Partition-wise join for join between (declaratively) partitioned tables

Started by Ashutosh Bapatover 9 years ago308 messages
#1Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
3 attachment(s)

Amit Langote is working on supporting declarative partitioning in
PostgreSQL [1]. /messages/by-id/55D3093C.5010800@lab.ntt.co.jp. I have started working on supporting partition-wise join.
This mail describes very high level design and some insight into the
performance improvements.

An equi-join between two partitioned tables can be broken down into
pair-wise join between their partitions. This technique is called
partition-wise join. Partition-wise joins between similarly partitioned
tables with equi-join condition can be efficient because [2]. https://users.cs.duke.edu/~shivnath/papers/sigmod295-herodotou.pdf
1. Each provably non-empty partition-wise join smaller. All such joins
collectively might be more efficient than the join between their parent.
2. Such joins are able to exploit properties of partitions like indexes,
their storage etc.
3. An N-way partition-wise join may have different efficient join orders
compared to the efficient join order between the parent tables.

A partition-wise join is processed in following stages [2]. https://users.cs.duke.edu/~shivnath/papers/sigmod295-herodotou.pdf, [3]. https://users.cs.duke.edu/~shivnath/tmp/paqo_draft.pdf.
1. Applicability testing: This phase checks if the join conditions match
the partitioning scheme. A partition-wise join is efficient if there is an
equi-join on the partition keys. E.g. join between tables R and S
partitioned by columns a and b resp. can be broken down into partition-wise
joins if there exists a join condition is R.a = S.b. Or in other words the
number of provably non-empty partition-wise joins is O(N) where N is the
number of partitions.

2. Matching: This phase determines which joins between the partitions of R
and S can potentially produce tuples in the join and prunes empty joins
between partitions.

3. Clustering: This phase aims at reducing the number of partition-wise
joins by clubbing together partitions from joining relations. E.g. clubbing
multiple partitions from either of the partitioned relations which can join
to a single partition from the other partitioned relation.

4. Path/plan creation: This phase creates multiple paths for each
partition-wise join. It also creates Append path/s representing the union
of partition-wise joins.

The work here focuses on a subset of use-cases discussed in [2]. https://users.cs.duke.edu/~shivnath/papers/sigmod295-herodotou.pdf. It only
considers partition-wise join for join between similarly partitioned tables
with same number of partitions with same properties, thus producing at most
as many partition-wise joins as there are partitions. It should be possible
to apply partition-wise join technique (with some special handling for
OUTER joins) if both relations have some extra partitions with
non-overlapping partition conditions, apart from the matching partitions.
But I am not planning to implement this optimization in the first cut.

The attached patch is a POC implementation of partition-wise join. It is is
based on the set of patches posted on 23rd May 2016 by Amit Langote for
declarative partitioning. The patch gives an idea about the approach used.
It has several TODOs, which I am working on.

Attached is a script with output which measures potential performance
improvement because of partition-wise join. The script uses a GUC
enable_partition_wise_join to disable/enable this feature for performance
measurement. The scripts measures performance improvement of a join between
two tables partitioned by range on integer column. Each table contains 50K
rows. Each table has an integer and a varchar column. It shows around
10-15% reduction in execution time when partition-wise join is used.
Accompanied with parallel query and FDWs, it opens up avenues for further
improvements for joins between partitioned tables.

[1]: . /messages/by-id/55D3093C.5010800@lab.ntt.co.jp
[2]: . https://users.cs.duke.edu/~shivnath/papers/sigmod295-herodotou.pdf
[3]: . https://users.cs.duke.edu/~shivnath/tmp/paqo_draft.pdf

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

Attachments:

partitioned_join.outapplication/octet-stream; name=partitioned_join.outDownload
partitioned_join.sqltext/x-sql; charset=US-ASCII; name=partitioned_join.sqlDownload
pg_dp_join_POC.patchtext/x-diff; charset=US-ASCII; name=pg_dp_join_POC.patchDownload
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 873a764..e8c7c76 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -987,21 +987,59 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		if (rel->has_eclass_joins || has_useful_pathkeys(root, rel))
 			add_child_rel_equivalences(root, appinfo, rel, childrel);
 		childrel->has_eclass_joins = rel->has_eclass_joins;
 
 		/*
 		 * Note: we could compute appropriate attr_needed data for the child's
 		 * variables, by transforming the parent's attr_needed through the
 		 * translated_vars mapping.  However, currently there's no need
 		 * because attr_needed is only examined for base relations not
 		 * otherrels.  So we just leave the child's attr_needed empty.
+		 * For a partitioned tables, individual partitions can participate in
+		 * the pair-wise joins. We need attr_needed data for buiding pair-wise
+		 * join relations. Partition tables should have same layout as the
+		 * parent table and hence should not need any translation. But rest of
+		 * the code still uses inheritance mechanism. So do we here.
+		 * TODO: do we need to translate the relids as well?
 		 */
+		if (rel->part_desc && rel->part_desc->nparts > 0)
+		{
+			AttrNumber attno;
+			for (attno = rel->min_attr; attno <= rel->max_attr; attno++)
+			{
+				int	index = attno - rel->min_attr;
+				Relids	attr_needed = bms_copy(rel->attr_needed[index]);
+
+				/*
+				 * System attributes do not need translation. In such a case,
+				 * the attribute numbers of the parent and the child should
+				 * start from the same minimum attribute.
+				 */
+				if (attno <= 0)
+				{
+					Assert(rel->min_attr == childrel->min_attr);
+					childrel->attr_needed[index] = attr_needed;
+				}
+				else
+				{
+					Var *var = list_nth(appinfo->translated_vars,
+										attno - 1); 
+					int child_index;
+
+					/* Parent Var translates to child Var. */ 
+					Assert(IsA(var, Var));
+
+					child_index = var->varattno - childrel->min_attr;
+					childrel->attr_needed[child_index] = attr_needed;
+				}
+			}
+		}
 
 		/*
 		 * Compute the child's size.
 		 */
 		set_rel_size(root, childrel, childRTindex, childRTE);
 
 		/*
 		 * It is possible that constraint exclusion detected a contradiction
 		 * within a child subquery, even though we didn't prove one above. If
 		 * so, we can skip this child.
@@ -1097,34 +1135,38 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 {
 	int			parentRTindex = rti;
 	List	   *live_childrels = NIL;
 	List	   *subpaths = NIL;
 	bool		subpaths_valid = true;
 	List	   *partial_subpaths = NIL;
 	bool		partial_subpaths_valid = true;
 	List	   *all_child_pathkeys = NIL;
 	List	   *all_child_outers = NIL;
 	ListCell   *l;
+	PartitionDesc	part_desc = rel->part_desc;
+	int			num_parts = part_desc ? part_desc->nparts : 0;
+	Oid		   *part_oids = part_desc ? part_desc->oids : NULL;
 
 	/*
 	 * Generate access paths for each member relation, and remember the
 	 * cheapest path for each one.  Also, identify all pathkeys (orderings)
 	 * and parameterizations (required_outer sets) available for the member
 	 * relations.
 	 */
 	foreach(l, root->append_rel_list)
 	{
 		AppendRelInfo *appinfo = (AppendRelInfo *) lfirst(l);
 		int			childRTindex;
 		RangeTblEntry *childRTE;
 		RelOptInfo *childrel;
 		ListCell   *lcp;
+		int			cnt_parts; 
 
 		/* append_rel_list contains all append rels; ignore others */
 		if (appinfo->parent_relid != parentRTindex)
 			continue;
 
 		/* Re-locate the child RTE and RelOptInfo */
 		childRTindex = appinfo->child_relid;
 		childRTE = root->simple_rte_array[childRTindex];
 		childrel = root->simple_rel_array[childRTindex];
 
@@ -1132,20 +1174,37 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 		 * Compute the child's access paths.
 		 */
 		set_rel_pathlist(root, childrel, childRTindex, childRTE);
 
 		/*
 		 * If child is dummy, ignore it.
 		 */
 		if (IS_DUMMY_REL(childrel))
 			continue;
 
+		/* 
+		 * Match the children to the partition to fill the partition scheme by
+		 * matching OID of the children in part_desc and RTE.
+		 * TODO: we are doing this here since we get hold of the partition
+		 * RelOptInfo here. But we should assess whether this is the right
+		 * place.
+		 */
+		for (cnt_parts = 0; cnt_parts < num_parts; cnt_parts++)
+		{
+			if (part_oids[cnt_parts] == childRTE->relid)
+			{
+				/* Every partition can be seen only once. */
+				Assert(!rel->part_rels[cnt_parts]);
+				rel->part_rels[cnt_parts] = childrel;
+			}
+		}
+
 		/*
 		 * Child is live, so add it to the live_childrels list for use below.
 		 */
 		live_childrels = lappend(live_childrels, childrel);
 
 		/*
 		 * If child has an unparameterized cheapest-total path, add that to
 		 * the unparameterized Append path we are constructing for the parent.
 		 * If not, there's no workable unparameterized path.
 		 */
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 4c9d8d9..9cd0361 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -120,20 +120,21 @@ bool		enable_indexscan = true;
 bool		enable_indexonlyscan = true;
 bool		enable_bitmapscan = true;
 bool		enable_tidscan = true;
 bool		enable_sort = true;
 bool		enable_hashagg = true;
 bool		enable_nestloop = true;
 bool		enable_material = true;
 bool		enable_mergejoin = true;
 bool		enable_hashjoin = true;
 bool		enable_fkey_estimates = true;
+bool		enable_partition_wise_join = true;
 
 typedef struct
 {
 	PlannerInfo *root;
 	QualCost	total;
 } cost_qual_eval_context;
 
 static List *extract_nonindex_conditions(List *qual_clauses, List *indexquals);
 static MergeScanSelCache *cached_scansel(PlannerInfo *root,
 			   RestrictInfo *rinfo,
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 01d4fea..b0cbc1b 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -10,35 +10,42 @@
  * IDENTIFICATION
  *	  src/backend/optimizer/path/joinrels.c
  *
  *-------------------------------------------------------------------------
  */
 #include "postgres.h"
 
 #include "optimizer/joininfo.h"
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
+#include "optimizer/cost.h"
 #include "utils/memutils.h"
 
 
 static void make_rels_by_clause_joins(PlannerInfo *root,
 						  RelOptInfo *old_rel,
 						  ListCell *other_rels);
 static void make_rels_by_clauseless_joins(PlannerInfo *root,
 							  RelOptInfo *old_rel,
 							  ListCell *other_rels);
 static bool has_join_restriction(PlannerInfo *root, RelOptInfo *rel);
 static bool has_legal_joinclause(PlannerInfo *root, RelOptInfo *rel);
 static bool is_dummy_rel(RelOptInfo *rel);
 static void mark_dummy_rel(RelOptInfo *rel);
 static bool restriction_is_constant_false(List *restrictlist,
 							  bool only_pushed_down);
+static bool are_partitions_joinable(RelOptInfo *rel1, RelOptInfo *rel2,
+								   SpecialJoinInfo *sjinfo);
+static PartitionDesc build_joinrel_partition_desc(RelOptInfo *rel1,
+												  RelOptInfo *rel2,
+												  SpecialJoinInfo *sjinfo);
+static void add_append_paths_to_joinrel(RelOptInfo *joinrel);
 
 
 /*
  * join_search_one_level
  *	  Consider ways to produce join relations containing exactly 'level'
  *	  jointree items.  (This is one step of the dynamic-programming method
  *	  embodied in standard_join_search.)  Join rel nodes for each feasible
  *	  combination of lower-level rels are created and returned in a list.
  *	  Implementation paths are created for each such joinrel, too.
  *
@@ -862,25 +869,182 @@ make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 			add_paths_to_joinrel(root, joinrel, rel1, rel2,
 								 JOIN_ANTI, sjinfo,
 								 restrictlist);
 			break;
 		default:
 			/* other values not expected here */
 			elog(ERROR, "unrecognized join type: %d", (int) sjinfo->jointype);
 			break;
 	}
 
+	/*
+	 * If both the relations are partitioned in the same way, and there is an
+	 * equi-join clause on partition key, try joining the partitions. Store the
+	 * partitioning scheme in joinrel for further joins.
+	 */
+
+	joinrel->part_desc = build_joinrel_partition_desc(rel1, rel2, sjinfo);
+
+	if (joinrel->part_desc && joinrel->part_desc->nparts > 0)
+	{
+		int	nparts = joinrel->part_desc->nparts;
+		int	cnt_parts;
+
+		/* Allocate space for holding the pair-wise join relations. */
+		joinrel->part_rels = (RelOptInfo **) palloc(sizeof(RelOptInfo *) *
+													nparts);
+		/* Create join relations for the partition relations. */
+		for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+		{
+			RelOptInfo	*part_join_rel;
+			part_join_rel = make_join_rel(root, rel1->part_rels[cnt_parts],
+										  rel2->part_rels[cnt_parts]);
+			joinrel->part_rels[cnt_parts] = part_join_rel;
+		}
+
+		/* Add append paths for pair-wise joins. */
+		add_append_paths_to_joinrel(joinrel);
+	}
+
 	bms_free(joinrelids);
 
 	return joinrel;
 }
 
+/*
+ * Given partitioin description of two joining relations, construct partition
+ * description for join between those relations.
+ *
+ * TODO find the right place for this function.
+ */
+static PartitionDesc
+build_joinrel_partition_desc(RelOptInfo *rel1, RelOptInfo *rel2,
+							 SpecialJoinInfo *sjinfo)
+{
+	PartitionDesc	part_desc;
+
+	/* Do nothing, if user doesn't want to try partition-wise join. */
+	if (!enable_partition_wise_join)
+		return NULL;
+
+	if (!are_partitions_joinable(rel1, rel2, sjinfo))
+		return NULL;
+
+	/* 
+	 * The result of join is partitioned the same way as the joining relations.
+	 * Construct the partitioning scheme from the joining relations.
+	 */
+	part_desc = (PartitionDesc) palloc0(sizeof(PartitionDescData));
+	part_desc->nparts = rel1->part_desc->nparts;
+	/* Fill up the rest of the fields. */	
+
+	return part_desc;
+}
+
+/*
+ * Assess whether the given relations are similarly partitioned and have
+ * equi-join clauses on partition keys.
+ * 
+ * Two relations are similarly partitioned if
+ * o. They have same number of partitions
+ * o. They have same number of partition keys*
+ * o. Partition keys have same types and opclasses*
+ * o. They have same upper and lower bounds (with inclusive/exclusive
+ * attributes) for all keys for range partitions. They have same list items for
+ * list partitions.
+ *
+ * Have same number of partition keys: It might be possible to join partitioned
+ * table which have different number of partition keys and suitable equi-join
+ * clauses to eliminate the possibilities. But right now, we do not consider
+ * this.
+ *
+ * Have same types and opclasses: Right now, we expect the partition keys to
+ * have exact same order of partion key types and opclasses. But it might
+ * be possible to relax this condition, if we can find which partition key
+ * matches which and also find corresponding equi-joins.
+ */
+static bool
+are_partitions_joinable(RelOptInfo *rel1, RelOptInfo *rel2,
+					   SpecialJoinInfo *sjinfo)
+{
+	PartitionDesc	part_desc1 = rel1->part_desc;
+	PartitionDesc	part_desc2 = rel2->part_desc;
+
+	/* If either of the relations is not partitioned, nothing to check here. */
+	if (!part_desc1 || part_desc1->nparts == 0 ||
+		!part_desc2 || part_desc2->nparts == 0)
+		return false;
+
+	/*
+	 * If the number of partitions on either side differs, partitioning schemes
+	 * do not match.
+	 * TODO: it should be possible to push an inner join down even if the number of
+	 * partitions differ but the common partitions match. In such a case pushing
+	 * down outer joins would be tricky, but still doable using empty relation
+	 * for non-existing partition.
+	 */
+	if (part_desc1->nparts != part_desc2->nparts)
+		return false;
+
+	/* TODO: */
+	/* All the artitions on either side should have same bounds or lists. */
+	/* Joining condition should have an equi-join on the partition key. */
+
+	/* By default, the partitions match. */
+	return true;
+}
+
+/*
+ * Add append paths for the join relation.
+ *
+ * Like set_append_rel_pathlist, this function considers pair-wise partition
+ * join paths with parameterization and pathkeys. 
+ * 
+ * TODO: right now the function just picks up the cheapest path from each of
+ * the partitions and creates an append path with those. 
+ *
+ * TODO: may be we should consider splitting set_append_rel_pathlist() so that
+ * it can be used for both inheritance and partitioning.
+ */
+static void
+add_append_paths_to_joinrel(RelOptInfo *joinrel)
+{
+	RelOptInfo **part_rels = joinrel->part_rels;
+	int		nparts;
+	int		cnt_parts;
+	List   *part_paths = NIL;
+
+	Assert(joinrel->part_desc);
+
+	nparts = joinrel->part_desc->nparts;
+
+	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	{
+		RelOptInfo *part_rel = part_rels[cnt_parts];
+
+		/* Find the cheapest path for partition relation. */
+		set_cheapest(part_rel);
+
+		/* We don't expect any parameterization here. */
+		Assert(!part_rel->cheapest_total_path->param_info);
+		
+		/*
+		 * Instead of lappend, we should use accumulate_append_subpath() to
+		 * pull up the paths in underlying append.
+		 */
+		part_paths = lappend(part_paths, part_rel->cheapest_total_path);
+	}
+
+	add_path(joinrel, (Path *) create_append_path(joinrel, part_paths, NULL,
+												  0));
+	return;
+}
 
 /*
  * have_join_order_restriction
  *		Detect whether the two relations should be joined to satisfy
  *		a join-order restriction arising from special or lateral joins.
  *
  * In practice this is always used with have_relevant_joinclause(), and so
  * could be merged with that function, but it seems clearer to separate the
  * two concerns.  We need this test because there are degenerate cases where
  * a clauseless join must be performed to satisfy join-order restrictions.
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index 1179643..602f231 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -480,20 +480,35 @@ get_relation_info(PlannerInfo *root, Oid relationObjectId, bool inhparent,
 	{
 		rel->serverid = GetForeignServerIdByRelId(RelationGetRelid(relation));
 		rel->fdwroutine = GetFdwRoutineForRelation(relation, true);
 	}
 	else
 	{
 		rel->serverid = InvalidOid;
 		rel->fdwroutine = NULL;
 	}
 
+	/*
+	 * Get the partitioning scheme.
+	 * TODO: this is temporary code to stick partitioning information in
+	 * RelOptInfo. We might change the way the information is stored. At every
+	 * relation, we need to match partitioning information relevant at that
+	 * level. So, storing only a single level partitioning information should
+	 * suffice, even for a multi-level partitioned table.
+	 */
+	rel->part_desc = RelationGetPartitionDesc(relation);
+	if (rel->part_desc && rel->part_desc->nparts > 0)
+	{
+		rel->part_rels = (RelOptInfo **) palloc0(sizeof(RelOptInfo *) *
+												 rel->part_desc->nparts);
+	}
+
 	heap_close(relation, NoLock);
 
 	/*
 	 * Allow a plugin to editorialize on the info we obtained from the
 	 * catalogs.  Actions might include altering the assumed relation size,
 	 * removing an index, or adding a hypothetical index to the indexlist.
 	 */
 	if (get_relation_info_hook)
 		(*get_relation_info_hook) (root, relationObjectId, inhparent, rel);
 }
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 1e87a73..6e7b1a4 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -377,25 +377,41 @@ build_join_rel(PlannerInfo *root,
 		 * pair of component relations.
 		 */
 		if (restrictlist_ptr)
 			*restrictlist_ptr = build_joinrel_restrictlist(root,
 														   joinrel,
 														   outer_rel,
 														   inner_rel);
 		return joinrel;
 	}
 
+	/* A partition relation can be joined with only partition relation. */
+	Assert(!(outer_rel->reloptkind == RELOPT_OTHER_JOINREL ||
+		    outer_rel->reloptkind == RELOPT_OTHER_MEMBER_REL) ||
+		   (inner_rel->reloptkind == RELOPT_OTHER_JOINREL ||
+			inner_rel->reloptkind == RELOPT_OTHER_MEMBER_REL));
+
 	/*
 	 * Nope, so make one.
 	 */
 	joinrel = makeNode(RelOptInfo);
-	joinrel->reloptkind = RELOPT_JOINREL;
+
+	/*
+	 * A join between partitions or child tables is different from join between
+	 * regular tables.
+	 */
+	if (outer_rel->reloptkind == RELOPT_OTHER_JOINREL ||
+		outer_rel->reloptkind == RELOPT_OTHER_MEMBER_REL)
+		joinrel->reloptkind = RELOPT_OTHER_JOINREL;
+	else
+		joinrel->reloptkind = RELOPT_JOINREL;
+
 	joinrel->relids = bms_copy(joinrelids);
 	joinrel->rows = 0;
 	/* cheap startup cost is interesting iff not all tuples to be retrieved */
 	joinrel->consider_startup = (root->tuple_fraction > 0);
 	joinrel->consider_param_startup = false;
 	joinrel->consider_parallel = false;
 	joinrel->reltarget = create_empty_pathtarget();
 	joinrel->pathlist = NIL;
 	joinrel->ppilist = NIL;
 	joinrel->partial_pathlist = NIL;
@@ -539,21 +555,21 @@ build_join_rel(PlannerInfo *root,
 		Assert(!found);
 		hentry->join_rel = joinrel;
 	}
 
 	/*
 	 * Also, if dynamic-programming join search is active, add the new joinrel
 	 * to the appropriate sublist.  Note: you might think the Assert on number
 	 * of members should be for equality, but some of the level 1 rels might
 	 * have been joinrels already, so we can only assert <=.
 	 */
-	if (root->join_rel_level)
+	if (root->join_rel_level && joinrel->reloptkind != RELOPT_OTHER_JOINREL)
 	{
 		Assert(root->join_cur_level > 0);
 		Assert(root->join_cur_level <= bms_num_members(joinrel->relids));
 		root->join_rel_level[root->join_cur_level] =
 			lappend(root->join_rel_level[root->join_cur_level], joinrel);
 	}
 
 	return joinrel;
 }
 
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index e246a9c..65e993e 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -879,20 +879,29 @@ static struct config_bool ConfigureNamesBool[] =
 	},
 	{
 		{"enable_fkey_estimates", PGC_USERSET, QUERY_TUNING_METHOD,
 			gettext_noop("Enables use of foreign keys for estimating joins."),
 			NULL
 		},
 		&enable_fkey_estimates,
 		true,
 		NULL, NULL, NULL
 	},
+	{
+		{"enable_partition_wise_join", PGC_USERSET, QUERY_TUNING_METHOD,
+			gettext_noop("Enables partition-wise join."),
+			NULL
+		},
+		&enable_partition_wise_join,
+		true,
+		NULL, NULL, NULL
+	},
 
 	{
 		{"geqo", PGC_USERSET, QUERY_TUNING_GEQO,
 			gettext_noop("Enables genetic query optimization."),
 			gettext_noop("This algorithm attempts to do planning without "
 						 "exhaustive searching.")
 		},
 		&enable_geqo,
 		true,
 		NULL, NULL, NULL
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 45739c3..332e83f 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -12,20 +12,21 @@
  *-------------------------------------------------------------------------
  */
 #ifndef RELATION_H
 #define RELATION_H
 
 #include "access/sdir.h"
 #include "lib/stringinfo.h"
 #include "nodes/params.h"
 #include "nodes/parsenodes.h"
 #include "storage/block.h"
+#include "catalog/partition.h"
 
 
 /*
  * Relids
  *		Set of relation identifiers (indexes into the rangetable).
  */
 typedef Bitmapset *Relids;
 
 /*
  * When looking for a "cheapest path", this enum specifies whether we want
@@ -342,20 +343,27 @@ typedef struct PlannerInfo
  * is present in the query join tree but the members are not.  The member
  * RTEs and otherrels are used to plan the scans of the individual tables or
  * subqueries of the append set; then the parent baserel is given Append
  * and/or MergeAppend paths comprising the best paths for the individual
  * member rels.  (See comments for AppendRelInfo for more information.)
  *
  * At one time we also made otherrels to represent join RTEs, for use in
  * handling join alias Vars.  Currently this is not needed because all join
  * alias Vars are expanded to non-aliased form during preprocess_expression.
  *
+ * We also have relations representing each of the pair-wise joins between
+ * partitioned tables with same partitioning scheme. These relations are not
+ * added to join_rel_level lists as they are not joined directly by the dynamic
+ * programming algorithm. Adding these two join_rel_level list also means that
+ * top level list has more than one join relation, which is symantically
+ * incorrect.
+ *
  * There is also a RelOptKind for "upper" relations, which are RelOptInfos
  * that describe post-scan/join processing steps, such as aggregation.
  * Many of the fields in these RelOptInfos are meaningless, but their Path
  * fields always hold Paths showing ways to do that processing step.
  *
  * Lastly, there is a RelOptKind for "dead" relations, which are base rels
  * that we have proven we don't need to join after all.
  *
  * Parts of this data structure are specific to various scan and join
  * mechanisms.  It didn't seem worth creating new node types for them.
@@ -460,20 +468,21 @@ typedef struct PlannerInfo
  * We store baserestrictcost in the RelOptInfo (for base relations) because
  * we know we will need it at least once (to price the sequential scan)
  * and may need it multiple times to price index scans.
  *----------
  */
 typedef enum RelOptKind
 {
 	RELOPT_BASEREL,
 	RELOPT_JOINREL,
 	RELOPT_OTHER_MEMBER_REL,
+	RELOPT_OTHER_JOINREL,
 	RELOPT_UPPER_REL,
 	RELOPT_DEADREL
 } RelOptKind;
 
 typedef struct RelOptInfo
 {
 	NodeTag		type;
 
 	RelOptKind	reloptkind;
 
@@ -532,20 +541,35 @@ typedef struct RelOptInfo
 	struct FdwRoutine *fdwroutine;
 	void	   *fdw_private;
 
 	/* used by various scans and joins: */
 	List	   *baserestrictinfo;		/* RestrictInfo structures (if base
 										 * rel) */
 	QualCost	baserestrictcost;		/* cost of evaluating the above */
 	List	   *joininfo;		/* RestrictInfo structures for join clauses
 								 * involving this rel */
 	bool		has_eclass_joins;		/* T means joininfo is incomplete */
+
+	/* For partitioned relations, joins or base relations. */
+	/* TODO: the partition hierarchy described by the members below, may be put
+	 * into a path rather than RelOptInfo and will be handy in case we start
+	 * supporting repartitioning of the data by different partitioning scheme.
+	 *
+	 * TODO: PartitionDescData contains the array of OIDs of the partitions, but that
+	 * doesn't work for the partitions obtained by pair-wise joins of
+	 * partitioned tables. We should probably create another data structure
+	 * like AppendRelInfo for storing those, but I am not sure.
+	 *
+	 * TODO: Notice recursive usage of RelOptInfo.
+	 */
+	PartitionDesc	part_desc;	/* Partitioning scheme if partitioned */
+	struct RelOptInfo	  **part_rels;	/* RelOptInfo of the partitions. */
 } RelOptInfo;
 
 /*
  * IndexOptInfo
  *		Per-index information for planning/optimization
  *
  *		indexkeys[], indexcollations[], opfamily[], and opcintype[]
  *		each have ncolumns entries.
  *
  *		sortopfamily[], reverse_sort[], and nulls_first[] likewise have
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 58ac163..ff2e3c7 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -60,20 +60,21 @@ extern bool enable_indexscan;
 extern bool enable_indexonlyscan;
 extern bool enable_bitmapscan;
 extern bool enable_tidscan;
 extern bool enable_sort;
 extern bool enable_hashagg;
 extern bool enable_nestloop;
 extern bool enable_material;
 extern bool enable_mergejoin;
 extern bool enable_hashjoin;
 extern bool enable_fkey_estimates;
+extern bool enable_partition_wise_join;
 extern int	constraint_exclusion;
 
 extern double clamp_row_est(double nrows);
 extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
 					double index_pages, PlannerInfo *root);
 extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 			 ParamPathInfo *param_info);
 extern void cost_samplescan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 				ParamPathInfo *param_info);
 extern void cost_index(IndexPath *path, PlannerInfo *root,
#2Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#1)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Jun 15, 2016 at 3:25 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Amit Langote is working on supporting declarative partitioning in PostgreSQL
[1]. I have started working on supporting partition-wise join. This mail
describes very high level design and some insight into the performance
improvements.

An equi-join between two partitioned tables can be broken down into
pair-wise join between their partitions. This technique is called
partition-wise join. Partition-wise joins between similarly partitioned
tables with equi-join condition can be efficient because [2]
1. Each provably non-empty partition-wise join smaller. All such joins
collectively might be more efficient than the join between their parent.
2. Such joins are able to exploit properties of partitions like indexes,
their storage etc.
3. An N-way partition-wise join may have different efficient join orders
compared to the efficient join order between the parent tables.

A partition-wise join is processed in following stages [2], [3].
1. Applicability testing: This phase checks if the join conditions match the
partitioning scheme. A partition-wise join is efficient if there is an
equi-join on the partition keys. E.g. join between tables R and S
partitioned by columns a and b resp. can be broken down into partition-wise
joins if there exists a join condition is R.a = S.b. Or in other words the
number of provably non-empty partition-wise joins is O(N) where N is the
number of partitions.

2. Matching: This phase determines which joins between the partitions of R
and S can potentially produce tuples in the join and prunes empty joins
between partitions.

3. Clustering: This phase aims at reducing the number of partition-wise
joins by clubbing together partitions from joining relations. E.g. clubbing
multiple partitions from either of the partitioned relations which can join
to a single partition from the other partitioned relation.

4. Path/plan creation: This phase creates multiple paths for each
partition-wise join. It also creates Append path/s representing the union of
partition-wise joins.

The work here focuses on a subset of use-cases discussed in [2]. It only
considers partition-wise join for join between similarly partitioned tables
with same number of partitions with same properties, thus producing at most
as many partition-wise joins as there are partitions. It should be possible
to apply partition-wise join technique (with some special handling for OUTER
joins) if both relations have some extra partitions with non-overlapping
partition conditions, apart from the matching partitions. But I am not
planning to implement this optimization in the first cut.

I haven't reviewed this code yet due to being busy with 9.6, but I
think this is a very important query planner improvement with the
potential for big wins on queries involving large amounts of data.

Suppose we have a pair of equi-partitioned tables. Right now, if we
choose to perform a hash join, we'll have to build a giant hash table
with all of the rows from every inner partition and then probe it for
every row in every outer partition. If there are few enough inner
rows that the resultant hash table still fits in work_mem, this is
somewhat inefficient but not terrible - but if it causes us to have to
batch the hash join where we otherwise would not need to do so, then
it really sucks. Similarly, if we decide to merge-join each pair of
partitions, a partitionwise join may be able to use an internal sort
on some or all partitions whereas if we had to deal with all of the
data at the same time we'd need an external sort, possibly multi-pass.
And if we choose a nested loop, say over an inner index-scan, we do
O(outer rows) index probes with this optimization but O(outer rows *
inner partitions) index probes without it.

In addition, parallel query can benefit significantly from this kind
of optimization. Tom recently raised the case of an appendrel where
every child has a parallel-safe path but not every child has a partial
path; currently, we can't go parallel in that case, but it's easy to
see that we could handle it by scheduling the appendrel's children
across a pool of workers. If we had this optimization, that sort of
thing would be much more likely to be useful, because it could create
appendrels where each member is an N-way join between equipartitioned
tables. That's particularly important right now because of the
restriction that a partial path must be driven by a Parallel SeqScan,
but even after that restriction is lifted it's easy to imagine that
the effective degree of parallelism for a single index scan may be
limited - so this kind of thing may significantly increase the number
of workers that a given query can use productively.

--
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

#3Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#2)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Jul 8, 2016 at 12:11 AM, Robert Haas <robertmhaas@gmail.com> wrote:

I haven't reviewed this code yet due to being busy with 9.6, but I
think this is a very important query planner improvement with the
potential for big wins on queries involving large amounts of data.

Suppose we have a pair of equi-partitioned tables. Right now, if we
choose to perform a hash join, we'll have to build a giant hash table
with all of the rows from every inner partition and then probe it for
every row in every outer partition. If there are few enough inner
rows that the resultant hash table still fits in work_mem, this is
somewhat inefficient but not terrible - but if it causes us to have to
batch the hash join where we otherwise would not need to do so, then
it really sucks. Similarly, if we decide to merge-join each pair of
partitions, a partitionwise join may be able to use an internal sort
on some or all partitions whereas if we had to deal with all of the
data at the same time we'd need an external sort, possibly multi-pass.

Or we might be able to use indexes directly without need of a MergeAppend.

And if we choose a nested loop, say over an inner index-scan, we do
O(outer rows) index probes with this optimization but O(outer rows *
inner partitions) index probes without it.

In addition, parallel query can benefit significantly from this kind
of optimization. Tom recently raised the case of an appendrel where
every child has a parallel-safe path but not every child has a partial
path; currently, we can't go parallel in that case, but it's easy to
see that we could handle it by scheduling the appendrel's children
across a pool of workers. If we had this optimization, that sort of
thing would be much more likely to be useful, because it could create
appendrels where each member is an N-way join between equipartitioned
tables. That's particularly important right now because of the
restriction that a partial path must be driven by a Parallel SeqScan,
but even after that restriction is lifted it's easy to imagine that
the effective degree of parallelism for a single index scan may be
limited - so this kind of thing may significantly increase the number
of workers that a given query can use productively.

+1.

The attached patch implements the logic to assess whether two partitioned
tables can be joined using partition-wise join technique described in my
last
mail on this thread.

Two partitioned relations are considered for partition-wise join if
following
conditions are met (See build_joinrel_part_info() for details):
1. Both the partitions have same number of partitions, with same number of
partition keys and partitioned by same strategy - range or list.
2. They have matching datatypes for partition keys (partkey_types_match())
3. For list partitioned relations, they have same lists for each pair of
partitions, paired by position in which they appear.
4. For range partitioned relations, they have same bounds for each pair of
partitions, paired by their position when ordered in ascending fashion on
the
upper bounds.
5. There exists an equi-join condition for each pair of partition keys,
paired
by the position in which they appear.

Partition-wise join technique can be applied under more lenient constraints
[1]: e.g. joins between tables with different number of partitions but having same bounds/lists for the common partitions. I am planning to defer that to a later version of this feature.
e.g. joins between tables with different number of partitions but having
same
bounds/lists for the common partitions. I am planning to defer that to a
later
version of this feature.

A join executed using partition-wise join technique is itself a relation
partitioned by the similar partitioning scheme as the joining relations with
the partition keys combined from the joining relations.

A PartitionOptInfo (uses name similar to RelOptInfo or IndexOptInfo)
structure
is used to store the partitioning information for a given base or relation.
In build_simple_rel(), we construct PartitionOptInfo structure for the given
base relation by copying the relation's PartitionDesc and PartitionKey
(structures from Amit Langote's patch). While doing so, all the partition
keys
are stored as expressions. The structure also holds the RelOptInfos of the
partition relations. For a join relation, most of the PartitionOptInfo is
copied from either of the joining relations, except the partition keys and
RelOptInfo of partition relations. Partition keys of the join relations are
created by combing partition keys from both the joining relations. The
logic to
cosnstruct RelOptInfo for the partition-wise join relations is yet to be
implemented.

Since the logic to create the paths and RelOptInfos for partition-wise join
relations is not implemented yet, a query which can use partition-wise join
fails with error
"ERROR: the relation was considered for partition-wise join, which is not
supported right now.". It will also print messages to show which of the
joins
can and can not use partition-wise join technique e.g.
"NOTICE: join between relations (b 1) and (b 2) is considered for
partition-wise join." The relations are indicated by their relid in the
query.
OR
"NOTICE: join between relations (b 1) and (b 2) is NOT considered for
partition-wise join.".
These messages are for debugging only, and will be removed once path
creation
logic is implemented.

The patch adds a test partition_join.sql, which has a number of positive and
negative testcases for joins between partitioned tables.

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

Attachments:

pg_dp_join_assess_phase.patchapplication/x-download; name=pg_dp_join_assess_phase.patchDownload
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 2e4b670..06fc90d 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -999,21 +999,59 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		if (rel->has_eclass_joins || has_useful_pathkeys(root, rel))
 			add_child_rel_equivalences(root, appinfo, rel, childrel);
 		childrel->has_eclass_joins = rel->has_eclass_joins;
 
 		/*
 		 * Note: we could compute appropriate attr_needed data for the child's
 		 * variables, by transforming the parent's attr_needed through the
 		 * translated_vars mapping.  However, currently there's no need
 		 * because attr_needed is only examined for base relations not
 		 * otherrels.  So we just leave the child's attr_needed empty.
+		 * For a partitioned tables, individual partitions can participate in
+		 * the pair-wise joins. We need attr_needed data for building pair-wise
+		 * join relations. Partitions should have same layout as the parent
+		 * table and hence should not need any translation, but we still use
+		 * the translated_vars to keep the code in sync with the inheritance
+		 * code.
 		 */
+		if (rel->part_info)
+		{
+			AttrNumber attno;
+			for (attno = rel->min_attr; attno <= rel->max_attr; attno++)
+			{
+				int	index = attno - rel->min_attr;
+				Relids	attr_needed = bms_copy(rel->attr_needed[index]);
+
+				/*
+				 * System attributes do not need translation. In such a case,
+				 * the attribute numbers of the parent and the child should
+				 * start from the same minimum attribute.
+				 */
+				if (attno <= 0)
+				{
+					Assert(rel->min_attr == childrel->min_attr);
+					childrel->attr_needed[index] = attr_needed;
+				}
+				else
+				{
+					Var *var = list_nth(appinfo->translated_vars,
+										attno - 1);
+					int child_index;
+
+					/* Parent Var translates to child Var. */
+					Assert(IsA(var, Var));
+
+					child_index = var->varattno - childrel->min_attr;
+					childrel->attr_needed[child_index] = attr_needed;
+				}
+			}
+		}
 
 		/*
 		 * Compute the child's size.
 		 */
 		set_rel_size(root, childrel, childRTindex, childRTE);
 
 		/*
 		 * It is possible that constraint exclusion detected a contradiction
 		 * within a child subquery, even though we didn't prove one above. If
 		 * so, we can skip this child.
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 8c1dccc..3ab6a9d 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -119,20 +119,21 @@ bool		enable_seqscan = true;
 bool		enable_indexscan = true;
 bool		enable_indexonlyscan = true;
 bool		enable_bitmapscan = true;
 bool		enable_tidscan = true;
 bool		enable_sort = true;
 bool		enable_hashagg = true;
 bool		enable_nestloop = true;
 bool		enable_material = true;
 bool		enable_mergejoin = true;
 bool		enable_hashjoin = true;
+bool		enable_partition_wise_join = true;
 
 typedef struct
 {
 	PlannerInfo *root;
 	QualCost	total;
 } cost_qual_eval_context;
 
 static List *extract_nonindex_conditions(List *qual_clauses, List *indexquals);
 static MergeScanSelCache *cached_scansel(PlannerInfo *root,
 			   RestrictInfo *rinfo,
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 01d4fea..e485e01 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -7,38 +7,56 @@
  * Portions Copyright (c) 1994, Regents of the University of California
  *
  *
  * IDENTIFICATION
  *	  src/backend/optimizer/path/joinrels.c
  *
  *-------------------------------------------------------------------------
  */
 #include "postgres.h"
 
+#include "optimizer/clauses.h"
 #include "optimizer/joininfo.h"
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
+#include "optimizer/cost.h"
 #include "utils/memutils.h"
 
 
 static void make_rels_by_clause_joins(PlannerInfo *root,
 						  RelOptInfo *old_rel,
 						  ListCell *other_rels);
 static void make_rels_by_clauseless_joins(PlannerInfo *root,
 							  RelOptInfo *old_rel,
 							  ListCell *other_rels);
 static bool has_join_restriction(PlannerInfo *root, RelOptInfo *rel);
 static bool has_legal_joinclause(PlannerInfo *root, RelOptInfo *rel);
 static bool is_dummy_rel(RelOptInfo *rel);
 static void mark_dummy_rel(RelOptInfo *rel);
 static bool restriction_is_constant_false(List *restrictlist,
 							  bool only_pushed_down);
+static bool have_partkey_equi_join(RelOptInfo *rel1, RelOptInfo *rel2,
+						   SpecialJoinInfo *sjinfo, List *restrictlist);
+static PartitionOptInfo *build_joinrel_part_info(RelOptInfo *rel1,
+							   RelOptInfo *rel2, SpecialJoinInfo *sjinfo,
+							   List *restrictlist);
+static void add_append_paths_to_joinrel(RelOptInfo *joinrel);
+static void match_expr_to_partition_keys(RestrictInfo *rinfo, Expr *expr,
+						  RelOptInfo *rel, List **pkclauses);
+static bool partkey_types_match(PartitionOptInfo *part_info1,
+					PartitionOptInfo *part_info2);
+static bool have_same_partition_lists(PartitionOptInfo *part_info1,
+					  PartitionOptInfo *part_info2);
+static bool have_same_partition_bounds(PartitionOptInfo *part_info1,
+					  PartitionOptInfo *part_info2);
+static bool compare_partition_bounds(PartitionOptInfo *part_info,
+					PartitionRangeBound *bound1, PartitionRangeBound *bound2);
 
 
 /*
  * join_search_one_level
  *	  Consider ways to produce join relations containing exactly 'level'
  *	  jointree items.  (This is one step of the dynamic-programming method
  *	  embodied in standard_join_search.)  Join rel nodes for each feasible
  *	  combination of lower-level rels are created and returned in a list.
  *	  Implementation paths are created for each such joinrel, too.
  *
@@ -652,20 +670,21 @@ join_is_legal(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
  */
 RelOptInfo *
 make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 {
 	Relids		joinrelids;
 	SpecialJoinInfo *sjinfo;
 	bool		reversed;
 	SpecialJoinInfo sjinfo_data;
 	RelOptInfo *joinrel;
 	List	   *restrictlist;
+	PartitionOptInfo	*part_info;
 
 	/* We should never try to join two overlapping sets of rels. */
 	Assert(!bms_overlap(rel1->relids, rel2->relids));
 
 	/* Construct Relids set that identifies the joinrel. */
 	joinrelids = bms_union(rel1->relids, rel2->relids);
 
 	/* Check validity and determine join type. */
 	if (!join_is_legal(root, rel1, rel2, joinrelids,
 					   &sjinfo, &reversed))
@@ -862,25 +881,372 @@ make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 			add_paths_to_joinrel(root, joinrel, rel1, rel2,
 								 JOIN_ANTI, sjinfo,
 								 restrictlist);
 			break;
 		default:
 			/* other values not expected here */
 			elog(ERROR, "unrecognized join type: %d", (int) sjinfo->jointype);
 			break;
 	}
 
+	/*
+	 * If both the relations are partitioned in the same way, and there is an
+	 * equi-join clause on partition key, try joining the partitions. Store the
+	 * partitioning scheme in joinrel for further joins.
+	 */
+	part_info = build_joinrel_part_info(rel1, rel2, sjinfo, restrictlist);
+	joinrel->part_info = part_info;
+	if (part_info)
+	{
+		PartitionOptInfo *part_info1 = rel1->part_info;
+		PartitionOptInfo *part_info2 = rel2->part_info;
+		int		nparts = part_info->nparts;
+		int		cnt_parts;
+		StringInfo	rel1_desc = makeStringInfo();
+		StringInfo	rel2_desc = makeStringInfo();
+
+		/* TODO: remove this notice when finalising the patch. */
+		outBitmapset(rel1_desc, rel1->relids);
+		outBitmapset(rel2_desc, rel2->relids);
+		elog(NOTICE, "join between relations %s and %s is considered for partition-wise join.",
+			 rel1_desc->data, rel2_desc->data);
+
+		/* Create join relations for the partition relations. */
+		for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+		{
+			RelOptInfo	*part_join_rel;
+
+			/*
+			 * TODO: make_join_rel can return NULL if the join is illegal. This
+			 * should not happen since the join between parents is deemed
+			 * legal.
+			 */
+			part_join_rel = make_join_rel(root,
+										  part_info1->part_rels[cnt_parts],
+										  part_info2->part_rels[cnt_parts]);
+			part_info->part_rels[cnt_parts] = part_join_rel;
+		}
+
+		/* Add append paths for pair-wise joins. */
+		add_append_paths_to_joinrel(joinrel);
+	}
+	/* TODO: remove this notice printing code while finalising the patch. */
+	else if (!IS_OTHER_REL(rel1->reloptkind) && !IS_OTHER_REL(rel2->reloptkind))
+	{
+		StringInfo	rel1_desc = makeStringInfo();
+		StringInfo	rel2_desc = makeStringInfo();
+
+		/* TODO: remove this notice when finalising the patch. */
+		outBitmapset(rel1_desc, rel1->relids);
+		outBitmapset(rel2_desc, rel2->relids);
+		elog(NOTICE, "join between relations %s and %s is NOT considered for partition-wise join.",
+			 rel1_desc->data, rel2_desc->data);
+	}
+
 	bms_free(joinrelids);
 
 	return joinrel;
 }
 
+/*
+ * build_joinrel_part_info
+ *
+ * Given partition description of two joining relations, construct partition
+ * description for join between those relations. If we can execute
+ * partition-wise join the resultant join is partitioned in the same way as the
+ * joining relations. Otherwise, it is not partitioned and in such case the
+ * function returns NULL.
+ *
+ * TODO find the right place for this function.
+ *
+ * TODO:
+ * This function and
+ * build_partition_info() should be named consistently e.g.
+ * build_joinrel_partition_info and build_simplerel_partition_info or
+ * build_baserel_partition_info resp.
+ */
+static PartitionOptInfo *
+build_joinrel_part_info(RelOptInfo *rel1, RelOptInfo *rel2,
+						SpecialJoinInfo *sjinfo, List *restrictlist)
+{
+	PartitionOptInfo	*part_info;
+	PartitionOptInfo	*part_info1 = rel1->part_info;
+	PartitionOptInfo	*part_info2 = rel2->part_info;
+	int		cnt_pks;
+	int		num_pks;
+
+	/* Do nothing, if user doesn't want to try partition-wise join. */
+	if (!enable_partition_wise_join)
+		return NULL;
+
+	/*
+	 * If number of partitions, number of partition keys and partitioning
+	 * strategy of the joining relations do not match, we can not apply
+	 * partition-wise join.
+	 *
+	 * TODO: it should be possible to push an inner join down even if the number of
+	 * partitions differ but the common partitions match. In such a case pushing
+	 * down outer joins would be tricky, but still doable using empty relation
+	 * for non-existing partition.
+	 */
+	if (!part_info1 || !part_info2 ||
+		part_info1->nparts != part_info2->nparts ||
+		part_info1->strategy != part_info2->strategy ||
+		part_info1->partnatts != part_info2->partnatts)
+		return NULL;
+
+	/*
+	 * Partition-wise join can not be applied if datatypes and collations of
+	 * the partition keys do not match.
+	 */
+	if (!partkey_types_match(part_info1, part_info2))
+		return NULL;
+
+	/*
+	 * Partition-wise join can not be applied if the partition bounds or lists
+	 * of joining relations do not match.
+	 */
+	switch (part_info1->strategy)
+	{
+		case PARTITION_STRAT_LIST:
+			if (!have_same_partition_lists(part_info1, part_info2))
+				return NULL;
+			break;
+
+		case PARTITION_STRAT_RANGE:
+			if (!have_same_partition_bounds(part_info1, part_info2))
+				return NULL;
+			break;
+
+		default:
+			/* Unknown partition strategy. */
+			return NULL;
+	}
+
+	/*
+	 * Partition-wise join can not be applied if there is no equi-join
+	 * condition between partition keys.
+	 */
+	if (!have_partkey_equi_join(rel1, rel2, sjinfo, restrictlist))
+		return NULL;
+
+	/* The join is partitioned the similar to the joining relations. */
+	part_info = makeNode(PartitionOptInfo);
+
+	/* Information related to the partititions. */
+	part_info->nparts = part_info1->nparts;
+	/* Allocate space for partition RelOptInfos, which will be filled later. */
+	part_info->part_rels = (RelOptInfo **) palloc(sizeof(RelOptInfo *) *
+												  part_info->nparts);
+	part_info->lists = part_info1->lists;
+	part_info->rangelowers = part_info1->rangelowers;
+	part_info->rangeuppers = part_info1->rangeuppers;
+
+	/* Information related to partition keys. */
+	part_info->strategy = part_info1->strategy;
+	part_info->partnatts = part_info1->partnatts;
+	part_info->partopfamily = part_info1->partopfamily;
+	part_info->partopcintype = part_info1->partopcintype;
+	part_info->partsupfunc = part_info1->partsupfunc;
+	part_info->tcinfo = part_info1->tcinfo;
+	num_pks = part_info->partnatts;
+	part_info->partexprs = (List **) palloc0(sizeof(List *) * num_pks);
+	for (cnt_pks = 0; cnt_pks < num_pks; cnt_pks++)
+	{
+		List *pkexpr = list_copy(part_info1->partexprs[cnt_pks]);
+
+		pkexpr = list_concat(pkexpr,
+							 list_copy(part_info2->partexprs[cnt_pks]));
+		part_info->partexprs[cnt_pks] = pkexpr;
+	}
+
+	return part_info;
+}
+
+/*
+ * Returns true if the given relations have equi-join clauses on all the
+ * corresponding partition keys.
+ */
+static bool
+have_partkey_equi_join(RelOptInfo *rel1, RelOptInfo *rel2,
+					 SpecialJoinInfo *sjinfo, List *restrictlist)
+{
+	PartitionOptInfo	*part_info1 = rel1->part_info;
+	PartitionOptInfo	*part_info2 = rel2->part_info;
+	ListCell	*lc;
+	List	   **pkclauses1;
+	List	   **pkclauses2;
+	int		cnt_pks;
+	int		num_pks;
+
+	Assert(part_info1->partnatts == part_info2->partnatts);
+	num_pks = part_info1->partnatts;
+	pkclauses1 = (List **) palloc0(sizeof(List *) * num_pks);
+	pkclauses2 = (List **) palloc0(sizeof(List *) * num_pks);
+
+	foreach (lc, restrictlist)
+	{
+		RestrictInfo *rinfo = lfirst(lc);
+		OpExpr		 *opexpr;
+		Expr		 *expr1;
+		Expr		 *expr2;
+
+		/* If processing an outer join, only use its own join clauses. */
+		if (IS_OUTER_JOIN(sjinfo->jointype) && rinfo->is_pushed_down)
+			continue;
+
+		/* Skip non-equi-join clauses. */
+		if (!rinfo->can_join ||
+			rinfo->hashjoinoperator == InvalidOid ||
+			!rinfo->mergeopfamilies)
+			continue;
+
+		opexpr = (OpExpr *) rinfo->clause;
+		Assert(is_opclause(opexpr));
+
+		/*
+		 * If clause of form rel1_expr op rel2_expr OR rel2_expr op rel1_expr,
+		 * match the operands to the relations. Otherwise, the clause is
+		 * not an equi-join between partition keys of joining relations.
+		 */
+		if (bms_is_subset(rinfo->left_relids, rel1->relids) &&
+			bms_is_subset(rinfo->right_relids, rel2->relids))
+		{
+			expr1 = linitial(opexpr->args);
+			expr2 = lsecond(opexpr->args);
+		}
+		else if (bms_is_subset(rinfo->left_relids, rel2->relids) &&
+				 bms_is_subset(rinfo->right_relids, rel1->relids))
+		{
+			expr1 = lsecond(opexpr->args);
+			expr2 = linitial(opexpr->args);
+		}
+		else
+			continue;
+
+		/* Associate matching clauses with partition keys. */
+		match_expr_to_partition_keys(rinfo, expr1, rel1, pkclauses1);
+		match_expr_to_partition_keys(rinfo, expr2, rel2, pkclauses2);
+	}
+
+	/*
+	 * If every pair of partition key from either of the joining relation has
+	 * at least one equi-join clause associated with it, we have an equi-join
+	 * between all corresponding partition keys.
+	 */
+	for (cnt_pks = 0; cnt_pks < num_pks; cnt_pks++)
+	{
+		List *li = list_intersection(pkclauses1[cnt_pks],
+									 pkclauses2[cnt_pks]);
+		if (!li)
+			return false;
+
+		/* We don't need the actual intersection list. */
+		list_free(li);
+	}
+	return true;
+}
+
+/*
+ * match_expr_to_partition_keys
+ *
+ * Find the partition key which is same as the given expression. If we are able
+ * to find one, add the clause to the list of clauses associated with that
+ * partition key. It is assumed that the given expression is part of the given
+ * clause.
+ *
+ * TODO: Since a given partition key appears only once in the partition by
+ * clause, it should be possible to return the position of the matching
+ * partition key and let the caller handle association of clauses. If we do so,
+ * we will be able to discard the clauses associated with different partition
+ * keys and just remember whether a pair of partition keys has associated
+ * clause or not (rather than remembering the whole list of clauses).
+ */
+static void
+match_expr_to_partition_keys(RestrictInfo *rinfo, Expr *expr, RelOptInfo *rel,
+							 List **pkclauses)
+{
+	PartitionOptInfo *part_info = rel->part_info;
+	int		cnt_pks;
+
+	/*
+	 * Remove the relabel decoration. We can assume that there is at most one
+	 * RelabelType node; eval_const_expressions() will have simplied if more
+	 * than one.
+	 */
+	if (IsA(expr, RelabelType))
+		expr = (Expr *) ((RelabelType *) expr)->arg;
+
+	for (cnt_pks = 0; cnt_pks < part_info->partnatts; cnt_pks++)
+	{
+		List	 *pkexprs = part_info->partexprs[cnt_pks];
+		ListCell *lc;
+
+		foreach(lc, pkexprs)
+		{
+			Expr *pkexpr = lfirst(lc);
+			if (equal(pkexpr, expr))
+			{
+				pkclauses[cnt_pks] = list_append_unique(pkclauses[cnt_pks],
+														rinfo);
+				break;
+			}
+		}
+	}
+}
+
+/*
+ * Add append paths for the join relation.
+ *
+ * Like set_append_rel_pathlist, this function considers pair-wise partition
+ * join paths with parameterization and pathkeys.
+ *
+ * TODO: right now the function just picks up the cheapest path from each of
+ * the partitions and creates an append path with those.
+ *
+ * TODO: may be we should consider splitting set_append_rel_pathlist() so that
+ * it can be used for both inheritance and partitioning.
+ */
+static void
+add_append_paths_to_joinrel(RelOptInfo *joinrel)
+{
+	PartitionOptInfo	*part_info = joinrel->part_info;
+	RelOptInfo **part_rels = part_info->part_rels;
+	int		nparts;
+	int		cnt_parts;
+	List   *part_paths = NIL;
+
+	Assert(part_info);
+
+	nparts = part_info->nparts;
+
+	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	{
+		RelOptInfo *part_rel = part_rels[cnt_parts];
+
+		/* Find the cheapest path for partition relation. */
+		set_cheapest(part_rel);
+
+		/* We don't expect any parameterization here. */
+		Assert(!part_rel->cheapest_total_path->param_info);
+
+		/*
+		 * Instead of lappend, we should use accumulate_append_subpath() to
+		 * pull up the paths in underlying append.
+		 */
+		part_paths = lappend(part_paths, part_rel->cheapest_total_path);
+	}
+
+	add_path(joinrel, (Path *) create_append_path(joinrel, part_paths, NULL,
+												  0));
+	return;
+}
 
 /*
  * have_join_order_restriction
  *		Detect whether the two relations should be joined to satisfy
  *		a join-order restriction arising from special or lateral joins.
  *
  * In practice this is always used with have_relevant_joinclause(), and so
  * could be merged with that function, but it seems clearer to separate the
  * two concerns.  We need this test because there are degenerate cases where
  * a clauseless join must be performed to satisfy join-order restrictions.
@@ -1242,10 +1608,180 @@ restriction_is_constant_false(List *restrictlist, bool only_pushed_down)
 
 			/* constant NULL is as good as constant FALSE for our purposes */
 			if (con->constisnull)
 				return true;
 			if (!DatumGetBool(con->constvalue))
 				return true;
 		}
 	}
 	return false;
 }
+
+/*
+ * have_same_partition_lists
+ *
+ * For given list partitioned relations, return true if lists for all the
+ * partitions of both the relations match. Return false otherwise.
+ */
+static bool
+have_same_partition_lists(PartitionOptInfo *part_info1,
+						  PartitionOptInfo *part_info2)
+{
+	int		cnt_parts;
+
+	Assert(part_info1->strategy == PARTITION_STRAT_LIST &&
+		   part_info2->strategy == PARTITION_STRAT_LIST);
+
+	Assert(part_info1->nparts == part_info2->nparts);
+
+	/* List partition has only one partition key. */
+	Assert(part_info1->partnatts == 1 && part_info2->partnatts == 1);
+
+	for (cnt_parts = 0; cnt_parts < part_info1->nparts; cnt_parts++)
+	{
+		PartitionListBound *list1 = part_info1->lists[cnt_parts];
+		PartitionListBound *list2 = part_info2->lists[cnt_parts];
+		int		cnt;
+
+		/*
+		 * If number of items in the lists do not match or the one of the lists
+		 * contains NULL value and other doesn't, lists do not match.
+		 *
+		 * TODO: This condition can be relaxed to accomodate subset of items,
+		 * but probably not in the first cut.
+		 */
+		if (list1->contains_null != list2->contains_null ||
+			list1->nvalues != list2->nvalues)
+			return false;
+
+		/* Compare actual values in the list. */
+		for (cnt = 0; cnt < list1->nvalues; cnt++)
+		{
+			Datum	cmpval;
+
+			cmpval = FunctionCall2Coll(&part_info1->partsupfunc[0],
+									   part_info1->tcinfo->typcoll[0],
+									   list1->values[cnt],
+									   list2->values[cnt]);
+			if (DatumGetInt32(cmpval))
+				return false;
+		}
+	}
+
+	/* Ok, everything matches, return true. */
+	return true;
+}
+
+/*
+ * have_same_partition_bounds
+ *
+ * For given partitioned relations, return true if the bounds of all the
+ * partitions of the both the relations match. Return false otherwise.
+ */
+static bool
+have_same_partition_bounds(PartitionOptInfo *part_info1,
+						   PartitionOptInfo *part_info2)
+{
+	int		cnt_parts;
+	Assert(part_info1->strategy == PARTITION_STRAT_RANGE &&
+		   part_info2->strategy == PARTITION_STRAT_RANGE);
+
+	Assert(part_info1->nparts == part_info2->nparts);
+
+	for (cnt_parts = 0; cnt_parts < part_info1->nparts; cnt_parts++)
+	{
+		PartitionRangeBound *lower1 = part_info1->rangelowers[cnt_parts];
+		PartitionRangeBound *lower2 = part_info2->rangelowers[cnt_parts];
+		PartitionRangeBound *upper1 = part_info1->rangeuppers[cnt_parts];
+		PartitionRangeBound *upper2 = part_info2->rangeuppers[cnt_parts];
+
+		if (!compare_partition_bounds(part_info1, lower1, lower2) ||
+			!compare_partition_bounds(part_info1, upper1, upper2))
+			return false;
+	}
+
+	/* Ok, everything matches. */
+	return true;
+}
+
+/*
+ * compare_partition_bounds
+ *
+ * Return true if given bound specifications match.
+ */
+static bool
+compare_partition_bounds(PartitionOptInfo *part_info,
+						 PartitionRangeBound *bound1,
+						 PartitionRangeBound *bound2)
+{
+	int		cnt_pks;
+	Datum  *val1 = bound1->val;
+	Datum  *val2 = bound2->val;
+
+	if (bound1->infinite != bound2->infinite ||
+		bound1->inclusive != bound2->inclusive	||
+		bound1->lower != bound2->lower)
+		return false;
+
+	/*
+	 * TODO:
+	 * Following part is common with compare_range_keys(), but that takes
+	 * PartitionKey as argument, which we don't have here.
+	 */
+	for (cnt_pks = 0; cnt_pks < part_info->partnatts; cnt_pks++)
+	{
+		Datum	cmpval;
+
+		cmpval = FunctionCall2Coll(&part_info->partsupfunc[cnt_pks],
+								   part_info->tcinfo->typcoll[cnt_pks],
+								   val1[cnt_pks], val2[cnt_pks]);
+
+		if (DatumGetInt32(cmpval))
+			return false;
+	}
+
+	/* Ok, everything matches. */
+	return true;
+}
+
+/*
+ * partkey_types_match
+ *
+ * Returns true if the partition key datatype and collation match for given
+ * partitioned relations. Returns false otherwise.
+ */
+static bool
+partkey_types_match(PartitionOptInfo *part_info1, PartitionOptInfo *part_info2)
+{
+	int		cnt_pks;
+	Oid	   *partopfamily1 = part_info1->partopfamily;
+	Oid	   *partopfamily2 = part_info2->partopfamily;
+	Oid	   *partopcintype1 = part_info1->partopcintype;
+	Oid	   *partopcintype2 = part_info2->partopcintype;
+	FmgrInfo *partsupfunc1 = part_info1->partsupfunc;
+	FmgrInfo *partsupfunc2 = part_info2->partsupfunc;
+	KeyTypeCollInfo *tcinfo1 = part_info1->tcinfo;
+	KeyTypeCollInfo *tcinfo2 = part_info2->tcinfo;
+
+	if (part_info1->partnatts != part_info1->partnatts)
+		return false;
+
+	for (cnt_pks = 0; cnt_pks < part_info1->partnatts; cnt_pks++)
+	{
+		/*
+		 * It suffices to check the OID of support function as it always has
+		 * two arguemnts and returns boolean. For types, it suffices to match
+		 * the type id, mod and collation; len, byval and align are depedent on
+		 * the first two.
+		 */
+		if (partopfamily1[cnt_pks] != partopfamily2[cnt_pks] ||
+			partopcintype1[cnt_pks] != partopcintype2[cnt_pks] ||
+			partsupfunc1[cnt_pks].fn_oid != partsupfunc2[cnt_pks].fn_oid ||
+			tcinfo1->typid[cnt_pks] != tcinfo2->typid[cnt_pks] ||
+			tcinfo1->typmod[cnt_pks] != tcinfo2->typmod[cnt_pks] ||
+			tcinfo1->typcoll[cnt_pks] != tcinfo2->typcoll[cnt_pks])
+			return false;
+	}
+
+	/* Everything matches. */
+	return true;
+}
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 9e5b60e..6cd8d8c 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -339,20 +339,25 @@ create_plan(PlannerInfo *root, Path *best_path)
 
 /*
  * create_plan_recurse
  *	  Recursive guts of create_plan().
  */
 static Plan *
 create_plan_recurse(PlannerInfo *root, Path *best_path, int flags)
 {
 	Plan	   *plan;
 
+	/* TODO: remove this code to continue partition-wise join planning. */
+	if (best_path->parent->reloptkind == RELOPT_JOINREL &&
+		best_path->parent->part_info)
+		elog(ERROR, "the relation was considered for partition-wise join, which is not supported right now.");
+
 	switch (best_path->pathtype)
 	{
 		case T_SeqScan:
 		case T_SampleScan:
 		case T_IndexScan:
 		case T_IndexOnlyScan:
 		case T_BitmapHeapScan:
 		case T_TidScan:
 		case T_SubqueryScan:
 		case T_FunctionScan:
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index 95aca08..739ce3d 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -1202,21 +1202,28 @@ get_relation_constraints(PlannerInfo *root,
 					ntest->nulltesttype = IS_NOT_NULL;
 					ntest->argisrow = type_is_rowtype(att->atttypid);
 					ntest->location = -1;
 					result = lappend(result, ntest);
 				}
 			}
 		}
 	}
 
 	/* Append partition check quals, if any */
-	pcqual = RelationGetPartitionCheckQual(relation);
+	/*
+	 * TODO the copyObject wrapper has been added by me to avoid changing
+	 * partitioned copy of cache. This should be fixed by Amit in his original
+	 * patches and mostly in get_check_qual_for_range() or
+	 * get_relation_constraints(). Once it's fixed their, remove the copyObject
+	 * wrapper.
+	 */
+	pcqual = copyObject(RelationGetPartitionCheckQual(relation));
 	if (pcqual)
 	{
 		/* Fix Vars to have the desired varno */
 		if (varno != 1)
 			ChangeVarNodes((Node *) pcqual, 1, varno, 0);
 
 		result = list_concat(result, pcqual);
 	}
 
 	heap_close(relation, NoLock);
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index a0a284b..b310c4d 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -8,31 +8,35 @@
  *
  *
  * IDENTIFICATION
  *	  src/backend/optimizer/util/relnode.c
  *
  *-------------------------------------------------------------------------
  */
 #include "postgres.h"
 
 #include "miscadmin.h"
+#include "catalog/heap.h"
 #include "catalog/pg_class.h"
 #include "foreign/foreign.h"
+#include "nodes/makefuncs.h"
 #include "optimizer/clauses.h"
 #include "optimizer/cost.h"
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
 #include "optimizer/placeholder.h"
 #include "optimizer/plancat.h"
 #include "optimizer/restrictinfo.h"
 #include "optimizer/tlist.h"
+#include "rewrite/rewriteManip.h"
 #include "utils/hsearch.h"
+#include "utils/rel.h"
 
 
 typedef struct JoinHashEntry
 {
 	Relids		join_relids;	/* hash key --- MUST BE FIRST */
 	RelOptInfo *join_rel;
 } JoinHashEntry;
 
 static void build_joinrel_tlist(PlannerInfo *root, RelOptInfo *joinrel,
 					RelOptInfo *input_rel);
@@ -42,20 +46,22 @@ static List *build_joinrel_restrictlist(PlannerInfo *root,
 						   RelOptInfo *inner_rel);
 static void build_joinrel_joinlist(RelOptInfo *joinrel,
 					   RelOptInfo *outer_rel,
 					   RelOptInfo *inner_rel);
 static List *subbuild_joinrel_restrictlist(RelOptInfo *joinrel,
 							  List *joininfo_list,
 							  List *new_restrictlist);
 static List *subbuild_joinrel_joinlist(RelOptInfo *joinrel,
 						  List *joininfo_list,
 						  List *new_joininfo);
+static PartitionOptInfo *build_partition_info(PlannerInfo *root,
+							   RelOptInfo *rel);
 
 
 /*
  * setup_simple_rel_arrays
  *	  Prepare the arrays we use for quickly accessing base relations.
  */
 void
 setup_simple_rel_arrays(PlannerInfo *root)
 {
 	Index		rti;
@@ -190,27 +196,33 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptKind reloptkind)
 
 		userid = OidIsValid(rte->checkAsUser) ? rte->checkAsUser : GetUserId();
 		rel->umid = GetUserMappingId(userid, rel->serverid, true);
 	}
 	else
 		rel->umid = InvalidOid;
 
 	/* Save the finished struct in the query's simple_rel_array */
 	root->simple_rel_array[relid] = rel;
 
+	/* Get the partitioning information, if any. */
+	if (rte->rtekind == RTE_RELATION)
+		rel->part_info = build_partition_info(root, rel);
+	else
+		rel->part_info = NULL;
+
 	/*
-	 * If this rel is an appendrel parent, recurse to build "other rel"
-	 * RelOptInfos for its children.  They are "other rels" because they are
-	 * not in the main join tree, but we will need RelOptInfos to plan access
-	 * to them.
+	 * If this rel is an unpartitioned appendrel parent, recurse to build
+	 * "other rel" RelOptInfos for its children.  They are "other rels" because
+	 * they are not in the main join tree, but we will need RelOptInfos to plan
+	 * access to them.
 	 */
-	if (rte->inh)
+	if (!rel->part_info && rte->inh)
 	{
 		ListCell   *l;
 
 		foreach(l, root->append_rel_list)
 		{
 			AppendRelInfo *appinfo = (AppendRelInfo *) lfirst(l);
 
 			/* append_rel_list contains all append rels; ignore others */
 			if (appinfo->parent_relid != relid)
 				continue;
@@ -381,21 +393,34 @@ build_join_rel(PlannerInfo *root,
 														   joinrel,
 														   outer_rel,
 														   inner_rel);
 		return joinrel;
 	}
 
 	/*
 	 * Nope, so make one.
 	 */
 	joinrel = makeNode(RelOptInfo);
-	joinrel->reloptkind = RELOPT_JOINREL;
+
+	/*
+	 * A join between partitions or child tables is different from join between
+	 * regular tables.
+	 */
+	if (IS_OTHER_REL(outer_rel->reloptkind))
+	{
+		/* A partition table can be joined only with other partition table. */
+		Assert(IS_OTHER_REL(inner_rel->reloptkind));
+		joinrel->reloptkind = RELOPT_OTHER_JOINREL;
+	}
+	else
+		joinrel->reloptkind = RELOPT_JOINREL;
+
 	joinrel->relids = bms_copy(joinrelids);
 	joinrel->rows = 0;
 	/* cheap startup cost is interesting iff not all tuples to be retrieved */
 	joinrel->consider_startup = (root->tuple_fraction > 0);
 	joinrel->consider_param_startup = false;
 	joinrel->consider_parallel = false;
 	joinrel->reltarget = create_empty_pathtarget();
 	joinrel->pathlist = NIL;
 	joinrel->ppilist = NIL;
 	joinrel->partial_pathlist = NIL;
@@ -540,21 +565,21 @@ build_join_rel(PlannerInfo *root,
 		Assert(!found);
 		hentry->join_rel = joinrel;
 	}
 
 	/*
 	 * Also, if dynamic-programming join search is active, add the new joinrel
 	 * to the appropriate sublist.  Note: you might think the Assert on number
 	 * of members should be for equality, but some of the level 1 rels might
 	 * have been joinrels already, so we can only assert <=.
 	 */
-	if (root->join_rel_level)
+	if (root->join_rel_level && joinrel->reloptkind != RELOPT_OTHER_JOINREL)
 	{
 		Assert(root->join_cur_level > 0);
 		Assert(root->join_cur_level <= bms_num_members(joinrel->relids));
 		root->join_rel_level[root->join_cur_level] =
 			lappend(root->join_rel_level[root->join_cur_level], joinrel);
 	}
 
 	return joinrel;
 }
 
@@ -1317,10 +1342,179 @@ get_appendrel_parampathinfo(RelOptInfo *appendrel, Relids required_outer)
 
 	/* Else build the ParamPathInfo */
 	ppi = makeNode(ParamPathInfo);
 	ppi->ppi_req_outer = required_outer;
 	ppi->ppi_rows = 0;
 	ppi->ppi_clauses = NIL;
 	appendrel->ppilist = lappend(appendrel->ppilist, ppi);
 
 	return ppi;
 }
+
+/*
+ * build_partition_info
+ *
+ * Retrieves partitioning information for given relation.
+ *
+ * The function also builds the RelOptInfos of the partitions recursively.
+ * TODO: complete the prologue.
+ */
+static PartitionOptInfo *
+build_partition_info(PlannerInfo *root, RelOptInfo *rel)
+{
+	Relation	relation;
+	PartitionKey	part_key;
+	PartitionDesc	part_desc;
+	PartitionOptInfo *part_info;
+	RangeTblEntry  *rte;
+	ListCell	*lc;
+	int		num_pkexprs;
+	int		cnt_pke;
+	int		cnt_parts;
+	Expr   *pkexpr;
+	Index	top_parent_relid;
+
+	/* The given relation should be simple relation. */
+	Assert(rel->reloptkind == RELOPT_BASEREL ||
+		   rel->reloptkind == RELOPT_OTHER_MEMBER_REL);
+	Assert(rel->relid != 0);
+
+	rte = root->simple_rte_array[rel->relid];
+	Assert(rte);
+
+	/*
+	 * For multi-level partitioned tables the children themselves can be
+	 * partitioned. In such a case, partitions at higher levels will have
+	 * parent relid of the topmost relation.
+	 */
+	if (rel->reloptkind == RELOPT_OTHER_MEMBER_REL)
+	{
+		RelOptInfo *parent_rel = find_childrel_top_parent(root, rel);
+		RangeTblEntry *parent_rte;
+
+		top_parent_relid = parent_rel->relid;
+		parent_rte = root->simple_rte_array[top_parent_relid];
+
+		/*
+		 * A relation appearing in its own partition hierarchy does not have
+		 * PartitionOptInfo in its RelOptInfo, lest it creates recursively
+		 * infinite chain of RelOptInfos and PartitionOptInfos.
+		 */
+		if (parent_rte->relid == rte->relid)
+			return NULL;
+	}
+	else
+		top_parent_relid = rel->relid;
+
+	/*
+	 * We need not lock the relation since it was already locked, either
+	 * by the rewriter or when expand_inherited_rtentry() added it to
+	 * the query's rangetable.
+	 */
+	relation = heap_open(rte->relid, NoLock);
+	part_desc = RelationGetPartitionDesc(relation);
+	part_key = RelationGetPartitionKey(relation);
+
+	/* Nothing to do for an unpartitioned relation. */
+	if (!part_desc || !part_key)
+	{
+		heap_close(relation, NoLock);
+		return NULL;
+	}
+
+	part_info = makeNode(PartitionOptInfo);
+
+	/* Store partition descriptor information. */
+	part_info->nparts = part_desc->nparts;
+	/* TODO: Should we copy the contents of the these arrays? */
+	part_info->lists = part_desc->lists;
+	part_info->rangelowers = part_desc->rangelowers;
+	part_info->rangeuppers = part_desc->rangeuppers;
+
+	/* Store partition key information. */
+	part_info->strategy = part_key->strategy;
+	part_info->partnatts = part_key->partnatts;
+	num_pkexprs = part_info->partnatts;
+	/* TODO: Should we copy the contents of these arrays? */
+	part_info->partopfamily = part_key->partopfamily;
+	part_info->partopcintype = part_key->partopcintype;
+	part_info->partsupfunc = part_key->partsupfunc;
+	part_info->tcinfo = part_key->tcinfo;
+
+	/* Store partition keys as single elements lists. */
+	part_info->partexprs = (List **) palloc(sizeof(List *) * num_pkexprs);
+	lc = list_head(part_key->partexprs);
+	for (cnt_pke = 0; cnt_pke < num_pkexprs; cnt_pke++)
+	{
+		AttrNumber attno = part_key->partattrs[cnt_pke];
+
+		if (attno != InvalidAttrNumber)
+		{
+			/* Single column partition key is stored as a Var node. */
+			Form_pg_attribute att_tup;
+
+			if (attno < 0)
+				att_tup = SystemAttributeDefinition(attno,
+												relation->rd_rel->relhasoids);
+			else
+				att_tup = relation->rd_att->attrs[attno - 1];
+
+			pkexpr = (Expr *) makeVar(rel->relid, attno, att_tup->atttypid,
+									  att_tup->atttypmod,
+									  att_tup->attcollation, 0);
+		}
+		else
+		{
+			if (!lc)
+				elog(ERROR, "wrong number of partition key expressions");
+			pkexpr = (Expr *) copyObject(lfirst(lc));
+			ChangeVarNodes((Node *) pkexpr, 1, rel->relid, 0);
+			lc = lnext(lc);
+		}
+
+		part_info->partexprs[cnt_pke] = list_make1(pkexpr);
+	}
+
+	/* Find RelOptInfo of the partitions. */
+	part_info->part_rels = (RelOptInfo **) palloc0(sizeof(RelOptInfo *) *
+												 part_info->nparts);
+	foreach(lc, root->append_rel_list)
+	{
+		AppendRelInfo *appinfo = (AppendRelInfo *) lfirst(lc);
+		int			childRTindex = appinfo->child_relid;
+		RangeTblEntry *childRTE = root->simple_rte_array[childRTindex];
+		RelOptInfo *childrel = root->simple_rel_array[childRTindex];
+
+		/* append_rel_list contains all append rels; ignore others */
+		if (appinfo->parent_relid != top_parent_relid)
+			continue;
+
+		/* If we haven't created a RelOptInfo aleady, create one. */
+		if (!childrel)
+			childrel = build_simple_rel(root, childRTindex,
+									RELOPT_OTHER_MEMBER_REL);
+		/*
+		 * OIDs of the partitions are arranged to match the partition bounds or
+		 * list in corresponding arrays. Arrange RelOptInfo's of partitions in
+		 * the same fashion.
+		 */
+		for (cnt_parts = 0; cnt_parts < part_info->nparts; cnt_parts++)
+		{
+			if (part_desc->oids[cnt_parts] == childRTE->relid)
+			{
+				/* Every partition can be seen only once. */
+				Assert(!part_info->part_rels[cnt_parts]);
+				part_info->part_rels[cnt_parts] = childrel;
+
+				break;
+			}
+		}
+	}
+
+	heap_close(relation, NoLock);
+
+	/* We must have found RelOptInfos of all the partitions. */
+	for (cnt_parts = 0; cnt_parts < part_info->nparts; cnt_parts++)
+		Assert(part_info->part_rels[cnt_parts]);
+
+	return part_info;
+}
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 60148b8..2beb0ec 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -870,20 +870,29 @@ static struct config_bool ConfigureNamesBool[] =
 	},
 	{
 		{"enable_hashjoin", PGC_USERSET, QUERY_TUNING_METHOD,
 			gettext_noop("Enables the planner's use of hash join plans."),
 			NULL
 		},
 		&enable_hashjoin,
 		true,
 		NULL, NULL, NULL
 	},
+	{
+		{"enable_partition_wise_join", PGC_USERSET, QUERY_TUNING_METHOD,
+			gettext_noop("Enables partition-wise join."),
+			NULL
+		},
+		&enable_partition_wise_join,
+		true,
+		NULL, NULL, NULL
+	},
 
 	{
 		{"geqo", PGC_USERSET, QUERY_TUNING_GEQO,
 			gettext_noop("Enables genetic query optimization."),
 			gettext_noop("This algorithm attempts to do planning without "
 						 "exhaustive searching.")
 		},
 		&enable_geqo,
 		true,
 		NULL, NULL, NULL
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index 365073b..b17de61 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -217,20 +217,21 @@ typedef enum NodeTag
 	T_DomainConstraintState,
 
 	/*
 	 * TAGS FOR PLANNER NODES (relation.h)
 	 */
 	T_PlannerInfo = 500,
 	T_PlannerGlobal,
 	T_RelOptInfo,
 	T_IndexOptInfo,
 	T_ForeignKeyOptInfo,
+	T_PartitionOptInfo,
 	T_ParamPathInfo,
 	T_Path,
 	T_IndexPath,
 	T_BitmapHeapPath,
 	T_BitmapAndPath,
 	T_BitmapOrPath,
 	T_TidPath,
 	T_SubqueryScanPath,
 	T_ForeignPath,
 	T_CustomPath,
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 0b5cb9e..bffd312 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -12,20 +12,21 @@
  *-------------------------------------------------------------------------
  */
 #ifndef RELATION_H
 #define RELATION_H
 
 #include "access/sdir.h"
 #include "lib/stringinfo.h"
 #include "nodes/params.h"
 #include "nodes/parsenodes.h"
 #include "storage/block.h"
+#include "catalog/partition.h"
 
 
 /*
  * Relids
  *		Set of relation identifiers (indexes into the rangetable).
  */
 typedef Bitmapset *Relids;
 
 /*
  * When looking for a "cheapest path", this enum specifies whether we want
@@ -347,20 +348,26 @@ typedef struct PlannerInfo
  * is present in the query join tree but the members are not.  The member
  * RTEs and otherrels are used to plan the scans of the individual tables or
  * subqueries of the append set; then the parent baserel is given Append
  * and/or MergeAppend paths comprising the best paths for the individual
  * member rels.  (See comments for AppendRelInfo for more information.)
  *
  * At one time we also made otherrels to represent join RTEs, for use in
  * handling join alias Vars.  Currently this is not needed because all join
  * alias Vars are expanded to non-aliased form during preprocess_expression.
  *
+ * We also have relations representing pair-wise joins between partitions of
+ * partitioned tables. These relations are not added to join_rel_level lists
+ * as they are not joined directly by the dynamic programming algorithm.
+ * Adding these two join_rel_level list also means that top level list has more
+ * than one join relation, which is symantically incorrect.
+ *
  * There is also a RelOptKind for "upper" relations, which are RelOptInfos
  * that describe post-scan/join processing steps, such as aggregation.
  * Many of the fields in these RelOptInfos are meaningless, but their Path
  * fields always hold Paths showing ways to do that processing step.
  *
  * Lastly, there is a RelOptKind for "dead" relations, which are base rels
  * that we have proven we don't need to join after all.
  *
  * Parts of this data structure are specific to various scan and join
  * mechanisms.  It didn't seem worth creating new node types for them.
@@ -465,24 +472,31 @@ typedef struct PlannerInfo
  * We store baserestrictcost in the RelOptInfo (for base relations) because
  * we know we will need it at least once (to price the sequential scan)
  * and may need it multiple times to price index scans.
  *----------
  */
 typedef enum RelOptKind
 {
 	RELOPT_BASEREL,
 	RELOPT_JOINREL,
 	RELOPT_OTHER_MEMBER_REL,
+	RELOPT_OTHER_JOINREL,
 	RELOPT_UPPER_REL,
 	RELOPT_DEADREL
 } RelOptKind;
 
+#define IS_OTHER_REL(reloptkind) \
+	((reloptkind) == RELOPT_OTHER_MEMBER_REL || \
+	 (reloptkind) == RELOPT_OTHER_JOINREL)
+
+typedef struct PartitionOptInfo PartitionOptInfo;
+
 typedef struct RelOptInfo
 {
 	NodeTag		type;
 
 	RelOptKind	reloptkind;
 
 	/* all relations included in this RelOptInfo */
 	Relids		relids;			/* set of base relids (rangetable indexes) */
 
 	/* size estimates generated by planner */
@@ -536,20 +550,26 @@ typedef struct RelOptInfo
 	struct FdwRoutine *fdwroutine;
 	void	   *fdw_private;
 
 	/* used by various scans and joins: */
 	List	   *baserestrictinfo;		/* RestrictInfo structures (if base
 										 * rel) */
 	QualCost	baserestrictcost;		/* cost of evaluating the above */
 	List	   *joininfo;		/* RestrictInfo structures for join clauses
 								 * involving this rel */
 	bool		has_eclass_joins;		/* T means joininfo is incomplete */
+
+	/* For partitioned relations, joins or base relations. NULL otherwise. */
+	/*
+	 * TODO: Notice recursive usage of RelOptInfo.
+	 */
+	PartitionOptInfo	*part_info;
 } RelOptInfo;
 
 /*
  * IndexOptInfo
  *		Per-index information for planning/optimization
  *
  *		indexkeys[], indexcollations[], opfamily[], and opcintype[]
  *		each have ncolumns entries.
  *
  *		sortopfamily[], reverse_sort[], and nulls_first[] likewise have
@@ -650,20 +670,46 @@ typedef struct ForeignKeyOptInfo
 	/* Derived info about whether FK's equality conditions match the query: */
 	int			nmatched_ec;	/* # of FK cols matched by ECs */
 	int			nmatched_rcols; /* # of FK cols matched by non-EC rinfos */
 	int			nmatched_ri;	/* total # of non-EC rinfos matched to FK */
 	/* Pointer to eclass matching each column's condition, if there is one */
 	struct EquivalenceClass *eclass[INDEX_MAX_KEYS];
 	/* List of non-EC RestrictInfos matching each column's condition */
 	List	   *rinfos[INDEX_MAX_KEYS];
 } ForeignKeyOptInfo;
 
+/*
+ * PartitionOptInfo
+ * 		Partitioning information for planning/optimization
+ *
+ * TODO: complete the comment
+ */
+typedef struct PartitionOptInfo
+{
+	NodeTag		type;
+
+	/* Information about partitions */
+	int			nparts;		/* number of partitions */
+	RelOptInfo			  **part_rels;	/* RelOptInfos of partitions */
+	PartitionListBound	  **lists;		/* list bounds */
+	PartitionRangeBound	  **rangelowers;	/* range lower bounds */
+	PartitionRangeBound	  **rangeuppers;	/* range upper bounds */
+
+	/* Information about partition keys */
+	char		strategy;		/* partition strategy */
+	int16		partnatts;		/* number of partition attributes */
+	List	  **partexprs;		/* partition key expressions. */
+	Oid		   *partopfamily;	/* OIDs of operator families */
+	Oid		   *partopcintype;	/* OIDs of opclass declared input data types */
+	FmgrInfo   *partsupfunc;	/* lookup info for support funcs */
+	KeyTypeCollInfo *tcinfo;	/* type and collation info (all columns) */
+} PartitionOptInfo;
 
 /*
  * EquivalenceClasses
  *
  * Whenever we can determine that a mergejoinable equality clause A = B is
  * not delayed by any outer join, we create an EquivalenceClass containing
  * the expressions A and B to record this knowledge.  If we later find another
  * equivalence B = C, we add C to the existing EquivalenceClass; this may
  * require merging two existing EquivalenceClasses.  At the end of the qual
  * distribution process, we have sets of values that are known all transitively
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 2a4df2f..1069726 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -59,20 +59,21 @@ extern bool enable_seqscan;
 extern bool enable_indexscan;
 extern bool enable_indexonlyscan;
 extern bool enable_bitmapscan;
 extern bool enable_tidscan;
 extern bool enable_sort;
 extern bool enable_hashagg;
 extern bool enable_nestloop;
 extern bool enable_material;
 extern bool enable_mergejoin;
 extern bool enable_hashjoin;
+extern bool enable_partition_wise_join;
 extern int	constraint_exclusion;
 
 extern double clamp_row_est(double nrows);
 extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
 					double index_pages, PlannerInfo *root);
 extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 			 ParamPathInfo *param_info);
 extern void cost_samplescan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 				ParamPathInfo *param_info);
 extern void cost_index(IndexPath *path, PlannerInfo *root,
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
new file mode 100644
index 0000000..a41f8b6
--- /dev/null
+++ b/src/test/regress/expected/partition_join.out
@@ -0,0 +1,1365 @@
+--
+-- PARTITION_JOIN
+-- Test partition-wise join between partitioned tables
+--
+-- TODO: We need to add tests for SEMI-ANTI joins.
+-- TODO: tests with placeholder vars
+--
+-- partitioned by a single column
+--
+CREATE TABLE pt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE pt1_p1 PARTITION OF pt1 FOR VALUES START (0) END (250);
+CREATE TABLE pt1_p2 PARTITION OF pt1 FOR VALUES START (250) END (500);
+CREATE TABLE pt1_p3 PARTITION OF pt1 FOR VALUES START (500) END (600);
+INSERT INTO pt1 SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE upt1 AS SELECT * FROM pt1;
+CREATE TABLE pt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE pt2_p1 PARTITION OF pt2 FOR VALUES START (0) END (250);
+CREATE TABLE pt2_p2 PARTITION OF pt2 FOR VALUES START (250) END (500);
+CREATE TABLE pt2_p3 PARTITION OF pt2 FOR VALUES START (500) END (600);
+INSERT INTO pt2 SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE upt2 AS SELECT * FROM pt2;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1 t1, pt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1 t1, pt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM upt1 t1, upt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is NOT considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0150 | 150 | 0150
+ 300 | 0300 | 300 | 0300
+ 450 | 0450 | 450 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1 t1 LEFT JOIN pt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1 t1 LEFT JOIN pt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM upt1 t1 LEFT JOIN upt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is NOT considered for partition-wise join.
+  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 |     | 
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1 t1 RIGHT JOIN pt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1 t1 RIGHT JOIN pt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM upt1 t1 RIGHT JOIN upt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is NOT considered for partition-wise join.
+  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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM pt1 WHERE pt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM pt2 WHERE pt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM pt1 WHERE pt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM pt2 WHERE pt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM upt1 t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM upt2 t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is NOT considered for partition-wise join.
+  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)
+
+--
+-- partitioned by expression
+--
+CREATE TABLE pt1_e (a int, b int, c varchar) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE pt1_e_p1 PARTITION OF pt1_e FOR VALUES START (0) END (250);
+CREATE TABLE pt1_e_p2 PARTITION OF pt1_e FOR VALUES START (250) END (500);
+CREATE TABLE pt1_e_p3 PARTITION OF pt1_e FOR VALUES START (500) END (600);
+INSERT INTO pt1_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE upt1_e AS SELECT * FROM pt1_e;
+CREATE TABLE pt2_e (a int, b int, c varchar) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE pt2_e_p1 PARTITION OF pt2_e FOR VALUES START (0) END (250);
+CREATE TABLE pt2_e_p2 PARTITION OF pt2_e FOR VALUES START (250) END (500);
+CREATE TABLE pt2_e_p3 PARTITION OF pt2_e FOR VALUES START (500) END (600);
+INSERT INTO pt2_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE upt2_e AS SELECT * FROM pt2_e;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_e t1, pt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_e t1, pt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM upt1_e t1, upt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is NOT considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0150 | 150 | 0150
+ 300 | 0300 | 300 | 0300
+ 450 | 0450 | 450 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_e t1 LEFT JOIN pt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_e t1 LEFT JOIN pt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM upt1_e t1 LEFT JOIN upt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is NOT considered for partition-wise join.
+  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 |     | 
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_e t1 RIGHT JOIN pt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_e t1 RIGHT JOIN pt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM upt1_e t1 RIGHT JOIN upt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is NOT considered for partition-wise join.
+  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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM pt1_e WHERE pt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM pt2_e WHERE pt2_e.b % 25 = 0) t2 ON ((t1.a + t1.b)/2 = (t2.b + t2.a)/2) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM pt1_e WHERE pt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM pt2_e WHERE pt2_e.b % 25 = 0) t2 ON ((t1.a + t1.b)/2 = (t2.b + t2.a)/2) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM upt1_e t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM upt2_e t2 WHERE t2.b % 25 = 0) t2 ON ((t1.a + t1.b)/2 = (t2.b + t2.a)/2) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is NOT considered for partition-wise join.
+  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)
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM pt1 t1, pt2 t2, pt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 3) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 3) and (b 1) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM pt1 t1, pt2 t2, pt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 3) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 3) and (b 1) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM upt1 t1, upt2 t2, upt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 3) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 3) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 1 3) and (b 2) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 2 3) and (b 1) is NOT considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (pt1 t1 LEFT JOIN pt2 t2 ON t1.a = t2.b) LEFT JOIN pt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (pt1 t1 LEFT JOIN pt2 t2 ON t1.a = t2.b) LEFT JOIN pt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (upt1 t1 LEFT JOIN upt2 t2 ON t1.a = t2.b) LEFT JOIN upt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is NOT considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (pt1 t1 LEFT JOIN pt2 t2 ON t1.a = t2.b) LEFT JOIN pt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2 4) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (pt1 t1 LEFT JOIN pt2 t2 ON t1.a = t2.b) LEFT JOIN pt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2 4) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (upt1 t1 LEFT JOIN upt2 t2 ON t1.a = t2.b) LEFT JOIN upt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 4) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2 4) is NOT considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |          | 
+ 100 | 0100 |     |      |          | 
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |          | 
+ 250 | 0250 |     |      |          | 
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |          | 
+ 400 | 0400 |     |      |          | 
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |          | 
+ 550 | 0550 |     |      |          | 
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (pt1 t1 LEFT JOIN pt2 t2 ON t1.a = t2.b) RIGHT JOIN pt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (pt1 t1 LEFT JOIN pt2 t2 ON t1.a = t2.b) RIGHT JOIN pt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (upt1 t1 LEFT JOIN upt2 t2 ON t1.a = t2.b) RIGHT JOIN upt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is NOT considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (pt1 t1 RIGHT JOIN pt2 t2 ON t1.a = t2.b) RIGHT JOIN pt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (pt1 t1 RIGHT JOIN pt2 t2 ON t1.a = t2.b) RIGHT JOIN pt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (upt1 t1 RIGHT JOIN upt2 t2 ON t1.a = t2.b) RIGHT JOIN upt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 4) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is NOT considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM pt1 WHERE pt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM pt2 WHERE pt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM pt1_e WHERE pt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 6) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM pt1 WHERE pt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM pt2 WHERE pt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM pt1_e WHERE pt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 6) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM upt1 WHERE upt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM upt2 WHERE upt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM upt1_e WHERE upt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 6) and (b 7) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is NOT considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+     |      |  75 | 0075 |          | 
+     |      | 225 | 0225 |          | 
+     |      | 375 | 0375 |          | 
+     |      | 525 | 0525 |          | 
+(16 rows)
+
+--
+-- partitioned by multiple columns
+--
+CREATE TABLE pt1_m (a int, b int, c varchar) PARTITION BY RANGE(a, ((a + b)/2));
+CREATE TABLE pt1_m_p1 PARTITION OF pt1_m FOR VALUES START (0, 0) END (250, 250);
+CREATE TABLE pt1_m_p2 PARTITION OF pt1_m FOR VALUES START (250, 250) END (500, 500);
+CREATE TABLE pt1_m_p3 PARTITION OF pt1_m FOR VALUES START (500, 500) END (600, 600);
+INSERT INTO pt1_m SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE upt1_m AS SELECT * FROM pt1_m;
+CREATE TABLE pt2_m (a int, b int, c varchar) PARTITION BY RANGE(((b + a)/2), b);
+CREATE TABLE pt2_m_p1 PARTITION OF pt2_m FOR VALUES START (0, 0) END (250, 250);
+CREATE TABLE pt2_m_p2 PARTITION OF pt2_m FOR VALUES START (250, 250) END (500, 500);
+CREATE TABLE pt2_m_p3 PARTITION OF pt2_m FOR VALUES START (500, 500) END (600, 600);
+INSERT INTO pt2_m SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE upt2_m AS SELECT * FROM pt2_m;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_m t1, pt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_m t1, pt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM upt1_m t1, upt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is NOT considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0150 | 150 | 0150
+ 300 | 0300 | 300 | 0300
+ 450 | 0450 | 450 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_m t1 LEFT JOIN pt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_m t1 LEFT JOIN pt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM upt1_m t1 LEFT JOIN upt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is NOT considered for partition-wise join.
+  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 |     | 
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_m t1 RIGHT JOIN pt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_m t1 RIGHT JOIN pt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM upt1_m t1 RIGHT JOIN upt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is NOT considered for partition-wise join.
+  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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM pt1_m WHERE pt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM pt2_m WHERE pt2_m.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM pt1_m WHERE pt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM pt2_m WHERE pt2_m.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM upt1_m t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM upt2_m t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is NOT considered for partition-wise join.
+  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)
+
+--
+-- multi-leveled partitions
+--
+CREATE TABLE pt1_l (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE pt1_l_p1 PARTITION OF pt1_l FOR VALUES START (0) END (250) PARTITION BY RANGE (b);
+CREATE TABLE pt1_l_p1_p1 PARTITION OF pt1_l_p1 FOR VALUES START (0) END (100);
+CREATE TABLE pt1_l_p1_p2 PARTITION OF pt1_l_p1 FOR VALUES START (100) END (250);
+CREATE TABLE pt1_l_p2 PARTITION OF pt1_l FOR VALUES START (250) END (500) PARTITION BY RANGE (c);
+CREATE TABLE pt1_l_p2_p1 PARTITION OF pt1_l_p2 FOR VALUES START ('0250') END ('0400');
+CREATE TABLE pt1_l_p2_p2 PARTITION OF pt1_l_p2 FOR VALUES START ('0400') END ('0500');
+CREATE TABLE pt1_l_p3 PARTITION OF pt1_l FOR VALUES START (500) END (600) PARTITION BY RANGE ((b + a));
+CREATE TABLE pt1_l_p3_p1 PARTITION OF pt1_l_p3 FOR VALUES START (1000) END (1100);
+CREATE TABLE pt1_l_p3_p2 PARTITION OF pt1_l_p3 FOR VALUES START (1100) END (1200);
+INSERT INTO pt1_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE upt1_l AS SELECT * FROM pt1_l;
+CREATE TABLE pt2_l (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE pt2_l_p1 PARTITION OF pt2_l FOR VALUES START (0) END (250) PARTITION BY RANGE (a);
+CREATE TABLE pt2_l_p1_p1 PARTITION OF pt2_l_p1 FOR VALUES START (0) END (100);
+CREATE TABLE pt2_l_p1_p2 PARTITION OF pt2_l_p1 FOR VALUES START (100) END (250);
+CREATE TABLE pt2_l_p2 PARTITION OF pt2_l FOR VALUES START (250) END (500) PARTITION BY RANGE (c);
+CREATE TABLE pt2_l_p2_p1 PARTITION OF pt2_l_p2 FOR VALUES START ('0250') END ('0400');
+CREATE TABLE pt2_l_p2_p2 PARTITION OF pt2_l_p2 FOR VALUES START ('0400') END ('0500');
+CREATE TABLE pt2_l_p3 PARTITION OF pt2_l FOR VALUES START (500) END (600) PARTITION BY RANGE ((a + b));
+CREATE TABLE pt2_l_p3_p1 PARTITION OF pt2_l_p3 FOR VALUES START (1000) END (1100);
+CREATE TABLE pt2_l_p3_p2 PARTITION OF pt2_l_p3 FOR VALUES START (1100) END (1200);
+INSERT INTO pt2_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE upt2_l AS SELECT * FROM pt2_l;
+--
+-- TODO: a varchar = varchar comparison is converted into a comparison
+-- varchar::text = varchar::text. Thus the partition key on c does not match
+-- the equi-join condition on c. This is fine, but can be improved. Also, as
+-- long as we have this behaviour, we get to test partial partition-wise join
+-- between multi-level partitioned relations.
+--
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_l t1, pt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 14) is considered for partition-wise join.
+NOTICE:  join between relations (b 5) and (b 15) is considered for partition-wise join.
+NOTICE:  join between relations (b 6) and (b 16) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_l t1, pt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 14) is considered for partition-wise join.
+NOTICE:  join between relations (b 5) and (b 15) is considered for partition-wise join.
+NOTICE:  join between relations (b 6) and (b 16) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM upt1_l t1, upt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is NOT considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0150 | 150 | 0150
+ 300 | 0300 | 300 | 0300
+ 450 | 0450 | 450 | 0450
+(4 rows)
+
+-- TODO for outer joins the equality conditions do not appear in joininfo of
+-- children, which means that the equi-join clauses are not present for
+-- subpartitions. Hence the joins between subpartitions are not currently being
+-- considered for partition-wise join.
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_l t1 LEFT JOIN pt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_l t1 LEFT JOIN pt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM upt1_l t1 LEFT JOIN upt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is NOT considered for partition-wise join.
+  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 |     | 
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_l t1 RIGHT JOIN pt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_l t1 RIGHT JOIN pt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM upt1_l t1 RIGHT JOIN upt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is NOT considered for partition-wise join.
+  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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM pt1_l WHERE pt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM pt2_l WHERE pt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM pt1_l WHERE pt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM pt2_l WHERE pt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM upt1_l t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM upt2_l t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is NOT considered for partition-wise join.
+  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)
+
+--
+-- 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;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1 AS SELECT * FROM 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;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt2 AS SELECT * FROM plt2;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.a = t2.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.a = t2.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1 t1, uplt2 t2 WHERE t1.c = t2.c AND t1.a = t2.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is NOT considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0003 | 150 | 0003
+ 300 | 0006 | 300 | 0006
+ 450 | 0009 | 450 | 0009
+(4 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.a AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.a AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.a AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is NOT considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  50 | 0001 |     | 
+ 100 | 0002 |     | 
+ 150 | 0003 | 150 | 0003
+ 200 | 0004 |     | 
+ 250 | 0005 |     | 
+ 300 | 0006 | 300 | 0006
+ 350 | 0007 |     | 
+ 400 | 0008 |     | 
+ 450 | 0009 | 450 | 0009
+ 500 | 0010 |     | 
+ 550 | 0011 |     | 
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is NOT considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0003 | 150 | 0003
+ 300 | 0006 | 300 | 0006
+ 450 | 0009 | 450 | 0009
+     |      |  75 | 0001
+     |      | 225 | 0004
+     |      | 375 | 0007
+     |      | 525 | 0010
+(8 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uplt1 t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is NOT considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  50 | 0001 |     | 
+ 100 | 0002 |     | 
+ 150 | 0003 | 150 | 0003
+ 200 | 0004 |     | 
+ 250 | 0005 |     | 
+ 300 | 0006 | 300 | 0006
+ 350 | 0007 |     | 
+ 400 | 0008 |     | 
+ 450 | 0009 | 450 | 0009
+ 500 | 0010 |     | 
+ 550 | 0011 |     | 
+     |      |  75 | 0001
+     |      | 225 | 0004
+     |      | 375 | 0007
+     |      | 525 | 0010
+(16 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;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1_e AS SELECT * FROM plt1_e;
+CREATE TABLE plt2_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
+CREATE TABLE plt2_e_p1 PARTITION OF plt2_e FOR VALUES IN ('0000', '0003', '0004', '0010');
+CREATE TABLE plt2_e_p2 PARTITION OF plt2_e FOR VALUES IN ('0001', '0005', '0002', '0009');
+CREATE TABLE plt2_e_p3 PARTITION OF plt2_e FOR VALUES IN ('0006', '0007', '0008', '0011');
+INSERT INTO plt2_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt2_e AS SELECT * FROM plt2_e;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1, plt2_e t2 WHERE t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1, plt2_e t2 WHERE t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_e t1, uplt2_e t2 WHERE t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is NOT considered for partition-wise join.
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+ 150 | A0003 | 150 | A0003
+ 300 | A0006 | 300 | A0006
+ 450 | A0009 | 450 | A0009
+(4 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 LEFT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 LEFT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_e t1 LEFT JOIN uplt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is NOT considered for partition-wise join.
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+  50 | A0001 |     | 
+ 100 | A0002 |     | 
+ 150 | A0003 | 150 | A0003
+ 200 | A0004 |     | 
+ 250 | A0005 |     | 
+ 300 | A0006 | 300 | A0006
+ 350 | A0007 |     | 
+ 400 | A0008 |     | 
+ 450 | A0009 | 450 | A0009
+ 500 | A0010 |     | 
+ 550 | A0011 |     | 
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 RIGHT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 RIGHT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_e t1 RIGHT JOIN uplt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is NOT considered for partition-wise join.
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+ 150 | A0003 | 150 | A0003
+ 300 | A0006 | 300 | A0006
+ 450 | A0009 | 450 | A0009
+     |       |  75 | A0001
+     |       | 225 | A0004
+     |       | 375 | A0007
+     |       | 525 | A0010
+(8 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2_e WHERE plt2_e.b % 25 = 0) t2 ON (t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A')) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2_e WHERE plt2_e.b % 25 = 0) t2 ON (t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A')) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uplt1_e t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2_e t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A')) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is NOT considered for partition-wise join.
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+  50 | A0001 |     | 
+ 100 | A0002 |     | 
+ 150 | A0003 | 150 | A0003
+ 200 | A0004 |     | 
+ 250 | A0005 |     | 
+ 300 | A0006 | 300 | A0006
+ 350 | A0007 |     | 
+ 400 | A0008 |     | 
+ 450 | A0009 | 450 | A0009
+ 500 | A0010 |     | 
+ 550 | A0011 |     | 
+     |       |  75 | A0001
+     |       | 225 | A0004
+     |       | 375 | A0007
+     |       | 525 | A0010
+(16 rows)
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.a = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c AND t1.a = t3.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 3) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 3) and (b 1) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.a = t2.b AND t1.c = t2.c AND t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 3) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 3) and (b 1) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM uplt1 t1, uplt2 t2, uplt1_e t3 WHERE t1.a = t2.b AND t1.c = t2.c AND t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 3) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 3) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 1 3) and (b 2) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 2 3) and (b 1) is NOT considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+(4 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is NOT considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2 4) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2 4) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 4) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2 4) is NOT considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |          | 
+ 100 | 0002 |     |      |          | 
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |          | 
+ 250 | 0005 |     |      |          | 
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |          | 
+ 400 | 0008 |     |      |          | 
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |          | 
+ 550 | 0011 |     |      |          | 
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is NOT considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 4) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is NOT considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+(4 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 6) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 6) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is considered for partition-wise join.
+ERROR:  the relation was considered for partition-wise join, which is not supported right now.
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 6) and (b 7) is NOT considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is NOT considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+--
+-- negative testcases
+--
+CREATE TABLE pt1_n (a int, b int, c varchar) PARTITION BY RANGE(c);
+CREATE TABLE pt1_n_p1 PARTITION OF pt1_n FOR VALUES START ('0000') END ('0250');
+CREATE TABLE pt1_n_p2 PARTITION OF pt1_n FOR VALUES START ('0250') END ('0500');
+INSERT INTO pt1_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 499, 2) i;
+CREATE TABLE pt2_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE pt2_n_p1 PARTITION OF pt2_n FOR VALUES IN ('0000', '0003', '0004', '0010', '0006', '0007');
+CREATE TABLE pt2_n_p2 PARTITION OF pt2_n FOR VALUES IN ('0001', '0005', '0002', '0009', '0008', '0011');
+INSERT INTO pt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE pt3_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE pt3_n_p1 PARTITION OF pt3_n FOR VALUES IN ('0000', '0004', '0006', '0007');
+CREATE TABLE pt3_n_p2 PARTITION OF pt3_n FOR VALUES IN ('0001', '0002', '0008', '0010');
+CREATE TABLE pt3_n_p3 PARTITION OF pt3_n FOR VALUES IN ('0003', '0005', '0009', '0011');
+INSERT INTO pt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+CREATE TABLE pt4_n (a int, b int, c text) PARTITION BY RANGE(a);
+CREATE TABLE pt4_n_p1 PARTITION OF pt4_n FOR VALUES START (0) END (300);
+CREATE TABLE pt4_n_p2 PARTITION OF pt4_n FOR VALUES START (300) END (500);
+CREATE TABLE pt4_n_p3 PARTITION OF pt4_n FOR VALUES START (500) END (600);
+INSERT INTO pt4_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+-- partition-wise join can not be applied if the partition ranges differ
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1 t1, pt4_n t2 WHERE t1.a = t2.a;
+NOTICE:  join between relations (b 1) and (b 2) is NOT considered for partition-wise join.
+                 QUERY PLAN                  
+---------------------------------------------
+ Merge Join
+   Merge Cond: (t1.a = t2.a)
+   ->  Sort
+         Sort Key: t1.a
+         ->  Append
+               ->  Seq Scan on pt1 t1
+               ->  Seq Scan on pt1_p1 t1_1
+               ->  Seq Scan on pt1_p2 t1_2
+               ->  Seq Scan on pt1_p3 t1_3
+   ->  Sort
+         Sort Key: t2.a
+         ->  Append
+               ->  Seq Scan on pt4_n t2
+               ->  Seq Scan on pt4_n_p1 t2_1
+               ->  Seq Scan on pt4_n_p2 t2_2
+               ->  Seq Scan on pt4_n_p3 t2_3
+(16 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1 t1 FULL JOIN pt4_n t2 ON t1.a = t2.a;
+NOTICE:  join between relations (b 1) and (b 2) is NOT considered for partition-wise join.
+                 QUERY PLAN                  
+---------------------------------------------
+ Merge Full Join
+   Merge Cond: (t1.a = t2.a)
+   ->  Sort
+         Sort Key: t1.a
+         ->  Append
+               ->  Seq Scan on pt1 t1
+               ->  Seq Scan on pt1_p1 t1_1
+               ->  Seq Scan on pt1_p2 t1_2
+               ->  Seq Scan on pt1_p3 t1_3
+   ->  Sort
+         Sort Key: t2.a
+         ->  Append
+               ->  Seq Scan on pt4_n t2
+               ->  Seq Scan on pt4_n_p1 t2_1
+               ->  Seq Scan on pt4_n_p2 t2_2
+               ->  Seq Scan on pt4_n_p3 t2_3
+(16 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 pt1 t1 LEFT JOIN pt2 t2 ON (t1.a < t2.b);
+NOTICE:  join between relations (b 1) and (b 2) is NOT considered for partition-wise join.
+                QUERY PLAN                 
+-------------------------------------------
+ Nested Loop Left Join
+   Join Filter: (t1.a < t2.b)
+   ->  Append
+         ->  Seq Scan on pt1 t1
+         ->  Seq Scan on pt1_p1 t1_1
+         ->  Seq Scan on pt1_p2 t1_2
+         ->  Seq Scan on pt1_p3 t1_3
+   ->  Materialize
+         ->  Append
+               ->  Seq Scan on pt2 t2
+               ->  Seq Scan on pt2_p1 t2_1
+               ->  Seq Scan on pt2_p2 t2_2
+               ->  Seq Scan on pt2_p3 t2_3
+(13 rows)
+
+-- equi-join with join condition on partial keys does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_m t1, pt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t1.a % 25 = 0;
+NOTICE:  join between relations (b 1) and (b 2) is NOT considered for partition-wise join.
+                 QUERY PLAN                  
+---------------------------------------------
+ Hash Join
+   Hash Cond: (((t2.b + t2.a) / 2) = t1.a)
+   ->  Append
+         ->  Seq Scan on pt2_m t2
+         ->  Seq Scan on pt2_m_p1 t2_1
+         ->  Seq Scan on pt2_m_p2 t2_2
+         ->  Seq Scan on pt2_m_p3 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on pt1_m t1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on pt1_m_p1 t1_1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on pt1_m_p2 t1_2
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on pt1_m_p3 t1_3
+                     Filter: ((a % 25) = 0)
+(17 rows)
+
+-- equi-join between out-of-order partition key columns does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_m t1 LEFT JOIN pt2_m t2 ON t1.a = t2.b WHERE t1.a % 25 = 0;
+NOTICE:  join between relations (b 1) and (b 2) is NOT considered for partition-wise join.
+                 QUERY PLAN                  
+---------------------------------------------
+ Hash Right Join
+   Hash Cond: (t2.b = t1.a)
+   ->  Append
+         ->  Seq Scan on pt2_m t2
+         ->  Seq Scan on pt2_m_p1 t2_1
+         ->  Seq Scan on pt2_m_p2 t2_2
+         ->  Seq Scan on pt2_m_p3 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on pt1_m t1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on pt1_m_p1 t1_1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on pt1_m_p2 t1_2
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on pt1_m_p3 t1_3
+                     Filter: ((a % 25) = 0)
+(17 rows)
+
+-- equi-join between non-key columns does not qualify for partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_m t1 LEFT JOIN pt2_m t2 ON t1.c = t2.c WHERE t1.a % 25 = 0;
+NOTICE:  join between relations (b 1) and (b 2) is NOT considered for partition-wise join.
+                 QUERY PLAN                  
+---------------------------------------------
+ Hash Right Join
+   Hash Cond: ((t2.c)::text = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on pt2_m t2
+         ->  Seq Scan on pt2_m_p1 t2_1
+         ->  Seq Scan on pt2_m_p2 t2_2
+         ->  Seq Scan on pt2_m_p3 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on pt1_m t1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on pt1_m_p1 t1_1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on pt1_m_p2 t1_2
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on pt1_m_p3 t1_3
+                     Filter: ((a % 25) = 0)
+(17 rows)
+
+-- TODO: an equalify condition on the partition keys in WHERE clause should not
+-- qualify for partition-wise join. But such a condition converts an OUTER
+-- join into an INNER join, in which case the conditions in WHERE clause can be
+-- treated as join conditions, thus qualifying the join for partition-wise
+-- join.
+-- partition-wise join can not be applied for a join between list and range
+-- partitioned table
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_n t1, pt2_n t2 WHERE t1.c = t2.c;
+NOTICE:  join between relations (b 1) and (b 2) is NOT considered for partition-wise join.
+                 QUERY PLAN                  
+---------------------------------------------
+ Merge Join
+   Merge Cond: ((t1.c)::text = t2.c)
+   ->  Sort
+         Sort Key: t1.c
+         ->  Append
+               ->  Seq Scan on pt1_n t1
+               ->  Seq Scan on pt1_n_p1 t1_1
+               ->  Seq Scan on pt1_n_p2 t1_2
+   ->  Sort
+         Sort Key: t2.c
+         ->  Append
+               ->  Seq Scan on pt2_n t2
+               ->  Seq Scan on pt2_n_p1 t2_1
+               ->  Seq Scan on pt2_n_p2 t2_2
+(14 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_n t1 LEFT JOIN pt2_n t2 ON (t1.c = t2.c);
+NOTICE:  join between relations (b 1) and (b 2) is NOT considered for partition-wise join.
+                 QUERY PLAN                  
+---------------------------------------------
+ Merge Left Join
+   Merge Cond: ((t1.c)::text = t2.c)
+   ->  Sort
+         Sort Key: t1.c
+         ->  Append
+               ->  Seq Scan on pt1_n t1
+               ->  Seq Scan on pt1_n_p1 t1_1
+               ->  Seq Scan on pt1_n_p2 t1_2
+   ->  Sort
+         Sort Key: t2.c
+         ->  Append
+               ->  Seq Scan on pt2_n t2
+               ->  Seq Scan on pt2_n_p1 t2_1
+               ->  Seq Scan on pt2_n_p2 t2_2
+(14 rows)
+
+-- partition-wise join can not be applied between tables with different
+-- partition lists
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_n t1 RIGHT JOIN pt1 t2 ON (t1.c = t2.c);
+NOTICE:  join between relations (b 2) and (b 1) is NOT considered for partition-wise join.
+                 QUERY PLAN                  
+---------------------------------------------
+ Merge Right Join
+   Merge Cond: ((t1.c)::text = (t2.c)::text)
+   ->  Sort
+         Sort Key: t1.c
+         ->  Append
+               ->  Seq Scan on pt1_n t1
+               ->  Seq Scan on pt1_n_p1 t1_1
+               ->  Seq Scan on pt1_n_p2 t1_2
+   ->  Sort
+         Sort Key: t2.c
+         ->  Append
+               ->  Seq Scan on pt1 t2
+               ->  Seq Scan on pt1_p1 t2_1
+               ->  Seq Scan on pt1_p2 t2_2
+               ->  Seq Scan on pt1_p3 t2_3
+(15 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_n t1 FULL JOIN pt1 t2 ON (t1.c = t2.c);
+NOTICE:  join between relations (b 1) and (b 2) is NOT considered for partition-wise join.
+                 QUERY PLAN                  
+---------------------------------------------
+ Merge Full Join
+   Merge Cond: ((t1.c)::text = (t2.c)::text)
+   ->  Sort
+         Sort Key: t1.c
+         ->  Append
+               ->  Seq Scan on pt1_n t1
+               ->  Seq Scan on pt1_n_p1 t1_1
+               ->  Seq Scan on pt1_n_p2 t1_2
+   ->  Sort
+         Sort Key: t2.c
+         ->  Append
+               ->  Seq Scan on pt1 t2
+               ->  Seq Scan on pt1_p1 t2_1
+               ->  Seq Scan on pt1_p2 t2_2
+               ->  Seq Scan on pt1_p3 t2_3
+(15 rows)
+
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
new file mode 100644
index 0000000..128987b
--- /dev/null
+++ b/src/test/regress/sql/partition_join.sql
@@ -0,0 +1,408 @@
+--
+-- PARTITION_JOIN
+-- Test partition-wise join between partitioned tables
+--
+
+-- TODO: We need to add tests for SEMI-ANTI joins.
+-- TODO: tests with placeholder vars
+
+--
+-- partitioned by a single column
+--
+CREATE TABLE pt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE pt1_p1 PARTITION OF pt1 FOR VALUES START (0) END (250);
+CREATE TABLE pt1_p2 PARTITION OF pt1 FOR VALUES START (250) END (500);
+CREATE TABLE pt1_p3 PARTITION OF pt1 FOR VALUES START (500) END (600);
+INSERT INTO pt1 SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE upt1 AS SELECT * FROM pt1;
+
+CREATE TABLE pt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE pt2_p1 PARTITION OF pt2 FOR VALUES START (0) END (250);
+CREATE TABLE pt2_p2 PARTITION OF pt2 FOR VALUES START (250) END (500);
+CREATE TABLE pt2_p3 PARTITION OF pt2 FOR VALUES START (500) END (600);
+INSERT INTO pt2 SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE upt2 AS SELECT * FROM pt2;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1 t1, pt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1 t1, pt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM upt1 t1, upt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1 t1 LEFT JOIN pt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1 t1 LEFT JOIN pt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM upt1 t1 LEFT JOIN upt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1 t1 RIGHT JOIN pt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1 t1 RIGHT JOIN pt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM upt1 t1 RIGHT JOIN upt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM pt1 WHERE pt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM pt2 WHERE pt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM pt1 WHERE pt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM pt2 WHERE pt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM upt1 t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM upt2 t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+
+--
+-- partitioned by expression
+--
+CREATE TABLE pt1_e (a int, b int, c varchar) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE pt1_e_p1 PARTITION OF pt1_e FOR VALUES START (0) END (250);
+CREATE TABLE pt1_e_p2 PARTITION OF pt1_e FOR VALUES START (250) END (500);
+CREATE TABLE pt1_e_p3 PARTITION OF pt1_e FOR VALUES START (500) END (600);
+INSERT INTO pt1_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE upt1_e AS SELECT * FROM pt1_e;
+
+CREATE TABLE pt2_e (a int, b int, c varchar) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE pt2_e_p1 PARTITION OF pt2_e FOR VALUES START (0) END (250);
+CREATE TABLE pt2_e_p2 PARTITION OF pt2_e FOR VALUES START (250) END (500);
+CREATE TABLE pt2_e_p3 PARTITION OF pt2_e FOR VALUES START (500) END (600);
+INSERT INTO pt2_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE upt2_e AS SELECT * FROM pt2_e;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_e t1, pt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_e t1, pt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM upt1_e t1, upt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_e t1 LEFT JOIN pt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_e t1 LEFT JOIN pt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM upt1_e t1 LEFT JOIN upt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_e t1 RIGHT JOIN pt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_e t1 RIGHT JOIN pt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM upt1_e t1 RIGHT JOIN upt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM pt1_e WHERE pt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM pt2_e WHERE pt2_e.b % 25 = 0) t2 ON ((t1.a + t1.b)/2 = (t2.b + t2.a)/2) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM pt1_e WHERE pt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM pt2_e WHERE pt2_e.b % 25 = 0) t2 ON ((t1.a + t1.b)/2 = (t2.b + t2.a)/2) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM upt1_e t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM upt2_e t2 WHERE t2.b % 25 = 0) t2 ON ((t1.a + t1.b)/2 = (t2.b + t2.a)/2) ORDER BY t1.a, t2.b;
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM pt1 t1, pt2 t2, pt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM pt1 t1, pt2 t2, pt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM upt1 t1, upt2 t2, upt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (pt1 t1 LEFT JOIN pt2 t2 ON t1.a = t2.b) LEFT JOIN pt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (pt1 t1 LEFT JOIN pt2 t2 ON t1.a = t2.b) LEFT JOIN pt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (upt1 t1 LEFT JOIN upt2 t2 ON t1.a = t2.b) LEFT JOIN upt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (pt1 t1 LEFT JOIN pt2 t2 ON t1.a = t2.b) LEFT JOIN pt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (pt1 t1 LEFT JOIN pt2 t2 ON t1.a = t2.b) LEFT JOIN pt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (upt1 t1 LEFT JOIN upt2 t2 ON t1.a = t2.b) LEFT JOIN upt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (pt1 t1 LEFT JOIN pt2 t2 ON t1.a = t2.b) RIGHT JOIN pt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (pt1 t1 LEFT JOIN pt2 t2 ON t1.a = t2.b) RIGHT JOIN pt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (upt1 t1 LEFT JOIN upt2 t2 ON t1.a = t2.b) RIGHT JOIN upt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (pt1 t1 RIGHT JOIN pt2 t2 ON t1.a = t2.b) RIGHT JOIN pt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (pt1 t1 RIGHT JOIN pt2 t2 ON t1.a = t2.b) RIGHT JOIN pt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (upt1 t1 RIGHT JOIN upt2 t2 ON t1.a = t2.b) RIGHT JOIN upt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM pt1 WHERE pt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM pt2 WHERE pt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM pt1_e WHERE pt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM pt1 WHERE pt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM pt2 WHERE pt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM pt1_e WHERE pt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM upt1 WHERE upt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM upt2 WHERE upt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM upt1_e WHERE upt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+--
+-- partitioned by multiple columns
+--
+CREATE TABLE pt1_m (a int, b int, c varchar) PARTITION BY RANGE(a, ((a + b)/2));
+CREATE TABLE pt1_m_p1 PARTITION OF pt1_m FOR VALUES START (0, 0) END (250, 250);
+CREATE TABLE pt1_m_p2 PARTITION OF pt1_m FOR VALUES START (250, 250) END (500, 500);
+CREATE TABLE pt1_m_p3 PARTITION OF pt1_m FOR VALUES START (500, 500) END (600, 600);
+INSERT INTO pt1_m SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE upt1_m AS SELECT * FROM pt1_m;
+
+CREATE TABLE pt2_m (a int, b int, c varchar) PARTITION BY RANGE(((b + a)/2), b);
+CREATE TABLE pt2_m_p1 PARTITION OF pt2_m FOR VALUES START (0, 0) END (250, 250);
+CREATE TABLE pt2_m_p2 PARTITION OF pt2_m FOR VALUES START (250, 250) END (500, 500);
+CREATE TABLE pt2_m_p3 PARTITION OF pt2_m FOR VALUES START (500, 500) END (600, 600);
+INSERT INTO pt2_m SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE upt2_m AS SELECT * FROM pt2_m;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_m t1, pt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_m t1, pt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM upt1_m t1, upt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_m t1 LEFT JOIN pt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_m t1 LEFT JOIN pt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM upt1_m t1 LEFT JOIN upt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_m t1 RIGHT JOIN pt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_m t1 RIGHT JOIN pt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM upt1_m t1 RIGHT JOIN upt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM pt1_m WHERE pt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM pt2_m WHERE pt2_m.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM pt1_m WHERE pt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM pt2_m WHERE pt2_m.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM upt1_m t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM upt2_m t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+
+--
+-- multi-leveled partitions
+--
+CREATE TABLE pt1_l (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE pt1_l_p1 PARTITION OF pt1_l FOR VALUES START (0) END (250) PARTITION BY RANGE (b);
+CREATE TABLE pt1_l_p1_p1 PARTITION OF pt1_l_p1 FOR VALUES START (0) END (100);
+CREATE TABLE pt1_l_p1_p2 PARTITION OF pt1_l_p1 FOR VALUES START (100) END (250);
+CREATE TABLE pt1_l_p2 PARTITION OF pt1_l FOR VALUES START (250) END (500) PARTITION BY RANGE (c);
+CREATE TABLE pt1_l_p2_p1 PARTITION OF pt1_l_p2 FOR VALUES START ('0250') END ('0400');
+CREATE TABLE pt1_l_p2_p2 PARTITION OF pt1_l_p2 FOR VALUES START ('0400') END ('0500');
+CREATE TABLE pt1_l_p3 PARTITION OF pt1_l FOR VALUES START (500) END (600) PARTITION BY RANGE ((b + a));
+CREATE TABLE pt1_l_p3_p1 PARTITION OF pt1_l_p3 FOR VALUES START (1000) END (1100);
+CREATE TABLE pt1_l_p3_p2 PARTITION OF pt1_l_p3 FOR VALUES START (1100) END (1200);
+INSERT INTO pt1_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE upt1_l AS SELECT * FROM pt1_l;
+
+CREATE TABLE pt2_l (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE pt2_l_p1 PARTITION OF pt2_l FOR VALUES START (0) END (250) PARTITION BY RANGE (a);
+CREATE TABLE pt2_l_p1_p1 PARTITION OF pt2_l_p1 FOR VALUES START (0) END (100);
+CREATE TABLE pt2_l_p1_p2 PARTITION OF pt2_l_p1 FOR VALUES START (100) END (250);
+CREATE TABLE pt2_l_p2 PARTITION OF pt2_l FOR VALUES START (250) END (500) PARTITION BY RANGE (c);
+CREATE TABLE pt2_l_p2_p1 PARTITION OF pt2_l_p2 FOR VALUES START ('0250') END ('0400');
+CREATE TABLE pt2_l_p2_p2 PARTITION OF pt2_l_p2 FOR VALUES START ('0400') END ('0500');
+CREATE TABLE pt2_l_p3 PARTITION OF pt2_l FOR VALUES START (500) END (600) PARTITION BY RANGE ((a + b));
+CREATE TABLE pt2_l_p3_p1 PARTITION OF pt2_l_p3 FOR VALUES START (1000) END (1100);
+CREATE TABLE pt2_l_p3_p2 PARTITION OF pt2_l_p3 FOR VALUES START (1100) END (1200);
+INSERT INTO pt2_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE upt2_l AS SELECT * FROM pt2_l;
+
+--
+-- TODO: a varchar = varchar comparison is converted into a comparison
+-- varchar::text = varchar::text. Thus the partition key on c does not match
+-- the equi-join condition on c. This is fine, but can be improved. Also, as
+-- long as we have this behaviour, we get to test partial partition-wise join
+-- between multi-level partitioned relations.
+--
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_l t1, pt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_l t1, pt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM upt1_l t1, upt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+-- TODO for outer joins the equality conditions do not appear in joininfo of
+-- children, which means that the equi-join clauses are not present for
+-- subpartitions. Hence the joins between subpartitions are not currently being
+-- considered for partition-wise join.
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_l t1 LEFT JOIN pt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_l t1 LEFT JOIN pt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM upt1_l t1 LEFT JOIN upt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_l t1 RIGHT JOIN pt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_l t1 RIGHT JOIN pt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM upt1_l t1 RIGHT JOIN upt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM pt1_l WHERE pt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM pt2_l WHERE pt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM pt1_l WHERE pt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM pt2_l WHERE pt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM upt1_l t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM upt2_l t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+
+--
+-- 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;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1 AS SELECT * FROM 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;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt2 AS SELECT * FROM plt2;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.a = t2.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.a = t2.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1 t1, uplt2 t2 WHERE t1.c = t2.c AND t1.a = t2.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.a AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.a AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.a AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 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 plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 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 uplt1 t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) ORDER BY t1.a, t2.b;
+
+--
+-- 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;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1_e AS SELECT * FROM plt1_e;
+
+CREATE TABLE plt2_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
+CREATE TABLE plt2_e_p1 PARTITION OF plt2_e FOR VALUES IN ('0000', '0003', '0004', '0010');
+CREATE TABLE plt2_e_p2 PARTITION OF plt2_e FOR VALUES IN ('0001', '0005', '0002', '0009');
+CREATE TABLE plt2_e_p3 PARTITION OF plt2_e FOR VALUES IN ('0006', '0007', '0008', '0011');
+INSERT INTO plt2_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt2_e AS SELECT * FROM plt2_e;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1, plt2_e t2 WHERE t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1, plt2_e t2 WHERE t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_e t1, uplt2_e t2 WHERE t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 LEFT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 LEFT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_e t1 LEFT JOIN uplt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 RIGHT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 RIGHT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_e t1 RIGHT JOIN uplt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2_e WHERE plt2_e.b % 25 = 0) t2 ON (t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A')) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2_e WHERE plt2_e.b % 25 = 0) t2 ON (t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A')) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uplt1_e t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2_e t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A')) ORDER BY t1.a, t2.b;
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.a = t2.b AND t1.c = t2.c AND ltrim(t3.c, 'A') = t1.c AND t1.a = t3.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM plt1 t1, plt2 t2, plt1_e t3 WHERE t1.a = t2.b AND t1.c = t2.c AND t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM uplt1 t1, uplt2 t2, uplt1_e t3 WHERE t1.a = t2.b AND t1.c = t2.c AND t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+--
+-- negative testcases
+--
+
+CREATE TABLE pt1_n (a int, b int, c varchar) PARTITION BY RANGE(c);
+CREATE TABLE pt1_n_p1 PARTITION OF pt1_n FOR VALUES START ('0000') END ('0250');
+CREATE TABLE pt1_n_p2 PARTITION OF pt1_n FOR VALUES START ('0250') END ('0500');
+INSERT INTO pt1_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 499, 2) i;
+
+CREATE TABLE pt2_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE pt2_n_p1 PARTITION OF pt2_n FOR VALUES IN ('0000', '0003', '0004', '0010', '0006', '0007');
+CREATE TABLE pt2_n_p2 PARTITION OF pt2_n FOR VALUES IN ('0001', '0005', '0002', '0009', '0008', '0011');
+INSERT INTO pt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+
+CREATE TABLE pt3_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE pt3_n_p1 PARTITION OF pt3_n FOR VALUES IN ('0000', '0004', '0006', '0007');
+CREATE TABLE pt3_n_p2 PARTITION OF pt3_n FOR VALUES IN ('0001', '0002', '0008', '0010');
+CREATE TABLE pt3_n_p3 PARTITION OF pt3_n FOR VALUES IN ('0003', '0005', '0009', '0011');
+INSERT INTO pt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+
+CREATE TABLE pt4_n (a int, b int, c text) PARTITION BY RANGE(a);
+CREATE TABLE pt4_n_p1 PARTITION OF pt4_n FOR VALUES START (0) END (300);
+CREATE TABLE pt4_n_p2 PARTITION OF pt4_n FOR VALUES START (300) END (500);
+CREATE TABLE pt4_n_p3 PARTITION OF pt4_n FOR VALUES START (500) END (600);
+INSERT INTO pt4_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+
+-- partition-wise join can not be applied if the partition ranges differ
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1 t1, pt4_n t2 WHERE t1.a = t2.a;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1 t1 FULL JOIN pt4_n t2 ON t1.a = t2.a;
+-- 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 pt1 t1 LEFT JOIN pt2 t2 ON (t1.a < t2.b);
+-- equi-join with join condition on partial keys does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_m t1, pt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t1.a % 25 = 0;
+-- equi-join between out-of-order partition key columns does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_m t1 LEFT JOIN pt2_m t2 ON t1.a = t2.b WHERE t1.a % 25 = 0;
+-- equi-join between non-key columns does not qualify for partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_m t1 LEFT JOIN pt2_m t2 ON t1.c = t2.c WHERE t1.a % 25 = 0;
+-- TODO: an equalify condition on the partition keys in WHERE clause should not
+-- qualify for partition-wise join. But such a condition converts an OUTER
+-- join into an INNER join, in which case the conditions in WHERE clause can be
+-- treated as join conditions, thus qualifying the join for partition-wise
+-- join.
+
+-- partition-wise join can not be applied for a join between list and range
+-- partitioned table
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_n t1, pt2_n t2 WHERE t1.c = t2.c;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_n t1 LEFT JOIN pt2_n t2 ON (t1.c = t2.c);
+
+-- partition-wise join can not be applied between tables with different
+-- partition lists
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_n t1 RIGHT JOIN pt1 t2 ON (t1.c = t2.c);
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM pt1_n t1 FULL JOIN pt1 t2 ON (t1.c = t2.c);
#4Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Ashutosh Bapat (#3)
Re: Partition-wise join for join between (declaratively) partitioned tables

Sorry forgot to mention: this patch applies on top of the v7 patches posted
by Amit Langote on 27th June (
/messages/by-id/81371428-bb4b-1e33-5ad6-8c5c51b52cb7@lab.ntt.co.jp
).

On Tue, Jul 19, 2016 at 7:41 PM, Ashutosh Bapat <
ashutosh.bapat@enterprisedb.com> wrote:

On Fri, Jul 8, 2016 at 12:11 AM, Robert Haas <robertmhaas@gmail.com>
wrote:

I haven't reviewed this code yet due to being busy with 9.6, but I
think this is a very important query planner improvement with the
potential for big wins on queries involving large amounts of data.

Suppose we have a pair of equi-partitioned tables. Right now, if we
choose to perform a hash join, we'll have to build a giant hash table
with all of the rows from every inner partition and then probe it for
every row in every outer partition. If there are few enough inner
rows that the resultant hash table still fits in work_mem, this is
somewhat inefficient but not terrible - but if it causes us to have to
batch the hash join where we otherwise would not need to do so, then
it really sucks. Similarly, if we decide to merge-join each pair of
partitions, a partitionwise join may be able to use an internal sort
on some or all partitions whereas if we had to deal with all of the
data at the same time we'd need an external sort, possibly multi-pass.

Or we might be able to use indexes directly without need of a MergeAppend.

And if we choose a nested loop, say over an inner index-scan, we do
O(outer rows) index probes with this optimization but O(outer rows *
inner partitions) index probes without it.

In addition, parallel query can benefit significantly from this kind
of optimization. Tom recently raised the case of an appendrel where
every child has a parallel-safe path but not every child has a partial
path; currently, we can't go parallel in that case, but it's easy to
see that we could handle it by scheduling the appendrel's children
across a pool of workers. If we had this optimization, that sort of
thing would be much more likely to be useful, because it could create
appendrels where each member is an N-way join between equipartitioned
tables. That's particularly important right now because of the
restriction that a partial path must be driven by a Parallel SeqScan,
but even after that restriction is lifted it's easy to imagine that
the effective degree of parallelism for a single index scan may be
limited - so this kind of thing may significantly increase the number
of workers that a given query can use productively.

+1.

The attached patch implements the logic to assess whether two partitioned
tables can be joined using partition-wise join technique described in my
last
mail on this thread.

Two partitioned relations are considered for partition-wise join if
following
conditions are met (See build_joinrel_part_info() for details):
1. Both the partitions have same number of partitions, with same number of
partition keys and partitioned by same strategy - range or list.
2. They have matching datatypes for partition keys (partkey_types_match())
3. For list partitioned relations, they have same lists for each pair of
partitions, paired by position in which they appear.
4. For range partitioned relations, they have same bounds for each pair of
partitions, paired by their position when ordered in ascending fashion on
the
upper bounds.
5. There exists an equi-join condition for each pair of partition keys,
paired
by the position in which they appear.

Partition-wise join technique can be applied under more lenient
constraints [1]
e.g. joins between tables with different number of partitions but having
same
bounds/lists for the common partitions. I am planning to defer that to a
later
version of this feature.

A join executed using partition-wise join technique is itself a relation
partitioned by the similar partitioning scheme as the joining relations
with
the partition keys combined from the joining relations.

A PartitionOptInfo (uses name similar to RelOptInfo or IndexOptInfo)
structure
is used to store the partitioning information for a given base or relation.
In build_simple_rel(), we construct PartitionOptInfo structure for the
given
base relation by copying the relation's PartitionDesc and PartitionKey
(structures from Amit Langote's patch). While doing so, all the partition
keys
are stored as expressions. The structure also holds the RelOptInfos of the
partition relations. For a join relation, most of the PartitionOptInfo is
copied from either of the joining relations, except the partition keys and
RelOptInfo of partition relations. Partition keys of the join relations are
created by combing partition keys from both the joining relations. The
logic to
cosnstruct RelOptInfo for the partition-wise join relations is yet to be
implemented.

Since the logic to create the paths and RelOptInfos for partition-wise join
relations is not implemented yet, a query which can use partition-wise join
fails with error
"ERROR: the relation was considered for partition-wise join, which is not
supported right now.". It will also print messages to show which of the
joins
can and can not use partition-wise join technique e.g.
"NOTICE: join between relations (b 1) and (b 2) is considered for
partition-wise join." The relations are indicated by their relid in the
query.
OR
"NOTICE: join between relations (b 1) and (b 2) is NOT considered for
partition-wise join.".
These messages are for debugging only, and will be removed once path
creation
logic is implemented.

The patch adds a test partition_join.sql, which has a number of positive
and
negative testcases for joins between partitioned tables.

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

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

#5Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Ashutosh Bapat (#4)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

Hi All,

PFA the patch to support partition-wise joins for partitioned tables. The
patch
is based on the declarative parition support patches provided by Amit
Langote
on 26th August 2016. The previous patch added support to assess whether two
tables can be joined using partition-wise join technique, but did not have
complete support to create plans which used partition-wise technique. This
patch implements three important pieces for supporting partition-wise join

1. Logic to assess whether join between two partitioned tables can be
executed
using partition-wise join technique.
2. Construct RelOptInfo's representating join between matching partitions of
the joining relations and add join paths to those RelOptInfo's
3. Add append paths to the RelOptInfo representing the join between
partitioned
tables. Rest of the planner code chooses the optimal path for join.

make_join_rel() now calls try_partition_wise_join(), which executes all of
the
steps listed above. If the joining partitioned relations are deemed fit for
partition-wise join, we create one RelOptInfo (if not already present)
representing a join between every pair of partitions to be joined. Since the
join between parents is deemed legal, the join between the partitions is
also
legal, hence legality of the join is not checked again. RelOptInfo
representing
the join between partitions is constructed by translating the relevant
members
of RelOptInfo of the parent join relation. Similarly SpecialJoinInfo,
restrictlist (for given join order) are constructed by translating those for
the parent join.

make_join_rel() is split into two portions, a. that deals with constructing
restrictlist and RelOptInfo for join relation b. that creates paths for the
join. The second portion is separated into a function
populate_joinrel_with_paths(), which is reused in try_partition_wise_join()
to
create paths for join between matching partitions.

set_append_rel_pathlist() generates paths for child relations, marks the
empty
children as dummy relations and creates append paths by collecting paths
with
similar properties (parameterization and pathkeys) from non-empty children.
It
then adds append paths to the parent relation. This patch divides
set_append_rel_pathlist() into two parts a. marking empty child relations as
dummy and generating paths for non-empty children. b. collecting children
paths
into append paths for parent. Part b is separate into a function
add_paths_to_append_rel() which is reused for collecting paths from
partition-wise join child relations to construct append paths for join
between
partitioned tables.

For an N-way join between partitioned tables, make_join_rel() is called as
many
times as the number of valid join orders exist. For each such call, we will
add
paths to join between partitions for corresponding join order between those
partitions. We can generate the append paths for parent joinrel only after
all
such join orders have been considered. Hence before setting cheapest path
forx
parent join relation, we set the cheapest path for each join relation
between
partitions, followed by creating append paths for the parent joinrel. This
method needs some readjustment for multi-level partitions (TODO item 2
below).

A GUC enable_partition_wise_join is added to enable or disable
partition-wise
join technique. I think the GUC is useful similar to other join related GUCs
like enable_hashjoin.

parameterized paths: While creating parameterized paths for child relations
of
a partitioned tables, we do not have an idea as to whether we will be able
to
use partition-wise join technique or not. Also we do not know the child
partition of the other partitioned table, to which a given partition would
join. Hence we do not create paths parameterized by child partitions of
other
partitioned relations. But path for child of a partitioned relation
parameterized by other parent relation, can be considered to be
parameterised
by any child relation of the other partitioned relation by replacing the
parent
parameters by corresponding child parameters. This condition is used to
eliminate parameterized paths while creating merge and hash joins, to decide
the resultant parameterization of a join between child partitions and to
create
nested loop paths with inner path parameterized by outer relation where
inner
and outer relations are child partitions. While creating such nest loop join
paths we translate the path parameterized by other parent partitioned
relation,
to that parameterized by the required child.

Functions like select_outer_pathkeys_for_merge(), make_sort_from_pathkeys(),
find_ec_member_for_tle() which did not expect to be called for a child
relation, are now used for child partition relations for joins. These
functions
are adjusted for that usage.

Testing:
I have added partition_join.sql testcase to test partition-wise join
feature.
That file has extensive tests for list, range, multi-level partitioning
schemes
and various kinds of joins including nested loop join with inner relation
parameterized by outer relationThat file has extensive tests for list,
range,
multi-level partitioning schemes and various kinds of joins including nested
loop join with inner relation parameterized by outer relation.

make check passes clean.

TODOs:

1. Instead of storing partitioning information in RelOptInfo of each of the
partitioned relations (base and join relations), we can keep a list of
canonical partition schemes in PlannerInfo. Every RelOptInfo gets a pointer
to
the member of list representing the partitioning scheme of corresponding
relation. RelOptInfo's of all similarly partitioned relations get the same
pointer thus making it easy to match the partitioning schemes by comparing
the
pointers. While we are supporting only exact partition matching scheme now,
it's possible to extend this method to match compatible partitioning
schemes by
maintaining a list of compatible partitioning schemes.

Right now, I have moved some partition related structures from partition.c
to
partition.h. These structures are still being reviewed and might change when
Amit Langote improves his patches. Having canonical partitioning scheme in
PlannerInfo may not require moving those structures out. So, that code is
still
under development. A related change is renaming RangeBound structure in Amit
Langote's patches to PartitionRangeBound to avoid name conflict with
rangetypes.h. That change too should vanish once we decide where to keep
that
structure and its final name.

2. Multi-level partitioned tables: For some reason path created for joining
partitions are not being picked up as the cheapest paths. I think, we need
to
finalize the lower level paths before moving upwards in the partition
hierarchy. But I am yet to investigate the issue here.
RelOptInfo::parent_relid
should point to top parents rather than immediate parents.

3. Testing: need more tests for testing partition-wise join with foreign
tables
as partitions. More tests for parameterized joins for multi-level
partitioned
joins.

4. Remove bms_to_char(): I have added this function to print Relids in the
debugger. I have found it very useful to quickly examine Relids in debugger,
which otherwise wasn't so easy. If others find it useful too, I can create a
separate patch to be considered for a separate commit.

5. In add_paths_to_append_rel() to find the possible set of outer relations
for
generating parameterized paths for a given join. This code needs to be
adjusted
to eliminate the parent relations possible set of outer relations for a join
between child partitions.

6. Add support to reparameterize more types of paths for child relations. I
will add this once we finalize the method to reparameterize a parent path
for
child partition.

7. The patch adds make_joinrel() (name needs to be changed because of its
similariy with make_join_rel()) to construct an empty RelOptInfo for a join
between partitions. The function copies code doing the same from
build_join_rel(). build_join_rel() too can use this function, if we decide
to
retain it.

8. Few small TODOs related to code reorganization, proper function,
variable naming etc. are in the patch. pg_indent run.

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

Attachments:

pg_dp_join.patchtext/x-patch; charset=US-ASCII; name=pg_dp_join.patchDownload
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index f17ac29..8ce704a 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -43,80 +43,20 @@
 #include "utils/builtins.h"
 #include "utils/datum.h"
 #include "utils/memutils.h"
 #include "utils/fmgroids.h"
 #include "utils/inval.h"
 #include "utils/lsyscache.h"
 #include "utils/ruleutils.h"
 #include "utils/rel.h"
 #include "utils/syscache.h"
 
-/* Type and collation information for partition key columns */
-typedef struct KeyTypeCollInfo
-{
-	Oid		*typid;
-	int32	*typmod;
-	int16	*typlen;
-	bool	*typbyval;
-	char	*typalign;
-	Oid		*typcoll;
-} KeyTypeCollInfo;
-
-/*
- * Partition key information
- */
-typedef struct PartitionKeyData
-{
-	char		strategy;		/* partition strategy */
-	int16		partnatts;		/* number of partition attributes */
-	AttrNumber *partattrs;		/* partition attnums */
-	Oid		   *partopfamily;	/* OIDs of operator families */
-	Oid		   *partopcintype;	/* OIDs of opclass declared input data types */
-	FmgrInfo   *partsupfunc;	/* lookup info for support funcs */
-	List	   *partexprs;		/* partition key expressions, if any */
-	char	  **partcolnames;	/* partition key column names */
-	KeyTypeCollInfo *tcinfo;	/* type and collation info (all columns) */
-} PartitionKeyData;
-
-/* Internal representation of a list partition bound */
-typedef struct PartitionListInfo
-{
-	int		nvalues;	/* number of values in the following array */
-	Datum  *values;		/* values contained in the list */
-	bool   *nulls;
-} PartitionListInfo;
-
-/* Internal representation of a range partition bound */
-typedef struct RangeBound
-{
-	Datum	   *val;			/* composite bound value, if any */
-	bool		infinite;		/* bound is +/- infinity */
-	bool		inclusive;		/* bound is inclusive (vs exclusive) */
-	bool		lower;			/* this is the lower (vs upper) bound */
-} RangeBound;
-
-typedef struct PartitionRangeInfo
-{
-	RangeBound	*lower;
-	RangeBound	*upper;
-} PartitionRangeInfo;
-
-/*
- * Information about a single partition
- */
-typedef struct PartitionInfoData
-{
-	Oid						oid;		/* partition OID */
-	PartitionListInfo	   *list;		/* list partition info */
-	PartitionRangeInfo	   *range;		/* range partition info */
-} PartitionInfoData;
-
 /*
  * PartitionKeyExecInfo
  *
  *		This struct holds the information needed to extract partition
  *		column values from a heap tuple.
  *
  *		Key					copy of the rd_partkey of rel
  *		ExpressionState		exec state for expressions, or NIL if none
  */
 typedef struct PartitionKeyExecInfo
@@ -226,37 +166,37 @@ static PartitionListInfo *make_list_from_spec(PartitionKey key,
 static PartitionListInfo *copy_list_info(PartitionListInfo *src,
 							PartitionKey key);
 static bool equal_list_info(PartitionKey key, PartitionListInfo *l1,
 				PartitionListInfo *l2);
 static bool partition_list_values_equal(PartitionKey key,
 						   Datum val1, Datum val2);
 
 /* Range partition related support functions */
 static PartitionRangeInfo *make_range_from_spec(PartitionKey key,
 							PartitionRangeSpec *range_spec);
-static RangeBound *make_range_bound(PartitionKey key, List *val, bool inclusive,
+static PartitionRangeBound *make_range_bound(PartitionKey key, List *val, bool inclusive,
 							bool lower);
 static PartitionRangeInfo *copy_range_info(PartitionRangeInfo *src,
 							PartitionKey key);
-static RangeBound *copy_range_bound(RangeBound *src, PartitionKey key);
+static PartitionRangeBound *copy_range_bound(PartitionRangeBound *src, PartitionKey key);
 static bool equal_range_info(PartitionKey key, PartitionRangeInfo *r1,
 				 PartitionRangeInfo *r2);
 static int32 partition_range_cmp(PartitionKey key, PartitionRangeInfo *r1,
 									  PartitionRangeInfo *r2);
-static int32 partition_range_bound_cmp(PartitionKey key, RangeBound *b1,
-							RangeBound *b2);
+static int32 partition_range_bound_cmp(PartitionKey key, PartitionRangeBound *b1,
+							PartitionRangeBound *b2);
 static int32 partition_range_tuple_cmp(PartitionKey key,
 						   Datum *val1, Datum *val2);
 static bool partition_range_overlaps(PartitionKey key,
 							PartitionRangeInfo *r1, PartitionRangeInfo *r2);
-static bool tuple_rightof_bound(PartitionKey key, Datum *tuple, RangeBound *bound);
-static bool tuple_leftof_bound(PartitionKey key, Datum *tuple, RangeBound *bound);
+static bool tuple_rightof_bound(PartitionKey key, Datum *tuple, PartitionRangeBound *bound);
+static bool tuple_leftof_bound(PartitionKey key, Datum *tuple, PartitionRangeBound *bound);
 static int range_partition_bsearch(PartitionKey key, PartitionDesc pdesc,
 						Datum *tuple);
 
 /*
  * Partition key related functions
  */
 
 /*
  * RelationBuildPartitionKey
  *		Build and attach to relcache partition key data of relation
@@ -1097,21 +1037,21 @@ list_overlaps_existing_partition(PartitionKey key,
 }
 
 
 /*
  * Is a new partition's range empty?
  */
 static bool
 partition_range_empty(PartitionKey key, PartitionRangeSpec *range_spec)
 {
 	PartitionRangeInfo *range;
-	RangeBound *lower,
+	PartitionRangeBound *lower,
 			   *upper;
 
 	range = make_range_from_spec(key, range_spec);
 	lower = range->lower;
 	upper = range->upper;
 
 	/*
 	 * Range is not empty if one (and only one) of the bounds is infinity.
 	 * Both cannot be infinite because of how the syntax is specified.
 	 */
@@ -2104,27 +2044,27 @@ make_range_from_spec(PartitionKey key, PartitionRangeSpec *range_spec)
 									range_spec->lowerinc,
 									true);
 	range->upper = make_range_bound(key,
 									range_spec->upper,
 									range_spec->upperinc,
 									false);
 
 	return range;
 }
 
-static RangeBound *
+static PartitionRangeBound *
 make_range_bound(PartitionKey key, List *val, bool inclusive, bool lower)
 {
-	RangeBound *bound;
+	PartitionRangeBound *bound;
 	ListCell *cell;
 
-	bound = (RangeBound *) palloc0(sizeof(RangeBound));
+	bound = (PartitionRangeBound *) palloc0(sizeof(PartitionRangeBound));
 	bound->infinite = (val == NIL);
 	bound->inclusive = inclusive;
 	bound->lower = lower;
 
 	if (val)
 	{
 		int		i;
 
 		bound->val = (Datum *) palloc0(key->partnatts * sizeof(Datum));
 
@@ -2157,28 +2097,28 @@ copy_range_info(PartitionRangeInfo *src, PartitionKey key)
 {
 	PartitionRangeInfo *result;
 
 	result = (PartitionRangeInfo *) palloc0(sizeof(PartitionRangeInfo));
 	result->lower = copy_range_bound(src->lower, key);
 	result->upper = copy_range_bound(src->upper, key);
 
 	return result;
 }
 
-static RangeBound *
-copy_range_bound(RangeBound *src, PartitionKey key)
+static PartitionRangeBound *
+copy_range_bound(PartitionRangeBound *src, PartitionKey key)
 {
 	int		i;
 	int		partnatts = get_partition_key_natts(key);
-	RangeBound  *result;
+	PartitionRangeBound  *result;
 
-	result = (RangeBound *) palloc0(sizeof(RangeBound));
+	result = (PartitionRangeBound *) palloc0(sizeof(PartitionRangeBound));
 	result->infinite = src->infinite;
 	result->inclusive = src->inclusive;
 	result->lower = src->lower;
 
 	if (src->val)
 	{
 		result->val = (Datum *) palloc0(partnatts * sizeof(Datum));
 		for (i = 0; i < partnatts; i++)
 			result->val[i] = datumCopy(src->val[i],
 									   key->tcinfo->typbyval[i],
@@ -2212,21 +2152,21 @@ partition_range_cmp(PartitionKey key, PartitionRangeInfo *r1,
 
 	cmp = partition_range_bound_cmp(key, r1->lower, r2->lower);
 	if (cmp == 0)
 		cmp = partition_range_bound_cmp(key, r1->upper, r2->upper);
 
 	return cmp;
 }
 
 /* Returns for two range partition bounds whether, b1 <=, =, >= b2 */
 static int32
-partition_range_bound_cmp(PartitionKey key, RangeBound *b1, RangeBound *b2)
+partition_range_bound_cmp(PartitionKey key, PartitionRangeBound *b1, PartitionRangeBound *b2)
 {
 	int32		result;
 
 	/*
 	 * First, handle cases involving infinity, which don't require invoking
 	 * the comparison proc.
 	 */
 	if (b1->infinite && b2->infinite)
 	{
 		/*
@@ -2337,31 +2277,31 @@ range_partition_bsearch(PartitionKey key, PartitionDesc pdesc,
 		}
 
 		low = idx + 1;
 	}
 
 	return -1;
 }
 
 /* Does range key lie to the right of partition bound */
 static bool
-tuple_rightof_bound(PartitionKey key, Datum *tuple, RangeBound *bound)
+tuple_rightof_bound(PartitionKey key, Datum *tuple, PartitionRangeBound *bound)
 {
 	int32	cmpval = partition_range_tuple_cmp(key, tuple, bound->val);
 
 	if (!cmpval)
 		return bound->lower ? bound->inclusive : !bound->inclusive;
 
 	return cmpval > 0;
 }
 
 /* Does range key lie to the left of partition bound */
 static bool
-tuple_leftof_bound(PartitionKey key, Datum *tuple, RangeBound *bound)
+tuple_leftof_bound(PartitionKey key, Datum *tuple, PartitionRangeBound *bound)
 {
 	int32	cmpval = partition_range_tuple_cmp(key, tuple, bound->val);
 
 	if (!cmpval)
 		return !bound->lower ? bound->inclusive : !bound->inclusive;
 
 	return cmpval < 0;
 }
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 3b8fa6b..cd14840 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -204,20 +204,31 @@ _outBitmapset(StringInfo str, const Bitmapset *bms)
 }
 
 /* for use by extensions which define extensible nodes */
 void
 outBitmapset(StringInfo str, const Bitmapset *bms)
 {
 	_outBitmapset(str, bms);
 }
 
 /*
+ * TODO: remove, used for debugging through gdb.
+ */
+char *
+bms_to_char(const Bitmapset *bms)
+{
+	StringInfo str = makeStringInfo();
+	outBitmapset(str, bms);
+	return str->data;
+}
+
+/*
  * Print the value of a Datum given its type.
  */
 void
 outDatum(StringInfo str, Datum value, int typlen, bool typbyval)
 {
 	Size		length,
 				i;
 	char	   *s;
 
 	length = datumGetSize(value, typbyval, typlen);
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 04264b4..09781be 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -119,20 +119,24 @@ static void check_output_expressions(Query *subquery,
 static void compare_tlist_datatypes(List *tlist, List *colTypes,
 						pushdown_safety_info *safetyInfo);
 static bool targetIsInAllPartitionLists(TargetEntry *tle, Query *query);
 static bool qual_is_pushdown_safe(Query *subquery, Index rti, Node *qual,
 					  pushdown_safety_info *safetyInfo);
 static void subquery_push_qual(Query *subquery,
 				   RangeTblEntry *rte, Index rti, Node *qual);
 static void recurse_push_qual(Node *setOp, Query *topquery,
 				  RangeTblEntry *rte, Index rti, Node *qual);
 static void remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel);
+static void add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
+									List *live_childrels);
+static void generate_partition_wise_join_paths(PlannerInfo *root,
+											   RelOptInfo *rel);
 
 
 /*
  * make_one_rel
  *	  Finds all possible access paths for executing a query, returning a
  *	  single rel that represents the join of all base rels in the query.
  */
 RelOptInfo *
 make_one_rel(PlannerInfo *root, List *joinlist)
 {
@@ -908,20 +912,40 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		childRTE = root->simple_rte_array[childRTindex];
 
 		/*
 		 * The child rel's RelOptInfo was already created during
 		 * add_base_rels_to_query.
 		 */
 		childrel = find_base_rel(root, childRTindex);
 		Assert(childrel->reloptkind == RELOPT_OTHER_MEMBER_REL);
 
 		/*
+		 * Copy/Modify targetlist. Partition-wise join technique may consider
+		 * an OUTER join of another child relation with this child relation.
+		 * In that case, even if this child is deemed empty, we will require
+		 * the targetlist of this child to construct the nullable side. Hence
+		 * set the targetlist before we prove that the child is empty and stop
+		 * processing further.
+		 *
+		 * NB: the resulting childrel->reltarget->exprs may contain arbitrary
+		 * expressions, which otherwise would not occur in a rel's targetlist.
+		 * Code that might be looking at an appendrel child must cope with
+		 * such.  (Normally, a rel's targetlist would only include Vars and
+		 * PlaceHolderVars.)  XXX we do not bother to update the cost or width
+		 * fields of childrel->reltarget; not clear if that would be useful.
+		 */
+		childrel->reltarget->exprs = (List *)
+			adjust_appendrel_attrs(root,
+								   (Node *) rel->reltarget->exprs,
+								   appinfo);
+
+		/*
 		 * We have to copy the parent's targetlist and quals to the child,
 		 * with appropriate substitution of variables.  However, only the
 		 * baserestrictinfo quals are needed before we can check for
 		 * constraint exclusion; so do that first and then check to see if we
 		 * can disregard this child.
 		 *
 		 * As of 8.4, the child rel's targetlist might contain non-Var
 		 * expressions, which means that substitution into the quals could
 		 * produce opportunities for const-simplification, and perhaps even
 		 * pseudoconstant quals.  To deal with this, we strip the RestrictInfo
@@ -953,38 +977,25 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		if (relation_excluded_by_constraints(root, childrel, childRTE))
 		{
 			/*
 			 * This child need not be scanned, so we can omit it from the
 			 * appendrel.
 			 */
 			set_dummy_rel_pathlist(childrel);
 			continue;
 		}
 
-		/*
-		 * CE failed, so finish copying/modifying targetlist and join quals.
-		 *
-		 * NB: the resulting childrel->reltarget->exprs may contain arbitrary
-		 * expressions, which otherwise would not occur in a rel's targetlist.
-		 * Code that might be looking at an appendrel child must cope with
-		 * such.  (Normally, a rel's targetlist would only include Vars and
-		 * PlaceHolderVars.)  XXX we do not bother to update the cost or width
-		 * fields of childrel->reltarget; not clear if that would be useful.
-		 */
+		/* CE failed, so finish copying/modifying join quals. */
 		childrel->joininfo = (List *)
 			adjust_appendrel_attrs(root,
 								   (Node *) rel->joininfo,
 								   appinfo);
-		childrel->reltarget->exprs = (List *)
-			adjust_appendrel_attrs(root,
-								   (Node *) rel->reltarget->exprs,
-								   appinfo);
 
 		/*
 		 * We have to make child entries in the EquivalenceClass data
 		 * structures as well.  This is needed either if the parent
 		 * participates in some eclass joins (because we will want to consider
 		 * inner-indexscan joins on the individual children) or if the parent
 		 * has useful pathkeys (because we should try to build MergeAppend
 		 * paths that produce those sort orderings).
 		 */
 		if (rel->has_eclass_joins || has_useful_pathkeys(root, rel))
@@ -1115,41 +1126,32 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 /*
  * set_append_rel_pathlist
  *	  Build access paths for an "append relation"
  */
 static void
 set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 						Index rti, RangeTblEntry *rte)
 {
 	int			parentRTindex = rti;
 	List	   *live_childrels = NIL;
-	List	   *subpaths = NIL;
-	bool		subpaths_valid = true;
-	List	   *partial_subpaths = NIL;
-	bool		partial_subpaths_valid = true;
-	List	   *all_child_pathkeys = NIL;
-	List	   *all_child_outers = NIL;
 	ListCell   *l;
 
 	/*
-	 * Generate access paths for each member relation, and remember the
-	 * cheapest path for each one.  Also, identify all pathkeys (orderings)
-	 * and parameterizations (required_outer sets) available for the member
-	 * relations.
+	 * Generate access paths for each member relation and remember the
+	 * non-dummy children.
 	 */
 	foreach(l, root->append_rel_list)
 	{
 		AppendRelInfo *appinfo = (AppendRelInfo *) lfirst(l);
 		int			childRTindex;
 		RangeTblEntry *childRTE;
 		RelOptInfo *childrel;
-		ListCell   *lcp;
 
 		/* append_rel_list contains all append rels; ignore others */
 		if (appinfo->parent_relid != parentRTindex)
 			continue;
 
 		/* Re-locate the child RTE and RelOptInfo */
 		childRTindex = appinfo->child_relid;
 		childRTE = root->simple_rte_array[childRTindex];
 		childrel = root->simple_rel_array[childRTindex];
 
@@ -1170,20 +1172,46 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 		/*
 		 * If child is dummy, ignore it.
 		 */
 		if (IS_DUMMY_REL(childrel))
 			continue;
 
 		/*
 		 * Child is live, so add it to the live_childrels list for use below.
 		 */
 		live_childrels = lappend(live_childrels, childrel);
+	}
+
+	add_paths_to_append_rel(root, rel, live_childrels);
+}
+
+static void
+add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
+						List *live_childrels)
+{
+	List	   *subpaths = NIL;
+	bool		subpaths_valid = true;
+	List	   *partial_subpaths = NIL;
+	bool		partial_subpaths_valid = true;
+	List	   *all_child_pathkeys = NIL;
+	List	   *all_child_outers = NIL;
+	ListCell   *l;
+
+	/*
+	 * For every non-dummy child, remember the cheapest path.  Also, identify
+	 * all pathkeys (orderings) and parameterizations (required_outer sets)
+	 * available for the non-dummy member relations.
+	 */
+	foreach (l, live_childrels)
+	{
+		RelOptInfo *childrel = lfirst(l);
+		ListCell   *lcp;
 
 		/*
 		 * If child has an unparameterized cheapest-total path, add that to
 		 * the unparameterized Append path we are constructing for the parent.
 		 * If not, there's no workable unparameterized path.
 		 */
 		if (childrel->cheapest_total_path->param_info == NULL)
 			subpaths = accumulate_append_subpath(subpaths,
 											  childrel->cheapest_total_path);
 		else
@@ -2188,20 +2216,22 @@ standard_join_search(PlannerInfo *root, int levels_needed, List *initial_rels)
 		 * Run generate_gather_paths() for each just-processed joinrel.  We
 		 * could not do this earlier because both regular and partial paths
 		 * can get added to a particular joinrel at multiple times within
 		 * join_search_one_level.  After that, we're done creating paths for
 		 * the joinrel, so run set_cheapest().
 		 */
 		foreach(lc, root->join_rel_level[lev])
 		{
 			rel = (RelOptInfo *) lfirst(lc);
 
+			generate_partition_wise_join_paths(root, rel);
+
 			/* Create GatherPaths for any useful partial paths for rel */
 			generate_gather_paths(root, rel);
 
 			/* Find and save the cheapest paths for this rel */
 			set_cheapest(rel);
 
 #ifdef OPTIMIZER_DEBUG
 			debug_print_rel(root, rel);
 #endif
 		}
@@ -2849,20 +2879,77 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
 		 * OK, we don't need it.  Replace the expression with a NULL constant.
 		 * Preserve the exposed type of the expression, in case something
 		 * looks at the rowtype of the subquery's result.
 		 */
 		tle->expr = (Expr *) makeNullConst(exprType(texpr),
 										   exprTypmod(texpr),
 										   exprCollation(texpr));
 	}
 }
 
+/*
+ * generate_partition_wise_join_paths
+ * 		Create appends paths containing partition-wise join paths for given
+ * 		join relation.
+ */
+static void
+generate_partition_wise_join_paths(PlannerInfo *root, RelOptInfo *rel)
+{
+	List	   *live_children = NIL;
+	int			cnt_part;
+	PartitionOptInfo *part_info = rel->part_info;
+
+	/* Handle only join relations. */
+	if (rel->reloptkind != RELOPT_JOINREL &&
+		rel->reloptkind != RELOPT_OTHER_JOINREL)
+		return;
+
+	/* If the relation is not partitioned, nothing to do. */
+	if (!part_info)
+		return;
+
+	for (cnt_part = 0; cnt_part < part_info->nparts; cnt_part++)
+	{
+		RelOptInfo	*child_rel = part_info->part_rels[cnt_part];
+
+		/* Ignore dummy child. */
+		if (!IS_DUMMY_REL(child_rel))
+		{
+			/* Collect the paths from child joinrel. */
+			/* TODO: we need to reassess how do we do this, since the join
+			 * between parents will require append paths for the child joinrel.
+			 */
+			generate_partition_wise_join_paths(root, child_rel);
+
+			/* Find the cheapest of the paths for this rel. */
+			set_cheapest(child_rel);
+
+			/* Should we generate any gather paths? */
+#ifdef OPTIMIZER_DEBUG
+			debug_print_rel(root, rel);
+#endif
+
+			live_children = lappend(live_children, child_rel);
+		}
+	}
+
+	/*
+	 * Create append paths by collecting sub paths from live children. Even if
+	 * there are no live children, we should create an append path with no
+	 * subpaths i.e. a dummy access path.
+	 */
+	add_paths_to_append_rel(root, rel, live_children);
+
+	if (live_children)
+		pfree(live_children);
+}
+
 /*****************************************************************************
  *			DEBUG SUPPORT
  *****************************************************************************/
 
 #ifdef OPTIMIZER_DEBUG
 
 static void
 print_relids(PlannerInfo *root, Relids relids)
 {
 	int			x;
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 2a49639..a23da1c 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -119,20 +119,21 @@ bool		enable_seqscan = true;
 bool		enable_indexscan = true;
 bool		enable_indexonlyscan = true;
 bool		enable_bitmapscan = true;
 bool		enable_tidscan = true;
 bool		enable_sort = true;
 bool		enable_hashagg = true;
 bool		enable_nestloop = true;
 bool		enable_material = true;
 bool		enable_mergejoin = true;
 bool		enable_hashjoin = true;
+bool		enable_partition_wise_join = true;
 
 typedef struct
 {
 	PlannerInfo *root;
 	QualCost	total;
 } cost_qual_eval_context;
 
 static List *extract_nonindex_conditions(List *qual_clauses, List *indexquals);
 static MergeScanSelCache *cached_scansel(PlannerInfo *root,
 			   RestrictInfo *rinfo,
diff --git a/src/backend/optimizer/path/equivclass.c b/src/backend/optimizer/path/equivclass.c
index 0e50ad5..62a3f3a 100644
--- a/src/backend/optimizer/path/equivclass.c
+++ b/src/backend/optimizer/path/equivclass.c
@@ -2359,20 +2359,22 @@ eclass_useful_for_merging(PlannerInfo *root,
 
 	/*
 	 * Note we don't test ec_broken; if we did, we'd need a separate code path
 	 * to look through ec_sources.  Checking the members anyway is OK as a
 	 * possibly-overoptimistic heuristic.
 	 */
 
 	/* If specified rel is a child, we must consider the topmost parent rel */
 	if (rel->reloptkind == RELOPT_OTHER_MEMBER_REL)
 		relids = find_childrel_top_parent(root, rel)->relids;
+	else if (rel->reloptkind == RELOPT_OTHER_JOINREL)
+		relids = rel->parent_relids;
 	else
 		relids = rel->relids;
 
 	/* If rel already includes all members of eclass, no point in searching */
 	if (bms_is_subset(eclass->ec_relids, relids))
 		return false;
 
 	/* To join, we need a member not in the given rel */
 	foreach(lc, eclass->ec_members)
 	{
diff --git a/src/backend/optimizer/path/joinpath.c b/src/backend/optimizer/path/joinpath.c
index cc7384f..8b34ab9 100644
--- a/src/backend/optimizer/path/joinpath.c
+++ b/src/backend/optimizer/path/joinpath.c
@@ -18,23 +18,29 @@
 
 #include "executor/executor.h"
 #include "foreign/fdwapi.h"
 #include "optimizer/cost.h"
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
 
 /* Hook for plugins to get control in add_paths_to_joinrel() */
 set_join_pathlist_hook_type set_join_pathlist_hook = NULL;
 
-#define PATH_PARAM_BY_REL(path, rel)  \
+#define PATH_PARAM_BY_PARENT(path, rel)	\
+	((path)->param_info && bms_overlap(PATH_REQ_OUTER(path),	\
+									   (rel)->parent_relids))
+#define PATH_PARAM_BY_REL_SELF(path, rel)  \
 	((path)->param_info && bms_overlap(PATH_REQ_OUTER(path), (rel)->relids))
 
+#define PATH_PARAM_BY_REL(path, rel)	\
+	(PATH_PARAM_BY_REL_SELF(path, rel) || PATH_PARAM_BY_PARENT(path, rel))
+
 static void sort_inner_and_outer(PlannerInfo *root, RelOptInfo *joinrel,
 					 RelOptInfo *outerrel, RelOptInfo *innerrel,
 					 JoinType jointype, JoinPathExtraData *extra);
 static void match_unsorted_outer(PlannerInfo *root, RelOptInfo *joinrel,
 					 RelOptInfo *outerrel, RelOptInfo *innerrel,
 					 JoinType jointype, JoinPathExtraData *extra);
 static void consider_parallel_nestloop(PlannerInfo *root,
 						   RelOptInfo *joinrel,
 						   RelOptInfo *outerrel,
 						   RelOptInfo *innerrel,
@@ -121,20 +127,22 @@ add_paths_to_joinrel(PlannerInfo *root,
 	 * Decide whether it's sensible to generate parameterized paths for this
 	 * joinrel, and if so, which relations such paths should require.  There
 	 * is usually no need to create a parameterized result path unless there
 	 * is a join order restriction that prevents joining one of our input rels
 	 * directly to the parameter source rel instead of joining to the other
 	 * input rel.  (But see allow_star_schema_join().)	This restriction
 	 * reduces the number of parameterized paths we have to deal with at
 	 * higher join levels, without compromising the quality of the resulting
 	 * plan.  We express the restriction as a Relids set that must overlap the
 	 * parameterization of any proposed join path.
+	 * TODO: do we need sjinfo created for children to added to the list in the
+	 * planner info? OR how do we handle the code below?
 	 */
 	foreach(lc, root->join_info_list)
 	{
 		SpecialJoinInfo *sjinfo = (SpecialJoinInfo *) lfirst(lc);
 
 		/*
 		 * SJ is relevant to this join if we have some part of its RHS
 		 * (possibly not all of it), and haven't yet joined to its LHS.  (This
 		 * test is pretty simplistic, but should be sufficient considering the
 		 * join has already been proven legal.)  If the SJ is relevant, it
@@ -272,20 +280,35 @@ try_nestloop_path(PlannerInfo *root,
 				  Path *outer_path,
 				  Path *inner_path,
 				  List *pathkeys,
 				  JoinType jointype,
 				  JoinPathExtraData *extra)
 {
 	Relids		required_outer;
 	JoinCostWorkspace workspace;
 
 	/*
+	 * An inner path parameterized by the parent relation of outer
+	 * relation needs to be reparameterized by the outer relation to be used
+	 * for parameterized nested loop join.
+	 */
+	if (PATH_PARAM_BY_PARENT(inner_path, outer_path->parent))
+	{
+		inner_path = reparameterize_path_for_child(root, inner_path,
+												   outer_path->parent);
+
+		/* If we could not translate the path, don't produce nest loop path. */
+		if (!inner_path)
+			return;
+	}
+
+	/*
 	 * Check to see if proposed path is still parameterized, and reject if the
 	 * parameterization wouldn't be sensible --- unless allow_star_schema_join
 	 * says to allow it anyway.  Also, we must reject if have_dangerous_phv
 	 * doesn't like the look of it, which could only happen if the nestloop is
 	 * still parameterized.
 	 */
 	required_outer = calc_nestloop_required_outer(outer_path,
 												  inner_path);
 	if (required_outer &&
 		((!bms_overlap(required_outer, extra->param_source_rels) &&
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 01d4fea..6378d2e 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -7,38 +7,71 @@
  * Portions Copyright (c) 1994, Regents of the University of California
  *
  *
  * IDENTIFICATION
  *	  src/backend/optimizer/path/joinrels.c
  *
  *-------------------------------------------------------------------------
  */
 #include "postgres.h"
 
+#include "optimizer/clauses.h"
 #include "optimizer/joininfo.h"
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
+#include "optimizer/prep.h"
+#include "optimizer/cost.h"
 #include "utils/memutils.h"
 
 
 static void make_rels_by_clause_joins(PlannerInfo *root,
 						  RelOptInfo *old_rel,
 						  ListCell *other_rels);
 static void make_rels_by_clauseless_joins(PlannerInfo *root,
 							  RelOptInfo *old_rel,
 							  ListCell *other_rels);
 static bool has_join_restriction(PlannerInfo *root, RelOptInfo *rel);
 static bool has_legal_joinclause(PlannerInfo *root, RelOptInfo *rel);
 static bool is_dummy_rel(RelOptInfo *rel);
 static void mark_dummy_rel(RelOptInfo *rel);
 static bool restriction_is_constant_false(List *restrictlist,
 							  bool only_pushed_down);
+static void populate_joinrel_with_paths(PlannerInfo *root, RelOptInfo *rel1,
+							RelOptInfo *rel2, RelOptInfo *joinrel,
+							SpecialJoinInfo *sjinfo, List *restrictlist);
+static void try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1,
+						  RelOptInfo *rel2, RelOptInfo *joinrel,
+						  SpecialJoinInfo *parent_sjinfo,
+						  List *parent_restrictlist);
+static SpecialJoinInfo * build_partition_join_sjinfo(PlannerInfo *root,
+										 SpecialJoinInfo *parent_sjinfo,
+										 List *append_rel_infos1,
+										 List *append_rel_infos2);
+static PartitionOptInfo *build_joinrel_part_info(RelOptInfo *rel1,
+							   RelOptInfo *rel2, JoinType jointype,
+							   List *restrictlist);
+static bool have_same_part_info(PartitionOptInfo *part_info1,
+								PartitionOptInfo *part_info2);
+static bool have_partkey_equi_join(RelOptInfo *rel1, RelOptInfo *rel2,
+						   JoinType jointype, List *restrictlist);
+static int match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel);
+static bool partkey_types_match(PartitionOptInfo *part_info1,
+					PartitionOptInfo *part_info2);
+static bool have_same_partition_lists(PartitionOptInfo *part_info1,
+					  PartitionOptInfo *part_info2);
+static bool have_same_partition_bounds(PartitionOptInfo *part_info1,
+					  PartitionOptInfo *part_info2);
+static bool compare_partition_bounds(PartitionOptInfo *part_info,
+					PartitionRangeBound *bound1, PartitionRangeBound *bound2);
+static bool match_joinrel_part_info(RelOptInfo *joinrel, RelOptInfo *rel1,
+						  RelOptInfo *rel2, JoinType jointype,
+						  List *restrictlist);
 
 
 /*
  * join_search_one_level
  *	  Consider ways to produce join relations containing exactly 'level'
  *	  jointree items.  (This is one step of the dynamic-programming method
  *	  embodied in standard_join_search.)  Join rel nodes for each feasible
  *	  combination of lower-level rels are created and returned in a list.
  *	  Implementation paths are created for each such joinrel, too.
  *
@@ -717,20 +750,44 @@ make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 	/*
 	 * If we've already proven this join is empty, we needn't consider any
 	 * more paths for it.
 	 */
 	if (is_dummy_rel(joinrel))
 	{
 		bms_free(joinrelids);
 		return joinrel;
 	}
 
+	/* Add paths to the join relation. */
+	populate_joinrel_with_paths(root, rel1, rel2, joinrel, sjinfo,
+								restrictlist);
+
+	/* Apply partition-wise join technique, if possible. */
+	try_partition_wise_join(root, rel1, rel2, joinrel, sjinfo, restrictlist);
+
+	bms_free(joinrelids);
+
+	return joinrel;
+}
+
+/*
+ * populate_joinrel_with_paths
+ *	  Create paths to join given input relation and add those to the given
+ *	  joinrel. The SpecialJoinInfo provides details about the join and the
+ *	  restrictlist contains the join clauses and the other clauses applicable
+ *	  for given pair of the joining relations.
+ */
+static void
+populate_joinrel_with_paths(PlannerInfo *root, RelOptInfo *rel1,
+							RelOptInfo *rel2, RelOptInfo *joinrel,
+							SpecialJoinInfo *sjinfo, List *restrictlist)
+{
 	/*
 	 * Consider paths using each rel as both outer and inner.  Depending on
 	 * the join type, a provably empty outer or inner rel might mean the join
 	 * is provably empty too; in which case throw away any previously computed
 	 * paths and mark the join as dummy.  (We do it this way since it's
 	 * conceivable that dummy-ness of a multi-element join might only be
 	 * noticeable for certain construction paths.)
 	 *
 	 * Also, a provably constant-false join restriction typically means that
 	 * we can skip evaluating one or both sides of the join.  We do this by
@@ -861,27 +918,22 @@ make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 				mark_dummy_rel(rel2);
 			add_paths_to_joinrel(root, joinrel, rel1, rel2,
 								 JOIN_ANTI, sjinfo,
 								 restrictlist);
 			break;
 		default:
 			/* other values not expected here */
 			elog(ERROR, "unrecognized join type: %d", (int) sjinfo->jointype);
 			break;
 	}
-
-	bms_free(joinrelids);
-
-	return joinrel;
 }
 
-
 /*
  * have_join_order_restriction
  *		Detect whether the two relations should be joined to satisfy
  *		a join-order restriction arising from special or lateral joins.
  *
  * In practice this is always used with have_relevant_joinclause(), and so
  * could be merged with that function, but it seems clearer to separate the
  * two concerns.  We need this test because there are degenerate cases where
  * a clauseless join must be performed to satisfy join-order restrictions.
  * Also, if one rel has a lateral reference to the other, or both are needed
@@ -1242,10 +1294,744 @@ restriction_is_constant_false(List *restrictlist, bool only_pushed_down)
 
 			/* constant NULL is as good as constant FALSE for our purposes */
 			if (con->constisnull)
 				return true;
 			if (!DatumGetBool(con->constvalue))
 				return true;
 		}
 	}
 	return false;
 }
+
+/*
+ * If the join between the given two relations can be executed as
+ * partition-wise join create the join relations for partition-wise join,
+ * create paths for those and then create append paths to combine
+ * partition-wise join results.
+ *
+ * TODO: naming convention in this function is screwd up, sometimes we use
+ * part_ to indicate a partition variable and sometimes we use parent_ to
+ * indicate parent variable. Need to organize it better.
+ */
+static void
+try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
+						RelOptInfo *joinrel, SpecialJoinInfo *parent_sjinfo,
+						List *parent_restrictlist)
+{
+	PartitionOptInfo *part_info1 = rel1->part_info;
+	PartitionOptInfo *part_info2 = rel2->part_info;
+	int		nparts;
+	int		cnt_parts;
+	StringInfo	rel1_desc;
+	StringInfo	rel2_desc;
+	PartitionOptInfo *part_info;
+
+	if (!joinrel->part_info)
+	{
+		/* Can this join be executed using partition-wise join? */
+		joinrel->part_info = build_joinrel_part_info(rel1, rel2,
+													 parent_sjinfo->jointype,
+													 parent_restrictlist);
+
+		/* If partition-wise join is not possible, nothing to do. */
+		if (!joinrel->part_info)
+			return;
+	}
+	else
+	{
+		/*
+		 * Generally, we should check whether the partition scheme for the join
+		 * is same as the partitioning scheme of both the joining relations. If
+		 * not, we should create a new PartitionOptInfo for new partitioning
+		 * scheme. But we allow partition-wise join only in those cases, when
+		 * the partitioning scheme of resultant relation is same irrespective
+		 * of the order in which the relations are joined. Hence make sure that
+		 * the already computed partition scheme for join matches that required
+		 * by the current pair of joining relations.
+		 */
+		Assert(match_joinrel_part_info(joinrel, rel1, rel2,
+									   parent_sjinfo->jointype,
+									   parent_restrictlist));
+	}
+
+	part_info = joinrel->part_info;
+	nparts = part_info->nparts;
+	rel1_desc = makeStringInfo();
+	rel2_desc = makeStringInfo();
+
+	/* TODO: remove this notice when finalising the patch. */
+	outBitmapset(rel1_desc, rel1->relids);
+	outBitmapset(rel2_desc, rel2->relids);
+	elog(NOTICE, "join between relations %s and %s is considered for partition-wise join.",
+		 rel1_desc->data, rel2_desc->data);
+
+	/*
+	 * Create join relations for the partition relations, if they do not exist
+	 * already. Add paths to those for the given pair of joining relations.
+	 */
+	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	{
+		RelOptInfo	*part_joinrel;
+		RelOptInfo	*part_rel1 = part_info1->part_rels[cnt_parts];
+		RelOptInfo	*part_rel2 = part_info2->part_rels[cnt_parts];
+		SpecialJoinInfo *sjinfo;
+		List		*restrictlist;
+		List		*join_aris;
+		List		*ari1;
+		List		*ari2;
+		Relids	joinrelids;
+
+		/* We should never try to join two overlapping sets of rels. */
+		Assert(!bms_overlap(part_rel1->relids, part_rel2->relids));
+
+		/* Construct Relids set that identifies the joinrel. */
+		joinrelids = bms_union(part_rel1->relids, part_rel2->relids);
+
+		/*
+		 * Gather the AppendRelInfos for base partition relations
+		 * partiticipating in the given partition relations. We need them
+		 * construct partition-wise join relation, special join info and
+		 * restriction list by substituting the Var and relids from parent to
+		 * child.
+		 */
+		ari1 = find_appendrelinfos_by_relids(root, part_rel1->relids);
+		ari2 = find_appendrelinfos_by_relids(root, part_rel2->relids);
+
+		/*
+		 * Construct SpecialJoinInfo from parent join relations's
+		 * SpecialJoinInfo.
+		 */
+		sjinfo = build_partition_join_sjinfo(root, parent_sjinfo, ari1, ari2);
+
+		/* Construct the parent-child relid map for the join relation. */
+		join_aris = list_concat(ari1, ari2);
+
+		/*
+		 * Construct restrictions applicable to the partition-wise join from
+		 * those applicable to the join between the parents.
+		 */
+		restrictlist = (List *) adjust_partitionrel_attrs(root,
+													(Node *)parent_restrictlist,
+													join_aris);
+
+		part_joinrel = part_info->part_rels[cnt_parts];
+
+		/* Construct the join relation for given partition of the join. */
+		if (!part_joinrel)
+		{
+			part_joinrel = make_joinrel(root, RELOPT_OTHER_JOINREL,
+										joinrelids);
+
+			/*
+			 * Fill partition join relation by translating information from the
+			 * parent join relation.
+			 */
+			fill_partition_join_rel(root, part_rel1, part_rel2, part_joinrel,
+									joinrel, sjinfo, restrictlist, join_aris);
+
+			part_info->part_rels[cnt_parts] = part_joinrel;
+
+			add_joinrel_to_list(root, part_joinrel);
+		}
+
+		/* TODO: set_append_rel_size() calls add_child_rel_equivalences() for
+		 * adding child equivalence classes one parent at a time. So, we do not
+		 * add equivalence classes with multiple child relations. Do we need to
+		 * do that? Possibily because of the parent node translation above we
+		 * don't need it. Anyway, is there a way to verify it?
+		 */
+		Assert(bms_equal(part_joinrel->relids, joinrelids));
+
+		bms_free(joinrelids);
+
+		/*
+		 * If we've already proven that this join is empty, we needn't consider
+		 * any more paths for it.
+		 */
+		if (is_dummy_rel(part_joinrel))
+			continue;
+
+		populate_joinrel_with_paths(root, part_rel1, part_rel2, part_joinrel,
+									sjinfo, restrictlist);
+
+		/*
+		 * If the partition themselves are partitioned, try partition-wise join
+		 * recursively.
+		 * TODO: we need to decide the correct place for this function to be
+		 * called. The recursive nature of this call probably requires
+		 * AppendPath to be added before we try to populate the other kinds of
+		 * join paths for parent relations. So, we may have to call this
+		 * function before populate_joinrel_with_paths().
+		 */
+		try_partition_wise_join(root, part_rel1, part_rel2, part_joinrel, sjinfo,
+								restrictlist);
+	}
+}
+
+/*
+ * Construct the SpecialJoinInfo for the partition-wise join using parents'
+ * special join info. Also, instead of
+ * constructing an sjinfo everytime, we should probably save it in
+ * root->join_info_list and search within it like join_is_legal?
+ */
+static SpecialJoinInfo *
+build_partition_join_sjinfo(PlannerInfo *root, SpecialJoinInfo *parent_sjinfo,
+							List *append_rel_infos1, List *append_rel_infos2)
+{
+	SpecialJoinInfo *sjinfo = copyObject(parent_sjinfo);
+	sjinfo->min_lefthand = adjust_partition_relids(sjinfo->min_lefthand,
+												   append_rel_infos1);
+	sjinfo->min_righthand = adjust_partition_relids(sjinfo->min_righthand,
+													append_rel_infos2);
+	sjinfo->syn_lefthand = adjust_partition_relids(sjinfo->syn_lefthand,
+												   append_rel_infos1);
+	sjinfo->syn_righthand = adjust_partition_relids(sjinfo->syn_righthand,
+													append_rel_infos2);
+
+	/* Replace the Var nodes of parent with those of children in expressions. */
+	sjinfo->semi_rhs_exprs = (List *) adjust_partitionrel_attrs(root,
+											   (Node *) sjinfo->semi_rhs_exprs,
+															append_rel_infos2);
+	return sjinfo;
+}
+
+/*
+ * have_same_partition_lists
+ *
+ * For given list partitioned relations, return true if lists for all the
+ * partitions of both the relations match. Return false otherwise.
+ */
+static bool
+have_same_partition_lists(PartitionOptInfo *part_info1,
+						  PartitionOptInfo *part_info2)
+{
+	int		cnt_parts;
+
+	Assert(part_info1->strategy == PARTITION_STRAT_LIST &&
+		   part_info2->strategy == PARTITION_STRAT_LIST);
+
+	Assert(part_info1->nparts == part_info2->nparts);
+
+	/* List partition has only one partition key. */
+	Assert(part_info1->partnatts == 1 && part_info2->partnatts == 1);
+
+	for (cnt_parts = 0; cnt_parts < part_info1->nparts; cnt_parts++)
+	{
+		PartitionListInfo *list1 = part_info1->lists[cnt_parts];
+		PartitionListInfo *list2 = part_info2->lists[cnt_parts];
+		int		cnt;
+
+		/*
+		 * If number of items in the lists do not match or the one of the lists
+		 * contains NULL value and other doesn't, lists do not match.
+		 *
+		 * TODO: This condition can be relaxed to accomodate subset of items,
+		 * but probably not in the first cut.
+		 */
+		if (list1->nvalues != list2->nvalues)
+			return false;
+
+		/* Compare actual values in the list. */
+		for (cnt = 0; cnt < list1->nvalues; cnt++)
+		{
+			Datum	cmpval;
+
+			if (list1->nulls[cnt] != list2->nulls[cnt])
+				return false;
+
+			/* Skip NULL value comparisons. */
+			if (list1->nulls[cnt])
+				continue;
+
+			cmpval = FunctionCall2Coll(&part_info1->partsupfunc[0],
+									   part_info1->tcinfo->typcoll[0],
+									   list1->values[cnt],
+									   list2->values[cnt]);
+			if (DatumGetInt32(cmpval))
+				return false;
+		}
+	}
+
+	/* Ok, everything matches, return true. */
+	return true;
+}
+
+/*
+ * have_same_partition_bounds
+ *
+ * For given partitioned relations, return true if the bounds of all the
+ * partitions of the both the relations match. Return false otherwise.
+ */
+static bool
+have_same_partition_bounds(PartitionOptInfo *part_info1,
+						   PartitionOptInfo *part_info2)
+{
+	int		cnt_parts;
+	Assert(part_info1->strategy == PARTITION_STRAT_RANGE &&
+		   part_info2->strategy == PARTITION_STRAT_RANGE);
+
+	Assert(part_info1->nparts == part_info2->nparts);
+
+	for (cnt_parts = 0; cnt_parts < part_info1->nparts; cnt_parts++)
+	{
+		PartitionRangeBound *lower1 = part_info1->ranges[cnt_parts]->lower;
+		PartitionRangeBound *lower2 = part_info2->ranges[cnt_parts]->lower;
+		PartitionRangeBound *upper1 = part_info1->ranges[cnt_parts]->upper;
+		PartitionRangeBound *upper2 = part_info2->ranges[cnt_parts]->upper;
+
+		if (!compare_partition_bounds(part_info1, lower1, lower2) ||
+			!compare_partition_bounds(part_info1, upper1, upper2))
+			return false;
+	}
+
+	/* Ok, everything matches. */
+	return true;
+}
+
+/*
+ * compare_partition_bounds
+ *
+ * Return true if given bound specifications match.
+ */
+static bool
+compare_partition_bounds(PartitionOptInfo *part_info,
+						 PartitionRangeBound *bound1,
+						 PartitionRangeBound *bound2)
+{
+	int		cnt_pks;
+	Datum  *val1 = bound1->val;
+	Datum  *val2 = bound2->val;
+
+	if (bound1->infinite != bound2->infinite ||
+		bound1->inclusive != bound2->inclusive	||
+		bound1->lower != bound2->lower)
+		return false;
+
+	/*
+	 * TODO:
+	 * Following part is common with compare_range_keys(), but that takes
+	 * PartitionKey as argument, which we don't have here.
+	 */
+	for (cnt_pks = 0; cnt_pks < part_info->partnatts; cnt_pks++)
+	{
+		Datum	cmpval;
+
+		cmpval = FunctionCall2Coll(&part_info->partsupfunc[cnt_pks],
+								   part_info->tcinfo->typcoll[cnt_pks],
+								   val1[cnt_pks], val2[cnt_pks]);
+
+		if (DatumGetInt32(cmpval))
+			return false;
+	}
+
+	/* Ok, everything matches. */
+	return true;
+}
+
+/*
+ * partkey_types_match
+ *
+ * Returns true if the partition key datatype and collation match for given
+ * partitioned relations. Returns false otherwise.
+ */
+static bool
+partkey_types_match(PartitionOptInfo *part_info1, PartitionOptInfo *part_info2)
+{
+	int		cnt_pks;
+	Oid	   *partopfamily1 = part_info1->partopfamily;
+	Oid	   *partopfamily2 = part_info2->partopfamily;
+	Oid	   *partopcintype1 = part_info1->partopcintype;
+	Oid	   *partopcintype2 = part_info2->partopcintype;
+	FmgrInfo *partsupfunc1 = part_info1->partsupfunc;
+	FmgrInfo *partsupfunc2 = part_info2->partsupfunc;
+	KeyTypeCollInfo *tcinfo1 = part_info1->tcinfo;
+	KeyTypeCollInfo *tcinfo2 = part_info2->tcinfo;
+
+	if (part_info1->partnatts != part_info1->partnatts)
+		return false;
+
+	for (cnt_pks = 0; cnt_pks < part_info1->partnatts; cnt_pks++)
+	{
+		/*
+		 * It suffices to check the OID of support function as it always has
+		 * two arguemnts and returns boolean. For types, it suffices to match
+		 * the type id, mod and collation; len, byval and align are depedent on
+		 * the first two.
+		 */
+		if (partopfamily1[cnt_pks] != partopfamily2[cnt_pks] ||
+			partopcintype1[cnt_pks] != partopcintype2[cnt_pks] ||
+			partsupfunc1[cnt_pks].fn_oid != partsupfunc2[cnt_pks].fn_oid ||
+			tcinfo1->typid[cnt_pks] != tcinfo2->typid[cnt_pks] ||
+			tcinfo1->typmod[cnt_pks] != tcinfo2->typmod[cnt_pks] ||
+			tcinfo1->typcoll[cnt_pks] != tcinfo2->typcoll[cnt_pks])
+			return false;
+	}
+
+	/* Everything matches. */
+	return true;
+}
+
+/*
+ * Substitute oldrelids with newrelids in the given Relids set. It recycles the
+ * given relids input.
+ */
+Relids
+adjust_partition_relids(Relids relids, List *append_rel_infos)
+{
+	ListCell	*lc;
+	foreach (lc, append_rel_infos)
+	{
+		AppendRelInfo	*ari = lfirst(lc);
+
+		/* Remove old, add new */
+		if (bms_is_member(ari->parent_relid, relids))
+		{
+			relids = bms_del_member(relids, ari->parent_relid);
+			relids = bms_add_member(relids, ari->child_relid);
+		}
+	}
+	return relids;
+}
+
+/*
+ * Check whether the given two partitioning schemes are same.
+ */
+static bool
+have_same_part_info(PartitionOptInfo *part_info1, PartitionOptInfo *part_info2)
+{
+	int		cnt;
+
+	/*
+	 * If number of partitions, number of partition keys and partitioning
+	 * strategy of the joining relations do not match, we can not apply
+	 * partition-wise join.
+	 *
+	 * TODO: it should be possible to push an inner join down even if the number of
+	 * partitions differ but the common partitions match. In such a case pushing
+	 * down outer joins would be tricky, but still doable using empty relation
+	 * for non-existing partition.
+	 */
+	if (!part_info1 || !part_info2 ||
+		part_info1->nparts != part_info2->nparts ||
+		part_info1->strategy != part_info2->strategy ||
+		part_info1->partnatts != part_info2->partnatts)
+		return false;
+
+	/*
+	 * Partition-wise join between multi-level partitioned tables is not
+	 * supported, since the partition hierarchy doesn't get translated into the
+	 * corresponding RelOptInfo hierarchy. Since the partition hierarchy is
+	 * flattened, if we add append paths to intermediate partition relations,
+	 * the corresponding rows will be added multiple times.
+	 */
+	for (cnt = 0; cnt < part_info1->nparts; cnt++)
+	{
+		RelOptInfo *part_rel1 = part_info1->part_rels[cnt];
+		RelOptInfo *part_rel2 = part_info2->part_rels[cnt];
+		if (!part_rel1 || !part_rel2 || part_rel1->part_info ||
+			part_rel2->part_info)
+			return false;
+	}
+
+	/*
+	 * Partition-wise join can not be applied if datatypes and collations of
+	 * the partition keys do not match.
+	 */
+	if (!partkey_types_match(part_info1, part_info2))
+		return false;
+
+	/*
+	 * Partition-wise join can not be applied if the partition bounds or lists
+	 * of joining relations do not match.
+	 */
+	switch (part_info1->strategy)
+	{
+		case PARTITION_STRAT_LIST:
+			if (!have_same_partition_lists(part_info1, part_info2))
+				return false;
+			break;
+
+		case PARTITION_STRAT_RANGE:
+			if (!have_same_partition_bounds(part_info1, part_info2))
+				return false;
+			break;
+
+		default:
+			/* Unknown partition strategy. */
+			return false;
+	}
+
+	return true;
+}
+
+/*
+ * build_joinrel_part_info
+ *
+ * Given partition description of two joining relations, construct partition
+ * description for join between those relations. If we can execute
+ * partition-wise join the resultant join is partitioned in the same way as the
+ * joining relations. Otherwise, it is not partitioned and in such case the
+ * function returns NULL.
+ *
+ * TODO find the right place for this function.
+ *
+ * TODO:
+ * This function and
+ * build_partition_info() should be named consistently e.g.
+ * build_joinrel_partition_info and build_simplerel_partition_info or
+ * build_baserel_partition_info resp.
+ *
+ * TODO: this code can be simplified a lot if we precompute the partitioning
+ * compatibility between relations. See Robert's idea in 9th August mail on
+ * thread "Is any ordering of N-way join always partition-wise joinable?"
+ */
+static PartitionOptInfo *
+build_joinrel_part_info(RelOptInfo *rel1, RelOptInfo *rel2,
+						JoinType jointype, List *restrictlist)
+{
+	PartitionOptInfo	*part_info;
+	PartitionOptInfo	*part_info1 = rel1->part_info;
+	PartitionOptInfo	*part_info2 = rel2->part_info;
+	int		cnt;
+	int		num_pks;
+
+	/* Do nothing, if user doesn't want to try partition-wise join. */
+	if (!enable_partition_wise_join)
+		return NULL;
+
+	if (!have_same_part_info(part_info1, part_info2))
+		return NULL;
+
+	/*
+	 * Partition-wise join can not be applied if there is no equi-join
+	 * condition between partition keys.
+	 */
+	if (!have_partkey_equi_join(rel1, rel2, jointype, restrictlist))
+		return NULL;
+
+	/* The join is partitioned the similar to the joining relations. */
+	part_info = makeNode(PartitionOptInfo);
+
+	/* Information related to the partititions. */
+	part_info->nparts = part_info1->nparts;
+	/* Allocate space for partition RelOptInfos, which will be filled later. */
+	part_info->part_rels = (RelOptInfo **) palloc0(sizeof(RelOptInfo *) *
+												  part_info->nparts);
+	part_info->lists = part_info1->lists;
+	part_info->ranges = part_info1->ranges;
+
+	/* Information related to partition keys. */
+	part_info->strategy = part_info1->strategy;
+	part_info->partnatts = part_info1->partnatts;
+	part_info->partopfamily = part_info1->partopfamily;
+	part_info->partopcintype = part_info1->partopcintype;
+	part_info->partsupfunc = part_info1->partsupfunc;
+	part_info->tcinfo = part_info1->tcinfo;
+	num_pks = part_info->partnatts;
+	part_info->partexprs = (List **) palloc0(sizeof(List *) * num_pks);
+	for (cnt = 0; cnt < num_pks; cnt++)
+	{
+		List *pkexpr = list_copy(part_info1->partexprs[cnt]);
+
+		pkexpr = list_concat(pkexpr,
+							 list_copy(part_info2->partexprs[cnt]));
+		part_info->partexprs[cnt] = pkexpr;
+	}
+
+	return part_info;
+}
+
+/*
+ * Returns true if the given relations have equi-join clauses on all the
+ * corresponding partition keys.
+ */
+static bool
+have_partkey_equi_join(RelOptInfo *rel1, RelOptInfo *rel2,
+					 JoinType jointype, List *restrictlist)
+{
+	PartitionOptInfo	*part_info1 = rel1->part_info;
+	PartitionOptInfo	*part_info2 = rel2->part_info;
+	ListCell	*lc;
+	int		cnt_pks;
+	int		num_pks;
+	bool   *pk_has_clause;
+
+	Assert(part_info1->partnatts == part_info2->partnatts);
+	num_pks = part_info1->partnatts;
+	pk_has_clause = (bool *) palloc0(sizeof(bool) * num_pks);
+
+	foreach (lc, restrictlist)
+	{
+		RestrictInfo *rinfo = lfirst(lc);
+		OpExpr		 *opexpr;
+		Expr		 *expr1;
+		Expr		 *expr2;
+		int		ipk1;
+		int		ipk2;
+
+		/* If processing an outer join, only use its own join clauses. */
+		if (IS_OUTER_JOIN(jointype) && rinfo->is_pushed_down)
+			continue;
+
+		/* Skip non-equi-join clauses. */
+		if (!rinfo->can_join ||
+			rinfo->hashjoinoperator == InvalidOid ||
+			!rinfo->mergeopfamilies)
+			continue;
+
+		opexpr = (OpExpr *) rinfo->clause;
+		Assert(is_opclause(opexpr));
+
+		/*
+		 * If clause of form rel1_expr op rel2_expr OR rel2_expr op rel1_expr,
+		 * match the operands to the relations. Otherwise, the clause is
+		 * not an equi-join between partition keys of joining relations.
+		 */
+		if (bms_is_subset(rinfo->left_relids, rel1->relids) &&
+			bms_is_subset(rinfo->right_relids, rel2->relids))
+		{
+			expr1 = linitial(opexpr->args);
+			expr2 = lsecond(opexpr->args);
+		}
+		else if (bms_is_subset(rinfo->left_relids, rel2->relids) &&
+				 bms_is_subset(rinfo->right_relids, rel1->relids))
+		{
+			expr1 = lsecond(opexpr->args);
+			expr2 = linitial(opexpr->args);
+		}
+		else
+			continue;
+
+		/* Associate matching clauses with partition keys. */
+		ipk1 = match_expr_to_partition_keys(expr1, rel1);
+		ipk2 = match_expr_to_partition_keys(expr2, rel2);
+
+		if (ipk1 == ipk2)
+			pk_has_clause[ipk1] = true;
+	}
+
+	/*
+	 * If every pair of partition key from either of the joining relation has
+	 * at least one equi-join clause associated with it, we have an equi-join
+	 * between all corresponding partition keys.
+	 */
+	for (cnt_pks = 0; cnt_pks < num_pks; cnt_pks++)
+	{
+		if (!pk_has_clause[cnt_pks])
+		{
+			pfree(pk_has_clause);
+			return false;
+		}
+	}
+
+	pfree(pk_has_clause);
+	return true;
+}
+
+/*
+ * match_expr_to_partition_keys
+ *
+ * Find the partition key which is same as the given expression. If found,
+ * return the index of the partition key, else return -1.
+ */
+static int
+match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel)
+{
+	PartitionOptInfo *part_info = rel->part_info;
+	int		cnt_pks;
+
+	/*
+	 * Remove the relabel decoration. We can assume that there is at most one
+	 * RelabelType node; eval_const_expressions() will have simplied if more
+	 * than one.
+	 */
+	if (IsA(expr, RelabelType))
+		expr = (Expr *) ((RelabelType *) expr)->arg;
+
+	for (cnt_pks = 0; cnt_pks < part_info->partnatts; cnt_pks++)
+	{
+		List	 *pkexprs = part_info->partexprs[cnt_pks];
+		ListCell *lc;
+
+		foreach(lc, pkexprs)
+		{
+			Expr *pkexpr = lfirst(lc);
+			if (equal(pkexpr, expr))
+				return cnt_pks;
+		}
+	}
+
+	return -1;
+}
+
+/*
+ * Checks
+ * 1. if the partitioning scheme of the join relation match that of the joining
+ *    relations.
+ * 2. if there exists equi-join condition between the partition keys of the
+ *    joining relations.
+ * 3. if the partition keys of the join contain the partition keys of both the
+ *    relations.
+ */
+static bool
+match_joinrel_part_info(RelOptInfo *joinrel, RelOptInfo *rel1,
+						RelOptInfo *rel2, JoinType jointype,
+						List *restrictlist)
+{
+	int		cnt;
+	int		num_pks;
+	PartitionOptInfo *join_part_info = joinrel->part_info;
+	PartitionOptInfo *part_info1 = rel1->part_info;
+	PartitionOptInfo *part_info2= rel1->part_info;
+
+	Assert(enable_partition_wise_join);
+
+	/*
+	 * If partitioning scheme of the join doesn't match that of the joining
+	 * relations, join relations is not partitioned in the same way as one or
+	 * both of the joining relations.
+	 */
+	if (!have_same_part_info(join_part_info, part_info1) ||
+		!have_same_part_info(join_part_info, part_info2))
+		return false;
+
+	/*
+	 * If an equi-join condition between the partition keys of the joining
+	 * relations does not exist, the given partition scheme can not be used for
+	 * partition-wise join between these two relations.
+	 */
+	if (!have_partkey_equi_join(rel1, rel2, jointype, restrictlist))
+		return false;
+
+	/*
+	 * If the partition keys from either of the joining relations are not part
+	 * of the partition keys of the join relation, again we can not use the
+	 * given partition scheme for joining the given relations partition-wise.
+	 */
+	Assert(join_part_info->partnatts == part_info1->partnatts);
+	Assert(join_part_info->partnatts == part_info2->partnatts);
+
+	num_pks = join_part_info->partnatts;
+	for (cnt = 0; cnt < num_pks; cnt++)
+	{
+		List *pkexpr = join_part_info->partexprs[cnt];
+		List *pkexpr1 = part_info1->partexprs[cnt];
+		List *pkexpr2 = part_info2->partexprs[cnt];
+		List *diff_list;
+
+		diff_list = list_difference(pkexpr1, pkexpr);
+		if (diff_list)
+		{
+			list_free(diff_list);
+			return false;
+		}
+
+		diff_list = list_difference(pkexpr2, pkexpr);
+		if (diff_list)
+		{
+			list_free(diff_list);
+			return false;
+		}
+	}
+
+	return true;
+}
diff --git a/src/backend/optimizer/path/pathkeys.c b/src/backend/optimizer/path/pathkeys.c
index 4436ac1..dba6772 100644
--- a/src/backend/optimizer/path/pathkeys.c
+++ b/src/backend/optimizer/path/pathkeys.c
@@ -1081,26 +1081,38 @@ select_outer_pathkeys_for_merge(PlannerInfo *root,
 								List *mergeclauses,
 								RelOptInfo *joinrel)
 {
 	List	   *pathkeys = NIL;
 	int			nClauses = list_length(mergeclauses);
 	EquivalenceClass **ecs;
 	int		   *scores;
 	int			necs;
 	ListCell   *lc;
 	int			j;
+	Relids		relids;
 
 	/* Might have no mergeclauses */
 	if (nClauses == 0)
 		return NIL;
 
 	/*
+	 * Code below scores equivalence classes by how many equivalence members
+	 * can produce join clauses for this join relation. Equivalence members
+	 * which do not cover the parents of a partition-wise join relation, can
+	 * produce join clauses for partition-wise join relation.
+	 */
+	if (joinrel->reloptkind == RELOPT_OTHER_JOINREL)
+		relids = joinrel->parent_relids;
+	else
+		relids = joinrel->relids;
+
+	/*
 	 * Make arrays of the ECs used by the mergeclauses (dropping any
 	 * duplicates) and their "popularity" scores.
 	 */
 	ecs = (EquivalenceClass **) palloc(nClauses * sizeof(EquivalenceClass *));
 	scores = (int *) palloc(nClauses * sizeof(int));
 	necs = 0;
 
 	foreach(lc, mergeclauses)
 	{
 		RestrictInfo *rinfo = (RestrictInfo *) lfirst(lc);
@@ -1126,21 +1138,21 @@ select_outer_pathkeys_for_merge(PlannerInfo *root,
 			continue;
 
 		/* compute score */
 		score = 0;
 		foreach(lc2, oeclass->ec_members)
 		{
 			EquivalenceMember *em = (EquivalenceMember *) lfirst(lc2);
 
 			/* Potential future join partner? */
 			if (!em->em_is_const && !em->em_is_child &&
-				!bms_overlap(em->em_relids, joinrel->relids))
+				!bms_overlap(em->em_relids, relids))
 				score++;
 		}
 
 		ecs[necs] = oeclass;
 		scores[necs] = score;
 		necs++;
 	}
 
 	/*
 	 * Find out if we have all the ECs mentioned in query_pathkeys; if so we
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 32f4031..b221e2c 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -235,21 +235,22 @@ static Plan *prepare_sort_from_pathkeys(Plan *lefttree, List *pathkeys,
 						   const AttrNumber *reqColIdx,
 						   bool adjust_tlist_in_place,
 						   int *p_numsortkeys,
 						   AttrNumber **p_sortColIdx,
 						   Oid **p_sortOperators,
 						   Oid **p_collations,
 						   bool **p_nullsFirst);
 static EquivalenceMember *find_ec_member_for_tle(EquivalenceClass *ec,
 					   TargetEntry *tle,
 					   Relids relids);
-static Sort *make_sort_from_pathkeys(Plan *lefttree, List *pathkeys);
+static Sort *make_sort_from_pathkeys(Plan *lefttree, List *pathkeys,
+									 Relids relids);
 static Sort *make_sort_from_groupcols(List *groupcls,
 						 AttrNumber *grpColIdx,
 						 Plan *lefttree);
 static Material *make_material(Plan *lefttree);
 static WindowAgg *make_windowagg(List *tlist, Index winref,
 			   int partNumCols, AttrNumber *partColIdx, Oid *partOperators,
 			   int ordNumCols, AttrNumber *ordColIdx, Oid *ordOperators,
 			   int frameOptions, Node *startOffset, Node *endOffset,
 			   Plan *lefttree);
 static Group *make_group(List *tlist, List *qual, int numGroupCols,
@@ -1507,21 +1508,21 @@ create_sort_plan(PlannerInfo *root, SortPath *best_path, int flags)
 	Plan	   *subplan;
 
 	/*
 	 * We don't want any excess columns in the sorted tuples, so request a
 	 * smaller tlist.  Otherwise, since Sort doesn't project, tlist
 	 * requirements pass through.
 	 */
 	subplan = create_plan_recurse(root, best_path->subpath,
 								  flags | CP_SMALL_TLIST);
 
-	plan = make_sort_from_pathkeys(subplan, best_path->path.pathkeys);
+	plan = make_sort_from_pathkeys(subplan, best_path->path.pathkeys, NULL);
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
 	return plan;
 }
 
 /*
  * create_group_plan
  *
  *	  Create a Group plan for 'best_path' and (recursively) plans
@@ -3517,31 +3518,33 @@ create_mergejoin_plan(PlannerInfo *root,
 	List	   *innerpathkeys;
 	int			nClauses;
 	Oid		   *mergefamilies;
 	Oid		   *mergecollations;
 	int		   *mergestrategies;
 	bool	   *mergenullsfirst;
 	int			i;
 	ListCell   *lc;
 	ListCell   *lop;
 	ListCell   *lip;
+	Path	   *outer_path = best_path->jpath.outerjoinpath;
+	Path	   *inner_path = best_path->jpath.innerjoinpath;
 
 	/*
 	 * MergeJoin can project, so we don't have to demand exact tlists from the
 	 * inputs.  However, if we're intending to sort an input's result, it's
 	 * best to request a small tlist so we aren't sorting more data than
 	 * necessary.
 	 */
-	outer_plan = create_plan_recurse(root, best_path->jpath.outerjoinpath,
+	outer_plan = create_plan_recurse(root, outer_path,
 					 (best_path->outersortkeys != NIL) ? CP_SMALL_TLIST : 0);
 
-	inner_plan = create_plan_recurse(root, best_path->jpath.innerjoinpath,
+	inner_plan = create_plan_recurse(root, inner_path,
 					 (best_path->innersortkeys != NIL) ? CP_SMALL_TLIST : 0);
 
 	/* Sort join qual clauses into best execution order */
 	/* NB: do NOT reorder the mergeclauses */
 	joinclauses = order_qual_clauses(root, best_path->jpath.joinrestrictinfo);
 
 	/* Get the join qual clauses (in plain expression form) */
 	/* Any pseudoconstant clauses are ignored here */
 	if (IS_OUTER_JOIN(best_path->jpath.jointype))
 	{
@@ -3573,48 +3576,52 @@ create_mergejoin_plan(PlannerInfo *root,
 		otherclauses = (List *)
 			replace_nestloop_params(root, (Node *) otherclauses);
 	}
 
 	/*
 	 * Rearrange mergeclauses, if needed, so that the outer variable is always
 	 * on the left; mark the mergeclause restrictinfos with correct
 	 * outer_is_left status.
 	 */
 	mergeclauses = get_switched_clauses(best_path->path_mergeclauses,
-							 best_path->jpath.outerjoinpath->parent->relids);
+							 outer_path->parent->relids);
 
 	/*
 	 * Create explicit sort nodes for the outer and inner paths if necessary.
 	 */
 	if (best_path->outersortkeys)
 	{
+		Relids		outer_relids = outer_path->parent->relids;
 		Sort	   *sort = make_sort_from_pathkeys(outer_plan,
-												   best_path->outersortkeys);
+												   best_path->outersortkeys,
+												   outer_relids);
 
 		label_sort_with_costsize(root, sort, -1.0);
 		outer_plan = (Plan *) sort;
 		outerpathkeys = best_path->outersortkeys;
 	}
 	else
-		outerpathkeys = best_path->jpath.outerjoinpath->pathkeys;
+		outerpathkeys = outer_path->pathkeys;
 
 	if (best_path->innersortkeys)
 	{
+		Relids		inner_relids = inner_path->parent->relids;
 		Sort	   *sort = make_sort_from_pathkeys(inner_plan,
-												   best_path->innersortkeys);
+												   best_path->innersortkeys,
+												   inner_relids);
 
 		label_sort_with_costsize(root, sort, -1.0);
 		inner_plan = (Plan *) sort;
 		innerpathkeys = best_path->innersortkeys;
 	}
 	else
-		innerpathkeys = best_path->jpath.innerjoinpath->pathkeys;
+		innerpathkeys = inner_path->pathkeys;
 
 	/*
 	 * If specified, add a materialize node to shield the inner plan from the
 	 * need to handle mark/restore.
 	 */
 	if (best_path->materialize_inner)
 	{
 		Plan	   *matplan = (Plan *) make_material(inner_plan);
 
 		/*
@@ -5330,25 +5337,25 @@ prepare_sort_from_pathkeys(Plan *lefttree, List *pathkeys,
 
 				/*
 				 * We shouldn't be trying to sort by an equivalence class that
 				 * contains a constant, so no need to consider such cases any
 				 * further.
 				 */
 				if (em->em_is_const)
 					continue;
 
 				/*
-				 * Ignore child members unless they match the rel being
+				 * Ignore child members unless they belong to the rel being
 				 * sorted.
 				 */
 				if (em->em_is_child &&
-					!bms_equal(em->em_relids, relids))
+					!bms_is_subset(em->em_relids, relids))
 					continue;
 
 				sortexpr = em->em_expr;
 				exprvars = pull_var_clause((Node *) sortexpr,
 										   PVC_INCLUDE_AGGREGATES |
 										   PVC_INCLUDE_WINDOWFUNCS |
 										   PVC_INCLUDE_PLACEHOLDERS);
 				foreach(k, exprvars)
 				{
 					if (!tlist_member_ignore_relabel(lfirst(k), tlist))
@@ -5445,57 +5452,58 @@ find_ec_member_for_tle(EquivalenceClass *ec,
 		Expr	   *emexpr;
 
 		/*
 		 * We shouldn't be trying to sort by an equivalence class that
 		 * contains a constant, so no need to consider such cases any further.
 		 */
 		if (em->em_is_const)
 			continue;
 
 		/*
-		 * Ignore child members unless they match the rel being sorted.
+		 * Ignore child members unless they belong to the rel being sorted.
 		 */
 		if (em->em_is_child &&
-			!bms_equal(em->em_relids, relids))
+			!bms_is_subset(em->em_relids, relids))
 			continue;
 
 		/* Match if same expression (after stripping relabel) */
 		emexpr = em->em_expr;
 		while (emexpr && IsA(emexpr, RelabelType))
 			emexpr = ((RelabelType *) emexpr)->arg;
 
 		if (equal(emexpr, tlexpr))
 			return em;
 	}
 
 	return NULL;
 }
 
 /*
  * make_sort_from_pathkeys
  *	  Create sort plan to sort according to given pathkeys
  *
  *	  'lefttree' is the node which yields input tuples
  *	  'pathkeys' is the list of pathkeys by which the result is to be sorted
+ *	  'relids' is the set of relations required by prepare_sort_from_pathkeys()
  */
 static Sort *
-make_sort_from_pathkeys(Plan *lefttree, List *pathkeys)
+make_sort_from_pathkeys(Plan *lefttree, List *pathkeys, Relids relids)
 {
 	int			numsortkeys;
 	AttrNumber *sortColIdx;
 	Oid		   *sortOperators;
 	Oid		   *collations;
 	bool	   *nullsFirst;
 
 	/* Compute sort column info, and adjust lefttree as needed */
 	lefttree = prepare_sort_from_pathkeys(lefttree, pathkeys,
-										  NULL,
+										  relids,
 										  NULL,
 										  false,
 										  &numsortkeys,
 										  &sortColIdx,
 										  &sortOperators,
 										  &collations,
 										  &nullsFirst);
 
 	/* Now build the Sort node */
 	return make_sort(lefttree, numsortkeys,
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index 592214b..7e3e3b8 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -1873,20 +1873,79 @@ adjust_appendrel_attrs(PlannerInfo *root, Node *node, AppendRelInfo *appinfo)
 										   appinfo);
 		}
 		result = (Node *) newnode;
 	}
 	else
 		result = adjust_appendrel_attrs_mutator(node, &context);
 
 	return result;
 }
 
+/*
+ * find_appendrelinfos_by_relids
+ * 		Find AppendRelInfo structures for all relations specified by relids.
+ */
+List *
+find_appendrelinfos_by_relids(PlannerInfo *root, Relids relids)
+{
+	ListCell	*lc;
+	List		*ari_list = NIL;
+
+	foreach (lc, root->append_rel_list)
+	{
+		AppendRelInfo *ari = lfirst(lc);
+
+		if (bms_is_member(ari->child_relid, relids))
+			ari_list = lappend(ari_list, ari);
+	}
+
+	Assert(list_length(ari_list) == bms_num_members(relids));
+	return ari_list;
+}
+
+/*
+ * adjust_partitionrel_attrs
+ *	    Replace the Var nodes in given node with the corresponding Var nodes
+ *	    of the child. Given list of AppendRelInfo nodes holds the mapping
+ *	    between parent and child Var nodes.
+ *
+ * TODO:
+ * Note: This function is expected to be called only in case of partitioned
+ * tables, where the child table has the same schema as the parent table. This
+ * allows us to just restamp the Var nodes with child's relid.
+ *
+ * While doing so, we need to make sure to translate non-expression information
+ * in nodes like RestrictInfo.
+ */
+Node *
+adjust_partitionrel_attrs(PlannerInfo *root, Node *node,
+						  List *append_rel_infos)
+{
+	ListCell   *lc;
+
+	/*
+	 * TODO: for partitioned tables, since the partitions have same structure
+	 * as that of their parents, it should suffice to just restamp the Var node
+	 * rather than copying, but both require a new tree being allocated.
+	 * TODO: Instead of copying and mutating the trees one child relation at a
+	 * time, we should be able to do this en-masse for all the partitions
+	 * involved.
+	 */
+	foreach (lc, append_rel_infos)
+	{
+		AppendRelInfo	*append_rel_info = lfirst(lc);
+		node = adjust_appendrel_attrs(root, node, append_rel_info);
+	}
+
+	return node;
+}
+
 static Node *
 adjust_appendrel_attrs_mutator(Node *node,
 							   adjust_appendrel_attrs_context *context)
 {
 	AppendRelInfo *appinfo = context->appinfo;
 
 	if (node == NULL)
 		return NULL;
 	if (IsA(node, Var))
 	{
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index abb7507..4e04133 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -16,21 +16,23 @@
 
 #include <math.h>
 
 #include "miscadmin.h"
 #include "nodes/nodeFuncs.h"
 #include "optimizer/clauses.h"
 #include "optimizer/cost.h"
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
 #include "optimizer/planmain.h"
+#include "optimizer/prep.h"
 #include "optimizer/restrictinfo.h"
+#include "optimizer/tlist.h"
 #include "optimizer/var.h"
 #include "parser/parsetree.h"
 #include "utils/lsyscache.h"
 #include "utils/selfuncs.h"
 
 
 typedef enum
 {
 	COSTS_EQUAL,				/* path costs are fuzzily equal */
 	COSTS_BETTER1,				/* first path is cheaper than second */
@@ -1889,20 +1891,21 @@ calc_nestloop_required_outer(Path *outer_path, Path *inner_path)
 	/* inner_path can require rels from outer path, but not vice versa */
 	Assert(!bms_overlap(outer_paramrels, inner_path->parent->relids));
 	/* easy case if inner path is not parameterized */
 	if (!inner_paramrels)
 		return bms_copy(outer_paramrels);
 	/* else, form the union ... */
 	required_outer = bms_union(outer_paramrels, inner_paramrels);
 	/* ... and remove any mention of now-satisfied outer rels */
 	required_outer = bms_del_members(required_outer,
 									 outer_path->parent->relids);
+
 	/* maintain invariant that required_outer is exactly NULL if empty */
 	if (bms_is_empty(required_outer))
 	{
 		bms_free(required_outer);
 		required_outer = NULL;
 	}
 	return required_outer;
 }
 
 /*
@@ -3202,10 +3205,160 @@ reparameterize_path(PlannerInfo *root, Path *path,
 														 rel,
 														 spath->subpath,
 														 spath->path.pathkeys,
 														 required_outer);
 			}
 		default:
 			break;
 	}
 	return NULL;
 }
+
+/*
+ * reparameterize_path_for_child
+ * 		Given a path parameterized by the parent of the given relation,
+ * 		translate the path to be parameterized by the given child relation.
+ *
+ * The function creates a new path of the same type as the given path, but
+ * parameterized by the given child relation. If it can not reparameterize the
+ * path as required, it returns NULL.
+ */
+Path *
+reparameterize_path_for_child(PlannerInfo *root, Path *path,
+							  RelOptInfo *child_rel)
+{
+	Path	   *new_path;
+	ParamPathInfo   *new_ppi;
+	ParamPathInfo   *old_ppi;
+	List	   *child_aris;
+
+	/*
+	 * If the path is not parameterized by parent of the given relation, it
+	 * doesn't need reparameterization.
+	 */
+	if (!path->param_info ||
+		!bms_overlap(PATH_REQ_OUTER(path), child_rel->parent_relids))
+	return path;
+
+	switch (nodeTag(path))
+	{
+		case T_Path:
+			new_path = makeNode(Path);
+			memcpy(new_path, path, sizeof(Path));
+			break;
+
+		case T_HashPath:
+			new_path = (Path *) makeNode(HashPath);
+			memcpy(new_path, path, sizeof(HashPath));
+			break;
+
+		case T_MergePath:
+			new_path = (Path *) makeNode(MergePath);
+			memcpy(new_path, path, sizeof(MergePath));
+			break;
+
+		case T_NestPath:
+			new_path = (Path *) makeNode(NestPath);
+			memcpy(new_path, path, sizeof(NestPath));
+			break;
+
+		case T_IndexPath:
+			new_path = (Path *) makeNode(IndexPath);
+			memcpy(new_path, path, sizeof(IndexPath));
+			break;
+
+		case T_AppendPath:
+			new_path = (Path *) makeNode(AppendPath);
+			memcpy(new_path, path, sizeof(AppendPath));
+			break;
+
+		/*
+		 * TODO:
+		 * If this method of translation is fine add more path types here.
+		 */
+
+		default:
+			/* Path type unsupported by this function. */
+			return NULL;
+	}
+
+	/*
+	 * Gather AppendRelInfos of the base partition relations in the outer child
+	 * relation. We need those for translating parent path to that of child by
+	 * substituting parent Var nodes and relids with those of children.
+	 */
+	child_aris = find_appendrelinfos_by_relids(root, child_rel->relids);
+
+	/* Adjust the parameterization information. */
+	old_ppi = new_path->param_info;
+	new_ppi = makeNode(ParamPathInfo);
+	new_ppi->ppi_req_outer = adjust_partition_relids(bms_copy(old_ppi->ppi_req_outer),
+													 child_aris);
+	new_ppi->ppi_rows = old_ppi->ppi_rows;
+	new_ppi->ppi_clauses = (List *) adjust_partitionrel_attrs(root,
+													 (Node *) old_ppi->ppi_clauses,
+													 child_aris);
+
+	/* Adjust the path target. */
+	new_path->pathtarget = copy_pathtarget(new_path->pathtarget);
+	new_path->pathtarget->exprs = (List *) adjust_partitionrel_attrs(root,
+													(Node *) new_path->pathtarget->exprs,
+													 child_aris);
+	new_path->param_info = new_ppi;
+
+	/*
+	 * Change parameterization of sub paths recursively. Also carry out any
+	 * pathtype specific adjustments.
+	 */
+	switch (nodeTag(path))
+	{
+		case T_HashPath:
+		case T_MergePath:
+		case T_NestPath:
+			{
+				JoinPath *jpath = (JoinPath *)new_path;
+
+				jpath->outerjoinpath = reparameterize_path_for_child(root,
+														 jpath->outerjoinpath,
+														 child_rel);
+				jpath->innerjoinpath = reparameterize_path_for_child(root,
+														 jpath->innerjoinpath,
+														 child_rel);
+				jpath->joinrestrictinfo = (List *) adjust_partitionrel_attrs(root,
+															(Node *) jpath->joinrestrictinfo,
+															child_aris);
+			}
+			break;
+
+		case T_AppendPath:
+			{
+				AppendPath	*apath = (AppendPath *)new_path;
+				List		*subpaths = NIL;
+				ListCell	*lc;
+
+				foreach (lc, apath->subpaths)
+					subpaths = lappend(subpaths,
+									   reparameterize_path_for_child(root,
+																	lfirst(lc),
+																	child_rel));
+				apath->subpaths = subpaths;
+			}
+
+		case T_IndexPath:
+			{
+				IndexPath *ipath = (IndexPath *)new_path;
+
+				ipath->indexquals = (List *) adjust_partitionrel_attrs(root,
+														(Node *) ipath->indexquals,
+														child_aris);
+				ipath->indexquals = (List *) adjust_partitionrel_attrs(root,
+														(Node *) ipath->indexorderbys,
+														child_aris);
+			}
+
+		default:
+			/* Nothing to do. */
+			break;
+	}
+
+	return new_path;
+}
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index deef560..cea06c7 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -8,29 +8,34 @@
  *
  *
  * IDENTIFICATION
  *	  src/backend/optimizer/util/relnode.c
  *
  *-------------------------------------------------------------------------
  */
 #include "postgres.h"
 
 #include "miscadmin.h"
+#include "catalog/heap.h"
 #include "optimizer/clauses.h"
 #include "optimizer/cost.h"
+#include "nodes/makefuncs.h"
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
 #include "optimizer/placeholder.h"
 #include "optimizer/plancat.h"
+#include "optimizer/prep.h"
 #include "optimizer/restrictinfo.h"
 #include "optimizer/tlist.h"
+#include "rewrite/rewriteManip.h"
 #include "utils/hsearch.h"
+#include "utils/rel.h"
 
 
 typedef struct JoinHashEntry
 {
 	Relids		join_relids;	/* hash key --- MUST BE FIRST */
 	RelOptInfo *join_rel;
 } JoinHashEntry;
 
 static void build_joinrel_tlist(PlannerInfo *root, RelOptInfo *joinrel,
 					RelOptInfo *input_rel);
@@ -40,20 +45,24 @@ static List *build_joinrel_restrictlist(PlannerInfo *root,
 						   RelOptInfo *inner_rel);
 static void build_joinrel_joinlist(RelOptInfo *joinrel,
 					   RelOptInfo *outer_rel,
 					   RelOptInfo *inner_rel);
 static List *subbuild_joinrel_restrictlist(RelOptInfo *joinrel,
 							  List *joininfo_list,
 							  List *new_restrictlist);
 static List *subbuild_joinrel_joinlist(RelOptInfo *joinrel,
 						  List *joininfo_list,
 						  List *new_joininfo);
+static PartitionOptInfo *build_partition_info(PlannerInfo *root,
+							   RelOptInfo *rel);
+static void set_foreign_rel_properties(RelOptInfo *joinrel,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel);
 
 
 /*
  * setup_simple_rel_arrays
  *	  Prepare the arrays we use for quickly accessing base relations.
  */
 void
 setup_simple_rel_arrays(PlannerInfo *root)
 {
 	Index		rti;
@@ -165,27 +174,33 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptKind reloptkind)
 			break;
 		default:
 			elog(ERROR, "unrecognized RTE kind: %d",
 				 (int) rte->rtekind);
 			break;
 	}
 
 	/* Save the finished struct in the query's simple_rel_array */
 	root->simple_rel_array[relid] = rel;
 
+	/* Get the partitioning information, if any. */
+	if (rte->rtekind == RTE_RELATION)
+		rel->part_info = build_partition_info(root, rel);
+	else
+		rel->part_info = NULL;
+
 	/*
 	 * If this rel is an appendrel parent, recurse to build "other rel"
 	 * RelOptInfos for its children.  They are "other rels" because they are
 	 * not in the main join tree, but we will need RelOptInfos to plan access
 	 * to them.
 	 */
-	if (rte->inh)
+	if (!rel->part_info && rte->inh)
 	{
 		ListCell   *l;
 
 		foreach(l, root->append_rel_list)
 		{
 			AppendRelInfo *appinfo = (AppendRelInfo *) lfirst(l);
 
 			/* append_rel_list contains all append rels; ignore others */
 			if (appinfo->parent_relid != relid)
 				continue;
@@ -307,20 +322,70 @@ find_join_rel(PlannerInfo *root, Relids relids)
 
 			if (bms_equal(rel->relids, relids))
 				return rel;
 		}
 	}
 
 	return NULL;
 }
 
 /*
+ * set_foreign_rel_properties
+ *		Set up foreign-join fields if outer and inner relation are foreign
+ * 		tables (or joins) belonging to the same server and assigned to the same
+ *		user to check access permissions as.
+ *
+ * In addition to an exact match of userid, we allow the case where one side
+ * has zero userid (implying current user) and the other side has explicit
+ * userid that happens to equal the current user; but in that case, pushdown of
+ * the join is only valid for the current user.  The useridiscurrent field
+ * records whether we had to make such an assumption for this join or any
+ * sub-join.
+ *
+ * Otherwise these fields are left invalid, so GetForeignJoinPaths will not be
+ * called for the join relation.
+ *
+ */
+static void
+set_foreign_rel_properties(RelOptInfo *joinrel, RelOptInfo *outer_rel,
+						   RelOptInfo *inner_rel)
+{
+	if (OidIsValid(outer_rel->serverid) &&
+		inner_rel->serverid == outer_rel->serverid)
+	{
+		if (inner_rel->userid == outer_rel->userid)
+		{
+			joinrel->serverid = outer_rel->serverid;
+			joinrel->userid = outer_rel->userid;
+			joinrel->useridiscurrent = outer_rel->useridiscurrent || inner_rel->useridiscurrent;
+			joinrel->fdwroutine = outer_rel->fdwroutine;
+		}
+		else if (!OidIsValid(inner_rel->userid) &&
+				 outer_rel->userid == GetUserId())
+		{
+			joinrel->serverid = outer_rel->serverid;
+			joinrel->userid = outer_rel->userid;
+			joinrel->useridiscurrent = true;
+			joinrel->fdwroutine = outer_rel->fdwroutine;
+		}
+		else if (!OidIsValid(outer_rel->userid) &&
+				 inner_rel->userid == GetUserId())
+		{
+			joinrel->serverid = outer_rel->serverid;
+			joinrel->userid = inner_rel->userid;
+			joinrel->useridiscurrent = true;
+			joinrel->fdwroutine = outer_rel->fdwroutine;
+		}
+	}
+}
+
+/*
  * build_join_rel
  *	  Returns relation entry corresponding to the union of two given rels,
  *	  creating a new relation entry if none already exists.
  *
  * 'joinrelids' is the Relids set that uniquely identifies the join
  * 'outer_rel' and 'inner_rel' are relation nodes for the relations to be
  *		joined
  * 'sjinfo': join context info
  * 'restrictlist_ptr': result variable.  If not NULL, *restrictlist_ptr
  *		receives the list of RestrictInfo nodes that apply to this
@@ -356,21 +421,25 @@ build_join_rel(PlannerInfo *root,
 														   joinrel,
 														   outer_rel,
 														   inner_rel);
 		return joinrel;
 	}
 
 	/*
 	 * Nope, so make one.
 	 */
 	joinrel = makeNode(RelOptInfo);
+
+	Assert(!IS_OTHER_REL(outer_rel->reloptkind) &&
+		   !IS_OTHER_REL(inner_rel->reloptkind));
 	joinrel->reloptkind = RELOPT_JOINREL;
+
 	joinrel->relids = bms_copy(joinrelids);
 	joinrel->rows = 0;
 	/* cheap startup cost is interesting iff not all tuples to be retrieved */
 	joinrel->consider_startup = (root->tuple_fraction > 0);
 	joinrel->consider_param_startup = false;
 	joinrel->consider_parallel = false;
 	joinrel->reltarget = create_empty_pathtarget();
 	joinrel->pathlist = NIL;
 	joinrel->ppilist = NIL;
 	joinrel->partial_pathlist = NIL;
@@ -402,61 +471,25 @@ build_join_rel(PlannerInfo *root,
 	joinrel->serverid = InvalidOid;
 	joinrel->userid = InvalidOid;
 	joinrel->useridiscurrent = false;
 	joinrel->fdwroutine = NULL;
 	joinrel->fdw_private = NULL;
 	joinrel->baserestrictinfo = NIL;
 	joinrel->baserestrictcost.startup = 0;
 	joinrel->baserestrictcost.per_tuple = 0;
 	joinrel->joininfo = NIL;
 	joinrel->has_eclass_joins = false;
+	joinrel->part_info = NULL;
+	joinrel->parent_relids = NULL;
 
-	/*
-	 * Set up foreign-join fields if outer and inner relation are foreign
-	 * tables (or joins) belonging to the same server and assigned to the same
-	 * user to check access permissions as.  In addition to an exact match of
-	 * userid, we allow the case where one side has zero userid (implying
-	 * current user) and the other side has explicit userid that happens to
-	 * equal the current user; but in that case, pushdown of the join is only
-	 * valid for the current user.  The useridiscurrent field records whether
-	 * we had to make such an assumption for this join or any sub-join.
-	 *
-	 * Otherwise these fields are left invalid, so GetForeignJoinPaths will
-	 * not be called for the join relation.
-	 */
-	if (OidIsValid(outer_rel->serverid) &&
-		inner_rel->serverid == outer_rel->serverid)
-	{
-		if (inner_rel->userid == outer_rel->userid)
-		{
-			joinrel->serverid = outer_rel->serverid;
-			joinrel->userid = outer_rel->userid;
-			joinrel->useridiscurrent = outer_rel->useridiscurrent || inner_rel->useridiscurrent;
-			joinrel->fdwroutine = outer_rel->fdwroutine;
-		}
-		else if (!OidIsValid(inner_rel->userid) &&
-				 outer_rel->userid == GetUserId())
-		{
-			joinrel->serverid = outer_rel->serverid;
-			joinrel->userid = outer_rel->userid;
-			joinrel->useridiscurrent = true;
-			joinrel->fdwroutine = outer_rel->fdwroutine;
-		}
-		else if (!OidIsValid(outer_rel->userid) &&
-				 inner_rel->userid == GetUserId())
-		{
-			joinrel->serverid = outer_rel->serverid;
-			joinrel->userid = inner_rel->userid;
-			joinrel->useridiscurrent = true;
-			joinrel->fdwroutine = outer_rel->fdwroutine;
-		}
-	}
+	/* Computer information relevant to the foreign relations. */
+	set_foreign_rel_properties(joinrel, outer_rel, inner_rel);
 
 	/*
 	 * Create a new tlist containing just the vars that need to be output from
 	 * this join (ie, are needed for higher joinclauses or final output).
 	 *
 	 * NOTE: the tlist order for a join rel will depend on which pair of outer
 	 * and inner rels we first try to build it from.  But the contents should
 	 * be the same regardless.
 	 */
 	build_joinrel_tlist(root, joinrel, outer_rel);
@@ -510,57 +543,146 @@ build_join_rel(PlannerInfo *root,
 	 * assume this doesn't matter, because we should hit all the same baserels
 	 * and joinclauses while building up to this joinrel no matter which we
 	 * take; therefore, we should make the same decision here however we get
 	 * here.
 	 */
 	if (inner_rel->consider_parallel && outer_rel->consider_parallel &&
 		is_parallel_safe(root, (Node *) restrictlist) &&
 		is_parallel_safe(root, (Node *) joinrel->reltarget->exprs))
 		joinrel->consider_parallel = true;
 
-	/*
-	 * Add the joinrel to the query's joinrel list, and store it into the
-	 * auxiliary hashtable if there is one.  NB: GEQO requires us to append
-	 * the new joinrel to the end of the list!
-	 */
-	root->join_rel_list = lappend(root->join_rel_list, joinrel);
-
-	if (root->join_rel_hash)
-	{
-		JoinHashEntry *hentry;
-		bool		found;
-
-		hentry = (JoinHashEntry *) hash_search(root->join_rel_hash,
-											   &(joinrel->relids),
-											   HASH_ENTER,
-											   &found);
-		Assert(!found);
-		hentry->join_rel = joinrel;
-	}
+	/* Add the joinrel to the query's PlannerInfo. */
+	add_joinrel_to_list(root, joinrel);
 
 	/*
 	 * Also, if dynamic-programming join search is active, add the new joinrel
 	 * to the appropriate sublist.  Note: you might think the Assert on number
 	 * of members should be for equality, but some of the level 1 rels might
 	 * have been joinrels already, so we can only assert <=.
 	 */
 	if (root->join_rel_level)
 	{
 		Assert(root->join_cur_level > 0);
 		Assert(root->join_cur_level <= bms_num_members(joinrel->relids));
 		root->join_rel_level[root->join_cur_level] =
 			lappend(root->join_rel_level[root->join_cur_level], joinrel);
 	}
 
 	return joinrel;
 }
 
+RelOptInfo *
+make_joinrel(PlannerInfo *root, RelOptKind reloptkind, Relids joinrelids)
+{
+	RelOptInfo *joinrel = makeNode(RelOptInfo);
+	joinrel->reloptkind = RELOPT_OTHER_JOINREL;
+	joinrel->relids = bms_copy(joinrelids);
+	joinrel->rows = 0;
+	/* cheap startup cost is interesting iff not all tuples to be retrieved */
+	joinrel->consider_startup = (root->tuple_fraction > 0);
+	joinrel->consider_param_startup = false;
+	joinrel->consider_parallel = false;
+	joinrel->reltarget = create_empty_pathtarget();
+	joinrel->pathlist = NIL;
+	joinrel->ppilist = NIL;
+	joinrel->partial_pathlist = NIL;
+	joinrel->cheapest_startup_path = NULL;
+	joinrel->cheapest_total_path = NULL;
+	joinrel->cheapest_unique_path = NULL;
+	joinrel->cheapest_parameterized_paths = NIL;
+	joinrel->direct_lateral_relids = NULL;
+	joinrel->lateral_relids = NULL;
+	joinrel->relid = 0;			/* indicates not a baserel */
+	joinrel->rtekind = RTE_JOIN;
+	joinrel->min_attr = 0;
+	joinrel->max_attr = 0;
+	joinrel->attr_needed = NULL;
+	joinrel->attr_widths = NULL;
+	joinrel->lateral_vars = NIL;
+	joinrel->lateral_referencers = NULL;
+	joinrel->indexlist = NIL;
+	joinrel->pages = 0;
+	joinrel->tuples = 0;
+	joinrel->allvisfrac = 0;
+	joinrel->subroot = NULL;
+	joinrel->subplan_params = NIL;
+	joinrel->serverid = InvalidOid;
+	joinrel->userid = InvalidOid;
+	joinrel->useridiscurrent = false;
+	joinrel->fdwroutine = NULL;
+	joinrel->fdw_private = NULL;
+	joinrel->baserestrictinfo = NIL;
+	joinrel->baserestrictcost.startup = 0;
+	joinrel->baserestrictcost.per_tuple = 0;
+	joinrel->joininfo = NIL;
+	joinrel->has_eclass_joins = false;
+	joinrel->part_info = NULL;
+	joinrel->parent_relids = NULL;
+
+	return joinrel;
+}
+
+RelOptInfo *
+fill_partition_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
+						RelOptInfo *joinrel, RelOptInfo *parent_joinrel,
+						SpecialJoinInfo *sjinfo, List *restrictlist,
+						List *join_aris)
+{
+	List	   *tmp_exprs;
+
+	/* Only joins between other relations land here. */
+	Assert(IS_OTHER_REL(rel1->reloptkind) && IS_OTHER_REL(rel2->reloptkind));
+
+	joinrel->parent_relids = bms_copy(parent_joinrel->relids);
+
+	/* Computer information relevant to foreign relations. */
+	set_foreign_rel_properties(joinrel, rel1, rel2);
+
+	/*
+	 * Produce partition-wise joinrel's targetlist by translating the parent
+	 * joinrel's targetlist. This will also include the required placeholder
+	 * Vars.
+	 */
+	joinrel->reltarget = copy_pathtarget(parent_joinrel->reltarget);
+	tmp_exprs = joinrel->reltarget->exprs;
+	joinrel->reltarget->exprs = (List *) adjust_partitionrel_attrs(root, (Node *) tmp_exprs,
+															join_aris);
+
+	/*
+	 * Lateral relids directly referred in this relation will be same as that
+	 * of the parent relation.
+	 */
+	joinrel->direct_lateral_relids = (Relids) bms_copy(parent_joinrel->direct_lateral_relids);
+	joinrel->lateral_relids = (Relids) bms_copy(parent_joinrel->lateral_relids);
+
+	joinrel->joininfo = (List *) adjust_partitionrel_attrs(root,
+											 (Node *) parent_joinrel->joininfo,
+													join_aris);
+
+	/*
+	 * If the parent joinrel has pending equivalence classes, so does the
+	 * child.
+	 */
+	joinrel->has_eclass_joins = parent_joinrel->has_eclass_joins;
+
+	/*
+	 * Set estimates of the joinrel's size.
+	 */
+	set_joinrel_size_estimates(root, joinrel, rel1, rel2, sjinfo,
+							   restrictlist);
+
+	/* Child joinrel is parallel safe if parent is parallel safe. */
+	joinrel->consider_parallel = parent_joinrel->consider_parallel;
+
+	return joinrel;
+}
+
 /*
  * min_join_parameterization
  *
  * Determine the minimum possible parameterization of a joinrel, that is, the
  * set of other rels it contains LATERAL references to.  We save this value in
  * the join's RelOptInfo.  This function is split out of build_join_rel()
  * because join_is_legal() needs the value to check a prospective join.
  */
 Relids
 min_join_parameterization(PlannerInfo *root,
@@ -1313,10 +1435,196 @@ get_appendrel_parampathinfo(RelOptInfo *appendrel, Relids required_outer)
 
 	/* Else build the ParamPathInfo */
 	ppi = makeNode(ParamPathInfo);
 	ppi->ppi_req_outer = required_outer;
 	ppi->ppi_rows = 0;
 	ppi->ppi_clauses = NIL;
 	appendrel->ppilist = lappend(appendrel->ppilist, ppi);
 
 	return ppi;
 }
+
+/*
+ * build_partition_info
+ *
+ * Retrieves partitioning information for given relation.
+ *
+ * The function also builds the RelOptInfos of the partitions recursively.
+ * TODO: complete the prologue.
+ */
+static PartitionOptInfo *
+build_partition_info(PlannerInfo *root, RelOptInfo *rel)
+{
+	Relation	relation;
+	PartitionKey	part_key;
+	PartitionDesc	part_desc;
+	PartitionOptInfo *part_info;
+	RangeTblEntry  *rte;
+	ListCell	*lc;
+	int		num_pkexprs;
+	int		cnt_pke;
+	int		nparts;
+	int		cnt_parts;
+	Expr   *pkexpr;
+
+	/* The given relation should be simple relation. */
+	Assert(rel->reloptkind == RELOPT_BASEREL ||
+		   rel->reloptkind == RELOPT_OTHER_MEMBER_REL);
+	Assert(rel->relid != 0);
+
+	rte = root->simple_rte_array[rel->relid];
+	Assert(rte);
+
+	/*
+	 * If it's not parent of the inheritance hierarchy, it can not be
+	 * partiioned relation.
+	 */
+	if (!rte->inh)
+		return NULL;
+
+	/*
+	 * We need not lock the relation since it was already locked, either
+	 * by the rewriter or when expand_inherited_rtentry() added it to
+	 * the query's rangetable.
+	 */
+	relation = heap_open(rte->relid, NoLock);
+	part_desc = RelationGetPartitionDesc(relation);
+	part_key = RelationGetPartitionKey(relation);
+
+	/* Nothing to do for an unpartitioned relation. */
+	if (!part_desc || !part_key)
+	{
+		heap_close(relation, NoLock);
+		return NULL;
+	}
+
+	part_info = makeNode(PartitionOptInfo);
+
+	/* Store partition descriptor information. */
+	nparts = part_info->nparts = part_desc->nparts;
+	/* TODO: Should we copy the contents of the these arrays? */
+	part_info->lists = (PartitionListInfo **) palloc(sizeof(PartitionListInfo *) * nparts);
+	part_info->ranges = (PartitionRangeInfo **) palloc(sizeof(PartitionRangeInfo *) * nparts);
+
+	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	{
+		part_info->lists[cnt_parts] = part_desc->parts[cnt_parts]->list;
+		part_info->ranges[cnt_parts] = part_desc->parts[cnt_parts]->range;
+	}
+
+	/* Store partition key information. */
+	part_info->strategy = part_key->strategy;
+	part_info->partnatts = part_key->partnatts;
+	num_pkexprs = part_info->partnatts;
+	/* TODO: Should we copy the contents of these arrays? */
+	part_info->partopfamily = part_key->partopfamily;
+	part_info->partopcintype = part_key->partopcintype;
+	part_info->partsupfunc = part_key->partsupfunc;
+	part_info->tcinfo = part_key->tcinfo;
+
+	/* Store partition keys as single elements lists. */
+	part_info->partexprs = (List **) palloc(sizeof(List *) * num_pkexprs);
+	lc = list_head(part_key->partexprs);
+	for (cnt_pke = 0; cnt_pke < num_pkexprs; cnt_pke++)
+	{
+		AttrNumber attno = part_key->partattrs[cnt_pke];
+
+		if (attno != InvalidAttrNumber)
+		{
+			/* Single column partition key is stored as a Var node. */
+			Form_pg_attribute att_tup;
+
+			if (attno < 0)
+				att_tup = SystemAttributeDefinition(attno,
+												relation->rd_rel->relhasoids);
+			else
+				att_tup = relation->rd_att->attrs[attno - 1];
+
+			pkexpr = (Expr *) makeVar(rel->relid, attno, att_tup->atttypid,
+									  att_tup->atttypmod,
+									  att_tup->attcollation, 0);
+		}
+		else
+		{
+			if (!lc)
+				elog(ERROR, "wrong number of partition key expressions");
+			pkexpr = (Expr *) copyObject(lfirst(lc));
+			ChangeVarNodes((Node *) pkexpr, 1, rel->relid, 0);
+			lc = lnext(lc);
+		}
+
+		part_info->partexprs[cnt_pke] = list_make1(pkexpr);
+	}
+
+	/* Find RelOptInfo of the partitions. */
+	part_info->part_rels = (RelOptInfo **) palloc0(sizeof(RelOptInfo *) *
+												   nparts);
+	foreach(lc, root->append_rel_list)
+	{
+		AppendRelInfo *appinfo = (AppendRelInfo *) lfirst(lc);
+		int			childRTindex = appinfo->child_relid;
+		RangeTblEntry *childRTE = root->simple_rte_array[childRTindex];
+		RelOptInfo *childrel = root->simple_rel_array[childRTindex];
+
+		/* append_rel_list contains all append rels; ignore others */
+		if (appinfo->parent_relid != rel->relid)
+			continue;
+
+		/* If we haven't created a RelOptInfo aleady, create one. */
+		if (!childrel)
+			childrel = build_simple_rel(root, childRTindex,
+									RELOPT_OTHER_MEMBER_REL);
+
+		/* Save the parent relids for parameterized path handling. */
+		childrel->parent_relids = bms_copy(rel->relids);
+
+		/*
+		 * OIDs of the partitions are arranged to match the partition bounds or
+		 * list in corresponding arrays. Arrange RelOptInfo's of partitions in
+		 * the same fashion.
+		 */
+		for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+		{
+			if (part_desc->parts[cnt_parts]->oid == childRTE->relid)
+			{
+				/* Every partition can be seen only once. */
+				Assert(!part_info->part_rels[cnt_parts]);
+				part_info->part_rels[cnt_parts] = childrel;
+
+				break;
+			}
+		}
+	}
+
+	heap_close(relation, NoLock);
+
+	/* We must have found RelOptInfos of all the partitions. */
+	for (cnt_parts = 0; cnt_parts < part_info->nparts; cnt_parts++)
+		Assert(part_info->part_rels[cnt_parts]);
+
+	return part_info;
+}
+
+/*
+ * Adds given join relation to the joinrel list and also to the hashtable if
+ * there is one.
+ */
+void
+add_joinrel_to_list(PlannerInfo *root, RelOptInfo *joinrel)
+{
+	/* GEQO requires us to append the new joinrel to the end of the list! */
+	root->join_rel_list = lappend(root->join_rel_list, joinrel);
+
+	/* store it into the auxiliary hashtable if there is one. */
+	if (root->join_rel_hash)
+	{
+		JoinHashEntry *hentry;
+		bool		found;
+
+		hentry = (JoinHashEntry *) hash_search(root->join_rel_hash,
+											   &(joinrel->relids),
+											   HASH_ENTER,
+											   &found);
+		Assert(!found);
+		hentry->join_rel = joinrel;
+	}
+}
diff --git a/src/backend/utils/adt/selfuncs.c b/src/backend/utils/adt/selfuncs.c
index 56943f2..16b2eac 100644
--- a/src/backend/utils/adt/selfuncs.c
+++ b/src/backend/utils/adt/selfuncs.c
@@ -3405,21 +3405,23 @@ estimate_num_groups(PlannerInfo *root, List *groupExprs, double input_rows,
 			else
 			{
 				/* not time to process varinfo2 yet */
 				newvarinfos = lcons(varinfo2, newvarinfos);
 			}
 		}
 
 		/*
 		 * Sanity check --- don't divide by zero if empty relation.
 		 */
-		Assert(rel->reloptkind == RELOPT_BASEREL);
+		Assert(rel->reloptkind == RELOPT_BASEREL ||
+			   rel->reloptkind == RELOPT_OTHER_MEMBER_REL);
+
 		if (rel->tuples > 0)
 		{
 			/*
 			 * Clamp to size of rel, or size of rel / 10 if multiple Vars. The
 			 * fudge factor is because the Vars are probably correlated but we
 			 * don't know by how much.  We should never clamp to less than the
 			 * largest ndistinct value for any of the Vars, though, since
 			 * there will surely be at least that many groups.
 			 */
 			double		clamp = rel->tuples;
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index c5178f7..3412eae 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -870,20 +870,29 @@ static struct config_bool ConfigureNamesBool[] =
 	},
 	{
 		{"enable_hashjoin", PGC_USERSET, QUERY_TUNING_METHOD,
 			gettext_noop("Enables the planner's use of hash join plans."),
 			NULL
 		},
 		&enable_hashjoin,
 		true,
 		NULL, NULL, NULL
 	},
+	{
+		{"enable_partition_wise_join", PGC_USERSET, QUERY_TUNING_METHOD,
+			gettext_noop("Enables partition-wise join."),
+			NULL
+		},
+		&enable_partition_wise_join,
+		true,
+		NULL, NULL, NULL
+	},
 
 	{
 		{"geqo", PGC_USERSET, QUERY_TUNING_GEQO,
 			gettext_noop("Enables genetic query optimization."),
 			gettext_noop("This algorithm attempts to do planning without "
 						 "exhaustive searching.")
 		},
 		&enable_geqo,
 		true,
 		NULL, NULL, NULL
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 14fd29e..8b928a3 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -12,22 +12,88 @@
  */
 #ifndef PARTITION_H
 #define PARTITION_H
 
 #include "fmgr.h"
 #include "executor/tuptable.h"
 #include "nodes/execnodes.h"
 #include "parser/parse_node.h"
 #include "utils/relcache.h"
 
+/* Type and collation information for partition key columns */
+typedef struct KeyTypeCollInfo
+{
+	Oid		*typid;
+	int32	*typmod;
+	int16	*typlen;
+	bool	*typbyval;
+	char	*typalign;
+	Oid		*typcoll;
+} KeyTypeCollInfo;
+
+/*
+ * Partition key information
+ */
+typedef struct PartitionKeyData
+{
+	char		strategy;		/* partition strategy */
+	int16		partnatts;		/* number of partition attributes */
+	AttrNumber *partattrs;		/* partition attnums */
+	Oid		   *partopfamily;	/* OIDs of operator families */
+	Oid		   *partopcintype;	/* OIDs of opclass declared input data types */
+	FmgrInfo   *partsupfunc;	/* lookup info for support funcs */
+	List	   *partexprs;		/* partition key expressions, if any */
+	char	  **partcolnames;	/* partition key column names */
+	KeyTypeCollInfo *tcinfo;	/* type and collation info (all columns) */
+} PartitionKeyData;
+
 typedef struct PartitionKeyData *PartitionKey;
 
+/* Internal representation of a list partition bound */
+typedef struct PartitionListInfo
+{
+	int		nvalues;	/* number of values in the following array */
+	Datum  *values;		/* values contained in the list */
+	bool   *nulls;
+} PartitionListInfo;
+
+/*
+ * TODO: the patch by Amit L named this structure as RangeBound, but that
+ * conflicts with a structure with the same name in rangetypes.h. Hence renamed
+ * it here. It should be changed to whatever Amit L uses in the next set of
+ * patches.
+ */
+/* Internal representation of a range partition bound */
+typedef struct PartitionRangeBound
+{
+	Datum		*val;			/* the bound value, if any */
+	bool		infinite;		/* bound is +/- infinity */
+	bool		inclusive;		/* bound is inclusive (vs exclusive) */
+	bool		lower;			/* this is the lower (vs upper) bound */
+} PartitionRangeBound;
+
+typedef struct PartitionRangeInfo
+{
+	PartitionRangeBound	*lower;
+	PartitionRangeBound	*upper;
+} PartitionRangeInfo;
+
+/*
+ * Information about a single partition
+ */
+typedef struct PartitionInfoData
+{
+	Oid						oid;		/* partition OID */
+	PartitionListInfo	   *list;		/* list partition info */
+	PartitionRangeInfo	   *range;		/* range partition info */
+} PartitionInfoData;
+
 /*
  * Information about partitions of a partitioned table.
  */
 typedef struct PartitionInfoData *PartitionInfo;
 typedef struct PartitionDescData
 {
 	int		nparts;			/* Number of partitions */
 	PartitionInfo *parts;	/* Array of PartitionInfoData pointers */
 } PartitionDescData;
 
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index bb62112..4bb5966 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -218,20 +218,21 @@ typedef enum NodeTag
 	T_DomainConstraintState,
 
 	/*
 	 * TAGS FOR PLANNER NODES (relation.h)
 	 */
 	T_PlannerInfo = 500,
 	T_PlannerGlobal,
 	T_RelOptInfo,
 	T_IndexOptInfo,
 	T_ForeignKeyOptInfo,
+	T_PartitionOptInfo,
 	T_ParamPathInfo,
 	T_Path,
 	T_IndexPath,
 	T_BitmapHeapPath,
 	T_BitmapAndPath,
 	T_BitmapOrPath,
 	T_TidPath,
 	T_SubqueryScanPath,
 	T_ForeignPath,
 	T_CustomPath,
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 2709cc7..fae6c1a 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -12,20 +12,21 @@
  *-------------------------------------------------------------------------
  */
 #ifndef RELATION_H
 #define RELATION_H
 
 #include "access/sdir.h"
 #include "lib/stringinfo.h"
 #include "nodes/params.h"
 #include "nodes/parsenodes.h"
 #include "storage/block.h"
+#include "catalog/partition.h"
 
 
 /*
  * Relids
  *		Set of relation identifiers (indexes into the rangetable).
  */
 typedef Bitmapset *Relids;
 
 /*
  * When looking for a "cheapest path", this enum specifies whether we want
@@ -345,20 +346,26 @@ typedef struct PlannerInfo
  * is present in the query join tree but the members are not.  The member
  * RTEs and otherrels are used to plan the scans of the individual tables or
  * subqueries of the append set; then the parent baserel is given Append
  * and/or MergeAppend paths comprising the best paths for the individual
  * member rels.  (See comments for AppendRelInfo for more information.)
  *
  * At one time we also made otherrels to represent join RTEs, for use in
  * handling join alias Vars.  Currently this is not needed because all join
  * alias Vars are expanded to non-aliased form during preprocess_expression.
  *
+ * We also have relations representing pair-wise joins between partitions of
+ * partitioned tables. These relations are not added to join_rel_level lists
+ * as they are not joined directly by the dynamic programming algorithm.
+ * Adding these two join_rel_level list also means that top level list has more
+ * than one join relation, which is symantically incorrect.
+ *
  * There is also a RelOptKind for "upper" relations, which are RelOptInfos
  * that describe post-scan/join processing steps, such as aggregation.
  * Many of the fields in these RelOptInfos are meaningless, but their Path
  * fields always hold Paths showing ways to do that processing step.
  *
  * Lastly, there is a RelOptKind for "dead" relations, which are base rels
  * that we have proven we don't need to join after all.
  *
  * Parts of this data structure are specific to various scan and join
  * mechanisms.  It didn't seem worth creating new node types for them.
@@ -464,24 +471,31 @@ typedef struct PlannerInfo
  * We store baserestrictcost in the RelOptInfo (for base relations) because
  * we know we will need it at least once (to price the sequential scan)
  * and may need it multiple times to price index scans.
  *----------
  */
 typedef enum RelOptKind
 {
 	RELOPT_BASEREL,
 	RELOPT_JOINREL,
 	RELOPT_OTHER_MEMBER_REL,
+	RELOPT_OTHER_JOINREL,
 	RELOPT_UPPER_REL,
 	RELOPT_DEADREL
 } RelOptKind;
 
+#define IS_OTHER_REL(reloptkind) \
+	((reloptkind) == RELOPT_OTHER_MEMBER_REL || \
+	 (reloptkind) == RELOPT_OTHER_JOINREL)
+
+typedef struct PartitionOptInfo PartitionOptInfo;
+
 typedef struct RelOptInfo
 {
 	NodeTag		type;
 
 	RelOptKind	reloptkind;
 
 	/* all relations included in this RelOptInfo */
 	Relids		relids;			/* set of base relids (rangetable indexes) */
 
 	/* size estimates generated by planner */
@@ -535,20 +549,29 @@ typedef struct RelOptInfo
 	struct FdwRoutine *fdwroutine;
 	void	   *fdw_private;
 
 	/* used by various scans and joins: */
 	List	   *baserestrictinfo;		/* RestrictInfo structures (if base
 										 * rel) */
 	QualCost	baserestrictcost;		/* cost of evaluating the above */
 	List	   *joininfo;		/* RestrictInfo structures for join clauses
 								 * involving this rel */
 	bool		has_eclass_joins;		/* T means joininfo is incomplete */
+
+	/* For partitioned relations, joins or base relations. NULL otherwise. */
+	/*
+	 * TODO: Notice recursive usage of RelOptInfo.
+	 */
+	PartitionOptInfo	*part_info;
+
+	/* Set only for "other" base or join relations. */
+	Relids		parent_relids;
 } RelOptInfo;
 
 /*
  * IndexOptInfo
  *		Per-index information for planning/optimization
  *
  *		indexkeys[], indexcollations[], opfamily[], and opcintype[]
  *		each have ncolumns entries.
  *
  *		sortopfamily[], reverse_sort[], and nulls_first[] likewise have
@@ -649,20 +672,45 @@ typedef struct ForeignKeyOptInfo
 	/* Derived info about whether FK's equality conditions match the query: */
 	int			nmatched_ec;	/* # of FK cols matched by ECs */
 	int			nmatched_rcols; /* # of FK cols matched by non-EC rinfos */
 	int			nmatched_ri;	/* total # of non-EC rinfos matched to FK */
 	/* Pointer to eclass matching each column's condition, if there is one */
 	struct EquivalenceClass *eclass[INDEX_MAX_KEYS];
 	/* List of non-EC RestrictInfos matching each column's condition */
 	List	   *rinfos[INDEX_MAX_KEYS];
 } ForeignKeyOptInfo;
 
+/*
+ * PartitionOptInfo
+ * 		Partitioning information for planning/optimization
+ *
+ * TODO: complete the comment
+ */
+typedef struct PartitionOptInfo
+{
+	NodeTag		type;
+
+	/* Information about partitions */
+	int			nparts;		/* number of partitions */
+	RelOptInfo			  **part_rels;	/* RelOptInfos of partitions */
+	PartitionListInfo	  **lists;		/* list bounds */
+	PartitionRangeInfo	  **ranges;	/* range lower bounds */
+
+	/* Information about partition keys */
+	char		strategy;		/* partition strategy */
+	int16		partnatts;		/* number of partition attributes */
+	List	  **partexprs;		/* partition key expressions. */
+	Oid		   *partopfamily;	/* OIDs of operator families */
+	Oid		   *partopcintype;	/* OIDs of opclass declared input data types */
+	FmgrInfo   *partsupfunc;	/* lookup info for support funcs */
+	KeyTypeCollInfo *tcinfo;	/* type and collation info (all columns) */
+} PartitionOptInfo;
 
 /*
  * EquivalenceClasses
  *
  * Whenever we can determine that a mergejoinable equality clause A = B is
  * not delayed by any outer join, we create an EquivalenceClass containing
  * the expressions A and B to record this knowledge.  If we later find another
  * equivalence B = C, we add C to the existing EquivalenceClass; this may
  * require merging two existing EquivalenceClasses.  At the end of the qual
  * distribution process, we have sets of values that are known all transitively
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 2a4df2f..1069726 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -59,20 +59,21 @@ extern bool enable_seqscan;
 extern bool enable_indexscan;
 extern bool enable_indexonlyscan;
 extern bool enable_bitmapscan;
 extern bool enable_tidscan;
 extern bool enable_sort;
 extern bool enable_hashagg;
 extern bool enable_nestloop;
 extern bool enable_material;
 extern bool enable_mergejoin;
 extern bool enable_hashjoin;
+extern bool enable_partition_wise_join;
 extern int	constraint_exclusion;
 
 extern double clamp_row_est(double nrows);
 extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
 					double index_pages, PlannerInfo *root);
 extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 			 ParamPathInfo *param_info);
 extern void cost_samplescan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 				ParamPathInfo *param_info);
 extern void cost_index(IndexPath *path, PlannerInfo *root,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 71d9154..5a4b054 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -222,20 +222,22 @@ extern ModifyTablePath *create_modifytable_path(PlannerInfo *root,
 						List *rowMarks, OnConflictExpr *onconflict,
 						int epqParam);
 extern LimitPath *create_limit_path(PlannerInfo *root, RelOptInfo *rel,
 				  Path *subpath,
 				  Node *limitOffset, Node *limitCount,
 				  int64 offset_est, int64 count_est);
 
 extern Path *reparameterize_path(PlannerInfo *root, Path *path,
 					Relids required_outer,
 					double loop_count);
+extern Path *reparameterize_path_for_child(PlannerInfo *root, Path *path,
+					RelOptInfo *child_rel);
 
 /*
  * prototypes for relnode.c
  */
 extern void setup_simple_rel_arrays(PlannerInfo *root);
 extern RelOptInfo *build_simple_rel(PlannerInfo *root, int relid,
 				 RelOptKind reloptkind);
 extern RelOptInfo *find_base_rel(PlannerInfo *root, int relid);
 extern RelOptInfo *find_join_rel(PlannerInfo *root, Relids relids);
 extern RelOptInfo *build_join_rel(PlannerInfo *root,
@@ -260,12 +262,20 @@ extern ParamPathInfo *get_baserel_parampathinfo(PlannerInfo *root,
 						  Relids required_outer);
 extern ParamPathInfo *get_joinrel_parampathinfo(PlannerInfo *root,
 						  RelOptInfo *joinrel,
 						  Path *outer_path,
 						  Path *inner_path,
 						  SpecialJoinInfo *sjinfo,
 						  Relids required_outer,
 						  List **restrict_clauses);
 extern ParamPathInfo *get_appendrel_parampathinfo(RelOptInfo *appendrel,
 							Relids required_outer);
+extern RelOptInfo *make_joinrel(PlannerInfo *root, RelOptKind reloptkind,
+						Relids joinrelids);
+extern RelOptInfo *fill_partition_join_rel(PlannerInfo *root, RelOptInfo *rel1,
+								  RelOptInfo *rel2, RelOptInfo *joinrel,
+								  RelOptInfo *parent_joinrel,
+								  SpecialJoinInfo *sjinfo,
+								  List *restrictlist, List *join_aris);
+extern void add_joinrel_to_list(PlannerInfo *root, RelOptInfo *joinrel);
 
 #endif   /* PATHNODE_H */
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 44abe83..152db2b 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -212,11 +212,14 @@ extern List *make_inner_pathkeys_for_merge(PlannerInfo *root,
 							  List *mergeclauses,
 							  List *outer_pathkeys);
 extern List *truncate_useless_pathkeys(PlannerInfo *root,
 						  RelOptInfo *rel,
 						  List *pathkeys);
 extern bool has_useful_pathkeys(PlannerInfo *root, RelOptInfo *rel);
 extern PathKey *make_canonical_pathkey(PlannerInfo *root,
 					   EquivalenceClass *eclass, Oid opfamily,
 					   int strategy, bool nulls_first);
 
+/* TODO: need a better place to save this function signature. */
+extern Relids adjust_partition_relids(Relids relids, List *append_rel_infos);
+
 #endif   /* PATHS_H */
diff --git a/src/include/optimizer/prep.h b/src/include/optimizer/prep.h
index fb35b68..7b149c3 100644
--- a/src/include/optimizer/prep.h
+++ b/src/include/optimizer/prep.h
@@ -21,20 +21,23 @@
 /*
  * prototypes for prepjointree.c
  */
 extern void pull_up_sublinks(PlannerInfo *root);
 extern void inline_set_returning_functions(PlannerInfo *root);
 extern void pull_up_subqueries(PlannerInfo *root);
 extern void flatten_simple_union_all(PlannerInfo *root);
 extern void reduce_outer_joins(PlannerInfo *root);
 extern Relids get_relids_in_jointree(Node *jtnode, bool include_joins);
 extern Relids get_relids_for_join(PlannerInfo *root, int joinrelid);
+extern Node *adjust_partitionrel_attrs(PlannerInfo *root, Node *node,
+									   List *append_rel_infos);
+extern List *find_appendrelinfos_by_relids(PlannerInfo *root, Relids relids);
 
 /*
  * prototypes for prepqual.c
  */
 extern Node *negate_clause(Node *node);
 extern Expr *canonicalize_qual(Expr *qual);
 
 /*
  * prototypes for prepsecurity.c
  */
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
new file mode 100644
index 0000000..307076e
--- /dev/null
+++ b/src/test/regress/expected/partition_join.out
@@ -0,0 +1,6838 @@
+--
+-- PARTITION_JOIN
+-- Test partition-wise join between partitioned tables
+--
+--
+-- partitioned by a single column
+--
+CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES START (0) END (250);
+CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES START (500) END (600);
+CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES START (250) END (500);
+INSERT INTO prt1 SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1 AS SELECT * FROM prt1;
+CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES START (0) END (250);
+CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES START (250) END (500);
+CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES START (500) END (600);
+INSERT INTO prt2 SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+CREATE TABLE uprt2 AS SELECT * FROM prt2;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on public.prt2_p1 t2
+                     Output: t2.b, t2.c
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+               Hash Cond: (t2_1.b = t1_2.a)
+               ->  Seq Scan on public.prt2_p2 t2_1
+                     Output: t2_1.b, t2_1.c
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+               Hash Cond: (t2_2.b = t1_1.a)
+               ->  Seq Scan on public.prt2_p3 t2_2
+                     Output: t2_2.b, t2_2.c
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0150 | 150 | 0150
+ 300 | 0300 | 300 | 0300
+ 450 | 0450 | 450 | 0450
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on public.prt2_p1 t2
+                     Output: t2.b, t2.c
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+               Hash Cond: (t2_1.b = t1_2.a)
+               ->  Seq Scan on public.prt2_p2 t2_1
+                     Output: t2_1.b, t2_1.c
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+               Hash Cond: (t2_2.b = t1_1.a)
+               ->  Seq Scan on public.prt2_p3 t2_2
+                     Output: t2_2.b, t2_2.c
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+  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 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: (t1.a = t2.b)
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t2.b, t2.c
+                           ->  Seq Scan on public.prt2_p1 t2
+                                 Output: t2.b, t2.c
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Hash Cond: (t1_2.a = t2_1.b)
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c
+                           ->  Seq Scan on public.prt2_p2 t2_1
+                                 Output: t2_1.b, t2_1.c
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Hash Cond: (t1_1.a = t2_2.b)
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c
+                           ->  Seq Scan on public.prt2_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+                                 Filter: ((t2_2.b % 25) = 0)
+(36 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+  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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+   Sort Key: prt1_p1.a, prt2_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+               Hash Cond: (prt1_p1.a = prt2_p1.b)
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c
+                     Filter: ((prt1_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p1.b, prt2_p1.c
+                     ->  Seq Scan on public.prt2_p1
+                           Output: prt2_p1.b, prt2_p1.c
+                           Filter: ((prt2_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt1_p2.a = prt2_p2.b)
+               ->  Seq Scan on public.prt1_p2
+                     Output: prt1_p2.a, prt1_p2.c
+                     Filter: ((prt1_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p2.b, prt2_p2.c
+                     ->  Seq Scan on public.prt2_p2
+                           Output: prt2_p2.b, prt2_p2.c
+                           Filter: ((prt2_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c
+               Hash Cond: (prt1_p3.a = prt2_p3.b)
+               ->  Seq Scan on public.prt1_p3
+                     Output: prt1_p3.a, prt1_p3.c
+                     Filter: ((prt1_p3.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p3.b, prt2_p3.c
+                     ->  Seq Scan on public.prt2_p3
+                           Output: prt2_p3.b, prt2_p3.c
+                           Filter: ((prt2_p3.b % 25) = 0)
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 t2 WHERE t2.b % 25 = 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)
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+   Sort Key: prt1_p1.a, prt2_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+               Hash Cond: (prt1_p1.a = prt2_p1.b)
+               Filter: (((50) = prt1_p1.b) OR ((75) = prt2_p1.b))
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c, prt1_p1.b, 50
+                     Filter: ((prt1_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p1.b, prt2_p1.c, (75)
+                     ->  Seq Scan on public.prt2_p1
+                           Output: prt2_p1.b, prt2_p1.c, 75
+                           Filter: ((prt2_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt1_p2.a = prt2_p2.b)
+               Filter: (((50) = prt1_p2.b) OR ((75) = prt2_p2.b))
+               ->  Seq Scan on public.prt1_p2
+                     Output: prt1_p2.a, prt1_p2.c, prt1_p2.b, 50
+                     Filter: ((prt1_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p2.b, prt2_p2.c, (75)
+                     ->  Seq Scan on public.prt2_p2
+                           Output: prt2_p2.b, prt2_p2.c, 75
+                           Filter: ((prt2_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c
+               Hash Cond: (prt1_p3.a = prt2_p3.b)
+               Filter: (((50) = prt1_p3.b) OR ((75) = prt2_p3.b))
+               ->  Seq Scan on public.prt1_p3
+                     Output: prt1_p3.a, prt1_p3.c, prt1_p3.b, 50
+                     Filter: ((prt1_p3.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p3.b, prt2_p3.c, (75)
+                     ->  Seq Scan on public.prt2_p3
+                           Output: prt2_p3.b, prt2_p3.c, 75
+                           Filter: ((prt2_p3.b % 25) = 0)
+(40 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+ a  |  c   | b  |  c   
+----+------+----+------
+ 50 | 0050 |    | 
+    |      | 75 | 0075
+(2 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+ a  |  c   | b  |  c   
+----+------+----+------
+ 50 | 0050 |    | 
+    |      | 75 | 0075
+(2 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, (25), prt2_p1.b, prt2_p1.c, (50)
+   Sort Key: prt1_p1.a, prt2_p1.b
+   ->  Result
+         Output: prt1_p1.a, prt1_p1.c, (25), prt2_p1.b, prt2_p1.c, (50)
+         ->  Append
+               ->  Hash Full Join
+                     Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, (25), (50)
+                     Hash Cond: (prt1_p1.a = prt2_p1.b)
+                     ->  Seq Scan on public.prt1_p1
+                           Output: prt1_p1.a, prt1_p1.c, 25
+                           Filter: ((prt1_p1.a % 25) = 0)
+                     ->  Hash
+                           Output: prt2_p1.b, prt2_p1.c, (50)
+                           ->  Seq Scan on public.prt2_p1
+                                 Output: prt2_p1.b, prt2_p1.c, 50
+                                 Filter: ((prt2_p1.b % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c, (25), (50)
+                     Hash Cond: (prt1_p2.a = prt2_p2.b)
+                     ->  Seq Scan on public.prt1_p2
+                           Output: prt1_p2.a, prt1_p2.c, 25
+                           Filter: ((prt1_p2.a % 25) = 0)
+                     ->  Hash
+                           Output: prt2_p2.b, prt2_p2.c, (50)
+                           ->  Seq Scan on public.prt2_p2
+                                 Output: prt2_p2.b, prt2_p2.c, 50
+                                 Filter: ((prt2_p2.b % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c, (25), (50)
+                     Hash Cond: (prt1_p3.a = prt2_p3.b)
+                     ->  Seq Scan on public.prt1_p3
+                           Output: prt1_p3.a, prt1_p3.c, 25
+                           Filter: ((prt1_p3.a % 25) = 0)
+                     ->  Hash
+                           Output: prt2_p3.b, prt2_p3.c, (50)
+                           ->  Seq Scan on public.prt2_p3
+                                 Output: prt2_p3.b, prt2_p3.c, 50
+                                 Filter: ((prt2_p3.b % 25) = 0)
+(39 rows)
+
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  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)
+
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM uprt2 WHERE uprt2.b % 25 = 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)
+
+-- Join with pruned partitions from joining relations
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on public.prt2_p2 t2
+                     Output: t2.b, t2.c
+                     Filter: (t2.b > 250)
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.prt1_p2 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a < 450) AND ((t1.a % 25) = 0))
+(15 rows)
+
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+(1 row)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a < 450 AND t2.b > 250 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+(1 row)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                    QUERY PLAN                                    
+----------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, b, c
+   Sort Key: prt1_p1.a, b
+   ->  Append
+         ->  Nested Loop Left Join
+               Output: prt1_p1.a, prt1_p1.c, b, c
+               Join Filter: (prt1_p1.a = b)
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c
+                     Filter: ((prt1_p1.a < 450) AND ((prt1_p1.a % 25) = 0))
+               ->  Result
+                     Output: b, c
+                     One-Time Filter: false
+         ->  Hash Right Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt2_p2.b = prt1_p2.a)
+               ->  Seq Scan on public.prt2_p2
+                     Output: prt2_p2.b, prt2_p2.c
+                     Filter: (prt2_p2.b > 250)
+               ->  Hash
+                     Output: prt1_p2.a, prt1_p2.c
+                     ->  Seq Scan on public.prt1_p2
+                           Output: prt1_p2.a, prt1_p2.c
+                           Filter: ((prt1_p2.a < 450) AND ((prt1_p2.a % 25) = 0))
+(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.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.a % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 5) and (b 4) is considered for partition-wise join.
+                                       QUERY PLAN                                       
+----------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+   Sort Key: prt1_p2.a, prt2_p2.b
+   ->  Result
+         Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: prt2_p2.b, prt2_p2.c, prt1_p2.a, prt1_p2.c
+                     Hash Cond: (prt1_p2.a = prt2_p2.b)
+                     ->  Seq Scan on public.prt1_p2
+                           Output: prt1_p2.a, prt1_p2.c
+                           Filter: (prt1_p2.a < 450)
+                     ->  Hash
+                           Output: prt2_p2.b, prt2_p2.c
+                           ->  Seq Scan on public.prt2_p2
+                                 Output: prt2_p2.b, prt2_p2.c
+                                 Filter: ((prt2_p2.b > 250) AND ((prt2_p2.a % 25) = 0))
+               ->  Nested Loop Left Join
+                     Output: prt2_p3.b, prt2_p3.c, a, c
+                     Join Filter: (a = prt2_p3.b)
+                     ->  Seq Scan on public.prt2_p3
+                           Output: prt2_p3.b, prt2_p3.c
+                           Filter: ((prt2_p3.b > 250) AND ((prt2_p3.a % 25) = 0))
+                     ->  Result
+                           Output: a, c
+                           One-Time Filter: false
+(26 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 5) and (b 4) is considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+     |      | 375 | 0375
+     |      | 450 | 0450
+     |      | 525 | 0525
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+     |      | 375 | 0375
+     |      | 450 | 0450
+     |      | 525 | 0525
+(4 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                    QUERY PLAN                                    
+----------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, b, c
+   Sort Key: prt1_p1.a, b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_p1.a, prt1_p1.c, b, c
+               Hash Cond: (prt1_p1.a = b)
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c
+                     Filter: ((prt1_p1.a < 450) AND ((prt1_p1.a % 25) = 0))
+               ->  Hash
+                     Output: b, c
+                     ->  Result
+                           Output: b, c
+                           One-Time Filter: false
+         ->  Hash Full Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt1_p2.a = prt2_p2.b)
+               ->  Seq Scan on public.prt1_p2
+                     Output: prt1_p2.a, prt1_p2.c
+                     Filter: ((prt1_p2.a < 450) AND ((prt1_p2.a % 25) = 0))
+               ->  Hash
+                     Output: prt2_p2.b, prt2_p2.c
+                     ->  Seq Scan on public.prt2_p2
+                           Output: prt2_p2.b, prt2_p2.c
+                           Filter: ((prt2_p2.b > 250) AND ((prt2_p2.b % 25) = 0))
+         ->  Hash Full Join
+               Output: a, c, prt2_p3.b, prt2_p3.c
+               Hash Cond: (prt2_p3.b = a)
+               ->  Seq Scan on public.prt2_p3
+                     Output: prt2_p3.b, prt2_p3.c
+                     Filter: ((prt2_p3.b > 250) AND ((prt2_p3.b % 25) = 0))
+               ->  Hash
+                     Output: a, c
+                     ->  Result
+                           Output: a, c
+                           One-Time Filter: false
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b 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)
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 1) and (b 3) is considered for partition-wise join.
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.a = t1_3.b)
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_3.b
+                     ->  Seq Scan on public.prt2_p1 t1_3
+                           Output: t1_3.b
+                           Filter: ((t1_3.b % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.a = t1_4.b)
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_4.b
+                     ->  Seq Scan on public.prt2_p2 t1_4
+                           Output: t1_4.b
+                           Filter: ((t1_4.b % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.a = t1_5.b)
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_5.b
+                     ->  Seq Scan on public.prt2_p3 t1_5
+                           Output: t1_5.b
+                           Filter: ((t1_5.b % 25) = 0)
+(37 rows)
+
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 1) and (b 3) is considered for partition-wise join.
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+-- lateral reference
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4 5) is considered for partition-wise join.
+                                             QUERY PLAN                                              
+-----------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   Sort Key: t1.a
+   ->  Result
+         Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+         ->  Append
+               ->  Nested Loop Left Join
+                     Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a))
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.b, t1.c
+                           Filter: ((t1.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+                           Hash Cond: (t3.b = t2.a)
+                           ->  Seq Scan on public.prt2_p1 t3
+                                 Output: t3.a, t3.b
+                           ->  Hash
+                                 Output: t2.a
+                                 ->  Seq Scan on public.prt1_p1 t2
+                                       Output: t2.a
+                                       Filter: (t1.a = t2.a)
+               ->  Nested Loop Left Join
+                     Output: t1_2.a, t1_2.b, t1_2.c, t2_2.a, t3_1.a, (LEAST(t1_2.a, t2_2.a, t3_1.a))
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.b, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2_2.a, t3_1.a, LEAST(t1_2.a, t2_2.a, t3_1.a)
+                           Hash Cond: (t3_1.b = t2_2.a)
+                           ->  Seq Scan on public.prt2_p2 t3_1
+                                 Output: t3_1.a, t3_1.b
+                           ->  Hash
+                                 Output: t2_2.a
+                                 ->  Seq Scan on public.prt1_p2 t2_2
+                                       Output: t2_2.a
+                                       Filter: (t1_2.a = t2_2.a)
+               ->  Nested Loop Left Join
+                     Output: t1_1.a, t1_1.b, t1_1.c, t2_1.a, t3_2.a, (LEAST(t1_1.a, t2_1.a, t3_2.a))
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.b, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2_1.a, t3_2.a, LEAST(t1_1.a, t2_1.a, t3_2.a)
+                           Hash Cond: (t3_2.b = t2_1.a)
+                           ->  Seq Scan on public.prt2_p3 t3_2
+                                 Output: t3_2.a, t3_2.b
+                           ->  Hash
+                                 Output: t2_1.a
+                                 ->  Seq Scan on public.prt1_p3 t2_1
+                                       Output: t2_1.a
+                                       Filter: (t1_1.a = t2_1.a)
+(51 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4 5) is considered for partition-wise join.
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   Sort Key: t1.a
+   ->  Nested Loop Left Join
+         Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+         ->  Append
+               ->  Seq Scan on public.prt1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Seq Scan on public.prt1_p1 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Seq Scan on public.prt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Seq Scan on public.prt1_p2 t1_3
+                     Output: t1_3.a, t1_3.b, t1_3.c
+                     Filter: ((t1_3.a % 25) = 0)
+         ->  Append
+               ->  Hash Join
+                     Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+                     Hash Cond: (t3.b = t2.a)
+                     ->  Seq Scan on public.prt2_p1 t3
+                           Output: t3.a, t3.b
+                     ->  Hash
+                           Output: t2.a
+                           ->  Seq Scan on public.prt1_p1 t2
+                                 Output: t2.a
+                                 Filter: (t1.b = t2.a)
+               ->  Hash Join
+                     Output: t2_2.a, t3_1.a, LEAST(t1.a, t2_2.a, t3_1.a)
+                     Hash Cond: (t3_1.b = t2_2.a)
+                     ->  Seq Scan on public.prt2_p2 t3_1
+                           Output: t3_1.a, t3_1.b
+                     ->  Hash
+                           Output: t2_2.a
+                           ->  Seq Scan on public.prt1_p2 t2_2
+                                 Output: t2_2.a
+                                 Filter: (t1.b = t2_2.a)
+               ->  Hash Join
+                     Output: t2_1.a, t3_2.a, LEAST(t1.a, t2_1.a, t3_2.a)
+                     Hash Cond: (t3_2.b = t2_1.a)
+                     ->  Seq Scan on public.prt2_p3 t3_2
+                           Output: t3_2.a, t3_2.b
+                     ->  Hash
+                           Output: t2_1.a
+                           ->  Seq Scan on public.prt1_p3 t2_1
+                                 Output: t2_1.a
+                                 Filter: (t1.b = t2_1.a)
+(49 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+--
+-- partitioned by expression
+--
+CREATE TABLE prt1_e (a int, b int, c varchar) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES START (0) END (250);
+CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES START (250) END (500);
+CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES START (500) END (600);
+INSERT INTO prt1_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_e AS SELECT * FROM prt1_e;
+CREATE TABLE prt2_e (a int, b int, c varchar) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES START (0) END (250);
+CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES START (250) END (500);
+CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES START (500) END (600);
+INSERT INTO prt2_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_e;
+ANALYZE prt2_e_p1;
+ANALYZE prt2_e_p2;
+ANALYZE prt2_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_e AS SELECT * FROM prt2_e;
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+                                  QUERY PLAN                                  
+------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_e_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_e_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
+               ->  Seq Scan on public.prt2_e_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_e_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+(34 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0150 | 150 | 0150
+ 300 | 0300 | 300 | 0300
+ 450 | 0450 | 450 | 0450
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1, uprt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.a % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+                                  QUERY PLAN                                  
+------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_e_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_e_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
+               ->  Seq Scan on public.prt2_e_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_e_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+  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 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1 LEFT JOIN uprt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 RIGHT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: (((t1.a + t1.b) / 2) = ((t2.b + t2.a) / 2))
+                     ->  Seq Scan on public.prt1_e_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                     ->  Hash
+                           Output: t2.b, t2.c, t2.a
+                           ->  Seq Scan on public.prt2_e_p1 t2
+                                 Output: t2.b, t2.c, t2.a
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: (((t1_1.a + t1_1.b) / 2) = ((t2_1.b + t2_1.a) / 2))
+                     ->  Seq Scan on public.prt1_e_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c, t2_1.a
+                           ->  Seq Scan on public.prt2_e_p2 t2_1
+                                 Output: t2_1.b, t2_1.c, t2_1.a
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: (((t1_2.a + t1_2.b) / 2) = ((t2_2.b + t2_2.a) / 2))
+                     ->  Seq Scan on public.prt1_e_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c, t2_2.a
+                           ->  Seq Scan on public.prt2_e_p3 t2_2
+                                 Output: t2_2.b, t2_2.c, t2_2.a
+                                 Filter: ((t2_2.b % 25) = 0)
+(36 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 RIGHT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+  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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1 RIGHT JOIN uprt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t2.b % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_e WHERE prt2_e.b % 25 = 0) t2 ON ((t1.a + t1.b)/2 = (t2.b + t2.a)/2) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                            QUERY PLAN                                            
+--------------------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_e_p1.a, prt1_e_p1.c, prt2_e_p1.b, prt2_e_p1.c
+   Sort Key: prt1_e_p1.a, prt2_e_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_e_p1.a, prt1_e_p1.c, prt2_e_p1.b, prt2_e_p1.c
+               Hash Cond: (((prt1_e_p1.a + prt1_e_p1.b) / 2) = ((prt2_e_p1.b + prt2_e_p1.a) / 2))
+               ->  Seq Scan on public.prt1_e_p1
+                     Output: prt1_e_p1.a, prt1_e_p1.c, prt1_e_p1.b
+                     Filter: ((prt1_e_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_e_p1.b, prt2_e_p1.c, prt2_e_p1.a
+                     ->  Seq Scan on public.prt2_e_p1
+                           Output: prt2_e_p1.b, prt2_e_p1.c, prt2_e_p1.a
+                           Filter: ((prt2_e_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_e_p2.a, prt1_e_p2.c, prt2_e_p2.b, prt2_e_p2.c
+               Hash Cond: (((prt1_e_p2.a + prt1_e_p2.b) / 2) = ((prt2_e_p2.b + prt2_e_p2.a) / 2))
+               ->  Seq Scan on public.prt1_e_p2
+                     Output: prt1_e_p2.a, prt1_e_p2.c, prt1_e_p2.b
+                     Filter: ((prt1_e_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_e_p2.b, prt2_e_p2.c, prt2_e_p2.a
+                     ->  Seq Scan on public.prt2_e_p2
+                           Output: prt2_e_p2.b, prt2_e_p2.c, prt2_e_p2.a
+                           Filter: ((prt2_e_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_e_p3.a, prt1_e_p3.c, prt2_e_p3.b, prt2_e_p3.c
+               Hash Cond: (((prt1_e_p3.a + prt1_e_p3.b) / 2) = ((prt2_e_p3.b + prt2_e_p3.a) / 2))
+               ->  Seq Scan on public.prt1_e_p3
+                     Output: prt1_e_p3.a, prt1_e_p3.c, prt1_e_p3.b
+                     Filter: ((prt1_e_p3.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_e_p3.b, prt2_e_p3.c, prt2_e_p3.a
+                     ->  Seq Scan on public.prt2_e_p3
+                           Output: prt2_e_p3.b, prt2_e_p3.c, prt2_e_p3.a
+                           Filter: ((prt2_e_p3.b % 25) = 0)
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_e WHERE prt2_e.b % 25 = 0) t2 ON ((t1.a + t1.b)/2 = (t2.b + t2.a)/2) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_e t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_e t2 WHERE t2.b % 25 = 0) t2 ON ((t1.a + t1.b)/2 = (t2.b + t2.a)/2) 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)
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 3) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 3) and (b 1) is considered for partition-wise join.
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on public.prt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = t1.a)
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_2.a)
+                     ->  Seq Scan on public.prt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                           ->  Hash Join
+                                 Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = t1_2.a)
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_1.a)
+                     ->  Seq Scan on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                           ->  Hash Join
+                                 Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = t1_1.a)
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+(57 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 3) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 3) and (b 1) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM uprt1 t1, uprt2 t2, uprt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on public.prt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = t1.a)
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_2.a)
+                     ->  Seq Scan on public.prt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = t1_2.a)
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_1.a)
+                     ->  Seq Scan on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = t1_1.a)
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+(57 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, 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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2 4) is considered for partition-wise join.
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: (((t3.a + t3.b) / 2) = t2.b)
+                     ->  Seq Scan on public.prt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = t1.a)
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: (((t3_1.a + t3_1.b) / 2) = t2_1.b)
+                     ->  Seq Scan on public.prt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = t1_2.a)
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: (((t3_2.a + t3_2.b) / 2) = t2_2.b)
+                     ->  Seq Scan on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = t1_1.a)
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+(57 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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2 4) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |          | 
+ 100 | 0100 |     |      |          | 
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |          | 
+ 250 | 0250 |     |      |          | 
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |          | 
+ 400 | 0400 |     |      |          | 
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |          | 
+ 550 | 0550 |     |      |          | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |          | 
+ 100 | 0100 |     |      |          | 
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |          | 
+ 250 | 0250 |     |      |          | 
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |          | 
+ 400 | 0400 |     |      |          | 
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |          | 
+ 550 | 0550 |     |      |          | 
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Hash Cond: (t2.b = t1.a)
+                     ->  Seq Scan on public.prt2_p1 t2
+                           Output: t2.b, t2.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Hash Cond: (t1.a = ((t3.a + t3.b) / 2))
+                                 ->  Seq Scan on public.prt1_p1 t1
+                                       Output: t1.a, t1.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                     Hash Cond: (t2_1.b = t1_2.a)
+                     ->  Seq Scan on public.prt2_p2 t2_1
+                           Output: t2_1.b, t2_1.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                                 Hash Cond: (t1_2.a = ((t3_1.a + t3_1.b) / 2))
+                                 ->  Seq Scan on public.prt1_p2 t1_2
+                                       Output: t1_2.a, t1_2.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                     Hash Cond: (t2_2.b = t1_1.a)
+                     ->  Seq Scan on public.prt2_p3 t2_2
+                           Output: t2_2.b, t2_2.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                                 Hash Cond: (t1_1.a = ((t3_2.a + t3_2.b) / 2))
+                                 ->  Seq Scan on public.prt1_p3 t1_1
+                                       Output: t1_1.a, t1_1.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: (t1.a = t2.b)
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = ((t3.a + t3.b) / 2))
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Hash Cond: (t1_2.a = t2_1.b)
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = ((t3_1.a + t3_1.b) / 2))
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Hash Cond: (t1_1.a = t2_2.b)
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = ((t3_2.a + t3_2.b) / 2))
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 6) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is considered for partition-wise join.
+                                                  QUERY PLAN                                                   
+---------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, ((prt1_e_p1.a + prt1_e_p1.b)), prt1_e_p1.c
+   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   ->  Result
+         Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, (prt1_e_p1.a + prt1_e_p1.b), prt1_e_p1.c
+         ->  Append
+               ->  Hash Full Join
+                     Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, prt1_e_p1.a, prt1_e_p1.b, prt1_e_p1.c
+                     Hash Cond: (prt1_p1.a = ((prt1_e_p1.a + prt1_e_p1.b) / 2))
+                     ->  Hash Full Join
+                           Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on public.prt1_p1
+                                 Output: prt1_p1.a, prt1_p1.c
+                                 Filter: ((prt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p1.b, prt2_p1.c
+                                 ->  Seq Scan on public.prt2_p1
+                                       Output: prt2_p1.b, prt2_p1.c
+                                       Filter: ((prt2_p1.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p1.a, prt1_e_p1.b, prt1_e_p1.c
+                           ->  Seq Scan on public.prt1_e_p1
+                                 Output: prt1_e_p1.a, prt1_e_p1.b, prt1_e_p1.c
+                                 Filter: ((prt1_e_p1.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c, prt1_e_p2.a, prt1_e_p2.b, prt1_e_p2.c
+                     Hash Cond: (prt1_p2.a = ((prt1_e_p2.a + prt1_e_p2.b) / 2))
+                     ->  Hash Full Join
+                           Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+                           Hash Cond: (prt1_p2.a = prt2_p2.b)
+                           ->  Seq Scan on public.prt1_p2
+                                 Output: prt1_p2.a, prt1_p2.c
+                                 Filter: ((prt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p2.b, prt2_p2.c
+                                 ->  Seq Scan on public.prt2_p2
+                                       Output: prt2_p2.b, prt2_p2.c
+                                       Filter: ((prt2_p2.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p2.a, prt1_e_p2.b, prt1_e_p2.c
+                           ->  Seq Scan on public.prt1_e_p2
+                                 Output: prt1_e_p2.a, prt1_e_p2.b, prt1_e_p2.c
+                                 Filter: ((prt1_e_p2.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c, prt1_e_p3.a, prt1_e_p3.b, prt1_e_p3.c
+                     Hash Cond: (prt1_p3.a = ((prt1_e_p3.a + prt1_e_p3.b) / 2))
+                     ->  Hash Full Join
+                           Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c
+                           Hash Cond: (prt1_p3.a = prt2_p3.b)
+                           ->  Seq Scan on public.prt1_p3
+                                 Output: prt1_p3.a, prt1_p3.c
+                                 Filter: ((prt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p3.b, prt2_p3.c
+                                 ->  Seq Scan on public.prt2_p3
+                                       Output: prt2_p3.b, prt2_p3.c
+                                       Filter: ((prt2_p3.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p3.a, prt1_e_p3.b, prt1_e_p3.c
+                           ->  Seq Scan on public.prt1_e_p3
+                                 Output: prt1_e_p3.a, prt1_e_p3.b, prt1_e_p3.c
+                                 Filter: ((prt1_e_p3.a % 25) = 0)
+(63 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 6) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+     |      |  75 | 0075 |          | 
+     |      | 225 | 0225 |          | 
+     |      | 375 | 0375 |          | 
+     |      | 525 | 0525 |          | 
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM uprt1_e WHERE uprt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+     |      |  75 | 0075 |          | 
+     |      | 225 | 0225 |          | 
+     |      | 375 | 0375 |          | 
+     |      | 525 | 0525 |          | 
+(16 rows)
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, 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.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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;
+NOTICE:  join between relations (b 6) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is considered for partition-wise join.
+                                                      QUERY PLAN                                                      
+----------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, (50), prt2_p1.b, (75), ((prt1_e_p1.a + prt1_e_p1.b)), (50)
+   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   ->  Result
+         Output: prt1_p1.a, (50), prt2_p1.b, (75), (prt1_e_p1.a + prt1_e_p1.b), (50)
+         ->  Append
+               ->  Hash Full Join
+                     Output: prt1_p1.a, prt2_p1.b, prt1_e_p1.a, prt1_e_p1.b, (50), (75), (50)
+                     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
+                           Output: prt1_p1.a, prt2_p1.b, (50), (75)
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on public.prt1_p1
+                                 Output: prt1_p1.a, 50
+                                 Filter: ((prt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p1.b, (75)
+                                 ->  Seq Scan on public.prt2_p1
+                                       Output: prt2_p1.b, 75
+                                       Filter: ((prt2_p1.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p1.a, prt1_e_p1.b, (50)
+                           ->  Seq Scan on public.prt1_e_p1
+                                 Output: prt1_e_p1.a, prt1_e_p1.b, 50
+                                 Filter: ((prt1_e_p1.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p2.a, prt2_p2.b, prt1_e_p2.a, prt1_e_p2.b, (50), (75), (50)
+                     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
+                           Output: prt1_p2.a, prt2_p2.b, (50), (75)
+                           Hash Cond: (prt1_p2.a = prt2_p2.b)
+                           ->  Seq Scan on public.prt1_p2
+                                 Output: prt1_p2.a, 50
+                                 Filter: ((prt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p2.b, (75)
+                                 ->  Seq Scan on public.prt2_p2
+                                       Output: prt2_p2.b, 75
+                                       Filter: ((prt2_p2.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p2.a, prt1_e_p2.b, (50)
+                           ->  Seq Scan on public.prt1_e_p2
+                                 Output: prt1_e_p2.a, prt1_e_p2.b, 50
+                                 Filter: ((prt1_e_p2.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p3.a, prt2_p3.b, prt1_e_p3.a, prt1_e_p3.b, (50), (75), (50)
+                     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
+                           Output: prt1_p3.a, prt2_p3.b, (50), (75)
+                           Hash Cond: (prt1_p3.a = prt2_p3.b)
+                           ->  Seq Scan on public.prt1_p3
+                                 Output: prt1_p3.a, 50
+                                 Filter: ((prt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p3.b, (75)
+                                 ->  Seq Scan on public.prt2_p3
+                                       Output: prt2_p3.b, 75
+                                       Filter: ((prt2_p3.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p3.a, prt1_e_p3.b, (50)
+                           ->  Seq Scan on public.prt1_e_p3
+                                 Output: prt1_e_p3.a, prt1_e_p3.b, 50
+                                 Filter: ((prt1_e_p3.a % 25) = 0)
+(66 rows)
+
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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;
+NOTICE:  join between relations (b 6) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is considered for partition-wise join.
+ a  | phv | b  | phv | ?column? | phv 
+----+-----+----+-----+----------+-----
+ 50 |  50 |    |     |      100 |  50
+    |     | 75 |  75 |          |    
+(2 rows)
+
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM uprt1_e WHERE uprt1_e.a % 25 = 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)
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 4) is considered for partition-wise join.
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.a = t1_3.b)
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_3.b, t2.a, t2.b
+                     ->  Hash Join
+                           Output: t1_3.b, t2.a, t2.b
+                           Hash Cond: (((t2.a + t2.b) / 2) = t1_3.b)
+                           ->  Seq Scan on public.prt1_e_p1 t2
+                                 Output: t2.a, t2.b
+                           ->  Hash
+                                 Output: t1_3.b
+                                 ->  Seq Scan on public.prt2_p1 t1_3
+                                       Output: t1_3.b
+                                       Filter: ((t1_3.b % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.a = t1_4.b)
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_4.b, t2_1.a, t2_1.b
+                     ->  Hash Join
+                           Output: t1_4.b, t2_1.a, t2_1.b
+                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_4.b)
+                           ->  Seq Scan on public.prt1_e_p2 t2_1
+                                 Output: t2_1.a, t2_1.b
+                           ->  Hash
+                                 Output: t1_4.b
+                                 ->  Seq Scan on public.prt2_p2 t1_4
+                                       Output: t1_4.b
+                                       Filter: ((t1_4.b % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.a = t1_5.b)
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_5.b, t2_2.a, t2_2.b
+                     ->  Hash Join
+                           Output: t1_5.b, t2_2.a, t2_2.b
+                           Hash Cond: (((t2_2.a + t2_2.b) / 2) = t1_5.b)
+                           ->  Seq Scan on public.prt1_e_p3 t2_2
+                                 Output: t2_2.a, t2_2.b
+                           ->  Hash
+                                 Output: t1_5.b
+                                 ->  Seq Scan on public.prt2_p3 t1_5
+                                       Output: t1_5.b
+                                       Filter: ((t1_5.b % 25) = 0)
+(58 rows)
+
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 4) is considered for partition-wise join.
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1, uprt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 5) is considered for partition-wise join.
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.a = t1_3.b)
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_3.b, t1_6.a, t1_6.b
+                     ->  Hash Semi Join
+                           Output: t1_3.b, t1_6.a, t1_6.b
+                           Hash Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
+                           ->  Seq Scan on public.prt2_p1 t1_3
+                                 Output: t1_3.b
+                           ->  Hash
+                                 Output: t1_6.a, t1_6.b
+                                 ->  Seq Scan on public.prt1_e_p1 t1_6
+                                       Output: t1_6.a, t1_6.b
+                                       Filter: ((t1_6.a % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.a = t1_4.b)
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_4.b, t1_7.a, t1_7.b
+                     ->  Hash Semi Join
+                           Output: t1_4.b, t1_7.a, t1_7.b
+                           Hash Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
+                           ->  Seq Scan on public.prt2_p2 t1_4
+                                 Output: t1_4.b
+                           ->  Hash
+                                 Output: t1_7.a, t1_7.b
+                                 ->  Seq Scan on public.prt1_e_p2 t1_7
+                                       Output: t1_7.a, t1_7.b
+                                       Filter: ((t1_7.a % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.a = t1_5.b)
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_5.b, t1_8.a, t1_8.b
+                     ->  Hash Semi Join
+                           Output: t1_5.b, t1_8.a, t1_8.b
+                           Hash Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
+                           ->  Seq Scan on public.prt2_p3 t1_5
+                                 Output: t1_5.b
+                           ->  Hash
+                                 Output: t1_8.a, t1_8.b
+                                 ->  Seq Scan on public.prt1_e_p3 t1_8
+                                       Output: t1_8.a, t1_8.b
+                                       Filter: ((t1_8.a % 25) = 0)
+(58 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 5) is considered for partition-wise join.
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+-- test merge joins with and without using indexes
+SET enable_hashjoin TO off;
+SET enable_nestloop TO off;
+CREATE INDEX iprt1_a on prt1(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 iprt2_b on prt2(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 iprt1_e_ab2 on prt1_e(((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));
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+                                             QUERY PLAN                                              
+-----------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Merge Cond: (t2.b = t1.a)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           Sort Key: t2.b
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Merge Cond: ((((t3.a + t3.b) / 2)) = t2.b)
+                                 ->  Sort
+                                       Output: t3.a, t3.b, t3.c, (((t3.a + t3.b) / 2))
+                                       Sort Key: (((t3.a + t3.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c, ((t3.a + t3.b) / 2)
+                                             Filter: ((t3.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2.b, t2.c
+                                       Sort Key: t2.b
+                                       ->  Seq Scan on public.prt2_p1 t2
+                                             Output: t2.b, t2.c
+                     ->  Sort
+                           Output: t1.a, t1.c
+                           Sort Key: t1.a
+                           ->  Seq Scan on public.prt1_p1 t1
+                                 Output: t1.a, t1.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Merge Cond: (t2_1.b = t1_2.a)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           Sort Key: t2_1.b
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t2_1.b)
+                                 ->  Sort
+                                       Output: t3_1.a, t3_1.b, t3_1.c, (((t3_1.a + t3_1.b) / 2))
+                                       Sort Key: (((t3_1.a + t3_1.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c, ((t3_1.a + t3_1.b) / 2)
+                                             Filter: ((t3_1.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2_1.b, t2_1.c
+                                       Sort Key: t2_1.b
+                                       ->  Seq Scan on public.prt2_p2 t2_1
+                                             Output: t2_1.b, t2_1.c
+                     ->  Sort
+                           Output: t1_2.a, t1_2.c
+                           Sort Key: t1_2.a
+                           ->  Seq Scan on public.prt1_p2 t1_2
+                                 Output: t1_2.a, t1_2.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Merge Cond: (t2_2.b = t1_1.a)
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           Sort Key: t2_2.b
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t2_2.b)
+                                 ->  Sort
+                                       Output: t3_2.a, t3_2.b, t3_2.c, (((t3_2.a + t3_2.b) / 2))
+                                       Sort Key: (((t3_2.a + t3_2.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c, ((t3_2.a + t3_2.b) / 2)
+                                             Filter: ((t3_2.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2_2.b, t2_2.c
+                                       Sort Key: t2_2.b
+                                       ->  Seq Scan on public.prt2_p3 t2_2
+                                             Output: t2_2.b, t2_2.c
+                     ->  Sort
+                           Output: t1_1.a, t1_1.c
+                           Sort Key: t1_1.a
+                           ->  Seq Scan on public.prt1_p3 t1_1
+                                 Output: t1_1.a, t1_1.c
+(81 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 5) is considered for partition-wise join.
+                                   QUERY PLAN                                    
+---------------------------------------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Output: t1.a, t1.b, t1.c
+         Merge Cond: (t1.a = t1_3.b)
+         ->  Sort
+               Output: t1.a, t1.b, t1.c
+               Sort Key: t1.a
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_3.b, t1_6.a, t1_6.b
+               ->  Merge Semi Join
+                     Output: t1_3.b, t1_6.a, t1_6.b
+                     Merge Cond: (t1_3.b = (((t1_6.a + t1_6.b) / 2)))
+                     ->  Sort
+                           Output: t1_3.b
+                           Sort Key: t1_3.b
+                           ->  Seq Scan on public.prt2_p1 t1_3
+                                 Output: t1_3.b
+                     ->  Sort
+                           Output: t1_6.a, t1_6.b, (((t1_6.a + t1_6.b) / 2))
+                           Sort Key: (((t1_6.a + t1_6.b) / 2))
+                           ->  Seq Scan on public.prt1_e_p1 t1_6
+                                 Output: t1_6.a, t1_6.b, ((t1_6.a + t1_6.b) / 2)
+                                 Filter: ((t1_6.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_2.a, t1_2.b, t1_2.c
+         Merge Cond: (t1_2.a = t1_4.b)
+         ->  Sort
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Sort Key: t1_2.a
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+         ->  Materialize
+               Output: t1_4.b, t1_7.a, t1_7.b
+               ->  Merge Semi Join
+                     Output: t1_4.b, t1_7.a, t1_7.b
+                     Merge Cond: (t1_4.b = (((t1_7.a + t1_7.b) / 2)))
+                     ->  Sort
+                           Output: t1_4.b
+                           Sort Key: t1_4.b
+                           ->  Seq Scan on public.prt2_p2 t1_4
+                                 Output: t1_4.b
+                     ->  Sort
+                           Output: t1_7.a, t1_7.b, (((t1_7.a + t1_7.b) / 2))
+                           Sort Key: (((t1_7.a + t1_7.b) / 2))
+                           ->  Seq Scan on public.prt1_e_p2 t1_7
+                                 Output: t1_7.a, t1_7.b, ((t1_7.a + t1_7.b) / 2)
+                                 Filter: ((t1_7.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_1.a, t1_1.b, t1_1.c
+         Merge Cond: (t1_1.a = t1_5.b)
+         ->  Sort
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Sort Key: t1_1.a
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_5.b, t1_8.a, t1_8.b
+               ->  Merge Semi Join
+                     Output: t1_5.b, t1_8.a, t1_8.b
+                     Merge Cond: (t1_5.b = (((t1_8.a + t1_8.b) / 2)))
+                     ->  Sort
+                           Output: t1_5.b
+                           Sort Key: t1_5.b
+                           ->  Seq Scan on public.prt2_p3 t1_5
+                                 Output: t1_5.b
+                     ->  Sort
+                           Output: t1_8.a, t1_8.b, (((t1_8.a + t1_8.b) / 2))
+                           Sort Key: (((t1_8.a + t1_8.b) / 2))
+                           ->  Seq Scan on public.prt1_e_p3 t1_8
+                                 Output: t1_8.a, t1_8.b, ((t1_8.a + t1_8.b) / 2)
+                                 Filter: ((t1_8.a % 25) = 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 5) is considered for partition-wise join.
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+                                             QUERY PLAN                                              
+-----------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Merge Cond: (t1.a = t2.b)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           Sort Key: t1.a
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Merge Cond: ((((t3.a + t3.b) / 2)) = t1.a)
+                                 ->  Sort
+                                       Output: t3.a, t3.b, t3.c, (((t3.a + t3.b) / 2))
+                                       Sort Key: (((t3.a + t3.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c, ((t3.a + t3.b) / 2)
+                                             Filter: ((t3.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t1.a, t1.c
+                                       Sort Key: t1.a
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                     ->  Sort
+                           Output: t2.b, t2.c
+                           Sort Key: t2.b
+                           ->  Seq Scan on public.prt2_p1 t2
+                                 Output: t2.b, t2.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                     Merge Cond: (t1_2.a = t2_1.b)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                           Sort Key: t1_2.a
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                                 Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t1_2.a)
+                                 ->  Sort
+                                       Output: t3_1.a, t3_1.b, t3_1.c, (((t3_1.a + t3_1.b) / 2))
+                                       Sort Key: (((t3_1.a + t3_1.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c, ((t3_1.a + t3_1.b) / 2)
+                                             Filter: ((t3_1.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t1_2.a, t1_2.c
+                                       Sort Key: t1_2.a
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                     ->  Sort
+                           Output: t2_1.b, t2_1.c
+                           Sort Key: t2_1.b
+                           ->  Seq Scan on public.prt2_p2 t2_1
+                                 Output: t2_1.b, t2_1.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                     Merge Cond: (t1_1.a = t2_2.b)
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                           Sort Key: t1_1.a
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                                 Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t1_1.a)
+                                 ->  Sort
+                                       Output: t3_2.a, t3_2.b, t3_2.c, (((t3_2.a + t3_2.b) / 2))
+                                       Sort Key: (((t3_2.a + t3_2.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c, ((t3_2.a + t3_2.b) / 2)
+                                             Filter: ((t3_2.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t1_1.a, t1_1.c
+                                       Sort Key: t1_1.a
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                     ->  Sort
+                           Output: t2_2.b, t2_2.c
+                           Sort Key: t2_2.b
+                           ->  Seq Scan on public.prt2_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+(81 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SET enable_seqscan TO off;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+                                          QUERY PLAN                                          
+----------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Merge Cond: (t2.b = t1.a)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           Sort Key: t2.b
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Merge Cond: (((t3.a + t3.b) / 2) = t2.b)
+                                 ->  Index Scan using iprt1_e_p1_ab2 on public.prt1_e_p1 t3
+                                       Output: t3.a, t3.b, t3.c
+                                       Filter: ((t3.a % 25) = 0)
+                                 ->  Index Scan using iprt2_p1_b on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                     ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Merge Cond: (t2_1.b = t1_2.a)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           Sort Key: t2_1.b
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Merge Cond: (((t3_1.a + t3_1.b) / 2) = t2_1.b)
+                                 ->  Index Scan using iprt1_e_p2_ab2 on public.prt1_e_p2 t3_1
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       Filter: ((t3_1.a % 25) = 0)
+                                 ->  Index Scan using iprt2_p2_b on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                     ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+               ->  Merge Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Merge Cond: (t2_2.b = ((t3_2.a + t3_2.b) / 2))
+                     ->  Merge Left Join
+                           Output: t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                           Merge Cond: (t2_2.b = t1_1.a)
+                           ->  Index Scan using iprt2_p3_b on public.prt2_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+                           ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_1
+                                 Output: t1_1.a, t1_1.c
+                     ->  Index Scan using iprt1_e_p3_ab2 on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                           Filter: ((t3_2.a % 25) = 0)
+(51 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 5) is considered for partition-wise join.
+                                    QUERY PLAN                                    
+----------------------------------------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Output: t1.a, t1.b, t1.c
+         Merge Cond: (t1.a = t1_3.b)
+         ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1
+               Output: t1.a, t1.b, t1.c
+               Filter: ((t1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_3.b, t1_6.a, t1_6.b
+               ->  Merge Semi Join
+                     Output: t1_3.b, t1_6.a, t1_6.b
+                     Merge Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
+                     ->  Index Only Scan using iprt2_p1_b on public.prt2_p1 t1_3
+                           Output: t1_3.b
+                     ->  Index Scan using iprt1_e_p1_ab2 on public.prt1_e_p1 t1_6
+                           Output: t1_6.a, t1_6.b
+                           Filter: ((t1_6.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_2.a, t1_2.b, t1_2.c
+         Merge Cond: (t1_2.a = t1_4.b)
+         ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_2
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Filter: ((t1_2.a % 25) = 0)
+         ->  Materialize
+               Output: t1_4.b, t1_7.a, t1_7.b
+               ->  Merge Semi Join
+                     Output: t1_4.b, t1_7.a, t1_7.b
+                     Merge Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
+                     ->  Index Only Scan using iprt2_p2_b on public.prt2_p2 t1_4
+                           Output: t1_4.b
+                     ->  Index Scan using iprt1_e_p2_ab2 on public.prt1_e_p2 t1_7
+                           Output: t1_7.a, t1_7.b
+                           Filter: ((t1_7.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_1.a, t1_1.b, t1_1.c
+         Merge Cond: (t1_1.a = t1_5.b)
+         ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_1
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Filter: ((t1_1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_5.b, t1_8.a, t1_8.b
+               ->  Merge Semi Join
+                     Output: t1_5.b, t1_8.a, t1_8.b
+                     Merge Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
+                     ->  Index Only Scan using iprt2_p3_b on public.prt2_p3 t1_5
+                           Output: t1_5.b
+                     ->  Index Scan using iprt1_e_p3_ab2 on public.prt1_e_p3 t1_8
+                           Output: t1_8.a, t1_8.b
+                           Filter: ((t1_8.a % 25) = 0)
+(50 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 5) is considered for partition-wise join.
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+                                          QUERY PLAN                                          
+----------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Merge Cond: (t1.a = t2.b)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           Sort Key: t1.a
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Merge Cond: (((t3.a + t3.b) / 2) = t1.a)
+                                 ->  Index Scan using iprt1_e_p1_ab2 on public.prt1_e_p1 t3
+                                       Output: t3.a, t3.b, t3.c
+                                       Filter: ((t3.a % 25) = 0)
+                                 ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1
+                                       Output: t1.a, t1.c
+                     ->  Index Scan using iprt2_p1_b on public.prt2_p1 t2
+                           Output: t2.b, t2.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                     Merge Cond: (t1_2.a = t2_1.b)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                           Sort Key: t1_2.a
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                                 Merge Cond: (((t3_1.a + t3_1.b) / 2) = t1_2.a)
+                                 ->  Index Scan using iprt1_e_p2_ab2 on public.prt1_e_p2 t3_1
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       Filter: ((t3_1.a % 25) = 0)
+                                 ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_2
+                                       Output: t1_2.a, t1_2.c
+                     ->  Index Scan using iprt2_p2_b on public.prt2_p2 t2_1
+                           Output: t2_1.b, t2_1.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                     Merge Cond: (t1_1.a = t2_2.b)
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                           Sort Key: t1_1.a
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                                 Merge Cond: (((t3_2.a + t3_2.b) / 2) = t1_1.a)
+                                 ->  Index Scan using iprt1_e_p3_ab2 on public.prt1_e_p3 t3_2
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       Filter: ((t3_2.a % 25) = 0)
+                                 ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_1
+                                       Output: t1_1.a, t1_1.c
+                     ->  Index Scan using iprt2_p3_b on public.prt2_p3 t2_2
+                           Output: t2_2.b, t2_2.c
+(54 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+-- lateral references and parameterized paths
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4 5) is considered for partition-wise join.
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
+ Nested Loop Left Join
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   ->  Merge Append
+         Sort Key: t1.a
+         ->  Index Scan using iprt1_a on public.prt1 t1
+               Output: t1.a, t1.b, t1.c
+               Filter: ((t1.a % 25) = 0)
+         ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1_1
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Filter: ((t1_1.a % 25) = 0)
+         ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_2
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Filter: ((t1_2.a % 25) = 0)
+         ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_3
+               Output: t1_3.a, t1_3.b, t1_3.c
+               Filter: ((t1_3.a % 25) = 0)
+   ->  Append
+         ->  Merge Join
+               Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+               Merge Cond: (t2.a = t3.b)
+               ->  Index Only Scan using iprt1_p1_a on public.prt1_p1 t2
+                     Output: t2.a
+                     Index Cond: (t2.a = t1.a)
+               ->  Index Scan using iprt2_p1_b on public.prt2_p1 t3
+                     Output: t3.a, t3.b
+         ->  Merge Join
+               Output: t2_2.a, t3_1.a, LEAST(t1.a, t2_2.a, t3_1.a)
+               Merge Cond: (t2_2.a = t3_1.b)
+               ->  Index Only Scan using iprt1_p2_a on public.prt1_p2 t2_2
+                     Output: t2_2.a
+                     Index Cond: (t2_2.a = t1.a)
+               ->  Index Scan using iprt2_p2_b on public.prt2_p2 t3_1
+                     Output: t3_1.a, t3_1.b
+         ->  Merge Join
+               Output: t2_1.a, t3_2.a, LEAST(t1.a, t2_1.a, t3_2.a)
+               Merge Cond: (t2_1.a = t3_2.b)
+               ->  Index Only Scan using iprt1_p3_a on public.prt1_p3 t2_1
+                     Output: t2_1.a
+                     Index Cond: (t2_1.a = t1.a)
+               ->  Index Scan using iprt2_p3_b on public.prt2_p3 t3_2
+                     Output: t3_2.a, t3_2.b
+(41 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4 5) is considered for partition-wise join.
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
+ Nested Loop Left Join
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   ->  Merge Append
+         Sort Key: t1.a
+         ->  Index Scan using iprt1_a on public.prt1 t1
+               Output: t1.a, t1.b, t1.c
+               Filter: ((t1.a % 25) = 0)
+         ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1_1
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Filter: ((t1_1.a % 25) = 0)
+         ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_2
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Filter: ((t1_2.a % 25) = 0)
+         ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_3
+               Output: t1_3.a, t1_3.b, t1_3.c
+               Filter: ((t1_3.a % 25) = 0)
+   ->  Append
+         ->  Merge Join
+               Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+               Merge Cond: (t2.a = t3.b)
+               ->  Index Only Scan using iprt1_p1_a on public.prt1_p1 t2
+                     Output: t2.a
+                     Index Cond: (t2.a = t1.b)
+               ->  Index Scan using iprt2_p1_b on public.prt2_p1 t3
+                     Output: t3.a, t3.b
+         ->  Merge Join
+               Output: t2_2.a, t3_1.a, LEAST(t1.a, t2_2.a, t3_1.a)
+               Merge Cond: (t2_2.a = t3_1.b)
+               ->  Index Only Scan using iprt1_p2_a on public.prt1_p2 t2_2
+                     Output: t2_2.a
+                     Index Cond: (t2_2.a = t1.b)
+               ->  Index Scan using iprt2_p2_b on public.prt2_p2 t3_1
+                     Output: t3_1.a, t3_1.b
+         ->  Merge Join
+               Output: t2_1.a, t3_2.a, LEAST(t1.a, t2_1.a, t3_2.a)
+               Merge Cond: (t2_1.a = t3_2.b)
+               ->  Index Only Scan using iprt1_p3_a on public.prt1_p3 t2_1
+                     Output: t2_1.a
+                     Index Cond: (t2_1.a = t1.b)
+               ->  Index Scan using iprt2_p3_b on public.prt2_p3 t3_2
+                     Output: t3_2.a, t3_2.b
+(41 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+RESET enable_hashjoin;
+RESET enable_nestloop;
+RESET enable_seqscan;
+--
+-- partitioned by multiple columns
+--
+CREATE TABLE prt1_m (a int, b int, c varchar) PARTITION BY RANGE(a, ((a + b)/2));
+CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES START (0, 0) END (250, 250);
+CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES START (250, 250) END (500, 500);
+CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES START (500, 500) END (600, 600);
+INSERT INTO prt1_m SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_m;
+ANALYZE prt1_m_p1;
+ANALYZE prt1_m_p2;
+ANALYZE prt1_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_m AS SELECT * FROM prt1_m;
+CREATE TABLE prt2_m (a int, b int, c varchar) PARTITION BY RANGE(((b + a)/2), b);
+CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES START (0, 0) END (250, 250);
+CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES START (250, 250) END (500, 500);
+CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES START (500, 500) END (600, 600);
+INSERT INTO prt2_m SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_m;
+ANALYZE prt2_m_p1;
+ANALYZE prt2_m_p2;
+ANALYZE prt2_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_m AS SELECT * FROM prt2_m;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1, prt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+                                              QUERY PLAN                                              
+------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((((t2.b + t2.a) / 2) = t1.a) AND (t2.b = ((t1.a + t1.b) / 2)))
+               ->  Seq Scan on public.prt2_m_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_m_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((((t2_1.b + t2_1.a) / 2) = t1_1.a) AND (t2_1.b = ((t1_1.a + t1_1.b) / 2)))
+               ->  Seq Scan on public.prt2_m_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_m_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((((t2_2.b + t2_2.a) / 2) = t1_2.a) AND (t2_2.b = ((t1_2.a + t1_2.b) / 2)))
+               ->  Seq Scan on public.prt2_m_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_m_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1, prt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0150 | 150 | 0150
+ 300 | 0300 | 300 | 0300
+ 450 | 0450 | 450 | 0450
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_m t1, uprt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 AND t1.a % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+                                              QUERY PLAN                                              
+------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((((t2.b + t2.a) / 2) = t1.a) AND (t2.b = ((t1.a + t1.b) / 2)))
+               ->  Seq Scan on public.prt2_m_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_m_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((((t2_1.b + t2_1.a) / 2) = t1_1.a) AND (t2_1.b = ((t1_1.a + t1_1.b) / 2)))
+               ->  Seq Scan on public.prt2_m_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_m_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((((t2_2.b + t2_2.a) / 2) = t1_2.a) AND (t2_2.b = ((t1_2.a + t1_2.b) / 2)))
+               ->  Seq Scan on public.prt2_m_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_m_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+  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 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_m t1 LEFT JOIN uprt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+                                                 QUERY PLAN                                                 
+------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: ((((t1.a + t1.b) / 2) = t2.b) AND (t1.a = ((t2.b + t2.a) / 2)))
+                     ->  Seq Scan on public.prt1_m_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                     ->  Hash
+                           Output: t2.b, t2.c, t2.a
+                           ->  Seq Scan on public.prt2_m_p1 t2
+                                 Output: t2.b, t2.c, t2.a
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: ((((t1_1.a + t1_1.b) / 2) = t2_1.b) AND (t1_1.a = ((t2_1.b + t2_1.a) / 2)))
+                     ->  Seq Scan on public.prt1_m_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c, t2_1.a
+                           ->  Seq Scan on public.prt2_m_p2 t2_1
+                                 Output: t2_1.b, t2_1.c, t2_1.a
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: ((((t1_2.a + t1_2.b) / 2) = t2_2.b) AND (t1_2.a = ((t2_2.b + t2_2.a) / 2)))
+                     ->  Seq Scan on public.prt1_m_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c, t2_2.a
+                           ->  Seq Scan on public.prt2_m_p3 t2_2
+                                 Output: t2_2.b, t2_2.c, t2_2.a
+                                 Filter: ((t2_2.b % 25) = 0)
+(36 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+  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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_m t1 RIGHT JOIN uprt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                                             QUERY PLAN                                                             
+------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_m_p1.a, prt1_m_p1.c, prt2_m_p1.b, prt2_m_p1.c
+   Sort Key: prt1_m_p1.a, prt2_m_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_m_p1.a, prt1_m_p1.c, prt2_m_p1.b, prt2_m_p1.c
+               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 public.prt1_m_p1
+                     Output: prt1_m_p1.a, prt1_m_p1.c, prt1_m_p1.b
+                     Filter: ((prt1_m_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_m_p1.b, prt2_m_p1.c, prt2_m_p1.a
+                     ->  Seq Scan on public.prt2_m_p1
+                           Output: prt2_m_p1.b, prt2_m_p1.c, prt2_m_p1.a
+                           Filter: ((prt2_m_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_m_p2.a, prt1_m_p2.c, prt2_m_p2.b, prt2_m_p2.c
+               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 public.prt1_m_p2
+                     Output: prt1_m_p2.a, prt1_m_p2.c, prt1_m_p2.b
+                     Filter: ((prt1_m_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_m_p2.b, prt2_m_p2.c, prt2_m_p2.a
+                     ->  Seq Scan on public.prt2_m_p2
+                           Output: prt2_m_p2.b, prt2_m_p2.c, prt2_m_p2.a
+                           Filter: ((prt2_m_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_m_p3.a, prt1_m_p3.c, prt2_m_p3.b, prt2_m_p3.c
+               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 public.prt1_m_p3
+                     Output: prt1_m_p3.a, prt1_m_p3.c, prt1_m_p3.b
+                     Filter: ((prt1_m_p3.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_m_p3.b, prt2_m_p3.c, prt2_m_p3.a
+                     ->  Seq Scan on public.prt2_m_p3
+                           Output: prt2_m_p3.b, prt2_m_p3.c, prt2_m_p3.a
+                           Filter: ((prt2_m_p3.b % 25) = 0)
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_m t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_m t2 WHERE t2.b % 25 = 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 | 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)
+
+--
+-- multi-leveled partitions
+--
+CREATE TABLE prt1_l (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_l_p1 PARTITION OF prt1_l FOR VALUES START (0) END (250) PARTITION BY RANGE (b);
+CREATE TABLE prt1_l_p1_p1 PARTITION OF prt1_l_p1 FOR VALUES START (0) END (100);
+CREATE TABLE prt1_l_p1_p2 PARTITION OF prt1_l_p1 FOR VALUES START (100) END (250);
+CREATE TABLE prt1_l_p2 PARTITION OF prt1_l FOR VALUES START (250) END (500) PARTITION BY RANGE (c);
+CREATE TABLE prt1_l_p2_p1 PARTITION OF prt1_l_p2 FOR VALUES START ('0250') END ('0400');
+CREATE TABLE prt1_l_p2_p2 PARTITION OF prt1_l_p2 FOR VALUES START ('0400') END ('0500');
+CREATE TABLE prt1_l_p3 PARTITION OF prt1_l FOR VALUES START (500) END (600) PARTITION BY RANGE ((b + a));
+CREATE TABLE prt1_l_p3_p1 PARTITION OF prt1_l_p3 FOR VALUES START (1000) END (1100);
+CREATE TABLE prt1_l_p3_p2 PARTITION OF prt1_l_p3 FOR VALUES START (1100) END (1200);
+INSERT INTO prt1_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_l;
+ANALYZE prt1_l_p1;
+ANALYZE prt1_l_p1_p1;
+ANALYZE prt1_l_p1_p2;
+ANALYZE prt1_l_p2;
+ANALYZE prt1_l_p2_p1;
+ANALYZE prt1_l_p2_p2;
+ANALYZE prt1_l_p3;
+ANALYZE prt1_l_p3_p1;
+ANALYZE prt1_l_p3_p2;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_l AS SELECT * FROM prt1_l;
+CREATE TABLE prt2_l (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_l_p1 PARTITION OF prt2_l FOR VALUES START (0) END (250) PARTITION BY RANGE (a);
+CREATE TABLE prt2_l_p1_p1 PARTITION OF prt2_l_p1 FOR VALUES START (0) END (100);
+CREATE TABLE prt2_l_p1_p2 PARTITION OF prt2_l_p1 FOR VALUES START (100) END (250);
+CREATE TABLE prt2_l_p2 PARTITION OF prt2_l FOR VALUES START (250) END (500) PARTITION BY RANGE (c);
+CREATE TABLE prt2_l_p2_p1 PARTITION OF prt2_l_p2 FOR VALUES START ('0250') END ('0400');
+CREATE TABLE prt2_l_p2_p2 PARTITION OF prt2_l_p2 FOR VALUES START ('0400') END ('0500');
+CREATE TABLE prt2_l_p3 PARTITION OF prt2_l FOR VALUES START (500) END (600) PARTITION BY RANGE ((a + b));
+CREATE TABLE prt2_l_p3_p1 PARTITION OF prt2_l_p3 FOR VALUES START (1000) END (1100);
+CREATE TABLE prt2_l_p3_p2 PARTITION OF prt2_l_p3 FOR VALUES START (1100) END (1200);
+INSERT INTO prt2_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_l;
+ANALYZE prt2_l_p1;
+ANALYZE prt2_l_p1_p1;
+ANALYZE prt2_l_p1_p2;
+ANALYZE prt2_l_p2;
+ANALYZE prt2_l_p2_p1;
+ANALYZE prt2_l_p2_p2;
+ANALYZE prt2_l_p3;
+ANALYZE prt2_l_p3_p1;
+ANALYZE prt2_l_p3_p2;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_l AS SELECT * FROM prt2_l;
+EXPLAIN (VERBOSE, 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 = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                                         QUERY PLAN                                                         
+----------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a
+   ->  Hash Join
+         Output: t1.a, t1.c, t2.b, t2.c
+         Hash Cond: ((t2.b = t1.a) AND (t2.a = t1.b) AND ((t2.c)::text = (t1.c)::text) AND ((t2.a + t2.b) = (t1.b + t1.a)))
+         ->  Append
+               ->  Seq Scan on public.prt2_l t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Seq Scan on public.prt2_l_p1 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Seq Scan on public.prt2_l_p1_p1 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Seq Scan on public.prt2_l_p1_p2 t2_3
+                     Output: t2_3.b, t2_3.c, t2_3.a
+               ->  Seq Scan on public.prt2_l_p2 t2_4
+                     Output: t2_4.b, t2_4.c, t2_4.a
+               ->  Seq Scan on public.prt2_l_p2_p1 t2_5
+                     Output: t2_5.b, t2_5.c, t2_5.a
+               ->  Seq Scan on public.prt2_l_p2_p2 t2_6
+                     Output: t2_6.b, t2_6.c, t2_6.a
+               ->  Seq Scan on public.prt2_l_p3 t2_7
+                     Output: t2_7.b, t2_7.c, t2_7.a
+               ->  Seq Scan on public.prt2_l_p3_p1 t2_8
+                     Output: t2_8.b, t2_8.c, t2_8.a
+               ->  Seq Scan on public.prt2_l_p3_p2 t2_9
+                     Output: t2_9.b, t2_9.c, t2_9.a
+         ->  Hash
+               Output: t1.a, t1.c, t1.b
+               ->  Append
+                     ->  Seq Scan on public.prt1_l t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+                     ->  Seq Scan on public.prt1_l_p1 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+                     ->  Seq Scan on public.prt1_l_p1_p1 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+                     ->  Seq Scan on public.prt1_l_p1_p2 t1_3
+                           Output: t1_3.a, t1_3.c, t1_3.b
+                           Filter: ((t1_3.a % 25) = 0)
+                     ->  Seq Scan on public.prt1_l_p2 t1_4
+                           Output: t1_4.a, t1_4.c, t1_4.b
+                           Filter: ((t1_4.a % 25) = 0)
+                     ->  Seq Scan on public.prt1_l_p2_p1 t1_5
+                           Output: t1_5.a, t1_5.c, t1_5.b
+                           Filter: ((t1_5.a % 25) = 0)
+                     ->  Seq Scan on public.prt1_l_p2_p2 t1_6
+                           Output: t1_6.a, t1_6.c, t1_6.b
+                           Filter: ((t1_6.a % 25) = 0)
+                     ->  Seq Scan on public.prt1_l_p3 t1_7
+                           Output: t1_7.a, t1_7.c, t1_7.b
+                           Filter: ((t1_7.a % 25) = 0)
+                     ->  Seq Scan on public.prt1_l_p3_p1 t1_8
+                           Output: t1_8.a, t1_8.c, t1_8.b
+                           Filter: ((t1_8.a % 25) = 0)
+                     ->  Seq Scan on public.prt1_l_p3_p2 t1_9
+                           Output: t1_9.a, t1_9.c, t1_9.b
+                           Filter: ((t1_9.a % 25) = 0)
+(60 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1, prt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1, uprt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 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)
+
+EXPLAIN (VERBOSE, 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                                         QUERY PLAN                                                         
+----------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Hash Right Join
+         Output: t1.a, t1.c, t2.b, t2.c
+         Hash Cond: ((t2.b = t1.a) AND (t2.a = t1.b) AND ((t2.c)::text = (t1.c)::text) AND ((t2.a + t2.b) = (t1.b + t1.a)))
+         ->  Append
+               ->  Seq Scan on public.prt2_l t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Seq Scan on public.prt2_l_p1 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Seq Scan on public.prt2_l_p1_p1 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Seq Scan on public.prt2_l_p1_p2 t2_3
+                     Output: t2_3.b, t2_3.c, t2_3.a
+               ->  Seq Scan on public.prt2_l_p2 t2_4
+                     Output: t2_4.b, t2_4.c, t2_4.a
+               ->  Seq Scan on public.prt2_l_p2_p1 t2_5
+                     Output: t2_5.b, t2_5.c, t2_5.a
+               ->  Seq Scan on public.prt2_l_p2_p2 t2_6
+                     Output: t2_6.b, t2_6.c, t2_6.a
+               ->  Seq Scan on public.prt2_l_p3 t2_7
+                     Output: t2_7.b, t2_7.c, t2_7.a
+               ->  Seq Scan on public.prt2_l_p3_p1 t2_8
+                     Output: t2_8.b, t2_8.c, t2_8.a
+               ->  Seq Scan on public.prt2_l_p3_p2 t2_9
+                     Output: t2_9.b, t2_9.c, t2_9.a
+         ->  Hash
+               Output: t1.a, t1.c, t1.b
+               ->  Append
+                     ->  Seq Scan on public.prt1_l t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+                     ->  Seq Scan on public.prt1_l_p1 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+                     ->  Seq Scan on public.prt1_l_p1_p1 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+                     ->  Seq Scan on public.prt1_l_p1_p2 t1_3
+                           Output: t1_3.a, t1_3.c, t1_3.b
+                           Filter: ((t1_3.a % 25) = 0)
+                     ->  Seq Scan on public.prt1_l_p2 t1_4
+                           Output: t1_4.a, t1_4.c, t1_4.b
+                           Filter: ((t1_4.a % 25) = 0)
+                     ->  Seq Scan on public.prt1_l_p2_p1 t1_5
+                           Output: t1_5.a, t1_5.c, t1_5.b
+                           Filter: ((t1_5.a % 25) = 0)
+                     ->  Seq Scan on public.prt1_l_p2_p2 t1_6
+                           Output: t1_6.a, t1_6.c, t1_6.b
+                           Filter: ((t1_6.a % 25) = 0)
+                     ->  Seq Scan on public.prt1_l_p3 t1_7
+                           Output: t1_7.a, t1_7.c, t1_7.b
+                           Filter: ((t1_7.a % 25) = 0)
+                     ->  Seq Scan on public.prt1_l_p3_p1 t1_8
+                           Output: t1_8.a, t1_8.c, t1_8.b
+                           Filter: ((t1_8.a % 25) = 0)
+                     ->  Seq Scan on public.prt1_l_p3_p2 t1_9
+                           Output: t1_9.a, t1_9.c, t1_9.b
+                           Filter: ((t1_9.a % 25) = 0)
+(60 rows)
+
+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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 LEFT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+                                                         QUERY PLAN                                                         
+----------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Hash Right Join
+         Output: t1.a, t1.c, t2.b, t2.c
+         Hash Cond: ((t1.a = t2.b) AND (t1.b = t2.a) AND ((t1.c)::text = (t2.c)::text) AND ((t1.b + t1.a) = (t2.a + t2.b)))
+         ->  Append
+               ->  Seq Scan on public.prt1_l t1
+                     Output: t1.a, t1.c, t1.b
+               ->  Seq Scan on public.prt1_l_p1 t1_1
+                     Output: t1_1.a, t1_1.c, t1_1.b
+               ->  Seq Scan on public.prt1_l_p1_p1 t1_2
+                     Output: t1_2.a, t1_2.c, t1_2.b
+               ->  Seq Scan on public.prt1_l_p1_p2 t1_3
+                     Output: t1_3.a, t1_3.c, t1_3.b
+               ->  Seq Scan on public.prt1_l_p2 t1_4
+                     Output: t1_4.a, t1_4.c, t1_4.b
+               ->  Seq Scan on public.prt1_l_p2_p1 t1_5
+                     Output: t1_5.a, t1_5.c, t1_5.b
+               ->  Seq Scan on public.prt1_l_p2_p2 t1_6
+                     Output: t1_6.a, t1_6.c, t1_6.b
+               ->  Seq Scan on public.prt1_l_p3 t1_7
+                     Output: t1_7.a, t1_7.c, t1_7.b
+               ->  Seq Scan on public.prt1_l_p3_p1 t1_8
+                     Output: t1_8.a, t1_8.c, t1_8.b
+               ->  Seq Scan on public.prt1_l_p3_p2 t1_9
+                     Output: t1_9.a, t1_9.c, t1_9.b
+         ->  Hash
+               Output: t2.b, t2.c, t2.a
+               ->  Append
+                     ->  Seq Scan on public.prt2_l t2
+                           Output: t2.b, t2.c, t2.a
+                           Filter: ((t2.b % 25) = 0)
+                     ->  Seq Scan on public.prt2_l_p1 t2_1
+                           Output: t2_1.b, t2_1.c, t2_1.a
+                           Filter: ((t2_1.b % 25) = 0)
+                     ->  Seq Scan on public.prt2_l_p1_p1 t2_2
+                           Output: t2_2.b, t2_2.c, t2_2.a
+                           Filter: ((t2_2.b % 25) = 0)
+                     ->  Seq Scan on public.prt2_l_p1_p2 t2_3
+                           Output: t2_3.b, t2_3.c, t2_3.a
+                           Filter: ((t2_3.b % 25) = 0)
+                     ->  Seq Scan on public.prt2_l_p2 t2_4
+                           Output: t2_4.b, t2_4.c, t2_4.a
+                           Filter: ((t2_4.b % 25) = 0)
+                     ->  Seq Scan on public.prt2_l_p2_p1 t2_5
+                           Output: t2_5.b, t2_5.c, t2_5.a
+                           Filter: ((t2_5.b % 25) = 0)
+                     ->  Seq Scan on public.prt2_l_p2_p2 t2_6
+                           Output: t2_6.b, t2_6.c, t2_6.a
+                           Filter: ((t2_6.b % 25) = 0)
+                     ->  Seq Scan on public.prt2_l_p3 t2_7
+                           Output: t2_7.b, t2_7.c, t2_7.a
+                           Filter: ((t2_7.b % 25) = 0)
+                     ->  Seq Scan on public.prt2_l_p3_p1 t2_8
+                           Output: t2_8.b, t2_8.c, t2_8.a
+                           Filter: ((t2_8.b % 25) = 0)
+                     ->  Seq Scan on public.prt2_l_p3_p2 t2_9
+                           Output: t2_9.b, t2_9.c, t2_9.a
+                           Filter: ((t2_9.b % 25) = 0)
+(60 rows)
+
+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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 RIGHT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+                                                                             QUERY PLAN                                                                             
+--------------------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_l.a, prt1_l.c, prt2_l.b, prt2_l.c
+   Sort Key: prt1_l.a, prt2_l.b
+   ->  Hash Full Join
+         Output: prt1_l.a, prt1_l.c, prt2_l.b, prt2_l.c
+         Hash Cond: ((prt1_l.a = prt2_l.b) AND (prt1_l.b = prt2_l.a) AND ((prt1_l.c)::text = (prt2_l.c)::text) AND ((prt1_l.b + prt1_l.a) = (prt2_l.a + prt2_l.b)))
+         ->  Append
+               ->  Seq Scan on public.prt1_l
+                     Output: prt1_l.a, prt1_l.c, prt1_l.b
+                     Filter: ((prt1_l.a % 25) = 0)
+               ->  Seq Scan on public.prt1_l_p1
+                     Output: prt1_l_p1.a, prt1_l_p1.c, prt1_l_p1.b
+                     Filter: ((prt1_l_p1.a % 25) = 0)
+               ->  Seq Scan on public.prt1_l_p1_p1
+                     Output: prt1_l_p1_p1.a, prt1_l_p1_p1.c, prt1_l_p1_p1.b
+                     Filter: ((prt1_l_p1_p1.a % 25) = 0)
+               ->  Seq Scan on public.prt1_l_p1_p2
+                     Output: prt1_l_p1_p2.a, prt1_l_p1_p2.c, prt1_l_p1_p2.b
+                     Filter: ((prt1_l_p1_p2.a % 25) = 0)
+               ->  Seq Scan on public.prt1_l_p2
+                     Output: prt1_l_p2.a, prt1_l_p2.c, prt1_l_p2.b
+                     Filter: ((prt1_l_p2.a % 25) = 0)
+               ->  Seq Scan on public.prt1_l_p2_p1
+                     Output: prt1_l_p2_p1.a, prt1_l_p2_p1.c, prt1_l_p2_p1.b
+                     Filter: ((prt1_l_p2_p1.a % 25) = 0)
+               ->  Seq Scan on public.prt1_l_p2_p2
+                     Output: prt1_l_p2_p2.a, prt1_l_p2_p2.c, prt1_l_p2_p2.b
+                     Filter: ((prt1_l_p2_p2.a % 25) = 0)
+               ->  Seq Scan on public.prt1_l_p3
+                     Output: prt1_l_p3.a, prt1_l_p3.c, prt1_l_p3.b
+                     Filter: ((prt1_l_p3.a % 25) = 0)
+               ->  Seq Scan on public.prt1_l_p3_p1
+                     Output: prt1_l_p3_p1.a, prt1_l_p3_p1.c, prt1_l_p3_p1.b
+                     Filter: ((prt1_l_p3_p1.a % 25) = 0)
+               ->  Seq Scan on public.prt1_l_p3_p2
+                     Output: prt1_l_p3_p2.a, prt1_l_p3_p2.c, prt1_l_p3_p2.b
+                     Filter: ((prt1_l_p3_p2.a % 25) = 0)
+         ->  Hash
+               Output: prt2_l.b, prt2_l.c, prt2_l.a
+               ->  Append
+                     ->  Seq Scan on public.prt2_l
+                           Output: prt2_l.b, prt2_l.c, prt2_l.a
+                           Filter: ((prt2_l.b % 25) = 0)
+                     ->  Seq Scan on public.prt2_l_p1
+                           Output: prt2_l_p1.b, prt2_l_p1.c, prt2_l_p1.a
+                           Filter: ((prt2_l_p1.b % 25) = 0)
+                     ->  Seq Scan on public.prt2_l_p1_p1
+                           Output: prt2_l_p1_p1.b, prt2_l_p1_p1.c, prt2_l_p1_p1.a
+                           Filter: ((prt2_l_p1_p1.b % 25) = 0)
+                     ->  Seq Scan on public.prt2_l_p1_p2
+                           Output: prt2_l_p1_p2.b, prt2_l_p1_p2.c, prt2_l_p1_p2.a
+                           Filter: ((prt2_l_p1_p2.b % 25) = 0)
+                     ->  Seq Scan on public.prt2_l_p2
+                           Output: prt2_l_p2.b, prt2_l_p2.c, prt2_l_p2.a
+                           Filter: ((prt2_l_p2.b % 25) = 0)
+                     ->  Seq Scan on public.prt2_l_p2_p1
+                           Output: prt2_l_p2_p1.b, prt2_l_p2_p1.c, prt2_l_p2_p1.a
+                           Filter: ((prt2_l_p2_p1.b % 25) = 0)
+                     ->  Seq Scan on public.prt2_l_p2_p2
+                           Output: prt2_l_p2_p2.b, prt2_l_p2_p2.c, prt2_l_p2_p2.a
+                           Filter: ((prt2_l_p2_p2.b % 25) = 0)
+                     ->  Seq Scan on public.prt2_l_p3
+                           Output: prt2_l_p3.b, prt2_l_p3.c, prt2_l_p3.a
+                           Filter: ((prt2_l_p3.b % 25) = 0)
+                     ->  Seq Scan on public.prt2_l_p3_p1
+                           Output: prt2_l_p3_p1.b, prt2_l_p3_p1.c, prt2_l_p3_p1.a
+                           Filter: ((prt2_l_p3_p1.b % 25) = 0)
+                     ->  Seq Scan on public.prt2_l_p3_p2
+                           Output: prt2_l_p3_p2.b, prt2_l_p3_p2.c, prt2_l_p3_p2.a
+                           Filter: ((prt2_l_p3_p2.b % 25) = 0)
+(70 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_l t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_l t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.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)
+
+--
+-- 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;
+ANALYZE plt1_p1;
+ANALYZE plt1_p2;
+ANALYZE plt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1 AS SELECT * FROM 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;
+ANALYZE plt2_p1;
+ANALYZE plt2_p2;
+ANALYZE plt2_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt2 AS SELECT * FROM plt2;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.a = t2.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((t2.c = t1.c) AND (t2.a = t1.a))
+               ->  Seq Scan on public.plt2_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.plt1_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((t2_1.c = t1_1.c) AND (t2_1.a = t1_1.a))
+               ->  Seq Scan on public.plt2_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.plt1_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((t2_2.c = t1_2.c) AND (t2_2.a = t1_2.a))
+               ->  Seq Scan on public.plt2_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.plt1_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.a = t2.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0003 | 150 | 0003
+ 300 | 0006 | 300 | 0006
+ 450 | 0009 | 450 | 0009
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1 t1, uplt2 t2 WHERE t1.c = t2.c AND t1.a = t2.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0003 | 150 | 0003
+ 300 | 0006 | 300 | 0006
+ 450 | 0009 | 450 | 0009
+(4 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.a AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+               ->  Seq Scan on public.plt2_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.plt1_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on public.plt2_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.plt1_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on public.plt2_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.plt1_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.a AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  50 | 0001 |     | 
+ 100 | 0002 |     | 
+ 150 | 0003 | 150 | 0003
+ 200 | 0004 |     | 
+ 250 | 0005 |     | 
+ 300 | 0006 | 300 | 0006
+ 350 | 0007 |     | 
+ 400 | 0008 |     | 
+ 450 | 0009 | 450 | 0009
+ 500 | 0010 |     | 
+ 550 | 0011 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.a AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  50 | 0001 |     | 
+ 100 | 0002 |     | 
+ 150 | 0003 | 150 | 0003
+ 200 | 0004 |     | 
+ 250 | 0005 |     | 
+ 300 | 0006 | 300 | 0006
+ 350 | 0007 |     | 
+ 400 | 0008 |     | 
+ 450 | 0009 | 450 | 0009
+ 500 | 0010 |     | 
+ 550 | 0011 |     | 
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+                                QUERY PLAN                                
+--------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: ((t1.a = t2.b) AND (t1.c = t2.c))
+                     ->  Seq Scan on public.plt1_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t2.b, t2.c
+                           ->  Seq Scan on public.plt2_p1 t2
+                                 Output: t2.b, t2.c
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: ((t1_1.a = t2_1.b) AND (t1_1.c = t2_1.c))
+                     ->  Seq Scan on public.plt1_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c
+                           ->  Seq Scan on public.plt2_p2 t2_1
+                                 Output: t2_1.b, t2_1.c
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: ((t1_2.a = t2_2.b) AND (t1_2.c = t2_2.c))
+                     ->  Seq Scan on public.plt1_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c
+                           ->  Seq Scan on public.plt2_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+                                 Filter: ((t2_2.b % 25) = 0)
+(36 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0003 | 150 | 0003
+ 300 | 0006 | 300 | 0006
+ 450 | 0009 | 450 | 0009
+     |      |  75 | 0001
+     |      | 225 | 0004
+     |      | 375 | 0007
+     |      | 525 | 0010
+(8 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0003 | 150 | 0003
+ 300 | 0006 | 300 | 0006
+ 450 | 0009 | 450 | 0009
+     |      |  75 | 0001
+     |      | 225 | 0004
+     |      | 375 | 0007
+     |      | 525 | 0010
+(8 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                   QUERY PLAN                                   
+--------------------------------------------------------------------------------
+ Sort
+   Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+   Sort Key: plt1_p1.a, plt2_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+               Hash Cond: ((plt1_p1.a = plt2_p1.b) AND (plt1_p1.c = plt2_p1.c))
+               ->  Seq Scan on public.plt1_p1
+                     Output: plt1_p1.a, plt1_p1.c
+                     Filter: ((plt1_p1.a % 25) = 0)
+               ->  Hash
+                     Output: plt2_p1.b, plt2_p1.c
+                     ->  Seq Scan on public.plt2_p1
+                           Output: plt2_p1.b, plt2_p1.c
+                           Filter: ((plt2_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+               Hash Cond: ((plt1_p2.a = plt2_p2.b) AND (plt1_p2.c = plt2_p2.c))
+               ->  Seq Scan on public.plt1_p2
+                     Output: plt1_p2.a, plt1_p2.c
+                     Filter: ((plt1_p2.a % 25) = 0)
+               ->  Hash
+                     Output: plt2_p2.b, plt2_p2.c
+                     ->  Seq Scan on public.plt2_p2
+                           Output: plt2_p2.b, plt2_p2.c
+                           Filter: ((plt2_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+               Hash Cond: ((plt1_p3.a = plt2_p3.b) AND (plt1_p3.c = plt2_p3.c))
+               ->  Seq Scan on public.plt1_p3
+                     Output: plt1_p3.a, plt1_p3.c
+                     Filter: ((plt1_p3.a % 25) = 0)
+               ->  Hash
+                     Output: plt2_p3.b, plt2_p3.c
+                     ->  Seq Scan on public.plt2_p3
+                           Output: plt2_p3.b, plt2_p3.c
+                           Filter: ((plt2_p3.b % 25) = 0)
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  50 | 0001 |     | 
+ 100 | 0002 |     | 
+ 150 | 0003 | 150 | 0003
+ 200 | 0004 |     | 
+ 250 | 0005 |     | 
+ 300 | 0006 | 300 | 0006
+ 350 | 0007 |     | 
+ 400 | 0008 |     | 
+ 450 | 0009 | 450 | 0009
+ 500 | 0010 |     | 
+ 550 | 0011 |     | 
+     |      |  75 | 0001
+     |      | 225 | 0004
+     |      | 375 | 0007
+     |      | 525 | 0010
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uplt1 t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  50 | 0001 |     | 
+ 100 | 0002 |     | 
+ 150 | 0003 | 150 | 0003
+ 200 | 0004 |     | 
+ 250 | 0005 |     | 
+ 300 | 0006 | 300 | 0006
+ 350 | 0007 |     | 
+ 400 | 0008 |     | 
+ 450 | 0009 | 450 | 0009
+ 500 | 0010 |     | 
+ 550 | 0011 |     | 
+     |      |  75 | 0001
+     |      | 225 | 0004
+     |      | 375 | 0007
+     |      | 525 | 0010
+(16 rows)
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM (SELECT 50 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ GroupAggregate
+   Output: sum(plt1_p1.a), plt1_p1.c, avg(plt2_p1.b), plt2_p1.c
+   Group Key: plt1_p1.c, plt2_p1.c
+   ->  Sort
+         Output: plt1_p1.c, plt2_p1.c, plt1_p1.a, plt2_p1.b
+         Sort Key: plt1_p1.c, plt2_p1.c
+         ->  Result
+               Output: plt1_p1.c, plt2_p1.c, plt1_p1.a, plt2_p1.b
+               ->  Append
+                     ->  Hash Full Join
+                           Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+                           Hash Cond: ((plt1_p1.c = plt2_p1.c) AND (plt1_p1.a = plt2_p1.b))
+                           ->  Seq Scan on public.plt1_p1
+                                 Output: plt1_p1.a, plt1_p1.c
+                                 Filter: ((plt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p1.b, plt2_p1.c
+                                 ->  Seq Scan on public.plt2_p1
+                                       Output: plt2_p1.b, plt2_p1.c
+                                       Filter: ((plt2_p1.b % 25) = 0)
+                     ->  Hash Full Join
+                           Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+                           Hash Cond: ((plt1_p2.c = plt2_p2.c) AND (plt1_p2.a = plt2_p2.b))
+                           ->  Seq Scan on public.plt1_p2
+                                 Output: plt1_p2.a, plt1_p2.c
+                                 Filter: ((plt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p2.b, plt2_p2.c
+                                 ->  Seq Scan on public.plt2_p2
+                                       Output: plt2_p2.b, plt2_p2.c
+                                       Filter: ((plt2_p2.b % 25) = 0)
+                     ->  Hash Full Join
+                           Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+                           Hash Cond: ((plt1_p3.c = plt2_p3.c) AND (plt1_p3.a = plt2_p3.b))
+                           ->  Seq Scan on public.plt1_p3
+                                 Output: plt1_p3.a, plt1_p3.c
+                                 Filter: ((plt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p3.b, plt2_p3.c
+                                 ->  Seq Scan on public.plt2_p3
+                                       Output: plt2_p3.b, plt2_p3.c
+                                       Filter: ((plt2_p3.b % 25) = 0)
+(42 rows)
+
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM (SELECT 50 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+ sum |  c   |          avg           |  c   
+-----+------+------------------------+------
+   0 | 0000 | 0.00000000000000000000 | 0000
+  50 | 0001 |                        | 
+ 100 | 0002 |                        | 
+ 150 | 0003 |   150.0000000000000000 | 0003
+ 200 | 0004 |                        | 
+ 250 | 0005 |                        | 
+ 300 | 0006 |   300.0000000000000000 | 0006
+ 350 | 0007 |                        | 
+ 400 | 0008 |                        | 
+ 450 | 0009 |   450.0000000000000000 | 0009
+ 500 | 0010 |                        | 
+ 550 | 0011 |                        | 
+     |      |    75.0000000000000000 | 0001
+     |      |   225.0000000000000000 | 0004
+     |      |   375.0000000000000000 | 0007
+     |      |   525.0000000000000000 | 0010
+(16 rows)
+
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM (SELECT 50 phv, * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+ sum |  c   |          avg           |  c   
+-----+------+------------------------+------
+   0 | 0000 | 0.00000000000000000000 | 0000
+  50 | 0001 |                        | 
+ 100 | 0002 |                        | 
+ 150 | 0003 |   150.0000000000000000 | 0003
+ 200 | 0004 |                        | 
+ 250 | 0005 |                        | 
+ 300 | 0006 |   300.0000000000000000 | 0006
+ 350 | 0007 |                        | 
+ 400 | 0008 |                        | 
+ 450 | 0009 |   450.0000000000000000 | 0009
+ 500 | 0010 |                        | 
+ 550 | 0011 |                        | 
+     |      |    75.0000000000000000 | 0001
+     |      |   225.0000000000000000 | 0004
+     |      |   375.0000000000000000 | 0007
+     |      |   525.0000000000000000 | 0010
+(16 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t1.phv), avg(t2.b), t2.c, avg(t2.phv) FROM (SELECT 25 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ GroupAggregate
+   Output: sum(plt1_p1.a), plt1_p1.c, sum((25)), avg(plt2_p1.b), plt2_p1.c, avg((50))
+   Group Key: plt1_p1.c, plt2_p1.c
+   ->  Sort
+         Output: plt1_p1.c, plt2_p1.c, plt1_p1.a, (25), plt2_p1.b, (50)
+         Sort Key: plt1_p1.c, plt2_p1.c
+         ->  Result
+               Output: plt1_p1.c, plt2_p1.c, plt1_p1.a, (25), plt2_p1.b, (50)
+               ->  Append
+                     ->  Hash Full Join
+                           Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, (25), (50)
+                           Hash Cond: ((plt1_p1.c = plt2_p1.c) AND (plt1_p1.a = plt2_p1.b))
+                           ->  Seq Scan on public.plt1_p1
+                                 Output: plt1_p1.a, plt1_p1.c, 25
+                                 Filter: ((plt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p1.b, plt2_p1.c, (50)
+                                 ->  Seq Scan on public.plt2_p1
+                                       Output: plt2_p1.b, plt2_p1.c, 50
+                                       Filter: ((plt2_p1.b % 25) = 0)
+                     ->  Hash Full Join
+                           Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c, (25), (50)
+                           Hash Cond: ((plt1_p2.c = plt2_p2.c) AND (plt1_p2.a = plt2_p2.b))
+                           ->  Seq Scan on public.plt1_p2
+                                 Output: plt1_p2.a, plt1_p2.c, 25
+                                 Filter: ((plt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p2.b, plt2_p2.c, (50)
+                                 ->  Seq Scan on public.plt2_p2
+                                       Output: plt2_p2.b, plt2_p2.c, 50
+                                       Filter: ((plt2_p2.b % 25) = 0)
+                     ->  Hash Full Join
+                           Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c, (25), (50)
+                           Hash Cond: ((plt1_p3.c = plt2_p3.c) AND (plt1_p3.a = plt2_p3.b))
+                           ->  Seq Scan on public.plt1_p3
+                                 Output: plt1_p3.a, plt1_p3.c, 25
+                                 Filter: ((plt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p3.b, plt2_p3.c, (50)
+                                 ->  Seq Scan on public.plt2_p3
+                                       Output: plt2_p3.b, plt2_p3.c, 50
+                                       Filter: ((plt2_p3.b % 25) = 0)
+(42 rows)
+
+SELECT sum(t1.a), t1.c, sum(t1.phv), avg(t2.b), t2.c, avg(t2.phv) FROM (SELECT 25 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+ sum |  c   | sum |          avg           |  c   |         avg         
+-----+------+-----+------------------------+------+---------------------
+   0 | 0000 |  25 | 0.00000000000000000000 | 0000 | 50.0000000000000000
+  50 | 0001 |  25 |                        |      |                    
+ 100 | 0002 |  25 |                        |      |                    
+ 150 | 0003 |  25 |   150.0000000000000000 | 0003 | 50.0000000000000000
+ 200 | 0004 |  25 |                        |      |                    
+ 250 | 0005 |  25 |                        |      |                    
+ 300 | 0006 |  25 |   300.0000000000000000 | 0006 | 50.0000000000000000
+ 350 | 0007 |  25 |                        |      |                    
+ 400 | 0008 |  25 |                        |      |                    
+ 450 | 0009 |  25 |   450.0000000000000000 | 0009 | 50.0000000000000000
+ 500 | 0010 |  25 |                        |      |                    
+ 550 | 0011 |  25 |                        |      |                    
+     |      |     |    75.0000000000000000 | 0001 | 50.0000000000000000
+     |      |     |   225.0000000000000000 | 0004 | 50.0000000000000000
+     |      |     |   375.0000000000000000 | 0007 | 50.0000000000000000
+     |      |     |   525.0000000000000000 | 0010 | 50.0000000000000000
+(16 rows)
+
+SELECT sum(t1.a), t1.c, sum(t1.phv), avg(t2.b), t2.c, avg(t2.phv) FROM (SELECT 25 phv, * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+ sum |  c   | sum |          avg           |  c   |         avg         
+-----+------+-----+------------------------+------+---------------------
+   0 | 0000 |  25 | 0.00000000000000000000 | 0000 | 50.0000000000000000
+  50 | 0001 |  25 |                        |      |                    
+ 100 | 0002 |  25 |                        |      |                    
+ 150 | 0003 |  25 |   150.0000000000000000 | 0003 | 50.0000000000000000
+ 200 | 0004 |  25 |                        |      |                    
+ 250 | 0005 |  25 |                        |      |                    
+ 300 | 0006 |  25 |   300.0000000000000000 | 0006 | 50.0000000000000000
+ 350 | 0007 |  25 |                        |      |                    
+ 400 | 0008 |  25 |                        |      |                    
+ 450 | 0009 |  25 |   450.0000000000000000 | 0009 | 50.0000000000000000
+ 500 | 0010 |  25 |                        |      |                    
+ 550 | 0011 |  25 |                        |      |                    
+     |      |     |    75.0000000000000000 | 0001 | 50.0000000000000000
+     |      |     |   225.0000000000000000 | 0004 | 50.0000000000000000
+     |      |     |   375.0000000000000000 | 0007 | 50.0000000000000000
+     |      |     |   525.0000000000000000 | 0010 | 50.0000000000000000
+(16 rows)
+
+-- Join with pruned partitions from joining relations
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.c NOT IN ('0001', '0005', '0002', '0009') AND t2.c NOT IN ('0000', '0003', '0004', '0010') GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Output: (sum(t1.a)), t1.c, (avg(t2.b)), t2.c
+   Sort Key: t1.c
+   ->  HashAggregate
+         Output: sum(t1.a), t1.c, avg(t2.b), t2.c
+         Group Key: t1.c, t2.c
+         ->  Result
+               Output: t1.c, t2.c, t1.a, t2.b
+               ->  Append
+                     ->  Hash Join
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           Hash Cond: (t1.c = t2.c)
+                           ->  Seq Scan on public.plt1_p3 t1
+                                 Output: t1.a, t1.c
+                                 Filter: (t1.c <> ALL ('{0001,0005,0002,0009}'::text[]))
+                           ->  Hash
+                                 Output: t2.b, t2.c
+                                 ->  Seq Scan on public.plt2_p3 t2
+                                       Output: t2.b, t2.c
+                                       Filter: (t2.c <> ALL ('{0000,0003,0004,0010}'::text[]))
+(20 rows)
+
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.c NOT IN ('0001', '0005', '0002', '0009') AND t2.c NOT IN ('0000', '0003', '0004', '0010') GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+  sum   |  c   |         avg          |  c   
+--------+------+----------------------+------
+ 137700 | 0006 | 324.0000000000000000 | 0006
+ 158950 | 0007 | 375.0000000000000000 | 0007
+ 169600 | 0008 | 424.5000000000000000 | 0008
+ 229600 | 0011 | 574.5000000000000000 | 0011
+(4 rows)
+
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM uplt1 t1, uplt2 t2 WHERE t1.c = t2.c AND t1.c NOT IN ('0001', '0005', '0002', '0009') AND t2.c NOT IN ('0000', '0003', '0004', '0010') GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+  sum   |  c   |         avg          |  c   
+--------+------+----------------------+------
+ 137700 | 0006 | 324.0000000000000000 | 0006
+ 158950 | 0007 | 375.0000000000000000 | 0007
+ 169600 | 0008 | 424.5000000000000000 | 0008
+ 229600 | 0011 | 574.5000000000000000 | 0011
+(4 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 LEFT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Output: (sum(t1.a)), t1.c, (sum(b)), c
+   Sort Key: t1.c, c
+   ->  HashAggregate
+         Output: sum(t1.a), t1.c, sum(b), c
+         Group Key: t1.c, c
+         ->  Result
+               Output: t1.c, c, t1.a, b
+               ->  Append
+                     ->  Hash Left Join
+                           Output: t1.a, t1.c, b, c
+                           Hash Cond: (t1.c = c)
+                           ->  Seq Scan on public.plt1_p1 t1
+                                 Output: t1.a, t1.c
+                                 Filter: (t1.c <> ALL ('{0001,0005,0002,0009}'::text[]))
+                           ->  Hash
+                                 Output: b, c
+                                 ->  Result
+                                       Output: b, c
+                                       One-Time Filter: false
+                     ->  Hash Left Join
+                           Output: t1_1.a, t1_1.c, t2.b, t2.c
+                           Hash Cond: (t1_1.c = t2.c)
+                           ->  Seq Scan on public.plt1_p3 t1_1
+                                 Output: t1_1.a, t1_1.c
+                                 Filter: (t1_1.c <> ALL ('{0001,0005,0002,0009}'::text[]))
+                           ->  Hash
+                                 Output: t2.b, t2.c
+                                 ->  Seq Scan on public.plt2_p3 t2
+                                       Output: t2.b, t2.c
+                                       Filter: (t2.c <> ALL ('{0000,0003,0004,0010}'::text[]))
+(31 rows)
+
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 LEFT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  sum   |  c   |  sum   |  c   
+--------+------+--------+------
+    600 | 0000 |        | 
+   4350 | 0003 |        | 
+   5600 | 0004 |        | 
+ 137700 | 0006 | 137700 | 0006
+ 158950 | 0007 | 159375 | 0007
+ 169600 | 0008 | 169800 | 0008
+  13100 | 0010 |        | 
+ 229600 | 0011 | 229800 | 0011
+(8 rows)
+
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM uplt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 LEFT JOIN (SELECT * FROM uplt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+  sum   |  c   |  sum   |  c   
+--------+------+--------+------
+    600 | 0000 |        | 
+   4350 | 0003 |        | 
+   5600 | 0004 |        | 
+ 137700 | 0006 | 137700 | 0006
+ 158950 | 0007 | 159375 | 0007
+ 169600 | 0008 | 169800 | 0008
+  13100 | 0010 |        | 
+ 229600 | 0011 | 229800 | 0011
+(8 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 RIGHT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+NOTICE:  join between relations (b 5) and (b 4) is considered for partition-wise join.
+                                           QUERY PLAN                                            
+-------------------------------------------------------------------------------------------------
+ Sort
+   Output: (sum(a)), c, (sum(t2.b)), t2.c
+   Sort Key: c, t2.c
+   ->  HashAggregate
+         Output: sum(a), c, sum(t2.b), t2.c
+         Group Key: c, t2.c
+         ->  Result
+               Output: c, t2.c, a, t2.b
+               ->  Append
+                     ->  Hash Left Join
+                           Output: t2.b, t2.c, a, c
+                           Hash Cond: (t2.c = c)
+                           ->  Seq Scan on public.plt2_p2 t2
+                                 Output: t2.b, t2.c
+                                 Filter: (t2.c <> ALL ('{0000,0003,0004,0010}'::text[]))
+                           ->  Hash
+                                 Output: a, c
+                                 ->  Result
+                                       Output: a, c
+                                       One-Time Filter: false
+                     ->  Hash Right Join
+                           Output: t2_1.b, t2_1.c, t1.a, t1.c
+                           Hash Cond: (t1.c = t2_1.c)
+                           ->  Seq Scan on public.plt1_p3 t1
+                                 Output: t1.a, t1.c
+                                 Filter: (t1.c <> ALL ('{0001,0005,0002,0009}'::text[]))
+                           ->  Hash
+                                 Output: t2_1.b, t2_1.c
+                                 ->  Seq Scan on public.plt2_p3 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                       Filter: (t2_1.c <> ALL ('{0000,0003,0004,0010}'::text[]))
+(31 rows)
+
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 RIGHT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+NOTICE:  join between relations (b 5) and (b 4) is considered for partition-wise join.
+  sum   |  c   |  sum   |  c   
+--------+------+--------+------
+ 137700 | 0006 | 137700 | 0006
+ 158950 | 0007 | 159375 | 0007
+ 169600 | 0008 | 169800 | 0008
+ 229600 | 0011 | 229800 | 0011
+        |      |   1275 | 0001
+        |      |   1992 | 0002
+        |      |   4392 | 0005
+        |      |   8058 | 0009
+(8 rows)
+
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM uplt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 RIGHT JOIN (SELECT * FROM uplt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+  sum   |  c   |  sum   |  c   
+--------+------+--------+------
+ 137700 | 0006 | 137700 | 0006
+ 158950 | 0007 | 159375 | 0007
+ 169600 | 0008 | 169800 | 0008
+ 229600 | 0011 | 229800 | 0011
+        |      |   1275 | 0001
+        |      |   1992 | 0002
+        |      |   4392 | 0005
+        |      |   8058 | 0009
+(8 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 FULL JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                           QUERY PLAN                                            
+-------------------------------------------------------------------------------------------------
+ Sort
+   Output: (sum(t1.a)), t1.c, (sum(b)), c
+   Sort Key: t1.c, c
+   ->  HashAggregate
+         Output: sum(t1.a), t1.c, sum(b), c
+         Group Key: t1.c, c
+         ->  Result
+               Output: t1.c, c, t1.a, b
+               ->  Append
+                     ->  Hash Full Join
+                           Output: t1.a, t1.c, b, c
+                           Hash Cond: (t1.c = c)
+                           ->  Seq Scan on public.plt1_p1 t1
+                                 Output: t1.a, t1.c
+                                 Filter: (t1.c <> ALL ('{0001,0005,0002,0009}'::text[]))
+                           ->  Hash
+                                 Output: b, c
+                                 ->  Result
+                                       Output: b, c
+                                       One-Time Filter: false
+                     ->  Hash Full Join
+                           Output: a, c, t2.b, t2.c
+                           Hash Cond: (t2.c = c)
+                           ->  Seq Scan on public.plt2_p2 t2
+                                 Output: t2.b, t2.c
+                                 Filter: (t2.c <> ALL ('{0000,0003,0004,0010}'::text[]))
+                           ->  Hash
+                                 Output: a, c
+                                 ->  Result
+                                       Output: a, c
+                                       One-Time Filter: false
+                     ->  Hash Full Join
+                           Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                           Hash Cond: (t1_1.c = t2_1.c)
+                           ->  Seq Scan on public.plt1_p3 t1_1
+                                 Output: t1_1.a, t1_1.c
+                                 Filter: (t1_1.c <> ALL ('{0001,0005,0002,0009}'::text[]))
+                           ->  Hash
+                                 Output: t2_1.b, t2_1.c
+                                 ->  Seq Scan on public.plt2_p3 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                       Filter: (t2_1.c <> ALL ('{0000,0003,0004,0010}'::text[]))
+(42 rows)
+
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 FULL JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  sum   |  c   |  sum   |  c   
+--------+------+--------+------
+    600 | 0000 |        | 
+   4350 | 0003 |        | 
+   5600 | 0004 |        | 
+ 137700 | 0006 | 137700 | 0006
+ 158950 | 0007 | 159375 | 0007
+ 169600 | 0008 | 169800 | 0008
+  13100 | 0010 |        | 
+ 229600 | 0011 | 229800 | 0011
+        |      |   1275 | 0001
+        |      |   1992 | 0002
+        |      |   4392 | 0005
+        |      |   8058 | 0009
+(12 rows)
+
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM uplt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 FULL JOIN (SELECT * FROM uplt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+  sum   |  c   |  sum   |  c   
+--------+------+--------+------
+    600 | 0000 |        | 
+   4350 | 0003 |        | 
+   5600 | 0004 |        | 
+ 137700 | 0006 | 137700 | 0006
+ 158950 | 0007 | 159375 | 0007
+ 169600 | 0008 | 169800 | 0008
+  13100 | 0010 |        | 
+ 229600 | 0011 | 229800 | 0011
+        |      |   1275 | 0001
+        |      |   1992 | 0002
+        |      |   4392 | 0005
+        |      |   8058 | 0009
+(12 rows)
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 1) and (b 3) is considered for partition-wise join.
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.c = t1_3.c)
+               ->  Seq Scan on public.plt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_3.c
+                     ->  Seq Scan on public.plt2_p1 t1_3
+                           Output: t1_3.c
+                           Filter: ((t1_3.b % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.c = t1_4.c)
+               ->  Seq Scan on public.plt1_p2 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_4.c
+                     ->  Seq Scan on public.plt2_p2 t1_4
+                           Output: t1_4.c
+                           Filter: ((t1_4.b % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.c = t1_5.c)
+               ->  Seq Scan on public.plt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_5.c
+                     ->  Seq Scan on public.plt2_p3 t1_5
+                           Output: t1_5.c
+                           Filter: ((t1_5.b % 25) = 0)
+(37 rows)
+
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 1) and (b 3) is considered for partition-wise join.
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+(8 rows)
+
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+(8 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;
+ANALYZE plt1_e;
+ANALYZE plt1_e_p1;
+ANALYZE plt1_e_p2;
+ANALYZE plt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1_e AS SELECT * FROM plt1_e;
+CREATE TABLE plt2_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
+CREATE TABLE plt2_e_p1 PARTITION OF plt2_e FOR VALUES IN ('0000', '0003', '0004', '0010');
+CREATE TABLE plt2_e_p2 PARTITION OF plt2_e FOR VALUES IN ('0001', '0005', '0002', '0009');
+CREATE TABLE plt2_e_p3 PARTITION OF plt2_e FOR VALUES IN ('0006', '0007', '0008', '0011');
+INSERT INTO plt2_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE plt2_e;
+ANALYZE plt2_e_p1;
+ANALYZE plt2_e_p2;
+ANALYZE plt2_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt2_e AS SELECT * FROM plt2_e;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1, plt2_e t2 WHERE t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((t2.b = t1.a) AND (ltrim(t2.c, 'A'::text) = ltrim(t1.c, 'A'::text)))
+               ->  Seq Scan on public.plt2_e_p1 t2
+                     Output: t2.b, t2.c
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.plt1_e_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((t2_1.b = t1_1.a) AND (ltrim(t2_1.c, 'A'::text) = ltrim(t1_1.c, 'A'::text)))
+               ->  Seq Scan on public.plt2_e_p2 t2_1
+                     Output: t2_1.b, t2_1.c
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.plt1_e_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((t2_2.b = t1_2.a) AND (ltrim(t2_2.c, 'A'::text) = ltrim(t1_2.c, 'A'::text)))
+               ->  Seq Scan on public.plt2_e_p3 t2_2
+                     Output: t2_2.b, t2_2.c
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.plt1_e_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1, plt2_e t2 WHERE t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+ 150 | A0003 | 150 | A0003
+ 300 | A0006 | 300 | A0006
+ 450 | A0009 | 450 | A0009
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_e t1, uplt2_e t2 WHERE t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+ 150 | A0003 | 150 | A0003
+ 300 | A0006 | 300 | A0006
+ 450 | A0009 | 450 | A0009
+(4 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 LEFT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((t2.b = t1.a) AND (ltrim(t2.c, 'A'::text) = ltrim(t1.c, 'A'::text)))
+               ->  Seq Scan on public.plt2_e_p1 t2
+                     Output: t2.b, t2.c
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.plt1_e_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((t2_1.b = t1_1.a) AND (ltrim(t2_1.c, 'A'::text) = ltrim(t1_1.c, 'A'::text)))
+               ->  Seq Scan on public.plt2_e_p2 t2_1
+                     Output: t2_1.b, t2_1.c
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.plt1_e_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((t2_2.b = t1_2.a) AND (ltrim(t2_2.c, 'A'::text) = ltrim(t1_2.c, 'A'::text)))
+               ->  Seq Scan on public.plt2_e_p3 t2_2
+                     Output: t2_2.b, t2_2.c
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.plt1_e_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 LEFT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+  50 | A0001 |     | 
+ 100 | A0002 |     | 
+ 150 | A0003 | 150 | A0003
+ 200 | A0004 |     | 
+ 250 | A0005 |     | 
+ 300 | A0006 | 300 | A0006
+ 350 | A0007 |     | 
+ 400 | A0008 |     | 
+ 450 | A0009 | 450 | A0009
+ 500 | A0010 |     | 
+ 550 | A0011 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_e t1 LEFT JOIN uplt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+  50 | A0001 |     | 
+ 100 | A0002 |     | 
+ 150 | A0003 | 150 | A0003
+ 200 | A0004 |     | 
+ 250 | A0005 |     | 
+ 300 | A0006 | 300 | A0006
+ 350 | A0007 |     | 
+ 400 | A0008 |     | 
+ 450 | A0009 | 450 | A0009
+ 500 | A0010 |     | 
+ 550 | A0011 |     | 
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 RIGHT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+                                                  QUERY PLAN                                                  
+--------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: ((t1.a = t2.b) AND (ltrim(t1.c, 'A'::text) = ltrim(t2.c, 'A'::text)))
+                     ->  Seq Scan on public.plt1_e_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t2.b, t2.c
+                           ->  Seq Scan on public.plt2_e_p1 t2
+                                 Output: t2.b, t2.c
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: ((t1_1.a = t2_1.b) AND (ltrim(t1_1.c, 'A'::text) = ltrim(t2_1.c, 'A'::text)))
+                     ->  Seq Scan on public.plt1_e_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c
+                           ->  Seq Scan on public.plt2_e_p2 t2_1
+                                 Output: t2_1.b, t2_1.c
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: ((t1_2.a = t2_2.b) AND (ltrim(t1_2.c, 'A'::text) = ltrim(t2_2.c, 'A'::text)))
+                     ->  Seq Scan on public.plt1_e_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c
+                           ->  Seq Scan on public.plt2_e_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+                                 Filter: ((t2_2.b % 25) = 0)
+(36 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 RIGHT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+ 150 | A0003 | 150 | A0003
+ 300 | A0006 | 300 | A0006
+ 450 | A0009 | 450 | A0009
+     |       |  75 | A0001
+     |       | 225 | A0004
+     |       | 375 | A0007
+     |       | 525 | A0010
+(8 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_e t1 RIGHT JOIN uplt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+ 150 | A0003 | 150 | A0003
+ 300 | A0006 | 300 | A0006
+ 450 | A0009 | 450 | A0009
+     |       |  75 | A0001
+     |       | 225 | A0004
+     |       | 375 | A0007
+     |       | 525 | A0010
+(8 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2_e WHERE plt2_e.b % 25 = 0) t2 ON (t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A')) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                                         QUERY PLAN                                                         
+----------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: plt1_e_p1.a, plt1_e_p1.c, plt2_e_p1.b, plt2_e_p1.c
+   Sort Key: plt1_e_p1.a, plt2_e_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: plt1_e_p1.a, plt1_e_p1.c, plt2_e_p1.b, plt2_e_p1.c
+               Hash Cond: ((plt1_e_p1.a = plt2_e_p1.b) AND (ltrim(plt1_e_p1.c, 'A'::text) = ltrim(plt2_e_p1.c, 'A'::text)))
+               ->  Seq Scan on public.plt1_e_p1
+                     Output: plt1_e_p1.a, plt1_e_p1.c
+                     Filter: ((plt1_e_p1.a % 25) = 0)
+               ->  Hash
+                     Output: plt2_e_p1.b, plt2_e_p1.c
+                     ->  Seq Scan on public.plt2_e_p1
+                           Output: plt2_e_p1.b, plt2_e_p1.c
+                           Filter: ((plt2_e_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: plt1_e_p2.a, plt1_e_p2.c, plt2_e_p2.b, plt2_e_p2.c
+               Hash Cond: ((plt1_e_p2.a = plt2_e_p2.b) AND (ltrim(plt1_e_p2.c, 'A'::text) = ltrim(plt2_e_p2.c, 'A'::text)))
+               ->  Seq Scan on public.plt1_e_p2
+                     Output: plt1_e_p2.a, plt1_e_p2.c
+                     Filter: ((plt1_e_p2.a % 25) = 0)
+               ->  Hash
+                     Output: plt2_e_p2.b, plt2_e_p2.c
+                     ->  Seq Scan on public.plt2_e_p2
+                           Output: plt2_e_p2.b, plt2_e_p2.c
+                           Filter: ((plt2_e_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: plt1_e_p3.a, plt1_e_p3.c, plt2_e_p3.b, plt2_e_p3.c
+               Hash Cond: ((plt1_e_p3.a = plt2_e_p3.b) AND (ltrim(plt1_e_p3.c, 'A'::text) = ltrim(plt2_e_p3.c, 'A'::text)))
+               ->  Seq Scan on public.plt1_e_p3
+                     Output: plt1_e_p3.a, plt1_e_p3.c
+                     Filter: ((plt1_e_p3.a % 25) = 0)
+               ->  Hash
+                     Output: plt2_e_p3.b, plt2_e_p3.c
+                     ->  Seq Scan on public.plt2_e_p3
+                           Output: plt2_e_p3.b, plt2_e_p3.c
+                           Filter: ((plt2_e_p3.b % 25) = 0)
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2_e WHERE plt2_e.b % 25 = 0) t2 ON (t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A')) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+  50 | A0001 |     | 
+ 100 | A0002 |     | 
+ 150 | A0003 | 150 | A0003
+ 200 | A0004 |     | 
+ 250 | A0005 |     | 
+ 300 | A0006 | 300 | A0006
+ 350 | A0007 |     | 
+ 400 | A0008 |     | 
+ 450 | A0009 | 450 | A0009
+ 500 | A0010 |     | 
+ 550 | A0011 |     | 
+     |       |  75 | A0001
+     |       | 225 | A0004
+     |       | 375 | A0007
+     |       | 525 | A0010
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uplt1_e t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2_e t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A')) ORDER BY t1.a, t2.b;
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+  50 | A0001 |     | 
+ 100 | A0002 |     | 
+ 150 | A0003 | 150 | A0003
+ 200 | A0004 |     | 
+ 250 | A0005 |     | 
+ 300 | A0006 | 300 | A0006
+ 350 | A0007 |     | 
+ 400 | A0008 |     | 
+ 450 | A0009 | 450 | A0009
+ 500 | A0010 |     | 
+ 550 | A0011 |     | 
+     |       |  75 | A0001
+     |       | 225 | A0004
+     |       | 375 | A0007
+     |       | 525 | A0010
+(16 rows)
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 3) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 3) and (b 1) is considered for partition-wise join.
+                                        QUERY PLAN                                        
+------------------------------------------------------------------------------------------
+ Sort
+   Output: (avg(t1.a)), (avg(t2.b)), (avg((t3.a + t3.b))), t1.c, t2.c, t3.c
+   Sort Key: t1.c, t3.c
+   ->  HashAggregate
+         Output: avg(t1.a), avg(t2.b), avg((t3.a + t3.b)), t1.c, t2.c, t3.c
+         Group Key: t1.c, t2.c, t3.c
+         ->  Result
+               Output: t1.c, t2.c, t3.c, t1.a, t2.b, t3.a, t3.b
+               ->  Append
+                     ->  Hash Join
+                           Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                           Hash Cond: (t1.c = t2.c)
+                           ->  Seq Scan on public.plt1_p1 t1
+                                 Output: t1.a, t1.c
+                           ->  Hash
+                                 Output: t2.b, t2.c, t3.a, t3.b, t3.c
+                                 ->  Hash Join
+                                       Output: t2.b, t2.c, t3.a, t3.b, t3.c
+                                       Hash Cond: (t2.c = ltrim(t3.c, 'A'::text))
+                                       ->  Seq Scan on public.plt2_p1 t2
+                                             Output: t2.b, t2.c
+                                       ->  Hash
+                                             Output: t3.a, t3.b, t3.c
+                                             ->  Seq Scan on public.plt1_e_p1 t3
+                                                   Output: t3.a, t3.b, t3.c
+                     ->  Hash Join
+                           Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                           Hash Cond: (t1_1.c = t2_1.c)
+                           ->  Seq Scan on public.plt1_p2 t1_1
+                                 Output: t1_1.a, t1_1.c
+                           ->  Hash
+                                 Output: t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                                 ->  Hash Join
+                                       Output: t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                                       Hash Cond: (t2_1.c = ltrim(t3_1.c, 'A'::text))
+                                       ->  Seq Scan on public.plt2_p2 t2_1
+                                             Output: t2_1.b, t2_1.c
+                                       ->  Hash
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             ->  Seq Scan on public.plt1_e_p2 t3_1
+                                                   Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash Join
+                           Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                           Hash Cond: (t1_2.c = t2_2.c)
+                           ->  Seq Scan on public.plt1_p3 t1_2
+                                 Output: t1_2.a, t1_2.c
+                           ->  Hash
+                                 Output: t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                                 ->  Hash Join
+                                       Output: t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                                       Hash Cond: (t2_2.c = ltrim(t3_2.c, 'A'::text))
+                                       ->  Seq Scan on public.plt2_p3 t2_2
+                                             Output: t2_2.b, t2_2.c
+                                       ->  Hash
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             ->  Seq Scan on public.plt1_e_p3 t3_2
+                                                   Output: t3_2.a, t3_2.b, t3_2.c
+(57 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;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 3) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 3) and (b 1) is considered for partition-wise join.
+         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)
+
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM uplt1 t1, uplt2 t2, uplt1_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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: ((t3.a = t1.a) AND (ltrim(t3.c, 'A'::text) = t1.c))
+                     ->  Seq Scan on public.plt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                                 ->  Seq Scan on public.plt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.plt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: ((t3_1.a = t1_1.a) AND (ltrim(t3_1.c, 'A'::text) = t1_1.c))
+                     ->  Seq Scan on public.plt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.c = t1_1.c))
+                                 ->  Seq Scan on public.plt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.plt1_p2 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: ((t3_2.a = t1_2.a) AND (ltrim(t3_2.c, 'A'::text) = t1_2.c))
+                     ->  Seq Scan on public.plt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                                 ->  Seq Scan on public.plt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.plt1_p3 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2 4) is considered for partition-wise join.
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: ((t3.a = t2.b) AND (ltrim(t3.c, 'A'::text) = t2.c))
+                     ->  Seq Scan on public.plt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                                 ->  Seq Scan on public.plt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.plt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: ((t3_1.a = t2_1.b) AND (ltrim(t3_1.c, 'A'::text) = t2_1.c))
+                     ->  Seq Scan on public.plt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.c = t1_1.c))
+                                 ->  Seq Scan on public.plt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.plt1_p2 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: ((t3_2.a = t2_2.b) AND (ltrim(t3_2.c, 'A'::text) = t2_2.c))
+                     ->  Seq Scan on public.plt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                                 ->  Seq Scan on public.plt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.plt1_p3 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2 4) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |          | 
+ 100 | 0002 |     |      |          | 
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |          | 
+ 250 | 0005 |     |      |          | 
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |          | 
+ 400 | 0008 |     |      |          | 
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |          | 
+ 550 | 0011 |     |      |          | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |          | 
+ 100 | 0002 |     |      |          | 
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |          | 
+ 250 | 0005 |     |      |          | 
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |          | 
+ 400 | 0008 |     |      |          | 
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |          | 
+ 550 | 0011 |     |      |          | 
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Hash Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                     ->  Seq Scan on public.plt2_p1 t2
+                           Output: t2.b, t2.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Hash Cond: ((t1.c = ltrim(t3.c, 'A'::text)) AND (t1.a = t3.a))
+                                 ->  Seq Scan on public.plt1_p1 t1
+                                       Output: t1.a, t1.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.plt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                     Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.c = t1_1.c))
+                     ->  Seq Scan on public.plt2_p2 t2_1
+                           Output: t2_1.b, t2_1.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_1.a, t1_1.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_1.a, t1_1.c
+                                 Hash Cond: ((t1_1.c = ltrim(t3_1.c, 'A'::text)) AND (t1_1.a = t3_1.a))
+                                 ->  Seq Scan on public.plt1_p2 t1_1
+                                       Output: t1_1.a, t1_1.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.plt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                     Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                     ->  Seq Scan on public.plt2_p3 t2_2
+                           Output: t2_2.b, t2_2.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_2.a, t1_2.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_2.a, t1_2.c
+                                 Hash Cond: ((t1_2.c = ltrim(t3_2.c, 'A'::text)) AND (t1_2.a = t3_2.a))
+                                 ->  Seq Scan on public.plt1_p3 t1_2
+                                       Output: t1_2.a, t1_2.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.plt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: ((t1.a = t2.b) AND (t1.c = t2.c))
+                     ->  Seq Scan on public.plt1_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Hash Cond: ((t2.b = t3.a) AND (t2.c = ltrim(t3.c, 'A'::text)))
+                                 ->  Seq Scan on public.plt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.plt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: ((t1_1.a = t2_1.b) AND (t1_1.c = t2_1.c))
+                     ->  Seq Scan on public.plt1_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: ((t2_1.b = t3_1.a) AND (t2_1.c = ltrim(t3_1.c, 'A'::text)))
+                                 ->  Seq Scan on public.plt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.plt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: ((t1_2.a = t2_2.b) AND (t1_2.c = t2_2.c))
+                     ->  Seq Scan on public.plt1_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: ((t2_2.b = t3_2.a) AND (t2_2.c = ltrim(t3_2.c, 'A'::text)))
+                                 ->  Seq Scan on public.plt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.plt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+     |      |     |      |      100 | A0001
+     |      |     |      |      200 | A0002
+     |      |     |      |      400 | A0004
+     |      |     |      |      500 | A0005
+     |      |     |      |      700 | A0007
+     |      |     |      |      800 | A0008
+     |      |     |      |     1000 | A0010
+     |      |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+     |      |     |      |      100 | A0001
+     |      |     |      |      200 | A0002
+     |      |     |      |      400 | A0004
+     |      |     |      |      500 | A0005
+     |      |     |      |      700 | A0007
+     |      |     |      |      800 | A0008
+     |      |     |      |     1000 | A0010
+     |      |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 6) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is considered for partition-wise join.
+                                                  QUERY PLAN                                                   
+---------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, ((plt1_e_p1.a + plt1_e_p1.b)), plt1_e_p1.c
+   Sort Key: plt1_p1.a, plt2_p1.b, ((plt1_e_p1.a + plt1_e_p1.b))
+   ->  Result
+         Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, (plt1_e_p1.a + plt1_e_p1.b), plt1_e_p1.c
+         ->  Append
+               ->  Hash Full Join
+                     Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                     Hash Cond: ((plt1_p1.a = plt1_e_p1.a) AND (plt1_p1.c = ltrim(plt1_e_p1.c, 'A'::text)))
+                     ->  Hash Full Join
+                           Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+                           Hash Cond: ((plt1_p1.a = plt2_p1.b) AND (plt1_p1.c = plt2_p1.c))
+                           ->  Seq Scan on public.plt1_p1
+                                 Output: plt1_p1.a, plt1_p1.c
+                                 Filter: ((plt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p1.b, plt2_p1.c
+                                 ->  Seq Scan on public.plt2_p1
+                                       Output: plt2_p1.b, plt2_p1.c
+                                       Filter: ((plt2_p1.b % 25) = 0)
+                     ->  Hash
+                           Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                           ->  Seq Scan on public.plt1_e_p1
+                                 Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                                 Filter: ((plt1_e_p1.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c, plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                     Hash Cond: ((plt1_p2.a = plt1_e_p2.a) AND (plt1_p2.c = ltrim(plt1_e_p2.c, 'A'::text)))
+                     ->  Hash Full Join
+                           Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+                           Hash Cond: ((plt1_p2.a = plt2_p2.b) AND (plt1_p2.c = plt2_p2.c))
+                           ->  Seq Scan on public.plt1_p2
+                                 Output: plt1_p2.a, plt1_p2.c
+                                 Filter: ((plt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p2.b, plt2_p2.c
+                                 ->  Seq Scan on public.plt2_p2
+                                       Output: plt2_p2.b, plt2_p2.c
+                                       Filter: ((plt2_p2.b % 25) = 0)
+                     ->  Hash
+                           Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                           ->  Seq Scan on public.plt1_e_p2
+                                 Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                                 Filter: ((plt1_e_p2.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c, plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                     Hash Cond: ((plt1_p3.a = plt1_e_p3.a) AND (plt1_p3.c = ltrim(plt1_e_p3.c, 'A'::text)))
+                     ->  Hash Full Join
+                           Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+                           Hash Cond: ((plt1_p3.a = plt2_p3.b) AND (plt1_p3.c = plt2_p3.c))
+                           ->  Seq Scan on public.plt1_p3
+                                 Output: plt1_p3.a, plt1_p3.c
+                                 Filter: ((plt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p3.b, plt2_p3.c
+                                 ->  Seq Scan on public.plt2_p3
+                                       Output: plt2_p3.b, plt2_p3.c
+                                       Filter: ((plt2_p3.b % 25) = 0)
+                     ->  Hash
+                           Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                           ->  Seq Scan on public.plt1_e_p3
+                                 Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                                 Filter: ((plt1_e_p3.a % 25) = 0)
+(63 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 6) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 4) is considered for partition-wise join.
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.c = t1_3.c)
+               ->  Seq Scan on public.plt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_3.c, t2.c
+                     Hash Cond: (t1_3.c = ltrim(t2.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p1 t1_3
+                           Output: t1_3.c
+                     ->  Hash
+                           Output: t2.c
+                           ->  Seq Scan on public.plt1_e_p1 t2
+                                 Output: t2.c
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.c = t1_4.c)
+               ->  Seq Scan on public.plt1_p2 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_4.c, t2_1.c
+                     Hash Cond: (t1_4.c = ltrim(t2_1.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p2 t1_4
+                           Output: t1_4.c
+                     ->  Hash
+                           Output: t2_1.c
+                           ->  Seq Scan on public.plt1_e_p2 t2_1
+                                 Output: t2_1.c
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.c = t1_5.c)
+               ->  Seq Scan on public.plt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_5.c, t2_2.c
+                     Hash Cond: (t1_5.c = ltrim(t2_2.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p3 t1_5
+                           Output: t1_5.c
+                     ->  Hash
+                           Output: t2_2.c
+                           ->  Seq Scan on public.plt1_e_p3 t2_2
+                                 Output: t2_2.c
+(49 rows)
+
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 4) is considered for partition-wise join.
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1, uplt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 5) is considered for partition-wise join.
+                                   QUERY PLAN                                   
+--------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.c = t1_3.c)
+               ->  Seq Scan on public.plt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_3.c, t1_6.c
+                     Hash Cond: (t1_3.c = ltrim(t1_6.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p1 t1_3
+                           Output: t1_3.c
+                     ->  Hash
+                           Output: t1_6.c
+                           ->  HashAggregate
+                                 Output: t1_6.c
+                                 Group Key: ltrim(t1_6.c, 'A'::text)
+                                 ->  Seq Scan on public.plt1_e_p1 t1_6
+                                       Output: t1_6.c, ltrim(t1_6.c, 'A'::text)
+                                       Filter: ((t1_6.a % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.c = t1_4.c)
+               ->  Seq Scan on public.plt1_p2 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_4.c, t1_7.c
+                     Hash Cond: (t1_4.c = ltrim(t1_7.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p2 t1_4
+                           Output: t1_4.c
+                     ->  Hash
+                           Output: t1_7.c
+                           ->  HashAggregate
+                                 Output: t1_7.c
+                                 Group Key: ltrim(t1_7.c, 'A'::text)
+                                 ->  Seq Scan on public.plt1_e_p2 t1_7
+                                       Output: t1_7.c, ltrim(t1_7.c, 'A'::text)
+                                       Filter: ((t1_7.a % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.c = t1_5.c)
+               ->  Seq Scan on public.plt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_5.c, t1_8.c
+                     Hash Cond: (t1_5.c = ltrim(t1_8.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p3 t1_5
+                           Output: t1_5.c
+                     ->  Hash
+                           Output: t1_8.c
+                           ->  HashAggregate
+                                 Output: t1_8.c
+                                 Group Key: ltrim(t1_8.c, 'A'::text)
+                                 ->  Seq Scan on public.plt1_e_p3 t1_8
+                                       Output: t1_8.c, ltrim(t1_8.c, 'A'::text)
+                                       Filter: ((t1_8.a % 25) = 0)
+(61 rows)
+
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 5) is considered for partition-wise join.
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+-- test merge join with and without index scan
+CREATE INDEX iplt1_c on plt1(c);
+CREATE INDEX iplt1_p1_c on plt1_p1(c);
+CREATE INDEX iplt1_p2_c on plt1_p2(c);
+CREATE INDEX iplt1_p3_c on plt1_p3(c);
+CREATE INDEX iplt2_c on plt2(c);
+CREATE INDEX iplt2_p1_c on plt2_p1(c);
+CREATE INDEX iplt2_p2_c on plt2_p2(c);
+CREATE INDEX iplt2_p3_c on plt2_p3(c);
+CREATE INDEX iplt1_e_c on plt1_e(ltrim(c, 'A'));
+CREATE INDEX iplt1_e_p1_c on plt1_e_p1(ltrim(c, 'A'));
+CREATE INDEX iplt1_e_p2_c on plt1_e_p2(ltrim(c, 'A'));
+CREATE INDEX iplt1_e_p3_c on plt1_e_p3(ltrim(c, 'A'));
+ANALYZE plt1;
+ANALYZE plt1_p1;
+ANALYZE plt1_p2;
+ANALYZE plt1_p3;
+ANALYZE plt2;
+ANALYZE plt2_p1;
+ANALYZE plt2_p2;
+ANALYZE plt2_p3;
+ANALYZE plt1_e;
+ANALYZE plt1_e_p1;
+ANALYZE plt1_e_p2;
+ANALYZE plt1_e_p3;
+SET enable_hashjoin TO off;
+SET enable_nestloop TO off;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+                                                QUERY PLAN                                                 
+-----------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Merge Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           Sort Key: t2.b, t2.c
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Merge Cond: ((t3.a = t2.b) AND ((ltrim(t3.c, 'A'::text)) = t2.c))
+                                 ->  Sort
+                                       Output: t3.a, t3.b, t3.c, (ltrim(t3.c, 'A'::text))
+                                       Sort Key: t3.a, (ltrim(t3.c, 'A'::text))
+                                       ->  Seq Scan on public.plt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c, ltrim(t3.c, 'A'::text)
+                                             Filter: ((t3.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2.b, t2.c
+                                       Sort Key: t2.b, t2.c
+                                       ->  Seq Scan on public.plt2_p1 t2
+                                             Output: t2.b, t2.c
+                     ->  Sort
+                           Output: t1.a, t1.c
+                           Sort Key: t1.a, t1.c
+                           ->  Seq Scan on public.plt1_p1 t1
+                                 Output: t1.a, t1.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Merge Cond: ((t2_1.c = t1_1.c) AND (t2_1.b = t1_1.a))
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           Sort Key: t2_1.c, t2_1.b
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Merge Cond: ((t3_1.a = t2_1.b) AND ((ltrim(t3_1.c, 'A'::text)) = t2_1.c))
+                                 ->  Sort
+                                       Output: t3_1.a, t3_1.b, t3_1.c, (ltrim(t3_1.c, 'A'::text))
+                                       Sort Key: t3_1.a, (ltrim(t3_1.c, 'A'::text))
+                                       ->  Seq Scan on public.plt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c, ltrim(t3_1.c, 'A'::text)
+                                             Filter: ((t3_1.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2_1.b, t2_1.c
+                                       Sort Key: t2_1.b, t2_1.c
+                                       ->  Seq Scan on public.plt2_p2 t2_1
+                                             Output: t2_1.b, t2_1.c
+                     ->  Sort
+                           Output: t1_1.a, t1_1.c
+                           Sort Key: t1_1.c, t1_1.a
+                           ->  Seq Scan on public.plt1_p2 t1_1
+                                 Output: t1_1.a, t1_1.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Merge Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           Sort Key: t2_2.b, t2_2.c
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Merge Cond: ((t3_2.a = t2_2.b) AND ((ltrim(t3_2.c, 'A'::text)) = t2_2.c))
+                                 ->  Sort
+                                       Output: t3_2.a, t3_2.b, t3_2.c, (ltrim(t3_2.c, 'A'::text))
+                                       Sort Key: t3_2.a, (ltrim(t3_2.c, 'A'::text))
+                                       ->  Seq Scan on public.plt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c, ltrim(t3_2.c, 'A'::text)
+                                             Filter: ((t3_2.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2_2.b, t2_2.c
+                                       Sort Key: t2_2.b, t2_2.c
+                                       ->  Seq Scan on public.plt2_p3 t2_2
+                                             Output: t2_2.b, t2_2.c
+                     ->  Sort
+                           Output: t1_2.a, t1_2.c
+                           Sort Key: t1_2.a, t1_2.c
+                           ->  Seq Scan on public.plt1_p3 t1_2
+                                 Output: t1_2.a, t1_2.c
+(81 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+     |      |     |      |      100 | A0001
+     |      |     |      |      200 | A0002
+     |      |     |      |      400 | A0004
+     |      |     |      |      500 | A0005
+     |      |     |      |      700 | A0007
+     |      |     |      |      800 | A0008
+     |      |     |      |     1000 | A0010
+     |      |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+     |      |     |      |      100 | A0001
+     |      |     |      |      200 | A0002
+     |      |     |      |      400 | A0004
+     |      |     |      |      500 | A0005
+     |      |     |      |      700 | A0007
+     |      |     |      |      800 | A0008
+     |      |     |      |     1000 | A0010
+     |      |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 6) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is considered for partition-wise join.
+                                                  QUERY PLAN                                                   
+---------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, ((plt1_e_p1.a + plt1_e_p1.b)), plt1_e_p1.c
+   Sort Key: plt1_p1.a, plt2_p1.b, ((plt1_e_p1.a + plt1_e_p1.b))
+   ->  Result
+         Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, (plt1_e_p1.a + plt1_e_p1.b), plt1_e_p1.c
+         ->  Append
+               ->  Merge Full Join
+                     Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                     Merge Cond: ((plt1_p1.a = plt1_e_p1.a) AND (plt1_p1.c = (ltrim(plt1_e_p1.c, 'A'::text))))
+                     ->  Sort
+                           Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+                           Sort Key: plt1_p1.a, plt1_p1.c
+                           ->  Merge Full Join
+                                 Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+                                 Merge Cond: ((plt1_p1.a = plt2_p1.b) AND (plt1_p1.c = plt2_p1.c))
+                                 ->  Sort
+                                       Output: plt1_p1.a, plt1_p1.c
+                                       Sort Key: plt1_p1.a, plt1_p1.c
+                                       ->  Seq Scan on public.plt1_p1
+                                             Output: plt1_p1.a, plt1_p1.c
+                                             Filter: ((plt1_p1.a % 25) = 0)
+                                 ->  Sort
+                                       Output: plt2_p1.b, plt2_p1.c
+                                       Sort Key: plt2_p1.b, plt2_p1.c
+                                       ->  Seq Scan on public.plt2_p1
+                                             Output: plt2_p1.b, plt2_p1.c
+                                             Filter: ((plt2_p1.b % 25) = 0)
+                     ->  Sort
+                           Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c, (ltrim(plt1_e_p1.c, 'A'::text))
+                           Sort Key: plt1_e_p1.a, (ltrim(plt1_e_p1.c, 'A'::text))
+                           ->  Seq Scan on public.plt1_e_p1
+                                 Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c, ltrim(plt1_e_p1.c, 'A'::text)
+                                 Filter: ((plt1_e_p1.a % 25) = 0)
+               ->  Merge Full Join
+                     Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c, plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                     Merge Cond: ((plt1_p2.a = plt1_e_p2.a) AND (plt1_p2.c = (ltrim(plt1_e_p2.c, 'A'::text))))
+                     ->  Sort
+                           Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+                           Sort Key: plt1_p2.a, plt1_p2.c
+                           ->  Merge Full Join
+                                 Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+                                 Merge Cond: ((plt1_p2.a = plt2_p2.b) AND (plt1_p2.c = plt2_p2.c))
+                                 ->  Sort
+                                       Output: plt1_p2.a, plt1_p2.c
+                                       Sort Key: plt1_p2.a, plt1_p2.c
+                                       ->  Seq Scan on public.plt1_p2
+                                             Output: plt1_p2.a, plt1_p2.c
+                                             Filter: ((plt1_p2.a % 25) = 0)
+                                 ->  Sort
+                                       Output: plt2_p2.b, plt2_p2.c
+                                       Sort Key: plt2_p2.b, plt2_p2.c
+                                       ->  Seq Scan on public.plt2_p2
+                                             Output: plt2_p2.b, plt2_p2.c
+                                             Filter: ((plt2_p2.b % 25) = 0)
+                     ->  Sort
+                           Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c, (ltrim(plt1_e_p2.c, 'A'::text))
+                           Sort Key: plt1_e_p2.a, (ltrim(plt1_e_p2.c, 'A'::text))
+                           ->  Seq Scan on public.plt1_e_p2
+                                 Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c, ltrim(plt1_e_p2.c, 'A'::text)
+                                 Filter: ((plt1_e_p2.a % 25) = 0)
+               ->  Merge Full Join
+                     Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c, plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                     Merge Cond: ((plt1_p3.a = plt1_e_p3.a) AND (plt1_p3.c = (ltrim(plt1_e_p3.c, 'A'::text))))
+                     ->  Sort
+                           Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+                           Sort Key: plt1_p3.a, plt1_p3.c
+                           ->  Merge Full Join
+                                 Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+                                 Merge Cond: ((plt1_p3.a = plt2_p3.b) AND (plt1_p3.c = plt2_p3.c))
+                                 ->  Sort
+                                       Output: plt1_p3.a, plt1_p3.c
+                                       Sort Key: plt1_p3.a, plt1_p3.c
+                                       ->  Seq Scan on public.plt1_p3
+                                             Output: plt1_p3.a, plt1_p3.c
+                                             Filter: ((plt1_p3.a % 25) = 0)
+                                 ->  Sort
+                                       Output: plt2_p3.b, plt2_p3.c
+                                       Sort Key: plt2_p3.b, plt2_p3.c
+                                       ->  Seq Scan on public.plt2_p3
+                                             Output: plt2_p3.b, plt2_p3.c
+                                             Filter: ((plt2_p3.b % 25) = 0)
+                     ->  Sort
+                           Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c, (ltrim(plt1_e_p3.c, 'A'::text))
+                           Sort Key: plt1_e_p3.a, (ltrim(plt1_e_p3.c, 'A'::text))
+                           ->  Seq Scan on public.plt1_e_p3
+                                 Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c, ltrim(plt1_e_p3.c, 'A'::text)
+                                 Filter: ((plt1_e_p3.a % 25) = 0)
+(87 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 6) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 5) is considered for partition-wise join.
+                                      QUERY PLAN                                      
+--------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Merge Semi Join
+               Output: t1.a, t1.b, t1.c
+               Merge Cond: (t1.c = t1_3.c)
+               ->  Sort
+                     Output: t1.a, t1.b, t1.c
+                     Sort Key: t1.c
+                     ->  Seq Scan on public.plt1_p1 t1
+                           Output: t1.a, t1.b, t1.c
+                           Filter: ((t1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_3.c, t1_6.c
+                     ->  Merge Join
+                           Output: t1_3.c, t1_6.c
+                           Merge Cond: ((ltrim(t1_6.c, 'A'::text)) = t1_3.c)
+                           ->  Sort
+                                 Output: t1_6.c, (ltrim(t1_6.c, 'A'::text))
+                                 Sort Key: (ltrim(t1_6.c, 'A'::text))
+                                 ->  HashAggregate
+                                       Output: t1_6.c, (ltrim(t1_6.c, 'A'::text))
+                                       Group Key: ltrim(t1_6.c, 'A'::text)
+                                       ->  Seq Scan on public.plt1_e_p1 t1_6
+                                             Output: t1_6.c, ltrim(t1_6.c, 'A'::text)
+                                             Filter: ((t1_6.a % 25) = 0)
+                           ->  Sort
+                                 Output: t1_3.c
+                                 Sort Key: t1_3.c
+                                 ->  Seq Scan on public.plt2_p1 t1_3
+                                       Output: t1_3.c
+         ->  Merge Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Merge Cond: (t1_1.c = t1_4.c)
+               ->  Sort
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Sort Key: t1_1.c
+                     ->  Seq Scan on public.plt1_p2 t1_1
+                           Output: t1_1.a, t1_1.b, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_4.c, t1_7.c
+                     ->  Merge Join
+                           Output: t1_4.c, t1_7.c
+                           Merge Cond: ((ltrim(t1_7.c, 'A'::text)) = t1_4.c)
+                           ->  Sort
+                                 Output: t1_7.c, (ltrim(t1_7.c, 'A'::text))
+                                 Sort Key: (ltrim(t1_7.c, 'A'::text))
+                                 ->  HashAggregate
+                                       Output: t1_7.c, (ltrim(t1_7.c, 'A'::text))
+                                       Group Key: ltrim(t1_7.c, 'A'::text)
+                                       ->  Seq Scan on public.plt1_e_p2 t1_7
+                                             Output: t1_7.c, ltrim(t1_7.c, 'A'::text)
+                                             Filter: ((t1_7.a % 25) = 0)
+                           ->  Sort
+                                 Output: t1_4.c
+                                 Sort Key: t1_4.c
+                                 ->  Seq Scan on public.plt2_p2 t1_4
+                                       Output: t1_4.c
+         ->  Merge Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Merge Cond: (t1_2.c = t1_5.c)
+               ->  Sort
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Sort Key: t1_2.c
+                     ->  Seq Scan on public.plt1_p3 t1_2
+                           Output: t1_2.a, t1_2.b, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_5.c, t1_8.c
+                     ->  Merge Join
+                           Output: t1_5.c, t1_8.c
+                           Merge Cond: ((ltrim(t1_8.c, 'A'::text)) = t1_5.c)
+                           ->  Sort
+                                 Output: t1_8.c, (ltrim(t1_8.c, 'A'::text))
+                                 Sort Key: (ltrim(t1_8.c, 'A'::text))
+                                 ->  HashAggregate
+                                       Output: t1_8.c, (ltrim(t1_8.c, 'A'::text))
+                                       Group Key: ltrim(t1_8.c, 'A'::text)
+                                       ->  Seq Scan on public.plt1_e_p3 t1_8
+                                             Output: t1_8.c, ltrim(t1_8.c, 'A'::text)
+                                             Filter: ((t1_8.a % 25) = 0)
+                           ->  Sort
+                                 Output: t1_5.c
+                                 Sort Key: t1_5.c
+                                 ->  Seq Scan on public.plt2_p3 t1_5
+                                       Output: t1_5.c
+(88 rows)
+
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 5) is considered for partition-wise join.
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+SET enable_seqscan TO off;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Merge Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           Sort Key: t2.b, t2.c
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Merge Cond: (ltrim(t3.c, 'A'::text) = t2.c)
+                                 Join Filter: (t2.b = t3.a)
+                                 ->  Index Scan using iplt1_e_p1_c on public.plt1_e_p1 t3
+                                       Output: t3.a, t3.b, t3.c
+                                       Filter: ((t3.a % 25) = 0)
+                                 ->  Index Scan using iplt2_p1_c on public.plt2_p1 t2
+                                       Output: t2.b, t2.c
+                     ->  Sort
+                           Output: t1.a, t1.c
+                           Sort Key: t1.a, t1.c
+                           ->  Index Scan using iplt1_p1_c on public.plt1_p1 t1
+                                 Output: t1.a, t1.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Merge Cond: ((t2_1.c = t1_1.c) AND (t2_1.b = t1_1.a))
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           Sort Key: t2_1.c, t2_1.b
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Merge Cond: (ltrim(t3_1.c, 'A'::text) = t2_1.c)
+                                 Join Filter: (t2_1.b = t3_1.a)
+                                 ->  Index Scan using iplt1_e_p2_c on public.plt1_e_p2 t3_1
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       Filter: ((t3_1.a % 25) = 0)
+                                 ->  Index Scan using iplt2_p2_c on public.plt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                     ->  Sort
+                           Output: t1_1.a, t1_1.c
+                           Sort Key: t1_1.c, t1_1.a
+                           ->  Index Scan using iplt1_p2_c on public.plt1_p2 t1_1
+                                 Output: t1_1.a, t1_1.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Merge Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           Sort Key: t2_2.b, t2_2.c
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Merge Cond: (ltrim(t3_2.c, 'A'::text) = t2_2.c)
+                                 Join Filter: (t2_2.b = t3_2.a)
+                                 ->  Index Scan using iplt1_e_p3_c on public.plt1_e_p3 t3_2
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       Filter: ((t3_2.a % 25) = 0)
+                                 ->  Index Scan using iplt2_p3_c on public.plt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                     ->  Sort
+                           Output: t1_2.a, t1_2.c
+                           Sort Key: t1_2.a, t1_2.c
+                           ->  Index Scan using iplt1_p3_c on public.plt1_p3 t1_2
+                                 Output: t1_2.a, t1_2.c
+(66 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+     |      |     |      |      100 | A0001
+     |      |     |      |      200 | A0002
+     |      |     |      |      400 | A0004
+     |      |     |      |      500 | A0005
+     |      |     |      |      700 | A0007
+     |      |     |      |      800 | A0008
+     |      |     |      |     1000 | A0010
+     |      |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+     |      |     |      |      100 | A0001
+     |      |     |      |      200 | A0002
+     |      |     |      |      400 | A0004
+     |      |     |      |      500 | A0005
+     |      |     |      |      700 | A0007
+     |      |     |      |      800 | A0008
+     |      |     |      |     1000 | A0010
+     |      |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 6) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is considered for partition-wise join.
+                                                  QUERY PLAN                                                   
+---------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, ((plt1_e_p1.a + plt1_e_p1.b)), plt1_e_p1.c
+   Sort Key: plt1_p1.a, plt2_p1.b, ((plt1_e_p1.a + plt1_e_p1.b))
+   ->  Result
+         Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, (plt1_e_p1.a + plt1_e_p1.b), plt1_e_p1.c
+         ->  Append
+               ->  Merge Full Join
+                     Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                     Merge Cond: ((plt1_p1.a = plt1_e_p1.a) AND (plt1_p1.c = (ltrim(plt1_e_p1.c, 'A'::text))))
+                     ->  Sort
+                           Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+                           Sort Key: plt1_p1.a, plt1_p1.c
+                           ->  Merge Full Join
+                                 Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+                                 Merge Cond: ((plt1_p1.a = plt2_p1.b) AND (plt1_p1.c = plt2_p1.c))
+                                 ->  Sort
+                                       Output: plt1_p1.a, plt1_p1.c
+                                       Sort Key: plt1_p1.a, plt1_p1.c
+                                       ->  Index Scan using iplt1_p1_c on public.plt1_p1
+                                             Output: plt1_p1.a, plt1_p1.c
+                                             Filter: ((plt1_p1.a % 25) = 0)
+                                 ->  Sort
+                                       Output: plt2_p1.b, plt2_p1.c
+                                       Sort Key: plt2_p1.b, plt2_p1.c
+                                       ->  Index Scan using iplt2_p1_c on public.plt2_p1
+                                             Output: plt2_p1.b, plt2_p1.c
+                                             Filter: ((plt2_p1.b % 25) = 0)
+                     ->  Sort
+                           Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c, (ltrim(plt1_e_p1.c, 'A'::text))
+                           Sort Key: plt1_e_p1.a, (ltrim(plt1_e_p1.c, 'A'::text))
+                           ->  Index Scan using iplt1_e_p1_c on public.plt1_e_p1
+                                 Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c, ltrim(plt1_e_p1.c, 'A'::text)
+                                 Filter: ((plt1_e_p1.a % 25) = 0)
+               ->  Merge Full Join
+                     Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c, plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                     Merge Cond: ((plt1_p2.a = plt1_e_p2.a) AND (plt1_p2.c = (ltrim(plt1_e_p2.c, 'A'::text))))
+                     ->  Sort
+                           Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+                           Sort Key: plt1_p2.a, plt1_p2.c
+                           ->  Merge Full Join
+                                 Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+                                 Merge Cond: ((plt1_p2.a = plt2_p2.b) AND (plt1_p2.c = plt2_p2.c))
+                                 ->  Sort
+                                       Output: plt1_p2.a, plt1_p2.c
+                                       Sort Key: plt1_p2.a, plt1_p2.c
+                                       ->  Index Scan using iplt1_p2_c on public.plt1_p2
+                                             Output: plt1_p2.a, plt1_p2.c
+                                             Filter: ((plt1_p2.a % 25) = 0)
+                                 ->  Sort
+                                       Output: plt2_p2.b, plt2_p2.c
+                                       Sort Key: plt2_p2.b, plt2_p2.c
+                                       ->  Index Scan using iplt2_p2_c on public.plt2_p2
+                                             Output: plt2_p2.b, plt2_p2.c
+                                             Filter: ((plt2_p2.b % 25) = 0)
+                     ->  Sort
+                           Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c, (ltrim(plt1_e_p2.c, 'A'::text))
+                           Sort Key: plt1_e_p2.a, (ltrim(plt1_e_p2.c, 'A'::text))
+                           ->  Index Scan using iplt1_e_p2_c on public.plt1_e_p2
+                                 Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c, ltrim(plt1_e_p2.c, 'A'::text)
+                                 Filter: ((plt1_e_p2.a % 25) = 0)
+               ->  Merge Full Join
+                     Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c, plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                     Merge Cond: ((plt1_p3.a = plt1_e_p3.a) AND (plt1_p3.c = (ltrim(plt1_e_p3.c, 'A'::text))))
+                     ->  Sort
+                           Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+                           Sort Key: plt1_p3.a, plt1_p3.c
+                           ->  Merge Full Join
+                                 Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+                                 Merge Cond: ((plt1_p3.a = plt2_p3.b) AND (plt1_p3.c = plt2_p3.c))
+                                 ->  Sort
+                                       Output: plt1_p3.a, plt1_p3.c
+                                       Sort Key: plt1_p3.a, plt1_p3.c
+                                       ->  Index Scan using iplt1_p3_c on public.plt1_p3
+                                             Output: plt1_p3.a, plt1_p3.c
+                                             Filter: ((plt1_p3.a % 25) = 0)
+                                 ->  Sort
+                                       Output: plt2_p3.b, plt2_p3.c
+                                       Sort Key: plt2_p3.b, plt2_p3.c
+                                       ->  Index Scan using iplt2_p3_c on public.plt2_p3
+                                             Output: plt2_p3.b, plt2_p3.c
+                                             Filter: ((plt2_p3.b % 25) = 0)
+                     ->  Sort
+                           Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c, (ltrim(plt1_e_p3.c, 'A'::text))
+                           Sort Key: plt1_e_p3.a, (ltrim(plt1_e_p3.c, 'A'::text))
+                           ->  Index Scan using iplt1_e_p3_c on public.plt1_e_p3
+                                 Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c, ltrim(plt1_e_p3.c, 'A'::text)
+                                 Filter: ((plt1_e_p3.a % 25) = 0)
+(87 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 6) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 5) is considered for partition-wise join.
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Merge Semi Join
+               Output: t1.a, t1.b, t1.c
+               Merge Cond: (t1.c = t1_3.c)
+               ->  Index Scan using iplt1_p1_c on public.plt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_3.c, t1_6.c
+                     ->  Merge Semi Join
+                           Output: t1_3.c, t1_6.c
+                           Merge Cond: (t1_3.c = ltrim(t1_6.c, 'A'::text))
+                           ->  Index Only Scan using iplt2_p1_c on public.plt2_p1 t1_3
+                                 Output: t1_3.c
+                           ->  Materialize
+                                 Output: t1_6.c
+                                 ->  Index Scan using iplt1_e_p1_c on public.plt1_e_p1 t1_6
+                                       Output: t1_6.c
+                                       Filter: ((t1_6.a % 25) = 0)
+         ->  Merge Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Merge Cond: (t1_1.c = t1_4.c)
+               ->  Index Scan using iplt1_p2_c on public.plt1_p2 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_4.c, t1_7.c
+                     ->  Merge Semi Join
+                           Output: t1_4.c, t1_7.c
+                           Merge Cond: (t1_4.c = ltrim(t1_7.c, 'A'::text))
+                           ->  Index Only Scan using iplt2_p2_c on public.plt2_p2 t1_4
+                                 Output: t1_4.c
+                           ->  Materialize
+                                 Output: t1_7.c
+                                 ->  Index Scan using iplt1_e_p2_c on public.plt1_e_p2 t1_7
+                                       Output: t1_7.c
+                                       Filter: ((t1_7.a % 25) = 0)
+         ->  Merge Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Merge Cond: (t1_2.c = t1_5.c)
+               ->  Index Scan using iplt1_p3_c on public.plt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_5.c, t1_8.c
+                     ->  Merge Semi Join
+                           Output: t1_5.c, t1_8.c
+                           Merge Cond: (t1_5.c = ltrim(t1_8.c, 'A'::text))
+                           ->  Index Only Scan using iplt2_p3_c on public.plt2_p3 t1_5
+                                 Output: t1_5.c
+                           ->  Materialize
+                                 Output: t1_8.c
+                                 ->  Index Scan using iplt1_e_p3_c on public.plt1_e_p3 t1_8
+                                       Output: t1_8.c
+                                       Filter: ((t1_8.a % 25) = 0)
+(58 rows)
+
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 5) is considered for partition-wise join.
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+RESET enable_hashjoin;
+RESET enable_nestloop;
+RESET enable_seqscan;
+--
+-- negative testcases
+--
+CREATE TABLE prt1_n (a int, b int, c varchar) PARTITION BY RANGE(c);
+CREATE TABLE prt1_n_p1 PARTITION OF prt1_n FOR VALUES START ('0000') END ('0250');
+CREATE TABLE prt1_n_p2 PARTITION OF prt1_n FOR VALUES START ('0250') END ('0500');
+INSERT INTO prt1_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 499, 2) i;
+ANALYZE prt1_n;
+ANALYZE prt1_n_p1;
+ANALYZE prt1_n_p2;
+CREATE TABLE prt2_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt2_n_p1 PARTITION OF prt2_n FOR VALUES IN ('0000', '0003', '0004', '0010', '0006', '0007');
+CREATE TABLE prt2_n_p2 PARTITION OF prt2_n FOR VALUES IN ('0001', '0005', '0002', '0009', '0008', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt2_n;
+ANALYZE prt2_n_p1;
+ANALYZE prt2_n_p2;
+CREATE TABLE prt3_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt3_n_p1 PARTITION OF prt3_n FOR VALUES IN ('0000', '0004', '0006', '0007');
+CREATE TABLE prt3_n_p2 PARTITION OF prt3_n FOR VALUES IN ('0001', '0002', '0008', '0010');
+CREATE TABLE prt3_n_p3 PARTITION OF prt3_n FOR VALUES IN ('0003', '0005', '0009', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt3_n;
+ANALYZE prt3_n_p1;
+ANALYZE prt3_n_p2;
+ANALYZE prt3_n_p3;
+CREATE TABLE prt4_n (a int, b int, c text) PARTITION BY RANGE(a);
+CREATE TABLE prt4_n_p1 PARTITION OF prt4_n FOR VALUES START (0) END (300);
+CREATE TABLE prt4_n_p2 PARTITION OF prt4_n FOR VALUES START (300) END (500);
+CREATE TABLE prt4_n_p3 PARTITION OF prt4_n FOR VALUES START (500) END (600);
+INSERT INTO prt4_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt4_n;
+ANALYZE prt4_n_p1;
+ANALYZE prt4_n_p2;
+ANALYZE prt4_n_p3;
+-- partition-wise join can not be applied if the partition ranges differ
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Join
+   Hash Cond: (t1.a = t2.a)
+   ->  Append
+         ->  Seq Scan on prt1 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p2 t1_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt4_n t2
+               ->  Seq Scan on prt4_n_p1 t2_1
+               ->  Seq Scan on prt4_n_p2 t2_2
+               ->  Seq Scan on prt4_n_p3 t2_3
+(13 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 FULL JOIN prt4_n t2 ON t1.a = t2.a;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Full Join
+   Hash Cond: (t1.a = t2.a)
+   ->  Append
+         ->  Seq Scan on prt1 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p2 t1_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt4_n t2
+               ->  Seq Scan on prt4_n_p1 t2_1
+               ->  Seq Scan on prt4_n_p2 t2_2
+               ->  Seq Scan on prt4_n_p3 t2_3
+(13 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                        
+---------------------------------------------------------
+ Nested Loop Left Join
+   ->  Append
+         ->  Seq Scan on prt1 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p2 t1_3
+   ->  Append
+         ->  Seq Scan on prt2 t2
+               Filter: (t1.a < b)
+         ->  Index Scan using iprt2_p1_b on prt2_p1 t2_1
+               Index Cond: (t1.a < b)
+         ->  Index Scan using iprt2_p2_b on prt2_p2 t2_2
+               Index Cond: (t1.a < b)
+         ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+               Index Cond: (t1.a < b)
+(15 rows)
+
+-- equi-join with join condition on partial keys does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1, prt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t1.a % 25 = 0;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Join
+   Hash Cond: (((t2.b + t2.a) / 2) = t1.a)
+   ->  Append
+         ->  Seq Scan on prt2_m t2
+         ->  Seq Scan on prt2_m_p1 t2_1
+         ->  Seq Scan on prt2_m_p2 t2_2
+         ->  Seq Scan on prt2_m_p3 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_m t1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p1 t1_1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p2 t1_2
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p3 t1_3
+                     Filter: ((a % 25) = 0)
+(17 rows)
+
+-- equi-join between out-of-order partition key columns does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.a = t2.b WHERE t1.a % 25 = 0;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Right Join
+   Hash Cond: (t2.b = t1.a)
+   ->  Append
+         ->  Seq Scan on prt2_m t2
+         ->  Seq Scan on prt2_m_p1 t2_1
+         ->  Seq Scan on prt2_m_p2 t2_2
+         ->  Seq Scan on prt2_m_p3 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_m t1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p1 t1_1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p2 t1_2
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p3 t1_3
+                     Filter: ((a % 25) = 0)
+(17 rows)
+
+-- equi-join between non-key columns does not qualify for partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.c = t2.c WHERE t1.a % 25 = 0;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Right Join
+   Hash Cond: ((t2.c)::text = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt2_m t2
+         ->  Seq Scan on prt2_m_p1 t2_1
+         ->  Seq Scan on prt2_m_p2 t2_2
+         ->  Seq Scan on prt2_m_p3 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_m t1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p1 t1_1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p2 t1_2
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p3 t1_3
+                     Filter: ((a % 25) = 0)
+(17 rows)
+
+-- partition-wise join can not be applied for a join between list and range
+-- partitioned table
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1, prt2_n t2 WHERE t1.c = t2.c;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Join
+   Hash Cond: (t2.c = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt2_n t2
+         ->  Seq Scan on prt2_n_p1 t2_1
+         ->  Seq Scan on prt2_n_p2 t2_2
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(11 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 LEFT JOIN prt2_n t2 ON (t1.c = t2.c);
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Right Join
+   Hash Cond: (t2.c = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt2_n t2
+         ->  Seq Scan on prt2_n_p1 t2_1
+         ->  Seq Scan on prt2_n_p2 t2_2
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(11 rows)
+
+-- partition-wise join can not be applied between tables with different
+-- partition lists
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 RIGHT JOIN prt1 t2 ON (t1.c = t2.c);
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Left Join
+   Hash Cond: ((t2.c)::text = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt1 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p2 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(12 rows)
+
+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);
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Full Join
+   Hash Cond: ((t2.c)::text = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt1 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p2 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(12 rows)
+
diff --git a/src/test/regress/expected/rangefuncs.out b/src/test/regress/expected/rangefuncs.out
index f06cfa4..16e7f56 100644
--- a/src/test/regress/expected/rangefuncs.out
+++ b/src/test/regress/expected/rangefuncs.out
@@ -1,25 +1,26 @@
 SELECT name, setting FROM pg_settings WHERE name LIKE 'enable%';
-         name         | setting 
-----------------------+---------
- enable_bitmapscan    | on
- enable_hashagg       | on
- enable_hashjoin      | on
- enable_indexonlyscan | on
- enable_indexscan     | on
- enable_material      | on
- enable_mergejoin     | on
- enable_nestloop      | on
- enable_seqscan       | on
- enable_sort          | on
- enable_tidscan       | on
-(11 rows)
+            name            | setting 
+----------------------------+---------
+ enable_bitmapscan          | on
+ enable_hashagg             | on
+ enable_hashjoin            | on
+ enable_indexonlyscan       | on
+ enable_indexscan           | on
+ enable_material            | on
+ enable_mergejoin           | on
+ enable_nestloop            | on
+ enable_partition_wise_join | on
+ enable_seqscan             | on
+ enable_sort                | on
+ enable_tidscan             | on
+(12 rows)
 
 CREATE TABLE foo2(fooid int, f2 int);
 INSERT INTO foo2 VALUES(1, 11);
 INSERT INTO foo2 VALUES(2, 22);
 INSERT INTO foo2 VALUES(1, 111);
 CREATE FUNCTION foot(int) returns setof foo2 as 'SELECT * FROM foo2 WHERE fooid = $1 ORDER BY f2;' LANGUAGE SQL;
 -- function with ORDINALITY
 select * from foot(1) with ordinality as z(a,b,ord);
  a |  b  | ord 
 ---+-----+-----
diff --git a/src/test/regress/parallel_schedule b/src/test/regress/parallel_schedule
index 1cb5dfc..d62841d 100644
--- a/src/test/regress/parallel_schedule
+++ b/src/test/regress/parallel_schedule
@@ -93,21 +93,21 @@ test: alter_generic alter_operator misc psql async dbsize misc_functions
 
 # rules cannot run concurrently with any test that creates a view
 test: rules psql_crosstab amutils
 
 # run by itself so it can run parallel workers
 test: select_parallel
 
 # ----------
 # Another group of parallel tests
 # ----------
-test: select_views portals_p2 foreign_key cluster dependency guc bitmapops combocid tsearch tsdicts foreign_data window xmlmap functional_deps advisory_lock json jsonb json_encoding indirect_toast equivclass
+test: select_views portals_p2 foreign_key cluster dependency guc bitmapops combocid tsearch tsdicts foreign_data window xmlmap functional_deps advisory_lock json jsonb json_encoding indirect_toast equivclass partition_join
 # ----------
 # Another group of parallel tests
 # NB: temp.sql does a reconnect which transiently uses 2 connections,
 # so keep this parallel group to at most 19 tests
 # ----------
 test: plancache limit plpgsql copy2 temp domain rangefuncs prepare without_oid conversion truncate alter_table sequence polymorphism rowtypes returning largeobject with xml
 
 # event triggers cannot run concurrently with any test that runs DDL
 test: event_trigger
 
diff --git a/src/test/regress/serial_schedule b/src/test/regress/serial_schedule
index 8958d8c..49a27b1 100644
--- a/src/test/regress/serial_schedule
+++ b/src/test/regress/serial_schedule
@@ -161,10 +161,11 @@ test: truncate
 test: alter_table
 test: sequence
 test: polymorphism
 test: rowtypes
 test: returning
 test: largeobject
 test: with
 test: xml
 test: event_trigger
 test: stats
+test: partition_join
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
new file mode 100644
index 0000000..19c7d29
--- /dev/null
+++ b/src/test/regress/sql/partition_join.sql
@@ -0,0 +1,746 @@
+--
+-- PARTITION_JOIN
+-- Test partition-wise join between partitioned tables
+--
+
+--
+-- partitioned by a single column
+--
+CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES START (0) END (250);
+CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES START (500) END (600);
+CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES START (250) END (500);
+INSERT INTO prt1 SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1 AS SELECT * FROM prt1;
+
+CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES START (0) END (250);
+CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES START (250) END (500);
+CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES START (500) END (600);
+INSERT INTO prt2 SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+CREATE TABLE uprt2 AS SELECT * FROM prt2;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, 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.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+
+-- Join with pruned partitions from joining relations
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a < 450 AND t2.b > 250 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+-- lateral reference
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+--
+-- partitioned by expression
+--
+CREATE TABLE prt1_e (a int, b int, c varchar) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES START (0) END (250);
+CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES START (250) END (500);
+CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES START (500) END (600);
+INSERT INTO prt1_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_e AS SELECT * FROM prt1_e;
+
+CREATE TABLE prt2_e (a int, b int, c varchar) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES START (0) END (250);
+CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES START (250) END (500);
+CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES START (500) END (600);
+INSERT INTO prt2_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_e;
+ANALYZE prt2_e_p1;
+ANALYZE prt2_e_p2;
+ANALYZE prt2_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_e AS SELECT * FROM prt2_e;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1, uprt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1 LEFT JOIN uprt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 RIGHT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 RIGHT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1 RIGHT JOIN uprt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_e WHERE prt2_e.b % 25 = 0) t2 ON ((t1.a + t1.b)/2 = (t2.b + t2.a)/2) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_e WHERE prt2_e.b % 25 = 0) t2 ON ((t1.a + t1.b)/2 = (t2.b + t2.a)/2) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_e t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_e t2 WHERE t2.b % 25 = 0) t2 ON ((t1.a + t1.b)/2 = (t2.b + t2.a)/2) ORDER BY t1.a, t2.b;
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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.a % 25 = 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM uprt1 t1, uprt2 t2, uprt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM uprt1_e WHERE uprt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, 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.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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;
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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;
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM uprt1_e WHERE uprt1_e.a % 25 = 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;
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1, uprt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+-- test merge joins with and without using indexes
+SET enable_hashjoin TO off;
+SET enable_nestloop TO off;
+
+CREATE INDEX iprt1_a on prt1(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 iprt2_b on prt2(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 iprt1_e_ab2 on prt1_e(((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));
+
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+SET enable_seqscan TO off;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+-- lateral references and parameterized paths
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+RESET enable_hashjoin;
+RESET enable_nestloop;
+RESET enable_seqscan;
+
+--
+-- partitioned by multiple columns
+--
+CREATE TABLE prt1_m (a int, b int, c varchar) PARTITION BY RANGE(a, ((a + b)/2));
+CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES START (0, 0) END (250, 250);
+CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES START (250, 250) END (500, 500);
+CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES START (500, 500) END (600, 600);
+INSERT INTO prt1_m SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_m;
+ANALYZE prt1_m_p1;
+ANALYZE prt1_m_p2;
+ANALYZE prt1_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_m AS SELECT * FROM prt1_m;
+
+CREATE TABLE prt2_m (a int, b int, c varchar) PARTITION BY RANGE(((b + a)/2), b);
+CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES START (0, 0) END (250, 250);
+CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES START (250, 250) END (500, 500);
+CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES START (500, 500) END (600, 600);
+INSERT INTO prt2_m SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_m;
+ANALYZE prt2_m_p1;
+ANALYZE prt2_m_p2;
+ANALYZE prt2_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_m AS SELECT * FROM prt2_m;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1, prt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1, prt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_m t1, uprt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_m t1 LEFT JOIN uprt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_m t1 RIGHT JOIN uprt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_m t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_m t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+
+--
+-- multi-leveled partitions
+--
+CREATE TABLE prt1_l (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_l_p1 PARTITION OF prt1_l FOR VALUES START (0) END (250) PARTITION BY RANGE (b);
+CREATE TABLE prt1_l_p1_p1 PARTITION OF prt1_l_p1 FOR VALUES START (0) END (100);
+CREATE TABLE prt1_l_p1_p2 PARTITION OF prt1_l_p1 FOR VALUES START (100) END (250);
+CREATE TABLE prt1_l_p2 PARTITION OF prt1_l FOR VALUES START (250) END (500) PARTITION BY RANGE (c);
+CREATE TABLE prt1_l_p2_p1 PARTITION OF prt1_l_p2 FOR VALUES START ('0250') END ('0400');
+CREATE TABLE prt1_l_p2_p2 PARTITION OF prt1_l_p2 FOR VALUES START ('0400') END ('0500');
+CREATE TABLE prt1_l_p3 PARTITION OF prt1_l FOR VALUES START (500) END (600) PARTITION BY RANGE ((b + a));
+CREATE TABLE prt1_l_p3_p1 PARTITION OF prt1_l_p3 FOR VALUES START (1000) END (1100);
+CREATE TABLE prt1_l_p3_p2 PARTITION OF prt1_l_p3 FOR VALUES START (1100) END (1200);
+INSERT INTO prt1_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_l;
+ANALYZE prt1_l_p1;
+ANALYZE prt1_l_p1_p1;
+ANALYZE prt1_l_p1_p2;
+ANALYZE prt1_l_p2;
+ANALYZE prt1_l_p2_p1;
+ANALYZE prt1_l_p2_p2;
+ANALYZE prt1_l_p3;
+ANALYZE prt1_l_p3_p1;
+ANALYZE prt1_l_p3_p2;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_l AS SELECT * FROM prt1_l;
+
+CREATE TABLE prt2_l (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_l_p1 PARTITION OF prt2_l FOR VALUES START (0) END (250) PARTITION BY RANGE (a);
+CREATE TABLE prt2_l_p1_p1 PARTITION OF prt2_l_p1 FOR VALUES START (0) END (100);
+CREATE TABLE prt2_l_p1_p2 PARTITION OF prt2_l_p1 FOR VALUES START (100) END (250);
+CREATE TABLE prt2_l_p2 PARTITION OF prt2_l FOR VALUES START (250) END (500) PARTITION BY RANGE (c);
+CREATE TABLE prt2_l_p2_p1 PARTITION OF prt2_l_p2 FOR VALUES START ('0250') END ('0400');
+CREATE TABLE prt2_l_p2_p2 PARTITION OF prt2_l_p2 FOR VALUES START ('0400') END ('0500');
+CREATE TABLE prt2_l_p3 PARTITION OF prt2_l FOR VALUES START (500) END (600) PARTITION BY RANGE ((a + b));
+CREATE TABLE prt2_l_p3_p1 PARTITION OF prt2_l_p3 FOR VALUES START (1000) END (1100);
+CREATE TABLE prt2_l_p3_p2 PARTITION OF prt2_l_p3 FOR VALUES START (1100) END (1200);
+INSERT INTO prt2_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_l;
+ANALYZE prt2_l_p1;
+ANALYZE prt2_l_p1_p1;
+ANALYZE prt2_l_p1_p2;
+ANALYZE prt2_l_p2;
+ANALYZE prt2_l_p2_p1;
+ANALYZE prt2_l_p2_p2;
+ANALYZE prt2_l_p3;
+ANALYZE prt2_l_p3_p1;
+ANALYZE prt2_l_p3_p2;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_l AS SELECT * FROM prt2_l;
+
+EXPLAIN (VERBOSE, 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 = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 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 = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1, uprt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 LEFT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 RIGHT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_l t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_l t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+
+--
+-- 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;
+ANALYZE plt1_p1;
+ANALYZE plt1_p2;
+ANALYZE plt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1 AS SELECT * FROM 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;
+ANALYZE plt2_p1;
+ANALYZE plt2_p2;
+ANALYZE plt2_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt2 AS SELECT * FROM plt2;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.a = t2.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.a = t2.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1 t1, uplt2 t2 WHERE t1.c = t2.c AND t1.a = t2.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.a AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.a AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.a AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 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 plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 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 uplt1 t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) ORDER BY t1.a, t2.b;
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM (SELECT 50 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM (SELECT 50 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM (SELECT 50 phv, * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t1.phv), avg(t2.b), t2.c, avg(t2.phv) FROM (SELECT 25 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t1.phv), avg(t2.b), t2.c, avg(t2.phv) FROM (SELECT 25 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t1.phv), avg(t2.b), t2.c, avg(t2.phv) FROM (SELECT 25 phv, * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
+-- Join with pruned partitions from joining relations
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.c NOT IN ('0001', '0005', '0002', '0009') AND t2.c NOT IN ('0000', '0003', '0004', '0010') GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.c NOT IN ('0001', '0005', '0002', '0009') AND t2.c NOT IN ('0000', '0003', '0004', '0010') GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM uplt1 t1, uplt2 t2 WHERE t1.c = t2.c AND t1.c NOT IN ('0001', '0005', '0002', '0009') AND t2.c NOT IN ('0000', '0003', '0004', '0010') GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 LEFT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 LEFT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM uplt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 LEFT JOIN (SELECT * FROM uplt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 RIGHT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 RIGHT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM uplt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 RIGHT JOIN (SELECT * FROM uplt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 FULL JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 FULL JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM uplt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 FULL JOIN (SELECT * FROM uplt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 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;
+ANALYZE plt1_e;
+ANALYZE plt1_e_p1;
+ANALYZE plt1_e_p2;
+ANALYZE plt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1_e AS SELECT * FROM plt1_e;
+
+CREATE TABLE plt2_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
+CREATE TABLE plt2_e_p1 PARTITION OF plt2_e FOR VALUES IN ('0000', '0003', '0004', '0010');
+CREATE TABLE plt2_e_p2 PARTITION OF plt2_e FOR VALUES IN ('0001', '0005', '0002', '0009');
+CREATE TABLE plt2_e_p3 PARTITION OF plt2_e FOR VALUES IN ('0006', '0007', '0008', '0011');
+INSERT INTO plt2_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE plt2_e;
+ANALYZE plt2_e_p1;
+ANALYZE plt2_e_p2;
+ANALYZE plt2_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt2_e AS SELECT * FROM plt2_e;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1, plt2_e t2 WHERE t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1, plt2_e t2 WHERE t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_e t1, uplt2_e t2 WHERE t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 LEFT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 LEFT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_e t1 LEFT JOIN uplt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 RIGHT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 RIGHT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_e t1 RIGHT JOIN uplt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2_e WHERE plt2_e.b % 25 = 0) t2 ON (t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A')) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2_e WHERE plt2_e.b % 25 = 0) t2 ON (t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A')) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uplt1_e t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2_e t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A')) ORDER BY t1.a, t2.b;
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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;
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM uplt1 t1, uplt2 t2, uplt1_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;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1, uplt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 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 ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+-- test merge join with and without index scan
+CREATE INDEX iplt1_c on plt1(c);
+CREATE INDEX iplt1_p1_c on plt1_p1(c);
+CREATE INDEX iplt1_p2_c on plt1_p2(c);
+CREATE INDEX iplt1_p3_c on plt1_p3(c);
+CREATE INDEX iplt2_c on plt2(c);
+CREATE INDEX iplt2_p1_c on plt2_p1(c);
+CREATE INDEX iplt2_p2_c on plt2_p2(c);
+CREATE INDEX iplt2_p3_c on plt2_p3(c);
+CREATE INDEX iplt1_e_c on plt1_e(ltrim(c, 'A'));
+CREATE INDEX iplt1_e_p1_c on plt1_e_p1(ltrim(c, 'A'));
+CREATE INDEX iplt1_e_p2_c on plt1_e_p2(ltrim(c, 'A'));
+CREATE INDEX iplt1_e_p3_c on plt1_e_p3(ltrim(c, 'A'));
+
+ANALYZE plt1;
+ANALYZE plt1_p1;
+ANALYZE plt1_p2;
+ANALYZE plt1_p3;
+ANALYZE plt2;
+ANALYZE plt2_p1;
+ANALYZE plt2_p2;
+ANALYZE plt2_p3;
+ANALYZE plt1_e;
+ANALYZE plt1_e_p1;
+ANALYZE plt1_e_p2;
+ANALYZE plt1_e_p3;
+
+SET enable_hashjoin TO off;
+SET enable_nestloop TO off;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 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 ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+SET enable_seqscan TO off;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 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 ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+RESET enable_hashjoin;
+RESET enable_nestloop;
+RESET enable_seqscan;
+
+--
+-- negative testcases
+--
+
+CREATE TABLE prt1_n (a int, b int, c varchar) PARTITION BY RANGE(c);
+CREATE TABLE prt1_n_p1 PARTITION OF prt1_n FOR VALUES START ('0000') END ('0250');
+CREATE TABLE prt1_n_p2 PARTITION OF prt1_n FOR VALUES START ('0250') END ('0500');
+INSERT INTO prt1_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 499, 2) i;
+ANALYZE prt1_n;
+ANALYZE prt1_n_p1;
+ANALYZE prt1_n_p2;
+
+CREATE TABLE prt2_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt2_n_p1 PARTITION OF prt2_n FOR VALUES IN ('0000', '0003', '0004', '0010', '0006', '0007');
+CREATE TABLE prt2_n_p2 PARTITION OF prt2_n FOR VALUES IN ('0001', '0005', '0002', '0009', '0008', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt2_n;
+ANALYZE prt2_n_p1;
+ANALYZE prt2_n_p2;
+
+CREATE TABLE prt3_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt3_n_p1 PARTITION OF prt3_n FOR VALUES IN ('0000', '0004', '0006', '0007');
+CREATE TABLE prt3_n_p2 PARTITION OF prt3_n FOR VALUES IN ('0001', '0002', '0008', '0010');
+CREATE TABLE prt3_n_p3 PARTITION OF prt3_n FOR VALUES IN ('0003', '0005', '0009', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt3_n;
+ANALYZE prt3_n_p1;
+ANALYZE prt3_n_p2;
+ANALYZE prt3_n_p3;
+
+CREATE TABLE prt4_n (a int, b int, c text) PARTITION BY RANGE(a);
+CREATE TABLE prt4_n_p1 PARTITION OF prt4_n FOR VALUES START (0) END (300);
+CREATE TABLE prt4_n_p2 PARTITION OF prt4_n FOR VALUES START (300) END (500);
+CREATE TABLE prt4_n_p3 PARTITION OF prt4_n FOR VALUES START (500) END (600);
+INSERT INTO prt4_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt4_n;
+ANALYZE prt4_n_p1;
+ANALYZE prt4_n_p2;
+ANALYZE prt4_n_p3;
+
+-- partition-wise join can not be applied if the partition ranges differ
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 FULL JOIN prt4_n t2 ON t1.a = t2.a;
+-- 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);
+-- equi-join with join condition on partial keys does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1, prt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t1.a % 25 = 0;
+-- equi-join between out-of-order partition key columns does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.a = t2.b WHERE t1.a % 25 = 0;
+-- equi-join between non-key columns does not qualify for partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.c = t2.c WHERE t1.a % 25 = 0;
+-- partition-wise join can not be applied for a join between list and range
+-- partitioned table
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1, prt2_n t2 WHERE t1.c = t2.c;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 LEFT JOIN prt2_n t2 ON (t1.c = t2.c);
+-- partition-wise join can not be applied between tables with different
+-- partition lists
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 RIGHT JOIN prt1 t2 ON (t1.c = t2.c);
+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);
#6Amit Kapila
amit.kapila16@gmail.com
In reply to: Ashutosh Bapat (#5)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Sep 9, 2016 at 3:17 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

4. Remove bms_to_char(): I have added this function to print Relids in the
debugger. I have found it very useful to quickly examine Relids in debugger,
which otherwise wasn't so easy. If others find it useful too, I can create a
separate patch to be considered for a separate commit.

+1 to have such a function. I often need something like that whenever
I debug the optimizer code.

--
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com

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

#7Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Ashutosh Bapat (#5)
Re: Partition-wise join for join between (declaratively) partitioned tables

On 2016/09/09 18:47, Ashutosh Bapat wrote:

A related change is renaming RangeBound structure in Amit
Langote's patches to PartitionRangeBound to avoid name conflict with
rangetypes.h. That change too should vanish once we decide where to keep
that structure and its final name.

This change has been incorporated into the latest patch I posted on Sep 9 [1]/messages/by-id/28ee345c-1278-700e-39a7-36a71f9a3b43@lab.ntt.co.jp.

Thanks,
Amit

[1]: /messages/by-id/28ee345c-1278-700e-39a7-36a71f9a3b43@lab.ntt.co.jp
/messages/by-id/28ee345c-1278-700e-39a7-36a71f9a3b43@lab.ntt.co.jp

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

#8Rajkumar Raghuwanshi
rajkumar.raghuwanshi@enterprisedb.com
In reply to: Ashutosh Bapat (#5)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Sep 9, 2016 at 3:17 PM, Ashutosh Bapat <
ashutosh.bapat@enterprisedb.com> wrote:

Hi All,

PFA the patch to support partition-wise joins for partitioned tables. The
patch
is based on the declarative parition support patches provided by Amit
Langote
on 26th August 2016.

I have applied declarative partitioning patches posted by Amit Langote on
26 Aug 2016 and then partition-wise-join patch, getting below error while
make install.

../../../../src/include/nodes/relation.h:706: error: redefinition of
typedef ‘PartitionOptInfo’
../../../../src/include/nodes/relation.h:490: note: previous declaration of
‘PartitionOptInfo’ was here
make[4]: *** [gistbuild.o] Error 1
make[4]: Leaving directory `/home/edb/Desktop/edb_work/WO
RKDB/PG/postgresql/src/backend/access/gist'
make[3]: *** [gist-recursive] Error 2
make[3]: Leaving directory `/home/edb/Desktop/edb_work/WO
RKDB/PG/postgresql/src/backend/access'
make[2]: *** [access-recursive] Error 2
make[2]: Leaving directory `/home/edb/Desktop/edb_work/WO
RKDB/PG/postgresql/src/backend'
make[1]: *** [all-backend-recurse] Error 2
make[1]: Leaving directory `/home/edb/Desktop/edb_work/WO
RKDB/PG/postgresql/src'
make: *** [all-src-recurse] Error 2

PS : I am using - gcc (GCC) 4.4.7 20120313 (Red Hat 4.4.7-17)

Attached the patch for the fix of above error.

Thanks & Regards,
Rajkumar Raghuwanshi
QMG, EnterpriseDB Corporation

Attachments:

pwj_install_fix.patchtext/x-patch; charset=US-ASCII; name=pwj_install_fix.patchDownload
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 1e9fed9..963b022 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -487,7 +487,7 @@ typedef enum RelOptKind
 	((reloptkind) == RELOPT_OTHER_MEMBER_REL || \
 	 (reloptkind) == RELOPT_OTHER_JOINREL)
 
-typedef struct PartitionOptInfo PartitionOptInfo;
+typedef struct PartitionOptInfo;
 
 typedef struct RelOptInfo
 {
@@ -561,7 +561,7 @@ typedef struct RelOptInfo
 	/*
 	 * TODO: Notice recursive usage of RelOptInfo.
 	 */
-	PartitionOptInfo	*part_info;
+	struct PartitionOptInfo	*part_info;
 
 	/* Set only for "other" base or join relations. */
 	Relids		parent_relids;
#9Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Rajkumar Raghuwanshi (#8)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Sep 16, 2016 at 6:00 PM, Rajkumar Raghuwanshi
<rajkumar.raghuwanshi@enterprisedb.com> wrote:

On Fri, Sep 9, 2016 at 3:17 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Hi All,

PFA the patch to support partition-wise joins for partitioned tables. The
patch
is based on the declarative parition support patches provided by Amit
Langote
on 26th August 2016.

I have applied declarative partitioning patches posted by Amit Langote on 26
Aug 2016 and then partition-wise-join patch, getting below error while make
install.

../../../../src/include/nodes/relation.h:706: error: redefinition of typedef
‘PartitionOptInfo’
../../../../src/include/nodes/relation.h:490: note: previous declaration of
‘PartitionOptInfo’ was here
make[4]: *** [gistbuild.o] Error 1
make[4]: Leaving directory
`/home/edb/Desktop/edb_work/WORKDB/PG/postgresql/src/backend/access/gist'
make[3]: *** [gist-recursive] Error 2
make[3]: Leaving directory
`/home/edb/Desktop/edb_work/WORKDB/PG/postgresql/src/backend/access'
make[2]: *** [access-recursive] Error 2
make[2]: Leaving directory
`/home/edb/Desktop/edb_work/WORKDB/PG/postgresql/src/backend'
make[1]: *** [all-backend-recurse] Error 2
make[1]: Leaving directory
`/home/edb/Desktop/edb_work/WORKDB/PG/postgresql/src'
make: *** [all-src-recurse] Error 2

PS : I am using - gcc (GCC) 4.4.7 20120313 (Red Hat 4.4.7-17)

Attached the patch for the fix of above error.

Thanks for the report. I will fix this in the next patch.

--
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

#10Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Ashutosh Bapat (#5)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

PFA patch which takes care of some of the TODOs mentioned in my
previous mail. The patch is based on the set of patches supporting
declarative partitioning by Amit Langoted posted on 26th August.

TODOs:

1. Instead of storing partitioning information in RelOptInfo of each of the
partitioned relations (base and join relations), we can keep a list of
canonical partition schemes in PlannerInfo. Every RelOptInfo gets a pointer
to
the member of list representing the partitioning scheme of corresponding
relation. RelOptInfo's of all similarly partitioned relations get the same
pointer thus making it easy to match the partitioning schemes by comparing
the
pointers. While we are supporting only exact partition matching scheme now,
it's possible to extend this method to match compatible partitioning schemes
by
maintaining a list of compatible partitioning schemes.

Right now, I have moved some partition related structures from partition.c
to
partition.h. These structures are still being reviewed and might change when
Amit Langote improves his patches. Having canonical partitioning scheme in
PlannerInfo may not require moving those structures out. So, that code is
still
under development. A related change is renaming RangeBound structure in Amit
Langote's patches to PartitionRangeBound to avoid name conflict with
rangetypes.h. That change too should vanish once we decide where to keep
that
structure and its final name.

Done.

2. Multi-level partitioned tables: For some reason path created for joining
partitions are not being picked up as the cheapest paths. I think, we need
to
finalize the lower level paths before moving upwards in the partition
hierarchy. But I am yet to investigate the issue here.
RelOptInfo::parent_relid
should point to top parents rather than immediate parents.

Done

3. Testing: need more tests for testing partition-wise join with foreign
tables
as partitions. More tests for parameterized joins for multi-level
partitioned
joins.

Needs to be done.

4. Remove bms_to_char(): I have added this function to print Relids in the
debugger. I have found it very useful to quickly examine Relids in debugger,
which otherwise wasn't so easy. If others find it useful too, I can create a
separate patch to be considered for a separate commit.

I will take care of this after rebasing the patch on the latest
sources and latest set of patches by Amit Langote.

5. In add_paths_to_append_rel() to find the possible set of outer relations
for
generating parameterized paths for a given join. This code needs to be
adjusted
to eliminate the parent relations possible set of outer relations for a join
between child partitions.

Done.

6. Add support to reparameterize more types of paths for child relations. I
will add this once we finalize the method to reparameterize a parent path
for
child partition.

Will wait for reviewer's opinion.

7. The patch adds make_joinrel() (name needs to be changed because of its
similariy with make_join_rel()) to construct an empty RelOptInfo for a join
between partitions. The function copies code doing the same from
build_join_rel(). build_join_rel() too can use this function, if we decide
to
retain it.

This will be done as a separate cleanup patch.

8. Few small TODOs related to code reorganization, proper function,
variable naming etc. are in the patch. pg_indent run.

I have taken care of most of the TODOs. But there are still some TODOs
remaining. I will take care of those in the next version of patches.

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

Attachments:

pg_dp_join_v2.patchinvalid/octet-stream; name=pg_dp_join_v2.patchDownload
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index f17ac29..4f840c1 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -13,38 +13,41 @@
  *-------------------------------------------------------------------------
 */
 
 #include "postgres.h"
 
 #include "access/heapam.h"
 #include "access/htup_details.h"
 #include "access/nbtree.h"
 #include "access/sysattr.h"
 #include "catalog/dependency.h"
+#include "catalog/heap.h"
 #include "catalog/indexing.h"
 #include "catalog/objectaddress.h"
 #include "catalog/partition.h"
 #include "catalog/pg_collation.h"
 #include "catalog/pg_inherits.h"
 #include "catalog/pg_inherits_fn.h"
 #include "catalog/pg_opclass.h"
 #include "catalog/pg_partitioned_table.h"
 #include "catalog/pg_partitioned_table_fn.h"
 #include "catalog/pg_type.h"
 #include "executor/executor.h"
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
 #include "nodes/parsenodes.h"
 #include "optimizer/clauses.h"
+#include "optimizer/cost.h"
 #include "optimizer/planmain.h"
 #include "optimizer/var.h"
+#include "rewrite/rewriteManip.h"
 #include "storage/lmgr.h"
 #include "utils/array.h"
 #include "utils/builtins.h"
 #include "utils/datum.h"
 #include "utils/memutils.h"
 #include "utils/fmgroids.h"
 #include "utils/inval.h"
 #include "utils/lsyscache.h"
 #include "utils/ruleutils.h"
 #include "utils/rel.h"
@@ -158,20 +161,50 @@ typedef struct PartitionTreeNodeData
 	PartitionDesc		pdesc;
 	Oid					relid;
 	int					index;
 	int					offset;
 	int					num_leaf_parts;
 
 	struct PartitionTreeNodeData *downlink;
 	struct PartitionTreeNodeData *next;
 } PartitionTreeNodeData;
 
+/*
+ * Partitioning scheme
+ *		Structure to hold partitioning scheme for a given relation.
+ *
+ * Multiple relations may be partitioned in the same way. The relations
+ * resulting from joining such relations may be partitioned in the same way as
+ * the joining relations.  Similarly, relations derived from such relations by
+ * grouping, sorting be partitioned in the same as the underlying relations.
+ *
+ * PlannerInfo stores a list of distinct "canonical" partitioning schemes.
+ * RelOptInfo of a partitioned relation holds the pointer to "canonical"
+ * partitioning scheme.
+ */
+typedef struct PartitionSchemeData
+{
+	/* Information about partitions */
+	int			nparts;		/* number of partitions */
+	PartitionListInfo	  **lists;		/* list bounds */
+	PartitionRangeInfo	  **ranges;	/* range lower bounds */
+
+	/* Information about partition keys */
+	char		strategy;		/* partition strategy */
+	int16		partnatts;		/* number of partition attributes */
+	Oid		   *partopfamily;	/* OIDs of operator families */
+	Oid		   *partopcintype;	/* OIDs of opclass declared input data types */
+	Oid		   *key_types;		/* OIDs of partition key data types. */
+	int32	   *key_typmods;	/* typmods of partition keys. */
+	Oid		   *key_collations;	/* OIDs of collations of partition keys. */
+} PartitionSchemeData;
+
 /* Support RelationBuildPartitionKey() */
 static PartitionKey copy_partition_key(PartitionKey fromkey);
 static KeyTypeCollInfo *copy_key_type_coll_info(int nkeycols,
 								KeyTypeCollInfo *tcinfo);
 
 /* Support RelationBuildPartitionDesc() */
 static int32 partition_cmp(const void *a, const void *b, void *arg);
 
 /* Support check_new_partition_bound() */
 static bool list_overlaps_existing_partition(PartitionKey key,
@@ -222,20 +255,23 @@ static int range_partition_for_tuple(PartitionKey key, PartitionDesc pdesc,
 
 /* List partition related support functions */
 static PartitionListInfo *make_list_from_spec(PartitionKey key,
 							PartitionListSpec *list_spec);
 static PartitionListInfo *copy_list_info(PartitionListInfo *src,
 							PartitionKey key);
 static bool equal_list_info(PartitionKey key, PartitionListInfo *l1,
 				PartitionListInfo *l2);
 static bool partition_list_values_equal(PartitionKey key,
 						   Datum val1, Datum val2);
+static bool have_same_partition_lists(PartitionDesc part_desc,
+									  PartitionKey part_key,
+									  PartitionScheme part_scheme);
 
 /* Range partition related support functions */
 static PartitionRangeInfo *make_range_from_spec(PartitionKey key,
 							PartitionRangeSpec *range_spec);
 static RangeBound *make_range_bound(PartitionKey key, List *val, bool inclusive,
 							bool lower);
 static PartitionRangeInfo *copy_range_info(PartitionRangeInfo *src,
 							PartitionKey key);
 static RangeBound *copy_range_bound(RangeBound *src, PartitionKey key);
 static bool equal_range_info(PartitionKey key, PartitionRangeInfo *r1,
@@ -245,20 +281,23 @@ static int32 partition_range_cmp(PartitionKey key, PartitionRangeInfo *r1,
 static int32 partition_range_bound_cmp(PartitionKey key, RangeBound *b1,
 							RangeBound *b2);
 static int32 partition_range_tuple_cmp(PartitionKey key,
 						   Datum *val1, Datum *val2);
 static bool partition_range_overlaps(PartitionKey key,
 							PartitionRangeInfo *r1, PartitionRangeInfo *r2);
 static bool tuple_rightof_bound(PartitionKey key, Datum *tuple, RangeBound *bound);
 static bool tuple_leftof_bound(PartitionKey key, Datum *tuple, RangeBound *bound);
 static int range_partition_bsearch(PartitionKey key, PartitionDesc pdesc,
 						Datum *tuple);
+static bool have_same_partition_bounds(PartitionDesc part_desc,
+								PartitionKey part_key,
+								PartitionScheme part_scheme);
 
 /*
  * Partition key related functions
  */
 
 /*
  * RelationBuildPartitionKey
  *		Build and attach to relcache partition key data of relation
  *
  * Note that the partition key data attached to a relcache entry must be
@@ -2358,10 +2397,357 @@ tuple_rightof_bound(PartitionKey key, Datum *tuple, RangeBound *bound)
 static bool
 tuple_leftof_bound(PartitionKey key, Datum *tuple, RangeBound *bound)
 {
 	int32	cmpval = partition_range_tuple_cmp(key, tuple, bound->val);
 
 	if (!cmpval)
 		return !bound->lower ? bound->inclusive : !bound->inclusive;
 
 	return cmpval < 0;
 }
+
+/*
+ * find_partition_scheme
+ * 		Find the "canonical" partition scheme for the given base table.
+ *
+ * 	The function searches the list of canonical partition schemes for one that
+ * 	exactly matches the partitioning properties of the given relation. If it
+ * 	does not find one, the function creates a canonical partition scheme
+ * 	structure and adds it to the list.
+ *
+ * 	For an umpartitioned table, it returns NULL.
+ */
+extern PartitionScheme
+find_partition_scheme(PlannerInfo *root, Relation relation)
+{
+	PartitionKey	part_key = RelationGetPartitionKey(relation);
+	PartitionDesc	part_desc = RelationGetPartitionDesc(relation);
+	ListCell	   *lc;
+	int		nparts;
+	int		partnatts;
+	int		cnt_pks;
+	int		cnt_parts;
+	PartitionScheme	part_scheme = NULL;
+
+	/* No partition scheme for an unpartitioned relation. */
+	if (!part_desc || !part_key)
+		return NULL;
+
+	nparts = part_desc->nparts;
+	partnatts = part_key->partnatts;
+
+	/* Search for a matching partition scheme and return if found one. */
+	foreach (lc, root->part_schemes)
+	{
+		part_scheme = lfirst(lc);
+
+		/* Match number of partitions and partitioning strategy. */
+		if (nparts != part_scheme->nparts ||
+			part_key->strategy != part_scheme->strategy ||
+			partnatts != part_scheme->partnatts)
+			continue;
+
+		/* Match the partition key types. */
+		for (cnt_pks = 0; cnt_pks < partnatts; cnt_pks++)
+		{
+			/*
+			 * It suffices to check the OID of support function as it always has
+			 * two arguemnts and returns boolean. For types, it suffices to match
+			 * the type id, mod and collation; len, byval and align are depedent on
+			 * the first two.
+			 */
+			if (part_key->partopfamily[cnt_pks] != part_scheme->partopfamily[cnt_pks] ||
+				part_key->partopcintype[cnt_pks] != part_scheme->partopcintype[cnt_pks] ||
+				part_key->tcinfo->typid[cnt_pks] != part_scheme->key_types[cnt_pks] ||
+				part_key->tcinfo->typmod[cnt_pks] != part_scheme->key_typmods[cnt_pks] ||
+				part_key->tcinfo->typcoll[cnt_pks] != part_scheme->key_collations[cnt_pks])
+				break;
+		}
+
+		/* Some partition key didn't match. Check next partitioning scheme. */
+		if (cnt_pks < partnatts)
+			continue;
+
+		/* Match partition bounds or lists. */
+		switch (part_scheme->strategy)
+		{
+			case PARTITION_STRAT_LIST:
+				if (!have_same_partition_lists(part_desc, part_key,
+											   part_scheme))
+					continue;
+				break;
+
+			case PARTITION_STRAT_RANGE:
+				if (!have_same_partition_bounds(part_desc, part_key,
+												part_scheme))
+					continue;
+				break;
+
+			default:
+				/* Unknown partition strategy. */
+				elog(ERROR, "unknown partition strategy code %d",
+					 part_key->strategy);
+				break;
+		}
+
+		/* Found matching partition scheme. */
+		return part_scheme;
+	}
+
+	/* Did not find matching partition scheme. Create one. */
+	part_scheme = (PartitionScheme) palloc0(sizeof(PartitionSchemeData));
+
+	/* Copy partition bounds/lists. */
+	part_scheme->nparts = part_desc->nparts;
+	part_scheme->strategy = part_key->strategy;
+	switch (part_scheme->strategy)
+	{
+		case PARTITION_STRAT_LIST:
+			part_scheme->lists = (PartitionListInfo **) palloc(sizeof(PartitionListInfo *) * nparts);
+			part_scheme->ranges = NULL;
+			for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+				part_scheme->lists[cnt_parts] = copy_list_info(part_desc->parts[cnt_parts]->list,
+															   part_key);
+			break;
+
+		case PARTITION_STRAT_RANGE:
+			part_scheme->ranges = (PartitionRangeInfo **) palloc(sizeof(PartitionRangeInfo *) * nparts);
+			part_scheme->lists = NULL;
+			for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+				part_scheme->ranges[cnt_parts] = copy_range_info(part_desc->parts[cnt_parts]->range,
+																 part_key);
+			break;
+
+		default:
+			elog(ERROR, "unknown partition strategy code %d",
+				 part_key->strategy);
+			break;
+	}
+
+	/* Store partition key information. */
+	part_scheme->partnatts = part_key->partnatts;
+
+	part_scheme->partopfamily = (Oid *) palloc(sizeof(Oid) * partnatts);
+	part_scheme->partopcintype = (Oid *) palloc(sizeof(Oid) * partnatts);
+	part_scheme->key_types = (Oid *) palloc(sizeof(Oid) * partnatts);
+	part_scheme->key_typmods = (int32 *) palloc(sizeof(int32) * partnatts);
+	part_scheme->key_collations = (Oid *) palloc(sizeof(Oid) * partnatts);
+
+	for (cnt_pks = 0; cnt_pks < partnatts; cnt_pks++)
+	{
+		part_scheme->partopfamily[cnt_pks] = part_key->partopfamily[cnt_pks];
+		part_scheme->partopcintype[cnt_pks] = part_key->partopcintype[cnt_pks];
+		part_scheme->key_types[cnt_pks] = part_key->tcinfo->typid[cnt_pks];
+		part_scheme->key_typmods[cnt_pks] = part_key->tcinfo->typmod[cnt_pks];
+		part_scheme->key_collations[cnt_pks] = part_key->tcinfo->typcoll[cnt_pks];
+	}
+
+	/* Add the partitioning scheme to PlannerInfo. */
+	root->part_schemes = lappend(root->part_schemes, part_scheme);
+
+	return part_scheme;
+}
+
+/*
+ * have_same_partition_lists
+ *
+ * For given list partitioned relations, return true if lists for all the
+ * partitions of both the relations match. Return false otherwise.
+ */
+static bool
+have_same_partition_lists(PartitionDesc part_desc, PartitionKey part_key,
+						  PartitionScheme part_scheme)
+{
+	int		cnt_parts;
+
+	Assert(part_scheme->strategy == part_key->strategy);
+	Assert(part_key->strategy == PARTITION_STRAT_LIST &&
+		   part_key->strategy == PARTITION_STRAT_LIST);
+
+	Assert(part_scheme->nparts == part_desc->nparts);
+
+	/* List partition has only one partition key. */
+	Assert(part_key->partnatts == 1 && part_scheme->partnatts == 1);
+
+	for (cnt_parts = 0; cnt_parts < part_desc->nparts; cnt_parts++)
+	{
+		if (!equal_list_info(part_key, part_scheme->lists[cnt_parts],
+							 part_desc->parts[cnt_parts]->list))
+			return false;
+	}
+
+	/* Ok, everything matches, return true. */
+	return true;
+}
+
+/*
+ * have_same_partition_bounds
+ *
+ * For given partitioned relations, return true if the bounds of all the
+ * partitions of the both the relations match. Return false otherwise.
+ */
+static bool
+have_same_partition_bounds(PartitionDesc part_desc, PartitionKey part_key,
+						   PartitionScheme part_scheme)
+{
+	int		cnt_parts;
+	Assert(part_key->strategy == PARTITION_STRAT_RANGE &&
+		   part_scheme->strategy == PARTITION_STRAT_RANGE);
+
+	Assert(part_desc->nparts == part_scheme->nparts);
+
+	for (cnt_parts = 0; cnt_parts < part_desc->nparts; cnt_parts++)
+	{
+		if (!equal_range_info(part_key, part_scheme->ranges[cnt_parts],
+							  part_desc->parts[cnt_parts]->range))
+			return false;
+	}
+
+	/* Ok, everything matches. */
+	return true;
+}
+
+extern List **
+build_baserel_partition_keys(PlannerInfo *root, Relation relation, Index varno)
+{
+	PartitionKey	part_key = RelationGetPartitionKey(relation);
+	int		num_pkexprs;
+	int		cnt_pke;
+	List		  **partexprs;
+	ListCell	   *lc;
+
+	if (!part_key || part_key->partnatts <= 0)
+		return NULL;
+
+	num_pkexprs = part_key->partnatts;
+
+	/* Store partition keys as single elements lists. */
+	partexprs = (List **) palloc(sizeof(List *) * num_pkexprs);
+
+	lc = list_head(part_key->partexprs);
+	for (cnt_pke = 0; cnt_pke < num_pkexprs; cnt_pke++)
+	{
+		AttrNumber attno = part_key->partattrs[cnt_pke];
+		Expr	  *pkexpr;
+
+		if (attno != InvalidAttrNumber)
+		{
+			/* Single column partition key is stored as a Var node. */
+			Form_pg_attribute att_tup;
+
+			if (attno < 0)
+				att_tup = SystemAttributeDefinition(attno,
+												relation->rd_rel->relhasoids);
+			else
+				att_tup = relation->rd_att->attrs[attno - 1];
+
+			pkexpr = (Expr *) makeVar(varno, attno, att_tup->atttypid,
+									  att_tup->atttypmod,
+									  att_tup->attcollation, 0);
+		}
+		else
+		{
+			if (!lc)
+				elog(ERROR, "wrong number of partition key expressions");
+			pkexpr = (Expr *) copyObject(lfirst(lc));
+			ChangeVarNodes((Node *) pkexpr, 1, varno, 0);
+			lc = lnext(lc);
+		}
+
+		partexprs[cnt_pke] = list_make1(pkexpr);
+	}
+
+	return partexprs;
+}
+
+/*
+ * Returns the number of partitions supported by the given partition scheme.
+ */
+extern int
+PartitionSchemeGetNumParts(PartitionScheme part_scheme)
+{
+	return part_scheme ? part_scheme->nparts : 0;
+}
+
+/*
+ * Returns the number of partition keys supported by the given partition
+ * scheme.
+ */
+extern int
+PartitionSchemeGetNumKeys(PartitionScheme part_scheme)
+{
+	return part_scheme ? part_scheme->partnatts : 0;
+}
+
+/*
+ * Set up partitioning scheme and partition keys for a join between given two
+ * relations.
+ */
+extern void
+build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
+							 RelOptInfo *inner_rel, JoinType jointype)
+{
+	int		num_pks;
+	int		cnt;
+
+	/*
+	 * Nothing to do if
+	 * a. partition-wise join is disabled.
+	 * b. joining relations are not partitioned.
+	 * c. partitioning schemes do not match.
+	 */
+	if (!enable_partition_wise_join ||
+		!outer_rel->part_scheme || !inner_rel->part_scheme ||
+		outer_rel->part_scheme != inner_rel->part_scheme)
+	{
+		joinrel->part_scheme = NULL;
+		return;
+	}
+
+	/*
+	 * This function will be called only once for each joinrel and thus we
+	 * should not see partition scheme, partition keys and array for storing
+	 * child relations set.
+	 */
+	Assert(!joinrel->part_scheme && !joinrel->partexprs &&
+		   !joinrel->part_rels);
+
+	num_pks = outer_rel->part_scheme->partnatts;
+
+	/* Join relation is partitioned in the same way as the joining relation. */
+	joinrel->part_scheme = outer_rel->part_scheme;
+
+	/*
+	 * Collect the partition key expressions.
+	 * TODO: for an outer join we may want to exclude the inner side. For
+	 * a FULL OUTER join this means, that it doesn't have any partition
+	 * keys.
+	 */
+	joinrel->partexprs = (List **) palloc0(sizeof(List *) * num_pks);
+	for (cnt = 0; cnt < num_pks; cnt++)
+	{
+		List *pkexpr = list_copy(outer_rel->partexprs[cnt]);
+
+		pkexpr = list_concat(pkexpr,
+							 list_copy(inner_rel->partexprs[cnt]));
+		joinrel->partexprs[cnt] = pkexpr;
+	}
+}
+
+/*
+ * Return palloc'ed array of OIDs of partitions.
+ */
+extern Oid *
+PartitionDescGetPartOids(PartitionDesc part_desc)
+{
+	Oid	   *part_oids;
+	int		cnt_parts;
+
+	if (!part_desc || part_desc->nparts <= 0)
+		return NULL;
+
+	part_oids = (Oid *) palloc(sizeof(Oid) * part_desc->nparts);
+	for (cnt_parts = 0; cnt_parts < part_desc->nparts; cnt_parts++)
+		part_oids[cnt_parts] = part_desc->parts[cnt_parts]->oid;
+
+	return part_oids;
+}
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 3b8fa6b..cd14840 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -204,20 +204,31 @@ _outBitmapset(StringInfo str, const Bitmapset *bms)
 }
 
 /* for use by extensions which define extensible nodes */
 void
 outBitmapset(StringInfo str, const Bitmapset *bms)
 {
 	_outBitmapset(str, bms);
 }
 
 /*
+ * TODO: remove, used for debugging through gdb.
+ */
+char *
+bms_to_char(const Bitmapset *bms)
+{
+	StringInfo str = makeStringInfo();
+	outBitmapset(str, bms);
+	return str->data;
+}
+
+/*
  * Print the value of a Datum given its type.
  */
 void
 outDatum(StringInfo str, Datum value, int typlen, bool typbyval)
 {
 	Size		length,
 				i;
 	char	   *s;
 
 	length = datumGetSize(value, typbyval, typlen);
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 04264b4..ad6ccf2 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -13,20 +13,21 @@
  *-------------------------------------------------------------------------
  */
 
 #include "postgres.h"
 
 #include <limits.h>
 #include <math.h>
 
 #include "access/sysattr.h"
 #include "access/tsmapi.h"
+#include "catalog/partition.h"
 #include "catalog/pg_class.h"
 #include "catalog/pg_operator.h"
 #include "catalog/pg_proc.h"
 #include "foreign/fdwapi.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
 #ifdef OPTIMIZER_DEBUG
 #include "nodes/print.h"
 #endif
 #include "optimizer/clauses.h"
@@ -37,20 +38,21 @@
 #include "optimizer/plancat.h"
 #include "optimizer/planner.h"
 #include "optimizer/prep.h"
 #include "optimizer/restrictinfo.h"
 #include "optimizer/tlist.h"
 #include "optimizer/var.h"
 #include "parser/parse_clause.h"
 #include "parser/parsetree.h"
 #include "rewrite/rewriteManip.h"
 #include "utils/lsyscache.h"
+#include "utils/rel.h"
 
 
 /* results of subquery_is_pushdown_safe */
 typedef struct pushdown_safety_info
 {
 	bool	   *unsafeColumns;	/* which output columns are unsafe to use */
 	bool		unsafeVolatile; /* don't push down volatile quals */
 	bool		unsafeLeaky;	/* don't push down leaky quals */
 } pushdown_safety_info;
 
@@ -119,20 +121,24 @@ static void check_output_expressions(Query *subquery,
 static void compare_tlist_datatypes(List *tlist, List *colTypes,
 						pushdown_safety_info *safetyInfo);
 static bool targetIsInAllPartitionLists(TargetEntry *tle, Query *query);
 static bool qual_is_pushdown_safe(Query *subquery, Index rti, Node *qual,
 					  pushdown_safety_info *safetyInfo);
 static void subquery_push_qual(Query *subquery,
 				   RangeTblEntry *rte, Index rti, Node *qual);
 static void recurse_push_qual(Node *setOp, Query *topquery,
 				  RangeTblEntry *rte, Index rti, Node *qual);
 static void remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel);
+static void add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
+									List *live_childrels);
+static void generate_partition_wise_join_paths(PlannerInfo *root,
+											   RelOptInfo *rel);
 
 
 /*
  * make_one_rel
  *	  Finds all possible access paths for executing a query, returning a
  *	  single rel that represents the join of all base rels in the query.
  */
 RelOptInfo *
 make_one_rel(PlannerInfo *root, List *joinlist)
 {
@@ -861,20 +867,48 @@ static void
 set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 					Index rti, RangeTblEntry *rte)
 {
 	int			parentRTindex = rti;
 	bool		has_live_children;
 	double		parent_rows;
 	double		parent_size;
 	double	   *parent_attrsizes;
 	int			nattrs;
 	ListCell   *l;
+	Oid		   *part_oids = NULL;
+	int			nparts = 0;
+
+	/*
+	 * For a partitioned relation, we will save the child RelOptInfos in parent
+	 * RelOptInfo in the same the order as corresponding bounds/lists are
+	 * stored in the partition scheme.
+	 */
+	if (rel->part_scheme)
+	{
+		RangeTblEntry	*rte = root->simple_rte_array[rel->relid];
+
+		/*
+		 * We need not lock the relation since it was already locked, either by
+		 * the rewriter or when expand_inherited_rtentry() added it to the
+		 * query's rangetable.
+		 */
+		Relation	relation = heap_open(rte->relid, NoLock);
+		PartitionDesc	part_desc = RelationGetPartitionDesc(relation);
+
+		part_oids = PartitionDescGetPartOids(part_desc);
+		nparts = part_desc->nparts;
+
+		Assert(part_oids && nparts > 0);
+
+		rel->part_rels = (RelOptInfo **)palloc0(sizeof(RelOptInfo *) * nparts);
+		heap_close(relation, NoLock);
+	}
 
 	/*
 	 * Initialize to compute size estimates for whole append relation.
 	 *
 	 * We handle width estimates by weighting the widths of different child
 	 * rels proportionally to their number of rows.  This is sensible because
 	 * the use of width estimates is mainly to compute the total relation
 	 * "footprint" if we have to sort or hash it.  To do this, we sum the
 	 * total equivalent size (in "double" arithmetic) and then divide by the
 	 * total rowcount estimate.  This is done separately for the total rel
@@ -892,36 +926,80 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 	foreach(l, root->append_rel_list)
 	{
 		AppendRelInfo *appinfo = (AppendRelInfo *) lfirst(l);
 		int			childRTindex;
 		RangeTblEntry *childRTE;
 		RelOptInfo *childrel;
 		List	   *childquals;
 		Node	   *childqual;
 		ListCell   *parentvars;
 		ListCell   *childvars;
+		int			cnt_parts;
 
 		/* append_rel_list contains all append rels; ignore others */
 		if (appinfo->parent_relid != parentRTindex)
 			continue;
 
 		childRTindex = appinfo->child_relid;
 		childRTE = root->simple_rte_array[childRTindex];
 
 		/*
 		 * The child rel's RelOptInfo was already created during
 		 * add_base_rels_to_query.
 		 */
 		childrel = find_base_rel(root, childRTindex);
+
+		/*
+		 * Save topmost parent's relid. If the parent itself is a child of some
+		 * other relation, use parent's topmost parent relids.
+		 */
+		if (rel->top_parent_relids)
+			childrel->top_parent_relids = rel->top_parent_relids;
+		else
+			childrel->top_parent_relids = bms_copy(rel->relids);
+
 		Assert(childrel->reloptkind == RELOPT_OTHER_MEMBER_REL);
 
 		/*
+		 * For a partitioned table, save the child RelOptInfo at its
+		 * appropriate place in the parent RelOptInfo.
+		 */
+		for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+		{
+			if (part_oids[cnt_parts] == childRTE->relid)
+			{
+				Assert(!rel->part_rels[cnt_parts]);
+				rel->part_rels[cnt_parts] = childrel;
+			}
+		}
+
+		/*
+		 * Copy/Modify targetlist. Partition-wise join technique may consider
+		 * an OUTER join of another child relation with this child relation.
+		 * In that case, even if this child is deemed empty, we will require
+		 * the targetlist of this child to construct the nullable side. Hence
+		 * set the targetlist before we prove that the child is empty and stop
+		 * processing further.
+		 *
+		 * NB: the resulting childrel->reltarget->exprs may contain arbitrary
+		 * expressions, which otherwise would not occur in a rel's targetlist.
+		 * Code that might be looking at an appendrel child must cope with
+		 * such.  (Normally, a rel's targetlist would only include Vars and
+		 * PlaceHolderVars.)  XXX we do not bother to update the cost or width
+		 * fields of childrel->reltarget; not clear if that would be useful.
+		 */
+		childrel->reltarget->exprs = (List *)
+			adjust_appendrel_attrs(root,
+								   (Node *) rel->reltarget->exprs,
+								   appinfo);
+
+		/*
 		 * We have to copy the parent's targetlist and quals to the child,
 		 * with appropriate substitution of variables.  However, only the
 		 * baserestrictinfo quals are needed before we can check for
 		 * constraint exclusion; so do that first and then check to see if we
 		 * can disregard this child.
 		 *
 		 * As of 8.4, the child rel's targetlist might contain non-Var
 		 * expressions, which means that substitution into the quals could
 		 * produce opportunities for const-simplification, and perhaps even
 		 * pseudoconstant quals.  To deal with this, we strip the RestrictInfo
@@ -953,38 +1031,25 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		if (relation_excluded_by_constraints(root, childrel, childRTE))
 		{
 			/*
 			 * This child need not be scanned, so we can omit it from the
 			 * appendrel.
 			 */
 			set_dummy_rel_pathlist(childrel);
 			continue;
 		}
 
-		/*
-		 * CE failed, so finish copying/modifying targetlist and join quals.
-		 *
-		 * NB: the resulting childrel->reltarget->exprs may contain arbitrary
-		 * expressions, which otherwise would not occur in a rel's targetlist.
-		 * Code that might be looking at an appendrel child must cope with
-		 * such.  (Normally, a rel's targetlist would only include Vars and
-		 * PlaceHolderVars.)  XXX we do not bother to update the cost or width
-		 * fields of childrel->reltarget; not clear if that would be useful.
-		 */
+		/* CE failed, so finish copying/modifying join quals. */
 		childrel->joininfo = (List *)
 			adjust_appendrel_attrs(root,
 								   (Node *) rel->joininfo,
 								   appinfo);
-		childrel->reltarget->exprs = (List *)
-			adjust_appendrel_attrs(root,
-								   (Node *) rel->reltarget->exprs,
-								   appinfo);
 
 		/*
 		 * We have to make child entries in the EquivalenceClass data
 		 * structures as well.  This is needed either if the parent
 		 * participates in some eclass joins (because we will want to consider
 		 * inner-indexscan joins on the individual children) or if the parent
 		 * has useful pathkeys (because we should try to build MergeAppend
 		 * paths that produce those sort orderings).
 		 */
 		if (rel->has_eclass_joins || has_useful_pathkeys(root, rel))
@@ -1073,20 +1138,28 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 				}
 				if (child_width <= 0)
 					child_width = get_typavgwidth(exprType(childvar),
 												  exprTypmod(childvar));
 				Assert(child_width > 0);
 				parent_attrsizes[pndx] += child_width * childrel->rows;
 			}
 		}
 	}
 
+	/* Should have found all the childrels of a partitioned relation. */
+	if (rel->part_scheme)
+	{
+		int		cnt_parts;
+		for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+			Assert(rel->part_rels[cnt_parts]);
+	}
+
 	if (has_live_children)
 	{
 		/*
 		 * Save the finished size estimates.
 		 */
 		int			i;
 
 		Assert(parent_rows > 0);
 		rel->rows = parent_rows;
 		rel->reltarget->width = rint(parent_size / parent_rows);
@@ -1115,41 +1188,32 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 /*
  * set_append_rel_pathlist
  *	  Build access paths for an "append relation"
  */
 static void
 set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 						Index rti, RangeTblEntry *rte)
 {
 	int			parentRTindex = rti;
 	List	   *live_childrels = NIL;
-	List	   *subpaths = NIL;
-	bool		subpaths_valid = true;
-	List	   *partial_subpaths = NIL;
-	bool		partial_subpaths_valid = true;
-	List	   *all_child_pathkeys = NIL;
-	List	   *all_child_outers = NIL;
 	ListCell   *l;
 
 	/*
-	 * Generate access paths for each member relation, and remember the
-	 * cheapest path for each one.  Also, identify all pathkeys (orderings)
-	 * and parameterizations (required_outer sets) available for the member
-	 * relations.
+	 * Generate access paths for each member relation and remember the
+	 * non-dummy children.
 	 */
 	foreach(l, root->append_rel_list)
 	{
 		AppendRelInfo *appinfo = (AppendRelInfo *) lfirst(l);
 		int			childRTindex;
 		RangeTblEntry *childRTE;
 		RelOptInfo *childrel;
-		ListCell   *lcp;
 
 		/* append_rel_list contains all append rels; ignore others */
 		if (appinfo->parent_relid != parentRTindex)
 			continue;
 
 		/* Re-locate the child RTE and RelOptInfo */
 		childRTindex = appinfo->child_relid;
 		childRTE = root->simple_rte_array[childRTindex];
 		childrel = root->simple_rel_array[childRTindex];
 
@@ -1170,20 +1234,46 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 		/*
 		 * If child is dummy, ignore it.
 		 */
 		if (IS_DUMMY_REL(childrel))
 			continue;
 
 		/*
 		 * Child is live, so add it to the live_childrels list for use below.
 		 */
 		live_childrels = lappend(live_childrels, childrel);
+	}
+
+	add_paths_to_append_rel(root, rel, live_childrels);
+}
+
+static void
+add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
+						List *live_childrels)
+{
+	List	   *subpaths = NIL;
+	bool		subpaths_valid = true;
+	List	   *partial_subpaths = NIL;
+	bool		partial_subpaths_valid = true;
+	List	   *all_child_pathkeys = NIL;
+	List	   *all_child_outers = NIL;
+	ListCell   *l;
+
+	/*
+	 * For every non-dummy child, remember the cheapest path.  Also, identify
+	 * all pathkeys (orderings) and parameterizations (required_outer sets)
+	 * available for the non-dummy member relations.
+	 */
+	foreach (l, live_childrels)
+	{
+		RelOptInfo *childrel = lfirst(l);
+		ListCell   *lcp;
 
 		/*
 		 * If child has an unparameterized cheapest-total path, add that to
 		 * the unparameterized Append path we are constructing for the parent.
 		 * If not, there's no workable unparameterized path.
 		 */
 		if (childrel->cheapest_total_path->param_info == NULL)
 			subpaths = accumulate_append_subpath(subpaths,
 											  childrel->cheapest_total_path);
 		else
@@ -2188,20 +2278,22 @@ standard_join_search(PlannerInfo *root, int levels_needed, List *initial_rels)
 		 * Run generate_gather_paths() for each just-processed joinrel.  We
 		 * could not do this earlier because both regular and partial paths
 		 * can get added to a particular joinrel at multiple times within
 		 * join_search_one_level.  After that, we're done creating paths for
 		 * the joinrel, so run set_cheapest().
 		 */
 		foreach(lc, root->join_rel_level[lev])
 		{
 			rel = (RelOptInfo *) lfirst(lc);
 
+			generate_partition_wise_join_paths(root, rel);
+
 			/* Create GatherPaths for any useful partial paths for rel */
 			generate_gather_paths(root, rel);
 
 			/* Find and save the cheapest paths for this rel */
 			set_cheapest(rel);
 
 #ifdef OPTIMIZER_DEBUG
 			debug_print_rel(root, rel);
 #endif
 		}
@@ -2849,20 +2941,73 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
 		 * OK, we don't need it.  Replace the expression with a NULL constant.
 		 * Preserve the exposed type of the expression, in case something
 		 * looks at the rowtype of the subquery's result.
 		 */
 		tle->expr = (Expr *) makeNullConst(exprType(texpr),
 										   exprTypmod(texpr),
 										   exprCollation(texpr));
 	}
 }
 
+/*
+ * generate_partition_wise_join_paths
+ * 		Create appends paths containing partition-wise join paths for given
+ * 		join relation.
+ */
+static void
+generate_partition_wise_join_paths(PlannerInfo *root, RelOptInfo *rel)
+{
+	List	   *live_children = NIL;
+	int			cnt_part;
+	int			nparts = PartitionSchemeGetNumParts(rel->part_scheme);
+
+	/* Handle only join relations. */
+	if (rel->reloptkind != RELOPT_JOINREL &&
+		rel->reloptkind != RELOPT_OTHER_JOINREL)
+		return;
+
+	/* If the relation is not partitioned, nothing to do. */
+	if (!rel->part_scheme || !rel->part_rels)
+		return;
+
+	for (cnt_part = 0; cnt_part < nparts; cnt_part++)
+	{
+		RelOptInfo	*child_rel = rel->part_rels[cnt_part];
+
+		/* Ignore dummy child. */
+		if (!IS_DUMMY_REL(child_rel))
+		{
+			/* Recursively collect the paths from child joinrel. */
+			generate_partition_wise_join_paths(root, child_rel);
+
+			/* Find the cheapest of the paths for this rel. */
+			set_cheapest(child_rel);
+
+#ifdef OPTIMIZER_DEBUG
+			debug_print_rel(root, rel);
+#endif
+
+			live_children = lappend(live_children, child_rel);
+		}
+	}
+
+	/*
+	 * Create append paths by collecting sub paths from live children. Even if
+	 * there are no live children, we should create an append path with no
+	 * subpaths i.e. a dummy access path.
+	 */
+	add_paths_to_append_rel(root, rel, live_children);
+
+	if (live_children)
+		pfree(live_children);
+}
+
 /*****************************************************************************
  *			DEBUG SUPPORT
  *****************************************************************************/
 
 #ifdef OPTIMIZER_DEBUG
 
 static void
 print_relids(PlannerInfo *root, Relids relids)
 {
 	int			x;
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 2a49639..a23da1c 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -119,20 +119,21 @@ bool		enable_seqscan = true;
 bool		enable_indexscan = true;
 bool		enable_indexonlyscan = true;
 bool		enable_bitmapscan = true;
 bool		enable_tidscan = true;
 bool		enable_sort = true;
 bool		enable_hashagg = true;
 bool		enable_nestloop = true;
 bool		enable_material = true;
 bool		enable_mergejoin = true;
 bool		enable_hashjoin = true;
+bool		enable_partition_wise_join = true;
 
 typedef struct
 {
 	PlannerInfo *root;
 	QualCost	total;
 } cost_qual_eval_context;
 
 static List *extract_nonindex_conditions(List *qual_clauses, List *indexquals);
 static MergeScanSelCache *cached_scansel(PlannerInfo *root,
 			   RestrictInfo *rinfo,
diff --git a/src/backend/optimizer/path/equivclass.c b/src/backend/optimizer/path/equivclass.c
index 0e50ad5..73026a3 100644
--- a/src/backend/optimizer/path/equivclass.c
+++ b/src/backend/optimizer/path/equivclass.c
@@ -2359,20 +2359,22 @@ eclass_useful_for_merging(PlannerInfo *root,
 
 	/*
 	 * Note we don't test ec_broken; if we did, we'd need a separate code path
 	 * to look through ec_sources.  Checking the members anyway is OK as a
 	 * possibly-overoptimistic heuristic.
 	 */
 
 	/* If specified rel is a child, we must consider the topmost parent rel */
 	if (rel->reloptkind == RELOPT_OTHER_MEMBER_REL)
 		relids = find_childrel_top_parent(root, rel)->relids;
+	else if (rel->reloptkind == RELOPT_OTHER_JOINREL)
+		relids = rel->top_parent_relids;
 	else
 		relids = rel->relids;
 
 	/* If rel already includes all members of eclass, no point in searching */
 	if (bms_is_subset(eclass->ec_relids, relids))
 		return false;
 
 	/* To join, we need a member not in the given rel */
 	foreach(lc, eclass->ec_members)
 	{
diff --git a/src/backend/optimizer/path/joinpath.c b/src/backend/optimizer/path/joinpath.c
index cc7384f..db0e469 100644
--- a/src/backend/optimizer/path/joinpath.c
+++ b/src/backend/optimizer/path/joinpath.c
@@ -18,23 +18,29 @@
 
 #include "executor/executor.h"
 #include "foreign/fdwapi.h"
 #include "optimizer/cost.h"
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
 
 /* Hook for plugins to get control in add_paths_to_joinrel() */
 set_join_pathlist_hook_type set_join_pathlist_hook = NULL;
 
-#define PATH_PARAM_BY_REL(path, rel)  \
+#define PATH_PARAM_BY_PARENT(path, rel)	\
+	((path)->param_info && bms_overlap(PATH_REQ_OUTER(path),	\
+									   (rel)->top_parent_relids))
+#define PATH_PARAM_BY_REL_SELF(path, rel)  \
 	((path)->param_info && bms_overlap(PATH_REQ_OUTER(path), (rel)->relids))
 
+#define PATH_PARAM_BY_REL(path, rel)	\
+	(PATH_PARAM_BY_REL_SELF(path, rel) || PATH_PARAM_BY_PARENT(path, rel))
+
 static void sort_inner_and_outer(PlannerInfo *root, RelOptInfo *joinrel,
 					 RelOptInfo *outerrel, RelOptInfo *innerrel,
 					 JoinType jointype, JoinPathExtraData *extra);
 static void match_unsorted_outer(PlannerInfo *root, RelOptInfo *joinrel,
 					 RelOptInfo *outerrel, RelOptInfo *innerrel,
 					 JoinType jointype, JoinPathExtraData *extra);
 static void consider_parallel_nestloop(PlannerInfo *root,
 						   RelOptInfo *joinrel,
 						   RelOptInfo *outerrel,
 						   RelOptInfo *innerrel,
@@ -125,38 +131,51 @@ add_paths_to_joinrel(PlannerInfo *root,
 	 * directly to the parameter source rel instead of joining to the other
 	 * input rel.  (But see allow_star_schema_join().)	This restriction
 	 * reduces the number of parameterized paths we have to deal with at
 	 * higher join levels, without compromising the quality of the resulting
 	 * plan.  We express the restriction as a Relids set that must overlap the
 	 * parameterization of any proposed join path.
 	 */
 	foreach(lc, root->join_info_list)
 	{
 		SpecialJoinInfo *sjinfo = (SpecialJoinInfo *) lfirst(lc);
+		Relids	joinrelids;
+
+		/*
+		 * PlannerInfo doesn't contain the SpecialJoinInfos created for joins
+		 * between partitions, even if there is a SpecialJoinInfo node for the
+		 * join between the topmost parents. Hence while calculating Relids set
+		 * representing the restriction, consider relids of topmost parent of
+		 * partitions.
+		 */
+		if (joinrel->reloptkind == RELOPT_OTHER_JOINREL)
+			joinrelids = joinrel->top_parent_relids;
+		else
+			joinrelids = joinrel->relids;
 
 		/*
 		 * SJ is relevant to this join if we have some part of its RHS
 		 * (possibly not all of it), and haven't yet joined to its LHS.  (This
 		 * test is pretty simplistic, but should be sufficient considering the
 		 * join has already been proven legal.)  If the SJ is relevant, it
 		 * presents constraints for joining to anything not in its RHS.
 		 */
-		if (bms_overlap(joinrel->relids, sjinfo->min_righthand) &&
-			!bms_overlap(joinrel->relids, sjinfo->min_lefthand))
+		if (bms_overlap(joinrelids, sjinfo->min_righthand) &&
+			!bms_overlap(joinrelids, sjinfo->min_lefthand))
 			extra.param_source_rels = bms_join(extra.param_source_rels,
 										   bms_difference(root->all_baserels,
 													 sjinfo->min_righthand));
 
 		/* full joins constrain both sides symmetrically */
 		if (sjinfo->jointype == JOIN_FULL &&
-			bms_overlap(joinrel->relids, sjinfo->min_lefthand) &&
-			!bms_overlap(joinrel->relids, sjinfo->min_righthand))
+			bms_overlap(joinrelids, sjinfo->min_lefthand) &&
+			!bms_overlap(joinrelids, sjinfo->min_righthand))
 			extra.param_source_rels = bms_join(extra.param_source_rels,
 										   bms_difference(root->all_baserels,
 													  sjinfo->min_lefthand));
 	}
 
 	/*
 	 * However, when a LATERAL subquery is involved, there will simply not be
 	 * any paths for the joinrel that aren't parameterized by whatever the
 	 * subquery is parameterized by, unless its parameterization is resolved
 	 * within the joinrel.  So we might as well allow additional dependencies
@@ -272,20 +291,35 @@ try_nestloop_path(PlannerInfo *root,
 				  Path *outer_path,
 				  Path *inner_path,
 				  List *pathkeys,
 				  JoinType jointype,
 				  JoinPathExtraData *extra)
 {
 	Relids		required_outer;
 	JoinCostWorkspace workspace;
 
 	/*
+	 * An inner path parameterized by the parent relation of outer
+	 * relation needs to be reparameterized by the outer relation to be used
+	 * for parameterized nested loop join.
+	 */
+	if (PATH_PARAM_BY_PARENT(inner_path, outer_path->parent))
+	{
+		inner_path = reparameterize_path_for_child(root, inner_path,
+												   outer_path->parent);
+
+		/* If we could not translate the path, don't produce nest loop path. */
+		if (!inner_path)
+			return;
+	}
+
+	/*
 	 * Check to see if proposed path is still parameterized, and reject if the
 	 * parameterization wouldn't be sensible --- unless allow_star_schema_join
 	 * says to allow it anyway.  Also, we must reject if have_dangerous_phv
 	 * doesn't like the look of it, which could only happen if the nestloop is
 	 * still parameterized.
 	 */
 	required_outer = calc_nestloop_required_outer(outer_path,
 												  inner_path);
 	if (required_outer &&
 		((!bms_overlap(required_outer, extra->param_source_rels) &&
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 01d4fea..4c51dd2 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -7,38 +7,56 @@
  * Portions Copyright (c) 1994, Regents of the University of California
  *
  *
  * IDENTIFICATION
  *	  src/backend/optimizer/path/joinrels.c
  *
  *-------------------------------------------------------------------------
  */
 #include "postgres.h"
 
+#include "catalog/partition.h"
+#include "optimizer/clauses.h"
 #include "optimizer/joininfo.h"
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
+#include "optimizer/prep.h"
+#include "optimizer/cost.h"
 #include "utils/memutils.h"
 
 
 static void make_rels_by_clause_joins(PlannerInfo *root,
 						  RelOptInfo *old_rel,
 						  ListCell *other_rels);
 static void make_rels_by_clauseless_joins(PlannerInfo *root,
 							  RelOptInfo *old_rel,
 							  ListCell *other_rels);
 static bool has_join_restriction(PlannerInfo *root, RelOptInfo *rel);
 static bool has_legal_joinclause(PlannerInfo *root, RelOptInfo *rel);
 static bool is_dummy_rel(RelOptInfo *rel);
 static void mark_dummy_rel(RelOptInfo *rel);
 static bool restriction_is_constant_false(List *restrictlist,
 							  bool only_pushed_down);
+static void populate_joinrel_with_paths(PlannerInfo *root, RelOptInfo *rel1,
+							RelOptInfo *rel2, RelOptInfo *joinrel,
+							SpecialJoinInfo *sjinfo, List *restrictlist);
+static void try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1,
+						  RelOptInfo *rel2, RelOptInfo *joinrel,
+						  SpecialJoinInfo *parent_sjinfo,
+						  List *parent_restrictlist);
+static SpecialJoinInfo * build_partition_join_sjinfo(PlannerInfo *root,
+										 SpecialJoinInfo *parent_sjinfo,
+										 List *append_rel_infos1,
+										 List *append_rel_infos2);
+static bool have_partkey_equi_join(RelOptInfo *rel1, RelOptInfo *rel2,
+						   JoinType jointype, List *restrictlist);
+static int match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel);
 
 
 /*
  * join_search_one_level
  *	  Consider ways to produce join relations containing exactly 'level'
  *	  jointree items.  (This is one step of the dynamic-programming method
  *	  embodied in standard_join_search.)  Join rel nodes for each feasible
  *	  combination of lower-level rels are created and returned in a list.
  *	  Implementation paths are created for each such joinrel, too.
  *
@@ -717,20 +735,44 @@ make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 	/*
 	 * If we've already proven this join is empty, we needn't consider any
 	 * more paths for it.
 	 */
 	if (is_dummy_rel(joinrel))
 	{
 		bms_free(joinrelids);
 		return joinrel;
 	}
 
+	/* Add paths to the join relation. */
+	populate_joinrel_with_paths(root, rel1, rel2, joinrel, sjinfo,
+								restrictlist);
+
+	/* Apply partition-wise join technique, if possible. */
+	try_partition_wise_join(root, rel1, rel2, joinrel, sjinfo, restrictlist);
+
+	bms_free(joinrelids);
+
+	return joinrel;
+}
+
+/*
+ * populate_joinrel_with_paths
+ *	  Create paths to join given input relation and add those to the given
+ *	  joinrel. The SpecialJoinInfo provides details about the join and the
+ *	  restrictlist contains the join clauses and the other clauses applicable
+ *	  for given pair of the joining relations.
+ */
+static void
+populate_joinrel_with_paths(PlannerInfo *root, RelOptInfo *rel1,
+							RelOptInfo *rel2, RelOptInfo *joinrel,
+							SpecialJoinInfo *sjinfo, List *restrictlist)
+{
 	/*
 	 * Consider paths using each rel as both outer and inner.  Depending on
 	 * the join type, a provably empty outer or inner rel might mean the join
 	 * is provably empty too; in which case throw away any previously computed
 	 * paths and mark the join as dummy.  (We do it this way since it's
 	 * conceivable that dummy-ness of a multi-element join might only be
 	 * noticeable for certain construction paths.)
 	 *
 	 * Also, a provably constant-false join restriction typically means that
 	 * we can skip evaluating one or both sides of the join.  We do this by
@@ -861,27 +903,22 @@ make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 				mark_dummy_rel(rel2);
 			add_paths_to_joinrel(root, joinrel, rel1, rel2,
 								 JOIN_ANTI, sjinfo,
 								 restrictlist);
 			break;
 		default:
 			/* other values not expected here */
 			elog(ERROR, "unrecognized join type: %d", (int) sjinfo->jointype);
 			break;
 	}
-
-	bms_free(joinrelids);
-
-	return joinrel;
 }
 
-
 /*
  * have_join_order_restriction
  *		Detect whether the two relations should be joined to satisfy
  *		a join-order restriction arising from special or lateral joins.
  *
  * In practice this is always used with have_relevant_joinclause(), and so
  * could be merged with that function, but it seems clearer to separate the
  * two concerns.  We need this test because there are degenerate cases where
  * a clauseless join must be performed to satisfy join-order restrictions.
  * Also, if one rel has a lateral reference to the other, or both are needed
@@ -1242,10 +1279,314 @@ restriction_is_constant_false(List *restrictlist, bool only_pushed_down)
 
 			/* constant NULL is as good as constant FALSE for our purposes */
 			if (con->constisnull)
 				return true;
 			if (!DatumGetBool(con->constvalue))
 				return true;
 		}
 	}
 	return false;
 }
+
+/*
+ * If the join between the given two relations can be executed as
+ * partition-wise join create the join relations for partition-wise join,
+ * create paths for those and then create append paths to combine
+ * partition-wise join results.
+ */
+static void
+try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
+						RelOptInfo *joinrel, SpecialJoinInfo *parent_sjinfo,
+						List *parent_restrictlist)
+{
+	int		nparts;
+	int		cnt_parts;
+	StringInfo	rel1_desc;
+	StringInfo	rel2_desc;
+	PartitionScheme	part_scheme;
+
+
+	/* Nothing to do, if the join relation is not partitioned. */
+	if (!joinrel->part_scheme)
+		return;
+
+	/*
+	 * If an equi-join condition between the partition keys of the joining
+	 * relations does not exist, the given partition scheme can not be used for
+	 * partition-wise join between these two relations.
+	 */
+	if (!have_partkey_equi_join(rel1, rel2, parent_sjinfo->jointype,
+								parent_restrictlist))
+		return;
+
+	Assert(joinrel->part_scheme == rel1->part_scheme &&
+		   joinrel->part_scheme == rel2->part_scheme);
+
+	part_scheme = joinrel->part_scheme;
+	nparts = PartitionSchemeGetNumParts(part_scheme);
+	rel1_desc = makeStringInfo();
+	rel2_desc = makeStringInfo();
+
+	/* TODO: remove this notice when finalising the patch. */
+	outBitmapset(rel1_desc, rel1->relids);
+	outBitmapset(rel2_desc, rel2->relids);
+	elog(NOTICE, "join between relations %s and %s is considered for partition-wise join.",
+		 rel1_desc->data, rel2_desc->data);
+
+	/*
+	 * We allocate the array for child RelOptInfos till we find at least one
+	 * join order which can use partition-wise join technique. If no join order
+	 * can use partition-wise join technique, there are no child relations.
+	 */
+
+	if (!joinrel->part_rels)
+		joinrel->part_rels = (RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
+
+	/*
+	 * Create join relations for the partition relations, if they do not exist
+	 * already. Add paths to those for the given pair of joining relations.
+	 */
+	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	{
+		RelOptInfo *child_joinrel;
+		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
+		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
+		SpecialJoinInfo *child_sjinfo;
+		List	   *child_restrictlist;
+		List	   *join_aris;
+		List	   *ari1;
+		List	   *ari2;
+
+		/* We should never try to join two overlapping sets of rels. */
+		Assert(!bms_overlap(child_rel1->relids, child_rel2->relids));
+
+		/*
+		 * Gather the AppendRelInfos for base partition relations
+		 * partiticipating in the given partition relations. We need them
+		 * construct partition-wise join relation, special join info and
+		 * restriction list by substituting the Var and relids from parent to
+		 * child.
+		 */
+		ari1 = find_appendrelinfos_by_relids(root, child_rel1->relids);
+		ari2 = find_appendrelinfos_by_relids(root, child_rel2->relids);
+
+		/*
+		 * Construct SpecialJoinInfo from parent join relations's
+		 * SpecialJoinInfo.
+		 */
+		child_sjinfo = build_partition_join_sjinfo(root, parent_sjinfo, ari1, ari2);
+
+		/* Construct the parent-child relid map for the join relation. */
+		join_aris = list_concat(ari1, ari2);
+
+		/*
+		 * Construct restrictions applicable to the partition-wise join from
+		 * those applicable to the join between the parents.
+		 */
+		child_restrictlist = (List *) adjust_partitionrel_attrs(root,
+													(Node *)parent_restrictlist,
+													join_aris);
+
+		child_joinrel = joinrel->part_rels[cnt_parts];
+
+		/* Construct the join relation for given partition of the join. */
+		if (!child_joinrel)
+		{
+			child_joinrel = build_partition_join_rel(root, child_rel1,
+													 child_rel2, joinrel,
+													 child_sjinfo,
+													 child_restrictlist,
+													 join_aris);
+
+			joinrel->part_rels[cnt_parts] = child_joinrel;
+
+		}
+
+		/*
+		 * If we've already proven that this join is empty, we needn't consider
+		 * any more paths for it.
+		 */
+		if (is_dummy_rel(child_joinrel))
+			continue;
+
+		populate_joinrel_with_paths(root, child_rel1, child_rel2, child_joinrel,
+									child_sjinfo, child_restrictlist);
+
+		/*
+		 * If the partition themselves are partitioned, try partition-wise join
+		 * recursively.
+		 */
+		try_partition_wise_join(root, child_rel1, child_rel2, child_joinrel,
+								child_sjinfo, child_restrictlist);
+	}
+}
+
+/*
+ * Construct the SpecialJoinInfo for the partition-wise join using parents'
+ * special join info. Also, instead of
+ * constructing an sjinfo everytime, we should probably save it in
+ * root->join_info_list and search within it like join_is_legal?
+ */
+static SpecialJoinInfo *
+build_partition_join_sjinfo(PlannerInfo *root, SpecialJoinInfo *parent_sjinfo,
+							List *append_rel_infos1, List *append_rel_infos2)
+{
+	SpecialJoinInfo *sjinfo = copyObject(parent_sjinfo);
+	sjinfo->min_lefthand = adjust_partition_relids(sjinfo->min_lefthand,
+												   append_rel_infos1);
+	sjinfo->min_righthand = adjust_partition_relids(sjinfo->min_righthand,
+													append_rel_infos2);
+	sjinfo->syn_lefthand = adjust_partition_relids(sjinfo->syn_lefthand,
+												   append_rel_infos1);
+	sjinfo->syn_righthand = adjust_partition_relids(sjinfo->syn_righthand,
+													append_rel_infos2);
+
+	/* Replace the Var nodes of parent with those of children in expressions. */
+	sjinfo->semi_rhs_exprs = (List *) adjust_partitionrel_attrs(root,
+											   (Node *) sjinfo->semi_rhs_exprs,
+															append_rel_infos2);
+	return sjinfo;
+}
+
+/*
+ * Substitute oldrelids with newrelids in the given Relids set. It recycles the
+ * given relids input.
+ */
+Relids
+adjust_partition_relids(Relids relids, List *append_rel_infos)
+{
+	ListCell	*lc;
+	foreach (lc, append_rel_infos)
+	{
+		AppendRelInfo	*ari = lfirst(lc);
+
+		/* Remove old, add new */
+		if (bms_is_member(ari->parent_relid, relids))
+		{
+			relids = bms_del_member(relids, ari->parent_relid);
+			relids = bms_add_member(relids, ari->child_relid);
+		}
+	}
+	return relids;
+}
+
+/*
+ * Returns true if the given relations have equi-join clauses on all the
+ * corresponding partition keys.
+ */
+static bool
+have_partkey_equi_join(RelOptInfo *rel1, RelOptInfo *rel2,
+					 JoinType jointype, List *restrictlist)
+{
+	PartitionScheme	part_scheme = rel1->part_scheme;
+	ListCell	*lc;
+	int		cnt_pks;
+	int		num_pks = PartitionSchemeGetNumKeys(part_scheme);
+	bool   *pk_has_clause;
+
+	Assert(rel1->part_scheme == rel2->part_scheme);
+
+	pk_has_clause = (bool *) palloc0(sizeof(bool) * num_pks);
+
+	foreach (lc, restrictlist)
+	{
+		RestrictInfo *rinfo = lfirst(lc);
+		OpExpr		 *opexpr;
+		Expr		 *expr1;
+		Expr		 *expr2;
+		int		ipk1;
+		int		ipk2;
+
+		/* If processing an outer join, only use its own join clauses. */
+		if (IS_OUTER_JOIN(jointype) && rinfo->is_pushed_down)
+			continue;
+
+		/* Skip non-equi-join clauses. */
+		if (!rinfo->can_join ||
+			rinfo->hashjoinoperator == InvalidOid ||
+			!rinfo->mergeopfamilies)
+			continue;
+
+		opexpr = (OpExpr *) rinfo->clause;
+		Assert(is_opclause(opexpr));
+
+		/*
+		 * If clause of form rel1_expr op rel2_expr OR rel2_expr op rel1_expr,
+		 * match the operands to the relations. Otherwise, the clause is
+		 * not an equi-join between partition keys of joining relations.
+		 */
+		if (bms_is_subset(rinfo->left_relids, rel1->relids) &&
+			bms_is_subset(rinfo->right_relids, rel2->relids))
+		{
+			expr1 = linitial(opexpr->args);
+			expr2 = lsecond(opexpr->args);
+		}
+		else if (bms_is_subset(rinfo->left_relids, rel2->relids) &&
+				 bms_is_subset(rinfo->right_relids, rel1->relids))
+		{
+			expr1 = lsecond(opexpr->args);
+			expr2 = linitial(opexpr->args);
+		}
+		else
+			continue;
+
+		/* Associate matching clauses with partition keys. */
+		ipk1 = match_expr_to_partition_keys(expr1, rel1);
+		ipk2 = match_expr_to_partition_keys(expr2, rel2);
+
+		if (ipk1 == ipk2)
+			pk_has_clause[ipk1] = true;
+	}
+
+	/*
+	 * If every pair of partition key from either of the joining relation has
+	 * at least one equi-join clause associated with it, we have an equi-join
+	 * between all corresponding partition keys.
+	 */
+	for (cnt_pks = 0; cnt_pks < num_pks; cnt_pks++)
+	{
+		if (!pk_has_clause[cnt_pks])
+		{
+			pfree(pk_has_clause);
+			return false;
+		}
+	}
+
+	pfree(pk_has_clause);
+	return true;
+}
+
+/*
+ * match_expr_to_partition_keys
+ *
+ * Find the partition key which is same as the given expression. If found,
+ * return the index of the partition key, else return -1.
+ */
+static int
+match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel)
+{
+	int		cnt_pks;
+	int		num_pks = PartitionSchemeGetNumKeys(rel->part_scheme);
+
+	/*
+	 * Remove the relabel decoration. We can assume that there is at most one
+	 * RelabelType node; eval_const_expressions() will have simplied if more
+	 * than one.
+	 */
+	if (IsA(expr, RelabelType))
+		expr = (Expr *) ((RelabelType *) expr)->arg;
+
+	for (cnt_pks = 0; cnt_pks < num_pks; cnt_pks++)
+	{
+		List	 *pkexprs = rel->partexprs[cnt_pks];
+		ListCell *lc;
+
+		foreach(lc, pkexprs)
+		{
+			Expr *pkexpr = lfirst(lc);
+			if (equal(pkexpr, expr))
+				return cnt_pks;
+		}
+	}
+
+	return -1;
+}
diff --git a/src/backend/optimizer/path/pathkeys.c b/src/backend/optimizer/path/pathkeys.c
index 4436ac1..c8870c8 100644
--- a/src/backend/optimizer/path/pathkeys.c
+++ b/src/backend/optimizer/path/pathkeys.c
@@ -1081,26 +1081,38 @@ select_outer_pathkeys_for_merge(PlannerInfo *root,
 								List *mergeclauses,
 								RelOptInfo *joinrel)
 {
 	List	   *pathkeys = NIL;
 	int			nClauses = list_length(mergeclauses);
 	EquivalenceClass **ecs;
 	int		   *scores;
 	int			necs;
 	ListCell   *lc;
 	int			j;
+	Relids		relids;
 
 	/* Might have no mergeclauses */
 	if (nClauses == 0)
 		return NIL;
 
 	/*
+	 * Code below scores equivalence classes by how many equivalence members
+	 * can produce join clauses for this join relation. Equivalence members
+	 * which do not cover the parents of a partition-wise join relation, can
+	 * produce join clauses for partition-wise join relation.
+	 */
+	if (joinrel->reloptkind == RELOPT_OTHER_JOINREL)
+		relids = joinrel->top_parent_relids;
+	else
+		relids = joinrel->relids;
+
+	/*
 	 * Make arrays of the ECs used by the mergeclauses (dropping any
 	 * duplicates) and their "popularity" scores.
 	 */
 	ecs = (EquivalenceClass **) palloc(nClauses * sizeof(EquivalenceClass *));
 	scores = (int *) palloc(nClauses * sizeof(int));
 	necs = 0;
 
 	foreach(lc, mergeclauses)
 	{
 		RestrictInfo *rinfo = (RestrictInfo *) lfirst(lc);
@@ -1126,21 +1138,21 @@ select_outer_pathkeys_for_merge(PlannerInfo *root,
 			continue;
 
 		/* compute score */
 		score = 0;
 		foreach(lc2, oeclass->ec_members)
 		{
 			EquivalenceMember *em = (EquivalenceMember *) lfirst(lc2);
 
 			/* Potential future join partner? */
 			if (!em->em_is_const && !em->em_is_child &&
-				!bms_overlap(em->em_relids, joinrel->relids))
+				!bms_overlap(em->em_relids, relids))
 				score++;
 		}
 
 		ecs[necs] = oeclass;
 		scores[necs] = score;
 		necs++;
 	}
 
 	/*
 	 * Find out if we have all the ECs mentioned in query_pathkeys; if so we
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 32f4031..b221e2c 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -235,21 +235,22 @@ static Plan *prepare_sort_from_pathkeys(Plan *lefttree, List *pathkeys,
 						   const AttrNumber *reqColIdx,
 						   bool adjust_tlist_in_place,
 						   int *p_numsortkeys,
 						   AttrNumber **p_sortColIdx,
 						   Oid **p_sortOperators,
 						   Oid **p_collations,
 						   bool **p_nullsFirst);
 static EquivalenceMember *find_ec_member_for_tle(EquivalenceClass *ec,
 					   TargetEntry *tle,
 					   Relids relids);
-static Sort *make_sort_from_pathkeys(Plan *lefttree, List *pathkeys);
+static Sort *make_sort_from_pathkeys(Plan *lefttree, List *pathkeys,
+									 Relids relids);
 static Sort *make_sort_from_groupcols(List *groupcls,
 						 AttrNumber *grpColIdx,
 						 Plan *lefttree);
 static Material *make_material(Plan *lefttree);
 static WindowAgg *make_windowagg(List *tlist, Index winref,
 			   int partNumCols, AttrNumber *partColIdx, Oid *partOperators,
 			   int ordNumCols, AttrNumber *ordColIdx, Oid *ordOperators,
 			   int frameOptions, Node *startOffset, Node *endOffset,
 			   Plan *lefttree);
 static Group *make_group(List *tlist, List *qual, int numGroupCols,
@@ -1507,21 +1508,21 @@ create_sort_plan(PlannerInfo *root, SortPath *best_path, int flags)
 	Plan	   *subplan;
 
 	/*
 	 * We don't want any excess columns in the sorted tuples, so request a
 	 * smaller tlist.  Otherwise, since Sort doesn't project, tlist
 	 * requirements pass through.
 	 */
 	subplan = create_plan_recurse(root, best_path->subpath,
 								  flags | CP_SMALL_TLIST);
 
-	plan = make_sort_from_pathkeys(subplan, best_path->path.pathkeys);
+	plan = make_sort_from_pathkeys(subplan, best_path->path.pathkeys, NULL);
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
 	return plan;
 }
 
 /*
  * create_group_plan
  *
  *	  Create a Group plan for 'best_path' and (recursively) plans
@@ -3517,31 +3518,33 @@ create_mergejoin_plan(PlannerInfo *root,
 	List	   *innerpathkeys;
 	int			nClauses;
 	Oid		   *mergefamilies;
 	Oid		   *mergecollations;
 	int		   *mergestrategies;
 	bool	   *mergenullsfirst;
 	int			i;
 	ListCell   *lc;
 	ListCell   *lop;
 	ListCell   *lip;
+	Path	   *outer_path = best_path->jpath.outerjoinpath;
+	Path	   *inner_path = best_path->jpath.innerjoinpath;
 
 	/*
 	 * MergeJoin can project, so we don't have to demand exact tlists from the
 	 * inputs.  However, if we're intending to sort an input's result, it's
 	 * best to request a small tlist so we aren't sorting more data than
 	 * necessary.
 	 */
-	outer_plan = create_plan_recurse(root, best_path->jpath.outerjoinpath,
+	outer_plan = create_plan_recurse(root, outer_path,
 					 (best_path->outersortkeys != NIL) ? CP_SMALL_TLIST : 0);
 
-	inner_plan = create_plan_recurse(root, best_path->jpath.innerjoinpath,
+	inner_plan = create_plan_recurse(root, inner_path,
 					 (best_path->innersortkeys != NIL) ? CP_SMALL_TLIST : 0);
 
 	/* Sort join qual clauses into best execution order */
 	/* NB: do NOT reorder the mergeclauses */
 	joinclauses = order_qual_clauses(root, best_path->jpath.joinrestrictinfo);
 
 	/* Get the join qual clauses (in plain expression form) */
 	/* Any pseudoconstant clauses are ignored here */
 	if (IS_OUTER_JOIN(best_path->jpath.jointype))
 	{
@@ -3573,48 +3576,52 @@ create_mergejoin_plan(PlannerInfo *root,
 		otherclauses = (List *)
 			replace_nestloop_params(root, (Node *) otherclauses);
 	}
 
 	/*
 	 * Rearrange mergeclauses, if needed, so that the outer variable is always
 	 * on the left; mark the mergeclause restrictinfos with correct
 	 * outer_is_left status.
 	 */
 	mergeclauses = get_switched_clauses(best_path->path_mergeclauses,
-							 best_path->jpath.outerjoinpath->parent->relids);
+							 outer_path->parent->relids);
 
 	/*
 	 * Create explicit sort nodes for the outer and inner paths if necessary.
 	 */
 	if (best_path->outersortkeys)
 	{
+		Relids		outer_relids = outer_path->parent->relids;
 		Sort	   *sort = make_sort_from_pathkeys(outer_plan,
-												   best_path->outersortkeys);
+												   best_path->outersortkeys,
+												   outer_relids);
 
 		label_sort_with_costsize(root, sort, -1.0);
 		outer_plan = (Plan *) sort;
 		outerpathkeys = best_path->outersortkeys;
 	}
 	else
-		outerpathkeys = best_path->jpath.outerjoinpath->pathkeys;
+		outerpathkeys = outer_path->pathkeys;
 
 	if (best_path->innersortkeys)
 	{
+		Relids		inner_relids = inner_path->parent->relids;
 		Sort	   *sort = make_sort_from_pathkeys(inner_plan,
-												   best_path->innersortkeys);
+												   best_path->innersortkeys,
+												   inner_relids);
 
 		label_sort_with_costsize(root, sort, -1.0);
 		inner_plan = (Plan *) sort;
 		innerpathkeys = best_path->innersortkeys;
 	}
 	else
-		innerpathkeys = best_path->jpath.innerjoinpath->pathkeys;
+		innerpathkeys = inner_path->pathkeys;
 
 	/*
 	 * If specified, add a materialize node to shield the inner plan from the
 	 * need to handle mark/restore.
 	 */
 	if (best_path->materialize_inner)
 	{
 		Plan	   *matplan = (Plan *) make_material(inner_plan);
 
 		/*
@@ -5330,25 +5337,25 @@ prepare_sort_from_pathkeys(Plan *lefttree, List *pathkeys,
 
 				/*
 				 * We shouldn't be trying to sort by an equivalence class that
 				 * contains a constant, so no need to consider such cases any
 				 * further.
 				 */
 				if (em->em_is_const)
 					continue;
 
 				/*
-				 * Ignore child members unless they match the rel being
+				 * Ignore child members unless they belong to the rel being
 				 * sorted.
 				 */
 				if (em->em_is_child &&
-					!bms_equal(em->em_relids, relids))
+					!bms_is_subset(em->em_relids, relids))
 					continue;
 
 				sortexpr = em->em_expr;
 				exprvars = pull_var_clause((Node *) sortexpr,
 										   PVC_INCLUDE_AGGREGATES |
 										   PVC_INCLUDE_WINDOWFUNCS |
 										   PVC_INCLUDE_PLACEHOLDERS);
 				foreach(k, exprvars)
 				{
 					if (!tlist_member_ignore_relabel(lfirst(k), tlist))
@@ -5445,57 +5452,58 @@ find_ec_member_for_tle(EquivalenceClass *ec,
 		Expr	   *emexpr;
 
 		/*
 		 * We shouldn't be trying to sort by an equivalence class that
 		 * contains a constant, so no need to consider such cases any further.
 		 */
 		if (em->em_is_const)
 			continue;
 
 		/*
-		 * Ignore child members unless they match the rel being sorted.
+		 * Ignore child members unless they belong to the rel being sorted.
 		 */
 		if (em->em_is_child &&
-			!bms_equal(em->em_relids, relids))
+			!bms_is_subset(em->em_relids, relids))
 			continue;
 
 		/* Match if same expression (after stripping relabel) */
 		emexpr = em->em_expr;
 		while (emexpr && IsA(emexpr, RelabelType))
 			emexpr = ((RelabelType *) emexpr)->arg;
 
 		if (equal(emexpr, tlexpr))
 			return em;
 	}
 
 	return NULL;
 }
 
 /*
  * make_sort_from_pathkeys
  *	  Create sort plan to sort according to given pathkeys
  *
  *	  'lefttree' is the node which yields input tuples
  *	  'pathkeys' is the list of pathkeys by which the result is to be sorted
+ *	  'relids' is the set of relations required by prepare_sort_from_pathkeys()
  */
 static Sort *
-make_sort_from_pathkeys(Plan *lefttree, List *pathkeys)
+make_sort_from_pathkeys(Plan *lefttree, List *pathkeys, Relids relids)
 {
 	int			numsortkeys;
 	AttrNumber *sortColIdx;
 	Oid		   *sortOperators;
 	Oid		   *collations;
 	bool	   *nullsFirst;
 
 	/* Compute sort column info, and adjust lefttree as needed */
 	lefttree = prepare_sort_from_pathkeys(lefttree, pathkeys,
-										  NULL,
+										  relids,
 										  NULL,
 										  false,
 										  &numsortkeys,
 										  &sortColIdx,
 										  &sortOperators,
 										  &collations,
 										  &nullsFirst);
 
 	/* Now build the Sort node */
 	return make_sort(lefttree, numsortkeys,
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index 592214b..2e327a8 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -1873,20 +1873,71 @@ adjust_appendrel_attrs(PlannerInfo *root, Node *node, AppendRelInfo *appinfo)
 										   appinfo);
 		}
 		result = (Node *) newnode;
 	}
 	else
 		result = adjust_appendrel_attrs_mutator(node, &context);
 
 	return result;
 }
 
+/*
+ * find_appendrelinfos_by_relids
+ * 		Find AppendRelInfo structures for all relations specified by relids.
+ */
+List *
+find_appendrelinfos_by_relids(PlannerInfo *root, Relids relids)
+{
+	ListCell	*lc;
+	List		*ari_list = NIL;
+
+	foreach (lc, root->append_rel_list)
+	{
+		AppendRelInfo *ari = lfirst(lc);
+
+		if (bms_is_member(ari->child_relid, relids))
+			ari_list = lappend(ari_list, ari);
+	}
+
+	Assert(list_length(ari_list) == bms_num_members(relids));
+	return ari_list;
+}
+
+/*
+ * adjust_partitionrel_attrs
+ *	    Replace the Var nodes in given node with the corresponding Var nodes
+ *	    of the child. Given list of AppendRelInfo nodes holds the mapping
+ *	    between parent and child Var nodes.
+ *
+ * While doing so, we need to make sure to translate non-expression information
+ * in nodes like RestrictInfo.
+ */
+Node *
+adjust_partitionrel_attrs(PlannerInfo *root, Node *node,
+						  List *append_rel_infos)
+{
+	ListCell   *lc;
+
+	/*
+	 * TODO: Instead of copying and mutating the trees one child relation at a
+	 * time, we should be able to do this en-masse for all the partitions
+	 * involved.
+	 */
+	foreach (lc, append_rel_infos)
+	{
+		AppendRelInfo	*append_rel_info = lfirst(lc);
+		node = adjust_appendrel_attrs(root, node, append_rel_info);
+	}
+
+	return node;
+}
+
 static Node *
 adjust_appendrel_attrs_mutator(Node *node,
 							   adjust_appendrel_attrs_context *context)
 {
 	AppendRelInfo *appinfo = context->appinfo;
 
 	if (node == NULL)
 		return NULL;
 	if (IsA(node, Var))
 	{
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index abb7507..ba64329 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -16,21 +16,23 @@
 
 #include <math.h>
 
 #include "miscadmin.h"
 #include "nodes/nodeFuncs.h"
 #include "optimizer/clauses.h"
 #include "optimizer/cost.h"
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
 #include "optimizer/planmain.h"
+#include "optimizer/prep.h"
 #include "optimizer/restrictinfo.h"
+#include "optimizer/tlist.h"
 #include "optimizer/var.h"
 #include "parser/parsetree.h"
 #include "utils/lsyscache.h"
 #include "utils/selfuncs.h"
 
 
 typedef enum
 {
 	COSTS_EQUAL,				/* path costs are fuzzily equal */
 	COSTS_BETTER1,				/* first path is cheaper than second */
@@ -3202,10 +3204,160 @@ reparameterize_path(PlannerInfo *root, Path *path,
 														 rel,
 														 spath->subpath,
 														 spath->path.pathkeys,
 														 required_outer);
 			}
 		default:
 			break;
 	}
 	return NULL;
 }
+
+/*
+ * reparameterize_path_for_child
+ * 		Given a path parameterized by the parent of the given relation,
+ * 		translate the path to be parameterized by the given child relation.
+ *
+ * The function creates a new path of the same type as the given path, but
+ * parameterized by the given child relation. If it can not reparameterize the
+ * path as required, it returns NULL.
+ */
+Path *
+reparameterize_path_for_child(PlannerInfo *root, Path *path,
+							  RelOptInfo *child_rel)
+{
+	Path	   *new_path;
+	ParamPathInfo   *new_ppi;
+	ParamPathInfo   *old_ppi;
+	List	   *child_aris;
+
+	/*
+	 * If the path is not parameterized by parent of the given relation, it
+	 * doesn't need reparameterization.
+	 */
+	if (!path->param_info ||
+		!bms_overlap(PATH_REQ_OUTER(path), child_rel->top_parent_relids))
+	return path;
+
+	switch (nodeTag(path))
+	{
+		case T_Path:
+			new_path = makeNode(Path);
+			memcpy(new_path, path, sizeof(Path));
+			break;
+
+		case T_HashPath:
+			new_path = (Path *) makeNode(HashPath);
+			memcpy(new_path, path, sizeof(HashPath));
+			break;
+
+		case T_MergePath:
+			new_path = (Path *) makeNode(MergePath);
+			memcpy(new_path, path, sizeof(MergePath));
+			break;
+
+		case T_NestPath:
+			new_path = (Path *) makeNode(NestPath);
+			memcpy(new_path, path, sizeof(NestPath));
+			break;
+
+		case T_IndexPath:
+			new_path = (Path *) makeNode(IndexPath);
+			memcpy(new_path, path, sizeof(IndexPath));
+			break;
+
+		case T_AppendPath:
+			new_path = (Path *) makeNode(AppendPath);
+			memcpy(new_path, path, sizeof(AppendPath));
+			break;
+
+		/*
+		 * TODO:
+		 * If this method of translation is fine add more path types here.
+		 */
+
+		default:
+			/* Path type unsupported by this function. */
+			return NULL;
+	}
+
+	/*
+	 * Gather AppendRelInfos of the base partition relations in the outer child
+	 * relation. We need those for translating parent path to that of child by
+	 * substituting parent Var nodes and relids with those of children.
+	 */
+	child_aris = find_appendrelinfos_by_relids(root, child_rel->relids);
+
+	/* Adjust the parameterization information. */
+	old_ppi = new_path->param_info;
+	new_ppi = makeNode(ParamPathInfo);
+	new_ppi->ppi_req_outer = adjust_partition_relids(bms_copy(old_ppi->ppi_req_outer),
+													 child_aris);
+	new_ppi->ppi_rows = old_ppi->ppi_rows;
+	new_ppi->ppi_clauses = (List *) adjust_partitionrel_attrs(root,
+													 (Node *) old_ppi->ppi_clauses,
+													 child_aris);
+
+	/* Adjust the path target. */
+	new_path->pathtarget = copy_pathtarget(new_path->pathtarget);
+	new_path->pathtarget->exprs = (List *) adjust_partitionrel_attrs(root,
+													(Node *) new_path->pathtarget->exprs,
+													 child_aris);
+	new_path->param_info = new_ppi;
+
+	/*
+	 * Change parameterization of sub paths recursively. Also carry out any
+	 * pathtype specific adjustments.
+	 */
+	switch (nodeTag(path))
+	{
+		case T_HashPath:
+		case T_MergePath:
+		case T_NestPath:
+			{
+				JoinPath *jpath = (JoinPath *)new_path;
+
+				jpath->outerjoinpath = reparameterize_path_for_child(root,
+														 jpath->outerjoinpath,
+														 child_rel);
+				jpath->innerjoinpath = reparameterize_path_for_child(root,
+														 jpath->innerjoinpath,
+														 child_rel);
+				jpath->joinrestrictinfo = (List *) adjust_partitionrel_attrs(root,
+															(Node *) jpath->joinrestrictinfo,
+															child_aris);
+			}
+			break;
+
+		case T_AppendPath:
+			{
+				AppendPath	*apath = (AppendPath *)new_path;
+				List		*subpaths = NIL;
+				ListCell	*lc;
+
+				foreach (lc, apath->subpaths)
+					subpaths = lappend(subpaths,
+									   reparameterize_path_for_child(root,
+																	lfirst(lc),
+																	child_rel));
+				apath->subpaths = subpaths;
+			}
+
+		case T_IndexPath:
+			{
+				IndexPath *ipath = (IndexPath *)new_path;
+
+				ipath->indexquals = (List *) adjust_partitionrel_attrs(root,
+														(Node *) ipath->indexquals,
+														child_aris);
+				ipath->indexquals = (List *) adjust_partitionrel_attrs(root,
+														(Node *) ipath->indexorderbys,
+														child_aris);
+			}
+
+		default:
+			/* Nothing to do. */
+			break;
+	}
+
+	return new_path;
+}
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index 6ef45d5..6512227 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -404,20 +404,35 @@ get_relation_info(PlannerInfo *root, Oid relationObjectId, bool inhparent,
 	}
 	else
 	{
 		rel->serverid = InvalidOid;
 		rel->fdwroutine = NULL;
 	}
 
 	/* Collect info about relation's foreign keys, if relevant */
 	get_relation_foreign_keys(root, rel, relation);
 
+	/*
+	 * Lookup partition scheme for the given relation. Only parent relations
+	 * can be partitioned.
+	 */
+	if (inhparent)
+		rel->part_scheme = find_partition_scheme(root, relation);
+	else
+		rel->part_scheme = NULL;
+
+	if (rel->part_scheme)
+		rel->partexprs = build_baserel_partition_keys(root, relation,
+													  rel->relid);
+	else
+		rel->partexprs = NULL;
+
 	heap_close(relation, NoLock);
 
 	/*
 	 * Allow a plugin to editorialize on the info we obtained from the
 	 * catalogs.  Actions might include altering the assumed relation size,
 	 * removing an index, or adding a hypothetical index to the indexlist.
 	 */
 	if (get_relation_info_hook)
 		(*get_relation_info_hook) (root, relationObjectId, inhparent, rel);
 }
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index deef560..d1f6c04 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -8,29 +8,35 @@
  *
  *
  * IDENTIFICATION
  *	  src/backend/optimizer/util/relnode.c
  *
  *-------------------------------------------------------------------------
  */
 #include "postgres.h"
 
 #include "miscadmin.h"
+#include "catalog/heap.h"
+#include "catalog/partition.h"
 #include "optimizer/clauses.h"
 #include "optimizer/cost.h"
+#include "nodes/makefuncs.h"
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
 #include "optimizer/placeholder.h"
 #include "optimizer/plancat.h"
+#include "optimizer/prep.h"
 #include "optimizer/restrictinfo.h"
 #include "optimizer/tlist.h"
+#include "rewrite/rewriteManip.h"
 #include "utils/hsearch.h"
+#include "utils/rel.h"
 
 
 typedef struct JoinHashEntry
 {
 	Relids		join_relids;	/* hash key --- MUST BE FIRST */
 	RelOptInfo *join_rel;
 } JoinHashEntry;
 
 static void build_joinrel_tlist(PlannerInfo *root, RelOptInfo *joinrel,
 					RelOptInfo *input_rel);
@@ -40,20 +46,22 @@ static List *build_joinrel_restrictlist(PlannerInfo *root,
 						   RelOptInfo *inner_rel);
 static void build_joinrel_joinlist(RelOptInfo *joinrel,
 					   RelOptInfo *outer_rel,
 					   RelOptInfo *inner_rel);
 static List *subbuild_joinrel_restrictlist(RelOptInfo *joinrel,
 							  List *joininfo_list,
 							  List *new_restrictlist);
 static List *subbuild_joinrel_joinlist(RelOptInfo *joinrel,
 						  List *joininfo_list,
 						  List *new_joininfo);
+static void set_foreign_rel_properties(RelOptInfo *joinrel,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel);
 
 
 /*
  * setup_simple_rel_arrays
  *	  Prepare the arrays we use for quickly accessing base relations.
  */
 void
 setup_simple_rel_arrays(PlannerInfo *root)
 {
 	Index		rti;
@@ -130,20 +138,24 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptKind reloptkind)
 	rel->serverid = InvalidOid;
 	rel->userid = rte->checkAsUser;
 	rel->useridiscurrent = false;
 	rel->fdwroutine = NULL;
 	rel->fdw_private = NULL;
 	rel->baserestrictinfo = NIL;
 	rel->baserestrictcost.startup = 0;
 	rel->baserestrictcost.per_tuple = 0;
 	rel->joininfo = NIL;
 	rel->has_eclass_joins = false;
+	rel->part_scheme = NULL;
+	rel->partexprs = NULL;
+	rel->top_parent_relids = NULL;
+	rel->part_rels = NULL;
 
 	/* Check type of rtable entry */
 	switch (rte->rtekind)
 	{
 		case RTE_RELATION:
 			/* Table --- retrieve statistics from the system catalogs */
 			get_relation_info(root, rte->relid, rte->inh, rel);
 			break;
 		case RTE_SUBQUERY:
 		case RTE_FUNCTION:
@@ -307,20 +319,70 @@ find_join_rel(PlannerInfo *root, Relids relids)
 
 			if (bms_equal(rel->relids, relids))
 				return rel;
 		}
 	}
 
 	return NULL;
 }
 
 /*
+ * set_foreign_rel_properties
+ *		Set up foreign-join fields if outer and inner relation are foreign
+ *		tables (or joins) belonging to the same server and assigned to the same
+ *		user to check access permissions as.
+ *
+ * In addition to an exact match of userid, we allow the case where one side
+ * has zero userid (implying current user) and the other side has explicit
+ * userid that happens to equal the current user; but in that case, pushdown of
+ * the join is only valid for the current user.  The useridiscurrent field
+ * records whether we had to make such an assumption for this join or any
+ * sub-join.
+ *
+ * Otherwise these fields are left invalid, so GetForeignJoinPaths will not be
+ * called for the join relation.
+ *
+ */
+static void
+set_foreign_rel_properties(RelOptInfo *joinrel, RelOptInfo *outer_rel,
+						   RelOptInfo *inner_rel)
+{
+	if (OidIsValid(outer_rel->serverid) &&
+		inner_rel->serverid == outer_rel->serverid)
+	{
+		if (inner_rel->userid == outer_rel->userid)
+		{
+			joinrel->serverid = outer_rel->serverid;
+			joinrel->userid = outer_rel->userid;
+			joinrel->useridiscurrent = outer_rel->useridiscurrent || inner_rel->useridiscurrent;
+			joinrel->fdwroutine = outer_rel->fdwroutine;
+		}
+		else if (!OidIsValid(inner_rel->userid) &&
+				 outer_rel->userid == GetUserId())
+		{
+			joinrel->serverid = outer_rel->serverid;
+			joinrel->userid = outer_rel->userid;
+			joinrel->useridiscurrent = true;
+			joinrel->fdwroutine = outer_rel->fdwroutine;
+		}
+		else if (!OidIsValid(outer_rel->userid) &&
+				 inner_rel->userid == GetUserId())
+		{
+			joinrel->serverid = outer_rel->serverid;
+			joinrel->userid = inner_rel->userid;
+			joinrel->useridiscurrent = true;
+			joinrel->fdwroutine = outer_rel->fdwroutine;
+		}
+	}
+}
+
+/*
  * build_join_rel
  *	  Returns relation entry corresponding to the union of two given rels,
  *	  creating a new relation entry if none already exists.
  *
  * 'joinrelids' is the Relids set that uniquely identifies the join
  * 'outer_rel' and 'inner_rel' are relation nodes for the relations to be
  *		joined
  * 'sjinfo': join context info
  * 'restrictlist_ptr': result variable.  If not NULL, *restrictlist_ptr
  *		receives the list of RestrictInfo nodes that apply to this
@@ -356,21 +418,25 @@ build_join_rel(PlannerInfo *root,
 														   joinrel,
 														   outer_rel,
 														   inner_rel);
 		return joinrel;
 	}
 
 	/*
 	 * Nope, so make one.
 	 */
 	joinrel = makeNode(RelOptInfo);
+
+	Assert(!IS_OTHER_REL(outer_rel->reloptkind) &&
+		   !IS_OTHER_REL(inner_rel->reloptkind));
 	joinrel->reloptkind = RELOPT_JOINREL;
+
 	joinrel->relids = bms_copy(joinrelids);
 	joinrel->rows = 0;
 	/* cheap startup cost is interesting iff not all tuples to be retrieved */
 	joinrel->consider_startup = (root->tuple_fraction > 0);
 	joinrel->consider_param_startup = false;
 	joinrel->consider_parallel = false;
 	joinrel->reltarget = create_empty_pathtarget();
 	joinrel->pathlist = NIL;
 	joinrel->ppilist = NIL;
 	joinrel->partial_pathlist = NIL;
@@ -402,61 +468,27 @@ build_join_rel(PlannerInfo *root,
 	joinrel->serverid = InvalidOid;
 	joinrel->userid = InvalidOid;
 	joinrel->useridiscurrent = false;
 	joinrel->fdwroutine = NULL;
 	joinrel->fdw_private = NULL;
 	joinrel->baserestrictinfo = NIL;
 	joinrel->baserestrictcost.startup = 0;
 	joinrel->baserestrictcost.per_tuple = 0;
 	joinrel->joininfo = NIL;
 	joinrel->has_eclass_joins = false;
+	joinrel->part_scheme = NULL;
+	joinrel->partexprs = NULL;
+	joinrel->top_parent_relids = NULL;
+	joinrel->part_rels = NULL;
 
-	/*
-	 * Set up foreign-join fields if outer and inner relation are foreign
-	 * tables (or joins) belonging to the same server and assigned to the same
-	 * user to check access permissions as.  In addition to an exact match of
-	 * userid, we allow the case where one side has zero userid (implying
-	 * current user) and the other side has explicit userid that happens to
-	 * equal the current user; but in that case, pushdown of the join is only
-	 * valid for the current user.  The useridiscurrent field records whether
-	 * we had to make such an assumption for this join or any sub-join.
-	 *
-	 * Otherwise these fields are left invalid, so GetForeignJoinPaths will
-	 * not be called for the join relation.
-	 */
-	if (OidIsValid(outer_rel->serverid) &&
-		inner_rel->serverid == outer_rel->serverid)
-	{
-		if (inner_rel->userid == outer_rel->userid)
-		{
-			joinrel->serverid = outer_rel->serverid;
-			joinrel->userid = outer_rel->userid;
-			joinrel->useridiscurrent = outer_rel->useridiscurrent || inner_rel->useridiscurrent;
-			joinrel->fdwroutine = outer_rel->fdwroutine;
-		}
-		else if (!OidIsValid(inner_rel->userid) &&
-				 outer_rel->userid == GetUserId())
-		{
-			joinrel->serverid = outer_rel->serverid;
-			joinrel->userid = outer_rel->userid;
-			joinrel->useridiscurrent = true;
-			joinrel->fdwroutine = outer_rel->fdwroutine;
-		}
-		else if (!OidIsValid(outer_rel->userid) &&
-				 inner_rel->userid == GetUserId())
-		{
-			joinrel->serverid = outer_rel->serverid;
-			joinrel->userid = inner_rel->userid;
-			joinrel->useridiscurrent = true;
-			joinrel->fdwroutine = outer_rel->fdwroutine;
-		}
-	}
+	/* Computer information relevant to the foreign relations. */
+	set_foreign_rel_properties(joinrel, outer_rel, inner_rel);
 
 	/*
 	 * Create a new tlist containing just the vars that need to be output from
 	 * this join (ie, are needed for higher joinclauses or final output).
 	 *
 	 * NOTE: the tlist order for a join rel will depend on which pair of outer
 	 * and inner rels we first try to build it from.  But the contents should
 	 * be the same regardless.
 	 */
 	build_joinrel_tlist(root, joinrel, outer_rel);
@@ -468,20 +500,24 @@ build_join_rel(PlannerInfo *root,
 	 * sets of any PlaceHolderVars computed here to direct_lateral_relids, so
 	 * now we can finish computing that.  This is much like the computation of
 	 * the transitively-closed lateral_relids in min_join_parameterization,
 	 * except that here we *do* have to consider the added PHVs.
 	 */
 	joinrel->direct_lateral_relids =
 		bms_del_members(joinrel->direct_lateral_relids, joinrel->relids);
 	if (bms_is_empty(joinrel->direct_lateral_relids))
 		joinrel->direct_lateral_relids = NULL;
 
+	/* Store the partition information. */
+	build_joinrel_partition_info(joinrel, outer_rel, inner_rel,
+								 sjinfo->jointype);
+
 	/*
 	 * Construct restrict and join clause lists for the new joinrel. (The
 	 * caller might or might not need the restrictlist, but I need it anyway
 	 * for set_joinrel_size_estimates().)
 	 */
 	restrictlist = build_joinrel_restrictlist(root, joinrel,
 											  outer_rel, inner_rel);
 	if (restrictlist_ptr)
 		*restrictlist_ptr = restrictlist;
 	build_joinrel_joinlist(joinrel, outer_rel, inner_rel);
@@ -510,57 +546,153 @@ build_join_rel(PlannerInfo *root,
 	 * assume this doesn't matter, because we should hit all the same baserels
 	 * and joinclauses while building up to this joinrel no matter which we
 	 * take; therefore, we should make the same decision here however we get
 	 * here.
 	 */
 	if (inner_rel->consider_parallel && outer_rel->consider_parallel &&
 		is_parallel_safe(root, (Node *) restrictlist) &&
 		is_parallel_safe(root, (Node *) joinrel->reltarget->exprs))
 		joinrel->consider_parallel = true;
 
-	/*
-	 * Add the joinrel to the query's joinrel list, and store it into the
-	 * auxiliary hashtable if there is one.  NB: GEQO requires us to append
-	 * the new joinrel to the end of the list!
-	 */
-	root->join_rel_list = lappend(root->join_rel_list, joinrel);
-
-	if (root->join_rel_hash)
-	{
-		JoinHashEntry *hentry;
-		bool		found;
-
-		hentry = (JoinHashEntry *) hash_search(root->join_rel_hash,
-											   &(joinrel->relids),
-											   HASH_ENTER,
-											   &found);
-		Assert(!found);
-		hentry->join_rel = joinrel;
-	}
+	/* Add the joinrel to the query's PlannerInfo. */
+	add_join_rel(root, joinrel);
 
 	/*
 	 * Also, if dynamic-programming join search is active, add the new joinrel
 	 * to the appropriate sublist.  Note: you might think the Assert on number
 	 * of members should be for equality, but some of the level 1 rels might
 	 * have been joinrels already, so we can only assert <=.
 	 */
 	if (root->join_rel_level)
 	{
 		Assert(root->join_cur_level > 0);
 		Assert(root->join_cur_level <= bms_num_members(joinrel->relids));
 		root->join_rel_level[root->join_cur_level] =
 			lappend(root->join_rel_level[root->join_cur_level], joinrel);
 	}
 
 	return joinrel;
 }
 
+RelOptInfo *
+build_partition_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
+						 RelOptInfo *inner_rel, RelOptInfo *parent_joinrel,
+						 SpecialJoinInfo *sjinfo, List *restrictlist,
+						 List *join_aris)
+{
+	List	   *tmp_exprs;
+	RelOptInfo *joinrel = makeNode(RelOptInfo);
+
+	joinrel->reloptkind = RELOPT_OTHER_JOINREL;
+	joinrel->relids = bms_union(outer_rel->relids, inner_rel->relids);
+	joinrel->rows = 0;
+	/* cheap startup cost is interesting iff not all tuples to be retrieved */
+	joinrel->consider_startup = (root->tuple_fraction > 0);
+	joinrel->consider_param_startup = false;
+	joinrel->consider_parallel = false;
+	joinrel->reltarget = create_empty_pathtarget();
+	joinrel->pathlist = NIL;
+	joinrel->ppilist = NIL;
+	joinrel->partial_pathlist = NIL;
+	joinrel->cheapest_startup_path = NULL;
+	joinrel->cheapest_total_path = NULL;
+	joinrel->cheapest_unique_path = NULL;
+	joinrel->cheapest_parameterized_paths = NIL;
+	joinrel->direct_lateral_relids = NULL;
+	joinrel->lateral_relids = NULL;
+	joinrel->relid = 0;			/* indicates not a baserel */
+	joinrel->rtekind = RTE_JOIN;
+	joinrel->min_attr = 0;
+	joinrel->max_attr = 0;
+	joinrel->attr_needed = NULL;
+	joinrel->attr_widths = NULL;
+	joinrel->lateral_vars = NIL;
+	joinrel->lateral_referencers = NULL;
+	joinrel->indexlist = NIL;
+	joinrel->pages = 0;
+	joinrel->tuples = 0;
+	joinrel->allvisfrac = 0;
+	joinrel->subroot = NULL;
+	joinrel->subplan_params = NIL;
+	joinrel->serverid = InvalidOid;
+	joinrel->userid = InvalidOid;
+	joinrel->useridiscurrent = false;
+	joinrel->fdwroutine = NULL;
+	joinrel->fdw_private = NULL;
+	joinrel->baserestrictinfo = NIL;
+	joinrel->baserestrictcost.startup = 0;
+	joinrel->baserestrictcost.per_tuple = 0;
+	joinrel->joininfo = NIL;
+	joinrel->has_eclass_joins = false;
+	joinrel->part_scheme = NULL;
+	joinrel->partexprs = NULL;
+	joinrel->top_parent_relids = NULL;
+	joinrel->part_rels = NULL;
+
+
+	/* Only joins between other relations land here. */
+	Assert(IS_OTHER_REL(outer_rel->reloptkind) &&
+		   IS_OTHER_REL(inner_rel->reloptkind));
+
+	joinrel->top_parent_relids = bms_union(outer_rel->top_parent_relids,
+										   inner_rel->top_parent_relids);
+
+	/* Computer information relevant to foreign relations. */
+	set_foreign_rel_properties(joinrel, outer_rel, inner_rel);
+
+	/*
+	 * Produce partition-wise joinrel's targetlist by translating the parent
+	 * joinrel's targetlist. This will also include the required placeholder
+	 * Vars.
+	 */
+	joinrel->reltarget = copy_pathtarget(parent_joinrel->reltarget);
+	tmp_exprs = joinrel->reltarget->exprs;
+	joinrel->reltarget->exprs = (List *) adjust_partitionrel_attrs(root, (Node *) tmp_exprs,
+															join_aris);
+
+	/*
+	 * Lateral relids directly referred in this relation will be same as that
+	 * of the parent relation.
+	 */
+	joinrel->direct_lateral_relids = (Relids) bms_copy(parent_joinrel->direct_lateral_relids);
+	joinrel->lateral_relids = (Relids) bms_copy(parent_joinrel->lateral_relids);
+
+	joinrel->joininfo = (List *) adjust_partitionrel_attrs(root,
+											 (Node *) parent_joinrel->joininfo,
+													join_aris);
+
+	/*
+	 * If the parent joinrel has pending equivalence classes, so does the
+	 * child.
+	 */
+	joinrel->has_eclass_joins = parent_joinrel->has_eclass_joins;
+
+	/* Is the join between partitions itself partitioned? */
+	build_joinrel_partition_info(joinrel, outer_rel, inner_rel, sjinfo->jointype);
+
+	/*
+	 * Set estimates of the joinrel's size.
+	 */
+	set_joinrel_size_estimates(root, joinrel, outer_rel, inner_rel, sjinfo,
+							   restrictlist);
+
+	/* Child joinrel is parallel safe if parent is parallel safe. */
+	joinrel->consider_parallel = parent_joinrel->consider_parallel;
+
+	Assert(!find_join_rel(root, joinrel->relids));
+
+	/* Add the relation to the PlannerInfo. */
+	add_join_rel(root, joinrel);
+
+	return joinrel;
+}
+
 /*
  * min_join_parameterization
  *
  * Determine the minimum possible parameterization of a joinrel, that is, the
  * set of other rels it contains LATERAL references to.  We save this value in
  * the join's RelOptInfo.  This function is split out of build_join_rel()
  * because join_is_legal() needs the value to check a prospective join.
  */
 Relids
 min_join_parameterization(PlannerInfo *root,
@@ -1313,10 +1445,35 @@ get_appendrel_parampathinfo(RelOptInfo *appendrel, Relids required_outer)
 
 	/* Else build the ParamPathInfo */
 	ppi = makeNode(ParamPathInfo);
 	ppi->ppi_req_outer = required_outer;
 	ppi->ppi_rows = 0;
 	ppi->ppi_clauses = NIL;
 	appendrel->ppilist = lappend(appendrel->ppilist, ppi);
 
 	return ppi;
 }
+
+/*
+ * Adds given join relation to the joinrel list and also to the hashtable if
+ * there is one.
+ */
+void
+add_join_rel(PlannerInfo *root, RelOptInfo *joinrel)
+{
+	/* GEQO requires us to append the new joinrel to the end of the list! */
+	root->join_rel_list = lappend(root->join_rel_list, joinrel);
+
+	/* store it into the auxiliary hashtable if there is one. */
+	if (root->join_rel_hash)
+	{
+		JoinHashEntry *hentry;
+		bool		found;
+
+		hentry = (JoinHashEntry *) hash_search(root->join_rel_hash,
+											   &(joinrel->relids),
+											   HASH_ENTER,
+											   &found);
+		Assert(!found);
+		hentry->join_rel = joinrel;
+	}
+}
diff --git a/src/backend/utils/adt/selfuncs.c b/src/backend/utils/adt/selfuncs.c
index 56943f2..16b2eac 100644
--- a/src/backend/utils/adt/selfuncs.c
+++ b/src/backend/utils/adt/selfuncs.c
@@ -3405,21 +3405,23 @@ estimate_num_groups(PlannerInfo *root, List *groupExprs, double input_rows,
 			else
 			{
 				/* not time to process varinfo2 yet */
 				newvarinfos = lcons(varinfo2, newvarinfos);
 			}
 		}
 
 		/*
 		 * Sanity check --- don't divide by zero if empty relation.
 		 */
-		Assert(rel->reloptkind == RELOPT_BASEREL);
+		Assert(rel->reloptkind == RELOPT_BASEREL ||
+			   rel->reloptkind == RELOPT_OTHER_MEMBER_REL);
+
 		if (rel->tuples > 0)
 		{
 			/*
 			 * Clamp to size of rel, or size of rel / 10 if multiple Vars. The
 			 * fudge factor is because the Vars are probably correlated but we
 			 * don't know by how much.  We should never clamp to less than the
 			 * largest ndistinct value for any of the Vars, though, since
 			 * there will surely be at least that many groups.
 			 */
 			double		clamp = rel->tuples;
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index c5178f7..3412eae 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -870,20 +870,29 @@ static struct config_bool ConfigureNamesBool[] =
 	},
 	{
 		{"enable_hashjoin", PGC_USERSET, QUERY_TUNING_METHOD,
 			gettext_noop("Enables the planner's use of hash join plans."),
 			NULL
 		},
 		&enable_hashjoin,
 		true,
 		NULL, NULL, NULL
 	},
+	{
+		{"enable_partition_wise_join", PGC_USERSET, QUERY_TUNING_METHOD,
+			gettext_noop("Enables partition-wise join."),
+			NULL
+		},
+		&enable_partition_wise_join,
+		true,
+		NULL, NULL, NULL
+	},
 
 	{
 		{"geqo", PGC_USERSET, QUERY_TUNING_GEQO,
 			gettext_noop("Enables genetic query optimization."),
 			gettext_noop("This algorithm attempts to do planning without "
 						 "exhaustive searching.")
 		},
 		&enable_geqo,
 		true,
 		NULL, NULL, NULL
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 14fd29e..9d552b3 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -9,37 +9,39 @@
  * src/include/utils/partition.h
  *
  *-------------------------------------------------------------------------
  */
 #ifndef PARTITION_H
 #define PARTITION_H
 
 #include "fmgr.h"
 #include "executor/tuptable.h"
 #include "nodes/execnodes.h"
+#include "nodes/relation.h"
 #include "parser/parse_node.h"
 #include "utils/relcache.h"
 
 typedef struct PartitionKeyData *PartitionKey;
 
 /*
  * Information about partitions of a partitioned table.
  */
 typedef struct PartitionInfoData *PartitionInfo;
 typedef struct PartitionDescData
 {
 	int		nparts;			/* Number of partitions */
 	PartitionInfo *parts;	/* Array of PartitionInfoData pointers */
 } PartitionDescData;
 
 typedef struct PartitionDescData *PartitionDesc;
 typedef struct PartitionTreeNodeData *PartitionTreeNode;
+typedef struct PartitionSchemeData	 *PartitionScheme;
 
 /* relcache support for partition key information */
 extern void RelationBuildPartitionKey(Relation relation);
 
 /* Partition key inquiry functions */
 extern int get_partition_key_strategy(PartitionKey key);
 extern int get_partition_key_natts(PartitionKey key);
 extern List *get_partition_key_exprs(PartitionKey key);
 
 /* Partition key inquiry functions - for a given column */
@@ -62,11 +64,21 @@ extern List *get_check_qual_from_partbound(Relation rel, Relation parent,
 extern List *RelationGetPartitionCheckQual(Relation rel);
 
 /* For tuple routing */
 extern PartitionTreeNode RelationGetPartitionTreeNode(Relation rel);
 extern List *get_leaf_partition_oids_v2(PartitionTreeNode ptnode);
 
 extern int get_partition_for_tuple(PartitionTreeNode ptnode,
 					TupleTableSlot *slot,
 					EState *estate,
 					Oid *failed_at);
+extern List **build_baserel_partition_keys(PlannerInfo *root,
+							Relation relation, Index varno);
+extern PartitionScheme find_partition_scheme(PlannerInfo *root, Relation rel);
+extern int PartitionSchemeGetNumParts(PartitionScheme part_scheme);
+extern int PartitionSchemeGetNumKeys(PartitionScheme part_scheme);
+extern Oid *PartitionDescGetPartOids(PartitionDesc part_desc);
+extern void build_joinrel_partition_info(RelOptInfo *joinrel,
+						  RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+						  JoinType jointype);
+
 #endif   /* PARTITION_H */
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 2709cc7..29b419a 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -256,20 +256,23 @@ typedef struct PlannerInfo
 
 	List	   *query_pathkeys; /* desired pathkeys for query_planner() */
 
 	List	   *group_pathkeys; /* groupClause pathkeys, if any */
 	List	   *window_pathkeys;	/* pathkeys of bottom window, if any */
 	List	   *distinct_pathkeys;		/* distinctClause pathkeys, if any */
 	List	   *sort_pathkeys;	/* sortClause pathkeys, if any */
 
 	List	   *initial_rels;	/* RelOptInfos we are now trying to join */
 
+	List	   *part_schemes;	/* Canonicalised partition schemes
+								 * used in the query. */
+
 	/* Use fetch_upper_rel() to get any particular upper rel */
 	List	   *upper_rels[UPPERREL_FINAL + 1]; /* upper-rel RelOptInfos */
 
 	/* Result tlists chosen by grouping_planner for upper-stage processing */
 	struct PathTarget *upper_targets[UPPERREL_FINAL + 1];
 
 	/*
 	 * grouping_planner passes back its final processed targetlist here, for
 	 * use in relabeling the topmost tlist of the finished Plan.
 	 */
@@ -345,20 +348,26 @@ typedef struct PlannerInfo
  * is present in the query join tree but the members are not.  The member
  * RTEs and otherrels are used to plan the scans of the individual tables or
  * subqueries of the append set; then the parent baserel is given Append
  * and/or MergeAppend paths comprising the best paths for the individual
  * member rels.  (See comments for AppendRelInfo for more information.)
  *
  * At one time we also made otherrels to represent join RTEs, for use in
  * handling join alias Vars.  Currently this is not needed because all join
  * alias Vars are expanded to non-aliased form during preprocess_expression.
  *
+ * We also have relations representing pair-wise joins between partitions of
+ * partitioned tables. These relations are not added to join_rel_level lists
+ * as they are not joined directly by the dynamic programming algorithm.
+ * Adding these two join_rel_level list also means that top level list has more
+ * than one join relation, which is symantically incorrect.
+ *
  * There is also a RelOptKind for "upper" relations, which are RelOptInfos
  * that describe post-scan/join processing steps, such as aggregation.
  * Many of the fields in these RelOptInfos are meaningless, but their Path
  * fields always hold Paths showing ways to do that processing step.
  *
  * Lastly, there is a RelOptKind for "dead" relations, which are base rels
  * that we have proven we don't need to join after all.
  *
  * Parts of this data structure are specific to various scan and join
  * mechanisms.  It didn't seem worth creating new node types for them.
@@ -464,24 +473,31 @@ typedef struct PlannerInfo
  * We store baserestrictcost in the RelOptInfo (for base relations) because
  * we know we will need it at least once (to price the sequential scan)
  * and may need it multiple times to price index scans.
  *----------
  */
 typedef enum RelOptKind
 {
 	RELOPT_BASEREL,
 	RELOPT_JOINREL,
 	RELOPT_OTHER_MEMBER_REL,
+	RELOPT_OTHER_JOINREL,
 	RELOPT_UPPER_REL,
 	RELOPT_DEADREL
 } RelOptKind;
 
+#define IS_OTHER_REL(reloptkind) \
+	((reloptkind) == RELOPT_OTHER_MEMBER_REL || \
+	 (reloptkind) == RELOPT_OTHER_JOINREL)
+
+typedef struct PartitionSchemeData *PartitionScheme;
+
 typedef struct RelOptInfo
 {
 	NodeTag		type;
 
 	RelOptKind	reloptkind;
 
 	/* all relations included in this RelOptInfo */
 	Relids		relids;			/* set of base relids (rangetable indexes) */
 
 	/* size estimates generated by planner */
@@ -535,20 +551,37 @@ typedef struct RelOptInfo
 	struct FdwRoutine *fdwroutine;
 	void	   *fdw_private;
 
 	/* used by various scans and joins: */
 	List	   *baserestrictinfo;		/* RestrictInfo structures (if base
 										 * rel) */
 	QualCost	baserestrictcost;		/* cost of evaluating the above */
 	List	   *joininfo;		/* RestrictInfo structures for join clauses
 								 * involving this rel */
 	bool		has_eclass_joins;		/* T means joininfo is incomplete */
+
+	/* For partitioned relations, joins or base relations. NULL otherwise. */
+	PartitionScheme	    part_scheme;	/* Partitioning scheme. */
+	struct RelOptInfo **part_rels;		/* Array of RelOptInfos of partitions,
+										 * stored in the same order as bounds
+										 * or lists in PartitionScheme.
+										 */
+	List			  **partexprs;		/* Array of list of partition key
+										 * expressions. For base relations
+										 * these are one element lists. For
+										 * join there can be as many elements
+										 * as there are number of joining
+										 * relations.
+										 */
+
+	/* Set only for "other" base or join relations. */
+	Relids		top_parent_relids;
 } RelOptInfo;
 
 /*
  * IndexOptInfo
  *		Per-index information for planning/optimization
  *
  *		indexkeys[], indexcollations[], opfamily[], and opcintype[]
  *		each have ncolumns entries.
  *
  *		sortopfamily[], reverse_sort[], and nulls_first[] likewise have
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 2a4df2f..1069726 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -59,20 +59,21 @@ extern bool enable_seqscan;
 extern bool enable_indexscan;
 extern bool enable_indexonlyscan;
 extern bool enable_bitmapscan;
 extern bool enable_tidscan;
 extern bool enable_sort;
 extern bool enable_hashagg;
 extern bool enable_nestloop;
 extern bool enable_material;
 extern bool enable_mergejoin;
 extern bool enable_hashjoin;
+extern bool enable_partition_wise_join;
 extern int	constraint_exclusion;
 
 extern double clamp_row_est(double nrows);
 extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
 					double index_pages, PlannerInfo *root);
 extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 			 ParamPathInfo *param_info);
 extern void cost_samplescan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 				ParamPathInfo *param_info);
 extern void cost_index(IndexPath *path, PlannerInfo *root,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 71d9154..2ab31d3 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -222,20 +222,22 @@ extern ModifyTablePath *create_modifytable_path(PlannerInfo *root,
 						List *rowMarks, OnConflictExpr *onconflict,
 						int epqParam);
 extern LimitPath *create_limit_path(PlannerInfo *root, RelOptInfo *rel,
 				  Path *subpath,
 				  Node *limitOffset, Node *limitCount,
 				  int64 offset_est, int64 count_est);
 
 extern Path *reparameterize_path(PlannerInfo *root, Path *path,
 					Relids required_outer,
 					double loop_count);
+extern Path *reparameterize_path_for_child(PlannerInfo *root, Path *path,
+					RelOptInfo *child_rel);
 
 /*
  * prototypes for relnode.c
  */
 extern void setup_simple_rel_arrays(PlannerInfo *root);
 extern RelOptInfo *build_simple_rel(PlannerInfo *root, int relid,
 				 RelOptKind reloptkind);
 extern RelOptInfo *find_base_rel(PlannerInfo *root, int relid);
 extern RelOptInfo *find_join_rel(PlannerInfo *root, Relids relids);
 extern RelOptInfo *build_join_rel(PlannerInfo *root,
@@ -260,12 +262,17 @@ extern ParamPathInfo *get_baserel_parampathinfo(PlannerInfo *root,
 						  Relids required_outer);
 extern ParamPathInfo *get_joinrel_parampathinfo(PlannerInfo *root,
 						  RelOptInfo *joinrel,
 						  Path *outer_path,
 						  Path *inner_path,
 						  SpecialJoinInfo *sjinfo,
 						  Relids required_outer,
 						  List **restrict_clauses);
 extern ParamPathInfo *get_appendrel_parampathinfo(RelOptInfo *appendrel,
 							Relids required_outer);
+extern RelOptInfo *build_partition_join_rel(PlannerInfo *root,
+						 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+						 RelOptInfo *parent_joinrel, SpecialJoinInfo *sjinfo,
+						 List *restrictlist, List *join_aris);
+extern void add_join_rel(PlannerInfo *root, RelOptInfo *joinrel);
 
 #endif   /* PATHNODE_H */
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 44abe83..a799428 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -212,11 +212,13 @@ extern List *make_inner_pathkeys_for_merge(PlannerInfo *root,
 							  List *mergeclauses,
 							  List *outer_pathkeys);
 extern List *truncate_useless_pathkeys(PlannerInfo *root,
 						  RelOptInfo *rel,
 						  List *pathkeys);
 extern bool has_useful_pathkeys(PlannerInfo *root, RelOptInfo *rel);
 extern PathKey *make_canonical_pathkey(PlannerInfo *root,
 					   EquivalenceClass *eclass, Oid opfamily,
 					   int strategy, bool nulls_first);
 
+extern Relids adjust_partition_relids(Relids relids, List *append_rel_infos);
+
 #endif   /* PATHS_H */
diff --git a/src/include/optimizer/prep.h b/src/include/optimizer/prep.h
index fb35b68..7b149c3 100644
--- a/src/include/optimizer/prep.h
+++ b/src/include/optimizer/prep.h
@@ -21,20 +21,23 @@
 /*
  * prototypes for prepjointree.c
  */
 extern void pull_up_sublinks(PlannerInfo *root);
 extern void inline_set_returning_functions(PlannerInfo *root);
 extern void pull_up_subqueries(PlannerInfo *root);
 extern void flatten_simple_union_all(PlannerInfo *root);
 extern void reduce_outer_joins(PlannerInfo *root);
 extern Relids get_relids_in_jointree(Node *jtnode, bool include_joins);
 extern Relids get_relids_for_join(PlannerInfo *root, int joinrelid);
+extern Node *adjust_partitionrel_attrs(PlannerInfo *root, Node *node,
+									   List *append_rel_infos);
+extern List *find_appendrelinfos_by_relids(PlannerInfo *root, Relids relids);
 
 /*
  * prototypes for prepqual.c
  */
 extern Node *negate_clause(Node *node);
 extern Expr *canonicalize_qual(Expr *qual);
 
 /*
  * prototypes for prepsecurity.c
  */
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
new file mode 100644
index 0000000..a90be5a
--- /dev/null
+++ b/src/test/regress/expected/partition_join.out
@@ -0,0 +1,6884 @@
+--
+-- PARTITION_JOIN
+-- Test partition-wise join between partitioned tables
+--
+--
+-- partitioned by a single column
+--
+CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES START (0) END (250);
+CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES START (500) END (600);
+CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES START (250) END (500);
+INSERT INTO prt1 SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1 AS SELECT * FROM prt1;
+CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES START (0) END (250);
+CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES START (250) END (500);
+CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES START (500) END (600);
+INSERT INTO prt2 SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+CREATE TABLE uprt2 AS SELECT * FROM prt2;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on public.prt2_p1 t2
+                     Output: t2.b, t2.c
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+               Hash Cond: (t2_1.b = t1_2.a)
+               ->  Seq Scan on public.prt2_p2 t2_1
+                     Output: t2_1.b, t2_1.c
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+               Hash Cond: (t2_2.b = t1_1.a)
+               ->  Seq Scan on public.prt2_p3 t2_2
+                     Output: t2_2.b, t2_2.c
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0150 | 150 | 0150
+ 300 | 0300 | 300 | 0300
+ 450 | 0450 | 450 | 0450
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on public.prt2_p1 t2
+                     Output: t2.b, t2.c
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+               Hash Cond: (t2_1.b = t1_2.a)
+               ->  Seq Scan on public.prt2_p2 t2_1
+                     Output: t2_1.b, t2_1.c
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+               Hash Cond: (t2_2.b = t1_1.a)
+               ->  Seq Scan on public.prt2_p3 t2_2
+                     Output: t2_2.b, t2_2.c
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+  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 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: (t1.a = t2.b)
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t2.b, t2.c
+                           ->  Seq Scan on public.prt2_p1 t2
+                                 Output: t2.b, t2.c
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Hash Cond: (t1_2.a = t2_1.b)
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c
+                           ->  Seq Scan on public.prt2_p2 t2_1
+                                 Output: t2_1.b, t2_1.c
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Hash Cond: (t1_1.a = t2_2.b)
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c
+                           ->  Seq Scan on public.prt2_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+                                 Filter: ((t2_2.b % 25) = 0)
+(36 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+  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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+   Sort Key: prt1_p1.a, prt2_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+               Hash Cond: (prt1_p1.a = prt2_p1.b)
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c
+                     Filter: ((prt1_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p1.b, prt2_p1.c
+                     ->  Seq Scan on public.prt2_p1
+                           Output: prt2_p1.b, prt2_p1.c
+                           Filter: ((prt2_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt1_p2.a = prt2_p2.b)
+               ->  Seq Scan on public.prt1_p2
+                     Output: prt1_p2.a, prt1_p2.c
+                     Filter: ((prt1_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p2.b, prt2_p2.c
+                     ->  Seq Scan on public.prt2_p2
+                           Output: prt2_p2.b, prt2_p2.c
+                           Filter: ((prt2_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c
+               Hash Cond: (prt1_p3.a = prt2_p3.b)
+               ->  Seq Scan on public.prt1_p3
+                     Output: prt1_p3.a, prt1_p3.c
+                     Filter: ((prt1_p3.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p3.b, prt2_p3.c
+                     ->  Seq Scan on public.prt2_p3
+                           Output: prt2_p3.b, prt2_p3.c
+                           Filter: ((prt2_p3.b % 25) = 0)
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 t2 WHERE t2.b % 25 = 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)
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+   Sort Key: prt1_p1.a, prt2_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+               Hash Cond: (prt1_p1.a = prt2_p1.b)
+               Filter: (((50) = prt1_p1.b) OR ((75) = prt2_p1.b))
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c, prt1_p1.b, 50
+                     Filter: ((prt1_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p1.b, prt2_p1.c, (75)
+                     ->  Seq Scan on public.prt2_p1
+                           Output: prt2_p1.b, prt2_p1.c, 75
+                           Filter: ((prt2_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt1_p2.a = prt2_p2.b)
+               Filter: (((50) = prt1_p2.b) OR ((75) = prt2_p2.b))
+               ->  Seq Scan on public.prt1_p2
+                     Output: prt1_p2.a, prt1_p2.c, prt1_p2.b, 50
+                     Filter: ((prt1_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p2.b, prt2_p2.c, (75)
+                     ->  Seq Scan on public.prt2_p2
+                           Output: prt2_p2.b, prt2_p2.c, 75
+                           Filter: ((prt2_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c
+               Hash Cond: (prt1_p3.a = prt2_p3.b)
+               Filter: (((50) = prt1_p3.b) OR ((75) = prt2_p3.b))
+               ->  Seq Scan on public.prt1_p3
+                     Output: prt1_p3.a, prt1_p3.c, prt1_p3.b, 50
+                     Filter: ((prt1_p3.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p3.b, prt2_p3.c, (75)
+                     ->  Seq Scan on public.prt2_p3
+                           Output: prt2_p3.b, prt2_p3.c, 75
+                           Filter: ((prt2_p3.b % 25) = 0)
+(40 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+ a  |  c   | b  |  c   
+----+------+----+------
+ 50 | 0050 |    | 
+    |      | 75 | 0075
+(2 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+ a  |  c   | b  |  c   
+----+------+----+------
+ 50 | 0050 |    | 
+    |      | 75 | 0075
+(2 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, (25), prt2_p1.b, prt2_p1.c, (50)
+   Sort Key: prt1_p1.a, prt2_p1.b
+   ->  Result
+         Output: prt1_p1.a, prt1_p1.c, (25), prt2_p1.b, prt2_p1.c, (50)
+         ->  Append
+               ->  Hash Full Join
+                     Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, (25), (50)
+                     Hash Cond: (prt1_p1.a = prt2_p1.b)
+                     ->  Seq Scan on public.prt1_p1
+                           Output: prt1_p1.a, prt1_p1.c, 25
+                           Filter: ((prt1_p1.a % 25) = 0)
+                     ->  Hash
+                           Output: prt2_p1.b, prt2_p1.c, (50)
+                           ->  Seq Scan on public.prt2_p1
+                                 Output: prt2_p1.b, prt2_p1.c, 50
+                                 Filter: ((prt2_p1.b % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c, (25), (50)
+                     Hash Cond: (prt1_p2.a = prt2_p2.b)
+                     ->  Seq Scan on public.prt1_p2
+                           Output: prt1_p2.a, prt1_p2.c, 25
+                           Filter: ((prt1_p2.a % 25) = 0)
+                     ->  Hash
+                           Output: prt2_p2.b, prt2_p2.c, (50)
+                           ->  Seq Scan on public.prt2_p2
+                                 Output: prt2_p2.b, prt2_p2.c, 50
+                                 Filter: ((prt2_p2.b % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c, (25), (50)
+                     Hash Cond: (prt1_p3.a = prt2_p3.b)
+                     ->  Seq Scan on public.prt1_p3
+                           Output: prt1_p3.a, prt1_p3.c, 25
+                           Filter: ((prt1_p3.a % 25) = 0)
+                     ->  Hash
+                           Output: prt2_p3.b, prt2_p3.c, (50)
+                           ->  Seq Scan on public.prt2_p3
+                                 Output: prt2_p3.b, prt2_p3.c, 50
+                                 Filter: ((prt2_p3.b % 25) = 0)
+(39 rows)
+
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  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)
+
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM uprt2 WHERE uprt2.b % 25 = 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)
+
+-- Join with pruned partitions from joining relations
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on public.prt2_p2 t2
+                     Output: t2.b, t2.c
+                     Filter: (t2.b > 250)
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.prt1_p2 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a < 450) AND ((t1.a % 25) = 0))
+(15 rows)
+
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+(1 row)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a < 450 AND t2.b > 250 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+(1 row)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                    QUERY PLAN                                    
+----------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, b, c
+   Sort Key: prt1_p1.a, b
+   ->  Append
+         ->  Nested Loop Left Join
+               Output: prt1_p1.a, prt1_p1.c, b, c
+               Join Filter: (prt1_p1.a = b)
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c
+                     Filter: ((prt1_p1.a < 450) AND ((prt1_p1.a % 25) = 0))
+               ->  Result
+                     Output: b, c
+                     One-Time Filter: false
+         ->  Hash Right Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt2_p2.b = prt1_p2.a)
+               ->  Seq Scan on public.prt2_p2
+                     Output: prt2_p2.b, prt2_p2.c
+                     Filter: (prt2_p2.b > 250)
+               ->  Hash
+                     Output: prt1_p2.a, prt1_p2.c
+                     ->  Seq Scan on public.prt1_p2
+                           Output: prt1_p2.a, prt1_p2.c
+                           Filter: ((prt1_p2.a < 450) AND ((prt1_p2.a % 25) = 0))
+(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.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.a % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 5) and (b 4) is considered for partition-wise join.
+                                       QUERY PLAN                                       
+----------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+   Sort Key: prt1_p2.a, prt2_p2.b
+   ->  Result
+         Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: prt2_p2.b, prt2_p2.c, prt1_p2.a, prt1_p2.c
+                     Hash Cond: (prt1_p2.a = prt2_p2.b)
+                     ->  Seq Scan on public.prt1_p2
+                           Output: prt1_p2.a, prt1_p2.c
+                           Filter: (prt1_p2.a < 450)
+                     ->  Hash
+                           Output: prt2_p2.b, prt2_p2.c
+                           ->  Seq Scan on public.prt2_p2
+                                 Output: prt2_p2.b, prt2_p2.c
+                                 Filter: ((prt2_p2.b > 250) AND ((prt2_p2.a % 25) = 0))
+               ->  Nested Loop Left Join
+                     Output: prt2_p3.b, prt2_p3.c, a, c
+                     Join Filter: (a = prt2_p3.b)
+                     ->  Seq Scan on public.prt2_p3
+                           Output: prt2_p3.b, prt2_p3.c
+                           Filter: ((prt2_p3.b > 250) AND ((prt2_p3.a % 25) = 0))
+                     ->  Result
+                           Output: a, c
+                           One-Time Filter: false
+(26 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 5) and (b 4) is considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+     |      | 375 | 0375
+     |      | 450 | 0450
+     |      | 525 | 0525
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+     |      | 375 | 0375
+     |      | 450 | 0450
+     |      | 525 | 0525
+(4 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                    QUERY PLAN                                    
+----------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, b, c
+   Sort Key: prt1_p1.a, b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_p1.a, prt1_p1.c, b, c
+               Hash Cond: (prt1_p1.a = b)
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c
+                     Filter: ((prt1_p1.a < 450) AND ((prt1_p1.a % 25) = 0))
+               ->  Hash
+                     Output: b, c
+                     ->  Result
+                           Output: b, c
+                           One-Time Filter: false
+         ->  Hash Full Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt1_p2.a = prt2_p2.b)
+               ->  Seq Scan on public.prt1_p2
+                     Output: prt1_p2.a, prt1_p2.c
+                     Filter: ((prt1_p2.a < 450) AND ((prt1_p2.a % 25) = 0))
+               ->  Hash
+                     Output: prt2_p2.b, prt2_p2.c
+                     ->  Seq Scan on public.prt2_p2
+                           Output: prt2_p2.b, prt2_p2.c
+                           Filter: ((prt2_p2.b > 250) AND ((prt2_p2.b % 25) = 0))
+         ->  Hash Full Join
+               Output: a, c, prt2_p3.b, prt2_p3.c
+               Hash Cond: (prt2_p3.b = a)
+               ->  Seq Scan on public.prt2_p3
+                     Output: prt2_p3.b, prt2_p3.c
+                     Filter: ((prt2_p3.b > 250) AND ((prt2_p3.b % 25) = 0))
+               ->  Hash
+                     Output: a, c
+                     ->  Result
+                           Output: a, c
+                           One-Time Filter: false
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b 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)
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 1) and (b 3) is considered for partition-wise join.
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.a = t1_3.b)
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_3.b
+                     ->  Seq Scan on public.prt2_p1 t1_3
+                           Output: t1_3.b
+                           Filter: ((t1_3.b % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.a = t1_4.b)
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_4.b
+                     ->  Seq Scan on public.prt2_p2 t1_4
+                           Output: t1_4.b
+                           Filter: ((t1_4.b % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.a = t1_5.b)
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_5.b
+                     ->  Seq Scan on public.prt2_p3 t1_5
+                           Output: t1_5.b
+                           Filter: ((t1_5.b % 25) = 0)
+(37 rows)
+
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 1) and (b 3) is considered for partition-wise join.
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+-- lateral reference
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4 5) is considered for partition-wise join.
+                                             QUERY PLAN                                              
+-----------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   Sort Key: t1.a
+   ->  Result
+         Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+         ->  Append
+               ->  Nested Loop Left Join
+                     Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a))
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.b, t1.c
+                           Filter: ((t1.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+                           Hash Cond: (t3.b = t2.a)
+                           ->  Seq Scan on public.prt2_p1 t3
+                                 Output: t3.a, t3.b
+                           ->  Hash
+                                 Output: t2.a
+                                 ->  Seq Scan on public.prt1_p1 t2
+                                       Output: t2.a
+                                       Filter: (t1.a = t2.a)
+               ->  Nested Loop Left Join
+                     Output: t1_2.a, t1_2.b, t1_2.c, t2_2.a, t3_1.a, (LEAST(t1_2.a, t2_2.a, t3_1.a))
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.b, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2_2.a, t3_1.a, LEAST(t1_2.a, t2_2.a, t3_1.a)
+                           Hash Cond: (t3_1.b = t2_2.a)
+                           ->  Seq Scan on public.prt2_p2 t3_1
+                                 Output: t3_1.a, t3_1.b
+                           ->  Hash
+                                 Output: t2_2.a
+                                 ->  Seq Scan on public.prt1_p2 t2_2
+                                       Output: t2_2.a
+                                       Filter: (t1_2.a = t2_2.a)
+               ->  Nested Loop Left Join
+                     Output: t1_1.a, t1_1.b, t1_1.c, t2_1.a, t3_2.a, (LEAST(t1_1.a, t2_1.a, t3_2.a))
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.b, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2_1.a, t3_2.a, LEAST(t1_1.a, t2_1.a, t3_2.a)
+                           Hash Cond: (t3_2.b = t2_1.a)
+                           ->  Seq Scan on public.prt2_p3 t3_2
+                                 Output: t3_2.a, t3_2.b
+                           ->  Hash
+                                 Output: t2_1.a
+                                 ->  Seq Scan on public.prt1_p3 t2_1
+                                       Output: t2_1.a
+                                       Filter: (t1_1.a = t2_1.a)
+(51 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4 5) is considered for partition-wise join.
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   Sort Key: t1.a
+   ->  Nested Loop Left Join
+         Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+         ->  Append
+               ->  Seq Scan on public.prt1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Seq Scan on public.prt1_p1 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Seq Scan on public.prt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Seq Scan on public.prt1_p2 t1_3
+                     Output: t1_3.a, t1_3.b, t1_3.c
+                     Filter: ((t1_3.a % 25) = 0)
+         ->  Append
+               ->  Hash Join
+                     Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+                     Hash Cond: (t3.b = t2.a)
+                     ->  Seq Scan on public.prt2_p1 t3
+                           Output: t3.a, t3.b
+                     ->  Hash
+                           Output: t2.a
+                           ->  Seq Scan on public.prt1_p1 t2
+                                 Output: t2.a
+                                 Filter: (t1.b = t2.a)
+               ->  Hash Join
+                     Output: t2_2.a, t3_1.a, LEAST(t1.a, t2_2.a, t3_1.a)
+                     Hash Cond: (t3_1.b = t2_2.a)
+                     ->  Seq Scan on public.prt2_p2 t3_1
+                           Output: t3_1.a, t3_1.b
+                     ->  Hash
+                           Output: t2_2.a
+                           ->  Seq Scan on public.prt1_p2 t2_2
+                                 Output: t2_2.a
+                                 Filter: (t1.b = t2_2.a)
+               ->  Hash Join
+                     Output: t2_1.a, t3_2.a, LEAST(t1.a, t2_1.a, t3_2.a)
+                     Hash Cond: (t3_2.b = t2_1.a)
+                     ->  Seq Scan on public.prt2_p3 t3_2
+                           Output: t3_2.a, t3_2.b
+                     ->  Hash
+                           Output: t2_1.a
+                           ->  Seq Scan on public.prt1_p3 t2_1
+                                 Output: t2_1.a
+                                 Filter: (t1.b = t2_1.a)
+(49 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+--
+-- partitioned by expression
+--
+CREATE TABLE prt1_e (a int, b int, c varchar) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES START (0) END (250);
+CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES START (250) END (500);
+CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES START (500) END (600);
+INSERT INTO prt1_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_e AS SELECT * FROM prt1_e;
+CREATE TABLE prt2_e (a int, b int, c varchar) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES START (0) END (250);
+CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES START (250) END (500);
+CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES START (500) END (600);
+INSERT INTO prt2_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_e;
+ANALYZE prt2_e_p1;
+ANALYZE prt2_e_p2;
+ANALYZE prt2_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_e AS SELECT * FROM prt2_e;
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+                                  QUERY PLAN                                  
+------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_e_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_e_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
+               ->  Seq Scan on public.prt2_e_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_e_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+(34 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0150 | 150 | 0150
+ 300 | 0300 | 300 | 0300
+ 450 | 0450 | 450 | 0450
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1, uprt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.a % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+                                  QUERY PLAN                                  
+------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_e_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_e_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
+               ->  Seq Scan on public.prt2_e_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_e_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+  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 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1 LEFT JOIN uprt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 RIGHT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: (((t1.a + t1.b) / 2) = ((t2.b + t2.a) / 2))
+                     ->  Seq Scan on public.prt1_e_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                     ->  Hash
+                           Output: t2.b, t2.c, t2.a
+                           ->  Seq Scan on public.prt2_e_p1 t2
+                                 Output: t2.b, t2.c, t2.a
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: (((t1_1.a + t1_1.b) / 2) = ((t2_1.b + t2_1.a) / 2))
+                     ->  Seq Scan on public.prt1_e_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c, t2_1.a
+                           ->  Seq Scan on public.prt2_e_p2 t2_1
+                                 Output: t2_1.b, t2_1.c, t2_1.a
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: (((t1_2.a + t1_2.b) / 2) = ((t2_2.b + t2_2.a) / 2))
+                     ->  Seq Scan on public.prt1_e_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c, t2_2.a
+                           ->  Seq Scan on public.prt2_e_p3 t2_2
+                                 Output: t2_2.b, t2_2.c, t2_2.a
+                                 Filter: ((t2_2.b % 25) = 0)
+(36 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 RIGHT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+  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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1 RIGHT JOIN uprt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t2.b % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_e WHERE prt2_e.b % 25 = 0) t2 ON ((t1.a + t1.b)/2 = (t2.b + t2.a)/2) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                            QUERY PLAN                                            
+--------------------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_e_p1.a, prt1_e_p1.c, prt2_e_p1.b, prt2_e_p1.c
+   Sort Key: prt1_e_p1.a, prt2_e_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_e_p1.a, prt1_e_p1.c, prt2_e_p1.b, prt2_e_p1.c
+               Hash Cond: (((prt1_e_p1.a + prt1_e_p1.b) / 2) = ((prt2_e_p1.b + prt2_e_p1.a) / 2))
+               ->  Seq Scan on public.prt1_e_p1
+                     Output: prt1_e_p1.a, prt1_e_p1.c, prt1_e_p1.b
+                     Filter: ((prt1_e_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_e_p1.b, prt2_e_p1.c, prt2_e_p1.a
+                     ->  Seq Scan on public.prt2_e_p1
+                           Output: prt2_e_p1.b, prt2_e_p1.c, prt2_e_p1.a
+                           Filter: ((prt2_e_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_e_p2.a, prt1_e_p2.c, prt2_e_p2.b, prt2_e_p2.c
+               Hash Cond: (((prt1_e_p2.a + prt1_e_p2.b) / 2) = ((prt2_e_p2.b + prt2_e_p2.a) / 2))
+               ->  Seq Scan on public.prt1_e_p2
+                     Output: prt1_e_p2.a, prt1_e_p2.c, prt1_e_p2.b
+                     Filter: ((prt1_e_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_e_p2.b, prt2_e_p2.c, prt2_e_p2.a
+                     ->  Seq Scan on public.prt2_e_p2
+                           Output: prt2_e_p2.b, prt2_e_p2.c, prt2_e_p2.a
+                           Filter: ((prt2_e_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_e_p3.a, prt1_e_p3.c, prt2_e_p3.b, prt2_e_p3.c
+               Hash Cond: (((prt1_e_p3.a + prt1_e_p3.b) / 2) = ((prt2_e_p3.b + prt2_e_p3.a) / 2))
+               ->  Seq Scan on public.prt1_e_p3
+                     Output: prt1_e_p3.a, prt1_e_p3.c, prt1_e_p3.b
+                     Filter: ((prt1_e_p3.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_e_p3.b, prt2_e_p3.c, prt2_e_p3.a
+                     ->  Seq Scan on public.prt2_e_p3
+                           Output: prt2_e_p3.b, prt2_e_p3.c, prt2_e_p3.a
+                           Filter: ((prt2_e_p3.b % 25) = 0)
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_e WHERE prt2_e.b % 25 = 0) t2 ON ((t1.a + t1.b)/2 = (t2.b + t2.a)/2) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_e t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_e t2 WHERE t2.b % 25 = 0) t2 ON ((t1.a + t1.b)/2 = (t2.b + t2.a)/2) 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)
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 3) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 3) and (b 1) is considered for partition-wise join.
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on public.prt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = t1.a)
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_2.a)
+                     ->  Seq Scan on public.prt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                           ->  Hash Join
+                                 Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = t1_2.a)
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_1.a)
+                     ->  Seq Scan on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                           ->  Hash Join
+                                 Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = t1_1.a)
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+(57 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 3) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 3) and (b 1) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM uprt1 t1, uprt2 t2, uprt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on public.prt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = t1.a)
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_2.a)
+                     ->  Seq Scan on public.prt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = t1_2.a)
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_1.a)
+                     ->  Seq Scan on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = t1_1.a)
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+(57 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, 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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2 4) is considered for partition-wise join.
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: (((t3.a + t3.b) / 2) = t2.b)
+                     ->  Seq Scan on public.prt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = t1.a)
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: (((t3_1.a + t3_1.b) / 2) = t2_1.b)
+                     ->  Seq Scan on public.prt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = t1_2.a)
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: (((t3_2.a + t3_2.b) / 2) = t2_2.b)
+                     ->  Seq Scan on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = t1_1.a)
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+(57 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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2 4) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |          | 
+ 100 | 0100 |     |      |          | 
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |          | 
+ 250 | 0250 |     |      |          | 
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |          | 
+ 400 | 0400 |     |      |          | 
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |          | 
+ 550 | 0550 |     |      |          | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |          | 
+ 100 | 0100 |     |      |          | 
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |          | 
+ 250 | 0250 |     |      |          | 
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |          | 
+ 400 | 0400 |     |      |          | 
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |          | 
+ 550 | 0550 |     |      |          | 
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Hash Cond: (t2.b = t1.a)
+                     ->  Seq Scan on public.prt2_p1 t2
+                           Output: t2.b, t2.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Hash Cond: (t1.a = ((t3.a + t3.b) / 2))
+                                 ->  Seq Scan on public.prt1_p1 t1
+                                       Output: t1.a, t1.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                     Hash Cond: (t2_1.b = t1_2.a)
+                     ->  Seq Scan on public.prt2_p2 t2_1
+                           Output: t2_1.b, t2_1.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                                 Hash Cond: (t1_2.a = ((t3_1.a + t3_1.b) / 2))
+                                 ->  Seq Scan on public.prt1_p2 t1_2
+                                       Output: t1_2.a, t1_2.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                     Hash Cond: (t2_2.b = t1_1.a)
+                     ->  Seq Scan on public.prt2_p3 t2_2
+                           Output: t2_2.b, t2_2.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                                 Hash Cond: (t1_1.a = ((t3_2.a + t3_2.b) / 2))
+                                 ->  Seq Scan on public.prt1_p3 t1_1
+                                       Output: t1_1.a, t1_1.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: (t1.a = t2.b)
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = ((t3.a + t3.b) / 2))
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Hash Cond: (t1_2.a = t2_1.b)
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = ((t3_1.a + t3_1.b) / 2))
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Hash Cond: (t1_1.a = t2_2.b)
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = ((t3_2.a + t3_2.b) / 2))
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 6) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is considered for partition-wise join.
+                                                  QUERY PLAN                                                   
+---------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, ((prt1_e_p1.a + prt1_e_p1.b)), prt1_e_p1.c
+   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   ->  Result
+         Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, (prt1_e_p1.a + prt1_e_p1.b), prt1_e_p1.c
+         ->  Append
+               ->  Hash Full Join
+                     Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, prt1_e_p1.a, prt1_e_p1.b, prt1_e_p1.c
+                     Hash Cond: (prt1_p1.a = ((prt1_e_p1.a + prt1_e_p1.b) / 2))
+                     ->  Hash Full Join
+                           Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on public.prt1_p1
+                                 Output: prt1_p1.a, prt1_p1.c
+                                 Filter: ((prt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p1.b, prt2_p1.c
+                                 ->  Seq Scan on public.prt2_p1
+                                       Output: prt2_p1.b, prt2_p1.c
+                                       Filter: ((prt2_p1.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p1.a, prt1_e_p1.b, prt1_e_p1.c
+                           ->  Seq Scan on public.prt1_e_p1
+                                 Output: prt1_e_p1.a, prt1_e_p1.b, prt1_e_p1.c
+                                 Filter: ((prt1_e_p1.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c, prt1_e_p2.a, prt1_e_p2.b, prt1_e_p2.c
+                     Hash Cond: (prt1_p2.a = ((prt1_e_p2.a + prt1_e_p2.b) / 2))
+                     ->  Hash Full Join
+                           Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+                           Hash Cond: (prt1_p2.a = prt2_p2.b)
+                           ->  Seq Scan on public.prt1_p2
+                                 Output: prt1_p2.a, prt1_p2.c
+                                 Filter: ((prt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p2.b, prt2_p2.c
+                                 ->  Seq Scan on public.prt2_p2
+                                       Output: prt2_p2.b, prt2_p2.c
+                                       Filter: ((prt2_p2.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p2.a, prt1_e_p2.b, prt1_e_p2.c
+                           ->  Seq Scan on public.prt1_e_p2
+                                 Output: prt1_e_p2.a, prt1_e_p2.b, prt1_e_p2.c
+                                 Filter: ((prt1_e_p2.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c, prt1_e_p3.a, prt1_e_p3.b, prt1_e_p3.c
+                     Hash Cond: (prt1_p3.a = ((prt1_e_p3.a + prt1_e_p3.b) / 2))
+                     ->  Hash Full Join
+                           Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c
+                           Hash Cond: (prt1_p3.a = prt2_p3.b)
+                           ->  Seq Scan on public.prt1_p3
+                                 Output: prt1_p3.a, prt1_p3.c
+                                 Filter: ((prt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p3.b, prt2_p3.c
+                                 ->  Seq Scan on public.prt2_p3
+                                       Output: prt2_p3.b, prt2_p3.c
+                                       Filter: ((prt2_p3.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p3.a, prt1_e_p3.b, prt1_e_p3.c
+                           ->  Seq Scan on public.prt1_e_p3
+                                 Output: prt1_e_p3.a, prt1_e_p3.b, prt1_e_p3.c
+                                 Filter: ((prt1_e_p3.a % 25) = 0)
+(63 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 6) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+     |      |  75 | 0075 |          | 
+     |      | 225 | 0225 |          | 
+     |      | 375 | 0375 |          | 
+     |      | 525 | 0525 |          | 
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM uprt1_e WHERE uprt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+     |      |  75 | 0075 |          | 
+     |      | 225 | 0225 |          | 
+     |      | 375 | 0375 |          | 
+     |      | 525 | 0525 |          | 
+(16 rows)
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, 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.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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;
+NOTICE:  join between relations (b 6) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is considered for partition-wise join.
+                                                      QUERY PLAN                                                      
+----------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, (50), prt2_p1.b, (75), ((prt1_e_p1.a + prt1_e_p1.b)), (50)
+   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   ->  Result
+         Output: prt1_p1.a, (50), prt2_p1.b, (75), (prt1_e_p1.a + prt1_e_p1.b), (50)
+         ->  Append
+               ->  Hash Full Join
+                     Output: prt1_p1.a, prt2_p1.b, prt1_e_p1.a, prt1_e_p1.b, (50), (75), (50)
+                     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
+                           Output: prt1_p1.a, prt2_p1.b, (50), (75)
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on public.prt1_p1
+                                 Output: prt1_p1.a, 50
+                                 Filter: ((prt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p1.b, (75)
+                                 ->  Seq Scan on public.prt2_p1
+                                       Output: prt2_p1.b, 75
+                                       Filter: ((prt2_p1.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p1.a, prt1_e_p1.b, (50)
+                           ->  Seq Scan on public.prt1_e_p1
+                                 Output: prt1_e_p1.a, prt1_e_p1.b, 50
+                                 Filter: ((prt1_e_p1.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p2.a, prt2_p2.b, prt1_e_p2.a, prt1_e_p2.b, (50), (75), (50)
+                     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
+                           Output: prt1_p2.a, prt2_p2.b, (50), (75)
+                           Hash Cond: (prt1_p2.a = prt2_p2.b)
+                           ->  Seq Scan on public.prt1_p2
+                                 Output: prt1_p2.a, 50
+                                 Filter: ((prt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p2.b, (75)
+                                 ->  Seq Scan on public.prt2_p2
+                                       Output: prt2_p2.b, 75
+                                       Filter: ((prt2_p2.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p2.a, prt1_e_p2.b, (50)
+                           ->  Seq Scan on public.prt1_e_p2
+                                 Output: prt1_e_p2.a, prt1_e_p2.b, 50
+                                 Filter: ((prt1_e_p2.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p3.a, prt2_p3.b, prt1_e_p3.a, prt1_e_p3.b, (50), (75), (50)
+                     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
+                           Output: prt1_p3.a, prt2_p3.b, (50), (75)
+                           Hash Cond: (prt1_p3.a = prt2_p3.b)
+                           ->  Seq Scan on public.prt1_p3
+                                 Output: prt1_p3.a, 50
+                                 Filter: ((prt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p3.b, (75)
+                                 ->  Seq Scan on public.prt2_p3
+                                       Output: prt2_p3.b, 75
+                                       Filter: ((prt2_p3.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p3.a, prt1_e_p3.b, (50)
+                           ->  Seq Scan on public.prt1_e_p3
+                                 Output: prt1_e_p3.a, prt1_e_p3.b, 50
+                                 Filter: ((prt1_e_p3.a % 25) = 0)
+(66 rows)
+
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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;
+NOTICE:  join between relations (b 6) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is considered for partition-wise join.
+ a  | phv | b  | phv | ?column? | phv 
+----+-----+----+-----+----------+-----
+ 50 |  50 |    |     |      100 |  50
+    |     | 75 |  75 |          |    
+(2 rows)
+
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM uprt1_e WHERE uprt1_e.a % 25 = 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)
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 4) is considered for partition-wise join.
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.a = t1_3.b)
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_3.b, t2.a, t2.b
+                     ->  Hash Join
+                           Output: t1_3.b, t2.a, t2.b
+                           Hash Cond: (((t2.a + t2.b) / 2) = t1_3.b)
+                           ->  Seq Scan on public.prt1_e_p1 t2
+                                 Output: t2.a, t2.b
+                           ->  Hash
+                                 Output: t1_3.b
+                                 ->  Seq Scan on public.prt2_p1 t1_3
+                                       Output: t1_3.b
+                                       Filter: ((t1_3.b % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.a = t1_4.b)
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_4.b, t2_1.a, t2_1.b
+                     ->  Hash Join
+                           Output: t1_4.b, t2_1.a, t2_1.b
+                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_4.b)
+                           ->  Seq Scan on public.prt1_e_p2 t2_1
+                                 Output: t2_1.a, t2_1.b
+                           ->  Hash
+                                 Output: t1_4.b
+                                 ->  Seq Scan on public.prt2_p2 t1_4
+                                       Output: t1_4.b
+                                       Filter: ((t1_4.b % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.a = t1_5.b)
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_5.b, t2_2.a, t2_2.b
+                     ->  Hash Join
+                           Output: t1_5.b, t2_2.a, t2_2.b
+                           Hash Cond: (((t2_2.a + t2_2.b) / 2) = t1_5.b)
+                           ->  Seq Scan on public.prt1_e_p3 t2_2
+                                 Output: t2_2.a, t2_2.b
+                           ->  Hash
+                                 Output: t1_5.b
+                                 ->  Seq Scan on public.prt2_p3 t1_5
+                                       Output: t1_5.b
+                                       Filter: ((t1_5.b % 25) = 0)
+(58 rows)
+
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 4) is considered for partition-wise join.
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1, uprt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 5) is considered for partition-wise join.
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.a = t1_3.b)
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_3.b, t1_6.a, t1_6.b
+                     ->  Hash Semi Join
+                           Output: t1_3.b, t1_6.a, t1_6.b
+                           Hash Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
+                           ->  Seq Scan on public.prt2_p1 t1_3
+                                 Output: t1_3.b
+                           ->  Hash
+                                 Output: t1_6.a, t1_6.b
+                                 ->  Seq Scan on public.prt1_e_p1 t1_6
+                                       Output: t1_6.a, t1_6.b
+                                       Filter: ((t1_6.a % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.a = t1_4.b)
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_4.b, t1_7.a, t1_7.b
+                     ->  Hash Semi Join
+                           Output: t1_4.b, t1_7.a, t1_7.b
+                           Hash Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
+                           ->  Seq Scan on public.prt2_p2 t1_4
+                                 Output: t1_4.b
+                           ->  Hash
+                                 Output: t1_7.a, t1_7.b
+                                 ->  Seq Scan on public.prt1_e_p2 t1_7
+                                       Output: t1_7.a, t1_7.b
+                                       Filter: ((t1_7.a % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.a = t1_5.b)
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_5.b, t1_8.a, t1_8.b
+                     ->  Hash Semi Join
+                           Output: t1_5.b, t1_8.a, t1_8.b
+                           Hash Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
+                           ->  Seq Scan on public.prt2_p3 t1_5
+                                 Output: t1_5.b
+                           ->  Hash
+                                 Output: t1_8.a, t1_8.b
+                                 ->  Seq Scan on public.prt1_e_p3 t1_8
+                                       Output: t1_8.a, t1_8.b
+                                       Filter: ((t1_8.a % 25) = 0)
+(58 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 5) is considered for partition-wise join.
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+-- test merge joins with and without using indexes
+SET enable_hashjoin TO off;
+SET enable_nestloop TO off;
+CREATE INDEX iprt1_a on prt1(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 iprt2_b on prt2(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 iprt1_e_ab2 on prt1_e(((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));
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+                                             QUERY PLAN                                              
+-----------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Merge Cond: (t2.b = t1.a)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           Sort Key: t2.b
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Merge Cond: ((((t3.a + t3.b) / 2)) = t2.b)
+                                 ->  Sort
+                                       Output: t3.a, t3.b, t3.c, (((t3.a + t3.b) / 2))
+                                       Sort Key: (((t3.a + t3.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c, ((t3.a + t3.b) / 2)
+                                             Filter: ((t3.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2.b, t2.c
+                                       Sort Key: t2.b
+                                       ->  Seq Scan on public.prt2_p1 t2
+                                             Output: t2.b, t2.c
+                     ->  Sort
+                           Output: t1.a, t1.c
+                           Sort Key: t1.a
+                           ->  Seq Scan on public.prt1_p1 t1
+                                 Output: t1.a, t1.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Merge Cond: (t2_1.b = t1_2.a)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           Sort Key: t2_1.b
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t2_1.b)
+                                 ->  Sort
+                                       Output: t3_1.a, t3_1.b, t3_1.c, (((t3_1.a + t3_1.b) / 2))
+                                       Sort Key: (((t3_1.a + t3_1.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c, ((t3_1.a + t3_1.b) / 2)
+                                             Filter: ((t3_1.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2_1.b, t2_1.c
+                                       Sort Key: t2_1.b
+                                       ->  Seq Scan on public.prt2_p2 t2_1
+                                             Output: t2_1.b, t2_1.c
+                     ->  Sort
+                           Output: t1_2.a, t1_2.c
+                           Sort Key: t1_2.a
+                           ->  Seq Scan on public.prt1_p2 t1_2
+                                 Output: t1_2.a, t1_2.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Merge Cond: (t2_2.b = t1_1.a)
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           Sort Key: t2_2.b
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t2_2.b)
+                                 ->  Sort
+                                       Output: t3_2.a, t3_2.b, t3_2.c, (((t3_2.a + t3_2.b) / 2))
+                                       Sort Key: (((t3_2.a + t3_2.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c, ((t3_2.a + t3_2.b) / 2)
+                                             Filter: ((t3_2.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2_2.b, t2_2.c
+                                       Sort Key: t2_2.b
+                                       ->  Seq Scan on public.prt2_p3 t2_2
+                                             Output: t2_2.b, t2_2.c
+                     ->  Sort
+                           Output: t1_1.a, t1_1.c
+                           Sort Key: t1_1.a
+                           ->  Seq Scan on public.prt1_p3 t1_1
+                                 Output: t1_1.a, t1_1.c
+(81 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 5) is considered for partition-wise join.
+                                   QUERY PLAN                                    
+---------------------------------------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Output: t1.a, t1.b, t1.c
+         Merge Cond: (t1.a = t1_3.b)
+         ->  Sort
+               Output: t1.a, t1.b, t1.c
+               Sort Key: t1.a
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_3.b, t1_6.a, t1_6.b
+               ->  Merge Semi Join
+                     Output: t1_3.b, t1_6.a, t1_6.b
+                     Merge Cond: (t1_3.b = (((t1_6.a + t1_6.b) / 2)))
+                     ->  Sort
+                           Output: t1_3.b
+                           Sort Key: t1_3.b
+                           ->  Seq Scan on public.prt2_p1 t1_3
+                                 Output: t1_3.b
+                     ->  Sort
+                           Output: t1_6.a, t1_6.b, (((t1_6.a + t1_6.b) / 2))
+                           Sort Key: (((t1_6.a + t1_6.b) / 2))
+                           ->  Seq Scan on public.prt1_e_p1 t1_6
+                                 Output: t1_6.a, t1_6.b, ((t1_6.a + t1_6.b) / 2)
+                                 Filter: ((t1_6.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_2.a, t1_2.b, t1_2.c
+         Merge Cond: (t1_2.a = t1_4.b)
+         ->  Sort
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Sort Key: t1_2.a
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+         ->  Materialize
+               Output: t1_4.b, t1_7.a, t1_7.b
+               ->  Merge Semi Join
+                     Output: t1_4.b, t1_7.a, t1_7.b
+                     Merge Cond: (t1_4.b = (((t1_7.a + t1_7.b) / 2)))
+                     ->  Sort
+                           Output: t1_4.b
+                           Sort Key: t1_4.b
+                           ->  Seq Scan on public.prt2_p2 t1_4
+                                 Output: t1_4.b
+                     ->  Sort
+                           Output: t1_7.a, t1_7.b, (((t1_7.a + t1_7.b) / 2))
+                           Sort Key: (((t1_7.a + t1_7.b) / 2))
+                           ->  Seq Scan on public.prt1_e_p2 t1_7
+                                 Output: t1_7.a, t1_7.b, ((t1_7.a + t1_7.b) / 2)
+                                 Filter: ((t1_7.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_1.a, t1_1.b, t1_1.c
+         Merge Cond: (t1_1.a = t1_5.b)
+         ->  Sort
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Sort Key: t1_1.a
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_5.b, t1_8.a, t1_8.b
+               ->  Merge Semi Join
+                     Output: t1_5.b, t1_8.a, t1_8.b
+                     Merge Cond: (t1_5.b = (((t1_8.a + t1_8.b) / 2)))
+                     ->  Sort
+                           Output: t1_5.b
+                           Sort Key: t1_5.b
+                           ->  Seq Scan on public.prt2_p3 t1_5
+                                 Output: t1_5.b
+                     ->  Sort
+                           Output: t1_8.a, t1_8.b, (((t1_8.a + t1_8.b) / 2))
+                           Sort Key: (((t1_8.a + t1_8.b) / 2))
+                           ->  Seq Scan on public.prt1_e_p3 t1_8
+                                 Output: t1_8.a, t1_8.b, ((t1_8.a + t1_8.b) / 2)
+                                 Filter: ((t1_8.a % 25) = 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 5) is considered for partition-wise join.
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+                                             QUERY PLAN                                              
+-----------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Merge Cond: (t1.a = t2.b)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           Sort Key: t1.a
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Merge Cond: ((((t3.a + t3.b) / 2)) = t1.a)
+                                 ->  Sort
+                                       Output: t3.a, t3.b, t3.c, (((t3.a + t3.b) / 2))
+                                       Sort Key: (((t3.a + t3.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c, ((t3.a + t3.b) / 2)
+                                             Filter: ((t3.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t1.a, t1.c
+                                       Sort Key: t1.a
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                     ->  Sort
+                           Output: t2.b, t2.c
+                           Sort Key: t2.b
+                           ->  Seq Scan on public.prt2_p1 t2
+                                 Output: t2.b, t2.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                     Merge Cond: (t1_2.a = t2_1.b)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                           Sort Key: t1_2.a
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                                 Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t1_2.a)
+                                 ->  Sort
+                                       Output: t3_1.a, t3_1.b, t3_1.c, (((t3_1.a + t3_1.b) / 2))
+                                       Sort Key: (((t3_1.a + t3_1.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c, ((t3_1.a + t3_1.b) / 2)
+                                             Filter: ((t3_1.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t1_2.a, t1_2.c
+                                       Sort Key: t1_2.a
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                     ->  Sort
+                           Output: t2_1.b, t2_1.c
+                           Sort Key: t2_1.b
+                           ->  Seq Scan on public.prt2_p2 t2_1
+                                 Output: t2_1.b, t2_1.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                     Merge Cond: (t1_1.a = t2_2.b)
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                           Sort Key: t1_1.a
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                                 Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t1_1.a)
+                                 ->  Sort
+                                       Output: t3_2.a, t3_2.b, t3_2.c, (((t3_2.a + t3_2.b) / 2))
+                                       Sort Key: (((t3_2.a + t3_2.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c, ((t3_2.a + t3_2.b) / 2)
+                                             Filter: ((t3_2.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t1_1.a, t1_1.c
+                                       Sort Key: t1_1.a
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                     ->  Sort
+                           Output: t2_2.b, t2_2.c
+                           Sort Key: t2_2.b
+                           ->  Seq Scan on public.prt2_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+(81 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SET enable_seqscan TO off;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+                                          QUERY PLAN                                          
+----------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Merge Cond: (t2.b = t1.a)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           Sort Key: t2.b
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Merge Cond: (((t3.a + t3.b) / 2) = t2.b)
+                                 ->  Index Scan using iprt1_e_p1_ab2 on public.prt1_e_p1 t3
+                                       Output: t3.a, t3.b, t3.c
+                                       Filter: ((t3.a % 25) = 0)
+                                 ->  Index Scan using iprt2_p1_b on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                     ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Merge Cond: (t2_1.b = t1_2.a)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           Sort Key: t2_1.b
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Merge Cond: (((t3_1.a + t3_1.b) / 2) = t2_1.b)
+                                 ->  Index Scan using iprt1_e_p2_ab2 on public.prt1_e_p2 t3_1
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       Filter: ((t3_1.a % 25) = 0)
+                                 ->  Index Scan using iprt2_p2_b on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                     ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+               ->  Merge Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Merge Cond: (t2_2.b = ((t3_2.a + t3_2.b) / 2))
+                     ->  Merge Left Join
+                           Output: t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                           Merge Cond: (t2_2.b = t1_1.a)
+                           ->  Index Scan using iprt2_p3_b on public.prt2_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+                           ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_1
+                                 Output: t1_1.a, t1_1.c
+                     ->  Index Scan using iprt1_e_p3_ab2 on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                           Filter: ((t3_2.a % 25) = 0)
+(51 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 5) is considered for partition-wise join.
+                                    QUERY PLAN                                    
+----------------------------------------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Output: t1.a, t1.b, t1.c
+         Merge Cond: (t1.a = t1_3.b)
+         ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1
+               Output: t1.a, t1.b, t1.c
+               Filter: ((t1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_3.b, t1_6.a, t1_6.b
+               ->  Merge Semi Join
+                     Output: t1_3.b, t1_6.a, t1_6.b
+                     Merge Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
+                     ->  Index Only Scan using iprt2_p1_b on public.prt2_p1 t1_3
+                           Output: t1_3.b
+                     ->  Index Scan using iprt1_e_p1_ab2 on public.prt1_e_p1 t1_6
+                           Output: t1_6.a, t1_6.b
+                           Filter: ((t1_6.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_2.a, t1_2.b, t1_2.c
+         Merge Cond: (t1_2.a = t1_4.b)
+         ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_2
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Filter: ((t1_2.a % 25) = 0)
+         ->  Materialize
+               Output: t1_4.b, t1_7.a, t1_7.b
+               ->  Merge Semi Join
+                     Output: t1_4.b, t1_7.a, t1_7.b
+                     Merge Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
+                     ->  Index Only Scan using iprt2_p2_b on public.prt2_p2 t1_4
+                           Output: t1_4.b
+                     ->  Index Scan using iprt1_e_p2_ab2 on public.prt1_e_p2 t1_7
+                           Output: t1_7.a, t1_7.b
+                           Filter: ((t1_7.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_1.a, t1_1.b, t1_1.c
+         Merge Cond: (t1_1.a = t1_5.b)
+         ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_1
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Filter: ((t1_1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_5.b, t1_8.a, t1_8.b
+               ->  Merge Semi Join
+                     Output: t1_5.b, t1_8.a, t1_8.b
+                     Merge Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
+                     ->  Index Only Scan using iprt2_p3_b on public.prt2_p3 t1_5
+                           Output: t1_5.b
+                     ->  Index Scan using iprt1_e_p3_ab2 on public.prt1_e_p3 t1_8
+                           Output: t1_8.a, t1_8.b
+                           Filter: ((t1_8.a % 25) = 0)
+(50 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 5) is considered for partition-wise join.
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+                                          QUERY PLAN                                          
+----------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Merge Cond: (t1.a = t2.b)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           Sort Key: t1.a
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Merge Cond: (((t3.a + t3.b) / 2) = t1.a)
+                                 ->  Index Scan using iprt1_e_p1_ab2 on public.prt1_e_p1 t3
+                                       Output: t3.a, t3.b, t3.c
+                                       Filter: ((t3.a % 25) = 0)
+                                 ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1
+                                       Output: t1.a, t1.c
+                     ->  Index Scan using iprt2_p1_b on public.prt2_p1 t2
+                           Output: t2.b, t2.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                     Merge Cond: (t1_2.a = t2_1.b)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                           Sort Key: t1_2.a
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                                 Merge Cond: (((t3_1.a + t3_1.b) / 2) = t1_2.a)
+                                 ->  Index Scan using iprt1_e_p2_ab2 on public.prt1_e_p2 t3_1
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       Filter: ((t3_1.a % 25) = 0)
+                                 ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_2
+                                       Output: t1_2.a, t1_2.c
+                     ->  Index Scan using iprt2_p2_b on public.prt2_p2 t2_1
+                           Output: t2_1.b, t2_1.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                     Merge Cond: (t1_1.a = t2_2.b)
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                           Sort Key: t1_1.a
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                                 Merge Cond: (((t3_2.a + t3_2.b) / 2) = t1_1.a)
+                                 ->  Index Scan using iprt1_e_p3_ab2 on public.prt1_e_p3 t3_2
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       Filter: ((t3_2.a % 25) = 0)
+                                 ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_1
+                                       Output: t1_1.a, t1_1.c
+                     ->  Index Scan using iprt2_p3_b on public.prt2_p3 t2_2
+                           Output: t2_2.b, t2_2.c
+(54 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+-- lateral references and parameterized paths
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4 5) is considered for partition-wise join.
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
+ Nested Loop Left Join
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   ->  Merge Append
+         Sort Key: t1.a
+         ->  Index Scan using iprt1_a on public.prt1 t1
+               Output: t1.a, t1.b, t1.c
+               Filter: ((t1.a % 25) = 0)
+         ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1_1
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Filter: ((t1_1.a % 25) = 0)
+         ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_2
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Filter: ((t1_2.a % 25) = 0)
+         ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_3
+               Output: t1_3.a, t1_3.b, t1_3.c
+               Filter: ((t1_3.a % 25) = 0)
+   ->  Append
+         ->  Merge Join
+               Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+               Merge Cond: (t2.a = t3.b)
+               ->  Index Only Scan using iprt1_p1_a on public.prt1_p1 t2
+                     Output: t2.a
+                     Index Cond: (t2.a = t1.a)
+               ->  Index Scan using iprt2_p1_b on public.prt2_p1 t3
+                     Output: t3.a, t3.b
+         ->  Merge Join
+               Output: t2_2.a, t3_1.a, LEAST(t1.a, t2_2.a, t3_1.a)
+               Merge Cond: (t2_2.a = t3_1.b)
+               ->  Index Only Scan using iprt1_p2_a on public.prt1_p2 t2_2
+                     Output: t2_2.a
+                     Index Cond: (t2_2.a = t1.a)
+               ->  Index Scan using iprt2_p2_b on public.prt2_p2 t3_1
+                     Output: t3_1.a, t3_1.b
+         ->  Merge Join
+               Output: t2_1.a, t3_2.a, LEAST(t1.a, t2_1.a, t3_2.a)
+               Merge Cond: (t2_1.a = t3_2.b)
+               ->  Index Only Scan using iprt1_p3_a on public.prt1_p3 t2_1
+                     Output: t2_1.a
+                     Index Cond: (t2_1.a = t1.a)
+               ->  Index Scan using iprt2_p3_b on public.prt2_p3 t3_2
+                     Output: t3_2.a, t3_2.b
+(41 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4 5) is considered for partition-wise join.
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
+ Nested Loop Left Join
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   ->  Merge Append
+         Sort Key: t1.a
+         ->  Index Scan using iprt1_a on public.prt1 t1
+               Output: t1.a, t1.b, t1.c
+               Filter: ((t1.a % 25) = 0)
+         ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1_1
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Filter: ((t1_1.a % 25) = 0)
+         ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_2
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Filter: ((t1_2.a % 25) = 0)
+         ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_3
+               Output: t1_3.a, t1_3.b, t1_3.c
+               Filter: ((t1_3.a % 25) = 0)
+   ->  Append
+         ->  Merge Join
+               Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+               Merge Cond: (t2.a = t3.b)
+               ->  Index Only Scan using iprt1_p1_a on public.prt1_p1 t2
+                     Output: t2.a
+                     Index Cond: (t2.a = t1.b)
+               ->  Index Scan using iprt2_p1_b on public.prt2_p1 t3
+                     Output: t3.a, t3.b
+         ->  Merge Join
+               Output: t2_2.a, t3_1.a, LEAST(t1.a, t2_2.a, t3_1.a)
+               Merge Cond: (t2_2.a = t3_1.b)
+               ->  Index Only Scan using iprt1_p2_a on public.prt1_p2 t2_2
+                     Output: t2_2.a
+                     Index Cond: (t2_2.a = t1.b)
+               ->  Index Scan using iprt2_p2_b on public.prt2_p2 t3_1
+                     Output: t3_1.a, t3_1.b
+         ->  Merge Join
+               Output: t2_1.a, t3_2.a, LEAST(t1.a, t2_1.a, t3_2.a)
+               Merge Cond: (t2_1.a = t3_2.b)
+               ->  Index Only Scan using iprt1_p3_a on public.prt1_p3 t2_1
+                     Output: t2_1.a
+                     Index Cond: (t2_1.a = t1.b)
+               ->  Index Scan using iprt2_p3_b on public.prt2_p3 t3_2
+                     Output: t3_2.a, t3_2.b
+(41 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+RESET enable_hashjoin;
+RESET enable_nestloop;
+RESET enable_seqscan;
+--
+-- partitioned by multiple columns
+--
+CREATE TABLE prt1_m (a int, b int, c varchar) PARTITION BY RANGE(a, ((a + b)/2));
+CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES START (0, 0) END (250, 250);
+CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES START (250, 250) END (500, 500);
+CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES START (500, 500) END (600, 600);
+INSERT INTO prt1_m SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_m;
+ANALYZE prt1_m_p1;
+ANALYZE prt1_m_p2;
+ANALYZE prt1_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_m AS SELECT * FROM prt1_m;
+CREATE TABLE prt2_m (a int, b int, c varchar) PARTITION BY RANGE(((b + a)/2), b);
+CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES START (0, 0) END (250, 250);
+CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES START (250, 250) END (500, 500);
+CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES START (500, 500) END (600, 600);
+INSERT INTO prt2_m SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_m;
+ANALYZE prt2_m_p1;
+ANALYZE prt2_m_p2;
+ANALYZE prt2_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_m AS SELECT * FROM prt2_m;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1, prt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+                                              QUERY PLAN                                              
+------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((((t2.b + t2.a) / 2) = t1.a) AND (t2.b = ((t1.a + t1.b) / 2)))
+               ->  Seq Scan on public.prt2_m_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_m_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((((t2_1.b + t2_1.a) / 2) = t1_1.a) AND (t2_1.b = ((t1_1.a + t1_1.b) / 2)))
+               ->  Seq Scan on public.prt2_m_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_m_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((((t2_2.b + t2_2.a) / 2) = t1_2.a) AND (t2_2.b = ((t1_2.a + t1_2.b) / 2)))
+               ->  Seq Scan on public.prt2_m_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_m_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1, prt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0150 | 150 | 0150
+ 300 | 0300 | 300 | 0300
+ 450 | 0450 | 450 | 0450
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_m t1, uprt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 AND t1.a % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+                                              QUERY PLAN                                              
+------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((((t2.b + t2.a) / 2) = t1.a) AND (t2.b = ((t1.a + t1.b) / 2)))
+               ->  Seq Scan on public.prt2_m_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_m_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((((t2_1.b + t2_1.a) / 2) = t1_1.a) AND (t2_1.b = ((t1_1.a + t1_1.b) / 2)))
+               ->  Seq Scan on public.prt2_m_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_m_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((((t2_2.b + t2_2.a) / 2) = t1_2.a) AND (t2_2.b = ((t1_2.a + t1_2.b) / 2)))
+               ->  Seq Scan on public.prt2_m_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_m_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+  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 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_m t1 LEFT JOIN uprt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+                                                 QUERY PLAN                                                 
+------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: ((((t1.a + t1.b) / 2) = t2.b) AND (t1.a = ((t2.b + t2.a) / 2)))
+                     ->  Seq Scan on public.prt1_m_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                     ->  Hash
+                           Output: t2.b, t2.c, t2.a
+                           ->  Seq Scan on public.prt2_m_p1 t2
+                                 Output: t2.b, t2.c, t2.a
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: ((((t1_1.a + t1_1.b) / 2) = t2_1.b) AND (t1_1.a = ((t2_1.b + t2_1.a) / 2)))
+                     ->  Seq Scan on public.prt1_m_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c, t2_1.a
+                           ->  Seq Scan on public.prt2_m_p2 t2_1
+                                 Output: t2_1.b, t2_1.c, t2_1.a
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: ((((t1_2.a + t1_2.b) / 2) = t2_2.b) AND (t1_2.a = ((t2_2.b + t2_2.a) / 2)))
+                     ->  Seq Scan on public.prt1_m_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c, t2_2.a
+                           ->  Seq Scan on public.prt2_m_p3 t2_2
+                                 Output: t2_2.b, t2_2.c, t2_2.a
+                                 Filter: ((t2_2.b % 25) = 0)
+(36 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+  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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_m t1 RIGHT JOIN uprt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                                             QUERY PLAN                                                             
+------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_m_p1.a, prt1_m_p1.c, prt2_m_p1.b, prt2_m_p1.c
+   Sort Key: prt1_m_p1.a, prt2_m_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_m_p1.a, prt1_m_p1.c, prt2_m_p1.b, prt2_m_p1.c
+               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 public.prt1_m_p1
+                     Output: prt1_m_p1.a, prt1_m_p1.c, prt1_m_p1.b
+                     Filter: ((prt1_m_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_m_p1.b, prt2_m_p1.c, prt2_m_p1.a
+                     ->  Seq Scan on public.prt2_m_p1
+                           Output: prt2_m_p1.b, prt2_m_p1.c, prt2_m_p1.a
+                           Filter: ((prt2_m_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_m_p2.a, prt1_m_p2.c, prt2_m_p2.b, prt2_m_p2.c
+               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 public.prt1_m_p2
+                     Output: prt1_m_p2.a, prt1_m_p2.c, prt1_m_p2.b
+                     Filter: ((prt1_m_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_m_p2.b, prt2_m_p2.c, prt2_m_p2.a
+                     ->  Seq Scan on public.prt2_m_p2
+                           Output: prt2_m_p2.b, prt2_m_p2.c, prt2_m_p2.a
+                           Filter: ((prt2_m_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_m_p3.a, prt1_m_p3.c, prt2_m_p3.b, prt2_m_p3.c
+               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 public.prt1_m_p3
+                     Output: prt1_m_p3.a, prt1_m_p3.c, prt1_m_p3.b
+                     Filter: ((prt1_m_p3.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_m_p3.b, prt2_m_p3.c, prt2_m_p3.a
+                     ->  Seq Scan on public.prt2_m_p3
+                           Output: prt2_m_p3.b, prt2_m_p3.c, prt2_m_p3.a
+                           Filter: ((prt2_m_p3.b % 25) = 0)
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_m t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_m t2 WHERE t2.b % 25 = 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 | 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)
+
+--
+-- multi-leveled partitions
+--
+CREATE TABLE prt1_l (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_l_p1 PARTITION OF prt1_l FOR VALUES START (0) END (250) PARTITION BY RANGE (b);
+CREATE TABLE prt1_l_p1_p1 PARTITION OF prt1_l_p1 FOR VALUES START (0) END (100);
+CREATE TABLE prt1_l_p1_p2 PARTITION OF prt1_l_p1 FOR VALUES START (100) END (250);
+CREATE TABLE prt1_l_p2 PARTITION OF prt1_l FOR VALUES START (250) END (500) PARTITION BY RANGE (c);
+CREATE TABLE prt1_l_p2_p1 PARTITION OF prt1_l_p2 FOR VALUES START ('0250') END ('0400');
+CREATE TABLE prt1_l_p2_p2 PARTITION OF prt1_l_p2 FOR VALUES START ('0400') END ('0500');
+CREATE TABLE prt1_l_p3 PARTITION OF prt1_l FOR VALUES START (500) END (600) PARTITION BY RANGE ((b + a));
+CREATE TABLE prt1_l_p3_p1 PARTITION OF prt1_l_p3 FOR VALUES START (1000) END (1100);
+CREATE TABLE prt1_l_p3_p2 PARTITION OF prt1_l_p3 FOR VALUES START (1100) END (1200);
+INSERT INTO prt1_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_l;
+ANALYZE prt1_l_p1;
+ANALYZE prt1_l_p1_p1;
+ANALYZE prt1_l_p1_p2;
+ANALYZE prt1_l_p2;
+ANALYZE prt1_l_p2_p1;
+ANALYZE prt1_l_p2_p2;
+ANALYZE prt1_l_p3;
+ANALYZE prt1_l_p3_p1;
+ANALYZE prt1_l_p3_p2;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_l AS SELECT * FROM prt1_l;
+CREATE TABLE prt2_l (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_l_p1 PARTITION OF prt2_l FOR VALUES START (0) END (250) PARTITION BY RANGE (a);
+CREATE TABLE prt2_l_p1_p1 PARTITION OF prt2_l_p1 FOR VALUES START (0) END (100);
+CREATE TABLE prt2_l_p1_p2 PARTITION OF prt2_l_p1 FOR VALUES START (100) END (250);
+CREATE TABLE prt2_l_p2 PARTITION OF prt2_l FOR VALUES START (250) END (500) PARTITION BY RANGE (c);
+CREATE TABLE prt2_l_p2_p1 PARTITION OF prt2_l_p2 FOR VALUES START ('0250') END ('0400');
+CREATE TABLE prt2_l_p2_p2 PARTITION OF prt2_l_p2 FOR VALUES START ('0400') END ('0500');
+CREATE TABLE prt2_l_p3 PARTITION OF prt2_l FOR VALUES START (500) END (600) PARTITION BY RANGE ((a + b));
+CREATE TABLE prt2_l_p3_p1 PARTITION OF prt2_l_p3 FOR VALUES START (1000) END (1100);
+CREATE TABLE prt2_l_p3_p2 PARTITION OF prt2_l_p3 FOR VALUES START (1100) END (1200);
+INSERT INTO prt2_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_l;
+ANALYZE prt2_l_p1;
+ANALYZE prt2_l_p1_p1;
+ANALYZE prt2_l_p1_p2;
+ANALYZE prt2_l_p2;
+ANALYZE prt2_l_p2_p1;
+ANALYZE prt2_l_p2_p2;
+ANALYZE prt2_l_p3;
+ANALYZE prt2_l_p3_p1;
+ANALYZE prt2_l_p3_p2;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_l AS SELECT * FROM prt2_l;
+EXPLAIN (VERBOSE, 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 = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 17) is considered for partition-wise join.
+NOTICE:  join between relations (b 8) and (b 21) is considered for partition-wise join.
+NOTICE:  join between relations (b 12) and (b 25) is considered for partition-wise join.
+                                                                      QUERY PLAN                                                                      
+------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((t2.b = t1.a) AND (t2.a = t1.b) AND ((t2.c)::text = (t1.c)::text) AND ((t2.a + t2.b) = (t1.b + t1.a)))
+               ->  Seq Scan on public.prt2_l_p1_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_l_p1_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.a = t1_1.b) AND ((t2_1.c)::text = (t1_1.c)::text) AND ((t2_1.a + t2_1.b) = (t1_1.b + t1_1.a)))
+               ->  Seq Scan on public.prt2_l_p1_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_l_p1_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.a = t1_2.b) AND ((t2_2.c)::text = (t1_2.c)::text) AND ((t2_2.a + t2_2.b) = (t1_2.b + t1_2.a)))
+               ->  Seq Scan on public.prt2_l_p2_p1 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_l_p2_p1 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_3.a, t1_3.c, t2_3.b, t2_3.c
+               Hash Cond: ((t2_3.b = t1_3.a) AND (t2_3.a = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text) AND ((t2_3.a + t2_3.b) = (t1_3.b + t1_3.a)))
+               ->  Seq Scan on public.prt2_l_p2_p2 t2_3
+                     Output: t2_3.b, t2_3.c, t2_3.a
+               ->  Hash
+                     Output: t1_3.a, t1_3.c, t1_3.b
+                     ->  Seq Scan on public.prt1_l_p2_p2 t1_3
+                           Output: t1_3.a, t1_3.c, t1_3.b
+                           Filter: ((t1_3.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_4.a, t1_4.c, t2_4.b, t2_4.c
+               Hash Cond: ((t2_4.b = t1_4.a) AND (t2_4.a = t1_4.b) AND ((t2_4.c)::text = (t1_4.c)::text) AND ((t2_4.a + t2_4.b) = (t1_4.b + t1_4.a)))
+               ->  Seq Scan on public.prt2_l_p3_p1 t2_4
+                     Output: t2_4.b, t2_4.c, t2_4.a
+               ->  Hash
+                     Output: t1_4.a, t1_4.c, t1_4.b
+                     ->  Seq Scan on public.prt1_l_p3_p1 t1_4
+                           Output: t1_4.a, t1_4.c, t1_4.b
+                           Filter: ((t1_4.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_5.a, t1_5.c, t2_5.b, t2_5.c
+               Hash Cond: ((t2_5.b = t1_5.a) AND (t2_5.a = t1_5.b) AND ((t2_5.c)::text = (t1_5.c)::text) AND ((t2_5.a + t2_5.b) = (t1_5.b + t1_5.a)))
+               ->  Seq Scan on public.prt2_l_p3_p2 t2_5
+                     Output: t2_5.b, t2_5.c, t2_5.a
+               ->  Hash
+                     Output: t1_5.a, t1_5.c, t1_5.b
+                     ->  Seq Scan on public.prt1_l_p3_p2 t1_5
+                           Output: t1_5.a, t1_5.c, t1_5.b
+                           Filter: ((t1_5.a % 25) = 0)
+(64 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1, prt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 17) is considered for partition-wise join.
+NOTICE:  join between relations (b 8) and (b 21) is considered for partition-wise join.
+NOTICE:  join between relations (b 12) and (b 25) is considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0150 | 150 | 0150
+ 300 | 0300 | 300 | 0300
+ 450 | 0450 | 450 | 0450
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1, uprt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 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)
+
+EXPLAIN (VERBOSE, 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 5) and (b 18) is considered for partition-wise join.
+NOTICE:  join between relations (b 9) and (b 22) is considered for partition-wise join.
+NOTICE:  join between relations (b 13) and (b 26) is considered for partition-wise join.
+                                                                      QUERY PLAN                                                                      
+------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((t2.b = t1.a) AND (t2.a = t1.b) AND ((t2.c)::text = (t1.c)::text) AND ((t2.a + t2.b) = (t1.b + t1.a)))
+               ->  Seq Scan on public.prt2_l_p1_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_l_p1_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.a = t1_1.b) AND ((t2_1.c)::text = (t1_1.c)::text) AND ((t2_1.a + t2_1.b) = (t1_1.b + t1_1.a)))
+               ->  Seq Scan on public.prt2_l_p1_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_l_p1_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.a = t1_2.b) AND ((t2_2.c)::text = (t1_2.c)::text) AND ((t2_2.a + t2_2.b) = (t1_2.b + t1_2.a)))
+               ->  Seq Scan on public.prt2_l_p2_p1 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_l_p2_p1 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_3.a, t1_3.c, t2_3.b, t2_3.c
+               Hash Cond: ((t2_3.b = t1_3.a) AND (t2_3.a = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text) AND ((t2_3.a + t2_3.b) = (t1_3.b + t1_3.a)))
+               ->  Seq Scan on public.prt2_l_p2_p2 t2_3
+                     Output: t2_3.b, t2_3.c, t2_3.a
+               ->  Hash
+                     Output: t1_3.a, t1_3.c, t1_3.b
+                     ->  Seq Scan on public.prt1_l_p2_p2 t1_3
+                           Output: t1_3.a, t1_3.c, t1_3.b
+                           Filter: ((t1_3.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_4.a, t1_4.c, t2_4.b, t2_4.c
+               Hash Cond: ((t2_4.b = t1_4.a) AND (t2_4.a = t1_4.b) AND ((t2_4.c)::text = (t1_4.c)::text) AND ((t2_4.a + t2_4.b) = (t1_4.b + t1_4.a)))
+               ->  Seq Scan on public.prt2_l_p3_p1 t2_4
+                     Output: t2_4.b, t2_4.c, t2_4.a
+               ->  Hash
+                     Output: t1_4.a, t1_4.c, t1_4.b
+                     ->  Seq Scan on public.prt1_l_p3_p1 t1_4
+                           Output: t1_4.a, t1_4.c, t1_4.b
+                           Filter: ((t1_4.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_5.a, t1_5.c, t2_5.b, t2_5.c
+               Hash Cond: ((t2_5.b = t1_5.a) AND (t2_5.a = t1_5.b) AND ((t2_5.c)::text = (t1_5.c)::text) AND ((t2_5.a + t2_5.b) = (t1_5.b + t1_5.a)))
+               ->  Seq Scan on public.prt2_l_p3_p2 t2_5
+                     Output: t2_5.b, t2_5.c, t2_5.a
+               ->  Hash
+                     Output: t1_5.a, t1_5.c, t1_5.b
+                     ->  Seq Scan on public.prt1_l_p3_p2 t1_5
+                           Output: t1_5.a, t1_5.c, t1_5.b
+                           Filter: ((t1_5.a % 25) = 0)
+(64 rows)
+
+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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 5) and (b 18) is considered for partition-wise join.
+NOTICE:  join between relations (b 9) and (b 22) is considered for partition-wise join.
+NOTICE:  join between relations (b 13) and (b 26) is considered for partition-wise join.
+  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 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 LEFT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 18) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 22) and (b 9) is considered for partition-wise join.
+NOTICE:  join between relations (b 26) and (b 13) is considered for partition-wise join.
+                                                                         QUERY PLAN                                                                         
+------------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: ((t1.a = t2.b) AND (t1.b = t2.a) AND ((t1.c)::text = (t2.c)::text) AND ((t1.b + t1.a) = (t2.a + t2.b)))
+                     ->  Seq Scan on public.prt1_l_p1_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                     ->  Hash
+                           Output: t2.b, t2.c, t2.a
+                           ->  Seq Scan on public.prt2_l_p1_p1 t2
+                                 Output: t2.b, t2.c, t2.a
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: ((t1_1.a = t2_1.b) AND (t1_1.b = t2_1.a) AND ((t1_1.c)::text = (t2_1.c)::text) AND ((t1_1.b + t1_1.a) = (t2_1.a + t2_1.b)))
+                     ->  Seq Scan on public.prt1_l_p1_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c, t2_1.a
+                           ->  Seq Scan on public.prt2_l_p1_p2 t2_1
+                                 Output: t2_1.b, t2_1.c, t2_1.a
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: ((t1_2.a = t2_2.b) AND (t1_2.b = t2_2.a) AND ((t1_2.c)::text = (t2_2.c)::text) AND ((t1_2.b + t1_2.a) = (t2_2.a + t2_2.b)))
+                     ->  Seq Scan on public.prt1_l_p2_p1 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c, t2_2.a
+                           ->  Seq Scan on public.prt2_l_p2_p1 t2_2
+                                 Output: t2_2.b, t2_2.c, t2_2.a
+                                 Filter: ((t2_2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_3.b, t2_3.c, t1_3.a, t1_3.c
+                     Hash Cond: ((t1_3.a = t2_3.b) AND (t1_3.b = t2_3.a) AND ((t1_3.c)::text = (t2_3.c)::text) AND ((t1_3.b + t1_3.a) = (t2_3.a + t2_3.b)))
+                     ->  Seq Scan on public.prt1_l_p2_p2 t1_3
+                           Output: t1_3.a, t1_3.c, t1_3.b
+                     ->  Hash
+                           Output: t2_3.b, t2_3.c, t2_3.a
+                           ->  Seq Scan on public.prt2_l_p2_p2 t2_3
+                                 Output: t2_3.b, t2_3.c, t2_3.a
+                                 Filter: ((t2_3.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_4.b, t2_4.c, t1_4.a, t1_4.c
+                     Hash Cond: ((t1_4.a = t2_4.b) AND (t1_4.b = t2_4.a) AND ((t1_4.c)::text = (t2_4.c)::text) AND ((t1_4.b + t1_4.a) = (t2_4.a + t2_4.b)))
+                     ->  Seq Scan on public.prt1_l_p3_p1 t1_4
+                           Output: t1_4.a, t1_4.c, t1_4.b
+                     ->  Hash
+                           Output: t2_4.b, t2_4.c, t2_4.a
+                           ->  Seq Scan on public.prt2_l_p3_p1 t2_4
+                                 Output: t2_4.b, t2_4.c, t2_4.a
+                                 Filter: ((t2_4.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_5.b, t2_5.c, t1_5.a, t1_5.c
+                     Hash Cond: ((t1_5.a = t2_5.b) AND (t1_5.b = t2_5.a) AND ((t1_5.c)::text = (t2_5.c)::text) AND ((t1_5.b + t1_5.a) = (t2_5.a + t2_5.b)))
+                     ->  Seq Scan on public.prt1_l_p3_p2 t1_5
+                           Output: t1_5.a, t1_5.c, t1_5.b
+                     ->  Hash
+                           Output: t2_5.b, t2_5.c, t2_5.a
+                           ->  Seq Scan on public.prt2_l_p3_p2 t2_5
+                                 Output: t2_5.b, t2_5.c, t2_5.a
+                                 Filter: ((t2_5.b % 25) = 0)
+(66 rows)
+
+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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 18) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 22) and (b 9) is considered for partition-wise join.
+NOTICE:  join between relations (b 26) and (b 13) is considered for partition-wise join.
+  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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 RIGHT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 7) and (b 20) is considered for partition-wise join.
+NOTICE:  join between relations (b 11) and (b 24) is considered for partition-wise join.
+NOTICE:  join between relations (b 15) and (b 28) is considered for partition-wise join.
+                                                                                                              QUERY PLAN                                                                                                              
+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_l_p1_p1.a, prt1_l_p1_p1.c, prt2_l_p1_p1.b, prt2_l_p1_p1.c
+   Sort Key: prt1_l_p1_p1.a, prt2_l_p1_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_l_p1_p1.a, prt1_l_p1_p1.c, prt2_l_p1_p1.b, prt2_l_p1_p1.c
+               Hash Cond: ((prt1_l_p1_p1.a = prt2_l_p1_p1.b) AND (prt1_l_p1_p1.b = prt2_l_p1_p1.a) AND ((prt1_l_p1_p1.c)::text = (prt2_l_p1_p1.c)::text) AND ((prt1_l_p1_p1.b + prt1_l_p1_p1.a) = (prt2_l_p1_p1.a + prt2_l_p1_p1.b)))
+               ->  Seq Scan on public.prt1_l_p1_p1
+                     Output: prt1_l_p1_p1.a, prt1_l_p1_p1.c, prt1_l_p1_p1.b
+                     Filter: ((prt1_l_p1_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p1_p1.b, prt2_l_p1_p1.c, prt2_l_p1_p1.a
+                     ->  Seq Scan on public.prt2_l_p1_p1
+                           Output: prt2_l_p1_p1.b, prt2_l_p1_p1.c, prt2_l_p1_p1.a
+                           Filter: ((prt2_l_p1_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_l_p1_p2.a, prt1_l_p1_p2.c, prt2_l_p1_p2.b, prt2_l_p1_p2.c
+               Hash Cond: ((prt1_l_p1_p2.a = prt2_l_p1_p2.b) AND (prt1_l_p1_p2.b = prt2_l_p1_p2.a) AND ((prt1_l_p1_p2.c)::text = (prt2_l_p1_p2.c)::text) AND ((prt1_l_p1_p2.b + prt1_l_p1_p2.a) = (prt2_l_p1_p2.a + prt2_l_p1_p2.b)))
+               ->  Seq Scan on public.prt1_l_p1_p2
+                     Output: prt1_l_p1_p2.a, prt1_l_p1_p2.c, prt1_l_p1_p2.b
+                     Filter: ((prt1_l_p1_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p1_p2.b, prt2_l_p1_p2.c, prt2_l_p1_p2.a
+                     ->  Seq Scan on public.prt2_l_p1_p2
+                           Output: prt2_l_p1_p2.b, prt2_l_p1_p2.c, prt2_l_p1_p2.a
+                           Filter: ((prt2_l_p1_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_l_p2_p1.a, prt1_l_p2_p1.c, prt2_l_p2_p1.b, prt2_l_p2_p1.c
+               Hash Cond: ((prt1_l_p2_p1.a = prt2_l_p2_p1.b) AND (prt1_l_p2_p1.b = prt2_l_p2_p1.a) AND ((prt1_l_p2_p1.c)::text = (prt2_l_p2_p1.c)::text) AND ((prt1_l_p2_p1.b + prt1_l_p2_p1.a) = (prt2_l_p2_p1.a + prt2_l_p2_p1.b)))
+               ->  Seq Scan on public.prt1_l_p2_p1
+                     Output: prt1_l_p2_p1.a, prt1_l_p2_p1.c, prt1_l_p2_p1.b
+                     Filter: ((prt1_l_p2_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p2_p1.b, prt2_l_p2_p1.c, prt2_l_p2_p1.a
+                     ->  Seq Scan on public.prt2_l_p2_p1
+                           Output: prt2_l_p2_p1.b, prt2_l_p2_p1.c, prt2_l_p2_p1.a
+                           Filter: ((prt2_l_p2_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_l_p2_p2.a, prt1_l_p2_p2.c, prt2_l_p2_p2.b, prt2_l_p2_p2.c
+               Hash Cond: ((prt1_l_p2_p2.a = prt2_l_p2_p2.b) AND (prt1_l_p2_p2.b = prt2_l_p2_p2.a) AND ((prt1_l_p2_p2.c)::text = (prt2_l_p2_p2.c)::text) AND ((prt1_l_p2_p2.b + prt1_l_p2_p2.a) = (prt2_l_p2_p2.a + prt2_l_p2_p2.b)))
+               ->  Seq Scan on public.prt1_l_p2_p2
+                     Output: prt1_l_p2_p2.a, prt1_l_p2_p2.c, prt1_l_p2_p2.b
+                     Filter: ((prt1_l_p2_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p2_p2.b, prt2_l_p2_p2.c, prt2_l_p2_p2.a
+                     ->  Seq Scan on public.prt2_l_p2_p2
+                           Output: prt2_l_p2_p2.b, prt2_l_p2_p2.c, prt2_l_p2_p2.a
+                           Filter: ((prt2_l_p2_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_l_p3_p1.a, prt1_l_p3_p1.c, prt2_l_p3_p1.b, prt2_l_p3_p1.c
+               Hash Cond: ((prt1_l_p3_p1.a = prt2_l_p3_p1.b) AND (prt1_l_p3_p1.b = prt2_l_p3_p1.a) AND ((prt1_l_p3_p1.c)::text = (prt2_l_p3_p1.c)::text) AND ((prt1_l_p3_p1.b + prt1_l_p3_p1.a) = (prt2_l_p3_p1.a + prt2_l_p3_p1.b)))
+               ->  Seq Scan on public.prt1_l_p3_p1
+                     Output: prt1_l_p3_p1.a, prt1_l_p3_p1.c, prt1_l_p3_p1.b
+                     Filter: ((prt1_l_p3_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p3_p1.b, prt2_l_p3_p1.c, prt2_l_p3_p1.a
+                     ->  Seq Scan on public.prt2_l_p3_p1
+                           Output: prt2_l_p3_p1.b, prt2_l_p3_p1.c, prt2_l_p3_p1.a
+                           Filter: ((prt2_l_p3_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_l_p3_p2.a, prt1_l_p3_p2.c, prt2_l_p3_p2.b, prt2_l_p3_p2.c
+               Hash Cond: ((prt1_l_p3_p2.a = prt2_l_p3_p2.b) AND (prt1_l_p3_p2.b = prt2_l_p3_p2.a) AND ((prt1_l_p3_p2.c)::text = (prt2_l_p3_p2.c)::text) AND ((prt1_l_p3_p2.b + prt1_l_p3_p2.a) = (prt2_l_p3_p2.a + prt2_l_p3_p2.b)))
+               ->  Seq Scan on public.prt1_l_p3_p2
+                     Output: prt1_l_p3_p2.a, prt1_l_p3_p2.c, prt1_l_p3_p2.b
+                     Filter: ((prt1_l_p3_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p3_p2.b, prt2_l_p3_p2.c, prt2_l_p3_p2.a
+                     ->  Seq Scan on public.prt2_l_p3_p2
+                           Output: prt2_l_p3_p2.b, prt2_l_p3_p2.c, prt2_l_p3_p2.a
+                           Filter: ((prt2_l_p3_p2.b % 25) = 0)
+(70 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 7) and (b 20) is considered for partition-wise join.
+NOTICE:  join between relations (b 11) and (b 24) is considered for partition-wise join.
+NOTICE:  join between relations (b 15) and (b 28) is considered for partition-wise join.
+  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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_l t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_l t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.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)
+
+--
+-- 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;
+ANALYZE plt1_p1;
+ANALYZE plt1_p2;
+ANALYZE plt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1 AS SELECT * FROM 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;
+ANALYZE plt2_p1;
+ANALYZE plt2_p2;
+ANALYZE plt2_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt2 AS SELECT * FROM plt2;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.a = t2.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((t2.c = t1.c) AND (t2.a = t1.a))
+               ->  Seq Scan on public.plt2_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.plt1_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((t2_1.c = t1_1.c) AND (t2_1.a = t1_1.a))
+               ->  Seq Scan on public.plt2_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.plt1_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((t2_2.c = t1_2.c) AND (t2_2.a = t1_2.a))
+               ->  Seq Scan on public.plt2_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.plt1_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.a = t2.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0003 | 150 | 0003
+ 300 | 0006 | 300 | 0006
+ 450 | 0009 | 450 | 0009
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1 t1, uplt2 t2 WHERE t1.c = t2.c AND t1.a = t2.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0003 | 150 | 0003
+ 300 | 0006 | 300 | 0006
+ 450 | 0009 | 450 | 0009
+(4 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.a AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+               ->  Seq Scan on public.plt2_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.plt1_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on public.plt2_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.plt1_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on public.plt2_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.plt1_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.a AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  50 | 0001 |     | 
+ 100 | 0002 |     | 
+ 150 | 0003 | 150 | 0003
+ 200 | 0004 |     | 
+ 250 | 0005 |     | 
+ 300 | 0006 | 300 | 0006
+ 350 | 0007 |     | 
+ 400 | 0008 |     | 
+ 450 | 0009 | 450 | 0009
+ 500 | 0010 |     | 
+ 550 | 0011 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.a AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  50 | 0001 |     | 
+ 100 | 0002 |     | 
+ 150 | 0003 | 150 | 0003
+ 200 | 0004 |     | 
+ 250 | 0005 |     | 
+ 300 | 0006 | 300 | 0006
+ 350 | 0007 |     | 
+ 400 | 0008 |     | 
+ 450 | 0009 | 450 | 0009
+ 500 | 0010 |     | 
+ 550 | 0011 |     | 
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+                                QUERY PLAN                                
+--------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: ((t1.a = t2.b) AND (t1.c = t2.c))
+                     ->  Seq Scan on public.plt1_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t2.b, t2.c
+                           ->  Seq Scan on public.plt2_p1 t2
+                                 Output: t2.b, t2.c
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: ((t1_1.a = t2_1.b) AND (t1_1.c = t2_1.c))
+                     ->  Seq Scan on public.plt1_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c
+                           ->  Seq Scan on public.plt2_p2 t2_1
+                                 Output: t2_1.b, t2_1.c
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: ((t1_2.a = t2_2.b) AND (t1_2.c = t2_2.c))
+                     ->  Seq Scan on public.plt1_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c
+                           ->  Seq Scan on public.plt2_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+                                 Filter: ((t2_2.b % 25) = 0)
+(36 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0003 | 150 | 0003
+ 300 | 0006 | 300 | 0006
+ 450 | 0009 | 450 | 0009
+     |      |  75 | 0001
+     |      | 225 | 0004
+     |      | 375 | 0007
+     |      | 525 | 0010
+(8 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0003 | 150 | 0003
+ 300 | 0006 | 300 | 0006
+ 450 | 0009 | 450 | 0009
+     |      |  75 | 0001
+     |      | 225 | 0004
+     |      | 375 | 0007
+     |      | 525 | 0010
+(8 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                   QUERY PLAN                                   
+--------------------------------------------------------------------------------
+ Sort
+   Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+   Sort Key: plt1_p1.a, plt2_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+               Hash Cond: ((plt1_p1.a = plt2_p1.b) AND (plt1_p1.c = plt2_p1.c))
+               ->  Seq Scan on public.plt1_p1
+                     Output: plt1_p1.a, plt1_p1.c
+                     Filter: ((plt1_p1.a % 25) = 0)
+               ->  Hash
+                     Output: plt2_p1.b, plt2_p1.c
+                     ->  Seq Scan on public.plt2_p1
+                           Output: plt2_p1.b, plt2_p1.c
+                           Filter: ((plt2_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+               Hash Cond: ((plt1_p2.a = plt2_p2.b) AND (plt1_p2.c = plt2_p2.c))
+               ->  Seq Scan on public.plt1_p2
+                     Output: plt1_p2.a, plt1_p2.c
+                     Filter: ((plt1_p2.a % 25) = 0)
+               ->  Hash
+                     Output: plt2_p2.b, plt2_p2.c
+                     ->  Seq Scan on public.plt2_p2
+                           Output: plt2_p2.b, plt2_p2.c
+                           Filter: ((plt2_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+               Hash Cond: ((plt1_p3.a = plt2_p3.b) AND (plt1_p3.c = plt2_p3.c))
+               ->  Seq Scan on public.plt1_p3
+                     Output: plt1_p3.a, plt1_p3.c
+                     Filter: ((plt1_p3.a % 25) = 0)
+               ->  Hash
+                     Output: plt2_p3.b, plt2_p3.c
+                     ->  Seq Scan on public.plt2_p3
+                           Output: plt2_p3.b, plt2_p3.c
+                           Filter: ((plt2_p3.b % 25) = 0)
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  50 | 0001 |     | 
+ 100 | 0002 |     | 
+ 150 | 0003 | 150 | 0003
+ 200 | 0004 |     | 
+ 250 | 0005 |     | 
+ 300 | 0006 | 300 | 0006
+ 350 | 0007 |     | 
+ 400 | 0008 |     | 
+ 450 | 0009 | 450 | 0009
+ 500 | 0010 |     | 
+ 550 | 0011 |     | 
+     |      |  75 | 0001
+     |      | 225 | 0004
+     |      | 375 | 0007
+     |      | 525 | 0010
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uplt1 t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  50 | 0001 |     | 
+ 100 | 0002 |     | 
+ 150 | 0003 | 150 | 0003
+ 200 | 0004 |     | 
+ 250 | 0005 |     | 
+ 300 | 0006 | 300 | 0006
+ 350 | 0007 |     | 
+ 400 | 0008 |     | 
+ 450 | 0009 | 450 | 0009
+ 500 | 0010 |     | 
+ 550 | 0011 |     | 
+     |      |  75 | 0001
+     |      | 225 | 0004
+     |      | 375 | 0007
+     |      | 525 | 0010
+(16 rows)
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM (SELECT 50 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ GroupAggregate
+   Output: sum(plt1_p1.a), plt1_p1.c, avg(plt2_p1.b), plt2_p1.c
+   Group Key: plt1_p1.c, plt2_p1.c
+   ->  Sort
+         Output: plt1_p1.c, plt2_p1.c, plt1_p1.a, plt2_p1.b
+         Sort Key: plt1_p1.c, plt2_p1.c
+         ->  Result
+               Output: plt1_p1.c, plt2_p1.c, plt1_p1.a, plt2_p1.b
+               ->  Append
+                     ->  Hash Full Join
+                           Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+                           Hash Cond: ((plt1_p1.c = plt2_p1.c) AND (plt1_p1.a = plt2_p1.b))
+                           ->  Seq Scan on public.plt1_p1
+                                 Output: plt1_p1.a, plt1_p1.c
+                                 Filter: ((plt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p1.b, plt2_p1.c
+                                 ->  Seq Scan on public.plt2_p1
+                                       Output: plt2_p1.b, plt2_p1.c
+                                       Filter: ((plt2_p1.b % 25) = 0)
+                     ->  Hash Full Join
+                           Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+                           Hash Cond: ((plt1_p2.c = plt2_p2.c) AND (plt1_p2.a = plt2_p2.b))
+                           ->  Seq Scan on public.plt1_p2
+                                 Output: plt1_p2.a, plt1_p2.c
+                                 Filter: ((plt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p2.b, plt2_p2.c
+                                 ->  Seq Scan on public.plt2_p2
+                                       Output: plt2_p2.b, plt2_p2.c
+                                       Filter: ((plt2_p2.b % 25) = 0)
+                     ->  Hash Full Join
+                           Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+                           Hash Cond: ((plt1_p3.c = plt2_p3.c) AND (plt1_p3.a = plt2_p3.b))
+                           ->  Seq Scan on public.plt1_p3
+                                 Output: plt1_p3.a, plt1_p3.c
+                                 Filter: ((plt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p3.b, plt2_p3.c
+                                 ->  Seq Scan on public.plt2_p3
+                                       Output: plt2_p3.b, plt2_p3.c
+                                       Filter: ((plt2_p3.b % 25) = 0)
+(42 rows)
+
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM (SELECT 50 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+ sum |  c   |          avg           |  c   
+-----+------+------------------------+------
+   0 | 0000 | 0.00000000000000000000 | 0000
+  50 | 0001 |                        | 
+ 100 | 0002 |                        | 
+ 150 | 0003 |   150.0000000000000000 | 0003
+ 200 | 0004 |                        | 
+ 250 | 0005 |                        | 
+ 300 | 0006 |   300.0000000000000000 | 0006
+ 350 | 0007 |                        | 
+ 400 | 0008 |                        | 
+ 450 | 0009 |   450.0000000000000000 | 0009
+ 500 | 0010 |                        | 
+ 550 | 0011 |                        | 
+     |      |    75.0000000000000000 | 0001
+     |      |   225.0000000000000000 | 0004
+     |      |   375.0000000000000000 | 0007
+     |      |   525.0000000000000000 | 0010
+(16 rows)
+
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM (SELECT 50 phv, * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+ sum |  c   |          avg           |  c   
+-----+------+------------------------+------
+   0 | 0000 | 0.00000000000000000000 | 0000
+  50 | 0001 |                        | 
+ 100 | 0002 |                        | 
+ 150 | 0003 |   150.0000000000000000 | 0003
+ 200 | 0004 |                        | 
+ 250 | 0005 |                        | 
+ 300 | 0006 |   300.0000000000000000 | 0006
+ 350 | 0007 |                        | 
+ 400 | 0008 |                        | 
+ 450 | 0009 |   450.0000000000000000 | 0009
+ 500 | 0010 |                        | 
+ 550 | 0011 |                        | 
+     |      |    75.0000000000000000 | 0001
+     |      |   225.0000000000000000 | 0004
+     |      |   375.0000000000000000 | 0007
+     |      |   525.0000000000000000 | 0010
+(16 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t1.phv), avg(t2.b), t2.c, avg(t2.phv) FROM (SELECT 25 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ GroupAggregate
+   Output: sum(plt1_p1.a), plt1_p1.c, sum((25)), avg(plt2_p1.b), plt2_p1.c, avg((50))
+   Group Key: plt1_p1.c, plt2_p1.c
+   ->  Sort
+         Output: plt1_p1.c, plt2_p1.c, plt1_p1.a, (25), plt2_p1.b, (50)
+         Sort Key: plt1_p1.c, plt2_p1.c
+         ->  Result
+               Output: plt1_p1.c, plt2_p1.c, plt1_p1.a, (25), plt2_p1.b, (50)
+               ->  Append
+                     ->  Hash Full Join
+                           Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, (25), (50)
+                           Hash Cond: ((plt1_p1.c = plt2_p1.c) AND (plt1_p1.a = plt2_p1.b))
+                           ->  Seq Scan on public.plt1_p1
+                                 Output: plt1_p1.a, plt1_p1.c, 25
+                                 Filter: ((plt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p1.b, plt2_p1.c, (50)
+                                 ->  Seq Scan on public.plt2_p1
+                                       Output: plt2_p1.b, plt2_p1.c, 50
+                                       Filter: ((plt2_p1.b % 25) = 0)
+                     ->  Hash Full Join
+                           Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c, (25), (50)
+                           Hash Cond: ((plt1_p2.c = plt2_p2.c) AND (plt1_p2.a = plt2_p2.b))
+                           ->  Seq Scan on public.plt1_p2
+                                 Output: plt1_p2.a, plt1_p2.c, 25
+                                 Filter: ((plt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p2.b, plt2_p2.c, (50)
+                                 ->  Seq Scan on public.plt2_p2
+                                       Output: plt2_p2.b, plt2_p2.c, 50
+                                       Filter: ((plt2_p2.b % 25) = 0)
+                     ->  Hash Full Join
+                           Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c, (25), (50)
+                           Hash Cond: ((plt1_p3.c = plt2_p3.c) AND (plt1_p3.a = plt2_p3.b))
+                           ->  Seq Scan on public.plt1_p3
+                                 Output: plt1_p3.a, plt1_p3.c, 25
+                                 Filter: ((plt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p3.b, plt2_p3.c, (50)
+                                 ->  Seq Scan on public.plt2_p3
+                                       Output: plt2_p3.b, plt2_p3.c, 50
+                                       Filter: ((plt2_p3.b % 25) = 0)
+(42 rows)
+
+SELECT sum(t1.a), t1.c, sum(t1.phv), avg(t2.b), t2.c, avg(t2.phv) FROM (SELECT 25 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+ sum |  c   | sum |          avg           |  c   |         avg         
+-----+------+-----+------------------------+------+---------------------
+   0 | 0000 |  25 | 0.00000000000000000000 | 0000 | 50.0000000000000000
+  50 | 0001 |  25 |                        |      |                    
+ 100 | 0002 |  25 |                        |      |                    
+ 150 | 0003 |  25 |   150.0000000000000000 | 0003 | 50.0000000000000000
+ 200 | 0004 |  25 |                        |      |                    
+ 250 | 0005 |  25 |                        |      |                    
+ 300 | 0006 |  25 |   300.0000000000000000 | 0006 | 50.0000000000000000
+ 350 | 0007 |  25 |                        |      |                    
+ 400 | 0008 |  25 |                        |      |                    
+ 450 | 0009 |  25 |   450.0000000000000000 | 0009 | 50.0000000000000000
+ 500 | 0010 |  25 |                        |      |                    
+ 550 | 0011 |  25 |                        |      |                    
+     |      |     |    75.0000000000000000 | 0001 | 50.0000000000000000
+     |      |     |   225.0000000000000000 | 0004 | 50.0000000000000000
+     |      |     |   375.0000000000000000 | 0007 | 50.0000000000000000
+     |      |     |   525.0000000000000000 | 0010 | 50.0000000000000000
+(16 rows)
+
+SELECT sum(t1.a), t1.c, sum(t1.phv), avg(t2.b), t2.c, avg(t2.phv) FROM (SELECT 25 phv, * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+ sum |  c   | sum |          avg           |  c   |         avg         
+-----+------+-----+------------------------+------+---------------------
+   0 | 0000 |  25 | 0.00000000000000000000 | 0000 | 50.0000000000000000
+  50 | 0001 |  25 |                        |      |                    
+ 100 | 0002 |  25 |                        |      |                    
+ 150 | 0003 |  25 |   150.0000000000000000 | 0003 | 50.0000000000000000
+ 200 | 0004 |  25 |                        |      |                    
+ 250 | 0005 |  25 |                        |      |                    
+ 300 | 0006 |  25 |   300.0000000000000000 | 0006 | 50.0000000000000000
+ 350 | 0007 |  25 |                        |      |                    
+ 400 | 0008 |  25 |                        |      |                    
+ 450 | 0009 |  25 |   450.0000000000000000 | 0009 | 50.0000000000000000
+ 500 | 0010 |  25 |                        |      |                    
+ 550 | 0011 |  25 |                        |      |                    
+     |      |     |    75.0000000000000000 | 0001 | 50.0000000000000000
+     |      |     |   225.0000000000000000 | 0004 | 50.0000000000000000
+     |      |     |   375.0000000000000000 | 0007 | 50.0000000000000000
+     |      |     |   525.0000000000000000 | 0010 | 50.0000000000000000
+(16 rows)
+
+-- Join with pruned partitions from joining relations
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.c NOT IN ('0001', '0005', '0002', '0009') AND t2.c NOT IN ('0000', '0003', '0004', '0010') GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Output: (sum(t1.a)), t1.c, (avg(t2.b)), t2.c
+   Sort Key: t1.c
+   ->  HashAggregate
+         Output: sum(t1.a), t1.c, avg(t2.b), t2.c
+         Group Key: t1.c, t2.c
+         ->  Result
+               Output: t1.c, t2.c, t1.a, t2.b
+               ->  Append
+                     ->  Hash Join
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           Hash Cond: (t1.c = t2.c)
+                           ->  Seq Scan on public.plt1_p3 t1
+                                 Output: t1.a, t1.c
+                                 Filter: (t1.c <> ALL ('{0001,0005,0002,0009}'::text[]))
+                           ->  Hash
+                                 Output: t2.b, t2.c
+                                 ->  Seq Scan on public.plt2_p3 t2
+                                       Output: t2.b, t2.c
+                                       Filter: (t2.c <> ALL ('{0000,0003,0004,0010}'::text[]))
+(20 rows)
+
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.c NOT IN ('0001', '0005', '0002', '0009') AND t2.c NOT IN ('0000', '0003', '0004', '0010') GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+  sum   |  c   |         avg          |  c   
+--------+------+----------------------+------
+ 137700 | 0006 | 324.0000000000000000 | 0006
+ 158950 | 0007 | 375.0000000000000000 | 0007
+ 169600 | 0008 | 424.5000000000000000 | 0008
+ 229600 | 0011 | 574.5000000000000000 | 0011
+(4 rows)
+
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM uplt1 t1, uplt2 t2 WHERE t1.c = t2.c AND t1.c NOT IN ('0001', '0005', '0002', '0009') AND t2.c NOT IN ('0000', '0003', '0004', '0010') GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+  sum   |  c   |         avg          |  c   
+--------+------+----------------------+------
+ 137700 | 0006 | 324.0000000000000000 | 0006
+ 158950 | 0007 | 375.0000000000000000 | 0007
+ 169600 | 0008 | 424.5000000000000000 | 0008
+ 229600 | 0011 | 574.5000000000000000 | 0011
+(4 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 LEFT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Output: (sum(t1.a)), t1.c, (sum(b)), c
+   Sort Key: t1.c, c
+   ->  HashAggregate
+         Output: sum(t1.a), t1.c, sum(b), c
+         Group Key: t1.c, c
+         ->  Result
+               Output: t1.c, c, t1.a, b
+               ->  Append
+                     ->  Hash Left Join
+                           Output: t1.a, t1.c, b, c
+                           Hash Cond: (t1.c = c)
+                           ->  Seq Scan on public.plt1_p1 t1
+                                 Output: t1.a, t1.c
+                                 Filter: (t1.c <> ALL ('{0001,0005,0002,0009}'::text[]))
+                           ->  Hash
+                                 Output: b, c
+                                 ->  Result
+                                       Output: b, c
+                                       One-Time Filter: false
+                     ->  Hash Left Join
+                           Output: t1_1.a, t1_1.c, t2.b, t2.c
+                           Hash Cond: (t1_1.c = t2.c)
+                           ->  Seq Scan on public.plt1_p3 t1_1
+                                 Output: t1_1.a, t1_1.c
+                                 Filter: (t1_1.c <> ALL ('{0001,0005,0002,0009}'::text[]))
+                           ->  Hash
+                                 Output: t2.b, t2.c
+                                 ->  Seq Scan on public.plt2_p3 t2
+                                       Output: t2.b, t2.c
+                                       Filter: (t2.c <> ALL ('{0000,0003,0004,0010}'::text[]))
+(31 rows)
+
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 LEFT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  sum   |  c   |  sum   |  c   
+--------+------+--------+------
+    600 | 0000 |        | 
+   4350 | 0003 |        | 
+   5600 | 0004 |        | 
+ 137700 | 0006 | 137700 | 0006
+ 158950 | 0007 | 159375 | 0007
+ 169600 | 0008 | 169800 | 0008
+  13100 | 0010 |        | 
+ 229600 | 0011 | 229800 | 0011
+(8 rows)
+
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM uplt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 LEFT JOIN (SELECT * FROM uplt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+  sum   |  c   |  sum   |  c   
+--------+------+--------+------
+    600 | 0000 |        | 
+   4350 | 0003 |        | 
+   5600 | 0004 |        | 
+ 137700 | 0006 | 137700 | 0006
+ 158950 | 0007 | 159375 | 0007
+ 169600 | 0008 | 169800 | 0008
+  13100 | 0010 |        | 
+ 229600 | 0011 | 229800 | 0011
+(8 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 RIGHT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+NOTICE:  join between relations (b 5) and (b 4) is considered for partition-wise join.
+                                           QUERY PLAN                                            
+-------------------------------------------------------------------------------------------------
+ Sort
+   Output: (sum(a)), c, (sum(t2.b)), t2.c
+   Sort Key: c, t2.c
+   ->  HashAggregate
+         Output: sum(a), c, sum(t2.b), t2.c
+         Group Key: c, t2.c
+         ->  Result
+               Output: c, t2.c, a, t2.b
+               ->  Append
+                     ->  Hash Left Join
+                           Output: t2.b, t2.c, a, c
+                           Hash Cond: (t2.c = c)
+                           ->  Seq Scan on public.plt2_p2 t2
+                                 Output: t2.b, t2.c
+                                 Filter: (t2.c <> ALL ('{0000,0003,0004,0010}'::text[]))
+                           ->  Hash
+                                 Output: a, c
+                                 ->  Result
+                                       Output: a, c
+                                       One-Time Filter: false
+                     ->  Hash Right Join
+                           Output: t2_1.b, t2_1.c, t1.a, t1.c
+                           Hash Cond: (t1.c = t2_1.c)
+                           ->  Seq Scan on public.plt1_p3 t1
+                                 Output: t1.a, t1.c
+                                 Filter: (t1.c <> ALL ('{0001,0005,0002,0009}'::text[]))
+                           ->  Hash
+                                 Output: t2_1.b, t2_1.c
+                                 ->  Seq Scan on public.plt2_p3 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                       Filter: (t2_1.c <> ALL ('{0000,0003,0004,0010}'::text[]))
+(31 rows)
+
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 RIGHT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+NOTICE:  join between relations (b 5) and (b 4) is considered for partition-wise join.
+  sum   |  c   |  sum   |  c   
+--------+------+--------+------
+ 137700 | 0006 | 137700 | 0006
+ 158950 | 0007 | 159375 | 0007
+ 169600 | 0008 | 169800 | 0008
+ 229600 | 0011 | 229800 | 0011
+        |      |   1275 | 0001
+        |      |   1992 | 0002
+        |      |   4392 | 0005
+        |      |   8058 | 0009
+(8 rows)
+
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM uplt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 RIGHT JOIN (SELECT * FROM uplt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+  sum   |  c   |  sum   |  c   
+--------+------+--------+------
+ 137700 | 0006 | 137700 | 0006
+ 158950 | 0007 | 159375 | 0007
+ 169600 | 0008 | 169800 | 0008
+ 229600 | 0011 | 229800 | 0011
+        |      |   1275 | 0001
+        |      |   1992 | 0002
+        |      |   4392 | 0005
+        |      |   8058 | 0009
+(8 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 FULL JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                           QUERY PLAN                                            
+-------------------------------------------------------------------------------------------------
+ Sort
+   Output: (sum(t1.a)), t1.c, (sum(b)), c
+   Sort Key: t1.c, c
+   ->  HashAggregate
+         Output: sum(t1.a), t1.c, sum(b), c
+         Group Key: t1.c, c
+         ->  Result
+               Output: t1.c, c, t1.a, b
+               ->  Append
+                     ->  Hash Full Join
+                           Output: t1.a, t1.c, b, c
+                           Hash Cond: (t1.c = c)
+                           ->  Seq Scan on public.plt1_p1 t1
+                                 Output: t1.a, t1.c
+                                 Filter: (t1.c <> ALL ('{0001,0005,0002,0009}'::text[]))
+                           ->  Hash
+                                 Output: b, c
+                                 ->  Result
+                                       Output: b, c
+                                       One-Time Filter: false
+                     ->  Hash Full Join
+                           Output: a, c, t2.b, t2.c
+                           Hash Cond: (t2.c = c)
+                           ->  Seq Scan on public.plt2_p2 t2
+                                 Output: t2.b, t2.c
+                                 Filter: (t2.c <> ALL ('{0000,0003,0004,0010}'::text[]))
+                           ->  Hash
+                                 Output: a, c
+                                 ->  Result
+                                       Output: a, c
+                                       One-Time Filter: false
+                     ->  Hash Full Join
+                           Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                           Hash Cond: (t1_1.c = t2_1.c)
+                           ->  Seq Scan on public.plt1_p3 t1_1
+                                 Output: t1_1.a, t1_1.c
+                                 Filter: (t1_1.c <> ALL ('{0001,0005,0002,0009}'::text[]))
+                           ->  Hash
+                                 Output: t2_1.b, t2_1.c
+                                 ->  Seq Scan on public.plt2_p3 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                       Filter: (t2_1.c <> ALL ('{0000,0003,0004,0010}'::text[]))
+(42 rows)
+
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 FULL JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  sum   |  c   |  sum   |  c   
+--------+------+--------+------
+    600 | 0000 |        | 
+   4350 | 0003 |        | 
+   5600 | 0004 |        | 
+ 137700 | 0006 | 137700 | 0006
+ 158950 | 0007 | 159375 | 0007
+ 169600 | 0008 | 169800 | 0008
+  13100 | 0010 |        | 
+ 229600 | 0011 | 229800 | 0011
+        |      |   1275 | 0001
+        |      |   1992 | 0002
+        |      |   4392 | 0005
+        |      |   8058 | 0009
+(12 rows)
+
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM uplt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 FULL JOIN (SELECT * FROM uplt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+  sum   |  c   |  sum   |  c   
+--------+------+--------+------
+    600 | 0000 |        | 
+   4350 | 0003 |        | 
+   5600 | 0004 |        | 
+ 137700 | 0006 | 137700 | 0006
+ 158950 | 0007 | 159375 | 0007
+ 169600 | 0008 | 169800 | 0008
+  13100 | 0010 |        | 
+ 229600 | 0011 | 229800 | 0011
+        |      |   1275 | 0001
+        |      |   1992 | 0002
+        |      |   4392 | 0005
+        |      |   8058 | 0009
+(12 rows)
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 1) and (b 3) is considered for partition-wise join.
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.c = t1_3.c)
+               ->  Seq Scan on public.plt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_3.c
+                     ->  Seq Scan on public.plt2_p1 t1_3
+                           Output: t1_3.c
+                           Filter: ((t1_3.b % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.c = t1_4.c)
+               ->  Seq Scan on public.plt1_p2 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_4.c
+                     ->  Seq Scan on public.plt2_p2 t1_4
+                           Output: t1_4.c
+                           Filter: ((t1_4.b % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.c = t1_5.c)
+               ->  Seq Scan on public.plt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_5.c
+                     ->  Seq Scan on public.plt2_p3 t1_5
+                           Output: t1_5.c
+                           Filter: ((t1_5.b % 25) = 0)
+(37 rows)
+
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 1) and (b 3) is considered for partition-wise join.
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+(8 rows)
+
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+(8 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;
+ANALYZE plt1_e;
+ANALYZE plt1_e_p1;
+ANALYZE plt1_e_p2;
+ANALYZE plt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1_e AS SELECT * FROM plt1_e;
+CREATE TABLE plt2_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
+CREATE TABLE plt2_e_p1 PARTITION OF plt2_e FOR VALUES IN ('0000', '0003', '0004', '0010');
+CREATE TABLE plt2_e_p2 PARTITION OF plt2_e FOR VALUES IN ('0001', '0005', '0002', '0009');
+CREATE TABLE plt2_e_p3 PARTITION OF plt2_e FOR VALUES IN ('0006', '0007', '0008', '0011');
+INSERT INTO plt2_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE plt2_e;
+ANALYZE plt2_e_p1;
+ANALYZE plt2_e_p2;
+ANALYZE plt2_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt2_e AS SELECT * FROM plt2_e;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1, plt2_e t2 WHERE t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((t2.b = t1.a) AND (ltrim(t2.c, 'A'::text) = ltrim(t1.c, 'A'::text)))
+               ->  Seq Scan on public.plt2_e_p1 t2
+                     Output: t2.b, t2.c
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.plt1_e_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((t2_1.b = t1_1.a) AND (ltrim(t2_1.c, 'A'::text) = ltrim(t1_1.c, 'A'::text)))
+               ->  Seq Scan on public.plt2_e_p2 t2_1
+                     Output: t2_1.b, t2_1.c
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.plt1_e_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((t2_2.b = t1_2.a) AND (ltrim(t2_2.c, 'A'::text) = ltrim(t1_2.c, 'A'::text)))
+               ->  Seq Scan on public.plt2_e_p3 t2_2
+                     Output: t2_2.b, t2_2.c
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.plt1_e_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1, plt2_e t2 WHERE t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+ 150 | A0003 | 150 | A0003
+ 300 | A0006 | 300 | A0006
+ 450 | A0009 | 450 | A0009
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_e t1, uplt2_e t2 WHERE t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+ 150 | A0003 | 150 | A0003
+ 300 | A0006 | 300 | A0006
+ 450 | A0009 | 450 | A0009
+(4 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 LEFT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((t2.b = t1.a) AND (ltrim(t2.c, 'A'::text) = ltrim(t1.c, 'A'::text)))
+               ->  Seq Scan on public.plt2_e_p1 t2
+                     Output: t2.b, t2.c
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.plt1_e_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((t2_1.b = t1_1.a) AND (ltrim(t2_1.c, 'A'::text) = ltrim(t1_1.c, 'A'::text)))
+               ->  Seq Scan on public.plt2_e_p2 t2_1
+                     Output: t2_1.b, t2_1.c
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.plt1_e_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((t2_2.b = t1_2.a) AND (ltrim(t2_2.c, 'A'::text) = ltrim(t1_2.c, 'A'::text)))
+               ->  Seq Scan on public.plt2_e_p3 t2_2
+                     Output: t2_2.b, t2_2.c
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.plt1_e_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 LEFT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+  50 | A0001 |     | 
+ 100 | A0002 |     | 
+ 150 | A0003 | 150 | A0003
+ 200 | A0004 |     | 
+ 250 | A0005 |     | 
+ 300 | A0006 | 300 | A0006
+ 350 | A0007 |     | 
+ 400 | A0008 |     | 
+ 450 | A0009 | 450 | A0009
+ 500 | A0010 |     | 
+ 550 | A0011 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_e t1 LEFT JOIN uplt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+  50 | A0001 |     | 
+ 100 | A0002 |     | 
+ 150 | A0003 | 150 | A0003
+ 200 | A0004 |     | 
+ 250 | A0005 |     | 
+ 300 | A0006 | 300 | A0006
+ 350 | A0007 |     | 
+ 400 | A0008 |     | 
+ 450 | A0009 | 450 | A0009
+ 500 | A0010 |     | 
+ 550 | A0011 |     | 
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 RIGHT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+                                                  QUERY PLAN                                                  
+--------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: ((t1.a = t2.b) AND (ltrim(t1.c, 'A'::text) = ltrim(t2.c, 'A'::text)))
+                     ->  Seq Scan on public.plt1_e_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t2.b, t2.c
+                           ->  Seq Scan on public.plt2_e_p1 t2
+                                 Output: t2.b, t2.c
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: ((t1_1.a = t2_1.b) AND (ltrim(t1_1.c, 'A'::text) = ltrim(t2_1.c, 'A'::text)))
+                     ->  Seq Scan on public.plt1_e_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c
+                           ->  Seq Scan on public.plt2_e_p2 t2_1
+                                 Output: t2_1.b, t2_1.c
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: ((t1_2.a = t2_2.b) AND (ltrim(t1_2.c, 'A'::text) = ltrim(t2_2.c, 'A'::text)))
+                     ->  Seq Scan on public.plt1_e_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c
+                           ->  Seq Scan on public.plt2_e_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+                                 Filter: ((t2_2.b % 25) = 0)
+(36 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 RIGHT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+ 150 | A0003 | 150 | A0003
+ 300 | A0006 | 300 | A0006
+ 450 | A0009 | 450 | A0009
+     |       |  75 | A0001
+     |       | 225 | A0004
+     |       | 375 | A0007
+     |       | 525 | A0010
+(8 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_e t1 RIGHT JOIN uplt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+ 150 | A0003 | 150 | A0003
+ 300 | A0006 | 300 | A0006
+ 450 | A0009 | 450 | A0009
+     |       |  75 | A0001
+     |       | 225 | A0004
+     |       | 375 | A0007
+     |       | 525 | A0010
+(8 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2_e WHERE plt2_e.b % 25 = 0) t2 ON (t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A')) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                                         QUERY PLAN                                                         
+----------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: plt1_e_p1.a, plt1_e_p1.c, plt2_e_p1.b, plt2_e_p1.c
+   Sort Key: plt1_e_p1.a, plt2_e_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: plt1_e_p1.a, plt1_e_p1.c, plt2_e_p1.b, plt2_e_p1.c
+               Hash Cond: ((plt1_e_p1.a = plt2_e_p1.b) AND (ltrim(plt1_e_p1.c, 'A'::text) = ltrim(plt2_e_p1.c, 'A'::text)))
+               ->  Seq Scan on public.plt1_e_p1
+                     Output: plt1_e_p1.a, plt1_e_p1.c
+                     Filter: ((plt1_e_p1.a % 25) = 0)
+               ->  Hash
+                     Output: plt2_e_p1.b, plt2_e_p1.c
+                     ->  Seq Scan on public.plt2_e_p1
+                           Output: plt2_e_p1.b, plt2_e_p1.c
+                           Filter: ((plt2_e_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: plt1_e_p2.a, plt1_e_p2.c, plt2_e_p2.b, plt2_e_p2.c
+               Hash Cond: ((plt1_e_p2.a = plt2_e_p2.b) AND (ltrim(plt1_e_p2.c, 'A'::text) = ltrim(plt2_e_p2.c, 'A'::text)))
+               ->  Seq Scan on public.plt1_e_p2
+                     Output: plt1_e_p2.a, plt1_e_p2.c
+                     Filter: ((plt1_e_p2.a % 25) = 0)
+               ->  Hash
+                     Output: plt2_e_p2.b, plt2_e_p2.c
+                     ->  Seq Scan on public.plt2_e_p2
+                           Output: plt2_e_p2.b, plt2_e_p2.c
+                           Filter: ((plt2_e_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: plt1_e_p3.a, plt1_e_p3.c, plt2_e_p3.b, plt2_e_p3.c
+               Hash Cond: ((plt1_e_p3.a = plt2_e_p3.b) AND (ltrim(plt1_e_p3.c, 'A'::text) = ltrim(plt2_e_p3.c, 'A'::text)))
+               ->  Seq Scan on public.plt1_e_p3
+                     Output: plt1_e_p3.a, plt1_e_p3.c
+                     Filter: ((plt1_e_p3.a % 25) = 0)
+               ->  Hash
+                     Output: plt2_e_p3.b, plt2_e_p3.c
+                     ->  Seq Scan on public.plt2_e_p3
+                           Output: plt2_e_p3.b, plt2_e_p3.c
+                           Filter: ((plt2_e_p3.b % 25) = 0)
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2_e WHERE plt2_e.b % 25 = 0) t2 ON (t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A')) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+  50 | A0001 |     | 
+ 100 | A0002 |     | 
+ 150 | A0003 | 150 | A0003
+ 200 | A0004 |     | 
+ 250 | A0005 |     | 
+ 300 | A0006 | 300 | A0006
+ 350 | A0007 |     | 
+ 400 | A0008 |     | 
+ 450 | A0009 | 450 | A0009
+ 500 | A0010 |     | 
+ 550 | A0011 |     | 
+     |       |  75 | A0001
+     |       | 225 | A0004
+     |       | 375 | A0007
+     |       | 525 | A0010
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uplt1_e t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2_e t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A')) ORDER BY t1.a, t2.b;
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+  50 | A0001 |     | 
+ 100 | A0002 |     | 
+ 150 | A0003 | 150 | A0003
+ 200 | A0004 |     | 
+ 250 | A0005 |     | 
+ 300 | A0006 | 300 | A0006
+ 350 | A0007 |     | 
+ 400 | A0008 |     | 
+ 450 | A0009 | 450 | A0009
+ 500 | A0010 |     | 
+ 550 | A0011 |     | 
+     |       |  75 | A0001
+     |       | 225 | A0004
+     |       | 375 | A0007
+     |       | 525 | A0010
+(16 rows)
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 3) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 3) and (b 1) is considered for partition-wise join.
+                                        QUERY PLAN                                        
+------------------------------------------------------------------------------------------
+ Sort
+   Output: (avg(t1.a)), (avg(t2.b)), (avg((t3.a + t3.b))), t1.c, t2.c, t3.c
+   Sort Key: t1.c, t3.c
+   ->  HashAggregate
+         Output: avg(t1.a), avg(t2.b), avg((t3.a + t3.b)), t1.c, t2.c, t3.c
+         Group Key: t1.c, t2.c, t3.c
+         ->  Result
+               Output: t1.c, t2.c, t3.c, t1.a, t2.b, t3.a, t3.b
+               ->  Append
+                     ->  Hash Join
+                           Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                           Hash Cond: (t1.c = t2.c)
+                           ->  Seq Scan on public.plt1_p1 t1
+                                 Output: t1.a, t1.c
+                           ->  Hash
+                                 Output: t2.b, t2.c, t3.a, t3.b, t3.c
+                                 ->  Hash Join
+                                       Output: t2.b, t2.c, t3.a, t3.b, t3.c
+                                       Hash Cond: (t2.c = ltrim(t3.c, 'A'::text))
+                                       ->  Seq Scan on public.plt2_p1 t2
+                                             Output: t2.b, t2.c
+                                       ->  Hash
+                                             Output: t3.a, t3.b, t3.c
+                                             ->  Seq Scan on public.plt1_e_p1 t3
+                                                   Output: t3.a, t3.b, t3.c
+                     ->  Hash Join
+                           Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                           Hash Cond: (t1_1.c = t2_1.c)
+                           ->  Seq Scan on public.plt1_p2 t1_1
+                                 Output: t1_1.a, t1_1.c
+                           ->  Hash
+                                 Output: t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                                 ->  Hash Join
+                                       Output: t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                                       Hash Cond: (t2_1.c = ltrim(t3_1.c, 'A'::text))
+                                       ->  Seq Scan on public.plt2_p2 t2_1
+                                             Output: t2_1.b, t2_1.c
+                                       ->  Hash
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             ->  Seq Scan on public.plt1_e_p2 t3_1
+                                                   Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash Join
+                           Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                           Hash Cond: (t1_2.c = t2_2.c)
+                           ->  Seq Scan on public.plt1_p3 t1_2
+                                 Output: t1_2.a, t1_2.c
+                           ->  Hash
+                                 Output: t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                                 ->  Hash Join
+                                       Output: t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                                       Hash Cond: (t2_2.c = ltrim(t3_2.c, 'A'::text))
+                                       ->  Seq Scan on public.plt2_p3 t2_2
+                                             Output: t2_2.b, t2_2.c
+                                       ->  Hash
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             ->  Seq Scan on public.plt1_e_p3 t3_2
+                                                   Output: t3_2.a, t3_2.b, t3_2.c
+(57 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;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 3) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 3) and (b 1) is considered for partition-wise join.
+         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)
+
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM uplt1 t1, uplt2 t2, uplt1_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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: ((t3.a = t1.a) AND (ltrim(t3.c, 'A'::text) = t1.c))
+                     ->  Seq Scan on public.plt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                                 ->  Seq Scan on public.plt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.plt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: ((t3_1.a = t1_1.a) AND (ltrim(t3_1.c, 'A'::text) = t1_1.c))
+                     ->  Seq Scan on public.plt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.c = t1_1.c))
+                                 ->  Seq Scan on public.plt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.plt1_p2 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: ((t3_2.a = t1_2.a) AND (ltrim(t3_2.c, 'A'::text) = t1_2.c))
+                     ->  Seq Scan on public.plt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                                 ->  Seq Scan on public.plt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.plt1_p3 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2 4) is considered for partition-wise join.
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: ((t3.a = t2.b) AND (ltrim(t3.c, 'A'::text) = t2.c))
+                     ->  Seq Scan on public.plt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                                 ->  Seq Scan on public.plt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.plt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: ((t3_1.a = t2_1.b) AND (ltrim(t3_1.c, 'A'::text) = t2_1.c))
+                     ->  Seq Scan on public.plt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.c = t1_1.c))
+                                 ->  Seq Scan on public.plt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.plt1_p2 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: ((t3_2.a = t2_2.b) AND (ltrim(t3_2.c, 'A'::text) = t2_2.c))
+                     ->  Seq Scan on public.plt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                                 ->  Seq Scan on public.plt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.plt1_p3 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2 4) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |          | 
+ 100 | 0002 |     |      |          | 
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |          | 
+ 250 | 0005 |     |      |          | 
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |          | 
+ 400 | 0008 |     |      |          | 
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |          | 
+ 550 | 0011 |     |      |          | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |          | 
+ 100 | 0002 |     |      |          | 
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |          | 
+ 250 | 0005 |     |      |          | 
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |          | 
+ 400 | 0008 |     |      |          | 
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |          | 
+ 550 | 0011 |     |      |          | 
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Hash Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                     ->  Seq Scan on public.plt2_p1 t2
+                           Output: t2.b, t2.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Hash Cond: ((t1.c = ltrim(t3.c, 'A'::text)) AND (t1.a = t3.a))
+                                 ->  Seq Scan on public.plt1_p1 t1
+                                       Output: t1.a, t1.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.plt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                     Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.c = t1_1.c))
+                     ->  Seq Scan on public.plt2_p2 t2_1
+                           Output: t2_1.b, t2_1.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_1.a, t1_1.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_1.a, t1_1.c
+                                 Hash Cond: ((t1_1.c = ltrim(t3_1.c, 'A'::text)) AND (t1_1.a = t3_1.a))
+                                 ->  Seq Scan on public.plt1_p2 t1_1
+                                       Output: t1_1.a, t1_1.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.plt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                     Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                     ->  Seq Scan on public.plt2_p3 t2_2
+                           Output: t2_2.b, t2_2.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_2.a, t1_2.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_2.a, t1_2.c
+                                 Hash Cond: ((t1_2.c = ltrim(t3_2.c, 'A'::text)) AND (t1_2.a = t3_2.a))
+                                 ->  Seq Scan on public.plt1_p3 t1_2
+                                       Output: t1_2.a, t1_2.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.plt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: ((t1.a = t2.b) AND (t1.c = t2.c))
+                     ->  Seq Scan on public.plt1_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Hash Cond: ((t2.b = t3.a) AND (t2.c = ltrim(t3.c, 'A'::text)))
+                                 ->  Seq Scan on public.plt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.plt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: ((t1_1.a = t2_1.b) AND (t1_1.c = t2_1.c))
+                     ->  Seq Scan on public.plt1_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: ((t2_1.b = t3_1.a) AND (t2_1.c = ltrim(t3_1.c, 'A'::text)))
+                                 ->  Seq Scan on public.plt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.plt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: ((t1_2.a = t2_2.b) AND (t1_2.c = t2_2.c))
+                     ->  Seq Scan on public.plt1_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: ((t2_2.b = t3_2.a) AND (t2_2.c = ltrim(t3_2.c, 'A'::text)))
+                                 ->  Seq Scan on public.plt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.plt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+     |      |     |      |      100 | A0001
+     |      |     |      |      200 | A0002
+     |      |     |      |      400 | A0004
+     |      |     |      |      500 | A0005
+     |      |     |      |      700 | A0007
+     |      |     |      |      800 | A0008
+     |      |     |      |     1000 | A0010
+     |      |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+     |      |     |      |      100 | A0001
+     |      |     |      |      200 | A0002
+     |      |     |      |      400 | A0004
+     |      |     |      |      500 | A0005
+     |      |     |      |      700 | A0007
+     |      |     |      |      800 | A0008
+     |      |     |      |     1000 | A0010
+     |      |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 6) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is considered for partition-wise join.
+                                                  QUERY PLAN                                                   
+---------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, ((plt1_e_p1.a + plt1_e_p1.b)), plt1_e_p1.c
+   Sort Key: plt1_p1.a, plt2_p1.b, ((plt1_e_p1.a + plt1_e_p1.b))
+   ->  Result
+         Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, (plt1_e_p1.a + plt1_e_p1.b), plt1_e_p1.c
+         ->  Append
+               ->  Hash Full Join
+                     Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                     Hash Cond: ((plt1_p1.a = plt1_e_p1.a) AND (plt1_p1.c = ltrim(plt1_e_p1.c, 'A'::text)))
+                     ->  Hash Full Join
+                           Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+                           Hash Cond: ((plt1_p1.a = plt2_p1.b) AND (plt1_p1.c = plt2_p1.c))
+                           ->  Seq Scan on public.plt1_p1
+                                 Output: plt1_p1.a, plt1_p1.c
+                                 Filter: ((plt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p1.b, plt2_p1.c
+                                 ->  Seq Scan on public.plt2_p1
+                                       Output: plt2_p1.b, plt2_p1.c
+                                       Filter: ((plt2_p1.b % 25) = 0)
+                     ->  Hash
+                           Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                           ->  Seq Scan on public.plt1_e_p1
+                                 Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                                 Filter: ((plt1_e_p1.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c, plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                     Hash Cond: ((plt1_p2.a = plt1_e_p2.a) AND (plt1_p2.c = ltrim(plt1_e_p2.c, 'A'::text)))
+                     ->  Hash Full Join
+                           Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+                           Hash Cond: ((plt1_p2.a = plt2_p2.b) AND (plt1_p2.c = plt2_p2.c))
+                           ->  Seq Scan on public.plt1_p2
+                                 Output: plt1_p2.a, plt1_p2.c
+                                 Filter: ((plt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p2.b, plt2_p2.c
+                                 ->  Seq Scan on public.plt2_p2
+                                       Output: plt2_p2.b, plt2_p2.c
+                                       Filter: ((plt2_p2.b % 25) = 0)
+                     ->  Hash
+                           Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                           ->  Seq Scan on public.plt1_e_p2
+                                 Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                                 Filter: ((plt1_e_p2.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c, plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                     Hash Cond: ((plt1_p3.a = plt1_e_p3.a) AND (plt1_p3.c = ltrim(plt1_e_p3.c, 'A'::text)))
+                     ->  Hash Full Join
+                           Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+                           Hash Cond: ((plt1_p3.a = plt2_p3.b) AND (plt1_p3.c = plt2_p3.c))
+                           ->  Seq Scan on public.plt1_p3
+                                 Output: plt1_p3.a, plt1_p3.c
+                                 Filter: ((plt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p3.b, plt2_p3.c
+                                 ->  Seq Scan on public.plt2_p3
+                                       Output: plt2_p3.b, plt2_p3.c
+                                       Filter: ((plt2_p3.b % 25) = 0)
+                     ->  Hash
+                           Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                           ->  Seq Scan on public.plt1_e_p3
+                                 Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                                 Filter: ((plt1_e_p3.a % 25) = 0)
+(63 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 6) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 4) is considered for partition-wise join.
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.c = t1_3.c)
+               ->  Seq Scan on public.plt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_3.c, t2.c
+                     Hash Cond: (t1_3.c = ltrim(t2.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p1 t1_3
+                           Output: t1_3.c
+                     ->  Hash
+                           Output: t2.c
+                           ->  Seq Scan on public.plt1_e_p1 t2
+                                 Output: t2.c
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.c = t1_4.c)
+               ->  Seq Scan on public.plt1_p2 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_4.c, t2_1.c
+                     Hash Cond: (t1_4.c = ltrim(t2_1.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p2 t1_4
+                           Output: t1_4.c
+                     ->  Hash
+                           Output: t2_1.c
+                           ->  Seq Scan on public.plt1_e_p2 t2_1
+                                 Output: t2_1.c
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.c = t1_5.c)
+               ->  Seq Scan on public.plt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_5.c, t2_2.c
+                     Hash Cond: (t1_5.c = ltrim(t2_2.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p3 t1_5
+                           Output: t1_5.c
+                     ->  Hash
+                           Output: t2_2.c
+                           ->  Seq Scan on public.plt1_e_p3 t2_2
+                                 Output: t2_2.c
+(49 rows)
+
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 4) is considered for partition-wise join.
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1, uplt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 5) is considered for partition-wise join.
+                                   QUERY PLAN                                   
+--------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.c = t1_3.c)
+               ->  Seq Scan on public.plt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_3.c, t1_6.c
+                     Hash Cond: (t1_3.c = ltrim(t1_6.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p1 t1_3
+                           Output: t1_3.c
+                     ->  Hash
+                           Output: t1_6.c
+                           ->  HashAggregate
+                                 Output: t1_6.c
+                                 Group Key: ltrim(t1_6.c, 'A'::text)
+                                 ->  Seq Scan on public.plt1_e_p1 t1_6
+                                       Output: t1_6.c, ltrim(t1_6.c, 'A'::text)
+                                       Filter: ((t1_6.a % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.c = t1_4.c)
+               ->  Seq Scan on public.plt1_p2 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_4.c, t1_7.c
+                     Hash Cond: (t1_4.c = ltrim(t1_7.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p2 t1_4
+                           Output: t1_4.c
+                     ->  Hash
+                           Output: t1_7.c
+                           ->  HashAggregate
+                                 Output: t1_7.c
+                                 Group Key: ltrim(t1_7.c, 'A'::text)
+                                 ->  Seq Scan on public.plt1_e_p2 t1_7
+                                       Output: t1_7.c, ltrim(t1_7.c, 'A'::text)
+                                       Filter: ((t1_7.a % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.c = t1_5.c)
+               ->  Seq Scan on public.plt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_5.c, t1_8.c
+                     Hash Cond: (t1_5.c = ltrim(t1_8.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p3 t1_5
+                           Output: t1_5.c
+                     ->  Hash
+                           Output: t1_8.c
+                           ->  HashAggregate
+                                 Output: t1_8.c
+                                 Group Key: ltrim(t1_8.c, 'A'::text)
+                                 ->  Seq Scan on public.plt1_e_p3 t1_8
+                                       Output: t1_8.c, ltrim(t1_8.c, 'A'::text)
+                                       Filter: ((t1_8.a % 25) = 0)
+(61 rows)
+
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 5) is considered for partition-wise join.
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+-- test merge join with and without index scan
+CREATE INDEX iplt1_c on plt1(c);
+CREATE INDEX iplt1_p1_c on plt1_p1(c);
+CREATE INDEX iplt1_p2_c on plt1_p2(c);
+CREATE INDEX iplt1_p3_c on plt1_p3(c);
+CREATE INDEX iplt2_c on plt2(c);
+CREATE INDEX iplt2_p1_c on plt2_p1(c);
+CREATE INDEX iplt2_p2_c on plt2_p2(c);
+CREATE INDEX iplt2_p3_c on plt2_p3(c);
+CREATE INDEX iplt1_e_c on plt1_e(ltrim(c, 'A'));
+CREATE INDEX iplt1_e_p1_c on plt1_e_p1(ltrim(c, 'A'));
+CREATE INDEX iplt1_e_p2_c on plt1_e_p2(ltrim(c, 'A'));
+CREATE INDEX iplt1_e_p3_c on plt1_e_p3(ltrim(c, 'A'));
+ANALYZE plt1;
+ANALYZE plt1_p1;
+ANALYZE plt1_p2;
+ANALYZE plt1_p3;
+ANALYZE plt2;
+ANALYZE plt2_p1;
+ANALYZE plt2_p2;
+ANALYZE plt2_p3;
+ANALYZE plt1_e;
+ANALYZE plt1_e_p1;
+ANALYZE plt1_e_p2;
+ANALYZE plt1_e_p3;
+SET enable_hashjoin TO off;
+SET enable_nestloop TO off;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+                                                QUERY PLAN                                                 
+-----------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Merge Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           Sort Key: t2.b, t2.c
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Merge Cond: ((t3.a = t2.b) AND ((ltrim(t3.c, 'A'::text)) = t2.c))
+                                 ->  Sort
+                                       Output: t3.a, t3.b, t3.c, (ltrim(t3.c, 'A'::text))
+                                       Sort Key: t3.a, (ltrim(t3.c, 'A'::text))
+                                       ->  Seq Scan on public.plt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c, ltrim(t3.c, 'A'::text)
+                                             Filter: ((t3.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2.b, t2.c
+                                       Sort Key: t2.b, t2.c
+                                       ->  Seq Scan on public.plt2_p1 t2
+                                             Output: t2.b, t2.c
+                     ->  Sort
+                           Output: t1.a, t1.c
+                           Sort Key: t1.a, t1.c
+                           ->  Seq Scan on public.plt1_p1 t1
+                                 Output: t1.a, t1.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Merge Cond: ((t2_1.c = t1_1.c) AND (t2_1.b = t1_1.a))
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           Sort Key: t2_1.c, t2_1.b
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Merge Cond: ((t3_1.a = t2_1.b) AND ((ltrim(t3_1.c, 'A'::text)) = t2_1.c))
+                                 ->  Sort
+                                       Output: t3_1.a, t3_1.b, t3_1.c, (ltrim(t3_1.c, 'A'::text))
+                                       Sort Key: t3_1.a, (ltrim(t3_1.c, 'A'::text))
+                                       ->  Seq Scan on public.plt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c, ltrim(t3_1.c, 'A'::text)
+                                             Filter: ((t3_1.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2_1.b, t2_1.c
+                                       Sort Key: t2_1.b, t2_1.c
+                                       ->  Seq Scan on public.plt2_p2 t2_1
+                                             Output: t2_1.b, t2_1.c
+                     ->  Sort
+                           Output: t1_1.a, t1_1.c
+                           Sort Key: t1_1.c, t1_1.a
+                           ->  Seq Scan on public.plt1_p2 t1_1
+                                 Output: t1_1.a, t1_1.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Merge Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           Sort Key: t2_2.b, t2_2.c
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Merge Cond: ((t3_2.a = t2_2.b) AND ((ltrim(t3_2.c, 'A'::text)) = t2_2.c))
+                                 ->  Sort
+                                       Output: t3_2.a, t3_2.b, t3_2.c, (ltrim(t3_2.c, 'A'::text))
+                                       Sort Key: t3_2.a, (ltrim(t3_2.c, 'A'::text))
+                                       ->  Seq Scan on public.plt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c, ltrim(t3_2.c, 'A'::text)
+                                             Filter: ((t3_2.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2_2.b, t2_2.c
+                                       Sort Key: t2_2.b, t2_2.c
+                                       ->  Seq Scan on public.plt2_p3 t2_2
+                                             Output: t2_2.b, t2_2.c
+                     ->  Sort
+                           Output: t1_2.a, t1_2.c
+                           Sort Key: t1_2.a, t1_2.c
+                           ->  Seq Scan on public.plt1_p3 t1_2
+                                 Output: t1_2.a, t1_2.c
+(81 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+     |      |     |      |      100 | A0001
+     |      |     |      |      200 | A0002
+     |      |     |      |      400 | A0004
+     |      |     |      |      500 | A0005
+     |      |     |      |      700 | A0007
+     |      |     |      |      800 | A0008
+     |      |     |      |     1000 | A0010
+     |      |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+     |      |     |      |      100 | A0001
+     |      |     |      |      200 | A0002
+     |      |     |      |      400 | A0004
+     |      |     |      |      500 | A0005
+     |      |     |      |      700 | A0007
+     |      |     |      |      800 | A0008
+     |      |     |      |     1000 | A0010
+     |      |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 6) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is considered for partition-wise join.
+                                                  QUERY PLAN                                                   
+---------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, ((plt1_e_p1.a + plt1_e_p1.b)), plt1_e_p1.c
+   Sort Key: plt1_p1.a, plt2_p1.b, ((plt1_e_p1.a + plt1_e_p1.b))
+   ->  Result
+         Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, (plt1_e_p1.a + plt1_e_p1.b), plt1_e_p1.c
+         ->  Append
+               ->  Merge Full Join
+                     Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                     Merge Cond: ((plt1_p1.a = plt1_e_p1.a) AND (plt1_p1.c = (ltrim(plt1_e_p1.c, 'A'::text))))
+                     ->  Sort
+                           Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+                           Sort Key: plt1_p1.a, plt1_p1.c
+                           ->  Merge Full Join
+                                 Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+                                 Merge Cond: ((plt1_p1.a = plt2_p1.b) AND (plt1_p1.c = plt2_p1.c))
+                                 ->  Sort
+                                       Output: plt1_p1.a, plt1_p1.c
+                                       Sort Key: plt1_p1.a, plt1_p1.c
+                                       ->  Seq Scan on public.plt1_p1
+                                             Output: plt1_p1.a, plt1_p1.c
+                                             Filter: ((plt1_p1.a % 25) = 0)
+                                 ->  Sort
+                                       Output: plt2_p1.b, plt2_p1.c
+                                       Sort Key: plt2_p1.b, plt2_p1.c
+                                       ->  Seq Scan on public.plt2_p1
+                                             Output: plt2_p1.b, plt2_p1.c
+                                             Filter: ((plt2_p1.b % 25) = 0)
+                     ->  Sort
+                           Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c, (ltrim(plt1_e_p1.c, 'A'::text))
+                           Sort Key: plt1_e_p1.a, (ltrim(plt1_e_p1.c, 'A'::text))
+                           ->  Seq Scan on public.plt1_e_p1
+                                 Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c, ltrim(plt1_e_p1.c, 'A'::text)
+                                 Filter: ((plt1_e_p1.a % 25) = 0)
+               ->  Merge Full Join
+                     Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c, plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                     Merge Cond: ((plt1_p2.a = plt1_e_p2.a) AND (plt1_p2.c = (ltrim(plt1_e_p2.c, 'A'::text))))
+                     ->  Sort
+                           Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+                           Sort Key: plt1_p2.a, plt1_p2.c
+                           ->  Merge Full Join
+                                 Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+                                 Merge Cond: ((plt1_p2.a = plt2_p2.b) AND (plt1_p2.c = plt2_p2.c))
+                                 ->  Sort
+                                       Output: plt1_p2.a, plt1_p2.c
+                                       Sort Key: plt1_p2.a, plt1_p2.c
+                                       ->  Seq Scan on public.plt1_p2
+                                             Output: plt1_p2.a, plt1_p2.c
+                                             Filter: ((plt1_p2.a % 25) = 0)
+                                 ->  Sort
+                                       Output: plt2_p2.b, plt2_p2.c
+                                       Sort Key: plt2_p2.b, plt2_p2.c
+                                       ->  Seq Scan on public.plt2_p2
+                                             Output: plt2_p2.b, plt2_p2.c
+                                             Filter: ((plt2_p2.b % 25) = 0)
+                     ->  Sort
+                           Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c, (ltrim(plt1_e_p2.c, 'A'::text))
+                           Sort Key: plt1_e_p2.a, (ltrim(plt1_e_p2.c, 'A'::text))
+                           ->  Seq Scan on public.plt1_e_p2
+                                 Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c, ltrim(plt1_e_p2.c, 'A'::text)
+                                 Filter: ((plt1_e_p2.a % 25) = 0)
+               ->  Merge Full Join
+                     Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c, plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                     Merge Cond: ((plt1_p3.a = plt1_e_p3.a) AND (plt1_p3.c = (ltrim(plt1_e_p3.c, 'A'::text))))
+                     ->  Sort
+                           Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+                           Sort Key: plt1_p3.a, plt1_p3.c
+                           ->  Merge Full Join
+                                 Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+                                 Merge Cond: ((plt1_p3.a = plt2_p3.b) AND (plt1_p3.c = plt2_p3.c))
+                                 ->  Sort
+                                       Output: plt1_p3.a, plt1_p3.c
+                                       Sort Key: plt1_p3.a, plt1_p3.c
+                                       ->  Seq Scan on public.plt1_p3
+                                             Output: plt1_p3.a, plt1_p3.c
+                                             Filter: ((plt1_p3.a % 25) = 0)
+                                 ->  Sort
+                                       Output: plt2_p3.b, plt2_p3.c
+                                       Sort Key: plt2_p3.b, plt2_p3.c
+                                       ->  Seq Scan on public.plt2_p3
+                                             Output: plt2_p3.b, plt2_p3.c
+                                             Filter: ((plt2_p3.b % 25) = 0)
+                     ->  Sort
+                           Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c, (ltrim(plt1_e_p3.c, 'A'::text))
+                           Sort Key: plt1_e_p3.a, (ltrim(plt1_e_p3.c, 'A'::text))
+                           ->  Seq Scan on public.plt1_e_p3
+                                 Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c, ltrim(plt1_e_p3.c, 'A'::text)
+                                 Filter: ((plt1_e_p3.a % 25) = 0)
+(87 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 6) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 5) is considered for partition-wise join.
+                                      QUERY PLAN                                      
+--------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Merge Semi Join
+               Output: t1.a, t1.b, t1.c
+               Merge Cond: (t1.c = t1_3.c)
+               ->  Sort
+                     Output: t1.a, t1.b, t1.c
+                     Sort Key: t1.c
+                     ->  Seq Scan on public.plt1_p1 t1
+                           Output: t1.a, t1.b, t1.c
+                           Filter: ((t1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_3.c, t1_6.c
+                     ->  Merge Join
+                           Output: t1_3.c, t1_6.c
+                           Merge Cond: ((ltrim(t1_6.c, 'A'::text)) = t1_3.c)
+                           ->  Sort
+                                 Output: t1_6.c, (ltrim(t1_6.c, 'A'::text))
+                                 Sort Key: (ltrim(t1_6.c, 'A'::text))
+                                 ->  HashAggregate
+                                       Output: t1_6.c, (ltrim(t1_6.c, 'A'::text))
+                                       Group Key: ltrim(t1_6.c, 'A'::text)
+                                       ->  Seq Scan on public.plt1_e_p1 t1_6
+                                             Output: t1_6.c, ltrim(t1_6.c, 'A'::text)
+                                             Filter: ((t1_6.a % 25) = 0)
+                           ->  Sort
+                                 Output: t1_3.c
+                                 Sort Key: t1_3.c
+                                 ->  Seq Scan on public.plt2_p1 t1_3
+                                       Output: t1_3.c
+         ->  Merge Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Merge Cond: (t1_1.c = t1_4.c)
+               ->  Sort
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Sort Key: t1_1.c
+                     ->  Seq Scan on public.plt1_p2 t1_1
+                           Output: t1_1.a, t1_1.b, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_4.c, t1_7.c
+                     ->  Merge Join
+                           Output: t1_4.c, t1_7.c
+                           Merge Cond: ((ltrim(t1_7.c, 'A'::text)) = t1_4.c)
+                           ->  Sort
+                                 Output: t1_7.c, (ltrim(t1_7.c, 'A'::text))
+                                 Sort Key: (ltrim(t1_7.c, 'A'::text))
+                                 ->  HashAggregate
+                                       Output: t1_7.c, (ltrim(t1_7.c, 'A'::text))
+                                       Group Key: ltrim(t1_7.c, 'A'::text)
+                                       ->  Seq Scan on public.plt1_e_p2 t1_7
+                                             Output: t1_7.c, ltrim(t1_7.c, 'A'::text)
+                                             Filter: ((t1_7.a % 25) = 0)
+                           ->  Sort
+                                 Output: t1_4.c
+                                 Sort Key: t1_4.c
+                                 ->  Seq Scan on public.plt2_p2 t1_4
+                                       Output: t1_4.c
+         ->  Merge Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Merge Cond: (t1_2.c = t1_5.c)
+               ->  Sort
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Sort Key: t1_2.c
+                     ->  Seq Scan on public.plt1_p3 t1_2
+                           Output: t1_2.a, t1_2.b, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_5.c, t1_8.c
+                     ->  Merge Join
+                           Output: t1_5.c, t1_8.c
+                           Merge Cond: ((ltrim(t1_8.c, 'A'::text)) = t1_5.c)
+                           ->  Sort
+                                 Output: t1_8.c, (ltrim(t1_8.c, 'A'::text))
+                                 Sort Key: (ltrim(t1_8.c, 'A'::text))
+                                 ->  HashAggregate
+                                       Output: t1_8.c, (ltrim(t1_8.c, 'A'::text))
+                                       Group Key: ltrim(t1_8.c, 'A'::text)
+                                       ->  Seq Scan on public.plt1_e_p3 t1_8
+                                             Output: t1_8.c, ltrim(t1_8.c, 'A'::text)
+                                             Filter: ((t1_8.a % 25) = 0)
+                           ->  Sort
+                                 Output: t1_5.c
+                                 Sort Key: t1_5.c
+                                 ->  Seq Scan on public.plt2_p3 t1_5
+                                       Output: t1_5.c
+(88 rows)
+
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 5) is considered for partition-wise join.
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+SET enable_seqscan TO off;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Merge Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           Sort Key: t2.b, t2.c
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Merge Cond: (ltrim(t3.c, 'A'::text) = t2.c)
+                                 Join Filter: (t2.b = t3.a)
+                                 ->  Index Scan using iplt1_e_p1_c on public.plt1_e_p1 t3
+                                       Output: t3.a, t3.b, t3.c
+                                       Filter: ((t3.a % 25) = 0)
+                                 ->  Index Scan using iplt2_p1_c on public.plt2_p1 t2
+                                       Output: t2.b, t2.c
+                     ->  Sort
+                           Output: t1.a, t1.c
+                           Sort Key: t1.a, t1.c
+                           ->  Index Scan using iplt1_p1_c on public.plt1_p1 t1
+                                 Output: t1.a, t1.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Merge Cond: ((t2_1.c = t1_1.c) AND (t2_1.b = t1_1.a))
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           Sort Key: t2_1.c, t2_1.b
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Merge Cond: (ltrim(t3_1.c, 'A'::text) = t2_1.c)
+                                 Join Filter: (t2_1.b = t3_1.a)
+                                 ->  Index Scan using iplt1_e_p2_c on public.plt1_e_p2 t3_1
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       Filter: ((t3_1.a % 25) = 0)
+                                 ->  Index Scan using iplt2_p2_c on public.plt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                     ->  Sort
+                           Output: t1_1.a, t1_1.c
+                           Sort Key: t1_1.c, t1_1.a
+                           ->  Index Scan using iplt1_p2_c on public.plt1_p2 t1_1
+                                 Output: t1_1.a, t1_1.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Merge Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           Sort Key: t2_2.b, t2_2.c
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Merge Cond: (ltrim(t3_2.c, 'A'::text) = t2_2.c)
+                                 Join Filter: (t2_2.b = t3_2.a)
+                                 ->  Index Scan using iplt1_e_p3_c on public.plt1_e_p3 t3_2
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       Filter: ((t3_2.a % 25) = 0)
+                                 ->  Index Scan using iplt2_p3_c on public.plt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                     ->  Sort
+                           Output: t1_2.a, t1_2.c
+                           Sort Key: t1_2.a, t1_2.c
+                           ->  Index Scan using iplt1_p3_c on public.plt1_p3 t1_2
+                                 Output: t1_2.a, t1_2.c
+(66 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+     |      |     |      |      100 | A0001
+     |      |     |      |      200 | A0002
+     |      |     |      |      400 | A0004
+     |      |     |      |      500 | A0005
+     |      |     |      |      700 | A0007
+     |      |     |      |      800 | A0008
+     |      |     |      |     1000 | A0010
+     |      |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+     |      |     |      |      100 | A0001
+     |      |     |      |      200 | A0002
+     |      |     |      |      400 | A0004
+     |      |     |      |      500 | A0005
+     |      |     |      |      700 | A0007
+     |      |     |      |      800 | A0008
+     |      |     |      |     1000 | A0010
+     |      |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 6) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is considered for partition-wise join.
+                                                  QUERY PLAN                                                   
+---------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, ((plt1_e_p1.a + plt1_e_p1.b)), plt1_e_p1.c
+   Sort Key: plt1_p1.a, plt2_p1.b, ((plt1_e_p1.a + plt1_e_p1.b))
+   ->  Result
+         Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, (plt1_e_p1.a + plt1_e_p1.b), plt1_e_p1.c
+         ->  Append
+               ->  Merge Full Join
+                     Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                     Merge Cond: ((plt1_p1.a = plt1_e_p1.a) AND (plt1_p1.c = (ltrim(plt1_e_p1.c, 'A'::text))))
+                     ->  Sort
+                           Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+                           Sort Key: plt1_p1.a, plt1_p1.c
+                           ->  Merge Full Join
+                                 Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+                                 Merge Cond: ((plt1_p1.a = plt2_p1.b) AND (plt1_p1.c = plt2_p1.c))
+                                 ->  Sort
+                                       Output: plt1_p1.a, plt1_p1.c
+                                       Sort Key: plt1_p1.a, plt1_p1.c
+                                       ->  Index Scan using iplt1_p1_c on public.plt1_p1
+                                             Output: plt1_p1.a, plt1_p1.c
+                                             Filter: ((plt1_p1.a % 25) = 0)
+                                 ->  Sort
+                                       Output: plt2_p1.b, plt2_p1.c
+                                       Sort Key: plt2_p1.b, plt2_p1.c
+                                       ->  Index Scan using iplt2_p1_c on public.plt2_p1
+                                             Output: plt2_p1.b, plt2_p1.c
+                                             Filter: ((plt2_p1.b % 25) = 0)
+                     ->  Sort
+                           Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c, (ltrim(plt1_e_p1.c, 'A'::text))
+                           Sort Key: plt1_e_p1.a, (ltrim(plt1_e_p1.c, 'A'::text))
+                           ->  Index Scan using iplt1_e_p1_c on public.plt1_e_p1
+                                 Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c, ltrim(plt1_e_p1.c, 'A'::text)
+                                 Filter: ((plt1_e_p1.a % 25) = 0)
+               ->  Merge Full Join
+                     Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c, plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                     Merge Cond: ((plt1_p2.a = plt1_e_p2.a) AND (plt1_p2.c = (ltrim(plt1_e_p2.c, 'A'::text))))
+                     ->  Sort
+                           Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+                           Sort Key: plt1_p2.a, plt1_p2.c
+                           ->  Merge Full Join
+                                 Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+                                 Merge Cond: ((plt1_p2.a = plt2_p2.b) AND (plt1_p2.c = plt2_p2.c))
+                                 ->  Sort
+                                       Output: plt1_p2.a, plt1_p2.c
+                                       Sort Key: plt1_p2.a, plt1_p2.c
+                                       ->  Index Scan using iplt1_p2_c on public.plt1_p2
+                                             Output: plt1_p2.a, plt1_p2.c
+                                             Filter: ((plt1_p2.a % 25) = 0)
+                                 ->  Sort
+                                       Output: plt2_p2.b, plt2_p2.c
+                                       Sort Key: plt2_p2.b, plt2_p2.c
+                                       ->  Index Scan using iplt2_p2_c on public.plt2_p2
+                                             Output: plt2_p2.b, plt2_p2.c
+                                             Filter: ((plt2_p2.b % 25) = 0)
+                     ->  Sort
+                           Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c, (ltrim(plt1_e_p2.c, 'A'::text))
+                           Sort Key: plt1_e_p2.a, (ltrim(plt1_e_p2.c, 'A'::text))
+                           ->  Index Scan using iplt1_e_p2_c on public.plt1_e_p2
+                                 Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c, ltrim(plt1_e_p2.c, 'A'::text)
+                                 Filter: ((plt1_e_p2.a % 25) = 0)
+               ->  Merge Full Join
+                     Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c, plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                     Merge Cond: ((plt1_p3.a = plt1_e_p3.a) AND (plt1_p3.c = (ltrim(plt1_e_p3.c, 'A'::text))))
+                     ->  Sort
+                           Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+                           Sort Key: plt1_p3.a, plt1_p3.c
+                           ->  Merge Full Join
+                                 Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+                                 Merge Cond: ((plt1_p3.a = plt2_p3.b) AND (plt1_p3.c = plt2_p3.c))
+                                 ->  Sort
+                                       Output: plt1_p3.a, plt1_p3.c
+                                       Sort Key: plt1_p3.a, plt1_p3.c
+                                       ->  Index Scan using iplt1_p3_c on public.plt1_p3
+                                             Output: plt1_p3.a, plt1_p3.c
+                                             Filter: ((plt1_p3.a % 25) = 0)
+                                 ->  Sort
+                                       Output: plt2_p3.b, plt2_p3.c
+                                       Sort Key: plt2_p3.b, plt2_p3.c
+                                       ->  Index Scan using iplt2_p3_c on public.plt2_p3
+                                             Output: plt2_p3.b, plt2_p3.c
+                                             Filter: ((plt2_p3.b % 25) = 0)
+                     ->  Sort
+                           Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c, (ltrim(plt1_e_p3.c, 'A'::text))
+                           Sort Key: plt1_e_p3.a, (ltrim(plt1_e_p3.c, 'A'::text))
+                           ->  Index Scan using iplt1_e_p3_c on public.plt1_e_p3
+                                 Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c, ltrim(plt1_e_p3.c, 'A'::text)
+                                 Filter: ((plt1_e_p3.a % 25) = 0)
+(87 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 6) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 5) is considered for partition-wise join.
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Merge Semi Join
+               Output: t1.a, t1.b, t1.c
+               Merge Cond: (t1.c = t1_3.c)
+               ->  Index Scan using iplt1_p1_c on public.plt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_3.c, t1_6.c
+                     ->  Merge Semi Join
+                           Output: t1_3.c, t1_6.c
+                           Merge Cond: (t1_3.c = ltrim(t1_6.c, 'A'::text))
+                           ->  Index Only Scan using iplt2_p1_c on public.plt2_p1 t1_3
+                                 Output: t1_3.c
+                           ->  Materialize
+                                 Output: t1_6.c
+                                 ->  Index Scan using iplt1_e_p1_c on public.plt1_e_p1 t1_6
+                                       Output: t1_6.c
+                                       Filter: ((t1_6.a % 25) = 0)
+         ->  Merge Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Merge Cond: (t1_1.c = t1_4.c)
+               ->  Index Scan using iplt1_p2_c on public.plt1_p2 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_4.c, t1_7.c
+                     ->  Merge Semi Join
+                           Output: t1_4.c, t1_7.c
+                           Merge Cond: (t1_4.c = ltrim(t1_7.c, 'A'::text))
+                           ->  Index Only Scan using iplt2_p2_c on public.plt2_p2 t1_4
+                                 Output: t1_4.c
+                           ->  Materialize
+                                 Output: t1_7.c
+                                 ->  Index Scan using iplt1_e_p2_c on public.plt1_e_p2 t1_7
+                                       Output: t1_7.c
+                                       Filter: ((t1_7.a % 25) = 0)
+         ->  Merge Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Merge Cond: (t1_2.c = t1_5.c)
+               ->  Index Scan using iplt1_p3_c on public.plt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_5.c, t1_8.c
+                     ->  Merge Semi Join
+                           Output: t1_5.c, t1_8.c
+                           Merge Cond: (t1_5.c = ltrim(t1_8.c, 'A'::text))
+                           ->  Index Only Scan using iplt2_p3_c on public.plt2_p3 t1_5
+                                 Output: t1_5.c
+                           ->  Materialize
+                                 Output: t1_8.c
+                                 ->  Index Scan using iplt1_e_p3_c on public.plt1_e_p3 t1_8
+                                       Output: t1_8.c
+                                       Filter: ((t1_8.a % 25) = 0)
+(58 rows)
+
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 5) is considered for partition-wise join.
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+RESET enable_hashjoin;
+RESET enable_nestloop;
+RESET enable_seqscan;
+--
+-- negative testcases
+--
+CREATE TABLE prt1_n (a int, b int, c varchar) PARTITION BY RANGE(c);
+CREATE TABLE prt1_n_p1 PARTITION OF prt1_n FOR VALUES START ('0000') END ('0250');
+CREATE TABLE prt1_n_p2 PARTITION OF prt1_n FOR VALUES START ('0250') END ('0500');
+INSERT INTO prt1_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 499, 2) i;
+ANALYZE prt1_n;
+ANALYZE prt1_n_p1;
+ANALYZE prt1_n_p2;
+CREATE TABLE prt2_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt2_n_p1 PARTITION OF prt2_n FOR VALUES IN ('0000', '0003', '0004', '0010', '0006', '0007');
+CREATE TABLE prt2_n_p2 PARTITION OF prt2_n FOR VALUES IN ('0001', '0005', '0002', '0009', '0008', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt2_n;
+ANALYZE prt2_n_p1;
+ANALYZE prt2_n_p2;
+CREATE TABLE prt3_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt3_n_p1 PARTITION OF prt3_n FOR VALUES IN ('0000', '0004', '0006', '0007');
+CREATE TABLE prt3_n_p2 PARTITION OF prt3_n FOR VALUES IN ('0001', '0002', '0008', '0010');
+CREATE TABLE prt3_n_p3 PARTITION OF prt3_n FOR VALUES IN ('0003', '0005', '0009', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt3_n;
+ANALYZE prt3_n_p1;
+ANALYZE prt3_n_p2;
+ANALYZE prt3_n_p3;
+CREATE TABLE prt4_n (a int, b int, c text) PARTITION BY RANGE(a);
+CREATE TABLE prt4_n_p1 PARTITION OF prt4_n FOR VALUES START (0) END (300);
+CREATE TABLE prt4_n_p2 PARTITION OF prt4_n FOR VALUES START (300) END (500);
+CREATE TABLE prt4_n_p3 PARTITION OF prt4_n FOR VALUES START (500) END (600);
+INSERT INTO prt4_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt4_n;
+ANALYZE prt4_n_p1;
+ANALYZE prt4_n_p2;
+ANALYZE prt4_n_p3;
+-- partition-wise join can not be applied if the partition ranges differ
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Join
+   Hash Cond: (t1.a = t2.a)
+   ->  Append
+         ->  Seq Scan on prt1 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p2 t1_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt4_n t2
+               ->  Seq Scan on prt4_n_p1 t2_1
+               ->  Seq Scan on prt4_n_p2 t2_2
+               ->  Seq Scan on prt4_n_p3 t2_3
+(13 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 FULL JOIN prt4_n t2 ON t1.a = t2.a;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Full Join
+   Hash Cond: (t1.a = t2.a)
+   ->  Append
+         ->  Seq Scan on prt1 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p2 t1_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt4_n t2
+               ->  Seq Scan on prt4_n_p1 t2_1
+               ->  Seq Scan on prt4_n_p2 t2_2
+               ->  Seq Scan on prt4_n_p3 t2_3
+(13 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                        
+---------------------------------------------------------
+ Nested Loop Left Join
+   ->  Append
+         ->  Seq Scan on prt1 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p2 t1_3
+   ->  Append
+         ->  Seq Scan on prt2 t2
+               Filter: (t1.a < b)
+         ->  Index Scan using iprt2_p1_b on prt2_p1 t2_1
+               Index Cond: (t1.a < b)
+         ->  Index Scan using iprt2_p2_b on prt2_p2 t2_2
+               Index Cond: (t1.a < b)
+         ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+               Index Cond: (t1.a < b)
+(15 rows)
+
+-- equi-join with join condition on partial keys does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1, prt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t1.a % 25 = 0;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Join
+   Hash Cond: (((t2.b + t2.a) / 2) = t1.a)
+   ->  Append
+         ->  Seq Scan on prt2_m t2
+         ->  Seq Scan on prt2_m_p1 t2_1
+         ->  Seq Scan on prt2_m_p2 t2_2
+         ->  Seq Scan on prt2_m_p3 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_m t1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p1 t1_1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p2 t1_2
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p3 t1_3
+                     Filter: ((a % 25) = 0)
+(17 rows)
+
+-- equi-join between out-of-order partition key columns does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.a = t2.b WHERE t1.a % 25 = 0;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Right Join
+   Hash Cond: (t2.b = t1.a)
+   ->  Append
+         ->  Seq Scan on prt2_m t2
+         ->  Seq Scan on prt2_m_p1 t2_1
+         ->  Seq Scan on prt2_m_p2 t2_2
+         ->  Seq Scan on prt2_m_p3 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_m t1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p1 t1_1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p2 t1_2
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p3 t1_3
+                     Filter: ((a % 25) = 0)
+(17 rows)
+
+-- equi-join between non-key columns does not qualify for partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.c = t2.c WHERE t1.a % 25 = 0;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Right Join
+   Hash Cond: ((t2.c)::text = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt2_m t2
+         ->  Seq Scan on prt2_m_p1 t2_1
+         ->  Seq Scan on prt2_m_p2 t2_2
+         ->  Seq Scan on prt2_m_p3 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_m t1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p1 t1_1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p2 t1_2
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p3 t1_3
+                     Filter: ((a % 25) = 0)
+(17 rows)
+
+-- partition-wise join can not be applied for a join between list and range
+-- partitioned table
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1, prt2_n t2 WHERE t1.c = t2.c;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Join
+   Hash Cond: (t2.c = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt2_n t2
+         ->  Seq Scan on prt2_n_p1 t2_1
+         ->  Seq Scan on prt2_n_p2 t2_2
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(11 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 LEFT JOIN prt2_n t2 ON (t1.c = t2.c);
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Right Join
+   Hash Cond: (t2.c = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt2_n t2
+         ->  Seq Scan on prt2_n_p1 t2_1
+         ->  Seq Scan on prt2_n_p2 t2_2
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(11 rows)
+
+-- partition-wise join can not be applied between tables with different
+-- partition lists
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 RIGHT JOIN prt1 t2 ON (t1.c = t2.c);
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Left Join
+   Hash Cond: ((t2.c)::text = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt1 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p2 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(12 rows)
+
+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);
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Full Join
+   Hash Cond: ((t2.c)::text = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt1 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p2 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(12 rows)
+
diff --git a/src/test/regress/expected/rangefuncs.out b/src/test/regress/expected/rangefuncs.out
index f06cfa4..16e7f56 100644
--- a/src/test/regress/expected/rangefuncs.out
+++ b/src/test/regress/expected/rangefuncs.out
@@ -1,25 +1,26 @@
 SELECT name, setting FROM pg_settings WHERE name LIKE 'enable%';
-         name         | setting 
-----------------------+---------
- enable_bitmapscan    | on
- enable_hashagg       | on
- enable_hashjoin      | on
- enable_indexonlyscan | on
- enable_indexscan     | on
- enable_material      | on
- enable_mergejoin     | on
- enable_nestloop      | on
- enable_seqscan       | on
- enable_sort          | on
- enable_tidscan       | on
-(11 rows)
+            name            | setting 
+----------------------------+---------
+ enable_bitmapscan          | on
+ enable_hashagg             | on
+ enable_hashjoin            | on
+ enable_indexonlyscan       | on
+ enable_indexscan           | on
+ enable_material            | on
+ enable_mergejoin           | on
+ enable_nestloop            | on
+ enable_partition_wise_join | on
+ enable_seqscan             | on
+ enable_sort                | on
+ enable_tidscan             | on
+(12 rows)
 
 CREATE TABLE foo2(fooid int, f2 int);
 INSERT INTO foo2 VALUES(1, 11);
 INSERT INTO foo2 VALUES(2, 22);
 INSERT INTO foo2 VALUES(1, 111);
 CREATE FUNCTION foot(int) returns setof foo2 as 'SELECT * FROM foo2 WHERE fooid = $1 ORDER BY f2;' LANGUAGE SQL;
 -- function with ORDINALITY
 select * from foot(1) with ordinality as z(a,b,ord);
  a |  b  | ord 
 ---+-----+-----
diff --git a/src/test/regress/parallel_schedule b/src/test/regress/parallel_schedule
index 1cb5dfc..d62841d 100644
--- a/src/test/regress/parallel_schedule
+++ b/src/test/regress/parallel_schedule
@@ -93,21 +93,21 @@ test: alter_generic alter_operator misc psql async dbsize misc_functions
 
 # rules cannot run concurrently with any test that creates a view
 test: rules psql_crosstab amutils
 
 # run by itself so it can run parallel workers
 test: select_parallel
 
 # ----------
 # Another group of parallel tests
 # ----------
-test: select_views portals_p2 foreign_key cluster dependency guc bitmapops combocid tsearch tsdicts foreign_data window xmlmap functional_deps advisory_lock json jsonb json_encoding indirect_toast equivclass
+test: select_views portals_p2 foreign_key cluster dependency guc bitmapops combocid tsearch tsdicts foreign_data window xmlmap functional_deps advisory_lock json jsonb json_encoding indirect_toast equivclass partition_join
 # ----------
 # Another group of parallel tests
 # NB: temp.sql does a reconnect which transiently uses 2 connections,
 # so keep this parallel group to at most 19 tests
 # ----------
 test: plancache limit plpgsql copy2 temp domain rangefuncs prepare without_oid conversion truncate alter_table sequence polymorphism rowtypes returning largeobject with xml
 
 # event triggers cannot run concurrently with any test that runs DDL
 test: event_trigger
 
diff --git a/src/test/regress/serial_schedule b/src/test/regress/serial_schedule
index 8958d8c..49a27b1 100644
--- a/src/test/regress/serial_schedule
+++ b/src/test/regress/serial_schedule
@@ -161,10 +161,11 @@ test: truncate
 test: alter_table
 test: sequence
 test: polymorphism
 test: rowtypes
 test: returning
 test: largeobject
 test: with
 test: xml
 test: event_trigger
 test: stats
+test: partition_join
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
new file mode 100644
index 0000000..e4f05c7
--- /dev/null
+++ b/src/test/regress/sql/partition_join.sql
@@ -0,0 +1,769 @@
+--
+-- PARTITION_JOIN
+-- Test partition-wise join between partitioned tables
+--
+
+--
+-- partitioned by a single column
+--
+CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES START (0) END (250);
+CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES START (500) END (600);
+CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES START (250) END (500);
+INSERT INTO prt1 SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1 AS SELECT * FROM prt1;
+
+CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES START (0) END (250);
+CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES START (250) END (500);
+CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES START (500) END (600);
+INSERT INTO prt2 SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+CREATE TABLE uprt2 AS SELECT * FROM prt2;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, 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.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+
+-- Join with pruned partitions from joining relations
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a < 450 AND t2.b > 250 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+-- lateral reference
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+--
+-- partitioned by expression
+--
+CREATE TABLE prt1_e (a int, b int, c varchar) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES START (0) END (250);
+CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES START (250) END (500);
+CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES START (500) END (600);
+INSERT INTO prt1_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_e AS SELECT * FROM prt1_e;
+
+CREATE TABLE prt2_e (a int, b int, c varchar) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES START (0) END (250);
+CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES START (250) END (500);
+CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES START (500) END (600);
+INSERT INTO prt2_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_e;
+ANALYZE prt2_e_p1;
+ANALYZE prt2_e_p2;
+ANALYZE prt2_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_e AS SELECT * FROM prt2_e;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1, uprt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1 LEFT JOIN uprt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 RIGHT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 RIGHT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1 RIGHT JOIN uprt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_e WHERE prt2_e.b % 25 = 0) t2 ON ((t1.a + t1.b)/2 = (t2.b + t2.a)/2) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_e WHERE prt2_e.b % 25 = 0) t2 ON ((t1.a + t1.b)/2 = (t2.b + t2.a)/2) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_e t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_e t2 WHERE t2.b % 25 = 0) t2 ON ((t1.a + t1.b)/2 = (t2.b + t2.a)/2) ORDER BY t1.a, t2.b;
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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.a % 25 = 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM uprt1 t1, uprt2 t2, uprt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM uprt1_e WHERE uprt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, 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.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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;
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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;
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM uprt1_e WHERE uprt1_e.a % 25 = 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;
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1, uprt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+-- test merge joins with and without using indexes
+SET enable_hashjoin TO off;
+SET enable_nestloop TO off;
+
+CREATE INDEX iprt1_a on prt1(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 iprt2_b on prt2(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 iprt1_e_ab2 on prt1_e(((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));
+
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+SET enable_seqscan TO off;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+-- lateral references and parameterized paths
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+RESET enable_hashjoin;
+RESET enable_nestloop;
+RESET enable_seqscan;
+
+--
+-- partitioned by multiple columns
+--
+CREATE TABLE prt1_m (a int, b int, c varchar) PARTITION BY RANGE(a, ((a + b)/2));
+CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES START (0, 0) END (250, 250);
+CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES START (250, 250) END (500, 500);
+CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES START (500, 500) END (600, 600);
+INSERT INTO prt1_m SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_m;
+ANALYZE prt1_m_p1;
+ANALYZE prt1_m_p2;
+ANALYZE prt1_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_m AS SELECT * FROM prt1_m;
+
+CREATE TABLE prt2_m (a int, b int, c varchar) PARTITION BY RANGE(((b + a)/2), b);
+CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES START (0, 0) END (250, 250);
+CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES START (250, 250) END (500, 500);
+CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES START (500, 500) END (600, 600);
+INSERT INTO prt2_m SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_m;
+ANALYZE prt2_m_p1;
+ANALYZE prt2_m_p2;
+ANALYZE prt2_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_m AS SELECT * FROM prt2_m;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1, prt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1, prt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_m t1, uprt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_m t1 LEFT JOIN uprt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_m t1 RIGHT JOIN uprt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_m t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_m t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+
+--
+-- multi-leveled partitions
+--
+CREATE TABLE prt1_l (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_l_p1 PARTITION OF prt1_l FOR VALUES START (0) END (250) PARTITION BY RANGE (b);
+CREATE TABLE prt1_l_p1_p1 PARTITION OF prt1_l_p1 FOR VALUES START (0) END (100);
+CREATE TABLE prt1_l_p1_p2 PARTITION OF prt1_l_p1 FOR VALUES START (100) END (250);
+CREATE TABLE prt1_l_p2 PARTITION OF prt1_l FOR VALUES START (250) END (500) PARTITION BY RANGE (c);
+CREATE TABLE prt1_l_p2_p1 PARTITION OF prt1_l_p2 FOR VALUES START ('0250') END ('0400');
+CREATE TABLE prt1_l_p2_p2 PARTITION OF prt1_l_p2 FOR VALUES START ('0400') END ('0500');
+CREATE TABLE prt1_l_p3 PARTITION OF prt1_l FOR VALUES START (500) END (600) PARTITION BY RANGE ((b + a));
+CREATE TABLE prt1_l_p3_p1 PARTITION OF prt1_l_p3 FOR VALUES START (1000) END (1100);
+CREATE TABLE prt1_l_p3_p2 PARTITION OF prt1_l_p3 FOR VALUES START (1100) END (1200);
+INSERT INTO prt1_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_l;
+ANALYZE prt1_l_p1;
+ANALYZE prt1_l_p1_p1;
+ANALYZE prt1_l_p1_p2;
+ANALYZE prt1_l_p2;
+ANALYZE prt1_l_p2_p1;
+ANALYZE prt1_l_p2_p2;
+ANALYZE prt1_l_p3;
+ANALYZE prt1_l_p3_p1;
+ANALYZE prt1_l_p3_p2;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_l AS SELECT * FROM prt1_l;
+
+CREATE TABLE prt2_l (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_l_p1 PARTITION OF prt2_l FOR VALUES START (0) END (250) PARTITION BY RANGE (a);
+CREATE TABLE prt2_l_p1_p1 PARTITION OF prt2_l_p1 FOR VALUES START (0) END (100);
+CREATE TABLE prt2_l_p1_p2 PARTITION OF prt2_l_p1 FOR VALUES START (100) END (250);
+CREATE TABLE prt2_l_p2 PARTITION OF prt2_l FOR VALUES START (250) END (500) PARTITION BY RANGE (c);
+CREATE TABLE prt2_l_p2_p1 PARTITION OF prt2_l_p2 FOR VALUES START ('0250') END ('0400');
+CREATE TABLE prt2_l_p2_p2 PARTITION OF prt2_l_p2 FOR VALUES START ('0400') END ('0500');
+CREATE TABLE prt2_l_p3 PARTITION OF prt2_l FOR VALUES START (500) END (600) PARTITION BY RANGE ((a + b));
+CREATE TABLE prt2_l_p3_p1 PARTITION OF prt2_l_p3 FOR VALUES START (1000) END (1100);
+CREATE TABLE prt2_l_p3_p2 PARTITION OF prt2_l_p3 FOR VALUES START (1100) END (1200);
+INSERT INTO prt2_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_l;
+ANALYZE prt2_l_p1;
+ANALYZE prt2_l_p1_p1;
+ANALYZE prt2_l_p1_p2;
+ANALYZE prt2_l_p2;
+ANALYZE prt2_l_p2_p1;
+ANALYZE prt2_l_p2_p2;
+ANALYZE prt2_l_p3;
+ANALYZE prt2_l_p3_p1;
+ANALYZE prt2_l_p3_p2;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_l AS SELECT * FROM prt2_l;
+
+EXPLAIN (VERBOSE, 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 = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 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 = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1, uprt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 LEFT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 RIGHT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_l t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_l t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+
+-- lateral reference
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
+			  ON t1.a = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
+			  ON t1.a = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1_l t2 JOIN uprt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
+			  ON t1.a = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
+			  ON t1.b = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
+			  ON t1.b = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1_l t2 JOIN uprt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
+			  ON t1.b = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+--
+-- 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;
+ANALYZE plt1_p1;
+ANALYZE plt1_p2;
+ANALYZE plt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1 AS SELECT * FROM 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;
+ANALYZE plt2_p1;
+ANALYZE plt2_p2;
+ANALYZE plt2_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt2 AS SELECT * FROM plt2;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.a = t2.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.a = t2.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1 t1, uplt2 t2 WHERE t1.c = t2.c AND t1.a = t2.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.a AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.a AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.a AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 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 plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 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 uplt1 t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) ORDER BY t1.a, t2.b;
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM (SELECT 50 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM (SELECT 50 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM (SELECT 50 phv, * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t1.phv), avg(t2.b), t2.c, avg(t2.phv) FROM (SELECT 25 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t1.phv), avg(t2.b), t2.c, avg(t2.phv) FROM (SELECT 25 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t1.phv), avg(t2.b), t2.c, avg(t2.phv) FROM (SELECT 25 phv, * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
+-- Join with pruned partitions from joining relations
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.c NOT IN ('0001', '0005', '0002', '0009') AND t2.c NOT IN ('0000', '0003', '0004', '0010') GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.c NOT IN ('0001', '0005', '0002', '0009') AND t2.c NOT IN ('0000', '0003', '0004', '0010') GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM uplt1 t1, uplt2 t2 WHERE t1.c = t2.c AND t1.c NOT IN ('0001', '0005', '0002', '0009') AND t2.c NOT IN ('0000', '0003', '0004', '0010') GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 LEFT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 LEFT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM uplt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 LEFT JOIN (SELECT * FROM uplt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 RIGHT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 RIGHT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM uplt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 RIGHT JOIN (SELECT * FROM uplt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 FULL JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 FULL JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM uplt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 FULL JOIN (SELECT * FROM uplt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 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;
+ANALYZE plt1_e;
+ANALYZE plt1_e_p1;
+ANALYZE plt1_e_p2;
+ANALYZE plt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1_e AS SELECT * FROM plt1_e;
+
+CREATE TABLE plt2_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
+CREATE TABLE plt2_e_p1 PARTITION OF plt2_e FOR VALUES IN ('0000', '0003', '0004', '0010');
+CREATE TABLE plt2_e_p2 PARTITION OF plt2_e FOR VALUES IN ('0001', '0005', '0002', '0009');
+CREATE TABLE plt2_e_p3 PARTITION OF plt2_e FOR VALUES IN ('0006', '0007', '0008', '0011');
+INSERT INTO plt2_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE plt2_e;
+ANALYZE plt2_e_p1;
+ANALYZE plt2_e_p2;
+ANALYZE plt2_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt2_e AS SELECT * FROM plt2_e;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1, plt2_e t2 WHERE t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1, plt2_e t2 WHERE t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_e t1, uplt2_e t2 WHERE t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 LEFT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 LEFT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_e t1 LEFT JOIN uplt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 RIGHT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 RIGHT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_e t1 RIGHT JOIN uplt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2_e WHERE plt2_e.b % 25 = 0) t2 ON (t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A')) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2_e WHERE plt2_e.b % 25 = 0) t2 ON (t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A')) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uplt1_e t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2_e t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A')) ORDER BY t1.a, t2.b;
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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;
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM uplt1 t1, uplt2 t2, uplt1_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;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1, uplt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 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 ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+-- test merge join with and without index scan
+CREATE INDEX iplt1_c on plt1(c);
+CREATE INDEX iplt1_p1_c on plt1_p1(c);
+CREATE INDEX iplt1_p2_c on plt1_p2(c);
+CREATE INDEX iplt1_p3_c on plt1_p3(c);
+CREATE INDEX iplt2_c on plt2(c);
+CREATE INDEX iplt2_p1_c on plt2_p1(c);
+CREATE INDEX iplt2_p2_c on plt2_p2(c);
+CREATE INDEX iplt2_p3_c on plt2_p3(c);
+CREATE INDEX iplt1_e_c on plt1_e(ltrim(c, 'A'));
+CREATE INDEX iplt1_e_p1_c on plt1_e_p1(ltrim(c, 'A'));
+CREATE INDEX iplt1_e_p2_c on plt1_e_p2(ltrim(c, 'A'));
+CREATE INDEX iplt1_e_p3_c on plt1_e_p3(ltrim(c, 'A'));
+
+ANALYZE plt1;
+ANALYZE plt1_p1;
+ANALYZE plt1_p2;
+ANALYZE plt1_p3;
+ANALYZE plt2;
+ANALYZE plt2_p1;
+ANALYZE plt2_p2;
+ANALYZE plt2_p3;
+ANALYZE plt1_e;
+ANALYZE plt1_e_p1;
+ANALYZE plt1_e_p2;
+ANALYZE plt1_e_p3;
+
+SET enable_hashjoin TO off;
+SET enable_nestloop TO off;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 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 ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+SET enable_seqscan TO off;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 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 ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+RESET enable_hashjoin;
+RESET enable_nestloop;
+RESET enable_seqscan;
+
+--
+-- negative testcases
+--
+
+CREATE TABLE prt1_n (a int, b int, c varchar) PARTITION BY RANGE(c);
+CREATE TABLE prt1_n_p1 PARTITION OF prt1_n FOR VALUES START ('0000') END ('0250');
+CREATE TABLE prt1_n_p2 PARTITION OF prt1_n FOR VALUES START ('0250') END ('0500');
+INSERT INTO prt1_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 499, 2) i;
+ANALYZE prt1_n;
+ANALYZE prt1_n_p1;
+ANALYZE prt1_n_p2;
+
+CREATE TABLE prt2_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt2_n_p1 PARTITION OF prt2_n FOR VALUES IN ('0000', '0003', '0004', '0010', '0006', '0007');
+CREATE TABLE prt2_n_p2 PARTITION OF prt2_n FOR VALUES IN ('0001', '0005', '0002', '0009', '0008', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt2_n;
+ANALYZE prt2_n_p1;
+ANALYZE prt2_n_p2;
+
+CREATE TABLE prt3_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt3_n_p1 PARTITION OF prt3_n FOR VALUES IN ('0000', '0004', '0006', '0007');
+CREATE TABLE prt3_n_p2 PARTITION OF prt3_n FOR VALUES IN ('0001', '0002', '0008', '0010');
+CREATE TABLE prt3_n_p3 PARTITION OF prt3_n FOR VALUES IN ('0003', '0005', '0009', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt3_n;
+ANALYZE prt3_n_p1;
+ANALYZE prt3_n_p2;
+ANALYZE prt3_n_p3;
+
+CREATE TABLE prt4_n (a int, b int, c text) PARTITION BY RANGE(a);
+CREATE TABLE prt4_n_p1 PARTITION OF prt4_n FOR VALUES START (0) END (300);
+CREATE TABLE prt4_n_p2 PARTITION OF prt4_n FOR VALUES START (300) END (500);
+CREATE TABLE prt4_n_p3 PARTITION OF prt4_n FOR VALUES START (500) END (600);
+INSERT INTO prt4_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt4_n;
+ANALYZE prt4_n_p1;
+ANALYZE prt4_n_p2;
+ANALYZE prt4_n_p3;
+
+-- partition-wise join can not be applied if the partition ranges differ
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 FULL JOIN prt4_n t2 ON t1.a = t2.a;
+-- 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);
+-- equi-join with join condition on partial keys does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1, prt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t1.a % 25 = 0;
+-- equi-join between out-of-order partition key columns does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.a = t2.b WHERE t1.a % 25 = 0;
+-- equi-join between non-key columns does not qualify for partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.c = t2.c WHERE t1.a % 25 = 0;
+-- partition-wise join can not be applied for a join between list and range
+-- partitioned table
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1, prt2_n t2 WHERE t1.c = t2.c;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 LEFT JOIN prt2_n t2 ON (t1.c = t2.c);
+-- partition-wise join can not be applied between tables with different
+-- partition lists
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 RIGHT JOIN prt1 t2 ON (t1.c = t2.c);
+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);
#11Rajkumar Raghuwanshi
rajkumar.raghuwanshi@enterprisedb.com
In reply to: Ashutosh Bapat (#10)
Re: Partition-wise join for join between (declaratively) partitioned tables

Hi,

I got a server crash with partition_wise_join, steps to reproduce given
below.

postgres=# set enable_partition_wise_join=true;
SET
postgres=# CREATE TABLE tbl (a int,c text) PARTITION BY LIST(a);
CREATE TABLE
postgres=# CREATE TABLE tbl_p1 PARTITION OF tbl FOR VALUES IN (1, 2);
CREATE TABLE
postgres=# CREATE TABLE tbl_p2 PARTITION OF tbl FOR VALUES IN (3, 4);
CREATE TABLE
postgres=# INSERT INTO tbl VALUES (1,'P1'),(2,'P1'),(3,'P2'),(4,'P2');
INSERT 0 4
postgres=# EXPLAIN (VERBOSE, COSTS OFF) SELECT * FROM tbl t1 INNER JOIN tbl
t2 ON (t1.a = t2.a) WHERE t1.c = 'P1' AND t1.c = 'P2';
NOTICE: join between relations (b 1) and (b 2) is considered for
partition-wise join.
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.
!>

Thanks & Regards,
Rajkumar Raghuwanshi
QMG, EnterpriseDB Corporation

#12Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Rajkumar Raghuwanshi (#8)
Re: Partition-wise join for join between (declaratively) partitioned tables

Hi Rajkumar,

On Fri, Sep 16, 2016 at 6:00 PM, Rajkumar Raghuwanshi
<rajkumar.raghuwanshi@enterprisedb.com> wrote:

On Fri, Sep 9, 2016 at 3:17 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Hi All,

PFA the patch to support partition-wise joins for partitioned tables. The
patch
is based on the declarative parition support patches provided by Amit
Langote
on 26th August 2016.

I have applied declarative partitioning patches posted by Amit Langote on 26
Aug 2016 and then partition-wise-join patch, getting below error while make
install.

../../../../src/include/nodes/relation.h:706: error: redefinition of typedef
‘PartitionOptInfo’
../../../../src/include/nodes/relation.h:490: note: previous declaration of
‘PartitionOptInfo’ was here
make[4]: *** [gistbuild.o] Error 1
make[4]: Leaving directory
`/home/edb/Desktop/edb_work/WORKDB/PG/postgresql/src/backend/access/gist'
make[3]: *** [gist-recursive] Error 2
make[3]: Leaving directory
`/home/edb/Desktop/edb_work/WORKDB/PG/postgresql/src/backend/access'
make[2]: *** [access-recursive] Error 2
make[2]: Leaving directory
`/home/edb/Desktop/edb_work/WORKDB/PG/postgresql/src/backend'
make[1]: *** [all-backend-recurse] Error 2
make[1]: Leaving directory
`/home/edb/Desktop/edb_work/WORKDB/PG/postgresql/src'
make: *** [all-src-recurse] Error 2

PS : I am using - gcc (GCC) 4.4.7 20120313 (Red Hat 4.4.7-17)

Thanks for the report and the patch.

This is fixed by the patch posted with
/messages/by-id/CAFjFpRdRFWMc4zNjeJB6p1Ncpznc9DMdXfZJmVK5X_us5zeD9Q@mail.gmail.com.

--
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

#13Rajkumar Raghuwanshi
rajkumar.raghuwanshi@enterprisedb.com
In reply to: Ashutosh Bapat (#10)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Sep 20, 2016 at 4:26 PM, Ashutosh Bapat <
ashutosh.bapat@enterprisedb.com> wrote:

PFA patch which takes care of some of the TODOs mentioned in my
previous mail. The patch is based on the set of patches supporting
declarative partitioning by Amit Langoted posted on 26th August.

I have applied declarative partitioning patches posted by Amit Langote on
26 Aug 2016 and then latest partition-wise-join patch, getting below error
while make install.

../../../../src/include/catalog/partition.h:37: error: redefinition of
typedef ‘PartitionScheme’
../../../../src/include/nodes/relation.h:492: note: previous declaration of
‘PartitionScheme’ was here
make[4]: *** [commit_ts.o] Error 1
make[4]: Leaving directory
`/home/edb/Desktop/edb_work/WORKDB/PG_PWJ/postgresql/src/backend/access/transam'
make[3]: *** [transam-recursive] Error 2
make[3]: Leaving directory
`/home/edb/Desktop/edb_work/WORKDB/PG_PWJ/postgresql/src/backend/access'
make[2]: *** [access-recursive] Error 2
make[2]: Leaving directory
`/home/edb/Desktop/edb_work/WORKDB/PG_PWJ/postgresql/src/backend'
make[1]: *** [all-backend-recurse] Error 2
make[1]: Leaving directory
`/home/edb/Desktop/edb_work/WORKDB/PG_PWJ/postgresql/src'
make: *** [all-src-recurse] Error 2

PS : I am using - gcc (GCC) 4.4.7 20120313 (Red Hat 4.4.7-17)

I have commented below statement in src/include/catalog/partition.h file
and then tried to install, it worked fine.

/* typedef struct PartitionSchemeData *PartitionScheme; */

Thanks & Regards,
Rajkumar Raghuwanshi

#14Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Rajkumar Raghuwanshi (#13)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

../../../../src/include/catalog/partition.h:37: error: redefinition of
typedef ‘PartitionScheme’
../../../../src/include/nodes/relation.h:492: note: previous declaration of
‘PartitionScheme’ was here

[...]

PS : I am using - gcc (GCC) 4.4.7 20120313 (Red Hat 4.4.7-17)

Thanks for the report. For some reason, I am not getting these errors
with my compiler

[ashutosh@ubuntu regress]gcc --version
gcc (Ubuntu/Linaro 4.6.3-1ubuntu5) 4.6.3

Anyway, I have fixed it in the attached patch.

The patch is based on sources upto commit

commit 2a7f4f76434d82eb0d1b5f4f7051043e1dd3ee1a
Author: Heikki Linnakangas <heikki.linnakangas@iki.fi>
Date: Wed Sep 21 13:24:13 2016 +0300

and Amit Langote's set of patches posted on 15th Sept. 2016 [1]. /messages/by-id/e5c1c9cf-3f5a-c4d7-6047-7351147aaef9@lab.ntt.co.jp

There are few implementation details that need to be worked out like
1. adjust_partitionrel_attrs() calls adjust_appendrel_attrs() as many
times as the number of base relations in the join, possibly producing
a new expression tree in every call. It can be optimized to call
adjust_appendrel_attrs() only once. I will work on that if reviewers
agree that adjust_partitionrel_attrs() is needed and should be
optimized.

2. As mentioned in earlier mails, the paths parameterized by parent
partitioned table are translated to be parameterized by child
partitions. That code needs to support more kinds of paths. I will
work on that, if reviewers agree that the approach of translating
paths is acceptable.

3. Because of an issue with declarative partitioning patch [2]. /messages/by-id/CAFjFpRc=T+CjpGNkNSdOkHza8VAPb35bngaCdAzPgBkhijmJhg@mail.gmail.com
multi-level partition table tests are failing in partition_join.sql.
Those were not failing with an earlier set of patches supporting
declarative partitions. Those will be fixed based on the discussion in
that thread.

4. More tests for foreign tables as partitions and for multi-level
partitioned tables.

5. The tests use unpartitioned tables for verifying results. Those
tables and corresponding SQL statements will be removed once the tests
are finalised.

[1]: . /messages/by-id/e5c1c9cf-3f5a-c4d7-6047-7351147aaef9@lab.ntt.co.jp
[2]: . /messages/by-id/CAFjFpRc=T+CjpGNkNSdOkHza8VAPb35bngaCdAzPgBkhijmJhg@mail.gmail.com

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

Attachments:

pg_dp_join_v3.patchinvalid/octet-stream; name=pg_dp_join_v3.patchDownload
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 1a60563..7b8406b 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -13,38 +13,41 @@
  *-------------------------------------------------------------------------
 */
 
 #include "postgres.h"
 
 #include "access/heapam.h"
 #include "access/htup_details.h"
 #include "access/nbtree.h"
 #include "access/sysattr.h"
 #include "catalog/dependency.h"
+#include "catalog/heap.h"
 #include "catalog/indexing.h"
 #include "catalog/objectaddress.h"
 #include "catalog/partition.h"
 #include "catalog/pg_collation.h"
 #include "catalog/pg_inherits.h"
 #include "catalog/pg_inherits_fn.h"
 #include "catalog/pg_opclass.h"
 #include "catalog/pg_partitioned_table.h"
 #include "catalog/pg_partitioned_table_fn.h"
 #include "catalog/pg_type.h"
 #include "executor/executor.h"
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
 #include "nodes/parsenodes.h"
 #include "optimizer/clauses.h"
+#include "optimizer/cost.h"
 #include "optimizer/planmain.h"
 #include "optimizer/var.h"
+#include "rewrite/rewriteManip.h"
 #include "storage/lmgr.h"
 #include "utils/array.h"
 #include "utils/builtins.h"
 #include "utils/datum.h"
 #include "utils/memutils.h"
 #include "utils/fmgroids.h"
 #include "utils/inval.h"
 #include "utils/lsyscache.h"
 #include "utils/ruleutils.h"
 #include "utils/rel.h"
@@ -208,20 +211,49 @@ typedef struct PartitionTreeNodeData
 	PartitionDesc		pdesc;
 	Oid					relid;
 	int					index;
 	int					offset;
 	int					num_leaf_parts;
 
 	struct PartitionTreeNodeData *downlink;
 	struct PartitionTreeNodeData *next;
 } PartitionTreeNodeData;
 
+/*
+ * Partitioning scheme
+ *		Structure to hold partitioning scheme for a given relation.
+ *
+ * Multiple relations may be partitioned in the same way. The relations
+ * resulting from joining such relations may be partitioned in the same way as
+ * the joining relations.  Similarly, relations derived from such relations by
+ * grouping, sorting be partitioned in the same as the underlying relations.
+ *
+ * PlannerInfo stores a list of distinct "canonical" partitioning schemes.
+ * RelOptInfo of a partitioned relation holds the pointer to "canonical"
+ * partitioning scheme.
+ */
+typedef struct PartitionSchemeData
+{
+	/* Information about partitions */
+	int			nparts;	/* number of partitions */
+	BoundCollection bounds; /* Partition bounds/lists */
+
+	/* Information about partition keys */
+	char		strategy;		/* partition strategy */
+	int16		partnatts;		/* number of partition attributes */
+	Oid		   *partopfamily;	/* OIDs of operator families */
+	Oid		   *partopcintype;	/* OIDs of opclass declared input data types */
+	Oid		   *key_types;		/* OIDs of partition key data types. */
+	int32	   *key_typmods;	/* typmods of partition keys. */
+	Oid		   *key_collations;	/* OIDs of collations of partition keys. */
+} PartitionSchemeData;
+
 /* Support RelationBuildPartitionKey() */
 static PartitionKey copy_partition_key(PartitionKey fromkey);
 static KeyTypeCollInfo *copy_key_type_coll_info(int nkeycols,
 								KeyTypeCollInfo *tcinfo);
 
 /* Support RelationBuildPartitionDesc() */
 static int32 list_value_cmp(const void *a, const void *b, void *arg);
 static int32 range_partition_cmp(const void *a, const void *b, void *arg);
 
 /* Support check_new_partition_bound() */
@@ -870,20 +902,21 @@ RelationBuildPartitionDesc(Relation rel)
 				result->bounds->rangeinfo = rangeinfo;
 				break;
 			}
 		}
 	}
 
 	MemoryContextSwitchTo(oldcxt);
 	rel->rd_partdesc = result;
 }
 
+
 /*
  * Are two partition bound collections logically equal?
  *
  * Used in the keep logic of relcache.c (ie, in RelationClearRelation()).
  * This is also useful when b1 and b2 are bound collections of two separate
  * relations, respectively, because BoundCollection is a canonical
  * representation of a set partition bounds (for given partitioning strategy).
  */
 bool
 partition_bounds_equal(PartitionKey key,
@@ -2411,20 +2444,73 @@ make_range_bound(PartitionKey key, List *val, bool inclusive, bool lower)
 				bound->val[i] = datumCopy(val->constvalue,
 										  key->tcinfo->typbyval[i],
 										  key->tcinfo->typlen[i]);
 			i++;
 		}
 	}
 
 	return bound;
 }
 
+static BoundCollection
+copy_bounds(BoundCollection src_bounds, PartitionKey key, int nparts)
+{
+	BoundCollection dst_bounds;
+	int		i;
+
+	dst_bounds = (BoundCollection) palloc(sizeof(BoundCollectionData));
+
+	if (src_bounds->listinfo)
+	{
+		ListInfo *dst_li = (ListInfo *) palloc(sizeof(ListInfo));
+		ListInfo *src_li = src_bounds->listinfo;
+
+		Assert(!src_bounds->rangeinfo);
+		dst_bounds->rangeinfo = NULL;
+
+		/* Copy the ListInfo structure. */
+		dst_li->nvalues = src_li->nvalues;
+		dst_li->has_null = src_li->has_null;
+		dst_li->null_index = src_li->null_index;
+
+		dst_li->values = (Datum *) palloc(sizeof(Datum) * dst_li->nvalues);
+		dst_li->indexes = (int *) palloc(sizeof(int) * dst_li->nvalues);
+		for (i = 0; i < dst_li->nvalues; i++)
+		{
+			dst_li->values[i] = datumCopy(src_li->values[i],
+													key->tcinfo->typbyval[0],
+													key->tcinfo->typlen[0]);
+			dst_li->indexes[i] = src_li->indexes[i];
+		}
+
+		dst_bounds->listinfo = dst_li;
+	}
+	else
+	{
+		RangeInfo *dst_ri = (RangeInfo *) palloc(sizeof(RangeInfo));
+		RangeInfo *src_ri = src_bounds->rangeinfo;
+
+		Assert(!src_bounds->listinfo && src_bounds->rangeinfo);
+		dst_bounds->listinfo = NULL;
+
+		/* Copy RangeInfo structure. */
+		dst_ri = (RangeInfo *) palloc(sizeof(RangeInfo));
+		dst_ri->ranges = (PartitionRange **) palloc(sizeof(PartitionRange *) * nparts);
+		for (i = 0; i < nparts; i++)
+			dst_ri->ranges[i] = copy_range(src_ri->ranges[i], key);
+
+		dst_bounds->rangeinfo = dst_ri;
+	}
+
+	return dst_bounds;
+}
+
 /*
  * Make and return a copy of input PartitionRange.
  */
 static PartitionRange *
 copy_range(PartitionRange *src, PartitionKey key)
 {
 	PartitionRange *result;
 
 	result = (PartitionRange *) palloc0(sizeof(PartitionRange));
 	result->lower = copy_range_bound(src->lower, key);
@@ -2647,10 +2733,210 @@ tuple_rightof_bound(PartitionKey key, Datum *tuple, PartitionRangeBound *bound)
 static bool
 tuple_leftof_bound(PartitionKey key, Datum *tuple, PartitionRangeBound *bound)
 {
 	int32	cmpval = partition_range_tuple_cmp(key, tuple, bound->val);
 
 	if (!cmpval)
 		return !bound->lower ? bound->inclusive : !bound->inclusive;
 
 	return cmpval < 0;
 }
+
+/*
+ * find_partition_scheme
+ * 		Find the "canonical" partition scheme for the given base table.
+ *
+ * 	The function searches the list of canonical partition schemes for one that
+ * 	exactly matches the partitioning properties of the given relation. If it
+ * 	does not find one, the function creates a canonical partition scheme
+ * 	structure and adds it to the list.
+ *
+ * 	For an umpartitioned table, it returns NULL.
+ */
+extern PartitionScheme
+find_partition_scheme(PlannerInfo *root, Relation relation)
+{
+	PartitionKey	part_key = RelationGetPartitionKey(relation);
+	PartitionDesc	part_desc = RelationGetPartitionDesc(relation);
+	ListCell	   *lc;
+	int		nparts;
+	int		partnatts;
+	int		cnt_pks;
+	PartitionScheme	part_scheme = NULL;
+
+	/* No partition scheme for an unpartitioned relation. */
+	if (!part_desc || !part_key)
+		return NULL;
+
+	nparts = part_desc->nparts;
+	partnatts = part_key->partnatts;
+
+	/* Search for a matching partition scheme and return if found one. */
+	foreach (lc, root->part_schemes)
+	{
+		part_scheme = lfirst(lc);
+
+		/* Match number of partitions and partitioning strategy. */
+		if (nparts != part_scheme->nparts ||
+			part_key->strategy != part_scheme->strategy ||
+			partnatts != part_scheme->partnatts)
+			continue;
+
+		/* Match the partition key types. */
+		for (cnt_pks = 0; cnt_pks < partnatts; cnt_pks++)
+		{
+			/*
+			 * It suffices to check the OID of support function as it always has
+			 * two arguemnts and returns boolean. For types, it suffices to match
+			 * the type id, mod and collation; len, byval and align are depedent on
+			 * the first two.
+			 */
+			if (part_key->partopfamily[cnt_pks] != part_scheme->partopfamily[cnt_pks] ||
+				part_key->partopcintype[cnt_pks] != part_scheme->partopcintype[cnt_pks] ||
+				part_key->tcinfo->typid[cnt_pks] != part_scheme->key_types[cnt_pks] ||
+				part_key->tcinfo->typmod[cnt_pks] != part_scheme->key_typmods[cnt_pks] ||
+				part_key->tcinfo->typcoll[cnt_pks] != part_scheme->key_collations[cnt_pks])
+				break;
+		}
+
+		/* Some partition key didn't match. Check next partitioning scheme. */
+		if (cnt_pks < partnatts)
+			continue;
+
+		if (!partition_bounds_equal(part_key, part_desc->bounds,
+									part_scheme->bounds, nparts))
+			continue;
+
+		/* Found matching partition scheme. */
+		return part_scheme;
+	}
+
+	/* Did not find matching partition scheme. Create one. */
+	part_scheme = (PartitionScheme) palloc0(sizeof(PartitionSchemeData));
+
+	/* Copy partition bounds/lists. */
+	part_scheme->nparts = part_desc->nparts;
+	part_scheme->strategy = part_key->strategy;
+	part_scheme->bounds = copy_bounds(part_desc->bounds, part_key,
+									  part_scheme->nparts);
+
+	/* Store partition key information. */
+	part_scheme->partnatts = part_key->partnatts;
+
+	part_scheme->partopfamily = (Oid *) palloc(sizeof(Oid) * partnatts);
+	part_scheme->partopcintype = (Oid *) palloc(sizeof(Oid) * partnatts);
+	part_scheme->key_types = (Oid *) palloc(sizeof(Oid) * partnatts);
+	part_scheme->key_typmods = (int32 *) palloc(sizeof(int32) * partnatts);
+	part_scheme->key_collations = (Oid *) palloc(sizeof(Oid) * partnatts);
+
+	for (cnt_pks = 0; cnt_pks < partnatts; cnt_pks++)
+	{
+		part_scheme->partopfamily[cnt_pks] = part_key->partopfamily[cnt_pks];
+		part_scheme->partopcintype[cnt_pks] = part_key->partopcintype[cnt_pks];
+		part_scheme->key_types[cnt_pks] = part_key->tcinfo->typid[cnt_pks];
+		part_scheme->key_typmods[cnt_pks] = part_key->tcinfo->typmod[cnt_pks];
+		part_scheme->key_collations[cnt_pks] = part_key->tcinfo->typcoll[cnt_pks];
+	}
+
+	/* Add the partitioning scheme to PlannerInfo. */
+	root->part_schemes = lappend(root->part_schemes, part_scheme);
+
+	return part_scheme;
+}
+
+/*
+ * build_baserel_partition_keys
+ *
+ * For a base relation, construct an array of partition key expressions. Each
+ * partition key expression is stored as a single member list to accomodate
+ * more partition keys when relations are joined.
+ */
+extern List **
+build_baserel_partition_keys(Relation relation, Index varno)
+{
+	PartitionKey	part_key = RelationGetPartitionKey(relation);
+	int		num_pkexprs;
+	int		cnt_pke;
+	List		  **partexprs;
+	ListCell	   *lc;
+
+	if (!part_key || part_key->partnatts <= 0)
+		return NULL;
+
+	num_pkexprs = part_key->partnatts;
+
+	/* Store partition keys as single elements lists. */
+	partexprs = (List **) palloc(sizeof(List *) * num_pkexprs);
+
+	lc = list_head(part_key->partexprs);
+	for (cnt_pke = 0; cnt_pke < num_pkexprs; cnt_pke++)
+	{
+		AttrNumber attno = part_key->partattrs[cnt_pke];
+		Expr	  *pkexpr;
+
+		if (attno != InvalidAttrNumber)
+		{
+			/* Single column partition key is stored as a Var node. */
+			Form_pg_attribute att_tup;
+
+			if (attno < 0)
+				att_tup = SystemAttributeDefinition(attno,
+												relation->rd_rel->relhasoids);
+			else
+				att_tup = relation->rd_att->attrs[attno - 1];
+
+			pkexpr = (Expr *) makeVar(varno, attno, att_tup->atttypid,
+									  att_tup->atttypmod,
+									  att_tup->attcollation, 0);
+		}
+		else
+		{
+			if (!lc)
+				elog(ERROR, "wrong number of partition key expressions");
+			pkexpr = (Expr *) copyObject(lfirst(lc));
+			ChangeVarNodes((Node *) pkexpr, 1, varno, 0);
+			lc = lnext(lc);
+		}
+
+		partexprs[cnt_pke] = list_make1(pkexpr);
+	}
+
+	return partexprs;
+}
+
+/*
+ * Returns the number of partitions supported by the given partition scheme.
+ */
+extern int
+PartitionSchemeGetNumParts(PartitionScheme part_scheme)
+{
+	return part_scheme ? part_scheme->nparts : 0;
+}
+
+/*
+ * Returns the number of partition keys supported by the given partition
+ * scheme.
+ */
+extern int
+PartitionSchemeGetNumKeys(PartitionScheme part_scheme)
+{
+	return part_scheme ? part_scheme->partnatts : 0;
+}
+
+/*
+ * Return palloc'ed array of OIDs of partitions.
+ */
+extern Oid *
+PartitionDescGetPartOids(PartitionDesc part_desc)
+{
+	Oid	   *part_oids;
+	int		cnt_parts;
+
+	if (!part_desc || part_desc->nparts <= 0)
+		return NULL;
+
+	part_oids = (Oid *) palloc(sizeof(Oid) * part_desc->nparts);
+	for (cnt_parts = 0; cnt_parts < part_desc->nparts; cnt_parts++)
+		part_oids[cnt_parts] = part_desc->oids[cnt_parts];
+
+	return part_oids;
+}
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 99b6bc8..1e43bab 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -13,20 +13,21 @@
  *-------------------------------------------------------------------------
  */
 
 #include "postgres.h"
 
 #include <limits.h>
 #include <math.h>
 
 #include "access/sysattr.h"
 #include "access/tsmapi.h"
+#include "catalog/partition.h"
 #include "catalog/pg_class.h"
 #include "catalog/pg_operator.h"
 #include "catalog/pg_proc.h"
 #include "foreign/fdwapi.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
 #ifdef OPTIMIZER_DEBUG
 #include "nodes/print.h"
 #endif
 #include "optimizer/clauses.h"
@@ -37,20 +38,21 @@
 #include "optimizer/plancat.h"
 #include "optimizer/planner.h"
 #include "optimizer/prep.h"
 #include "optimizer/restrictinfo.h"
 #include "optimizer/tlist.h"
 #include "optimizer/var.h"
 #include "parser/parse_clause.h"
 #include "parser/parsetree.h"
 #include "rewrite/rewriteManip.h"
 #include "utils/lsyscache.h"
+#include "utils/rel.h"
 
 
 /* results of subquery_is_pushdown_safe */
 typedef struct pushdown_safety_info
 {
 	bool	   *unsafeColumns;	/* which output columns are unsafe to use */
 	bool		unsafeVolatile; /* don't push down volatile quals */
 	bool		unsafeLeaky;	/* don't push down leaky quals */
 } pushdown_safety_info;
 
@@ -119,20 +121,24 @@ static void check_output_expressions(Query *subquery,
 static void compare_tlist_datatypes(List *tlist, List *colTypes,
 						pushdown_safety_info *safetyInfo);
 static bool targetIsInAllPartitionLists(TargetEntry *tle, Query *query);
 static bool qual_is_pushdown_safe(Query *subquery, Index rti, Node *qual,
 					  pushdown_safety_info *safetyInfo);
 static void subquery_push_qual(Query *subquery,
 				   RangeTblEntry *rte, Index rti, Node *qual);
 static void recurse_push_qual(Node *setOp, Query *topquery,
 				  RangeTblEntry *rte, Index rti, Node *qual);
 static void remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel);
+static void add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
+									List *live_childrels);
+static void generate_partition_wise_join_paths(PlannerInfo *root,
+											   RelOptInfo *rel);
 
 
 /*
  * make_one_rel
  *	  Finds all possible access paths for executing a query, returning a
  *	  single rel that represents the join of all base rels in the query.
  */
 RelOptInfo *
 make_one_rel(PlannerInfo *root, List *joinlist)
 {
@@ -861,20 +867,48 @@ static void
 set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 					Index rti, RangeTblEntry *rte)
 {
 	int			parentRTindex = rti;
 	bool		has_live_children;
 	double		parent_rows;
 	double		parent_size;
 	double	   *parent_attrsizes;
 	int			nattrs;
 	ListCell   *l;
+	Oid		   *part_oids = NULL;
+	int			nparts = 0;
+
+	/*
+	 * For a partitioned relation, we will save the child RelOptInfos in parent
+	 * RelOptInfo in the same the order as corresponding bounds/lists are
+	 * stored in the partition scheme.
+	 */
+	if (rel->part_scheme)
+	{
+		RangeTblEntry	*rte = root->simple_rte_array[rel->relid];
+
+		/*
+		 * We need not lock the relation since it was already locked, either by
+		 * the rewriter or when expand_inherited_rtentry() added it to the
+		 * query's rangetable.
+		 */
+		Relation	relation = heap_open(rte->relid, NoLock);
+		PartitionDesc	part_desc = RelationGetPartitionDesc(relation);
+
+		part_oids = PartitionDescGetPartOids(part_desc);
+		nparts = part_desc->nparts;
+
+		Assert(part_oids && nparts > 0);
+
+		rel->part_rels = (RelOptInfo **)palloc0(sizeof(RelOptInfo *) * nparts);
+		heap_close(relation, NoLock);
+	}
 
 	/*
 	 * Initialize to compute size estimates for whole append relation.
 	 *
 	 * We handle width estimates by weighting the widths of different child
 	 * rels proportionally to their number of rows.  This is sensible because
 	 * the use of width estimates is mainly to compute the total relation
 	 * "footprint" if we have to sort or hash it.  To do this, we sum the
 	 * total equivalent size (in "double" arithmetic) and then divide by the
 	 * total rowcount estimate.  This is done separately for the total rel
@@ -892,36 +926,80 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 	foreach(l, root->append_rel_list)
 	{
 		AppendRelInfo *appinfo = (AppendRelInfo *) lfirst(l);
 		int			childRTindex;
 		RangeTblEntry *childRTE;
 		RelOptInfo *childrel;
 		List	   *childquals;
 		Node	   *childqual;
 		ListCell   *parentvars;
 		ListCell   *childvars;
+		int			cnt_parts;
 
 		/* append_rel_list contains all append rels; ignore others */
 		if (appinfo->parent_relid != parentRTindex)
 			continue;
 
 		childRTindex = appinfo->child_relid;
 		childRTE = root->simple_rte_array[childRTindex];
 
 		/*
 		 * The child rel's RelOptInfo was already created during
 		 * add_base_rels_to_query.
 		 */
 		childrel = find_base_rel(root, childRTindex);
+
+		/*
+		 * Save topmost parent's relid. If the parent itself is a child of some
+		 * other relation, use parent's topmost parent relids.
+		 */
+		if (rel->top_parent_relids)
+			childrel->top_parent_relids = rel->top_parent_relids;
+		else
+			childrel->top_parent_relids = bms_copy(rel->relids);
+
 		Assert(childrel->reloptkind == RELOPT_OTHER_MEMBER_REL);
 
 		/*
+		 * For a partitioned table, save the child RelOptInfo at its
+		 * appropriate place in the parent RelOptInfo.
+		 */
+		for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+		{
+			if (part_oids[cnt_parts] == childRTE->relid)
+			{
+				Assert(!rel->part_rels[cnt_parts]);
+				rel->part_rels[cnt_parts] = childrel;
+			}
+		}
+
+		/*
+		 * Copy/Modify targetlist. Partition-wise join technique may consider
+		 * an OUTER join of another child relation with this child relation.
+		 * In that case, even if this child is deemed empty, we will require
+		 * the targetlist of this child to construct the nullable side. Hence
+		 * set the targetlist before we prove that the child is empty and stop
+		 * processing further.
+		 *
+		 * NB: the resulting childrel->reltarget->exprs may contain arbitrary
+		 * expressions, which otherwise would not occur in a rel's targetlist.
+		 * Code that might be looking at an appendrel child must cope with
+		 * such.  (Normally, a rel's targetlist would only include Vars and
+		 * PlaceHolderVars.)  XXX we do not bother to update the cost or width
+		 * fields of childrel->reltarget; not clear if that would be useful.
+		 */
+		childrel->reltarget->exprs = (List *)
+			adjust_appendrel_attrs(root,
+								   (Node *) rel->reltarget->exprs,
+								   appinfo);
+
+		/*
 		 * We have to copy the parent's targetlist and quals to the child,
 		 * with appropriate substitution of variables.  However, only the
 		 * baserestrictinfo quals are needed before we can check for
 		 * constraint exclusion; so do that first and then check to see if we
 		 * can disregard this child.
 		 *
 		 * As of 8.4, the child rel's targetlist might contain non-Var
 		 * expressions, which means that substitution into the quals could
 		 * produce opportunities for const-simplification, and perhaps even
 		 * pseudoconstant quals.  To deal with this, we strip the RestrictInfo
@@ -953,38 +1031,25 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		if (relation_excluded_by_constraints(root, childrel, childRTE))
 		{
 			/*
 			 * This child need not be scanned, so we can omit it from the
 			 * appendrel.
 			 */
 			set_dummy_rel_pathlist(childrel);
 			continue;
 		}
 
-		/*
-		 * CE failed, so finish copying/modifying targetlist and join quals.
-		 *
-		 * NB: the resulting childrel->reltarget->exprs may contain arbitrary
-		 * expressions, which otherwise would not occur in a rel's targetlist.
-		 * Code that might be looking at an appendrel child must cope with
-		 * such.  (Normally, a rel's targetlist would only include Vars and
-		 * PlaceHolderVars.)  XXX we do not bother to update the cost or width
-		 * fields of childrel->reltarget; not clear if that would be useful.
-		 */
+		/* CE failed, so finish copying/modifying join quals. */
 		childrel->joininfo = (List *)
 			adjust_appendrel_attrs(root,
 								   (Node *) rel->joininfo,
 								   appinfo);
-		childrel->reltarget->exprs = (List *)
-			adjust_appendrel_attrs(root,
-								   (Node *) rel->reltarget->exprs,
-								   appinfo);
 
 		/*
 		 * We have to make child entries in the EquivalenceClass data
 		 * structures as well.  This is needed either if the parent
 		 * participates in some eclass joins (because we will want to consider
 		 * inner-indexscan joins on the individual children) or if the parent
 		 * has useful pathkeys (because we should try to build MergeAppend
 		 * paths that produce those sort orderings).
 		 */
 		if (rel->has_eclass_joins || has_useful_pathkeys(root, rel))
@@ -1073,20 +1138,28 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 				}
 				if (child_width <= 0)
 					child_width = get_typavgwidth(exprType(childvar),
 												  exprTypmod(childvar));
 				Assert(child_width > 0);
 				parent_attrsizes[pndx] += child_width * childrel->rows;
 			}
 		}
 	}
 
+	/* Should have found all the childrels of a partitioned relation. */
+	if (rel->part_scheme)
+	{
+		int		cnt_parts;
+		for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+			Assert(rel->part_rels[cnt_parts]);
+	}
+
 	if (has_live_children)
 	{
 		/*
 		 * Save the finished size estimates.
 		 */
 		int			i;
 
 		Assert(parent_rows > 0);
 		rel->rows = parent_rows;
 		rel->reltarget->width = rint(parent_size / parent_rows);
@@ -1115,41 +1188,32 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 /*
  * set_append_rel_pathlist
  *	  Build access paths for an "append relation"
  */
 static void
 set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 						Index rti, RangeTblEntry *rte)
 {
 	int			parentRTindex = rti;
 	List	   *live_childrels = NIL;
-	List	   *subpaths = NIL;
-	bool		subpaths_valid = true;
-	List	   *partial_subpaths = NIL;
-	bool		partial_subpaths_valid = true;
-	List	   *all_child_pathkeys = NIL;
-	List	   *all_child_outers = NIL;
 	ListCell   *l;
 
 	/*
-	 * Generate access paths for each member relation, and remember the
-	 * cheapest path for each one.  Also, identify all pathkeys (orderings)
-	 * and parameterizations (required_outer sets) available for the member
-	 * relations.
+	 * Generate access paths for each member relation and remember the
+	 * non-dummy children.
 	 */
 	foreach(l, root->append_rel_list)
 	{
 		AppendRelInfo *appinfo = (AppendRelInfo *) lfirst(l);
 		int			childRTindex;
 		RangeTblEntry *childRTE;
 		RelOptInfo *childrel;
-		ListCell   *lcp;
 
 		/* append_rel_list contains all append rels; ignore others */
 		if (appinfo->parent_relid != parentRTindex)
 			continue;
 
 		/* Re-locate the child RTE and RelOptInfo */
 		childRTindex = appinfo->child_relid;
 		childRTE = root->simple_rte_array[childRTindex];
 		childrel = root->simple_rel_array[childRTindex];
 
@@ -1170,20 +1234,57 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 		/*
 		 * If child is dummy, ignore it.
 		 */
 		if (IS_DUMMY_REL(childrel))
 			continue;
 
 		/*
 		 * Child is live, so add it to the live_childrels list for use below.
 		 */
 		live_childrels = lappend(live_childrels, childrel);
+	}
+
+	add_paths_to_append_rel(root, rel, live_childrels);
+}
+
+static void
+add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
+						List *live_childrels)
+{
+	List	   *subpaths = NIL;
+	bool		subpaths_valid = true;
+	List	   *partial_subpaths = NIL;
+	bool		partial_subpaths_valid = true;
+	List	   *all_child_pathkeys = NIL;
+	List	   *all_child_outers = NIL;
+	ListCell   *l;
+
+	/* An append relation with all its children dummy is dummy. */
+	if (!live_childrels)
+	{
+		/* Mark the relation as dummy, if not already done so. */
+		if (!IS_DUMMY_REL(rel))
+			set_dummy_rel_pathlist(rel);
+
+		/* No more paths need to be added. */
+		return;
+	}
+
+	/*
+	 * For every non-dummy child, remember the cheapest path.  Also, identify
+	 * all pathkeys (orderings) and parameterizations (required_outer sets)
+	 * available for the non-dummy member relations.
+	 */
+	foreach (l, live_childrels)
+	{
+		RelOptInfo *childrel = lfirst(l);
+		ListCell   *lcp;
 
 		/*
 		 * If child has an unparameterized cheapest-total path, add that to
 		 * the unparameterized Append path we are constructing for the parent.
 		 * If not, there's no workable unparameterized path.
 		 */
 		if (childrel->cheapest_total_path->param_info == NULL)
 			subpaths = accumulate_append_subpath(subpaths,
 											  childrel->cheapest_total_path);
 		else
@@ -2188,20 +2289,22 @@ standard_join_search(PlannerInfo *root, int levels_needed, List *initial_rels)
 		 * Run generate_gather_paths() for each just-processed joinrel.  We
 		 * could not do this earlier because both regular and partial paths
 		 * can get added to a particular joinrel at multiple times within
 		 * join_search_one_level.  After that, we're done creating paths for
 		 * the joinrel, so run set_cheapest().
 		 */
 		foreach(lc, root->join_rel_level[lev])
 		{
 			rel = (RelOptInfo *) lfirst(lc);
 
+			generate_partition_wise_join_paths(root, rel);
+
 			/* Create GatherPaths for any useful partial paths for rel */
 			generate_gather_paths(root, rel);
 
 			/* Find and save the cheapest paths for this rel */
 			set_cheapest(rel);
 
 #ifdef OPTIMIZER_DEBUG
 			debug_print_rel(root, rel);
 #endif
 		}
@@ -2851,20 +2954,73 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
 		 * OK, we don't need it.  Replace the expression with a NULL constant.
 		 * Preserve the exposed type of the expression, in case something
 		 * looks at the rowtype of the subquery's result.
 		 */
 		tle->expr = (Expr *) makeNullConst(exprType(texpr),
 										   exprTypmod(texpr),
 										   exprCollation(texpr));
 	}
 }
 
+/*
+ * generate_partition_wise_join_paths
+ * 		Create appends paths containing partition-wise join paths for given
+ * 		join relation.
+ */
+static void
+generate_partition_wise_join_paths(PlannerInfo *root, RelOptInfo *rel)
+{
+	List	   *live_children = NIL;
+	int			cnt_part;
+	int			nparts = PartitionSchemeGetNumParts(rel->part_scheme);
+
+	/* Handle only join relations. */
+	if (rel->reloptkind != RELOPT_JOINREL &&
+		rel->reloptkind != RELOPT_OTHER_JOINREL)
+		return;
+
+	/* If the relation is not partitioned or is proven dummy, nothing to do. */
+	if (!rel->part_scheme || !rel->part_rels || IS_DUMMY_REL(rel))
+		return;
+
+	for (cnt_part = 0; cnt_part < nparts; cnt_part++)
+	{
+		RelOptInfo	*child_rel = rel->part_rels[cnt_part];
+
+		/* Ignore dummy child. */
+		if (!IS_DUMMY_REL(child_rel))
+		{
+			/* Recursively collect the paths from child joinrel. */
+			generate_partition_wise_join_paths(root, child_rel);
+
+			/* Find the cheapest of the paths for this rel. */
+			set_cheapest(child_rel);
+
+#ifdef OPTIMIZER_DEBUG
+			debug_print_rel(root, rel);
+#endif
+
+			live_children = lappend(live_children, child_rel);
+		}
+	}
+
+	/*
+	 * Create append paths by collecting sub paths from live children. Even if
+	 * there are no live children, we should create an append path with no
+	 * subpaths i.e. a dummy access path.
+	 */
+	add_paths_to_append_rel(root, rel, live_children);
+
+	if (live_children)
+		pfree(live_children);
+}
+
 /*****************************************************************************
  *			DEBUG SUPPORT
  *****************************************************************************/
 
 #ifdef OPTIMIZER_DEBUG
 
 static void
 print_relids(PlannerInfo *root, Relids relids)
 {
 	int			x;
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 2a49639..a23da1c 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -119,20 +119,21 @@ bool		enable_seqscan = true;
 bool		enable_indexscan = true;
 bool		enable_indexonlyscan = true;
 bool		enable_bitmapscan = true;
 bool		enable_tidscan = true;
 bool		enable_sort = true;
 bool		enable_hashagg = true;
 bool		enable_nestloop = true;
 bool		enable_material = true;
 bool		enable_mergejoin = true;
 bool		enable_hashjoin = true;
+bool		enable_partition_wise_join = true;
 
 typedef struct
 {
 	PlannerInfo *root;
 	QualCost	total;
 } cost_qual_eval_context;
 
 static List *extract_nonindex_conditions(List *qual_clauses, List *indexquals);
 static MergeScanSelCache *cached_scansel(PlannerInfo *root,
 			   RestrictInfo *rinfo,
diff --git a/src/backend/optimizer/path/equivclass.c b/src/backend/optimizer/path/equivclass.c
index 0e50ad5..73026a3 100644
--- a/src/backend/optimizer/path/equivclass.c
+++ b/src/backend/optimizer/path/equivclass.c
@@ -2359,20 +2359,22 @@ eclass_useful_for_merging(PlannerInfo *root,
 
 	/*
 	 * Note we don't test ec_broken; if we did, we'd need a separate code path
 	 * to look through ec_sources.  Checking the members anyway is OK as a
 	 * possibly-overoptimistic heuristic.
 	 */
 
 	/* If specified rel is a child, we must consider the topmost parent rel */
 	if (rel->reloptkind == RELOPT_OTHER_MEMBER_REL)
 		relids = find_childrel_top_parent(root, rel)->relids;
+	else if (rel->reloptkind == RELOPT_OTHER_JOINREL)
+		relids = rel->top_parent_relids;
 	else
 		relids = rel->relids;
 
 	/* If rel already includes all members of eclass, no point in searching */
 	if (bms_is_subset(eclass->ec_relids, relids))
 		return false;
 
 	/* To join, we need a member not in the given rel */
 	foreach(lc, eclass->ec_members)
 	{
diff --git a/src/backend/optimizer/path/joinpath.c b/src/backend/optimizer/path/joinpath.c
index cc7384f..db0e469 100644
--- a/src/backend/optimizer/path/joinpath.c
+++ b/src/backend/optimizer/path/joinpath.c
@@ -18,23 +18,29 @@
 
 #include "executor/executor.h"
 #include "foreign/fdwapi.h"
 #include "optimizer/cost.h"
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
 
 /* Hook for plugins to get control in add_paths_to_joinrel() */
 set_join_pathlist_hook_type set_join_pathlist_hook = NULL;
 
-#define PATH_PARAM_BY_REL(path, rel)  \
+#define PATH_PARAM_BY_PARENT(path, rel)	\
+	((path)->param_info && bms_overlap(PATH_REQ_OUTER(path),	\
+									   (rel)->top_parent_relids))
+#define PATH_PARAM_BY_REL_SELF(path, rel)  \
 	((path)->param_info && bms_overlap(PATH_REQ_OUTER(path), (rel)->relids))
 
+#define PATH_PARAM_BY_REL(path, rel)	\
+	(PATH_PARAM_BY_REL_SELF(path, rel) || PATH_PARAM_BY_PARENT(path, rel))
+
 static void sort_inner_and_outer(PlannerInfo *root, RelOptInfo *joinrel,
 					 RelOptInfo *outerrel, RelOptInfo *innerrel,
 					 JoinType jointype, JoinPathExtraData *extra);
 static void match_unsorted_outer(PlannerInfo *root, RelOptInfo *joinrel,
 					 RelOptInfo *outerrel, RelOptInfo *innerrel,
 					 JoinType jointype, JoinPathExtraData *extra);
 static void consider_parallel_nestloop(PlannerInfo *root,
 						   RelOptInfo *joinrel,
 						   RelOptInfo *outerrel,
 						   RelOptInfo *innerrel,
@@ -125,38 +131,51 @@ add_paths_to_joinrel(PlannerInfo *root,
 	 * directly to the parameter source rel instead of joining to the other
 	 * input rel.  (But see allow_star_schema_join().)	This restriction
 	 * reduces the number of parameterized paths we have to deal with at
 	 * higher join levels, without compromising the quality of the resulting
 	 * plan.  We express the restriction as a Relids set that must overlap the
 	 * parameterization of any proposed join path.
 	 */
 	foreach(lc, root->join_info_list)
 	{
 		SpecialJoinInfo *sjinfo = (SpecialJoinInfo *) lfirst(lc);
+		Relids	joinrelids;
+
+		/*
+		 * PlannerInfo doesn't contain the SpecialJoinInfos created for joins
+		 * between partitions, even if there is a SpecialJoinInfo node for the
+		 * join between the topmost parents. Hence while calculating Relids set
+		 * representing the restriction, consider relids of topmost parent of
+		 * partitions.
+		 */
+		if (joinrel->reloptkind == RELOPT_OTHER_JOINREL)
+			joinrelids = joinrel->top_parent_relids;
+		else
+			joinrelids = joinrel->relids;
 
 		/*
 		 * SJ is relevant to this join if we have some part of its RHS
 		 * (possibly not all of it), and haven't yet joined to its LHS.  (This
 		 * test is pretty simplistic, but should be sufficient considering the
 		 * join has already been proven legal.)  If the SJ is relevant, it
 		 * presents constraints for joining to anything not in its RHS.
 		 */
-		if (bms_overlap(joinrel->relids, sjinfo->min_righthand) &&
-			!bms_overlap(joinrel->relids, sjinfo->min_lefthand))
+		if (bms_overlap(joinrelids, sjinfo->min_righthand) &&
+			!bms_overlap(joinrelids, sjinfo->min_lefthand))
 			extra.param_source_rels = bms_join(extra.param_source_rels,
 										   bms_difference(root->all_baserels,
 													 sjinfo->min_righthand));
 
 		/* full joins constrain both sides symmetrically */
 		if (sjinfo->jointype == JOIN_FULL &&
-			bms_overlap(joinrel->relids, sjinfo->min_lefthand) &&
-			!bms_overlap(joinrel->relids, sjinfo->min_righthand))
+			bms_overlap(joinrelids, sjinfo->min_lefthand) &&
+			!bms_overlap(joinrelids, sjinfo->min_righthand))
 			extra.param_source_rels = bms_join(extra.param_source_rels,
 										   bms_difference(root->all_baserels,
 													  sjinfo->min_lefthand));
 	}
 
 	/*
 	 * However, when a LATERAL subquery is involved, there will simply not be
 	 * any paths for the joinrel that aren't parameterized by whatever the
 	 * subquery is parameterized by, unless its parameterization is resolved
 	 * within the joinrel.  So we might as well allow additional dependencies
@@ -272,20 +291,35 @@ try_nestloop_path(PlannerInfo *root,
 				  Path *outer_path,
 				  Path *inner_path,
 				  List *pathkeys,
 				  JoinType jointype,
 				  JoinPathExtraData *extra)
 {
 	Relids		required_outer;
 	JoinCostWorkspace workspace;
 
 	/*
+	 * An inner path parameterized by the parent relation of outer
+	 * relation needs to be reparameterized by the outer relation to be used
+	 * for parameterized nested loop join.
+	 */
+	if (PATH_PARAM_BY_PARENT(inner_path, outer_path->parent))
+	{
+		inner_path = reparameterize_path_for_child(root, inner_path,
+												   outer_path->parent);
+
+		/* If we could not translate the path, don't produce nest loop path. */
+		if (!inner_path)
+			return;
+	}
+
+	/*
 	 * Check to see if proposed path is still parameterized, and reject if the
 	 * parameterization wouldn't be sensible --- unless allow_star_schema_join
 	 * says to allow it anyway.  Also, we must reject if have_dangerous_phv
 	 * doesn't like the look of it, which could only happen if the nestloop is
 	 * still parameterized.
 	 */
 	required_outer = calc_nestloop_required_outer(outer_path,
 												  inner_path);
 	if (required_outer &&
 		((!bms_overlap(required_outer, extra->param_source_rels) &&
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 01d4fea..4c51dd2 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -7,38 +7,56 @@
  * Portions Copyright (c) 1994, Regents of the University of California
  *
  *
  * IDENTIFICATION
  *	  src/backend/optimizer/path/joinrels.c
  *
  *-------------------------------------------------------------------------
  */
 #include "postgres.h"
 
+#include "catalog/partition.h"
+#include "optimizer/clauses.h"
 #include "optimizer/joininfo.h"
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
+#include "optimizer/prep.h"
+#include "optimizer/cost.h"
 #include "utils/memutils.h"
 
 
 static void make_rels_by_clause_joins(PlannerInfo *root,
 						  RelOptInfo *old_rel,
 						  ListCell *other_rels);
 static void make_rels_by_clauseless_joins(PlannerInfo *root,
 							  RelOptInfo *old_rel,
 							  ListCell *other_rels);
 static bool has_join_restriction(PlannerInfo *root, RelOptInfo *rel);
 static bool has_legal_joinclause(PlannerInfo *root, RelOptInfo *rel);
 static bool is_dummy_rel(RelOptInfo *rel);
 static void mark_dummy_rel(RelOptInfo *rel);
 static bool restriction_is_constant_false(List *restrictlist,
 							  bool only_pushed_down);
+static void populate_joinrel_with_paths(PlannerInfo *root, RelOptInfo *rel1,
+							RelOptInfo *rel2, RelOptInfo *joinrel,
+							SpecialJoinInfo *sjinfo, List *restrictlist);
+static void try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1,
+						  RelOptInfo *rel2, RelOptInfo *joinrel,
+						  SpecialJoinInfo *parent_sjinfo,
+						  List *parent_restrictlist);
+static SpecialJoinInfo * build_partition_join_sjinfo(PlannerInfo *root,
+										 SpecialJoinInfo *parent_sjinfo,
+										 List *append_rel_infos1,
+										 List *append_rel_infos2);
+static bool have_partkey_equi_join(RelOptInfo *rel1, RelOptInfo *rel2,
+						   JoinType jointype, List *restrictlist);
+static int match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel);
 
 
 /*
  * join_search_one_level
  *	  Consider ways to produce join relations containing exactly 'level'
  *	  jointree items.  (This is one step of the dynamic-programming method
  *	  embodied in standard_join_search.)  Join rel nodes for each feasible
  *	  combination of lower-level rels are created and returned in a list.
  *	  Implementation paths are created for each such joinrel, too.
  *
@@ -717,20 +735,44 @@ make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 	/*
 	 * If we've already proven this join is empty, we needn't consider any
 	 * more paths for it.
 	 */
 	if (is_dummy_rel(joinrel))
 	{
 		bms_free(joinrelids);
 		return joinrel;
 	}
 
+	/* Add paths to the join relation. */
+	populate_joinrel_with_paths(root, rel1, rel2, joinrel, sjinfo,
+								restrictlist);
+
+	/* Apply partition-wise join technique, if possible. */
+	try_partition_wise_join(root, rel1, rel2, joinrel, sjinfo, restrictlist);
+
+	bms_free(joinrelids);
+
+	return joinrel;
+}
+
+/*
+ * populate_joinrel_with_paths
+ *	  Create paths to join given input relation and add those to the given
+ *	  joinrel. The SpecialJoinInfo provides details about the join and the
+ *	  restrictlist contains the join clauses and the other clauses applicable
+ *	  for given pair of the joining relations.
+ */
+static void
+populate_joinrel_with_paths(PlannerInfo *root, RelOptInfo *rel1,
+							RelOptInfo *rel2, RelOptInfo *joinrel,
+							SpecialJoinInfo *sjinfo, List *restrictlist)
+{
 	/*
 	 * Consider paths using each rel as both outer and inner.  Depending on
 	 * the join type, a provably empty outer or inner rel might mean the join
 	 * is provably empty too; in which case throw away any previously computed
 	 * paths and mark the join as dummy.  (We do it this way since it's
 	 * conceivable that dummy-ness of a multi-element join might only be
 	 * noticeable for certain construction paths.)
 	 *
 	 * Also, a provably constant-false join restriction typically means that
 	 * we can skip evaluating one or both sides of the join.  We do this by
@@ -861,27 +903,22 @@ make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 				mark_dummy_rel(rel2);
 			add_paths_to_joinrel(root, joinrel, rel1, rel2,
 								 JOIN_ANTI, sjinfo,
 								 restrictlist);
 			break;
 		default:
 			/* other values not expected here */
 			elog(ERROR, "unrecognized join type: %d", (int) sjinfo->jointype);
 			break;
 	}
-
-	bms_free(joinrelids);
-
-	return joinrel;
 }
 
-
 /*
  * have_join_order_restriction
  *		Detect whether the two relations should be joined to satisfy
  *		a join-order restriction arising from special or lateral joins.
  *
  * In practice this is always used with have_relevant_joinclause(), and so
  * could be merged with that function, but it seems clearer to separate the
  * two concerns.  We need this test because there are degenerate cases where
  * a clauseless join must be performed to satisfy join-order restrictions.
  * Also, if one rel has a lateral reference to the other, or both are needed
@@ -1242,10 +1279,314 @@ restriction_is_constant_false(List *restrictlist, bool only_pushed_down)
 
 			/* constant NULL is as good as constant FALSE for our purposes */
 			if (con->constisnull)
 				return true;
 			if (!DatumGetBool(con->constvalue))
 				return true;
 		}
 	}
 	return false;
 }
+
+/*
+ * If the join between the given two relations can be executed as
+ * partition-wise join create the join relations for partition-wise join,
+ * create paths for those and then create append paths to combine
+ * partition-wise join results.
+ */
+static void
+try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
+						RelOptInfo *joinrel, SpecialJoinInfo *parent_sjinfo,
+						List *parent_restrictlist)
+{
+	int		nparts;
+	int		cnt_parts;
+	StringInfo	rel1_desc;
+	StringInfo	rel2_desc;
+	PartitionScheme	part_scheme;
+
+
+	/* Nothing to do, if the join relation is not partitioned. */
+	if (!joinrel->part_scheme)
+		return;
+
+	/*
+	 * If an equi-join condition between the partition keys of the joining
+	 * relations does not exist, the given partition scheme can not be used for
+	 * partition-wise join between these two relations.
+	 */
+	if (!have_partkey_equi_join(rel1, rel2, parent_sjinfo->jointype,
+								parent_restrictlist))
+		return;
+
+	Assert(joinrel->part_scheme == rel1->part_scheme &&
+		   joinrel->part_scheme == rel2->part_scheme);
+
+	part_scheme = joinrel->part_scheme;
+	nparts = PartitionSchemeGetNumParts(part_scheme);
+	rel1_desc = makeStringInfo();
+	rel2_desc = makeStringInfo();
+
+	/* TODO: remove this notice when finalising the patch. */
+	outBitmapset(rel1_desc, rel1->relids);
+	outBitmapset(rel2_desc, rel2->relids);
+	elog(NOTICE, "join between relations %s and %s is considered for partition-wise join.",
+		 rel1_desc->data, rel2_desc->data);
+
+	/*
+	 * We allocate the array for child RelOptInfos till we find at least one
+	 * join order which can use partition-wise join technique. If no join order
+	 * can use partition-wise join technique, there are no child relations.
+	 */
+
+	if (!joinrel->part_rels)
+		joinrel->part_rels = (RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
+
+	/*
+	 * Create join relations for the partition relations, if they do not exist
+	 * already. Add paths to those for the given pair of joining relations.
+	 */
+	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	{
+		RelOptInfo *child_joinrel;
+		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
+		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
+		SpecialJoinInfo *child_sjinfo;
+		List	   *child_restrictlist;
+		List	   *join_aris;
+		List	   *ari1;
+		List	   *ari2;
+
+		/* We should never try to join two overlapping sets of rels. */
+		Assert(!bms_overlap(child_rel1->relids, child_rel2->relids));
+
+		/*
+		 * Gather the AppendRelInfos for base partition relations
+		 * partiticipating in the given partition relations. We need them
+		 * construct partition-wise join relation, special join info and
+		 * restriction list by substituting the Var and relids from parent to
+		 * child.
+		 */
+		ari1 = find_appendrelinfos_by_relids(root, child_rel1->relids);
+		ari2 = find_appendrelinfos_by_relids(root, child_rel2->relids);
+
+		/*
+		 * Construct SpecialJoinInfo from parent join relations's
+		 * SpecialJoinInfo.
+		 */
+		child_sjinfo = build_partition_join_sjinfo(root, parent_sjinfo, ari1, ari2);
+
+		/* Construct the parent-child relid map for the join relation. */
+		join_aris = list_concat(ari1, ari2);
+
+		/*
+		 * Construct restrictions applicable to the partition-wise join from
+		 * those applicable to the join between the parents.
+		 */
+		child_restrictlist = (List *) adjust_partitionrel_attrs(root,
+													(Node *)parent_restrictlist,
+													join_aris);
+
+		child_joinrel = joinrel->part_rels[cnt_parts];
+
+		/* Construct the join relation for given partition of the join. */
+		if (!child_joinrel)
+		{
+			child_joinrel = build_partition_join_rel(root, child_rel1,
+													 child_rel2, joinrel,
+													 child_sjinfo,
+													 child_restrictlist,
+													 join_aris);
+
+			joinrel->part_rels[cnt_parts] = child_joinrel;
+
+		}
+
+		/*
+		 * If we've already proven that this join is empty, we needn't consider
+		 * any more paths for it.
+		 */
+		if (is_dummy_rel(child_joinrel))
+			continue;
+
+		populate_joinrel_with_paths(root, child_rel1, child_rel2, child_joinrel,
+									child_sjinfo, child_restrictlist);
+
+		/*
+		 * If the partition themselves are partitioned, try partition-wise join
+		 * recursively.
+		 */
+		try_partition_wise_join(root, child_rel1, child_rel2, child_joinrel,
+								child_sjinfo, child_restrictlist);
+	}
+}
+
+/*
+ * Construct the SpecialJoinInfo for the partition-wise join using parents'
+ * special join info. Also, instead of
+ * constructing an sjinfo everytime, we should probably save it in
+ * root->join_info_list and search within it like join_is_legal?
+ */
+static SpecialJoinInfo *
+build_partition_join_sjinfo(PlannerInfo *root, SpecialJoinInfo *parent_sjinfo,
+							List *append_rel_infos1, List *append_rel_infos2)
+{
+	SpecialJoinInfo *sjinfo = copyObject(parent_sjinfo);
+	sjinfo->min_lefthand = adjust_partition_relids(sjinfo->min_lefthand,
+												   append_rel_infos1);
+	sjinfo->min_righthand = adjust_partition_relids(sjinfo->min_righthand,
+													append_rel_infos2);
+	sjinfo->syn_lefthand = adjust_partition_relids(sjinfo->syn_lefthand,
+												   append_rel_infos1);
+	sjinfo->syn_righthand = adjust_partition_relids(sjinfo->syn_righthand,
+													append_rel_infos2);
+
+	/* Replace the Var nodes of parent with those of children in expressions. */
+	sjinfo->semi_rhs_exprs = (List *) adjust_partitionrel_attrs(root,
+											   (Node *) sjinfo->semi_rhs_exprs,
+															append_rel_infos2);
+	return sjinfo;
+}
+
+/*
+ * Substitute oldrelids with newrelids in the given Relids set. It recycles the
+ * given relids input.
+ */
+Relids
+adjust_partition_relids(Relids relids, List *append_rel_infos)
+{
+	ListCell	*lc;
+	foreach (lc, append_rel_infos)
+	{
+		AppendRelInfo	*ari = lfirst(lc);
+
+		/* Remove old, add new */
+		if (bms_is_member(ari->parent_relid, relids))
+		{
+			relids = bms_del_member(relids, ari->parent_relid);
+			relids = bms_add_member(relids, ari->child_relid);
+		}
+	}
+	return relids;
+}
+
+/*
+ * Returns true if the given relations have equi-join clauses on all the
+ * corresponding partition keys.
+ */
+static bool
+have_partkey_equi_join(RelOptInfo *rel1, RelOptInfo *rel2,
+					 JoinType jointype, List *restrictlist)
+{
+	PartitionScheme	part_scheme = rel1->part_scheme;
+	ListCell	*lc;
+	int		cnt_pks;
+	int		num_pks = PartitionSchemeGetNumKeys(part_scheme);
+	bool   *pk_has_clause;
+
+	Assert(rel1->part_scheme == rel2->part_scheme);
+
+	pk_has_clause = (bool *) palloc0(sizeof(bool) * num_pks);
+
+	foreach (lc, restrictlist)
+	{
+		RestrictInfo *rinfo = lfirst(lc);
+		OpExpr		 *opexpr;
+		Expr		 *expr1;
+		Expr		 *expr2;
+		int		ipk1;
+		int		ipk2;
+
+		/* If processing an outer join, only use its own join clauses. */
+		if (IS_OUTER_JOIN(jointype) && rinfo->is_pushed_down)
+			continue;
+
+		/* Skip non-equi-join clauses. */
+		if (!rinfo->can_join ||
+			rinfo->hashjoinoperator == InvalidOid ||
+			!rinfo->mergeopfamilies)
+			continue;
+
+		opexpr = (OpExpr *) rinfo->clause;
+		Assert(is_opclause(opexpr));
+
+		/*
+		 * If clause of form rel1_expr op rel2_expr OR rel2_expr op rel1_expr,
+		 * match the operands to the relations. Otherwise, the clause is
+		 * not an equi-join between partition keys of joining relations.
+		 */
+		if (bms_is_subset(rinfo->left_relids, rel1->relids) &&
+			bms_is_subset(rinfo->right_relids, rel2->relids))
+		{
+			expr1 = linitial(opexpr->args);
+			expr2 = lsecond(opexpr->args);
+		}
+		else if (bms_is_subset(rinfo->left_relids, rel2->relids) &&
+				 bms_is_subset(rinfo->right_relids, rel1->relids))
+		{
+			expr1 = lsecond(opexpr->args);
+			expr2 = linitial(opexpr->args);
+		}
+		else
+			continue;
+
+		/* Associate matching clauses with partition keys. */
+		ipk1 = match_expr_to_partition_keys(expr1, rel1);
+		ipk2 = match_expr_to_partition_keys(expr2, rel2);
+
+		if (ipk1 == ipk2)
+			pk_has_clause[ipk1] = true;
+	}
+
+	/*
+	 * If every pair of partition key from either of the joining relation has
+	 * at least one equi-join clause associated with it, we have an equi-join
+	 * between all corresponding partition keys.
+	 */
+	for (cnt_pks = 0; cnt_pks < num_pks; cnt_pks++)
+	{
+		if (!pk_has_clause[cnt_pks])
+		{
+			pfree(pk_has_clause);
+			return false;
+		}
+	}
+
+	pfree(pk_has_clause);
+	return true;
+}
+
+/*
+ * match_expr_to_partition_keys
+ *
+ * Find the partition key which is same as the given expression. If found,
+ * return the index of the partition key, else return -1.
+ */
+static int
+match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel)
+{
+	int		cnt_pks;
+	int		num_pks = PartitionSchemeGetNumKeys(rel->part_scheme);
+
+	/*
+	 * Remove the relabel decoration. We can assume that there is at most one
+	 * RelabelType node; eval_const_expressions() will have simplied if more
+	 * than one.
+	 */
+	if (IsA(expr, RelabelType))
+		expr = (Expr *) ((RelabelType *) expr)->arg;
+
+	for (cnt_pks = 0; cnt_pks < num_pks; cnt_pks++)
+	{
+		List	 *pkexprs = rel->partexprs[cnt_pks];
+		ListCell *lc;
+
+		foreach(lc, pkexprs)
+		{
+			Expr *pkexpr = lfirst(lc);
+			if (equal(pkexpr, expr))
+				return cnt_pks;
+		}
+	}
+
+	return -1;
+}
diff --git a/src/backend/optimizer/path/pathkeys.c b/src/backend/optimizer/path/pathkeys.c
index 4436ac1..c8870c8 100644
--- a/src/backend/optimizer/path/pathkeys.c
+++ b/src/backend/optimizer/path/pathkeys.c
@@ -1081,26 +1081,38 @@ select_outer_pathkeys_for_merge(PlannerInfo *root,
 								List *mergeclauses,
 								RelOptInfo *joinrel)
 {
 	List	   *pathkeys = NIL;
 	int			nClauses = list_length(mergeclauses);
 	EquivalenceClass **ecs;
 	int		   *scores;
 	int			necs;
 	ListCell   *lc;
 	int			j;
+	Relids		relids;
 
 	/* Might have no mergeclauses */
 	if (nClauses == 0)
 		return NIL;
 
 	/*
+	 * Code below scores equivalence classes by how many equivalence members
+	 * can produce join clauses for this join relation. Equivalence members
+	 * which do not cover the parents of a partition-wise join relation, can
+	 * produce join clauses for partition-wise join relation.
+	 */
+	if (joinrel->reloptkind == RELOPT_OTHER_JOINREL)
+		relids = joinrel->top_parent_relids;
+	else
+		relids = joinrel->relids;
+
+	/*
 	 * Make arrays of the ECs used by the mergeclauses (dropping any
 	 * duplicates) and their "popularity" scores.
 	 */
 	ecs = (EquivalenceClass **) palloc(nClauses * sizeof(EquivalenceClass *));
 	scores = (int *) palloc(nClauses * sizeof(int));
 	necs = 0;
 
 	foreach(lc, mergeclauses)
 	{
 		RestrictInfo *rinfo = (RestrictInfo *) lfirst(lc);
@@ -1126,21 +1138,21 @@ select_outer_pathkeys_for_merge(PlannerInfo *root,
 			continue;
 
 		/* compute score */
 		score = 0;
 		foreach(lc2, oeclass->ec_members)
 		{
 			EquivalenceMember *em = (EquivalenceMember *) lfirst(lc2);
 
 			/* Potential future join partner? */
 			if (!em->em_is_const && !em->em_is_child &&
-				!bms_overlap(em->em_relids, joinrel->relids))
+				!bms_overlap(em->em_relids, relids))
 				score++;
 		}
 
 		ecs[necs] = oeclass;
 		scores[necs] = score;
 		necs++;
 	}
 
 	/*
 	 * Find out if we have all the ECs mentioned in query_pathkeys; if so we
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 32f4031..b221e2c 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -235,21 +235,22 @@ static Plan *prepare_sort_from_pathkeys(Plan *lefttree, List *pathkeys,
 						   const AttrNumber *reqColIdx,
 						   bool adjust_tlist_in_place,
 						   int *p_numsortkeys,
 						   AttrNumber **p_sortColIdx,
 						   Oid **p_sortOperators,
 						   Oid **p_collations,
 						   bool **p_nullsFirst);
 static EquivalenceMember *find_ec_member_for_tle(EquivalenceClass *ec,
 					   TargetEntry *tle,
 					   Relids relids);
-static Sort *make_sort_from_pathkeys(Plan *lefttree, List *pathkeys);
+static Sort *make_sort_from_pathkeys(Plan *lefttree, List *pathkeys,
+									 Relids relids);
 static Sort *make_sort_from_groupcols(List *groupcls,
 						 AttrNumber *grpColIdx,
 						 Plan *lefttree);
 static Material *make_material(Plan *lefttree);
 static WindowAgg *make_windowagg(List *tlist, Index winref,
 			   int partNumCols, AttrNumber *partColIdx, Oid *partOperators,
 			   int ordNumCols, AttrNumber *ordColIdx, Oid *ordOperators,
 			   int frameOptions, Node *startOffset, Node *endOffset,
 			   Plan *lefttree);
 static Group *make_group(List *tlist, List *qual, int numGroupCols,
@@ -1507,21 +1508,21 @@ create_sort_plan(PlannerInfo *root, SortPath *best_path, int flags)
 	Plan	   *subplan;
 
 	/*
 	 * We don't want any excess columns in the sorted tuples, so request a
 	 * smaller tlist.  Otherwise, since Sort doesn't project, tlist
 	 * requirements pass through.
 	 */
 	subplan = create_plan_recurse(root, best_path->subpath,
 								  flags | CP_SMALL_TLIST);
 
-	plan = make_sort_from_pathkeys(subplan, best_path->path.pathkeys);
+	plan = make_sort_from_pathkeys(subplan, best_path->path.pathkeys, NULL);
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
 	return plan;
 }
 
 /*
  * create_group_plan
  *
  *	  Create a Group plan for 'best_path' and (recursively) plans
@@ -3517,31 +3518,33 @@ create_mergejoin_plan(PlannerInfo *root,
 	List	   *innerpathkeys;
 	int			nClauses;
 	Oid		   *mergefamilies;
 	Oid		   *mergecollations;
 	int		   *mergestrategies;
 	bool	   *mergenullsfirst;
 	int			i;
 	ListCell   *lc;
 	ListCell   *lop;
 	ListCell   *lip;
+	Path	   *outer_path = best_path->jpath.outerjoinpath;
+	Path	   *inner_path = best_path->jpath.innerjoinpath;
 
 	/*
 	 * MergeJoin can project, so we don't have to demand exact tlists from the
 	 * inputs.  However, if we're intending to sort an input's result, it's
 	 * best to request a small tlist so we aren't sorting more data than
 	 * necessary.
 	 */
-	outer_plan = create_plan_recurse(root, best_path->jpath.outerjoinpath,
+	outer_plan = create_plan_recurse(root, outer_path,
 					 (best_path->outersortkeys != NIL) ? CP_SMALL_TLIST : 0);
 
-	inner_plan = create_plan_recurse(root, best_path->jpath.innerjoinpath,
+	inner_plan = create_plan_recurse(root, inner_path,
 					 (best_path->innersortkeys != NIL) ? CP_SMALL_TLIST : 0);
 
 	/* Sort join qual clauses into best execution order */
 	/* NB: do NOT reorder the mergeclauses */
 	joinclauses = order_qual_clauses(root, best_path->jpath.joinrestrictinfo);
 
 	/* Get the join qual clauses (in plain expression form) */
 	/* Any pseudoconstant clauses are ignored here */
 	if (IS_OUTER_JOIN(best_path->jpath.jointype))
 	{
@@ -3573,48 +3576,52 @@ create_mergejoin_plan(PlannerInfo *root,
 		otherclauses = (List *)
 			replace_nestloop_params(root, (Node *) otherclauses);
 	}
 
 	/*
 	 * Rearrange mergeclauses, if needed, so that the outer variable is always
 	 * on the left; mark the mergeclause restrictinfos with correct
 	 * outer_is_left status.
 	 */
 	mergeclauses = get_switched_clauses(best_path->path_mergeclauses,
-							 best_path->jpath.outerjoinpath->parent->relids);
+							 outer_path->parent->relids);
 
 	/*
 	 * Create explicit sort nodes for the outer and inner paths if necessary.
 	 */
 	if (best_path->outersortkeys)
 	{
+		Relids		outer_relids = outer_path->parent->relids;
 		Sort	   *sort = make_sort_from_pathkeys(outer_plan,
-												   best_path->outersortkeys);
+												   best_path->outersortkeys,
+												   outer_relids);
 
 		label_sort_with_costsize(root, sort, -1.0);
 		outer_plan = (Plan *) sort;
 		outerpathkeys = best_path->outersortkeys;
 	}
 	else
-		outerpathkeys = best_path->jpath.outerjoinpath->pathkeys;
+		outerpathkeys = outer_path->pathkeys;
 
 	if (best_path->innersortkeys)
 	{
+		Relids		inner_relids = inner_path->parent->relids;
 		Sort	   *sort = make_sort_from_pathkeys(inner_plan,
-												   best_path->innersortkeys);
+												   best_path->innersortkeys,
+												   inner_relids);
 
 		label_sort_with_costsize(root, sort, -1.0);
 		inner_plan = (Plan *) sort;
 		innerpathkeys = best_path->innersortkeys;
 	}
 	else
-		innerpathkeys = best_path->jpath.innerjoinpath->pathkeys;
+		innerpathkeys = inner_path->pathkeys;
 
 	/*
 	 * If specified, add a materialize node to shield the inner plan from the
 	 * need to handle mark/restore.
 	 */
 	if (best_path->materialize_inner)
 	{
 		Plan	   *matplan = (Plan *) make_material(inner_plan);
 
 		/*
@@ -5330,25 +5337,25 @@ prepare_sort_from_pathkeys(Plan *lefttree, List *pathkeys,
 
 				/*
 				 * We shouldn't be trying to sort by an equivalence class that
 				 * contains a constant, so no need to consider such cases any
 				 * further.
 				 */
 				if (em->em_is_const)
 					continue;
 
 				/*
-				 * Ignore child members unless they match the rel being
+				 * Ignore child members unless they belong to the rel being
 				 * sorted.
 				 */
 				if (em->em_is_child &&
-					!bms_equal(em->em_relids, relids))
+					!bms_is_subset(em->em_relids, relids))
 					continue;
 
 				sortexpr = em->em_expr;
 				exprvars = pull_var_clause((Node *) sortexpr,
 										   PVC_INCLUDE_AGGREGATES |
 										   PVC_INCLUDE_WINDOWFUNCS |
 										   PVC_INCLUDE_PLACEHOLDERS);
 				foreach(k, exprvars)
 				{
 					if (!tlist_member_ignore_relabel(lfirst(k), tlist))
@@ -5445,57 +5452,58 @@ find_ec_member_for_tle(EquivalenceClass *ec,
 		Expr	   *emexpr;
 
 		/*
 		 * We shouldn't be trying to sort by an equivalence class that
 		 * contains a constant, so no need to consider such cases any further.
 		 */
 		if (em->em_is_const)
 			continue;
 
 		/*
-		 * Ignore child members unless they match the rel being sorted.
+		 * Ignore child members unless they belong to the rel being sorted.
 		 */
 		if (em->em_is_child &&
-			!bms_equal(em->em_relids, relids))
+			!bms_is_subset(em->em_relids, relids))
 			continue;
 
 		/* Match if same expression (after stripping relabel) */
 		emexpr = em->em_expr;
 		while (emexpr && IsA(emexpr, RelabelType))
 			emexpr = ((RelabelType *) emexpr)->arg;
 
 		if (equal(emexpr, tlexpr))
 			return em;
 	}
 
 	return NULL;
 }
 
 /*
  * make_sort_from_pathkeys
  *	  Create sort plan to sort according to given pathkeys
  *
  *	  'lefttree' is the node which yields input tuples
  *	  'pathkeys' is the list of pathkeys by which the result is to be sorted
+ *	  'relids' is the set of relations required by prepare_sort_from_pathkeys()
  */
 static Sort *
-make_sort_from_pathkeys(Plan *lefttree, List *pathkeys)
+make_sort_from_pathkeys(Plan *lefttree, List *pathkeys, Relids relids)
 {
 	int			numsortkeys;
 	AttrNumber *sortColIdx;
 	Oid		   *sortOperators;
 	Oid		   *collations;
 	bool	   *nullsFirst;
 
 	/* Compute sort column info, and adjust lefttree as needed */
 	lefttree = prepare_sort_from_pathkeys(lefttree, pathkeys,
-										  NULL,
+										  relids,
 										  NULL,
 										  false,
 										  &numsortkeys,
 										  &sortColIdx,
 										  &sortOperators,
 										  &collations,
 										  &nullsFirst);
 
 	/* Now build the Sort node */
 	return make_sort(lefttree, numsortkeys,
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index 193b2c9..10335d2 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -1873,20 +1873,71 @@ adjust_appendrel_attrs(PlannerInfo *root, Node *node, AppendRelInfo *appinfo)
 										   appinfo);
 		}
 		result = (Node *) newnode;
 	}
 	else
 		result = adjust_appendrel_attrs_mutator(node, &context);
 
 	return result;
 }
 
+/*
+ * find_appendrelinfos_by_relids
+ * 		Find AppendRelInfo structures for all relations specified by relids.
+ */
+List *
+find_appendrelinfos_by_relids(PlannerInfo *root, Relids relids)
+{
+	ListCell	*lc;
+	List		*ari_list = NIL;
+
+	foreach (lc, root->append_rel_list)
+	{
+		AppendRelInfo *ari = lfirst(lc);
+
+		if (bms_is_member(ari->child_relid, relids))
+			ari_list = lappend(ari_list, ari);
+	}
+
+	Assert(list_length(ari_list) == bms_num_members(relids));
+	return ari_list;
+}
+
+/*
+ * adjust_partitionrel_attrs
+ *	    Replace the Var nodes in given node with the corresponding Var nodes
+ *	    of the child. Given list of AppendRelInfo nodes holds the mapping
+ *	    between parent and child Var nodes.
+ *
+ * While doing so, we need to make sure to translate non-expression information
+ * in nodes like RestrictInfo.
+ */
+Node *
+adjust_partitionrel_attrs(PlannerInfo *root, Node *node,
+						  List *append_rel_infos)
+{
+	ListCell   *lc;
+
+	/*
+	 * TODO: Instead of copying and mutating the trees one child relation at a
+	 * time, we should be able to do this en-masse for all the partitions
+	 * involved.
+	 */
+	foreach (lc, append_rel_infos)
+	{
+		AppendRelInfo	*append_rel_info = lfirst(lc);
+		node = adjust_appendrel_attrs(root, node, append_rel_info);
+	}
+
+	return node;
+}
+
 static Node *
 adjust_appendrel_attrs_mutator(Node *node,
 							   adjust_appendrel_attrs_context *context)
 {
 	AppendRelInfo *appinfo = context->appinfo;
 
 	if (node == NULL)
 		return NULL;
 	if (IsA(node, Var))
 	{
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index abb7507..ba64329 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -16,21 +16,23 @@
 
 #include <math.h>
 
 #include "miscadmin.h"
 #include "nodes/nodeFuncs.h"
 #include "optimizer/clauses.h"
 #include "optimizer/cost.h"
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
 #include "optimizer/planmain.h"
+#include "optimizer/prep.h"
 #include "optimizer/restrictinfo.h"
+#include "optimizer/tlist.h"
 #include "optimizer/var.h"
 #include "parser/parsetree.h"
 #include "utils/lsyscache.h"
 #include "utils/selfuncs.h"
 
 
 typedef enum
 {
 	COSTS_EQUAL,				/* path costs are fuzzily equal */
 	COSTS_BETTER1,				/* first path is cheaper than second */
@@ -3202,10 +3204,160 @@ reparameterize_path(PlannerInfo *root, Path *path,
 														 rel,
 														 spath->subpath,
 														 spath->path.pathkeys,
 														 required_outer);
 			}
 		default:
 			break;
 	}
 	return NULL;
 }
+
+/*
+ * reparameterize_path_for_child
+ * 		Given a path parameterized by the parent of the given relation,
+ * 		translate the path to be parameterized by the given child relation.
+ *
+ * The function creates a new path of the same type as the given path, but
+ * parameterized by the given child relation. If it can not reparameterize the
+ * path as required, it returns NULL.
+ */
+Path *
+reparameterize_path_for_child(PlannerInfo *root, Path *path,
+							  RelOptInfo *child_rel)
+{
+	Path	   *new_path;
+	ParamPathInfo   *new_ppi;
+	ParamPathInfo   *old_ppi;
+	List	   *child_aris;
+
+	/*
+	 * If the path is not parameterized by parent of the given relation, it
+	 * doesn't need reparameterization.
+	 */
+	if (!path->param_info ||
+		!bms_overlap(PATH_REQ_OUTER(path), child_rel->top_parent_relids))
+	return path;
+
+	switch (nodeTag(path))
+	{
+		case T_Path:
+			new_path = makeNode(Path);
+			memcpy(new_path, path, sizeof(Path));
+			break;
+
+		case T_HashPath:
+			new_path = (Path *) makeNode(HashPath);
+			memcpy(new_path, path, sizeof(HashPath));
+			break;
+
+		case T_MergePath:
+			new_path = (Path *) makeNode(MergePath);
+			memcpy(new_path, path, sizeof(MergePath));
+			break;
+
+		case T_NestPath:
+			new_path = (Path *) makeNode(NestPath);
+			memcpy(new_path, path, sizeof(NestPath));
+			break;
+
+		case T_IndexPath:
+			new_path = (Path *) makeNode(IndexPath);
+			memcpy(new_path, path, sizeof(IndexPath));
+			break;
+
+		case T_AppendPath:
+			new_path = (Path *) makeNode(AppendPath);
+			memcpy(new_path, path, sizeof(AppendPath));
+			break;
+
+		/*
+		 * TODO:
+		 * If this method of translation is fine add more path types here.
+		 */
+
+		default:
+			/* Path type unsupported by this function. */
+			return NULL;
+	}
+
+	/*
+	 * Gather AppendRelInfos of the base partition relations in the outer child
+	 * relation. We need those for translating parent path to that of child by
+	 * substituting parent Var nodes and relids with those of children.
+	 */
+	child_aris = find_appendrelinfos_by_relids(root, child_rel->relids);
+
+	/* Adjust the parameterization information. */
+	old_ppi = new_path->param_info;
+	new_ppi = makeNode(ParamPathInfo);
+	new_ppi->ppi_req_outer = adjust_partition_relids(bms_copy(old_ppi->ppi_req_outer),
+													 child_aris);
+	new_ppi->ppi_rows = old_ppi->ppi_rows;
+	new_ppi->ppi_clauses = (List *) adjust_partitionrel_attrs(root,
+													 (Node *) old_ppi->ppi_clauses,
+													 child_aris);
+
+	/* Adjust the path target. */
+	new_path->pathtarget = copy_pathtarget(new_path->pathtarget);
+	new_path->pathtarget->exprs = (List *) adjust_partitionrel_attrs(root,
+													(Node *) new_path->pathtarget->exprs,
+													 child_aris);
+	new_path->param_info = new_ppi;
+
+	/*
+	 * Change parameterization of sub paths recursively. Also carry out any
+	 * pathtype specific adjustments.
+	 */
+	switch (nodeTag(path))
+	{
+		case T_HashPath:
+		case T_MergePath:
+		case T_NestPath:
+			{
+				JoinPath *jpath = (JoinPath *)new_path;
+
+				jpath->outerjoinpath = reparameterize_path_for_child(root,
+														 jpath->outerjoinpath,
+														 child_rel);
+				jpath->innerjoinpath = reparameterize_path_for_child(root,
+														 jpath->innerjoinpath,
+														 child_rel);
+				jpath->joinrestrictinfo = (List *) adjust_partitionrel_attrs(root,
+															(Node *) jpath->joinrestrictinfo,
+															child_aris);
+			}
+			break;
+
+		case T_AppendPath:
+			{
+				AppendPath	*apath = (AppendPath *)new_path;
+				List		*subpaths = NIL;
+				ListCell	*lc;
+
+				foreach (lc, apath->subpaths)
+					subpaths = lappend(subpaths,
+									   reparameterize_path_for_child(root,
+																	lfirst(lc),
+																	child_rel));
+				apath->subpaths = subpaths;
+			}
+
+		case T_IndexPath:
+			{
+				IndexPath *ipath = (IndexPath *)new_path;
+
+				ipath->indexquals = (List *) adjust_partitionrel_attrs(root,
+														(Node *) ipath->indexquals,
+														child_aris);
+				ipath->indexquals = (List *) adjust_partitionrel_attrs(root,
+														(Node *) ipath->indexorderbys,
+														child_aris);
+			}
+
+		default:
+			/* Nothing to do. */
+			break;
+	}
+
+	return new_path;
+}
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index f8bfa4b..a3f27d1 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -404,20 +404,34 @@ get_relation_info(PlannerInfo *root, Oid relationObjectId, bool inhparent,
 	}
 	else
 	{
 		rel->serverid = InvalidOid;
 		rel->fdwroutine = NULL;
 	}
 
 	/* Collect info about relation's foreign keys, if relevant */
 	get_relation_foreign_keys(root, rel, relation);
 
+	/*
+	 * Lookup partition scheme for the given relation. Only parent relations
+	 * can be partitioned.
+	 */
+	if (inhparent)
+		rel->part_scheme = find_partition_scheme(root, relation);
+	else
+		rel->part_scheme = NULL;
+
+	if (rel->part_scheme)
+		rel->partexprs = build_baserel_partition_keys(relation, rel->relid);
+	else
+		rel->partexprs = NULL;
+
 	heap_close(relation, NoLock);
 
 	/*
 	 * Allow a plugin to editorialize on the info we obtained from the
 	 * catalogs.  Actions might include altering the assumed relation size,
 	 * removing an index, or adding a hypothetical index to the indexlist.
 	 */
 	if (get_relation_info_hook)
 		(*get_relation_info_hook) (root, relationObjectId, inhparent, rel);
 }
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index deef560..bd0b956 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -8,29 +8,35 @@
  *
  *
  * IDENTIFICATION
  *	  src/backend/optimizer/util/relnode.c
  *
  *-------------------------------------------------------------------------
  */
 #include "postgres.h"
 
 #include "miscadmin.h"
+#include "catalog/heap.h"
+#include "catalog/partition.h"
 #include "optimizer/clauses.h"
 #include "optimizer/cost.h"
+#include "nodes/makefuncs.h"
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
 #include "optimizer/placeholder.h"
 #include "optimizer/plancat.h"
+#include "optimizer/prep.h"
 #include "optimizer/restrictinfo.h"
 #include "optimizer/tlist.h"
+#include "rewrite/rewriteManip.h"
 #include "utils/hsearch.h"
+#include "utils/rel.h"
 
 
 typedef struct JoinHashEntry
 {
 	Relids		join_relids;	/* hash key --- MUST BE FIRST */
 	RelOptInfo *join_rel;
 } JoinHashEntry;
 
 static void build_joinrel_tlist(PlannerInfo *root, RelOptInfo *joinrel,
 					RelOptInfo *input_rel);
@@ -40,20 +46,25 @@ static List *build_joinrel_restrictlist(PlannerInfo *root,
 						   RelOptInfo *inner_rel);
 static void build_joinrel_joinlist(RelOptInfo *joinrel,
 					   RelOptInfo *outer_rel,
 					   RelOptInfo *inner_rel);
 static List *subbuild_joinrel_restrictlist(RelOptInfo *joinrel,
 							  List *joininfo_list,
 							  List *new_restrictlist);
 static List *subbuild_joinrel_joinlist(RelOptInfo *joinrel,
 						  List *joininfo_list,
 						  List *new_joininfo);
+static void set_foreign_rel_properties(RelOptInfo *joinrel,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel);
+static void build_joinrel_partition_info(RelOptInfo *joinrel,
+						  RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+						  JoinType jointype);
 
 
 /*
  * setup_simple_rel_arrays
  *	  Prepare the arrays we use for quickly accessing base relations.
  */
 void
 setup_simple_rel_arrays(PlannerInfo *root)
 {
 	Index		rti;
@@ -130,20 +141,24 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptKind reloptkind)
 	rel->serverid = InvalidOid;
 	rel->userid = rte->checkAsUser;
 	rel->useridiscurrent = false;
 	rel->fdwroutine = NULL;
 	rel->fdw_private = NULL;
 	rel->baserestrictinfo = NIL;
 	rel->baserestrictcost.startup = 0;
 	rel->baserestrictcost.per_tuple = 0;
 	rel->joininfo = NIL;
 	rel->has_eclass_joins = false;
+	rel->part_scheme = NULL;
+	rel->partexprs = NULL;
+	rel->top_parent_relids = NULL;
+	rel->part_rels = NULL;
 
 	/* Check type of rtable entry */
 	switch (rte->rtekind)
 	{
 		case RTE_RELATION:
 			/* Table --- retrieve statistics from the system catalogs */
 			get_relation_info(root, rte->relid, rte->inh, rel);
 			break;
 		case RTE_SUBQUERY:
 		case RTE_FUNCTION:
@@ -307,20 +322,70 @@ find_join_rel(PlannerInfo *root, Relids relids)
 
 			if (bms_equal(rel->relids, relids))
 				return rel;
 		}
 	}
 
 	return NULL;
 }
 
 /*
+ * set_foreign_rel_properties
+ *		Set up foreign-join fields if outer and inner relation are foreign
+ *		tables (or joins) belonging to the same server and assigned to the same
+ *		user to check access permissions as.
+ *
+ * In addition to an exact match of userid, we allow the case where one side
+ * has zero userid (implying current user) and the other side has explicit
+ * userid that happens to equal the current user; but in that case, pushdown of
+ * the join is only valid for the current user.  The useridiscurrent field
+ * records whether we had to make such an assumption for this join or any
+ * sub-join.
+ *
+ * Otherwise these fields are left invalid, so GetForeignJoinPaths will not be
+ * called for the join relation.
+ *
+ */
+static void
+set_foreign_rel_properties(RelOptInfo *joinrel, RelOptInfo *outer_rel,
+						   RelOptInfo *inner_rel)
+{
+	if (OidIsValid(outer_rel->serverid) &&
+		inner_rel->serverid == outer_rel->serverid)
+	{
+		if (inner_rel->userid == outer_rel->userid)
+		{
+			joinrel->serverid = outer_rel->serverid;
+			joinrel->userid = outer_rel->userid;
+			joinrel->useridiscurrent = outer_rel->useridiscurrent || inner_rel->useridiscurrent;
+			joinrel->fdwroutine = outer_rel->fdwroutine;
+		}
+		else if (!OidIsValid(inner_rel->userid) &&
+				 outer_rel->userid == GetUserId())
+		{
+			joinrel->serverid = outer_rel->serverid;
+			joinrel->userid = outer_rel->userid;
+			joinrel->useridiscurrent = true;
+			joinrel->fdwroutine = outer_rel->fdwroutine;
+		}
+		else if (!OidIsValid(outer_rel->userid) &&
+				 inner_rel->userid == GetUserId())
+		{
+			joinrel->serverid = outer_rel->serverid;
+			joinrel->userid = inner_rel->userid;
+			joinrel->useridiscurrent = true;
+			joinrel->fdwroutine = outer_rel->fdwroutine;
+		}
+	}
+}
+
+/*
  * build_join_rel
  *	  Returns relation entry corresponding to the union of two given rels,
  *	  creating a new relation entry if none already exists.
  *
  * 'joinrelids' is the Relids set that uniquely identifies the join
  * 'outer_rel' and 'inner_rel' are relation nodes for the relations to be
  *		joined
  * 'sjinfo': join context info
  * 'restrictlist_ptr': result variable.  If not NULL, *restrictlist_ptr
  *		receives the list of RestrictInfo nodes that apply to this
@@ -356,21 +421,25 @@ build_join_rel(PlannerInfo *root,
 														   joinrel,
 														   outer_rel,
 														   inner_rel);
 		return joinrel;
 	}
 
 	/*
 	 * Nope, so make one.
 	 */
 	joinrel = makeNode(RelOptInfo);
+
+	Assert(!IS_OTHER_REL(outer_rel->reloptkind) &&
+		   !IS_OTHER_REL(inner_rel->reloptkind));
 	joinrel->reloptkind = RELOPT_JOINREL;
+
 	joinrel->relids = bms_copy(joinrelids);
 	joinrel->rows = 0;
 	/* cheap startup cost is interesting iff not all tuples to be retrieved */
 	joinrel->consider_startup = (root->tuple_fraction > 0);
 	joinrel->consider_param_startup = false;
 	joinrel->consider_parallel = false;
 	joinrel->reltarget = create_empty_pathtarget();
 	joinrel->pathlist = NIL;
 	joinrel->ppilist = NIL;
 	joinrel->partial_pathlist = NIL;
@@ -402,61 +471,27 @@ build_join_rel(PlannerInfo *root,
 	joinrel->serverid = InvalidOid;
 	joinrel->userid = InvalidOid;
 	joinrel->useridiscurrent = false;
 	joinrel->fdwroutine = NULL;
 	joinrel->fdw_private = NULL;
 	joinrel->baserestrictinfo = NIL;
 	joinrel->baserestrictcost.startup = 0;
 	joinrel->baserestrictcost.per_tuple = 0;
 	joinrel->joininfo = NIL;
 	joinrel->has_eclass_joins = false;
+	joinrel->part_scheme = NULL;
+	joinrel->partexprs = NULL;
+	joinrel->top_parent_relids = NULL;
+	joinrel->part_rels = NULL;
 
-	/*
-	 * Set up foreign-join fields if outer and inner relation are foreign
-	 * tables (or joins) belonging to the same server and assigned to the same
-	 * user to check access permissions as.  In addition to an exact match of
-	 * userid, we allow the case where one side has zero userid (implying
-	 * current user) and the other side has explicit userid that happens to
-	 * equal the current user; but in that case, pushdown of the join is only
-	 * valid for the current user.  The useridiscurrent field records whether
-	 * we had to make such an assumption for this join or any sub-join.
-	 *
-	 * Otherwise these fields are left invalid, so GetForeignJoinPaths will
-	 * not be called for the join relation.
-	 */
-	if (OidIsValid(outer_rel->serverid) &&
-		inner_rel->serverid == outer_rel->serverid)
-	{
-		if (inner_rel->userid == outer_rel->userid)
-		{
-			joinrel->serverid = outer_rel->serverid;
-			joinrel->userid = outer_rel->userid;
-			joinrel->useridiscurrent = outer_rel->useridiscurrent || inner_rel->useridiscurrent;
-			joinrel->fdwroutine = outer_rel->fdwroutine;
-		}
-		else if (!OidIsValid(inner_rel->userid) &&
-				 outer_rel->userid == GetUserId())
-		{
-			joinrel->serverid = outer_rel->serverid;
-			joinrel->userid = outer_rel->userid;
-			joinrel->useridiscurrent = true;
-			joinrel->fdwroutine = outer_rel->fdwroutine;
-		}
-		else if (!OidIsValid(outer_rel->userid) &&
-				 inner_rel->userid == GetUserId())
-		{
-			joinrel->serverid = outer_rel->serverid;
-			joinrel->userid = inner_rel->userid;
-			joinrel->useridiscurrent = true;
-			joinrel->fdwroutine = outer_rel->fdwroutine;
-		}
-	}
+	/* Computer information relevant to the foreign relations. */
+	set_foreign_rel_properties(joinrel, outer_rel, inner_rel);
 
 	/*
 	 * Create a new tlist containing just the vars that need to be output from
 	 * this join (ie, are needed for higher joinclauses or final output).
 	 *
 	 * NOTE: the tlist order for a join rel will depend on which pair of outer
 	 * and inner rels we first try to build it from.  But the contents should
 	 * be the same regardless.
 	 */
 	build_joinrel_tlist(root, joinrel, outer_rel);
@@ -468,20 +503,24 @@ build_join_rel(PlannerInfo *root,
 	 * sets of any PlaceHolderVars computed here to direct_lateral_relids, so
 	 * now we can finish computing that.  This is much like the computation of
 	 * the transitively-closed lateral_relids in min_join_parameterization,
 	 * except that here we *do* have to consider the added PHVs.
 	 */
 	joinrel->direct_lateral_relids =
 		bms_del_members(joinrel->direct_lateral_relids, joinrel->relids);
 	if (bms_is_empty(joinrel->direct_lateral_relids))
 		joinrel->direct_lateral_relids = NULL;
 
+	/* Store the partition information. */
+	build_joinrel_partition_info(joinrel, outer_rel, inner_rel,
+								 sjinfo->jointype);
+
 	/*
 	 * Construct restrict and join clause lists for the new joinrel. (The
 	 * caller might or might not need the restrictlist, but I need it anyway
 	 * for set_joinrel_size_estimates().)
 	 */
 	restrictlist = build_joinrel_restrictlist(root, joinrel,
 											  outer_rel, inner_rel);
 	if (restrictlist_ptr)
 		*restrictlist_ptr = restrictlist;
 	build_joinrel_joinlist(joinrel, outer_rel, inner_rel);
@@ -510,58 +549,170 @@ build_join_rel(PlannerInfo *root,
 	 * assume this doesn't matter, because we should hit all the same baserels
 	 * and joinclauses while building up to this joinrel no matter which we
 	 * take; therefore, we should make the same decision here however we get
 	 * here.
 	 */
 	if (inner_rel->consider_parallel && outer_rel->consider_parallel &&
 		is_parallel_safe(root, (Node *) restrictlist) &&
 		is_parallel_safe(root, (Node *) joinrel->reltarget->exprs))
 		joinrel->consider_parallel = true;
 
-	/*
-	 * Add the joinrel to the query's joinrel list, and store it into the
-	 * auxiliary hashtable if there is one.  NB: GEQO requires us to append
-	 * the new joinrel to the end of the list!
-	 */
-	root->join_rel_list = lappend(root->join_rel_list, joinrel);
-
-	if (root->join_rel_hash)
-	{
-		JoinHashEntry *hentry;
-		bool		found;
-
-		hentry = (JoinHashEntry *) hash_search(root->join_rel_hash,
-											   &(joinrel->relids),
-											   HASH_ENTER,
-											   &found);
-		Assert(!found);
-		hentry->join_rel = joinrel;
-	}
+	/* Add the joinrel to the query's PlannerInfo. */
+	add_join_rel(root, joinrel);
 
 	/*
 	 * Also, if dynamic-programming join search is active, add the new joinrel
 	 * to the appropriate sublist.  Note: you might think the Assert on number
 	 * of members should be for equality, but some of the level 1 rels might
 	 * have been joinrels already, so we can only assert <=.
 	 */
 	if (root->join_rel_level)
 	{
 		Assert(root->join_cur_level > 0);
 		Assert(root->join_cur_level <= bms_num_members(joinrel->relids));
 		root->join_rel_level[root->join_cur_level] =
 			lappend(root->join_rel_level[root->join_cur_level], joinrel);
 	}
 
 	return joinrel;
 }
 
 /*
+ * build_partition_join_rel
+ *	  Returns relation entry corresponding to the of join between two given
+ *	  partition relations.
+ *
+ * 'outer_rel' and 'inner_rel' are the RelOptInfos of partition relations being
+ *		joined.
+ * 'parent_joinrel' is the RelOptInfo representing the join between parent
+ *		relations. Most of the members of new RelOptInfo are produced by
+ *		translating corresponding members of this RelOptInfo.
+ * 'sjinfo': context info for join between the partitions
+ * 'restrictlist': list of RestrictInfo nodes that apply to this particular
+ *		pair of joinable relations.
+ * 'join_aris': list of AppendRelInfo nodes for base child relations involved
+ *		in this join.
+ */
+RelOptInfo *
+build_partition_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
+						 RelOptInfo *inner_rel, RelOptInfo *parent_joinrel,
+						 SpecialJoinInfo *sjinfo, List *restrictlist,
+						 List *join_aris)
+{
+	List	   *tmp_exprs;
+	RelOptInfo *joinrel = makeNode(RelOptInfo);
+
+	joinrel->reloptkind = RELOPT_OTHER_JOINREL;
+	joinrel->relids = bms_union(outer_rel->relids, inner_rel->relids);
+	joinrel->rows = 0;
+	/* cheap startup cost is interesting iff not all tuples to be retrieved */
+	joinrel->consider_startup = (root->tuple_fraction > 0);
+	joinrel->consider_param_startup = false;
+	joinrel->consider_parallel = false;
+	joinrel->reltarget = create_empty_pathtarget();
+	joinrel->pathlist = NIL;
+	joinrel->ppilist = NIL;
+	joinrel->partial_pathlist = NIL;
+	joinrel->cheapest_startup_path = NULL;
+	joinrel->cheapest_total_path = NULL;
+	joinrel->cheapest_unique_path = NULL;
+	joinrel->cheapest_parameterized_paths = NIL;
+	joinrel->direct_lateral_relids = NULL;
+	joinrel->lateral_relids = NULL;
+	joinrel->relid = 0;			/* indicates not a baserel */
+	joinrel->rtekind = RTE_JOIN;
+	joinrel->min_attr = 0;
+	joinrel->max_attr = 0;
+	joinrel->attr_needed = NULL;
+	joinrel->attr_widths = NULL;
+	joinrel->lateral_vars = NIL;
+	joinrel->lateral_referencers = NULL;
+	joinrel->indexlist = NIL;
+	joinrel->pages = 0;
+	joinrel->tuples = 0;
+	joinrel->allvisfrac = 0;
+	joinrel->subroot = NULL;
+	joinrel->subplan_params = NIL;
+	joinrel->serverid = InvalidOid;
+	joinrel->userid = InvalidOid;
+	joinrel->useridiscurrent = false;
+	joinrel->fdwroutine = NULL;
+	joinrel->fdw_private = NULL;
+	joinrel->baserestrictinfo = NIL;
+	joinrel->baserestrictcost.startup = 0;
+	joinrel->baserestrictcost.per_tuple = 0;
+	joinrel->joininfo = NIL;
+	joinrel->has_eclass_joins = false;
+	joinrel->part_scheme = NULL;
+	joinrel->partexprs = NULL;
+	joinrel->top_parent_relids = NULL;
+	joinrel->part_rels = NULL;
+
+
+	/* Only joins between other relations land here. */
+	Assert(IS_OTHER_REL(outer_rel->reloptkind) &&
+		   IS_OTHER_REL(inner_rel->reloptkind));
+
+	joinrel->top_parent_relids = bms_union(outer_rel->top_parent_relids,
+										   inner_rel->top_parent_relids);
+
+	/* Computer information relevant to foreign relations. */
+	set_foreign_rel_properties(joinrel, outer_rel, inner_rel);
+
+	/*
+	 * Produce partition-wise joinrel's targetlist by translating the parent
+	 * joinrel's targetlist. This will also include the required placeholder
+	 * Vars.
+	 */
+	joinrel->reltarget = copy_pathtarget(parent_joinrel->reltarget);
+	tmp_exprs = joinrel->reltarget->exprs;
+	joinrel->reltarget->exprs = (List *) adjust_partitionrel_attrs(root, (Node *) tmp_exprs,
+															join_aris);
+
+	/*
+	 * Lateral relids directly referred in this relation will be same as that
+	 * of the parent relation.
+	 */
+	joinrel->direct_lateral_relids = (Relids) bms_copy(parent_joinrel->direct_lateral_relids);
+	joinrel->lateral_relids = (Relids) bms_copy(parent_joinrel->lateral_relids);
+
+	joinrel->joininfo = (List *) adjust_partitionrel_attrs(root,
+											 (Node *) parent_joinrel->joininfo,
+													join_aris);
+
+	/*
+	 * If the parent joinrel has pending equivalence classes, so does the
+	 * child.
+	 */
+	joinrel->has_eclass_joins = parent_joinrel->has_eclass_joins;
+
+	/* Is the join between partitions itself partitioned? */
+	build_joinrel_partition_info(joinrel, outer_rel, inner_rel, sjinfo->jointype);
+
+	/*
+	 * Set estimates of the joinrel's size.
+	 */
+	set_joinrel_size_estimates(root, joinrel, outer_rel, inner_rel, sjinfo,
+							   restrictlist);
+
+	/* Child joinrel is parallel safe if parent is parallel safe. */
+	joinrel->consider_parallel = parent_joinrel->consider_parallel;
+
+	Assert(!find_join_rel(root, joinrel->relids));
+
+	/* Add the relation to the PlannerInfo. */
+	add_join_rel(root, joinrel);
+
+	return joinrel;
+}
+
+/*
  * min_join_parameterization
  *
  * Determine the minimum possible parameterization of a joinrel, that is, the
  * set of other rels it contains LATERAL references to.  We save this value in
  * the join's RelOptInfo.  This function is split out of build_join_rel()
  * because join_is_legal() needs the value to check a prospective join.
  */
 Relids
 min_join_parameterization(PlannerInfo *root,
 						  Relids joinrelids,
@@ -1313,10 +1464,92 @@ get_appendrel_parampathinfo(RelOptInfo *appendrel, Relids required_outer)
 
 	/* Else build the ParamPathInfo */
 	ppi = makeNode(ParamPathInfo);
 	ppi->ppi_req_outer = required_outer;
 	ppi->ppi_rows = 0;
 	ppi->ppi_clauses = NIL;
 	appendrel->ppilist = lappend(appendrel->ppilist, ppi);
 
 	return ppi;
 }
+
+/*
+ * Adds given join relation to the joinrel list and also to the hashtable if
+ * there is one.
+ */
+void
+add_join_rel(PlannerInfo *root, RelOptInfo *joinrel)
+{
+	/* GEQO requires us to append the new joinrel to the end of the list! */
+	root->join_rel_list = lappend(root->join_rel_list, joinrel);
+
+	/* store it into the auxiliary hashtable if there is one. */
+	if (root->join_rel_hash)
+	{
+		JoinHashEntry *hentry;
+		bool		found;
+
+		hentry = (JoinHashEntry *) hash_search(root->join_rel_hash,
+											   &(joinrel->relids),
+											   HASH_ENTER,
+											   &found);
+		Assert(!found);
+		hentry->join_rel = joinrel;
+	}
+}
+
+/*
+ * Set up partitioning scheme and partition keys for a join between given two
+ * relations.
+ */
+static void
+build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
+							 RelOptInfo *inner_rel, JoinType jointype)
+{
+	int		num_pks;
+	int		cnt;
+
+	/*
+	 * Nothing to do if
+	 * a. partition-wise join is disabled.
+	 * b. joining relations are not partitioned.
+	 * c. partitioning schemes do not match.
+	 */
+	if (!enable_partition_wise_join ||
+		!outer_rel->part_scheme || !inner_rel->part_scheme ||
+		outer_rel->part_scheme != inner_rel->part_scheme)
+	{
+		joinrel->part_scheme = NULL;
+		return;
+	}
+
+	/*
+	 * This function will be called only once for each joinrel and thus we
+	 * should not see partition scheme, partition keys and array for storing
+	 * child relations set.
+	 */
+	Assert(!joinrel->part_scheme && !joinrel->partexprs &&
+		   !joinrel->part_rels);
+
+	num_pks = PartitionSchemeGetNumKeys(outer_rel->part_scheme);
+
+	/* Join relation is partitioned in the same way as the joining relation. */
+	joinrel->part_scheme = outer_rel->part_scheme;
+
+	/*
+	 * Collect the partition key expressions. An OUTER join will produce rows
+	 * where the partition key columns of inner side are NULL and may not fit
+	 * the partitioning scheme with inner partition keys. Since two NULL values
+	 * are not considered equal, an equi-join involing inner partition keys
+	 * still prohibits cross-partition joins while joining with another
+	 * similarly partitioned relation.
+	 */
+	joinrel->partexprs = (List **) palloc0(sizeof(List *) * num_pks);
+	for (cnt = 0; cnt < num_pks; cnt++)
+	{
+		List *pkexpr = list_copy(outer_rel->partexprs[cnt]);
+
+		pkexpr = list_concat(pkexpr,
+							 list_copy(inner_rel->partexprs[cnt]));
+		joinrel->partexprs[cnt] = pkexpr;
+	}
+}
diff --git a/src/backend/utils/adt/selfuncs.c b/src/backend/utils/adt/selfuncs.c
index 56943f2..16b2eac 100644
--- a/src/backend/utils/adt/selfuncs.c
+++ b/src/backend/utils/adt/selfuncs.c
@@ -3405,21 +3405,23 @@ estimate_num_groups(PlannerInfo *root, List *groupExprs, double input_rows,
 			else
 			{
 				/* not time to process varinfo2 yet */
 				newvarinfos = lcons(varinfo2, newvarinfos);
 			}
 		}
 
 		/*
 		 * Sanity check --- don't divide by zero if empty relation.
 		 */
-		Assert(rel->reloptkind == RELOPT_BASEREL);
+		Assert(rel->reloptkind == RELOPT_BASEREL ||
+			   rel->reloptkind == RELOPT_OTHER_MEMBER_REL);
+
 		if (rel->tuples > 0)
 		{
 			/*
 			 * Clamp to size of rel, or size of rel / 10 if multiple Vars. The
 			 * fudge factor is because the Vars are probably correlated but we
 			 * don't know by how much.  We should never clamp to less than the
 			 * largest ndistinct value for any of the Vars, though, since
 			 * there will surely be at least that many groups.
 			 */
 			double		clamp = rel->tuples;
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index ce4eef9..edc7e58 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -870,20 +870,29 @@ static struct config_bool ConfigureNamesBool[] =
 	},
 	{
 		{"enable_hashjoin", PGC_USERSET, QUERY_TUNING_METHOD,
 			gettext_noop("Enables the planner's use of hash join plans."),
 			NULL
 		},
 		&enable_hashjoin,
 		true,
 		NULL, NULL, NULL
 	},
+	{
+		{"enable_partition_wise_join", PGC_USERSET, QUERY_TUNING_METHOD,
+			gettext_noop("Enables partition-wise join."),
+			NULL
+		},
+		&enable_partition_wise_join,
+		true,
+		NULL, NULL, NULL
+	},
 
 	{
 		{"geqo", PGC_USERSET, QUERY_TUNING_GEQO,
 			gettext_noop("Enables genetic query optimization."),
 			gettext_noop("This algorithm attempts to do planning without "
 						 "exhaustive searching.")
 		},
 		&enable_geqo,
 		true,
 		NULL, NULL, NULL
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 81a4b91..3c59619 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -41,20 +41,24 @@ typedef struct BoundCollectionData *BoundCollection;
  */
 typedef struct PartitionDescData
 {
 	int					nparts;		/* Number of partitions */
 	Oid				   *oids;		/* OIDs of partitions */
 	BoundCollection		bounds;		/* collection of list or range bounds */
 } PartitionDescData;
 
 typedef struct PartitionDescData *PartitionDesc;
 typedef struct PartitionTreeNodeData *PartitionTreeNode;
+typedef struct PartitionSchemeData *PartitionScheme;
+
+/* Include here to avoid circular dependency with relation.h. */
+struct PlannerInfo;
 
 /* relcache support for partition key information */
 extern void RelationBuildPartitionKey(Relation relation);
 
 /* Partition key inquiry functions */
 extern int get_partition_key_strategy(PartitionKey key);
 extern int get_partition_key_natts(PartitionKey key);
 extern List *get_partition_key_exprs(PartitionKey key);
 
 /* Partition key inquiry functions - for a given column */
@@ -77,11 +81,18 @@ extern List *get_qual_from_partbound(Relation rel, Relation parent, Node *bound)
 extern List *RelationGetPartitionQual(Relation rel, bool recurse);
 
 /* For tuple routing */
 extern PartitionTreeNode RelationGetPartitionTreeNode(Relation rel);
 extern List *get_leaf_partition_oids_v2(PartitionTreeNode ptnode);
 
 extern int get_partition_for_tuple(PartitionTreeNode ptnode,
 					TupleTableSlot *slot,
 					EState *estate,
 					Oid *failed_at);
+extern List **build_baserel_partition_keys(Relation relation, Index varno);
+extern PartitionScheme find_partition_scheme(struct PlannerInfo *root,
+											 Relation rel);
+extern int PartitionSchemeGetNumParts(PartitionScheme part_scheme);
+extern int PartitionSchemeGetNumKeys(PartitionScheme part_scheme);
+extern Oid *PartitionDescGetPartOids(PartitionDesc part_desc);
+
 #endif   /* PARTITION_H */
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 3a1255a..b00eedc 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -8,20 +8,21 @@
  * Portions Copyright (c) 1994, Regents of the University of California
  *
  * src/include/nodes/relation.h
  *
  *-------------------------------------------------------------------------
  */
 #ifndef RELATION_H
 #define RELATION_H
 
 #include "access/sdir.h"
+#include "catalog/partition.h"
 #include "lib/stringinfo.h"
 #include "nodes/params.h"
 #include "nodes/parsenodes.h"
 #include "storage/block.h"
 
 
 /*
  * Relids
  *		Set of relation identifiers (indexes into the rangetable).
  */
@@ -256,20 +257,23 @@ typedef struct PlannerInfo
 
 	List	   *query_pathkeys; /* desired pathkeys for query_planner() */
 
 	List	   *group_pathkeys; /* groupClause pathkeys, if any */
 	List	   *window_pathkeys;	/* pathkeys of bottom window, if any */
 	List	   *distinct_pathkeys;		/* distinctClause pathkeys, if any */
 	List	   *sort_pathkeys;	/* sortClause pathkeys, if any */
 
 	List	   *initial_rels;	/* RelOptInfos we are now trying to join */
 
+	List	   *part_schemes;	/* Canonicalised partition schemes
+								 * used in the query. */
+
 	/* Use fetch_upper_rel() to get any particular upper rel */
 	List	   *upper_rels[UPPERREL_FINAL + 1]; /* upper-rel RelOptInfos */
 
 	/* Result tlists chosen by grouping_planner for upper-stage processing */
 	struct PathTarget *upper_targets[UPPERREL_FINAL + 1];
 
 	/*
 	 * grouping_planner passes back its final processed targetlist here, for
 	 * use in relabeling the topmost tlist of the finished Plan.
 	 */
@@ -345,20 +349,26 @@ typedef struct PlannerInfo
  * is present in the query join tree but the members are not.  The member
  * RTEs and otherrels are used to plan the scans of the individual tables or
  * subqueries of the append set; then the parent baserel is given Append
  * and/or MergeAppend paths comprising the best paths for the individual
  * member rels.  (See comments for AppendRelInfo for more information.)
  *
  * At one time we also made otherrels to represent join RTEs, for use in
  * handling join alias Vars.  Currently this is not needed because all join
  * alias Vars are expanded to non-aliased form during preprocess_expression.
  *
+ * We also have relations representing pair-wise joins between partitions of
+ * partitioned tables. These relations are not added to join_rel_level lists
+ * as they are not joined directly by the dynamic programming algorithm.
+ * Adding these two join_rel_level list also means that top level list has more
+ * than one join relation, which is symantically incorrect.
+ *
  * There is also a RelOptKind for "upper" relations, which are RelOptInfos
  * that describe post-scan/join processing steps, such as aggregation.
  * Many of the fields in these RelOptInfos are meaningless, but their Path
  * fields always hold Paths showing ways to do that processing step.
  *
  * Lastly, there is a RelOptKind for "dead" relations, which are base rels
  * that we have proven we don't need to join after all.
  *
  * Parts of this data structure are specific to various scan and join
  * mechanisms.  It didn't seem worth creating new node types for them.
@@ -464,24 +474,29 @@ typedef struct PlannerInfo
  * We store baserestrictcost in the RelOptInfo (for base relations) because
  * we know we will need it at least once (to price the sequential scan)
  * and may need it multiple times to price index scans.
  *----------
  */
 typedef enum RelOptKind
 {
 	RELOPT_BASEREL,
 	RELOPT_JOINREL,
 	RELOPT_OTHER_MEMBER_REL,
+	RELOPT_OTHER_JOINREL,
 	RELOPT_UPPER_REL,
 	RELOPT_DEADREL
 } RelOptKind;
 
+#define IS_OTHER_REL(reloptkind) \
+	((reloptkind) == RELOPT_OTHER_MEMBER_REL || \
+	 (reloptkind) == RELOPT_OTHER_JOINREL)
+
 typedef struct RelOptInfo
 {
 	NodeTag		type;
 
 	RelOptKind	reloptkind;
 
 	/* all relations included in this RelOptInfo */
 	Relids		relids;			/* set of base relids (rangetable indexes) */
 
 	/* size estimates generated by planner */
@@ -535,20 +550,37 @@ typedef struct RelOptInfo
 	struct FdwRoutine *fdwroutine;
 	void	   *fdw_private;
 
 	/* used by various scans and joins: */
 	List	   *baserestrictinfo;		/* RestrictInfo structures (if base
 										 * rel) */
 	QualCost	baserestrictcost;		/* cost of evaluating the above */
 	List	   *joininfo;		/* RestrictInfo structures for join clauses
 								 * involving this rel */
 	bool		has_eclass_joins;		/* T means joininfo is incomplete */
+
+	/* For partitioned relations, joins or base relations. NULL otherwise. */
+	PartitionScheme	    part_scheme;	/* Partitioning scheme. */
+	struct RelOptInfo **part_rels;		/* Array of RelOptInfos of partitions,
+										 * stored in the same order as bounds
+										 * or lists in PartitionScheme.
+										 */
+	List			  **partexprs;		/* Array of list of partition key
+										 * expressions. For base relations
+										 * these are one element lists. For
+										 * join there can be as many elements
+										 * as there are number of joining
+										 * relations.
+										 */
+
+	/* Set only for "other" base or join relations. */
+	Relids		top_parent_relids;
 } RelOptInfo;
 
 /*
  * IndexOptInfo
  *		Per-index information for planning/optimization
  *
  *		indexkeys[], indexcollations[], opfamily[], and opcintype[]
  *		each have ncolumns entries.
  *
  *		sortopfamily[], reverse_sort[], and nulls_first[] likewise have
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 2a4df2f..1069726 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -59,20 +59,21 @@ extern bool enable_seqscan;
 extern bool enable_indexscan;
 extern bool enable_indexonlyscan;
 extern bool enable_bitmapscan;
 extern bool enable_tidscan;
 extern bool enable_sort;
 extern bool enable_hashagg;
 extern bool enable_nestloop;
 extern bool enable_material;
 extern bool enable_mergejoin;
 extern bool enable_hashjoin;
+extern bool enable_partition_wise_join;
 extern int	constraint_exclusion;
 
 extern double clamp_row_est(double nrows);
 extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
 					double index_pages, PlannerInfo *root);
 extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 			 ParamPathInfo *param_info);
 extern void cost_samplescan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 				ParamPathInfo *param_info);
 extern void cost_index(IndexPath *path, PlannerInfo *root,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 71d9154..2ab31d3 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -222,20 +222,22 @@ extern ModifyTablePath *create_modifytable_path(PlannerInfo *root,
 						List *rowMarks, OnConflictExpr *onconflict,
 						int epqParam);
 extern LimitPath *create_limit_path(PlannerInfo *root, RelOptInfo *rel,
 				  Path *subpath,
 				  Node *limitOffset, Node *limitCount,
 				  int64 offset_est, int64 count_est);
 
 extern Path *reparameterize_path(PlannerInfo *root, Path *path,
 					Relids required_outer,
 					double loop_count);
+extern Path *reparameterize_path_for_child(PlannerInfo *root, Path *path,
+					RelOptInfo *child_rel);
 
 /*
  * prototypes for relnode.c
  */
 extern void setup_simple_rel_arrays(PlannerInfo *root);
 extern RelOptInfo *build_simple_rel(PlannerInfo *root, int relid,
 				 RelOptKind reloptkind);
 extern RelOptInfo *find_base_rel(PlannerInfo *root, int relid);
 extern RelOptInfo *find_join_rel(PlannerInfo *root, Relids relids);
 extern RelOptInfo *build_join_rel(PlannerInfo *root,
@@ -260,12 +262,17 @@ extern ParamPathInfo *get_baserel_parampathinfo(PlannerInfo *root,
 						  Relids required_outer);
 extern ParamPathInfo *get_joinrel_parampathinfo(PlannerInfo *root,
 						  RelOptInfo *joinrel,
 						  Path *outer_path,
 						  Path *inner_path,
 						  SpecialJoinInfo *sjinfo,
 						  Relids required_outer,
 						  List **restrict_clauses);
 extern ParamPathInfo *get_appendrel_parampathinfo(RelOptInfo *appendrel,
 							Relids required_outer);
+extern RelOptInfo *build_partition_join_rel(PlannerInfo *root,
+						 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+						 RelOptInfo *parent_joinrel, SpecialJoinInfo *sjinfo,
+						 List *restrictlist, List *join_aris);
+extern void add_join_rel(PlannerInfo *root, RelOptInfo *joinrel);
 
 #endif   /* PATHNODE_H */
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 44abe83..a799428 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -212,11 +212,13 @@ extern List *make_inner_pathkeys_for_merge(PlannerInfo *root,
 							  List *mergeclauses,
 							  List *outer_pathkeys);
 extern List *truncate_useless_pathkeys(PlannerInfo *root,
 						  RelOptInfo *rel,
 						  List *pathkeys);
 extern bool has_useful_pathkeys(PlannerInfo *root, RelOptInfo *rel);
 extern PathKey *make_canonical_pathkey(PlannerInfo *root,
 					   EquivalenceClass *eclass, Oid opfamily,
 					   int strategy, bool nulls_first);
 
+extern Relids adjust_partition_relids(Relids relids, List *append_rel_infos);
+
 #endif   /* PATHS_H */
diff --git a/src/include/optimizer/prep.h b/src/include/optimizer/prep.h
index fb35b68..7b149c3 100644
--- a/src/include/optimizer/prep.h
+++ b/src/include/optimizer/prep.h
@@ -21,20 +21,23 @@
 /*
  * prototypes for prepjointree.c
  */
 extern void pull_up_sublinks(PlannerInfo *root);
 extern void inline_set_returning_functions(PlannerInfo *root);
 extern void pull_up_subqueries(PlannerInfo *root);
 extern void flatten_simple_union_all(PlannerInfo *root);
 extern void reduce_outer_joins(PlannerInfo *root);
 extern Relids get_relids_in_jointree(Node *jtnode, bool include_joins);
 extern Relids get_relids_for_join(PlannerInfo *root, int joinrelid);
+extern Node *adjust_partitionrel_attrs(PlannerInfo *root, Node *node,
+									   List *append_rel_infos);
+extern List *find_appendrelinfos_by_relids(PlannerInfo *root, Relids relids);
 
 /*
  * prototypes for prepqual.c
  */
 extern Node *negate_clause(Node *node);
 extern Expr *canonicalize_qual(Expr *qual);
 
 /*
  * prototypes for prepsecurity.c
  */
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
new file mode 100644
index 0000000..0eecade
--- /dev/null
+++ b/src/test/regress/expected/partition_join.out
@@ -0,0 +1,7031 @@
+--
+-- PARTITION_JOIN
+-- Test partition-wise join between partitioned tables
+--
+--
+-- partitioned by a single column
+--
+CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES START (0) END (250);
+CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES START (500) END (600);
+CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES START (250) END (500);
+INSERT INTO prt1 SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1 AS SELECT * FROM prt1;
+CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES START (0) END (250);
+CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES START (250) END (500);
+CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES START (500) END (600);
+INSERT INTO prt2 SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+CREATE TABLE uprt2 AS SELECT * FROM prt2;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on public.prt2_p1 t2
+                     Output: t2.b, t2.c
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+               Hash Cond: (t2_1.b = t1_2.a)
+               ->  Seq Scan on public.prt2_p2 t2_1
+                     Output: t2_1.b, t2_1.c
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+               Hash Cond: (t2_2.b = t1_1.a)
+               ->  Seq Scan on public.prt2_p3 t2_2
+                     Output: t2_2.b, t2_2.c
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0150 | 150 | 0150
+ 300 | 0300 | 300 | 0300
+ 450 | 0450 | 450 | 0450
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on public.prt2_p1 t2
+                     Output: t2.b, t2.c
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+               Hash Cond: (t2_1.b = t1_2.a)
+               ->  Seq Scan on public.prt2_p2 t2_1
+                     Output: t2_1.b, t2_1.c
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+               Hash Cond: (t2_2.b = t1_1.a)
+               ->  Seq Scan on public.prt2_p3 t2_2
+                     Output: t2_2.b, t2_2.c
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+  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 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: (t1.a = t2.b)
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t2.b, t2.c
+                           ->  Seq Scan on public.prt2_p1 t2
+                                 Output: t2.b, t2.c
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Hash Cond: (t1_2.a = t2_1.b)
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c
+                           ->  Seq Scan on public.prt2_p2 t2_1
+                                 Output: t2_1.b, t2_1.c
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Hash Cond: (t1_1.a = t2_2.b)
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c
+                           ->  Seq Scan on public.prt2_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+                                 Filter: ((t2_2.b % 25) = 0)
+(36 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+  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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+   Sort Key: prt1_p1.a, prt2_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+               Hash Cond: (prt1_p1.a = prt2_p1.b)
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c
+                     Filter: ((prt1_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p1.b, prt2_p1.c
+                     ->  Seq Scan on public.prt2_p1
+                           Output: prt2_p1.b, prt2_p1.c
+                           Filter: ((prt2_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt1_p2.a = prt2_p2.b)
+               ->  Seq Scan on public.prt1_p2
+                     Output: prt1_p2.a, prt1_p2.c
+                     Filter: ((prt1_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p2.b, prt2_p2.c
+                     ->  Seq Scan on public.prt2_p2
+                           Output: prt2_p2.b, prt2_p2.c
+                           Filter: ((prt2_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c
+               Hash Cond: (prt1_p3.a = prt2_p3.b)
+               ->  Seq Scan on public.prt1_p3
+                     Output: prt1_p3.a, prt1_p3.c
+                     Filter: ((prt1_p3.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p3.b, prt2_p3.c
+                     ->  Seq Scan on public.prt2_p3
+                           Output: prt2_p3.b, prt2_p3.c
+                           Filter: ((prt2_p3.b % 25) = 0)
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 t2 WHERE t2.b % 25 = 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)
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+   Sort Key: prt1_p1.a, prt2_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+               Hash Cond: (prt1_p1.a = prt2_p1.b)
+               Filter: (((50) = prt1_p1.b) OR ((75) = prt2_p1.b))
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c, prt1_p1.b, 50
+                     Filter: ((prt1_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p1.b, prt2_p1.c, (75)
+                     ->  Seq Scan on public.prt2_p1
+                           Output: prt2_p1.b, prt2_p1.c, 75
+                           Filter: ((prt2_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt1_p2.a = prt2_p2.b)
+               Filter: (((50) = prt1_p2.b) OR ((75) = prt2_p2.b))
+               ->  Seq Scan on public.prt1_p2
+                     Output: prt1_p2.a, prt1_p2.c, prt1_p2.b, 50
+                     Filter: ((prt1_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p2.b, prt2_p2.c, (75)
+                     ->  Seq Scan on public.prt2_p2
+                           Output: prt2_p2.b, prt2_p2.c, 75
+                           Filter: ((prt2_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c
+               Hash Cond: (prt1_p3.a = prt2_p3.b)
+               Filter: (((50) = prt1_p3.b) OR ((75) = prt2_p3.b))
+               ->  Seq Scan on public.prt1_p3
+                     Output: prt1_p3.a, prt1_p3.c, prt1_p3.b, 50
+                     Filter: ((prt1_p3.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p3.b, prt2_p3.c, (75)
+                     ->  Seq Scan on public.prt2_p3
+                           Output: prt2_p3.b, prt2_p3.c, 75
+                           Filter: ((prt2_p3.b % 25) = 0)
+(40 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+ a  |  c   | b  |  c   
+----+------+----+------
+ 50 | 0050 |    | 
+    |      | 75 | 0075
+(2 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+ a  |  c   | b  |  c   
+----+------+----+------
+ 50 | 0050 |    | 
+    |      | 75 | 0075
+(2 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, (25), prt2_p1.b, prt2_p1.c, (50)
+   Sort Key: prt1_p1.a, prt2_p1.b
+   ->  Result
+         Output: prt1_p1.a, prt1_p1.c, (25), prt2_p1.b, prt2_p1.c, (50)
+         ->  Append
+               ->  Hash Full Join
+                     Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, (25), (50)
+                     Hash Cond: (prt1_p1.a = prt2_p1.b)
+                     ->  Seq Scan on public.prt1_p1
+                           Output: prt1_p1.a, prt1_p1.c, 25
+                           Filter: ((prt1_p1.a % 25) = 0)
+                     ->  Hash
+                           Output: prt2_p1.b, prt2_p1.c, (50)
+                           ->  Seq Scan on public.prt2_p1
+                                 Output: prt2_p1.b, prt2_p1.c, 50
+                                 Filter: ((prt2_p1.b % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c, (25), (50)
+                     Hash Cond: (prt1_p2.a = prt2_p2.b)
+                     ->  Seq Scan on public.prt1_p2
+                           Output: prt1_p2.a, prt1_p2.c, 25
+                           Filter: ((prt1_p2.a % 25) = 0)
+                     ->  Hash
+                           Output: prt2_p2.b, prt2_p2.c, (50)
+                           ->  Seq Scan on public.prt2_p2
+                                 Output: prt2_p2.b, prt2_p2.c, 50
+                                 Filter: ((prt2_p2.b % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c, (25), (50)
+                     Hash Cond: (prt1_p3.a = prt2_p3.b)
+                     ->  Seq Scan on public.prt1_p3
+                           Output: prt1_p3.a, prt1_p3.c, 25
+                           Filter: ((prt1_p3.a % 25) = 0)
+                     ->  Hash
+                           Output: prt2_p3.b, prt2_p3.c, (50)
+                           ->  Seq Scan on public.prt2_p3
+                                 Output: prt2_p3.b, prt2_p3.c, 50
+                                 Filter: ((prt2_p3.b % 25) = 0)
+(39 rows)
+
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  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)
+
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM uprt2 WHERE uprt2.b % 25 = 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)
+
+-- Join with pruned partitions from joining relations
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on public.prt2_p2 t2
+                     Output: t2.b, t2.c
+                     Filter: (t2.b > 250)
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.prt1_p2 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a < 450) AND ((t1.a % 25) = 0))
+(15 rows)
+
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+(1 row)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a < 450 AND t2.b > 250 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+(1 row)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                    QUERY PLAN                                    
+----------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, b, c
+   Sort Key: prt1_p1.a, b
+   ->  Append
+         ->  Nested Loop Left Join
+               Output: prt1_p1.a, prt1_p1.c, b, c
+               Join Filter: (prt1_p1.a = b)
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c
+                     Filter: ((prt1_p1.a < 450) AND ((prt1_p1.a % 25) = 0))
+               ->  Result
+                     Output: b, c
+                     One-Time Filter: false
+         ->  Hash Right Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt2_p2.b = prt1_p2.a)
+               ->  Seq Scan on public.prt2_p2
+                     Output: prt2_p2.b, prt2_p2.c
+                     Filter: (prt2_p2.b > 250)
+               ->  Hash
+                     Output: prt1_p2.a, prt1_p2.c
+                     ->  Seq Scan on public.prt1_p2
+                           Output: prt1_p2.a, prt1_p2.c
+                           Filter: ((prt1_p2.a < 450) AND ((prt1_p2.a % 25) = 0))
+(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.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.a % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 5) and (b 4) is considered for partition-wise join.
+                                       QUERY PLAN                                       
+----------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+   Sort Key: prt1_p2.a, prt2_p2.b
+   ->  Result
+         Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: prt2_p2.b, prt2_p2.c, prt1_p2.a, prt1_p2.c
+                     Hash Cond: (prt1_p2.a = prt2_p2.b)
+                     ->  Seq Scan on public.prt1_p2
+                           Output: prt1_p2.a, prt1_p2.c
+                           Filter: (prt1_p2.a < 450)
+                     ->  Hash
+                           Output: prt2_p2.b, prt2_p2.c
+                           ->  Seq Scan on public.prt2_p2
+                                 Output: prt2_p2.b, prt2_p2.c
+                                 Filter: ((prt2_p2.b > 250) AND ((prt2_p2.a % 25) = 0))
+               ->  Nested Loop Left Join
+                     Output: prt2_p3.b, prt2_p3.c, a, c
+                     Join Filter: (a = prt2_p3.b)
+                     ->  Seq Scan on public.prt2_p3
+                           Output: prt2_p3.b, prt2_p3.c
+                           Filter: ((prt2_p3.b > 250) AND ((prt2_p3.a % 25) = 0))
+                     ->  Result
+                           Output: a, c
+                           One-Time Filter: false
+(26 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 5) and (b 4) is considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+     |      | 375 | 0375
+     |      | 450 | 0450
+     |      | 525 | 0525
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+     |      | 375 | 0375
+     |      | 450 | 0450
+     |      | 525 | 0525
+(4 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                    QUERY PLAN                                    
+----------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, b, c
+   Sort Key: prt1_p1.a, b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_p1.a, prt1_p1.c, b, c
+               Hash Cond: (prt1_p1.a = b)
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c
+                     Filter: ((prt1_p1.a < 450) AND ((prt1_p1.a % 25) = 0))
+               ->  Hash
+                     Output: b, c
+                     ->  Result
+                           Output: b, c
+                           One-Time Filter: false
+         ->  Hash Full Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt1_p2.a = prt2_p2.b)
+               ->  Seq Scan on public.prt1_p2
+                     Output: prt1_p2.a, prt1_p2.c
+                     Filter: ((prt1_p2.a < 450) AND ((prt1_p2.a % 25) = 0))
+               ->  Hash
+                     Output: prt2_p2.b, prt2_p2.c
+                     ->  Seq Scan on public.prt2_p2
+                           Output: prt2_p2.b, prt2_p2.c
+                           Filter: ((prt2_p2.b > 250) AND ((prt2_p2.b % 25) = 0))
+         ->  Hash Full Join
+               Output: a, c, prt2_p3.b, prt2_p3.c
+               Hash Cond: (prt2_p3.b = a)
+               ->  Seq Scan on public.prt2_p3
+                     Output: prt2_p3.b, prt2_p3.c
+                     Filter: ((prt2_p3.b > 250) AND ((prt2_p3.b % 25) = 0))
+               ->  Hash
+                     Output: a, c
+                     ->  Result
+                           Output: a, c
+                           One-Time Filter: false
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b 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)
+
+-- joins where one of the relations is proven empty
+EXPLAIN (VERBOSE, 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;
+            QUERY PLAN            
+----------------------------------
+ Result
+   Output: t1.a, t1.c, t2.b, t2.c
+   One-Time Filter: false
+(3 rows)
+
+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;
+ a | c | b | c 
+---+---+---+---
+(0 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 LEFT JOIN prt2 t2 ON t1.a = t2.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+              QUERY PLAN              
+--------------------------------------
+ Result
+   Output: prt1.a, prt1.c, t2.b, t2.c
+   One-Time Filter: false
+(3 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 LEFT JOIN prt2 t2 ON t1.a = t2.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+ a | c | b | c 
+---+---+---+---
+(0 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 4) is considered for partition-wise join.
+                    QUERY PLAN                     
+---------------------------------------------------
+ Sort
+   Output: a, c, t2.b, t2.c
+   Sort Key: a, t2.b
+   ->  Hash Left Join
+         Output: a, c, t2.b, t2.c
+         Hash Cond: (t2.b = a)
+         ->  Append
+               ->  Seq Scan on public.prt2 t2
+                     Output: t2.b, t2.c
+                     Filter: ((t2.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p1 t2_1
+                     Output: t2_1.b, t2_1.c
+                     Filter: ((t2_1.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p2 t2_2
+                     Output: t2_2.b, t2_2.c
+                     Filter: ((t2_2.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p3 t2_3
+                     Output: t2_3.b, t2_3.c
+                     Filter: ((t2_3.a % 25) = 0)
+         ->  Hash
+               Output: a, c
+               ->  Result
+                     Output: a, c
+                     One-Time Filter: false
+(24 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 4) is considered for partition-wise join.
+ a | c |  b  |  c   
+---+---+-----+------
+   |   |   0 | 0000
+   |   |  75 | 0075
+   |   | 150 | 0150
+   |   | 225 | 0225
+   |   | 300 | 0300
+   |   | 375 | 0375
+   |   | 450 | 0450
+   |   | 525 | 0525
+(8 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a = 1 AND a = 2) t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+ a | c |  b  |  c   
+---+---+-----+------
+   |   |   0 | 0000
+   |   |  75 | 0075
+   |   | 150 | 0150
+   |   | 225 | 0225
+   |   | 300 | 0300
+   |   | 375 | 0375
+   |   | 450 | 0450
+   |   | 525 | 0525
+(8 rows)
+
+EXPLAIN (VERBOSE, 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 % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 4) is considered for partition-wise join.
+                    QUERY PLAN                     
+---------------------------------------------------
+ Sort
+   Output: a, c, t2.b, t2.c
+   Sort Key: a, t2.b
+   ->  Hash Left Join
+         Output: a, c, t2.b, t2.c
+         Hash Cond: (t2.b = a)
+         ->  Append
+               ->  Seq Scan on public.prt2 t2
+                     Output: t2.b, t2.c
+                     Filter: ((t2.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p1 t2_1
+                     Output: t2_1.b, t2_1.c
+                     Filter: ((t2_1.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p2 t2_2
+                     Output: t2_2.b, t2_2.c
+                     Filter: ((t2_2.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p3 t2_3
+                     Output: t2_3.b, t2_3.c
+                     Filter: ((t2_3.a % 25) = 0)
+         ->  Hash
+               Output: a, c
+               ->  Result
+                     Output: a, c
+                     One-Time Filter: false
+(24 rows)
+
+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 % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 4) is considered for partition-wise join.
+ a | c |  b  |  c   
+---+---+-----+------
+   |   |   0 | 0000
+   |   |  75 | 0075
+   |   | 150 | 0150
+   |   | 225 | 0225
+   |   | 300 | 0300
+   |   | 375 | 0375
+   |   | 450 | 0450
+   |   | 525 | 0525
+(8 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a = 1 AND a = 2) t1 FULL JOIN uprt2 t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+ a | c |  b  |  c   
+---+---+-----+------
+   |   |   0 | 0000
+   |   |  75 | 0075
+   |   | 150 | 0150
+   |   | 225 | 0225
+   |   | 300 | 0300
+   |   | 375 | 0375
+   |   | 450 | 0450
+   |   | 525 | 0525
+(8 rows)
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 1) and (b 3) is considered for partition-wise join.
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.a = t1_3.b)
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_3.b
+                     ->  Seq Scan on public.prt2_p1 t1_3
+                           Output: t1_3.b
+                           Filter: ((t1_3.b % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.a = t1_4.b)
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_4.b
+                     ->  Seq Scan on public.prt2_p2 t1_4
+                           Output: t1_4.b
+                           Filter: ((t1_4.b % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.a = t1_5.b)
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_5.b
+                     ->  Seq Scan on public.prt2_p3 t1_5
+                           Output: t1_5.b
+                           Filter: ((t1_5.b % 25) = 0)
+(37 rows)
+
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 1) and (b 3) is considered for partition-wise join.
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+-- lateral reference
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4 5) is considered for partition-wise join.
+                                             QUERY PLAN                                              
+-----------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   Sort Key: t1.a
+   ->  Result
+         Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+         ->  Append
+               ->  Nested Loop Left Join
+                     Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a))
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.b, t1.c
+                           Filter: ((t1.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+                           Hash Cond: (t3.b = t2.a)
+                           ->  Seq Scan on public.prt2_p1 t3
+                                 Output: t3.a, t3.b
+                           ->  Hash
+                                 Output: t2.a
+                                 ->  Seq Scan on public.prt1_p1 t2
+                                       Output: t2.a
+                                       Filter: (t1.a = t2.a)
+               ->  Nested Loop Left Join
+                     Output: t1_2.a, t1_2.b, t1_2.c, t2_2.a, t3_1.a, (LEAST(t1_2.a, t2_2.a, t3_1.a))
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.b, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2_2.a, t3_1.a, LEAST(t1_2.a, t2_2.a, t3_1.a)
+                           Hash Cond: (t3_1.b = t2_2.a)
+                           ->  Seq Scan on public.prt2_p2 t3_1
+                                 Output: t3_1.a, t3_1.b
+                           ->  Hash
+                                 Output: t2_2.a
+                                 ->  Seq Scan on public.prt1_p2 t2_2
+                                       Output: t2_2.a
+                                       Filter: (t1_2.a = t2_2.a)
+               ->  Nested Loop Left Join
+                     Output: t1_1.a, t1_1.b, t1_1.c, t2_1.a, t3_2.a, (LEAST(t1_1.a, t2_1.a, t3_2.a))
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.b, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2_1.a, t3_2.a, LEAST(t1_1.a, t2_1.a, t3_2.a)
+                           Hash Cond: (t3_2.b = t2_1.a)
+                           ->  Seq Scan on public.prt2_p3 t3_2
+                                 Output: t3_2.a, t3_2.b
+                           ->  Hash
+                                 Output: t2_1.a
+                                 ->  Seq Scan on public.prt1_p3 t2_1
+                                       Output: t2_1.a
+                                       Filter: (t1_1.a = t2_1.a)
+(51 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4 5) is considered for partition-wise join.
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   Sort Key: t1.a
+   ->  Nested Loop Left Join
+         Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+         ->  Append
+               ->  Seq Scan on public.prt1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Seq Scan on public.prt1_p1 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Seq Scan on public.prt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Seq Scan on public.prt1_p2 t1_3
+                     Output: t1_3.a, t1_3.b, t1_3.c
+                     Filter: ((t1_3.a % 25) = 0)
+         ->  Append
+               ->  Hash Join
+                     Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+                     Hash Cond: (t3.b = t2.a)
+                     ->  Seq Scan on public.prt2_p1 t3
+                           Output: t3.a, t3.b
+                     ->  Hash
+                           Output: t2.a
+                           ->  Seq Scan on public.prt1_p1 t2
+                                 Output: t2.a
+                                 Filter: (t1.b = t2.a)
+               ->  Hash Join
+                     Output: t2_2.a, t3_1.a, LEAST(t1.a, t2_2.a, t3_1.a)
+                     Hash Cond: (t3_1.b = t2_2.a)
+                     ->  Seq Scan on public.prt2_p2 t3_1
+                           Output: t3_1.a, t3_1.b
+                     ->  Hash
+                           Output: t2_2.a
+                           ->  Seq Scan on public.prt1_p2 t2_2
+                                 Output: t2_2.a
+                                 Filter: (t1.b = t2_2.a)
+               ->  Hash Join
+                     Output: t2_1.a, t3_2.a, LEAST(t1.a, t2_1.a, t3_2.a)
+                     Hash Cond: (t3_2.b = t2_1.a)
+                     ->  Seq Scan on public.prt2_p3 t3_2
+                           Output: t3_2.a, t3_2.b
+                     ->  Hash
+                           Output: t2_1.a
+                           ->  Seq Scan on public.prt1_p3 t2_1
+                                 Output: t2_1.a
+                                 Filter: (t1.b = t2_1.a)
+(49 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+--
+-- partitioned by expression
+--
+CREATE TABLE prt1_e (a int, b int, c varchar) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES START (0) END (250);
+CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES START (250) END (500);
+CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES START (500) END (600);
+INSERT INTO prt1_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_e AS SELECT * FROM prt1_e;
+CREATE TABLE prt2_e (a int, b int, c varchar) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES START (0) END (250);
+CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES START (250) END (500);
+CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES START (500) END (600);
+INSERT INTO prt2_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_e;
+ANALYZE prt2_e_p1;
+ANALYZE prt2_e_p2;
+ANALYZE prt2_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_e AS SELECT * FROM prt2_e;
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+                                  QUERY PLAN                                  
+------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_e_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_e_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
+               ->  Seq Scan on public.prt2_e_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_e_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+(34 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0150 | 150 | 0150
+ 300 | 0300 | 300 | 0300
+ 450 | 0450 | 450 | 0450
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1, uprt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.a % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+                                  QUERY PLAN                                  
+------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_e_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_e_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
+               ->  Seq Scan on public.prt2_e_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_e_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+  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 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1 LEFT JOIN uprt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 RIGHT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: (((t1.a + t1.b) / 2) = ((t2.b + t2.a) / 2))
+                     ->  Seq Scan on public.prt1_e_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                     ->  Hash
+                           Output: t2.b, t2.c, t2.a
+                           ->  Seq Scan on public.prt2_e_p1 t2
+                                 Output: t2.b, t2.c, t2.a
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: (((t1_1.a + t1_1.b) / 2) = ((t2_1.b + t2_1.a) / 2))
+                     ->  Seq Scan on public.prt1_e_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c, t2_1.a
+                           ->  Seq Scan on public.prt2_e_p2 t2_1
+                                 Output: t2_1.b, t2_1.c, t2_1.a
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: (((t1_2.a + t1_2.b) / 2) = ((t2_2.b + t2_2.a) / 2))
+                     ->  Seq Scan on public.prt1_e_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c, t2_2.a
+                           ->  Seq Scan on public.prt2_e_p3 t2_2
+                                 Output: t2_2.b, t2_2.c, t2_2.a
+                                 Filter: ((t2_2.b % 25) = 0)
+(36 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 RIGHT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+  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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1 RIGHT JOIN uprt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t2.b % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_e WHERE prt2_e.b % 25 = 0) t2 ON ((t1.a + t1.b)/2 = (t2.b + t2.a)/2) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                            QUERY PLAN                                            
+--------------------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_e_p1.a, prt1_e_p1.c, prt2_e_p1.b, prt2_e_p1.c
+   Sort Key: prt1_e_p1.a, prt2_e_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_e_p1.a, prt1_e_p1.c, prt2_e_p1.b, prt2_e_p1.c
+               Hash Cond: (((prt1_e_p1.a + prt1_e_p1.b) / 2) = ((prt2_e_p1.b + prt2_e_p1.a) / 2))
+               ->  Seq Scan on public.prt1_e_p1
+                     Output: prt1_e_p1.a, prt1_e_p1.c, prt1_e_p1.b
+                     Filter: ((prt1_e_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_e_p1.b, prt2_e_p1.c, prt2_e_p1.a
+                     ->  Seq Scan on public.prt2_e_p1
+                           Output: prt2_e_p1.b, prt2_e_p1.c, prt2_e_p1.a
+                           Filter: ((prt2_e_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_e_p2.a, prt1_e_p2.c, prt2_e_p2.b, prt2_e_p2.c
+               Hash Cond: (((prt1_e_p2.a + prt1_e_p2.b) / 2) = ((prt2_e_p2.b + prt2_e_p2.a) / 2))
+               ->  Seq Scan on public.prt1_e_p2
+                     Output: prt1_e_p2.a, prt1_e_p2.c, prt1_e_p2.b
+                     Filter: ((prt1_e_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_e_p2.b, prt2_e_p2.c, prt2_e_p2.a
+                     ->  Seq Scan on public.prt2_e_p2
+                           Output: prt2_e_p2.b, prt2_e_p2.c, prt2_e_p2.a
+                           Filter: ((prt2_e_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_e_p3.a, prt1_e_p3.c, prt2_e_p3.b, prt2_e_p3.c
+               Hash Cond: (((prt1_e_p3.a + prt1_e_p3.b) / 2) = ((prt2_e_p3.b + prt2_e_p3.a) / 2))
+               ->  Seq Scan on public.prt1_e_p3
+                     Output: prt1_e_p3.a, prt1_e_p3.c, prt1_e_p3.b
+                     Filter: ((prt1_e_p3.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_e_p3.b, prt2_e_p3.c, prt2_e_p3.a
+                     ->  Seq Scan on public.prt2_e_p3
+                           Output: prt2_e_p3.b, prt2_e_p3.c, prt2_e_p3.a
+                           Filter: ((prt2_e_p3.b % 25) = 0)
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_e WHERE prt2_e.b % 25 = 0) t2 ON ((t1.a + t1.b)/2 = (t2.b + t2.a)/2) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_e t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_e t2 WHERE t2.b % 25 = 0) t2 ON ((t1.a + t1.b)/2 = (t2.b + t2.a)/2) 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)
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 3) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 3) and (b 1) is considered for partition-wise join.
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on public.prt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = t1.a)
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_2.a)
+                     ->  Seq Scan on public.prt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                           ->  Hash Join
+                                 Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = t1_2.a)
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_1.a)
+                     ->  Seq Scan on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                           ->  Hash Join
+                                 Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = t1_1.a)
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+(57 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 3) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 3) and (b 1) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM uprt1 t1, uprt2 t2, uprt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on public.prt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = t1.a)
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_2.a)
+                     ->  Seq Scan on public.prt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = t1_2.a)
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_1.a)
+                     ->  Seq Scan on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = t1_1.a)
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+(57 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, 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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2 4) is considered for partition-wise join.
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: (((t3.a + t3.b) / 2) = t2.b)
+                     ->  Seq Scan on public.prt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = t1.a)
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: (((t3_1.a + t3_1.b) / 2) = t2_1.b)
+                     ->  Seq Scan on public.prt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = t1_2.a)
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: (((t3_2.a + t3_2.b) / 2) = t2_2.b)
+                     ->  Seq Scan on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = t1_1.a)
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+(57 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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2 4) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |          | 
+ 100 | 0100 |     |      |          | 
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |          | 
+ 250 | 0250 |     |      |          | 
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |          | 
+ 400 | 0400 |     |      |          | 
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |          | 
+ 550 | 0550 |     |      |          | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |          | 
+ 100 | 0100 |     |      |          | 
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |          | 
+ 250 | 0250 |     |      |          | 
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |          | 
+ 400 | 0400 |     |      |          | 
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |          | 
+ 550 | 0550 |     |      |          | 
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Hash Cond: (t2.b = t1.a)
+                     ->  Seq Scan on public.prt2_p1 t2
+                           Output: t2.b, t2.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Hash Cond: (t1.a = ((t3.a + t3.b) / 2))
+                                 ->  Seq Scan on public.prt1_p1 t1
+                                       Output: t1.a, t1.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                     Hash Cond: (t2_1.b = t1_2.a)
+                     ->  Seq Scan on public.prt2_p2 t2_1
+                           Output: t2_1.b, t2_1.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                                 Hash Cond: (t1_2.a = ((t3_1.a + t3_1.b) / 2))
+                                 ->  Seq Scan on public.prt1_p2 t1_2
+                                       Output: t1_2.a, t1_2.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                     Hash Cond: (t2_2.b = t1_1.a)
+                     ->  Seq Scan on public.prt2_p3 t2_2
+                           Output: t2_2.b, t2_2.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                                 Hash Cond: (t1_1.a = ((t3_2.a + t3_2.b) / 2))
+                                 ->  Seq Scan on public.prt1_p3 t1_1
+                                       Output: t1_1.a, t1_1.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: (t1.a = t2.b)
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = ((t3.a + t3.b) / 2))
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Hash Cond: (t1_2.a = t2_1.b)
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = ((t3_1.a + t3_1.b) / 2))
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Hash Cond: (t1_1.a = t2_2.b)
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = ((t3_2.a + t3_2.b) / 2))
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 6) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is considered for partition-wise join.
+                                                  QUERY PLAN                                                   
+---------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, ((prt1_e_p1.a + prt1_e_p1.b)), prt1_e_p1.c
+   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   ->  Result
+         Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, (prt1_e_p1.a + prt1_e_p1.b), prt1_e_p1.c
+         ->  Append
+               ->  Hash Full Join
+                     Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, prt1_e_p1.a, prt1_e_p1.b, prt1_e_p1.c
+                     Hash Cond: (prt1_p1.a = ((prt1_e_p1.a + prt1_e_p1.b) / 2))
+                     ->  Hash Full Join
+                           Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on public.prt1_p1
+                                 Output: prt1_p1.a, prt1_p1.c
+                                 Filter: ((prt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p1.b, prt2_p1.c
+                                 ->  Seq Scan on public.prt2_p1
+                                       Output: prt2_p1.b, prt2_p1.c
+                                       Filter: ((prt2_p1.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p1.a, prt1_e_p1.b, prt1_e_p1.c
+                           ->  Seq Scan on public.prt1_e_p1
+                                 Output: prt1_e_p1.a, prt1_e_p1.b, prt1_e_p1.c
+                                 Filter: ((prt1_e_p1.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c, prt1_e_p2.a, prt1_e_p2.b, prt1_e_p2.c
+                     Hash Cond: (prt1_p2.a = ((prt1_e_p2.a + prt1_e_p2.b) / 2))
+                     ->  Hash Full Join
+                           Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+                           Hash Cond: (prt1_p2.a = prt2_p2.b)
+                           ->  Seq Scan on public.prt1_p2
+                                 Output: prt1_p2.a, prt1_p2.c
+                                 Filter: ((prt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p2.b, prt2_p2.c
+                                 ->  Seq Scan on public.prt2_p2
+                                       Output: prt2_p2.b, prt2_p2.c
+                                       Filter: ((prt2_p2.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p2.a, prt1_e_p2.b, prt1_e_p2.c
+                           ->  Seq Scan on public.prt1_e_p2
+                                 Output: prt1_e_p2.a, prt1_e_p2.b, prt1_e_p2.c
+                                 Filter: ((prt1_e_p2.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c, prt1_e_p3.a, prt1_e_p3.b, prt1_e_p3.c
+                     Hash Cond: (prt1_p3.a = ((prt1_e_p3.a + prt1_e_p3.b) / 2))
+                     ->  Hash Full Join
+                           Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c
+                           Hash Cond: (prt1_p3.a = prt2_p3.b)
+                           ->  Seq Scan on public.prt1_p3
+                                 Output: prt1_p3.a, prt1_p3.c
+                                 Filter: ((prt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p3.b, prt2_p3.c
+                                 ->  Seq Scan on public.prt2_p3
+                                       Output: prt2_p3.b, prt2_p3.c
+                                       Filter: ((prt2_p3.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p3.a, prt1_e_p3.b, prt1_e_p3.c
+                           ->  Seq Scan on public.prt1_e_p3
+                                 Output: prt1_e_p3.a, prt1_e_p3.b, prt1_e_p3.c
+                                 Filter: ((prt1_e_p3.a % 25) = 0)
+(63 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 6) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+     |      |  75 | 0075 |          | 
+     |      | 225 | 0225 |          | 
+     |      | 375 | 0375 |          | 
+     |      | 525 | 0525 |          | 
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM uprt1_e WHERE uprt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+     |      |  75 | 0075 |          | 
+     |      | 225 | 0225 |          | 
+     |      | 375 | 0375 |          | 
+     |      | 525 | 0525 |          | 
+(16 rows)
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, 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.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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;
+NOTICE:  join between relations (b 6) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is considered for partition-wise join.
+                                                      QUERY PLAN                                                      
+----------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, (50), prt2_p1.b, (75), ((prt1_e_p1.a + prt1_e_p1.b)), (50)
+   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   ->  Result
+         Output: prt1_p1.a, (50), prt2_p1.b, (75), (prt1_e_p1.a + prt1_e_p1.b), (50)
+         ->  Append
+               ->  Hash Full Join
+                     Output: prt1_p1.a, prt2_p1.b, prt1_e_p1.a, prt1_e_p1.b, (50), (75), (50)
+                     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
+                           Output: prt1_p1.a, prt2_p1.b, (50), (75)
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on public.prt1_p1
+                                 Output: prt1_p1.a, 50
+                                 Filter: ((prt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p1.b, (75)
+                                 ->  Seq Scan on public.prt2_p1
+                                       Output: prt2_p1.b, 75
+                                       Filter: ((prt2_p1.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p1.a, prt1_e_p1.b, (50)
+                           ->  Seq Scan on public.prt1_e_p1
+                                 Output: prt1_e_p1.a, prt1_e_p1.b, 50
+                                 Filter: ((prt1_e_p1.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p2.a, prt2_p2.b, prt1_e_p2.a, prt1_e_p2.b, (50), (75), (50)
+                     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
+                           Output: prt1_p2.a, prt2_p2.b, (50), (75)
+                           Hash Cond: (prt1_p2.a = prt2_p2.b)
+                           ->  Seq Scan on public.prt1_p2
+                                 Output: prt1_p2.a, 50
+                                 Filter: ((prt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p2.b, (75)
+                                 ->  Seq Scan on public.prt2_p2
+                                       Output: prt2_p2.b, 75
+                                       Filter: ((prt2_p2.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p2.a, prt1_e_p2.b, (50)
+                           ->  Seq Scan on public.prt1_e_p2
+                                 Output: prt1_e_p2.a, prt1_e_p2.b, 50
+                                 Filter: ((prt1_e_p2.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p3.a, prt2_p3.b, prt1_e_p3.a, prt1_e_p3.b, (50), (75), (50)
+                     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
+                           Output: prt1_p3.a, prt2_p3.b, (50), (75)
+                           Hash Cond: (prt1_p3.a = prt2_p3.b)
+                           ->  Seq Scan on public.prt1_p3
+                                 Output: prt1_p3.a, 50
+                                 Filter: ((prt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p3.b, (75)
+                                 ->  Seq Scan on public.prt2_p3
+                                       Output: prt2_p3.b, 75
+                                       Filter: ((prt2_p3.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p3.a, prt1_e_p3.b, (50)
+                           ->  Seq Scan on public.prt1_e_p3
+                                 Output: prt1_e_p3.a, prt1_e_p3.b, 50
+                                 Filter: ((prt1_e_p3.a % 25) = 0)
+(66 rows)
+
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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;
+NOTICE:  join between relations (b 6) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is considered for partition-wise join.
+ a  | phv | b  | phv | ?column? | phv 
+----+-----+----+-----+----------+-----
+ 50 |  50 |    |     |      100 |  50
+    |     | 75 |  75 |          |    
+(2 rows)
+
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM uprt1_e WHERE uprt1_e.a % 25 = 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)
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 4) is considered for partition-wise join.
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.a = t1_3.b)
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_3.b, t2.a, t2.b
+                     ->  Hash Join
+                           Output: t1_3.b, t2.a, t2.b
+                           Hash Cond: (((t2.a + t2.b) / 2) = t1_3.b)
+                           ->  Seq Scan on public.prt1_e_p1 t2
+                                 Output: t2.a, t2.b
+                           ->  Hash
+                                 Output: t1_3.b
+                                 ->  Seq Scan on public.prt2_p1 t1_3
+                                       Output: t1_3.b
+                                       Filter: ((t1_3.b % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.a = t1_4.b)
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_4.b, t2_1.a, t2_1.b
+                     ->  Hash Join
+                           Output: t1_4.b, t2_1.a, t2_1.b
+                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_4.b)
+                           ->  Seq Scan on public.prt1_e_p2 t2_1
+                                 Output: t2_1.a, t2_1.b
+                           ->  Hash
+                                 Output: t1_4.b
+                                 ->  Seq Scan on public.prt2_p2 t1_4
+                                       Output: t1_4.b
+                                       Filter: ((t1_4.b % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.a = t1_5.b)
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_5.b, t2_2.a, t2_2.b
+                     ->  Hash Join
+                           Output: t1_5.b, t2_2.a, t2_2.b
+                           Hash Cond: (((t2_2.a + t2_2.b) / 2) = t1_5.b)
+                           ->  Seq Scan on public.prt1_e_p3 t2_2
+                                 Output: t2_2.a, t2_2.b
+                           ->  Hash
+                                 Output: t1_5.b
+                                 ->  Seq Scan on public.prt2_p3 t1_5
+                                       Output: t1_5.b
+                                       Filter: ((t1_5.b % 25) = 0)
+(58 rows)
+
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 4) is considered for partition-wise join.
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1, uprt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 5) is considered for partition-wise join.
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.a = t1_3.b)
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_3.b, t1_6.a, t1_6.b
+                     ->  Hash Semi Join
+                           Output: t1_3.b, t1_6.a, t1_6.b
+                           Hash Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
+                           ->  Seq Scan on public.prt2_p1 t1_3
+                                 Output: t1_3.b
+                           ->  Hash
+                                 Output: t1_6.a, t1_6.b
+                                 ->  Seq Scan on public.prt1_e_p1 t1_6
+                                       Output: t1_6.a, t1_6.b
+                                       Filter: ((t1_6.a % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.a = t1_4.b)
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_4.b, t1_7.a, t1_7.b
+                     ->  Hash Semi Join
+                           Output: t1_4.b, t1_7.a, t1_7.b
+                           Hash Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
+                           ->  Seq Scan on public.prt2_p2 t1_4
+                                 Output: t1_4.b
+                           ->  Hash
+                                 Output: t1_7.a, t1_7.b
+                                 ->  Seq Scan on public.prt1_e_p2 t1_7
+                                       Output: t1_7.a, t1_7.b
+                                       Filter: ((t1_7.a % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.a = t1_5.b)
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_5.b, t1_8.a, t1_8.b
+                     ->  Hash Semi Join
+                           Output: t1_5.b, t1_8.a, t1_8.b
+                           Hash Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
+                           ->  Seq Scan on public.prt2_p3 t1_5
+                                 Output: t1_5.b
+                           ->  Hash
+                                 Output: t1_8.a, t1_8.b
+                                 ->  Seq Scan on public.prt1_e_p3 t1_8
+                                       Output: t1_8.a, t1_8.b
+                                       Filter: ((t1_8.a % 25) = 0)
+(58 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 5) is considered for partition-wise join.
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+-- test merge joins with and without using indexes
+SET enable_hashjoin TO off;
+SET enable_nestloop TO off;
+CREATE INDEX iprt1_a on prt1(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 iprt2_b on prt2(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 iprt1_e_ab2 on prt1_e(((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));
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+                                             QUERY PLAN                                              
+-----------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Merge Cond: (t2.b = t1.a)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           Sort Key: t2.b
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Merge Cond: ((((t3.a + t3.b) / 2)) = t2.b)
+                                 ->  Sort
+                                       Output: t3.a, t3.b, t3.c, (((t3.a + t3.b) / 2))
+                                       Sort Key: (((t3.a + t3.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c, ((t3.a + t3.b) / 2)
+                                             Filter: ((t3.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2.b, t2.c
+                                       Sort Key: t2.b
+                                       ->  Seq Scan on public.prt2_p1 t2
+                                             Output: t2.b, t2.c
+                     ->  Sort
+                           Output: t1.a, t1.c
+                           Sort Key: t1.a
+                           ->  Seq Scan on public.prt1_p1 t1
+                                 Output: t1.a, t1.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Merge Cond: (t2_1.b = t1_2.a)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           Sort Key: t2_1.b
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t2_1.b)
+                                 ->  Sort
+                                       Output: t3_1.a, t3_1.b, t3_1.c, (((t3_1.a + t3_1.b) / 2))
+                                       Sort Key: (((t3_1.a + t3_1.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c, ((t3_1.a + t3_1.b) / 2)
+                                             Filter: ((t3_1.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2_1.b, t2_1.c
+                                       Sort Key: t2_1.b
+                                       ->  Seq Scan on public.prt2_p2 t2_1
+                                             Output: t2_1.b, t2_1.c
+                     ->  Sort
+                           Output: t1_2.a, t1_2.c
+                           Sort Key: t1_2.a
+                           ->  Seq Scan on public.prt1_p2 t1_2
+                                 Output: t1_2.a, t1_2.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Merge Cond: (t2_2.b = t1_1.a)
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           Sort Key: t2_2.b
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t2_2.b)
+                                 ->  Sort
+                                       Output: t3_2.a, t3_2.b, t3_2.c, (((t3_2.a + t3_2.b) / 2))
+                                       Sort Key: (((t3_2.a + t3_2.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c, ((t3_2.a + t3_2.b) / 2)
+                                             Filter: ((t3_2.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2_2.b, t2_2.c
+                                       Sort Key: t2_2.b
+                                       ->  Seq Scan on public.prt2_p3 t2_2
+                                             Output: t2_2.b, t2_2.c
+                     ->  Sort
+                           Output: t1_1.a, t1_1.c
+                           Sort Key: t1_1.a
+                           ->  Seq Scan on public.prt1_p3 t1_1
+                                 Output: t1_1.a, t1_1.c
+(81 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 5) is considered for partition-wise join.
+                                   QUERY PLAN                                    
+---------------------------------------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Output: t1.a, t1.b, t1.c
+         Merge Cond: (t1.a = t1_3.b)
+         ->  Sort
+               Output: t1.a, t1.b, t1.c
+               Sort Key: t1.a
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_3.b, t1_6.a, t1_6.b
+               ->  Merge Semi Join
+                     Output: t1_3.b, t1_6.a, t1_6.b
+                     Merge Cond: (t1_3.b = (((t1_6.a + t1_6.b) / 2)))
+                     ->  Sort
+                           Output: t1_3.b
+                           Sort Key: t1_3.b
+                           ->  Seq Scan on public.prt2_p1 t1_3
+                                 Output: t1_3.b
+                     ->  Sort
+                           Output: t1_6.a, t1_6.b, (((t1_6.a + t1_6.b) / 2))
+                           Sort Key: (((t1_6.a + t1_6.b) / 2))
+                           ->  Seq Scan on public.prt1_e_p1 t1_6
+                                 Output: t1_6.a, t1_6.b, ((t1_6.a + t1_6.b) / 2)
+                                 Filter: ((t1_6.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_2.a, t1_2.b, t1_2.c
+         Merge Cond: (t1_2.a = t1_4.b)
+         ->  Sort
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Sort Key: t1_2.a
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+         ->  Materialize
+               Output: t1_4.b, t1_7.a, t1_7.b
+               ->  Merge Semi Join
+                     Output: t1_4.b, t1_7.a, t1_7.b
+                     Merge Cond: (t1_4.b = (((t1_7.a + t1_7.b) / 2)))
+                     ->  Sort
+                           Output: t1_4.b
+                           Sort Key: t1_4.b
+                           ->  Seq Scan on public.prt2_p2 t1_4
+                                 Output: t1_4.b
+                     ->  Sort
+                           Output: t1_7.a, t1_7.b, (((t1_7.a + t1_7.b) / 2))
+                           Sort Key: (((t1_7.a + t1_7.b) / 2))
+                           ->  Seq Scan on public.prt1_e_p2 t1_7
+                                 Output: t1_7.a, t1_7.b, ((t1_7.a + t1_7.b) / 2)
+                                 Filter: ((t1_7.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_1.a, t1_1.b, t1_1.c
+         Merge Cond: (t1_1.a = t1_5.b)
+         ->  Sort
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Sort Key: t1_1.a
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_5.b, t1_8.a, t1_8.b
+               ->  Merge Semi Join
+                     Output: t1_5.b, t1_8.a, t1_8.b
+                     Merge Cond: (t1_5.b = (((t1_8.a + t1_8.b) / 2)))
+                     ->  Sort
+                           Output: t1_5.b
+                           Sort Key: t1_5.b
+                           ->  Seq Scan on public.prt2_p3 t1_5
+                                 Output: t1_5.b
+                     ->  Sort
+                           Output: t1_8.a, t1_8.b, (((t1_8.a + t1_8.b) / 2))
+                           Sort Key: (((t1_8.a + t1_8.b) / 2))
+                           ->  Seq Scan on public.prt1_e_p3 t1_8
+                                 Output: t1_8.a, t1_8.b, ((t1_8.a + t1_8.b) / 2)
+                                 Filter: ((t1_8.a % 25) = 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 5) is considered for partition-wise join.
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+                                             QUERY PLAN                                              
+-----------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Merge Cond: (t1.a = t2.b)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           Sort Key: t1.a
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Merge Cond: ((((t3.a + t3.b) / 2)) = t1.a)
+                                 ->  Sort
+                                       Output: t3.a, t3.b, t3.c, (((t3.a + t3.b) / 2))
+                                       Sort Key: (((t3.a + t3.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c, ((t3.a + t3.b) / 2)
+                                             Filter: ((t3.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t1.a, t1.c
+                                       Sort Key: t1.a
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                     ->  Sort
+                           Output: t2.b, t2.c
+                           Sort Key: t2.b
+                           ->  Seq Scan on public.prt2_p1 t2
+                                 Output: t2.b, t2.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                     Merge Cond: (t1_2.a = t2_1.b)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                           Sort Key: t1_2.a
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                                 Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t1_2.a)
+                                 ->  Sort
+                                       Output: t3_1.a, t3_1.b, t3_1.c, (((t3_1.a + t3_1.b) / 2))
+                                       Sort Key: (((t3_1.a + t3_1.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c, ((t3_1.a + t3_1.b) / 2)
+                                             Filter: ((t3_1.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t1_2.a, t1_2.c
+                                       Sort Key: t1_2.a
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                     ->  Sort
+                           Output: t2_1.b, t2_1.c
+                           Sort Key: t2_1.b
+                           ->  Seq Scan on public.prt2_p2 t2_1
+                                 Output: t2_1.b, t2_1.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                     Merge Cond: (t1_1.a = t2_2.b)
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                           Sort Key: t1_1.a
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                                 Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t1_1.a)
+                                 ->  Sort
+                                       Output: t3_2.a, t3_2.b, t3_2.c, (((t3_2.a + t3_2.b) / 2))
+                                       Sort Key: (((t3_2.a + t3_2.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c, ((t3_2.a + t3_2.b) / 2)
+                                             Filter: ((t3_2.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t1_1.a, t1_1.c
+                                       Sort Key: t1_1.a
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                     ->  Sort
+                           Output: t2_2.b, t2_2.c
+                           Sort Key: t2_2.b
+                           ->  Seq Scan on public.prt2_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+(81 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SET enable_seqscan TO off;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+                                          QUERY PLAN                                          
+----------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Merge Cond: (t2.b = t1.a)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           Sort Key: t2.b
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Merge Cond: (((t3.a + t3.b) / 2) = t2.b)
+                                 ->  Index Scan using iprt1_e_p1_ab2 on public.prt1_e_p1 t3
+                                       Output: t3.a, t3.b, t3.c
+                                       Filter: ((t3.a % 25) = 0)
+                                 ->  Index Scan using iprt2_p1_b on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                     ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Merge Cond: (t2_1.b = t1_2.a)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           Sort Key: t2_1.b
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Merge Cond: (((t3_1.a + t3_1.b) / 2) = t2_1.b)
+                                 ->  Index Scan using iprt1_e_p2_ab2 on public.prt1_e_p2 t3_1
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       Filter: ((t3_1.a % 25) = 0)
+                                 ->  Index Scan using iprt2_p2_b on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                     ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+               ->  Merge Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Merge Cond: (t2_2.b = ((t3_2.a + t3_2.b) / 2))
+                     ->  Merge Left Join
+                           Output: t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                           Merge Cond: (t2_2.b = t1_1.a)
+                           ->  Index Scan using iprt2_p3_b on public.prt2_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+                           ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_1
+                                 Output: t1_1.a, t1_1.c
+                     ->  Index Scan using iprt1_e_p3_ab2 on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                           Filter: ((t3_2.a % 25) = 0)
+(51 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 5) is considered for partition-wise join.
+                                    QUERY PLAN                                    
+----------------------------------------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Output: t1.a, t1.b, t1.c
+         Merge Cond: (t1.a = t1_3.b)
+         ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1
+               Output: t1.a, t1.b, t1.c
+               Filter: ((t1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_3.b, t1_6.a, t1_6.b
+               ->  Merge Semi Join
+                     Output: t1_3.b, t1_6.a, t1_6.b
+                     Merge Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
+                     ->  Index Only Scan using iprt2_p1_b on public.prt2_p1 t1_3
+                           Output: t1_3.b
+                     ->  Index Scan using iprt1_e_p1_ab2 on public.prt1_e_p1 t1_6
+                           Output: t1_6.a, t1_6.b
+                           Filter: ((t1_6.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_2.a, t1_2.b, t1_2.c
+         Merge Cond: (t1_2.a = t1_4.b)
+         ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_2
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Filter: ((t1_2.a % 25) = 0)
+         ->  Materialize
+               Output: t1_4.b, t1_7.a, t1_7.b
+               ->  Merge Semi Join
+                     Output: t1_4.b, t1_7.a, t1_7.b
+                     Merge Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
+                     ->  Index Only Scan using iprt2_p2_b on public.prt2_p2 t1_4
+                           Output: t1_4.b
+                     ->  Index Scan using iprt1_e_p2_ab2 on public.prt1_e_p2 t1_7
+                           Output: t1_7.a, t1_7.b
+                           Filter: ((t1_7.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_1.a, t1_1.b, t1_1.c
+         Merge Cond: (t1_1.a = t1_5.b)
+         ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_1
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Filter: ((t1_1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_5.b, t1_8.a, t1_8.b
+               ->  Merge Semi Join
+                     Output: t1_5.b, t1_8.a, t1_8.b
+                     Merge Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
+                     ->  Index Only Scan using iprt2_p3_b on public.prt2_p3 t1_5
+                           Output: t1_5.b
+                     ->  Index Scan using iprt1_e_p3_ab2 on public.prt1_e_p3 t1_8
+                           Output: t1_8.a, t1_8.b
+                           Filter: ((t1_8.a % 25) = 0)
+(50 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 5) is considered for partition-wise join.
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+                                          QUERY PLAN                                          
+----------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Merge Cond: (t1.a = t2.b)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           Sort Key: t1.a
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Merge Cond: (((t3.a + t3.b) / 2) = t1.a)
+                                 ->  Index Scan using iprt1_e_p1_ab2 on public.prt1_e_p1 t3
+                                       Output: t3.a, t3.b, t3.c
+                                       Filter: ((t3.a % 25) = 0)
+                                 ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1
+                                       Output: t1.a, t1.c
+                     ->  Index Scan using iprt2_p1_b on public.prt2_p1 t2
+                           Output: t2.b, t2.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                     Merge Cond: (t1_2.a = t2_1.b)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                           Sort Key: t1_2.a
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                                 Merge Cond: (((t3_1.a + t3_1.b) / 2) = t1_2.a)
+                                 ->  Index Scan using iprt1_e_p2_ab2 on public.prt1_e_p2 t3_1
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       Filter: ((t3_1.a % 25) = 0)
+                                 ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_2
+                                       Output: t1_2.a, t1_2.c
+                     ->  Index Scan using iprt2_p2_b on public.prt2_p2 t2_1
+                           Output: t2_1.b, t2_1.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                     Merge Cond: (t1_1.a = t2_2.b)
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                           Sort Key: t1_1.a
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                                 Merge Cond: (((t3_2.a + t3_2.b) / 2) = t1_1.a)
+                                 ->  Index Scan using iprt1_e_p3_ab2 on public.prt1_e_p3 t3_2
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       Filter: ((t3_2.a % 25) = 0)
+                                 ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_1
+                                       Output: t1_1.a, t1_1.c
+                     ->  Index Scan using iprt2_p3_b on public.prt2_p3 t2_2
+                           Output: t2_2.b, t2_2.c
+(54 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+-- lateral references and parameterized paths
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4 5) is considered for partition-wise join.
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
+ Nested Loop Left Join
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   ->  Merge Append
+         Sort Key: t1.a
+         ->  Index Scan using iprt1_a on public.prt1 t1
+               Output: t1.a, t1.b, t1.c
+               Filter: ((t1.a % 25) = 0)
+         ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1_1
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Filter: ((t1_1.a % 25) = 0)
+         ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_2
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Filter: ((t1_2.a % 25) = 0)
+         ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_3
+               Output: t1_3.a, t1_3.b, t1_3.c
+               Filter: ((t1_3.a % 25) = 0)
+   ->  Append
+         ->  Merge Join
+               Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+               Merge Cond: (t2.a = t3.b)
+               ->  Index Only Scan using iprt1_p1_a on public.prt1_p1 t2
+                     Output: t2.a
+                     Index Cond: (t2.a = t1.a)
+               ->  Index Scan using iprt2_p1_b on public.prt2_p1 t3
+                     Output: t3.a, t3.b
+         ->  Merge Join
+               Output: t2_2.a, t3_1.a, LEAST(t1.a, t2_2.a, t3_1.a)
+               Merge Cond: (t2_2.a = t3_1.b)
+               ->  Index Only Scan using iprt1_p2_a on public.prt1_p2 t2_2
+                     Output: t2_2.a
+                     Index Cond: (t2_2.a = t1.a)
+               ->  Index Scan using iprt2_p2_b on public.prt2_p2 t3_1
+                     Output: t3_1.a, t3_1.b
+         ->  Merge Join
+               Output: t2_1.a, t3_2.a, LEAST(t1.a, t2_1.a, t3_2.a)
+               Merge Cond: (t2_1.a = t3_2.b)
+               ->  Index Only Scan using iprt1_p3_a on public.prt1_p3 t2_1
+                     Output: t2_1.a
+                     Index Cond: (t2_1.a = t1.a)
+               ->  Index Scan using iprt2_p3_b on public.prt2_p3 t3_2
+                     Output: t3_2.a, t3_2.b
+(41 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4 5) is considered for partition-wise join.
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
+ Nested Loop Left Join
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   ->  Merge Append
+         Sort Key: t1.a
+         ->  Index Scan using iprt1_a on public.prt1 t1
+               Output: t1.a, t1.b, t1.c
+               Filter: ((t1.a % 25) = 0)
+         ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1_1
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Filter: ((t1_1.a % 25) = 0)
+         ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_2
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Filter: ((t1_2.a % 25) = 0)
+         ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_3
+               Output: t1_3.a, t1_3.b, t1_3.c
+               Filter: ((t1_3.a % 25) = 0)
+   ->  Append
+         ->  Merge Join
+               Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+               Merge Cond: (t2.a = t3.b)
+               ->  Index Only Scan using iprt1_p1_a on public.prt1_p1 t2
+                     Output: t2.a
+                     Index Cond: (t2.a = t1.b)
+               ->  Index Scan using iprt2_p1_b on public.prt2_p1 t3
+                     Output: t3.a, t3.b
+         ->  Merge Join
+               Output: t2_2.a, t3_1.a, LEAST(t1.a, t2_2.a, t3_1.a)
+               Merge Cond: (t2_2.a = t3_1.b)
+               ->  Index Only Scan using iprt1_p2_a on public.prt1_p2 t2_2
+                     Output: t2_2.a
+                     Index Cond: (t2_2.a = t1.b)
+               ->  Index Scan using iprt2_p2_b on public.prt2_p2 t3_1
+                     Output: t3_1.a, t3_1.b
+         ->  Merge Join
+               Output: t2_1.a, t3_2.a, LEAST(t1.a, t2_1.a, t3_2.a)
+               Merge Cond: (t2_1.a = t3_2.b)
+               ->  Index Only Scan using iprt1_p3_a on public.prt1_p3 t2_1
+                     Output: t2_1.a
+                     Index Cond: (t2_1.a = t1.b)
+               ->  Index Scan using iprt2_p3_b on public.prt2_p3 t3_2
+                     Output: t3_2.a, t3_2.b
+(41 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+RESET enable_hashjoin;
+RESET enable_nestloop;
+RESET enable_seqscan;
+--
+-- partitioned by multiple columns
+--
+CREATE TABLE prt1_m (a int, b int, c varchar) PARTITION BY RANGE(a, ((a + b)/2));
+CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES START (0, 0) END (250, 250);
+CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES START (250, 250) END (500, 500);
+CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES START (500, 500) END (600, 600);
+INSERT INTO prt1_m SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_m;
+ANALYZE prt1_m_p1;
+ANALYZE prt1_m_p2;
+ANALYZE prt1_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_m AS SELECT * FROM prt1_m;
+CREATE TABLE prt2_m (a int, b int, c varchar) PARTITION BY RANGE(((b + a)/2), b);
+CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES START (0, 0) END (250, 250);
+CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES START (250, 250) END (500, 500);
+CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES START (500, 500) END (600, 600);
+INSERT INTO prt2_m SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_m;
+ANALYZE prt2_m_p1;
+ANALYZE prt2_m_p2;
+ANALYZE prt2_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_m AS SELECT * FROM prt2_m;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1, prt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+                                              QUERY PLAN                                              
+------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((((t2.b + t2.a) / 2) = t1.a) AND (t2.b = ((t1.a + t1.b) / 2)))
+               ->  Seq Scan on public.prt2_m_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_m_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((((t2_1.b + t2_1.a) / 2) = t1_1.a) AND (t2_1.b = ((t1_1.a + t1_1.b) / 2)))
+               ->  Seq Scan on public.prt2_m_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_m_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((((t2_2.b + t2_2.a) / 2) = t1_2.a) AND (t2_2.b = ((t1_2.a + t1_2.b) / 2)))
+               ->  Seq Scan on public.prt2_m_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_m_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1, prt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0150 | 150 | 0150
+ 300 | 0300 | 300 | 0300
+ 450 | 0450 | 450 | 0450
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_m t1, uprt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 AND t1.a % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+                                              QUERY PLAN                                              
+------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((((t2.b + t2.a) / 2) = t1.a) AND (t2.b = ((t1.a + t1.b) / 2)))
+               ->  Seq Scan on public.prt2_m_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_m_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((((t2_1.b + t2_1.a) / 2) = t1_1.a) AND (t2_1.b = ((t1_1.a + t1_1.b) / 2)))
+               ->  Seq Scan on public.prt2_m_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_m_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((((t2_2.b + t2_2.a) / 2) = t1_2.a) AND (t2_2.b = ((t1_2.a + t1_2.b) / 2)))
+               ->  Seq Scan on public.prt2_m_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_m_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+  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 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_m t1 LEFT JOIN uprt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+                                                 QUERY PLAN                                                 
+------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: ((((t1.a + t1.b) / 2) = t2.b) AND (t1.a = ((t2.b + t2.a) / 2)))
+                     ->  Seq Scan on public.prt1_m_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                     ->  Hash
+                           Output: t2.b, t2.c, t2.a
+                           ->  Seq Scan on public.prt2_m_p1 t2
+                                 Output: t2.b, t2.c, t2.a
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: ((((t1_1.a + t1_1.b) / 2) = t2_1.b) AND (t1_1.a = ((t2_1.b + t2_1.a) / 2)))
+                     ->  Seq Scan on public.prt1_m_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c, t2_1.a
+                           ->  Seq Scan on public.prt2_m_p2 t2_1
+                                 Output: t2_1.b, t2_1.c, t2_1.a
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: ((((t1_2.a + t1_2.b) / 2) = t2_2.b) AND (t1_2.a = ((t2_2.b + t2_2.a) / 2)))
+                     ->  Seq Scan on public.prt1_m_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c, t2_2.a
+                           ->  Seq Scan on public.prt2_m_p3 t2_2
+                                 Output: t2_2.b, t2_2.c, t2_2.a
+                                 Filter: ((t2_2.b % 25) = 0)
+(36 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+  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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_m t1 RIGHT JOIN uprt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                                             QUERY PLAN                                                             
+------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_m_p1.a, prt1_m_p1.c, prt2_m_p1.b, prt2_m_p1.c
+   Sort Key: prt1_m_p1.a, prt2_m_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_m_p1.a, prt1_m_p1.c, prt2_m_p1.b, prt2_m_p1.c
+               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 public.prt1_m_p1
+                     Output: prt1_m_p1.a, prt1_m_p1.c, prt1_m_p1.b
+                     Filter: ((prt1_m_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_m_p1.b, prt2_m_p1.c, prt2_m_p1.a
+                     ->  Seq Scan on public.prt2_m_p1
+                           Output: prt2_m_p1.b, prt2_m_p1.c, prt2_m_p1.a
+                           Filter: ((prt2_m_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_m_p2.a, prt1_m_p2.c, prt2_m_p2.b, prt2_m_p2.c
+               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 public.prt1_m_p2
+                     Output: prt1_m_p2.a, prt1_m_p2.c, prt1_m_p2.b
+                     Filter: ((prt1_m_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_m_p2.b, prt2_m_p2.c, prt2_m_p2.a
+                     ->  Seq Scan on public.prt2_m_p2
+                           Output: prt2_m_p2.b, prt2_m_p2.c, prt2_m_p2.a
+                           Filter: ((prt2_m_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_m_p3.a, prt1_m_p3.c, prt2_m_p3.b, prt2_m_p3.c
+               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 public.prt1_m_p3
+                     Output: prt1_m_p3.a, prt1_m_p3.c, prt1_m_p3.b
+                     Filter: ((prt1_m_p3.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_m_p3.b, prt2_m_p3.c, prt2_m_p3.a
+                     ->  Seq Scan on public.prt2_m_p3
+                           Output: prt2_m_p3.b, prt2_m_p3.c, prt2_m_p3.a
+                           Filter: ((prt2_m_p3.b % 25) = 0)
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_m t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_m t2 WHERE t2.b % 25 = 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 | 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)
+
+--
+-- multi-leveled partitions
+--
+CREATE TABLE prt1_l (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_l_p1 PARTITION OF prt1_l FOR VALUES START (0) END (250) PARTITION BY RANGE (b);
+CREATE TABLE prt1_l_p1_p1 PARTITION OF prt1_l_p1 FOR VALUES START (0) END (100);
+CREATE TABLE prt1_l_p1_p2 PARTITION OF prt1_l_p1 FOR VALUES START (100) END (250);
+CREATE TABLE prt1_l_p2 PARTITION OF prt1_l FOR VALUES START (250) END (500) PARTITION BY RANGE (c);
+CREATE TABLE prt1_l_p2_p1 PARTITION OF prt1_l_p2 FOR VALUES START ('0250') END ('0400');
+CREATE TABLE prt1_l_p2_p2 PARTITION OF prt1_l_p2 FOR VALUES START ('0400') END ('0500');
+CREATE TABLE prt1_l_p3 PARTITION OF prt1_l FOR VALUES START (500) END (600) PARTITION BY RANGE ((b + a));
+CREATE TABLE prt1_l_p3_p1 PARTITION OF prt1_l_p3 FOR VALUES START (1000) END (1100);
+CREATE TABLE prt1_l_p3_p2 PARTITION OF prt1_l_p3 FOR VALUES START (1100) END (1200);
+INSERT INTO prt1_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_l;
+ANALYZE prt1_l_p1;
+ANALYZE prt1_l_p1_p1;
+ANALYZE prt1_l_p1_p2;
+ANALYZE prt1_l_p2;
+ANALYZE prt1_l_p2_p1;
+ANALYZE prt1_l_p2_p2;
+ANALYZE prt1_l_p3;
+ANALYZE prt1_l_p3_p1;
+ANALYZE prt1_l_p3_p2;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_l AS SELECT * FROM prt1_l;
+CREATE TABLE prt2_l (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_l_p1 PARTITION OF prt2_l FOR VALUES START (0) END (250) PARTITION BY RANGE (a);
+CREATE TABLE prt2_l_p1_p1 PARTITION OF prt2_l_p1 FOR VALUES START (0) END (100);
+CREATE TABLE prt2_l_p1_p2 PARTITION OF prt2_l_p1 FOR VALUES START (100) END (250);
+CREATE TABLE prt2_l_p2 PARTITION OF prt2_l FOR VALUES START (250) END (500) PARTITION BY RANGE (c);
+CREATE TABLE prt2_l_p2_p1 PARTITION OF prt2_l_p2 FOR VALUES START ('0250') END ('0400');
+CREATE TABLE prt2_l_p2_p2 PARTITION OF prt2_l_p2 FOR VALUES START ('0400') END ('0500');
+CREATE TABLE prt2_l_p3 PARTITION OF prt2_l FOR VALUES START (500) END (600) PARTITION BY RANGE ((a + b));
+CREATE TABLE prt2_l_p3_p1 PARTITION OF prt2_l_p3 FOR VALUES START (1000) END (1100);
+CREATE TABLE prt2_l_p3_p2 PARTITION OF prt2_l_p3 FOR VALUES START (1100) END (1200);
+INSERT INTO prt2_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_l;
+ANALYZE prt2_l_p1;
+ANALYZE prt2_l_p1_p1;
+ANALYZE prt2_l_p1_p2;
+ANALYZE prt2_l_p2;
+ANALYZE prt2_l_p2_p1;
+ANALYZE prt2_l_p2_p2;
+ANALYZE prt2_l_p3;
+ANALYZE prt2_l_p3_p1;
+ANALYZE prt2_l_p3_p2;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_l AS SELECT * FROM prt2_l;
+EXPLAIN (VERBOSE, 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 = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 17) is considered for partition-wise join.
+NOTICE:  join between relations (b 8) and (b 21) is considered for partition-wise join.
+NOTICE:  join between relations (b 12) and (b 25) is considered for partition-wise join.
+                                                                      QUERY PLAN                                                                      
+------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((t2.b = t1.a) AND (t2.a = t1.b) AND ((t2.c)::text = (t1.c)::text) AND ((t2.a + t2.b) = (t1.b + t1.a)))
+               ->  Seq Scan on public.prt2_l_p1_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_l_p1_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.a = t1_1.b) AND ((t2_1.c)::text = (t1_1.c)::text) AND ((t2_1.a + t2_1.b) = (t1_1.b + t1_1.a)))
+               ->  Seq Scan on public.prt2_l_p1_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_l_p1_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.a = t1_2.b) AND ((t2_2.c)::text = (t1_2.c)::text) AND ((t2_2.a + t2_2.b) = (t1_2.b + t1_2.a)))
+               ->  Seq Scan on public.prt2_l_p2_p1 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_l_p2_p1 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_3.a, t1_3.c, t2_3.b, t2_3.c
+               Hash Cond: ((t2_3.b = t1_3.a) AND (t2_3.a = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text) AND ((t2_3.a + t2_3.b) = (t1_3.b + t1_3.a)))
+               ->  Seq Scan on public.prt2_l_p2_p2 t2_3
+                     Output: t2_3.b, t2_3.c, t2_3.a
+               ->  Hash
+                     Output: t1_3.a, t1_3.c, t1_3.b
+                     ->  Seq Scan on public.prt1_l_p2_p2 t1_3
+                           Output: t1_3.a, t1_3.c, t1_3.b
+                           Filter: ((t1_3.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_4.a, t1_4.c, t2_4.b, t2_4.c
+               Hash Cond: ((t2_4.b = t1_4.a) AND (t2_4.a = t1_4.b) AND ((t2_4.c)::text = (t1_4.c)::text) AND ((t2_4.a + t2_4.b) = (t1_4.b + t1_4.a)))
+               ->  Seq Scan on public.prt2_l_p3_p1 t2_4
+                     Output: t2_4.b, t2_4.c, t2_4.a
+               ->  Hash
+                     Output: t1_4.a, t1_4.c, t1_4.b
+                     ->  Seq Scan on public.prt1_l_p3_p1 t1_4
+                           Output: t1_4.a, t1_4.c, t1_4.b
+                           Filter: ((t1_4.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_5.a, t1_5.c, t2_5.b, t2_5.c
+               Hash Cond: ((t2_5.b = t1_5.a) AND (t2_5.a = t1_5.b) AND ((t2_5.c)::text = (t1_5.c)::text) AND ((t2_5.a + t2_5.b) = (t1_5.b + t1_5.a)))
+               ->  Seq Scan on public.prt2_l_p3_p2 t2_5
+                     Output: t2_5.b, t2_5.c, t2_5.a
+               ->  Hash
+                     Output: t1_5.a, t1_5.c, t1_5.b
+                     ->  Seq Scan on public.prt1_l_p3_p2 t1_5
+                           Output: t1_5.a, t1_5.c, t1_5.b
+                           Filter: ((t1_5.a % 25) = 0)
+(64 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1, prt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 17) is considered for partition-wise join.
+NOTICE:  join between relations (b 8) and (b 21) is considered for partition-wise join.
+NOTICE:  join between relations (b 12) and (b 25) is considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0150 | 150 | 0150
+ 300 | 0300 | 300 | 0300
+ 450 | 0450 | 450 | 0450
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1, uprt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 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)
+
+EXPLAIN (VERBOSE, 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 5) and (b 18) is considered for partition-wise join.
+NOTICE:  join between relations (b 9) and (b 22) is considered for partition-wise join.
+NOTICE:  join between relations (b 13) and (b 26) is considered for partition-wise join.
+                                                                      QUERY PLAN                                                                      
+------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((t2.b = t1.a) AND (t2.a = t1.b) AND ((t2.c)::text = (t1.c)::text) AND ((t2.a + t2.b) = (t1.b + t1.a)))
+               ->  Seq Scan on public.prt2_l_p1_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_l_p1_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.a = t1_1.b) AND ((t2_1.c)::text = (t1_1.c)::text) AND ((t2_1.a + t2_1.b) = (t1_1.b + t1_1.a)))
+               ->  Seq Scan on public.prt2_l_p1_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_l_p1_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.a = t1_2.b) AND ((t2_2.c)::text = (t1_2.c)::text) AND ((t2_2.a + t2_2.b) = (t1_2.b + t1_2.a)))
+               ->  Seq Scan on public.prt2_l_p2_p1 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_l_p2_p1 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_3.a, t1_3.c, t2_3.b, t2_3.c
+               Hash Cond: ((t2_3.b = t1_3.a) AND (t2_3.a = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text) AND ((t2_3.a + t2_3.b) = (t1_3.b + t1_3.a)))
+               ->  Seq Scan on public.prt2_l_p2_p2 t2_3
+                     Output: t2_3.b, t2_3.c, t2_3.a
+               ->  Hash
+                     Output: t1_3.a, t1_3.c, t1_3.b
+                     ->  Seq Scan on public.prt1_l_p2_p2 t1_3
+                           Output: t1_3.a, t1_3.c, t1_3.b
+                           Filter: ((t1_3.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_4.a, t1_4.c, t2_4.b, t2_4.c
+               Hash Cond: ((t2_4.b = t1_4.a) AND (t2_4.a = t1_4.b) AND ((t2_4.c)::text = (t1_4.c)::text) AND ((t2_4.a + t2_4.b) = (t1_4.b + t1_4.a)))
+               ->  Seq Scan on public.prt2_l_p3_p1 t2_4
+                     Output: t2_4.b, t2_4.c, t2_4.a
+               ->  Hash
+                     Output: t1_4.a, t1_4.c, t1_4.b
+                     ->  Seq Scan on public.prt1_l_p3_p1 t1_4
+                           Output: t1_4.a, t1_4.c, t1_4.b
+                           Filter: ((t1_4.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_5.a, t1_5.c, t2_5.b, t2_5.c
+               Hash Cond: ((t2_5.b = t1_5.a) AND (t2_5.a = t1_5.b) AND ((t2_5.c)::text = (t1_5.c)::text) AND ((t2_5.a + t2_5.b) = (t1_5.b + t1_5.a)))
+               ->  Seq Scan on public.prt2_l_p3_p2 t2_5
+                     Output: t2_5.b, t2_5.c, t2_5.a
+               ->  Hash
+                     Output: t1_5.a, t1_5.c, t1_5.b
+                     ->  Seq Scan on public.prt1_l_p3_p2 t1_5
+                           Output: t1_5.a, t1_5.c, t1_5.b
+                           Filter: ((t1_5.a % 25) = 0)
+(64 rows)
+
+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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 5) and (b 18) is considered for partition-wise join.
+NOTICE:  join between relations (b 9) and (b 22) is considered for partition-wise join.
+NOTICE:  join between relations (b 13) and (b 26) is considered for partition-wise join.
+  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 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 LEFT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 18) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 22) and (b 9) is considered for partition-wise join.
+NOTICE:  join between relations (b 26) and (b 13) is considered for partition-wise join.
+                                                                         QUERY PLAN                                                                         
+------------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: ((t1.a = t2.b) AND (t1.b = t2.a) AND ((t1.c)::text = (t2.c)::text) AND ((t1.b + t1.a) = (t2.a + t2.b)))
+                     ->  Seq Scan on public.prt1_l_p1_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                     ->  Hash
+                           Output: t2.b, t2.c, t2.a
+                           ->  Seq Scan on public.prt2_l_p1_p1 t2
+                                 Output: t2.b, t2.c, t2.a
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: ((t1_1.a = t2_1.b) AND (t1_1.b = t2_1.a) AND ((t1_1.c)::text = (t2_1.c)::text) AND ((t1_1.b + t1_1.a) = (t2_1.a + t2_1.b)))
+                     ->  Seq Scan on public.prt1_l_p1_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c, t2_1.a
+                           ->  Seq Scan on public.prt2_l_p1_p2 t2_1
+                                 Output: t2_1.b, t2_1.c, t2_1.a
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: ((t1_2.a = t2_2.b) AND (t1_2.b = t2_2.a) AND ((t1_2.c)::text = (t2_2.c)::text) AND ((t1_2.b + t1_2.a) = (t2_2.a + t2_2.b)))
+                     ->  Seq Scan on public.prt1_l_p2_p1 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c, t2_2.a
+                           ->  Seq Scan on public.prt2_l_p2_p1 t2_2
+                                 Output: t2_2.b, t2_2.c, t2_2.a
+                                 Filter: ((t2_2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_3.b, t2_3.c, t1_3.a, t1_3.c
+                     Hash Cond: ((t1_3.a = t2_3.b) AND (t1_3.b = t2_3.a) AND ((t1_3.c)::text = (t2_3.c)::text) AND ((t1_3.b + t1_3.a) = (t2_3.a + t2_3.b)))
+                     ->  Seq Scan on public.prt1_l_p2_p2 t1_3
+                           Output: t1_3.a, t1_3.c, t1_3.b
+                     ->  Hash
+                           Output: t2_3.b, t2_3.c, t2_3.a
+                           ->  Seq Scan on public.prt2_l_p2_p2 t2_3
+                                 Output: t2_3.b, t2_3.c, t2_3.a
+                                 Filter: ((t2_3.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_4.b, t2_4.c, t1_4.a, t1_4.c
+                     Hash Cond: ((t1_4.a = t2_4.b) AND (t1_4.b = t2_4.a) AND ((t1_4.c)::text = (t2_4.c)::text) AND ((t1_4.b + t1_4.a) = (t2_4.a + t2_4.b)))
+                     ->  Seq Scan on public.prt1_l_p3_p1 t1_4
+                           Output: t1_4.a, t1_4.c, t1_4.b
+                     ->  Hash
+                           Output: t2_4.b, t2_4.c, t2_4.a
+                           ->  Seq Scan on public.prt2_l_p3_p1 t2_4
+                                 Output: t2_4.b, t2_4.c, t2_4.a
+                                 Filter: ((t2_4.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_5.b, t2_5.c, t1_5.a, t1_5.c
+                     Hash Cond: ((t1_5.a = t2_5.b) AND (t1_5.b = t2_5.a) AND ((t1_5.c)::text = (t2_5.c)::text) AND ((t1_5.b + t1_5.a) = (t2_5.a + t2_5.b)))
+                     ->  Seq Scan on public.prt1_l_p3_p2 t1_5
+                           Output: t1_5.a, t1_5.c, t1_5.b
+                     ->  Hash
+                           Output: t2_5.b, t2_5.c, t2_5.a
+                           ->  Seq Scan on public.prt2_l_p3_p2 t2_5
+                                 Output: t2_5.b, t2_5.c, t2_5.a
+                                 Filter: ((t2_5.b % 25) = 0)
+(66 rows)
+
+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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 18) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 22) and (b 9) is considered for partition-wise join.
+NOTICE:  join between relations (b 26) and (b 13) is considered for partition-wise join.
+  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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 RIGHT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 7) and (b 20) is considered for partition-wise join.
+NOTICE:  join between relations (b 11) and (b 24) is considered for partition-wise join.
+NOTICE:  join between relations (b 15) and (b 28) is considered for partition-wise join.
+                                                                                                              QUERY PLAN                                                                                                              
+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_l_p1_p1.a, prt1_l_p1_p1.c, prt2_l_p1_p1.b, prt2_l_p1_p1.c
+   Sort Key: prt1_l_p1_p1.a, prt2_l_p1_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_l_p1_p1.a, prt1_l_p1_p1.c, prt2_l_p1_p1.b, prt2_l_p1_p1.c
+               Hash Cond: ((prt1_l_p1_p1.a = prt2_l_p1_p1.b) AND (prt1_l_p1_p1.b = prt2_l_p1_p1.a) AND ((prt1_l_p1_p1.c)::text = (prt2_l_p1_p1.c)::text) AND ((prt1_l_p1_p1.b + prt1_l_p1_p1.a) = (prt2_l_p1_p1.a + prt2_l_p1_p1.b)))
+               ->  Seq Scan on public.prt1_l_p1_p1
+                     Output: prt1_l_p1_p1.a, prt1_l_p1_p1.c, prt1_l_p1_p1.b
+                     Filter: ((prt1_l_p1_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p1_p1.b, prt2_l_p1_p1.c, prt2_l_p1_p1.a
+                     ->  Seq Scan on public.prt2_l_p1_p1
+                           Output: prt2_l_p1_p1.b, prt2_l_p1_p1.c, prt2_l_p1_p1.a
+                           Filter: ((prt2_l_p1_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_l_p1_p2.a, prt1_l_p1_p2.c, prt2_l_p1_p2.b, prt2_l_p1_p2.c
+               Hash Cond: ((prt1_l_p1_p2.a = prt2_l_p1_p2.b) AND (prt1_l_p1_p2.b = prt2_l_p1_p2.a) AND ((prt1_l_p1_p2.c)::text = (prt2_l_p1_p2.c)::text) AND ((prt1_l_p1_p2.b + prt1_l_p1_p2.a) = (prt2_l_p1_p2.a + prt2_l_p1_p2.b)))
+               ->  Seq Scan on public.prt1_l_p1_p2
+                     Output: prt1_l_p1_p2.a, prt1_l_p1_p2.c, prt1_l_p1_p2.b
+                     Filter: ((prt1_l_p1_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p1_p2.b, prt2_l_p1_p2.c, prt2_l_p1_p2.a
+                     ->  Seq Scan on public.prt2_l_p1_p2
+                           Output: prt2_l_p1_p2.b, prt2_l_p1_p2.c, prt2_l_p1_p2.a
+                           Filter: ((prt2_l_p1_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_l_p2_p1.a, prt1_l_p2_p1.c, prt2_l_p2_p1.b, prt2_l_p2_p1.c
+               Hash Cond: ((prt1_l_p2_p1.a = prt2_l_p2_p1.b) AND (prt1_l_p2_p1.b = prt2_l_p2_p1.a) AND ((prt1_l_p2_p1.c)::text = (prt2_l_p2_p1.c)::text) AND ((prt1_l_p2_p1.b + prt1_l_p2_p1.a) = (prt2_l_p2_p1.a + prt2_l_p2_p1.b)))
+               ->  Seq Scan on public.prt1_l_p2_p1
+                     Output: prt1_l_p2_p1.a, prt1_l_p2_p1.c, prt1_l_p2_p1.b
+                     Filter: ((prt1_l_p2_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p2_p1.b, prt2_l_p2_p1.c, prt2_l_p2_p1.a
+                     ->  Seq Scan on public.prt2_l_p2_p1
+                           Output: prt2_l_p2_p1.b, prt2_l_p2_p1.c, prt2_l_p2_p1.a
+                           Filter: ((prt2_l_p2_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_l_p2_p2.a, prt1_l_p2_p2.c, prt2_l_p2_p2.b, prt2_l_p2_p2.c
+               Hash Cond: ((prt1_l_p2_p2.a = prt2_l_p2_p2.b) AND (prt1_l_p2_p2.b = prt2_l_p2_p2.a) AND ((prt1_l_p2_p2.c)::text = (prt2_l_p2_p2.c)::text) AND ((prt1_l_p2_p2.b + prt1_l_p2_p2.a) = (prt2_l_p2_p2.a + prt2_l_p2_p2.b)))
+               ->  Seq Scan on public.prt1_l_p2_p2
+                     Output: prt1_l_p2_p2.a, prt1_l_p2_p2.c, prt1_l_p2_p2.b
+                     Filter: ((prt1_l_p2_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p2_p2.b, prt2_l_p2_p2.c, prt2_l_p2_p2.a
+                     ->  Seq Scan on public.prt2_l_p2_p2
+                           Output: prt2_l_p2_p2.b, prt2_l_p2_p2.c, prt2_l_p2_p2.a
+                           Filter: ((prt2_l_p2_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_l_p3_p1.a, prt1_l_p3_p1.c, prt2_l_p3_p1.b, prt2_l_p3_p1.c
+               Hash Cond: ((prt1_l_p3_p1.a = prt2_l_p3_p1.b) AND (prt1_l_p3_p1.b = prt2_l_p3_p1.a) AND ((prt1_l_p3_p1.c)::text = (prt2_l_p3_p1.c)::text) AND ((prt1_l_p3_p1.b + prt1_l_p3_p1.a) = (prt2_l_p3_p1.a + prt2_l_p3_p1.b)))
+               ->  Seq Scan on public.prt1_l_p3_p1
+                     Output: prt1_l_p3_p1.a, prt1_l_p3_p1.c, prt1_l_p3_p1.b
+                     Filter: ((prt1_l_p3_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p3_p1.b, prt2_l_p3_p1.c, prt2_l_p3_p1.a
+                     ->  Seq Scan on public.prt2_l_p3_p1
+                           Output: prt2_l_p3_p1.b, prt2_l_p3_p1.c, prt2_l_p3_p1.a
+                           Filter: ((prt2_l_p3_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_l_p3_p2.a, prt1_l_p3_p2.c, prt2_l_p3_p2.b, prt2_l_p3_p2.c
+               Hash Cond: ((prt1_l_p3_p2.a = prt2_l_p3_p2.b) AND (prt1_l_p3_p2.b = prt2_l_p3_p2.a) AND ((prt1_l_p3_p2.c)::text = (prt2_l_p3_p2.c)::text) AND ((prt1_l_p3_p2.b + prt1_l_p3_p2.a) = (prt2_l_p3_p2.a + prt2_l_p3_p2.b)))
+               ->  Seq Scan on public.prt1_l_p3_p2
+                     Output: prt1_l_p3_p2.a, prt1_l_p3_p2.c, prt1_l_p3_p2.b
+                     Filter: ((prt1_l_p3_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p3_p2.b, prt2_l_p3_p2.c, prt2_l_p3_p2.a
+                     ->  Seq Scan on public.prt2_l_p3_p2
+                           Output: prt2_l_p3_p2.b, prt2_l_p3_p2.c, prt2_l_p3_p2.a
+                           Filter: ((prt2_l_p3_p2.b % 25) = 0)
+(70 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 7) and (b 20) is considered for partition-wise join.
+NOTICE:  join between relations (b 11) and (b 24) is considered for partition-wise join.
+NOTICE:  join between relations (b 15) and (b 28) is considered for partition-wise join.
+  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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_l t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_l t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.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)
+
+--
+-- 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;
+ANALYZE plt1_p1;
+ANALYZE plt1_p2;
+ANALYZE plt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1 AS SELECT * FROM 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;
+ANALYZE plt2_p1;
+ANALYZE plt2_p2;
+ANALYZE plt2_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt2 AS SELECT * FROM plt2;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.a = t2.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((t2.c = t1.c) AND (t2.a = t1.a))
+               ->  Seq Scan on public.plt2_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.plt1_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((t2_1.c = t1_1.c) AND (t2_1.a = t1_1.a))
+               ->  Seq Scan on public.plt2_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.plt1_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((t2_2.c = t1_2.c) AND (t2_2.a = t1_2.a))
+               ->  Seq Scan on public.plt2_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.plt1_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.a = t2.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0003 | 150 | 0003
+ 300 | 0006 | 300 | 0006
+ 450 | 0009 | 450 | 0009
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1 t1, uplt2 t2 WHERE t1.c = t2.c AND t1.a = t2.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0003 | 150 | 0003
+ 300 | 0006 | 300 | 0006
+ 450 | 0009 | 450 | 0009
+(4 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.a AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+               ->  Seq Scan on public.plt2_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.plt1_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on public.plt2_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.plt1_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on public.plt2_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.plt1_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.a AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  50 | 0001 |     | 
+ 100 | 0002 |     | 
+ 150 | 0003 | 150 | 0003
+ 200 | 0004 |     | 
+ 250 | 0005 |     | 
+ 300 | 0006 | 300 | 0006
+ 350 | 0007 |     | 
+ 400 | 0008 |     | 
+ 450 | 0009 | 450 | 0009
+ 500 | 0010 |     | 
+ 550 | 0011 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.a AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  50 | 0001 |     | 
+ 100 | 0002 |     | 
+ 150 | 0003 | 150 | 0003
+ 200 | 0004 |     | 
+ 250 | 0005 |     | 
+ 300 | 0006 | 300 | 0006
+ 350 | 0007 |     | 
+ 400 | 0008 |     | 
+ 450 | 0009 | 450 | 0009
+ 500 | 0010 |     | 
+ 550 | 0011 |     | 
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+                                QUERY PLAN                                
+--------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: ((t1.a = t2.b) AND (t1.c = t2.c))
+                     ->  Seq Scan on public.plt1_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t2.b, t2.c
+                           ->  Seq Scan on public.plt2_p1 t2
+                                 Output: t2.b, t2.c
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: ((t1_1.a = t2_1.b) AND (t1_1.c = t2_1.c))
+                     ->  Seq Scan on public.plt1_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c
+                           ->  Seq Scan on public.plt2_p2 t2_1
+                                 Output: t2_1.b, t2_1.c
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: ((t1_2.a = t2_2.b) AND (t1_2.c = t2_2.c))
+                     ->  Seq Scan on public.plt1_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c
+                           ->  Seq Scan on public.plt2_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+                                 Filter: ((t2_2.b % 25) = 0)
+(36 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0003 | 150 | 0003
+ 300 | 0006 | 300 | 0006
+ 450 | 0009 | 450 | 0009
+     |      |  75 | 0001
+     |      | 225 | 0004
+     |      | 375 | 0007
+     |      | 525 | 0010
+(8 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0003 | 150 | 0003
+ 300 | 0006 | 300 | 0006
+ 450 | 0009 | 450 | 0009
+     |      |  75 | 0001
+     |      | 225 | 0004
+     |      | 375 | 0007
+     |      | 525 | 0010
+(8 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                   QUERY PLAN                                   
+--------------------------------------------------------------------------------
+ Sort
+   Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+   Sort Key: plt1_p1.a, plt2_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+               Hash Cond: ((plt1_p1.a = plt2_p1.b) AND (plt1_p1.c = plt2_p1.c))
+               ->  Seq Scan on public.plt1_p1
+                     Output: plt1_p1.a, plt1_p1.c
+                     Filter: ((plt1_p1.a % 25) = 0)
+               ->  Hash
+                     Output: plt2_p1.b, plt2_p1.c
+                     ->  Seq Scan on public.plt2_p1
+                           Output: plt2_p1.b, plt2_p1.c
+                           Filter: ((plt2_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+               Hash Cond: ((plt1_p2.a = plt2_p2.b) AND (plt1_p2.c = plt2_p2.c))
+               ->  Seq Scan on public.plt1_p2
+                     Output: plt1_p2.a, plt1_p2.c
+                     Filter: ((plt1_p2.a % 25) = 0)
+               ->  Hash
+                     Output: plt2_p2.b, plt2_p2.c
+                     ->  Seq Scan on public.plt2_p2
+                           Output: plt2_p2.b, plt2_p2.c
+                           Filter: ((plt2_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+               Hash Cond: ((plt1_p3.a = plt2_p3.b) AND (plt1_p3.c = plt2_p3.c))
+               ->  Seq Scan on public.plt1_p3
+                     Output: plt1_p3.a, plt1_p3.c
+                     Filter: ((plt1_p3.a % 25) = 0)
+               ->  Hash
+                     Output: plt2_p3.b, plt2_p3.c
+                     ->  Seq Scan on public.plt2_p3
+                           Output: plt2_p3.b, plt2_p3.c
+                           Filter: ((plt2_p3.b % 25) = 0)
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  50 | 0001 |     | 
+ 100 | 0002 |     | 
+ 150 | 0003 | 150 | 0003
+ 200 | 0004 |     | 
+ 250 | 0005 |     | 
+ 300 | 0006 | 300 | 0006
+ 350 | 0007 |     | 
+ 400 | 0008 |     | 
+ 450 | 0009 | 450 | 0009
+ 500 | 0010 |     | 
+ 550 | 0011 |     | 
+     |      |  75 | 0001
+     |      | 225 | 0004
+     |      | 375 | 0007
+     |      | 525 | 0010
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uplt1 t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  50 | 0001 |     | 
+ 100 | 0002 |     | 
+ 150 | 0003 | 150 | 0003
+ 200 | 0004 |     | 
+ 250 | 0005 |     | 
+ 300 | 0006 | 300 | 0006
+ 350 | 0007 |     | 
+ 400 | 0008 |     | 
+ 450 | 0009 | 450 | 0009
+ 500 | 0010 |     | 
+ 550 | 0011 |     | 
+     |      |  75 | 0001
+     |      | 225 | 0004
+     |      | 375 | 0007
+     |      | 525 | 0010
+(16 rows)
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM (SELECT 50 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ GroupAggregate
+   Output: sum(plt1_p1.a), plt1_p1.c, avg(plt2_p1.b), plt2_p1.c
+   Group Key: plt1_p1.c, plt2_p1.c
+   ->  Sort
+         Output: plt1_p1.c, plt2_p1.c, plt1_p1.a, plt2_p1.b
+         Sort Key: plt1_p1.c, plt2_p1.c
+         ->  Result
+               Output: plt1_p1.c, plt2_p1.c, plt1_p1.a, plt2_p1.b
+               ->  Append
+                     ->  Hash Full Join
+                           Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+                           Hash Cond: ((plt1_p1.c = plt2_p1.c) AND (plt1_p1.a = plt2_p1.b))
+                           ->  Seq Scan on public.plt1_p1
+                                 Output: plt1_p1.a, plt1_p1.c
+                                 Filter: ((plt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p1.b, plt2_p1.c
+                                 ->  Seq Scan on public.plt2_p1
+                                       Output: plt2_p1.b, plt2_p1.c
+                                       Filter: ((plt2_p1.b % 25) = 0)
+                     ->  Hash Full Join
+                           Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+                           Hash Cond: ((plt1_p2.c = plt2_p2.c) AND (plt1_p2.a = plt2_p2.b))
+                           ->  Seq Scan on public.plt1_p2
+                                 Output: plt1_p2.a, plt1_p2.c
+                                 Filter: ((plt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p2.b, plt2_p2.c
+                                 ->  Seq Scan on public.plt2_p2
+                                       Output: plt2_p2.b, plt2_p2.c
+                                       Filter: ((plt2_p2.b % 25) = 0)
+                     ->  Hash Full Join
+                           Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+                           Hash Cond: ((plt1_p3.c = plt2_p3.c) AND (plt1_p3.a = plt2_p3.b))
+                           ->  Seq Scan on public.plt1_p3
+                                 Output: plt1_p3.a, plt1_p3.c
+                                 Filter: ((plt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p3.b, plt2_p3.c
+                                 ->  Seq Scan on public.plt2_p3
+                                       Output: plt2_p3.b, plt2_p3.c
+                                       Filter: ((plt2_p3.b % 25) = 0)
+(42 rows)
+
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM (SELECT 50 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+ sum |  c   |          avg           |  c   
+-----+------+------------------------+------
+   0 | 0000 | 0.00000000000000000000 | 0000
+  50 | 0001 |                        | 
+ 100 | 0002 |                        | 
+ 150 | 0003 |   150.0000000000000000 | 0003
+ 200 | 0004 |                        | 
+ 250 | 0005 |                        | 
+ 300 | 0006 |   300.0000000000000000 | 0006
+ 350 | 0007 |                        | 
+ 400 | 0008 |                        | 
+ 450 | 0009 |   450.0000000000000000 | 0009
+ 500 | 0010 |                        | 
+ 550 | 0011 |                        | 
+     |      |    75.0000000000000000 | 0001
+     |      |   225.0000000000000000 | 0004
+     |      |   375.0000000000000000 | 0007
+     |      |   525.0000000000000000 | 0010
+(16 rows)
+
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM (SELECT 50 phv, * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+ sum |  c   |          avg           |  c   
+-----+------+------------------------+------
+   0 | 0000 | 0.00000000000000000000 | 0000
+  50 | 0001 |                        | 
+ 100 | 0002 |                        | 
+ 150 | 0003 |   150.0000000000000000 | 0003
+ 200 | 0004 |                        | 
+ 250 | 0005 |                        | 
+ 300 | 0006 |   300.0000000000000000 | 0006
+ 350 | 0007 |                        | 
+ 400 | 0008 |                        | 
+ 450 | 0009 |   450.0000000000000000 | 0009
+ 500 | 0010 |                        | 
+ 550 | 0011 |                        | 
+     |      |    75.0000000000000000 | 0001
+     |      |   225.0000000000000000 | 0004
+     |      |   375.0000000000000000 | 0007
+     |      |   525.0000000000000000 | 0010
+(16 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t1.phv), avg(t2.b), t2.c, avg(t2.phv) FROM (SELECT 25 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ GroupAggregate
+   Output: sum(plt1_p1.a), plt1_p1.c, sum((25)), avg(plt2_p1.b), plt2_p1.c, avg((50))
+   Group Key: plt1_p1.c, plt2_p1.c
+   ->  Sort
+         Output: plt1_p1.c, plt2_p1.c, plt1_p1.a, (25), plt2_p1.b, (50)
+         Sort Key: plt1_p1.c, plt2_p1.c
+         ->  Result
+               Output: plt1_p1.c, plt2_p1.c, plt1_p1.a, (25), plt2_p1.b, (50)
+               ->  Append
+                     ->  Hash Full Join
+                           Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, (25), (50)
+                           Hash Cond: ((plt1_p1.c = plt2_p1.c) AND (plt1_p1.a = plt2_p1.b))
+                           ->  Seq Scan on public.plt1_p1
+                                 Output: plt1_p1.a, plt1_p1.c, 25
+                                 Filter: ((plt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p1.b, plt2_p1.c, (50)
+                                 ->  Seq Scan on public.plt2_p1
+                                       Output: plt2_p1.b, plt2_p1.c, 50
+                                       Filter: ((plt2_p1.b % 25) = 0)
+                     ->  Hash Full Join
+                           Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c, (25), (50)
+                           Hash Cond: ((plt1_p2.c = plt2_p2.c) AND (plt1_p2.a = plt2_p2.b))
+                           ->  Seq Scan on public.plt1_p2
+                                 Output: plt1_p2.a, plt1_p2.c, 25
+                                 Filter: ((plt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p2.b, plt2_p2.c, (50)
+                                 ->  Seq Scan on public.plt2_p2
+                                       Output: plt2_p2.b, plt2_p2.c, 50
+                                       Filter: ((plt2_p2.b % 25) = 0)
+                     ->  Hash Full Join
+                           Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c, (25), (50)
+                           Hash Cond: ((plt1_p3.c = plt2_p3.c) AND (plt1_p3.a = plt2_p3.b))
+                           ->  Seq Scan on public.plt1_p3
+                                 Output: plt1_p3.a, plt1_p3.c, 25
+                                 Filter: ((plt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p3.b, plt2_p3.c, (50)
+                                 ->  Seq Scan on public.plt2_p3
+                                       Output: plt2_p3.b, plt2_p3.c, 50
+                                       Filter: ((plt2_p3.b % 25) = 0)
+(42 rows)
+
+SELECT sum(t1.a), t1.c, sum(t1.phv), avg(t2.b), t2.c, avg(t2.phv) FROM (SELECT 25 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+ sum |  c   | sum |          avg           |  c   |         avg         
+-----+------+-----+------------------------+------+---------------------
+   0 | 0000 |  25 | 0.00000000000000000000 | 0000 | 50.0000000000000000
+  50 | 0001 |  25 |                        |      |                    
+ 100 | 0002 |  25 |                        |      |                    
+ 150 | 0003 |  25 |   150.0000000000000000 | 0003 | 50.0000000000000000
+ 200 | 0004 |  25 |                        |      |                    
+ 250 | 0005 |  25 |                        |      |                    
+ 300 | 0006 |  25 |   300.0000000000000000 | 0006 | 50.0000000000000000
+ 350 | 0007 |  25 |                        |      |                    
+ 400 | 0008 |  25 |                        |      |                    
+ 450 | 0009 |  25 |   450.0000000000000000 | 0009 | 50.0000000000000000
+ 500 | 0010 |  25 |                        |      |                    
+ 550 | 0011 |  25 |                        |      |                    
+     |      |     |    75.0000000000000000 | 0001 | 50.0000000000000000
+     |      |     |   225.0000000000000000 | 0004 | 50.0000000000000000
+     |      |     |   375.0000000000000000 | 0007 | 50.0000000000000000
+     |      |     |   525.0000000000000000 | 0010 | 50.0000000000000000
+(16 rows)
+
+SELECT sum(t1.a), t1.c, sum(t1.phv), avg(t2.b), t2.c, avg(t2.phv) FROM (SELECT 25 phv, * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+ sum |  c   | sum |          avg           |  c   |         avg         
+-----+------+-----+------------------------+------+---------------------
+   0 | 0000 |  25 | 0.00000000000000000000 | 0000 | 50.0000000000000000
+  50 | 0001 |  25 |                        |      |                    
+ 100 | 0002 |  25 |                        |      |                    
+ 150 | 0003 |  25 |   150.0000000000000000 | 0003 | 50.0000000000000000
+ 200 | 0004 |  25 |                        |      |                    
+ 250 | 0005 |  25 |                        |      |                    
+ 300 | 0006 |  25 |   300.0000000000000000 | 0006 | 50.0000000000000000
+ 350 | 0007 |  25 |                        |      |                    
+ 400 | 0008 |  25 |                        |      |                    
+ 450 | 0009 |  25 |   450.0000000000000000 | 0009 | 50.0000000000000000
+ 500 | 0010 |  25 |                        |      |                    
+ 550 | 0011 |  25 |                        |      |                    
+     |      |     |    75.0000000000000000 | 0001 | 50.0000000000000000
+     |      |     |   225.0000000000000000 | 0004 | 50.0000000000000000
+     |      |     |   375.0000000000000000 | 0007 | 50.0000000000000000
+     |      |     |   525.0000000000000000 | 0010 | 50.0000000000000000
+(16 rows)
+
+-- Join with pruned partitions from joining relations
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.c NOT IN ('0001', '0005', '0002', '0009') AND t2.c NOT IN ('0000', '0003', '0004', '0010') GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Output: (sum(t1.a)), t1.c, (avg(t2.b)), t2.c
+   Sort Key: t1.c
+   ->  HashAggregate
+         Output: sum(t1.a), t1.c, avg(t2.b), t2.c
+         Group Key: t1.c, t2.c
+         ->  Result
+               Output: t1.c, t2.c, t1.a, t2.b
+               ->  Append
+                     ->  Hash Join
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           Hash Cond: (t1.c = t2.c)
+                           ->  Seq Scan on public.plt1_p3 t1
+                                 Output: t1.a, t1.c
+                                 Filter: (t1.c <> ALL ('{0001,0005,0002,0009}'::text[]))
+                           ->  Hash
+                                 Output: t2.b, t2.c
+                                 ->  Seq Scan on public.plt2_p3 t2
+                                       Output: t2.b, t2.c
+                                       Filter: (t2.c <> ALL ('{0000,0003,0004,0010}'::text[]))
+(20 rows)
+
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.c NOT IN ('0001', '0005', '0002', '0009') AND t2.c NOT IN ('0000', '0003', '0004', '0010') GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+  sum   |  c   |         avg          |  c   
+--------+------+----------------------+------
+ 137700 | 0006 | 324.0000000000000000 | 0006
+ 158950 | 0007 | 375.0000000000000000 | 0007
+ 169600 | 0008 | 424.5000000000000000 | 0008
+ 229600 | 0011 | 574.5000000000000000 | 0011
+(4 rows)
+
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM uplt1 t1, uplt2 t2 WHERE t1.c = t2.c AND t1.c NOT IN ('0001', '0005', '0002', '0009') AND t2.c NOT IN ('0000', '0003', '0004', '0010') GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+  sum   |  c   |         avg          |  c   
+--------+------+----------------------+------
+ 137700 | 0006 | 324.0000000000000000 | 0006
+ 158950 | 0007 | 375.0000000000000000 | 0007
+ 169600 | 0008 | 424.5000000000000000 | 0008
+ 229600 | 0011 | 574.5000000000000000 | 0011
+(4 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 LEFT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Output: (sum(t1.a)), t1.c, (sum(b)), c
+   Sort Key: t1.c, c
+   ->  HashAggregate
+         Output: sum(t1.a), t1.c, sum(b), c
+         Group Key: t1.c, c
+         ->  Result
+               Output: t1.c, c, t1.a, b
+               ->  Append
+                     ->  Hash Left Join
+                           Output: t1.a, t1.c, b, c
+                           Hash Cond: (t1.c = c)
+                           ->  Seq Scan on public.plt1_p1 t1
+                                 Output: t1.a, t1.c
+                                 Filter: (t1.c <> ALL ('{0001,0005,0002,0009}'::text[]))
+                           ->  Hash
+                                 Output: b, c
+                                 ->  Result
+                                       Output: b, c
+                                       One-Time Filter: false
+                     ->  Hash Left Join
+                           Output: t1_1.a, t1_1.c, t2.b, t2.c
+                           Hash Cond: (t1_1.c = t2.c)
+                           ->  Seq Scan on public.plt1_p3 t1_1
+                                 Output: t1_1.a, t1_1.c
+                                 Filter: (t1_1.c <> ALL ('{0001,0005,0002,0009}'::text[]))
+                           ->  Hash
+                                 Output: t2.b, t2.c
+                                 ->  Seq Scan on public.plt2_p3 t2
+                                       Output: t2.b, t2.c
+                                       Filter: (t2.c <> ALL ('{0000,0003,0004,0010}'::text[]))
+(31 rows)
+
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 LEFT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  sum   |  c   |  sum   |  c   
+--------+------+--------+------
+    600 | 0000 |        | 
+   4350 | 0003 |        | 
+   5600 | 0004 |        | 
+ 137700 | 0006 | 137700 | 0006
+ 158950 | 0007 | 159375 | 0007
+ 169600 | 0008 | 169800 | 0008
+  13100 | 0010 |        | 
+ 229600 | 0011 | 229800 | 0011
+(8 rows)
+
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM uplt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 LEFT JOIN (SELECT * FROM uplt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+  sum   |  c   |  sum   |  c   
+--------+------+--------+------
+    600 | 0000 |        | 
+   4350 | 0003 |        | 
+   5600 | 0004 |        | 
+ 137700 | 0006 | 137700 | 0006
+ 158950 | 0007 | 159375 | 0007
+ 169600 | 0008 | 169800 | 0008
+  13100 | 0010 |        | 
+ 229600 | 0011 | 229800 | 0011
+(8 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 RIGHT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+NOTICE:  join between relations (b 5) and (b 4) is considered for partition-wise join.
+                                           QUERY PLAN                                            
+-------------------------------------------------------------------------------------------------
+ Sort
+   Output: (sum(a)), c, (sum(t2.b)), t2.c
+   Sort Key: c, t2.c
+   ->  HashAggregate
+         Output: sum(a), c, sum(t2.b), t2.c
+         Group Key: c, t2.c
+         ->  Result
+               Output: c, t2.c, a, t2.b
+               ->  Append
+                     ->  Hash Left Join
+                           Output: t2.b, t2.c, a, c
+                           Hash Cond: (t2.c = c)
+                           ->  Seq Scan on public.plt2_p2 t2
+                                 Output: t2.b, t2.c
+                                 Filter: (t2.c <> ALL ('{0000,0003,0004,0010}'::text[]))
+                           ->  Hash
+                                 Output: a, c
+                                 ->  Result
+                                       Output: a, c
+                                       One-Time Filter: false
+                     ->  Hash Right Join
+                           Output: t2_1.b, t2_1.c, t1.a, t1.c
+                           Hash Cond: (t1.c = t2_1.c)
+                           ->  Seq Scan on public.plt1_p3 t1
+                                 Output: t1.a, t1.c
+                                 Filter: (t1.c <> ALL ('{0001,0005,0002,0009}'::text[]))
+                           ->  Hash
+                                 Output: t2_1.b, t2_1.c
+                                 ->  Seq Scan on public.plt2_p3 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                       Filter: (t2_1.c <> ALL ('{0000,0003,0004,0010}'::text[]))
+(31 rows)
+
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 RIGHT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+NOTICE:  join between relations (b 5) and (b 4) is considered for partition-wise join.
+  sum   |  c   |  sum   |  c   
+--------+------+--------+------
+ 137700 | 0006 | 137700 | 0006
+ 158950 | 0007 | 159375 | 0007
+ 169600 | 0008 | 169800 | 0008
+ 229600 | 0011 | 229800 | 0011
+        |      |   1275 | 0001
+        |      |   1992 | 0002
+        |      |   4392 | 0005
+        |      |   8058 | 0009
+(8 rows)
+
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM uplt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 RIGHT JOIN (SELECT * FROM uplt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+  sum   |  c   |  sum   |  c   
+--------+------+--------+------
+ 137700 | 0006 | 137700 | 0006
+ 158950 | 0007 | 159375 | 0007
+ 169600 | 0008 | 169800 | 0008
+ 229600 | 0011 | 229800 | 0011
+        |      |   1275 | 0001
+        |      |   1992 | 0002
+        |      |   4392 | 0005
+        |      |   8058 | 0009
+(8 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 FULL JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                           QUERY PLAN                                            
+-------------------------------------------------------------------------------------------------
+ Sort
+   Output: (sum(t1.a)), t1.c, (sum(b)), c
+   Sort Key: t1.c, c
+   ->  HashAggregate
+         Output: sum(t1.a), t1.c, sum(b), c
+         Group Key: t1.c, c
+         ->  Result
+               Output: t1.c, c, t1.a, b
+               ->  Append
+                     ->  Hash Full Join
+                           Output: t1.a, t1.c, b, c
+                           Hash Cond: (t1.c = c)
+                           ->  Seq Scan on public.plt1_p1 t1
+                                 Output: t1.a, t1.c
+                                 Filter: (t1.c <> ALL ('{0001,0005,0002,0009}'::text[]))
+                           ->  Hash
+                                 Output: b, c
+                                 ->  Result
+                                       Output: b, c
+                                       One-Time Filter: false
+                     ->  Hash Full Join
+                           Output: a, c, t2.b, t2.c
+                           Hash Cond: (t2.c = c)
+                           ->  Seq Scan on public.plt2_p2 t2
+                                 Output: t2.b, t2.c
+                                 Filter: (t2.c <> ALL ('{0000,0003,0004,0010}'::text[]))
+                           ->  Hash
+                                 Output: a, c
+                                 ->  Result
+                                       Output: a, c
+                                       One-Time Filter: false
+                     ->  Hash Full Join
+                           Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                           Hash Cond: (t1_1.c = t2_1.c)
+                           ->  Seq Scan on public.plt1_p3 t1_1
+                                 Output: t1_1.a, t1_1.c
+                                 Filter: (t1_1.c <> ALL ('{0001,0005,0002,0009}'::text[]))
+                           ->  Hash
+                                 Output: t2_1.b, t2_1.c
+                                 ->  Seq Scan on public.plt2_p3 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                       Filter: (t2_1.c <> ALL ('{0000,0003,0004,0010}'::text[]))
+(42 rows)
+
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 FULL JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  sum   |  c   |  sum   |  c   
+--------+------+--------+------
+    600 | 0000 |        | 
+   4350 | 0003 |        | 
+   5600 | 0004 |        | 
+ 137700 | 0006 | 137700 | 0006
+ 158950 | 0007 | 159375 | 0007
+ 169600 | 0008 | 169800 | 0008
+  13100 | 0010 |        | 
+ 229600 | 0011 | 229800 | 0011
+        |      |   1275 | 0001
+        |      |   1992 | 0002
+        |      |   4392 | 0005
+        |      |   8058 | 0009
+(12 rows)
+
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM uplt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 FULL JOIN (SELECT * FROM uplt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+  sum   |  c   |  sum   |  c   
+--------+------+--------+------
+    600 | 0000 |        | 
+   4350 | 0003 |        | 
+   5600 | 0004 |        | 
+ 137700 | 0006 | 137700 | 0006
+ 158950 | 0007 | 159375 | 0007
+ 169600 | 0008 | 169800 | 0008
+  13100 | 0010 |        | 
+ 229600 | 0011 | 229800 | 0011
+        |      |   1275 | 0001
+        |      |   1992 | 0002
+        |      |   4392 | 0005
+        |      |   8058 | 0009
+(12 rows)
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 1) and (b 3) is considered for partition-wise join.
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.c = t1_3.c)
+               ->  Seq Scan on public.plt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_3.c
+                     ->  Seq Scan on public.plt2_p1 t1_3
+                           Output: t1_3.c
+                           Filter: ((t1_3.b % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.c = t1_4.c)
+               ->  Seq Scan on public.plt1_p2 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_4.c
+                     ->  Seq Scan on public.plt2_p2 t1_4
+                           Output: t1_4.c
+                           Filter: ((t1_4.b % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.c = t1_5.c)
+               ->  Seq Scan on public.plt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_5.c
+                     ->  Seq Scan on public.plt2_p3 t1_5
+                           Output: t1_5.c
+                           Filter: ((t1_5.b % 25) = 0)
+(37 rows)
+
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 1) and (b 3) is considered for partition-wise join.
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+(8 rows)
+
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+(8 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;
+ANALYZE plt1_e;
+ANALYZE plt1_e_p1;
+ANALYZE plt1_e_p2;
+ANALYZE plt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1_e AS SELECT * FROM plt1_e;
+CREATE TABLE plt2_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
+CREATE TABLE plt2_e_p1 PARTITION OF plt2_e FOR VALUES IN ('0000', '0003', '0004', '0010');
+CREATE TABLE plt2_e_p2 PARTITION OF plt2_e FOR VALUES IN ('0001', '0005', '0002', '0009');
+CREATE TABLE plt2_e_p3 PARTITION OF plt2_e FOR VALUES IN ('0006', '0007', '0008', '0011');
+INSERT INTO plt2_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE plt2_e;
+ANALYZE plt2_e_p1;
+ANALYZE plt2_e_p2;
+ANALYZE plt2_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt2_e AS SELECT * FROM plt2_e;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1, plt2_e t2 WHERE t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((t2.b = t1.a) AND (ltrim(t2.c, 'A'::text) = ltrim(t1.c, 'A'::text)))
+               ->  Seq Scan on public.plt2_e_p1 t2
+                     Output: t2.b, t2.c
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.plt1_e_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((t2_1.b = t1_1.a) AND (ltrim(t2_1.c, 'A'::text) = ltrim(t1_1.c, 'A'::text)))
+               ->  Seq Scan on public.plt2_e_p2 t2_1
+                     Output: t2_1.b, t2_1.c
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.plt1_e_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((t2_2.b = t1_2.a) AND (ltrim(t2_2.c, 'A'::text) = ltrim(t1_2.c, 'A'::text)))
+               ->  Seq Scan on public.plt2_e_p3 t2_2
+                     Output: t2_2.b, t2_2.c
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.plt1_e_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1, plt2_e t2 WHERE t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+ 150 | A0003 | 150 | A0003
+ 300 | A0006 | 300 | A0006
+ 450 | A0009 | 450 | A0009
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_e t1, uplt2_e t2 WHERE t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+ 150 | A0003 | 150 | A0003
+ 300 | A0006 | 300 | A0006
+ 450 | A0009 | 450 | A0009
+(4 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 LEFT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((t2.b = t1.a) AND (ltrim(t2.c, 'A'::text) = ltrim(t1.c, 'A'::text)))
+               ->  Seq Scan on public.plt2_e_p1 t2
+                     Output: t2.b, t2.c
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.plt1_e_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((t2_1.b = t1_1.a) AND (ltrim(t2_1.c, 'A'::text) = ltrim(t1_1.c, 'A'::text)))
+               ->  Seq Scan on public.plt2_e_p2 t2_1
+                     Output: t2_1.b, t2_1.c
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.plt1_e_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((t2_2.b = t1_2.a) AND (ltrim(t2_2.c, 'A'::text) = ltrim(t1_2.c, 'A'::text)))
+               ->  Seq Scan on public.plt2_e_p3 t2_2
+                     Output: t2_2.b, t2_2.c
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.plt1_e_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 LEFT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+  50 | A0001 |     | 
+ 100 | A0002 |     | 
+ 150 | A0003 | 150 | A0003
+ 200 | A0004 |     | 
+ 250 | A0005 |     | 
+ 300 | A0006 | 300 | A0006
+ 350 | A0007 |     | 
+ 400 | A0008 |     | 
+ 450 | A0009 | 450 | A0009
+ 500 | A0010 |     | 
+ 550 | A0011 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_e t1 LEFT JOIN uplt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+  50 | A0001 |     | 
+ 100 | A0002 |     | 
+ 150 | A0003 | 150 | A0003
+ 200 | A0004 |     | 
+ 250 | A0005 |     | 
+ 300 | A0006 | 300 | A0006
+ 350 | A0007 |     | 
+ 400 | A0008 |     | 
+ 450 | A0009 | 450 | A0009
+ 500 | A0010 |     | 
+ 550 | A0011 |     | 
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 RIGHT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+                                                  QUERY PLAN                                                  
+--------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: ((t1.a = t2.b) AND (ltrim(t1.c, 'A'::text) = ltrim(t2.c, 'A'::text)))
+                     ->  Seq Scan on public.plt1_e_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t2.b, t2.c
+                           ->  Seq Scan on public.plt2_e_p1 t2
+                                 Output: t2.b, t2.c
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: ((t1_1.a = t2_1.b) AND (ltrim(t1_1.c, 'A'::text) = ltrim(t2_1.c, 'A'::text)))
+                     ->  Seq Scan on public.plt1_e_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c
+                           ->  Seq Scan on public.plt2_e_p2 t2_1
+                                 Output: t2_1.b, t2_1.c
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: ((t1_2.a = t2_2.b) AND (ltrim(t1_2.c, 'A'::text) = ltrim(t2_2.c, 'A'::text)))
+                     ->  Seq Scan on public.plt1_e_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c
+                           ->  Seq Scan on public.plt2_e_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+                                 Filter: ((t2_2.b % 25) = 0)
+(36 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 RIGHT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+ 150 | A0003 | 150 | A0003
+ 300 | A0006 | 300 | A0006
+ 450 | A0009 | 450 | A0009
+     |       |  75 | A0001
+     |       | 225 | A0004
+     |       | 375 | A0007
+     |       | 525 | A0010
+(8 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_e t1 RIGHT JOIN uplt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+ 150 | A0003 | 150 | A0003
+ 300 | A0006 | 300 | A0006
+ 450 | A0009 | 450 | A0009
+     |       |  75 | A0001
+     |       | 225 | A0004
+     |       | 375 | A0007
+     |       | 525 | A0010
+(8 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2_e WHERE plt2_e.b % 25 = 0) t2 ON (t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A')) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                                         QUERY PLAN                                                         
+----------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: plt1_e_p1.a, plt1_e_p1.c, plt2_e_p1.b, plt2_e_p1.c
+   Sort Key: plt1_e_p1.a, plt2_e_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: plt1_e_p1.a, plt1_e_p1.c, plt2_e_p1.b, plt2_e_p1.c
+               Hash Cond: ((plt1_e_p1.a = plt2_e_p1.b) AND (ltrim(plt1_e_p1.c, 'A'::text) = ltrim(plt2_e_p1.c, 'A'::text)))
+               ->  Seq Scan on public.plt1_e_p1
+                     Output: plt1_e_p1.a, plt1_e_p1.c
+                     Filter: ((plt1_e_p1.a % 25) = 0)
+               ->  Hash
+                     Output: plt2_e_p1.b, plt2_e_p1.c
+                     ->  Seq Scan on public.plt2_e_p1
+                           Output: plt2_e_p1.b, plt2_e_p1.c
+                           Filter: ((plt2_e_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: plt1_e_p2.a, plt1_e_p2.c, plt2_e_p2.b, plt2_e_p2.c
+               Hash Cond: ((plt1_e_p2.a = plt2_e_p2.b) AND (ltrim(plt1_e_p2.c, 'A'::text) = ltrim(plt2_e_p2.c, 'A'::text)))
+               ->  Seq Scan on public.plt1_e_p2
+                     Output: plt1_e_p2.a, plt1_e_p2.c
+                     Filter: ((plt1_e_p2.a % 25) = 0)
+               ->  Hash
+                     Output: plt2_e_p2.b, plt2_e_p2.c
+                     ->  Seq Scan on public.plt2_e_p2
+                           Output: plt2_e_p2.b, plt2_e_p2.c
+                           Filter: ((plt2_e_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: plt1_e_p3.a, plt1_e_p3.c, plt2_e_p3.b, plt2_e_p3.c
+               Hash Cond: ((plt1_e_p3.a = plt2_e_p3.b) AND (ltrim(plt1_e_p3.c, 'A'::text) = ltrim(plt2_e_p3.c, 'A'::text)))
+               ->  Seq Scan on public.plt1_e_p3
+                     Output: plt1_e_p3.a, plt1_e_p3.c
+                     Filter: ((plt1_e_p3.a % 25) = 0)
+               ->  Hash
+                     Output: plt2_e_p3.b, plt2_e_p3.c
+                     ->  Seq Scan on public.plt2_e_p3
+                           Output: plt2_e_p3.b, plt2_e_p3.c
+                           Filter: ((plt2_e_p3.b % 25) = 0)
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2_e WHERE plt2_e.b % 25 = 0) t2 ON (t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A')) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+  50 | A0001 |     | 
+ 100 | A0002 |     | 
+ 150 | A0003 | 150 | A0003
+ 200 | A0004 |     | 
+ 250 | A0005 |     | 
+ 300 | A0006 | 300 | A0006
+ 350 | A0007 |     | 
+ 400 | A0008 |     | 
+ 450 | A0009 | 450 | A0009
+ 500 | A0010 |     | 
+ 550 | A0011 |     | 
+     |       |  75 | A0001
+     |       | 225 | A0004
+     |       | 375 | A0007
+     |       | 525 | A0010
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uplt1_e t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2_e t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A')) ORDER BY t1.a, t2.b;
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+  50 | A0001 |     | 
+ 100 | A0002 |     | 
+ 150 | A0003 | 150 | A0003
+ 200 | A0004 |     | 
+ 250 | A0005 |     | 
+ 300 | A0006 | 300 | A0006
+ 350 | A0007 |     | 
+ 400 | A0008 |     | 
+ 450 | A0009 | 450 | A0009
+ 500 | A0010 |     | 
+ 550 | A0011 |     | 
+     |       |  75 | A0001
+     |       | 225 | A0004
+     |       | 375 | A0007
+     |       | 525 | A0010
+(16 rows)
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 3) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 3) and (b 1) is considered for partition-wise join.
+                                        QUERY PLAN                                        
+------------------------------------------------------------------------------------------
+ Sort
+   Output: (avg(t1.a)), (avg(t2.b)), (avg((t3.a + t3.b))), t1.c, t2.c, t3.c
+   Sort Key: t1.c, t3.c
+   ->  HashAggregate
+         Output: avg(t1.a), avg(t2.b), avg((t3.a + t3.b)), t1.c, t2.c, t3.c
+         Group Key: t1.c, t2.c, t3.c
+         ->  Result
+               Output: t1.c, t2.c, t3.c, t1.a, t2.b, t3.a, t3.b
+               ->  Append
+                     ->  Hash Join
+                           Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                           Hash Cond: (t1.c = t2.c)
+                           ->  Seq Scan on public.plt1_p1 t1
+                                 Output: t1.a, t1.c
+                           ->  Hash
+                                 Output: t2.b, t2.c, t3.a, t3.b, t3.c
+                                 ->  Hash Join
+                                       Output: t2.b, t2.c, t3.a, t3.b, t3.c
+                                       Hash Cond: (t2.c = ltrim(t3.c, 'A'::text))
+                                       ->  Seq Scan on public.plt2_p1 t2
+                                             Output: t2.b, t2.c
+                                       ->  Hash
+                                             Output: t3.a, t3.b, t3.c
+                                             ->  Seq Scan on public.plt1_e_p1 t3
+                                                   Output: t3.a, t3.b, t3.c
+                     ->  Hash Join
+                           Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                           Hash Cond: (t1_1.c = t2_1.c)
+                           ->  Seq Scan on public.plt1_p2 t1_1
+                                 Output: t1_1.a, t1_1.c
+                           ->  Hash
+                                 Output: t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                                 ->  Hash Join
+                                       Output: t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                                       Hash Cond: (t2_1.c = ltrim(t3_1.c, 'A'::text))
+                                       ->  Seq Scan on public.plt2_p2 t2_1
+                                             Output: t2_1.b, t2_1.c
+                                       ->  Hash
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             ->  Seq Scan on public.plt1_e_p2 t3_1
+                                                   Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash Join
+                           Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                           Hash Cond: (t1_2.c = t2_2.c)
+                           ->  Seq Scan on public.plt1_p3 t1_2
+                                 Output: t1_2.a, t1_2.c
+                           ->  Hash
+                                 Output: t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                                 ->  Hash Join
+                                       Output: t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                                       Hash Cond: (t2_2.c = ltrim(t3_2.c, 'A'::text))
+                                       ->  Seq Scan on public.plt2_p3 t2_2
+                                             Output: t2_2.b, t2_2.c
+                                       ->  Hash
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             ->  Seq Scan on public.plt1_e_p3 t3_2
+                                                   Output: t3_2.a, t3_2.b, t3_2.c
+(57 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;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 3) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 3) and (b 1) is considered for partition-wise join.
+         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)
+
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM uplt1 t1, uplt2 t2, uplt1_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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: ((t3.a = t1.a) AND (ltrim(t3.c, 'A'::text) = t1.c))
+                     ->  Seq Scan on public.plt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                                 ->  Seq Scan on public.plt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.plt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: ((t3_1.a = t1_1.a) AND (ltrim(t3_1.c, 'A'::text) = t1_1.c))
+                     ->  Seq Scan on public.plt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.c = t1_1.c))
+                                 ->  Seq Scan on public.plt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.plt1_p2 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: ((t3_2.a = t1_2.a) AND (ltrim(t3_2.c, 'A'::text) = t1_2.c))
+                     ->  Seq Scan on public.plt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                                 ->  Seq Scan on public.plt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.plt1_p3 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2 4) is considered for partition-wise join.
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: ((t3.a = t2.b) AND (ltrim(t3.c, 'A'::text) = t2.c))
+                     ->  Seq Scan on public.plt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                                 ->  Seq Scan on public.plt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.plt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: ((t3_1.a = t2_1.b) AND (ltrim(t3_1.c, 'A'::text) = t2_1.c))
+                     ->  Seq Scan on public.plt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.c = t1_1.c))
+                                 ->  Seq Scan on public.plt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.plt1_p2 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: ((t3_2.a = t2_2.b) AND (ltrim(t3_2.c, 'A'::text) = t2_2.c))
+                     ->  Seq Scan on public.plt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                                 ->  Seq Scan on public.plt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.plt1_p3 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 2) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2 4) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |          | 
+ 100 | 0002 |     |      |          | 
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |          | 
+ 250 | 0005 |     |      |          | 
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |          | 
+ 400 | 0008 |     |      |          | 
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |          | 
+ 550 | 0011 |     |      |          | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |          | 
+ 100 | 0002 |     |      |          | 
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |          | 
+ 250 | 0005 |     |      |          | 
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |          | 
+ 400 | 0008 |     |      |          | 
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |          | 
+ 550 | 0011 |     |      |          | 
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Hash Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                     ->  Seq Scan on public.plt2_p1 t2
+                           Output: t2.b, t2.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Hash Cond: ((t1.c = ltrim(t3.c, 'A'::text)) AND (t1.a = t3.a))
+                                 ->  Seq Scan on public.plt1_p1 t1
+                                       Output: t1.a, t1.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.plt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                     Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.c = t1_1.c))
+                     ->  Seq Scan on public.plt2_p2 t2_1
+                           Output: t2_1.b, t2_1.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_1.a, t1_1.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_1.a, t1_1.c
+                                 Hash Cond: ((t1_1.c = ltrim(t3_1.c, 'A'::text)) AND (t1_1.a = t3_1.a))
+                                 ->  Seq Scan on public.plt1_p2 t1_1
+                                       Output: t1_1.a, t1_1.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.plt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                     Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                     ->  Seq Scan on public.plt2_p3 t2_2
+                           Output: t2_2.b, t2_2.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_2.a, t1_2.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_2.a, t1_2.c
+                                 Hash Cond: ((t1_2.c = ltrim(t3_2.c, 'A'::text)) AND (t1_2.a = t3_2.a))
+                                 ->  Seq Scan on public.plt1_p3 t1_2
+                                       Output: t1_2.a, t1_2.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.plt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 1 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: ((t1.a = t2.b) AND (t1.c = t2.c))
+                     ->  Seq Scan on public.plt1_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Hash Cond: ((t2.b = t3.a) AND (t2.c = ltrim(t3.c, 'A'::text)))
+                                 ->  Seq Scan on public.plt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.plt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: ((t1_1.a = t2_1.b) AND (t1_1.c = t2_1.c))
+                     ->  Seq Scan on public.plt1_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: ((t2_1.b = t3_1.a) AND (t2_1.c = ltrim(t3_1.c, 'A'::text)))
+                                 ->  Seq Scan on public.plt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.plt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: ((t1_2.a = t2_2.b) AND (t1_2.c = t2_2.c))
+                     ->  Seq Scan on public.plt1_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: ((t2_2.b = t3_2.a) AND (t2_2.c = ltrim(t3_2.c, 'A'::text)))
+                                 ->  Seq Scan on public.plt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.plt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+     |      |     |      |      100 | A0001
+     |      |     |      |      200 | A0002
+     |      |     |      |      400 | A0004
+     |      |     |      |      500 | A0005
+     |      |     |      |      700 | A0007
+     |      |     |      |      800 | A0008
+     |      |     |      |     1000 | A0010
+     |      |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+     |      |     |      |      100 | A0001
+     |      |     |      |      200 | A0002
+     |      |     |      |      400 | A0004
+     |      |     |      |      500 | A0005
+     |      |     |      |      700 | A0007
+     |      |     |      |      800 | A0008
+     |      |     |      |     1000 | A0010
+     |      |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 6) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is considered for partition-wise join.
+                                                  QUERY PLAN                                                   
+---------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, ((plt1_e_p1.a + plt1_e_p1.b)), plt1_e_p1.c
+   Sort Key: plt1_p1.a, plt2_p1.b, ((plt1_e_p1.a + plt1_e_p1.b))
+   ->  Result
+         Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, (plt1_e_p1.a + plt1_e_p1.b), plt1_e_p1.c
+         ->  Append
+               ->  Hash Full Join
+                     Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                     Hash Cond: ((plt1_p1.a = plt1_e_p1.a) AND (plt1_p1.c = ltrim(plt1_e_p1.c, 'A'::text)))
+                     ->  Hash Full Join
+                           Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+                           Hash Cond: ((plt1_p1.a = plt2_p1.b) AND (plt1_p1.c = plt2_p1.c))
+                           ->  Seq Scan on public.plt1_p1
+                                 Output: plt1_p1.a, plt1_p1.c
+                                 Filter: ((plt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p1.b, plt2_p1.c
+                                 ->  Seq Scan on public.plt2_p1
+                                       Output: plt2_p1.b, plt2_p1.c
+                                       Filter: ((plt2_p1.b % 25) = 0)
+                     ->  Hash
+                           Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                           ->  Seq Scan on public.plt1_e_p1
+                                 Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                                 Filter: ((plt1_e_p1.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c, plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                     Hash Cond: ((plt1_p2.a = plt1_e_p2.a) AND (plt1_p2.c = ltrim(plt1_e_p2.c, 'A'::text)))
+                     ->  Hash Full Join
+                           Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+                           Hash Cond: ((plt1_p2.a = plt2_p2.b) AND (plt1_p2.c = plt2_p2.c))
+                           ->  Seq Scan on public.plt1_p2
+                                 Output: plt1_p2.a, plt1_p2.c
+                                 Filter: ((plt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p2.b, plt2_p2.c
+                                 ->  Seq Scan on public.plt2_p2
+                                       Output: plt2_p2.b, plt2_p2.c
+                                       Filter: ((plt2_p2.b % 25) = 0)
+                     ->  Hash
+                           Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                           ->  Seq Scan on public.plt1_e_p2
+                                 Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                                 Filter: ((plt1_e_p2.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c, plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                     Hash Cond: ((plt1_p3.a = plt1_e_p3.a) AND (plt1_p3.c = ltrim(plt1_e_p3.c, 'A'::text)))
+                     ->  Hash Full Join
+                           Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+                           Hash Cond: ((plt1_p3.a = plt2_p3.b) AND (plt1_p3.c = plt2_p3.c))
+                           ->  Seq Scan on public.plt1_p3
+                                 Output: plt1_p3.a, plt1_p3.c
+                                 Filter: ((plt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p3.b, plt2_p3.c
+                                 ->  Seq Scan on public.plt2_p3
+                                       Output: plt2_p3.b, plt2_p3.c
+                                       Filter: ((plt2_p3.b % 25) = 0)
+                     ->  Hash
+                           Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                           ->  Seq Scan on public.plt1_e_p3
+                                 Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                                 Filter: ((plt1_e_p3.a % 25) = 0)
+(63 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 6) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 4) is considered for partition-wise join.
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.c = t1_3.c)
+               ->  Seq Scan on public.plt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_3.c, t2.c
+                     Hash Cond: (t1_3.c = ltrim(t2.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p1 t1_3
+                           Output: t1_3.c
+                     ->  Hash
+                           Output: t2.c
+                           ->  Seq Scan on public.plt1_e_p1 t2
+                                 Output: t2.c
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.c = t1_4.c)
+               ->  Seq Scan on public.plt1_p2 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_4.c, t2_1.c
+                     Hash Cond: (t1_4.c = ltrim(t2_1.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p2 t1_4
+                           Output: t1_4.c
+                     ->  Hash
+                           Output: t2_1.c
+                           ->  Seq Scan on public.plt1_e_p2 t2_1
+                                 Output: t2_1.c
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.c = t1_5.c)
+               ->  Seq Scan on public.plt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_5.c, t2_2.c
+                     Hash Cond: (t1_5.c = ltrim(t2_2.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p3 t1_5
+                           Output: t1_5.c
+                     ->  Hash
+                           Output: t2_2.c
+                           ->  Seq Scan on public.plt1_e_p3 t2_2
+                                 Output: t2_2.c
+(49 rows)
+
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 4) is considered for partition-wise join.
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1, uplt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 5) is considered for partition-wise join.
+                                   QUERY PLAN                                   
+--------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.c = t1_3.c)
+               ->  Seq Scan on public.plt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_3.c, t1_6.c
+                     Hash Cond: (t1_3.c = ltrim(t1_6.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p1 t1_3
+                           Output: t1_3.c
+                     ->  Hash
+                           Output: t1_6.c
+                           ->  HashAggregate
+                                 Output: t1_6.c
+                                 Group Key: ltrim(t1_6.c, 'A'::text)
+                                 ->  Seq Scan on public.plt1_e_p1 t1_6
+                                       Output: t1_6.c, ltrim(t1_6.c, 'A'::text)
+                                       Filter: ((t1_6.a % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.c = t1_4.c)
+               ->  Seq Scan on public.plt1_p2 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_4.c, t1_7.c
+                     Hash Cond: (t1_4.c = ltrim(t1_7.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p2 t1_4
+                           Output: t1_4.c
+                     ->  Hash
+                           Output: t1_7.c
+                           ->  HashAggregate
+                                 Output: t1_7.c
+                                 Group Key: ltrim(t1_7.c, 'A'::text)
+                                 ->  Seq Scan on public.plt1_e_p2 t1_7
+                                       Output: t1_7.c, ltrim(t1_7.c, 'A'::text)
+                                       Filter: ((t1_7.a % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.c = t1_5.c)
+               ->  Seq Scan on public.plt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_5.c, t1_8.c
+                     Hash Cond: (t1_5.c = ltrim(t1_8.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p3 t1_5
+                           Output: t1_5.c
+                     ->  Hash
+                           Output: t1_8.c
+                           ->  HashAggregate
+                                 Output: t1_8.c
+                                 Group Key: ltrim(t1_8.c, 'A'::text)
+                                 ->  Seq Scan on public.plt1_e_p3 t1_8
+                                       Output: t1_8.c, ltrim(t1_8.c, 'A'::text)
+                                       Filter: ((t1_8.a % 25) = 0)
+(61 rows)
+
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 5) is considered for partition-wise join.
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+-- test merge join with and without index scan
+CREATE INDEX iplt1_c on plt1(c);
+CREATE INDEX iplt1_p1_c on plt1_p1(c);
+CREATE INDEX iplt1_p2_c on plt1_p2(c);
+CREATE INDEX iplt1_p3_c on plt1_p3(c);
+CREATE INDEX iplt2_c on plt2(c);
+CREATE INDEX iplt2_p1_c on plt2_p1(c);
+CREATE INDEX iplt2_p2_c on plt2_p2(c);
+CREATE INDEX iplt2_p3_c on plt2_p3(c);
+CREATE INDEX iplt1_e_c on plt1_e(ltrim(c, 'A'));
+CREATE INDEX iplt1_e_p1_c on plt1_e_p1(ltrim(c, 'A'));
+CREATE INDEX iplt1_e_p2_c on plt1_e_p2(ltrim(c, 'A'));
+CREATE INDEX iplt1_e_p3_c on plt1_e_p3(ltrim(c, 'A'));
+ANALYZE plt1;
+ANALYZE plt1_p1;
+ANALYZE plt1_p2;
+ANALYZE plt1_p3;
+ANALYZE plt2;
+ANALYZE plt2_p1;
+ANALYZE plt2_p2;
+ANALYZE plt2_p3;
+ANALYZE plt1_e;
+ANALYZE plt1_e_p1;
+ANALYZE plt1_e_p2;
+ANALYZE plt1_e_p3;
+SET enable_hashjoin TO off;
+SET enable_nestloop TO off;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+                                                QUERY PLAN                                                 
+-----------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Merge Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           Sort Key: t2.b, t2.c
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Merge Cond: ((t3.a = t2.b) AND ((ltrim(t3.c, 'A'::text)) = t2.c))
+                                 ->  Sort
+                                       Output: t3.a, t3.b, t3.c, (ltrim(t3.c, 'A'::text))
+                                       Sort Key: t3.a, (ltrim(t3.c, 'A'::text))
+                                       ->  Seq Scan on public.plt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c, ltrim(t3.c, 'A'::text)
+                                             Filter: ((t3.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2.b, t2.c
+                                       Sort Key: t2.b, t2.c
+                                       ->  Seq Scan on public.plt2_p1 t2
+                                             Output: t2.b, t2.c
+                     ->  Sort
+                           Output: t1.a, t1.c
+                           Sort Key: t1.a, t1.c
+                           ->  Seq Scan on public.plt1_p1 t1
+                                 Output: t1.a, t1.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Merge Cond: ((t2_1.c = t1_1.c) AND (t2_1.b = t1_1.a))
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           Sort Key: t2_1.c, t2_1.b
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Merge Cond: ((t3_1.a = t2_1.b) AND ((ltrim(t3_1.c, 'A'::text)) = t2_1.c))
+                                 ->  Sort
+                                       Output: t3_1.a, t3_1.b, t3_1.c, (ltrim(t3_1.c, 'A'::text))
+                                       Sort Key: t3_1.a, (ltrim(t3_1.c, 'A'::text))
+                                       ->  Seq Scan on public.plt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c, ltrim(t3_1.c, 'A'::text)
+                                             Filter: ((t3_1.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2_1.b, t2_1.c
+                                       Sort Key: t2_1.b, t2_1.c
+                                       ->  Seq Scan on public.plt2_p2 t2_1
+                                             Output: t2_1.b, t2_1.c
+                     ->  Sort
+                           Output: t1_1.a, t1_1.c
+                           Sort Key: t1_1.c, t1_1.a
+                           ->  Seq Scan on public.plt1_p2 t1_1
+                                 Output: t1_1.a, t1_1.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Merge Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           Sort Key: t2_2.b, t2_2.c
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Merge Cond: ((t3_2.a = t2_2.b) AND ((ltrim(t3_2.c, 'A'::text)) = t2_2.c))
+                                 ->  Sort
+                                       Output: t3_2.a, t3_2.b, t3_2.c, (ltrim(t3_2.c, 'A'::text))
+                                       Sort Key: t3_2.a, (ltrim(t3_2.c, 'A'::text))
+                                       ->  Seq Scan on public.plt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c, ltrim(t3_2.c, 'A'::text)
+                                             Filter: ((t3_2.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2_2.b, t2_2.c
+                                       Sort Key: t2_2.b, t2_2.c
+                                       ->  Seq Scan on public.plt2_p3 t2_2
+                                             Output: t2_2.b, t2_2.c
+                     ->  Sort
+                           Output: t1_2.a, t1_2.c
+                           Sort Key: t1_2.a, t1_2.c
+                           ->  Seq Scan on public.plt1_p3 t1_2
+                                 Output: t1_2.a, t1_2.c
+(81 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+     |      |     |      |      100 | A0001
+     |      |     |      |      200 | A0002
+     |      |     |      |      400 | A0004
+     |      |     |      |      500 | A0005
+     |      |     |      |      700 | A0007
+     |      |     |      |      800 | A0008
+     |      |     |      |     1000 | A0010
+     |      |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+     |      |     |      |      100 | A0001
+     |      |     |      |      200 | A0002
+     |      |     |      |      400 | A0004
+     |      |     |      |      500 | A0005
+     |      |     |      |      700 | A0007
+     |      |     |      |      800 | A0008
+     |      |     |      |     1000 | A0010
+     |      |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 6) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is considered for partition-wise join.
+                                                  QUERY PLAN                                                   
+---------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, ((plt1_e_p1.a + plt1_e_p1.b)), plt1_e_p1.c
+   Sort Key: plt1_p1.a, plt2_p1.b, ((plt1_e_p1.a + plt1_e_p1.b))
+   ->  Result
+         Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, (plt1_e_p1.a + plt1_e_p1.b), plt1_e_p1.c
+         ->  Append
+               ->  Merge Full Join
+                     Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                     Merge Cond: ((plt1_p1.a = plt1_e_p1.a) AND (plt1_p1.c = (ltrim(plt1_e_p1.c, 'A'::text))))
+                     ->  Sort
+                           Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+                           Sort Key: plt1_p1.a, plt1_p1.c
+                           ->  Merge Full Join
+                                 Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+                                 Merge Cond: ((plt1_p1.a = plt2_p1.b) AND (plt1_p1.c = plt2_p1.c))
+                                 ->  Sort
+                                       Output: plt1_p1.a, plt1_p1.c
+                                       Sort Key: plt1_p1.a, plt1_p1.c
+                                       ->  Seq Scan on public.plt1_p1
+                                             Output: plt1_p1.a, plt1_p1.c
+                                             Filter: ((plt1_p1.a % 25) = 0)
+                                 ->  Sort
+                                       Output: plt2_p1.b, plt2_p1.c
+                                       Sort Key: plt2_p1.b, plt2_p1.c
+                                       ->  Seq Scan on public.plt2_p1
+                                             Output: plt2_p1.b, plt2_p1.c
+                                             Filter: ((plt2_p1.b % 25) = 0)
+                     ->  Sort
+                           Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c, (ltrim(plt1_e_p1.c, 'A'::text))
+                           Sort Key: plt1_e_p1.a, (ltrim(plt1_e_p1.c, 'A'::text))
+                           ->  Seq Scan on public.plt1_e_p1
+                                 Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c, ltrim(plt1_e_p1.c, 'A'::text)
+                                 Filter: ((plt1_e_p1.a % 25) = 0)
+               ->  Merge Full Join
+                     Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c, plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                     Merge Cond: ((plt1_p2.a = plt1_e_p2.a) AND (plt1_p2.c = (ltrim(plt1_e_p2.c, 'A'::text))))
+                     ->  Sort
+                           Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+                           Sort Key: plt1_p2.a, plt1_p2.c
+                           ->  Merge Full Join
+                                 Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+                                 Merge Cond: ((plt1_p2.a = plt2_p2.b) AND (plt1_p2.c = plt2_p2.c))
+                                 ->  Sort
+                                       Output: plt1_p2.a, plt1_p2.c
+                                       Sort Key: plt1_p2.a, plt1_p2.c
+                                       ->  Seq Scan on public.plt1_p2
+                                             Output: plt1_p2.a, plt1_p2.c
+                                             Filter: ((plt1_p2.a % 25) = 0)
+                                 ->  Sort
+                                       Output: plt2_p2.b, plt2_p2.c
+                                       Sort Key: plt2_p2.b, plt2_p2.c
+                                       ->  Seq Scan on public.plt2_p2
+                                             Output: plt2_p2.b, plt2_p2.c
+                                             Filter: ((plt2_p2.b % 25) = 0)
+                     ->  Sort
+                           Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c, (ltrim(plt1_e_p2.c, 'A'::text))
+                           Sort Key: plt1_e_p2.a, (ltrim(plt1_e_p2.c, 'A'::text))
+                           ->  Seq Scan on public.plt1_e_p2
+                                 Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c, ltrim(plt1_e_p2.c, 'A'::text)
+                                 Filter: ((plt1_e_p2.a % 25) = 0)
+               ->  Merge Full Join
+                     Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c, plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                     Merge Cond: ((plt1_p3.a = plt1_e_p3.a) AND (plt1_p3.c = (ltrim(plt1_e_p3.c, 'A'::text))))
+                     ->  Sort
+                           Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+                           Sort Key: plt1_p3.a, plt1_p3.c
+                           ->  Merge Full Join
+                                 Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+                                 Merge Cond: ((plt1_p3.a = plt2_p3.b) AND (plt1_p3.c = plt2_p3.c))
+                                 ->  Sort
+                                       Output: plt1_p3.a, plt1_p3.c
+                                       Sort Key: plt1_p3.a, plt1_p3.c
+                                       ->  Seq Scan on public.plt1_p3
+                                             Output: plt1_p3.a, plt1_p3.c
+                                             Filter: ((plt1_p3.a % 25) = 0)
+                                 ->  Sort
+                                       Output: plt2_p3.b, plt2_p3.c
+                                       Sort Key: plt2_p3.b, plt2_p3.c
+                                       ->  Seq Scan on public.plt2_p3
+                                             Output: plt2_p3.b, plt2_p3.c
+                                             Filter: ((plt2_p3.b % 25) = 0)
+                     ->  Sort
+                           Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c, (ltrim(plt1_e_p3.c, 'A'::text))
+                           Sort Key: plt1_e_p3.a, (ltrim(plt1_e_p3.c, 'A'::text))
+                           ->  Seq Scan on public.plt1_e_p3
+                                 Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c, ltrim(plt1_e_p3.c, 'A'::text)
+                                 Filter: ((plt1_e_p3.a % 25) = 0)
+(87 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 6) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 5) is considered for partition-wise join.
+                                      QUERY PLAN                                      
+--------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Merge Semi Join
+               Output: t1.a, t1.b, t1.c
+               Merge Cond: (t1.c = t1_3.c)
+               ->  Sort
+                     Output: t1.a, t1.b, t1.c
+                     Sort Key: t1.c
+                     ->  Seq Scan on public.plt1_p1 t1
+                           Output: t1.a, t1.b, t1.c
+                           Filter: ((t1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_3.c, t1_6.c
+                     ->  Merge Join
+                           Output: t1_3.c, t1_6.c
+                           Merge Cond: ((ltrim(t1_6.c, 'A'::text)) = t1_3.c)
+                           ->  Sort
+                                 Output: t1_6.c, (ltrim(t1_6.c, 'A'::text))
+                                 Sort Key: (ltrim(t1_6.c, 'A'::text))
+                                 ->  HashAggregate
+                                       Output: t1_6.c, (ltrim(t1_6.c, 'A'::text))
+                                       Group Key: ltrim(t1_6.c, 'A'::text)
+                                       ->  Seq Scan on public.plt1_e_p1 t1_6
+                                             Output: t1_6.c, ltrim(t1_6.c, 'A'::text)
+                                             Filter: ((t1_6.a % 25) = 0)
+                           ->  Sort
+                                 Output: t1_3.c
+                                 Sort Key: t1_3.c
+                                 ->  Seq Scan on public.plt2_p1 t1_3
+                                       Output: t1_3.c
+         ->  Merge Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Merge Cond: (t1_1.c = t1_4.c)
+               ->  Sort
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Sort Key: t1_1.c
+                     ->  Seq Scan on public.plt1_p2 t1_1
+                           Output: t1_1.a, t1_1.b, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_4.c, t1_7.c
+                     ->  Merge Join
+                           Output: t1_4.c, t1_7.c
+                           Merge Cond: ((ltrim(t1_7.c, 'A'::text)) = t1_4.c)
+                           ->  Sort
+                                 Output: t1_7.c, (ltrim(t1_7.c, 'A'::text))
+                                 Sort Key: (ltrim(t1_7.c, 'A'::text))
+                                 ->  HashAggregate
+                                       Output: t1_7.c, (ltrim(t1_7.c, 'A'::text))
+                                       Group Key: ltrim(t1_7.c, 'A'::text)
+                                       ->  Seq Scan on public.plt1_e_p2 t1_7
+                                             Output: t1_7.c, ltrim(t1_7.c, 'A'::text)
+                                             Filter: ((t1_7.a % 25) = 0)
+                           ->  Sort
+                                 Output: t1_4.c
+                                 Sort Key: t1_4.c
+                                 ->  Seq Scan on public.plt2_p2 t1_4
+                                       Output: t1_4.c
+         ->  Merge Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Merge Cond: (t1_2.c = t1_5.c)
+               ->  Sort
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Sort Key: t1_2.c
+                     ->  Seq Scan on public.plt1_p3 t1_2
+                           Output: t1_2.a, t1_2.b, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_5.c, t1_8.c
+                     ->  Merge Join
+                           Output: t1_5.c, t1_8.c
+                           Merge Cond: ((ltrim(t1_8.c, 'A'::text)) = t1_5.c)
+                           ->  Sort
+                                 Output: t1_8.c, (ltrim(t1_8.c, 'A'::text))
+                                 Sort Key: (ltrim(t1_8.c, 'A'::text))
+                                 ->  HashAggregate
+                                       Output: t1_8.c, (ltrim(t1_8.c, 'A'::text))
+                                       Group Key: ltrim(t1_8.c, 'A'::text)
+                                       ->  Seq Scan on public.plt1_e_p3 t1_8
+                                             Output: t1_8.c, ltrim(t1_8.c, 'A'::text)
+                                             Filter: ((t1_8.a % 25) = 0)
+                           ->  Sort
+                                 Output: t1_5.c
+                                 Sort Key: t1_5.c
+                                 ->  Seq Scan on public.plt2_p3 t1_5
+                                       Output: t1_5.c
+(88 rows)
+
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 5) is considered for partition-wise join.
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+SET enable_seqscan TO off;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Merge Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           Sort Key: t2.b, t2.c
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Merge Cond: (ltrim(t3.c, 'A'::text) = t2.c)
+                                 Join Filter: (t2.b = t3.a)
+                                 ->  Index Scan using iplt1_e_p1_c on public.plt1_e_p1 t3
+                                       Output: t3.a, t3.b, t3.c
+                                       Filter: ((t3.a % 25) = 0)
+                                 ->  Index Scan using iplt2_p1_c on public.plt2_p1 t2
+                                       Output: t2.b, t2.c
+                     ->  Sort
+                           Output: t1.a, t1.c
+                           Sort Key: t1.a, t1.c
+                           ->  Index Scan using iplt1_p1_c on public.plt1_p1 t1
+                                 Output: t1.a, t1.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Merge Cond: ((t2_1.c = t1_1.c) AND (t2_1.b = t1_1.a))
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           Sort Key: t2_1.c, t2_1.b
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Merge Cond: (ltrim(t3_1.c, 'A'::text) = t2_1.c)
+                                 Join Filter: (t2_1.b = t3_1.a)
+                                 ->  Index Scan using iplt1_e_p2_c on public.plt1_e_p2 t3_1
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       Filter: ((t3_1.a % 25) = 0)
+                                 ->  Index Scan using iplt2_p2_c on public.plt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                     ->  Sort
+                           Output: t1_1.a, t1_1.c
+                           Sort Key: t1_1.c, t1_1.a
+                           ->  Index Scan using iplt1_p2_c on public.plt1_p2 t1_1
+                                 Output: t1_1.a, t1_1.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Merge Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           Sort Key: t2_2.b, t2_2.c
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Merge Cond: (ltrim(t3_2.c, 'A'::text) = t2_2.c)
+                                 Join Filter: (t2_2.b = t3_2.a)
+                                 ->  Index Scan using iplt1_e_p3_c on public.plt1_e_p3 t3_2
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       Filter: ((t3_2.a % 25) = 0)
+                                 ->  Index Scan using iplt2_p3_c on public.plt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                     ->  Sort
+                           Output: t1_2.a, t1_2.c
+                           Sort Key: t1_2.a, t1_2.c
+                           ->  Index Scan using iplt1_p3_c on public.plt1_p3 t1_2
+                                 Output: t1_2.a, t1_2.c
+(66 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 4) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 2) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 2 4) and (b 1) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 1 2) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+     |      |     |      |      100 | A0001
+     |      |     |      |      200 | A0002
+     |      |     |      |      400 | A0004
+     |      |     |      |      500 | A0005
+     |      |     |      |      700 | A0007
+     |      |     |      |      800 | A0008
+     |      |     |      |     1000 | A0010
+     |      |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+     |      |     |      |      100 | A0001
+     |      |     |      |      200 | A0002
+     |      |     |      |      400 | A0004
+     |      |     |      |      500 | A0005
+     |      |     |      |      700 | A0007
+     |      |     |      |      800 | A0008
+     |      |     |      |     1000 | A0010
+     |      |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 6) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is considered for partition-wise join.
+                                                  QUERY PLAN                                                   
+---------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, ((plt1_e_p1.a + plt1_e_p1.b)), plt1_e_p1.c
+   Sort Key: plt1_p1.a, plt2_p1.b, ((plt1_e_p1.a + plt1_e_p1.b))
+   ->  Result
+         Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, (plt1_e_p1.a + plt1_e_p1.b), plt1_e_p1.c
+         ->  Append
+               ->  Merge Full Join
+                     Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                     Merge Cond: ((plt1_p1.a = plt1_e_p1.a) AND (plt1_p1.c = (ltrim(plt1_e_p1.c, 'A'::text))))
+                     ->  Sort
+                           Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+                           Sort Key: plt1_p1.a, plt1_p1.c
+                           ->  Merge Full Join
+                                 Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+                                 Merge Cond: ((plt1_p1.a = plt2_p1.b) AND (plt1_p1.c = plt2_p1.c))
+                                 ->  Sort
+                                       Output: plt1_p1.a, plt1_p1.c
+                                       Sort Key: plt1_p1.a, plt1_p1.c
+                                       ->  Index Scan using iplt1_p1_c on public.plt1_p1
+                                             Output: plt1_p1.a, plt1_p1.c
+                                             Filter: ((plt1_p1.a % 25) = 0)
+                                 ->  Sort
+                                       Output: plt2_p1.b, plt2_p1.c
+                                       Sort Key: plt2_p1.b, plt2_p1.c
+                                       ->  Index Scan using iplt2_p1_c on public.plt2_p1
+                                             Output: plt2_p1.b, plt2_p1.c
+                                             Filter: ((plt2_p1.b % 25) = 0)
+                     ->  Sort
+                           Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c, (ltrim(plt1_e_p1.c, 'A'::text))
+                           Sort Key: plt1_e_p1.a, (ltrim(plt1_e_p1.c, 'A'::text))
+                           ->  Index Scan using iplt1_e_p1_c on public.plt1_e_p1
+                                 Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c, ltrim(plt1_e_p1.c, 'A'::text)
+                                 Filter: ((plt1_e_p1.a % 25) = 0)
+               ->  Merge Full Join
+                     Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c, plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                     Merge Cond: ((plt1_p2.a = plt1_e_p2.a) AND (plt1_p2.c = (ltrim(plt1_e_p2.c, 'A'::text))))
+                     ->  Sort
+                           Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+                           Sort Key: plt1_p2.a, plt1_p2.c
+                           ->  Merge Full Join
+                                 Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+                                 Merge Cond: ((plt1_p2.a = plt2_p2.b) AND (plt1_p2.c = plt2_p2.c))
+                                 ->  Sort
+                                       Output: plt1_p2.a, plt1_p2.c
+                                       Sort Key: plt1_p2.a, plt1_p2.c
+                                       ->  Index Scan using iplt1_p2_c on public.plt1_p2
+                                             Output: plt1_p2.a, plt1_p2.c
+                                             Filter: ((plt1_p2.a % 25) = 0)
+                                 ->  Sort
+                                       Output: plt2_p2.b, plt2_p2.c
+                                       Sort Key: plt2_p2.b, plt2_p2.c
+                                       ->  Index Scan using iplt2_p2_c on public.plt2_p2
+                                             Output: plt2_p2.b, plt2_p2.c
+                                             Filter: ((plt2_p2.b % 25) = 0)
+                     ->  Sort
+                           Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c, (ltrim(plt1_e_p2.c, 'A'::text))
+                           Sort Key: plt1_e_p2.a, (ltrim(plt1_e_p2.c, 'A'::text))
+                           ->  Index Scan using iplt1_e_p2_c on public.plt1_e_p2
+                                 Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c, ltrim(plt1_e_p2.c, 'A'::text)
+                                 Filter: ((plt1_e_p2.a % 25) = 0)
+               ->  Merge Full Join
+                     Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c, plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                     Merge Cond: ((plt1_p3.a = plt1_e_p3.a) AND (plt1_p3.c = (ltrim(plt1_e_p3.c, 'A'::text))))
+                     ->  Sort
+                           Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+                           Sort Key: plt1_p3.a, plt1_p3.c
+                           ->  Merge Full Join
+                                 Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+                                 Merge Cond: ((plt1_p3.a = plt2_p3.b) AND (plt1_p3.c = plt2_p3.c))
+                                 ->  Sort
+                                       Output: plt1_p3.a, plt1_p3.c
+                                       Sort Key: plt1_p3.a, plt1_p3.c
+                                       ->  Index Scan using iplt1_p3_c on public.plt1_p3
+                                             Output: plt1_p3.a, plt1_p3.c
+                                             Filter: ((plt1_p3.a % 25) = 0)
+                                 ->  Sort
+                                       Output: plt2_p3.b, plt2_p3.c
+                                       Sort Key: plt2_p3.b, plt2_p3.c
+                                       ->  Index Scan using iplt2_p3_c on public.plt2_p3
+                                             Output: plt2_p3.b, plt2_p3.c
+                                             Filter: ((plt2_p3.b % 25) = 0)
+                     ->  Sort
+                           Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c, (ltrim(plt1_e_p3.c, 'A'::text))
+                           Sort Key: plt1_e_p3.a, (ltrim(plt1_e_p3.c, 'A'::text))
+                           ->  Index Scan using iplt1_e_p3_c on public.plt1_e_p3
+                                 Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c, ltrim(plt1_e_p3.c, 'A'::text)
+                                 Filter: ((plt1_e_p3.a % 25) = 0)
+(87 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+NOTICE:  join between relations (b 6) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is considered for partition-wise join.
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 5) is considered for partition-wise join.
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Merge Semi Join
+               Output: t1.a, t1.b, t1.c
+               Merge Cond: (t1.c = t1_3.c)
+               ->  Index Scan using iplt1_p1_c on public.plt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_3.c, t1_6.c
+                     ->  Merge Semi Join
+                           Output: t1_3.c, t1_6.c
+                           Merge Cond: (t1_3.c = ltrim(t1_6.c, 'A'::text))
+                           ->  Index Only Scan using iplt2_p1_c on public.plt2_p1 t1_3
+                                 Output: t1_3.c
+                           ->  Materialize
+                                 Output: t1_6.c
+                                 ->  Index Scan using iplt1_e_p1_c on public.plt1_e_p1 t1_6
+                                       Output: t1_6.c
+                                       Filter: ((t1_6.a % 25) = 0)
+         ->  Merge Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Merge Cond: (t1_1.c = t1_4.c)
+               ->  Index Scan using iplt1_p2_c on public.plt1_p2 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_4.c, t1_7.c
+                     ->  Merge Semi Join
+                           Output: t1_4.c, t1_7.c
+                           Merge Cond: (t1_4.c = ltrim(t1_7.c, 'A'::text))
+                           ->  Index Only Scan using iplt2_p2_c on public.plt2_p2 t1_4
+                                 Output: t1_4.c
+                           ->  Materialize
+                                 Output: t1_7.c
+                                 ->  Index Scan using iplt1_e_p2_c on public.plt1_e_p2 t1_7
+                                       Output: t1_7.c
+                                       Filter: ((t1_7.a % 25) = 0)
+         ->  Merge Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Merge Cond: (t1_2.c = t1_5.c)
+               ->  Index Scan using iplt1_p3_c on public.plt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_5.c, t1_8.c
+                     ->  Merge Semi Join
+                           Output: t1_5.c, t1_8.c
+                           Merge Cond: (t1_5.c = ltrim(t1_8.c, 'A'::text))
+                           ->  Index Only Scan using iplt2_p3_c on public.plt2_p3 t1_5
+                                 Output: t1_5.c
+                           ->  Materialize
+                                 Output: t1_8.c
+                                 ->  Index Scan using iplt1_e_p3_c on public.plt1_e_p3 t1_8
+                                       Output: t1_8.c
+                                       Filter: ((t1_8.a % 25) = 0)
+(58 rows)
+
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 3) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 3 5) is considered for partition-wise join.
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+RESET enable_hashjoin;
+RESET enable_nestloop;
+RESET enable_seqscan;
+--
+-- negative testcases
+--
+CREATE TABLE prt1_n (a int, b int, c varchar) PARTITION BY RANGE(c);
+CREATE TABLE prt1_n_p1 PARTITION OF prt1_n FOR VALUES START ('0000') END ('0250');
+CREATE TABLE prt1_n_p2 PARTITION OF prt1_n FOR VALUES START ('0250') END ('0500');
+INSERT INTO prt1_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 499, 2) i;
+ANALYZE prt1_n;
+ANALYZE prt1_n_p1;
+ANALYZE prt1_n_p2;
+CREATE TABLE prt2_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt2_n_p1 PARTITION OF prt2_n FOR VALUES IN ('0000', '0003', '0004', '0010', '0006', '0007');
+CREATE TABLE prt2_n_p2 PARTITION OF prt2_n FOR VALUES IN ('0001', '0005', '0002', '0009', '0008', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt2_n;
+ANALYZE prt2_n_p1;
+ANALYZE prt2_n_p2;
+CREATE TABLE prt3_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt3_n_p1 PARTITION OF prt3_n FOR VALUES IN ('0000', '0004', '0006', '0007');
+CREATE TABLE prt3_n_p2 PARTITION OF prt3_n FOR VALUES IN ('0001', '0002', '0008', '0010');
+CREATE TABLE prt3_n_p3 PARTITION OF prt3_n FOR VALUES IN ('0003', '0005', '0009', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt3_n;
+ANALYZE prt3_n_p1;
+ANALYZE prt3_n_p2;
+ANALYZE prt3_n_p3;
+CREATE TABLE prt4_n (a int, b int, c text) PARTITION BY RANGE(a);
+CREATE TABLE prt4_n_p1 PARTITION OF prt4_n FOR VALUES START (0) END (300);
+CREATE TABLE prt4_n_p2 PARTITION OF prt4_n FOR VALUES START (300) END (500);
+CREATE TABLE prt4_n_p3 PARTITION OF prt4_n FOR VALUES START (500) END (600);
+INSERT INTO prt4_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt4_n;
+ANALYZE prt4_n_p1;
+ANALYZE prt4_n_p2;
+ANALYZE prt4_n_p3;
+-- partition-wise join can not be applied if the partition ranges differ
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Join
+   Hash Cond: (t1.a = t2.a)
+   ->  Append
+         ->  Seq Scan on prt1 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p2 t1_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt4_n t2
+               ->  Seq Scan on prt4_n_p1 t2_1
+               ->  Seq Scan on prt4_n_p2 t2_2
+               ->  Seq Scan on prt4_n_p3 t2_3
+(13 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 FULL JOIN prt4_n t2 ON t1.a = t2.a;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Full Join
+   Hash Cond: (t1.a = t2.a)
+   ->  Append
+         ->  Seq Scan on prt1 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p2 t1_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt4_n t2
+               ->  Seq Scan on prt4_n_p1 t2_1
+               ->  Seq Scan on prt4_n_p2 t2_2
+               ->  Seq Scan on prt4_n_p3 t2_3
+(13 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                        
+---------------------------------------------------------
+ Nested Loop Left Join
+   ->  Append
+         ->  Seq Scan on prt1 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p2 t1_3
+   ->  Append
+         ->  Seq Scan on prt2 t2
+               Filter: (t1.a < b)
+         ->  Index Scan using iprt2_p1_b on prt2_p1 t2_1
+               Index Cond: (t1.a < b)
+         ->  Index Scan using iprt2_p2_b on prt2_p2 t2_2
+               Index Cond: (t1.a < b)
+         ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+               Index Cond: (t1.a < b)
+(15 rows)
+
+-- equi-join with join condition on partial keys does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1, prt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t1.a % 25 = 0;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Join
+   Hash Cond: (((t2.b + t2.a) / 2) = t1.a)
+   ->  Append
+         ->  Seq Scan on prt2_m t2
+         ->  Seq Scan on prt2_m_p1 t2_1
+         ->  Seq Scan on prt2_m_p2 t2_2
+         ->  Seq Scan on prt2_m_p3 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_m t1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p1 t1_1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p2 t1_2
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p3 t1_3
+                     Filter: ((a % 25) = 0)
+(17 rows)
+
+-- equi-join between out-of-order partition key columns does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.a = t2.b WHERE t1.a % 25 = 0;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Right Join
+   Hash Cond: (t2.b = t1.a)
+   ->  Append
+         ->  Seq Scan on prt2_m t2
+         ->  Seq Scan on prt2_m_p1 t2_1
+         ->  Seq Scan on prt2_m_p2 t2_2
+         ->  Seq Scan on prt2_m_p3 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_m t1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p1 t1_1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p2 t1_2
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p3 t1_3
+                     Filter: ((a % 25) = 0)
+(17 rows)
+
+-- equi-join between non-key columns does not qualify for partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.c = t2.c WHERE t1.a % 25 = 0;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Right Join
+   Hash Cond: ((t2.c)::text = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt2_m t2
+         ->  Seq Scan on prt2_m_p1 t2_1
+         ->  Seq Scan on prt2_m_p2 t2_2
+         ->  Seq Scan on prt2_m_p3 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_m t1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p1 t1_1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p2 t1_2
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p3 t1_3
+                     Filter: ((a % 25) = 0)
+(17 rows)
+
+-- partition-wise join can not be applied for a join between list and range
+-- partitioned table
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1, prt2_n t2 WHERE t1.c = t2.c;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Join
+   Hash Cond: (t2.c = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt2_n t2
+         ->  Seq Scan on prt2_n_p1 t2_1
+         ->  Seq Scan on prt2_n_p2 t2_2
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(11 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 LEFT JOIN prt2_n t2 ON (t1.c = t2.c);
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Right Join
+   Hash Cond: (t2.c = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt2_n t2
+         ->  Seq Scan on prt2_n_p1 t2_1
+         ->  Seq Scan on prt2_n_p2 t2_2
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(11 rows)
+
+-- partition-wise join can not be applied between tables with different
+-- partition lists
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 RIGHT JOIN prt1 t2 ON (t1.c = t2.c);
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Left Join
+   Hash Cond: ((t2.c)::text = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt1 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p2 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(12 rows)
+
+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);
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Full Join
+   Hash Cond: ((t2.c)::text = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt1 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p2 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(12 rows)
+
diff --git a/src/test/regress/expected/rangefuncs.out b/src/test/regress/expected/rangefuncs.out
index f06cfa4..16e7f56 100644
--- a/src/test/regress/expected/rangefuncs.out
+++ b/src/test/regress/expected/rangefuncs.out
@@ -1,25 +1,26 @@
 SELECT name, setting FROM pg_settings WHERE name LIKE 'enable%';
-         name         | setting 
-----------------------+---------
- enable_bitmapscan    | on
- enable_hashagg       | on
- enable_hashjoin      | on
- enable_indexonlyscan | on
- enable_indexscan     | on
- enable_material      | on
- enable_mergejoin     | on
- enable_nestloop      | on
- enable_seqscan       | on
- enable_sort          | on
- enable_tidscan       | on
-(11 rows)
+            name            | setting 
+----------------------------+---------
+ enable_bitmapscan          | on
+ enable_hashagg             | on
+ enable_hashjoin            | on
+ enable_indexonlyscan       | on
+ enable_indexscan           | on
+ enable_material            | on
+ enable_mergejoin           | on
+ enable_nestloop            | on
+ enable_partition_wise_join | on
+ enable_seqscan             | on
+ enable_sort                | on
+ enable_tidscan             | on
+(12 rows)
 
 CREATE TABLE foo2(fooid int, f2 int);
 INSERT INTO foo2 VALUES(1, 11);
 INSERT INTO foo2 VALUES(2, 22);
 INSERT INTO foo2 VALUES(1, 111);
 CREATE FUNCTION foot(int) returns setof foo2 as 'SELECT * FROM foo2 WHERE fooid = $1 ORDER BY f2;' LANGUAGE SQL;
 -- function with ORDINALITY
 select * from foot(1) with ordinality as z(a,b,ord);
  a |  b  | ord 
 ---+-----+-----
diff --git a/src/test/regress/parallel_schedule b/src/test/regress/parallel_schedule
index 8641769..5ad149d 100644
--- a/src/test/regress/parallel_schedule
+++ b/src/test/regress/parallel_schedule
@@ -93,21 +93,21 @@ test: alter_generic alter_operator misc psql async dbsize misc_functions tsrf
 
 # rules cannot run concurrently with any test that creates a view
 test: rules psql_crosstab amutils
 
 # run by itself so it can run parallel workers
 test: select_parallel
 
 # ----------
 # Another group of parallel tests
 # ----------
-test: select_views portals_p2 foreign_key cluster dependency guc bitmapops combocid tsearch tsdicts foreign_data window xmlmap functional_deps advisory_lock json jsonb json_encoding indirect_toast equivclass
+test: select_views portals_p2 foreign_key cluster dependency guc bitmapops combocid tsearch tsdicts foreign_data window xmlmap functional_deps advisory_lock json jsonb json_encoding indirect_toast equivclass partition_join
 # ----------
 # Another group of parallel tests
 # NB: temp.sql does a reconnect which transiently uses 2 connections,
 # so keep this parallel group to at most 19 tests
 # ----------
 test: plancache limit plpgsql copy2 temp domain rangefuncs prepare without_oid conversion truncate alter_table sequence polymorphism rowtypes returning largeobject with xml
 
 # event triggers cannot run concurrently with any test that runs DDL
 test: event_trigger
 
diff --git a/src/test/regress/serial_schedule b/src/test/regress/serial_schedule
index 835cf35..b4773b8 100644
--- a/src/test/regress/serial_schedule
+++ b/src/test/regress/serial_schedule
@@ -162,10 +162,11 @@ test: truncate
 test: alter_table
 test: sequence
 test: polymorphism
 test: rowtypes
 test: returning
 test: largeobject
 test: with
 test: xml
 test: event_trigger
 test: stats
+test: partition_join
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
new file mode 100644
index 0000000..45e8a64
--- /dev/null
+++ b/src/test/regress/sql/partition_join.sql
@@ -0,0 +1,788 @@
+--
+-- PARTITION_JOIN
+-- Test partition-wise join between partitioned tables
+--
+
+--
+-- partitioned by a single column
+--
+CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES START (0) END (250);
+CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES START (500) END (600);
+CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES START (250) END (500);
+INSERT INTO prt1 SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1 AS SELECT * FROM prt1;
+
+CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES START (0) END (250);
+CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES START (250) END (500);
+CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES START (500) END (600);
+INSERT INTO prt2 SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+CREATE TABLE uprt2 AS SELECT * FROM prt2;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, 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.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+
+-- Join with pruned partitions from joining relations
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a < 450 AND t2.b > 250 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+
+-- joins where one of the relations is proven empty
+EXPLAIN (VERBOSE, 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;
+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;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 LEFT JOIN prt2 t2 ON t1.a = t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 LEFT JOIN prt2 t2 ON t1.a = t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a = 1 AND a = 2) t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, 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 % 25 = 0 ORDER BY t1.a, t2.b;
+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 % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a = 1 AND a = 2) t1 FULL JOIN uprt2 t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+-- lateral reference
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+--
+-- partitioned by expression
+--
+CREATE TABLE prt1_e (a int, b int, c varchar) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES START (0) END (250);
+CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES START (250) END (500);
+CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES START (500) END (600);
+INSERT INTO prt1_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_e AS SELECT * FROM prt1_e;
+
+CREATE TABLE prt2_e (a int, b int, c varchar) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES START (0) END (250);
+CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES START (250) END (500);
+CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES START (500) END (600);
+INSERT INTO prt2_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_e;
+ANALYZE prt2_e_p1;
+ANALYZE prt2_e_p2;
+ANALYZE prt2_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_e AS SELECT * FROM prt2_e;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1, uprt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1 LEFT JOIN uprt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 RIGHT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 RIGHT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1 RIGHT JOIN uprt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_e WHERE prt2_e.b % 25 = 0) t2 ON ((t1.a + t1.b)/2 = (t2.b + t2.a)/2) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_e WHERE prt2_e.b % 25 = 0) t2 ON ((t1.a + t1.b)/2 = (t2.b + t2.a)/2) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_e t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_e t2 WHERE t2.b % 25 = 0) t2 ON ((t1.a + t1.b)/2 = (t2.b + t2.a)/2) ORDER BY t1.a, t2.b;
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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.a % 25 = 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM uprt1 t1, uprt2 t2, uprt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM uprt1_e WHERE uprt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, 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.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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;
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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;
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM uprt1_e WHERE uprt1_e.a % 25 = 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;
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1, uprt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+-- test merge joins with and without using indexes
+SET enable_hashjoin TO off;
+SET enable_nestloop TO off;
+
+CREATE INDEX iprt1_a on prt1(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 iprt2_b on prt2(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 iprt1_e_ab2 on prt1_e(((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));
+
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+SET enable_seqscan TO off;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+-- lateral references and parameterized paths
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+RESET enable_hashjoin;
+RESET enable_nestloop;
+RESET enable_seqscan;
+
+--
+-- partitioned by multiple columns
+--
+CREATE TABLE prt1_m (a int, b int, c varchar) PARTITION BY RANGE(a, ((a + b)/2));
+CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES START (0, 0) END (250, 250);
+CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES START (250, 250) END (500, 500);
+CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES START (500, 500) END (600, 600);
+INSERT INTO prt1_m SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_m;
+ANALYZE prt1_m_p1;
+ANALYZE prt1_m_p2;
+ANALYZE prt1_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_m AS SELECT * FROM prt1_m;
+
+CREATE TABLE prt2_m (a int, b int, c varchar) PARTITION BY RANGE(((b + a)/2), b);
+CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES START (0, 0) END (250, 250);
+CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES START (250, 250) END (500, 500);
+CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES START (500, 500) END (600, 600);
+INSERT INTO prt2_m SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_m;
+ANALYZE prt2_m_p1;
+ANALYZE prt2_m_p2;
+ANALYZE prt2_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_m AS SELECT * FROM prt2_m;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1, prt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1, prt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_m t1, uprt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_m t1 LEFT JOIN uprt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_m t1 RIGHT JOIN uprt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_m t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_m t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+
+--
+-- multi-leveled partitions
+--
+CREATE TABLE prt1_l (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_l_p1 PARTITION OF prt1_l FOR VALUES START (0) END (250) PARTITION BY RANGE (b);
+CREATE TABLE prt1_l_p1_p1 PARTITION OF prt1_l_p1 FOR VALUES START (0) END (100);
+CREATE TABLE prt1_l_p1_p2 PARTITION OF prt1_l_p1 FOR VALUES START (100) END (250);
+CREATE TABLE prt1_l_p2 PARTITION OF prt1_l FOR VALUES START (250) END (500) PARTITION BY RANGE (c);
+CREATE TABLE prt1_l_p2_p1 PARTITION OF prt1_l_p2 FOR VALUES START ('0250') END ('0400');
+CREATE TABLE prt1_l_p2_p2 PARTITION OF prt1_l_p2 FOR VALUES START ('0400') END ('0500');
+CREATE TABLE prt1_l_p3 PARTITION OF prt1_l FOR VALUES START (500) END (600) PARTITION BY RANGE ((b + a));
+CREATE TABLE prt1_l_p3_p1 PARTITION OF prt1_l_p3 FOR VALUES START (1000) END (1100);
+CREATE TABLE prt1_l_p3_p2 PARTITION OF prt1_l_p3 FOR VALUES START (1100) END (1200);
+INSERT INTO prt1_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_l;
+ANALYZE prt1_l_p1;
+ANALYZE prt1_l_p1_p1;
+ANALYZE prt1_l_p1_p2;
+ANALYZE prt1_l_p2;
+ANALYZE prt1_l_p2_p1;
+ANALYZE prt1_l_p2_p2;
+ANALYZE prt1_l_p3;
+ANALYZE prt1_l_p3_p1;
+ANALYZE prt1_l_p3_p2;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_l AS SELECT * FROM prt1_l;
+
+CREATE TABLE prt2_l (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_l_p1 PARTITION OF prt2_l FOR VALUES START (0) END (250) PARTITION BY RANGE (a);
+CREATE TABLE prt2_l_p1_p1 PARTITION OF prt2_l_p1 FOR VALUES START (0) END (100);
+CREATE TABLE prt2_l_p1_p2 PARTITION OF prt2_l_p1 FOR VALUES START (100) END (250);
+CREATE TABLE prt2_l_p2 PARTITION OF prt2_l FOR VALUES START (250) END (500) PARTITION BY RANGE (c);
+CREATE TABLE prt2_l_p2_p1 PARTITION OF prt2_l_p2 FOR VALUES START ('0250') END ('0400');
+CREATE TABLE prt2_l_p2_p2 PARTITION OF prt2_l_p2 FOR VALUES START ('0400') END ('0500');
+CREATE TABLE prt2_l_p3 PARTITION OF prt2_l FOR VALUES START (500) END (600) PARTITION BY RANGE ((a + b));
+CREATE TABLE prt2_l_p3_p1 PARTITION OF prt2_l_p3 FOR VALUES START (1000) END (1100);
+CREATE TABLE prt2_l_p3_p2 PARTITION OF prt2_l_p3 FOR VALUES START (1100) END (1200);
+INSERT INTO prt2_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_l;
+ANALYZE prt2_l_p1;
+ANALYZE prt2_l_p1_p1;
+ANALYZE prt2_l_p1_p2;
+ANALYZE prt2_l_p2;
+ANALYZE prt2_l_p2_p1;
+ANALYZE prt2_l_p2_p2;
+ANALYZE prt2_l_p3;
+ANALYZE prt2_l_p3_p1;
+ANALYZE prt2_l_p3_p2;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_l AS SELECT * FROM prt2_l;
+
+EXPLAIN (VERBOSE, 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 = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 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 = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1, uprt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 LEFT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 RIGHT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_l t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_l t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+
+-- lateral reference
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
+			  ON t1.a = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
+			  ON t1.a = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1_l t2 JOIN uprt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
+			  ON t1.a = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
+			  ON t1.b = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
+			  ON t1.b = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1_l t2 JOIN uprt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
+			  ON t1.b = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+--
+-- 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;
+ANALYZE plt1_p1;
+ANALYZE plt1_p2;
+ANALYZE plt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1 AS SELECT * FROM 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;
+ANALYZE plt2_p1;
+ANALYZE plt2_p2;
+ANALYZE plt2_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt2 AS SELECT * FROM plt2;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.a = t2.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.a = t2.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1 t1, uplt2 t2 WHERE t1.c = t2.c AND t1.a = t2.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.a AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.a AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.a AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 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 plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 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 uplt1 t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) ORDER BY t1.a, t2.b;
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM (SELECT 50 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM (SELECT 50 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM (SELECT 50 phv, * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t1.phv), avg(t2.b), t2.c, avg(t2.phv) FROM (SELECT 25 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t1.phv), avg(t2.b), t2.c, avg(t2.phv) FROM (SELECT 25 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t1.phv), avg(t2.b), t2.c, avg(t2.phv) FROM (SELECT 25 phv, * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
+-- Join with pruned partitions from joining relations
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.c NOT IN ('0001', '0005', '0002', '0009') AND t2.c NOT IN ('0000', '0003', '0004', '0010') GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.c NOT IN ('0001', '0005', '0002', '0009') AND t2.c NOT IN ('0000', '0003', '0004', '0010') GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM uplt1 t1, uplt2 t2 WHERE t1.c = t2.c AND t1.c NOT IN ('0001', '0005', '0002', '0009') AND t2.c NOT IN ('0000', '0003', '0004', '0010') GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 LEFT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 LEFT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM uplt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 LEFT JOIN (SELECT * FROM uplt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 RIGHT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 RIGHT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM uplt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 RIGHT JOIN (SELECT * FROM uplt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 FULL JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 FULL JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM uplt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 FULL JOIN (SELECT * FROM uplt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 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;
+ANALYZE plt1_e;
+ANALYZE plt1_e_p1;
+ANALYZE plt1_e_p2;
+ANALYZE plt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1_e AS SELECT * FROM plt1_e;
+
+CREATE TABLE plt2_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
+CREATE TABLE plt2_e_p1 PARTITION OF plt2_e FOR VALUES IN ('0000', '0003', '0004', '0010');
+CREATE TABLE plt2_e_p2 PARTITION OF plt2_e FOR VALUES IN ('0001', '0005', '0002', '0009');
+CREATE TABLE plt2_e_p3 PARTITION OF plt2_e FOR VALUES IN ('0006', '0007', '0008', '0011');
+INSERT INTO plt2_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE plt2_e;
+ANALYZE plt2_e_p1;
+ANALYZE plt2_e_p2;
+ANALYZE plt2_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt2_e AS SELECT * FROM plt2_e;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1, plt2_e t2 WHERE t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1, plt2_e t2 WHERE t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_e t1, uplt2_e t2 WHERE t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 LEFT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 LEFT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_e t1 LEFT JOIN uplt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 RIGHT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 RIGHT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_e t1 RIGHT JOIN uplt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2_e WHERE plt2_e.b % 25 = 0) t2 ON (t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A')) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2_e WHERE plt2_e.b % 25 = 0) t2 ON (t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A')) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uplt1_e t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2_e t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A')) ORDER BY t1.a, t2.b;
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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;
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM uplt1 t1, uplt2 t2, uplt1_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;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1, uplt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 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 ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+-- test merge join with and without index scan
+CREATE INDEX iplt1_c on plt1(c);
+CREATE INDEX iplt1_p1_c on plt1_p1(c);
+CREATE INDEX iplt1_p2_c on plt1_p2(c);
+CREATE INDEX iplt1_p3_c on plt1_p3(c);
+CREATE INDEX iplt2_c on plt2(c);
+CREATE INDEX iplt2_p1_c on plt2_p1(c);
+CREATE INDEX iplt2_p2_c on plt2_p2(c);
+CREATE INDEX iplt2_p3_c on plt2_p3(c);
+CREATE INDEX iplt1_e_c on plt1_e(ltrim(c, 'A'));
+CREATE INDEX iplt1_e_p1_c on plt1_e_p1(ltrim(c, 'A'));
+CREATE INDEX iplt1_e_p2_c on plt1_e_p2(ltrim(c, 'A'));
+CREATE INDEX iplt1_e_p3_c on plt1_e_p3(ltrim(c, 'A'));
+
+ANALYZE plt1;
+ANALYZE plt1_p1;
+ANALYZE plt1_p2;
+ANALYZE plt1_p3;
+ANALYZE plt2;
+ANALYZE plt2_p1;
+ANALYZE plt2_p2;
+ANALYZE plt2_p3;
+ANALYZE plt1_e;
+ANALYZE plt1_e_p1;
+ANALYZE plt1_e_p2;
+ANALYZE plt1_e_p3;
+
+SET enable_hashjoin TO off;
+SET enable_nestloop TO off;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 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 ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+SET enable_seqscan TO off;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 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 ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+RESET enable_hashjoin;
+RESET enable_nestloop;
+RESET enable_seqscan;
+
+--
+-- negative testcases
+--
+
+CREATE TABLE prt1_n (a int, b int, c varchar) PARTITION BY RANGE(c);
+CREATE TABLE prt1_n_p1 PARTITION OF prt1_n FOR VALUES START ('0000') END ('0250');
+CREATE TABLE prt1_n_p2 PARTITION OF prt1_n FOR VALUES START ('0250') END ('0500');
+INSERT INTO prt1_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 499, 2) i;
+ANALYZE prt1_n;
+ANALYZE prt1_n_p1;
+ANALYZE prt1_n_p2;
+
+CREATE TABLE prt2_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt2_n_p1 PARTITION OF prt2_n FOR VALUES IN ('0000', '0003', '0004', '0010', '0006', '0007');
+CREATE TABLE prt2_n_p2 PARTITION OF prt2_n FOR VALUES IN ('0001', '0005', '0002', '0009', '0008', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt2_n;
+ANALYZE prt2_n_p1;
+ANALYZE prt2_n_p2;
+
+CREATE TABLE prt3_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt3_n_p1 PARTITION OF prt3_n FOR VALUES IN ('0000', '0004', '0006', '0007');
+CREATE TABLE prt3_n_p2 PARTITION OF prt3_n FOR VALUES IN ('0001', '0002', '0008', '0010');
+CREATE TABLE prt3_n_p3 PARTITION OF prt3_n FOR VALUES IN ('0003', '0005', '0009', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt3_n;
+ANALYZE prt3_n_p1;
+ANALYZE prt3_n_p2;
+ANALYZE prt3_n_p3;
+
+CREATE TABLE prt4_n (a int, b int, c text) PARTITION BY RANGE(a);
+CREATE TABLE prt4_n_p1 PARTITION OF prt4_n FOR VALUES START (0) END (300);
+CREATE TABLE prt4_n_p2 PARTITION OF prt4_n FOR VALUES START (300) END (500);
+CREATE TABLE prt4_n_p3 PARTITION OF prt4_n FOR VALUES START (500) END (600);
+INSERT INTO prt4_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt4_n;
+ANALYZE prt4_n_p1;
+ANALYZE prt4_n_p2;
+ANALYZE prt4_n_p3;
+
+-- partition-wise join can not be applied if the partition ranges differ
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 FULL JOIN prt4_n t2 ON t1.a = t2.a;
+-- 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);
+-- equi-join with join condition on partial keys does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1, prt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t1.a % 25 = 0;
+-- equi-join between out-of-order partition key columns does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.a = t2.b WHERE t1.a % 25 = 0;
+-- equi-join between non-key columns does not qualify for partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.c = t2.c WHERE t1.a % 25 = 0;
+-- partition-wise join can not be applied for a join between list and range
+-- partitioned table
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1, prt2_n t2 WHERE t1.c = t2.c;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 LEFT JOIN prt2_n t2 ON (t1.c = t2.c);
+-- partition-wise join can not be applied between tables with different
+-- partition lists
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 RIGHT JOIN prt1 t2 ON (t1.c = t2.c);
+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);
#15Rajkumar Raghuwanshi
rajkumar.raghuwanshi@enterprisedb.com
In reply to: Ashutosh Bapat (#14)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Sep 22, 2016 at 4:11 PM, Ashutosh Bapat <
ashutosh.bapat@enterprisedb.com> wrote:

The patch is based on sources upto commit

commit 2a7f4f76434d82eb0d1b5f4f7051043e1dd3ee1a
Author: Heikki Linnakangas <heikki.linnakangas@iki.fi>
Date: Wed Sep 21 13:24:13 2016 +0300

and Amit Langote's set of patches posted on 15th Sept. 2016 [1]

I have applied your patch on top of Amit patches posted on 15th Sept. 2016,
and tried to create some test cases on list and multi-level partition based
on test cases written for range partition.

I got some server crash and errors which I have mentioned as comment in
expected output file, which need to be updated once these issues will get
fix. also for these issue expected output is generated by creating same
query for non-partition table with same data.

Attached patch created on top to Ashutosh's patch posted on 22 Sept 2016.

Thanks & Regards,
Rajkumar Raghuwanshi
QMG, EnterpriseDB Corporation

Attachments:

partition_join_extra_testcases.patchinvalid/octet-stream; name=partition_join_extra_testcases.patchDownload
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index 0eecade..16ed8e6 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -3993,6 +3993,416 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_l t1 WHERE t1.a % 25 = 0
      |      | 525 | 0525
 (16 rows)
 
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM (SELECT 50 phv, * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b and t1.b = t2.a and t1.c = t2.c and t2.a + t2.b = t1.b + t1.a) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY 1,2,3,4,5,6,7,8;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 7) and (b 20) is considered for partition-wise join.
+NOTICE:  join between relations (b 11) and (b 24) is considered for partition-wise join.
+NOTICE:  join between relations (b 15) and (b 28) is considered for partition-wise join.
+                                                                                                                 QUERY PLAN                                                                                                                 
+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: (50), prt1_l_p1_p1.a, prt1_l_p1_p1.b, prt1_l_p1_p1.c, (75), prt2_l_p1_p1.a, prt2_l_p1_p1.b, prt2_l_p1_p1.c
+   Sort Key: (50), prt1_l_p1_p1.a, prt1_l_p1_p1.b, prt1_l_p1_p1.c, (75), prt2_l_p1_p1.a, prt2_l_p1_p1.b, prt2_l_p1_p1.c
+   ->  Result
+         Output: (50), prt1_l_p1_p1.a, prt1_l_p1_p1.b, prt1_l_p1_p1.c, (75), prt2_l_p1_p1.a, prt2_l_p1_p1.b, prt2_l_p1_p1.c
+         ->  Append
+               ->  Hash Full Join
+                     Output: prt1_l_p1_p1.a, prt1_l_p1_p1.b, prt1_l_p1_p1.c, prt2_l_p1_p1.a, prt2_l_p1_p1.b, prt2_l_p1_p1.c, (50), (75)
+                     Hash Cond: ((prt1_l_p1_p1.a = prt2_l_p1_p1.b) AND (prt1_l_p1_p1.b = prt2_l_p1_p1.a) AND ((prt1_l_p1_p1.c)::text = (prt2_l_p1_p1.c)::text) AND ((prt1_l_p1_p1.b + prt1_l_p1_p1.a) = (prt2_l_p1_p1.a + prt2_l_p1_p1.b)))
+                     Filter: (((50) = prt1_l_p1_p1.b) OR ((75) = prt2_l_p1_p1.b))
+                     ->  Seq Scan on public.prt1_l_p1_p1
+                           Output: prt1_l_p1_p1.a, prt1_l_p1_p1.b, prt1_l_p1_p1.c, 50
+                           Filter: ((prt1_l_p1_p1.a % 25) = 0)
+                     ->  Hash
+                           Output: prt2_l_p1_p1.a, prt2_l_p1_p1.b, prt2_l_p1_p1.c, (75)
+                           ->  Seq Scan on public.prt2_l_p1_p1
+                                 Output: prt2_l_p1_p1.a, prt2_l_p1_p1.b, prt2_l_p1_p1.c, 75
+                                 Filter: ((prt2_l_p1_p1.b % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_l_p1_p2.a, prt1_l_p1_p2.b, prt1_l_p1_p2.c, prt2_l_p1_p2.a, prt2_l_p1_p2.b, prt2_l_p1_p2.c, (50), (75)
+                     Hash Cond: ((prt1_l_p1_p2.a = prt2_l_p1_p2.b) AND (prt1_l_p1_p2.b = prt2_l_p1_p2.a) AND ((prt1_l_p1_p2.c)::text = (prt2_l_p1_p2.c)::text) AND ((prt1_l_p1_p2.b + prt1_l_p1_p2.a) = (prt2_l_p1_p2.a + prt2_l_p1_p2.b)))
+                     Filter: (((50) = prt1_l_p1_p2.b) OR ((75) = prt2_l_p1_p2.b))
+                     ->  Seq Scan on public.prt1_l_p1_p2
+                           Output: prt1_l_p1_p2.a, prt1_l_p1_p2.b, prt1_l_p1_p2.c, 50
+                           Filter: ((prt1_l_p1_p2.a % 25) = 0)
+                     ->  Hash
+                           Output: prt2_l_p1_p2.a, prt2_l_p1_p2.b, prt2_l_p1_p2.c, (75)
+                           ->  Seq Scan on public.prt2_l_p1_p2
+                                 Output: prt2_l_p1_p2.a, prt2_l_p1_p2.b, prt2_l_p1_p2.c, 75
+                                 Filter: ((prt2_l_p1_p2.b % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_l_p2_p1.a, prt1_l_p2_p1.b, prt1_l_p2_p1.c, prt2_l_p2_p1.a, prt2_l_p2_p1.b, prt2_l_p2_p1.c, (50), (75)
+                     Hash Cond: ((prt1_l_p2_p1.a = prt2_l_p2_p1.b) AND (prt1_l_p2_p1.b = prt2_l_p2_p1.a) AND ((prt1_l_p2_p1.c)::text = (prt2_l_p2_p1.c)::text) AND ((prt1_l_p2_p1.b + prt1_l_p2_p1.a) = (prt2_l_p2_p1.a + prt2_l_p2_p1.b)))
+                     Filter: (((50) = prt1_l_p2_p1.b) OR ((75) = prt2_l_p2_p1.b))
+                     ->  Seq Scan on public.prt1_l_p2_p1
+                           Output: prt1_l_p2_p1.a, prt1_l_p2_p1.b, prt1_l_p2_p1.c, 50
+                           Filter: ((prt1_l_p2_p1.a % 25) = 0)
+                     ->  Hash
+                           Output: prt2_l_p2_p1.a, prt2_l_p2_p1.b, prt2_l_p2_p1.c, (75)
+                           ->  Seq Scan on public.prt2_l_p2_p1
+                                 Output: prt2_l_p2_p1.a, prt2_l_p2_p1.b, prt2_l_p2_p1.c, 75
+                                 Filter: ((prt2_l_p2_p1.b % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_l_p2_p2.a, prt1_l_p2_p2.b, prt1_l_p2_p2.c, prt2_l_p2_p2.a, prt2_l_p2_p2.b, prt2_l_p2_p2.c, (50), (75)
+                     Hash Cond: ((prt1_l_p2_p2.a = prt2_l_p2_p2.b) AND (prt1_l_p2_p2.b = prt2_l_p2_p2.a) AND ((prt1_l_p2_p2.c)::text = (prt2_l_p2_p2.c)::text) AND ((prt1_l_p2_p2.b + prt1_l_p2_p2.a) = (prt2_l_p2_p2.a + prt2_l_p2_p2.b)))
+                     Filter: (((50) = prt1_l_p2_p2.b) OR ((75) = prt2_l_p2_p2.b))
+                     ->  Seq Scan on public.prt1_l_p2_p2
+                           Output: prt1_l_p2_p2.a, prt1_l_p2_p2.b, prt1_l_p2_p2.c, 50
+                           Filter: ((prt1_l_p2_p2.a % 25) = 0)
+                     ->  Hash
+                           Output: prt2_l_p2_p2.a, prt2_l_p2_p2.b, prt2_l_p2_p2.c, (75)
+                           ->  Seq Scan on public.prt2_l_p2_p2
+                                 Output: prt2_l_p2_p2.a, prt2_l_p2_p2.b, prt2_l_p2_p2.c, 75
+                                 Filter: ((prt2_l_p2_p2.b % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_l_p3_p1.a, prt1_l_p3_p1.b, prt1_l_p3_p1.c, prt2_l_p3_p1.a, prt2_l_p3_p1.b, prt2_l_p3_p1.c, (50), (75)
+                     Hash Cond: ((prt1_l_p3_p1.a = prt2_l_p3_p1.b) AND (prt1_l_p3_p1.b = prt2_l_p3_p1.a) AND ((prt1_l_p3_p1.c)::text = (prt2_l_p3_p1.c)::text) AND ((prt1_l_p3_p1.b + prt1_l_p3_p1.a) = (prt2_l_p3_p1.a + prt2_l_p3_p1.b)))
+                     Filter: (((50) = prt1_l_p3_p1.b) OR ((75) = prt2_l_p3_p1.b))
+                     ->  Seq Scan on public.prt1_l_p3_p1
+                           Output: prt1_l_p3_p1.a, prt1_l_p3_p1.b, prt1_l_p3_p1.c, 50
+                           Filter: ((prt1_l_p3_p1.a % 25) = 0)
+                     ->  Hash
+                           Output: prt2_l_p3_p1.a, prt2_l_p3_p1.b, prt2_l_p3_p1.c, (75)
+                           ->  Seq Scan on public.prt2_l_p3_p1
+                                 Output: prt2_l_p3_p1.a, prt2_l_p3_p1.b, prt2_l_p3_p1.c, 75
+                                 Filter: ((prt2_l_p3_p1.b % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_l_p3_p2.a, prt1_l_p3_p2.b, prt1_l_p3_p2.c, prt2_l_p3_p2.a, prt2_l_p3_p2.b, prt2_l_p3_p2.c, (50), (75)
+                     Hash Cond: ((prt1_l_p3_p2.a = prt2_l_p3_p2.b) AND (prt1_l_p3_p2.b = prt2_l_p3_p2.a) AND ((prt1_l_p3_p2.c)::text = (prt2_l_p3_p2.c)::text) AND ((prt1_l_p3_p2.b + prt1_l_p3_p2.a) = (prt2_l_p3_p2.a + prt2_l_p3_p2.b)))
+                     Filter: (((50) = prt1_l_p3_p2.b) OR ((75) = prt2_l_p3_p2.b))
+                     ->  Seq Scan on public.prt1_l_p3_p2
+                           Output: prt1_l_p3_p2.a, prt1_l_p3_p2.b, prt1_l_p3_p2.c, 50
+                           Filter: ((prt1_l_p3_p2.a % 25) = 0)
+                     ->  Hash
+                           Output: prt2_l_p3_p2.a, prt2_l_p3_p2.b, prt2_l_p3_p2.c, (75)
+                           ->  Seq Scan on public.prt2_l_p3_p2
+                                 Output: prt2_l_p3_p2.a, prt2_l_p3_p2.b, prt2_l_p3_p2.c, 75
+                                 Filter: ((prt2_l_p3_p2.b % 25) = 0)
+(78 rows)
+
+SELECT * FROM (SELECT 50 phv, * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b and t1.b = t2.a and t1.c = t2.c and t2.a + t2.b = t1.b + t1.a) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY 1,2,3,4,5,6,7,8;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 7) and (b 20) is considered for partition-wise join.
+NOTICE:  join between relations (b 11) and (b 24) is considered for partition-wise join.
+NOTICE:  join between relations (b 15) and (b 28) is considered for partition-wise join.
+ phv | a  | b  |  c   | phv | a  | b  |  c   
+-----+----+----+------+-----+----+----+------
+  50 | 50 | 50 | 0050 |     |    |    | 
+     |    |    |      |  75 | 75 | 75 | 0075
+(2 rows)
+
+SELECT * FROM (SELECT 50 phv, * FROM uprt1_l WHERE uprt1_l.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2_l WHERE uprt2_l.b % 25 = 0) t2 ON (t1.a = t2.b and t1.b = t2.a and t1.c = t2.c and t2.a + t2.b = t1.b + t1.a) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY 1,2,3,4,5,6,7,8;
+ phv | a  | b  |  c   | phv | a  | b  |  c   
+-----+----+----+------+-----+----+----+------
+  50 | 50 | 50 | 0050 |     |    |    | 
+     |    |    |      |  75 | 75 | 75 | 0075
+(2 rows)
+
+-- Join with pruned partitions from joining relations
+--Getting server crash, need to be fix to get query output
+EXPLAIN (VERBOSE, 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 = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a < 450 AND t2.b > 250 AND t1.a % 25 = 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 = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a < 450 AND t2.b > 250 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+(1 row)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1, uprt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a < 450 AND t2.b > 250 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+(1 row)
+
+--Getting server crash, need to be fix to get query output
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2_l WHERE b > 250) t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2_l WHERE b > 250) t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_l WHERE a < 450) t1 LEFT JOIN (SELECT * FROM uprt2_l WHERE b > 250) t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 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)
+
+--Getting server crash, need to be fix to get query output
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2_l WHERE b > 250) t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2_l WHERE b > 250) t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+     |      | 375 | 0375
+     |      | 450 | 0450
+     |      | 525 | 0525
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_l WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM uprt2_l WHERE b > 250) t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+     |      | 375 | 0375
+     |      | 450 | 0450
+     |      | 525 | 0525
+(4 rows)
+
+--Getting server crash, need to be fix to get query output
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_l WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_l WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b 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)
+
+-- joins where one of the relations is proven empty
+EXPLAIN (VERBOSE, 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 = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a = 1 AND t1.a = 2;
+            QUERY PLAN            
+----------------------------------
+ Result
+   Output: t1.a, t1.c, t2.b, t2.c
+   One-Time Filter: false
+(3 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1, prt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a = 1 AND t1.a = 2;
+ a | c | b | c 
+---+---+---+---
+(0 rows)
+
+--Getting server crash, need to be fix to get query output
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE a = 1 AND a = 2) t1 LEFT JOIN prt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE a = 1 AND a = 2) t1 LEFT JOIN prt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b;
+ a | c | b | c 
+---+---+---+---
+(0 rows)
+
+--Getting server crash, need to be fix to get query output
+EXPLAIN (VERBOSE, 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 AND t1.b + t1.a = t2.a + t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+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 AND t1.b + t1.a = t2.a + t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+ a | c |  b  |  c   
+---+---+-----+------
+   |   |   0 | 0000
+   |   |  75 | 0075
+   |   | 150 | 0150
+   |   | 225 | 0225
+   |   | 300 | 0300
+   |   | 375 | 0375
+   |   | 450 | 0450
+   |   | 525 | 0525
+(8 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_l WHERE a = 1 AND a = 2) t1 RIGHT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+ a | c |  b  |  c   
+---+---+-----+------
+   |   |   0 | 0000
+   |   |  75 | 0075
+   |   | 150 | 0150
+   |   | 225 | 0225
+   |   | 300 | 0300
+   |   | 375 | 0375
+   |   | 450 | 0450
+   |   | 525 | 0525
+(8 rows)
+
+--Getting server crash, need to be fix to get query output
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE a = 1 AND a = 2) t1 FULL JOIN prt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE a = 1 AND a = 2) t1 FULL JOIN prt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+ a | c |  b  |  c   
+---+---+-----+------
+   |   |   0 | 0000
+   |   |  75 | 0075
+   |   | 150 | 0150
+   |   | 225 | 0225
+   |   | 300 | 0300
+   |   | 375 | 0375
+   |   | 450 | 0450
+   |   | 525 | 0525
+(8 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_l WHERE a = 1 AND a = 2) t1 FULL JOIN uprt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+ a | c |  b  |  c   
+---+---+-----+------
+   |   |   0 | 0000
+   |   |  75 | 0075
+   |   | 150 | 0150
+   |   | 225 | 0225
+   |   | 300 | 0300
+   |   | 375 | 0375
+   |   | 450 | 0450
+   |   | 525 | 0525
+(8 rows)
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1_l t1 WHERE t1.a IN (SELECT t1.b FROM prt2_l t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 1) and (b 3) is considered for partition-wise join.
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Output: t1.a, t1.b, t1.c
+         Merge Cond: (t1.a = t1_9.b)
+         ->  Sort
+               Output: t1.a, t1.b, t1.c
+               Sort Key: t1.a
+               ->  Append
+                     ->  Seq Scan on public.prt1_l_p1 t1
+                           Output: t1.a, t1.b, t1.c
+                           Filter: ((t1.a % 25) = 0)
+                     ->  Seq Scan on public.prt1_l_p1_p1 t1_1
+                           Output: t1_1.a, t1_1.b, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+                     ->  Seq Scan on public.prt1_l_p1_p2 t1_2
+                           Output: t1_2.a, t1_2.b, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Sort
+               Output: t1_9.b
+               Sort Key: t1_9.b
+               ->  Append
+                     ->  Seq Scan on public.prt2_l_p1 t1_9
+                           Output: t1_9.b
+                           Filter: ((t1_9.b % 25) = 0)
+                     ->  Seq Scan on public.prt2_l_p1_p1 t1_10
+                           Output: t1_10.b
+                           Filter: ((t1_10.b % 25) = 0)
+                     ->  Seq Scan on public.prt2_l_p1_p2 t1_11
+                           Output: t1_11.b
+                           Filter: ((t1_11.b % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_3.a, t1_3.b, t1_3.c
+         Merge Cond: (t1_3.a = t1_12.b)
+         ->  Sort
+               Output: t1_3.a, t1_3.b, t1_3.c
+               Sort Key: t1_3.a
+               ->  Append
+                     ->  Seq Scan on public.prt1_l_p2 t1_3
+                           Output: t1_3.a, t1_3.b, t1_3.c
+                           Filter: ((t1_3.a % 25) = 0)
+                     ->  Seq Scan on public.prt1_l_p2_p1 t1_4
+                           Output: t1_4.a, t1_4.b, t1_4.c
+                           Filter: ((t1_4.a % 25) = 0)
+                     ->  Seq Scan on public.prt1_l_p2_p2 t1_5
+                           Output: t1_5.a, t1_5.b, t1_5.c
+                           Filter: ((t1_5.a % 25) = 0)
+         ->  Sort
+               Output: t1_12.b
+               Sort Key: t1_12.b
+               ->  Append
+                     ->  Seq Scan on public.prt2_l_p2 t1_12
+                           Output: t1_12.b
+                           Filter: ((t1_12.b % 25) = 0)
+                     ->  Seq Scan on public.prt2_l_p2_p1 t1_13
+                           Output: t1_13.b
+                           Filter: ((t1_13.b % 25) = 0)
+                     ->  Seq Scan on public.prt2_l_p2_p2 t1_14
+                           Output: t1_14.b
+                           Filter: ((t1_14.b % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_6.a, t1_6.b, t1_6.c
+         Merge Cond: (t1_6.a = t1_15.b)
+         ->  Sort
+               Output: t1_6.a, t1_6.b, t1_6.c
+               Sort Key: t1_6.a
+               ->  Append
+                     ->  Seq Scan on public.prt1_l_p3 t1_6
+                           Output: t1_6.a, t1_6.b, t1_6.c
+                           Filter: ((t1_6.a % 25) = 0)
+                     ->  Seq Scan on public.prt1_l_p3_p1 t1_7
+                           Output: t1_7.a, t1_7.b, t1_7.c
+                           Filter: ((t1_7.a % 25) = 0)
+                     ->  Seq Scan on public.prt1_l_p3_p2 t1_8
+                           Output: t1_8.a, t1_8.b, t1_8.c
+                           Filter: ((t1_8.a % 25) = 0)
+         ->  Sort
+               Output: t1_15.b
+               Sort Key: t1_15.b
+               ->  Append
+                     ->  Seq Scan on public.prt2_l_p3 t1_15
+                           Output: t1_15.b
+                           Filter: ((t1_15.b % 25) = 0)
+                     ->  Seq Scan on public.prt2_l_p3_p1 t1_16
+                           Output: t1_16.b
+                           Filter: ((t1_16.b % 25) = 0)
+                     ->  Seq Scan on public.prt2_l_p3_p2 t1_17
+                           Output: t1_17.b
+                           Filter: ((t1_17.b % 25) = 0)
+(89 rows)
+
+SELECT t1.* FROM prt1_l t1 WHERE t1.a IN (SELECT t1.b FROM prt2_l t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 1) and (b 3) is considered for partition-wise join.
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1_l t1 WHERE t1.a IN (SELECT t1.b FROM uprt2_l t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
 --
 -- tests for list partitioned tables.
 --
@@ -4832,6 +5242,350 @@ SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.b %
  500 | 500 | 0010
 (8 rows)
 
+-- joins where one of the relations is proven empty
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1, plt2 t2 WHERE t1.a = t2.b AND t1.a = 1 AND t1.a = 2;
+            QUERY PLAN            
+----------------------------------
+ Result
+   Output: t1.a, t1.c, t2.b, t2.c
+   One-Time Filter: false
+(3 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1, plt2 t2 WHERE t1.a = t2.b AND t1.a = 1 AND t1.a = 2;
+ a | c | b | c 
+---+---+---+---
+(0 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1 WHERE a = 1 AND a = 2) t1 LEFT JOIN plt2 t2 ON t1.a = t2.b;
+              QUERY PLAN              
+--------------------------------------
+ Result
+   Output: plt1.a, plt1.c, t2.b, t2.c
+   One-Time Filter: false
+(3 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1 WHERE a = 1 AND a = 2) t1 LEFT JOIN plt2 t2 ON t1.a = t2.b;
+ a | c | b | c 
+---+---+---+---
+(0 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1 WHERE a = 1 AND a = 2) t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                     
+---------------------------------------------------
+ Sort
+   Output: a, c, t2.b, t2.c
+   Sort Key: a, t2.b
+   ->  Hash Left Join
+         Output: a, c, t2.b, t2.c
+         Hash Cond: (t2.b = a)
+         ->  Append
+               ->  Seq Scan on public.plt2 t2
+                     Output: t2.b, t2.c
+                     Filter: ((t2.a % 25) = 0)
+               ->  Seq Scan on public.plt2_p1 t2_1
+                     Output: t2_1.b, t2_1.c
+                     Filter: ((t2_1.a % 25) = 0)
+               ->  Seq Scan on public.plt2_p2 t2_2
+                     Output: t2_2.b, t2_2.c
+                     Filter: ((t2_2.a % 25) = 0)
+               ->  Seq Scan on public.plt2_p3 t2_3
+                     Output: t2_3.b, t2_3.c
+                     Filter: ((t2_3.a % 25) = 0)
+         ->  Hash
+               Output: a, c
+               ->  Result
+                     Output: a, c
+                     One-Time Filter: false
+(24 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1 WHERE a = 1 AND a = 2) t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+ a | c |  b  |  c   
+---+---+-----+------
+   |   |   0 | 0000
+   |   |  75 | 0001
+   |   | 150 | 0003
+   |   | 225 | 0004
+   |   | 300 | 0006
+   |   | 375 | 0007
+   |   | 450 | 0009
+   |   | 525 | 0010
+(8 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uplt1 WHERE a = 1 AND a = 2) t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+ a | c |  b  |  c   
+---+---+-----+------
+   |   |   0 | 0000
+   |   |  75 | 0001
+   |   | 150 | 0003
+   |   | 225 | 0004
+   |   | 300 | 0006
+   |   | 375 | 0007
+   |   | 450 | 0009
+   |   | 525 | 0010
+(8 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1 WHERE a = 1 AND a = 2) t1 FULL JOIN plt2 t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                     
+---------------------------------------------------
+ Sort
+   Output: a, c, t2.b, t2.c
+   Sort Key: a, t2.b
+   ->  Hash Left Join
+         Output: a, c, t2.b, t2.c
+         Hash Cond: (t2.b = a)
+         ->  Append
+               ->  Seq Scan on public.plt2 t2
+                     Output: t2.b, t2.c
+                     Filter: ((t2.a % 25) = 0)
+               ->  Seq Scan on public.plt2_p1 t2_1
+                     Output: t2_1.b, t2_1.c
+                     Filter: ((t2_1.a % 25) = 0)
+               ->  Seq Scan on public.plt2_p2 t2_2
+                     Output: t2_2.b, t2_2.c
+                     Filter: ((t2_2.a % 25) = 0)
+               ->  Seq Scan on public.plt2_p3 t2_3
+                     Output: t2_3.b, t2_3.c
+                     Filter: ((t2_3.a % 25) = 0)
+         ->  Hash
+               Output: a, c
+               ->  Result
+                     Output: a, c
+                     One-Time Filter: false
+(24 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1 WHERE a = 1 AND a = 2) t1 FULL JOIN plt2 t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+ a | c |  b  |  c   
+---+---+-----+------
+   |   |   0 | 0000
+   |   |  75 | 0001
+   |   | 150 | 0003
+   |   | 225 | 0004
+   |   | 300 | 0006
+   |   | 375 | 0007
+   |   | 450 | 0009
+   |   | 525 | 0010
+(8 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uplt1 WHERE a = 1 AND a = 2) t1 FULL JOIN uplt2 t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+ a | c |  b  |  c   
+---+---+-----+------
+   |   |   0 | 0000
+   |   |  75 | 0001
+   |   | 150 | 0003
+   |   | 225 | 0004
+   |   | 300 | 0006
+   |   | 375 | 0007
+   |   | 450 | 0009
+   |   | 525 | 0010
+(8 rows)
+
+-- lateral reference
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT distinct * FROM plt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.c AS t2c, t3.c AS t3c, least(t1.c,t2.c,t3.c) FROM plt1 t2 JOIN plt2 t3 ON (t2.c = t3.c)) ss
+			  ON t1.c = ss.t2c WHERE t1.a % 25 = 0 ORDER BY 1,2,3,4,5,6;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4 5) is considered for partition-wise join.
+                                             QUERY PLAN                                              
+-----------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c, t2.c, t3.c, (LEAST(t1.c, t2.c, t3.c))
+   Sort Key: t1.a, t1.b, t1.c, t2.c, (LEAST(t1.c, t2.c, t3.c))
+   ->  HashAggregate
+         Output: t1.a, t1.b, t1.c, t2.c, t3.c, (LEAST(t1.c, t2.c, t3.c))
+         Group Key: t1.a, t1.b, t1.c, t2.c, t3.c, (LEAST(t1.c, t2.c, t3.c))
+         ->  Append
+               ->  Nested Loop Left Join
+                     Output: t1.a, t1.b, t1.c, t2.c, t3.c, (LEAST(t1.c, t2.c, t3.c))
+                     ->  Seq Scan on public.plt1_p1 t1
+                           Output: t1.a, t1.b, t1.c
+                           Filter: ((t1.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2.c, t3.c, LEAST(t1.c, t2.c, t3.c)
+                           Hash Cond: (t3.c = t2.c)
+                           ->  Seq Scan on public.plt2_p1 t3
+                                 Output: t3.c
+                           ->  Hash
+                                 Output: t2.c
+                                 ->  Seq Scan on public.plt1_p1 t2
+                                       Output: t2.c
+                                       Filter: (t1.c = t2.c)
+               ->  Nested Loop Left Join
+                     Output: t1_1.a, t1_1.b, t1_1.c, t2_1.c, t3_1.c, (LEAST(t1_1.c, t2_1.c, t3_1.c))
+                     ->  Seq Scan on public.plt1_p2 t1_1
+                           Output: t1_1.a, t1_1.b, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2_1.c, t3_1.c, LEAST(t1_1.c, t2_1.c, t3_1.c)
+                           Hash Cond: (t3_1.c = t2_1.c)
+                           ->  Seq Scan on public.plt2_p2 t3_1
+                                 Output: t3_1.c
+                           ->  Hash
+                                 Output: t2_1.c
+                                 ->  Seq Scan on public.plt1_p2 t2_1
+                                       Output: t2_1.c
+                                       Filter: (t1_1.c = t2_1.c)
+               ->  Nested Loop Left Join
+                     Output: t1_2.a, t1_2.b, t1_2.c, t2_2.c, t3_2.c, (LEAST(t1_2.c, t2_2.c, t3_2.c))
+                     ->  Seq Scan on public.plt1_p3 t1_2
+                           Output: t1_2.a, t1_2.b, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2_2.c, t3_2.c, LEAST(t1_2.c, t2_2.c, t3_2.c)
+                           Hash Cond: (t3_2.c = t2_2.c)
+                           ->  Seq Scan on public.plt2_p3 t3_2
+                                 Output: t3_2.c
+                           ->  Hash
+                                 Output: t2_2.c
+                                 ->  Seq Scan on public.plt1_p3 t2_2
+                                       Output: t2_2.c
+                                       Filter: (t1_2.c = t2_2.c)
+(52 rows)
+
+SELECT distinct * FROM plt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.c AS t2c, t3.c AS t3c, least(t1.c,t2.c,t3.c) FROM plt1 t2 JOIN plt2 t3 ON (t2.c = t3.c)) ss
+			  ON t1.c = ss.t2c WHERE t1.a % 25 = 0 ORDER BY 1,2,3,4,5,6;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4 5) is considered for partition-wise join.
+  a  |  b  |  c   | t2c  | t3c  | least 
+-----+-----+------+------+------+-------
+   0 |   0 | 0000 | 0000 | 0000 | 0000
+  50 |  50 | 0001 | 0001 | 0001 | 0001
+ 100 | 100 | 0002 | 0002 | 0002 | 0002
+ 150 | 150 | 0003 | 0003 | 0003 | 0003
+ 200 | 200 | 0004 | 0004 | 0004 | 0004
+ 250 | 250 | 0005 | 0005 | 0005 | 0005
+ 300 | 300 | 0006 | 0006 | 0006 | 0006
+ 350 | 350 | 0007 | 0007 | 0007 | 0007
+ 400 | 400 | 0008 | 0008 | 0008 | 0008
+ 450 | 450 | 0009 | 0009 | 0009 | 0009
+ 500 | 500 | 0010 | 0010 | 0010 | 0010
+ 550 | 550 | 0011 | 0011 | 0011 | 0011
+(12 rows)
+
+SELECT distinct * FROM uplt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.c AS t2c, t3.c AS t3c, least(t1.c,t2.c,t3.c) FROM uplt1 t2 JOIN uplt2 t3 ON (t2.c = t3.c)) ss
+			  ON t1.c = ss.t2c WHERE t1.a % 25 = 0 ORDER BY 1,2,3,4,5,6;
+  a  |  b  |  c   | t2c  | t3c  | least 
+-----+-----+------+------+------+-------
+   0 |   0 | 0000 | 0000 | 0000 | 0000
+  50 |  50 | 0001 | 0001 | 0001 | 0001
+ 100 | 100 | 0002 | 0002 | 0002 | 0002
+ 150 | 150 | 0003 | 0003 | 0003 | 0003
+ 200 | 200 | 0004 | 0004 | 0004 | 0004
+ 250 | 250 | 0005 | 0005 | 0005 | 0005
+ 300 | 300 | 0006 | 0006 | 0006 | 0006
+ 350 | 350 | 0007 | 0007 | 0007 | 0007
+ 400 | 400 | 0008 | 0008 | 0008 | 0008
+ 450 | 450 | 0009 | 0009 | 0009 | 0009
+ 500 | 500 | 0010 | 0010 | 0010 | 0010
+ 550 | 550 | 0011 | 0011 | 0011 | 0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT distinct * FROM plt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.c AS t3c, least(t1.c,t2.c,t3.c) FROM plt1 t2 JOIN plt2 t3 ON (t2.c = t3.c)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY 1,2,3,4,5,6;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Unique
+   Output: t1.a, t1.b, t1.c, t2.a, t3.c, (LEAST(t1.c, t2.c, t3.c))
+   ->  Sort
+         Output: t1.a, t1.b, t1.c, t2.a, t3.c, (LEAST(t1.c, t2.c, t3.c))
+         Sort Key: t1.a, t1.b, t1.c, t2.a, t3.c, (LEAST(t1.c, t2.c, t3.c))
+         ->  Nested Loop Left Join
+               Output: t1.a, t1.b, t1.c, t2.a, t3.c, (LEAST(t1.c, t2.c, t3.c))
+               ->  Append
+                     ->  Seq Scan on public.plt1 t1
+                           Output: t1.a, t1.b, t1.c
+                           Filter: ((t1.a % 25) = 0)
+                     ->  Seq Scan on public.plt1_p1 t1_1
+                           Output: t1_1.a, t1_1.b, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+                     ->  Seq Scan on public.plt1_p2 t1_2
+                           Output: t1_2.a, t1_2.b, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+                     ->  Seq Scan on public.plt1_p3 t1_3
+                           Output: t1_3.a, t1_3.b, t1_3.c
+                           Filter: ((t1_3.a % 25) = 0)
+               ->  Append
+                     ->  Hash Join
+                           Output: t2.a, t3.c, LEAST(t1.c, t2.c, t3.c)
+                           Hash Cond: (t3.c = t2.c)
+                           ->  Seq Scan on public.plt2_p1 t3
+                                 Output: t3.c
+                           ->  Hash
+                                 Output: t2.a, t2.c
+                                 ->  Seq Scan on public.plt1_p1 t2
+                                       Output: t2.a, t2.c
+                                       Filter: (t1.b = t2.a)
+                     ->  Hash Join
+                           Output: t2_1.a, t3_1.c, LEAST(t1.c, t2_1.c, t3_1.c)
+                           Hash Cond: (t3_1.c = t2_1.c)
+                           ->  Seq Scan on public.plt2_p2 t3_1
+                                 Output: t3_1.c
+                           ->  Hash
+                                 Output: t2_1.a, t2_1.c
+                                 ->  Seq Scan on public.plt1_p2 t2_1
+                                       Output: t2_1.a, t2_1.c
+                                       Filter: (t1.b = t2_1.a)
+                     ->  Hash Join
+                           Output: t2_2.a, t3_2.c, LEAST(t1.c, t2_2.c, t3_2.c)
+                           Hash Cond: (t3_2.c = t2_2.c)
+                           ->  Seq Scan on public.plt2_p3 t3_2
+                                 Output: t3_2.c
+                           ->  Hash
+                                 Output: t2_2.a, t2_2.c
+                                 ->  Seq Scan on public.plt1_p3 t2_2
+                                       Output: t2_2.a, t2_2.c
+                                       Filter: (t1.b = t2_2.a)
+(51 rows)
+
+SELECT distinct * FROM plt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.c AS t3c, least(t1.c,t2.c,t3.c) FROM plt1 t2 JOIN plt2 t3 ON (t2.c = t3.c)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY 1,2,3,4,5,6;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  a  |  b  |  c   | t2a | t3c  | least 
+-----+-----+------+-----+------+-------
+   0 |   0 | 0000 |   0 | 0000 | 0000
+  50 |  50 | 0001 |  50 | 0001 | 0001
+ 100 | 100 | 0002 | 100 | 0002 | 0002
+ 150 | 150 | 0003 | 150 | 0003 | 0003
+ 200 | 200 | 0004 | 200 | 0004 | 0004
+ 250 | 250 | 0005 | 250 | 0005 | 0005
+ 300 | 300 | 0006 | 300 | 0006 | 0006
+ 350 | 350 | 0007 | 350 | 0007 | 0007
+ 400 | 400 | 0008 | 400 | 0008 | 0008
+ 450 | 450 | 0009 | 450 | 0009 | 0009
+ 500 | 500 | 0010 | 500 | 0010 | 0010
+ 550 | 550 | 0011 | 550 | 0011 | 0011
+(12 rows)
+
+SELECT distinct * FROM plt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.c AS t3c, least(t1.c,t2.c,t3.c) FROM plt1 t2 JOIN plt2 t3 ON (t2.c = t3.c)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY 1,2,3,4,5,6;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  a  |  b  |  c   | t2a | t3c  | least 
+-----+-----+------+-----+------+-------
+   0 |   0 | 0000 |   0 | 0000 | 0000
+  50 |  50 | 0001 |  50 | 0001 | 0001
+ 100 | 100 | 0002 | 100 | 0002 | 0002
+ 150 | 150 | 0003 | 150 | 0003 | 0003
+ 200 | 200 | 0004 | 200 | 0004 | 0004
+ 250 | 250 | 0005 | 250 | 0005 | 0005
+ 300 | 300 | 0006 | 300 | 0006 | 0006
+ 350 | 350 | 0007 | 350 | 0007 | 0007
+ 400 | 400 | 0008 | 400 | 0008 | 0008
+ 450 | 450 | 0009 | 450 | 0009 | 0009
+ 500 | 500 | 0010 | 500 | 0010 | 0010
+ 550 | 550 | 0011 | 550 | 0011 | 0011
+(12 rows)
+
 --
 -- list partitioned by expression
 --
@@ -5998,6 +6752,88 @@ SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN
  550 | 550 | 0011
 (12 rows)
 
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.phv, t2.b, t2.phv, ltrim(t3.c,'A'), t3.phv FROM ((SELECT 50 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c)) FULL JOIN (SELECT '0002'::text phv, * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.c = ltrim(t3.c,'A')) WHERE t1.a = t1.phv OR t2.b = t2.phv OR ltrim(t3.c,'A') = t3.phv ORDER BY t1.a, t2.b, ltrim(t3.c,'A');
+NOTICE:  join between relations (b 6) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is considered for partition-wise join.
+                                                 QUERY PLAN                                                  
+-------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: plt1_p1.a, (50), plt2_p1.b, (75), (ltrim(plt1_e.c, 'A'::text)), ('0002'::text)
+   Sort Key: plt1_p1.a, plt2_p1.b, (ltrim(plt1_e.c, 'A'::text))
+   ->  Hash Full Join
+         Output: plt1_p1.a, (50), plt2_p1.b, (75), ltrim(plt1_e.c, 'A'::text), ('0002'::text)
+         Hash Cond: (plt1_p1.c = ltrim(plt1_e.c, 'A'::text))
+         Filter: ((plt1_p1.a = (50)) OR (plt2_p1.b = (75)) OR (ltrim(plt1_e.c, 'A'::text) = ('0002'::text)))
+         ->  Append
+               ->  Hash Full Join
+                     Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, (50), (75)
+                     Hash Cond: (plt1_p1.c = plt2_p1.c)
+                     ->  Seq Scan on public.plt1_p1
+                           Output: plt1_p1.a, plt1_p1.c, 50
+                           Filter: ((plt1_p1.a % 25) = 0)
+                     ->  Hash
+                           Output: plt2_p1.b, plt2_p1.c, (75)
+                           ->  Seq Scan on public.plt2_p1
+                                 Output: plt2_p1.b, plt2_p1.c, 75
+                                 Filter: ((plt2_p1.b % 25) = 0)
+               ->  Hash Full Join
+                     Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, (50), (75)
+                     Hash Cond: (plt1_p2.c = plt2_p2.c)
+                     ->  Seq Scan on public.plt1_p2
+                           Output: plt1_p2.a, plt1_p2.c, 50
+                           Filter: ((plt1_p2.a % 25) = 0)
+                     ->  Hash
+                           Output: plt2_p2.b, plt2_p2.c, (75)
+                           ->  Seq Scan on public.plt2_p2
+                                 Output: plt2_p2.b, plt2_p2.c, 75
+                                 Filter: ((plt2_p2.b % 25) = 0)
+               ->  Hash Full Join
+                     Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, (50), (75)
+                     Hash Cond: (plt1_p3.c = plt2_p3.c)
+                     ->  Seq Scan on public.plt1_p3
+                           Output: plt1_p3.a, plt1_p3.c, 50
+                           Filter: ((plt1_p3.a % 25) = 0)
+                     ->  Hash
+                           Output: plt2_p3.b, plt2_p3.c, (75)
+                           ->  Seq Scan on public.plt2_p3
+                                 Output: plt2_p3.b, plt2_p3.c, 75
+                                 Filter: ((plt2_p3.b % 25) = 0)
+         ->  Hash
+               Output: plt1_e.c, ('0002'::text)
+               ->  Append
+                     ->  Seq Scan on public.plt1_e
+                           Output: plt1_e.c, '0002'::text
+                           Filter: ((plt1_e.a % 25) = 0)
+                     ->  Seq Scan on public.plt1_e_p1
+                           Output: plt1_e_p1.c, '0002'::text
+                           Filter: ((plt1_e_p1.a % 25) = 0)
+                     ->  Seq Scan on public.plt1_e_p2
+                           Output: plt1_e_p2.c, '0002'::text
+                           Filter: ((plt1_e_p2.a % 25) = 0)
+                     ->  Seq Scan on public.plt1_e_p3
+                           Output: plt1_e_p3.c, '0002'::text
+                           Filter: ((plt1_e_p3.a % 25) = 0)
+(56 rows)
+
+SELECT t1.a, t1.phv, t2.b, t2.phv, ltrim(t3.c,'A'), t3.phv FROM ((SELECT 50 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c)) FULL JOIN (SELECT '0002'::text phv, * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.c = ltrim(t3.c,'A')) WHERE t1.a = t1.phv OR t2.b = t2.phv OR ltrim(t3.c,'A') = t3.phv ORDER BY t1.a, t2.b, ltrim(t3.c,'A');
+NOTICE:  join between relations (b 6) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 6 7) and (b 8) is considered for partition-wise join.
+  a  | phv | b  | phv | ltrim | phv  
+-----+-----+----+-----+-------+------
+  50 |  50 | 75 |  75 | 0001  | 0002
+ 100 |  50 |    |     | 0002  | 0002
+(2 rows)
+
+SELECT t1.a, t1.phv, t2.b, t2.phv, ltrim(t3.c,'A'), t3.phv FROM ((SELECT 50 phv, * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.c = t2.c)) FULL JOIN (SELECT '0002'::text phv, * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.c = ltrim(t3.c,'A')) WHERE t1.a = t1.phv OR t2.b = t2.phv OR ltrim(t3.c,'A') = t3.phv ORDER BY t1.a, t2.b, ltrim(t3.c,'A');
+  a  | phv | b  | phv | ltrim | phv  
+-----+-----+----+-----+-------+------
+  50 |  50 | 75 |  75 | 0001  | 0002
+ 100 |  50 |    |     | 0002  | 0002
+(2 rows)
+
 -- test merge join with and without index scan
 CREATE INDEX iplt1_c on plt1(c);
 CREATE INDEX iplt1_p1_c on plt1_p1(c);
@@ -6781,9 +7617,1510 @@ SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN
  550 | 550 | 0011
 (12 rows)
 
+-- lateral references and parameterized paths
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT distinct * FROM plt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.c AS t2c, t3.c AS t3c, least(t1.c,t2.c,t3.c) FROM plt1 t2 JOIN plt2 t3 ON (t2.c = t3.c)) ss
+			  ON t1.c = ss.t2c WHERE t1.a % 25 = 0 ORDER BY 1,2,3,4,5,6;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4 5) is considered for partition-wise join.
+                                         QUERY PLAN                                          
+---------------------------------------------------------------------------------------------
+ Unique
+   Output: t1.a, t1.b, t1.c, t2.c, t3.c, (LEAST(t1.c, t2.c, t3.c))
+   ->  Sort
+         Output: t1.a, t1.b, t1.c, t2.c, t3.c, (LEAST(t1.c, t2.c, t3.c))
+         Sort Key: t1.a, t1.b, t1.c, t2.c, (LEAST(t1.c, t2.c, t3.c))
+         ->  Nested Loop Left Join
+               Output: t1.a, t1.b, t1.c, t2.c, t3.c, (LEAST(t1.c, t2.c, t3.c))
+               ->  Append
+                     ->  Index Scan using iplt1_c on public.plt1 t1
+                           Output: t1.a, t1.b, t1.c
+                           Filter: ((t1.a % 25) = 0)
+                     ->  Index Scan using iplt1_p1_c on public.plt1_p1 t1_1
+                           Output: t1_1.a, t1_1.b, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+                     ->  Index Scan using iplt1_p2_c on public.plt1_p2 t1_2
+                           Output: t1_2.a, t1_2.b, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+                     ->  Index Scan using iplt1_p3_c on public.plt1_p3 t1_3
+                           Output: t1_3.a, t1_3.b, t1_3.c
+                           Filter: ((t1_3.a % 25) = 0)
+               ->  Append
+                     ->  Merge Join
+                           Output: t2.c, t3.c, LEAST(t1.c, t2.c, t3.c)
+                           Merge Cond: (t2.c = t3.c)
+                           ->  Index Only Scan using iplt1_p1_c on public.plt1_p1 t2
+                                 Output: t2.c
+                                 Index Cond: (t2.c = t1.c)
+                           ->  Materialize
+                                 Output: t3.c
+                                 ->  Index Only Scan using iplt2_p1_c on public.plt2_p1 t3
+                                       Output: t3.c
+                     ->  Merge Join
+                           Output: t2_1.c, t3_1.c, LEAST(t1.c, t2_1.c, t3_1.c)
+                           Merge Cond: (t2_1.c = t3_1.c)
+                           ->  Index Only Scan using iplt1_p2_c on public.plt1_p2 t2_1
+                                 Output: t2_1.c
+                                 Index Cond: (t2_1.c = t1.c)
+                           ->  Materialize
+                                 Output: t3_1.c
+                                 ->  Index Only Scan using iplt2_p2_c on public.plt2_p2 t3_1
+                                       Output: t3_1.c
+                     ->  Merge Join
+                           Output: t2_2.c, t3_2.c, LEAST(t1.c, t2_2.c, t3_2.c)
+                           Merge Cond: (t2_2.c = t3_2.c)
+                           ->  Index Only Scan using iplt1_p3_c on public.plt1_p3 t2_2
+                                 Output: t2_2.c
+                                 Index Cond: (t2_2.c = t1.c)
+                           ->  Materialize
+                                 Output: t3_2.c
+                                 ->  Index Only Scan using iplt2_p3_c on public.plt2_p3 t3_2
+                                       Output: t3_2.c
+(51 rows)
+
+SELECT distinct * FROM plt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.c AS t2c, t3.c AS t3c, least(t1.c,t2.c,t3.c) FROM plt1 t2 JOIN plt2 t3 ON (t2.c = t3.c)) ss
+			  ON t1.c = ss.t2c WHERE t1.a % 25 = 0 ORDER BY 1,2,3,4,5,6;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 1) and (b 4 5) is considered for partition-wise join.
+  a  |  b  |  c   | t2c  | t3c  | least 
+-----+-----+------+------+------+-------
+   0 |   0 | 0000 | 0000 | 0000 | 0000
+  50 |  50 | 0001 | 0001 | 0001 | 0001
+ 100 | 100 | 0002 | 0002 | 0002 | 0002
+ 150 | 150 | 0003 | 0003 | 0003 | 0003
+ 200 | 200 | 0004 | 0004 | 0004 | 0004
+ 250 | 250 | 0005 | 0005 | 0005 | 0005
+ 300 | 300 | 0006 | 0006 | 0006 | 0006
+ 350 | 350 | 0007 | 0007 | 0007 | 0007
+ 400 | 400 | 0008 | 0008 | 0008 | 0008
+ 450 | 450 | 0009 | 0009 | 0009 | 0009
+ 500 | 500 | 0010 | 0010 | 0010 | 0010
+ 550 | 550 | 0011 | 0011 | 0011 | 0011
+(12 rows)
+
+SELECT distinct * FROM uplt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.c AS t2c, t3.c AS t3c, least(t1.c,t2.c,t3.c) FROM uplt1 t2 JOIN uplt2 t3 ON (t2.c = t3.c)) ss
+			  ON t1.c = ss.t2c WHERE t1.a % 25 = 0 ORDER BY 1,2,3,4,5,6;
+  a  |  b  |  c   | t2c  | t3c  | least 
+-----+-----+------+------+------+-------
+   0 |   0 | 0000 | 0000 | 0000 | 0000
+  50 |  50 | 0001 | 0001 | 0001 | 0001
+ 100 | 100 | 0002 | 0002 | 0002 | 0002
+ 150 | 150 | 0003 | 0003 | 0003 | 0003
+ 200 | 200 | 0004 | 0004 | 0004 | 0004
+ 250 | 250 | 0005 | 0005 | 0005 | 0005
+ 300 | 300 | 0006 | 0006 | 0006 | 0006
+ 350 | 350 | 0007 | 0007 | 0007 | 0007
+ 400 | 400 | 0008 | 0008 | 0008 | 0008
+ 450 | 450 | 0009 | 0009 | 0009 | 0009
+ 500 | 500 | 0010 | 0010 | 0010 | 0010
+ 550 | 550 | 0011 | 0011 | 0011 | 0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT distinct * FROM plt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.c AS t3c, least(t1.c,t2.c,t3.c) FROM plt1 t2 JOIN plt2 t3 ON (t2.c = t3.c)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY 1,2,3,4,5,6;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+                                      QUERY PLAN                                       
+---------------------------------------------------------------------------------------
+ Unique
+   Output: t1.a, t1.b, t1.c, t2.a, t3.c, (LEAST(t1.c, t2.c, t3.c))
+   ->  Sort
+         Output: t1.a, t1.b, t1.c, t2.a, t3.c, (LEAST(t1.c, t2.c, t3.c))
+         Sort Key: t1.a, t1.b, t1.c, t2.a, t3.c, (LEAST(t1.c, t2.c, t3.c))
+         ->  Nested Loop Left Join
+               Output: t1.a, t1.b, t1.c, t2.a, t3.c, (LEAST(t1.c, t2.c, t3.c))
+               ->  Append
+                     ->  Seq Scan on public.plt1 t1
+                           Output: t1.a, t1.b, t1.c
+                           Filter: ((t1.a % 25) = 0)
+                     ->  Seq Scan on public.plt1_p1 t1_1
+                           Output: t1_1.a, t1_1.b, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+                     ->  Seq Scan on public.plt1_p2 t1_2
+                           Output: t1_2.a, t1_2.b, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+                     ->  Seq Scan on public.plt1_p3 t1_3
+                           Output: t1_3.a, t1_3.b, t1_3.c
+                           Filter: ((t1_3.a % 25) = 0)
+               ->  Append
+                     ->  Merge Join
+                           Output: t2.a, t3.c, LEAST(t1.c, t2.c, t3.c)
+                           Merge Cond: (t2.c = t3.c)
+                           ->  Index Scan using iplt1_p1_c on public.plt1_p1 t2
+                                 Output: t2.a, t2.c
+                                 Filter: (t1.b = t2.a)
+                           ->  Index Only Scan using iplt2_p1_c on public.plt2_p1 t3
+                                 Output: t3.c
+                     ->  Merge Join
+                           Output: t2_1.a, t3_1.c, LEAST(t1.c, t2_1.c, t3_1.c)
+                           Merge Cond: (t2_1.c = t3_1.c)
+                           ->  Index Scan using iplt1_p2_c on public.plt1_p2 t2_1
+                                 Output: t2_1.a, t2_1.c
+                                 Filter: (t1.b = t2_1.a)
+                           ->  Index Only Scan using iplt2_p2_c on public.plt2_p2 t3_1
+                                 Output: t3_1.c
+                     ->  Merge Join
+                           Output: t2_2.a, t3_2.c, LEAST(t1.c, t2_2.c, t3_2.c)
+                           Merge Cond: (t2_2.c = t3_2.c)
+                           ->  Index Scan using iplt1_p3_c on public.plt1_p3 t2_2
+                                 Output: t2_2.a, t2_2.c
+                                 Filter: (t1.b = t2_2.a)
+                           ->  Index Only Scan using iplt2_p3_c on public.plt2_p3 t3_2
+                                 Output: t3_2.c
+(45 rows)
+
+SELECT distinct * FROM plt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.c AS t3c, least(t1.c,t2.c,t3.c) FROM plt1 t2 JOIN plt2 t3 ON (t2.c = t3.c)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY 1,2,3,4,5,6;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+  a  |  b  |  c   | t2a | t3c  | least 
+-----+-----+------+-----+------+-------
+   0 |   0 | 0000 |   0 | 0000 | 0000
+  50 |  50 | 0001 |  50 | 0001 | 0001
+ 100 | 100 | 0002 | 100 | 0002 | 0002
+ 150 | 150 | 0003 | 150 | 0003 | 0003
+ 200 | 200 | 0004 | 200 | 0004 | 0004
+ 250 | 250 | 0005 | 250 | 0005 | 0005
+ 300 | 300 | 0006 | 300 | 0006 | 0006
+ 350 | 350 | 0007 | 350 | 0007 | 0007
+ 400 | 400 | 0008 | 400 | 0008 | 0008
+ 450 | 450 | 0009 | 450 | 0009 | 0009
+ 500 | 500 | 0010 | 500 | 0010 | 0010
+ 550 | 550 | 0011 | 550 | 0011 | 0011
+(12 rows)
+
+SELECT distinct * FROM uplt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.c AS t3c, least(t1.c,t2.c,t3.c) FROM uplt1 t2 JOIN uplt2 t3 ON (t2.c = t3.c)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY 1,2,3,4,5,6;
+  a  |  b  |  c   | t2a | t3c  | least 
+-----+-----+------+-----+------+-------
+   0 |   0 | 0000 |   0 | 0000 | 0000
+  50 |  50 | 0001 |  50 | 0001 | 0001
+ 100 | 100 | 0002 | 100 | 0002 | 0002
+ 150 | 150 | 0003 | 150 | 0003 | 0003
+ 200 | 200 | 0004 | 200 | 0004 | 0004
+ 250 | 250 | 0005 | 250 | 0005 | 0005
+ 300 | 300 | 0006 | 300 | 0006 | 0006
+ 350 | 350 | 0007 | 350 | 0007 | 0007
+ 400 | 400 | 0008 | 400 | 0008 | 0008
+ 450 | 450 | 0009 | 450 | 0009 | 0009
+ 500 | 500 | 0010 | 500 | 0010 | 0010
+ 550 | 550 | 0011 | 550 | 0011 | 0011
+(12 rows)
+
 RESET enable_hashjoin;
 RESET enable_nestloop;
 RESET enable_seqscan;
+
+--
+-- multi-leveled partitions
+--
+CREATE TABLE plt1_l (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_l_p1 PARTITION OF plt1_l FOR VALUES IN ('0000', '0003', '0004', '0010') PARTITION BY LIST (c);
+CREATE TABLE plt1_l_p1_p1 PARTITION OF plt1_l_p1 FOR VALUES IN ('0000', '0003');
+CREATE TABLE plt1_l_p1_p2 PARTITION OF plt1_l_p1 FOR VALUES IN ('0004', '0010');
+CREATE TABLE plt1_l_p2 PARTITION OF plt1_l FOR VALUES IN ('0001', '0005', '0002', '0009') PARTITION BY LIST (c);
+CREATE TABLE plt1_l_p2_p1 PARTITION OF plt1_l_p2 FOR VALUES IN ('0001', '0005');
+CREATE TABLE plt1_l_p2_p2 PARTITION OF plt1_l_p2 FOR VALUES IN ('0002', '0009');
+CREATE TABLE plt1_l_p3 PARTITION OF plt1_l FOR VALUES IN ('0006', '0007', '0008', '0011') PARTITION BY LIST (ltrim(c,'A'));
+CREATE TABLE plt1_l_p3_p1 PARTITION OF plt1_l_p3 FOR VALUES IN ('0006', '0007');
+CREATE TABLE plt1_l_p3_p2 PARTITION OF plt1_l_p3 FOR VALUES IN ('0008', '0011');
+INSERT INTO plt1_l SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE plt1_l;
+ANALYZE plt1_l_p1;
+ANALYZE plt1_l_p1_p1;
+ANALYZE plt1_l_p1_p2;
+ANALYZE plt1_l_p2;
+ANALYZE plt1_l_p2_p1;
+ANALYZE plt1_l_p2_p2;
+ANALYZE plt1_l_p3;
+ANALYZE plt1_l_p3_p1;
+ANALYZE plt1_l_p3_p2;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1_l AS SELECT * FROM plt1_l;
+CREATE TABLE plt2_l (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_l_p1 PARTITION OF plt2_l FOR VALUES IN ('0000', '0003', '0004', '0010') PARTITION BY LIST (c);
+CREATE TABLE plt2_l_p1_p1 PARTITION OF plt2_l_p1 FOR VALUES IN ('0000', '0003');
+CREATE TABLE plt2_l_p1_p2 PARTITION OF plt2_l_p1 FOR VALUES IN ('0004', '0010');
+CREATE TABLE plt2_l_p2 PARTITION OF plt2_l FOR VALUES IN ('0001', '0005', '0002', '0009') PARTITION BY LIST (c);
+CREATE TABLE plt2_l_p2_p1 PARTITION OF plt2_l_p2 FOR VALUES IN ('0001', '0005');
+CREATE TABLE plt2_l_p2_p2 PARTITION OF plt2_l_p2 FOR VALUES IN ('0002', '0009');
+CREATE TABLE plt2_l_p3 PARTITION OF plt2_l FOR VALUES IN ('0006', '0007', '0008', '0011') PARTITION BY LIST (ltrim(c,'A'));
+CREATE TABLE plt2_l_p3_p1 PARTITION OF plt2_l_p3 FOR VALUES IN ('0006', '0007');
+CREATE TABLE plt2_l_p3_p2 PARTITION OF plt2_l_p3 FOR VALUES IN ('0008', '0011');
+INSERT INTO plt2_l SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE plt2_l;
+ANALYZE plt2_l_p1;
+ANALYZE plt2_l_p1_p1;
+ANALYZE plt2_l_p1_p2;
+ANALYZE plt2_l_p2;
+ANALYZE plt2_l_p2_p1;
+ANALYZE plt2_l_p2_p2;
+ANALYZE plt2_l_p3;
+ANALYZE plt2_l_p3_p1;
+ANALYZE plt2_l_p3_p2;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt2_l AS SELECT * FROM plt2_l;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_l t1, plt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 17) is considered for partition-wise join.
+NOTICE:  join between relations (b 8) and (b 21) is considered for partition-wise join.
+NOTICE:  join between relations (b 12) and (b 25) is considered for partition-wise join.
+                                                                                     QUERY PLAN                                                                                     
+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((t2.b = t1.a) AND (t2.a = t1.b) AND ((t2.c)::text = (t1.c)::text) AND (ltrim((t2.c)::text, 'A'::text) = ltrim((t1.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt2_l_p1_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.plt1_l_p1_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.a = t1_1.b) AND ((t2_1.c)::text = (t1_1.c)::text) AND (ltrim((t2_1.c)::text, 'A'::text) = ltrim((t1_1.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt2_l_p1_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.plt1_l_p1_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.a = t1_2.b) AND ((t2_2.c)::text = (t1_2.c)::text) AND (ltrim((t2_2.c)::text, 'A'::text) = ltrim((t1_2.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt2_l_p2_p1 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.plt1_l_p2_p1 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_3.a, t1_3.c, t2_3.b, t2_3.c
+               Hash Cond: ((t2_3.b = t1_3.a) AND (t2_3.a = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text) AND (ltrim((t2_3.c)::text, 'A'::text) = ltrim((t1_3.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt2_l_p2_p2 t2_3
+                     Output: t2_3.b, t2_3.c, t2_3.a
+               ->  Hash
+                     Output: t1_3.a, t1_3.c, t1_3.b
+                     ->  Seq Scan on public.plt1_l_p2_p2 t1_3
+                           Output: t1_3.a, t1_3.c, t1_3.b
+                           Filter: ((t1_3.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_4.a, t1_4.c, t2_4.b, t2_4.c
+               Hash Cond: ((t2_4.b = t1_4.a) AND (t2_4.a = t1_4.b) AND ((t2_4.c)::text = (t1_4.c)::text) AND (ltrim((t2_4.c)::text, 'A'::text) = ltrim((t1_4.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt2_l_p3_p1 t2_4
+                     Output: t2_4.b, t2_4.c, t2_4.a
+               ->  Hash
+                     Output: t1_4.a, t1_4.c, t1_4.b
+                     ->  Seq Scan on public.plt1_l_p3_p1 t1_4
+                           Output: t1_4.a, t1_4.c, t1_4.b
+                           Filter: ((t1_4.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_5.a, t1_5.c, t2_5.b, t2_5.c
+               Hash Cond: ((t2_5.b = t1_5.a) AND (t2_5.a = t1_5.b) AND ((t2_5.c)::text = (t1_5.c)::text) AND (ltrim((t2_5.c)::text, 'A'::text) = ltrim((t1_5.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt2_l_p3_p2 t2_5
+                     Output: t2_5.b, t2_5.c, t2_5.a
+               ->  Hash
+                     Output: t1_5.a, t1_5.c, t1_5.b
+                     ->  Seq Scan on public.plt1_l_p3_p2 t1_5
+                           Output: t1_5.a, t1_5.c, t1_5.b
+                           Filter: ((t1_5.a % 25) = 0)
+(64 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_l t1, plt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 17) is considered for partition-wise join.
+NOTICE:  join between relations (b 8) and (b 21) is considered for partition-wise join.
+NOTICE:  join between relations (b 12) and (b 25) is considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0003 | 150 | 0003
+ 300 | 0006 | 300 | 0006
+ 450 | 0009 | 450 | 0009
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_l t1, uplt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0003 | 150 | 0003
+ 300 | 0006 | 300 | 0006
+ 450 | 0009 | 450 | 0009
+(4 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_l t1 LEFT JOIN plt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 5) and (b 18) is considered for partition-wise join.
+NOTICE:  join between relations (b 9) and (b 22) is considered for partition-wise join.
+NOTICE:  join between relations (b 13) and (b 26) is considered for partition-wise join.
+                                                                                     QUERY PLAN                                                                                     
+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((t2.b = t1.a) AND (t2.a = t1.b) AND ((t2.c)::text = (t1.c)::text) AND (ltrim((t2.c)::text, 'A'::text) = ltrim((t1.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt2_l_p1_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.plt1_l_p1_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.a = t1_1.b) AND ((t2_1.c)::text = (t1_1.c)::text) AND (ltrim((t2_1.c)::text, 'A'::text) = ltrim((t1_1.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt2_l_p1_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.plt1_l_p1_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.a = t1_2.b) AND ((t2_2.c)::text = (t1_2.c)::text) AND (ltrim((t2_2.c)::text, 'A'::text) = ltrim((t1_2.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt2_l_p2_p1 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.plt1_l_p2_p1 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_3.a, t1_3.c, t2_3.b, t2_3.c
+               Hash Cond: ((t2_3.b = t1_3.a) AND (t2_3.a = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text) AND (ltrim((t2_3.c)::text, 'A'::text) = ltrim((t1_3.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt2_l_p2_p2 t2_3
+                     Output: t2_3.b, t2_3.c, t2_3.a
+               ->  Hash
+                     Output: t1_3.a, t1_3.c, t1_3.b
+                     ->  Seq Scan on public.plt1_l_p2_p2 t1_3
+                           Output: t1_3.a, t1_3.c, t1_3.b
+                           Filter: ((t1_3.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_4.a, t1_4.c, t2_4.b, t2_4.c
+               Hash Cond: ((t2_4.b = t1_4.a) AND (t2_4.a = t1_4.b) AND ((t2_4.c)::text = (t1_4.c)::text) AND (ltrim((t2_4.c)::text, 'A'::text) = ltrim((t1_4.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt2_l_p3_p1 t2_4
+                     Output: t2_4.b, t2_4.c, t2_4.a
+               ->  Hash
+                     Output: t1_4.a, t1_4.c, t1_4.b
+                     ->  Seq Scan on public.plt1_l_p3_p1 t1_4
+                           Output: t1_4.a, t1_4.c, t1_4.b
+                           Filter: ((t1_4.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_5.a, t1_5.c, t2_5.b, t2_5.c
+               Hash Cond: ((t2_5.b = t1_5.a) AND (t2_5.a = t1_5.b) AND ((t2_5.c)::text = (t1_5.c)::text) AND (ltrim((t2_5.c)::text, 'A'::text) = ltrim((t1_5.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt2_l_p3_p2 t2_5
+                     Output: t2_5.b, t2_5.c, t2_5.a
+               ->  Hash
+                     Output: t1_5.a, t1_5.c, t1_5.b
+                     ->  Seq Scan on public.plt1_l_p3_p2 t1_5
+                           Output: t1_5.a, t1_5.c, t1_5.b
+                           Filter: ((t1_5.a % 25) = 0)
+(64 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_l t1 LEFT JOIN plt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 5) and (b 18) is considered for partition-wise join.
+NOTICE:  join between relations (b 9) and (b 22) is considered for partition-wise join.
+NOTICE:  join between relations (b 13) and (b 26) is considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  50 | 0001 |     | 
+ 100 | 0002 |     | 
+ 150 | 0003 | 150 | 0003
+ 200 | 0004 |     | 
+ 250 | 0005 |     | 
+ 300 | 0006 | 300 | 0006
+ 350 | 0007 |     | 
+ 400 | 0008 |     | 
+ 450 | 0009 | 450 | 0009
+ 500 | 0010 |     | 
+ 550 | 0011 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_l t1 LEFT JOIN uplt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  50 | 0001 |     | 
+ 100 | 0002 |     | 
+ 150 | 0003 | 150 | 0003
+ 200 | 0004 |     | 
+ 250 | 0005 |     | 
+ 300 | 0006 | 300 | 0006
+ 350 | 0007 |     | 
+ 400 | 0008 |     | 
+ 450 | 0009 | 450 | 0009
+ 500 | 0010 |     | 
+ 550 | 0011 |     | 
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_l t1 RIGHT JOIN plt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 5) and (b 18) is considered for partition-wise join.
+NOTICE:  join between relations (b 9) and (b 22) is considered for partition-wise join.
+NOTICE:  join between relations (b 13) and (b 26) is considered for partition-wise join.
+                                                                                     QUERY PLAN                                                                                     
+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((t2.b = t1.a) AND (t2.a = t1.b) AND ((t2.c)::text = (t1.c)::text) AND (ltrim((t2.c)::text, 'A'::text) = ltrim((t1.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt2_l_p1_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.plt1_l_p1_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.a = t1_1.b) AND ((t2_1.c)::text = (t1_1.c)::text) AND (ltrim((t2_1.c)::text, 'A'::text) = ltrim((t1_1.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt2_l_p1_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.plt1_l_p1_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.a = t1_2.b) AND ((t2_2.c)::text = (t1_2.c)::text) AND (ltrim((t2_2.c)::text, 'A'::text) = ltrim((t1_2.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt2_l_p2_p1 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.plt1_l_p2_p1 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_3.a, t1_3.c, t2_3.b, t2_3.c
+               Hash Cond: ((t2_3.b = t1_3.a) AND (t2_3.a = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text) AND (ltrim((t2_3.c)::text, 'A'::text) = ltrim((t1_3.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt2_l_p2_p2 t2_3
+                     Output: t2_3.b, t2_3.c, t2_3.a
+               ->  Hash
+                     Output: t1_3.a, t1_3.c, t1_3.b
+                     ->  Seq Scan on public.plt1_l_p2_p2 t1_3
+                           Output: t1_3.a, t1_3.c, t1_3.b
+                           Filter: ((t1_3.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_4.a, t1_4.c, t2_4.b, t2_4.c
+               Hash Cond: ((t2_4.b = t1_4.a) AND (t2_4.a = t1_4.b) AND ((t2_4.c)::text = (t1_4.c)::text) AND (ltrim((t2_4.c)::text, 'A'::text) = ltrim((t1_4.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt2_l_p3_p1 t2_4
+                     Output: t2_4.b, t2_4.c, t2_4.a
+               ->  Hash
+                     Output: t1_4.a, t1_4.c, t1_4.b
+                     ->  Seq Scan on public.plt1_l_p3_p1 t1_4
+                           Output: t1_4.a, t1_4.c, t1_4.b
+                           Filter: ((t1_4.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_5.a, t1_5.c, t2_5.b, t2_5.c
+               Hash Cond: ((t2_5.b = t1_5.a) AND (t2_5.a = t1_5.b) AND ((t2_5.c)::text = (t1_5.c)::text) AND (ltrim((t2_5.c)::text, 'A'::text) = ltrim((t1_5.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt2_l_p3_p2 t2_5
+                     Output: t2_5.b, t2_5.c, t2_5.a
+               ->  Hash
+                     Output: t1_5.a, t1_5.c, t1_5.b
+                     ->  Seq Scan on public.plt1_l_p3_p2 t1_5
+                           Output: t1_5.a, t1_5.c, t1_5.b
+                           Filter: ((t1_5.a % 25) = 0)
+(64 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_l t1 RIGHT JOIN plt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 5) and (b 18) is considered for partition-wise join.
+NOTICE:  join between relations (b 9) and (b 22) is considered for partition-wise join.
+NOTICE:  join between relations (b 13) and (b 26) is considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0003 | 150 | 0003
+ 300 | 0006 | 300 | 0006
+ 450 | 0009 | 450 | 0009
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_l t1 RIGHT JOIN uplt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0003 | 150 | 0003
+ 300 | 0006 | 300 | 0006
+ 450 | 0009 | 450 | 0009
+(4 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_l WHERE plt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2_l WHERE plt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A')) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 7) and (b 20) is considered for partition-wise join.
+NOTICE:  join between relations (b 11) and (b 24) is considered for partition-wise join.
+NOTICE:  join between relations (b 15) and (b 28) is considered for partition-wise join.
+                                                                                                                     QUERY PLAN                                                                                                                     
+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: plt1_l_p1_p1.a, plt1_l_p1_p1.c, plt2_l_p1_p1.b, plt2_l_p1_p1.c
+   Sort Key: plt1_l_p1_p1.a, plt2_l_p1_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: plt1_l_p1_p1.a, plt1_l_p1_p1.c, plt2_l_p1_p1.b, plt2_l_p1_p1.c
+               Hash Cond: ((plt1_l_p1_p1.a = plt2_l_p1_p1.b) AND (plt1_l_p1_p1.b = plt2_l_p1_p1.a) AND ((plt1_l_p1_p1.c)::text = (plt2_l_p1_p1.c)::text) AND (ltrim((plt1_l_p1_p1.c)::text, 'A'::text) = ltrim((plt2_l_p1_p1.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt1_l_p1_p1
+                     Output: plt1_l_p1_p1.a, plt1_l_p1_p1.c, plt1_l_p1_p1.b
+                     Filter: ((plt1_l_p1_p1.a % 25) = 0)
+               ->  Hash
+                     Output: plt2_l_p1_p1.b, plt2_l_p1_p1.c, plt2_l_p1_p1.a
+                     ->  Seq Scan on public.plt2_l_p1_p1
+                           Output: plt2_l_p1_p1.b, plt2_l_p1_p1.c, plt2_l_p1_p1.a
+                           Filter: ((plt2_l_p1_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: plt1_l_p1_p2.a, plt1_l_p1_p2.c, plt2_l_p1_p2.b, plt2_l_p1_p2.c
+               Hash Cond: ((plt1_l_p1_p2.a = plt2_l_p1_p2.b) AND (plt1_l_p1_p2.b = plt2_l_p1_p2.a) AND ((plt1_l_p1_p2.c)::text = (plt2_l_p1_p2.c)::text) AND (ltrim((plt1_l_p1_p2.c)::text, 'A'::text) = ltrim((plt2_l_p1_p2.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt1_l_p1_p2
+                     Output: plt1_l_p1_p2.a, plt1_l_p1_p2.c, plt1_l_p1_p2.b
+                     Filter: ((plt1_l_p1_p2.a % 25) = 0)
+               ->  Hash
+                     Output: plt2_l_p1_p2.b, plt2_l_p1_p2.c, plt2_l_p1_p2.a
+                     ->  Seq Scan on public.plt2_l_p1_p2
+                           Output: plt2_l_p1_p2.b, plt2_l_p1_p2.c, plt2_l_p1_p2.a
+                           Filter: ((plt2_l_p1_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: plt1_l_p2_p1.a, plt1_l_p2_p1.c, plt2_l_p2_p1.b, plt2_l_p2_p1.c
+               Hash Cond: ((plt1_l_p2_p1.a = plt2_l_p2_p1.b) AND (plt1_l_p2_p1.b = plt2_l_p2_p1.a) AND ((plt1_l_p2_p1.c)::text = (plt2_l_p2_p1.c)::text) AND (ltrim((plt1_l_p2_p1.c)::text, 'A'::text) = ltrim((plt2_l_p2_p1.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt1_l_p2_p1
+                     Output: plt1_l_p2_p1.a, plt1_l_p2_p1.c, plt1_l_p2_p1.b
+                     Filter: ((plt1_l_p2_p1.a % 25) = 0)
+               ->  Hash
+                     Output: plt2_l_p2_p1.b, plt2_l_p2_p1.c, plt2_l_p2_p1.a
+                     ->  Seq Scan on public.plt2_l_p2_p1
+                           Output: plt2_l_p2_p1.b, plt2_l_p2_p1.c, plt2_l_p2_p1.a
+                           Filter: ((plt2_l_p2_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: plt1_l_p2_p2.a, plt1_l_p2_p2.c, plt2_l_p2_p2.b, plt2_l_p2_p2.c
+               Hash Cond: ((plt1_l_p2_p2.a = plt2_l_p2_p2.b) AND (plt1_l_p2_p2.b = plt2_l_p2_p2.a) AND ((plt1_l_p2_p2.c)::text = (plt2_l_p2_p2.c)::text) AND (ltrim((plt1_l_p2_p2.c)::text, 'A'::text) = ltrim((plt2_l_p2_p2.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt1_l_p2_p2
+                     Output: plt1_l_p2_p2.a, plt1_l_p2_p2.c, plt1_l_p2_p2.b
+                     Filter: ((plt1_l_p2_p2.a % 25) = 0)
+               ->  Hash
+                     Output: plt2_l_p2_p2.b, plt2_l_p2_p2.c, plt2_l_p2_p2.a
+                     ->  Seq Scan on public.plt2_l_p2_p2
+                           Output: plt2_l_p2_p2.b, plt2_l_p2_p2.c, plt2_l_p2_p2.a
+                           Filter: ((plt2_l_p2_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: plt1_l_p3_p1.a, plt1_l_p3_p1.c, plt2_l_p3_p1.b, plt2_l_p3_p1.c
+               Hash Cond: ((plt1_l_p3_p1.a = plt2_l_p3_p1.b) AND (plt1_l_p3_p1.b = plt2_l_p3_p1.a) AND ((plt1_l_p3_p1.c)::text = (plt2_l_p3_p1.c)::text) AND (ltrim((plt1_l_p3_p1.c)::text, 'A'::text) = ltrim((plt2_l_p3_p1.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt1_l_p3_p1
+                     Output: plt1_l_p3_p1.a, plt1_l_p3_p1.c, plt1_l_p3_p1.b
+                     Filter: ((plt1_l_p3_p1.a % 25) = 0)
+               ->  Hash
+                     Output: plt2_l_p3_p1.b, plt2_l_p3_p1.c, plt2_l_p3_p1.a
+                     ->  Seq Scan on public.plt2_l_p3_p1
+                           Output: plt2_l_p3_p1.b, plt2_l_p3_p1.c, plt2_l_p3_p1.a
+                           Filter: ((plt2_l_p3_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: plt1_l_p3_p2.a, plt1_l_p3_p2.c, plt2_l_p3_p2.b, plt2_l_p3_p2.c
+               Hash Cond: ((plt1_l_p3_p2.a = plt2_l_p3_p2.b) AND (plt1_l_p3_p2.b = plt2_l_p3_p2.a) AND ((plt1_l_p3_p2.c)::text = (plt2_l_p3_p2.c)::text) AND (ltrim((plt1_l_p3_p2.c)::text, 'A'::text) = ltrim((plt2_l_p3_p2.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt1_l_p3_p2
+                     Output: plt1_l_p3_p2.a, plt1_l_p3_p2.c, plt1_l_p3_p2.b
+                     Filter: ((plt1_l_p3_p2.a % 25) = 0)
+               ->  Hash
+                     Output: plt2_l_p3_p2.b, plt2_l_p3_p2.c, plt2_l_p3_p2.a
+                     ->  Seq Scan on public.plt2_l_p3_p2
+                           Output: plt2_l_p3_p2.b, plt2_l_p3_p2.c, plt2_l_p3_p2.a
+                           Filter: ((plt2_l_p3_p2.b % 25) = 0)
+(70 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_l WHERE plt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2_l WHERE plt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A')) ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 7) and (b 20) is considered for partition-wise join.
+NOTICE:  join between relations (b 11) and (b 24) is considered for partition-wise join.
+NOTICE:  join between relations (b 15) and (b 28) is considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  50 | 0001 |     | 
+ 100 | 0002 |     | 
+ 150 | 0003 | 150 | 0003
+ 200 | 0004 |     | 
+ 250 | 0005 |     | 
+ 300 | 0006 | 300 | 0006
+ 350 | 0007 |     | 
+ 400 | 0008 |     | 
+ 450 | 0009 | 450 | 0009
+ 500 | 0010 |     | 
+ 550 | 0011 |     | 
+     |      |  75 | 0001
+     |      | 225 | 0004
+     |      | 375 | 0007
+     |      | 525 | 0010
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uplt1_l t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2_l t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A')) ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  50 | 0001 |     | 
+ 100 | 0002 |     | 
+ 150 | 0003 | 150 | 0003
+ 200 | 0004 |     | 
+ 250 | 0005 |     | 
+ 300 | 0006 | 300 | 0006
+ 350 | 0007 |     | 
+ 400 | 0008 |     | 
+ 450 | 0009 | 450 | 0009
+ 500 | 0010 |     | 
+ 550 | 0011 |     | 
+     |      |  75 | 0001
+     |      | 225 | 0004
+     |      | 375 | 0007
+     |      | 525 | 0010
+(16 rows)
+
+-- lateral reference
+--Getting could not devise a query plan error, need to be fix to get query output
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM plt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.c,t2.c,t3.c) FROM plt1_l t2 JOIN plt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND ltrim(t2.c,'A') = ltrim(t3.c,'A'))) ss
+			  ON t1.a = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND ltrim(t1.c,'A') = ltrim(ss.t2c,'A') WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM plt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.c,t2.c,t3.c) FROM plt1_l t2 JOIN plt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND ltrim(t2.c,'A') = ltrim(t3.c,'A'))) ss
+			  ON t1.a = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND ltrim(t1.c,'A') = ltrim(ss.t2c,'A') WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t2c  | t2b | t3a | least 
+-----+-----+------+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 | 0000 |   0 |   0 | 0000
+  50 |  50 | 0001 |     |      |     |     | 
+ 100 | 100 | 0002 |     |      |     |     | 
+ 150 | 150 | 0003 | 150 | 0003 | 150 | 150 | 0003
+ 200 | 200 | 0004 |     |      |     |     | 
+ 250 | 250 | 0005 |     |      |     |     | 
+ 300 | 300 | 0006 | 300 | 0006 | 300 | 300 | 0006
+ 350 | 350 | 0007 |     |      |     |     | 
+ 400 | 400 | 0008 |     |      |     |     | 
+ 450 | 450 | 0009 | 450 | 0009 | 450 | 450 | 0009
+ 500 | 500 | 0010 |     |      |     |     | 
+ 550 | 550 | 0011 |     |      |     |     | 
+(12 rows)
+
+SELECT * FROM uplt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.c,t2.c,t3.c) FROM uplt1_l t2 JOIN uplt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND ltrim(t2.c,'A') = ltrim(t3.c,'A'))) ss
+			  ON t1.a = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND ltrim(t1.c,'A') = ltrim(ss.t2c,'A') WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t2c  | t2b | t3a | least 
+-----+-----+------+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 | 0000 |   0 |   0 | 0000
+  50 |  50 | 0001 |     |      |     |     | 
+ 100 | 100 | 0002 |     |      |     |     | 
+ 150 | 150 | 0003 | 150 | 0003 | 150 | 150 | 0003
+ 200 | 200 | 0004 |     |      |     |     | 
+ 250 | 250 | 0005 |     |      |     |     | 
+ 300 | 300 | 0006 | 300 | 0006 | 300 | 300 | 0006
+ 350 | 350 | 0007 |     |      |     |     | 
+ 400 | 400 | 0008 |     |      |     |     | 
+ 450 | 450 | 0009 | 450 | 0009 | 450 | 450 | 0009
+ 500 | 500 | 0010 |     |      |     |     | 
+ 550 | 550 | 0011 |     |      |     |     | 
+(12 rows)
+
+--Getting could not devise a query plan error, need to be fix to get query output
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM plt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.c,t2.c,t3.c) FROM plt1_l t2 JOIN plt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND ltrim(t2.c,'A') = ltrim(t3.c,'A'))) ss
+			  ON t1.b = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND ltrim(t1.c,'A') = ltrim(ss.t2c,'A') WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM plt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.c,t2.c,t3.c) FROM plt1_l t2 JOIN plt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND ltrim(t2.c,'A') = ltrim(t3.c,'A'))) ss
+			  ON t1.b = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND ltrim(t1.c,'A') = ltrim(ss.t2c,'A') WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t2c  | t2b | t3a | least 
+-----+-----+------+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 | 0000 |   0 |   0 | 0000
+  50 |  50 | 0001 |     |      |     |     | 
+ 100 | 100 | 0002 |     |      |     |     | 
+ 150 | 150 | 0003 | 150 | 0003 | 150 | 150 | 0003
+ 200 | 200 | 0004 |     |      |     |     | 
+ 250 | 250 | 0005 |     |      |     |     | 
+ 300 | 300 | 0006 | 300 | 0006 | 300 | 300 | 0006
+ 350 | 350 | 0007 |     |      |     |     | 
+ 400 | 400 | 0008 |     |      |     |     | 
+ 450 | 450 | 0009 | 450 | 0009 | 450 | 450 | 0009
+ 500 | 500 | 0010 |     |      |     |     | 
+ 550 | 550 | 0011 |     |      |     |     | 
+(12 rows)
+
+SELECT * FROM uplt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.c,t2.c,t3.c) FROM uplt1_l t2 JOIN uplt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND ltrim(t2.c,'A') = ltrim(t3.c,'A'))) ss
+			  ON t1.b = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND ltrim(t1.c,'A') = ltrim(ss.t2c,'A') WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t2c  | t2b | t3a | least 
+-----+-----+------+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 | 0000 |   0 |   0 | 0000
+  50 |  50 | 0001 |     |      |     |     | 
+ 100 | 100 | 0002 |     |      |     |     | 
+ 150 | 150 | 0003 | 150 | 0003 | 150 | 150 | 0003
+ 200 | 200 | 0004 |     |      |     |     | 
+ 250 | 250 | 0005 |     |      |     |     | 
+ 300 | 300 | 0006 | 300 | 0006 | 300 | 300 | 0006
+ 350 | 350 | 0007 |     |      |     |     | 
+ 400 | 400 | 0008 |     |      |     |     | 
+ 450 | 450 | 0009 | 450 | 0009 | 450 | 450 | 0009
+ 500 | 500 | 0010 |     |      |     |     | 
+ 550 | 550 | 0011 |     |      |     |     | 
+(12 rows)
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM (SELECT 50 phv, * FROM plt1_l WHERE plt1_l.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM plt2_l WHERE plt2_l.b % 25 = 0) t2 ON (t1.a = t2.b and t1.b = t2.a and t1.c = t2.c and ltrim(t2.c,'A') = ltrim(t1.c,'A')) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY 1,2,3,4,5,6,7,8;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 7) and (b 20) is considered for partition-wise join.
+NOTICE:  join between relations (b 11) and (b 24) is considered for partition-wise join.
+NOTICE:  join between relations (b 15) and (b 28) is considered for partition-wise join.
+                                                                                                                        QUERY PLAN                                                                                                                        
+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: (50), plt1_l_p1_p1.a, plt1_l_p1_p1.b, plt1_l_p1_p1.c, (75), plt2_l_p1_p1.a, plt2_l_p1_p1.b, plt2_l_p1_p1.c
+   Sort Key: (50), plt1_l_p1_p1.a, plt1_l_p1_p1.b, plt1_l_p1_p1.c, (75), plt2_l_p1_p1.a, plt2_l_p1_p1.b, plt2_l_p1_p1.c
+   ->  Result
+         Output: (50), plt1_l_p1_p1.a, plt1_l_p1_p1.b, plt1_l_p1_p1.c, (75), plt2_l_p1_p1.a, plt2_l_p1_p1.b, plt2_l_p1_p1.c
+         ->  Append
+               ->  Hash Full Join
+                     Output: plt1_l_p1_p1.a, plt1_l_p1_p1.b, plt1_l_p1_p1.c, plt2_l_p1_p1.a, plt2_l_p1_p1.b, plt2_l_p1_p1.c, (50), (75)
+                     Hash Cond: ((plt1_l_p1_p1.a = plt2_l_p1_p1.b) AND (plt1_l_p1_p1.b = plt2_l_p1_p1.a) AND ((plt1_l_p1_p1.c)::text = (plt2_l_p1_p1.c)::text) AND (ltrim((plt1_l_p1_p1.c)::text, 'A'::text) = ltrim((plt2_l_p1_p1.c)::text, 'A'::text)))
+                     Filter: (((50) = plt1_l_p1_p1.b) OR ((75) = plt2_l_p1_p1.b))
+                     ->  Seq Scan on public.plt1_l_p1_p1
+                           Output: plt1_l_p1_p1.a, plt1_l_p1_p1.b, plt1_l_p1_p1.c, 50
+                           Filter: ((plt1_l_p1_p1.a % 25) = 0)
+                     ->  Hash
+                           Output: plt2_l_p1_p1.a, plt2_l_p1_p1.b, plt2_l_p1_p1.c, (75)
+                           ->  Seq Scan on public.plt2_l_p1_p1
+                                 Output: plt2_l_p1_p1.a, plt2_l_p1_p1.b, plt2_l_p1_p1.c, 75
+                                 Filter: ((plt2_l_p1_p1.b % 25) = 0)
+               ->  Hash Full Join
+                     Output: plt1_l_p1_p2.a, plt1_l_p1_p2.b, plt1_l_p1_p2.c, plt2_l_p1_p2.a, plt2_l_p1_p2.b, plt2_l_p1_p2.c, (50), (75)
+                     Hash Cond: ((plt1_l_p1_p2.a = plt2_l_p1_p2.b) AND (plt1_l_p1_p2.b = plt2_l_p1_p2.a) AND ((plt1_l_p1_p2.c)::text = (plt2_l_p1_p2.c)::text) AND (ltrim((plt1_l_p1_p2.c)::text, 'A'::text) = ltrim((plt2_l_p1_p2.c)::text, 'A'::text)))
+                     Filter: (((50) = plt1_l_p1_p2.b) OR ((75) = plt2_l_p1_p2.b))
+                     ->  Seq Scan on public.plt1_l_p1_p2
+                           Output: plt1_l_p1_p2.a, plt1_l_p1_p2.b, plt1_l_p1_p2.c, 50
+                           Filter: ((plt1_l_p1_p2.a % 25) = 0)
+                     ->  Hash
+                           Output: plt2_l_p1_p2.a, plt2_l_p1_p2.b, plt2_l_p1_p2.c, (75)
+                           ->  Seq Scan on public.plt2_l_p1_p2
+                                 Output: plt2_l_p1_p2.a, plt2_l_p1_p2.b, plt2_l_p1_p2.c, 75
+                                 Filter: ((plt2_l_p1_p2.b % 25) = 0)
+               ->  Hash Full Join
+                     Output: plt1_l_p2_p1.a, plt1_l_p2_p1.b, plt1_l_p2_p1.c, plt2_l_p2_p1.a, plt2_l_p2_p1.b, plt2_l_p2_p1.c, (50), (75)
+                     Hash Cond: ((plt1_l_p2_p1.a = plt2_l_p2_p1.b) AND (plt1_l_p2_p1.b = plt2_l_p2_p1.a) AND ((plt1_l_p2_p1.c)::text = (plt2_l_p2_p1.c)::text) AND (ltrim((plt1_l_p2_p1.c)::text, 'A'::text) = ltrim((plt2_l_p2_p1.c)::text, 'A'::text)))
+                     Filter: (((50) = plt1_l_p2_p1.b) OR ((75) = plt2_l_p2_p1.b))
+                     ->  Seq Scan on public.plt1_l_p2_p1
+                           Output: plt1_l_p2_p1.a, plt1_l_p2_p1.b, plt1_l_p2_p1.c, 50
+                           Filter: ((plt1_l_p2_p1.a % 25) = 0)
+                     ->  Hash
+                           Output: plt2_l_p2_p1.a, plt2_l_p2_p1.b, plt2_l_p2_p1.c, (75)
+                           ->  Seq Scan on public.plt2_l_p2_p1
+                                 Output: plt2_l_p2_p1.a, plt2_l_p2_p1.b, plt2_l_p2_p1.c, 75
+                                 Filter: ((plt2_l_p2_p1.b % 25) = 0)
+               ->  Hash Full Join
+                     Output: plt1_l_p2_p2.a, plt1_l_p2_p2.b, plt1_l_p2_p2.c, plt2_l_p2_p2.a, plt2_l_p2_p2.b, plt2_l_p2_p2.c, (50), (75)
+                     Hash Cond: ((plt1_l_p2_p2.a = plt2_l_p2_p2.b) AND (plt1_l_p2_p2.b = plt2_l_p2_p2.a) AND ((plt1_l_p2_p2.c)::text = (plt2_l_p2_p2.c)::text) AND (ltrim((plt1_l_p2_p2.c)::text, 'A'::text) = ltrim((plt2_l_p2_p2.c)::text, 'A'::text)))
+                     Filter: (((50) = plt1_l_p2_p2.b) OR ((75) = plt2_l_p2_p2.b))
+                     ->  Seq Scan on public.plt1_l_p2_p2
+                           Output: plt1_l_p2_p2.a, plt1_l_p2_p2.b, plt1_l_p2_p2.c, 50
+                           Filter: ((plt1_l_p2_p2.a % 25) = 0)
+                     ->  Hash
+                           Output: plt2_l_p2_p2.a, plt2_l_p2_p2.b, plt2_l_p2_p2.c, (75)
+                           ->  Seq Scan on public.plt2_l_p2_p2
+                                 Output: plt2_l_p2_p2.a, plt2_l_p2_p2.b, plt2_l_p2_p2.c, 75
+                                 Filter: ((plt2_l_p2_p2.b % 25) = 0)
+               ->  Hash Full Join
+                     Output: plt1_l_p3_p1.a, plt1_l_p3_p1.b, plt1_l_p3_p1.c, plt2_l_p3_p1.a, plt2_l_p3_p1.b, plt2_l_p3_p1.c, (50), (75)
+                     Hash Cond: ((plt1_l_p3_p1.a = plt2_l_p3_p1.b) AND (plt1_l_p3_p1.b = plt2_l_p3_p1.a) AND ((plt1_l_p3_p1.c)::text = (plt2_l_p3_p1.c)::text) AND (ltrim((plt1_l_p3_p1.c)::text, 'A'::text) = ltrim((plt2_l_p3_p1.c)::text, 'A'::text)))
+                     Filter: (((50) = plt1_l_p3_p1.b) OR ((75) = plt2_l_p3_p1.b))
+                     ->  Seq Scan on public.plt1_l_p3_p1
+                           Output: plt1_l_p3_p1.a, plt1_l_p3_p1.b, plt1_l_p3_p1.c, 50
+                           Filter: ((plt1_l_p3_p1.a % 25) = 0)
+                     ->  Hash
+                           Output: plt2_l_p3_p1.a, plt2_l_p3_p1.b, plt2_l_p3_p1.c, (75)
+                           ->  Seq Scan on public.plt2_l_p3_p1
+                                 Output: plt2_l_p3_p1.a, plt2_l_p3_p1.b, plt2_l_p3_p1.c, 75
+                                 Filter: ((plt2_l_p3_p1.b % 25) = 0)
+               ->  Hash Full Join
+                     Output: plt1_l_p3_p2.a, plt1_l_p3_p2.b, plt1_l_p3_p2.c, plt2_l_p3_p2.a, plt2_l_p3_p2.b, plt2_l_p3_p2.c, (50), (75)
+                     Hash Cond: ((plt1_l_p3_p2.a = plt2_l_p3_p2.b) AND (plt1_l_p3_p2.b = plt2_l_p3_p2.a) AND ((plt1_l_p3_p2.c)::text = (plt2_l_p3_p2.c)::text) AND (ltrim((plt1_l_p3_p2.c)::text, 'A'::text) = ltrim((plt2_l_p3_p2.c)::text, 'A'::text)))
+                     Filter: (((50) = plt1_l_p3_p2.b) OR ((75) = plt2_l_p3_p2.b))
+                     ->  Seq Scan on public.plt1_l_p3_p2
+                           Output: plt1_l_p3_p2.a, plt1_l_p3_p2.b, plt1_l_p3_p2.c, 50
+                           Filter: ((plt1_l_p3_p2.a % 25) = 0)
+                     ->  Hash
+                           Output: plt2_l_p3_p2.a, plt2_l_p3_p2.b, plt2_l_p3_p2.c, (75)
+                           ->  Seq Scan on public.plt2_l_p3_p2
+                                 Output: plt2_l_p3_p2.a, plt2_l_p3_p2.b, plt2_l_p3_p2.c, 75
+                                 Filter: ((plt2_l_p3_p2.b % 25) = 0)
+(78 rows)
+
+SELECT * FROM (SELECT 50 phv, * FROM plt1_l WHERE plt1_l.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM plt2_l WHERE plt2_l.b % 25 = 0) t2 ON (t1.a = t2.b and t1.b = t2.a and t1.c = t2.c and ltrim(t2.c,'A') = ltrim(t1.c,'A')) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY 1,2,3,4,5,6,7,8;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 7) and (b 20) is considered for partition-wise join.
+NOTICE:  join between relations (b 11) and (b 24) is considered for partition-wise join.
+NOTICE:  join between relations (b 15) and (b 28) is considered for partition-wise join.
+ phv | a  | b  |  c   | phv | a  | b  |  c   
+-----+----+----+------+-----+----+----+------
+  50 | 50 | 50 | 0001 |     |    |    | 
+     |    |    |      |  75 | 75 | 75 | 0001
+(2 rows)
+
+SELECT * FROM (SELECT 50 phv, * FROM uplt1_l WHERE uplt1_l.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uplt2_l WHERE uplt2_l.b % 25 = 0) t2 ON (t1.a = t2.b and t1.b = t2.a and t1.c = t2.c and ltrim(t2.c,'A') = ltrim(t1.c,'A')) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY 1,2,3,4,5,6,7,8;
+ phv | a  | b  |  c   | phv | a  | b  |  c   
+-----+----+----+------+-----+----+----+------
+  50 | 50 | 50 | 0001 |     |    |    | 
+     |    |    |      |  75 | 75 | 75 | 0001
+(2 rows)
+
+-- Join with pruned partitions from joining relations
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_l t1, plt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') AND t1.a < 450 AND t2.b > 250 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 17) is considered for partition-wise join.
+NOTICE:  join between relations (b 8) and (b 21) is considered for partition-wise join.
+NOTICE:  join between relations (b 12) and (b 25) is considered for partition-wise join.
+                                                                                     QUERY PLAN                                                                                     
+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop
+               Output: t1.a, t1.c, t2.b, t2.c
+               Join Filter: ((t1.a = t2.b) AND (t1.b = t2.a) AND ((t1.c)::text = (t2.c)::text) AND (ltrim((t1.c)::text, 'A'::text) = ltrim((t2.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt1_l_p1_p1 t1
+                     Output: t1.a, t1.c, t1.b
+                     Filter: ((t1.a < 450) AND ((t1.a % 25) = 0))
+               ->  Seq Scan on public.plt2_l_p1_p1 t2
+                     Output: t2.b, t2.c, t2.a
+                     Filter: (t2.b > 250)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.a = t1_1.b) AND ((t2_1.c)::text = (t1_1.c)::text) AND (ltrim((t2_1.c)::text, 'A'::text) = ltrim((t1_1.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt2_l_p1_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+                     Filter: (t2_1.b > 250)
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.plt1_l_p1_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a < 450) AND ((t1_1.a % 25) = 0))
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.a = t1_2.b) AND ((t2_2.c)::text = (t1_2.c)::text) AND (ltrim((t2_2.c)::text, 'A'::text) = ltrim((t1_2.c)::text, 'A'::text)))
+               ->  Append
+                     ->  Seq Scan on public.plt2_l_p2 t2_2
+                           Output: t2_2.b, t2_2.c, t2_2.a
+                           Filter: (t2_2.b > 250)
+                     ->  Seq Scan on public.plt2_l_p2_p1 t2_3
+                           Output: t2_3.b, t2_3.c, t2_3.a
+                           Filter: (t2_3.b > 250)
+                     ->  Seq Scan on public.plt2_l_p2_p2 t2_4
+                           Output: t2_4.b, t2_4.c, t2_4.a
+                           Filter: (t2_4.b > 250)
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Append
+                           ->  Seq Scan on public.plt1_l_p2 t1_2
+                                 Output: t1_2.a, t1_2.c, t1_2.b
+                                 Filter: ((t1_2.a < 450) AND ((t1_2.a % 25) = 0))
+                           ->  Seq Scan on public.plt1_l_p2_p1 t1_3
+                                 Output: t1_3.a, t1_3.c, t1_3.b
+                                 Filter: ((t1_3.a < 450) AND ((t1_3.a % 25) = 0))
+                           ->  Seq Scan on public.plt1_l_p2_p2 t1_4
+                                 Output: t1_4.a, t1_4.c, t1_4.b
+                                 Filter: ((t1_4.a < 450) AND ((t1_4.a % 25) = 0))
+         ->  Hash Join
+               Output: t1_5.a, t1_5.c, t2_5.b, t2_5.c
+               Hash Cond: ((t2_5.b = t1_5.a) AND (t2_5.a = t1_5.b) AND ((t2_5.c)::text = (t1_5.c)::text) AND (ltrim((t2_5.c)::text, 'A'::text) = ltrim((t1_5.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt2_l_p3_p1 t2_5
+                     Output: t2_5.b, t2_5.c, t2_5.a
+                     Filter: (t2_5.b > 250)
+               ->  Hash
+                     Output: t1_5.a, t1_5.c, t1_5.b
+                     ->  Seq Scan on public.plt1_l_p3_p1 t1_5
+                           Output: t1_5.a, t1_5.c, t1_5.b
+                           Filter: ((t1_5.a < 450) AND ((t1_5.a % 25) = 0))
+         ->  Hash Join
+               Output: t1_6.a, t1_6.c, t2_6.b, t2_6.c
+               Hash Cond: ((t2_6.b = t1_6.a) AND (t2_6.a = t1_6.b) AND ((t2_6.c)::text = (t1_6.c)::text) AND (ltrim((t2_6.c)::text, 'A'::text) = ltrim((t1_6.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt2_l_p3_p2 t2_6
+                     Output: t2_6.b, t2_6.c, t2_6.a
+                     Filter: (t2_6.b > 250)
+               ->  Hash
+                     Output: t1_6.a, t1_6.c, t1_6.b
+                     ->  Seq Scan on public.plt1_l_p3_p2 t1_6
+                           Output: t1_6.a, t1_6.c, t1_6.b
+                           Filter: ((t1_6.a < 450) AND ((t1_6.a % 25) = 0))
+(71 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_l t1, plt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') AND t1.a < 450 AND t2.b > 250 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 1) and (b 2) is considered for partition-wise join.
+NOTICE:  join between relations (b 4) and (b 17) is considered for partition-wise join.
+NOTICE:  join between relations (b 8) and (b 21) is considered for partition-wise join.
+NOTICE:  join between relations (b 12) and (b 25) is considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0006 | 300 | 0006
+(1 row)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_l t1, uplt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') AND t1.a < 450 AND t2.b > 250 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0006 | 300 | 0006
+(1 row)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_l WHERE a < 450) t1 LEFT JOIN (SELECT * FROM plt2_l WHERE b > 250) t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 7) and (b 20) is considered for partition-wise join.
+NOTICE:  join between relations (b 11) and (b 24) is considered for partition-wise join.
+NOTICE:  join between relations (b 15) and (b 28) is considered for partition-wise join.
+                                                                                                                      QUERY PLAN                                                                                                                      
+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: plt1_l_p1_p1.a, plt1_l_p1_p1.c, plt2_l_p1_p1.b, plt2_l_p1_p1.c
+   Sort Key: plt1_l_p1_p1.a, plt2_l_p1_p1.b
+   ->  Append
+         ->  Nested Loop Left Join
+               Output: plt1_l_p1_p1.a, plt1_l_p1_p1.c, plt2_l_p1_p1.b, plt2_l_p1_p1.c
+               Join Filter: ((plt1_l_p1_p1.a = plt2_l_p1_p1.b) AND (plt1_l_p1_p1.b = plt2_l_p1_p1.a) AND ((plt1_l_p1_p1.c)::text = (plt2_l_p1_p1.c)::text) AND (ltrim((plt1_l_p1_p1.c)::text, 'A'::text) = ltrim((plt2_l_p1_p1.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt1_l_p1_p1
+                     Output: plt1_l_p1_p1.a, plt1_l_p1_p1.c, plt1_l_p1_p1.b
+                     Filter: ((plt1_l_p1_p1.a < 450) AND ((plt1_l_p1_p1.a % 25) = 0))
+               ->  Seq Scan on public.plt2_l_p1_p1
+                     Output: plt2_l_p1_p1.b, plt2_l_p1_p1.c, plt2_l_p1_p1.a
+                     Filter: (plt2_l_p1_p1.b > 250)
+         ->  Hash Right Join
+               Output: plt1_l_p1_p2.a, plt1_l_p1_p2.c, plt2_l_p1_p2.b, plt2_l_p1_p2.c
+               Hash Cond: ((plt2_l_p1_p2.b = plt1_l_p1_p2.a) AND (plt2_l_p1_p2.a = plt1_l_p1_p2.b) AND ((plt2_l_p1_p2.c)::text = (plt1_l_p1_p2.c)::text) AND (ltrim((plt2_l_p1_p2.c)::text, 'A'::text) = ltrim((plt1_l_p1_p2.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt2_l_p1_p2
+                     Output: plt2_l_p1_p2.b, plt2_l_p1_p2.c, plt2_l_p1_p2.a
+                     Filter: (plt2_l_p1_p2.b > 250)
+               ->  Hash
+                     Output: plt1_l_p1_p2.a, plt1_l_p1_p2.c, plt1_l_p1_p2.b
+                     ->  Seq Scan on public.plt1_l_p1_p2
+                           Output: plt1_l_p1_p2.a, plt1_l_p1_p2.c, plt1_l_p1_p2.b
+                           Filter: ((plt1_l_p1_p2.a < 450) AND ((plt1_l_p1_p2.a % 25) = 0))
+         ->  Hash Left Join
+               Output: plt1_l_p2_p1.a, plt1_l_p2_p1.c, plt2_l_p2_p1.b, plt2_l_p2_p1.c
+               Hash Cond: ((plt1_l_p2_p1.a = plt2_l_p2_p1.b) AND (plt1_l_p2_p1.b = plt2_l_p2_p1.a) AND ((plt1_l_p2_p1.c)::text = (plt2_l_p2_p1.c)::text) AND (ltrim((plt1_l_p2_p1.c)::text, 'A'::text) = ltrim((plt2_l_p2_p1.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt1_l_p2_p1
+                     Output: plt1_l_p2_p1.a, plt1_l_p2_p1.c, plt1_l_p2_p1.b
+                     Filter: ((plt1_l_p2_p1.a < 450) AND ((plt1_l_p2_p1.a % 25) = 0))
+               ->  Hash
+                     Output: plt2_l_p2_p1.b, plt2_l_p2_p1.c, plt2_l_p2_p1.a
+                     ->  Seq Scan on public.plt2_l_p2_p1
+                           Output: plt2_l_p2_p1.b, plt2_l_p2_p1.c, plt2_l_p2_p1.a
+                           Filter: (plt2_l_p2_p1.b > 250)
+         ->  Hash Right Join
+               Output: plt1_l_p2_p2.a, plt1_l_p2_p2.c, plt2_l_p2_p2.b, plt2_l_p2_p2.c
+               Hash Cond: ((plt2_l_p2_p2.b = plt1_l_p2_p2.a) AND (plt2_l_p2_p2.a = plt1_l_p2_p2.b) AND ((plt2_l_p2_p2.c)::text = (plt1_l_p2_p2.c)::text) AND (ltrim((plt2_l_p2_p2.c)::text, 'A'::text) = ltrim((plt1_l_p2_p2.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt2_l_p2_p2
+                     Output: plt2_l_p2_p2.b, plt2_l_p2_p2.c, plt2_l_p2_p2.a
+                     Filter: (plt2_l_p2_p2.b > 250)
+               ->  Hash
+                     Output: plt1_l_p2_p2.a, plt1_l_p2_p2.c, plt1_l_p2_p2.b
+                     ->  Seq Scan on public.plt1_l_p2_p2
+                           Output: plt1_l_p2_p2.a, plt1_l_p2_p2.c, plt1_l_p2_p2.b
+                           Filter: ((plt1_l_p2_p2.a < 450) AND ((plt1_l_p2_p2.a % 25) = 0))
+         ->  Hash Right Join
+               Output: plt1_l_p3_p1.a, plt1_l_p3_p1.c, plt2_l_p3_p1.b, plt2_l_p3_p1.c
+               Hash Cond: ((plt2_l_p3_p1.b = plt1_l_p3_p1.a) AND (plt2_l_p3_p1.a = plt1_l_p3_p1.b) AND ((plt2_l_p3_p1.c)::text = (plt1_l_p3_p1.c)::text) AND (ltrim((plt2_l_p3_p1.c)::text, 'A'::text) = ltrim((plt1_l_p3_p1.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt2_l_p3_p1
+                     Output: plt2_l_p3_p1.b, plt2_l_p3_p1.c, plt2_l_p3_p1.a
+                     Filter: (plt2_l_p3_p1.b > 250)
+               ->  Hash
+                     Output: plt1_l_p3_p1.a, plt1_l_p3_p1.c, plt1_l_p3_p1.b
+                     ->  Seq Scan on public.plt1_l_p3_p1
+                           Output: plt1_l_p3_p1.a, plt1_l_p3_p1.c, plt1_l_p3_p1.b
+                           Filter: ((plt1_l_p3_p1.a < 450) AND ((plt1_l_p3_p1.a % 25) = 0))
+         ->  Hash Right Join
+               Output: plt1_l_p3_p2.a, plt1_l_p3_p2.c, plt2_l_p3_p2.b, plt2_l_p3_p2.c
+               Hash Cond: ((plt2_l_p3_p2.b = plt1_l_p3_p2.a) AND (plt2_l_p3_p2.a = plt1_l_p3_p2.b) AND ((plt2_l_p3_p2.c)::text = (plt1_l_p3_p2.c)::text) AND (ltrim((plt2_l_p3_p2.c)::text, 'A'::text) = ltrim((plt1_l_p3_p2.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt2_l_p3_p2
+                     Output: plt2_l_p3_p2.b, plt2_l_p3_p2.c, plt2_l_p3_p2.a
+                     Filter: (plt2_l_p3_p2.b > 250)
+               ->  Hash
+                     Output: plt1_l_p3_p2.a, plt1_l_p3_p2.c, plt1_l_p3_p2.b
+                     ->  Seq Scan on public.plt1_l_p3_p2
+                           Output: plt1_l_p3_p2.a, plt1_l_p3_p2.c, plt1_l_p3_p2.b
+                           Filter: ((plt1_l_p3_p2.a < 450) AND ((plt1_l_p3_p2.a % 25) = 0))
+(68 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_l WHERE a < 450) t1 LEFT JOIN (SELECT * FROM plt2_l WHERE b > 250) t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 7) and (b 20) is considered for partition-wise join.
+NOTICE:  join between relations (b 11) and (b 24) is considered for partition-wise join.
+NOTICE:  join between relations (b 15) and (b 28) is considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |     | 
+  50 | 0001 |     | 
+ 100 | 0002 |     | 
+ 150 | 0003 |     | 
+ 200 | 0004 |     | 
+ 250 | 0005 |     | 
+ 300 | 0006 | 300 | 0006
+ 350 | 0007 |     | 
+ 400 | 0008 |     | 
+(9 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uplt1_l WHERE a < 450) t1 LEFT JOIN (SELECT * FROM uplt2_l WHERE b > 250) t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |     | 
+  50 | 0001 |     | 
+ 100 | 0002 |     | 
+ 150 | 0003 |     | 
+ 200 | 0004 |     | 
+ 250 | 0005 |     | 
+ 300 | 0006 | 300 | 0006
+ 350 | 0007 |     | 
+ 400 | 0008 |     | 
+(9 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_l WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM plt2_l WHERE b > 250) t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 5) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 20) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 24) and (b 11) is considered for partition-wise join.
+NOTICE:  join between relations (b 28) and (b 15) is considered for partition-wise join.
+                                                                                                                        QUERY PLAN                                                                                                                        
+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: plt1_l_p1_p1.a, plt1_l_p1_p1.c, plt2_l_p1_p1.b, plt2_l_p1_p1.c
+   Sort Key: plt1_l_p1_p1.a, plt2_l_p1_p1.b
+   ->  Result
+         Output: plt1_l_p1_p1.a, plt1_l_p1_p1.c, plt2_l_p1_p1.b, plt2_l_p1_p1.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: plt2_l_p1_p1.b, plt2_l_p1_p1.c, plt1_l_p1_p1.a, plt1_l_p1_p1.c
+                     Hash Cond: ((plt1_l_p1_p1.a = plt2_l_p1_p1.b) AND (plt1_l_p1_p1.b = plt2_l_p1_p1.a) AND ((plt1_l_p1_p1.c)::text = (plt2_l_p1_p1.c)::text) AND (ltrim((plt1_l_p1_p1.c)::text, 'A'::text) = ltrim((plt2_l_p1_p1.c)::text, 'A'::text)))
+                     ->  Seq Scan on public.plt1_l_p1_p1
+                           Output: plt1_l_p1_p1.a, plt1_l_p1_p1.c, plt1_l_p1_p1.b
+                           Filter: (plt1_l_p1_p1.a < 450)
+                     ->  Hash
+                           Output: plt2_l_p1_p1.b, plt2_l_p1_p1.c, plt2_l_p1_p1.a
+                           ->  Seq Scan on public.plt2_l_p1_p1
+                                 Output: plt2_l_p1_p1.b, plt2_l_p1_p1.c, plt2_l_p1_p1.a
+                                 Filter: ((plt2_l_p1_p1.b > 250) AND ((plt2_l_p1_p1.a % 25) = 0))
+               ->  Hash Right Join
+                     Output: plt2_l_p1_p2.b, plt2_l_p1_p2.c, plt1_l_p1_p2.a, plt1_l_p1_p2.c
+                     Hash Cond: ((plt1_l_p1_p2.a = plt2_l_p1_p2.b) AND (plt1_l_p1_p2.b = plt2_l_p1_p2.a) AND ((plt1_l_p1_p2.c)::text = (plt2_l_p1_p2.c)::text) AND (ltrim((plt1_l_p1_p2.c)::text, 'A'::text) = ltrim((plt2_l_p1_p2.c)::text, 'A'::text)))
+                     ->  Seq Scan on public.plt1_l_p1_p2
+                           Output: plt1_l_p1_p2.a, plt1_l_p1_p2.c, plt1_l_p1_p2.b
+                           Filter: (plt1_l_p1_p2.a < 450)
+                     ->  Hash
+                           Output: plt2_l_p1_p2.b, plt2_l_p1_p2.c, plt2_l_p1_p2.a
+                           ->  Seq Scan on public.plt2_l_p1_p2
+                                 Output: plt2_l_p1_p2.b, plt2_l_p1_p2.c, plt2_l_p1_p2.a
+                                 Filter: ((plt2_l_p1_p2.b > 250) AND ((plt2_l_p1_p2.a % 25) = 0))
+               ->  Hash Right Join
+                     Output: plt2_l_p2_p1.b, plt2_l_p2_p1.c, plt1_l_p2_p1.a, plt1_l_p2_p1.c
+                     Hash Cond: ((plt1_l_p2_p1.a = plt2_l_p2_p1.b) AND (plt1_l_p2_p1.b = plt2_l_p2_p1.a) AND ((plt1_l_p2_p1.c)::text = (plt2_l_p2_p1.c)::text) AND (ltrim((plt1_l_p2_p1.c)::text, 'A'::text) = ltrim((plt2_l_p2_p1.c)::text, 'A'::text)))
+                     ->  Seq Scan on public.plt1_l_p2_p1
+                           Output: plt1_l_p2_p1.a, plt1_l_p2_p1.c, plt1_l_p2_p1.b
+                           Filter: (plt1_l_p2_p1.a < 450)
+                     ->  Hash
+                           Output: plt2_l_p2_p1.b, plt2_l_p2_p1.c, plt2_l_p2_p1.a
+                           ->  Seq Scan on public.plt2_l_p2_p1
+                                 Output: plt2_l_p2_p1.b, plt2_l_p2_p1.c, plt2_l_p2_p1.a
+                                 Filter: ((plt2_l_p2_p1.b > 250) AND ((plt2_l_p2_p1.a % 25) = 0))
+               ->  Hash Right Join
+                     Output: plt2_l_p2_p2.b, plt2_l_p2_p2.c, plt1_l_p2_p2.a, plt1_l_p2_p2.c
+                     Hash Cond: ((plt1_l_p2_p2.a = plt2_l_p2_p2.b) AND (plt1_l_p2_p2.b = plt2_l_p2_p2.a) AND ((plt1_l_p2_p2.c)::text = (plt2_l_p2_p2.c)::text) AND (ltrim((plt1_l_p2_p2.c)::text, 'A'::text) = ltrim((plt2_l_p2_p2.c)::text, 'A'::text)))
+                     ->  Seq Scan on public.plt1_l_p2_p2
+                           Output: plt1_l_p2_p2.a, plt1_l_p2_p2.c, plt1_l_p2_p2.b
+                           Filter: (plt1_l_p2_p2.a < 450)
+                     ->  Hash
+                           Output: plt2_l_p2_p2.b, plt2_l_p2_p2.c, plt2_l_p2_p2.a
+                           ->  Seq Scan on public.plt2_l_p2_p2
+                                 Output: plt2_l_p2_p2.b, plt2_l_p2_p2.c, plt2_l_p2_p2.a
+                                 Filter: ((plt2_l_p2_p2.b > 250) AND ((plt2_l_p2_p2.a % 25) = 0))
+               ->  Hash Right Join
+                     Output: plt2_l_p3_p1.b, plt2_l_p3_p1.c, plt1_l_p3_p1.a, plt1_l_p3_p1.c
+                     Hash Cond: ((plt1_l_p3_p1.a = plt2_l_p3_p1.b) AND (plt1_l_p3_p1.b = plt2_l_p3_p1.a) AND ((plt1_l_p3_p1.c)::text = (plt2_l_p3_p1.c)::text) AND (ltrim((plt1_l_p3_p1.c)::text, 'A'::text) = ltrim((plt2_l_p3_p1.c)::text, 'A'::text)))
+                     ->  Seq Scan on public.plt1_l_p3_p1
+                           Output: plt1_l_p3_p1.a, plt1_l_p3_p1.c, plt1_l_p3_p1.b
+                           Filter: (plt1_l_p3_p1.a < 450)
+                     ->  Hash
+                           Output: plt2_l_p3_p1.b, plt2_l_p3_p1.c, plt2_l_p3_p1.a
+                           ->  Seq Scan on public.plt2_l_p3_p1
+                                 Output: plt2_l_p3_p1.b, plt2_l_p3_p1.c, plt2_l_p3_p1.a
+                                 Filter: ((plt2_l_p3_p1.b > 250) AND ((plt2_l_p3_p1.a % 25) = 0))
+               ->  Hash Right Join
+                     Output: plt2_l_p3_p2.b, plt2_l_p3_p2.c, plt1_l_p3_p2.a, plt1_l_p3_p2.c
+                     Hash Cond: ((plt1_l_p3_p2.a = plt2_l_p3_p2.b) AND (plt1_l_p3_p2.b = plt2_l_p3_p2.a) AND ((plt1_l_p3_p2.c)::text = (plt2_l_p3_p2.c)::text) AND (ltrim((plt1_l_p3_p2.c)::text, 'A'::text) = ltrim((plt2_l_p3_p2.c)::text, 'A'::text)))
+                     ->  Seq Scan on public.plt1_l_p3_p2
+                           Output: plt1_l_p3_p2.a, plt1_l_p3_p2.c, plt1_l_p3_p2.b
+                           Filter: (plt1_l_p3_p2.a < 450)
+                     ->  Hash
+                           Output: plt2_l_p3_p2.b, plt2_l_p3_p2.c, plt2_l_p3_p2.a
+                           ->  Seq Scan on public.plt2_l_p3_p2
+                                 Output: plt2_l_p3_p2.b, plt2_l_p3_p2.c, plt2_l_p3_p2.a
+                                 Filter: ((plt2_l_p3_p2.b > 250) AND ((plt2_l_p3_p2.a % 25) = 0))
+(72 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_l WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM plt2_l WHERE b > 250) t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 5) and (b 4) is considered for partition-wise join.
+NOTICE:  join between relations (b 20) and (b 7) is considered for partition-wise join.
+NOTICE:  join between relations (b 24) and (b 11) is considered for partition-wise join.
+NOTICE:  join between relations (b 28) and (b 15) is considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0006 | 300 | 0006
+     |      | 375 | 0007
+     |      | 450 | 0009
+     |      | 525 | 0010
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uplt1_l WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM uplt2_l WHERE b > 250) t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0006 | 300 | 0006
+     |      | 375 | 0007
+     |      | 450 | 0009
+     |      | 525 | 0010
+(4 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_l WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2_l WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 7) and (b 20) is considered for partition-wise join.
+NOTICE:  join between relations (b 11) and (b 24) is considered for partition-wise join.
+NOTICE:  join between relations (b 15) and (b 28) is considered for partition-wise join.
+                                                                                                                     QUERY PLAN                                                                                                                     
+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: plt1_l_p1_p1.a, plt1_l_p1_p1.c, plt2_l_p1_p1.b, plt2_l_p1_p1.c
+   Sort Key: plt1_l_p1_p1.a, plt2_l_p1_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: plt1_l_p1_p1.a, plt1_l_p1_p1.c, plt2_l_p1_p1.b, plt2_l_p1_p1.c
+               Hash Cond: ((plt1_l_p1_p1.a = plt2_l_p1_p1.b) AND (plt1_l_p1_p1.b = plt2_l_p1_p1.a) AND ((plt1_l_p1_p1.c)::text = (plt2_l_p1_p1.c)::text) AND (ltrim((plt1_l_p1_p1.c)::text, 'A'::text) = ltrim((plt2_l_p1_p1.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt1_l_p1_p1
+                     Output: plt1_l_p1_p1.a, plt1_l_p1_p1.c, plt1_l_p1_p1.b
+                     Filter: ((plt1_l_p1_p1.a < 450) AND ((plt1_l_p1_p1.a % 25) = 0))
+               ->  Hash
+                     Output: plt2_l_p1_p1.b, plt2_l_p1_p1.c, plt2_l_p1_p1.a
+                     ->  Seq Scan on public.plt2_l_p1_p1
+                           Output: plt2_l_p1_p1.b, plt2_l_p1_p1.c, plt2_l_p1_p1.a
+                           Filter: ((plt2_l_p1_p1.b > 250) AND ((plt2_l_p1_p1.b % 25) = 0))
+         ->  Hash Full Join
+               Output: plt1_l_p1_p2.a, plt1_l_p1_p2.c, plt2_l_p1_p2.b, plt2_l_p1_p2.c
+               Hash Cond: ((plt1_l_p1_p2.a = plt2_l_p1_p2.b) AND (plt1_l_p1_p2.b = plt2_l_p1_p2.a) AND ((plt1_l_p1_p2.c)::text = (plt2_l_p1_p2.c)::text) AND (ltrim((plt1_l_p1_p2.c)::text, 'A'::text) = ltrim((plt2_l_p1_p2.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt1_l_p1_p2
+                     Output: plt1_l_p1_p2.a, plt1_l_p1_p2.c, plt1_l_p1_p2.b
+                     Filter: ((plt1_l_p1_p2.a < 450) AND ((plt1_l_p1_p2.a % 25) = 0))
+               ->  Hash
+                     Output: plt2_l_p1_p2.b, plt2_l_p1_p2.c, plt2_l_p1_p2.a
+                     ->  Seq Scan on public.plt2_l_p1_p2
+                           Output: plt2_l_p1_p2.b, plt2_l_p1_p2.c, plt2_l_p1_p2.a
+                           Filter: ((plt2_l_p1_p2.b > 250) AND ((plt2_l_p1_p2.b % 25) = 0))
+         ->  Hash Full Join
+               Output: plt1_l_p2_p1.a, plt1_l_p2_p1.c, plt2_l_p2_p1.b, plt2_l_p2_p1.c
+               Hash Cond: ((plt1_l_p2_p1.a = plt2_l_p2_p1.b) AND (plt1_l_p2_p1.b = plt2_l_p2_p1.a) AND ((plt1_l_p2_p1.c)::text = (plt2_l_p2_p1.c)::text) AND (ltrim((plt1_l_p2_p1.c)::text, 'A'::text) = ltrim((plt2_l_p2_p1.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt1_l_p2_p1
+                     Output: plt1_l_p2_p1.a, plt1_l_p2_p1.c, plt1_l_p2_p1.b
+                     Filter: ((plt1_l_p2_p1.a < 450) AND ((plt1_l_p2_p1.a % 25) = 0))
+               ->  Hash
+                     Output: plt2_l_p2_p1.b, plt2_l_p2_p1.c, plt2_l_p2_p1.a
+                     ->  Seq Scan on public.plt2_l_p2_p1
+                           Output: plt2_l_p2_p1.b, plt2_l_p2_p1.c, plt2_l_p2_p1.a
+                           Filter: ((plt2_l_p2_p1.b > 250) AND ((plt2_l_p2_p1.b % 25) = 0))
+         ->  Hash Full Join
+               Output: plt1_l_p2_p2.a, plt1_l_p2_p2.c, plt2_l_p2_p2.b, plt2_l_p2_p2.c
+               Hash Cond: ((plt1_l_p2_p2.a = plt2_l_p2_p2.b) AND (plt1_l_p2_p2.b = plt2_l_p2_p2.a) AND ((plt1_l_p2_p2.c)::text = (plt2_l_p2_p2.c)::text) AND (ltrim((plt1_l_p2_p2.c)::text, 'A'::text) = ltrim((plt2_l_p2_p2.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt1_l_p2_p2
+                     Output: plt1_l_p2_p2.a, plt1_l_p2_p2.c, plt1_l_p2_p2.b
+                     Filter: ((plt1_l_p2_p2.a < 450) AND ((plt1_l_p2_p2.a % 25) = 0))
+               ->  Hash
+                     Output: plt2_l_p2_p2.b, plt2_l_p2_p2.c, plt2_l_p2_p2.a
+                     ->  Seq Scan on public.plt2_l_p2_p2
+                           Output: plt2_l_p2_p2.b, plt2_l_p2_p2.c, plt2_l_p2_p2.a
+                           Filter: ((plt2_l_p2_p2.b > 250) AND ((plt2_l_p2_p2.b % 25) = 0))
+         ->  Hash Full Join
+               Output: plt1_l_p3_p1.a, plt1_l_p3_p1.c, plt2_l_p3_p1.b, plt2_l_p3_p1.c
+               Hash Cond: ((plt1_l_p3_p1.a = plt2_l_p3_p1.b) AND (plt1_l_p3_p1.b = plt2_l_p3_p1.a) AND ((plt1_l_p3_p1.c)::text = (plt2_l_p3_p1.c)::text) AND (ltrim((plt1_l_p3_p1.c)::text, 'A'::text) = ltrim((plt2_l_p3_p1.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt1_l_p3_p1
+                     Output: plt1_l_p3_p1.a, plt1_l_p3_p1.c, plt1_l_p3_p1.b
+                     Filter: ((plt1_l_p3_p1.a < 450) AND ((plt1_l_p3_p1.a % 25) = 0))
+               ->  Hash
+                     Output: plt2_l_p3_p1.b, plt2_l_p3_p1.c, plt2_l_p3_p1.a
+                     ->  Seq Scan on public.plt2_l_p3_p1
+                           Output: plt2_l_p3_p1.b, plt2_l_p3_p1.c, plt2_l_p3_p1.a
+                           Filter: ((plt2_l_p3_p1.b > 250) AND ((plt2_l_p3_p1.b % 25) = 0))
+         ->  Hash Full Join
+               Output: plt1_l_p3_p2.a, plt1_l_p3_p2.c, plt2_l_p3_p2.b, plt2_l_p3_p2.c
+               Hash Cond: ((plt1_l_p3_p2.a = plt2_l_p3_p2.b) AND (plt1_l_p3_p2.b = plt2_l_p3_p2.a) AND ((plt1_l_p3_p2.c)::text = (plt2_l_p3_p2.c)::text) AND (ltrim((plt1_l_p3_p2.c)::text, 'A'::text) = ltrim((plt2_l_p3_p2.c)::text, 'A'::text)))
+               ->  Seq Scan on public.plt1_l_p3_p2
+                     Output: plt1_l_p3_p2.a, plt1_l_p3_p2.c, plt1_l_p3_p2.b
+                     Filter: ((plt1_l_p3_p2.a < 450) AND ((plt1_l_p3_p2.a % 25) = 0))
+               ->  Hash
+                     Output: plt2_l_p3_p2.b, plt2_l_p3_p2.c, plt2_l_p3_p2.a
+                     ->  Seq Scan on public.plt2_l_p3_p2
+                           Output: plt2_l_p3_p2.b, plt2_l_p3_p2.c, plt2_l_p3_p2.a
+                           Filter: ((plt2_l_p3_p2.b > 250) AND ((plt2_l_p3_p2.b % 25) = 0))
+(70 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_l WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2_l WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') ORDER BY t1.a, t2.b;
+NOTICE:  join between relations (b 4) and (b 5) is considered for partition-wise join.
+NOTICE:  join between relations (b 7) and (b 20) is considered for partition-wise join.
+NOTICE:  join between relations (b 11) and (b 24) is considered for partition-wise join.
+NOTICE:  join between relations (b 15) and (b 28) is considered for partition-wise join.
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |     | 
+  50 | 0001 |     | 
+ 100 | 0002 |     | 
+ 150 | 0003 |     | 
+ 200 | 0004 |     | 
+ 250 | 0005 |     | 
+ 300 | 0006 | 300 | 0006
+ 350 | 0007 |     | 
+ 400 | 0008 |     | 
+     |      | 375 | 0007
+     |      | 450 | 0009
+     |      | 525 | 0010
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uplt1_l WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2_l WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |     | 
+  50 | 0001 |     | 
+ 100 | 0002 |     | 
+ 150 | 0003 |     | 
+ 200 | 0004 |     | 
+ 250 | 0005 |     | 
+ 300 | 0006 | 300 | 0006
+ 350 | 0007 |     | 
+ 400 | 0008 |     | 
+     |      | 375 | 0007
+     |      | 450 | 0009
+     |      | 525 | 0010
+(12 rows)
+
+-- joins where one of the relations is proven empty
+--Getting cannot use column or expression from ancestor partition key error
+--for declarative partitioning code posted on 15 sep, and getting server crash
+--for partition-wise-join code posted on 2009, so need to generate output once
+--declarative code fixed.
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_l t1, plt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') AND t1.a = 1 AND t1.a = 2;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_l t1, plt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') AND t1.a = 1 AND t1.a = 2;
+ a | c | b | c 
+---+---+---+---
+(0 rows)
+
+--Getting cannot use column or expression from ancestor partition key error
+--for declarative partitioning code posted on 15 sep, and getting server crash
+--for partition-wise-join code posted on 2009, so need to generate output once
+--declarative code fixed.
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_l WHERE a = 1 AND a = 2) t1 LEFT JOIN plt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A');
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_l WHERE a = 1 AND a = 2) t1 LEFT JOIN plt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A');
+ a | c | b | c 
+---+---+---+---
+(0 rows)
+
+--Getting cannot use column or expression from ancestor partition key error
+--for declarative partitioning code posted on 15 sep, and getting server crash
+--for partition-wise-join code posted on 2009, so need to generate output once
+--declarative code fixed.
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_l WHERE a = 1 AND a = 2) t1 RIGHT JOIN plt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_l WHERE a = 1 AND a = 2) t1 RIGHT JOIN plt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+ a | c |  b  |  c   
+---+---+-----+------
+   |   |   0 | 0000
+   |   |  75 | 0001
+   |   | 150 | 0003
+   |   | 225 | 0004
+   |   | 300 | 0006
+   |   | 375 | 0007
+   |   | 450 | 0009
+   |   | 525 | 0010
+(8 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uplt1_l WHERE a = 1 AND a = 2) t1 RIGHT JOIN uplt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+ a | c |  b  |  c   
+---+---+-----+------
+   |   |   0 | 0000
+   |   |  75 | 0001
+   |   | 150 | 0003
+   |   | 225 | 0004
+   |   | 300 | 0006
+   |   | 375 | 0007
+   |   | 450 | 0009
+   |   | 525 | 0010
+(8 rows)
+
+--Getting cannot use column or expression from ancestor partition key error
+--for declarative partitioning code posted on 15 sep, and getting server crash
+--for partition-wise-join code posted on 2009, so need to generate output once
+--declarative code fixed.
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_l WHERE a = 1 AND a = 2) t1 FULL JOIN plt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_l WHERE a = 1 AND a = 2) t1 FULL JOIN plt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+ a | c |  b  |  c   
+---+---+-----+------
+   |   |   0 | 0000
+   |   |  75 | 0001
+   |   | 150 | 0003
+   |   | 225 | 0004
+   |   | 300 | 0006
+   |   | 375 | 0007
+   |   | 450 | 0009
+   |   | 525 | 0010
+(8 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uplt1_l WHERE a = 1 AND a = 2) t1 FULL JOIN uplt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+ a | c |  b  |  c   
+---+---+-----+------
+   |   |   0 | 0000
+   |   |  75 | 0001
+   |   | 150 | 0003
+   |   | 225 | 0004
+   |   | 300 | 0006
+   |   | 375 | 0007
+   |   | 450 | 0009
+   |   | 525 | 0010
+(8 rows)
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1_l t1 WHERE t1.c IN (SELECT t1.c FROM plt2_l t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 1) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 5) and (b 18) is considered for partition-wise join.
+NOTICE:  join between relations (b 9) and (b 22) is considered for partition-wise join.
+                             QUERY PLAN                              
+---------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: ((t1.c)::text = (t1_7.c)::text)
+               ->  Seq Scan on public.plt1_l_p1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_7.c
+                     ->  Seq Scan on public.plt2_l_p1_p1 t1_7
+                           Output: t1_7.c
+                           Filter: ((t1_7.b % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: ((t1_1.c)::text = (t1_8.c)::text)
+               ->  Seq Scan on public.plt1_l_p1_p2 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_8.c
+                     ->  Seq Scan on public.plt2_l_p1_p2 t1_8
+                           Output: t1_8.c
+                           Filter: ((t1_8.b % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: ((t1_2.c)::text = (t1_9.c)::text)
+               ->  Seq Scan on public.plt1_l_p2_p1 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_9.c
+                     ->  Seq Scan on public.plt2_l_p2_p1 t1_9
+                           Output: t1_9.c
+                           Filter: ((t1_9.b % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_3.a, t1_3.b, t1_3.c
+               Join Filter: ((t1_3.c)::text = (t1_10.c)::text)
+               ->  Seq Scan on public.plt1_l_p2_p2 t1_3
+                     Output: t1_3.a, t1_3.b, t1_3.c
+                     Filter: ((t1_3.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_10.c
+                     ->  Seq Scan on public.plt2_l_p2_p2 t1_10
+                           Output: t1_10.c
+                           Filter: ((t1_10.b % 25) = 0)
+         ->  Hash Semi Join
+               Output: t1_4.a, t1_4.b, t1_4.c
+               Hash Cond: ((t1_4.c)::text = (t1_11.c)::text)
+               ->  Append
+                     ->  Seq Scan on public.plt1_l_p3 t1_4
+                           Output: t1_4.a, t1_4.b, t1_4.c
+                           Filter: ((t1_4.a % 25) = 0)
+                     ->  Seq Scan on public.plt1_l_p3_p1 t1_5
+                           Output: t1_5.a, t1_5.b, t1_5.c
+                           Filter: ((t1_5.a % 25) = 0)
+                     ->  Seq Scan on public.plt1_l_p3_p2 t1_6
+                           Output: t1_6.a, t1_6.b, t1_6.c
+                           Filter: ((t1_6.a % 25) = 0)
+               ->  Hash
+                     Output: t1_11.c
+                     ->  Append
+                           ->  Seq Scan on public.plt2_l_p3 t1_11
+                                 Output: t1_11.c
+                                 Filter: ((t1_11.b % 25) = 0)
+                           ->  Seq Scan on public.plt2_l_p3_p1 t1_12
+                                 Output: t1_12.c
+                                 Filter: ((t1_12.b % 25) = 0)
+                           ->  Seq Scan on public.plt2_l_p3_p2 t1_13
+                                 Output: t1_13.c
+                                 Filter: ((t1_13.b % 25) = 0)
+(73 rows)
+
+SELECT t1.* FROM plt1_l t1 WHERE t1.c IN (SELECT t1.c FROM plt2_l t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+NOTICE:  join between relations (b 1) and (b 3) is considered for partition-wise join.
+NOTICE:  join between relations (b 5) and (b 18) is considered for partition-wise join.
+NOTICE:  join between relations (b 9) and (b 22) is considered for partition-wise join.
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+(8 rows)
+
+SELECT t1.* FROM uplt1_l t1 WHERE t1.c IN (SELECT t1.c FROM uplt2_l t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+(8 rows)
+
 --
 -- negative testcases
 --
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index 45e8a64..fa16354 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -467,6 +467,59 @@ SELECT * FROM uprt1_l t1 LEFT JOIN LATERAL
 			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1_l t2 JOIN uprt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
 			  ON t1.b = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
 
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM (SELECT 50 phv, * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b and t1.b = t2.a and t1.c = t2.c and t2.a + t2.b = t1.b + t1.a) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY 1,2,3,4,5,6,7,8;
+SELECT * FROM (SELECT 50 phv, * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b and t1.b = t2.a and t1.c = t2.c and t2.a + t2.b = t1.b + t1.a) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY 1,2,3,4,5,6,7,8;
+SELECT * FROM (SELECT 50 phv, * FROM uprt1_l WHERE uprt1_l.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2_l WHERE uprt2_l.b % 25 = 0) t2 ON (t1.a = t2.b and t1.b = t2.a and t1.c = t2.c and t2.a + t2.b = t1.b + t1.a) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY 1,2,3,4,5,6,7,8;
+
+-- Join with pruned partitions from joining relations
+EXPLAIN (VERBOSE, 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 = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a < 450 AND t2.b > 250 AND t1.a % 25 = 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 = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a < 450 AND t2.b > 250 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1, uprt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a < 450 AND t2.b > 250 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2_l WHERE b > 250) t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE a < 450) t1 LEFT JOIN (SELECT * FROM prt2_l WHERE b > 250) t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_l WHERE a < 450) t1 LEFT JOIN (SELECT * FROM uprt2_l WHERE b > 250) t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2_l WHERE b > 250) t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2_l WHERE b > 250) t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_l WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM uprt2_l WHERE b > 250) t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_l WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_l WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b ORDER BY t1.a, t2.b;
+
+-- joins where one of the relations is proven empty
+EXPLAIN (VERBOSE, 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 = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a = 1 AND t1.a = 2;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1, prt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a = 1 AND t1.a = 2;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE a = 1 AND a = 2) t1 LEFT JOIN prt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE a = 1 AND a = 2) t1 LEFT JOIN prt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b;
+
+EXPLAIN (VERBOSE, 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 AND t1.b + t1.a = t2.a + t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+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 AND t1.b + t1.a = t2.a + t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_l WHERE a = 1 AND a = 2) t1 RIGHT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE a = 1 AND a = 2) t1 FULL JOIN prt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE a = 1 AND a = 2) t1 FULL JOIN prt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_l WHERE a = 1 AND a = 2) t1 FULL JOIN uprt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1_l t1 WHERE t1.a IN (SELECT t1.b FROM prt2_l t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1_l t1 WHERE t1.a IN (SELECT t1.b FROM prt2_l t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1_l t1 WHERE t1.a IN (SELECT t1.b FROM uprt2_l t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+
 --
 -- tests for list partitioned tables.
 --
@@ -555,6 +608,48 @@ SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.b % 25
 SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
 SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
 
+-- joins where one of the relations is proven empty
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1, plt2 t2 WHERE t1.a = t2.b AND t1.a = 1 AND t1.a = 2;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1, plt2 t2 WHERE t1.a = t2.b AND t1.a = 1 AND t1.a = 2;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1 WHERE a = 1 AND a = 2) t1 LEFT JOIN plt2 t2 ON t1.a = t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1 WHERE a = 1 AND a = 2) t1 LEFT JOIN plt2 t2 ON t1.a = t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1 WHERE a = 1 AND a = 2) t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1 WHERE a = 1 AND a = 2) t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uplt1 WHERE a = 1 AND a = 2) t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1 WHERE a = 1 AND a = 2) t1 FULL JOIN plt2 t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1 WHERE a = 1 AND a = 2) t1 FULL JOIN plt2 t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uplt1 WHERE a = 1 AND a = 2) t1 FULL JOIN uplt2 t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+-- lateral reference
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT distinct * FROM plt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.c AS t2c, t3.c AS t3c, least(t1.c,t2.c,t3.c) FROM plt1 t2 JOIN plt2 t3 ON (t2.c = t3.c)) ss
+			  ON t1.c = ss.t2c WHERE t1.a % 25 = 0 ORDER BY 1,2,3,4,5,6;
+SELECT distinct * FROM plt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.c AS t2c, t3.c AS t3c, least(t1.c,t2.c,t3.c) FROM plt1 t2 JOIN plt2 t3 ON (t2.c = t3.c)) ss
+			  ON t1.c = ss.t2c WHERE t1.a % 25 = 0 ORDER BY 1,2,3,4,5,6;
+SELECT distinct * FROM uplt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.c AS t2c, t3.c AS t3c, least(t1.c,t2.c,t3.c) FROM uplt1 t2 JOIN uplt2 t3 ON (t2.c = t3.c)) ss
+			  ON t1.c = ss.t2c WHERE t1.a % 25 = 0 ORDER BY 1,2,3,4,5,6;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT distinct * FROM plt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.c AS t3c, least(t1.c,t2.c,t3.c) FROM plt1 t2 JOIN plt2 t3 ON (t2.c = t3.c)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY 1,2,3,4,5,6;
+SELECT distinct * FROM plt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.c AS t3c, least(t1.c,t2.c,t3.c) FROM plt1 t2 JOIN plt2 t3 ON (t2.c = t3.c)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY 1,2,3,4,5,6;
+SELECT distinct * FROM plt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.c AS t3c, least(t1.c,t2.c,t3.c) FROM plt1 t2 JOIN plt2 t3 ON (t2.c = t3.c)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY 1,2,3,4,5,6;
+
 --
 -- list partitioned by expression
 --
@@ -648,6 +743,13 @@ SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (
 SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
 SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
 
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.phv, t2.b, t2.phv, ltrim(t3.c,'A'), t3.phv FROM ((SELECT 50 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c)) FULL JOIN (SELECT '0002'::text phv, * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.c = ltrim(t3.c,'A')) WHERE t1.a = t1.phv OR t2.b = t2.phv OR ltrim(t3.c,'A') = t3.phv ORDER BY t1.a, t2.b, ltrim(t3.c,'A');
+SELECT t1.a, t1.phv, t2.b, t2.phv, ltrim(t3.c,'A'), t3.phv FROM ((SELECT 50 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c)) FULL JOIN (SELECT '0002'::text phv, * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.c = ltrim(t3.c,'A')) WHERE t1.a = t1.phv OR t2.b = t2.phv OR ltrim(t3.c,'A') = t3.phv ORDER BY t1.a, t2.b, ltrim(t3.c,'A');
+SELECT t1.a, t1.phv, t2.b, t2.phv, ltrim(t3.c,'A'), t3.phv FROM ((SELECT 50 phv, * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.c = t2.c)) FULL JOIN (SELECT '0002'::text phv, * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.c = ltrim(t3.c,'A')) WHERE t1.a = t1.phv OR t2.b = t2.phv OR ltrim(t3.c,'A') = t3.phv ORDER BY t1.a, t2.b, ltrim(t3.c,'A');
+
 -- test merge join with and without index scan
 CREATE INDEX iplt1_c on plt1(c);
 CREATE INDEX iplt1_p1_c on plt1_p1(c);
@@ -710,11 +812,182 @@ SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (
 SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
 SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
 
+-- lateral references and parameterized paths
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT distinct * FROM plt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.c AS t2c, t3.c AS t3c, least(t1.c,t2.c,t3.c) FROM plt1 t2 JOIN plt2 t3 ON (t2.c = t3.c)) ss
+			  ON t1.c = ss.t2c WHERE t1.a % 25 = 0 ORDER BY 1,2,3,4,5,6;
+SELECT distinct * FROM plt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.c AS t2c, t3.c AS t3c, least(t1.c,t2.c,t3.c) FROM plt1 t2 JOIN plt2 t3 ON (t2.c = t3.c)) ss
+			  ON t1.c = ss.t2c WHERE t1.a % 25 = 0 ORDER BY 1,2,3,4,5,6;
+SELECT distinct * FROM uplt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.c AS t2c, t3.c AS t3c, least(t1.c,t2.c,t3.c) FROM uplt1 t2 JOIN uplt2 t3 ON (t2.c = t3.c)) ss
+			  ON t1.c = ss.t2c WHERE t1.a % 25 = 0 ORDER BY 1,2,3,4,5,6;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT distinct * FROM plt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.c AS t3c, least(t1.c,t2.c,t3.c) FROM plt1 t2 JOIN plt2 t3 ON (t2.c = t3.c)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY 1,2,3,4,5,6;
+SELECT distinct * FROM plt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.c AS t3c, least(t1.c,t2.c,t3.c) FROM plt1 t2 JOIN plt2 t3 ON (t2.c = t3.c)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY 1,2,3,4,5,6;
+SELECT distinct * FROM uplt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.c AS t3c, least(t1.c,t2.c,t3.c) FROM uplt1 t2 JOIN uplt2 t3 ON (t2.c = t3.c)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY 1,2,3,4,5,6;
 RESET enable_hashjoin;
 RESET enable_nestloop;
 RESET enable_seqscan;
 
 --
+-- multi-leveled partitions
+--
+CREATE TABLE plt1_l (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt1_l_p1 PARTITION OF plt1_l FOR VALUES IN ('0000', '0003', '0004', '0010') PARTITION BY LIST (c);
+CREATE TABLE plt1_l_p1_p1 PARTITION OF plt1_l_p1 FOR VALUES IN ('0000', '0003');
+CREATE TABLE plt1_l_p1_p2 PARTITION OF plt1_l_p1 FOR VALUES IN ('0004', '0010');
+CREATE TABLE plt1_l_p2 PARTITION OF plt1_l FOR VALUES IN ('0001', '0005', '0002', '0009') PARTITION BY LIST (c);
+CREATE TABLE plt1_l_p2_p1 PARTITION OF plt1_l_p2 FOR VALUES IN ('0001', '0005');
+CREATE TABLE plt1_l_p2_p2 PARTITION OF plt1_l_p2 FOR VALUES IN ('0002', '0009');
+CREATE TABLE plt1_l_p3 PARTITION OF plt1_l FOR VALUES IN ('0006', '0007', '0008', '0011') PARTITION BY LIST (ltrim(c,'A'));
+CREATE TABLE plt1_l_p3_p1 PARTITION OF plt1_l_p3 FOR VALUES IN ('0006', '0007');
+CREATE TABLE plt1_l_p3_p2 PARTITION OF plt1_l_p3 FOR VALUES IN ('0008', '0011');
+INSERT INTO plt1_l SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE plt1_l;
+ANALYZE plt1_l_p1;
+ANALYZE plt1_l_p1_p1;
+ANALYZE plt1_l_p1_p2;
+ANALYZE plt1_l_p2;
+ANALYZE plt1_l_p2_p1;
+ANALYZE plt1_l_p2_p2;
+ANALYZE plt1_l_p3;
+ANALYZE plt1_l_p3_p1;
+ANALYZE plt1_l_p3_p2;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1_l AS SELECT * FROM plt1_l;
+
+CREATE TABLE plt2_l (a int, b int, c varchar) PARTITION BY LIST(c);
+CREATE TABLE plt2_l_p1 PARTITION OF plt2_l FOR VALUES IN ('0000', '0003', '0004', '0010') PARTITION BY LIST (c);
+CREATE TABLE plt2_l_p1_p1 PARTITION OF plt2_l_p1 FOR VALUES IN ('0000', '0003');
+CREATE TABLE plt2_l_p1_p2 PARTITION OF plt2_l_p1 FOR VALUES IN ('0004', '0010');
+CREATE TABLE plt2_l_p2 PARTITION OF plt2_l FOR VALUES IN ('0001', '0005', '0002', '0009') PARTITION BY LIST (c);
+CREATE TABLE plt2_l_p2_p1 PARTITION OF plt2_l_p2 FOR VALUES IN ('0001', '0005');
+CREATE TABLE plt2_l_p2_p2 PARTITION OF plt2_l_p2 FOR VALUES IN ('0002', '0009');
+CREATE TABLE plt2_l_p3 PARTITION OF plt2_l FOR VALUES IN ('0006', '0007', '0008', '0011') PARTITION BY LIST (ltrim(c,'A'));
+CREATE TABLE plt2_l_p3_p1 PARTITION OF plt2_l_p3 FOR VALUES IN ('0006', '0007');
+CREATE TABLE plt2_l_p3_p2 PARTITION OF plt2_l_p3 FOR VALUES IN ('0008', '0011');
+INSERT INTO plt2_l SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE plt2_l;
+ANALYZE plt2_l_p1;
+ANALYZE plt2_l_p1_p1;
+ANALYZE plt2_l_p1_p2;
+ANALYZE plt2_l_p2;
+ANALYZE plt2_l_p2_p1;
+ANALYZE plt2_l_p2_p2;
+ANALYZE plt2_l_p3;
+ANALYZE plt2_l_p3_p1;
+ANALYZE plt2_l_p3_p2;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt2_l AS SELECT * FROM plt2_l;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_l t1, plt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_l t1, plt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_l t1, uplt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_l t1 LEFT JOIN plt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_l t1 LEFT JOIN plt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_l t1 LEFT JOIN uplt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_l t1 RIGHT JOIN plt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_l t1 RIGHT JOIN plt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_l t1 RIGHT JOIN uplt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_l WHERE plt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2_l WHERE plt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A')) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_l WHERE plt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2_l WHERE plt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A')) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uplt1_l t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2_l t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A')) ORDER BY t1.a, t2.b;
+
+-- lateral reference
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM plt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.c,t2.c,t3.c) FROM plt1_l t2 JOIN plt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND ltrim(t2.c,'A') = ltrim(t3.c,'A'))) ss
+			  ON t1.a = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND ltrim(t1.c,'A') = ltrim(ss.t2c,'A') WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM plt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.c,t2.c,t3.c) FROM plt1_l t2 JOIN plt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND ltrim(t2.c,'A') = ltrim(t3.c,'A'))) ss
+			  ON t1.a = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND ltrim(t1.c,'A') = ltrim(ss.t2c,'A') WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uplt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.c,t2.c,t3.c) FROM uplt1_l t2 JOIN uplt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND ltrim(t2.c,'A') = ltrim(t3.c,'A'))) ss
+			  ON t1.a = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND ltrim(t1.c,'A') = ltrim(ss.t2c,'A') WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM plt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.c,t2.c,t3.c) FROM plt1_l t2 JOIN plt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND ltrim(t2.c,'A') = ltrim(t3.c,'A'))) ss
+			  ON t1.b = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND ltrim(t1.c,'A') = ltrim(ss.t2c,'A') WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM plt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.c,t2.c,t3.c) FROM plt1_l t2 JOIN plt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND ltrim(t2.c,'A') = ltrim(t3.c,'A'))) ss
+			  ON t1.b = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND ltrim(t1.c,'A') = ltrim(ss.t2c,'A') WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uplt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.c,t2.c,t3.c) FROM uplt1_l t2 JOIN uplt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND ltrim(t2.c,'A') = ltrim(t3.c,'A'))) ss
+			  ON t1.b = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND ltrim(t1.c,'A') = ltrim(ss.t2c,'A') WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM (SELECT 50 phv, * FROM plt1_l WHERE plt1_l.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM plt2_l WHERE plt2_l.b % 25 = 0) t2 ON (t1.a = t2.b and t1.b = t2.a and t1.c = t2.c and ltrim(t2.c,'A') = ltrim(t1.c,'A')) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY 1,2,3,4,5,6,7,8;
+SELECT * FROM (SELECT 50 phv, * FROM plt1_l WHERE plt1_l.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM plt2_l WHERE plt2_l.b % 25 = 0) t2 ON (t1.a = t2.b and t1.b = t2.a and t1.c = t2.c and ltrim(t2.c,'A') = ltrim(t1.c,'A')) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY 1,2,3,4,5,6,7,8;
+SELECT * FROM (SELECT 50 phv, * FROM uplt1_l WHERE uplt1_l.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uplt2_l WHERE uplt2_l.b % 25 = 0) t2 ON (t1.a = t2.b and t1.b = t2.a and t1.c = t2.c and ltrim(t2.c,'A') = ltrim(t1.c,'A')) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY 1,2,3,4,5,6,7,8;
+
+-- Join with pruned partitions from joining relations
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_l t1, plt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') AND t1.a < 450 AND t2.b > 250 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_l t1, plt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') AND t1.a < 450 AND t2.b > 250 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_l t1, uplt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') AND t1.a < 450 AND t2.b > 250 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_l WHERE a < 450) t1 LEFT JOIN (SELECT * FROM plt2_l WHERE b > 250) t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_l WHERE a < 450) t1 LEFT JOIN (SELECT * FROM plt2_l WHERE b > 250) t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uplt1_l WHERE a < 450) t1 LEFT JOIN (SELECT * FROM uplt2_l WHERE b > 250) t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_l WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM plt2_l WHERE b > 250) t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_l WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM plt2_l WHERE b > 250) t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uplt1_l WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM uplt2_l WHERE b > 250) t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_l WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2_l WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_l WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2_l WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uplt1_l WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2_l WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') ORDER BY t1.a, t2.b;
+
+-- joins where one of the relations is proven empty
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_l t1, plt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') AND t1.a = 1 AND t1.a = 2;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_l t1, plt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') AND t1.a = 1 AND t1.a = 2;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_l WHERE a = 1 AND a = 2) t1 LEFT JOIN plt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A');
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_l WHERE a = 1 AND a = 2) t1 LEFT JOIN plt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A');
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_l WHERE a = 1 AND a = 2) t1 RIGHT JOIN plt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_l WHERE a = 1 AND a = 2) t1 RIGHT JOIN plt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uplt1_l WHERE a = 1 AND a = 2) t1 RIGHT JOIN uplt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_l WHERE a = 1 AND a = 2) t1 FULL JOIN plt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_l WHERE a = 1 AND a = 2) t1 FULL JOIN plt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uplt1_l WHERE a = 1 AND a = 2) t1 FULL JOIN uplt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND ltrim(t1.c,'A') = ltrim(t2.c,'A') WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1_l t1 WHERE t1.c IN (SELECT t1.c FROM plt2_l t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM plt1_l t1 WHERE t1.c IN (SELECT t1.c FROM plt2_l t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uplt1_l t1 WHERE t1.c IN (SELECT t1.c FROM uplt2_l t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+--
 -- negative testcases
 --
 
#16Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#14)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Sep 22, 2016 at 6:41 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

[ new patch ]

This should probably get updated since Rajkumar reported a crash.
Meanwhile, here are some comments from an initial read-through:

+ * Multiple relations may be partitioned in the same way. The relations
+ * resulting from joining such relations may be partitioned in the same way as
+ * the joining relations.  Similarly, relations derived from such relations by
+ * grouping, sorting be partitioned in the same as the underlying relations.

I think you should change "may be partitioned in the same way" to "are
partitioned in the same way" or "can be regarded as partitioned in the
same way". The sentence that begins with "Similarly," is not
grammatical; it should say something like: ...by grouping or sorting
are partitioned in the same way as the underlying relations.

@@ -870,20 +902,21 @@ RelationBuildPartitionDesc(Relation rel)
result->bounds->rangeinfo = rangeinfo;
break;
}
}
}

MemoryContextSwitchTo(oldcxt);
rel->rd_partdesc = result;
}

+
/*
* Are two partition bound collections logically equal?
*
* Used in the keep logic of relcache.c (ie, in RelationClearRelation()).
* This is also useful when b1 and b2 are bound collections of two separate
* relations, respectively, because BoundCollection is a canonical
* representation of a set partition bounds (for given partitioning strategy).
*/
bool
partition_bounds_equal(PartitionKey key,

Spurious hunk.

+ * For an umpartitioned table, it returns NULL.

Spelling.

+ * two arguemnts and returns boolean. For types, it
suffices to match

Spelling.

+ * partition key expression is stored as a single member list to accomodate

Spelling.

+ * For a base relation, construct an array of partition key expressions. Each
+ * partition key expression is stored as a single member list to accomodate
+ * more partition keys when relations are joined.

How would joining relations result in more partitioning keys getting
added? Especially given the comment for the preceding function, which
says that a new PartitionScheme gets created unless an exact match is
found.

+ if (!lc)

Test lc == NIL instead of !lc.

+extern int
+PartitionSchemeGetNumParts(PartitionScheme part_scheme)
+{
+    return part_scheme ? part_scheme->nparts : 0;
+}

I'm not convinced it's a very good idea for this function to have
special handling for when part_scheme is NULL. In
try_partition_wise_join() that checks is not needed because it's
already been done, and in generate_partition_wise_join_paths it is
needed but only because you are initializing nparts too early. If you
move this initialization down below the IS_DUMMY_REL() check you won't
need the NULL guard. I would ditch this function and let the callers
access the structure member directly.

+extern int
+PartitionSchemeGetNumKeys(PartitionScheme part_scheme)
+{
+    return part_scheme ? part_scheme->partnatts : 0;
+}

Similarly here. have_partkey_equi_join should probably have a
quick-exit path when part_scheme is NULL, and then num_pks can be set
afterwards unconditionally. Same for match_expr_to_partition_keys.
build_joinrel_partition_info already has it and doesn't need this
double-check.

+extern Oid *
+PartitionDescGetPartOids(PartitionDesc part_desc)
+{
+    Oid       *part_oids;
+    int        cnt_parts;
+
+    if (!part_desc || part_desc->nparts <= 0)
+        return NULL;
+
+    part_oids = (Oid *) palloc(sizeof(Oid) * part_desc->nparts);
+    for (cnt_parts = 0; cnt_parts < part_desc->nparts; cnt_parts++)
+        part_oids[cnt_parts] = part_desc->oids[cnt_parts];
+
+    return part_oids;
+}

I may be missing something, but this looks like a bad idea in multiple
ways. First, you've got checks for part_desc's validity here that
should be in the caller, as noted above. Second, you're copying an
array by looping instead of using memcpy(). Third, the one and only
caller is set_append_rel_size, which doesn't seem to have any need to
copy this data in the first place. If there is any possibility that
the PartitionDesc is going to change under us while that function is
running, something is deeply broken. Nothing in the planner is going
to cope with the table structure changing under us, so it had better
not.

+    /*
+     * For a partitioned relation, we will save the child RelOptInfos in parent
+     * RelOptInfo in the same the order as corresponding bounds/lists are
+     * stored in the partition scheme.
+     */

This comment seems misplaced; shouldn't it be next to the code that is
actually doing this, rather than the code that is merely setting up
for it? And, also, the comment implies that we're doing this instead
of what we'd normally do, whereas I think we are actually doing
something additional.

+        /*
+         * Save topmost parent's relid. If the parent itself is a child of some
+         * other relation, use parent's topmost parent relids.
+         */
+        if (rel->top_parent_relids)
+            childrel->top_parent_relids = rel->top_parent_relids;
+        else
+            childrel->top_parent_relids = bms_copy(rel->relids);

Comment should explain why we're doing it, not what we're doing. The
comment as written just restates what anybody who's likely to be
looking at this can already see to be true from looking at the code
that follows. The question is why do it.

+    /* Set only for "other" base or join relations. */
+    Relids        top_parent_relids;

Comment should say what it is, not just when it's set.

+    /* Should have found all the childrels of a partitioned relation. */
+    if (rel->part_scheme)
+    {
+        int        cnt_parts;
+        for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+            Assert(rel->part_rels[cnt_parts]);
+    }

A block that does nothing but Assert() should be guarded by #ifdef
USE_ASSERT_CHECKING. Although, actually, maybe this should be an
elog(), just in case?

+    }
+
+    add_paths_to_append_rel(root, rel, live_childrels);
+}
+
+static void
+add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
+                        List *live_childrels)

The new function should have a header comment, which should include an
explanation of why this is now separate from
set_append_rel_pathlist().

+ if (!live_childrels)

As before, I think live_childrels == NIL is better style.

+ generate_partition_wise_join_paths(root, rel);

Needs an update to the comment earlier in the hunk. It's important to
explain why this has to be done here and not within
join_search_one_level.

+            /* Recursively collect the paths from child joinrel. */
+            generate_partition_wise_join_paths(root, child_rel);

Given the recursion, check_stack_depth() at top of function is
probably appropriate. Same for try_partition_wise_join().

+ if (live_children)
+ pfree(live_children);

Given that none of the substructure, including ListCells, will be
freed, this seems utterly pointless. If it's necessary to recover
memory here at all, we probably need to be more aggressive about it.
Have you tested the effect of this patch on planner memory consumption
with multi-way joins between tables with many partitions? If you
haven't, you probably should. (Testing runtime would be good, too.)
Does it grow linearly? Quadratically? Exponentially? Minor leaks
don't matter, but if we're generating too much garbage we'll have to
make sure it gets cleaned up soon enough to prevent runaway memory
usage.

     /*
+     * An inner path parameterized by the parent relation of outer
+     * relation needs to be reparameterized by the outer relation to be used
+     * for parameterized nested loop join.
+     */

No doubt, but I think the comment is missing the bigger picture -- it
doesn't say anything about this being here to support partition-wise
joins, which seems like a key point.

+        /* If we could not translate the path, don't produce nest loop path. */
+        if (!inner_path)
+            return;

Why would that ever happen?

+/*
+ * If the join between the given two relations can be executed as
+ * partition-wise join create the join relations for partition-wise join,
+ * create paths for those and then create append paths to combine
+ * partition-wise join results.
+ */
+static void
+try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
+                        RelOptInfo *joinrel, SpecialJoinInfo *parent_sjinfo,
+                        List *parent_restrictlist)

This comment doesn't accurately describe what the function does. No
append paths are created here; that happens at a much later stage. I
think this comment needs quite a bit more work, and maybe the function
should be renamed, too. There are really two steps involved here:
first, we create paths for each child, attached to a new RelOptInfo
flagged as RELOPT_OTHER_JOINREL paths; later, we create additional
paths for the parent RelOptInfo by appending a path for each child.

Broadly, I think there's a lack of adequate documentation of the
overall theory of operation of this patch. I believe that an update
to the optimizer README would be appropriate, probably with a new
section but maybe incorporating the new material into an existing
section. In addition, the comments for individual comments and chunks
of code need to do a better job explaining how each part of the patch
contributes to the overall picture. I also think we need to do a
better join hammering out the terminology. I don't particularly like
the term "partition-wise join" in the first place, although I don't
know what would be better, but we certainly need to avoid confusing a
partition-wise join -- which is a join performed by joining each
partition of one partitioned rel to the corresponding partition of a
similarly partitioned rel rather than by the usual execution strategy
of joining the parent rels -- with the concept of an other-join-rel,
which an other-member-rel analogue for joins. I don't think the patch
is currently very clear about this right now, either in the code or in
the comments. Maybe this function ought to be named something like
make_child_joins() or make_child_join_paths(), and we could use "child
joins" and/or "child join paths" as standard terminology throughout
the patch.

+    rel1_desc = makeStringInfo();
+    rel2_desc = makeStringInfo();
+
+    /* TODO: remove this notice when finalising the patch. */
+    outBitmapset(rel1_desc, rel1->relids);
+    outBitmapset(rel2_desc, rel2->relids);
+    elog(NOTICE, "join between relations %s and %s is considered for
partition-wise join.",
+         rel1_desc->data, rel2_desc->data);

Please remove your debugging cruft before submitting patches to
pgsql-hackers, or at least put #ifdef NOT_USED or something around it.

+     * We allocate the array for child RelOptInfos till we find at least one
+     * join order which can use partition-wise join technique. If no join order
+     * can use partition-wise join technique, there are no child relations.

This comment has problems. I think "till" is supposed to be "until",
and there's supposed to be a "don't" in there somewhere. But really,
I think what you're going for is just /* Allocate when first needed */
which would be a lot shorter and also more clear.

+     * Create join relations for the partition relations, if they do not exist
+     * already. Add paths to those for the given pair of joining relations.

I think the comment could be a bit more explanatory here. Something
like: "This joinrel is partitioned, so iterate over the partitions and
create paths for each one, allowing us to eventually build an
append-of-joins path for the parent. Since this routine may be called
multiple times for various join orders, the RelOptInfo needed for each
child join may or may not already exist, but the paths for this join
order definitely do not. Note that we don't create any actual
AppendPath at this stage; it only makes sense to do that at the end,
after each possible join order has been considered for each child
join. The best join order may differ from child to child."

+ * partiticipating in the given partition relations. We need them

Spelling.

+/*
+ * Construct the SpecialJoinInfo for the partition-wise join using parents'
+ * special join info. Also, instead of
+ * constructing an sjinfo everytime, we should probably save it in
+ * root->join_info_list and search within it like join_is_legal?
+ */

The lines here are of very different lengths for no particularly good
reason, and it should end with a period, not a question mark.

On the substance of the issue, it seems like the way you're doing this
right now could allocate a very large number of SpecialJoinInfo
structures. For every join relation, you'll create one
SpecialJoinInfo per legal join order per partition. That seems like
it could get to be a big number. I don't know if that's going to be a
problem from a memory-usage standpoint, but it seems like it might.
It's not just the SpecialJoinInfo itself; all of the substructure gets
duplicated, too.

+    SpecialJoinInfo *sjinfo = copyObject(parent_sjinfo);
+    sjinfo->min_lefthand = adjust_partition_relids(sjinfo->min_lefthand,
+                                                   append_rel_infos1);

Missing a blank line here.

+ AppendRelInfo *ari = lfirst(lc);

Standard naming convention for an AppendRelInfo variable seems to be
appinfo, not ari. (I just did "git grep AppendRelInfo".)

+        /* Skip non-equi-join clauses. */
+        if (!rinfo->can_join ||
+            rinfo->hashjoinoperator == InvalidOid ||
+            !rinfo->mergeopfamilies)
+            continue;

There's definitely something ugly about this. If rinfo->can_join is
false, then we're done. But suppose one of mergeopfamilies == NIL and
rinfo->hashoperator == InvalidOid is true and the other is false. Are
we really precluded from doing a partiion-wise join in that case, or
are we just prohibited from using certain join strategies? In most
places where we make similar tests, we're careful not to require more
than we need.

I also think that these tests need to consider the partitioning
operator in use. Suppose that the partition key is of a type T which
has two operator classes X and Y. Both relations are partitioned
using an operator from opfamily X, but the join condition mentions
opfamily Y. I'm pretty sure this precludes a partitionwise join. If
the join condition used opfamily X, then we would have a guarantee
that two rows which compared as equal would be in the same partition,
but because it uses opfamily Y, that's not guaranteed. For example,
if T is a text type, X might test for exact equality using "C"
collation rules, while Y might test for equality using some
case-insensitive set of rules. If the partition boundaries are such
that "foo" and "FOO" are in different partitions, a partitionwise join
using the case-insensitive operator will produce wrong results. You
can also imagine this happening with numeric, if you have one opclass
(like the default one) that considers 5.0 and 5.00 to be equal, but
another opclass that thinks they are different; if the latter is used
to set the partition bounds, 5.0 and 5.00 could end up in different
partitions - which will be fine if an operator from that opclass is
used for the join, but not if an operator from the regular opclass is
used.

After thinking this over a bit, I think the right way to think about this is:

1. Amit's patch currently only ever uses btree opfamilies for
partitioning. It uses those for both range partitioning and list
partitioning. If we ever support hash partitioning, we would
presumably use hash opfamilies for that purpose, but right now it's
all about btree opfamilies.

2. Therefore, if A and B are partitioned but the btree opfamilies
don't match, they don't have the same partitioning scheme and this
code should never be reached. Similarly, if they use the same
opfamily but different collations, the partitioning schemes shouldn't
match and therefore this code should not be reached.

3. If A and B are partitioned and the partitioning opfamilies - which
are necessarily btree opfamilies - do match, then the operator which
appears in the query needs to be from the same opfamily and have
amopstrategy of BTEqualStrategyNumber within that opfamily. If not,
then a partition-wise join is not possible.

4. Assuming the above conditions are met, have_partkey_equi_join
doesn't need to care whether the operator chosen has mergeopfamilies
or a valid hashjoinoperator. Those factors will control which join
methods are legal, but not whether a partitionwise join is possible in
principle.

Let me know whether that seems right.

+ * RelabelType node; eval_const_expressions() will have simplied if more

Spelling.

     /*
+     * Code below scores equivalence classes by how many equivalence members
+     * can produce join clauses for this join relation. Equivalence members
+     * which do not cover the parents of a partition-wise join relation, can
+     * produce join clauses for partition-wise join relation.
+     */

I don't know what that means. The comma in the second sentence
doesn't belong there.

+    /*
+     * TODO: Instead of copying and mutating the trees one child relation at a
+     * time, we should be able to do this en-masse for all the partitions
+     * involved.
+     */

I don't see how that would be possible, but if it's a TODO, you'd
better do it (or decide not to do it and remove or change the
comment).

     /*
      * Create explicit sort nodes for the outer and inner paths if necessary.
      */
     if (best_path->outersortkeys)
     {
+        Relids        outer_relids = outer_path->parent->relids;
         Sort       *sort = make_sort_from_pathkeys(outer_plan,
-                                                   best_path->outersortkeys);
+                                                   best_path->outersortkeys,
+                                                   outer_relids);

The changes related to make_sort_from_pathkeys() are pretty opaque to
me. Can you explain?

+ * Change parameterization of sub paths recursively. Also carry out any

"sub paths" should not be two words, here or anywhere.

+reparameterize_path_for_child(PlannerInfo *root, Path *path,
+                              RelOptInfo *child_rel)

This is suspiciously unlike reparameterize_path. Why?

+    /* Computer information relevant to the foreign relations. */
+    set_foreign_rel_properties(joinrel, outer_rel, inner_rel);

Perhaps this refactoring could be split out into a preliminary patch,
which would then simplify this patch. And same for add_join_rel().

+     * Produce partition-wise joinrel's targetlist by translating the parent
+     * joinrel's targetlist. This will also include the required placeholder

Again the confusion between a "child" join and a partition-wise join...

+    /*
+     * Nothing to do if
+     * a. partition-wise join is disabled.
+     * b. joining relations are not partitioned.
+     * c. partitioning schemes do not match.
+     */
+

I don't think that's going to survive pgindent.

+ * are not considered equal, an equi-join involing inner partition keys

Spelling.

+     * Collect the partition key expressions. An OUTER join will produce rows
+     * where the partition key columns of inner side are NULL and may not fit
+     * the partitioning scheme with inner partition keys. Since two NULL values
+     * are not considered equal, an equi-join involing inner partition keys
+     * still prohibits cross-partition joins while joining with another
+     * similarly partitioned relation.

I can't figure out what this comment is trying to tell me. Possibly I
just need more caffeine.

+ * Adding these two join_rel_level list also means that top level list has more
+ * than one join relation, which is symantically incorrect.

I don't understand this, either; also, spelling.

As a general comment, the ratio of tests-to-code in this patch is way
out of line with PostgreSQL's normal practices. The total patch file
is 10965 lines. The test cases begin at line 3047, meaning that in
round figures you've got about one-quarter code and about
three-quarters test cases. I suspect that a large fraction of those
test cases aren't adding any meaningful code coverage and will just
take work to maintain. That needs to be slimmed down substantially in
any version of this considered for commit.

--
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

#17Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#16)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Sep 28, 2016 at 2:02 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Thu, Sep 22, 2016 at 6:41 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

[ new patch ]

This should probably get updated since Rajkumar reported a crash.
Meanwhile, here are some comments from an initial read-through:

Done. Fixed those crashes. Also fixed some crashes in foreign table
code and postgres_fdw. The tests were provided by Rajkumar. I am
working on including those in my patch. The attached patch is still
based on Amit's patches set of patches posted on 15th Sept. 2016. He
is addressing your comments on his patches. So, I am expecting a more
stable version arrive soon. I will rebase my patches then. Because of
a bug in those patches related to multi-level partitioned tables and
lateral joins and also a restriction on sharing partition keys across
levels of partitions, the testcase is still failing. I will work on
that while rebasing the patch.

+ * Multiple relations may be partitioned in the same way. The relations
+ * resulting from joining such relations may be partitioned in the same way as
+ * the joining relations.  Similarly, relations derived from such relations by
+ * grouping, sorting be partitioned in the same as the underlying relations.

I think you should change "may be partitioned in the same way" to "are
partitioned in the same way" or "can be regarded as partitioned in the
same way".

The relations resulting from joining partitioned relations are
partitioned in the same way, if there exist equi-join condition/s
between their partition keys. If such equi-joins do not exist, the
join is *not* partitioned. Hence I did not use "are" or "can be" which
indicate a certainty. Instead I used "may" which indicates
"uncertainty". I am not sure whether that's a good place to explain
the conditions under which such relations are partitioned. Those
conditions will change as we implement more and more partition-wise
join strategies. But that comment conveys two things 1. partition
scheme makes sense for all kinds of relations 2. multiple relations
(of any kind) may share partition scheme. I have slightly changed the
wording to make this point clear. Please let me know if it looks
better.

The sentence that begins with "Similarly," is not
grammatical; it should say something like: ...by grouping or sorting
are partitioned in the same way as the underlying relations.

Done. Instead of "are" I have used "may" for the same reason as above.

@@ -870,20 +902,21 @@ RelationBuildPartitionDesc(Relation rel)
result->bounds->rangeinfo = rangeinfo;
break;
}
}
}

MemoryContextSwitchTo(oldcxt);
rel->rd_partdesc = result;
}

+
/*
* Are two partition bound collections logically equal?
*
* Used in the keep logic of relcache.c (ie, in RelationClearRelation()).
* This is also useful when b1 and b2 are bound collections of two separate
* relations, respectively, because BoundCollection is a canonical
* representation of a set partition bounds (for given partitioning strategy).
*/
bool
partition_bounds_equal(PartitionKey key,

Spurious hunk.

Thanks. Done.

+ * For an umpartitioned table, it returns NULL.

Spelling.

Done. Thanks.

+ * two arguemnts and returns boolean. For types, it
suffices to match

Spelling.

Thanks. Done.

+ * partition key expression is stored as a single member list to accomodate

Spelling.

Thanks. Done.

+ * For a base relation, construct an array of partition key expressions. Each
+ * partition key expression is stored as a single member list to accomodate
+ * more partition keys when relations are joined.

How would joining relations result in more partitioning keys getting
added? Especially given the comment for the preceding function, which
says that a new PartitionScheme gets created unless an exact match is
found.

Let's assume that relation A and B are partitioned by columns a and b
resp. and have same partitioning scheme. This means that the datatypes
of a and b as well as the opclass used for comparing partition key
values of A and B are same. A join between A and B with condition A.a
= B.b is partitioned by both A.a and B.b. We need to keep track of
both the keys in case AB joins with C which is partitioned in the same
manner. I guess, the confusion is with the term "partition keys" -
which is being used to indicate the class of partition key as well as
instance of partition key. In the above example, the datatype of
partition key and the opclass together indicate partition key class
whereas A.a and B.b are instances of that class. Increase in partition
keys may mean both increase in the number of classes or increase in
the number of instances. In the above comment I used to mean number of
instances. May be we should use "partition key expressions" to
indicate the partition key instances and "partition key" to indicate
partition key class. I have changed the comments to use partition keys
and partition key expressions appropriately. Please let me know if the
comments are worded correctly.

PartitionScheme does not hold the actual partition key expressions. It
holds the partition key type and opclass used for comparison, which
should be same for all the relations sharing the partition scheme.

+ if (!lc)

Test lc == NIL instead of !lc.

NIL is defined as (List *) NULL and lc is ListCell *. So changed the
test to lc == NULL instead of !lc.

+extern int
+PartitionSchemeGetNumParts(PartitionScheme part_scheme)
+{
+    return part_scheme ? part_scheme->nparts : 0;
+}

I'm not convinced it's a very good idea for this function to have
special handling for when part_scheme is NULL. In
try_partition_wise_join() that checks is not needed because it's
already been done, and in generate_partition_wise_join_paths it is
needed but only because you are initializing nparts too early. If you
move this initialization down below the IS_DUMMY_REL() check you won't
need the NULL guard. I would ditch this function and let the callers
access the structure member directly.

+extern int
+PartitionSchemeGetNumKeys(PartitionScheme part_scheme)
+{
+    return part_scheme ? part_scheme->partnatts : 0;
+}

Similarly here. have_partkey_equi_join should probably have a
quick-exit path when part_scheme is NULL, and then num_pks can be set
afterwards unconditionally. Same for match_expr_to_partition_keys.
build_joinrel_partition_info already has it and doesn't need this
double-check.

+extern Oid *
+PartitionDescGetPartOids(PartitionDesc part_desc)
+{
+    Oid       *part_oids;
+    int        cnt_parts;
+
+    if (!part_desc || part_desc->nparts <= 0)
+        return NULL;
+
+    part_oids = (Oid *) palloc(sizeof(Oid) * part_desc->nparts);
+    for (cnt_parts = 0; cnt_parts < part_desc->nparts; cnt_parts++)
+        part_oids[cnt_parts] = part_desc->oids[cnt_parts];
+
+    return part_oids;
+}

I may be missing something, but this looks like a bad idea in multiple
ways. First, you've got checks for part_desc's validity here that
should be in the caller, as noted above. Second, you're copying an
array by looping instead of using memcpy(). Third, the one and only
caller is set_append_rel_size, which doesn't seem to have any need to
copy this data in the first place. If there is any possibility that
the PartitionDesc is going to change under us while that function is
running, something is deeply broken. Nothing in the planner is going
to cope with the table structure changing under us, so it had better
not.

These three functions were written based on Amit Langote's patches
which did not expose partition related structures outside partition.c.
Hence they required wrappers. I have moved PartitionSchemeData to
partition.h and removed these functions. Instead the members are
accessed directly.

+    /*
+     * For a partitioned relation, we will save the child RelOptInfos in parent
+     * RelOptInfo in the same the order as corresponding bounds/lists are
+     * stored in the partition scheme.
+     */

This comment seems misplaced; shouldn't it be next to the code that is
actually doing this, rather than the code that is merely setting up
for it? And, also, the comment implies that we're doing this instead
of what we'd normally do, whereas I think we are actually doing
something additional.

Ok. I have moved the comment few line below, near the code which saves
the partition RelOptInfos.

+        /*
+         * Save topmost parent's relid. If the parent itself is a child of some
+         * other relation, use parent's topmost parent relids.
+         */
+        if (rel->top_parent_relids)
+            childrel->top_parent_relids = rel->top_parent_relids;
+        else
+            childrel->top_parent_relids = bms_copy(rel->relids);

Comment should explain why we're doing it, not what we're doing. The
comment as written just restates what anybody who's likely to be
looking at this can already see to be true from looking at the code
that follows. The question is why do it.

The point of that comment is to explain how it percolates down the
hierarchy, which is not so clear from the code. I have changed it to
read
/*
* Recursively save topmost parent's relid in RelOptInfos of
* partitions.
*/

Or you are expecting that the comment to explain the purpose of
top_parent_relids? I don't think that's a good idea, since the purpose
will change over the time and the comment will soon be out of sync
with the actual code, unless the developers expanding the usage
remember to update the comment. I have not seen the comments,
explaining purpose, next to the assignments. Take for example
RelOptInfo::relids.

+    /* Set only for "other" base or join relations. */
+    Relids        top_parent_relids;

Comment should say what it is, not just when it's set.

Done. Check if it looks good.

+    /* Should have found all the childrels of a partitioned relation. */
+    if (rel->part_scheme)
+    {
+        int        cnt_parts;
+        for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+            Assert(rel->part_rels[cnt_parts]);
+    }

A block that does nothing but Assert() should be guarded by #ifdef
USE_ASSERT_CHECKING. Although, actually, maybe this should be an
elog(), just in case?

Changed it to elog().

+    }
+
+    add_paths_to_append_rel(root, rel, live_childrels);
+}
+
+static void
+add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
+                        List *live_childrels)

The new function should have a header comment, which should include an
explanation of why this is now separate from
set_append_rel_pathlist().

Sorry for missing it. Added the prologue. Let me know, if it looks
good. I have made sure that all functions have a prologue and tried to
match the style with surrounding functions. Let me know if I have
still missed any or the styles do not match.

+ if (!live_childrels)

As before, I think live_childrels == NIL is better style.

Fixed.

+ generate_partition_wise_join_paths(root, rel);

Needs an update to the comment earlier in the hunk. It's important to
explain why this has to be done here and not within
join_search_one_level.

Thanks for pointing that out. Similar to generate_gather_paths(), we
need to add explanation in standard_join_search() as well as in the
function prologue. Did that. Let me know if it looks good.

+            /* Recursively collect the paths from child joinrel. */
+            generate_partition_wise_join_paths(root, child_rel);

Given the recursion, check_stack_depth() at top of function is
probably appropriate. Same for try_partition_wise_join().

Done. I wouldn't imagine a user creating that many levels of
partitions, but it's good to guard against some automated script that
has gone berserk.

+ if (live_children)
+ pfree(live_children);

Given that none of the substructure, including ListCells, will be
freed, this seems utterly pointless. If it's necessary to recover
memory here at all, we probably need to be more aggressive about it.

I intended to use list_free() instead of pfree(). Fixed that.

Have you tested the effect of this patch on planner memory consumption
with multi-way joins between tables with many partitions? If you
haven't, you probably should. (Testing runtime would be good, too.)
Does it grow linearly? Quadratically? Exponentially? Minor leaks
don't matter, but if we're generating too much garbage we'll have to
make sure it gets cleaned up soon enough to prevent runaway memory
usage.

I tried to check memory usage with various combinations of number of
partitions and number of relations being joined. For higher number of
relations being joined like 10 with 100 partitions, OOM killer kicked
in during the planning phase. I am suspecting
adjust_partitionrel_attrs() (changed that name to
adjust_join_appendrel_attrs() to be in sync with
adjust_appendrel_attrs()) to be the culprit. It copies expression
trees every time for joining two children. That's an exponentially
increasing number as the number of legal joins increases
exponentially. I am still investigating this.

As a side question, do we have a function to free an expression tree?
I didn't find any.

/*
+     * An inner path parameterized by the parent relation of outer
+     * relation needs to be reparameterized by the outer relation to be used
+     * for parameterized nested loop join.
+     */

No doubt, but I think the comment is missing the bigger picture -- it
doesn't say anything about this being here to support partition-wise
joins, which seems like a key point.

I have tried to explain the partition-wise join context. Let me know
if it looks good.

+        /* If we could not translate the path, don't produce nest loop path. */
+        if (!inner_path)
+            return;

Why would that ever happen?

Right now, reparameterize_path_for_child() does not support all kinds
of paths. So I have added that condition. I will add support for more
path types there once we agree that this is the right way to translate
the paths and that the path translation is required.

+/*
+ * If the join between the given two relations can be executed as
+ * partition-wise join create the join relations for partition-wise join,
+ * create paths for those and then create append paths to combine
+ * partition-wise join results.
+ */
+static void
+try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
+                        RelOptInfo *joinrel, SpecialJoinInfo *parent_sjinfo,
+                        List *parent_restrictlist)

This comment doesn't accurately describe what the function does. No
append paths are created here; that happens at a much later stage.

Removed reference to the append paths. Sorry for leaving it there,
when I moved the append path creation to a later stage.

I
think this comment needs quite a bit more work, and maybe the function
should be renamed, too.

Improved the comments in the prologue and inside the function. Please
let me know, if they look good.

There are really two steps involved here:
first, we create paths for each child, attached to a new RelOptInfo
flagged as RELOPT_OTHER_JOINREL paths; later, we create additional
paths for the parent RelOptInfo by appending a path for each child.

Right, the first one is done in try_partition_wise_join() and the
later is done in generate_partition_wise_join_paths()

Broadly, I think there's a lack of adequate documentation of the
overall theory of operation of this patch. I believe that an update
to the optimizer README would be appropriate, probably with a new
section but maybe incorporating the new material into an existing
section.

Done. I have added a separate section to optimizer/README

In addition, the comments for individual comments and chunks
of code need to do a better job explaining how each part of the patch
contributes to the overall picture.

I also think we need to do a
better join hammering out the terminology. I don't particularly like
the term "partition-wise join" in the first place, although I don't
know what would be better, but we certainly need to avoid confusing a
partition-wise join -- which is a join performed by joining each
partition of one partitioned rel to the corresponding partition of a
similarly partitioned rel rather than by the usual execution strategy
of joining the parent rels -- with the concept of an other-join-rel,
which an other-member-rel analogue for joins. I don't think the patch
is currently very clear about this right now, either in the code or in
the comments. Maybe this function ought to be named something like
make_child_joins() or make_child_join_paths(), and we could use "child
joins" and/or "child join paths" as standard terminology throughout
the patch.

Partition-wise join is widely used term in the literature. Other
DBMSes use the same term as well. So, I think we should stick with
"partition-wise join". Partition-wise join as you have described is a
join performed by joining each partition of one partitioned rel to the
corresponding partition of a similarly partitioned rel rather than by
the usual execution strategy of joining the parent rels. I have
usually used the term "partition-wise join technique" to refer to this
method. I have changed the other usages of this term to use wording
like child joins or join between partiitions or join between child
relations as appropriate. Also, I have changed the names of functions
dealing with joins between partitions to use child_join instead of
partition_join or partition_wise_join.

Since partition-wise join is a method to join two relations just like
other methods, try_partition_wise_join() fits into the naming
convention try_<join technique> like try_nestloop_join.

+    rel1_desc = makeStringInfo();
+    rel2_desc = makeStringInfo();
+
+    /* TODO: remove this notice when finalising the patch. */
+    outBitmapset(rel1_desc, rel1->relids);
+    outBitmapset(rel2_desc, rel2->relids);
+    elog(NOTICE, "join between relations %s and %s is considered for
partition-wise join.",
+         rel1_desc->data, rel2_desc->data);

Please remove your debugging cruft before submitting patches to
pgsql-hackers, or at least put #ifdef NOT_USED or something around it.

I kept this one intentionally. But as the TODO comment says, I do
intend to remove it once testing is over. Those messages make it very
easy to know whether partition-wise join was considered for a given
join or not. Without those messages, one has to break into
try_partition_wise_join() to figure out whether partition-wise join
was used or not. The final plan may not come out to be partition-wise
join plan even if partition-wise join was considered. Although, I have
now used DEBUG3 instead of NOTICE and removed those lines from the
expected output.

+     * We allocate the array for child RelOptInfos till we find at least one
+     * join order which can use partition-wise join technique. If no join order
+     * can use partition-wise join technique, there are no child relations.

This comment has problems. I think "till" is supposed to be "until",
and there's supposed to be a "don't" in there somewhere. But really,
I think what you're going for is just /* Allocate when first needed */
which would be a lot shorter and also more clear.

Sorry for those mistakes. Yes, shorter version is better. Fixed the
comment as per your suggestion.

+     * Create join relations for the partition relations, if they do not exist
+     * already. Add paths to those for the given pair of joining relations.

I think the comment could be a bit more explanatory here. Something
like: "This joinrel is partitioned, so iterate over the partitions and
create paths for each one, allowing us to eventually build an
append-of-joins path for the parent. Since this routine may be called
multiple times for various join orders, the RelOptInfo needed for each
child join may or may not already exist, but the paths for this join
order definitely do not. Note that we don't create any actual
AppendPath at this stage; it only makes sense to do that at the end,
after each possible join order has been considered for each child
join. The best join order may differ from child to child."

Copied verbatim. Thanks for the detailed comment.

+ * partiticipating in the given partition relations. We need them

Spelling.

Done. Also fixed other grammatical mistakes and typos in that comment.

+/*
+ * Construct the SpecialJoinInfo for the partition-wise join using parents'
+ * special join info. Also, instead of
+ * constructing an sjinfo everytime, we should probably save it in
+ * root->join_info_list and search within it like join_is_legal?
+ */

The lines here are of very different lengths for no particularly good
reason, and it should end with a period, not a question mark.

My bad. Sorry. Fixed.

On the substance of the issue, it seems like the way you're doing this
right now could allocate a very large number of SpecialJoinInfo
structures. For every join relation, you'll create one
SpecialJoinInfo per legal join order per partition. That seems like
it could get to be a big number. I don't know if that's going to be a
problem from a memory-usage standpoint, but it seems like it might.
It's not just the SpecialJoinInfo itself; all of the substructure gets
duplicated, too.

Yes. We need the SpecialJoinInfo structures for the existing path
creation to work. The code will be complicated if we try to use parent
SpecialJoinInfo instead of creating those for children. We may free
memory allocated in SpecialJoinInfo to save some memory.
SpecialJoinInfos are not needed once the paths are created. Still we
will waste some memory for semi_rhs_exprs, which are reused for unique
paths. But otherwise we will reclaim the rest of the memory. Memory
wastage in adjust_partition_relids() may be minimized by modifying
adjust_appendrel_attrs() to accept list of AppendRelInfos and mutating
the tree only once rather than doing it N times for an N-way join.

+    SpecialJoinInfo *sjinfo = copyObject(parent_sjinfo);
+    sjinfo->min_lefthand = adjust_partition_relids(sjinfo->min_lefthand,
+                                                   append_rel_infos1);

Missing a blank line here.

Done.

+ AppendRelInfo *ari = lfirst(lc);

Standard naming convention for an AppendRelInfo variable seems to be
appinfo, not ari. (I just did "git grep AppendRelInfo".)

Done.

+        /* Skip non-equi-join clauses. */
+        if (!rinfo->can_join ||
+            rinfo->hashjoinoperator == InvalidOid ||
+            !rinfo->mergeopfamilies)
+            continue;

There's definitely something ugly about this. If rinfo->can_join is
false, then we're done. But suppose one of mergeopfamilies == NIL and
rinfo->hashoperator == InvalidOid is true and the other is false. Are
we really precluded from doing a partiion-wise join in that case, or
are we just prohibited from using certain join strategies? In most
places where we make similar tests, we're careful not to require more
than we need.

Right. That condition is flawed. Corrected it.

I also think that these tests need to consider the partitioning
operator in use. Suppose that the partition key is of a type T which
has two operator classes X and Y. Both relations are partitioned
using an operator from opfamily X, but the join condition mentions
opfamily Y. I'm pretty sure this precludes a partitionwise join. If
the join condition used opfamily X, then we would have a guarantee
that two rows which compared as equal would be in the same partition,
but because it uses opfamily Y, that's not guaranteed. For example,
if T is a text type, X might test for exact equality using "C"
collation rules, while Y might test for equality using some
case-insensitive set of rules. If the partition boundaries are such
that "foo" and "FOO" are in different partitions, a partitionwise join
using the case-insensitive operator will produce wrong results. You
can also imagine this happening with numeric, if you have one opclass
(like the default one) that considers 5.0 and 5.00 to be equal, but
another opclass that thinks they are different; if the latter is used
to set the partition bounds, 5.0 and 5.00 could end up in different
partitions - which will be fine if an operator from that opclass is
used for the join, but not if an operator from the regular opclass is
used.

Your description above uses opfamily and opclass interchangeably. It
starts saying X and Y are classed but then also refers to them as
families. But I got the point. I guess, similar to
relation_has_unique_index_for(), I have to check whether the operator
family specified in the partition scheme is present in the
mergeopfamilies in RestrictInfo for matching partition key. I have
added that check and restructured that portion of code to be readable.

After thinking this over a bit, I think the right way to think about this is:

1. Amit's patch currently only ever uses btree opfamilies for
partitioning. It uses those for both range partitioning and list
partitioning. If we ever support hash partitioning, we would
presumably use hash opfamilies for that purpose, but right now it's
all about btree opfamilies.

2. Therefore, if A and B are partitioned but the btree opfamilies
don't match, they don't have the same partitioning scheme and this
code should never be reached. Similarly, if they use the same
opfamily but different collations, the partitioning schemes shouldn't
match and therefore this code should not be reached.

That's right.

3. If A and B are partitioned and the partitioning opfamilies - which
are necessarily btree opfamilies - do match, then the operator which
appears in the query needs to be from the same opfamily and have
amopstrategy of BTEqualStrategyNumber within that opfamily. If not,
then a partition-wise join is not possible.

I think this is achieved by checking whether the opfamily for given
partition key is present in the mergeopfamilies of corresponding
RestrictInfo, as stated above.

4. Assuming the above conditions are met, have_partkey_equi_join
doesn't need to care whether the operator chosen has mergeopfamilies
or a valid hashjoinoperator. Those factors will control which join
methods are legal, but not whether a partitionwise join is possible in
principle.

If mergeopfamilies is NIL, above check will fail anyway. But skipping
a clause which has mergeopfamilies NIL will save some cycles in
matching expressions.

There is something strange happening with Amit's patch. When we create
a table partitioned by range on a column of type int2vector, it
somehow gets a btree operator family, but doesn't have mergeopfamilies
set in RestrictInfo of equality condition on that column. Instead the
RestrictInfo has hashjoinoperator. In this case if we ignore
hashjoinoperator, we won't be able to apply partition-wise join. I
guess, in such case we want to play safe and not apply partition-wise
join, even though applying it will give the correct result.

+ * RelabelType node; eval_const_expressions() will have simplied if more

Spelling.

Thanks. Done.

/*
+     * Code below scores equivalence classes by how many equivalence members
+     * can produce join clauses for this join relation. Equivalence members
+     * which do not cover the parents of a partition-wise join relation, can
+     * produce join clauses for partition-wise join relation.
+     */

I don't know what that means. The comma in the second sentence
doesn't belong there.

Sorry for that construction. I have changed the comment to be
something more meaningful.

+    /*
+     * TODO: Instead of copying and mutating the trees one child relation at a
+     * time, we should be able to do this en-masse for all the partitions
+     * involved.
+     */

I don't see how that would be possible, but if it's a TODO, you'd
better do it (or decide not to do it and remove or change the
comment).

That should be doable by passing a list of AppendRelInfo structures to
adjust_appendrel_attrs_mutator(). In the mutator, we have to check
each appinfo instead of just one. But that's a lot of refactoring. May
be done as a separate patch, if we are consuming too much memory. I
have removed TODO for now.

/*
* Create explicit sort nodes for the outer and inner paths if necessary.
*/
if (best_path->outersortkeys)
{
+        Relids        outer_relids = outer_path->parent->relids;
Sort       *sort = make_sort_from_pathkeys(outer_plan,
-                                                   best_path->outersortkeys);
+                                                   best_path->outersortkeys,
+                                                   outer_relids);

The changes related to make_sort_from_pathkeys() are pretty opaque to
me. Can you explain?

prepare_sort_from_pathkeys() accepts Relids as one of the argument to
find equivalence members belonging to child relations. The function
does not expect relids when searching equivalence members for parent
relations. Before this patch, make_sort_from_pathkeys() passed NULL to
this function, because it didn't expect child relations before.
Because of partition-wise joins, we need to sort child relations for
merge join or to create unique paths. So, make_sort_from_pathkeys() is
required to pass relids to prepare_sort_from_pathkeys() when
processing child relations, so that the later does not skip child
members.

+ * Change parameterization of sub paths recursively. Also carry out any

"sub paths" should not be two words, here or anywhere.

Fixed.

+reparameterize_path_for_child(PlannerInfo *root, Path *path,
+                              RelOptInfo *child_rel)

This is suspiciously unlike reparameterize_path. Why?

reparameterize_path() tries to create path with new parameterization
from an existing parameterized path. So, it looks for additional
conditions to expand the parameterization. But this functions
translates a path parameterized by parent to be parameterized by its
child. That does not involve looking for any extra conditions, but
involves translating the existing ones so that they can be used with a
child. A right name would be translate_parampath_to_child() or
something which uses word "translate" instead of "reparameterize". But
every name like that is getting too long. For now I have renamed it as
reparameterize_path_by_child(). Also added a comment in the function
prologue about cost, rows, width etc.

+    /* Computer information relevant to the foreign relations. */
+    set_foreign_rel_properties(joinrel, outer_rel, inner_rel);

Perhaps this refactoring could be split out into a preliminary patch,
which would then simplify this patch. And same for add_join_rel().

Yes, that's better. I will separate the code out in a separate patch.

There's code in build_join_rel() and build_partition_join_rel() (I
will change that name) which creates a joinrel RelOptInfo. Most of
that code simply sets NULL or 0 fields and is duplicated in both the
functions. Do you see any value in separating it out in its own
function?

Also, makeNode() uses palloc0(), thus makeNode(RelOptInfo) would set
most of the fields to 0 or NULL. Why do we then again set those fields
as NULL or 0? Should I try to remove unnecessary assignments?

+     * Produce partition-wise joinrel's targetlist by translating the parent
+     * joinrel's targetlist. This will also include the required placeholder

Again the confusion between a "child" join and a partition-wise join...

+    /*
+     * Nothing to do if
+     * a. partition-wise join is disabled.
+     * b. joining relations are not partitioned.
+     * c. partitioning schemes do not match.
+     */
+

I don't think that's going to survive pgindent.

Changed this code a bit.

+ * are not considered equal, an equi-join involing inner partition keys

Spelling.

+     * Collect the partition key expressions. An OUTER join will produce rows
+     * where the partition key columns of inner side are NULL and may not fit
+     * the partitioning scheme with inner partition keys. Since two NULL values
+     * are not considered equal, an equi-join involing inner partition keys
+     * still prohibits cross-partition joins while joining with another
+     * similarly partitioned relation.

I can't figure out what this comment is trying to tell me. Possibly I
just need more caffeine.

Re-wrote the comment with examples and detailed explanation. The
comment talks about whether inner partition key expressions should be
considered as the partition key expressions for the join, given that
for an OUTER join the inner partition key expressions may go NULL. The
comment explains why it's safe to do so. If we don't do that, any FULL
OUTER join will have no partition expressions and thus partition-wise
join technique will be useless for a N-way FULL OUTER join even if
it's safe to use it.

+ * Adding these two join_rel_level list also means that top level list has more
+ * than one join relation, which is symantically incorrect.

I don't understand this, either; also, spelling.

I think, that sentence is not required. Removed it.

As a general comment, the ratio of tests-to-code in this patch is way
out of line with PostgreSQL's normal practices. The total patch file
is 10965 lines. The test cases begin at line 3047, meaning that in
round figures you've got about one-quarter code and about
three-quarters test cases. I suspect that a large fraction of those
test cases aren't adding any meaningful code coverage and will just
take work to maintain. That needs to be slimmed down substantially in
any version of this considered for commit.

I agree. We require two kinds of tests 1. those which test partition
scheme matching 2. those test the planner code, which deals with path
creation. I have added both kinds of testcases for all kinds of
partitioning schemes (range, list, multi-level, partition key being
expressions, columns). That's not required. We need 1st kind of tests
for all partitioning schemes and 2nd kind of testcases only for one of
the partitioning schemes. So, definitely the number of tests will
reduce. A possible extreme would be to use a single multi-level
partitioned tests, which includes all kinds of partitioning schemes at
various partition levels. But that kind of testcase will be highly
unreadable and harder to maintain. Let me know what do you think. I
will work on that in the next version of patch. The test still fails
because of a bug in Amit's earlier set of patches

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

Attachments:

pg_dp_join_v4.patchtext/plain; charset=US-ASCII; name=pg_dp_join_v4.patchDownload
diff --git a/contrib/postgres_fdw/deparse.c b/contrib/postgres_fdw/deparse.c
index 691658f..287c7d5 100644
--- a/contrib/postgres_fdw/deparse.c
+++ b/contrib/postgres_fdw/deparse.c
@@ -763,21 +763,21 @@ build_tlist_to_deparse(RelOptInfo *foreignrel)
  * List of columns selected is returned in retrieved_attrs.
  */
 extern void
 deparseSelectStmtForRel(StringInfo buf, PlannerInfo *root, RelOptInfo *rel,
 						List *tlist, List *remote_conds, List *pathkeys,
 						List **retrieved_attrs, List **params_list)
 {
 	deparse_expr_cxt context;
 
 	/* We handle relations for foreign tables and joins between those */
-	Assert(rel->reloptkind == RELOPT_JOINREL ||
+	Assert(IS_JOIN_REL(rel) ||
 		   rel->reloptkind == RELOPT_BASEREL ||
 		   rel->reloptkind == RELOPT_OTHER_MEMBER_REL);
 
 	/* Fill portions of context common to join and base relation */
 	context.buf = buf;
 	context.root = root;
 	context.foreignrel = rel;
 	context.params_list = params_list;
 
 	/* Construct SELECT clause and FROM clause */
@@ -817,21 +817,21 @@ deparseSelectSql(List *tlist, List **retrieved_attrs, deparse_expr_cxt *context)
 	StringInfo	buf = context->buf;
 	RelOptInfo *foreignrel = context->foreignrel;
 	PlannerInfo *root = context->root;
 	PgFdwRelationInfo *fpinfo = (PgFdwRelationInfo *) foreignrel->fdw_private;
 
 	/*
 	 * Construct SELECT list
 	 */
 	appendStringInfoString(buf, "SELECT ");
 
-	if (foreignrel->reloptkind == RELOPT_JOINREL)
+	if (IS_JOIN_REL(foreignrel))
 	{
 		/* For a join relation use the input tlist */
 		deparseExplicitTargetList(tlist, retrieved_attrs, context);
 	}
 	else
 	{
 		/*
 		 * For a base relation fpinfo->attrs_used gives the list of columns
 		 * required to be fetched from the foreign server.
 		 */
@@ -845,22 +845,21 @@ deparseSelectSql(List *tlist, List **retrieved_attrs, deparse_expr_cxt *context)
 
 		deparseTargetList(buf, root, foreignrel->relid, rel, false,
 						  fpinfo->attrs_used, false, retrieved_attrs);
 		heap_close(rel, NoLock);
 	}
 
 	/*
 	 * Construct FROM clause
 	 */
 	appendStringInfoString(buf, " FROM ");
-	deparseFromExprForRel(buf, root, foreignrel,
-						  (foreignrel->reloptkind == RELOPT_JOINREL),
+	deparseFromExprForRel(buf, root, foreignrel, IS_JOIN_REL(foreignrel),
 						  context->params_list);
 }
 
 /*
  * Emit a target list that retrieves the columns specified in attrs_used.
  * This is used for both SELECT and RETURNING targetlists; the is_returning
  * parameter is true only for a RETURNING targetlist.
  *
  * The tlist text is appended to buf, and we also create an integer List
  * of the columns being retrieved, which is returned to *retrieved_attrs.
@@ -981,21 +980,21 @@ deparseLockingClause(deparse_expr_cxt *context)
 		 * before 8.3.
 		 */
 		if (relid == root->parse->resultRelation &&
 			(root->parse->commandType == CMD_UPDATE ||
 			 root->parse->commandType == CMD_DELETE))
 		{
 			/* Relation is UPDATE/DELETE target, so use FOR UPDATE */
 			appendStringInfoString(buf, " FOR UPDATE");
 
 			/* Add the relation alias if we are here for a join relation */
-			if (rel->reloptkind == RELOPT_JOINREL)
+			if (IS_JOIN_REL(rel))
 				appendStringInfo(buf, " OF %s%d", REL_ALIAS_PREFIX, relid);
 		}
 		else
 		{
 			PlanRowMark *rc = get_plan_rowmark(root->rowMarks, relid);
 
 			if (rc)
 			{
 				/*
 				 * Relation is specified as a FOR UPDATE/SHARE target, so
@@ -1017,22 +1016,21 @@ deparseLockingClause(deparse_expr_cxt *context)
 					case LCS_FORSHARE:
 						appendStringInfoString(buf, " FOR SHARE");
 						break;
 					case LCS_FORNOKEYUPDATE:
 					case LCS_FORUPDATE:
 						appendStringInfoString(buf, " FOR UPDATE");
 						break;
 				}
 
 				/* Add the relation alias if we are here for a join relation */
-				if (rel->reloptkind == RELOPT_JOINREL &&
-					rc->strength != LCS_NONE)
+				if (IS_JOIN_REL(rel) && rc->strength != LCS_NONE)
 					appendStringInfo(buf, " OF %s%d", REL_ALIAS_PREFIX, relid);
 			}
 		}
 	}
 }
 
 /*
  * Deparse conditions from the provided list and append them to buf.
  *
  * The conditions in the list are assumed to be ANDed. This function is used to
@@ -1155,21 +1153,21 @@ deparseExplicitTargetList(List *tlist, List **retrieved_attrs,
  * The function constructs ... JOIN ... ON ... for join relation. For a base
  * relation it just returns schema-qualified tablename, with the appropriate
  * alias if so requested.
  */
 static void
 deparseFromExprForRel(StringInfo buf, PlannerInfo *root, RelOptInfo *foreignrel,
 					  bool use_alias, List **params_list)
 {
 	PgFdwRelationInfo *fpinfo = (PgFdwRelationInfo *) foreignrel->fdw_private;
 
-	if (foreignrel->reloptkind == RELOPT_JOINREL)
+	if (IS_JOIN_REL(foreignrel))
 	{
 		RelOptInfo *rel_o = fpinfo->outerrel;
 		RelOptInfo *rel_i = fpinfo->innerrel;
 		StringInfoData join_sql_o;
 		StringInfoData join_sql_i;
 
 		/* Deparse outer relation */
 		initStringInfo(&join_sql_o);
 		deparseFromExprForRel(&join_sql_o, root, rel_o, true, params_list);
 
@@ -1860,21 +1858,21 @@ deparseExpr(Expr *node, deparse_expr_cxt *context)
  * Deparse given Var node into context->buf.
  *
  * If the Var belongs to the foreign relation, just print its remote name.
  * Otherwise, it's effectively a Param (and will in fact be a Param at
  * run time).  Handle it the same way we handle plain Params --- see
  * deparseParam for comments.
  */
 static void
 deparseVar(Var *node, deparse_expr_cxt *context)
 {
-	bool		qualify_col = (context->foreignrel->reloptkind == RELOPT_JOINREL);
+	bool		qualify_col = IS_JOIN_REL(context->foreignrel);
 
 	if (bms_is_member(node->varno, context->foreignrel->relids) &&
 		node->varlevelsup == 0)
 		deparseColumnRef(context->buf, node->varno, node->varattno,
 						 context->root, qualify_col);
 	else
 	{
 		/* Treat like a Param */
 		if (context->params_list)
 		{
diff --git a/contrib/postgres_fdw/postgres_fdw.c b/contrib/postgres_fdw/postgres_fdw.c
index daf0438..594292a 100644
--- a/contrib/postgres_fdw/postgres_fdw.c
+++ b/contrib/postgres_fdw/postgres_fdw.c
@@ -1163,21 +1163,21 @@ postgresGetForeignPlan(PlannerInfo *root,
 			local_exprs = lappend(local_exprs, rinfo->clause);
 		else if (is_foreign_expr(root, foreignrel, rinfo->clause))
 		{
 			remote_conds = lappend(remote_conds, rinfo);
 			remote_exprs = lappend(remote_exprs, rinfo->clause);
 		}
 		else
 			local_exprs = lappend(local_exprs, rinfo->clause);
 	}
 
-	if (foreignrel->reloptkind == RELOPT_JOINREL)
+	if (IS_JOIN_REL(foreignrel))
 	{
 		/* For a join relation, get the conditions from fdw_private structure */
 		remote_conds = fpinfo->remote_conds;
 		local_exprs = fpinfo->local_conds;
 
 		/* Build the list of columns to be fetched from the foreign server. */
 		fdw_scan_tlist = build_tlist_to_deparse(foreignrel);
 
 		/*
 		 * Ensure that the outer plan produces a tuple whose descriptor
@@ -1221,21 +1221,21 @@ postgresGetForeignPlan(PlannerInfo *root,
 							&retrieved_attrs, &params_list);
 
 	/*
 	 * Build the fdw_private list that will be available to the executor.
 	 * Items in the list must match order in enum FdwScanPrivateIndex.
 	 */
 	fdw_private = list_make4(makeString(sql.data),
 							 remote_conds,
 							 retrieved_attrs,
 							 makeInteger(fpinfo->fetch_size));
-	if (foreignrel->reloptkind == RELOPT_JOINREL)
+	if (IS_JOIN_REL(foreignrel))
 		fdw_private = lappend(fdw_private,
 							  makeString(fpinfo->relation_name->data));
 
 	/*
 	 * Create the ForeignScan node for the given relation.
 	 *
 	 * Note that the remote parameter expressions are stored in the fdw_exprs
 	 * field of the finished plan node; we can't keep them in private state
 	 * because then they wouldn't be subject to later planner processing.
 	 */
@@ -2498,21 +2498,21 @@ estimate_path_cost_size(PlannerInfo *root,
 		List	   *retrieved_attrs;
 
 		/*
 		 * param_join_conds might contain both clauses that are safe to send
 		 * across, and clauses that aren't.
 		 */
 		classifyConditions(root, foreignrel, param_join_conds,
 						   &remote_param_join_conds, &local_param_join_conds);
 
 		/* Build the list of columns to be fetched from the foreign server. */
-		if (foreignrel->reloptkind == RELOPT_JOINREL)
+		if (IS_JOIN_REL(foreignrel))
 			fdw_scan_tlist = build_tlist_to_deparse(foreignrel);
 		else
 			fdw_scan_tlist = NIL;
 
 		/*
 		 * The complete list of remote conditions includes everything from
 		 * baserestrictinfo plus any extra join_conds relevant to this
 		 * particular path.
 		 */
 		remote_conds = list_concat(list_copy(remote_param_join_conds),
@@ -2579,21 +2579,21 @@ estimate_path_cost_size(PlannerInfo *root,
 		 * We will come here again and again with different set of pathkeys
 		 * that caller wants to cost. We don't need to calculate the cost of
 		 * bare scan each time. Instead, use the costs if we have cached them
 		 * already.
 		 */
 		if (fpinfo->rel_startup_cost > 0 && fpinfo->rel_total_cost > 0)
 		{
 			startup_cost = fpinfo->rel_startup_cost;
 			run_cost = fpinfo->rel_total_cost - fpinfo->rel_startup_cost;
 		}
-		else if (foreignrel->reloptkind != RELOPT_JOINREL)
+		else if (!IS_JOIN_REL(foreignrel))
 		{
 			/* Clamp retrieved rows estimates to at most foreignrel->tuples. */
 			retrieved_rows = Min(retrieved_rows, foreignrel->tuples);
 
 			/*
 			 * Cost as though this were a seqscan, which is pessimistic.  We
 			 * effectively imagine the local_conds are being evaluated
 			 * remotely, too.
 			 */
 			startup_cost = 0;
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 1a60563..dc2b34b 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -13,38 +13,41 @@
  *-------------------------------------------------------------------------
 */
 
 #include "postgres.h"
 
 #include "access/heapam.h"
 #include "access/htup_details.h"
 #include "access/nbtree.h"
 #include "access/sysattr.h"
 #include "catalog/dependency.h"
+#include "catalog/heap.h"
 #include "catalog/indexing.h"
 #include "catalog/objectaddress.h"
 #include "catalog/partition.h"
 #include "catalog/pg_collation.h"
 #include "catalog/pg_inherits.h"
 #include "catalog/pg_inherits_fn.h"
 #include "catalog/pg_opclass.h"
 #include "catalog/pg_partitioned_table.h"
 #include "catalog/pg_partitioned_table_fn.h"
 #include "catalog/pg_type.h"
 #include "executor/executor.h"
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
 #include "nodes/parsenodes.h"
 #include "optimizer/clauses.h"
+#include "optimizer/cost.h"
 #include "optimizer/planmain.h"
 #include "optimizer/var.h"
+#include "rewrite/rewriteManip.h"
 #include "storage/lmgr.h"
 #include "utils/array.h"
 #include "utils/builtins.h"
 #include "utils/datum.h"
 #include "utils/memutils.h"
 #include "utils/fmgroids.h"
 #include "utils/inval.h"
 #include "utils/lsyscache.h"
 #include "utils/ruleutils.h"
 #include "utils/rel.h"
@@ -2412,20 +2415,78 @@ make_range_bound(PartitionKey key, List *val, bool inclusive, bool lower)
 										  key->tcinfo->typbyval[i],
 										  key->tcinfo->typlen[i]);
 			i++;
 		}
 	}
 
 	return bound;
 }
 
 /*
+ * Return a copy of input BoundCollection structure containg nparts number of
+ * partitions. The data types of bounds are described by given partition key
+ * specificiation.
+ */
+static BoundCollection
+copy_bounds(BoundCollection src_bounds, PartitionKey key, int nparts)
+{
+	BoundCollection dst_bounds;
+	int		i;
+
+	dst_bounds = (BoundCollection) palloc(sizeof(BoundCollectionData));
+
+	if (src_bounds->listinfo)
+	{
+		ListInfo *dst_li = (ListInfo *) palloc(sizeof(ListInfo));
+		ListInfo *src_li = src_bounds->listinfo;
+
+		Assert(!src_bounds->rangeinfo);
+		dst_bounds->rangeinfo = NULL;
+
+		/* Copy the ListInfo structure. */
+		dst_li->nvalues = src_li->nvalues;
+		dst_li->has_null = src_li->has_null;
+		dst_li->null_index = src_li->null_index;
+
+		dst_li->values = (Datum *) palloc(sizeof(Datum) * dst_li->nvalues);
+		dst_li->indexes = (int *) palloc(sizeof(int) * dst_li->nvalues);
+		for (i = 0; i < dst_li->nvalues; i++)
+		{
+			dst_li->values[i] = datumCopy(src_li->values[i],
+													key->tcinfo->typbyval[0],
+													key->tcinfo->typlen[0]);
+			dst_li->indexes[i] = src_li->indexes[i];
+		}
+
+		dst_bounds->listinfo = dst_li;
+	}
+	else
+	{
+		RangeInfo *dst_ri = (RangeInfo *) palloc(sizeof(RangeInfo));
+		RangeInfo *src_ri = src_bounds->rangeinfo;
+
+		Assert(!src_bounds->listinfo && src_bounds->rangeinfo);
+		dst_bounds->listinfo = NULL;
+
+		/* Copy RangeInfo structure. */
+		dst_ri = (RangeInfo *) palloc(sizeof(RangeInfo));
+		dst_ri->ranges = (PartitionRange **) palloc(sizeof(PartitionRange *) * nparts);
+		for (i = 0; i < nparts; i++)
+			dst_ri->ranges[i] = copy_range(src_ri->ranges[i], key);
+
+		dst_bounds->rangeinfo = dst_ri;
+	}
+
+	return dst_bounds;
+}
+
+/*
  * Make and return a copy of input PartitionRange.
  */
 static PartitionRange *
 copy_range(PartitionRange *src, PartitionKey key)
 {
 	PartitionRange *result;
 
 	result = (PartitionRange *) palloc0(sizeof(PartitionRange));
 	result->lower = copy_range_bound(src->lower, key);
 	result->upper = copy_range_bound(src->upper, key);
@@ -2647,10 +2708,175 @@ tuple_rightof_bound(PartitionKey key, Datum *tuple, PartitionRangeBound *bound)
 static bool
 tuple_leftof_bound(PartitionKey key, Datum *tuple, PartitionRangeBound *bound)
 {
 	int32	cmpval = partition_range_tuple_cmp(key, tuple, bound->val);
 
 	if (!cmpval)
 		return !bound->lower ? bound->inclusive : !bound->inclusive;
 
 	return cmpval < 0;
 }
+
+/*
+ * find_partition_scheme
+ * 		Find the "canonical" partition scheme for the given base table.
+ *
+ * The function searches the list of canonical partition schemes for one that
+ * exactly matches the partitioning properties of the given relation. If it
+ * does not find one, the function creates a canonical partition scheme
+ * structure and adds it to the list.
+ *
+ * For an unpartitioned table, it returns NULL.
+ */
+
+extern PartitionScheme
+find_partition_scheme(PlannerInfo *root, Relation relation)
+{
+	PartitionKey	part_key = RelationGetPartitionKey(relation);
+	PartitionDesc	part_desc = RelationGetPartitionDesc(relation);
+	ListCell	   *lc;
+	int		nparts;
+	int		partnatts;
+	int		cnt_pks;
+	PartitionScheme	part_scheme = NULL;
+
+	/* No partition scheme for an unpartitioned relation. */
+	if (!part_desc || !part_key)
+		return NULL;
+
+	nparts = part_desc->nparts;
+	partnatts = part_key->partnatts;
+
+	/* Search for a matching partition scheme and return if found one. */
+	foreach (lc, root->part_schemes)
+	{
+		part_scheme = lfirst(lc);
+
+		/* Match number of partitions and partitioning strategy. */
+		if (nparts != part_scheme->nparts ||
+			part_key->strategy != part_scheme->strategy ||
+			partnatts != part_scheme->partnatts)
+			continue;
+
+		/* Match the partition key types. */
+		for (cnt_pks = 0; cnt_pks < partnatts; cnt_pks++)
+		{
+			/*
+			 * It suffices to check the OID of support function as it always has
+			 * two arguments and returns boolean. For types, it suffices to match
+			 * the type id, mod and collation; len, byval and align are depedent on
+			 * the first two.
+			 */
+			if (part_key->partopfamily[cnt_pks] != part_scheme->partopfamily[cnt_pks] ||
+				part_key->partopcintype[cnt_pks] != part_scheme->partopcintype[cnt_pks] ||
+				part_key->tcinfo->typid[cnt_pks] != part_scheme->key_types[cnt_pks] ||
+				part_key->tcinfo->typmod[cnt_pks] != part_scheme->key_typmods[cnt_pks] ||
+				part_key->tcinfo->typcoll[cnt_pks] != part_scheme->key_collations[cnt_pks])
+				break;
+		}
+
+		/* Some partition key didn't match. Check next partitioning scheme. */
+		if (cnt_pks < partnatts)
+			continue;
+
+		if (!partition_bounds_equal(part_key, part_desc->bounds,
+									part_scheme->bounds, nparts))
+			continue;
+
+		/* Found matching partition scheme. */
+		return part_scheme;
+	}
+
+	/* Did not find matching partition scheme. Create one. */
+	part_scheme = (PartitionScheme) palloc0(sizeof(PartitionSchemeData));
+
+	/* Copy partition bounds/lists. */
+	part_scheme->nparts = part_desc->nparts;
+	part_scheme->strategy = part_key->strategy;
+	part_scheme->bounds = copy_bounds(part_desc->bounds, part_key,
+									  part_scheme->nparts);
+
+	/* Store partition key information. */
+	part_scheme->partnatts = part_key->partnatts;
+
+	part_scheme->partopfamily = (Oid *) palloc(sizeof(Oid) * partnatts);
+	part_scheme->partopcintype = (Oid *) palloc(sizeof(Oid) * partnatts);
+	part_scheme->key_types = (Oid *) palloc(sizeof(Oid) * partnatts);
+	part_scheme->key_typmods = (int32 *) palloc(sizeof(int32) * partnatts);
+	part_scheme->key_collations = (Oid *) palloc(sizeof(Oid) * partnatts);
+
+	for (cnt_pks = 0; cnt_pks < partnatts; cnt_pks++)
+	{
+		part_scheme->partopfamily[cnt_pks] = part_key->partopfamily[cnt_pks];
+		part_scheme->partopcintype[cnt_pks] = part_key->partopcintype[cnt_pks];
+		part_scheme->key_types[cnt_pks] = part_key->tcinfo->typid[cnt_pks];
+		part_scheme->key_typmods[cnt_pks] = part_key->tcinfo->typmod[cnt_pks];
+		part_scheme->key_collations[cnt_pks] = part_key->tcinfo->typcoll[cnt_pks];
+	}
+
+	/* Add the partitioning scheme to PlannerInfo. */
+	root->part_schemes = lappend(root->part_schemes, part_scheme);
+
+	return part_scheme;
+}
+
+/*
+ * build_baserel_partition_key_exprs
+ *		Collect partition key expressions for a given base relation.
+ *
+ * The function converts single column partition keys into corresponding Var
+ * nodes. It restamps Var nodes in partition key expressions by given varno.
+ * The partition key expressions are returned as an array of Lists to be stored
+ * in RelOptInfo of the base relation.
+ */
+extern List **
+build_baserel_partition_key_exprs(Relation relation, Index varno)
+{
+	PartitionKey	part_key = RelationGetPartitionKey(relation);
+	int		num_pkexprs;
+	int		cnt_pke;
+	List		  **partexprs;
+	ListCell	   *lc;
+
+	if (!part_key || part_key->partnatts <= 0)
+		return NULL;
+
+	num_pkexprs = part_key->partnatts;
+	partexprs = (List **) palloc(sizeof(List *) * num_pkexprs);
+	lc = list_head(part_key->partexprs);
+
+	for (cnt_pke = 0; cnt_pke < num_pkexprs; cnt_pke++)
+	{
+		AttrNumber attno = part_key->partattrs[cnt_pke];
+		Expr	  *pkexpr;
+
+		if (attno != InvalidAttrNumber)
+		{
+			/* Single column partition key is stored as a Var node. */
+			Form_pg_attribute att_tup;
+
+			if (attno < 0)
+				att_tup = SystemAttributeDefinition(attno,
+												relation->rd_rel->relhasoids);
+			else
+				att_tup = relation->rd_att->attrs[attno - 1];
+
+			pkexpr = (Expr *) makeVar(varno, attno, att_tup->atttypid,
+									  att_tup->atttypmod,
+									  att_tup->attcollation, 0);
+		}
+		else
+		{
+			if (lc == NULL)
+				elog(ERROR, "wrong number of partition key expressions");
+
+			/* Re-stamp the expressions with given varno. */
+			pkexpr = (Expr *) copyObject(lfirst(lc));
+			ChangeVarNodes((Node *) pkexpr, 1, varno, 0);
+			lc = lnext(lc);
+		}
+
+		partexprs[cnt_pke] = list_make1(pkexpr);
+	}
+
+	return partexprs;
+}
diff --git a/src/backend/foreign/foreign.c b/src/backend/foreign/foreign.c
index 242d6d2..75c95e4 100644
--- a/src/backend/foreign/foreign.c
+++ b/src/backend/foreign/foreign.c
@@ -714,21 +714,21 @@ get_foreign_server_oid(const char *servername, bool missing_ok)
  * Since the plan created using this path will presumably only be used to
  * execute EPQ checks, efficiency of the path is not a concern. But since the
  * path list in RelOptInfo is anyway sorted by total cost we are likely to
  * choose the most efficient path, which is all for the best.
  */
 extern Path *
 GetExistingLocalJoinPath(RelOptInfo *joinrel)
 {
 	ListCell   *lc;
 
-	Assert(joinrel->reloptkind == RELOPT_JOINREL);
+	Assert(IS_JOIN_REL(joinrel));
 
 	foreach(lc, joinrel->pathlist)
 	{
 		Path	   *path = (Path *) lfirst(lc);
 		JoinPath   *joinpath = NULL;
 
 		/* Skip parameterised paths. */
 		if (path->param_info != NULL)
 			continue;
 
@@ -779,27 +779,27 @@ GetExistingLocalJoinPath(RelOptInfo *joinrel)
 		 * If either inner or outer path is a ForeignPath corresponding to a
 		 * pushed down join, replace it with the fdw_outerpath, so that we
 		 * maintain path for EPQ checks built entirely of local join
 		 * strategies.
 		 */
 		if (IsA(joinpath->outerjoinpath, ForeignPath))
 		{
 			ForeignPath *foreign_path;
 
 			foreign_path = (ForeignPath *) joinpath->outerjoinpath;
-			if (foreign_path->path.parent->reloptkind == RELOPT_JOINREL)
+			if (IS_JOIN_REL(foreign_path->path.parent))
 				joinpath->outerjoinpath = foreign_path->fdw_outerpath;
 		}
 
 		if (IsA(joinpath->innerjoinpath, ForeignPath))
 		{
 			ForeignPath *foreign_path;
 
 			foreign_path = (ForeignPath *) joinpath->innerjoinpath;
-			if (foreign_path->path.parent->reloptkind == RELOPT_JOINREL)
+			if (IS_JOIN_REL(foreign_path->path.parent))
 				joinpath->innerjoinpath = foreign_path->fdw_outerpath;
 		}
 
 		return (Path *) joinpath;
 	}
 	return NULL;
 }
diff --git a/src/backend/optimizer/README b/src/backend/optimizer/README
index 775bcc3..29f2bb1 100644
--- a/src/backend/optimizer/README
+++ b/src/backend/optimizer/README
@@ -967,10 +967,51 @@ beneath a Gather node - which we call "partial" paths since they return
 only a subset of the results in each worker - must be kept separate from
 ordinary paths (see RelOptInfo's partial_pathlist and the function
 add_partial_path).
 
 One of the keys to making parallel query effective is to run as much of
 the query in parallel as possible.  Therefore, we expect it to generally
 be desirable to postpone the Gather stage until as near to the top of the
 plan as possible.  Expanding the range of cases in which more work can be
 pushed below the Gather (and costing them accurately) is likely to keep us
 busy for a long time to come.
+
+Partition-wise 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 for a given row in a given partition of a given
+partitioned table, its joining row, if exists, should exist only in the
+matching partition of the other partitioned table; no row from non-matching
+partitions in the other partitioned table can join with the given row from the
+first table. This condition allows the join between partitioned table 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
+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 partition tables
+into join between their partitions is called partition-wise join. We will use
+term "partitioned relation" for both partitioned table as well as join between
+partitioned tables which can use partition-wise join technique.
+
+Partitioning properties of a partitioned table are stored in
+PartitionSchemeData structure. Planner maintains a list of canonical partition
+schemes (distinct PartitionSchemeData objects) so that any two partitioned
+relations with same partitioning scheme share the same PartitionSchemeData
+object. This reduces memory consumed by PartitionSchemeData objects and makes
+it easy to compare the partition schemes of joining relations. RelOptInfos of
+partitioned relations hold partition key expressions and the RelOptInfos of
+the partition relations of that relation.
+
+Partition-wise joins are planned in two phases
+
+1. First phase creates the RelOptInfos for joins between matching partitions
+and creates join paths for these joins. The join paths for join between
+partitions are created using the same techniques as described above.
+
+2. After creating all possible paths for joins between the partitions,
+Append/MergeAppend paths are created to construct join between the partitioned
+relations by choosing the one path from each of the RelOptInfos created
+in the first phase. Append/Merge append paths with different possible
+parameterizations and pathkeys are created based on the paths created for joins
+between partitions.
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 99b6bc8..ee737f2 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -11,22 +11,24 @@
  *	  src/backend/optimizer/path/allpaths.c
  *
  *-------------------------------------------------------------------------
  */
 
 #include "postgres.h"
 
 #include <limits.h>
 #include <math.h>
 
+#include "miscadmin.h"
 #include "access/sysattr.h"
 #include "access/tsmapi.h"
+#include "catalog/partition.h"
 #include "catalog/pg_class.h"
 #include "catalog/pg_operator.h"
 #include "catalog/pg_proc.h"
 #include "foreign/fdwapi.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
 #ifdef OPTIMIZER_DEBUG
 #include "nodes/print.h"
 #endif
 #include "optimizer/clauses.h"
@@ -37,20 +39,21 @@
 #include "optimizer/plancat.h"
 #include "optimizer/planner.h"
 #include "optimizer/prep.h"
 #include "optimizer/restrictinfo.h"
 #include "optimizer/tlist.h"
 #include "optimizer/var.h"
 #include "parser/parse_clause.h"
 #include "parser/parsetree.h"
 #include "rewrite/rewriteManip.h"
 #include "utils/lsyscache.h"
+#include "utils/rel.h"
 
 
 /* results of subquery_is_pushdown_safe */
 typedef struct pushdown_safety_info
 {
 	bool	   *unsafeColumns;	/* which output columns are unsafe to use */
 	bool		unsafeVolatile; /* don't push down volatile quals */
 	bool		unsafeLeaky;	/* don't push down leaky quals */
 } pushdown_safety_info;
 
@@ -119,20 +122,24 @@ static void check_output_expressions(Query *subquery,
 static void compare_tlist_datatypes(List *tlist, List *colTypes,
 						pushdown_safety_info *safetyInfo);
 static bool targetIsInAllPartitionLists(TargetEntry *tle, Query *query);
 static bool qual_is_pushdown_safe(Query *subquery, Index rti, Node *qual,
 					  pushdown_safety_info *safetyInfo);
 static void subquery_push_qual(Query *subquery,
 				   RangeTblEntry *rte, Index rti, Node *qual);
 static void recurse_push_qual(Node *setOp, Query *topquery,
 				  RangeTblEntry *rte, Index rti, Node *qual);
 static void remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel);
+static void add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
+									List *live_childrels);
+static void generate_partition_wise_join_paths(PlannerInfo *root,
+											   RelOptInfo *rel);
 
 
 /*
  * make_one_rel
  *	  Finds all possible access paths for executing a query, returning a
  *	  single rel that represents the join of all base rels in the query.
  */
 RelOptInfo *
 make_one_rel(PlannerInfo *root, List *joinlist)
 {
@@ -861,20 +868,44 @@ static void
 set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 					Index rti, RangeTblEntry *rte)
 {
 	int			parentRTindex = rti;
 	bool		has_live_children;
 	double		parent_rows;
 	double		parent_size;
 	double	   *parent_attrsizes;
 	int			nattrs;
 	ListCell   *l;
+	Oid		   *part_oids = NULL;
+	int			nparts = 0;
+
+	/* Fetch the number of partitions of a partitioned table and their Oids. */
+	if (rel->part_scheme)
+	{
+		RangeTblEntry	*rte = root->simple_rte_array[rel->relid];
+
+		/*
+		 * We need not lock the relation since it was already locked, either by
+		 * the rewriter or when expand_inherited_rtentry() added it to the
+		 * query's rangetable.
+		 */
+		Relation	relation = heap_open(rte->relid, NoLock);
+		PartitionDesc	part_desc = RelationGetPartitionDesc(relation);
+
+		part_oids = part_desc->oids;
+		nparts = part_desc->nparts;
+
+		Assert(part_oids && nparts > 0);
+
+		rel->part_rels = (RelOptInfo **)palloc0(sizeof(RelOptInfo *) * nparts);
+		heap_close(relation, NoLock);
+	}
 
 	/*
 	 * Initialize to compute size estimates for whole append relation.
 	 *
 	 * We handle width estimates by weighting the widths of different child
 	 * rels proportionally to their number of rows.  This is sensible because
 	 * the use of width estimates is mainly to compute the total relation
 	 * "footprint" if we have to sort or hash it.  To do this, we sum the
 	 * total equivalent size (in "double" arithmetic) and then divide by the
 	 * total rowcount estimate.  This is done separately for the total rel
@@ -892,35 +923,84 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 	foreach(l, root->append_rel_list)
 	{
 		AppendRelInfo *appinfo = (AppendRelInfo *) lfirst(l);
 		int			childRTindex;
 		RangeTblEntry *childRTE;
 		RelOptInfo *childrel;
 		List	   *childquals;
 		Node	   *childqual;
 		ListCell   *parentvars;
 		ListCell   *childvars;
+		int			cnt_parts;
 
 		/* append_rel_list contains all append rels; ignore others */
 		if (appinfo->parent_relid != parentRTindex)
 			continue;
 
 		childRTindex = appinfo->child_relid;
 		childRTE = root->simple_rte_array[childRTindex];
 
 		/*
 		 * The child rel's RelOptInfo was already created during
 		 * add_base_rels_to_query.
 		 */
 		childrel = find_base_rel(root, childRTindex);
+
+		/*
+		 * Recursively save topmost parent's relid in RelOptInfos of
+		 * partitions.
+		 */
+		if (rel->top_parent_relids)
+			childrel->top_parent_relids = rel->top_parent_relids;
+		else
+			childrel->top_parent_relids = bms_copy(rel->relids);
+
 		Assert(childrel->reloptkind == RELOPT_OTHER_MEMBER_REL);
 
+
+		/*
+		 * For two partitioned tables with the same partitioning scheme, it is
+		 * assumed that the Oids of matching partitions from both the tables
+		 * are placed at the same position in the array of partition oids in
+		 * respective partition descriptors. Saving the RelOptInfo of a
+		 * partition in the same location as its Oid makes it easy to find the
+		 * RelOptInfos of matching partitions for partition-wise join.
+		 */
+		for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+		{
+			if (part_oids[cnt_parts] == childRTE->relid)
+			{
+				Assert(!rel->part_rels[cnt_parts]);
+				rel->part_rels[cnt_parts] = childrel;
+			}
+		}
+
+		/*
+		 * Copy/Modify targetlist. Partition-wise join technique may join this
+		 * child with a child of another partitioned table, such that this
+		 * child forms the nullable side of the outer join. In such a case, we
+		 * will need the targetlist of this child, even if it's deemed empty.
+		 * Hence set the targetlist before bailing out in case the child is
+		 * proven empty.
+		 *
+		 * NB: the resulting childrel->reltarget->exprs may contain arbitrary
+		 * expressions, which otherwise would not occur in a rel's targetlist.
+		 * Code that might be looking at an appendrel child must cope with
+		 * such.  (Normally, a rel's targetlist would only include Vars and
+		 * PlaceHolderVars.)  XXX we do not bother to update the cost or width
+		 * fields of childrel->reltarget; not clear if that would be useful.
+		 */
+		childrel->reltarget->exprs = (List *)
+			adjust_appendrel_attrs(root,
+								   (Node *) rel->reltarget->exprs,
+								   appinfo);
+
 		/*
 		 * We have to copy the parent's targetlist and quals to the child,
 		 * with appropriate substitution of variables.  However, only the
 		 * baserestrictinfo quals are needed before we can check for
 		 * constraint exclusion; so do that first and then check to see if we
 		 * can disregard this child.
 		 *
 		 * As of 8.4, the child rel's targetlist might contain non-Var
 		 * expressions, which means that substitution into the quals could
 		 * produce opportunities for const-simplification, and perhaps even
@@ -953,38 +1033,25 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		if (relation_excluded_by_constraints(root, childrel, childRTE))
 		{
 			/*
 			 * This child need not be scanned, so we can omit it from the
 			 * appendrel.
 			 */
 			set_dummy_rel_pathlist(childrel);
 			continue;
 		}
 
-		/*
-		 * CE failed, so finish copying/modifying targetlist and join quals.
-		 *
-		 * NB: the resulting childrel->reltarget->exprs may contain arbitrary
-		 * expressions, which otherwise would not occur in a rel's targetlist.
-		 * Code that might be looking at an appendrel child must cope with
-		 * such.  (Normally, a rel's targetlist would only include Vars and
-		 * PlaceHolderVars.)  XXX we do not bother to update the cost or width
-		 * fields of childrel->reltarget; not clear if that would be useful.
-		 */
+		/* CE failed, so finish copying/modifying join quals. */
 		childrel->joininfo = (List *)
 			adjust_appendrel_attrs(root,
 								   (Node *) rel->joininfo,
 								   appinfo);
-		childrel->reltarget->exprs = (List *)
-			adjust_appendrel_attrs(root,
-								   (Node *) rel->reltarget->exprs,
-								   appinfo);
 
 		/*
 		 * We have to make child entries in the EquivalenceClass data
 		 * structures as well.  This is needed either if the parent
 		 * participates in some eclass joins (because we will want to consider
 		 * inner-indexscan joins on the individual children) or if the parent
 		 * has useful pathkeys (because we should try to build MergeAppend
 		 * paths that produce those sort orderings).
 		 */
 		if (rel->has_eclass_joins || has_useful_pathkeys(root, rel))
@@ -1073,20 +1140,30 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 				}
 				if (child_width <= 0)
 					child_width = get_typavgwidth(exprType(childvar),
 												  exprTypmod(childvar));
 				Assert(child_width > 0);
 				parent_attrsizes[pndx] += child_width * childrel->rows;
 			}
 		}
 	}
 
+	/* Should have found all the childrels of a partitioned relation. */
+	if (rel->part_scheme)
+	{
+		int		cnt_parts;
+		for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+			if (!rel->part_rels[cnt_parts])
+				elog(ERROR, "could not find the RelOptInfo of a partition with oid %u",
+					 part_oids[cnt_parts]);
+	}
+
 	if (has_live_children)
 	{
 		/*
 		 * Save the finished size estimates.
 		 */
 		int			i;
 
 		Assert(parent_rows > 0);
 		rel->rows = parent_rows;
 		rel->reltarget->width = rint(parent_size / parent_rows);
@@ -1115,41 +1192,32 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 /*
  * set_append_rel_pathlist
  *	  Build access paths for an "append relation"
  */
 static void
 set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 						Index rti, RangeTblEntry *rte)
 {
 	int			parentRTindex = rti;
 	List	   *live_childrels = NIL;
-	List	   *subpaths = NIL;
-	bool		subpaths_valid = true;
-	List	   *partial_subpaths = NIL;
-	bool		partial_subpaths_valid = true;
-	List	   *all_child_pathkeys = NIL;
-	List	   *all_child_outers = NIL;
 	ListCell   *l;
 
 	/*
-	 * Generate access paths for each member relation, and remember the
-	 * cheapest path for each one.  Also, identify all pathkeys (orderings)
-	 * and parameterizations (required_outer sets) available for the member
-	 * relations.
+	 * Generate access paths for each member relation and remember the
+	 * non-dummy children.
 	 */
 	foreach(l, root->append_rel_list)
 	{
 		AppendRelInfo *appinfo = (AppendRelInfo *) lfirst(l);
 		int			childRTindex;
 		RangeTblEntry *childRTE;
 		RelOptInfo *childrel;
-		ListCell   *lcp;
 
 		/* append_rel_list contains all append rels; ignore others */
 		if (appinfo->parent_relid != parentRTindex)
 			continue;
 
 		/* Re-locate the child RTE and RelOptInfo */
 		childRTindex = appinfo->child_relid;
 		childRTE = root->simple_rte_array[childRTindex];
 		childrel = root->simple_rel_array[childRTindex];
 
@@ -1170,20 +1238,70 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 		/*
 		 * If child is dummy, ignore it.
 		 */
 		if (IS_DUMMY_REL(childrel))
 			continue;
 
 		/*
 		 * Child is live, so add it to the live_childrels list for use below.
 		 */
 		live_childrels = lappend(live_childrels, childrel);
+	}
+
+	/* Add Append/MergeAppend paths to the "append" relation. */
+	add_paths_to_append_rel(root, rel, live_childrels);
+}
+
+/*
+ * add_paths_to_append_rel
+ *		Generate Append/MergeAppend paths for given "append" relation. An
+ *		"append" relation can be a base parent relation or a join between
+ *		partitioned tables.
+ *
+ * The function collects all parameterizations and orderings supported by the
+ * non-dummy children. For every such parameterization or ordering, it creates
+ * an append path collecting one path from each non-dummy child with given
+ * parameterization or ordering. Similarly it collects partial paths from
+ * non-dummy children to create partial append paths.
+ */
+static void
+add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
+						List *live_childrels)
+{
+	List	   *subpaths = NIL;
+	bool		subpaths_valid = true;
+	List	   *partial_subpaths = NIL;
+	bool		partial_subpaths_valid = true;
+	List	   *all_child_pathkeys = NIL;
+	List	   *all_child_outers = NIL;
+	ListCell   *l;
+
+	/* An append relation with all its children dummy is dummy. */
+	if (live_childrels == NIL)
+	{
+		/* Mark the relation as dummy, if not already done so. */
+		if (!IS_DUMMY_REL(rel))
+			set_dummy_rel_pathlist(rel);
+
+		/* No more paths need to be added. */
+		return;
+	}
+
+	/*
+	 * For every non-dummy child, remember the cheapest path.  Also, identify
+	 * all pathkeys (orderings) and parameterizations (required_outer sets)
+	 * available for the non-dummy member relations.
+	 */
+	foreach (l, live_childrels)
+	{
+		RelOptInfo *childrel = lfirst(l);
+		ListCell   *lcp;
 
 		/*
 		 * If child has an unparameterized cheapest-total path, add that to
 		 * the unparameterized Append path we are constructing for the parent.
 		 * If not, there's no workable unparameterized path.
 		 */
 		if (childrel->cheapest_total_path->param_info == NULL)
 			subpaths = accumulate_append_subpath(subpaths,
 											  childrel->cheapest_total_path);
 		else
@@ -2181,27 +2299,34 @@ standard_join_search(PlannerInfo *root, int levels_needed, List *initial_rels)
 		 * Determine all possible pairs of relations to be joined at this
 		 * level, and build paths for making each one from every available
 		 * pair of lower-level relations.
 		 */
 		join_search_one_level(root, lev);
 
 		/*
 		 * Run generate_gather_paths() for each just-processed joinrel.  We
 		 * could not do this earlier because both regular and partial paths
 		 * can get added to a particular joinrel at multiple times within
+		 * join_search_one_level.  Similarly, create append paths for joinrels
+		 * which used partition-wise join technique.  We can not do this
+		 * earlier because the paths can get added to a relation representing
+		 * join between children at multiple times within
 		 * join_search_one_level.  After that, we're done creating paths for
 		 * the joinrel, so run set_cheapest().
 		 */
 		foreach(lc, root->join_rel_level[lev])
 		{
 			rel = (RelOptInfo *) lfirst(lc);
 
+			/* Create Append/MergeAppend paths for partition-wise joins. */
+			generate_partition_wise_join_paths(root, rel);
+
 			/* Create GatherPaths for any useful partial paths for rel */
 			generate_gather_paths(root, rel);
 
 			/* Find and save the cheapest paths for this rel */
 			set_cheapest(rel);
 
 #ifdef OPTIMIZER_DEBUG
 			debug_print_rel(root, rel);
 #endif
 		}
@@ -2851,20 +2976,92 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
 		 * OK, we don't need it.  Replace the expression with a NULL constant.
 		 * Preserve the exposed type of the expression, in case something
 		 * looks at the rowtype of the subquery's result.
 		 */
 		tle->expr = (Expr *) makeNullConst(exprType(texpr),
 										   exprTypmod(texpr),
 										   exprCollation(texpr));
 	}
 }
 
+/*
+ * generate_partition_wise_join_paths
+ * 		Create appends paths for given join relation, if partition-wise join
+ * 		technique was used for this join.
+ *
+ * The function collects the non-dummy children and hands them off to
+ * add_paths_to_append_rel(), which does the actual work.
+ *
+ * This must not be called until after we're done creating all paths for the
+ * join between children for specified join between parents. (Otherwise,
+ * add_path might delete a path that some Append/MergeAppend path has a
+ * reference to.)
+ */
+
+static void
+generate_partition_wise_join_paths(PlannerInfo *root, RelOptInfo *rel)
+{
+	List	   *live_children = NIL;
+	int			cnt_part;
+
+	/* Handle only join relations. */
+	if (!IS_JOIN_REL(rel))
+		return;
+
+	/*
+	 * If partition-wise join technique was not used for any of the join
+	 * orders, the join is not partitioned. Reset the partitioning scheme.
+	 * TODO: find a condition when some joining order would use partition-wise
+	 * join technique and other wouldn't.
+	 */
+	if (!rel->part_rels)
+		rel->part_scheme = NULL;
+
+	/* If the relation is not partitioned or is proven dummy, nothing to do. */
+	if (!rel->part_scheme || IS_DUMMY_REL(rel))
+		return;
+
+	/* Guard against stack overflow due to overly deep partition hierarchy. */
+	check_stack_depth();
+
+	for (cnt_part = 0; cnt_part < rel->part_scheme->nparts; cnt_part++)
+	{
+		RelOptInfo	*child_rel = rel->part_rels[cnt_part];
+
+		/* Ignore dummy child. */
+		if (!IS_DUMMY_REL(child_rel))
+		{
+			/* Recursively collect the paths from child joinrel. */
+			generate_partition_wise_join_paths(root, child_rel);
+
+			/* Find the cheapest of the paths for this rel. */
+			set_cheapest(child_rel);
+
+#ifdef OPTIMIZER_DEBUG
+			debug_print_rel(root, rel);
+#endif
+
+			live_children = lappend(live_children, child_rel);
+		}
+	}
+
+	/*
+	 * Create append paths by collecting subpaths from live children. Even if
+	 * there are no live children, we should create an append path with no
+	 * subpaths i.e. a dummy access path.
+	 */
+	add_paths_to_append_rel(root, rel, live_children);
+
+	if (live_children)
+		list_free(live_children);
+}
+
 /*****************************************************************************
  *			DEBUG SUPPORT
  *****************************************************************************/
 
 #ifdef OPTIMIZER_DEBUG
 
 static void
 print_relids(PlannerInfo *root, Relids relids)
 {
 	int			x;
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 2a49639..a23da1c 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -119,20 +119,21 @@ bool		enable_seqscan = true;
 bool		enable_indexscan = true;
 bool		enable_indexonlyscan = true;
 bool		enable_bitmapscan = true;
 bool		enable_tidscan = true;
 bool		enable_sort = true;
 bool		enable_hashagg = true;
 bool		enable_nestloop = true;
 bool		enable_material = true;
 bool		enable_mergejoin = true;
 bool		enable_hashjoin = true;
+bool		enable_partition_wise_join = true;
 
 typedef struct
 {
 	PlannerInfo *root;
 	QualCost	total;
 } cost_qual_eval_context;
 
 static List *extract_nonindex_conditions(List *qual_clauses, List *indexquals);
 static MergeScanSelCache *cached_scansel(PlannerInfo *root,
 			   RestrictInfo *rinfo,
diff --git a/src/backend/optimizer/path/equivclass.c b/src/backend/optimizer/path/equivclass.c
index 0e50ad5..73026a3 100644
--- a/src/backend/optimizer/path/equivclass.c
+++ b/src/backend/optimizer/path/equivclass.c
@@ -2359,20 +2359,22 @@ eclass_useful_for_merging(PlannerInfo *root,
 
 	/*
 	 * Note we don't test ec_broken; if we did, we'd need a separate code path
 	 * to look through ec_sources.  Checking the members anyway is OK as a
 	 * possibly-overoptimistic heuristic.
 	 */
 
 	/* If specified rel is a child, we must consider the topmost parent rel */
 	if (rel->reloptkind == RELOPT_OTHER_MEMBER_REL)
 		relids = find_childrel_top_parent(root, rel)->relids;
+	else if (rel->reloptkind == RELOPT_OTHER_JOINREL)
+		relids = rel->top_parent_relids;
 	else
 		relids = rel->relids;
 
 	/* If rel already includes all members of eclass, no point in searching */
 	if (bms_is_subset(eclass->ec_relids, relids))
 		return false;
 
 	/* To join, we need a member not in the given rel */
 	foreach(lc, eclass->ec_members)
 	{
diff --git a/src/backend/optimizer/path/joinpath.c b/src/backend/optimizer/path/joinpath.c
index cc7384f..fae15de 100644
--- a/src/backend/optimizer/path/joinpath.c
+++ b/src/backend/optimizer/path/joinpath.c
@@ -18,23 +18,33 @@
 
 #include "executor/executor.h"
 #include "foreign/fdwapi.h"
 #include "optimizer/cost.h"
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
 
 /* Hook for plugins to get control in add_paths_to_joinrel() */
 set_join_pathlist_hook_type set_join_pathlist_hook = NULL;
 
-#define PATH_PARAM_BY_REL(path, rel)  \
+/*
+ * Paths parameterized by the parent can be considered to be parameterized by
+ * any of its child.
+ */
+#define PATH_PARAM_BY_PARENT(path, rel)	\
+	((path)->param_info && bms_overlap(PATH_REQ_OUTER(path),	\
+									   (rel)->top_parent_relids))
+#define PATH_PARAM_BY_REL_SELF(path, rel)  \
 	((path)->param_info && bms_overlap(PATH_REQ_OUTER(path), (rel)->relids))
 
+#define PATH_PARAM_BY_REL(path, rel)	\
+	(PATH_PARAM_BY_REL_SELF(path, rel) || PATH_PARAM_BY_PARENT(path, rel))
+
 static void sort_inner_and_outer(PlannerInfo *root, RelOptInfo *joinrel,
 					 RelOptInfo *outerrel, RelOptInfo *innerrel,
 					 JoinType jointype, JoinPathExtraData *extra);
 static void match_unsorted_outer(PlannerInfo *root, RelOptInfo *joinrel,
 					 RelOptInfo *outerrel, RelOptInfo *innerrel,
 					 JoinType jointype, JoinPathExtraData *extra);
 static void consider_parallel_nestloop(PlannerInfo *root,
 						   RelOptInfo *joinrel,
 						   RelOptInfo *outerrel,
 						   RelOptInfo *innerrel,
@@ -125,38 +135,51 @@ add_paths_to_joinrel(PlannerInfo *root,
 	 * directly to the parameter source rel instead of joining to the other
 	 * input rel.  (But see allow_star_schema_join().)	This restriction
 	 * reduces the number of parameterized paths we have to deal with at
 	 * higher join levels, without compromising the quality of the resulting
 	 * plan.  We express the restriction as a Relids set that must overlap the
 	 * parameterization of any proposed join path.
 	 */
 	foreach(lc, root->join_info_list)
 	{
 		SpecialJoinInfo *sjinfo = (SpecialJoinInfo *) lfirst(lc);
+		Relids	joinrelids;
+
+		/*
+		 * PlannerInfo doesn't contain the SpecialJoinInfos created for joins
+		 * between child relations, even if there is a SpecialJoinInfo node for
+		 * the join between the topmost parents. Hence while calculating Relids
+		 * set representing the restriction, consider relids of topmost parent
+		 * of partitions.
+		 */
+		if (joinrel->reloptkind == RELOPT_OTHER_JOINREL)
+			joinrelids = joinrel->top_parent_relids;
+		else
+			joinrelids = joinrel->relids;
 
 		/*
 		 * SJ is relevant to this join if we have some part of its RHS
 		 * (possibly not all of it), and haven't yet joined to its LHS.  (This
 		 * test is pretty simplistic, but should be sufficient considering the
 		 * join has already been proven legal.)  If the SJ is relevant, it
 		 * presents constraints for joining to anything not in its RHS.
 		 */
-		if (bms_overlap(joinrel->relids, sjinfo->min_righthand) &&
-			!bms_overlap(joinrel->relids, sjinfo->min_lefthand))
+		if (bms_overlap(joinrelids, sjinfo->min_righthand) &&
+			!bms_overlap(joinrelids, sjinfo->min_lefthand))
 			extra.param_source_rels = bms_join(extra.param_source_rels,
 										   bms_difference(root->all_baserels,
 													 sjinfo->min_righthand));
 
 		/* full joins constrain both sides symmetrically */
 		if (sjinfo->jointype == JOIN_FULL &&
-			bms_overlap(joinrel->relids, sjinfo->min_lefthand) &&
-			!bms_overlap(joinrel->relids, sjinfo->min_righthand))
+			bms_overlap(joinrelids, sjinfo->min_lefthand) &&
+			!bms_overlap(joinrelids, sjinfo->min_righthand))
 			extra.param_source_rels = bms_join(extra.param_source_rels,
 										   bms_difference(root->all_baserels,
 													  sjinfo->min_lefthand));
 	}
 
 	/*
 	 * However, when a LATERAL subquery is involved, there will simply not be
 	 * any paths for the joinrel that aren't parameterized by whatever the
 	 * subquery is parameterized by, unless its parameterization is resolved
 	 * within the joinrel.  So we might as well allow additional dependencies
@@ -272,20 +295,38 @@ try_nestloop_path(PlannerInfo *root,
 				  Path *outer_path,
 				  Path *inner_path,
 				  List *pathkeys,
 				  JoinType jointype,
 				  JoinPathExtraData *extra)
 {
 	Relids		required_outer;
 	JoinCostWorkspace workspace;
 
 	/*
+	 * For a join between child relations, if the inner path is parameterized
+	 * by the parent of the outer relation, it can be considered to be
+	 * parameterized by the outer relation. We will be able to create a
+	 * nestloop join path with inner relation parameterized by the outer
+	 * relation by translating the inner path to be parameterized by the outer
+	 * child relation.
+	 */
+	if (PATH_PARAM_BY_PARENT(inner_path, outer_path->parent))
+	{
+		inner_path = reparameterize_path_by_child(root, inner_path,
+												   outer_path->parent);
+
+		/* If we could not translate the path, don't produce nest loop path. */
+		if (!inner_path)
+			return;
+	}
+
+	/*
 	 * Check to see if proposed path is still parameterized, and reject if the
 	 * parameterization wouldn't be sensible --- unless allow_star_schema_join
 	 * says to allow it anyway.  Also, we must reject if have_dangerous_phv
 	 * doesn't like the look of it, which could only happen if the nestloop is
 	 * still parameterized.
 	 */
 	required_outer = calc_nestloop_required_outer(outer_path,
 												  inner_path);
 	if (required_outer &&
 		((!bms_overlap(required_outer, extra->param_source_rels) &&
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 01d4fea..da8ad83 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -7,38 +7,57 @@
  * Portions Copyright (c) 1994, Regents of the University of California
  *
  *
  * IDENTIFICATION
  *	  src/backend/optimizer/path/joinrels.c
  *
  *-------------------------------------------------------------------------
  */
 #include "postgres.h"
 
+#include "miscadmin.h"
+#include "catalog/partition.h"
+#include "optimizer/clauses.h"
 #include "optimizer/joininfo.h"
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
+#include "optimizer/prep.h"
+#include "optimizer/cost.h"
 #include "utils/memutils.h"
 
 
 static void make_rels_by_clause_joins(PlannerInfo *root,
 						  RelOptInfo *old_rel,
 						  ListCell *other_rels);
 static void make_rels_by_clauseless_joins(PlannerInfo *root,
 							  RelOptInfo *old_rel,
 							  ListCell *other_rels);
 static bool has_join_restriction(PlannerInfo *root, RelOptInfo *rel);
 static bool has_legal_joinclause(PlannerInfo *root, RelOptInfo *rel);
 static bool is_dummy_rel(RelOptInfo *rel);
 static void mark_dummy_rel(RelOptInfo *rel);
 static bool restriction_is_constant_false(List *restrictlist,
 							  bool only_pushed_down);
+static void populate_joinrel_with_paths(PlannerInfo *root, RelOptInfo *rel1,
+							RelOptInfo *rel2, RelOptInfo *joinrel,
+							SpecialJoinInfo *sjinfo, List *restrictlist);
+static void try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1,
+						  RelOptInfo *rel2, RelOptInfo *joinrel,
+						  SpecialJoinInfo *parent_sjinfo,
+						  List *parent_restrictlist);
+static SpecialJoinInfo * build_child_join_sjinfo(PlannerInfo *root,
+										 SpecialJoinInfo *parent_sjinfo,
+										 List *append_rel_infos1,
+										 List *append_rel_infos2);
+static bool have_partkey_equi_join(RelOptInfo *rel1, RelOptInfo *rel2,
+						   JoinType jointype, List *restrictlist);
+static int match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel);
 
 
 /*
  * join_search_one_level
  *	  Consider ways to produce join relations containing exactly 'level'
  *	  jointree items.  (This is one step of the dynamic-programming method
  *	  embodied in standard_join_search.)  Join rel nodes for each feasible
  *	  combination of lower-level rels are created and returned in a list.
  *	  Implementation paths are created for each such joinrel, too.
  *
@@ -717,20 +736,44 @@ make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 	/*
 	 * If we've already proven this join is empty, we needn't consider any
 	 * more paths for it.
 	 */
 	if (is_dummy_rel(joinrel))
 	{
 		bms_free(joinrelids);
 		return joinrel;
 	}
 
+	/* Add paths to the join relation. */
+	populate_joinrel_with_paths(root, rel1, rel2, joinrel, sjinfo,
+								restrictlist);
+
+	/* Apply partition-wise join technique, if possible. */
+	try_partition_wise_join(root, rel1, rel2, joinrel, sjinfo, restrictlist);
+
+	bms_free(joinrelids);
+
+	return joinrel;
+}
+
+/*
+ * populate_joinrel_with_paths
+ *	  Add paths joining given input relations to the given joinrel. The
+ *	  SpecialJoinInfo provides details about the join and the restrictlist
+ *	  contains the join clauses and the other clauses applicable for given pair
+ *	  of the joining relations.
+ */
+static void
+populate_joinrel_with_paths(PlannerInfo *root, RelOptInfo *rel1,
+							RelOptInfo *rel2, RelOptInfo *joinrel,
+							SpecialJoinInfo *sjinfo, List *restrictlist)
+{
 	/*
 	 * Consider paths using each rel as both outer and inner.  Depending on
 	 * the join type, a provably empty outer or inner rel might mean the join
 	 * is provably empty too; in which case throw away any previously computed
 	 * paths and mark the join as dummy.  (We do it this way since it's
 	 * conceivable that dummy-ness of a multi-element join might only be
 	 * noticeable for certain construction paths.)
 	 *
 	 * Also, a provably constant-false join restriction typically means that
 	 * we can skip evaluating one or both sides of the join.  We do this by
@@ -861,27 +904,22 @@ make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 				mark_dummy_rel(rel2);
 			add_paths_to_joinrel(root, joinrel, rel1, rel2,
 								 JOIN_ANTI, sjinfo,
 								 restrictlist);
 			break;
 		default:
 			/* other values not expected here */
 			elog(ERROR, "unrecognized join type: %d", (int) sjinfo->jointype);
 			break;
 	}
-
-	bms_free(joinrelids);
-
-	return joinrel;
 }
 
-
 /*
  * have_join_order_restriction
  *		Detect whether the two relations should be joined to satisfy
  *		a join-order restriction arising from special or lateral joins.
  *
  * In practice this is always used with have_relevant_joinclause(), and so
  * could be merged with that function, but it seems clearer to separate the
  * two concerns.  We need this test because there are degenerate cases where
  * a clauseless join must be performed to satisfy join-order restrictions.
  * Also, if one rel has a lateral reference to the other, or both are needed
@@ -1242,10 +1280,378 @@ restriction_is_constant_false(List *restrictlist, bool only_pushed_down)
 
 			/* constant NULL is as good as constant FALSE for our purposes */
 			if (con->constisnull)
 				return true;
 			if (!DatumGetBool(con->constvalue))
 				return true;
 		}
 	}
 	return false;
 }
+
+/*
+ * TODO: In addition, the comments for individual comments and chunks of code
+ * need to do a better job explaining how each part of the patch contributes to
+ * the overall picture. Make a distinction between the partition-wise join as a
+ * join performed by breaking down join between partitioned tables and
+ * other-join-rel which is anologous to other-base-rel.
+ */
+
+/*
+ * Assess whether join between given two partitioned relations can be broken
+ * down into joins between matching partitions; a technique called
+ * "partition-wise join"
+ *
+ * Partition-wise 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.
+ *
+ * Partition-wise join is planned in two phases
+ *
+ * 1. Create the RelOptInfos for joins between matching partitions and add join
+ * paths to those. This function is responsible for this phase.
+ *
+ * 2. Add Append/MergeAppend paths to the RelOptInfo representing the join
+ * between the partitioned relations by choosing one path from each of the
+ * RelOptInfos created in the first phase. The second phase is implemented by
+ * generate_partition_wise_join_paths().
+ *
+ * The RelOptInfo, SpecialJoinInfo and restrictlist for each child join are
+ * obtained by translating the respective parent join structures.
+ */
+static void
+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;
+
+
+	/* Nothing to do, if the join relation is not partitioned. */
+	if (!joinrel->part_scheme)
+		return;
+
+	/*
+	 * If any of the joining relation is proven empty, either the join will be
+	 * empty (INNER join) or will have the inner side all nullified. We take
+	 * care of such cases when creating join paths for parent relations.
+	 * Nothing to be done here. Also, nothing to do, if the join is proven
+	 * empty.
+	 */
+	if (IS_DUMMY_REL(rel1) || IS_DUMMY_REL(rel2) || IS_DUMMY_REL(joinrel))
+		return;
+
+	/*
+	 * Partitioning scheme in join relation indicates a possibilty that the
+	 * join may be partitioned, but it's not necessary that every join order
+	 * can use partition-wise join technique. If one of joining relations turns
+	 * out to be unpartitioned, this pair of joining relations can not use
+	 * partition-wise join technique.
+	 */
+	if (!rel1->part_scheme || !rel2->part_scheme)
+		return;
+
+	/*
+	 * If an equi-join condition between the partition keys of the joining
+	 * relations does not exist, this pair of joining relations can not use
+	 * partition-wise technique.
+	 */
+	if (!have_partkey_equi_join(rel1, rel2, parent_sjinfo->jointype,
+								parent_restrictlist))
+		return;
+
+	/*
+	 * 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);
+
+	/* Make sure we have RelOptInfos of the partitions available. */
+	Assert(rel1->part_rels && rel2->part_rels);
+
+	/* Guard against stack overflow due to overly deep partition hierarchy. */
+	check_stack_depth();
+
+	part_scheme = joinrel->part_scheme;
+	nparts = part_scheme->nparts;
+
+	elog(DEBUG3, "join between relations %s and %s is considered for partition-wise join.",
+		 bmsToString(rel1->relids), bmsToString(rel2->relids));
+
+	/* Allocate the array for child RelOptInfos if not done already. */
+	if (!joinrel->part_rels)
+		joinrel->part_rels = (RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
+
+	/*
+	 * This joinrel is partitioned, so iterate over the partitions and create
+	 * paths for each one, allowing us to eventually build an append-of-joins
+	 * path for the parent. Since this routine may be called multiple times
+	 * for various join orders, the RelOptInfo needed for each child join may
+	 * or may not already exist, but the paths for this join order definitely
+	 * do not. Note that we don't create any actual AppendPath at this stage;
+	 * it only makes sense to do that at the end, after each possible join
+	 * order has been considered for each child join.  The best join order may
+	 * differ from child to child.
+	 */
+	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	{
+		RelOptInfo *child_joinrel;
+		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
+		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
+		SpecialJoinInfo *child_sjinfo;
+		List	   *child_restrictlist;
+		List	   *join_appinfos;
+		List	   *appinfos1;
+		List	   *appinfos2;
+
+		/* We should have a valid RelOptInfo for partitions being joined. */
+		Assert(child_rel1 && child_rel2);
+
+		/* We should never try to join two overlapping sets of rels. */
+		Assert(!bms_overlap(child_rel1->relids, child_rel2->relids));
+
+		appinfos1 = find_appinfos_by_relids(root, child_rel1->relids);
+		appinfos2 = find_appinfos_by_relids(root, child_rel2->relids);
+
+		/*
+		 * Construct SpecialJoinInfo from parent join relations's
+		 * SpecialJoinInfo.
+		 */
+		child_sjinfo = build_child_join_sjinfo(root, parent_sjinfo,
+												   appinfos1, appinfos2);
+
+		/* Construct the parent-child relid map for the join relation. */
+		join_appinfos = list_concat(appinfos1, appinfos2);
+
+		/*
+		 * Construct restrictions applicable to the child join from
+		 * those applicable to the parent join.
+		 */
+		child_restrictlist = (List *) adjust_join_appendrel_attrs(root,
+												   (Node *)parent_restrictlist,
+															   join_appinfos);
+
+		child_joinrel = joinrel->part_rels[cnt_parts];
+
+		/* Construct the join relation for given partition of the join. */
+		if (!child_joinrel)
+		{
+			child_joinrel = build_child_join_rel(root, child_rel1,
+													 child_rel2, joinrel,
+													 child_sjinfo,
+													 child_restrictlist,
+													 join_appinfos);
+
+			joinrel->part_rels[cnt_parts] = child_joinrel;
+
+		}
+
+		/*
+		 * If we've already proven that this join is empty, we needn't consider
+		 * any more paths for it.
+		 */
+		if (is_dummy_rel(child_joinrel))
+			continue;
+
+		populate_joinrel_with_paths(root, child_rel1, child_rel2, child_joinrel,
+									child_sjinfo, child_restrictlist);
+
+		/*
+		 * If the child relations themselves are partitioned, try partition-wise join
+		 * recursively.
+		 */
+		try_partition_wise_join(root, child_rel1, child_rel2, child_joinrel,
+								child_sjinfo, child_restrictlist);
+	}
+}
+
+/*
+ * Construct the SpecialJoinInfo for the join between children by translating
+ * SpecialJoinInfo for the join between parents.
+ */
+static SpecialJoinInfo *
+build_child_join_sjinfo(PlannerInfo *root, SpecialJoinInfo *parent_sjinfo,
+							List *append_rel_infos1, List *append_rel_infos2)
+{
+	SpecialJoinInfo *sjinfo = copyObject(parent_sjinfo);
+
+	sjinfo->min_lefthand = adjust_child_relids(sjinfo->min_lefthand,
+											   append_rel_infos1);
+	sjinfo->min_righthand = adjust_child_relids(sjinfo->min_righthand,
+												append_rel_infos2);
+	sjinfo->syn_lefthand = adjust_child_relids(sjinfo->syn_lefthand,
+											   append_rel_infos1);
+	sjinfo->syn_righthand = adjust_child_relids(sjinfo->syn_righthand,
+												append_rel_infos2);
+
+	/* Replace the Var nodes of parent with those of children in expressions. */
+	sjinfo->semi_rhs_exprs = (List *) adjust_join_appendrel_attrs(root,
+											   (Node *) sjinfo->semi_rhs_exprs,
+															append_rel_infos2);
+	return sjinfo;
+}
+
+/*
+ * Replace parent relids by child relids in the given relid set.
+ */
+Relids
+adjust_child_relids(Relids relids, List *append_rel_infos)
+{
+	ListCell	*lc;
+	foreach (lc, append_rel_infos)
+	{
+		AppendRelInfo	*appinfo = lfirst(lc);
+
+		/* Remove parent, add child*/
+		if (bms_is_member(appinfo->parent_relid, relids))
+		{
+			relids = bms_del_member(relids, appinfo->parent_relid);
+			relids = bms_add_member(relids, appinfo->child_relid);
+		}
+	}
+	return relids;
+}
+
+/*
+ * Returns true if there exists an equi-join condition for each pair of
+ * partition key from given relations being joined.
+ */
+static bool
+have_partkey_equi_join(RelOptInfo *rel1, RelOptInfo *rel2,
+					 JoinType jointype, List *restrictlist)
+{
+	PartitionScheme	part_scheme = rel1->part_scheme;
+	ListCell	*lc;
+	int		cnt_pks;
+	int		num_pks;
+	bool   *pk_has_clause;
+
+	/*
+	 * This function should be called when the joining relations have same
+	 * partitioning scheme.
+	 */
+	Assert(rel1->part_scheme == rel2->part_scheme);
+	Assert(part_scheme);
+
+	num_pks = part_scheme->partnatts;
+
+	pk_has_clause = (bool *) palloc0(sizeof(bool) * num_pks);
+
+	foreach (lc, restrictlist)
+	{
+		RestrictInfo *rinfo = lfirst(lc);
+		OpExpr		 *opexpr;
+		Expr		 *expr1;
+		Expr		 *expr2;
+		int		ipk1;
+		int		ipk2;
+
+		/* If processing an outer join, only use its own join clauses. */
+		if (IS_OUTER_JOIN(jointype) && rinfo->is_pushed_down)
+			continue;
+
+		/* Skip clauses which can not be used for a join. */
+		if (!rinfo->can_join)
+			continue;
+
+		/* Skip clauses which are not equality conditions. */
+		if (rinfo->hashjoinoperator == InvalidOid && !rinfo->mergeopfamilies)
+			continue;
+
+		opexpr = (OpExpr *) rinfo->clause;
+		Assert(is_opclause(opexpr));
+
+
+		/* Match the operands to the relation. */
+		if (bms_is_subset(rinfo->left_relids, rel1->relids) &&
+			bms_is_subset(rinfo->right_relids, rel2->relids))
+		{
+			expr1 = linitial(opexpr->args);
+			expr2 = lsecond(opexpr->args);
+		}
+		else if (bms_is_subset(rinfo->left_relids, rel2->relids) &&
+				 bms_is_subset(rinfo->right_relids, rel1->relids))
+		{
+			expr1 = lsecond(opexpr->args);
+			expr2 = linitial(opexpr->args);
+		}
+		else
+			continue;
+
+		/* Associate matching clauses with partition keys. */
+		ipk1 = match_expr_to_partition_keys(expr1, rel1);
+		ipk2 = match_expr_to_partition_keys(expr2, rel2);
+
+		/*
+		 * If the clause refers to different partition keys from
+		 * both relations, it can not be used for partition-wise join.
+		 */
+		if (ipk1 != ipk2)
+			continue;
+
+		/*
+		 * The clause allows partition-wise join if only it uses the same
+		 * operator family as that specified by the partition key.
+		 */
+		if (!list_member_oid(rinfo->mergeopfamilies,
+							 part_scheme->partopfamily[ipk1]))
+			continue;
+
+		/* Mark the partition key as having an equi-join clause. */
+		pk_has_clause[ipk1] = true;
+	}
+
+	/* Check whether every partition key has an equi-join condition. */
+	for (cnt_pks = 0; cnt_pks < num_pks; cnt_pks++)
+	{
+		if (!pk_has_clause[cnt_pks])
+		{
+			pfree(pk_has_clause);
+			return false;
+		}
+	}
+
+	pfree(pk_has_clause);
+	return true;
+}
+
+/*
+ * Find the partition key from the given relation matching the given
+ * expression. If found, return the index of the partition key, else return -1.
+ */
+static int
+match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel)
+{
+	int		cnt_pks;
+	int		num_pks;
+
+	/* This function should be called only for partitioned relations. */
+	Assert(rel->part_scheme);
+
+	num_pks = rel->part_scheme->partnatts;
+
+	/*
+	 * Remove the relabel decoration. We can assume that there is at most one
+	 * RelabelType node; eval_const_expressions() simplifies multiple
+	 * RelabelType nodes into one.
+	 */
+	if (IsA(expr, RelabelType))
+		expr = (Expr *) ((RelabelType *) expr)->arg;
+
+	for (cnt_pks = 0; cnt_pks < num_pks; cnt_pks++)
+	{
+		List	 *pkexprs = rel->partexprs[cnt_pks];
+		ListCell *lc;
+
+		foreach(lc, pkexprs)
+		{
+			Expr *pkexpr = lfirst(lc);
+			if (equal(pkexpr, expr))
+				return cnt_pks;
+		}
+	}
+
+	return -1;
+}
diff --git a/src/backend/optimizer/path/pathkeys.c b/src/backend/optimizer/path/pathkeys.c
index 4436ac1..6996590 100644
--- a/src/backend/optimizer/path/pathkeys.c
+++ b/src/backend/optimizer/path/pathkeys.c
@@ -1081,26 +1081,38 @@ select_outer_pathkeys_for_merge(PlannerInfo *root,
 								List *mergeclauses,
 								RelOptInfo *joinrel)
 {
 	List	   *pathkeys = NIL;
 	int			nClauses = list_length(mergeclauses);
 	EquivalenceClass **ecs;
 	int		   *scores;
 	int			necs;
 	ListCell   *lc;
 	int			j;
+	Relids		relids;
 
 	/* Might have no mergeclauses */
 	if (nClauses == 0)
 		return NIL;
 
 	/*
+	 * For a child join relation, use parent relids to find potential
+	 * join partners from equivalence classes. A potential join partner of
+	 * parent also indicates potential join partner of the child. By using
+	 * parent relids we eliminate duplicates arising out of many children.
+	 */
+	if (joinrel->reloptkind == RELOPT_OTHER_JOINREL)
+		relids = joinrel->top_parent_relids;
+	else
+		relids = joinrel->relids;
+
+	/*
 	 * Make arrays of the ECs used by the mergeclauses (dropping any
 	 * duplicates) and their "popularity" scores.
 	 */
 	ecs = (EquivalenceClass **) palloc(nClauses * sizeof(EquivalenceClass *));
 	scores = (int *) palloc(nClauses * sizeof(int));
 	necs = 0;
 
 	foreach(lc, mergeclauses)
 	{
 		RestrictInfo *rinfo = (RestrictInfo *) lfirst(lc);
@@ -1126,21 +1138,21 @@ select_outer_pathkeys_for_merge(PlannerInfo *root,
 			continue;
 
 		/* compute score */
 		score = 0;
 		foreach(lc2, oeclass->ec_members)
 		{
 			EquivalenceMember *em = (EquivalenceMember *) lfirst(lc2);
 
 			/* Potential future join partner? */
 			if (!em->em_is_const && !em->em_is_child &&
-				!bms_overlap(em->em_relids, joinrel->relids))
+				!bms_overlap(em->em_relids, relids))
 				score++;
 		}
 
 		ecs[necs] = oeclass;
 		scores[necs] = score;
 		necs++;
 	}
 
 	/*
 	 * Find out if we have all the ECs mentioned in query_pathkeys; if so we
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 32f4031..b221e2c 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -235,21 +235,22 @@ static Plan *prepare_sort_from_pathkeys(Plan *lefttree, List *pathkeys,
 						   const AttrNumber *reqColIdx,
 						   bool adjust_tlist_in_place,
 						   int *p_numsortkeys,
 						   AttrNumber **p_sortColIdx,
 						   Oid **p_sortOperators,
 						   Oid **p_collations,
 						   bool **p_nullsFirst);
 static EquivalenceMember *find_ec_member_for_tle(EquivalenceClass *ec,
 					   TargetEntry *tle,
 					   Relids relids);
-static Sort *make_sort_from_pathkeys(Plan *lefttree, List *pathkeys);
+static Sort *make_sort_from_pathkeys(Plan *lefttree, List *pathkeys,
+									 Relids relids);
 static Sort *make_sort_from_groupcols(List *groupcls,
 						 AttrNumber *grpColIdx,
 						 Plan *lefttree);
 static Material *make_material(Plan *lefttree);
 static WindowAgg *make_windowagg(List *tlist, Index winref,
 			   int partNumCols, AttrNumber *partColIdx, Oid *partOperators,
 			   int ordNumCols, AttrNumber *ordColIdx, Oid *ordOperators,
 			   int frameOptions, Node *startOffset, Node *endOffset,
 			   Plan *lefttree);
 static Group *make_group(List *tlist, List *qual, int numGroupCols,
@@ -1507,21 +1508,21 @@ create_sort_plan(PlannerInfo *root, SortPath *best_path, int flags)
 	Plan	   *subplan;
 
 	/*
 	 * We don't want any excess columns in the sorted tuples, so request a
 	 * smaller tlist.  Otherwise, since Sort doesn't project, tlist
 	 * requirements pass through.
 	 */
 	subplan = create_plan_recurse(root, best_path->subpath,
 								  flags | CP_SMALL_TLIST);
 
-	plan = make_sort_from_pathkeys(subplan, best_path->path.pathkeys);
+	plan = make_sort_from_pathkeys(subplan, best_path->path.pathkeys, NULL);
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
 	return plan;
 }
 
 /*
  * create_group_plan
  *
  *	  Create a Group plan for 'best_path' and (recursively) plans
@@ -3517,31 +3518,33 @@ create_mergejoin_plan(PlannerInfo *root,
 	List	   *innerpathkeys;
 	int			nClauses;
 	Oid		   *mergefamilies;
 	Oid		   *mergecollations;
 	int		   *mergestrategies;
 	bool	   *mergenullsfirst;
 	int			i;
 	ListCell   *lc;
 	ListCell   *lop;
 	ListCell   *lip;
+	Path	   *outer_path = best_path->jpath.outerjoinpath;
+	Path	   *inner_path = best_path->jpath.innerjoinpath;
 
 	/*
 	 * MergeJoin can project, so we don't have to demand exact tlists from the
 	 * inputs.  However, if we're intending to sort an input's result, it's
 	 * best to request a small tlist so we aren't sorting more data than
 	 * necessary.
 	 */
-	outer_plan = create_plan_recurse(root, best_path->jpath.outerjoinpath,
+	outer_plan = create_plan_recurse(root, outer_path,
 					 (best_path->outersortkeys != NIL) ? CP_SMALL_TLIST : 0);
 
-	inner_plan = create_plan_recurse(root, best_path->jpath.innerjoinpath,
+	inner_plan = create_plan_recurse(root, inner_path,
 					 (best_path->innersortkeys != NIL) ? CP_SMALL_TLIST : 0);
 
 	/* Sort join qual clauses into best execution order */
 	/* NB: do NOT reorder the mergeclauses */
 	joinclauses = order_qual_clauses(root, best_path->jpath.joinrestrictinfo);
 
 	/* Get the join qual clauses (in plain expression form) */
 	/* Any pseudoconstant clauses are ignored here */
 	if (IS_OUTER_JOIN(best_path->jpath.jointype))
 	{
@@ -3573,48 +3576,52 @@ create_mergejoin_plan(PlannerInfo *root,
 		otherclauses = (List *)
 			replace_nestloop_params(root, (Node *) otherclauses);
 	}
 
 	/*
 	 * Rearrange mergeclauses, if needed, so that the outer variable is always
 	 * on the left; mark the mergeclause restrictinfos with correct
 	 * outer_is_left status.
 	 */
 	mergeclauses = get_switched_clauses(best_path->path_mergeclauses,
-							 best_path->jpath.outerjoinpath->parent->relids);
+							 outer_path->parent->relids);
 
 	/*
 	 * Create explicit sort nodes for the outer and inner paths if necessary.
 	 */
 	if (best_path->outersortkeys)
 	{
+		Relids		outer_relids = outer_path->parent->relids;
 		Sort	   *sort = make_sort_from_pathkeys(outer_plan,
-												   best_path->outersortkeys);
+												   best_path->outersortkeys,
+												   outer_relids);
 
 		label_sort_with_costsize(root, sort, -1.0);
 		outer_plan = (Plan *) sort;
 		outerpathkeys = best_path->outersortkeys;
 	}
 	else
-		outerpathkeys = best_path->jpath.outerjoinpath->pathkeys;
+		outerpathkeys = outer_path->pathkeys;
 
 	if (best_path->innersortkeys)
 	{
+		Relids		inner_relids = inner_path->parent->relids;
 		Sort	   *sort = make_sort_from_pathkeys(inner_plan,
-												   best_path->innersortkeys);
+												   best_path->innersortkeys,
+												   inner_relids);
 
 		label_sort_with_costsize(root, sort, -1.0);
 		inner_plan = (Plan *) sort;
 		innerpathkeys = best_path->innersortkeys;
 	}
 	else
-		innerpathkeys = best_path->jpath.innerjoinpath->pathkeys;
+		innerpathkeys = inner_path->pathkeys;
 
 	/*
 	 * If specified, add a materialize node to shield the inner plan from the
 	 * need to handle mark/restore.
 	 */
 	if (best_path->materialize_inner)
 	{
 		Plan	   *matplan = (Plan *) make_material(inner_plan);
 
 		/*
@@ -5330,25 +5337,25 @@ prepare_sort_from_pathkeys(Plan *lefttree, List *pathkeys,
 
 				/*
 				 * We shouldn't be trying to sort by an equivalence class that
 				 * contains a constant, so no need to consider such cases any
 				 * further.
 				 */
 				if (em->em_is_const)
 					continue;
 
 				/*
-				 * Ignore child members unless they match the rel being
+				 * Ignore child members unless they belong to the rel being
 				 * sorted.
 				 */
 				if (em->em_is_child &&
-					!bms_equal(em->em_relids, relids))
+					!bms_is_subset(em->em_relids, relids))
 					continue;
 
 				sortexpr = em->em_expr;
 				exprvars = pull_var_clause((Node *) sortexpr,
 										   PVC_INCLUDE_AGGREGATES |
 										   PVC_INCLUDE_WINDOWFUNCS |
 										   PVC_INCLUDE_PLACEHOLDERS);
 				foreach(k, exprvars)
 				{
 					if (!tlist_member_ignore_relabel(lfirst(k), tlist))
@@ -5445,57 +5452,58 @@ find_ec_member_for_tle(EquivalenceClass *ec,
 		Expr	   *emexpr;
 
 		/*
 		 * We shouldn't be trying to sort by an equivalence class that
 		 * contains a constant, so no need to consider such cases any further.
 		 */
 		if (em->em_is_const)
 			continue;
 
 		/*
-		 * Ignore child members unless they match the rel being sorted.
+		 * Ignore child members unless they belong to the rel being sorted.
 		 */
 		if (em->em_is_child &&
-			!bms_equal(em->em_relids, relids))
+			!bms_is_subset(em->em_relids, relids))
 			continue;
 
 		/* Match if same expression (after stripping relabel) */
 		emexpr = em->em_expr;
 		while (emexpr && IsA(emexpr, RelabelType))
 			emexpr = ((RelabelType *) emexpr)->arg;
 
 		if (equal(emexpr, tlexpr))
 			return em;
 	}
 
 	return NULL;
 }
 
 /*
  * make_sort_from_pathkeys
  *	  Create sort plan to sort according to given pathkeys
  *
  *	  'lefttree' is the node which yields input tuples
  *	  'pathkeys' is the list of pathkeys by which the result is to be sorted
+ *	  'relids' is the set of relations required by prepare_sort_from_pathkeys()
  */
 static Sort *
-make_sort_from_pathkeys(Plan *lefttree, List *pathkeys)
+make_sort_from_pathkeys(Plan *lefttree, List *pathkeys, Relids relids)
 {
 	int			numsortkeys;
 	AttrNumber *sortColIdx;
 	Oid		   *sortOperators;
 	Oid		   *collations;
 	bool	   *nullsFirst;
 
 	/* Compute sort column info, and adjust lefttree as needed */
 	lefttree = prepare_sort_from_pathkeys(lefttree, pathkeys,
-										  NULL,
+										  relids,
 										  NULL,
 										  false,
 										  &numsortkeys,
 										  &sortColIdx,
 										  &sortOperators,
 										  &collations,
 										  &nullsFirst);
 
 	/* Now build the Sort node */
 	return make_sort(lefttree, numsortkeys,
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index 193b2c9..6c703b8 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -1873,20 +1873,71 @@ adjust_appendrel_attrs(PlannerInfo *root, Node *node, AppendRelInfo *appinfo)
 										   appinfo);
 		}
 		result = (Node *) newnode;
 	}
 	else
 		result = adjust_appendrel_attrs_mutator(node, &context);
 
 	return result;
 }
 
+/*
+ * find_appinfos_by_relids
+ * 		Find AppendRelInfo structures for all relations specified by relids.
+ */
+List *
+find_appinfos_by_relids(PlannerInfo *root, Relids relids)
+{
+	ListCell	*lc;
+	List		*appinfo_list = NIL;
+
+	foreach (lc, root->append_rel_list)
+	{
+		AppendRelInfo *appinfo = lfirst(lc);
+
+		if (bms_is_member(appinfo->child_relid, relids))
+			appinfo_list = lappend(appinfo_list, appinfo);
+	}
+
+	Assert(list_length(appinfo_list) == bms_num_members(relids));
+	return appinfo_list;
+}
+
+/*
+ * adjust_join_appendrel_attrs
+ *
+ *	    Replace the parent references in the given node by the child references
+ *	    specified by the list of AppendRelInfo.
+ *
+ * This function is a wrapper around adjust_appendrel_attrs() which handles
+ * only one AppendRelInfo at a time.
+ *
+ * TODO: measure how much memory we are leaking, how does the memory usage grow
+ * for N-way joins between tables having many many partitions with N very
+ * large.
+ */
+
+Node *
+adjust_join_appendrel_attrs(PlannerInfo *root, Node *node,
+						  List *append_rel_infos)
+{
+	ListCell   *lc;
+
+	foreach (lc, append_rel_infos)
+	{
+		AppendRelInfo	*appinfo = lfirst(lc);
+		node = adjust_appendrel_attrs(root, node, appinfo);
+	}
+
+	return node;
+}
+
 static Node *
 adjust_appendrel_attrs_mutator(Node *node,
 							   adjust_appendrel_attrs_context *context)
 {
 	AppendRelInfo *appinfo = context->appinfo;
 
 	if (node == NULL)
 		return NULL;
 	if (IsA(node, Var))
 	{
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index abb7507..0463369 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -16,21 +16,23 @@
 
 #include <math.h>
 
 #include "miscadmin.h"
 #include "nodes/nodeFuncs.h"
 #include "optimizer/clauses.h"
 #include "optimizer/cost.h"
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
 #include "optimizer/planmain.h"
+#include "optimizer/prep.h"
 #include "optimizer/restrictinfo.h"
+#include "optimizer/tlist.h"
 #include "optimizer/var.h"
 #include "parser/parsetree.h"
 #include "utils/lsyscache.h"
 #include "utils/selfuncs.h"
 
 
 typedef enum
 {
 	COSTS_EQUAL,				/* path costs are fuzzily equal */
 	COSTS_BETTER1,				/* first path is cheaper than second */
@@ -3202,10 +3204,165 @@ reparameterize_path(PlannerInfo *root, Path *path,
 														 rel,
 														 spath->subpath,
 														 spath->path.pathkeys,
 														 required_outer);
 			}
 		default:
 			break;
 	}
 	return NULL;
 }
+
+/*
+ * reparameterize_path_by_child
+ * 		Given a path parameterized by the parent of the given relation,
+ * 		translate the path to be parameterized by the given child relation.
+ *
+ * The function creates a new path of the same type as the given path, but
+ * parameterized by the given child relation. If it can not reparameterize the
+ * path as required, it returns NULL.
+ *
+ * The cost, number of rows, width and parallel path properties depend upon
+ * path->parent, which does not change during the translation. Hence those
+ * members are copied as they are.
+ */
+
+Path *
+reparameterize_path_by_child(PlannerInfo *root, Path *path,
+							  RelOptInfo *child_rel)
+{
+	Path	   *new_path;
+	ParamPathInfo   *new_ppi;
+	ParamPathInfo   *old_ppi;
+	List	   *child_aris;
+
+	/*
+	 * If the path is not parameterized by parent of the given relation, it
+	 * doesn't need reparameterization.
+	 */
+	if (!path->param_info ||
+		!bms_overlap(PATH_REQ_OUTER(path), child_rel->top_parent_relids))
+	return path;
+
+	switch (nodeTag(path))
+	{
+		case T_Path:
+			new_path = makeNode(Path);
+			memcpy(new_path, path, sizeof(Path));
+			break;
+
+		case T_HashPath:
+			new_path = (Path *) makeNode(HashPath);
+			memcpy(new_path, path, sizeof(HashPath));
+			break;
+
+		case T_MergePath:
+			new_path = (Path *) makeNode(MergePath);
+			memcpy(new_path, path, sizeof(MergePath));
+			break;
+
+		case T_NestPath:
+			new_path = (Path *) makeNode(NestPath);
+			memcpy(new_path, path, sizeof(NestPath));
+			break;
+
+		case T_IndexPath:
+			new_path = (Path *) makeNode(IndexPath);
+			memcpy(new_path, path, sizeof(IndexPath));
+			break;
+
+		case T_AppendPath:
+			new_path = (Path *) makeNode(AppendPath);
+			memcpy(new_path, path, sizeof(AppendPath));
+			break;
+
+		/*
+		 * TODO:
+		 * If this method of translation is fine add more path types here.
+		 */
+
+		default:
+			/* Path type unsupported by this function. */
+			return NULL;
+	}
+
+	/*
+	 * Gather AppendRelInfos of the base partition relations in the outer child
+	 * relation. We need those for translating parent path to that of child by
+	 * substituting parent Var nodes and relids with those of children.
+	 */
+	child_aris = find_appinfos_by_relids(root, child_rel->relids);
+
+	/* Adjust the parameterization information. */
+	old_ppi = new_path->param_info;
+	new_ppi = makeNode(ParamPathInfo);
+	new_ppi->ppi_req_outer = adjust_child_relids(bms_copy(old_ppi->ppi_req_outer),
+													 child_aris);
+	new_ppi->ppi_rows = old_ppi->ppi_rows;
+	new_ppi->ppi_clauses = (List *) adjust_join_appendrel_attrs(root,
+													 (Node *) old_ppi->ppi_clauses,
+													 child_aris);
+
+	/* Adjust the path target. */
+	new_path->pathtarget = copy_pathtarget(new_path->pathtarget);
+	new_path->pathtarget->exprs = (List *) adjust_join_appendrel_attrs(root,
+													(Node *) new_path->pathtarget->exprs,
+													 child_aris);
+	new_path->param_info = new_ppi;
+
+	/*
+	 * Change parameterization of subpaths recursively. Also carry out any
+	 * pathtype specific adjustments.
+	 */
+	switch (nodeTag(path))
+	{
+		case T_HashPath:
+		case T_MergePath:
+		case T_NestPath:
+			{
+				JoinPath *jpath = (JoinPath *)new_path;
+
+				jpath->outerjoinpath = reparameterize_path_by_child(root,
+														 jpath->outerjoinpath,
+														 child_rel);
+				jpath->innerjoinpath = reparameterize_path_by_child(root,
+														 jpath->innerjoinpath,
+														 child_rel);
+				jpath->joinrestrictinfo = (List *) adjust_join_appendrel_attrs(root,
+															(Node *) jpath->joinrestrictinfo,
+															child_aris);
+			}
+			break;
+
+		case T_AppendPath:
+			{
+				AppendPath	*apath = (AppendPath *)new_path;
+				List		*subpaths = NIL;
+				ListCell	*lc;
+
+				foreach (lc, apath->subpaths)
+					subpaths = lappend(subpaths,
+									   reparameterize_path_by_child(root,
+																	lfirst(lc),
+																	child_rel));
+				apath->subpaths = subpaths;
+			}
+
+		case T_IndexPath:
+			{
+				IndexPath *ipath = (IndexPath *)new_path;
+
+				ipath->indexquals = (List *) adjust_join_appendrel_attrs(root,
+														(Node *) ipath->indexquals,
+														child_aris);
+				ipath->indexquals = (List *) adjust_join_appendrel_attrs(root,
+														(Node *) ipath->indexorderbys,
+														child_aris);
+			}
+
+		default:
+			/* Nothing to do. */
+			break;
+	}
+
+	return new_path;
+}
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index f8bfa4b..a0e17d5 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -404,20 +404,35 @@ get_relation_info(PlannerInfo *root, Oid relationObjectId, bool inhparent,
 	}
 	else
 	{
 		rel->serverid = InvalidOid;
 		rel->fdwroutine = NULL;
 	}
 
 	/* Collect info about relation's foreign keys, if relevant */
 	get_relation_foreign_keys(root, rel, relation);
 
+	/*
+	 * Lookup partition scheme for the given relation. Only parent relations
+	 * can be partitioned.
+	 */
+	if (inhparent)
+		rel->part_scheme = find_partition_scheme(root, relation);
+	else
+		rel->part_scheme = NULL;
+
+	if (rel->part_scheme)
+		rel->partexprs = build_baserel_partition_key_exprs(relation,
+														   rel->relid);
+	else
+		rel->partexprs = NULL;
+
 	heap_close(relation, NoLock);
 
 	/*
 	 * Allow a plugin to editorialize on the info we obtained from the
 	 * catalogs.  Actions might include altering the assumed relation size,
 	 * removing an index, or adding a hypothetical index to the indexlist.
 	 */
 	if (get_relation_info_hook)
 		(*get_relation_info_hook) (root, relationObjectId, inhparent, rel);
 }
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index deef560..93275b4 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -8,29 +8,35 @@
  *
  *
  * IDENTIFICATION
  *	  src/backend/optimizer/util/relnode.c
  *
  *-------------------------------------------------------------------------
  */
 #include "postgres.h"
 
 #include "miscadmin.h"
+#include "catalog/heap.h"
+#include "catalog/partition.h"
 #include "optimizer/clauses.h"
 #include "optimizer/cost.h"
+#include "nodes/makefuncs.h"
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
 #include "optimizer/placeholder.h"
 #include "optimizer/plancat.h"
+#include "optimizer/prep.h"
 #include "optimizer/restrictinfo.h"
 #include "optimizer/tlist.h"
+#include "rewrite/rewriteManip.h"
 #include "utils/hsearch.h"
+#include "utils/rel.h"
 
 
 typedef struct JoinHashEntry
 {
 	Relids		join_relids;	/* hash key --- MUST BE FIRST */
 	RelOptInfo *join_rel;
 } JoinHashEntry;
 
 static void build_joinrel_tlist(PlannerInfo *root, RelOptInfo *joinrel,
 					RelOptInfo *input_rel);
@@ -40,20 +46,25 @@ static List *build_joinrel_restrictlist(PlannerInfo *root,
 						   RelOptInfo *inner_rel);
 static void build_joinrel_joinlist(RelOptInfo *joinrel,
 					   RelOptInfo *outer_rel,
 					   RelOptInfo *inner_rel);
 static List *subbuild_joinrel_restrictlist(RelOptInfo *joinrel,
 							  List *joininfo_list,
 							  List *new_restrictlist);
 static List *subbuild_joinrel_joinlist(RelOptInfo *joinrel,
 						  List *joininfo_list,
 						  List *new_joininfo);
+static void set_foreign_rel_properties(RelOptInfo *joinrel,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel);
+static void build_joinrel_partition_info(RelOptInfo *joinrel,
+						  RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+						  JoinType jointype);
 
 
 /*
  * setup_simple_rel_arrays
  *	  Prepare the arrays we use for quickly accessing base relations.
  */
 void
 setup_simple_rel_arrays(PlannerInfo *root)
 {
 	Index		rti;
@@ -130,20 +141,24 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptKind reloptkind)
 	rel->serverid = InvalidOid;
 	rel->userid = rte->checkAsUser;
 	rel->useridiscurrent = false;
 	rel->fdwroutine = NULL;
 	rel->fdw_private = NULL;
 	rel->baserestrictinfo = NIL;
 	rel->baserestrictcost.startup = 0;
 	rel->baserestrictcost.per_tuple = 0;
 	rel->joininfo = NIL;
 	rel->has_eclass_joins = false;
+	rel->part_scheme = NULL;
+	rel->partexprs = NULL;
+	rel->top_parent_relids = NULL;
+	rel->part_rels = NULL;
 
 	/* Check type of rtable entry */
 	switch (rte->rtekind)
 	{
 		case RTE_RELATION:
 			/* Table --- retrieve statistics from the system catalogs */
 			get_relation_info(root, rte->relid, rte->inh, rel);
 			break;
 		case RTE_SUBQUERY:
 		case RTE_FUNCTION:
@@ -307,20 +322,70 @@ find_join_rel(PlannerInfo *root, Relids relids)
 
 			if (bms_equal(rel->relids, relids))
 				return rel;
 		}
 	}
 
 	return NULL;
 }
 
 /*
+ * set_foreign_rel_properties
+ *		Set up foreign-join fields if outer and inner relation are foreign
+ *		tables (or joins) belonging to the same server and assigned to the same
+ *		user to check access permissions as.
+ *
+ * In addition to an exact match of userid, we allow the case where one side
+ * has zero userid (implying current user) and the other side has explicit
+ * userid that happens to equal the current user; but in that case, pushdown of
+ * the join is only valid for the current user.  The useridiscurrent field
+ * records whether we had to make such an assumption for this join or any
+ * sub-join.
+ *
+ * Otherwise these fields are left invalid, so GetForeignJoinPaths will not be
+ * called for the join relation.
+ *
+ */
+static void
+set_foreign_rel_properties(RelOptInfo *joinrel, RelOptInfo *outer_rel,
+						   RelOptInfo *inner_rel)
+{
+	if (OidIsValid(outer_rel->serverid) &&
+		inner_rel->serverid == outer_rel->serverid)
+	{
+		if (inner_rel->userid == outer_rel->userid)
+		{
+			joinrel->serverid = outer_rel->serverid;
+			joinrel->userid = outer_rel->userid;
+			joinrel->useridiscurrent = outer_rel->useridiscurrent || inner_rel->useridiscurrent;
+			joinrel->fdwroutine = outer_rel->fdwroutine;
+		}
+		else if (!OidIsValid(inner_rel->userid) &&
+				 outer_rel->userid == GetUserId())
+		{
+			joinrel->serverid = outer_rel->serverid;
+			joinrel->userid = outer_rel->userid;
+			joinrel->useridiscurrent = true;
+			joinrel->fdwroutine = outer_rel->fdwroutine;
+		}
+		else if (!OidIsValid(outer_rel->userid) &&
+				 inner_rel->userid == GetUserId())
+		{
+			joinrel->serverid = outer_rel->serverid;
+			joinrel->userid = inner_rel->userid;
+			joinrel->useridiscurrent = true;
+			joinrel->fdwroutine = outer_rel->fdwroutine;
+		}
+	}
+}
+
+/*
  * build_join_rel
  *	  Returns relation entry corresponding to the union of two given rels,
  *	  creating a new relation entry if none already exists.
  *
  * 'joinrelids' is the Relids set that uniquely identifies the join
  * 'outer_rel' and 'inner_rel' are relation nodes for the relations to be
  *		joined
  * 'sjinfo': join context info
  * 'restrictlist_ptr': result variable.  If not NULL, *restrictlist_ptr
  *		receives the list of RestrictInfo nodes that apply to this
@@ -356,21 +421,25 @@ build_join_rel(PlannerInfo *root,
 														   joinrel,
 														   outer_rel,
 														   inner_rel);
 		return joinrel;
 	}
 
 	/*
 	 * Nope, so make one.
 	 */
 	joinrel = makeNode(RelOptInfo);
+
+	Assert(!IS_OTHER_REL(outer_rel->reloptkind) &&
+		   !IS_OTHER_REL(inner_rel->reloptkind));
 	joinrel->reloptkind = RELOPT_JOINREL;
+
 	joinrel->relids = bms_copy(joinrelids);
 	joinrel->rows = 0;
 	/* cheap startup cost is interesting iff not all tuples to be retrieved */
 	joinrel->consider_startup = (root->tuple_fraction > 0);
 	joinrel->consider_param_startup = false;
 	joinrel->consider_parallel = false;
 	joinrel->reltarget = create_empty_pathtarget();
 	joinrel->pathlist = NIL;
 	joinrel->ppilist = NIL;
 	joinrel->partial_pathlist = NIL;
@@ -402,61 +471,27 @@ build_join_rel(PlannerInfo *root,
 	joinrel->serverid = InvalidOid;
 	joinrel->userid = InvalidOid;
 	joinrel->useridiscurrent = false;
 	joinrel->fdwroutine = NULL;
 	joinrel->fdw_private = NULL;
 	joinrel->baserestrictinfo = NIL;
 	joinrel->baserestrictcost.startup = 0;
 	joinrel->baserestrictcost.per_tuple = 0;
 	joinrel->joininfo = NIL;
 	joinrel->has_eclass_joins = false;
+	joinrel->part_scheme = NULL;
+	joinrel->partexprs = NULL;
+	joinrel->top_parent_relids = NULL;
+	joinrel->part_rels = NULL;
 
-	/*
-	 * Set up foreign-join fields if outer and inner relation are foreign
-	 * tables (or joins) belonging to the same server and assigned to the same
-	 * user to check access permissions as.  In addition to an exact match of
-	 * userid, we allow the case where one side has zero userid (implying
-	 * current user) and the other side has explicit userid that happens to
-	 * equal the current user; but in that case, pushdown of the join is only
-	 * valid for the current user.  The useridiscurrent field records whether
-	 * we had to make such an assumption for this join or any sub-join.
-	 *
-	 * Otherwise these fields are left invalid, so GetForeignJoinPaths will
-	 * not be called for the join relation.
-	 */
-	if (OidIsValid(outer_rel->serverid) &&
-		inner_rel->serverid == outer_rel->serverid)
-	{
-		if (inner_rel->userid == outer_rel->userid)
-		{
-			joinrel->serverid = outer_rel->serverid;
-			joinrel->userid = outer_rel->userid;
-			joinrel->useridiscurrent = outer_rel->useridiscurrent || inner_rel->useridiscurrent;
-			joinrel->fdwroutine = outer_rel->fdwroutine;
-		}
-		else if (!OidIsValid(inner_rel->userid) &&
-				 outer_rel->userid == GetUserId())
-		{
-			joinrel->serverid = outer_rel->serverid;
-			joinrel->userid = outer_rel->userid;
-			joinrel->useridiscurrent = true;
-			joinrel->fdwroutine = outer_rel->fdwroutine;
-		}
-		else if (!OidIsValid(outer_rel->userid) &&
-				 inner_rel->userid == GetUserId())
-		{
-			joinrel->serverid = outer_rel->serverid;
-			joinrel->userid = inner_rel->userid;
-			joinrel->useridiscurrent = true;
-			joinrel->fdwroutine = outer_rel->fdwroutine;
-		}
-	}
+	/* Compute information relevant to the foreign relations. */
+	set_foreign_rel_properties(joinrel, outer_rel, inner_rel);
 
 	/*
 	 * Create a new tlist containing just the vars that need to be output from
 	 * this join (ie, are needed for higher joinclauses or final output).
 	 *
 	 * NOTE: the tlist order for a join rel will depend on which pair of outer
 	 * and inner rels we first try to build it from.  But the contents should
 	 * be the same regardless.
 	 */
 	build_joinrel_tlist(root, joinrel, outer_rel);
@@ -468,20 +503,24 @@ build_join_rel(PlannerInfo *root,
 	 * sets of any PlaceHolderVars computed here to direct_lateral_relids, so
 	 * now we can finish computing that.  This is much like the computation of
 	 * the transitively-closed lateral_relids in min_join_parameterization,
 	 * except that here we *do* have to consider the added PHVs.
 	 */
 	joinrel->direct_lateral_relids =
 		bms_del_members(joinrel->direct_lateral_relids, joinrel->relids);
 	if (bms_is_empty(joinrel->direct_lateral_relids))
 		joinrel->direct_lateral_relids = NULL;
 
+	/* Store the partition information. */
+	build_joinrel_partition_info(joinrel, outer_rel, inner_rel,
+								 sjinfo->jointype);
+
 	/*
 	 * Construct restrict and join clause lists for the new joinrel. (The
 	 * caller might or might not need the restrictlist, but I need it anyway
 	 * for set_joinrel_size_estimates().)
 	 */
 	restrictlist = build_joinrel_restrictlist(root, joinrel,
 											  outer_rel, inner_rel);
 	if (restrictlist_ptr)
 		*restrictlist_ptr = restrictlist;
 	build_joinrel_joinlist(joinrel, outer_rel, inner_rel);
@@ -510,58 +549,168 @@ build_join_rel(PlannerInfo *root,
 	 * assume this doesn't matter, because we should hit all the same baserels
 	 * and joinclauses while building up to this joinrel no matter which we
 	 * take; therefore, we should make the same decision here however we get
 	 * here.
 	 */
 	if (inner_rel->consider_parallel && outer_rel->consider_parallel &&
 		is_parallel_safe(root, (Node *) restrictlist) &&
 		is_parallel_safe(root, (Node *) joinrel->reltarget->exprs))
 		joinrel->consider_parallel = true;
 
-	/*
-	 * Add the joinrel to the query's joinrel list, and store it into the
-	 * auxiliary hashtable if there is one.  NB: GEQO requires us to append
-	 * the new joinrel to the end of the list!
-	 */
-	root->join_rel_list = lappend(root->join_rel_list, joinrel);
-
-	if (root->join_rel_hash)
-	{
-		JoinHashEntry *hentry;
-		bool		found;
-
-		hentry = (JoinHashEntry *) hash_search(root->join_rel_hash,
-											   &(joinrel->relids),
-											   HASH_ENTER,
-											   &found);
-		Assert(!found);
-		hentry->join_rel = joinrel;
-	}
+	/* Add the joinrel to the query's PlannerInfo. */
+	add_join_rel(root, joinrel);
 
 	/*
 	 * Also, if dynamic-programming join search is active, add the new joinrel
 	 * to the appropriate sublist.  Note: you might think the Assert on number
 	 * of members should be for equality, but some of the level 1 rels might
 	 * have been joinrels already, so we can only assert <=.
 	 */
 	if (root->join_rel_level)
 	{
 		Assert(root->join_cur_level > 0);
 		Assert(root->join_cur_level <= bms_num_members(joinrel->relids));
 		root->join_rel_level[root->join_cur_level] =
 			lappend(root->join_rel_level[root->join_cur_level], joinrel);
 	}
 
 	return joinrel;
 }
 
 /*
+ * build_child_join_rel
+ *		Builds RelOptInfo for joining given two child relations from RelOptInfo
+ *		representing the join between their parents.
+ *
+ * 'outer_rel' and 'inner_rel' are the RelOptInfos of child relations being
+ *		joined.
+ * 'parent_joinrel' is the RelOptInfo representing the join between parent
+ *		relations. Most of the members of new RelOptInfo are produced by
+ *		translating corresponding members of this RelOptInfo.
+ * 'sjinfo': context info for child join
+ * 'restrictlist': list of RestrictInfo nodes that apply to this particular
+ *		pair of joinable relations.
+ * 'join_appinfos': list of AppendRelInfo nodes for base child relations involved
+ *		in this join.
+ */
+RelOptInfo *
+build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
+						 RelOptInfo *inner_rel, RelOptInfo *parent_joinrel,
+						 SpecialJoinInfo *sjinfo, List *restrictlist,
+						 List *join_appinfos)
+{
+	List	   *tmp_exprs;
+	RelOptInfo *joinrel = makeNode(RelOptInfo);
+
+	joinrel->reloptkind = RELOPT_OTHER_JOINREL;
+	joinrel->relids = bms_union(outer_rel->relids, inner_rel->relids);
+	joinrel->rows = 0;
+	/* cheap startup cost is interesting iff not all tuples to be retrieved */
+	joinrel->consider_startup = (root->tuple_fraction > 0);
+	joinrel->consider_param_startup = false;
+	joinrel->consider_parallel = false;
+	joinrel->reltarget = create_empty_pathtarget();
+	joinrel->pathlist = NIL;
+	joinrel->ppilist = NIL;
+	joinrel->partial_pathlist = NIL;
+	joinrel->cheapest_startup_path = NULL;
+	joinrel->cheapest_total_path = NULL;
+	joinrel->cheapest_unique_path = NULL;
+	joinrel->cheapest_parameterized_paths = NIL;
+	joinrel->direct_lateral_relids = NULL;
+	joinrel->lateral_relids = NULL;
+	joinrel->relid = 0;			/* indicates not a baserel */
+	joinrel->rtekind = RTE_JOIN;
+	joinrel->min_attr = 0;
+	joinrel->max_attr = 0;
+	joinrel->attr_needed = NULL;
+	joinrel->attr_widths = NULL;
+	joinrel->lateral_vars = NIL;
+	joinrel->lateral_referencers = NULL;
+	joinrel->indexlist = NIL;
+	joinrel->pages = 0;
+	joinrel->tuples = 0;
+	joinrel->allvisfrac = 0;
+	joinrel->subroot = NULL;
+	joinrel->subplan_params = NIL;
+	joinrel->serverid = InvalidOid;
+	joinrel->userid = InvalidOid;
+	joinrel->useridiscurrent = false;
+	joinrel->fdwroutine = NULL;
+	joinrel->fdw_private = NULL;
+	joinrel->baserestrictinfo = NIL;
+	joinrel->baserestrictcost.startup = 0;
+	joinrel->baserestrictcost.per_tuple = 0;
+	joinrel->joininfo = NIL;
+	joinrel->has_eclass_joins = false;
+	joinrel->part_scheme = NULL;
+	joinrel->partexprs = NULL;
+	joinrel->top_parent_relids = NULL;
+	joinrel->part_rels = NULL;
+
+
+	/* Only joins between other relations land here. */
+	Assert(IS_OTHER_REL(outer_rel->reloptkind) &&
+		   IS_OTHER_REL(inner_rel->reloptkind));
+
+	joinrel->top_parent_relids = bms_union(outer_rel->top_parent_relids,
+										   inner_rel->top_parent_relids);
+
+	/* Compute information relevant to foreign relations. */
+	set_foreign_rel_properties(joinrel, outer_rel, inner_rel);
+
+	/* Translate targetlist and joininfo. */
+	joinrel->reltarget = copy_pathtarget(parent_joinrel->reltarget);
+	tmp_exprs = joinrel->reltarget->exprs;
+	joinrel->reltarget->exprs = (List *) adjust_join_appendrel_attrs(root,
+															(Node *) tmp_exprs,
+																join_appinfos);
+	joinrel->joininfo = (List *) adjust_join_appendrel_attrs(root,
+											 (Node *) parent_joinrel->joininfo,
+																join_appinfos);
+
+
+	/*
+	 * Lateral relids referred in child join will be same as that referred in
+	 * the parent relation.
+	 */
+	joinrel->direct_lateral_relids = (Relids) bms_copy(parent_joinrel->direct_lateral_relids);
+	joinrel->lateral_relids = (Relids) bms_copy(parent_joinrel->lateral_relids);
+
+	/*
+	 * If the parent joinrel has pending equivalence classes, so does the
+	 * child.
+	 */
+	joinrel->has_eclass_joins = parent_joinrel->has_eclass_joins;
+
+	/* Is the join between partitions itself partitioned? */
+	build_joinrel_partition_info(joinrel, outer_rel, inner_rel, sjinfo->jointype);
+
+	/*
+	 * Set estimates of the joinrel's size.
+	 */
+	set_joinrel_size_estimates(root, joinrel, outer_rel, inner_rel, sjinfo,
+							   restrictlist);
+
+	/* Child joinrel is parallel safe if parent is parallel safe. */
+	joinrel->consider_parallel = parent_joinrel->consider_parallel;
+
+	/* We build the join only once. */
+	Assert(!find_join_rel(root, joinrel->relids));
+
+	/* Add the relation to the PlannerInfo. */
+	add_join_rel(root, joinrel);
+
+	return joinrel;
+}
+
+/*
  * min_join_parameterization
  *
  * Determine the minimum possible parameterization of a joinrel, that is, the
  * set of other rels it contains LATERAL references to.  We save this value in
  * the join's RelOptInfo.  This function is split out of build_join_rel()
  * because join_is_legal() needs the value to check a prospective join.
  */
 Relids
 min_join_parameterization(PlannerInfo *root,
 						  Relids joinrelids,
@@ -1313,10 +1462,118 @@ get_appendrel_parampathinfo(RelOptInfo *appendrel, Relids required_outer)
 
 	/* Else build the ParamPathInfo */
 	ppi = makeNode(ParamPathInfo);
 	ppi->ppi_req_outer = required_outer;
 	ppi->ppi_rows = 0;
 	ppi->ppi_clauses = NIL;
 	appendrel->ppilist = lappend(appendrel->ppilist, ppi);
 
 	return ppi;
 }
+
+/*
+ * add_join_rel
+ *		Add given join relation to the list of join relations in the given
+ *		PlannerInfo. Also add it to the auxiliary hashtable if there is one.
+ */
+void
+add_join_rel(PlannerInfo *root, RelOptInfo *joinrel)
+{
+	/* GEQO requires us to append the new joinrel to the end of the list! */
+	root->join_rel_list = lappend(root->join_rel_list, joinrel);
+
+	/* store it into the auxiliary hashtable if there is one. */
+	if (root->join_rel_hash)
+	{
+		JoinHashEntry *hentry;
+		bool		found;
+
+		hentry = (JoinHashEntry *) hash_search(root->join_rel_hash,
+											   &(joinrel->relids),
+											   HASH_ENTER,
+											   &found);
+		Assert(!found);
+		hentry->join_rel = joinrel;
+	}
+}
+
+/*
+ * build_joinrel_partition_info
+ *		If the join between given partitioned relations is possibly partitioned
+ *		set the partitioning scheme and partition keys expressions for the
+ *		join.
+ *
+ * If the two relations have same partitioning scheme, their join may be
+ * partitioned and will follow the same partitioning scheme as the joining
+ * relations.
+ */
+static void
+build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
+							 RelOptInfo *inner_rel, JoinType jointype)
+{
+	int		num_pks;
+	int		cnt;
+
+	/* Nothing to do if partition-wise join technique is disabled. */
+	if (!enable_partition_wise_join)
+	{
+		joinrel->part_scheme = NULL;
+		return;
+	}
+
+	/*
+	 * The join is not partitioned, if any of the relations being joined are
+	 * not partitioned or they do not have same partitioning scheme.
+	 */
+	if (!outer_rel->part_scheme || !inner_rel->part_scheme ||
+		outer_rel->part_scheme != inner_rel->part_scheme)
+	{
+		joinrel->part_scheme = NULL;
+		return;
+	}
+
+	/*
+	 * This function will be called only once for each joinrel, hence it should
+	 * not have partition scheme, partition key expressions and array for
+	 * storing child relations set.
+	 */
+	Assert(!joinrel->part_scheme && !joinrel->partexprs &&
+		   !joinrel->part_rels);
+
+	/*
+	 * Join relation is partitioned using same partitioning scheme as the
+	 * joining relations.
+	 */
+	joinrel->part_scheme = outer_rel->part_scheme;
+	num_pks = joinrel->part_scheme->partnatts;
+
+	/*
+	 * Construct partition keys for the join.
+	 *
+	 * An INNER join between two partitioned relations is partition by key
+	 * expressions from both the relations. For tables A and B partitioned by a and b
+	 * respectively, (A INNER JOIN B ON A.a = B.b) is partitioned by both A.a
+	 * and B.b.
+	 *
+	 * An OUTER join like (A LEFT JOIN B ON A.a = B.b) may produce rows with
+	 * B.b NULL. These rows may not fit the partitioning conditions imposed on
+	 * B.b. Hence, strictly speaking, the join is not partitioned by B.b.
+	 * Strictly speaking, partition keys of an OUTER join should include
+	 * partition key expressions from the OUTER side only. Consider a join like
+	 * (A LEFT JOIN B on (A.a = B.b) LEFT JOIN C ON B.b = C.c. If we do not
+	 * include B.b as partition key expression for (AB), it prohibits us from
+	 * using partition-wise join when joining (AB) with C as there is no
+	 * equi-join between partition keys of joining relations. But two NULL
+	 * values are never equal and no two rows from mis-matching partitions can
+	 * join. Hence it's safe to include B.b as partition key expression for
+	 * (AB), even though rows in (AB) are not strictly partitioned by B.b.
+	 */
+	joinrel->partexprs = (List **) palloc0(sizeof(List *) * num_pks);
+	for (cnt = 0; cnt < num_pks; cnt++)
+	{
+		List *pkexpr = list_copy(outer_rel->partexprs[cnt]);
+
+		pkexpr = list_concat(pkexpr,
+							 list_copy(inner_rel->partexprs[cnt]));
+		joinrel->partexprs[cnt] = pkexpr;
+	}
+}
diff --git a/src/backend/utils/adt/selfuncs.c b/src/backend/utils/adt/selfuncs.c
index 56943f2..16b2eac 100644
--- a/src/backend/utils/adt/selfuncs.c
+++ b/src/backend/utils/adt/selfuncs.c
@@ -3405,21 +3405,23 @@ estimate_num_groups(PlannerInfo *root, List *groupExprs, double input_rows,
 			else
 			{
 				/* not time to process varinfo2 yet */
 				newvarinfos = lcons(varinfo2, newvarinfos);
 			}
 		}
 
 		/*
 		 * Sanity check --- don't divide by zero if empty relation.
 		 */
-		Assert(rel->reloptkind == RELOPT_BASEREL);
+		Assert(rel->reloptkind == RELOPT_BASEREL ||
+			   rel->reloptkind == RELOPT_OTHER_MEMBER_REL);
+
 		if (rel->tuples > 0)
 		{
 			/*
 			 * Clamp to size of rel, or size of rel / 10 if multiple Vars. The
 			 * fudge factor is because the Vars are probably correlated but we
 			 * don't know by how much.  We should never clamp to less than the
 			 * largest ndistinct value for any of the Vars, though, since
 			 * there will surely be at least that many groups.
 			 */
 			double		clamp = rel->tuples;
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index ce4eef9..edc7e58 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -870,20 +870,29 @@ static struct config_bool ConfigureNamesBool[] =
 	},
 	{
 		{"enable_hashjoin", PGC_USERSET, QUERY_TUNING_METHOD,
 			gettext_noop("Enables the planner's use of hash join plans."),
 			NULL
 		},
 		&enable_hashjoin,
 		true,
 		NULL, NULL, NULL
 	},
+	{
+		{"enable_partition_wise_join", PGC_USERSET, QUERY_TUNING_METHOD,
+			gettext_noop("Enables partition-wise join."),
+			NULL
+		},
+		&enable_partition_wise_join,
+		true,
+		NULL, NULL, NULL
+	},
 
 	{
 		{"geqo", PGC_USERSET, QUERY_TUNING_GEQO,
 			gettext_noop("Enables genetic query optimization."),
 			gettext_noop("This algorithm attempts to do planning without "
 						 "exhaustive searching.")
 		},
 		&enable_geqo,
 		true,
 		NULL, NULL, NULL
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 81a4b91..19b7744 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -39,22 +39,57 @@ typedef struct BoundCollectionData *BoundCollection;
  * is list partitioned.  Whereas in case of a range partitioned table, they
  * are ordered to match the ascending order of partition ranges.
  */
 typedef struct PartitionDescData
 {
 	int					nparts;		/* Number of partitions */
 	Oid				   *oids;		/* OIDs of partitions */
 	BoundCollection		bounds;		/* collection of list or range bounds */
 } PartitionDescData;
 
+/*
+ * Partitioning scheme
+ *		Structure to hold partitioning scheme for a given relation.
+ *
+ * Multiple relations may be partitioned in the same way. The relations
+ * resulting from joining such relations may be partitioned in the same way as
+ * the joining relations. Similarly, relations derived from such relations by
+ * grouping, sorting may be partitioned in the same way as the underlying
+ * scan relations. All such relations partitioned in the same way share the
+ * partitioning scheme.
+ *
+ * PlannerInfo stores a list of distinct "canonical" partitioning schemes.
+ * RelOptInfo of a partitioned relation holds the pointer to "canonical"
+ * partitioning scheme.
+ */
+typedef struct PartitionSchemeData
+{
+	/* Information about partitions */
+	int			nparts;	/* number of partitions */
+	BoundCollection bounds; /* Partition bounds/lists */
+
+	/* Information about partition keys */
+	char		strategy;		/* partition strategy */
+	int16		partnatts;		/* number of partition attributes */
+	Oid		   *partopfamily;	/* OIDs of operator families */
+	Oid		   *partopcintype;	/* OIDs of opclass declared input data types */
+	Oid		   *key_types;		/* OIDs of partition key data types. */
+	int32	   *key_typmods;	/* typmods of partition keys. */
+	Oid		   *key_collations;	/* OIDs of collations of partition keys. */
+} PartitionSchemeData;
+
 typedef struct PartitionDescData *PartitionDesc;
 typedef struct PartitionTreeNodeData *PartitionTreeNode;
+typedef struct PartitionSchemeData *PartitionScheme;
+
+/* Include here to avoid circular dependency with relation.h. */
+struct PlannerInfo;
 
 /* relcache support for partition key information */
 extern void RelationBuildPartitionKey(Relation relation);
 
 /* Partition key inquiry functions */
 extern int get_partition_key_strategy(PartitionKey key);
 extern int get_partition_key_natts(PartitionKey key);
 extern List *get_partition_key_exprs(PartitionKey key);
 
 /* Partition key inquiry functions - for a given column */
@@ -77,11 +112,16 @@ extern List *get_qual_from_partbound(Relation rel, Relation parent, Node *bound)
 extern List *RelationGetPartitionQual(Relation rel, bool recurse);
 
 /* For tuple routing */
 extern PartitionTreeNode RelationGetPartitionTreeNode(Relation rel);
 extern List *get_leaf_partition_oids_v2(PartitionTreeNode ptnode);
 
 extern int get_partition_for_tuple(PartitionTreeNode ptnode,
 					TupleTableSlot *slot,
 					EState *estate,
 					Oid *failed_at);
+extern List **build_baserel_partition_key_exprs(Relation relation,
+												Index varno);
+extern PartitionScheme find_partition_scheme(struct PlannerInfo *root,
+											 Relation rel);
+
 #endif   /* PARTITION_H */
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 3a1255a..e79fb09 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -8,20 +8,21 @@
  * Portions Copyright (c) 1994, Regents of the University of California
  *
  * src/include/nodes/relation.h
  *
  *-------------------------------------------------------------------------
  */
 #ifndef RELATION_H
 #define RELATION_H
 
 #include "access/sdir.h"
+#include "catalog/partition.h"
 #include "lib/stringinfo.h"
 #include "nodes/params.h"
 #include "nodes/parsenodes.h"
 #include "storage/block.h"
 
 
 /*
  * Relids
  *		Set of relation identifiers (indexes into the rangetable).
  */
@@ -256,20 +257,23 @@ typedef struct PlannerInfo
 
 	List	   *query_pathkeys; /* desired pathkeys for query_planner() */
 
 	List	   *group_pathkeys; /* groupClause pathkeys, if any */
 	List	   *window_pathkeys;	/* pathkeys of bottom window, if any */
 	List	   *distinct_pathkeys;		/* distinctClause pathkeys, if any */
 	List	   *sort_pathkeys;	/* sortClause pathkeys, if any */
 
 	List	   *initial_rels;	/* RelOptInfos we are now trying to join */
 
+	List	   *part_schemes;	/* Canonicalised partition schemes
+								 * used in the query. */
+
 	/* Use fetch_upper_rel() to get any particular upper rel */
 	List	   *upper_rels[UPPERREL_FINAL + 1]; /* upper-rel RelOptInfos */
 
 	/* Result tlists chosen by grouping_planner for upper-stage processing */
 	struct PathTarget *upper_targets[UPPERREL_FINAL + 1];
 
 	/*
 	 * grouping_planner passes back its final processed targetlist here, for
 	 * use in relabeling the topmost tlist of the finished Plan.
 	 */
@@ -345,20 +349,25 @@ typedef struct PlannerInfo
  * is present in the query join tree but the members are not.  The member
  * RTEs and otherrels are used to plan the scans of the individual tables or
  * subqueries of the append set; then the parent baserel is given Append
  * and/or MergeAppend paths comprising the best paths for the individual
  * member rels.  (See comments for AppendRelInfo for more information.)
  *
  * At one time we also made otherrels to represent join RTEs, for use in
  * handling join alias Vars.  Currently this is not needed because all join
  * alias Vars are expanded to non-aliased form during preprocess_expression.
  *
+ * We also have relations representing joins between child relations of
+ * different partitioned tables. These relations are not added to
+ * join_rel_level lists as they are not joined directly by the dynamic
+ * programming algorithm.
+ *
  * There is also a RelOptKind for "upper" relations, which are RelOptInfos
  * that describe post-scan/join processing steps, such as aggregation.
  * Many of the fields in these RelOptInfos are meaningless, but their Path
  * fields always hold Paths showing ways to do that processing step.
  *
  * Lastly, there is a RelOptKind for "dead" relations, which are base rels
  * that we have proven we don't need to join after all.
  *
  * Parts of this data structure are specific to various scan and join
  * mechanisms.  It didn't seem worth creating new node types for them.
@@ -464,24 +473,33 @@ typedef struct PlannerInfo
  * We store baserestrictcost in the RelOptInfo (for base relations) because
  * we know we will need it at least once (to price the sequential scan)
  * and may need it multiple times to price index scans.
  *----------
  */
 typedef enum RelOptKind
 {
 	RELOPT_BASEREL,
 	RELOPT_JOINREL,
 	RELOPT_OTHER_MEMBER_REL,
+	RELOPT_OTHER_JOINREL,
 	RELOPT_UPPER_REL,
 	RELOPT_DEADREL
 } RelOptKind;
 
+#define IS_OTHER_REL(reloptkind) \
+	((reloptkind) == RELOPT_OTHER_MEMBER_REL || \
+	 (reloptkind) == RELOPT_OTHER_JOINREL)
+
+#define IS_JOIN_REL(rel)	\
+	((rel->reloptkind) == RELOPT_JOINREL || \
+	 (rel->reloptkind) == RELOPT_OTHER_JOINREL)
+
 typedef struct RelOptInfo
 {
 	NodeTag		type;
 
 	RelOptKind	reloptkind;
 
 	/* all relations included in this RelOptInfo */
 	Relids		relids;			/* set of base relids (rangetable indexes) */
 
 	/* size estimates generated by planner */
@@ -535,20 +553,37 @@ typedef struct RelOptInfo
 	struct FdwRoutine *fdwroutine;
 	void	   *fdw_private;
 
 	/* used by various scans and joins: */
 	List	   *baserestrictinfo;		/* RestrictInfo structures (if base
 										 * rel) */
 	QualCost	baserestrictcost;		/* cost of evaluating the above */
 	List	   *joininfo;		/* RestrictInfo structures for join clauses
 								 * involving this rel */
 	bool		has_eclass_joins;		/* T means joininfo is incomplete */
+
+	/* For partitioned relations. */
+	PartitionScheme	    part_scheme;	/* Partitioning scheme. */
+	struct RelOptInfo **part_rels;		/* Array of RelOptInfos of partitions,
+										 * stored in the same order as bounds
+										 * or lists in PartitionScheme.
+										 */
+	List			  **partexprs;		/* Array of list of partition key
+										 * expressions. For base relations
+										 * these are one element lists. For
+										 * join there may be as many elements
+										 * as the number of joining
+										 * relations.
+										 */
+
+	/* Set only for "other" base or join relations. */
+	Relids		top_parent_relids;		/* Relids of topmost parents. */
 } RelOptInfo;
 
 /*
  * IndexOptInfo
  *		Per-index information for planning/optimization
  *
  *		indexkeys[], indexcollations[], opfamily[], and opcintype[]
  *		each have ncolumns entries.
  *
  *		sortopfamily[], reverse_sort[], and nulls_first[] likewise have
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 2a4df2f..1069726 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -59,20 +59,21 @@ extern bool enable_seqscan;
 extern bool enable_indexscan;
 extern bool enable_indexonlyscan;
 extern bool enable_bitmapscan;
 extern bool enable_tidscan;
 extern bool enable_sort;
 extern bool enable_hashagg;
 extern bool enable_nestloop;
 extern bool enable_material;
 extern bool enable_mergejoin;
 extern bool enable_hashjoin;
+extern bool enable_partition_wise_join;
 extern int	constraint_exclusion;
 
 extern double clamp_row_est(double nrows);
 extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
 					double index_pages, PlannerInfo *root);
 extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 			 ParamPathInfo *param_info);
 extern void cost_samplescan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
 				ParamPathInfo *param_info);
 extern void cost_index(IndexPath *path, PlannerInfo *root,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 71d9154..3c2a72e 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -222,20 +222,22 @@ extern ModifyTablePath *create_modifytable_path(PlannerInfo *root,
 						List *rowMarks, OnConflictExpr *onconflict,
 						int epqParam);
 extern LimitPath *create_limit_path(PlannerInfo *root, RelOptInfo *rel,
 				  Path *subpath,
 				  Node *limitOffset, Node *limitCount,
 				  int64 offset_est, int64 count_est);
 
 extern Path *reparameterize_path(PlannerInfo *root, Path *path,
 					Relids required_outer,
 					double loop_count);
+extern Path *reparameterize_path_by_child(PlannerInfo *root, Path *path,
+					RelOptInfo *child_rel);
 
 /*
  * prototypes for relnode.c
  */
 extern void setup_simple_rel_arrays(PlannerInfo *root);
 extern RelOptInfo *build_simple_rel(PlannerInfo *root, int relid,
 				 RelOptKind reloptkind);
 extern RelOptInfo *find_base_rel(PlannerInfo *root, int relid);
 extern RelOptInfo *find_join_rel(PlannerInfo *root, Relids relids);
 extern RelOptInfo *build_join_rel(PlannerInfo *root,
@@ -260,12 +262,17 @@ extern ParamPathInfo *get_baserel_parampathinfo(PlannerInfo *root,
 						  Relids required_outer);
 extern ParamPathInfo *get_joinrel_parampathinfo(PlannerInfo *root,
 						  RelOptInfo *joinrel,
 						  Path *outer_path,
 						  Path *inner_path,
 						  SpecialJoinInfo *sjinfo,
 						  Relids required_outer,
 						  List **restrict_clauses);
 extern ParamPathInfo *get_appendrel_parampathinfo(RelOptInfo *appendrel,
 							Relids required_outer);
+extern RelOptInfo *build_child_join_rel(PlannerInfo *root,
+						 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+						 RelOptInfo *parent_joinrel, SpecialJoinInfo *sjinfo,
+						 List *restrictlist, List *join_appinfos);
+extern void add_join_rel(PlannerInfo *root, RelOptInfo *joinrel);
 
 #endif   /* PATHNODE_H */
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 44abe83..e57a166 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -212,11 +212,13 @@ extern List *make_inner_pathkeys_for_merge(PlannerInfo *root,
 							  List *mergeclauses,
 							  List *outer_pathkeys);
 extern List *truncate_useless_pathkeys(PlannerInfo *root,
 						  RelOptInfo *rel,
 						  List *pathkeys);
 extern bool has_useful_pathkeys(PlannerInfo *root, RelOptInfo *rel);
 extern PathKey *make_canonical_pathkey(PlannerInfo *root,
 					   EquivalenceClass *eclass, Oid opfamily,
 					   int strategy, bool nulls_first);
 
+extern Relids adjust_child_relids(Relids relids, List *append_rel_infos);
+
 #endif   /* PATHS_H */
diff --git a/src/include/optimizer/prep.h b/src/include/optimizer/prep.h
index fb35b68..a7f6271 100644
--- a/src/include/optimizer/prep.h
+++ b/src/include/optimizer/prep.h
@@ -21,20 +21,23 @@
 /*
  * prototypes for prepjointree.c
  */
 extern void pull_up_sublinks(PlannerInfo *root);
 extern void inline_set_returning_functions(PlannerInfo *root);
 extern void pull_up_subqueries(PlannerInfo *root);
 extern void flatten_simple_union_all(PlannerInfo *root);
 extern void reduce_outer_joins(PlannerInfo *root);
 extern Relids get_relids_in_jointree(Node *jtnode, bool include_joins);
 extern Relids get_relids_for_join(PlannerInfo *root, int joinrelid);
+extern Node *adjust_join_appendrel_attrs(PlannerInfo *root, Node *node,
+										 List *append_rel_infos);
+extern List *find_appinfos_by_relids(PlannerInfo *root, Relids relids);
 
 /*
  * prototypes for prepqual.c
  */
 extern Node *negate_clause(Node *node);
 extern Expr *canonicalize_qual(Expr *qual);
 
 /*
  * prototypes for prepsecurity.c
  */
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
new file mode 100644
index 0000000..b56eba9
--- /dev/null
+++ b/src/test/regress/expected/partition_join.out
@@ -0,0 +1,6663 @@
+--
+-- PARTITION_JOIN
+-- Test partition-wise join between partitioned tables
+--
+--
+-- partitioned by a single column
+--
+CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES START (0) END (250);
+CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES START (500) END (600);
+CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES START (250) END (500);
+INSERT INTO prt1 SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1 AS SELECT * FROM prt1;
+CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES START (0) END (250);
+CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES START (250) END (500);
+CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES START (500) END (600);
+INSERT INTO prt2 SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+CREATE TABLE uprt2 AS SELECT * FROM prt2;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on public.prt2_p1 t2
+                     Output: t2.b, t2.c
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+               Hash Cond: (t2_1.b = t1_2.a)
+               ->  Seq Scan on public.prt2_p2 t2_1
+                     Output: t2_1.b, t2_1.c
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+               Hash Cond: (t2_2.b = t1_1.a)
+               ->  Seq Scan on public.prt2_p3 t2_2
+                     Output: t2_2.b, t2_2.c
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on public.prt2_p1 t2
+                     Output: t2.b, t2.c
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+               Hash Cond: (t2_1.b = t1_2.a)
+               ->  Seq Scan on public.prt2_p2 t2_1
+                     Output: t2_1.b, t2_1.c
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+               Hash Cond: (t2_2.b = t1_1.a)
+               ->  Seq Scan on public.prt2_p3 t2_2
+                     Output: t2_2.b, t2_2.c
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.b % 25 = 0 ORDER BY t1.a, t2.b;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: (t1.a = t2.b)
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t2.b, t2.c
+                           ->  Seq Scan on public.prt2_p1 t2
+                                 Output: t2.b, t2.c
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Hash Cond: (t1_2.a = t2_1.b)
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c
+                           ->  Seq Scan on public.prt2_p2 t2_1
+                                 Output: t2_1.b, t2_1.c
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Hash Cond: (t1_1.a = t2_2.b)
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c
+                           ->  Seq Scan on public.prt2_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+                                 Filter: ((t2_2.b % 25) = 0)
+(36 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+   Sort Key: prt1_p1.a, prt2_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+               Hash Cond: (prt1_p1.a = prt2_p1.b)
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c
+                     Filter: ((prt1_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p1.b, prt2_p1.c
+                     ->  Seq Scan on public.prt2_p1
+                           Output: prt2_p1.b, prt2_p1.c
+                           Filter: ((prt2_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt1_p2.a = prt2_p2.b)
+               ->  Seq Scan on public.prt1_p2
+                     Output: prt1_p2.a, prt1_p2.c
+                     Filter: ((prt1_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p2.b, prt2_p2.c
+                     ->  Seq Scan on public.prt2_p2
+                           Output: prt2_p2.b, prt2_p2.c
+                           Filter: ((prt2_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c
+               Hash Cond: (prt1_p3.a = prt2_p3.b)
+               ->  Seq Scan on public.prt1_p3
+                     Output: prt1_p3.a, prt1_p3.c
+                     Filter: ((prt1_p3.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p3.b, prt2_p3.c
+                     ->  Seq Scan on public.prt2_p3
+                           Output: prt2_p3.b, prt2_p3.c
+                           Filter: ((prt2_p3.b % 25) = 0)
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 t2 WHERE t2.b % 25 = 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)
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+   Sort Key: prt1_p1.a, prt2_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+               Hash Cond: (prt1_p1.a = prt2_p1.b)
+               Filter: (((50) = prt1_p1.b) OR ((75) = prt2_p1.b))
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c, prt1_p1.b, 50
+                     Filter: ((prt1_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p1.b, prt2_p1.c, (75)
+                     ->  Seq Scan on public.prt2_p1
+                           Output: prt2_p1.b, prt2_p1.c, 75
+                           Filter: ((prt2_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt1_p2.a = prt2_p2.b)
+               Filter: (((50) = prt1_p2.b) OR ((75) = prt2_p2.b))
+               ->  Seq Scan on public.prt1_p2
+                     Output: prt1_p2.a, prt1_p2.c, prt1_p2.b, 50
+                     Filter: ((prt1_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p2.b, prt2_p2.c, (75)
+                     ->  Seq Scan on public.prt2_p2
+                           Output: prt2_p2.b, prt2_p2.c, 75
+                           Filter: ((prt2_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c
+               Hash Cond: (prt1_p3.a = prt2_p3.b)
+               Filter: (((50) = prt1_p3.b) OR ((75) = prt2_p3.b))
+               ->  Seq Scan on public.prt1_p3
+                     Output: prt1_p3.a, prt1_p3.c, prt1_p3.b, 50
+                     Filter: ((prt1_p3.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p3.b, prt2_p3.c, (75)
+                     ->  Seq Scan on public.prt2_p3
+                           Output: prt2_p3.b, prt2_p3.c, 75
+                           Filter: ((prt2_p3.b % 25) = 0)
+(40 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+ a  |  c   | b  |  c   
+----+------+----+------
+ 50 | 0050 |    | 
+    |      | 75 | 0075
+(2 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+ a  |  c   | b  |  c   
+----+------+----+------
+ 50 | 0050 |    | 
+    |      | 75 | 0075
+(2 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, (25), prt2_p1.b, prt2_p1.c, (50)
+   Sort Key: prt1_p1.a, prt2_p1.b
+   ->  Result
+         Output: prt1_p1.a, prt1_p1.c, (25), prt2_p1.b, prt2_p1.c, (50)
+         ->  Append
+               ->  Hash Full Join
+                     Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, (25), (50)
+                     Hash Cond: (prt1_p1.a = prt2_p1.b)
+                     ->  Seq Scan on public.prt1_p1
+                           Output: prt1_p1.a, prt1_p1.c, 25
+                           Filter: ((prt1_p1.a % 25) = 0)
+                     ->  Hash
+                           Output: prt2_p1.b, prt2_p1.c, (50)
+                           ->  Seq Scan on public.prt2_p1
+                                 Output: prt2_p1.b, prt2_p1.c, 50
+                                 Filter: ((prt2_p1.b % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c, (25), (50)
+                     Hash Cond: (prt1_p2.a = prt2_p2.b)
+                     ->  Seq Scan on public.prt1_p2
+                           Output: prt1_p2.a, prt1_p2.c, 25
+                           Filter: ((prt1_p2.a % 25) = 0)
+                     ->  Hash
+                           Output: prt2_p2.b, prt2_p2.c, (50)
+                           ->  Seq Scan on public.prt2_p2
+                                 Output: prt2_p2.b, prt2_p2.c, 50
+                                 Filter: ((prt2_p2.b % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c, (25), (50)
+                     Hash Cond: (prt1_p3.a = prt2_p3.b)
+                     ->  Seq Scan on public.prt1_p3
+                           Output: prt1_p3.a, prt1_p3.c, 25
+                           Filter: ((prt1_p3.a % 25) = 0)
+                     ->  Hash
+                           Output: prt2_p3.b, prt2_p3.c, (50)
+                           ->  Seq Scan on public.prt2_p3
+                                 Output: prt2_p3.b, prt2_p3.c, 50
+                                 Filter: ((prt2_p3.b % 25) = 0)
+(39 rows)
+
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 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)
+
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM uprt2 WHERE uprt2.b % 25 = 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)
+
+-- Join with pruned partitions from joining relations
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on public.prt2_p2 t2
+                     Output: t2.b, t2.c
+                     Filter: (t2.b > 250)
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.prt1_p2 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a < 450) AND ((t1.a % 25) = 0))
+(15 rows)
+
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+(1 row)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a < 450 AND t2.b > 250 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+(1 row)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                    QUERY PLAN                                    
+----------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, b, c
+   Sort Key: prt1_p1.a, b
+   ->  Append
+         ->  Nested Loop Left Join
+               Output: prt1_p1.a, prt1_p1.c, b, c
+               Join Filter: (prt1_p1.a = b)
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c
+                     Filter: ((prt1_p1.a < 450) AND ((prt1_p1.a % 25) = 0))
+               ->  Result
+                     Output: b, c
+                     One-Time Filter: false
+         ->  Hash Right Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt2_p2.b = prt1_p2.a)
+               ->  Seq Scan on public.prt2_p2
+                     Output: prt2_p2.b, prt2_p2.c
+                     Filter: (prt2_p2.b > 250)
+               ->  Hash
+                     Output: prt1_p2.a, prt1_p2.c
+                     ->  Seq Scan on public.prt1_p2
+                           Output: prt1_p2.a, prt1_p2.c
+                           Filter: ((prt1_p2.a < 450) AND ((prt1_p2.a % 25) = 0))
+(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.a % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.a % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                       QUERY PLAN                                       
+----------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+   Sort Key: prt1_p2.a, prt2_p2.b
+   ->  Result
+         Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: prt2_p2.b, prt2_p2.c, prt1_p2.a, prt1_p2.c
+                     Hash Cond: (prt1_p2.a = prt2_p2.b)
+                     ->  Seq Scan on public.prt1_p2
+                           Output: prt1_p2.a, prt1_p2.c
+                           Filter: (prt1_p2.a < 450)
+                     ->  Hash
+                           Output: prt2_p2.b, prt2_p2.c
+                           ->  Seq Scan on public.prt2_p2
+                                 Output: prt2_p2.b, prt2_p2.c
+                                 Filter: ((prt2_p2.b > 250) AND ((prt2_p2.a % 25) = 0))
+               ->  Nested Loop Left Join
+                     Output: prt2_p3.b, prt2_p3.c, a, c
+                     Join Filter: (a = prt2_p3.b)
+                     ->  Seq Scan on public.prt2_p3
+                           Output: prt2_p3.b, prt2_p3.c
+                           Filter: ((prt2_p3.b > 250) AND ((prt2_p3.a % 25) = 0))
+                     ->  Result
+                           Output: a, c
+                           One-Time Filter: false
+(26 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+     |      | 375 | 0375
+     |      | 450 | 0450
+     |      | 525 | 0525
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+     |      | 375 | 0375
+     |      | 450 | 0450
+     |      | 525 | 0525
+(4 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+                                    QUERY PLAN                                    
+----------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, b, c
+   Sort Key: prt1_p1.a, b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_p1.a, prt1_p1.c, b, c
+               Hash Cond: (prt1_p1.a = b)
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c
+                     Filter: ((prt1_p1.a < 450) AND ((prt1_p1.a % 25) = 0))
+               ->  Hash
+                     Output: b, c
+                     ->  Result
+                           Output: b, c
+                           One-Time Filter: false
+         ->  Hash Full Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt1_p2.a = prt2_p2.b)
+               ->  Seq Scan on public.prt1_p2
+                     Output: prt1_p2.a, prt1_p2.c
+                     Filter: ((prt1_p2.a < 450) AND ((prt1_p2.a % 25) = 0))
+               ->  Hash
+                     Output: prt2_p2.b, prt2_p2.c
+                     ->  Seq Scan on public.prt2_p2
+                           Output: prt2_p2.b, prt2_p2.c
+                           Filter: ((prt2_p2.b > 250) AND ((prt2_p2.b % 25) = 0))
+         ->  Hash Full Join
+               Output: a, c, prt2_p3.b, prt2_p3.c
+               Hash Cond: (prt2_p3.b = a)
+               ->  Seq Scan on public.prt2_p3
+                     Output: prt2_p3.b, prt2_p3.c
+                     Filter: ((prt2_p3.b > 250) AND ((prt2_p3.b % 25) = 0))
+               ->  Hash
+                     Output: a, c
+                     ->  Result
+                           Output: a, c
+                           One-Time Filter: false
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b 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)
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.a = t1_3.b)
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_3.b
+                     ->  Seq Scan on public.prt2_p1 t1_3
+                           Output: t1_3.b
+                           Filter: ((t1_3.b % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.a = t1_4.b)
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_4.b
+                     ->  Seq Scan on public.prt2_p2 t1_4
+                           Output: t1_4.b
+                           Filter: ((t1_4.b % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.a = t1_5.b)
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_5.b
+                     ->  Seq Scan on public.prt2_p3 t1_5
+                           Output: t1_5.b
+                           Filter: ((t1_5.b % 25) = 0)
+(37 rows)
+
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+-- lateral reference
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+                                             QUERY PLAN                                              
+-----------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   Sort Key: t1.a
+   ->  Result
+         Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+         ->  Append
+               ->  Nested Loop Left Join
+                     Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a))
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.b, t1.c
+                           Filter: ((t1.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+                           Hash Cond: (t3.b = t2.a)
+                           ->  Seq Scan on public.prt2_p1 t3
+                                 Output: t3.a, t3.b
+                           ->  Hash
+                                 Output: t2.a
+                                 ->  Seq Scan on public.prt1_p1 t2
+                                       Output: t2.a
+                                       Filter: (t1.a = t2.a)
+               ->  Nested Loop Left Join
+                     Output: t1_2.a, t1_2.b, t1_2.c, t2_2.a, t3_1.a, (LEAST(t1_2.a, t2_2.a, t3_1.a))
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.b, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2_2.a, t3_1.a, LEAST(t1_2.a, t2_2.a, t3_1.a)
+                           Hash Cond: (t3_1.b = t2_2.a)
+                           ->  Seq Scan on public.prt2_p2 t3_1
+                                 Output: t3_1.a, t3_1.b
+                           ->  Hash
+                                 Output: t2_2.a
+                                 ->  Seq Scan on public.prt1_p2 t2_2
+                                       Output: t2_2.a
+                                       Filter: (t1_2.a = t2_2.a)
+               ->  Nested Loop Left Join
+                     Output: t1_1.a, t1_1.b, t1_1.c, t2_1.a, t3_2.a, (LEAST(t1_1.a, t2_1.a, t3_2.a))
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.b, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2_1.a, t3_2.a, LEAST(t1_1.a, t2_1.a, t3_2.a)
+                           Hash Cond: (t3_2.b = t2_1.a)
+                           ->  Seq Scan on public.prt2_p3 t3_2
+                                 Output: t3_2.a, t3_2.b
+                           ->  Hash
+                                 Output: t2_1.a
+                                 ->  Seq Scan on public.prt1_p3 t2_1
+                                       Output: t2_1.a
+                                       Filter: (t1_1.a = t2_1.a)
+(51 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   Sort Key: t1.a
+   ->  Nested Loop Left Join
+         Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+         ->  Append
+               ->  Seq Scan on public.prt1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Seq Scan on public.prt1_p1 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Seq Scan on public.prt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Seq Scan on public.prt1_p2 t1_3
+                     Output: t1_3.a, t1_3.b, t1_3.c
+                     Filter: ((t1_3.a % 25) = 0)
+         ->  Append
+               ->  Hash Join
+                     Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+                     Hash Cond: (t3.b = t2.a)
+                     ->  Seq Scan on public.prt2_p1 t3
+                           Output: t3.a, t3.b
+                     ->  Hash
+                           Output: t2.a
+                           ->  Seq Scan on public.prt1_p1 t2
+                                 Output: t2.a
+                                 Filter: (t1.b = t2.a)
+               ->  Hash Join
+                     Output: t2_2.a, t3_1.a, LEAST(t1.a, t2_2.a, t3_1.a)
+                     Hash Cond: (t3_1.b = t2_2.a)
+                     ->  Seq Scan on public.prt2_p2 t3_1
+                           Output: t3_1.a, t3_1.b
+                     ->  Hash
+                           Output: t2_2.a
+                           ->  Seq Scan on public.prt1_p2 t2_2
+                                 Output: t2_2.a
+                                 Filter: (t1.b = t2_2.a)
+               ->  Hash Join
+                     Output: t2_1.a, t3_2.a, LEAST(t1.a, t2_1.a, t3_2.a)
+                     Hash Cond: (t3_2.b = t2_1.a)
+                     ->  Seq Scan on public.prt2_p3 t3_2
+                           Output: t3_2.a, t3_2.b
+                     ->  Hash
+                           Output: t2_1.a
+                           ->  Seq Scan on public.prt1_p3 t2_1
+                                 Output: t2_1.a
+                                 Filter: (t1.b = t2_1.a)
+(49 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+--
+-- partitioned by expression
+--
+CREATE TABLE prt1_e (a int, b int, c varchar) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES START (0) END (250);
+CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES START (250) END (500);
+CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES START (500) END (600);
+INSERT INTO prt1_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_e AS SELECT * FROM prt1_e;
+CREATE TABLE prt2_e (a int, b int, c varchar) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES START (0) END (250);
+CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES START (250) END (500);
+CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES START (500) END (600);
+INSERT INTO prt2_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_e;
+ANALYZE prt2_e_p1;
+ANALYZE prt2_e_p2;
+ANALYZE prt2_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_e AS SELECT * FROM prt2_e;
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                  QUERY PLAN                                  
+------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_e_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_e_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
+               ->  Seq Scan on public.prt2_e_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_e_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+(34 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.a % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1, uprt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.a % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                  QUERY PLAN                                  
+------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_e_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_e_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
+               ->  Seq Scan on public.prt2_e_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_e_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1 LEFT JOIN uprt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 RIGHT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: (((t1.a + t1.b) / 2) = ((t2.b + t2.a) / 2))
+                     ->  Seq Scan on public.prt1_e_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                     ->  Hash
+                           Output: t2.b, t2.c, t2.a
+                           ->  Seq Scan on public.prt2_e_p1 t2
+                                 Output: t2.b, t2.c, t2.a
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: (((t1_1.a + t1_1.b) / 2) = ((t2_1.b + t2_1.a) / 2))
+                     ->  Seq Scan on public.prt1_e_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c, t2_1.a
+                           ->  Seq Scan on public.prt2_e_p2 t2_1
+                                 Output: t2_1.b, t2_1.c, t2_1.a
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: (((t1_2.a + t1_2.b) / 2) = ((t2_2.b + t2_2.a) / 2))
+                     ->  Seq Scan on public.prt1_e_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c, t2_2.a
+                           ->  Seq Scan on public.prt2_e_p3 t2_2
+                                 Output: t2_2.b, t2_2.c, t2_2.a
+                                 Filter: ((t2_2.b % 25) = 0)
+(36 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 RIGHT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t2.b % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1 RIGHT JOIN uprt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t2.b % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_e WHERE prt2_e.b % 25 = 0) t2 ON ((t1.a + t1.b)/2 = (t2.b + t2.a)/2) ORDER BY t1.a, t2.b;
+                                            QUERY PLAN                                            
+--------------------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_e_p1.a, prt1_e_p1.c, prt2_e_p1.b, prt2_e_p1.c
+   Sort Key: prt1_e_p1.a, prt2_e_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_e_p1.a, prt1_e_p1.c, prt2_e_p1.b, prt2_e_p1.c
+               Hash Cond: (((prt1_e_p1.a + prt1_e_p1.b) / 2) = ((prt2_e_p1.b + prt2_e_p1.a) / 2))
+               ->  Seq Scan on public.prt1_e_p1
+                     Output: prt1_e_p1.a, prt1_e_p1.c, prt1_e_p1.b
+                     Filter: ((prt1_e_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_e_p1.b, prt2_e_p1.c, prt2_e_p1.a
+                     ->  Seq Scan on public.prt2_e_p1
+                           Output: prt2_e_p1.b, prt2_e_p1.c, prt2_e_p1.a
+                           Filter: ((prt2_e_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_e_p2.a, prt1_e_p2.c, prt2_e_p2.b, prt2_e_p2.c
+               Hash Cond: (((prt1_e_p2.a + prt1_e_p2.b) / 2) = ((prt2_e_p2.b + prt2_e_p2.a) / 2))
+               ->  Seq Scan on public.prt1_e_p2
+                     Output: prt1_e_p2.a, prt1_e_p2.c, prt1_e_p2.b
+                     Filter: ((prt1_e_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_e_p2.b, prt2_e_p2.c, prt2_e_p2.a
+                     ->  Seq Scan on public.prt2_e_p2
+                           Output: prt2_e_p2.b, prt2_e_p2.c, prt2_e_p2.a
+                           Filter: ((prt2_e_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_e_p3.a, prt1_e_p3.c, prt2_e_p3.b, prt2_e_p3.c
+               Hash Cond: (((prt1_e_p3.a + prt1_e_p3.b) / 2) = ((prt2_e_p3.b + prt2_e_p3.a) / 2))
+               ->  Seq Scan on public.prt1_e_p3
+                     Output: prt1_e_p3.a, prt1_e_p3.c, prt1_e_p3.b
+                     Filter: ((prt1_e_p3.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_e_p3.b, prt2_e_p3.c, prt2_e_p3.a
+                     ->  Seq Scan on public.prt2_e_p3
+                           Output: prt2_e_p3.b, prt2_e_p3.c, prt2_e_p3.a
+                           Filter: ((prt2_e_p3.b % 25) = 0)
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_e WHERE prt2_e.b % 25 = 0) t2 ON ((t1.a + t1.b)/2 = (t2.b + t2.a)/2) 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_e t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_e t2 WHERE t2.b % 25 = 0) t2 ON ((t1.a + t1.b)/2 = (t2.b + t2.a)/2) 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)
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on public.prt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = t1.a)
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_2.a)
+                     ->  Seq Scan on public.prt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                           ->  Hash Join
+                                 Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = t1_2.a)
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_1.a)
+                     ->  Seq Scan on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                           ->  Hash Join
+                                 Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = t1_1.a)
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+(57 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM uprt1 t1, uprt2 t2, uprt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on public.prt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = t1.a)
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_2.a)
+                     ->  Seq Scan on public.prt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = t1_2.a)
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_1.a)
+                     ->  Seq Scan on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = t1_1.a)
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+(57 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, 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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: (((t3.a + t3.b) / 2) = t2.b)
+                     ->  Seq Scan on public.prt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = t1.a)
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: (((t3_1.a + t3_1.b) / 2) = t2_1.b)
+                     ->  Seq Scan on public.prt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = t1_2.a)
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: (((t3_2.a + t3_2.b) / 2) = t2_2.b)
+                     ->  Seq Scan on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = t1_1.a)
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+(57 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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |          | 
+ 100 | 0100 |     |      |          | 
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |          | 
+ 250 | 0250 |     |      |          | 
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |          | 
+ 400 | 0400 |     |      |          | 
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |          | 
+ 550 | 0550 |     |      |          | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |          | 
+ 100 | 0100 |     |      |          | 
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |          | 
+ 250 | 0250 |     |      |          | 
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |          | 
+ 400 | 0400 |     |      |          | 
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |          | 
+ 550 | 0550 |     |      |          | 
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Hash Cond: (t2.b = t1.a)
+                     ->  Seq Scan on public.prt2_p1 t2
+                           Output: t2.b, t2.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Hash Cond: (t1.a = ((t3.a + t3.b) / 2))
+                                 ->  Seq Scan on public.prt1_p1 t1
+                                       Output: t1.a, t1.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                     Hash Cond: (t2_1.b = t1_2.a)
+                     ->  Seq Scan on public.prt2_p2 t2_1
+                           Output: t2_1.b, t2_1.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                                 Hash Cond: (t1_2.a = ((t3_1.a + t3_1.b) / 2))
+                                 ->  Seq Scan on public.prt1_p2 t1_2
+                                       Output: t1_2.a, t1_2.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                     Hash Cond: (t2_2.b = t1_1.a)
+                     ->  Seq Scan on public.prt2_p3 t2_2
+                           Output: t2_2.b, t2_2.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                                 Hash Cond: (t1_1.a = ((t3_2.a + t3_2.b) / 2))
+                                 ->  Seq Scan on public.prt1_p3 t1_1
+                                       Output: t1_1.a, t1_1.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: (t1.a = t2.b)
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = ((t3.a + t3.b) / 2))
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Hash Cond: (t1_2.a = t2_1.b)
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = ((t3_1.a + t3_1.b) / 2))
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Hash Cond: (t1_1.a = t2_2.b)
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = ((t3_2.a + t3_2.b) / 2))
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                                  QUERY PLAN                                                   
+---------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, ((prt1_e_p1.a + prt1_e_p1.b)), prt1_e_p1.c
+   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   ->  Result
+         Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, (prt1_e_p1.a + prt1_e_p1.b), prt1_e_p1.c
+         ->  Append
+               ->  Hash Full Join
+                     Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, prt1_e_p1.a, prt1_e_p1.b, prt1_e_p1.c
+                     Hash Cond: (prt1_p1.a = ((prt1_e_p1.a + prt1_e_p1.b) / 2))
+                     ->  Hash Full Join
+                           Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on public.prt1_p1
+                                 Output: prt1_p1.a, prt1_p1.c
+                                 Filter: ((prt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p1.b, prt2_p1.c
+                                 ->  Seq Scan on public.prt2_p1
+                                       Output: prt2_p1.b, prt2_p1.c
+                                       Filter: ((prt2_p1.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p1.a, prt1_e_p1.b, prt1_e_p1.c
+                           ->  Seq Scan on public.prt1_e_p1
+                                 Output: prt1_e_p1.a, prt1_e_p1.b, prt1_e_p1.c
+                                 Filter: ((prt1_e_p1.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c, prt1_e_p2.a, prt1_e_p2.b, prt1_e_p2.c
+                     Hash Cond: (prt1_p2.a = ((prt1_e_p2.a + prt1_e_p2.b) / 2))
+                     ->  Hash Full Join
+                           Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+                           Hash Cond: (prt1_p2.a = prt2_p2.b)
+                           ->  Seq Scan on public.prt1_p2
+                                 Output: prt1_p2.a, prt1_p2.c
+                                 Filter: ((prt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p2.b, prt2_p2.c
+                                 ->  Seq Scan on public.prt2_p2
+                                       Output: prt2_p2.b, prt2_p2.c
+                                       Filter: ((prt2_p2.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p2.a, prt1_e_p2.b, prt1_e_p2.c
+                           ->  Seq Scan on public.prt1_e_p2
+                                 Output: prt1_e_p2.a, prt1_e_p2.b, prt1_e_p2.c
+                                 Filter: ((prt1_e_p2.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c, prt1_e_p3.a, prt1_e_p3.b, prt1_e_p3.c
+                     Hash Cond: (prt1_p3.a = ((prt1_e_p3.a + prt1_e_p3.b) / 2))
+                     ->  Hash Full Join
+                           Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c
+                           Hash Cond: (prt1_p3.a = prt2_p3.b)
+                           ->  Seq Scan on public.prt1_p3
+                                 Output: prt1_p3.a, prt1_p3.c
+                                 Filter: ((prt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p3.b, prt2_p3.c
+                                 ->  Seq Scan on public.prt2_p3
+                                       Output: prt2_p3.b, prt2_p3.c
+                                       Filter: ((prt2_p3.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p3.a, prt1_e_p3.b, prt1_e_p3.c
+                           ->  Seq Scan on public.prt1_e_p3
+                                 Output: prt1_e_p3.a, prt1_e_p3.b, prt1_e_p3.c
+                                 Filter: ((prt1_e_p3.a % 25) = 0)
+(63 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+     |      |  75 | 0075 |          | 
+     |      | 225 | 0225 |          | 
+     |      | 375 | 0375 |          | 
+     |      | 525 | 0525 |          | 
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM uprt1_e WHERE uprt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+     |      |  75 | 0075 |          | 
+     |      | 225 | 0225 |          | 
+     |      | 375 | 0375 |          | 
+     |      | 525 | 0525 |          | 
+(16 rows)
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, 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.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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                                                      
+----------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, (50), prt2_p1.b, (75), ((prt1_e_p1.a + prt1_e_p1.b)), (50)
+   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   ->  Result
+         Output: prt1_p1.a, (50), prt2_p1.b, (75), (prt1_e_p1.a + prt1_e_p1.b), (50)
+         ->  Append
+               ->  Hash Full Join
+                     Output: prt1_p1.a, prt2_p1.b, prt1_e_p1.a, prt1_e_p1.b, (50), (75), (50)
+                     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
+                           Output: prt1_p1.a, prt2_p1.b, (50), (75)
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on public.prt1_p1
+                                 Output: prt1_p1.a, 50
+                                 Filter: ((prt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p1.b, (75)
+                                 ->  Seq Scan on public.prt2_p1
+                                       Output: prt2_p1.b, 75
+                                       Filter: ((prt2_p1.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p1.a, prt1_e_p1.b, (50)
+                           ->  Seq Scan on public.prt1_e_p1
+                                 Output: prt1_e_p1.a, prt1_e_p1.b, 50
+                                 Filter: ((prt1_e_p1.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p2.a, prt2_p2.b, prt1_e_p2.a, prt1_e_p2.b, (50), (75), (50)
+                     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
+                           Output: prt1_p2.a, prt2_p2.b, (50), (75)
+                           Hash Cond: (prt1_p2.a = prt2_p2.b)
+                           ->  Seq Scan on public.prt1_p2
+                                 Output: prt1_p2.a, 50
+                                 Filter: ((prt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p2.b, (75)
+                                 ->  Seq Scan on public.prt2_p2
+                                       Output: prt2_p2.b, 75
+                                       Filter: ((prt2_p2.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p2.a, prt1_e_p2.b, (50)
+                           ->  Seq Scan on public.prt1_e_p2
+                                 Output: prt1_e_p2.a, prt1_e_p2.b, 50
+                                 Filter: ((prt1_e_p2.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p3.a, prt2_p3.b, prt1_e_p3.a, prt1_e_p3.b, (50), (75), (50)
+                     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
+                           Output: prt1_p3.a, prt2_p3.b, (50), (75)
+                           Hash Cond: (prt1_p3.a = prt2_p3.b)
+                           ->  Seq Scan on public.prt1_p3
+                                 Output: prt1_p3.a, 50
+                                 Filter: ((prt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p3.b, (75)
+                                 ->  Seq Scan on public.prt2_p3
+                                       Output: prt2_p3.b, 75
+                                       Filter: ((prt2_p3.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p3.a, prt1_e_p3.b, (50)
+                           ->  Seq Scan on public.prt1_e_p3
+                                 Output: prt1_e_p3.a, prt1_e_p3.b, 50
+                                 Filter: ((prt1_e_p3.a % 25) = 0)
+(66 rows)
+
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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)
+
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM uprt1_e WHERE uprt1_e.a % 25 = 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)
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.a = t1_3.b)
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_3.b, t2.a, t2.b
+                     ->  Hash Join
+                           Output: t1_3.b, t2.a, t2.b
+                           Hash Cond: (((t2.a + t2.b) / 2) = t1_3.b)
+                           ->  Seq Scan on public.prt1_e_p1 t2
+                                 Output: t2.a, t2.b
+                           ->  Hash
+                                 Output: t1_3.b
+                                 ->  Seq Scan on public.prt2_p1 t1_3
+                                       Output: t1_3.b
+                                       Filter: ((t1_3.b % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.a = t1_4.b)
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_4.b, t2_1.a, t2_1.b
+                     ->  Hash Join
+                           Output: t1_4.b, t2_1.a, t2_1.b
+                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_4.b)
+                           ->  Seq Scan on public.prt1_e_p2 t2_1
+                                 Output: t2_1.a, t2_1.b
+                           ->  Hash
+                                 Output: t1_4.b
+                                 ->  Seq Scan on public.prt2_p2 t1_4
+                                       Output: t1_4.b
+                                       Filter: ((t1_4.b % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.a = t1_5.b)
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_5.b, t2_2.a, t2_2.b
+                     ->  Hash Join
+                           Output: t1_5.b, t2_2.a, t2_2.b
+                           Hash Cond: (((t2_2.a + t2_2.b) / 2) = t1_5.b)
+                           ->  Seq Scan on public.prt1_e_p3 t2_2
+                                 Output: t2_2.a, t2_2.b
+                           ->  Hash
+                                 Output: t1_5.b
+                                 ->  Seq Scan on public.prt2_p3 t1_5
+                                       Output: t1_5.b
+                                       Filter: ((t1_5.b % 25) = 0)
+(58 rows)
+
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1, uprt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.a = t1_3.b)
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_3.b, t1_6.a, t1_6.b
+                     ->  Hash Semi Join
+                           Output: t1_3.b, t1_6.a, t1_6.b
+                           Hash Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
+                           ->  Seq Scan on public.prt2_p1 t1_3
+                                 Output: t1_3.b
+                           ->  Hash
+                                 Output: t1_6.a, t1_6.b
+                                 ->  Seq Scan on public.prt1_e_p1 t1_6
+                                       Output: t1_6.a, t1_6.b
+                                       Filter: ((t1_6.a % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.a = t1_4.b)
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_4.b, t1_7.a, t1_7.b
+                     ->  Hash Semi Join
+                           Output: t1_4.b, t1_7.a, t1_7.b
+                           Hash Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
+                           ->  Seq Scan on public.prt2_p2 t1_4
+                                 Output: t1_4.b
+                           ->  Hash
+                                 Output: t1_7.a, t1_7.b
+                                 ->  Seq Scan on public.prt1_e_p2 t1_7
+                                       Output: t1_7.a, t1_7.b
+                                       Filter: ((t1_7.a % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.a = t1_5.b)
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_5.b, t1_8.a, t1_8.b
+                     ->  Hash Semi Join
+                           Output: t1_5.b, t1_8.a, t1_8.b
+                           Hash Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
+                           ->  Seq Scan on public.prt2_p3 t1_5
+                                 Output: t1_5.b
+                           ->  Hash
+                                 Output: t1_8.a, t1_8.b
+                                 ->  Seq Scan on public.prt1_e_p3 t1_8
+                                       Output: t1_8.a, t1_8.b
+                                       Filter: ((t1_8.a % 25) = 0)
+(58 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+-- test merge joins with and without using indexes
+SET enable_hashjoin TO off;
+SET enable_nestloop TO off;
+CREATE INDEX iprt1_a on prt1(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 iprt2_b on prt2(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 iprt1_e_ab2 on prt1_e(((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));
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                             QUERY PLAN                                              
+-----------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Merge Cond: (t2.b = t1.a)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           Sort Key: t2.b
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Merge Cond: ((((t3.a + t3.b) / 2)) = t2.b)
+                                 ->  Sort
+                                       Output: t3.a, t3.b, t3.c, (((t3.a + t3.b) / 2))
+                                       Sort Key: (((t3.a + t3.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c, ((t3.a + t3.b) / 2)
+                                             Filter: ((t3.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2.b, t2.c
+                                       Sort Key: t2.b
+                                       ->  Seq Scan on public.prt2_p1 t2
+                                             Output: t2.b, t2.c
+                     ->  Sort
+                           Output: t1.a, t1.c
+                           Sort Key: t1.a
+                           ->  Seq Scan on public.prt1_p1 t1
+                                 Output: t1.a, t1.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Merge Cond: (t2_1.b = t1_2.a)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           Sort Key: t2_1.b
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t2_1.b)
+                                 ->  Sort
+                                       Output: t3_1.a, t3_1.b, t3_1.c, (((t3_1.a + t3_1.b) / 2))
+                                       Sort Key: (((t3_1.a + t3_1.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c, ((t3_1.a + t3_1.b) / 2)
+                                             Filter: ((t3_1.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2_1.b, t2_1.c
+                                       Sort Key: t2_1.b
+                                       ->  Seq Scan on public.prt2_p2 t2_1
+                                             Output: t2_1.b, t2_1.c
+                     ->  Sort
+                           Output: t1_2.a, t1_2.c
+                           Sort Key: t1_2.a
+                           ->  Seq Scan on public.prt1_p2 t1_2
+                                 Output: t1_2.a, t1_2.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Merge Cond: (t2_2.b = t1_1.a)
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           Sort Key: t2_2.b
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t2_2.b)
+                                 ->  Sort
+                                       Output: t3_2.a, t3_2.b, t3_2.c, (((t3_2.a + t3_2.b) / 2))
+                                       Sort Key: (((t3_2.a + t3_2.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c, ((t3_2.a + t3_2.b) / 2)
+                                             Filter: ((t3_2.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2_2.b, t2_2.c
+                                       Sort Key: t2_2.b
+                                       ->  Seq Scan on public.prt2_p3 t2_2
+                                             Output: t2_2.b, t2_2.c
+                     ->  Sort
+                           Output: t1_1.a, t1_1.c
+                           Sort Key: t1_1.a
+                           ->  Seq Scan on public.prt1_p3 t1_1
+                                 Output: t1_1.a, t1_1.c
+(81 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+                                   QUERY PLAN                                    
+---------------------------------------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Output: t1.a, t1.b, t1.c
+         Merge Cond: (t1.a = t1_3.b)
+         ->  Sort
+               Output: t1.a, t1.b, t1.c
+               Sort Key: t1.a
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_3.b, t1_6.a, t1_6.b
+               ->  Merge Semi Join
+                     Output: t1_3.b, t1_6.a, t1_6.b
+                     Merge Cond: (t1_3.b = (((t1_6.a + t1_6.b) / 2)))
+                     ->  Sort
+                           Output: t1_3.b
+                           Sort Key: t1_3.b
+                           ->  Seq Scan on public.prt2_p1 t1_3
+                                 Output: t1_3.b
+                     ->  Sort
+                           Output: t1_6.a, t1_6.b, (((t1_6.a + t1_6.b) / 2))
+                           Sort Key: (((t1_6.a + t1_6.b) / 2))
+                           ->  Seq Scan on public.prt1_e_p1 t1_6
+                                 Output: t1_6.a, t1_6.b, ((t1_6.a + t1_6.b) / 2)
+                                 Filter: ((t1_6.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_2.a, t1_2.b, t1_2.c
+         Merge Cond: (t1_2.a = t1_4.b)
+         ->  Sort
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Sort Key: t1_2.a
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+         ->  Materialize
+               Output: t1_4.b, t1_7.a, t1_7.b
+               ->  Merge Semi Join
+                     Output: t1_4.b, t1_7.a, t1_7.b
+                     Merge Cond: (t1_4.b = (((t1_7.a + t1_7.b) / 2)))
+                     ->  Sort
+                           Output: t1_4.b
+                           Sort Key: t1_4.b
+                           ->  Seq Scan on public.prt2_p2 t1_4
+                                 Output: t1_4.b
+                     ->  Sort
+                           Output: t1_7.a, t1_7.b, (((t1_7.a + t1_7.b) / 2))
+                           Sort Key: (((t1_7.a + t1_7.b) / 2))
+                           ->  Seq Scan on public.prt1_e_p2 t1_7
+                                 Output: t1_7.a, t1_7.b, ((t1_7.a + t1_7.b) / 2)
+                                 Filter: ((t1_7.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_1.a, t1_1.b, t1_1.c
+         Merge Cond: (t1_1.a = t1_5.b)
+         ->  Sort
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Sort Key: t1_1.a
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_5.b, t1_8.a, t1_8.b
+               ->  Merge Semi Join
+                     Output: t1_5.b, t1_8.a, t1_8.b
+                     Merge Cond: (t1_5.b = (((t1_8.a + t1_8.b) / 2)))
+                     ->  Sort
+                           Output: t1_5.b
+                           Sort Key: t1_5.b
+                           ->  Seq Scan on public.prt2_p3 t1_5
+                                 Output: t1_5.b
+                     ->  Sort
+                           Output: t1_8.a, t1_8.b, (((t1_8.a + t1_8.b) / 2))
+                           Sort Key: (((t1_8.a + t1_8.b) / 2))
+                           ->  Seq Scan on public.prt1_e_p3 t1_8
+                                 Output: t1_8.a, t1_8.b, ((t1_8.a + t1_8.b) / 2)
+                                 Filter: ((t1_8.a % 25) = 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                             QUERY PLAN                                              
+-----------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Merge Cond: (t1.a = t2.b)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           Sort Key: t1.a
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Merge Cond: ((((t3.a + t3.b) / 2)) = t1.a)
+                                 ->  Sort
+                                       Output: t3.a, t3.b, t3.c, (((t3.a + t3.b) / 2))
+                                       Sort Key: (((t3.a + t3.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c, ((t3.a + t3.b) / 2)
+                                             Filter: ((t3.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t1.a, t1.c
+                                       Sort Key: t1.a
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                     ->  Sort
+                           Output: t2.b, t2.c
+                           Sort Key: t2.b
+                           ->  Seq Scan on public.prt2_p1 t2
+                                 Output: t2.b, t2.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                     Merge Cond: (t1_2.a = t2_1.b)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                           Sort Key: t1_2.a
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                                 Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t1_2.a)
+                                 ->  Sort
+                                       Output: t3_1.a, t3_1.b, t3_1.c, (((t3_1.a + t3_1.b) / 2))
+                                       Sort Key: (((t3_1.a + t3_1.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c, ((t3_1.a + t3_1.b) / 2)
+                                             Filter: ((t3_1.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t1_2.a, t1_2.c
+                                       Sort Key: t1_2.a
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                     ->  Sort
+                           Output: t2_1.b, t2_1.c
+                           Sort Key: t2_1.b
+                           ->  Seq Scan on public.prt2_p2 t2_1
+                                 Output: t2_1.b, t2_1.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                     Merge Cond: (t1_1.a = t2_2.b)
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                           Sort Key: t1_1.a
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                                 Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t1_1.a)
+                                 ->  Sort
+                                       Output: t3_2.a, t3_2.b, t3_2.c, (((t3_2.a + t3_2.b) / 2))
+                                       Sort Key: (((t3_2.a + t3_2.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c, ((t3_2.a + t3_2.b) / 2)
+                                             Filter: ((t3_2.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t1_1.a, t1_1.c
+                                       Sort Key: t1_1.a
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                     ->  Sort
+                           Output: t2_2.b, t2_2.c
+                           Sort Key: t2_2.b
+                           ->  Seq Scan on public.prt2_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+(81 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SET enable_seqscan TO off;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                          QUERY PLAN                                          
+----------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Merge Cond: (t2.b = t1.a)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           Sort Key: t2.b
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Merge Cond: (((t3.a + t3.b) / 2) = t2.b)
+                                 ->  Index Scan using iprt1_e_p1_ab2 on public.prt1_e_p1 t3
+                                       Output: t3.a, t3.b, t3.c
+                                       Filter: ((t3.a % 25) = 0)
+                                 ->  Index Scan using iprt2_p1_b on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                     ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Merge Cond: (t2_1.b = t1_2.a)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           Sort Key: t2_1.b
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Merge Cond: (((t3_1.a + t3_1.b) / 2) = t2_1.b)
+                                 ->  Index Scan using iprt1_e_p2_ab2 on public.prt1_e_p2 t3_1
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       Filter: ((t3_1.a % 25) = 0)
+                                 ->  Index Scan using iprt2_p2_b on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                     ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+               ->  Merge Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Merge Cond: (t2_2.b = ((t3_2.a + t3_2.b) / 2))
+                     ->  Merge Left Join
+                           Output: t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                           Merge Cond: (t2_2.b = t1_1.a)
+                           ->  Index Scan using iprt2_p3_b on public.prt2_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+                           ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_1
+                                 Output: t1_1.a, t1_1.c
+                     ->  Index Scan using iprt1_e_p3_ab2 on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                           Filter: ((t3_2.a % 25) = 0)
+(51 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+                                    QUERY PLAN                                    
+----------------------------------------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Output: t1.a, t1.b, t1.c
+         Merge Cond: (t1.a = t1_3.b)
+         ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1
+               Output: t1.a, t1.b, t1.c
+               Filter: ((t1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_3.b, t1_6.a, t1_6.b
+               ->  Merge Semi Join
+                     Output: t1_3.b, t1_6.a, t1_6.b
+                     Merge Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
+                     ->  Index Only Scan using iprt2_p1_b on public.prt2_p1 t1_3
+                           Output: t1_3.b
+                     ->  Index Scan using iprt1_e_p1_ab2 on public.prt1_e_p1 t1_6
+                           Output: t1_6.a, t1_6.b
+                           Filter: ((t1_6.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_2.a, t1_2.b, t1_2.c
+         Merge Cond: (t1_2.a = t1_4.b)
+         ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_2
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Filter: ((t1_2.a % 25) = 0)
+         ->  Materialize
+               Output: t1_4.b, t1_7.a, t1_7.b
+               ->  Merge Semi Join
+                     Output: t1_4.b, t1_7.a, t1_7.b
+                     Merge Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
+                     ->  Index Only Scan using iprt2_p2_b on public.prt2_p2 t1_4
+                           Output: t1_4.b
+                     ->  Index Scan using iprt1_e_p2_ab2 on public.prt1_e_p2 t1_7
+                           Output: t1_7.a, t1_7.b
+                           Filter: ((t1_7.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_1.a, t1_1.b, t1_1.c
+         Merge Cond: (t1_1.a = t1_5.b)
+         ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_1
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Filter: ((t1_1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_5.b, t1_8.a, t1_8.b
+               ->  Merge Semi Join
+                     Output: t1_5.b, t1_8.a, t1_8.b
+                     Merge Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
+                     ->  Index Only Scan using iprt2_p3_b on public.prt2_p3 t1_5
+                           Output: t1_5.b
+                     ->  Index Scan using iprt1_e_p3_ab2 on public.prt1_e_p3 t1_8
+                           Output: t1_8.a, t1_8.b
+                           Filter: ((t1_8.a % 25) = 0)
+(50 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                          QUERY PLAN                                          
+----------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Merge Cond: (t1.a = t2.b)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           Sort Key: t1.a
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Merge Cond: (((t3.a + t3.b) / 2) = t1.a)
+                                 ->  Index Scan using iprt1_e_p1_ab2 on public.prt1_e_p1 t3
+                                       Output: t3.a, t3.b, t3.c
+                                       Filter: ((t3.a % 25) = 0)
+                                 ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1
+                                       Output: t1.a, t1.c
+                     ->  Index Scan using iprt2_p1_b on public.prt2_p1 t2
+                           Output: t2.b, t2.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                     Merge Cond: (t1_2.a = t2_1.b)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                           Sort Key: t1_2.a
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                                 Merge Cond: (((t3_1.a + t3_1.b) / 2) = t1_2.a)
+                                 ->  Index Scan using iprt1_e_p2_ab2 on public.prt1_e_p2 t3_1
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       Filter: ((t3_1.a % 25) = 0)
+                                 ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_2
+                                       Output: t1_2.a, t1_2.c
+                     ->  Index Scan using iprt2_p2_b on public.prt2_p2 t2_1
+                           Output: t2_1.b, t2_1.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                     Merge Cond: (t1_1.a = t2_2.b)
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                           Sort Key: t1_1.a
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                                 Merge Cond: (((t3_2.a + t3_2.b) / 2) = t1_1.a)
+                                 ->  Index Scan using iprt1_e_p3_ab2 on public.prt1_e_p3 t3_2
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       Filter: ((t3_2.a % 25) = 0)
+                                 ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_1
+                                       Output: t1_1.a, t1_1.c
+                     ->  Index Scan using iprt2_p3_b on public.prt2_p3 t2_2
+                           Output: t2_2.b, t2_2.c
+(54 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+-- lateral references and parameterized paths
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
+ Nested Loop Left Join
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   ->  Merge Append
+         Sort Key: t1.a
+         ->  Index Scan using iprt1_a on public.prt1 t1
+               Output: t1.a, t1.b, t1.c
+               Filter: ((t1.a % 25) = 0)
+         ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1_1
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Filter: ((t1_1.a % 25) = 0)
+         ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_2
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Filter: ((t1_2.a % 25) = 0)
+         ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_3
+               Output: t1_3.a, t1_3.b, t1_3.c
+               Filter: ((t1_3.a % 25) = 0)
+   ->  Append
+         ->  Merge Join
+               Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+               Merge Cond: (t2.a = t3.b)
+               ->  Index Only Scan using iprt1_p1_a on public.prt1_p1 t2
+                     Output: t2.a
+                     Index Cond: (t2.a = t1.a)
+               ->  Index Scan using iprt2_p1_b on public.prt2_p1 t3
+                     Output: t3.a, t3.b
+         ->  Merge Join
+               Output: t2_2.a, t3_1.a, LEAST(t1.a, t2_2.a, t3_1.a)
+               Merge Cond: (t2_2.a = t3_1.b)
+               ->  Index Only Scan using iprt1_p2_a on public.prt1_p2 t2_2
+                     Output: t2_2.a
+                     Index Cond: (t2_2.a = t1.a)
+               ->  Index Scan using iprt2_p2_b on public.prt2_p2 t3_1
+                     Output: t3_1.a, t3_1.b
+         ->  Merge Join
+               Output: t2_1.a, t3_2.a, LEAST(t1.a, t2_1.a, t3_2.a)
+               Merge Cond: (t2_1.a = t3_2.b)
+               ->  Index Only Scan using iprt1_p3_a on public.prt1_p3 t2_1
+                     Output: t2_1.a
+                     Index Cond: (t2_1.a = t1.a)
+               ->  Index Scan using iprt2_p3_b on public.prt2_p3 t3_2
+                     Output: t3_2.a, t3_2.b
+(41 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
+ Nested Loop Left Join
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   ->  Merge Append
+         Sort Key: t1.a
+         ->  Index Scan using iprt1_a on public.prt1 t1
+               Output: t1.a, t1.b, t1.c
+               Filter: ((t1.a % 25) = 0)
+         ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1_1
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Filter: ((t1_1.a % 25) = 0)
+         ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_2
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Filter: ((t1_2.a % 25) = 0)
+         ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_3
+               Output: t1_3.a, t1_3.b, t1_3.c
+               Filter: ((t1_3.a % 25) = 0)
+   ->  Append
+         ->  Merge Join
+               Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+               Merge Cond: (t2.a = t3.b)
+               ->  Index Only Scan using iprt1_p1_a on public.prt1_p1 t2
+                     Output: t2.a
+                     Index Cond: (t2.a = t1.b)
+               ->  Index Scan using iprt2_p1_b on public.prt2_p1 t3
+                     Output: t3.a, t3.b
+         ->  Merge Join
+               Output: t2_2.a, t3_1.a, LEAST(t1.a, t2_2.a, t3_1.a)
+               Merge Cond: (t2_2.a = t3_1.b)
+               ->  Index Only Scan using iprt1_p2_a on public.prt1_p2 t2_2
+                     Output: t2_2.a
+                     Index Cond: (t2_2.a = t1.b)
+               ->  Index Scan using iprt2_p2_b on public.prt2_p2 t3_1
+                     Output: t3_1.a, t3_1.b
+         ->  Merge Join
+               Output: t2_1.a, t3_2.a, LEAST(t1.a, t2_1.a, t3_2.a)
+               Merge Cond: (t2_1.a = t3_2.b)
+               ->  Index Only Scan using iprt1_p3_a on public.prt1_p3 t2_1
+                     Output: t2_1.a
+                     Index Cond: (t2_1.a = t1.b)
+               ->  Index Scan using iprt2_p3_b on public.prt2_p3 t3_2
+                     Output: t3_2.a, t3_2.b
+(41 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+RESET enable_hashjoin;
+RESET enable_nestloop;
+RESET enable_seqscan;
+--
+-- partitioned by multiple columns
+--
+CREATE TABLE prt1_m (a int, b int, c varchar) PARTITION BY RANGE(a, ((a + b)/2));
+CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES START (0, 0) END (250, 250);
+CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES START (250, 250) END (500, 500);
+CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES START (500, 500) END (600, 600);
+INSERT INTO prt1_m SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_m;
+ANALYZE prt1_m_p1;
+ANALYZE prt1_m_p2;
+ANALYZE prt1_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_m AS SELECT * FROM prt1_m;
+CREATE TABLE prt2_m (a int, b int, c varchar) PARTITION BY RANGE(((b + a)/2), b);
+CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES START (0, 0) END (250, 250);
+CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES START (250, 250) END (500, 500);
+CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES START (500, 500) END (600, 600);
+INSERT INTO prt2_m SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_m;
+ANALYZE prt2_m_p1;
+ANALYZE prt2_m_p2;
+ANALYZE prt2_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_m AS SELECT * FROM prt2_m;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1, prt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                              QUERY PLAN                                              
+------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((((t2.b + t2.a) / 2) = t1.a) AND (t2.b = ((t1.a + t1.b) / 2)))
+               ->  Seq Scan on public.prt2_m_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_m_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((((t2_1.b + t2_1.a) / 2) = t1_1.a) AND (t2_1.b = ((t1_1.a + t1_1.b) / 2)))
+               ->  Seq Scan on public.prt2_m_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_m_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((((t2_2.b + t2_2.a) / 2) = t1_2.a) AND (t2_2.b = ((t1_2.a + t1_2.b) / 2)))
+               ->  Seq Scan on public.prt2_m_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_m_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1, prt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 AND t1.a % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_m t1, uprt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 AND t1.a % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                              QUERY PLAN                                              
+------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((((t2.b + t2.a) / 2) = t1.a) AND (t2.b = ((t1.a + t1.b) / 2)))
+               ->  Seq Scan on public.prt2_m_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_m_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((((t2_1.b + t2_1.a) / 2) = t1_1.a) AND (t2_1.b = ((t1_1.a + t1_1.b) / 2)))
+               ->  Seq Scan on public.prt2_m_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_m_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((((t2_2.b + t2_2.a) / 2) = t1_2.a) AND (t2_2.b = ((t1_2.a + t1_2.b) / 2)))
+               ->  Seq Scan on public.prt2_m_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_m_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_m t1 LEFT JOIN uprt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+                                                 QUERY PLAN                                                 
+------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: ((((t1.a + t1.b) / 2) = t2.b) AND (t1.a = ((t2.b + t2.a) / 2)))
+                     ->  Seq Scan on public.prt1_m_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                     ->  Hash
+                           Output: t2.b, t2.c, t2.a
+                           ->  Seq Scan on public.prt2_m_p1 t2
+                                 Output: t2.b, t2.c, t2.a
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: ((((t1_1.a + t1_1.b) / 2) = t2_1.b) AND (t1_1.a = ((t2_1.b + t2_1.a) / 2)))
+                     ->  Seq Scan on public.prt1_m_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c, t2_1.a
+                           ->  Seq Scan on public.prt2_m_p2 t2_1
+                                 Output: t2_1.b, t2_1.c, t2_1.a
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: ((((t1_2.a + t1_2.b) / 2) = t2_2.b) AND (t1_2.a = ((t2_2.b + t2_2.a) / 2)))
+                     ->  Seq Scan on public.prt1_m_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c, t2_2.a
+                           ->  Seq Scan on public.prt2_m_p3 t2_2
+                                 Output: t2_2.b, t2_2.c, t2_2.a
+                                 Filter: ((t2_2.b % 25) = 0)
+(36 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_m t1 RIGHT JOIN uprt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 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
+   Output: prt1_m_p1.a, prt1_m_p1.c, prt2_m_p1.b, prt2_m_p1.c
+   Sort Key: prt1_m_p1.a, prt2_m_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_m_p1.a, prt1_m_p1.c, prt2_m_p1.b, prt2_m_p1.c
+               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 public.prt1_m_p1
+                     Output: prt1_m_p1.a, prt1_m_p1.c, prt1_m_p1.b
+                     Filter: ((prt1_m_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_m_p1.b, prt2_m_p1.c, prt2_m_p1.a
+                     ->  Seq Scan on public.prt2_m_p1
+                           Output: prt2_m_p1.b, prt2_m_p1.c, prt2_m_p1.a
+                           Filter: ((prt2_m_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_m_p2.a, prt1_m_p2.c, prt2_m_p2.b, prt2_m_p2.c
+               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 public.prt1_m_p2
+                     Output: prt1_m_p2.a, prt1_m_p2.c, prt1_m_p2.b
+                     Filter: ((prt1_m_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_m_p2.b, prt2_m_p2.c, prt2_m_p2.a
+                     ->  Seq Scan on public.prt2_m_p2
+                           Output: prt2_m_p2.b, prt2_m_p2.c, prt2_m_p2.a
+                           Filter: ((prt2_m_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_m_p3.a, prt1_m_p3.c, prt2_m_p3.b, prt2_m_p3.c
+               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 public.prt1_m_p3
+                     Output: prt1_m_p3.a, prt1_m_p3.c, prt1_m_p3.b
+                     Filter: ((prt1_m_p3.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_m_p3.b, prt2_m_p3.c, prt2_m_p3.a
+                     ->  Seq Scan on public.prt2_m_p3
+                           Output: prt2_m_p3.b, prt2_m_p3.c, prt2_m_p3.a
+                           Filter: ((prt2_m_p3.b % 25) = 0)
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 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 | 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_m t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_m t2 WHERE t2.b % 25 = 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 | 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)
+
+--
+-- multi-leveled partitions
+--
+CREATE TABLE prt1_l (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_l_p1 PARTITION OF prt1_l FOR VALUES START (0) END (250) PARTITION BY RANGE (b);
+CREATE TABLE prt1_l_p1_p1 PARTITION OF prt1_l_p1 FOR VALUES START (0) END (100);
+CREATE TABLE prt1_l_p1_p2 PARTITION OF prt1_l_p1 FOR VALUES START (100) END (250);
+CREATE TABLE prt1_l_p2 PARTITION OF prt1_l FOR VALUES START (250) END (500) PARTITION BY RANGE (c);
+CREATE TABLE prt1_l_p2_p1 PARTITION OF prt1_l_p2 FOR VALUES START ('0250') END ('0400');
+CREATE TABLE prt1_l_p2_p2 PARTITION OF prt1_l_p2 FOR VALUES START ('0400') END ('0500');
+CREATE TABLE prt1_l_p3 PARTITION OF prt1_l FOR VALUES START (500) END (600) PARTITION BY RANGE ((b + a));
+CREATE TABLE prt1_l_p3_p1 PARTITION OF prt1_l_p3 FOR VALUES START (1000) END (1100);
+CREATE TABLE prt1_l_p3_p2 PARTITION OF prt1_l_p3 FOR VALUES START (1100) END (1200);
+INSERT INTO prt1_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_l;
+ANALYZE prt1_l_p1;
+ANALYZE prt1_l_p1_p1;
+ANALYZE prt1_l_p1_p2;
+ANALYZE prt1_l_p2;
+ANALYZE prt1_l_p2_p1;
+ANALYZE prt1_l_p2_p2;
+ANALYZE prt1_l_p3;
+ANALYZE prt1_l_p3_p1;
+ANALYZE prt1_l_p3_p2;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_l AS SELECT * FROM prt1_l;
+CREATE TABLE prt2_l (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_l_p1 PARTITION OF prt2_l FOR VALUES START (0) END (250) PARTITION BY RANGE (a);
+CREATE TABLE prt2_l_p1_p1 PARTITION OF prt2_l_p1 FOR VALUES START (0) END (100);
+CREATE TABLE prt2_l_p1_p2 PARTITION OF prt2_l_p1 FOR VALUES START (100) END (250);
+CREATE TABLE prt2_l_p2 PARTITION OF prt2_l FOR VALUES START (250) END (500) PARTITION BY RANGE (c);
+CREATE TABLE prt2_l_p2_p1 PARTITION OF prt2_l_p2 FOR VALUES START ('0250') END ('0400');
+CREATE TABLE prt2_l_p2_p2 PARTITION OF prt2_l_p2 FOR VALUES START ('0400') END ('0500');
+CREATE TABLE prt2_l_p3 PARTITION OF prt2_l FOR VALUES START (500) END (600) PARTITION BY RANGE ((a + b));
+CREATE TABLE prt2_l_p3_p1 PARTITION OF prt2_l_p3 FOR VALUES START (1000) END (1100);
+CREATE TABLE prt2_l_p3_p2 PARTITION OF prt2_l_p3 FOR VALUES START (1100) END (1200);
+INSERT INTO prt2_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_l;
+ANALYZE prt2_l_p1;
+ANALYZE prt2_l_p1_p1;
+ANALYZE prt2_l_p1_p2;
+ANALYZE prt2_l_p2;
+ANALYZE prt2_l_p2_p1;
+ANALYZE prt2_l_p2_p2;
+ANALYZE prt2_l_p3;
+ANALYZE prt2_l_p3_p1;
+ANALYZE prt2_l_p3_p2;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_l AS SELECT * FROM prt2_l;
+EXPLAIN (VERBOSE, 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 = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                                                      QUERY PLAN                                                                      
+------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((t2.b = t1.a) AND (t2.a = t1.b) AND ((t2.c)::text = (t1.c)::text) AND ((t2.a + t2.b) = (t1.b + t1.a)))
+               ->  Seq Scan on public.prt2_l_p1_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_l_p1_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.a = t1_1.b) AND ((t2_1.c)::text = (t1_1.c)::text) AND ((t2_1.a + t2_1.b) = (t1_1.b + t1_1.a)))
+               ->  Seq Scan on public.prt2_l_p1_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_l_p1_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.a = t1_2.b) AND ((t2_2.c)::text = (t1_2.c)::text) AND ((t2_2.a + t2_2.b) = (t1_2.b + t1_2.a)))
+               ->  Seq Scan on public.prt2_l_p2_p1 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_l_p2_p1 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_3.a, t1_3.c, t2_3.b, t2_3.c
+               Hash Cond: ((t2_3.b = t1_3.a) AND (t2_3.a = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text) AND ((t2_3.a + t2_3.b) = (t1_3.b + t1_3.a)))
+               ->  Seq Scan on public.prt2_l_p2_p2 t2_3
+                     Output: t2_3.b, t2_3.c, t2_3.a
+               ->  Hash
+                     Output: t1_3.a, t1_3.c, t1_3.b
+                     ->  Seq Scan on public.prt1_l_p2_p2 t1_3
+                           Output: t1_3.a, t1_3.c, t1_3.b
+                           Filter: ((t1_3.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_4.a, t1_4.c, t2_4.b, t2_4.c
+               Hash Cond: ((t2_4.b = t1_4.a) AND (t2_4.a = t1_4.b) AND ((t2_4.c)::text = (t1_4.c)::text) AND ((t2_4.a + t2_4.b) = (t1_4.b + t1_4.a)))
+               ->  Seq Scan on public.prt2_l_p3_p1 t2_4
+                     Output: t2_4.b, t2_4.c, t2_4.a
+               ->  Hash
+                     Output: t1_4.a, t1_4.c, t1_4.b
+                     ->  Seq Scan on public.prt1_l_p3_p1 t1_4
+                           Output: t1_4.a, t1_4.c, t1_4.b
+                           Filter: ((t1_4.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_5.a, t1_5.c, t2_5.b, t2_5.c
+               Hash Cond: ((t2_5.b = t1_5.a) AND (t2_5.a = t1_5.b) AND ((t2_5.c)::text = (t1_5.c)::text) AND ((t2_5.a + t2_5.b) = (t1_5.b + t1_5.a)))
+               ->  Seq Scan on public.prt2_l_p3_p2 t2_5
+                     Output: t2_5.b, t2_5.c, t2_5.a
+               ->  Hash
+                     Output: t1_5.a, t1_5.c, t1_5.b
+                     ->  Seq Scan on public.prt1_l_p3_p2 t1_5
+                           Output: t1_5.a, t1_5.c, t1_5.b
+                           Filter: ((t1_5.a % 25) = 0)
+(64 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1, prt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1, uprt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 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)
+
+EXPLAIN (VERBOSE, 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                                                      QUERY PLAN                                                                      
+------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((t2.b = t1.a) AND (t2.a = t1.b) AND ((t2.c)::text = (t1.c)::text) AND ((t2.a + t2.b) = (t1.b + t1.a)))
+               ->  Seq Scan on public.prt2_l_p1_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_l_p1_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.a = t1_1.b) AND ((t2_1.c)::text = (t1_1.c)::text) AND ((t2_1.a + t2_1.b) = (t1_1.b + t1_1.a)))
+               ->  Seq Scan on public.prt2_l_p1_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_l_p1_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.a = t1_2.b) AND ((t2_2.c)::text = (t1_2.c)::text) AND ((t2_2.a + t2_2.b) = (t1_2.b + t1_2.a)))
+               ->  Seq Scan on public.prt2_l_p2_p1 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_l_p2_p1 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_3.a, t1_3.c, t2_3.b, t2_3.c
+               Hash Cond: ((t2_3.b = t1_3.a) AND (t2_3.a = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text) AND ((t2_3.a + t2_3.b) = (t1_3.b + t1_3.a)))
+               ->  Seq Scan on public.prt2_l_p2_p2 t2_3
+                     Output: t2_3.b, t2_3.c, t2_3.a
+               ->  Hash
+                     Output: t1_3.a, t1_3.c, t1_3.b
+                     ->  Seq Scan on public.prt1_l_p2_p2 t1_3
+                           Output: t1_3.a, t1_3.c, t1_3.b
+                           Filter: ((t1_3.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_4.a, t1_4.c, t2_4.b, t2_4.c
+               Hash Cond: ((t2_4.b = t1_4.a) AND (t2_4.a = t1_4.b) AND ((t2_4.c)::text = (t1_4.c)::text) AND ((t2_4.a + t2_4.b) = (t1_4.b + t1_4.a)))
+               ->  Seq Scan on public.prt2_l_p3_p1 t2_4
+                     Output: t2_4.b, t2_4.c, t2_4.a
+               ->  Hash
+                     Output: t1_4.a, t1_4.c, t1_4.b
+                     ->  Seq Scan on public.prt1_l_p3_p1 t1_4
+                           Output: t1_4.a, t1_4.c, t1_4.b
+                           Filter: ((t1_4.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_5.a, t1_5.c, t2_5.b, t2_5.c
+               Hash Cond: ((t2_5.b = t1_5.a) AND (t2_5.a = t1_5.b) AND ((t2_5.c)::text = (t1_5.c)::text) AND ((t2_5.a + t2_5.b) = (t1_5.b + t1_5.a)))
+               ->  Seq Scan on public.prt2_l_p3_p2 t2_5
+                     Output: t2_5.b, t2_5.c, t2_5.a
+               ->  Hash
+                     Output: t1_5.a, t1_5.c, t1_5.b
+                     ->  Seq Scan on public.prt1_l_p3_p2 t1_5
+                           Output: t1_5.a, t1_5.c, t1_5.b
+                           Filter: ((t1_5.a % 25) = 0)
+(64 rows)
+
+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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 LEFT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+                                                                         QUERY PLAN                                                                         
+------------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: ((t1.a = t2.b) AND (t1.b = t2.a) AND ((t1.c)::text = (t2.c)::text) AND ((t1.b + t1.a) = (t2.a + t2.b)))
+                     ->  Seq Scan on public.prt1_l_p1_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                     ->  Hash
+                           Output: t2.b, t2.c, t2.a
+                           ->  Seq Scan on public.prt2_l_p1_p1 t2
+                                 Output: t2.b, t2.c, t2.a
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: ((t1_1.a = t2_1.b) AND (t1_1.b = t2_1.a) AND ((t1_1.c)::text = (t2_1.c)::text) AND ((t1_1.b + t1_1.a) = (t2_1.a + t2_1.b)))
+                     ->  Seq Scan on public.prt1_l_p1_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c, t2_1.a
+                           ->  Seq Scan on public.prt2_l_p1_p2 t2_1
+                                 Output: t2_1.b, t2_1.c, t2_1.a
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: ((t1_2.a = t2_2.b) AND (t1_2.b = t2_2.a) AND ((t1_2.c)::text = (t2_2.c)::text) AND ((t1_2.b + t1_2.a) = (t2_2.a + t2_2.b)))
+                     ->  Seq Scan on public.prt1_l_p2_p1 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c, t2_2.a
+                           ->  Seq Scan on public.prt2_l_p2_p1 t2_2
+                                 Output: t2_2.b, t2_2.c, t2_2.a
+                                 Filter: ((t2_2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_3.b, t2_3.c, t1_3.a, t1_3.c
+                     Hash Cond: ((t1_3.a = t2_3.b) AND (t1_3.b = t2_3.a) AND ((t1_3.c)::text = (t2_3.c)::text) AND ((t1_3.b + t1_3.a) = (t2_3.a + t2_3.b)))
+                     ->  Seq Scan on public.prt1_l_p2_p2 t1_3
+                           Output: t1_3.a, t1_3.c, t1_3.b
+                     ->  Hash
+                           Output: t2_3.b, t2_3.c, t2_3.a
+                           ->  Seq Scan on public.prt2_l_p2_p2 t2_3
+                                 Output: t2_3.b, t2_3.c, t2_3.a
+                                 Filter: ((t2_3.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_4.b, t2_4.c, t1_4.a, t1_4.c
+                     Hash Cond: ((t1_4.a = t2_4.b) AND (t1_4.b = t2_4.a) AND ((t1_4.c)::text = (t2_4.c)::text) AND ((t1_4.b + t1_4.a) = (t2_4.a + t2_4.b)))
+                     ->  Seq Scan on public.prt1_l_p3_p1 t1_4
+                           Output: t1_4.a, t1_4.c, t1_4.b
+                     ->  Hash
+                           Output: t2_4.b, t2_4.c, t2_4.a
+                           ->  Seq Scan on public.prt2_l_p3_p1 t2_4
+                                 Output: t2_4.b, t2_4.c, t2_4.a
+                                 Filter: ((t2_4.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_5.b, t2_5.c, t1_5.a, t1_5.c
+                     Hash Cond: ((t1_5.a = t2_5.b) AND (t1_5.b = t2_5.a) AND ((t1_5.c)::text = (t2_5.c)::text) AND ((t1_5.b + t1_5.a) = (t2_5.a + t2_5.b)))
+                     ->  Seq Scan on public.prt1_l_p3_p2 t1_5
+                           Output: t1_5.a, t1_5.c, t1_5.b
+                     ->  Hash
+                           Output: t2_5.b, t2_5.c, t2_5.a
+                           ->  Seq Scan on public.prt2_l_p3_p2 t2_5
+                                 Output: t2_5.b, t2_5.c, t2_5.a
+                                 Filter: ((t2_5.b % 25) = 0)
+(66 rows)
+
+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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 RIGHT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+                                                                                                              QUERY PLAN                                                                                                              
+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_l_p1_p1.a, prt1_l_p1_p1.c, prt2_l_p1_p1.b, prt2_l_p1_p1.c
+   Sort Key: prt1_l_p1_p1.a, prt2_l_p1_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_l_p1_p1.a, prt1_l_p1_p1.c, prt2_l_p1_p1.b, prt2_l_p1_p1.c
+               Hash Cond: ((prt1_l_p1_p1.a = prt2_l_p1_p1.b) AND (prt1_l_p1_p1.b = prt2_l_p1_p1.a) AND ((prt1_l_p1_p1.c)::text = (prt2_l_p1_p1.c)::text) AND ((prt1_l_p1_p1.b + prt1_l_p1_p1.a) = (prt2_l_p1_p1.a + prt2_l_p1_p1.b)))
+               ->  Seq Scan on public.prt1_l_p1_p1
+                     Output: prt1_l_p1_p1.a, prt1_l_p1_p1.c, prt1_l_p1_p1.b
+                     Filter: ((prt1_l_p1_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p1_p1.b, prt2_l_p1_p1.c, prt2_l_p1_p1.a
+                     ->  Seq Scan on public.prt2_l_p1_p1
+                           Output: prt2_l_p1_p1.b, prt2_l_p1_p1.c, prt2_l_p1_p1.a
+                           Filter: ((prt2_l_p1_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_l_p1_p2.a, prt1_l_p1_p2.c, prt2_l_p1_p2.b, prt2_l_p1_p2.c
+               Hash Cond: ((prt1_l_p1_p2.a = prt2_l_p1_p2.b) AND (prt1_l_p1_p2.b = prt2_l_p1_p2.a) AND ((prt1_l_p1_p2.c)::text = (prt2_l_p1_p2.c)::text) AND ((prt1_l_p1_p2.b + prt1_l_p1_p2.a) = (prt2_l_p1_p2.a + prt2_l_p1_p2.b)))
+               ->  Seq Scan on public.prt1_l_p1_p2
+                     Output: prt1_l_p1_p2.a, prt1_l_p1_p2.c, prt1_l_p1_p2.b
+                     Filter: ((prt1_l_p1_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p1_p2.b, prt2_l_p1_p2.c, prt2_l_p1_p2.a
+                     ->  Seq Scan on public.prt2_l_p1_p2
+                           Output: prt2_l_p1_p2.b, prt2_l_p1_p2.c, prt2_l_p1_p2.a
+                           Filter: ((prt2_l_p1_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_l_p2_p1.a, prt1_l_p2_p1.c, prt2_l_p2_p1.b, prt2_l_p2_p1.c
+               Hash Cond: ((prt1_l_p2_p1.a = prt2_l_p2_p1.b) AND (prt1_l_p2_p1.b = prt2_l_p2_p1.a) AND ((prt1_l_p2_p1.c)::text = (prt2_l_p2_p1.c)::text) AND ((prt1_l_p2_p1.b + prt1_l_p2_p1.a) = (prt2_l_p2_p1.a + prt2_l_p2_p1.b)))
+               ->  Seq Scan on public.prt1_l_p2_p1
+                     Output: prt1_l_p2_p1.a, prt1_l_p2_p1.c, prt1_l_p2_p1.b
+                     Filter: ((prt1_l_p2_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p2_p1.b, prt2_l_p2_p1.c, prt2_l_p2_p1.a
+                     ->  Seq Scan on public.prt2_l_p2_p1
+                           Output: prt2_l_p2_p1.b, prt2_l_p2_p1.c, prt2_l_p2_p1.a
+                           Filter: ((prt2_l_p2_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_l_p2_p2.a, prt1_l_p2_p2.c, prt2_l_p2_p2.b, prt2_l_p2_p2.c
+               Hash Cond: ((prt1_l_p2_p2.a = prt2_l_p2_p2.b) AND (prt1_l_p2_p2.b = prt2_l_p2_p2.a) AND ((prt1_l_p2_p2.c)::text = (prt2_l_p2_p2.c)::text) AND ((prt1_l_p2_p2.b + prt1_l_p2_p2.a) = (prt2_l_p2_p2.a + prt2_l_p2_p2.b)))
+               ->  Seq Scan on public.prt1_l_p2_p2
+                     Output: prt1_l_p2_p2.a, prt1_l_p2_p2.c, prt1_l_p2_p2.b
+                     Filter: ((prt1_l_p2_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p2_p2.b, prt2_l_p2_p2.c, prt2_l_p2_p2.a
+                     ->  Seq Scan on public.prt2_l_p2_p2
+                           Output: prt2_l_p2_p2.b, prt2_l_p2_p2.c, prt2_l_p2_p2.a
+                           Filter: ((prt2_l_p2_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_l_p3_p1.a, prt1_l_p3_p1.c, prt2_l_p3_p1.b, prt2_l_p3_p1.c
+               Hash Cond: ((prt1_l_p3_p1.a = prt2_l_p3_p1.b) AND (prt1_l_p3_p1.b = prt2_l_p3_p1.a) AND ((prt1_l_p3_p1.c)::text = (prt2_l_p3_p1.c)::text) AND ((prt1_l_p3_p1.b + prt1_l_p3_p1.a) = (prt2_l_p3_p1.a + prt2_l_p3_p1.b)))
+               ->  Seq Scan on public.prt1_l_p3_p1
+                     Output: prt1_l_p3_p1.a, prt1_l_p3_p1.c, prt1_l_p3_p1.b
+                     Filter: ((prt1_l_p3_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p3_p1.b, prt2_l_p3_p1.c, prt2_l_p3_p1.a
+                     ->  Seq Scan on public.prt2_l_p3_p1
+                           Output: prt2_l_p3_p1.b, prt2_l_p3_p1.c, prt2_l_p3_p1.a
+                           Filter: ((prt2_l_p3_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_l_p3_p2.a, prt1_l_p3_p2.c, prt2_l_p3_p2.b, prt2_l_p3_p2.c
+               Hash Cond: ((prt1_l_p3_p2.a = prt2_l_p3_p2.b) AND (prt1_l_p3_p2.b = prt2_l_p3_p2.a) AND ((prt1_l_p3_p2.c)::text = (prt2_l_p3_p2.c)::text) AND ((prt1_l_p3_p2.b + prt1_l_p3_p2.a) = (prt2_l_p3_p2.a + prt2_l_p3_p2.b)))
+               ->  Seq Scan on public.prt1_l_p3_p2
+                     Output: prt1_l_p3_p2.a, prt1_l_p3_p2.c, prt1_l_p3_p2.b
+                     Filter: ((prt1_l_p3_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p3_p2.b, prt2_l_p3_p2.c, prt2_l_p3_p2.a
+                     ->  Seq Scan on public.prt2_l_p3_p2
+                           Output: prt2_l_p3_p2.b, prt2_l_p3_p2.c, prt2_l_p3_p2.a
+                           Filter: ((prt2_l_p3_p2.b % 25) = 0)
+(70 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_l t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_l t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.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)
+
+--
+-- 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;
+ANALYZE plt1_p1;
+ANALYZE plt1_p2;
+ANALYZE plt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1 AS SELECT * FROM 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;
+ANALYZE plt2_p1;
+ANALYZE plt2_p2;
+ANALYZE plt2_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt2 AS SELECT * FROM plt2;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.a = t2.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((t2.c = t1.c) AND (t2.a = t1.a))
+               ->  Seq Scan on public.plt2_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.plt1_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((t2_1.c = t1_1.c) AND (t2_1.a = t1_1.a))
+               ->  Seq Scan on public.plt2_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.plt1_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((t2_2.c = t1_2.c) AND (t2_2.a = t1_2.a))
+               ->  Seq Scan on public.plt2_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.plt1_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.a = t2.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0003 | 150 | 0003
+ 300 | 0006 | 300 | 0006
+ 450 | 0009 | 450 | 0009
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1 t1, uplt2 t2 WHERE t1.c = t2.c AND t1.a = t2.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0003 | 150 | 0003
+ 300 | 0006 | 300 | 0006
+ 450 | 0009 | 450 | 0009
+(4 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.a AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+               ->  Seq Scan on public.plt2_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.plt1_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on public.plt2_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.plt1_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on public.plt2_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.plt1_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.a AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  50 | 0001 |     | 
+ 100 | 0002 |     | 
+ 150 | 0003 | 150 | 0003
+ 200 | 0004 |     | 
+ 250 | 0005 |     | 
+ 300 | 0006 | 300 | 0006
+ 350 | 0007 |     | 
+ 400 | 0008 |     | 
+ 450 | 0009 | 450 | 0009
+ 500 | 0010 |     | 
+ 550 | 0011 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.a AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  50 | 0001 |     | 
+ 100 | 0002 |     | 
+ 150 | 0003 | 150 | 0003
+ 200 | 0004 |     | 
+ 250 | 0005 |     | 
+ 300 | 0006 | 300 | 0006
+ 350 | 0007 |     | 
+ 400 | 0008 |     | 
+ 450 | 0009 | 450 | 0009
+ 500 | 0010 |     | 
+ 550 | 0011 |     | 
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+                                QUERY PLAN                                
+--------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: ((t1.a = t2.b) AND (t1.c = t2.c))
+                     ->  Seq Scan on public.plt1_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t2.b, t2.c
+                           ->  Seq Scan on public.plt2_p1 t2
+                                 Output: t2.b, t2.c
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: ((t1_1.a = t2_1.b) AND (t1_1.c = t2_1.c))
+                     ->  Seq Scan on public.plt1_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c
+                           ->  Seq Scan on public.plt2_p2 t2_1
+                                 Output: t2_1.b, t2_1.c
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: ((t1_2.a = t2_2.b) AND (t1_2.c = t2_2.c))
+                     ->  Seq Scan on public.plt1_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c
+                           ->  Seq Scan on public.plt2_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+                                 Filter: ((t2_2.b % 25) = 0)
+(36 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0003 | 150 | 0003
+ 300 | 0006 | 300 | 0006
+ 450 | 0009 | 450 | 0009
+     |      |  75 | 0001
+     |      | 225 | 0004
+     |      | 375 | 0007
+     |      | 525 | 0010
+(8 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0003 | 150 | 0003
+ 300 | 0006 | 300 | 0006
+ 450 | 0009 | 450 | 0009
+     |      |  75 | 0001
+     |      | 225 | 0004
+     |      | 375 | 0007
+     |      | 525 | 0010
+(8 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) ORDER BY t1.a, t2.b;
+                                   QUERY PLAN                                   
+--------------------------------------------------------------------------------
+ Sort
+   Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+   Sort Key: plt1_p1.a, plt2_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+               Hash Cond: ((plt1_p1.a = plt2_p1.b) AND (plt1_p1.c = plt2_p1.c))
+               ->  Seq Scan on public.plt1_p1
+                     Output: plt1_p1.a, plt1_p1.c
+                     Filter: ((plt1_p1.a % 25) = 0)
+               ->  Hash
+                     Output: plt2_p1.b, plt2_p1.c
+                     ->  Seq Scan on public.plt2_p1
+                           Output: plt2_p1.b, plt2_p1.c
+                           Filter: ((plt2_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+               Hash Cond: ((plt1_p2.a = plt2_p2.b) AND (plt1_p2.c = plt2_p2.c))
+               ->  Seq Scan on public.plt1_p2
+                     Output: plt1_p2.a, plt1_p2.c
+                     Filter: ((plt1_p2.a % 25) = 0)
+               ->  Hash
+                     Output: plt2_p2.b, plt2_p2.c
+                     ->  Seq Scan on public.plt2_p2
+                           Output: plt2_p2.b, plt2_p2.c
+                           Filter: ((plt2_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+               Hash Cond: ((plt1_p3.a = plt2_p3.b) AND (plt1_p3.c = plt2_p3.c))
+               ->  Seq Scan on public.plt1_p3
+                     Output: plt1_p3.a, plt1_p3.c
+                     Filter: ((plt1_p3.a % 25) = 0)
+               ->  Hash
+                     Output: plt2_p3.b, plt2_p3.c
+                     ->  Seq Scan on public.plt2_p3
+                           Output: plt2_p3.b, plt2_p3.c
+                           Filter: ((plt2_p3.b % 25) = 0)
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  50 | 0001 |     | 
+ 100 | 0002 |     | 
+ 150 | 0003 | 150 | 0003
+ 200 | 0004 |     | 
+ 250 | 0005 |     | 
+ 300 | 0006 | 300 | 0006
+ 350 | 0007 |     | 
+ 400 | 0008 |     | 
+ 450 | 0009 | 450 | 0009
+ 500 | 0010 |     | 
+ 550 | 0011 |     | 
+     |      |  75 | 0001
+     |      | 225 | 0004
+     |      | 375 | 0007
+     |      | 525 | 0010
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uplt1 t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  50 | 0001 |     | 
+ 100 | 0002 |     | 
+ 150 | 0003 | 150 | 0003
+ 200 | 0004 |     | 
+ 250 | 0005 |     | 
+ 300 | 0006 | 300 | 0006
+ 350 | 0007 |     | 
+ 400 | 0008 |     | 
+ 450 | 0009 | 450 | 0009
+ 500 | 0010 |     | 
+ 550 | 0011 |     | 
+     |      |  75 | 0001
+     |      | 225 | 0004
+     |      | 375 | 0007
+     |      | 525 | 0010
+(16 rows)
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM (SELECT 50 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ GroupAggregate
+   Output: sum(plt1_p1.a), plt1_p1.c, avg(plt2_p1.b), plt2_p1.c
+   Group Key: plt1_p1.c, plt2_p1.c
+   ->  Sort
+         Output: plt1_p1.c, plt2_p1.c, plt1_p1.a, plt2_p1.b
+         Sort Key: plt1_p1.c, plt2_p1.c
+         ->  Result
+               Output: plt1_p1.c, plt2_p1.c, plt1_p1.a, plt2_p1.b
+               ->  Append
+                     ->  Hash Full Join
+                           Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+                           Hash Cond: ((plt1_p1.c = plt2_p1.c) AND (plt1_p1.a = plt2_p1.b))
+                           ->  Seq Scan on public.plt1_p1
+                                 Output: plt1_p1.a, plt1_p1.c
+                                 Filter: ((plt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p1.b, plt2_p1.c
+                                 ->  Seq Scan on public.plt2_p1
+                                       Output: plt2_p1.b, plt2_p1.c
+                                       Filter: ((plt2_p1.b % 25) = 0)
+                     ->  Hash Full Join
+                           Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+                           Hash Cond: ((plt1_p2.c = plt2_p2.c) AND (plt1_p2.a = plt2_p2.b))
+                           ->  Seq Scan on public.plt1_p2
+                                 Output: plt1_p2.a, plt1_p2.c
+                                 Filter: ((plt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p2.b, plt2_p2.c
+                                 ->  Seq Scan on public.plt2_p2
+                                       Output: plt2_p2.b, plt2_p2.c
+                                       Filter: ((plt2_p2.b % 25) = 0)
+                     ->  Hash Full Join
+                           Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+                           Hash Cond: ((plt1_p3.c = plt2_p3.c) AND (plt1_p3.a = plt2_p3.b))
+                           ->  Seq Scan on public.plt1_p3
+                                 Output: plt1_p3.a, plt1_p3.c
+                                 Filter: ((plt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p3.b, plt2_p3.c
+                                 ->  Seq Scan on public.plt2_p3
+                                       Output: plt2_p3.b, plt2_p3.c
+                                       Filter: ((plt2_p3.b % 25) = 0)
+(42 rows)
+
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM (SELECT 50 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+ sum |  c   |          avg           |  c   
+-----+------+------------------------+------
+   0 | 0000 | 0.00000000000000000000 | 0000
+  50 | 0001 |                        | 
+ 100 | 0002 |                        | 
+ 150 | 0003 |   150.0000000000000000 | 0003
+ 200 | 0004 |                        | 
+ 250 | 0005 |                        | 
+ 300 | 0006 |   300.0000000000000000 | 0006
+ 350 | 0007 |                        | 
+ 400 | 0008 |                        | 
+ 450 | 0009 |   450.0000000000000000 | 0009
+ 500 | 0010 |                        | 
+ 550 | 0011 |                        | 
+     |      |    75.0000000000000000 | 0001
+     |      |   225.0000000000000000 | 0004
+     |      |   375.0000000000000000 | 0007
+     |      |   525.0000000000000000 | 0010
+(16 rows)
+
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM (SELECT 50 phv, * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+ sum |  c   |          avg           |  c   
+-----+------+------------------------+------
+   0 | 0000 | 0.00000000000000000000 | 0000
+  50 | 0001 |                        | 
+ 100 | 0002 |                        | 
+ 150 | 0003 |   150.0000000000000000 | 0003
+ 200 | 0004 |                        | 
+ 250 | 0005 |                        | 
+ 300 | 0006 |   300.0000000000000000 | 0006
+ 350 | 0007 |                        | 
+ 400 | 0008 |                        | 
+ 450 | 0009 |   450.0000000000000000 | 0009
+ 500 | 0010 |                        | 
+ 550 | 0011 |                        | 
+     |      |    75.0000000000000000 | 0001
+     |      |   225.0000000000000000 | 0004
+     |      |   375.0000000000000000 | 0007
+     |      |   525.0000000000000000 | 0010
+(16 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t1.phv), avg(t2.b), t2.c, avg(t2.phv) FROM (SELECT 25 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ GroupAggregate
+   Output: sum(plt1_p1.a), plt1_p1.c, sum((25)), avg(plt2_p1.b), plt2_p1.c, avg((50))
+   Group Key: plt1_p1.c, plt2_p1.c
+   ->  Sort
+         Output: plt1_p1.c, plt2_p1.c, plt1_p1.a, (25), plt2_p1.b, (50)
+         Sort Key: plt1_p1.c, plt2_p1.c
+         ->  Result
+               Output: plt1_p1.c, plt2_p1.c, plt1_p1.a, (25), plt2_p1.b, (50)
+               ->  Append
+                     ->  Hash Full Join
+                           Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, (25), (50)
+                           Hash Cond: ((plt1_p1.c = plt2_p1.c) AND (plt1_p1.a = plt2_p1.b))
+                           ->  Seq Scan on public.plt1_p1
+                                 Output: plt1_p1.a, plt1_p1.c, 25
+                                 Filter: ((plt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p1.b, plt2_p1.c, (50)
+                                 ->  Seq Scan on public.plt2_p1
+                                       Output: plt2_p1.b, plt2_p1.c, 50
+                                       Filter: ((plt2_p1.b % 25) = 0)
+                     ->  Hash Full Join
+                           Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c, (25), (50)
+                           Hash Cond: ((plt1_p2.c = plt2_p2.c) AND (plt1_p2.a = plt2_p2.b))
+                           ->  Seq Scan on public.plt1_p2
+                                 Output: plt1_p2.a, plt1_p2.c, 25
+                                 Filter: ((plt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p2.b, plt2_p2.c, (50)
+                                 ->  Seq Scan on public.plt2_p2
+                                       Output: plt2_p2.b, plt2_p2.c, 50
+                                       Filter: ((plt2_p2.b % 25) = 0)
+                     ->  Hash Full Join
+                           Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c, (25), (50)
+                           Hash Cond: ((plt1_p3.c = plt2_p3.c) AND (plt1_p3.a = plt2_p3.b))
+                           ->  Seq Scan on public.plt1_p3
+                                 Output: plt1_p3.a, plt1_p3.c, 25
+                                 Filter: ((plt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p3.b, plt2_p3.c, (50)
+                                 ->  Seq Scan on public.plt2_p3
+                                       Output: plt2_p3.b, plt2_p3.c, 50
+                                       Filter: ((plt2_p3.b % 25) = 0)
+(42 rows)
+
+SELECT sum(t1.a), t1.c, sum(t1.phv), avg(t2.b), t2.c, avg(t2.phv) FROM (SELECT 25 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+ sum |  c   | sum |          avg           |  c   |         avg         
+-----+------+-----+------------------------+------+---------------------
+   0 | 0000 |  25 | 0.00000000000000000000 | 0000 | 50.0000000000000000
+  50 | 0001 |  25 |                        |      |                    
+ 100 | 0002 |  25 |                        |      |                    
+ 150 | 0003 |  25 |   150.0000000000000000 | 0003 | 50.0000000000000000
+ 200 | 0004 |  25 |                        |      |                    
+ 250 | 0005 |  25 |                        |      |                    
+ 300 | 0006 |  25 |   300.0000000000000000 | 0006 | 50.0000000000000000
+ 350 | 0007 |  25 |                        |      |                    
+ 400 | 0008 |  25 |                        |      |                    
+ 450 | 0009 |  25 |   450.0000000000000000 | 0009 | 50.0000000000000000
+ 500 | 0010 |  25 |                        |      |                    
+ 550 | 0011 |  25 |                        |      |                    
+     |      |     |    75.0000000000000000 | 0001 | 50.0000000000000000
+     |      |     |   225.0000000000000000 | 0004 | 50.0000000000000000
+     |      |     |   375.0000000000000000 | 0007 | 50.0000000000000000
+     |      |     |   525.0000000000000000 | 0010 | 50.0000000000000000
+(16 rows)
+
+SELECT sum(t1.a), t1.c, sum(t1.phv), avg(t2.b), t2.c, avg(t2.phv) FROM (SELECT 25 phv, * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+ sum |  c   | sum |          avg           |  c   |         avg         
+-----+------+-----+------------------------+------+---------------------
+   0 | 0000 |  25 | 0.00000000000000000000 | 0000 | 50.0000000000000000
+  50 | 0001 |  25 |                        |      |                    
+ 100 | 0002 |  25 |                        |      |                    
+ 150 | 0003 |  25 |   150.0000000000000000 | 0003 | 50.0000000000000000
+ 200 | 0004 |  25 |                        |      |                    
+ 250 | 0005 |  25 |                        |      |                    
+ 300 | 0006 |  25 |   300.0000000000000000 | 0006 | 50.0000000000000000
+ 350 | 0007 |  25 |                        |      |                    
+ 400 | 0008 |  25 |                        |      |                    
+ 450 | 0009 |  25 |   450.0000000000000000 | 0009 | 50.0000000000000000
+ 500 | 0010 |  25 |                        |      |                    
+ 550 | 0011 |  25 |                        |      |                    
+     |      |     |    75.0000000000000000 | 0001 | 50.0000000000000000
+     |      |     |   225.0000000000000000 | 0004 | 50.0000000000000000
+     |      |     |   375.0000000000000000 | 0007 | 50.0000000000000000
+     |      |     |   525.0000000000000000 | 0010 | 50.0000000000000000
+(16 rows)
+
+-- Join with pruned partitions from joining relations
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.c NOT IN ('0001', '0005', '0002', '0009') AND t2.c NOT IN ('0000', '0003', '0004', '0010') GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Output: (sum(t1.a)), t1.c, (avg(t2.b)), t2.c
+   Sort Key: t1.c
+   ->  HashAggregate
+         Output: sum(t1.a), t1.c, avg(t2.b), t2.c
+         Group Key: t1.c, t2.c
+         ->  Result
+               Output: t1.c, t2.c, t1.a, t2.b
+               ->  Append
+                     ->  Hash Join
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           Hash Cond: (t1.c = t2.c)
+                           ->  Seq Scan on public.plt1_p3 t1
+                                 Output: t1.a, t1.c
+                                 Filter: (t1.c <> ALL ('{0001,0005,0002,0009}'::text[]))
+                           ->  Hash
+                                 Output: t2.b, t2.c
+                                 ->  Seq Scan on public.plt2_p3 t2
+                                       Output: t2.b, t2.c
+                                       Filter: (t2.c <> ALL ('{0000,0003,0004,0010}'::text[]))
+(20 rows)
+
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.c NOT IN ('0001', '0005', '0002', '0009') AND t2.c NOT IN ('0000', '0003', '0004', '0010') GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+  sum   |  c   |         avg          |  c   
+--------+------+----------------------+------
+ 137700 | 0006 | 324.0000000000000000 | 0006
+ 158950 | 0007 | 375.0000000000000000 | 0007
+ 169600 | 0008 | 424.5000000000000000 | 0008
+ 229600 | 0011 | 574.5000000000000000 | 0011
+(4 rows)
+
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM uplt1 t1, uplt2 t2 WHERE t1.c = t2.c AND t1.c NOT IN ('0001', '0005', '0002', '0009') AND t2.c NOT IN ('0000', '0003', '0004', '0010') GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+  sum   |  c   |         avg          |  c   
+--------+------+----------------------+------
+ 137700 | 0006 | 324.0000000000000000 | 0006
+ 158950 | 0007 | 375.0000000000000000 | 0007
+ 169600 | 0008 | 424.5000000000000000 | 0008
+ 229600 | 0011 | 574.5000000000000000 | 0011
+(4 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 LEFT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Output: (sum(t1.a)), t1.c, (sum(b)), c
+   Sort Key: t1.c, c
+   ->  HashAggregate
+         Output: sum(t1.a), t1.c, sum(b), c
+         Group Key: t1.c, c
+         ->  Result
+               Output: t1.c, c, t1.a, b
+               ->  Append
+                     ->  Hash Left Join
+                           Output: t1.a, t1.c, b, c
+                           Hash Cond: (t1.c = c)
+                           ->  Seq Scan on public.plt1_p1 t1
+                                 Output: t1.a, t1.c
+                                 Filter: (t1.c <> ALL ('{0001,0005,0002,0009}'::text[]))
+                           ->  Hash
+                                 Output: b, c
+                                 ->  Result
+                                       Output: b, c
+                                       One-Time Filter: false
+                     ->  Hash Left Join
+                           Output: t1_1.a, t1_1.c, t2.b, t2.c
+                           Hash Cond: (t1_1.c = t2.c)
+                           ->  Seq Scan on public.plt1_p3 t1_1
+                                 Output: t1_1.a, t1_1.c
+                                 Filter: (t1_1.c <> ALL ('{0001,0005,0002,0009}'::text[]))
+                           ->  Hash
+                                 Output: t2.b, t2.c
+                                 ->  Seq Scan on public.plt2_p3 t2
+                                       Output: t2.b, t2.c
+                                       Filter: (t2.c <> ALL ('{0000,0003,0004,0010}'::text[]))
+(31 rows)
+
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 LEFT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+  sum   |  c   |  sum   |  c   
+--------+------+--------+------
+    600 | 0000 |        | 
+   4350 | 0003 |        | 
+   5600 | 0004 |        | 
+ 137700 | 0006 | 137700 | 0006
+ 158950 | 0007 | 159375 | 0007
+ 169600 | 0008 | 169800 | 0008
+  13100 | 0010 |        | 
+ 229600 | 0011 | 229800 | 0011
+(8 rows)
+
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM uplt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 LEFT JOIN (SELECT * FROM uplt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+  sum   |  c   |  sum   |  c   
+--------+------+--------+------
+    600 | 0000 |        | 
+   4350 | 0003 |        | 
+   5600 | 0004 |        | 
+ 137700 | 0006 | 137700 | 0006
+ 158950 | 0007 | 159375 | 0007
+ 169600 | 0008 | 169800 | 0008
+  13100 | 0010 |        | 
+ 229600 | 0011 | 229800 | 0011
+(8 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 RIGHT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+                                           QUERY PLAN                                            
+-------------------------------------------------------------------------------------------------
+ Sort
+   Output: (sum(a)), c, (sum(t2.b)), t2.c
+   Sort Key: c, t2.c
+   ->  HashAggregate
+         Output: sum(a), c, sum(t2.b), t2.c
+         Group Key: c, t2.c
+         ->  Result
+               Output: c, t2.c, a, t2.b
+               ->  Append
+                     ->  Hash Left Join
+                           Output: t2.b, t2.c, a, c
+                           Hash Cond: (t2.c = c)
+                           ->  Seq Scan on public.plt2_p2 t2
+                                 Output: t2.b, t2.c
+                                 Filter: (t2.c <> ALL ('{0000,0003,0004,0010}'::text[]))
+                           ->  Hash
+                                 Output: a, c
+                                 ->  Result
+                                       Output: a, c
+                                       One-Time Filter: false
+                     ->  Hash Right Join
+                           Output: t2_1.b, t2_1.c, t1.a, t1.c
+                           Hash Cond: (t1.c = t2_1.c)
+                           ->  Seq Scan on public.plt1_p3 t1
+                                 Output: t1.a, t1.c
+                                 Filter: (t1.c <> ALL ('{0001,0005,0002,0009}'::text[]))
+                           ->  Hash
+                                 Output: t2_1.b, t2_1.c
+                                 ->  Seq Scan on public.plt2_p3 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                       Filter: (t2_1.c <> ALL ('{0000,0003,0004,0010}'::text[]))
+(31 rows)
+
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 RIGHT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+  sum   |  c   |  sum   |  c   
+--------+------+--------+------
+ 137700 | 0006 | 137700 | 0006
+ 158950 | 0007 | 159375 | 0007
+ 169600 | 0008 | 169800 | 0008
+ 229600 | 0011 | 229800 | 0011
+        |      |   1275 | 0001
+        |      |   1992 | 0002
+        |      |   4392 | 0005
+        |      |   8058 | 0009
+(8 rows)
+
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM uplt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 RIGHT JOIN (SELECT * FROM uplt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+  sum   |  c   |  sum   |  c   
+--------+------+--------+------
+ 137700 | 0006 | 137700 | 0006
+ 158950 | 0007 | 159375 | 0007
+ 169600 | 0008 | 169800 | 0008
+ 229600 | 0011 | 229800 | 0011
+        |      |   1275 | 0001
+        |      |   1992 | 0002
+        |      |   4392 | 0005
+        |      |   8058 | 0009
+(8 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 FULL JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+                                           QUERY PLAN                                            
+-------------------------------------------------------------------------------------------------
+ Sort
+   Output: (sum(t1.a)), t1.c, (sum(b)), c
+   Sort Key: t1.c, c
+   ->  HashAggregate
+         Output: sum(t1.a), t1.c, sum(b), c
+         Group Key: t1.c, c
+         ->  Result
+               Output: t1.c, c, t1.a, b
+               ->  Append
+                     ->  Hash Full Join
+                           Output: t1.a, t1.c, b, c
+                           Hash Cond: (t1.c = c)
+                           ->  Seq Scan on public.plt1_p1 t1
+                                 Output: t1.a, t1.c
+                                 Filter: (t1.c <> ALL ('{0001,0005,0002,0009}'::text[]))
+                           ->  Hash
+                                 Output: b, c
+                                 ->  Result
+                                       Output: b, c
+                                       One-Time Filter: false
+                     ->  Hash Full Join
+                           Output: a, c, t2.b, t2.c
+                           Hash Cond: (t2.c = c)
+                           ->  Seq Scan on public.plt2_p2 t2
+                                 Output: t2.b, t2.c
+                                 Filter: (t2.c <> ALL ('{0000,0003,0004,0010}'::text[]))
+                           ->  Hash
+                                 Output: a, c
+                                 ->  Result
+                                       Output: a, c
+                                       One-Time Filter: false
+                     ->  Hash Full Join
+                           Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                           Hash Cond: (t1_1.c = t2_1.c)
+                           ->  Seq Scan on public.plt1_p3 t1_1
+                                 Output: t1_1.a, t1_1.c
+                                 Filter: (t1_1.c <> ALL ('{0001,0005,0002,0009}'::text[]))
+                           ->  Hash
+                                 Output: t2_1.b, t2_1.c
+                                 ->  Seq Scan on public.plt2_p3 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                       Filter: (t2_1.c <> ALL ('{0000,0003,0004,0010}'::text[]))
+(42 rows)
+
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 FULL JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+  sum   |  c   |  sum   |  c   
+--------+------+--------+------
+    600 | 0000 |        | 
+   4350 | 0003 |        | 
+   5600 | 0004 |        | 
+ 137700 | 0006 | 137700 | 0006
+ 158950 | 0007 | 159375 | 0007
+ 169600 | 0008 | 169800 | 0008
+  13100 | 0010 |        | 
+ 229600 | 0011 | 229800 | 0011
+        |      |   1275 | 0001
+        |      |   1992 | 0002
+        |      |   4392 | 0005
+        |      |   8058 | 0009
+(12 rows)
+
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM uplt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 FULL JOIN (SELECT * FROM uplt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+  sum   |  c   |  sum   |  c   
+--------+------+--------+------
+    600 | 0000 |        | 
+   4350 | 0003 |        | 
+   5600 | 0004 |        | 
+ 137700 | 0006 | 137700 | 0006
+ 158950 | 0007 | 159375 | 0007
+ 169600 | 0008 | 169800 | 0008
+  13100 | 0010 |        | 
+ 229600 | 0011 | 229800 | 0011
+        |      |   1275 | 0001
+        |      |   1992 | 0002
+        |      |   4392 | 0005
+        |      |   8058 | 0009
+(12 rows)
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.c = t1_3.c)
+               ->  Seq Scan on public.plt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_3.c
+                     ->  Seq Scan on public.plt2_p1 t1_3
+                           Output: t1_3.c
+                           Filter: ((t1_3.b % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.c = t1_4.c)
+               ->  Seq Scan on public.plt1_p2 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_4.c
+                     ->  Seq Scan on public.plt2_p2 t1_4
+                           Output: t1_4.c
+                           Filter: ((t1_4.b % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.c = t1_5.c)
+               ->  Seq Scan on public.plt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_5.c
+                     ->  Seq Scan on public.plt2_p3 t1_5
+                           Output: t1_5.c
+                           Filter: ((t1_5.b % 25) = 0)
+(37 rows)
+
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+(8 rows)
+
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+(8 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;
+ANALYZE plt1_e;
+ANALYZE plt1_e_p1;
+ANALYZE plt1_e_p2;
+ANALYZE plt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1_e AS SELECT * FROM plt1_e;
+CREATE TABLE plt2_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
+CREATE TABLE plt2_e_p1 PARTITION OF plt2_e FOR VALUES IN ('0000', '0003', '0004', '0010');
+CREATE TABLE plt2_e_p2 PARTITION OF plt2_e FOR VALUES IN ('0001', '0005', '0002', '0009');
+CREATE TABLE plt2_e_p3 PARTITION OF plt2_e FOR VALUES IN ('0006', '0007', '0008', '0011');
+INSERT INTO plt2_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE plt2_e;
+ANALYZE plt2_e_p1;
+ANALYZE plt2_e_p2;
+ANALYZE plt2_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt2_e AS SELECT * FROM plt2_e;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1, plt2_e t2 WHERE t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((t2.b = t1.a) AND (ltrim(t2.c, 'A'::text) = ltrim(t1.c, 'A'::text)))
+               ->  Seq Scan on public.plt2_e_p1 t2
+                     Output: t2.b, t2.c
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.plt1_e_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((t2_1.b = t1_1.a) AND (ltrim(t2_1.c, 'A'::text) = ltrim(t1_1.c, 'A'::text)))
+               ->  Seq Scan on public.plt2_e_p2 t2_1
+                     Output: t2_1.b, t2_1.c
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.plt1_e_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((t2_2.b = t1_2.a) AND (ltrim(t2_2.c, 'A'::text) = ltrim(t1_2.c, 'A'::text)))
+               ->  Seq Scan on public.plt2_e_p3 t2_2
+                     Output: t2_2.b, t2_2.c
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.plt1_e_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1, plt2_e t2 WHERE t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+ 150 | A0003 | 150 | A0003
+ 300 | A0006 | 300 | A0006
+ 450 | A0009 | 450 | A0009
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_e t1, uplt2_e t2 WHERE t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+ 150 | A0003 | 150 | A0003
+ 300 | A0006 | 300 | A0006
+ 450 | A0009 | 450 | A0009
+(4 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 LEFT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((t2.b = t1.a) AND (ltrim(t2.c, 'A'::text) = ltrim(t1.c, 'A'::text)))
+               ->  Seq Scan on public.plt2_e_p1 t2
+                     Output: t2.b, t2.c
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.plt1_e_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((t2_1.b = t1_1.a) AND (ltrim(t2_1.c, 'A'::text) = ltrim(t1_1.c, 'A'::text)))
+               ->  Seq Scan on public.plt2_e_p2 t2_1
+                     Output: t2_1.b, t2_1.c
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.plt1_e_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((t2_2.b = t1_2.a) AND (ltrim(t2_2.c, 'A'::text) = ltrim(t1_2.c, 'A'::text)))
+               ->  Seq Scan on public.plt2_e_p3 t2_2
+                     Output: t2_2.b, t2_2.c
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.plt1_e_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 LEFT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+  50 | A0001 |     | 
+ 100 | A0002 |     | 
+ 150 | A0003 | 150 | A0003
+ 200 | A0004 |     | 
+ 250 | A0005 |     | 
+ 300 | A0006 | 300 | A0006
+ 350 | A0007 |     | 
+ 400 | A0008 |     | 
+ 450 | A0009 | 450 | A0009
+ 500 | A0010 |     | 
+ 550 | A0011 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_e t1 LEFT JOIN uplt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+  50 | A0001 |     | 
+ 100 | A0002 |     | 
+ 150 | A0003 | 150 | A0003
+ 200 | A0004 |     | 
+ 250 | A0005 |     | 
+ 300 | A0006 | 300 | A0006
+ 350 | A0007 |     | 
+ 400 | A0008 |     | 
+ 450 | A0009 | 450 | A0009
+ 500 | A0010 |     | 
+ 550 | A0011 |     | 
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 RIGHT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+                                                  QUERY PLAN                                                  
+--------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: ((t1.a = t2.b) AND (ltrim(t1.c, 'A'::text) = ltrim(t2.c, 'A'::text)))
+                     ->  Seq Scan on public.plt1_e_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t2.b, t2.c
+                           ->  Seq Scan on public.plt2_e_p1 t2
+                                 Output: t2.b, t2.c
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: ((t1_1.a = t2_1.b) AND (ltrim(t1_1.c, 'A'::text) = ltrim(t2_1.c, 'A'::text)))
+                     ->  Seq Scan on public.plt1_e_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c
+                           ->  Seq Scan on public.plt2_e_p2 t2_1
+                                 Output: t2_1.b, t2_1.c
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: ((t1_2.a = t2_2.b) AND (ltrim(t1_2.c, 'A'::text) = ltrim(t2_2.c, 'A'::text)))
+                     ->  Seq Scan on public.plt1_e_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c
+                           ->  Seq Scan on public.plt2_e_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+                                 Filter: ((t2_2.b % 25) = 0)
+(36 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 RIGHT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+ 150 | A0003 | 150 | A0003
+ 300 | A0006 | 300 | A0006
+ 450 | A0009 | 450 | A0009
+     |       |  75 | A0001
+     |       | 225 | A0004
+     |       | 375 | A0007
+     |       | 525 | A0010
+(8 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_e t1 RIGHT JOIN uplt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+ 150 | A0003 | 150 | A0003
+ 300 | A0006 | 300 | A0006
+ 450 | A0009 | 450 | A0009
+     |       |  75 | A0001
+     |       | 225 | A0004
+     |       | 375 | A0007
+     |       | 525 | A0010
+(8 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2_e WHERE plt2_e.b % 25 = 0) t2 ON (t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A')) ORDER BY t1.a, t2.b;
+                                                         QUERY PLAN                                                         
+----------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: plt1_e_p1.a, plt1_e_p1.c, plt2_e_p1.b, plt2_e_p1.c
+   Sort Key: plt1_e_p1.a, plt2_e_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: plt1_e_p1.a, plt1_e_p1.c, plt2_e_p1.b, plt2_e_p1.c
+               Hash Cond: ((plt1_e_p1.a = plt2_e_p1.b) AND (ltrim(plt1_e_p1.c, 'A'::text) = ltrim(plt2_e_p1.c, 'A'::text)))
+               ->  Seq Scan on public.plt1_e_p1
+                     Output: plt1_e_p1.a, plt1_e_p1.c
+                     Filter: ((plt1_e_p1.a % 25) = 0)
+               ->  Hash
+                     Output: plt2_e_p1.b, plt2_e_p1.c
+                     ->  Seq Scan on public.plt2_e_p1
+                           Output: plt2_e_p1.b, plt2_e_p1.c
+                           Filter: ((plt2_e_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: plt1_e_p2.a, plt1_e_p2.c, plt2_e_p2.b, plt2_e_p2.c
+               Hash Cond: ((plt1_e_p2.a = plt2_e_p2.b) AND (ltrim(plt1_e_p2.c, 'A'::text) = ltrim(plt2_e_p2.c, 'A'::text)))
+               ->  Seq Scan on public.plt1_e_p2
+                     Output: plt1_e_p2.a, plt1_e_p2.c
+                     Filter: ((plt1_e_p2.a % 25) = 0)
+               ->  Hash
+                     Output: plt2_e_p2.b, plt2_e_p2.c
+                     ->  Seq Scan on public.plt2_e_p2
+                           Output: plt2_e_p2.b, plt2_e_p2.c
+                           Filter: ((plt2_e_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: plt1_e_p3.a, plt1_e_p3.c, plt2_e_p3.b, plt2_e_p3.c
+               Hash Cond: ((plt1_e_p3.a = plt2_e_p3.b) AND (ltrim(plt1_e_p3.c, 'A'::text) = ltrim(plt2_e_p3.c, 'A'::text)))
+               ->  Seq Scan on public.plt1_e_p3
+                     Output: plt1_e_p3.a, plt1_e_p3.c
+                     Filter: ((plt1_e_p3.a % 25) = 0)
+               ->  Hash
+                     Output: plt2_e_p3.b, plt2_e_p3.c
+                     ->  Seq Scan on public.plt2_e_p3
+                           Output: plt2_e_p3.b, plt2_e_p3.c
+                           Filter: ((plt2_e_p3.b % 25) = 0)
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2_e WHERE plt2_e.b % 25 = 0) t2 ON (t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A')) ORDER BY t1.a, t2.b;
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+  50 | A0001 |     | 
+ 100 | A0002 |     | 
+ 150 | A0003 | 150 | A0003
+ 200 | A0004 |     | 
+ 250 | A0005 |     | 
+ 300 | A0006 | 300 | A0006
+ 350 | A0007 |     | 
+ 400 | A0008 |     | 
+ 450 | A0009 | 450 | A0009
+ 500 | A0010 |     | 
+ 550 | A0011 |     | 
+     |       |  75 | A0001
+     |       | 225 | A0004
+     |       | 375 | A0007
+     |       | 525 | A0010
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uplt1_e t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2_e t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A')) ORDER BY t1.a, t2.b;
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+  50 | A0001 |     | 
+ 100 | A0002 |     | 
+ 150 | A0003 | 150 | A0003
+ 200 | A0004 |     | 
+ 250 | A0005 |     | 
+ 300 | A0006 | 300 | A0006
+ 350 | A0007 |     | 
+ 400 | A0008 |     | 
+ 450 | A0009 | 450 | A0009
+ 500 | A0010 |     | 
+ 550 | A0011 |     | 
+     |       |  75 | A0001
+     |       | 225 | A0004
+     |       | 375 | A0007
+     |       | 525 | A0010
+(16 rows)
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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
+   Output: (avg(t1.a)), (avg(t2.b)), (avg((t3.a + t3.b))), t1.c, t2.c, t3.c
+   Sort Key: t1.c, t3.c
+   ->  HashAggregate
+         Output: avg(t1.a), avg(t2.b), avg((t3.a + t3.b)), t1.c, t2.c, t3.c
+         Group Key: t1.c, t2.c, t3.c
+         ->  Result
+               Output: t1.c, t2.c, t3.c, t1.a, t2.b, t3.a, t3.b
+               ->  Append
+                     ->  Hash Join
+                           Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                           Hash Cond: (t1.c = t2.c)
+                           ->  Seq Scan on public.plt1_p1 t1
+                                 Output: t1.a, t1.c
+                           ->  Hash
+                                 Output: t2.b, t2.c, t3.a, t3.b, t3.c
+                                 ->  Hash Join
+                                       Output: t2.b, t2.c, t3.a, t3.b, t3.c
+                                       Hash Cond: (t2.c = ltrim(t3.c, 'A'::text))
+                                       ->  Seq Scan on public.plt2_p1 t2
+                                             Output: t2.b, t2.c
+                                       ->  Hash
+                                             Output: t3.a, t3.b, t3.c
+                                             ->  Seq Scan on public.plt1_e_p1 t3
+                                                   Output: t3.a, t3.b, t3.c
+                     ->  Hash Join
+                           Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                           Hash Cond: (t1_1.c = t2_1.c)
+                           ->  Seq Scan on public.plt1_p2 t1_1
+                                 Output: t1_1.a, t1_1.c
+                           ->  Hash
+                                 Output: t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                                 ->  Hash Join
+                                       Output: t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                                       Hash Cond: (t2_1.c = ltrim(t3_1.c, 'A'::text))
+                                       ->  Seq Scan on public.plt2_p2 t2_1
+                                             Output: t2_1.b, t2_1.c
+                                       ->  Hash
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             ->  Seq Scan on public.plt1_e_p2 t3_1
+                                                   Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash Join
+                           Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                           Hash Cond: (t1_2.c = t2_2.c)
+                           ->  Seq Scan on public.plt1_p3 t1_2
+                                 Output: t1_2.a, t1_2.c
+                           ->  Hash
+                                 Output: t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                                 ->  Hash Join
+                                       Output: t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                                       Hash Cond: (t2_2.c = ltrim(t3_2.c, 'A'::text))
+                                       ->  Seq Scan on public.plt2_p3 t2_2
+                                             Output: t2_2.b, t2_2.c
+                                       ->  Hash
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             ->  Seq Scan on public.plt1_e_p3 t3_2
+                                                   Output: t3_2.a, t3_2.b, t3_2.c
+(57 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)
+
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM uplt1 t1, uplt2 t2, uplt1_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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: ((t3.a = t1.a) AND (ltrim(t3.c, 'A'::text) = t1.c))
+                     ->  Seq Scan on public.plt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                                 ->  Seq Scan on public.plt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.plt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: ((t3_1.a = t1_1.a) AND (ltrim(t3_1.c, 'A'::text) = t1_1.c))
+                     ->  Seq Scan on public.plt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.c = t1_1.c))
+                                 ->  Seq Scan on public.plt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.plt1_p2 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: ((t3_2.a = t1_2.a) AND (ltrim(t3_2.c, 'A'::text) = t1_2.c))
+                     ->  Seq Scan on public.plt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                                 ->  Seq Scan on public.plt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.plt1_p3 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: ((t3.a = t2.b) AND (ltrim(t3.c, 'A'::text) = t2.c))
+                     ->  Seq Scan on public.plt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                                 ->  Seq Scan on public.plt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.plt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: ((t3_1.a = t2_1.b) AND (ltrim(t3_1.c, 'A'::text) = t2_1.c))
+                     ->  Seq Scan on public.plt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.c = t1_1.c))
+                                 ->  Seq Scan on public.plt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.plt1_p2 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: ((t3_2.a = t2_2.b) AND (ltrim(t3_2.c, 'A'::text) = t2_2.c))
+                     ->  Seq Scan on public.plt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                                 ->  Seq Scan on public.plt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.plt1_p3 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |          | 
+ 100 | 0002 |     |      |          | 
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |          | 
+ 250 | 0005 |     |      |          | 
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |          | 
+ 400 | 0008 |     |      |          | 
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |          | 
+ 550 | 0011 |     |      |          | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |          | 
+ 100 | 0002 |     |      |          | 
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |          | 
+ 250 | 0005 |     |      |          | 
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |          | 
+ 400 | 0008 |     |      |          | 
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |          | 
+ 550 | 0011 |     |      |          | 
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Hash Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                     ->  Seq Scan on public.plt2_p1 t2
+                           Output: t2.b, t2.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Hash Cond: ((t1.c = ltrim(t3.c, 'A'::text)) AND (t1.a = t3.a))
+                                 ->  Seq Scan on public.plt1_p1 t1
+                                       Output: t1.a, t1.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.plt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                     Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.c = t1_1.c))
+                     ->  Seq Scan on public.plt2_p2 t2_1
+                           Output: t2_1.b, t2_1.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_1.a, t1_1.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_1.a, t1_1.c
+                                 Hash Cond: ((t1_1.c = ltrim(t3_1.c, 'A'::text)) AND (t1_1.a = t3_1.a))
+                                 ->  Seq Scan on public.plt1_p2 t1_1
+                                       Output: t1_1.a, t1_1.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.plt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                     Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                     ->  Seq Scan on public.plt2_p3 t2_2
+                           Output: t2_2.b, t2_2.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_2.a, t1_2.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_2.a, t1_2.c
+                                 Hash Cond: ((t1_2.c = ltrim(t3_2.c, 'A'::text)) AND (t1_2.a = t3_2.a))
+                                 ->  Seq Scan on public.plt1_p3 t1_2
+                                       Output: t1_2.a, t1_2.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.plt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: ((t1.a = t2.b) AND (t1.c = t2.c))
+                     ->  Seq Scan on public.plt1_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Hash Cond: ((t2.b = t3.a) AND (t2.c = ltrim(t3.c, 'A'::text)))
+                                 ->  Seq Scan on public.plt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.plt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: ((t1_1.a = t2_1.b) AND (t1_1.c = t2_1.c))
+                     ->  Seq Scan on public.plt1_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: ((t2_1.b = t3_1.a) AND (t2_1.c = ltrim(t3_1.c, 'A'::text)))
+                                 ->  Seq Scan on public.plt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.plt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: ((t1_2.a = t2_2.b) AND (t1_2.c = t2_2.c))
+                     ->  Seq Scan on public.plt1_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: ((t2_2.b = t3_2.a) AND (t2_2.c = ltrim(t3_2.c, 'A'::text)))
+                                 ->  Seq Scan on public.plt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.plt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+     |      |     |      |      100 | A0001
+     |      |     |      |      200 | A0002
+     |      |     |      |      400 | A0004
+     |      |     |      |      500 | A0005
+     |      |     |      |      700 | A0007
+     |      |     |      |      800 | A0008
+     |      |     |      |     1000 | A0010
+     |      |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+     |      |     |      |      100 | A0001
+     |      |     |      |      200 | A0002
+     |      |     |      |      400 | A0004
+     |      |     |      |      500 | A0005
+     |      |     |      |      700 | A0007
+     |      |     |      |      800 | A0008
+     |      |     |      |     1000 | A0010
+     |      |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                                  QUERY PLAN                                                   
+---------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, ((plt1_e_p1.a + plt1_e_p1.b)), plt1_e_p1.c
+   Sort Key: plt1_p1.a, plt2_p1.b, ((plt1_e_p1.a + plt1_e_p1.b))
+   ->  Result
+         Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, (plt1_e_p1.a + plt1_e_p1.b), plt1_e_p1.c
+         ->  Append
+               ->  Hash Full Join
+                     Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                     Hash Cond: ((plt1_p1.a = plt1_e_p1.a) AND (plt1_p1.c = ltrim(plt1_e_p1.c, 'A'::text)))
+                     ->  Hash Full Join
+                           Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+                           Hash Cond: ((plt1_p1.a = plt2_p1.b) AND (plt1_p1.c = plt2_p1.c))
+                           ->  Seq Scan on public.plt1_p1
+                                 Output: plt1_p1.a, plt1_p1.c
+                                 Filter: ((plt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p1.b, plt2_p1.c
+                                 ->  Seq Scan on public.plt2_p1
+                                       Output: plt2_p1.b, plt2_p1.c
+                                       Filter: ((plt2_p1.b % 25) = 0)
+                     ->  Hash
+                           Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                           ->  Seq Scan on public.plt1_e_p1
+                                 Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                                 Filter: ((plt1_e_p1.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c, plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                     Hash Cond: ((plt1_p2.a = plt1_e_p2.a) AND (plt1_p2.c = ltrim(plt1_e_p2.c, 'A'::text)))
+                     ->  Hash Full Join
+                           Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+                           Hash Cond: ((plt1_p2.a = plt2_p2.b) AND (plt1_p2.c = plt2_p2.c))
+                           ->  Seq Scan on public.plt1_p2
+                                 Output: plt1_p2.a, plt1_p2.c
+                                 Filter: ((plt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p2.b, plt2_p2.c
+                                 ->  Seq Scan on public.plt2_p2
+                                       Output: plt2_p2.b, plt2_p2.c
+                                       Filter: ((plt2_p2.b % 25) = 0)
+                     ->  Hash
+                           Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                           ->  Seq Scan on public.plt1_e_p2
+                                 Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                                 Filter: ((plt1_e_p2.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c, plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                     Hash Cond: ((plt1_p3.a = plt1_e_p3.a) AND (plt1_p3.c = ltrim(plt1_e_p3.c, 'A'::text)))
+                     ->  Hash Full Join
+                           Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+                           Hash Cond: ((plt1_p3.a = plt2_p3.b) AND (plt1_p3.c = plt2_p3.c))
+                           ->  Seq Scan on public.plt1_p3
+                                 Output: plt1_p3.a, plt1_p3.c
+                                 Filter: ((plt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p3.b, plt2_p3.c
+                                 ->  Seq Scan on public.plt2_p3
+                                       Output: plt2_p3.b, plt2_p3.c
+                                       Filter: ((plt2_p3.b % 25) = 0)
+                     ->  Hash
+                           Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                           ->  Seq Scan on public.plt1_e_p3
+                                 Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                                 Filter: ((plt1_e_p3.a % 25) = 0)
+(63 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.c = t1_3.c)
+               ->  Seq Scan on public.plt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_3.c, t2.c
+                     Hash Cond: (t1_3.c = ltrim(t2.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p1 t1_3
+                           Output: t1_3.c
+                     ->  Hash
+                           Output: t2.c
+                           ->  Seq Scan on public.plt1_e_p1 t2
+                                 Output: t2.c
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.c = t1_4.c)
+               ->  Seq Scan on public.plt1_p2 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_4.c, t2_1.c
+                     Hash Cond: (t1_4.c = ltrim(t2_1.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p2 t1_4
+                           Output: t1_4.c
+                     ->  Hash
+                           Output: t2_1.c
+                           ->  Seq Scan on public.plt1_e_p2 t2_1
+                                 Output: t2_1.c
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.c = t1_5.c)
+               ->  Seq Scan on public.plt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_5.c, t2_2.c
+                     Hash Cond: (t1_5.c = ltrim(t2_2.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p3 t1_5
+                           Output: t1_5.c
+                     ->  Hash
+                           Output: t2_2.c
+                           ->  Seq Scan on public.plt1_e_p3 t2_2
+                                 Output: t2_2.c
+(49 rows)
+
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1, uplt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+                                   QUERY PLAN                                   
+--------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.c = t1_3.c)
+               ->  Seq Scan on public.plt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_3.c, t1_6.c
+                     Hash Cond: (t1_3.c = ltrim(t1_6.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p1 t1_3
+                           Output: t1_3.c
+                     ->  Hash
+                           Output: t1_6.c
+                           ->  HashAggregate
+                                 Output: t1_6.c
+                                 Group Key: ltrim(t1_6.c, 'A'::text)
+                                 ->  Seq Scan on public.plt1_e_p1 t1_6
+                                       Output: t1_6.c, ltrim(t1_6.c, 'A'::text)
+                                       Filter: ((t1_6.a % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.c = t1_4.c)
+               ->  Seq Scan on public.plt1_p2 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_4.c, t1_7.c
+                     Hash Cond: (t1_4.c = ltrim(t1_7.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p2 t1_4
+                           Output: t1_4.c
+                     ->  Hash
+                           Output: t1_7.c
+                           ->  HashAggregate
+                                 Output: t1_7.c
+                                 Group Key: ltrim(t1_7.c, 'A'::text)
+                                 ->  Seq Scan on public.plt1_e_p2 t1_7
+                                       Output: t1_7.c, ltrim(t1_7.c, 'A'::text)
+                                       Filter: ((t1_7.a % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.c = t1_5.c)
+               ->  Seq Scan on public.plt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_5.c, t1_8.c
+                     Hash Cond: (t1_5.c = ltrim(t1_8.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p3 t1_5
+                           Output: t1_5.c
+                     ->  Hash
+                           Output: t1_8.c
+                           ->  HashAggregate
+                                 Output: t1_8.c
+                                 Group Key: ltrim(t1_8.c, 'A'::text)
+                                 ->  Seq Scan on public.plt1_e_p3 t1_8
+                                       Output: t1_8.c, ltrim(t1_8.c, 'A'::text)
+                                       Filter: ((t1_8.a % 25) = 0)
+(61 rows)
+
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+-- test merge join with and without index scan
+CREATE INDEX iplt1_c on plt1(c);
+CREATE INDEX iplt1_p1_c on plt1_p1(c);
+CREATE INDEX iplt1_p2_c on plt1_p2(c);
+CREATE INDEX iplt1_p3_c on plt1_p3(c);
+CREATE INDEX iplt2_c on plt2(c);
+CREATE INDEX iplt2_p1_c on plt2_p1(c);
+CREATE INDEX iplt2_p2_c on plt2_p2(c);
+CREATE INDEX iplt2_p3_c on plt2_p3(c);
+CREATE INDEX iplt1_e_c on plt1_e(ltrim(c, 'A'));
+CREATE INDEX iplt1_e_p1_c on plt1_e_p1(ltrim(c, 'A'));
+CREATE INDEX iplt1_e_p2_c on plt1_e_p2(ltrim(c, 'A'));
+CREATE INDEX iplt1_e_p3_c on plt1_e_p3(ltrim(c, 'A'));
+ANALYZE plt1;
+ANALYZE plt1_p1;
+ANALYZE plt1_p2;
+ANALYZE plt1_p3;
+ANALYZE plt2;
+ANALYZE plt2_p1;
+ANALYZE plt2_p2;
+ANALYZE plt2_p3;
+ANALYZE plt1_e;
+ANALYZE plt1_e_p1;
+ANALYZE plt1_e_p2;
+ANALYZE plt1_e_p3;
+SET enable_hashjoin TO off;
+SET enable_nestloop TO off;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                                QUERY PLAN                                                 
+-----------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Merge Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           Sort Key: t2.b, t2.c
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Merge Cond: ((t3.a = t2.b) AND ((ltrim(t3.c, 'A'::text)) = t2.c))
+                                 ->  Sort
+                                       Output: t3.a, t3.b, t3.c, (ltrim(t3.c, 'A'::text))
+                                       Sort Key: t3.a, (ltrim(t3.c, 'A'::text))
+                                       ->  Seq Scan on public.plt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c, ltrim(t3.c, 'A'::text)
+                                             Filter: ((t3.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2.b, t2.c
+                                       Sort Key: t2.b, t2.c
+                                       ->  Seq Scan on public.plt2_p1 t2
+                                             Output: t2.b, t2.c
+                     ->  Sort
+                           Output: t1.a, t1.c
+                           Sort Key: t1.a, t1.c
+                           ->  Seq Scan on public.plt1_p1 t1
+                                 Output: t1.a, t1.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Merge Cond: ((t2_1.c = t1_1.c) AND (t2_1.b = t1_1.a))
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           Sort Key: t2_1.c, t2_1.b
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Merge Cond: ((t3_1.a = t2_1.b) AND ((ltrim(t3_1.c, 'A'::text)) = t2_1.c))
+                                 ->  Sort
+                                       Output: t3_1.a, t3_1.b, t3_1.c, (ltrim(t3_1.c, 'A'::text))
+                                       Sort Key: t3_1.a, (ltrim(t3_1.c, 'A'::text))
+                                       ->  Seq Scan on public.plt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c, ltrim(t3_1.c, 'A'::text)
+                                             Filter: ((t3_1.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2_1.b, t2_1.c
+                                       Sort Key: t2_1.b, t2_1.c
+                                       ->  Seq Scan on public.plt2_p2 t2_1
+                                             Output: t2_1.b, t2_1.c
+                     ->  Sort
+                           Output: t1_1.a, t1_1.c
+                           Sort Key: t1_1.c, t1_1.a
+                           ->  Seq Scan on public.plt1_p2 t1_1
+                                 Output: t1_1.a, t1_1.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Merge Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           Sort Key: t2_2.b, t2_2.c
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Merge Cond: ((t3_2.a = t2_2.b) AND ((ltrim(t3_2.c, 'A'::text)) = t2_2.c))
+                                 ->  Sort
+                                       Output: t3_2.a, t3_2.b, t3_2.c, (ltrim(t3_2.c, 'A'::text))
+                                       Sort Key: t3_2.a, (ltrim(t3_2.c, 'A'::text))
+                                       ->  Seq Scan on public.plt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c, ltrim(t3_2.c, 'A'::text)
+                                             Filter: ((t3_2.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2_2.b, t2_2.c
+                                       Sort Key: t2_2.b, t2_2.c
+                                       ->  Seq Scan on public.plt2_p3 t2_2
+                                             Output: t2_2.b, t2_2.c
+                     ->  Sort
+                           Output: t1_2.a, t1_2.c
+                           Sort Key: t1_2.a, t1_2.c
+                           ->  Seq Scan on public.plt1_p3 t1_2
+                                 Output: t1_2.a, t1_2.c
+(81 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+     |      |     |      |      100 | A0001
+     |      |     |      |      200 | A0002
+     |      |     |      |      400 | A0004
+     |      |     |      |      500 | A0005
+     |      |     |      |      700 | A0007
+     |      |     |      |      800 | A0008
+     |      |     |      |     1000 | A0010
+     |      |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+     |      |     |      |      100 | A0001
+     |      |     |      |      200 | A0002
+     |      |     |      |      400 | A0004
+     |      |     |      |      500 | A0005
+     |      |     |      |      700 | A0007
+     |      |     |      |      800 | A0008
+     |      |     |      |     1000 | A0010
+     |      |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                                  QUERY PLAN                                                   
+---------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, ((plt1_e_p1.a + plt1_e_p1.b)), plt1_e_p1.c
+   Sort Key: plt1_p1.a, plt2_p1.b, ((plt1_e_p1.a + plt1_e_p1.b))
+   ->  Result
+         Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, (plt1_e_p1.a + plt1_e_p1.b), plt1_e_p1.c
+         ->  Append
+               ->  Merge Full Join
+                     Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                     Merge Cond: ((plt1_p1.a = plt1_e_p1.a) AND (plt1_p1.c = (ltrim(plt1_e_p1.c, 'A'::text))))
+                     ->  Sort
+                           Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+                           Sort Key: plt1_p1.a, plt1_p1.c
+                           ->  Merge Full Join
+                                 Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+                                 Merge Cond: ((plt1_p1.a = plt2_p1.b) AND (plt1_p1.c = plt2_p1.c))
+                                 ->  Sort
+                                       Output: plt1_p1.a, plt1_p1.c
+                                       Sort Key: plt1_p1.a, plt1_p1.c
+                                       ->  Seq Scan on public.plt1_p1
+                                             Output: plt1_p1.a, plt1_p1.c
+                                             Filter: ((plt1_p1.a % 25) = 0)
+                                 ->  Sort
+                                       Output: plt2_p1.b, plt2_p1.c
+                                       Sort Key: plt2_p1.b, plt2_p1.c
+                                       ->  Seq Scan on public.plt2_p1
+                                             Output: plt2_p1.b, plt2_p1.c
+                                             Filter: ((plt2_p1.b % 25) = 0)
+                     ->  Sort
+                           Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c, (ltrim(plt1_e_p1.c, 'A'::text))
+                           Sort Key: plt1_e_p1.a, (ltrim(plt1_e_p1.c, 'A'::text))
+                           ->  Seq Scan on public.plt1_e_p1
+                                 Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c, ltrim(plt1_e_p1.c, 'A'::text)
+                                 Filter: ((plt1_e_p1.a % 25) = 0)
+               ->  Merge Full Join
+                     Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c, plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                     Merge Cond: ((plt1_p2.a = plt1_e_p2.a) AND (plt1_p2.c = (ltrim(plt1_e_p2.c, 'A'::text))))
+                     ->  Sort
+                           Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+                           Sort Key: plt1_p2.a, plt1_p2.c
+                           ->  Merge Full Join
+                                 Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+                                 Merge Cond: ((plt1_p2.a = plt2_p2.b) AND (plt1_p2.c = plt2_p2.c))
+                                 ->  Sort
+                                       Output: plt1_p2.a, plt1_p2.c
+                                       Sort Key: plt1_p2.a, plt1_p2.c
+                                       ->  Seq Scan on public.plt1_p2
+                                             Output: plt1_p2.a, plt1_p2.c
+                                             Filter: ((plt1_p2.a % 25) = 0)
+                                 ->  Sort
+                                       Output: plt2_p2.b, plt2_p2.c
+                                       Sort Key: plt2_p2.b, plt2_p2.c
+                                       ->  Seq Scan on public.plt2_p2
+                                             Output: plt2_p2.b, plt2_p2.c
+                                             Filter: ((plt2_p2.b % 25) = 0)
+                     ->  Sort
+                           Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c, (ltrim(plt1_e_p2.c, 'A'::text))
+                           Sort Key: plt1_e_p2.a, (ltrim(plt1_e_p2.c, 'A'::text))
+                           ->  Seq Scan on public.plt1_e_p2
+                                 Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c, ltrim(plt1_e_p2.c, 'A'::text)
+                                 Filter: ((plt1_e_p2.a % 25) = 0)
+               ->  Merge Full Join
+                     Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c, plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                     Merge Cond: ((plt1_p3.a = plt1_e_p3.a) AND (plt1_p3.c = (ltrim(plt1_e_p3.c, 'A'::text))))
+                     ->  Sort
+                           Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+                           Sort Key: plt1_p3.a, plt1_p3.c
+                           ->  Merge Full Join
+                                 Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+                                 Merge Cond: ((plt1_p3.a = plt2_p3.b) AND (plt1_p3.c = plt2_p3.c))
+                                 ->  Sort
+                                       Output: plt1_p3.a, plt1_p3.c
+                                       Sort Key: plt1_p3.a, plt1_p3.c
+                                       ->  Seq Scan on public.plt1_p3
+                                             Output: plt1_p3.a, plt1_p3.c
+                                             Filter: ((plt1_p3.a % 25) = 0)
+                                 ->  Sort
+                                       Output: plt2_p3.b, plt2_p3.c
+                                       Sort Key: plt2_p3.b, plt2_p3.c
+                                       ->  Seq Scan on public.plt2_p3
+                                             Output: plt2_p3.b, plt2_p3.c
+                                             Filter: ((plt2_p3.b % 25) = 0)
+                     ->  Sort
+                           Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c, (ltrim(plt1_e_p3.c, 'A'::text))
+                           Sort Key: plt1_e_p3.a, (ltrim(plt1_e_p3.c, 'A'::text))
+                           ->  Seq Scan on public.plt1_e_p3
+                                 Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c, ltrim(plt1_e_p3.c, 'A'::text)
+                                 Filter: ((plt1_e_p3.a % 25) = 0)
+(87 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+                                      QUERY PLAN                                      
+--------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Merge Semi Join
+               Output: t1.a, t1.b, t1.c
+               Merge Cond: (t1.c = t1_3.c)
+               ->  Sort
+                     Output: t1.a, t1.b, t1.c
+                     Sort Key: t1.c
+                     ->  Seq Scan on public.plt1_p1 t1
+                           Output: t1.a, t1.b, t1.c
+                           Filter: ((t1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_3.c, t1_6.c
+                     ->  Merge Join
+                           Output: t1_3.c, t1_6.c
+                           Merge Cond: ((ltrim(t1_6.c, 'A'::text)) = t1_3.c)
+                           ->  Sort
+                                 Output: t1_6.c, (ltrim(t1_6.c, 'A'::text))
+                                 Sort Key: (ltrim(t1_6.c, 'A'::text))
+                                 ->  HashAggregate
+                                       Output: t1_6.c, (ltrim(t1_6.c, 'A'::text))
+                                       Group Key: ltrim(t1_6.c, 'A'::text)
+                                       ->  Seq Scan on public.plt1_e_p1 t1_6
+                                             Output: t1_6.c, ltrim(t1_6.c, 'A'::text)
+                                             Filter: ((t1_6.a % 25) = 0)
+                           ->  Sort
+                                 Output: t1_3.c
+                                 Sort Key: t1_3.c
+                                 ->  Seq Scan on public.plt2_p1 t1_3
+                                       Output: t1_3.c
+         ->  Merge Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Merge Cond: (t1_1.c = t1_4.c)
+               ->  Sort
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Sort Key: t1_1.c
+                     ->  Seq Scan on public.plt1_p2 t1_1
+                           Output: t1_1.a, t1_1.b, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_4.c, t1_7.c
+                     ->  Merge Join
+                           Output: t1_4.c, t1_7.c
+                           Merge Cond: ((ltrim(t1_7.c, 'A'::text)) = t1_4.c)
+                           ->  Sort
+                                 Output: t1_7.c, (ltrim(t1_7.c, 'A'::text))
+                                 Sort Key: (ltrim(t1_7.c, 'A'::text))
+                                 ->  HashAggregate
+                                       Output: t1_7.c, (ltrim(t1_7.c, 'A'::text))
+                                       Group Key: ltrim(t1_7.c, 'A'::text)
+                                       ->  Seq Scan on public.plt1_e_p2 t1_7
+                                             Output: t1_7.c, ltrim(t1_7.c, 'A'::text)
+                                             Filter: ((t1_7.a % 25) = 0)
+                           ->  Sort
+                                 Output: t1_4.c
+                                 Sort Key: t1_4.c
+                                 ->  Seq Scan on public.plt2_p2 t1_4
+                                       Output: t1_4.c
+         ->  Merge Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Merge Cond: (t1_2.c = t1_5.c)
+               ->  Sort
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Sort Key: t1_2.c
+                     ->  Seq Scan on public.plt1_p3 t1_2
+                           Output: t1_2.a, t1_2.b, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_5.c, t1_8.c
+                     ->  Merge Join
+                           Output: t1_5.c, t1_8.c
+                           Merge Cond: ((ltrim(t1_8.c, 'A'::text)) = t1_5.c)
+                           ->  Sort
+                                 Output: t1_8.c, (ltrim(t1_8.c, 'A'::text))
+                                 Sort Key: (ltrim(t1_8.c, 'A'::text))
+                                 ->  HashAggregate
+                                       Output: t1_8.c, (ltrim(t1_8.c, 'A'::text))
+                                       Group Key: ltrim(t1_8.c, 'A'::text)
+                                       ->  Seq Scan on public.plt1_e_p3 t1_8
+                                             Output: t1_8.c, ltrim(t1_8.c, 'A'::text)
+                                             Filter: ((t1_8.a % 25) = 0)
+                           ->  Sort
+                                 Output: t1_5.c
+                                 Sort Key: t1_5.c
+                                 ->  Seq Scan on public.plt2_p3 t1_5
+                                       Output: t1_5.c
+(88 rows)
+
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+SET enable_seqscan TO off;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Merge Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           Sort Key: t2.b, t2.c
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Merge Cond: (ltrim(t3.c, 'A'::text) = t2.c)
+                                 Join Filter: (t2.b = t3.a)
+                                 ->  Index Scan using iplt1_e_p1_c on public.plt1_e_p1 t3
+                                       Output: t3.a, t3.b, t3.c
+                                       Filter: ((t3.a % 25) = 0)
+                                 ->  Index Scan using iplt2_p1_c on public.plt2_p1 t2
+                                       Output: t2.b, t2.c
+                     ->  Sort
+                           Output: t1.a, t1.c
+                           Sort Key: t1.a, t1.c
+                           ->  Index Scan using iplt1_p1_c on public.plt1_p1 t1
+                                 Output: t1.a, t1.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Merge Cond: ((t2_1.c = t1_1.c) AND (t2_1.b = t1_1.a))
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           Sort Key: t2_1.c, t2_1.b
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Merge Cond: (ltrim(t3_1.c, 'A'::text) = t2_1.c)
+                                 Join Filter: (t2_1.b = t3_1.a)
+                                 ->  Index Scan using iplt1_e_p2_c on public.plt1_e_p2 t3_1
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       Filter: ((t3_1.a % 25) = 0)
+                                 ->  Index Scan using iplt2_p2_c on public.plt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                     ->  Sort
+                           Output: t1_1.a, t1_1.c
+                           Sort Key: t1_1.c, t1_1.a
+                           ->  Index Scan using iplt1_p2_c on public.plt1_p2 t1_1
+                                 Output: t1_1.a, t1_1.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Merge Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           Sort Key: t2_2.b, t2_2.c
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Merge Cond: (ltrim(t3_2.c, 'A'::text) = t2_2.c)
+                                 Join Filter: (t2_2.b = t3_2.a)
+                                 ->  Index Scan using iplt1_e_p3_c on public.plt1_e_p3 t3_2
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       Filter: ((t3_2.a % 25) = 0)
+                                 ->  Index Scan using iplt2_p3_c on public.plt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                     ->  Sort
+                           Output: t1_2.a, t1_2.c
+                           Sort Key: t1_2.a, t1_2.c
+                           ->  Index Scan using iplt1_p3_c on public.plt1_p3 t1_2
+                                 Output: t1_2.a, t1_2.c
+(66 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+     |      |     |      |      100 | A0001
+     |      |     |      |      200 | A0002
+     |      |     |      |      400 | A0004
+     |      |     |      |      500 | A0005
+     |      |     |      |      700 | A0007
+     |      |     |      |      800 | A0008
+     |      |     |      |     1000 | A0010
+     |      |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+     |      |     |      |      100 | A0001
+     |      |     |      |      200 | A0002
+     |      |     |      |      400 | A0004
+     |      |     |      |      500 | A0005
+     |      |     |      |      700 | A0007
+     |      |     |      |      800 | A0008
+     |      |     |      |     1000 | A0010
+     |      |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                                  QUERY PLAN                                                   
+---------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, ((plt1_e_p1.a + plt1_e_p1.b)), plt1_e_p1.c
+   Sort Key: plt1_p1.a, plt2_p1.b, ((plt1_e_p1.a + plt1_e_p1.b))
+   ->  Result
+         Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, (plt1_e_p1.a + plt1_e_p1.b), plt1_e_p1.c
+         ->  Append
+               ->  Merge Full Join
+                     Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                     Merge Cond: ((plt1_p1.a = plt1_e_p1.a) AND (plt1_p1.c = (ltrim(plt1_e_p1.c, 'A'::text))))
+                     ->  Sort
+                           Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+                           Sort Key: plt1_p1.a, plt1_p1.c
+                           ->  Merge Full Join
+                                 Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+                                 Merge Cond: ((plt1_p1.a = plt2_p1.b) AND (plt1_p1.c = plt2_p1.c))
+                                 ->  Sort
+                                       Output: plt1_p1.a, plt1_p1.c
+                                       Sort Key: plt1_p1.a, plt1_p1.c
+                                       ->  Index Scan using iplt1_p1_c on public.plt1_p1
+                                             Output: plt1_p1.a, plt1_p1.c
+                                             Filter: ((plt1_p1.a % 25) = 0)
+                                 ->  Sort
+                                       Output: plt2_p1.b, plt2_p1.c
+                                       Sort Key: plt2_p1.b, plt2_p1.c
+                                       ->  Index Scan using iplt2_p1_c on public.plt2_p1
+                                             Output: plt2_p1.b, plt2_p1.c
+                                             Filter: ((plt2_p1.b % 25) = 0)
+                     ->  Sort
+                           Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c, (ltrim(plt1_e_p1.c, 'A'::text))
+                           Sort Key: plt1_e_p1.a, (ltrim(plt1_e_p1.c, 'A'::text))
+                           ->  Index Scan using iplt1_e_p1_c on public.plt1_e_p1
+                                 Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c, ltrim(plt1_e_p1.c, 'A'::text)
+                                 Filter: ((plt1_e_p1.a % 25) = 0)
+               ->  Merge Full Join
+                     Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c, plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                     Merge Cond: ((plt1_p2.a = plt1_e_p2.a) AND (plt1_p2.c = (ltrim(plt1_e_p2.c, 'A'::text))))
+                     ->  Sort
+                           Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+                           Sort Key: plt1_p2.a, plt1_p2.c
+                           ->  Merge Full Join
+                                 Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+                                 Merge Cond: ((plt1_p2.a = plt2_p2.b) AND (plt1_p2.c = plt2_p2.c))
+                                 ->  Sort
+                                       Output: plt1_p2.a, plt1_p2.c
+                                       Sort Key: plt1_p2.a, plt1_p2.c
+                                       ->  Index Scan using iplt1_p2_c on public.plt1_p2
+                                             Output: plt1_p2.a, plt1_p2.c
+                                             Filter: ((plt1_p2.a % 25) = 0)
+                                 ->  Sort
+                                       Output: plt2_p2.b, plt2_p2.c
+                                       Sort Key: plt2_p2.b, plt2_p2.c
+                                       ->  Index Scan using iplt2_p2_c on public.plt2_p2
+                                             Output: plt2_p2.b, plt2_p2.c
+                                             Filter: ((plt2_p2.b % 25) = 0)
+                     ->  Sort
+                           Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c, (ltrim(plt1_e_p2.c, 'A'::text))
+                           Sort Key: plt1_e_p2.a, (ltrim(plt1_e_p2.c, 'A'::text))
+                           ->  Index Scan using iplt1_e_p2_c on public.plt1_e_p2
+                                 Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c, ltrim(plt1_e_p2.c, 'A'::text)
+                                 Filter: ((plt1_e_p2.a % 25) = 0)
+               ->  Merge Full Join
+                     Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c, plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                     Merge Cond: ((plt1_p3.a = plt1_e_p3.a) AND (plt1_p3.c = (ltrim(plt1_e_p3.c, 'A'::text))))
+                     ->  Sort
+                           Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+                           Sort Key: plt1_p3.a, plt1_p3.c
+                           ->  Merge Full Join
+                                 Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+                                 Merge Cond: ((plt1_p3.a = plt2_p3.b) AND (plt1_p3.c = plt2_p3.c))
+                                 ->  Sort
+                                       Output: plt1_p3.a, plt1_p3.c
+                                       Sort Key: plt1_p3.a, plt1_p3.c
+                                       ->  Index Scan using iplt1_p3_c on public.plt1_p3
+                                             Output: plt1_p3.a, plt1_p3.c
+                                             Filter: ((plt1_p3.a % 25) = 0)
+                                 ->  Sort
+                                       Output: plt2_p3.b, plt2_p3.c
+                                       Sort Key: plt2_p3.b, plt2_p3.c
+                                       ->  Index Scan using iplt2_p3_c on public.plt2_p3
+                                             Output: plt2_p3.b, plt2_p3.c
+                                             Filter: ((plt2_p3.b % 25) = 0)
+                     ->  Sort
+                           Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c, (ltrim(plt1_e_p3.c, 'A'::text))
+                           Sort Key: plt1_e_p3.a, (ltrim(plt1_e_p3.c, 'A'::text))
+                           ->  Index Scan using iplt1_e_p3_c on public.plt1_e_p3
+                                 Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c, ltrim(plt1_e_p3.c, 'A'::text)
+                                 Filter: ((plt1_e_p3.a % 25) = 0)
+(87 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Merge Semi Join
+               Output: t1.a, t1.b, t1.c
+               Merge Cond: (t1.c = t1_3.c)
+               ->  Index Scan using iplt1_p1_c on public.plt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_3.c, t1_6.c
+                     ->  Merge Semi Join
+                           Output: t1_3.c, t1_6.c
+                           Merge Cond: (t1_3.c = ltrim(t1_6.c, 'A'::text))
+                           ->  Index Only Scan using iplt2_p1_c on public.plt2_p1 t1_3
+                                 Output: t1_3.c
+                           ->  Materialize
+                                 Output: t1_6.c
+                                 ->  Index Scan using iplt1_e_p1_c on public.plt1_e_p1 t1_6
+                                       Output: t1_6.c
+                                       Filter: ((t1_6.a % 25) = 0)
+         ->  Merge Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Merge Cond: (t1_1.c = t1_4.c)
+               ->  Index Scan using iplt1_p2_c on public.plt1_p2 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_4.c, t1_7.c
+                     ->  Merge Semi Join
+                           Output: t1_4.c, t1_7.c
+                           Merge Cond: (t1_4.c = ltrim(t1_7.c, 'A'::text))
+                           ->  Index Only Scan using iplt2_p2_c on public.plt2_p2 t1_4
+                                 Output: t1_4.c
+                           ->  Materialize
+                                 Output: t1_7.c
+                                 ->  Index Scan using iplt1_e_p2_c on public.plt1_e_p2 t1_7
+                                       Output: t1_7.c
+                                       Filter: ((t1_7.a % 25) = 0)
+         ->  Merge Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Merge Cond: (t1_2.c = t1_5.c)
+               ->  Index Scan using iplt1_p3_c on public.plt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_5.c, t1_8.c
+                     ->  Merge Semi Join
+                           Output: t1_5.c, t1_8.c
+                           Merge Cond: (t1_5.c = ltrim(t1_8.c, 'A'::text))
+                           ->  Index Only Scan using iplt2_p3_c on public.plt2_p3 t1_5
+                                 Output: t1_5.c
+                           ->  Materialize
+                                 Output: t1_8.c
+                                 ->  Index Scan using iplt1_e_p3_c on public.plt1_e_p3 t1_8
+                                       Output: t1_8.c
+                                       Filter: ((t1_8.a % 25) = 0)
+(58 rows)
+
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+RESET enable_hashjoin;
+RESET enable_nestloop;
+RESET enable_seqscan;
+--
+-- negative testcases
+--
+-- joins where one of the relations is proven empty
+EXPLAIN (VERBOSE, 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;
+            QUERY PLAN            
+----------------------------------
+ Result
+   Output: t1.a, t1.c, t2.b, t2.c
+   One-Time Filter: false
+(3 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 LEFT JOIN prt2 t2 ON t1.a = t2.b;
+              QUERY PLAN              
+--------------------------------------
+ Result
+   Output: prt1.a, prt1.c, t2.b, t2.c
+   One-Time Filter: false
+(3 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                     
+---------------------------------------------------
+ Sort
+   Output: a, c, t2.b, t2.c
+   Sort Key: a, t2.b
+   ->  Hash Left Join
+         Output: a, c, t2.b, t2.c
+         Hash Cond: (t2.b = a)
+         ->  Append
+               ->  Seq Scan on public.prt2 t2
+                     Output: t2.b, t2.c
+                     Filter: ((t2.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p1 t2_1
+                     Output: t2_1.b, t2_1.c
+                     Filter: ((t2_1.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p2 t2_2
+                     Output: t2_2.b, t2_2.c
+                     Filter: ((t2_2.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p3 t2_3
+                     Output: t2_3.b, t2_3.c
+                     Filter: ((t2_3.a % 25) = 0)
+         ->  Hash
+               Output: a, c
+               ->  Result
+                     Output: a, c
+                     One-Time Filter: false
+(24 rows)
+
+EXPLAIN (VERBOSE, 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 % 25 = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                     
+---------------------------------------------------
+ Sort
+   Output: a, c, t2.b, t2.c
+   Sort Key: a, t2.b
+   ->  Hash Left Join
+         Output: a, c, t2.b, t2.c
+         Hash Cond: (t2.b = a)
+         ->  Append
+               ->  Seq Scan on public.prt2 t2
+                     Output: t2.b, t2.c
+                     Filter: ((t2.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p1 t2_1
+                     Output: t2_1.b, t2_1.c
+                     Filter: ((t2_1.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p2 t2_2
+                     Output: t2_2.b, t2_2.c
+                     Filter: ((t2_2.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p3 t2_3
+                     Output: t2_3.b, t2_3.c
+                     Filter: ((t2_3.a % 25) = 0)
+         ->  Hash
+               Output: a, c
+               ->  Result
+                     Output: a, c
+                     One-Time Filter: false
+(24 rows)
+
+CREATE TABLE prt1_n (a int, b int, c varchar) PARTITION BY RANGE(c);
+CREATE TABLE prt1_n_p1 PARTITION OF prt1_n FOR VALUES START ('0000') END ('0250');
+CREATE TABLE prt1_n_p2 PARTITION OF prt1_n FOR VALUES START ('0250') END ('0500');
+INSERT INTO prt1_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 499, 2) i;
+ANALYZE prt1_n;
+ANALYZE prt1_n_p1;
+ANALYZE prt1_n_p2;
+CREATE TABLE prt2_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt2_n_p1 PARTITION OF prt2_n FOR VALUES IN ('0000', '0003', '0004', '0010', '0006', '0007');
+CREATE TABLE prt2_n_p2 PARTITION OF prt2_n FOR VALUES IN ('0001', '0005', '0002', '0009', '0008', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt2_n;
+ANALYZE prt2_n_p1;
+ANALYZE prt2_n_p2;
+CREATE TABLE prt3_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt3_n_p1 PARTITION OF prt3_n FOR VALUES IN ('0000', '0004', '0006', '0007');
+CREATE TABLE prt3_n_p2 PARTITION OF prt3_n FOR VALUES IN ('0001', '0002', '0008', '0010');
+CREATE TABLE prt3_n_p3 PARTITION OF prt3_n FOR VALUES IN ('0003', '0005', '0009', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt3_n;
+ANALYZE prt3_n_p1;
+ANALYZE prt3_n_p2;
+ANALYZE prt3_n_p3;
+CREATE TABLE prt4_n (a int, b int, c text) PARTITION BY RANGE(a);
+CREATE TABLE prt4_n_p1 PARTITION OF prt4_n FOR VALUES START (0) END (300);
+CREATE TABLE prt4_n_p2 PARTITION OF prt4_n FOR VALUES START (300) END (500);
+CREATE TABLE prt4_n_p3 PARTITION OF prt4_n FOR VALUES START (500) END (600);
+INSERT INTO prt4_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt4_n;
+ANALYZE prt4_n_p1;
+ANALYZE prt4_n_p2;
+ANALYZE prt4_n_p3;
+-- partition-wise join can not be applied if the partition ranges differ
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Join
+   Hash Cond: (t1.a = t2.a)
+   ->  Append
+         ->  Seq Scan on prt1 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p2 t1_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt4_n t2
+               ->  Seq Scan on prt4_n_p1 t2_1
+               ->  Seq Scan on prt4_n_p2 t2_2
+               ->  Seq Scan on prt4_n_p3 t2_3
+(13 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 FULL JOIN prt4_n t2 ON t1.a = t2.a;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Full Join
+   Hash Cond: (t1.a = t2.a)
+   ->  Append
+         ->  Seq Scan on prt1 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p2 t1_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt4_n t2
+               ->  Seq Scan on prt4_n_p1 t2_1
+               ->  Seq Scan on prt4_n_p2 t2_2
+               ->  Seq Scan on prt4_n_p3 t2_3
+(13 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                        
+---------------------------------------------------------
+ Nested Loop Left Join
+   ->  Append
+         ->  Seq Scan on prt1 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p2 t1_3
+   ->  Append
+         ->  Seq Scan on prt2 t2
+               Filter: (t1.a < b)
+         ->  Index Scan using iprt2_p1_b on prt2_p1 t2_1
+               Index Cond: (t1.a < b)
+         ->  Index Scan using iprt2_p2_b on prt2_p2 t2_2
+               Index Cond: (t1.a < b)
+         ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+               Index Cond: (t1.a < b)
+(15 rows)
+
+-- equi-join with join condition on partial keys does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1, prt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t1.a % 25 = 0;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Join
+   Hash Cond: (((t2.b + t2.a) / 2) = t1.a)
+   ->  Append
+         ->  Seq Scan on prt2_m t2
+         ->  Seq Scan on prt2_m_p1 t2_1
+         ->  Seq Scan on prt2_m_p2 t2_2
+         ->  Seq Scan on prt2_m_p3 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_m t1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p1 t1_1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p2 t1_2
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p3 t1_3
+                     Filter: ((a % 25) = 0)
+(17 rows)
+
+-- equi-join between out-of-order partition key columns does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.a = t2.b WHERE t1.a % 25 = 0;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Right Join
+   Hash Cond: (t2.b = t1.a)
+   ->  Append
+         ->  Seq Scan on prt2_m t2
+         ->  Seq Scan on prt2_m_p1 t2_1
+         ->  Seq Scan on prt2_m_p2 t2_2
+         ->  Seq Scan on prt2_m_p3 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_m t1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p1 t1_1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p2 t1_2
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p3 t1_3
+                     Filter: ((a % 25) = 0)
+(17 rows)
+
+-- equi-join between non-key columns does not qualify for partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.c = t2.c WHERE t1.a % 25 = 0;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Right Join
+   Hash Cond: ((t2.c)::text = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt2_m t2
+         ->  Seq Scan on prt2_m_p1 t2_1
+         ->  Seq Scan on prt2_m_p2 t2_2
+         ->  Seq Scan on prt2_m_p3 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_m t1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p1 t1_1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p2 t1_2
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p3 t1_3
+                     Filter: ((a % 25) = 0)
+(17 rows)
+
+-- partition-wise join can not be applied for a join between list and range
+-- partitioned table
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1, prt2_n t2 WHERE t1.c = t2.c;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Join
+   Hash Cond: (t2.c = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt2_n t2
+         ->  Seq Scan on prt2_n_p1 t2_1
+         ->  Seq Scan on prt2_n_p2 t2_2
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(11 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 LEFT JOIN prt2_n t2 ON (t1.c = t2.c);
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Right Join
+   Hash Cond: (t2.c = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt2_n t2
+         ->  Seq Scan on prt2_n_p1 t2_1
+         ->  Seq Scan on prt2_n_p2 t2_2
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(11 rows)
+
+-- partition-wise join can not be applied between tables with different
+-- partition lists
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 RIGHT JOIN prt1 t2 ON (t1.c = t2.c);
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Left Join
+   Hash Cond: ((t2.c)::text = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt1 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p2 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(12 rows)
+
+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);
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Full Join
+   Hash Cond: ((t2.c)::text = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt1 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p2 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(12 rows)
+
diff --git a/src/test/regress/expected/rangefuncs.out b/src/test/regress/expected/rangefuncs.out
index f06cfa4..16e7f56 100644
--- a/src/test/regress/expected/rangefuncs.out
+++ b/src/test/regress/expected/rangefuncs.out
@@ -1,25 +1,26 @@
 SELECT name, setting FROM pg_settings WHERE name LIKE 'enable%';
-         name         | setting 
-----------------------+---------
- enable_bitmapscan    | on
- enable_hashagg       | on
- enable_hashjoin      | on
- enable_indexonlyscan | on
- enable_indexscan     | on
- enable_material      | on
- enable_mergejoin     | on
- enable_nestloop      | on
- enable_seqscan       | on
- enable_sort          | on
- enable_tidscan       | on
-(11 rows)
+            name            | setting 
+----------------------------+---------
+ enable_bitmapscan          | on
+ enable_hashagg             | on
+ enable_hashjoin            | on
+ enable_indexonlyscan       | on
+ enable_indexscan           | on
+ enable_material            | on
+ enable_mergejoin           | on
+ enable_nestloop            | on
+ enable_partition_wise_join | on
+ enable_seqscan             | on
+ enable_sort                | on
+ enable_tidscan             | on
+(12 rows)
 
 CREATE TABLE foo2(fooid int, f2 int);
 INSERT INTO foo2 VALUES(1, 11);
 INSERT INTO foo2 VALUES(2, 22);
 INSERT INTO foo2 VALUES(1, 111);
 CREATE FUNCTION foot(int) returns setof foo2 as 'SELECT * FROM foo2 WHERE fooid = $1 ORDER BY f2;' LANGUAGE SQL;
 -- function with ORDINALITY
 select * from foot(1) with ordinality as z(a,b,ord);
  a |  b  | ord 
 ---+-----+-----
diff --git a/src/test/regress/parallel_schedule b/src/test/regress/parallel_schedule
index 8641769..5ad149d 100644
--- a/src/test/regress/parallel_schedule
+++ b/src/test/regress/parallel_schedule
@@ -93,21 +93,21 @@ test: alter_generic alter_operator misc psql async dbsize misc_functions tsrf
 
 # rules cannot run concurrently with any test that creates a view
 test: rules psql_crosstab amutils
 
 # run by itself so it can run parallel workers
 test: select_parallel
 
 # ----------
 # Another group of parallel tests
 # ----------
-test: select_views portals_p2 foreign_key cluster dependency guc bitmapops combocid tsearch tsdicts foreign_data window xmlmap functional_deps advisory_lock json jsonb json_encoding indirect_toast equivclass
+test: select_views portals_p2 foreign_key cluster dependency guc bitmapops combocid tsearch tsdicts foreign_data window xmlmap functional_deps advisory_lock json jsonb json_encoding indirect_toast equivclass partition_join
 # ----------
 # Another group of parallel tests
 # NB: temp.sql does a reconnect which transiently uses 2 connections,
 # so keep this parallel group to at most 19 tests
 # ----------
 test: plancache limit plpgsql copy2 temp domain rangefuncs prepare without_oid conversion truncate alter_table sequence polymorphism rowtypes returning largeobject with xml
 
 # event triggers cannot run concurrently with any test that runs DDL
 test: event_trigger
 
diff --git a/src/test/regress/serial_schedule b/src/test/regress/serial_schedule
index 835cf35..b4773b8 100644
--- a/src/test/regress/serial_schedule
+++ b/src/test/regress/serial_schedule
@@ -162,10 +162,11 @@ test: truncate
 test: alter_table
 test: sequence
 test: polymorphism
 test: rowtypes
 test: returning
 test: largeobject
 test: with
 test: xml
 test: event_trigger
 test: stats
+test: partition_join
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
new file mode 100644
index 0000000..fedaff4
--- /dev/null
+++ b/src/test/regress/sql/partition_join.sql
@@ -0,0 +1,782 @@
+--
+-- PARTITION_JOIN
+-- Test partition-wise join between partitioned tables
+--
+
+--
+-- partitioned by a single column
+--
+CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES START (0) END (250);
+CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES START (500) END (600);
+CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES START (250) END (500);
+INSERT INTO prt1 SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1 AS SELECT * FROM prt1;
+
+CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES START (0) END (250);
+CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES START (250) END (500);
+CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES START (500) END (600);
+INSERT INTO prt2 SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+CREATE TABLE uprt2 AS SELECT * FROM prt2;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, 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.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+
+-- Join with pruned partitions from joining relations
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a < 450 AND t2.b > 250 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+-- lateral reference
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+--
+-- partitioned by expression
+--
+CREATE TABLE prt1_e (a int, b int, c varchar) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES START (0) END (250);
+CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES START (250) END (500);
+CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES START (500) END (600);
+INSERT INTO prt1_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_e AS SELECT * FROM prt1_e;
+
+CREATE TABLE prt2_e (a int, b int, c varchar) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES START (0) END (250);
+CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES START (250) END (500);
+CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES START (500) END (600);
+INSERT INTO prt2_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_e;
+ANALYZE prt2_e_p1;
+ANALYZE prt2_e_p2;
+ANALYZE prt2_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_e AS SELECT * FROM prt2_e;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1, uprt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1 LEFT JOIN uprt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 RIGHT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 RIGHT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1 RIGHT JOIN uprt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_e WHERE prt2_e.b % 25 = 0) t2 ON ((t1.a + t1.b)/2 = (t2.b + t2.a)/2) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_e WHERE prt2_e.b % 25 = 0) t2 ON ((t1.a + t1.b)/2 = (t2.b + t2.a)/2) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_e t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_e t2 WHERE t2.b % 25 = 0) t2 ON ((t1.a + t1.b)/2 = (t2.b + t2.a)/2) ORDER BY t1.a, t2.b;
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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.a % 25 = 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM uprt1 t1, uprt2 t2, uprt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM uprt1_e WHERE uprt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, 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.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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;
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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;
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM uprt1_e WHERE uprt1_e.a % 25 = 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;
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1, uprt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+-- test merge joins with and without using indexes
+SET enable_hashjoin TO off;
+SET enable_nestloop TO off;
+
+CREATE INDEX iprt1_a on prt1(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 iprt2_b on prt2(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 iprt1_e_ab2 on prt1_e(((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));
+
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+SET enable_seqscan TO off;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+-- lateral references and parameterized paths
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+RESET enable_hashjoin;
+RESET enable_nestloop;
+RESET enable_seqscan;
+
+--
+-- partitioned by multiple columns
+--
+CREATE TABLE prt1_m (a int, b int, c varchar) PARTITION BY RANGE(a, ((a + b)/2));
+CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES START (0, 0) END (250, 250);
+CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES START (250, 250) END (500, 500);
+CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES START (500, 500) END (600, 600);
+INSERT INTO prt1_m SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_m;
+ANALYZE prt1_m_p1;
+ANALYZE prt1_m_p2;
+ANALYZE prt1_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_m AS SELECT * FROM prt1_m;
+
+CREATE TABLE prt2_m (a int, b int, c varchar) PARTITION BY RANGE(((b + a)/2), b);
+CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES START (0, 0) END (250, 250);
+CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES START (250, 250) END (500, 500);
+CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES START (500, 500) END (600, 600);
+INSERT INTO prt2_m SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_m;
+ANALYZE prt2_m_p1;
+ANALYZE prt2_m_p2;
+ANALYZE prt2_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_m AS SELECT * FROM prt2_m;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1, prt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1, prt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_m t1, uprt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_m t1 LEFT JOIN uprt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_m t1 RIGHT JOIN uprt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_m t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_m t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+
+--
+-- multi-leveled partitions
+--
+CREATE TABLE prt1_l (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_l_p1 PARTITION OF prt1_l FOR VALUES START (0) END (250) PARTITION BY RANGE (b);
+CREATE TABLE prt1_l_p1_p1 PARTITION OF prt1_l_p1 FOR VALUES START (0) END (100);
+CREATE TABLE prt1_l_p1_p2 PARTITION OF prt1_l_p1 FOR VALUES START (100) END (250);
+CREATE TABLE prt1_l_p2 PARTITION OF prt1_l FOR VALUES START (250) END (500) PARTITION BY RANGE (c);
+CREATE TABLE prt1_l_p2_p1 PARTITION OF prt1_l_p2 FOR VALUES START ('0250') END ('0400');
+CREATE TABLE prt1_l_p2_p2 PARTITION OF prt1_l_p2 FOR VALUES START ('0400') END ('0500');
+CREATE TABLE prt1_l_p3 PARTITION OF prt1_l FOR VALUES START (500) END (600) PARTITION BY RANGE ((b + a));
+CREATE TABLE prt1_l_p3_p1 PARTITION OF prt1_l_p3 FOR VALUES START (1000) END (1100);
+CREATE TABLE prt1_l_p3_p2 PARTITION OF prt1_l_p3 FOR VALUES START (1100) END (1200);
+INSERT INTO prt1_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_l;
+ANALYZE prt1_l_p1;
+ANALYZE prt1_l_p1_p1;
+ANALYZE prt1_l_p1_p2;
+ANALYZE prt1_l_p2;
+ANALYZE prt1_l_p2_p1;
+ANALYZE prt1_l_p2_p2;
+ANALYZE prt1_l_p3;
+ANALYZE prt1_l_p3_p1;
+ANALYZE prt1_l_p3_p2;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_l AS SELECT * FROM prt1_l;
+
+CREATE TABLE prt2_l (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_l_p1 PARTITION OF prt2_l FOR VALUES START (0) END (250) PARTITION BY RANGE (a);
+CREATE TABLE prt2_l_p1_p1 PARTITION OF prt2_l_p1 FOR VALUES START (0) END (100);
+CREATE TABLE prt2_l_p1_p2 PARTITION OF prt2_l_p1 FOR VALUES START (100) END (250);
+CREATE TABLE prt2_l_p2 PARTITION OF prt2_l FOR VALUES START (250) END (500) PARTITION BY RANGE (c);
+CREATE TABLE prt2_l_p2_p1 PARTITION OF prt2_l_p2 FOR VALUES START ('0250') END ('0400');
+CREATE TABLE prt2_l_p2_p2 PARTITION OF prt2_l_p2 FOR VALUES START ('0400') END ('0500');
+CREATE TABLE prt2_l_p3 PARTITION OF prt2_l FOR VALUES START (500) END (600) PARTITION BY RANGE ((a + b));
+CREATE TABLE prt2_l_p3_p1 PARTITION OF prt2_l_p3 FOR VALUES START (1000) END (1100);
+CREATE TABLE prt2_l_p3_p2 PARTITION OF prt2_l_p3 FOR VALUES START (1100) END (1200);
+INSERT INTO prt2_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_l;
+ANALYZE prt2_l_p1;
+ANALYZE prt2_l_p1_p1;
+ANALYZE prt2_l_p1_p2;
+ANALYZE prt2_l_p2;
+ANALYZE prt2_l_p2_p1;
+ANALYZE prt2_l_p2_p2;
+ANALYZE prt2_l_p3;
+ANALYZE prt2_l_p3_p1;
+ANALYZE prt2_l_p3_p2;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_l AS SELECT * FROM prt2_l;
+
+EXPLAIN (VERBOSE, 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 = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 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 = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1, uprt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 LEFT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 RIGHT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_l t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_l t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+
+-- lateral reference
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
+			  ON t1.a = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
+			  ON t1.a = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1_l t2 JOIN uprt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
+			  ON t1.a = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
+			  ON t1.b = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
+			  ON t1.b = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1_l t2 JOIN uprt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
+			  ON t1.b = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+--
+-- 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;
+ANALYZE plt1_p1;
+ANALYZE plt1_p2;
+ANALYZE plt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1 AS SELECT * FROM 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;
+ANALYZE plt2_p1;
+ANALYZE plt2_p2;
+ANALYZE plt2_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt2 AS SELECT * FROM plt2;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.a = t2.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.a = t2.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1 t1, uplt2 t2 WHERE t1.c = t2.c AND t1.a = t2.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.a AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.a AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.a AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 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 plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 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 uplt1 t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) ORDER BY t1.a, t2.b;
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM (SELECT 50 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM (SELECT 50 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM (SELECT 50 phv, * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t1.phv), avg(t2.b), t2.c, avg(t2.phv) FROM (SELECT 25 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t1.phv), avg(t2.b), t2.c, avg(t2.phv) FROM (SELECT 25 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t1.phv), avg(t2.b), t2.c, avg(t2.phv) FROM (SELECT 25 phv, * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
+-- Join with pruned partitions from joining relations
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.c NOT IN ('0001', '0005', '0002', '0009') AND t2.c NOT IN ('0000', '0003', '0004', '0010') GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.c NOT IN ('0001', '0005', '0002', '0009') AND t2.c NOT IN ('0000', '0003', '0004', '0010') GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM uplt1 t1, uplt2 t2 WHERE t1.c = t2.c AND t1.c NOT IN ('0001', '0005', '0002', '0009') AND t2.c NOT IN ('0000', '0003', '0004', '0010') GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 LEFT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 LEFT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM uplt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 LEFT JOIN (SELECT * FROM uplt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 RIGHT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 RIGHT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM uplt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 RIGHT JOIN (SELECT * FROM uplt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 FULL JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 FULL JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM uplt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 FULL JOIN (SELECT * FROM uplt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 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;
+ANALYZE plt1_e;
+ANALYZE plt1_e_p1;
+ANALYZE plt1_e_p2;
+ANALYZE plt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1_e AS SELECT * FROM plt1_e;
+
+CREATE TABLE plt2_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
+CREATE TABLE plt2_e_p1 PARTITION OF plt2_e FOR VALUES IN ('0000', '0003', '0004', '0010');
+CREATE TABLE plt2_e_p2 PARTITION OF plt2_e FOR VALUES IN ('0001', '0005', '0002', '0009');
+CREATE TABLE plt2_e_p3 PARTITION OF plt2_e FOR VALUES IN ('0006', '0007', '0008', '0011');
+INSERT INTO plt2_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE plt2_e;
+ANALYZE plt2_e_p1;
+ANALYZE plt2_e_p2;
+ANALYZE plt2_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt2_e AS SELECT * FROM plt2_e;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1, plt2_e t2 WHERE t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1, plt2_e t2 WHERE t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_e t1, uplt2_e t2 WHERE t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 LEFT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 LEFT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_e t1 LEFT JOIN uplt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 RIGHT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 RIGHT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_e t1 RIGHT JOIN uplt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2_e WHERE plt2_e.b % 25 = 0) t2 ON (t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A')) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2_e WHERE plt2_e.b % 25 = 0) t2 ON (t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A')) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uplt1_e t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2_e t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A')) ORDER BY t1.a, t2.b;
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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;
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM uplt1 t1, uplt2 t2, uplt1_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;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1, uplt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 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 ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+-- test merge join with and without index scan
+CREATE INDEX iplt1_c on plt1(c);
+CREATE INDEX iplt1_p1_c on plt1_p1(c);
+CREATE INDEX iplt1_p2_c on plt1_p2(c);
+CREATE INDEX iplt1_p3_c on plt1_p3(c);
+CREATE INDEX iplt2_c on plt2(c);
+CREATE INDEX iplt2_p1_c on plt2_p1(c);
+CREATE INDEX iplt2_p2_c on plt2_p2(c);
+CREATE INDEX iplt2_p3_c on plt2_p3(c);
+CREATE INDEX iplt1_e_c on plt1_e(ltrim(c, 'A'));
+CREATE INDEX iplt1_e_p1_c on plt1_e_p1(ltrim(c, 'A'));
+CREATE INDEX iplt1_e_p2_c on plt1_e_p2(ltrim(c, 'A'));
+CREATE INDEX iplt1_e_p3_c on plt1_e_p3(ltrim(c, 'A'));
+
+ANALYZE plt1;
+ANALYZE plt1_p1;
+ANALYZE plt1_p2;
+ANALYZE plt1_p3;
+ANALYZE plt2;
+ANALYZE plt2_p1;
+ANALYZE plt2_p2;
+ANALYZE plt2_p3;
+ANALYZE plt1_e;
+ANALYZE plt1_e_p1;
+ANALYZE plt1_e_p2;
+ANALYZE plt1_e_p3;
+
+SET enable_hashjoin TO off;
+SET enable_nestloop TO off;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 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 ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+SET enable_seqscan TO off;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 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 ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+RESET enable_hashjoin;
+RESET enable_nestloop;
+RESET enable_seqscan;
+
+--
+-- negative testcases
+--
+
+-- joins where one of the relations is proven empty
+EXPLAIN (VERBOSE, 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;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 LEFT JOIN prt2 t2 ON t1.a = t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, 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 % 25 = 0 ORDER BY t1.a, t2.b;
+
+CREATE TABLE prt1_n (a int, b int, c varchar) PARTITION BY RANGE(c);
+CREATE TABLE prt1_n_p1 PARTITION OF prt1_n FOR VALUES START ('0000') END ('0250');
+CREATE TABLE prt1_n_p2 PARTITION OF prt1_n FOR VALUES START ('0250') END ('0500');
+INSERT INTO prt1_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 499, 2) i;
+ANALYZE prt1_n;
+ANALYZE prt1_n_p1;
+ANALYZE prt1_n_p2;
+
+CREATE TABLE prt2_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt2_n_p1 PARTITION OF prt2_n FOR VALUES IN ('0000', '0003', '0004', '0010', '0006', '0007');
+CREATE TABLE prt2_n_p2 PARTITION OF prt2_n FOR VALUES IN ('0001', '0005', '0002', '0009', '0008', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt2_n;
+ANALYZE prt2_n_p1;
+ANALYZE prt2_n_p2;
+
+CREATE TABLE prt3_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt3_n_p1 PARTITION OF prt3_n FOR VALUES IN ('0000', '0004', '0006', '0007');
+CREATE TABLE prt3_n_p2 PARTITION OF prt3_n FOR VALUES IN ('0001', '0002', '0008', '0010');
+CREATE TABLE prt3_n_p3 PARTITION OF prt3_n FOR VALUES IN ('0003', '0005', '0009', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt3_n;
+ANALYZE prt3_n_p1;
+ANALYZE prt3_n_p2;
+ANALYZE prt3_n_p3;
+
+CREATE TABLE prt4_n (a int, b int, c text) PARTITION BY RANGE(a);
+CREATE TABLE prt4_n_p1 PARTITION OF prt4_n FOR VALUES START (0) END (300);
+CREATE TABLE prt4_n_p2 PARTITION OF prt4_n FOR VALUES START (300) END (500);
+CREATE TABLE prt4_n_p3 PARTITION OF prt4_n FOR VALUES START (500) END (600);
+INSERT INTO prt4_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt4_n;
+ANALYZE prt4_n_p1;
+ANALYZE prt4_n_p2;
+ANALYZE prt4_n_p3;
+
+-- partition-wise join can not be applied if the partition ranges differ
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 FULL JOIN prt4_n t2 ON t1.a = t2.a;
+-- 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);
+-- equi-join with join condition on partial keys does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1, prt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t1.a % 25 = 0;
+-- equi-join between out-of-order partition key columns does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.a = t2.b WHERE t1.a % 25 = 0;
+-- equi-join between non-key columns does not qualify for partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.c = t2.c WHERE t1.a % 25 = 0;
+-- partition-wise join can not be applied for a join between list and range
+-- partitioned table
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1, prt2_n t2 WHERE t1.c = t2.c;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 LEFT JOIN prt2_n t2 ON (t1.c = t2.c);
+-- partition-wise join can not be applied between tables with different
+-- partition lists
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 RIGHT JOIN prt1 t2 ON (t1.c = t2.c);
+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);
#18Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#17)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Oct 14, 2016 at 12:37 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Have you tested the effect of this patch on planner memory consumption
with multi-way joins between tables with many partitions? If you
haven't, you probably should. (Testing runtime would be good, too.)
Does it grow linearly? Quadratically? Exponentially? Minor leaks
don't matter, but if we're generating too much garbage we'll have to
make sure it gets cleaned up soon enough to prevent runaway memory
usage.

I tried to check memory usage with various combinations of number of
partitions and number of relations being joined. For higher number of
relations being joined like 10 with 100 partitions, OOM killer kicked
in during the planning phase. I am suspecting
adjust_partitionrel_attrs() (changed that name to
adjust_join_appendrel_attrs() to be in sync with
adjust_appendrel_attrs()) to be the culprit. It copies expression
trees every time for joining two children. That's an exponentially
increasing number as the number of legal joins increases
exponentially. I am still investigating this.

I think the root of this problem is that the existing paths shares a
lot more substructure than the ones created by the new code. Without
a partition-wise join, the incremental memory usage for a joinrel
isn't any different whether the underlying rel is partitioned or not.
If it's partitioned, we'll be pointing to an AppendPath; if not, we'll
be pointing to some kind of Scan. But the join itself creates exactly
the same amount of new stuff regardless of what's underneath it. With
partitionwise join, that ceases to be true. Every joinrel - and the
number of those grows exponentially in the number of baserels, IICU -
needs its own list of paths for every member rel. So if a
non-partition-wise join created X paths, and there are K partitions, a
partition-wise join creates X * K paths. That's a lot.

Although we might be able to save some memory by tightening things up
here and there - for example, right now the planner isn't real smart
about recycling paths that are evicted by add_path(), and there's
probably other wastage as well - I suspect that what this shows is
that the basic design of this patch is not going to be viable.
Intuitively, it's often going to be the case that we want the "same
plan" for every partition-set. That is, if we have A JOIN B ON A.x =
B.x JOIN C ON A.y = B.y, and if A, B, and C are all compatibility
partitioned, then the result should be an Append plan with 100 join
plans under it, and all 100 of those plans should be basically mirror
images of each other. Of course, that's not really right in general:
for example, it could be that A1 is big and A2 is small while B1 is
small and B2 is big, so that the right plan for (A1 JOIN B1) and for
(A2 JOIN B2) are totally different from each other. But in many
practical cases we'll want to end up with a plan of precisely the same
shape for all children, and the current design ignores this, expending
both memory and CPU time to compute essentially-equivalent paths
across all children.

One way of attacking this problem is to gang together partitions which
are equivalent for planning purposes, as discussed in the paper "Join
Optimization Techniques for Partitioned Tables" by Herodotou, Borisov,
and Babu. However, it's not exactly clear how to do this: we could
gang together partitions that have the same index definitions, but the
sizes of the heaps, the sizes of their indexes, and the row counts
will vary from one partition to the next, and any of those things
could cause the plan choice to be different for one partition vs. the
next. We could try to come up with heuristics for when those things
are likely to be true. For example, suppose we compute the set of
partitions such that all joined relations have matching index
definitions on all tables; then, we take the biggest table in the set
and consider all tables more than half that size as part of one gang.
The biggest table becomes the leader and we compute partition-wise
paths for just that partition; the other members of the gang will
eventually get a plan that is of the same shape, but we don't actually
create it that plan until after scan/join planning is concluded.

Another idea is to try to reduce peak memory usage by performing
planning separately for each partition-set. For example, suppose we
decide to do a partition-wise join of A, B, and C. Initially, this
gets represented as a PartitionJoinPath tree, like this:

PartitionJoinPath
-> AppendPath for A
-> PartitionJoinPath
-> AppendPath for B
-> AppendPath for C

Because we haven't created individual join paths for the members, this
doesn't use much memory. Somehow, we come up with a cost for the
PartitionJoinPath; it probably won't be entirely accurate. Once
scan/join planning is concluded, if our final path contains a
PartitionJoinPath, we go back and loop over the partitions. For each
partition, we switch to a new memory context, perform planning, copy
the best path and its substructure back to the parent context, and
then reset the context. In that way, peak memory usage only grows by
about a factor of 2 rather than a factor equal to the partition count,
because we don't need to keep every possibly-useful path for every
partition all at the same time, but rather every possibly-useful path
for a single partition.

Maybe there are other ideas but I have a feeling any way you slice it
this is going to be a lot of work.

--
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

#19Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#18)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Oct 18, 2016 at 9:09 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Fri, Oct 14, 2016 at 12:37 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Have you tested the effect of this patch on planner memory consumption
with multi-way joins between tables with many partitions? If you
haven't, you probably should. (Testing runtime would be good, too.)
Does it grow linearly? Quadratically? Exponentially? Minor leaks
don't matter, but if we're generating too much garbage we'll have to
make sure it gets cleaned up soon enough to prevent runaway memory
usage.

I tried to check memory usage with various combinations of number of
partitions and number of relations being joined. For higher number of
relations being joined like 10 with 100 partitions, OOM killer kicked
in during the planning phase. I am suspecting
adjust_partitionrel_attrs() (changed that name to
adjust_join_appendrel_attrs() to be in sync with
adjust_appendrel_attrs()) to be the culprit. It copies expression
trees every time for joining two children. That's an exponentially
increasing number as the number of legal joins increases
exponentially. I am still investigating this.

I think the root of this problem is that the existing paths shares a
lot more substructure than the ones created by the new code. Without
a partition-wise join, the incremental memory usage for a joinrel
isn't any different whether the underlying rel is partitioned or not.
If it's partitioned, we'll be pointing to an AppendPath; if not, we'll
be pointing to some kind of Scan. But the join itself creates exactly
the same amount of new stuff regardless of what's underneath it. With
partitionwise join, that ceases to be true. Every joinrel - and the
number of those grows exponentially in the number of baserels, IICU -
needs its own list of paths for every member rel. So if a
non-partition-wise join created X paths, and there are K partitions, a
partition-wise join creates X * K paths. That's a lot.

Although we might be able to save some memory by tightening things up
here and there - for example, right now the planner isn't real smart
about recycling paths that are evicted by add_path(), and there's
probably other wastage as well - I suspect that what this shows is
that the basic design of this patch is not going to be viable.
Intuitively, it's often going to be the case that we want the "same
plan" for every partition-set. That is, if we have A JOIN B ON A.x =
B.x JOIN C ON A.y = B.y, and if A, B, and C are all compatibility
partitioned, then the result should be an Append plan with 100 join
plans under it, and all 100 of those plans should be basically mirror
images of each other. Of course, that's not really right in general:
for example, it could be that A1 is big and A2 is small while B1 is
small and B2 is big, so that the right plan for (A1 JOIN B1) and for
(A2 JOIN B2) are totally different from each other. But in many
practical cases we'll want to end up with a plan of precisely the same
shape for all children, and the current design ignores this, expending
both memory and CPU time to compute essentially-equivalent paths
across all children.

I think there are going to be two kinds of partitioning use-cases.
First, carefully hand-crafted by DBAs so that every partition is
different from other and so is every join between two partitions.
There will be lesser number of partitions, but creating paths for each
join between partitions will be crucial from performance point of
view. Consider, for example, systems which use partitions to
consolidate results from different sources for analytical purposes or
sharding. If we consider various points you have listed in [1]http://postgresql.nabble.com/design-for-a-partitioning-feature-was-inheritance-td5921603.html as to
why a partition is equivalent to a table, each join between partitions
is going to have very different characteristics and thus deserves a
set of paths for its own. Add to that possibility of partition pruning
or certain conditions affecting particular partitions, the need for
detailed planning evident.

The other usage of partitioning is to distribute the data and/or
quickly eliminate the data by partition pruning. In such case, all
partitions of a given table will have very similar properties. There
is a large chance that we will end up having same plans for every
partition and for joins between partitions. In such cases, I think it
suffices to create paths for just one or may be a handful partitions
of join and repeat that plan for other partitions of join. But in such
cases it also makes sense to have a light-weight representation for
partitions as compared to partitions being a full-fledged tables. If
we have such a light-weight representation, we may not even create
RelOptInfos representing joins between partitions, and different paths
for each join between partitions.

One way of attacking this problem is to gang together partitions which
are equivalent for planning purposes, as discussed in the paper "Join
Optimization Techniques for Partitioned Tables" by Herodotou, Borisov,
and Babu. However, it's not exactly clear how to do this: we could
gang together partitions that have the same index definitions, but the
sizes of the heaps, the sizes of their indexes, and the row counts
will vary from one partition to the next, and any of those things
could cause the plan choice to be different for one partition vs. the
next. We could try to come up with heuristics for when those things
are likely to be true. For example, suppose we compute the set of
partitions such that all joined relations have matching index
definitions on all tables; then, we take the biggest table in the set
and consider all tables more than half that size as part of one gang.
The biggest table becomes the leader and we compute partition-wise
paths for just that partition; the other members of the gang will
eventually get a plan that is of the same shape, but we don't actually
create it that plan until after scan/join planning is concluded.

Section 5 of that paper talks about clustering partitions together for
joining, only when there is 1:m OR n:1 partition matching for join. In
such a case, it clusters all the partitions from one relation that are
all joined with a single partition of the other relation. But I think
your idea to gang up partitions with similar properties may reduce the
number of paths we create but as you have mentioned how to gang them
up is not very clear. There are just too many factors like
availability of the indexes, sizes of tables, size of intermediate
results etc. which make it difficult to identify the properties used
for ganging up. Even after we do that, in the worst case, we will
still end up creating paths for all partitions of all joins, thus
causing increase in paths proportionate to the number of partitions.

In the section 6.3, the paper mentions that the number of paths
retained are linear in the number of child joins per parent join. So,
it's clear that the paper never considered linear increase in the
paths to be a problem or at least a problem that that work had to
solve. Now, it's surprising that their memory usage increased by 7% to
10%. But 1. they might be measuring total memory and not the memory
used by the planner and they experimented with PostgreSQL 8.3.7, which
probably tried much less number of paths than the current optimizer.

Another idea is to try to reduce peak memory usage by performing
planning separately for each partition-set. For example, suppose we
decide to do a partition-wise join of A, B, and C. Initially, this
gets represented as a PartitionJoinPath tree, like this:

PartitionJoinPath
-> AppendPath for A
-> PartitionJoinPath
-> AppendPath for B
-> AppendPath for C

Because we haven't created individual join paths for the members, this
doesn't use much memory. Somehow, we come up with a cost for the
PartitionJoinPath; it probably won't be entirely accurate. Once
scan/join planning is concluded, if our final path contains a
PartitionJoinPath, we go back and loop over the partitions.

A typical join tree will be composite: some portion partitioned and
some portion unpartitioned or different portions partitioned by
different partition schemes. In such case, inaccurate costs for
PartitionJoinPath, can affect the plan heavily, causing a suboptimal
path to be picked. Assuming that partitioning will be useful for large
sets of data, choosing a suboptimal plan can be more dangerous than
consuming memory for creating paths.

If we could come up with costs for PartitionJoinPath using some
methods of interpolation, say by sampling few partitions and then
extrapolating their costs for entire PartitionJoinPath, we can use
this method. But unless the partitions have very similar
characteristics or have such characteristics that costs can be guessed
based on the differences between the characteristics, I do not see how
that can happen. For example, while costing a PartitionJoinPath with
pathkeys, the costs will change a lot based on whether underlying
relations have indexes, or which join methods are used, which in turn
is based on properties on the partitions. Same is the case for paths
with parameterization. All such paths are important when a partitioned
join relation joins with other unpartitioned relation or a partitioned
relation with different partitioning scheme.

When each partition of base relation being joined has different
properties, the cost for join between one set of partitions can differ
from join between other set of partitions. Not only that, the costs
for various properties of resultant paths like pathkeys,
parameterization can vary a lot, depending upon the available indexes
and estimates of rows for each join. So, we need to come up with these
cost estimates separately for each join between partitions to come up
with cost of each PartitionJoinPath. If we have to calculate those
costs to create PartitionJoinPath, we better save them in paths rather
than recalculating them in the second round of planning for joins
between partitions.

For each
partition, we switch to a new memory context, perform planning, copy
the best path and its substructure back to the parent context, and
then reset the context.

This could be rather tricky. It assumes that all the code that creates
paths for joins, should not allocate any memory which is linked to
some object in a context that lives longer than the path creation
context. There is some code like create_join_clause() or
make_canonical_pathkey(), which carefully chooses which memory context
to allocate memory in. But can we ensure it always? postgres_fdw for
example allocates memory for PgFdwRelationInfo in current memory
context and attaches it in RelOptInfo, which should be in the
planner's original context. So, if we create a new memory context for
each partition, fpinfos would be invalidated when those contexts are
released. Not that, we can not enforce some restriction on the memory
usage while planning, it's hard to enforce it and bugs arising from it
may go unnoticed. GEQO planner might have its own problems with this
approach. Third party FDWs will pose a problem.

A possible solution would be to keep the track of used paths using a
reference count. Once the paths for given join tree are created, free
up the unused paths by traversing pathlist in each of the RelOptInfos.
Attached patch has a prototype implementation for the same. There are
some paths which are not linked to RelOptInfos, which need a bit
different treatment, but they can be handled too.

In that way, peak memory usage only grows by
about a factor of 2 rather than a factor equal to the partition count,
because we don't need to keep every possibly-useful path for every
partition all at the same time, but rather every possibly-useful path
for a single partition.

Maybe there are other ideas but I have a feeling any way you slice it
this is going to be a lot of work.

For the case of carefully hand-crafted partitions, I think, users
would expect the planner to use really the best plan and thus may be
willing to accommodate for increased memory usage. Going by any
approach that does not create the paths for joins between partitions
is not guaranteed to give the best plan. Users willing to provide
increased memory will be unhappy if we do not give them the best path.

The user who creates hundreds of partitions, will ideally be using
pretty powerful servers with a lot of memory. On such servers, the
linear increase in memory for paths may not be as bad as you are
portraying above, as long as its producing the best plan.

Just joining partitioned tables with hundreds of partitions does not
increase the number of paths. Number of paths increases when two
partitioned tables with similar partitioning scheme are joined with
equality condition on partition key. Unless we consider
repartitioning, how many of the joining relations share same
partitioning scheme? Section 8.6 mentions, "no TPC-H query plan,
regardless of the partitioning scheme, contains n-way child joins for
n >= 4. Maximum partitions that the paper mentions is 168 (Table 3).
My VM which has 8GB RAM and 4 cores handled that case pretty well. We
may add logic to free up space used by useless paths post-join to free
up some memory for next stages of query execution.

There will still be users, for whom the increase in the memory usage
is unexpected. Those will need to be educated or for them we might
take heuristic PartitionJoinPath based approach discussed above. But I
don't think that heuristic approach should be the default case. May be
we should supply a GUC which can switch between the approaches.

Some ideas for GUCs are 1. delay_partition_wise_join - when ON uses
the heuristic approach of PartitionJoinPath.
2. A GUC similar to join_collapse_limit may be used to limit the
number of partitioned relations being joined using partition-wise join
technique. A value of 1, indicates enable_partition_wise_join = false.
So, we may replace enable_partition_wise_join withe this GUC.
3. A GUC max_joinable_partitions (open to suggestions for name) may
specify the maximum number of partitions that two relations may have
to be eligible for partition-wise join.

I guess, using these GUCs allows a user handle the trade-off between
getting the best plan and memory usage consciously. I think, users
would like to accept a suboptimal plans consciously than being thrown
a suboptimal plan without choice.

[1]: http://postgresql.nabble.com/design-for-a-partitioning-feature-was-inheritance-td5921603.html

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

Attachments:

free_unused_paths.patchtext/x-patch; charset=US-ASCII; name=free_unused_paths.patchDownload
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index e42ef98..6a730ca 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -2200,20 +2200,75 @@ standard_join_search(PlannerInfo *root, int levels_needed, List *initial_rels)
 
 			/* Find and save the cheapest paths for this rel */
 			set_cheapest(rel);
 
 #ifdef OPTIMIZER_DEBUG
 			debug_print_rel(root, rel);
 #endif
 		}
 	}
 
+	/* Release all the memory consumed by unreferenced paths. */
+	for (lev = levels_needed - 1; lev >= 1; lev--)
+	{
+		ListCell *lc;
+		foreach (lc, root->join_rel_level[lev])
+		{
+			ListCell	*lc_path;
+			RelOptInfo *rel = (RelOptInfo *) lfirst(lc);
+
+			elog(NOTICE, "path list length before deleting %d", list_length(rel->pathlist));
+
+			lc_path = list_head(rel->pathlist);
+			while(lc_path)
+			{
+				Path *path = (Path *) lfirst(lc_path);
+
+				lc_path = lnext(lc_path);
+
+				/* Free the path if none references it. */
+				if (path->num_refs == 1)
+				{
+					elog(NOTICE, "freed unreferenced path %p of type %d", path,
+						 path->pathtype);
+
+					rel->pathlist = list_delete_ptr(rel->pathlist, path);
+					free_path(path);
+				}
+			}
+
+			elog(NOTICE, "path list length after deleting %d", list_length(rel->pathlist));
+
+			elog(NOTICE, "partial path list length before deleting %d", list_length(rel->partial_pathlist));
+			/* Do the same for partial pathlist. */
+			lc_path = list_head(rel->partial_pathlist);
+			while(lc_path)
+			{
+				Path *path = (Path *) lfirst(lc_path);
+
+				lc_path = lnext(lc_path);
+
+				/* Free the path if none references it. */
+				if (path->num_refs == 1)
+				{
+					elog(NOTICE, "freed unreferenced path %p of type %d", path,
+						 path->pathtype);
+
+					rel->partial_pathlist = list_delete_ptr(rel->partial_pathlist, path);
+					free_path(path);
+				}
+			}
+
+			elog(NOTICE, "partial path list length after deleting %d", list_length(rel->partial_pathlist));
+		}
+	}
+
 	/*
 	 * We should have a single rel at the final level.
 	 */
 	if (root->join_rel_level[levels_needed] == NIL)
 		elog(ERROR, "failed to build any %d-way joins", levels_needed);
 	Assert(list_length(root->join_rel_level[levels_needed]) == 1);
 
 	rel = (RelOptInfo *) linitial(root->join_rel_level[levels_needed]);
 
 	root->join_rel_level = NULL;
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index abb7507..6b34f6a 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -39,21 +39,21 @@ typedef enum
 } PathCostComparison;
 
 /*
  * STD_FUZZ_FACTOR is the normal fuzz factor for compare_path_costs_fuzzily.
  * XXX is it worth making this user-controllable?  It provides a tradeoff
  * between planner runtime and the accuracy of path cost comparisons.
  */
 #define STD_FUZZ_FACTOR 1.01
 
 static List *translate_sub_tlist(List *tlist, int relid);
-
+static void unref_path(Path *path);
 
 /*****************************************************************************
  *		MISC. PATH UTILITIES
  *****************************************************************************/
 
 /*
  * compare_path_costs
  *	  Return -1, 0, or +1 according as path1 is cheaper, the same cost,
  *	  or more expensive than path2 for the specified criterion.
  */
@@ -581,22 +581,21 @@ add_path(RelOptInfo *parent_rel, Path *new_path)
 		 * Remove current element from pathlist if dominated by new.
 		 */
 		if (remove_old)
 		{
 			parent_rel->pathlist = list_delete_cell(parent_rel->pathlist,
 													p1, p1_prev);
 
 			/*
 			 * Delete the data pointed-to by the deleted cell, if possible
 			 */
-			if (!IsA(old_path, IndexPath))
-				pfree(old_path);
+			free_path(old_path);
 			/* p1_prev does not advance */
 		}
 		else
 		{
 			/* new belongs after this old path if it has cost >= old's */
 			if (new_path->total_cost >= old_path->total_cost)
 				insert_after = p1;
 			/* p1_prev advances */
 			p1_prev = p1;
 		}
@@ -610,26 +609,26 @@ add_path(RelOptInfo *parent_rel, Path *new_path)
 			break;
 	}
 
 	if (accept_new)
 	{
 		/* Accept the new path: insert it at proper place in pathlist */
 		if (insert_after)
 			lappend_cell(parent_rel->pathlist, insert_after, new_path);
 		else
 			parent_rel->pathlist = lcons(new_path, parent_rel->pathlist);
+		new_path->num_refs++;
 	}
 	else
 	{
 		/* Reject and recycle the new path */
-		if (!IsA(new_path, IndexPath))
-			pfree(new_path);
+		free_path(new_path);
 	}
 }
 
 /*
  * add_path_precheck
  *	  Check whether a proposed new path could possibly get accepted.
  *	  We assume we know the path's pathkeys and parameterization accurately,
  *	  and have lower bounds for its costs.
  *
  * Note that we do not know the path's rowcount, since getting an estimate for
@@ -821,21 +820,21 @@ add_partial_path(RelOptInfo *parent_rel, Path *new_path)
 
 		/*
 		 * Remove current element from partial_pathlist if dominated by new.
 		 */
 		if (remove_old)
 		{
 			parent_rel->partial_pathlist =
 				list_delete_cell(parent_rel->partial_pathlist, p1, p1_prev);
 			/* we should not see IndexPaths here, so always safe to delete */
 			Assert(!IsA(old_path, IndexPath));
-			pfree(old_path);
+			free_path(old_path);
 			/* p1_prev does not advance */
 		}
 		else
 		{
 			/* new belongs after this old path if it has cost >= old's */
 			if (new_path->total_cost >= old_path->total_cost)
 				insert_after = p1;
 			/* p1_prev advances */
 			p1_prev = p1;
 		}
@@ -850,27 +849,28 @@ add_partial_path(RelOptInfo *parent_rel, Path *new_path)
 	}
 
 	if (accept_new)
 	{
 		/* Accept the new path: insert it at proper place */
 		if (insert_after)
 			lappend_cell(parent_rel->partial_pathlist, insert_after, new_path);
 		else
 			parent_rel->partial_pathlist =
 				lcons(new_path, parent_rel->partial_pathlist);
+		new_path->num_refs++;
 	}
 	else
 	{
 		/* we should not see IndexPaths here, so always safe to delete */
 		Assert(!IsA(new_path, IndexPath));
 		/* Reject and recycle the new path */
-		pfree(new_path);
+		free_path(new_path);
 	}
 }
 
 /*
  * add_partial_path_precheck
  *	  Check whether a proposed new partial path could possibly get accepted.
  *
  * Unlike add_path_precheck, we can ignore startup cost and parameterization,
  * since they don't matter for partial paths (see add_partial_path).  But
  * we do want to make sure we don't add a partial path if there's already
@@ -1079,20 +1079,21 @@ create_bitmap_heap_path(PlannerInfo *root,
 	pathnode->path.parent = rel;
 	pathnode->path.pathtarget = rel->reltarget;
 	pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
 														  required_outer);
 	pathnode->path.parallel_aware = false;
 	pathnode->path.parallel_safe = rel->consider_parallel;
 	pathnode->path.parallel_workers = 0;
 	pathnode->path.pathkeys = NIL;		/* always unordered */
 
 	pathnode->bitmapqual = bitmapqual;
+	bitmapqual->num_refs++;
 
 	cost_bitmap_heap_scan(&pathnode->path, root, rel,
 						  pathnode->path.param_info,
 						  bitmapqual, loop_count);
 
 	return pathnode;
 }
 
 /*
  * create_bitmap_and_path
@@ -1228,20 +1229,22 @@ create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
 	 * but since it doesn't do any selection or projection, it is a pretty
 	 * cheap node.
 	 */
 	pathnode->path.rows = 0;
 	pathnode->path.startup_cost = 0;
 	pathnode->path.total_cost = 0;
 	foreach(l, subpaths)
 	{
 		Path	   *subpath = (Path *) lfirst(l);
 
+		subpath->num_refs++;
+
 		pathnode->path.rows += subpath->rows;
 
 		if (l == list_head(subpaths))	/* first node? */
 			pathnode->path.startup_cost = subpath->startup_cost;
 		pathnode->path.total_cost += subpath->total_cost;
 		pathnode->path.parallel_safe = pathnode->path.parallel_safe &&
 			subpath->parallel_safe;
 
 		/* All child paths must have same parameterization */
 		Assert(bms_equal(PATH_REQ_OUTER(subpath), required_outer));
@@ -1290,20 +1293,21 @@ create_merge_append_path(PlannerInfo *root,
 	/*
 	 * Add up the sizes and costs of the input paths.
 	 */
 	pathnode->path.rows = 0;
 	input_startup_cost = 0;
 	input_total_cost = 0;
 	foreach(l, subpaths)
 	{
 		Path	   *subpath = (Path *) lfirst(l);
 
+		subpath->num_refs++;
 		pathnode->path.rows += subpath->rows;
 		pathnode->path.parallel_safe = pathnode->path.parallel_safe &&
 			subpath->parallel_safe;
 
 		if (pathkeys_contained_in(pathkeys, subpath->pathkeys))
 		{
 			/* Subpath is adequately ordered, we won't need to sort it */
 			input_startup_cost += subpath->startup_cost;
 			input_total_cost += subpath->total_cost;
 		}
@@ -1678,20 +1682,21 @@ create_gather_path(PlannerInfo *root, RelOptInfo *rel, Path *subpath,
 	pathnode->path.parent = rel;
 	pathnode->path.pathtarget = target;
 	pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
 														  required_outer);
 	pathnode->path.parallel_aware = false;
 	pathnode->path.parallel_safe = false;
 	pathnode->path.parallel_workers = subpath->parallel_workers;
 	pathnode->path.pathkeys = NIL;		/* Gather has unordered result */
 
 	pathnode->subpath = subpath;
+	subpath->num_refs++;
 	pathnode->single_copy = false;
 
 	if (pathnode->path.parallel_workers == 0)
 	{
 		pathnode->path.parallel_workers = 1;
 		pathnode->path.pathkeys = subpath->pathkeys;
 		pathnode->single_copy = true;
 	}
 
 	cost_gather(pathnode, root, rel, pathnode->path.param_info, rows);
@@ -1999,21 +2004,23 @@ create_nestloop_path(PlannerInfo *root,
 								  required_outer,
 								  &restrict_clauses);
 	pathnode->path.parallel_aware = false;
 	pathnode->path.parallel_safe = joinrel->consider_parallel &&
 		outer_path->parallel_safe && inner_path->parallel_safe;
 	/* This is a foolish way to estimate parallel_workers, but for now... */
 	pathnode->path.parallel_workers = outer_path->parallel_workers;
 	pathnode->path.pathkeys = pathkeys;
 	pathnode->jointype = jointype;
 	pathnode->outerjoinpath = outer_path;
+	outer_path->num_refs++;
 	pathnode->innerjoinpath = inner_path;
+	inner_path->num_refs++;
 	pathnode->joinrestrictinfo = restrict_clauses;
 
 	final_cost_nestloop(root, pathnode, workspace, sjinfo, semifactors);
 
 	return pathnode;
 }
 
 /*
  * create_mergejoin_path
  *	  Creates a pathnode corresponding to a mergejoin join between
@@ -2062,21 +2069,23 @@ create_mergejoin_path(PlannerInfo *root,
 								  required_outer,
 								  &restrict_clauses);
 	pathnode->jpath.path.parallel_aware = false;
 	pathnode->jpath.path.parallel_safe = joinrel->consider_parallel &&
 		outer_path->parallel_safe && inner_path->parallel_safe;
 	/* This is a foolish way to estimate parallel_workers, but for now... */
 	pathnode->jpath.path.parallel_workers = outer_path->parallel_workers;
 	pathnode->jpath.path.pathkeys = pathkeys;
 	pathnode->jpath.jointype = jointype;
 	pathnode->jpath.outerjoinpath = outer_path;
+	outer_path->num_refs++;
 	pathnode->jpath.innerjoinpath = inner_path;
+	inner_path->num_refs++;
 	pathnode->jpath.joinrestrictinfo = restrict_clauses;
 	pathnode->path_mergeclauses = mergeclauses;
 	pathnode->outersortkeys = outersortkeys;
 	pathnode->innersortkeys = innersortkeys;
 	/* pathnode->materialize_inner will be set by final_cost_mergejoin */
 
 	final_cost_mergejoin(root, pathnode, workspace, sjinfo);
 
 	return pathnode;
 }
@@ -2136,21 +2145,23 @@ create_hashjoin_path(PlannerInfo *root,
 	 * and then we could assume that the output inherits the outer relation's
 	 * ordering, which might save a sort step.  However there is considerable
 	 * downside if our estimate of the inner relation size is badly off. For
 	 * the moment we don't risk it.  (Note also that if we wanted to take this
 	 * seriously, joinpath.c would have to consider many more paths for the
 	 * outer rel than it does now.)
 	 */
 	pathnode->jpath.path.pathkeys = NIL;
 	pathnode->jpath.jointype = jointype;
 	pathnode->jpath.outerjoinpath = outer_path;
+	outer_path->num_refs++;
 	pathnode->jpath.innerjoinpath = inner_path;
+	inner_path->num_refs++;
 	pathnode->jpath.joinrestrictinfo = restrict_clauses;
 	pathnode->path_hashclauses = hashclauses;
 	/* final_cost_hashjoin will fill in pathnode->num_batches */
 
 	final_cost_hashjoin(root, pathnode, workspace, sjinfo, semifactors);
 
 	return pathnode;
 }
 
 /*
@@ -3202,10 +3213,87 @@ reparameterize_path(PlannerInfo *root, Path *path,
 														 rel,
 														 spath->subpath,
 														 spath->path.pathkeys,
 														 required_outer);
 			}
 		default:
 			break;
 	}
 	return NULL;
 }
+
+void
+free_path(Path *path)
+{
+	/* Decrement the reference counts of paths referenced by this one. */
+	switch(path->pathtype)
+	{
+		case T_SeqScan:
+		case T_IndexScan:
+		case T_IndexOnlyScan:
+			/* Simple paths do nothing. */
+			break;
+
+		case T_MergeJoin:
+		case T_HashJoin:
+		case T_NestLoop:
+			{
+				JoinPath *jpath = (JoinPath *)path;
+				unref_path(jpath->outerjoinpath);
+				unref_path(jpath->innerjoinpath);
+			}
+			break;
+
+		case T_Append:
+		case T_MergeAppend:
+			{
+				AppendPath *appath = (AppendPath *)path;
+				ListCell   *lc;
+
+				foreach (lc, appath->subpaths)
+				{
+					Path *path = lfirst(lc);
+					unref_path(path);
+				}
+			}
+			break;
+
+		case T_Gather:
+			{
+				GatherPath *gpath = (GatherPath *) path;
+				unref_path(gpath->subpath);
+			}
+			break;
+
+		case T_BitmapHeapScan:
+			{
+				BitmapHeapPath *bhpath = (BitmapHeapPath *)path;
+				unref_path(bhpath->bitmapqual);
+			}
+			break;
+
+		default:
+			elog(ERROR, "unrecognized path type %d", path->pathtype);
+			break;
+	}
+
+	/* Now reclaim the memory. */
+	if (!IsA(path, IndexPath))
+		pfree(path);
+}
+
+static void
+unref_path(Path *path)
+{
+	if (!path)
+		return;
+
+	path->num_refs--;
+
+	if (path->num_refs == 0)
+	{
+		elog(NOTICE, "freed an unreferenced path %p of type %d not added to rel pathlist",
+			 path, path->pathtype);
+
+		free_path(path);
+	}
+}
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 3a1255a..a58d71b 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -891,20 +891,21 @@ typedef struct Path
 	int			parallel_workers;		/* desired # of workers; 0 = not
 										 * parallel */
 
 	/* estimated size/costs for path (see costsize.c for more info) */
 	double		rows;			/* estimated number of result tuples */
 	Cost		startup_cost;	/* cost expended before fetching any tuples */
 	Cost		total_cost;		/* total cost (assuming all tuples fetched) */
 
 	List	   *pathkeys;		/* sort ordering of path's output */
 	/* pathkeys is a List of PathKey nodes; see above */
+	int			num_refs;		/* Number of objects referencing this path. */
 } Path;
 
 /* Macro for extracting a path's parameterization relids; beware double eval */
 #define PATH_REQ_OUTER(path)  \
 	((path)->param_info ? (path)->param_info->ppi_req_outer : (Relids) NULL)
 
 /*----------
  * IndexPath represents an index scan over a single index.
  *
  * This struct is used for both regular indexscans and index-only scans;
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 44abe83..d658cd3 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -211,12 +211,13 @@ extern List *select_outer_pathkeys_for_merge(PlannerInfo *root,
 extern List *make_inner_pathkeys_for_merge(PlannerInfo *root,
 							  List *mergeclauses,
 							  List *outer_pathkeys);
 extern List *truncate_useless_pathkeys(PlannerInfo *root,
 						  RelOptInfo *rel,
 						  List *pathkeys);
 extern bool has_useful_pathkeys(PlannerInfo *root, RelOptInfo *rel);
 extern PathKey *make_canonical_pathkey(PlannerInfo *root,
 					   EquivalenceClass *eclass, Oid opfamily,
 					   int strategy, bool nulls_first);
+extern void free_path(Path *path);
 
 #endif   /* PATHS_H */
#20Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#19)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Oct 28, 2016 at 3:09 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

I think there are going to be two kinds of partitioning use-cases.
First, carefully hand-crafted by DBAs so that every partition is
different from other and so is every join between two partitions.
There will be lesser number of partitions, but creating paths for each
join between partitions will be crucial from performance point of
view. Consider, for example, systems which use partitions to
consolidate results from different sources for analytical purposes or
sharding. If we consider various points you have listed in [1] as to
why a partition is equivalent to a table, each join between partitions
is going to have very different characteristics and thus deserves a
set of paths for its own. Add to that possibility of partition pruning
or certain conditions affecting particular partitions, the need for
detailed planning evident.

The other usage of partitioning is to distribute the data and/or
quickly eliminate the data by partition pruning. In such case, all
partitions of a given table will have very similar properties. There
is a large chance that we will end up having same plans for every
partition and for joins between partitions. In such cases, I think it
suffices to create paths for just one or may be a handful partitions
of join and repeat that plan for other partitions of join. But in such
cases it also makes sense to have a light-weight representation for
partitions as compared to partitions being a full-fledged tables. If
we have such a light-weight representation, we may not even create
RelOptInfos representing joins between partitions, and different paths
for each join between partitions.

I'm not sure I see a real distinction between these two use cases. I
think that the problem of differing data distribution between
partitions is almost always going to be an issue. Take the simple
case of an "orders" table which is partitioned by month. First, the
month that's currently in progress may be much smaller than a typical
completed month. Second, many businesses are seasonal and may have
many more orders at certain times of year. For example, in American
retail, many businesses have large spikes in December. I think some
businesses may do four times as much business in December as any other
month, for example. So you will have that sort of variation, at
least.

A typical join tree will be composite: some portion partitioned and
some portion unpartitioned or different portions partitioned by
different partition schemes. In such case, inaccurate costs for
PartitionJoinPath, can affect the plan heavily, causing a suboptimal
path to be picked. Assuming that partitioning will be useful for large
sets of data, choosing a suboptimal plan can be more dangerous than
consuming memory for creating paths.

Well, sure. But, I mean, every simplifying assumption which the
planner makes to limit resource consumption could have that effect.
join_collapse_limit, for example, can cause horrible plans. However,
we have it anyway, because the alternative of having planning take far
too long is unpalatable. Planning is always, at some level,
guesswork.

For each
partition, we switch to a new memory context, perform planning, copy
the best path and its substructure back to the parent context, and
then reset the context.

This could be rather tricky. It assumes that all the code that creates
paths for joins, should not allocate any memory which is linked to
some object in a context that lives longer than the path creation
context. There is some code like create_join_clause() or
make_canonical_pathkey(), which carefully chooses which memory context
to allocate memory in. But can we ensure it always? postgres_fdw for
example allocates memory for PgFdwRelationInfo in current memory
context and attaches it in RelOptInfo, which should be in the
planner's original context. So, if we create a new memory context for
each partition, fpinfos would be invalidated when those contexts are
released. Not that, we can not enforce some restriction on the memory
usage while planning, it's hard to enforce it and bugs arising from it
may go unnoticed. GEQO planner might have its own problems with this
approach. Third party FDWs will pose a problem.

Yep, there are problems. :-)

A possible solution would be to keep the track of used paths using a
reference count. Once the paths for given join tree are created, free
up the unused paths by traversing pathlist in each of the RelOptInfos.
Attached patch has a prototype implementation for the same. There are
some paths which are not linked to RelOptInfos, which need a bit
different treatment, but they can be handled too.

So, if you apply this with your previous patch, how much does it cut
down memory consumption?

In that way, peak memory usage only grows by
about a factor of 2 rather than a factor equal to the partition count,
because we don't need to keep every possibly-useful path for every
partition all at the same time, but rather every possibly-useful path
for a single partition.

Maybe there are other ideas but I have a feeling any way you slice it
this is going to be a lot of work.

For the case of carefully hand-crafted partitions, I think, users
would expect the planner to use really the best plan and thus may be
willing to accommodate for increased memory usage. Going by any
approach that does not create the paths for joins between partitions
is not guaranteed to give the best plan. Users willing to provide
increased memory will be unhappy if we do not give them the best path.

The user who creates hundreds of partitions, will ideally be using
pretty powerful servers with a lot of memory. On such servers, the
linear increase in memory for paths may not be as bad as you are
portraying above, as long as its producing the best plan.

No, I don't agree. We should be trying to build something that scales
well. I've heard reports of customers with hundreds or even thousands
of partitions; I think it is quite reasonable to think that we need to
scale to 1000 partitions. If we use 3MB of memory to plan a query
involving unpartitioned, using 3GB to plan a query where the main
tables have been partitioned 1000 ways does not seem reasonable to me.

--
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

#21Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#20)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Mon, Oct 31, 2016 at 6:37 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Fri, Oct 28, 2016 at 3:09 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

I think there are going to be two kinds of partitioning use-cases.
First, carefully hand-crafted by DBAs so that every partition is
different from other and so is every join between two partitions.
There will be lesser number of partitions, but creating paths for each
join between partitions will be crucial from performance point of
view. Consider, for example, systems which use partitions to
consolidate results from different sources for analytical purposes or
sharding. If we consider various points you have listed in [1] as to
why a partition is equivalent to a table, each join between partitions
is going to have very different characteristics and thus deserves a
set of paths for its own. Add to that possibility of partition pruning
or certain conditions affecting particular partitions, the need for
detailed planning evident.

The other usage of partitioning is to distribute the data and/or
quickly eliminate the data by partition pruning. In such case, all
partitions of a given table will have very similar properties. There
is a large chance that we will end up having same plans for every
partition and for joins between partitions. In such cases, I think it
suffices to create paths for just one or may be a handful partitions
of join and repeat that plan for other partitions of join. But in such
cases it also makes sense to have a light-weight representation for
partitions as compared to partitions being a full-fledged tables. If
we have such a light-weight representation, we may not even create
RelOptInfos representing joins between partitions, and different paths
for each join between partitions.

I'm not sure I see a real distinction between these two use cases. I
think that the problem of differing data distribution between
partitions is almost always going to be an issue. Take the simple
case of an "orders" table which is partitioned by month. First, the
month that's currently in progress may be much smaller than a typical
completed month. Second, many businesses are seasonal and may have
many more orders at certain times of year. For example, in American
retail, many businesses have large spikes in December. I think some
businesses may do four times as much business in December as any other
month, for example. So you will have that sort of variation, at
least.

A typical join tree will be composite: some portion partitioned and
some portion unpartitioned or different portions partitioned by
different partition schemes. In such case, inaccurate costs for
PartitionJoinPath, can affect the plan heavily, causing a suboptimal
path to be picked. Assuming that partitioning will be useful for large
sets of data, choosing a suboptimal plan can be more dangerous than
consuming memory for creating paths.

Well, sure. But, I mean, every simplifying assumption which the
planner makes to limit resource consumption could have that effect.
join_collapse_limit, for example, can cause horrible plans. However,
we have it anyway, because the alternative of having planning take far
too long is unpalatable. Planning is always, at some level,
guesswork.

My point is, this behaviour is configurable. Users who are ready to
spend time and resources to get the best plan are still able to do so,
by choosing a higher limit on join_collapse_limit. Those who can not
afford to do so, are ready to use inferior plans willingly by setting
join_collapse_limit to a lower number.

A possible solution would be to keep the track of used paths using a
reference count. Once the paths for given join tree are created, free
up the unused paths by traversing pathlist in each of the RelOptInfos.
Attached patch has a prototype implementation for the same. There are
some paths which are not linked to RelOptInfos, which need a bit
different treatment, but they can be handled too.

So, if you apply this with your previous patch, how much does it cut
down memory consumption?

Answered this below:

In that way, peak memory usage only grows by
about a factor of 2 rather than a factor equal to the partition count,
because we don't need to keep every possibly-useful path for every
partition all at the same time, but rather every possibly-useful path
for a single partition.

Maybe there are other ideas but I have a feeling any way you slice it
this is going to be a lot of work.

For the case of carefully hand-crafted partitions, I think, users
would expect the planner to use really the best plan and thus may be
willing to accommodate for increased memory usage. Going by any
approach that does not create the paths for joins between partitions
is not guaranteed to give the best plan. Users willing to provide
increased memory will be unhappy if we do not give them the best path.

The user who creates hundreds of partitions, will ideally be using
pretty powerful servers with a lot of memory. On such servers, the
linear increase in memory for paths may not be as bad as you are
portraying above, as long as its producing the best plan.

No, I don't agree. We should be trying to build something that scales
well. I've heard reports of customers with hundreds or even thousands
of partitions; I think it is quite reasonable to think that we need to
scale to 1000 partitions. If we use 3MB of memory to plan a query
involving unpartitioned, using 3GB to plan a query where the main
tables have been partitioned 1000 ways does not seem reasonable to me.

Here are memory consumption numbers.

For a simple query "select * from v5_prt100", where v5_prt100 is a
view on a 5 way self join of table prt100, which is a plain table with
100 partitions without any indexes.
postgres=# \d+ v5_prt100
View "part_mem_usage.v5_prt100"
Column | Type | Modifiers | Storage | Description
--------+--------+-----------+----------+-------------
t1 | prt100 | | extended |
t2 | prt100 | | extended |
t3 | prt100 | | extended |
t4 | prt100 | | extended |
t5 | prt100 | | extended |
View definition:
SELECT t1.*::prt100 AS t1,
t2.*::prt100 AS t2,
t3.*::prt100 AS t3,
t4.*::prt100 AS t4,
t5.*::prt100 AS t5
FROM prt100 t1,
prt100 t2,
prt100 t3,
prt100 t4,
prt100 t5
WHERE t1.a = t2.a AND t2.a = t3.a AND t3.a = t4.a AND t4.a = t5.a;

postgres=# \d prt100
Table "part_mem_usage.prt100"
Column | Type | Modifiers
--------+-------------------+-----------
a | integer |
b | integer |
c | character varying |
Partition Key: RANGE (a)
Number of partitions: 100 (Use \d+ to list them.)

Without partition-wise join the standard_planner() consumes 4311 kB
memory of which 150 kB is consumed in add_paths_to_joinrel().

With partition-wise join standard_planner() consumes 65MB memory,
which is 16 times more (not 100 times more as you suspected above). Of
this bloat 16MB is consumed for creating child join paths whereas
651kB is consumed in creating append paths. That's 100 times bloat for
path creation. Rest of the memory bloat is broken down as 9MB to
create child join RelOptInfos, 29MB to translate restrict clauses, 8MB
to translate target lists. 2MB for creating special join info for
children, 2MB goes into creating plans.

If we apply logic to free unused paths, the memory consumption reduces
as follows

Without partition-wise join standard_planner() consumes 4268 kB
(against 4311kB earlier) of which 123kB (against 150kB earlier) is
consumed in add_paths_to_joinrel().

With partition-wise join, standard_planner() consumes 63MB (against
65MB earlier). Child join paths still consume 13 MB (against 16MB
earlier), which is still 100 times that without using partition-wise
join. We may shave off some memory consumption by using better methods
than translating expressions, but we will continue to have bloats
introduced by paths, RelOptInfos for child joins etc.

So, I am thinking about your approach of creating PartitionJoinPaths
without actually creating child paths and then at a later stage
actually plan the child joins. Here's rough sketch of how that may be
done.

At the time of creating regular paths, we identify the join orders
which can use partition-wise join and save those in the RelOptInfo of
the parent table. If no such join order exists, we do not create
PartitionJoinPaths for that relation. Otherwise, once we have
considered all the join orders i.e. in
generate_partition_wise_join_paths(), we create one PartitionJoinPath
for every path that has survived in the parent or at least for every
path that has distinct properties like pathkeys or parameterisation,
with those properties.

At the time of creating plans, if PartitionJoinPath is chosen, we
actually create paths for every partition of that relation
recursively. The path creation logic is carried out in a different
memory context. Amongst the paths that survive, we choose the best
path that has the same properties as PartitionJoinPath. We would
expect all parameterized paths to be retained and any unparameterized
path can be sorted to match the pathkeys of reference
PartitionJoinPath. We then create the plan out of this path and copy
it into the outer memory context and release the memory context used
for path creation. This is similar to how prepared statements save
their plans. Once we have the plan, the memory consumed by paths won't
be referenced, and hence can not create problems. At the end we create
an Append/MergeAppend plan with all the child plans and return it.

Costing PartitionJoinPath needs more thought so that we don't end up
with bad overall plans. Here's an idea. Partition-wise joins are
better compared to the unpartitioned ones, because of the smaller
sizes of partitions. If we think of join as O(MN) operation where M
and N are sizes of unpartitioned tables being joined, partition-wise
join computes P joins each with average O(M/P * N/P) order where P is
the number of partitions, which is still O(MN) with constant factor
reduced by P times. I think, we need to apply similar logic to
costing. Let's say cost of a join is J(M, N) = S (M, N) + R (M, N)
where S and R are setup cost and joining cost (for M, N rows) resp.
Cost of partition-wise join would be P * J(M/P, N/P) = P * S(M/P, N/P)
+ P * R(M/P, N/P). Each of the join methods will have different S and
R functions and may not be linear on the number of rows. So,
PartitionJoinPath costs are obtained from corresponding regular path
costs subjected to above transformation. This way, we will be
protected from choosing a PartitionJoinPath when it's not optimal.
Take example of a join where the joining relations are very small in
size, thus hash join on full relation is optimal compared to hash join
of each partition because of setup cost. In such a case, the function
which calculates the cost of hash table setup, would result in almost
same cost for full table as well as each of the partitions, thus
increasing P * S(M/P, N/P) as compared to S(M, N).

Let me know your comments.

--
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

#22Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Ashutosh Bapat (#21)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

So, I am thinking about your approach of creating PartitionJoinPaths
without actually creating child paths and then at a later stage
actually plan the child joins. Here's rough sketch of how that may be
done.

At the time of creating regular paths, we identify the join orders
which can use partition-wise join and save those in the RelOptInfo of
the parent table. If no such join order exists, we do not create
PartitionJoinPaths for that relation. Otherwise, once we have
considered all the join orders i.e. in
generate_partition_wise_join_paths(), we create one PartitionJoinPath
for every path that has survived in the parent or at least for every
path that has distinct properties like pathkeys or parameterisation,
with those properties.

At the time of creating plans, if PartitionJoinPath is chosen, we
actually create paths for every partition of that relation
recursively. The path creation logic is carried out in a different
memory context. Amongst the paths that survive, we choose the best
path that has the same properties as PartitionJoinPath. We would
expect all parameterized paths to be retained and any unparameterized
path can be sorted to match the pathkeys of reference
PartitionJoinPath. We then create the plan out of this path and copy
it into the outer memory context and release the memory context used
for path creation. This is similar to how prepared statements save
their plans. Once we have the plan, the memory consumed by paths won't
be referenced, and hence can not create problems. At the end we create
an Append/MergeAppend plan with all the child plans and return it.

Costing PartitionJoinPath needs more thought so that we don't end up
with bad overall plans. Here's an idea. Partition-wise joins are
better compared to the unpartitioned ones, because of the smaller
sizes of partitions. If we think of join as O(MN) operation where M
and N are sizes of unpartitioned tables being joined, partition-wise
join computes P joins each with average O(M/P * N/P) order where P is
the number of partitions, which is still O(MN) with constant factor
reduced by P times. I think, we need to apply similar logic to
costing. Let's say cost of a join is J(M, N) = S (M, N) + R (M, N)
where S and R are setup cost and joining cost (for M, N rows) resp.
Cost of partition-wise join would be P * J(M/P, N/P) = P * S(M/P, N/P)
+ P * R(M/P, N/P). Each of the join methods will have different S and
R functions and may not be linear on the number of rows. So,
PartitionJoinPath costs are obtained from corresponding regular path
costs subjected to above transformation. This way, we will be
protected from choosing a PartitionJoinPath when it's not optimal.
Take example of a join where the joining relations are very small in
size, thus hash join on full relation is optimal compared to hash join
of each partition because of setup cost. In such a case, the function
which calculates the cost of hash table setup, would result in almost
same cost for full table as well as each of the partitions, thus
increasing P * S(M/P, N/P) as compared to S(M, N).

Let me know your comments.

I tried to measure the impact of having a memory context reset 1000
times (once for each partition) with the attached patch. Without this
patch make check in regress/ takes about 24 seconds on my laptop and
with this patch it takes 26 seconds. This is almost 10% increase in
time. I hope that's fine.

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

Attachments:

memory_context_change.patchtext/x-patch; charset=US-ASCII; name=memory_context_change.patchDownload
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index d8c5dd3..abc34aa 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -54,6 +54,7 @@
 #include "utils/rel.h"
 #include "utils/selfuncs.h"
 #include "utils/lsyscache.h"
+#include "utils/memutils.h"
 #include "utils/syscache.h"
 
 
@@ -192,6 +193,9 @@ standard_planner(Query *parse, int cursorOptions, ParamListInfo boundParams)
 	Plan	   *top_plan;
 	ListCell   *lp,
 			   *lr;
+	MemoryContext	temp_context;
+	int			i;
+	MemoryContext	default_context;
 
 	/* Cursor options may come from caller or from DECLARE CURSOR stmt */
 	if (parse->utilityStmt &&
@@ -432,6 +436,24 @@ standard_planner(Query *parse, int cursorOptions, ParamListInfo boundParams)
 	result->invalItems = glob->invalItems;
 	result->nParamExec = glob->nParamExec;
 
+	temp_context = AllocSetContextCreate(CurrentMemoryContext,
+										   "TemporaryContext",
+										   ALLOCSET_DEFAULT_SIZES);
+
+	/* Test the time impact of creating and destroying 1000 memory contexts. */
+	for (i = 0; i < 1000; i++)
+	{
+		RelOptInfo *rel;
+		default_context = MemoryContextSwitchTo(temp_context);
+		rel = makeNode(RelOptInfo);
+		pfree(rel);
+		MemoryContextSwitchTo(default_context);
+		MemoryContextResetAndDeleteChildren(temp_context);
+	}
+
+	MemoryContextSwitchTo(default_context);
+	MemoryContextDelete(temp_context);
+
 	return result;
 }
 
#23Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#21)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Nov 4, 2016 at 6:52 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Costing PartitionJoinPath needs more thought so that we don't end up
with bad overall plans. Here's an idea. Partition-wise joins are
better compared to the unpartitioned ones, because of the smaller
sizes of partitions. If we think of join as O(MN) operation where M
and N are sizes of unpartitioned tables being joined, partition-wise
join computes P joins each with average O(M/P * N/P) order where P is
the number of partitions, which is still O(MN) with constant factor
reduced by P times. I think, we need to apply similar logic to
costing. Let's say cost of a join is J(M, N) = S (M, N) + R (M, N)
where S and R are setup cost and joining cost (for M, N rows) resp.
Cost of partition-wise join would be P * J(M/P, N/P) = P * S(M/P, N/P)
+ P * R(M/P, N/P). Each of the join methods will have different S and
R functions and may not be linear on the number of rows. So,
PartitionJoinPath costs are obtained from corresponding regular path
costs subjected to above transformation. This way, we will be
protected from choosing a PartitionJoinPath when it's not optimal.

I'm not sure that I really understand the stuff with big-O notation
and M, N, and P. But I think what you are saying is that we could
cost a PartitionJoinPath by costing some of the partitions (it might
be a good idea to choose the biggest ones) and assuming the cost for
the remaining ones will be roughly proportional. That does seem like
a reasonable strategy to me.

--
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

#24Tom Lane
tgl@sss.pgh.pa.us
In reply to: Robert Haas (#23)
Re: Partition-wise join for join between (declaratively) partitioned tables

Robert Haas <robertmhaas@gmail.com> writes:

On Fri, Nov 4, 2016 at 6:52 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Costing PartitionJoinPath needs more thought so that we don't end up
with bad overall plans. Here's an idea. Partition-wise joins are
better compared to the unpartitioned ones, because of the smaller
sizes of partitions. If we think of join as O(MN) operation where M
and N are sizes of unpartitioned tables being joined, partition-wise
join computes P joins each with average O(M/P * N/P) order where P is
the number of partitions, which is still O(MN) with constant factor
reduced by P times. I think, we need to apply similar logic to
costing. Let's say cost of a join is J(M, N) = S (M, N) + R (M, N)
where S and R are setup cost and joining cost (for M, N rows) resp.
Cost of partition-wise join would be P * J(M/P, N/P) = P * S(M/P, N/P)
+ P * R(M/P, N/P). Each of the join methods will have different S and
R functions and may not be linear on the number of rows. So,
PartitionJoinPath costs are obtained from corresponding regular path
costs subjected to above transformation. This way, we will be
protected from choosing a PartitionJoinPath when it's not optimal.

I'm not sure that I really understand the stuff with big-O notation
and M, N, and P. But I think what you are saying is that we could
cost a PartitionJoinPath by costing some of the partitions (it might
be a good idea to choose the biggest ones) and assuming the cost for
the remaining ones will be roughly proportional. That does seem like
a reasonable strategy to me.

I'm not sure to what extent the above argument depends on the assumption
that join is O(MN), but I will point out that in no case of practical
interest for large tables is it actually O(MN). That would be true
only for the stupidest possible nested-loop join method. It would be
wise to convince ourselves that the argument holds for more realistic
big-O costs, eg hash join is more like O(M+N) if all goes well.

regards, tom lane

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

#25Robert Haas
robertmhaas@gmail.com
In reply to: Tom Lane (#24)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Mon, Nov 14, 2016 at 9:57 AM, Tom Lane <tgl@sss.pgh.pa.us> wrote:

Robert Haas <robertmhaas@gmail.com> writes:

On Fri, Nov 4, 2016 at 6:52 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Costing PartitionJoinPath needs more thought so that we don't end up
with bad overall plans. Here's an idea. Partition-wise joins are
better compared to the unpartitioned ones, because of the smaller
sizes of partitions. If we think of join as O(MN) operation where M
and N are sizes of unpartitioned tables being joined, partition-wise
join computes P joins each with average O(M/P * N/P) order where P is
the number of partitions, which is still O(MN) with constant factor
reduced by P times. I think, we need to apply similar logic to
costing. Let's say cost of a join is J(M, N) = S (M, N) + R (M, N)
where S and R are setup cost and joining cost (for M, N rows) resp.
Cost of partition-wise join would be P * J(M/P, N/P) = P * S(M/P, N/P)
+ P * R(M/P, N/P). Each of the join methods will have different S and
R functions and may not be linear on the number of rows. So,
PartitionJoinPath costs are obtained from corresponding regular path
costs subjected to above transformation. This way, we will be
protected from choosing a PartitionJoinPath when it's not optimal.

I'm not sure that I really understand the stuff with big-O notation
and M, N, and P. But I think what you are saying is that we could
cost a PartitionJoinPath by costing some of the partitions (it might
be a good idea to choose the biggest ones) and assuming the cost for
the remaining ones will be roughly proportional. That does seem like
a reasonable strategy to me.

I'm not sure to what extent the above argument depends on the assumption
that join is O(MN), but I will point out that in no case of practical
interest for large tables is it actually O(MN). That would be true
only for the stupidest possible nested-loop join method. It would be
wise to convince ourselves that the argument holds for more realistic
big-O costs, eg hash join is more like O(M+N) if all goes well.

Yeah, I agree. To recap briefly, the problem we're trying to solve
here is how to build a path for a partitionwise join without an
explosion in the amount of memory the planner uses or the number of
paths created. In the initial design, if there are N partitions per
relation, the total number of paths generated by the planner increases
by a factor of N+1, which gets ugly if, say, N = 1000, or even N =
100. To reign that in, we want to do a rough cut at costing the
partitionwise join that will be good enough to let us throw away
obviously inferior paths, and then work out the exact paths we're
going to use only for partitionwise joins that are actually selected.
I think costing one or a few of the larger sub-joins and assuming
those costs are representative is probably a reasonable approach to
that problem.

--
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

#26Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#23)
2 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

Hi Robert,
Sorry for delayed response.

The attached patch implements following ideas:
1. At the time of creating paths - If the joining relations are both
partitioned and join can use partition-wise join, we create paths for
few child-joins. Similar to inheritance relations
(set_append_rel_pathlist()), we collect paths with similar properties
from all sampled child-joins and create one PartitionJoinPath with
each set of paths. The cost of the PartitionJoinPath is obtained by
multiplying the sum of costs of paths in the given set by the ratio of
(number of rows estimated in the parent-join/sum of rows in
child-joins).

2. If the PartitionJoinPath emerges as the best path, we create paths
for each of the remaining child-joins. Then we collect paths with
properties same as the given PartitionJoinPath, one from each
child-join. These paths are converted into plans and a Merge/Append
plan is created combing these plans. The paths and plans for
child-join are created in a temporary memory context. The final plan
for each child-join is copied into planner's context and the temporary
memory context is reset.

Right now, we choose 1% or 1 (whichever is higher) child-joins to base
PartitionJoinPath costs on.

Memory consumption
-----------------------------
I tested a 5-way self-join for a table with 1000 partitions, each
partition having 1M rows. The memory consumed in standard_planner()
was measured with some granular tracking
(mem_usage_func_wise_measurement_slabwise.patch). Partition-wise join
consumed total of 289MB memory which is approx 6.6 times more than
non-partition-wise join which consumed 44MB. That's much better than
the earlier 16 times consumption for 5-way join with 100 partitions.

The extra 245MB memory was consumed by child-join RelOptInfos (48MB),
SpecialJoinInfos for child-joins (64MB), restrictlist translation
(92MB), paths for sampled child-joins (1.5MB), building targetlists
for child-joins (7MB).

In order to choose representative child-joins based on the sizes of
child-joins, we need to create all the child-join RelOptInfos. In
order to estimate sizes of child-joins, we need to create
SpecialJoinInfos and restrictlists for at least one join order for all
child-joins. For every representative child-join, we need to create
SpecialJoinInfo and restrictlist for all join orders for that
child-join. We might be able to save of restrictlist translation, if
we create restrict lists from joininfo similar to parent joins. I
haven't tried that yet.

Choosing representative child-joins:
--------------------------------------------------
There's another angle to choosing representative child joins. In a
partitioned N-way join, different joins covering different subsets of
N relations, will have different size distributions across the
partitions. This means that the child-joins costed for (N-k) joins,
may be different for those required for (N-k+1) joins. With a factor
of 1% sampling, N is such that a child-join participates in 100 joins,
we will end up creating paths for all partitions before creating
PartitionJoinPaths for the final N-way join. Hopefully that will be a
rare case and usually we will end up using paths already created. We
can not avoid creating PartitionJoinPaths for subset joins, as there
might be cases when partition-wise join will be optimal for an N-k way
join but not for N-way join. We may avoid this if we choose
representative child-joins based on their positions, in which case, we
may end up with some or all of those being empty and thus skewing the
costs heavily.

Partial paths
-----------------
AFAIU, we create partial paths for append relation, when all the
children have partial paths. Unlike parameterized paths or path with
pathkeys, there is no way to create a partial path for a normal path.
This means that unless we create paths for all child-joins, we can not
create partial paths for appendrel comprising of child-joins, and thus
can not use parallel query right now. This may not be that bad, since
it would be more efficient to run each child-join in a separate
worker, rather than using multiple workers for a single child-join.

regression tests
----------------------
I observed that for small relations (1000 rows in each partition and
100 partitions), the size estimates in append relations and sum of
those in child relations are very different. As a result, the
extrapolated costs for PartitionJoinPaths as described above, are way
higher than costs of join of appends (or even append of joins if we
are to create paths for all child-joins). Thus with this approach, we
choose partition-wise join for large number of partitions with large
data (e.g. 1000 partitions with 1M rows each). These are certainly the
cases when partition-wise join is a big win. I have not tried to find
out a threshold above which partition-wise join gets chosen with above
approach, but it's going to be a larger threshold. That makes writing
regression tests difficult, as those will require large data. So, we
have to find a way so that we can test partition-wise join with
smaller data. There are few possibilities like 1. convert the fraction
of representative child-joins into GUC and setting it to 100% would
start choosing partition-wise joins for tables with a few hundred rows
per partition, like it did in earlier approach, 2. provide a way to
force partition-wise join whenever possible, by say costing
partition-wise joins much lesser than non-partition-wise join when a
GUC is set (e.g. enable_partition_wise_join with values always, never,
optimal or something like that).

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

Attachments:

mem_usage_func_wise_measurement_slabwise.patchbinary/octet-stream; name=mem_usage_func_wise_measurement_slabwise.patchDownload
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index b6ec32b..ba57118 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -47,6 +47,8 @@
 #include "rewrite/rewriteManip.h"
 #include "utils/lsyscache.h"
 #include "utils/rel.h"
+#include "utils/memutils.h"
+#include "nodes/memnodes.h"
 
 
 /* results of subquery_is_pushdown_safe */
@@ -3092,6 +3094,10 @@ generate_partition_wise_join_paths(PlannerInfo *root, RelOptInfo *rel)
 	int		num_dummy_parts;
 	ListCell   *lc;
 
+	MemoryContextCounters mem_start;
+ 
+	MemoryContextFuncStatsStart(CurrentMemoryContext, &mem_start, __FUNCTION__);
+
 	/* Handle only join relations. */
 	if (!IS_JOIN_REL(rel))
 		return;
@@ -3211,6 +3217,8 @@ generate_partition_wise_join_paths(PlannerInfo *root, RelOptInfo *rel)
 
 	if (sampled_children)
 		list_free(sampled_children);
+
+	MemoryContextFuncStatsEnd(CurrentMemoryContext, &mem_start, __FUNCTION__);
 }
 
 /*****************************************************************************
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index b4220eb..ded1b7e 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -23,6 +23,7 @@
 #include "optimizer/prep.h"
 #include "optimizer/cost.h"
 #include "utils/memutils.h"
+#include "nodes/memnodes.h"
 
 
 static void make_rels_by_clause_joins(PlannerInfo *root,
@@ -766,6 +767,13 @@ populate_joinrel_with_paths(PlannerInfo *root, RelOptInfo *rel1,
 							RelOptInfo *rel2, RelOptInfo *joinrel,
 							SpecialJoinInfo *sjinfo, List *restrictlist)
 {
+	MemoryContextCounters mem_start;
+	char *label;
+
+	label = joinrel->reloptkind == RELOPT_OTHER_JOINREL ? "child_join_path_creation" : "parent_join_path_creation";
+
+	MemoryContextFuncStatsStart(CurrentMemoryContext, &mem_start, label);
+
 	/*
 	 * Consider paths using each rel as both outer and inner.  Depending on
 	 * the join type, a provably empty outer or inner rel might mean the join
@@ -910,6 +918,8 @@ populate_joinrel_with_paths(PlannerInfo *root, RelOptInfo *rel1,
 			elog(ERROR, "unrecognized join type: %d", (int) sjinfo->jointype);
 			break;
 	}
+
+	MemoryContextFuncStatsEnd(CurrentMemoryContext, &mem_start, label);
 }
 
 /*
@@ -1322,6 +1332,10 @@ try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 	int		cnt_parts;
 	PartitionScheme		part_scheme;
 	PartitionedJoin	   *partitioned_join;
+	MemoryContextCounters start_mem;
+
+	/* Start measuring memory */
+	MemoryContextFuncStatsStart(CurrentMemoryContext, &start_mem, __FUNCTION__);
 
 	/* Guard against stack overflow due to overly deep partition hierarchy. */
 	check_stack_depth();
@@ -1415,6 +1429,7 @@ try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		List	   *join_appinfos;
 		List	   *appinfos1;
 		List	   *appinfos2;
+		MemoryContextCounters restrict_mem;
 
 		/* We should never try to join two overlapping sets of rels. */
 		Assert(!bms_overlap(child_rel1->relids, child_rel2->relids));
@@ -1431,6 +1446,7 @@ try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 		appinfos2 = find_appinfos_by_relids(root, child_rel2->relids);
 		join_appinfos = list_concat(appinfos1, appinfos2);
 
+		MemoryContextFuncStatsStart(CurrentMemoryContext, &restrict_mem, "restrictlist translation");
 		/*
 		 * Construct restrictions applicable to the child join from
 		 * those applicable to the parent join.
@@ -1439,6 +1455,8 @@ try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 												   (Node *)parent_restrictlist,
 																join_appinfos);
 
+		MemoryContextFuncStatsEnd(CurrentMemoryContext, &restrict_mem, "restrictlist translation");
+
 		/*
 		 * Construct SpecialJoinInfo from parent join relations's
 		 * SpecialJoinInfo.
@@ -1462,6 +1480,9 @@ 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);
 	}
+
+	/* Stop measuring memory and print the stats. */
+	MemoryContextFuncStatsEnd(CurrentMemoryContext, &start_mem, __FUNCTION__);
 }
 
 /*
@@ -1481,6 +1502,7 @@ add_paths_to_child_joinrel(PlannerInfo *root, RelOptInfo *parent_joinrel,
 {
 	ListCell	*lc;
 	RelOptInfo	   *child_joinrel = parent_joinrel->part_rels[child_id];
+	MemoryContextCounters rest_mem_usage;
 
 	Assert(IS_JOIN_REL(parent_joinrel));
 
@@ -1531,6 +1553,7 @@ add_paths_to_child_joinrel(PlannerInfo *root, RelOptInfo *parent_joinrel,
 		child_sjinfo = build_child_join_sjinfo(root, pj->sjinfo, appinfos1,
 											   appinfos2);
 
+		MemoryContextFuncStatsStart(CurrentMemoryContext, &rest_mem_usage, "restrictlist translation");
 		/*
 		 * Construct restrictions applicable to the child join from
 		 * those applicable to the parent join.
@@ -1539,6 +1562,8 @@ add_paths_to_child_joinrel(PlannerInfo *root, RelOptInfo *parent_joinrel,
 													 (Node *) pj->restrictlist,
 																join_appinfos);
 
+		MemoryContextFuncStatsEnd(CurrentMemoryContext, &rest_mem_usage, "restrictlist translation");
+
 		/* The list is not needed anymore. */
 		list_free(join_appinfos);
 
@@ -1565,8 +1590,11 @@ static SpecialJoinInfo *
 build_child_join_sjinfo(PlannerInfo *root, SpecialJoinInfo *parent_sjinfo,
 							List *append_rel_infos1, List *append_rel_infos2)
 {
-	SpecialJoinInfo *sjinfo = copyObject(parent_sjinfo);
+	MemoryContextCounters mem_start;
+	SpecialJoinInfo *sjinfo;
 
+	MemoryContextFuncStatsStart(CurrentMemoryContext, &mem_start, __FUNCTION__);
+	sjinfo = copyObject(parent_sjinfo);
 	sjinfo->min_lefthand = adjust_child_relids(sjinfo->min_lefthand,
 											   append_rel_infos1);
 	sjinfo->min_righthand = adjust_child_relids(sjinfo->min_righthand,
@@ -1580,6 +1608,8 @@ build_child_join_sjinfo(PlannerInfo *root, SpecialJoinInfo *parent_sjinfo,
 	sjinfo->semi_rhs_exprs = (List *) adjust_join_appendrel_attrs(root,
 											   (Node *) sjinfo->semi_rhs_exprs,
 															append_rel_infos2);
+
+	MemoryContextFuncStatsEnd(CurrentMemoryContext, &mem_start, __FUNCTION__);
 	return sjinfo;
 }
 
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 3c0898a..e9ada93 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -55,6 +55,8 @@
 #include "utils/selfuncs.h"
 #include "utils/lsyscache.h"
 #include "utils/syscache.h"
+#include "nodes/memnodes.h"
+#include "utils/memutils.h"
 
 
 /* GUC parameters */
@@ -192,6 +194,9 @@ standard_planner(Query *parse, int cursorOptions, ParamListInfo boundParams)
 	Plan	   *top_plan;
 	ListCell   *lp,
 			   *lr;
+	MemoryContextCounters mem_start;
+
+	MemoryContextFuncStatsStart(CurrentMemoryContext, &mem_start, __FUNCTION__);
 
 	/* Cursor options may come from caller or from DECLARE CURSOR stmt */
 	if (parse->utilityStmt &&
@@ -432,6 +437,8 @@ standard_planner(Query *parse, int cursorOptions, ParamListInfo boundParams)
 	result->invalItems = glob->invalItems;
 	result->nParamExec = glob->nParamExec;
 
+	MemoryContextFuncStatsEnd(CurrentMemoryContext, &mem_start, __FUNCTION__);
+
 	return result;
 }
 
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 8510775..c27c59b 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -30,6 +30,8 @@
 #include "parser/parsetree.h"
 #include "utils/lsyscache.h"
 #include "utils/selfuncs.h"
+#include "utils/memutils.h"
+#include "nodes/memnodes.h"
 
 
 typedef enum
@@ -3390,6 +3392,9 @@ reparameterize_path_by_child(PlannerInfo *root, Path *path,
 	ParamPathInfo   *new_ppi;
 	ParamPathInfo   *old_ppi;
 	List	   *child_aris;
+	MemoryContextCounters	mem_start;
+
+	MemoryContextFuncStatsStart(CurrentMemoryContext, &mem_start, __FUNCTION__);
 
 	/*
 	 * If the path is not parameterized by parent of the given relation, it
@@ -3520,5 +3525,7 @@ reparameterize_path_by_child(PlannerInfo *root, Path *path,
 			break;
 	}
 
+	MemoryContextFuncStatsEnd(CurrentMemoryContext, &mem_start, __FUNCTION__);
+
 	return new_path;
 }
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index be08b6e..5d00c85 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -31,6 +31,8 @@
 #include "rewrite/rewriteManip.h"
 #include "utils/hsearch.h"
 #include "utils/rel.h"
+#include "nodes/memnodes.h"
+#include "utils/memutils.h"
 
 
 typedef struct JoinHashEntry
@@ -598,9 +600,16 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
 						 RelOptInfo *inner_rel, RelOptInfo *parent_joinrel,
 						 JoinType jointype)
 {
-	RelOptInfo *joinrel = makeNode(RelOptInfo);
+	RelOptInfo *joinrel;
 	List	   *join_appinfos;
 
+	MemoryContextCounters mem_start;
+	MemoryContextCounters tlist_mem;
+	MemoryContextCounters jlist_mem;
+
+	MemoryContextFuncStatsStart(CurrentMemoryContext, &mem_start, __FUNCTION__);
+
+	joinrel = makeNode(RelOptInfo);
 	joinrel->reloptkind = RELOPT_OTHER_JOINREL;
 	joinrel->relids = bms_union(outer_rel->relids, inner_rel->relids);
 	joinrel->rows = 0;
@@ -659,16 +668,20 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
 	set_foreign_rel_properties(joinrel, outer_rel, inner_rel);
 
 	/* Build targetlist */
+	MemoryContextFuncStatsStart(CurrentMemoryContext, &tlist_mem, "targetlist");
 	build_joinrel_tlist(root, joinrel, outer_rel);
 	build_joinrel_tlist(root, joinrel, inner_rel);
 	/* Add placeholder variables. */
 	add_placeholders_to_joinrel(root, joinrel, outer_rel, inner_rel);
+	MemoryContextFuncStatsEnd(CurrentMemoryContext, &tlist_mem, "targetlist");
 
 	/* Translate joininfo. */
 	join_appinfos = find_appinfos_by_relids(root, joinrel->relids);
+	MemoryContextFuncStatsStart(CurrentMemoryContext, &jlist_mem, "joininfo");
 	joinrel->joininfo = (List *) adjust_join_appendrel_attrs(root,
 											 (Node *) parent_joinrel->joininfo,
 																join_appinfos);
+	MemoryContextFuncStatsEnd(CurrentMemoryContext, &jlist_mem, "joininfo");
 
 	/*
 	 * Lateral relids referred in child join will be same as that referred in
@@ -700,6 +713,8 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
 
 	pfree(join_appinfos);
 
+	MemoryContextFuncStatsEnd(CurrentMemoryContext, &mem_start, __FUNCTION__);
+
 	return joinrel;
 }
 
diff --git a/src/backend/utils/mmgr/mcxt.c b/src/backend/utils/mmgr/mcxt.c
index 5cf388f..bd68a10 100644
--- a/src/backend/utils/mmgr/mcxt.c
+++ b/src/backend/utils/mmgr/mcxt.c
@@ -514,7 +514,7 @@ MemoryContextStatsDetail(MemoryContext context, int max_children)
  * Print this context if print is true, but in any case accumulate counts into
  * *totals (if given).
  */
-static void
+void
 MemoryContextStatsInternal(MemoryContext context, int level,
 						   bool print, int max_children,
 						   MemoryContextCounters *totals)
@@ -1181,3 +1181,29 @@ pnstrdup(const char *in, Size len)
 	out[len] = '\0';
 	return out;
 }
+
+void
+MemoryContextFuncStatsStart(MemoryContext context,
+							MemoryContextCounters *start_counts,
+							const char *label)
+{
+	memset(start_counts, 0, sizeof(*start_counts));
+	MemoryContextStatsInternal(context, 0, false, 100, start_counts);
+}
+
+void
+MemoryContextFuncStatsEnd(MemoryContext context,
+						  MemoryContextCounters *start_counts,
+						  const char *label)
+{
+	MemoryContextCounters end_counts;
+	Size	start_used_space = start_counts->totalspace - start_counts->freespace;
+	Size	end_used_space;
+
+	memset(&end_counts, 0, sizeof(end_counts));
+	MemoryContextStatsInternal(context, 0, false, 100, &end_counts);
+	end_used_space = end_counts.totalspace - end_counts.freespace;
+
+	elog(NOTICE, "%s,%s,%zu,%zu,%ld", label, context->name,
+		 start_used_space, end_used_space, end_used_space - start_used_space);
+}
diff --git a/src/include/utils/memutils.h b/src/include/utils/memutils.h
index e6334a2..6a3ed55 100644
--- a/src/include/utils/memutils.h
+++ b/src/include/utils/memutils.h
@@ -122,6 +122,12 @@ extern MemoryContext MemoryContextCreate(NodeTag tag, Size size,
 					MemoryContextMethods *methods,
 					MemoryContext parent,
 					const char *name);
+extern void MemoryContextFuncStatsStart(MemoryContext context,
+										MemoryContextCounters *start_counts,
+										const char *func_label);
+extern void MemoryContextFuncStatsEnd(MemoryContext context,
+									  MemoryContextCounters *start_counts,
+									  const char *func_label);
 
 
 /*
pg_dp_join_v5.patchbinary/octet-stream; name=pg_dp_join_v5.patchDownload
diff --git a/contrib/postgres_fdw/deparse.c b/contrib/postgres_fdw/deparse.c
index 691658f..287c7d5 100644
--- a/contrib/postgres_fdw/deparse.c
+++ b/contrib/postgres_fdw/deparse.c
@@ -770,7 +770,7 @@ deparseSelectStmtForRel(StringInfo buf, PlannerInfo *root, RelOptInfo *rel,
 	deparse_expr_cxt context;
 
 	/* We handle relations for foreign tables and joins between those */
-	Assert(rel->reloptkind == RELOPT_JOINREL ||
+	Assert(IS_JOIN_REL(rel) ||
 		   rel->reloptkind == RELOPT_BASEREL ||
 		   rel->reloptkind == RELOPT_OTHER_MEMBER_REL);
 
@@ -824,7 +824,7 @@ deparseSelectSql(List *tlist, List **retrieved_attrs, deparse_expr_cxt *context)
 	 */
 	appendStringInfoString(buf, "SELECT ");
 
-	if (foreignrel->reloptkind == RELOPT_JOINREL)
+	if (IS_JOIN_REL(foreignrel))
 	{
 		/* For a join relation use the input tlist */
 		deparseExplicitTargetList(tlist, retrieved_attrs, context);
@@ -852,8 +852,7 @@ deparseSelectSql(List *tlist, List **retrieved_attrs, deparse_expr_cxt *context)
 	 * Construct FROM clause
 	 */
 	appendStringInfoString(buf, " FROM ");
-	deparseFromExprForRel(buf, root, foreignrel,
-						  (foreignrel->reloptkind == RELOPT_JOINREL),
+	deparseFromExprForRel(buf, root, foreignrel, IS_JOIN_REL(foreignrel),
 						  context->params_list);
 }
 
@@ -988,7 +987,7 @@ deparseLockingClause(deparse_expr_cxt *context)
 			appendStringInfoString(buf, " FOR UPDATE");
 
 			/* Add the relation alias if we are here for a join relation */
-			if (rel->reloptkind == RELOPT_JOINREL)
+			if (IS_JOIN_REL(rel))
 				appendStringInfo(buf, " OF %s%d", REL_ALIAS_PREFIX, relid);
 		}
 		else
@@ -1024,8 +1023,7 @@ deparseLockingClause(deparse_expr_cxt *context)
 				}
 
 				/* Add the relation alias if we are here for a join relation */
-				if (rel->reloptkind == RELOPT_JOINREL &&
-					rc->strength != LCS_NONE)
+				if (IS_JOIN_REL(rel) && rc->strength != LCS_NONE)
 					appendStringInfo(buf, " OF %s%d", REL_ALIAS_PREFIX, relid);
 			}
 		}
@@ -1162,7 +1160,7 @@ deparseFromExprForRel(StringInfo buf, PlannerInfo *root, RelOptInfo *foreignrel,
 {
 	PgFdwRelationInfo *fpinfo = (PgFdwRelationInfo *) foreignrel->fdw_private;
 
-	if (foreignrel->reloptkind == RELOPT_JOINREL)
+	if (IS_JOIN_REL(foreignrel))
 	{
 		RelOptInfo *rel_o = fpinfo->outerrel;
 		RelOptInfo *rel_i = fpinfo->innerrel;
@@ -1867,7 +1865,7 @@ deparseExpr(Expr *node, deparse_expr_cxt *context)
 static void
 deparseVar(Var *node, deparse_expr_cxt *context)
 {
-	bool		qualify_col = (context->foreignrel->reloptkind == RELOPT_JOINREL);
+	bool		qualify_col = IS_JOIN_REL(context->foreignrel);
 
 	if (bms_is_member(node->varno, context->foreignrel->relids) &&
 		node->varlevelsup == 0)
diff --git a/contrib/postgres_fdw/postgres_fdw.c b/contrib/postgres_fdw/postgres_fdw.c
index daf0438..594292a 100644
--- a/contrib/postgres_fdw/postgres_fdw.c
+++ b/contrib/postgres_fdw/postgres_fdw.c
@@ -1170,7 +1170,7 @@ postgresGetForeignPlan(PlannerInfo *root,
 			local_exprs = lappend(local_exprs, rinfo->clause);
 	}
 
-	if (foreignrel->reloptkind == RELOPT_JOINREL)
+	if (IS_JOIN_REL(foreignrel))
 	{
 		/* For a join relation, get the conditions from fdw_private structure */
 		remote_conds = fpinfo->remote_conds;
@@ -1228,7 +1228,7 @@ postgresGetForeignPlan(PlannerInfo *root,
 							 remote_conds,
 							 retrieved_attrs,
 							 makeInteger(fpinfo->fetch_size));
-	if (foreignrel->reloptkind == RELOPT_JOINREL)
+	if (IS_JOIN_REL(foreignrel))
 		fdw_private = lappend(fdw_private,
 							  makeString(fpinfo->relation_name->data));
 
@@ -2505,7 +2505,7 @@ estimate_path_cost_size(PlannerInfo *root,
 						   &remote_param_join_conds, &local_param_join_conds);
 
 		/* Build the list of columns to be fetched from the foreign server. */
-		if (foreignrel->reloptkind == RELOPT_JOINREL)
+		if (IS_JOIN_REL(foreignrel))
 			fdw_scan_tlist = build_tlist_to_deparse(foreignrel);
 		else
 			fdw_scan_tlist = NIL;
@@ -2586,7 +2586,7 @@ estimate_path_cost_size(PlannerInfo *root,
 			startup_cost = fpinfo->rel_startup_cost;
 			run_cost = fpinfo->rel_total_cost - fpinfo->rel_startup_cost;
 		}
-		else if (foreignrel->reloptkind != RELOPT_JOINREL)
+		else if (!IS_JOIN_REL(foreignrel))
 		{
 			/* Clamp retrieved rows estimates to at most foreignrel->tuples. */
 			retrieved_rows = Min(retrieved_rows, foreignrel->tuples);
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 1a60563..dc2b34b 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -20,6 +20,7 @@
 #include "access/nbtree.h"
 #include "access/sysattr.h"
 #include "catalog/dependency.h"
+#include "catalog/heap.h"
 #include "catalog/indexing.h"
 #include "catalog/objectaddress.h"
 #include "catalog/partition.h"
@@ -36,8 +37,10 @@
 #include "nodes/nodeFuncs.h"
 #include "nodes/parsenodes.h"
 #include "optimizer/clauses.h"
+#include "optimizer/cost.h"
 #include "optimizer/planmain.h"
 #include "optimizer/var.h"
+#include "rewrite/rewriteManip.h"
 #include "storage/lmgr.h"
 #include "utils/array.h"
 #include "utils/builtins.h"
@@ -2419,6 +2422,64 @@ make_range_bound(PartitionKey key, List *val, bool inclusive, bool lower)
 }
 
 /*
+ * Return a copy of input BoundCollection structure containg nparts number of
+ * partitions. The data types of bounds are described by given partition key
+ * specificiation.
+ */
+static BoundCollection
+copy_bounds(BoundCollection src_bounds, PartitionKey key, int nparts)
+{
+	BoundCollection dst_bounds;
+	int		i;
+
+	dst_bounds = (BoundCollection) palloc(sizeof(BoundCollectionData));
+
+	if (src_bounds->listinfo)
+	{
+		ListInfo *dst_li = (ListInfo *) palloc(sizeof(ListInfo));
+		ListInfo *src_li = src_bounds->listinfo;
+
+		Assert(!src_bounds->rangeinfo);
+		dst_bounds->rangeinfo = NULL;
+
+		/* Copy the ListInfo structure. */
+		dst_li->nvalues = src_li->nvalues;
+		dst_li->has_null = src_li->has_null;
+		dst_li->null_index = src_li->null_index;
+
+		dst_li->values = (Datum *) palloc(sizeof(Datum) * dst_li->nvalues);
+		dst_li->indexes = (int *) palloc(sizeof(int) * dst_li->nvalues);
+		for (i = 0; i < dst_li->nvalues; i++)
+		{
+			dst_li->values[i] = datumCopy(src_li->values[i],
+													key->tcinfo->typbyval[0],
+													key->tcinfo->typlen[0]);
+			dst_li->indexes[i] = src_li->indexes[i];
+		}
+
+		dst_bounds->listinfo = dst_li;
+	}
+	else
+	{
+		RangeInfo *dst_ri = (RangeInfo *) palloc(sizeof(RangeInfo));
+		RangeInfo *src_ri = src_bounds->rangeinfo;
+
+		Assert(!src_bounds->listinfo && src_bounds->rangeinfo);
+		dst_bounds->listinfo = NULL;
+
+		/* Copy RangeInfo structure. */
+		dst_ri = (RangeInfo *) palloc(sizeof(RangeInfo));
+		dst_ri->ranges = (PartitionRange **) palloc(sizeof(PartitionRange *) * nparts);
+		for (i = 0; i < nparts; i++)
+			dst_ri->ranges[i] = copy_range(src_ri->ranges[i], key);
+
+		dst_bounds->rangeinfo = dst_ri;
+	}
+
+	return dst_bounds;
+}
+
+/*
  * Make and return a copy of input PartitionRange.
  */
 static PartitionRange *
@@ -2654,3 +2715,168 @@ tuple_leftof_bound(PartitionKey key, Datum *tuple, PartitionRangeBound *bound)
 
 	return cmpval < 0;
 }
+
+/*
+ * find_partition_scheme
+ * 		Find the "canonical" partition scheme for the given base table.
+ *
+ * The function searches the list of canonical partition schemes for one that
+ * exactly matches the partitioning properties of the given relation. If it
+ * does not find one, the function creates a canonical partition scheme
+ * structure and adds it to the list.
+ *
+ * For an unpartitioned table, it returns NULL.
+ */
+
+extern PartitionScheme
+find_partition_scheme(PlannerInfo *root, Relation relation)
+{
+	PartitionKey	part_key = RelationGetPartitionKey(relation);
+	PartitionDesc	part_desc = RelationGetPartitionDesc(relation);
+	ListCell	   *lc;
+	int		nparts;
+	int		partnatts;
+	int		cnt_pks;
+	PartitionScheme	part_scheme = NULL;
+
+	/* No partition scheme for an unpartitioned relation. */
+	if (!part_desc || !part_key)
+		return NULL;
+
+	nparts = part_desc->nparts;
+	partnatts = part_key->partnatts;
+
+	/* Search for a matching partition scheme and return if found one. */
+	foreach (lc, root->part_schemes)
+	{
+		part_scheme = lfirst(lc);
+
+		/* Match number of partitions and partitioning strategy. */
+		if (nparts != part_scheme->nparts ||
+			part_key->strategy != part_scheme->strategy ||
+			partnatts != part_scheme->partnatts)
+			continue;
+
+		/* Match the partition key types. */
+		for (cnt_pks = 0; cnt_pks < partnatts; cnt_pks++)
+		{
+			/*
+			 * It suffices to check the OID of support function as it always has
+			 * two arguments and returns boolean. For types, it suffices to match
+			 * the type id, mod and collation; len, byval and align are depedent on
+			 * the first two.
+			 */
+			if (part_key->partopfamily[cnt_pks] != part_scheme->partopfamily[cnt_pks] ||
+				part_key->partopcintype[cnt_pks] != part_scheme->partopcintype[cnt_pks] ||
+				part_key->tcinfo->typid[cnt_pks] != part_scheme->key_types[cnt_pks] ||
+				part_key->tcinfo->typmod[cnt_pks] != part_scheme->key_typmods[cnt_pks] ||
+				part_key->tcinfo->typcoll[cnt_pks] != part_scheme->key_collations[cnt_pks])
+				break;
+		}
+
+		/* Some partition key didn't match. Check next partitioning scheme. */
+		if (cnt_pks < partnatts)
+			continue;
+
+		if (!partition_bounds_equal(part_key, part_desc->bounds,
+									part_scheme->bounds, nparts))
+			continue;
+
+		/* Found matching partition scheme. */
+		return part_scheme;
+	}
+
+	/* Did not find matching partition scheme. Create one. */
+	part_scheme = (PartitionScheme) palloc0(sizeof(PartitionSchemeData));
+
+	/* Copy partition bounds/lists. */
+	part_scheme->nparts = part_desc->nparts;
+	part_scheme->strategy = part_key->strategy;
+	part_scheme->bounds = copy_bounds(part_desc->bounds, part_key,
+									  part_scheme->nparts);
+
+	/* Store partition key information. */
+	part_scheme->partnatts = part_key->partnatts;
+
+	part_scheme->partopfamily = (Oid *) palloc(sizeof(Oid) * partnatts);
+	part_scheme->partopcintype = (Oid *) palloc(sizeof(Oid) * partnatts);
+	part_scheme->key_types = (Oid *) palloc(sizeof(Oid) * partnatts);
+	part_scheme->key_typmods = (int32 *) palloc(sizeof(int32) * partnatts);
+	part_scheme->key_collations = (Oid *) palloc(sizeof(Oid) * partnatts);
+
+	for (cnt_pks = 0; cnt_pks < partnatts; cnt_pks++)
+	{
+		part_scheme->partopfamily[cnt_pks] = part_key->partopfamily[cnt_pks];
+		part_scheme->partopcintype[cnt_pks] = part_key->partopcintype[cnt_pks];
+		part_scheme->key_types[cnt_pks] = part_key->tcinfo->typid[cnt_pks];
+		part_scheme->key_typmods[cnt_pks] = part_key->tcinfo->typmod[cnt_pks];
+		part_scheme->key_collations[cnt_pks] = part_key->tcinfo->typcoll[cnt_pks];
+	}
+
+	/* Add the partitioning scheme to PlannerInfo. */
+	root->part_schemes = lappend(root->part_schemes, part_scheme);
+
+	return part_scheme;
+}
+
+/*
+ * build_baserel_partition_key_exprs
+ *		Collect partition key expressions for a given base relation.
+ *
+ * The function converts single column partition keys into corresponding Var
+ * nodes. It restamps Var nodes in partition key expressions by given varno.
+ * The partition key expressions are returned as an array of Lists to be stored
+ * in RelOptInfo of the base relation.
+ */
+extern List **
+build_baserel_partition_key_exprs(Relation relation, Index varno)
+{
+	PartitionKey	part_key = RelationGetPartitionKey(relation);
+	int		num_pkexprs;
+	int		cnt_pke;
+	List		  **partexprs;
+	ListCell	   *lc;
+
+	if (!part_key || part_key->partnatts <= 0)
+		return NULL;
+
+	num_pkexprs = part_key->partnatts;
+	partexprs = (List **) palloc(sizeof(List *) * num_pkexprs);
+	lc = list_head(part_key->partexprs);
+
+	for (cnt_pke = 0; cnt_pke < num_pkexprs; cnt_pke++)
+	{
+		AttrNumber attno = part_key->partattrs[cnt_pke];
+		Expr	  *pkexpr;
+
+		if (attno != InvalidAttrNumber)
+		{
+			/* Single column partition key is stored as a Var node. */
+			Form_pg_attribute att_tup;
+
+			if (attno < 0)
+				att_tup = SystemAttributeDefinition(attno,
+												relation->rd_rel->relhasoids);
+			else
+				att_tup = relation->rd_att->attrs[attno - 1];
+
+			pkexpr = (Expr *) makeVar(varno, attno, att_tup->atttypid,
+									  att_tup->atttypmod,
+									  att_tup->attcollation, 0);
+		}
+		else
+		{
+			if (lc == NULL)
+				elog(ERROR, "wrong number of partition key expressions");
+
+			/* Re-stamp the expressions with given varno. */
+			pkexpr = (Expr *) copyObject(lfirst(lc));
+			ChangeVarNodes((Node *) pkexpr, 1, varno, 0);
+			lc = lnext(lc);
+		}
+
+		partexprs[cnt_pke] = list_make1(pkexpr);
+	}
+
+	return partexprs;
+}
diff --git a/src/backend/foreign/foreign.c b/src/backend/foreign/foreign.c
index 242d6d2..75c95e4 100644
--- a/src/backend/foreign/foreign.c
+++ b/src/backend/foreign/foreign.c
@@ -721,7 +721,7 @@ GetExistingLocalJoinPath(RelOptInfo *joinrel)
 {
 	ListCell   *lc;
 
-	Assert(joinrel->reloptkind == RELOPT_JOINREL);
+	Assert(IS_JOIN_REL(joinrel));
 
 	foreach(lc, joinrel->pathlist)
 	{
@@ -786,7 +786,7 @@ GetExistingLocalJoinPath(RelOptInfo *joinrel)
 			ForeignPath *foreign_path;
 
 			foreign_path = (ForeignPath *) joinpath->outerjoinpath;
-			if (foreign_path->path.parent->reloptkind == RELOPT_JOINREL)
+			if (IS_JOIN_REL(foreign_path->path.parent))
 				joinpath->outerjoinpath = foreign_path->fdw_outerpath;
 		}
 
@@ -795,7 +795,7 @@ GetExistingLocalJoinPath(RelOptInfo *joinrel)
 			ForeignPath *foreign_path;
 
 			foreign_path = (ForeignPath *) joinpath->innerjoinpath;
-			if (foreign_path->path.parent->reloptkind == RELOPT_JOINREL)
+			if (IS_JOIN_REL(foreign_path->path.parent))
 				joinpath->innerjoinpath = foreign_path->fdw_outerpath;
 		}
 
diff --git a/src/backend/optimizer/README b/src/backend/optimizer/README
index 775bcc3..f203dc5 100644
--- a/src/backend/optimizer/README
+++ b/src/backend/optimizer/README
@@ -974,3 +974,56 @@ be desirable to postpone the Gather stage until as near to the top of the
 plan as possible.  Expanding the range of cases in which more work can be
 pushed below the Gather (and costing them accurately) is likely to keep us
 busy for a long time to come.
+
+Partition-wise 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 for a given row in a given partition of a given
+partitioned table, its joining row, if exists, should exist only in the
+matching partition of the other partitioned table; no row from non-matching
+partitions in the other partitioned table can join with the given row from the
+first table. This condition allows the join between partitioned table 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
+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 partition tables
+into join between their partitions is called partition-wise join. We will use
+term "partitioned relation" for both partitioned table as well as join between
+partitioned tables which can use partition-wise join technique.
+
+Partitioning properties of a partitioned table are stored in
+PartitionSchemeData structure. Planner maintains a list of canonical partition
+schemes (distinct PartitionSchemeData objects) so that any two partitioned
+relations with same partitioning scheme share the same PartitionSchemeData
+object. This reduces memory consumed by PartitionSchemeData objects and makes
+it easy to compare the partition schemes of joining relations. RelOptInfos of
+partitioned relations hold partition key expressions and the RelOptInfos of
+the partition relations of that relation.
+
+Partition-wise joins are planned in two phases
+
+1. First phase creates the RelOptInfos for joins between matching partitions
+and creates join paths for those using the same techniques described above.
+The number of paths created for a child-join i.e. join between partitions is
+same as the number of paths created for join between parents. That number grows
+exponentially with the number of base relations being joined. The time and
+memory consumed to create paths for each child-join will be proporional to the
+number of partitions. This will not scale well with thousands of partitions.
+Instead of that we estimate partition-wise join cost based on the costs of
+sampled child-joins. We choose child-joins with higher sizes to have realistic
+estimates. If the number of sampled child-joins is same as the number of live
+child-joins, we create append paths as we know costs of all required
+child-joins. Otherwise we create PartitionJoinPaths with cost estimates based
+on the costs of sampled child-joins. While creating append paths or
+PartitionJoin paths we create paths for all the different possible
+parameterizations and pathkeys available in the sampled child-joins.
+
+2. If PartitionJoinPath emerges as the best possible path, we create paths for
+each unsampled child-join and choose the cheapest one with same
+parameterization or pathkeys as the PartitionJoinPath. This path is converted
+into a plan and all the child-join plans are combined using an Append or
+MergeAppend plan as appropriate. We use a fresh memory context for planning
+each unsampled child-join, thus reducing memory consumption.
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 99b6bc8..fbacb3c 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -18,8 +18,10 @@
 #include <limits.h>
 #include <math.h>
 
+#include "miscadmin.h"
 #include "access/sysattr.h"
 #include "access/tsmapi.h"
+#include "catalog/partition.h"
 #include "catalog/pg_class.h"
 #include "catalog/pg_operator.h"
 #include "catalog/pg_proc.h"
@@ -44,6 +46,7 @@
 #include "parser/parsetree.h"
 #include "rewrite/rewriteManip.h"
 #include "utils/lsyscache.h"
+#include "utils/rel.h"
 
 
 /* results of subquery_is_pushdown_safe */
@@ -93,8 +96,8 @@ static void set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 static void set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 						Index rti, RangeTblEntry *rte);
 static void generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
-						   List *live_childrels,
-						   List *all_child_pathkeys);
+						   List *live_childrels, List *all_child_pathkeys,
+						   bool partition_join_path);
 static Path *get_cheapest_parameterized_child_path(PlannerInfo *root,
 									  RelOptInfo *rel,
 									  Relids required_outer);
@@ -126,6 +129,8 @@ static void subquery_push_qual(Query *subquery,
 static void recurse_push_qual(Node *setOp, Query *topquery,
 				  RangeTblEntry *rte, Index rti, Node *qual);
 static void remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel);
+static void add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
+								List *live_childrels, bool partition_join_path);
 
 
 /*
@@ -868,6 +873,30 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 	double	   *parent_attrsizes;
 	int			nattrs;
 	ListCell   *l;
+	Oid		   *part_oids = NULL;
+	int			nparts = 0;
+
+	/* Fetch the number of partitions of a partitioned table and their Oids. */
+	if (rel->part_scheme)
+	{
+		RangeTblEntry	*rte = root->simple_rte_array[rel->relid];
+
+		/*
+		 * We need not lock the relation since it was already locked, either by
+		 * the rewriter or when expand_inherited_rtentry() added it to the
+		 * query's rangetable.
+		 */
+		Relation	relation = heap_open(rte->relid, NoLock);
+		PartitionDesc	part_desc = RelationGetPartitionDesc(relation);
+
+		part_oids = part_desc->oids;
+		nparts = part_desc->nparts;
+
+		Assert(part_oids && nparts > 0);
+
+		rel->part_rels = (RelOptInfo **)palloc0(sizeof(RelOptInfo *) * nparts);
+		heap_close(relation, NoLock);
+	}
 
 	/*
 	 * Initialize to compute size estimates for whole append relation.
@@ -899,6 +928,7 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		Node	   *childqual;
 		ListCell   *parentvars;
 		ListCell   *childvars;
+		int			cnt_parts;
 
 		/* append_rel_list contains all append rels; ignore others */
 		if (appinfo->parent_relid != parentRTindex)
@@ -912,8 +942,101 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		 * add_base_rels_to_query.
 		 */
 		childrel = find_base_rel(root, childRTindex);
+
+		/*
+		 * Recursively save topmost parent's relid in RelOptInfos of
+		 * partitions.
+		 */
+		if (rel->top_parent_relids)
+			childrel->top_parent_relids = rel->top_parent_relids;
+		else
+			childrel->top_parent_relids = bms_copy(rel->relids);
+
 		Assert(childrel->reloptkind == RELOPT_OTHER_MEMBER_REL);
 
+
+		/*
+		 * For two partitioned tables with the same partitioning scheme, it is
+		 * assumed that the Oids of matching partitions from both the tables
+		 * are placed at the same position in the array of partition oids in
+		 * respective partition descriptors. Saving the RelOptInfo of a
+		 * partition in the same location as its Oid makes it easy to find the
+		 * RelOptInfos of matching partitions for partition-wise join.
+		 */
+		for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+		{
+			if (part_oids[cnt_parts] == childRTE->relid)
+			{
+				Assert(!rel->part_rels[cnt_parts]);
+				rel->part_rels[cnt_parts] = childrel;
+			}
+		}
+
+		/*
+		 * Note: we could compute appropriate attr_needed data for the child's
+		 * variables, by transforming the parent's attr_needed through the
+		 * translated_vars mapping.  However, currently there's no need
+		 * because attr_needed is only examined for base relations not
+		 * otherrels.  So we just leave the child's attr_needed empty.
+		 * For a partitioned tables, individual partitions can participate in
+		 * the pair-wise joins. We need attr_needed data for buiding pair-wise
+		 * join relations. Partition tables should have same layout as the
+		 * parent table and hence should not need any translation. But rest of
+		 * the code still uses inheritance mechanism. So do we here.
+		 */
+		if (rel->part_scheme)
+		{
+			AttrNumber attno;
+			for (attno = rel->min_attr; attno <= rel->max_attr; attno++)
+			{
+				int	index = attno - rel->min_attr;
+				Relids	attr_needed = bms_copy(rel->attr_needed[index]);
+
+				/*
+				 * System attributes do not need translation. In such a case,
+				 * the attribute numbers of the parent and the child should
+				 * start from the same minimum attribute.
+				 */
+				if (attno <= 0)
+				{
+					Assert(rel->min_attr == childrel->min_attr);
+					childrel->attr_needed[index] = attr_needed;
+				}
+				else
+				{
+					Var *var = list_nth(appinfo->translated_vars,
+										attno - 1);
+					int child_index;
+
+					/* Parent Var translates to child Var. */
+					Assert(IsA(var, Var));
+
+					child_index = var->varattno - childrel->min_attr;
+					childrel->attr_needed[child_index] = attr_needed;
+				}
+			}
+		}
+
+		/*
+		 * Copy/Modify targetlist. Partition-wise join technique may join this
+		 * child with a child of another partitioned table, such that this
+		 * child forms the nullable side of the outer join. In such a case, we
+		 * will need the targetlist of this child, even if it's deemed empty.
+		 * Hence set the targetlist before bailing out in case the child is
+		 * proven empty.
+		 *
+		 * NB: the resulting childrel->reltarget->exprs may contain arbitrary
+		 * expressions, which otherwise would not occur in a rel's targetlist.
+		 * Code that might be looking at an appendrel child must cope with
+		 * such.  (Normally, a rel's targetlist would only include Vars and
+		 * PlaceHolderVars.)  XXX we do not bother to update the cost or width
+		 * fields of childrel->reltarget; not clear if that would be useful.
+		 */
+		childrel->reltarget->exprs = (List *)
+			adjust_appendrel_attrs(root,
+								   (Node *) rel->reltarget->exprs,
+								   list_make1(appinfo));
+
 		/*
 		 * We have to copy the parent's targetlist and quals to the child,
 		 * with appropriate substitution of variables.  However, only the
@@ -931,7 +1054,7 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		childquals = get_all_actual_clauses(rel->baserestrictinfo);
 		childquals = (List *) adjust_appendrel_attrs(root,
 													 (Node *) childquals,
-													 appinfo);
+													 list_make1(appinfo));
 		childqual = eval_const_expressions(root, (Node *)
 										   make_ands_explicit(childquals));
 		if (childqual && IsA(childqual, Const) &&
@@ -960,24 +1083,11 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 			continue;
 		}
 
-		/*
-		 * CE failed, so finish copying/modifying targetlist and join quals.
-		 *
-		 * NB: the resulting childrel->reltarget->exprs may contain arbitrary
-		 * expressions, which otherwise would not occur in a rel's targetlist.
-		 * Code that might be looking at an appendrel child must cope with
-		 * such.  (Normally, a rel's targetlist would only include Vars and
-		 * PlaceHolderVars.)  XXX we do not bother to update the cost or width
-		 * fields of childrel->reltarget; not clear if that would be useful.
-		 */
+		/* CE failed, so finish copying/modifying join quals. */
 		childrel->joininfo = (List *)
 			adjust_appendrel_attrs(root,
 								   (Node *) rel->joininfo,
-								   appinfo);
-		childrel->reltarget->exprs = (List *)
-			adjust_appendrel_attrs(root,
-								   (Node *) rel->reltarget->exprs,
-								   appinfo);
+								   list_make1(appinfo));
 
 		/*
 		 * We have to make child entries in the EquivalenceClass data
@@ -992,14 +1102,6 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		childrel->has_eclass_joins = rel->has_eclass_joins;
 
 		/*
-		 * Note: we could compute appropriate attr_needed data for the child's
-		 * variables, by transforming the parent's attr_needed through the
-		 * translated_vars mapping.  However, currently there's no need
-		 * because attr_needed is only examined for base relations not
-		 * otherrels.  So we just leave the child's attr_needed empty.
-		 */
-
-		/*
 		 * If parallelism is allowable for this query in general, see whether
 		 * it's allowable for this childrel in particular.  But if we've
 		 * already decided the appendrel is not parallel-safe as a whole,
@@ -1080,6 +1182,16 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		}
 	}
 
+	/* Should have found all the childrels of a partitioned relation. */
+	if (rel->part_scheme)
+	{
+		int		cnt_parts;
+		for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+			if (!rel->part_rels[cnt_parts])
+				elog(ERROR, "could not find the RelOptInfo of a partition with oid %u",
+					 part_oids[cnt_parts]);
+	}
+
 	if (has_live_children)
 	{
 		/*
@@ -1122,19 +1234,11 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 {
 	int			parentRTindex = rti;
 	List	   *live_childrels = NIL;
-	List	   *subpaths = NIL;
-	bool		subpaths_valid = true;
-	List	   *partial_subpaths = NIL;
-	bool		partial_subpaths_valid = true;
-	List	   *all_child_pathkeys = NIL;
-	List	   *all_child_outers = NIL;
 	ListCell   *l;
 
 	/*
-	 * Generate access paths for each member relation, and remember the
-	 * cheapest path for each one.  Also, identify all pathkeys (orderings)
-	 * and parameterizations (required_outer sets) available for the member
-	 * relations.
+	 * Generate access paths for each member relation and remember the
+	 * non-dummy children.
 	 */
 	foreach(l, root->append_rel_list)
 	{
@@ -1142,7 +1246,6 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 		int			childRTindex;
 		RangeTblEntry *childRTE;
 		RelOptInfo *childrel;
-		ListCell   *lcp;
 
 		/* append_rel_list contains all append rels; ignore others */
 		if (appinfo->parent_relid != parentRTindex)
@@ -1177,6 +1280,64 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 		 * Child is live, so add it to the live_childrels list for use below.
 		 */
 		live_childrels = lappend(live_childrels, childrel);
+	}
+
+	/* Add Append/MergeAppend paths to the "append" relation. */
+	add_paths_to_append_rel(root, rel, live_childrels, false);
+}
+
+/*
+ * add_paths_to_append_rel
+ *		Generate Append/MergeAppend paths for given "append" relation. An
+ *		"append" relation can be a base parent relation or a join between
+ *		partitioned tables.
+ *
+ * The function collects all parameterizations and orderings supported by the
+ * non-dummy children. For every such parameterization or ordering, it creates
+ * an append path collecting one path from each non-dummy child with given
+ * parameterization or ordering. Similarly it collects partial paths from
+ * non-dummy children to create partial append paths.
+ */
+static void
+add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
+						List *live_childrels, bool partition_join_path)
+{
+	List	   *subpaths = NIL;
+	bool		subpaths_valid = true;
+	List	   *partial_subpaths = NIL;
+	bool		partial_subpaths_valid;
+	List	   *all_child_pathkeys = NIL;
+	List	   *all_child_outers = NIL;
+	ListCell   *l;
+
+	/*
+	 * While creating PartitionJoinPath, we sample paths from only a few child
+	 * relations. Even if all of sampled children have partial paths, it's not
+	 * guaranteed that all the unsampled children will have partial paths.
+	 * Hence we do not create partial PartitionJoinPaths.
+	 */
+	partial_subpaths_valid = !partition_join_path ? true : false;
+
+	/* An append relation with all its children dummy is dummy. */
+	if (live_childrels == NIL)
+	{
+		/* Mark the relation as dummy, if not already done so. */
+		if (!IS_DUMMY_REL(rel))
+			set_dummy_rel_pathlist(rel);
+
+		/* No more paths need to be added. */
+		return;
+	}
+
+	/*
+	 * For every non-dummy child, remember the cheapest path.  Also, identify
+	 * all pathkeys (orderings) and parameterizations (required_outer sets)
+	 * available for the non-dummy member relations.
+	 */
+	foreach (l, live_childrels)
+	{
+		RelOptInfo *childrel = lfirst(l);
+		ListCell   *lcp;
 
 		/*
 		 * If child has an unparameterized cheapest-total path, add that to
@@ -1267,7 +1428,17 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 	 * if we have zero or one live subpath due to constraint exclusion.)
 	 */
 	if (subpaths_valid)
-		add_path(rel, (Path *) create_append_path(rel, subpaths, NULL, 0));
+	{
+		Path *path;
+
+		if (partition_join_path)
+			path = (Path *) create_partition_join_path(rel, subpaths,
+													   NULL);
+		else
+			path = (Path *) create_append_path(rel, subpaths, NULL, 0);
+
+		add_path(rel, path);
+	}
 
 	/*
 	 * Consider an append of partial unordered, unparameterized partial paths.
@@ -1278,6 +1449,8 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 		ListCell   *lc;
 		int			parallel_workers = 0;
 
+		Assert(!partition_join_path);
+
 		/*
 		 * Decide on the number of workers to request for this append path.
 		 * For now, we just use the maximum value from among the members.  It
@@ -1304,7 +1477,7 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 	 */
 	if (subpaths_valid)
 		generate_mergeappend_paths(root, rel, live_childrels,
-								   all_child_pathkeys);
+								   all_child_pathkeys, partition_join_path);
 
 	/*
 	 * Build Append paths for each parameterization seen among the child rels.
@@ -1345,8 +1518,16 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 		}
 
 		if (subpaths_valid)
-			add_path(rel, (Path *)
-					 create_append_path(rel, subpaths, required_outer, 0));
+		{
+			Path *path;
+
+			if (partition_join_path)
+				path = (Path *) create_partition_join_path(rel, subpaths, required_outer);
+			else
+				path = (Path *) create_append_path(rel, subpaths, required_outer, 0);
+
+			add_path(rel, path);
+		}
 	}
 }
 
@@ -1376,7 +1557,7 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 static void
 generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
 						   List *live_childrels,
-						   List *all_child_pathkeys)
+						   List *all_child_pathkeys, bool partition_join_path)
 {
 	ListCell   *lcp;
 
@@ -1387,6 +1568,7 @@ generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
 		List	   *total_subpaths = NIL;
 		bool		startup_neq_total = false;
 		ListCell   *lcr;
+		Path	   *path;
 
 		/* Select the child paths for this ordering... */
 		foreach(lcr, live_childrels)
@@ -1434,17 +1616,29 @@ generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
 		}
 
 		/* ... and build the MergeAppend paths */
-		add_path(rel, (Path *) create_merge_append_path(root,
-														rel,
-														startup_subpaths,
-														pathkeys,
-														NULL));
+		if (partition_join_path)
+			path = (Path *) create_partition_join_path_with_pathkeys(root, rel,
+															 startup_subpaths,
+															 pathkeys, NULL);
+		else
+			path = (Path *) create_merge_append_path(root, rel,
+													 startup_subpaths,
+													 pathkeys, NULL);
+		add_path(rel, path);
+
 		if (startup_neq_total)
-			add_path(rel, (Path *) create_merge_append_path(root,
-															rel,
-															total_subpaths,
-															pathkeys,
-															NULL));
+		{
+			if (partition_join_path)
+				path = (Path *) create_partition_join_path_with_pathkeys(root,
+																rel,
+																total_subpaths,
+																pathkeys, NULL);
+			else
+				path = (Path *) create_merge_append_path(root, rel,
+														 total_subpaths,
+														 pathkeys, NULL);
+			add_path(rel, path);
+		}
 	}
 }
 
@@ -2188,6 +2382,10 @@ standard_join_search(PlannerInfo *root, int levels_needed, List *initial_rels)
 		 * Run generate_gather_paths() for each just-processed joinrel.  We
 		 * could not do this earlier because both regular and partial paths
 		 * can get added to a particular joinrel at multiple times within
+		 * join_search_one_level.  Similarly, create append paths for joinrels
+		 * which used partition-wise join technique.  We can not do this
+		 * earlier because the paths can get added to a relation representing
+		 * join between children at multiple times within
 		 * join_search_one_level.  After that, we're done creating paths for
 		 * the joinrel, so run set_cheapest().
 		 */
@@ -2195,6 +2393,9 @@ standard_join_search(PlannerInfo *root, int levels_needed, List *initial_rels)
 		{
 			rel = (RelOptInfo *) lfirst(lc);
 
+			/* Create Append/MergeAppend paths for partition-wise joins. */
+			generate_partition_wise_join_paths(root, rel);
+
 			/* Create GatherPaths for any useful partial paths for rel */
 			generate_gather_paths(root, rel);
 
@@ -2858,6 +3059,160 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
 	}
 }
 
+/* Fraction of to base cost on. Probably we should turn this into a GUC? */
+#define FRACTION_PARTS_TO_PLAN 0.01
+
+/*
+ * generate_partition_wise_join_paths
+ * 		Create paths representing partition-wise join for given partitioned
+ * 		join relation.
+ *
+ * The number of paths created for a child-join is same as the number of paths
+ * created for join between parents. That number grows exponentially with the
+ * number of base relations being joined. The time and memory consumed to
+ * create paths for each child-join will be proporional to the number of
+ * partitions. This will not scale well with thousands of partitions. Instead
+ * of that we estimate partition-wise join cost based on the costs of sampled
+ * child-joins. We choose child-joins with higher sizes to have realistic
+ * estimates.
+ *
+ * This must be called after we have considered all joining orders for
+ * parent-join, and required child-joins (Otherwise, add_path might delete a
+ * path that some Append/MergeAppend path has a reference to.)
+ */
+void
+generate_partition_wise_join_paths(PlannerInfo *root, RelOptInfo *rel)
+{
+	List   *sampled_children = NIL;
+	List   *sampled_child_nos = NIL;
+	int		cnt_part;
+	int		num_part_to_plan;
+	int		num_parts;
+	bool	partition_join_path = false;
+	int		num_dummy_parts;
+	ListCell   *lc;
+
+	/* Handle only join relations. */
+	if (!IS_JOIN_REL(rel))
+		return;
+
+	/*
+	 * If partition-wise join technique was not used for any of the join
+	 * orders, the join is not partitioned. Reset the partitioning scheme.
+	 */
+	if (!rel->part_rels)
+		rel->part_scheme = NULL;
+
+	/* If the relation is not partitioned or is proven dummy, nothing to do. */
+	if (!rel->part_scheme || IS_DUMMY_REL(rel))
+		return;
+
+	/* Guard against stack overflow due to overly deep partition hierarchy. */
+	check_stack_depth();
+
+	num_parts = rel->part_scheme->nparts;
+
+	/* Calculate number of child-joins to sample. */
+	num_part_to_plan = num_parts * FRACTION_PARTS_TO_PLAN;
+	if (num_part_to_plan < 1)
+		num_part_to_plan = 1;
+
+	/* Sample the child-joins with higher sizes. */
+	for (cnt_part = 0; cnt_part < num_parts; cnt_part++)
+	{
+		RelOptInfo *child_rel = rel->part_rels[cnt_part];
+		ListCell   *insert_after;
+
+		if (IS_DUMMY_REL(child_rel))
+		{
+			num_dummy_parts++;
+			continue;
+		}
+
+		insert_after = NULL;
+
+		/*
+		 * Add this relation to the list of samples ordered by the increasing
+		 * number of rows at appropriate place.
+		 */
+		foreach (lc, sampled_child_nos)
+		{
+			int	child_no = lfirst_int(lc);
+			RelOptInfo *old_childrel = rel->part_rels[child_no];
+
+			/*
+			 * Keep track of child with lowest number of rows but higher than the
+			 * that of the child being inserted. Insert the child before a
+			 * child with highest number of rows lesser than it.
+			 */
+			if (child_rel->rows <= old_childrel->rows)
+				insert_after = lc;
+			else
+				break;
+		}
+
+		/*
+		 * If we have collected required number of child-joins and current
+		 * child-join has lesser number of rows than all the child-joins
+		 * collected so far, ignore it in this phase.
+		 */
+		if (insert_after == list_tail(sampled_child_nos) &&
+			list_length(sampled_child_nos) == num_part_to_plan)
+			continue;
+
+		if (insert_after)
+			lappend_cell_int(sampled_child_nos, insert_after, cnt_part);
+		else
+			sampled_child_nos = lcons_int(cnt_part, sampled_child_nos);
+
+		/* Trim down list to the required number of children. */
+		if (list_length(sampled_child_nos) > num_part_to_plan)
+		{
+			/*
+			 * List is trimmed every time it grows after adding one child. So,
+			 * it can have at most one extra element.
+			 */
+			Assert(list_length(sampled_child_nos) == num_part_to_plan + 1);
+
+			list_delete_cell(sampled_child_nos,
+							 list_nth_cell(sampled_child_nos,
+										   list_length(sampled_child_nos) - 1),
+							 list_nth_cell(sampled_child_nos,
+										   list_length(sampled_child_nos) - 2));
+		}
+
+		Assert(list_length(sampled_child_nos) <= num_part_to_plan);
+	}
+
+	/* Create paths for all the sampled child-joins. */
+	foreach (lc, sampled_child_nos)
+	{
+		int	child_no = lfirst_int(lc);
+
+		/* Create paths for this child. */
+		add_paths_to_child_joinrel(root, rel, child_no);
+
+#ifdef OPTIMIZER_DEBUG
+		debug_print_rel(root, rel);
+#endif
+		sampled_children = lappend(sampled_children, rel->part_rels[child_no]);
+	}
+
+	/*
+	 * If the number of samples is same as the number of live children, an
+	 * append path will do. Otherwise, we will cost the partition-wise join
+	 * based on the sampled children using PartitionJoinPath.
+	 */
+	if (num_part_to_plan < num_parts - num_dummy_parts)
+		partition_join_path = true;
+
+	/* Add paths for partition-wise join based on the sampled children. */
+	add_paths_to_append_rel(root, rel, sampled_children, partition_join_path);
+
+	if (sampled_children)
+		list_free(sampled_children);
+}
+
 /*****************************************************************************
  *			DEBUG SUPPORT
  *****************************************************************************/
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 2a49639..a23da1c 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -126,6 +126,7 @@ bool		enable_nestloop = true;
 bool		enable_material = true;
 bool		enable_mergejoin = true;
 bool		enable_hashjoin = true;
+bool		enable_partition_wise_join = true;
 
 typedef struct
 {
diff --git a/src/backend/optimizer/path/equivclass.c b/src/backend/optimizer/path/equivclass.c
index 0e50ad5..fc40b69 100644
--- a/src/backend/optimizer/path/equivclass.c
+++ b/src/backend/optimizer/path/equivclass.c
@@ -2062,7 +2062,7 @@ add_child_rel_equivalences(PlannerInfo *root,
 				child_expr = (Expr *)
 					adjust_appendrel_attrs(root,
 										   (Node *) cur_em->em_expr,
-										   appinfo);
+										   list_make1(appinfo));
 
 				/*
 				 * Transform em_relids to match.  Note we do *not* do
@@ -2366,6 +2366,8 @@ eclass_useful_for_merging(PlannerInfo *root,
 	/* If specified rel is a child, we must consider the topmost parent rel */
 	if (rel->reloptkind == RELOPT_OTHER_MEMBER_REL)
 		relids = find_childrel_top_parent(root, rel)->relids;
+	else if (rel->reloptkind == RELOPT_OTHER_JOINREL)
+		relids = rel->top_parent_relids;
 	else
 		relids = rel->relids;
 
diff --git a/src/backend/optimizer/path/joinpath.c b/src/backend/optimizer/path/joinpath.c
index cc7384f..fae15de 100644
--- a/src/backend/optimizer/path/joinpath.c
+++ b/src/backend/optimizer/path/joinpath.c
@@ -25,9 +25,19 @@
 /* Hook for plugins to get control in add_paths_to_joinrel() */
 set_join_pathlist_hook_type set_join_pathlist_hook = NULL;
 
-#define PATH_PARAM_BY_REL(path, rel)  \
+/*
+ * Paths parameterized by the parent can be considered to be parameterized by
+ * any of its child.
+ */
+#define PATH_PARAM_BY_PARENT(path, rel)	\
+	((path)->param_info && bms_overlap(PATH_REQ_OUTER(path),	\
+									   (rel)->top_parent_relids))
+#define PATH_PARAM_BY_REL_SELF(path, rel)  \
 	((path)->param_info && bms_overlap(PATH_REQ_OUTER(path), (rel)->relids))
 
+#define PATH_PARAM_BY_REL(path, rel)	\
+	(PATH_PARAM_BY_REL_SELF(path, rel) || PATH_PARAM_BY_PARENT(path, rel))
+
 static void sort_inner_and_outer(PlannerInfo *root, RelOptInfo *joinrel,
 					 RelOptInfo *outerrel, RelOptInfo *innerrel,
 					 JoinType jointype, JoinPathExtraData *extra);
@@ -132,6 +142,19 @@ add_paths_to_joinrel(PlannerInfo *root,
 	foreach(lc, root->join_info_list)
 	{
 		SpecialJoinInfo *sjinfo = (SpecialJoinInfo *) lfirst(lc);
+		Relids	joinrelids;
+
+		/*
+		 * PlannerInfo doesn't contain the SpecialJoinInfos created for joins
+		 * between child relations, even if there is a SpecialJoinInfo node for
+		 * the join between the topmost parents. Hence while calculating Relids
+		 * set representing the restriction, consider relids of topmost parent
+		 * of partitions.
+		 */
+		if (joinrel->reloptkind == RELOPT_OTHER_JOINREL)
+			joinrelids = joinrel->top_parent_relids;
+		else
+			joinrelids = joinrel->relids;
 
 		/*
 		 * SJ is relevant to this join if we have some part of its RHS
@@ -140,16 +163,16 @@ add_paths_to_joinrel(PlannerInfo *root,
 		 * join has already been proven legal.)  If the SJ is relevant, it
 		 * presents constraints for joining to anything not in its RHS.
 		 */
-		if (bms_overlap(joinrel->relids, sjinfo->min_righthand) &&
-			!bms_overlap(joinrel->relids, sjinfo->min_lefthand))
+		if (bms_overlap(joinrelids, sjinfo->min_righthand) &&
+			!bms_overlap(joinrelids, sjinfo->min_lefthand))
 			extra.param_source_rels = bms_join(extra.param_source_rels,
 										   bms_difference(root->all_baserels,
 													 sjinfo->min_righthand));
 
 		/* full joins constrain both sides symmetrically */
 		if (sjinfo->jointype == JOIN_FULL &&
-			bms_overlap(joinrel->relids, sjinfo->min_lefthand) &&
-			!bms_overlap(joinrel->relids, sjinfo->min_righthand))
+			bms_overlap(joinrelids, sjinfo->min_lefthand) &&
+			!bms_overlap(joinrelids, sjinfo->min_righthand))
 			extra.param_source_rels = bms_join(extra.param_source_rels,
 										   bms_difference(root->all_baserels,
 													  sjinfo->min_lefthand));
@@ -279,6 +302,24 @@ try_nestloop_path(PlannerInfo *root,
 	JoinCostWorkspace workspace;
 
 	/*
+	 * For a join between child relations, if the inner path is parameterized
+	 * by the parent of the outer relation, it can be considered to be
+	 * parameterized by the outer relation. We will be able to create a
+	 * nestloop join path with inner relation parameterized by the outer
+	 * relation by translating the inner path to be parameterized by the outer
+	 * child relation.
+	 */
+	if (PATH_PARAM_BY_PARENT(inner_path, outer_path->parent))
+	{
+		inner_path = reparameterize_path_by_child(root, inner_path,
+												   outer_path->parent);
+
+		/* If we could not translate the path, don't produce nest loop path. */
+		if (!inner_path)
+			return;
+	}
+
+	/*
 	 * Check to see if proposed path is still parameterized, and reject if the
 	 * parameterization wouldn't be sensible --- unless allow_star_schema_join
 	 * says to allow it anyway.  Also, we must reject if have_dangerous_phv
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 01d4fea..b4220eb 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -14,9 +14,14 @@
  */
 #include "postgres.h"
 
+#include "miscadmin.h"
+#include "catalog/partition.h"
+#include "optimizer/clauses.h"
 #include "optimizer/joininfo.h"
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
+#include "optimizer/prep.h"
+#include "optimizer/cost.h"
 #include "utils/memutils.h"
 
 
@@ -32,7 +37,20 @@ static bool is_dummy_rel(RelOptInfo *rel);
 static void mark_dummy_rel(RelOptInfo *rel);
 static bool restriction_is_constant_false(List *restrictlist,
 							  bool only_pushed_down);
-
+static void populate_joinrel_with_paths(PlannerInfo *root, RelOptInfo *rel1,
+							RelOptInfo *rel2, RelOptInfo *joinrel,
+							SpecialJoinInfo *sjinfo, List *restrictlist);
+static void try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1,
+						  RelOptInfo *rel2, RelOptInfo *joinrel,
+						  SpecialJoinInfo *parent_sjinfo,
+						  List *parent_restrictlist);
+static SpecialJoinInfo * build_child_join_sjinfo(PlannerInfo *root,
+										 SpecialJoinInfo *parent_sjinfo,
+										 List *append_rel_infos1,
+										 List *append_rel_infos2);
+static bool have_partkey_equi_join(RelOptInfo *rel1, RelOptInfo *rel2,
+						   JoinType jointype, List *restrictlist);
+static int match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel);
 
 /*
  * join_search_one_level
@@ -724,6 +742,30 @@ make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 		return joinrel;
 	}
 
+	/* Add paths to the join relation. */
+	populate_joinrel_with_paths(root, rel1, rel2, joinrel, sjinfo,
+								restrictlist);
+
+	/* Apply partition-wise join technique, if possible. */
+	try_partition_wise_join(root, rel1, rel2, joinrel, sjinfo, restrictlist);
+
+	bms_free(joinrelids);
+
+	return joinrel;
+}
+
+/*
+ * populate_joinrel_with_paths
+ *	  Add paths joining given input relations to the given joinrel. The
+ *	  SpecialJoinInfo provides details about the join and the restrictlist
+ *	  contains the join clauses and the other clauses applicable for given pair
+ *	  of the joining relations.
+ */
+static void
+populate_joinrel_with_paths(PlannerInfo *root, RelOptInfo *rel1,
+							RelOptInfo *rel2, RelOptInfo *joinrel,
+							SpecialJoinInfo *sjinfo, List *restrictlist)
+{
 	/*
 	 * Consider paths using each rel as both outer and inner.  Depending on
 	 * the join type, a provably empty outer or inner rel might mean the join
@@ -868,13 +910,8 @@ make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 			elog(ERROR, "unrecognized join type: %d", (int) sjinfo->jointype);
 			break;
 	}
-
-	bms_free(joinrelids);
-
-	return joinrel;
 }
 
-
 /*
  * have_join_order_restriction
  *		Detect whether the two relations should be joined to satisfy
@@ -1249,3 +1286,462 @@ restriction_is_constant_false(List *restrictlist, bool only_pushed_down)
 	}
 	return false;
 }
+
+/*
+ * Assess whether join between given two partitioned relations can be broken
+ * down into joins between matching partitions; a technique called
+ * "partition-wise join"
+ *
+ * Partition-wise 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.
+ *
+ * Partition-wise join is planned in three phases (details: optimizer/README.)
+ *
+ * 1. Create the RelOptInfos for joins between matching partitions i.e
+ * child-joins and estimate sizes of those. This function is responsible for
+ * this phase.
+ *
+ * 2. Add paths representing partition-wise join. The second phase is
+ * implemented by generate_partition_wise_join_paths(). In order to save time
+ * and memory consumed in creating paths for every child-join, we create paths
+ * for only few child-joins. Paths for remaining child-joins are created in the
+ * next phase.
+ *
+ * 3. Create merge/append plan to combining plans for every child-join.
+ *
+ * The RelOptInfo, SpecialJoinInfo and restrictlist for each child join are
+ * obtained by translating the respective parent join structures.
+ */
+static void
+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;
+	PartitionedJoin	   *partitioned_join;
+
+	/* 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)
+		return;
+
+	/*
+	 * If any of the joining parent relations is proven empty, either the join
+	 * will be empty (INNER join) or will have the inner side all nullified. We
+	 * take care of such cases when creating join paths for parent relations.
+	 * Nothing to be done here. Also, nothing to do, if the parent join is
+	 * proven empty.
+	 */
+	if (IS_DUMMY_REL(rel1) || IS_DUMMY_REL(rel2) || IS_DUMMY_REL(joinrel))
+		return;
+
+	/*
+	 * Partitioning scheme in join relation indicates a possibilty that the
+	 * join may be partitioned, but it's not necessary that every pair of
+	 * joining relations can use partition-wise join technique. If one of
+	 * joining relations turns out to be unpartitioned, this pair of joining
+	 * relations can not use partition-wise join technique.
+	 */
+	if (!rel1->part_scheme || !rel2->part_scheme)
+		return;
+
+	/*
+	 * If an equi-join condition between the partition keys of the joining
+	 * relations does not exist, this pair of joining relations can not use
+	 * partition-wise technique.
+	 */
+	if (!have_partkey_equi_join(rel1, rel2, parent_sjinfo->jointype,
+								parent_restrictlist))
+		return;
+
+	/*
+	 * 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);
+
+	/* We should have RelOptInfos of the partitions available. */
+	Assert(rel1->part_rels && rel2->part_rels);
+
+	part_scheme = joinrel->part_scheme;
+	nparts = part_scheme->nparts;
+
+	/*
+	 * We do not store information about valid pairs of joining child
+	 * relations. The pair of joining relations for a child-join can be
+	 * derived by valid pairs of joining parent relations. Amongst the valid
+	 * pairs of parent joining relations, only those which result in
+	 * partitioned join matter for partition-wise join. Remember those so that
+	 * we can use them for creating paths child-joins for PartitionJoinPaths
+	 * later.
+	 */
+	partitioned_join = (PartitionedJoin *) palloc(sizeof(PartitionedJoin));
+	partitioned_join->rel1 = rel1;
+	partitioned_join->rel2 = rel2;
+	partitioned_join->restrictlist = parent_restrictlist;
+	partitioned_join->sjinfo = copyObject(parent_sjinfo);
+	joinrel->partitioned_joins = lappend(joinrel->partitioned_joins,
+										 partitioned_join);
+
+	elog(DEBUG3, "join between relations %s and %s is considered for partition-wise join.",
+		 bmsToString(rel1->relids), bmsToString(rel2->relids));
+
+	/* We are done if child RelOptInfos are already created. */
+	if (joinrel->part_rels)
+		return;
+
+	/* Create all the child RelOptInfos. */
+	joinrel->part_rels = (RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
+
+	/*
+	 * Create child join relations for this partitioned join. While doing so,
+	 * we estimate sizes of these child join relations. These estimates are
+	 * used to find the representative child relations used for costing the
+	 * partition-wise join later.
+	 */
+	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	{
+		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
+		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
+		SpecialJoinInfo	*child_sjinfo;
+		List	*child_restrictlist;
+		RelOptInfo *child_joinrel;
+		List	   *join_appinfos;
+		List	   *appinfos1;
+		List	   *appinfos2;
+
+		/* We should never try to join two overlapping sets of rels. */
+		Assert(!bms_overlap(child_rel1->relids, child_rel2->relids));
+
+		Assert (!joinrel->part_rels[cnt_parts]);
+
+		child_joinrel = build_child_join_rel(root, child_rel1, child_rel2,
+											 joinrel, parent_sjinfo->jointype);
+
+		joinrel->part_rels[cnt_parts] = child_joinrel;
+
+		/* Get parent-child mapping for translating nodes. */
+		appinfos1 = find_appinfos_by_relids(root, child_rel1->relids);
+		appinfos2 = find_appinfos_by_relids(root, child_rel2->relids);
+		join_appinfos = list_concat(appinfos1, appinfos2);
+
+		/*
+		 * Construct restrictions applicable to the child join from
+		 * those applicable to the parent join.
+		 */
+		child_restrictlist = (List *) adjust_join_appendrel_attrs(root,
+												   (Node *)parent_restrictlist,
+																join_appinfos);
+
+		/*
+		 * Construct SpecialJoinInfo from parent join relations's
+		 * SpecialJoinInfo.
+		 */
+		child_sjinfo = build_child_join_sjinfo(root, parent_sjinfo, appinfos1,
+											   appinfos2);
+
+		/*
+		 * Set estimates of the child-joinrel's size.
+		 */
+		set_joinrel_size_estimates(root, child_joinrel, child_rel1, child_rel2,
+								   child_sjinfo, child_restrictlist);
+
+		/* The list is not needed anymore. */
+		list_free(join_appinfos);
+
+		/*
+		 * If the child relations themselves are partitioned, try partition-wise join
+		 * recursively.
+		 */
+		try_partition_wise_join(root, child_rel1, child_rel2, child_joinrel,
+								child_sjinfo, child_restrictlist);
+	}
+}
+
+/*
+ * add_paths_to_child_join
+ *
+ * 		Add paths to 'child_id'th child of given parent join relation.
+ *
+ * The function creates paths for given child join by considering join between
+ * corresponding children of every pair of joining parent relation which
+ * produces partitioned join. Since we create paths only for sampled
+ * child-joins, either of the children being joined may not have paths. In that
+ * case, this function is called recursively to populate paths for those.
+ */
+void
+add_paths_to_child_joinrel(PlannerInfo *root, RelOptInfo *parent_joinrel,
+						   int child_id)
+{
+	ListCell	*lc;
+	RelOptInfo	   *child_joinrel = parent_joinrel->part_rels[child_id];
+
+	Assert(IS_JOIN_REL(parent_joinrel));
+
+	/* If this child relation already has paths, nothing to do. */
+	if (child_joinrel->cheapest_total_path)
+		return;
+
+	/* A dummy relation will have a dummy path as the cheapest path. */
+	Assert(!is_dummy_rel(child_joinrel));
+
+	/*
+	 * For every partitioned join order, calculate paths for the joining
+	 * child relations and then calculate paths for given child.
+	 */
+	foreach (lc, parent_joinrel->partitioned_joins)
+	{
+		PartitionedJoin	   *pj = lfirst(lc);
+		RelOptInfo *rel1 = pj->rel1;
+		RelOptInfo *rel2 = pj->rel2;
+		RelOptInfo *child_rel1 = rel1->part_rels[child_id];
+		RelOptInfo *child_rel2 = rel2->part_rels[child_id];
+		SpecialJoinInfo	   *child_sjinfo;
+		List	   *child_restrictlist;
+		List	   *join_appinfos;
+		List	   *appinfos1;
+		List	   *appinfos2;
+
+		/*
+		 * Add paths to joining relation if it is a join itself.
+		 * Paths for child base relations are created in
+		 * set_append_rel_pathlist().
+		 */
+		if (IS_JOIN_REL(pj->rel1))
+			add_paths_to_child_joinrel(root, rel1, child_id);
+
+		if (IS_JOIN_REL(pj->rel2))
+			add_paths_to_child_joinrel(root, rel2, child_id);
+
+		/* Get parent-child mapping for translating nodes. */
+		appinfos1 = find_appinfos_by_relids(root, child_rel1->relids);
+		appinfos2 = find_appinfos_by_relids(root, child_rel2->relids);
+		join_appinfos = list_concat(appinfos1, appinfos2);
+
+		/*
+		 * Construct SpecialJoinInfo from parent join relations's
+		 * SpecialJoinInfo.
+		 */
+		child_sjinfo = build_child_join_sjinfo(root, pj->sjinfo, appinfos1,
+											   appinfos2);
+
+		/*
+		 * Construct restrictions applicable to the child join from
+		 * those applicable to the parent join.
+		 */
+		child_restrictlist = (List *) adjust_join_appendrel_attrs(root,
+													 (Node *) pj->restrictlist,
+																join_appinfos);
+
+		/* The list is not needed anymore. */
+		list_free(join_appinfos);
+
+		/* Add paths for child join. */
+		populate_joinrel_with_paths(root, rel1->part_rels[child_id],
+									rel2->part_rels[child_id], child_joinrel,
+									child_sjinfo, child_restrictlist);
+
+		/*
+		 * In case the child is partitioned, add partition-wise join paths for
+		 * it.
+		 */
+		generate_partition_wise_join_paths(root, child_joinrel);
+	}
+
+	set_cheapest(child_joinrel);
+}
+
+/*
+ * Construct the SpecialJoinInfo for the join between children by translating
+ * SpecialJoinInfo for the join between parents.
+ */
+static SpecialJoinInfo *
+build_child_join_sjinfo(PlannerInfo *root, SpecialJoinInfo *parent_sjinfo,
+							List *append_rel_infos1, List *append_rel_infos2)
+{
+	SpecialJoinInfo *sjinfo = copyObject(parent_sjinfo);
+
+	sjinfo->min_lefthand = adjust_child_relids(sjinfo->min_lefthand,
+											   append_rel_infos1);
+	sjinfo->min_righthand = adjust_child_relids(sjinfo->min_righthand,
+												append_rel_infos2);
+	sjinfo->syn_lefthand = adjust_child_relids(sjinfo->syn_lefthand,
+											   append_rel_infos1);
+	sjinfo->syn_righthand = adjust_child_relids(sjinfo->syn_righthand,
+												append_rel_infos2);
+
+	/* Replace the Var nodes of parent with those of children in expressions. */
+	sjinfo->semi_rhs_exprs = (List *) adjust_join_appendrel_attrs(root,
+											   (Node *) sjinfo->semi_rhs_exprs,
+															append_rel_infos2);
+	return sjinfo;
+}
+
+/*
+ * Replace parent relids by child relids in the given relid set.
+ */
+Relids
+adjust_child_relids(Relids relids, List *append_rel_infos)
+{
+	ListCell	*lc;
+	foreach (lc, append_rel_infos)
+	{
+		AppendRelInfo	*appinfo = lfirst(lc);
+
+		/* Remove parent, add child*/
+		if (bms_is_member(appinfo->parent_relid, relids))
+		{
+			relids = bms_del_member(relids, appinfo->parent_relid);
+			relids = bms_add_member(relids, appinfo->child_relid);
+		}
+	}
+	return relids;
+}
+
+/*
+ * Returns true if there exists an equi-join condition for each pair of
+ * partition key from given relations being joined.
+ */
+static bool
+have_partkey_equi_join(RelOptInfo *rel1, RelOptInfo *rel2,
+					 JoinType jointype, List *restrictlist)
+{
+	PartitionScheme	part_scheme = rel1->part_scheme;
+	ListCell	*lc;
+	int		cnt_pks;
+	int		num_pks;
+	bool   *pk_has_clause;
+
+	/*
+	 * This function should be called when the joining relations have same
+	 * partitioning scheme.
+	 */
+	Assert(rel1->part_scheme == rel2->part_scheme);
+	Assert(part_scheme);
+
+	num_pks = part_scheme->partnatts;
+
+	pk_has_clause = (bool *) palloc0(sizeof(bool) * num_pks);
+
+	foreach (lc, restrictlist)
+	{
+		RestrictInfo *rinfo = lfirst(lc);
+		OpExpr		 *opexpr;
+		Expr		 *expr1;
+		Expr		 *expr2;
+		int		ipk1;
+		int		ipk2;
+
+		/* If processing an outer join, only use its own join clauses. */
+		if (IS_OUTER_JOIN(jointype) && rinfo->is_pushed_down)
+			continue;
+
+		/* Skip clauses which can not be used for a join. */
+		if (!rinfo->can_join)
+			continue;
+
+		/* Skip clauses which are not equality conditions. */
+		if (rinfo->hashjoinoperator == InvalidOid && !rinfo->mergeopfamilies)
+			continue;
+
+		opexpr = (OpExpr *) rinfo->clause;
+		Assert(is_opclause(opexpr));
+
+
+		/* Match the operands to the relation. */
+		if (bms_is_subset(rinfo->left_relids, rel1->relids) &&
+			bms_is_subset(rinfo->right_relids, rel2->relids))
+		{
+			expr1 = linitial(opexpr->args);
+			expr2 = lsecond(opexpr->args);
+		}
+		else if (bms_is_subset(rinfo->left_relids, rel2->relids) &&
+				 bms_is_subset(rinfo->right_relids, rel1->relids))
+		{
+			expr1 = lsecond(opexpr->args);
+			expr2 = linitial(opexpr->args);
+		}
+		else
+			continue;
+
+		/* Associate matching clauses with partition keys. */
+		ipk1 = match_expr_to_partition_keys(expr1, rel1);
+		ipk2 = match_expr_to_partition_keys(expr2, rel2);
+
+		/*
+		 * If the clause refers to different partition keys from
+		 * both relations, it can not be used for partition-wise join.
+		 */
+		if (ipk1 != ipk2)
+			continue;
+
+		/*
+		 * The clause allows partition-wise join if only it uses the same
+		 * operator family as that specified by the partition key.
+		 */
+		if (!list_member_oid(rinfo->mergeopfamilies,
+							 part_scheme->partopfamily[ipk1]))
+			continue;
+
+		/* Mark the partition key as having an equi-join clause. */
+		pk_has_clause[ipk1] = true;
+	}
+
+	/* Check whether every partition key has an equi-join condition. */
+	for (cnt_pks = 0; cnt_pks < num_pks; cnt_pks++)
+	{
+		if (!pk_has_clause[cnt_pks])
+		{
+			pfree(pk_has_clause);
+			return false;
+		}
+	}
+
+	pfree(pk_has_clause);
+	return true;
+}
+
+/*
+ * Find the partition key from the given relation matching the given
+ * expression. If found, return the index of the partition key, else return -1.
+ */
+static int
+match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel)
+{
+	int		cnt_pks;
+	int		num_pks;
+
+	/* This function should be called only for partitioned relations. */
+	Assert(rel->part_scheme);
+
+	num_pks = rel->part_scheme->partnatts;
+
+	/*
+	 * Remove the relabel decoration. We can assume that there is at most one
+	 * RelabelType node; eval_const_expressions() simplifies multiple
+	 * RelabelType nodes into one.
+	 */
+	if (IsA(expr, RelabelType))
+		expr = (Expr *) ((RelabelType *) expr)->arg;
+
+	for (cnt_pks = 0; cnt_pks < num_pks; cnt_pks++)
+	{
+		List	 *pkexprs = rel->partexprs[cnt_pks];
+		ListCell *lc;
+
+		foreach(lc, pkexprs)
+		{
+			Expr *pkexpr = lfirst(lc);
+			if (equal(pkexpr, expr))
+				return cnt_pks;
+		}
+	}
+
+	return -1;
+}
diff --git a/src/backend/optimizer/path/pathkeys.c b/src/backend/optimizer/path/pathkeys.c
index 4436ac1..6996590 100644
--- a/src/backend/optimizer/path/pathkeys.c
+++ b/src/backend/optimizer/path/pathkeys.c
@@ -1088,12 +1088,24 @@ select_outer_pathkeys_for_merge(PlannerInfo *root,
 	int			necs;
 	ListCell   *lc;
 	int			j;
+	Relids		relids;
 
 	/* Might have no mergeclauses */
 	if (nClauses == 0)
 		return NIL;
 
 	/*
+	 * For a child join relation, use parent relids to find potential
+	 * join partners from equivalence classes. A potential join partner of
+	 * parent also indicates potential join partner of the child. By using
+	 * parent relids we eliminate duplicates arising out of many children.
+	 */
+	if (joinrel->reloptkind == RELOPT_OTHER_JOINREL)
+		relids = joinrel->top_parent_relids;
+	else
+		relids = joinrel->relids;
+
+	/*
 	 * Make arrays of the ECs used by the mergeclauses (dropping any
 	 * duplicates) and their "popularity" scores.
 	 */
@@ -1133,7 +1145,7 @@ select_outer_pathkeys_for_merge(PlannerInfo *root,
 
 			/* Potential future join partner? */
 			if (!em->em_is_const && !em->em_is_child &&
-				!bms_overlap(em->em_relids, joinrel->relids))
+				!bms_overlap(em->em_relids, relids))
 				score++;
 		}
 
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 32f4031..d7397b3 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -31,6 +31,7 @@
 #include "optimizer/clauses.h"
 #include "optimizer/cost.h"
 #include "optimizer/paths.h"
+#include "optimizer/pathnode.h"
 #include "optimizer/placeholder.h"
 #include "optimizer/plancat.h"
 #include "optimizer/planmain.h"
@@ -43,6 +44,7 @@
 #include "parser/parse_clause.h"
 #include "parser/parsetree.h"
 #include "utils/lsyscache.h"
+#include "utils/memutils.h"
 
 
 /*
@@ -146,6 +148,9 @@ static CustomScan *create_customscan_plan(PlannerInfo *root,
 static NestLoop *create_nestloop_plan(PlannerInfo *root, NestPath *best_path);
 static MergeJoin *create_mergejoin_plan(PlannerInfo *root, MergePath *best_path);
 static HashJoin *create_hashjoin_plan(PlannerInfo *root, HashPath *best_path);
+static Plan *create_partition_plan(PlannerInfo *root, Path *best_path);
+static Plan *create_partition_join_plan(PlannerInfo *root,
+										PartitionJoinPath *best_path);
 static Node *replace_nestloop_params(PlannerInfo *root, Node *expr);
 static Node *replace_nestloop_params_mutator(Node *node, PlannerInfo *root);
 static void process_subquery_nestloop_params(PlannerInfo *root,
@@ -242,7 +247,8 @@ static Plan *prepare_sort_from_pathkeys(Plan *lefttree, List *pathkeys,
 static EquivalenceMember *find_ec_member_for_tle(EquivalenceClass *ec,
 					   TargetEntry *tle,
 					   Relids relids);
-static Sort *make_sort_from_pathkeys(Plan *lefttree, List *pathkeys);
+static Sort *make_sort_from_pathkeys(Plan *lefttree, List *pathkeys,
+									 Relids relids);
 static Sort *make_sort_from_groupcols(List *groupcls,
 						 AttrNumber *grpColIdx,
 						 Plan *lefttree);
@@ -368,12 +374,8 @@ create_plan_recurse(PlannerInfo *root, Path *best_path, int flags)
 									(JoinPath *) best_path);
 			break;
 		case T_Append:
-			plan = create_append_plan(root,
-									  (AppendPath *) best_path);
-			break;
 		case T_MergeAppend:
-			plan = create_merge_append_plan(root,
-											(MergeAppendPath *) best_path);
+			plan = create_partition_plan(root, best_path);
 			break;
 		case T_Result:
 			if (IsA(best_path, ProjectionPath))
@@ -1514,7 +1516,7 @@ create_sort_plan(PlannerInfo *root, SortPath *best_path, int flags)
 	subplan = create_plan_recurse(root, best_path->subpath,
 								  flags | CP_SMALL_TLIST);
 
-	plan = make_sort_from_pathkeys(subplan, best_path->path.pathkeys);
+	plan = make_sort_from_pathkeys(subplan, best_path->path.pathkeys, NULL);
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
@@ -3524,6 +3526,8 @@ create_mergejoin_plan(PlannerInfo *root,
 	ListCell   *lc;
 	ListCell   *lop;
 	ListCell   *lip;
+	Path	   *outer_path = best_path->jpath.outerjoinpath;
+	Path	   *inner_path = best_path->jpath.innerjoinpath;
 
 	/*
 	 * MergeJoin can project, so we don't have to demand exact tlists from the
@@ -3531,10 +3535,10 @@ create_mergejoin_plan(PlannerInfo *root,
 	 * best to request a small tlist so we aren't sorting more data than
 	 * necessary.
 	 */
-	outer_plan = create_plan_recurse(root, best_path->jpath.outerjoinpath,
+	outer_plan = create_plan_recurse(root, outer_path,
 					 (best_path->outersortkeys != NIL) ? CP_SMALL_TLIST : 0);
 
-	inner_plan = create_plan_recurse(root, best_path->jpath.innerjoinpath,
+	inner_plan = create_plan_recurse(root, inner_path,
 					 (best_path->innersortkeys != NIL) ? CP_SMALL_TLIST : 0);
 
 	/* Sort join qual clauses into best execution order */
@@ -3580,34 +3584,38 @@ create_mergejoin_plan(PlannerInfo *root,
 	 * outer_is_left status.
 	 */
 	mergeclauses = get_switched_clauses(best_path->path_mergeclauses,
-							 best_path->jpath.outerjoinpath->parent->relids);
+							 outer_path->parent->relids);
 
 	/*
 	 * Create explicit sort nodes for the outer and inner paths if necessary.
 	 */
 	if (best_path->outersortkeys)
 	{
+		Relids		outer_relids = outer_path->parent->relids;
 		Sort	   *sort = make_sort_from_pathkeys(outer_plan,
-												   best_path->outersortkeys);
+												   best_path->outersortkeys,
+												   outer_relids);
 
 		label_sort_with_costsize(root, sort, -1.0);
 		outer_plan = (Plan *) sort;
 		outerpathkeys = best_path->outersortkeys;
 	}
 	else
-		outerpathkeys = best_path->jpath.outerjoinpath->pathkeys;
+		outerpathkeys = outer_path->pathkeys;
 
 	if (best_path->innersortkeys)
 	{
+		Relids		inner_relids = inner_path->parent->relids;
 		Sort	   *sort = make_sort_from_pathkeys(inner_plan,
-												   best_path->innersortkeys);
+												   best_path->innersortkeys,
+												   inner_relids);
 
 		label_sort_with_costsize(root, sort, -1.0);
 		inner_plan = (Plan *) sort;
 		innerpathkeys = best_path->innersortkeys;
 	}
 	else
-		innerpathkeys = best_path->jpath.innerjoinpath->pathkeys;
+		innerpathkeys = inner_path->pathkeys;
 
 	/*
 	 * If specified, add a materialize node to shield the inner plan from the
@@ -3945,6 +3953,226 @@ create_hashjoin_plan(PlannerInfo *root,
 	return join_plan;
 }
 
+static Plan *
+create_partition_plan(PlannerInfo *root, Path *best_path)
+{
+	Plan   *plan;
+
+	if (IsA(best_path, PartitionJoinPath))
+		plan = create_partition_join_plan(root, (PartitionJoinPath *)best_path);
+	else if (best_path->pathtype == T_Append)
+		plan = create_append_plan(root, (AppendPath *) best_path);
+	else
+	{
+		Assert(best_path->pathtype == T_MergeAppend);
+		plan = create_merge_append_plan(root, (MergeAppendPath *) best_path);
+	}
+
+	return plan;
+}
+
+static Plan *
+create_partition_join_plan(PlannerInfo *root, PartitionJoinPath *best_path)
+{
+	RelOptInfo *joinrel = best_path->path.parent;
+	int		nparts;
+	int		cnt_parts;
+	List	   *child_plans = NIL;
+	List	   *tlist = build_path_tlist(root, &best_path->path);
+	Plan	   *plan;
+	MemoryContext	child_context;
+	MemoryContext	old_context;
+	List	   *pathkeys = best_path->path.pathkeys;
+	StringInfoData	mem_context_name;
+
+	/* The relation should be a partitioned join relation. */
+	Assert(IS_JOIN_REL(joinrel) && joinrel->part_scheme &&
+		   joinrel->partitioned_joins);
+
+	nparts = joinrel->part_scheme->nparts;
+
+	/* Create MergeAppend plan when result is expected to be ordered. */
+	if (pathkeys)
+	{
+		MergeAppend *node = makeNode(MergeAppend);
+		plan = &node->plan;
+
+		plan->targetlist = tlist;
+
+		/* Compute sorting info, and adjust MergeAppend's tlist as needed. */
+		(void) prepare_sort_from_pathkeys(plan, pathkeys,
+										  best_path->path.parent->relids,
+										  NULL,
+										  true,
+										  &node->numCols,
+										  &node->sortColIdx,
+										  &node->sortOperators,
+										  &node->collations,
+										  &node->nullsFirst);
+	}
+	else
+	{
+		Append  *node = makeNode(Append);
+		plan = &node->plan;
+		plan->targetlist = tlist;
+	}
+
+	/* Fill costs, so that we can cost Sort node, if required. */
+	copy_generic_path_info(plan, (Path *) best_path);
+
+	/*
+	 * Create a new memory context for planning child joins. Since this routine
+	 * may be called recursively for tables with subpartitions, we use
+	 * a unique context name for every level of partition by using the lowest
+	 * relid amongst the base relations being joined.
+	 */
+	initStringInfo(&mem_context_name);
+	appendStringInfo(&mem_context_name, "%s_%d", "ChildJoinContext",
+					 bms_next_member(joinrel->relids, -1));
+	child_context = AllocSetContextCreate(CurrentMemoryContext,
+										  pstrdup(mem_context_name.data),
+										  ALLOCSET_DEFAULT_SIZES);
+	pfree(mem_context_name.data);
+	resetStringInfo(&mem_context_name);
+
+	/*
+	 * Create a paths for all child joins, one child join at a time. The paths
+	 * for every child join are independent i.e. one child does not require
+	 * paths created for the other. In order to avoid accumulating memory
+	 * consumed while creating paths for every child join, we use a fresh
+	 * memory context for every child join.
+	 */
+	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	{
+		RelOptInfo *child_join;
+		Path	   *child_path;
+		Plan	   *child_plan;
+		int			numsortkeys;
+		AttrNumber *sortColIdx;
+		Oid		   *sortOperators;
+		Oid		   *collations;
+		bool	   *nullsFirst;
+
+		/* Switch to the child_join context to plan for the child join */
+		old_context = MemoryContextSwitchTo(child_context);
+
+		add_paths_to_child_joinrel(root, joinrel, cnt_parts);
+		child_join = joinrel->part_rels[cnt_parts];
+
+		/* Skip empty child. */
+		if (IS_DUMMY_REL(child_join))
+		{
+			MemoryContextSwitchTo(old_context);
+			MemoryContextResetAndDeleteChildren(child_context);
+
+			continue;
+		}
+
+#ifdef OPTIMIZER_DEBUG
+		debug_print_rel(root, rel);
+#endif
+
+		/*
+		 * Search for a child path with pathkeys or parameterization
+		 * matching that of the given path.
+		 */
+		child_path = get_cheapest_path_for_pathkeys(child_join->pathlist,
+													best_path->path.pathkeys,
+											 PATH_REQ_OUTER(&best_path->path),
+													TOTAL_COST);
+
+		if (!child_path)
+			elog(ERROR, "Could not find a path with required pathkeys.");
+
+		/* Create plan for the current child. */
+		child_plan = create_plan_recurse(root, child_path, CP_EXACT_TLIST);
+
+		if (pathkeys)
+		{
+			MergeAppend *node = (MergeAppend *) plan;
+
+			Assert(IsA(node, MergeAppend));
+
+			/* Compute sorting info, and adjust subplan's tlist as needed */
+			child_plan = prepare_sort_from_pathkeys(child_plan, pathkeys,
+												 child_path->parent->relids,
+												 node->sortColIdx,
+												 false,
+												 &numsortkeys,
+												 &sortColIdx,
+												 &sortOperators,
+												 &collations,
+												 &nullsFirst);
+
+			/*
+			 * Check that we got the same sort key information.  We just Assert
+			 * that the sortops match, since those depend only on the pathkeys;
+			 * but it seems like a good idea to check the sort column numbers
+			 * explicitly, to ensure the tlists really do match up.
+			 */
+			Assert(numsortkeys == node->numCols);
+			if (memcmp(sortColIdx, node->sortColIdx,
+					   numsortkeys * sizeof(AttrNumber)) != 0)
+				elog(ERROR, "MergeAppend child's targetlist doesn't match MergeAppend");
+			Assert(memcmp(sortOperators, node->sortOperators,
+						  numsortkeys * sizeof(Oid)) == 0);
+			Assert(memcmp(collations, node->collations,
+						  numsortkeys * sizeof(Oid)) == 0);
+			Assert(memcmp(nullsFirst, node->nullsFirst,
+						  numsortkeys * sizeof(bool)) == 0);
+
+			/* Now, insert a Sort node if subplan isn't sufficiently ordered */
+			if (!pathkeys_contained_in(pathkeys, child_path->pathkeys))
+			{
+				Sort	   *sort = make_sort(child_plan, numsortkeys,
+											 sortColIdx, sortOperators,
+											 collations, nullsFirst);
+				label_sort_with_costsize(root, sort, -1.0);
+				child_plan = (Plan *) sort;
+			}
+		}
+
+		/*
+		 * Switch to the original context so that we can copy the plan in the
+		 * same context as rest of the plan tree.
+		 */
+		MemoryContextSwitchTo(old_context);
+		child_plan = copyObject(child_plan);
+
+		child_plans = lappend(child_plans, child_plan);
+
+		/* Reset the child_join memory context to reclaim the memory. */
+		MemoryContextResetAndDeleteChildren(child_context);
+	}
+
+	/* Destroy the child context as we do not need it anymore. */
+	Assert(CurrentMemoryContext == old_context);
+	MemoryContextDelete(child_context);
+
+	/* Partitioned relation with all empty children gets a dummy path. */
+	Assert(child_plans != NIL);
+
+	if (IsA(plan, MergeAppend))
+	{
+		MergeAppend *node = (MergeAppend *)plan;
+
+		node->mergeplans = child_plans;
+	}
+	else
+	{
+		Append  *node = (Append *)plan;
+
+		Assert(IsA(plan, Append));
+		node->appendplans = child_plans;
+	}
+
+	/* Complete rest of the plan. */
+	plan->qual = NIL;
+	plan->lefttree = NULL;
+	plan->righttree = NULL;
+	return plan;
+}
+
 
 /*****************************************************************************
  *
@@ -3971,6 +4199,8 @@ replace_nestloop_params(PlannerInfo *root, Node *expr)
 static Node *
 replace_nestloop_params_mutator(Node *node, PlannerInfo *root)
 {
+	MemoryContext old_context;
+
 	if (node == NULL)
 		return NULL;
 	if (IsA(node, Var))
@@ -3999,10 +4229,13 @@ replace_nestloop_params_mutator(Node *node, PlannerInfo *root)
 			}
 		}
 		/* No, so add it */
+		old_context = MemoryContextSwitchTo(root->planner_cxt);
 		nlp = makeNode(NestLoopParam);
 		nlp->paramno = param->paramid;
 		nlp->paramval = var;
 		root->curOuterParams = lappend(root->curOuterParams, nlp);
+		MemoryContextSwitchTo(old_context);
+
 		/* And return the replacement Param */
 		return (Node *) param;
 	}
@@ -4062,10 +4295,13 @@ replace_nestloop_params_mutator(Node *node, PlannerInfo *root)
 			}
 		}
 		/* No, so add it */
+		old_context = MemoryContextSwitchTo(root->planner_cxt);
 		nlp = makeNode(NestLoopParam);
 		nlp->paramno = param->paramid;
 		nlp->paramval = (Var *) phv;
 		root->curOuterParams = lappend(root->curOuterParams, nlp);
+		MemoryContextSwitchTo(old_context);
+
 		/* And return the replacement Param */
 		return (Node *) param;
 	}
@@ -5337,11 +5573,11 @@ prepare_sort_from_pathkeys(Plan *lefttree, List *pathkeys,
 					continue;
 
 				/*
-				 * Ignore child members unless they match the rel being
+				 * Ignore child members unless they belong to the rel being
 				 * sorted.
 				 */
 				if (em->em_is_child &&
-					!bms_equal(em->em_relids, relids))
+					!bms_is_subset(em->em_relids, relids))
 					continue;
 
 				sortexpr = em->em_expr;
@@ -5452,10 +5688,10 @@ find_ec_member_for_tle(EquivalenceClass *ec,
 			continue;
 
 		/*
-		 * Ignore child members unless they match the rel being sorted.
+		 * Ignore child members unless they belong to the rel being sorted.
 		 */
 		if (em->em_is_child &&
-			!bms_equal(em->em_relids, relids))
+			!bms_is_subset(em->em_relids, relids))
 			continue;
 
 		/* Match if same expression (after stripping relabel) */
@@ -5476,9 +5712,10 @@ find_ec_member_for_tle(EquivalenceClass *ec,
  *
  *	  'lefttree' is the node which yields input tuples
  *	  'pathkeys' is the list of pathkeys by which the result is to be sorted
+ *	  'relids' is the set of relations required by prepare_sort_from_pathkeys()
  */
 static Sort *
-make_sort_from_pathkeys(Plan *lefttree, List *pathkeys)
+make_sort_from_pathkeys(Plan *lefttree, List *pathkeys, Relids relids)
 {
 	int			numsortkeys;
 	AttrNumber *sortColIdx;
@@ -5488,7 +5725,7 @@ make_sort_from_pathkeys(Plan *lefttree, List *pathkeys)
 
 	/* Compute sort column info, and adjust lefttree as needed */
 	lefttree = prepare_sort_from_pathkeys(lefttree, pathkeys,
-										  NULL,
+										  relids,
 										  NULL,
 										  false,
 										  &numsortkeys,
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index f657ffc..3c0898a 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -1108,7 +1108,7 @@ inheritance_planner(PlannerInfo *root)
 		subroot->parse = (Query *)
 			adjust_appendrel_attrs(root,
 								   (Node *) parse,
-								   appinfo);
+								   list_make1(appinfo));
 
 		/*
 		 * The rowMarks list might contain references to subquery RTEs, so
diff --git a/src/backend/optimizer/plan/subselect.c b/src/backend/optimizer/plan/subselect.c
index 263ba45..6f3270a 100644
--- a/src/backend/optimizer/plan/subselect.c
+++ b/src/backend/optimizer/plan/subselect.c
@@ -96,6 +96,7 @@ assign_param_for_var(PlannerInfo *root, Var *var)
 	ListCell   *ppl;
 	PlannerParamItem *pitem;
 	Index		levelsup;
+	MemoryContext	old_context;
 
 	/* Find the query level the Var belongs to */
 	for (levelsup = var->varlevelsup; levelsup > 0; levelsup--)
@@ -124,7 +125,9 @@ assign_param_for_var(PlannerInfo *root, Var *var)
 		}
 	}
 
-	/* Nope, so make a new one */
+
+	/* Nope, so make a new one in the planner context */
+	old_context = MemoryContextSwitchTo(root->planner_cxt);
 	var = (Var *) copyObject(var);
 	var->varlevelsup = 0;
 
@@ -134,6 +137,8 @@ assign_param_for_var(PlannerInfo *root, Var *var)
 
 	root->plan_params = lappend(root->plan_params, pitem);
 
+	MemoryContextSwitchTo(old_context);
+
 	return pitem->paramId;
 }
 
@@ -204,6 +209,7 @@ assign_param_for_placeholdervar(PlannerInfo *root, PlaceHolderVar *phv)
 	ListCell   *ppl;
 	PlannerParamItem *pitem;
 	Index		levelsup;
+	MemoryContext old_context;
 
 	/* Find the query level the PHV belongs to */
 	for (levelsup = phv->phlevelsup; levelsup > 0; levelsup--)
@@ -223,7 +229,8 @@ assign_param_for_placeholdervar(PlannerInfo *root, PlaceHolderVar *phv)
 		}
 	}
 
-	/* Nope, so make a new one */
+	/* Nope, so make a new one in the planner's context. */
+	old_context = MemoryContextSwitchTo(root->planner_cxt);
 	phv = (PlaceHolderVar *) copyObject(phv);
 	if (phv->phlevelsup != 0)
 	{
@@ -237,6 +244,8 @@ assign_param_for_placeholdervar(PlannerInfo *root, PlaceHolderVar *phv)
 
 	root->plan_params = lappend(root->plan_params, pitem);
 
+	MemoryContextSwitchTo(old_context);
+
 	return pitem->paramId;
 }
 
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index 193b2c9..1ce142e 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -55,7 +55,7 @@
 typedef struct
 {
 	PlannerInfo *root;
-	AppendRelInfo *appinfo;
+	List		*appinfos;
 	int			sublevels_up;
 } adjust_appendrel_attrs_context;
 
@@ -108,7 +108,6 @@ static Bitmapset *translate_col_privs(const Bitmapset *parent_privs,
 					List *translated_vars);
 static Node *adjust_appendrel_attrs_mutator(Node *node,
 							   adjust_appendrel_attrs_context *context);
-static Relids adjust_relid_set(Relids relids, Index oldrelid, Index newrelid);
 static List *adjust_inherited_tlist(List *tlist,
 					   AppendRelInfo *context);
 static List *expand_inherited_rte_internal(PlannerInfo *root, RangeTblEntry *rte,
@@ -1828,10 +1827,11 @@ translate_col_privs(const Bitmapset *parent_privs,
 
 /*
  * adjust_appendrel_attrs
- *	  Copy the specified query or expression and translate Vars referring
- *	  to the parent rel of the specified AppendRelInfo to refer to the
- *	  child rel instead.  We also update rtindexes appearing outside Vars,
- *	  such as resultRelation and jointree relids.
+ *
+ *	  Copy the specified query or expression and translate Vars referring to
+ *	  the parent rels specified in the given list of AppendRelInfos to refer to
+ *	  the corresponding child rels instead.  We also update rtindexes appearing
+ *	  outside Vars, such as resultRelation and jointree relids.
  *
  * Note: this is applied after conversion of sublinks to subplans in the
  * query jointree, but there may still be sublinks in the security barrier
@@ -1841,15 +1841,18 @@ translate_col_privs(const Bitmapset *parent_privs,
  * maybe we should try to fold the two routines together.
  */
 Node *
-adjust_appendrel_attrs(PlannerInfo *root, Node *node, AppendRelInfo *appinfo)
+adjust_appendrel_attrs(PlannerInfo *root, Node *node, List *appinfos)
 {
 	Node	   *result;
 	adjust_appendrel_attrs_context context;
+	ListCell   *lc;
 
 	context.root = root;
-	context.appinfo = appinfo;
+	context.appinfos = appinfos;
 	context.sublevels_up = 0;
 
+	Assert(appinfos && list_length(appinfos) >= 1);
+
 	/*
 	 * Must be prepared to start with a Query or a bare expression tree; if
 	 * it's a Query, go straight to query_tree_walker to make sure that
@@ -1858,11 +1861,20 @@ adjust_appendrel_attrs(PlannerInfo *root, Node *node, AppendRelInfo *appinfo)
 	if (node && IsA(node, Query))
 	{
 		Query	   *newnode;
+		AppendRelInfo *appinfo;
 
 		newnode = query_tree_mutator((Query *) node,
 									 adjust_appendrel_attrs_mutator,
 									 (void *) &context,
 									 QTW_IGNORE_RC_SUBQUERIES);
+		foreach (lc, appinfos)
+		{
+			appinfo = lfirst(lc);
+
+			if (newnode->resultRelation == appinfo->parent_relid)
+				break;
+		}
+
 		if (newnode->resultRelation == appinfo->parent_relid)
 		{
 			newnode->resultRelation = appinfo->child_relid;
@@ -1880,17 +1892,73 @@ adjust_appendrel_attrs(PlannerInfo *root, Node *node, AppendRelInfo *appinfo)
 	return result;
 }
 
+/*
+ * find_appinfos_by_relids
+ * 		Find AppendRelInfo structures for all relations specified by relids.
+ */
+List *
+find_appinfos_by_relids(PlannerInfo *root, Relids relids)
+{
+	ListCell	*lc;
+	List		*appinfo_list = NIL;
+
+	foreach (lc, root->append_rel_list)
+	{
+		AppendRelInfo *appinfo = lfirst(lc);
+
+		if (bms_is_member(appinfo->child_relid, relids))
+			appinfo_list = lappend(appinfo_list, appinfo);
+	}
+
+	Assert(list_length(appinfo_list) == bms_num_members(relids));
+	return appinfo_list;
+}
+
+/*
+ * adjust_join_appendrel_attrs
+ *
+ *	    Replace the parent references in the given node by the child references
+ *	    specified by the list of AppendRelInfo.
+ *
+ * This function is a wrapper around adjust_appendrel_attrs() which handles
+ * only one AppendRelInfo at a time.
+ */
+
+Node *
+adjust_join_appendrel_attrs(PlannerInfo *root, Node *node,
+						  List *append_rel_infos)
+{
+	return adjust_appendrel_attrs(root, node, append_rel_infos);
+}
+
 static Node *
 adjust_appendrel_attrs_mutator(Node *node,
 							   adjust_appendrel_attrs_context *context)
 {
-	AppendRelInfo *appinfo = context->appinfo;
+	List   *appinfos = context->appinfos;
+	ListCell   *lc;
+
+	Assert(appinfos && list_length(appinfos) >= 1);
 
 	if (node == NULL)
 		return NULL;
 	if (IsA(node, Var))
 	{
 		Var		   *var = (Var *) copyObject(node);
+		AppendRelInfo *appinfo;
+
+		/*
+		 * Find an appinfo, parent in which matches the Var. If none found, set
+		 * it to the last one. Rest of the code takes care of non-matching
+		 * appinfos.
+		 */
+		foreach (lc, appinfos)
+		{
+			appinfo = lfirst(lc);
+
+			if (var->varno == appinfo->parent_relid)
+				break;
+		}
 
 		if (var->varlevelsup == context->sublevels_up &&
 			var->varno == appinfo->parent_relid)
@@ -1981,32 +2049,58 @@ adjust_appendrel_attrs_mutator(Node *node,
 	{
 		CurrentOfExpr *cexpr = (CurrentOfExpr *) copyObject(node);
 
-		if (context->sublevels_up == 0 &&
-			cexpr->cvarno == appinfo->parent_relid)
-			cexpr->cvarno = appinfo->child_relid;
+		foreach (lc, appinfos)
+		{
+			AppendRelInfo *appinfo = lfirst(lc);
+
+			if (context->sublevels_up == 0 &&
+				cexpr->cvarno == appinfo->parent_relid)
+			{
+				cexpr->cvarno = appinfo->child_relid;
+				break;
+			}
+		}
 		return (Node *) cexpr;
 	}
 	if (IsA(node, RangeTblRef))
 	{
 		RangeTblRef *rtr = (RangeTblRef *) copyObject(node);
 
-		if (context->sublevels_up == 0 &&
-			rtr->rtindex == appinfo->parent_relid)
-			rtr->rtindex = appinfo->child_relid;
+		foreach (lc, appinfos)
+		{
+			AppendRelInfo *appinfo = lfirst(lc);
+
+			if (context->sublevels_up == 0 &&
+				rtr->rtindex == appinfo->parent_relid)
+			{
+				rtr->rtindex = appinfo->child_relid;
+				break;
+			}
+		}
 		return (Node *) rtr;
 	}
 	if (IsA(node, JoinExpr))
 	{
 		/* Copy the JoinExpr node with correct mutation of subnodes */
 		JoinExpr   *j;
+		AppendRelInfo *appinfo;
 
 		j = (JoinExpr *) expression_tree_mutator(node,
 											  adjust_appendrel_attrs_mutator,
 												 (void *) context);
+
 		/* now fix JoinExpr's rtindex (probably never happens) */
-		if (context->sublevels_up == 0 &&
-			j->rtindex == appinfo->parent_relid)
-			j->rtindex = appinfo->child_relid;
+		foreach (lc, appinfos)
+		{
+			appinfo = lfirst(lc);
+
+			if (context->sublevels_up == 0 &&
+				j->rtindex == appinfo->parent_relid)
+			{
+				j->rtindex = appinfo->child_relid;
+				break;
+			}
+		}
 		return (Node *) j;
 	}
 	if (IsA(node, PlaceHolderVar))
@@ -2019,9 +2113,8 @@ adjust_appendrel_attrs_mutator(Node *node,
 														 (void *) context);
 		/* now fix PlaceHolderVar's relid sets */
 		if (phv->phlevelsup == context->sublevels_up)
-			phv->phrels = adjust_relid_set(phv->phrels,
-										   appinfo->parent_relid,
-										   appinfo->child_relid);
+			phv->phrels = adjust_child_relids(bms_copy(phv->phrels),
+											  context->appinfos);
 		return (Node *) phv;
 	}
 	/* Shouldn't need to handle planner auxiliary nodes here */
@@ -2052,24 +2145,18 @@ adjust_appendrel_attrs_mutator(Node *node,
 			adjust_appendrel_attrs_mutator((Node *) oldinfo->orclause, context);
 
 		/* adjust relid sets too */
-		newinfo->clause_relids = adjust_relid_set(oldinfo->clause_relids,
-												  appinfo->parent_relid,
-												  appinfo->child_relid);
-		newinfo->required_relids = adjust_relid_set(oldinfo->required_relids,
-													appinfo->parent_relid,
-													appinfo->child_relid);
-		newinfo->outer_relids = adjust_relid_set(oldinfo->outer_relids,
-												 appinfo->parent_relid,
-												 appinfo->child_relid);
-		newinfo->nullable_relids = adjust_relid_set(oldinfo->nullable_relids,
-													appinfo->parent_relid,
-													appinfo->child_relid);
-		newinfo->left_relids = adjust_relid_set(oldinfo->left_relids,
-												appinfo->parent_relid,
-												appinfo->child_relid);
-		newinfo->right_relids = adjust_relid_set(oldinfo->right_relids,
-												 appinfo->parent_relid,
-												 appinfo->child_relid);
+		newinfo->clause_relids = adjust_child_relids(bms_copy(oldinfo->clause_relids),
+													 context->appinfos);
+		newinfo->required_relids = adjust_child_relids(bms_copy(oldinfo->required_relids),
+													 context->appinfos);
+		newinfo->outer_relids = adjust_child_relids(bms_copy(oldinfo->outer_relids),
+													 context->appinfos);
+		newinfo->nullable_relids = adjust_child_relids(bms_copy(oldinfo->nullable_relids),
+													   context->appinfos);
+		newinfo->left_relids = adjust_child_relids(bms_copy(oldinfo->left_relids),
+												   context->appinfos);
+		newinfo->right_relids = adjust_child_relids(bms_copy(oldinfo->right_relids),
+													context->appinfos);
 
 		/*
 		 * Reset cached derivative fields, since these might need to have
@@ -2118,23 +2205,6 @@ adjust_appendrel_attrs_mutator(Node *node,
 }
 
 /*
- * Substitute newrelid for oldrelid in a Relid set
- */
-static Relids
-adjust_relid_set(Relids relids, Index oldrelid, Index newrelid)
-{
-	if (bms_is_member(oldrelid, relids))
-	{
-		/* Ensure we have a modifiable copy */
-		relids = bms_copy(relids);
-		/* Remove old, add new */
-		relids = bms_del_member(relids, oldrelid);
-		relids = bms_add_member(relids, newrelid);
-	}
-	return relids;
-}
-
-/*
  * Adjust the targetlist entries of an inherited UPDATE operation
  *
  * The expressions have already been fixed, but we have to make sure that
@@ -2251,5 +2321,5 @@ adjust_appendrel_attrs_multilevel(PlannerInfo *root, Node *node,
 	else
 		Assert(parent_rel->reloptkind == RELOPT_BASEREL);
 	/* Now translate for this child */
-	return adjust_appendrel_attrs(root, node, appinfo);
+	return adjust_appendrel_attrs(root, node, list_make1(appinfo));
 }
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index abb7507..8510775 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -23,7 +23,9 @@
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
 #include "optimizer/planmain.h"
+#include "optimizer/prep.h"
 #include "optimizer/restrictinfo.h"
+#include "optimizer/tlist.h"
 #include "optimizer/var.h"
 #include "parser/parsetree.h"
 #include "utils/lsyscache.h"
@@ -2154,6 +2156,162 @@ create_hashjoin_path(PlannerInfo *root,
 }
 
 /*
+ * create_partition_join_path
+ * 		Creates a pathnode that represents partition-wise join for given
+ * 		partitioned join relation.
+ *
+ * This function is called when we haven't created paths for all the child
+ * joins. It estimates the number of rows and cost of the PartitionJoinPath
+ * based upon the number of rows and the costs of the list of paths for
+ * representative child-joins.
+ */
+PartitionJoinPath *
+create_partition_join_path(RelOptInfo *rel, List *subpaths,
+						   Bitmapset *required_outer)
+{
+	PartitionJoinPath *pathnode = makeNode(PartitionJoinPath);
+	double	subpath_rows = 0;
+	double	subpath_startup_cost = 0;
+	double	subpath_total_cost = 0;
+	double	child_rel_rows = 0;
+	ListCell   *lc;
+
+	Assert(rel->part_scheme);
+
+	pathnode->path.pathtype = T_Append;
+	pathnode->path.parent = rel;
+	pathnode->path.pathtarget = rel->reltarget;
+	pathnode->path.param_info = get_appendrel_parampathinfo(rel,
+															required_outer);
+	pathnode->path.pathkeys = NULL;
+
+	/* No parallel paths here. See more details in add_paths_to_append_rel() */
+	pathnode->path.parallel_aware = false;
+	pathnode->path.parallel_safe = false;
+	pathnode->path.parallel_workers = 0;
+
+	/* Accumulate the number of rows and costs from the given subpaths. */
+	foreach (lc, subpaths)
+	{
+		Path   *subpath = lfirst(lc);
+
+		subpath_rows += subpath->rows;
+		child_rel_rows += subpath->parent->rows;
+		subpath_total_cost += subpath->total_cost;
+
+		/*
+		 * Startup cost of an append relation is the startup cost of the first
+		 * subpath. Assume that the given first child will be the first child
+		 * in the final plan as well.
+		 */
+		if (lc == list_head(subpaths))
+			subpath_startup_cost = subpath->startup_cost;
+	}
+
+	/*
+	 * For a parameterized path, extrapolate the number of rows for the append
+	 * relation by considering the average selectivity of the parameterization
+	 * across the given children.
+	 */
+	if (bms_is_empty(required_outer))
+		pathnode->path.rows = rel->rows;
+	else
+		pathnode->path.rows = rel->rows * (subpath_rows / child_rel_rows);
+
+	pathnode->path.startup_cost = subpath_startup_cost;
+
+	/* Extrapolate the total cost to account for yet-to-be planned children. */
+	pathnode->path.total_cost = (subpath_total_cost * pathnode->path.rows) / subpath_rows;
+
+	return pathnode;
+}
+
+/*
+ * create_partition_join_path_with_pathkeys
+ * 		Creates a pathnode that represents an ordered partition-wise join for
+ * 		given partitioned join relation.
+ *
+ * This function is called when we haven't created paths for all the child
+ * joins. It estimates the number of rows and cost of the PartitionJoinPath
+ * based upon the number of rows and the costs of the list of paths for
+ * representative child-joins.
+ */
+PartitionJoinPath *
+create_partition_join_path_with_pathkeys(PlannerInfo *root, RelOptInfo *rel,
+						   List *subpaths, List *pathkeys,
+						   Bitmapset *required_outer)
+{
+	PartitionJoinPath *pathnode = makeNode(PartitionJoinPath);
+	double	subpath_rows = 0;
+	double	subpath_startup_cost = 0;
+	double	subpath_total_cost = 0;
+	double	child_rel_rows = 0;
+	ListCell   *lc;
+
+	Assert(rel->part_scheme);
+
+	pathnode->path.pathtype = T_MergeAppend;
+	pathnode->path.parent = rel;
+	pathnode->path.pathtarget = rel->reltarget;
+	pathnode->path.param_info = get_appendrel_parampathinfo(rel,
+															required_outer);
+	pathnode->path.pathkeys = pathkeys;
+
+	/* No parallel paths here. See more details in add_paths_to_append_rel() */
+	pathnode->path.parallel_aware = false;
+	pathnode->path.parallel_safe = false;
+	pathnode->path.parallel_workers = 0;
+
+	/* Accumulate the number of rows and costs from the given subpaths. */
+	foreach (lc, subpaths)
+	{
+		Path   *subpath = lfirst(lc);
+
+		if (pathkeys_contained_in(pathkeys, subpath->pathkeys))
+		{
+			/* Subpath is adequately ordered, we won't need to sort it */
+			subpath_startup_cost += subpath->startup_cost;
+			subpath_total_cost += subpath->total_cost;
+		}
+		else
+		{
+			/* We'll need to insert a Sort node, so include cost for that */
+			Path		sort_path;		/* dummy for result of cost_sort */
+
+			cost_sort(&sort_path,
+					  root,
+					  pathkeys,
+					  subpath->total_cost,
+					  subpath->parent->tuples,
+					  subpath->pathtarget->width,
+					  0.0,
+					  work_mem,
+					  -1);
+			subpath_startup_cost += sort_path.startup_cost;
+			subpath_total_cost += sort_path.total_cost;
+		}
+
+		subpath_rows += subpath->rows;
+		child_rel_rows += subpath->parent->rows;
+	}
+
+	/*
+	 * For a parameterized path, extrapolate the number of rows for the append
+	 * relation by considering the average selectivity of the parameterization
+	 * across the given children.
+	 */
+	if (bms_is_empty(required_outer))
+		pathnode->path.rows = rel->rows;
+	else
+		pathnode->path.rows = rel->rows * (subpath_rows / child_rel_rows);
+
+	/* Extrapolate the total cost to account for yet-to-be planned children. */
+	pathnode->path.startup_cost = (subpath_startup_cost * pathnode->path.rows) / subpath_rows;
+	pathnode->path.total_cost = (subpath_total_cost * pathnode->path.rows) / subpath_rows;
+
+	return pathnode;
+}
+/*
  * create_projection_path
  *	  Creates a pathnode that represents performing a projection.
  *
@@ -3209,3 +3367,158 @@ reparameterize_path(PlannerInfo *root, Path *path,
 	}
 	return NULL;
 }
+
+/*
+ * reparameterize_path_by_child
+ * 		Given a path parameterized by the parent of the given relation,
+ * 		translate the path to be parameterized by the given child relation.
+ *
+ * The function creates a new path of the same type as the given path, but
+ * parameterized by the given child relation. If it can not reparameterize the
+ * path as required, it returns NULL.
+ *
+ * The cost, number of rows, width and parallel path properties depend upon
+ * path->parent, which does not change during the translation. Hence those
+ * members are copied as they are.
+ */
+
+Path *
+reparameterize_path_by_child(PlannerInfo *root, Path *path,
+							  RelOptInfo *child_rel)
+{
+	Path	   *new_path;
+	ParamPathInfo   *new_ppi;
+	ParamPathInfo   *old_ppi;
+	List	   *child_aris;
+
+	/*
+	 * If the path is not parameterized by parent of the given relation, it
+	 * doesn't need reparameterization.
+	 */
+	if (!path->param_info ||
+		!bms_overlap(PATH_REQ_OUTER(path), child_rel->top_parent_relids))
+	return path;
+
+	switch (nodeTag(path))
+	{
+		case T_Path:
+			new_path = makeNode(Path);
+			memcpy(new_path, path, sizeof(Path));
+			break;
+
+		case T_HashPath:
+			new_path = (Path *) makeNode(HashPath);
+			memcpy(new_path, path, sizeof(HashPath));
+			break;
+
+		case T_MergePath:
+			new_path = (Path *) makeNode(MergePath);
+			memcpy(new_path, path, sizeof(MergePath));
+			break;
+
+		case T_NestPath:
+			new_path = (Path *) makeNode(NestPath);
+			memcpy(new_path, path, sizeof(NestPath));
+			break;
+
+		case T_IndexPath:
+			new_path = (Path *) makeNode(IndexPath);
+			memcpy(new_path, path, sizeof(IndexPath));
+			break;
+
+		case T_AppendPath:
+			new_path = (Path *) makeNode(AppendPath);
+			memcpy(new_path, path, sizeof(AppendPath));
+			break;
+
+		/*
+		 * TODO:
+		 * If this method of translation is fine add more path types here.
+		 */
+
+		default:
+			/* Path type unsupported by this function. */
+			return NULL;
+	}
+
+	/*
+	 * Gather AppendRelInfos of the base partition relations in the outer child
+	 * relation. We need those for translating parent path to that of child by
+	 * substituting parent Var nodes and relids with those of children.
+	 */
+	child_aris = find_appinfos_by_relids(root, child_rel->relids);
+
+	/* Adjust the parameterization information. */
+	old_ppi = new_path->param_info;
+	new_ppi = makeNode(ParamPathInfo);
+	new_ppi->ppi_req_outer = adjust_child_relids(bms_copy(old_ppi->ppi_req_outer),
+													 child_aris);
+	new_ppi->ppi_rows = old_ppi->ppi_rows;
+	new_ppi->ppi_clauses = (List *) adjust_join_appendrel_attrs(root,
+													 (Node *) old_ppi->ppi_clauses,
+													 child_aris);
+
+	/* Adjust the path target. */
+	new_path->pathtarget = copy_pathtarget(new_path->pathtarget);
+	new_path->pathtarget->exprs = (List *) adjust_join_appendrel_attrs(root,
+													(Node *) new_path->pathtarget->exprs,
+													 child_aris);
+	new_path->param_info = new_ppi;
+
+	/*
+	 * Change parameterization of subpaths recursively. Also carry out any
+	 * pathtype specific adjustments.
+	 */
+	switch (nodeTag(path))
+	{
+		case T_HashPath:
+		case T_MergePath:
+		case T_NestPath:
+			{
+				JoinPath *jpath = (JoinPath *)new_path;
+
+				jpath->outerjoinpath = reparameterize_path_by_child(root,
+														 jpath->outerjoinpath,
+														 child_rel);
+				jpath->innerjoinpath = reparameterize_path_by_child(root,
+														 jpath->innerjoinpath,
+														 child_rel);
+				jpath->joinrestrictinfo = (List *) adjust_join_appendrel_attrs(root,
+															(Node *) jpath->joinrestrictinfo,
+															child_aris);
+			}
+			break;
+
+		case T_AppendPath:
+			{
+				AppendPath	*apath = (AppendPath *)new_path;
+				List		*subpaths = NIL;
+				ListCell	*lc;
+
+				foreach (lc, apath->subpaths)
+					subpaths = lappend(subpaths,
+									   reparameterize_path_by_child(root,
+																	lfirst(lc),
+																	child_rel));
+				apath->subpaths = subpaths;
+			}
+
+		case T_IndexPath:
+			{
+				IndexPath *ipath = (IndexPath *)new_path;
+
+				ipath->indexquals = (List *) adjust_join_appendrel_attrs(root,
+														(Node *) ipath->indexquals,
+														child_aris);
+				ipath->indexquals = (List *) adjust_join_appendrel_attrs(root,
+														(Node *) ipath->indexorderbys,
+														child_aris);
+			}
+
+		default:
+			/* Nothing to do. */
+			break;
+	}
+
+	return new_path;
+}
diff --git a/src/backend/optimizer/util/placeholder.c b/src/backend/optimizer/util/placeholder.c
index b210914..98e95e0 100644
--- a/src/backend/optimizer/util/placeholder.c
+++ b/src/backend/optimizer/util/placeholder.c
@@ -21,6 +21,7 @@
 #include "optimizer/placeholder.h"
 #include "optimizer/planmain.h"
 #include "optimizer/var.h"
+#include "optimizer/prep.h"
 #include "utils/lsyscache.h"
 
 /* Local functions */
@@ -411,9 +412,15 @@ void
 add_placeholders_to_joinrel(PlannerInfo *root, RelOptInfo *joinrel,
 							RelOptInfo *outer_rel, RelOptInfo *inner_rel)
 {
-	Relids		relids = joinrel->relids;
+	Relids		relids;
 	ListCell   *lc;
 
+	/* PlaceHolderInfo refers to parent relids and not those of a child. */
+	if (joinrel->top_parent_relids)
+		relids = joinrel->top_parent_relids;
+	else
+		relids = joinrel->relids;
+
 	foreach(lc, root->placeholder_list)
 	{
 		PlaceHolderInfo *phinfo = (PlaceHolderInfo *) lfirst(lc);
@@ -424,9 +431,27 @@ add_placeholders_to_joinrel(PlannerInfo *root, RelOptInfo *joinrel,
 			/* Is it computable here? */
 			if (bms_is_subset(phinfo->ph_eval_at, relids))
 			{
+				PlaceHolderVar *phv = phinfo->ph_var;
+
+				/*
+				 * In case the placeholder Var refers to any of the parent
+				 * relation, translate it to refer to the corresponding child.
+				 */
+				if (bms_overlap(phv->phrels, relids) &&
+					joinrel->reloptkind == RELOPT_OTHER_JOINREL)
+				{
+					List *append_rel_infos;
+
+					append_rel_infos = find_appinfos_by_relids(root,
+															  joinrel->relids);
+					phv = (PlaceHolderVar *) adjust_join_appendrel_attrs(root,
+																  (Node *) phv,
+															 append_rel_infos);
+				}
+
 				/* Yup, add it to the output */
 				joinrel->reltarget->exprs = lappend(joinrel->reltarget->exprs,
-													phinfo->ph_var);
+													phv);
 				joinrel->reltarget->width += phinfo->ph_width;
 
 				/*
@@ -445,7 +470,7 @@ add_placeholders_to_joinrel(PlannerInfo *root, RelOptInfo *joinrel,
 				{
 					QualCost	cost;
 
-					cost_qual_eval_node(&cost, (Node *) phinfo->ph_var->phexpr,
+					cost_qual_eval_node(&cost, (Node *) phv->phexpr,
 										root);
 					joinrel->reltarget->cost.startup += cost.startup;
 					joinrel->reltarget->cost.per_tuple += cost.per_tuple;
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index f8bfa4b..a0e17d5 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -411,6 +411,21 @@ get_relation_info(PlannerInfo *root, Oid relationObjectId, bool inhparent,
 	/* Collect info about relation's foreign keys, if relevant */
 	get_relation_foreign_keys(root, rel, relation);
 
+	/*
+	 * Lookup partition scheme for the given relation. Only parent relations
+	 * can be partitioned.
+	 */
+	if (inhparent)
+		rel->part_scheme = find_partition_scheme(root, relation);
+	else
+		rel->part_scheme = NULL;
+
+	if (rel->part_scheme)
+		rel->partexprs = build_baserel_partition_key_exprs(relation,
+														   rel->relid);
+	else
+		rel->partexprs = NULL;
+
 	heap_close(relation, NoLock);
 
 	/*
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index deef560..be08b6e 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -15,15 +15,22 @@
 #include "postgres.h"
 
 #include "miscadmin.h"
+#include "catalog/heap.h"
+#include "catalog/partition.h"
 #include "optimizer/clauses.h"
 #include "optimizer/cost.h"
+#include "nodes/makefuncs.h"
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
 #include "optimizer/placeholder.h"
 #include "optimizer/plancat.h"
+#include "optimizer/prep.h"
 #include "optimizer/restrictinfo.h"
 #include "optimizer/tlist.h"
+#include "optimizer/var.h"
+#include "rewrite/rewriteManip.h"
 #include "utils/hsearch.h"
+#include "utils/rel.h"
 
 
 typedef struct JoinHashEntry
@@ -47,6 +54,11 @@ static List *subbuild_joinrel_restrictlist(RelOptInfo *joinrel,
 static List *subbuild_joinrel_joinlist(RelOptInfo *joinrel,
 						  List *joininfo_list,
 						  List *new_joininfo);
+static void set_foreign_rel_properties(RelOptInfo *joinrel,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel);
+static void build_joinrel_partition_info(RelOptInfo *joinrel,
+						  RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+						  JoinType jointype);
 
 
 /*
@@ -137,6 +149,10 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptKind reloptkind)
 	rel->baserestrictcost.per_tuple = 0;
 	rel->joininfo = NIL;
 	rel->has_eclass_joins = false;
+	rel->part_scheme = NULL;
+	rel->partexprs = NULL;
+	rel->top_parent_relids = NULL;
+	rel->part_rels = NULL;
 
 	/* Check type of rtable entry */
 	switch (rte->rtekind)
@@ -314,6 +330,56 @@ find_join_rel(PlannerInfo *root, Relids relids)
 }
 
 /*
+ * set_foreign_rel_properties
+ *		Set up foreign-join fields if outer and inner relation are foreign
+ *		tables (or joins) belonging to the same server and assigned to the same
+ *		user to check access permissions as.
+ *
+ * In addition to an exact match of userid, we allow the case where one side
+ * has zero userid (implying current user) and the other side has explicit
+ * userid that happens to equal the current user; but in that case, pushdown of
+ * the join is only valid for the current user.  The useridiscurrent field
+ * records whether we had to make such an assumption for this join or any
+ * sub-join.
+ *
+ * Otherwise these fields are left invalid, so GetForeignJoinPaths will not be
+ * called for the join relation.
+ *
+ */
+static void
+set_foreign_rel_properties(RelOptInfo *joinrel, RelOptInfo *outer_rel,
+						   RelOptInfo *inner_rel)
+{
+	if (OidIsValid(outer_rel->serverid) &&
+		inner_rel->serverid == outer_rel->serverid)
+	{
+		if (inner_rel->userid == outer_rel->userid)
+		{
+			joinrel->serverid = outer_rel->serverid;
+			joinrel->userid = outer_rel->userid;
+			joinrel->useridiscurrent = outer_rel->useridiscurrent || inner_rel->useridiscurrent;
+			joinrel->fdwroutine = outer_rel->fdwroutine;
+		}
+		else if (!OidIsValid(inner_rel->userid) &&
+				 outer_rel->userid == GetUserId())
+		{
+			joinrel->serverid = outer_rel->serverid;
+			joinrel->userid = outer_rel->userid;
+			joinrel->useridiscurrent = true;
+			joinrel->fdwroutine = outer_rel->fdwroutine;
+		}
+		else if (!OidIsValid(outer_rel->userid) &&
+				 inner_rel->userid == GetUserId())
+		{
+			joinrel->serverid = outer_rel->serverid;
+			joinrel->userid = inner_rel->userid;
+			joinrel->useridiscurrent = true;
+			joinrel->fdwroutine = outer_rel->fdwroutine;
+		}
+	}
+}
+
+/*
  * build_join_rel
  *	  Returns relation entry corresponding to the union of two given rels,
  *	  creating a new relation entry if none already exists.
@@ -363,7 +429,11 @@ build_join_rel(PlannerInfo *root,
 	 * Nope, so make one.
 	 */
 	joinrel = makeNode(RelOptInfo);
+
+	Assert(!IS_OTHER_REL(outer_rel->reloptkind) &&
+		   !IS_OTHER_REL(inner_rel->reloptkind));
 	joinrel->reloptkind = RELOPT_JOINREL;
+
 	joinrel->relids = bms_copy(joinrelids);
 	joinrel->rows = 0;
 	/* cheap startup cost is interesting iff not all tuples to be retrieved */
@@ -409,47 +479,13 @@ build_join_rel(PlannerInfo *root,
 	joinrel->baserestrictcost.per_tuple = 0;
 	joinrel->joininfo = NIL;
 	joinrel->has_eclass_joins = false;
+	joinrel->part_scheme = NULL;
+	joinrel->partexprs = NULL;
+	joinrel->top_parent_relids = NULL;
+	joinrel->part_rels = NULL;
 
-	/*
-	 * Set up foreign-join fields if outer and inner relation are foreign
-	 * tables (or joins) belonging to the same server and assigned to the same
-	 * user to check access permissions as.  In addition to an exact match of
-	 * userid, we allow the case where one side has zero userid (implying
-	 * current user) and the other side has explicit userid that happens to
-	 * equal the current user; but in that case, pushdown of the join is only
-	 * valid for the current user.  The useridiscurrent field records whether
-	 * we had to make such an assumption for this join or any sub-join.
-	 *
-	 * Otherwise these fields are left invalid, so GetForeignJoinPaths will
-	 * not be called for the join relation.
-	 */
-	if (OidIsValid(outer_rel->serverid) &&
-		inner_rel->serverid == outer_rel->serverid)
-	{
-		if (inner_rel->userid == outer_rel->userid)
-		{
-			joinrel->serverid = outer_rel->serverid;
-			joinrel->userid = outer_rel->userid;
-			joinrel->useridiscurrent = outer_rel->useridiscurrent || inner_rel->useridiscurrent;
-			joinrel->fdwroutine = outer_rel->fdwroutine;
-		}
-		else if (!OidIsValid(inner_rel->userid) &&
-				 outer_rel->userid == GetUserId())
-		{
-			joinrel->serverid = outer_rel->serverid;
-			joinrel->userid = outer_rel->userid;
-			joinrel->useridiscurrent = true;
-			joinrel->fdwroutine = outer_rel->fdwroutine;
-		}
-		else if (!OidIsValid(outer_rel->userid) &&
-				 inner_rel->userid == GetUserId())
-		{
-			joinrel->serverid = outer_rel->serverid;
-			joinrel->userid = inner_rel->userid;
-			joinrel->useridiscurrent = true;
-			joinrel->fdwroutine = outer_rel->fdwroutine;
-		}
-	}
+	/* Compute information relevant to the foreign relations. */
+	set_foreign_rel_properties(joinrel, outer_rel, inner_rel);
 
 	/*
 	 * Create a new tlist containing just the vars that need to be output from
@@ -475,6 +511,10 @@ build_join_rel(PlannerInfo *root,
 	if (bms_is_empty(joinrel->direct_lateral_relids))
 		joinrel->direct_lateral_relids = NULL;
 
+	/* Store the partition information. */
+	build_joinrel_partition_info(joinrel, outer_rel, inner_rel,
+								 sjinfo->jointype);
+
 	/*
 	 * Construct restrict and join clause lists for the new joinrel. (The
 	 * caller might or might not need the restrictlist, but I need it anyway
@@ -517,25 +557,8 @@ build_join_rel(PlannerInfo *root,
 		is_parallel_safe(root, (Node *) joinrel->reltarget->exprs))
 		joinrel->consider_parallel = true;
 
-	/*
-	 * Add the joinrel to the query's joinrel list, and store it into the
-	 * auxiliary hashtable if there is one.  NB: GEQO requires us to append
-	 * the new joinrel to the end of the list!
-	 */
-	root->join_rel_list = lappend(root->join_rel_list, joinrel);
-
-	if (root->join_rel_hash)
-	{
-		JoinHashEntry *hentry;
-		bool		found;
-
-		hentry = (JoinHashEntry *) hash_search(root->join_rel_hash,
-											   &(joinrel->relids),
-											   HASH_ENTER,
-											   &found);
-		Assert(!found);
-		hentry->join_rel = joinrel;
-	}
+	/* Add the joinrel to the query's PlannerInfo. */
+	add_join_rel(root, joinrel);
 
 	/*
 	 * Also, if dynamic-programming join search is active, add the new joinrel
@@ -555,6 +578,132 @@ build_join_rel(PlannerInfo *root,
 }
 
 /*
+ * build_child_join_rel
+ *		Builds RelOptInfo for joining given two child relations from RelOptInfo
+ *		representing the join between their parents.
+ *
+ * 'outer_rel' and 'inner_rel' are the RelOptInfos of child relations being
+ *		joined.
+ * 'parent_joinrel' is the RelOptInfo representing the join between parent
+ *		relations. Most of the members of new RelOptInfo are produced by
+ *		translating corresponding members of this RelOptInfo.
+ * 'sjinfo': context info for child join
+ * 'restrictlist': list of RestrictInfo nodes that apply to this particular
+ *		pair of joinable relations.
+ * 'join_appinfos': list of AppendRelInfo nodes for base child relations involved
+ *		in this join.
+ */
+RelOptInfo *
+build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
+						 RelOptInfo *inner_rel, RelOptInfo *parent_joinrel,
+						 JoinType jointype)
+{
+	RelOptInfo *joinrel = makeNode(RelOptInfo);
+	List	   *join_appinfos;
+
+	joinrel->reloptkind = RELOPT_OTHER_JOINREL;
+	joinrel->relids = bms_union(outer_rel->relids, inner_rel->relids);
+	joinrel->rows = 0;
+	/* cheap startup cost is interesting iff not all tuples to be retrieved */
+	joinrel->consider_startup = (root->tuple_fraction > 0);
+	joinrel->consider_param_startup = false;
+	joinrel->consider_parallel = false;
+	joinrel->reltarget = create_empty_pathtarget();
+	joinrel->pathlist = NIL;
+	joinrel->ppilist = NIL;
+	joinrel->partial_pathlist = NIL;
+	joinrel->cheapest_startup_path = NULL;
+	joinrel->cheapest_total_path = NULL;
+	joinrel->cheapest_unique_path = NULL;
+	joinrel->cheapest_parameterized_paths = NIL;
+	joinrel->direct_lateral_relids = NULL;
+	joinrel->lateral_relids = NULL;
+	joinrel->relid = 0;			/* indicates not a baserel */
+	joinrel->rtekind = RTE_JOIN;
+	joinrel->min_attr = 0;
+	joinrel->max_attr = 0;
+	joinrel->attr_needed = NULL;
+	joinrel->attr_widths = NULL;
+	joinrel->lateral_vars = NIL;
+	joinrel->lateral_referencers = NULL;
+	joinrel->indexlist = NIL;
+	joinrel->pages = 0;
+	joinrel->tuples = 0;
+	joinrel->allvisfrac = 0;
+	joinrel->subroot = NULL;
+	joinrel->subplan_params = NIL;
+	joinrel->serverid = InvalidOid;
+	joinrel->userid = InvalidOid;
+	joinrel->useridiscurrent = false;
+	joinrel->fdwroutine = NULL;
+	joinrel->fdw_private = NULL;
+	joinrel->baserestrictinfo = NIL;
+	joinrel->baserestrictcost.startup = 0;
+	joinrel->baserestrictcost.per_tuple = 0;
+	joinrel->joininfo = NIL;
+	joinrel->has_eclass_joins = false;
+	joinrel->part_scheme = NULL;
+	joinrel->partexprs = NULL;
+	joinrel->top_parent_relids = NULL;
+	joinrel->part_rels = NULL;
+
+
+	/* Only joins between other relations land here. */
+	Assert(IS_OTHER_REL(outer_rel->reloptkind) &&
+		   IS_OTHER_REL(inner_rel->reloptkind));
+
+	joinrel->top_parent_relids = bms_union(outer_rel->top_parent_relids,
+										   inner_rel->top_parent_relids);
+
+	/* Compute information relevant to foreign relations. */
+	set_foreign_rel_properties(joinrel, outer_rel, inner_rel);
+
+	/* Build targetlist */
+	build_joinrel_tlist(root, joinrel, outer_rel);
+	build_joinrel_tlist(root, joinrel, inner_rel);
+	/* Add placeholder variables. */
+	add_placeholders_to_joinrel(root, joinrel, outer_rel, inner_rel);
+
+	/* Translate joininfo. */
+	join_appinfos = find_appinfos_by_relids(root, joinrel->relids);
+	joinrel->joininfo = (List *) adjust_join_appendrel_attrs(root,
+											 (Node *) parent_joinrel->joininfo,
+																join_appinfos);
+
+	/*
+	 * Lateral relids referred in child join will be same as that referred in
+	 * the parent relation. Throw any partial result computed while building
+	 * the targetlist.
+	 */
+	bms_free(joinrel->direct_lateral_relids);
+	bms_free(joinrel->lateral_relids);
+	joinrel->direct_lateral_relids = (Relids) bms_copy(parent_joinrel->direct_lateral_relids);
+	joinrel->lateral_relids = (Relids) bms_copy(parent_joinrel->lateral_relids);
+
+	/*
+	 * If the parent joinrel has pending equivalence classes, so does the
+	 * child.
+	 */
+	joinrel->has_eclass_joins = parent_joinrel->has_eclass_joins;
+
+	/* Is the join between partitions itself partitioned? */
+	build_joinrel_partition_info(joinrel, outer_rel, inner_rel, jointype);
+
+	/* Child joinrel is parallel safe if parent is parallel safe. */
+	joinrel->consider_parallel = parent_joinrel->consider_parallel;
+
+	/* We build the join only once. */
+	Assert(!find_join_rel(root, joinrel->relids));
+
+	/* Add the relation to the PlannerInfo. */
+	add_join_rel(root, joinrel);
+
+	pfree(join_appinfos);
+
+	return joinrel;
+}
+
+/*
  * min_join_parameterization
  *
  * Determine the minimum possible parameterization of a joinrel, that is, the
@@ -609,9 +758,15 @@ static void
 build_joinrel_tlist(PlannerInfo *root, RelOptInfo *joinrel,
 					RelOptInfo *input_rel)
 {
-	Relids		relids = joinrel->relids;
+	Relids		relids;
 	ListCell   *vars;
 
+	/* attrs_needed refers to parent relids and not those of a child. */
+	if (joinrel->top_parent_relids)
+		relids = joinrel->top_parent_relids;
+	else
+		relids = joinrel->relids;
+
 	foreach(vars, input_rel->reltarget->exprs)
 	{
 		Var		   *var = (Var *) lfirst(vars);
@@ -627,23 +782,47 @@ build_joinrel_tlist(PlannerInfo *root, RelOptInfo *joinrel,
 
 		/*
 		 * Otherwise, anything in a baserel or joinrel targetlist ought to be
-		 * a Var.  (More general cases can only appear in appendrel child
-		 * rels, which will never be seen here.)
+		 * a Var or ConvertRowtypeExpr introduced while translating parent
+		 * targetlist to that of the child.
 		 */
-		if (!IsA(var, Var))
+		if (IsA(var, Var))
+		{
+			/* Get the Var's original base rel */
+			baserel = find_base_rel(root, var->varno);
+
+			/* Is it still needed above this joinrel? */
+			ndx = var->varattno - baserel->min_attr;
+		}
+		else if (IsA(var, ConvertRowtypeExpr))
+		{
+			ConvertRowtypeExpr *child_expr = (ConvertRowtypeExpr *) var;
+			Var	 *childvar = (Var *) child_expr->arg;
+
+			/*
+			 * Child's whole-row references are converted to that of parent
+			 * using ConvertRowtypeExpr. In this case, the argument to
+			 * ConvertRowtypeExpr is expected to be a whole-row reference of
+			 * the child.
+			 */
+			Assert(IsA(childvar, Var) && childvar->varattno == 0);
+
+			baserel = find_base_rel(root, childvar->varno);
+			ndx = 0 - baserel->min_attr;
+		}
+		else
 			elog(ERROR, "unexpected node type in rel targetlist: %d",
 				 (int) nodeTag(var));
 
-		/* Get the Var's original base rel */
-		baserel = find_base_rel(root, var->varno);
-
-		/* 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 */
 			joinrel->reltarget->exprs = lappend(joinrel->reltarget->exprs, var);
-			/* Vars have cost zero, so no need to adjust reltarget->cost */
+
+			/*
+			 * Vars have cost zero, so no need to adjust reltarget->cost. Even
+			 * if, it's a ConvertRowtypeExpr, it will be computed only for the
+			 * base relation, costing nothing for a join.
+			 */
 			joinrel->reltarget->width += baserel->attr_widths[ndx];
 		}
 	}
@@ -1320,3 +1499,111 @@ get_appendrel_parampathinfo(RelOptInfo *appendrel, Relids required_outer)
 
 	return ppi;
 }
+
+/*
+ * add_join_rel
+ *		Add given join relation to the list of join relations in the given
+ *		PlannerInfo. Also add it to the auxiliary hashtable if there is one.
+ */
+void
+add_join_rel(PlannerInfo *root, RelOptInfo *joinrel)
+{
+	/* GEQO requires us to append the new joinrel to the end of the list! */
+	root->join_rel_list = lappend(root->join_rel_list, joinrel);
+
+	/* store it into the auxiliary hashtable if there is one. */
+	if (root->join_rel_hash)
+	{
+		JoinHashEntry *hentry;
+		bool		found;
+
+		hentry = (JoinHashEntry *) hash_search(root->join_rel_hash,
+											   &(joinrel->relids),
+											   HASH_ENTER,
+											   &found);
+		Assert(!found);
+		hentry->join_rel = joinrel;
+	}
+}
+
+/*
+ * build_joinrel_partition_info
+ *		If the join between given partitioned relations is possibly partitioned
+ *		set the partitioning scheme and partition keys expressions for the
+ *		join.
+ *
+ * If the two relations have same partitioning scheme, their join may be
+ * partitioned and will follow the same partitioning scheme as the joining
+ * relations.
+ */
+static void
+build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
+							 RelOptInfo *inner_rel, JoinType jointype)
+{
+	int		num_pks;
+	int		cnt;
+
+	/* Nothing to do if partition-wise join technique is disabled. */
+	if (!enable_partition_wise_join)
+	{
+		joinrel->part_scheme = NULL;
+		return;
+	}
+
+	/*
+	 * The join is not partitioned, if any of the relations being joined are
+	 * not partitioned or they do not have same partitioning scheme.
+	 */
+	if (!outer_rel->part_scheme || !inner_rel->part_scheme ||
+		outer_rel->part_scheme != inner_rel->part_scheme)
+	{
+		joinrel->part_scheme = NULL;
+		return;
+	}
+
+	/*
+	 * This function will be called only once for each joinrel, hence it should
+	 * not have partition scheme, partition key expressions and array for
+	 * storing child relations set.
+	 */
+	Assert(!joinrel->part_scheme && !joinrel->partexprs &&
+		   !joinrel->part_rels);
+
+	/*
+	 * Join relation is partitioned using same partitioning scheme as the
+	 * joining relations.
+	 */
+	joinrel->part_scheme = outer_rel->part_scheme;
+	num_pks = joinrel->part_scheme->partnatts;
+
+	/*
+	 * Construct partition keys for the join.
+	 *
+	 * An INNER join between two partitioned relations is partition by key
+	 * expressions from both the relations. For tables A and B partitioned by a and b
+	 * respectively, (A INNER JOIN B ON A.a = B.b) is partitioned by both A.a
+	 * and B.b.
+	 *
+	 * An OUTER join like (A LEFT JOIN B ON A.a = B.b) may produce rows with
+	 * B.b NULL. These rows may not fit the partitioning conditions imposed on
+	 * B.b. Hence, strictly speaking, the join is not partitioned by B.b.
+	 * Strictly speaking, partition keys of an OUTER join should include
+	 * partition key expressions from the OUTER side only. Consider a join like
+	 * (A LEFT JOIN B on (A.a = B.b) LEFT JOIN C ON B.b = C.c. If we do not
+	 * include B.b as partition key expression for (AB), it prohibits us from
+	 * using partition-wise join when joining (AB) with C as there is no
+	 * equi-join between partition keys of joining relations. But two NULL
+	 * values are never equal and no two rows from mis-matching partitions can
+	 * join. Hence it's safe to include B.b as partition key expression for
+	 * (AB), even though rows in (AB) are not strictly partitioned by B.b.
+	 */
+	joinrel->partexprs = (List **) palloc0(sizeof(List *) * num_pks);
+	for (cnt = 0; cnt < num_pks; cnt++)
+	{
+		List *pkexpr = list_copy(outer_rel->partexprs[cnt]);
+
+		pkexpr = list_concat(pkexpr,
+							 list_copy(inner_rel->partexprs[cnt]));
+		joinrel->partexprs[cnt] = pkexpr;
+	}
+}
diff --git a/src/backend/postmaster/startup.c b/src/backend/postmaster/startup.c
index a7ae7e3..6787df6 100644
--- a/src/backend/postmaster/startup.c
+++ b/src/backend/postmaster/startup.c
@@ -169,7 +169,6 @@ HandleStartupProcInterrupts(void)
 		exit(1);
 }
 
-
 /* ----------------------------------
  *	Startup Process main entry point
  * ----------------------------------
diff --git a/src/backend/utils/adt/selfuncs.c b/src/backend/utils/adt/selfuncs.c
index 56943f2..16b2eac 100644
--- a/src/backend/utils/adt/selfuncs.c
+++ b/src/backend/utils/adt/selfuncs.c
@@ -3412,7 +3412,9 @@ estimate_num_groups(PlannerInfo *root, List *groupExprs, double input_rows,
 		/*
 		 * Sanity check --- don't divide by zero if empty relation.
 		 */
-		Assert(rel->reloptkind == RELOPT_BASEREL);
+		Assert(rel->reloptkind == RELOPT_BASEREL ||
+			   rel->reloptkind == RELOPT_OTHER_MEMBER_REL);
+
 		if (rel->tuples > 0)
 		{
 			/*
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index ce4eef9..edc7e58 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -877,6 +877,15 @@ static struct config_bool ConfigureNamesBool[] =
 		true,
 		NULL, NULL, NULL
 	},
+	{
+		{"enable_partition_wise_join", PGC_USERSET, QUERY_TUNING_METHOD,
+			gettext_noop("Enables partition-wise join."),
+			NULL
+		},
+		&enable_partition_wise_join,
+		true,
+		NULL, NULL, NULL
+	},
 
 	{
 		{"geqo", PGC_USERSET, QUERY_TUNING_GEQO,
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 81a4b91..19b7744 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -46,8 +46,43 @@ typedef struct PartitionDescData
 	BoundCollection		bounds;		/* collection of list or range bounds */
 } PartitionDescData;
 
+/*
+ * Partitioning scheme
+ *		Structure to hold partitioning scheme for a given relation.
+ *
+ * Multiple relations may be partitioned in the same way. The relations
+ * resulting from joining such relations may be partitioned in the same way as
+ * the joining relations. Similarly, relations derived from such relations by
+ * grouping, sorting may be partitioned in the same way as the underlying
+ * scan relations. All such relations partitioned in the same way share the
+ * partitioning scheme.
+ *
+ * PlannerInfo stores a list of distinct "canonical" partitioning schemes.
+ * RelOptInfo of a partitioned relation holds the pointer to "canonical"
+ * partitioning scheme.
+ */
+typedef struct PartitionSchemeData
+{
+	/* Information about partitions */
+	int			nparts;	/* number of partitions */
+	BoundCollection bounds; /* Partition bounds/lists */
+
+	/* Information about partition keys */
+	char		strategy;		/* partition strategy */
+	int16		partnatts;		/* number of partition attributes */
+	Oid		   *partopfamily;	/* OIDs of operator families */
+	Oid		   *partopcintype;	/* OIDs of opclass declared input data types */
+	Oid		   *key_types;		/* OIDs of partition key data types. */
+	int32	   *key_typmods;	/* typmods of partition keys. */
+	Oid		   *key_collations;	/* OIDs of collations of partition keys. */
+} PartitionSchemeData;
+
 typedef struct PartitionDescData *PartitionDesc;
 typedef struct PartitionTreeNodeData *PartitionTreeNode;
+typedef struct PartitionSchemeData *PartitionScheme;
+
+/* Include here to avoid circular dependency with relation.h. */
+struct PlannerInfo;
 
 /* relcache support for partition key information */
 extern void RelationBuildPartitionKey(Relation relation);
@@ -84,4 +119,9 @@ extern int get_partition_for_tuple(PartitionTreeNode ptnode,
 					TupleTableSlot *slot,
 					EState *estate,
 					Oid *failed_at);
+extern List **build_baserel_partition_key_exprs(Relation relation,
+												Index varno);
+extern PartitionScheme find_partition_scheme(struct PlannerInfo *root,
+											 Relation rel);
+
 #endif   /* PARTITION_H */
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index 2f9e7d3..94bce51 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -238,6 +238,7 @@ typedef enum NodeTag
 	T_NestPath,
 	T_MergePath,
 	T_HashPath,
+	T_PartitionJoinPath,
 	T_AppendPath,
 	T_MergeAppendPath,
 	T_ResultPath,
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 3a1255a..18f1e5c 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 "catalog/partition.h"
 #include "lib/stringinfo.h"
 #include "nodes/params.h"
 #include "nodes/parsenodes.h"
@@ -263,6 +264,9 @@ typedef struct PlannerInfo
 
 	List	   *initial_rels;	/* RelOptInfos we are now trying to join */
 
+	List	   *part_schemes;	/* Canonicalised partition schemes
+								 * used in the query. */
+
 	/* Use fetch_upper_rel() to get any particular upper rel */
 	List	   *upper_rels[UPPERREL_FINAL + 1]; /* upper-rel RelOptInfos */
 
@@ -352,6 +356,11 @@ typedef struct PlannerInfo
  * handling join alias Vars.  Currently this is not needed because all join
  * alias Vars are expanded to non-aliased form during preprocess_expression.
  *
+ * We also have relations representing joins between child relations of
+ * different partitioned tables. These relations are not added to
+ * join_rel_level lists as they are not joined directly by the dynamic
+ * programming algorithm.
+ *
  * There is also a RelOptKind for "upper" relations, which are RelOptInfos
  * that describe post-scan/join processing steps, such as aggregation.
  * Many of the fields in these RelOptInfos are meaningless, but their Path
@@ -471,10 +480,19 @@ typedef enum RelOptKind
 	RELOPT_BASEREL,
 	RELOPT_JOINREL,
 	RELOPT_OTHER_MEMBER_REL,
+	RELOPT_OTHER_JOINREL,
 	RELOPT_UPPER_REL,
 	RELOPT_DEADREL
 } RelOptKind;
 
+#define IS_OTHER_REL(reloptkind) \
+	((reloptkind) == RELOPT_OTHER_MEMBER_REL || \
+	 (reloptkind) == RELOPT_OTHER_JOINREL)
+
+#define IS_JOIN_REL(rel)	\
+	((rel->reloptkind) == RELOPT_JOINREL || \
+	 (rel->reloptkind) == RELOPT_OTHER_JOINREL)
+
 typedef struct RelOptInfo
 {
 	NodeTag		type;
@@ -542,6 +560,27 @@ typedef struct RelOptInfo
 	List	   *joininfo;		/* RestrictInfo structures for join clauses
 								 * involving this rel */
 	bool		has_eclass_joins;		/* T means joininfo is incomplete */
+
+	/* For partitioned relations. */
+	PartitionScheme	    part_scheme;	/* Partitioning scheme. */
+	struct RelOptInfo **part_rels;		/* Array of RelOptInfos of partitions,
+										 * stored in the same order as bounds
+										 * or lists in PartitionScheme.
+										 */
+	List			  **partexprs;		/* Array of list of partition key
+										 * expressions. For base relations
+										 * these are one element lists. For
+										 * join there may be as many elements
+										 * as the number of joining
+										 * relations.
+										 */
+	List			   *partitioned_joins;	/* List of join orders which yield
+											 * relations partitioned by above
+											 * partition scheme.
+											 */
+
+	/* Set only for "other" base or join relations. */
+	Relids		top_parent_relids;		/* Relids of topmost parents. */
 } RelOptInfo;
 
 /*
@@ -1469,6 +1508,14 @@ typedef struct LimitPath
 	Node	   *limitCount;		/* COUNT parameter, or NULL if none */
 } LimitPath;
 
+/*
+ * PartitionJoinPath represents partition-wise join between two partitioned
+ * tables.
+ */
+typedef struct PartitionJoinPath
+{
+	Path		path;
+} PartitionJoinPath;
 
 /*
  * Restriction clause info.
@@ -1785,6 +1832,20 @@ typedef struct SpecialJoinInfo
 } SpecialJoinInfo;
 
 /*
+ * Partitioned join information
+ *
+ * Saves information about relations which can be joined partition-wise and
+ * thus produce result partitioned by the partition scheme of the relation.
+ */
+typedef struct PartitionedJoin
+{
+	RelOptInfo	   *rel1;
+	RelOptInfo	   *rel2;
+	SpecialJoinInfo	   *sjinfo;
+	List		   *restrictlist;
+} PartitionedJoin;
+
+/*
  * Append-relation info.
  *
  * When we expand an inheritable table or a UNION-ALL subselect into an
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 2a4df2f..1069726 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -66,6 +66,7 @@ extern bool enable_nestloop;
 extern bool enable_material;
 extern bool enable_mergejoin;
 extern bool enable_hashjoin;
+extern bool enable_partition_wise_join;
 extern int	constraint_exclusion;
 
 extern double clamp_row_est(double nrows);
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 71d9154..44dc801 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -225,10 +225,17 @@ extern LimitPath *create_limit_path(PlannerInfo *root, RelOptInfo *rel,
 				  Path *subpath,
 				  Node *limitOffset, Node *limitCount,
 				  int64 offset_est, int64 count_est);
+extern PartitionJoinPath *create_partition_join_path(RelOptInfo *rel,
+								 List *subpaths, Bitmapset *required_outer);
+extern PartitionJoinPath *create_partition_join_path_with_pathkeys(PlannerInfo *root,
+								 RelOptInfo *rel, List *subpaths,
+								 List *pathkeys, Bitmapset *required_outer);
 
 extern Path *reparameterize_path(PlannerInfo *root, Path *path,
 					Relids required_outer,
 					double loop_count);
+extern Path *reparameterize_path_by_child(PlannerInfo *root, Path *path,
+					RelOptInfo *child_rel);
 
 /*
  * prototypes for relnode.c
@@ -267,5 +274,10 @@ extern ParamPathInfo *get_joinrel_parampathinfo(PlannerInfo *root,
 						  List **restrict_clauses);
 extern ParamPathInfo *get_appendrel_parampathinfo(RelOptInfo *appendrel,
 							Relids required_outer);
+extern RelOptInfo *build_child_join_rel(PlannerInfo *root,
+						 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+						 RelOptInfo *parent_joinrel, JoinType jointype);
+
+extern void add_join_rel(PlannerInfo *root, RelOptInfo *joinrel);
 
 #endif   /* PATHNODE_H */
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 44abe83..5d7bcd9 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -53,6 +53,8 @@ extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 					 List *initial_rels);
 
 extern void generate_gather_paths(PlannerInfo *root, RelOptInfo *rel);
+extern void generate_partition_wise_join_paths(PlannerInfo *root,
+											   RelOptInfo *rel);
 
 #ifdef OPTIMIZER_DEBUG
 extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
@@ -104,6 +106,9 @@ extern bool have_join_order_restriction(PlannerInfo *root,
 							RelOptInfo *rel1, RelOptInfo *rel2);
 extern bool have_dangerous_phv(PlannerInfo *root,
 				   Relids outer_relids, Relids inner_params);
+extern void add_paths_to_child_joinrel(PlannerInfo *root,
+									   RelOptInfo *parent_joinrel,
+									   int child_id);
 
 /*
  * equivclass.c
@@ -219,4 +224,6 @@ extern PathKey *make_canonical_pathkey(PlannerInfo *root,
 					   EquivalenceClass *eclass, Oid opfamily,
 					   int strategy, bool nulls_first);
 
+extern Relids adjust_child_relids(Relids relids, List *append_rel_infos);
+
 #endif   /* PATHS_H */
diff --git a/src/include/optimizer/prep.h b/src/include/optimizer/prep.h
index fb35b68..58df2f4 100644
--- a/src/include/optimizer/prep.h
+++ b/src/include/optimizer/prep.h
@@ -28,6 +28,9 @@ extern void flatten_simple_union_all(PlannerInfo *root);
 extern void reduce_outer_joins(PlannerInfo *root);
 extern Relids get_relids_in_jointree(Node *jtnode, bool include_joins);
 extern Relids get_relids_for_join(PlannerInfo *root, int joinrelid);
+extern Node *adjust_join_appendrel_attrs(PlannerInfo *root, Node *node,
+										 List *append_rel_infos);
+extern List *find_appinfos_by_relids(PlannerInfo *root, Relids relids);
 
 /*
  * prototypes for prepqual.c
@@ -58,7 +61,7 @@ extern RelOptInfo *plan_set_operations(PlannerInfo *root);
 extern void expand_inherited_tables(PlannerInfo *root);
 
 extern Node *adjust_appendrel_attrs(PlannerInfo *root, Node *node,
-					   AppendRelInfo *appinfo);
+					   List *appinfos);
 
 extern Node *adjust_appendrel_attrs_multilevel(PlannerInfo *root, Node *node,
 								  RelOptInfo *child_rel);
diff --git a/src/test/regress/expected/multi_level_partition_join.out b/src/test/regress/expected/multi_level_partition_join.out
new file mode 100644
index 0000000..2fcf779
--- /dev/null
+++ b/src/test/regress/expected/multi_level_partition_join.out
@@ -0,0 +1,458 @@
+--
+-- multi-leveled partitions
+--
+CREATE TABLE prt1_l (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_l_p1 PARTITION OF prt1_l FOR VALUES START (0) END (250) PARTITION BY RANGE (b);
+CREATE TABLE prt1_l_p1_p1 PARTITION OF prt1_l_p1 FOR VALUES START (0) END (100);
+CREATE TABLE prt1_l_p1_p2 PARTITION OF prt1_l_p1 FOR VALUES START (100) END (250);
+CREATE TABLE prt1_l_p2 PARTITION OF prt1_l FOR VALUES START (250) END (500) PARTITION BY RANGE (c);
+CREATE TABLE prt1_l_p2_p1 PARTITION OF prt1_l_p2 FOR VALUES START ('0250') END ('0400');
+CREATE TABLE prt1_l_p2_p2 PARTITION OF prt1_l_p2 FOR VALUES START ('0400') END ('0500');
+CREATE TABLE prt1_l_p3 PARTITION OF prt1_l FOR VALUES START (500) END (600) PARTITION BY RANGE ((b + a));
+CREATE TABLE prt1_l_p3_p1 PARTITION OF prt1_l_p3 FOR VALUES START (1000) END (1100);
+CREATE TABLE prt1_l_p3_p2 PARTITION OF prt1_l_p3 FOR VALUES START (1100) END (1200);
+INSERT INTO prt1_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_l;
+ANALYZE prt1_l_p1;
+ANALYZE prt1_l_p1_p1;
+ANALYZE prt1_l_p1_p2;
+ANALYZE prt1_l_p2;
+ANALYZE prt1_l_p2_p1;
+ANALYZE prt1_l_p2_p2;
+ANALYZE prt1_l_p3;
+ANALYZE prt1_l_p3_p1;
+ANALYZE prt1_l_p3_p2;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_l AS SELECT * FROM prt1_l;
+CREATE TABLE prt2_l (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_l_p1 PARTITION OF prt2_l FOR VALUES START (0) END (250) PARTITION BY RANGE (a);
+CREATE TABLE prt2_l_p1_p1 PARTITION OF prt2_l_p1 FOR VALUES START (0) END (100);
+CREATE TABLE prt2_l_p1_p2 PARTITION OF prt2_l_p1 FOR VALUES START (100) END (250);
+CREATE TABLE prt2_l_p2 PARTITION OF prt2_l FOR VALUES START (250) END (500) PARTITION BY RANGE (c);
+CREATE TABLE prt2_l_p2_p1 PARTITION OF prt2_l_p2 FOR VALUES START ('0250') END ('0400');
+CREATE TABLE prt2_l_p2_p2 PARTITION OF prt2_l_p2 FOR VALUES START ('0400') END ('0500');
+CREATE TABLE prt2_l_p3 PARTITION OF prt2_l FOR VALUES START (500) END (600) PARTITION BY RANGE ((a + b));
+CREATE TABLE prt2_l_p3_p1 PARTITION OF prt2_l_p3 FOR VALUES START (1000) END (1100);
+CREATE TABLE prt2_l_p3_p2 PARTITION OF prt2_l_p3 FOR VALUES START (1100) END (1200);
+INSERT INTO prt2_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_l;
+ANALYZE prt2_l_p1;
+ANALYZE prt2_l_p1_p1;
+ANALYZE prt2_l_p1_p2;
+ANALYZE prt2_l_p2;
+ANALYZE prt2_l_p2_p1;
+ANALYZE prt2_l_p2_p2;
+ANALYZE prt2_l_p3;
+ANALYZE prt2_l_p3_p1;
+ANALYZE prt2_l_p3_p2;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_l AS SELECT * FROM prt2_l;
+EXPLAIN (VERBOSE, 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 = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                                                      QUERY PLAN                                                                      
+------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((t2.b = t1.a) AND (t2.a = t1.b) AND ((t2.c)::text = (t1.c)::text) AND ((t2.a + t2.b) = (t1.b + t1.a)))
+               ->  Seq Scan on public.prt2_l_p1_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_l_p1_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.a = t1_1.b) AND ((t2_1.c)::text = (t1_1.c)::text) AND ((t2_1.a + t2_1.b) = (t1_1.b + t1_1.a)))
+               ->  Seq Scan on public.prt2_l_p1_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_l_p1_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.a = t1_2.b) AND ((t2_2.c)::text = (t1_2.c)::text) AND ((t2_2.a + t2_2.b) = (t1_2.b + t1_2.a)))
+               ->  Seq Scan on public.prt2_l_p2_p1 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_l_p2_p1 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_3.a, t1_3.c, t2_3.b, t2_3.c
+               Hash Cond: ((t2_3.b = t1_3.a) AND (t2_3.a = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text) AND ((t2_3.a + t2_3.b) = (t1_3.b + t1_3.a)))
+               ->  Seq Scan on public.prt2_l_p2_p2 t2_3
+                     Output: t2_3.b, t2_3.c, t2_3.a
+               ->  Hash
+                     Output: t1_3.a, t1_3.c, t1_3.b
+                     ->  Seq Scan on public.prt1_l_p2_p2 t1_3
+                           Output: t1_3.a, t1_3.c, t1_3.b
+                           Filter: ((t1_3.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_4.a, t1_4.c, t2_4.b, t2_4.c
+               Hash Cond: ((t2_4.b = t1_4.a) AND (t2_4.a = t1_4.b) AND ((t2_4.c)::text = (t1_4.c)::text) AND ((t2_4.a + t2_4.b) = (t1_4.b + t1_4.a)))
+               ->  Seq Scan on public.prt2_l_p3_p1 t2_4
+                     Output: t2_4.b, t2_4.c, t2_4.a
+               ->  Hash
+                     Output: t1_4.a, t1_4.c, t1_4.b
+                     ->  Seq Scan on public.prt1_l_p3_p1 t1_4
+                           Output: t1_4.a, t1_4.c, t1_4.b
+                           Filter: ((t1_4.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_5.a, t1_5.c, t2_5.b, t2_5.c
+               Hash Cond: ((t2_5.b = t1_5.a) AND (t2_5.a = t1_5.b) AND ((t2_5.c)::text = (t1_5.c)::text) AND ((t2_5.a + t2_5.b) = (t1_5.b + t1_5.a)))
+               ->  Seq Scan on public.prt2_l_p3_p2 t2_5
+                     Output: t2_5.b, t2_5.c, t2_5.a
+               ->  Hash
+                     Output: t1_5.a, t1_5.c, t1_5.b
+                     ->  Seq Scan on public.prt1_l_p3_p2 t1_5
+                           Output: t1_5.a, t1_5.c, t1_5.b
+                           Filter: ((t1_5.a % 25) = 0)
+(64 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1, prt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1, uprt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 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)
+
+EXPLAIN (VERBOSE, 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                                                      QUERY PLAN                                                                      
+------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((t2.b = t1.a) AND (t2.a = t1.b) AND ((t2.c)::text = (t1.c)::text) AND ((t2.a + t2.b) = (t1.b + t1.a)))
+               ->  Seq Scan on public.prt2_l_p1_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_l_p1_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.a = t1_1.b) AND ((t2_1.c)::text = (t1_1.c)::text) AND ((t2_1.a + t2_1.b) = (t1_1.b + t1_1.a)))
+               ->  Seq Scan on public.prt2_l_p1_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_l_p1_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.a = t1_2.b) AND ((t2_2.c)::text = (t1_2.c)::text) AND ((t2_2.a + t2_2.b) = (t1_2.b + t1_2.a)))
+               ->  Seq Scan on public.prt2_l_p2_p1 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_l_p2_p1 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_3.a, t1_3.c, t2_3.b, t2_3.c
+               Hash Cond: ((t2_3.b = t1_3.a) AND (t2_3.a = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text) AND ((t2_3.a + t2_3.b) = (t1_3.b + t1_3.a)))
+               ->  Seq Scan on public.prt2_l_p2_p2 t2_3
+                     Output: t2_3.b, t2_3.c, t2_3.a
+               ->  Hash
+                     Output: t1_3.a, t1_3.c, t1_3.b
+                     ->  Seq Scan on public.prt1_l_p2_p2 t1_3
+                           Output: t1_3.a, t1_3.c, t1_3.b
+                           Filter: ((t1_3.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_4.a, t1_4.c, t2_4.b, t2_4.c
+               Hash Cond: ((t2_4.b = t1_4.a) AND (t2_4.a = t1_4.b) AND ((t2_4.c)::text = (t1_4.c)::text) AND ((t2_4.a + t2_4.b) = (t1_4.b + t1_4.a)))
+               ->  Seq Scan on public.prt2_l_p3_p1 t2_4
+                     Output: t2_4.b, t2_4.c, t2_4.a
+               ->  Hash
+                     Output: t1_4.a, t1_4.c, t1_4.b
+                     ->  Seq Scan on public.prt1_l_p3_p1 t1_4
+                           Output: t1_4.a, t1_4.c, t1_4.b
+                           Filter: ((t1_4.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_5.a, t1_5.c, t2_5.b, t2_5.c
+               Hash Cond: ((t2_5.b = t1_5.a) AND (t2_5.a = t1_5.b) AND ((t2_5.c)::text = (t1_5.c)::text) AND ((t2_5.a + t2_5.b) = (t1_5.b + t1_5.a)))
+               ->  Seq Scan on public.prt2_l_p3_p2 t2_5
+                     Output: t2_5.b, t2_5.c, t2_5.a
+               ->  Hash
+                     Output: t1_5.a, t1_5.c, t1_5.b
+                     ->  Seq Scan on public.prt1_l_p3_p2 t1_5
+                           Output: t1_5.a, t1_5.c, t1_5.b
+                           Filter: ((t1_5.a % 25) = 0)
+(64 rows)
+
+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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 LEFT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+                                                                         QUERY PLAN                                                                         
+------------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: ((t1.a = t2.b) AND (t1.b = t2.a) AND ((t1.c)::text = (t2.c)::text) AND ((t1.b + t1.a) = (t2.a + t2.b)))
+                     ->  Seq Scan on public.prt1_l_p1_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                     ->  Hash
+                           Output: t2.b, t2.c, t2.a
+                           ->  Seq Scan on public.prt2_l_p1_p1 t2
+                                 Output: t2.b, t2.c, t2.a
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: ((t1_1.a = t2_1.b) AND (t1_1.b = t2_1.a) AND ((t1_1.c)::text = (t2_1.c)::text) AND ((t1_1.b + t1_1.a) = (t2_1.a + t2_1.b)))
+                     ->  Seq Scan on public.prt1_l_p1_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c, t2_1.a
+                           ->  Seq Scan on public.prt2_l_p1_p2 t2_1
+                                 Output: t2_1.b, t2_1.c, t2_1.a
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: ((t1_2.a = t2_2.b) AND (t1_2.b = t2_2.a) AND ((t1_2.c)::text = (t2_2.c)::text) AND ((t1_2.b + t1_2.a) = (t2_2.a + t2_2.b)))
+                     ->  Seq Scan on public.prt1_l_p2_p1 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c, t2_2.a
+                           ->  Seq Scan on public.prt2_l_p2_p1 t2_2
+                                 Output: t2_2.b, t2_2.c, t2_2.a
+                                 Filter: ((t2_2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_3.b, t2_3.c, t1_3.a, t1_3.c
+                     Hash Cond: ((t1_3.a = t2_3.b) AND (t1_3.b = t2_3.a) AND ((t1_3.c)::text = (t2_3.c)::text) AND ((t1_3.b + t1_3.a) = (t2_3.a + t2_3.b)))
+                     ->  Seq Scan on public.prt1_l_p2_p2 t1_3
+                           Output: t1_3.a, t1_3.c, t1_3.b
+                     ->  Hash
+                           Output: t2_3.b, t2_3.c, t2_3.a
+                           ->  Seq Scan on public.prt2_l_p2_p2 t2_3
+                                 Output: t2_3.b, t2_3.c, t2_3.a
+                                 Filter: ((t2_3.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_4.b, t2_4.c, t1_4.a, t1_4.c
+                     Hash Cond: ((t1_4.a = t2_4.b) AND (t1_4.b = t2_4.a) AND ((t1_4.c)::text = (t2_4.c)::text) AND ((t1_4.b + t1_4.a) = (t2_4.a + t2_4.b)))
+                     ->  Seq Scan on public.prt1_l_p3_p1 t1_4
+                           Output: t1_4.a, t1_4.c, t1_4.b
+                     ->  Hash
+                           Output: t2_4.b, t2_4.c, t2_4.a
+                           ->  Seq Scan on public.prt2_l_p3_p1 t2_4
+                                 Output: t2_4.b, t2_4.c, t2_4.a
+                                 Filter: ((t2_4.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_5.b, t2_5.c, t1_5.a, t1_5.c
+                     Hash Cond: ((t1_5.a = t2_5.b) AND (t1_5.b = t2_5.a) AND ((t1_5.c)::text = (t2_5.c)::text) AND ((t1_5.b + t1_5.a) = (t2_5.a + t2_5.b)))
+                     ->  Seq Scan on public.prt1_l_p3_p2 t1_5
+                           Output: t1_5.a, t1_5.c, t1_5.b
+                     ->  Hash
+                           Output: t2_5.b, t2_5.c, t2_5.a
+                           ->  Seq Scan on public.prt2_l_p3_p2 t2_5
+                                 Output: t2_5.b, t2_5.c, t2_5.a
+                                 Filter: ((t2_5.b % 25) = 0)
+(66 rows)
+
+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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 RIGHT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+                                                                                                              QUERY PLAN                                                                                                              
+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_l_p1_p1.a, prt1_l_p1_p1.c, prt2_l_p1_p1.b, prt2_l_p1_p1.c
+   Sort Key: prt1_l_p1_p1.a, prt2_l_p1_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_l_p1_p1.a, prt1_l_p1_p1.c, prt2_l_p1_p1.b, prt2_l_p1_p1.c
+               Hash Cond: ((prt1_l_p1_p1.a = prt2_l_p1_p1.b) AND (prt1_l_p1_p1.b = prt2_l_p1_p1.a) AND ((prt1_l_p1_p1.c)::text = (prt2_l_p1_p1.c)::text) AND ((prt1_l_p1_p1.b + prt1_l_p1_p1.a) = (prt2_l_p1_p1.a + prt2_l_p1_p1.b)))
+               ->  Seq Scan on public.prt1_l_p1_p1
+                     Output: prt1_l_p1_p1.a, prt1_l_p1_p1.c, prt1_l_p1_p1.b
+                     Filter: ((prt1_l_p1_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p1_p1.b, prt2_l_p1_p1.c, prt2_l_p1_p1.a
+                     ->  Seq Scan on public.prt2_l_p1_p1
+                           Output: prt2_l_p1_p1.b, prt2_l_p1_p1.c, prt2_l_p1_p1.a
+                           Filter: ((prt2_l_p1_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_l_p1_p2.a, prt1_l_p1_p2.c, prt2_l_p1_p2.b, prt2_l_p1_p2.c
+               Hash Cond: ((prt1_l_p1_p2.a = prt2_l_p1_p2.b) AND (prt1_l_p1_p2.b = prt2_l_p1_p2.a) AND ((prt1_l_p1_p2.c)::text = (prt2_l_p1_p2.c)::text) AND ((prt1_l_p1_p2.b + prt1_l_p1_p2.a) = (prt2_l_p1_p2.a + prt2_l_p1_p2.b)))
+               ->  Seq Scan on public.prt1_l_p1_p2
+                     Output: prt1_l_p1_p2.a, prt1_l_p1_p2.c, prt1_l_p1_p2.b
+                     Filter: ((prt1_l_p1_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p1_p2.b, prt2_l_p1_p2.c, prt2_l_p1_p2.a
+                     ->  Seq Scan on public.prt2_l_p1_p2
+                           Output: prt2_l_p1_p2.b, prt2_l_p1_p2.c, prt2_l_p1_p2.a
+                           Filter: ((prt2_l_p1_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_l_p2_p1.a, prt1_l_p2_p1.c, prt2_l_p2_p1.b, prt2_l_p2_p1.c
+               Hash Cond: ((prt1_l_p2_p1.a = prt2_l_p2_p1.b) AND (prt1_l_p2_p1.b = prt2_l_p2_p1.a) AND ((prt1_l_p2_p1.c)::text = (prt2_l_p2_p1.c)::text) AND ((prt1_l_p2_p1.b + prt1_l_p2_p1.a) = (prt2_l_p2_p1.a + prt2_l_p2_p1.b)))
+               ->  Seq Scan on public.prt1_l_p2_p1
+                     Output: prt1_l_p2_p1.a, prt1_l_p2_p1.c, prt1_l_p2_p1.b
+                     Filter: ((prt1_l_p2_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p2_p1.b, prt2_l_p2_p1.c, prt2_l_p2_p1.a
+                     ->  Seq Scan on public.prt2_l_p2_p1
+                           Output: prt2_l_p2_p1.b, prt2_l_p2_p1.c, prt2_l_p2_p1.a
+                           Filter: ((prt2_l_p2_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_l_p2_p2.a, prt1_l_p2_p2.c, prt2_l_p2_p2.b, prt2_l_p2_p2.c
+               Hash Cond: ((prt1_l_p2_p2.a = prt2_l_p2_p2.b) AND (prt1_l_p2_p2.b = prt2_l_p2_p2.a) AND ((prt1_l_p2_p2.c)::text = (prt2_l_p2_p2.c)::text) AND ((prt1_l_p2_p2.b + prt1_l_p2_p2.a) = (prt2_l_p2_p2.a + prt2_l_p2_p2.b)))
+               ->  Seq Scan on public.prt1_l_p2_p2
+                     Output: prt1_l_p2_p2.a, prt1_l_p2_p2.c, prt1_l_p2_p2.b
+                     Filter: ((prt1_l_p2_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p2_p2.b, prt2_l_p2_p2.c, prt2_l_p2_p2.a
+                     ->  Seq Scan on public.prt2_l_p2_p2
+                           Output: prt2_l_p2_p2.b, prt2_l_p2_p2.c, prt2_l_p2_p2.a
+                           Filter: ((prt2_l_p2_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_l_p3_p1.a, prt1_l_p3_p1.c, prt2_l_p3_p1.b, prt2_l_p3_p1.c
+               Hash Cond: ((prt1_l_p3_p1.a = prt2_l_p3_p1.b) AND (prt1_l_p3_p1.b = prt2_l_p3_p1.a) AND ((prt1_l_p3_p1.c)::text = (prt2_l_p3_p1.c)::text) AND ((prt1_l_p3_p1.b + prt1_l_p3_p1.a) = (prt2_l_p3_p1.a + prt2_l_p3_p1.b)))
+               ->  Seq Scan on public.prt1_l_p3_p1
+                     Output: prt1_l_p3_p1.a, prt1_l_p3_p1.c, prt1_l_p3_p1.b
+                     Filter: ((prt1_l_p3_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p3_p1.b, prt2_l_p3_p1.c, prt2_l_p3_p1.a
+                     ->  Seq Scan on public.prt2_l_p3_p1
+                           Output: prt2_l_p3_p1.b, prt2_l_p3_p1.c, prt2_l_p3_p1.a
+                           Filter: ((prt2_l_p3_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_l_p3_p2.a, prt1_l_p3_p2.c, prt2_l_p3_p2.b, prt2_l_p3_p2.c
+               Hash Cond: ((prt1_l_p3_p2.a = prt2_l_p3_p2.b) AND (prt1_l_p3_p2.b = prt2_l_p3_p2.a) AND ((prt1_l_p3_p2.c)::text = (prt2_l_p3_p2.c)::text) AND ((prt1_l_p3_p2.b + prt1_l_p3_p2.a) = (prt2_l_p3_p2.a + prt2_l_p3_p2.b)))
+               ->  Seq Scan on public.prt1_l_p3_p2
+                     Output: prt1_l_p3_p2.a, prt1_l_p3_p2.c, prt1_l_p3_p2.b
+                     Filter: ((prt1_l_p3_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p3_p2.b, prt2_l_p3_p2.c, prt2_l_p3_p2.a
+                     ->  Seq Scan on public.prt2_l_p3_p2
+                           Output: prt2_l_p3_p2.b, prt2_l_p3_p2.c, prt2_l_p3_p2.a
+                           Filter: ((prt2_l_p3_p2.b % 25) = 0)
+(70 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_l t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_l t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.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)
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
new file mode 100644
index 0000000..eb2015c
--- /dev/null
+++ b/src/test/regress/expected/partition_join.out
@@ -0,0 +1,5043 @@
+--
+-- PARTITION_JOIN
+-- Test partition-wise join between partitioned tables
+--
+--
+-- partitioned by a single column
+--
+CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES START (0) END (250);
+CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES START (500) END (600);
+CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES START (250) END (500);
+INSERT INTO prt1 SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1 AS SELECT * FROM prt1;
+CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES START (0) END (250);
+CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES START (250) END (500);
+CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES START (500) END (600);
+INSERT INTO prt2 SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+CREATE TABLE uprt2 AS SELECT * FROM prt2;
+-- inner join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on public.prt2_p1 t2
+                     Output: t2.b, t2.c
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+               Hash Cond: (t2_1.b = t1_2.a)
+               ->  Seq Scan on public.prt2_p2 t2_1
+                     Output: t2_1.b, t2_1.c
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+               Hash Cond: (t2_2.b = t1_1.a)
+               ->  Seq Scan on public.prt2_p3 t2_2
+                     Output: t2_2.b, t2_2.c
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 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)
+
+-- left outer join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on public.prt2_p1 t2
+                     Output: t2.b, t2.c
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+               Hash Cond: (t2_1.b = t1_2.a)
+               ->  Seq Scan on public.prt2_p2 t2_1
+                     Output: t2_1.b, t2_1.c
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+               Hash Cond: (t2_2.b = t1_1.a)
+               ->  Seq Scan on public.prt2_p3 t2_2
+                     Output: t2_2.b, t2_2.c
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+-- right outer join
+EXPLAIN (VERBOSE, 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.b % 25 = 0 ORDER BY t1.a, t2.b;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: (t1.a = t2.b)
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t2.b, t2.c
+                           ->  Seq Scan on public.prt2_p1 t2
+                                 Output: t2.b, t2.c
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Hash Cond: (t1_2.a = t2_1.b)
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c
+                           ->  Seq Scan on public.prt2_p2 t2_1
+                                 Output: t2_1.b, t2_1.c
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Hash Cond: (t1_1.a = t2_2.b)
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c
+                           ->  Seq Scan on public.prt2_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+                                 Filter: ((t2_2.b % 25) = 0)
+(36 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 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)
+
+-- full outer join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+   Sort Key: prt1_p1.a, prt2_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+               Hash Cond: (prt1_p1.a = prt2_p1.b)
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c
+                     Filter: ((prt1_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p1.b, prt2_p1.c
+                     ->  Seq Scan on public.prt2_p1
+                           Output: prt2_p1.b, prt2_p1.c
+                           Filter: ((prt2_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt1_p2.a = prt2_p2.b)
+               ->  Seq Scan on public.prt1_p2
+                     Output: prt1_p2.a, prt1_p2.c
+                     Filter: ((prt1_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p2.b, prt2_p2.c
+                     ->  Seq Scan on public.prt2_p2
+                           Output: prt2_p2.b, prt2_p2.c
+                           Filter: ((prt2_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c
+               Hash Cond: (prt1_p3.a = prt2_p3.b)
+               ->  Seq Scan on public.prt1_p3
+                     Output: prt1_p3.a, prt1_p3.c
+                     Filter: ((prt1_p3.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p3.b, prt2_p3.c
+                     ->  Seq Scan on public.prt2_p3
+                           Output: prt2_p3.b, prt2_p3.c
+                           Filter: ((prt2_p3.b % 25) = 0)
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 t2 WHERE t2.b % 25 = 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)
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+   Sort Key: prt1_p1.a, prt2_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+               Hash Cond: (prt1_p1.a = prt2_p1.b)
+               Filter: (((50) = prt1_p1.b) OR ((75) = prt2_p1.b))
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c, prt1_p1.b, 50
+                     Filter: ((prt1_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p1.b, prt2_p1.c, (75)
+                     ->  Seq Scan on public.prt2_p1
+                           Output: prt2_p1.b, prt2_p1.c, 75
+                           Filter: ((prt2_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt1_p2.a = prt2_p2.b)
+               Filter: (((50) = prt1_p2.b) OR ((75) = prt2_p2.b))
+               ->  Seq Scan on public.prt1_p2
+                     Output: prt1_p2.a, prt1_p2.c, prt1_p2.b, 50
+                     Filter: ((prt1_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p2.b, prt2_p2.c, (75)
+                     ->  Seq Scan on public.prt2_p2
+                           Output: prt2_p2.b, prt2_p2.c, 75
+                           Filter: ((prt2_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c
+               Hash Cond: (prt1_p3.a = prt2_p3.b)
+               Filter: (((50) = prt1_p3.b) OR ((75) = prt2_p3.b))
+               ->  Seq Scan on public.prt1_p3
+                     Output: prt1_p3.a, prt1_p3.c, prt1_p3.b, 50
+                     Filter: ((prt1_p3.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p3.b, prt2_p3.c, (75)
+                     ->  Seq Scan on public.prt2_p3
+                           Output: prt2_p3.b, prt2_p3.c, 75
+                           Filter: ((prt2_p3.b % 25) = 0)
+(40 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+ a  |  c   | b  |  c   
+----+------+----+------
+ 50 | 0050 |    | 
+    |      | 75 | 0075
+(2 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+ a  |  c   | b  |  c   
+----+------+----+------
+ 50 | 0050 |    | 
+    |      | 75 | 0075
+(2 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, (25), prt2_p1.b, prt2_p1.c, (50)
+   Sort Key: prt1_p1.a, prt2_p1.b
+   ->  Result
+         Output: prt1_p1.a, prt1_p1.c, (25), prt2_p1.b, prt2_p1.c, (50)
+         ->  Append
+               ->  Hash Full Join
+                     Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, (25), (50)
+                     Hash Cond: (prt1_p1.a = prt2_p1.b)
+                     ->  Seq Scan on public.prt1_p1
+                           Output: prt1_p1.a, prt1_p1.c, 25
+                           Filter: ((prt1_p1.a % 25) = 0)
+                     ->  Hash
+                           Output: prt2_p1.b, prt2_p1.c, (50)
+                           ->  Seq Scan on public.prt2_p1
+                                 Output: prt2_p1.b, prt2_p1.c, 50
+                                 Filter: ((prt2_p1.b % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c, (25), (50)
+                     Hash Cond: (prt1_p2.a = prt2_p2.b)
+                     ->  Seq Scan on public.prt1_p2
+                           Output: prt1_p2.a, prt1_p2.c, 25
+                           Filter: ((prt1_p2.a % 25) = 0)
+                     ->  Hash
+                           Output: prt2_p2.b, prt2_p2.c, (50)
+                           ->  Seq Scan on public.prt2_p2
+                                 Output: prt2_p2.b, prt2_p2.c, 50
+                                 Filter: ((prt2_p2.b % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c, (25), (50)
+                     Hash Cond: (prt1_p3.a = prt2_p3.b)
+                     ->  Seq Scan on public.prt1_p3
+                           Output: prt1_p3.a, prt1_p3.c, 25
+                           Filter: ((prt1_p3.a % 25) = 0)
+                     ->  Hash
+                           Output: prt2_p3.b, prt2_p3.c, (50)
+                           ->  Seq Scan on public.prt2_p3
+                                 Output: prt2_p3.b, prt2_p3.c, 50
+                                 Filter: ((prt2_p3.b % 25) = 0)
+(39 rows)
+
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 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)
+
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM uprt2 WHERE uprt2.b % 25 = 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)
+
+-- Join with pruned partitions from joining relations
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on public.prt2_p2 t2
+                     Output: t2.b, t2.c
+                     Filter: (t2.b > 250)
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.prt1_p2 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a < 450) AND ((t1.a % 25) = 0))
+(15 rows)
+
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+(1 row)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a < 450 AND t2.b > 250 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+(1 row)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                    QUERY PLAN                                    
+----------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, b, c
+   Sort Key: prt1_p1.a, b
+   ->  Append
+         ->  Nested Loop Left Join
+               Output: prt1_p1.a, prt1_p1.c, b, c
+               Join Filter: (prt1_p1.a = b)
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c
+                     Filter: ((prt1_p1.a < 450) AND ((prt1_p1.a % 25) = 0))
+               ->  Result
+                     Output: b, c
+                     One-Time Filter: false
+         ->  Hash Right Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt2_p2.b = prt1_p2.a)
+               ->  Seq Scan on public.prt2_p2
+                     Output: prt2_p2.b, prt2_p2.c
+                     Filter: (prt2_p2.b > 250)
+               ->  Hash
+                     Output: prt1_p2.a, prt1_p2.c
+                     ->  Seq Scan on public.prt1_p2
+                           Output: prt1_p2.a, prt1_p2.c
+                           Filter: ((prt1_p2.a < 450) AND ((prt1_p2.a % 25) = 0))
+(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.a % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.a % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                       QUERY PLAN                                       
+----------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+   Sort Key: prt1_p2.a, prt2_p2.b
+   ->  Result
+         Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: prt2_p2.b, prt2_p2.c, prt1_p2.a, prt1_p2.c
+                     Hash Cond: (prt1_p2.a = prt2_p2.b)
+                     ->  Seq Scan on public.prt1_p2
+                           Output: prt1_p2.a, prt1_p2.c
+                           Filter: (prt1_p2.a < 450)
+                     ->  Hash
+                           Output: prt2_p2.b, prt2_p2.c
+                           ->  Seq Scan on public.prt2_p2
+                                 Output: prt2_p2.b, prt2_p2.c
+                                 Filter: ((prt2_p2.b > 250) AND ((prt2_p2.a % 25) = 0))
+               ->  Nested Loop Left Join
+                     Output: prt2_p3.b, prt2_p3.c, a, c
+                     Join Filter: (a = prt2_p3.b)
+                     ->  Seq Scan on public.prt2_p3
+                           Output: prt2_p3.b, prt2_p3.c
+                           Filter: ((prt2_p3.b > 250) AND ((prt2_p3.a % 25) = 0))
+                     ->  Result
+                           Output: a, c
+                           One-Time Filter: false
+(26 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+     |      | 375 | 0375
+     |      | 450 | 0450
+     |      | 525 | 0525
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+     |      | 375 | 0375
+     |      | 450 | 0450
+     |      | 525 | 0525
+(4 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+                                    QUERY PLAN                                    
+----------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, b, c
+   Sort Key: prt1_p1.a, b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_p1.a, prt1_p1.c, b, c
+               Hash Cond: (prt1_p1.a = b)
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c
+                     Filter: ((prt1_p1.a < 450) AND ((prt1_p1.a % 25) = 0))
+               ->  Hash
+                     Output: b, c
+                     ->  Result
+                           Output: b, c
+                           One-Time Filter: false
+         ->  Hash Full Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt1_p2.a = prt2_p2.b)
+               ->  Seq Scan on public.prt1_p2
+                     Output: prt1_p2.a, prt1_p2.c
+                     Filter: ((prt1_p2.a < 450) AND ((prt1_p2.a % 25) = 0))
+               ->  Hash
+                     Output: prt2_p2.b, prt2_p2.c
+                     ->  Seq Scan on public.prt2_p2
+                           Output: prt2_p2.b, prt2_p2.c
+                           Filter: ((prt2_p2.b > 250) AND ((prt2_p2.b % 25) = 0))
+         ->  Hash Full Join
+               Output: a, c, prt2_p3.b, prt2_p3.c
+               Hash Cond: (prt2_p3.b = a)
+               ->  Seq Scan on public.prt2_p3
+                     Output: prt2_p3.b, prt2_p3.c
+                     Filter: ((prt2_p3.b > 250) AND ((prt2_p3.b % 25) = 0))
+               ->  Hash
+                     Output: a, c
+                     ->  Result
+                           Output: a, c
+                           One-Time Filter: false
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b 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)
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.a = t1_3.b)
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_3.b
+                     ->  Seq Scan on public.prt2_p1 t1_3
+                           Output: t1_3.b
+                           Filter: ((t1_3.b % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.a = t1_4.b)
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_4.b
+                     ->  Seq Scan on public.prt2_p2 t1_4
+                           Output: t1_4.b
+                           Filter: ((t1_4.b % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.a = t1_5.b)
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_5.b
+                     ->  Seq Scan on public.prt2_p3 t1_5
+                           Output: t1_5.b
+                           Filter: ((t1_5.b % 25) = 0)
+(37 rows)
+
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+-- lateral reference
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+                                             QUERY PLAN                                              
+-----------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   Sort Key: t1.a
+   ->  Result
+         Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+         ->  Append
+               ->  Nested Loop Left Join
+                     Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a))
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.b, t1.c
+                           Filter: ((t1.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+                           Hash Cond: (t3.b = t2.a)
+                           ->  Seq Scan on public.prt2_p1 t3
+                                 Output: t3.a, t3.b
+                           ->  Hash
+                                 Output: t2.a
+                                 ->  Seq Scan on public.prt1_p1 t2
+                                       Output: t2.a
+                                       Filter: (t1.a = t2.a)
+               ->  Nested Loop Left Join
+                     Output: t1_2.a, t1_2.b, t1_2.c, t2_2.a, t3_1.a, (LEAST(t1_2.a, t2_2.a, t3_1.a))
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.b, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2_2.a, t3_1.a, LEAST(t1_2.a, t2_2.a, t3_1.a)
+                           Hash Cond: (t3_1.b = t2_2.a)
+                           ->  Seq Scan on public.prt2_p2 t3_1
+                                 Output: t3_1.a, t3_1.b
+                           ->  Hash
+                                 Output: t2_2.a
+                                 ->  Seq Scan on public.prt1_p2 t2_2
+                                       Output: t2_2.a
+                                       Filter: (t1_2.a = t2_2.a)
+               ->  Nested Loop Left Join
+                     Output: t1_1.a, t1_1.b, t1_1.c, t2_1.a, t3_2.a, (LEAST(t1_1.a, t2_1.a, t3_2.a))
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.b, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2_1.a, t3_2.a, LEAST(t1_1.a, t2_1.a, t3_2.a)
+                           Hash Cond: (t3_2.b = t2_1.a)
+                           ->  Seq Scan on public.prt2_p3 t3_2
+                                 Output: t3_2.a, t3_2.b
+                           ->  Hash
+                                 Output: t2_1.a
+                                 ->  Seq Scan on public.prt1_p3 t2_1
+                                       Output: t2_1.a
+                                       Filter: (t1_1.a = t2_1.a)
+(51 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   Sort Key: t1.a
+   ->  Nested Loop Left Join
+         Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+         ->  Append
+               ->  Seq Scan on public.prt1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Seq Scan on public.prt1_p1 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Seq Scan on public.prt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Seq Scan on public.prt1_p2 t1_3
+                     Output: t1_3.a, t1_3.b, t1_3.c
+                     Filter: ((t1_3.a % 25) = 0)
+         ->  Append
+               ->  Hash Join
+                     Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+                     Hash Cond: (t3.b = t2.a)
+                     ->  Seq Scan on public.prt2_p1 t3
+                           Output: t3.a, t3.b
+                     ->  Hash
+                           Output: t2.a
+                           ->  Seq Scan on public.prt1_p1 t2
+                                 Output: t2.a
+                                 Filter: (t1.b = t2.a)
+               ->  Hash Join
+                     Output: t2_2.a, t3_1.a, LEAST(t1.a, t2_2.a, t3_1.a)
+                     Hash Cond: (t3_1.b = t2_2.a)
+                     ->  Seq Scan on public.prt2_p2 t3_1
+                           Output: t3_1.a, t3_1.b
+                     ->  Hash
+                           Output: t2_2.a
+                           ->  Seq Scan on public.prt1_p2 t2_2
+                                 Output: t2_2.a
+                                 Filter: (t1.b = t2_2.a)
+               ->  Hash Join
+                     Output: t2_1.a, t3_2.a, LEAST(t1.a, t2_1.a, t3_2.a)
+                     Hash Cond: (t3_2.b = t2_1.a)
+                     ->  Seq Scan on public.prt2_p3 t3_2
+                           Output: t3_2.a, t3_2.b
+                     ->  Hash
+                           Output: t2_1.a
+                           ->  Seq Scan on public.prt1_p3 t2_1
+                                 Output: t2_1.a
+                                 Filter: (t1.b = t2_1.a)
+(49 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+--
+-- partitioned by expression
+--
+CREATE TABLE prt1_e (a int, b int, c varchar) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES START (0) END (250);
+CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES START (250) END (500);
+CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES START (500) END (600);
+INSERT INTO prt1_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_e AS SELECT * FROM prt1_e;
+CREATE TABLE prt2_e (a int, b int, c varchar) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES START (0) END (250);
+CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES START (250) END (500);
+CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES START (500) END (600);
+INSERT INTO prt2_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_e;
+ANALYZE prt2_e_p1;
+ANALYZE prt2_e_p2;
+ANALYZE prt2_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_e AS SELECT * FROM prt2_e;
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                  QUERY PLAN                                  
+------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_e_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_e_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
+               ->  Seq Scan on public.prt2_e_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_e_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+(34 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.a % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1, uprt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.a % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                  QUERY PLAN                                  
+------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_e_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_e_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
+               ->  Seq Scan on public.prt2_e_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_e_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1 LEFT JOIN uprt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on public.prt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = t1.a)
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_2.a)
+                     ->  Seq Scan on public.prt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                           ->  Hash Join
+                                 Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = t1_2.a)
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_1.a)
+                     ->  Seq Scan on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                           ->  Hash Join
+                                 Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = t1_1.a)
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+(57 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM uprt1 t1, uprt2 t2, uprt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on public.prt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = t1.a)
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_2.a)
+                     ->  Seq Scan on public.prt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = t1_2.a)
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_1.a)
+                     ->  Seq Scan on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = t1_1.a)
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+(57 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, 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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: (((t3.a + t3.b) / 2) = t2.b)
+                     ->  Seq Scan on public.prt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = t1.a)
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: (((t3_1.a + t3_1.b) / 2) = t2_1.b)
+                     ->  Seq Scan on public.prt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = t1_2.a)
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: (((t3_2.a + t3_2.b) / 2) = t2_2.b)
+                     ->  Seq Scan on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = t1_1.a)
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+(57 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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |          | 
+ 100 | 0100 |     |      |          | 
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |          | 
+ 250 | 0250 |     |      |          | 
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |          | 
+ 400 | 0400 |     |      |          | 
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |          | 
+ 550 | 0550 |     |      |          | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |          | 
+ 100 | 0100 |     |      |          | 
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |          | 
+ 250 | 0250 |     |      |          | 
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |          | 
+ 400 | 0400 |     |      |          | 
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |          | 
+ 550 | 0550 |     |      |          | 
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Hash Cond: (t2.b = t1.a)
+                     ->  Seq Scan on public.prt2_p1 t2
+                           Output: t2.b, t2.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Hash Cond: (t1.a = ((t3.a + t3.b) / 2))
+                                 ->  Seq Scan on public.prt1_p1 t1
+                                       Output: t1.a, t1.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                     Hash Cond: (t2_1.b = t1_2.a)
+                     ->  Seq Scan on public.prt2_p2 t2_1
+                           Output: t2_1.b, t2_1.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                                 Hash Cond: (t1_2.a = ((t3_1.a + t3_1.b) / 2))
+                                 ->  Seq Scan on public.prt1_p2 t1_2
+                                       Output: t1_2.a, t1_2.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                     Hash Cond: (t2_2.b = t1_1.a)
+                     ->  Seq Scan on public.prt2_p3 t2_2
+                           Output: t2_2.b, t2_2.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                                 Hash Cond: (t1_1.a = ((t3_2.a + t3_2.b) / 2))
+                                 ->  Seq Scan on public.prt1_p3 t1_1
+                                       Output: t1_1.a, t1_1.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: (t1.a = t2.b)
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = ((t3.a + t3.b) / 2))
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Hash Cond: (t1_2.a = t2_1.b)
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = ((t3_1.a + t3_1.b) / 2))
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Hash Cond: (t1_1.a = t2_2.b)
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = ((t3_2.a + t3_2.b) / 2))
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                                  QUERY PLAN                                                   
+---------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, ((prt1_e_p1.a + prt1_e_p1.b)), prt1_e_p1.c
+   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   ->  Result
+         Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, (prt1_e_p1.a + prt1_e_p1.b), prt1_e_p1.c
+         ->  Append
+               ->  Hash Full Join
+                     Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, prt1_e_p1.a, prt1_e_p1.b, prt1_e_p1.c
+                     Hash Cond: (prt1_p1.a = ((prt1_e_p1.a + prt1_e_p1.b) / 2))
+                     ->  Hash Full Join
+                           Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on public.prt1_p1
+                                 Output: prt1_p1.a, prt1_p1.c
+                                 Filter: ((prt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p1.b, prt2_p1.c
+                                 ->  Seq Scan on public.prt2_p1
+                                       Output: prt2_p1.b, prt2_p1.c
+                                       Filter: ((prt2_p1.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p1.a, prt1_e_p1.b, prt1_e_p1.c
+                           ->  Seq Scan on public.prt1_e_p1
+                                 Output: prt1_e_p1.a, prt1_e_p1.b, prt1_e_p1.c
+                                 Filter: ((prt1_e_p1.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c, prt1_e_p2.a, prt1_e_p2.b, prt1_e_p2.c
+                     Hash Cond: (prt1_p2.a = ((prt1_e_p2.a + prt1_e_p2.b) / 2))
+                     ->  Hash Full Join
+                           Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+                           Hash Cond: (prt1_p2.a = prt2_p2.b)
+                           ->  Seq Scan on public.prt1_p2
+                                 Output: prt1_p2.a, prt1_p2.c
+                                 Filter: ((prt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p2.b, prt2_p2.c
+                                 ->  Seq Scan on public.prt2_p2
+                                       Output: prt2_p2.b, prt2_p2.c
+                                       Filter: ((prt2_p2.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p2.a, prt1_e_p2.b, prt1_e_p2.c
+                           ->  Seq Scan on public.prt1_e_p2
+                                 Output: prt1_e_p2.a, prt1_e_p2.b, prt1_e_p2.c
+                                 Filter: ((prt1_e_p2.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c, prt1_e_p3.a, prt1_e_p3.b, prt1_e_p3.c
+                     Hash Cond: (prt1_p3.a = ((prt1_e_p3.a + prt1_e_p3.b) / 2))
+                     ->  Hash Full Join
+                           Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c
+                           Hash Cond: (prt1_p3.a = prt2_p3.b)
+                           ->  Seq Scan on public.prt1_p3
+                                 Output: prt1_p3.a, prt1_p3.c
+                                 Filter: ((prt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p3.b, prt2_p3.c
+                                 ->  Seq Scan on public.prt2_p3
+                                       Output: prt2_p3.b, prt2_p3.c
+                                       Filter: ((prt2_p3.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p3.a, prt1_e_p3.b, prt1_e_p3.c
+                           ->  Seq Scan on public.prt1_e_p3
+                                 Output: prt1_e_p3.a, prt1_e_p3.b, prt1_e_p3.c
+                                 Filter: ((prt1_e_p3.a % 25) = 0)
+(63 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+     |      |  75 | 0075 |          | 
+     |      | 225 | 0225 |          | 
+     |      | 375 | 0375 |          | 
+     |      | 525 | 0525 |          | 
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM uprt1_e WHERE uprt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+     |      |  75 | 0075 |          | 
+     |      | 225 | 0225 |          | 
+     |      | 375 | 0375 |          | 
+     |      | 525 | 0525 |          | 
+(16 rows)
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, 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.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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                                                      
+----------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, (50), prt2_p1.b, (75), ((prt1_e_p1.a + prt1_e_p1.b)), (50)
+   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   ->  Result
+         Output: prt1_p1.a, (50), prt2_p1.b, (75), (prt1_e_p1.a + prt1_e_p1.b), (50)
+         ->  Append
+               ->  Hash Full Join
+                     Output: prt1_p1.a, prt2_p1.b, prt1_e_p1.a, prt1_e_p1.b, (50), (75), (50)
+                     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
+                           Output: prt1_p1.a, prt2_p1.b, (50), (75)
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on public.prt1_p1
+                                 Output: prt1_p1.a, 50
+                                 Filter: ((prt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p1.b, (75)
+                                 ->  Seq Scan on public.prt2_p1
+                                       Output: prt2_p1.b, 75
+                                       Filter: ((prt2_p1.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p1.a, prt1_e_p1.b, (50)
+                           ->  Seq Scan on public.prt1_e_p1
+                                 Output: prt1_e_p1.a, prt1_e_p1.b, 50
+                                 Filter: ((prt1_e_p1.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p2.a, prt2_p2.b, prt1_e_p2.a, prt1_e_p2.b, (50), (75), (50)
+                     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
+                           Output: prt1_p2.a, prt2_p2.b, (50), (75)
+                           Hash Cond: (prt1_p2.a = prt2_p2.b)
+                           ->  Seq Scan on public.prt1_p2
+                                 Output: prt1_p2.a, 50
+                                 Filter: ((prt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p2.b, (75)
+                                 ->  Seq Scan on public.prt2_p2
+                                       Output: prt2_p2.b, 75
+                                       Filter: ((prt2_p2.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p2.a, prt1_e_p2.b, (50)
+                           ->  Seq Scan on public.prt1_e_p2
+                                 Output: prt1_e_p2.a, prt1_e_p2.b, 50
+                                 Filter: ((prt1_e_p2.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p3.a, prt2_p3.b, prt1_e_p3.a, prt1_e_p3.b, (50), (75), (50)
+                     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
+                           Output: prt1_p3.a, prt2_p3.b, (50), (75)
+                           Hash Cond: (prt1_p3.a = prt2_p3.b)
+                           ->  Seq Scan on public.prt1_p3
+                                 Output: prt1_p3.a, 50
+                                 Filter: ((prt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p3.b, (75)
+                                 ->  Seq Scan on public.prt2_p3
+                                       Output: prt2_p3.b, 75
+                                       Filter: ((prt2_p3.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p3.a, prt1_e_p3.b, (50)
+                           ->  Seq Scan on public.prt1_e_p3
+                                 Output: prt1_e_p3.a, prt1_e_p3.b, 50
+                                 Filter: ((prt1_e_p3.a % 25) = 0)
+(66 rows)
+
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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)
+
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM uprt1_e WHERE uprt1_e.a % 25 = 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)
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.a = t1_3.b)
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_3.b, t2.a, t2.b
+                     ->  Hash Join
+                           Output: t1_3.b, t2.a, t2.b
+                           Hash Cond: (((t2.a + t2.b) / 2) = t1_3.b)
+                           ->  Seq Scan on public.prt1_e_p1 t2
+                                 Output: t2.a, t2.b
+                           ->  Hash
+                                 Output: t1_3.b
+                                 ->  Seq Scan on public.prt2_p1 t1_3
+                                       Output: t1_3.b
+                                       Filter: ((t1_3.b % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.a = t1_4.b)
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_4.b, t2_1.a, t2_1.b
+                     ->  Hash Join
+                           Output: t1_4.b, t2_1.a, t2_1.b
+                           Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_4.b)
+                           ->  Seq Scan on public.prt1_e_p2 t2_1
+                                 Output: t2_1.a, t2_1.b
+                           ->  Hash
+                                 Output: t1_4.b
+                                 ->  Seq Scan on public.prt2_p2 t1_4
+                                       Output: t1_4.b
+                                       Filter: ((t1_4.b % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.a = t1_5.b)
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_5.b, t2_2.a, t2_2.b
+                     ->  Hash Join
+                           Output: t1_5.b, t2_2.a, t2_2.b
+                           Hash Cond: (((t2_2.a + t2_2.b) / 2) = t1_5.b)
+                           ->  Seq Scan on public.prt1_e_p3 t2_2
+                                 Output: t2_2.a, t2_2.b
+                           ->  Hash
+                                 Output: t1_5.b
+                                 ->  Seq Scan on public.prt2_p3 t1_5
+                                       Output: t1_5.b
+                                       Filter: ((t1_5.b % 25) = 0)
+(58 rows)
+
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1, uprt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.a = t1_3.b)
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_3.b, t1_6.a, t1_6.b
+                     ->  Hash Semi Join
+                           Output: t1_3.b, t1_6.a, t1_6.b
+                           Hash Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
+                           ->  Seq Scan on public.prt2_p1 t1_3
+                                 Output: t1_3.b
+                           ->  Hash
+                                 Output: t1_6.a, t1_6.b
+                                 ->  Seq Scan on public.prt1_e_p1 t1_6
+                                       Output: t1_6.a, t1_6.b
+                                       Filter: ((t1_6.a % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.a = t1_4.b)
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_4.b, t1_7.a, t1_7.b
+                     ->  Hash Semi Join
+                           Output: t1_4.b, t1_7.a, t1_7.b
+                           Hash Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
+                           ->  Seq Scan on public.prt2_p2 t1_4
+                                 Output: t1_4.b
+                           ->  Hash
+                                 Output: t1_7.a, t1_7.b
+                                 ->  Seq Scan on public.prt1_e_p2 t1_7
+                                       Output: t1_7.a, t1_7.b
+                                       Filter: ((t1_7.a % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.a = t1_5.b)
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_5.b, t1_8.a, t1_8.b
+                     ->  Hash Semi Join
+                           Output: t1_5.b, t1_8.a, t1_8.b
+                           Hash Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
+                           ->  Seq Scan on public.prt2_p3 t1_5
+                                 Output: t1_5.b
+                           ->  Hash
+                                 Output: t1_8.a, t1_8.b
+                                 ->  Seq Scan on public.prt1_e_p3 t1_8
+                                       Output: t1_8.a, t1_8.b
+                                       Filter: ((t1_8.a % 25) = 0)
+(58 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+-- test merge joins with and without using indexes
+SET enable_hashjoin TO off;
+SET enable_nestloop TO off;
+CREATE INDEX iprt1_a on prt1(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 iprt2_b on prt2(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 iprt1_e_ab2 on prt1_e(((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));
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                             QUERY PLAN                                              
+-----------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Merge Cond: (t2.b = t1.a)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           Sort Key: t2.b
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Merge Cond: ((((t3.a + t3.b) / 2)) = t2.b)
+                                 ->  Sort
+                                       Output: t3.a, t3.b, t3.c, (((t3.a + t3.b) / 2))
+                                       Sort Key: (((t3.a + t3.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c, ((t3.a + t3.b) / 2)
+                                             Filter: ((t3.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2.b, t2.c
+                                       Sort Key: t2.b
+                                       ->  Seq Scan on public.prt2_p1 t2
+                                             Output: t2.b, t2.c
+                     ->  Sort
+                           Output: t1.a, t1.c
+                           Sort Key: t1.a
+                           ->  Seq Scan on public.prt1_p1 t1
+                                 Output: t1.a, t1.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Merge Cond: (t2_1.b = t1_2.a)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           Sort Key: t2_1.b
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t2_1.b)
+                                 ->  Sort
+                                       Output: t3_1.a, t3_1.b, t3_1.c, (((t3_1.a + t3_1.b) / 2))
+                                       Sort Key: (((t3_1.a + t3_1.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c, ((t3_1.a + t3_1.b) / 2)
+                                             Filter: ((t3_1.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2_1.b, t2_1.c
+                                       Sort Key: t2_1.b
+                                       ->  Seq Scan on public.prt2_p2 t2_1
+                                             Output: t2_1.b, t2_1.c
+                     ->  Sort
+                           Output: t1_2.a, t1_2.c
+                           Sort Key: t1_2.a
+                           ->  Seq Scan on public.prt1_p2 t1_2
+                                 Output: t1_2.a, t1_2.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Merge Cond: (t2_2.b = t1_1.a)
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           Sort Key: t2_2.b
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t2_2.b)
+                                 ->  Sort
+                                       Output: t3_2.a, t3_2.b, t3_2.c, (((t3_2.a + t3_2.b) / 2))
+                                       Sort Key: (((t3_2.a + t3_2.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c, ((t3_2.a + t3_2.b) / 2)
+                                             Filter: ((t3_2.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2_2.b, t2_2.c
+                                       Sort Key: t2_2.b
+                                       ->  Seq Scan on public.prt2_p3 t2_2
+                                             Output: t2_2.b, t2_2.c
+                     ->  Sort
+                           Output: t1_1.a, t1_1.c
+                           Sort Key: t1_1.a
+                           ->  Seq Scan on public.prt1_p3 t1_1
+                                 Output: t1_1.a, t1_1.c
+(81 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+                                   QUERY PLAN                                    
+---------------------------------------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Output: t1.a, t1.b, t1.c
+         Merge Cond: (t1.a = t1_3.b)
+         ->  Sort
+               Output: t1.a, t1.b, t1.c
+               Sort Key: t1.a
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_3.b, t1_6.a, t1_6.b
+               ->  Merge Semi Join
+                     Output: t1_3.b, t1_6.a, t1_6.b
+                     Merge Cond: (t1_3.b = (((t1_6.a + t1_6.b) / 2)))
+                     ->  Sort
+                           Output: t1_3.b
+                           Sort Key: t1_3.b
+                           ->  Seq Scan on public.prt2_p1 t1_3
+                                 Output: t1_3.b
+                     ->  Sort
+                           Output: t1_6.a, t1_6.b, (((t1_6.a + t1_6.b) / 2))
+                           Sort Key: (((t1_6.a + t1_6.b) / 2))
+                           ->  Seq Scan on public.prt1_e_p1 t1_6
+                                 Output: t1_6.a, t1_6.b, ((t1_6.a + t1_6.b) / 2)
+                                 Filter: ((t1_6.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_2.a, t1_2.b, t1_2.c
+         Merge Cond: (t1_2.a = t1_4.b)
+         ->  Sort
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Sort Key: t1_2.a
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+         ->  Materialize
+               Output: t1_4.b, t1_7.a, t1_7.b
+               ->  Merge Semi Join
+                     Output: t1_4.b, t1_7.a, t1_7.b
+                     Merge Cond: (t1_4.b = (((t1_7.a + t1_7.b) / 2)))
+                     ->  Sort
+                           Output: t1_4.b
+                           Sort Key: t1_4.b
+                           ->  Seq Scan on public.prt2_p2 t1_4
+                                 Output: t1_4.b
+                     ->  Sort
+                           Output: t1_7.a, t1_7.b, (((t1_7.a + t1_7.b) / 2))
+                           Sort Key: (((t1_7.a + t1_7.b) / 2))
+                           ->  Seq Scan on public.prt1_e_p2 t1_7
+                                 Output: t1_7.a, t1_7.b, ((t1_7.a + t1_7.b) / 2)
+                                 Filter: ((t1_7.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_1.a, t1_1.b, t1_1.c
+         Merge Cond: (t1_1.a = t1_5.b)
+         ->  Sort
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Sort Key: t1_1.a
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_5.b, t1_8.a, t1_8.b
+               ->  Merge Semi Join
+                     Output: t1_5.b, t1_8.a, t1_8.b
+                     Merge Cond: (t1_5.b = (((t1_8.a + t1_8.b) / 2)))
+                     ->  Sort
+                           Output: t1_5.b
+                           Sort Key: t1_5.b
+                           ->  Seq Scan on public.prt2_p3 t1_5
+                                 Output: t1_5.b
+                     ->  Sort
+                           Output: t1_8.a, t1_8.b, (((t1_8.a + t1_8.b) / 2))
+                           Sort Key: (((t1_8.a + t1_8.b) / 2))
+                           ->  Seq Scan on public.prt1_e_p3 t1_8
+                                 Output: t1_8.a, t1_8.b, ((t1_8.a + t1_8.b) / 2)
+                                 Filter: ((t1_8.a % 25) = 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                             QUERY PLAN                                              
+-----------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Merge Cond: (t1.a = t2.b)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           Sort Key: t1.a
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Merge Cond: ((((t3.a + t3.b) / 2)) = t1.a)
+                                 ->  Sort
+                                       Output: t3.a, t3.b, t3.c, (((t3.a + t3.b) / 2))
+                                       Sort Key: (((t3.a + t3.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c, ((t3.a + t3.b) / 2)
+                                             Filter: ((t3.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t1.a, t1.c
+                                       Sort Key: t1.a
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                     ->  Sort
+                           Output: t2.b, t2.c
+                           Sort Key: t2.b
+                           ->  Seq Scan on public.prt2_p1 t2
+                                 Output: t2.b, t2.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                     Merge Cond: (t1_2.a = t2_1.b)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                           Sort Key: t1_2.a
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                                 Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t1_2.a)
+                                 ->  Sort
+                                       Output: t3_1.a, t3_1.b, t3_1.c, (((t3_1.a + t3_1.b) / 2))
+                                       Sort Key: (((t3_1.a + t3_1.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c, ((t3_1.a + t3_1.b) / 2)
+                                             Filter: ((t3_1.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t1_2.a, t1_2.c
+                                       Sort Key: t1_2.a
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                     ->  Sort
+                           Output: t2_1.b, t2_1.c
+                           Sort Key: t2_1.b
+                           ->  Seq Scan on public.prt2_p2 t2_1
+                                 Output: t2_1.b, t2_1.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                     Merge Cond: (t1_1.a = t2_2.b)
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                           Sort Key: t1_1.a
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                                 Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t1_1.a)
+                                 ->  Sort
+                                       Output: t3_2.a, t3_2.b, t3_2.c, (((t3_2.a + t3_2.b) / 2))
+                                       Sort Key: (((t3_2.a + t3_2.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c, ((t3_2.a + t3_2.b) / 2)
+                                             Filter: ((t3_2.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t1_1.a, t1_1.c
+                                       Sort Key: t1_1.a
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                     ->  Sort
+                           Output: t2_2.b, t2_2.c
+                           Sort Key: t2_2.b
+                           ->  Seq Scan on public.prt2_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+(81 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SET enable_seqscan TO off;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                          QUERY PLAN                                          
+----------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Merge Cond: (t2.b = t1.a)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           Sort Key: t2.b
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Merge Cond: (((t3.a + t3.b) / 2) = t2.b)
+                                 ->  Index Scan using iprt1_e_p1_ab2 on public.prt1_e_p1 t3
+                                       Output: t3.a, t3.b, t3.c
+                                       Filter: ((t3.a % 25) = 0)
+                                 ->  Index Scan using iprt2_p1_b on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                     ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Merge Cond: (t2_1.b = t1_2.a)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           Sort Key: t2_1.b
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Merge Cond: (((t3_1.a + t3_1.b) / 2) = t2_1.b)
+                                 ->  Index Scan using iprt1_e_p2_ab2 on public.prt1_e_p2 t3_1
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       Filter: ((t3_1.a % 25) = 0)
+                                 ->  Index Scan using iprt2_p2_b on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                     ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+               ->  Merge Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Merge Cond: (t2_2.b = ((t3_2.a + t3_2.b) / 2))
+                     ->  Merge Left Join
+                           Output: t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                           Merge Cond: (t2_2.b = t1_1.a)
+                           ->  Index Scan using iprt2_p3_b on public.prt2_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+                           ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_1
+                                 Output: t1_1.a, t1_1.c
+                     ->  Index Scan using iprt1_e_p3_ab2 on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                           Filter: ((t3_2.a % 25) = 0)
+(51 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+                                    QUERY PLAN                                    
+----------------------------------------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Output: t1.a, t1.b, t1.c
+         Merge Cond: (t1.a = t1_3.b)
+         ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1
+               Output: t1.a, t1.b, t1.c
+               Filter: ((t1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_3.b, t1_6.a, t1_6.b
+               ->  Merge Semi Join
+                     Output: t1_3.b, t1_6.a, t1_6.b
+                     Merge Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
+                     ->  Index Only Scan using iprt2_p1_b on public.prt2_p1 t1_3
+                           Output: t1_3.b
+                     ->  Index Scan using iprt1_e_p1_ab2 on public.prt1_e_p1 t1_6
+                           Output: t1_6.a, t1_6.b
+                           Filter: ((t1_6.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_2.a, t1_2.b, t1_2.c
+         Merge Cond: (t1_2.a = t1_4.b)
+         ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_2
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Filter: ((t1_2.a % 25) = 0)
+         ->  Materialize
+               Output: t1_4.b, t1_7.a, t1_7.b
+               ->  Merge Semi Join
+                     Output: t1_4.b, t1_7.a, t1_7.b
+                     Merge Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
+                     ->  Index Only Scan using iprt2_p2_b on public.prt2_p2 t1_4
+                           Output: t1_4.b
+                     ->  Index Scan using iprt1_e_p2_ab2 on public.prt1_e_p2 t1_7
+                           Output: t1_7.a, t1_7.b
+                           Filter: ((t1_7.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_1.a, t1_1.b, t1_1.c
+         Merge Cond: (t1_1.a = t1_5.b)
+         ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_1
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Filter: ((t1_1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_5.b, t1_8.a, t1_8.b
+               ->  Merge Semi Join
+                     Output: t1_5.b, t1_8.a, t1_8.b
+                     Merge Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
+                     ->  Index Only Scan using iprt2_p3_b on public.prt2_p3 t1_5
+                           Output: t1_5.b
+                     ->  Index Scan using iprt1_e_p3_ab2 on public.prt1_e_p3 t1_8
+                           Output: t1_8.a, t1_8.b
+                           Filter: ((t1_8.a % 25) = 0)
+(50 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                          QUERY PLAN                                          
+----------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Merge Cond: (t1.a = t2.b)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           Sort Key: t1.a
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Merge Cond: (((t3.a + t3.b) / 2) = t1.a)
+                                 ->  Index Scan using iprt1_e_p1_ab2 on public.prt1_e_p1 t3
+                                       Output: t3.a, t3.b, t3.c
+                                       Filter: ((t3.a % 25) = 0)
+                                 ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1
+                                       Output: t1.a, t1.c
+                     ->  Index Scan using iprt2_p1_b on public.prt2_p1 t2
+                           Output: t2.b, t2.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                     Merge Cond: (t1_2.a = t2_1.b)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                           Sort Key: t1_2.a
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                                 Merge Cond: (((t3_1.a + t3_1.b) / 2) = t1_2.a)
+                                 ->  Index Scan using iprt1_e_p2_ab2 on public.prt1_e_p2 t3_1
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       Filter: ((t3_1.a % 25) = 0)
+                                 ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_2
+                                       Output: t1_2.a, t1_2.c
+                     ->  Index Scan using iprt2_p2_b on public.prt2_p2 t2_1
+                           Output: t2_1.b, t2_1.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                     Merge Cond: (t1_1.a = t2_2.b)
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                           Sort Key: t1_1.a
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                                 Merge Cond: (((t3_2.a + t3_2.b) / 2) = t1_1.a)
+                                 ->  Index Scan using iprt1_e_p3_ab2 on public.prt1_e_p3 t3_2
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       Filter: ((t3_2.a % 25) = 0)
+                                 ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_1
+                                       Output: t1_1.a, t1_1.c
+                     ->  Index Scan using iprt2_p3_b on public.prt2_p3 t2_2
+                           Output: t2_2.b, t2_2.c
+(54 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+-- lateral references and parameterized paths
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
+ Nested Loop Left Join
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   ->  Merge Append
+         Sort Key: t1.a
+         ->  Index Scan using iprt1_a on public.prt1 t1
+               Output: t1.a, t1.b, t1.c
+               Filter: ((t1.a % 25) = 0)
+         ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1_1
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Filter: ((t1_1.a % 25) = 0)
+         ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_2
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Filter: ((t1_2.a % 25) = 0)
+         ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_3
+               Output: t1_3.a, t1_3.b, t1_3.c
+               Filter: ((t1_3.a % 25) = 0)
+   ->  Append
+         ->  Merge Join
+               Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+               Merge Cond: (t2.a = t3.b)
+               ->  Index Only Scan using iprt1_p1_a on public.prt1_p1 t2
+                     Output: t2.a
+                     Index Cond: (t2.a = t1.a)
+               ->  Index Scan using iprt2_p1_b on public.prt2_p1 t3
+                     Output: t3.a, t3.b
+         ->  Merge Join
+               Output: t2_2.a, t3_1.a, LEAST(t1.a, t2_2.a, t3_1.a)
+               Merge Cond: (t2_2.a = t3_1.b)
+               ->  Index Only Scan using iprt1_p2_a on public.prt1_p2 t2_2
+                     Output: t2_2.a
+                     Index Cond: (t2_2.a = t1.a)
+               ->  Index Scan using iprt2_p2_b on public.prt2_p2 t3_1
+                     Output: t3_1.a, t3_1.b
+         ->  Merge Join
+               Output: t2_1.a, t3_2.a, LEAST(t1.a, t2_1.a, t3_2.a)
+               Merge Cond: (t2_1.a = t3_2.b)
+               ->  Index Only Scan using iprt1_p3_a on public.prt1_p3 t2_1
+                     Output: t2_1.a
+                     Index Cond: (t2_1.a = t1.a)
+               ->  Index Scan using iprt2_p3_b on public.prt2_p3 t3_2
+                     Output: t3_2.a, t3_2.b
+(41 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
+ Nested Loop Left Join
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   ->  Merge Append
+         Sort Key: t1.a
+         ->  Index Scan using iprt1_a on public.prt1 t1
+               Output: t1.a, t1.b, t1.c
+               Filter: ((t1.a % 25) = 0)
+         ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1_1
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Filter: ((t1_1.a % 25) = 0)
+         ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_2
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Filter: ((t1_2.a % 25) = 0)
+         ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_3
+               Output: t1_3.a, t1_3.b, t1_3.c
+               Filter: ((t1_3.a % 25) = 0)
+   ->  Append
+         ->  Merge Join
+               Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+               Merge Cond: (t2.a = t3.b)
+               ->  Index Only Scan using iprt1_p1_a on public.prt1_p1 t2
+                     Output: t2.a
+                     Index Cond: (t2.a = t1.b)
+               ->  Index Scan using iprt2_p1_b on public.prt2_p1 t3
+                     Output: t3.a, t3.b
+         ->  Merge Join
+               Output: t2_2.a, t3_1.a, LEAST(t1.a, t2_2.a, t3_1.a)
+               Merge Cond: (t2_2.a = t3_1.b)
+               ->  Index Only Scan using iprt1_p2_a on public.prt1_p2 t2_2
+                     Output: t2_2.a
+                     Index Cond: (t2_2.a = t1.b)
+               ->  Index Scan using iprt2_p2_b on public.prt2_p2 t3_1
+                     Output: t3_1.a, t3_1.b
+         ->  Merge Join
+               Output: t2_1.a, t3_2.a, LEAST(t1.a, t2_1.a, t3_2.a)
+               Merge Cond: (t2_1.a = t3_2.b)
+               ->  Index Only Scan using iprt1_p3_a on public.prt1_p3 t2_1
+                     Output: t2_1.a
+                     Index Cond: (t2_1.a = t1.b)
+               ->  Index Scan using iprt2_p3_b on public.prt2_p3 t3_2
+                     Output: t3_2.a, t3_2.b
+(41 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+RESET enable_hashjoin;
+RESET enable_nestloop;
+RESET enable_seqscan;
+--
+-- partitioned by multiple columns
+--
+CREATE TABLE prt1_m (a int, b int, c varchar) PARTITION BY RANGE(a, ((a + b)/2));
+CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES START (0, 0) END (250, 250);
+CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES START (250, 250) END (500, 500);
+CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES START (500, 500) END (600, 600);
+INSERT INTO prt1_m SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_m;
+ANALYZE prt1_m_p1;
+ANALYZE prt1_m_p2;
+ANALYZE prt1_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_m AS SELECT * FROM prt1_m;
+CREATE TABLE prt2_m (a int, b int, c varchar) PARTITION BY RANGE(((b + a)/2), b);
+CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES START (0, 0) END (250, 250);
+CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES START (250, 250) END (500, 500);
+CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES START (500, 500) END (600, 600);
+INSERT INTO prt2_m SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_m;
+ANALYZE prt2_m_p1;
+ANALYZE prt2_m_p2;
+ANALYZE prt2_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_m AS SELECT * FROM prt2_m;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+                                                 QUERY PLAN                                                 
+------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: ((((t1.a + t1.b) / 2) = t2.b) AND (t1.a = ((t2.b + t2.a) / 2)))
+                     ->  Seq Scan on public.prt1_m_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                     ->  Hash
+                           Output: t2.b, t2.c, t2.a
+                           ->  Seq Scan on public.prt2_m_p1 t2
+                                 Output: t2.b, t2.c, t2.a
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: ((((t1_1.a + t1_1.b) / 2) = t2_1.b) AND (t1_1.a = ((t2_1.b + t2_1.a) / 2)))
+                     ->  Seq Scan on public.prt1_m_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c, t2_1.a
+                           ->  Seq Scan on public.prt2_m_p2 t2_1
+                                 Output: t2_1.b, t2_1.c, t2_1.a
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: ((((t1_2.a + t1_2.b) / 2) = t2_2.b) AND (t1_2.a = ((t2_2.b + t2_2.a) / 2)))
+                     ->  Seq Scan on public.prt1_m_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c, t2_2.a
+                           ->  Seq Scan on public.prt2_m_p3 t2_2
+                                 Output: t2_2.b, t2_2.c, t2_2.a
+                                 Filter: ((t2_2.b % 25) = 0)
+(36 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_m t1 RIGHT JOIN uprt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 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
+   Output: prt1_m_p1.a, prt1_m_p1.c, prt2_m_p1.b, prt2_m_p1.c
+   Sort Key: prt1_m_p1.a, prt2_m_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_m_p1.a, prt1_m_p1.c, prt2_m_p1.b, prt2_m_p1.c
+               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 public.prt1_m_p1
+                     Output: prt1_m_p1.a, prt1_m_p1.c, prt1_m_p1.b
+                     Filter: ((prt1_m_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_m_p1.b, prt2_m_p1.c, prt2_m_p1.a
+                     ->  Seq Scan on public.prt2_m_p1
+                           Output: prt2_m_p1.b, prt2_m_p1.c, prt2_m_p1.a
+                           Filter: ((prt2_m_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_m_p2.a, prt1_m_p2.c, prt2_m_p2.b, prt2_m_p2.c
+               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 public.prt1_m_p2
+                     Output: prt1_m_p2.a, prt1_m_p2.c, prt1_m_p2.b
+                     Filter: ((prt1_m_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_m_p2.b, prt2_m_p2.c, prt2_m_p2.a
+                     ->  Seq Scan on public.prt2_m_p2
+                           Output: prt2_m_p2.b, prt2_m_p2.c, prt2_m_p2.a
+                           Filter: ((prt2_m_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_m_p3.a, prt1_m_p3.c, prt2_m_p3.b, prt2_m_p3.c
+               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 public.prt1_m_p3
+                     Output: prt1_m_p3.a, prt1_m_p3.c, prt1_m_p3.b
+                     Filter: ((prt1_m_p3.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_m_p3.b, prt2_m_p3.c, prt2_m_p3.a
+                     ->  Seq Scan on public.prt2_m_p3
+                           Output: prt2_m_p3.b, prt2_m_p3.c, prt2_m_p3.a
+                           Filter: ((prt2_m_p3.b % 25) = 0)
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 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 | 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_m t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_m t2 WHERE t2.b % 25 = 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 | 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)
+
+--
+-- 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;
+ANALYZE plt1_p1;
+ANALYZE plt1_p2;
+ANALYZE plt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1 AS SELECT * FROM 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;
+ANALYZE plt2_p1;
+ANALYZE plt2_p2;
+ANALYZE plt2_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt2 AS SELECT * FROM plt2;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.a = t2.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((t2.c = t1.c) AND (t2.a = t1.a))
+               ->  Seq Scan on public.plt2_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.plt1_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((t2_1.c = t1_1.c) AND (t2_1.a = t1_1.a))
+               ->  Seq Scan on public.plt2_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.plt1_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((t2_2.c = t1_2.c) AND (t2_2.a = t1_2.a))
+               ->  Seq Scan on public.plt2_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.plt1_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.a = t2.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0003 | 150 | 0003
+ 300 | 0006 | 300 | 0006
+ 450 | 0009 | 450 | 0009
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1 t1, uplt2 t2 WHERE t1.c = t2.c AND t1.a = t2.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0003 | 150 | 0003
+ 300 | 0006 | 300 | 0006
+ 450 | 0009 | 450 | 0009
+(4 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.a AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c))
+               ->  Seq Scan on public.plt2_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.plt1_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c))
+               ->  Seq Scan on public.plt2_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.plt1_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c))
+               ->  Seq Scan on public.plt2_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.plt1_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.a AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  50 | 0001 |     | 
+ 100 | 0002 |     | 
+ 150 | 0003 | 150 | 0003
+ 200 | 0004 |     | 
+ 250 | 0005 |     | 
+ 300 | 0006 | 300 | 0006
+ 350 | 0007 |     | 
+ 400 | 0008 |     | 
+ 450 | 0009 | 450 | 0009
+ 500 | 0010 |     | 
+ 550 | 0011 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.a AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  50 | 0001 |     | 
+ 100 | 0002 |     | 
+ 150 | 0003 | 150 | 0003
+ 200 | 0004 |     | 
+ 250 | 0005 |     | 
+ 300 | 0006 | 300 | 0006
+ 350 | 0007 |     | 
+ 400 | 0008 |     | 
+ 450 | 0009 | 450 | 0009
+ 500 | 0010 |     | 
+ 550 | 0011 |     | 
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+                                QUERY PLAN                                
+--------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: ((t1.a = t2.b) AND (t1.c = t2.c))
+                     ->  Seq Scan on public.plt1_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t2.b, t2.c
+                           ->  Seq Scan on public.plt2_p1 t2
+                                 Output: t2.b, t2.c
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: ((t1_1.a = t2_1.b) AND (t1_1.c = t2_1.c))
+                     ->  Seq Scan on public.plt1_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c
+                           ->  Seq Scan on public.plt2_p2 t2_1
+                                 Output: t2_1.b, t2_1.c
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: ((t1_2.a = t2_2.b) AND (t1_2.c = t2_2.c))
+                     ->  Seq Scan on public.plt1_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c
+                           ->  Seq Scan on public.plt2_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+                                 Filter: ((t2_2.b % 25) = 0)
+(36 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0003 | 150 | 0003
+ 300 | 0006 | 300 | 0006
+ 450 | 0009 | 450 | 0009
+     |      |  75 | 0001
+     |      | 225 | 0004
+     |      | 375 | 0007
+     |      | 525 | 0010
+(8 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+ 150 | 0003 | 150 | 0003
+ 300 | 0006 | 300 | 0006
+ 450 | 0009 | 450 | 0009
+     |      |  75 | 0001
+     |      | 225 | 0004
+     |      | 375 | 0007
+     |      | 525 | 0010
+(8 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) ORDER BY t1.a, t2.b;
+                                   QUERY PLAN                                   
+--------------------------------------------------------------------------------
+ Sort
+   Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+   Sort Key: plt1_p1.a, plt2_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+               Hash Cond: ((plt1_p1.a = plt2_p1.b) AND (plt1_p1.c = plt2_p1.c))
+               ->  Seq Scan on public.plt1_p1
+                     Output: plt1_p1.a, plt1_p1.c
+                     Filter: ((plt1_p1.a % 25) = 0)
+               ->  Hash
+                     Output: plt2_p1.b, plt2_p1.c
+                     ->  Seq Scan on public.plt2_p1
+                           Output: plt2_p1.b, plt2_p1.c
+                           Filter: ((plt2_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+               Hash Cond: ((plt1_p2.a = plt2_p2.b) AND (plt1_p2.c = plt2_p2.c))
+               ->  Seq Scan on public.plt1_p2
+                     Output: plt1_p2.a, plt1_p2.c
+                     Filter: ((plt1_p2.a % 25) = 0)
+               ->  Hash
+                     Output: plt2_p2.b, plt2_p2.c
+                     ->  Seq Scan on public.plt2_p2
+                           Output: plt2_p2.b, plt2_p2.c
+                           Filter: ((plt2_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+               Hash Cond: ((plt1_p3.a = plt2_p3.b) AND (plt1_p3.c = plt2_p3.c))
+               ->  Seq Scan on public.plt1_p3
+                     Output: plt1_p3.a, plt1_p3.c
+                     Filter: ((plt1_p3.a % 25) = 0)
+               ->  Hash
+                     Output: plt2_p3.b, plt2_p3.c
+                     ->  Seq Scan on public.plt2_p3
+                           Output: plt2_p3.b, plt2_p3.c
+                           Filter: ((plt2_p3.b % 25) = 0)
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  50 | 0001 |     | 
+ 100 | 0002 |     | 
+ 150 | 0003 | 150 | 0003
+ 200 | 0004 |     | 
+ 250 | 0005 |     | 
+ 300 | 0006 | 300 | 0006
+ 350 | 0007 |     | 
+ 400 | 0008 |     | 
+ 450 | 0009 | 450 | 0009
+ 500 | 0010 |     | 
+ 550 | 0011 |     | 
+     |      |  75 | 0001
+     |      | 225 | 0004
+     |      | 375 | 0007
+     |      | 525 | 0010
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uplt1 t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  50 | 0001 |     | 
+ 100 | 0002 |     | 
+ 150 | 0003 | 150 | 0003
+ 200 | 0004 |     | 
+ 250 | 0005 |     | 
+ 300 | 0006 | 300 | 0006
+ 350 | 0007 |     | 
+ 400 | 0008 |     | 
+ 450 | 0009 | 450 | 0009
+ 500 | 0010 |     | 
+ 550 | 0011 |     | 
+     |      |  75 | 0001
+     |      | 225 | 0004
+     |      | 375 | 0007
+     |      | 525 | 0010
+(16 rows)
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM (SELECT 50 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ GroupAggregate
+   Output: sum(plt1_p1.a), plt1_p1.c, avg(plt2_p1.b), plt2_p1.c
+   Group Key: plt1_p1.c, plt2_p1.c
+   ->  Sort
+         Output: plt1_p1.c, plt2_p1.c, plt1_p1.a, plt2_p1.b
+         Sort Key: plt1_p1.c, plt2_p1.c
+         ->  Result
+               Output: plt1_p1.c, plt2_p1.c, plt1_p1.a, plt2_p1.b
+               ->  Append
+                     ->  Hash Full Join
+                           Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+                           Hash Cond: ((plt1_p1.c = plt2_p1.c) AND (plt1_p1.a = plt2_p1.b))
+                           ->  Seq Scan on public.plt1_p1
+                                 Output: plt1_p1.a, plt1_p1.c
+                                 Filter: ((plt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p1.b, plt2_p1.c
+                                 ->  Seq Scan on public.plt2_p1
+                                       Output: plt2_p1.b, plt2_p1.c
+                                       Filter: ((plt2_p1.b % 25) = 0)
+                     ->  Hash Full Join
+                           Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+                           Hash Cond: ((plt1_p2.c = plt2_p2.c) AND (plt1_p2.a = plt2_p2.b))
+                           ->  Seq Scan on public.plt1_p2
+                                 Output: plt1_p2.a, plt1_p2.c
+                                 Filter: ((plt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p2.b, plt2_p2.c
+                                 ->  Seq Scan on public.plt2_p2
+                                       Output: plt2_p2.b, plt2_p2.c
+                                       Filter: ((plt2_p2.b % 25) = 0)
+                     ->  Hash Full Join
+                           Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+                           Hash Cond: ((plt1_p3.c = plt2_p3.c) AND (plt1_p3.a = plt2_p3.b))
+                           ->  Seq Scan on public.plt1_p3
+                                 Output: plt1_p3.a, plt1_p3.c
+                                 Filter: ((plt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p3.b, plt2_p3.c
+                                 ->  Seq Scan on public.plt2_p3
+                                       Output: plt2_p3.b, plt2_p3.c
+                                       Filter: ((plt2_p3.b % 25) = 0)
+(42 rows)
+
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM (SELECT 50 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+ sum |  c   |          avg           |  c   
+-----+------+------------------------+------
+   0 | 0000 | 0.00000000000000000000 | 0000
+  50 | 0001 |                        | 
+ 100 | 0002 |                        | 
+ 150 | 0003 |   150.0000000000000000 | 0003
+ 200 | 0004 |                        | 
+ 250 | 0005 |                        | 
+ 300 | 0006 |   300.0000000000000000 | 0006
+ 350 | 0007 |                        | 
+ 400 | 0008 |                        | 
+ 450 | 0009 |   450.0000000000000000 | 0009
+ 500 | 0010 |                        | 
+ 550 | 0011 |                        | 
+     |      |    75.0000000000000000 | 0001
+     |      |   225.0000000000000000 | 0004
+     |      |   375.0000000000000000 | 0007
+     |      |   525.0000000000000000 | 0010
+(16 rows)
+
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM (SELECT 50 phv, * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+ sum |  c   |          avg           |  c   
+-----+------+------------------------+------
+   0 | 0000 | 0.00000000000000000000 | 0000
+  50 | 0001 |                        | 
+ 100 | 0002 |                        | 
+ 150 | 0003 |   150.0000000000000000 | 0003
+ 200 | 0004 |                        | 
+ 250 | 0005 |                        | 
+ 300 | 0006 |   300.0000000000000000 | 0006
+ 350 | 0007 |                        | 
+ 400 | 0008 |                        | 
+ 450 | 0009 |   450.0000000000000000 | 0009
+ 500 | 0010 |                        | 
+ 550 | 0011 |                        | 
+     |      |    75.0000000000000000 | 0001
+     |      |   225.0000000000000000 | 0004
+     |      |   375.0000000000000000 | 0007
+     |      |   525.0000000000000000 | 0010
+(16 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t1.phv), avg(t2.b), t2.c, avg(t2.phv) FROM (SELECT 25 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ GroupAggregate
+   Output: sum(plt1_p1.a), plt1_p1.c, sum((25)), avg(plt2_p1.b), plt2_p1.c, avg((50))
+   Group Key: plt1_p1.c, plt2_p1.c
+   ->  Sort
+         Output: plt1_p1.c, plt2_p1.c, plt1_p1.a, (25), plt2_p1.b, (50)
+         Sort Key: plt1_p1.c, plt2_p1.c
+         ->  Result
+               Output: plt1_p1.c, plt2_p1.c, plt1_p1.a, (25), plt2_p1.b, (50)
+               ->  Append
+                     ->  Hash Full Join
+                           Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, (25), (50)
+                           Hash Cond: ((plt1_p1.c = plt2_p1.c) AND (plt1_p1.a = plt2_p1.b))
+                           ->  Seq Scan on public.plt1_p1
+                                 Output: plt1_p1.a, plt1_p1.c, 25
+                                 Filter: ((plt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p1.b, plt2_p1.c, (50)
+                                 ->  Seq Scan on public.plt2_p1
+                                       Output: plt2_p1.b, plt2_p1.c, 50
+                                       Filter: ((plt2_p1.b % 25) = 0)
+                     ->  Hash Full Join
+                           Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c, (25), (50)
+                           Hash Cond: ((plt1_p2.c = plt2_p2.c) AND (plt1_p2.a = plt2_p2.b))
+                           ->  Seq Scan on public.plt1_p2
+                                 Output: plt1_p2.a, plt1_p2.c, 25
+                                 Filter: ((plt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p2.b, plt2_p2.c, (50)
+                                 ->  Seq Scan on public.plt2_p2
+                                       Output: plt2_p2.b, plt2_p2.c, 50
+                                       Filter: ((plt2_p2.b % 25) = 0)
+                     ->  Hash Full Join
+                           Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c, (25), (50)
+                           Hash Cond: ((plt1_p3.c = plt2_p3.c) AND (plt1_p3.a = plt2_p3.b))
+                           ->  Seq Scan on public.plt1_p3
+                                 Output: plt1_p3.a, plt1_p3.c, 25
+                                 Filter: ((plt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p3.b, plt2_p3.c, (50)
+                                 ->  Seq Scan on public.plt2_p3
+                                       Output: plt2_p3.b, plt2_p3.c, 50
+                                       Filter: ((plt2_p3.b % 25) = 0)
+(42 rows)
+
+SELECT sum(t1.a), t1.c, sum(t1.phv), avg(t2.b), t2.c, avg(t2.phv) FROM (SELECT 25 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+ sum |  c   | sum |          avg           |  c   |         avg         
+-----+------+-----+------------------------+------+---------------------
+   0 | 0000 |  25 | 0.00000000000000000000 | 0000 | 50.0000000000000000
+  50 | 0001 |  25 |                        |      |                    
+ 100 | 0002 |  25 |                        |      |                    
+ 150 | 0003 |  25 |   150.0000000000000000 | 0003 | 50.0000000000000000
+ 200 | 0004 |  25 |                        |      |                    
+ 250 | 0005 |  25 |                        |      |                    
+ 300 | 0006 |  25 |   300.0000000000000000 | 0006 | 50.0000000000000000
+ 350 | 0007 |  25 |                        |      |                    
+ 400 | 0008 |  25 |                        |      |                    
+ 450 | 0009 |  25 |   450.0000000000000000 | 0009 | 50.0000000000000000
+ 500 | 0010 |  25 |                        |      |                    
+ 550 | 0011 |  25 |                        |      |                    
+     |      |     |    75.0000000000000000 | 0001 | 50.0000000000000000
+     |      |     |   225.0000000000000000 | 0004 | 50.0000000000000000
+     |      |     |   375.0000000000000000 | 0007 | 50.0000000000000000
+     |      |     |   525.0000000000000000 | 0010 | 50.0000000000000000
+(16 rows)
+
+SELECT sum(t1.a), t1.c, sum(t1.phv), avg(t2.b), t2.c, avg(t2.phv) FROM (SELECT 25 phv, * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+ sum |  c   | sum |          avg           |  c   |         avg         
+-----+------+-----+------------------------+------+---------------------
+   0 | 0000 |  25 | 0.00000000000000000000 | 0000 | 50.0000000000000000
+  50 | 0001 |  25 |                        |      |                    
+ 100 | 0002 |  25 |                        |      |                    
+ 150 | 0003 |  25 |   150.0000000000000000 | 0003 | 50.0000000000000000
+ 200 | 0004 |  25 |                        |      |                    
+ 250 | 0005 |  25 |                        |      |                    
+ 300 | 0006 |  25 |   300.0000000000000000 | 0006 | 50.0000000000000000
+ 350 | 0007 |  25 |                        |      |                    
+ 400 | 0008 |  25 |                        |      |                    
+ 450 | 0009 |  25 |   450.0000000000000000 | 0009 | 50.0000000000000000
+ 500 | 0010 |  25 |                        |      |                    
+ 550 | 0011 |  25 |                        |      |                    
+     |      |     |    75.0000000000000000 | 0001 | 50.0000000000000000
+     |      |     |   225.0000000000000000 | 0004 | 50.0000000000000000
+     |      |     |   375.0000000000000000 | 0007 | 50.0000000000000000
+     |      |     |   525.0000000000000000 | 0010 | 50.0000000000000000
+(16 rows)
+
+-- Join with pruned partitions from joining relations
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.c NOT IN ('0001', '0005', '0002', '0009') AND t2.c NOT IN ('0000', '0003', '0004', '0010') GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Output: (sum(t1.a)), t1.c, (avg(t2.b)), t2.c
+   Sort Key: t1.c
+   ->  HashAggregate
+         Output: sum(t1.a), t1.c, avg(t2.b), t2.c
+         Group Key: t1.c, t2.c
+         ->  Result
+               Output: t1.c, t2.c, t1.a, t2.b
+               ->  Append
+                     ->  Hash Join
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           Hash Cond: (t1.c = t2.c)
+                           ->  Seq Scan on public.plt1_p3 t1
+                                 Output: t1.a, t1.c
+                                 Filter: (t1.c <> ALL ('{0001,0005,0002,0009}'::text[]))
+                           ->  Hash
+                                 Output: t2.b, t2.c
+                                 ->  Seq Scan on public.plt2_p3 t2
+                                       Output: t2.b, t2.c
+                                       Filter: (t2.c <> ALL ('{0000,0003,0004,0010}'::text[]))
+(20 rows)
+
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.c NOT IN ('0001', '0005', '0002', '0009') AND t2.c NOT IN ('0000', '0003', '0004', '0010') GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+  sum   |  c   |         avg          |  c   
+--------+------+----------------------+------
+ 137700 | 0006 | 324.0000000000000000 | 0006
+ 158950 | 0007 | 375.0000000000000000 | 0007
+ 169600 | 0008 | 424.5000000000000000 | 0008
+ 229600 | 0011 | 574.5000000000000000 | 0011
+(4 rows)
+
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM uplt1 t1, uplt2 t2 WHERE t1.c = t2.c AND t1.c NOT IN ('0001', '0005', '0002', '0009') AND t2.c NOT IN ('0000', '0003', '0004', '0010') GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+  sum   |  c   |         avg          |  c   
+--------+------+----------------------+------
+ 137700 | 0006 | 324.0000000000000000 | 0006
+ 158950 | 0007 | 375.0000000000000000 | 0007
+ 169600 | 0008 | 424.5000000000000000 | 0008
+ 229600 | 0011 | 574.5000000000000000 | 0011
+(4 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 LEFT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Output: (sum(t1.a)), t1.c, (sum(b)), c
+   Sort Key: t1.c, c
+   ->  HashAggregate
+         Output: sum(t1.a), t1.c, sum(b), c
+         Group Key: t1.c, c
+         ->  Result
+               Output: t1.c, c, t1.a, b
+               ->  Append
+                     ->  Hash Left Join
+                           Output: t1.a, t1.c, b, c
+                           Hash Cond: (t1.c = c)
+                           ->  Seq Scan on public.plt1_p1 t1
+                                 Output: t1.a, t1.c
+                                 Filter: (t1.c <> ALL ('{0001,0005,0002,0009}'::text[]))
+                           ->  Hash
+                                 Output: b, c
+                                 ->  Result
+                                       Output: b, c
+                                       One-Time Filter: false
+                     ->  Hash Left Join
+                           Output: t1_1.a, t1_1.c, t2.b, t2.c
+                           Hash Cond: (t1_1.c = t2.c)
+                           ->  Seq Scan on public.plt1_p3 t1_1
+                                 Output: t1_1.a, t1_1.c
+                                 Filter: (t1_1.c <> ALL ('{0001,0005,0002,0009}'::text[]))
+                           ->  Hash
+                                 Output: t2.b, t2.c
+                                 ->  Seq Scan on public.plt2_p3 t2
+                                       Output: t2.b, t2.c
+                                       Filter: (t2.c <> ALL ('{0000,0003,0004,0010}'::text[]))
+(31 rows)
+
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 LEFT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+  sum   |  c   |  sum   |  c   
+--------+------+--------+------
+    600 | 0000 |        | 
+   4350 | 0003 |        | 
+   5600 | 0004 |        | 
+ 137700 | 0006 | 137700 | 0006
+ 158950 | 0007 | 159375 | 0007
+ 169600 | 0008 | 169800 | 0008
+  13100 | 0010 |        | 
+ 229600 | 0011 | 229800 | 0011
+(8 rows)
+
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM uplt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 LEFT JOIN (SELECT * FROM uplt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+  sum   |  c   |  sum   |  c   
+--------+------+--------+------
+    600 | 0000 |        | 
+   4350 | 0003 |        | 
+   5600 | 0004 |        | 
+ 137700 | 0006 | 137700 | 0006
+ 158950 | 0007 | 159375 | 0007
+ 169600 | 0008 | 169800 | 0008
+  13100 | 0010 |        | 
+ 229600 | 0011 | 229800 | 0011
+(8 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 RIGHT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+                                           QUERY PLAN                                            
+-------------------------------------------------------------------------------------------------
+ Sort
+   Output: (sum(a)), c, (sum(t2.b)), t2.c
+   Sort Key: c, t2.c
+   ->  HashAggregate
+         Output: sum(a), c, sum(t2.b), t2.c
+         Group Key: c, t2.c
+         ->  Result
+               Output: c, t2.c, a, t2.b
+               ->  Append
+                     ->  Hash Left Join
+                           Output: t2.b, t2.c, a, c
+                           Hash Cond: (t2.c = c)
+                           ->  Seq Scan on public.plt2_p2 t2
+                                 Output: t2.b, t2.c
+                                 Filter: (t2.c <> ALL ('{0000,0003,0004,0010}'::text[]))
+                           ->  Hash
+                                 Output: a, c
+                                 ->  Result
+                                       Output: a, c
+                                       One-Time Filter: false
+                     ->  Hash Right Join
+                           Output: t2_1.b, t2_1.c, t1.a, t1.c
+                           Hash Cond: (t1.c = t2_1.c)
+                           ->  Seq Scan on public.plt1_p3 t1
+                                 Output: t1.a, t1.c
+                                 Filter: (t1.c <> ALL ('{0001,0005,0002,0009}'::text[]))
+                           ->  Hash
+                                 Output: t2_1.b, t2_1.c
+                                 ->  Seq Scan on public.plt2_p3 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                       Filter: (t2_1.c <> ALL ('{0000,0003,0004,0010}'::text[]))
+(31 rows)
+
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 RIGHT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+  sum   |  c   |  sum   |  c   
+--------+------+--------+------
+ 137700 | 0006 | 137700 | 0006
+ 158950 | 0007 | 159375 | 0007
+ 169600 | 0008 | 169800 | 0008
+ 229600 | 0011 | 229800 | 0011
+        |      |   1275 | 0001
+        |      |   1992 | 0002
+        |      |   4392 | 0005
+        |      |   8058 | 0009
+(8 rows)
+
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM uplt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 RIGHT JOIN (SELECT * FROM uplt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+  sum   |  c   |  sum   |  c   
+--------+------+--------+------
+ 137700 | 0006 | 137700 | 0006
+ 158950 | 0007 | 159375 | 0007
+ 169600 | 0008 | 169800 | 0008
+ 229600 | 0011 | 229800 | 0011
+        |      |   1275 | 0001
+        |      |   1992 | 0002
+        |      |   4392 | 0005
+        |      |   8058 | 0009
+(8 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 FULL JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+                                           QUERY PLAN                                            
+-------------------------------------------------------------------------------------------------
+ Sort
+   Output: (sum(t1.a)), t1.c, (sum(b)), c
+   Sort Key: t1.c, c
+   ->  HashAggregate
+         Output: sum(t1.a), t1.c, sum(b), c
+         Group Key: t1.c, c
+         ->  Result
+               Output: t1.c, c, t1.a, b
+               ->  Append
+                     ->  Hash Full Join
+                           Output: t1.a, t1.c, b, c
+                           Hash Cond: (t1.c = c)
+                           ->  Seq Scan on public.plt1_p1 t1
+                                 Output: t1.a, t1.c
+                                 Filter: (t1.c <> ALL ('{0001,0005,0002,0009}'::text[]))
+                           ->  Hash
+                                 Output: b, c
+                                 ->  Result
+                                       Output: b, c
+                                       One-Time Filter: false
+                     ->  Hash Full Join
+                           Output: a, c, t2.b, t2.c
+                           Hash Cond: (t2.c = c)
+                           ->  Seq Scan on public.plt2_p2 t2
+                                 Output: t2.b, t2.c
+                                 Filter: (t2.c <> ALL ('{0000,0003,0004,0010}'::text[]))
+                           ->  Hash
+                                 Output: a, c
+                                 ->  Result
+                                       Output: a, c
+                                       One-Time Filter: false
+                     ->  Hash Full Join
+                           Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                           Hash Cond: (t1_1.c = t2_1.c)
+                           ->  Seq Scan on public.plt1_p3 t1_1
+                                 Output: t1_1.a, t1_1.c
+                                 Filter: (t1_1.c <> ALL ('{0001,0005,0002,0009}'::text[]))
+                           ->  Hash
+                                 Output: t2_1.b, t2_1.c
+                                 ->  Seq Scan on public.plt2_p3 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                       Filter: (t2_1.c <> ALL ('{0000,0003,0004,0010}'::text[]))
+(42 rows)
+
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 FULL JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+  sum   |  c   |  sum   |  c   
+--------+------+--------+------
+    600 | 0000 |        | 
+   4350 | 0003 |        | 
+   5600 | 0004 |        | 
+ 137700 | 0006 | 137700 | 0006
+ 158950 | 0007 | 159375 | 0007
+ 169600 | 0008 | 169800 | 0008
+  13100 | 0010 |        | 
+ 229600 | 0011 | 229800 | 0011
+        |      |   1275 | 0001
+        |      |   1992 | 0002
+        |      |   4392 | 0005
+        |      |   8058 | 0009
+(12 rows)
+
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM uplt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 FULL JOIN (SELECT * FROM uplt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+  sum   |  c   |  sum   |  c   
+--------+------+--------+------
+    600 | 0000 |        | 
+   4350 | 0003 |        | 
+   5600 | 0004 |        | 
+ 137700 | 0006 | 137700 | 0006
+ 158950 | 0007 | 159375 | 0007
+ 169600 | 0008 | 169800 | 0008
+  13100 | 0010 |        | 
+ 229600 | 0011 | 229800 | 0011
+        |      |   1275 | 0001
+        |      |   1992 | 0002
+        |      |   4392 | 0005
+        |      |   8058 | 0009
+(12 rows)
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.c = t1_3.c)
+               ->  Seq Scan on public.plt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_3.c
+                     ->  Seq Scan on public.plt2_p1 t1_3
+                           Output: t1_3.c
+                           Filter: ((t1_3.b % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.c = t1_4.c)
+               ->  Seq Scan on public.plt1_p2 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_4.c
+                     ->  Seq Scan on public.plt2_p2 t1_4
+                           Output: t1_4.c
+                           Filter: ((t1_4.b % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.c = t1_5.c)
+               ->  Seq Scan on public.plt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Materialize
+                     Output: t1_5.c
+                     ->  Seq Scan on public.plt2_p3 t1_5
+                           Output: t1_5.c
+                           Filter: ((t1_5.b % 25) = 0)
+(37 rows)
+
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+(8 rows)
+
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+(8 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;
+ANALYZE plt1_e;
+ANALYZE plt1_e_p1;
+ANALYZE plt1_e_p2;
+ANALYZE plt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1_e AS SELECT * FROM plt1_e;
+CREATE TABLE plt2_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
+CREATE TABLE plt2_e_p1 PARTITION OF plt2_e FOR VALUES IN ('0000', '0003', '0004', '0010');
+CREATE TABLE plt2_e_p2 PARTITION OF plt2_e FOR VALUES IN ('0001', '0005', '0002', '0009');
+CREATE TABLE plt2_e_p3 PARTITION OF plt2_e FOR VALUES IN ('0006', '0007', '0008', '0011');
+INSERT INTO plt2_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE plt2_e;
+ANALYZE plt2_e_p1;
+ANALYZE plt2_e_p2;
+ANALYZE plt2_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt2_e AS SELECT * FROM plt2_e;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 LEFT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((t2.b = t1.a) AND (ltrim(t2.c, 'A'::text) = ltrim(t1.c, 'A'::text)))
+               ->  Seq Scan on public.plt2_e_p1 t2
+                     Output: t2.b, t2.c
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.plt1_e_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((t2_1.b = t1_1.a) AND (ltrim(t2_1.c, 'A'::text) = ltrim(t1_1.c, 'A'::text)))
+               ->  Seq Scan on public.plt2_e_p2 t2_1
+                     Output: t2_1.b, t2_1.c
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.plt1_e_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((t2_2.b = t1_2.a) AND (ltrim(t2_2.c, 'A'::text) = ltrim(t1_2.c, 'A'::text)))
+               ->  Seq Scan on public.plt2_e_p3 t2_2
+                     Output: t2_2.b, t2_2.c
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.plt1_e_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 LEFT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+  50 | A0001 |     | 
+ 100 | A0002 |     | 
+ 150 | A0003 | 150 | A0003
+ 200 | A0004 |     | 
+ 250 | A0005 |     | 
+ 300 | A0006 | 300 | A0006
+ 350 | A0007 |     | 
+ 400 | A0008 |     | 
+ 450 | A0009 | 450 | A0009
+ 500 | A0010 |     | 
+ 550 | A0011 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_e t1 LEFT JOIN uplt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+  50 | A0001 |     | 
+ 100 | A0002 |     | 
+ 150 | A0003 | 150 | A0003
+ 200 | A0004 |     | 
+ 250 | A0005 |     | 
+ 300 | A0006 | 300 | A0006
+ 350 | A0007 |     | 
+ 400 | A0008 |     | 
+ 450 | A0009 | 450 | A0009
+ 500 | A0010 |     | 
+ 550 | A0011 |     | 
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2_e WHERE plt2_e.b % 25 = 0) t2 ON (t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A')) ORDER BY t1.a, t2.b;
+                                                         QUERY PLAN                                                         
+----------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: plt1_e_p1.a, plt1_e_p1.c, plt2_e_p1.b, plt2_e_p1.c
+   Sort Key: plt1_e_p1.a, plt2_e_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: plt1_e_p1.a, plt1_e_p1.c, plt2_e_p1.b, plt2_e_p1.c
+               Hash Cond: ((plt1_e_p1.a = plt2_e_p1.b) AND (ltrim(plt1_e_p1.c, 'A'::text) = ltrim(plt2_e_p1.c, 'A'::text)))
+               ->  Seq Scan on public.plt1_e_p1
+                     Output: plt1_e_p1.a, plt1_e_p1.c
+                     Filter: ((plt1_e_p1.a % 25) = 0)
+               ->  Hash
+                     Output: plt2_e_p1.b, plt2_e_p1.c
+                     ->  Seq Scan on public.plt2_e_p1
+                           Output: plt2_e_p1.b, plt2_e_p1.c
+                           Filter: ((plt2_e_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: plt1_e_p2.a, plt1_e_p2.c, plt2_e_p2.b, plt2_e_p2.c
+               Hash Cond: ((plt1_e_p2.a = plt2_e_p2.b) AND (ltrim(plt1_e_p2.c, 'A'::text) = ltrim(plt2_e_p2.c, 'A'::text)))
+               ->  Seq Scan on public.plt1_e_p2
+                     Output: plt1_e_p2.a, plt1_e_p2.c
+                     Filter: ((plt1_e_p2.a % 25) = 0)
+               ->  Hash
+                     Output: plt2_e_p2.b, plt2_e_p2.c
+                     ->  Seq Scan on public.plt2_e_p2
+                           Output: plt2_e_p2.b, plt2_e_p2.c
+                           Filter: ((plt2_e_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: plt1_e_p3.a, plt1_e_p3.c, plt2_e_p3.b, plt2_e_p3.c
+               Hash Cond: ((plt1_e_p3.a = plt2_e_p3.b) AND (ltrim(plt1_e_p3.c, 'A'::text) = ltrim(plt2_e_p3.c, 'A'::text)))
+               ->  Seq Scan on public.plt1_e_p3
+                     Output: plt1_e_p3.a, plt1_e_p3.c
+                     Filter: ((plt1_e_p3.a % 25) = 0)
+               ->  Hash
+                     Output: plt2_e_p3.b, plt2_e_p3.c
+                     ->  Seq Scan on public.plt2_e_p3
+                           Output: plt2_e_p3.b, plt2_e_p3.c
+                           Filter: ((plt2_e_p3.b % 25) = 0)
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2_e WHERE plt2_e.b % 25 = 0) t2 ON (t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A')) ORDER BY t1.a, t2.b;
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+  50 | A0001 |     | 
+ 100 | A0002 |     | 
+ 150 | A0003 | 150 | A0003
+ 200 | A0004 |     | 
+ 250 | A0005 |     | 
+ 300 | A0006 | 300 | A0006
+ 350 | A0007 |     | 
+ 400 | A0008 |     | 
+ 450 | A0009 | 450 | A0009
+ 500 | A0010 |     | 
+ 550 | A0011 |     | 
+     |       |  75 | A0001
+     |       | 225 | A0004
+     |       | 375 | A0007
+     |       | 525 | A0010
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uplt1_e t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2_e t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A')) ORDER BY t1.a, t2.b;
+  a  |   c   |  b  |   c   
+-----+-------+-----+-------
+   0 | A0000 |   0 | A0000
+  50 | A0001 |     | 
+ 100 | A0002 |     | 
+ 150 | A0003 | 150 | A0003
+ 200 | A0004 |     | 
+ 250 | A0005 |     | 
+ 300 | A0006 | 300 | A0006
+ 350 | A0007 |     | 
+ 400 | A0008 |     | 
+ 450 | A0009 | 450 | A0009
+ 500 | A0010 |     | 
+ 550 | A0011 |     | 
+     |       |  75 | A0001
+     |       | 225 | A0004
+     |       | 375 | A0007
+     |       | 525 | A0010
+(16 rows)
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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
+   Output: (avg(t1.a)), (avg(t2.b)), (avg((t3.a + t3.b))), t1.c, t2.c, t3.c
+   Sort Key: t1.c, t3.c
+   ->  HashAggregate
+         Output: avg(t1.a), avg(t2.b), avg((t3.a + t3.b)), t1.c, t2.c, t3.c
+         Group Key: t1.c, t2.c, t3.c
+         ->  Result
+               Output: t1.c, t2.c, t3.c, t1.a, t2.b, t3.a, t3.b
+               ->  Append
+                     ->  Hash Join
+                           Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                           Hash Cond: (t1.c = t2.c)
+                           ->  Seq Scan on public.plt1_p1 t1
+                                 Output: t1.a, t1.c
+                           ->  Hash
+                                 Output: t2.b, t2.c, t3.a, t3.b, t3.c
+                                 ->  Hash Join
+                                       Output: t2.b, t2.c, t3.a, t3.b, t3.c
+                                       Hash Cond: (t2.c = ltrim(t3.c, 'A'::text))
+                                       ->  Seq Scan on public.plt2_p1 t2
+                                             Output: t2.b, t2.c
+                                       ->  Hash
+                                             Output: t3.a, t3.b, t3.c
+                                             ->  Seq Scan on public.plt1_e_p1 t3
+                                                   Output: t3.a, t3.b, t3.c
+                     ->  Hash Join
+                           Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                           Hash Cond: (t1_1.c = t2_1.c)
+                           ->  Seq Scan on public.plt1_p2 t1_1
+                                 Output: t1_1.a, t1_1.c
+                           ->  Hash
+                                 Output: t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                                 ->  Hash Join
+                                       Output: t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                                       Hash Cond: (t2_1.c = ltrim(t3_1.c, 'A'::text))
+                                       ->  Seq Scan on public.plt2_p2 t2_1
+                                             Output: t2_1.b, t2_1.c
+                                       ->  Hash
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             ->  Seq Scan on public.plt1_e_p2 t3_1
+                                                   Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash Join
+                           Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                           Hash Cond: (t1_2.c = t2_2.c)
+                           ->  Seq Scan on public.plt1_p3 t1_2
+                                 Output: t1_2.a, t1_2.c
+                           ->  Hash
+                                 Output: t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                                 ->  Hash Join
+                                       Output: t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                                       Hash Cond: (t2_2.c = ltrim(t3_2.c, 'A'::text))
+                                       ->  Seq Scan on public.plt2_p3 t2_2
+                                             Output: t2_2.b, t2_2.c
+                                       ->  Hash
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             ->  Seq Scan on public.plt1_e_p3 t3_2
+                                                   Output: t3_2.a, t3_2.b, t3_2.c
+(57 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)
+
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM uplt1 t1, uplt2 t2, uplt1_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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: ((t3.a = t1.a) AND (ltrim(t3.c, 'A'::text) = t1.c))
+                     ->  Seq Scan on public.plt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                                 ->  Seq Scan on public.plt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.plt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: ((t3_1.a = t1_1.a) AND (ltrim(t3_1.c, 'A'::text) = t1_1.c))
+                     ->  Seq Scan on public.plt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.c = t1_1.c))
+                                 ->  Seq Scan on public.plt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.plt1_p2 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: ((t3_2.a = t1_2.a) AND (ltrim(t3_2.c, 'A'::text) = t1_2.c))
+                     ->  Seq Scan on public.plt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                                 ->  Seq Scan on public.plt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.plt1_p3 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: ((t3.a = t2.b) AND (ltrim(t3.c, 'A'::text) = t2.c))
+                     ->  Seq Scan on public.plt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                                 ->  Seq Scan on public.plt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.plt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: ((t3_1.a = t2_1.b) AND (ltrim(t3_1.c, 'A'::text) = t2_1.c))
+                     ->  Seq Scan on public.plt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.c = t1_1.c))
+                                 ->  Seq Scan on public.plt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.plt1_p2 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: ((t3_2.a = t2_2.b) AND (ltrim(t3_2.c, 'A'::text) = t2_2.c))
+                     ->  Seq Scan on public.plt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                                 ->  Seq Scan on public.plt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.plt1_p3 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |          | 
+ 100 | 0002 |     |      |          | 
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |          | 
+ 250 | 0005 |     |      |          | 
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |          | 
+ 400 | 0008 |     |      |          | 
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |          | 
+ 550 | 0011 |     |      |          | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |          | 
+ 100 | 0002 |     |      |          | 
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |          | 
+ 250 | 0005 |     |      |          | 
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |          | 
+ 400 | 0008 |     |      |          | 
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |          | 
+ 550 | 0011 |     |      |          | 
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Hash Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                     ->  Seq Scan on public.plt2_p1 t2
+                           Output: t2.b, t2.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Hash Cond: ((t1.c = ltrim(t3.c, 'A'::text)) AND (t1.a = t3.a))
+                                 ->  Seq Scan on public.plt1_p1 t1
+                                       Output: t1.a, t1.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.plt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                     Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.c = t1_1.c))
+                     ->  Seq Scan on public.plt2_p2 t2_1
+                           Output: t2_1.b, t2_1.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_1.a, t1_1.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_1.a, t1_1.c
+                                 Hash Cond: ((t1_1.c = ltrim(t3_1.c, 'A'::text)) AND (t1_1.a = t3_1.a))
+                                 ->  Seq Scan on public.plt1_p2 t1_1
+                                       Output: t1_1.a, t1_1.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.plt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                     Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                     ->  Seq Scan on public.plt2_p3 t2_2
+                           Output: t2_2.b, t2_2.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_2.a, t1_2.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_2.a, t1_2.c
+                                 Hash Cond: ((t1_2.c = ltrim(t3_2.c, 'A'::text)) AND (t1_2.a = t3_2.a))
+                                 ->  Seq Scan on public.plt1_p3 t1_2
+                                       Output: t1_2.a, t1_2.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.plt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: ((t1.a = t2.b) AND (t1.c = t2.c))
+                     ->  Seq Scan on public.plt1_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Hash Cond: ((t2.b = t3.a) AND (t2.c = ltrim(t3.c, 'A'::text)))
+                                 ->  Seq Scan on public.plt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.plt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: ((t1_1.a = t2_1.b) AND (t1_1.c = t2_1.c))
+                     ->  Seq Scan on public.plt1_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: ((t2_1.b = t3_1.a) AND (t2_1.c = ltrim(t3_1.c, 'A'::text)))
+                                 ->  Seq Scan on public.plt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.plt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: ((t1_2.a = t2_2.b) AND (t1_2.c = t2_2.c))
+                     ->  Seq Scan on public.plt1_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: ((t2_2.b = t3_2.a) AND (t2_2.c = ltrim(t3_2.c, 'A'::text)))
+                                 ->  Seq Scan on public.plt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.plt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+     |      |     |      |      100 | A0001
+     |      |     |      |      200 | A0002
+     |      |     |      |      400 | A0004
+     |      |     |      |      500 | A0005
+     |      |     |      |      700 | A0007
+     |      |     |      |      800 | A0008
+     |      |     |      |     1000 | A0010
+     |      |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+     |      |     |      |      100 | A0001
+     |      |     |      |      200 | A0002
+     |      |     |      |      400 | A0004
+     |      |     |      |      500 | A0005
+     |      |     |      |      700 | A0007
+     |      |     |      |      800 | A0008
+     |      |     |      |     1000 | A0010
+     |      |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                                  QUERY PLAN                                                   
+---------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, ((plt1_e_p1.a + plt1_e_p1.b)), plt1_e_p1.c
+   Sort Key: plt1_p1.a, plt2_p1.b, ((plt1_e_p1.a + plt1_e_p1.b))
+   ->  Result
+         Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, (plt1_e_p1.a + plt1_e_p1.b), plt1_e_p1.c
+         ->  Append
+               ->  Hash Full Join
+                     Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                     Hash Cond: ((plt1_p1.a = plt1_e_p1.a) AND (plt1_p1.c = ltrim(plt1_e_p1.c, 'A'::text)))
+                     ->  Hash Full Join
+                           Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+                           Hash Cond: ((plt1_p1.a = plt2_p1.b) AND (plt1_p1.c = plt2_p1.c))
+                           ->  Seq Scan on public.plt1_p1
+                                 Output: plt1_p1.a, plt1_p1.c
+                                 Filter: ((plt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p1.b, plt2_p1.c
+                                 ->  Seq Scan on public.plt2_p1
+                                       Output: plt2_p1.b, plt2_p1.c
+                                       Filter: ((plt2_p1.b % 25) = 0)
+                     ->  Hash
+                           Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                           ->  Seq Scan on public.plt1_e_p1
+                                 Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                                 Filter: ((plt1_e_p1.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c, plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                     Hash Cond: ((plt1_p2.a = plt1_e_p2.a) AND (plt1_p2.c = ltrim(plt1_e_p2.c, 'A'::text)))
+                     ->  Hash Full Join
+                           Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+                           Hash Cond: ((plt1_p2.a = plt2_p2.b) AND (plt1_p2.c = plt2_p2.c))
+                           ->  Seq Scan on public.plt1_p2
+                                 Output: plt1_p2.a, plt1_p2.c
+                                 Filter: ((plt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p2.b, plt2_p2.c
+                                 ->  Seq Scan on public.plt2_p2
+                                       Output: plt2_p2.b, plt2_p2.c
+                                       Filter: ((plt2_p2.b % 25) = 0)
+                     ->  Hash
+                           Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                           ->  Seq Scan on public.plt1_e_p2
+                                 Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                                 Filter: ((plt1_e_p2.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c, plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                     Hash Cond: ((plt1_p3.a = plt1_e_p3.a) AND (plt1_p3.c = ltrim(plt1_e_p3.c, 'A'::text)))
+                     ->  Hash Full Join
+                           Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+                           Hash Cond: ((plt1_p3.a = plt2_p3.b) AND (plt1_p3.c = plt2_p3.c))
+                           ->  Seq Scan on public.plt1_p3
+                                 Output: plt1_p3.a, plt1_p3.c
+                                 Filter: ((plt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p3.b, plt2_p3.c
+                                 ->  Seq Scan on public.plt2_p3
+                                       Output: plt2_p3.b, plt2_p3.c
+                                       Filter: ((plt2_p3.b % 25) = 0)
+                     ->  Hash
+                           Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                           ->  Seq Scan on public.plt1_e_p3
+                                 Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                                 Filter: ((plt1_e_p3.a % 25) = 0)
+(63 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.c = t1_3.c)
+               ->  Seq Scan on public.plt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_3.c, t2.c
+                     Hash Cond: (t1_3.c = ltrim(t2.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p1 t1_3
+                           Output: t1_3.c
+                     ->  Hash
+                           Output: t2.c
+                           ->  Seq Scan on public.plt1_e_p1 t2
+                                 Output: t2.c
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.c = t1_4.c)
+               ->  Seq Scan on public.plt1_p2 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_4.c, t2_1.c
+                     Hash Cond: (t1_4.c = ltrim(t2_1.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p2 t1_4
+                           Output: t1_4.c
+                     ->  Hash
+                           Output: t2_1.c
+                           ->  Seq Scan on public.plt1_e_p2 t2_1
+                                 Output: t2_1.c
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.c = t1_5.c)
+               ->  Seq Scan on public.plt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_5.c, t2_2.c
+                     Hash Cond: (t1_5.c = ltrim(t2_2.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p3 t1_5
+                           Output: t1_5.c
+                     ->  Hash
+                           Output: t2_2.c
+                           ->  Seq Scan on public.plt1_e_p3 t2_2
+                                 Output: t2_2.c
+(49 rows)
+
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1, uplt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+                                   QUERY PLAN                                   
+--------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.c = t1_3.c)
+               ->  Seq Scan on public.plt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_3.c, t1_6.c
+                     Hash Cond: (t1_3.c = ltrim(t1_6.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p1 t1_3
+                           Output: t1_3.c
+                     ->  Hash
+                           Output: t1_6.c
+                           ->  HashAggregate
+                                 Output: t1_6.c
+                                 Group Key: ltrim(t1_6.c, 'A'::text)
+                                 ->  Seq Scan on public.plt1_e_p1 t1_6
+                                       Output: t1_6.c, ltrim(t1_6.c, 'A'::text)
+                                       Filter: ((t1_6.a % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.c = t1_4.c)
+               ->  Seq Scan on public.plt1_p2 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_4.c, t1_7.c
+                     Hash Cond: (t1_4.c = ltrim(t1_7.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p2 t1_4
+                           Output: t1_4.c
+                     ->  Hash
+                           Output: t1_7.c
+                           ->  HashAggregate
+                                 Output: t1_7.c
+                                 Group Key: ltrim(t1_7.c, 'A'::text)
+                                 ->  Seq Scan on public.plt1_e_p2 t1_7
+                                       Output: t1_7.c, ltrim(t1_7.c, 'A'::text)
+                                       Filter: ((t1_7.a % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.c = t1_5.c)
+               ->  Seq Scan on public.plt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_5.c, t1_8.c
+                     Hash Cond: (t1_5.c = ltrim(t1_8.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p3 t1_5
+                           Output: t1_5.c
+                     ->  Hash
+                           Output: t1_8.c
+                           ->  HashAggregate
+                                 Output: t1_8.c
+                                 Group Key: ltrim(t1_8.c, 'A'::text)
+                                 ->  Seq Scan on public.plt1_e_p3 t1_8
+                                       Output: t1_8.c, ltrim(t1_8.c, 'A'::text)
+                                       Filter: ((t1_8.a % 25) = 0)
+(61 rows)
+
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+--
+-- negative testcases
+--
+-- joins where one of the relations is proven empty
+EXPLAIN (VERBOSE, 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;
+            QUERY PLAN            
+----------------------------------
+ Result
+   Output: t1.a, t1.c, t2.b, t2.c
+   One-Time Filter: false
+(3 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 LEFT JOIN prt2 t2 ON t1.a = t2.b;
+              QUERY PLAN              
+--------------------------------------
+ Result
+   Output: prt1.a, prt1.c, t2.b, t2.c
+   One-Time Filter: false
+(3 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                     
+---------------------------------------------------
+ Sort
+   Output: a, c, t2.b, t2.c
+   Sort Key: a, t2.b
+   ->  Hash Left Join
+         Output: a, c, t2.b, t2.c
+         Hash Cond: (t2.b = a)
+         ->  Append
+               ->  Seq Scan on public.prt2 t2
+                     Output: t2.b, t2.c
+                     Filter: ((t2.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p1 t2_1
+                     Output: t2_1.b, t2_1.c
+                     Filter: ((t2_1.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p2 t2_2
+                     Output: t2_2.b, t2_2.c
+                     Filter: ((t2_2.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p3 t2_3
+                     Output: t2_3.b, t2_3.c
+                     Filter: ((t2_3.a % 25) = 0)
+         ->  Hash
+               Output: a, c
+               ->  Result
+                     Output: a, c
+                     One-Time Filter: false
+(24 rows)
+
+EXPLAIN (VERBOSE, 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 % 25 = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                     
+---------------------------------------------------
+ Sort
+   Output: a, c, t2.b, t2.c
+   Sort Key: a, t2.b
+   ->  Hash Left Join
+         Output: a, c, t2.b, t2.c
+         Hash Cond: (t2.b = a)
+         ->  Append
+               ->  Seq Scan on public.prt2 t2
+                     Output: t2.b, t2.c
+                     Filter: ((t2.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p1 t2_1
+                     Output: t2_1.b, t2_1.c
+                     Filter: ((t2_1.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p2 t2_2
+                     Output: t2_2.b, t2_2.c
+                     Filter: ((t2_2.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p3 t2_3
+                     Output: t2_3.b, t2_3.c
+                     Filter: ((t2_3.a % 25) = 0)
+         ->  Hash
+               Output: a, c
+               ->  Result
+                     Output: a, c
+                     One-Time Filter: false
+(24 rows)
+
+CREATE TABLE prt1_n (a int, b int, c varchar) PARTITION BY RANGE(c);
+CREATE TABLE prt1_n_p1 PARTITION OF prt1_n FOR VALUES START ('0000') END ('0250');
+CREATE TABLE prt1_n_p2 PARTITION OF prt1_n FOR VALUES START ('0250') END ('0500');
+INSERT INTO prt1_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 499, 2) i;
+ANALYZE prt1_n;
+ANALYZE prt1_n_p1;
+ANALYZE prt1_n_p2;
+CREATE TABLE prt2_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt2_n_p1 PARTITION OF prt2_n FOR VALUES IN ('0000', '0003', '0004', '0010', '0006', '0007');
+CREATE TABLE prt2_n_p2 PARTITION OF prt2_n FOR VALUES IN ('0001', '0005', '0002', '0009', '0008', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt2_n;
+ANALYZE prt2_n_p1;
+ANALYZE prt2_n_p2;
+CREATE TABLE prt3_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt3_n_p1 PARTITION OF prt3_n FOR VALUES IN ('0000', '0004', '0006', '0007');
+CREATE TABLE prt3_n_p2 PARTITION OF prt3_n FOR VALUES IN ('0001', '0002', '0008', '0010');
+CREATE TABLE prt3_n_p3 PARTITION OF prt3_n FOR VALUES IN ('0003', '0005', '0009', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt3_n;
+ANALYZE prt3_n_p1;
+ANALYZE prt3_n_p2;
+ANALYZE prt3_n_p3;
+CREATE TABLE prt4_n (a int, b int, c text) PARTITION BY RANGE(a);
+CREATE TABLE prt4_n_p1 PARTITION OF prt4_n FOR VALUES START (0) END (300);
+CREATE TABLE prt4_n_p2 PARTITION OF prt4_n FOR VALUES START (300) END (500);
+CREATE TABLE prt4_n_p3 PARTITION OF prt4_n FOR VALUES START (500) END (600);
+INSERT INTO prt4_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt4_n;
+ANALYZE prt4_n_p1;
+ANALYZE prt4_n_p2;
+ANALYZE prt4_n_p3;
+-- partition-wise join can not be applied if the partition ranges differ
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Join
+   Hash Cond: (t1.a = t2.a)
+   ->  Append
+         ->  Seq Scan on prt1 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p2 t1_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt4_n t2
+               ->  Seq Scan on prt4_n_p1 t2_1
+               ->  Seq Scan on prt4_n_p2 t2_2
+               ->  Seq Scan on prt4_n_p3 t2_3
+(13 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 FULL JOIN prt4_n t2 ON t1.a = t2.a;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Full Join
+   Hash Cond: (t1.a = t2.a)
+   ->  Append
+         ->  Seq Scan on prt1 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p2 t1_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt4_n t2
+               ->  Seq Scan on prt4_n_p1 t2_1
+               ->  Seq Scan on prt4_n_p2 t2_2
+               ->  Seq Scan on prt4_n_p3 t2_3
+(13 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                        
+---------------------------------------------------------
+ Nested Loop Left Join
+   ->  Append
+         ->  Seq Scan on prt1 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p2 t1_3
+   ->  Append
+         ->  Seq Scan on prt2 t2
+               Filter: (t1.a < b)
+         ->  Index Scan using iprt2_p1_b on prt2_p1 t2_1
+               Index Cond: (t1.a < b)
+         ->  Index Scan using iprt2_p2_b on prt2_p2 t2_2
+               Index Cond: (t1.a < b)
+         ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+               Index Cond: (t1.a < b)
+(15 rows)
+
+-- equi-join with join condition on partial keys does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1, prt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t1.a % 25 = 0;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Join
+   Hash Cond: (((t2.b + t2.a) / 2) = t1.a)
+   ->  Append
+         ->  Seq Scan on prt2_m t2
+         ->  Seq Scan on prt2_m_p1 t2_1
+         ->  Seq Scan on prt2_m_p2 t2_2
+         ->  Seq Scan on prt2_m_p3 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_m t1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p1 t1_1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p2 t1_2
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p3 t1_3
+                     Filter: ((a % 25) = 0)
+(17 rows)
+
+-- equi-join between out-of-order partition key columns does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.a = t2.b WHERE t1.a % 25 = 0;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Right Join
+   Hash Cond: (t2.b = t1.a)
+   ->  Append
+         ->  Seq Scan on prt2_m t2
+         ->  Seq Scan on prt2_m_p1 t2_1
+         ->  Seq Scan on prt2_m_p2 t2_2
+         ->  Seq Scan on prt2_m_p3 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_m t1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p1 t1_1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p2 t1_2
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p3 t1_3
+                     Filter: ((a % 25) = 0)
+(17 rows)
+
+-- equi-join between non-key columns does not qualify for partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.c = t2.c WHERE t1.a % 25 = 0;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Right Join
+   Hash Cond: ((t2.c)::text = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt2_m t2
+         ->  Seq Scan on prt2_m_p1 t2_1
+         ->  Seq Scan on prt2_m_p2 t2_2
+         ->  Seq Scan on prt2_m_p3 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_m t1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p1 t1_1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p2 t1_2
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p3 t1_3
+                     Filter: ((a % 25) = 0)
+(17 rows)
+
+-- partition-wise join can not be applied for a join between list and range
+-- partitioned table
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1, prt2_n t2 WHERE t1.c = t2.c;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Join
+   Hash Cond: (t2.c = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt2_n t2
+         ->  Seq Scan on prt2_n_p1 t2_1
+         ->  Seq Scan on prt2_n_p2 t2_2
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(11 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 LEFT JOIN prt2_n t2 ON (t1.c = t2.c);
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Right Join
+   Hash Cond: (t2.c = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt2_n t2
+         ->  Seq Scan on prt2_n_p1 t2_1
+         ->  Seq Scan on prt2_n_p2 t2_2
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(11 rows)
+
+-- partition-wise join can not be applied between tables with different
+-- partition lists
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 RIGHT JOIN prt1 t2 ON (t1.c = t2.c);
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Left Join
+   Hash Cond: ((t2.c)::text = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt1 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p2 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(12 rows)
+
+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);
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Full Join
+   Hash Cond: ((t2.c)::text = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt1 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p2 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(12 rows)
+
diff --git a/src/test/regress/expected/rangefuncs.out b/src/test/regress/expected/rangefuncs.out
index f06cfa4..16e7f56 100644
--- a/src/test/regress/expected/rangefuncs.out
+++ b/src/test/regress/expected/rangefuncs.out
@@ -1,18 +1,19 @@
 SELECT name, setting FROM pg_settings WHERE name LIKE 'enable%';
-         name         | setting 
-----------------------+---------
- enable_bitmapscan    | on
- enable_hashagg       | on
- enable_hashjoin      | on
- enable_indexonlyscan | on
- enable_indexscan     | on
- enable_material      | on
- enable_mergejoin     | on
- enable_nestloop      | on
- enable_seqscan       | on
- enable_sort          | on
- enable_tidscan       | on
-(11 rows)
+            name            | setting 
+----------------------------+---------
+ enable_bitmapscan          | on
+ enable_hashagg             | on
+ enable_hashjoin            | on
+ enable_indexonlyscan       | on
+ enable_indexscan           | on
+ enable_material            | on
+ enable_mergejoin           | on
+ enable_nestloop            | on
+ enable_partition_wise_join | on
+ enable_seqscan             | on
+ enable_sort                | on
+ enable_tidscan             | on
+(12 rows)
 
 CREATE TABLE foo2(fooid int, f2 int);
 INSERT INTO foo2 VALUES(1, 11);
diff --git a/src/test/regress/parallel_schedule b/src/test/regress/parallel_schedule
index 8641769..b61ca3b 100644
--- a/src/test/regress/parallel_schedule
+++ b/src/test/regress/parallel_schedule
@@ -99,8 +99,9 @@ test: select_parallel
 
 # ----------
 # Another group of parallel tests
+# TODO: merge partition_join and multi_level_partition_join
 # ----------
-test: select_views portals_p2 foreign_key cluster dependency guc bitmapops combocid tsearch tsdicts foreign_data window xmlmap functional_deps advisory_lock json jsonb json_encoding indirect_toast equivclass
+test: select_views portals_p2 foreign_key cluster dependency guc bitmapops combocid tsearch tsdicts foreign_data window xmlmap functional_deps advisory_lock json jsonb json_encoding indirect_toast equivclass partition_join multi_level_partition_join
 # ----------
 # Another group of parallel tests
 # NB: temp.sql does a reconnect which transiently uses 2 connections,
diff --git a/src/test/regress/serial_schedule b/src/test/regress/serial_schedule
index 835cf35..5b167b6 100644
--- a/src/test/regress/serial_schedule
+++ b/src/test/regress/serial_schedule
@@ -169,3 +169,5 @@ test: with
 test: xml
 test: event_trigger
 test: stats
+test: partition_join
+test: multi_level_partition_join
diff --git a/src/test/regress/sql/multi_level_partition_join.sql b/src/test/regress/sql/multi_level_partition_join.sql
new file mode 100644
index 0000000..e5895ce
--- /dev/null
+++ b/src/test/regress/sql/multi_level_partition_join.sql
@@ -0,0 +1,95 @@
+--
+-- multi-leveled partitions
+--
+CREATE TABLE prt1_l (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_l_p1 PARTITION OF prt1_l FOR VALUES START (0) END (250) PARTITION BY RANGE (b);
+CREATE TABLE prt1_l_p1_p1 PARTITION OF prt1_l_p1 FOR VALUES START (0) END (100);
+CREATE TABLE prt1_l_p1_p2 PARTITION OF prt1_l_p1 FOR VALUES START (100) END (250);
+CREATE TABLE prt1_l_p2 PARTITION OF prt1_l FOR VALUES START (250) END (500) PARTITION BY RANGE (c);
+CREATE TABLE prt1_l_p2_p1 PARTITION OF prt1_l_p2 FOR VALUES START ('0250') END ('0400');
+CREATE TABLE prt1_l_p2_p2 PARTITION OF prt1_l_p2 FOR VALUES START ('0400') END ('0500');
+CREATE TABLE prt1_l_p3 PARTITION OF prt1_l FOR VALUES START (500) END (600) PARTITION BY RANGE ((b + a));
+CREATE TABLE prt1_l_p3_p1 PARTITION OF prt1_l_p3 FOR VALUES START (1000) END (1100);
+CREATE TABLE prt1_l_p3_p2 PARTITION OF prt1_l_p3 FOR VALUES START (1100) END (1200);
+INSERT INTO prt1_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_l;
+ANALYZE prt1_l_p1;
+ANALYZE prt1_l_p1_p1;
+ANALYZE prt1_l_p1_p2;
+ANALYZE prt1_l_p2;
+ANALYZE prt1_l_p2_p1;
+ANALYZE prt1_l_p2_p2;
+ANALYZE prt1_l_p3;
+ANALYZE prt1_l_p3_p1;
+ANALYZE prt1_l_p3_p2;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_l AS SELECT * FROM prt1_l;
+
+CREATE TABLE prt2_l (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_l_p1 PARTITION OF prt2_l FOR VALUES START (0) END (250) PARTITION BY RANGE (a);
+CREATE TABLE prt2_l_p1_p1 PARTITION OF prt2_l_p1 FOR VALUES START (0) END (100);
+CREATE TABLE prt2_l_p1_p2 PARTITION OF prt2_l_p1 FOR VALUES START (100) END (250);
+CREATE TABLE prt2_l_p2 PARTITION OF prt2_l FOR VALUES START (250) END (500) PARTITION BY RANGE (c);
+CREATE TABLE prt2_l_p2_p1 PARTITION OF prt2_l_p2 FOR VALUES START ('0250') END ('0400');
+CREATE TABLE prt2_l_p2_p2 PARTITION OF prt2_l_p2 FOR VALUES START ('0400') END ('0500');
+CREATE TABLE prt2_l_p3 PARTITION OF prt2_l FOR VALUES START (500) END (600) PARTITION BY RANGE ((a + b));
+CREATE TABLE prt2_l_p3_p1 PARTITION OF prt2_l_p3 FOR VALUES START (1000) END (1100);
+CREATE TABLE prt2_l_p3_p2 PARTITION OF prt2_l_p3 FOR VALUES START (1100) END (1200);
+INSERT INTO prt2_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_l;
+ANALYZE prt2_l_p1;
+ANALYZE prt2_l_p1_p1;
+ANALYZE prt2_l_p1_p2;
+ANALYZE prt2_l_p2;
+ANALYZE prt2_l_p2_p1;
+ANALYZE prt2_l_p2_p2;
+ANALYZE prt2_l_p3;
+ANALYZE prt2_l_p3_p1;
+ANALYZE prt2_l_p3_p2;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_l AS SELECT * FROM prt2_l;
+
+EXPLAIN (VERBOSE, 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 = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 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 = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1, uprt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 LEFT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 RIGHT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_l t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_l t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+
+-- lateral reference
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
+			  ON t1.a = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
+			  ON t1.a = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1_l t2 JOIN uprt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
+			  ON t1.a = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
+			  ON t1.b = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
+			  ON t1.b = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1_l t2 JOIN uprt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
+			  ON t1.b = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
new file mode 100644
index 0000000..b4945df
--- /dev/null
+++ b/src/test/regress/sql/partition_join.sql
@@ -0,0 +1,600 @@
+--
+-- PARTITION_JOIN
+-- Test partition-wise join between partitioned tables
+--
+
+--
+-- partitioned by a single column
+--
+CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES START (0) END (250);
+CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES START (500) END (600);
+CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES START (250) END (500);
+INSERT INTO prt1 SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1 AS SELECT * FROM prt1;
+
+CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES START (0) END (250);
+CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES START (250) END (500);
+CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES START (500) END (600);
+INSERT INTO prt2 SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+CREATE TABLE uprt2 AS SELECT * FROM prt2;
+
+-- inner join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND (t1.a + t2.b) % 120 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND (t1.a + t2.b) % 120 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND (t1.a + t2.b) % 120 = 0 ORDER BY t1.a, t2.b;
+
+-- left outer join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 OR t2.b is null ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+-- right outer join
+EXPLAIN (VERBOSE, 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.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+-- full outer join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+
+-- Join with pruned partitions from joining relations
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a < 450 AND t2.b > 250 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+-- lateral reference
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+--
+-- partitioned by expression
+--
+CREATE TABLE prt1_e (a int, b int, c varchar) PARTITION BY RANGE(((a + b)/2));
+CREATE TABLE prt1_e_p1 PARTITION OF prt1_e FOR VALUES START (0) END (250);
+CREATE TABLE prt1_e_p2 PARTITION OF prt1_e FOR VALUES START (250) END (500);
+CREATE TABLE prt1_e_p3 PARTITION OF prt1_e FOR VALUES START (500) END (600);
+INSERT INTO prt1_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_e AS SELECT * FROM prt1_e;
+
+CREATE TABLE prt2_e (a int, b int, c varchar) PARTITION BY RANGE(((b + a)/2));
+CREATE TABLE prt2_e_p1 PARTITION OF prt2_e FOR VALUES START (0) END (250);
+CREATE TABLE prt2_e_p2 PARTITION OF prt2_e FOR VALUES START (250) END (500);
+CREATE TABLE prt2_e_p3 PARTITION OF prt2_e FOR VALUES START (500) END (600);
+INSERT INTO prt2_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_e;
+ANALYZE prt2_e_p1;
+ANALYZE prt2_e_p2;
+ANALYZE prt2_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_e AS SELECT * FROM prt2_e;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1, uprt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1 LEFT JOIN uprt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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.a % 25 = 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM uprt1 t1, uprt2 t2, uprt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM uprt1_e WHERE uprt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, 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.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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;
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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;
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM uprt1_e WHERE uprt1_e.a % 25 = 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;
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1, uprt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+-- test merge joins with and without using indexes
+SET enable_hashjoin TO off;
+SET enable_nestloop TO off;
+
+CREATE INDEX iprt1_a on prt1(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 iprt2_b on prt2(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 iprt1_e_ab2 on prt1_e(((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));
+
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+SET enable_seqscan TO off;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+-- lateral references and parameterized paths
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+RESET enable_hashjoin;
+RESET enable_nestloop;
+RESET enable_seqscan;
+
+--
+-- partitioned by multiple columns
+--
+CREATE TABLE prt1_m (a int, b int, c varchar) PARTITION BY RANGE(a, ((a + b)/2));
+CREATE TABLE prt1_m_p1 PARTITION OF prt1_m FOR VALUES START (0, 0) END (250, 250);
+CREATE TABLE prt1_m_p2 PARTITION OF prt1_m FOR VALUES START (250, 250) END (500, 500);
+CREATE TABLE prt1_m_p3 PARTITION OF prt1_m FOR VALUES START (500, 500) END (600, 600);
+INSERT INTO prt1_m SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_m;
+ANALYZE prt1_m_p1;
+ANALYZE prt1_m_p2;
+ANALYZE prt1_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_m AS SELECT * FROM prt1_m;
+
+CREATE TABLE prt2_m (a int, b int, c varchar) PARTITION BY RANGE(((b + a)/2), b);
+CREATE TABLE prt2_m_p1 PARTITION OF prt2_m FOR VALUES START (0, 0) END (250, 250);
+CREATE TABLE prt2_m_p2 PARTITION OF prt2_m FOR VALUES START (250, 250) END (500, 500);
+CREATE TABLE prt2_m_p3 PARTITION OF prt2_m FOR VALUES START (500, 500) END (600, 600);
+INSERT INTO prt2_m SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_m;
+ANALYZE prt2_m_p1;
+ANALYZE prt2_m_p2;
+ANALYZE prt2_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_m AS SELECT * FROM prt2_m;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_m t1 RIGHT JOIN uprt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_m t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_m t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+
+--
+-- 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;
+ANALYZE plt1_p1;
+ANALYZE plt1_p2;
+ANALYZE plt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1 AS SELECT * FROM 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;
+ANALYZE plt2_p1;
+ANALYZE plt2_p2;
+ANALYZE plt2_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt2 AS SELECT * FROM plt2;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.a = t2.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.a = t2.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1 t1, uplt2 t2 WHERE t1.c = t2.c AND t1.a = t2.a AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.a AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.a AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.a AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 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 plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 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 uplt1 t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) ORDER BY t1.a, t2.b;
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM (SELECT 50 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM (SELECT 50 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM (SELECT 50 phv, * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t1.phv), avg(t2.b), t2.c, avg(t2.phv) FROM (SELECT 25 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t1.phv), avg(t2.b), t2.c, avg(t2.phv) FROM (SELECT 25 phv, * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t1.phv), avg(t2.b), t2.c, avg(t2.phv) FROM (SELECT 25 phv, * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.c = t2.c AND t1.a = t2.b) GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
+-- Join with pruned partitions from joining relations
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.c NOT IN ('0001', '0005', '0002', '0009') AND t2.c NOT IN ('0000', '0003', '0004', '0010') GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.c NOT IN ('0001', '0005', '0002', '0009') AND t2.c NOT IN ('0000', '0003', '0004', '0010') GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM uplt1 t1, uplt2 t2 WHERE t1.c = t2.c AND t1.c NOT IN ('0001', '0005', '0002', '0009') AND t2.c NOT IN ('0000', '0003', '0004', '0010') GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 LEFT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 LEFT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM uplt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 LEFT JOIN (SELECT * FROM uplt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 RIGHT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 RIGHT JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM uplt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 RIGHT JOIN (SELECT * FROM uplt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 FULL JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM plt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 FULL JOIN (SELECT * FROM plt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, sum(t2.b), t2.c FROM (SELECT * FROM uplt1 t1 WHERE t1.c NOT IN ('0001', '0005', '0002', '0009')) t1 FULL JOIN (SELECT * FROM uplt2 t2 WHERE t2.c NOT IN ('0000', '0003', '0004', '0010')) t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 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;
+ANALYZE plt1_e;
+ANALYZE plt1_e_p1;
+ANALYZE plt1_e_p2;
+ANALYZE plt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1_e AS SELECT * FROM plt1_e;
+
+CREATE TABLE plt2_e (a int, b int, c text) PARTITION BY LIST(ltrim(c, 'A'));
+CREATE TABLE plt2_e_p1 PARTITION OF plt2_e FOR VALUES IN ('0000', '0003', '0004', '0010');
+CREATE TABLE plt2_e_p2 PARTITION OF plt2_e FOR VALUES IN ('0001', '0005', '0002', '0009');
+CREATE TABLE plt2_e_p3 PARTITION OF plt2_e FOR VALUES IN ('0006', '0007', '0008', '0011');
+INSERT INTO plt2_e SELECT i, i, 'A' || to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE plt2_e;
+ANALYZE plt2_e_p1;
+ANALYZE plt2_e_p2;
+ANALYZE plt2_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt2_e AS SELECT * FROM plt2_e;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 LEFT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM plt1_e t1 LEFT JOIN plt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uplt1_e t1 LEFT JOIN uplt2_e t2 ON t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A') WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2_e WHERE plt2_e.b % 25 = 0) t2 ON (t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A')) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2_e WHERE plt2_e.b % 25 = 0) t2 ON (t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A')) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uplt1_e t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2_e t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND ltrim(t1.c, 'A') = ltrim(t2.c, 'A')) ORDER BY t1.a, t2.b;
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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;
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM uplt1 t1, uplt2 t2, uplt1_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;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1, uplt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 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 ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+--
+-- negative testcases
+--
+
+-- joins where one of the relations is proven empty
+EXPLAIN (VERBOSE, 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;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 LEFT JOIN prt2 t2 ON t1.a = t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, 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 % 25 = 0 ORDER BY t1.a, t2.b;
+
+CREATE TABLE prt1_n (a int, b int, c varchar) PARTITION BY RANGE(c);
+CREATE TABLE prt1_n_p1 PARTITION OF prt1_n FOR VALUES START ('0000') END ('0250');
+CREATE TABLE prt1_n_p2 PARTITION OF prt1_n FOR VALUES START ('0250') END ('0500');
+INSERT INTO prt1_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 499, 2) i;
+ANALYZE prt1_n;
+ANALYZE prt1_n_p1;
+ANALYZE prt1_n_p2;
+
+CREATE TABLE prt2_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt2_n_p1 PARTITION OF prt2_n FOR VALUES IN ('0000', '0003', '0004', '0010', '0006', '0007');
+CREATE TABLE prt2_n_p2 PARTITION OF prt2_n FOR VALUES IN ('0001', '0005', '0002', '0009', '0008', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt2_n;
+ANALYZE prt2_n_p1;
+ANALYZE prt2_n_p2;
+
+CREATE TABLE prt3_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt3_n_p1 PARTITION OF prt3_n FOR VALUES IN ('0000', '0004', '0006', '0007');
+CREATE TABLE prt3_n_p2 PARTITION OF prt3_n FOR VALUES IN ('0001', '0002', '0008', '0010');
+CREATE TABLE prt3_n_p3 PARTITION OF prt3_n FOR VALUES IN ('0003', '0005', '0009', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt3_n;
+ANALYZE prt3_n_p1;
+ANALYZE prt3_n_p2;
+ANALYZE prt3_n_p3;
+
+CREATE TABLE prt4_n (a int, b int, c text) PARTITION BY RANGE(a);
+CREATE TABLE prt4_n_p1 PARTITION OF prt4_n FOR VALUES START (0) END (300);
+CREATE TABLE prt4_n_p2 PARTITION OF prt4_n FOR VALUES START (300) END (500);
+CREATE TABLE prt4_n_p3 PARTITION OF prt4_n FOR VALUES START (500) END (600);
+INSERT INTO prt4_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt4_n;
+ANALYZE prt4_n_p1;
+ANALYZE prt4_n_p2;
+ANALYZE prt4_n_p3;
+
+-- partition-wise join can not be applied if the partition ranges differ
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 FULL JOIN prt4_n t2 ON t1.a = t2.a;
+
+-- 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);
+
+-- equi-join with join condition on partial keys does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1, prt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t1.a % 25 = 0;
+
+-- equi-join between out-of-order partition key columns does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.a = t2.b WHERE t1.a % 25 = 0;
+
+-- equi-join between non-key columns does not qualify for partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.c = t2.c WHERE t1.a % 25 = 0;
+
+-- partition-wise join can not be applied for a join between list and range
+-- partitioned table
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1, prt2_n t2 WHERE t1.c = t2.c;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 LEFT JOIN prt2_n t2 ON (t1.c = t2.c);
+
+-- partition-wise join can not be applied between tables with different
+-- partition lists
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 RIGHT JOIN prt1 t2 ON (t1.c = t2.c);
+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);
#27Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Ashutosh Bapat (#26)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

PFA patch rebased after partitioning code was committed.

On Thu, Dec 1, 2016 at 4:32 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Hi Robert,
Sorry for delayed response.

The attached patch implements following ideas:
1. At the time of creating paths - If the joining relations are both
partitioned and join can use partition-wise join, we create paths for
few child-joins. Similar to inheritance relations
(set_append_rel_pathlist()), we collect paths with similar properties
from all sampled child-joins and create one PartitionJoinPath with
each set of paths. The cost of the PartitionJoinPath is obtained by
multiplying the sum of costs of paths in the given set by the ratio of
(number of rows estimated in the parent-join/sum of rows in
child-joins).

2. If the PartitionJoinPath emerges as the best path, we create paths
for each of the remaining child-joins. Then we collect paths with
properties same as the given PartitionJoinPath, one from each
child-join. These paths are converted into plans and a Merge/Append
plan is created combing these plans. The paths and plans for
child-join are created in a temporary memory context. The final plan
for each child-join is copied into planner's context and the temporary
memory context is reset.

Right now, we choose 1% or 1 (whichever is higher) child-joins to base
PartitionJoinPath costs on.

Memory consumption
-----------------------------
I tested a 5-way self-join for a table with 1000 partitions, each
partition having 1M rows. The memory consumed in standard_planner()
was measured with some granular tracking
(mem_usage_func_wise_measurement_slabwise.patch). Partition-wise join
consumed total of 289MB memory which is approx 6.6 times more than
non-partition-wise join which consumed 44MB. That's much better than
the earlier 16 times consumption for 5-way join with 100 partitions.

The extra 245MB memory was consumed by child-join RelOptInfos (48MB),
SpecialJoinInfos for child-joins (64MB), restrictlist translation
(92MB), paths for sampled child-joins (1.5MB), building targetlists
for child-joins (7MB).

In the earlier implementation, a given clause which was applicable to
multiple join orders was getting translated as many times as the join
orders it was applicable in. I changed RestrictInfo for parent to
store a list of RestrictInfos applicable to children to avoid multiple
translations.

My earlier patch created the child-join plans in a temporary context
and then copied them into planner context since the translated clauses
were allocated memory in temporary memory context then. Now that they
are stored in planner's context, we can directly create the plan in
the planner's context.

Third, I added code to free up child SpecialJoinInfos after using those.

As a result the total memory consumption now is 192MB, which is approx
4.4 times the memory consumed during planning in case of
non-partition-wise join.

Choosing representative child-joins:
--------------------------------------------------
There's another angle to choosing representative child joins. In a
partitioned N-way join, different joins covering different subsets of
N relations, will have different size distributions across the
partitions. This means that the child-joins costed for (N-k) joins,
may be different for those required for (N-k+1) joins. With a factor
of 1% sampling, N is such that a child-join participates in 100 joins,
we will end up creating paths for all partitions before creating
PartitionJoinPaths for the final N-way join. Hopefully that will be a
rare case and usually we will end up using paths already created. We
can not avoid creating PartitionJoinPaths for subset joins, as there
might be cases when partition-wise join will be optimal for an N-k way
join but not for N-way join. We may avoid this if we choose
representative child-joins based on their positions, in which case, we
may end up with some or all of those being empty and thus skewing the
costs heavily.

Partial paths
-----------------
AFAIU, we create partial paths for append relation, when all the
children have partial paths. Unlike parameterized paths or path with
pathkeys, there is no way to create a partial path for a normal path.
This means that unless we create paths for all child-joins, we can not
create partial paths for appendrel comprising of child-joins, and thus
can not use parallel query right now. This may not be that bad, since
it would be more efficient to run each child-join in a separate
worker, rather than using multiple workers for a single child-join.

This still applies.

regression tests
----------------------
I observed that for small relations (1000 rows in each partition and
100 partitions), the size estimates in append relations and sum of
those in child relations are very different. As a result, the
extrapolated costs for PartitionJoinPaths as described above, are way
higher than costs of join of appends (or even append of joins if we
are to create paths for all child-joins). Thus with this approach, we
choose partition-wise join for large number of partitions with large
data (e.g. 1000 partitions with 1M rows each). These are certainly the
cases when partition-wise join is a big win. I have not tried to find
out a threshold above which partition-wise join gets chosen with above
approach, but it's going to be a larger threshold. That makes writing
regression tests difficult, as those will require large data. So, we
have to find a way so that we can test partition-wise join with
smaller data. There are few possibilities like 1. convert the fraction
of representative child-joins into GUC and setting it to 100% would
start choosing partition-wise joins for tables with a few hundred rows
per partition, like it did in earlier approach, 2. provide a way to
force partition-wise join whenever possible, by say costing
partition-wise joins much lesser than non-partition-wise join when a
GUC is set (e.g. enable_partition_wise_join with values always, never,
optimal or something like that).

For now I have added a float GUC partition_wise_plan_weight. The
partition-wise join cost derived from the samples is multiplied by
this GUC and set as the cost of ParitionJoinPath. A value of 1 means
that the cost derived from the samples are used as is. A value higher
than 1 discourages use of partition-wise join and that lower than 1
encourages use of partition-wise join. I am not very keen on keeping
this GUC, in this form. But we need some way to run regression with
smaller data.

For now I have disabled partition-wise join for multi-level
partitions. I will post a patch soon with that enabled.
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company

Attachments:

pg_dp_join_v5.patchbinary/octet-stream; name=pg_dp_join_v5.patchDownload
diff --git a/contrib/postgres_fdw/deparse.c b/contrib/postgres_fdw/deparse.c
index 66b059a..b1ea55a 100644
--- a/contrib/postgres_fdw/deparse.c
+++ b/contrib/postgres_fdw/deparse.c
@@ -911,7 +911,7 @@ deparseSelectStmtForRel(StringInfo buf, PlannerInfo *root, RelOptInfo *rel,
 	 * We handle relations for foreign tables, joins between those and upper
 	 * relations.
 	 */
-	Assert(rel->reloptkind == RELOPT_JOINREL ||
+	Assert(IS_JOIN_REL(rel) ||
 		   rel->reloptkind == RELOPT_BASEREL ||
 		   rel->reloptkind == RELOPT_OTHER_MEMBER_REL ||
 		   rel->reloptkind == RELOPT_UPPER_REL);
@@ -990,7 +990,7 @@ deparseSelectSql(List *tlist, List **retrieved_attrs, deparse_expr_cxt *context)
 	 */
 	appendStringInfoString(buf, "SELECT ");
 
-	if (foreignrel->reloptkind == RELOPT_JOINREL ||
+	if (IS_JOIN_REL(foreignrel) ||
 		foreignrel->reloptkind == RELOPT_UPPER_REL)
 	{
 		/* For a join relation use the input tlist */
@@ -1030,7 +1030,7 @@ deparseFromExpr(List *quals, deparse_expr_cxt *context)
 
 	/* For upper relations, scanrel must be either a joinrel or a baserel */
 	Assert(context->foreignrel->reloptkind != RELOPT_UPPER_REL ||
-		   scanrel->reloptkind == RELOPT_JOINREL ||
+		   IS_JOIN_REL(scanrel) ||
 		   scanrel->reloptkind == RELOPT_BASEREL);
 
 	/* Construct FROM clause */
@@ -1178,7 +1178,7 @@ deparseLockingClause(deparse_expr_cxt *context)
 			appendStringInfoString(buf, " FOR UPDATE");
 
 			/* Add the relation alias if we are here for a join relation */
-			if (rel->reloptkind == RELOPT_JOINREL)
+			if (IS_JOIN_REL(rel))
 				appendStringInfo(buf, " OF %s%d", REL_ALIAS_PREFIX, relid);
 		}
 		else
@@ -1345,7 +1345,7 @@ deparseFromExprForRel(StringInfo buf, PlannerInfo *root, RelOptInfo *foreignrel,
 {
 	PgFdwRelationInfo *fpinfo = (PgFdwRelationInfo *) foreignrel->fdw_private;
 
-	if (foreignrel->reloptkind == RELOPT_JOINREL)
+	if (IS_JOIN_REL(foreignrel))
 	{
 		RelOptInfo *rel_o = fpinfo->outerrel;
 		RelOptInfo *rel_i = fpinfo->innerrel;
diff --git a/contrib/postgres_fdw/postgres_fdw.c b/contrib/postgres_fdw/postgres_fdw.c
index fbe6929..7b3d8bb 100644
--- a/contrib/postgres_fdw/postgres_fdw.c
+++ b/contrib/postgres_fdw/postgres_fdw.c
@@ -721,8 +721,8 @@ get_useful_ecs_for_relation(PlannerInfo *root, RelOptInfo *rel)
 		return useful_eclass_list;
 
 	/* If this is a child rel, we must use the topmost parent rel to search. */
-	if (rel->reloptkind == RELOPT_OTHER_MEMBER_REL)
-		relids = find_childrel_top_parent(root, rel)->relids;
+	if (IS_OTHER_REL(rel))
+		relids = find_childrel_top_parent(root, rel);
 	else
 		relids = rel->relids;
 
@@ -1183,7 +1183,7 @@ postgresGetForeignPlan(PlannerInfo *root,
 			local_exprs = lappend(local_exprs, rinfo->clause);
 	}
 
-	if (foreignrel->reloptkind == RELOPT_JOINREL ||
+	if (IS_JOIN_REL(foreignrel) ||
 		foreignrel->reloptkind == RELOPT_UPPER_REL)
 	{
 		/* For a join relation, get the conditions from fdw_private structure */
@@ -1249,7 +1249,7 @@ postgresGetForeignPlan(PlannerInfo *root,
 							 remote_conds,
 							 retrieved_attrs,
 							 makeInteger(fpinfo->fetch_size));
-	if (foreignrel->reloptkind == RELOPT_JOINREL ||
+	if (IS_JOIN_REL(foreignrel->reloptkind) ||
 		foreignrel->reloptkind == RELOPT_UPPER_REL)
 		fdw_private = lappend(fdw_private,
 							  makeString(fpinfo->relation_name->data));
@@ -2529,7 +2529,7 @@ estimate_path_cost_size(PlannerInfo *root,
 						   &remote_param_join_conds, &local_param_join_conds);
 
 		/* Build the list of columns to be fetched from the foreign server. */
-		if (foreignrel->reloptkind == RELOPT_JOINREL ||
+		if (IS_JOIN_REL(foreignrel) ||
 			foreignrel->reloptkind == RELOPT_UPPER_REL)
 			fdw_scan_tlist = build_tlist_to_deparse(foreignrel);
 		else
@@ -2611,7 +2611,7 @@ estimate_path_cost_size(PlannerInfo *root,
 			startup_cost = fpinfo->rel_startup_cost;
 			run_cost = fpinfo->rel_total_cost - fpinfo->rel_startup_cost;
 		}
-		else if (foreignrel->reloptkind == RELOPT_JOINREL)
+		else if (IS_JOIN_REL(foreignrel))
 		{
 			PgFdwRelationInfo *fpinfo_i;
 			PgFdwRelationInfo *fpinfo_o;
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 9980582..41ffc94 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -20,6 +20,7 @@
 #include "access/nbtree.h"
 #include "access/sysattr.h"
 #include "catalog/dependency.h"
+#include "catalog/heap.h"
 #include "catalog/indexing.h"
 #include "catalog/objectaddress.h"
 #include "catalog/partition.h"
@@ -139,6 +140,8 @@ static int32 partition_bound_cmp(PartitionKey key,
 static int partition_bound_bsearch(PartitionKey key,
 						PartitionBoundInfo boundinfo,
 						void *probe, bool probe_is_bound, bool *is_equal);
+static PartitionBoundInfo partition_bounds_copy(PartitionBoundInfo src,
+									PartitionKey key);
 
 /* Support get_partition_for_tuple() */
 static void FormPartitionKeyDatum(PartitionDispatch pd,
@@ -1942,3 +1945,250 @@ partition_bound_bsearch(PartitionKey key, PartitionBoundInfo boundinfo,
 
 	return lo;
 }
+
+ /*
+ * Return a copy of given PartitionBoundInfo structure. The data types of bounds
+ * are described by given partition key specificiation.
+ */
+static PartitionBoundInfo
+partition_bounds_copy(PartitionBoundInfo src, PartitionKey key)
+{
+	PartitionBoundInfo	dest;
+	int		i;
+	int		ndatums;
+	int		partnatts;
+	int		num_indexes;
+
+	dest = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+
+	dest->strategy = src->strategy;
+	ndatums = dest->ndatums = src->ndatums;
+	partnatts = key->partnatts;
+
+	/* Range partitioned table has an extra index. */
+	num_indexes = key->strategy == PARTITION_STRATEGY_RANGE ? ndatums + 1 : ndatums;
+
+	/* List partitioned tables have only a single partition key. */
+	Assert(key->strategy != PARTITION_STRATEGY_LIST || partnatts == 1);
+
+	dest->datums = (Datum **) palloc(sizeof(Datum *) * ndatums);
+
+	for (i = 0; i < ndatums; i++)
+	{
+		int		j;
+		dest->datums[i] = (Datum *) palloc(sizeof(Datum) * partnatts);
+
+		for (j = 0; j < partnatts; j++)
+			dest->datums[i][j] = datumCopy(src->datums[i][j],
+										   key->parttypbyval[j],
+										   key->parttyplen[j]);
+	}
+
+	if (src->content)
+	{
+		dest->content = (RangeDatumContent **) palloc(ndatums *
+												sizeof(RangeDatumContent *));
+		for (i = 0; i < ndatums; i++)
+		{
+			dest->content[i] = (RangeDatumContent *) palloc(partnatts *
+												sizeof(RangeDatumContent));
+
+			memcpy(dest->content[i], src->content[i],
+				   sizeof(RangeDatumContent) * key->partnatts);
+		}
+	}
+	else
+		dest->content = NULL;
+
+	dest->indexes = (int *) palloc(sizeof(int) * num_indexes);
+	memcpy(dest->indexes, src->indexes, sizeof(int) * num_indexes);
+
+	dest->has_null = src->has_null;
+	dest->null_index = src->null_index;
+
+	return dest;
+}
+
+/*
+ * find_partition_scheme
+ *
+ * The function returns a canonical partition scheme which exactly matches the
+ * partitioning properties of the given relation if one exists in the of
+ * canonical partitioning schemes maintained in PlannerInfo. If none of the
+ * existing partitioning schemes match, the function creates a canonical
+ * partition scheme and adds it to the list.
+ *
+ * For an unpartitioned table or for a multi-level partitioned table it returns
+ * NULL.
+ */
+extern PartitionScheme
+find_partition_scheme(PlannerInfo *root, Relation relation)
+{
+	PartitionKey	part_key = RelationGetPartitionKey(relation);
+	PartitionDesc	part_desc = RelationGetPartitionDesc(relation);
+	ListCell	   *lc;
+	int		nparts;
+	int		partnatts;
+	int		cnt_pks;
+	int		cnt_parts;
+	PartitionScheme	part_scheme = NULL;
+
+	/* No partition scheme for an unpartitioned relation. */
+	if (!part_desc || !part_key)
+		return NULL;
+
+	nparts = part_desc->nparts;
+	partnatts = part_key->partnatts;
+
+	/*
+	 * For a multi-level partitioned table, we do not retain the partitioning
+	 * hierarchy while expanding RTE for the topmost parent. Thus the number of
+	 * children as per root->append_rel_list does not match the number of
+	 * partitions specified in the partition descriptor and hence the
+	 * partitioning scheme of a multi-partitioned table does not reflect the
+	 * true picture. So for now, treat a multi-partitioned table as not
+	 * partitioned.
+	 */
+	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	{
+		if (has_subclass(part_desc->oids[cnt_parts]))
+			return NULL;
+	}
+
+	/* Search for a matching partition scheme and return if found one. */
+	foreach (lc, root->part_schemes)
+	{
+		part_scheme = lfirst(lc);
+
+		/* Match number of partitions and partitioning strategy. */
+		if (nparts != part_scheme->nparts ||
+			part_key->strategy != part_scheme->strategy ||
+			partnatts != part_scheme->partnatts)
+			continue;
+
+		/* Match the partition key types. */
+		for (cnt_pks = 0; cnt_pks < partnatts; cnt_pks++)
+		{
+			/*
+			 * For types, it suffices to match the type id, mod and collation;
+			 * len, byval and align are depedent on the first two.
+			 */
+			if (part_key->partopfamily[cnt_pks] != part_scheme->partopfamily[cnt_pks] ||
+				part_key->partopcintype[cnt_pks] != part_scheme->partopcintype[cnt_pks] ||
+				part_key->parttypid[cnt_pks] != part_scheme->key_types[cnt_pks] ||
+				part_key->parttypmod[cnt_pks] != part_scheme->key_typmods[cnt_pks] ||
+				part_key->parttypcoll[cnt_pks] != part_scheme->key_collations[cnt_pks])
+				break;
+		}
+
+		/* Some partition key didn't match. Check next partitioning scheme. */
+		if (cnt_pks < partnatts)
+			continue;
+
+		if (!partition_bounds_equal(part_key, part_desc->boundinfo,
+									part_scheme->boundinfo))
+			continue;
+
+		/* Found matching partition scheme. */
+		return part_scheme;
+	}
+
+	/* Did not find matching partition scheme. Create one. */
+	part_scheme = (PartitionScheme) palloc0(sizeof(PartitionSchemeData));
+
+	/* Copy partition bounds/lists. */
+	part_scheme->nparts = part_desc->nparts;
+	part_scheme->strategy = part_key->strategy;
+	part_scheme->boundinfo = partition_bounds_copy(part_desc->boundinfo,
+												   part_key);
+
+	/* Store partition key information. */
+	part_scheme->partnatts = part_key->partnatts;
+
+	part_scheme->partopfamily = (Oid *) palloc(sizeof(Oid) * partnatts);
+	memcpy(part_scheme->partopfamily, part_key->partopfamily,
+		   sizeof(Oid) * partnatts);
+
+	part_scheme->partopcintype = (Oid *) palloc(sizeof(Oid) * partnatts);
+	memcpy(part_scheme->partopcintype, part_key->partopcintype,
+		   sizeof(Oid) * partnatts);
+
+	part_scheme->key_types = (Oid *) palloc(sizeof(Oid) * partnatts);
+	memcpy(part_scheme->key_types, part_key->parttypid,
+		   sizeof(Oid) * partnatts);
+
+	part_scheme->key_typmods = (int32 *) palloc(sizeof(int32) * partnatts);
+	memcpy(part_scheme->key_typmods, part_key->parttypmod,
+		   sizeof(int32) * partnatts);
+
+	part_scheme->key_collations = (Oid *) palloc(sizeof(Oid) * partnatts);
+	memcpy(part_scheme->key_collations, part_key->parttypcoll,
+		   sizeof(Oid) * partnatts);
+
+	/* Add the partitioning scheme to PlannerInfo. */
+	root->part_schemes = lappend(root->part_schemes, part_scheme);
+
+	return part_scheme;
+}
+
+/*
+ * build_baserel_partition_key_exprs
+ *
+ * Collect partition key expressions for a given base relation. The function
+ * converts any single column partition keys into corresponding Var nodes. It
+ * restamps Var nodes in partition key expressions by given varno. The
+ * partition key expressions are returned as an array of single element Lists
+ * to be stored in RelOptInfo of the base relation.
+ */
+extern List **
+build_baserel_partition_key_exprs(Relation relation, Index varno)
+{
+	PartitionKey	part_key = RelationGetPartitionKey(relation);
+	int		num_pkexprs;
+	int		cnt_pke;
+	List		  **partexprs;
+	ListCell	   *lc;
+
+	if (!part_key || part_key->partnatts <= 0)
+		return NULL;
+
+	num_pkexprs = part_key->partnatts;
+	partexprs = (List **) palloc(sizeof(List *) * num_pkexprs);
+	lc = list_head(part_key->partexprs);
+
+	for (cnt_pke = 0; cnt_pke < num_pkexprs; cnt_pke++)
+	{
+		AttrNumber attno = part_key->partattrs[cnt_pke];
+		Expr	  *pkexpr;
+
+		if (attno != InvalidAttrNumber)
+		{
+			/* Single column partition key is stored as a Var node. */
+			Form_pg_attribute att_tup;
+
+			if (attno < 0)
+				att_tup = SystemAttributeDefinition(attno,
+												relation->rd_rel->relhasoids);
+			else
+				att_tup = relation->rd_att->attrs[attno - 1];
+
+			pkexpr = (Expr *) makeVar(varno, attno, att_tup->atttypid,
+									  att_tup->atttypmod,
+									  att_tup->attcollation, 0);
+		}
+		else
+		{
+			if (lc == NULL)
+				elog(ERROR, "wrong number of partition key expressions");
+
+			/* Re-stamp the expressions with given varno. */
+			pkexpr = (Expr *) copyObject(lfirst(lc));
+			ChangeVarNodes((Node *) pkexpr, 1, varno, 0);
+			lc = lnext(lc);
+		}
+
+		partexprs[cnt_pke] = list_make1(pkexpr);
+	}
+
+	return partexprs;
+}
diff --git a/src/backend/foreign/foreign.c b/src/backend/foreign/foreign.c
index 242d6d2..75c95e4 100644
--- a/src/backend/foreign/foreign.c
+++ b/src/backend/foreign/foreign.c
@@ -721,7 +721,7 @@ GetExistingLocalJoinPath(RelOptInfo *joinrel)
 {
 	ListCell   *lc;
 
-	Assert(joinrel->reloptkind == RELOPT_JOINREL);
+	Assert(IS_JOIN_REL(joinrel));
 
 	foreach(lc, joinrel->pathlist)
 	{
@@ -786,7 +786,7 @@ GetExistingLocalJoinPath(RelOptInfo *joinrel)
 			ForeignPath *foreign_path;
 
 			foreign_path = (ForeignPath *) joinpath->outerjoinpath;
-			if (foreign_path->path.parent->reloptkind == RELOPT_JOINREL)
+			if (IS_JOIN_REL(foreign_path->path.parent))
 				joinpath->outerjoinpath = foreign_path->fdw_outerpath;
 		}
 
@@ -795,7 +795,7 @@ GetExistingLocalJoinPath(RelOptInfo *joinrel)
 			ForeignPath *foreign_path;
 
 			foreign_path = (ForeignPath *) joinpath->innerjoinpath;
-			if (foreign_path->path.parent->reloptkind == RELOPT_JOINREL)
+			if (IS_JOIN_REL(foreign_path->path.parent))
 				joinpath->innerjoinpath = foreign_path->fdw_outerpath;
 		}
 
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index d973225..fa0569c 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -2049,6 +2049,14 @@ _copyRestrictInfo(const RestrictInfo *from)
 	COPY_SCALAR_FIELD(hashjoinoperator);
 	COPY_SCALAR_FIELD(left_bucketsize);
 	COPY_SCALAR_FIELD(right_bucketsize);
+	/*
+	 * Do not copy parent_rinfo and child_rinfos because 1. they create a
+	 * circular dependency between child and parent RestrictInfo 2. dropping
+	 * those links just means that we loose some memory optimizations. 3. There
+	 * is a possibility that the child and parent RestrictInfots themselves may
+	 * have got copied and thus the old links may no longer be valid. The
+	 * caller may set up those links itself, if needed.
+	 */
 
 	return newnode;
 }
diff --git a/src/backend/optimizer/README b/src/backend/optimizer/README
index 775bcc3..d755f1f 100644
--- a/src/backend/optimizer/README
+++ b/src/backend/optimizer/README
@@ -974,3 +974,56 @@ be desirable to postpone the Gather stage until as near to the top of the
 plan as possible.  Expanding the range of cases in which more work can be
 pushed below the Gather (and costing them accurately) is likely to keep us
 busy for a long time to come.
+
+Partition-wise 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 for a given row in a given partition of a given
+partitioned table, its joining row, if exists, should exist only in the
+matching partition of the other partitioned table; no row from non-matching
+partitions in the other partitioned table can join with the given row from the
+first table. This condition allows the join between partitioned table 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
+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 partition tables
+into join between their partitions is called partition-wise join. We will use
+term "partitioned relation" for both partitioned table as well as join between
+partitioned tables which can use partition-wise join technique.
+
+Partitioning properties of a partitioned table are stored in
+PartitionSchemeData structure. Planner maintains a list of canonical partition
+schemes (distinct PartitionSchemeData objects) so that any two partitioned
+relations with same partitioning scheme share the same PartitionSchemeData
+object. This reduces memory consumed by PartitionSchemeData objects and makes
+it easy to compare the partition schemes of joining relations. RelOptInfos of
+partitioned relations hold partition key expressions and the RelOptInfos of
+the partition relations of that relation.
+
+Partition-wise joins are planned in two phases
+
+1. First phase creates the RelOptInfos for joins between matching partitions,
+henceforth referred to as child-joins. The number of paths created for a
+child-join i.e. join between partitions is same as the number of paths created
+for join between parents. That number grows exponentially with the number of
+base relations being joined. The time and memory consumed to create paths for
+each child-join will be proporional to the number of partitions. This will not
+scale well with thousands of partitions. Instead of that we estimate
+partition-wise join cost based on the costs of sampled child-joins. We choose
+child-joins with higher sizes to have realistic estimates. If the number of
+sampled child-joins is same as the number of live child-joins, we create append
+paths as we know costs of all required child-joins. Otherwise we create
+PartitionJoinPaths with cost estimates based on the costs of sampled
+child-joins. While creating append paths or PartitionJoin paths we create paths
+for all the different possible parameterizations and pathkeys available in the
+sampled child-joins.
+
+2. If PartitionJoinPath emerges as the best possible path, we create paths for
+each unsampled child-join. From every child-join we choose the cheapest path
+with same parameterization or pathkeys as the PartitionJoinPath. This path is
+converted into a plan and all the child-join plans are combined using an Append
+or MergeAppend plan as appropriate. We use a fresh memory context for planning
+each unsampled child-join, thus reducing memory consumption.
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 9753a26..0a2c131 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -18,8 +18,10 @@
 #include <limits.h>
 #include <math.h>
 
+#include "miscadmin.h"
 #include "access/sysattr.h"
 #include "access/tsmapi.h"
+#include "catalog/partition.h"
 #include "catalog/pg_class.h"
 #include "catalog/pg_operator.h"
 #include "catalog/pg_proc.h"
@@ -44,6 +46,7 @@
 #include "parser/parsetree.h"
 #include "rewrite/rewriteManip.h"
 #include "utils/lsyscache.h"
+#include "utils/rel.h"
 
 
 /* results of subquery_is_pushdown_safe */
@@ -93,8 +96,8 @@ static void set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 static void set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 						Index rti, RangeTblEntry *rte);
 static void generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
-						   List *live_childrels,
-						   List *all_child_pathkeys);
+						   List *live_childrels, List *all_child_pathkeys,
+						   bool partition_join_path);
 static Path *get_cheapest_parameterized_child_path(PlannerInfo *root,
 									  RelOptInfo *rel,
 									  Relids required_outer);
@@ -126,6 +129,8 @@ static void subquery_push_qual(Query *subquery,
 static void recurse_push_qual(Node *setOp, Query *topquery,
 				  RangeTblEntry *rte, Index rti, Node *qual);
 static void remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel);
+static void add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
+								List *live_childrels, bool partition_join_path);
 
 
 /*
@@ -868,6 +873,34 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 	double	   *parent_attrsizes;
 	int			nattrs;
 	ListCell   *l;
+	Oid		   *part_oids = NULL;
+	int			nparts = 0;
+
+	/*
+	 * We require OIDs of the partitions to arrange the child RelOptInfos to
+	 * match the lists/ranges specified in the partitioning scheme. Fetch those
+	 * here so as keep those handy when going to child RelOptInfos below.
+	 */
+	if (rel->part_scheme)
+	{
+		RangeTblEntry	*rte = root->simple_rte_array[rel->relid];
+
+		/*
+		 * We need not lock the relation since it was already locked, either by
+		 * the rewriter or when expand_inherited_rtentry() added it to the
+		 * query's rangetable.
+		 */
+		Relation	relation = heap_open(rte->relid, NoLock);
+		PartitionDesc	part_desc = RelationGetPartitionDesc(relation);
+
+		part_oids = part_desc->oids;
+		nparts = part_desc->nparts;
+
+		Assert(part_oids && nparts > 0);
+
+		rel->part_rels = (RelOptInfo **)palloc0(sizeof(RelOptInfo *) * nparts);
+		heap_close(relation, NoLock);
+	}
 
 	/*
 	 * Initialize to compute size estimates for whole append relation.
@@ -899,6 +932,8 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		Node	   *childqual;
 		ListCell   *parentvars;
 		ListCell   *childvars;
+		int			cnt_parts;
+		List	   *appinfos = list_make1(appinfo);
 
 		/* append_rel_list contains all append rels; ignore others */
 		if (appinfo->parent_relid != parentRTindex)
@@ -912,8 +947,95 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		 * add_base_rels_to_query.
 		 */
 		childrel = find_base_rel(root, childRTindex);
+
+		/*
+		 * Recursively save topmost parent's relid in RelOptInfos of
+		 * partitions.
+		 */
+		if (rel->top_parent_relids)
+			childrel->top_parent_relids = rel->top_parent_relids;
+		else
+			childrel->top_parent_relids = bms_copy(rel->relids);
+
 		Assert(childrel->reloptkind == RELOPT_OTHER_MEMBER_REL);
 
+
+		/*
+		 * For two partitioned tables with the same partitioning scheme, it is
+		 * assumed that the Oids of matching partitions from both the tables
+		 * are placed at the same position in the array of partition oids in
+		 * respective partition descriptors. Saving the RelOptInfo of a
+		 * partition in the same cardinal position as its Oid makes it easy to
+		 * find the RelOptInfos of matching partitions for partition-wise join.
+		 */
+		for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+		{
+			if (part_oids[cnt_parts] == childRTE->relid)
+			{
+				Assert(!rel->part_rels[cnt_parts]);
+				rel->part_rels[cnt_parts] = childrel;
+			}
+		}
+
+		/*
+		 * For a partitioned tables, individual partitions can participate in
+		 * the pair-wise joins. We need attr_needed data for building pair-wise
+		 * join relations. Partition tables should have same layout as the
+		 * parent table and hence should not need any translation. But rest of
+		 * the code still uses inheritance mechanism. So does this code. For
+		 * other inherited children, attr_needed is only examined for base
+		 * relations, no otherrels. So we compute attr_needed only for children
+		 * of a partitioned table.
+		 */
+		if (rel->part_scheme)
+		{
+			AttrNumber attno;
+			for (attno = rel->min_attr; attno <= rel->max_attr; attno++)
+			{
+				int	index = attno - rel->min_attr;
+				Relids	attr_needed = bms_copy(rel->attr_needed[index]);
+
+				/*
+				 * System attributes do not need translation. In such a case,
+				 * the attribute numbers of the parent and the child should
+				 * start from the same minimum attribute.
+				 */
+				if (attno <= 0)
+				{
+					Assert(rel->min_attr == childrel->min_attr);
+					childrel->attr_needed[index] = attr_needed;
+				}
+				else
+				{
+					Var *var = list_nth(appinfo->translated_vars,
+										attno - 1);
+					int child_index;
+
+					/* Parent Var translates to child Var. */
+					Assert(IsA(var, Var));
+
+					child_index = var->varattno - childrel->min_attr;
+					childrel->attr_needed[child_index] = attr_needed;
+				}
+			}
+		}
+
+		/*
+		 * Copy/Modify targetlist. Even if this child is deemed empty, we need
+		 * its targetlist in case it falls on nullable side in a child-join
+		 * because of partition-wise join.
+		 *
+		 * NB: the resulting childrel->reltarget->exprs may contain arbitrary
+		 * expressions, which otherwise would not occur in a rel's targetlist.
+		 * Code that might be looking at an appendrel child must cope with
+		 * such.  (Normally, a rel's targetlist would only include Vars and
+		 * PlaceHolderVars.)  XXX we do not bother to update the cost or width
+		 * fields of childrel->reltarget; not clear if that would be useful.
+		 */
+		childrel->reltarget->exprs = (List *)
+			adjust_appendrel_attrs(root,
+								   (Node *) rel->reltarget->exprs, appinfos);
+
 		/*
 		 * We have to copy the parent's targetlist and quals to the child,
 		 * with appropriate substitution of variables.  However, only the
@@ -931,7 +1053,7 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		childquals = get_all_actual_clauses(rel->baserestrictinfo);
 		childquals = (List *) adjust_appendrel_attrs(root,
 													 (Node *) childquals,
-													 appinfo);
+													 appinfos);
 		childqual = eval_const_expressions(root, (Node *)
 										   make_ands_explicit(childquals));
 		if (childqual && IsA(childqual, Const) &&
@@ -960,24 +1082,9 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 			continue;
 		}
 
-		/*
-		 * CE failed, so finish copying/modifying targetlist and join quals.
-		 *
-		 * NB: the resulting childrel->reltarget->exprs may contain arbitrary
-		 * expressions, which otherwise would not occur in a rel's targetlist.
-		 * Code that might be looking at an appendrel child must cope with
-		 * such.  (Normally, a rel's targetlist would only include Vars and
-		 * PlaceHolderVars.)  XXX we do not bother to update the cost or width
-		 * fields of childrel->reltarget; not clear if that would be useful.
-		 */
-		childrel->joininfo = (List *)
-			adjust_appendrel_attrs(root,
-								   (Node *) rel->joininfo,
-								   appinfo);
-		childrel->reltarget->exprs = (List *)
-			adjust_appendrel_attrs(root,
-								   (Node *) rel->reltarget->exprs,
-								   appinfo);
+		/* CE failed, so finish copying/modifying join quals. */
+		childrel->joininfo = build_child_clauses(root, rel->joininfo,
+												 appinfos);
 
 		/*
 		 * We have to make child entries in the EquivalenceClass data
@@ -992,14 +1099,6 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		childrel->has_eclass_joins = rel->has_eclass_joins;
 
 		/*
-		 * Note: we could compute appropriate attr_needed data for the child's
-		 * variables, by transforming the parent's attr_needed through the
-		 * translated_vars mapping.  However, currently there's no need
-		 * because attr_needed is only examined for base relations not
-		 * otherrels.  So we just leave the child's attr_needed empty.
-		 */
-
-		/*
 		 * If parallelism is allowable for this query in general, see whether
 		 * it's allowable for this childrel in particular.  But if we've
 		 * already decided the appendrel is not parallel-safe as a whole,
@@ -1080,6 +1179,16 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		}
 	}
 
+	/* Should have found all the childrels of a partitioned relation. */
+	if (rel->part_scheme)
+	{
+		int		cnt_parts;
+		for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+			if (!rel->part_rels[cnt_parts])
+				elog(ERROR, "could not find the RelOptInfo of a partition with oid %u",
+					 part_oids[cnt_parts]);
+	}
+
 	if (has_live_children)
 	{
 		/*
@@ -1122,19 +1231,11 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 {
 	int			parentRTindex = rti;
 	List	   *live_childrels = NIL;
-	List	   *subpaths = NIL;
-	bool		subpaths_valid = true;
-	List	   *partial_subpaths = NIL;
-	bool		partial_subpaths_valid = true;
-	List	   *all_child_pathkeys = NIL;
-	List	   *all_child_outers = NIL;
 	ListCell   *l;
 
 	/*
-	 * Generate access paths for each member relation, and remember the
-	 * cheapest path for each one.  Also, identify all pathkeys (orderings)
-	 * and parameterizations (required_outer sets) available for the member
-	 * relations.
+	 * Generate access paths for each member relation and remember the
+	 * non-dummy children.
 	 */
 	foreach(l, root->append_rel_list)
 	{
@@ -1142,7 +1243,6 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 		int			childRTindex;
 		RangeTblEntry *childRTE;
 		RelOptInfo *childrel;
-		ListCell   *lcp;
 
 		/* append_rel_list contains all append rels; ignore others */
 		if (appinfo->parent_relid != parentRTindex)
@@ -1177,6 +1277,64 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 		 * Child is live, so add it to the live_childrels list for use below.
 		 */
 		live_childrels = lappend(live_childrels, childrel);
+	}
+
+	/* Add Append/MergeAppend paths to the "append" relation. */
+	add_paths_to_append_rel(root, rel, live_childrels, false);
+}
+
+/*
+ * add_paths_to_append_rel
+ *		Generate Append/MergeAppend paths for given "append" relation. An
+ *		"append" relation can be a base parent relation or a join between
+ *		partitioned tables.
+ *
+ * The function collects all parameterizations and orderings supported by the
+ * non-dummy children. For every such parameterization or ordering, it creates
+ * an append path collecting one path from each non-dummy child with given
+ * parameterization or ordering. Similarly it collects partial paths from
+ * non-dummy children to create partial append paths.
+ */
+static void
+add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
+						List *live_childrels, bool partition_join_path)
+{
+	List	   *subpaths = NIL;
+	bool		subpaths_valid = true;
+	List	   *partial_subpaths = NIL;
+	bool		partial_subpaths_valid;
+	List	   *all_child_pathkeys = NIL;
+	List	   *all_child_outers = NIL;
+	ListCell   *l;
+
+	/*
+	 * While creating PartitionJoinPath, we sample paths from only a few child
+	 * relations. Even if all of sampled children have partial paths, it's not
+	 * guaranteed that all the unsampled children will have partial paths.
+	 * Hence we do not create partial PartitionJoinPaths.
+	 */
+	partial_subpaths_valid = !partition_join_path ? true : false;
+
+	/* An append relation with all its children dummy is dummy. */
+	if (live_childrels == NIL)
+	{
+		/* Mark the relation as dummy, if not already done so. */
+		if (!IS_DUMMY_REL(rel))
+			set_dummy_rel_pathlist(rel);
+
+		/* No more paths need to be added. */
+		return;
+	}
+
+	/*
+	 * For every non-dummy child, remember the cheapest path.  Also, identify
+	 * all pathkeys (orderings) and parameterizations (required_outer sets)
+	 * available for the non-dummy member relations.
+	 */
+	foreach (l, live_childrels)
+	{
+		RelOptInfo *childrel = lfirst(l);
+		ListCell   *lcp;
 
 		/*
 		 * If child has an unparameterized cheapest-total path, add that to
@@ -1267,7 +1425,17 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 	 * if we have zero or one live subpath due to constraint exclusion.)
 	 */
 	if (subpaths_valid)
-		add_path(rel, (Path *) create_append_path(rel, subpaths, NULL, 0));
+	{
+		Path *path;
+
+		if (partition_join_path)
+			path = (Path *) create_partition_join_path(rel, subpaths,
+													   NULL);
+		else
+			path = (Path *) create_append_path(rel, subpaths, NULL, 0);
+
+		add_path(rel, path);
+	}
 
 	/*
 	 * Consider an append of partial unordered, unparameterized partial paths.
@@ -1278,6 +1446,8 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 		ListCell   *lc;
 		int			parallel_workers = 0;
 
+		Assert(!partition_join_path);
+
 		/*
 		 * Decide on the number of workers to request for this append path.
 		 * For now, we just use the maximum value from among the members.  It
@@ -1304,7 +1474,7 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 	 */
 	if (subpaths_valid)
 		generate_mergeappend_paths(root, rel, live_childrels,
-								   all_child_pathkeys);
+								   all_child_pathkeys, partition_join_path);
 
 	/*
 	 * Build Append paths for each parameterization seen among the child rels.
@@ -1345,8 +1515,16 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 		}
 
 		if (subpaths_valid)
-			add_path(rel, (Path *)
-					 create_append_path(rel, subpaths, required_outer, 0));
+		{
+			Path *path;
+
+			if (partition_join_path)
+				path = (Path *) create_partition_join_path(rel, subpaths, required_outer);
+			else
+				path = (Path *) create_append_path(rel, subpaths, required_outer, 0);
+
+			add_path(rel, path);
+		}
 	}
 }
 
@@ -1376,7 +1554,7 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 static void
 generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
 						   List *live_childrels,
-						   List *all_child_pathkeys)
+						   List *all_child_pathkeys, bool partition_join_path)
 {
 	ListCell   *lcp;
 
@@ -1387,6 +1565,7 @@ generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
 		List	   *total_subpaths = NIL;
 		bool		startup_neq_total = false;
 		ListCell   *lcr;
+		Path	   *path;
 
 		/* Select the child paths for this ordering... */
 		foreach(lcr, live_childrels)
@@ -1434,17 +1613,29 @@ generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
 		}
 
 		/* ... and build the MergeAppend paths */
-		add_path(rel, (Path *) create_merge_append_path(root,
-														rel,
-														startup_subpaths,
-														pathkeys,
-														NULL));
+		if (partition_join_path)
+			path = (Path *) create_partition_join_path_with_pathkeys(root, rel,
+															 startup_subpaths,
+															 pathkeys, NULL);
+		else
+			path = (Path *) create_merge_append_path(root, rel,
+													 startup_subpaths,
+													 pathkeys, NULL);
+		add_path(rel, path);
+
 		if (startup_neq_total)
-			add_path(rel, (Path *) create_merge_append_path(root,
-															rel,
-															total_subpaths,
-															pathkeys,
-															NULL));
+		{
+			if (partition_join_path)
+				path = (Path *) create_partition_join_path_with_pathkeys(root,
+																rel,
+																total_subpaths,
+																pathkeys, NULL);
+			else
+				path = (Path *) create_merge_append_path(root, rel,
+														 total_subpaths,
+														 pathkeys, NULL);
+			add_path(rel, path);
+		}
 	}
 }
 
@@ -2186,15 +2377,27 @@ standard_join_search(PlannerInfo *root, int levels_needed, List *initial_rels)
 
 		/*
 		 * Run generate_gather_paths() for each just-processed joinrel.  We
-		 * could not do this earlier because both regular and partial paths
-		 * can get added to a particular joinrel at multiple times within
-		 * join_search_one_level.  After that, we're done creating paths for
-		 * the joinrel, so run set_cheapest().
+		 * could not do this earlier because both regular and partial paths can
+		 * get added to a particular joinrel at multiple times within
+		 * join_search_one_level.
+		 *
+		 * Similarly, create paths for joinrels which used partition-wise join
+		 * technique. generate_partition_wise_join_paths() creates paths for
+		 * only few of the child-joins with highest sizes. Though we calculate
+		 * size of a child-join only once; when it gets created, it may be
+		 * deemed empty while considering various join orders within
+		 * join_search_one_level.
+		 *
+		 * After that, we're done creating paths for the joinrel, so run
+		 * set_cheapest().
 		 */
 		foreach(lc, root->join_rel_level[lev])
 		{
 			rel = (RelOptInfo *) lfirst(lc);
 
+			/* Create paths for partition-wise joins. */
+			generate_partition_wise_join_paths(root, rel);
+
 			/* Create GatherPaths for any useful partial paths for rel */
 			generate_gather_paths(root, rel);
 
@@ -2866,6 +3069,151 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
 	}
 }
 
+/* Fraction of child relations to base cost on. */
+#define FRACTION_PARTS_TO_PLAN 0.01
+
+/*
+ * generate_partition_wise_join_paths
+ *
+ * 		Create paths representing partition-wise join for given partitioned
+ * 		join relation.
+ *
+ * The number of paths created for a child-join is same as the number of paths
+ * created for join between parents. That number grows exponentially with the
+ * number of base relations being joined. The time and memory consumed to
+ * create paths for each child-join will be proporional to the number of
+ * partitions. This will not scale well with thousands of partitions. Instead
+ * of that we estimate partition-wise join cost based on the costs of sampled
+ * child-joins. We choose child-joins with higher sizes to have realistic
+ * estimates.
+ *
+ * This must be called after we have considered all joining orders since
+ * certain join orders may allow us to deem a child-join as dummy.
+ */
+void
+generate_partition_wise_join_paths(PlannerInfo *root, RelOptInfo *rel)
+{
+	List   *sampled_children = NIL;
+	List   *ordered_child_nos = NIL;
+	int		cnt_part;
+	int		num_part_to_plan;
+	int		num_parts;
+	bool	partition_join_path = false;
+	int		num_dummy_parts = 0;
+	ListCell   *lc;
+
+	/* Handle only join relations. */
+	if (!IS_JOIN_REL(rel))
+		return;
+
+	/*
+	 * If none of the join orders for this relation could use partition-wise
+	 * join technique, the join is not partitioned. Reset the partitioning
+	 * scheme.
+	 */
+	if (!rel->part_rels)
+		rel->part_scheme = NULL;
+
+	/* If the relation is not partitioned or is proven dummy, nothing to do. */
+	if (!rel->part_scheme || IS_DUMMY_REL(rel))
+		return;
+
+	/* Guard against stack overflow due to overly deep partition hierarchy. */
+	check_stack_depth();
+
+	num_parts = rel->part_scheme->nparts;
+
+	/* Calculate number of child-joins to sample. */
+	num_part_to_plan = num_parts * FRACTION_PARTS_TO_PLAN;
+	if (num_part_to_plan < 1)
+		num_part_to_plan = 1;
+
+	/* Order the child-join relations by their size. */
+	for (cnt_part = 0; cnt_part < num_parts; cnt_part++)
+	{
+		RelOptInfo *child_rel = rel->part_rels[cnt_part];
+		ListCell   *insert_after;
+
+		insert_after = NULL;
+
+		/* Dummy children will not be scanned, so ingore those. */
+		if (IS_DUMMY_REL(child_rel))
+		{
+			num_dummy_parts++;
+			continue;
+		}
+
+		/*
+		 * Add this relation to the list of samples ordered by the increasing
+		 * number of rows at appropriate place.
+		 */
+		foreach (lc, ordered_child_nos)
+		{
+			int	child_no = lfirst_int(lc);
+			RelOptInfo *other_childrel = rel->part_rels[child_no];
+
+			/*
+			 * Keep track of child with lowest number of rows but higher than the
+			 * that of the child being inserted. Insert the child before a
+			 * child with highest number of rows lesser than it.
+			 */
+			if (child_rel->rows <= other_childrel->rows)
+				insert_after = lc;
+			else
+				break;
+		}
+
+		if (insert_after)
+			lappend_cell_int(ordered_child_nos, insert_after, cnt_part);
+		else
+			ordered_child_nos = lcons_int(cnt_part, ordered_child_nos);
+	}
+
+	/*
+	 * Create paths for the child-joins as they appear in the list ordered by
+	 * their size. Stop when we have created paths for required number of
+	 * child-joins.
+	 */
+	foreach (lc, ordered_child_nos)
+	{
+		int	child_no = lfirst_int(lc);
+		RelOptInfo *child_rel = rel->part_rels[child_no];
+
+		/* Create paths for this child. */
+		add_paths_to_child_joinrel(root, rel, child_no);
+
+		/* Dummy children will not be scanned, so ingore those. */
+		if (IS_DUMMY_REL(child_rel))
+		{
+			num_dummy_parts++;
+			continue;
+		}
+
+#ifdef OPTIMIZER_DEBUG
+		debug_print_rel(root, rel);
+#endif
+
+		sampled_children = lappend(sampled_children, child_rel);
+
+		if (list_length(sampled_children) >= num_part_to_plan)
+			break;
+	}
+
+	/*
+	 * If the number of samples is same as the number of live children, an
+	 * append path will do. Otherwise, we will cost the partition-wise join
+	 * based on the sampled children using PartitionJoinPath.
+	 */
+	if (num_part_to_plan < num_parts - num_dummy_parts)
+		partition_join_path = true;
+
+	/* Add paths for partition-wise join based on the sampled children. */
+	add_paths_to_append_rel(root, rel, sampled_children, partition_join_path);
+
+	if (sampled_children)
+		list_free(sampled_children);
+}
+
 /*****************************************************************************
  *			DEBUG SUPPORT
  *****************************************************************************/
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 415edad..ba2b238 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -126,6 +126,8 @@ bool		enable_nestloop = true;
 bool		enable_material = true;
 bool		enable_mergejoin = true;
 bool		enable_hashjoin = true;
+bool		enable_partition_wise_join = true;
+double		partition_wise_plan_weight = DEFAULT_PARTITION_WISE_PLAN_WEIGHT;
 
 typedef struct
 {
diff --git a/src/backend/optimizer/path/equivclass.c b/src/backend/optimizer/path/equivclass.c
index 0e50ad5..b1ea2ba 100644
--- a/src/backend/optimizer/path/equivclass.c
+++ b/src/backend/optimizer/path/equivclass.c
@@ -972,6 +972,10 @@ generate_base_implied_equalities_broken(PlannerInfo *root,
  * appropriate clauses using child EC members.  add_child_rel_equivalences
  * must already have been done for the child rel.
  *
+ * For a join between child relations, joinrelids, outer_relids and
+ * inner_rel all point to child relations. In this case, we need to find the
+ * parent relids to search the applicable equivalence classes.
+ *
  * The results are sufficient for use in merge, hash, and plain nestloop join
  * methods.  We do not worry here about selecting clauses that are optimal
  * for use in a parameterized indexscan.  indxpath.c makes its own selections
@@ -1021,12 +1025,27 @@ generate_join_implied_equalities_for_ecs(PlannerInfo *root,
 	ListCell   *lc;
 
 	/* If inner rel is a child, extra setup work is needed */
-	if (inner_rel->reloptkind == RELOPT_OTHER_MEMBER_REL)
+	if (IS_OTHER_REL(inner_rel))
 	{
+		RelOptInfo *outer_rel;
+		Relids		nominal_outer_relids;
+
+		if (bms_num_members(outer_relids) > 1)
+			outer_rel = find_join_rel(root, outer_relids);
+		else
+			outer_rel = find_base_rel(root, bms_singleton_member(outer_relids));
+
 		/* Fetch relid set for the topmost parent rel */
-		nominal_inner_relids = find_childrel_top_parent(root, inner_rel)->relids;
-		/* ECs will be marked with the parent's relid, not the child's */
-		nominal_join_relids = bms_union(outer_relids, nominal_inner_relids);
+		nominal_inner_relids = find_childrel_top_parent(root, inner_rel);
+
+		/* ECs will be marked with the parent's relid, not the child's. */
+		if (outer_rel && IS_OTHER_REL(outer_rel))
+		{
+			nominal_outer_relids = find_childrel_top_parent(root, outer_rel);
+			nominal_join_relids = bms_union(nominal_outer_relids, nominal_inner_relids);
+		}
+		else
+			nominal_join_relids = bms_union(outer_relids, nominal_inner_relids);
 	}
 	else
 	{
@@ -2062,7 +2081,7 @@ add_child_rel_equivalences(PlannerInfo *root,
 				child_expr = (Expr *)
 					adjust_appendrel_attrs(root,
 										   (Node *) cur_em->em_expr,
-										   appinfo);
+										   list_make1(appinfo));
 
 				/*
 				 * Transform em_relids to match.  Note we do *not* do
@@ -2364,8 +2383,8 @@ eclass_useful_for_merging(PlannerInfo *root,
 	 */
 
 	/* If specified rel is a child, we must consider the topmost parent rel */
-	if (rel->reloptkind == RELOPT_OTHER_MEMBER_REL)
-		relids = find_childrel_top_parent(root, rel)->relids;
+	if (IS_OTHER_REL(rel))
+		relids = find_childrel_top_parent(root, rel);
 	else
 		relids = rel->relids;
 
diff --git a/src/backend/optimizer/path/joinpath.c b/src/backend/optimizer/path/joinpath.c
index 96f00fc..efa4af0 100644
--- a/src/backend/optimizer/path/joinpath.c
+++ b/src/backend/optimizer/path/joinpath.c
@@ -25,9 +25,19 @@
 /* Hook for plugins to get control in add_paths_to_joinrel() */
 set_join_pathlist_hook_type set_join_pathlist_hook = NULL;
 
-#define PATH_PARAM_BY_REL(path, rel)  \
+/*
+ * Paths parameterized by the parent can be considered to be parameterized by
+ * any of its child.
+ */
+#define PATH_PARAM_BY_PARENT(path, rel)	\
+	((path)->param_info && bms_overlap(PATH_REQ_OUTER(path),	\
+									   (rel)->top_parent_relids))
+#define PATH_PARAM_BY_REL_SELF(path, rel)  \
 	((path)->param_info && bms_overlap(PATH_REQ_OUTER(path), (rel)->relids))
 
+#define PATH_PARAM_BY_REL(path, rel)	\
+	(PATH_PARAM_BY_REL_SELF(path, rel) || PATH_PARAM_BY_PARENT(path, rel))
+
 static void sort_inner_and_outer(PlannerInfo *root, RelOptInfo *joinrel,
 					 RelOptInfo *outerrel, RelOptInfo *innerrel,
 					 JoinType jointype, JoinPathExtraData *extra);
@@ -132,6 +142,19 @@ add_paths_to_joinrel(PlannerInfo *root,
 	foreach(lc, root->join_info_list)
 	{
 		SpecialJoinInfo *sjinfo2 = (SpecialJoinInfo *) lfirst(lc);
+		Relids	joinrelids;
+
+		/*
+		 * PlannerInfo doesn't contain the SpecialJoinInfos created for joins
+		 * between child relations, even if there is a SpecialJoinInfo node for
+		 * the join between the topmost parents. Hence while calculating Relids
+		 * set representing the restriction, consider relids of topmost parent
+		 * of partitions.
+		 */
+		if (joinrel->reloptkind == RELOPT_OTHER_JOINREL)
+			joinrelids = joinrel->top_parent_relids;
+		else
+			joinrelids = joinrel->relids;
 
 		/*
 		 * SJ is relevant to this join if we have some part of its RHS
@@ -140,16 +163,16 @@ add_paths_to_joinrel(PlannerInfo *root,
 		 * join has already been proven legal.)  If the SJ is relevant, it
 		 * presents constraints for joining to anything not in its RHS.
 		 */
-		if (bms_overlap(joinrel->relids, sjinfo2->min_righthand) &&
-			!bms_overlap(joinrel->relids, sjinfo2->min_lefthand))
+		if (bms_overlap(joinrelids, sjinfo2->min_righthand) &&
+			!bms_overlap(joinrelids, sjinfo2->min_lefthand))
 			extra.param_source_rels = bms_join(extra.param_source_rels,
 										   bms_difference(root->all_baserels,
 													sjinfo2->min_righthand));
 
 		/* full joins constrain both sides symmetrically */
 		if (sjinfo2->jointype == JOIN_FULL &&
-			bms_overlap(joinrel->relids, sjinfo2->min_lefthand) &&
-			!bms_overlap(joinrel->relids, sjinfo2->min_righthand))
+			bms_overlap(joinrelids, sjinfo2->min_lefthand) &&
+			!bms_overlap(joinrelids, sjinfo2->min_righthand))
 			extra.param_source_rels = bms_join(extra.param_source_rels,
 										   bms_difference(root->all_baserels,
 													 sjinfo2->min_lefthand));
@@ -279,6 +302,22 @@ try_nestloop_path(PlannerInfo *root,
 	JoinCostWorkspace workspace;
 
 	/*
+	 * For a join between child relations, if the inner path is parameterized
+	 * by the parent of the outer relation, create a nestloop join path with
+	 * inner relation parameterized by the outer relation by translating the
+	 * inner path to be parameterized by the outer child relation.
+	 */
+	if (PATH_PARAM_BY_PARENT(inner_path, outer_path->parent))
+	{
+		inner_path = reparameterize_path_by_child(root, inner_path,
+												   outer_path->parent);
+
+		/* If we could not translate the path, don't produce nest loop path. */
+		if (!inner_path)
+			return;
+	}
+
+	/*
 	 * Check to see if proposed path is still parameterized, and reject if the
 	 * parameterization wouldn't be sensible --- unless allow_star_schema_join
 	 * says to allow it anyway.  Also, we must reject if have_dangerous_phv
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 01d4fea..47ff915 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -14,9 +14,14 @@
  */
 #include "postgres.h"
 
+#include "miscadmin.h"
+#include "catalog/partition.h"
+#include "optimizer/clauses.h"
 #include "optimizer/joininfo.h"
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
+#include "optimizer/prep.h"
+#include "optimizer/cost.h"
 #include "utils/memutils.h"
 
 
@@ -32,7 +37,19 @@ static bool is_dummy_rel(RelOptInfo *rel);
 static void mark_dummy_rel(RelOptInfo *rel);
 static bool restriction_is_constant_false(List *restrictlist,
 							  bool only_pushed_down);
-
+static void populate_joinrel_with_paths(PlannerInfo *root, RelOptInfo *rel1,
+							RelOptInfo *rel2, RelOptInfo *joinrel,
+							SpecialJoinInfo *sjinfo, List *restrictlist);
+static void try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1,
+						  RelOptInfo *rel2, RelOptInfo *joinrel,
+						  SpecialJoinInfo *parent_sjinfo,
+						  List *parent_restrictlist);
+static SpecialJoinInfo *build_child_join_sjinfo(PlannerInfo *root,
+									SpecialJoinInfo *parent_sjinfo,
+									Relids left_relids, Relids right_relids);
+static bool have_partkey_equi_join(RelOptInfo *rel1, RelOptInfo *rel2,
+						   JoinType jointype, List *restrictlist);
+static int match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel);
 
 /*
  * join_search_one_level
@@ -724,6 +741,31 @@ make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 		return joinrel;
 	}
 
+	/* Add paths to the join relation. */
+	populate_joinrel_with_paths(root, rel1, rel2, joinrel, sjinfo,
+								restrictlist);
+
+	/* Apply partition-wise join technique, if possible. */
+	try_partition_wise_join(root, rel1, rel2, joinrel, sjinfo, restrictlist);
+
+	bms_free(joinrelids);
+
+	return joinrel;
+}
+
+/*
+ * populate_joinrel_with_paths
+ *	  Add paths to the given joinrel for given pair of joining relations. The
+ *	  SpecialJoinInfo provides details about the join and the restrictlist
+ *	  contains the join clauses and the other clauses applicable for given pair
+ *	  of the joining relations.
+ */
+
+static void
+populate_joinrel_with_paths(PlannerInfo *root, RelOptInfo *rel1,
+							RelOptInfo *rel2, RelOptInfo *joinrel,
+							SpecialJoinInfo *sjinfo, List *restrictlist)
+{
 	/*
 	 * Consider paths using each rel as both outer and inner.  Depending on
 	 * the join type, a provably empty outer or inner rel might mean the join
@@ -868,13 +910,8 @@ make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 			elog(ERROR, "unrecognized join type: %d", (int) sjinfo->jointype);
 			break;
 	}
-
-	bms_free(joinrelids);
-
-	return joinrel;
 }
 
-
 /*
  * have_join_order_restriction
  *		Detect whether the two relations should be joined to satisfy
@@ -1249,3 +1286,477 @@ restriction_is_constant_false(List *restrictlist, bool only_pushed_down)
 	}
 	return false;
 }
+
+/* Free SpecialJoinInfo. */
+static void
+free_special_join_info(SpecialJoinInfo *sjinfo)
+{
+	bms_free(sjinfo->min_lefthand);
+	bms_free(sjinfo->syn_lefthand);
+	bms_free(sjinfo->syn_righthand);
+	pfree(sjinfo);
+}
+
+/*
+ * Assess whether join between given two partitioned relations can be broken
+ * down into joins between matching partitions; a technique called
+ * "partition-wise join"
+ *
+ * Partition-wise 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.
+ *
+ * Partition-wise join is planned as follows (details: optimizer/README.)
+ *
+ * 1. Create the RelOptInfos for joins between matching partitions i.e
+ * child-joins and estimate sizes of those. This function is responsible for
+ * this phase.
+ *
+ * 2. Add paths representing partition-wise join. The second phase is
+ * implemented by generate_partition_wise_join_paths(). In order to save time
+ * and memory consumed in creating paths for every child-join, we create paths
+ * for only few child-joins.
+ *
+ * 3. Create merge/append plan to combining plans for every child-join,
+ * creating paths for remaining child-joins.
+ *
+ * The RelOptInfo, SpecialJoinInfo and restrictlist for each child join are
+ * obtained by translating the respective parent join structures.
+ */
+static void
+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;
+	PartitionedJoin	   *partitioned_join;
+
+	/* 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)
+		return;
+
+	/*
+	 * If any of the joining parent relations is proven empty, either the join
+	 * will be empty (INNER join) or will have the inner side all nullified. We
+	 * take care of such cases when creating join paths for parent relations.
+	 * Nothing to be done here. Also, nothing to do, if the parent join is
+	 * proven empty.
+	 */
+	if (IS_DUMMY_REL(rel1) || IS_DUMMY_REL(rel2) || IS_DUMMY_REL(joinrel))
+		return;
+
+	/*
+	 * Partitioning scheme in join relation indicates a possibilty that the
+	 * join may be partitioned, but it's not necessary that every pair of
+	 * joining relations can use partition-wise join technique. If one of
+	 * joining relations turns out to be unpartitioned, this pair of joining
+	 * relations can not use partition-wise join technique.
+	 */
+	if (!rel1->part_scheme || !rel2->part_scheme)
+		return;
+
+	/*
+	 * If an equi-join condition between the partition keys of the joining
+	 * relations does not exist, this pair of joining relations can not use
+	 * partition-wise technique.
+	 */
+	if (!have_partkey_equi_join(rel1, rel2, parent_sjinfo->jointype,
+								parent_restrictlist))
+		return;
+
+	/*
+	 * 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);
+
+	/* We should have RelOptInfos of the partitions available. */
+	Assert(rel1->part_rels && rel2->part_rels);
+
+	part_scheme = joinrel->part_scheme;
+	nparts = part_scheme->nparts;
+
+	/*
+	 * We do not store information about valid pairs of joining child
+	 * relations. The pair of joining relations for a child-join can be derived
+	 * from valid pairs of joining parent relations. Amongst the valid pairs of
+	 * parent joining relations, only those which result in partitioned join
+	 * matter for partition-wise join. Remember those so that we can use them
+	 * for creating paths for few child-joins in
+	 * generate_partition_wise_join_paths() later.
+	 */
+	partitioned_join = (PartitionedJoin *) palloc(sizeof(PartitionedJoin));
+	partitioned_join->rel1 = rel1;
+	partitioned_join->rel2 = rel2;
+	partitioned_join->sjinfo = copyObject(parent_sjinfo);
+	joinrel->partitioned_joins = lappend(joinrel->partitioned_joins,
+										 partitioned_join);
+
+	elog(DEBUG3, "join between relations %s and %s is considered for partition-wise join.",
+		 bmsToString(rel1->relids), bmsToString(rel2->relids));
+
+	/* We are done if child RelOptInfos are already created. */
+	if (joinrel->part_rels)
+		return;
+
+	/* Create all the child RelOptInfos. */
+	joinrel->part_rels = (RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
+
+	/*
+	 * Create child join relations for this partitioned join. While doing so,
+	 * we estimate sizes of these child join relations. These estimates are
+	 * used to find the representative child relations used for costing the
+	 * partition-wise join later.
+	 */
+	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	{
+		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
+		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
+		SpecialJoinInfo	*child_sjinfo;
+		List	*child_restrictlist;
+		RelOptInfo *child_joinrel;
+
+		/* We should never try to join two overlapping sets of rels. */
+		Assert(!bms_overlap(child_rel1->relids, child_rel2->relids));
+
+		Assert (!joinrel->part_rels[cnt_parts]);
+
+		child_joinrel = build_child_join_rel(root, child_rel1, child_rel2,
+											 joinrel, parent_sjinfo->jointype);
+
+		joinrel->part_rels[cnt_parts] = child_joinrel;
+
+		/*
+		 * Construct restrictions applicable to the child join from
+		 * those applicable to the parent join.
+		 */
+		child_restrictlist = build_child_joinrel_restrictlist(root,
+															  child_joinrel,
+															  child_rel1,
+															  child_rel2);
+
+		/*
+		 * Construct SpecialJoinInfo from parent join relations's
+		 * SpecialJoinInfo.
+		 */
+		child_sjinfo = build_child_join_sjinfo(root, parent_sjinfo,
+											   child_rel1->relids,
+											   child_rel2->relids);
+
+		/*
+		 * Set estimates of the child-joinrel's size.
+		 */
+		set_joinrel_size_estimates(root, child_joinrel, child_rel1, child_rel2,
+								   child_sjinfo, child_restrictlist);
+
+		/*
+		 * If the child relations themselves are partitioned, try partition-wise join
+		 * recursively.
+		 */
+		try_partition_wise_join(root, child_rel1, child_rel2, child_joinrel,
+								child_sjinfo, child_restrictlist);
+
+		free_special_join_info(child_sjinfo);
+		child_sjinfo = NULL;
+	}
+}
+
+/*
+ * add_paths_to_child_join
+ * 		Add paths to 'child_id'th child of given parent join relation.
+ *
+ * The function creates paths for given child-join by joining corresponding
+ * children of every pair of joining parent relations which produces
+ * partitioned join. Since we create paths only for sampled child-joins, either
+ * of the children being joined may not have paths. In that case, this function
+ * is called recursively to populate paths for those.
+ */
+void
+add_paths_to_child_joinrel(PlannerInfo *root, RelOptInfo *parent_joinrel,
+						   int child_id)
+{
+	ListCell	*lc;
+	RelOptInfo	   *child_joinrel = parent_joinrel->part_rels[child_id];
+
+	Assert(IS_JOIN_REL(parent_joinrel));
+
+	/* If this child relation already has paths, nothing to do. */
+	if (child_joinrel->cheapest_total_path)
+		return;
+
+	/* A dummy relation will have a dummy path as the cheapest path. */
+	Assert(!is_dummy_rel(child_joinrel));
+
+	/*
+	 * For every partitioned join order, calculate paths for the joining
+	 * child relations and then calculate paths for given child.
+	 */
+	foreach (lc, parent_joinrel->partitioned_joins)
+	{
+		PartitionedJoin	   *pj = lfirst(lc);
+		RelOptInfo *rel1 = pj->rel1;
+		RelOptInfo *rel2 = pj->rel2;
+		RelOptInfo *child_rel1 = rel1->part_rels[child_id];
+		RelOptInfo *child_rel2 = rel2->part_rels[child_id];
+		SpecialJoinInfo	   *child_sjinfo;
+		List	   *child_restrictlist;
+
+		/*
+		 * Add paths to joining relation if it is a join itself.
+		 * Paths for child base relations are created in
+		 * set_append_rel_pathlist().
+		 */
+		if (IS_JOIN_REL(pj->rel1))
+			add_paths_to_child_joinrel(root, rel1, child_id);
+
+		if (IS_JOIN_REL(pj->rel2))
+			add_paths_to_child_joinrel(root, rel2, child_id);
+
+		/*
+		 * Construct SpecialJoinInfo from parent join relations's
+		 * SpecialJoinInfo.
+		 */
+		child_sjinfo = build_child_join_sjinfo(root, pj->sjinfo,
+											   child_rel1->relids,
+											   child_rel2->relids);
+
+
+		/*
+		 * Construct restrictions applicable to the child join from
+		 * those applicable to the parent join.
+		 */
+		child_restrictlist = build_child_joinrel_restrictlist(root,
+															  child_joinrel,
+															  child_rel1,
+															  child_rel2);
+
+		/* Add paths for child join. */
+		populate_joinrel_with_paths(root, rel1->part_rels[child_id],
+									rel2->part_rels[child_id], child_joinrel,
+									child_sjinfo, child_restrictlist);
+
+		/* Add partition-wise join paths for partitioned child-joins. */
+		generate_partition_wise_join_paths(root, child_joinrel);
+
+		free_special_join_info(child_sjinfo);
+		child_sjinfo = NULL;
+	}
+
+	set_cheapest(child_joinrel);
+}
+
+/*
+ * Construct the SpecialJoinInfo for a child-join by translating
+ * SpecialJoinInfo for the join between parents. left_relids and right_relids
+ * are the relids of left and right side of the join respectively.
+ */
+static SpecialJoinInfo *
+build_child_join_sjinfo(PlannerInfo *root, SpecialJoinInfo *parent_sjinfo,
+						Relids left_relids, Relids right_relids)
+{
+	SpecialJoinInfo *sjinfo = makeNode(SpecialJoinInfo);
+	MemoryContext	old_context;
+	List	   *left_appinfos = find_appinfos_by_relids(root, left_relids);
+	List	   *right_appinfos = find_appinfos_by_relids(root, right_relids);
+
+	memcpy(sjinfo, parent_sjinfo, sizeof(SpecialJoinInfo));
+
+	sjinfo->min_lefthand = adjust_child_relids(sjinfo->min_lefthand,
+											   left_appinfos);
+	sjinfo->min_righthand = adjust_child_relids(sjinfo->min_righthand,
+												right_appinfos);
+	sjinfo->syn_lefthand = adjust_child_relids(sjinfo->syn_lefthand,
+											   left_appinfos);
+	sjinfo->syn_righthand = adjust_child_relids(sjinfo->syn_righthand,
+												right_appinfos);
+
+	/*
+	 * Replace the Var nodes of parent with those of children in expressions.
+	 * This function may be called within a temporary context, but the
+	 * expressions will be shallow-copied into the plan. Hence copy those in
+	 * the planner's context.
+	 */
+	old_context = MemoryContextSwitchTo(root->planner_cxt);
+	sjinfo->semi_rhs_exprs = (List *) adjust_appendrel_attrs(root,
+											   (Node *) sjinfo->semi_rhs_exprs,
+															 right_appinfos);
+	MemoryContextSwitchTo(old_context);
+
+	list_free(left_appinfos);
+	list_free(right_appinfos);
+
+	return sjinfo;
+}
+
+/*
+ * Replace parent relids by child relids in the copy of given relid set.
+ */
+Relids
+adjust_child_relids(Relids relids, List *append_rel_infos)
+{
+	ListCell   *lc;
+
+	/* Ensure we have a modifiable copy. */
+	relids = bms_copy(relids);
+	foreach (lc, append_rel_infos)
+	{
+		AppendRelInfo	*appinfo = lfirst(lc);
+
+		/* Remove parent, add child */
+		if (bms_is_member(appinfo->parent_relid, relids))
+		{
+			relids = bms_del_member(relids, appinfo->parent_relid);
+			relids = bms_add_member(relids, appinfo->child_relid);
+		}
+	}
+
+	return relids;
+}
+
+/*
+ * Returns true if there exists an equi-join condition for each pair of
+ * partition key from given relations being joined.
+ */
+static bool
+have_partkey_equi_join(RelOptInfo *rel1, RelOptInfo *rel2,
+					 JoinType jointype, List *restrictlist)
+{
+	PartitionScheme	part_scheme = rel1->part_scheme;
+	ListCell	*lc;
+	int		cnt_pks;
+	int		num_pks;
+	bool   *pk_has_clause;
+
+	/*
+	 * This function should be called when the joining relations have same
+	 * partitioning scheme.
+	 */
+	Assert(rel1->part_scheme == rel2->part_scheme);
+	Assert(part_scheme);
+
+	num_pks = part_scheme->partnatts;
+
+	pk_has_clause = (bool *) palloc0(sizeof(bool) * num_pks);
+
+	foreach (lc, restrictlist)
+	{
+		RestrictInfo *rinfo = lfirst(lc);
+		OpExpr		 *opexpr;
+		Expr		 *expr1;
+		Expr		 *expr2;
+		int		ipk1;
+		int		ipk2;
+
+		/* If processing an outer join, only use its own join clauses. */
+		if (IS_OUTER_JOIN(jointype) && rinfo->is_pushed_down)
+			continue;
+
+		/* Skip clauses which can not be used for a join. */
+		if (!rinfo->can_join)
+			continue;
+
+		/* Skip clauses which are not equality conditions. */
+		if (rinfo->hashjoinoperator == InvalidOid && !rinfo->mergeopfamilies)
+			continue;
+
+		opexpr = (OpExpr *) rinfo->clause;
+		Assert(is_opclause(opexpr));
+
+
+		/* Match the operands to the relation. */
+		if (bms_is_subset(rinfo->left_relids, rel1->relids) &&
+			bms_is_subset(rinfo->right_relids, rel2->relids))
+		{
+			expr1 = linitial(opexpr->args);
+			expr2 = lsecond(opexpr->args);
+		}
+		else if (bms_is_subset(rinfo->left_relids, rel2->relids) &&
+				 bms_is_subset(rinfo->right_relids, rel1->relids))
+		{
+			expr1 = lsecond(opexpr->args);
+			expr2 = linitial(opexpr->args);
+		}
+		else
+			continue;
+
+		/* Associate matching clauses with partition keys. */
+		ipk1 = match_expr_to_partition_keys(expr1, rel1);
+		ipk2 = match_expr_to_partition_keys(expr2, rel2);
+
+		/*
+		 * If the clause refers to different partition keys from
+		 * both relations, it can not be used for partition-wise join.
+		 */
+		if (ipk1 != ipk2)
+			continue;
+
+		/*
+		 * The clause allows partition-wise join if only it uses the same
+		 * operator family as that specified by the partition key.
+		 */
+		if (!list_member_oid(rinfo->mergeopfamilies,
+							 part_scheme->partopfamily[ipk1]))
+			continue;
+
+		/* Mark the partition key as having an equi-join clause. */
+		pk_has_clause[ipk1] = true;
+	}
+
+	/* Check whether every partition key has an equi-join condition. */
+	for (cnt_pks = 0; cnt_pks < num_pks; cnt_pks++)
+	{
+		if (!pk_has_clause[cnt_pks])
+		{
+			pfree(pk_has_clause);
+			return false;
+		}
+	}
+
+	pfree(pk_has_clause);
+	return true;
+}
+
+/*
+ * Find the partition key from the given relation matching the given
+ * expression. If found, return the index of the partition key, else return -1.
+ */
+static int
+match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel)
+{
+	int		cnt_pks;
+	int		num_pks;
+
+	/* This function should be called only for partitioned relations. */
+	Assert(rel->part_scheme);
+
+	num_pks = rel->part_scheme->partnatts;
+
+	/*
+	 * Remove the relabel decoration. We can assume that there is at most one
+	 * RelabelType node; eval_const_expressions() simplifies multiple
+	 * RelabelType nodes into one.
+	 */
+	if (IsA(expr, RelabelType))
+		expr = (Expr *) ((RelabelType *) expr)->arg;
+
+	for (cnt_pks = 0; cnt_pks < num_pks; cnt_pks++)
+	{
+		List	 *pkexprs = rel->partexprs[cnt_pks];
+		ListCell *lc;
+
+		foreach(lc, pkexprs)
+		{
+			Expr *pkexpr = lfirst(lc);
+			if (equal(pkexpr, expr))
+				return cnt_pks;
+		}
+	}
+
+	return -1;
+}
diff --git a/src/backend/optimizer/path/pathkeys.c b/src/backend/optimizer/path/pathkeys.c
index 4436ac1..def64e3 100644
--- a/src/backend/optimizer/path/pathkeys.c
+++ b/src/backend/optimizer/path/pathkeys.c
@@ -1088,12 +1088,25 @@ select_outer_pathkeys_for_merge(PlannerInfo *root,
 	int			necs;
 	ListCell   *lc;
 	int			j;
+	Relids		relids;
 
 	/* Might have no mergeclauses */
 	if (nClauses == 0)
 		return NIL;
 
 	/*
+	 * For a child join relation, use parent relids to find potential join
+	 * partners (see code below) from equivalence classes. A potential join
+	 * partner of parent also indicates potential join partner of the child. By
+	 * using only parent relids, we avoid scoring an equivalence class multiple
+	 * times once for parent and then for all of its children.
+	 */
+	if (joinrel->reloptkind == RELOPT_OTHER_JOINREL)
+		relids = joinrel->top_parent_relids;
+	else
+		relids = joinrel->relids;
+
+	/*
 	 * Make arrays of the ECs used by the mergeclauses (dropping any
 	 * duplicates) and their "popularity" scores.
 	 */
@@ -1133,7 +1146,7 @@ select_outer_pathkeys_for_merge(PlannerInfo *root,
 
 			/* Potential future join partner? */
 			if (!em->em_is_const && !em->em_is_child &&
-				!bms_overlap(em->em_relids, joinrel->relids))
+				!bms_overlap(em->em_relids, relids))
 				score++;
 		}
 
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index ad49674..b754d90 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -30,6 +30,7 @@
 #include "optimizer/clauses.h"
 #include "optimizer/cost.h"
 #include "optimizer/paths.h"
+#include "optimizer/pathnode.h"
 #include "optimizer/placeholder.h"
 #include "optimizer/plancat.h"
 #include "optimizer/planmain.h"
@@ -42,6 +43,7 @@
 #include "parser/parse_clause.h"
 #include "parser/parsetree.h"
 #include "utils/lsyscache.h"
+#include "utils/memutils.h"
 
 
 /*
@@ -145,6 +147,9 @@ static CustomScan *create_customscan_plan(PlannerInfo *root,
 static NestLoop *create_nestloop_plan(PlannerInfo *root, NestPath *best_path);
 static MergeJoin *create_mergejoin_plan(PlannerInfo *root, MergePath *best_path);
 static HashJoin *create_hashjoin_plan(PlannerInfo *root, HashPath *best_path);
+static Plan *create_partition_plan(PlannerInfo *root, Path *best_path);
+static Plan *create_partition_join_plan(PlannerInfo *root,
+										PartitionJoinPath *best_path);
 static Node *replace_nestloop_params(PlannerInfo *root, Node *expr);
 static Node *replace_nestloop_params_mutator(Node *node, PlannerInfo *root);
 static void process_subquery_nestloop_params(PlannerInfo *root,
@@ -241,7 +246,8 @@ static Plan *prepare_sort_from_pathkeys(Plan *lefttree, List *pathkeys,
 static EquivalenceMember *find_ec_member_for_tle(EquivalenceClass *ec,
 					   TargetEntry *tle,
 					   Relids relids);
-static Sort *make_sort_from_pathkeys(Plan *lefttree, List *pathkeys);
+static Sort *make_sort_from_pathkeys(Plan *lefttree, List *pathkeys,
+									 Relids relids);
 static Sort *make_sort_from_groupcols(List *groupcls,
 						 AttrNumber *grpColIdx,
 						 Plan *lefttree);
@@ -367,12 +373,8 @@ create_plan_recurse(PlannerInfo *root, Path *best_path, int flags)
 									(JoinPath *) best_path);
 			break;
 		case T_Append:
-			plan = create_append_plan(root,
-									  (AppendPath *) best_path);
-			break;
 		case T_MergeAppend:
-			plan = create_merge_append_plan(root,
-											(MergeAppendPath *) best_path);
+			plan = create_partition_plan(root, best_path);
 			break;
 		case T_Result:
 			if (IsA(best_path, ProjectionPath))
@@ -1115,6 +1117,30 @@ create_merge_append_plan(PlannerInfo *root, MergeAppendPath *best_path)
 }
 
 /*
+ * create_partition_plan
+ *		Creates an Merge/Append plan as specified by the "best path".
+ *
+ *		Returns a Plan node.
+ */
+static Plan *
+create_partition_plan(PlannerInfo *root, Path *best_path)
+{
+	Plan   *plan;
+
+	if (IsA(best_path, PartitionJoinPath))
+		plan = create_partition_join_plan(root, (PartitionJoinPath *)best_path);
+	else if (best_path->pathtype == T_Append)
+		plan = create_append_plan(root, (AppendPath *) best_path);
+	else
+	{
+		Assert(best_path->pathtype == T_MergeAppend);
+		plan = create_merge_append_plan(root, (MergeAppendPath *) best_path);
+	}
+
+	return plan;
+}
+
+/*
  * create_result_plan
  *	  Create a Result plan for 'best_path'.
  *	  This is only used for degenerate cases, such as a query with an empty
@@ -1513,7 +1539,7 @@ create_sort_plan(PlannerInfo *root, SortPath *best_path, int flags)
 	subplan = create_plan_recurse(root, best_path->subpath,
 								  flags | CP_SMALL_TLIST);
 
-	plan = make_sort_from_pathkeys(subplan, best_path->path.pathkeys);
+	plan = make_sort_from_pathkeys(subplan, best_path->path.pathkeys, NULL);
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
@@ -3530,6 +3556,8 @@ create_mergejoin_plan(PlannerInfo *root,
 	ListCell   *lc;
 	ListCell   *lop;
 	ListCell   *lip;
+	Path	   *outer_path = best_path->jpath.outerjoinpath;
+	Path	   *inner_path = best_path->jpath.innerjoinpath;
 
 	/*
 	 * MergeJoin can project, so we don't have to demand exact tlists from the
@@ -3537,10 +3565,10 @@ create_mergejoin_plan(PlannerInfo *root,
 	 * best to request a small tlist so we aren't sorting more data than
 	 * necessary.
 	 */
-	outer_plan = create_plan_recurse(root, best_path->jpath.outerjoinpath,
+	outer_plan = create_plan_recurse(root, outer_path,
 					 (best_path->outersortkeys != NIL) ? CP_SMALL_TLIST : 0);
 
-	inner_plan = create_plan_recurse(root, best_path->jpath.innerjoinpath,
+	inner_plan = create_plan_recurse(root, inner_path,
 					 (best_path->innersortkeys != NIL) ? CP_SMALL_TLIST : 0);
 
 	/* Sort join qual clauses into best execution order */
@@ -3586,34 +3614,38 @@ create_mergejoin_plan(PlannerInfo *root,
 	 * outer_is_left status.
 	 */
 	mergeclauses = get_switched_clauses(best_path->path_mergeclauses,
-							 best_path->jpath.outerjoinpath->parent->relids);
+							 outer_path->parent->relids);
 
 	/*
 	 * Create explicit sort nodes for the outer and inner paths if necessary.
 	 */
 	if (best_path->outersortkeys)
 	{
+		Relids		outer_relids = outer_path->parent->relids;
 		Sort	   *sort = make_sort_from_pathkeys(outer_plan,
-												   best_path->outersortkeys);
+												   best_path->outersortkeys,
+												   outer_relids);
 
 		label_sort_with_costsize(root, sort, -1.0);
 		outer_plan = (Plan *) sort;
 		outerpathkeys = best_path->outersortkeys;
 	}
 	else
-		outerpathkeys = best_path->jpath.outerjoinpath->pathkeys;
+		outerpathkeys = outer_path->pathkeys;
 
 	if (best_path->innersortkeys)
 	{
+		Relids		inner_relids = inner_path->parent->relids;
 		Sort	   *sort = make_sort_from_pathkeys(inner_plan,
-												   best_path->innersortkeys);
+												   best_path->innersortkeys,
+												   inner_relids);
 
 		label_sort_with_costsize(root, sort, -1.0);
 		inner_plan = (Plan *) sort;
 		innerpathkeys = best_path->innersortkeys;
 	}
 	else
-		innerpathkeys = best_path->jpath.innerjoinpath->pathkeys;
+		innerpathkeys = inner_path->pathkeys;
 
 	/*
 	 * If specified, add a materialize node to shield the inner plan from the
@@ -3951,6 +3983,212 @@ create_hashjoin_plan(PlannerInfo *root,
 	return join_plan;
 }
 
+/*
+ * create_partition_join_plan
+ *		Creates Merge/Append plan consisting of join plans for child-join.
+ *
+ *		Returns a Plan node.
+ */
+static Plan *
+create_partition_join_plan(PlannerInfo *root, PartitionJoinPath *best_path)
+{
+	RelOptInfo *joinrel = best_path->path.parent;
+	int		nparts;
+	int		cnt_parts;
+	List	   *child_plans = NIL;
+	List	   *tlist = build_path_tlist(root, &best_path->path);
+	Plan	   *plan;
+	MemoryContext	child_context;
+	MemoryContext	old_context;
+	List	   *pathkeys = best_path->path.pathkeys;
+	StringInfoData	mem_context_name;
+
+	/* The relation should be a partitioned join relation. */
+	Assert(IS_JOIN_REL(joinrel) && joinrel->part_scheme &&
+		   joinrel->partitioned_joins);
+
+	nparts = joinrel->part_scheme->nparts;
+
+	/* Create MergeAppend plan when result is expected to be ordered. */
+	if (pathkeys)
+	{
+		MergeAppend *node = makeNode(MergeAppend);
+		plan = &node->plan;
+
+		plan->targetlist = tlist;
+
+		/* Compute sorting info, and adjust MergeAppend's tlist as needed. */
+		(void) prepare_sort_from_pathkeys(plan, pathkeys,
+										  best_path->path.parent->relids,
+										  NULL,
+										  true,
+										  &node->numCols,
+										  &node->sortColIdx,
+										  &node->sortOperators,
+										  &node->collations,
+										  &node->nullsFirst);
+	}
+	else
+	{
+		Append  *node = makeNode(Append);
+		plan = &node->plan;
+		plan->targetlist = tlist;
+	}
+
+	/* Fill costs, so that we can cost Sort node, if required. */
+	copy_generic_path_info(plan, (Path *) best_path);
+
+	/*
+	 * Create a new memory context for planning child joins. Since this routine
+	 * may be called recursively for tables with subpartitions, we use
+	 * a unique context name for every level of partition by using the lowest
+	 * relid amongst the base relations being joined.
+	 */
+	initStringInfo(&mem_context_name);
+	appendStringInfo(&mem_context_name, "%s_%d", "ChildJoinContext",
+					 bms_next_member(joinrel->relids, -1));
+	child_context = AllocSetContextCreate(CurrentMemoryContext,
+										  pstrdup(mem_context_name.data),
+										  ALLOCSET_DEFAULT_SIZES);
+	pfree(mem_context_name.data);
+	resetStringInfo(&mem_context_name);
+
+	/*
+	 * Create a paths for all child joins, one child join at a time. The paths
+	 * for every child join are independent i.e. one child does not require
+	 * paths created for the other. In order to avoid accumulating memory
+	 * consumed while creating paths for every child join, we use a fresh
+	 * memory context for every child join.
+	 */
+	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	{
+		RelOptInfo *child_join;
+		Path	   *child_path;
+		Plan	   *child_plan;
+		int			numsortkeys;
+		AttrNumber *sortColIdx;
+		Oid		   *sortOperators;
+		Oid		   *collations;
+		bool	   *nullsFirst;
+
+		/*
+		 * Create paths for the child join in a separate context, so that we
+		 * can reuse the memory used by those paths.
+		 */
+		old_context = MemoryContextSwitchTo(child_context);
+
+		add_paths_to_child_joinrel(root, joinrel, cnt_parts);
+
+		MemoryContextSwitchTo(old_context);
+
+		child_join = joinrel->part_rels[cnt_parts];
+
+
+		/* Skip empty child. */
+		if (IS_DUMMY_REL(child_join))
+			continue;
+
+#ifdef OPTIMIZER_DEBUG
+		debug_print_rel(root, rel);
+#endif
+
+		/*
+		 * Search for a child path with pathkeys or parameterization
+		 * matching that of the given path.
+		 */
+		child_path = get_cheapest_path_for_pathkeys(child_join->pathlist,
+													best_path->path.pathkeys,
+											 PATH_REQ_OUTER(&best_path->path),
+													TOTAL_COST);
+
+		if (!child_path)
+			elog(ERROR, "Could not find a path with required pathkeys.");
+
+		/* Create plan for the current child. */
+		child_plan = create_plan_recurse(root, child_path, CP_EXACT_TLIST);
+
+		if (pathkeys)
+		{
+			MergeAppend *node = (MergeAppend *) plan;
+
+			Assert(IsA(node, MergeAppend));
+
+			/* Compute sorting info, and adjust subplan's tlist as needed */
+			child_plan = prepare_sort_from_pathkeys(child_plan, pathkeys,
+												 child_path->parent->relids,
+												 node->sortColIdx,
+												 false,
+												 &numsortkeys,
+												 &sortColIdx,
+												 &sortOperators,
+												 &collations,
+												 &nullsFirst);
+
+			/*
+			 * Check that we got the same sort key information.  We just Assert
+			 * that the sortops match, since those depend only on the pathkeys;
+			 * but it seems like a good idea to check the sort column numbers
+			 * explicitly, to ensure the tlists really do match up.
+			 */
+			Assert(numsortkeys == node->numCols);
+			if (memcmp(sortColIdx, node->sortColIdx,
+					   numsortkeys * sizeof(AttrNumber)) != 0)
+				elog(ERROR, "MergeAppend child's targetlist doesn't match MergeAppend");
+			Assert(memcmp(sortOperators, node->sortOperators,
+						  numsortkeys * sizeof(Oid)) == 0);
+			Assert(memcmp(collations, node->collations,
+						  numsortkeys * sizeof(Oid)) == 0);
+			Assert(memcmp(nullsFirst, node->nullsFirst,
+						  numsortkeys * sizeof(bool)) == 0);
+
+			/* Now, insert a Sort node if subplan isn't sufficiently ordered */
+			if (!pathkeys_contained_in(pathkeys, child_path->pathkeys))
+			{
+				Sort	   *sort = make_sort(child_plan, numsortkeys,
+											 sortColIdx, sortOperators,
+											 collations, nullsFirst);
+				label_sort_with_costsize(root, sort, -1.0);
+				child_plan = (Plan *) sort;
+			}
+		}
+
+		child_plans = lappend(child_plans, child_plan);
+
+		/*
+		 * Reset the child_join memory context to reclaim the memory consumed
+		 * while creating paths.
+		 */
+		MemoryContextResetAndDeleteChildren(child_context);
+	}
+
+	/* Destroy the child context as we do not need it anymore. */
+	Assert(CurrentMemoryContext == old_context);
+	MemoryContextDelete(child_context);
+
+	/* Partitioned relation with all empty children gets a dummy path. */
+	Assert(child_plans != NIL);
+
+	if (IsA(plan, MergeAppend))
+	{
+		MergeAppend *node = (MergeAppend *)plan;
+
+		node->mergeplans = child_plans;
+	}
+	else
+	{
+		Append  *node = (Append *)plan;
+
+		Assert(IsA(plan, Append));
+		node->appendplans = child_plans;
+	}
+
+	/* Complete rest of the plan. */
+	plan->qual = NIL;
+	plan->lefttree = NULL;
+	plan->righttree = NULL;
+	return plan;
+}
+
 
 /*****************************************************************************
  *
@@ -4009,6 +4247,7 @@ replace_nestloop_params_mutator(Node *node, PlannerInfo *root)
 		nlp->paramno = param->paramid;
 		nlp->paramval = var;
 		root->curOuterParams = lappend(root->curOuterParams, nlp);
+
 		/* And return the replacement Param */
 		return (Node *) param;
 	}
@@ -4072,6 +4311,7 @@ replace_nestloop_params_mutator(Node *node, PlannerInfo *root)
 		nlp->paramno = param->paramid;
 		nlp->paramval = (Var *) phv;
 		root->curOuterParams = lappend(root->curOuterParams, nlp);
+
 		/* And return the replacement Param */
 		return (Node *) param;
 	}
@@ -5343,11 +5583,11 @@ prepare_sort_from_pathkeys(Plan *lefttree, List *pathkeys,
 					continue;
 
 				/*
-				 * Ignore child members unless they match the rel being
+				 * Ignore child members unless they belong to the rel being
 				 * sorted.
 				 */
 				if (em->em_is_child &&
-					!bms_equal(em->em_relids, relids))
+					!bms_is_subset(em->em_relids, relids))
 					continue;
 
 				sortexpr = em->em_expr;
@@ -5458,10 +5698,10 @@ find_ec_member_for_tle(EquivalenceClass *ec,
 			continue;
 
 		/*
-		 * Ignore child members unless they match the rel being sorted.
+		 * Ignore child members unless they belong to the rel being sorted.
 		 */
 		if (em->em_is_child &&
-			!bms_equal(em->em_relids, relids))
+			!bms_is_subset(em->em_relids, relids))
 			continue;
 
 		/* Match if same expression (after stripping relabel) */
@@ -5482,9 +5722,10 @@ find_ec_member_for_tle(EquivalenceClass *ec,
  *
  *	  'lefttree' is the node which yields input tuples
  *	  'pathkeys' is the list of pathkeys by which the result is to be sorted
+ *	  'relids' is the set of relations required by prepare_sort_from_pathkeys()
  */
 static Sort *
-make_sort_from_pathkeys(Plan *lefttree, List *pathkeys)
+make_sort_from_pathkeys(Plan *lefttree, List *pathkeys, Relids relids)
 {
 	int			numsortkeys;
 	AttrNumber *sortColIdx;
@@ -5494,7 +5735,7 @@ make_sort_from_pathkeys(Plan *lefttree, List *pathkeys)
 
 	/* Compute sort column info, and adjust lefttree as needed */
 	lefttree = prepare_sort_from_pathkeys(lefttree, pathkeys,
-										  NULL,
+										  relids,
 										  NULL,
 										  false,
 										  &numsortkeys,
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 41dde50..fedbb43 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -1105,7 +1105,7 @@ inheritance_planner(PlannerInfo *root)
 		subroot->parse = (Query *)
 			adjust_appendrel_attrs(root,
 								   (Node *) parse,
-								   appinfo);
+								   list_make1(appinfo));
 
 		/*
 		 * The rowMarks list might contain references to subquery RTEs, so
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index b714783..899e46f 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -55,7 +55,7 @@
 typedef struct
 {
 	PlannerInfo *root;
-	AppendRelInfo *appinfo;
+	List	   *appinfos;
 	int			sublevels_up;
 } adjust_appendrel_attrs_context;
 
@@ -108,7 +108,6 @@ static Bitmapset *translate_col_privs(const Bitmapset *parent_privs,
 					List *translated_vars);
 static Node *adjust_appendrel_attrs_mutator(Node *node,
 							   adjust_appendrel_attrs_context *context);
-static Relids adjust_relid_set(Relids relids, Index oldrelid, Index newrelid);
 static List *adjust_inherited_tlist(List *tlist,
 					   AppendRelInfo *context);
 
@@ -1712,10 +1711,10 @@ translate_col_privs(const Bitmapset *parent_privs,
 
 /*
  * adjust_appendrel_attrs
- *	  Copy the specified query or expression and translate Vars referring
- *	  to the parent rel of the specified AppendRelInfo to refer to the
- *	  child rel instead.  We also update rtindexes appearing outside Vars,
- *	  such as resultRelation and jointree relids.
+ *	  Copy the specified query or expression and translate Vars referring to
+ *	  the parent rels specified in the given list of AppendRelInfos to refer to
+ *	  the corresponding child rels instead.  We also update rtindexes appearing
+ *	  outside Vars, such as resultRelation and jointree relids.
  *
  * Note: this is applied after conversion of sublinks to subplans in the
  * query jointree, but there may still be sublinks in the security barrier
@@ -1725,15 +1724,18 @@ translate_col_privs(const Bitmapset *parent_privs,
  * maybe we should try to fold the two routines together.
  */
 Node *
-adjust_appendrel_attrs(PlannerInfo *root, Node *node, AppendRelInfo *appinfo)
+adjust_appendrel_attrs(PlannerInfo *root, Node *node, List *appinfos)
 {
 	Node	   *result;
 	adjust_appendrel_attrs_context context;
+	ListCell   *lc;
 
 	context.root = root;
-	context.appinfo = appinfo;
+	context.appinfos = appinfos;
 	context.sublevels_up = 0;
 
+	Assert(appinfos && list_length(appinfos) >= 1);
+
 	/*
 	 * Must be prepared to start with a Query or a bare expression tree; if
 	 * it's a Query, go straight to query_tree_walker to make sure that
@@ -1742,11 +1744,20 @@ adjust_appendrel_attrs(PlannerInfo *root, Node *node, AppendRelInfo *appinfo)
 	if (node && IsA(node, Query))
 	{
 		Query	   *newnode;
+		AppendRelInfo *appinfo;
 
 		newnode = query_tree_mutator((Query *) node,
 									 adjust_appendrel_attrs_mutator,
 									 (void *) &context,
 									 QTW_IGNORE_RC_SUBQUERIES);
+		foreach (lc, appinfos)
+		{
+			appinfo = lfirst(lc);
+
+			if (newnode->resultRelation == appinfo->parent_relid)
+				break;
+		}
+
 		if (newnode->resultRelation == appinfo->parent_relid)
 		{
 			newnode->resultRelation = appinfo->child_relid;
@@ -1764,17 +1775,51 @@ adjust_appendrel_attrs(PlannerInfo *root, Node *node, AppendRelInfo *appinfo)
 	return result;
 }
 
+/*
+ * find_appinfos_by_relids
+ * 		Find AppendRelInfo structures for all relations specified by relids.
+ */
+List *
+find_appinfos_by_relids(PlannerInfo *root, Relids relids)
+{
+	ListCell	*lc;
+	List		*appinfo_list = NIL;
+
+	foreach (lc, root->append_rel_list)
+	{
+		AppendRelInfo *appinfo = lfirst(lc);
+
+		if (bms_is_member(appinfo->child_relid, relids))
+			appinfo_list = lappend(appinfo_list, appinfo);
+	}
+
+	Assert(list_length(appinfo_list) == bms_num_members(relids));
+	return appinfo_list;
+}
+
 static Node *
 adjust_appendrel_attrs_mutator(Node *node,
 							   adjust_appendrel_attrs_context *context)
 {
-	AppendRelInfo *appinfo = context->appinfo;
+	List   *appinfos = context->appinfos;
+	ListCell   *lc;
+
+	Assert(appinfos && list_length(appinfos) >= 1);
 
 	if (node == NULL)
 		return NULL;
 	if (IsA(node, Var))
 	{
 		Var		   *var = (Var *) copyObject(node);
+		AppendRelInfo *appinfo;
+
+		foreach (lc, appinfos)
+		{
+			appinfo = lfirst(lc);
+
+			if (var->varno == appinfo->parent_relid)
+				break;
+		}
 
 		if (var->varlevelsup == context->sublevels_up &&
 			var->varno == appinfo->parent_relid)
@@ -1865,32 +1910,58 @@ adjust_appendrel_attrs_mutator(Node *node,
 	{
 		CurrentOfExpr *cexpr = (CurrentOfExpr *) copyObject(node);
 
-		if (context->sublevels_up == 0 &&
-			cexpr->cvarno == appinfo->parent_relid)
-			cexpr->cvarno = appinfo->child_relid;
+		foreach (lc, appinfos)
+		{
+			AppendRelInfo *appinfo = lfirst(lc);
+
+			if (context->sublevels_up == 0 &&
+				cexpr->cvarno == appinfo->parent_relid)
+			{
+				cexpr->cvarno = appinfo->child_relid;
+				break;
+			}
+		}
 		return (Node *) cexpr;
 	}
 	if (IsA(node, RangeTblRef))
 	{
 		RangeTblRef *rtr = (RangeTblRef *) copyObject(node);
 
-		if (context->sublevels_up == 0 &&
-			rtr->rtindex == appinfo->parent_relid)
-			rtr->rtindex = appinfo->child_relid;
+		foreach (lc, appinfos)
+		{
+			AppendRelInfo *appinfo = lfirst(lc);
+
+			if (context->sublevels_up == 0 &&
+				rtr->rtindex == appinfo->parent_relid)
+			{
+				rtr->rtindex = appinfo->child_relid;
+				break;
+			}
+		}
 		return (Node *) rtr;
 	}
 	if (IsA(node, JoinExpr))
 	{
 		/* Copy the JoinExpr node with correct mutation of subnodes */
 		JoinExpr   *j;
+		AppendRelInfo *appinfo;
 
 		j = (JoinExpr *) expression_tree_mutator(node,
 											  adjust_appendrel_attrs_mutator,
 												 (void *) context);
+
 		/* now fix JoinExpr's rtindex (probably never happens) */
-		if (context->sublevels_up == 0 &&
-			j->rtindex == appinfo->parent_relid)
-			j->rtindex = appinfo->child_relid;
+		foreach (lc, appinfos)
+		{
+			appinfo = lfirst(lc);
+
+			if (context->sublevels_up == 0 &&
+				j->rtindex == appinfo->parent_relid)
+			{
+				j->rtindex = appinfo->child_relid;
+				break;
+			}
+		}
 		return (Node *) j;
 	}
 	if (IsA(node, PlaceHolderVar))
@@ -1903,9 +1974,8 @@ adjust_appendrel_attrs_mutator(Node *node,
 														 (void *) context);
 		/* now fix PlaceHolderVar's relid sets */
 		if (phv->phlevelsup == context->sublevels_up)
-			phv->phrels = adjust_relid_set(phv->phrels,
-										   appinfo->parent_relid,
-										   appinfo->child_relid);
+			phv->phrels = adjust_child_relids(phv->phrels, context->appinfos);
+
 		return (Node *) phv;
 	}
 	/* Shouldn't need to handle planner auxiliary nodes here */
@@ -1936,24 +2006,18 @@ adjust_appendrel_attrs_mutator(Node *node,
 			adjust_appendrel_attrs_mutator((Node *) oldinfo->orclause, context);
 
 		/* adjust relid sets too */
-		newinfo->clause_relids = adjust_relid_set(oldinfo->clause_relids,
-												  appinfo->parent_relid,
-												  appinfo->child_relid);
-		newinfo->required_relids = adjust_relid_set(oldinfo->required_relids,
-													appinfo->parent_relid,
-													appinfo->child_relid);
-		newinfo->outer_relids = adjust_relid_set(oldinfo->outer_relids,
-												 appinfo->parent_relid,
-												 appinfo->child_relid);
-		newinfo->nullable_relids = adjust_relid_set(oldinfo->nullable_relids,
-													appinfo->parent_relid,
-													appinfo->child_relid);
-		newinfo->left_relids = adjust_relid_set(oldinfo->left_relids,
-												appinfo->parent_relid,
-												appinfo->child_relid);
-		newinfo->right_relids = adjust_relid_set(oldinfo->right_relids,
-												 appinfo->parent_relid,
-												 appinfo->child_relid);
+		newinfo->clause_relids = adjust_child_relids(oldinfo->clause_relids,
+													 context->appinfos);
+		newinfo->required_relids = adjust_child_relids(oldinfo->required_relids,
+													 context->appinfos);
+		newinfo->outer_relids = adjust_child_relids(oldinfo->outer_relids,
+													 context->appinfos);
+		newinfo->nullable_relids = adjust_child_relids(oldinfo->nullable_relids,
+													   context->appinfos);
+		newinfo->left_relids = adjust_child_relids(oldinfo->left_relids,
+												   context->appinfos);
+		newinfo->right_relids = adjust_child_relids(oldinfo->right_relids,
+													context->appinfos);
 
 		/*
 		 * Reset cached derivative fields, since these might need to have
@@ -2002,23 +2066,6 @@ adjust_appendrel_attrs_mutator(Node *node,
 }
 
 /*
- * Substitute newrelid for oldrelid in a Relid set
- */
-static Relids
-adjust_relid_set(Relids relids, Index oldrelid, Index newrelid)
-{
-	if (bms_is_member(oldrelid, relids))
-	{
-		/* Ensure we have a modifiable copy */
-		relids = bms_copy(relids);
-		/* Remove old, add new */
-		relids = bms_del_member(relids, oldrelid);
-		relids = bms_add_member(relids, newrelid);
-	}
-	return relids;
-}
-
-/*
  * Adjust the targetlist entries of an inherited UPDATE operation
  *
  * The expressions have already been fixed, but we have to make sure that
@@ -2135,5 +2182,91 @@ adjust_appendrel_attrs_multilevel(PlannerInfo *root, Node *node,
 	else
 		Assert(parent_rel->reloptkind == RELOPT_BASEREL);
 	/* Now translate for this child */
-	return adjust_appendrel_attrs(root, node, appinfo);
+	return adjust_appendrel_attrs(root, node, list_make1(appinfo));
+}
+
+/*
+ * build_child_restrictinfo
+ *		Returns a RestrictInfo which is derived from the given RestrictInfo by
+ *		applying the parent-child translation specified by the list of
+ *		AppendRelInfos.
+ *
+ * The topmost parent's RestrictInfo maintains a list of child RestrictInfos
+ * derived from it. If a suitable RestrictInfo is found in that list, it is
+ * returned as is. If there is no such child RestrictInfo, we translate the given
+ * RestrictInfo using the given list of AppendRelInfos and stick it in the
+ * topmost parent's list before returning it to the caller.
+ */
+RestrictInfo *
+build_child_restrictinfo(PlannerInfo *root, RestrictInfo *rinfo,
+						 List *append_rel_infos)
+{
+	Relids child_required_relids;
+	ListCell   *lc;
+	RestrictInfo   *parent_rinfo;
+	RestrictInfo   *child_rinfo;
+	MemoryContext	old_context;
+
+	child_required_relids = adjust_child_relids(rinfo->required_relids,
+													append_rel_infos);
+	/*
+	 * Check if we already have the RestrictInfo for the given child in the
+	 * topmost parent's RestrictInfo.
+	 */
+	parent_rinfo = rinfo->parent_rinfo ? rinfo->parent_rinfo : rinfo;
+	foreach (lc, parent_rinfo->child_rinfos)
+	{
+		child_rinfo = lfirst(lc);
+
+		if (bms_equal(child_rinfo->required_relids, child_required_relids))
+		{
+			bms_free(child_required_relids);
+			return child_rinfo;
+		}
+	}
+
+	/*
+	 * We didn't find any child restrictinfo for the given child, translate the
+	 * given RestrictInfo and stick it into the parent's list. The clause
+	 * expression may get used in plan, so create the child RestrictInfo in the
+	 * planner's context.
+	 */
+	old_context = MemoryContextSwitchTo(root->planner_cxt);
+	child_rinfo = (RestrictInfo *) adjust_appendrel_attrs(root, (Node *) rinfo,
+														  append_rel_infos);
+	bms_free(child_required_relids);
+	parent_rinfo->child_rinfos = lappend(parent_rinfo->child_rinfos,
+										 child_rinfo);
+	child_rinfo->parent_rinfo = parent_rinfo;
+
+	MemoryContextSwitchTo(old_context);
+
+	return child_rinfo;
+}
+
+/*
+ * build_child_clauses
+ *		Convenience routine to call build_child_restrictinfo on a list of
+ *		clauses.
+ */
+List *
+build_child_clauses(PlannerInfo *root, List *clauses, List *append_rel_infos)
+{
+	List   *child_clauses = NIL;
+	ListCell   *lc;
+
+	foreach (lc, clauses)
+	{
+		RestrictInfo   *parent_rinfo = lfirst(lc);
+		RestrictInfo   *child_rinfo;
+
+		Assert(IsA(parent_rinfo, RestrictInfo));
+
+		child_rinfo = build_child_restrictinfo(root, parent_rinfo,
+											   append_rel_infos);
+
+		child_clauses = lappend(child_clauses, child_rinfo);
+	}
+
+	return child_clauses;
 }
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 6d3ccfd..37e5f38 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -23,7 +23,9 @@
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
 #include "optimizer/planmain.h"
+#include "optimizer/prep.h"
 #include "optimizer/restrictinfo.h"
+#include "optimizer/tlist.h"
 #include "optimizer/var.h"
 #include "parser/parsetree.h"
 #include "utils/lsyscache.h"
@@ -2154,6 +2156,176 @@ create_hashjoin_path(PlannerInfo *root,
 }
 
 /*
+ * create_partition_join_path
+ * 		Creates a pathnode that represents partition-wise join for given
+ * 		partitioned join relation.
+ *
+ * This function is called when we haven't created paths for all the child
+ * joins. It estimates the number of rows and cost of the PartitionJoinPath
+ * based upon the number of rows and the cost of representative child-joins
+ * paths.
+ */
+PartitionJoinPath *
+create_partition_join_path(RelOptInfo *rel, List *subpaths,
+						   Bitmapset *required_outer)
+{
+	PartitionJoinPath *pathnode = makeNode(PartitionJoinPath);
+	double	subpath_rows = 0;
+	double	subpath_startup_cost = 0;
+	double	subpath_total_cost = 0;
+	double	child_rel_rows = 0;
+	ListCell   *lc;
+
+	Assert(rel->part_scheme);
+
+	pathnode->path.pathtype = T_Append;
+	pathnode->path.parent = rel;
+	pathnode->path.pathtarget = rel->reltarget;
+	pathnode->path.param_info = get_appendrel_parampathinfo(rel,
+															required_outer);
+	pathnode->path.pathkeys = NULL;
+
+	/* No parallel paths here. See more details in add_paths_to_append_rel() */
+	pathnode->path.parallel_aware = false;
+	pathnode->path.parallel_safe = false;
+	pathnode->path.parallel_workers = 0;
+
+	/* Accumulate the number of rows and costs from the given subpaths. */
+	foreach (lc, subpaths)
+	{
+		Path   *subpath = lfirst(lc);
+
+		subpath_rows += subpath->rows;
+		child_rel_rows += subpath->parent->rows;
+		subpath_total_cost += subpath->total_cost;
+
+		/*
+		 * Startup cost of an append relation is the startup cost of the first
+		 * subpath. Assume that the given first child will be the first child
+		 * in the final plan as well.
+		 */
+		if (lc == list_head(subpaths))
+			subpath_startup_cost = subpath->startup_cost;
+	}
+
+	/*
+	 * For a parameterized path, extrapolate the number of rows for the append
+	 * relation by considering the average selectivity of the parameterization
+	 * across the given children.
+	 */
+	if (bms_is_empty(required_outer))
+		pathnode->path.rows = rel->rows;
+	else
+		pathnode->path.rows = rel->rows * (subpath_rows / child_rel_rows);
+
+	pathnode->path.startup_cost = subpath_startup_cost;
+
+	/* Extrapolate the total cost to account for yet-to-be planned children. */
+	pathnode->path.total_cost = (subpath_total_cost * pathnode->path.rows) / subpath_rows;
+
+	/*
+	 * Multiply the costs with scaling factor as specified. Used to encourage
+	 * or discourage use of partition-wise join plans.
+	 */
+	pathnode->path.startup_cost *= partition_wise_plan_weight;
+	pathnode->path.total_cost *= partition_wise_plan_weight;
+
+	return pathnode;
+}
+
+/*
+ * create_partition_join_path_with_pathkeys
+ * 		Creates a pathnode that represents an ordered partition-wise join for
+ * 		given partitioned join relation.
+ *
+ * This function is called when we haven't created paths for all the child
+ * joins. It estimates the number of rows and cost of the PartitionJoinPath
+ * based upon the number of rows and the cost of representative child-joins
+ * paths.
+ */
+PartitionJoinPath *
+create_partition_join_path_with_pathkeys(PlannerInfo *root, RelOptInfo *rel,
+						   List *subpaths, List *pathkeys,
+						   Bitmapset *required_outer)
+{
+	PartitionJoinPath *pathnode = makeNode(PartitionJoinPath);
+	double	subpath_rows = 0;
+	double	subpath_startup_cost = 0;
+	double	subpath_total_cost = 0;
+	double	child_rel_rows = 0;
+	ListCell   *lc;
+
+	Assert(rel->part_scheme);
+
+	pathnode->path.pathtype = T_MergeAppend;
+	pathnode->path.parent = rel;
+	pathnode->path.pathtarget = rel->reltarget;
+	pathnode->path.param_info = get_appendrel_parampathinfo(rel,
+															required_outer);
+	pathnode->path.pathkeys = pathkeys;
+
+	/* No parallel paths here. See more details in add_paths_to_append_rel() */
+	pathnode->path.parallel_aware = false;
+	pathnode->path.parallel_safe = false;
+	pathnode->path.parallel_workers = 0;
+
+	/* Accumulate the number of rows and costs from the given subpaths. */
+	foreach (lc, subpaths)
+	{
+		Path   *subpath = lfirst(lc);
+
+		if (pathkeys_contained_in(pathkeys, subpath->pathkeys))
+		{
+			/* Subpath is adequately ordered, we won't need to sort it */
+			subpath_startup_cost += subpath->startup_cost;
+			subpath_total_cost += subpath->total_cost;
+		}
+		else
+		{
+			/* We'll need to insert a Sort node, so include cost for that */
+			Path		sort_path;		/* dummy for result of cost_sort */
+
+			cost_sort(&sort_path,
+					  root,
+					  pathkeys,
+					  subpath->total_cost,
+					  subpath->parent->tuples,
+					  subpath->pathtarget->width,
+					  0.0,
+					  work_mem,
+					  -1);
+			subpath_startup_cost += sort_path.startup_cost;
+			subpath_total_cost += sort_path.total_cost;
+		}
+
+		subpath_rows += subpath->rows;
+		child_rel_rows += subpath->parent->rows;
+	}
+
+	/*
+	 * For a parameterized path, extrapolate the number of rows for the append
+	 * relation by considering the average selectivity of the parameterization
+	 * across the given children.
+	 */
+	if (bms_is_empty(required_outer))
+		pathnode->path.rows = rel->rows;
+	else
+		pathnode->path.rows = rel->rows * (subpath_rows / child_rel_rows);
+
+	/* Extrapolate the total cost to account for yet-to-be planned children. */
+	pathnode->path.startup_cost = (subpath_startup_cost * pathnode->path.rows) / subpath_rows;
+	pathnode->path.total_cost = (subpath_total_cost * pathnode->path.rows) / subpath_rows;
+
+	/*
+	 * Multiply the costs with scaling factor as specified. Used to encourage
+	 * or discourage use of partition-wise join plans.
+	 */
+	pathnode->path.startup_cost *= partition_wise_plan_weight;
+	pathnode->path.total_cost *= partition_wise_plan_weight;
+
+	return pathnode;
+}
+/*
  * create_projection_path
  *	  Creates a pathnode that represents performing a projection.
  *
@@ -3209,3 +3381,182 @@ reparameterize_path(PlannerInfo *root, Path *path,
 	}
 	return NULL;
 }
+
+/*
+ * reparameterize_path_by_child
+ * 		Given a path parameterized by the parent of the given relation,
+ * 		translate the path to be parameterized by the given child relation.
+ *
+ * The function creates a new path of the same type as the given path, but
+ * parameterized by the given child relation. If it can not reparameterize the
+ * path as required, it returns NULL.
+ *
+ * The cost, number of rows, width and parallel path properties depend upon
+ * path->parent, which does not change during the translation. Hence those
+ * members are copied as they are.
+ */
+
+Path *
+reparameterize_path_by_child(PlannerInfo *root, Path *path,
+							  RelOptInfo *child_rel)
+{
+	Path	   *new_path;
+	ParamPathInfo   *new_ppi;
+	ParamPathInfo   *old_ppi;
+	List	   *child_aris;
+	Relids		required_outer;
+
+	/*
+	 * If the path is not parameterized by parent of the given relation or it it
+	 * doesn't need reparameterization.
+	 */
+	if (!path->param_info ||
+		!bms_overlap(PATH_REQ_OUTER(path), child_rel->top_parent_relids))
+	return path;
+
+	switch (nodeTag(path))
+	{
+		case T_Path:
+			new_path = makeNode(Path);
+			memcpy(new_path, path, sizeof(Path));
+			break;
+
+		case T_HashPath:
+			new_path = (Path *) makeNode(HashPath);
+			memcpy(new_path, path, sizeof(HashPath));
+			break;
+
+		case T_MergePath:
+			new_path = (Path *) makeNode(MergePath);
+			memcpy(new_path, path, sizeof(MergePath));
+			break;
+
+		case T_NestPath:
+			new_path = (Path *) makeNode(NestPath);
+			memcpy(new_path, path, sizeof(NestPath));
+			break;
+
+		case T_IndexPath:
+			new_path = (Path *) makeNode(IndexPath);
+			memcpy(new_path, path, sizeof(IndexPath));
+			break;
+
+		case T_AppendPath:
+			new_path = (Path *) makeNode(AppendPath);
+			memcpy(new_path, path, sizeof(AppendPath));
+			break;
+
+		/*
+		 * TODO:
+		 * If this method of translation is fine add more path types here.
+		 */
+
+		default:
+			/* Path type unsupported by this function. */
+			return NULL;
+	}
+
+	/*
+	 * Gather AppendRelInfos of the base partition relations in the outer child
+	 * relation. We need those for translating parent path to that of child by
+	 * substituting parent Var nodes and relids with those of children.
+	 */
+	child_aris = find_appinfos_by_relids(root, child_rel->relids);
+
+	/* Adjust the parameterization information. */
+	old_ppi = new_path->param_info;
+	required_outer = adjust_child_relids(old_ppi->ppi_req_outer, child_aris);
+
+	/* If we already have a PPI for this parameterization, just return it */
+	new_ppi = find_param_path_info(new_path->parent, required_outer);
+
+	/* If not build a new one and link it to the list of PPIs. */
+	if (!new_ppi)
+	{
+		new_ppi = makeNode(ParamPathInfo);
+		new_ppi->ppi_req_outer = required_outer;
+		new_ppi->ppi_rows = old_ppi->ppi_rows;
+		new_ppi->ppi_clauses = build_child_clauses(root, old_ppi->ppi_clauses,
+												   child_aris);
+		new_path->param_info = new_ppi;
+		new_path->parent->ppilist = lappend(new_path->parent->ppilist, new_ppi);
+	}
+	else
+		bms_free(required_outer);
+
+	/*
+	 * Adjust the path target if the parent of the outer relation is referenced
+	 * in the targetlist. This can happen when only the parent of outer relation is
+	 * laterally referenced in this relation.
+	 */
+	if (bms_overlap(path->parent->lateral_relids, child_rel->top_parent_relids))
+	{
+		MemoryContext	old_context;
+
+		/*
+		 * Allocate the target in planner's context, since they are copies as
+		 * is from path while creating plans.
+		 */
+		old_context = MemoryContextSwitchTo(root->planner_cxt);
+		new_path->pathtarget = copy_pathtarget(new_path->pathtarget);
+		new_path->pathtarget->exprs = (List *) adjust_appendrel_attrs(root,
+											(Node *) new_path->pathtarget->exprs,
+																	child_aris);
+		MemoryContextSwitchTo(old_context);
+	}
+
+	/*
+	 * Change parameterization of subpaths recursively. Also carry out any
+	 * pathtype specific adjustments.
+	 */
+	switch (nodeTag(path))
+	{
+		case T_HashPath:
+		case T_MergePath:
+		case T_NestPath:
+			{
+				JoinPath *jpath = (JoinPath *)new_path;
+
+				jpath->outerjoinpath = reparameterize_path_by_child(root,
+														 jpath->outerjoinpath,
+														 child_rel);
+				jpath->innerjoinpath = reparameterize_path_by_child(root,
+														 jpath->innerjoinpath,
+														 child_rel);
+				jpath->joinrestrictinfo = build_child_clauses(root,
+															  jpath->joinrestrictinfo,
+															  child_aris);
+			}
+			break;
+
+		case T_AppendPath:
+			{
+				AppendPath	*apath = (AppendPath *)new_path;
+				List		*subpaths = NIL;
+				ListCell	*lc;
+
+				foreach (lc, apath->subpaths)
+					subpaths = lappend(subpaths,
+									   reparameterize_path_by_child(root,
+																	lfirst(lc),
+																	child_rel));
+				apath->subpaths = subpaths;
+			}
+
+		case T_IndexPath:
+			{
+				IndexPath *ipath = (IndexPath *)new_path;
+
+				ipath->indexclauses = build_child_clauses(root, ipath->indexclauses,
+														  child_aris);
+				ipath->indexquals = build_child_clauses(root, ipath->indexquals,
+														child_aris);
+			}
+
+		default:
+			/* Nothing to do. */
+			break;
+	}
+
+	return new_path;
+}
diff --git a/src/backend/optimizer/util/placeholder.c b/src/backend/optimizer/util/placeholder.c
index b210914..859e6a6 100644
--- a/src/backend/optimizer/util/placeholder.c
+++ b/src/backend/optimizer/util/placeholder.c
@@ -21,6 +21,7 @@
 #include "optimizer/placeholder.h"
 #include "optimizer/planmain.h"
 #include "optimizer/var.h"
+#include "optimizer/prep.h"
 #include "utils/lsyscache.h"
 
 /* Local functions */
@@ -411,9 +412,15 @@ void
 add_placeholders_to_joinrel(PlannerInfo *root, RelOptInfo *joinrel,
 							RelOptInfo *outer_rel, RelOptInfo *inner_rel)
 {
-	Relids		relids = joinrel->relids;
+	Relids		relids;
 	ListCell   *lc;
 
+	/* PlaceHolderInfo refers to parent relids and not those of a child. */
+	if (joinrel->top_parent_relids)
+		relids = joinrel->top_parent_relids;
+	else
+		relids = joinrel->relids;
+
 	foreach(lc, root->placeholder_list)
 	{
 		PlaceHolderInfo *phinfo = (PlaceHolderInfo *) lfirst(lc);
@@ -424,9 +431,27 @@ add_placeholders_to_joinrel(PlannerInfo *root, RelOptInfo *joinrel,
 			/* Is it computable here? */
 			if (bms_is_subset(phinfo->ph_eval_at, relids))
 			{
+				PlaceHolderVar *phv = phinfo->ph_var;
+
+				/*
+				 * In case the placeholder Var refers to any of the parent
+				 * relation, translate it to refer to the corresponding child.
+				 */
+				if (bms_overlap(phv->phrels, relids) &&
+					joinrel->reloptkind == RELOPT_OTHER_JOINREL)
+				{
+					List *append_rel_infos;
+
+					append_rel_infos = find_appinfos_by_relids(root,
+															  joinrel->relids);
+					phv = (PlaceHolderVar *) adjust_appendrel_attrs(root,
+																  (Node *) phv,
+															 append_rel_infos);
+				}
+
 				/* Yup, add it to the output */
 				joinrel->reltarget->exprs = lappend(joinrel->reltarget->exprs,
-													phinfo->ph_var);
+													phv);
 				joinrel->reltarget->width += phinfo->ph_width;
 
 				/*
@@ -445,7 +470,7 @@ add_placeholders_to_joinrel(PlannerInfo *root, RelOptInfo *joinrel,
 				{
 					QualCost	cost;
 
-					cost_qual_eval_node(&cost, (Node *) phinfo->ph_var->phexpr,
+					cost_qual_eval_node(&cost, (Node *) phv->phexpr,
 										root);
 					joinrel->reltarget->cost.startup += cost.startup;
 					joinrel->reltarget->cost.per_tuple += cost.per_tuple;
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index 72272d9..88e66e4 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -412,6 +412,21 @@ get_relation_info(PlannerInfo *root, Oid relationObjectId, bool inhparent,
 	/* Collect info about relation's foreign keys, if relevant */
 	get_relation_foreign_keys(root, rel, relation, inhparent);
 
+	/*
+	 * Lookup partition scheme for the given relation. Only parent relations
+	 * can be partitioned.
+	 */
+	if (inhparent)
+		rel->part_scheme = find_partition_scheme(root, relation);
+	else
+		rel->part_scheme = NULL;
+
+	if (rel->part_scheme)
+		rel->partexprs = build_baserel_partition_key_exprs(relation,
+														   rel->relid);
+	else
+		rel->partexprs = NULL;
+
 	heap_close(relation, NoLock);
 
 	/*
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index d5326e6..d1433ec 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -15,15 +15,22 @@
 #include "postgres.h"
 
 #include "miscadmin.h"
+#include "catalog/heap.h"
+#include "catalog/partition.h"
 #include "optimizer/clauses.h"
 #include "optimizer/cost.h"
+#include "nodes/makefuncs.h"
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
 #include "optimizer/placeholder.h"
 #include "optimizer/plancat.h"
+#include "optimizer/prep.h"
 #include "optimizer/restrictinfo.h"
 #include "optimizer/tlist.h"
+#include "optimizer/var.h"
+#include "rewrite/rewriteManip.h"
 #include "utils/hsearch.h"
+#include "utils/rel.h"
 
 
 typedef struct JoinHashEntry
@@ -34,10 +41,6 @@ typedef struct JoinHashEntry
 
 static void build_joinrel_tlist(PlannerInfo *root, RelOptInfo *joinrel,
 					RelOptInfo *input_rel);
-static List *build_joinrel_restrictlist(PlannerInfo *root,
-						   RelOptInfo *joinrel,
-						   RelOptInfo *outer_rel,
-						   RelOptInfo *inner_rel);
 static void build_joinrel_joinlist(RelOptInfo *joinrel,
 					   RelOptInfo *outer_rel,
 					   RelOptInfo *inner_rel);
@@ -47,6 +50,20 @@ static List *subbuild_joinrel_restrictlist(RelOptInfo *joinrel,
 static List *subbuild_joinrel_joinlist(RelOptInfo *joinrel,
 						  List *joininfo_list,
 						  List *new_joininfo);
+static void set_foreign_rel_properties(RelOptInfo *joinrel,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel);
+static void build_joinrel_partition_info(RelOptInfo *joinrel,
+						  RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+						  JoinType jointype);
+static void build_child_joinrel_joinlist(PlannerInfo *root,
+								 RelOptInfo *joinrel, RelOptInfo *outer_rel,
+								 RelOptInfo *inner_rel);
+static List *subbuild_child_joinrel_joinlist(PlannerInfo *root,
+							RelOptInfo *joinrel, List *joininfo_list,
+							RelOptInfo *other_rel, List *new_joininfo);
+static List *subbuild_child_joinrel_restrictlist(PlannerInfo *root,
+									RelOptInfo *joinrel, List *joininfo_list,
+									RelOptInfo *other_rel, List *new_restrictlist);
 
 
 /*
@@ -137,6 +154,10 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptKind reloptkind)
 	rel->baserestrictcost.per_tuple = 0;
 	rel->joininfo = NIL;
 	rel->has_eclass_joins = false;
+	rel->part_scheme = NULL;
+	rel->partexprs = NULL;
+	rel->top_parent_relids = NULL;
+	rel->part_rels = NULL;
 
 	/* Check type of rtable entry */
 	switch (rte->rtekind)
@@ -314,6 +335,56 @@ find_join_rel(PlannerInfo *root, Relids relids)
 }
 
 /*
+ * set_foreign_rel_properties
+ *		Set up foreign-join fields if outer and inner relation are foreign
+ *		tables (or joins) belonging to the same server and assigned to the same
+ *		user to check access permissions as.
+ *
+ * In addition to an exact match of userid, we allow the case where one side
+ * has zero userid (implying current user) and the other side has explicit
+ * userid that happens to equal the current user; but in that case, pushdown of
+ * the join is only valid for the current user.  The useridiscurrent field
+ * records whether we had to make such an assumption for this join or any
+ * sub-join.
+ *
+ * Otherwise these fields are left invalid, so GetForeignJoinPaths will not be
+ * called for the join relation.
+ *
+ */
+static void
+set_foreign_rel_properties(RelOptInfo *joinrel, RelOptInfo *outer_rel,
+						   RelOptInfo *inner_rel)
+{
+	if (OidIsValid(outer_rel->serverid) &&
+		inner_rel->serverid == outer_rel->serverid)
+	{
+		if (inner_rel->userid == outer_rel->userid)
+		{
+			joinrel->serverid = outer_rel->serverid;
+			joinrel->userid = outer_rel->userid;
+			joinrel->useridiscurrent = outer_rel->useridiscurrent || inner_rel->useridiscurrent;
+			joinrel->fdwroutine = outer_rel->fdwroutine;
+		}
+		else if (!OidIsValid(inner_rel->userid) &&
+				 outer_rel->userid == GetUserId())
+		{
+			joinrel->serverid = outer_rel->serverid;
+			joinrel->userid = outer_rel->userid;
+			joinrel->useridiscurrent = true;
+			joinrel->fdwroutine = outer_rel->fdwroutine;
+		}
+		else if (!OidIsValid(outer_rel->userid) &&
+				 inner_rel->userid == GetUserId())
+		{
+			joinrel->serverid = outer_rel->serverid;
+			joinrel->userid = inner_rel->userid;
+			joinrel->useridiscurrent = true;
+			joinrel->fdwroutine = outer_rel->fdwroutine;
+		}
+	}
+}
+
+/*
  * build_join_rel
  *	  Returns relation entry corresponding to the union of two given rels,
  *	  creating a new relation entry if none already exists.
@@ -363,7 +434,11 @@ build_join_rel(PlannerInfo *root,
 	 * Nope, so make one.
 	 */
 	joinrel = makeNode(RelOptInfo);
+
+	Assert(!IS_OTHER_REL(outer_rel) && !IS_OTHER_REL(inner_rel));
+
 	joinrel->reloptkind = RELOPT_JOINREL;
+
 	joinrel->relids = bms_copy(joinrelids);
 	joinrel->rows = 0;
 	/* cheap startup cost is interesting iff not all tuples to be retrieved */
@@ -409,47 +484,13 @@ build_join_rel(PlannerInfo *root,
 	joinrel->baserestrictcost.per_tuple = 0;
 	joinrel->joininfo = NIL;
 	joinrel->has_eclass_joins = false;
+	joinrel->part_scheme = NULL;
+	joinrel->partexprs = NULL;
+	joinrel->top_parent_relids = NULL;
+	joinrel->part_rels = NULL;
 
-	/*
-	 * Set up foreign-join fields if outer and inner relation are foreign
-	 * tables (or joins) belonging to the same server and assigned to the same
-	 * user to check access permissions as.  In addition to an exact match of
-	 * userid, we allow the case where one side has zero userid (implying
-	 * current user) and the other side has explicit userid that happens to
-	 * equal the current user; but in that case, pushdown of the join is only
-	 * valid for the current user.  The useridiscurrent field records whether
-	 * we had to make such an assumption for this join or any sub-join.
-	 *
-	 * Otherwise these fields are left invalid, so GetForeignJoinPaths will
-	 * not be called for the join relation.
-	 */
-	if (OidIsValid(outer_rel->serverid) &&
-		inner_rel->serverid == outer_rel->serverid)
-	{
-		if (inner_rel->userid == outer_rel->userid)
-		{
-			joinrel->serverid = outer_rel->serverid;
-			joinrel->userid = outer_rel->userid;
-			joinrel->useridiscurrent = outer_rel->useridiscurrent || inner_rel->useridiscurrent;
-			joinrel->fdwroutine = outer_rel->fdwroutine;
-		}
-		else if (!OidIsValid(inner_rel->userid) &&
-				 outer_rel->userid == GetUserId())
-		{
-			joinrel->serverid = outer_rel->serverid;
-			joinrel->userid = outer_rel->userid;
-			joinrel->useridiscurrent = true;
-			joinrel->fdwroutine = outer_rel->fdwroutine;
-		}
-		else if (!OidIsValid(outer_rel->userid) &&
-				 inner_rel->userid == GetUserId())
-		{
-			joinrel->serverid = outer_rel->serverid;
-			joinrel->userid = inner_rel->userid;
-			joinrel->useridiscurrent = true;
-			joinrel->fdwroutine = outer_rel->fdwroutine;
-		}
-	}
+	/* Compute information relevant to the foreign relations. */
+	set_foreign_rel_properties(joinrel, outer_rel, inner_rel);
 
 	/*
 	 * Create a new tlist containing just the vars that need to be output from
@@ -475,6 +516,10 @@ build_join_rel(PlannerInfo *root,
 	if (bms_is_empty(joinrel->direct_lateral_relids))
 		joinrel->direct_lateral_relids = NULL;
 
+	/* Store the partition information. */
+	build_joinrel_partition_info(joinrel, outer_rel, inner_rel,
+								 sjinfo->jointype);
+
 	/*
 	 * Construct restrict and join clause lists for the new joinrel. (The
 	 * caller might or might not need the restrictlist, but I need it anyway
@@ -517,25 +562,8 @@ build_join_rel(PlannerInfo *root,
 		is_parallel_safe(root, (Node *) joinrel->reltarget->exprs))
 		joinrel->consider_parallel = true;
 
-	/*
-	 * Add the joinrel to the query's joinrel list, and store it into the
-	 * auxiliary hashtable if there is one.  NB: GEQO requires us to append
-	 * the new joinrel to the end of the list!
-	 */
-	root->join_rel_list = lappend(root->join_rel_list, joinrel);
-
-	if (root->join_rel_hash)
-	{
-		JoinHashEntry *hentry;
-		bool		found;
-
-		hentry = (JoinHashEntry *) hash_search(root->join_rel_hash,
-											   &(joinrel->relids),
-											   HASH_ENTER,
-											   &found);
-		Assert(!found);
-		hentry->join_rel = joinrel;
-	}
+	/* Add the joinrel to the query's PlannerInfo. */
+	add_join_rel(root, joinrel);
 
 	/*
 	 * Also, if dynamic-programming join search is active, add the new joinrel
@@ -555,6 +583,125 @@ build_join_rel(PlannerInfo *root,
 }
 
 /*
+ * build_child_join_rel
+ *		Builds RelOptInfo for joining given two child relations from RelOptInfo
+ *		representing the join between their parents.
+ *
+ * 'outer_rel' and 'inner_rel' are the RelOptInfos of child relations being
+ *		joined.
+ * 'parent_joinrel' is the RelOptInfo representing the join between parent
+ *		relations. Most of the members of new RelOptInfo are produced by
+ *		translating corresponding members of this RelOptInfo.
+ * 'sjinfo': context info for child join
+ * 'restrictlist': list of RestrictInfo nodes that apply to this particular
+ *		pair of joinable relations.
+ * 'join_appinfos': list of AppendRelInfo nodes for base child relations involved
+ *		in this join.
+ */
+RelOptInfo *
+build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
+						 RelOptInfo *inner_rel, RelOptInfo *parent_joinrel,
+						 JoinType jointype)
+{
+	RelOptInfo *joinrel = makeNode(RelOptInfo);
+
+	joinrel->reloptkind = RELOPT_OTHER_JOINREL;
+	joinrel->relids = bms_union(outer_rel->relids, inner_rel->relids);
+	joinrel->rows = 0;
+	/* cheap startup cost is interesting iff not all tuples to be retrieved */
+	joinrel->consider_startup = (root->tuple_fraction > 0);
+	joinrel->consider_param_startup = false;
+	joinrel->consider_parallel = false;
+	joinrel->reltarget = create_empty_pathtarget();
+	joinrel->pathlist = NIL;
+	joinrel->ppilist = NIL;
+	joinrel->partial_pathlist = NIL;
+	joinrel->cheapest_startup_path = NULL;
+	joinrel->cheapest_total_path = NULL;
+	joinrel->cheapest_unique_path = NULL;
+	joinrel->cheapest_parameterized_paths = NIL;
+	joinrel->direct_lateral_relids = NULL;
+	joinrel->lateral_relids = NULL;
+	joinrel->relid = 0;			/* indicates not a baserel */
+	joinrel->rtekind = RTE_JOIN;
+	joinrel->min_attr = 0;
+	joinrel->max_attr = 0;
+	joinrel->attr_needed = NULL;
+	joinrel->attr_widths = NULL;
+	joinrel->lateral_vars = NIL;
+	joinrel->lateral_referencers = NULL;
+	joinrel->indexlist = NIL;
+	joinrel->pages = 0;
+	joinrel->tuples = 0;
+	joinrel->allvisfrac = 0;
+	joinrel->subroot = NULL;
+	joinrel->subplan_params = NIL;
+	joinrel->serverid = InvalidOid;
+	joinrel->userid = InvalidOid;
+	joinrel->useridiscurrent = false;
+	joinrel->fdwroutine = NULL;
+	joinrel->fdw_private = NULL;
+	joinrel->baserestrictinfo = NIL;
+	joinrel->baserestrictcost.startup = 0;
+	joinrel->baserestrictcost.per_tuple = 0;
+	joinrel->joininfo = NIL;
+	joinrel->has_eclass_joins = false;
+	joinrel->part_scheme = NULL;
+	joinrel->partexprs = NULL;
+	joinrel->top_parent_relids = NULL;
+	joinrel->part_rels = NULL;
+
+
+	/* Only joins between other relations land here. */
+	Assert(IS_OTHER_REL(outer_rel) && IS_OTHER_REL(inner_rel));
+
+	joinrel->top_parent_relids = bms_union(outer_rel->top_parent_relids,
+										   inner_rel->top_parent_relids);
+
+	/* Compute information relevant to foreign relations. */
+	set_foreign_rel_properties(joinrel, outer_rel, inner_rel);
+
+	/* Build targetlist */
+	build_joinrel_tlist(root, joinrel, outer_rel);
+	build_joinrel_tlist(root, joinrel, inner_rel);
+	/* Add placeholder variables. */
+	add_placeholders_to_joinrel(root, joinrel, outer_rel, inner_rel);
+
+	/* Construct joininfo list. */
+	build_child_joinrel_joinlist(root, joinrel, outer_rel, inner_rel);
+
+	/*
+	 * Lateral relids referred in child join will be same as that referred in
+	 * the parent relation. Throw any partial result computed while building
+	 * the targetlist.
+	 */
+	bms_free(joinrel->direct_lateral_relids);
+	bms_free(joinrel->lateral_relids);
+	joinrel->direct_lateral_relids = (Relids) bms_copy(parent_joinrel->direct_lateral_relids);
+	joinrel->lateral_relids = (Relids) bms_copy(parent_joinrel->lateral_relids);
+
+	/*
+	 * If the parent joinrel has pending equivalence classes, so does the
+	 * child.
+	 */
+	joinrel->has_eclass_joins = parent_joinrel->has_eclass_joins;
+
+	/* Is the join between partitions itself partitioned? */
+	build_joinrel_partition_info(joinrel, outer_rel, inner_rel, jointype);
+
+	/* Child joinrel is parallel safe if parent is parallel safe. */
+	joinrel->consider_parallel = parent_joinrel->consider_parallel;
+
+	/* We build the join only once. */
+	Assert(!find_join_rel(root, joinrel->relids));
+
+	/* Add the relation to the PlannerInfo. */
+	add_join_rel(root, joinrel);
+
+	return joinrel;
+}
+
+/*
  * min_join_parameterization
  *
  * Determine the minimum possible parameterization of a joinrel, that is, the
@@ -609,9 +756,15 @@ static void
 build_joinrel_tlist(PlannerInfo *root, RelOptInfo *joinrel,
 					RelOptInfo *input_rel)
 {
-	Relids		relids = joinrel->relids;
+	Relids		relids;
 	ListCell   *vars;
 
+	/* attrs_needed refers to parent relids and not those of a child. */
+	if (joinrel->top_parent_relids)
+		relids = joinrel->top_parent_relids;
+	else
+		relids = joinrel->relids;
+
 	foreach(vars, input_rel->reltarget->exprs)
 	{
 		Var		   *var = (Var *) lfirst(vars);
@@ -627,23 +780,47 @@ build_joinrel_tlist(PlannerInfo *root, RelOptInfo *joinrel,
 
 		/*
 		 * Otherwise, anything in a baserel or joinrel targetlist ought to be
-		 * a Var.  (More general cases can only appear in appendrel child
-		 * rels, which will never be seen here.)
+		 * a Var or ConvertRowtypeExpr introduced while translating parent
+		 * targetlist to that of the child.
 		 */
-		if (!IsA(var, Var))
+		if (IsA(var, Var))
+		{
+			/* Get the Var's original base rel */
+			baserel = find_base_rel(root, var->varno);
+
+			/* Is it still needed above this joinrel? */
+			ndx = var->varattno - baserel->min_attr;
+		}
+		else if (IsA(var, ConvertRowtypeExpr))
+		{
+			ConvertRowtypeExpr *child_expr = (ConvertRowtypeExpr *) var;
+			Var	 *childvar = (Var *) child_expr->arg;
+
+			/*
+			 * Child's whole-row references are converted to that of parent
+			 * using ConvertRowtypeExpr. In this case, the argument to
+			 * ConvertRowtypeExpr is expected to be a whole-row reference of
+			 * the child.
+			 */
+			Assert(IsA(childvar, Var) && childvar->varattno == 0);
+
+			baserel = find_base_rel(root, childvar->varno);
+			ndx = 0 - baserel->min_attr;
+		}
+		else
 			elog(ERROR, "unexpected node type in rel targetlist: %d",
 				 (int) nodeTag(var));
 
-		/* Get the Var's original base rel */
-		baserel = find_base_rel(root, var->varno);
-
-		/* 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 */
 			joinrel->reltarget->exprs = lappend(joinrel->reltarget->exprs, var);
-			/* Vars have cost zero, so no need to adjust reltarget->cost */
+
+			/*
+			 * Vars have cost zero, so no need to adjust reltarget->cost. Even
+			 * if, it's a ConvertRowtypeExpr, it will be computed only for the
+			 * base relation, costing nothing for a join.
+			 */
 			joinrel->reltarget->width += baserel->attr_widths[ndx];
 		}
 	}
@@ -691,7 +868,7 @@ build_joinrel_tlist(PlannerInfo *root, RelOptInfo *joinrel,
  * RestrictInfo nodes are no longer context-dependent.  Instead, just include
  * the original nodes in the lists made for the join relation.
  */
-static List *
+List *
 build_joinrel_restrictlist(PlannerInfo *root,
 						   RelOptInfo *joinrel,
 						   RelOptInfo *outer_rel,
@@ -780,6 +957,8 @@ subbuild_joinrel_joinlist(RelOptInfo *joinrel,
 {
 	ListCell   *l;
 
+	Assert(joinrel->reloptkind == RELOPT_JOINREL);
+
 	foreach(l, joininfo_list)
 	{
 		RestrictInfo *rinfo = (RestrictInfo *) lfirst(l);
@@ -808,6 +987,154 @@ subbuild_joinrel_joinlist(RelOptInfo *joinrel,
 	return new_joininfo;
 }
 
+/* Similar to build_joinrel_joinlist, but used for child-join relations. */
+static void
+build_child_joinrel_joinlist(PlannerInfo *root, RelOptInfo *joinrel,
+									RelOptInfo *outer_rel, RelOptInfo *inner_rel)
+{
+	List   *result;
+
+	Assert(joinrel->reloptkind == RELOPT_OTHER_JOINREL);
+
+	result = subbuild_child_joinrel_joinlist(root, joinrel,
+											 outer_rel->joininfo, inner_rel,
+											 NIL);
+	result = subbuild_child_joinrel_joinlist(root, joinrel,
+											 inner_rel->joininfo, outer_rel,
+											 result);
+	joinrel->joininfo = result;
+}
+
+/* Similar to subbuild_joinrel_restrictlist(), but used for child-joins. */
+List *
+build_child_joinrel_restrictlist(PlannerInfo *root, RelOptInfo *joinrel,
+								 RelOptInfo *outer_rel, RelOptInfo *inner_rel)
+{
+	List   *result;
+
+	Assert(joinrel->reloptkind == RELOPT_OTHER_JOINREL);
+
+	result = subbuild_child_joinrel_restrictlist(root, joinrel,
+												 outer_rel->joininfo,
+												 inner_rel, NIL);
+	result = subbuild_child_joinrel_restrictlist(root, joinrel,
+												 inner_rel->joininfo,
+												 outer_rel, result);
+
+	/*
+	 * Add on any clauses derived from EquivalenceClasses.  These cannot be
+	 * redundant with the clauses in the joininfo lists, so don't bother
+	 * checking.
+	 */
+	result = list_concat(result,
+						 generate_join_implied_equalities(root,
+														  joinrel->relids,
+														  outer_rel->relids,
+														  inner_rel));
+
+	return result;
+}
+
+/*
+ * Similar to subbuild_joinrel_joinlist() but used for child-joins.
+ */
+static List *
+subbuild_child_joinrel_joinlist(PlannerInfo *root, RelOptInfo *joinrel,
+								List *joininfo_list, RelOptInfo *other_rel,
+								List *new_joininfo)
+{
+	ListCell   *l;
+	List	   *append_rel_infos = find_appinfos_by_relids(root,
+														   other_rel->relids);
+
+	Assert(joinrel->reloptkind == RELOPT_OTHER_JOINREL);
+
+	foreach(l, joininfo_list)
+	{
+		RestrictInfo *rinfo = (RestrictInfo *) lfirst(l);
+		RestrictInfo *parent_rinfo;
+
+		parent_rinfo = rinfo->parent_rinfo ? rinfo->parent_rinfo : rinfo;
+
+		if (bms_is_subset(parent_rinfo->required_relids, joinrel->top_parent_relids))
+		{
+			/*
+			 * The child clause derived from this clause becomes a restriction
+			 * clause for the joinrel, since it refers to no outside rels.  So
+			 * we can ignore it in this routine.
+			 */
+		}
+		else
+		{
+			/*
+			 * If the clause references parent of other joining relation, we
+			 * need to translate those references to the child relation.
+			 */
+			if (bms_overlap(rinfo->required_relids, other_rel->top_parent_relids))
+				rinfo = build_child_restrictinfo(root, rinfo, append_rel_infos);
+
+			/*
+			 * This clause is still a join clause at this level, so add it to
+			 * the new joininfo list, being careful to eliminate duplicates.
+			 * (Since RestrictInfo nodes in different joinlists will have been
+			 * multiply-linked rather than copied, pointer equality should be
+			 * a sufficient test.)
+			 */
+			new_joininfo = list_append_unique_ptr(new_joininfo, rinfo);
+		}
+	}
+
+	list_free(append_rel_infos);
+
+	return new_joininfo;
+}
+
+static List *
+subbuild_child_joinrel_restrictlist(PlannerInfo *root, RelOptInfo *joinrel,
+									List *joininfo_list, RelOptInfo *other_rel,
+									List *new_restrictlist)
+{
+	ListCell   *l;
+	List	   *append_rel_infos = find_appinfos_by_relids(root,
+														   other_rel->relids);
+
+	Assert(joinrel->reloptkind == RELOPT_OTHER_JOINREL);
+
+	foreach(l, joininfo_list)
+	{
+		RestrictInfo *rinfo = (RestrictInfo *) lfirst(l);
+		RestrictInfo *parent_rinfo;
+
+		parent_rinfo = rinfo->parent_rinfo ? rinfo->parent_rinfo : rinfo;
+
+		if (bms_is_subset(parent_rinfo->required_relids, joinrel->top_parent_relids))
+		{
+			/*
+			 * The child clause derived from this clause becomes a restriction
+			 * clause for the joinrel, since it refers to no outside rels.  Add
+			 * it to the list, being careful to eliminate duplicates. (Since
+			 * RestrictInfo nodes in different joinlists will have been
+			 * multiply-linked rather than copied, pointer equality should be a
+			 * sufficient test.)
+			 */
+			Assert(bms_overlap(rinfo->required_relids,
+							   other_rel->top_parent_relids));
+
+			rinfo = build_child_restrictinfo(root, rinfo, append_rel_infos);
+			new_restrictlist = list_append_unique_ptr(new_restrictlist, rinfo);
+		}
+		else
+		{
+			/*
+			 * This clause is still a join clause at this level, so we ignore
+			 * it in this routine.
+			 */
+		}
+	}
+
+	return new_restrictlist;
+}
+
 
 /*
  * build_empty_join_rel
@@ -933,9 +1260,17 @@ find_childrel_appendrelinfo(PlannerInfo *root, RelOptInfo *rel)
  * appendrel ancestors.  This function locates the topmost ancestor,
  * which will be a regular baserel not an otherrel.
  */
-RelOptInfo *
+Relids
 find_childrel_top_parent(PlannerInfo *root, RelOptInfo *rel)
 {
+	Assert(IS_OTHER_REL(rel));
+
+	/* Child-join relations cache this in their RelOptInfo. */
+	if (rel->reloptkind == RELOPT_OTHER_JOINREL)
+		return rel->top_parent_relids;
+
+	Assert(rel->reloptkind == RELOPT_OTHER_MEMBER_REL);
+
 	do
 	{
 		AppendRelInfo *appinfo = find_childrel_appendrelinfo(root, rel);
@@ -947,7 +1282,7 @@ find_childrel_top_parent(PlannerInfo *root, RelOptInfo *rel)
 
 	Assert(rel->reloptkind == RELOPT_BASEREL);
 
-	return rel;
+	return rel->relids;
 }
 
 
@@ -1009,12 +1344,8 @@ get_baserel_parampathinfo(PlannerInfo *root, RelOptInfo *baserel,
 	Assert(!bms_overlap(baserel->relids, required_outer));
 
 	/* If we already have a PPI for this parameterization, just return it */
-	foreach(lc, baserel->ppilist)
-	{
-		ppi = (ParamPathInfo *) lfirst(lc);
-		if (bms_equal(ppi->ppi_req_outer, required_outer))
-			return ppi;
-	}
+	if ((ppi = find_param_path_info(baserel, required_outer)))
+		return ppi;
 
 	/*
 	 * Identify all joinclauses that are movable to this base rel given this
@@ -1251,12 +1582,8 @@ get_joinrel_parampathinfo(PlannerInfo *root, RelOptInfo *joinrel,
 	*restrict_clauses = list_concat(pclauses, *restrict_clauses);
 
 	/* If we already have a PPI for this parameterization, just return it */
-	foreach(lc, joinrel->ppilist)
-	{
-		ppi = (ParamPathInfo *) lfirst(lc);
-		if (bms_equal(ppi->ppi_req_outer, required_outer))
-			return ppi;
-	}
+	if ((ppi = find_param_path_info(joinrel, required_outer)))
+		return ppi;
 
 	/* Estimate the number of rows returned by the parameterized join */
 	rows = get_parameterized_joinrel_size(root, joinrel,
@@ -1295,7 +1622,6 @@ ParamPathInfo *
 get_appendrel_parampathinfo(RelOptInfo *appendrel, Relids required_outer)
 {
 	ParamPathInfo *ppi;
-	ListCell   *lc;
 
 	/* Unparameterized paths have no ParamPathInfo */
 	if (bms_is_empty(required_outer))
@@ -1304,12 +1630,8 @@ get_appendrel_parampathinfo(RelOptInfo *appendrel, Relids required_outer)
 	Assert(!bms_overlap(appendrel->relids, required_outer));
 
 	/* If we already have a PPI for this parameterization, just return it */
-	foreach(lc, appendrel->ppilist)
-	{
-		ppi = (ParamPathInfo *) lfirst(lc);
-		if (bms_equal(ppi->ppi_req_outer, required_outer))
-			return ppi;
-	}
+	if ((ppi = find_param_path_info(appendrel, required_outer)))
+		return ppi;
 
 	/* Else build the ParamPathInfo */
 	ppi = makeNode(ParamPathInfo);
@@ -1320,3 +1642,130 @@ get_appendrel_parampathinfo(RelOptInfo *appendrel, Relids required_outer)
 
 	return ppi;
 }
+
+/*
+ * add_join_rel
+ *		Add given join relation to the list of join relations in the given
+ *		PlannerInfo. Also add it to the auxiliary hashtable if there is one.
+ */
+void
+add_join_rel(PlannerInfo *root, RelOptInfo *joinrel)
+{
+	/* GEQO requires us to append the new joinrel to the end of the list! */
+	root->join_rel_list = lappend(root->join_rel_list, joinrel);
+
+	/* store it into the auxiliary hashtable if there is one. */
+	if (root->join_rel_hash)
+	{
+		JoinHashEntry *hentry;
+		bool		found;
+
+		hentry = (JoinHashEntry *) hash_search(root->join_rel_hash,
+											   &(joinrel->relids),
+											   HASH_ENTER,
+											   &found);
+		Assert(!found);
+		hentry->join_rel = joinrel;
+	}
+}
+
+/*
+ * build_joinrel_partition_info
+ *		If the join between given partitioned relations is possibly partitioned
+ *		set the partitioning scheme and partition keys expressions for the
+ *		join.
+ *
+ * If the two relations have same partitioning scheme, their join may be
+ * partitioned and will follow the same partitioning scheme as the joining
+ * relations.
+ */
+static void
+build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
+							 RelOptInfo *inner_rel, JoinType jointype)
+{
+	int		num_pks;
+	int		cnt;
+
+	/* Nothing to do if partition-wise join technique is disabled. */
+	if (!enable_partition_wise_join)
+	{
+		joinrel->part_scheme = NULL;
+		return;
+	}
+
+	/*
+	 * The join is not partitioned, if any of the relations being joined are
+	 * not partitioned or they do not have same partitioning scheme.
+	 */
+	if (!outer_rel->part_scheme || !inner_rel->part_scheme ||
+		outer_rel->part_scheme != inner_rel->part_scheme)
+	{
+		joinrel->part_scheme = NULL;
+		return;
+	}
+
+	/*
+	 * This function will be called only once for each joinrel, hence it should
+	 * not have partition scheme, partition key expressions and array for
+	 * storing child relations set.
+	 */
+	Assert(!joinrel->part_scheme && !joinrel->partexprs &&
+		   !joinrel->part_rels);
+
+	/*
+	 * Join relation is partitioned using same partitioning scheme as the
+	 * joining relations.
+	 */
+	joinrel->part_scheme = outer_rel->part_scheme;
+	num_pks = joinrel->part_scheme->partnatts;
+
+	/*
+	 * Construct partition keys for the join.
+	 *
+	 * An INNER join between two partitioned relations is partition by key
+	 * expressions from both the relations. For tables A and B partitioned by a and b
+	 * respectively, (A INNER JOIN B ON A.a = B.b) is partitioned by both A.a
+	 * and B.b.
+	 *
+	 * An OUTER join like (A LEFT JOIN B ON A.a = B.b) may produce rows with
+	 * B.b NULL. These rows may not fit the partitioning conditions imposed on
+	 * B.b. Hence, strictly speaking, the join is not partitioned by B.b.
+	 * Strictly speaking, partition keys of an OUTER join should include
+	 * partition key expressions from the OUTER side only. Consider a join like
+	 * (A LEFT JOIN B on (A.a = B.b) LEFT JOIN C ON B.b = C.c. If we do not
+	 * include B.b as partition key expression for (AB), it prohibits us from
+	 * using partition-wise join when joining (AB) with C as there is no
+	 * equi-join between partition keys of joining relations. But two NULL
+	 * values are never equal and no two rows from mis-matching partitions can
+	 * join. Hence it's safe to include B.b as partition key expression for
+	 * (AB), even though rows in (AB) are not strictly partitioned by B.b.
+	 */
+	joinrel->partexprs = (List **) palloc0(sizeof(List *) * num_pks);
+	for (cnt = 0; cnt < num_pks; cnt++)
+	{
+		List *pkexpr = list_copy(outer_rel->partexprs[cnt]);
+
+		pkexpr = list_concat(pkexpr,
+							 list_copy(inner_rel->partexprs[cnt]));
+		joinrel->partexprs[cnt] = pkexpr;
+	}
+}
+
+/*
+ * Returns a ParamPathInfo for outer relations specified by required_outer, if
+ * already available in the given rel. Returns NULL otherwise.
+ */
+ParamPathInfo *
+find_param_path_info(RelOptInfo *rel, Relids required_outer)
+{
+	ListCell   *lc;
+
+	foreach(lc, rel->ppilist)
+	{
+		ParamPathInfo  *ppi = (ParamPathInfo *) lfirst(lc);
+		if (bms_equal(ppi->ppi_req_outer, required_outer))
+			return ppi;
+	}
+
+	return NULL;
+}
diff --git a/src/backend/postmaster/startup.c b/src/backend/postmaster/startup.c
index a7ae7e3..6787df6 100644
--- a/src/backend/postmaster/startup.c
+++ b/src/backend/postmaster/startup.c
@@ -169,7 +169,6 @@ HandleStartupProcInterrupts(void)
 		exit(1);
 }
 
-
 /* ----------------------------------
  *	Startup Process main entry point
  * ----------------------------------
diff --git a/src/backend/utils/adt/selfuncs.c b/src/backend/utils/adt/selfuncs.c
index 4973396..713e361 100644
--- a/src/backend/utils/adt/selfuncs.c
+++ b/src/backend/utils/adt/selfuncs.c
@@ -3426,7 +3426,9 @@ estimate_num_groups(PlannerInfo *root, List *groupExprs, double input_rows,
 		/*
 		 * Sanity check --- don't divide by zero if empty relation.
 		 */
-		Assert(rel->reloptkind == RELOPT_BASEREL);
+		Assert(rel->reloptkind == RELOPT_BASEREL ||
+			   rel->reloptkind == RELOPT_OTHER_MEMBER_REL);
+
 		if (rel->tuples > 0)
 		{
 			/*
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index a025117..ee0c4f9 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -894,6 +894,15 @@ static struct config_bool ConfigureNamesBool[] =
 		true,
 		NULL, NULL, NULL
 	},
+	{
+		{"enable_partition_wise_join", PGC_USERSET, QUERY_TUNING_METHOD,
+			gettext_noop("Enables partition-wise join."),
+			NULL
+		},
+		&enable_partition_wise_join,
+		true,
+		NULL, NULL, NULL
+	},
 
 	{
 		{"geqo", PGC_USERSET, QUERY_TUNING_GEQO,
@@ -2937,6 +2946,16 @@ static struct config_real ConfigureNamesReal[] =
 	},
 
 	{
+		{"partition_wise_plan_weight", PGC_USERSET, QUERY_TUNING_OTHER,
+			gettext_noop("Multiplication factor for partition-wise plan costs."),
+			NULL
+		},
+		&partition_wise_plan_weight,
+		DEFAULT_PARTITION_WISE_PLAN_WEIGHT, 0, DBL_MAX,
+		NULL, NULL, NULL
+	},
+
+	{
 		{"bgwriter_lru_multiplier", PGC_SIGHUP, RESOURCES_BGWRITER,
 			gettext_noop("Multiple of the average buffer usage to free per round."),
 			NULL
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 21effbf..e174923 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -63,6 +63,42 @@ typedef struct PartitionDispatchData
 
 typedef struct PartitionDispatchData *PartitionDispatch;
 
+/*
+ * Partitioning scheme
+ *		Structure to hold partitioning scheme for a given relation.
+ *
+ * Multiple relations may be partitioned in the same way. The relations
+ * resulting from joining such relations may be partitioned in the same way as
+ * the joining relations. Similarly, relations derived from such relations by
+ * grouping, sorting may be partitioned in the same way as the underlying
+ * scan relations. All such relations partitioned in the same way share the
+ * partitioning scheme.
+ *
+ * PlannerInfo stores a list of distinct "canonical" partitioning schemes.
+ * RelOptInfo of a partitioned relation holds the pointer to "canonical"
+ * partitioning scheme.
+ */
+typedef struct PartitionSchemeData
+{
+	/* Information about partitions */
+	int			nparts;			/* number of partitions */
+	PartitionBoundInfo boundinfo;	/* Partition bounds/lists */
+
+	/* Information about partition keys */
+	char		strategy;		/* partition strategy */
+	int16		partnatts;		/* number of partition attributes */
+	Oid		   *partopfamily;	/* OIDs of operator families */
+	Oid		   *partopcintype;	/* OIDs of opclass declared input data types */
+	Oid		   *key_types;		/* OIDs of partition key data types. */
+	int32	   *key_typmods;	/* typmods of partition keys. */
+	Oid		   *key_collations;	/* OIDs of collations of partition keys. */
+} PartitionSchemeData;
+
+typedef struct PartitionSchemeData *PartitionScheme;
+
+/* Include here to avoid circular dependency with relation.h. */
+struct PlannerInfo;
+
 extern void RelationBuildPartitionDesc(Relation relation);
 extern bool partition_bounds_equal(PartitionKey key,
 					   PartitionBoundInfo p1, PartitionBoundInfo p2);
@@ -80,4 +116,9 @@ extern int get_partition_for_tuple(PartitionDispatch *pd,
 					TupleTableSlot *slot,
 					EState *estate,
 					Oid *failed_at);
+extern List **build_baserel_partition_key_exprs(Relation relation,
+												Index varno);
+extern PartitionScheme find_partition_scheme(struct PlannerInfo *root,
+											 Relation rel);
+
 #endif   /* PARTITION_H */
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index c514d3f..297ec92 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -238,6 +238,7 @@ typedef enum NodeTag
 	T_NestPath,
 	T_MergePath,
 	T_HashPath,
+	T_PartitionJoinPath,
 	T_AppendPath,
 	T_MergeAppendPath,
 	T_ResultPath,
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 3a1255a..0a7f21a 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 "catalog/partition.h"
 #include "lib/stringinfo.h"
 #include "nodes/params.h"
 #include "nodes/parsenodes.h"
@@ -263,6 +264,9 @@ typedef struct PlannerInfo
 
 	List	   *initial_rels;	/* RelOptInfos we are now trying to join */
 
+	List	   *part_schemes;	/* Canonicalised partition schemes
+								 * used in the query. */
+
 	/* Use fetch_upper_rel() to get any particular upper rel */
 	List	   *upper_rels[UPPERREL_FINAL + 1]; /* upper-rel RelOptInfos */
 
@@ -352,6 +356,11 @@ typedef struct PlannerInfo
  * handling join alias Vars.  Currently this is not needed because all join
  * alias Vars are expanded to non-aliased form during preprocess_expression.
  *
+ * We also have relations representing joins between child relations of
+ * different partitioned tables. These relations are not added to
+ * join_rel_level lists as they are not joined directly by the dynamic
+ * programming algorithm.
+ *
  * There is also a RelOptKind for "upper" relations, which are RelOptInfos
  * that describe post-scan/join processing steps, such as aggregation.
  * Many of the fields in these RelOptInfos are meaningless, but their Path
@@ -471,10 +480,19 @@ typedef enum RelOptKind
 	RELOPT_BASEREL,
 	RELOPT_JOINREL,
 	RELOPT_OTHER_MEMBER_REL,
+	RELOPT_OTHER_JOINREL,
 	RELOPT_UPPER_REL,
 	RELOPT_DEADREL
 } RelOptKind;
 
+#define IS_OTHER_REL(rel) \
+	((rel)->reloptkind == RELOPT_OTHER_MEMBER_REL || \
+	 (rel)->reloptkind == RELOPT_OTHER_JOINREL)
+
+#define IS_JOIN_REL(rel)	\
+	((rel)->reloptkind == RELOPT_JOINREL || \
+	 (rel)->reloptkind == RELOPT_OTHER_JOINREL)
+
 typedef struct RelOptInfo
 {
 	NodeTag		type;
@@ -542,6 +560,27 @@ typedef struct RelOptInfo
 	List	   *joininfo;		/* RestrictInfo structures for join clauses
 								 * involving this rel */
 	bool		has_eclass_joins;		/* T means joininfo is incomplete */
+
+	/* For partitioned relations. */
+	PartitionScheme	    part_scheme;	/* Partitioning scheme. */
+	struct RelOptInfo **part_rels;		/* Array of RelOptInfos of partitions,
+										 * stored in the same order as bounds
+										 * or lists in PartitionScheme.
+										 */
+	List			  **partexprs;		/* Array of list of partition key
+										 * expressions. For base relations
+										 * these are one element lists. For
+										 * join there may be as many elements
+										 * as the number of joining
+										 * relations.
+										 */
+	List			   *partitioned_joins;	/* List of join orders which yield
+											 * relations partitioned by above
+											 * partition scheme.
+											 */
+
+	/* Set only for "other" base or join relations. */
+	Relids		top_parent_relids;		/* Relids of topmost parents. */
 } RelOptInfo;
 
 /*
@@ -1469,6 +1508,14 @@ typedef struct LimitPath
 	Node	   *limitCount;		/* COUNT parameter, or NULL if none */
 } LimitPath;
 
+/*
+ * PartitionJoinPath represents partition-wise join between two partitioned
+ * tables.
+ */
+typedef struct PartitionJoinPath
+{
+	Path		path;
+} PartitionJoinPath;
 
 /*
  * Restriction clause info.
@@ -1663,6 +1710,12 @@ typedef struct RestrictInfo
 	/* cache space for hashclause processing; -1 if not yet set */
 	Selectivity left_bucketsize;	/* avg bucketsize of left side */
 	Selectivity right_bucketsize;		/* avg bucketsize of right side */
+
+	/* Only one of these two can be set. */
+	List	   *child_rinfos;		/* RestrictInfos derived for children. */
+	struct RestrictInfo *parent_rinfo;		/* Parent restrictinfo this
+											 * RestrictInf is derived from.
+											 */
 } RestrictInfo;
 
 /*
@@ -1785,6 +1838,19 @@ typedef struct SpecialJoinInfo
 } SpecialJoinInfo;
 
 /*
+ * Partitioned join information
+ *
+ * Saves information about relations which can be joined partition-wise and
+ * thus produce result partitioned by the partition scheme of the relation.
+ */
+typedef struct PartitionedJoin
+{
+	RelOptInfo	   *rel1;
+	RelOptInfo	   *rel2;
+	SpecialJoinInfo	   *sjinfo;
+} PartitionedJoin;
+
+/*
  * Append-relation info.
  *
  * When we expand an inheritable table or a UNION-ALL subselect into an
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 2a4df2f..aff7ab7 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -30,6 +30,7 @@
 #define DEFAULT_PARALLEL_SETUP_COST  1000.0
 
 #define DEFAULT_EFFECTIVE_CACHE_SIZE  524288	/* measured in pages */
+#define DEFAULT_PARTITION_WISE_PLAN_WEIGHT 1
 
 typedef enum
 {
@@ -66,7 +67,9 @@ extern bool enable_nestloop;
 extern bool enable_material;
 extern bool enable_mergejoin;
 extern bool enable_hashjoin;
+extern bool enable_partition_wise_join;
 extern int	constraint_exclusion;
+extern double partition_wise_plan_weight;
 
 extern double clamp_row_est(double nrows);
 extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 71d9154..f36277f 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -225,10 +225,17 @@ extern LimitPath *create_limit_path(PlannerInfo *root, RelOptInfo *rel,
 				  Path *subpath,
 				  Node *limitOffset, Node *limitCount,
 				  int64 offset_est, int64 count_est);
+extern PartitionJoinPath *create_partition_join_path(RelOptInfo *rel,
+								 List *subpaths, Bitmapset *required_outer);
+extern PartitionJoinPath *create_partition_join_path_with_pathkeys(PlannerInfo *root,
+								 RelOptInfo *rel, List *subpaths,
+								 List *pathkeys, Bitmapset *required_outer);
 
 extern Path *reparameterize_path(PlannerInfo *root, Path *path,
 					Relids required_outer,
 					double loop_count);
+extern Path *reparameterize_path_by_child(PlannerInfo *root, Path *path,
+					RelOptInfo *child_rel);
 
 /*
  * prototypes for relnode.c
@@ -253,7 +260,7 @@ extern RelOptInfo *fetch_upper_rel(PlannerInfo *root, UpperRelationKind kind,
 				Relids relids);
 extern AppendRelInfo *find_childrel_appendrelinfo(PlannerInfo *root,
 							RelOptInfo *rel);
-extern RelOptInfo *find_childrel_top_parent(PlannerInfo *root, RelOptInfo *rel);
+extern Relids find_childrel_top_parent(PlannerInfo *root, RelOptInfo *rel);
 extern Relids find_childrel_parents(PlannerInfo *root, RelOptInfo *rel);
 extern ParamPathInfo *get_baserel_parampathinfo(PlannerInfo *root,
 						  RelOptInfo *baserel,
@@ -267,5 +274,18 @@ extern ParamPathInfo *get_joinrel_parampathinfo(PlannerInfo *root,
 						  List **restrict_clauses);
 extern ParamPathInfo *get_appendrel_parampathinfo(RelOptInfo *appendrel,
 							Relids required_outer);
+extern RelOptInfo *build_child_join_rel(PlannerInfo *root,
+						 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+						 RelOptInfo *parent_joinrel, JoinType jointype);
+extern List *build_joinrel_restrictlist(PlannerInfo *root,
+						   RelOptInfo *joinrel,
+						   RelOptInfo *outer_rel,
+						   RelOptInfo *inner_rel);
+extern void add_join_rel(PlannerInfo *root, RelOptInfo *joinrel);
+extern List *build_child_joinrel_restrictlist(PlannerInfo *root,
+								 RelOptInfo *joinrel, RelOptInfo *outer_rel,
+								 RelOptInfo *inner_rel);
+extern ParamPathInfo *find_param_path_info(RelOptInfo *rel,
+									  Relids required_outer);
 
 #endif   /* PATHNODE_H */
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 44abe83..5d7bcd9 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -53,6 +53,8 @@ extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 					 List *initial_rels);
 
 extern void generate_gather_paths(PlannerInfo *root, RelOptInfo *rel);
+extern void generate_partition_wise_join_paths(PlannerInfo *root,
+											   RelOptInfo *rel);
 
 #ifdef OPTIMIZER_DEBUG
 extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
@@ -104,6 +106,9 @@ extern bool have_join_order_restriction(PlannerInfo *root,
 							RelOptInfo *rel1, RelOptInfo *rel2);
 extern bool have_dangerous_phv(PlannerInfo *root,
 				   Relids outer_relids, Relids inner_params);
+extern void add_paths_to_child_joinrel(PlannerInfo *root,
+									   RelOptInfo *parent_joinrel,
+									   int child_id);
 
 /*
  * equivclass.c
@@ -219,4 +224,6 @@ extern PathKey *make_canonical_pathkey(PlannerInfo *root,
 					   EquivalenceClass *eclass, Oid opfamily,
 					   int strategy, bool nulls_first);
 
+extern Relids adjust_child_relids(Relids relids, List *append_rel_infos);
+
 #endif   /* PATHS_H */
diff --git a/src/include/optimizer/prep.h b/src/include/optimizer/prep.h
index fb35b68..2483303 100644
--- a/src/include/optimizer/prep.h
+++ b/src/include/optimizer/prep.h
@@ -28,6 +28,9 @@ extern void flatten_simple_union_all(PlannerInfo *root);
 extern void reduce_outer_joins(PlannerInfo *root);
 extern Relids get_relids_in_jointree(Node *jtnode, bool include_joins);
 extern Relids get_relids_for_join(PlannerInfo *root, int joinrelid);
+extern Node *adjust_join_appendrel_attrs(PlannerInfo *root, Node *node,
+										 List *append_rel_infos);
+extern List *find_appinfos_by_relids(PlannerInfo *root, Relids relids);
 
 /*
  * prototypes for prepqual.c
@@ -58,9 +61,13 @@ extern RelOptInfo *plan_set_operations(PlannerInfo *root);
 extern void expand_inherited_tables(PlannerInfo *root);
 
 extern Node *adjust_appendrel_attrs(PlannerInfo *root, Node *node,
-					   AppendRelInfo *appinfo);
+					   List *appinfos);
 
 extern Node *adjust_appendrel_attrs_multilevel(PlannerInfo *root, Node *node,
 								  RelOptInfo *child_rel);
+extern RestrictInfo *build_child_restrictinfo(PlannerInfo *root,
+								 RestrictInfo *rinfo, List *append_rel_infos);
+extern List *build_child_clauses(PlannerInfo *root, List *clauses,
+								 List *append_rel_infos);
 
 #endif   /* PREP_H */
diff --git a/src/test/regress/expected/multi_level_partition_join.out b/src/test/regress/expected/multi_level_partition_join.out
new file mode 100644
index 0000000..d40ae55
--- /dev/null
+++ b/src/test/regress/expected/multi_level_partition_join.out
@@ -0,0 +1,458 @@
+--
+-- multi-leveled partitions
+--
+CREATE TABLE prt1_l (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_l_p1 PARTITION OF prt1_l FOR VALUES FROM (0) TO (250) PARTITION BY RANGE (b);
+CREATE TABLE prt1_l_p1_p1 PARTITION OF prt1_l_p1 FOR VALUES FROM (0) TO (100);
+CREATE TABLE prt1_l_p1_p2 PARTITION OF prt1_l_p1 FOR VALUES FROM (100) TO (250);
+CREATE TABLE prt1_l_p2 PARTITION OF prt1_l FOR VALUES FROM (250) TO (500) PARTITION BY RANGE (c);
+CREATE TABLE prt1_l_p2_p1 PARTITION OF prt1_l_p2 FOR VALUES FROM ('0250') TO ('0400');
+CREATE TABLE prt1_l_p2_p2 PARTITION OF prt1_l_p2 FOR VALUES FROM ('0400') TO ('0500');
+CREATE TABLE prt1_l_p3 PARTITION OF prt1_l FOR VALUES FROM (500) TO (600) PARTITION BY RANGE ((b + a));
+CREATE TABLE prt1_l_p3_p1 PARTITION OF prt1_l_p3 FOR VALUES FROM (1000) TO (1100);
+CREATE TABLE prt1_l_p3_p2 PARTITION OF prt1_l_p3 FOR VALUES FROM (1100) TO (1200);
+INSERT INTO prt1_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_l;
+ANALYZE prt1_l_p1;
+ANALYZE prt1_l_p1_p1;
+ANALYZE prt1_l_p1_p2;
+ANALYZE prt1_l_p2;
+ANALYZE prt1_l_p2_p1;
+ANALYZE prt1_l_p2_p2;
+ANALYZE prt1_l_p3;
+ANALYZE prt1_l_p3_p1;
+ANALYZE prt1_l_p3_p2;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_l AS SELECT * FROM prt1_l;
+CREATE TABLE prt2_l (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_l_p1 PARTITION OF prt2_l FOR VALUES FROM (0) TO (250) PARTITION BY RANGE (a);
+CREATE TABLE prt2_l_p1_p1 PARTITION OF prt2_l_p1 FOR VALUES FROM (0) TO (100);
+CREATE TABLE prt2_l_p1_p2 PARTITION OF prt2_l_p1 FOR VALUES FROM (100) TO (250);
+CREATE TABLE prt2_l_p2 PARTITION OF prt2_l FOR VALUES FROM (250) TO (500) PARTITION BY RANGE (c);
+CREATE TABLE prt2_l_p2_p1 PARTITION OF prt2_l_p2 FOR VALUES FROM ('0250') TO ('0400');
+CREATE TABLE prt2_l_p2_p2 PARTITION OF prt2_l_p2 FOR VALUES FROM ('0400') TO ('0500');
+CREATE TABLE prt2_l_p3 PARTITION OF prt2_l FOR VALUES FROM (500) TO (600) PARTITION BY RANGE ((a + b));
+CREATE TABLE prt2_l_p3_p1 PARTITION OF prt2_l_p3 FOR VALUES FROM (1000) TO (1100);
+CREATE TABLE prt2_l_p3_p2 PARTITION OF prt2_l_p3 FOR VALUES FROM (1100) TO (1200);
+INSERT INTO prt2_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_l;
+ANALYZE prt2_l_p1;
+ANALYZE prt2_l_p1_p1;
+ANALYZE prt2_l_p1_p2;
+ANALYZE prt2_l_p2;
+ANALYZE prt2_l_p2_p1;
+ANALYZE prt2_l_p2_p2;
+ANALYZE prt2_l_p3;
+ANALYZE prt2_l_p3_p1;
+ANALYZE prt2_l_p3_p2;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_l AS SELECT * FROM prt2_l;
+EXPLAIN (VERBOSE, 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 = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                                                      QUERY PLAN                                                                      
+------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((t2.b = t1.a) AND (t2.a = t1.b) AND ((t2.c)::text = (t1.c)::text) AND ((t2.a + t2.b) = (t1.b + t1.a)))
+               ->  Seq Scan on public.prt2_l_p1_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_l_p1_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.a = t1_1.b) AND ((t2_1.c)::text = (t1_1.c)::text) AND ((t2_1.a + t2_1.b) = (t1_1.b + t1_1.a)))
+               ->  Seq Scan on public.prt2_l_p1_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_l_p1_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.a = t1_2.b) AND ((t2_2.c)::text = (t1_2.c)::text) AND ((t2_2.a + t2_2.b) = (t1_2.b + t1_2.a)))
+               ->  Seq Scan on public.prt2_l_p2_p1 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_l_p2_p1 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_3.a, t1_3.c, t2_3.b, t2_3.c
+               Hash Cond: ((t2_3.b = t1_3.a) AND (t2_3.a = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text) AND ((t2_3.a + t2_3.b) = (t1_3.b + t1_3.a)))
+               ->  Seq Scan on public.prt2_l_p2_p2 t2_3
+                     Output: t2_3.b, t2_3.c, t2_3.a
+               ->  Hash
+                     Output: t1_3.a, t1_3.c, t1_3.b
+                     ->  Seq Scan on public.prt1_l_p2_p2 t1_3
+                           Output: t1_3.a, t1_3.c, t1_3.b
+                           Filter: ((t1_3.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_4.a, t1_4.c, t2_4.b, t2_4.c
+               Hash Cond: ((t2_4.b = t1_4.a) AND (t2_4.a = t1_4.b) AND ((t2_4.c)::text = (t1_4.c)::text) AND ((t2_4.a + t2_4.b) = (t1_4.b + t1_4.a)))
+               ->  Seq Scan on public.prt2_l_p3_p1 t2_4
+                     Output: t2_4.b, t2_4.c, t2_4.a
+               ->  Hash
+                     Output: t1_4.a, t1_4.c, t1_4.b
+                     ->  Seq Scan on public.prt1_l_p3_p1 t1_4
+                           Output: t1_4.a, t1_4.c, t1_4.b
+                           Filter: ((t1_4.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_5.a, t1_5.c, t2_5.b, t2_5.c
+               Hash Cond: ((t2_5.b = t1_5.a) AND (t2_5.a = t1_5.b) AND ((t2_5.c)::text = (t1_5.c)::text) AND ((t2_5.a + t2_5.b) = (t1_5.b + t1_5.a)))
+               ->  Seq Scan on public.prt2_l_p3_p2 t2_5
+                     Output: t2_5.b, t2_5.c, t2_5.a
+               ->  Hash
+                     Output: t1_5.a, t1_5.c, t1_5.b
+                     ->  Seq Scan on public.prt1_l_p3_p2 t1_5
+                           Output: t1_5.a, t1_5.c, t1_5.b
+                           Filter: ((t1_5.a % 25) = 0)
+(64 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1, prt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1, uprt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 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)
+
+EXPLAIN (VERBOSE, 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                                                      QUERY PLAN                                                                      
+------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((t2.b = t1.a) AND (t2.a = t1.b) AND ((t2.c)::text = (t1.c)::text) AND ((t2.a + t2.b) = (t1.b + t1.a)))
+               ->  Seq Scan on public.prt2_l_p1_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_l_p1_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.a = t1_1.b) AND ((t2_1.c)::text = (t1_1.c)::text) AND ((t2_1.a + t2_1.b) = (t1_1.b + t1_1.a)))
+               ->  Seq Scan on public.prt2_l_p1_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_l_p1_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.a = t1_2.b) AND ((t2_2.c)::text = (t1_2.c)::text) AND ((t2_2.a + t2_2.b) = (t1_2.b + t1_2.a)))
+               ->  Seq Scan on public.prt2_l_p2_p1 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_l_p2_p1 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_3.a, t1_3.c, t2_3.b, t2_3.c
+               Hash Cond: ((t2_3.b = t1_3.a) AND (t2_3.a = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text) AND ((t2_3.a + t2_3.b) = (t1_3.b + t1_3.a)))
+               ->  Seq Scan on public.prt2_l_p2_p2 t2_3
+                     Output: t2_3.b, t2_3.c, t2_3.a
+               ->  Hash
+                     Output: t1_3.a, t1_3.c, t1_3.b
+                     ->  Seq Scan on public.prt1_l_p2_p2 t1_3
+                           Output: t1_3.a, t1_3.c, t1_3.b
+                           Filter: ((t1_3.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_4.a, t1_4.c, t2_4.b, t2_4.c
+               Hash Cond: ((t2_4.b = t1_4.a) AND (t2_4.a = t1_4.b) AND ((t2_4.c)::text = (t1_4.c)::text) AND ((t2_4.a + t2_4.b) = (t1_4.b + t1_4.a)))
+               ->  Seq Scan on public.prt2_l_p3_p1 t2_4
+                     Output: t2_4.b, t2_4.c, t2_4.a
+               ->  Hash
+                     Output: t1_4.a, t1_4.c, t1_4.b
+                     ->  Seq Scan on public.prt1_l_p3_p1 t1_4
+                           Output: t1_4.a, t1_4.c, t1_4.b
+                           Filter: ((t1_4.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_5.a, t1_5.c, t2_5.b, t2_5.c
+               Hash Cond: ((t2_5.b = t1_5.a) AND (t2_5.a = t1_5.b) AND ((t2_5.c)::text = (t1_5.c)::text) AND ((t2_5.a + t2_5.b) = (t1_5.b + t1_5.a)))
+               ->  Seq Scan on public.prt2_l_p3_p2 t2_5
+                     Output: t2_5.b, t2_5.c, t2_5.a
+               ->  Hash
+                     Output: t1_5.a, t1_5.c, t1_5.b
+                     ->  Seq Scan on public.prt1_l_p3_p2 t1_5
+                           Output: t1_5.a, t1_5.c, t1_5.b
+                           Filter: ((t1_5.a % 25) = 0)
+(64 rows)
+
+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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 LEFT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+                                                                         QUERY PLAN                                                                         
+------------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: ((t1.a = t2.b) AND (t1.b = t2.a) AND ((t1.c)::text = (t2.c)::text) AND ((t1.b + t1.a) = (t2.a + t2.b)))
+                     ->  Seq Scan on public.prt1_l_p1_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                     ->  Hash
+                           Output: t2.b, t2.c, t2.a
+                           ->  Seq Scan on public.prt2_l_p1_p1 t2
+                                 Output: t2.b, t2.c, t2.a
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: ((t1_1.a = t2_1.b) AND (t1_1.b = t2_1.a) AND ((t1_1.c)::text = (t2_1.c)::text) AND ((t1_1.b + t1_1.a) = (t2_1.a + t2_1.b)))
+                     ->  Seq Scan on public.prt1_l_p1_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c, t2_1.a
+                           ->  Seq Scan on public.prt2_l_p1_p2 t2_1
+                                 Output: t2_1.b, t2_1.c, t2_1.a
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: ((t1_2.a = t2_2.b) AND (t1_2.b = t2_2.a) AND ((t1_2.c)::text = (t2_2.c)::text) AND ((t1_2.b + t1_2.a) = (t2_2.a + t2_2.b)))
+                     ->  Seq Scan on public.prt1_l_p2_p1 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c, t2_2.a
+                           ->  Seq Scan on public.prt2_l_p2_p1 t2_2
+                                 Output: t2_2.b, t2_2.c, t2_2.a
+                                 Filter: ((t2_2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_3.b, t2_3.c, t1_3.a, t1_3.c
+                     Hash Cond: ((t1_3.a = t2_3.b) AND (t1_3.b = t2_3.a) AND ((t1_3.c)::text = (t2_3.c)::text) AND ((t1_3.b + t1_3.a) = (t2_3.a + t2_3.b)))
+                     ->  Seq Scan on public.prt1_l_p2_p2 t1_3
+                           Output: t1_3.a, t1_3.c, t1_3.b
+                     ->  Hash
+                           Output: t2_3.b, t2_3.c, t2_3.a
+                           ->  Seq Scan on public.prt2_l_p2_p2 t2_3
+                                 Output: t2_3.b, t2_3.c, t2_3.a
+                                 Filter: ((t2_3.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_4.b, t2_4.c, t1_4.a, t1_4.c
+                     Hash Cond: ((t1_4.a = t2_4.b) AND (t1_4.b = t2_4.a) AND ((t1_4.c)::text = (t2_4.c)::text) AND ((t1_4.b + t1_4.a) = (t2_4.a + t2_4.b)))
+                     ->  Seq Scan on public.prt1_l_p3_p1 t1_4
+                           Output: t1_4.a, t1_4.c, t1_4.b
+                     ->  Hash
+                           Output: t2_4.b, t2_4.c, t2_4.a
+                           ->  Seq Scan on public.prt2_l_p3_p1 t2_4
+                                 Output: t2_4.b, t2_4.c, t2_4.a
+                                 Filter: ((t2_4.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_5.b, t2_5.c, t1_5.a, t1_5.c
+                     Hash Cond: ((t1_5.a = t2_5.b) AND (t1_5.b = t2_5.a) AND ((t1_5.c)::text = (t2_5.c)::text) AND ((t1_5.b + t1_5.a) = (t2_5.a + t2_5.b)))
+                     ->  Seq Scan on public.prt1_l_p3_p2 t1_5
+                           Output: t1_5.a, t1_5.c, t1_5.b
+                     ->  Hash
+                           Output: t2_5.b, t2_5.c, t2_5.a
+                           ->  Seq Scan on public.prt2_l_p3_p2 t2_5
+                                 Output: t2_5.b, t2_5.c, t2_5.a
+                                 Filter: ((t2_5.b % 25) = 0)
+(66 rows)
+
+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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 RIGHT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+                                                                                                              QUERY PLAN                                                                                                              
+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_l_p1_p1.a, prt1_l_p1_p1.c, prt2_l_p1_p1.b, prt2_l_p1_p1.c
+   Sort Key: prt1_l_p1_p1.a, prt2_l_p1_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_l_p1_p1.a, prt1_l_p1_p1.c, prt2_l_p1_p1.b, prt2_l_p1_p1.c
+               Hash Cond: ((prt1_l_p1_p1.a = prt2_l_p1_p1.b) AND (prt1_l_p1_p1.b = prt2_l_p1_p1.a) AND ((prt1_l_p1_p1.c)::text = (prt2_l_p1_p1.c)::text) AND ((prt1_l_p1_p1.b + prt1_l_p1_p1.a) = (prt2_l_p1_p1.a + prt2_l_p1_p1.b)))
+               ->  Seq Scan on public.prt1_l_p1_p1
+                     Output: prt1_l_p1_p1.a, prt1_l_p1_p1.c, prt1_l_p1_p1.b
+                     Filter: ((prt1_l_p1_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p1_p1.b, prt2_l_p1_p1.c, prt2_l_p1_p1.a
+                     ->  Seq Scan on public.prt2_l_p1_p1
+                           Output: prt2_l_p1_p1.b, prt2_l_p1_p1.c, prt2_l_p1_p1.a
+                           Filter: ((prt2_l_p1_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_l_p1_p2.a, prt1_l_p1_p2.c, prt2_l_p1_p2.b, prt2_l_p1_p2.c
+               Hash Cond: ((prt1_l_p1_p2.a = prt2_l_p1_p2.b) AND (prt1_l_p1_p2.b = prt2_l_p1_p2.a) AND ((prt1_l_p1_p2.c)::text = (prt2_l_p1_p2.c)::text) AND ((prt1_l_p1_p2.b + prt1_l_p1_p2.a) = (prt2_l_p1_p2.a + prt2_l_p1_p2.b)))
+               ->  Seq Scan on public.prt1_l_p1_p2
+                     Output: prt1_l_p1_p2.a, prt1_l_p1_p2.c, prt1_l_p1_p2.b
+                     Filter: ((prt1_l_p1_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p1_p2.b, prt2_l_p1_p2.c, prt2_l_p1_p2.a
+                     ->  Seq Scan on public.prt2_l_p1_p2
+                           Output: prt2_l_p1_p2.b, prt2_l_p1_p2.c, prt2_l_p1_p2.a
+                           Filter: ((prt2_l_p1_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_l_p2_p1.a, prt1_l_p2_p1.c, prt2_l_p2_p1.b, prt2_l_p2_p1.c
+               Hash Cond: ((prt1_l_p2_p1.a = prt2_l_p2_p1.b) AND (prt1_l_p2_p1.b = prt2_l_p2_p1.a) AND ((prt1_l_p2_p1.c)::text = (prt2_l_p2_p1.c)::text) AND ((prt1_l_p2_p1.b + prt1_l_p2_p1.a) = (prt2_l_p2_p1.a + prt2_l_p2_p1.b)))
+               ->  Seq Scan on public.prt1_l_p2_p1
+                     Output: prt1_l_p2_p1.a, prt1_l_p2_p1.c, prt1_l_p2_p1.b
+                     Filter: ((prt1_l_p2_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p2_p1.b, prt2_l_p2_p1.c, prt2_l_p2_p1.a
+                     ->  Seq Scan on public.prt2_l_p2_p1
+                           Output: prt2_l_p2_p1.b, prt2_l_p2_p1.c, prt2_l_p2_p1.a
+                           Filter: ((prt2_l_p2_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_l_p2_p2.a, prt1_l_p2_p2.c, prt2_l_p2_p2.b, prt2_l_p2_p2.c
+               Hash Cond: ((prt1_l_p2_p2.a = prt2_l_p2_p2.b) AND (prt1_l_p2_p2.b = prt2_l_p2_p2.a) AND ((prt1_l_p2_p2.c)::text = (prt2_l_p2_p2.c)::text) AND ((prt1_l_p2_p2.b + prt1_l_p2_p2.a) = (prt2_l_p2_p2.a + prt2_l_p2_p2.b)))
+               ->  Seq Scan on public.prt1_l_p2_p2
+                     Output: prt1_l_p2_p2.a, prt1_l_p2_p2.c, prt1_l_p2_p2.b
+                     Filter: ((prt1_l_p2_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p2_p2.b, prt2_l_p2_p2.c, prt2_l_p2_p2.a
+                     ->  Seq Scan on public.prt2_l_p2_p2
+                           Output: prt2_l_p2_p2.b, prt2_l_p2_p2.c, prt2_l_p2_p2.a
+                           Filter: ((prt2_l_p2_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_l_p3_p1.a, prt1_l_p3_p1.c, prt2_l_p3_p1.b, prt2_l_p3_p1.c
+               Hash Cond: ((prt1_l_p3_p1.a = prt2_l_p3_p1.b) AND (prt1_l_p3_p1.b = prt2_l_p3_p1.a) AND ((prt1_l_p3_p1.c)::text = (prt2_l_p3_p1.c)::text) AND ((prt1_l_p3_p1.b + prt1_l_p3_p1.a) = (prt2_l_p3_p1.a + prt2_l_p3_p1.b)))
+               ->  Seq Scan on public.prt1_l_p3_p1
+                     Output: prt1_l_p3_p1.a, prt1_l_p3_p1.c, prt1_l_p3_p1.b
+                     Filter: ((prt1_l_p3_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p3_p1.b, prt2_l_p3_p1.c, prt2_l_p3_p1.a
+                     ->  Seq Scan on public.prt2_l_p3_p1
+                           Output: prt2_l_p3_p1.b, prt2_l_p3_p1.c, prt2_l_p3_p1.a
+                           Filter: ((prt2_l_p3_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_l_p3_p2.a, prt1_l_p3_p2.c, prt2_l_p3_p2.b, prt2_l_p3_p2.c
+               Hash Cond: ((prt1_l_p3_p2.a = prt2_l_p3_p2.b) AND (prt1_l_p3_p2.b = prt2_l_p3_p2.a) AND ((prt1_l_p3_p2.c)::text = (prt2_l_p3_p2.c)::text) AND ((prt1_l_p3_p2.b + prt1_l_p3_p2.a) = (prt2_l_p3_p2.a + prt2_l_p3_p2.b)))
+               ->  Seq Scan on public.prt1_l_p3_p2
+                     Output: prt1_l_p3_p2.a, prt1_l_p3_p2.c, prt1_l_p3_p2.b
+                     Filter: ((prt1_l_p3_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p3_p2.b, prt2_l_p3_p2.c, prt2_l_p3_p2.a
+                     ->  Seq Scan on public.prt2_l_p3_p2
+                           Output: prt2_l_p3_p2.b, prt2_l_p3_p2.c, prt2_l_p3_p2.a
+                           Filter: ((prt2_l_p3_p2.b % 25) = 0)
+(70 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_l t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_l t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.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)
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
new file mode 100644
index 0000000..79779d6
--- /dev/null
+++ b/src/test/regress/expected/partition_join.out
@@ -0,0 +1,4118 @@
+--
+-- PARTITION_JOIN
+-- Test partition-wise join between partitioned tables
+--
+-- Usually partition-wise join paths are chosen when data is large, which would
+-- take regression tests to run longer. So, weigh partition-wise joins cheaper
+-- to force those even for smaller data.
+SET partition_wise_plan_weight to 0.2;
+--
+-- partitioned by a single column
+--
+CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+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, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1 AS SELECT * FROM prt1;
+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_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, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+CREATE TABLE uprt2 AS SELECT * FROM prt2;
+-- inner join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on public.prt2_p1 t2
+                     Output: t2.b, t2.c
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+               Hash Cond: (t2_1.b = t1_2.a)
+               ->  Seq Scan on public.prt2_p2 t2_1
+                     Output: t2_1.b, t2_1.c
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+               Hash Cond: (t2_2.b = t1_1.a)
+               ->  Seq Scan on public.prt2_p3 t2_2
+                     Output: t2_2.b, t2_2.c
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 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)
+
+-- left outer join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on public.prt2_p1 t2
+                     Output: t2.b, t2.c
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+               Hash Cond: (t2_1.b = t1_2.a)
+               ->  Seq Scan on public.prt2_p2 t2_1
+                     Output: t2_1.b, t2_1.c
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+               Hash Cond: (t2_2.b = t1_1.a)
+               ->  Seq Scan on public.prt2_p3 t2_2
+                     Output: t2_2.b, t2_2.c
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+-- right outer join
+EXPLAIN (VERBOSE, 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.b % 25 = 0 ORDER BY t1.a, t2.b;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: (t1.a = t2.b)
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t2.b, t2.c
+                           ->  Seq Scan on public.prt2_p1 t2
+                                 Output: t2.b, t2.c
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Hash Cond: (t1_2.a = t2_1.b)
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c
+                           ->  Seq Scan on public.prt2_p2 t2_1
+                                 Output: t2_1.b, t2_1.c
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Hash Cond: (t1_1.a = t2_2.b)
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c
+                           ->  Seq Scan on public.prt2_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+                                 Filter: ((t2_2.b % 25) = 0)
+(36 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 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)
+
+-- full outer join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+   Sort Key: prt1_p1.a, prt2_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+               Hash Cond: (prt1_p1.a = prt2_p1.b)
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c
+                     Filter: ((prt1_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p1.b, prt2_p1.c
+                     ->  Seq Scan on public.prt2_p1
+                           Output: prt2_p1.b, prt2_p1.c
+                           Filter: ((prt2_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt1_p2.a = prt2_p2.b)
+               ->  Seq Scan on public.prt1_p2
+                     Output: prt1_p2.a, prt1_p2.c
+                     Filter: ((prt1_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p2.b, prt2_p2.c
+                     ->  Seq Scan on public.prt2_p2
+                           Output: prt2_p2.b, prt2_p2.c
+                           Filter: ((prt2_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c
+               Hash Cond: (prt1_p3.a = prt2_p3.b)
+               ->  Seq Scan on public.prt1_p3
+                     Output: prt1_p3.a, prt1_p3.c
+                     Filter: ((prt1_p3.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p3.b, prt2_p3.c
+                     ->  Seq Scan on public.prt2_p3
+                           Output: prt2_p3.b, prt2_p3.c
+                           Filter: ((prt2_p3.b % 25) = 0)
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 t2 WHERE t2.b % 25 = 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)
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+   Sort Key: prt1_p1.a, prt2_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+               Hash Cond: (prt1_p1.a = prt2_p1.b)
+               Filter: (((50) = prt1_p1.b) OR ((75) = prt2_p1.b))
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c, prt1_p1.b, 50
+                     Filter: ((prt1_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p1.b, prt2_p1.c, (75)
+                     ->  Seq Scan on public.prt2_p1
+                           Output: prt2_p1.b, prt2_p1.c, 75
+                           Filter: ((prt2_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt1_p2.a = prt2_p2.b)
+               Filter: (((50) = prt1_p2.b) OR ((75) = prt2_p2.b))
+               ->  Seq Scan on public.prt1_p2
+                     Output: prt1_p2.a, prt1_p2.c, prt1_p2.b, 50
+                     Filter: ((prt1_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p2.b, prt2_p2.c, (75)
+                     ->  Seq Scan on public.prt2_p2
+                           Output: prt2_p2.b, prt2_p2.c, 75
+                           Filter: ((prt2_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c
+               Hash Cond: (prt1_p3.a = prt2_p3.b)
+               Filter: (((50) = prt1_p3.b) OR ((75) = prt2_p3.b))
+               ->  Seq Scan on public.prt1_p3
+                     Output: prt1_p3.a, prt1_p3.c, prt1_p3.b, 50
+                     Filter: ((prt1_p3.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p3.b, prt2_p3.c, (75)
+                     ->  Seq Scan on public.prt2_p3
+                           Output: prt2_p3.b, prt2_p3.c, 75
+                           Filter: ((prt2_p3.b % 25) = 0)
+(40 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+ a  |  c   | b  |  c   
+----+------+----+------
+ 50 | 0050 |    | 
+    |      | 75 | 0075
+(2 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+ a  |  c   | b  |  c   
+----+------+----+------
+ 50 | 0050 |    | 
+    |      | 75 | 0075
+(2 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, (25), prt2_p1.b, prt2_p1.c, (50)
+   Sort Key: prt1_p1.a, prt2_p1.b
+   ->  Result
+         Output: prt1_p1.a, prt1_p1.c, (25), prt2_p1.b, prt2_p1.c, (50)
+         ->  Append
+               ->  Hash Full Join
+                     Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, (25), (50)
+                     Hash Cond: (prt1_p1.a = prt2_p1.b)
+                     ->  Seq Scan on public.prt1_p1
+                           Output: prt1_p1.a, prt1_p1.c, 25
+                           Filter: ((prt1_p1.a % 25) = 0)
+                     ->  Hash
+                           Output: prt2_p1.b, prt2_p1.c, (50)
+                           ->  Seq Scan on public.prt2_p1
+                                 Output: prt2_p1.b, prt2_p1.c, 50
+                                 Filter: ((prt2_p1.b % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c, (25), (50)
+                     Hash Cond: (prt1_p2.a = prt2_p2.b)
+                     ->  Seq Scan on public.prt1_p2
+                           Output: prt1_p2.a, prt1_p2.c, 25
+                           Filter: ((prt1_p2.a % 25) = 0)
+                     ->  Hash
+                           Output: prt2_p2.b, prt2_p2.c, (50)
+                           ->  Seq Scan on public.prt2_p2
+                                 Output: prt2_p2.b, prt2_p2.c, 50
+                                 Filter: ((prt2_p2.b % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c, (25), (50)
+                     Hash Cond: (prt1_p3.a = prt2_p3.b)
+                     ->  Seq Scan on public.prt1_p3
+                           Output: prt1_p3.a, prt1_p3.c, 25
+                           Filter: ((prt1_p3.a % 25) = 0)
+                     ->  Hash
+                           Output: prt2_p3.b, prt2_p3.c, (50)
+                           ->  Seq Scan on public.prt2_p3
+                                 Output: prt2_p3.b, prt2_p3.c, 50
+                                 Filter: ((prt2_p3.b % 25) = 0)
+(39 rows)
+
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 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)
+
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM uprt2 WHERE uprt2.b % 25 = 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)
+
+-- Join with pruned partitions from joining relations
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on public.prt2_p2 t2
+                     Output: t2.b, t2.c
+                     Filter: (t2.b > 250)
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.prt1_p2 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a < 450) AND ((t1.a % 25) = 0))
+(15 rows)
+
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+(1 row)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a < 450 AND t2.b > 250 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+(1 row)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                    QUERY PLAN                                    
+----------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, b, c
+   Sort Key: prt1_p1.a, b
+   ->  Append
+         ->  Nested Loop Left Join
+               Output: prt1_p1.a, prt1_p1.c, b, c
+               Join Filter: (prt1_p1.a = b)
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c
+                     Filter: ((prt1_p1.a < 450) AND ((prt1_p1.a % 25) = 0))
+               ->  Result
+                     Output: b, c
+                     One-Time Filter: false
+         ->  Hash Right Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt2_p2.b = prt1_p2.a)
+               ->  Seq Scan on public.prt2_p2
+                     Output: prt2_p2.b, prt2_p2.c
+                     Filter: (prt2_p2.b > 250)
+               ->  Hash
+                     Output: prt1_p2.a, prt1_p2.c
+                     ->  Seq Scan on public.prt1_p2
+                           Output: prt1_p2.a, prt1_p2.c
+                           Filter: ((prt1_p2.a < 450) AND ((prt1_p2.a % 25) = 0))
+(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.a % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.a % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                       QUERY PLAN                                       
+----------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+   Sort Key: prt1_p2.a, prt2_p2.b
+   ->  Result
+         Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: prt2_p2.b, prt2_p2.c, prt1_p2.a, prt1_p2.c
+                     Hash Cond: (prt1_p2.a = prt2_p2.b)
+                     ->  Seq Scan on public.prt1_p2
+                           Output: prt1_p2.a, prt1_p2.c
+                           Filter: (prt1_p2.a < 450)
+                     ->  Hash
+                           Output: prt2_p2.b, prt2_p2.c
+                           ->  Seq Scan on public.prt2_p2
+                                 Output: prt2_p2.b, prt2_p2.c
+                                 Filter: ((prt2_p2.b > 250) AND ((prt2_p2.a % 25) = 0))
+               ->  Nested Loop Left Join
+                     Output: prt2_p3.b, prt2_p3.c, a, c
+                     Join Filter: (a = prt2_p3.b)
+                     ->  Seq Scan on public.prt2_p3
+                           Output: prt2_p3.b, prt2_p3.c
+                           Filter: ((prt2_p3.b > 250) AND ((prt2_p3.a % 25) = 0))
+                     ->  Result
+                           Output: a, c
+                           One-Time Filter: false
+(26 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+     |      | 375 | 0375
+     |      | 450 | 0450
+     |      | 525 | 0525
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+     |      | 375 | 0375
+     |      | 450 | 0450
+     |      | 525 | 0525
+(4 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+                                    QUERY PLAN                                    
+----------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, b, c
+   Sort Key: prt1_p1.a, b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_p1.a, prt1_p1.c, b, c
+               Hash Cond: (prt1_p1.a = b)
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c
+                     Filter: ((prt1_p1.a < 450) AND ((prt1_p1.a % 25) = 0))
+               ->  Hash
+                     Output: b, c
+                     ->  Result
+                           Output: b, c
+                           One-Time Filter: false
+         ->  Hash Full Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt1_p2.a = prt2_p2.b)
+               ->  Seq Scan on public.prt1_p2
+                     Output: prt1_p2.a, prt1_p2.c
+                     Filter: ((prt1_p2.a < 450) AND ((prt1_p2.a % 25) = 0))
+               ->  Hash
+                     Output: prt2_p2.b, prt2_p2.c
+                     ->  Seq Scan on public.prt2_p2
+                           Output: prt2_p2.b, prt2_p2.c
+                           Filter: ((prt2_p2.b > 250) AND ((prt2_p2.b % 25) = 0))
+         ->  Hash Full Join
+               Output: a, c, prt2_p3.b, prt2_p3.c
+               Hash Cond: (prt2_p3.b = a)
+               ->  Seq Scan on public.prt2_p3
+                     Output: prt2_p3.b, prt2_p3.c
+                     Filter: ((prt2_p3.b > 250) AND ((prt2_p3.b % 25) = 0))
+               ->  Hash
+                     Output: a, c
+                     ->  Result
+                           Output: a, c
+                           One-Time Filter: false
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b 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)
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+                     QUERY PLAN                     
+----------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Output: t1.a, t1.b, t1.c
+         Merge Cond: (t1.a = t1_3.b)
+         ->  Sort
+               Output: t1.a, t1.b, t1.c
+               Sort Key: t1.a
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+         ->  Sort
+               Output: t1_3.b
+               Sort Key: t1_3.b
+               ->  Seq Scan on public.prt2_p1 t1_3
+                     Output: t1_3.b
+                     Filter: ((t1_3.b % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_2.a, t1_2.b, t1_2.c
+         Merge Cond: (t1_2.a = t1_4.b)
+         ->  Sort
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Sort Key: t1_2.a
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+         ->  Sort
+               Output: t1_4.b
+               Sort Key: t1_4.b
+               ->  Seq Scan on public.prt2_p2 t1_4
+                     Output: t1_4.b
+                     Filter: ((t1_4.b % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_1.a, t1_1.b, t1_1.c
+         Merge Cond: (t1_1.a = t1_5.b)
+         ->  Sort
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Sort Key: t1_1.a
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+         ->  Sort
+               Output: t1_5.b
+               Sort Key: t1_5.b
+               ->  Seq Scan on public.prt2_p3 t1_5
+                     Output: t1_5.b
+                     Filter: ((t1_5.b % 25) = 0)
+(47 rows)
+
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+-- lateral reference
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+                                             QUERY PLAN                                              
+-----------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   Sort Key: t1.a
+   ->  Result
+         Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+         ->  Append
+               ->  Nested Loop Left Join
+                     Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a))
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.b, t1.c
+                           Filter: ((t1.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+                           Hash Cond: (t3.b = t2.a)
+                           ->  Seq Scan on public.prt2_p1 t3
+                                 Output: t3.a, t3.b
+                           ->  Hash
+                                 Output: t2.a
+                                 ->  Seq Scan on public.prt1_p1 t2
+                                       Output: t2.a
+                                       Filter: (t1.a = t2.a)
+               ->  Nested Loop Left Join
+                     Output: t1_2.a, t1_2.b, t1_2.c, t2_2.a, t3_1.a, (LEAST(t1_2.a, t2_2.a, t3_1.a))
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.b, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2_2.a, t3_1.a, LEAST(t1_2.a, t2_2.a, t3_1.a)
+                           Hash Cond: (t3_1.b = t2_2.a)
+                           ->  Seq Scan on public.prt2_p2 t3_1
+                                 Output: t3_1.a, t3_1.b
+                           ->  Hash
+                                 Output: t2_2.a
+                                 ->  Seq Scan on public.prt1_p2 t2_2
+                                       Output: t2_2.a
+                                       Filter: (t1_2.a = t2_2.a)
+               ->  Nested Loop Left Join
+                     Output: t1_1.a, t1_1.b, t1_1.c, t2_1.a, t3_2.a, (LEAST(t1_1.a, t2_1.a, t3_2.a))
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.b, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2_1.a, t3_2.a, LEAST(t1_1.a, t2_1.a, t3_2.a)
+                           Hash Cond: (t3_2.b = t2_1.a)
+                           ->  Seq Scan on public.prt2_p3 t3_2
+                                 Output: t3_2.a, t3_2.b
+                           ->  Hash
+                                 Output: t2_1.a
+                                 ->  Seq Scan on public.prt1_p3 t2_1
+                                       Output: t2_1.a
+                                       Filter: (t1_1.a = t2_1.a)
+(51 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   Sort Key: t1.a
+   ->  Nested Loop Left Join
+         Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+         ->  Append
+               ->  Seq Scan on public.prt1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Seq Scan on public.prt1_p1 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Seq Scan on public.prt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Seq Scan on public.prt1_p2 t1_3
+                     Output: t1_3.a, t1_3.b, t1_3.c
+                     Filter: ((t1_3.a % 25) = 0)
+         ->  Append
+               ->  Hash Join
+                     Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+                     Hash Cond: (t3.b = t2.a)
+                     ->  Seq Scan on public.prt2_p1 t3
+                           Output: t3.a, t3.b
+                     ->  Hash
+                           Output: t2.a
+                           ->  Seq Scan on public.prt1_p1 t2
+                                 Output: t2.a
+                                 Filter: (t1.b = t2.a)
+               ->  Hash Join
+                     Output: t2_2.a, t3_1.a, LEAST(t1.a, t2_2.a, t3_1.a)
+                     Hash Cond: (t3_1.b = t2_2.a)
+                     ->  Seq Scan on public.prt2_p2 t3_1
+                           Output: t3_1.a, t3_1.b
+                     ->  Hash
+                           Output: t2_2.a
+                           ->  Seq Scan on public.prt1_p2 t2_2
+                                 Output: t2_2.a
+                                 Filter: (t1.b = t2_2.a)
+               ->  Hash Join
+                     Output: t2_1.a, t3_2.a, LEAST(t1.a, t2_1.a, t3_2.a)
+                     Hash Cond: (t3_2.b = t2_1.a)
+                     ->  Seq Scan on public.prt2_p3 t3_2
+                           Output: t3_2.a, t3_2.b
+                     ->  Hash
+                           Output: t2_1.a
+                           ->  Seq Scan on public.prt1_p3 t2_1
+                                 Output: t2_1.a
+                                 Filter: (t1.b = t2_1.a)
+(49 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+--
+-- partitioned by expression
+--
+CREATE TABLE prt1_e (a int, b int, c varchar) PARTITION BY RANGE(((a + b)/2));
+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);
+INSERT INTO prt1_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_e AS SELECT * FROM prt1_e;
+CREATE TABLE prt2_e (a int, b int, c varchar) PARTITION BY RANGE(((b + a)/2));
+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);
+INSERT INTO prt2_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_e;
+ANALYZE prt2_e_p1;
+ANALYZE prt2_e_p2;
+ANALYZE prt2_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_e AS SELECT * FROM prt2_e;
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                  QUERY PLAN                                  
+------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_e_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_e_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
+               ->  Seq Scan on public.prt2_e_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_e_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+(34 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.a % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1, uprt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.a % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                  QUERY PLAN                                  
+------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_e_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_e_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
+               ->  Seq Scan on public.prt2_e_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_e_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1 LEFT JOIN uprt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on public.prt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = t1.a)
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_2.a)
+                     ->  Seq Scan on public.prt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                           ->  Hash Join
+                                 Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = t1_2.a)
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_1.a)
+                     ->  Seq Scan on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                           ->  Hash Join
+                                 Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = t1_1.a)
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+(57 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM uprt1 t1, uprt2 t2, uprt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on public.prt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = t1.a)
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_2.a)
+                     ->  Seq Scan on public.prt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = t1_2.a)
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_1.a)
+                     ->  Seq Scan on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = t1_1.a)
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+(57 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, 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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: (((t3.a + t3.b) / 2) = t2.b)
+                     ->  Seq Scan on public.prt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = t1.a)
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: (((t3_1.a + t3_1.b) / 2) = t2_1.b)
+                     ->  Seq Scan on public.prt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = t1_2.a)
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: (((t3_2.a + t3_2.b) / 2) = t2_2.b)
+                     ->  Seq Scan on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = t1_1.a)
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+(57 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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |          | 
+ 100 | 0100 |     |      |          | 
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |          | 
+ 250 | 0250 |     |      |          | 
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |          | 
+ 400 | 0400 |     |      |          | 
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |          | 
+ 550 | 0550 |     |      |          | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |          | 
+ 100 | 0100 |     |      |          | 
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |          | 
+ 250 | 0250 |     |      |          | 
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |          | 
+ 400 | 0400 |     |      |          | 
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |          | 
+ 550 | 0550 |     |      |          | 
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Hash Cond: (t2.b = t1.a)
+                     ->  Seq Scan on public.prt2_p1 t2
+                           Output: t2.b, t2.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Hash Cond: (t1.a = ((t3.a + t3.b) / 2))
+                                 ->  Seq Scan on public.prt1_p1 t1
+                                       Output: t1.a, t1.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                     Hash Cond: (t2_1.b = t1_2.a)
+                     ->  Seq Scan on public.prt2_p2 t2_1
+                           Output: t2_1.b, t2_1.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                                 Hash Cond: (t1_2.a = ((t3_1.a + t3_1.b) / 2))
+                                 ->  Seq Scan on public.prt1_p2 t1_2
+                                       Output: t1_2.a, t1_2.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                     Hash Cond: (t2_2.b = t1_1.a)
+                     ->  Seq Scan on public.prt2_p3 t2_2
+                           Output: t2_2.b, t2_2.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                                 Hash Cond: (t1_1.a = ((t3_2.a + t3_2.b) / 2))
+                                 ->  Seq Scan on public.prt1_p3 t1_1
+                                       Output: t1_1.a, t1_1.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: (t1.a = t2.b)
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = ((t3.a + t3.b) / 2))
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Hash Cond: (t1_2.a = t2_1.b)
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = ((t3_1.a + t3_1.b) / 2))
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Hash Cond: (t1_1.a = t2_2.b)
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = ((t3_2.a + t3_2.b) / 2))
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                                  QUERY PLAN                                                   
+---------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, ((prt1_e_p1.a + prt1_e_p1.b)), prt1_e_p1.c
+   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   ->  Result
+         Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, (prt1_e_p1.a + prt1_e_p1.b), prt1_e_p1.c
+         ->  Append
+               ->  Hash Full Join
+                     Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, prt1_e_p1.a, prt1_e_p1.b, prt1_e_p1.c
+                     Hash Cond: (prt1_p1.a = ((prt1_e_p1.a + prt1_e_p1.b) / 2))
+                     ->  Hash Full Join
+                           Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on public.prt1_p1
+                                 Output: prt1_p1.a, prt1_p1.c
+                                 Filter: ((prt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p1.b, prt2_p1.c
+                                 ->  Seq Scan on public.prt2_p1
+                                       Output: prt2_p1.b, prt2_p1.c
+                                       Filter: ((prt2_p1.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p1.a, prt1_e_p1.b, prt1_e_p1.c
+                           ->  Seq Scan on public.prt1_e_p1
+                                 Output: prt1_e_p1.a, prt1_e_p1.b, prt1_e_p1.c
+                                 Filter: ((prt1_e_p1.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c, prt1_e_p2.a, prt1_e_p2.b, prt1_e_p2.c
+                     Hash Cond: (prt1_p2.a = ((prt1_e_p2.a + prt1_e_p2.b) / 2))
+                     ->  Hash Full Join
+                           Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+                           Hash Cond: (prt1_p2.a = prt2_p2.b)
+                           ->  Seq Scan on public.prt1_p2
+                                 Output: prt1_p2.a, prt1_p2.c
+                                 Filter: ((prt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p2.b, prt2_p2.c
+                                 ->  Seq Scan on public.prt2_p2
+                                       Output: prt2_p2.b, prt2_p2.c
+                                       Filter: ((prt2_p2.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p2.a, prt1_e_p2.b, prt1_e_p2.c
+                           ->  Seq Scan on public.prt1_e_p2
+                                 Output: prt1_e_p2.a, prt1_e_p2.b, prt1_e_p2.c
+                                 Filter: ((prt1_e_p2.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c, prt1_e_p3.a, prt1_e_p3.b, prt1_e_p3.c
+                     Hash Cond: (prt1_p3.a = ((prt1_e_p3.a + prt1_e_p3.b) / 2))
+                     ->  Hash Full Join
+                           Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c
+                           Hash Cond: (prt1_p3.a = prt2_p3.b)
+                           ->  Seq Scan on public.prt1_p3
+                                 Output: prt1_p3.a, prt1_p3.c
+                                 Filter: ((prt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p3.b, prt2_p3.c
+                                 ->  Seq Scan on public.prt2_p3
+                                       Output: prt2_p3.b, prt2_p3.c
+                                       Filter: ((prt2_p3.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p3.a, prt1_e_p3.b, prt1_e_p3.c
+                           ->  Seq Scan on public.prt1_e_p3
+                                 Output: prt1_e_p3.a, prt1_e_p3.b, prt1_e_p3.c
+                                 Filter: ((prt1_e_p3.a % 25) = 0)
+(63 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+     |      |  75 | 0075 |          | 
+     |      | 225 | 0225 |          | 
+     |      | 375 | 0375 |          | 
+     |      | 525 | 0525 |          | 
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM uprt1_e WHERE uprt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+     |      |  75 | 0075 |          | 
+     |      | 225 | 0225 |          | 
+     |      | 375 | 0375 |          | 
+     |      | 525 | 0525 |          | 
+(16 rows)
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, 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.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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                                                      
+----------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, (50), prt2_p1.b, (75), ((prt1_e_p1.a + prt1_e_p1.b)), (50)
+   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   ->  Result
+         Output: prt1_p1.a, (50), prt2_p1.b, (75), (prt1_e_p1.a + prt1_e_p1.b), (50)
+         ->  Append
+               ->  Hash Full Join
+                     Output: prt1_p1.a, prt2_p1.b, prt1_e_p1.a, prt1_e_p1.b, (50), (75), (50)
+                     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
+                           Output: prt1_p1.a, prt2_p1.b, (50), (75)
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on public.prt1_p1
+                                 Output: prt1_p1.a, 50
+                                 Filter: ((prt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p1.b, (75)
+                                 ->  Seq Scan on public.prt2_p1
+                                       Output: prt2_p1.b, 75
+                                       Filter: ((prt2_p1.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p1.a, prt1_e_p1.b, (50)
+                           ->  Seq Scan on public.prt1_e_p1
+                                 Output: prt1_e_p1.a, prt1_e_p1.b, 50
+                                 Filter: ((prt1_e_p1.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p2.a, prt2_p2.b, prt1_e_p2.a, prt1_e_p2.b, (50), (75), (50)
+                     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
+                           Output: prt1_p2.a, prt2_p2.b, (50), (75)
+                           Hash Cond: (prt1_p2.a = prt2_p2.b)
+                           ->  Seq Scan on public.prt1_p2
+                                 Output: prt1_p2.a, 50
+                                 Filter: ((prt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p2.b, (75)
+                                 ->  Seq Scan on public.prt2_p2
+                                       Output: prt2_p2.b, 75
+                                       Filter: ((prt2_p2.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p2.a, prt1_e_p2.b, (50)
+                           ->  Seq Scan on public.prt1_e_p2
+                                 Output: prt1_e_p2.a, prt1_e_p2.b, 50
+                                 Filter: ((prt1_e_p2.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p3.a, prt2_p3.b, prt1_e_p3.a, prt1_e_p3.b, (50), (75), (50)
+                     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
+                           Output: prt1_p3.a, prt2_p3.b, (50), (75)
+                           Hash Cond: (prt1_p3.a = prt2_p3.b)
+                           ->  Seq Scan on public.prt1_p3
+                                 Output: prt1_p3.a, 50
+                                 Filter: ((prt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p3.b, (75)
+                                 ->  Seq Scan on public.prt2_p3
+                                       Output: prt2_p3.b, 75
+                                       Filter: ((prt2_p3.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p3.a, prt1_e_p3.b, (50)
+                           ->  Seq Scan on public.prt1_e_p3
+                                 Output: prt1_e_p3.a, prt1_e_p3.b, 50
+                                 Filter: ((prt1_e_p3.a % 25) = 0)
+(66 rows)
+
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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)
+
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM uprt1_e WHERE uprt1_e.a % 25 = 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)
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+                            QUERY PLAN                             
+-------------------------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Output: t1.a, t1.b, t1.c
+         Merge Cond: (t1.a = t1_3.b)
+         ->  Sort
+               Output: t1.a, t1.b, t1.c
+               Sort Key: t1.a
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+         ->  Sort
+               Output: t1_3.b, t2.a, t2.b
+               Sort Key: t1_3.b
+               ->  Hash Join
+                     Output: t1_3.b, t2.a, t2.b
+                     Hash Cond: (((t2.a + t2.b) / 2) = t1_3.b)
+                     ->  Seq Scan on public.prt1_e_p1 t2
+                           Output: t2.a, t2.b
+                     ->  Hash
+                           Output: t1_3.b
+                           ->  Seq Scan on public.prt2_p1 t1_3
+                                 Output: t1_3.b
+                                 Filter: ((t1_3.b % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_2.a, t1_2.b, t1_2.c
+         Merge Cond: (t1_2.a = t1_4.b)
+         ->  Sort
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Sort Key: t1_2.a
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+         ->  Sort
+               Output: t1_4.b, t2_1.a, t2_1.b
+               Sort Key: t1_4.b
+               ->  Hash Join
+                     Output: t1_4.b, t2_1.a, t2_1.b
+                     Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_4.b)
+                     ->  Seq Scan on public.prt1_e_p2 t2_1
+                           Output: t2_1.a, t2_1.b
+                     ->  Hash
+                           Output: t1_4.b
+                           ->  Seq Scan on public.prt2_p2 t1_4
+                                 Output: t1_4.b
+                                 Filter: ((t1_4.b % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_1.a, t1_1.b, t1_1.c
+         Merge Cond: (t1_1.a = t1_5.b)
+         ->  Sort
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Sort Key: t1_1.a
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+         ->  Sort
+               Output: t1_5.b, t2_2.a, t2_2.b
+               Sort Key: t1_5.b
+               ->  Hash Join
+                     Output: t1_5.b, t2_2.a, t2_2.b
+                     Hash Cond: (((t2_2.a + t2_2.b) / 2) = t1_5.b)
+                     ->  Seq Scan on public.prt1_e_p3 t2_2
+                           Output: t2_2.a, t2_2.b
+                     ->  Hash
+                           Output: t1_5.b
+                           ->  Seq Scan on public.prt2_p3 t1_5
+                                 Output: t1_5.b
+                                 Filter: ((t1_5.b % 25) = 0)
+(68 rows)
+
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1, uprt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+                            QUERY PLAN                             
+-------------------------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Output: t1.a, t1.b, t1.c
+         Merge Cond: (t1.a = t1_3.b)
+         ->  Sort
+               Output: t1.a, t1.b, t1.c
+               Sort Key: t1.a
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+         ->  Sort
+               Output: t1_3.b, t1_6.a, t1_6.b
+               Sort Key: t1_3.b
+               ->  Hash Semi Join
+                     Output: t1_3.b, t1_6.a, t1_6.b
+                     Hash Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
+                     ->  Seq Scan on public.prt2_p1 t1_3
+                           Output: t1_3.b
+                     ->  Hash
+                           Output: t1_6.a, t1_6.b
+                           ->  Seq Scan on public.prt1_e_p1 t1_6
+                                 Output: t1_6.a, t1_6.b
+                                 Filter: ((t1_6.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_2.a, t1_2.b, t1_2.c
+         Merge Cond: (t1_2.a = t1_4.b)
+         ->  Sort
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Sort Key: t1_2.a
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+         ->  Sort
+               Output: t1_4.b, t1_7.a, t1_7.b
+               Sort Key: t1_4.b
+               ->  Hash Semi Join
+                     Output: t1_4.b, t1_7.a, t1_7.b
+                     Hash Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
+                     ->  Seq Scan on public.prt2_p2 t1_4
+                           Output: t1_4.b
+                     ->  Hash
+                           Output: t1_7.a, t1_7.b
+                           ->  Seq Scan on public.prt1_e_p2 t1_7
+                                 Output: t1_7.a, t1_7.b
+                                 Filter: ((t1_7.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_1.a, t1_1.b, t1_1.c
+         Merge Cond: (t1_1.a = t1_5.b)
+         ->  Sort
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Sort Key: t1_1.a
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+         ->  Sort
+               Output: t1_5.b, t1_8.a, t1_8.b
+               Sort Key: t1_5.b
+               ->  Hash Semi Join
+                     Output: t1_5.b, t1_8.a, t1_8.b
+                     Hash Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
+                     ->  Seq Scan on public.prt2_p3 t1_5
+                           Output: t1_5.b
+                     ->  Hash
+                           Output: t1_8.a, t1_8.b
+                           ->  Seq Scan on public.prt1_e_p3 t1_8
+                                 Output: t1_8.a, t1_8.b
+                                 Filter: ((t1_8.a % 25) = 0)
+(68 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+-- test merge joins with and without using indexes
+SET enable_hashjoin TO off;
+SET enable_nestloop TO off;
+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 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 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));
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                             QUERY PLAN                                              
+-----------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Merge Cond: (t2.b = t1.a)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           Sort Key: t2.b
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Merge Cond: ((((t3.a + t3.b) / 2)) = t2.b)
+                                 ->  Sort
+                                       Output: t3.a, t3.b, t3.c, (((t3.a + t3.b) / 2))
+                                       Sort Key: (((t3.a + t3.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c, ((t3.a + t3.b) / 2)
+                                             Filter: ((t3.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2.b, t2.c
+                                       Sort Key: t2.b
+                                       ->  Seq Scan on public.prt2_p1 t2
+                                             Output: t2.b, t2.c
+                     ->  Sort
+                           Output: t1.a, t1.c
+                           Sort Key: t1.a
+                           ->  Seq Scan on public.prt1_p1 t1
+                                 Output: t1.a, t1.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Merge Cond: (t2_1.b = t1_2.a)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           Sort Key: t2_1.b
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t2_1.b)
+                                 ->  Sort
+                                       Output: t3_1.a, t3_1.b, t3_1.c, (((t3_1.a + t3_1.b) / 2))
+                                       Sort Key: (((t3_1.a + t3_1.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c, ((t3_1.a + t3_1.b) / 2)
+                                             Filter: ((t3_1.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2_1.b, t2_1.c
+                                       Sort Key: t2_1.b
+                                       ->  Seq Scan on public.prt2_p2 t2_1
+                                             Output: t2_1.b, t2_1.c
+                     ->  Sort
+                           Output: t1_2.a, t1_2.c
+                           Sort Key: t1_2.a
+                           ->  Seq Scan on public.prt1_p2 t1_2
+                                 Output: t1_2.a, t1_2.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Merge Cond: (t2_2.b = t1_1.a)
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           Sort Key: t2_2.b
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t2_2.b)
+                                 ->  Sort
+                                       Output: t3_2.a, t3_2.b, t3_2.c, (((t3_2.a + t3_2.b) / 2))
+                                       Sort Key: (((t3_2.a + t3_2.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c, ((t3_2.a + t3_2.b) / 2)
+                                             Filter: ((t3_2.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2_2.b, t2_2.c
+                                       Sort Key: t2_2.b
+                                       ->  Seq Scan on public.prt2_p3 t2_2
+                                             Output: t2_2.b, t2_2.c
+                     ->  Sort
+                           Output: t1_1.a, t1_1.c
+                           Sort Key: t1_1.a
+                           ->  Seq Scan on public.prt1_p3 t1_1
+                                 Output: t1_1.a, t1_1.c
+(81 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+                                   QUERY PLAN                                    
+---------------------------------------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Output: t1.a, t1.b, t1.c
+         Merge Cond: (t1.a = t1_3.b)
+         ->  Sort
+               Output: t1.a, t1.b, t1.c
+               Sort Key: t1.a
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_3.b, t1_6.a, t1_6.b
+               ->  Merge Semi Join
+                     Output: t1_3.b, t1_6.a, t1_6.b
+                     Merge Cond: (t1_3.b = (((t1_6.a + t1_6.b) / 2)))
+                     ->  Sort
+                           Output: t1_3.b
+                           Sort Key: t1_3.b
+                           ->  Seq Scan on public.prt2_p1 t1_3
+                                 Output: t1_3.b
+                     ->  Sort
+                           Output: t1_6.a, t1_6.b, (((t1_6.a + t1_6.b) / 2))
+                           Sort Key: (((t1_6.a + t1_6.b) / 2))
+                           ->  Seq Scan on public.prt1_e_p1 t1_6
+                                 Output: t1_6.a, t1_6.b, ((t1_6.a + t1_6.b) / 2)
+                                 Filter: ((t1_6.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_2.a, t1_2.b, t1_2.c
+         Merge Cond: (t1_2.a = t1_4.b)
+         ->  Sort
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Sort Key: t1_2.a
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+         ->  Materialize
+               Output: t1_4.b, t1_7.a, t1_7.b
+               ->  Merge Semi Join
+                     Output: t1_4.b, t1_7.a, t1_7.b
+                     Merge Cond: (t1_4.b = (((t1_7.a + t1_7.b) / 2)))
+                     ->  Sort
+                           Output: t1_4.b
+                           Sort Key: t1_4.b
+                           ->  Seq Scan on public.prt2_p2 t1_4
+                                 Output: t1_4.b
+                     ->  Sort
+                           Output: t1_7.a, t1_7.b, (((t1_7.a + t1_7.b) / 2))
+                           Sort Key: (((t1_7.a + t1_7.b) / 2))
+                           ->  Seq Scan on public.prt1_e_p2 t1_7
+                                 Output: t1_7.a, t1_7.b, ((t1_7.a + t1_7.b) / 2)
+                                 Filter: ((t1_7.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_1.a, t1_1.b, t1_1.c
+         Merge Cond: (t1_1.a = t1_5.b)
+         ->  Sort
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Sort Key: t1_1.a
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_5.b, t1_8.a, t1_8.b
+               ->  Merge Semi Join
+                     Output: t1_5.b, t1_8.a, t1_8.b
+                     Merge Cond: (t1_5.b = (((t1_8.a + t1_8.b) / 2)))
+                     ->  Sort
+                           Output: t1_5.b
+                           Sort Key: t1_5.b
+                           ->  Seq Scan on public.prt2_p3 t1_5
+                                 Output: t1_5.b
+                     ->  Sort
+                           Output: t1_8.a, t1_8.b, (((t1_8.a + t1_8.b) / 2))
+                           Sort Key: (((t1_8.a + t1_8.b) / 2))
+                           ->  Seq Scan on public.prt1_e_p3 t1_8
+                                 Output: t1_8.a, t1_8.b, ((t1_8.a + t1_8.b) / 2)
+                                 Filter: ((t1_8.a % 25) = 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                             QUERY PLAN                                              
+-----------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Merge Cond: (t1.a = t2.b)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           Sort Key: t1.a
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Merge Cond: ((((t3.a + t3.b) / 2)) = t1.a)
+                                 ->  Sort
+                                       Output: t3.a, t3.b, t3.c, (((t3.a + t3.b) / 2))
+                                       Sort Key: (((t3.a + t3.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c, ((t3.a + t3.b) / 2)
+                                             Filter: ((t3.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t1.a, t1.c
+                                       Sort Key: t1.a
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                     ->  Sort
+                           Output: t2.b, t2.c
+                           Sort Key: t2.b
+                           ->  Seq Scan on public.prt2_p1 t2
+                                 Output: t2.b, t2.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                     Merge Cond: (t1_2.a = t2_1.b)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                           Sort Key: t1_2.a
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                                 Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t1_2.a)
+                                 ->  Sort
+                                       Output: t3_1.a, t3_1.b, t3_1.c, (((t3_1.a + t3_1.b) / 2))
+                                       Sort Key: (((t3_1.a + t3_1.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c, ((t3_1.a + t3_1.b) / 2)
+                                             Filter: ((t3_1.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t1_2.a, t1_2.c
+                                       Sort Key: t1_2.a
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                     ->  Sort
+                           Output: t2_1.b, t2_1.c
+                           Sort Key: t2_1.b
+                           ->  Seq Scan on public.prt2_p2 t2_1
+                                 Output: t2_1.b, t2_1.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                     Merge Cond: (t1_1.a = t2_2.b)
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                           Sort Key: t1_1.a
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                                 Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t1_1.a)
+                                 ->  Sort
+                                       Output: t3_2.a, t3_2.b, t3_2.c, (((t3_2.a + t3_2.b) / 2))
+                                       Sort Key: (((t3_2.a + t3_2.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c, ((t3_2.a + t3_2.b) / 2)
+                                             Filter: ((t3_2.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t1_1.a, t1_1.c
+                                       Sort Key: t1_1.a
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                     ->  Sort
+                           Output: t2_2.b, t2_2.c
+                           Sort Key: t2_2.b
+                           ->  Seq Scan on public.prt2_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+(81 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SET enable_seqscan TO off;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                          QUERY PLAN                                          
+----------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Merge Cond: (t2.b = t1.a)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           Sort Key: t2.b
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Merge Cond: (((t3.a + t3.b) / 2) = t2.b)
+                                 ->  Index Scan using iprt1_e_p1_ab2 on public.prt1_e_p1 t3
+                                       Output: t3.a, t3.b, t3.c
+                                       Filter: ((t3.a % 25) = 0)
+                                 ->  Index Scan using iprt2_p1_b on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                     ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Merge Cond: (t2_1.b = t1_2.a)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           Sort Key: t2_1.b
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Merge Cond: (((t3_1.a + t3_1.b) / 2) = t2_1.b)
+                                 ->  Index Scan using iprt1_e_p2_ab2 on public.prt1_e_p2 t3_1
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       Filter: ((t3_1.a % 25) = 0)
+                                 ->  Index Scan using iprt2_p2_b on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                     ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+               ->  Merge Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Merge Cond: (t2_2.b = ((t3_2.a + t3_2.b) / 2))
+                     ->  Merge Left Join
+                           Output: t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                           Merge Cond: (t2_2.b = t1_1.a)
+                           ->  Index Scan using iprt2_p3_b on public.prt2_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+                           ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_1
+                                 Output: t1_1.a, t1_1.c
+                     ->  Index Scan using iprt1_e_p3_ab2 on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                           Filter: ((t3_2.a % 25) = 0)
+(51 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+                                    QUERY PLAN                                    
+----------------------------------------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Output: t1.a, t1.b, t1.c
+         Merge Cond: (t1.a = t1_3.b)
+         ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1
+               Output: t1.a, t1.b, t1.c
+               Filter: ((t1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_3.b, t1_6.a, t1_6.b
+               ->  Merge Semi Join
+                     Output: t1_3.b, t1_6.a, t1_6.b
+                     Merge Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
+                     ->  Index Only Scan using iprt2_p1_b on public.prt2_p1 t1_3
+                           Output: t1_3.b
+                     ->  Index Scan using iprt1_e_p1_ab2 on public.prt1_e_p1 t1_6
+                           Output: t1_6.a, t1_6.b
+                           Filter: ((t1_6.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_2.a, t1_2.b, t1_2.c
+         Merge Cond: (t1_2.a = t1_4.b)
+         ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_2
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Filter: ((t1_2.a % 25) = 0)
+         ->  Materialize
+               Output: t1_4.b, t1_7.a, t1_7.b
+               ->  Merge Semi Join
+                     Output: t1_4.b, t1_7.a, t1_7.b
+                     Merge Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
+                     ->  Index Only Scan using iprt2_p2_b on public.prt2_p2 t1_4
+                           Output: t1_4.b
+                     ->  Index Scan using iprt1_e_p2_ab2 on public.prt1_e_p2 t1_7
+                           Output: t1_7.a, t1_7.b
+                           Filter: ((t1_7.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_1.a, t1_1.b, t1_1.c
+         Merge Cond: (t1_1.a = t1_5.b)
+         ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_1
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Filter: ((t1_1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_5.b, t1_8.a, t1_8.b
+               ->  Merge Semi Join
+                     Output: t1_5.b, t1_8.a, t1_8.b
+                     Merge Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
+                     ->  Index Only Scan using iprt2_p3_b on public.prt2_p3 t1_5
+                           Output: t1_5.b
+                     ->  Index Scan using iprt1_e_p3_ab2 on public.prt1_e_p3 t1_8
+                           Output: t1_8.a, t1_8.b
+                           Filter: ((t1_8.a % 25) = 0)
+(50 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                          QUERY PLAN                                          
+----------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Merge Cond: (t1.a = t2.b)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           Sort Key: t1.a
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Merge Cond: (((t3.a + t3.b) / 2) = t1.a)
+                                 ->  Index Scan using iprt1_e_p1_ab2 on public.prt1_e_p1 t3
+                                       Output: t3.a, t3.b, t3.c
+                                       Filter: ((t3.a % 25) = 0)
+                                 ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1
+                                       Output: t1.a, t1.c
+                     ->  Index Scan using iprt2_p1_b on public.prt2_p1 t2
+                           Output: t2.b, t2.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                     Merge Cond: (t1_2.a = t2_1.b)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                           Sort Key: t1_2.a
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                                 Merge Cond: (((t3_1.a + t3_1.b) / 2) = t1_2.a)
+                                 ->  Index Scan using iprt1_e_p2_ab2 on public.prt1_e_p2 t3_1
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       Filter: ((t3_1.a % 25) = 0)
+                                 ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_2
+                                       Output: t1_2.a, t1_2.c
+                     ->  Index Scan using iprt2_p2_b on public.prt2_p2 t2_1
+                           Output: t2_1.b, t2_1.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                     Merge Cond: (t1_1.a = t2_2.b)
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                           Sort Key: t1_1.a
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                                 Merge Cond: (((t3_2.a + t3_2.b) / 2) = t1_1.a)
+                                 ->  Index Scan using iprt1_e_p3_ab2 on public.prt1_e_p3 t3_2
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       Filter: ((t3_2.a % 25) = 0)
+                                 ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_1
+                                       Output: t1_1.a, t1_1.c
+                     ->  Index Scan using iprt2_p3_b on public.prt2_p3 t2_2
+                           Output: t2_2.b, t2_2.c
+(54 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+-- lateral references and parameterized paths
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Result
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   ->  Merge Append
+         Sort Key: t1.a
+         ->  Nested Loop Left Join
+               Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a))
+               ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Merge Join
+                     Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+                     Merge Cond: (t2.a = t3.b)
+                     ->  Index Only Scan using iprt1_p1_a on public.prt1_p1 t2
+                           Output: t2.a
+                           Index Cond: (t2.a = t1.a)
+                     ->  Index Scan using iprt2_p1_b on public.prt2_p1 t3
+                           Output: t3.a, t3.b
+         ->  Nested Loop Left Join
+               Output: t1_2.a, t1_2.b, t1_2.c, t2_2.a, t3_1.a, (LEAST(t1_2.a, t2_2.a, t3_1.a))
+               ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Merge Join
+                     Output: t2_2.a, t3_1.a, LEAST(t1_2.a, t2_2.a, t3_1.a)
+                     Merge Cond: (t2_2.a = t3_1.b)
+                     ->  Index Only Scan using iprt1_p2_a on public.prt1_p2 t2_2
+                           Output: t2_2.a
+                           Index Cond: (t2_2.a = t1_2.a)
+                     ->  Index Scan using iprt2_p2_b on public.prt2_p2 t3_1
+                           Output: t3_1.a, t3_1.b
+         ->  Nested Loop Left Join
+               Output: t1_1.a, t1_1.b, t1_1.c, t2_1.a, t3_2.a, (LEAST(t1_1.a, t2_1.a, t3_2.a))
+               ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Merge Join
+                     Output: t2_1.a, t3_2.a, LEAST(t1_1.a, t2_1.a, t3_2.a)
+                     Merge Cond: (t2_1.a = t3_2.b)
+                     ->  Index Only Scan using iprt1_p3_a on public.prt1_p3 t2_1
+                           Output: t2_1.a
+                           Index Cond: (t2_1.a = t1_1.a)
+                     ->  Index Scan using iprt2_p3_b on public.prt2_p3 t3_2
+                           Output: t3_2.a, t3_2.b
+(43 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
+ Nested Loop Left Join
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   ->  Merge Append
+         Sort Key: t1.a
+         ->  Sort
+               Output: t1.a, t1.b, t1.c
+               Sort Key: t1.a
+               ->  Seq Scan on public.prt1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+         ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1_1
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Filter: ((t1_1.a % 25) = 0)
+         ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_2
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Filter: ((t1_2.a % 25) = 0)
+         ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_3
+               Output: t1_3.a, t1_3.b, t1_3.c
+               Filter: ((t1_3.a % 25) = 0)
+   ->  Append
+         ->  Merge Join
+               Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+               Merge Cond: (t2.a = t3.b)
+               ->  Index Only Scan using iprt1_p1_a on public.prt1_p1 t2
+                     Output: t2.a
+                     Index Cond: (t2.a = t1.b)
+               ->  Index Scan using iprt2_p1_b on public.prt2_p1 t3
+                     Output: t3.a, t3.b
+         ->  Merge Join
+               Output: t2_2.a, t3_1.a, LEAST(t1.a, t2_2.a, t3_1.a)
+               Merge Cond: (t2_2.a = t3_1.b)
+               ->  Index Only Scan using iprt1_p2_a on public.prt1_p2 t2_2
+                     Output: t2_2.a
+                     Index Cond: (t2_2.a = t1.b)
+               ->  Index Scan using iprt2_p2_b on public.prt2_p2 t3_1
+                     Output: t3_1.a, t3_1.b
+         ->  Merge Join
+               Output: t2_1.a, t3_2.a, LEAST(t1.a, t2_1.a, t3_2.a)
+               Merge Cond: (t2_1.a = t3_2.b)
+               ->  Index Only Scan using iprt1_p3_a on public.prt1_p3 t2_1
+                     Output: t2_1.a
+                     Index Cond: (t2_1.a = t1.b)
+               ->  Index Scan using iprt2_p3_b on public.prt2_p3 t3_2
+                     Output: t3_2.a, t3_2.b
+(44 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+RESET enable_hashjoin;
+RESET enable_nestloop;
+RESET enable_seqscan;
+--
+-- partitioned by multiple columns
+--
+CREATE TABLE prt1_m (a int, b int, c varchar) 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, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_m;
+ANALYZE prt1_m_p1;
+ANALYZE prt1_m_p2;
+ANALYZE prt1_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_m AS SELECT * FROM prt1_m;
+CREATE TABLE prt2_m (a int, b int, c varchar) 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, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_m;
+ANALYZE prt2_m_p1;
+ANALYZE prt2_m_p2;
+ANALYZE prt2_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_m AS SELECT * FROM prt2_m;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+                                                 QUERY PLAN                                                 
+------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: ((((t1.a + t1.b) / 2) = t2.b) AND (t1.a = ((t2.b + t2.a) / 2)))
+                     ->  Seq Scan on public.prt1_m_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                     ->  Hash
+                           Output: t2.b, t2.c, t2.a
+                           ->  Seq Scan on public.prt2_m_p1 t2
+                                 Output: t2.b, t2.c, t2.a
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: ((((t1_1.a + t1_1.b) / 2) = t2_1.b) AND (t1_1.a = ((t2_1.b + t2_1.a) / 2)))
+                     ->  Seq Scan on public.prt1_m_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c, t2_1.a
+                           ->  Seq Scan on public.prt2_m_p2 t2_1
+                                 Output: t2_1.b, t2_1.c, t2_1.a
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: ((((t1_2.a + t1_2.b) / 2) = t2_2.b) AND (t1_2.a = ((t2_2.b + t2_2.a) / 2)))
+                     ->  Seq Scan on public.prt1_m_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c, t2_2.a
+                           ->  Seq Scan on public.prt2_m_p3 t2_2
+                                 Output: t2_2.b, t2_2.c, t2_2.a
+                                 Filter: ((t2_2.b % 25) = 0)
+(36 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_m t1 RIGHT JOIN uprt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 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
+   Output: prt1_m_p1.a, prt1_m_p1.c, prt2_m_p1.b, prt2_m_p1.c
+   Sort Key: prt1_m_p1.a, prt2_m_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_m_p1.a, prt1_m_p1.c, prt2_m_p1.b, prt2_m_p1.c
+               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 public.prt1_m_p1
+                     Output: prt1_m_p1.a, prt1_m_p1.c, prt1_m_p1.b
+                     Filter: ((prt1_m_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_m_p1.b, prt2_m_p1.c, prt2_m_p1.a
+                     ->  Seq Scan on public.prt2_m_p1
+                           Output: prt2_m_p1.b, prt2_m_p1.c, prt2_m_p1.a
+                           Filter: ((prt2_m_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_m_p2.a, prt1_m_p2.c, prt2_m_p2.b, prt2_m_p2.c
+               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 public.prt1_m_p2
+                     Output: prt1_m_p2.a, prt1_m_p2.c, prt1_m_p2.b
+                     Filter: ((prt1_m_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_m_p2.b, prt2_m_p2.c, prt2_m_p2.a
+                     ->  Seq Scan on public.prt2_m_p2
+                           Output: prt2_m_p2.b, prt2_m_p2.c, prt2_m_p2.a
+                           Filter: ((prt2_m_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_m_p3.a, prt1_m_p3.c, prt2_m_p3.b, prt2_m_p3.c
+               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 public.prt1_m_p3
+                     Output: prt1_m_p3.a, prt1_m_p3.c, prt1_m_p3.b
+                     Filter: ((prt1_m_p3.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_m_p3.b, prt2_m_p3.c, prt2_m_p3.a
+                     ->  Seq Scan on public.prt2_m_p3
+                           Output: prt2_m_p3.b, prt2_m_p3.c, prt2_m_p3.a
+                           Filter: ((prt2_m_p3.b % 25) = 0)
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 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 | 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_m t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_m t2 WHERE t2.b % 25 = 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 | 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)
+
+--
+-- 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;
+ANALYZE plt1_p1;
+ANALYZE plt1_p2;
+ANALYZE plt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1 AS SELECT * FROM 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;
+ANALYZE plt2_p1;
+ANALYZE plt2_p2;
+ANALYZE plt2_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt2 AS SELECT * FROM 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;
+ANALYZE plt1_e_p1;
+ANALYZE plt1_e_p2;
+ANALYZE plt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1_e AS SELECT * FROM plt1_e;
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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
+   Output: (avg(t1.a)), (avg(t2.b)), (avg((t3.a + t3.b))), t1.c, t2.c, t3.c
+   Sort Key: t1.c, t3.c
+   ->  HashAggregate
+         Output: avg(t1.a), avg(t2.b), avg((t3.a + t3.b)), t1.c, t2.c, t3.c
+         Group Key: t1.c, t2.c, t3.c
+         ->  Result
+               Output: t1.c, t2.c, t3.c, t1.a, t2.b, t3.a, t3.b
+               ->  Append
+                     ->  Hash Join
+                           Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                           Hash Cond: (t1.c = t2.c)
+                           ->  Seq Scan on public.plt1_p1 t1
+                                 Output: t1.a, t1.c
+                           ->  Hash
+                                 Output: t2.b, t2.c, t3.a, t3.b, t3.c
+                                 ->  Hash Join
+                                       Output: t2.b, t2.c, t3.a, t3.b, t3.c
+                                       Hash Cond: (t2.c = ltrim(t3.c, 'A'::text))
+                                       ->  Seq Scan on public.plt2_p1 t2
+                                             Output: t2.b, t2.c
+                                       ->  Hash
+                                             Output: t3.a, t3.b, t3.c
+                                             ->  Seq Scan on public.plt1_e_p1 t3
+                                                   Output: t3.a, t3.b, t3.c
+                     ->  Hash Join
+                           Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                           Hash Cond: (t1_1.c = t2_1.c)
+                           ->  Seq Scan on public.plt1_p2 t1_1
+                                 Output: t1_1.a, t1_1.c
+                           ->  Hash
+                                 Output: t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                                 ->  Hash Join
+                                       Output: t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                                       Hash Cond: (t2_1.c = ltrim(t3_1.c, 'A'::text))
+                                       ->  Seq Scan on public.plt2_p2 t2_1
+                                             Output: t2_1.b, t2_1.c
+                                       ->  Hash
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             ->  Seq Scan on public.plt1_e_p2 t3_1
+                                                   Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash Join
+                           Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                           Hash Cond: (t1_2.c = t2_2.c)
+                           ->  Seq Scan on public.plt1_p3 t1_2
+                                 Output: t1_2.a, t1_2.c
+                           ->  Hash
+                                 Output: t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                                 ->  Hash Join
+                                       Output: t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                                       Hash Cond: (t2_2.c = ltrim(t3_2.c, 'A'::text))
+                                       ->  Seq Scan on public.plt2_p3 t2_2
+                                             Output: t2_2.b, t2_2.c
+                                       ->  Hash
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             ->  Seq Scan on public.plt1_e_p3 t3_2
+                                                   Output: t3_2.a, t3_2.b, t3_2.c
+(57 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)
+
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM uplt1 t1, uplt2 t2, uplt1_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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: ((t3.a = t1.a) AND (ltrim(t3.c, 'A'::text) = t1.c))
+                     ->  Seq Scan on public.plt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                                 ->  Seq Scan on public.plt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.plt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: ((t3_1.a = t1_1.a) AND (ltrim(t3_1.c, 'A'::text) = t1_1.c))
+                     ->  Seq Scan on public.plt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.c = t1_1.c))
+                                 ->  Seq Scan on public.plt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.plt1_p2 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: ((t3_2.a = t1_2.a) AND (ltrim(t3_2.c, 'A'::text) = t1_2.c))
+                     ->  Seq Scan on public.plt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                                 ->  Seq Scan on public.plt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.plt1_p3 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: ((t3.a = t2.b) AND (ltrim(t3.c, 'A'::text) = t2.c))
+                     ->  Seq Scan on public.plt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                                 ->  Seq Scan on public.plt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.plt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: ((t3_1.a = t2_1.b) AND (ltrim(t3_1.c, 'A'::text) = t2_1.c))
+                     ->  Seq Scan on public.plt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.c = t1_1.c))
+                                 ->  Seq Scan on public.plt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.plt1_p2 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: ((t3_2.a = t2_2.b) AND (ltrim(t3_2.c, 'A'::text) = t2_2.c))
+                     ->  Seq Scan on public.plt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                                 ->  Seq Scan on public.plt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.plt1_p3 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |          | 
+ 100 | 0002 |     |      |          | 
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |          | 
+ 250 | 0005 |     |      |          | 
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |          | 
+ 400 | 0008 |     |      |          | 
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |          | 
+ 550 | 0011 |     |      |          | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |          | 
+ 100 | 0002 |     |      |          | 
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |          | 
+ 250 | 0005 |     |      |          | 
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |          | 
+ 400 | 0008 |     |      |          | 
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |          | 
+ 550 | 0011 |     |      |          | 
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Hash Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                     ->  Seq Scan on public.plt2_p1 t2
+                           Output: t2.b, t2.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Hash Cond: ((t1.c = ltrim(t3.c, 'A'::text)) AND (t1.a = t3.a))
+                                 ->  Seq Scan on public.plt1_p1 t1
+                                       Output: t1.a, t1.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.plt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                     Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.c = t1_1.c))
+                     ->  Seq Scan on public.plt2_p2 t2_1
+                           Output: t2_1.b, t2_1.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_1.a, t1_1.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_1.a, t1_1.c
+                                 Hash Cond: ((t1_1.c = ltrim(t3_1.c, 'A'::text)) AND (t1_1.a = t3_1.a))
+                                 ->  Seq Scan on public.plt1_p2 t1_1
+                                       Output: t1_1.a, t1_1.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.plt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                     Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                     ->  Seq Scan on public.plt2_p3 t2_2
+                           Output: t2_2.b, t2_2.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_2.a, t1_2.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_2.a, t1_2.c
+                                 Hash Cond: ((t1_2.c = ltrim(t3_2.c, 'A'::text)) AND (t1_2.a = t3_2.a))
+                                 ->  Seq Scan on public.plt1_p3 t1_2
+                                       Output: t1_2.a, t1_2.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.plt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: ((t1.a = t2.b) AND (t1.c = t2.c))
+                     ->  Seq Scan on public.plt1_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Hash Cond: ((t2.b = t3.a) AND (t2.c = ltrim(t3.c, 'A'::text)))
+                                 ->  Seq Scan on public.plt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.plt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: ((t1_1.a = t2_1.b) AND (t1_1.c = t2_1.c))
+                     ->  Seq Scan on public.plt1_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: ((t2_1.b = t3_1.a) AND (t2_1.c = ltrim(t3_1.c, 'A'::text)))
+                                 ->  Seq Scan on public.plt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.plt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: ((t1_2.a = t2_2.b) AND (t1_2.c = t2_2.c))
+                     ->  Seq Scan on public.plt1_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: ((t2_2.b = t3_2.a) AND (t2_2.c = ltrim(t3_2.c, 'A'::text)))
+                                 ->  Seq Scan on public.plt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.plt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+     |      |     |      |      100 | A0001
+     |      |     |      |      200 | A0002
+     |      |     |      |      400 | A0004
+     |      |     |      |      500 | A0005
+     |      |     |      |      700 | A0007
+     |      |     |      |      800 | A0008
+     |      |     |      |     1000 | A0010
+     |      |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+     |      |     |      |      100 | A0001
+     |      |     |      |      200 | A0002
+     |      |     |      |      400 | A0004
+     |      |     |      |      500 | A0005
+     |      |     |      |      700 | A0007
+     |      |     |      |      800 | A0008
+     |      |     |      |     1000 | A0010
+     |      |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                                  QUERY PLAN                                                   
+---------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, ((plt1_e_p1.a + plt1_e_p1.b)), plt1_e_p1.c
+   Sort Key: plt1_p1.a, plt2_p1.b, ((plt1_e_p1.a + plt1_e_p1.b))
+   ->  Result
+         Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, (plt1_e_p1.a + plt1_e_p1.b), plt1_e_p1.c
+         ->  Append
+               ->  Hash Full Join
+                     Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                     Hash Cond: ((plt1_p1.a = plt1_e_p1.a) AND (plt1_p1.c = ltrim(plt1_e_p1.c, 'A'::text)))
+                     ->  Hash Full Join
+                           Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+                           Hash Cond: ((plt1_p1.a = plt2_p1.b) AND (plt1_p1.c = plt2_p1.c))
+                           ->  Seq Scan on public.plt1_p1
+                                 Output: plt1_p1.a, plt1_p1.c
+                                 Filter: ((plt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p1.b, plt2_p1.c
+                                 ->  Seq Scan on public.plt2_p1
+                                       Output: plt2_p1.b, plt2_p1.c
+                                       Filter: ((plt2_p1.b % 25) = 0)
+                     ->  Hash
+                           Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                           ->  Seq Scan on public.plt1_e_p1
+                                 Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                                 Filter: ((plt1_e_p1.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c, plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                     Hash Cond: ((plt1_p2.a = plt1_e_p2.a) AND (plt1_p2.c = ltrim(plt1_e_p2.c, 'A'::text)))
+                     ->  Hash Full Join
+                           Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+                           Hash Cond: ((plt1_p2.a = plt2_p2.b) AND (plt1_p2.c = plt2_p2.c))
+                           ->  Seq Scan on public.plt1_p2
+                                 Output: plt1_p2.a, plt1_p2.c
+                                 Filter: ((plt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p2.b, plt2_p2.c
+                                 ->  Seq Scan on public.plt2_p2
+                                       Output: plt2_p2.b, plt2_p2.c
+                                       Filter: ((plt2_p2.b % 25) = 0)
+                     ->  Hash
+                           Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                           ->  Seq Scan on public.plt1_e_p2
+                                 Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                                 Filter: ((plt1_e_p2.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c, plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                     Hash Cond: ((plt1_p3.a = plt1_e_p3.a) AND (plt1_p3.c = ltrim(plt1_e_p3.c, 'A'::text)))
+                     ->  Hash Full Join
+                           Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+                           Hash Cond: ((plt1_p3.a = plt2_p3.b) AND (plt1_p3.c = plt2_p3.c))
+                           ->  Seq Scan on public.plt1_p3
+                                 Output: plt1_p3.a, plt1_p3.c
+                                 Filter: ((plt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p3.b, plt2_p3.c
+                                 ->  Seq Scan on public.plt2_p3
+                                       Output: plt2_p3.b, plt2_p3.c
+                                       Filter: ((plt2_p3.b % 25) = 0)
+                     ->  Hash
+                           Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                           ->  Seq Scan on public.plt1_e_p3
+                                 Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                                 Filter: ((plt1_e_p3.a % 25) = 0)
+(63 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.c = t1_3.c)
+               ->  Seq Scan on public.plt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_3.c, t2.c
+                     Hash Cond: (t1_3.c = ltrim(t2.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p1 t1_3
+                           Output: t1_3.c
+                     ->  Hash
+                           Output: t2.c
+                           ->  Seq Scan on public.plt1_e_p1 t2
+                                 Output: t2.c
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.c = t1_4.c)
+               ->  Seq Scan on public.plt1_p2 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_4.c, t2_1.c
+                     Hash Cond: (t1_4.c = ltrim(t2_1.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p2 t1_4
+                           Output: t1_4.c
+                     ->  Hash
+                           Output: t2_1.c
+                           ->  Seq Scan on public.plt1_e_p2 t2_1
+                                 Output: t2_1.c
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.c = t1_5.c)
+               ->  Seq Scan on public.plt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_5.c, t2_2.c
+                     Hash Cond: (t1_5.c = ltrim(t2_2.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p3 t1_5
+                           Output: t1_5.c
+                     ->  Hash
+                           Output: t2_2.c
+                           ->  Seq Scan on public.plt1_e_p3 t2_2
+                                 Output: t2_2.c
+(49 rows)
+
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1, uplt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+                                   QUERY PLAN                                   
+--------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.c = t1_3.c)
+               ->  Seq Scan on public.plt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_3.c, t1_6.c
+                     Hash Cond: (t1_3.c = ltrim(t1_6.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p1 t1_3
+                           Output: t1_3.c
+                     ->  Hash
+                           Output: t1_6.c
+                           ->  HashAggregate
+                                 Output: t1_6.c
+                                 Group Key: ltrim(t1_6.c, 'A'::text)
+                                 ->  Seq Scan on public.plt1_e_p1 t1_6
+                                       Output: t1_6.c, ltrim(t1_6.c, 'A'::text)
+                                       Filter: ((t1_6.a % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.c = t1_4.c)
+               ->  Seq Scan on public.plt1_p2 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_4.c, t1_7.c
+                     Hash Cond: (t1_4.c = ltrim(t1_7.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p2 t1_4
+                           Output: t1_4.c
+                     ->  Hash
+                           Output: t1_7.c
+                           ->  HashAggregate
+                                 Output: t1_7.c
+                                 Group Key: ltrim(t1_7.c, 'A'::text)
+                                 ->  Seq Scan on public.plt1_e_p2 t1_7
+                                       Output: t1_7.c, ltrim(t1_7.c, 'A'::text)
+                                       Filter: ((t1_7.a % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.c = t1_5.c)
+               ->  Seq Scan on public.plt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_5.c, t1_8.c
+                     Hash Cond: (t1_5.c = ltrim(t1_8.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p3 t1_5
+                           Output: t1_5.c
+                     ->  Hash
+                           Output: t1_8.c
+                           ->  HashAggregate
+                                 Output: t1_8.c
+                                 Group Key: ltrim(t1_8.c, 'A'::text)
+                                 ->  Seq Scan on public.plt1_e_p3 t1_8
+                                       Output: t1_8.c, ltrim(t1_8.c, 'A'::text)
+                                       Filter: ((t1_8.a % 25) = 0)
+(61 rows)
+
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+--
+-- negative testcases
+--
+-- joins where one of the relations is proven empty
+EXPLAIN (VERBOSE, 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;
+            QUERY PLAN            
+----------------------------------
+ Result
+   Output: t1.a, t1.c, t2.b, t2.c
+   One-Time Filter: false
+(3 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 LEFT JOIN prt2 t2 ON t1.a = t2.b;
+              QUERY PLAN              
+--------------------------------------
+ Result
+   Output: prt1.a, prt1.c, t2.b, t2.c
+   One-Time Filter: false
+(3 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                     
+---------------------------------------------------
+ Sort
+   Output: a, c, t2.b, t2.c
+   Sort Key: a, t2.b
+   ->  Hash Left Join
+         Output: a, c, t2.b, t2.c
+         Hash Cond: (t2.b = a)
+         ->  Append
+               ->  Seq Scan on public.prt2 t2
+                     Output: t2.b, t2.c
+                     Filter: ((t2.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p1 t2_1
+                     Output: t2_1.b, t2_1.c
+                     Filter: ((t2_1.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p2 t2_2
+                     Output: t2_2.b, t2_2.c
+                     Filter: ((t2_2.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p3 t2_3
+                     Output: t2_3.b, t2_3.c
+                     Filter: ((t2_3.a % 25) = 0)
+         ->  Hash
+               Output: a, c
+               ->  Result
+                     Output: a, c
+                     One-Time Filter: false
+(24 rows)
+
+EXPLAIN (VERBOSE, 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 % 25 = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                     
+---------------------------------------------------
+ Sort
+   Output: a, c, t2.b, t2.c
+   Sort Key: a, t2.b
+   ->  Hash Left Join
+         Output: a, c, t2.b, t2.c
+         Hash Cond: (t2.b = a)
+         ->  Append
+               ->  Seq Scan on public.prt2 t2
+                     Output: t2.b, t2.c
+                     Filter: ((t2.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p1 t2_1
+                     Output: t2_1.b, t2_1.c
+                     Filter: ((t2_1.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p2 t2_2
+                     Output: t2_2.b, t2_2.c
+                     Filter: ((t2_2.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p3 t2_3
+                     Output: t2_3.b, t2_3.c
+                     Filter: ((t2_3.a % 25) = 0)
+         ->  Hash
+               Output: a, c
+               ->  Result
+                     Output: a, c
+                     One-Time Filter: false
+(24 rows)
+
+CREATE TABLE prt1_n (a int, b int, c varchar) PARTITION BY RANGE(c);
+CREATE TABLE prt1_n_p1 PARTITION OF prt1_n FOR VALUES FROM ('0000') TO ('0250');
+CREATE TABLE prt1_n_p2 PARTITION OF prt1_n FOR VALUES FROM ('0250') TO ('0500');
+INSERT INTO prt1_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 499, 2) i;
+ANALYZE prt1_n;
+ANALYZE prt1_n_p1;
+ANALYZE prt1_n_p2;
+CREATE TABLE prt2_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt2_n_p1 PARTITION OF prt2_n FOR VALUES IN ('0000', '0003', '0004', '0010', '0006', '0007');
+CREATE TABLE prt2_n_p2 PARTITION OF prt2_n FOR VALUES IN ('0001', '0005', '0002', '0009', '0008', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt2_n;
+ANALYZE prt2_n_p1;
+ANALYZE prt2_n_p2;
+CREATE TABLE prt3_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt3_n_p1 PARTITION OF prt3_n FOR VALUES IN ('0000', '0004', '0006', '0007');
+CREATE TABLE prt3_n_p2 PARTITION OF prt3_n FOR VALUES IN ('0001', '0002', '0008', '0010');
+CREATE TABLE prt3_n_p3 PARTITION OF prt3_n FOR VALUES IN ('0003', '0005', '0009', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt3_n;
+ANALYZE prt3_n_p1;
+ANALYZE prt3_n_p2;
+ANALYZE prt3_n_p3;
+CREATE TABLE prt4_n (a int, b int, c text) PARTITION BY RANGE(a);
+CREATE TABLE prt4_n_p1 PARTITION OF prt4_n FOR VALUES FROM (0) TO (300);
+CREATE TABLE prt4_n_p2 PARTITION OF prt4_n FOR VALUES FROM (300) TO (500);
+CREATE TABLE prt4_n_p3 PARTITION OF prt4_n FOR VALUES FROM (500) TO (600);
+INSERT INTO prt4_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt4_n;
+ANALYZE prt4_n_p1;
+ANALYZE prt4_n_p2;
+ANALYZE prt4_n_p3;
+-- partition-wise join can not be applied if the partition ranges differ
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Join
+   Hash Cond: (t1.a = t2.a)
+   ->  Append
+         ->  Seq Scan on prt1 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p2 t1_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt4_n t2
+               ->  Seq Scan on prt4_n_p1 t2_1
+               ->  Seq Scan on prt4_n_p2 t2_2
+               ->  Seq Scan on prt4_n_p3 t2_3
+(13 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 FULL JOIN prt4_n t2 ON t1.a = t2.a;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Full Join
+   Hash Cond: (t1.a = t2.a)
+   ->  Append
+         ->  Seq Scan on prt1 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p2 t1_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt4_n t2
+               ->  Seq Scan on prt4_n_p1 t2_1
+               ->  Seq Scan on prt4_n_p2 t2_2
+               ->  Seq Scan on prt4_n_p3 t2_3
+(13 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                        
+---------------------------------------------------------
+ Nested Loop Left Join
+   ->  Append
+         ->  Seq Scan on prt1 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p2 t1_3
+   ->  Append
+         ->  Seq Scan on prt2 t2
+               Filter: (t1.a < b)
+         ->  Index Scan using iprt2_p1_b on prt2_p1 t2_1
+               Index Cond: (t1.a < b)
+         ->  Index Scan using iprt2_p2_b on prt2_p2 t2_2
+               Index Cond: (t1.a < b)
+         ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+               Index Cond: (t1.a < b)
+(15 rows)
+
+-- equi-join with join condition on partial keys does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1, prt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t1.a % 25 = 0;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Join
+   Hash Cond: (((t2.b + t2.a) / 2) = t1.a)
+   ->  Append
+         ->  Seq Scan on prt2_m t2
+         ->  Seq Scan on prt2_m_p1 t2_1
+         ->  Seq Scan on prt2_m_p2 t2_2
+         ->  Seq Scan on prt2_m_p3 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_m t1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p1 t1_1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p2 t1_2
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p3 t1_3
+                     Filter: ((a % 25) = 0)
+(17 rows)
+
+-- equi-join between out-of-order partition key columns does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.a = t2.b WHERE t1.a % 25 = 0;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Right Join
+   Hash Cond: (t2.b = t1.a)
+   ->  Append
+         ->  Seq Scan on prt2_m t2
+         ->  Seq Scan on prt2_m_p1 t2_1
+         ->  Seq Scan on prt2_m_p2 t2_2
+         ->  Seq Scan on prt2_m_p3 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_m t1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p1 t1_1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p2 t1_2
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p3 t1_3
+                     Filter: ((a % 25) = 0)
+(17 rows)
+
+-- equi-join between non-key columns does not qualify for partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.c = t2.c WHERE t1.a % 25 = 0;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Right Join
+   Hash Cond: ((t2.c)::text = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt2_m t2
+         ->  Seq Scan on prt2_m_p1 t2_1
+         ->  Seq Scan on prt2_m_p2 t2_2
+         ->  Seq Scan on prt2_m_p3 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_m t1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p1 t1_1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p2 t1_2
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p3 t1_3
+                     Filter: ((a % 25) = 0)
+(17 rows)
+
+-- partition-wise join can not be applied for a join between list and range
+-- partitioned table
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1, prt2_n t2 WHERE t1.c = t2.c;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Join
+   Hash Cond: (t2.c = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt2_n t2
+         ->  Seq Scan on prt2_n_p1 t2_1
+         ->  Seq Scan on prt2_n_p2 t2_2
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(11 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 LEFT JOIN prt2_n t2 ON (t1.c = t2.c);
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Right Join
+   Hash Cond: (t2.c = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt2_n t2
+         ->  Seq Scan on prt2_n_p1 t2_1
+         ->  Seq Scan on prt2_n_p2 t2_2
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(11 rows)
+
+-- partition-wise join can not be applied between tables with different
+-- partition lists
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 RIGHT JOIN prt1 t2 ON (t1.c = t2.c);
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Left Join
+   Hash Cond: ((t2.c)::text = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt1 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p2 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(12 rows)
+
+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);
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Full Join
+   Hash Cond: ((t2.c)::text = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt1 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p2 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(12 rows)
+
diff --git a/src/test/regress/expected/rangefuncs.out b/src/test/regress/expected/rangefuncs.out
index f06cfa4..16e7f56 100644
--- a/src/test/regress/expected/rangefuncs.out
+++ b/src/test/regress/expected/rangefuncs.out
@@ -1,18 +1,19 @@
 SELECT name, setting FROM pg_settings WHERE name LIKE 'enable%';
-         name         | setting 
-----------------------+---------
- enable_bitmapscan    | on
- enable_hashagg       | on
- enable_hashjoin      | on
- enable_indexonlyscan | on
- enable_indexscan     | on
- enable_material      | on
- enable_mergejoin     | on
- enable_nestloop      | on
- enable_seqscan       | on
- enable_sort          | on
- enable_tidscan       | on
-(11 rows)
+            name            | setting 
+----------------------------+---------
+ enable_bitmapscan          | on
+ enable_hashagg             | on
+ enable_hashjoin            | on
+ enable_indexonlyscan       | on
+ enable_indexscan           | on
+ enable_material            | on
+ enable_mergejoin           | on
+ enable_nestloop            | on
+ enable_partition_wise_join | on
+ enable_seqscan             | on
+ enable_sort                | on
+ enable_tidscan             | on
+(12 rows)
 
 CREATE TABLE foo2(fooid int, f2 int);
 INSERT INTO foo2 VALUES(1, 11);
diff --git a/src/test/regress/parallel_schedule b/src/test/regress/parallel_schedule
index 8641769..b61ca3b 100644
--- a/src/test/regress/parallel_schedule
+++ b/src/test/regress/parallel_schedule
@@ -99,8 +99,9 @@ test: select_parallel
 
 # ----------
 # Another group of parallel tests
+# TODO: merge partition_join and multi_level_partition_join
 # ----------
-test: select_views portals_p2 foreign_key cluster dependency guc bitmapops combocid tsearch tsdicts foreign_data window xmlmap functional_deps advisory_lock json jsonb json_encoding indirect_toast equivclass
+test: select_views portals_p2 foreign_key cluster dependency guc bitmapops combocid tsearch tsdicts foreign_data window xmlmap functional_deps advisory_lock json jsonb json_encoding indirect_toast equivclass partition_join multi_level_partition_join
 # ----------
 # Another group of parallel tests
 # NB: temp.sql does a reconnect which transiently uses 2 connections,
diff --git a/src/test/regress/serial_schedule b/src/test/regress/serial_schedule
index 835cf35..5b167b6 100644
--- a/src/test/regress/serial_schedule
+++ b/src/test/regress/serial_schedule
@@ -169,3 +169,5 @@ test: with
 test: xml
 test: event_trigger
 test: stats
+test: partition_join
+test: multi_level_partition_join
diff --git a/src/test/regress/sql/multi_level_partition_join.sql b/src/test/regress/sql/multi_level_partition_join.sql
new file mode 100644
index 0000000..31f0281
--- /dev/null
+++ b/src/test/regress/sql/multi_level_partition_join.sql
@@ -0,0 +1,95 @@
+--
+-- multi-leveled partitions
+--
+CREATE TABLE prt1_l (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_l_p1 PARTITION OF prt1_l FOR VALUES FROM (0) TO (250) PARTITION BY RANGE (b);
+CREATE TABLE prt1_l_p1_p1 PARTITION OF prt1_l_p1 FOR VALUES FROM (0) TO (100);
+CREATE TABLE prt1_l_p1_p2 PARTITION OF prt1_l_p1 FOR VALUES FROM (100) TO (250);
+CREATE TABLE prt1_l_p2 PARTITION OF prt1_l FOR VALUES FROM (250) TO (500) PARTITION BY RANGE (c);
+CREATE TABLE prt1_l_p2_p1 PARTITION OF prt1_l_p2 FOR VALUES FROM ('0250') TO ('0400');
+CREATE TABLE prt1_l_p2_p2 PARTITION OF prt1_l_p2 FOR VALUES FROM ('0400') TO ('0500');
+CREATE TABLE prt1_l_p3 PARTITION OF prt1_l FOR VALUES FROM (500) TO (600) PARTITION BY RANGE ((b + a));
+CREATE TABLE prt1_l_p3_p1 PARTITION OF prt1_l_p3 FOR VALUES FROM (1000) TO (1100);
+CREATE TABLE prt1_l_p3_p2 PARTITION OF prt1_l_p3 FOR VALUES FROM (1100) TO (1200);
+INSERT INTO prt1_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_l;
+ANALYZE prt1_l_p1;
+ANALYZE prt1_l_p1_p1;
+ANALYZE prt1_l_p1_p2;
+ANALYZE prt1_l_p2;
+ANALYZE prt1_l_p2_p1;
+ANALYZE prt1_l_p2_p2;
+ANALYZE prt1_l_p3;
+ANALYZE prt1_l_p3_p1;
+ANALYZE prt1_l_p3_p2;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_l AS SELECT * FROM prt1_l;
+
+CREATE TABLE prt2_l (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_l_p1 PARTITION OF prt2_l FOR VALUES FROM (0) TO (250) PARTITION BY RANGE (a);
+CREATE TABLE prt2_l_p1_p1 PARTITION OF prt2_l_p1 FOR VALUES FROM (0) TO (100);
+CREATE TABLE prt2_l_p1_p2 PARTITION OF prt2_l_p1 FOR VALUES FROM (100) TO (250);
+CREATE TABLE prt2_l_p2 PARTITION OF prt2_l FOR VALUES FROM (250) TO (500) PARTITION BY RANGE (c);
+CREATE TABLE prt2_l_p2_p1 PARTITION OF prt2_l_p2 FOR VALUES FROM ('0250') TO ('0400');
+CREATE TABLE prt2_l_p2_p2 PARTITION OF prt2_l_p2 FOR VALUES FROM ('0400') TO ('0500');
+CREATE TABLE prt2_l_p3 PARTITION OF prt2_l FOR VALUES FROM (500) TO (600) PARTITION BY RANGE ((a + b));
+CREATE TABLE prt2_l_p3_p1 PARTITION OF prt2_l_p3 FOR VALUES FROM (1000) TO (1100);
+CREATE TABLE prt2_l_p3_p2 PARTITION OF prt2_l_p3 FOR VALUES FROM (1100) TO (1200);
+INSERT INTO prt2_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_l;
+ANALYZE prt2_l_p1;
+ANALYZE prt2_l_p1_p1;
+ANALYZE prt2_l_p1_p2;
+ANALYZE prt2_l_p2;
+ANALYZE prt2_l_p2_p1;
+ANALYZE prt2_l_p2_p2;
+ANALYZE prt2_l_p3;
+ANALYZE prt2_l_p3_p1;
+ANALYZE prt2_l_p3_p2;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_l AS SELECT * FROM prt2_l;
+
+EXPLAIN (VERBOSE, 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 = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 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 = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1, uprt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 LEFT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 RIGHT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_l t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_l t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+
+-- lateral reference
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
+			  ON t1.a = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
+			  ON t1.a = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1_l t2 JOIN uprt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
+			  ON t1.a = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
+			  ON t1.b = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
+			  ON t1.b = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1_l t2 JOIN uprt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
+			  ON t1.b = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
new file mode 100644
index 0000000..9b2baeb
--- /dev/null
+++ b/src/test/regress/sql/partition_join.sql
@@ -0,0 +1,520 @@
+--
+-- PARTITION_JOIN
+-- Test partition-wise join between partitioned tables
+--
+
+-- Usually partition-wise join paths are chosen when data is large, which would
+-- take regression tests to run longer. So, weigh partition-wise joins cheaper
+-- to force those even for smaller data.
+SET partition_wise_plan_weight to 0.2;
+
+--
+-- partitioned by a single column
+--
+CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+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, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1 AS SELECT * FROM prt1;
+
+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_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, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+CREATE TABLE uprt2 AS SELECT * FROM prt2;
+
+-- inner join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+-- left outer join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+-- right outer join
+EXPLAIN (VERBOSE, 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.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+-- full outer join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+
+-- Join with pruned partitions from joining relations
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a < 450 AND t2.b > 250 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+-- lateral reference
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+--
+-- partitioned by expression
+--
+CREATE TABLE prt1_e (a int, b int, c varchar) PARTITION BY RANGE(((a + b)/2));
+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);
+INSERT INTO prt1_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_e AS SELECT * FROM prt1_e;
+
+CREATE TABLE prt2_e (a int, b int, c varchar) PARTITION BY RANGE(((b + a)/2));
+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);
+INSERT INTO prt2_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_e;
+ANALYZE prt2_e_p1;
+ANALYZE prt2_e_p2;
+ANALYZE prt2_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_e AS SELECT * FROM prt2_e;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1, uprt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1 LEFT JOIN uprt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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.a % 25 = 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM uprt1 t1, uprt2 t2, uprt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM uprt1_e WHERE uprt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, 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.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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;
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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;
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM uprt1_e WHERE uprt1_e.a % 25 = 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;
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1, uprt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+-- test merge joins with and without using indexes
+SET enable_hashjoin TO off;
+SET enable_nestloop TO off;
+
+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 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 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));
+
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+SET enable_seqscan TO off;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+-- lateral references and parameterized paths
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+RESET enable_hashjoin;
+RESET enable_nestloop;
+RESET enable_seqscan;
+
+--
+-- partitioned by multiple columns
+--
+CREATE TABLE prt1_m (a int, b int, c varchar) 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, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_m;
+ANALYZE prt1_m_p1;
+ANALYZE prt1_m_p2;
+ANALYZE prt1_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_m AS SELECT * FROM prt1_m;
+
+CREATE TABLE prt2_m (a int, b int, c varchar) 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, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_m;
+ANALYZE prt2_m_p1;
+ANALYZE prt2_m_p2;
+ANALYZE prt2_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_m AS SELECT * FROM prt2_m;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_m t1 RIGHT JOIN uprt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_m t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_m t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+
+--
+-- 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;
+ANALYZE plt1_p1;
+ANALYZE plt1_p2;
+ANALYZE plt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1 AS SELECT * FROM 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;
+ANALYZE plt2_p1;
+ANALYZE plt2_p2;
+ANALYZE plt2_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt2 AS SELECT * FROM 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;
+ANALYZE plt1_e_p1;
+ANALYZE plt1_e_p2;
+ANALYZE plt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1_e AS SELECT * FROM plt1_e;
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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;
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM uplt1 t1, uplt2 t2, uplt1_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;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1, uplt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 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 ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+--
+-- negative testcases
+--
+
+-- joins where one of the relations is proven empty
+EXPLAIN (VERBOSE, 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;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 LEFT JOIN prt2 t2 ON t1.a = t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, 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 % 25 = 0 ORDER BY t1.a, t2.b;
+
+CREATE TABLE prt1_n (a int, b int, c varchar) PARTITION BY RANGE(c);
+CREATE TABLE prt1_n_p1 PARTITION OF prt1_n FOR VALUES FROM ('0000') TO ('0250');
+CREATE TABLE prt1_n_p2 PARTITION OF prt1_n FOR VALUES FROM ('0250') TO ('0500');
+INSERT INTO prt1_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 499, 2) i;
+ANALYZE prt1_n;
+ANALYZE prt1_n_p1;
+ANALYZE prt1_n_p2;
+
+CREATE TABLE prt2_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt2_n_p1 PARTITION OF prt2_n FOR VALUES IN ('0000', '0003', '0004', '0010', '0006', '0007');
+CREATE TABLE prt2_n_p2 PARTITION OF prt2_n FOR VALUES IN ('0001', '0005', '0002', '0009', '0008', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt2_n;
+ANALYZE prt2_n_p1;
+ANALYZE prt2_n_p2;
+
+CREATE TABLE prt3_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt3_n_p1 PARTITION OF prt3_n FOR VALUES IN ('0000', '0004', '0006', '0007');
+CREATE TABLE prt3_n_p2 PARTITION OF prt3_n FOR VALUES IN ('0001', '0002', '0008', '0010');
+CREATE TABLE prt3_n_p3 PARTITION OF prt3_n FOR VALUES IN ('0003', '0005', '0009', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt3_n;
+ANALYZE prt3_n_p1;
+ANALYZE prt3_n_p2;
+ANALYZE prt3_n_p3;
+
+CREATE TABLE prt4_n (a int, b int, c text) PARTITION BY RANGE(a);
+CREATE TABLE prt4_n_p1 PARTITION OF prt4_n FOR VALUES FROM (0) TO (300);
+CREATE TABLE prt4_n_p2 PARTITION OF prt4_n FOR VALUES FROM (300) TO (500);
+CREATE TABLE prt4_n_p3 PARTITION OF prt4_n FOR VALUES FROM (500) TO (600);
+INSERT INTO prt4_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt4_n;
+ANALYZE prt4_n_p1;
+ANALYZE prt4_n_p2;
+ANALYZE prt4_n_p3;
+
+-- partition-wise join can not be applied if the partition ranges differ
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 FULL JOIN prt4_n t2 ON t1.a = t2.a;
+
+-- 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);
+
+-- equi-join with join condition on partial keys does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1, prt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t1.a % 25 = 0;
+
+-- equi-join between out-of-order partition key columns does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.a = t2.b WHERE t1.a % 25 = 0;
+
+-- equi-join between non-key columns does not qualify for partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.c = t2.c WHERE t1.a % 25 = 0;
+
+-- partition-wise join can not be applied for a join between list and range
+-- partitioned table
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1, prt2_n t2 WHERE t1.c = t2.c;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 LEFT JOIN prt2_n t2 ON (t1.c = t2.c);
+
+-- partition-wise join can not be applied between tables with different
+-- partition lists
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 RIGHT JOIN prt1 t2 ON (t1.c = t2.c);
+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);
#28Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Ashutosh Bapat (#27)
2 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Dec 27, 2016 at 11:01 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

PFA patch rebased after partitioning code was committed.

On Thu, Dec 1, 2016 at 4:32 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Hi Robert,
Sorry for delayed response.

The attached patch implements following ideas:
1. At the time of creating paths - If the joining relations are both
partitioned and join can use partition-wise join, we create paths for
few child-joins. Similar to inheritance relations
(set_append_rel_pathlist()), we collect paths with similar properties
from all sampled child-joins and create one PartitionJoinPath with
each set of paths. The cost of the PartitionJoinPath is obtained by
multiplying the sum of costs of paths in the given set by the ratio of
(number of rows estimated in the parent-join/sum of rows in
child-joins).

2. If the PartitionJoinPath emerges as the best path, we create paths
for each of the remaining child-joins. Then we collect paths with
properties same as the given PartitionJoinPath, one from each
child-join. These paths are converted into plans and a Merge/Append
plan is created combing these plans. The paths and plans for
child-join are created in a temporary memory context. The final plan
for each child-join is copied into planner's context and the temporary
memory context is reset.

Right now, we choose 1% or 1 (whichever is higher) child-joins to base
PartitionJoinPath costs on.

Memory consumption
-----------------------------
I tested a 5-way self-join for a table with 1000 partitions, each
partition having 1M rows. The memory consumed in standard_planner()
was measured with some granular tracking
(mem_usage_func_wise_measurement_slabwise.patch). Partition-wise join
consumed total of 289MB memory which is approx 6.6 times more than
non-partition-wise join which consumed 44MB. That's much better than
the earlier 16 times consumption for 5-way join with 100 partitions.

The extra 245MB memory was consumed by child-join RelOptInfos (48MB),
SpecialJoinInfos for child-joins (64MB), restrictlist translation
(92MB), paths for sampled child-joins (1.5MB), building targetlists
for child-joins (7MB).

In the earlier implementation, a given clause which was applicable to
multiple join orders was getting translated as many times as the join
orders it was applicable in. I changed RestrictInfo for parent to
store a list of RestrictInfos applicable to children to avoid multiple
translations.

My earlier patch created the child-join plans in a temporary context
and then copied them into planner context since the translated clauses
were allocated memory in temporary memory context then. Now that they
are stored in planner's context, we can directly create the plan in
the planner's context.

Third, I added code to free up child SpecialJoinInfos after using those.

As a result the total memory consumption now is 192MB, which is approx
4.4 times the memory consumed during planning in case of
non-partition-wise join.

Choosing representative child-joins:
--------------------------------------------------
There's another angle to choosing representative child joins. In a
partitioned N-way join, different joins covering different subsets of
N relations, will have different size distributions across the
partitions. This means that the child-joins costed for (N-k) joins,
may be different for those required for (N-k+1) joins. With a factor
of 1% sampling, N is such that a child-join participates in 100 joins,
we will end up creating paths for all partitions before creating
PartitionJoinPaths for the final N-way join. Hopefully that will be a
rare case and usually we will end up using paths already created. We
can not avoid creating PartitionJoinPaths for subset joins, as there
might be cases when partition-wise join will be optimal for an N-k way
join but not for N-way join. We may avoid this if we choose
representative child-joins based on their positions, in which case, we
may end up with some or all of those being empty and thus skewing the
costs heavily.

Partial paths
-----------------
AFAIU, we create partial paths for append relation, when all the
children have partial paths. Unlike parameterized paths or path with
pathkeys, there is no way to create a partial path for a normal path.
This means that unless we create paths for all child-joins, we can not
create partial paths for appendrel comprising of child-joins, and thus
can not use parallel query right now. This may not be that bad, since
it would be more efficient to run each child-join in a separate
worker, rather than using multiple workers for a single child-join.

This still applies.

regression tests
----------------------
I observed that for small relations (1000 rows in each partition and
100 partitions), the size estimates in append relations and sum of
those in child relations are very different. As a result, the
extrapolated costs for PartitionJoinPaths as described above, are way
higher than costs of join of appends (or even append of joins if we
are to create paths for all child-joins). Thus with this approach, we
choose partition-wise join for large number of partitions with large
data (e.g. 1000 partitions with 1M rows each). These are certainly the
cases when partition-wise join is a big win. I have not tried to find
out a threshold above which partition-wise join gets chosen with above
approach, but it's going to be a larger threshold. That makes writing
regression tests difficult, as those will require large data. So, we
have to find a way so that we can test partition-wise join with
smaller data. There are few possibilities like 1. convert the fraction
of representative child-joins into GUC and setting it to 100% would
start choosing partition-wise joins for tables with a few hundred rows
per partition, like it did in earlier approach, 2. provide a way to
force partition-wise join whenever possible, by say costing
partition-wise joins much lesser than non-partition-wise join when a
GUC is set (e.g. enable_partition_wise_join with values always, never,
optimal or something like that).

For now I have added a float GUC partition_wise_plan_weight. The
partition-wise join cost derived from the samples is multiplied by
this GUC and set as the cost of ParitionJoinPath. A value of 1 means
that the cost derived from the samples are used as is. A value higher
than 1 discourages use of partition-wise join and that lower than 1
encourages use of partition-wise join. I am not very keen on keeping
this GUC, in this form. But we need some way to run regression with
smaller data.

For now I have disabled partition-wise join for multi-level
partitions. I will post a patch soon with that enabled.

PFA the patch (pg_dp_join_v6.patch) with some bugs fixed and rebased
on the latest code.

Also, PFA patch to support partition-wise join between multi-level
partitioned tables. I copied the Amit Langote's patch for translating
partition hierarchy into inheritance hierarchy and added code to
support partition-wise join. You had expressed some concerns about
Amit's approach in [1]/messages/by-id/CA+TgmoaEU10Kmdy44izcqJYLh1fkh58_6sbGGu0Q4b7PPE46eA@mail.gmail.com, but that discussion is still open. So, I
haven't merged those changes to partition-wise join patch. We may
continue to work on it as separate patch or I can include it in
partition-wise join main patch.

BTW, INSERT into multi-level partitioned tables is crashing with
latest head. The issue was reported in [2]/messages/by-id/CAKcux6=m1qyqB2k6cjniuMMrYXb75O-MB4qGQMu8zg-iGGLjDw@mail.gmail.com. Because of that
multi_level_partition_join test crashes in pg_dp_join_v6.patch.
Intestingly the crash vanishes when we apply patch supporting
mult-level partition-wise join.

[1]: /messages/by-id/CA+TgmoaEU10Kmdy44izcqJYLh1fkh58_6sbGGu0Q4b7PPE46eA@mail.gmail.com
[2]: /messages/by-id/CAKcux6=m1qyqB2k6cjniuMMrYXb75O-MB4qGQMu8zg-iGGLjDw@mail.gmail.com

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

Attachments:

pg_dp_join_v6.patchapplication/x-download; name=pg_dp_join_v6.patchDownload
diff --git a/contrib/postgres_fdw/deparse.c b/contrib/postgres_fdw/deparse.c
index 66b059a..b1ea55a 100644
--- a/contrib/postgres_fdw/deparse.c
+++ b/contrib/postgres_fdw/deparse.c
@@ -911,7 +911,7 @@ deparseSelectStmtForRel(StringInfo buf, PlannerInfo *root, RelOptInfo *rel,
 	 * We handle relations for foreign tables, joins between those and upper
 	 * relations.
 	 */
-	Assert(rel->reloptkind == RELOPT_JOINREL ||
+	Assert(IS_JOIN_REL(rel) ||
 		   rel->reloptkind == RELOPT_BASEREL ||
 		   rel->reloptkind == RELOPT_OTHER_MEMBER_REL ||
 		   rel->reloptkind == RELOPT_UPPER_REL);
@@ -990,7 +990,7 @@ deparseSelectSql(List *tlist, List **retrieved_attrs, deparse_expr_cxt *context)
 	 */
 	appendStringInfoString(buf, "SELECT ");
 
-	if (foreignrel->reloptkind == RELOPT_JOINREL ||
+	if (IS_JOIN_REL(foreignrel) ||
 		foreignrel->reloptkind == RELOPT_UPPER_REL)
 	{
 		/* For a join relation use the input tlist */
@@ -1030,7 +1030,7 @@ deparseFromExpr(List *quals, deparse_expr_cxt *context)
 
 	/* For upper relations, scanrel must be either a joinrel or a baserel */
 	Assert(context->foreignrel->reloptkind != RELOPT_UPPER_REL ||
-		   scanrel->reloptkind == RELOPT_JOINREL ||
+		   IS_JOIN_REL(scanrel) ||
 		   scanrel->reloptkind == RELOPT_BASEREL);
 
 	/* Construct FROM clause */
@@ -1178,7 +1178,7 @@ deparseLockingClause(deparse_expr_cxt *context)
 			appendStringInfoString(buf, " FOR UPDATE");
 
 			/* Add the relation alias if we are here for a join relation */
-			if (rel->reloptkind == RELOPT_JOINREL)
+			if (IS_JOIN_REL(rel))
 				appendStringInfo(buf, " OF %s%d", REL_ALIAS_PREFIX, relid);
 		}
 		else
@@ -1345,7 +1345,7 @@ deparseFromExprForRel(StringInfo buf, PlannerInfo *root, RelOptInfo *foreignrel,
 {
 	PgFdwRelationInfo *fpinfo = (PgFdwRelationInfo *) foreignrel->fdw_private;
 
-	if (foreignrel->reloptkind == RELOPT_JOINREL)
+	if (IS_JOIN_REL(foreignrel))
 	{
 		RelOptInfo *rel_o = fpinfo->outerrel;
 		RelOptInfo *rel_i = fpinfo->innerrel;
diff --git a/contrib/postgres_fdw/postgres_fdw.c b/contrib/postgres_fdw/postgres_fdw.c
index fbe6929..74fca6b 100644
--- a/contrib/postgres_fdw/postgres_fdw.c
+++ b/contrib/postgres_fdw/postgres_fdw.c
@@ -721,8 +721,8 @@ get_useful_ecs_for_relation(PlannerInfo *root, RelOptInfo *rel)
 		return useful_eclass_list;
 
 	/* If this is a child rel, we must use the topmost parent rel to search. */
-	if (rel->reloptkind == RELOPT_OTHER_MEMBER_REL)
-		relids = find_childrel_top_parent(root, rel)->relids;
+	if (IS_OTHER_REL(rel))
+		relids = find_childrel_top_parent(root, rel);
 	else
 		relids = rel->relids;
 
@@ -1183,7 +1183,7 @@ postgresGetForeignPlan(PlannerInfo *root,
 			local_exprs = lappend(local_exprs, rinfo->clause);
 	}
 
-	if (foreignrel->reloptkind == RELOPT_JOINREL ||
+	if (IS_JOIN_REL(foreignrel) ||
 		foreignrel->reloptkind == RELOPT_UPPER_REL)
 	{
 		/* For a join relation, get the conditions from fdw_private structure */
@@ -1249,7 +1249,7 @@ postgresGetForeignPlan(PlannerInfo *root,
 							 remote_conds,
 							 retrieved_attrs,
 							 makeInteger(fpinfo->fetch_size));
-	if (foreignrel->reloptkind == RELOPT_JOINREL ||
+	if (IS_JOIN_REL(foreignrel) ||
 		foreignrel->reloptkind == RELOPT_UPPER_REL)
 		fdw_private = lappend(fdw_private,
 							  makeString(fpinfo->relation_name->data));
@@ -2529,7 +2529,7 @@ estimate_path_cost_size(PlannerInfo *root,
 						   &remote_param_join_conds, &local_param_join_conds);
 
 		/* Build the list of columns to be fetched from the foreign server. */
-		if (foreignrel->reloptkind == RELOPT_JOINREL ||
+		if (IS_JOIN_REL(foreignrel) ||
 			foreignrel->reloptkind == RELOPT_UPPER_REL)
 			fdw_scan_tlist = build_tlist_to_deparse(foreignrel);
 		else
@@ -2611,7 +2611,7 @@ estimate_path_cost_size(PlannerInfo *root,
 			startup_cost = fpinfo->rel_startup_cost;
 			run_cost = fpinfo->rel_total_cost - fpinfo->rel_startup_cost;
 		}
-		else if (foreignrel->reloptkind == RELOPT_JOINREL)
+		else if (IS_JOIN_REL(foreignrel))
 		{
 			PgFdwRelationInfo *fpinfo_i;
 			PgFdwRelationInfo *fpinfo_o;
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index fca8747..98fed55 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -20,6 +20,7 @@
 #include "access/nbtree.h"
 #include "access/sysattr.h"
 #include "catalog/dependency.h"
+#include "catalog/heap.h"
 #include "catalog/indexing.h"
 #include "catalog/objectaddress.h"
 #include "catalog/partition.h"
@@ -139,6 +140,8 @@ static int32 partition_bound_cmp(PartitionKey key,
 static int partition_bound_bsearch(PartitionKey key,
 						PartitionBoundInfo boundinfo,
 						void *probe, bool probe_is_bound, bool *is_equal);
+static PartitionBoundInfo partition_bounds_copy(PartitionBoundInfo src,
+									PartitionKey key);
 
 /* Support get_partition_for_tuple() */
 static void FormPartitionKeyDatum(PartitionDispatch pd,
@@ -1990,3 +1993,250 @@ partition_bound_bsearch(PartitionKey key, PartitionBoundInfo boundinfo,
 
 	return lo;
 }
+
+ /*
+ * Return a copy of given PartitionBoundInfo structure. The data types of bounds
+ * are described by given partition key specificiation.
+ */
+static PartitionBoundInfo
+partition_bounds_copy(PartitionBoundInfo src, PartitionKey key)
+{
+	PartitionBoundInfo	dest;
+	int		i;
+	int		ndatums;
+	int		partnatts;
+	int		num_indexes;
+
+	dest = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+
+	dest->strategy = src->strategy;
+	ndatums = dest->ndatums = src->ndatums;
+	partnatts = key->partnatts;
+
+	/* Range partitioned table has an extra index. */
+	num_indexes = key->strategy == PARTITION_STRATEGY_RANGE ? ndatums + 1 : ndatums;
+
+	/* List partitioned tables have only a single partition key. */
+	Assert(key->strategy != PARTITION_STRATEGY_LIST || partnatts == 1);
+
+	dest->datums = (Datum **) palloc(sizeof(Datum *) * ndatums);
+
+	for (i = 0; i < ndatums; i++)
+	{
+		int		j;
+		dest->datums[i] = (Datum *) palloc(sizeof(Datum) * partnatts);
+
+		for (j = 0; j < partnatts; j++)
+			dest->datums[i][j] = datumCopy(src->datums[i][j],
+										   key->parttypbyval[j],
+										   key->parttyplen[j]);
+	}
+
+	if (src->content)
+	{
+		dest->content = (RangeDatumContent **) palloc(ndatums *
+												sizeof(RangeDatumContent *));
+		for (i = 0; i < ndatums; i++)
+		{
+			dest->content[i] = (RangeDatumContent *) palloc(partnatts *
+												sizeof(RangeDatumContent));
+
+			memcpy(dest->content[i], src->content[i],
+				   sizeof(RangeDatumContent) * key->partnatts);
+		}
+	}
+	else
+		dest->content = NULL;
+
+	dest->indexes = (int *) palloc(sizeof(int) * num_indexes);
+	memcpy(dest->indexes, src->indexes, sizeof(int) * num_indexes);
+
+	dest->has_null = src->has_null;
+	dest->null_index = src->null_index;
+
+	return dest;
+}
+
+/*
+ * find_partition_scheme
+ *
+ * The function returns a canonical partition scheme which exactly matches the
+ * partitioning properties of the given relation if one exists in the of
+ * canonical partitioning schemes maintained in PlannerInfo. If none of the
+ * existing partitioning schemes match, the function creates a canonical
+ * partition scheme and adds it to the list.
+ *
+ * For an unpartitioned table or for a multi-level partitioned table it returns
+ * NULL.
+ */
+extern PartitionScheme
+find_partition_scheme(PlannerInfo *root, Relation relation)
+{
+	PartitionKey	part_key = RelationGetPartitionKey(relation);
+	PartitionDesc	part_desc = RelationGetPartitionDesc(relation);
+	ListCell	   *lc;
+	int		nparts;
+	int		partnatts;
+	int		cnt_pks;
+	int		cnt_parts;
+	PartitionScheme	part_scheme = NULL;
+
+	/* No partition scheme for an unpartitioned relation. */
+	if (!part_desc || !part_key)
+		return NULL;
+
+	nparts = part_desc->nparts;
+	partnatts = part_key->partnatts;
+
+	/*
+	 * For a multi-level partitioned table, we do not retain the partitioning
+	 * hierarchy while expanding RTE for the topmost parent. Thus the number of
+	 * children as per root->append_rel_list does not match the number of
+	 * partitions specified in the partition descriptor and hence the
+	 * partitioning scheme of a multi-partitioned table does not reflect the
+	 * true picture. So for now, treat a multi-partitioned table as not
+	 * partitioned.
+	 */
+	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	{
+		if (has_subclass(part_desc->oids[cnt_parts]))
+			return NULL;
+	}
+
+	/* Search for a matching partition scheme and return if found one. */
+	foreach (lc, root->part_schemes)
+	{
+		part_scheme = lfirst(lc);
+
+		/* Match number of partitions and partitioning strategy. */
+		if (nparts != part_scheme->nparts ||
+			part_key->strategy != part_scheme->strategy ||
+			partnatts != part_scheme->partnatts)
+			continue;
+
+		/* Match the partition key types. */
+		for (cnt_pks = 0; cnt_pks < partnatts; cnt_pks++)
+		{
+			/*
+			 * For types, it suffices to match the type id, mod and collation;
+			 * len, byval and align are depedent on the first two.
+			 */
+			if (part_key->partopfamily[cnt_pks] != part_scheme->partopfamily[cnt_pks] ||
+				part_key->partopcintype[cnt_pks] != part_scheme->partopcintype[cnt_pks] ||
+				part_key->parttypid[cnt_pks] != part_scheme->key_types[cnt_pks] ||
+				part_key->parttypmod[cnt_pks] != part_scheme->key_typmods[cnt_pks] ||
+				part_key->parttypcoll[cnt_pks] != part_scheme->key_collations[cnt_pks])
+				break;
+		}
+
+		/* Some partition key didn't match. Check next partitioning scheme. */
+		if (cnt_pks < partnatts)
+			continue;
+
+		if (!partition_bounds_equal(part_key, part_desc->boundinfo,
+									part_scheme->boundinfo))
+			continue;
+
+		/* Found matching partition scheme. */
+		return part_scheme;
+	}
+
+	/* Did not find matching partition scheme. Create one. */
+	part_scheme = (PartitionScheme) palloc0(sizeof(PartitionSchemeData));
+
+	/* Copy partition bounds/lists. */
+	part_scheme->nparts = part_desc->nparts;
+	part_scheme->strategy = part_key->strategy;
+	part_scheme->boundinfo = partition_bounds_copy(part_desc->boundinfo,
+												   part_key);
+
+	/* Store partition key information. */
+	part_scheme->partnatts = part_key->partnatts;
+
+	part_scheme->partopfamily = (Oid *) palloc(sizeof(Oid) * partnatts);
+	memcpy(part_scheme->partopfamily, part_key->partopfamily,
+		   sizeof(Oid) * partnatts);
+
+	part_scheme->partopcintype = (Oid *) palloc(sizeof(Oid) * partnatts);
+	memcpy(part_scheme->partopcintype, part_key->partopcintype,
+		   sizeof(Oid) * partnatts);
+
+	part_scheme->key_types = (Oid *) palloc(sizeof(Oid) * partnatts);
+	memcpy(part_scheme->key_types, part_key->parttypid,
+		   sizeof(Oid) * partnatts);
+
+	part_scheme->key_typmods = (int32 *) palloc(sizeof(int32) * partnatts);
+	memcpy(part_scheme->key_typmods, part_key->parttypmod,
+		   sizeof(int32) * partnatts);
+
+	part_scheme->key_collations = (Oid *) palloc(sizeof(Oid) * partnatts);
+	memcpy(part_scheme->key_collations, part_key->parttypcoll,
+		   sizeof(Oid) * partnatts);
+
+	/* Add the partitioning scheme to PlannerInfo. */
+	root->part_schemes = lappend(root->part_schemes, part_scheme);
+
+	return part_scheme;
+}
+
+/*
+ * build_baserel_partition_key_exprs
+ *
+ * Collect partition key expressions for a given base relation. The function
+ * converts any single column partition keys into corresponding Var nodes. It
+ * restamps Var nodes in partition key expressions by given varno. The
+ * partition key expressions are returned as an array of single element Lists
+ * to be stored in RelOptInfo of the base relation.
+ */
+extern List **
+build_baserel_partition_key_exprs(Relation relation, Index varno)
+{
+	PartitionKey	part_key = RelationGetPartitionKey(relation);
+	int		num_pkexprs;
+	int		cnt_pke;
+	List		  **partexprs;
+	ListCell	   *lc;
+
+	if (!part_key || part_key->partnatts <= 0)
+		return NULL;
+
+	num_pkexprs = part_key->partnatts;
+	partexprs = (List **) palloc(sizeof(List *) * num_pkexprs);
+	lc = list_head(part_key->partexprs);
+
+	for (cnt_pke = 0; cnt_pke < num_pkexprs; cnt_pke++)
+	{
+		AttrNumber attno = part_key->partattrs[cnt_pke];
+		Expr	  *pkexpr;
+
+		if (attno != InvalidAttrNumber)
+		{
+			/* Single column partition key is stored as a Var node. */
+			Form_pg_attribute att_tup;
+
+			if (attno < 0)
+				att_tup = SystemAttributeDefinition(attno,
+												relation->rd_rel->relhasoids);
+			else
+				att_tup = relation->rd_att->attrs[attno - 1];
+
+			pkexpr = (Expr *) makeVar(varno, attno, att_tup->atttypid,
+									  att_tup->atttypmod,
+									  att_tup->attcollation, 0);
+		}
+		else
+		{
+			if (lc == NULL)
+				elog(ERROR, "wrong number of partition key expressions");
+
+			/* Re-stamp the expressions with given varno. */
+			pkexpr = (Expr *) copyObject(lfirst(lc));
+			ChangeVarNodes((Node *) pkexpr, 1, varno, 0);
+			lc = lnext(lc);
+		}
+
+		partexprs[cnt_pke] = list_make1(pkexpr);
+	}
+
+	return partexprs;
+}
diff --git a/src/backend/foreign/foreign.c b/src/backend/foreign/foreign.c
index 242d6d2..75c95e4 100644
--- a/src/backend/foreign/foreign.c
+++ b/src/backend/foreign/foreign.c
@@ -721,7 +721,7 @@ GetExistingLocalJoinPath(RelOptInfo *joinrel)
 {
 	ListCell   *lc;
 
-	Assert(joinrel->reloptkind == RELOPT_JOINREL);
+	Assert(IS_JOIN_REL(joinrel));
 
 	foreach(lc, joinrel->pathlist)
 	{
@@ -786,7 +786,7 @@ GetExistingLocalJoinPath(RelOptInfo *joinrel)
 			ForeignPath *foreign_path;
 
 			foreign_path = (ForeignPath *) joinpath->outerjoinpath;
-			if (foreign_path->path.parent->reloptkind == RELOPT_JOINREL)
+			if (IS_JOIN_REL(foreign_path->path.parent))
 				joinpath->outerjoinpath = foreign_path->fdw_outerpath;
 		}
 
@@ -795,7 +795,7 @@ GetExistingLocalJoinPath(RelOptInfo *joinrel)
 			ForeignPath *foreign_path;
 
 			foreign_path = (ForeignPath *) joinpath->innerjoinpath;
-			if (foreign_path->path.parent->reloptkind == RELOPT_JOINREL)
+			if (IS_JOIN_REL(foreign_path->path.parent))
 				joinpath->innerjoinpath = foreign_path->fdw_outerpath;
 		}
 
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 6955298..6d26688 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -2049,6 +2049,14 @@ _copyRestrictInfo(const RestrictInfo *from)
 	COPY_SCALAR_FIELD(hashjoinoperator);
 	COPY_SCALAR_FIELD(left_bucketsize);
 	COPY_SCALAR_FIELD(right_bucketsize);
+	/*
+	 * Do not copy parent_rinfo and child_rinfos because 1. they create a
+	 * circular dependency between child and parent RestrictInfo 2. dropping
+	 * those links just means that we loose some memory optimizations. 3. There
+	 * is a possibility that the child and parent RestrictInfots themselves may
+	 * have got copied and thus the old links may no longer be valid. The
+	 * caller may set up those links itself, if needed.
+	 */
 
 	return newnode;
 }
diff --git a/src/backend/optimizer/README b/src/backend/optimizer/README
index 775bcc3..d755f1f 100644
--- a/src/backend/optimizer/README
+++ b/src/backend/optimizer/README
@@ -974,3 +974,56 @@ be desirable to postpone the Gather stage until as near to the top of the
 plan as possible.  Expanding the range of cases in which more work can be
 pushed below the Gather (and costing them accurately) is likely to keep us
 busy for a long time to come.
+
+Partition-wise 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 for a given row in a given partition of a given
+partitioned table, its joining row, if exists, should exist only in the
+matching partition of the other partitioned table; no row from non-matching
+partitions in the other partitioned table can join with the given row from the
+first table. This condition allows the join between partitioned table 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
+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 partition tables
+into join between their partitions is called partition-wise join. We will use
+term "partitioned relation" for both partitioned table as well as join between
+partitioned tables which can use partition-wise join technique.
+
+Partitioning properties of a partitioned table are stored in
+PartitionSchemeData structure. Planner maintains a list of canonical partition
+schemes (distinct PartitionSchemeData objects) so that any two partitioned
+relations with same partitioning scheme share the same PartitionSchemeData
+object. This reduces memory consumed by PartitionSchemeData objects and makes
+it easy to compare the partition schemes of joining relations. RelOptInfos of
+partitioned relations hold partition key expressions and the RelOptInfos of
+the partition relations of that relation.
+
+Partition-wise joins are planned in two phases
+
+1. First phase creates the RelOptInfos for joins between matching partitions,
+henceforth referred to as child-joins. The number of paths created for a
+child-join i.e. join between partitions is same as the number of paths created
+for join between parents. That number grows exponentially with the number of
+base relations being joined. The time and memory consumed to create paths for
+each child-join will be proporional to the number of partitions. This will not
+scale well with thousands of partitions. Instead of that we estimate
+partition-wise join cost based on the costs of sampled child-joins. We choose
+child-joins with higher sizes to have realistic estimates. If the number of
+sampled child-joins is same as the number of live child-joins, we create append
+paths as we know costs of all required child-joins. Otherwise we create
+PartitionJoinPaths with cost estimates based on the costs of sampled
+child-joins. While creating append paths or PartitionJoin paths we create paths
+for all the different possible parameterizations and pathkeys available in the
+sampled child-joins.
+
+2. If PartitionJoinPath emerges as the best possible path, we create paths for
+each unsampled child-join. From every child-join we choose the cheapest path
+with same parameterization or pathkeys as the PartitionJoinPath. This path is
+converted into a plan and all the child-join plans are combined using an Append
+or MergeAppend plan as appropriate. We use a fresh memory context for planning
+each unsampled child-join, thus reducing memory consumption.
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 9753a26..0a2c131 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -18,8 +18,10 @@
 #include <limits.h>
 #include <math.h>
 
+#include "miscadmin.h"
 #include "access/sysattr.h"
 #include "access/tsmapi.h"
+#include "catalog/partition.h"
 #include "catalog/pg_class.h"
 #include "catalog/pg_operator.h"
 #include "catalog/pg_proc.h"
@@ -44,6 +46,7 @@
 #include "parser/parsetree.h"
 #include "rewrite/rewriteManip.h"
 #include "utils/lsyscache.h"
+#include "utils/rel.h"
 
 
 /* results of subquery_is_pushdown_safe */
@@ -93,8 +96,8 @@ static void set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 static void set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 						Index rti, RangeTblEntry *rte);
 static void generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
-						   List *live_childrels,
-						   List *all_child_pathkeys);
+						   List *live_childrels, List *all_child_pathkeys,
+						   bool partition_join_path);
 static Path *get_cheapest_parameterized_child_path(PlannerInfo *root,
 									  RelOptInfo *rel,
 									  Relids required_outer);
@@ -126,6 +129,8 @@ static void subquery_push_qual(Query *subquery,
 static void recurse_push_qual(Node *setOp, Query *topquery,
 				  RangeTblEntry *rte, Index rti, Node *qual);
 static void remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel);
+static void add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
+								List *live_childrels, bool partition_join_path);
 
 
 /*
@@ -868,6 +873,34 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 	double	   *parent_attrsizes;
 	int			nattrs;
 	ListCell   *l;
+	Oid		   *part_oids = NULL;
+	int			nparts = 0;
+
+	/*
+	 * We require OIDs of the partitions to arrange the child RelOptInfos to
+	 * match the lists/ranges specified in the partitioning scheme. Fetch those
+	 * here so as keep those handy when going to child RelOptInfos below.
+	 */
+	if (rel->part_scheme)
+	{
+		RangeTblEntry	*rte = root->simple_rte_array[rel->relid];
+
+		/*
+		 * We need not lock the relation since it was already locked, either by
+		 * the rewriter or when expand_inherited_rtentry() added it to the
+		 * query's rangetable.
+		 */
+		Relation	relation = heap_open(rte->relid, NoLock);
+		PartitionDesc	part_desc = RelationGetPartitionDesc(relation);
+
+		part_oids = part_desc->oids;
+		nparts = part_desc->nparts;
+
+		Assert(part_oids && nparts > 0);
+
+		rel->part_rels = (RelOptInfo **)palloc0(sizeof(RelOptInfo *) * nparts);
+		heap_close(relation, NoLock);
+	}
 
 	/*
 	 * Initialize to compute size estimates for whole append relation.
@@ -899,6 +932,8 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		Node	   *childqual;
 		ListCell   *parentvars;
 		ListCell   *childvars;
+		int			cnt_parts;
+		List	   *appinfos = list_make1(appinfo);
 
 		/* append_rel_list contains all append rels; ignore others */
 		if (appinfo->parent_relid != parentRTindex)
@@ -912,8 +947,95 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		 * add_base_rels_to_query.
 		 */
 		childrel = find_base_rel(root, childRTindex);
+
+		/*
+		 * Recursively save topmost parent's relid in RelOptInfos of
+		 * partitions.
+		 */
+		if (rel->top_parent_relids)
+			childrel->top_parent_relids = rel->top_parent_relids;
+		else
+			childrel->top_parent_relids = bms_copy(rel->relids);
+
 		Assert(childrel->reloptkind == RELOPT_OTHER_MEMBER_REL);
 
+
+		/*
+		 * For two partitioned tables with the same partitioning scheme, it is
+		 * assumed that the Oids of matching partitions from both the tables
+		 * are placed at the same position in the array of partition oids in
+		 * respective partition descriptors. Saving the RelOptInfo of a
+		 * partition in the same cardinal position as its Oid makes it easy to
+		 * find the RelOptInfos of matching partitions for partition-wise join.
+		 */
+		for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+		{
+			if (part_oids[cnt_parts] == childRTE->relid)
+			{
+				Assert(!rel->part_rels[cnt_parts]);
+				rel->part_rels[cnt_parts] = childrel;
+			}
+		}
+
+		/*
+		 * For a partitioned tables, individual partitions can participate in
+		 * the pair-wise joins. We need attr_needed data for building pair-wise
+		 * join relations. Partition tables should have same layout as the
+		 * parent table and hence should not need any translation. But rest of
+		 * the code still uses inheritance mechanism. So does this code. For
+		 * other inherited children, attr_needed is only examined for base
+		 * relations, no otherrels. So we compute attr_needed only for children
+		 * of a partitioned table.
+		 */
+		if (rel->part_scheme)
+		{
+			AttrNumber attno;
+			for (attno = rel->min_attr; attno <= rel->max_attr; attno++)
+			{
+				int	index = attno - rel->min_attr;
+				Relids	attr_needed = bms_copy(rel->attr_needed[index]);
+
+				/*
+				 * System attributes do not need translation. In such a case,
+				 * the attribute numbers of the parent and the child should
+				 * start from the same minimum attribute.
+				 */
+				if (attno <= 0)
+				{
+					Assert(rel->min_attr == childrel->min_attr);
+					childrel->attr_needed[index] = attr_needed;
+				}
+				else
+				{
+					Var *var = list_nth(appinfo->translated_vars,
+										attno - 1);
+					int child_index;
+
+					/* Parent Var translates to child Var. */
+					Assert(IsA(var, Var));
+
+					child_index = var->varattno - childrel->min_attr;
+					childrel->attr_needed[child_index] = attr_needed;
+				}
+			}
+		}
+
+		/*
+		 * Copy/Modify targetlist. Even if this child is deemed empty, we need
+		 * its targetlist in case it falls on nullable side in a child-join
+		 * because of partition-wise join.
+		 *
+		 * NB: the resulting childrel->reltarget->exprs may contain arbitrary
+		 * expressions, which otherwise would not occur in a rel's targetlist.
+		 * Code that might be looking at an appendrel child must cope with
+		 * such.  (Normally, a rel's targetlist would only include Vars and
+		 * PlaceHolderVars.)  XXX we do not bother to update the cost or width
+		 * fields of childrel->reltarget; not clear if that would be useful.
+		 */
+		childrel->reltarget->exprs = (List *)
+			adjust_appendrel_attrs(root,
+								   (Node *) rel->reltarget->exprs, appinfos);
+
 		/*
 		 * We have to copy the parent's targetlist and quals to the child,
 		 * with appropriate substitution of variables.  However, only the
@@ -931,7 +1053,7 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		childquals = get_all_actual_clauses(rel->baserestrictinfo);
 		childquals = (List *) adjust_appendrel_attrs(root,
 													 (Node *) childquals,
-													 appinfo);
+													 appinfos);
 		childqual = eval_const_expressions(root, (Node *)
 										   make_ands_explicit(childquals));
 		if (childqual && IsA(childqual, Const) &&
@@ -960,24 +1082,9 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 			continue;
 		}
 
-		/*
-		 * CE failed, so finish copying/modifying targetlist and join quals.
-		 *
-		 * NB: the resulting childrel->reltarget->exprs may contain arbitrary
-		 * expressions, which otherwise would not occur in a rel's targetlist.
-		 * Code that might be looking at an appendrel child must cope with
-		 * such.  (Normally, a rel's targetlist would only include Vars and
-		 * PlaceHolderVars.)  XXX we do not bother to update the cost or width
-		 * fields of childrel->reltarget; not clear if that would be useful.
-		 */
-		childrel->joininfo = (List *)
-			adjust_appendrel_attrs(root,
-								   (Node *) rel->joininfo,
-								   appinfo);
-		childrel->reltarget->exprs = (List *)
-			adjust_appendrel_attrs(root,
-								   (Node *) rel->reltarget->exprs,
-								   appinfo);
+		/* CE failed, so finish copying/modifying join quals. */
+		childrel->joininfo = build_child_clauses(root, rel->joininfo,
+												 appinfos);
 
 		/*
 		 * We have to make child entries in the EquivalenceClass data
@@ -992,14 +1099,6 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		childrel->has_eclass_joins = rel->has_eclass_joins;
 
 		/*
-		 * Note: we could compute appropriate attr_needed data for the child's
-		 * variables, by transforming the parent's attr_needed through the
-		 * translated_vars mapping.  However, currently there's no need
-		 * because attr_needed is only examined for base relations not
-		 * otherrels.  So we just leave the child's attr_needed empty.
-		 */
-
-		/*
 		 * If parallelism is allowable for this query in general, see whether
 		 * it's allowable for this childrel in particular.  But if we've
 		 * already decided the appendrel is not parallel-safe as a whole,
@@ -1080,6 +1179,16 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		}
 	}
 
+	/* Should have found all the childrels of a partitioned relation. */
+	if (rel->part_scheme)
+	{
+		int		cnt_parts;
+		for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+			if (!rel->part_rels[cnt_parts])
+				elog(ERROR, "could not find the RelOptInfo of a partition with oid %u",
+					 part_oids[cnt_parts]);
+	}
+
 	if (has_live_children)
 	{
 		/*
@@ -1122,19 +1231,11 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 {
 	int			parentRTindex = rti;
 	List	   *live_childrels = NIL;
-	List	   *subpaths = NIL;
-	bool		subpaths_valid = true;
-	List	   *partial_subpaths = NIL;
-	bool		partial_subpaths_valid = true;
-	List	   *all_child_pathkeys = NIL;
-	List	   *all_child_outers = NIL;
 	ListCell   *l;
 
 	/*
-	 * Generate access paths for each member relation, and remember the
-	 * cheapest path for each one.  Also, identify all pathkeys (orderings)
-	 * and parameterizations (required_outer sets) available for the member
-	 * relations.
+	 * Generate access paths for each member relation and remember the
+	 * non-dummy children.
 	 */
 	foreach(l, root->append_rel_list)
 	{
@@ -1142,7 +1243,6 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 		int			childRTindex;
 		RangeTblEntry *childRTE;
 		RelOptInfo *childrel;
-		ListCell   *lcp;
 
 		/* append_rel_list contains all append rels; ignore others */
 		if (appinfo->parent_relid != parentRTindex)
@@ -1177,6 +1277,64 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 		 * Child is live, so add it to the live_childrels list for use below.
 		 */
 		live_childrels = lappend(live_childrels, childrel);
+	}
+
+	/* Add Append/MergeAppend paths to the "append" relation. */
+	add_paths_to_append_rel(root, rel, live_childrels, false);
+}
+
+/*
+ * add_paths_to_append_rel
+ *		Generate Append/MergeAppend paths for given "append" relation. An
+ *		"append" relation can be a base parent relation or a join between
+ *		partitioned tables.
+ *
+ * The function collects all parameterizations and orderings supported by the
+ * non-dummy children. For every such parameterization or ordering, it creates
+ * an append path collecting one path from each non-dummy child with given
+ * parameterization or ordering. Similarly it collects partial paths from
+ * non-dummy children to create partial append paths.
+ */
+static void
+add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
+						List *live_childrels, bool partition_join_path)
+{
+	List	   *subpaths = NIL;
+	bool		subpaths_valid = true;
+	List	   *partial_subpaths = NIL;
+	bool		partial_subpaths_valid;
+	List	   *all_child_pathkeys = NIL;
+	List	   *all_child_outers = NIL;
+	ListCell   *l;
+
+	/*
+	 * While creating PartitionJoinPath, we sample paths from only a few child
+	 * relations. Even if all of sampled children have partial paths, it's not
+	 * guaranteed that all the unsampled children will have partial paths.
+	 * Hence we do not create partial PartitionJoinPaths.
+	 */
+	partial_subpaths_valid = !partition_join_path ? true : false;
+
+	/* An append relation with all its children dummy is dummy. */
+	if (live_childrels == NIL)
+	{
+		/* Mark the relation as dummy, if not already done so. */
+		if (!IS_DUMMY_REL(rel))
+			set_dummy_rel_pathlist(rel);
+
+		/* No more paths need to be added. */
+		return;
+	}
+
+	/*
+	 * For every non-dummy child, remember the cheapest path.  Also, identify
+	 * all pathkeys (orderings) and parameterizations (required_outer sets)
+	 * available for the non-dummy member relations.
+	 */
+	foreach (l, live_childrels)
+	{
+		RelOptInfo *childrel = lfirst(l);
+		ListCell   *lcp;
 
 		/*
 		 * If child has an unparameterized cheapest-total path, add that to
@@ -1267,7 +1425,17 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 	 * if we have zero or one live subpath due to constraint exclusion.)
 	 */
 	if (subpaths_valid)
-		add_path(rel, (Path *) create_append_path(rel, subpaths, NULL, 0));
+	{
+		Path *path;
+
+		if (partition_join_path)
+			path = (Path *) create_partition_join_path(rel, subpaths,
+													   NULL);
+		else
+			path = (Path *) create_append_path(rel, subpaths, NULL, 0);
+
+		add_path(rel, path);
+	}
 
 	/*
 	 * Consider an append of partial unordered, unparameterized partial paths.
@@ -1278,6 +1446,8 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 		ListCell   *lc;
 		int			parallel_workers = 0;
 
+		Assert(!partition_join_path);
+
 		/*
 		 * Decide on the number of workers to request for this append path.
 		 * For now, we just use the maximum value from among the members.  It
@@ -1304,7 +1474,7 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 	 */
 	if (subpaths_valid)
 		generate_mergeappend_paths(root, rel, live_childrels,
-								   all_child_pathkeys);
+								   all_child_pathkeys, partition_join_path);
 
 	/*
 	 * Build Append paths for each parameterization seen among the child rels.
@@ -1345,8 +1515,16 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 		}
 
 		if (subpaths_valid)
-			add_path(rel, (Path *)
-					 create_append_path(rel, subpaths, required_outer, 0));
+		{
+			Path *path;
+
+			if (partition_join_path)
+				path = (Path *) create_partition_join_path(rel, subpaths, required_outer);
+			else
+				path = (Path *) create_append_path(rel, subpaths, required_outer, 0);
+
+			add_path(rel, path);
+		}
 	}
 }
 
@@ -1376,7 +1554,7 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 static void
 generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
 						   List *live_childrels,
-						   List *all_child_pathkeys)
+						   List *all_child_pathkeys, bool partition_join_path)
 {
 	ListCell   *lcp;
 
@@ -1387,6 +1565,7 @@ generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
 		List	   *total_subpaths = NIL;
 		bool		startup_neq_total = false;
 		ListCell   *lcr;
+		Path	   *path;
 
 		/* Select the child paths for this ordering... */
 		foreach(lcr, live_childrels)
@@ -1434,17 +1613,29 @@ generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
 		}
 
 		/* ... and build the MergeAppend paths */
-		add_path(rel, (Path *) create_merge_append_path(root,
-														rel,
-														startup_subpaths,
-														pathkeys,
-														NULL));
+		if (partition_join_path)
+			path = (Path *) create_partition_join_path_with_pathkeys(root, rel,
+															 startup_subpaths,
+															 pathkeys, NULL);
+		else
+			path = (Path *) create_merge_append_path(root, rel,
+													 startup_subpaths,
+													 pathkeys, NULL);
+		add_path(rel, path);
+
 		if (startup_neq_total)
-			add_path(rel, (Path *) create_merge_append_path(root,
-															rel,
-															total_subpaths,
-															pathkeys,
-															NULL));
+		{
+			if (partition_join_path)
+				path = (Path *) create_partition_join_path_with_pathkeys(root,
+																rel,
+																total_subpaths,
+																pathkeys, NULL);
+			else
+				path = (Path *) create_merge_append_path(root, rel,
+														 total_subpaths,
+														 pathkeys, NULL);
+			add_path(rel, path);
+		}
 	}
 }
 
@@ -2186,15 +2377,27 @@ standard_join_search(PlannerInfo *root, int levels_needed, List *initial_rels)
 
 		/*
 		 * Run generate_gather_paths() for each just-processed joinrel.  We
-		 * could not do this earlier because both regular and partial paths
-		 * can get added to a particular joinrel at multiple times within
-		 * join_search_one_level.  After that, we're done creating paths for
-		 * the joinrel, so run set_cheapest().
+		 * could not do this earlier because both regular and partial paths can
+		 * get added to a particular joinrel at multiple times within
+		 * join_search_one_level.
+		 *
+		 * Similarly, create paths for joinrels which used partition-wise join
+		 * technique. generate_partition_wise_join_paths() creates paths for
+		 * only few of the child-joins with highest sizes. Though we calculate
+		 * size of a child-join only once; when it gets created, it may be
+		 * deemed empty while considering various join orders within
+		 * join_search_one_level.
+		 *
+		 * After that, we're done creating paths for the joinrel, so run
+		 * set_cheapest().
 		 */
 		foreach(lc, root->join_rel_level[lev])
 		{
 			rel = (RelOptInfo *) lfirst(lc);
 
+			/* Create paths for partition-wise joins. */
+			generate_partition_wise_join_paths(root, rel);
+
 			/* Create GatherPaths for any useful partial paths for rel */
 			generate_gather_paths(root, rel);
 
@@ -2866,6 +3069,151 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel)
 	}
 }
 
+/* Fraction of child relations to base cost on. */
+#define FRACTION_PARTS_TO_PLAN 0.01
+
+/*
+ * generate_partition_wise_join_paths
+ *
+ * 		Create paths representing partition-wise join for given partitioned
+ * 		join relation.
+ *
+ * The number of paths created for a child-join is same as the number of paths
+ * created for join between parents. That number grows exponentially with the
+ * number of base relations being joined. The time and memory consumed to
+ * create paths for each child-join will be proporional to the number of
+ * partitions. This will not scale well with thousands of partitions. Instead
+ * of that we estimate partition-wise join cost based on the costs of sampled
+ * child-joins. We choose child-joins with higher sizes to have realistic
+ * estimates.
+ *
+ * This must be called after we have considered all joining orders since
+ * certain join orders may allow us to deem a child-join as dummy.
+ */
+void
+generate_partition_wise_join_paths(PlannerInfo *root, RelOptInfo *rel)
+{
+	List   *sampled_children = NIL;
+	List   *ordered_child_nos = NIL;
+	int		cnt_part;
+	int		num_part_to_plan;
+	int		num_parts;
+	bool	partition_join_path = false;
+	int		num_dummy_parts = 0;
+	ListCell   *lc;
+
+	/* Handle only join relations. */
+	if (!IS_JOIN_REL(rel))
+		return;
+
+	/*
+	 * If none of the join orders for this relation could use partition-wise
+	 * join technique, the join is not partitioned. Reset the partitioning
+	 * scheme.
+	 */
+	if (!rel->part_rels)
+		rel->part_scheme = NULL;
+
+	/* If the relation is not partitioned or is proven dummy, nothing to do. */
+	if (!rel->part_scheme || IS_DUMMY_REL(rel))
+		return;
+
+	/* Guard against stack overflow due to overly deep partition hierarchy. */
+	check_stack_depth();
+
+	num_parts = rel->part_scheme->nparts;
+
+	/* Calculate number of child-joins to sample. */
+	num_part_to_plan = num_parts * FRACTION_PARTS_TO_PLAN;
+	if (num_part_to_plan < 1)
+		num_part_to_plan = 1;
+
+	/* Order the child-join relations by their size. */
+	for (cnt_part = 0; cnt_part < num_parts; cnt_part++)
+	{
+		RelOptInfo *child_rel = rel->part_rels[cnt_part];
+		ListCell   *insert_after;
+
+		insert_after = NULL;
+
+		/* Dummy children will not be scanned, so ingore those. */
+		if (IS_DUMMY_REL(child_rel))
+		{
+			num_dummy_parts++;
+			continue;
+		}
+
+		/*
+		 * Add this relation to the list of samples ordered by the increasing
+		 * number of rows at appropriate place.
+		 */
+		foreach (lc, ordered_child_nos)
+		{
+			int	child_no = lfirst_int(lc);
+			RelOptInfo *other_childrel = rel->part_rels[child_no];
+
+			/*
+			 * Keep track of child with lowest number of rows but higher than the
+			 * that of the child being inserted. Insert the child before a
+			 * child with highest number of rows lesser than it.
+			 */
+			if (child_rel->rows <= other_childrel->rows)
+				insert_after = lc;
+			else
+				break;
+		}
+
+		if (insert_after)
+			lappend_cell_int(ordered_child_nos, insert_after, cnt_part);
+		else
+			ordered_child_nos = lcons_int(cnt_part, ordered_child_nos);
+	}
+
+	/*
+	 * Create paths for the child-joins as they appear in the list ordered by
+	 * their size. Stop when we have created paths for required number of
+	 * child-joins.
+	 */
+	foreach (lc, ordered_child_nos)
+	{
+		int	child_no = lfirst_int(lc);
+		RelOptInfo *child_rel = rel->part_rels[child_no];
+
+		/* Create paths for this child. */
+		add_paths_to_child_joinrel(root, rel, child_no);
+
+		/* Dummy children will not be scanned, so ingore those. */
+		if (IS_DUMMY_REL(child_rel))
+		{
+			num_dummy_parts++;
+			continue;
+		}
+
+#ifdef OPTIMIZER_DEBUG
+		debug_print_rel(root, rel);
+#endif
+
+		sampled_children = lappend(sampled_children, child_rel);
+
+		if (list_length(sampled_children) >= num_part_to_plan)
+			break;
+	}
+
+	/*
+	 * If the number of samples is same as the number of live children, an
+	 * append path will do. Otherwise, we will cost the partition-wise join
+	 * based on the sampled children using PartitionJoinPath.
+	 */
+	if (num_part_to_plan < num_parts - num_dummy_parts)
+		partition_join_path = true;
+
+	/* Add paths for partition-wise join based on the sampled children. */
+	add_paths_to_append_rel(root, rel, sampled_children, partition_join_path);
+
+	if (sampled_children)
+		list_free(sampled_children);
+}
+
 /*****************************************************************************
  *			DEBUG SUPPORT
  *****************************************************************************/
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 415edad..ba2b238 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -126,6 +126,8 @@ bool		enable_nestloop = true;
 bool		enable_material = true;
 bool		enable_mergejoin = true;
 bool		enable_hashjoin = true;
+bool		enable_partition_wise_join = true;
+double		partition_wise_plan_weight = DEFAULT_PARTITION_WISE_PLAN_WEIGHT;
 
 typedef struct
 {
diff --git a/src/backend/optimizer/path/equivclass.c b/src/backend/optimizer/path/equivclass.c
index 0e50ad5..b1ea2ba 100644
--- a/src/backend/optimizer/path/equivclass.c
+++ b/src/backend/optimizer/path/equivclass.c
@@ -972,6 +972,10 @@ generate_base_implied_equalities_broken(PlannerInfo *root,
  * appropriate clauses using child EC members.  add_child_rel_equivalences
  * must already have been done for the child rel.
  *
+ * For a join between child relations, joinrelids, outer_relids and
+ * inner_rel all point to child relations. In this case, we need to find the
+ * parent relids to search the applicable equivalence classes.
+ *
  * The results are sufficient for use in merge, hash, and plain nestloop join
  * methods.  We do not worry here about selecting clauses that are optimal
  * for use in a parameterized indexscan.  indxpath.c makes its own selections
@@ -1021,12 +1025,27 @@ generate_join_implied_equalities_for_ecs(PlannerInfo *root,
 	ListCell   *lc;
 
 	/* If inner rel is a child, extra setup work is needed */
-	if (inner_rel->reloptkind == RELOPT_OTHER_MEMBER_REL)
+	if (IS_OTHER_REL(inner_rel))
 	{
+		RelOptInfo *outer_rel;
+		Relids		nominal_outer_relids;
+
+		if (bms_num_members(outer_relids) > 1)
+			outer_rel = find_join_rel(root, outer_relids);
+		else
+			outer_rel = find_base_rel(root, bms_singleton_member(outer_relids));
+
 		/* Fetch relid set for the topmost parent rel */
-		nominal_inner_relids = find_childrel_top_parent(root, inner_rel)->relids;
-		/* ECs will be marked with the parent's relid, not the child's */
-		nominal_join_relids = bms_union(outer_relids, nominal_inner_relids);
+		nominal_inner_relids = find_childrel_top_parent(root, inner_rel);
+
+		/* ECs will be marked with the parent's relid, not the child's. */
+		if (outer_rel && IS_OTHER_REL(outer_rel))
+		{
+			nominal_outer_relids = find_childrel_top_parent(root, outer_rel);
+			nominal_join_relids = bms_union(nominal_outer_relids, nominal_inner_relids);
+		}
+		else
+			nominal_join_relids = bms_union(outer_relids, nominal_inner_relids);
 	}
 	else
 	{
@@ -2062,7 +2081,7 @@ add_child_rel_equivalences(PlannerInfo *root,
 				child_expr = (Expr *)
 					adjust_appendrel_attrs(root,
 										   (Node *) cur_em->em_expr,
-										   appinfo);
+										   list_make1(appinfo));
 
 				/*
 				 * Transform em_relids to match.  Note we do *not* do
@@ -2364,8 +2383,8 @@ eclass_useful_for_merging(PlannerInfo *root,
 	 */
 
 	/* If specified rel is a child, we must consider the topmost parent rel */
-	if (rel->reloptkind == RELOPT_OTHER_MEMBER_REL)
-		relids = find_childrel_top_parent(root, rel)->relids;
+	if (IS_OTHER_REL(rel))
+		relids = find_childrel_top_parent(root, rel);
 	else
 		relids = rel->relids;
 
diff --git a/src/backend/optimizer/path/joinpath.c b/src/backend/optimizer/path/joinpath.c
index b5cbcf4..d794038 100644
--- a/src/backend/optimizer/path/joinpath.c
+++ b/src/backend/optimizer/path/joinpath.c
@@ -25,9 +25,19 @@
 /* Hook for plugins to get control in add_paths_to_joinrel() */
 set_join_pathlist_hook_type set_join_pathlist_hook = NULL;
 
-#define PATH_PARAM_BY_REL(path, rel)  \
+/*
+ * Paths parameterized by the parent can be considered to be parameterized by
+ * any of its child.
+ */
+#define PATH_PARAM_BY_PARENT(path, rel)	\
+	((path)->param_info && bms_overlap(PATH_REQ_OUTER(path),	\
+									   (rel)->top_parent_relids))
+#define PATH_PARAM_BY_REL_SELF(path, rel)  \
 	((path)->param_info && bms_overlap(PATH_REQ_OUTER(path), (rel)->relids))
 
+#define PATH_PARAM_BY_REL(path, rel)	\
+	(PATH_PARAM_BY_REL_SELF(path, rel) || PATH_PARAM_BY_PARENT(path, rel))
+
 static void sort_inner_and_outer(PlannerInfo *root, RelOptInfo *joinrel,
 					 RelOptInfo *outerrel, RelOptInfo *innerrel,
 					 JoinType jointype, JoinPathExtraData *extra);
@@ -141,6 +151,19 @@ add_paths_to_joinrel(PlannerInfo *root,
 	foreach(lc, root->join_info_list)
 	{
 		SpecialJoinInfo *sjinfo2 = (SpecialJoinInfo *) lfirst(lc);
+		Relids	joinrelids;
+
+		/*
+		 * PlannerInfo doesn't contain the SpecialJoinInfos created for joins
+		 * between child relations, even if there is a SpecialJoinInfo node for
+		 * the join between the topmost parents. Hence while calculating Relids
+		 * set representing the restriction, consider relids of topmost parent
+		 * of partitions.
+		 */
+		if (joinrel->reloptkind == RELOPT_OTHER_JOINREL)
+			joinrelids = joinrel->top_parent_relids;
+		else
+			joinrelids = joinrel->relids;
 
 		/*
 		 * SJ is relevant to this join if we have some part of its RHS
@@ -149,16 +172,16 @@ add_paths_to_joinrel(PlannerInfo *root,
 		 * join has already been proven legal.)  If the SJ is relevant, it
 		 * presents constraints for joining to anything not in its RHS.
 		 */
-		if (bms_overlap(joinrel->relids, sjinfo2->min_righthand) &&
-			!bms_overlap(joinrel->relids, sjinfo2->min_lefthand))
+		if (bms_overlap(joinrelids, sjinfo2->min_righthand) &&
+			!bms_overlap(joinrelids, sjinfo2->min_lefthand))
 			extra.param_source_rels = bms_join(extra.param_source_rels,
 										   bms_difference(root->all_baserels,
 													sjinfo2->min_righthand));
 
 		/* full joins constrain both sides symmetrically */
 		if (sjinfo2->jointype == JOIN_FULL &&
-			bms_overlap(joinrel->relids, sjinfo2->min_lefthand) &&
-			!bms_overlap(joinrel->relids, sjinfo2->min_righthand))
+			bms_overlap(joinrelids, sjinfo2->min_lefthand) &&
+			!bms_overlap(joinrelids, sjinfo2->min_righthand))
 			extra.param_source_rels = bms_join(extra.param_source_rels,
 										   bms_difference(root->all_baserels,
 													 sjinfo2->min_lefthand));
@@ -288,6 +311,22 @@ try_nestloop_path(PlannerInfo *root,
 	JoinCostWorkspace workspace;
 
 	/*
+	 * For a join between child relations, if the inner path is parameterized
+	 * by the parent of the outer relation, create a nestloop join path with
+	 * inner relation parameterized by the outer relation by translating the
+	 * inner path to be parameterized by the outer child relation.
+	 */
+	if (PATH_PARAM_BY_PARENT(inner_path, outer_path->parent))
+	{
+		inner_path = reparameterize_path_by_child(root, inner_path,
+												   outer_path->parent);
+
+		/* If we could not translate the path, don't produce nest loop path. */
+		if (!inner_path)
+			return;
+	}
+
+	/*
 	 * Check to see if proposed path is still parameterized, and reject if the
 	 * parameterization wouldn't be sensible --- unless allow_star_schema_join
 	 * says to allow it anyway.  Also, we must reject if have_dangerous_phv
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 01d4fea..7839f0f 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -14,9 +14,14 @@
  */
 #include "postgres.h"
 
+#include "miscadmin.h"
+#include "catalog/partition.h"
+#include "optimizer/clauses.h"
 #include "optimizer/joininfo.h"
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
+#include "optimizer/prep.h"
+#include "optimizer/cost.h"
 #include "utils/memutils.h"
 
 
@@ -32,7 +37,19 @@ static bool is_dummy_rel(RelOptInfo *rel);
 static void mark_dummy_rel(RelOptInfo *rel);
 static bool restriction_is_constant_false(List *restrictlist,
 							  bool only_pushed_down);
-
+static void populate_joinrel_with_paths(PlannerInfo *root, RelOptInfo *rel1,
+							RelOptInfo *rel2, RelOptInfo *joinrel,
+							SpecialJoinInfo *sjinfo, List *restrictlist);
+static void try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1,
+						  RelOptInfo *rel2, RelOptInfo *joinrel,
+						  SpecialJoinInfo *parent_sjinfo,
+						  List *parent_restrictlist);
+static SpecialJoinInfo *build_child_join_sjinfo(PlannerInfo *root,
+									SpecialJoinInfo *parent_sjinfo,
+									Relids left_relids, Relids right_relids);
+static bool have_partkey_equi_join(RelOptInfo *rel1, RelOptInfo *rel2,
+						   JoinType jointype, List *restrictlist);
+static int match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel);
 
 /*
  * join_search_one_level
@@ -724,6 +741,31 @@ make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 		return joinrel;
 	}
 
+	/* Add paths to the join relation. */
+	populate_joinrel_with_paths(root, rel1, rel2, joinrel, sjinfo,
+								restrictlist);
+
+	/* Apply partition-wise join technique, if possible. */
+	try_partition_wise_join(root, rel1, rel2, joinrel, sjinfo, restrictlist);
+
+	bms_free(joinrelids);
+
+	return joinrel;
+}
+
+/*
+ * populate_joinrel_with_paths
+ *	  Add paths to the given joinrel for given pair of joining relations. The
+ *	  SpecialJoinInfo provides details about the join and the restrictlist
+ *	  contains the join clauses and the other clauses applicable for given pair
+ *	  of the joining relations.
+ */
+
+static void
+populate_joinrel_with_paths(PlannerInfo *root, RelOptInfo *rel1,
+							RelOptInfo *rel2, RelOptInfo *joinrel,
+							SpecialJoinInfo *sjinfo, List *restrictlist)
+{
 	/*
 	 * Consider paths using each rel as both outer and inner.  Depending on
 	 * the join type, a provably empty outer or inner rel might mean the join
@@ -868,13 +910,8 @@ make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 			elog(ERROR, "unrecognized join type: %d", (int) sjinfo->jointype);
 			break;
 	}
-
-	bms_free(joinrelids);
-
-	return joinrel;
 }
 
-
 /*
  * have_join_order_restriction
  *		Detect whether the two relations should be joined to satisfy
@@ -1249,3 +1286,476 @@ restriction_is_constant_false(List *restrictlist, bool only_pushed_down)
 	}
 	return false;
 }
+
+/* Free SpecialJoinInfo. */
+static void
+free_special_join_info(SpecialJoinInfo *sjinfo)
+{
+	bms_free(sjinfo->min_lefthand);
+	bms_free(sjinfo->syn_lefthand);
+	bms_free(sjinfo->syn_righthand);
+	pfree(sjinfo);
+}
+
+/*
+ * Assess whether join between given two partitioned relations can be broken
+ * down into joins between matching partitions; a technique called
+ * "partition-wise join"
+ *
+ * Partition-wise 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.
+ *
+ * Partition-wise join is planned as follows (details: optimizer/README.)
+ *
+ * 1. Create the RelOptInfos for joins between matching partitions i.e
+ * child-joins and estimate sizes of those. This function is responsible for
+ * this phase.
+ *
+ * 2. Add paths representing partition-wise join. The second phase is
+ * implemented by generate_partition_wise_join_paths(). In order to save time
+ * and memory consumed in creating paths for every child-join, we create paths
+ * for only few child-joins.
+ *
+ * 3. Create merge/append plan to combining plans for every child-join,
+ * creating paths for remaining child-joins.
+ *
+ * The RelOptInfo, SpecialJoinInfo and restrictlist for each child join are
+ * obtained by translating the respective parent join structures.
+ */
+static void
+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;
+	PartitionedJoin	   *partitioned_join;
+
+	/* 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)
+		return;
+
+	/*
+	 * If any of the joining parent relations is proven empty, either the join
+	 * will be empty (INNER join) or will have the inner side all nullified. We
+	 * take care of such cases when creating join paths for parent relations.
+	 * Nothing to be done here. Also, nothing to do, if the parent join is
+	 * proven empty.
+	 */
+	if (IS_DUMMY_REL(rel1) || IS_DUMMY_REL(rel2) || IS_DUMMY_REL(joinrel))
+		return;
+
+	/*
+	 * Partitioning scheme in join relation indicates a possibilty that the
+	 * join may be partitioned, but it's not necessary that every pair of
+	 * joining relations can use partition-wise join technique. If one of
+	 * joining relations turns out to be unpartitioned, this pair of joining
+	 * relations can not use partition-wise join technique.
+	 */
+	if (!rel1->part_scheme || !rel2->part_scheme)
+		return;
+
+	/*
+	 * If an equi-join condition between the partition keys of the joining
+	 * relations does not exist, this pair of joining relations can not use
+	 * partition-wise technique.
+	 */
+	if (!have_partkey_equi_join(rel1, rel2, parent_sjinfo->jointype,
+								parent_restrictlist))
+		return;
+
+	/*
+	 * 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);
+
+	/* We should have RelOptInfos of the partitions available. */
+	Assert(rel1->part_rels && rel2->part_rels);
+
+	part_scheme = joinrel->part_scheme;
+	nparts = part_scheme->nparts;
+
+	/*
+	 * We do not store information about valid pairs of joining child
+	 * relations. The pair of joining relations for a child-join can be derived
+	 * from valid pairs of joining parent relations. Amongst the valid pairs of
+	 * parent joining relations, only those which result in partitioned join
+	 * matter for partition-wise join. Remember those so that we can use them
+	 * for creating paths for few child-joins in
+	 * generate_partition_wise_join_paths() later.
+	 */
+	partitioned_join = (PartitionedJoin *) palloc(sizeof(PartitionedJoin));
+	partitioned_join->rel1 = rel1;
+	partitioned_join->rel2 = rel2;
+	partitioned_join->sjinfo = copyObject(parent_sjinfo);
+	partitioned_join->restrictlist = parent_restrictlist;
+	joinrel->partitioned_joins = lappend(joinrel->partitioned_joins,
+										 partitioned_join);
+
+	elog(DEBUG3, "join between relations %s and %s is considered for partition-wise join.",
+		 bmsToString(rel1->relids), bmsToString(rel2->relids));
+
+	/* We are done if child RelOptInfos are already created. */
+	if (joinrel->part_rels)
+		return;
+
+	/* Create all the child RelOptInfos. */
+	joinrel->part_rels = (RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
+
+	/*
+	 * Create child join relations for this partitioned join. While doing so,
+	 * we estimate sizes of these child join relations. These estimates are
+	 * used to find the representative child relations used for costing the
+	 * partition-wise join later.
+	 */
+	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	{
+		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
+		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
+		SpecialJoinInfo	*child_sjinfo;
+		List	*child_restrictlist;
+		RelOptInfo *child_joinrel;
+
+		/* We should never try to join two overlapping sets of rels. */
+		Assert(!bms_overlap(child_rel1->relids, child_rel2->relids));
+
+		Assert (!joinrel->part_rels[cnt_parts]);
+
+		child_joinrel = build_child_join_rel(root, child_rel1, child_rel2,
+											 joinrel, parent_sjinfo->jointype);
+
+		joinrel->part_rels[cnt_parts] = child_joinrel;
+
+		/*
+		 * Construct restrictions applicable to the child join from
+		 * those applicable to the parent join.
+		 */
+		child_restrictlist = build_child_clauses(root, parent_restrictlist,
+												 find_appinfos_by_relids(root,
+													   child_joinrel->relids));
+
+		/*
+		 * Construct SpecialJoinInfo from parent join relations's
+		 * SpecialJoinInfo.
+		 */
+		child_sjinfo = build_child_join_sjinfo(root, parent_sjinfo,
+											   child_rel1->relids,
+											   child_rel2->relids);
+
+		/*
+		 * Set estimates of the child-joinrel's size.
+		 */
+		set_joinrel_size_estimates(root, child_joinrel, child_rel1, child_rel2,
+								   child_sjinfo, child_restrictlist);
+
+		/*
+		 * If the child relations themselves are partitioned, try partition-wise join
+		 * recursively.
+		 */
+		try_partition_wise_join(root, child_rel1, child_rel2, child_joinrel,
+								child_sjinfo, child_restrictlist);
+
+		free_special_join_info(child_sjinfo);
+		child_sjinfo = NULL;
+	}
+}
+
+/*
+ * add_paths_to_child_join
+ * 		Add paths to 'child_id'th child of given parent join relation.
+ *
+ * The function creates paths for given child-join by joining corresponding
+ * children of every pair of joining parent relations which produces
+ * partitioned join. Since we create paths only for sampled child-joins, either
+ * of the children being joined may not have paths. In that case, this function
+ * is called recursively to populate paths for those.
+ */
+void
+add_paths_to_child_joinrel(PlannerInfo *root, RelOptInfo *parent_joinrel,
+						   int child_id)
+{
+	ListCell	*lc;
+	RelOptInfo	   *child_joinrel = parent_joinrel->part_rels[child_id];
+
+	Assert(IS_JOIN_REL(parent_joinrel));
+
+	/* If this child relation already has paths, nothing to do. */
+	if (child_joinrel->cheapest_total_path)
+		return;
+
+	/* A dummy relation will have a dummy path as the cheapest path. */
+	Assert(!is_dummy_rel(child_joinrel));
+
+	/*
+	 * For every partitioned join order, calculate paths for the joining
+	 * child relations and then calculate paths for given child.
+	 */
+	foreach (lc, parent_joinrel->partitioned_joins)
+	{
+		PartitionedJoin	   *pj = lfirst(lc);
+		RelOptInfo *rel1 = pj->rel1;
+		RelOptInfo *rel2 = pj->rel2;
+		RelOptInfo *child_rel1 = rel1->part_rels[child_id];
+		RelOptInfo *child_rel2 = rel2->part_rels[child_id];
+		SpecialJoinInfo	   *child_sjinfo;
+		List	   *child_restrictlist;
+
+		/*
+		 * Add paths to joining relation if it is a join itself.
+		 * Paths for child base relations are created in
+		 * set_append_rel_pathlist().
+		 */
+		if (IS_JOIN_REL(pj->rel1))
+			add_paths_to_child_joinrel(root, rel1, child_id);
+
+		if (IS_JOIN_REL(pj->rel2))
+			add_paths_to_child_joinrel(root, rel2, child_id);
+
+		/*
+		 * Construct SpecialJoinInfo from parent join relations's
+		 * SpecialJoinInfo.
+		 */
+		child_sjinfo = build_child_join_sjinfo(root, pj->sjinfo,
+											   child_rel1->relids,
+											   child_rel2->relids);
+
+
+		/*
+		 * Construct restrictions applicable to the child join from
+		 * those applicable to the parent join.
+		 */
+		child_restrictlist = build_child_clauses(root, pj->restrictlist,
+												 find_appinfos_by_relids(root,
+													   child_joinrel->relids));
+
+		/* Add paths for child join. */
+		populate_joinrel_with_paths(root, rel1->part_rels[child_id],
+									rel2->part_rels[child_id], child_joinrel,
+									child_sjinfo, child_restrictlist);
+
+		/* Add partition-wise join paths for partitioned child-joins. */
+		generate_partition_wise_join_paths(root, child_joinrel);
+
+		free_special_join_info(child_sjinfo);
+		child_sjinfo = NULL;
+	}
+
+	set_cheapest(child_joinrel);
+}
+
+/*
+ * Construct the SpecialJoinInfo for a child-join by translating
+ * SpecialJoinInfo for the join between parents. left_relids and right_relids
+ * are the relids of left and right side of the join respectively.
+ */
+static SpecialJoinInfo *
+build_child_join_sjinfo(PlannerInfo *root, SpecialJoinInfo *parent_sjinfo,
+						Relids left_relids, Relids right_relids)
+{
+	SpecialJoinInfo *sjinfo = makeNode(SpecialJoinInfo);
+	MemoryContext	old_context;
+	List	   *left_appinfos = find_appinfos_by_relids(root, left_relids);
+	List	   *right_appinfos = find_appinfos_by_relids(root, right_relids);
+
+	memcpy(sjinfo, parent_sjinfo, sizeof(SpecialJoinInfo));
+
+	sjinfo->min_lefthand = adjust_child_relids(sjinfo->min_lefthand,
+											   left_appinfos);
+	sjinfo->min_righthand = adjust_child_relids(sjinfo->min_righthand,
+												right_appinfos);
+	sjinfo->syn_lefthand = adjust_child_relids(sjinfo->syn_lefthand,
+											   left_appinfos);
+	sjinfo->syn_righthand = adjust_child_relids(sjinfo->syn_righthand,
+												right_appinfos);
+
+	/*
+	 * Replace the Var nodes of parent with those of children in expressions.
+	 * This function may be called within a temporary context, but the
+	 * expressions will be shallow-copied into the plan. Hence copy those in
+	 * the planner's context.
+	 */
+	old_context = MemoryContextSwitchTo(root->planner_cxt);
+	sjinfo->semi_rhs_exprs = (List *) adjust_appendrel_attrs(root,
+											   (Node *) sjinfo->semi_rhs_exprs,
+															 right_appinfos);
+	MemoryContextSwitchTo(old_context);
+
+	list_free(left_appinfos);
+	list_free(right_appinfos);
+
+	return sjinfo;
+}
+
+/*
+ * Replace parent relids by child relids in the copy of given relid set.
+ */
+Relids
+adjust_child_relids(Relids relids, List *append_rel_infos)
+{
+	ListCell   *lc;
+
+	/* Ensure we have a modifiable copy. */
+	relids = bms_copy(relids);
+	foreach (lc, append_rel_infos)
+	{
+		AppendRelInfo	*appinfo = lfirst(lc);
+
+		/* Remove parent, add child */
+		if (bms_is_member(appinfo->parent_relid, relids))
+		{
+			relids = bms_del_member(relids, appinfo->parent_relid);
+			relids = bms_add_member(relids, appinfo->child_relid);
+		}
+	}
+
+	return relids;
+}
+
+/*
+ * Returns true if there exists an equi-join condition for each pair of
+ * partition key from given relations being joined.
+ */
+static bool
+have_partkey_equi_join(RelOptInfo *rel1, RelOptInfo *rel2,
+					 JoinType jointype, List *restrictlist)
+{
+	PartitionScheme	part_scheme = rel1->part_scheme;
+	ListCell	*lc;
+	int		cnt_pks;
+	int		num_pks;
+	bool   *pk_has_clause;
+
+	/*
+	 * This function should be called when the joining relations have same
+	 * partitioning scheme.
+	 */
+	Assert(rel1->part_scheme == rel2->part_scheme);
+	Assert(part_scheme);
+
+	num_pks = part_scheme->partnatts;
+
+	pk_has_clause = (bool *) palloc0(sizeof(bool) * num_pks);
+
+	foreach (lc, restrictlist)
+	{
+		RestrictInfo *rinfo = lfirst(lc);
+		OpExpr		 *opexpr;
+		Expr		 *expr1;
+		Expr		 *expr2;
+		int		ipk1;
+		int		ipk2;
+
+		/* If processing an outer join, only use its own join clauses. */
+		if (IS_OUTER_JOIN(jointype) && rinfo->is_pushed_down)
+			continue;
+
+		/* Skip clauses which can not be used for a join. */
+		if (!rinfo->can_join)
+			continue;
+
+		/* Skip clauses which are not equality conditions. */
+		if (rinfo->hashjoinoperator == InvalidOid && !rinfo->mergeopfamilies)
+			continue;
+
+		opexpr = (OpExpr *) rinfo->clause;
+		Assert(is_opclause(opexpr));
+
+
+		/* Match the operands to the relation. */
+		if (bms_is_subset(rinfo->left_relids, rel1->relids) &&
+			bms_is_subset(rinfo->right_relids, rel2->relids))
+		{
+			expr1 = linitial(opexpr->args);
+			expr2 = lsecond(opexpr->args);
+		}
+		else if (bms_is_subset(rinfo->left_relids, rel2->relids) &&
+				 bms_is_subset(rinfo->right_relids, rel1->relids))
+		{
+			expr1 = lsecond(opexpr->args);
+			expr2 = linitial(opexpr->args);
+		}
+		else
+			continue;
+
+		/* Associate matching clauses with partition keys. */
+		ipk1 = match_expr_to_partition_keys(expr1, rel1);
+		ipk2 = match_expr_to_partition_keys(expr2, rel2);
+
+		/*
+		 * If the clause refers to different partition keys from
+		 * both relations, it can not be used for partition-wise join.
+		 */
+		if (ipk1 != ipk2)
+			continue;
+
+		/*
+		 * The clause allows partition-wise join if only it uses the same
+		 * operator family as that specified by the partition key.
+		 */
+		if (!list_member_oid(rinfo->mergeopfamilies,
+							 part_scheme->partopfamily[ipk1]))
+			continue;
+
+		/* Mark the partition key as having an equi-join clause. */
+		pk_has_clause[ipk1] = true;
+	}
+
+	/* Check whether every partition key has an equi-join condition. */
+	for (cnt_pks = 0; cnt_pks < num_pks; cnt_pks++)
+	{
+		if (!pk_has_clause[cnt_pks])
+		{
+			pfree(pk_has_clause);
+			return false;
+		}
+	}
+
+	pfree(pk_has_clause);
+	return true;
+}
+
+/*
+ * Find the partition key from the given relation matching the given
+ * expression. If found, return the index of the partition key, else return -1.
+ */
+static int
+match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel)
+{
+	int		cnt_pks;
+	int		num_pks;
+
+	/* This function should be called only for partitioned relations. */
+	Assert(rel->part_scheme);
+
+	num_pks = rel->part_scheme->partnatts;
+
+	/*
+	 * Remove the relabel decoration. We can assume that there is at most one
+	 * RelabelType node; eval_const_expressions() simplifies multiple
+	 * RelabelType nodes into one.
+	 */
+	if (IsA(expr, RelabelType))
+		expr = (Expr *) ((RelabelType *) expr)->arg;
+
+	for (cnt_pks = 0; cnt_pks < num_pks; cnt_pks++)
+	{
+		List	 *pkexprs = rel->partexprs[cnt_pks];
+		ListCell *lc;
+
+		foreach(lc, pkexprs)
+		{
+			Expr *pkexpr = lfirst(lc);
+			if (equal(pkexpr, expr))
+				return cnt_pks;
+		}
+	}
+
+	return -1;
+}
diff --git a/src/backend/optimizer/path/pathkeys.c b/src/backend/optimizer/path/pathkeys.c
index 4436ac1..def64e3 100644
--- a/src/backend/optimizer/path/pathkeys.c
+++ b/src/backend/optimizer/path/pathkeys.c
@@ -1088,12 +1088,25 @@ select_outer_pathkeys_for_merge(PlannerInfo *root,
 	int			necs;
 	ListCell   *lc;
 	int			j;
+	Relids		relids;
 
 	/* Might have no mergeclauses */
 	if (nClauses == 0)
 		return NIL;
 
 	/*
+	 * For a child join relation, use parent relids to find potential join
+	 * partners (see code below) from equivalence classes. A potential join
+	 * partner of parent also indicates potential join partner of the child. By
+	 * using only parent relids, we avoid scoring an equivalence class multiple
+	 * times once for parent and then for all of its children.
+	 */
+	if (joinrel->reloptkind == RELOPT_OTHER_JOINREL)
+		relids = joinrel->top_parent_relids;
+	else
+		relids = joinrel->relids;
+
+	/*
 	 * Make arrays of the ECs used by the mergeclauses (dropping any
 	 * duplicates) and their "popularity" scores.
 	 */
@@ -1133,7 +1146,7 @@ select_outer_pathkeys_for_merge(PlannerInfo *root,
 
 			/* Potential future join partner? */
 			if (!em->em_is_const && !em->em_is_child &&
-				!bms_overlap(em->em_relids, joinrel->relids))
+				!bms_overlap(em->em_relids, relids))
 				score++;
 		}
 
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index ad49674..3a223c8 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -30,6 +30,7 @@
 #include "optimizer/clauses.h"
 #include "optimizer/cost.h"
 #include "optimizer/paths.h"
+#include "optimizer/pathnode.h"
 #include "optimizer/placeholder.h"
 #include "optimizer/plancat.h"
 #include "optimizer/planmain.h"
@@ -42,6 +43,7 @@
 #include "parser/parse_clause.h"
 #include "parser/parsetree.h"
 #include "utils/lsyscache.h"
+#include "utils/memutils.h"
 
 
 /*
@@ -145,6 +147,9 @@ static CustomScan *create_customscan_plan(PlannerInfo *root,
 static NestLoop *create_nestloop_plan(PlannerInfo *root, NestPath *best_path);
 static MergeJoin *create_mergejoin_plan(PlannerInfo *root, MergePath *best_path);
 static HashJoin *create_hashjoin_plan(PlannerInfo *root, HashPath *best_path);
+static Plan *create_partition_plan(PlannerInfo *root, Path *best_path);
+static Plan *create_partition_join_plan(PlannerInfo *root,
+										PartitionJoinPath *best_path);
 static Node *replace_nestloop_params(PlannerInfo *root, Node *expr);
 static Node *replace_nestloop_params_mutator(Node *node, PlannerInfo *root);
 static void process_subquery_nestloop_params(PlannerInfo *root,
@@ -241,7 +246,8 @@ static Plan *prepare_sort_from_pathkeys(Plan *lefttree, List *pathkeys,
 static EquivalenceMember *find_ec_member_for_tle(EquivalenceClass *ec,
 					   TargetEntry *tle,
 					   Relids relids);
-static Sort *make_sort_from_pathkeys(Plan *lefttree, List *pathkeys);
+static Sort *make_sort_from_pathkeys(Plan *lefttree, List *pathkeys,
+									 Relids relids);
 static Sort *make_sort_from_groupcols(List *groupcls,
 						 AttrNumber *grpColIdx,
 						 Plan *lefttree);
@@ -367,12 +373,8 @@ create_plan_recurse(PlannerInfo *root, Path *best_path, int flags)
 									(JoinPath *) best_path);
 			break;
 		case T_Append:
-			plan = create_append_plan(root,
-									  (AppendPath *) best_path);
-			break;
 		case T_MergeAppend:
-			plan = create_merge_append_plan(root,
-											(MergeAppendPath *) best_path);
+			plan = create_partition_plan(root, best_path);
 			break;
 		case T_Result:
 			if (IsA(best_path, ProjectionPath))
@@ -1115,6 +1117,30 @@ create_merge_append_plan(PlannerInfo *root, MergeAppendPath *best_path)
 }
 
 /*
+ * create_partition_plan
+ *		Creates an Merge/Append plan as specified by the "best path".
+ *
+ *		Returns a Plan node.
+ */
+static Plan *
+create_partition_plan(PlannerInfo *root, Path *best_path)
+{
+	Plan   *plan;
+
+	if (IsA(best_path, PartitionJoinPath))
+		plan = create_partition_join_plan(root, (PartitionJoinPath *)best_path);
+	else if (best_path->pathtype == T_Append)
+		plan = create_append_plan(root, (AppendPath *) best_path);
+	else
+	{
+		Assert(best_path->pathtype == T_MergeAppend);
+		plan = create_merge_append_plan(root, (MergeAppendPath *) best_path);
+	}
+
+	return plan;
+}
+
+/*
  * create_result_plan
  *	  Create a Result plan for 'best_path'.
  *	  This is only used for degenerate cases, such as a query with an empty
@@ -1513,7 +1539,7 @@ create_sort_plan(PlannerInfo *root, SortPath *best_path, int flags)
 	subplan = create_plan_recurse(root, best_path->subpath,
 								  flags | CP_SMALL_TLIST);
 
-	plan = make_sort_from_pathkeys(subplan, best_path->path.pathkeys);
+	plan = make_sort_from_pathkeys(subplan, best_path->path.pathkeys, NULL);
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
@@ -3530,6 +3556,8 @@ create_mergejoin_plan(PlannerInfo *root,
 	ListCell   *lc;
 	ListCell   *lop;
 	ListCell   *lip;
+	Path	   *outer_path = best_path->jpath.outerjoinpath;
+	Path	   *inner_path = best_path->jpath.innerjoinpath;
 
 	/*
 	 * MergeJoin can project, so we don't have to demand exact tlists from the
@@ -3537,10 +3565,10 @@ create_mergejoin_plan(PlannerInfo *root,
 	 * best to request a small tlist so we aren't sorting more data than
 	 * necessary.
 	 */
-	outer_plan = create_plan_recurse(root, best_path->jpath.outerjoinpath,
+	outer_plan = create_plan_recurse(root, outer_path,
 					 (best_path->outersortkeys != NIL) ? CP_SMALL_TLIST : 0);
 
-	inner_plan = create_plan_recurse(root, best_path->jpath.innerjoinpath,
+	inner_plan = create_plan_recurse(root, inner_path,
 					 (best_path->innersortkeys != NIL) ? CP_SMALL_TLIST : 0);
 
 	/* Sort join qual clauses into best execution order */
@@ -3586,34 +3614,38 @@ create_mergejoin_plan(PlannerInfo *root,
 	 * outer_is_left status.
 	 */
 	mergeclauses = get_switched_clauses(best_path->path_mergeclauses,
-							 best_path->jpath.outerjoinpath->parent->relids);
+							 outer_path->parent->relids);
 
 	/*
 	 * Create explicit sort nodes for the outer and inner paths if necessary.
 	 */
 	if (best_path->outersortkeys)
 	{
+		Relids		outer_relids = outer_path->parent->relids;
 		Sort	   *sort = make_sort_from_pathkeys(outer_plan,
-												   best_path->outersortkeys);
+												   best_path->outersortkeys,
+												   outer_relids);
 
 		label_sort_with_costsize(root, sort, -1.0);
 		outer_plan = (Plan *) sort;
 		outerpathkeys = best_path->outersortkeys;
 	}
 	else
-		outerpathkeys = best_path->jpath.outerjoinpath->pathkeys;
+		outerpathkeys = outer_path->pathkeys;
 
 	if (best_path->innersortkeys)
 	{
+		Relids		inner_relids = inner_path->parent->relids;
 		Sort	   *sort = make_sort_from_pathkeys(inner_plan,
-												   best_path->innersortkeys);
+												   best_path->innersortkeys,
+												   inner_relids);
 
 		label_sort_with_costsize(root, sort, -1.0);
 		inner_plan = (Plan *) sort;
 		innerpathkeys = best_path->innersortkeys;
 	}
 	else
-		innerpathkeys = best_path->jpath.innerjoinpath->pathkeys;
+		innerpathkeys = inner_path->pathkeys;
 
 	/*
 	 * If specified, add a materialize node to shield the inner plan from the
@@ -3951,6 +3983,215 @@ create_hashjoin_plan(PlannerInfo *root,
 	return join_plan;
 }
 
+/*
+ * create_partition_join_plan
+ *		Creates Merge/Append plan consisting of join plans for child-join.
+ *
+ *		Returns a Plan node.
+ */
+static Plan *
+create_partition_join_plan(PlannerInfo *root, PartitionJoinPath *best_path)
+{
+	RelOptInfo *joinrel = best_path->path.parent;
+	int		nparts;
+	int		cnt_parts;
+	List	   *child_plans = NIL;
+	List	   *tlist = build_path_tlist(root, &best_path->path);
+	Plan	   *plan;
+	MemoryContext	child_context;
+	MemoryContext	old_context;
+	List	   *pathkeys = best_path->path.pathkeys;
+	StringInfoData	mem_context_name;
+
+	/* The relation should be a partitioned join relation. */
+	Assert(IS_JOIN_REL(joinrel) && joinrel->part_scheme &&
+		   joinrel->partitioned_joins);
+
+	nparts = joinrel->part_scheme->nparts;
+
+	/* Create MergeAppend plan when result is expected to be ordered. */
+	if (pathkeys)
+	{
+		MergeAppend *node = makeNode(MergeAppend);
+		plan = &node->plan;
+
+		plan->targetlist = tlist;
+
+		/* Compute sorting info, and adjust MergeAppend's tlist as needed. */
+		(void) prepare_sort_from_pathkeys(plan, pathkeys,
+										  best_path->path.parent->relids,
+										  NULL,
+										  true,
+										  &node->numCols,
+										  &node->sortColIdx,
+										  &node->sortOperators,
+										  &node->collations,
+										  &node->nullsFirst);
+	}
+	else
+	{
+		Append  *node = makeNode(Append);
+		plan = &node->plan;
+		plan->targetlist = tlist;
+	}
+
+	/* Fill costs, so that we can cost Sort node, if required. */
+	copy_generic_path_info(plan, (Path *) best_path);
+
+	/*
+	 * Create a new memory context for planning child joins. Since this routine
+	 * may be called recursively for tables with subpartitions, we use
+	 * a unique context name for every level of partition by using the lowest
+	 * relid amongst the base relations being joined.
+	 */
+	initStringInfo(&mem_context_name);
+	appendStringInfo(&mem_context_name, "%s_%d", "ChildJoinContext",
+					 bms_next_member(joinrel->relids, -1));
+	child_context = AllocSetContextCreate(CurrentMemoryContext,
+										  pstrdup(mem_context_name.data),
+										  ALLOCSET_DEFAULT_SIZES);
+	pfree(mem_context_name.data);
+	resetStringInfo(&mem_context_name);
+
+	/*
+	 * Create a paths for all child joins, one child join at a time. The paths
+	 * for every child join are independent i.e. one child does not require
+	 * paths created for the other. In order to avoid accumulating memory
+	 * consumed while creating paths for every child join, we use a fresh
+	 * memory context for every child join.
+	 */
+	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	{
+		RelOptInfo *child_join;
+		Path	   *child_path = NULL;
+		Plan	   *child_plan;
+		int			numsortkeys;
+		AttrNumber *sortColIdx;
+		Oid		   *sortOperators;
+		Oid		   *collations;
+		bool	   *nullsFirst;
+
+		/*
+		 * Create paths for the child join in a separate context, so that we
+		 * can reuse the memory used by those paths.
+		 */
+		old_context = MemoryContextSwitchTo(child_context);
+
+		add_paths_to_child_joinrel(root, joinrel, cnt_parts);
+
+		child_join = joinrel->part_rels[cnt_parts];
+
+
+		/* Skip empty child. */
+		if (IS_DUMMY_REL(child_join))
+		{
+			MemoryContextSwitchTo(old_context);
+			continue;
+		}
+
+#ifdef OPTIMIZER_DEBUG
+		debug_print_rel(root, rel);
+#endif
+
+		/*
+		 * Search for a child path with pathkeys or parameterization
+		 * matching that of the given path.
+		 */
+		child_path = get_cheapest_path_for_pathkeys(child_join->pathlist,
+													best_path->path.pathkeys,
+											 PATH_REQ_OUTER(&best_path->path),
+													TOTAL_COST);
+
+		if (!child_path)
+			elog(ERROR, "Could not find a path with required pathkeys.");
+
+		MemoryContextSwitchTo(old_context);
+
+		/* Create plan for the current child. */
+		child_plan = create_plan_recurse(root, child_path, CP_EXACT_TLIST);
+
+		if (pathkeys)
+		{
+			MergeAppend *node = (MergeAppend *) plan;
+
+			Assert(IsA(node, MergeAppend));
+
+			/* Compute sorting info, and adjust subplan's tlist as needed */
+			child_plan = prepare_sort_from_pathkeys(child_plan, pathkeys,
+												 child_path->parent->relids,
+												 node->sortColIdx,
+												 false,
+												 &numsortkeys,
+												 &sortColIdx,
+												 &sortOperators,
+												 &collations,
+												 &nullsFirst);
+
+			/*
+			 * Check that we got the same sort key information.  We just Assert
+			 * that the sortops match, since those depend only on the pathkeys;
+			 * but it seems like a good idea to check the sort column numbers
+			 * explicitly, to ensure the tlists really do match up.
+			 */
+			Assert(numsortkeys == node->numCols);
+			if (memcmp(sortColIdx, node->sortColIdx,
+					   numsortkeys * sizeof(AttrNumber)) != 0)
+				elog(ERROR, "MergeAppend child's targetlist doesn't match MergeAppend");
+			Assert(memcmp(sortOperators, node->sortOperators,
+						  numsortkeys * sizeof(Oid)) == 0);
+			Assert(memcmp(collations, node->collations,
+						  numsortkeys * sizeof(Oid)) == 0);
+			Assert(memcmp(nullsFirst, node->nullsFirst,
+						  numsortkeys * sizeof(bool)) == 0);
+
+			/* Now, insert a Sort node if subplan isn't sufficiently ordered */
+			if (!pathkeys_contained_in(pathkeys, child_path->pathkeys))
+			{
+				Sort	   *sort = make_sort(child_plan, numsortkeys,
+											 sortColIdx, sortOperators,
+											 collations, nullsFirst);
+				label_sort_with_costsize(root, sort, -1.0);
+				child_plan = (Plan *) sort;
+			}
+		}
+
+		child_plans = lappend(child_plans, child_plan);
+
+		/*
+		 * Reset the child_join memory context to reclaim the memory consumed
+		 * while creating paths.
+		 */
+		MemoryContextResetAndDeleteChildren(child_context);
+	}
+
+	/* Destroy the child context as we do not need it anymore. */
+	Assert(CurrentMemoryContext == old_context);
+	MemoryContextDelete(child_context);
+
+	/* Partitioned relation with all empty children gets a dummy path. */
+	Assert(child_plans != NIL);
+
+	if (IsA(plan, MergeAppend))
+	{
+		MergeAppend *node = (MergeAppend *)plan;
+
+		node->mergeplans = child_plans;
+	}
+	else
+	{
+		Append  *node = (Append *)plan;
+
+		Assert(IsA(plan, Append));
+		node->appendplans = child_plans;
+	}
+
+	/* Complete rest of the plan. */
+	plan->qual = NIL;
+	plan->lefttree = NULL;
+	plan->righttree = NULL;
+	return plan;
+}
+
 
 /*****************************************************************************
  *
@@ -4009,6 +4250,7 @@ replace_nestloop_params_mutator(Node *node, PlannerInfo *root)
 		nlp->paramno = param->paramid;
 		nlp->paramval = var;
 		root->curOuterParams = lappend(root->curOuterParams, nlp);
+
 		/* And return the replacement Param */
 		return (Node *) param;
 	}
@@ -4072,6 +4314,7 @@ replace_nestloop_params_mutator(Node *node, PlannerInfo *root)
 		nlp->paramno = param->paramid;
 		nlp->paramval = (Var *) phv;
 		root->curOuterParams = lappend(root->curOuterParams, nlp);
+
 		/* And return the replacement Param */
 		return (Node *) param;
 	}
@@ -5343,11 +5586,11 @@ prepare_sort_from_pathkeys(Plan *lefttree, List *pathkeys,
 					continue;
 
 				/*
-				 * Ignore child members unless they match the rel being
+				 * Ignore child members unless they belong to the rel being
 				 * sorted.
 				 */
 				if (em->em_is_child &&
-					!bms_equal(em->em_relids, relids))
+					!bms_is_subset(em->em_relids, relids))
 					continue;
 
 				sortexpr = em->em_expr;
@@ -5458,10 +5701,10 @@ find_ec_member_for_tle(EquivalenceClass *ec,
 			continue;
 
 		/*
-		 * Ignore child members unless they match the rel being sorted.
+		 * Ignore child members unless they belong to the rel being sorted.
 		 */
 		if (em->em_is_child &&
-			!bms_equal(em->em_relids, relids))
+			!bms_is_subset(em->em_relids, relids))
 			continue;
 
 		/* Match if same expression (after stripping relabel) */
@@ -5482,9 +5725,10 @@ find_ec_member_for_tle(EquivalenceClass *ec,
  *
  *	  'lefttree' is the node which yields input tuples
  *	  'pathkeys' is the list of pathkeys by which the result is to be sorted
+ *	  'relids' is the set of relations required by prepare_sort_from_pathkeys()
  */
 static Sort *
-make_sort_from_pathkeys(Plan *lefttree, List *pathkeys)
+make_sort_from_pathkeys(Plan *lefttree, List *pathkeys, Relids relids)
 {
 	int			numsortkeys;
 	AttrNumber *sortColIdx;
@@ -5494,7 +5738,7 @@ make_sort_from_pathkeys(Plan *lefttree, List *pathkeys)
 
 	/* Compute sort column info, and adjust lefttree as needed */
 	lefttree = prepare_sort_from_pathkeys(lefttree, pathkeys,
-										  NULL,
+										  relids,
 										  NULL,
 										  false,
 										  &numsortkeys,
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 41dde50..fedbb43 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -1105,7 +1105,7 @@ inheritance_planner(PlannerInfo *root)
 		subroot->parse = (Query *)
 			adjust_appendrel_attrs(root,
 								   (Node *) parse,
-								   appinfo);
+								   list_make1(appinfo));
 
 		/*
 		 * The rowMarks list might contain references to subquery RTEs, so
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index b714783..8b6a183 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -55,7 +55,7 @@
 typedef struct
 {
 	PlannerInfo *root;
-	AppendRelInfo *appinfo;
+	List	   *appinfos;
 	int			sublevels_up;
 } adjust_appendrel_attrs_context;
 
@@ -108,7 +108,6 @@ static Bitmapset *translate_col_privs(const Bitmapset *parent_privs,
 					List *translated_vars);
 static Node *adjust_appendrel_attrs_mutator(Node *node,
 							   adjust_appendrel_attrs_context *context);
-static Relids adjust_relid_set(Relids relids, Index oldrelid, Index newrelid);
 static List *adjust_inherited_tlist(List *tlist,
 					   AppendRelInfo *context);
 
@@ -1712,10 +1711,10 @@ translate_col_privs(const Bitmapset *parent_privs,
 
 /*
  * adjust_appendrel_attrs
- *	  Copy the specified query or expression and translate Vars referring
- *	  to the parent rel of the specified AppendRelInfo to refer to the
- *	  child rel instead.  We also update rtindexes appearing outside Vars,
- *	  such as resultRelation and jointree relids.
+ *	  Copy the specified query or expression and translate Vars referring to
+ *	  the parent rels specified in the given list of AppendRelInfos to refer to
+ *	  the corresponding child rels instead.  We also update rtindexes appearing
+ *	  outside Vars, such as resultRelation and jointree relids.
  *
  * Note: this is applied after conversion of sublinks to subplans in the
  * query jointree, but there may still be sublinks in the security barrier
@@ -1725,15 +1724,18 @@ translate_col_privs(const Bitmapset *parent_privs,
  * maybe we should try to fold the two routines together.
  */
 Node *
-adjust_appendrel_attrs(PlannerInfo *root, Node *node, AppendRelInfo *appinfo)
+adjust_appendrel_attrs(PlannerInfo *root, Node *node, List *appinfos)
 {
 	Node	   *result;
 	adjust_appendrel_attrs_context context;
+	ListCell   *lc;
 
 	context.root = root;
-	context.appinfo = appinfo;
+	context.appinfos = appinfos;
 	context.sublevels_up = 0;
 
+	Assert(appinfos && list_length(appinfos) >= 1);
+
 	/*
 	 * Must be prepared to start with a Query or a bare expression tree; if
 	 * it's a Query, go straight to query_tree_walker to make sure that
@@ -1742,11 +1744,20 @@ adjust_appendrel_attrs(PlannerInfo *root, Node *node, AppendRelInfo *appinfo)
 	if (node && IsA(node, Query))
 	{
 		Query	   *newnode;
+		AppendRelInfo *appinfo;
 
 		newnode = query_tree_mutator((Query *) node,
 									 adjust_appendrel_attrs_mutator,
 									 (void *) &context,
 									 QTW_IGNORE_RC_SUBQUERIES);
+		foreach (lc, appinfos)
+		{
+			appinfo = lfirst(lc);
+
+			if (newnode->resultRelation == appinfo->parent_relid)
+				break;
+		}
+
 		if (newnode->resultRelation == appinfo->parent_relid)
 		{
 			newnode->resultRelation = appinfo->child_relid;
@@ -1764,17 +1775,51 @@ adjust_appendrel_attrs(PlannerInfo *root, Node *node, AppendRelInfo *appinfo)
 	return result;
 }
 
+/*
+ * find_appinfos_by_relids
+ * 		Find AppendRelInfo structures for all relations specified by relids.
+ */
+List *
+find_appinfos_by_relids(PlannerInfo *root, Relids relids)
+{
+	ListCell	*lc;
+	List		*appinfo_list = NIL;
+
+	foreach (lc, root->append_rel_list)
+	{
+		AppendRelInfo *appinfo = lfirst(lc);
+
+		if (bms_is_member(appinfo->child_relid, relids))
+			appinfo_list = lappend(appinfo_list, appinfo);
+	}
+
+	Assert(list_length(appinfo_list) == bms_num_members(relids));
+	return appinfo_list;
+}
+
 static Node *
 adjust_appendrel_attrs_mutator(Node *node,
 							   adjust_appendrel_attrs_context *context)
 {
-	AppendRelInfo *appinfo = context->appinfo;
+	List   *appinfos = context->appinfos;
+	ListCell   *lc;
+
+	Assert(appinfos && list_length(appinfos) >= 1);
 
 	if (node == NULL)
 		return NULL;
 	if (IsA(node, Var))
 	{
 		Var		   *var = (Var *) copyObject(node);
+		AppendRelInfo *appinfo;
+
+		foreach (lc, appinfos)
+		{
+			appinfo = lfirst(lc);
+
+			if (var->varno == appinfo->parent_relid)
+				break;
+		}
 
 		if (var->varlevelsup == context->sublevels_up &&
 			var->varno == appinfo->parent_relid)
@@ -1865,32 +1910,58 @@ adjust_appendrel_attrs_mutator(Node *node,
 	{
 		CurrentOfExpr *cexpr = (CurrentOfExpr *) copyObject(node);
 
-		if (context->sublevels_up == 0 &&
-			cexpr->cvarno == appinfo->parent_relid)
-			cexpr->cvarno = appinfo->child_relid;
+		foreach (lc, appinfos)
+		{
+			AppendRelInfo *appinfo = lfirst(lc);
+
+			if (context->sublevels_up == 0 &&
+				cexpr->cvarno == appinfo->parent_relid)
+			{
+				cexpr->cvarno = appinfo->child_relid;
+				break;
+			}
+		}
 		return (Node *) cexpr;
 	}
 	if (IsA(node, RangeTblRef))
 	{
 		RangeTblRef *rtr = (RangeTblRef *) copyObject(node);
 
-		if (context->sublevels_up == 0 &&
-			rtr->rtindex == appinfo->parent_relid)
-			rtr->rtindex = appinfo->child_relid;
+		foreach (lc, appinfos)
+		{
+			AppendRelInfo *appinfo = lfirst(lc);
+
+			if (context->sublevels_up == 0 &&
+				rtr->rtindex == appinfo->parent_relid)
+			{
+				rtr->rtindex = appinfo->child_relid;
+				break;
+			}
+		}
 		return (Node *) rtr;
 	}
 	if (IsA(node, JoinExpr))
 	{
 		/* Copy the JoinExpr node with correct mutation of subnodes */
 		JoinExpr   *j;
+		AppendRelInfo *appinfo;
 
 		j = (JoinExpr *) expression_tree_mutator(node,
 											  adjust_appendrel_attrs_mutator,
 												 (void *) context);
+
 		/* now fix JoinExpr's rtindex (probably never happens) */
-		if (context->sublevels_up == 0 &&
-			j->rtindex == appinfo->parent_relid)
-			j->rtindex = appinfo->child_relid;
+		foreach (lc, appinfos)
+		{
+			appinfo = lfirst(lc);
+
+			if (context->sublevels_up == 0 &&
+				j->rtindex == appinfo->parent_relid)
+			{
+				j->rtindex = appinfo->child_relid;
+				break;
+			}
+		}
 		return (Node *) j;
 	}
 	if (IsA(node, PlaceHolderVar))
@@ -1903,9 +1974,8 @@ adjust_appendrel_attrs_mutator(Node *node,
 														 (void *) context);
 		/* now fix PlaceHolderVar's relid sets */
 		if (phv->phlevelsup == context->sublevels_up)
-			phv->phrels = adjust_relid_set(phv->phrels,
-										   appinfo->parent_relid,
-										   appinfo->child_relid);
+			phv->phrels = adjust_child_relids(phv->phrels, context->appinfos);
+
 		return (Node *) phv;
 	}
 	/* Shouldn't need to handle planner auxiliary nodes here */
@@ -1936,24 +2006,18 @@ adjust_appendrel_attrs_mutator(Node *node,
 			adjust_appendrel_attrs_mutator((Node *) oldinfo->orclause, context);
 
 		/* adjust relid sets too */
-		newinfo->clause_relids = adjust_relid_set(oldinfo->clause_relids,
-												  appinfo->parent_relid,
-												  appinfo->child_relid);
-		newinfo->required_relids = adjust_relid_set(oldinfo->required_relids,
-													appinfo->parent_relid,
-													appinfo->child_relid);
-		newinfo->outer_relids = adjust_relid_set(oldinfo->outer_relids,
-												 appinfo->parent_relid,
-												 appinfo->child_relid);
-		newinfo->nullable_relids = adjust_relid_set(oldinfo->nullable_relids,
-													appinfo->parent_relid,
-													appinfo->child_relid);
-		newinfo->left_relids = adjust_relid_set(oldinfo->left_relids,
-												appinfo->parent_relid,
-												appinfo->child_relid);
-		newinfo->right_relids = adjust_relid_set(oldinfo->right_relids,
-												 appinfo->parent_relid,
-												 appinfo->child_relid);
+		newinfo->clause_relids = adjust_child_relids(oldinfo->clause_relids,
+													 context->appinfos);
+		newinfo->required_relids = adjust_child_relids(oldinfo->required_relids,
+													 context->appinfos);
+		newinfo->outer_relids = adjust_child_relids(oldinfo->outer_relids,
+													 context->appinfos);
+		newinfo->nullable_relids = adjust_child_relids(oldinfo->nullable_relids,
+													   context->appinfos);
+		newinfo->left_relids = adjust_child_relids(oldinfo->left_relids,
+												   context->appinfos);
+		newinfo->right_relids = adjust_child_relids(oldinfo->right_relids,
+													context->appinfos);
 
 		/*
 		 * Reset cached derivative fields, since these might need to have
@@ -2002,23 +2066,6 @@ adjust_appendrel_attrs_mutator(Node *node,
 }
 
 /*
- * Substitute newrelid for oldrelid in a Relid set
- */
-static Relids
-adjust_relid_set(Relids relids, Index oldrelid, Index newrelid)
-{
-	if (bms_is_member(oldrelid, relids))
-	{
-		/* Ensure we have a modifiable copy */
-		relids = bms_copy(relids);
-		/* Remove old, add new */
-		relids = bms_del_member(relids, oldrelid);
-		relids = bms_add_member(relids, newrelid);
-	}
-	return relids;
-}
-
-/*
  * Adjust the targetlist entries of an inherited UPDATE operation
  *
  * The expressions have already been fixed, but we have to make sure that
@@ -2135,5 +2182,100 @@ adjust_appendrel_attrs_multilevel(PlannerInfo *root, Node *node,
 	else
 		Assert(parent_rel->reloptkind == RELOPT_BASEREL);
 	/* Now translate for this child */
-	return adjust_appendrel_attrs(root, node, appinfo);
+	return adjust_appendrel_attrs(root, node, list_make1(appinfo));
+}
+
+/*
+ * build_child_restrictinfo
+ *		Returns a RestrictInfo which is derived from the given RestrictInfo by
+ *		applying the parent-child translation specified by the list of
+ *		AppendRelInfos.
+ *
+ * The topmost parent's RestrictInfo maintains a list of child RestrictInfos
+ * derived from it. If a suitable RestrictInfo is found in that list, it is
+ * returned as is. If there is no such child RestrictInfo, we translate the given
+ * RestrictInfo using the given list of AppendRelInfos and stick it in the
+ * topmost parent's list before returning it to the caller.
+ */
+RestrictInfo *
+build_child_restrictinfo(PlannerInfo *root, RestrictInfo *rinfo,
+						 List *append_rel_infos)
+{
+	Relids child_required_relids;
+	ListCell   *lc;
+	RestrictInfo   *parent_rinfo;
+	RestrictInfo   *child_rinfo;
+	MemoryContext	old_context;
+
+	child_required_relids = adjust_child_relids(rinfo->required_relids,
+												append_rel_infos);
+
+
+	/* Nothing to do, if the clause does not need any translation. */
+	if (bms_equal(child_required_relids, rinfo->required_relids))
+	{
+		bms_free(child_required_relids);
+		return rinfo;
+	}
+
+	/*
+	 * Check if we already have the RestrictInfo for the given child in the
+	 * topmost parent's RestrictInfo.
+	 */
+	parent_rinfo = rinfo->parent_rinfo ? rinfo->parent_rinfo : rinfo;
+	foreach (lc, parent_rinfo->child_rinfos)
+	{
+		child_rinfo = lfirst(lc);
+
+		if (bms_equal(child_rinfo->required_relids, child_required_relids))
+		{
+			bms_free(child_required_relids);
+			return child_rinfo;
+		}
+	}
+
+	/*
+	 * We didn't find any child restrictinfo for the given child, translate the
+	 * given RestrictInfo and stick it into the parent's list. The clause
+	 * expression may get used in plan, so create the child RestrictInfo in the
+	 * planner's context.
+	 */
+	old_context = MemoryContextSwitchTo(root->planner_cxt);
+	child_rinfo = (RestrictInfo *) adjust_appendrel_attrs(root, (Node *) rinfo,
+														  append_rel_infos);
+	bms_free(child_required_relids);
+	parent_rinfo->child_rinfos = lappend(parent_rinfo->child_rinfos,
+										 child_rinfo);
+	child_rinfo->parent_rinfo = parent_rinfo;
+
+	MemoryContextSwitchTo(old_context);
+
+	return child_rinfo;
+}
+
+/*
+ * build_child_clauses
+ *		Convenience routine to call build_child_restrictinfo on a list of
+ *		clauses.
+ */
+List *
+build_child_clauses(PlannerInfo *root, List *clauses, List *append_rel_infos)
+{
+	List   *child_clauses = NIL;
+	ListCell   *lc;
+
+	foreach (lc, clauses)
+	{
+		RestrictInfo   *parent_rinfo = lfirst(lc);
+		RestrictInfo   *child_rinfo;
+
+		Assert(IsA(parent_rinfo, RestrictInfo));
+
+		child_rinfo = build_child_restrictinfo(root, parent_rinfo,
+											   append_rel_infos);
+
+		child_clauses = lappend(child_clauses, child_rinfo);
+	}
+
+	return child_clauses;
 }
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 6d3ccfd..f5010e4 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -23,7 +23,9 @@
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
 #include "optimizer/planmain.h"
+#include "optimizer/prep.h"
 #include "optimizer/restrictinfo.h"
+#include "optimizer/tlist.h"
 #include "optimizer/var.h"
 #include "parser/parsetree.h"
 #include "utils/lsyscache.h"
@@ -2154,6 +2156,176 @@ create_hashjoin_path(PlannerInfo *root,
 }
 
 /*
+ * create_partition_join_path
+ * 		Creates a pathnode that represents partition-wise join for given
+ * 		partitioned join relation.
+ *
+ * This function is called when we haven't created paths for all the child
+ * joins. It estimates the number of rows and cost of the PartitionJoinPath
+ * based upon the number of rows and the cost of representative child-joins
+ * paths.
+ */
+PartitionJoinPath *
+create_partition_join_path(RelOptInfo *rel, List *subpaths,
+						   Bitmapset *required_outer)
+{
+	PartitionJoinPath *pathnode = makeNode(PartitionJoinPath);
+	double	subpath_rows = 0;
+	double	subpath_startup_cost = 0;
+	double	subpath_total_cost = 0;
+	double	child_rel_rows = 0;
+	ListCell   *lc;
+
+	Assert(rel->part_scheme);
+
+	pathnode->path.pathtype = T_Append;
+	pathnode->path.parent = rel;
+	pathnode->path.pathtarget = rel->reltarget;
+	pathnode->path.param_info = get_appendrel_parampathinfo(rel,
+															required_outer);
+	pathnode->path.pathkeys = NULL;
+
+	/* No parallel paths here. See more details in add_paths_to_append_rel() */
+	pathnode->path.parallel_aware = false;
+	pathnode->path.parallel_safe = false;
+	pathnode->path.parallel_workers = 0;
+
+	/* Accumulate the number of rows and costs from the given subpaths. */
+	foreach (lc, subpaths)
+	{
+		Path   *subpath = lfirst(lc);
+
+		subpath_rows += subpath->rows;
+		child_rel_rows += subpath->parent->rows;
+		subpath_total_cost += subpath->total_cost;
+
+		/*
+		 * Startup cost of an append relation is the startup cost of the first
+		 * subpath. Assume that the given first child will be the first child
+		 * in the final plan as well.
+		 */
+		if (lc == list_head(subpaths))
+			subpath_startup_cost = subpath->startup_cost;
+	}
+
+	/*
+	 * For a parameterized path, extrapolate the number of rows for the append
+	 * relation by considering the average selectivity of the parameterization
+	 * across the given children.
+	 */
+	if (bms_is_empty(required_outer))
+		pathnode->path.rows = rel->rows;
+	else
+		pathnode->path.rows = rel->rows * (subpath_rows / child_rel_rows);
+
+	pathnode->path.startup_cost = subpath_startup_cost;
+
+	/* Extrapolate the total cost to account for yet-to-be planned children. */
+	pathnode->path.total_cost = (subpath_total_cost * pathnode->path.rows) / subpath_rows;
+
+	/*
+	 * Multiply the costs with scaling factor as specified. Used to encourage
+	 * or discourage use of partition-wise join plans.
+	 */
+	pathnode->path.startup_cost *= partition_wise_plan_weight;
+	pathnode->path.total_cost *= partition_wise_plan_weight;
+
+	return pathnode;
+}
+
+/*
+ * create_partition_join_path_with_pathkeys
+ * 		Creates a pathnode that represents an ordered partition-wise join for
+ * 		given partitioned join relation.
+ *
+ * This function is called when we haven't created paths for all the child
+ * joins. It estimates the number of rows and cost of the PartitionJoinPath
+ * based upon the number of rows and the cost of representative child-joins
+ * paths.
+ */
+PartitionJoinPath *
+create_partition_join_path_with_pathkeys(PlannerInfo *root, RelOptInfo *rel,
+						   List *subpaths, List *pathkeys,
+						   Bitmapset *required_outer)
+{
+	PartitionJoinPath *pathnode = makeNode(PartitionJoinPath);
+	double	subpath_rows = 0;
+	double	subpath_startup_cost = 0;
+	double	subpath_total_cost = 0;
+	double	child_rel_rows = 0;
+	ListCell   *lc;
+
+	Assert(rel->part_scheme);
+
+	pathnode->path.pathtype = T_MergeAppend;
+	pathnode->path.parent = rel;
+	pathnode->path.pathtarget = rel->reltarget;
+	pathnode->path.param_info = get_appendrel_parampathinfo(rel,
+															required_outer);
+	pathnode->path.pathkeys = pathkeys;
+
+	/* No parallel paths here. See more details in add_paths_to_append_rel() */
+	pathnode->path.parallel_aware = false;
+	pathnode->path.parallel_safe = false;
+	pathnode->path.parallel_workers = 0;
+
+	/* Accumulate the number of rows and costs from the given subpaths. */
+	foreach (lc, subpaths)
+	{
+		Path   *subpath = lfirst(lc);
+
+		if (pathkeys_contained_in(pathkeys, subpath->pathkeys))
+		{
+			/* Subpath is adequately ordered, we won't need to sort it */
+			subpath_startup_cost += subpath->startup_cost;
+			subpath_total_cost += subpath->total_cost;
+		}
+		else
+		{
+			/* We'll need to insert a Sort node, so include cost for that */
+			Path		sort_path;		/* dummy for result of cost_sort */
+
+			cost_sort(&sort_path,
+					  root,
+					  pathkeys,
+					  subpath->total_cost,
+					  subpath->parent->tuples,
+					  subpath->pathtarget->width,
+					  0.0,
+					  work_mem,
+					  -1);
+			subpath_startup_cost += sort_path.startup_cost;
+			subpath_total_cost += sort_path.total_cost;
+		}
+
+		subpath_rows += subpath->rows;
+		child_rel_rows += subpath->parent->rows;
+	}
+
+	/*
+	 * For a parameterized path, extrapolate the number of rows for the append
+	 * relation by considering the average selectivity of the parameterization
+	 * across the given children.
+	 */
+	if (bms_is_empty(required_outer))
+		pathnode->path.rows = rel->rows;
+	else
+		pathnode->path.rows = rel->rows * (subpath_rows / child_rel_rows);
+
+	/* Extrapolate the total cost to account for yet-to-be planned children. */
+	pathnode->path.startup_cost = (subpath_startup_cost * pathnode->path.rows) / subpath_rows;
+	pathnode->path.total_cost = (subpath_total_cost * pathnode->path.rows) / subpath_rows;
+
+	/*
+	 * Multiply the costs with scaling factor as specified. Used to encourage
+	 * or discourage use of partition-wise join plans.
+	 */
+	pathnode->path.startup_cost *= partition_wise_plan_weight;
+	pathnode->path.total_cost *= partition_wise_plan_weight;
+
+	return pathnode;
+}
+/*
  * create_projection_path
  *	  Creates a pathnode that represents performing a projection.
  *
@@ -3209,3 +3381,183 @@ reparameterize_path(PlannerInfo *root, Path *path,
 	}
 	return NULL;
 }
+
+/*
+ * reparameterize_path_by_child
+ * 		Given a path parameterized by the parent of the given relation,
+ * 		translate the path to be parameterized by the given child relation.
+ *
+ * The function creates a new path of the same type as the given path, but
+ * parameterized by the given child relation. If it can not reparameterize the
+ * path as required, it returns NULL.
+ *
+ * The cost, number of rows, width and parallel path properties depend upon
+ * path->parent, which does not change during the translation. Hence those
+ * members are copied as they are.
+ */
+
+Path *
+reparameterize_path_by_child(PlannerInfo *root, Path *path,
+							  RelOptInfo *child_rel)
+{
+	Path	   *new_path;
+	ParamPathInfo   *new_ppi;
+	ParamPathInfo   *old_ppi;
+	List	   *child_aris;
+	Relids		required_outer;
+
+	/*
+	 * If the path is not parameterized by parent of the given relation or it it
+	 * doesn't need reparameterization.
+	 */
+	if (!path->param_info ||
+		!bms_overlap(PATH_REQ_OUTER(path), child_rel->top_parent_relids))
+	return path;
+
+	switch (nodeTag(path))
+	{
+		case T_Path:
+			new_path = makeNode(Path);
+			memcpy(new_path, path, sizeof(Path));
+			break;
+
+		case T_HashPath:
+			new_path = (Path *) makeNode(HashPath);
+			memcpy(new_path, path, sizeof(HashPath));
+			break;
+
+		case T_MergePath:
+			new_path = (Path *) makeNode(MergePath);
+			memcpy(new_path, path, sizeof(MergePath));
+			break;
+
+		case T_NestPath:
+			new_path = (Path *) makeNode(NestPath);
+			memcpy(new_path, path, sizeof(NestPath));
+			break;
+
+		case T_IndexPath:
+			new_path = (Path *) makeNode(IndexPath);
+			memcpy(new_path, path, sizeof(IndexPath));
+			break;
+
+		case T_AppendPath:
+			new_path = (Path *) makeNode(AppendPath);
+			memcpy(new_path, path, sizeof(AppendPath));
+			break;
+
+		/*
+		 * TODO:
+		 * If this method of translation is fine add more path types here.
+		 */
+
+		default:
+			/* Path type unsupported by this function. */
+			return NULL;
+	}
+
+	/*
+	 * Gather AppendRelInfos of the base partition relations in the outer child
+	 * relation. We need those for translating parent path to that of child by
+	 * substituting parent Var nodes and relids with those of children.
+	 */
+	child_aris = find_appinfos_by_relids(root, child_rel->relids);
+
+	/* Adjust the parameterization information. */
+	old_ppi = new_path->param_info;
+	required_outer = adjust_child_relids(old_ppi->ppi_req_outer, child_aris);
+
+	/* If we already have a PPI for this parameterization, just return it */
+	new_ppi = find_param_path_info(new_path->parent, required_outer);
+
+	/* If not build a new one and link it to the list of PPIs. */
+	if (!new_ppi)
+	{
+		new_ppi = makeNode(ParamPathInfo);
+		new_ppi->ppi_req_outer = required_outer;
+		new_ppi->ppi_rows = old_ppi->ppi_rows;
+		new_ppi->ppi_clauses = build_child_clauses(root, old_ppi->ppi_clauses,
+												   child_aris);
+		new_path->parent->ppilist = lappend(new_path->parent->ppilist, new_ppi);
+	}
+	else
+		bms_free(required_outer);
+
+	new_path->param_info = new_ppi;
+
+	/*
+	 * Adjust the path target if the parent of the outer relation is referenced
+	 * in the targetlist. This can happen when only the parent of outer relation is
+	 * laterally referenced in this relation.
+	 */
+	if (bms_overlap(path->parent->lateral_relids, child_rel->top_parent_relids))
+	{
+		MemoryContext	old_context;
+
+		/*
+		 * Allocate the target in planner's context, since they are copies as
+		 * is from path while creating plans.
+		 */
+		old_context = MemoryContextSwitchTo(root->planner_cxt);
+		new_path->pathtarget = copy_pathtarget(new_path->pathtarget);
+		new_path->pathtarget->exprs = (List *) adjust_appendrel_attrs(root,
+											(Node *) new_path->pathtarget->exprs,
+																	child_aris);
+		MemoryContextSwitchTo(old_context);
+	}
+
+	/*
+	 * Change parameterization of subpaths recursively. Also carry out any
+	 * pathtype specific adjustments.
+	 */
+	switch (nodeTag(path))
+	{
+		case T_HashPath:
+		case T_MergePath:
+		case T_NestPath:
+			{
+				JoinPath *jpath = (JoinPath *)new_path;
+
+				jpath->outerjoinpath = reparameterize_path_by_child(root,
+														 jpath->outerjoinpath,
+														 child_rel);
+				jpath->innerjoinpath = reparameterize_path_by_child(root,
+														 jpath->innerjoinpath,
+														 child_rel);
+				jpath->joinrestrictinfo = build_child_clauses(root,
+															  jpath->joinrestrictinfo,
+															  child_aris);
+			}
+			break;
+
+		case T_AppendPath:
+			{
+				AppendPath	*apath = (AppendPath *)new_path;
+				List		*subpaths = NIL;
+				ListCell	*lc;
+
+				foreach (lc, apath->subpaths)
+					subpaths = lappend(subpaths,
+									   reparameterize_path_by_child(root,
+																	lfirst(lc),
+																	child_rel));
+				apath->subpaths = subpaths;
+			}
+
+		case T_IndexPath:
+			{
+				IndexPath *ipath = (IndexPath *)new_path;
+
+				ipath->indexclauses = build_child_clauses(root, ipath->indexclauses,
+														  child_aris);
+				ipath->indexquals = build_child_clauses(root, ipath->indexquals,
+														child_aris);
+			}
+
+		default:
+			/* Nothing to do. */
+			break;
+	}
+
+	return new_path;
+}
diff --git a/src/backend/optimizer/util/placeholder.c b/src/backend/optimizer/util/placeholder.c
index b210914..b67c62f 100644
--- a/src/backend/optimizer/util/placeholder.c
+++ b/src/backend/optimizer/util/placeholder.c
@@ -21,6 +21,7 @@
 #include "optimizer/placeholder.h"
 #include "optimizer/planmain.h"
 #include "optimizer/var.h"
+#include "optimizer/prep.h"
 #include "utils/lsyscache.h"
 
 /* Local functions */
@@ -414,6 +415,10 @@ add_placeholders_to_joinrel(PlannerInfo *root, RelOptInfo *joinrel,
 	Relids		relids = joinrel->relids;
 	ListCell   *lc;
 
+	/* This function is called only on the parent relations. */
+	Assert(!IS_OTHER_REL(joinrel) && !IS_OTHER_REL(outer_rel) &&
+		   !IS_OTHER_REL(inner_rel));
+
 	foreach(lc, root->placeholder_list)
 	{
 		PlaceHolderInfo *phinfo = (PlaceHolderInfo *) lfirst(lc);
@@ -424,9 +429,11 @@ add_placeholders_to_joinrel(PlannerInfo *root, RelOptInfo *joinrel,
 			/* Is it computable here? */
 			if (bms_is_subset(phinfo->ph_eval_at, relids))
 			{
+				PlaceHolderVar *phv = phinfo->ph_var;
+
 				/* Yup, add it to the output */
 				joinrel->reltarget->exprs = lappend(joinrel->reltarget->exprs,
-													phinfo->ph_var);
+													phv);
 				joinrel->reltarget->width += phinfo->ph_width;
 
 				/*
@@ -445,7 +452,7 @@ add_placeholders_to_joinrel(PlannerInfo *root, RelOptInfo *joinrel,
 				{
 					QualCost	cost;
 
-					cost_qual_eval_node(&cost, (Node *) phinfo->ph_var->phexpr,
+					cost_qual_eval_node(&cost, (Node *) phv->phexpr,
 										root);
 					joinrel->reltarget->cost.startup += cost.startup;
 					joinrel->reltarget->cost.per_tuple += cost.per_tuple;
@@ -459,3 +466,53 @@ add_placeholders_to_joinrel(PlannerInfo *root, RelOptInfo *joinrel,
 		}
 	}
 }
+
+/*
+ * add_placeholders_to_child_joinrel
+ *		Translate the PHVs in parent's targetlist and add them to the child's
+ *		targetlist. Also adjust the cost
+ */
+void
+add_placeholders_to_child_joinrel(PlannerInfo *root, RelOptInfo *childrel,
+								  RelOptInfo *parentrel)
+{
+	ListCell  *lc;
+
+	/* This function is called only for join relations. */
+	Assert(IS_JOIN_REL(childrel) && IS_JOIN_REL(parentrel));
+
+	/* Ensure child relations is really what it claims to be. */
+	Assert(IS_OTHER_REL(childrel));
+
+	foreach (lc, parentrel->reltarget->exprs)
+	{
+		PlaceHolderVar *phv = lfirst(lc);
+
+		if (IsA(phv, PlaceHolderVar))
+		{
+			/*
+			 * In case the placeholder Var refers to any of the parent
+			 * relations, translate it to refer to the corresponding child.
+			 */
+			if (bms_overlap(phv->phrels, parentrel->relids) &&
+				childrel->reloptkind == RELOPT_OTHER_JOINREL)
+			{
+				List *append_rel_infos;
+
+				append_rel_infos = find_appinfos_by_relids(root,
+														  childrel->relids);
+				phv = (PlaceHolderVar *) adjust_appendrel_attrs(root,
+															  (Node *) phv,
+														 append_rel_infos);
+			}
+
+			childrel->reltarget->exprs = lappend(childrel->reltarget->exprs,
+												 phv);
+		}
+	}
+
+	/* Adjust the cost and width of child targetlist. */
+	childrel->reltarget->cost.startup = parentrel->reltarget->cost.startup;
+	childrel->reltarget->cost.per_tuple = parentrel->reltarget->cost.per_tuple;
+	childrel->reltarget->width = parentrel->reltarget->width;
+}
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index 72272d9..88e66e4 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -412,6 +412,21 @@ get_relation_info(PlannerInfo *root, Oid relationObjectId, bool inhparent,
 	/* Collect info about relation's foreign keys, if relevant */
 	get_relation_foreign_keys(root, rel, relation, inhparent);
 
+	/*
+	 * Lookup partition scheme for the given relation. Only parent relations
+	 * can be partitioned.
+	 */
+	if (inhparent)
+		rel->part_scheme = find_partition_scheme(root, relation);
+	else
+		rel->part_scheme = NULL;
+
+	if (rel->part_scheme)
+		rel->partexprs = build_baserel_partition_key_exprs(relation,
+														   rel->relid);
+	else
+		rel->partexprs = NULL;
+
 	heap_close(relation, NoLock);
 
 	/*
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index d5326e6..a9a707f 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -15,15 +15,22 @@
 #include "postgres.h"
 
 #include "miscadmin.h"
+#include "catalog/heap.h"
+#include "catalog/partition.h"
 #include "optimizer/clauses.h"
 #include "optimizer/cost.h"
+#include "nodes/makefuncs.h"
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
 #include "optimizer/placeholder.h"
 #include "optimizer/plancat.h"
+#include "optimizer/prep.h"
 #include "optimizer/restrictinfo.h"
 #include "optimizer/tlist.h"
+#include "optimizer/var.h"
+#include "rewrite/rewriteManip.h"
 #include "utils/hsearch.h"
+#include "utils/rel.h"
 
 
 typedef struct JoinHashEntry
@@ -34,10 +41,6 @@ typedef struct JoinHashEntry
 
 static void build_joinrel_tlist(PlannerInfo *root, RelOptInfo *joinrel,
 					RelOptInfo *input_rel);
-static List *build_joinrel_restrictlist(PlannerInfo *root,
-						   RelOptInfo *joinrel,
-						   RelOptInfo *outer_rel,
-						   RelOptInfo *inner_rel);
 static void build_joinrel_joinlist(RelOptInfo *joinrel,
 					   RelOptInfo *outer_rel,
 					   RelOptInfo *inner_rel);
@@ -47,6 +50,11 @@ static List *subbuild_joinrel_restrictlist(RelOptInfo *joinrel,
 static List *subbuild_joinrel_joinlist(RelOptInfo *joinrel,
 						  List *joininfo_list,
 						  List *new_joininfo);
+static void set_foreign_rel_properties(RelOptInfo *joinrel,
+							RelOptInfo *outer_rel, RelOptInfo *inner_rel);
+static void build_joinrel_partition_info(RelOptInfo *joinrel,
+						  RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+						  JoinType jointype);
 
 
 /*
@@ -137,6 +145,10 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptKind reloptkind)
 	rel->baserestrictcost.per_tuple = 0;
 	rel->joininfo = NIL;
 	rel->has_eclass_joins = false;
+	rel->part_scheme = NULL;
+	rel->partexprs = NULL;
+	rel->top_parent_relids = NULL;
+	rel->part_rels = NULL;
 
 	/* Check type of rtable entry */
 	switch (rte->rtekind)
@@ -314,6 +326,56 @@ find_join_rel(PlannerInfo *root, Relids relids)
 }
 
 /*
+ * set_foreign_rel_properties
+ *		Set up foreign-join fields if outer and inner relation are foreign
+ *		tables (or joins) belonging to the same server and assigned to the same
+ *		user to check access permissions as.
+ *
+ * In addition to an exact match of userid, we allow the case where one side
+ * has zero userid (implying current user) and the other side has explicit
+ * userid that happens to equal the current user; but in that case, pushdown of
+ * the join is only valid for the current user.  The useridiscurrent field
+ * records whether we had to make such an assumption for this join or any
+ * sub-join.
+ *
+ * Otherwise these fields are left invalid, so GetForeignJoinPaths will not be
+ * called for the join relation.
+ *
+ */
+static void
+set_foreign_rel_properties(RelOptInfo *joinrel, RelOptInfo *outer_rel,
+						   RelOptInfo *inner_rel)
+{
+	if (OidIsValid(outer_rel->serverid) &&
+		inner_rel->serverid == outer_rel->serverid)
+	{
+		if (inner_rel->userid == outer_rel->userid)
+		{
+			joinrel->serverid = outer_rel->serverid;
+			joinrel->userid = outer_rel->userid;
+			joinrel->useridiscurrent = outer_rel->useridiscurrent || inner_rel->useridiscurrent;
+			joinrel->fdwroutine = outer_rel->fdwroutine;
+		}
+		else if (!OidIsValid(inner_rel->userid) &&
+				 outer_rel->userid == GetUserId())
+		{
+			joinrel->serverid = outer_rel->serverid;
+			joinrel->userid = outer_rel->userid;
+			joinrel->useridiscurrent = true;
+			joinrel->fdwroutine = outer_rel->fdwroutine;
+		}
+		else if (!OidIsValid(outer_rel->userid) &&
+				 inner_rel->userid == GetUserId())
+		{
+			joinrel->serverid = outer_rel->serverid;
+			joinrel->userid = inner_rel->userid;
+			joinrel->useridiscurrent = true;
+			joinrel->fdwroutine = outer_rel->fdwroutine;
+		}
+	}
+}
+
+/*
  * build_join_rel
  *	  Returns relation entry corresponding to the union of two given rels,
  *	  creating a new relation entry if none already exists.
@@ -363,7 +425,11 @@ build_join_rel(PlannerInfo *root,
 	 * Nope, so make one.
 	 */
 	joinrel = makeNode(RelOptInfo);
+
+	Assert(!IS_OTHER_REL(outer_rel) && !IS_OTHER_REL(inner_rel));
+
 	joinrel->reloptkind = RELOPT_JOINREL;
+
 	joinrel->relids = bms_copy(joinrelids);
 	joinrel->rows = 0;
 	/* cheap startup cost is interesting iff not all tuples to be retrieved */
@@ -409,47 +475,13 @@ build_join_rel(PlannerInfo *root,
 	joinrel->baserestrictcost.per_tuple = 0;
 	joinrel->joininfo = NIL;
 	joinrel->has_eclass_joins = false;
+	joinrel->part_scheme = NULL;
+	joinrel->partexprs = NULL;
+	joinrel->top_parent_relids = NULL;
+	joinrel->part_rels = NULL;
 
-	/*
-	 * Set up foreign-join fields if outer and inner relation are foreign
-	 * tables (or joins) belonging to the same server and assigned to the same
-	 * user to check access permissions as.  In addition to an exact match of
-	 * userid, we allow the case where one side has zero userid (implying
-	 * current user) and the other side has explicit userid that happens to
-	 * equal the current user; but in that case, pushdown of the join is only
-	 * valid for the current user.  The useridiscurrent field records whether
-	 * we had to make such an assumption for this join or any sub-join.
-	 *
-	 * Otherwise these fields are left invalid, so GetForeignJoinPaths will
-	 * not be called for the join relation.
-	 */
-	if (OidIsValid(outer_rel->serverid) &&
-		inner_rel->serverid == outer_rel->serverid)
-	{
-		if (inner_rel->userid == outer_rel->userid)
-		{
-			joinrel->serverid = outer_rel->serverid;
-			joinrel->userid = outer_rel->userid;
-			joinrel->useridiscurrent = outer_rel->useridiscurrent || inner_rel->useridiscurrent;
-			joinrel->fdwroutine = outer_rel->fdwroutine;
-		}
-		else if (!OidIsValid(inner_rel->userid) &&
-				 outer_rel->userid == GetUserId())
-		{
-			joinrel->serverid = outer_rel->serverid;
-			joinrel->userid = outer_rel->userid;
-			joinrel->useridiscurrent = true;
-			joinrel->fdwroutine = outer_rel->fdwroutine;
-		}
-		else if (!OidIsValid(outer_rel->userid) &&
-				 inner_rel->userid == GetUserId())
-		{
-			joinrel->serverid = outer_rel->serverid;
-			joinrel->userid = inner_rel->userid;
-			joinrel->useridiscurrent = true;
-			joinrel->fdwroutine = outer_rel->fdwroutine;
-		}
-	}
+	/* Compute information relevant to the foreign relations. */
+	set_foreign_rel_properties(joinrel, outer_rel, inner_rel);
 
 	/*
 	 * Create a new tlist containing just the vars that need to be output from
@@ -475,6 +507,10 @@ build_join_rel(PlannerInfo *root,
 	if (bms_is_empty(joinrel->direct_lateral_relids))
 		joinrel->direct_lateral_relids = NULL;
 
+	/* Store the partition information. */
+	build_joinrel_partition_info(joinrel, outer_rel, inner_rel,
+								 sjinfo->jointype);
+
 	/*
 	 * Construct restrict and join clause lists for the new joinrel. (The
 	 * caller might or might not need the restrictlist, but I need it anyway
@@ -517,25 +553,8 @@ build_join_rel(PlannerInfo *root,
 		is_parallel_safe(root, (Node *) joinrel->reltarget->exprs))
 		joinrel->consider_parallel = true;
 
-	/*
-	 * Add the joinrel to the query's joinrel list, and store it into the
-	 * auxiliary hashtable if there is one.  NB: GEQO requires us to append
-	 * the new joinrel to the end of the list!
-	 */
-	root->join_rel_list = lappend(root->join_rel_list, joinrel);
-
-	if (root->join_rel_hash)
-	{
-		JoinHashEntry *hentry;
-		bool		found;
-
-		hentry = (JoinHashEntry *) hash_search(root->join_rel_hash,
-											   &(joinrel->relids),
-											   HASH_ENTER,
-											   &found);
-		Assert(!found);
-		hentry->join_rel = joinrel;
-	}
+	/* Add the joinrel to the query's PlannerInfo. */
+	add_join_rel(root, joinrel);
 
 	/*
 	 * Also, if dynamic-programming join search is active, add the new joinrel
@@ -555,6 +574,127 @@ build_join_rel(PlannerInfo *root,
 }
 
 /*
+ * build_child_join_rel
+ *		Builds RelOptInfo for joining given two child relations from RelOptInfo
+ *		representing the join between their parents.
+ *
+ * 'outer_rel' and 'inner_rel' are the RelOptInfos of child relations being
+ *		joined.
+ * 'parent_joinrel' is the RelOptInfo representing the join between parent
+ *		relations. Most of the members of new RelOptInfo are produced by
+ *		translating corresponding members of this RelOptInfo.
+ * 'sjinfo': context info for child join
+ * 'restrictlist': list of RestrictInfo nodes that apply to this particular
+ *		pair of joinable relations.
+ * 'join_appinfos': list of AppendRelInfo nodes for base child relations involved
+ *		in this join.
+ */
+RelOptInfo *
+build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
+						 RelOptInfo *inner_rel, RelOptInfo *parent_joinrel,
+						 JoinType jointype)
+{
+	RelOptInfo *joinrel = makeNode(RelOptInfo);
+
+	joinrel->reloptkind = RELOPT_OTHER_JOINREL;
+	joinrel->relids = bms_union(outer_rel->relids, inner_rel->relids);
+	joinrel->rows = 0;
+	/* cheap startup cost is interesting iff not all tuples to be retrieved */
+	joinrel->consider_startup = (root->tuple_fraction > 0);
+	joinrel->consider_param_startup = false;
+	joinrel->consider_parallel = false;
+	joinrel->reltarget = create_empty_pathtarget();
+	joinrel->pathlist = NIL;
+	joinrel->ppilist = NIL;
+	joinrel->partial_pathlist = NIL;
+	joinrel->cheapest_startup_path = NULL;
+	joinrel->cheapest_total_path = NULL;
+	joinrel->cheapest_unique_path = NULL;
+	joinrel->cheapest_parameterized_paths = NIL;
+	joinrel->direct_lateral_relids = NULL;
+	joinrel->lateral_relids = NULL;
+	joinrel->relid = 0;			/* indicates not a baserel */
+	joinrel->rtekind = RTE_JOIN;
+	joinrel->min_attr = 0;
+	joinrel->max_attr = 0;
+	joinrel->attr_needed = NULL;
+	joinrel->attr_widths = NULL;
+	joinrel->lateral_vars = NIL;
+	joinrel->lateral_referencers = NULL;
+	joinrel->indexlist = NIL;
+	joinrel->pages = 0;
+	joinrel->tuples = 0;
+	joinrel->allvisfrac = 0;
+	joinrel->subroot = NULL;
+	joinrel->subplan_params = NIL;
+	joinrel->serverid = InvalidOid;
+	joinrel->userid = InvalidOid;
+	joinrel->useridiscurrent = false;
+	joinrel->fdwroutine = NULL;
+	joinrel->fdw_private = NULL;
+	joinrel->baserestrictinfo = NIL;
+	joinrel->baserestrictcost.startup = 0;
+	joinrel->baserestrictcost.per_tuple = 0;
+	joinrel->joininfo = NIL;
+	joinrel->has_eclass_joins = false;
+	joinrel->part_scheme = NULL;
+	joinrel->partexprs = NULL;
+	joinrel->top_parent_relids = NULL;
+	joinrel->part_rels = NULL;
+
+
+	/* Only joins between other relations land here. */
+	Assert(IS_OTHER_REL(outer_rel) && IS_OTHER_REL(inner_rel));
+
+	joinrel->top_parent_relids = bms_union(outer_rel->top_parent_relids,
+										   inner_rel->top_parent_relids);
+
+	/* Compute information relevant to foreign relations. */
+	set_foreign_rel_properties(joinrel, outer_rel, inner_rel);
+
+	/* Build targetlist */
+	build_joinrel_tlist(root, joinrel, outer_rel);
+	build_joinrel_tlist(root, joinrel, inner_rel);
+	/* Add placeholder variables. */
+	add_placeholders_to_child_joinrel(root, joinrel, parent_joinrel);
+
+	/* Construct joininfo list. */
+	joinrel->joininfo = build_child_clauses(root, parent_joinrel->joininfo,
+											find_appinfos_by_relids(root,
+															 joinrel->relids));
+
+	/*
+	 * Lateral relids referred in child join will be same as that referred in
+	 * the parent relation. Throw any partial result computed while building
+	 * the targetlist.
+	 */
+	bms_free(joinrel->direct_lateral_relids);
+	bms_free(joinrel->lateral_relids);
+	joinrel->direct_lateral_relids = (Relids) bms_copy(parent_joinrel->direct_lateral_relids);
+	joinrel->lateral_relids = (Relids) bms_copy(parent_joinrel->lateral_relids);
+
+	/*
+	 * If the parent joinrel has pending equivalence classes, so does the
+	 * child.
+	 */
+	joinrel->has_eclass_joins = parent_joinrel->has_eclass_joins;
+
+	/* Is the join between partitions itself partitioned? */
+	build_joinrel_partition_info(joinrel, outer_rel, inner_rel, jointype);
+
+	/* Child joinrel is parallel safe if parent is parallel safe. */
+	joinrel->consider_parallel = parent_joinrel->consider_parallel;
+
+	/* We build the join only once. */
+	Assert(!find_join_rel(root, joinrel->relids));
+
+	/* Add the relation to the PlannerInfo. */
+	add_join_rel(root, joinrel);
+
+	return joinrel;
+}
+
+/*
  * min_join_parameterization
  *
  * Determine the minimum possible parameterization of a joinrel, that is, the
@@ -609,9 +749,15 @@ static void
 build_joinrel_tlist(PlannerInfo *root, RelOptInfo *joinrel,
 					RelOptInfo *input_rel)
 {
-	Relids		relids = joinrel->relids;
+	Relids		relids;
 	ListCell   *vars;
 
+	/* attrs_needed refers to parent relids and not those of a child. */
+	if (joinrel->top_parent_relids)
+		relids = joinrel->top_parent_relids;
+	else
+		relids = joinrel->relids;
+
 	foreach(vars, input_rel->reltarget->exprs)
 	{
 		Var		   *var = (Var *) lfirst(vars);
@@ -627,23 +773,47 @@ build_joinrel_tlist(PlannerInfo *root, RelOptInfo *joinrel,
 
 		/*
 		 * Otherwise, anything in a baserel or joinrel targetlist ought to be
-		 * a Var.  (More general cases can only appear in appendrel child
-		 * rels, which will never be seen here.)
+		 * a Var or ConvertRowtypeExpr introduced while translating parent
+		 * targetlist to that of the child.
 		 */
-		if (!IsA(var, Var))
+		if (IsA(var, Var))
+		{
+			/* Get the Var's original base rel */
+			baserel = find_base_rel(root, var->varno);
+
+			/* Is it still needed above this joinrel? */
+			ndx = var->varattno - baserel->min_attr;
+		}
+		else if (IsA(var, ConvertRowtypeExpr))
+		{
+			ConvertRowtypeExpr *child_expr = (ConvertRowtypeExpr *) var;
+			Var	 *childvar = (Var *) child_expr->arg;
+
+			/*
+			 * Child's whole-row references are converted to that of parent
+			 * using ConvertRowtypeExpr. In this case, the argument to
+			 * ConvertRowtypeExpr is expected to be a whole-row reference of
+			 * the child.
+			 */
+			Assert(IsA(childvar, Var) && childvar->varattno == 0);
+
+			baserel = find_base_rel(root, childvar->varno);
+			ndx = 0 - baserel->min_attr;
+		}
+		else
 			elog(ERROR, "unexpected node type in rel targetlist: %d",
 				 (int) nodeTag(var));
 
-		/* Get the Var's original base rel */
-		baserel = find_base_rel(root, var->varno);
-
-		/* 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 */
 			joinrel->reltarget->exprs = lappend(joinrel->reltarget->exprs, var);
-			/* Vars have cost zero, so no need to adjust reltarget->cost */
+
+			/*
+			 * Vars have cost zero, so no need to adjust reltarget->cost. Even
+			 * if, it's a ConvertRowtypeExpr, it will be computed only for the
+			 * base relation, costing nothing for a join.
+			 */
 			joinrel->reltarget->width += baserel->attr_widths[ndx];
 		}
 	}
@@ -691,7 +861,7 @@ build_joinrel_tlist(PlannerInfo *root, RelOptInfo *joinrel,
  * RestrictInfo nodes are no longer context-dependent.  Instead, just include
  * the original nodes in the lists made for the join relation.
  */
-static List *
+List *
 build_joinrel_restrictlist(PlannerInfo *root,
 						   RelOptInfo *joinrel,
 						   RelOptInfo *outer_rel,
@@ -780,6 +950,8 @@ subbuild_joinrel_joinlist(RelOptInfo *joinrel,
 {
 	ListCell   *l;
 
+	Assert(joinrel->reloptkind == RELOPT_JOINREL);
+
 	foreach(l, joininfo_list)
 	{
 		RestrictInfo *rinfo = (RestrictInfo *) lfirst(l);
@@ -808,7 +980,6 @@ subbuild_joinrel_joinlist(RelOptInfo *joinrel,
 	return new_joininfo;
 }
 
-
 /*
  * build_empty_join_rel
  *		Build a dummy join relation describing an empty set of base rels.
@@ -933,9 +1104,17 @@ find_childrel_appendrelinfo(PlannerInfo *root, RelOptInfo *rel)
  * appendrel ancestors.  This function locates the topmost ancestor,
  * which will be a regular baserel not an otherrel.
  */
-RelOptInfo *
+Relids
 find_childrel_top_parent(PlannerInfo *root, RelOptInfo *rel)
 {
+	Assert(IS_OTHER_REL(rel));
+
+	/* Child-join relations cache this in their RelOptInfo. */
+	if (rel->reloptkind == RELOPT_OTHER_JOINREL)
+		return rel->top_parent_relids;
+
+	Assert(rel->reloptkind == RELOPT_OTHER_MEMBER_REL);
+
 	do
 	{
 		AppendRelInfo *appinfo = find_childrel_appendrelinfo(root, rel);
@@ -947,7 +1126,7 @@ find_childrel_top_parent(PlannerInfo *root, RelOptInfo *rel)
 
 	Assert(rel->reloptkind == RELOPT_BASEREL);
 
-	return rel;
+	return rel->relids;
 }
 
 
@@ -1009,12 +1188,8 @@ get_baserel_parampathinfo(PlannerInfo *root, RelOptInfo *baserel,
 	Assert(!bms_overlap(baserel->relids, required_outer));
 
 	/* If we already have a PPI for this parameterization, just return it */
-	foreach(lc, baserel->ppilist)
-	{
-		ppi = (ParamPathInfo *) lfirst(lc);
-		if (bms_equal(ppi->ppi_req_outer, required_outer))
-			return ppi;
-	}
+	if ((ppi = find_param_path_info(baserel, required_outer)))
+		return ppi;
 
 	/*
 	 * Identify all joinclauses that are movable to this base rel given this
@@ -1251,12 +1426,8 @@ get_joinrel_parampathinfo(PlannerInfo *root, RelOptInfo *joinrel,
 	*restrict_clauses = list_concat(pclauses, *restrict_clauses);
 
 	/* If we already have a PPI for this parameterization, just return it */
-	foreach(lc, joinrel->ppilist)
-	{
-		ppi = (ParamPathInfo *) lfirst(lc);
-		if (bms_equal(ppi->ppi_req_outer, required_outer))
-			return ppi;
-	}
+	if ((ppi = find_param_path_info(joinrel, required_outer)))
+		return ppi;
 
 	/* Estimate the number of rows returned by the parameterized join */
 	rows = get_parameterized_joinrel_size(root, joinrel,
@@ -1295,7 +1466,6 @@ ParamPathInfo *
 get_appendrel_parampathinfo(RelOptInfo *appendrel, Relids required_outer)
 {
 	ParamPathInfo *ppi;
-	ListCell   *lc;
 
 	/* Unparameterized paths have no ParamPathInfo */
 	if (bms_is_empty(required_outer))
@@ -1304,12 +1474,8 @@ get_appendrel_parampathinfo(RelOptInfo *appendrel, Relids required_outer)
 	Assert(!bms_overlap(appendrel->relids, required_outer));
 
 	/* If we already have a PPI for this parameterization, just return it */
-	foreach(lc, appendrel->ppilist)
-	{
-		ppi = (ParamPathInfo *) lfirst(lc);
-		if (bms_equal(ppi->ppi_req_outer, required_outer))
-			return ppi;
-	}
+	if ((ppi = find_param_path_info(appendrel, required_outer)))
+		return ppi;
 
 	/* Else build the ParamPathInfo */
 	ppi = makeNode(ParamPathInfo);
@@ -1320,3 +1486,130 @@ get_appendrel_parampathinfo(RelOptInfo *appendrel, Relids required_outer)
 
 	return ppi;
 }
+
+/*
+ * add_join_rel
+ *		Add given join relation to the list of join relations in the given
+ *		PlannerInfo. Also add it to the auxiliary hashtable if there is one.
+ */
+void
+add_join_rel(PlannerInfo *root, RelOptInfo *joinrel)
+{
+	/* GEQO requires us to append the new joinrel to the end of the list! */
+	root->join_rel_list = lappend(root->join_rel_list, joinrel);
+
+	/* store it into the auxiliary hashtable if there is one. */
+	if (root->join_rel_hash)
+	{
+		JoinHashEntry *hentry;
+		bool		found;
+
+		hentry = (JoinHashEntry *) hash_search(root->join_rel_hash,
+											   &(joinrel->relids),
+											   HASH_ENTER,
+											   &found);
+		Assert(!found);
+		hentry->join_rel = joinrel;
+	}
+}
+
+/*
+ * build_joinrel_partition_info
+ *		If the join between given partitioned relations is possibly partitioned
+ *		set the partitioning scheme and partition keys expressions for the
+ *		join.
+ *
+ * If the two relations have same partitioning scheme, their join may be
+ * partitioned and will follow the same partitioning scheme as the joining
+ * relations.
+ */
+static void
+build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
+							 RelOptInfo *inner_rel, JoinType jointype)
+{
+	int		num_pks;
+	int		cnt;
+
+	/* Nothing to do if partition-wise join technique is disabled. */
+	if (!enable_partition_wise_join)
+	{
+		joinrel->part_scheme = NULL;
+		return;
+	}
+
+	/*
+	 * The join is not partitioned, if any of the relations being joined are
+	 * not partitioned or they do not have same partitioning scheme.
+	 */
+	if (!outer_rel->part_scheme || !inner_rel->part_scheme ||
+		outer_rel->part_scheme != inner_rel->part_scheme)
+	{
+		joinrel->part_scheme = NULL;
+		return;
+	}
+
+	/*
+	 * This function will be called only once for each joinrel, hence it should
+	 * not have partition scheme, partition key expressions and array for
+	 * storing child relations set.
+	 */
+	Assert(!joinrel->part_scheme && !joinrel->partexprs &&
+		   !joinrel->part_rels);
+
+	/*
+	 * Join relation is partitioned using same partitioning scheme as the
+	 * joining relations.
+	 */
+	joinrel->part_scheme = outer_rel->part_scheme;
+	num_pks = joinrel->part_scheme->partnatts;
+
+	/*
+	 * Construct partition keys for the join.
+	 *
+	 * An INNER join between two partitioned relations is partition by key
+	 * expressions from both the relations. For tables A and B partitioned by a and b
+	 * respectively, (A INNER JOIN B ON A.a = B.b) is partitioned by both A.a
+	 * and B.b.
+	 *
+	 * An OUTER join like (A LEFT JOIN B ON A.a = B.b) may produce rows with
+	 * B.b NULL. These rows may not fit the partitioning conditions imposed on
+	 * B.b. Hence, strictly speaking, the join is not partitioned by B.b.
+	 * Strictly speaking, partition keys of an OUTER join should include
+	 * partition key expressions from the OUTER side only. Consider a join like
+	 * (A LEFT JOIN B on (A.a = B.b) LEFT JOIN C ON B.b = C.c. If we do not
+	 * include B.b as partition key expression for (AB), it prohibits us from
+	 * using partition-wise join when joining (AB) with C as there is no
+	 * equi-join between partition keys of joining relations. But two NULL
+	 * values are never equal and no two rows from mis-matching partitions can
+	 * join. Hence it's safe to include B.b as partition key expression for
+	 * (AB), even though rows in (AB) are not strictly partitioned by B.b.
+	 */
+	joinrel->partexprs = (List **) palloc0(sizeof(List *) * num_pks);
+	for (cnt = 0; cnt < num_pks; cnt++)
+	{
+		List *pkexpr = list_copy(outer_rel->partexprs[cnt]);
+
+		pkexpr = list_concat(pkexpr,
+							 list_copy(inner_rel->partexprs[cnt]));
+		joinrel->partexprs[cnt] = pkexpr;
+	}
+}
+
+/*
+ * Returns a ParamPathInfo for outer relations specified by required_outer, if
+ * already available in the given rel. Returns NULL otherwise.
+ */
+ParamPathInfo *
+find_param_path_info(RelOptInfo *rel, Relids required_outer)
+{
+	ListCell   *lc;
+
+	foreach(lc, rel->ppilist)
+	{
+		ParamPathInfo  *ppi = (ParamPathInfo *) lfirst(lc);
+		if (bms_equal(ppi->ppi_req_outer, required_outer))
+			return ppi;
+	}
+
+	return NULL;
+}
diff --git a/src/backend/postmaster/startup.c b/src/backend/postmaster/startup.c
index a7ae7e3..6787df6 100644
--- a/src/backend/postmaster/startup.c
+++ b/src/backend/postmaster/startup.c
@@ -169,7 +169,6 @@ HandleStartupProcInterrupts(void)
 		exit(1);
 }
 
-
 /* ----------------------------------
  *	Startup Process main entry point
  * ----------------------------------
diff --git a/src/backend/utils/adt/selfuncs.c b/src/backend/utils/adt/selfuncs.c
index 4973396..713e361 100644
--- a/src/backend/utils/adt/selfuncs.c
+++ b/src/backend/utils/adt/selfuncs.c
@@ -3426,7 +3426,9 @@ estimate_num_groups(PlannerInfo *root, List *groupExprs, double input_rows,
 		/*
 		 * Sanity check --- don't divide by zero if empty relation.
 		 */
-		Assert(rel->reloptkind == RELOPT_BASEREL);
+		Assert(rel->reloptkind == RELOPT_BASEREL ||
+			   rel->reloptkind == RELOPT_OTHER_MEMBER_REL);
+
 		if (rel->tuples > 0)
 		{
 			/*
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 946ba9e..f3d40a1 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -893,6 +893,15 @@ static struct config_bool ConfigureNamesBool[] =
 		true,
 		NULL, NULL, NULL
 	},
+	{
+		{"enable_partition_wise_join", PGC_USERSET, QUERY_TUNING_METHOD,
+			gettext_noop("Enables partition-wise join."),
+			NULL
+		},
+		&enable_partition_wise_join,
+		true,
+		NULL, NULL, NULL
+	},
 
 	{
 		{"geqo", PGC_USERSET, QUERY_TUNING_GEQO,
@@ -2927,6 +2936,16 @@ static struct config_real ConfigureNamesReal[] =
 	},
 
 	{
+		{"partition_wise_plan_weight", PGC_USERSET, QUERY_TUNING_OTHER,
+			gettext_noop("Multiplication factor for partition-wise plan costs."),
+			NULL
+		},
+		&partition_wise_plan_weight,
+		DEFAULT_PARTITION_WISE_PLAN_WEIGHT, 0, DBL_MAX,
+		NULL, NULL, NULL
+	},
+
+	{
 		{"bgwriter_lru_multiplier", PGC_SIGHUP, RESOURCES_BGWRITER,
 			gettext_noop("Multiple of the average buffer usage to free per round."),
 			NULL
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index bf38df5..90c0179 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -70,6 +70,42 @@ typedef struct PartitionDispatchData
 
 typedef struct PartitionDispatchData *PartitionDispatch;
 
+/*
+ * Partitioning scheme
+ *		Structure to hold partitioning scheme for a given relation.
+ *
+ * Multiple relations may be partitioned in the same way. The relations
+ * resulting from joining such relations may be partitioned in the same way as
+ * the joining relations. Similarly, relations derived from such relations by
+ * grouping, sorting may be partitioned in the same way as the underlying
+ * scan relations. All such relations partitioned in the same way share the
+ * partitioning scheme.
+ *
+ * PlannerInfo stores a list of distinct "canonical" partitioning schemes.
+ * RelOptInfo of a partitioned relation holds the pointer to "canonical"
+ * partitioning scheme.
+ */
+typedef struct PartitionSchemeData
+{
+	/* Information about partitions */
+	int			nparts;			/* number of partitions */
+	PartitionBoundInfo boundinfo;	/* Partition bounds/lists */
+
+	/* Information about partition keys */
+	char		strategy;		/* partition strategy */
+	int16		partnatts;		/* number of partition attributes */
+	Oid		   *partopfamily;	/* OIDs of operator families */
+	Oid		   *partopcintype;	/* OIDs of opclass declared input data types */
+	Oid		   *key_types;		/* OIDs of partition key data types. */
+	int32	   *key_typmods;	/* typmods of partition keys. */
+	Oid		   *key_collations;	/* OIDs of collations of partition keys. */
+} PartitionSchemeData;
+
+typedef struct PartitionSchemeData *PartitionScheme;
+
+/* Include here to avoid circular dependency with relation.h. */
+struct PlannerInfo;
+
 extern void RelationBuildPartitionDesc(Relation relation);
 extern bool partition_bounds_equal(PartitionKey key,
 					   PartitionBoundInfo p1, PartitionBoundInfo p2);
@@ -87,4 +123,9 @@ extern int get_partition_for_tuple(PartitionDispatch *pd,
 					TupleTableSlot *slot,
 					EState *estate,
 					Oid *failed_at);
+extern List **build_baserel_partition_key_exprs(Relation relation,
+												Index varno);
+extern PartitionScheme find_partition_scheme(struct PlannerInfo *root,
+											 Relation rel);
+
 #endif   /* PARTITION_H */
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index 201f248..42775d0 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -238,6 +238,7 @@ typedef enum NodeTag
 	T_NestPath,
 	T_MergePath,
 	T_HashPath,
+	T_PartitionJoinPath,
 	T_AppendPath,
 	T_MergeAppendPath,
 	T_ResultPath,
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 3a1255a..df5b60f 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 "catalog/partition.h"
 #include "lib/stringinfo.h"
 #include "nodes/params.h"
 #include "nodes/parsenodes.h"
@@ -263,6 +264,9 @@ typedef struct PlannerInfo
 
 	List	   *initial_rels;	/* RelOptInfos we are now trying to join */
 
+	List	   *part_schemes;	/* Canonicalised partition schemes
+								 * used in the query. */
+
 	/* Use fetch_upper_rel() to get any particular upper rel */
 	List	   *upper_rels[UPPERREL_FINAL + 1]; /* upper-rel RelOptInfos */
 
@@ -352,6 +356,11 @@ typedef struct PlannerInfo
  * handling join alias Vars.  Currently this is not needed because all join
  * alias Vars are expanded to non-aliased form during preprocess_expression.
  *
+ * We also have relations representing joins between child relations of
+ * different partitioned tables. These relations are not added to
+ * join_rel_level lists as they are not joined directly by the dynamic
+ * programming algorithm.
+ *
  * There is also a RelOptKind for "upper" relations, which are RelOptInfos
  * that describe post-scan/join processing steps, such as aggregation.
  * Many of the fields in these RelOptInfos are meaningless, but their Path
@@ -471,10 +480,19 @@ typedef enum RelOptKind
 	RELOPT_BASEREL,
 	RELOPT_JOINREL,
 	RELOPT_OTHER_MEMBER_REL,
+	RELOPT_OTHER_JOINREL,
 	RELOPT_UPPER_REL,
 	RELOPT_DEADREL
 } RelOptKind;
 
+#define IS_OTHER_REL(rel) \
+	((rel)->reloptkind == RELOPT_OTHER_MEMBER_REL || \
+	 (rel)->reloptkind == RELOPT_OTHER_JOINREL)
+
+#define IS_JOIN_REL(rel)	\
+	((rel)->reloptkind == RELOPT_JOINREL || \
+	 (rel)->reloptkind == RELOPT_OTHER_JOINREL)
+
 typedef struct RelOptInfo
 {
 	NodeTag		type;
@@ -542,6 +560,27 @@ typedef struct RelOptInfo
 	List	   *joininfo;		/* RestrictInfo structures for join clauses
 								 * involving this rel */
 	bool		has_eclass_joins;		/* T means joininfo is incomplete */
+
+	/* For partitioned relations. */
+	PartitionScheme	    part_scheme;	/* Partitioning scheme. */
+	struct RelOptInfo **part_rels;		/* Array of RelOptInfos of partitions,
+										 * stored in the same order as bounds
+										 * or lists in PartitionScheme.
+										 */
+	List			  **partexprs;		/* Array of list of partition key
+										 * expressions. For base relations
+										 * these are one element lists. For
+										 * join there may be as many elements
+										 * as the number of joining
+										 * relations.
+										 */
+	List			   *partitioned_joins;	/* List of join orders which yield
+											 * relations partitioned by above
+											 * partition scheme.
+											 */
+
+	/* Set only for "other" base or join relations. */
+	Relids		top_parent_relids;		/* Relids of topmost parents. */
 } RelOptInfo;
 
 /*
@@ -1469,6 +1508,14 @@ typedef struct LimitPath
 	Node	   *limitCount;		/* COUNT parameter, or NULL if none */
 } LimitPath;
 
+/*
+ * PartitionJoinPath represents partition-wise join between two partitioned
+ * tables.
+ */
+typedef struct PartitionJoinPath
+{
+	Path		path;
+} PartitionJoinPath;
 
 /*
  * Restriction clause info.
@@ -1663,6 +1710,12 @@ typedef struct RestrictInfo
 	/* cache space for hashclause processing; -1 if not yet set */
 	Selectivity left_bucketsize;	/* avg bucketsize of left side */
 	Selectivity right_bucketsize;		/* avg bucketsize of right side */
+
+	/* Only one of these two can be set. */
+	List	   *child_rinfos;		/* RestrictInfos derived for children. */
+	struct RestrictInfo *parent_rinfo;		/* Parent restrictinfo this
+											 * RestrictInf is derived from.
+											 */
 } RestrictInfo;
 
 /*
@@ -1785,6 +1838,21 @@ typedef struct SpecialJoinInfo
 } SpecialJoinInfo;
 
 /*
+ * Partitioned join information
+ *
+ * Saves information about relations which can be joined partition-wise and
+ * thus produce result which is partitioned by the partition scheme of the
+ * relation.
+ */
+typedef struct PartitionedJoin
+{
+	RelOptInfo	   *rel1;
+	RelOptInfo	   *rel2;
+	SpecialJoinInfo	   *sjinfo;	/* SpecialJoinInfo applicable. */
+	List		   *restrictlist;	/* applicable join clauses. */
+} PartitionedJoin;
+
+/*
  * Append-relation info.
  *
  * When we expand an inheritable table or a UNION-ALL subselect into an
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 2a4df2f..aff7ab7 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -30,6 +30,7 @@
 #define DEFAULT_PARALLEL_SETUP_COST  1000.0
 
 #define DEFAULT_EFFECTIVE_CACHE_SIZE  524288	/* measured in pages */
+#define DEFAULT_PARTITION_WISE_PLAN_WEIGHT 1
 
 typedef enum
 {
@@ -66,7 +67,9 @@ extern bool enable_nestloop;
 extern bool enable_material;
 extern bool enable_mergejoin;
 extern bool enable_hashjoin;
+extern bool enable_partition_wise_join;
 extern int	constraint_exclusion;
+extern double partition_wise_plan_weight;
 
 extern double clamp_row_est(double nrows);
 extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 71d9154..d75d56f 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -225,10 +225,17 @@ extern LimitPath *create_limit_path(PlannerInfo *root, RelOptInfo *rel,
 				  Path *subpath,
 				  Node *limitOffset, Node *limitCount,
 				  int64 offset_est, int64 count_est);
+extern PartitionJoinPath *create_partition_join_path(RelOptInfo *rel,
+								 List *subpaths, Bitmapset *required_outer);
+extern PartitionJoinPath *create_partition_join_path_with_pathkeys(PlannerInfo *root,
+								 RelOptInfo *rel, List *subpaths,
+								 List *pathkeys, Bitmapset *required_outer);
 
 extern Path *reparameterize_path(PlannerInfo *root, Path *path,
 					Relids required_outer,
 					double loop_count);
+extern Path *reparameterize_path_by_child(PlannerInfo *root, Path *path,
+					RelOptInfo *child_rel);
 
 /*
  * prototypes for relnode.c
@@ -253,7 +260,7 @@ extern RelOptInfo *fetch_upper_rel(PlannerInfo *root, UpperRelationKind kind,
 				Relids relids);
 extern AppendRelInfo *find_childrel_appendrelinfo(PlannerInfo *root,
 							RelOptInfo *rel);
-extern RelOptInfo *find_childrel_top_parent(PlannerInfo *root, RelOptInfo *rel);
+extern Relids find_childrel_top_parent(PlannerInfo *root, RelOptInfo *rel);
 extern Relids find_childrel_parents(PlannerInfo *root, RelOptInfo *rel);
 extern ParamPathInfo *get_baserel_parampathinfo(PlannerInfo *root,
 						  RelOptInfo *baserel,
@@ -267,5 +274,15 @@ extern ParamPathInfo *get_joinrel_parampathinfo(PlannerInfo *root,
 						  List **restrict_clauses);
 extern ParamPathInfo *get_appendrel_parampathinfo(RelOptInfo *appendrel,
 							Relids required_outer);
+extern RelOptInfo *build_child_join_rel(PlannerInfo *root,
+						 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+						 RelOptInfo *parent_joinrel, JoinType jointype);
+extern List *build_joinrel_restrictlist(PlannerInfo *root,
+						   RelOptInfo *joinrel,
+						   RelOptInfo *outer_rel,
+						   RelOptInfo *inner_rel);
+extern void add_join_rel(PlannerInfo *root, RelOptInfo *joinrel);
+extern ParamPathInfo *find_param_path_info(RelOptInfo *rel,
+									  Relids required_outer);
 
 #endif   /* PATHNODE_H */
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 44abe83..5d7bcd9 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -53,6 +53,8 @@ extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 					 List *initial_rels);
 
 extern void generate_gather_paths(PlannerInfo *root, RelOptInfo *rel);
+extern void generate_partition_wise_join_paths(PlannerInfo *root,
+											   RelOptInfo *rel);
 
 #ifdef OPTIMIZER_DEBUG
 extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
@@ -104,6 +106,9 @@ extern bool have_join_order_restriction(PlannerInfo *root,
 							RelOptInfo *rel1, RelOptInfo *rel2);
 extern bool have_dangerous_phv(PlannerInfo *root,
 				   Relids outer_relids, Relids inner_params);
+extern void add_paths_to_child_joinrel(PlannerInfo *root,
+									   RelOptInfo *parent_joinrel,
+									   int child_id);
 
 /*
  * equivclass.c
@@ -219,4 +224,6 @@ extern PathKey *make_canonical_pathkey(PlannerInfo *root,
 					   EquivalenceClass *eclass, Oid opfamily,
 					   int strategy, bool nulls_first);
 
+extern Relids adjust_child_relids(Relids relids, List *append_rel_infos);
+
 #endif   /* PATHS_H */
diff --git a/src/include/optimizer/placeholder.h b/src/include/optimizer/placeholder.h
index 54d0216..1652eca 100644
--- a/src/include/optimizer/placeholder.h
+++ b/src/include/optimizer/placeholder.h
@@ -28,5 +28,7 @@ extern void fix_placeholder_input_needed_levels(PlannerInfo *root);
 extern void add_placeholders_to_base_rels(PlannerInfo *root);
 extern void add_placeholders_to_joinrel(PlannerInfo *root, RelOptInfo *joinrel,
 							RelOptInfo *outer_rel, RelOptInfo *inner_rel);
+extern void add_placeholders_to_child_joinrel(PlannerInfo *root,
+							RelOptInfo *childrel, RelOptInfo *parentrel);
 
 #endif   /* PLACEHOLDER_H */
diff --git a/src/include/optimizer/prep.h b/src/include/optimizer/prep.h
index fb35b68..2483303 100644
--- a/src/include/optimizer/prep.h
+++ b/src/include/optimizer/prep.h
@@ -28,6 +28,9 @@ extern void flatten_simple_union_all(PlannerInfo *root);
 extern void reduce_outer_joins(PlannerInfo *root);
 extern Relids get_relids_in_jointree(Node *jtnode, bool include_joins);
 extern Relids get_relids_for_join(PlannerInfo *root, int joinrelid);
+extern Node *adjust_join_appendrel_attrs(PlannerInfo *root, Node *node,
+										 List *append_rel_infos);
+extern List *find_appinfos_by_relids(PlannerInfo *root, Relids relids);
 
 /*
  * prototypes for prepqual.c
@@ -58,9 +61,13 @@ extern RelOptInfo *plan_set_operations(PlannerInfo *root);
 extern void expand_inherited_tables(PlannerInfo *root);
 
 extern Node *adjust_appendrel_attrs(PlannerInfo *root, Node *node,
-					   AppendRelInfo *appinfo);
+					   List *appinfos);
 
 extern Node *adjust_appendrel_attrs_multilevel(PlannerInfo *root, Node *node,
 								  RelOptInfo *child_rel);
+extern RestrictInfo *build_child_restrictinfo(PlannerInfo *root,
+								 RestrictInfo *rinfo, List *append_rel_infos);
+extern List *build_child_clauses(PlannerInfo *root, List *clauses,
+								 List *append_rel_infos);
 
 #endif   /* PREP_H */
diff --git a/src/test/regress/expected/multi_level_partition_join.out b/src/test/regress/expected/multi_level_partition_join.out
new file mode 100644
index 0000000..d40ae55
--- /dev/null
+++ b/src/test/regress/expected/multi_level_partition_join.out
@@ -0,0 +1,458 @@
+--
+-- multi-leveled partitions
+--
+CREATE TABLE prt1_l (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_l_p1 PARTITION OF prt1_l FOR VALUES FROM (0) TO (250) PARTITION BY RANGE (b);
+CREATE TABLE prt1_l_p1_p1 PARTITION OF prt1_l_p1 FOR VALUES FROM (0) TO (100);
+CREATE TABLE prt1_l_p1_p2 PARTITION OF prt1_l_p1 FOR VALUES FROM (100) TO (250);
+CREATE TABLE prt1_l_p2 PARTITION OF prt1_l FOR VALUES FROM (250) TO (500) PARTITION BY RANGE (c);
+CREATE TABLE prt1_l_p2_p1 PARTITION OF prt1_l_p2 FOR VALUES FROM ('0250') TO ('0400');
+CREATE TABLE prt1_l_p2_p2 PARTITION OF prt1_l_p2 FOR VALUES FROM ('0400') TO ('0500');
+CREATE TABLE prt1_l_p3 PARTITION OF prt1_l FOR VALUES FROM (500) TO (600) PARTITION BY RANGE ((b + a));
+CREATE TABLE prt1_l_p3_p1 PARTITION OF prt1_l_p3 FOR VALUES FROM (1000) TO (1100);
+CREATE TABLE prt1_l_p3_p2 PARTITION OF prt1_l_p3 FOR VALUES FROM (1100) TO (1200);
+INSERT INTO prt1_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_l;
+ANALYZE prt1_l_p1;
+ANALYZE prt1_l_p1_p1;
+ANALYZE prt1_l_p1_p2;
+ANALYZE prt1_l_p2;
+ANALYZE prt1_l_p2_p1;
+ANALYZE prt1_l_p2_p2;
+ANALYZE prt1_l_p3;
+ANALYZE prt1_l_p3_p1;
+ANALYZE prt1_l_p3_p2;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_l AS SELECT * FROM prt1_l;
+CREATE TABLE prt2_l (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_l_p1 PARTITION OF prt2_l FOR VALUES FROM (0) TO (250) PARTITION BY RANGE (a);
+CREATE TABLE prt2_l_p1_p1 PARTITION OF prt2_l_p1 FOR VALUES FROM (0) TO (100);
+CREATE TABLE prt2_l_p1_p2 PARTITION OF prt2_l_p1 FOR VALUES FROM (100) TO (250);
+CREATE TABLE prt2_l_p2 PARTITION OF prt2_l FOR VALUES FROM (250) TO (500) PARTITION BY RANGE (c);
+CREATE TABLE prt2_l_p2_p1 PARTITION OF prt2_l_p2 FOR VALUES FROM ('0250') TO ('0400');
+CREATE TABLE prt2_l_p2_p2 PARTITION OF prt2_l_p2 FOR VALUES FROM ('0400') TO ('0500');
+CREATE TABLE prt2_l_p3 PARTITION OF prt2_l FOR VALUES FROM (500) TO (600) PARTITION BY RANGE ((a + b));
+CREATE TABLE prt2_l_p3_p1 PARTITION OF prt2_l_p3 FOR VALUES FROM (1000) TO (1100);
+CREATE TABLE prt2_l_p3_p2 PARTITION OF prt2_l_p3 FOR VALUES FROM (1100) TO (1200);
+INSERT INTO prt2_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_l;
+ANALYZE prt2_l_p1;
+ANALYZE prt2_l_p1_p1;
+ANALYZE prt2_l_p1_p2;
+ANALYZE prt2_l_p2;
+ANALYZE prt2_l_p2_p1;
+ANALYZE prt2_l_p2_p2;
+ANALYZE prt2_l_p3;
+ANALYZE prt2_l_p3_p1;
+ANALYZE prt2_l_p3_p2;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_l AS SELECT * FROM prt2_l;
+EXPLAIN (VERBOSE, 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 = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                                                      QUERY PLAN                                                                      
+------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((t2.b = t1.a) AND (t2.a = t1.b) AND ((t2.c)::text = (t1.c)::text) AND ((t2.a + t2.b) = (t1.b + t1.a)))
+               ->  Seq Scan on public.prt2_l_p1_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_l_p1_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.a = t1_1.b) AND ((t2_1.c)::text = (t1_1.c)::text) AND ((t2_1.a + t2_1.b) = (t1_1.b + t1_1.a)))
+               ->  Seq Scan on public.prt2_l_p1_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_l_p1_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.a = t1_2.b) AND ((t2_2.c)::text = (t1_2.c)::text) AND ((t2_2.a + t2_2.b) = (t1_2.b + t1_2.a)))
+               ->  Seq Scan on public.prt2_l_p2_p1 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_l_p2_p1 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_3.a, t1_3.c, t2_3.b, t2_3.c
+               Hash Cond: ((t2_3.b = t1_3.a) AND (t2_3.a = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text) AND ((t2_3.a + t2_3.b) = (t1_3.b + t1_3.a)))
+               ->  Seq Scan on public.prt2_l_p2_p2 t2_3
+                     Output: t2_3.b, t2_3.c, t2_3.a
+               ->  Hash
+                     Output: t1_3.a, t1_3.c, t1_3.b
+                     ->  Seq Scan on public.prt1_l_p2_p2 t1_3
+                           Output: t1_3.a, t1_3.c, t1_3.b
+                           Filter: ((t1_3.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_4.a, t1_4.c, t2_4.b, t2_4.c
+               Hash Cond: ((t2_4.b = t1_4.a) AND (t2_4.a = t1_4.b) AND ((t2_4.c)::text = (t1_4.c)::text) AND ((t2_4.a + t2_4.b) = (t1_4.b + t1_4.a)))
+               ->  Seq Scan on public.prt2_l_p3_p1 t2_4
+                     Output: t2_4.b, t2_4.c, t2_4.a
+               ->  Hash
+                     Output: t1_4.a, t1_4.c, t1_4.b
+                     ->  Seq Scan on public.prt1_l_p3_p1 t1_4
+                           Output: t1_4.a, t1_4.c, t1_4.b
+                           Filter: ((t1_4.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_5.a, t1_5.c, t2_5.b, t2_5.c
+               Hash Cond: ((t2_5.b = t1_5.a) AND (t2_5.a = t1_5.b) AND ((t2_5.c)::text = (t1_5.c)::text) AND ((t2_5.a + t2_5.b) = (t1_5.b + t1_5.a)))
+               ->  Seq Scan on public.prt2_l_p3_p2 t2_5
+                     Output: t2_5.b, t2_5.c, t2_5.a
+               ->  Hash
+                     Output: t1_5.a, t1_5.c, t1_5.b
+                     ->  Seq Scan on public.prt1_l_p3_p2 t1_5
+                           Output: t1_5.a, t1_5.c, t1_5.b
+                           Filter: ((t1_5.a % 25) = 0)
+(64 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1, prt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1, uprt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 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)
+
+EXPLAIN (VERBOSE, 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                                                      QUERY PLAN                                                                      
+------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((t2.b = t1.a) AND (t2.a = t1.b) AND ((t2.c)::text = (t1.c)::text) AND ((t2.a + t2.b) = (t1.b + t1.a)))
+               ->  Seq Scan on public.prt2_l_p1_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_l_p1_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.a = t1_1.b) AND ((t2_1.c)::text = (t1_1.c)::text) AND ((t2_1.a + t2_1.b) = (t1_1.b + t1_1.a)))
+               ->  Seq Scan on public.prt2_l_p1_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_l_p1_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.a = t1_2.b) AND ((t2_2.c)::text = (t1_2.c)::text) AND ((t2_2.a + t2_2.b) = (t1_2.b + t1_2.a)))
+               ->  Seq Scan on public.prt2_l_p2_p1 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_l_p2_p1 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_3.a, t1_3.c, t2_3.b, t2_3.c
+               Hash Cond: ((t2_3.b = t1_3.a) AND (t2_3.a = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text) AND ((t2_3.a + t2_3.b) = (t1_3.b + t1_3.a)))
+               ->  Seq Scan on public.prt2_l_p2_p2 t2_3
+                     Output: t2_3.b, t2_3.c, t2_3.a
+               ->  Hash
+                     Output: t1_3.a, t1_3.c, t1_3.b
+                     ->  Seq Scan on public.prt1_l_p2_p2 t1_3
+                           Output: t1_3.a, t1_3.c, t1_3.b
+                           Filter: ((t1_3.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_4.a, t1_4.c, t2_4.b, t2_4.c
+               Hash Cond: ((t2_4.b = t1_4.a) AND (t2_4.a = t1_4.b) AND ((t2_4.c)::text = (t1_4.c)::text) AND ((t2_4.a + t2_4.b) = (t1_4.b + t1_4.a)))
+               ->  Seq Scan on public.prt2_l_p3_p1 t2_4
+                     Output: t2_4.b, t2_4.c, t2_4.a
+               ->  Hash
+                     Output: t1_4.a, t1_4.c, t1_4.b
+                     ->  Seq Scan on public.prt1_l_p3_p1 t1_4
+                           Output: t1_4.a, t1_4.c, t1_4.b
+                           Filter: ((t1_4.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_5.a, t1_5.c, t2_5.b, t2_5.c
+               Hash Cond: ((t2_5.b = t1_5.a) AND (t2_5.a = t1_5.b) AND ((t2_5.c)::text = (t1_5.c)::text) AND ((t2_5.a + t2_5.b) = (t1_5.b + t1_5.a)))
+               ->  Seq Scan on public.prt2_l_p3_p2 t2_5
+                     Output: t2_5.b, t2_5.c, t2_5.a
+               ->  Hash
+                     Output: t1_5.a, t1_5.c, t1_5.b
+                     ->  Seq Scan on public.prt1_l_p3_p2 t1_5
+                           Output: t1_5.a, t1_5.c, t1_5.b
+                           Filter: ((t1_5.a % 25) = 0)
+(64 rows)
+
+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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 LEFT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+                                                                         QUERY PLAN                                                                         
+------------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: ((t1.a = t2.b) AND (t1.b = t2.a) AND ((t1.c)::text = (t2.c)::text) AND ((t1.b + t1.a) = (t2.a + t2.b)))
+                     ->  Seq Scan on public.prt1_l_p1_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                     ->  Hash
+                           Output: t2.b, t2.c, t2.a
+                           ->  Seq Scan on public.prt2_l_p1_p1 t2
+                                 Output: t2.b, t2.c, t2.a
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: ((t1_1.a = t2_1.b) AND (t1_1.b = t2_1.a) AND ((t1_1.c)::text = (t2_1.c)::text) AND ((t1_1.b + t1_1.a) = (t2_1.a + t2_1.b)))
+                     ->  Seq Scan on public.prt1_l_p1_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c, t2_1.a
+                           ->  Seq Scan on public.prt2_l_p1_p2 t2_1
+                                 Output: t2_1.b, t2_1.c, t2_1.a
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: ((t1_2.a = t2_2.b) AND (t1_2.b = t2_2.a) AND ((t1_2.c)::text = (t2_2.c)::text) AND ((t1_2.b + t1_2.a) = (t2_2.a + t2_2.b)))
+                     ->  Seq Scan on public.prt1_l_p2_p1 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c, t2_2.a
+                           ->  Seq Scan on public.prt2_l_p2_p1 t2_2
+                                 Output: t2_2.b, t2_2.c, t2_2.a
+                                 Filter: ((t2_2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_3.b, t2_3.c, t1_3.a, t1_3.c
+                     Hash Cond: ((t1_3.a = t2_3.b) AND (t1_3.b = t2_3.a) AND ((t1_3.c)::text = (t2_3.c)::text) AND ((t1_3.b + t1_3.a) = (t2_3.a + t2_3.b)))
+                     ->  Seq Scan on public.prt1_l_p2_p2 t1_3
+                           Output: t1_3.a, t1_3.c, t1_3.b
+                     ->  Hash
+                           Output: t2_3.b, t2_3.c, t2_3.a
+                           ->  Seq Scan on public.prt2_l_p2_p2 t2_3
+                                 Output: t2_3.b, t2_3.c, t2_3.a
+                                 Filter: ((t2_3.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_4.b, t2_4.c, t1_4.a, t1_4.c
+                     Hash Cond: ((t1_4.a = t2_4.b) AND (t1_4.b = t2_4.a) AND ((t1_4.c)::text = (t2_4.c)::text) AND ((t1_4.b + t1_4.a) = (t2_4.a + t2_4.b)))
+                     ->  Seq Scan on public.prt1_l_p3_p1 t1_4
+                           Output: t1_4.a, t1_4.c, t1_4.b
+                     ->  Hash
+                           Output: t2_4.b, t2_4.c, t2_4.a
+                           ->  Seq Scan on public.prt2_l_p3_p1 t2_4
+                                 Output: t2_4.b, t2_4.c, t2_4.a
+                                 Filter: ((t2_4.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_5.b, t2_5.c, t1_5.a, t1_5.c
+                     Hash Cond: ((t1_5.a = t2_5.b) AND (t1_5.b = t2_5.a) AND ((t1_5.c)::text = (t2_5.c)::text) AND ((t1_5.b + t1_5.a) = (t2_5.a + t2_5.b)))
+                     ->  Seq Scan on public.prt1_l_p3_p2 t1_5
+                           Output: t1_5.a, t1_5.c, t1_5.b
+                     ->  Hash
+                           Output: t2_5.b, t2_5.c, t2_5.a
+                           ->  Seq Scan on public.prt2_l_p3_p2 t2_5
+                                 Output: t2_5.b, t2_5.c, t2_5.a
+                                 Filter: ((t2_5.b % 25) = 0)
+(66 rows)
+
+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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 RIGHT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+                                                                                                              QUERY PLAN                                                                                                              
+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_l_p1_p1.a, prt1_l_p1_p1.c, prt2_l_p1_p1.b, prt2_l_p1_p1.c
+   Sort Key: prt1_l_p1_p1.a, prt2_l_p1_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_l_p1_p1.a, prt1_l_p1_p1.c, prt2_l_p1_p1.b, prt2_l_p1_p1.c
+               Hash Cond: ((prt1_l_p1_p1.a = prt2_l_p1_p1.b) AND (prt1_l_p1_p1.b = prt2_l_p1_p1.a) AND ((prt1_l_p1_p1.c)::text = (prt2_l_p1_p1.c)::text) AND ((prt1_l_p1_p1.b + prt1_l_p1_p1.a) = (prt2_l_p1_p1.a + prt2_l_p1_p1.b)))
+               ->  Seq Scan on public.prt1_l_p1_p1
+                     Output: prt1_l_p1_p1.a, prt1_l_p1_p1.c, prt1_l_p1_p1.b
+                     Filter: ((prt1_l_p1_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p1_p1.b, prt2_l_p1_p1.c, prt2_l_p1_p1.a
+                     ->  Seq Scan on public.prt2_l_p1_p1
+                           Output: prt2_l_p1_p1.b, prt2_l_p1_p1.c, prt2_l_p1_p1.a
+                           Filter: ((prt2_l_p1_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_l_p1_p2.a, prt1_l_p1_p2.c, prt2_l_p1_p2.b, prt2_l_p1_p2.c
+               Hash Cond: ((prt1_l_p1_p2.a = prt2_l_p1_p2.b) AND (prt1_l_p1_p2.b = prt2_l_p1_p2.a) AND ((prt1_l_p1_p2.c)::text = (prt2_l_p1_p2.c)::text) AND ((prt1_l_p1_p2.b + prt1_l_p1_p2.a) = (prt2_l_p1_p2.a + prt2_l_p1_p2.b)))
+               ->  Seq Scan on public.prt1_l_p1_p2
+                     Output: prt1_l_p1_p2.a, prt1_l_p1_p2.c, prt1_l_p1_p2.b
+                     Filter: ((prt1_l_p1_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p1_p2.b, prt2_l_p1_p2.c, prt2_l_p1_p2.a
+                     ->  Seq Scan on public.prt2_l_p1_p2
+                           Output: prt2_l_p1_p2.b, prt2_l_p1_p2.c, prt2_l_p1_p2.a
+                           Filter: ((prt2_l_p1_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_l_p2_p1.a, prt1_l_p2_p1.c, prt2_l_p2_p1.b, prt2_l_p2_p1.c
+               Hash Cond: ((prt1_l_p2_p1.a = prt2_l_p2_p1.b) AND (prt1_l_p2_p1.b = prt2_l_p2_p1.a) AND ((prt1_l_p2_p1.c)::text = (prt2_l_p2_p1.c)::text) AND ((prt1_l_p2_p1.b + prt1_l_p2_p1.a) = (prt2_l_p2_p1.a + prt2_l_p2_p1.b)))
+               ->  Seq Scan on public.prt1_l_p2_p1
+                     Output: prt1_l_p2_p1.a, prt1_l_p2_p1.c, prt1_l_p2_p1.b
+                     Filter: ((prt1_l_p2_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p2_p1.b, prt2_l_p2_p1.c, prt2_l_p2_p1.a
+                     ->  Seq Scan on public.prt2_l_p2_p1
+                           Output: prt2_l_p2_p1.b, prt2_l_p2_p1.c, prt2_l_p2_p1.a
+                           Filter: ((prt2_l_p2_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_l_p2_p2.a, prt1_l_p2_p2.c, prt2_l_p2_p2.b, prt2_l_p2_p2.c
+               Hash Cond: ((prt1_l_p2_p2.a = prt2_l_p2_p2.b) AND (prt1_l_p2_p2.b = prt2_l_p2_p2.a) AND ((prt1_l_p2_p2.c)::text = (prt2_l_p2_p2.c)::text) AND ((prt1_l_p2_p2.b + prt1_l_p2_p2.a) = (prt2_l_p2_p2.a + prt2_l_p2_p2.b)))
+               ->  Seq Scan on public.prt1_l_p2_p2
+                     Output: prt1_l_p2_p2.a, prt1_l_p2_p2.c, prt1_l_p2_p2.b
+                     Filter: ((prt1_l_p2_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p2_p2.b, prt2_l_p2_p2.c, prt2_l_p2_p2.a
+                     ->  Seq Scan on public.prt2_l_p2_p2
+                           Output: prt2_l_p2_p2.b, prt2_l_p2_p2.c, prt2_l_p2_p2.a
+                           Filter: ((prt2_l_p2_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_l_p3_p1.a, prt1_l_p3_p1.c, prt2_l_p3_p1.b, prt2_l_p3_p1.c
+               Hash Cond: ((prt1_l_p3_p1.a = prt2_l_p3_p1.b) AND (prt1_l_p3_p1.b = prt2_l_p3_p1.a) AND ((prt1_l_p3_p1.c)::text = (prt2_l_p3_p1.c)::text) AND ((prt1_l_p3_p1.b + prt1_l_p3_p1.a) = (prt2_l_p3_p1.a + prt2_l_p3_p1.b)))
+               ->  Seq Scan on public.prt1_l_p3_p1
+                     Output: prt1_l_p3_p1.a, prt1_l_p3_p1.c, prt1_l_p3_p1.b
+                     Filter: ((prt1_l_p3_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p3_p1.b, prt2_l_p3_p1.c, prt2_l_p3_p1.a
+                     ->  Seq Scan on public.prt2_l_p3_p1
+                           Output: prt2_l_p3_p1.b, prt2_l_p3_p1.c, prt2_l_p3_p1.a
+                           Filter: ((prt2_l_p3_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_l_p3_p2.a, prt1_l_p3_p2.c, prt2_l_p3_p2.b, prt2_l_p3_p2.c
+               Hash Cond: ((prt1_l_p3_p2.a = prt2_l_p3_p2.b) AND (prt1_l_p3_p2.b = prt2_l_p3_p2.a) AND ((prt1_l_p3_p2.c)::text = (prt2_l_p3_p2.c)::text) AND ((prt1_l_p3_p2.b + prt1_l_p3_p2.a) = (prt2_l_p3_p2.a + prt2_l_p3_p2.b)))
+               ->  Seq Scan on public.prt1_l_p3_p2
+                     Output: prt1_l_p3_p2.a, prt1_l_p3_p2.c, prt1_l_p3_p2.b
+                     Filter: ((prt1_l_p3_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p3_p2.b, prt2_l_p3_p2.c, prt2_l_p3_p2.a
+                     ->  Seq Scan on public.prt2_l_p3_p2
+                           Output: prt2_l_p3_p2.b, prt2_l_p3_p2.c, prt2_l_p3_p2.a
+                           Filter: ((prt2_l_p3_p2.b % 25) = 0)
+(70 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_l t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_l t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.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)
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
new file mode 100644
index 0000000..79779d6
--- /dev/null
+++ b/src/test/regress/expected/partition_join.out
@@ -0,0 +1,4118 @@
+--
+-- PARTITION_JOIN
+-- Test partition-wise join between partitioned tables
+--
+-- Usually partition-wise join paths are chosen when data is large, which would
+-- take regression tests to run longer. So, weigh partition-wise joins cheaper
+-- to force those even for smaller data.
+SET partition_wise_plan_weight to 0.2;
+--
+-- partitioned by a single column
+--
+CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+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, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1 AS SELECT * FROM prt1;
+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_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, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+CREATE TABLE uprt2 AS SELECT * FROM prt2;
+-- inner join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on public.prt2_p1 t2
+                     Output: t2.b, t2.c
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+               Hash Cond: (t2_1.b = t1_2.a)
+               ->  Seq Scan on public.prt2_p2 t2_1
+                     Output: t2_1.b, t2_1.c
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+               Hash Cond: (t2_2.b = t1_1.a)
+               ->  Seq Scan on public.prt2_p3 t2_2
+                     Output: t2_2.b, t2_2.c
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 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)
+
+-- left outer join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on public.prt2_p1 t2
+                     Output: t2.b, t2.c
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+               Hash Cond: (t2_1.b = t1_2.a)
+               ->  Seq Scan on public.prt2_p2 t2_1
+                     Output: t2_1.b, t2_1.c
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+               Hash Cond: (t2_2.b = t1_1.a)
+               ->  Seq Scan on public.prt2_p3 t2_2
+                     Output: t2_2.b, t2_2.c
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+-- right outer join
+EXPLAIN (VERBOSE, 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.b % 25 = 0 ORDER BY t1.a, t2.b;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: (t1.a = t2.b)
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t2.b, t2.c
+                           ->  Seq Scan on public.prt2_p1 t2
+                                 Output: t2.b, t2.c
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Hash Cond: (t1_2.a = t2_1.b)
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c
+                           ->  Seq Scan on public.prt2_p2 t2_1
+                                 Output: t2_1.b, t2_1.c
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Hash Cond: (t1_1.a = t2_2.b)
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c
+                           ->  Seq Scan on public.prt2_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+                                 Filter: ((t2_2.b % 25) = 0)
+(36 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 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)
+
+-- full outer join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+   Sort Key: prt1_p1.a, prt2_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+               Hash Cond: (prt1_p1.a = prt2_p1.b)
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c
+                     Filter: ((prt1_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p1.b, prt2_p1.c
+                     ->  Seq Scan on public.prt2_p1
+                           Output: prt2_p1.b, prt2_p1.c
+                           Filter: ((prt2_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt1_p2.a = prt2_p2.b)
+               ->  Seq Scan on public.prt1_p2
+                     Output: prt1_p2.a, prt1_p2.c
+                     Filter: ((prt1_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p2.b, prt2_p2.c
+                     ->  Seq Scan on public.prt2_p2
+                           Output: prt2_p2.b, prt2_p2.c
+                           Filter: ((prt2_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c
+               Hash Cond: (prt1_p3.a = prt2_p3.b)
+               ->  Seq Scan on public.prt1_p3
+                     Output: prt1_p3.a, prt1_p3.c
+                     Filter: ((prt1_p3.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p3.b, prt2_p3.c
+                     ->  Seq Scan on public.prt2_p3
+                           Output: prt2_p3.b, prt2_p3.c
+                           Filter: ((prt2_p3.b % 25) = 0)
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 t2 WHERE t2.b % 25 = 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)
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+   Sort Key: prt1_p1.a, prt2_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+               Hash Cond: (prt1_p1.a = prt2_p1.b)
+               Filter: (((50) = prt1_p1.b) OR ((75) = prt2_p1.b))
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c, prt1_p1.b, 50
+                     Filter: ((prt1_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p1.b, prt2_p1.c, (75)
+                     ->  Seq Scan on public.prt2_p1
+                           Output: prt2_p1.b, prt2_p1.c, 75
+                           Filter: ((prt2_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt1_p2.a = prt2_p2.b)
+               Filter: (((50) = prt1_p2.b) OR ((75) = prt2_p2.b))
+               ->  Seq Scan on public.prt1_p2
+                     Output: prt1_p2.a, prt1_p2.c, prt1_p2.b, 50
+                     Filter: ((prt1_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p2.b, prt2_p2.c, (75)
+                     ->  Seq Scan on public.prt2_p2
+                           Output: prt2_p2.b, prt2_p2.c, 75
+                           Filter: ((prt2_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c
+               Hash Cond: (prt1_p3.a = prt2_p3.b)
+               Filter: (((50) = prt1_p3.b) OR ((75) = prt2_p3.b))
+               ->  Seq Scan on public.prt1_p3
+                     Output: prt1_p3.a, prt1_p3.c, prt1_p3.b, 50
+                     Filter: ((prt1_p3.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p3.b, prt2_p3.c, (75)
+                     ->  Seq Scan on public.prt2_p3
+                           Output: prt2_p3.b, prt2_p3.c, 75
+                           Filter: ((prt2_p3.b % 25) = 0)
+(40 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+ a  |  c   | b  |  c   
+----+------+----+------
+ 50 | 0050 |    | 
+    |      | 75 | 0075
+(2 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+ a  |  c   | b  |  c   
+----+------+----+------
+ 50 | 0050 |    | 
+    |      | 75 | 0075
+(2 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, (25), prt2_p1.b, prt2_p1.c, (50)
+   Sort Key: prt1_p1.a, prt2_p1.b
+   ->  Result
+         Output: prt1_p1.a, prt1_p1.c, (25), prt2_p1.b, prt2_p1.c, (50)
+         ->  Append
+               ->  Hash Full Join
+                     Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, (25), (50)
+                     Hash Cond: (prt1_p1.a = prt2_p1.b)
+                     ->  Seq Scan on public.prt1_p1
+                           Output: prt1_p1.a, prt1_p1.c, 25
+                           Filter: ((prt1_p1.a % 25) = 0)
+                     ->  Hash
+                           Output: prt2_p1.b, prt2_p1.c, (50)
+                           ->  Seq Scan on public.prt2_p1
+                                 Output: prt2_p1.b, prt2_p1.c, 50
+                                 Filter: ((prt2_p1.b % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c, (25), (50)
+                     Hash Cond: (prt1_p2.a = prt2_p2.b)
+                     ->  Seq Scan on public.prt1_p2
+                           Output: prt1_p2.a, prt1_p2.c, 25
+                           Filter: ((prt1_p2.a % 25) = 0)
+                     ->  Hash
+                           Output: prt2_p2.b, prt2_p2.c, (50)
+                           ->  Seq Scan on public.prt2_p2
+                                 Output: prt2_p2.b, prt2_p2.c, 50
+                                 Filter: ((prt2_p2.b % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c, (25), (50)
+                     Hash Cond: (prt1_p3.a = prt2_p3.b)
+                     ->  Seq Scan on public.prt1_p3
+                           Output: prt1_p3.a, prt1_p3.c, 25
+                           Filter: ((prt1_p3.a % 25) = 0)
+                     ->  Hash
+                           Output: prt2_p3.b, prt2_p3.c, (50)
+                           ->  Seq Scan on public.prt2_p3
+                                 Output: prt2_p3.b, prt2_p3.c, 50
+                                 Filter: ((prt2_p3.b % 25) = 0)
+(39 rows)
+
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 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)
+
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM uprt2 WHERE uprt2.b % 25 = 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)
+
+-- Join with pruned partitions from joining relations
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on public.prt2_p2 t2
+                     Output: t2.b, t2.c
+                     Filter: (t2.b > 250)
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.prt1_p2 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a < 450) AND ((t1.a % 25) = 0))
+(15 rows)
+
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+(1 row)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a < 450 AND t2.b > 250 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+(1 row)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                    QUERY PLAN                                    
+----------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, b, c
+   Sort Key: prt1_p1.a, b
+   ->  Append
+         ->  Nested Loop Left Join
+               Output: prt1_p1.a, prt1_p1.c, b, c
+               Join Filter: (prt1_p1.a = b)
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c
+                     Filter: ((prt1_p1.a < 450) AND ((prt1_p1.a % 25) = 0))
+               ->  Result
+                     Output: b, c
+                     One-Time Filter: false
+         ->  Hash Right Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt2_p2.b = prt1_p2.a)
+               ->  Seq Scan on public.prt2_p2
+                     Output: prt2_p2.b, prt2_p2.c
+                     Filter: (prt2_p2.b > 250)
+               ->  Hash
+                     Output: prt1_p2.a, prt1_p2.c
+                     ->  Seq Scan on public.prt1_p2
+                           Output: prt1_p2.a, prt1_p2.c
+                           Filter: ((prt1_p2.a < 450) AND ((prt1_p2.a % 25) = 0))
+(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.a % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.a % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                       QUERY PLAN                                       
+----------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+   Sort Key: prt1_p2.a, prt2_p2.b
+   ->  Result
+         Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: prt2_p2.b, prt2_p2.c, prt1_p2.a, prt1_p2.c
+                     Hash Cond: (prt1_p2.a = prt2_p2.b)
+                     ->  Seq Scan on public.prt1_p2
+                           Output: prt1_p2.a, prt1_p2.c
+                           Filter: (prt1_p2.a < 450)
+                     ->  Hash
+                           Output: prt2_p2.b, prt2_p2.c
+                           ->  Seq Scan on public.prt2_p2
+                                 Output: prt2_p2.b, prt2_p2.c
+                                 Filter: ((prt2_p2.b > 250) AND ((prt2_p2.a % 25) = 0))
+               ->  Nested Loop Left Join
+                     Output: prt2_p3.b, prt2_p3.c, a, c
+                     Join Filter: (a = prt2_p3.b)
+                     ->  Seq Scan on public.prt2_p3
+                           Output: prt2_p3.b, prt2_p3.c
+                           Filter: ((prt2_p3.b > 250) AND ((prt2_p3.a % 25) = 0))
+                     ->  Result
+                           Output: a, c
+                           One-Time Filter: false
+(26 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+     |      | 375 | 0375
+     |      | 450 | 0450
+     |      | 525 | 0525
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+     |      | 375 | 0375
+     |      | 450 | 0450
+     |      | 525 | 0525
+(4 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+                                    QUERY PLAN                                    
+----------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, b, c
+   Sort Key: prt1_p1.a, b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_p1.a, prt1_p1.c, b, c
+               Hash Cond: (prt1_p1.a = b)
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c
+                     Filter: ((prt1_p1.a < 450) AND ((prt1_p1.a % 25) = 0))
+               ->  Hash
+                     Output: b, c
+                     ->  Result
+                           Output: b, c
+                           One-Time Filter: false
+         ->  Hash Full Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt1_p2.a = prt2_p2.b)
+               ->  Seq Scan on public.prt1_p2
+                     Output: prt1_p2.a, prt1_p2.c
+                     Filter: ((prt1_p2.a < 450) AND ((prt1_p2.a % 25) = 0))
+               ->  Hash
+                     Output: prt2_p2.b, prt2_p2.c
+                     ->  Seq Scan on public.prt2_p2
+                           Output: prt2_p2.b, prt2_p2.c
+                           Filter: ((prt2_p2.b > 250) AND ((prt2_p2.b % 25) = 0))
+         ->  Hash Full Join
+               Output: a, c, prt2_p3.b, prt2_p3.c
+               Hash Cond: (prt2_p3.b = a)
+               ->  Seq Scan on public.prt2_p3
+                     Output: prt2_p3.b, prt2_p3.c
+                     Filter: ((prt2_p3.b > 250) AND ((prt2_p3.b % 25) = 0))
+               ->  Hash
+                     Output: a, c
+                     ->  Result
+                           Output: a, c
+                           One-Time Filter: false
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b 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)
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+                     QUERY PLAN                     
+----------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Output: t1.a, t1.b, t1.c
+         Merge Cond: (t1.a = t1_3.b)
+         ->  Sort
+               Output: t1.a, t1.b, t1.c
+               Sort Key: t1.a
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+         ->  Sort
+               Output: t1_3.b
+               Sort Key: t1_3.b
+               ->  Seq Scan on public.prt2_p1 t1_3
+                     Output: t1_3.b
+                     Filter: ((t1_3.b % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_2.a, t1_2.b, t1_2.c
+         Merge Cond: (t1_2.a = t1_4.b)
+         ->  Sort
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Sort Key: t1_2.a
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+         ->  Sort
+               Output: t1_4.b
+               Sort Key: t1_4.b
+               ->  Seq Scan on public.prt2_p2 t1_4
+                     Output: t1_4.b
+                     Filter: ((t1_4.b % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_1.a, t1_1.b, t1_1.c
+         Merge Cond: (t1_1.a = t1_5.b)
+         ->  Sort
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Sort Key: t1_1.a
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+         ->  Sort
+               Output: t1_5.b
+               Sort Key: t1_5.b
+               ->  Seq Scan on public.prt2_p3 t1_5
+                     Output: t1_5.b
+                     Filter: ((t1_5.b % 25) = 0)
+(47 rows)
+
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+-- lateral reference
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+                                             QUERY PLAN                                              
+-----------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   Sort Key: t1.a
+   ->  Result
+         Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+         ->  Append
+               ->  Nested Loop Left Join
+                     Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a))
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.b, t1.c
+                           Filter: ((t1.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+                           Hash Cond: (t3.b = t2.a)
+                           ->  Seq Scan on public.prt2_p1 t3
+                                 Output: t3.a, t3.b
+                           ->  Hash
+                                 Output: t2.a
+                                 ->  Seq Scan on public.prt1_p1 t2
+                                       Output: t2.a
+                                       Filter: (t1.a = t2.a)
+               ->  Nested Loop Left Join
+                     Output: t1_2.a, t1_2.b, t1_2.c, t2_2.a, t3_1.a, (LEAST(t1_2.a, t2_2.a, t3_1.a))
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.b, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2_2.a, t3_1.a, LEAST(t1_2.a, t2_2.a, t3_1.a)
+                           Hash Cond: (t3_1.b = t2_2.a)
+                           ->  Seq Scan on public.prt2_p2 t3_1
+                                 Output: t3_1.a, t3_1.b
+                           ->  Hash
+                                 Output: t2_2.a
+                                 ->  Seq Scan on public.prt1_p2 t2_2
+                                       Output: t2_2.a
+                                       Filter: (t1_2.a = t2_2.a)
+               ->  Nested Loop Left Join
+                     Output: t1_1.a, t1_1.b, t1_1.c, t2_1.a, t3_2.a, (LEAST(t1_1.a, t2_1.a, t3_2.a))
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.b, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2_1.a, t3_2.a, LEAST(t1_1.a, t2_1.a, t3_2.a)
+                           Hash Cond: (t3_2.b = t2_1.a)
+                           ->  Seq Scan on public.prt2_p3 t3_2
+                                 Output: t3_2.a, t3_2.b
+                           ->  Hash
+                                 Output: t2_1.a
+                                 ->  Seq Scan on public.prt1_p3 t2_1
+                                       Output: t2_1.a
+                                       Filter: (t1_1.a = t2_1.a)
+(51 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   Sort Key: t1.a
+   ->  Nested Loop Left Join
+         Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+         ->  Append
+               ->  Seq Scan on public.prt1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Seq Scan on public.prt1_p1 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Seq Scan on public.prt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Seq Scan on public.prt1_p2 t1_3
+                     Output: t1_3.a, t1_3.b, t1_3.c
+                     Filter: ((t1_3.a % 25) = 0)
+         ->  Append
+               ->  Hash Join
+                     Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+                     Hash Cond: (t3.b = t2.a)
+                     ->  Seq Scan on public.prt2_p1 t3
+                           Output: t3.a, t3.b
+                     ->  Hash
+                           Output: t2.a
+                           ->  Seq Scan on public.prt1_p1 t2
+                                 Output: t2.a
+                                 Filter: (t1.b = t2.a)
+               ->  Hash Join
+                     Output: t2_2.a, t3_1.a, LEAST(t1.a, t2_2.a, t3_1.a)
+                     Hash Cond: (t3_1.b = t2_2.a)
+                     ->  Seq Scan on public.prt2_p2 t3_1
+                           Output: t3_1.a, t3_1.b
+                     ->  Hash
+                           Output: t2_2.a
+                           ->  Seq Scan on public.prt1_p2 t2_2
+                                 Output: t2_2.a
+                                 Filter: (t1.b = t2_2.a)
+               ->  Hash Join
+                     Output: t2_1.a, t3_2.a, LEAST(t1.a, t2_1.a, t3_2.a)
+                     Hash Cond: (t3_2.b = t2_1.a)
+                     ->  Seq Scan on public.prt2_p3 t3_2
+                           Output: t3_2.a, t3_2.b
+                     ->  Hash
+                           Output: t2_1.a
+                           ->  Seq Scan on public.prt1_p3 t2_1
+                                 Output: t2_1.a
+                                 Filter: (t1.b = t2_1.a)
+(49 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+--
+-- partitioned by expression
+--
+CREATE TABLE prt1_e (a int, b int, c varchar) PARTITION BY RANGE(((a + b)/2));
+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);
+INSERT INTO prt1_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_e AS SELECT * FROM prt1_e;
+CREATE TABLE prt2_e (a int, b int, c varchar) PARTITION BY RANGE(((b + a)/2));
+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);
+INSERT INTO prt2_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_e;
+ANALYZE prt2_e_p1;
+ANALYZE prt2_e_p2;
+ANALYZE prt2_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_e AS SELECT * FROM prt2_e;
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                  QUERY PLAN                                  
+------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_e_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_e_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
+               ->  Seq Scan on public.prt2_e_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_e_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+(34 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.a % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1, uprt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.a % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                  QUERY PLAN                                  
+------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_e_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_e_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
+               ->  Seq Scan on public.prt2_e_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_e_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1 LEFT JOIN uprt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on public.prt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = t1.a)
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_2.a)
+                     ->  Seq Scan on public.prt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                           ->  Hash Join
+                                 Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = t1_2.a)
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_1.a)
+                     ->  Seq Scan on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                           ->  Hash Join
+                                 Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = t1_1.a)
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+(57 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM uprt1 t1, uprt2 t2, uprt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on public.prt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = t1.a)
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_2.a)
+                     ->  Seq Scan on public.prt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = t1_2.a)
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_1.a)
+                     ->  Seq Scan on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = t1_1.a)
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+(57 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, 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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: (((t3.a + t3.b) / 2) = t2.b)
+                     ->  Seq Scan on public.prt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = t1.a)
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: (((t3_1.a + t3_1.b) / 2) = t2_1.b)
+                     ->  Seq Scan on public.prt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = t1_2.a)
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: (((t3_2.a + t3_2.b) / 2) = t2_2.b)
+                     ->  Seq Scan on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = t1_1.a)
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+(57 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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |          | 
+ 100 | 0100 |     |      |          | 
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |          | 
+ 250 | 0250 |     |      |          | 
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |          | 
+ 400 | 0400 |     |      |          | 
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |          | 
+ 550 | 0550 |     |      |          | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |          | 
+ 100 | 0100 |     |      |          | 
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |          | 
+ 250 | 0250 |     |      |          | 
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |          | 
+ 400 | 0400 |     |      |          | 
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |          | 
+ 550 | 0550 |     |      |          | 
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Hash Cond: (t2.b = t1.a)
+                     ->  Seq Scan on public.prt2_p1 t2
+                           Output: t2.b, t2.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Hash Cond: (t1.a = ((t3.a + t3.b) / 2))
+                                 ->  Seq Scan on public.prt1_p1 t1
+                                       Output: t1.a, t1.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                     Hash Cond: (t2_1.b = t1_2.a)
+                     ->  Seq Scan on public.prt2_p2 t2_1
+                           Output: t2_1.b, t2_1.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                                 Hash Cond: (t1_2.a = ((t3_1.a + t3_1.b) / 2))
+                                 ->  Seq Scan on public.prt1_p2 t1_2
+                                       Output: t1_2.a, t1_2.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                     Hash Cond: (t2_2.b = t1_1.a)
+                     ->  Seq Scan on public.prt2_p3 t2_2
+                           Output: t2_2.b, t2_2.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                                 Hash Cond: (t1_1.a = ((t3_2.a + t3_2.b) / 2))
+                                 ->  Seq Scan on public.prt1_p3 t1_1
+                                       Output: t1_1.a, t1_1.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: (t1.a = t2.b)
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = ((t3.a + t3.b) / 2))
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Hash Cond: (t1_2.a = t2_1.b)
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = ((t3_1.a + t3_1.b) / 2))
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Hash Cond: (t1_1.a = t2_2.b)
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = ((t3_2.a + t3_2.b) / 2))
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                                  QUERY PLAN                                                   
+---------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, ((prt1_e_p1.a + prt1_e_p1.b)), prt1_e_p1.c
+   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   ->  Result
+         Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, (prt1_e_p1.a + prt1_e_p1.b), prt1_e_p1.c
+         ->  Append
+               ->  Hash Full Join
+                     Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, prt1_e_p1.a, prt1_e_p1.b, prt1_e_p1.c
+                     Hash Cond: (prt1_p1.a = ((prt1_e_p1.a + prt1_e_p1.b) / 2))
+                     ->  Hash Full Join
+                           Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on public.prt1_p1
+                                 Output: prt1_p1.a, prt1_p1.c
+                                 Filter: ((prt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p1.b, prt2_p1.c
+                                 ->  Seq Scan on public.prt2_p1
+                                       Output: prt2_p1.b, prt2_p1.c
+                                       Filter: ((prt2_p1.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p1.a, prt1_e_p1.b, prt1_e_p1.c
+                           ->  Seq Scan on public.prt1_e_p1
+                                 Output: prt1_e_p1.a, prt1_e_p1.b, prt1_e_p1.c
+                                 Filter: ((prt1_e_p1.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c, prt1_e_p2.a, prt1_e_p2.b, prt1_e_p2.c
+                     Hash Cond: (prt1_p2.a = ((prt1_e_p2.a + prt1_e_p2.b) / 2))
+                     ->  Hash Full Join
+                           Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+                           Hash Cond: (prt1_p2.a = prt2_p2.b)
+                           ->  Seq Scan on public.prt1_p2
+                                 Output: prt1_p2.a, prt1_p2.c
+                                 Filter: ((prt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p2.b, prt2_p2.c
+                                 ->  Seq Scan on public.prt2_p2
+                                       Output: prt2_p2.b, prt2_p2.c
+                                       Filter: ((prt2_p2.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p2.a, prt1_e_p2.b, prt1_e_p2.c
+                           ->  Seq Scan on public.prt1_e_p2
+                                 Output: prt1_e_p2.a, prt1_e_p2.b, prt1_e_p2.c
+                                 Filter: ((prt1_e_p2.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c, prt1_e_p3.a, prt1_e_p3.b, prt1_e_p3.c
+                     Hash Cond: (prt1_p3.a = ((prt1_e_p3.a + prt1_e_p3.b) / 2))
+                     ->  Hash Full Join
+                           Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c
+                           Hash Cond: (prt1_p3.a = prt2_p3.b)
+                           ->  Seq Scan on public.prt1_p3
+                                 Output: prt1_p3.a, prt1_p3.c
+                                 Filter: ((prt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p3.b, prt2_p3.c
+                                 ->  Seq Scan on public.prt2_p3
+                                       Output: prt2_p3.b, prt2_p3.c
+                                       Filter: ((prt2_p3.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p3.a, prt1_e_p3.b, prt1_e_p3.c
+                           ->  Seq Scan on public.prt1_e_p3
+                                 Output: prt1_e_p3.a, prt1_e_p3.b, prt1_e_p3.c
+                                 Filter: ((prt1_e_p3.a % 25) = 0)
+(63 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+     |      |  75 | 0075 |          | 
+     |      | 225 | 0225 |          | 
+     |      | 375 | 0375 |          | 
+     |      | 525 | 0525 |          | 
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM uprt1_e WHERE uprt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+     |      |  75 | 0075 |          | 
+     |      | 225 | 0225 |          | 
+     |      | 375 | 0375 |          | 
+     |      | 525 | 0525 |          | 
+(16 rows)
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, 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.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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                                                      
+----------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, (50), prt2_p1.b, (75), ((prt1_e_p1.a + prt1_e_p1.b)), (50)
+   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   ->  Result
+         Output: prt1_p1.a, (50), prt2_p1.b, (75), (prt1_e_p1.a + prt1_e_p1.b), (50)
+         ->  Append
+               ->  Hash Full Join
+                     Output: prt1_p1.a, prt2_p1.b, prt1_e_p1.a, prt1_e_p1.b, (50), (75), (50)
+                     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
+                           Output: prt1_p1.a, prt2_p1.b, (50), (75)
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on public.prt1_p1
+                                 Output: prt1_p1.a, 50
+                                 Filter: ((prt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p1.b, (75)
+                                 ->  Seq Scan on public.prt2_p1
+                                       Output: prt2_p1.b, 75
+                                       Filter: ((prt2_p1.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p1.a, prt1_e_p1.b, (50)
+                           ->  Seq Scan on public.prt1_e_p1
+                                 Output: prt1_e_p1.a, prt1_e_p1.b, 50
+                                 Filter: ((prt1_e_p1.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p2.a, prt2_p2.b, prt1_e_p2.a, prt1_e_p2.b, (50), (75), (50)
+                     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
+                           Output: prt1_p2.a, prt2_p2.b, (50), (75)
+                           Hash Cond: (prt1_p2.a = prt2_p2.b)
+                           ->  Seq Scan on public.prt1_p2
+                                 Output: prt1_p2.a, 50
+                                 Filter: ((prt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p2.b, (75)
+                                 ->  Seq Scan on public.prt2_p2
+                                       Output: prt2_p2.b, 75
+                                       Filter: ((prt2_p2.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p2.a, prt1_e_p2.b, (50)
+                           ->  Seq Scan on public.prt1_e_p2
+                                 Output: prt1_e_p2.a, prt1_e_p2.b, 50
+                                 Filter: ((prt1_e_p2.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p3.a, prt2_p3.b, prt1_e_p3.a, prt1_e_p3.b, (50), (75), (50)
+                     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
+                           Output: prt1_p3.a, prt2_p3.b, (50), (75)
+                           Hash Cond: (prt1_p3.a = prt2_p3.b)
+                           ->  Seq Scan on public.prt1_p3
+                                 Output: prt1_p3.a, 50
+                                 Filter: ((prt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p3.b, (75)
+                                 ->  Seq Scan on public.prt2_p3
+                                       Output: prt2_p3.b, 75
+                                       Filter: ((prt2_p3.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p3.a, prt1_e_p3.b, (50)
+                           ->  Seq Scan on public.prt1_e_p3
+                                 Output: prt1_e_p3.a, prt1_e_p3.b, 50
+                                 Filter: ((prt1_e_p3.a % 25) = 0)
+(66 rows)
+
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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)
+
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM uprt1_e WHERE uprt1_e.a % 25 = 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)
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+                            QUERY PLAN                             
+-------------------------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Output: t1.a, t1.b, t1.c
+         Merge Cond: (t1.a = t1_3.b)
+         ->  Sort
+               Output: t1.a, t1.b, t1.c
+               Sort Key: t1.a
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+         ->  Sort
+               Output: t1_3.b, t2.a, t2.b
+               Sort Key: t1_3.b
+               ->  Hash Join
+                     Output: t1_3.b, t2.a, t2.b
+                     Hash Cond: (((t2.a + t2.b) / 2) = t1_3.b)
+                     ->  Seq Scan on public.prt1_e_p1 t2
+                           Output: t2.a, t2.b
+                     ->  Hash
+                           Output: t1_3.b
+                           ->  Seq Scan on public.prt2_p1 t1_3
+                                 Output: t1_3.b
+                                 Filter: ((t1_3.b % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_2.a, t1_2.b, t1_2.c
+         Merge Cond: (t1_2.a = t1_4.b)
+         ->  Sort
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Sort Key: t1_2.a
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+         ->  Sort
+               Output: t1_4.b, t2_1.a, t2_1.b
+               Sort Key: t1_4.b
+               ->  Hash Join
+                     Output: t1_4.b, t2_1.a, t2_1.b
+                     Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_4.b)
+                     ->  Seq Scan on public.prt1_e_p2 t2_1
+                           Output: t2_1.a, t2_1.b
+                     ->  Hash
+                           Output: t1_4.b
+                           ->  Seq Scan on public.prt2_p2 t1_4
+                                 Output: t1_4.b
+                                 Filter: ((t1_4.b % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_1.a, t1_1.b, t1_1.c
+         Merge Cond: (t1_1.a = t1_5.b)
+         ->  Sort
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Sort Key: t1_1.a
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+         ->  Sort
+               Output: t1_5.b, t2_2.a, t2_2.b
+               Sort Key: t1_5.b
+               ->  Hash Join
+                     Output: t1_5.b, t2_2.a, t2_2.b
+                     Hash Cond: (((t2_2.a + t2_2.b) / 2) = t1_5.b)
+                     ->  Seq Scan on public.prt1_e_p3 t2_2
+                           Output: t2_2.a, t2_2.b
+                     ->  Hash
+                           Output: t1_5.b
+                           ->  Seq Scan on public.prt2_p3 t1_5
+                                 Output: t1_5.b
+                                 Filter: ((t1_5.b % 25) = 0)
+(68 rows)
+
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1, uprt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+                            QUERY PLAN                             
+-------------------------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Output: t1.a, t1.b, t1.c
+         Merge Cond: (t1.a = t1_3.b)
+         ->  Sort
+               Output: t1.a, t1.b, t1.c
+               Sort Key: t1.a
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+         ->  Sort
+               Output: t1_3.b, t1_6.a, t1_6.b
+               Sort Key: t1_3.b
+               ->  Hash Semi Join
+                     Output: t1_3.b, t1_6.a, t1_6.b
+                     Hash Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
+                     ->  Seq Scan on public.prt2_p1 t1_3
+                           Output: t1_3.b
+                     ->  Hash
+                           Output: t1_6.a, t1_6.b
+                           ->  Seq Scan on public.prt1_e_p1 t1_6
+                                 Output: t1_6.a, t1_6.b
+                                 Filter: ((t1_6.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_2.a, t1_2.b, t1_2.c
+         Merge Cond: (t1_2.a = t1_4.b)
+         ->  Sort
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Sort Key: t1_2.a
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+         ->  Sort
+               Output: t1_4.b, t1_7.a, t1_7.b
+               Sort Key: t1_4.b
+               ->  Hash Semi Join
+                     Output: t1_4.b, t1_7.a, t1_7.b
+                     Hash Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
+                     ->  Seq Scan on public.prt2_p2 t1_4
+                           Output: t1_4.b
+                     ->  Hash
+                           Output: t1_7.a, t1_7.b
+                           ->  Seq Scan on public.prt1_e_p2 t1_7
+                                 Output: t1_7.a, t1_7.b
+                                 Filter: ((t1_7.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_1.a, t1_1.b, t1_1.c
+         Merge Cond: (t1_1.a = t1_5.b)
+         ->  Sort
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Sort Key: t1_1.a
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+         ->  Sort
+               Output: t1_5.b, t1_8.a, t1_8.b
+               Sort Key: t1_5.b
+               ->  Hash Semi Join
+                     Output: t1_5.b, t1_8.a, t1_8.b
+                     Hash Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
+                     ->  Seq Scan on public.prt2_p3 t1_5
+                           Output: t1_5.b
+                     ->  Hash
+                           Output: t1_8.a, t1_8.b
+                           ->  Seq Scan on public.prt1_e_p3 t1_8
+                                 Output: t1_8.a, t1_8.b
+                                 Filter: ((t1_8.a % 25) = 0)
+(68 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+-- test merge joins with and without using indexes
+SET enable_hashjoin TO off;
+SET enable_nestloop TO off;
+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 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 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));
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                             QUERY PLAN                                              
+-----------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Merge Cond: (t2.b = t1.a)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           Sort Key: t2.b
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Merge Cond: ((((t3.a + t3.b) / 2)) = t2.b)
+                                 ->  Sort
+                                       Output: t3.a, t3.b, t3.c, (((t3.a + t3.b) / 2))
+                                       Sort Key: (((t3.a + t3.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c, ((t3.a + t3.b) / 2)
+                                             Filter: ((t3.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2.b, t2.c
+                                       Sort Key: t2.b
+                                       ->  Seq Scan on public.prt2_p1 t2
+                                             Output: t2.b, t2.c
+                     ->  Sort
+                           Output: t1.a, t1.c
+                           Sort Key: t1.a
+                           ->  Seq Scan on public.prt1_p1 t1
+                                 Output: t1.a, t1.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Merge Cond: (t2_1.b = t1_2.a)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           Sort Key: t2_1.b
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t2_1.b)
+                                 ->  Sort
+                                       Output: t3_1.a, t3_1.b, t3_1.c, (((t3_1.a + t3_1.b) / 2))
+                                       Sort Key: (((t3_1.a + t3_1.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c, ((t3_1.a + t3_1.b) / 2)
+                                             Filter: ((t3_1.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2_1.b, t2_1.c
+                                       Sort Key: t2_1.b
+                                       ->  Seq Scan on public.prt2_p2 t2_1
+                                             Output: t2_1.b, t2_1.c
+                     ->  Sort
+                           Output: t1_2.a, t1_2.c
+                           Sort Key: t1_2.a
+                           ->  Seq Scan on public.prt1_p2 t1_2
+                                 Output: t1_2.a, t1_2.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Merge Cond: (t2_2.b = t1_1.a)
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           Sort Key: t2_2.b
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t2_2.b)
+                                 ->  Sort
+                                       Output: t3_2.a, t3_2.b, t3_2.c, (((t3_2.a + t3_2.b) / 2))
+                                       Sort Key: (((t3_2.a + t3_2.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c, ((t3_2.a + t3_2.b) / 2)
+                                             Filter: ((t3_2.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2_2.b, t2_2.c
+                                       Sort Key: t2_2.b
+                                       ->  Seq Scan on public.prt2_p3 t2_2
+                                             Output: t2_2.b, t2_2.c
+                     ->  Sort
+                           Output: t1_1.a, t1_1.c
+                           Sort Key: t1_1.a
+                           ->  Seq Scan on public.prt1_p3 t1_1
+                                 Output: t1_1.a, t1_1.c
+(81 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+                                   QUERY PLAN                                    
+---------------------------------------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Output: t1.a, t1.b, t1.c
+         Merge Cond: (t1.a = t1_3.b)
+         ->  Sort
+               Output: t1.a, t1.b, t1.c
+               Sort Key: t1.a
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_3.b, t1_6.a, t1_6.b
+               ->  Merge Semi Join
+                     Output: t1_3.b, t1_6.a, t1_6.b
+                     Merge Cond: (t1_3.b = (((t1_6.a + t1_6.b) / 2)))
+                     ->  Sort
+                           Output: t1_3.b
+                           Sort Key: t1_3.b
+                           ->  Seq Scan on public.prt2_p1 t1_3
+                                 Output: t1_3.b
+                     ->  Sort
+                           Output: t1_6.a, t1_6.b, (((t1_6.a + t1_6.b) / 2))
+                           Sort Key: (((t1_6.a + t1_6.b) / 2))
+                           ->  Seq Scan on public.prt1_e_p1 t1_6
+                                 Output: t1_6.a, t1_6.b, ((t1_6.a + t1_6.b) / 2)
+                                 Filter: ((t1_6.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_2.a, t1_2.b, t1_2.c
+         Merge Cond: (t1_2.a = t1_4.b)
+         ->  Sort
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Sort Key: t1_2.a
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+         ->  Materialize
+               Output: t1_4.b, t1_7.a, t1_7.b
+               ->  Merge Semi Join
+                     Output: t1_4.b, t1_7.a, t1_7.b
+                     Merge Cond: (t1_4.b = (((t1_7.a + t1_7.b) / 2)))
+                     ->  Sort
+                           Output: t1_4.b
+                           Sort Key: t1_4.b
+                           ->  Seq Scan on public.prt2_p2 t1_4
+                                 Output: t1_4.b
+                     ->  Sort
+                           Output: t1_7.a, t1_7.b, (((t1_7.a + t1_7.b) / 2))
+                           Sort Key: (((t1_7.a + t1_7.b) / 2))
+                           ->  Seq Scan on public.prt1_e_p2 t1_7
+                                 Output: t1_7.a, t1_7.b, ((t1_7.a + t1_7.b) / 2)
+                                 Filter: ((t1_7.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_1.a, t1_1.b, t1_1.c
+         Merge Cond: (t1_1.a = t1_5.b)
+         ->  Sort
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Sort Key: t1_1.a
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_5.b, t1_8.a, t1_8.b
+               ->  Merge Semi Join
+                     Output: t1_5.b, t1_8.a, t1_8.b
+                     Merge Cond: (t1_5.b = (((t1_8.a + t1_8.b) / 2)))
+                     ->  Sort
+                           Output: t1_5.b
+                           Sort Key: t1_5.b
+                           ->  Seq Scan on public.prt2_p3 t1_5
+                                 Output: t1_5.b
+                     ->  Sort
+                           Output: t1_8.a, t1_8.b, (((t1_8.a + t1_8.b) / 2))
+                           Sort Key: (((t1_8.a + t1_8.b) / 2))
+                           ->  Seq Scan on public.prt1_e_p3 t1_8
+                                 Output: t1_8.a, t1_8.b, ((t1_8.a + t1_8.b) / 2)
+                                 Filter: ((t1_8.a % 25) = 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                             QUERY PLAN                                              
+-----------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Merge Cond: (t1.a = t2.b)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           Sort Key: t1.a
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Merge Cond: ((((t3.a + t3.b) / 2)) = t1.a)
+                                 ->  Sort
+                                       Output: t3.a, t3.b, t3.c, (((t3.a + t3.b) / 2))
+                                       Sort Key: (((t3.a + t3.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c, ((t3.a + t3.b) / 2)
+                                             Filter: ((t3.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t1.a, t1.c
+                                       Sort Key: t1.a
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                     ->  Sort
+                           Output: t2.b, t2.c
+                           Sort Key: t2.b
+                           ->  Seq Scan on public.prt2_p1 t2
+                                 Output: t2.b, t2.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                     Merge Cond: (t1_2.a = t2_1.b)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                           Sort Key: t1_2.a
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                                 Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t1_2.a)
+                                 ->  Sort
+                                       Output: t3_1.a, t3_1.b, t3_1.c, (((t3_1.a + t3_1.b) / 2))
+                                       Sort Key: (((t3_1.a + t3_1.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c, ((t3_1.a + t3_1.b) / 2)
+                                             Filter: ((t3_1.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t1_2.a, t1_2.c
+                                       Sort Key: t1_2.a
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                     ->  Sort
+                           Output: t2_1.b, t2_1.c
+                           Sort Key: t2_1.b
+                           ->  Seq Scan on public.prt2_p2 t2_1
+                                 Output: t2_1.b, t2_1.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                     Merge Cond: (t1_1.a = t2_2.b)
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                           Sort Key: t1_1.a
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                                 Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t1_1.a)
+                                 ->  Sort
+                                       Output: t3_2.a, t3_2.b, t3_2.c, (((t3_2.a + t3_2.b) / 2))
+                                       Sort Key: (((t3_2.a + t3_2.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c, ((t3_2.a + t3_2.b) / 2)
+                                             Filter: ((t3_2.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t1_1.a, t1_1.c
+                                       Sort Key: t1_1.a
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                     ->  Sort
+                           Output: t2_2.b, t2_2.c
+                           Sort Key: t2_2.b
+                           ->  Seq Scan on public.prt2_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+(81 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SET enable_seqscan TO off;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                          QUERY PLAN                                          
+----------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Merge Cond: (t2.b = t1.a)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           Sort Key: t2.b
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Merge Cond: (((t3.a + t3.b) / 2) = t2.b)
+                                 ->  Index Scan using iprt1_e_p1_ab2 on public.prt1_e_p1 t3
+                                       Output: t3.a, t3.b, t3.c
+                                       Filter: ((t3.a % 25) = 0)
+                                 ->  Index Scan using iprt2_p1_b on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                     ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Merge Cond: (t2_1.b = t1_2.a)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           Sort Key: t2_1.b
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Merge Cond: (((t3_1.a + t3_1.b) / 2) = t2_1.b)
+                                 ->  Index Scan using iprt1_e_p2_ab2 on public.prt1_e_p2 t3_1
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       Filter: ((t3_1.a % 25) = 0)
+                                 ->  Index Scan using iprt2_p2_b on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                     ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+               ->  Merge Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Merge Cond: (t2_2.b = ((t3_2.a + t3_2.b) / 2))
+                     ->  Merge Left Join
+                           Output: t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                           Merge Cond: (t2_2.b = t1_1.a)
+                           ->  Index Scan using iprt2_p3_b on public.prt2_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+                           ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_1
+                                 Output: t1_1.a, t1_1.c
+                     ->  Index Scan using iprt1_e_p3_ab2 on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                           Filter: ((t3_2.a % 25) = 0)
+(51 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+                                    QUERY PLAN                                    
+----------------------------------------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Output: t1.a, t1.b, t1.c
+         Merge Cond: (t1.a = t1_3.b)
+         ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1
+               Output: t1.a, t1.b, t1.c
+               Filter: ((t1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_3.b, t1_6.a, t1_6.b
+               ->  Merge Semi Join
+                     Output: t1_3.b, t1_6.a, t1_6.b
+                     Merge Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
+                     ->  Index Only Scan using iprt2_p1_b on public.prt2_p1 t1_3
+                           Output: t1_3.b
+                     ->  Index Scan using iprt1_e_p1_ab2 on public.prt1_e_p1 t1_6
+                           Output: t1_6.a, t1_6.b
+                           Filter: ((t1_6.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_2.a, t1_2.b, t1_2.c
+         Merge Cond: (t1_2.a = t1_4.b)
+         ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_2
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Filter: ((t1_2.a % 25) = 0)
+         ->  Materialize
+               Output: t1_4.b, t1_7.a, t1_7.b
+               ->  Merge Semi Join
+                     Output: t1_4.b, t1_7.a, t1_7.b
+                     Merge Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
+                     ->  Index Only Scan using iprt2_p2_b on public.prt2_p2 t1_4
+                           Output: t1_4.b
+                     ->  Index Scan using iprt1_e_p2_ab2 on public.prt1_e_p2 t1_7
+                           Output: t1_7.a, t1_7.b
+                           Filter: ((t1_7.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_1.a, t1_1.b, t1_1.c
+         Merge Cond: (t1_1.a = t1_5.b)
+         ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_1
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Filter: ((t1_1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_5.b, t1_8.a, t1_8.b
+               ->  Merge Semi Join
+                     Output: t1_5.b, t1_8.a, t1_8.b
+                     Merge Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
+                     ->  Index Only Scan using iprt2_p3_b on public.prt2_p3 t1_5
+                           Output: t1_5.b
+                     ->  Index Scan using iprt1_e_p3_ab2 on public.prt1_e_p3 t1_8
+                           Output: t1_8.a, t1_8.b
+                           Filter: ((t1_8.a % 25) = 0)
+(50 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                          QUERY PLAN                                          
+----------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Merge Cond: (t1.a = t2.b)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           Sort Key: t1.a
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Merge Cond: (((t3.a + t3.b) / 2) = t1.a)
+                                 ->  Index Scan using iprt1_e_p1_ab2 on public.prt1_e_p1 t3
+                                       Output: t3.a, t3.b, t3.c
+                                       Filter: ((t3.a % 25) = 0)
+                                 ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1
+                                       Output: t1.a, t1.c
+                     ->  Index Scan using iprt2_p1_b on public.prt2_p1 t2
+                           Output: t2.b, t2.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                     Merge Cond: (t1_2.a = t2_1.b)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                           Sort Key: t1_2.a
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                                 Merge Cond: (((t3_1.a + t3_1.b) / 2) = t1_2.a)
+                                 ->  Index Scan using iprt1_e_p2_ab2 on public.prt1_e_p2 t3_1
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       Filter: ((t3_1.a % 25) = 0)
+                                 ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_2
+                                       Output: t1_2.a, t1_2.c
+                     ->  Index Scan using iprt2_p2_b on public.prt2_p2 t2_1
+                           Output: t2_1.b, t2_1.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                     Merge Cond: (t1_1.a = t2_2.b)
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                           Sort Key: t1_1.a
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                                 Merge Cond: (((t3_2.a + t3_2.b) / 2) = t1_1.a)
+                                 ->  Index Scan using iprt1_e_p3_ab2 on public.prt1_e_p3 t3_2
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       Filter: ((t3_2.a % 25) = 0)
+                                 ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_1
+                                       Output: t1_1.a, t1_1.c
+                     ->  Index Scan using iprt2_p3_b on public.prt2_p3 t2_2
+                           Output: t2_2.b, t2_2.c
+(54 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+-- lateral references and parameterized paths
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Result
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   ->  Merge Append
+         Sort Key: t1.a
+         ->  Nested Loop Left Join
+               Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a))
+               ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Merge Join
+                     Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+                     Merge Cond: (t2.a = t3.b)
+                     ->  Index Only Scan using iprt1_p1_a on public.prt1_p1 t2
+                           Output: t2.a
+                           Index Cond: (t2.a = t1.a)
+                     ->  Index Scan using iprt2_p1_b on public.prt2_p1 t3
+                           Output: t3.a, t3.b
+         ->  Nested Loop Left Join
+               Output: t1_2.a, t1_2.b, t1_2.c, t2_2.a, t3_1.a, (LEAST(t1_2.a, t2_2.a, t3_1.a))
+               ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Merge Join
+                     Output: t2_2.a, t3_1.a, LEAST(t1_2.a, t2_2.a, t3_1.a)
+                     Merge Cond: (t2_2.a = t3_1.b)
+                     ->  Index Only Scan using iprt1_p2_a on public.prt1_p2 t2_2
+                           Output: t2_2.a
+                           Index Cond: (t2_2.a = t1_2.a)
+                     ->  Index Scan using iprt2_p2_b on public.prt2_p2 t3_1
+                           Output: t3_1.a, t3_1.b
+         ->  Nested Loop Left Join
+               Output: t1_1.a, t1_1.b, t1_1.c, t2_1.a, t3_2.a, (LEAST(t1_1.a, t2_1.a, t3_2.a))
+               ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Merge Join
+                     Output: t2_1.a, t3_2.a, LEAST(t1_1.a, t2_1.a, t3_2.a)
+                     Merge Cond: (t2_1.a = t3_2.b)
+                     ->  Index Only Scan using iprt1_p3_a on public.prt1_p3 t2_1
+                           Output: t2_1.a
+                           Index Cond: (t2_1.a = t1_1.a)
+                     ->  Index Scan using iprt2_p3_b on public.prt2_p3 t3_2
+                           Output: t3_2.a, t3_2.b
+(43 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
+ Nested Loop Left Join
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   ->  Merge Append
+         Sort Key: t1.a
+         ->  Sort
+               Output: t1.a, t1.b, t1.c
+               Sort Key: t1.a
+               ->  Seq Scan on public.prt1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+         ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1_1
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Filter: ((t1_1.a % 25) = 0)
+         ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_2
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Filter: ((t1_2.a % 25) = 0)
+         ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_3
+               Output: t1_3.a, t1_3.b, t1_3.c
+               Filter: ((t1_3.a % 25) = 0)
+   ->  Append
+         ->  Merge Join
+               Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+               Merge Cond: (t2.a = t3.b)
+               ->  Index Only Scan using iprt1_p1_a on public.prt1_p1 t2
+                     Output: t2.a
+                     Index Cond: (t2.a = t1.b)
+               ->  Index Scan using iprt2_p1_b on public.prt2_p1 t3
+                     Output: t3.a, t3.b
+         ->  Merge Join
+               Output: t2_2.a, t3_1.a, LEAST(t1.a, t2_2.a, t3_1.a)
+               Merge Cond: (t2_2.a = t3_1.b)
+               ->  Index Only Scan using iprt1_p2_a on public.prt1_p2 t2_2
+                     Output: t2_2.a
+                     Index Cond: (t2_2.a = t1.b)
+               ->  Index Scan using iprt2_p2_b on public.prt2_p2 t3_1
+                     Output: t3_1.a, t3_1.b
+         ->  Merge Join
+               Output: t2_1.a, t3_2.a, LEAST(t1.a, t2_1.a, t3_2.a)
+               Merge Cond: (t2_1.a = t3_2.b)
+               ->  Index Only Scan using iprt1_p3_a on public.prt1_p3 t2_1
+                     Output: t2_1.a
+                     Index Cond: (t2_1.a = t1.b)
+               ->  Index Scan using iprt2_p3_b on public.prt2_p3 t3_2
+                     Output: t3_2.a, t3_2.b
+(44 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+RESET enable_hashjoin;
+RESET enable_nestloop;
+RESET enable_seqscan;
+--
+-- partitioned by multiple columns
+--
+CREATE TABLE prt1_m (a int, b int, c varchar) 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, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_m;
+ANALYZE prt1_m_p1;
+ANALYZE prt1_m_p2;
+ANALYZE prt1_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_m AS SELECT * FROM prt1_m;
+CREATE TABLE prt2_m (a int, b int, c varchar) 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, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_m;
+ANALYZE prt2_m_p1;
+ANALYZE prt2_m_p2;
+ANALYZE prt2_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_m AS SELECT * FROM prt2_m;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+                                                 QUERY PLAN                                                 
+------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: ((((t1.a + t1.b) / 2) = t2.b) AND (t1.a = ((t2.b + t2.a) / 2)))
+                     ->  Seq Scan on public.prt1_m_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                     ->  Hash
+                           Output: t2.b, t2.c, t2.a
+                           ->  Seq Scan on public.prt2_m_p1 t2
+                                 Output: t2.b, t2.c, t2.a
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: ((((t1_1.a + t1_1.b) / 2) = t2_1.b) AND (t1_1.a = ((t2_1.b + t2_1.a) / 2)))
+                     ->  Seq Scan on public.prt1_m_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c, t2_1.a
+                           ->  Seq Scan on public.prt2_m_p2 t2_1
+                                 Output: t2_1.b, t2_1.c, t2_1.a
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: ((((t1_2.a + t1_2.b) / 2) = t2_2.b) AND (t1_2.a = ((t2_2.b + t2_2.a) / 2)))
+                     ->  Seq Scan on public.prt1_m_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c, t2_2.a
+                           ->  Seq Scan on public.prt2_m_p3 t2_2
+                                 Output: t2_2.b, t2_2.c, t2_2.a
+                                 Filter: ((t2_2.b % 25) = 0)
+(36 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_m t1 RIGHT JOIN uprt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 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
+   Output: prt1_m_p1.a, prt1_m_p1.c, prt2_m_p1.b, prt2_m_p1.c
+   Sort Key: prt1_m_p1.a, prt2_m_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_m_p1.a, prt1_m_p1.c, prt2_m_p1.b, prt2_m_p1.c
+               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 public.prt1_m_p1
+                     Output: prt1_m_p1.a, prt1_m_p1.c, prt1_m_p1.b
+                     Filter: ((prt1_m_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_m_p1.b, prt2_m_p1.c, prt2_m_p1.a
+                     ->  Seq Scan on public.prt2_m_p1
+                           Output: prt2_m_p1.b, prt2_m_p1.c, prt2_m_p1.a
+                           Filter: ((prt2_m_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_m_p2.a, prt1_m_p2.c, prt2_m_p2.b, prt2_m_p2.c
+               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 public.prt1_m_p2
+                     Output: prt1_m_p2.a, prt1_m_p2.c, prt1_m_p2.b
+                     Filter: ((prt1_m_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_m_p2.b, prt2_m_p2.c, prt2_m_p2.a
+                     ->  Seq Scan on public.prt2_m_p2
+                           Output: prt2_m_p2.b, prt2_m_p2.c, prt2_m_p2.a
+                           Filter: ((prt2_m_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_m_p3.a, prt1_m_p3.c, prt2_m_p3.b, prt2_m_p3.c
+               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 public.prt1_m_p3
+                     Output: prt1_m_p3.a, prt1_m_p3.c, prt1_m_p3.b
+                     Filter: ((prt1_m_p3.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_m_p3.b, prt2_m_p3.c, prt2_m_p3.a
+                     ->  Seq Scan on public.prt2_m_p3
+                           Output: prt2_m_p3.b, prt2_m_p3.c, prt2_m_p3.a
+                           Filter: ((prt2_m_p3.b % 25) = 0)
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 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 | 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_m t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_m t2 WHERE t2.b % 25 = 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 | 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)
+
+--
+-- 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;
+ANALYZE plt1_p1;
+ANALYZE plt1_p2;
+ANALYZE plt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1 AS SELECT * FROM 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;
+ANALYZE plt2_p1;
+ANALYZE plt2_p2;
+ANALYZE plt2_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt2 AS SELECT * FROM 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;
+ANALYZE plt1_e_p1;
+ANALYZE plt1_e_p2;
+ANALYZE plt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1_e AS SELECT * FROM plt1_e;
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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
+   Output: (avg(t1.a)), (avg(t2.b)), (avg((t3.a + t3.b))), t1.c, t2.c, t3.c
+   Sort Key: t1.c, t3.c
+   ->  HashAggregate
+         Output: avg(t1.a), avg(t2.b), avg((t3.a + t3.b)), t1.c, t2.c, t3.c
+         Group Key: t1.c, t2.c, t3.c
+         ->  Result
+               Output: t1.c, t2.c, t3.c, t1.a, t2.b, t3.a, t3.b
+               ->  Append
+                     ->  Hash Join
+                           Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                           Hash Cond: (t1.c = t2.c)
+                           ->  Seq Scan on public.plt1_p1 t1
+                                 Output: t1.a, t1.c
+                           ->  Hash
+                                 Output: t2.b, t2.c, t3.a, t3.b, t3.c
+                                 ->  Hash Join
+                                       Output: t2.b, t2.c, t3.a, t3.b, t3.c
+                                       Hash Cond: (t2.c = ltrim(t3.c, 'A'::text))
+                                       ->  Seq Scan on public.plt2_p1 t2
+                                             Output: t2.b, t2.c
+                                       ->  Hash
+                                             Output: t3.a, t3.b, t3.c
+                                             ->  Seq Scan on public.plt1_e_p1 t3
+                                                   Output: t3.a, t3.b, t3.c
+                     ->  Hash Join
+                           Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                           Hash Cond: (t1_1.c = t2_1.c)
+                           ->  Seq Scan on public.plt1_p2 t1_1
+                                 Output: t1_1.a, t1_1.c
+                           ->  Hash
+                                 Output: t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                                 ->  Hash Join
+                                       Output: t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                                       Hash Cond: (t2_1.c = ltrim(t3_1.c, 'A'::text))
+                                       ->  Seq Scan on public.plt2_p2 t2_1
+                                             Output: t2_1.b, t2_1.c
+                                       ->  Hash
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             ->  Seq Scan on public.plt1_e_p2 t3_1
+                                                   Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash Join
+                           Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                           Hash Cond: (t1_2.c = t2_2.c)
+                           ->  Seq Scan on public.plt1_p3 t1_2
+                                 Output: t1_2.a, t1_2.c
+                           ->  Hash
+                                 Output: t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                                 ->  Hash Join
+                                       Output: t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                                       Hash Cond: (t2_2.c = ltrim(t3_2.c, 'A'::text))
+                                       ->  Seq Scan on public.plt2_p3 t2_2
+                                             Output: t2_2.b, t2_2.c
+                                       ->  Hash
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             ->  Seq Scan on public.plt1_e_p3 t3_2
+                                                   Output: t3_2.a, t3_2.b, t3_2.c
+(57 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)
+
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM uplt1 t1, uplt2 t2, uplt1_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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: ((t3.a = t1.a) AND (ltrim(t3.c, 'A'::text) = t1.c))
+                     ->  Seq Scan on public.plt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                                 ->  Seq Scan on public.plt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.plt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: ((t3_1.a = t1_1.a) AND (ltrim(t3_1.c, 'A'::text) = t1_1.c))
+                     ->  Seq Scan on public.plt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.c = t1_1.c))
+                                 ->  Seq Scan on public.plt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.plt1_p2 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: ((t3_2.a = t1_2.a) AND (ltrim(t3_2.c, 'A'::text) = t1_2.c))
+                     ->  Seq Scan on public.plt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                                 ->  Seq Scan on public.plt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.plt1_p3 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: ((t3.a = t2.b) AND (ltrim(t3.c, 'A'::text) = t2.c))
+                     ->  Seq Scan on public.plt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                                 ->  Seq Scan on public.plt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.plt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: ((t3_1.a = t2_1.b) AND (ltrim(t3_1.c, 'A'::text) = t2_1.c))
+                     ->  Seq Scan on public.plt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.c = t1_1.c))
+                                 ->  Seq Scan on public.plt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.plt1_p2 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: ((t3_2.a = t2_2.b) AND (ltrim(t3_2.c, 'A'::text) = t2_2.c))
+                     ->  Seq Scan on public.plt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                                 ->  Seq Scan on public.plt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.plt1_p3 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |          | 
+ 100 | 0002 |     |      |          | 
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |          | 
+ 250 | 0005 |     |      |          | 
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |          | 
+ 400 | 0008 |     |      |          | 
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |          | 
+ 550 | 0011 |     |      |          | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |          | 
+ 100 | 0002 |     |      |          | 
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |          | 
+ 250 | 0005 |     |      |          | 
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |          | 
+ 400 | 0008 |     |      |          | 
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |          | 
+ 550 | 0011 |     |      |          | 
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Hash Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                     ->  Seq Scan on public.plt2_p1 t2
+                           Output: t2.b, t2.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Hash Cond: ((t1.c = ltrim(t3.c, 'A'::text)) AND (t1.a = t3.a))
+                                 ->  Seq Scan on public.plt1_p1 t1
+                                       Output: t1.a, t1.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.plt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                     Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.c = t1_1.c))
+                     ->  Seq Scan on public.plt2_p2 t2_1
+                           Output: t2_1.b, t2_1.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_1.a, t1_1.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_1.a, t1_1.c
+                                 Hash Cond: ((t1_1.c = ltrim(t3_1.c, 'A'::text)) AND (t1_1.a = t3_1.a))
+                                 ->  Seq Scan on public.plt1_p2 t1_1
+                                       Output: t1_1.a, t1_1.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.plt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                     Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                     ->  Seq Scan on public.plt2_p3 t2_2
+                           Output: t2_2.b, t2_2.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_2.a, t1_2.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_2.a, t1_2.c
+                                 Hash Cond: ((t1_2.c = ltrim(t3_2.c, 'A'::text)) AND (t1_2.a = t3_2.a))
+                                 ->  Seq Scan on public.plt1_p3 t1_2
+                                       Output: t1_2.a, t1_2.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.plt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: ((t1.a = t2.b) AND (t1.c = t2.c))
+                     ->  Seq Scan on public.plt1_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Hash Cond: ((t2.b = t3.a) AND (t2.c = ltrim(t3.c, 'A'::text)))
+                                 ->  Seq Scan on public.plt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.plt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: ((t1_1.a = t2_1.b) AND (t1_1.c = t2_1.c))
+                     ->  Seq Scan on public.plt1_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: ((t2_1.b = t3_1.a) AND (t2_1.c = ltrim(t3_1.c, 'A'::text)))
+                                 ->  Seq Scan on public.plt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.plt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: ((t1_2.a = t2_2.b) AND (t1_2.c = t2_2.c))
+                     ->  Seq Scan on public.plt1_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: ((t2_2.b = t3_2.a) AND (t2_2.c = ltrim(t3_2.c, 'A'::text)))
+                                 ->  Seq Scan on public.plt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.plt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+     |      |     |      |      100 | A0001
+     |      |     |      |      200 | A0002
+     |      |     |      |      400 | A0004
+     |      |     |      |      500 | A0005
+     |      |     |      |      700 | A0007
+     |      |     |      |      800 | A0008
+     |      |     |      |     1000 | A0010
+     |      |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+     |      |     |      |      100 | A0001
+     |      |     |      |      200 | A0002
+     |      |     |      |      400 | A0004
+     |      |     |      |      500 | A0005
+     |      |     |      |      700 | A0007
+     |      |     |      |      800 | A0008
+     |      |     |      |     1000 | A0010
+     |      |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                                  QUERY PLAN                                                   
+---------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, ((plt1_e_p1.a + plt1_e_p1.b)), plt1_e_p1.c
+   Sort Key: plt1_p1.a, plt2_p1.b, ((plt1_e_p1.a + plt1_e_p1.b))
+   ->  Result
+         Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, (plt1_e_p1.a + plt1_e_p1.b), plt1_e_p1.c
+         ->  Append
+               ->  Hash Full Join
+                     Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                     Hash Cond: ((plt1_p1.a = plt1_e_p1.a) AND (plt1_p1.c = ltrim(plt1_e_p1.c, 'A'::text)))
+                     ->  Hash Full Join
+                           Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+                           Hash Cond: ((plt1_p1.a = plt2_p1.b) AND (plt1_p1.c = plt2_p1.c))
+                           ->  Seq Scan on public.plt1_p1
+                                 Output: plt1_p1.a, plt1_p1.c
+                                 Filter: ((plt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p1.b, plt2_p1.c
+                                 ->  Seq Scan on public.plt2_p1
+                                       Output: plt2_p1.b, plt2_p1.c
+                                       Filter: ((plt2_p1.b % 25) = 0)
+                     ->  Hash
+                           Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                           ->  Seq Scan on public.plt1_e_p1
+                                 Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                                 Filter: ((plt1_e_p1.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c, plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                     Hash Cond: ((plt1_p2.a = plt1_e_p2.a) AND (plt1_p2.c = ltrim(plt1_e_p2.c, 'A'::text)))
+                     ->  Hash Full Join
+                           Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+                           Hash Cond: ((plt1_p2.a = plt2_p2.b) AND (plt1_p2.c = plt2_p2.c))
+                           ->  Seq Scan on public.plt1_p2
+                                 Output: plt1_p2.a, plt1_p2.c
+                                 Filter: ((plt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p2.b, plt2_p2.c
+                                 ->  Seq Scan on public.plt2_p2
+                                       Output: plt2_p2.b, plt2_p2.c
+                                       Filter: ((plt2_p2.b % 25) = 0)
+                     ->  Hash
+                           Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                           ->  Seq Scan on public.plt1_e_p2
+                                 Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                                 Filter: ((plt1_e_p2.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c, plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                     Hash Cond: ((plt1_p3.a = plt1_e_p3.a) AND (plt1_p3.c = ltrim(plt1_e_p3.c, 'A'::text)))
+                     ->  Hash Full Join
+                           Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+                           Hash Cond: ((plt1_p3.a = plt2_p3.b) AND (plt1_p3.c = plt2_p3.c))
+                           ->  Seq Scan on public.plt1_p3
+                                 Output: plt1_p3.a, plt1_p3.c
+                                 Filter: ((plt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p3.b, plt2_p3.c
+                                 ->  Seq Scan on public.plt2_p3
+                                       Output: plt2_p3.b, plt2_p3.c
+                                       Filter: ((plt2_p3.b % 25) = 0)
+                     ->  Hash
+                           Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                           ->  Seq Scan on public.plt1_e_p3
+                                 Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                                 Filter: ((plt1_e_p3.a % 25) = 0)
+(63 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.c = t1_3.c)
+               ->  Seq Scan on public.plt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_3.c, t2.c
+                     Hash Cond: (t1_3.c = ltrim(t2.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p1 t1_3
+                           Output: t1_3.c
+                     ->  Hash
+                           Output: t2.c
+                           ->  Seq Scan on public.plt1_e_p1 t2
+                                 Output: t2.c
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.c = t1_4.c)
+               ->  Seq Scan on public.plt1_p2 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_4.c, t2_1.c
+                     Hash Cond: (t1_4.c = ltrim(t2_1.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p2 t1_4
+                           Output: t1_4.c
+                     ->  Hash
+                           Output: t2_1.c
+                           ->  Seq Scan on public.plt1_e_p2 t2_1
+                                 Output: t2_1.c
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.c = t1_5.c)
+               ->  Seq Scan on public.plt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_5.c, t2_2.c
+                     Hash Cond: (t1_5.c = ltrim(t2_2.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p3 t1_5
+                           Output: t1_5.c
+                     ->  Hash
+                           Output: t2_2.c
+                           ->  Seq Scan on public.plt1_e_p3 t2_2
+                                 Output: t2_2.c
+(49 rows)
+
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1, uplt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+                                   QUERY PLAN                                   
+--------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.c = t1_3.c)
+               ->  Seq Scan on public.plt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_3.c, t1_6.c
+                     Hash Cond: (t1_3.c = ltrim(t1_6.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p1 t1_3
+                           Output: t1_3.c
+                     ->  Hash
+                           Output: t1_6.c
+                           ->  HashAggregate
+                                 Output: t1_6.c
+                                 Group Key: ltrim(t1_6.c, 'A'::text)
+                                 ->  Seq Scan on public.plt1_e_p1 t1_6
+                                       Output: t1_6.c, ltrim(t1_6.c, 'A'::text)
+                                       Filter: ((t1_6.a % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.c = t1_4.c)
+               ->  Seq Scan on public.plt1_p2 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_4.c, t1_7.c
+                     Hash Cond: (t1_4.c = ltrim(t1_7.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p2 t1_4
+                           Output: t1_4.c
+                     ->  Hash
+                           Output: t1_7.c
+                           ->  HashAggregate
+                                 Output: t1_7.c
+                                 Group Key: ltrim(t1_7.c, 'A'::text)
+                                 ->  Seq Scan on public.plt1_e_p2 t1_7
+                                       Output: t1_7.c, ltrim(t1_7.c, 'A'::text)
+                                       Filter: ((t1_7.a % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.c = t1_5.c)
+               ->  Seq Scan on public.plt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_5.c, t1_8.c
+                     Hash Cond: (t1_5.c = ltrim(t1_8.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p3 t1_5
+                           Output: t1_5.c
+                     ->  Hash
+                           Output: t1_8.c
+                           ->  HashAggregate
+                                 Output: t1_8.c
+                                 Group Key: ltrim(t1_8.c, 'A'::text)
+                                 ->  Seq Scan on public.plt1_e_p3 t1_8
+                                       Output: t1_8.c, ltrim(t1_8.c, 'A'::text)
+                                       Filter: ((t1_8.a % 25) = 0)
+(61 rows)
+
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+--
+-- negative testcases
+--
+-- joins where one of the relations is proven empty
+EXPLAIN (VERBOSE, 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;
+            QUERY PLAN            
+----------------------------------
+ Result
+   Output: t1.a, t1.c, t2.b, t2.c
+   One-Time Filter: false
+(3 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 LEFT JOIN prt2 t2 ON t1.a = t2.b;
+              QUERY PLAN              
+--------------------------------------
+ Result
+   Output: prt1.a, prt1.c, t2.b, t2.c
+   One-Time Filter: false
+(3 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                     
+---------------------------------------------------
+ Sort
+   Output: a, c, t2.b, t2.c
+   Sort Key: a, t2.b
+   ->  Hash Left Join
+         Output: a, c, t2.b, t2.c
+         Hash Cond: (t2.b = a)
+         ->  Append
+               ->  Seq Scan on public.prt2 t2
+                     Output: t2.b, t2.c
+                     Filter: ((t2.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p1 t2_1
+                     Output: t2_1.b, t2_1.c
+                     Filter: ((t2_1.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p2 t2_2
+                     Output: t2_2.b, t2_2.c
+                     Filter: ((t2_2.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p3 t2_3
+                     Output: t2_3.b, t2_3.c
+                     Filter: ((t2_3.a % 25) = 0)
+         ->  Hash
+               Output: a, c
+               ->  Result
+                     Output: a, c
+                     One-Time Filter: false
+(24 rows)
+
+EXPLAIN (VERBOSE, 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 % 25 = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                     
+---------------------------------------------------
+ Sort
+   Output: a, c, t2.b, t2.c
+   Sort Key: a, t2.b
+   ->  Hash Left Join
+         Output: a, c, t2.b, t2.c
+         Hash Cond: (t2.b = a)
+         ->  Append
+               ->  Seq Scan on public.prt2 t2
+                     Output: t2.b, t2.c
+                     Filter: ((t2.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p1 t2_1
+                     Output: t2_1.b, t2_1.c
+                     Filter: ((t2_1.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p2 t2_2
+                     Output: t2_2.b, t2_2.c
+                     Filter: ((t2_2.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p3 t2_3
+                     Output: t2_3.b, t2_3.c
+                     Filter: ((t2_3.a % 25) = 0)
+         ->  Hash
+               Output: a, c
+               ->  Result
+                     Output: a, c
+                     One-Time Filter: false
+(24 rows)
+
+CREATE TABLE prt1_n (a int, b int, c varchar) PARTITION BY RANGE(c);
+CREATE TABLE prt1_n_p1 PARTITION OF prt1_n FOR VALUES FROM ('0000') TO ('0250');
+CREATE TABLE prt1_n_p2 PARTITION OF prt1_n FOR VALUES FROM ('0250') TO ('0500');
+INSERT INTO prt1_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 499, 2) i;
+ANALYZE prt1_n;
+ANALYZE prt1_n_p1;
+ANALYZE prt1_n_p2;
+CREATE TABLE prt2_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt2_n_p1 PARTITION OF prt2_n FOR VALUES IN ('0000', '0003', '0004', '0010', '0006', '0007');
+CREATE TABLE prt2_n_p2 PARTITION OF prt2_n FOR VALUES IN ('0001', '0005', '0002', '0009', '0008', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt2_n;
+ANALYZE prt2_n_p1;
+ANALYZE prt2_n_p2;
+CREATE TABLE prt3_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt3_n_p1 PARTITION OF prt3_n FOR VALUES IN ('0000', '0004', '0006', '0007');
+CREATE TABLE prt3_n_p2 PARTITION OF prt3_n FOR VALUES IN ('0001', '0002', '0008', '0010');
+CREATE TABLE prt3_n_p3 PARTITION OF prt3_n FOR VALUES IN ('0003', '0005', '0009', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt3_n;
+ANALYZE prt3_n_p1;
+ANALYZE prt3_n_p2;
+ANALYZE prt3_n_p3;
+CREATE TABLE prt4_n (a int, b int, c text) PARTITION BY RANGE(a);
+CREATE TABLE prt4_n_p1 PARTITION OF prt4_n FOR VALUES FROM (0) TO (300);
+CREATE TABLE prt4_n_p2 PARTITION OF prt4_n FOR VALUES FROM (300) TO (500);
+CREATE TABLE prt4_n_p3 PARTITION OF prt4_n FOR VALUES FROM (500) TO (600);
+INSERT INTO prt4_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt4_n;
+ANALYZE prt4_n_p1;
+ANALYZE prt4_n_p2;
+ANALYZE prt4_n_p3;
+-- partition-wise join can not be applied if the partition ranges differ
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Join
+   Hash Cond: (t1.a = t2.a)
+   ->  Append
+         ->  Seq Scan on prt1 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p2 t1_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt4_n t2
+               ->  Seq Scan on prt4_n_p1 t2_1
+               ->  Seq Scan on prt4_n_p2 t2_2
+               ->  Seq Scan on prt4_n_p3 t2_3
+(13 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 FULL JOIN prt4_n t2 ON t1.a = t2.a;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Full Join
+   Hash Cond: (t1.a = t2.a)
+   ->  Append
+         ->  Seq Scan on prt1 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p2 t1_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt4_n t2
+               ->  Seq Scan on prt4_n_p1 t2_1
+               ->  Seq Scan on prt4_n_p2 t2_2
+               ->  Seq Scan on prt4_n_p3 t2_3
+(13 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                        
+---------------------------------------------------------
+ Nested Loop Left Join
+   ->  Append
+         ->  Seq Scan on prt1 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p2 t1_3
+   ->  Append
+         ->  Seq Scan on prt2 t2
+               Filter: (t1.a < b)
+         ->  Index Scan using iprt2_p1_b on prt2_p1 t2_1
+               Index Cond: (t1.a < b)
+         ->  Index Scan using iprt2_p2_b on prt2_p2 t2_2
+               Index Cond: (t1.a < b)
+         ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+               Index Cond: (t1.a < b)
+(15 rows)
+
+-- equi-join with join condition on partial keys does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1, prt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t1.a % 25 = 0;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Join
+   Hash Cond: (((t2.b + t2.a) / 2) = t1.a)
+   ->  Append
+         ->  Seq Scan on prt2_m t2
+         ->  Seq Scan on prt2_m_p1 t2_1
+         ->  Seq Scan on prt2_m_p2 t2_2
+         ->  Seq Scan on prt2_m_p3 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_m t1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p1 t1_1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p2 t1_2
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p3 t1_3
+                     Filter: ((a % 25) = 0)
+(17 rows)
+
+-- equi-join between out-of-order partition key columns does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.a = t2.b WHERE t1.a % 25 = 0;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Right Join
+   Hash Cond: (t2.b = t1.a)
+   ->  Append
+         ->  Seq Scan on prt2_m t2
+         ->  Seq Scan on prt2_m_p1 t2_1
+         ->  Seq Scan on prt2_m_p2 t2_2
+         ->  Seq Scan on prt2_m_p3 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_m t1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p1 t1_1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p2 t1_2
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p3 t1_3
+                     Filter: ((a % 25) = 0)
+(17 rows)
+
+-- equi-join between non-key columns does not qualify for partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.c = t2.c WHERE t1.a % 25 = 0;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Right Join
+   Hash Cond: ((t2.c)::text = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt2_m t2
+         ->  Seq Scan on prt2_m_p1 t2_1
+         ->  Seq Scan on prt2_m_p2 t2_2
+         ->  Seq Scan on prt2_m_p3 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_m t1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p1 t1_1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p2 t1_2
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p3 t1_3
+                     Filter: ((a % 25) = 0)
+(17 rows)
+
+-- partition-wise join can not be applied for a join between list and range
+-- partitioned table
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1, prt2_n t2 WHERE t1.c = t2.c;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Join
+   Hash Cond: (t2.c = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt2_n t2
+         ->  Seq Scan on prt2_n_p1 t2_1
+         ->  Seq Scan on prt2_n_p2 t2_2
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(11 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 LEFT JOIN prt2_n t2 ON (t1.c = t2.c);
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Right Join
+   Hash Cond: (t2.c = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt2_n t2
+         ->  Seq Scan on prt2_n_p1 t2_1
+         ->  Seq Scan on prt2_n_p2 t2_2
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(11 rows)
+
+-- partition-wise join can not be applied between tables with different
+-- partition lists
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 RIGHT JOIN prt1 t2 ON (t1.c = t2.c);
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Left Join
+   Hash Cond: ((t2.c)::text = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt1 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p2 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(12 rows)
+
+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);
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Full Join
+   Hash Cond: ((t2.c)::text = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt1 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p2 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(12 rows)
+
diff --git a/src/test/regress/expected/rangefuncs.out b/src/test/regress/expected/rangefuncs.out
index f06cfa4..16e7f56 100644
--- a/src/test/regress/expected/rangefuncs.out
+++ b/src/test/regress/expected/rangefuncs.out
@@ -1,18 +1,19 @@
 SELECT name, setting FROM pg_settings WHERE name LIKE 'enable%';
-         name         | setting 
-----------------------+---------
- enable_bitmapscan    | on
- enable_hashagg       | on
- enable_hashjoin      | on
- enable_indexonlyscan | on
- enable_indexscan     | on
- enable_material      | on
- enable_mergejoin     | on
- enable_nestloop      | on
- enable_seqscan       | on
- enable_sort          | on
- enable_tidscan       | on
-(11 rows)
+            name            | setting 
+----------------------------+---------
+ enable_bitmapscan          | on
+ enable_hashagg             | on
+ enable_hashjoin            | on
+ enable_indexonlyscan       | on
+ enable_indexscan           | on
+ enable_material            | on
+ enable_mergejoin           | on
+ enable_nestloop            | on
+ enable_partition_wise_join | on
+ enable_seqscan             | on
+ enable_sort                | on
+ enable_tidscan             | on
+(12 rows)
 
 CREATE TABLE foo2(fooid int, f2 int);
 INSERT INTO foo2 VALUES(1, 11);
diff --git a/src/test/regress/parallel_schedule b/src/test/regress/parallel_schedule
index 8641769..b61ca3b 100644
--- a/src/test/regress/parallel_schedule
+++ b/src/test/regress/parallel_schedule
@@ -99,8 +99,9 @@ test: select_parallel
 
 # ----------
 # Another group of parallel tests
+# TODO: merge partition_join and multi_level_partition_join
 # ----------
-test: select_views portals_p2 foreign_key cluster dependency guc bitmapops combocid tsearch tsdicts foreign_data window xmlmap functional_deps advisory_lock json jsonb json_encoding indirect_toast equivclass
+test: select_views portals_p2 foreign_key cluster dependency guc bitmapops combocid tsearch tsdicts foreign_data window xmlmap functional_deps advisory_lock json jsonb json_encoding indirect_toast equivclass partition_join multi_level_partition_join
 # ----------
 # Another group of parallel tests
 # NB: temp.sql does a reconnect which transiently uses 2 connections,
diff --git a/src/test/regress/serial_schedule b/src/test/regress/serial_schedule
index 835cf35..5b167b6 100644
--- a/src/test/regress/serial_schedule
+++ b/src/test/regress/serial_schedule
@@ -169,3 +169,5 @@ test: with
 test: xml
 test: event_trigger
 test: stats
+test: partition_join
+test: multi_level_partition_join
diff --git a/src/test/regress/sql/multi_level_partition_join.sql b/src/test/regress/sql/multi_level_partition_join.sql
new file mode 100644
index 0000000..31f0281
--- /dev/null
+++ b/src/test/regress/sql/multi_level_partition_join.sql
@@ -0,0 +1,95 @@
+--
+-- multi-leveled partitions
+--
+CREATE TABLE prt1_l (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_l_p1 PARTITION OF prt1_l FOR VALUES FROM (0) TO (250) PARTITION BY RANGE (b);
+CREATE TABLE prt1_l_p1_p1 PARTITION OF prt1_l_p1 FOR VALUES FROM (0) TO (100);
+CREATE TABLE prt1_l_p1_p2 PARTITION OF prt1_l_p1 FOR VALUES FROM (100) TO (250);
+CREATE TABLE prt1_l_p2 PARTITION OF prt1_l FOR VALUES FROM (250) TO (500) PARTITION BY RANGE (c);
+CREATE TABLE prt1_l_p2_p1 PARTITION OF prt1_l_p2 FOR VALUES FROM ('0250') TO ('0400');
+CREATE TABLE prt1_l_p2_p2 PARTITION OF prt1_l_p2 FOR VALUES FROM ('0400') TO ('0500');
+CREATE TABLE prt1_l_p3 PARTITION OF prt1_l FOR VALUES FROM (500) TO (600) PARTITION BY RANGE ((b + a));
+CREATE TABLE prt1_l_p3_p1 PARTITION OF prt1_l_p3 FOR VALUES FROM (1000) TO (1100);
+CREATE TABLE prt1_l_p3_p2 PARTITION OF prt1_l_p3 FOR VALUES FROM (1100) TO (1200);
+INSERT INTO prt1_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_l;
+ANALYZE prt1_l_p1;
+ANALYZE prt1_l_p1_p1;
+ANALYZE prt1_l_p1_p2;
+ANALYZE prt1_l_p2;
+ANALYZE prt1_l_p2_p1;
+ANALYZE prt1_l_p2_p2;
+ANALYZE prt1_l_p3;
+ANALYZE prt1_l_p3_p1;
+ANALYZE prt1_l_p3_p2;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_l AS SELECT * FROM prt1_l;
+
+CREATE TABLE prt2_l (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_l_p1 PARTITION OF prt2_l FOR VALUES FROM (0) TO (250) PARTITION BY RANGE (a);
+CREATE TABLE prt2_l_p1_p1 PARTITION OF prt2_l_p1 FOR VALUES FROM (0) TO (100);
+CREATE TABLE prt2_l_p1_p2 PARTITION OF prt2_l_p1 FOR VALUES FROM (100) TO (250);
+CREATE TABLE prt2_l_p2 PARTITION OF prt2_l FOR VALUES FROM (250) TO (500) PARTITION BY RANGE (c);
+CREATE TABLE prt2_l_p2_p1 PARTITION OF prt2_l_p2 FOR VALUES FROM ('0250') TO ('0400');
+CREATE TABLE prt2_l_p2_p2 PARTITION OF prt2_l_p2 FOR VALUES FROM ('0400') TO ('0500');
+CREATE TABLE prt2_l_p3 PARTITION OF prt2_l FOR VALUES FROM (500) TO (600) PARTITION BY RANGE ((a + b));
+CREATE TABLE prt2_l_p3_p1 PARTITION OF prt2_l_p3 FOR VALUES FROM (1000) TO (1100);
+CREATE TABLE prt2_l_p3_p2 PARTITION OF prt2_l_p3 FOR VALUES FROM (1100) TO (1200);
+INSERT INTO prt2_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_l;
+ANALYZE prt2_l_p1;
+ANALYZE prt2_l_p1_p1;
+ANALYZE prt2_l_p1_p2;
+ANALYZE prt2_l_p2;
+ANALYZE prt2_l_p2_p1;
+ANALYZE prt2_l_p2_p2;
+ANALYZE prt2_l_p3;
+ANALYZE prt2_l_p3_p1;
+ANALYZE prt2_l_p3_p2;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_l AS SELECT * FROM prt2_l;
+
+EXPLAIN (VERBOSE, 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 = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 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 = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1, uprt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 LEFT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 RIGHT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_l t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_l t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+
+-- lateral reference
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
+			  ON t1.a = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
+			  ON t1.a = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1_l t2 JOIN uprt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
+			  ON t1.a = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
+			  ON t1.b = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
+			  ON t1.b = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1_l t2 JOIN uprt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
+			  ON t1.b = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
new file mode 100644
index 0000000..9b2baeb
--- /dev/null
+++ b/src/test/regress/sql/partition_join.sql
@@ -0,0 +1,520 @@
+--
+-- PARTITION_JOIN
+-- Test partition-wise join between partitioned tables
+--
+
+-- Usually partition-wise join paths are chosen when data is large, which would
+-- take regression tests to run longer. So, weigh partition-wise joins cheaper
+-- to force those even for smaller data.
+SET partition_wise_plan_weight to 0.2;
+
+--
+-- partitioned by a single column
+--
+CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+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, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1 AS SELECT * FROM prt1;
+
+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_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, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+CREATE TABLE uprt2 AS SELECT * FROM prt2;
+
+-- inner join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+-- left outer join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+-- right outer join
+EXPLAIN (VERBOSE, 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.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+-- full outer join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+
+-- Join with pruned partitions from joining relations
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a < 450 AND t2.b > 250 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+-- lateral reference
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+--
+-- partitioned by expression
+--
+CREATE TABLE prt1_e (a int, b int, c varchar) PARTITION BY RANGE(((a + b)/2));
+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);
+INSERT INTO prt1_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_e AS SELECT * FROM prt1_e;
+
+CREATE TABLE prt2_e (a int, b int, c varchar) PARTITION BY RANGE(((b + a)/2));
+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);
+INSERT INTO prt2_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_e;
+ANALYZE prt2_e_p1;
+ANALYZE prt2_e_p2;
+ANALYZE prt2_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_e AS SELECT * FROM prt2_e;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1, uprt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1 LEFT JOIN uprt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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.a % 25 = 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM uprt1 t1, uprt2 t2, uprt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM uprt1_e WHERE uprt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, 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.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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;
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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;
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM uprt1_e WHERE uprt1_e.a % 25 = 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;
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1, uprt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+-- test merge joins with and without using indexes
+SET enable_hashjoin TO off;
+SET enable_nestloop TO off;
+
+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 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 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));
+
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+SET enable_seqscan TO off;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+-- lateral references and parameterized paths
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+RESET enable_hashjoin;
+RESET enable_nestloop;
+RESET enable_seqscan;
+
+--
+-- partitioned by multiple columns
+--
+CREATE TABLE prt1_m (a int, b int, c varchar) 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, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_m;
+ANALYZE prt1_m_p1;
+ANALYZE prt1_m_p2;
+ANALYZE prt1_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_m AS SELECT * FROM prt1_m;
+
+CREATE TABLE prt2_m (a int, b int, c varchar) 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, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_m;
+ANALYZE prt2_m_p1;
+ANALYZE prt2_m_p2;
+ANALYZE prt2_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_m AS SELECT * FROM prt2_m;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_m t1 RIGHT JOIN uprt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_m t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_m t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+
+--
+-- 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;
+ANALYZE plt1_p1;
+ANALYZE plt1_p2;
+ANALYZE plt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1 AS SELECT * FROM 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;
+ANALYZE plt2_p1;
+ANALYZE plt2_p2;
+ANALYZE plt2_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt2 AS SELECT * FROM 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;
+ANALYZE plt1_e_p1;
+ANALYZE plt1_e_p2;
+ANALYZE plt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1_e AS SELECT * FROM plt1_e;
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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;
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM uplt1 t1, uplt2 t2, uplt1_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;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1, uplt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 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 ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+--
+-- negative testcases
+--
+
+-- joins where one of the relations is proven empty
+EXPLAIN (VERBOSE, 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;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 LEFT JOIN prt2 t2 ON t1.a = t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, 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 % 25 = 0 ORDER BY t1.a, t2.b;
+
+CREATE TABLE prt1_n (a int, b int, c varchar) PARTITION BY RANGE(c);
+CREATE TABLE prt1_n_p1 PARTITION OF prt1_n FOR VALUES FROM ('0000') TO ('0250');
+CREATE TABLE prt1_n_p2 PARTITION OF prt1_n FOR VALUES FROM ('0250') TO ('0500');
+INSERT INTO prt1_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 499, 2) i;
+ANALYZE prt1_n;
+ANALYZE prt1_n_p1;
+ANALYZE prt1_n_p2;
+
+CREATE TABLE prt2_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt2_n_p1 PARTITION OF prt2_n FOR VALUES IN ('0000', '0003', '0004', '0010', '0006', '0007');
+CREATE TABLE prt2_n_p2 PARTITION OF prt2_n FOR VALUES IN ('0001', '0005', '0002', '0009', '0008', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt2_n;
+ANALYZE prt2_n_p1;
+ANALYZE prt2_n_p2;
+
+CREATE TABLE prt3_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt3_n_p1 PARTITION OF prt3_n FOR VALUES IN ('0000', '0004', '0006', '0007');
+CREATE TABLE prt3_n_p2 PARTITION OF prt3_n FOR VALUES IN ('0001', '0002', '0008', '0010');
+CREATE TABLE prt3_n_p3 PARTITION OF prt3_n FOR VALUES IN ('0003', '0005', '0009', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt3_n;
+ANALYZE prt3_n_p1;
+ANALYZE prt3_n_p2;
+ANALYZE prt3_n_p3;
+
+CREATE TABLE prt4_n (a int, b int, c text) PARTITION BY RANGE(a);
+CREATE TABLE prt4_n_p1 PARTITION OF prt4_n FOR VALUES FROM (0) TO (300);
+CREATE TABLE prt4_n_p2 PARTITION OF prt4_n FOR VALUES FROM (300) TO (500);
+CREATE TABLE prt4_n_p3 PARTITION OF prt4_n FOR VALUES FROM (500) TO (600);
+INSERT INTO prt4_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt4_n;
+ANALYZE prt4_n_p1;
+ANALYZE prt4_n_p2;
+ANALYZE prt4_n_p3;
+
+-- partition-wise join can not be applied if the partition ranges differ
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 FULL JOIN prt4_n t2 ON t1.a = t2.a;
+
+-- 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);
+
+-- equi-join with join condition on partial keys does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1, prt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t1.a % 25 = 0;
+
+-- equi-join between out-of-order partition key columns does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.a = t2.b WHERE t1.a % 25 = 0;
+
+-- equi-join between non-key columns does not qualify for partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.c = t2.c WHERE t1.a % 25 = 0;
+
+-- partition-wise join can not be applied for a join between list and range
+-- partitioned table
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1, prt2_n t2 WHERE t1.c = t2.c;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 LEFT JOIN prt2_n t2 ON (t1.c = t2.c);
+
+-- partition-wise join can not be applied between tables with different
+-- partition lists
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 RIGHT JOIN prt1 t2 ON (t1.c = t2.c);
+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);
multi_level_partition_join.patchapplication/x-download; name=multi_level_partition_join.patchDownload
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 98fed55..66537cf 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -2078,7 +2078,6 @@ find_partition_scheme(PlannerInfo *root, Relation relation)
 	int		nparts;
 	int		partnatts;
 	int		cnt_pks;
-	int		cnt_parts;
 	PartitionScheme	part_scheme = NULL;
 
 	/* No partition scheme for an unpartitioned relation. */
@@ -2088,21 +2087,6 @@ find_partition_scheme(PlannerInfo *root, Relation relation)
 	nparts = part_desc->nparts;
 	partnatts = part_key->partnatts;
 
-	/*
-	 * For a multi-level partitioned table, we do not retain the partitioning
-	 * hierarchy while expanding RTE for the topmost parent. Thus the number of
-	 * children as per root->append_rel_list does not match the number of
-	 * partitions specified in the partition descriptor and hence the
-	 * partitioning scheme of a multi-partitioned table does not reflect the
-	 * true picture. So for now, treat a multi-partitioned table as not
-	 * partitioned.
-	 */
-	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
-	{
-		if (has_subclass(part_desc->oids[cnt_parts]))
-			return NULL;
-	}
-
 	/* Search for a matching partition scheme and return if found one. */
 	foreach (lc, root->part_schemes)
 	{
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 0a2c131..b8af23e 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -98,9 +98,6 @@ static void set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 static void generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
 						   List *live_childrels, List *all_child_pathkeys,
 						   bool partition_join_path);
-static Path *get_cheapest_parameterized_child_path(PlannerInfo *root,
-									  RelOptInfo *rel,
-									  Relids required_outer);
 static List *accumulate_append_subpath(List *subpaths, Path *path);
 static void set_subquery_pathlist(PlannerInfo *root, RelOptInfo *rel,
 					  Index rti, RangeTblEntry *rte);
@@ -1646,7 +1643,7 @@ generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
  *
  * Returns NULL if unable to create such a path.
  */
-static Path *
+Path *
 get_cheapest_parameterized_child_path(PlannerInfo *root, RelOptInfo *rel,
 									  Relids required_outer)
 {
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 7839f0f..e7e06c8 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1526,7 +1526,6 @@ add_paths_to_child_joinrel(PlannerInfo *root, RelOptInfo *parent_joinrel,
 											   child_rel1->relids,
 											   child_rel2->relids);
 
-
 		/*
 		 * Construct restrictions applicable to the child join from
 		 * those applicable to the parent join.
@@ -1619,6 +1618,58 @@ adjust_child_relids(Relids relids, List *append_rel_infos)
 }
 
 /*
+ * Replace any relid present in top_parent_relids with its child in
+ * child_relids. Members of child_relids can be multiple levels below top
+ * parent in the partition hierarchy.
+ */
+Relids
+adjust_child_relids_multilevel(PlannerInfo *root, Relids relids,
+							   Relids child_relids, Relids top_parent_relids)
+{
+	List	   *appinfos;
+	Relids		parent_relids = NULL;
+	ListCell   *lc;
+	Relids		result;
+	Relids		tmp_result = NULL;
+
+	/*
+	 * If the given relids set doesn't contain any of the top parent relids,
+	 * it will remain unchanged.
+	 */
+	if (!bms_overlap(relids, top_parent_relids))
+		return relids;
+
+	appinfos = find_appinfos_by_relids(root, child_relids);
+
+	/* Construct relids set for the immediate parent of the given child. */
+	foreach (lc, appinfos)
+	{
+		AppendRelInfo   *appinfo = lfirst(lc);
+
+		parent_relids = bms_add_member(parent_relids, appinfo->parent_relid);
+	}
+
+	/* Recurse if immediate parent is not the top parent. */
+	if (!bms_equal(parent_relids, top_parent_relids))
+	{
+		tmp_result = adjust_child_relids_multilevel(root, relids,
+													parent_relids,
+													top_parent_relids);
+		relids = tmp_result;
+	}
+
+	result = adjust_child_relids(relids, appinfos);
+
+	/* Free memory consumed by any immediate result. */
+	if (tmp_result)
+		bms_free(tmp_result);
+
+	bms_free(parent_relids);
+	list_free(appinfos);
+	return result;
+}
+
+/*
  * Returns true if there exists an equi-join condition for each pair of
  * partition key from given relations being joined.
  */
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 3a223c8..baf0fdf 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -4097,13 +4097,32 @@ create_partition_join_plan(PlannerInfo *root, PartitionJoinPath *best_path)
 		 * Search for a child path with pathkeys or parameterization
 		 * matching that of the given path.
 		 */
-		child_path = get_cheapest_path_for_pathkeys(child_join->pathlist,
-													best_path->path.pathkeys,
-											 PATH_REQ_OUTER(&best_path->path),
-													TOTAL_COST);
+		if (!PATH_REQ_OUTER(&best_path->path))
+			child_path = get_cheapest_path_for_pathkeys(child_join->pathlist,
+													  best_path->path.pathkeys,
+														NULL,
+														TOTAL_COST);
+		else if (best_path->reparamterized_by)
+		{
+			child_path = get_cheapest_parameterized_child_path(root,
+															   child_join,
+													best_path->old_param);
+
+			if (!child_path)
+				elog(ERROR, "Could not find a child-join path with required pathkeys or parameterization.");
+
+			child_path = reparameterize_path_by_child(root, child_path,
+												 best_path->reparamterized_by);
+		}
+		else
+			child_path = get_cheapest_parameterized_child_path(root,
+															   child_join,
+											  PATH_REQ_OUTER(&best_path->path));
 
 		if (!child_path)
-			elog(ERROR, "Could not find a path with required pathkeys.");
+			elog(ERROR, "Could not find a child-join path with required pathkeys or parameterization.");
+
+		MemoryContextSwitchTo(old_context);
 
 		MemoryContextSwitchTo(old_context);
 
@@ -4155,7 +4174,19 @@ create_partition_join_plan(PlannerInfo *root, PartitionJoinPath *best_path)
 			}
 		}
 
-		child_plans = lappend(child_plans, child_plan);
+		/* Flatten Merge/Append plans. */
+		if (IsA(child_plan, Append))
+		{
+			Append	   *append = (Append *) child_plan;
+			child_plans = list_concat(child_plans, append->appendplans);
+		}
+		else if (IsA(child_plan, MergeAppend))
+		{
+			MergeAppend	   *mappend = (MergeAppend *) child_plan;
+			child_plans = list_concat(child_plans, mappend->mergeplans);
+		}
+		else
+			child_plans = lappend(child_plans, child_plan);
 
 		/*
 		 * Reset the child_join memory context to reclaim the memory consumed
diff --git a/src/backend/optimizer/plan/initsplan.c b/src/backend/optimizer/plan/initsplan.c
index 84ce6b3..61f3886 100644
--- a/src/backend/optimizer/plan/initsplan.c
+++ b/src/backend/optimizer/plan/initsplan.c
@@ -14,6 +14,7 @@
  */
 #include "postgres.h"
 
+#include "catalog/pg_class.h"
 #include "catalog/pg_type.h"
 #include "nodes/nodeFuncs.h"
 #include "optimizer/clauses.h"
@@ -623,8 +624,22 @@ create_lateral_join_info(PlannerInfo *root)
 	for (rti = 1; rti < root->simple_rel_array_size; rti++)
 	{
 		RelOptInfo *brel = root->simple_rel_array[rti];
+		RangeTblEntry *rte = root->simple_rte_array[rti];
 
-		if (brel == NULL || brel->reloptkind != RELOPT_BASEREL)
+		if (brel == NULL)
+			continue;
+
+		/*
+		 * If an "other rel" RTE is a "partitioned table", we must propagate
+		 * the lateral info inherited all the way from the root parent to its
+		 * children. That's because the children are not linked directly with
+		 * the root parent via AppendRelInfo's unlike in case of a regular
+		 * inheritance set (see expand_inherited_rtentry()).  Failing to
+		 * do this would result in those children not getting marked with the
+		 * appropriate lateral info.
+		 */
+		if (brel->reloptkind != RELOPT_BASEREL &&
+			rte->relkind != RELKIND_PARTITIONED_TABLE)
 			continue;
 
 		if (root->simple_rte_array[rti]->inh)
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index 8b6a183..ecaffd0 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -110,6 +110,14 @@ static Node *adjust_appendrel_attrs_mutator(Node *node,
 							   adjust_appendrel_attrs_context *context);
 static List *adjust_inherited_tlist(List *tlist,
 					   AppendRelInfo *context);
+static List *expand_inherited_rte_internal(PlannerInfo *root, RangeTblEntry *rte,
+							 Index rti, PlanRowMark *oldrc,
+							 LOCKMODE lockmode, bool flatten);
+static AppendRelInfo *process_one_child_table(PlannerInfo *root,
+						RangeTblEntry *parentRTE, Index parentRTindex,
+						Relation parentrel, Relation childrel,
+						PlanRowMark *parent_rc, bool inh,
+						RangeTblEntry **childRTE, Index *childRTindex);
 
 
 /*
@@ -1323,7 +1331,10 @@ expand_inherited_tables(PlannerInfo *root)
 
 	/*
 	 * expand_inherited_rtentry may add RTEs to parse->rtable; there is no
-	 * need to scan them since they can't have inh=true.  So just scan as far
+	 * need to scan them here since they can't normally have inh=true.  If
+	 * the inheritance set represents a partitioned table, some newly added
+	 * RTEs will break the above rule if they are partitioned tables
+	 * themselves, but they are expanded recursively.  So just scan as far
 	 * as the original end of the rtable list.
 	 */
 	nrtes = list_length(root->parse->rtable);
@@ -1346,9 +1357,11 @@ expand_inherited_tables(PlannerInfo *root)
  *		"inh" flag to prevent later code from looking for AppendRelInfos.
  *
  * Note that the original RTE is considered to represent the whole
- * inheritance set.  The first of the generated RTEs is an RTE for the same
- * table, but with inh = false, to represent the parent table in its role
- * as a simple member of the inheritance set.
+ * inheritance set.  If the RTE represents a partitioned table, inheritance
+ * set is expanded recursively.  The first of the generated RTEs is an RTE
+ * for the same table, but with inh = false, to represent the parent table
+ * in its role as a simple member of the inheritance set.  The same applies
+ * to each individual inheritance set in the recursive expansion case.
  *
  * A childless table is never considered to be an inheritance set; therefore
  * a parent RTE must always have at least two associated AppendRelInfos.
@@ -1359,11 +1372,8 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 	Query	   *parse = root->parse;
 	Oid			parentOID;
 	PlanRowMark *oldrc;
-	Relation	oldrelation;
 	LOCKMODE	lockmode;
-	List	   *inhOIDs;
 	List	   *appinfos;
-	ListCell   *l;
 
 	/* Does RT entry allow inheritance? */
 	if (!rte->inh)
@@ -1404,19 +1414,69 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 	else
 		lockmode = AccessShareLock;
 
-	/* Scan for all members of inheritance set, acquire needed locks */
-	inhOIDs = find_all_inheritors(parentOID, lockmode, NULL);
+	/*
+	 * Do not flatten the inheritance hierarchy if partitioned table, unless
+	 * this is the result relation.
+	 */
+	if (rte->relkind == RELKIND_PARTITIONED_TABLE &&
+		rti != root->parse->resultRelation)
+		appinfos = expand_inherited_rte_internal(root, rte, rti, oldrc,
+												 lockmode, false);
+	else
+		appinfos = expand_inherited_rte_internal(root, rte, rti, oldrc,
+												 lockmode, true);
+
+	/* Add to root->append_rel_list */
+	root->append_rel_list = list_concat(root->append_rel_list, appinfos);
+}
+
+/*
+ * expand_inherited_rte_internal
+ *		Expand an inheritance set in either non-recursive (flatten=true) or
+ *		recursive (flatten=false) manner.
+ *
+ * A inheritance hierarchy is not flttened if it represents a partitioned
+ * table.  This allows later planning steps to apply any partitioning
+ * related optimizations in suitable manner.
+ */
+static List *
+expand_inherited_rte_internal(PlannerInfo *root, RangeTblEntry *rte,
+							  Index rti, PlanRowMark *oldrc,
+							  LOCKMODE lockmode, bool flatten)
+{
+	Oid			parentOID;
+	Relation	oldrelation;
+	List	   *inhOIDs;
+	List	   *appinfos = NIL;
+	ListCell   *l;
+	bool		has_descendents;
+
+	Assert(rte->rtekind == RTE_RELATION);
+	parentOID = rte->relid;
 
 	/*
-	 * Check that there's at least one descendant, else treat as no-child
+	 * Get the list of inheritors.
+	 *
+	 * Also check that there's at least one descendant, else treat as no-child
 	 * case.  This could happen despite above has_subclass() check, if table
 	 * once had a child but no longer does.
 	 */
-	if (list_length(inhOIDs) < 2)
+	if (flatten)
+	{
+		inhOIDs = find_all_inheritors(parentOID, lockmode, NULL);
+		has_descendents = list_length(inhOIDs) >= 2;
+	}
+	else
+	{
+		inhOIDs = find_inheritance_children(parentOID, lockmode);
+		has_descendents = list_length(inhOIDs) >= 1;
+	}
+
+	if (!has_descendents)
 	{
 		/* Clear flag before returning */
 		rte->inh = false;
-		return;
+		return NIL;
 	}
 
 	/*
@@ -1433,15 +1493,24 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 	 */
 	oldrelation = heap_open(parentOID, NoLock);
 
+	/*
+	 * Process parent relation in its role as inheritance set member; remember
+	 * that parent table OID is not in inhOIDs if we did not flatten the
+	 * inheritance tree.
+	 */
+	if (!flatten)
+		appinfos = list_make1(process_one_child_table(root, rte, rti,
+													  oldrelation, oldrelation,
+													  oldrc, false,
+													  NULL, NULL));
+
 	/* Scan the inheritance set and expand it */
-	appinfos = NIL;
 	foreach(l, inhOIDs)
 	{
 		Oid			childOID = lfirst_oid(l);
 		Relation	newrelation;
 		RangeTblEntry *childrte;
 		Index		childRTindex;
-		AppendRelInfo *appinfo;
 
 		/* Open rel if needed; we already have required locks */
 		if (childOID != parentOID)
@@ -1462,75 +1531,29 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 		}
 
 		/*
-		 * Build an RTE for the child, and attach to query's rangetable list.
-		 * We copy most fields of the parent's RTE, but replace relation OID
-		 * and relkind, and set inh = false.  Also, set requiredPerms to zero
-		 * since all required permissions checks are done on the original RTE.
-		 */
-		childrte = copyObject(rte);
-		childrte->relid = childOID;
-		childrte->relkind = newrelation->rd_rel->relkind;
-		childrte->inh = false;
-		childrte->requiredPerms = 0;
-		parse->rtable = lappend(parse->rtable, childrte);
-		childRTindex = list_length(parse->rtable);
-
-		/*
-		 * Build an AppendRelInfo for this parent and child.
-		 */
-		appinfo = makeNode(AppendRelInfo);
-		appinfo->parent_relid = rti;
-		appinfo->child_relid = childRTindex;
-		appinfo->parent_reltype = oldrelation->rd_rel->reltype;
-		appinfo->child_reltype = newrelation->rd_rel->reltype;
-		make_inh_translation_list(oldrelation, newrelation, childRTindex,
-								  &appinfo->translated_vars);
-		appinfo->parent_reloid = parentOID;
-		appinfos = lappend(appinfos, appinfo);
-
-		/*
-		 * Translate the column permissions bitmaps to the child's attnums (we
-		 * have to build the translated_vars list before we can do this). But
-		 * if this is the parent table, leave copyObject's result alone.
+		 * process_one_child_table() performs the following actions for the
+		 * child table:
 		 *
-		 * Note: we need to do this even though the executor won't run any
-		 * permissions checks on the child RTE.  The insertedCols/updatedCols
-		 * bitmaps may be examined for trigger-firing purposes.
-		 */
-		if (childOID != parentOID)
-		{
-			childrte->selectedCols = translate_col_privs(rte->selectedCols,
-												   appinfo->translated_vars);
-			childrte->insertedCols = translate_col_privs(rte->insertedCols,
-												   appinfo->translated_vars);
-			childrte->updatedCols = translate_col_privs(rte->updatedCols,
-												   appinfo->translated_vars);
-		}
-
-		/*
-		 * Build a PlanRowMark if parent is marked FOR UPDATE/SHARE.
+		 * 1. add a new RTE to the query rtable,
+		 * 2. builds a PlanRowMark and adds to the root->rowMarks list
+		 * 3. builds and returns AppendRelInfo for parent-child pair
 		 */
-		if (oldrc)
+		appinfos = lappend(appinfos,
+						   process_one_child_table(root, rte, rti,
+												   oldrelation, newrelation,
+												   oldrc, false,
+												   &childrte, &childRTindex));
+
+		/* Recurse if we did not flatten the inheritance tree */
+		if (!flatten && has_subclass(childOID))
 		{
-			PlanRowMark *newrc = makeNode(PlanRowMark);
-
-			newrc->rti = childRTindex;
-			newrc->prti = rti;
-			newrc->rowmarkId = oldrc->rowmarkId;
-			/* Reselect rowmark type, because relkind might not match parent */
-			newrc->markType = select_rowmark_type(childrte, oldrc->strength);
-			newrc->allMarkTypes = (1 << newrc->markType);
-			newrc->strength = oldrc->strength;
-			newrc->waitPolicy = oldrc->waitPolicy;
-			newrc->isParent = false;
-
-			/* Include child's rowmark type in parent's allMarkTypes */
-			oldrc->allMarkTypes |= newrc->allMarkTypes;
-
-			root->rowMarks = lappend(root->rowMarks, newrc);
+			Assert(childrte->relkind == RELKIND_PARTITIONED_TABLE);
+			childrte->inh = true;
+			appinfos = list_concat(appinfos,
+							   expand_inherited_rte_internal(root, childrte,
+										childRTindex, oldrc, lockmode, flatten));
 		}
 
-		/* Close child relations, but keep locks */
 		if (childOID != parentOID)
 			heap_close(newrelation, NoLock);
 	}
@@ -1546,11 +1569,108 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 	{
 		/* Clear flag before returning */
 		rte->inh = false;
-		return;
+		return NIL;
 	}
+	return appinfos;
+}
 
-	/* Otherwise, OK to add to root->append_rel_list */
-	root->append_rel_list = list_concat(root->append_rel_list, appinfos);
+/*
+ * process_one_child_table
+ *		Process one child table in context of inheritance expansion for a
+ *		query
+ *
+ * *childRTE & *childRTindex are output variables when non-NULL.
+ */
+static AppendRelInfo *
+process_one_child_table(PlannerInfo *root,
+						RangeTblEntry *parentRTE, Index parentRTindex,
+						Relation parentrel, Relation childrel,
+						PlanRowMark *parent_rc, bool inh,
+						RangeTblEntry **childRTE, Index *childRTindex)
+{
+	Query  *parse = root->parse;
+	Oid		parentOID = RelationGetRelid(parentrel),
+			childOID = RelationGetRelid(childrel);
+	RangeTblEntry  *newrte;
+	Index			newrti;
+	AppendRelInfo  *appinfo;
+
+	/*
+	 * Build an RTE for the child, and attach to query's rangetable list.
+	 * We copy most fields of the parent's RTE, but replace relation OID
+	 * and relkind, and set inh as requested.  Also, set requiredPerms to
+	 * zero since all required permissions checks are done on the original
+	 * RTE.
+	 */
+	newrte = copyObject(parentRTE);
+	newrte->relid = RelationGetRelid(childrel);
+	newrte->relkind = childrel->rd_rel->relkind;
+	newrte->inh = inh;
+	newrte->requiredPerms = 0;
+	parse->rtable = lappend(parse->rtable, newrte);
+	newrti = list_length(parse->rtable);
+
+	/* Return the child table RT entry and index if requested */
+	if (childRTE)
+		*childRTE = newrte;
+	if (childRTindex)
+		*childRTindex = newrti;
+
+	/*
+	 * Build an AppendRelInfo for this parent and child.
+	 */
+	appinfo = makeNode(AppendRelInfo);
+	appinfo->parent_relid = parentRTindex;
+	appinfo->child_relid = newrti;
+	appinfo->parent_reltype = parentrel->rd_rel->reltype;
+	appinfo->child_reltype = childrel->rd_rel->reltype;
+	make_inh_translation_list(parentrel, childrel, newrti,
+							  &appinfo->translated_vars);
+	appinfo->parent_reloid = parentOID;
+
+	/*
+	 * Translate the column permissions bitmaps to the child's attnums (we
+	 * have to build the translated_vars list before we can do this). But
+	 * if this is the parent table, leave copyObject's result alone.
+	 *
+	 * Note: we need to do this even though the executor won't run any
+	 * permissions checks on the child RTE.  The insertedCols/updatedCols
+	 * bitmaps may be examined for trigger-firing purposes.
+	 */
+	if (childOID != parentOID)
+	{
+		newrte->selectedCols = translate_col_privs(parentRTE->selectedCols,
+											   appinfo->translated_vars);
+		newrte->insertedCols = translate_col_privs(parentRTE->insertedCols,
+											   appinfo->translated_vars);
+		newrte->updatedCols = translate_col_privs(parentRTE->updatedCols,
+											   appinfo->translated_vars);
+	}
+
+	/*
+	 * Build a PlanRowMark if parent is marked FOR UPDATE/SHARE.
+	 */
+	if (parent_rc)
+	{
+		PlanRowMark *newrc = makeNode(PlanRowMark);
+
+		newrc->rti = newrti;
+		newrc->prti = parentRTindex;
+		newrc->rowmarkId = parent_rc->rowmarkId;
+		/* Reselect rowmark type, because relkind might not match parent */
+		newrc->markType = select_rowmark_type(newrte, parent_rc->strength);
+		newrc->allMarkTypes = (1 << newrc->markType);
+		newrc->strength = parent_rc->strength;
+		newrc->waitPolicy = parent_rc->waitPolicy;
+		newrc->isParent = false;
+
+		/* Include child's rowmark type in parent's allMarkTypes */
+		parent_rc->allMarkTypes |= newrc->allMarkTypes;
+
+		root->rowMarks = lappend(root->rowMarks, newrc);
+	}
+
+	return appinfo;
 }
 
 /*
@@ -2166,23 +2286,47 @@ adjust_inherited_tlist(List *tlist, AppendRelInfo *context)
  * adjust_appendrel_attrs_multilevel
  *	  Apply Var translations from a toplevel appendrel parent down to a child.
  *
- * In some cases we need to translate expressions referencing a baserel
+ * In some cases we need to translate expressions referencing a parent relation
  * to reference an appendrel child that's multiple levels removed from it.
  */
 Node *
 adjust_appendrel_attrs_multilevel(PlannerInfo *root, Node *node,
 								  RelOptInfo *child_rel)
 {
-	AppendRelInfo *appinfo = find_childrel_appendrelinfo(root, child_rel);
-	RelOptInfo *parent_rel = find_base_rel(root, appinfo->parent_relid);
+	List	   *appinfos = find_appinfos_by_relids(root, child_rel->relids);
+	RelOptInfo *parent_rel;
+
+	if (child_rel->reloptkind == RELOPT_OTHER_MEMBER_REL)
+	{
+		AppendRelInfo *appinfo = linitial(appinfos);
+		parent_rel = find_base_rel(root, appinfo->parent_relid);
+	}
+	else
+	{
+		Relids		parent_relids = NULL;
+		ListCell   *lc;
+
+		/* Construct relids set for the immediate parent of the given child. */
+		foreach (lc, appinfos)
+		{
+			AppendRelInfo   *appinfo = lfirst(lc);
+
+			parent_relids = bms_add_member(parent_relids, appinfo->parent_relid);
+		}
+
+		parent_rel = find_join_rel(root, parent_relids);
+		bms_free(parent_relids);
+	}
 
 	/* If parent is also a child, first recurse to apply its translations */
-	if (parent_rel->reloptkind == RELOPT_OTHER_MEMBER_REL)
+	if (IS_OTHER_REL(parent_rel))
 		node = adjust_appendrel_attrs_multilevel(root, node, parent_rel);
 	else
-		Assert(parent_rel->reloptkind == RELOPT_BASEREL);
+		Assert(parent_rel->reloptkind == RELOPT_BASEREL ||
+			   parent_rel->reloptkind == RELOPT_JOINREL);
+
 	/* Now translate for this child */
-	return adjust_appendrel_attrs(root, node, list_make1(appinfo));
+	return adjust_appendrel_attrs(root, node, appinfos);
 }
 
 /*
@@ -2210,7 +2354,6 @@ build_child_restrictinfo(PlannerInfo *root, RestrictInfo *rinfo,
 	child_required_relids = adjust_child_relids(rinfo->required_relids,
 												append_rel_infos);
 
-
 	/* Nothing to do, if the clause does not need any translation. */
 	if (bms_equal(child_required_relids, rinfo->required_relids))
 	{
@@ -2279,3 +2422,50 @@ build_child_clauses(PlannerInfo *root, List *clauses, List *append_rel_infos)
 
 	return child_clauses;
 }
+
+/*
+ * build_child_clauses_multilevel
+ *		Similar to build_child_clauses but used when the child relation
+ *		represented by child_relids is
+ *		deeper down in the partition hierarchy reprepresented by parent
+ *		relation with relids top_parent_relids.
+ */
+List *
+build_child_clauses_multilevel(PlannerInfo *root, List *clauses,
+							   Relids child_relids, Relids top_parent_relids)
+{
+	List	   *appinfos;
+	Relids		parent_relids = NULL;
+	ListCell   *lc;
+	List	   *result;
+	List	   *tmp_result = NIL;
+
+	appinfos = find_appinfos_by_relids(root, child_relids);
+
+	/* Construct relids set for the immediate parent of the given child. */
+	foreach (lc, appinfos)
+	{
+		AppendRelInfo   *appinfo = lfirst(lc);
+
+		parent_relids = bms_add_member(parent_relids, appinfo->parent_relid);
+	}
+
+	/* Recurse if immediate parent is not the top parent. */
+	if (!bms_equal(parent_relids, top_parent_relids))
+	{
+		clauses = build_child_clauses_multilevel(root, clauses, parent_relids,
+												  top_parent_relids);
+		tmp_result = clauses;
+	}
+
+	result = build_child_clauses(root, clauses, appinfos);
+
+	/* Free memory consumed by any itermediate list. */
+	if (tmp_result)
+		list_free(tmp_result);
+
+	bms_free(parent_relids);
+	list_free(appinfos);
+
+	return result;
+}
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index f5010e4..2fa0023 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -3403,7 +3403,6 @@ reparameterize_path_by_child(PlannerInfo *root, Path *path,
 	Path	   *new_path;
 	ParamPathInfo   *new_ppi;
 	ParamPathInfo   *old_ppi;
-	List	   *child_aris;
 	Relids		required_outer;
 
 	/*
@@ -3446,6 +3445,11 @@ reparameterize_path_by_child(PlannerInfo *root, Path *path,
 			memcpy(new_path, path, sizeof(AppendPath));
 			break;
 
+		case T_PartitionJoinPath:
+			new_path = (Path *) makeNode(PartitionJoinPath);
+			memcpy(new_path, path, sizeof(PartitionJoinPath));
+			break;
+
 		/*
 		 * TODO:
 		 * If this method of translation is fine add more path types here.
@@ -3456,16 +3460,12 @@ reparameterize_path_by_child(PlannerInfo *root, Path *path,
 			return NULL;
 	}
 
-	/*
-	 * Gather AppendRelInfos of the base partition relations in the outer child
-	 * relation. We need those for translating parent path to that of child by
-	 * substituting parent Var nodes and relids with those of children.
-	 */
-	child_aris = find_appinfos_by_relids(root, child_rel->relids);
-
 	/* Adjust the parameterization information. */
 	old_ppi = new_path->param_info;
-	required_outer = adjust_child_relids(old_ppi->ppi_req_outer, child_aris);
+	required_outer = adjust_child_relids_multilevel(root,
+													old_ppi->ppi_req_outer,
+													child_rel->relids,
+												 child_rel->top_parent_relids);
 
 	/* If we already have a PPI for this parameterization, just return it */
 	new_ppi = find_param_path_info(new_path->parent, required_outer);
@@ -3476,8 +3476,10 @@ reparameterize_path_by_child(PlannerInfo *root, Path *path,
 		new_ppi = makeNode(ParamPathInfo);
 		new_ppi->ppi_req_outer = required_outer;
 		new_ppi->ppi_rows = old_ppi->ppi_rows;
-		new_ppi->ppi_clauses = build_child_clauses(root, old_ppi->ppi_clauses,
-												   child_aris);
+		new_ppi->ppi_clauses = build_child_clauses_multilevel(root,
+														  old_ppi->ppi_clauses,
+															 child_rel->relids,
+												 child_rel->top_parent_relids);
 		new_path->parent->ppilist = lappend(new_path->parent->ppilist, new_ppi);
 	}
 	else
@@ -3493,6 +3495,7 @@ reparameterize_path_by_child(PlannerInfo *root, Path *path,
 	if (bms_overlap(path->parent->lateral_relids, child_rel->top_parent_relids))
 	{
 		MemoryContext	old_context;
+		List	   *exprs;
 
 		/*
 		 * Allocate the target in planner's context, since they are copies as
@@ -3500,9 +3503,12 @@ reparameterize_path_by_child(PlannerInfo *root, Path *path,
 		 */
 		old_context = MemoryContextSwitchTo(root->planner_cxt);
 		new_path->pathtarget = copy_pathtarget(new_path->pathtarget);
-		new_path->pathtarget->exprs = (List *) adjust_appendrel_attrs(root,
-											(Node *) new_path->pathtarget->exprs,
-																	child_aris);
+		exprs = new_path->pathtarget->exprs;
+		exprs = (List *) adjust_appendrel_attrs_multilevel(root,
+														   (Node *) exprs,
+														   child_rel);
+		new_path->pathtarget->exprs = exprs;
+
 		MemoryContextSwitchTo(old_context);
 	}
 
@@ -3524,9 +3530,10 @@ reparameterize_path_by_child(PlannerInfo *root, Path *path,
 				jpath->innerjoinpath = reparameterize_path_by_child(root,
 														 jpath->innerjoinpath,
 														 child_rel);
-				jpath->joinrestrictinfo = build_child_clauses(root,
-															  jpath->joinrestrictinfo,
-															  child_aris);
+				jpath->joinrestrictinfo = build_child_clauses_multilevel(root,
+													  jpath->joinrestrictinfo,
+															child_rel->relids,
+												child_rel->top_parent_relids);
 			}
 			break;
 
@@ -3543,16 +3550,38 @@ reparameterize_path_by_child(PlannerInfo *root, Path *path,
 																	child_rel));
 				apath->subpaths = subpaths;
 			}
+			break;
 
 		case T_IndexPath:
 			{
 				IndexPath *ipath = (IndexPath *)new_path;
 
-				ipath->indexclauses = build_child_clauses(root, ipath->indexclauses,
-														  child_aris);
-				ipath->indexquals = build_child_clauses(root, ipath->indexquals,
-														child_aris);
+				ipath->indexclauses = build_child_clauses_multilevel(root,
+														   ipath->indexclauses,
+															 child_rel->relids,
+												 child_rel->top_parent_relids);
+
+				ipath->indexquals = build_child_clauses_multilevel(root,
+															 ipath->indexquals,
+															 child_rel->relids,
+												 child_rel->top_parent_relids);
 			}
+			break;
+
+		case T_PartitionJoinPath:
+			{
+				PartitionJoinPath *pjpath = (PartitionJoinPath *)new_path;	
+
+				/*
+				 * The paths for the child-joins covered by PartitionJoinPath will
+				 * be created later. If this PartitionJoinPath was reparameterized
+				 * by child of another relations, we will need that relation to
+				 * reparamterize the child-join paths.
+				 */
+				pjpath->old_param = PATH_REQ_OUTER(path); 
+				pjpath->reparamterized_by = child_rel;
+			}
+			break;
 
 		default:
 			/* Nothing to do. */
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index 88e66e4..83074e0 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -1335,8 +1335,13 @@ relation_excluded_by_constraints(PlannerInfo *root,
 	if (predicate_refuted_by(safe_restrictions, safe_restrictions))
 		return true;
 
-	/* Only plain relations have constraints */
-	if (rte->rtekind != RTE_RELATION || rte->inh)
+	/*
+	 * Only plain relations have constraints.  We represent a partitioned
+	 * table append member as its own append relation and hence would have
+	 * set rte->inh in that case.
+	 */
+	if (rte->rtekind != RTE_RELATION ||
+		(rte->inh && rte->relkind != RELKIND_PARTITIONED_TABLE))
 		return false;
 
 	/*
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index df5b60f..6bac09d 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -1515,6 +1515,12 @@ typedef struct LimitPath
 typedef struct PartitionJoinPath
 {
 	Path		path;
+
+	/* If this path is result of reparameterization by child. */
+	Relids		old_param;			/* Older parameterization. */
+	RelOptInfo *reparamterized_by;	/* Child relation by which this paths was
+									 * reparamterized.
+									 */
 } PartitionJoinPath;
 
 /*
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 5d7bcd9..361c081 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -55,6 +55,8 @@ extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 extern void generate_gather_paths(PlannerInfo *root, RelOptInfo *rel);
 extern void generate_partition_wise_join_paths(PlannerInfo *root,
 											   RelOptInfo *rel);
+extern Path *get_cheapest_parameterized_child_path(PlannerInfo *root,
+									   RelOptInfo *rel, Relids required_outer);
 
 #ifdef OPTIMIZER_DEBUG
 extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
@@ -225,5 +227,8 @@ extern PathKey *make_canonical_pathkey(PlannerInfo *root,
 					   int strategy, bool nulls_first);
 
 extern Relids adjust_child_relids(Relids relids, List *append_rel_infos);
+extern Relids adjust_child_relids_multilevel(PlannerInfo *root, Relids relids,
+											 Relids child_relids,
+											 Relids top_parent_relids);
 
 #endif   /* PATHS_H */
diff --git a/src/include/optimizer/prep.h b/src/include/optimizer/prep.h
index 2483303..51c9057 100644
--- a/src/include/optimizer/prep.h
+++ b/src/include/optimizer/prep.h
@@ -69,5 +69,7 @@ extern RestrictInfo *build_child_restrictinfo(PlannerInfo *root,
 								 RestrictInfo *rinfo, List *append_rel_infos);
 extern List *build_child_clauses(PlannerInfo *root, List *clauses,
 								 List *append_rel_infos);
+extern List *build_child_clauses_multilevel(PlannerInfo *root, List *clauses,
+								Relids child_relids, Relids top_parent_relids);
 
 #endif   /* PREP_H */
diff --git a/src/test/regress/expected/inherit.out b/src/test/regress/expected/inherit.out
index 38ea8e8..3e6dd49 100644
--- a/src/test/regress/expected/inherit.out
+++ b/src/test/regress/expected/inherit.out
@@ -1645,15 +1645,15 @@ explain (costs off) select * from range_list_parted;
  Append
    ->  Seq Scan on range_list_parted
    ->  Seq Scan on part_1_10
-   ->  Seq Scan on part_10_20
-   ->  Seq Scan on part_21_30
-   ->  Seq Scan on part_40_inf
    ->  Seq Scan on part_1_10_ab
    ->  Seq Scan on part_1_10_cd
+   ->  Seq Scan on part_10_20
    ->  Seq Scan on part_10_20_ab
    ->  Seq Scan on part_10_20_cd
+   ->  Seq Scan on part_21_30
    ->  Seq Scan on part_21_30_ab
    ->  Seq Scan on part_21_30_cd
+   ->  Seq Scan on part_40_inf
    ->  Seq Scan on part_40_inf_ab
    ->  Seq Scan on part_40_inf_cd
    ->  Seq Scan on part_40_inf_null
@@ -1681,18 +1681,18 @@ explain (costs off) select * from range_list_parted where b = 'ab';
          Filter: (b = 'ab'::bpchar)
    ->  Seq Scan on part_1_10
          Filter: (b = 'ab'::bpchar)
-   ->  Seq Scan on part_10_20
-         Filter: (b = 'ab'::bpchar)
-   ->  Seq Scan on part_21_30
-         Filter: (b = 'ab'::bpchar)
-   ->  Seq Scan on part_40_inf
-         Filter: (b = 'ab'::bpchar)
    ->  Seq Scan on part_1_10_ab
          Filter: (b = 'ab'::bpchar)
+   ->  Seq Scan on part_10_20
+         Filter: (b = 'ab'::bpchar)
    ->  Seq Scan on part_10_20_ab
          Filter: (b = 'ab'::bpchar)
+   ->  Seq Scan on part_21_30
+         Filter: (b = 'ab'::bpchar)
    ->  Seq Scan on part_21_30_ab
          Filter: (b = 'ab'::bpchar)
+   ->  Seq Scan on part_40_inf
+         Filter: (b = 'ab'::bpchar)
    ->  Seq Scan on part_40_inf_ab
          Filter: (b = 'ab'::bpchar)
 (19 rows)
@@ -1705,14 +1705,14 @@ explain (costs off) select * from range_list_parted where a between 3 and 23 and
          Filter: ((a >= 3) AND (a <= 23) AND (b = 'ab'::bpchar))
    ->  Seq Scan on part_1_10
          Filter: ((a >= 3) AND (a <= 23) AND (b = 'ab'::bpchar))
-   ->  Seq Scan on part_10_20
-         Filter: ((a >= 3) AND (a <= 23) AND (b = 'ab'::bpchar))
-   ->  Seq Scan on part_21_30
-         Filter: ((a >= 3) AND (a <= 23) AND (b = 'ab'::bpchar))
    ->  Seq Scan on part_1_10_ab
          Filter: ((a >= 3) AND (a <= 23) AND (b = 'ab'::bpchar))
+   ->  Seq Scan on part_10_20
+         Filter: ((a >= 3) AND (a <= 23) AND (b = 'ab'::bpchar))
    ->  Seq Scan on part_10_20_ab
          Filter: ((a >= 3) AND (a <= 23) AND (b = 'ab'::bpchar))
+   ->  Seq Scan on part_21_30
+         Filter: ((a >= 3) AND (a <= 23) AND (b = 'ab'::bpchar))
    ->  Seq Scan on part_21_30_ab
          Filter: ((a >= 3) AND (a <= 23) AND (b = 'ab'::bpchar))
 (15 rows)
@@ -1752,24 +1752,24 @@ explain (costs off) select * from range_list_parted where a is not null and a <
          Filter: ((a IS NOT NULL) AND (a < 67))
    ->  Seq Scan on part_1_10
          Filter: ((a IS NOT NULL) AND (a < 67))
-   ->  Seq Scan on part_10_20
-         Filter: ((a IS NOT NULL) AND (a < 67))
-   ->  Seq Scan on part_21_30
-         Filter: ((a IS NOT NULL) AND (a < 67))
-   ->  Seq Scan on part_40_inf
-         Filter: ((a IS NOT NULL) AND (a < 67))
    ->  Seq Scan on part_1_10_ab
          Filter: ((a IS NOT NULL) AND (a < 67))
    ->  Seq Scan on part_1_10_cd
          Filter: ((a IS NOT NULL) AND (a < 67))
+   ->  Seq Scan on part_10_20
+         Filter: ((a IS NOT NULL) AND (a < 67))
    ->  Seq Scan on part_10_20_ab
          Filter: ((a IS NOT NULL) AND (a < 67))
    ->  Seq Scan on part_10_20_cd
          Filter: ((a IS NOT NULL) AND (a < 67))
+   ->  Seq Scan on part_21_30
+         Filter: ((a IS NOT NULL) AND (a < 67))
    ->  Seq Scan on part_21_30_ab
          Filter: ((a IS NOT NULL) AND (a < 67))
    ->  Seq Scan on part_21_30_cd
          Filter: ((a IS NOT NULL) AND (a < 67))
+   ->  Seq Scan on part_40_inf
+         Filter: ((a IS NOT NULL) AND (a < 67))
    ->  Seq Scan on part_40_inf_ab
          Filter: ((a IS NOT NULL) AND (a < 67))
    ->  Seq Scan on part_40_inf_cd
diff --git a/src/test/regress/expected/multi_level_partition_join.out b/src/test/regress/expected/multi_level_partition_join.out
index d40ae55..97b1e79 100644
--- a/src/test/regress/expected/multi_level_partition_join.out
+++ b/src/test/regress/expected/multi_level_partition_join.out
@@ -1,21 +1,19 @@
+-- Encourage partition-wise join plans.
+SET partition_wise_plan_weight TO 0.1;
 --
 -- multi-leveled partitions
 --
 CREATE TABLE prt1_l (a int, b int, c varchar) PARTITION BY RANGE(a);
-CREATE TABLE prt1_l_p1 PARTITION OF prt1_l FOR VALUES FROM (0) TO (250) PARTITION BY RANGE (b);
-CREATE TABLE prt1_l_p1_p1 PARTITION OF prt1_l_p1 FOR VALUES FROM (0) TO (100);
-CREATE TABLE prt1_l_p1_p2 PARTITION OF prt1_l_p1 FOR VALUES FROM (100) TO (250);
+CREATE TABLE prt1_l_p1 PARTITION OF prt1_l FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_l_p2 PARTITION OF prt1_l FOR VALUES FROM (250) TO (500) PARTITION BY RANGE (c);
 CREATE TABLE prt1_l_p2_p1 PARTITION OF prt1_l_p2 FOR VALUES FROM ('0250') TO ('0400');
 CREATE TABLE prt1_l_p2_p2 PARTITION OF prt1_l_p2 FOR VALUES FROM ('0400') TO ('0500');
-CREATE TABLE prt1_l_p3 PARTITION OF prt1_l FOR VALUES FROM (500) TO (600) PARTITION BY RANGE ((b + a));
-CREATE TABLE prt1_l_p3_p1 PARTITION OF prt1_l_p3 FOR VALUES FROM (1000) TO (1100);
-CREATE TABLE prt1_l_p3_p2 PARTITION OF prt1_l_p3 FOR VALUES FROM (1100) TO (1200);
+CREATE TABLE prt1_l_p3 PARTITION OF prt1_l FOR VALUES FROM (500) TO (600) PARTITION BY RANGE (b);
+CREATE TABLE prt1_l_p3_p1 PARTITION OF prt1_l_p3 FOR VALUES FROM (500) TO (550);
+CREATE TABLE prt1_l_p3_p2 PARTITION OF prt1_l_p3 FOR VALUES FROM (550) TO (600);
 INSERT INTO prt1_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
 ANALYZE prt1_l;
 ANALYZE prt1_l_p1;
-ANALYZE prt1_l_p1_p1;
-ANALYZE prt1_l_p1_p2;
 ANALYZE prt1_l_p2;
 ANALYZE prt1_l_p2_p1;
 ANALYZE prt1_l_p2_p2;
@@ -26,20 +24,16 @@ ANALYZE prt1_l_p3_p2;
 -- results are tested.
 CREATE TABLE uprt1_l AS SELECT * FROM prt1_l;
 CREATE TABLE prt2_l (a int, b int, c varchar) PARTITION BY RANGE(b);
-CREATE TABLE prt2_l_p1 PARTITION OF prt2_l FOR VALUES FROM (0) TO (250) PARTITION BY RANGE (a);
-CREATE TABLE prt2_l_p1_p1 PARTITION OF prt2_l_p1 FOR VALUES FROM (0) TO (100);
-CREATE TABLE prt2_l_p1_p2 PARTITION OF prt2_l_p1 FOR VALUES FROM (100) TO (250);
+CREATE TABLE prt2_l_p1 PARTITION OF prt2_l FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_l_p2 PARTITION OF prt2_l FOR VALUES FROM (250) TO (500) PARTITION BY RANGE (c);
 CREATE TABLE prt2_l_p2_p1 PARTITION OF prt2_l_p2 FOR VALUES FROM ('0250') TO ('0400');
 CREATE TABLE prt2_l_p2_p2 PARTITION OF prt2_l_p2 FOR VALUES FROM ('0400') TO ('0500');
-CREATE TABLE prt2_l_p3 PARTITION OF prt2_l FOR VALUES FROM (500) TO (600) PARTITION BY RANGE ((a + b));
-CREATE TABLE prt2_l_p3_p1 PARTITION OF prt2_l_p3 FOR VALUES FROM (1000) TO (1100);
-CREATE TABLE prt2_l_p3_p2 PARTITION OF prt2_l_p3 FOR VALUES FROM (1100) TO (1200);
+CREATE TABLE prt2_l_p3 PARTITION OF prt2_l FOR VALUES FROM (500) TO (600) PARTITION BY RANGE (a);
+CREATE TABLE prt2_l_p3_p1 PARTITION OF prt2_l_p3 FOR VALUES FROM (500) TO (525);
+CREATE TABLE prt2_l_p3_p2 PARTITION OF prt2_l_p3 FOR VALUES FROM (525) TO (600);
 INSERT INTO prt2_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
 ANALYZE prt2_l;
 ANALYZE prt2_l_p1;
-ANALYZE prt2_l_p1_p1;
-ANALYZE prt2_l_p1_p2;
 ANALYZE prt2_l_p2;
 ANALYZE prt2_l_p2_p1;
 ANALYZE prt2_l_p2_p2;
@@ -49,77 +43,70 @@ ANALYZE prt2_l_p3_p2;
 -- TODO: This table is created only for testing the results. Remove once
 -- results are tested.
 CREATE TABLE uprt2_l AS SELECT * FROM prt2_l;
+-- inner join
 EXPLAIN (VERBOSE, 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 = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
-                                                                      QUERY PLAN                                                                      
-------------------------------------------------------------------------------------------------------------------------------------------------------
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1, prt2_l t2 WHERE t1.a = t2.b AND t1.c = t2.c AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
  Sort
    Output: t1.a, t1.c, t2.b, t2.c
    Sort Key: t1.a
    ->  Append
          ->  Hash Join
                Output: t1.a, t1.c, t2.b, t2.c
-               Hash Cond: ((t2.b = t1.a) AND (t2.a = t1.b) AND ((t2.c)::text = (t1.c)::text) AND ((t2.a + t2.b) = (t1.b + t1.a)))
-               ->  Seq Scan on public.prt2_l_p1_p1 t2
-                     Output: t2.b, t2.c, t2.a
+               Hash Cond: ((t2.b = t1.a) AND ((t2.c)::text = (t1.c)::text))
+               ->  Seq Scan on public.prt2_l_p1 t2
+                     Output: t2.b, t2.c
                ->  Hash
-                     Output: t1.a, t1.c, t1.b
-                     ->  Seq Scan on public.prt1_l_p1_p1 t1
-                           Output: t1.a, t1.c, t1.b
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.prt1_l_p1 t1
+                           Output: t1.a, t1.c
                            Filter: ((t1.a % 25) = 0)
          ->  Hash Join
                Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
-               Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.a = t1_1.b) AND ((t2_1.c)::text = (t1_1.c)::text) AND ((t2_1.a + t2_1.b) = (t1_1.b + t1_1.a)))
-               ->  Seq Scan on public.prt2_l_p1_p2 t2_1
-                     Output: t2_1.b, t2_1.c, t2_1.a
+               Hash Cond: ((t2_1.b = t1_1.a) AND ((t2_1.c)::text = (t1_1.c)::text))
+               ->  Seq Scan on public.prt2_l_p2_p1 t2_1
+                     Output: t2_1.b, t2_1.c
                ->  Hash
-                     Output: t1_1.a, t1_1.c, t1_1.b
-                     ->  Seq Scan on public.prt1_l_p1_p2 t1_1
-                           Output: t1_1.a, t1_1.c, t1_1.b
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.prt1_l_p2_p1 t1_1
+                           Output: t1_1.a, t1_1.c
                            Filter: ((t1_1.a % 25) = 0)
          ->  Hash Join
                Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
-               Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.a = t1_2.b) AND ((t2_2.c)::text = (t1_2.c)::text) AND ((t2_2.a + t2_2.b) = (t1_2.b + t1_2.a)))
-               ->  Seq Scan on public.prt2_l_p2_p1 t2_2
-                     Output: t2_2.b, t2_2.c, t2_2.a
+               Hash Cond: ((t2_2.b = t1_2.a) AND ((t2_2.c)::text = (t1_2.c)::text))
+               ->  Seq Scan on public.prt2_l_p2_p2 t2_2
+                     Output: t2_2.b, t2_2.c
                ->  Hash
-                     Output: t1_2.a, t1_2.c, t1_2.b
-                     ->  Seq Scan on public.prt1_l_p2_p1 t1_2
-                           Output: t1_2.a, t1_2.c, t1_2.b
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.prt1_l_p2_p2 t1_2
+                           Output: t1_2.a, t1_2.c
                            Filter: ((t1_2.a % 25) = 0)
          ->  Hash Join
                Output: t1_3.a, t1_3.c, t2_3.b, t2_3.c
-               Hash Cond: ((t2_3.b = t1_3.a) AND (t2_3.a = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text) AND ((t2_3.a + t2_3.b) = (t1_3.b + t1_3.a)))
-               ->  Seq Scan on public.prt2_l_p2_p2 t2_3
-                     Output: t2_3.b, t2_3.c, t2_3.a
+               Hash Cond: ((t2_3.b = t1_3.a) AND ((t2_3.c)::text = (t1_3.c)::text))
+               ->  Append
+                     ->  Seq Scan on public.prt2_l_p3 t2_3
+                           Output: t2_3.b, t2_3.c
+                     ->  Seq Scan on public.prt2_l_p3_p1 t2_4
+                           Output: t2_4.b, t2_4.c
+                     ->  Seq Scan on public.prt2_l_p3_p2 t2_5
+                           Output: t2_5.b, t2_5.c
                ->  Hash
-                     Output: t1_3.a, t1_3.c, t1_3.b
-                     ->  Seq Scan on public.prt1_l_p2_p2 t1_3
-                           Output: t1_3.a, t1_3.c, t1_3.b
-                           Filter: ((t1_3.a % 25) = 0)
-         ->  Hash Join
-               Output: t1_4.a, t1_4.c, t2_4.b, t2_4.c
-               Hash Cond: ((t2_4.b = t1_4.a) AND (t2_4.a = t1_4.b) AND ((t2_4.c)::text = (t1_4.c)::text) AND ((t2_4.a + t2_4.b) = (t1_4.b + t1_4.a)))
-               ->  Seq Scan on public.prt2_l_p3_p1 t2_4
-                     Output: t2_4.b, t2_4.c, t2_4.a
-               ->  Hash
-                     Output: t1_4.a, t1_4.c, t1_4.b
-                     ->  Seq Scan on public.prt1_l_p3_p1 t1_4
-                           Output: t1_4.a, t1_4.c, t1_4.b
-                           Filter: ((t1_4.a % 25) = 0)
-         ->  Hash Join
-               Output: t1_5.a, t1_5.c, t2_5.b, t2_5.c
-               Hash Cond: ((t2_5.b = t1_5.a) AND (t2_5.a = t1_5.b) AND ((t2_5.c)::text = (t1_5.c)::text) AND ((t2_5.a + t2_5.b) = (t1_5.b + t1_5.a)))
-               ->  Seq Scan on public.prt2_l_p3_p2 t2_5
-                     Output: t2_5.b, t2_5.c, t2_5.a
-               ->  Hash
-                     Output: t1_5.a, t1_5.c, t1_5.b
-                     ->  Seq Scan on public.prt1_l_p3_p2 t1_5
-                           Output: t1_5.a, t1_5.c, t1_5.b
-                           Filter: ((t1_5.a % 25) = 0)
-(64 rows)
+                     Output: t1_3.a, t1_3.c
+                     ->  Append
+                           ->  Seq Scan on public.prt1_l_p3 t1_3
+                                 Output: t1_3.a, t1_3.c
+                                 Filter: ((t1_3.a % 25) = 0)
+                           ->  Seq Scan on public.prt1_l_p3_p1 t1_4
+                                 Output: t1_4.a, t1_4.c
+                                 Filter: ((t1_4.a % 25) = 0)
+                           ->  Seq Scan on public.prt1_l_p3_p2 t1_5
+                                 Output: t1_5.a, t1_5.c
+                                 Filter: ((t1_5.a % 25) = 0)
+(56 rows)
 
-SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1, prt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 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.c = t2.c AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
   a  |  c   |  b  |  c   
 -----+------+-----+------
    0 | 0000 |   0 | 0000
@@ -128,7 +115,7 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1, prt2_l t2 WHERE t1.a = t2.b AND t1
  450 | 0450 | 450 | 0450
 (4 rows)
 
-SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1, uprt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1, uprt2_l t2 WHERE t1.a = t2.b AND t1.c = t2.c AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
   a  |  c   |  b  |  c   
 -----+------+-----+------
    0 | 0000 |   0 | 0000
@@ -137,77 +124,70 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1, uprt2_l t2 WHERE t1.a = t2.b AND
  450 | 0450 | 450 | 0450
 (4 rows)
 
+-- left join
 EXPLAIN (VERBOSE, 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
-                                                                      QUERY PLAN                                                                      
-------------------------------------------------------------------------------------------------------------------------------------------------------
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
  Sort
    Output: t1.a, t1.c, t2.b, t2.c
    Sort Key: t1.a, t2.b
    ->  Append
          ->  Hash Right Join
                Output: t1.a, t1.c, t2.b, t2.c
-               Hash Cond: ((t2.b = t1.a) AND (t2.a = t1.b) AND ((t2.c)::text = (t1.c)::text) AND ((t2.a + t2.b) = (t1.b + t1.a)))
-               ->  Seq Scan on public.prt2_l_p1_p1 t2
-                     Output: t2.b, t2.c, t2.a
+               Hash Cond: ((t2.b = t1.a) AND ((t2.c)::text = (t1.c)::text))
+               ->  Seq Scan on public.prt2_l_p1 t2
+                     Output: t2.b, t2.c
                ->  Hash
-                     Output: t1.a, t1.c, t1.b
-                     ->  Seq Scan on public.prt1_l_p1_p1 t1
-                           Output: t1.a, t1.c, t1.b
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.prt1_l_p1 t1
+                           Output: t1.a, t1.c
                            Filter: ((t1.a % 25) = 0)
          ->  Hash Right Join
                Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
-               Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.a = t1_1.b) AND ((t2_1.c)::text = (t1_1.c)::text) AND ((t2_1.a + t2_1.b) = (t1_1.b + t1_1.a)))
-               ->  Seq Scan on public.prt2_l_p1_p2 t2_1
-                     Output: t2_1.b, t2_1.c, t2_1.a
+               Hash Cond: ((t2_1.b = t1_1.a) AND ((t2_1.c)::text = (t1_1.c)::text))
+               ->  Seq Scan on public.prt2_l_p2_p1 t2_1
+                     Output: t2_1.b, t2_1.c
                ->  Hash
-                     Output: t1_1.a, t1_1.c, t1_1.b
-                     ->  Seq Scan on public.prt1_l_p1_p2 t1_1
-                           Output: t1_1.a, t1_1.c, t1_1.b
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.prt1_l_p2_p1 t1_1
+                           Output: t1_1.a, t1_1.c
                            Filter: ((t1_1.a % 25) = 0)
          ->  Hash Right Join
                Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
-               Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.a = t1_2.b) AND ((t2_2.c)::text = (t1_2.c)::text) AND ((t2_2.a + t2_2.b) = (t1_2.b + t1_2.a)))
-               ->  Seq Scan on public.prt2_l_p2_p1 t2_2
-                     Output: t2_2.b, t2_2.c, t2_2.a
+               Hash Cond: ((t2_2.b = t1_2.a) AND ((t2_2.c)::text = (t1_2.c)::text))
+               ->  Seq Scan on public.prt2_l_p2_p2 t2_2
+                     Output: t2_2.b, t2_2.c
                ->  Hash
-                     Output: t1_2.a, t1_2.c, t1_2.b
-                     ->  Seq Scan on public.prt1_l_p2_p1 t1_2
-                           Output: t1_2.a, t1_2.c, t1_2.b
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.prt1_l_p2_p2 t1_2
+                           Output: t1_2.a, t1_2.c
                            Filter: ((t1_2.a % 25) = 0)
          ->  Hash Right Join
                Output: t1_3.a, t1_3.c, t2_3.b, t2_3.c
-               Hash Cond: ((t2_3.b = t1_3.a) AND (t2_3.a = t1_3.b) AND ((t2_3.c)::text = (t1_3.c)::text) AND ((t2_3.a + t2_3.b) = (t1_3.b + t1_3.a)))
-               ->  Seq Scan on public.prt2_l_p2_p2 t2_3
-                     Output: t2_3.b, t2_3.c, t2_3.a
-               ->  Hash
-                     Output: t1_3.a, t1_3.c, t1_3.b
-                     ->  Seq Scan on public.prt1_l_p2_p2 t1_3
-                           Output: t1_3.a, t1_3.c, t1_3.b
-                           Filter: ((t1_3.a % 25) = 0)
-         ->  Hash Right Join
-               Output: t1_4.a, t1_4.c, t2_4.b, t2_4.c
-               Hash Cond: ((t2_4.b = t1_4.a) AND (t2_4.a = t1_4.b) AND ((t2_4.c)::text = (t1_4.c)::text) AND ((t2_4.a + t2_4.b) = (t1_4.b + t1_4.a)))
-               ->  Seq Scan on public.prt2_l_p3_p1 t2_4
-                     Output: t2_4.b, t2_4.c, t2_4.a
+               Hash Cond: ((t2_3.b = t1_3.a) AND ((t2_3.c)::text = (t1_3.c)::text))
+               ->  Append
+                     ->  Seq Scan on public.prt2_l_p3 t2_3
+                           Output: t2_3.b, t2_3.c
+                     ->  Seq Scan on public.prt2_l_p3_p1 t2_4
+                           Output: t2_4.b, t2_4.c
+                     ->  Seq Scan on public.prt2_l_p3_p2 t2_5
+                           Output: t2_5.b, t2_5.c
                ->  Hash
-                     Output: t1_4.a, t1_4.c, t1_4.b
-                     ->  Seq Scan on public.prt1_l_p3_p1 t1_4
-                           Output: t1_4.a, t1_4.c, t1_4.b
-                           Filter: ((t1_4.a % 25) = 0)
-         ->  Hash Right Join
-               Output: t1_5.a, t1_5.c, t2_5.b, t2_5.c
-               Hash Cond: ((t2_5.b = t1_5.a) AND (t2_5.a = t1_5.b) AND ((t2_5.c)::text = (t1_5.c)::text) AND ((t2_5.a + t2_5.b) = (t1_5.b + t1_5.a)))
-               ->  Seq Scan on public.prt2_l_p3_p2 t2_5
-                     Output: t2_5.b, t2_5.c, t2_5.a
-               ->  Hash
-                     Output: t1_5.a, t1_5.c, t1_5.b
-                     ->  Seq Scan on public.prt1_l_p3_p2 t1_5
-                           Output: t1_5.a, t1_5.c, t1_5.b
-                           Filter: ((t1_5.a % 25) = 0)
-(64 rows)
+                     Output: t1_3.a, t1_3.c
+                     ->  Append
+                           ->  Seq Scan on public.prt1_l_p3 t1_3
+                                 Output: t1_3.a, t1_3.c
+                                 Filter: ((t1_3.a % 25) = 0)
+                           ->  Seq Scan on public.prt1_l_p3_p1 t1_4
+                                 Output: t1_4.a, t1_4.c
+                                 Filter: ((t1_4.a % 25) = 0)
+                           ->  Seq Scan on public.prt1_l_p3_p2 t1_5
+                                 Output: t1_5.a, t1_5.c
+                                 Filter: ((t1_5.a % 25) = 0)
+(56 rows)
 
-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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 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.a % 25 = 0 ORDER BY t1.a, t2.b;
   a  |  c   |  b  |  c   
 -----+------+-----+------
    0 | 0000 |   0 | 0000
@@ -224,7 +204,7 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1 LEFT JOIN prt2_l t2 ON t1.a = t2.b
  550 | 0550 |     | 
 (12 rows)
 
-SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 LEFT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 LEFT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
   a  |  c   |  b  |  c   
 -----+------+-----+------
    0 | 0000 |   0 | 0000
@@ -241,10 +221,11 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 LEFT JOIN uprt2_l t2 ON t1.a = t2.
  550 | 0550 |     | 
 (12 rows)
 
+-- right join
 EXPLAIN (VERBOSE, 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
-                                                                         QUERY PLAN                                                                         
-------------------------------------------------------------------------------------------------------------------------------------------------------------
+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.b % 25 = 0 ORDER BY t1.a, t2.b;
+                                        QUERY PLAN                                        
+------------------------------------------------------------------------------------------
  Sort
    Output: t1.a, t1.c, t2.b, t2.c
    Sort Key: t1.a, t2.b
@@ -253,67 +234,59 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1 RIGHT JOIN prt2_l t2 ON t1.a = t2.b
          ->  Append
                ->  Hash Right Join
                      Output: t2.b, t2.c, t1.a, t1.c
-                     Hash Cond: ((t1.a = t2.b) AND (t1.b = t2.a) AND ((t1.c)::text = (t2.c)::text) AND ((t1.b + t1.a) = (t2.a + t2.b)))
-                     ->  Seq Scan on public.prt1_l_p1_p1 t1
-                           Output: t1.a, t1.c, t1.b
+                     Hash Cond: ((t1.a = t2.b) AND ((t1.c)::text = (t2.c)::text))
+                     ->  Seq Scan on public.prt1_l_p1 t1
+                           Output: t1.a, t1.c
                      ->  Hash
-                           Output: t2.b, t2.c, t2.a
-                           ->  Seq Scan on public.prt2_l_p1_p1 t2
-                                 Output: t2.b, t2.c, t2.a
+                           Output: t2.b, t2.c
+                           ->  Seq Scan on public.prt2_l_p1 t2
+                                 Output: t2.b, t2.c
                                  Filter: ((t2.b % 25) = 0)
                ->  Hash Right Join
                      Output: t2_1.b, t2_1.c, t1_1.a, t1_1.c
-                     Hash Cond: ((t1_1.a = t2_1.b) AND (t1_1.b = t2_1.a) AND ((t1_1.c)::text = (t2_1.c)::text) AND ((t1_1.b + t1_1.a) = (t2_1.a + t2_1.b)))
-                     ->  Seq Scan on public.prt1_l_p1_p2 t1_1
-                           Output: t1_1.a, t1_1.c, t1_1.b
+                     Hash Cond: ((t1_1.a = t2_1.b) AND ((t1_1.c)::text = (t2_1.c)::text))
+                     ->  Seq Scan on public.prt1_l_p2_p1 t1_1
+                           Output: t1_1.a, t1_1.c
                      ->  Hash
-                           Output: t2_1.b, t2_1.c, t2_1.a
-                           ->  Seq Scan on public.prt2_l_p1_p2 t2_1
-                                 Output: t2_1.b, t2_1.c, t2_1.a
+                           Output: t2_1.b, t2_1.c
+                           ->  Seq Scan on public.prt2_l_p2_p1 t2_1
+                                 Output: t2_1.b, t2_1.c
                                  Filter: ((t2_1.b % 25) = 0)
                ->  Hash Right Join
                      Output: t2_2.b, t2_2.c, t1_2.a, t1_2.c
-                     Hash Cond: ((t1_2.a = t2_2.b) AND (t1_2.b = t2_2.a) AND ((t1_2.c)::text = (t2_2.c)::text) AND ((t1_2.b + t1_2.a) = (t2_2.a + t2_2.b)))
-                     ->  Seq Scan on public.prt1_l_p2_p1 t1_2
-                           Output: t1_2.a, t1_2.c, t1_2.b
+                     Hash Cond: ((t1_2.a = t2_2.b) AND ((t1_2.c)::text = (t2_2.c)::text))
+                     ->  Seq Scan on public.prt1_l_p2_p2 t1_2
+                           Output: t1_2.a, t1_2.c
                      ->  Hash
-                           Output: t2_2.b, t2_2.c, t2_2.a
-                           ->  Seq Scan on public.prt2_l_p2_p1 t2_2
-                                 Output: t2_2.b, t2_2.c, t2_2.a
+                           Output: t2_2.b, t2_2.c
+                           ->  Seq Scan on public.prt2_l_p2_p2 t2_2
+                                 Output: t2_2.b, t2_2.c
                                  Filter: ((t2_2.b % 25) = 0)
                ->  Hash Right Join
                      Output: t2_3.b, t2_3.c, t1_3.a, t1_3.c
-                     Hash Cond: ((t1_3.a = t2_3.b) AND (t1_3.b = t2_3.a) AND ((t1_3.c)::text = (t2_3.c)::text) AND ((t1_3.b + t1_3.a) = (t2_3.a + t2_3.b)))
-                     ->  Seq Scan on public.prt1_l_p2_p2 t1_3
-                           Output: t1_3.a, t1_3.c, t1_3.b
-                     ->  Hash
-                           Output: t2_3.b, t2_3.c, t2_3.a
-                           ->  Seq Scan on public.prt2_l_p2_p2 t2_3
-                                 Output: t2_3.b, t2_3.c, t2_3.a
-                                 Filter: ((t2_3.b % 25) = 0)
-               ->  Hash Right Join
-                     Output: t2_4.b, t2_4.c, t1_4.a, t1_4.c
-                     Hash Cond: ((t1_4.a = t2_4.b) AND (t1_4.b = t2_4.a) AND ((t1_4.c)::text = (t2_4.c)::text) AND ((t1_4.b + t1_4.a) = (t2_4.a + t2_4.b)))
-                     ->  Seq Scan on public.prt1_l_p3_p1 t1_4
-                           Output: t1_4.a, t1_4.c, t1_4.b
-                     ->  Hash
-                           Output: t2_4.b, t2_4.c, t2_4.a
-                           ->  Seq Scan on public.prt2_l_p3_p1 t2_4
-                                 Output: t2_4.b, t2_4.c, t2_4.a
-                                 Filter: ((t2_4.b % 25) = 0)
-               ->  Hash Right Join
-                     Output: t2_5.b, t2_5.c, t1_5.a, t1_5.c
-                     Hash Cond: ((t1_5.a = t2_5.b) AND (t1_5.b = t2_5.a) AND ((t1_5.c)::text = (t2_5.c)::text) AND ((t1_5.b + t1_5.a) = (t2_5.a + t2_5.b)))
-                     ->  Seq Scan on public.prt1_l_p3_p2 t1_5
-                           Output: t1_5.a, t1_5.c, t1_5.b
+                     Hash Cond: ((t1_3.a = t2_3.b) AND ((t1_3.c)::text = (t2_3.c)::text))
+                     ->  Append
+                           ->  Seq Scan on public.prt1_l_p3 t1_3
+                                 Output: t1_3.a, t1_3.c
+                           ->  Seq Scan on public.prt1_l_p3_p1 t1_4
+                                 Output: t1_4.a, t1_4.c
+                           ->  Seq Scan on public.prt1_l_p3_p2 t1_5
+                                 Output: t1_5.a, t1_5.c
                      ->  Hash
-                           Output: t2_5.b, t2_5.c, t2_5.a
-                           ->  Seq Scan on public.prt2_l_p3_p2 t2_5
-                                 Output: t2_5.b, t2_5.c, t2_5.a
-                                 Filter: ((t2_5.b % 25) = 0)
-(66 rows)
+                           Output: t2_3.b, t2_3.c
+                           ->  Append
+                                 ->  Seq Scan on public.prt2_l_p3 t2_3
+                                       Output: t2_3.b, t2_3.c
+                                       Filter: ((t2_3.b % 25) = 0)
+                                 ->  Seq Scan on public.prt2_l_p3_p1 t2_4
+                                       Output: t2_4.b, t2_4.c
+                                       Filter: ((t2_4.b % 25) = 0)
+                                 ->  Seq Scan on public.prt2_l_p3_p2 t2_5
+                                       Output: t2_5.b, t2_5.c
+                                       Filter: ((t2_5.b % 25) = 0)
+(58 rows)
 
-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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 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.b % 25 = 0 ORDER BY t1.a, t2.b;
   a  |  c   |  b  |  c   
 -----+------+-----+------
    0 | 0000 |   0 | 0000
@@ -326,7 +299,7 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1 RIGHT JOIN prt2_l t2 ON t1.a = t2.b
      |      | 525 | 0525
 (8 rows)
 
-SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 RIGHT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 RIGHT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
   a  |  c   |  b  |  c   
 -----+------+-----+------
    0 | 0000 |   0 | 0000
@@ -339,83 +312,76 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 RIGHT JOIN uprt2_l t2 ON t1.a = t2
      |      | 525 | 0525
 (8 rows)
 
+-- full join
 EXPLAIN (VERBOSE, COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
-                                                                                                              QUERY PLAN                                                                                                              
---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) ORDER BY t1.a, t2.b;
+                                                     QUERY PLAN                                                     
+--------------------------------------------------------------------------------------------------------------------
  Sort
-   Output: prt1_l_p1_p1.a, prt1_l_p1_p1.c, prt2_l_p1_p1.b, prt2_l_p1_p1.c
-   Sort Key: prt1_l_p1_p1.a, prt2_l_p1_p1.b
+   Output: prt1_l_p1.a, prt1_l_p1.c, prt2_l_p1.b, prt2_l_p1.c
+   Sort Key: prt1_l_p1.a, prt2_l_p1.b
    ->  Append
          ->  Hash Full Join
-               Output: prt1_l_p1_p1.a, prt1_l_p1_p1.c, prt2_l_p1_p1.b, prt2_l_p1_p1.c
-               Hash Cond: ((prt1_l_p1_p1.a = prt2_l_p1_p1.b) AND (prt1_l_p1_p1.b = prt2_l_p1_p1.a) AND ((prt1_l_p1_p1.c)::text = (prt2_l_p1_p1.c)::text) AND ((prt1_l_p1_p1.b + prt1_l_p1_p1.a) = (prt2_l_p1_p1.a + prt2_l_p1_p1.b)))
-               ->  Seq Scan on public.prt1_l_p1_p1
-                     Output: prt1_l_p1_p1.a, prt1_l_p1_p1.c, prt1_l_p1_p1.b
-                     Filter: ((prt1_l_p1_p1.a % 25) = 0)
-               ->  Hash
-                     Output: prt2_l_p1_p1.b, prt2_l_p1_p1.c, prt2_l_p1_p1.a
-                     ->  Seq Scan on public.prt2_l_p1_p1
-                           Output: prt2_l_p1_p1.b, prt2_l_p1_p1.c, prt2_l_p1_p1.a
-                           Filter: ((prt2_l_p1_p1.b % 25) = 0)
-         ->  Hash Full Join
-               Output: prt1_l_p1_p2.a, prt1_l_p1_p2.c, prt2_l_p1_p2.b, prt2_l_p1_p2.c
-               Hash Cond: ((prt1_l_p1_p2.a = prt2_l_p1_p2.b) AND (prt1_l_p1_p2.b = prt2_l_p1_p2.a) AND ((prt1_l_p1_p2.c)::text = (prt2_l_p1_p2.c)::text) AND ((prt1_l_p1_p2.b + prt1_l_p1_p2.a) = (prt2_l_p1_p2.a + prt2_l_p1_p2.b)))
-               ->  Seq Scan on public.prt1_l_p1_p2
-                     Output: prt1_l_p1_p2.a, prt1_l_p1_p2.c, prt1_l_p1_p2.b
-                     Filter: ((prt1_l_p1_p2.a % 25) = 0)
+               Output: prt1_l_p1.a, prt1_l_p1.c, prt2_l_p1.b, prt2_l_p1.c
+               Hash Cond: ((prt1_l_p1.a = prt2_l_p1.b) AND ((prt1_l_p1.c)::text = (prt2_l_p1.c)::text))
+               ->  Seq Scan on public.prt1_l_p1
+                     Output: prt1_l_p1.a, prt1_l_p1.c
+                     Filter: ((prt1_l_p1.a % 25) = 0)
                ->  Hash
-                     Output: prt2_l_p1_p2.b, prt2_l_p1_p2.c, prt2_l_p1_p2.a
-                     ->  Seq Scan on public.prt2_l_p1_p2
-                           Output: prt2_l_p1_p2.b, prt2_l_p1_p2.c, prt2_l_p1_p2.a
-                           Filter: ((prt2_l_p1_p2.b % 25) = 0)
+                     Output: prt2_l_p1.b, prt2_l_p1.c
+                     ->  Seq Scan on public.prt2_l_p1
+                           Output: prt2_l_p1.b, prt2_l_p1.c
+                           Filter: ((prt2_l_p1.b % 25) = 0)
          ->  Hash Full Join
                Output: prt1_l_p2_p1.a, prt1_l_p2_p1.c, prt2_l_p2_p1.b, prt2_l_p2_p1.c
-               Hash Cond: ((prt1_l_p2_p1.a = prt2_l_p2_p1.b) AND (prt1_l_p2_p1.b = prt2_l_p2_p1.a) AND ((prt1_l_p2_p1.c)::text = (prt2_l_p2_p1.c)::text) AND ((prt1_l_p2_p1.b + prt1_l_p2_p1.a) = (prt2_l_p2_p1.a + prt2_l_p2_p1.b)))
+               Hash Cond: ((prt1_l_p2_p1.a = prt2_l_p2_p1.b) AND ((prt1_l_p2_p1.c)::text = (prt2_l_p2_p1.c)::text))
                ->  Seq Scan on public.prt1_l_p2_p1
-                     Output: prt1_l_p2_p1.a, prt1_l_p2_p1.c, prt1_l_p2_p1.b
+                     Output: prt1_l_p2_p1.a, prt1_l_p2_p1.c
                      Filter: ((prt1_l_p2_p1.a % 25) = 0)
                ->  Hash
-                     Output: prt2_l_p2_p1.b, prt2_l_p2_p1.c, prt2_l_p2_p1.a
+                     Output: prt2_l_p2_p1.b, prt2_l_p2_p1.c
                      ->  Seq Scan on public.prt2_l_p2_p1
-                           Output: prt2_l_p2_p1.b, prt2_l_p2_p1.c, prt2_l_p2_p1.a
+                           Output: prt2_l_p2_p1.b, prt2_l_p2_p1.c
                            Filter: ((prt2_l_p2_p1.b % 25) = 0)
          ->  Hash Full Join
                Output: prt1_l_p2_p2.a, prt1_l_p2_p2.c, prt2_l_p2_p2.b, prt2_l_p2_p2.c
-               Hash Cond: ((prt1_l_p2_p2.a = prt2_l_p2_p2.b) AND (prt1_l_p2_p2.b = prt2_l_p2_p2.a) AND ((prt1_l_p2_p2.c)::text = (prt2_l_p2_p2.c)::text) AND ((prt1_l_p2_p2.b + prt1_l_p2_p2.a) = (prt2_l_p2_p2.a + prt2_l_p2_p2.b)))
+               Hash Cond: ((prt1_l_p2_p2.a = prt2_l_p2_p2.b) AND ((prt1_l_p2_p2.c)::text = (prt2_l_p2_p2.c)::text))
                ->  Seq Scan on public.prt1_l_p2_p2
-                     Output: prt1_l_p2_p2.a, prt1_l_p2_p2.c, prt1_l_p2_p2.b
+                     Output: prt1_l_p2_p2.a, prt1_l_p2_p2.c
                      Filter: ((prt1_l_p2_p2.a % 25) = 0)
                ->  Hash
-                     Output: prt2_l_p2_p2.b, prt2_l_p2_p2.c, prt2_l_p2_p2.a
+                     Output: prt2_l_p2_p2.b, prt2_l_p2_p2.c
                      ->  Seq Scan on public.prt2_l_p2_p2
-                           Output: prt2_l_p2_p2.b, prt2_l_p2_p2.c, prt2_l_p2_p2.a
+                           Output: prt2_l_p2_p2.b, prt2_l_p2_p2.c
                            Filter: ((prt2_l_p2_p2.b % 25) = 0)
          ->  Hash Full Join
-               Output: prt1_l_p3_p1.a, prt1_l_p3_p1.c, prt2_l_p3_p1.b, prt2_l_p3_p1.c
-               Hash Cond: ((prt1_l_p3_p1.a = prt2_l_p3_p1.b) AND (prt1_l_p3_p1.b = prt2_l_p3_p1.a) AND ((prt1_l_p3_p1.c)::text = (prt2_l_p3_p1.c)::text) AND ((prt1_l_p3_p1.b + prt1_l_p3_p1.a) = (prt2_l_p3_p1.a + prt2_l_p3_p1.b)))
-               ->  Seq Scan on public.prt1_l_p3_p1
-                     Output: prt1_l_p3_p1.a, prt1_l_p3_p1.c, prt1_l_p3_p1.b
-                     Filter: ((prt1_l_p3_p1.a % 25) = 0)
+               Output: prt1_l_p3.a, prt1_l_p3.c, prt2_l_p3.b, prt2_l_p3.c
+               Hash Cond: ((prt1_l_p3.a = prt2_l_p3.b) AND ((prt1_l_p3.c)::text = (prt2_l_p3.c)::text))
+               ->  Append
+                     ->  Seq Scan on public.prt1_l_p3
+                           Output: prt1_l_p3.a, prt1_l_p3.c
+                           Filter: ((prt1_l_p3.a % 25) = 0)
+                     ->  Seq Scan on public.prt1_l_p3_p1
+                           Output: prt1_l_p3_p1.a, prt1_l_p3_p1.c
+                           Filter: ((prt1_l_p3_p1.a % 25) = 0)
+                     ->  Seq Scan on public.prt1_l_p3_p2
+                           Output: prt1_l_p3_p2.a, prt1_l_p3_p2.c
+                           Filter: ((prt1_l_p3_p2.a % 25) = 0)
                ->  Hash
-                     Output: prt2_l_p3_p1.b, prt2_l_p3_p1.c, prt2_l_p3_p1.a
-                     ->  Seq Scan on public.prt2_l_p3_p1
-                           Output: prt2_l_p3_p1.b, prt2_l_p3_p1.c, prt2_l_p3_p1.a
-                           Filter: ((prt2_l_p3_p1.b % 25) = 0)
-         ->  Hash Full Join
-               Output: prt1_l_p3_p2.a, prt1_l_p3_p2.c, prt2_l_p3_p2.b, prt2_l_p3_p2.c
-               Hash Cond: ((prt1_l_p3_p2.a = prt2_l_p3_p2.b) AND (prt1_l_p3_p2.b = prt2_l_p3_p2.a) AND ((prt1_l_p3_p2.c)::text = (prt2_l_p3_p2.c)::text) AND ((prt1_l_p3_p2.b + prt1_l_p3_p2.a) = (prt2_l_p3_p2.a + prt2_l_p3_p2.b)))
-               ->  Seq Scan on public.prt1_l_p3_p2
-                     Output: prt1_l_p3_p2.a, prt1_l_p3_p2.c, prt1_l_p3_p2.b
-                     Filter: ((prt1_l_p3_p2.a % 25) = 0)
-               ->  Hash
-                     Output: prt2_l_p3_p2.b, prt2_l_p3_p2.c, prt2_l_p3_p2.a
-                     ->  Seq Scan on public.prt2_l_p3_p2
-                           Output: prt2_l_p3_p2.b, prt2_l_p3_p2.c, prt2_l_p3_p2.a
-                           Filter: ((prt2_l_p3_p2.b % 25) = 0)
-(70 rows)
+                     Output: prt2_l_p3.b, prt2_l_p3.c
+                     ->  Append
+                           ->  Seq Scan on public.prt2_l_p3
+                                 Output: prt2_l_p3.b, prt2_l_p3.c
+                                 Filter: ((prt2_l_p3.b % 25) = 0)
+                           ->  Seq Scan on public.prt2_l_p3_p1
+                                 Output: prt2_l_p3_p1.b, prt2_l_p3_p1.c
+                                 Filter: ((prt2_l_p3_p1.b % 25) = 0)
+                           ->  Seq Scan on public.prt2_l_p3_p2
+                                 Output: prt2_l_p3_p2.b, prt2_l_p3_p2.c
+                                 Filter: ((prt2_l_p3_p2.b % 25) = 0)
+(62 rows)
 
-SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) ORDER BY t1.a, t2.b;
   a  |  c   |  b  |  c   
 -----+------+-----+------
    0 | 0000 |   0 | 0000
@@ -436,7 +402,7 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0
      |      | 525 | 0525
 (16 rows)
 
-SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_l t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_l t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_l WHERE uprt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_l WHERE uprt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) ORDER BY t1.a, t2.b;
   a  |  c   |  b  |  c   
 -----+------+-----+------
    0 | 0000 |   0 | 0000
@@ -456,3 +422,266 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_l t1 WHERE t1.a % 25 = 0
      |      | 375 | 0375
      |      | 525 | 0525
 (16 rows)
+
+-- lateral partition-wise join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) 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.a % 25 = 0 ORDER BY t1.a;
+                                                     QUERY PLAN                                                      
+---------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c, t2.a, t2.c, t2.b, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   Sort Key: t1.a
+   ->  Result
+         Output: t1.a, t1.b, t1.c, t2.a, t2.c, t2.b, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+         ->  Append
+               ->  Nested Loop Left Join
+                     Output: t1.a, t1.b, t1.c, t2.a, t2.c, t2.b, t3.a, (LEAST(t1.a, t2.a, t3.a))
+                     ->  Seq Scan on public.prt1_l_p1 t1
+                           Output: t1.a, t1.b, t1.c
+                           Filter: ((t1.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2.a, t2.c, t2.b, t3.a, LEAST(t1.a, t2.a, t3.a)
+                           Hash Cond: ((t3.b = t2.a) AND ((t3.c)::text = (t2.c)::text))
+                           ->  Seq Scan on public.prt2_l_p1 t3
+                                 Output: t3.a, t3.b, t3.c
+                           ->  Hash
+                                 Output: t2.a, t2.c, t2.b
+                                 ->  Seq Scan on public.prt1_l_p1 t2
+                                       Output: t2.a, t2.c, t2.b
+                                       Filter: ((t1.a = t2.a) AND ((t1.c)::text = (t2.c)::text))
+               ->  Nested Loop Left Join
+                     Output: t1_1.a, t1_1.b, t1_1.c, t2_1.a, t2_1.c, t2_1.b, t3_1.a, (LEAST(t1_1.a, t2_1.a, t3_1.a))
+                     ->  Seq Scan on public.prt1_l_p2_p1 t1_1
+                           Output: t1_1.a, t1_1.b, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2_1.a, t2_1.c, t2_1.b, t3_1.a, LEAST(t1_1.a, t2_1.a, t3_1.a)
+                           Hash Cond: ((t3_1.b = t2_1.a) AND ((t3_1.c)::text = (t2_1.c)::text))
+                           ->  Seq Scan on public.prt2_l_p2_p1 t3_1
+                                 Output: t3_1.a, t3_1.b, t3_1.c
+                           ->  Hash
+                                 Output: t2_1.a, t2_1.c, t2_1.b
+                                 ->  Seq Scan on public.prt1_l_p2_p1 t2_1
+                                       Output: t2_1.a, t2_1.c, t2_1.b
+                                       Filter: ((t1_1.a = t2_1.a) AND ((t1_1.c)::text = (t2_1.c)::text))
+               ->  Nested Loop Left Join
+                     Output: t1_2.a, t1_2.b, t1_2.c, t2_2.a, t2_2.c, t2_2.b, t3_2.a, (LEAST(t1_2.a, t2_2.a, t3_2.a))
+                     ->  Seq Scan on public.prt1_l_p2_p2 t1_2
+                           Output: t1_2.a, t1_2.b, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2_2.a, t2_2.c, t2_2.b, t3_2.a, LEAST(t1_2.a, t2_2.a, t3_2.a)
+                           Hash Cond: ((t3_2.b = t2_2.a) AND ((t3_2.c)::text = (t2_2.c)::text))
+                           ->  Seq Scan on public.prt2_l_p2_p2 t3_2
+                                 Output: t3_2.a, t3_2.b, t3_2.c
+                           ->  Hash
+                                 Output: t2_2.a, t2_2.c, t2_2.b
+                                 ->  Seq Scan on public.prt1_l_p2_p2 t2_2
+                                       Output: t2_2.a, t2_2.c, t2_2.b
+                                       Filter: ((t1_2.a = t2_2.a) AND ((t1_2.c)::text = (t2_2.c)::text))
+               ->  Nested Loop Left Join
+                     Output: t1_3.a, t1_3.b, t1_3.c, t2_3.a, t2_3.c, t2_3.b, t3_3.a, (LEAST(t1_3.a, t2_3.a, t3_3.a))
+                     ->  Append
+                           ->  Seq Scan on public.prt1_l_p3 t1_3
+                                 Output: t1_3.a, t1_3.b, t1_3.c
+                                 Filter: ((t1_3.a % 25) = 0)
+                           ->  Seq Scan on public.prt1_l_p3_p1 t1_4
+                                 Output: t1_4.a, t1_4.b, t1_4.c
+                                 Filter: ((t1_4.a % 25) = 0)
+                           ->  Seq Scan on public.prt1_l_p3_p2 t1_5
+                                 Output: t1_5.a, t1_5.b, t1_5.c
+                                 Filter: ((t1_5.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2_3.a, t2_3.c, t2_3.b, t3_3.a, LEAST(t1_3.a, t2_3.a, t3_3.a)
+                           Hash Cond: ((t3_3.b = t2_3.a) AND ((t3_3.c)::text = (t2_3.c)::text))
+                           ->  Append
+                                 ->  Seq Scan on public.prt2_l_p3 t3_3
+                                       Output: t3_3.a, t3_3.b, t3_3.c
+                                 ->  Seq Scan on public.prt2_l_p3_p1 t3_4
+                                       Output: t3_4.a, t3_4.b, t3_4.c
+                                 ->  Seq Scan on public.prt2_l_p3_p2 t3_5
+                                       Output: t3_5.a, t3_5.b, t3_5.c
+                           ->  Hash
+                                 Output: t2_3.a, t2_3.c, t2_3.b
+                                 ->  Append
+                                       ->  Seq Scan on public.prt1_l_p3 t2_3
+                                             Output: t2_3.a, t2_3.c, t2_3.b
+                                             Filter: ((t1_3.a = t2_3.a) AND ((t1_3.c)::text = (t2_3.c)::text))
+                                       ->  Seq Scan on public.prt1_l_p3_p1 t2_4
+                                             Output: t2_4.a, t2_4.c, t2_4.b
+                                             Filter: ((t1_3.a = t2_4.a) AND ((t1_3.c)::text = (t2_4.c)::text))
+                                       ->  Seq Scan on public.prt1_l_p3_p2 t2_5
+                                             Output: t2_5.a, t2_5.c, t2_5.b
+                                             Filter: ((t1_3.a = t2_5.a) AND ((t1_3.c)::text = (t2_5.c)::text))
+(85 rows)
+
+SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) 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.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t2c  | t2b | t3a | least 
+-----+-----+------+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |      |     |     |      
+ 100 | 100 | 0100 |     |      |     |     |      
+ 150 | 150 | 0150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |      |     |     |      
+ 250 | 250 | 0250 |     |      |     |     |      
+ 300 | 300 | 0300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |      |     |     |      
+ 400 | 400 | 0400 |     |      |     |     |      
+ 450 | 450 | 0450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |      |     |     |      
+ 550 | 550 | 0550 |     |      |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1_l t2 JOIN uprt2_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.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t2c  | t2b | t3a | least 
+-----+-----+------+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |      |     |     |      
+ 100 | 100 | 0100 |     |      |     |     |      
+ 150 | 150 | 0150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |      |     |     |      
+ 250 | 250 | 0250 |     |      |     |     |      
+ 300 | 300 | 0300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |      |     |     |      
+ 400 | 400 | 0400 |     |      |     |     |      
+ 450 | 450 | 0450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |      |     |     |      
+ 550 | 550 | 0550 |     |      |     |     |      
+(12 rows)
+
+-- lateral references with clauses without equi-join on partition key
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.c = t3.c)) ss
+			  ON t1.b = ss.t2a AND t1.c = ss.t2c WHERE t1.a % 25 = 0 ORDER BY t1.a;
+                                             QUERY PLAN                                              
+-----------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c, t2.a, t2.c, t2.b, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   Sort Key: t1.a
+   ->  Nested Loop Left Join
+         Output: t1.a, t1.b, t1.c, t2.a, t2.c, t2.b, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+         ->  Append
+               ->  Seq Scan on public.prt1_l t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Seq Scan on public.prt1_l_p1 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Seq Scan on public.prt1_l_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Seq Scan on public.prt1_l_p2_p1 t1_3
+                     Output: t1_3.a, t1_3.b, t1_3.c
+                     Filter: ((t1_3.a % 25) = 0)
+               ->  Seq Scan on public.prt1_l_p2_p2 t1_4
+                     Output: t1_4.a, t1_4.b, t1_4.c
+                     Filter: ((t1_4.a % 25) = 0)
+               ->  Seq Scan on public.prt1_l_p3 t1_5
+                     Output: t1_5.a, t1_5.b, t1_5.c
+                     Filter: ((t1_5.a % 25) = 0)
+               ->  Seq Scan on public.prt1_l_p3_p1 t1_6
+                     Output: t1_6.a, t1_6.b, t1_6.c
+                     Filter: ((t1_6.a % 25) = 0)
+               ->  Seq Scan on public.prt1_l_p3_p2 t1_7
+                     Output: t1_7.a, t1_7.b, t1_7.c
+                     Filter: ((t1_7.a % 25) = 0)
+         ->  Append
+               ->  Hash Join
+                     Output: t2.a, t2.c, t2.b, t3.a, LEAST(t1.a, t2.a, t3.a)
+                     Hash Cond: ((t3.b = t2.a) AND ((t3.c)::text = (t2.c)::text))
+                     ->  Seq Scan on public.prt2_l_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t2.a, t2.c, t2.b
+                           ->  Seq Scan on public.prt1_l_p1 t2
+                                 Output: t2.a, t2.c, t2.b
+                                 Filter: ((t1.b = t2.a) AND ((t1.c)::text = (t2.c)::text))
+               ->  Hash Join
+                     Output: t2_1.a, t2_1.c, t2_1.b, t3_1.a, LEAST(t1.a, t2_1.a, t3_1.a)
+                     Hash Cond: ((t3_1.b = t2_1.a) AND ((t3_1.c)::text = (t2_1.c)::text))
+                     ->  Seq Scan on public.prt2_l_p2_p1 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t2_1.a, t2_1.c, t2_1.b
+                           ->  Seq Scan on public.prt1_l_p2_p1 t2_1
+                                 Output: t2_1.a, t2_1.c, t2_1.b
+                                 Filter: ((t1.b = t2_1.a) AND ((t1.c)::text = (t2_1.c)::text))
+               ->  Hash Join
+                     Output: t2_2.a, t2_2.c, t2_2.b, t3_2.a, LEAST(t1.a, t2_2.a, t3_2.a)
+                     Hash Cond: ((t3_2.b = t2_2.a) AND ((t3_2.c)::text = (t2_2.c)::text))
+                     ->  Seq Scan on public.prt2_l_p2_p2 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t2_2.a, t2_2.c, t2_2.b
+                           ->  Seq Scan on public.prt1_l_p2_p2 t2_2
+                                 Output: t2_2.a, t2_2.c, t2_2.b
+                                 Filter: ((t1.b = t2_2.a) AND ((t1.c)::text = (t2_2.c)::text))
+               ->  Hash Join
+                     Output: t2_3.a, t2_3.c, t2_3.b, t3_3.a, LEAST(t1.a, t2_3.a, t3_3.a)
+                     Hash Cond: ((t3_3.b = t2_3.a) AND ((t3_3.c)::text = (t2_3.c)::text))
+                     ->  Append
+                           ->  Seq Scan on public.prt2_l_p3 t3_3
+                                 Output: t3_3.a, t3_3.b, t3_3.c
+                           ->  Seq Scan on public.prt2_l_p3_p1 t3_4
+                                 Output: t3_4.a, t3_4.b, t3_4.c
+                           ->  Seq Scan on public.prt2_l_p3_p2 t3_5
+                                 Output: t3_5.a, t3_5.b, t3_5.c
+                     ->  Hash
+                           Output: t2_3.a, t2_3.c, t2_3.b
+                           ->  Append
+                                 ->  Seq Scan on public.prt1_l_p3 t2_3
+                                       Output: t2_3.a, t2_3.c, t2_3.b
+                                       Filter: ((t1.b = t2_3.a) AND ((t1.c)::text = (t2_3.c)::text))
+                                 ->  Seq Scan on public.prt1_l_p3_p1 t2_4
+                                       Output: t2_4.a, t2_4.c, t2_4.b
+                                       Filter: ((t1.b = t2_4.a) AND ((t1.c)::text = (t2_4.c)::text))
+                                 ->  Seq Scan on public.prt1_l_p3_p2 t2_5
+                                       Output: t2_5.a, t2_5.c, t2_5.b
+                                       Filter: ((t1.b = t2_5.a) AND ((t1.c)::text = (t2_5.c)::text))
+(83 rows)
+
+SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.c = t3.c)) ss
+			  ON t1.b = ss.t2a AND t1.c = ss.t2c WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t2c  | t2b | t3a | least 
+-----+-----+------+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |      |     |     |      
+ 100 | 100 | 0100 |     |      |     |     |      
+ 150 | 150 | 0150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |      |     |     |      
+ 250 | 250 | 0250 |     |      |     |     |      
+ 300 | 300 | 0300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |      |     |     |      
+ 400 | 400 | 0400 |     |      |     |     |      
+ 450 | 450 | 0450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |      |     |     |      
+ 550 | 550 | 0550 |     |      |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1_l t2 JOIN uprt2_l t3 ON (t2.a = t3.b AND t2.c = t3.c)) ss
+			  ON t1.b = ss.t2a AND t1.c = ss.t2c WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t2c  | t2b | t3a | least 
+-----+-----+------+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |      |     |     |      
+ 100 | 100 | 0100 |     |      |     |     |      
+ 150 | 150 | 0150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |      |     |     |      
+ 250 | 250 | 0250 |     |      |     |     |      
+ 300 | 300 | 0300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |      |     |     |      
+ 400 | 400 | 0400 |     |      |     |     |      
+ 450 | 450 | 0450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |      |     |     |      
+ 550 | 550 | 0550 |     |      |     |     |      
+(12 rows)
+
diff --git a/src/test/regress/sql/multi_level_partition_join.sql b/src/test/regress/sql/multi_level_partition_join.sql
index 31f0281..821c095 100644
--- a/src/test/regress/sql/multi_level_partition_join.sql
+++ b/src/test/regress/sql/multi_level_partition_join.sql
@@ -1,21 +1,20 @@
+-- Encourage partition-wise join plans.
+SET partition_wise_plan_weight TO 0.1;
+
 --
 -- multi-leveled partitions
 --
 CREATE TABLE prt1_l (a int, b int, c varchar) PARTITION BY RANGE(a);
-CREATE TABLE prt1_l_p1 PARTITION OF prt1_l FOR VALUES FROM (0) TO (250) PARTITION BY RANGE (b);
-CREATE TABLE prt1_l_p1_p1 PARTITION OF prt1_l_p1 FOR VALUES FROM (0) TO (100);
-CREATE TABLE prt1_l_p1_p2 PARTITION OF prt1_l_p1 FOR VALUES FROM (100) TO (250);
+CREATE TABLE prt1_l_p1 PARTITION OF prt1_l FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt1_l_p2 PARTITION OF prt1_l FOR VALUES FROM (250) TO (500) PARTITION BY RANGE (c);
 CREATE TABLE prt1_l_p2_p1 PARTITION OF prt1_l_p2 FOR VALUES FROM ('0250') TO ('0400');
 CREATE TABLE prt1_l_p2_p2 PARTITION OF prt1_l_p2 FOR VALUES FROM ('0400') TO ('0500');
-CREATE TABLE prt1_l_p3 PARTITION OF prt1_l FOR VALUES FROM (500) TO (600) PARTITION BY RANGE ((b + a));
-CREATE TABLE prt1_l_p3_p1 PARTITION OF prt1_l_p3 FOR VALUES FROM (1000) TO (1100);
-CREATE TABLE prt1_l_p3_p2 PARTITION OF prt1_l_p3 FOR VALUES FROM (1100) TO (1200);
+CREATE TABLE prt1_l_p3 PARTITION OF prt1_l FOR VALUES FROM (500) TO (600) PARTITION BY RANGE (b);
+CREATE TABLE prt1_l_p3_p1 PARTITION OF prt1_l_p3 FOR VALUES FROM (500) TO (550);
+CREATE TABLE prt1_l_p3_p2 PARTITION OF prt1_l_p3 FOR VALUES FROM (550) TO (600);
 INSERT INTO prt1_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
 ANALYZE prt1_l;
 ANALYZE prt1_l_p1;
-ANALYZE prt1_l_p1_p1;
-ANALYZE prt1_l_p1_p2;
 ANALYZE prt1_l_p2;
 ANALYZE prt1_l_p2_p1;
 ANALYZE prt1_l_p2_p2;
@@ -27,20 +26,16 @@ ANALYZE prt1_l_p3_p2;
 CREATE TABLE uprt1_l AS SELECT * FROM prt1_l;
 
 CREATE TABLE prt2_l (a int, b int, c varchar) PARTITION BY RANGE(b);
-CREATE TABLE prt2_l_p1 PARTITION OF prt2_l FOR VALUES FROM (0) TO (250) PARTITION BY RANGE (a);
-CREATE TABLE prt2_l_p1_p1 PARTITION OF prt2_l_p1 FOR VALUES FROM (0) TO (100);
-CREATE TABLE prt2_l_p1_p2 PARTITION OF prt2_l_p1 FOR VALUES FROM (100) TO (250);
+CREATE TABLE prt2_l_p1 PARTITION OF prt2_l FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt2_l_p2 PARTITION OF prt2_l FOR VALUES FROM (250) TO (500) PARTITION BY RANGE (c);
 CREATE TABLE prt2_l_p2_p1 PARTITION OF prt2_l_p2 FOR VALUES FROM ('0250') TO ('0400');
 CREATE TABLE prt2_l_p2_p2 PARTITION OF prt2_l_p2 FOR VALUES FROM ('0400') TO ('0500');
-CREATE TABLE prt2_l_p3 PARTITION OF prt2_l FOR VALUES FROM (500) TO (600) PARTITION BY RANGE ((a + b));
-CREATE TABLE prt2_l_p3_p1 PARTITION OF prt2_l_p3 FOR VALUES FROM (1000) TO (1100);
-CREATE TABLE prt2_l_p3_p2 PARTITION OF prt2_l_p3 FOR VALUES FROM (1100) TO (1200);
+CREATE TABLE prt2_l_p3 PARTITION OF prt2_l FOR VALUES FROM (500) TO (600) PARTITION BY RANGE (a);
+CREATE TABLE prt2_l_p3_p1 PARTITION OF prt2_l_p3 FOR VALUES FROM (500) TO (525);
+CREATE TABLE prt2_l_p3_p2 PARTITION OF prt2_l_p3 FOR VALUES FROM (525) TO (600);
 INSERT INTO prt2_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
 ANALYZE prt2_l;
 ANALYZE prt2_l_p1;
-ANALYZE prt2_l_p1_p1;
-ANALYZE prt2_l_p1_p2;
 ANALYZE prt2_l_p2;
 ANALYZE prt2_l_p2_p1;
 ANALYZE prt2_l_p2_p2;
@@ -51,45 +46,50 @@ ANALYZE prt2_l_p3_p2;
 -- results are tested.
 CREATE TABLE uprt2_l AS SELECT * FROM prt2_l;
 
+-- inner join
 EXPLAIN (VERBOSE, 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 = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 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 = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
-SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1, uprt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a % 25 = 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.c = t2.c AND t1.a % 25 = 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.c = t2.c AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1, uprt2_l t2 WHERE t1.a = t2.b AND t1.c = t2.c AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
 
+-- left join
 EXPLAIN (VERBOSE, 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
-SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 LEFT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t1.a % 25 = 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.a % 25 = 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 LEFT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
 
+-- right join
 EXPLAIN (VERBOSE, 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 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.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
-SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 RIGHT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b WHERE t2.b % 25 = 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.b % 25 = 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.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 RIGHT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
 
+-- full join
 EXPLAIN (VERBOSE, COSTS OFF)
-SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
-SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
-SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_l t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_l t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 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.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 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 uprt1_l WHERE uprt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_l WHERE uprt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) ORDER BY t1.a, t2.b;
 
--- lateral reference
+-- lateral partition-wise join
 EXPLAIN (VERBOSE, COSTS OFF)
 SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
-			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
-			  ON t1.a = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) 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.a % 25 = 0 ORDER BY t1.a;
 SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
-			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
-			  ON t1.a = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) 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.a % 25 = 0 ORDER BY t1.a;
 SELECT * FROM uprt1_l t1 LEFT JOIN LATERAL
-			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1_l t2 JOIN uprt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
-			  ON t1.a = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1_l t2 JOIN uprt2_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.a % 25 = 0 ORDER BY t1.a;
 
+-- lateral references with clauses without equi-join on partition key
 EXPLAIN (VERBOSE, COSTS OFF)
 SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
-			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
-			  ON t1.b = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.c = t3.c)) ss
+			  ON t1.b = ss.t2a AND t1.c = ss.t2c WHERE t1.a % 25 = 0 ORDER BY t1.a;
 SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
-			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
-			  ON t1.b = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.c = t3.c)) ss
+			  ON t1.b = ss.t2a AND t1.c = ss.t2c WHERE t1.a % 25 = 0 ORDER BY t1.a;
 SELECT * FROM uprt1_l t1 LEFT JOIN LATERAL
-			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1_l t2 JOIN uprt2_l t3 ON (t2.a = t3.b AND t2.b = t3.a AND t2.c = t3.c AND t2.b + t2.a = t3.a + t3.b)) ss
-			  ON t1.b = ss.t2a AND t1.b = ss.t2a AND t1.c = ss.t2c AND t1.b + t1.a = ss.t2a + ss.t2b WHERE t1.a % 25 = 0 ORDER BY t1.a;
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1_l t2 JOIN uprt2_l t3 ON (t2.a = t3.b AND t2.c = t3.c)) ss
+			  ON t1.b = ss.t2a AND t1.c = ss.t2c WHERE t1.a % 25 = 0 ORDER BY t1.a;
#29Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#28)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Mon, Jan 2, 2017 at 7:32 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

PFA the patch (pg_dp_join_v6.patch) with some bugs fixed and rebased
on the latest code.

Maybe not surprisingly given how fast things are moving around here
these days, this needs a rebase.

Apart from that, my overall comment on this patch is that it's huge:

37 files changed, 7993 insertions(+), 287 deletions(-)

Now, more than half of that is regression test cases and their output,
which you will certainly be asked to pare down in any version of this
intended for commit. But even excluding those, it's still a fairly
patch:

30 files changed, 2783 insertions(+), 272 deletions(-)

I think the reason this is so large is because there's a fair amount
of refactoring work that has been done as a precondition of the actual
meat of the patch, and no attempt has been made to separate the
refactoring work from the main body of the patch. I think that's
something that needs to be done. If you look at the way Amit Langote
submitted the partitioning patches and the follow-up bug fixes, he had
a series of patches 0001-blah, 0002-quux, etc. generated using
format-patch. Each patch had its own commit message written by him
explaining the purpose of that patch, links to relevant discussion,
etc. If you can separate this into more digestible chunks it will be
easier to get committed.

Other questions/comments:

Why does find_partition_scheme need to copy the partition bound
information instead of just pointing to it? Amit went to some trouble
to make sure that this can't change under us while we hold a lock on
the relation, and we'd better hold a lock on the relation if we're
planning a query against it.

I think the PartitionScheme stuff should live in the optimizer rather
that src/backend/catalog/partition.c. Maybe plancat.c? Perhaps we
eventually need a new file in the optimizer just for partitioning
stuff, but I'm not sure about that yet.

The fact that set_append_rel_size needs to reopen the relation to
extract a few more bits of information is not desirable. You need to
fish this information through in some other way; for example, you
could have get_relation_info() stash the needed bits in the
RelOptInfo.

+                * For two partitioned tables with the same
partitioning scheme, it is
+                * assumed that the Oids of matching partitions from
both the tables
+                * are placed at the same position in the array of
partition oids in

Rather than saying that we assume this, you should say why it has to
be true. (If it doesn't have to be true, we shouldn't assume it.)

+                * join relations. Partition tables should have same
layout as the
+                * parent table and hence should not need any
translation. But rest of

The same attributes have to be present with the same types, but they
can be rearranged. This comment seems to imply the contrary.

FRACTION_PARTS_TO_PLAN seems like it should be a GUC.

+               /*
+                * Add this relation to the list of samples ordered by
the increasing
+                * number of rows at appropriate place.
+                */
+               foreach (lc, ordered_child_nos)
+               {
+                       int     child_no = lfirst_int(lc);
+                       RelOptInfo *other_childrel = rel->part_rels[child_no];
+
+                       /*
+                        * Keep track of child with lowest number of
rows but higher than the
+                        * that of the child being inserted. Insert
the child before a
+                        * child with highest number of rows lesser than it.
+                        */
+                       if (child_rel->rows <= other_childrel->rows)
+                               insert_after = lc;
+                       else
+                               break;
+               }

Can we use quicksort instead of a hand-coded insertion sort?

+ if (bms_num_members(outer_relids) > 1)

Seems like bms_get_singleton_member could be used.

+ * Partitioning scheme in join relation indicates a possibilty that the

Spelling.

There seems to be no reason for create_partition_plan to be separated
from create_plan_recurse. You can just add another case for the new
path type.

Why does create_partition_join_path need to be separate from
create_partition_join_path_with_pathkeys? Couldn't that be combined
into a single function with a pathkeys argument that might sometimes
be NIL? I assume most of the logic is common.

From a sort of theoretical standpoint, the biggest danger of this
patch seems to be that by deferring path creation until a later stage
than normal, we could miss some important processing.
subquery_planner() does a lot of stuff after
expand_inherited_tables(); if any of those things, especially the ones
that happen AFTER path generation, have an effect on the paths, then
this code needs to compensate for those changes somehow. It seems
like having the planning of unsampled children get deferred until
create_plan() time is awfully surprising; here we are creating the
plan and suddenly what used to be a straightforward path->plan
translation is running around doing major planning work. I can't
entirely justify it, but I somehow have a feeling that work ought to
be moved earlier. Not sure exactly where.

This is not really a full review, mostly because I can't easily figure
out the motivation for all of the changes the patch makes. It makes a
lot of changes in a lot of places, and it's not really very easy to
understand why those changes are necessary. My comments above about
splitting the patch into a series of patches that can potentially be
reviewed and applied independently, with the main patch being the last
in the series, are a suggestion as to how to tackle that. There might
be some work that needs to or could be done on the comments, too. For
example, the patch splits out add_paths_to_append_rel from
set_append_rel_pathlist, but the comments don't say anything helpful
like "we need to do X after Y, because Z". They just say that we do
it. To some extent I think the comments in the optimizer have that
problem generally, so it's not entirely the fault of this patch;
still, the lack of those explanations makes the code reorganization
harder to follow, and might confuse future patch authors, too.

--
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

#30Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#29)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Feb 2, 2017 at 2:41 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Mon, Jan 2, 2017 at 7:32 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

PFA the patch (pg_dp_join_v6.patch) with some bugs fixed and rebased
on the latest code.

Maybe not surprisingly given how fast things are moving around here
these days, this needs a rebase.

Apart from that, my overall comment on this patch is that it's huge:

37 files changed, 7993 insertions(+), 287 deletions(-)

Now, more than half of that is regression test cases and their output,
which you will certainly be asked to pare down in any version of this
intended for commit.

Yes. I will work on that once the design and implementation is in
acceptable state. I have already toned down testcases compared to the
previous patch.

But even excluding those, it's still a fairly
patch:

30 files changed, 2783 insertions(+), 272 deletions(-)

I think the reason this is so large is because there's a fair amount
of refactoring work that has been done as a precondition of the actual
meat of the patch, and no attempt has been made to separate the
refactoring work from the main body of the patch. I think that's
something that needs to be done. If you look at the way Amit Langote
submitted the partitioning patches and the follow-up bug fixes, he had
a series of patches 0001-blah, 0002-quux, etc. generated using
format-patch. Each patch had its own commit message written by him
explaining the purpose of that patch, links to relevant discussion,
etc. If you can separate this into more digestible chunks it will be
easier to get committed.

I will try to break down the patch into smaller, easy-to-review,
logically cohesive patches.

Other questions/comments:

Why does find_partition_scheme need to copy the partition bound
information instead of just pointing to it? Amit went to some trouble
to make sure that this can't change under us while we hold a lock on
the relation, and we'd better hold a lock on the relation if we're
planning a query against it.

PartitionScheme is shared across multiple relations, join or base,
partitioned similarly. Obviously it can't and does not need to point
partition bound informations (which should all be same) of all those
base relations. O the the face of it, it looks weird that it points to
only one of them, mostly the one which it encounters first. But, since
it's going to be the same partition bound information, it doesn't
matter which one. So, I think, we can point of any one of those. Do
you agree?

I think the PartitionScheme stuff should live in the optimizer rather
that src/backend/catalog/partition.c. Maybe plancat.c? Perhaps we
eventually need a new file in the optimizer just for partitioning
stuff, but I'm not sure about that yet.

I placed PartitionScheme stuff in partition.c because most of the
functions and structures in partition.c are not visible outside that
file. But I will try again to locate PartitionScheme to optimizer.

The fact that set_append_rel_size needs to reopen the relation to
extract a few more bits of information is not desirable. You need to
fish this information through in some other way; for example, you
could have get_relation_info() stash the needed bits in the
RelOptInfo.

I considered this option and discarded it, since not all partitioned
relations will have OIDs for partitions e.g. partitioned joins will
not have OIDs for their partitions. But now that I think of it, we
should probably store those OIDs just for the base relation and leave
them unused for non-base relations just like other base relation
specific fields in RelOptInfo.

+                * For two partitioned tables with the same
partitioning scheme, it is
+                * assumed that the Oids of matching partitions from
both the tables
+                * are placed at the same position in the array of
partition oids in

Rather than saying that we assume this, you should say why it has to
be true. (If it doesn't have to be true, we shouldn't assume it.)

Will take care of this.

+                * join relations. Partition tables should have same
layout as the
+                * parent table and hence should not need any
translation. But rest of

The same attributes have to be present with the same types, but they
can be rearranged. This comment seems to imply the contrary.

Hmm, will take care of this.

FRACTION_PARTS_TO_PLAN seems like it should be a GUC.

+1. Will take care of this. Does "representative_partitions_fraction"
or "sample_partition_fraction" look like a good GUC name? Any other
suggestions?

+               /*
+                * Add this relation to the list of samples ordered by
the increasing
+                * number of rows at appropriate place.
+                */
+               foreach (lc, ordered_child_nos)
+               {
+                       int     child_no = lfirst_int(lc);
+                       RelOptInfo *other_childrel = rel->part_rels[child_no];
+
+                       /*
+                        * Keep track of child with lowest number of
rows but higher than the
+                        * that of the child being inserted. Insert
the child before a
+                        * child with highest number of rows lesser than it.
+                        */
+                       if (child_rel->rows <= other_childrel->rows)
+                               insert_after = lc;
+                       else
+                               break;
+               }

Can we use quicksort instead of a hand-coded insertion sort?

I guess so, if I write comparison functions, which shouldn't be a
problem. Will try that.

+ if (bms_num_members(outer_relids) > 1)

Seems like bms_get_singleton_member could be used.

+ * Partitioning scheme in join relation indicates a possibilty that the

Spelling.

There seems to be no reason for create_partition_plan to be separated
from create_plan_recurse. You can just add another case for the new
path type.

Why does create_partition_join_path need to be separate from
create_partition_join_path_with_pathkeys? Couldn't that be combined
into a single function with a pathkeys argument that might sometimes
be NIL? I assume most of the logic is common.

From a sort of theoretical standpoint, the biggest danger of this
patch seems to be that by deferring path creation until a later stage
than normal, we could miss some important processing.
subquery_planner() does a lot of stuff after
expand_inherited_tables(); if any of those things, especially the ones
that happen AFTER path generation, have an effect on the paths, then
this code needs to compensate for those changes somehow. It seems
like having the planning of unsampled children get deferred until
create_plan() time is awfully surprising; here we are creating the
plan and suddenly what used to be a straightforward path->plan
translation is running around doing major planning work. I can't
entirely justify it, but I somehow have a feeling that work ought to
be moved earlier. Not sure exactly where.

This is not really a full review, mostly because I can't easily figure
out the motivation for all of the changes the patch makes. It makes a
lot of changes in a lot of places, and it's not really very easy to
understand why those changes are necessary. My comments above about
splitting the patch into a series of patches that can potentially be
reviewed and applied independently, with the main patch being the last
in the series, are a suggestion as to how to tackle that. There might
be some work that needs to or could be done on the comments, too. For
example, the patch splits out add_paths_to_append_rel from
set_append_rel_pathlist, but the comments don't say anything helpful
like "we need to do X after Y, because Z". They just say that we do
it. To some extent I think the comments in the optimizer have that
problem generally, so it's not entirely the fault of this patch;
still, the lack of those explanations makes the code reorganization
harder to follow, and might confuse future patch authors, too.

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

--
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

#31Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Ashutosh Bapat (#30)
Re: Partition-wise join for join between (declaratively) partitioned tables

sent the previous mail before completing my reply. Sorry for that.
Here's the rest of the reply.

+ if (bms_num_members(outer_relids) > 1)

Seems like bms_get_singleton_member could be used.

+ * Partitioning scheme in join relation indicates a possibilty that the

Spelling.

Will take care of this.

There seems to be no reason for create_partition_plan to be separated
from create_plan_recurse. You can just add another case for the new
path type.

Will take care of this.

Why does create_partition_join_path need to be separate from
create_partition_join_path_with_pathkeys? Couldn't that be combined
into a single function with a pathkeys argument that might sometimes
be NIL? I assume most of the logic is common.

Agreed. will take care of this.

From a sort of theoretical standpoint, the biggest danger of this
patch seems to be that by deferring path creation until a later stage
than normal, we could miss some important processing.
subquery_planner() does a lot of stuff after
expand_inherited_tables(); if any of those things, especially the ones
that happen AFTER path generation, have an effect on the paths, then
this code needs to compensate for those changes somehow. It seems
like having the planning of unsampled children get deferred until
create_plan() time is awfully surprising; here we are creating the
plan and suddenly what used to be a straightforward path->plan
translation is running around doing major planning work. I can't
entirely justify it, but I somehow have a feeling that work ought to
be moved earlier. Not sure exactly where.

I agree with this. Probably we should add a path tree mutator before
SS_identify_outer_params() to replace any Partition*Paths with
Merge/Append paths. The mutator will create paths for child-joins
within temporary memory context, copy the relevant paths and create
Merge/Append paths. There are two problems there 1. We have to write
code to copy paths; most of the paths would be flat copy but custom
scan paths might have some unexpected problems. 2. There will be many
surviving PartitionPaths, and all the corresponding child paths would
need copying and consume memory. In order to reduce that consumption,
we have run this mutator after set_cheapest() in subquery_planner();
but then nothing interesting happens between that and create_plan().
Expanding PartitionPaths during create_plan() does not need any path
copying and we expand only the PartitionPaths which will be converted
to plans. That does save a lot of memory; the reason why we defer
creating paths for child-joins.

This is not really a full review, mostly because I can't easily figure
out the motivation for all of the changes the patch makes. It makes a
lot of changes in a lot of places, and it's not really very easy to
understand why those changes are necessary. My comments above about
splitting the patch into a series of patches that can potentially be
reviewed and applied independently, with the main patch being the last
in the series, are a suggestion as to how to tackle that. There might
be some work that needs to or could be done on the comments, too. For
example, the patch splits out add_paths_to_append_rel from
set_append_rel_pathlist, but the comments don't say anything helpful
like "we need to do X after Y, because Z". They just say that we do
it. To some extent I think the comments in the optimizer have that
problem generally, so it's not entirely the fault of this patch;
still, the lack of those explanations makes the code reorganization
harder to follow, and might confuse future patch authors, too.

Specifically about add_paths_to_append_rel(), what do you expect the
comment to say? It would be obvious why we split that functionality
into a separate function: in fact, we don't necessarily explain why
certain code resides in a separate function in the comments. I think,
that particular comment (or for that matter other such comments in the
optimizer) can be removed altogether, since it just writes the
function names as an "English" sentence. I sometimes find those
comments useful, because I can read just those comments and forget
about the code, making comprehension easy. If highlighting is ON, your
brain habitually ignores the non-comment portions when required. I am
open to suggestions.

--
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

#32Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Ashutosh Bapat (#30)
11 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

Per your suggestion I have split the patch into many smaller patches.

0001-Refactor-set_append_rel_pathlist.patch
0002-Refactor-make_join_rel.patch
0003-Refactor-adjust_appendrel_attrs.patch
0004-Refactor-build_join_rel.patch
0005-Add-function-find_param_path_info.patch

These four refactor existing code.

0006-Canonical-partition-scheme.patch
0007-Partition-wise-join-tests.patch -- just tests, they fail
0008-Partition-wise-join.patch -- actual patch implementing
partition-wise join, still some tests fail\

0009-Adjust-join-related-to-code-to-accept-child-relation.patch
0010-Parameterized-path-fixes.patch
0011-Use-IS_JOIN_REL-instead-of-RELOPT_JOINREL.patch

The last three patches change existing code to expect child(-join)
relations where they were not expected earlier.

Each patch has summary of the changes.

Partition-wise join for multi-level partitioned tables is not covered
by these patches. I will post those patches soon.

Other questions/comments:

Why does find_partition_scheme need to copy the partition bound
information instead of just pointing to it? Amit went to some trouble
to make sure that this can't change under us while we hold a lock on
the relation, and we'd better hold a lock on the relation if we're
planning a query against it.

PartitionScheme is shared across multiple relations, join or base,
partitioned similarly. Obviously it can't and does not need to point
partition bound informations (which should all be same) of all those
base relations. O the the face of it, it looks weird that it points to
only one of them, mostly the one which it encounters first. But, since
it's going to be the same partition bound information, it doesn't
matter which one. So, I think, we can point of any one of those. Do
you agree?

Instead of copying PartitionBoundInfo, used pointer of the first
encountered one.

I think the PartitionScheme stuff should live in the optimizer rather
that src/backend/catalog/partition.c. Maybe plancat.c? Perhaps we
eventually need a new file in the optimizer just for partitioning
stuff, but I'm not sure about that yet.

I placed PartitionScheme stuff in partition.c because most of the
functions and structures in partition.c are not visible outside that
file. But I will try again to locate PartitionScheme to optimizer.

Moved the code as per your suggestion.

The fact that set_append_rel_size needs to reopen the relation to
extract a few more bits of information is not desirable. You need to
fish this information through in some other way; for example, you
could have get_relation_info() stash the needed bits in the
RelOptInfo.

I considered this option and discarded it, since not all partitioned
relations will have OIDs for partitions e.g. partitioned joins will
not have OIDs for their partitions. But now that I think of it, we
should probably store those OIDs just for the base relation and leave
them unused for non-base relations just like other base relation
specific fields in RelOptInfo.

Changed as per your suggestions.

+                * For two partitioned tables with the same
partitioning scheme, it is
+                * assumed that the Oids of matching partitions from
both the tables
+                * are placed at the same position in the array of
partition oids in

Rather than saying that we assume this, you should say why it has to
be true. (If it doesn't have to be true, we shouldn't assume it.)

Will take care of this.

Done. Please check.

+                * join relations. Partition tables should have same
layout as the
+                * parent table and hence should not need any
translation. But rest of

The same attributes have to be present with the same types, but they
can be rearranged. This comment seems to imply the contrary.

Hmm, will take care of this.

Done.

FRACTION_PARTS_TO_PLAN seems like it should be a GUC.

+1. Will take care of this. Does "representative_partitions_fraction"
or "sample_partition_fraction" look like a good GUC name? Any other
suggestions?

used "sample_partition_fraction" for now. Suggestions are welcome.

+               /*
+                * Add this relation to the list of samples ordered by
the increasing
+                * number of rows at appropriate place.
+                */
+               foreach (lc, ordered_child_nos)
+               {
+                       int     child_no = lfirst_int(lc);
+                       RelOptInfo *other_childrel = rel->part_rels[child_no];
+
+                       /*
+                        * Keep track of child with lowest number of
rows but higher than the
+                        * that of the child being inserted. Insert
the child before a
+                        * child with highest number of rows lesser than it.
+                        */
+                       if (child_rel->rows <= other_childrel->rows)
+                               insert_after = lc;
+                       else
+                               break;
+               }

Can we use quicksort instead of a hand-coded insertion sort?

I guess so, if I write comparison functions, which shouldn't be a
problem. Will try that.

Done.

+ if (bms_num_members(outer_relids) > 1)

Seems like bms_get_singleton_member could be used.

That code is not required any more.

+ * Partitioning scheme in join relation indicates a possibilty that the

Spelling.

Done.

There seems to be no reason for create_partition_plan to be separated
from create_plan_recurse. You can just add another case for the new
path type.

Done.

Why does create_partition_join_path need to be separate from
create_partition_join_path_with_pathkeys? Couldn't that be combined
into a single function with a pathkeys argument that might sometimes
be NIL? I assume most of the logic is common.

Combined those into a single function.

From a sort of theoretical standpoint, the biggest danger of this
patch seems to be that by deferring path creation until a later stage
than normal, we could miss some important processing.
subquery_planner() does a lot of stuff after
expand_inherited_tables(); if any of those things, especially the ones
that happen AFTER path generation, have an effect on the paths, then
this code needs to compensate for those changes somehow. It seems
like having the planning of unsampled children get deferred until
create_plan() time is awfully surprising; here we are creating the
plan and suddenly what used to be a straightforward path->plan
translation is running around doing major planning work. I can't
entirely justify it, but I somehow have a feeling that work ought to
be moved earlier. Not sure exactly where.

Pasting my previous replies here to keep everything in one mail.

I agree with this. Probably we should add a path tree mutator before
SS_identify_outer_params() to replace any Partition*Paths with
Merge/Append paths. The mutator will create paths for child-joins
within temporary memory context, copy the relevant paths and create
Merge/Append paths. There are two problems there 1. We have to write
code to copy paths; most of the paths would be flat copy but custom
scan paths might have some unexpected problems. 2. There will be many
surviving PartitionPaths, and all the corresponding child paths would
need copying and consume memory. In order to reduce that consumption,
we have run this mutator after set_cheapest() in subquery_planner();
but then nothing interesting happens between that and create_plan().
Expanding PartitionPaths during create_plan() does not need any path
copying and we expand only the PartitionPaths which will be converted
to plans. That does save a lot of memory; the reason why we defer
creating paths for child-joins.

This is not really a full review, mostly because I can't easily figure
out the motivation for all of the changes the patch makes. It makes a
lot of changes in a lot of places, and it's not really very easy to
understand why those changes are necessary. My comments above about
splitting the patch into a series of patches that can potentially be
reviewed and applied independently, with the main patch being the last
in the series, are a suggestion as to how to tackle that. There might
be some work that needs to or could be done on the comments, too. For
example, the patch splits out add_paths_to_append_rel from
set_append_rel_pathlist, but the comments don't say anything helpful
like "we need to do X after Y, because Z". They just say that we do
it. To some extent I think the comments in the optimizer have that
problem generally, so it's not entirely the fault of this patch;
still, the lack of those explanations makes the code reorganization
harder to follow, and might confuse future patch authors, too.

Specifically about add_paths_to_append_rel(), what do you expect the
comment to say? It would be obvious why we split that functionality
into a separate function: in fact, we don't necessarily explain why
certain code resides in a separate function in the comments. I think,
that particular comment (or for that matter other such comments in the
optimizer) can be removed altogether, since it just writes the
function names as an "English" sentence. I sometimes find those
comments useful, because I can read just those comments and forget
about the code, making comprehension easy. If highlighting is ON, your
brain habitually ignores the non-comment portions when required. I am
open to suggestions.

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

Attachments:

0001-Refactor-set_append_rel_pathlist.patchapplication/octet-stream; name=0001-Refactor-set_append_rel_pathlist.patchDownload
From cf978fcadc1985c27783c114b8a367fe795efe9e Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Fri, 3 Feb 2017 14:45:37 +0530
Subject: [PATCH 01/11] Refactor set_append_rel_pathlist.

set_append_rel_pathlist() creates paths for each child relation and then
creates append paths for the "append" relation. Right now only a base relation
can have children, but with partition-wise join and aggregation a join or an
upper relation can have children as well. While the code to add paths to the
child relations differs for base, join and upper child relations, the code to
create append paths can be shared by all the three relations. Hence separating
it into a new function add_paths_to_append_rel() so that it can be re-used for
all kinds of relations.
---
 src/backend/optimizer/path/allpaths.c |   51 ++++++++++++++++++++++++++-------
 1 file changed, 41 insertions(+), 10 deletions(-)

diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 5c18987..d797d6a 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -127,6 +127,8 @@ static void recurse_push_qual(Node *setOp, Query *topquery,
 				  RangeTblEntry *rte, Index rti, Node *qual);
 static void remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel);
 static int	compute_parallel_worker(RelOptInfo *rel, BlockNumber pages);
+static void add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
+									List *live_childrels);
 
 
 /*
@@ -1169,19 +1171,11 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 {
 	int			parentRTindex = rti;
 	List	   *live_childrels = NIL;
-	List	   *subpaths = NIL;
-	bool		subpaths_valid = true;
-	List	   *partial_subpaths = NIL;
-	bool		partial_subpaths_valid = true;
-	List	   *all_child_pathkeys = NIL;
-	List	   *all_child_outers = NIL;
 	ListCell   *l;
 
 	/*
 	 * Generate access paths for each member relation, and remember the
-	 * cheapest path for each one.  Also, identify all pathkeys (orderings)
-	 * and parameterizations (required_outer sets) available for the member
-	 * relations.
+	 * non-dummy children.
 	 */
 	foreach(l, root->append_rel_list)
 	{
@@ -1189,7 +1183,6 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 		int			childRTindex;
 		RangeTblEntry *childRTE;
 		RelOptInfo *childrel;
-		ListCell   *lcp;
 
 		/* append_rel_list contains all append rels; ignore others */
 		if (appinfo->parent_relid != parentRTindex)
@@ -1224,6 +1217,44 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 		 * Child is live, so add it to the live_childrels list for use below.
 		 */
 		live_childrels = lappend(live_childrels, childrel);
+	}
+
+	/* Add Append/MergeAppend paths to the "append" relation. */
+	add_paths_to_append_rel(root, rel, live_childrels);
+}
+
+
+/*
+ * add_paths_to_append_rel
+ *		Generate Append/MergeAppend paths for given "append" relation.
+ *
+ * The function collects all parameterizations and orderings supported by the
+ * non-dummy children. For every such parameterization or ordering, it creates
+ * an append path collecting one path from each non-dummy child with given
+ * parameterization or ordering. Similarly it collects partial paths from
+ * non-dummy children to create partial append paths.
+ */
+static void
+add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
+						List *live_childrels)
+{
+	List	   *subpaths = NIL;
+	bool		subpaths_valid = true;
+	List	   *partial_subpaths = NIL;
+	bool		partial_subpaths_valid = true;
+	List	   *all_child_pathkeys = NIL;
+	List	   *all_child_outers = NIL;
+	ListCell   *l;
+
+	/*
+	 * For every non-dummy child, remember the cheapest path.  Also, identify
+	 * all pathkeys (orderings) and parameterizations (required_outer sets)
+	 * available for the non-dummy member relations.
+	 */
+	foreach (l, live_childrels)
+	{
+		RelOptInfo *childrel = lfirst(l);
+		ListCell   *lcp;
 
 		/*
 		 * If child has an unparameterized cheapest-total path, add that to
-- 
1.7.9.5

0002-Refactor-make_join_rel.patchapplication/octet-stream; name=0002-Refactor-make_join_rel.patchDownload
From 80861ce9a56d639833f015392bb6d4035a9deda2 Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Fri, 3 Feb 2017 15:41:39 +0530
Subject: [PATCH 02/11] Refactor make_join_rel().

The code in make_join_rel() to add paths to join relation for a given pair of
joining relations can be re-used to add paths to a child join relation, which
do not need the other functionality offered by make_join_rel(). Separate this
code into populate_joinrel_with_paths(). This patch does just refactors
make_join_rel() to pave the way for partition-wise join.
---
 src/backend/optimizer/path/joinrels.c |   28 ++++++++++++++++++++++++----
 1 file changed, 24 insertions(+), 4 deletions(-)

diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 6f3c20b..936ee0c 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -32,6 +32,9 @@ static bool is_dummy_rel(RelOptInfo *rel);
 static void mark_dummy_rel(RelOptInfo *rel);
 static bool restriction_is_constant_false(List *restrictlist,
 							  bool only_pushed_down);
+static void populate_joinrel_with_paths(PlannerInfo *root, RelOptInfo *rel1,
+							RelOptInfo *rel2, RelOptInfo *joinrel,
+							SpecialJoinInfo *sjinfo, List *restrictlist);
 
 
 /*
@@ -724,6 +727,27 @@ make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 		return joinrel;
 	}
 
+	/* Add paths to the join relation. */
+	populate_joinrel_with_paths(root, rel1, rel2, joinrel, sjinfo,
+								restrictlist);
+
+	bms_free(joinrelids);
+
+	return joinrel;
+}
+
+/*
+ * populate_joinrel_with_paths
+ *	  Add paths to the given joinrel for given pair of joining relations. The
+ *	  SpecialJoinInfo provides details about the join and the restrictlist
+ *	  contains the join clauses and the other clauses applicable for given pair
+ *	  of the joining relations.
+ */
+static void
+populate_joinrel_with_paths(PlannerInfo *root, RelOptInfo *rel1,
+							RelOptInfo *rel2, RelOptInfo *joinrel,
+							SpecialJoinInfo *sjinfo, List *restrictlist)
+{
 	/*
 	 * Consider paths using each rel as both outer and inner.  Depending on
 	 * the join type, a provably empty outer or inner rel might mean the join
@@ -868,10 +892,6 @@ make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 			elog(ERROR, "unrecognized join type: %d", (int) sjinfo->jointype);
 			break;
 	}
-
-	bms_free(joinrelids);
-
-	return joinrel;
 }
 
 
-- 
1.7.9.5

0003-Refactor-adjust_appendrel_attrs.patchapplication/octet-stream; name=0003-Refactor-adjust_appendrel_attrs.patchDownload
From 74d38c91f9c37f8c6ede2a1a63ae3fec6731a10d Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Fri, 3 Feb 2017 17:22:52 +0530
Subject: [PATCH 03/11] Refactor adjust_appendrel_attrs.

adjust_appendrel_attrs() is used to translate nodes for a parent relation to
those for a child relation by replacing the parent specific nodes like Var
nodes with corresponding nodes specific to the child. Right now this function
works with a single parent-child pair. For partition-wise join and
partition-wise aggregation/grouping, we require to translate nodes for multiple
parent-child pairs. This patch modifies adjust_appendrel_attrs() to work with
multiple parent-child pairs.
---
 src/backend/optimizer/path/allpaths.c   |    7 +-
 src/backend/optimizer/path/equivclass.c |    2 +-
 src/backend/optimizer/plan/planner.c    |    2 +-
 src/backend/optimizer/prep/prepunion.c  |  161 ++++++++++++++++++++++---------
 src/include/optimizer/prep.h            |    2 +-
 5 files changed, 121 insertions(+), 53 deletions(-)

diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index d797d6a..d8fac14 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -862,6 +862,7 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		ListCell   *parentvars;
 		ListCell   *childvars;
 		ListCell   *lc;
+		List	   *appinfo_list = list_make1(appinfo);
 
 		/* append_rel_list contains all append rels; ignore others */
 		if (appinfo->parent_relid != parentRTindex)
@@ -903,7 +904,7 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 			Assert(IsA(rinfo, RestrictInfo));
 			childqual = adjust_appendrel_attrs(root,
 											   (Node *) rinfo->clause,
-											   appinfo);
+											   appinfo_list);
 			childqual = eval_const_expressions(root, childqual);
 			/* check for flat-out constant */
 			if (childqual && IsA(childqual, Const))
@@ -1022,11 +1023,11 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		childrel->joininfo = (List *)
 			adjust_appendrel_attrs(root,
 								   (Node *) rel->joininfo,
-								   appinfo);
+								   appinfo_list);
 		childrel->reltarget->exprs = (List *)
 			adjust_appendrel_attrs(root,
 								   (Node *) rel->reltarget->exprs,
-								   appinfo);
+								   appinfo_list);
 
 		/*
 		 * We have to make child entries in the EquivalenceClass data
diff --git a/src/backend/optimizer/path/equivclass.c b/src/backend/optimizer/path/equivclass.c
index a329dd1..bcce142 100644
--- a/src/backend/optimizer/path/equivclass.c
+++ b/src/backend/optimizer/path/equivclass.c
@@ -2111,7 +2111,7 @@ add_child_rel_equivalences(PlannerInfo *root,
 				child_expr = (Expr *)
 					adjust_appendrel_attrs(root,
 										   (Node *) cur_em->em_expr,
-										   appinfo);
+										   list_make1(appinfo));
 
 				/*
 				 * Transform em_relids to match.  Note we do *not* do
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 881742f..24a48b8 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -1085,7 +1085,7 @@ inheritance_planner(PlannerInfo *root)
 		subroot->parse = (Query *)
 			adjust_appendrel_attrs(root,
 								   (Node *) parse,
-								   appinfo);
+								   list_make1(appinfo));
 
 		/*
 		 * If there are securityQuals attached to the parent, move them to the
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index 06e843d..6f41979 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -55,7 +55,7 @@
 typedef struct
 {
 	PlannerInfo *root;
-	AppendRelInfo *appinfo;
+	List	    *appinfos;
 } adjust_appendrel_attrs_context;
 
 static Path *recurse_set_operations(Node *setOp, PlannerInfo *root,
@@ -107,9 +107,9 @@ static Bitmapset *translate_col_privs(const Bitmapset *parent_privs,
 					List *translated_vars);
 static Node *adjust_appendrel_attrs_mutator(Node *node,
 							   adjust_appendrel_attrs_context *context);
-static Relids adjust_relid_set(Relids relids, Index oldrelid, Index newrelid);
 static List *adjust_inherited_tlist(List *tlist,
 					   AppendRelInfo *context);
+static Relids adjust_relid_set(Relids relids, List *append_rel_infos);
 
 
 /*
@@ -1719,10 +1719,10 @@ translate_col_privs(const Bitmapset *parent_privs,
 
 /*
  * adjust_appendrel_attrs
- *	  Copy the specified query or expression and translate Vars referring
- *	  to the parent rel of the specified AppendRelInfo to refer to the
- *	  child rel instead.  We also update rtindexes appearing outside Vars,
- *	  such as resultRelation and jointree relids.
+ *	  Copy the specified query or expression and translate Vars referring to
+ *	  the parent rels of the specified in the given list of AppendRelInfos to
+ *	  refer to the corresponding child rel instead.  We also update rtindexes
+ *	  appearing outside Vars, such as resultRelation and jointree relids.
  *
  * Note: this is only applied after conversion of sublinks to subplans,
  * so we don't need to cope with recursion into sub-queries.
@@ -1731,13 +1731,20 @@ translate_col_privs(const Bitmapset *parent_privs,
  * maybe we should try to fold the two routines together.
  */
 Node *
-adjust_appendrel_attrs(PlannerInfo *root, Node *node, AppendRelInfo *appinfo)
+adjust_appendrel_attrs(PlannerInfo *root, Node *node, List *appinfos)
 {
 	Node	   *result;
 	adjust_appendrel_attrs_context context;
+	ListCell   *lc;
 
 	context.root = root;
-	context.appinfo = appinfo;
+	context.appinfos = appinfos;
+
+	/*
+	 * Catch a caller who wants to adjust expressions, but doesn't pass any
+	 * AppendRelInfo.
+	 */
+	Assert(appinfos && list_length(appinfos) >= 1);
 
 	/*
 	 * Must be prepared to start with a Query or a bare expression tree.
@@ -1745,20 +1752,28 @@ adjust_appendrel_attrs(PlannerInfo *root, Node *node, AppendRelInfo *appinfo)
 	if (node && IsA(node, Query))
 	{
 		Query	   *newnode;
+		AppendRelInfo *appinfo;
 
 		newnode = query_tree_mutator((Query *) node,
 									 adjust_appendrel_attrs_mutator,
 									 (void *) &context,
 									 QTW_IGNORE_RC_SUBQUERIES);
-		if (newnode->resultRelation == appinfo->parent_relid)
+		foreach (lc, appinfos)
 		{
-			newnode->resultRelation = appinfo->child_relid;
-			/* Fix tlist resnos too, if it's inherited UPDATE */
-			if (newnode->commandType == CMD_UPDATE)
-				newnode->targetList =
-					adjust_inherited_tlist(newnode->targetList,
-										   appinfo);
+			appinfo = lfirst(lc);
+
+			if (newnode->resultRelation == appinfo->parent_relid)
+			{
+				newnode->resultRelation = appinfo->child_relid;
+				/* Fix tlist resnos too, if it's inherited UPDATE */
+				if (newnode->commandType == CMD_UPDATE)
+					newnode->targetList =
+									adjust_inherited_tlist(newnode->targetList,
+														   appinfo);
+				break;
+			}
 		}
+
 		result = (Node *) newnode;
 	}
 	else
@@ -1771,13 +1786,29 @@ static Node *
 adjust_appendrel_attrs_mutator(Node *node,
 							   adjust_appendrel_attrs_context *context)
 {
-	AppendRelInfo *appinfo = context->appinfo;
+	List   *appinfos = context->appinfos;
+	ListCell   *lc;
+
+	/*
+	 * Catch a caller who wants to adjust expressions, but doesn't pass any
+	 * AppendRelInfo.
+	 */
+	Assert(appinfos && list_length(appinfos) >= 1);
 
 	if (node == NULL)
 		return NULL;
 	if (IsA(node, Var))
 	{
 		Var		   *var = (Var *) copyObject(node);
+		AppendRelInfo *appinfo;
+
+		foreach (lc, appinfos)
+		{
+			appinfo = lfirst(lc);
+
+			if (var->varno == appinfo->parent_relid)
+				break;
+		}
 
 		if (var->varlevelsup == 0 &&
 			var->varno == appinfo->parent_relid)
@@ -1860,29 +1891,54 @@ adjust_appendrel_attrs_mutator(Node *node,
 	{
 		CurrentOfExpr *cexpr = (CurrentOfExpr *) copyObject(node);
 
-		if (cexpr->cvarno == appinfo->parent_relid)
-			cexpr->cvarno = appinfo->child_relid;
+		foreach (lc, appinfos)
+		{
+			AppendRelInfo *appinfo = lfirst(lc);
+
+			if (cexpr->cvarno == appinfo->parent_relid)
+			{
+				cexpr->cvarno = appinfo->child_relid;
+				break;
+			}
+		}
 		return (Node *) cexpr;
 	}
 	if (IsA(node, RangeTblRef))
 	{
 		RangeTblRef *rtr = (RangeTblRef *) copyObject(node);
 
-		if (rtr->rtindex == appinfo->parent_relid)
-			rtr->rtindex = appinfo->child_relid;
+		foreach (lc, appinfos)
+		{
+			AppendRelInfo *appinfo = lfirst(lc);
+
+			if (rtr->rtindex == appinfo->parent_relid)
+			{
+				rtr->rtindex = appinfo->child_relid;
+				break;
+			}
+		}
 		return (Node *) rtr;
 	}
 	if (IsA(node, JoinExpr))
 	{
 		/* Copy the JoinExpr node with correct mutation of subnodes */
 		JoinExpr   *j;
+		AppendRelInfo *appinfo;
 
 		j = (JoinExpr *) expression_tree_mutator(node,
 											  adjust_appendrel_attrs_mutator,
 												 (void *) context);
 		/* now fix JoinExpr's rtindex (probably never happens) */
-		if (j->rtindex == appinfo->parent_relid)
-			j->rtindex = appinfo->child_relid;
+		foreach (lc, appinfos)
+		{
+			appinfo = lfirst(lc);
+
+			if (j->rtindex == appinfo->parent_relid)
+			{
+				j->rtindex = appinfo->child_relid;
+				break;
+			}
+		}
 		return (Node *) j;
 	}
 	if (IsA(node, PlaceHolderVar))
@@ -1895,9 +1951,7 @@ adjust_appendrel_attrs_mutator(Node *node,
 														 (void *) context);
 		/* now fix PlaceHolderVar's relid sets */
 		if (phv->phlevelsup == 0)
-			phv->phrels = adjust_relid_set(phv->phrels,
-										   appinfo->parent_relid,
-										   appinfo->child_relid);
+			phv->phrels = adjust_relid_set(phv->phrels, context->appinfos);
 		return (Node *) phv;
 	}
 	/* Shouldn't need to handle planner auxiliary nodes here */
@@ -1929,23 +1983,17 @@ adjust_appendrel_attrs_mutator(Node *node,
 
 		/* adjust relid sets too */
 		newinfo->clause_relids = adjust_relid_set(oldinfo->clause_relids,
-												  appinfo->parent_relid,
-												  appinfo->child_relid);
+													 context->appinfos);
 		newinfo->required_relids = adjust_relid_set(oldinfo->required_relids,
-													appinfo->parent_relid,
-													appinfo->child_relid);
+													 context->appinfos);
 		newinfo->outer_relids = adjust_relid_set(oldinfo->outer_relids,
-												 appinfo->parent_relid,
-												 appinfo->child_relid);
+													 context->appinfos);
 		newinfo->nullable_relids = adjust_relid_set(oldinfo->nullable_relids,
-													appinfo->parent_relid,
-													appinfo->child_relid);
+													   context->appinfos);
 		newinfo->left_relids = adjust_relid_set(oldinfo->left_relids,
-												appinfo->parent_relid,
-												appinfo->child_relid);
+												   context->appinfos);
 		newinfo->right_relids = adjust_relid_set(oldinfo->right_relids,
-												 appinfo->parent_relid,
-												 appinfo->child_relid);
+													context->appinfos);
 
 		/*
 		 * Reset cached derivative fields, since these might need to have
@@ -1977,19 +2025,38 @@ adjust_appendrel_attrs_mutator(Node *node,
 }
 
 /*
- * Substitute newrelid for oldrelid in a Relid set
+ * Replace parent relids by child relids in the copy of given relid set
+ * according to the given list of AppendRelInfos. The given relid set is
+ * returned as is if it contains no parent in the given list, otherwise, the
+ * given relid set is not changed.
  */
-static Relids
-adjust_relid_set(Relids relids, Index oldrelid, Index newrelid)
+Relids
+adjust_relid_set(Relids relids, List *append_rel_infos)
 {
-	if (bms_is_member(oldrelid, relids))
+	ListCell   *lc;
+	Bitmapset  *result = NULL;
+
+	foreach (lc, append_rel_infos)
 	{
-		/* Ensure we have a modifiable copy */
-		relids = bms_copy(relids);
-		/* Remove old, add new */
-		relids = bms_del_member(relids, oldrelid);
-		relids = bms_add_member(relids, newrelid);
+		AppendRelInfo	*appinfo = lfirst(lc);
+
+		/* Remove parent, add child */
+		if (bms_is_member(appinfo->parent_relid, relids))
+		{
+			/* Make a copy if we are changing the set. */
+			if (!result)
+				result = bms_copy(relids);
+
+			result = bms_del_member(result, appinfo->parent_relid);
+			result = bms_add_member(result, appinfo->child_relid);
+		}
 	}
+
+	/* Return new set if we modified the given set. */
+	if (result)
+		return result;
+
+	/* Else return the given relids set as is. */
 	return relids;
 }
 
@@ -2110,5 +2177,5 @@ adjust_appendrel_attrs_multilevel(PlannerInfo *root, Node *node,
 	else
 		Assert(parent_rel->reloptkind == RELOPT_BASEREL);
 	/* Now translate for this child */
-	return adjust_appendrel_attrs(root, node, appinfo);
+	return adjust_appendrel_attrs(root, node, list_make1(appinfo));
 }
diff --git a/src/include/optimizer/prep.h b/src/include/optimizer/prep.h
index 2b20b36..a02e06a 100644
--- a/src/include/optimizer/prep.h
+++ b/src/include/optimizer/prep.h
@@ -53,7 +53,7 @@ extern RelOptInfo *plan_set_operations(PlannerInfo *root);
 extern void expand_inherited_tables(PlannerInfo *root);
 
 extern Node *adjust_appendrel_attrs(PlannerInfo *root, Node *node,
-					   AppendRelInfo *appinfo);
+					   List *appinfos);
 
 extern Node *adjust_appendrel_attrs_multilevel(PlannerInfo *root, Node *node,
 								  RelOptInfo *child_rel);
-- 
1.7.9.5

0004-Refactor-build_join_rel.patchapplication/octet-stream; name=0004-Refactor-build_join_rel.patchDownload
From 5606e2526b997606e8485c00fc34ac1f5ccf2fe9 Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Mon, 6 Feb 2017 10:58:48 +0530
Subject: [PATCH 04/11] Refactor build_join_rel.

Partition-wise joins do not use build_join_rel() to build child-join relations,
but it still requires code to set foreign relation properties as well as code
to add join relation into PlannerInfo. Separate that code into
set_foreign_rel_properties() and add_join_rel() resp. to be called while
building child joins.
---
 src/backend/optimizer/util/relnode.c |  142 ++++++++++++++++++++--------------
 1 file changed, 83 insertions(+), 59 deletions(-)

diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index adc1db9..160ed6d 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -49,6 +49,9 @@ static List *subbuild_joinrel_restrictlist(RelOptInfo *joinrel,
 static List *subbuild_joinrel_joinlist(RelOptInfo *joinrel,
 						  List *joininfo_list,
 						  List *new_joininfo);
+static void set_foreign_rel_properties(RelOptInfo *joinrel,
+						   RelOptInfo *outer_rel, RelOptInfo *inner_rel);
+static void add_join_rel(PlannerInfo *root, RelOptInfo *joinrel);
 
 
 /*
@@ -327,6 +330,82 @@ find_join_rel(PlannerInfo *root, Relids relids)
 }
 
 /*
+ * set_foreign_rel_properties
+ *		Set up foreign-join fields if outer and inner relation are foreign
+ *		tables (or joins) belonging to the same server and assigned to the same
+ *		user to check access permissions as.
+ *
+ * In addition to an exact match of userid, we allow the case where one side
+ * has zero userid (implying current user) and the other side has explicit
+ * userid that happens to equal the current user; but in that case, pushdown of
+ * the join is only valid for the current user.  The useridiscurrent field
+ * records whether we had to make such an assumption for this join or any
+ * sub-join.
+ *
+ * Otherwise these fields are left invalid, so GetForeignJoinPaths will not be
+ * called for the join relation.
+ *
+ */
+static void
+set_foreign_rel_properties(RelOptInfo *joinrel, RelOptInfo *outer_rel,
+						   RelOptInfo *inner_rel)
+{
+	if (OidIsValid(outer_rel->serverid) &&
+		inner_rel->serverid == outer_rel->serverid)
+	{
+		if (inner_rel->userid == outer_rel->userid)
+		{
+			joinrel->serverid = outer_rel->serverid;
+			joinrel->userid = outer_rel->userid;
+			joinrel->useridiscurrent = outer_rel->useridiscurrent || inner_rel->useridiscurrent;
+			joinrel->fdwroutine = outer_rel->fdwroutine;
+		}
+		else if (!OidIsValid(inner_rel->userid) &&
+				 outer_rel->userid == GetUserId())
+		{
+			joinrel->serverid = outer_rel->serverid;
+			joinrel->userid = outer_rel->userid;
+			joinrel->useridiscurrent = true;
+			joinrel->fdwroutine = outer_rel->fdwroutine;
+		}
+		else if (!OidIsValid(outer_rel->userid) &&
+				 inner_rel->userid == GetUserId())
+		{
+			joinrel->serverid = outer_rel->serverid;
+			joinrel->userid = inner_rel->userid;
+			joinrel->useridiscurrent = true;
+			joinrel->fdwroutine = outer_rel->fdwroutine;
+		}
+	}
+}
+
+/*
+ * add_join_rel
+ *		Add given join relation to the list of join relations in the given
+ *		PlannerInfo. Also add it to the auxiliary hashtable if there is one.
+ */
+void
+add_join_rel(PlannerInfo *root, RelOptInfo *joinrel)
+{
+	/* GEQO requires us to append the new joinrel to the end of the list! */
+	root->join_rel_list = lappend(root->join_rel_list, joinrel);
+
+	/* store it into the auxiliary hashtable if there is one. */
+	if (root->join_rel_hash)
+	{
+		JoinHashEntry *hentry;
+		bool		found;
+
+		hentry = (JoinHashEntry *) hash_search(root->join_rel_hash,
+											   &(joinrel->relids),
+											   HASH_ENTER,
+											   &found);
+		Assert(!found);
+		hentry->join_rel = joinrel;
+	}
+}
+
+/*
  * build_join_rel
  *	  Returns relation entry corresponding to the union of two given rels,
  *	  creating a new relation entry if none already exists.
@@ -424,46 +503,8 @@ build_join_rel(PlannerInfo *root,
 	joinrel->joininfo = NIL;
 	joinrel->has_eclass_joins = false;
 
-	/*
-	 * Set up foreign-join fields if outer and inner relation are foreign
-	 * tables (or joins) belonging to the same server and assigned to the same
-	 * user to check access permissions as.  In addition to an exact match of
-	 * userid, we allow the case where one side has zero userid (implying
-	 * current user) and the other side has explicit userid that happens to
-	 * equal the current user; but in that case, pushdown of the join is only
-	 * valid for the current user.  The useridiscurrent field records whether
-	 * we had to make such an assumption for this join or any sub-join.
-	 *
-	 * Otherwise these fields are left invalid, so GetForeignJoinPaths will
-	 * not be called for the join relation.
-	 */
-	if (OidIsValid(outer_rel->serverid) &&
-		inner_rel->serverid == outer_rel->serverid)
-	{
-		if (inner_rel->userid == outer_rel->userid)
-		{
-			joinrel->serverid = outer_rel->serverid;
-			joinrel->userid = outer_rel->userid;
-			joinrel->useridiscurrent = outer_rel->useridiscurrent || inner_rel->useridiscurrent;
-			joinrel->fdwroutine = outer_rel->fdwroutine;
-		}
-		else if (!OidIsValid(inner_rel->userid) &&
-				 outer_rel->userid == GetUserId())
-		{
-			joinrel->serverid = outer_rel->serverid;
-			joinrel->userid = outer_rel->userid;
-			joinrel->useridiscurrent = true;
-			joinrel->fdwroutine = outer_rel->fdwroutine;
-		}
-		else if (!OidIsValid(outer_rel->userid) &&
-				 inner_rel->userid == GetUserId())
-		{
-			joinrel->serverid = outer_rel->serverid;
-			joinrel->userid = inner_rel->userid;
-			joinrel->useridiscurrent = true;
-			joinrel->fdwroutine = outer_rel->fdwroutine;
-		}
-	}
+	/* Compute information relevant to the foreign relations. */
+	set_foreign_rel_properties(joinrel, outer_rel, inner_rel);
 
 	/*
 	 * Create a new tlist containing just the vars that need to be output from
@@ -531,25 +572,8 @@ build_join_rel(PlannerInfo *root,
 		is_parallel_safe(root, (Node *) joinrel->reltarget->exprs))
 		joinrel->consider_parallel = true;
 
-	/*
-	 * Add the joinrel to the query's joinrel list, and store it into the
-	 * auxiliary hashtable if there is one.  NB: GEQO requires us to append
-	 * the new joinrel to the end of the list!
-	 */
-	root->join_rel_list = lappend(root->join_rel_list, joinrel);
-
-	if (root->join_rel_hash)
-	{
-		JoinHashEntry *hentry;
-		bool		found;
-
-		hentry = (JoinHashEntry *) hash_search(root->join_rel_hash,
-											   &(joinrel->relids),
-											   HASH_ENTER,
-											   &found);
-		Assert(!found);
-		hentry->join_rel = joinrel;
-	}
+	/* Add the joinrel to the PlannerInfo. */
+	add_join_rel(root, joinrel);
 
 	/*
 	 * Also, if dynamic-programming join search is active, add the new joinrel
-- 
1.7.9.5

0005-Add-function-find_param_path_info.patchapplication/octet-stream; name=0005-Add-function-find_param_path_info.patchDownload
From ec2984ab4ea387d5e91fbd354209ff45f114b603 Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Mon, 6 Feb 2017 12:14:06 +0530
Subject: [PATCH 05/11] Add function find_param_path_info.

The code to search ParamPathInfo for a set of required outer relations in the
list of ParamPathInfos of a given relation is duplicated in
get_*rel_parampathinfo() functions. Separate this code into
find_param_path_info() and call it from get_*rel_parampathinfo() functions.
---
 src/backend/optimizer/util/relnode.c |   46 ++++++++++++++++++++--------------
 1 file changed, 27 insertions(+), 19 deletions(-)

diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 160ed6d..19982dc 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -52,6 +52,8 @@ static List *subbuild_joinrel_joinlist(RelOptInfo *joinrel,
 static void set_foreign_rel_properties(RelOptInfo *joinrel,
 						   RelOptInfo *outer_rel, RelOptInfo *inner_rel);
 static void add_join_rel(PlannerInfo *root, RelOptInfo *joinrel);
+extern ParamPathInfo *find_param_path_info(RelOptInfo *rel,
+									  Relids required_outer);
 
 
 /*
@@ -1047,12 +1049,8 @@ get_baserel_parampathinfo(PlannerInfo *root, RelOptInfo *baserel,
 	Assert(!bms_overlap(baserel->relids, required_outer));
 
 	/* If we already have a PPI for this parameterization, just return it */
-	foreach(lc, baserel->ppilist)
-	{
-		ppi = (ParamPathInfo *) lfirst(lc);
-		if (bms_equal(ppi->ppi_req_outer, required_outer))
-			return ppi;
-	}
+	if ((ppi = find_param_path_info(baserel, required_outer)))
+		return ppi;
 
 	/*
 	 * Identify all joinclauses that are movable to this base rel given this
@@ -1289,12 +1287,8 @@ get_joinrel_parampathinfo(PlannerInfo *root, RelOptInfo *joinrel,
 	*restrict_clauses = list_concat(pclauses, *restrict_clauses);
 
 	/* If we already have a PPI for this parameterization, just return it */
-	foreach(lc, joinrel->ppilist)
-	{
-		ppi = (ParamPathInfo *) lfirst(lc);
-		if (bms_equal(ppi->ppi_req_outer, required_outer))
-			return ppi;
-	}
+	if ((ppi = find_param_path_info(joinrel, required_outer)))
+		return ppi;
 
 	/* Estimate the number of rows returned by the parameterized join */
 	rows = get_parameterized_joinrel_size(root, joinrel,
@@ -1333,7 +1327,6 @@ ParamPathInfo *
 get_appendrel_parampathinfo(RelOptInfo *appendrel, Relids required_outer)
 {
 	ParamPathInfo *ppi;
-	ListCell   *lc;
 
 	/* Unparameterized paths have no ParamPathInfo */
 	if (bms_is_empty(required_outer))
@@ -1342,12 +1335,8 @@ get_appendrel_parampathinfo(RelOptInfo *appendrel, Relids required_outer)
 	Assert(!bms_overlap(appendrel->relids, required_outer));
 
 	/* If we already have a PPI for this parameterization, just return it */
-	foreach(lc, appendrel->ppilist)
-	{
-		ppi = (ParamPathInfo *) lfirst(lc);
-		if (bms_equal(ppi->ppi_req_outer, required_outer))
-			return ppi;
-	}
+	if ((ppi = find_param_path_info(appendrel, required_outer)))
+		return ppi;
 
 	/* Else build the ParamPathInfo */
 	ppi = makeNode(ParamPathInfo);
@@ -1358,3 +1347,22 @@ get_appendrel_parampathinfo(RelOptInfo *appendrel, Relids required_outer)
 
 	return ppi;
 }
+
+/*
+ * Returns a ParamPathInfo for outer relations specified by required_outer, if
+ * already available in the given rel. Returns NULL otherwise.
+ */
+ParamPathInfo *
+find_param_path_info(RelOptInfo *rel, Relids required_outer)
+{
+	ListCell   *lc;
+
+	foreach(lc, rel->ppilist)
+	{
+		ParamPathInfo  *ppi = (ParamPathInfo *) lfirst(lc);
+		if (bms_equal(ppi->ppi_req_outer, required_outer))
+			return ppi;
+	}
+
+	return NULL;
+}
-- 
1.7.9.5

0006-Canonical-partition-scheme.patchapplication/octet-stream; name=0006-Canonical-partition-scheme.patchDownload
From 53c7c239715824278c2abe19c15abdb1ed3d7d91 Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Tue, 7 Feb 2017 10:47:49 +0530
Subject: [PATCH 06/11] Canonical partition scheme.

For a single level partitioned table, annotate RelOptInfo of a partitioned
table with canonical partition scheme. All partitioned tables, with the same
partitioning scheme share the same canonical partitioning scheme. We store the
RelOptInfo's corresponding to the partitions in RelOptInfo of the partitioned
table. Those are arranged in the same order as the partition bound indices in
the partition scheme.

We do not handle multi-level partitioned tables since inheritance hierarchy
does not retain the partition hierarchy.  All the partitions at any level
appear as children of the top-level partitioned table. Thus making it hard to
associate a partition relation with corresponding partition bounds.

Multi-level partitioned tables will be handled in a separate patch.
---
 src/backend/optimizer/path/allpaths.c |   48 +++++++
 src/backend/optimizer/util/plancat.c  |  232 +++++++++++++++++++++++++++++++++
 src/include/nodes/relation.h          |   51 ++++++++
 3 files changed, 331 insertions(+)

diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index d8fac14..0eb56f3 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -829,6 +829,7 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 	double	   *parent_attrsizes;
 	int			nattrs;
 	ListCell   *l;
+	int			nparts;
 
 	/*
 	 * Initialize to compute size estimates for whole append relation.
@@ -850,6 +851,18 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 	nattrs = rel->max_attr - rel->min_attr + 1;
 	parent_attrsizes = (double *) palloc0(nattrs * sizeof(double));
 
+	/*
+	 * For a partitioned table, allocate an array to hold RelOptInfo's of the
+	 * partitions. It will be filled while handling the children below.
+	 */
+	if (rel->part_scheme)
+	{
+		nparts = rel->part_scheme->nparts;
+		rel->part_rels = (RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
+	}
+	else
+		nparts = 0;
+
 	foreach(l, root->append_rel_list)
 	{
 		AppendRelInfo *appinfo = (AppendRelInfo *) lfirst(l);
@@ -879,6 +892,30 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		Assert(childrel->reloptkind == RELOPT_OTHER_MEMBER_REL);
 
 		/*
+		 * Two partitioned tables with the same partitioning scheme, have their
+		 * partition bounds arranged in the same order. The order of partition
+		 * OIDs in RelOptInfo corresponds to the partition bound order. Thus
+		 * the OIDs of matching partitions from both the tables are placed at
+		 * the same position in the array of partition OIDs in the respective
+		 * RelOptInfos. Arranging RelOptInfos of partitions in the same order
+		 * as their OIDs makes it easy to find the RelOptInfos of matching
+		 * partitions for partition-wise join.
+		 */
+		if (rel->part_scheme)
+		{
+			int		cnt_parts;
+
+			for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+			{
+				if (rel->part_oids[cnt_parts] == childRTE->relid)
+				{
+					Assert(!rel->part_rels[cnt_parts]);
+					rel->part_rels[cnt_parts] = childrel;
+				}
+			}
+		}
+
+		/*
 		 * We have to copy the parent's targetlist and quals to the child,
 		 * with appropriate substitution of variables.  However, only the
 		 * baserestrictinfo quals are needed before we can check for
@@ -1130,6 +1167,17 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		}
 	}
 
+	/* Should have found all the childrels of a partitioned relation. */
+	if (rel->part_scheme)
+	{
+		int		cnt_parts;
+
+		for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+			if (!rel->part_rels[cnt_parts])
+				elog(ERROR, "could not find the RelOptInfo of a partition with oid %u",
+					 rel->part_oids[cnt_parts]);
+	}
+
 	if (has_live_children)
 	{
 		/*
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index 7836e6b..01ba885 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -27,6 +27,7 @@
 #include "catalog/catalog.h"
 #include "catalog/dependency.h"
 #include "catalog/heap.h"
+#include "catalog/pg_inherits_fn.h"
 #include "catalog/partition.h"
 #include "catalog/pg_am.h"
 #include "foreign/fdwapi.h"
@@ -63,6 +64,13 @@ static List *get_relation_constraints(PlannerInfo *root,
 						 bool include_notnull);
 static List *build_index_tlist(PlannerInfo *root, IndexOptInfo *index,
 				  Relation heapRelation);
+static List **build_baserel_partition_key_exprs(Relation relation,
+												Index varno);
+static PartitionScheme find_partition_scheme(struct PlannerInfo *root,
+											 Relation rel);
+static void get_relation_partition_info(PlannerInfo *root, RelOptInfo *rel,
+										Relation relation, bool inhparent);
+
 
 
 /*
@@ -412,6 +420,9 @@ get_relation_info(PlannerInfo *root, Oid relationObjectId, bool inhparent,
 	/* Collect info about relation's foreign keys, if relevant */
 	get_relation_foreign_keys(root, rel, relation, inhparent);
 
+	/* Collect info about relation's partitioning scheme, if any. */
+	get_relation_partition_info(root, rel, relation, inhparent);
+
 	heap_close(relation, NoLock);
 
 	/*
@@ -1716,3 +1727,224 @@ has_row_triggers(PlannerInfo *root, Index rti, CmdType event)
 	heap_close(relation, NoLock);
 	return result;
 }
+
+/*
+ * get_relation_partition_info
+ *
+ * Retrieves partitioning information for a given relation.
+ *
+ * Partitioning scheme, partition key expressions and OIDs of partitions are
+ * added to the given RelOptInfo. A partitioned table can participate in the
+ * query as a simple relation or an inheritance parent. Only the later can have
+ * child relations, and hence partitions. From the point of view of the query
+ * optimizer only such relations are considered to be partitioned. Hence
+ * partitioning information is set only for an inheritance parent.
+ */
+static void
+get_relation_partition_info(PlannerInfo *root, RelOptInfo *rel,
+							Relation relation, bool inhparent)
+{
+	/* No partitioning information for an unpartitioned relation. */
+	if (relation->rd_rel->relkind != RELKIND_PARTITIONED_TABLE ||
+		!inhparent ||
+		!(rel->part_scheme = find_partition_scheme(root, relation)))
+	{
+		rel->partexprs = NULL;
+		rel->part_rels = NULL;
+		rel->part_oids = NULL;
+		return;
+	}
+
+	rel->partexprs = build_baserel_partition_key_exprs(relation, rel->relid);
+	rel->part_oids = RelationGetPartitionDesc(relation)->oids;
+
+	/*
+	 * RelOptInfos of the partitions will be filled in when we build those for
+	 * the child relations.
+	 */
+	rel->part_rels = NULL;
+	return;
+}
+
+/*
+ * find_partition_scheme
+ *
+ * The function returns a canonical partition scheme which exactly matches the
+ * partitioning properties of the given relation if one exists in the of
+ * canonical partitioning schemes maintained in PlannerInfo. If none of the
+ * existing partitioning schemes match, the function creates a canonical
+ * partition scheme and adds it to the list.
+ *
+ * For an unpartitioned table or for a multi-level partitioned table it returns
+ * NULL. See comments in the function for more details.
+ */
+static PartitionScheme
+find_partition_scheme(PlannerInfo *root, Relation relation)
+{
+	PartitionKey	part_key = RelationGetPartitionKey(relation);
+	PartitionDesc	part_desc = RelationGetPartitionDesc(relation);
+	ListCell	   *lc;
+	int		nparts;
+	int		partnatts;
+	int		cnt_pks;
+	int		cnt_parts;
+	PartitionScheme	part_scheme = NULL;
+
+	/* No partition scheme for an unpartitioned relation. */
+	if (!part_desc || !part_key)
+		return NULL;
+
+	nparts = part_desc->nparts;
+	partnatts = part_key->partnatts;
+
+	/*
+	 * For a multi-level partitioned table, we do not retain the partitioning
+	 * hierarchy while expanding RTE for the topmost parent. Thus the number of
+	 * children as per root->append_rel_list does not match the number of
+	 * partitions specified in the partition descriptor and hence the
+	 * partitioning scheme of a multi-partitioned table does not reflect the
+	 * true picture. So for now, treat a multi-partitioned table as not
+	 * partitioned.
+	 */
+	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	{
+		if (has_subclass(part_desc->oids[cnt_parts]))
+			return NULL;
+	}
+
+	/* Search for a matching partition scheme and return if found one. */
+	foreach (lc, root->part_schemes)
+	{
+		part_scheme = lfirst(lc);
+
+		/* Match number of partitions and partitioning strategy. */
+		if (nparts != part_scheme->nparts ||
+			part_key->strategy != part_scheme->strategy ||
+			partnatts != part_scheme->partnatts)
+			continue;
+
+		/* Match the partition key types. */
+		for (cnt_pks = 0; cnt_pks < partnatts; cnt_pks++)
+		{
+			/*
+			 * For types, it suffices to match the type id, mod and collation;
+			 * len, byval and align are depedent on the first two.
+			 */
+			if (part_key->partopfamily[cnt_pks] != part_scheme->partopfamily[cnt_pks] ||
+				part_key->partopcintype[cnt_pks] != part_scheme->partopcintype[cnt_pks] ||
+				part_key->parttypid[cnt_pks] != part_scheme->key_types[cnt_pks] ||
+				part_key->parttypmod[cnt_pks] != part_scheme->key_typmods[cnt_pks] ||
+				part_key->parttypcoll[cnt_pks] != part_scheme->key_collations[cnt_pks])
+				break;
+		}
+
+		/* Some partition key didn't match. Check next partitioning scheme. */
+		if (cnt_pks < partnatts)
+			continue;
+
+		if (!partition_bounds_equal(part_key, part_desc->boundinfo,
+									part_scheme->boundinfo))
+			continue;
+
+		/* Found matching partition scheme. */
+		return part_scheme;
+	}
+
+	/* Did not find matching partition scheme. Create one. */
+	part_scheme = (PartitionScheme) palloc0(sizeof(PartitionSchemeData));
+
+	/* Copy partition bounds/lists. */
+	part_scheme->nparts = part_desc->nparts;
+	part_scheme->strategy = part_key->strategy;
+	part_scheme->boundinfo = part_desc->boundinfo;
+
+	/* Store partition key information. */
+	part_scheme->partnatts = part_key->partnatts;
+
+	part_scheme->partopfamily = (Oid *) palloc(sizeof(Oid) * partnatts);
+	memcpy(part_scheme->partopfamily, part_key->partopfamily,
+		   sizeof(Oid) * partnatts);
+
+	part_scheme->partopcintype = (Oid *) palloc(sizeof(Oid) * partnatts);
+	memcpy(part_scheme->partopcintype, part_key->partopcintype,
+		   sizeof(Oid) * partnatts);
+
+	part_scheme->key_types = (Oid *) palloc(sizeof(Oid) * partnatts);
+	memcpy(part_scheme->key_types, part_key->parttypid,
+		   sizeof(Oid) * partnatts);
+
+	part_scheme->key_typmods = (int32 *) palloc(sizeof(int32) * partnatts);
+	memcpy(part_scheme->key_typmods, part_key->parttypmod,
+		   sizeof(int32) * partnatts);
+
+	part_scheme->key_collations = (Oid *) palloc(sizeof(Oid) * partnatts);
+	memcpy(part_scheme->key_collations, part_key->parttypcoll,
+		   sizeof(Oid) * partnatts);
+
+	/* Add the partitioning scheme to PlannerInfo. */
+	root->part_schemes = lappend(root->part_schemes, part_scheme);
+
+	return part_scheme;
+}
+
+/*
+ * build_baserel_partition_key_exprs
+ *
+ * Collect partition key expressions for a given base relation. The function
+ * converts any single column partition keys into corresponding Var nodes. It
+ * restamps Var nodes in partition key expressions by given varno. The
+ * partition key expressions are returned as an array of single element Lists
+ * to be stored in RelOptInfo of the base relation.
+ */
+static List **
+build_baserel_partition_key_exprs(Relation relation, Index varno)
+{
+	PartitionKey	part_key = RelationGetPartitionKey(relation);
+	int		num_pkexprs;
+	int		cnt_pke;
+	List		  **partexprs;
+	ListCell	   *lc;
+
+	if (!part_key || part_key->partnatts <= 0)
+		return NULL;
+
+	num_pkexprs = part_key->partnatts;
+	partexprs = (List **) palloc(sizeof(List *) * num_pkexprs);
+	lc = list_head(part_key->partexprs);
+
+	for (cnt_pke = 0; cnt_pke < num_pkexprs; cnt_pke++)
+	{
+		AttrNumber attno = part_key->partattrs[cnt_pke];
+		Expr	  *pkexpr;
+
+		if (attno != InvalidAttrNumber)
+		{
+			/* Single column partition key is stored as a Var node. */
+			Form_pg_attribute att_tup;
+
+			if (attno < 0)
+				att_tup = SystemAttributeDefinition(attno,
+												relation->rd_rel->relhasoids);
+			else
+				att_tup = relation->rd_att->attrs[attno - 1];
+
+			pkexpr = (Expr *) makeVar(varno, attno, att_tup->atttypid,
+									  att_tup->atttypmod,
+									  att_tup->attcollation, 0);
+		}
+		else
+		{
+			if (lc == NULL)
+				elog(ERROR, "wrong number of partition key expressions");
+
+			/* Re-stamp the expressions with given varno. */
+			pkexpr = (Expr *) copyObject(lfirst(lc));
+			ChangeVarNodes((Node *) pkexpr, 1, varno, 0);
+			lc = lnext(lc);
+		}
+
+		partexprs[cnt_pke] = list_make1(pkexpr);
+	}
+
+	return partexprs;
+}
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 643be54..4f99184 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 "catalog/partition.h"
 #include "lib/stringinfo.h"
 #include "nodes/params.h"
 #include "nodes/parsenodes.h"
@@ -261,6 +262,9 @@ typedef struct PlannerInfo
 	List	   *distinct_pathkeys;		/* distinctClause pathkeys, if any */
 	List	   *sort_pathkeys;	/* sortClause pathkeys, if any */
 
+	List	   *part_schemes;	/* Canonicalised partition schemes
+								 * used in the query. */
+
 	List	   *initial_rels;	/* RelOptInfos we are now trying to join */
 
 	/* Use fetch_upper_rel() to get any particular upper rel */
@@ -321,6 +325,38 @@ typedef struct PlannerInfo
 	((root)->simple_rte_array ? (root)->simple_rte_array[rti] : \
 	 rt_fetch(rti, (root)->parse->rtable))
 
+/*
+ * Partitioning scheme
+ *		Structure to hold partitioning scheme for a given relation.
+ *
+ * Multiple relations may be partitioned in the same way. The relations
+ * resulting from joining such relations may be partitioned in the same way as
+ * the joining relations. Similarly, relations derived from such relations by
+ * grouping, sorting may be partitioned in the same way as the underlying
+ * scan relations. All such relations partitioned in the same way share the
+ * partitioning scheme.
+ *
+ * PlannerInfo stores a list of distinct "canonical" partitioning schemes.
+ * RelOptInfo of a partitioned relation holds the pointer to "canonical"
+ * partitioning scheme.
+ */
+typedef struct PartitionSchemeData
+{
+	/* Information about partitions */
+	int			nparts;			/* number of partitions */
+	PartitionBoundInfo boundinfo;	/* Partition bounds/lists */
+
+	/* Information about partition keys */
+	char		strategy;		/* partition strategy */
+	int16		partnatts;		/* number of partition attributes */
+	Oid		   *partopfamily;	/* OIDs of operator families */
+	Oid		   *partopcintype;	/* OIDs of opclass declared input data types */
+	Oid		   *key_types;		/* OIDs of partition key data types. */
+	int32	   *key_typmods;	/* typmods of partition keys. */
+	Oid		   *key_collations;	/* OIDs of collations of partition keys. */
+} PartitionSchemeData;
+
+typedef struct PartitionSchemeData *PartitionScheme;
 
 /*----------
  * RelOptInfo
@@ -531,6 +567,7 @@ typedef struct RelOptInfo
 	PlannerInfo *subroot;		/* if subquery */
 	List	   *subplan_params; /* if subquery */
 	int			rel_parallel_workers;	/* wanted number of parallel workers */
+	Oid		   *part_oids;		/* OIDs of partitions */
 
 	/* Information about foreign tables and foreign joins */
 	Oid			serverid;		/* identifies server for the table or join */
@@ -549,6 +586,20 @@ typedef struct RelOptInfo
 	List	   *joininfo;		/* RestrictInfo structures for join clauses
 								 * involving this rel */
 	bool		has_eclass_joins;		/* T means joininfo is incomplete */
+
+	/* For all the partitioned relations. */
+	PartitionScheme	    part_scheme;	/* Partitioning scheme. */
+	struct RelOptInfo **part_rels;		/* Array of RelOptInfos of partitions,
+										 * stored in the same order as bounds
+										 * or lists in PartitionScheme.
+										 */
+	List			  **partexprs;		/* Array of list of partition key
+										 * expressions. For base relations
+										 * these are one element lists. For
+										 * join there may be as many elements
+										 * as the number of joining
+										 * relations.
+										 */
 } RelOptInfo;
 
 /*
-- 
1.7.9.5

0007-Partition-wise-join-tests.patchapplication/octet-stream; name=0007-Partition-wise-join-tests.patchDownload
From 82e4452ff549ede9f34684d363f66291a3bb632b Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Tue, 7 Feb 2017 11:57:30 +0530
Subject: [PATCH 07/11] Partition-wise join tests.

This file does not contain tests for joins between multi-leveled partitioned
tables. Those will be added later.
---
 src/test/regress/expected/partition_join.out | 4114 ++++++++++++++++++++++++++
 src/test/regress/parallel_schedule           |    2 +-
 src/test/regress/serial_schedule             |    1 +
 src/test/regress/sql/partition_join.sql      |  515 ++++
 4 files changed, 4631 insertions(+), 1 deletion(-)
 create mode 100644 src/test/regress/expected/partition_join.out
 create mode 100644 src/test/regress/sql/partition_join.sql

diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
new file mode 100644
index 0000000..18238fa
--- /dev/null
+++ b/src/test/regress/expected/partition_join.out
@@ -0,0 +1,4114 @@
+--
+-- PARTITION_JOIN
+-- Test partition-wise join between partitioned tables
+--
+--
+-- partitioned by a single column
+--
+CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+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, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1 AS SELECT * FROM prt1;
+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_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, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+CREATE TABLE uprt2 AS SELECT * FROM prt2;
+-- inner join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on public.prt2_p1 t2
+                     Output: t2.b, t2.c
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+               Hash Cond: (t2_1.b = t1_2.a)
+               ->  Seq Scan on public.prt2_p2 t2_1
+                     Output: t2_1.b, t2_1.c
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+               Hash Cond: (t2_2.b = t1_1.a)
+               ->  Seq Scan on public.prt2_p3 t2_2
+                     Output: t2_2.b, t2_2.c
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 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)
+
+-- left outer join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on public.prt2_p1 t2
+                     Output: t2.b, t2.c
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+               Hash Cond: (t2_1.b = t1_2.a)
+               ->  Seq Scan on public.prt2_p2 t2_1
+                     Output: t2_1.b, t2_1.c
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+               Hash Cond: (t2_2.b = t1_1.a)
+               ->  Seq Scan on public.prt2_p3 t2_2
+                     Output: t2_2.b, t2_2.c
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+-- right outer join
+EXPLAIN (VERBOSE, 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.b % 25 = 0 ORDER BY t1.a, t2.b;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: (t1.a = t2.b)
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t2.b, t2.c
+                           ->  Seq Scan on public.prt2_p1 t2
+                                 Output: t2.b, t2.c
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Hash Cond: (t1_2.a = t2_1.b)
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c
+                           ->  Seq Scan on public.prt2_p2 t2_1
+                                 Output: t2_1.b, t2_1.c
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Hash Cond: (t1_1.a = t2_2.b)
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c
+                           ->  Seq Scan on public.prt2_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+                                 Filter: ((t2_2.b % 25) = 0)
+(36 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 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)
+
+-- full outer join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+   Sort Key: prt1_p1.a, prt2_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+               Hash Cond: (prt1_p1.a = prt2_p1.b)
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c
+                     Filter: ((prt1_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p1.b, prt2_p1.c
+                     ->  Seq Scan on public.prt2_p1
+                           Output: prt2_p1.b, prt2_p1.c
+                           Filter: ((prt2_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt1_p2.a = prt2_p2.b)
+               ->  Seq Scan on public.prt1_p2
+                     Output: prt1_p2.a, prt1_p2.c
+                     Filter: ((prt1_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p2.b, prt2_p2.c
+                     ->  Seq Scan on public.prt2_p2
+                           Output: prt2_p2.b, prt2_p2.c
+                           Filter: ((prt2_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c
+               Hash Cond: (prt1_p3.a = prt2_p3.b)
+               ->  Seq Scan on public.prt1_p3
+                     Output: prt1_p3.a, prt1_p3.c
+                     Filter: ((prt1_p3.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p3.b, prt2_p3.c
+                     ->  Seq Scan on public.prt2_p3
+                           Output: prt2_p3.b, prt2_p3.c
+                           Filter: ((prt2_p3.b % 25) = 0)
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 t2 WHERE t2.b % 25 = 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)
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+   Sort Key: prt1_p1.a, prt2_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+               Hash Cond: (prt1_p1.a = prt2_p1.b)
+               Filter: (((50) = prt1_p1.b) OR ((75) = prt2_p1.b))
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c, prt1_p1.b, 50
+                     Filter: ((prt1_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p1.b, prt2_p1.c, (75)
+                     ->  Seq Scan on public.prt2_p1
+                           Output: prt2_p1.b, prt2_p1.c, 75
+                           Filter: ((prt2_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt1_p2.a = prt2_p2.b)
+               Filter: (((50) = prt1_p2.b) OR ((75) = prt2_p2.b))
+               ->  Seq Scan on public.prt1_p2
+                     Output: prt1_p2.a, prt1_p2.c, prt1_p2.b, 50
+                     Filter: ((prt1_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p2.b, prt2_p2.c, (75)
+                     ->  Seq Scan on public.prt2_p2
+                           Output: prt2_p2.b, prt2_p2.c, 75
+                           Filter: ((prt2_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c
+               Hash Cond: (prt1_p3.a = prt2_p3.b)
+               Filter: (((50) = prt1_p3.b) OR ((75) = prt2_p3.b))
+               ->  Seq Scan on public.prt1_p3
+                     Output: prt1_p3.a, prt1_p3.c, prt1_p3.b, 50
+                     Filter: ((prt1_p3.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p3.b, prt2_p3.c, (75)
+                     ->  Seq Scan on public.prt2_p3
+                           Output: prt2_p3.b, prt2_p3.c, 75
+                           Filter: ((prt2_p3.b % 25) = 0)
+(40 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+ a  |  c   | b  |  c   
+----+------+----+------
+ 50 | 0050 |    | 
+    |      | 75 | 0075
+(2 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+ a  |  c   | b  |  c   
+----+------+----+------
+ 50 | 0050 |    | 
+    |      | 75 | 0075
+(2 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, (25), prt2_p1.b, prt2_p1.c, (50)
+   Sort Key: prt1_p1.a, prt2_p1.b
+   ->  Result
+         Output: prt1_p1.a, prt1_p1.c, (25), prt2_p1.b, prt2_p1.c, (50)
+         ->  Append
+               ->  Hash Full Join
+                     Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, (25), (50)
+                     Hash Cond: (prt1_p1.a = prt2_p1.b)
+                     ->  Seq Scan on public.prt1_p1
+                           Output: prt1_p1.a, prt1_p1.c, 25
+                           Filter: ((prt1_p1.a % 25) = 0)
+                     ->  Hash
+                           Output: prt2_p1.b, prt2_p1.c, (50)
+                           ->  Seq Scan on public.prt2_p1
+                                 Output: prt2_p1.b, prt2_p1.c, 50
+                                 Filter: ((prt2_p1.b % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c, (25), (50)
+                     Hash Cond: (prt1_p2.a = prt2_p2.b)
+                     ->  Seq Scan on public.prt1_p2
+                           Output: prt1_p2.a, prt1_p2.c, 25
+                           Filter: ((prt1_p2.a % 25) = 0)
+                     ->  Hash
+                           Output: prt2_p2.b, prt2_p2.c, (50)
+                           ->  Seq Scan on public.prt2_p2
+                                 Output: prt2_p2.b, prt2_p2.c, 50
+                                 Filter: ((prt2_p2.b % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c, (25), (50)
+                     Hash Cond: (prt1_p3.a = prt2_p3.b)
+                     ->  Seq Scan on public.prt1_p3
+                           Output: prt1_p3.a, prt1_p3.c, 25
+                           Filter: ((prt1_p3.a % 25) = 0)
+                     ->  Hash
+                           Output: prt2_p3.b, prt2_p3.c, (50)
+                           ->  Seq Scan on public.prt2_p3
+                                 Output: prt2_p3.b, prt2_p3.c, 50
+                                 Filter: ((prt2_p3.b % 25) = 0)
+(39 rows)
+
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 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)
+
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM uprt2 WHERE uprt2.b % 25 = 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)
+
+-- Join with pruned partitions from joining relations
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on public.prt2_p2 t2
+                     Output: t2.b, t2.c
+                     Filter: (t2.b > 250)
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.prt1_p2 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a < 450) AND ((t1.a % 25) = 0))
+(15 rows)
+
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+(1 row)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a < 450 AND t2.b > 250 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+(1 row)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                    QUERY PLAN                                    
+----------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, b, c
+   Sort Key: prt1_p1.a, b
+   ->  Append
+         ->  Nested Loop Left Join
+               Output: prt1_p1.a, prt1_p1.c, b, c
+               Join Filter: (prt1_p1.a = b)
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c
+                     Filter: ((prt1_p1.a < 450) AND ((prt1_p1.a % 25) = 0))
+               ->  Result
+                     Output: b, c
+                     One-Time Filter: false
+         ->  Hash Right Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt2_p2.b = prt1_p2.a)
+               ->  Seq Scan on public.prt2_p2
+                     Output: prt2_p2.b, prt2_p2.c
+                     Filter: (prt2_p2.b > 250)
+               ->  Hash
+                     Output: prt1_p2.a, prt1_p2.c
+                     ->  Seq Scan on public.prt1_p2
+                           Output: prt1_p2.a, prt1_p2.c
+                           Filter: ((prt1_p2.a < 450) AND ((prt1_p2.a % 25) = 0))
+(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.a % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.a % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                       QUERY PLAN                                       
+----------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+   Sort Key: prt1_p2.a, prt2_p2.b
+   ->  Result
+         Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: prt2_p2.b, prt2_p2.c, prt1_p2.a, prt1_p2.c
+                     Hash Cond: (prt1_p2.a = prt2_p2.b)
+                     ->  Seq Scan on public.prt1_p2
+                           Output: prt1_p2.a, prt1_p2.c
+                           Filter: (prt1_p2.a < 450)
+                     ->  Hash
+                           Output: prt2_p2.b, prt2_p2.c
+                           ->  Seq Scan on public.prt2_p2
+                                 Output: prt2_p2.b, prt2_p2.c
+                                 Filter: ((prt2_p2.b > 250) AND ((prt2_p2.a % 25) = 0))
+               ->  Nested Loop Left Join
+                     Output: prt2_p3.b, prt2_p3.c, a, c
+                     Join Filter: (a = prt2_p3.b)
+                     ->  Seq Scan on public.prt2_p3
+                           Output: prt2_p3.b, prt2_p3.c
+                           Filter: ((prt2_p3.b > 250) AND ((prt2_p3.a % 25) = 0))
+                     ->  Result
+                           Output: a, c
+                           One-Time Filter: false
+(26 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+     |      | 375 | 0375
+     |      | 450 | 0450
+     |      | 525 | 0525
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+     |      | 375 | 0375
+     |      | 450 | 0450
+     |      | 525 | 0525
+(4 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+                                    QUERY PLAN                                    
+----------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, b, c
+   Sort Key: prt1_p1.a, b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_p1.a, prt1_p1.c, b, c
+               Hash Cond: (prt1_p1.a = b)
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c
+                     Filter: ((prt1_p1.a < 450) AND ((prt1_p1.a % 25) = 0))
+               ->  Hash
+                     Output: b, c
+                     ->  Result
+                           Output: b, c
+                           One-Time Filter: false
+         ->  Hash Full Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt1_p2.a = prt2_p2.b)
+               ->  Seq Scan on public.prt1_p2
+                     Output: prt1_p2.a, prt1_p2.c
+                     Filter: ((prt1_p2.a < 450) AND ((prt1_p2.a % 25) = 0))
+               ->  Hash
+                     Output: prt2_p2.b, prt2_p2.c
+                     ->  Seq Scan on public.prt2_p2
+                           Output: prt2_p2.b, prt2_p2.c
+                           Filter: ((prt2_p2.b > 250) AND ((prt2_p2.b % 25) = 0))
+         ->  Hash Full Join
+               Output: a, c, prt2_p3.b, prt2_p3.c
+               Hash Cond: (prt2_p3.b = a)
+               ->  Seq Scan on public.prt2_p3
+                     Output: prt2_p3.b, prt2_p3.c
+                     Filter: ((prt2_p3.b > 250) AND ((prt2_p3.b % 25) = 0))
+               ->  Hash
+                     Output: a, c
+                     ->  Result
+                           Output: a, c
+                           One-Time Filter: false
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b 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)
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+                     QUERY PLAN                     
+----------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Output: t1.a, t1.b, t1.c
+         Merge Cond: (t1.a = t1_3.b)
+         ->  Sort
+               Output: t1.a, t1.b, t1.c
+               Sort Key: t1.a
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+         ->  Sort
+               Output: t1_3.b
+               Sort Key: t1_3.b
+               ->  Seq Scan on public.prt2_p1 t1_3
+                     Output: t1_3.b
+                     Filter: ((t1_3.b % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_2.a, t1_2.b, t1_2.c
+         Merge Cond: (t1_2.a = t1_4.b)
+         ->  Sort
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Sort Key: t1_2.a
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+         ->  Sort
+               Output: t1_4.b
+               Sort Key: t1_4.b
+               ->  Seq Scan on public.prt2_p2 t1_4
+                     Output: t1_4.b
+                     Filter: ((t1_4.b % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_1.a, t1_1.b, t1_1.c
+         Merge Cond: (t1_1.a = t1_5.b)
+         ->  Sort
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Sort Key: t1_1.a
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+         ->  Sort
+               Output: t1_5.b
+               Sort Key: t1_5.b
+               ->  Seq Scan on public.prt2_p3 t1_5
+                     Output: t1_5.b
+                     Filter: ((t1_5.b % 25) = 0)
+(47 rows)
+
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+-- lateral reference
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+                                             QUERY PLAN                                              
+-----------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   Sort Key: t1.a
+   ->  Result
+         Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+         ->  Append
+               ->  Nested Loop Left Join
+                     Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a))
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.b, t1.c
+                           Filter: ((t1.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+                           Hash Cond: (t3.b = t2.a)
+                           ->  Seq Scan on public.prt2_p1 t3
+                                 Output: t3.a, t3.b
+                           ->  Hash
+                                 Output: t2.a
+                                 ->  Seq Scan on public.prt1_p1 t2
+                                       Output: t2.a
+                                       Filter: (t1.a = t2.a)
+               ->  Nested Loop Left Join
+                     Output: t1_2.a, t1_2.b, t1_2.c, t2_2.a, t3_1.a, (LEAST(t1_2.a, t2_2.a, t3_1.a))
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.b, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2_2.a, t3_1.a, LEAST(t1_2.a, t2_2.a, t3_1.a)
+                           Hash Cond: (t3_1.b = t2_2.a)
+                           ->  Seq Scan on public.prt2_p2 t3_1
+                                 Output: t3_1.a, t3_1.b
+                           ->  Hash
+                                 Output: t2_2.a
+                                 ->  Seq Scan on public.prt1_p2 t2_2
+                                       Output: t2_2.a
+                                       Filter: (t1_2.a = t2_2.a)
+               ->  Nested Loop Left Join
+                     Output: t1_1.a, t1_1.b, t1_1.c, t2_1.a, t3_2.a, (LEAST(t1_1.a, t2_1.a, t3_2.a))
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.b, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2_1.a, t3_2.a, LEAST(t1_1.a, t2_1.a, t3_2.a)
+                           Hash Cond: (t3_2.b = t2_1.a)
+                           ->  Seq Scan on public.prt2_p3 t3_2
+                                 Output: t3_2.a, t3_2.b
+                           ->  Hash
+                                 Output: t2_1.a
+                                 ->  Seq Scan on public.prt1_p3 t2_1
+                                       Output: t2_1.a
+                                       Filter: (t1_1.a = t2_1.a)
+(51 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   Sort Key: t1.a
+   ->  Nested Loop Left Join
+         Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+         ->  Append
+               ->  Seq Scan on public.prt1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Seq Scan on public.prt1_p1 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Seq Scan on public.prt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Seq Scan on public.prt1_p2 t1_3
+                     Output: t1_3.a, t1_3.b, t1_3.c
+                     Filter: ((t1_3.a % 25) = 0)
+         ->  Append
+               ->  Hash Join
+                     Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+                     Hash Cond: (t3.b = t2.a)
+                     ->  Seq Scan on public.prt2_p1 t3
+                           Output: t3.a, t3.b
+                     ->  Hash
+                           Output: t2.a
+                           ->  Seq Scan on public.prt1_p1 t2
+                                 Output: t2.a
+                                 Filter: (t1.b = t2.a)
+               ->  Hash Join
+                     Output: t2_2.a, t3_1.a, LEAST(t1.a, t2_2.a, t3_1.a)
+                     Hash Cond: (t3_1.b = t2_2.a)
+                     ->  Seq Scan on public.prt2_p2 t3_1
+                           Output: t3_1.a, t3_1.b
+                     ->  Hash
+                           Output: t2_2.a
+                           ->  Seq Scan on public.prt1_p2 t2_2
+                                 Output: t2_2.a
+                                 Filter: (t1.b = t2_2.a)
+               ->  Hash Join
+                     Output: t2_1.a, t3_2.a, LEAST(t1.a, t2_1.a, t3_2.a)
+                     Hash Cond: (t3_2.b = t2_1.a)
+                     ->  Seq Scan on public.prt2_p3 t3_2
+                           Output: t3_2.a, t3_2.b
+                     ->  Hash
+                           Output: t2_1.a
+                           ->  Seq Scan on public.prt1_p3 t2_1
+                                 Output: t2_1.a
+                                 Filter: (t1.b = t2_1.a)
+(49 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+--
+-- partitioned by expression
+--
+CREATE TABLE prt1_e (a int, b int, c varchar) PARTITION BY RANGE(((a + b)/2));
+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);
+INSERT INTO prt1_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_e AS SELECT * FROM prt1_e;
+CREATE TABLE prt2_e (a int, b int, c varchar) PARTITION BY RANGE(((b + a)/2));
+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);
+INSERT INTO prt2_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_e;
+ANALYZE prt2_e_p1;
+ANALYZE prt2_e_p2;
+ANALYZE prt2_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_e AS SELECT * FROM prt2_e;
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                  QUERY PLAN                                  
+------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_e_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_e_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
+               ->  Seq Scan on public.prt2_e_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_e_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+(34 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.a % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1, uprt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.a % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                  QUERY PLAN                                  
+------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_e_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_e_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
+               ->  Seq Scan on public.prt2_e_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_e_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1 LEFT JOIN uprt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on public.prt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = t1.a)
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_2.a)
+                     ->  Seq Scan on public.prt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                           ->  Hash Join
+                                 Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = t1_2.a)
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_1.a)
+                     ->  Seq Scan on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                           ->  Hash Join
+                                 Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = t1_1.a)
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+(57 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM uprt1 t1, uprt2 t2, uprt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on public.prt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = t1.a)
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_2.a)
+                     ->  Seq Scan on public.prt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = t1_2.a)
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_1.a)
+                     ->  Seq Scan on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = t1_1.a)
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+(57 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, 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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: (((t3.a + t3.b) / 2) = t2.b)
+                     ->  Seq Scan on public.prt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = t1.a)
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: (((t3_1.a + t3_1.b) / 2) = t2_1.b)
+                     ->  Seq Scan on public.prt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = t1_2.a)
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: (((t3_2.a + t3_2.b) / 2) = t2_2.b)
+                     ->  Seq Scan on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = t1_1.a)
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+(57 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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |          | 
+ 100 | 0100 |     |      |          | 
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |          | 
+ 250 | 0250 |     |      |          | 
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |          | 
+ 400 | 0400 |     |      |          | 
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |          | 
+ 550 | 0550 |     |      |          | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |          | 
+ 100 | 0100 |     |      |          | 
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |          | 
+ 250 | 0250 |     |      |          | 
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |          | 
+ 400 | 0400 |     |      |          | 
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |          | 
+ 550 | 0550 |     |      |          | 
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Hash Cond: (t2.b = t1.a)
+                     ->  Seq Scan on public.prt2_p1 t2
+                           Output: t2.b, t2.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Hash Cond: (t1.a = ((t3.a + t3.b) / 2))
+                                 ->  Seq Scan on public.prt1_p1 t1
+                                       Output: t1.a, t1.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                     Hash Cond: (t2_1.b = t1_2.a)
+                     ->  Seq Scan on public.prt2_p2 t2_1
+                           Output: t2_1.b, t2_1.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                                 Hash Cond: (t1_2.a = ((t3_1.a + t3_1.b) / 2))
+                                 ->  Seq Scan on public.prt1_p2 t1_2
+                                       Output: t1_2.a, t1_2.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                     Hash Cond: (t2_2.b = t1_1.a)
+                     ->  Seq Scan on public.prt2_p3 t2_2
+                           Output: t2_2.b, t2_2.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                                 Hash Cond: (t1_1.a = ((t3_2.a + t3_2.b) / 2))
+                                 ->  Seq Scan on public.prt1_p3 t1_1
+                                       Output: t1_1.a, t1_1.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: (t1.a = t2.b)
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = ((t3.a + t3.b) / 2))
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Hash Cond: (t1_2.a = t2_1.b)
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = ((t3_1.a + t3_1.b) / 2))
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Hash Cond: (t1_1.a = t2_2.b)
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = ((t3_2.a + t3_2.b) / 2))
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                                  QUERY PLAN                                                   
+---------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, ((prt1_e_p1.a + prt1_e_p1.b)), prt1_e_p1.c
+   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   ->  Result
+         Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, (prt1_e_p1.a + prt1_e_p1.b), prt1_e_p1.c
+         ->  Append
+               ->  Hash Full Join
+                     Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, prt1_e_p1.a, prt1_e_p1.b, prt1_e_p1.c
+                     Hash Cond: (prt1_p1.a = ((prt1_e_p1.a + prt1_e_p1.b) / 2))
+                     ->  Hash Full Join
+                           Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on public.prt1_p1
+                                 Output: prt1_p1.a, prt1_p1.c
+                                 Filter: ((prt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p1.b, prt2_p1.c
+                                 ->  Seq Scan on public.prt2_p1
+                                       Output: prt2_p1.b, prt2_p1.c
+                                       Filter: ((prt2_p1.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p1.a, prt1_e_p1.b, prt1_e_p1.c
+                           ->  Seq Scan on public.prt1_e_p1
+                                 Output: prt1_e_p1.a, prt1_e_p1.b, prt1_e_p1.c
+                                 Filter: ((prt1_e_p1.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c, prt1_e_p2.a, prt1_e_p2.b, prt1_e_p2.c
+                     Hash Cond: (prt1_p2.a = ((prt1_e_p2.a + prt1_e_p2.b) / 2))
+                     ->  Hash Full Join
+                           Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+                           Hash Cond: (prt1_p2.a = prt2_p2.b)
+                           ->  Seq Scan on public.prt1_p2
+                                 Output: prt1_p2.a, prt1_p2.c
+                                 Filter: ((prt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p2.b, prt2_p2.c
+                                 ->  Seq Scan on public.prt2_p2
+                                       Output: prt2_p2.b, prt2_p2.c
+                                       Filter: ((prt2_p2.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p2.a, prt1_e_p2.b, prt1_e_p2.c
+                           ->  Seq Scan on public.prt1_e_p2
+                                 Output: prt1_e_p2.a, prt1_e_p2.b, prt1_e_p2.c
+                                 Filter: ((prt1_e_p2.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c, prt1_e_p3.a, prt1_e_p3.b, prt1_e_p3.c
+                     Hash Cond: (prt1_p3.a = ((prt1_e_p3.a + prt1_e_p3.b) / 2))
+                     ->  Hash Full Join
+                           Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c
+                           Hash Cond: (prt1_p3.a = prt2_p3.b)
+                           ->  Seq Scan on public.prt1_p3
+                                 Output: prt1_p3.a, prt1_p3.c
+                                 Filter: ((prt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p3.b, prt2_p3.c
+                                 ->  Seq Scan on public.prt2_p3
+                                       Output: prt2_p3.b, prt2_p3.c
+                                       Filter: ((prt2_p3.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p3.a, prt1_e_p3.b, prt1_e_p3.c
+                           ->  Seq Scan on public.prt1_e_p3
+                                 Output: prt1_e_p3.a, prt1_e_p3.b, prt1_e_p3.c
+                                 Filter: ((prt1_e_p3.a % 25) = 0)
+(63 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+     |      |  75 | 0075 |          | 
+     |      | 225 | 0225 |          | 
+     |      | 375 | 0375 |          | 
+     |      | 525 | 0525 |          | 
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM uprt1_e WHERE uprt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+     |      |  75 | 0075 |          | 
+     |      | 225 | 0225 |          | 
+     |      | 375 | 0375 |          | 
+     |      | 525 | 0525 |          | 
+(16 rows)
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, 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.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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                                                      
+----------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, (50), prt2_p1.b, (75), ((prt1_e_p1.a + prt1_e_p1.b)), (50)
+   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   ->  Result
+         Output: prt1_p1.a, (50), prt2_p1.b, (75), (prt1_e_p1.a + prt1_e_p1.b), (50)
+         ->  Append
+               ->  Hash Full Join
+                     Output: prt1_p1.a, prt2_p1.b, prt1_e_p1.a, prt1_e_p1.b, (50), (75), (50)
+                     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
+                           Output: prt1_p1.a, prt2_p1.b, (50), (75)
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on public.prt1_p1
+                                 Output: prt1_p1.a, 50
+                                 Filter: ((prt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p1.b, (75)
+                                 ->  Seq Scan on public.prt2_p1
+                                       Output: prt2_p1.b, 75
+                                       Filter: ((prt2_p1.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p1.a, prt1_e_p1.b, (50)
+                           ->  Seq Scan on public.prt1_e_p1
+                                 Output: prt1_e_p1.a, prt1_e_p1.b, 50
+                                 Filter: ((prt1_e_p1.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p2.a, prt2_p2.b, prt1_e_p2.a, prt1_e_p2.b, (50), (75), (50)
+                     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
+                           Output: prt1_p2.a, prt2_p2.b, (50), (75)
+                           Hash Cond: (prt1_p2.a = prt2_p2.b)
+                           ->  Seq Scan on public.prt1_p2
+                                 Output: prt1_p2.a, 50
+                                 Filter: ((prt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p2.b, (75)
+                                 ->  Seq Scan on public.prt2_p2
+                                       Output: prt2_p2.b, 75
+                                       Filter: ((prt2_p2.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p2.a, prt1_e_p2.b, (50)
+                           ->  Seq Scan on public.prt1_e_p2
+                                 Output: prt1_e_p2.a, prt1_e_p2.b, 50
+                                 Filter: ((prt1_e_p2.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p3.a, prt2_p3.b, prt1_e_p3.a, prt1_e_p3.b, (50), (75), (50)
+                     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
+                           Output: prt1_p3.a, prt2_p3.b, (50), (75)
+                           Hash Cond: (prt1_p3.a = prt2_p3.b)
+                           ->  Seq Scan on public.prt1_p3
+                                 Output: prt1_p3.a, 50
+                                 Filter: ((prt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p3.b, (75)
+                                 ->  Seq Scan on public.prt2_p3
+                                       Output: prt2_p3.b, 75
+                                       Filter: ((prt2_p3.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p3.a, prt1_e_p3.b, (50)
+                           ->  Seq Scan on public.prt1_e_p3
+                                 Output: prt1_e_p3.a, prt1_e_p3.b, 50
+                                 Filter: ((prt1_e_p3.a % 25) = 0)
+(66 rows)
+
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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)
+
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM uprt1_e WHERE uprt1_e.a % 25 = 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)
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+                            QUERY PLAN                             
+-------------------------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Output: t1.a, t1.b, t1.c
+         Merge Cond: (t1.a = t1_3.b)
+         ->  Sort
+               Output: t1.a, t1.b, t1.c
+               Sort Key: t1.a
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+         ->  Sort
+               Output: t1_3.b, t2.a, t2.b
+               Sort Key: t1_3.b
+               ->  Hash Join
+                     Output: t1_3.b, t2.a, t2.b
+                     Hash Cond: (((t2.a + t2.b) / 2) = t1_3.b)
+                     ->  Seq Scan on public.prt1_e_p1 t2
+                           Output: t2.a, t2.b
+                     ->  Hash
+                           Output: t1_3.b
+                           ->  Seq Scan on public.prt2_p1 t1_3
+                                 Output: t1_3.b
+                                 Filter: ((t1_3.b % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_2.a, t1_2.b, t1_2.c
+         Merge Cond: (t1_2.a = t1_4.b)
+         ->  Sort
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Sort Key: t1_2.a
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+         ->  Sort
+               Output: t1_4.b, t2_1.a, t2_1.b
+               Sort Key: t1_4.b
+               ->  Hash Join
+                     Output: t1_4.b, t2_1.a, t2_1.b
+                     Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_4.b)
+                     ->  Seq Scan on public.prt1_e_p2 t2_1
+                           Output: t2_1.a, t2_1.b
+                     ->  Hash
+                           Output: t1_4.b
+                           ->  Seq Scan on public.prt2_p2 t1_4
+                                 Output: t1_4.b
+                                 Filter: ((t1_4.b % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_1.a, t1_1.b, t1_1.c
+         Merge Cond: (t1_1.a = t1_5.b)
+         ->  Sort
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Sort Key: t1_1.a
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+         ->  Sort
+               Output: t1_5.b, t2_2.a, t2_2.b
+               Sort Key: t1_5.b
+               ->  Hash Join
+                     Output: t1_5.b, t2_2.a, t2_2.b
+                     Hash Cond: (((t2_2.a + t2_2.b) / 2) = t1_5.b)
+                     ->  Seq Scan on public.prt1_e_p3 t2_2
+                           Output: t2_2.a, t2_2.b
+                     ->  Hash
+                           Output: t1_5.b
+                           ->  Seq Scan on public.prt2_p3 t1_5
+                                 Output: t1_5.b
+                                 Filter: ((t1_5.b % 25) = 0)
+(68 rows)
+
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1, uprt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+                            QUERY PLAN                             
+-------------------------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Output: t1.a, t1.b, t1.c
+         Merge Cond: (t1.a = t1_3.b)
+         ->  Sort
+               Output: t1.a, t1.b, t1.c
+               Sort Key: t1.a
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+         ->  Sort
+               Output: t1_3.b, t1_6.a, t1_6.b
+               Sort Key: t1_3.b
+               ->  Hash Semi Join
+                     Output: t1_3.b, t1_6.a, t1_6.b
+                     Hash Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
+                     ->  Seq Scan on public.prt2_p1 t1_3
+                           Output: t1_3.b
+                     ->  Hash
+                           Output: t1_6.a, t1_6.b
+                           ->  Seq Scan on public.prt1_e_p1 t1_6
+                                 Output: t1_6.a, t1_6.b
+                                 Filter: ((t1_6.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_2.a, t1_2.b, t1_2.c
+         Merge Cond: (t1_2.a = t1_4.b)
+         ->  Sort
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Sort Key: t1_2.a
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+         ->  Sort
+               Output: t1_4.b, t1_7.a, t1_7.b
+               Sort Key: t1_4.b
+               ->  Hash Semi Join
+                     Output: t1_4.b, t1_7.a, t1_7.b
+                     Hash Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
+                     ->  Seq Scan on public.prt2_p2 t1_4
+                           Output: t1_4.b
+                     ->  Hash
+                           Output: t1_7.a, t1_7.b
+                           ->  Seq Scan on public.prt1_e_p2 t1_7
+                                 Output: t1_7.a, t1_7.b
+                                 Filter: ((t1_7.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_1.a, t1_1.b, t1_1.c
+         Merge Cond: (t1_1.a = t1_5.b)
+         ->  Sort
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Sort Key: t1_1.a
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+         ->  Sort
+               Output: t1_5.b, t1_8.a, t1_8.b
+               Sort Key: t1_5.b
+               ->  Hash Semi Join
+                     Output: t1_5.b, t1_8.a, t1_8.b
+                     Hash Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
+                     ->  Seq Scan on public.prt2_p3 t1_5
+                           Output: t1_5.b
+                     ->  Hash
+                           Output: t1_8.a, t1_8.b
+                           ->  Seq Scan on public.prt1_e_p3 t1_8
+                                 Output: t1_8.a, t1_8.b
+                                 Filter: ((t1_8.a % 25) = 0)
+(68 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+-- test merge joins with and without using indexes
+SET enable_hashjoin TO off;
+SET enable_nestloop TO off;
+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 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 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));
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                             QUERY PLAN                                              
+-----------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Merge Cond: (t2.b = t1.a)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           Sort Key: t2.b
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Merge Cond: ((((t3.a + t3.b) / 2)) = t2.b)
+                                 ->  Sort
+                                       Output: t3.a, t3.b, t3.c, (((t3.a + t3.b) / 2))
+                                       Sort Key: (((t3.a + t3.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c, ((t3.a + t3.b) / 2)
+                                             Filter: ((t3.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2.b, t2.c
+                                       Sort Key: t2.b
+                                       ->  Seq Scan on public.prt2_p1 t2
+                                             Output: t2.b, t2.c
+                     ->  Sort
+                           Output: t1.a, t1.c
+                           Sort Key: t1.a
+                           ->  Seq Scan on public.prt1_p1 t1
+                                 Output: t1.a, t1.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Merge Cond: (t2_1.b = t1_2.a)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           Sort Key: t2_1.b
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t2_1.b)
+                                 ->  Sort
+                                       Output: t3_1.a, t3_1.b, t3_1.c, (((t3_1.a + t3_1.b) / 2))
+                                       Sort Key: (((t3_1.a + t3_1.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c, ((t3_1.a + t3_1.b) / 2)
+                                             Filter: ((t3_1.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2_1.b, t2_1.c
+                                       Sort Key: t2_1.b
+                                       ->  Seq Scan on public.prt2_p2 t2_1
+                                             Output: t2_1.b, t2_1.c
+                     ->  Sort
+                           Output: t1_2.a, t1_2.c
+                           Sort Key: t1_2.a
+                           ->  Seq Scan on public.prt1_p2 t1_2
+                                 Output: t1_2.a, t1_2.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Merge Cond: (t2_2.b = t1_1.a)
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           Sort Key: t2_2.b
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t2_2.b)
+                                 ->  Sort
+                                       Output: t3_2.a, t3_2.b, t3_2.c, (((t3_2.a + t3_2.b) / 2))
+                                       Sort Key: (((t3_2.a + t3_2.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c, ((t3_2.a + t3_2.b) / 2)
+                                             Filter: ((t3_2.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2_2.b, t2_2.c
+                                       Sort Key: t2_2.b
+                                       ->  Seq Scan on public.prt2_p3 t2_2
+                                             Output: t2_2.b, t2_2.c
+                     ->  Sort
+                           Output: t1_1.a, t1_1.c
+                           Sort Key: t1_1.a
+                           ->  Seq Scan on public.prt1_p3 t1_1
+                                 Output: t1_1.a, t1_1.c
+(81 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+                                   QUERY PLAN                                    
+---------------------------------------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Output: t1.a, t1.b, t1.c
+         Merge Cond: (t1.a = t1_3.b)
+         ->  Sort
+               Output: t1.a, t1.b, t1.c
+               Sort Key: t1.a
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_3.b, t1_6.a, t1_6.b
+               ->  Merge Semi Join
+                     Output: t1_3.b, t1_6.a, t1_6.b
+                     Merge Cond: (t1_3.b = (((t1_6.a + t1_6.b) / 2)))
+                     ->  Sort
+                           Output: t1_3.b
+                           Sort Key: t1_3.b
+                           ->  Seq Scan on public.prt2_p1 t1_3
+                                 Output: t1_3.b
+                     ->  Sort
+                           Output: t1_6.a, t1_6.b, (((t1_6.a + t1_6.b) / 2))
+                           Sort Key: (((t1_6.a + t1_6.b) / 2))
+                           ->  Seq Scan on public.prt1_e_p1 t1_6
+                                 Output: t1_6.a, t1_6.b, ((t1_6.a + t1_6.b) / 2)
+                                 Filter: ((t1_6.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_2.a, t1_2.b, t1_2.c
+         Merge Cond: (t1_2.a = t1_4.b)
+         ->  Sort
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Sort Key: t1_2.a
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+         ->  Materialize
+               Output: t1_4.b, t1_7.a, t1_7.b
+               ->  Merge Semi Join
+                     Output: t1_4.b, t1_7.a, t1_7.b
+                     Merge Cond: (t1_4.b = (((t1_7.a + t1_7.b) / 2)))
+                     ->  Sort
+                           Output: t1_4.b
+                           Sort Key: t1_4.b
+                           ->  Seq Scan on public.prt2_p2 t1_4
+                                 Output: t1_4.b
+                     ->  Sort
+                           Output: t1_7.a, t1_7.b, (((t1_7.a + t1_7.b) / 2))
+                           Sort Key: (((t1_7.a + t1_7.b) / 2))
+                           ->  Seq Scan on public.prt1_e_p2 t1_7
+                                 Output: t1_7.a, t1_7.b, ((t1_7.a + t1_7.b) / 2)
+                                 Filter: ((t1_7.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_1.a, t1_1.b, t1_1.c
+         Merge Cond: (t1_1.a = t1_5.b)
+         ->  Sort
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Sort Key: t1_1.a
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_5.b, t1_8.a, t1_8.b
+               ->  Merge Semi Join
+                     Output: t1_5.b, t1_8.a, t1_8.b
+                     Merge Cond: (t1_5.b = (((t1_8.a + t1_8.b) / 2)))
+                     ->  Sort
+                           Output: t1_5.b
+                           Sort Key: t1_5.b
+                           ->  Seq Scan on public.prt2_p3 t1_5
+                                 Output: t1_5.b
+                     ->  Sort
+                           Output: t1_8.a, t1_8.b, (((t1_8.a + t1_8.b) / 2))
+                           Sort Key: (((t1_8.a + t1_8.b) / 2))
+                           ->  Seq Scan on public.prt1_e_p3 t1_8
+                                 Output: t1_8.a, t1_8.b, ((t1_8.a + t1_8.b) / 2)
+                                 Filter: ((t1_8.a % 25) = 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                             QUERY PLAN                                              
+-----------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Merge Cond: (t1.a = t2.b)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           Sort Key: t1.a
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Merge Cond: ((((t3.a + t3.b) / 2)) = t1.a)
+                                 ->  Sort
+                                       Output: t3.a, t3.b, t3.c, (((t3.a + t3.b) / 2))
+                                       Sort Key: (((t3.a + t3.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c, ((t3.a + t3.b) / 2)
+                                             Filter: ((t3.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t1.a, t1.c
+                                       Sort Key: t1.a
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                     ->  Sort
+                           Output: t2.b, t2.c
+                           Sort Key: t2.b
+                           ->  Seq Scan on public.prt2_p1 t2
+                                 Output: t2.b, t2.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                     Merge Cond: (t1_2.a = t2_1.b)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                           Sort Key: t1_2.a
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                                 Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t1_2.a)
+                                 ->  Sort
+                                       Output: t3_1.a, t3_1.b, t3_1.c, (((t3_1.a + t3_1.b) / 2))
+                                       Sort Key: (((t3_1.a + t3_1.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c, ((t3_1.a + t3_1.b) / 2)
+                                             Filter: ((t3_1.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t1_2.a, t1_2.c
+                                       Sort Key: t1_2.a
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                     ->  Sort
+                           Output: t2_1.b, t2_1.c
+                           Sort Key: t2_1.b
+                           ->  Seq Scan on public.prt2_p2 t2_1
+                                 Output: t2_1.b, t2_1.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                     Merge Cond: (t1_1.a = t2_2.b)
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                           Sort Key: t1_1.a
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                                 Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t1_1.a)
+                                 ->  Sort
+                                       Output: t3_2.a, t3_2.b, t3_2.c, (((t3_2.a + t3_2.b) / 2))
+                                       Sort Key: (((t3_2.a + t3_2.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c, ((t3_2.a + t3_2.b) / 2)
+                                             Filter: ((t3_2.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t1_1.a, t1_1.c
+                                       Sort Key: t1_1.a
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                     ->  Sort
+                           Output: t2_2.b, t2_2.c
+                           Sort Key: t2_2.b
+                           ->  Seq Scan on public.prt2_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+(81 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SET enable_seqscan TO off;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                          QUERY PLAN                                          
+----------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Merge Cond: (t2.b = t1.a)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           Sort Key: t2.b
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Merge Cond: (((t3.a + t3.b) / 2) = t2.b)
+                                 ->  Index Scan using iprt1_e_p1_ab2 on public.prt1_e_p1 t3
+                                       Output: t3.a, t3.b, t3.c
+                                       Filter: ((t3.a % 25) = 0)
+                                 ->  Index Scan using iprt2_p1_b on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                     ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Merge Cond: (t2_1.b = t1_2.a)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           Sort Key: t2_1.b
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Merge Cond: (((t3_1.a + t3_1.b) / 2) = t2_1.b)
+                                 ->  Index Scan using iprt1_e_p2_ab2 on public.prt1_e_p2 t3_1
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       Filter: ((t3_1.a % 25) = 0)
+                                 ->  Index Scan using iprt2_p2_b on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                     ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+               ->  Merge Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Merge Cond: (t2_2.b = ((t3_2.a + t3_2.b) / 2))
+                     ->  Merge Left Join
+                           Output: t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                           Merge Cond: (t2_2.b = t1_1.a)
+                           ->  Index Scan using iprt2_p3_b on public.prt2_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+                           ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_1
+                                 Output: t1_1.a, t1_1.c
+                     ->  Index Scan using iprt1_e_p3_ab2 on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                           Filter: ((t3_2.a % 25) = 0)
+(51 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+                                    QUERY PLAN                                    
+----------------------------------------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Output: t1.a, t1.b, t1.c
+         Merge Cond: (t1.a = t1_3.b)
+         ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1
+               Output: t1.a, t1.b, t1.c
+               Filter: ((t1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_3.b, t1_6.a, t1_6.b
+               ->  Merge Semi Join
+                     Output: t1_3.b, t1_6.a, t1_6.b
+                     Merge Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
+                     ->  Index Only Scan using iprt2_p1_b on public.prt2_p1 t1_3
+                           Output: t1_3.b
+                     ->  Index Scan using iprt1_e_p1_ab2 on public.prt1_e_p1 t1_6
+                           Output: t1_6.a, t1_6.b
+                           Filter: ((t1_6.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_2.a, t1_2.b, t1_2.c
+         Merge Cond: (t1_2.a = t1_4.b)
+         ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_2
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Filter: ((t1_2.a % 25) = 0)
+         ->  Materialize
+               Output: t1_4.b, t1_7.a, t1_7.b
+               ->  Merge Semi Join
+                     Output: t1_4.b, t1_7.a, t1_7.b
+                     Merge Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
+                     ->  Index Only Scan using iprt2_p2_b on public.prt2_p2 t1_4
+                           Output: t1_4.b
+                     ->  Index Scan using iprt1_e_p2_ab2 on public.prt1_e_p2 t1_7
+                           Output: t1_7.a, t1_7.b
+                           Filter: ((t1_7.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_1.a, t1_1.b, t1_1.c
+         Merge Cond: (t1_1.a = t1_5.b)
+         ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_1
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Filter: ((t1_1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_5.b, t1_8.a, t1_8.b
+               ->  Merge Semi Join
+                     Output: t1_5.b, t1_8.a, t1_8.b
+                     Merge Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
+                     ->  Index Only Scan using iprt2_p3_b on public.prt2_p3 t1_5
+                           Output: t1_5.b
+                     ->  Index Scan using iprt1_e_p3_ab2 on public.prt1_e_p3 t1_8
+                           Output: t1_8.a, t1_8.b
+                           Filter: ((t1_8.a % 25) = 0)
+(50 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                          QUERY PLAN                                          
+----------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Merge Cond: (t1.a = t2.b)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           Sort Key: t1.a
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Merge Cond: (((t3.a + t3.b) / 2) = t1.a)
+                                 ->  Index Scan using iprt1_e_p1_ab2 on public.prt1_e_p1 t3
+                                       Output: t3.a, t3.b, t3.c
+                                       Filter: ((t3.a % 25) = 0)
+                                 ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1
+                                       Output: t1.a, t1.c
+                     ->  Index Scan using iprt2_p1_b on public.prt2_p1 t2
+                           Output: t2.b, t2.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                     Merge Cond: (t1_2.a = t2_1.b)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                           Sort Key: t1_2.a
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                                 Merge Cond: (((t3_1.a + t3_1.b) / 2) = t1_2.a)
+                                 ->  Index Scan using iprt1_e_p2_ab2 on public.prt1_e_p2 t3_1
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       Filter: ((t3_1.a % 25) = 0)
+                                 ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_2
+                                       Output: t1_2.a, t1_2.c
+                     ->  Index Scan using iprt2_p2_b on public.prt2_p2 t2_1
+                           Output: t2_1.b, t2_1.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                     Merge Cond: (t1_1.a = t2_2.b)
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                           Sort Key: t1_1.a
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                                 Merge Cond: (((t3_2.a + t3_2.b) / 2) = t1_1.a)
+                                 ->  Index Scan using iprt1_e_p3_ab2 on public.prt1_e_p3 t3_2
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       Filter: ((t3_2.a % 25) = 0)
+                                 ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_1
+                                       Output: t1_1.a, t1_1.c
+                     ->  Index Scan using iprt2_p3_b on public.prt2_p3 t2_2
+                           Output: t2_2.b, t2_2.c
+(54 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+-- lateral references and parameterized paths
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Result
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   ->  Merge Append
+         Sort Key: t1.a
+         ->  Nested Loop Left Join
+               Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a))
+               ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Merge Join
+                     Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+                     Merge Cond: (t2.a = t3.b)
+                     ->  Index Only Scan using iprt1_p1_a on public.prt1_p1 t2
+                           Output: t2.a
+                           Index Cond: (t2.a = t1.a)
+                     ->  Index Scan using iprt2_p1_b on public.prt2_p1 t3
+                           Output: t3.a, t3.b
+         ->  Nested Loop Left Join
+               Output: t1_2.a, t1_2.b, t1_2.c, t2_2.a, t3_1.a, (LEAST(t1_2.a, t2_2.a, t3_1.a))
+               ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Merge Join
+                     Output: t2_2.a, t3_1.a, LEAST(t1_2.a, t2_2.a, t3_1.a)
+                     Merge Cond: (t2_2.a = t3_1.b)
+                     ->  Index Only Scan using iprt1_p2_a on public.prt1_p2 t2_2
+                           Output: t2_2.a
+                           Index Cond: (t2_2.a = t1_2.a)
+                     ->  Index Scan using iprt2_p2_b on public.prt2_p2 t3_1
+                           Output: t3_1.a, t3_1.b
+         ->  Nested Loop Left Join
+               Output: t1_1.a, t1_1.b, t1_1.c, t2_1.a, t3_2.a, (LEAST(t1_1.a, t2_1.a, t3_2.a))
+               ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Merge Join
+                     Output: t2_1.a, t3_2.a, LEAST(t1_1.a, t2_1.a, t3_2.a)
+                     Merge Cond: (t2_1.a = t3_2.b)
+                     ->  Index Only Scan using iprt1_p3_a on public.prt1_p3 t2_1
+                           Output: t2_1.a
+                           Index Cond: (t2_1.a = t1_1.a)
+                     ->  Index Scan using iprt2_p3_b on public.prt2_p3 t3_2
+                           Output: t3_2.a, t3_2.b
+(43 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
+ Nested Loop Left Join
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   ->  Merge Append
+         Sort Key: t1.a
+         ->  Sort
+               Output: t1.a, t1.b, t1.c
+               Sort Key: t1.a
+               ->  Seq Scan on public.prt1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+         ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1_1
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Filter: ((t1_1.a % 25) = 0)
+         ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_2
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Filter: ((t1_2.a % 25) = 0)
+         ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_3
+               Output: t1_3.a, t1_3.b, t1_3.c
+               Filter: ((t1_3.a % 25) = 0)
+   ->  Append
+         ->  Merge Join
+               Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+               Merge Cond: (t2.a = t3.b)
+               ->  Index Only Scan using iprt1_p1_a on public.prt1_p1 t2
+                     Output: t2.a
+                     Index Cond: (t2.a = t1.b)
+               ->  Index Scan using iprt2_p1_b on public.prt2_p1 t3
+                     Output: t3.a, t3.b
+         ->  Merge Join
+               Output: t2_2.a, t3_1.a, LEAST(t1.a, t2_2.a, t3_1.a)
+               Merge Cond: (t2_2.a = t3_1.b)
+               ->  Index Only Scan using iprt1_p2_a on public.prt1_p2 t2_2
+                     Output: t2_2.a
+                     Index Cond: (t2_2.a = t1.b)
+               ->  Index Scan using iprt2_p2_b on public.prt2_p2 t3_1
+                     Output: t3_1.a, t3_1.b
+         ->  Merge Join
+               Output: t2_1.a, t3_2.a, LEAST(t1.a, t2_1.a, t3_2.a)
+               Merge Cond: (t2_1.a = t3_2.b)
+               ->  Index Only Scan using iprt1_p3_a on public.prt1_p3 t2_1
+                     Output: t2_1.a
+                     Index Cond: (t2_1.a = t1.b)
+               ->  Index Scan using iprt2_p3_b on public.prt2_p3 t3_2
+                     Output: t3_2.a, t3_2.b
+(44 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+RESET enable_hashjoin;
+RESET enable_nestloop;
+RESET enable_seqscan;
+--
+-- partitioned by multiple columns
+--
+CREATE TABLE prt1_m (a int, b int, c varchar) 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, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_m;
+ANALYZE prt1_m_p1;
+ANALYZE prt1_m_p2;
+ANALYZE prt1_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_m AS SELECT * FROM prt1_m;
+CREATE TABLE prt2_m (a int, b int, c varchar) 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, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_m;
+ANALYZE prt2_m_p1;
+ANALYZE prt2_m_p2;
+ANALYZE prt2_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_m AS SELECT * FROM prt2_m;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+                                                 QUERY PLAN                                                 
+------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: ((((t1.a + t1.b) / 2) = t2.b) AND (t1.a = ((t2.b + t2.a) / 2)))
+                     ->  Seq Scan on public.prt1_m_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                     ->  Hash
+                           Output: t2.b, t2.c, t2.a
+                           ->  Seq Scan on public.prt2_m_p1 t2
+                                 Output: t2.b, t2.c, t2.a
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: ((((t1_1.a + t1_1.b) / 2) = t2_1.b) AND (t1_1.a = ((t2_1.b + t2_1.a) / 2)))
+                     ->  Seq Scan on public.prt1_m_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c, t2_1.a
+                           ->  Seq Scan on public.prt2_m_p2 t2_1
+                                 Output: t2_1.b, t2_1.c, t2_1.a
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: ((((t1_2.a + t1_2.b) / 2) = t2_2.b) AND (t1_2.a = ((t2_2.b + t2_2.a) / 2)))
+                     ->  Seq Scan on public.prt1_m_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c, t2_2.a
+                           ->  Seq Scan on public.prt2_m_p3 t2_2
+                                 Output: t2_2.b, t2_2.c, t2_2.a
+                                 Filter: ((t2_2.b % 25) = 0)
+(36 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_m t1 RIGHT JOIN uprt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 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
+   Output: prt1_m_p1.a, prt1_m_p1.c, prt2_m_p1.b, prt2_m_p1.c
+   Sort Key: prt1_m_p1.a, prt2_m_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_m_p1.a, prt1_m_p1.c, prt2_m_p1.b, prt2_m_p1.c
+               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 public.prt1_m_p1
+                     Output: prt1_m_p1.a, prt1_m_p1.c, prt1_m_p1.b
+                     Filter: ((prt1_m_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_m_p1.b, prt2_m_p1.c, prt2_m_p1.a
+                     ->  Seq Scan on public.prt2_m_p1
+                           Output: prt2_m_p1.b, prt2_m_p1.c, prt2_m_p1.a
+                           Filter: ((prt2_m_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_m_p2.a, prt1_m_p2.c, prt2_m_p2.b, prt2_m_p2.c
+               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 public.prt1_m_p2
+                     Output: prt1_m_p2.a, prt1_m_p2.c, prt1_m_p2.b
+                     Filter: ((prt1_m_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_m_p2.b, prt2_m_p2.c, prt2_m_p2.a
+                     ->  Seq Scan on public.prt2_m_p2
+                           Output: prt2_m_p2.b, prt2_m_p2.c, prt2_m_p2.a
+                           Filter: ((prt2_m_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_m_p3.a, prt1_m_p3.c, prt2_m_p3.b, prt2_m_p3.c
+               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 public.prt1_m_p3
+                     Output: prt1_m_p3.a, prt1_m_p3.c, prt1_m_p3.b
+                     Filter: ((prt1_m_p3.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_m_p3.b, prt2_m_p3.c, prt2_m_p3.a
+                     ->  Seq Scan on public.prt2_m_p3
+                           Output: prt2_m_p3.b, prt2_m_p3.c, prt2_m_p3.a
+                           Filter: ((prt2_m_p3.b % 25) = 0)
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 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 | 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_m t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_m t2 WHERE t2.b % 25 = 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 | 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)
+
+--
+-- 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;
+ANALYZE plt1_p1;
+ANALYZE plt1_p2;
+ANALYZE plt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1 AS SELECT * FROM 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;
+ANALYZE plt2_p1;
+ANALYZE plt2_p2;
+ANALYZE plt2_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt2 AS SELECT * FROM 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;
+ANALYZE plt1_e_p1;
+ANALYZE plt1_e_p2;
+ANALYZE plt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1_e AS SELECT * FROM plt1_e;
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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
+   Output: (avg(t1.a)), (avg(t2.b)), (avg((t3.a + t3.b))), t1.c, t2.c, t3.c
+   Sort Key: t1.c, t3.c
+   ->  HashAggregate
+         Output: avg(t1.a), avg(t2.b), avg((t3.a + t3.b)), t1.c, t2.c, t3.c
+         Group Key: t1.c, t2.c, t3.c
+         ->  Result
+               Output: t1.c, t2.c, t3.c, t1.a, t2.b, t3.a, t3.b
+               ->  Append
+                     ->  Hash Join
+                           Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                           Hash Cond: (t1.c = t2.c)
+                           ->  Seq Scan on public.plt1_p1 t1
+                                 Output: t1.a, t1.c
+                           ->  Hash
+                                 Output: t2.b, t2.c, t3.a, t3.b, t3.c
+                                 ->  Hash Join
+                                       Output: t2.b, t2.c, t3.a, t3.b, t3.c
+                                       Hash Cond: (t2.c = ltrim(t3.c, 'A'::text))
+                                       ->  Seq Scan on public.plt2_p1 t2
+                                             Output: t2.b, t2.c
+                                       ->  Hash
+                                             Output: t3.a, t3.b, t3.c
+                                             ->  Seq Scan on public.plt1_e_p1 t3
+                                                   Output: t3.a, t3.b, t3.c
+                     ->  Hash Join
+                           Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                           Hash Cond: (t1_1.c = t2_1.c)
+                           ->  Seq Scan on public.plt1_p2 t1_1
+                                 Output: t1_1.a, t1_1.c
+                           ->  Hash
+                                 Output: t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                                 ->  Hash Join
+                                       Output: t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                                       Hash Cond: (t2_1.c = ltrim(t3_1.c, 'A'::text))
+                                       ->  Seq Scan on public.plt2_p2 t2_1
+                                             Output: t2_1.b, t2_1.c
+                                       ->  Hash
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             ->  Seq Scan on public.plt1_e_p2 t3_1
+                                                   Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash Join
+                           Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                           Hash Cond: (t1_2.c = t2_2.c)
+                           ->  Seq Scan on public.plt1_p3 t1_2
+                                 Output: t1_2.a, t1_2.c
+                           ->  Hash
+                                 Output: t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                                 ->  Hash Join
+                                       Output: t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                                       Hash Cond: (t2_2.c = ltrim(t3_2.c, 'A'::text))
+                                       ->  Seq Scan on public.plt2_p3 t2_2
+                                             Output: t2_2.b, t2_2.c
+                                       ->  Hash
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             ->  Seq Scan on public.plt1_e_p3 t3_2
+                                                   Output: t3_2.a, t3_2.b, t3_2.c
+(57 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)
+
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM uplt1 t1, uplt2 t2, uplt1_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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: ((t3.a = t1.a) AND (ltrim(t3.c, 'A'::text) = t1.c))
+                     ->  Seq Scan on public.plt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                                 ->  Seq Scan on public.plt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.plt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: ((t3_1.a = t1_1.a) AND (ltrim(t3_1.c, 'A'::text) = t1_1.c))
+                     ->  Seq Scan on public.plt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.c = t1_1.c))
+                                 ->  Seq Scan on public.plt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.plt1_p2 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: ((t3_2.a = t1_2.a) AND (ltrim(t3_2.c, 'A'::text) = t1_2.c))
+                     ->  Seq Scan on public.plt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                                 ->  Seq Scan on public.plt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.plt1_p3 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: ((t3.a = t2.b) AND (ltrim(t3.c, 'A'::text) = t2.c))
+                     ->  Seq Scan on public.plt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                                 ->  Seq Scan on public.plt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.plt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: ((t3_1.a = t2_1.b) AND (ltrim(t3_1.c, 'A'::text) = t2_1.c))
+                     ->  Seq Scan on public.plt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.c = t1_1.c))
+                                 ->  Seq Scan on public.plt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.plt1_p2 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: ((t3_2.a = t2_2.b) AND (ltrim(t3_2.c, 'A'::text) = t2_2.c))
+                     ->  Seq Scan on public.plt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                                 ->  Seq Scan on public.plt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.plt1_p3 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |          | 
+ 100 | 0002 |     |      |          | 
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |          | 
+ 250 | 0005 |     |      |          | 
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |          | 
+ 400 | 0008 |     |      |          | 
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |          | 
+ 550 | 0011 |     |      |          | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |          | 
+ 100 | 0002 |     |      |          | 
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |          | 
+ 250 | 0005 |     |      |          | 
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |          | 
+ 400 | 0008 |     |      |          | 
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |          | 
+ 550 | 0011 |     |      |          | 
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Hash Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                     ->  Seq Scan on public.plt2_p1 t2
+                           Output: t2.b, t2.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Hash Cond: ((t1.c = ltrim(t3.c, 'A'::text)) AND (t1.a = t3.a))
+                                 ->  Seq Scan on public.plt1_p1 t1
+                                       Output: t1.a, t1.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.plt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                     Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.c = t1_1.c))
+                     ->  Seq Scan on public.plt2_p2 t2_1
+                           Output: t2_1.b, t2_1.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_1.a, t1_1.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_1.a, t1_1.c
+                                 Hash Cond: ((t1_1.c = ltrim(t3_1.c, 'A'::text)) AND (t1_1.a = t3_1.a))
+                                 ->  Seq Scan on public.plt1_p2 t1_1
+                                       Output: t1_1.a, t1_1.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.plt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                     Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                     ->  Seq Scan on public.plt2_p3 t2_2
+                           Output: t2_2.b, t2_2.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_2.a, t1_2.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_2.a, t1_2.c
+                                 Hash Cond: ((t1_2.c = ltrim(t3_2.c, 'A'::text)) AND (t1_2.a = t3_2.a))
+                                 ->  Seq Scan on public.plt1_p3 t1_2
+                                       Output: t1_2.a, t1_2.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.plt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: ((t1.a = t2.b) AND (t1.c = t2.c))
+                     ->  Seq Scan on public.plt1_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Hash Cond: ((t2.b = t3.a) AND (t2.c = ltrim(t3.c, 'A'::text)))
+                                 ->  Seq Scan on public.plt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.plt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: ((t1_1.a = t2_1.b) AND (t1_1.c = t2_1.c))
+                     ->  Seq Scan on public.plt1_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: ((t2_1.b = t3_1.a) AND (t2_1.c = ltrim(t3_1.c, 'A'::text)))
+                                 ->  Seq Scan on public.plt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.plt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: ((t1_2.a = t2_2.b) AND (t1_2.c = t2_2.c))
+                     ->  Seq Scan on public.plt1_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: ((t2_2.b = t3_2.a) AND (t2_2.c = ltrim(t3_2.c, 'A'::text)))
+                                 ->  Seq Scan on public.plt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.plt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+     |      |     |      |      100 | A0001
+     |      |     |      |      200 | A0002
+     |      |     |      |      400 | A0004
+     |      |     |      |      500 | A0005
+     |      |     |      |      700 | A0007
+     |      |     |      |      800 | A0008
+     |      |     |      |     1000 | A0010
+     |      |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+     |      |     |      |      100 | A0001
+     |      |     |      |      200 | A0002
+     |      |     |      |      400 | A0004
+     |      |     |      |      500 | A0005
+     |      |     |      |      700 | A0007
+     |      |     |      |      800 | A0008
+     |      |     |      |     1000 | A0010
+     |      |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                                  QUERY PLAN                                                   
+---------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, ((plt1_e_p1.a + plt1_e_p1.b)), plt1_e_p1.c
+   Sort Key: plt1_p1.a, plt2_p1.b, ((plt1_e_p1.a + plt1_e_p1.b))
+   ->  Result
+         Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, (plt1_e_p1.a + plt1_e_p1.b), plt1_e_p1.c
+         ->  Append
+               ->  Hash Full Join
+                     Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                     Hash Cond: ((plt1_p1.a = plt1_e_p1.a) AND (plt1_p1.c = ltrim(plt1_e_p1.c, 'A'::text)))
+                     ->  Hash Full Join
+                           Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+                           Hash Cond: ((plt1_p1.a = plt2_p1.b) AND (plt1_p1.c = plt2_p1.c))
+                           ->  Seq Scan on public.plt1_p1
+                                 Output: plt1_p1.a, plt1_p1.c
+                                 Filter: ((plt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p1.b, plt2_p1.c
+                                 ->  Seq Scan on public.plt2_p1
+                                       Output: plt2_p1.b, plt2_p1.c
+                                       Filter: ((plt2_p1.b % 25) = 0)
+                     ->  Hash
+                           Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                           ->  Seq Scan on public.plt1_e_p1
+                                 Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                                 Filter: ((plt1_e_p1.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c, plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                     Hash Cond: ((plt1_p2.a = plt1_e_p2.a) AND (plt1_p2.c = ltrim(plt1_e_p2.c, 'A'::text)))
+                     ->  Hash Full Join
+                           Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+                           Hash Cond: ((plt1_p2.a = plt2_p2.b) AND (plt1_p2.c = plt2_p2.c))
+                           ->  Seq Scan on public.plt1_p2
+                                 Output: plt1_p2.a, plt1_p2.c
+                                 Filter: ((plt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p2.b, plt2_p2.c
+                                 ->  Seq Scan on public.plt2_p2
+                                       Output: plt2_p2.b, plt2_p2.c
+                                       Filter: ((plt2_p2.b % 25) = 0)
+                     ->  Hash
+                           Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                           ->  Seq Scan on public.plt1_e_p2
+                                 Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                                 Filter: ((plt1_e_p2.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c, plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                     Hash Cond: ((plt1_p3.a = plt1_e_p3.a) AND (plt1_p3.c = ltrim(plt1_e_p3.c, 'A'::text)))
+                     ->  Hash Full Join
+                           Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+                           Hash Cond: ((plt1_p3.a = plt2_p3.b) AND (plt1_p3.c = plt2_p3.c))
+                           ->  Seq Scan on public.plt1_p3
+                                 Output: plt1_p3.a, plt1_p3.c
+                                 Filter: ((plt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p3.b, plt2_p3.c
+                                 ->  Seq Scan on public.plt2_p3
+                                       Output: plt2_p3.b, plt2_p3.c
+                                       Filter: ((plt2_p3.b % 25) = 0)
+                     ->  Hash
+                           Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                           ->  Seq Scan on public.plt1_e_p3
+                                 Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                                 Filter: ((plt1_e_p3.a % 25) = 0)
+(63 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.c = t1_3.c)
+               ->  Seq Scan on public.plt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_3.c, t2.c
+                     Hash Cond: (t1_3.c = ltrim(t2.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p1 t1_3
+                           Output: t1_3.c
+                     ->  Hash
+                           Output: t2.c
+                           ->  Seq Scan on public.plt1_e_p1 t2
+                                 Output: t2.c
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.c = t1_4.c)
+               ->  Seq Scan on public.plt1_p2 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_4.c, t2_1.c
+                     Hash Cond: (t1_4.c = ltrim(t2_1.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p2 t1_4
+                           Output: t1_4.c
+                     ->  Hash
+                           Output: t2_1.c
+                           ->  Seq Scan on public.plt1_e_p2 t2_1
+                                 Output: t2_1.c
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.c = t1_5.c)
+               ->  Seq Scan on public.plt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_5.c, t2_2.c
+                     Hash Cond: (t1_5.c = ltrim(t2_2.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p3 t1_5
+                           Output: t1_5.c
+                     ->  Hash
+                           Output: t2_2.c
+                           ->  Seq Scan on public.plt1_e_p3 t2_2
+                                 Output: t2_2.c
+(49 rows)
+
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1, uplt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+                                   QUERY PLAN                                   
+--------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.c = t1_3.c)
+               ->  Seq Scan on public.plt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_3.c, t1_6.c
+                     Hash Cond: (t1_3.c = ltrim(t1_6.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p1 t1_3
+                           Output: t1_3.c
+                     ->  Hash
+                           Output: t1_6.c
+                           ->  HashAggregate
+                                 Output: t1_6.c
+                                 Group Key: ltrim(t1_6.c, 'A'::text)
+                                 ->  Seq Scan on public.plt1_e_p1 t1_6
+                                       Output: t1_6.c, ltrim(t1_6.c, 'A'::text)
+                                       Filter: ((t1_6.a % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.c = t1_4.c)
+               ->  Seq Scan on public.plt1_p2 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_4.c, t1_7.c
+                     Hash Cond: (t1_4.c = ltrim(t1_7.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p2 t1_4
+                           Output: t1_4.c
+                     ->  Hash
+                           Output: t1_7.c
+                           ->  HashAggregate
+                                 Output: t1_7.c
+                                 Group Key: ltrim(t1_7.c, 'A'::text)
+                                 ->  Seq Scan on public.plt1_e_p2 t1_7
+                                       Output: t1_7.c, ltrim(t1_7.c, 'A'::text)
+                                       Filter: ((t1_7.a % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.c = t1_5.c)
+               ->  Seq Scan on public.plt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_5.c, t1_8.c
+                     Hash Cond: (t1_5.c = ltrim(t1_8.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p3 t1_5
+                           Output: t1_5.c
+                     ->  Hash
+                           Output: t1_8.c
+                           ->  HashAggregate
+                                 Output: t1_8.c
+                                 Group Key: ltrim(t1_8.c, 'A'::text)
+                                 ->  Seq Scan on public.plt1_e_p3 t1_8
+                                       Output: t1_8.c, ltrim(t1_8.c, 'A'::text)
+                                       Filter: ((t1_8.a % 25) = 0)
+(61 rows)
+
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+--
+-- negative testcases
+--
+-- joins where one of the relations is proven empty
+EXPLAIN (VERBOSE, 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;
+            QUERY PLAN            
+----------------------------------
+ Result
+   Output: t1.a, t1.c, t2.b, t2.c
+   One-Time Filter: false
+(3 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 LEFT JOIN prt2 t2 ON t1.a = t2.b;
+              QUERY PLAN              
+--------------------------------------
+ Result
+   Output: prt1.a, prt1.c, t2.b, t2.c
+   One-Time Filter: false
+(3 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                     
+---------------------------------------------------
+ Sort
+   Output: a, c, t2.b, t2.c
+   Sort Key: a, t2.b
+   ->  Hash Left Join
+         Output: a, c, t2.b, t2.c
+         Hash Cond: (t2.b = a)
+         ->  Append
+               ->  Seq Scan on public.prt2 t2
+                     Output: t2.b, t2.c
+                     Filter: ((t2.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p1 t2_1
+                     Output: t2_1.b, t2_1.c
+                     Filter: ((t2_1.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p2 t2_2
+                     Output: t2_2.b, t2_2.c
+                     Filter: ((t2_2.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p3 t2_3
+                     Output: t2_3.b, t2_3.c
+                     Filter: ((t2_3.a % 25) = 0)
+         ->  Hash
+               Output: a, c
+               ->  Result
+                     Output: a, c
+                     One-Time Filter: false
+(24 rows)
+
+EXPLAIN (VERBOSE, 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 % 25 = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                     
+---------------------------------------------------
+ Sort
+   Output: a, c, t2.b, t2.c
+   Sort Key: a, t2.b
+   ->  Hash Left Join
+         Output: a, c, t2.b, t2.c
+         Hash Cond: (t2.b = a)
+         ->  Append
+               ->  Seq Scan on public.prt2 t2
+                     Output: t2.b, t2.c
+                     Filter: ((t2.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p1 t2_1
+                     Output: t2_1.b, t2_1.c
+                     Filter: ((t2_1.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p2 t2_2
+                     Output: t2_2.b, t2_2.c
+                     Filter: ((t2_2.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p3 t2_3
+                     Output: t2_3.b, t2_3.c
+                     Filter: ((t2_3.a % 25) = 0)
+         ->  Hash
+               Output: a, c
+               ->  Result
+                     Output: a, c
+                     One-Time Filter: false
+(24 rows)
+
+CREATE TABLE prt1_n (a int, b int, c varchar) PARTITION BY RANGE(c);
+CREATE TABLE prt1_n_p1 PARTITION OF prt1_n FOR VALUES FROM ('0000') TO ('0250');
+CREATE TABLE prt1_n_p2 PARTITION OF prt1_n FOR VALUES FROM ('0250') TO ('0500');
+INSERT INTO prt1_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 499, 2) i;
+ANALYZE prt1_n;
+ANALYZE prt1_n_p1;
+ANALYZE prt1_n_p2;
+CREATE TABLE prt2_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt2_n_p1 PARTITION OF prt2_n FOR VALUES IN ('0000', '0003', '0004', '0010', '0006', '0007');
+CREATE TABLE prt2_n_p2 PARTITION OF prt2_n FOR VALUES IN ('0001', '0005', '0002', '0009', '0008', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt2_n;
+ANALYZE prt2_n_p1;
+ANALYZE prt2_n_p2;
+CREATE TABLE prt3_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt3_n_p1 PARTITION OF prt3_n FOR VALUES IN ('0000', '0004', '0006', '0007');
+CREATE TABLE prt3_n_p2 PARTITION OF prt3_n FOR VALUES IN ('0001', '0002', '0008', '0010');
+CREATE TABLE prt3_n_p3 PARTITION OF prt3_n FOR VALUES IN ('0003', '0005', '0009', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt3_n;
+ANALYZE prt3_n_p1;
+ANALYZE prt3_n_p2;
+ANALYZE prt3_n_p3;
+CREATE TABLE prt4_n (a int, b int, c text) PARTITION BY RANGE(a);
+CREATE TABLE prt4_n_p1 PARTITION OF prt4_n FOR VALUES FROM (0) TO (300);
+CREATE TABLE prt4_n_p2 PARTITION OF prt4_n FOR VALUES FROM (300) TO (500);
+CREATE TABLE prt4_n_p3 PARTITION OF prt4_n FOR VALUES FROM (500) TO (600);
+INSERT INTO prt4_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt4_n;
+ANALYZE prt4_n_p1;
+ANALYZE prt4_n_p2;
+ANALYZE prt4_n_p3;
+-- partition-wise join can not be applied if the partition ranges differ
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Join
+   Hash Cond: (t1.a = t2.a)
+   ->  Append
+         ->  Seq Scan on prt1 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p2 t1_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt4_n t2
+               ->  Seq Scan on prt4_n_p1 t2_1
+               ->  Seq Scan on prt4_n_p2 t2_2
+               ->  Seq Scan on prt4_n_p3 t2_3
+(13 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 FULL JOIN prt4_n t2 ON t1.a = t2.a;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Full Join
+   Hash Cond: (t1.a = t2.a)
+   ->  Append
+         ->  Seq Scan on prt1 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p2 t1_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt4_n t2
+               ->  Seq Scan on prt4_n_p1 t2_1
+               ->  Seq Scan on prt4_n_p2 t2_2
+               ->  Seq Scan on prt4_n_p3 t2_3
+(13 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                        
+---------------------------------------------------------
+ Nested Loop Left Join
+   ->  Append
+         ->  Seq Scan on prt1 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p2 t1_3
+   ->  Append
+         ->  Seq Scan on prt2 t2
+               Filter: (t1.a < b)
+         ->  Index Scan using iprt2_p1_b on prt2_p1 t2_1
+               Index Cond: (t1.a < b)
+         ->  Index Scan using iprt2_p2_b on prt2_p2 t2_2
+               Index Cond: (t1.a < b)
+         ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+               Index Cond: (t1.a < b)
+(15 rows)
+
+-- equi-join with join condition on partial keys does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1, prt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t1.a % 25 = 0;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Join
+   Hash Cond: (((t2.b + t2.a) / 2) = t1.a)
+   ->  Append
+         ->  Seq Scan on prt2_m t2
+         ->  Seq Scan on prt2_m_p1 t2_1
+         ->  Seq Scan on prt2_m_p2 t2_2
+         ->  Seq Scan on prt2_m_p3 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_m t1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p1 t1_1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p2 t1_2
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p3 t1_3
+                     Filter: ((a % 25) = 0)
+(17 rows)
+
+-- equi-join between out-of-order partition key columns does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.a = t2.b WHERE t1.a % 25 = 0;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Right Join
+   Hash Cond: (t2.b = t1.a)
+   ->  Append
+         ->  Seq Scan on prt2_m t2
+         ->  Seq Scan on prt2_m_p1 t2_1
+         ->  Seq Scan on prt2_m_p2 t2_2
+         ->  Seq Scan on prt2_m_p3 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_m t1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p1 t1_1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p2 t1_2
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p3 t1_3
+                     Filter: ((a % 25) = 0)
+(17 rows)
+
+-- equi-join between non-key columns does not qualify for partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.c = t2.c WHERE t1.a % 25 = 0;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Right Join
+   Hash Cond: ((t2.c)::text = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt2_m t2
+         ->  Seq Scan on prt2_m_p1 t2_1
+         ->  Seq Scan on prt2_m_p2 t2_2
+         ->  Seq Scan on prt2_m_p3 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_m t1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p1 t1_1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p2 t1_2
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p3 t1_3
+                     Filter: ((a % 25) = 0)
+(17 rows)
+
+-- partition-wise join can not be applied for a join between list and range
+-- partitioned table
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1, prt2_n t2 WHERE t1.c = t2.c;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Join
+   Hash Cond: (t2.c = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt2_n t2
+         ->  Seq Scan on prt2_n_p1 t2_1
+         ->  Seq Scan on prt2_n_p2 t2_2
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(11 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 LEFT JOIN prt2_n t2 ON (t1.c = t2.c);
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Right Join
+   Hash Cond: (t2.c = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt2_n t2
+         ->  Seq Scan on prt2_n_p1 t2_1
+         ->  Seq Scan on prt2_n_p2 t2_2
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(11 rows)
+
+-- partition-wise join can not be applied between tables with different
+-- partition lists
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 RIGHT JOIN prt1 t2 ON (t1.c = t2.c);
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Left Join
+   Hash Cond: ((t2.c)::text = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt1 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p2 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(12 rows)
+
+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);
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Full Join
+   Hash Cond: ((t2.c)::text = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt1 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p2 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(12 rows)
+
diff --git a/src/test/regress/parallel_schedule b/src/test/regress/parallel_schedule
index edeb2d6..ac38f50 100644
--- a/src/test/regress/parallel_schedule
+++ b/src/test/regress/parallel_schedule
@@ -103,7 +103,7 @@ test: publication subscription
 # ----------
 # Another group of parallel tests
 # ----------
-test: select_views portals_p2 foreign_key cluster dependency guc bitmapops combocid tsearch tsdicts foreign_data window xmlmap functional_deps advisory_lock json jsonb json_encoding indirect_toast equivclass
+test: select_views portals_p2 foreign_key cluster dependency guc bitmapops combocid tsearch tsdicts foreign_data window xmlmap functional_deps advisory_lock json jsonb json_encoding indirect_toast equivclass partition_join
 # ----------
 # Another group of parallel tests
 # NB: temp.sql does a reconnect which transiently uses 2 connections,
diff --git a/src/test/regress/serial_schedule b/src/test/regress/serial_schedule
index 27a46d7..1bf98a4 100644
--- a/src/test/regress/serial_schedule
+++ b/src/test/regress/serial_schedule
@@ -172,3 +172,4 @@ test: with
 test: xml
 test: event_trigger
 test: stats
+test: partition_join
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
new file mode 100644
index 0000000..0322f1e
--- /dev/null
+++ b/src/test/regress/sql/partition_join.sql
@@ -0,0 +1,515 @@
+--
+-- PARTITION_JOIN
+-- Test partition-wise join between partitioned tables
+--
+
+--
+-- partitioned by a single column
+--
+CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+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, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1 AS SELECT * FROM prt1;
+
+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_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, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+CREATE TABLE uprt2 AS SELECT * FROM prt2;
+
+-- inner join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+-- left outer join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+-- right outer join
+EXPLAIN (VERBOSE, 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.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+-- full outer join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+
+-- Join with pruned partitions from joining relations
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a < 450 AND t2.b > 250 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+-- lateral reference
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+--
+-- partitioned by expression
+--
+CREATE TABLE prt1_e (a int, b int, c varchar) PARTITION BY RANGE(((a + b)/2));
+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);
+INSERT INTO prt1_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_e AS SELECT * FROM prt1_e;
+
+CREATE TABLE prt2_e (a int, b int, c varchar) PARTITION BY RANGE(((b + a)/2));
+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);
+INSERT INTO prt2_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_e;
+ANALYZE prt2_e_p1;
+ANALYZE prt2_e_p2;
+ANALYZE prt2_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_e AS SELECT * FROM prt2_e;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1, uprt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1 LEFT JOIN uprt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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.a % 25 = 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM uprt1 t1, uprt2 t2, uprt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM uprt1_e WHERE uprt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, 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.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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;
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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;
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM uprt1_e WHERE uprt1_e.a % 25 = 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;
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1, uprt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+-- test merge joins with and without using indexes
+SET enable_hashjoin TO off;
+SET enable_nestloop TO off;
+
+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 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 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));
+
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+SET enable_seqscan TO off;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+-- lateral references and parameterized paths
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+RESET enable_hashjoin;
+RESET enable_nestloop;
+RESET enable_seqscan;
+
+--
+-- partitioned by multiple columns
+--
+CREATE TABLE prt1_m (a int, b int, c varchar) 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, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_m;
+ANALYZE prt1_m_p1;
+ANALYZE prt1_m_p2;
+ANALYZE prt1_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_m AS SELECT * FROM prt1_m;
+
+CREATE TABLE prt2_m (a int, b int, c varchar) 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, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_m;
+ANALYZE prt2_m_p1;
+ANALYZE prt2_m_p2;
+ANALYZE prt2_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_m AS SELECT * FROM prt2_m;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_m t1 RIGHT JOIN uprt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_m t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_m t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+
+--
+-- 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;
+ANALYZE plt1_p1;
+ANALYZE plt1_p2;
+ANALYZE plt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1 AS SELECT * FROM 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;
+ANALYZE plt2_p1;
+ANALYZE plt2_p2;
+ANALYZE plt2_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt2 AS SELECT * FROM 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;
+ANALYZE plt1_e_p1;
+ANALYZE plt1_e_p2;
+ANALYZE plt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1_e AS SELECT * FROM plt1_e;
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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;
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM uplt1 t1, uplt2 t2, uplt1_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;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1, uplt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 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 ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+--
+-- negative testcases
+--
+
+-- joins where one of the relations is proven empty
+EXPLAIN (VERBOSE, 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;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 LEFT JOIN prt2 t2 ON t1.a = t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, 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 % 25 = 0 ORDER BY t1.a, t2.b;
+
+CREATE TABLE prt1_n (a int, b int, c varchar) PARTITION BY RANGE(c);
+CREATE TABLE prt1_n_p1 PARTITION OF prt1_n FOR VALUES FROM ('0000') TO ('0250');
+CREATE TABLE prt1_n_p2 PARTITION OF prt1_n FOR VALUES FROM ('0250') TO ('0500');
+INSERT INTO prt1_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 499, 2) i;
+ANALYZE prt1_n;
+ANALYZE prt1_n_p1;
+ANALYZE prt1_n_p2;
+
+CREATE TABLE prt2_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt2_n_p1 PARTITION OF prt2_n FOR VALUES IN ('0000', '0003', '0004', '0010', '0006', '0007');
+CREATE TABLE prt2_n_p2 PARTITION OF prt2_n FOR VALUES IN ('0001', '0005', '0002', '0009', '0008', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt2_n;
+ANALYZE prt2_n_p1;
+ANALYZE prt2_n_p2;
+
+CREATE TABLE prt3_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt3_n_p1 PARTITION OF prt3_n FOR VALUES IN ('0000', '0004', '0006', '0007');
+CREATE TABLE prt3_n_p2 PARTITION OF prt3_n FOR VALUES IN ('0001', '0002', '0008', '0010');
+CREATE TABLE prt3_n_p3 PARTITION OF prt3_n FOR VALUES IN ('0003', '0005', '0009', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt3_n;
+ANALYZE prt3_n_p1;
+ANALYZE prt3_n_p2;
+ANALYZE prt3_n_p3;
+
+CREATE TABLE prt4_n (a int, b int, c text) PARTITION BY RANGE(a);
+CREATE TABLE prt4_n_p1 PARTITION OF prt4_n FOR VALUES FROM (0) TO (300);
+CREATE TABLE prt4_n_p2 PARTITION OF prt4_n FOR VALUES FROM (300) TO (500);
+CREATE TABLE prt4_n_p3 PARTITION OF prt4_n FOR VALUES FROM (500) TO (600);
+INSERT INTO prt4_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt4_n;
+ANALYZE prt4_n_p1;
+ANALYZE prt4_n_p2;
+ANALYZE prt4_n_p3;
+
+-- partition-wise join can not be applied if the partition ranges differ
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 FULL JOIN prt4_n t2 ON t1.a = t2.a;
+
+-- 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);
+
+-- equi-join with join condition on partial keys does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1, prt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t1.a % 25 = 0;
+
+-- equi-join between out-of-order partition key columns does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.a = t2.b WHERE t1.a % 25 = 0;
+
+-- equi-join between non-key columns does not qualify for partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.c = t2.c WHERE t1.a % 25 = 0;
+
+-- partition-wise join can not be applied for a join between list and range
+-- partitioned table
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1, prt2_n t2 WHERE t1.c = t2.c;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 LEFT JOIN prt2_n t2 ON (t1.c = t2.c);
+
+-- partition-wise join can not be applied between tables with different
+-- partition lists
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 RIGHT JOIN prt1 t2 ON (t1.c = t2.c);
+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);
-- 
1.7.9.5

0008-Partition-wise-join.patchapplication/octet-stream; name=0008-Partition-wise-join.patchDownload
From b9ceb880a2377ae2570a3fb2aa4f862d5b8846fd Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Tue, 7 Feb 2017 16:04:03 +0530
Subject: [PATCH 08/11] Partition-wise join

Implement partition-wise join for join between single level partitioned tables.
The details of this technique can be found in optimizer/README, where most of
the implementation has been explained.

We obtain clauses applicable to a child-join by translating corresponding
clauses of the parent. Because child-join can be computed by different
combinations of joining child relations, a given clause is required to be
translated multiple times. In order to reduce the memory consumption, we keep a
repository of child-clauses derived from a parent clause and search in that
repository before translating.

Tests for semi-join, those forcing a merge join for child-join and those
testing lateral join will crash with this patch. The tests testing joins with
partition pruning will also fail. Those crashes and failures are because the
existing code does not expect a child-join to appear in certain cases.
Following patches will fix that code.
---
 src/backend/nodes/copyfuncs.c                |    9 +
 src/backend/optimizer/README                 |   53 ++++
 src/backend/optimizer/path/allpaths.c        |  305 ++++++++++++++++---
 src/backend/optimizer/path/costsize.c        |    3 +
 src/backend/optimizer/path/joinpath.c        |   21 +-
 src/backend/optimizer/path/joinrels.c        |  421 ++++++++++++++++++++++++++
 src/backend/optimizer/plan/createplan.c      |  227 +++++++++++++-
 src/backend/optimizer/prep/prepunion.c       |  160 ++++++++++
 src/backend/optimizer/util/pathnode.c        |  113 +++++++
 src/backend/optimizer/util/placeholder.c     |   55 ++++
 src/backend/optimizer/util/relnode.c         |  266 +++++++++++++++-
 src/backend/utils/misc/guc.c                 |   28 ++
 src/include/nodes/nodes.h                    |    1 +
 src/include/nodes/relation.h                 |   57 ++++
 src/include/optimizer/cost.h                 |    5 +
 src/include/optimizer/pathnode.h             |    6 +
 src/include/optimizer/paths.h                |    5 +
 src/include/optimizer/placeholder.h          |    2 +
 src/include/optimizer/prep.h                 |    8 +
 src/test/regress/expected/partition_join.out |    4 +
 src/test/regress/expected/sysviews.out       |   29 +-
 src/test/regress/sql/partition_join.sql      |    5 +
 22 files changed, 1713 insertions(+), 70 deletions(-)

diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 30d733e..72c021e 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -2070,6 +2070,15 @@ _copyRestrictInfo(const RestrictInfo *from)
 	COPY_SCALAR_FIELD(left_bucketsize);
 	COPY_SCALAR_FIELD(right_bucketsize);
 
+	/*
+	 * Do not copy parent_rinfo and child_rinfos because 1. they create a
+	 * circular dependency between child and parent RestrictInfo 2. dropping
+	 * those links just means that we loose some memory optimizations. 3. There
+	 * is a possibility that the child and parent RestrictInfots themselves may
+	 * have got copied and thus the old links may no longer be valid. The
+	 * caller may set up those links itself, if needed.
+	 */
+
 	return newnode;
 }
 
diff --git a/src/backend/optimizer/README b/src/backend/optimizer/README
index fc0fca4..7565ae4 100644
--- a/src/backend/optimizer/README
+++ b/src/backend/optimizer/README
@@ -1076,3 +1076,56 @@ be desirable to postpone the Gather stage until as near to the top of the
 plan as possible.  Expanding the range of cases in which more work can be
 pushed below the Gather (and costing them accurately) is likely to keep us
 busy for a long time to come.
+
+Partition-wise 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 for a given row in a given partition of a given
+partitioned table, its joining row, if exists, should exist only in the
+matching partition of the other partitioned table; no row from non-matching
+partitions in the other partitioned table can join with the given row from the
+first table. This condition allows the join between partitioned table 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
+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 partition tables
+into join between their partitions is called partition-wise join. We will use
+term "partitioned relation" for both partitioned table as well as join between
+partitioned tables which can use partition-wise join technique.
+
+Partitioning properties of a partitioned table are stored in
+PartitionSchemeData structure. Planner maintains a list of canonical partition
+schemes (distinct PartitionSchemeData objects) so that any two partitioned
+relations with same partitioning scheme share the same PartitionSchemeData
+object. This reduces memory consumed by PartitionSchemeData objects and makes
+it easy to compare the partition schemes of joining relations. RelOptInfos of
+partitioned relations hold partition key expressions and the RelOptInfos of
+the partition relations of that relation.
+
+Partition-wise joins are planned in two phases
+
+1. First phase creates the RelOptInfos for joins between matching partitions,
+henceforth referred to as child-joins. The number of paths created for a
+child-join i.e. join between partitions is same as the number of paths created
+for join between parents. That number grows exponentially with the number of
+base relations being joined. The time and memory consumed to create paths for
+each child-join will be proporional to the number of partitions. This will not
+scale well with thousands of partitions. Instead of that we estimate
+partition-wise join cost based on the costs of sampled child-joins. We choose
+child-joins with higher sizes to have realistic estimates. If the number of
+sampled child-joins is same as the number of live child-joins, we create append
+paths as we know costs of all required child-joins. Otherwise we create
+PartitionJoinPaths with cost estimates based on the costs of sampled
+child-joins. While creating append paths or PartitionJoin paths we create paths
+for all the different possible parameterizations and pathkeys available in the
+sampled child-joins.
+
+2. If PartitionJoinPath emerges as the best possible path, we create paths for
+each unsampled child-join. From every child-join we choose the cheapest path
+with same parameterization or pathkeys as the PartitionJoinPath. This path is
+converted into a plan and all the child-join plans are combined using an Append
+or MergeAppend plan as appropriate. We use a fresh memory context for planning
+each unsampled child-join, thus reducing memory consumption.
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 0eb56f3..1adf6ba 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -15,6 +15,7 @@
 
 #include "postgres.h"
 
+#include "miscadmin.h"
 #include <limits.h>
 #include <math.h>
 
@@ -93,8 +94,8 @@ static void set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 static void set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 						Index rti, RangeTblEntry *rte);
 static void generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
-						   List *live_childrels,
-						   List *all_child_pathkeys);
+						   List *live_childrels, List *all_child_pathkeys,
+						   bool partition_join_path);
 static Path *get_cheapest_parameterized_child_path(PlannerInfo *root,
 									  RelOptInfo *rel,
 									  Relids required_outer);
@@ -128,8 +129,8 @@ static void recurse_push_qual(Node *setOp, Query *topquery,
 static void remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel);
 static int	compute_parallel_worker(RelOptInfo *rel, BlockNumber pages);
 static void add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
-									List *live_childrels);
-
+							   List *live_childrels, bool partition_join_path);
+static int compare_rel_size(const void *rel1_p, const void *rel2_p);
 
 /*
  * make_one_rel
@@ -891,6 +892,12 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		childrel = find_base_rel(root, childRTindex);
 		Assert(childrel->reloptkind == RELOPT_OTHER_MEMBER_REL);
 
+		/* Pass top parent's relids down the inheritance hierarchy. */
+		if (rel->top_parent_relids)
+			childrel->top_parent_relids = rel->top_parent_relids;
+		else
+			childrel->top_parent_relids = bms_copy(rel->relids);
+
 		/*
 		 * Two partitioned tables with the same partitioning scheme, have their
 		 * partition bounds arranged in the same order. The order of partition
@@ -900,10 +907,15 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		 * RelOptInfos. Arranging RelOptInfos of partitions in the same order
 		 * as their OIDs makes it easy to find the RelOptInfos of matching
 		 * partitions for partition-wise join.
+		 *
+		 * For a partitioned tables, individual partitions can participate in
+		 * the pair-wise joins. We need attr_needed data for building
+		 * child-join targetlists.
 		 */
 		if (rel->part_scheme)
 		{
 			int		cnt_parts;
+			AttrNumber		attno;
 
 			for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
 			{
@@ -913,6 +925,38 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 					rel->part_rels[cnt_parts] = childrel;
 				}
 			}
+
+			for (attno = rel->min_attr; attno <= rel->max_attr; attno++)
+			{
+				int	index = attno - rel->min_attr;
+				Relids	attr_needed = bms_copy(rel->attr_needed[index]);
+
+				/*
+				 * System attributes do not need translation. In such a case,
+				 * the attribute numbers of the parent and the child should
+				 * start from the same minimum attribute.
+				 */
+				if (attno <= 0)
+				{
+					Assert(rel->min_attr == childrel->min_attr);
+					childrel->attr_needed[index] = attr_needed;
+				}
+				else
+				{
+					Var *var = list_nth(appinfo->translated_vars,
+										attno - 1);
+					int child_index;
+
+					/*
+					 * Parent Var for a user defined attribute translates to
+					 * child Var.
+					 */
+					Assert(IsA(var, Var));
+
+					child_index = var->varattno - childrel->min_attr;
+					childrel->attr_needed[child_index] = attr_needed;
+				}
+			}
 		}
 
 		/*
@@ -1057,10 +1101,8 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		 * PlaceHolderVars.)  XXX we do not bother to update the cost or width
 		 * fields of childrel->reltarget; not clear if that would be useful.
 		 */
-		childrel->joininfo = (List *)
-			adjust_appendrel_attrs(root,
-								   (Node *) rel->joininfo,
-								   appinfo_list);
+		childrel->joininfo = build_child_clauses(root, rel->joininfo,
+												 appinfo_list);
 		childrel->reltarget->exprs = (List *)
 			adjust_appendrel_attrs(root,
 								   (Node *) rel->reltarget->exprs,
@@ -1079,14 +1121,6 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		childrel->has_eclass_joins = rel->has_eclass_joins;
 
 		/*
-		 * Note: we could compute appropriate attr_needed data for the child's
-		 * variables, by transforming the parent's attr_needed through the
-		 * translated_vars mapping.  However, currently there's no need
-		 * because attr_needed is only examined for base relations not
-		 * otherrels.  So we just leave the child's attr_needed empty.
-		 */
-
-		/*
 		 * If parallelism is allowable for this query in general, see whether
 		 * it's allowable for this childrel in particular.  But if we've
 		 * already decided the appendrel is not parallel-safe as a whole,
@@ -1269,10 +1303,9 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 	}
 
 	/* Add Append/MergeAppend paths to the "append" relation. */
-	add_paths_to_append_rel(root, rel, live_childrels);
+	add_paths_to_append_rel(root, rel, live_childrels, false);
 }
 
-
 /*
  * add_paths_to_append_rel
  *		Generate Append/MergeAppend paths for given "append" relation.
@@ -1282,20 +1315,44 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
  * an append path collecting one path from each non-dummy child with given
  * parameterization or ordering. Similarly it collects partial paths from
  * non-dummy children to create partial append paths.
+ *
+ * When called on partitioned join relation with partition_join_path = true, it
+ * adds PartitionJoinPath instead of Merge/Append path. This path is costed
+ * based on the costs of sampled child-join and is expanded later into
+ * Merge/Append plan.
  */
 static void
 add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
-						List *live_childrels)
+						List *live_childrels, bool partition_join_path)
 {
 	List	   *subpaths = NIL;
 	bool		subpaths_valid = true;
 	List	   *partial_subpaths = NIL;
-	bool		partial_subpaths_valid = true;
+	bool		partial_subpaths_valid;
 	List	   *all_child_pathkeys = NIL;
 	List	   *all_child_outers = NIL;
 	ListCell   *l;
 
 	/*
+	 * While creating PartitionJoinPath, we sample paths from only a few child
+	 * relations. Even if all of sampled children have partial paths, it's not
+	 * guaranteed that all the unsampled children will have partial paths.
+	 * Hence we do not create partial PartitionJoinPaths.
+	 */
+	partial_subpaths_valid = !partition_join_path ? true : false;
+
+	/* An append relation with all its children dummy is dummy. */
+	if (live_childrels == NIL)
+	{
+		/* Mark the relation as dummy, if not already done so. */
+		if (!IS_DUMMY_REL(rel))
+			set_dummy_rel_pathlist(rel);
+
+		/* No more paths need to be added. */
+		return;
+	}
+
+	/*
 	 * For every non-dummy child, remember the cheapest path.  Also, identify
 	 * all pathkeys (orderings) and parameterizations (required_outer sets)
 	 * available for the non-dummy member relations.
@@ -1394,7 +1451,17 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	 * if we have zero or one live subpath due to constraint exclusion.)
 	 */
 	if (subpaths_valid)
-		add_path(rel, (Path *) create_append_path(rel, subpaths, NULL, 0));
+	{
+		Path *path;
+
+		if (partition_join_path)
+			path = (Path *) create_partition_join_path(root, rel, subpaths,
+													   NULL, NIL);
+		else
+			path = (Path *) create_append_path(rel, subpaths, NULL, 0);
+
+		add_path(rel, path);
+	}
 
 	/*
 	 * Consider an append of partial unordered, unparameterized partial paths.
@@ -1405,6 +1472,8 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		ListCell   *lc;
 		int			parallel_workers = 0;
 
+		Assert(!partition_join_path);
+
 		/*
 		 * Decide on the number of workers to request for this append path.
 		 * For now, we just use the maximum value from among the members.  It
@@ -1431,7 +1500,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	 */
 	if (subpaths_valid)
 		generate_mergeappend_paths(root, rel, live_childrels,
-								   all_child_pathkeys);
+								   all_child_pathkeys, partition_join_path);
 
 	/*
 	 * Build Append paths for each parameterization seen among the child rels.
@@ -1472,8 +1541,18 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		}
 
 		if (subpaths_valid)
-			add_path(rel, (Path *)
-					 create_append_path(rel, subpaths, required_outer, 0));
+		{
+			Path *path;
+
+			if (partition_join_path)
+				path = (Path *) create_partition_join_path(root, rel, subpaths,
+														   required_outer, NIL);
+			else
+				path = (Path *) create_append_path(rel, subpaths,
+												   required_outer, 0);
+
+			add_path(rel, path);
+		}
 	}
 }
 
@@ -1499,11 +1578,16 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
  * parameterized mergejoin plans, it might be worth adding support for
  * parameterized MergeAppends to feed such joins.  (See notes in
  * optimizer/README for why that might not ever happen, though.)
+ *
+ * When called on partitioned join relation with partition_join_path = true, it
+ * adds PartitionJoinPath with pathkeys instead of MergeAppend path. This path
+ * is costed based on the costs of sampled child-join and is expanded later
+ * into MergeAppend plan.
  */
 static void
 generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
 						   List *live_childrels,
-						   List *all_child_pathkeys)
+						   List *all_child_pathkeys, bool partition_join_path)
 {
 	ListCell   *lcp;
 
@@ -1514,6 +1598,7 @@ generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
 		List	   *total_subpaths = NIL;
 		bool		startup_neq_total = false;
 		ListCell   *lcr;
+		Path	   *path;
 
 		/* Select the child paths for this ordering... */
 		foreach(lcr, live_childrels)
@@ -1560,18 +1645,29 @@ generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
 				accumulate_append_subpath(total_subpaths, cheapest_total);
 		}
 
-		/* ... and build the MergeAppend paths */
-		add_path(rel, (Path *) create_merge_append_path(root,
-														rel,
-														startup_subpaths,
-														pathkeys,
-														NULL));
+		/* ... and build the paths */
+		if (partition_join_path)
+			path = (Path *) create_partition_join_path(root, rel,
+													   startup_subpaths,
+													   NULL, pathkeys);
+		else
+			path = (Path *) create_merge_append_path(root, rel,
+													 startup_subpaths,
+													 pathkeys, NULL);
+		add_path(rel, path);
+
 		if (startup_neq_total)
-			add_path(rel, (Path *) create_merge_append_path(root,
-															rel,
-															total_subpaths,
-															pathkeys,
-															NULL));
+		{
+			if (partition_join_path)
+				path = (Path *) create_partition_join_path(root, rel,
+														   total_subpaths,
+														   NULL, pathkeys);
+			else
+				path = (Path *) create_merge_append_path(root, rel,
+														 total_subpaths,
+														 pathkeys, NULL);
+			add_path(rel, path);
+		}
 	}
 }
 
@@ -2316,8 +2412,17 @@ standard_join_search(PlannerInfo *root, int levels_needed, List *initial_rels)
 		 * Run generate_gather_paths() for each just-processed joinrel.  We
 		 * could not do this earlier because both regular and partial paths
 		 * can get added to a particular joinrel at multiple times within
-		 * join_search_one_level.  After that, we're done creating paths for
-		 * the joinrel, so run set_cheapest().
+		 * join_search_one_level.
+		 *
+		 * Similarly, create paths for joinrels which used partition-wise join
+		 * technique. generate_partition_wise_join_paths() creates paths for
+		 * only few of the child-joins with highest sizes. Though we calculate
+		 * size of a child-join only once; when it gets created, it may be
+		 * deemed empty while considering various join orders within
+		 * join_search_one_level.
+		 *
+		 * After that, we're done creating paths for the joinrel, so run
+		 * set_cheapest().
 		 */
 		foreach(lc, root->join_rel_level[lev])
 		{
@@ -2326,6 +2431,9 @@ standard_join_search(PlannerInfo *root, int levels_needed, List *initial_rels)
 			/* Create GatherPaths for any useful partial paths for rel */
 			generate_gather_paths(root, rel);
 
+			/* Create paths for partition-wise joins. */
+			generate_partition_wise_join_paths(root, rel);
+
 			/* Find and save the cheapest paths for this rel */
 			set_cheapest(rel);
 
@@ -3011,6 +3119,127 @@ compute_parallel_worker(RelOptInfo *rel, BlockNumber pages)
 	return parallel_workers;
 }
 
+/*
+ * Function to compare estimated sizes of two relations to be used with
+ * qsort().
+ */
+static int
+compare_rel_size(const void *rel1_p, const void *rel2_p)
+{
+	RelOptInfo *rel1 = *(RelOptInfo **) rel1_p;
+	RelOptInfo *rel2 = *(RelOptInfo **) rel2_p;
+
+	return (int) (rel1->rows - rel2->rows);
+}
+
+/*
+ * generate_partition_wise_join_paths
+ *
+ * 		Create paths representing partition-wise join for given partitioned
+ * 		join relation.
+ *
+ * The number of paths created for a child-join is same as the number of paths
+ * created for join between parents. That number grows exponentially with the
+ * number of base relations being joined. The time and memory consumed to
+ * create paths for each child-join will be proporional to the number of
+ * partitions. This will not scale well with thousands of partitions. Instead
+ * of that we estimate partition-wise join cost based on the costs of sampled
+ * child-joins. We choose child-joins with higher sizes to have realistic
+ * estimates.
+ *
+ * This must be called after we have considered all joining orders since
+ * certain join orders may allow us to deem a child-join as dummy.
+ */
+void
+generate_partition_wise_join_paths(PlannerInfo *root, RelOptInfo *rel)
+{
+	List   *sampled_children = NIL;
+	int		cnt_parts;
+	int		num_part_to_plan;
+	int		num_parts;
+	bool	partition_join_path = false;
+	int		num_dummy_parts = 0;
+	RelOptInfo	  **ordered_part_rels;
+
+	/* Handle only join relations. */
+	if (!IS_JOIN_REL(rel))
+		return;
+
+	/*
+	 * If none of the join orders for this relation could use partition-wise
+	 * join technique, the join is not partitioned. Reset the partitioning
+	 * scheme.
+	 */
+	if (!rel->part_rels)
+		rel->part_scheme = NULL;
+
+	/* If the relation is not partitioned or is proven dummy, nothing to do. */
+	if (!rel->part_scheme || IS_DUMMY_REL(rel))
+		return;
+
+	/* Guard against stack overflow due to overly deep partition hierarchy. */
+	check_stack_depth();
+
+	num_parts = rel->part_scheme->nparts;
+
+	/* Calculate number of child-joins to sample. */
+	num_part_to_plan = num_parts * sample_partition_fraction;
+	if (num_part_to_plan < 1)
+		num_part_to_plan = 1;
+
+	/* Order the child-join relations by their size. */
+	ordered_part_rels = (RelOptInfo **) palloc(sizeof(RelOptInfo *) *
+															num_parts);
+	memcpy(ordered_part_rels, rel->part_rels,
+		   sizeof(RelOptInfo *) * num_parts);
+	qsort(ordered_part_rels, num_parts, sizeof(RelOptInfo *),
+		  compare_rel_size);
+
+	/*
+	 * Create paths for the child-joins for required number of largest
+	 * relations. qsort() returns relations ordered in ascending sizes, so
+	 * start from the end of the array.
+	 */
+	for (cnt_parts = num_parts - 1; cnt_parts >= 0; cnt_parts--)
+	{
+		RelOptInfo *child_rel = rel->part_rels[cnt_parts];
+
+		/* Create paths for this child. */
+		add_paths_to_child_joinrel(root, rel, cnt_parts);
+
+		/* Dummy children will not be scanned, so ingore those. */
+		if (IS_DUMMY_REL(child_rel))
+		{
+			num_dummy_parts++;
+			continue;
+		}
+
+#ifdef OPTIMIZER_DEBUG
+		debug_print_rel(root, rel);
+#endif
+
+		sampled_children = lappend(sampled_children, child_rel);
+
+		if (list_length(sampled_children) >= num_part_to_plan)
+			break;
+	}
+	pfree(ordered_part_rels);
+
+	/*
+	 * If the number of samples is same as the number of live children, an
+	 * append path will do. Otherwise, we will cost the partition-wise join
+	 * based on the sampled children using PartitionJoinPath.
+	 */
+	if (num_part_to_plan < num_parts - num_dummy_parts)
+		partition_join_path = true;
+
+	/* Add paths for partition-wise join based on the sampled children. */
+	add_paths_to_append_rel(root, rel, sampled_children, partition_join_path);
+
+	if (sampled_children)
+		list_free(sampled_children);
+}
+
 
 /*****************************************************************************
  *			DEBUG SUPPORT
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index a43daa7..c720115 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -126,6 +126,9 @@ bool		enable_nestloop = true;
 bool		enable_material = true;
 bool		enable_mergejoin = true;
 bool		enable_hashjoin = true;
+bool		enable_partition_wise_join = true;
+double		partition_wise_plan_weight = DEFAULT_PARTITION_WISE_PLAN_WEIGHT;
+double		sample_partition_fraction = DEFAULT_SAMPLE_PARTITION_FRACTION;
 
 typedef struct
 {
diff --git a/src/backend/optimizer/path/joinpath.c b/src/backend/optimizer/path/joinpath.c
index 2897245..f80fb25 100644
--- a/src/backend/optimizer/path/joinpath.c
+++ b/src/backend/optimizer/path/joinpath.c
@@ -96,6 +96,19 @@ add_paths_to_joinrel(PlannerInfo *root,
 	JoinPathExtraData extra;
 	bool		mergejoin_allowed = true;
 	ListCell   *lc;
+	Relids		joinrelids;
+
+	/*
+	 * PlannerInfo doesn't contain the SpecialJoinInfos created for joins
+	 * between child relations, even if there is a SpecialJoinInfo node for
+	 * the join between the topmost parents. Hence while calculating Relids
+	 * set representing the restriction, consider relids of topmost parent
+	 * of partitions.
+	 */
+	if (joinrel->reloptkind == RELOPT_OTHER_JOINREL)
+		joinrelids = joinrel->top_parent_relids;
+	else
+		joinrelids = joinrel->relids;
 
 	extra.restrictlist = restrictlist;
 	extra.mergeclause_list = NIL;
@@ -149,16 +162,16 @@ add_paths_to_joinrel(PlannerInfo *root,
 		 * join has already been proven legal.)  If the SJ is relevant, it
 		 * presents constraints for joining to anything not in its RHS.
 		 */
-		if (bms_overlap(joinrel->relids, sjinfo2->min_righthand) &&
-			!bms_overlap(joinrel->relids, sjinfo2->min_lefthand))
+		if (bms_overlap(joinrelids, sjinfo2->min_righthand) &&
+			!bms_overlap(joinrelids, sjinfo2->min_lefthand))
 			extra.param_source_rels = bms_join(extra.param_source_rels,
 										   bms_difference(root->all_baserels,
 													sjinfo2->min_righthand));
 
 		/* full joins constrain both sides symmetrically */
 		if (sjinfo2->jointype == JOIN_FULL &&
-			bms_overlap(joinrel->relids, sjinfo2->min_lefthand) &&
-			!bms_overlap(joinrel->relids, sjinfo2->min_righthand))
+			bms_overlap(joinrelids, sjinfo2->min_lefthand) &&
+			!bms_overlap(joinrelids, sjinfo2->min_righthand))
 			extra.param_source_rels = bms_join(extra.param_source_rels,
 										   bms_difference(root->all_baserels,
 													 sjinfo2->min_lefthand));
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 936ee0c..7476e8e 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -14,9 +14,14 @@
  */
 #include "postgres.h"
 
+#include "miscadmin.h"
+#include "nodes/relation.h"
+#include "optimizer/clauses.h"
 #include "optimizer/joininfo.h"
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
+#include "optimizer/prep.h"
+#include "optimizer/cost.h"
 #include "utils/memutils.h"
 
 
@@ -35,6 +40,14 @@ static bool restriction_is_constant_false(List *restrictlist,
 static void populate_joinrel_with_paths(PlannerInfo *root, RelOptInfo *rel1,
 							RelOptInfo *rel2, RelOptInfo *joinrel,
 							SpecialJoinInfo *sjinfo, List *restrictlist);
+static void try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1,
+						  RelOptInfo *rel2, RelOptInfo *joinrel,
+						  SpecialJoinInfo *parent_sjinfo,
+						  List *parent_restrictlist);
+static bool have_partkey_equi_join(RelOptInfo *rel1, RelOptInfo *rel2,
+						   JoinType jointype, List *restrictlist);
+static int match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel);
+static void free_special_join_info(SpecialJoinInfo *sjinfo);
 
 
 /*
@@ -731,6 +744,9 @@ make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 	populate_joinrel_with_paths(root, rel1, rel2, joinrel, sjinfo,
 								restrictlist);
 
+	/* Apply partition-wise join technique, if possible. */
+	try_partition_wise_join(root, rel1, rel2, joinrel, sjinfo, restrictlist);
+
 	bms_free(joinrelids);
 
 	return joinrel;
@@ -1269,3 +1285,408 @@ restriction_is_constant_false(List *restrictlist, bool only_pushed_down)
 	}
 	return false;
 }
+
+/* Free memory used by SpecialJoinInfo. */
+static void
+free_special_join_info(SpecialJoinInfo *sjinfo)
+{
+	bms_free(sjinfo->min_lefthand);
+	bms_free(sjinfo->syn_lefthand);
+	bms_free(sjinfo->syn_righthand);
+	pfree(sjinfo);
+}
+
+/*
+ * Assess whether join between given two partitioned relations can be broken
+ * down into joins between matching partitions; a technique called
+ * "partition-wise join"
+ *
+ * Partition-wise 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.
+ *
+ * Partition-wise join is planned as follows (details: optimizer/README.)
+ *
+ * 1. Create the RelOptInfos for joins between matching partitions i.e
+ * child-joins and estimate sizes of those. This function is responsible for
+ * this phase.
+ *
+ * 2. Add paths representing partition-wise join. The second phase is
+ * implemented by generate_partition_wise_join_paths(). In order to save time
+ * and memory consumed in creating paths for every child-join, we create paths
+ * for only few child-joins.
+ *
+ * 3. Create merge/append plan to combining plans for every child-join,
+ * creating paths for remaining child-joins.
+ *
+ * The RelOptInfo, SpecialJoinInfo and restrictlist for each child join are
+ * obtained by translating the respective parent join structures.
+ */
+static void
+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;
+	PartitionedJoin	   *partitioned_join;
+
+	/* 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)
+		return;
+
+	/*
+	 * If any of the joining parent relations is proven empty, either the join
+	 * will be empty (INNER join) or will have the inner side all nullified. We
+	 * take care of such cases when creating join paths for parent relations.
+	 * Nothing to be done here. Also, nothing to do, if the parent join is
+	 * proven empty.
+	 */
+	if (IS_DUMMY_REL(rel1) || IS_DUMMY_REL(rel2) || IS_DUMMY_REL(joinrel))
+		return;
+
+	/*
+	 * Partitioning scheme in join relation indicates a possibility that the
+	 * join may be partitioned, but it's not necessary that every pair of
+	 * joining relations can use partition-wise join technique. If one of
+	 * joining relations turns out to be unpartitioned, this pair of joining
+	 * relations can not use partition-wise join technique.
+	 */
+	if (!rel1->part_scheme || !rel2->part_scheme)
+		return;
+
+	/*
+	 * If an equi-join condition between the partition keys of the joining
+	 * relations does not exist, this pair of joining relations can not use
+	 * partition-wise technique.
+	 */
+	if (!have_partkey_equi_join(rel1, rel2, parent_sjinfo->jointype,
+								parent_restrictlist))
+		return;
+
+	/*
+	 * 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);
+
+	/* We should have RelOptInfos of the partitions available. */
+	Assert(rel1->part_rels && rel2->part_rels);
+
+	part_scheme = joinrel->part_scheme;
+	nparts = part_scheme->nparts;
+
+	/*
+	 * We do not store information about valid pairs of joining child
+	 * relations. The pair of joining relations for a child-join can be derived
+	 * from valid pairs of joining parent relations. Amongst the valid pairs of
+	 * parent joining relations, only those which result in partitioned join
+	 * matter for partition-wise join. Remember those so that we can use them
+	 * for creating paths for few child-joins in
+	 * generate_partition_wise_join_paths() later.
+	 */
+	partitioned_join = (PartitionedJoin *) palloc(sizeof(PartitionedJoin));
+	partitioned_join->rel1 = rel1;
+	partitioned_join->rel2 = rel2;
+	partitioned_join->sjinfo = copyObject(parent_sjinfo);
+	partitioned_join->restrictlist = parent_restrictlist;
+	joinrel->partitioned_joins = lappend(joinrel->partitioned_joins,
+										 partitioned_join);
+
+	elog(DEBUG3, "join between relations %s and %s is considered for partition-wise join.",
+		 bmsToString(rel1->relids), bmsToString(rel2->relids));
+
+	/* We are done if child RelOptInfos are already created. */
+	if (joinrel->part_rels)
+		return;
+
+	/* Create all the child RelOptInfos. */
+	joinrel->part_rels = (RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
+
+	/*
+	 * Create child join relations for this partitioned join. While doing so,
+	 * we estimate sizes of these child join relations. These estimates are
+	 * used to find the representative child relations used for costing the
+	 * partition-wise join later.
+	 */
+	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	{
+		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
+		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
+		SpecialJoinInfo	*child_sjinfo;
+		List	*child_restrictlist;
+		RelOptInfo *child_joinrel;
+
+		/* We should never try to join two overlapping sets of rels. */
+		Assert(!bms_overlap(child_rel1->relids, child_rel2->relids));
+
+		Assert (!joinrel->part_rels[cnt_parts]);
+
+		child_joinrel = build_child_join_rel(root, child_rel1, child_rel2,
+											 joinrel, parent_sjinfo->jointype);
+
+		joinrel->part_rels[cnt_parts] = child_joinrel;
+
+		/*
+		 * Construct restrictions applicable to the child join from
+		 * those applicable to the parent join.
+		 */
+		child_restrictlist = build_child_clauses(root, parent_restrictlist,
+												 find_appinfos_by_relids(root,
+													   child_joinrel->relids));
+
+		/*
+		 * Construct SpecialJoinInfo from parent join relations's
+		 * SpecialJoinInfo.
+		 */
+		child_sjinfo = build_child_join_sjinfo(root, parent_sjinfo,
+											   child_rel1->relids,
+											   child_rel2->relids);
+
+		/*
+		 * Set estimates of the child-joinrel's size.
+		 */
+		set_joinrel_size_estimates(root, child_joinrel, child_rel1, child_rel2,
+								   child_sjinfo, child_restrictlist);
+
+		/*
+		 * If the child relations themselves are partitioned, try partition-wise join
+		 * recursively.
+		 */
+		try_partition_wise_join(root, child_rel1, child_rel2, child_joinrel,
+								child_sjinfo, child_restrictlist);
+
+		free_special_join_info(child_sjinfo);
+		child_sjinfo = NULL;
+	}
+}
+
+/*
+ * add_paths_to_child_join
+ * 		Add paths to 'child_id'th child of given parent join relation.
+ *
+ * The function creates paths for given child-join by joining corresponding
+ * children of every pair of joining parent relations which produces
+ * partitioned join. Since we create paths only for sampled child-joins, either
+ * of the children being joined may not have paths. In that case, this function
+ * is called recursively to populate paths for those.
+ */
+void
+add_paths_to_child_joinrel(PlannerInfo *root, RelOptInfo *parent_joinrel,
+						   int child_id)
+{
+	ListCell	*lc;
+	RelOptInfo	   *child_joinrel = parent_joinrel->part_rels[child_id];
+
+	Assert(IS_JOIN_REL(parent_joinrel));
+
+	/* If this child relation already has paths, nothing to do. */
+	if (child_joinrel->cheapest_total_path)
+		return;
+
+	/* A dummy relation will have a dummy path as the cheapest path. */
+	Assert(!is_dummy_rel(child_joinrel));
+
+	/*
+	 * For every partitioned join order, calculate paths for the joining
+	 * child relations and then calculate paths for given child.
+	 */
+	foreach (lc, parent_joinrel->partitioned_joins)
+	{
+		PartitionedJoin	   *pj = lfirst(lc);
+		RelOptInfo *rel1 = pj->rel1;
+		RelOptInfo *rel2 = pj->rel2;
+		RelOptInfo *child_rel1 = rel1->part_rels[child_id];
+		RelOptInfo *child_rel2 = rel2->part_rels[child_id];
+		SpecialJoinInfo	   *child_sjinfo;
+		List	   *child_restrictlist;
+
+		/*
+		 * Add paths to joining relation if it is a join itself.
+		 * Paths for child base relations are created in
+		 * set_append_rel_pathlist().
+		 */
+		if (IS_JOIN_REL(pj->rel1))
+			add_paths_to_child_joinrel(root, rel1, child_id);
+
+		if (IS_JOIN_REL(pj->rel2))
+			add_paths_to_child_joinrel(root, rel2, child_id);
+
+		/*
+		 * Construct SpecialJoinInfo from parent join relations's
+		 * SpecialJoinInfo.
+		 */
+		child_sjinfo = build_child_join_sjinfo(root, pj->sjinfo,
+											   child_rel1->relids,
+											   child_rel2->relids);
+
+
+		/*
+		 * Construct restrictions applicable to the child join from
+		 * those applicable to the parent join.
+		 */
+		child_restrictlist = build_child_clauses(root, pj->restrictlist,
+												 find_appinfos_by_relids(root,
+													   child_joinrel->relids));
+
+		/* Add paths for child join. */
+		populate_joinrel_with_paths(root, rel1->part_rels[child_id],
+									rel2->part_rels[child_id], child_joinrel,
+									child_sjinfo, child_restrictlist);
+
+		/* Add partition-wise join paths for partitioned child-joins. */
+		generate_partition_wise_join_paths(root, child_joinrel);
+
+		free_special_join_info(child_sjinfo);
+		child_sjinfo = NULL;
+	}
+
+	set_cheapest(child_joinrel);
+}
+
+/*
+ * Returns true if there exists an equi-join condition for each pair of
+ * partition key from given relations being joined.
+ */
+static bool
+have_partkey_equi_join(RelOptInfo *rel1, RelOptInfo *rel2,
+					 JoinType jointype, List *restrictlist)
+{
+	PartitionScheme	part_scheme = rel1->part_scheme;
+	ListCell	*lc;
+	int		cnt_pks;
+	int		num_pks;
+	bool   *pk_has_clause;
+
+	/*
+	 * This function should be called when the joining relations have same
+	 * partitioning scheme.
+	 */
+	Assert(rel1->part_scheme == rel2->part_scheme);
+	Assert(part_scheme);
+
+	num_pks = part_scheme->partnatts;
+
+	pk_has_clause = (bool *) palloc0(sizeof(bool) * num_pks);
+
+	foreach (lc, restrictlist)
+	{
+		RestrictInfo *rinfo = lfirst(lc);
+		OpExpr		 *opexpr;
+		Expr		 *expr1;
+		Expr		 *expr2;
+		int		ipk1;
+		int		ipk2;
+
+		/* If processing an outer join, only use its own join clauses. */
+		if (IS_OUTER_JOIN(jointype) && rinfo->is_pushed_down)
+			continue;
+
+		/* Skip clauses which can not be used for a join. */
+		if (!rinfo->can_join)
+			continue;
+
+		/* Skip clauses which are not equality conditions. */
+		if (rinfo->hashjoinoperator == InvalidOid && !rinfo->mergeopfamilies)
+			continue;
+
+		opexpr = (OpExpr *) rinfo->clause;
+		Assert(is_opclause(opexpr));
+
+
+		/* Match the operands to the relation. */
+		if (bms_is_subset(rinfo->left_relids, rel1->relids) &&
+			bms_is_subset(rinfo->right_relids, rel2->relids))
+		{
+			expr1 = linitial(opexpr->args);
+			expr2 = lsecond(opexpr->args);
+		}
+		else if (bms_is_subset(rinfo->left_relids, rel2->relids) &&
+				 bms_is_subset(rinfo->right_relids, rel1->relids))
+		{
+			expr1 = lsecond(opexpr->args);
+			expr2 = linitial(opexpr->args);
+		}
+		else
+			continue;
+
+		/* Associate matching clauses with partition keys. */
+		ipk1 = match_expr_to_partition_keys(expr1, rel1);
+		ipk2 = match_expr_to_partition_keys(expr2, rel2);
+
+		/*
+		 * If the clause refers to different partition keys from
+		 * both relations, it can not be used for partition-wise join.
+		 */
+		if (ipk1 != ipk2)
+			continue;
+
+		/*
+		 * The clause allows partition-wise join if only it uses the same
+		 * operator family as that specified by the partition key.
+		 */
+		if (!list_member_oid(rinfo->mergeopfamilies,
+							 part_scheme->partopfamily[ipk1]))
+			continue;
+
+		/* Mark the partition key as having an equi-join clause. */
+		pk_has_clause[ipk1] = true;
+	}
+
+	/* Check whether every partition key has an equi-join condition. */
+	for (cnt_pks = 0; cnt_pks < num_pks; cnt_pks++)
+	{
+		if (!pk_has_clause[cnt_pks])
+		{
+			pfree(pk_has_clause);
+			return false;
+		}
+	}
+
+	pfree(pk_has_clause);
+	return true;
+}
+
+/*
+ * Find the partition key from the given relation matching the given
+ * expression. If found, return the index of the partition key, else return -1.
+ */
+static int
+match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel)
+{
+	int		cnt_pks;
+	int		num_pks;
+
+	/* This function should be called only for partitioned relations. */
+	Assert(rel->part_scheme);
+
+	num_pks = rel->part_scheme->partnatts;
+
+	/*
+	 * Remove the relabel decoration. We can assume that there is at most one
+	 * RelabelType node; eval_const_expressions() simplifies multiple
+	 * RelabelType nodes into one.
+	 */
+	if (IsA(expr, RelabelType))
+		expr = (Expr *) ((RelabelType *) expr)->arg;
+
+	for (cnt_pks = 0; cnt_pks < num_pks; cnt_pks++)
+	{
+		List	 *pkexprs = rel->partexprs[cnt_pks];
+		ListCell *lc;
+
+		foreach(lc, pkexprs)
+		{
+			Expr *pkexpr = lfirst(lc);
+			if (equal(pkexpr, expr))
+				return cnt_pks;
+		}
+	}
+
+	return -1;
+}
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 997bdcf..fe6b7f8 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -42,6 +42,7 @@
 #include "parser/parse_clause.h"
 #include "parser/parsetree.h"
 #include "utils/lsyscache.h"
+#include "utils/memutils.h"
 
 
 /*
@@ -146,6 +147,8 @@ static CustomScan *create_customscan_plan(PlannerInfo *root,
 static NestLoop *create_nestloop_plan(PlannerInfo *root, NestPath *best_path);
 static MergeJoin *create_mergejoin_plan(PlannerInfo *root, MergePath *best_path);
 static HashJoin *create_hashjoin_plan(PlannerInfo *root, HashPath *best_path);
+static Plan *create_partition_join_plan(PlannerInfo *root,
+										PartitionJoinPath *best_path);
 static Node *replace_nestloop_params(PlannerInfo *root, Node *expr);
 static Node *replace_nestloop_params_mutator(Node *node, PlannerInfo *root);
 static void process_subquery_nestloop_params(PlannerInfo *root,
@@ -369,12 +372,20 @@ create_plan_recurse(PlannerInfo *root, Path *best_path, int flags)
 									(JoinPath *) best_path);
 			break;
 		case T_Append:
-			plan = create_append_plan(root,
-									  (AppendPath *) best_path);
+			if (IsA(best_path, PartitionJoinPath))
+				plan = create_partition_join_plan(root,
+											   (PartitionJoinPath *)best_path);
+			else
+				plan = create_append_plan(root,
+										  (AppendPath *) best_path);
 			break;
 		case T_MergeAppend:
-			plan = create_merge_append_plan(root,
-											(MergeAppendPath *) best_path);
+			if (IsA(best_path, PartitionJoinPath))
+				plan = create_partition_join_plan(root,
+											   (PartitionJoinPath *)best_path);
+			else
+				plan = create_merge_append_plan(root,
+												(MergeAppendPath *) best_path);
 			break;
 		case T_Result:
 			if (IsA(best_path, ProjectionPath))
@@ -3982,6 +3993,214 @@ create_hashjoin_plan(PlannerInfo *root,
 	return join_plan;
 }
 
+/*
+ * create_partition_join_plan
+ *		Creates Merge/Append plan consisting of join plans for child-join.
+ *
+ *		Returns a Plan node.
+ */
+static Plan *
+create_partition_join_plan(PlannerInfo *root, PartitionJoinPath *best_path)
+{
+	RelOptInfo *joinrel = best_path->path.parent;
+	int		nparts;
+	int		cnt_parts;
+	List	   *child_plans = NIL;
+	List	   *tlist = build_path_tlist(root, &best_path->path);
+	Plan	   *plan;
+	MemoryContext	child_context;
+	MemoryContext	old_context;
+	List	   *pathkeys = best_path->path.pathkeys;
+	StringInfoData	mem_context_name;
+
+	/* The relation should be a partitioned join relation. */
+	Assert(IS_JOIN_REL(joinrel) && joinrel->part_scheme &&
+		   joinrel->partitioned_joins);
+
+	nparts = joinrel->part_scheme->nparts;
+
+	/* Create MergeAppend plan when result is expected to be ordered. */
+	if (pathkeys)
+	{
+		MergeAppend *node = makeNode(MergeAppend);
+		plan = &node->plan;
+
+		plan->targetlist = tlist;
+
+		/* Compute sorting info, and adjust MergeAppend's tlist as needed. */
+		(void) prepare_sort_from_pathkeys(plan, pathkeys,
+										  best_path->path.parent->relids,
+										  NULL,
+										  true,
+										  &node->numCols,
+										  &node->sortColIdx,
+										  &node->sortOperators,
+										  &node->collations,
+										  &node->nullsFirst);
+	}
+	else
+	{
+		Append  *node = makeNode(Append);
+		plan = &node->plan;
+		plan->targetlist = tlist;
+	}
+
+	/* Fill costs, so that we can cost Sort node, if required. */
+	copy_generic_path_info(plan, (Path *) best_path);
+
+	/*
+	 * Create a new memory context for planning child joins. Since this routine
+	 * may be called recursively for tables with subpartitions, we use
+	 * a unique context name for every level of partition by using the lowest
+	 * relid amongst the base relations being joined.
+	 */
+	initStringInfo(&mem_context_name);
+	appendStringInfo(&mem_context_name, "%s_%d", "ChildJoinContext",
+					 bms_next_member(joinrel->relids, -1));
+	child_context = AllocSetContextCreate(CurrentMemoryContext,
+										  pstrdup(mem_context_name.data),
+										  ALLOCSET_DEFAULT_SIZES);
+	pfree(mem_context_name.data);
+	resetStringInfo(&mem_context_name);
+
+	/*
+	 * Create a paths for all child joins, one child join at a time. The paths
+	 * for every child join are independent i.e. one child does not require
+	 * paths created for the other. In order to avoid accumulating memory
+	 * consumed while creating paths for every child join, we use a fresh
+	 * memory context for every child join.
+	 */
+	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	{
+		RelOptInfo *child_join;
+		Path	   *child_path = NULL;
+		Plan	   *child_plan;
+		int			numsortkeys;
+		AttrNumber *sortColIdx;
+		Oid		   *sortOperators;
+		Oid		   *collations;
+		bool	   *nullsFirst;
+
+		/*
+		 * Create paths for the child join in a separate context, so that we
+		 * can reuse the memory used by those paths.
+		 */
+		old_context = MemoryContextSwitchTo(child_context);
+
+		add_paths_to_child_joinrel(root, joinrel, cnt_parts);
+
+		child_join = joinrel->part_rels[cnt_parts];
+
+
+		/* Skip empty child. */
+		if (IS_DUMMY_REL(child_join))
+		{
+			MemoryContextSwitchTo(old_context);
+			continue;
+		}
+
+#ifdef OPTIMIZER_DEBUG
+		debug_print_rel(root, rel);
+#endif
+
+		/*
+		 * Search for a child path with pathkeys or parameterization
+		 * matching that of the given path.
+		 */
+		child_path = get_cheapest_path_for_pathkeys(child_join->pathlist,
+													best_path->path.pathkeys,
+											 PATH_REQ_OUTER(&best_path->path),
+													TOTAL_COST);
+
+		if (!child_path)
+			elog(ERROR, "Could not find a path with required pathkeys.");
+
+		MemoryContextSwitchTo(old_context);
+
+		/* Create plan for the current child. */
+		child_plan = create_plan_recurse(root, child_path, CP_EXACT_TLIST);
+
+		if (pathkeys)
+		{
+			MergeAppend *node = (MergeAppend *) plan;
+
+			Assert(IsA(node, MergeAppend));
+
+			/* Compute sorting info, and adjust subplan's tlist as needed */
+			child_plan = prepare_sort_from_pathkeys(child_plan, pathkeys,
+												 child_path->parent->relids,
+												 node->sortColIdx,
+												 false,
+												 &numsortkeys,
+												 &sortColIdx,
+												 &sortOperators,
+												 &collations,
+												 &nullsFirst);
+
+			/*
+			 * Check that we got the same sort key information.  We just Assert
+			 * that the sortops match, since those depend only on the pathkeys;
+			 * but it seems like a good idea to check the sort column numbers
+			 * explicitly, to ensure the tlists really do match up.
+			 */
+			Assert(numsortkeys == node->numCols);
+			if (memcmp(sortColIdx, node->sortColIdx,
+					   numsortkeys * sizeof(AttrNumber)) != 0)
+				elog(ERROR, "MergeAppend child's targetlist doesn't match MergeAppend");
+			Assert(memcmp(sortOperators, node->sortOperators,
+						  numsortkeys * sizeof(Oid)) == 0);
+			Assert(memcmp(collations, node->collations,
+						  numsortkeys * sizeof(Oid)) == 0);
+			Assert(memcmp(nullsFirst, node->nullsFirst,
+						  numsortkeys * sizeof(bool)) == 0);
+
+			/* Now, insert a Sort node if subplan isn't sufficiently ordered */
+			if (!pathkeys_contained_in(pathkeys, child_path->pathkeys))
+			{
+				Sort	   *sort = make_sort(child_plan, numsortkeys,
+											 sortColIdx, sortOperators,
+											 collations, nullsFirst);
+				label_sort_with_costsize(root, sort, -1.0);
+				child_plan = (Plan *) sort;
+			}
+		}
+
+		child_plans = lappend(child_plans, child_plan);
+
+		/*
+		 * Reset the child_join memory context to reclaim the memory consumed
+		 * while creating paths.
+		 */
+		MemoryContextResetAndDeleteChildren(child_context);
+	}
+
+	/* Destroy the child context as we do not need it anymore. */
+	Assert(CurrentMemoryContext == old_context);
+	MemoryContextDelete(child_context);
+
+	/* Partitioned relation with all empty children gets a dummy path. */
+	Assert(child_plans != NIL);
+
+	if (IsA(plan, MergeAppend))
+	{
+		MergeAppend *node = (MergeAppend *)plan;
+
+		node->mergeplans = child_plans;
+	}
+	else
+	{
+		Append  *node = (Append *)plan;
+
+		Assert(IsA(plan, Append));
+		node->appendplans = child_plans;
+	}
+
+	/* Complete rest of the plan. */
+	plan->qual = NIL;
+	plan->lefttree = NULL;
+	plan->righttree = NULL;
+	return plan;
+}
 
 /*****************************************************************************
  *
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index 6f41979..676204f 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -2179,3 +2179,163 @@ adjust_appendrel_attrs_multilevel(PlannerInfo *root, Node *node,
 	/* Now translate for this child */
 	return adjust_appendrel_attrs(root, node, list_make1(appinfo));
 }
+
+/*
+ * build_child_restrictinfo
+ *		Returns a RestrictInfo which is derived from the given RestrictInfo by
+ *		applying the parent-child translation specified by the list of
+ *		AppendRelInfos.
+ *
+ * The topmost parent's RestrictInfo maintains a list of child RestrictInfos
+ * derived from it. If a suitable RestrictInfo is found in that list, it is
+ * returned as is. If there is no such child RestrictInfo, we translate the given
+ * RestrictInfo using the given list of AppendRelInfos and stick it in the
+ * topmost parent's list before returning it to the caller.
+ */
+RestrictInfo *
+build_child_restrictinfo(PlannerInfo *root, RestrictInfo *rinfo,
+						 List *append_rel_infos)
+{
+	Relids child_required_relids;
+	ListCell   *lc;
+	RestrictInfo   *parent_rinfo;
+	RestrictInfo   *child_rinfo;
+	MemoryContext	old_context;
+
+	child_required_relids = adjust_relid_set(rinfo->required_relids,
+												append_rel_infos);
+
+
+	/* Nothing to do, if the clause does not need any translation. */
+	if (bms_equal(child_required_relids, rinfo->required_relids))
+	{
+		bms_free(child_required_relids);
+		return rinfo;
+	}
+
+	/*
+	 * Check if we already have the RestrictInfo for the given child in the
+	 * topmost parent's RestrictInfo.
+	 */
+	parent_rinfo = rinfo->parent_rinfo ? rinfo->parent_rinfo : rinfo;
+	foreach (lc, parent_rinfo->child_rinfos)
+	{
+		child_rinfo = lfirst(lc);
+
+		if (bms_equal(child_rinfo->required_relids, child_required_relids))
+		{
+			bms_free(child_required_relids);
+			return child_rinfo;
+		}
+	}
+
+	/*
+	 * We didn't find any child restrictinfo for the given child, translate the
+	 * given RestrictInfo and stick it into the parent's list. The clause
+	 * expression may get used in plan, so create the child RestrictInfo in the
+	 * planner's context.
+	 */
+	old_context = MemoryContextSwitchTo(root->planner_cxt);
+	child_rinfo = (RestrictInfo *) adjust_appendrel_attrs(root, (Node *) rinfo,
+														  append_rel_infos);
+	bms_free(child_required_relids);
+	parent_rinfo->child_rinfos = lappend(parent_rinfo->child_rinfos,
+										 child_rinfo);
+	child_rinfo->parent_rinfo = parent_rinfo;
+
+	MemoryContextSwitchTo(old_context);
+
+	return child_rinfo;
+}
+
+/*
+ * build_child_clauses
+ *		Convenience routine to call build_child_restrictinfo on a list of
+ *		clauses.
+ */
+List *
+build_child_clauses(PlannerInfo *root, List *clauses, List *append_rel_infos)
+{
+	List   *child_clauses = NIL;
+	ListCell   *lc;
+
+	foreach (lc, clauses)
+	{
+		RestrictInfo   *parent_rinfo = lfirst(lc);
+		RestrictInfo   *child_rinfo;
+
+		Assert(IsA(parent_rinfo, RestrictInfo));
+
+		child_rinfo = build_child_restrictinfo(root, parent_rinfo,
+											   append_rel_infos);
+
+		child_clauses = lappend(child_clauses, child_rinfo);
+	}
+
+	return child_clauses;
+}
+
+/*
+ * find_appinfos_by_relids
+ * 		Find AppendRelInfo structures for all relations specified by relids.
+ */
+List *
+find_appinfos_by_relids(PlannerInfo *root, Relids relids)
+{
+	ListCell	*lc;
+	List		*appinfo_list = NIL;
+
+	foreach (lc, root->append_rel_list)
+	{
+		AppendRelInfo *appinfo = lfirst(lc);
+
+		if (bms_is_member(appinfo->child_relid, relids))
+			appinfo_list = lappend(appinfo_list, appinfo);
+	}
+
+	Assert(list_length(appinfo_list) == bms_num_members(relids));
+	return appinfo_list;
+}
+
+/*
+ * Construct the SpecialJoinInfo for a child-join by translating
+ * SpecialJoinInfo for the join between parents. left_relids and right_relids
+ * are the relids of left and right side of the join respectively.
+ */
+SpecialJoinInfo *
+build_child_join_sjinfo(PlannerInfo *root, SpecialJoinInfo *parent_sjinfo,
+						Relids left_relids, Relids right_relids)
+{
+	SpecialJoinInfo *sjinfo = makeNode(SpecialJoinInfo);
+	MemoryContext	old_context;
+	List	   *left_appinfos = find_appinfos_by_relids(root, left_relids);
+	List	   *right_appinfos = find_appinfos_by_relids(root, right_relids);
+
+	memcpy(sjinfo, parent_sjinfo, sizeof(SpecialJoinInfo));
+
+	sjinfo->min_lefthand = adjust_relid_set(sjinfo->min_lefthand,
+											   left_appinfos);
+	sjinfo->min_righthand = adjust_relid_set(sjinfo->min_righthand,
+												right_appinfos);
+	sjinfo->syn_lefthand = adjust_relid_set(sjinfo->syn_lefthand,
+											   left_appinfos);
+	sjinfo->syn_righthand = adjust_relid_set(sjinfo->syn_righthand,
+												right_appinfos);
+
+	/*
+	 * Replace the Var nodes of parent with those of children in expressions.
+	 * This function may be called within a temporary context, but the
+	 * expressions will be shallow-copied into the plan. Hence copy those in
+	 * the planner's context.
+	 */
+	old_context = MemoryContextSwitchTo(root->planner_cxt);
+	sjinfo->semi_rhs_exprs = (List *) adjust_appendrel_attrs(root,
+											   (Node *) sjinfo->semi_rhs_exprs,
+															 right_appinfos);
+	MemoryContextSwitchTo(old_context);
+
+	list_free(left_appinfos);
+	list_free(right_appinfos);
+
+	return sjinfo;
+}
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index f440875..d861a49 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -23,7 +23,9 @@
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
 #include "optimizer/planmain.h"
+#include "optimizer/prep.h"
 #include "optimizer/restrictinfo.h"
+#include "optimizer/tlist.h"
 #include "optimizer/var.h"
 #include "parser/parsetree.h"
 #include "utils/lsyscache.h"
@@ -2154,6 +2156,117 @@ create_hashjoin_path(PlannerInfo *root,
 }
 
 /*
+ * create_partition_join_path
+ * 		Creates a pathnode that represents partition-wise join for given
+ * 		partitioned join relation.
+ *
+ * This function is called when we haven't created paths for all the child
+ * joins. It estimates the number of rows and cost of the PartitionJoinPath
+ * based upon the number of rows and the cost of representative child-joins
+ * paths.
+ */
+PartitionJoinPath *
+create_partition_join_path(PlannerInfo *root, RelOptInfo *rel, List *subpaths,
+						   Bitmapset *required_outer, List *pathkeys)
+{
+	PartitionJoinPath *pathnode = makeNode(PartitionJoinPath);
+	double	subpath_rows = 0;
+	double	subpath_startup_cost = 0;
+	double	subpath_total_cost = 0;
+	double	child_rel_rows = 0;
+	ListCell   *lc;
+
+	Assert(rel->part_scheme);
+
+	pathnode->path.pathtype = pathkeys ? T_MergeAppend : T_Append;
+	pathnode->path.parent = rel;
+	pathnode->path.pathtarget = rel->reltarget;
+	pathnode->path.param_info = get_appendrel_parampathinfo(rel,
+															required_outer);
+	pathnode->path.pathkeys = pathkeys;
+
+	/* No parallel paths here. See more details in add_paths_to_append_rel() */
+	pathnode->path.parallel_aware = false;
+	pathnode->path.parallel_safe = false;
+	pathnode->path.parallel_workers = 0;
+
+	/* Accumulate the number of rows and costs from the given subpaths. */
+	foreach (lc, subpaths)
+	{
+		Path   *subpath = lfirst(lc);
+
+		if (!pathkeys)
+		{
+			/*
+			 * Startup cost of an append relation is the startup cost of the
+			 * first subpath. Assume that the given first child will be the
+			 * first child in the final plan as well.
+			 */
+			if (lc == list_head(subpaths))
+				subpath_startup_cost = subpath->startup_cost;
+			subpath_total_cost += subpath->total_cost;
+		}
+		else if (pathkeys_contained_in(pathkeys, subpath->pathkeys))
+		{
+			/*
+			 * Subpath is adequately ordered, we won't need to sort it. We need
+			 * all the subplans to return their respective first rows, before
+			 * returning a row. So add the startup costs.
+			 */
+			subpath_startup_cost += subpath->startup_cost;
+			subpath_total_cost += subpath->total_cost;
+		}
+		else
+		{
+			/* We'll need to insert a Sort node, so include cost for that */
+			Path		sort_path;		/* dummy for result of cost_sort */
+
+			cost_sort(&sort_path,
+					  root,
+					  pathkeys,
+					  subpath->total_cost,
+					  subpath->parent->tuples,
+					  subpath->pathtarget->width,
+					  0.0,
+					  work_mem,
+					  -1);
+			subpath_startup_cost += sort_path.startup_cost;
+			subpath_total_cost += sort_path.total_cost;
+		}
+
+		subpath_rows += subpath->rows;
+		child_rel_rows += subpath->parent->rows;
+
+	}
+
+	/*
+	 * For a parameterized path, extrapolate the number of rows for the append
+	 * relation by considering the average selectivity of the parameterization
+	 * across the given children.
+	 */
+	if (bms_is_empty(required_outer))
+		pathnode->path.rows = rel->rows;
+	else
+		pathnode->path.rows = rel->rows * (subpath_rows / child_rel_rows);
+
+	/* Extrapolate the total cost to account for yet-to-be planned children. */
+	if (!pathkeys)
+		pathnode->path.startup_cost = subpath_startup_cost;
+	else
+		pathnode->path.startup_cost = (subpath_startup_cost * pathnode->path.rows) / subpath_rows;
+	pathnode->path.total_cost = (subpath_total_cost * pathnode->path.rows) / subpath_rows;
+
+	/*
+	 * Multiply the costs with scaling factor as specified. Used to encourage
+	 * or discourage use of partition-wise join plans.
+	 */
+	pathnode->path.startup_cost *= partition_wise_plan_weight;
+	pathnode->path.total_cost *= partition_wise_plan_weight;
+
+	return pathnode;
+}
+
+/*
  * create_projection_path
  *	  Creates a pathnode that represents performing a projection.
  *
diff --git a/src/backend/optimizer/util/placeholder.c b/src/backend/optimizer/util/placeholder.c
index 698a387..e06bccc 100644
--- a/src/backend/optimizer/util/placeholder.c
+++ b/src/backend/optimizer/util/placeholder.c
@@ -20,6 +20,7 @@
 #include "optimizer/pathnode.h"
 #include "optimizer/placeholder.h"
 #include "optimizer/planmain.h"
+#include "optimizer/prep.h"
 #include "optimizer/var.h"
 #include "utils/lsyscache.h"
 
@@ -414,6 +415,10 @@ add_placeholders_to_joinrel(PlannerInfo *root, RelOptInfo *joinrel,
 	Relids		relids = joinrel->relids;
 	ListCell   *lc;
 
+	/* This function is called only on the parent relations. */
+	Assert(!IS_OTHER_REL(joinrel) && !IS_OTHER_REL(outer_rel) &&
+		   !IS_OTHER_REL(inner_rel));
+
 	foreach(lc, root->placeholder_list)
 	{
 		PlaceHolderInfo *phinfo = (PlaceHolderInfo *) lfirst(lc);
@@ -459,3 +464,53 @@ add_placeholders_to_joinrel(PlannerInfo *root, RelOptInfo *joinrel,
 		}
 	}
 }
+
+/*
+ * add_placeholders_to_child_joinrel
+ *		Translate the PHVs in parent's targetlist and add them to the child's
+ *		targetlist. Also adjust the cost
+ */
+void
+add_placeholders_to_child_joinrel(PlannerInfo *root, RelOptInfo *childrel,
+								  RelOptInfo *parentrel)
+{
+	ListCell  *lc;
+
+	/* This function is called only for join relations. */
+	Assert(IS_JOIN_REL(childrel) && IS_JOIN_REL(parentrel));
+
+	/* Ensure child relations is really what it claims to be. */
+	Assert(IS_OTHER_REL(childrel));
+
+	foreach (lc, parentrel->reltarget->exprs)
+	{
+		PlaceHolderVar *phv = lfirst(lc);
+
+		if (IsA(phv, PlaceHolderVar))
+		{
+			/*
+			 * In case the placeholder Var refers to any of the parent
+			 * relations, translate it to refer to the corresponding child.
+			 */
+			if (bms_overlap(phv->phrels, parentrel->relids) &&
+				childrel->reloptkind == RELOPT_OTHER_JOINREL)
+			{
+				List *append_rel_infos;
+
+				append_rel_infos = find_appinfos_by_relids(root,
+														  childrel->relids);
+				phv = (PlaceHolderVar *) adjust_appendrel_attrs(root,
+															  (Node *) phv,
+														 append_rel_infos);
+			}
+
+			childrel->reltarget->exprs = lappend(childrel->reltarget->exprs,
+												 phv);
+		}
+	}
+
+	/* Adjust the cost and width of child targetlist. */
+	childrel->reltarget->cost.startup = parentrel->reltarget->cost.startup;
+	childrel->reltarget->cost.per_tuple = parentrel->reltarget->cost.per_tuple;
+	childrel->reltarget->width = parentrel->reltarget->width;
+}
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 19982dc..1eed987 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -23,6 +23,7 @@
 #include "optimizer/paths.h"
 #include "optimizer/placeholder.h"
 #include "optimizer/plancat.h"
+#include "optimizer/prep.h"
 #include "optimizer/restrictinfo.h"
 #include "optimizer/tlist.h"
 #include "utils/hsearch.h"
@@ -54,6 +55,9 @@ static void set_foreign_rel_properties(RelOptInfo *joinrel,
 static void add_join_rel(PlannerInfo *root, RelOptInfo *joinrel);
 extern ParamPathInfo *find_param_path_info(RelOptInfo *rel,
 									  Relids required_outer);
+static void build_joinrel_partition_info(RelOptInfo *joinrel,
+						  RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+						  JoinType jointype);
 
 
 /*
@@ -434,6 +438,9 @@ build_join_rel(PlannerInfo *root,
 	RelOptInfo *joinrel;
 	List	   *restrictlist;
 
+	/* This function should be used only for join between parents. */
+	Assert(!IS_OTHER_REL(outer_rel) && !IS_OTHER_REL(inner_rel));
+
 	/*
 	 * See if we already have a joinrel for this set of base rels.
 	 */
@@ -532,6 +539,10 @@ build_join_rel(PlannerInfo *root,
 	if (bms_is_empty(joinrel->direct_lateral_relids))
 		joinrel->direct_lateral_relids = NULL;
 
+	/* Store the partition information. */
+	build_joinrel_partition_info(joinrel, outer_rel, inner_rel,
+								 sjinfo->jointype);
+
 	/*
 	 * Construct restrict and join clause lists for the new joinrel. (The
 	 * caller might or might not need the restrictlist, but I need it anyway
@@ -594,6 +605,126 @@ build_join_rel(PlannerInfo *root,
 	return joinrel;
 }
 
+ /*
+ * build_child_join_rel
+ *		Builds RelOptInfo for joining given two child relations from RelOptInfo
+ *		representing the join between their parents.
+ *
+ * 'outer_rel' and 'inner_rel' are the RelOptInfos of child relations being
+ *		joined.
+ * 'parent_joinrel' is the RelOptInfo representing the join between parent
+ *		relations. Most of the members of new RelOptInfo are produced by
+ *		translating corresponding members of this RelOptInfo.
+ * 'sjinfo': context info for child join
+ * 'restrictlist': list of RestrictInfo nodes that apply to this particular
+ *		pair of joinable relations.
+ * 'join_appinfos': list of AppendRelInfo nodes for base child relations involved
+ *		in this join.
+ */
+RelOptInfo *
+build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
+						 RelOptInfo *inner_rel, RelOptInfo *parent_joinrel,
+						 JoinType jointype)
+{
+	RelOptInfo *joinrel = makeNode(RelOptInfo);
+
+	/* Only joins between other relations land here. */
+	Assert(IS_OTHER_REL(outer_rel) && IS_OTHER_REL(inner_rel));
+
+	joinrel->reloptkind = RELOPT_OTHER_JOINREL;
+	joinrel->relids = bms_union(outer_rel->relids, inner_rel->relids);
+	joinrel->rows = 0;
+	/* cheap startup cost is interesting iff not all tuples to be retrieved */
+	joinrel->consider_startup = (root->tuple_fraction > 0);
+	joinrel->consider_param_startup = false;
+	joinrel->consider_parallel = false;
+	joinrel->reltarget = create_empty_pathtarget();
+	joinrel->pathlist = NIL;
+	joinrel->ppilist = NIL;
+	joinrel->partial_pathlist = NIL;
+	joinrel->cheapest_startup_path = NULL;
+	joinrel->cheapest_total_path = NULL;
+	joinrel->cheapest_unique_path = NULL;
+	joinrel->cheapest_parameterized_paths = NIL;
+	joinrel->direct_lateral_relids = NULL;
+	joinrel->lateral_relids = NULL;
+	joinrel->relid = 0;			/* indicates not a baserel */
+	joinrel->rtekind = RTE_JOIN;
+	joinrel->min_attr = 0;
+	joinrel->max_attr = 0;
+	joinrel->attr_needed = NULL;
+	joinrel->attr_widths = NULL;
+	joinrel->lateral_vars = NIL;
+	joinrel->lateral_referencers = NULL;
+	joinrel->indexlist = NIL;
+	joinrel->pages = 0;
+	joinrel->tuples = 0;
+	joinrel->allvisfrac = 0;
+	joinrel->subroot = NULL;
+	joinrel->subplan_params = NIL;
+	joinrel->serverid = InvalidOid;
+	joinrel->userid = InvalidOid;
+	joinrel->useridiscurrent = false;
+	joinrel->fdwroutine = NULL;
+	joinrel->fdw_private = NULL;
+	joinrel->baserestrictinfo = NIL;
+	joinrel->baserestrictcost.startup = 0;
+	joinrel->baserestrictcost.per_tuple = 0;
+	joinrel->joininfo = NIL;
+	joinrel->has_eclass_joins = false;
+	joinrel->part_scheme = NULL;
+	joinrel->partexprs = NULL;
+	joinrel->top_parent_relids = NULL;
+	joinrel->part_rels = NULL;
+
+	joinrel->top_parent_relids = bms_union(outer_rel->top_parent_relids,
+										   inner_rel->top_parent_relids);
+
+	/* Compute information relevant to foreign relations. */
+	set_foreign_rel_properties(joinrel, outer_rel, inner_rel);
+
+	/* Build targetlist */
+	build_joinrel_tlist(root, joinrel, outer_rel);
+	build_joinrel_tlist(root, joinrel, inner_rel);
+	/* Add placeholder variables. */
+	add_placeholders_to_child_joinrel(root, joinrel, parent_joinrel);
+
+	/* Construct joininfo list. */
+	joinrel->joininfo = build_child_clauses(root, parent_joinrel->joininfo,
+											find_appinfos_by_relids(root,
+															 joinrel->relids));
+
+	/*
+	 * Lateral relids referred in child join will be same as that referred in
+	 * the parent relation. Throw any partial result computed while building
+	 * the targetlist.
+	 */
+	bms_free(joinrel->direct_lateral_relids);
+	bms_free(joinrel->lateral_relids);
+	joinrel->direct_lateral_relids = (Relids) bms_copy(parent_joinrel->direct_lateral_relids);
+	joinrel->lateral_relids = (Relids) bms_copy(parent_joinrel->lateral_relids);
+
+	/*
+	 * If the parent joinrel has pending equivalence classes, so does the
+	 * child.
+	 */
+	joinrel->has_eclass_joins = parent_joinrel->has_eclass_joins;
+
+	/* Is the join between partitions itself partitioned? */
+	build_joinrel_partition_info(joinrel, outer_rel, inner_rel, jointype);
+
+	/* Child joinrel is parallel safe if parent is parallel safe. */
+	joinrel->consider_parallel = parent_joinrel->consider_parallel;
+
+	/* We build the join only once. */
+	Assert(!find_join_rel(root, joinrel->relids));
+
+	/* Add the relation to the PlannerInfo. */
+	add_join_rel(root, joinrel);
+
+	return joinrel;
+}
+
 /*
  * min_join_parameterization
  *
@@ -649,9 +780,15 @@ static void
 build_joinrel_tlist(PlannerInfo *root, RelOptInfo *joinrel,
 					RelOptInfo *input_rel)
 {
-	Relids		relids = joinrel->relids;
+	Relids		relids;
 	ListCell   *vars;
 
+	/* attrs_needed refers to parent relids and not those of a child. */
+	if (joinrel->top_parent_relids)
+		relids = joinrel->top_parent_relids;
+	else
+		relids = joinrel->relids;
+
 	foreach(vars, input_rel->reltarget->exprs)
 	{
 		Var		   *var = (Var *) lfirst(vars);
@@ -667,23 +804,47 @@ build_joinrel_tlist(PlannerInfo *root, RelOptInfo *joinrel,
 
 		/*
 		 * Otherwise, anything in a baserel or joinrel targetlist ought to be
-		 * a Var.  (More general cases can only appear in appendrel child
-		 * rels, which will never be seen here.)
+		 * a Var or ConvertRowtypeExpr introduced while translating parent
+		 * targetlist to that of the child.
 		 */
-		if (!IsA(var, Var))
+		if (IsA(var, Var))
+		{
+			/* Get the Var's original base rel */
+			baserel = find_base_rel(root, var->varno);
+
+			/* Is it still needed above this joinrel? */
+			ndx = var->varattno - baserel->min_attr;
+		}
+		else if (IsA(var, ConvertRowtypeExpr))
+		{
+			ConvertRowtypeExpr *child_expr = (ConvertRowtypeExpr *) var;
+			Var	 *childvar = (Var *) child_expr->arg;
+
+			/*
+			 * Child's whole-row references are converted to that of parent
+			 * using ConvertRowtypeExpr. In this case, the argument to
+			 * ConvertRowtypeExpr is expected to be a whole-row reference of
+			 * the child.
+			 */
+			Assert(IsA(childvar, Var) && childvar->varattno == 0);
+
+			baserel = find_base_rel(root, childvar->varno);
+			ndx = 0 - baserel->min_attr;
+		}
+		else
 			elog(ERROR, "unexpected node type in rel targetlist: %d",
 				 (int) nodeTag(var));
 
-		/* Get the Var's original base rel */
-		baserel = find_base_rel(root, var->varno);
-
-		/* 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 */
 			joinrel->reltarget->exprs = lappend(joinrel->reltarget->exprs, var);
-			/* Vars have cost zero, so no need to adjust reltarget->cost */
+
+			/*
+			 * Vars have cost zero, so no need to adjust reltarget->cost. Even
+			 * if, it's a ConvertRowtypeExpr, it will be computed only for the
+			 * base relation, costing nothing for a join.
+			 */
 			joinrel->reltarget->width += baserel->attr_widths[ndx];
 		}
 	}
@@ -820,6 +981,9 @@ subbuild_joinrel_joinlist(RelOptInfo *joinrel,
 {
 	ListCell   *l;
 
+	/* Expected to be called only for join between parent relations. */
+	Assert(joinrel->reloptkind == RELOPT_JOINREL);
+
 	foreach(l, joininfo_list)
 	{
 		RestrictInfo *rinfo = (RestrictInfo *) lfirst(l);
@@ -1366,3 +1530,85 @@ find_param_path_info(RelOptInfo *rel, Relids required_outer)
 
 	return NULL;
 }
+
+/*
+ * build_joinrel_partition_info
+ *		If the join between given partitioned relations is possibly partitioned
+ *		set the partitioning scheme and partition keys expressions for the
+ *		join.
+ *
+ * If the two relations have same partitioning scheme, their join may be
+ * partitioned and will follow the same partitioning scheme as the joining
+ * relations.
+ */
+static void
+build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
+							 RelOptInfo *inner_rel, JoinType jointype)
+{
+	int		num_pks;
+	int		cnt;
+
+	/* Nothing to do if partition-wise join technique is disabled. */
+	if (!enable_partition_wise_join)
+	{
+		joinrel->part_scheme = NULL;
+		return;
+	}
+
+	/*
+	 * The join is not partitioned, if any of the relations being joined are
+	 * not partitioned or they do not have same partitioning scheme.
+	 */
+	if (!outer_rel->part_scheme || !inner_rel->part_scheme ||
+		outer_rel->part_scheme != inner_rel->part_scheme)
+	{
+		joinrel->part_scheme = NULL;
+		return;
+	}
+
+	/*
+	 * This function will be called only once for each joinrel, hence it should
+	 * not have partition scheme, partition key expressions and array for
+	 * storing child relations set.
+	 */
+	Assert(!joinrel->part_scheme && !joinrel->partexprs &&
+		   !joinrel->part_rels);
+
+	/*
+	 * Join relation is partitioned using same partitioning scheme as the
+	 * joining relations.
+	 */
+	joinrel->part_scheme = outer_rel->part_scheme;
+	num_pks = joinrel->part_scheme->partnatts;
+
+	/*
+	 * Construct partition keys for the join.
+	 *
+	 * An INNER join between two partitioned relations is partition by key
+	 * expressions from both the relations. For tables A and B partitioned by a and b
+	 * respectively, (A INNER JOIN B ON A.a = B.b) is partitioned by both A.a
+	 * and B.b.
+	 *
+	 * An OUTER join like (A LEFT JOIN B ON A.a = B.b) may produce rows with
+	 * B.b NULL. These rows may not fit the partitioning conditions imposed on
+	 * B.b. Hence, strictly speaking, the join is not partitioned by B.b.
+	 * Strictly speaking, partition keys of an OUTER join should include
+	 * partition key expressions from the OUTER side only. Consider a join like
+	 * (A LEFT JOIN B on (A.a = B.b) LEFT JOIN C ON B.b = C.c. If we do not
+	 * include B.b as partition key expression for (AB), it prohibits us from
+	 * using partition-wise join when joining (AB) with C as there is no
+	 * equi-join between partition keys of joining relations. But two NULL
+	 * values are never equal and no two rows from mis-matching partitions can
+	 * join. Hence it's safe to include B.b as partition key expression for
+	 * (AB), even though rows in (AB) are not strictly partitioned by B.b.
+	 */
+	joinrel->partexprs = (List **) palloc0(sizeof(List *) * num_pks);
+	for (cnt = 0; cnt < num_pks; cnt++)
+	{
+		List *pkexpr = list_copy(outer_rel->partexprs[cnt]);
+
+		pkexpr = list_concat(pkexpr,
+							 list_copy(inner_rel->partexprs[cnt]));
+		joinrel->partexprs[cnt] = pkexpr;
+	}
+}
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index de85eca..afd0c23 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -901,6 +901,15 @@ static struct config_bool ConfigureNamesBool[] =
 		true,
 		NULL, NULL, NULL
 	},
+	{
+		{"enable_partition_wise_join", PGC_USERSET, QUERY_TUNING_METHOD,
+			gettext_noop("Enables partition-wise join."),
+			NULL
+		},
+		&enable_partition_wise_join,
+		true,
+		NULL, NULL, NULL
+	},
 
 	{
 		{"geqo", PGC_USERSET, QUERY_TUNING_GEQO,
@@ -2947,6 +2956,25 @@ static struct config_real ConfigureNamesReal[] =
 	},
 
 	{
+		{"partition_wise_plan_weight", PGC_USERSET, QUERY_TUNING_OTHER,
+			gettext_noop("Multiplication factor for partition-wise plan costs."),
+			NULL
+		},
+		&partition_wise_plan_weight,
+		DEFAULT_PARTITION_WISE_PLAN_WEIGHT, 0, DBL_MAX,
+		NULL, NULL, NULL
+	},
+	{
+		{"sample_partition_fraction", PGC_USERSET, QUERY_TUNING_OTHER,
+			gettext_noop("Fraction of partitions to be used as sample for calculating total cost of partition-wise plans."),
+			NULL
+		},
+		&sample_partition_fraction,
+		DEFAULT_SAMPLE_PARTITION_FRACTION, 0, 1,
+		NULL, NULL, NULL
+	},
+
+	{
 		{"bgwriter_lru_multiplier", PGC_SIGHUP, RESOURCES_BGWRITER,
 			gettext_noop("Multiple of the average buffer usage to free per round."),
 			NULL
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index 95dd8ba..292d9a6 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -240,6 +240,7 @@ typedef enum NodeTag
 	T_NestPath,
 	T_MergePath,
 	T_HashPath,
+	T_PartitionJoinPath,
 	T_AppendPath,
 	T_MergeAppendPath,
 	T_ResultPath,
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 4f99184..146d53b 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -391,6 +391,11 @@ typedef struct PartitionSchemeData *PartitionScheme;
  * handling join alias Vars.  Currently this is not needed because all join
  * alias Vars are expanded to non-aliased form during preprocess_expression.
  *
+ * We also have relations representing joins between child relations of
+ * different partitioned tables. These relations are not added to
+ * join_rel_level lists as they are not joined directly by the dynamic
+ * programming algorithm.
+ *
  * There is also a RelOptKind for "upper" relations, which are RelOptInfos
  * that describe post-scan/join processing steps, such as aggregation.
  * Many of the fields in these RelOptInfos are meaningless, but their Path
@@ -512,10 +517,19 @@ typedef enum RelOptKind
 	RELOPT_BASEREL,
 	RELOPT_JOINREL,
 	RELOPT_OTHER_MEMBER_REL,
+	RELOPT_OTHER_JOINREL,
 	RELOPT_UPPER_REL,
 	RELOPT_DEADREL
 } RelOptKind;
 
+#define IS_OTHER_REL(rel) \
+	((rel)->reloptkind == RELOPT_OTHER_MEMBER_REL || \
+	 (rel)->reloptkind == RELOPT_OTHER_JOINREL)
+
+#define IS_JOIN_REL(rel)	\
+	((rel)->reloptkind == RELOPT_JOINREL || \
+	 (rel)->reloptkind == RELOPT_OTHER_JOINREL)
+
 typedef struct RelOptInfo
 {
 	NodeTag		type;
@@ -600,6 +614,14 @@ typedef struct RelOptInfo
 										 * as the number of joining
 										 * relations.
 										 */
+
+	/* For joins between partitioned tables. */
+	List			   *partitioned_joins;	/* List of join orders which yield
+											 * relations partitioned by above
+											 * partition scheme.
+											 */
+	/* Set only for "other" base or "other" join relations. */
+	Relids		top_parent_relids;		/* Relids of topmost parents. */
 } RelOptInfo;
 
 /*
@@ -1540,6 +1562,14 @@ typedef struct LimitPath
 	Node	   *limitCount;		/* COUNT parameter, or NULL if none */
 } LimitPath;
 
+/*
+ * PartitionJoinPath represents partition-wise join between two partitioned
+ * tables.
+ */
+typedef struct PartitionJoinPath
+{
+	Path		path;
+} PartitionJoinPath;
 
 /*
  * Restriction clause info.
@@ -1747,6 +1777,18 @@ typedef struct RestrictInfo
 	/* cache space for hashclause processing; -1 if not yet set */
 	Selectivity left_bucketsize;	/* avg bucketsize of left side */
 	Selectivity right_bucketsize;		/* avg bucketsize of right side */
+
+	/*
+	 * Repository to locate child RestrictInfos derived from parent
+	 * RestrictInfo. Every derived child RestrictInfo points to the parent
+	 * RestrictInfo from which it is derived. Parent RestrictInfo maintains a
+	 * list of all derived child RestrictInfos. So only one of the following
+	 * should be set.
+	 */
+	List	   *child_rinfos;		/* RestrictInfos derived for children. */
+	struct RestrictInfo *parent_rinfo;		/* Parent restrictinfo this
+											 * RestrictInf is derived from.
+											 */
 } RestrictInfo;
 
 /*
@@ -1869,6 +1911,21 @@ typedef struct SpecialJoinInfo
 } SpecialJoinInfo;
 
 /*
+ * Partitioned join information
+ *
+ * Saves information about relations which can be joined partition-wise and
+ * thus produce result which is partitioned by the partition scheme of the
+ * relation.
+ */
+typedef struct PartitionedJoin
+{
+	RelOptInfo	   *rel1;
+	RelOptInfo	   *rel2;
+	SpecialJoinInfo	   *sjinfo;	/* SpecialJoinInfo applicable. */
+	List		   *restrictlist;	/* applicable join clauses. */
+} PartitionedJoin;
+
+/*
  * Append-relation info.
  *
  * When we expand an inheritable table or a UNION-ALL subselect into an
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 0e68264..a13eff1 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -30,6 +30,8 @@
 #define DEFAULT_PARALLEL_SETUP_COST  1000.0
 
 #define DEFAULT_EFFECTIVE_CACHE_SIZE  524288	/* measured in pages */
+#define DEFAULT_PARTITION_WISE_PLAN_WEIGHT 1
+#define DEFAULT_SAMPLE_PARTITION_FRACTION 0.01
 
 typedef enum
 {
@@ -66,7 +68,10 @@ extern bool enable_nestloop;
 extern bool enable_material;
 extern bool enable_mergejoin;
 extern bool enable_hashjoin;
+extern bool enable_partition_wise_join;
 extern int	constraint_exclusion;
+extern double partition_wise_plan_weight;
+extern double sample_partition_fraction;
 
 extern double clamp_row_est(double nrows);
 extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 7b41317..81d637a 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -229,6 +229,9 @@ extern LimitPath *create_limit_path(PlannerInfo *root, RelOptInfo *rel,
 				  Path *subpath,
 				  Node *limitOffset, Node *limitCount,
 				  int64 offset_est, int64 count_est);
+extern PartitionJoinPath *create_partition_join_path(PlannerInfo *root,
+								RelOptInfo *rel, List *subpaths,
+								Bitmapset *required_outer, List *pathkeys);
 
 extern Path *reparameterize_path(PlannerInfo *root, Path *path,
 					Relids required_outer,
@@ -271,5 +274,8 @@ extern ParamPathInfo *get_joinrel_parampathinfo(PlannerInfo *root,
 						  List **restrict_clauses);
 extern ParamPathInfo *get_appendrel_parampathinfo(RelOptInfo *appendrel,
 							Relids required_outer);
+extern RelOptInfo *build_child_join_rel(PlannerInfo *root,
+						 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+						 RelOptInfo *parent_joinrel, JoinType jointype);
 
 #endif   /* PATHNODE_H */
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 81a9be7..7ad19be 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -53,6 +53,8 @@ extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 					 List *initial_rels);
 
 extern void generate_gather_paths(PlannerInfo *root, RelOptInfo *rel);
+extern void generate_partition_wise_join_paths(PlannerInfo *root,
+											   RelOptInfo *rel);
 
 #ifdef OPTIMIZER_DEBUG
 extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
@@ -106,6 +108,9 @@ extern bool have_join_order_restriction(PlannerInfo *root,
 							RelOptInfo *rel1, RelOptInfo *rel2);
 extern bool have_dangerous_phv(PlannerInfo *root,
 				   Relids outer_relids, Relids inner_params);
+extern void add_paths_to_child_joinrel(PlannerInfo *root,
+									   RelOptInfo *parent_joinrel,
+									   int child_id);
 
 /*
  * equivclass.c
diff --git a/src/include/optimizer/placeholder.h b/src/include/optimizer/placeholder.h
index 11e6403..8598268 100644
--- a/src/include/optimizer/placeholder.h
+++ b/src/include/optimizer/placeholder.h
@@ -28,5 +28,7 @@ extern void fix_placeholder_input_needed_levels(PlannerInfo *root);
 extern void add_placeholders_to_base_rels(PlannerInfo *root);
 extern void add_placeholders_to_joinrel(PlannerInfo *root, RelOptInfo *joinrel,
 							RelOptInfo *outer_rel, RelOptInfo *inner_rel);
+extern void add_placeholders_to_child_joinrel(PlannerInfo *root,
+							RelOptInfo *childrel, RelOptInfo *parentrel);
 
 #endif   /* PLACEHOLDER_H */
diff --git a/src/include/optimizer/prep.h b/src/include/optimizer/prep.h
index a02e06a..5832130 100644
--- a/src/include/optimizer/prep.h
+++ b/src/include/optimizer/prep.h
@@ -57,5 +57,13 @@ extern Node *adjust_appendrel_attrs(PlannerInfo *root, Node *node,
 
 extern Node *adjust_appendrel_attrs_multilevel(PlannerInfo *root, Node *node,
 								  RelOptInfo *child_rel);
+extern RestrictInfo *build_child_restrictinfo(PlannerInfo *root,
+								 RestrictInfo *rinfo, List *append_rel_infos);
+extern List *build_child_clauses(PlannerInfo *root, List *clauses,
+								 List *append_rel_infos);
+extern List *find_appinfos_by_relids(PlannerInfo *root, Relids relids);
+extern SpecialJoinInfo *build_child_join_sjinfo(PlannerInfo *root,
+									SpecialJoinInfo *parent_sjinfo,
+									Relids left_relids, Relids right_relids);
 
 #endif   /* PREP_H */
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index 18238fa..79779d6 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -2,6 +2,10 @@
 -- PARTITION_JOIN
 -- Test partition-wise join between partitioned tables
 --
+-- Usually partition-wise join paths are chosen when data is large, which would
+-- take regression tests to run longer. So, weigh partition-wise joins cheaper
+-- to force those even for smaller data.
+SET partition_wise_plan_weight to 0.2;
 --
 -- partitioned by a single column
 --
diff --git a/src/test/regress/expected/sysviews.out b/src/test/regress/expected/sysviews.out
index d48abd7..c6c1405 100644
--- a/src/test/regress/expected/sysviews.out
+++ b/src/test/regress/expected/sysviews.out
@@ -70,20 +70,21 @@ select count(*) >= 0 as ok from pg_prepared_xacts;
 -- This is to record the prevailing planner enable_foo settings during
 -- a regression test run.
 select name, setting from pg_settings where name like 'enable%';
-         name         | setting 
-----------------------+---------
- enable_bitmapscan    | on
- enable_hashagg       | on
- enable_hashjoin      | on
- enable_indexonlyscan | on
- enable_indexscan     | on
- enable_material      | on
- enable_mergejoin     | on
- enable_nestloop      | on
- enable_seqscan       | on
- enable_sort          | on
- enable_tidscan       | on
-(11 rows)
+            name            | setting 
+----------------------------+---------
+ enable_bitmapscan          | on
+ enable_hashagg             | on
+ enable_hashjoin            | on
+ enable_indexonlyscan       | on
+ enable_indexscan           | on
+ enable_material            | on
+ enable_mergejoin           | on
+ enable_nestloop            | on
+ enable_partition_wise_join | on
+ enable_seqscan             | on
+ enable_sort                | on
+ enable_tidscan             | on
+(12 rows)
 
 -- Test that the pg_timezone_names and pg_timezone_abbrevs views are
 -- more-or-less working.  We can't test their contents in any great detail
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index 0322f1e..9b2baeb 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -3,6 +3,11 @@
 -- Test partition-wise join between partitioned tables
 --
 
+-- Usually partition-wise join paths are chosen when data is large, which would
+-- take regression tests to run longer. So, weigh partition-wise joins cheaper
+-- to force those even for smaller data.
+SET partition_wise_plan_weight to 0.2;
+
 --
 -- partitioned by a single column
 --
-- 
1.7.9.5

0009-Adjust-join-related-to-code-to-accept-child-relation.patchapplication/octet-stream; name=0009-Adjust-join-related-to-code-to-accept-child-relation.patchDownload
From e8fb61d8be1596b7c9727b29869cecdeb932ee1f Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Tue, 7 Feb 2017 17:16:46 +0530
Subject: [PATCH 09/11] Adjust join related to code to accept child relations.

Existing join related code doesn't expect child relations to be joined. This
patch contains various fixes to change that.

1. Uniqe-ifying joining relations.
=================================
For semi-joins we unique-ify the joining relations, which tries to estimate
nummber of unique values using estimate_num_groups(). This function doesn't
expect a Var from a child relation and contained an assertion to that effect.
With partition-wise joins, we may compute a join between child relations. This
commit changes that assertion to include child relation. The function doesn't
need any change other than that to accomodate child relations.

2. OUTER joins require dummy child relations to have targetlist.
================================================================
We need a targetlist defining nullable columns for an outer join, even if the
relation on the nullable side is deemed to be empty. Prior to partition-wise
join an empty child relation never had a targetlist since it was eliminated
from planning. But with partition-wise join an empty child relation may
participate in an outer join with another non-empty child relation. Hence set
targetlist for a child relation even if it's dummy.

3. prepare_sort_from_pathkeys fixes.
====================================
Before partition-wise join feature were never required to be directly sorted,
let's say for merge joins. With partition-wise join feature, the child
relations will participate directly in the join and also need to be sorted
directly for the purpose of merge join. In order to sort a relation, we use
pathkeys. The expression on which to sort a particular relation is provided by
the equivalence member corresponding to that relation in the equivalence class
referred by the pathkeys. Since the code doesn't expect child relations to
bubble up to the sorting, the function prepare_sort_from_pathkeys() skips any
child members (those set with em_is_child) unless the caller specifically asks
for child relations by passing relids. make_sort_from_pathkeys() calls
prepare_sort_from_pathkeys() to create Sort plan for outer and inner plans
without passing relids of the relation to be sorted. For partition-wise joins
the outer and inner plans produce child relations and thus
prepare_sort_from_pathkeys() does not find equivalence members since it skips
child members for the want of relids. This particular instance can be fixed by
passing outer/inner_path->parent->relids to prepare_sort_from_pathkeys().

All the callers of prepare_sort_from_pathkeys() viz.
create_merge_append_plan(), create_merge_append_plan(),
create_windowagg_plan() except make_sort_from_pathkeys() pass relids to
prepare_sort_from_pathkeys(). make_sort_from_pathkeys() as well passes those
with this patch.

make_sort_from_pathkeys() itself doesn't know the relids of relation being
sorted. It just gets the plan. Hence we need to pass relids to
make_sort_from_pathkeys() and thus change each of its callers to pass relids,
if required.

It has two callers as of now.
1. create_sort_plan(PlannerInfo *root, SortPath *best_path, int flags): does
not handle child relations yet, so doesn't need any change.
2. create_mergejoin_plan(PlannerInfo *root, MergePath *best_path):
It requires this change and the relids can be obtained from the outer and inner
path's parent RelOptInfo.

4. Handling em_is_child cases.
==============================
Right now, when comparing relids for child relations, only exact match is
considered. This is fine as long as em_relids has only a single member in it
and the passed in relids has only a single member in it. But with
partition-wise join, relids can have multiple members and em_relids may not
exactly match the given relids. But we need to find the member which covers
subset of given relids.
---
 src/backend/optimizer/path/allpaths.c   |   41 +++++++++++++++++--------------
 src/backend/optimizer/plan/createplan.c |   28 +++++++++++++--------
 src/backend/utils/adt/selfuncs.c        |    3 ++-
 3 files changed, 42 insertions(+), 30 deletions(-)

diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 1adf6ba..b4cd5ab 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -960,11 +960,27 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		}
 
 		/*
-		 * We have to copy the parent's targetlist and quals to the child,
-		 * with appropriate substitution of variables.  However, only the
-		 * baserestrictinfo quals are needed before we can check for
-		 * constraint exclusion; so do that first and then check to see if we
-		 * can disregard this child.
+		 * Copy/Modify targetlist. Even if this child is deemed empty, we need
+		 * its targetlist in case it falls on nullable side in a child-join
+		 * because of partition-wise join.
+		 *
+		 * NB: the resulting childrel->reltarget->exprs may contain arbitrary
+		 * expressions, which otherwise would not occur in a rel's targetlist.
+		 * Code that might be looking at an appendrel child must cope with
+		 * such.  (Normally, a rel's targetlist would only include Vars and
+		 * PlaceHolderVars.)  XXX we do not bother to update the cost or width
+		 * fields of childrel->reltarget; not clear if that would be useful.
+		 */
+		childrel->reltarget->exprs = (List *)
+			adjust_appendrel_attrs(root,
+								   (Node *) rel->reltarget->exprs,
+								   appinfo_list);
+
+		/*
+		 * We have to copy the parent's quals to the child, with appropriate
+		 * substitution of variables.  However, only the baserestrictinfo quals
+		 * are needed before we can check for constraint exclusion; so do that
+		 * first and then check to see if we can disregard this child.
 		 *
 		 * The child rel's targetlist might contain non-Var expressions, which
 		 * means that substitution into the quals could produce opportunities
@@ -1091,22 +1107,9 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 			continue;
 		}
 
-		/*
-		 * CE failed, so finish copying/modifying targetlist and join quals.
-		 *
-		 * NB: the resulting childrel->reltarget->exprs may contain arbitrary
-		 * expressions, which otherwise would not occur in a rel's targetlist.
-		 * Code that might be looking at an appendrel child must cope with
-		 * such.  (Normally, a rel's targetlist would only include Vars and
-		 * PlaceHolderVars.)  XXX we do not bother to update the cost or width
-		 * fields of childrel->reltarget; not clear if that would be useful.
-		 */
+		/* CE failed, so finish copying/modifying targetlist and join quals. */
 		childrel->joininfo = build_child_clauses(root, rel->joininfo,
 												 appinfo_list);
-		childrel->reltarget->exprs = (List *)
-			adjust_appendrel_attrs(root,
-								   (Node *) rel->reltarget->exprs,
-								   appinfo_list);
 
 		/*
 		 * We have to make child entries in the EquivalenceClass data
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index fe6b7f8..d0705dc 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -245,7 +245,8 @@ static Plan *prepare_sort_from_pathkeys(Plan *lefttree, List *pathkeys,
 static EquivalenceMember *find_ec_member_for_tle(EquivalenceClass *ec,
 					   TargetEntry *tle,
 					   Relids relids);
-static Sort *make_sort_from_pathkeys(Plan *lefttree, List *pathkeys);
+static Sort *make_sort_from_pathkeys(Plan *lefttree, List *pathkeys,
+									 Relids relids);
 static Sort *make_sort_from_groupcols(List *groupcls,
 						 AttrNumber *grpColIdx,
 						 Plan *lefttree);
@@ -1555,7 +1556,7 @@ create_sort_plan(PlannerInfo *root, SortPath *best_path, int flags)
 	subplan = create_plan_recurse(root, best_path->subpath,
 								  flags | CP_SMALL_TLIST);
 
-	plan = make_sort_from_pathkeys(subplan, best_path->path.pathkeys);
+	plan = make_sort_from_pathkeys(subplan, best_path->path.pathkeys, NULL);
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
@@ -3572,6 +3573,8 @@ create_mergejoin_plan(PlannerInfo *root,
 	ListCell   *lc;
 	ListCell   *lop;
 	ListCell   *lip;
+	Path	   *outer_path = best_path->jpath.outerjoinpath;
+	Path	   *inner_path = best_path->jpath.innerjoinpath;
 
 	/*
 	 * MergeJoin can project, so we don't have to demand exact tlists from the
@@ -3635,8 +3638,10 @@ create_mergejoin_plan(PlannerInfo *root,
 	 */
 	if (best_path->outersortkeys)
 	{
+		Relids		outer_relids = outer_path->parent->relids;
 		Sort	   *sort = make_sort_from_pathkeys(outer_plan,
-												   best_path->outersortkeys);
+												   best_path->outersortkeys,
+												   outer_relids);
 
 		label_sort_with_costsize(root, sort, -1.0);
 		outer_plan = (Plan *) sort;
@@ -3647,8 +3652,10 @@ create_mergejoin_plan(PlannerInfo *root,
 
 	if (best_path->innersortkeys)
 	{
+		Relids		inner_relids = inner_path->parent->relids;
 		Sort	   *sort = make_sort_from_pathkeys(inner_plan,
-												   best_path->innersortkeys);
+												   best_path->innersortkeys,
+												   inner_relids);
 
 		label_sort_with_costsize(root, sort, -1.0);
 		inner_plan = (Plan *) sort;
@@ -5630,11 +5637,11 @@ prepare_sort_from_pathkeys(Plan *lefttree, List *pathkeys,
 					continue;
 
 				/*
-				 * Ignore child members unless they match the rel being
+				 * Ignore child members unless they belong to the rel being
 				 * sorted.
 				 */
 				if (em->em_is_child &&
-					!bms_equal(em->em_relids, relids))
+					!bms_is_subset(em->em_relids, relids))
 					continue;
 
 				sortexpr = em->em_expr;
@@ -5745,10 +5752,10 @@ find_ec_member_for_tle(EquivalenceClass *ec,
 			continue;
 
 		/*
-		 * Ignore child members unless they match the rel being sorted.
+		 * Ignore child members unless they belong to the rel being sorted.
 		 */
 		if (em->em_is_child &&
-			!bms_equal(em->em_relids, relids))
+			!bms_is_subset(em->em_relids, relids))
 			continue;
 
 		/* Match if same expression (after stripping relabel) */
@@ -5769,9 +5776,10 @@ find_ec_member_for_tle(EquivalenceClass *ec,
  *
  *	  'lefttree' is the node which yields input tuples
  *	  'pathkeys' is the list of pathkeys by which the result is to be sorted
+ *	  'relids' is the set of relations required by prepare_sort_from_pathkeys()
  */
 static Sort *
-make_sort_from_pathkeys(Plan *lefttree, List *pathkeys)
+make_sort_from_pathkeys(Plan *lefttree, List *pathkeys, Relids relids)
 {
 	int			numsortkeys;
 	AttrNumber *sortColIdx;
@@ -5781,7 +5789,7 @@ make_sort_from_pathkeys(Plan *lefttree, List *pathkeys)
 
 	/* Compute sort column info, and adjust lefttree as needed */
 	lefttree = prepare_sort_from_pathkeys(lefttree, pathkeys,
-										  NULL,
+										  relids,
 										  NULL,
 										  false,
 										  &numsortkeys,
diff --git a/src/backend/utils/adt/selfuncs.c b/src/backend/utils/adt/selfuncs.c
index fa32e9e..c833846 100644
--- a/src/backend/utils/adt/selfuncs.c
+++ b/src/backend/utils/adt/selfuncs.c
@@ -3427,7 +3427,8 @@ estimate_num_groups(PlannerInfo *root, List *groupExprs, double input_rows,
 		/*
 		 * Sanity check --- don't divide by zero if empty relation.
 		 */
-		Assert(rel->reloptkind == RELOPT_BASEREL);
+		Assert(rel->reloptkind == RELOPT_BASEREL ||
+			   rel->reloptkind == RELOPT_OTHER_MEMBER_REL);
 		if (rel->tuples > 0)
 		{
 			/*
-- 
1.7.9.5

0010-Parameterized-path-fixes.patchapplication/octet-stream; name=0010-Parameterized-path-fixes.patchDownload
From 406fdb63f146b2baca9afaa78735ddc121ec9671 Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Tue, 7 Feb 2017 17:25:25 +0530
Subject: [PATCH 10/11] Parameterized path fixes.

We do not create merge or hash join paths when the inner path is parameterized
by the outer and vice-versa. Parameterization information in path refers to the
top-most parent relation. Current tests (PATH_PARAM_BY_REL) to avoid joining
such paths fail while joining child relations; the paths from either child may
be paramterized by other's parent. Modify the tests to consider paths
parameterized by parent as parameterized by any of its child.

If the inner path is parameterized by outer path, we can create a nested loop
join using those two paths with inner relation parameterized by the outer
relation. For LATERAL JOINs this is the only legal way to plan a join. In case
of partitioned joins, the lateral references refer to the topmost parent and
hence inner paths are parameterized by the topmost parent. In such cases, it's
possible to translate the inner path to be parameterized by
the child and create nested loop join. When presented with a pair of child
relation paths, where the inner paths is parameterized by the parent of outer
child, this patch translates the path to be parameterized by the outer child
and creates a nested loop join path.

The function reparameterize_path_by_child() needs to call adjust_relid_set() to
substitute parent relids by child relids in Path::param_info::ppi_req_outer.
Hence "extern"alized that function. Since there is already another
static adjust_relid_set() in rewriteManip.c, renamed this one to
adjust_child_relids().

Also "extern"alized find_param_path_info() required by
reparameterize_path_by_child().
---
 src/backend/optimizer/path/joinpath.c  |   33 +++++-
 src/backend/optimizer/prep/prepunion.c |   42 ++++----
 src/backend/optimizer/util/pathnode.c  |  180 ++++++++++++++++++++++++++++++++
 src/backend/optimizer/util/relnode.c   |    2 -
 src/include/optimizer/pathnode.h       |    4 +
 src/include/optimizer/prep.h           |    1 +
 6 files changed, 237 insertions(+), 25 deletions(-)

diff --git a/src/backend/optimizer/path/joinpath.c b/src/backend/optimizer/path/joinpath.c
index f80fb25..4d4a183 100644
--- a/src/backend/optimizer/path/joinpath.c
+++ b/src/backend/optimizer/path/joinpath.c
@@ -25,9 +25,19 @@
 /* Hook for plugins to get control in add_paths_to_joinrel() */
 set_join_pathlist_hook_type set_join_pathlist_hook = NULL;
 
-#define PATH_PARAM_BY_REL(path, rel)  \
+/*
+ * Paths parameterized by the parent can be considered to be parameterized by
+ * any of its child.
+ */
+#define PATH_PARAM_BY_PARENT(path, rel)	\
+	((path)->param_info && bms_overlap(PATH_REQ_OUTER(path),	\
+									   (rel)->top_parent_relids))
+#define PATH_PARAM_BY_REL_SELF(path, rel)  \
 	((path)->param_info && bms_overlap(PATH_REQ_OUTER(path), (rel)->relids))
 
+#define PATH_PARAM_BY_REL(path, rel)	\
+	(PATH_PARAM_BY_REL_SELF(path, rel) || PATH_PARAM_BY_PARENT(path, rel))
+
 static void sort_inner_and_outer(PlannerInfo *root, RelOptInfo *joinrel,
 					 RelOptInfo *outerrel, RelOptInfo *innerrel,
 					 JoinType jointype, JoinPathExtraData *extra);
@@ -301,6 +311,27 @@ try_nestloop_path(PlannerInfo *root,
 	JoinCostWorkspace workspace;
 
 	/*
+	 * Since result produced by a child is part of the result produced by its
+	 * topmost parent and has same properties, the parameters representing that
+	 * parent may be substituted by values from a child. Hence expressions and
+	 * hence paths using those expressions, parameterized by a parent can be
+	 * said to be parameterized by any of its child.  For a join between child
+	 * relations, if the inner path is parameterized by the parent of the outer
+	 * relation,  create a nestloop join path with inner relation parameterized
+	 * by the outer relation by translating the inner path to be parameterized
+	 * by the outer child relation.
+	 */
+	if (PATH_PARAM_BY_PARENT(inner_path, outer_path->parent))
+	{
+		inner_path = reparameterize_path_by_child(root, inner_path,
+												   outer_path->parent);
+
+		/* If we could not translate the path, don't produce nest loop path. */
+		if (!inner_path)
+			return;
+	}
+
+	/*
 	 * Check to see if proposed path is still parameterized, and reject if the
 	 * parameterization wouldn't be sensible --- unless allow_star_schema_join
 	 * says to allow it anyway.  Also, we must reject if have_dangerous_phv
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index 676204f..d459e95 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -109,7 +109,6 @@ static Node *adjust_appendrel_attrs_mutator(Node *node,
 							   adjust_appendrel_attrs_context *context);
 static List *adjust_inherited_tlist(List *tlist,
 					   AppendRelInfo *context);
-static Relids adjust_relid_set(Relids relids, List *append_rel_infos);
 
 
 /*
@@ -1951,7 +1950,7 @@ adjust_appendrel_attrs_mutator(Node *node,
 														 (void *) context);
 		/* now fix PlaceHolderVar's relid sets */
 		if (phv->phlevelsup == 0)
-			phv->phrels = adjust_relid_set(phv->phrels, context->appinfos);
+			phv->phrels = adjust_child_relids(phv->phrels, context->appinfos);
 		return (Node *) phv;
 	}
 	/* Shouldn't need to handle planner auxiliary nodes here */
@@ -1982,17 +1981,17 @@ adjust_appendrel_attrs_mutator(Node *node,
 			adjust_appendrel_attrs_mutator((Node *) oldinfo->orclause, context);
 
 		/* adjust relid sets too */
-		newinfo->clause_relids = adjust_relid_set(oldinfo->clause_relids,
+		newinfo->clause_relids = adjust_child_relids(oldinfo->clause_relids,
 													 context->appinfos);
-		newinfo->required_relids = adjust_relid_set(oldinfo->required_relids,
+		newinfo->required_relids = adjust_child_relids(oldinfo->required_relids,
 													 context->appinfos);
-		newinfo->outer_relids = adjust_relid_set(oldinfo->outer_relids,
+		newinfo->outer_relids = adjust_child_relids(oldinfo->outer_relids,
 													 context->appinfos);
-		newinfo->nullable_relids = adjust_relid_set(oldinfo->nullable_relids,
+		newinfo->nullable_relids = adjust_child_relids(oldinfo->nullable_relids,
 													   context->appinfos);
-		newinfo->left_relids = adjust_relid_set(oldinfo->left_relids,
+		newinfo->left_relids = adjust_child_relids(oldinfo->left_relids,
 												   context->appinfos);
-		newinfo->right_relids = adjust_relid_set(oldinfo->right_relids,
+		newinfo->right_relids = adjust_child_relids(oldinfo->right_relids,
 													context->appinfos);
 
 		/*
@@ -2026,15 +2025,18 @@ adjust_appendrel_attrs_mutator(Node *node,
 
 /*
  * Replace parent relids by child relids in the copy of given relid set
- * according to the given list of AppendRelInfos. The given relid set is
- * returned as is if it contains no parent in the given list, otherwise, the
- * given relid set is not changed.
+ * according to the given list of AppendRelInfos.
  */
 Relids
-adjust_relid_set(Relids relids, List *append_rel_infos)
+adjust_child_relids(Relids relids, List *append_rel_infos)
 {
 	ListCell   *lc;
-	Bitmapset  *result = NULL;
+
+	/*
+	 * The new relids set may be expected to be in a memory context different
+	 * from the given one. Make a copy here.
+	 */
+	Bitmapset  *result = bms_copy(relids);
 
 	foreach (lc, append_rel_infos)
 	{
@@ -2043,10 +2045,6 @@ adjust_relid_set(Relids relids, List *append_rel_infos)
 		/* Remove parent, add child */
 		if (bms_is_member(appinfo->parent_relid, relids))
 		{
-			/* Make a copy if we are changing the set. */
-			if (!result)
-				result = bms_copy(relids);
-
 			result = bms_del_member(result, appinfo->parent_relid);
 			result = bms_add_member(result, appinfo->child_relid);
 		}
@@ -2202,7 +2200,7 @@ build_child_restrictinfo(PlannerInfo *root, RestrictInfo *rinfo,
 	RestrictInfo   *child_rinfo;
 	MemoryContext	old_context;
 
-	child_required_relids = adjust_relid_set(rinfo->required_relids,
+	child_required_relids = adjust_child_relids(rinfo->required_relids,
 												append_rel_infos);
 
 
@@ -2313,13 +2311,13 @@ build_child_join_sjinfo(PlannerInfo *root, SpecialJoinInfo *parent_sjinfo,
 
 	memcpy(sjinfo, parent_sjinfo, sizeof(SpecialJoinInfo));
 
-	sjinfo->min_lefthand = adjust_relid_set(sjinfo->min_lefthand,
+	sjinfo->min_lefthand = adjust_child_relids(sjinfo->min_lefthand,
 											   left_appinfos);
-	sjinfo->min_righthand = adjust_relid_set(sjinfo->min_righthand,
+	sjinfo->min_righthand = adjust_child_relids(sjinfo->min_righthand,
 												right_appinfos);
-	sjinfo->syn_lefthand = adjust_relid_set(sjinfo->syn_lefthand,
+	sjinfo->syn_lefthand = adjust_child_relids(sjinfo->syn_lefthand,
 											   left_appinfos);
-	sjinfo->syn_righthand = adjust_relid_set(sjinfo->syn_righthand,
+	sjinfo->syn_righthand = adjust_child_relids(sjinfo->syn_righthand,
 												right_appinfos);
 
 	/*
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index d861a49..f322320 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -3388,3 +3388,183 @@ reparameterize_path(PlannerInfo *root, Path *path,
 	}
 	return NULL;
 }
+
+/*
+ * reparameterize_path_by_child
+ * 		Given a path parameterized by the parent of the given relation,
+ * 		translate the path to be parameterized by the given child relation.
+ *
+ * The function creates a new path of the same type as the given path, but
+ * parameterized by the given child relation. If it can not reparameterize the
+ * path as required, it returns NULL.
+ *
+ * The cost, number of rows, width and parallel path properties depend upon
+ * path->parent, which does not change during the translation. Hence those
+ * members are copied as they are.
+ */
+
+Path *
+reparameterize_path_by_child(PlannerInfo *root, Path *path,
+							  RelOptInfo *child_rel)
+{
+	Path	   *new_path;
+	ParamPathInfo   *new_ppi;
+	ParamPathInfo   *old_ppi;
+	List	   *child_aris;
+	Relids		required_outer;
+
+	/*
+	 * If the path is not parameterized by parent of the given relation or it it
+	 * doesn't need reparameterization.
+	 */
+	if (!path->param_info ||
+		!bms_overlap(PATH_REQ_OUTER(path), child_rel->top_parent_relids))
+	return path;
+
+	switch (nodeTag(path))
+	{
+		case T_Path:
+			new_path = makeNode(Path);
+			memcpy(new_path, path, sizeof(Path));
+			break;
+
+		case T_HashPath:
+			new_path = (Path *) makeNode(HashPath);
+			memcpy(new_path, path, sizeof(HashPath));
+			break;
+
+		case T_MergePath:
+			new_path = (Path *) makeNode(MergePath);
+			memcpy(new_path, path, sizeof(MergePath));
+			break;
+
+		case T_NestPath:
+			new_path = (Path *) makeNode(NestPath);
+			memcpy(new_path, path, sizeof(NestPath));
+			break;
+
+		case T_IndexPath:
+			new_path = (Path *) makeNode(IndexPath);
+			memcpy(new_path, path, sizeof(IndexPath));
+			break;
+
+		case T_AppendPath:
+			new_path = (Path *) makeNode(AppendPath);
+			memcpy(new_path, path, sizeof(AppendPath));
+			break;
+
+		/*
+		 * TODO:
+		 * If this method of translation is fine add more path types here.
+		 */
+
+		default:
+			/* Path type unsupported by this function. */
+			return NULL;
+	}
+
+	/*
+	 * Gather AppendRelInfos of the base partition relations in the outer child
+	 * relation. We need those for translating parent path to that of child by
+	 * substituting parent Var nodes and relids with those of children.
+	 */
+	child_aris = find_appinfos_by_relids(root, child_rel->relids);
+
+	/* Adjust the parameterization information. */
+	old_ppi = new_path->param_info;
+	required_outer = adjust_child_relids(old_ppi->ppi_req_outer, child_aris);
+
+	/* If we already have a PPI for this parameterization, just return it */
+	new_ppi = find_param_path_info(new_path->parent, required_outer);
+
+	/* If not build a new one and link it to the list of PPIs. */
+	if (!new_ppi)
+	{
+		new_ppi = makeNode(ParamPathInfo);
+		new_ppi->ppi_req_outer = required_outer;
+		new_ppi->ppi_rows = old_ppi->ppi_rows;
+		new_ppi->ppi_clauses = build_child_clauses(root, old_ppi->ppi_clauses,
+												   child_aris);
+		new_path->parent->ppilist = lappend(new_path->parent->ppilist, new_ppi);
+	}
+	else
+		bms_free(required_outer);
+
+	new_path->param_info = new_ppi;
+
+	/*
+	 * Adjust the path target if the parent of the outer relation is referenced
+	 * in the targetlist. This can happen when only the parent of outer relation is
+	 * laterally referenced in this relation.
+	 */
+	if (bms_overlap(path->parent->lateral_relids, child_rel->top_parent_relids))
+	{
+		MemoryContext	old_context;
+
+		/*
+		 * Allocate the target in planner's context, since they are copies as
+		 * is from path while creating plans.
+		 */
+		old_context = MemoryContextSwitchTo(root->planner_cxt);
+		new_path->pathtarget = copy_pathtarget(new_path->pathtarget);
+		new_path->pathtarget->exprs = (List *) adjust_appendrel_attrs(root,
+											(Node *) new_path->pathtarget->exprs,
+																	child_aris);
+		MemoryContextSwitchTo(old_context);
+	}
+
+	/*
+	 * Change parameterization of subpaths recursively. Also carry out any
+	 * pathtype specific adjustments.
+	 */
+	switch (nodeTag(path))
+	{
+		case T_HashPath:
+		case T_MergePath:
+		case T_NestPath:
+			{
+				JoinPath *jpath = (JoinPath *)new_path;
+
+				jpath->outerjoinpath = reparameterize_path_by_child(root,
+														 jpath->outerjoinpath,
+														 child_rel);
+				jpath->innerjoinpath = reparameterize_path_by_child(root,
+														 jpath->innerjoinpath,
+														 child_rel);
+				jpath->joinrestrictinfo = build_child_clauses(root,
+															  jpath->joinrestrictinfo,
+															  child_aris);
+			}
+			break;
+
+		case T_AppendPath:
+			{
+				AppendPath	*apath = (AppendPath *)new_path;
+				List		*subpaths = NIL;
+				ListCell	*lc;
+
+				foreach (lc, apath->subpaths)
+					subpaths = lappend(subpaths,
+									   reparameterize_path_by_child(root,
+																	lfirst(lc),
+																	child_rel));
+				apath->subpaths = subpaths;
+			}
+
+		case T_IndexPath:
+			{
+				IndexPath *ipath = (IndexPath *)new_path;
+
+				ipath->indexclauses = build_child_clauses(root, ipath->indexclauses,
+														  child_aris);
+				ipath->indexquals = build_child_clauses(root, ipath->indexquals,
+														child_aris);
+			}
+
+		default:
+			/* Nothing to do. */
+			break;
+	}
+
+	return new_path;
+}
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 1eed987..46eea02 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -53,8 +53,6 @@ static List *subbuild_joinrel_joinlist(RelOptInfo *joinrel,
 static void set_foreign_rel_properties(RelOptInfo *joinrel,
 						   RelOptInfo *outer_rel, RelOptInfo *inner_rel);
 static void add_join_rel(PlannerInfo *root, RelOptInfo *joinrel);
-extern ParamPathInfo *find_param_path_info(RelOptInfo *rel,
-									  Relids required_outer);
 static void build_joinrel_partition_info(RelOptInfo *joinrel,
 						  RelOptInfo *outer_rel, RelOptInfo *inner_rel,
 						  JoinType jointype);
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 81d637a..b9f5b11 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -236,6 +236,8 @@ extern PartitionJoinPath *create_partition_join_path(PlannerInfo *root,
 extern Path *reparameterize_path(PlannerInfo *root, Path *path,
 					Relids required_outer,
 					double loop_count);
+extern Path *reparameterize_path_by_child(PlannerInfo *root, Path *path,
+					RelOptInfo *child_rel);
 
 /*
  * prototypes for relnode.c
@@ -277,5 +279,7 @@ extern ParamPathInfo *get_appendrel_parampathinfo(RelOptInfo *appendrel,
 extern RelOptInfo *build_child_join_rel(PlannerInfo *root,
 						 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
 						 RelOptInfo *parent_joinrel, JoinType jointype);
+extern ParamPathInfo *find_param_path_info(RelOptInfo *rel,
+									  Relids required_outer);
 
 #endif   /* PATHNODE_H */
diff --git a/src/include/optimizer/prep.h b/src/include/optimizer/prep.h
index 5832130..0347b37 100644
--- a/src/include/optimizer/prep.h
+++ b/src/include/optimizer/prep.h
@@ -65,5 +65,6 @@ extern List *find_appinfos_by_relids(PlannerInfo *root, Relids relids);
 extern SpecialJoinInfo *build_child_join_sjinfo(PlannerInfo *root,
 									SpecialJoinInfo *parent_sjinfo,
 									Relids left_relids, Relids right_relids);
+extern Relids adjust_child_relids(Relids relids, List *append_rel_infos);
 
 #endif   /* PREP_H */
-- 
1.7.9.5

0011-Use-IS_JOIN_REL-instead-of-RELOPT_JOINREL.patchapplication/octet-stream; name=0011-Use-IS_JOIN_REL-instead-of-RELOPT_JOINREL.patchDownload
From 38c9b261a37a402c980b0b64427a4f76ea44f345 Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Wed, 8 Feb 2017 14:42:15 +0530
Subject: [PATCH 11/11] Use IS_JOIN_REL() instead of RELOPT_JOINREL

FDW code uses RELOPT_JOINREL to check whether a given relation is a join or
not. Partition-wise joins create child-join relations, which are marked as
RELOPT_OTHER_JOINREL. Macro IS_JOIN_REL() returns true for both of those kinds,
replace RELOPT_JOINREL tests with IS_JOIN_REL() test.

Similarly replace RELOPT_OTHER_MEMBER_REL test with IS_OTHER_REL() where we
want to test for child relations of all kinds.
---
 contrib/postgres_fdw/deparse.c      |   10 +++++-----
 contrib/postgres_fdw/postgres_fdw.c |   10 ++++++----
 src/backend/foreign/foreign.c       |    6 +++---
 3 files changed, 14 insertions(+), 12 deletions(-)

diff --git a/contrib/postgres_fdw/deparse.c b/contrib/postgres_fdw/deparse.c
index d2b94aa..a2171d7 100644
--- a/contrib/postgres_fdw/deparse.c
+++ b/contrib/postgres_fdw/deparse.c
@@ -911,7 +911,7 @@ deparseSelectStmtForRel(StringInfo buf, PlannerInfo *root, RelOptInfo *rel,
 	 * We handle relations for foreign tables, joins between those and upper
 	 * relations.
 	 */
-	Assert(rel->reloptkind == RELOPT_JOINREL ||
+	Assert(IS_JOIN_REL(rel) ||
 		   rel->reloptkind == RELOPT_BASEREL ||
 		   rel->reloptkind == RELOPT_OTHER_MEMBER_REL ||
 		   rel->reloptkind == RELOPT_UPPER_REL);
@@ -990,7 +990,7 @@ deparseSelectSql(List *tlist, List **retrieved_attrs, deparse_expr_cxt *context)
 	 */
 	appendStringInfoString(buf, "SELECT ");
 
-	if (foreignrel->reloptkind == RELOPT_JOINREL ||
+	if (IS_JOIN_REL(foreignrel) ||
 		foreignrel->reloptkind == RELOPT_UPPER_REL)
 	{
 		/* For a join relation use the input tlist */
@@ -1030,7 +1030,7 @@ deparseFromExpr(List *quals, deparse_expr_cxt *context)
 
 	/* For upper relations, scanrel must be either a joinrel or a baserel */
 	Assert(context->foreignrel->reloptkind != RELOPT_UPPER_REL ||
-		   scanrel->reloptkind == RELOPT_JOINREL ||
+		   IS_JOIN_REL(scanrel) ||
 		   scanrel->reloptkind == RELOPT_BASEREL);
 
 	/* Construct FROM clause */
@@ -1178,7 +1178,7 @@ deparseLockingClause(deparse_expr_cxt *context)
 			appendStringInfoString(buf, " FOR UPDATE");
 
 			/* Add the relation alias if we are here for a join relation */
-			if (rel->reloptkind == RELOPT_JOINREL)
+			if (IS_JOIN_REL(rel))
 				appendStringInfo(buf, " OF %s%d", REL_ALIAS_PREFIX, relid);
 		}
 		else
@@ -1342,7 +1342,7 @@ deparseFromExprForRel(StringInfo buf, PlannerInfo *root, RelOptInfo *foreignrel,
 {
 	PgFdwRelationInfo *fpinfo = (PgFdwRelationInfo *) foreignrel->fdw_private;
 
-	if (foreignrel->reloptkind == RELOPT_JOINREL)
+	if (IS_JOIN_REL(foreignrel))
 	{
 		RelOptInfo *rel_o = fpinfo->outerrel;
 		RelOptInfo *rel_i = fpinfo->innerrel;
diff --git a/contrib/postgres_fdw/postgres_fdw.c b/contrib/postgres_fdw/postgres_fdw.c
index 5d270b9..2487f26 100644
--- a/contrib/postgres_fdw/postgres_fdw.c
+++ b/contrib/postgres_fdw/postgres_fdw.c
@@ -723,6 +723,8 @@ get_useful_ecs_for_relation(PlannerInfo *root, RelOptInfo *rel)
 	/* If this is a child rel, we must use the topmost parent rel to search. */
 	if (rel->reloptkind == RELOPT_OTHER_MEMBER_REL)
 		relids = find_childrel_top_parent(root, rel)->relids;
+	else if (rel->reloptkind == RELOPT_OTHER_JOINREL)
+		relids = rel->top_parent_relids;
 	else
 		relids = rel->relids;
 
@@ -1181,7 +1183,7 @@ postgresGetForeignPlan(PlannerInfo *root,
 			local_exprs = lappend(local_exprs, rinfo->clause);
 	}
 
-	if (foreignrel->reloptkind == RELOPT_JOINREL ||
+	if (IS_JOIN_REL(foreignrel) ||
 		foreignrel->reloptkind == RELOPT_UPPER_REL)
 	{
 		/* For a join relation, get the conditions from fdw_private structure */
@@ -1247,7 +1249,7 @@ postgresGetForeignPlan(PlannerInfo *root,
 							 remote_conds,
 							 retrieved_attrs,
 							 makeInteger(fpinfo->fetch_size));
-	if (foreignrel->reloptkind == RELOPT_JOINREL ||
+	if (IS_JOIN_REL(foreignrel) ||
 		foreignrel->reloptkind == RELOPT_UPPER_REL)
 		fdw_private = lappend(fdw_private,
 							  makeString(fpinfo->relation_name->data));
@@ -2527,7 +2529,7 @@ estimate_path_cost_size(PlannerInfo *root,
 						   &remote_param_join_conds, &local_param_join_conds);
 
 		/* Build the list of columns to be fetched from the foreign server. */
-		if (foreignrel->reloptkind == RELOPT_JOINREL ||
+		if (IS_JOIN_REL(foreignrel) ||
 			foreignrel->reloptkind == RELOPT_UPPER_REL)
 			fdw_scan_tlist = build_tlist_to_deparse(foreignrel);
 		else
@@ -2609,7 +2611,7 @@ estimate_path_cost_size(PlannerInfo *root,
 			startup_cost = fpinfo->rel_startup_cost;
 			run_cost = fpinfo->rel_total_cost - fpinfo->rel_startup_cost;
 		}
-		else if (foreignrel->reloptkind == RELOPT_JOINREL)
+		else if (IS_JOIN_REL(foreignrel))
 		{
 			PgFdwRelationInfo *fpinfo_i;
 			PgFdwRelationInfo *fpinfo_o;
diff --git a/src/backend/foreign/foreign.c b/src/backend/foreign/foreign.c
index fdb4f71..e8ca7df 100644
--- a/src/backend/foreign/foreign.c
+++ b/src/backend/foreign/foreign.c
@@ -717,7 +717,7 @@ GetExistingLocalJoinPath(RelOptInfo *joinrel)
 {
 	ListCell   *lc;
 
-	Assert(joinrel->reloptkind == RELOPT_JOINREL);
+	Assert(IS_JOIN_REL(joinrel));
 
 	foreach(lc, joinrel->pathlist)
 	{
@@ -782,7 +782,7 @@ GetExistingLocalJoinPath(RelOptInfo *joinrel)
 			ForeignPath *foreign_path;
 
 			foreign_path = (ForeignPath *) joinpath->outerjoinpath;
-			if (foreign_path->path.parent->reloptkind == RELOPT_JOINREL)
+			if (IS_JOIN_REL(foreign_path->path.parent))
 				joinpath->outerjoinpath = foreign_path->fdw_outerpath;
 		}
 
@@ -791,7 +791,7 @@ GetExistingLocalJoinPath(RelOptInfo *joinrel)
 			ForeignPath *foreign_path;
 
 			foreign_path = (ForeignPath *) joinpath->innerjoinpath;
-			if (foreign_path->path.parent->reloptkind == RELOPT_JOINREL)
+			if (IS_JOIN_REL(foreign_path->path.parent))
 				joinpath->innerjoinpath = foreign_path->fdw_outerpath;
 		}
 
-- 
1.7.9.5

#33Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Ashutosh Bapat (#32)
11 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

Fixed a problem with the way qsort was being used in the earlier set
of patches. Attached PFA the set of patches with that fixed.

On Thu, Feb 9, 2017 at 4:20 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Per your suggestion I have split the patch into many smaller patches.

0001-Refactor-set_append_rel_pathlist.patch
0002-Refactor-make_join_rel.patch
0003-Refactor-adjust_appendrel_attrs.patch
0004-Refactor-build_join_rel.patch
0005-Add-function-find_param_path_info.patch

These four refactor existing code.

0006-Canonical-partition-scheme.patch
0007-Partition-wise-join-tests.patch -- just tests, they fail
0008-Partition-wise-join.patch -- actual patch implementing
partition-wise join, still some tests fail\

0009-Adjust-join-related-to-code-to-accept-child-relation.patch
0010-Parameterized-path-fixes.patch
0011-Use-IS_JOIN_REL-instead-of-RELOPT_JOINREL.patch

The last three patches change existing code to expect child(-join)
relations where they were not expected earlier.

Each patch has summary of the changes.

Partition-wise join for multi-level partitioned tables is not covered
by these patches. I will post those patches soon.

Other questions/comments:

Why does find_partition_scheme need to copy the partition bound
information instead of just pointing to it? Amit went to some trouble
to make sure that this can't change under us while we hold a lock on
the relation, and we'd better hold a lock on the relation if we're
planning a query against it.

PartitionScheme is shared across multiple relations, join or base,
partitioned similarly. Obviously it can't and does not need to point
partition bound informations (which should all be same) of all those
base relations. O the the face of it, it looks weird that it points to
only one of them, mostly the one which it encounters first. But, since
it's going to be the same partition bound information, it doesn't
matter which one. So, I think, we can point of any one of those. Do
you agree?

Instead of copying PartitionBoundInfo, used pointer of the first
encountered one.

I think the PartitionScheme stuff should live in the optimizer rather
that src/backend/catalog/partition.c. Maybe plancat.c? Perhaps we
eventually need a new file in the optimizer just for partitioning
stuff, but I'm not sure about that yet.

I placed PartitionScheme stuff in partition.c because most of the
functions and structures in partition.c are not visible outside that
file. But I will try again to locate PartitionScheme to optimizer.

Moved the code as per your suggestion.

The fact that set_append_rel_size needs to reopen the relation to
extract a few more bits of information is not desirable. You need to
fish this information through in some other way; for example, you
could have get_relation_info() stash the needed bits in the
RelOptInfo.

I considered this option and discarded it, since not all partitioned
relations will have OIDs for partitions e.g. partitioned joins will
not have OIDs for their partitions. But now that I think of it, we
should probably store those OIDs just for the base relation and leave
them unused for non-base relations just like other base relation
specific fields in RelOptInfo.

Changed as per your suggestions.

+                * For two partitioned tables with the same
partitioning scheme, it is
+                * assumed that the Oids of matching partitions from
both the tables
+                * are placed at the same position in the array of
partition oids in

Rather than saying that we assume this, you should say why it has to
be true. (If it doesn't have to be true, we shouldn't assume it.)

Will take care of this.

Done. Please check.

+                * join relations. Partition tables should have same
layout as the
+                * parent table and hence should not need any
translation. But rest of

The same attributes have to be present with the same types, but they
can be rearranged. This comment seems to imply the contrary.

Hmm, will take care of this.

Done.

FRACTION_PARTS_TO_PLAN seems like it should be a GUC.

+1. Will take care of this. Does "representative_partitions_fraction"
or "sample_partition_fraction" look like a good GUC name? Any other
suggestions?

used "sample_partition_fraction" for now. Suggestions are welcome.

+               /*
+                * Add this relation to the list of samples ordered by
the increasing
+                * number of rows at appropriate place.
+                */
+               foreach (lc, ordered_child_nos)
+               {
+                       int     child_no = lfirst_int(lc);
+                       RelOptInfo *other_childrel = rel->part_rels[child_no];
+
+                       /*
+                        * Keep track of child with lowest number of
rows but higher than the
+                        * that of the child being inserted. Insert
the child before a
+                        * child with highest number of rows lesser than it.
+                        */
+                       if (child_rel->rows <= other_childrel->rows)
+                               insert_after = lc;
+                       else
+                               break;
+               }

Can we use quicksort instead of a hand-coded insertion sort?

I guess so, if I write comparison functions, which shouldn't be a
problem. Will try that.

Done.

+ if (bms_num_members(outer_relids) > 1)

Seems like bms_get_singleton_member could be used.

That code is not required any more.

+ * Partitioning scheme in join relation indicates a possibilty that the

Spelling.

Done.

There seems to be no reason for create_partition_plan to be separated
from create_plan_recurse. You can just add another case for the new
path type.

Done.

Why does create_partition_join_path need to be separate from
create_partition_join_path_with_pathkeys? Couldn't that be combined
into a single function with a pathkeys argument that might sometimes
be NIL? I assume most of the logic is common.

Combined those into a single function.

From a sort of theoretical standpoint, the biggest danger of this
patch seems to be that by deferring path creation until a later stage
than normal, we could miss some important processing.
subquery_planner() does a lot of stuff after
expand_inherited_tables(); if any of those things, especially the ones
that happen AFTER path generation, have an effect on the paths, then
this code needs to compensate for those changes somehow. It seems
like having the planning of unsampled children get deferred until
create_plan() time is awfully surprising; here we are creating the
plan and suddenly what used to be a straightforward path->plan
translation is running around doing major planning work. I can't
entirely justify it, but I somehow have a feeling that work ought to
be moved earlier. Not sure exactly where.

Pasting my previous replies here to keep everything in one mail.

I agree with this. Probably we should add a path tree mutator before
SS_identify_outer_params() to replace any Partition*Paths with
Merge/Append paths. The mutator will create paths for child-joins
within temporary memory context, copy the relevant paths and create
Merge/Append paths. There are two problems there 1. We have to write
code to copy paths; most of the paths would be flat copy but custom
scan paths might have some unexpected problems. 2. There will be many
surviving PartitionPaths, and all the corresponding child paths would
need copying and consume memory. In order to reduce that consumption,
we have run this mutator after set_cheapest() in subquery_planner();
but then nothing interesting happens between that and create_plan().
Expanding PartitionPaths during create_plan() does not need any path
copying and we expand only the PartitionPaths which will be converted
to plans. That does save a lot of memory; the reason why we defer
creating paths for child-joins.

This is not really a full review, mostly because I can't easily figure
out the motivation for all of the changes the patch makes. It makes a
lot of changes in a lot of places, and it's not really very easy to
understand why those changes are necessary. My comments above about
splitting the patch into a series of patches that can potentially be
reviewed and applied independently, with the main patch being the last
in the series, are a suggestion as to how to tackle that. There might
be some work that needs to or could be done on the comments, too. For
example, the patch splits out add_paths_to_append_rel from
set_append_rel_pathlist, but the comments don't say anything helpful
like "we need to do X after Y, because Z". They just say that we do
it. To some extent I think the comments in the optimizer have that
problem generally, so it's not entirely the fault of this patch;
still, the lack of those explanations makes the code reorganization
harder to follow, and might confuse future patch authors, too.

Specifically about add_paths_to_append_rel(), what do you expect the
comment to say? It would be obvious why we split that functionality
into a separate function: in fact, we don't necessarily explain why
certain code resides in a separate function in the comments. I think,
that particular comment (or for that matter other such comments in the
optimizer) can be removed altogether, since it just writes the
function names as an "English" sentence. I sometimes find those
comments useful, because I can read just those comments and forget
about the code, making comprehension easy. If highlighting is ON, your
brain habitually ignores the non-comment portions when required. I am
open to suggestions.

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

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

Attachments:

0001-Refactor-set_append_rel_pathlist.patchapplication/octet-stream; name=0001-Refactor-set_append_rel_pathlist.patchDownload
From cf978fcadc1985c27783c114b8a367fe795efe9e Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Fri, 3 Feb 2017 14:45:37 +0530
Subject: [PATCH 01/11] Refactor set_append_rel_pathlist.

set_append_rel_pathlist() creates paths for each child relation and then
creates append paths for the "append" relation. Right now only a base relation
can have children, but with partition-wise join and aggregation a join or an
upper relation can have children as well. While the code to add paths to the
child relations differs for base, join and upper child relations, the code to
create append paths can be shared by all the three relations. Hence separating
it into a new function add_paths_to_append_rel() so that it can be re-used for
all kinds of relations.
---
 src/backend/optimizer/path/allpaths.c |   51 ++++++++++++++++++++++++++-------
 1 file changed, 41 insertions(+), 10 deletions(-)

diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 5c18987..d797d6a 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -127,6 +127,8 @@ static void recurse_push_qual(Node *setOp, Query *topquery,
 				  RangeTblEntry *rte, Index rti, Node *qual);
 static void remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel);
 static int	compute_parallel_worker(RelOptInfo *rel, BlockNumber pages);
+static void add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
+									List *live_childrels);
 
 
 /*
@@ -1169,19 +1171,11 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 {
 	int			parentRTindex = rti;
 	List	   *live_childrels = NIL;
-	List	   *subpaths = NIL;
-	bool		subpaths_valid = true;
-	List	   *partial_subpaths = NIL;
-	bool		partial_subpaths_valid = true;
-	List	   *all_child_pathkeys = NIL;
-	List	   *all_child_outers = NIL;
 	ListCell   *l;
 
 	/*
 	 * Generate access paths for each member relation, and remember the
-	 * cheapest path for each one.  Also, identify all pathkeys (orderings)
-	 * and parameterizations (required_outer sets) available for the member
-	 * relations.
+	 * non-dummy children.
 	 */
 	foreach(l, root->append_rel_list)
 	{
@@ -1189,7 +1183,6 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 		int			childRTindex;
 		RangeTblEntry *childRTE;
 		RelOptInfo *childrel;
-		ListCell   *lcp;
 
 		/* append_rel_list contains all append rels; ignore others */
 		if (appinfo->parent_relid != parentRTindex)
@@ -1224,6 +1217,44 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 		 * Child is live, so add it to the live_childrels list for use below.
 		 */
 		live_childrels = lappend(live_childrels, childrel);
+	}
+
+	/* Add Append/MergeAppend paths to the "append" relation. */
+	add_paths_to_append_rel(root, rel, live_childrels);
+}
+
+
+/*
+ * add_paths_to_append_rel
+ *		Generate Append/MergeAppend paths for given "append" relation.
+ *
+ * The function collects all parameterizations and orderings supported by the
+ * non-dummy children. For every such parameterization or ordering, it creates
+ * an append path collecting one path from each non-dummy child with given
+ * parameterization or ordering. Similarly it collects partial paths from
+ * non-dummy children to create partial append paths.
+ */
+static void
+add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
+						List *live_childrels)
+{
+	List	   *subpaths = NIL;
+	bool		subpaths_valid = true;
+	List	   *partial_subpaths = NIL;
+	bool		partial_subpaths_valid = true;
+	List	   *all_child_pathkeys = NIL;
+	List	   *all_child_outers = NIL;
+	ListCell   *l;
+
+	/*
+	 * For every non-dummy child, remember the cheapest path.  Also, identify
+	 * all pathkeys (orderings) and parameterizations (required_outer sets)
+	 * available for the non-dummy member relations.
+	 */
+	foreach (l, live_childrels)
+	{
+		RelOptInfo *childrel = lfirst(l);
+		ListCell   *lcp;
 
 		/*
 		 * If child has an unparameterized cheapest-total path, add that to
-- 
1.7.9.5

0002-Refactor-make_join_rel.patchapplication/octet-stream; name=0002-Refactor-make_join_rel.patchDownload
From 80861ce9a56d639833f015392bb6d4035a9deda2 Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Fri, 3 Feb 2017 15:41:39 +0530
Subject: [PATCH 02/11] Refactor make_join_rel().

The code in make_join_rel() to add paths to join relation for a given pair of
joining relations can be re-used to add paths to a child join relation, which
do not need the other functionality offered by make_join_rel(). Separate this
code into populate_joinrel_with_paths(). This patch does just refactors
make_join_rel() to pave the way for partition-wise join.
---
 src/backend/optimizer/path/joinrels.c |   28 ++++++++++++++++++++++++----
 1 file changed, 24 insertions(+), 4 deletions(-)

diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 6f3c20b..936ee0c 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -32,6 +32,9 @@ static bool is_dummy_rel(RelOptInfo *rel);
 static void mark_dummy_rel(RelOptInfo *rel);
 static bool restriction_is_constant_false(List *restrictlist,
 							  bool only_pushed_down);
+static void populate_joinrel_with_paths(PlannerInfo *root, RelOptInfo *rel1,
+							RelOptInfo *rel2, RelOptInfo *joinrel,
+							SpecialJoinInfo *sjinfo, List *restrictlist);
 
 
 /*
@@ -724,6 +727,27 @@ make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 		return joinrel;
 	}
 
+	/* Add paths to the join relation. */
+	populate_joinrel_with_paths(root, rel1, rel2, joinrel, sjinfo,
+								restrictlist);
+
+	bms_free(joinrelids);
+
+	return joinrel;
+}
+
+/*
+ * populate_joinrel_with_paths
+ *	  Add paths to the given joinrel for given pair of joining relations. The
+ *	  SpecialJoinInfo provides details about the join and the restrictlist
+ *	  contains the join clauses and the other clauses applicable for given pair
+ *	  of the joining relations.
+ */
+static void
+populate_joinrel_with_paths(PlannerInfo *root, RelOptInfo *rel1,
+							RelOptInfo *rel2, RelOptInfo *joinrel,
+							SpecialJoinInfo *sjinfo, List *restrictlist)
+{
 	/*
 	 * Consider paths using each rel as both outer and inner.  Depending on
 	 * the join type, a provably empty outer or inner rel might mean the join
@@ -868,10 +892,6 @@ make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 			elog(ERROR, "unrecognized join type: %d", (int) sjinfo->jointype);
 			break;
 	}
-
-	bms_free(joinrelids);
-
-	return joinrel;
 }
 
 
-- 
1.7.9.5

0003-Refactor-adjust_appendrel_attrs.patchapplication/octet-stream; name=0003-Refactor-adjust_appendrel_attrs.patchDownload
From 74d38c91f9c37f8c6ede2a1a63ae3fec6731a10d Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Fri, 3 Feb 2017 17:22:52 +0530
Subject: [PATCH 03/11] Refactor adjust_appendrel_attrs.

adjust_appendrel_attrs() is used to translate nodes for a parent relation to
those for a child relation by replacing the parent specific nodes like Var
nodes with corresponding nodes specific to the child. Right now this function
works with a single parent-child pair. For partition-wise join and
partition-wise aggregation/grouping, we require to translate nodes for multiple
parent-child pairs. This patch modifies adjust_appendrel_attrs() to work with
multiple parent-child pairs.
---
 src/backend/optimizer/path/allpaths.c   |    7 +-
 src/backend/optimizer/path/equivclass.c |    2 +-
 src/backend/optimizer/plan/planner.c    |    2 +-
 src/backend/optimizer/prep/prepunion.c  |  161 ++++++++++++++++++++++---------
 src/include/optimizer/prep.h            |    2 +-
 5 files changed, 121 insertions(+), 53 deletions(-)

diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index d797d6a..d8fac14 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -862,6 +862,7 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		ListCell   *parentvars;
 		ListCell   *childvars;
 		ListCell   *lc;
+		List	   *appinfo_list = list_make1(appinfo);
 
 		/* append_rel_list contains all append rels; ignore others */
 		if (appinfo->parent_relid != parentRTindex)
@@ -903,7 +904,7 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 			Assert(IsA(rinfo, RestrictInfo));
 			childqual = adjust_appendrel_attrs(root,
 											   (Node *) rinfo->clause,
-											   appinfo);
+											   appinfo_list);
 			childqual = eval_const_expressions(root, childqual);
 			/* check for flat-out constant */
 			if (childqual && IsA(childqual, Const))
@@ -1022,11 +1023,11 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		childrel->joininfo = (List *)
 			adjust_appendrel_attrs(root,
 								   (Node *) rel->joininfo,
-								   appinfo);
+								   appinfo_list);
 		childrel->reltarget->exprs = (List *)
 			adjust_appendrel_attrs(root,
 								   (Node *) rel->reltarget->exprs,
-								   appinfo);
+								   appinfo_list);
 
 		/*
 		 * We have to make child entries in the EquivalenceClass data
diff --git a/src/backend/optimizer/path/equivclass.c b/src/backend/optimizer/path/equivclass.c
index a329dd1..bcce142 100644
--- a/src/backend/optimizer/path/equivclass.c
+++ b/src/backend/optimizer/path/equivclass.c
@@ -2111,7 +2111,7 @@ add_child_rel_equivalences(PlannerInfo *root,
 				child_expr = (Expr *)
 					adjust_appendrel_attrs(root,
 										   (Node *) cur_em->em_expr,
-										   appinfo);
+										   list_make1(appinfo));
 
 				/*
 				 * Transform em_relids to match.  Note we do *not* do
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 881742f..24a48b8 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -1085,7 +1085,7 @@ inheritance_planner(PlannerInfo *root)
 		subroot->parse = (Query *)
 			adjust_appendrel_attrs(root,
 								   (Node *) parse,
-								   appinfo);
+								   list_make1(appinfo));
 
 		/*
 		 * If there are securityQuals attached to the parent, move them to the
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index 06e843d..6f41979 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -55,7 +55,7 @@
 typedef struct
 {
 	PlannerInfo *root;
-	AppendRelInfo *appinfo;
+	List	    *appinfos;
 } adjust_appendrel_attrs_context;
 
 static Path *recurse_set_operations(Node *setOp, PlannerInfo *root,
@@ -107,9 +107,9 @@ static Bitmapset *translate_col_privs(const Bitmapset *parent_privs,
 					List *translated_vars);
 static Node *adjust_appendrel_attrs_mutator(Node *node,
 							   adjust_appendrel_attrs_context *context);
-static Relids adjust_relid_set(Relids relids, Index oldrelid, Index newrelid);
 static List *adjust_inherited_tlist(List *tlist,
 					   AppendRelInfo *context);
+static Relids adjust_relid_set(Relids relids, List *append_rel_infos);
 
 
 /*
@@ -1719,10 +1719,10 @@ translate_col_privs(const Bitmapset *parent_privs,
 
 /*
  * adjust_appendrel_attrs
- *	  Copy the specified query or expression and translate Vars referring
- *	  to the parent rel of the specified AppendRelInfo to refer to the
- *	  child rel instead.  We also update rtindexes appearing outside Vars,
- *	  such as resultRelation and jointree relids.
+ *	  Copy the specified query or expression and translate Vars referring to
+ *	  the parent rels of the specified in the given list of AppendRelInfos to
+ *	  refer to the corresponding child rel instead.  We also update rtindexes
+ *	  appearing outside Vars, such as resultRelation and jointree relids.
  *
  * Note: this is only applied after conversion of sublinks to subplans,
  * so we don't need to cope with recursion into sub-queries.
@@ -1731,13 +1731,20 @@ translate_col_privs(const Bitmapset *parent_privs,
  * maybe we should try to fold the two routines together.
  */
 Node *
-adjust_appendrel_attrs(PlannerInfo *root, Node *node, AppendRelInfo *appinfo)
+adjust_appendrel_attrs(PlannerInfo *root, Node *node, List *appinfos)
 {
 	Node	   *result;
 	adjust_appendrel_attrs_context context;
+	ListCell   *lc;
 
 	context.root = root;
-	context.appinfo = appinfo;
+	context.appinfos = appinfos;
+
+	/*
+	 * Catch a caller who wants to adjust expressions, but doesn't pass any
+	 * AppendRelInfo.
+	 */
+	Assert(appinfos && list_length(appinfos) >= 1);
 
 	/*
 	 * Must be prepared to start with a Query or a bare expression tree.
@@ -1745,20 +1752,28 @@ adjust_appendrel_attrs(PlannerInfo *root, Node *node, AppendRelInfo *appinfo)
 	if (node && IsA(node, Query))
 	{
 		Query	   *newnode;
+		AppendRelInfo *appinfo;
 
 		newnode = query_tree_mutator((Query *) node,
 									 adjust_appendrel_attrs_mutator,
 									 (void *) &context,
 									 QTW_IGNORE_RC_SUBQUERIES);
-		if (newnode->resultRelation == appinfo->parent_relid)
+		foreach (lc, appinfos)
 		{
-			newnode->resultRelation = appinfo->child_relid;
-			/* Fix tlist resnos too, if it's inherited UPDATE */
-			if (newnode->commandType == CMD_UPDATE)
-				newnode->targetList =
-					adjust_inherited_tlist(newnode->targetList,
-										   appinfo);
+			appinfo = lfirst(lc);
+
+			if (newnode->resultRelation == appinfo->parent_relid)
+			{
+				newnode->resultRelation = appinfo->child_relid;
+				/* Fix tlist resnos too, if it's inherited UPDATE */
+				if (newnode->commandType == CMD_UPDATE)
+					newnode->targetList =
+									adjust_inherited_tlist(newnode->targetList,
+														   appinfo);
+				break;
+			}
 		}
+
 		result = (Node *) newnode;
 	}
 	else
@@ -1771,13 +1786,29 @@ static Node *
 adjust_appendrel_attrs_mutator(Node *node,
 							   adjust_appendrel_attrs_context *context)
 {
-	AppendRelInfo *appinfo = context->appinfo;
+	List   *appinfos = context->appinfos;
+	ListCell   *lc;
+
+	/*
+	 * Catch a caller who wants to adjust expressions, but doesn't pass any
+	 * AppendRelInfo.
+	 */
+	Assert(appinfos && list_length(appinfos) >= 1);
 
 	if (node == NULL)
 		return NULL;
 	if (IsA(node, Var))
 	{
 		Var		   *var = (Var *) copyObject(node);
+		AppendRelInfo *appinfo;
+
+		foreach (lc, appinfos)
+		{
+			appinfo = lfirst(lc);
+
+			if (var->varno == appinfo->parent_relid)
+				break;
+		}
 
 		if (var->varlevelsup == 0 &&
 			var->varno == appinfo->parent_relid)
@@ -1860,29 +1891,54 @@ adjust_appendrel_attrs_mutator(Node *node,
 	{
 		CurrentOfExpr *cexpr = (CurrentOfExpr *) copyObject(node);
 
-		if (cexpr->cvarno == appinfo->parent_relid)
-			cexpr->cvarno = appinfo->child_relid;
+		foreach (lc, appinfos)
+		{
+			AppendRelInfo *appinfo = lfirst(lc);
+
+			if (cexpr->cvarno == appinfo->parent_relid)
+			{
+				cexpr->cvarno = appinfo->child_relid;
+				break;
+			}
+		}
 		return (Node *) cexpr;
 	}
 	if (IsA(node, RangeTblRef))
 	{
 		RangeTblRef *rtr = (RangeTblRef *) copyObject(node);
 
-		if (rtr->rtindex == appinfo->parent_relid)
-			rtr->rtindex = appinfo->child_relid;
+		foreach (lc, appinfos)
+		{
+			AppendRelInfo *appinfo = lfirst(lc);
+
+			if (rtr->rtindex == appinfo->parent_relid)
+			{
+				rtr->rtindex = appinfo->child_relid;
+				break;
+			}
+		}
 		return (Node *) rtr;
 	}
 	if (IsA(node, JoinExpr))
 	{
 		/* Copy the JoinExpr node with correct mutation of subnodes */
 		JoinExpr   *j;
+		AppendRelInfo *appinfo;
 
 		j = (JoinExpr *) expression_tree_mutator(node,
 											  adjust_appendrel_attrs_mutator,
 												 (void *) context);
 		/* now fix JoinExpr's rtindex (probably never happens) */
-		if (j->rtindex == appinfo->parent_relid)
-			j->rtindex = appinfo->child_relid;
+		foreach (lc, appinfos)
+		{
+			appinfo = lfirst(lc);
+
+			if (j->rtindex == appinfo->parent_relid)
+			{
+				j->rtindex = appinfo->child_relid;
+				break;
+			}
+		}
 		return (Node *) j;
 	}
 	if (IsA(node, PlaceHolderVar))
@@ -1895,9 +1951,7 @@ adjust_appendrel_attrs_mutator(Node *node,
 														 (void *) context);
 		/* now fix PlaceHolderVar's relid sets */
 		if (phv->phlevelsup == 0)
-			phv->phrels = adjust_relid_set(phv->phrels,
-										   appinfo->parent_relid,
-										   appinfo->child_relid);
+			phv->phrels = adjust_relid_set(phv->phrels, context->appinfos);
 		return (Node *) phv;
 	}
 	/* Shouldn't need to handle planner auxiliary nodes here */
@@ -1929,23 +1983,17 @@ adjust_appendrel_attrs_mutator(Node *node,
 
 		/* adjust relid sets too */
 		newinfo->clause_relids = adjust_relid_set(oldinfo->clause_relids,
-												  appinfo->parent_relid,
-												  appinfo->child_relid);
+													 context->appinfos);
 		newinfo->required_relids = adjust_relid_set(oldinfo->required_relids,
-													appinfo->parent_relid,
-													appinfo->child_relid);
+													 context->appinfos);
 		newinfo->outer_relids = adjust_relid_set(oldinfo->outer_relids,
-												 appinfo->parent_relid,
-												 appinfo->child_relid);
+													 context->appinfos);
 		newinfo->nullable_relids = adjust_relid_set(oldinfo->nullable_relids,
-													appinfo->parent_relid,
-													appinfo->child_relid);
+													   context->appinfos);
 		newinfo->left_relids = adjust_relid_set(oldinfo->left_relids,
-												appinfo->parent_relid,
-												appinfo->child_relid);
+												   context->appinfos);
 		newinfo->right_relids = adjust_relid_set(oldinfo->right_relids,
-												 appinfo->parent_relid,
-												 appinfo->child_relid);
+													context->appinfos);
 
 		/*
 		 * Reset cached derivative fields, since these might need to have
@@ -1977,19 +2025,38 @@ adjust_appendrel_attrs_mutator(Node *node,
 }
 
 /*
- * Substitute newrelid for oldrelid in a Relid set
+ * Replace parent relids by child relids in the copy of given relid set
+ * according to the given list of AppendRelInfos. The given relid set is
+ * returned as is if it contains no parent in the given list, otherwise, the
+ * given relid set is not changed.
  */
-static Relids
-adjust_relid_set(Relids relids, Index oldrelid, Index newrelid)
+Relids
+adjust_relid_set(Relids relids, List *append_rel_infos)
 {
-	if (bms_is_member(oldrelid, relids))
+	ListCell   *lc;
+	Bitmapset  *result = NULL;
+
+	foreach (lc, append_rel_infos)
 	{
-		/* Ensure we have a modifiable copy */
-		relids = bms_copy(relids);
-		/* Remove old, add new */
-		relids = bms_del_member(relids, oldrelid);
-		relids = bms_add_member(relids, newrelid);
+		AppendRelInfo	*appinfo = lfirst(lc);
+
+		/* Remove parent, add child */
+		if (bms_is_member(appinfo->parent_relid, relids))
+		{
+			/* Make a copy if we are changing the set. */
+			if (!result)
+				result = bms_copy(relids);
+
+			result = bms_del_member(result, appinfo->parent_relid);
+			result = bms_add_member(result, appinfo->child_relid);
+		}
 	}
+
+	/* Return new set if we modified the given set. */
+	if (result)
+		return result;
+
+	/* Else return the given relids set as is. */
 	return relids;
 }
 
@@ -2110,5 +2177,5 @@ adjust_appendrel_attrs_multilevel(PlannerInfo *root, Node *node,
 	else
 		Assert(parent_rel->reloptkind == RELOPT_BASEREL);
 	/* Now translate for this child */
-	return adjust_appendrel_attrs(root, node, appinfo);
+	return adjust_appendrel_attrs(root, node, list_make1(appinfo));
 }
diff --git a/src/include/optimizer/prep.h b/src/include/optimizer/prep.h
index 2b20b36..a02e06a 100644
--- a/src/include/optimizer/prep.h
+++ b/src/include/optimizer/prep.h
@@ -53,7 +53,7 @@ extern RelOptInfo *plan_set_operations(PlannerInfo *root);
 extern void expand_inherited_tables(PlannerInfo *root);
 
 extern Node *adjust_appendrel_attrs(PlannerInfo *root, Node *node,
-					   AppendRelInfo *appinfo);
+					   List *appinfos);
 
 extern Node *adjust_appendrel_attrs_multilevel(PlannerInfo *root, Node *node,
 								  RelOptInfo *child_rel);
-- 
1.7.9.5

0004-Refactor-build_join_rel.patchapplication/octet-stream; name=0004-Refactor-build_join_rel.patchDownload
From 5606e2526b997606e8485c00fc34ac1f5ccf2fe9 Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Mon, 6 Feb 2017 10:58:48 +0530
Subject: [PATCH 04/11] Refactor build_join_rel.

Partition-wise joins do not use build_join_rel() to build child-join relations,
but it still requires code to set foreign relation properties as well as code
to add join relation into PlannerInfo. Separate that code into
set_foreign_rel_properties() and add_join_rel() resp. to be called while
building child joins.
---
 src/backend/optimizer/util/relnode.c |  142 ++++++++++++++++++++--------------
 1 file changed, 83 insertions(+), 59 deletions(-)

diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index adc1db9..160ed6d 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -49,6 +49,9 @@ static List *subbuild_joinrel_restrictlist(RelOptInfo *joinrel,
 static List *subbuild_joinrel_joinlist(RelOptInfo *joinrel,
 						  List *joininfo_list,
 						  List *new_joininfo);
+static void set_foreign_rel_properties(RelOptInfo *joinrel,
+						   RelOptInfo *outer_rel, RelOptInfo *inner_rel);
+static void add_join_rel(PlannerInfo *root, RelOptInfo *joinrel);
 
 
 /*
@@ -327,6 +330,82 @@ find_join_rel(PlannerInfo *root, Relids relids)
 }
 
 /*
+ * set_foreign_rel_properties
+ *		Set up foreign-join fields if outer and inner relation are foreign
+ *		tables (or joins) belonging to the same server and assigned to the same
+ *		user to check access permissions as.
+ *
+ * In addition to an exact match of userid, we allow the case where one side
+ * has zero userid (implying current user) and the other side has explicit
+ * userid that happens to equal the current user; but in that case, pushdown of
+ * the join is only valid for the current user.  The useridiscurrent field
+ * records whether we had to make such an assumption for this join or any
+ * sub-join.
+ *
+ * Otherwise these fields are left invalid, so GetForeignJoinPaths will not be
+ * called for the join relation.
+ *
+ */
+static void
+set_foreign_rel_properties(RelOptInfo *joinrel, RelOptInfo *outer_rel,
+						   RelOptInfo *inner_rel)
+{
+	if (OidIsValid(outer_rel->serverid) &&
+		inner_rel->serverid == outer_rel->serverid)
+	{
+		if (inner_rel->userid == outer_rel->userid)
+		{
+			joinrel->serverid = outer_rel->serverid;
+			joinrel->userid = outer_rel->userid;
+			joinrel->useridiscurrent = outer_rel->useridiscurrent || inner_rel->useridiscurrent;
+			joinrel->fdwroutine = outer_rel->fdwroutine;
+		}
+		else if (!OidIsValid(inner_rel->userid) &&
+				 outer_rel->userid == GetUserId())
+		{
+			joinrel->serverid = outer_rel->serverid;
+			joinrel->userid = outer_rel->userid;
+			joinrel->useridiscurrent = true;
+			joinrel->fdwroutine = outer_rel->fdwroutine;
+		}
+		else if (!OidIsValid(outer_rel->userid) &&
+				 inner_rel->userid == GetUserId())
+		{
+			joinrel->serverid = outer_rel->serverid;
+			joinrel->userid = inner_rel->userid;
+			joinrel->useridiscurrent = true;
+			joinrel->fdwroutine = outer_rel->fdwroutine;
+		}
+	}
+}
+
+/*
+ * add_join_rel
+ *		Add given join relation to the list of join relations in the given
+ *		PlannerInfo. Also add it to the auxiliary hashtable if there is one.
+ */
+void
+add_join_rel(PlannerInfo *root, RelOptInfo *joinrel)
+{
+	/* GEQO requires us to append the new joinrel to the end of the list! */
+	root->join_rel_list = lappend(root->join_rel_list, joinrel);
+
+	/* store it into the auxiliary hashtable if there is one. */
+	if (root->join_rel_hash)
+	{
+		JoinHashEntry *hentry;
+		bool		found;
+
+		hentry = (JoinHashEntry *) hash_search(root->join_rel_hash,
+											   &(joinrel->relids),
+											   HASH_ENTER,
+											   &found);
+		Assert(!found);
+		hentry->join_rel = joinrel;
+	}
+}
+
+/*
  * build_join_rel
  *	  Returns relation entry corresponding to the union of two given rels,
  *	  creating a new relation entry if none already exists.
@@ -424,46 +503,8 @@ build_join_rel(PlannerInfo *root,
 	joinrel->joininfo = NIL;
 	joinrel->has_eclass_joins = false;
 
-	/*
-	 * Set up foreign-join fields if outer and inner relation are foreign
-	 * tables (or joins) belonging to the same server and assigned to the same
-	 * user to check access permissions as.  In addition to an exact match of
-	 * userid, we allow the case where one side has zero userid (implying
-	 * current user) and the other side has explicit userid that happens to
-	 * equal the current user; but in that case, pushdown of the join is only
-	 * valid for the current user.  The useridiscurrent field records whether
-	 * we had to make such an assumption for this join or any sub-join.
-	 *
-	 * Otherwise these fields are left invalid, so GetForeignJoinPaths will
-	 * not be called for the join relation.
-	 */
-	if (OidIsValid(outer_rel->serverid) &&
-		inner_rel->serverid == outer_rel->serverid)
-	{
-		if (inner_rel->userid == outer_rel->userid)
-		{
-			joinrel->serverid = outer_rel->serverid;
-			joinrel->userid = outer_rel->userid;
-			joinrel->useridiscurrent = outer_rel->useridiscurrent || inner_rel->useridiscurrent;
-			joinrel->fdwroutine = outer_rel->fdwroutine;
-		}
-		else if (!OidIsValid(inner_rel->userid) &&
-				 outer_rel->userid == GetUserId())
-		{
-			joinrel->serverid = outer_rel->serverid;
-			joinrel->userid = outer_rel->userid;
-			joinrel->useridiscurrent = true;
-			joinrel->fdwroutine = outer_rel->fdwroutine;
-		}
-		else if (!OidIsValid(outer_rel->userid) &&
-				 inner_rel->userid == GetUserId())
-		{
-			joinrel->serverid = outer_rel->serverid;
-			joinrel->userid = inner_rel->userid;
-			joinrel->useridiscurrent = true;
-			joinrel->fdwroutine = outer_rel->fdwroutine;
-		}
-	}
+	/* Compute information relevant to the foreign relations. */
+	set_foreign_rel_properties(joinrel, outer_rel, inner_rel);
 
 	/*
 	 * Create a new tlist containing just the vars that need to be output from
@@ -531,25 +572,8 @@ build_join_rel(PlannerInfo *root,
 		is_parallel_safe(root, (Node *) joinrel->reltarget->exprs))
 		joinrel->consider_parallel = true;
 
-	/*
-	 * Add the joinrel to the query's joinrel list, and store it into the
-	 * auxiliary hashtable if there is one.  NB: GEQO requires us to append
-	 * the new joinrel to the end of the list!
-	 */
-	root->join_rel_list = lappend(root->join_rel_list, joinrel);
-
-	if (root->join_rel_hash)
-	{
-		JoinHashEntry *hentry;
-		bool		found;
-
-		hentry = (JoinHashEntry *) hash_search(root->join_rel_hash,
-											   &(joinrel->relids),
-											   HASH_ENTER,
-											   &found);
-		Assert(!found);
-		hentry->join_rel = joinrel;
-	}
+	/* Add the joinrel to the PlannerInfo. */
+	add_join_rel(root, joinrel);
 
 	/*
 	 * Also, if dynamic-programming join search is active, add the new joinrel
-- 
1.7.9.5

0005-Add-function-find_param_path_info.patchapplication/octet-stream; name=0005-Add-function-find_param_path_info.patchDownload
From ec2984ab4ea387d5e91fbd354209ff45f114b603 Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Mon, 6 Feb 2017 12:14:06 +0530
Subject: [PATCH 05/11] Add function find_param_path_info.

The code to search ParamPathInfo for a set of required outer relations in the
list of ParamPathInfos of a given relation is duplicated in
get_*rel_parampathinfo() functions. Separate this code into
find_param_path_info() and call it from get_*rel_parampathinfo() functions.
---
 src/backend/optimizer/util/relnode.c |   46 ++++++++++++++++++++--------------
 1 file changed, 27 insertions(+), 19 deletions(-)

diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 160ed6d..19982dc 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -52,6 +52,8 @@ static List *subbuild_joinrel_joinlist(RelOptInfo *joinrel,
 static void set_foreign_rel_properties(RelOptInfo *joinrel,
 						   RelOptInfo *outer_rel, RelOptInfo *inner_rel);
 static void add_join_rel(PlannerInfo *root, RelOptInfo *joinrel);
+extern ParamPathInfo *find_param_path_info(RelOptInfo *rel,
+									  Relids required_outer);
 
 
 /*
@@ -1047,12 +1049,8 @@ get_baserel_parampathinfo(PlannerInfo *root, RelOptInfo *baserel,
 	Assert(!bms_overlap(baserel->relids, required_outer));
 
 	/* If we already have a PPI for this parameterization, just return it */
-	foreach(lc, baserel->ppilist)
-	{
-		ppi = (ParamPathInfo *) lfirst(lc);
-		if (bms_equal(ppi->ppi_req_outer, required_outer))
-			return ppi;
-	}
+	if ((ppi = find_param_path_info(baserel, required_outer)))
+		return ppi;
 
 	/*
 	 * Identify all joinclauses that are movable to this base rel given this
@@ -1289,12 +1287,8 @@ get_joinrel_parampathinfo(PlannerInfo *root, RelOptInfo *joinrel,
 	*restrict_clauses = list_concat(pclauses, *restrict_clauses);
 
 	/* If we already have a PPI for this parameterization, just return it */
-	foreach(lc, joinrel->ppilist)
-	{
-		ppi = (ParamPathInfo *) lfirst(lc);
-		if (bms_equal(ppi->ppi_req_outer, required_outer))
-			return ppi;
-	}
+	if ((ppi = find_param_path_info(joinrel, required_outer)))
+		return ppi;
 
 	/* Estimate the number of rows returned by the parameterized join */
 	rows = get_parameterized_joinrel_size(root, joinrel,
@@ -1333,7 +1327,6 @@ ParamPathInfo *
 get_appendrel_parampathinfo(RelOptInfo *appendrel, Relids required_outer)
 {
 	ParamPathInfo *ppi;
-	ListCell   *lc;
 
 	/* Unparameterized paths have no ParamPathInfo */
 	if (bms_is_empty(required_outer))
@@ -1342,12 +1335,8 @@ get_appendrel_parampathinfo(RelOptInfo *appendrel, Relids required_outer)
 	Assert(!bms_overlap(appendrel->relids, required_outer));
 
 	/* If we already have a PPI for this parameterization, just return it */
-	foreach(lc, appendrel->ppilist)
-	{
-		ppi = (ParamPathInfo *) lfirst(lc);
-		if (bms_equal(ppi->ppi_req_outer, required_outer))
-			return ppi;
-	}
+	if ((ppi = find_param_path_info(appendrel, required_outer)))
+		return ppi;
 
 	/* Else build the ParamPathInfo */
 	ppi = makeNode(ParamPathInfo);
@@ -1358,3 +1347,22 @@ get_appendrel_parampathinfo(RelOptInfo *appendrel, Relids required_outer)
 
 	return ppi;
 }
+
+/*
+ * Returns a ParamPathInfo for outer relations specified by required_outer, if
+ * already available in the given rel. Returns NULL otherwise.
+ */
+ParamPathInfo *
+find_param_path_info(RelOptInfo *rel, Relids required_outer)
+{
+	ListCell   *lc;
+
+	foreach(lc, rel->ppilist)
+	{
+		ParamPathInfo  *ppi = (ParamPathInfo *) lfirst(lc);
+		if (bms_equal(ppi->ppi_req_outer, required_outer))
+			return ppi;
+	}
+
+	return NULL;
+}
-- 
1.7.9.5

0006-Canonical-partition-scheme.patchapplication/octet-stream; name=0006-Canonical-partition-scheme.patchDownload
From 53c7c239715824278c2abe19c15abdb1ed3d7d91 Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Tue, 7 Feb 2017 10:47:49 +0530
Subject: [PATCH 06/11] Canonical partition scheme.

For a single level partitioned table, annotate RelOptInfo of a partitioned
table with canonical partition scheme. All partitioned tables, with the same
partitioning scheme share the same canonical partitioning scheme. We store the
RelOptInfo's corresponding to the partitions in RelOptInfo of the partitioned
table. Those are arranged in the same order as the partition bound indices in
the partition scheme.

We do not handle multi-level partitioned tables since inheritance hierarchy
does not retain the partition hierarchy.  All the partitions at any level
appear as children of the top-level partitioned table. Thus making it hard to
associate a partition relation with corresponding partition bounds.

Multi-level partitioned tables will be handled in a separate patch.
---
 src/backend/optimizer/path/allpaths.c |   48 +++++++
 src/backend/optimizer/util/plancat.c  |  232 +++++++++++++++++++++++++++++++++
 src/include/nodes/relation.h          |   51 ++++++++
 3 files changed, 331 insertions(+)

diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index d8fac14..0eb56f3 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -829,6 +829,7 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 	double	   *parent_attrsizes;
 	int			nattrs;
 	ListCell   *l;
+	int			nparts;
 
 	/*
 	 * Initialize to compute size estimates for whole append relation.
@@ -850,6 +851,18 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 	nattrs = rel->max_attr - rel->min_attr + 1;
 	parent_attrsizes = (double *) palloc0(nattrs * sizeof(double));
 
+	/*
+	 * For a partitioned table, allocate an array to hold RelOptInfo's of the
+	 * partitions. It will be filled while handling the children below.
+	 */
+	if (rel->part_scheme)
+	{
+		nparts = rel->part_scheme->nparts;
+		rel->part_rels = (RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
+	}
+	else
+		nparts = 0;
+
 	foreach(l, root->append_rel_list)
 	{
 		AppendRelInfo *appinfo = (AppendRelInfo *) lfirst(l);
@@ -879,6 +892,30 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		Assert(childrel->reloptkind == RELOPT_OTHER_MEMBER_REL);
 
 		/*
+		 * Two partitioned tables with the same partitioning scheme, have their
+		 * partition bounds arranged in the same order. The order of partition
+		 * OIDs in RelOptInfo corresponds to the partition bound order. Thus
+		 * the OIDs of matching partitions from both the tables are placed at
+		 * the same position in the array of partition OIDs in the respective
+		 * RelOptInfos. Arranging RelOptInfos of partitions in the same order
+		 * as their OIDs makes it easy to find the RelOptInfos of matching
+		 * partitions for partition-wise join.
+		 */
+		if (rel->part_scheme)
+		{
+			int		cnt_parts;
+
+			for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+			{
+				if (rel->part_oids[cnt_parts] == childRTE->relid)
+				{
+					Assert(!rel->part_rels[cnt_parts]);
+					rel->part_rels[cnt_parts] = childrel;
+				}
+			}
+		}
+
+		/*
 		 * We have to copy the parent's targetlist and quals to the child,
 		 * with appropriate substitution of variables.  However, only the
 		 * baserestrictinfo quals are needed before we can check for
@@ -1130,6 +1167,17 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		}
 	}
 
+	/* Should have found all the childrels of a partitioned relation. */
+	if (rel->part_scheme)
+	{
+		int		cnt_parts;
+
+		for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+			if (!rel->part_rels[cnt_parts])
+				elog(ERROR, "could not find the RelOptInfo of a partition with oid %u",
+					 rel->part_oids[cnt_parts]);
+	}
+
 	if (has_live_children)
 	{
 		/*
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index 7836e6b..01ba885 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -27,6 +27,7 @@
 #include "catalog/catalog.h"
 #include "catalog/dependency.h"
 #include "catalog/heap.h"
+#include "catalog/pg_inherits_fn.h"
 #include "catalog/partition.h"
 #include "catalog/pg_am.h"
 #include "foreign/fdwapi.h"
@@ -63,6 +64,13 @@ static List *get_relation_constraints(PlannerInfo *root,
 						 bool include_notnull);
 static List *build_index_tlist(PlannerInfo *root, IndexOptInfo *index,
 				  Relation heapRelation);
+static List **build_baserel_partition_key_exprs(Relation relation,
+												Index varno);
+static PartitionScheme find_partition_scheme(struct PlannerInfo *root,
+											 Relation rel);
+static void get_relation_partition_info(PlannerInfo *root, RelOptInfo *rel,
+										Relation relation, bool inhparent);
+
 
 
 /*
@@ -412,6 +420,9 @@ get_relation_info(PlannerInfo *root, Oid relationObjectId, bool inhparent,
 	/* Collect info about relation's foreign keys, if relevant */
 	get_relation_foreign_keys(root, rel, relation, inhparent);
 
+	/* Collect info about relation's partitioning scheme, if any. */
+	get_relation_partition_info(root, rel, relation, inhparent);
+
 	heap_close(relation, NoLock);
 
 	/*
@@ -1716,3 +1727,224 @@ has_row_triggers(PlannerInfo *root, Index rti, CmdType event)
 	heap_close(relation, NoLock);
 	return result;
 }
+
+/*
+ * get_relation_partition_info
+ *
+ * Retrieves partitioning information for a given relation.
+ *
+ * Partitioning scheme, partition key expressions and OIDs of partitions are
+ * added to the given RelOptInfo. A partitioned table can participate in the
+ * query as a simple relation or an inheritance parent. Only the later can have
+ * child relations, and hence partitions. From the point of view of the query
+ * optimizer only such relations are considered to be partitioned. Hence
+ * partitioning information is set only for an inheritance parent.
+ */
+static void
+get_relation_partition_info(PlannerInfo *root, RelOptInfo *rel,
+							Relation relation, bool inhparent)
+{
+	/* No partitioning information for an unpartitioned relation. */
+	if (relation->rd_rel->relkind != RELKIND_PARTITIONED_TABLE ||
+		!inhparent ||
+		!(rel->part_scheme = find_partition_scheme(root, relation)))
+	{
+		rel->partexprs = NULL;
+		rel->part_rels = NULL;
+		rel->part_oids = NULL;
+		return;
+	}
+
+	rel->partexprs = build_baserel_partition_key_exprs(relation, rel->relid);
+	rel->part_oids = RelationGetPartitionDesc(relation)->oids;
+
+	/*
+	 * RelOptInfos of the partitions will be filled in when we build those for
+	 * the child relations.
+	 */
+	rel->part_rels = NULL;
+	return;
+}
+
+/*
+ * find_partition_scheme
+ *
+ * The function returns a canonical partition scheme which exactly matches the
+ * partitioning properties of the given relation if one exists in the of
+ * canonical partitioning schemes maintained in PlannerInfo. If none of the
+ * existing partitioning schemes match, the function creates a canonical
+ * partition scheme and adds it to the list.
+ *
+ * For an unpartitioned table or for a multi-level partitioned table it returns
+ * NULL. See comments in the function for more details.
+ */
+static PartitionScheme
+find_partition_scheme(PlannerInfo *root, Relation relation)
+{
+	PartitionKey	part_key = RelationGetPartitionKey(relation);
+	PartitionDesc	part_desc = RelationGetPartitionDesc(relation);
+	ListCell	   *lc;
+	int		nparts;
+	int		partnatts;
+	int		cnt_pks;
+	int		cnt_parts;
+	PartitionScheme	part_scheme = NULL;
+
+	/* No partition scheme for an unpartitioned relation. */
+	if (!part_desc || !part_key)
+		return NULL;
+
+	nparts = part_desc->nparts;
+	partnatts = part_key->partnatts;
+
+	/*
+	 * For a multi-level partitioned table, we do not retain the partitioning
+	 * hierarchy while expanding RTE for the topmost parent. Thus the number of
+	 * children as per root->append_rel_list does not match the number of
+	 * partitions specified in the partition descriptor and hence the
+	 * partitioning scheme of a multi-partitioned table does not reflect the
+	 * true picture. So for now, treat a multi-partitioned table as not
+	 * partitioned.
+	 */
+	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	{
+		if (has_subclass(part_desc->oids[cnt_parts]))
+			return NULL;
+	}
+
+	/* Search for a matching partition scheme and return if found one. */
+	foreach (lc, root->part_schemes)
+	{
+		part_scheme = lfirst(lc);
+
+		/* Match number of partitions and partitioning strategy. */
+		if (nparts != part_scheme->nparts ||
+			part_key->strategy != part_scheme->strategy ||
+			partnatts != part_scheme->partnatts)
+			continue;
+
+		/* Match the partition key types. */
+		for (cnt_pks = 0; cnt_pks < partnatts; cnt_pks++)
+		{
+			/*
+			 * For types, it suffices to match the type id, mod and collation;
+			 * len, byval and align are depedent on the first two.
+			 */
+			if (part_key->partopfamily[cnt_pks] != part_scheme->partopfamily[cnt_pks] ||
+				part_key->partopcintype[cnt_pks] != part_scheme->partopcintype[cnt_pks] ||
+				part_key->parttypid[cnt_pks] != part_scheme->key_types[cnt_pks] ||
+				part_key->parttypmod[cnt_pks] != part_scheme->key_typmods[cnt_pks] ||
+				part_key->parttypcoll[cnt_pks] != part_scheme->key_collations[cnt_pks])
+				break;
+		}
+
+		/* Some partition key didn't match. Check next partitioning scheme. */
+		if (cnt_pks < partnatts)
+			continue;
+
+		if (!partition_bounds_equal(part_key, part_desc->boundinfo,
+									part_scheme->boundinfo))
+			continue;
+
+		/* Found matching partition scheme. */
+		return part_scheme;
+	}
+
+	/* Did not find matching partition scheme. Create one. */
+	part_scheme = (PartitionScheme) palloc0(sizeof(PartitionSchemeData));
+
+	/* Copy partition bounds/lists. */
+	part_scheme->nparts = part_desc->nparts;
+	part_scheme->strategy = part_key->strategy;
+	part_scheme->boundinfo = part_desc->boundinfo;
+
+	/* Store partition key information. */
+	part_scheme->partnatts = part_key->partnatts;
+
+	part_scheme->partopfamily = (Oid *) palloc(sizeof(Oid) * partnatts);
+	memcpy(part_scheme->partopfamily, part_key->partopfamily,
+		   sizeof(Oid) * partnatts);
+
+	part_scheme->partopcintype = (Oid *) palloc(sizeof(Oid) * partnatts);
+	memcpy(part_scheme->partopcintype, part_key->partopcintype,
+		   sizeof(Oid) * partnatts);
+
+	part_scheme->key_types = (Oid *) palloc(sizeof(Oid) * partnatts);
+	memcpy(part_scheme->key_types, part_key->parttypid,
+		   sizeof(Oid) * partnatts);
+
+	part_scheme->key_typmods = (int32 *) palloc(sizeof(int32) * partnatts);
+	memcpy(part_scheme->key_typmods, part_key->parttypmod,
+		   sizeof(int32) * partnatts);
+
+	part_scheme->key_collations = (Oid *) palloc(sizeof(Oid) * partnatts);
+	memcpy(part_scheme->key_collations, part_key->parttypcoll,
+		   sizeof(Oid) * partnatts);
+
+	/* Add the partitioning scheme to PlannerInfo. */
+	root->part_schemes = lappend(root->part_schemes, part_scheme);
+
+	return part_scheme;
+}
+
+/*
+ * build_baserel_partition_key_exprs
+ *
+ * Collect partition key expressions for a given base relation. The function
+ * converts any single column partition keys into corresponding Var nodes. It
+ * restamps Var nodes in partition key expressions by given varno. The
+ * partition key expressions are returned as an array of single element Lists
+ * to be stored in RelOptInfo of the base relation.
+ */
+static List **
+build_baserel_partition_key_exprs(Relation relation, Index varno)
+{
+	PartitionKey	part_key = RelationGetPartitionKey(relation);
+	int		num_pkexprs;
+	int		cnt_pke;
+	List		  **partexprs;
+	ListCell	   *lc;
+
+	if (!part_key || part_key->partnatts <= 0)
+		return NULL;
+
+	num_pkexprs = part_key->partnatts;
+	partexprs = (List **) palloc(sizeof(List *) * num_pkexprs);
+	lc = list_head(part_key->partexprs);
+
+	for (cnt_pke = 0; cnt_pke < num_pkexprs; cnt_pke++)
+	{
+		AttrNumber attno = part_key->partattrs[cnt_pke];
+		Expr	  *pkexpr;
+
+		if (attno != InvalidAttrNumber)
+		{
+			/* Single column partition key is stored as a Var node. */
+			Form_pg_attribute att_tup;
+
+			if (attno < 0)
+				att_tup = SystemAttributeDefinition(attno,
+												relation->rd_rel->relhasoids);
+			else
+				att_tup = relation->rd_att->attrs[attno - 1];
+
+			pkexpr = (Expr *) makeVar(varno, attno, att_tup->atttypid,
+									  att_tup->atttypmod,
+									  att_tup->attcollation, 0);
+		}
+		else
+		{
+			if (lc == NULL)
+				elog(ERROR, "wrong number of partition key expressions");
+
+			/* Re-stamp the expressions with given varno. */
+			pkexpr = (Expr *) copyObject(lfirst(lc));
+			ChangeVarNodes((Node *) pkexpr, 1, varno, 0);
+			lc = lnext(lc);
+		}
+
+		partexprs[cnt_pke] = list_make1(pkexpr);
+	}
+
+	return partexprs;
+}
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 643be54..4f99184 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 "catalog/partition.h"
 #include "lib/stringinfo.h"
 #include "nodes/params.h"
 #include "nodes/parsenodes.h"
@@ -261,6 +262,9 @@ typedef struct PlannerInfo
 	List	   *distinct_pathkeys;		/* distinctClause pathkeys, if any */
 	List	   *sort_pathkeys;	/* sortClause pathkeys, if any */
 
+	List	   *part_schemes;	/* Canonicalised partition schemes
+								 * used in the query. */
+
 	List	   *initial_rels;	/* RelOptInfos we are now trying to join */
 
 	/* Use fetch_upper_rel() to get any particular upper rel */
@@ -321,6 +325,38 @@ typedef struct PlannerInfo
 	((root)->simple_rte_array ? (root)->simple_rte_array[rti] : \
 	 rt_fetch(rti, (root)->parse->rtable))
 
+/*
+ * Partitioning scheme
+ *		Structure to hold partitioning scheme for a given relation.
+ *
+ * Multiple relations may be partitioned in the same way. The relations
+ * resulting from joining such relations may be partitioned in the same way as
+ * the joining relations. Similarly, relations derived from such relations by
+ * grouping, sorting may be partitioned in the same way as the underlying
+ * scan relations. All such relations partitioned in the same way share the
+ * partitioning scheme.
+ *
+ * PlannerInfo stores a list of distinct "canonical" partitioning schemes.
+ * RelOptInfo of a partitioned relation holds the pointer to "canonical"
+ * partitioning scheme.
+ */
+typedef struct PartitionSchemeData
+{
+	/* Information about partitions */
+	int			nparts;			/* number of partitions */
+	PartitionBoundInfo boundinfo;	/* Partition bounds/lists */
+
+	/* Information about partition keys */
+	char		strategy;		/* partition strategy */
+	int16		partnatts;		/* number of partition attributes */
+	Oid		   *partopfamily;	/* OIDs of operator families */
+	Oid		   *partopcintype;	/* OIDs of opclass declared input data types */
+	Oid		   *key_types;		/* OIDs of partition key data types. */
+	int32	   *key_typmods;	/* typmods of partition keys. */
+	Oid		   *key_collations;	/* OIDs of collations of partition keys. */
+} PartitionSchemeData;
+
+typedef struct PartitionSchemeData *PartitionScheme;
 
 /*----------
  * RelOptInfo
@@ -531,6 +567,7 @@ typedef struct RelOptInfo
 	PlannerInfo *subroot;		/* if subquery */
 	List	   *subplan_params; /* if subquery */
 	int			rel_parallel_workers;	/* wanted number of parallel workers */
+	Oid		   *part_oids;		/* OIDs of partitions */
 
 	/* Information about foreign tables and foreign joins */
 	Oid			serverid;		/* identifies server for the table or join */
@@ -549,6 +586,20 @@ typedef struct RelOptInfo
 	List	   *joininfo;		/* RestrictInfo structures for join clauses
 								 * involving this rel */
 	bool		has_eclass_joins;		/* T means joininfo is incomplete */
+
+	/* For all the partitioned relations. */
+	PartitionScheme	    part_scheme;	/* Partitioning scheme. */
+	struct RelOptInfo **part_rels;		/* Array of RelOptInfos of partitions,
+										 * stored in the same order as bounds
+										 * or lists in PartitionScheme.
+										 */
+	List			  **partexprs;		/* Array of list of partition key
+										 * expressions. For base relations
+										 * these are one element lists. For
+										 * join there may be as many elements
+										 * as the number of joining
+										 * relations.
+										 */
 } RelOptInfo;
 
 /*
-- 
1.7.9.5

0007-Partition-wise-join-tests.patchapplication/octet-stream; name=0007-Partition-wise-join-tests.patchDownload
From 82e4452ff549ede9f34684d363f66291a3bb632b Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Tue, 7 Feb 2017 11:57:30 +0530
Subject: [PATCH 07/11] Partition-wise join tests.

This file does not contain tests for joins between multi-leveled partitioned
tables. Those will be added later.
---
 src/test/regress/expected/partition_join.out | 4114 ++++++++++++++++++++++++++
 src/test/regress/parallel_schedule           |    2 +-
 src/test/regress/serial_schedule             |    1 +
 src/test/regress/sql/partition_join.sql      |  515 ++++
 4 files changed, 4631 insertions(+), 1 deletion(-)
 create mode 100644 src/test/regress/expected/partition_join.out
 create mode 100644 src/test/regress/sql/partition_join.sql

diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
new file mode 100644
index 0000000..18238fa
--- /dev/null
+++ b/src/test/regress/expected/partition_join.out
@@ -0,0 +1,4114 @@
+--
+-- PARTITION_JOIN
+-- Test partition-wise join between partitioned tables
+--
+--
+-- partitioned by a single column
+--
+CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+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, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1 AS SELECT * FROM prt1;
+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_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, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+CREATE TABLE uprt2 AS SELECT * FROM prt2;
+-- inner join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on public.prt2_p1 t2
+                     Output: t2.b, t2.c
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+               Hash Cond: (t2_1.b = t1_2.a)
+               ->  Seq Scan on public.prt2_p2 t2_1
+                     Output: t2_1.b, t2_1.c
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+               Hash Cond: (t2_2.b = t1_1.a)
+               ->  Seq Scan on public.prt2_p3 t2_2
+                     Output: t2_2.b, t2_2.c
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 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)
+
+-- left outer join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on public.prt2_p1 t2
+                     Output: t2.b, t2.c
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+               Hash Cond: (t2_1.b = t1_2.a)
+               ->  Seq Scan on public.prt2_p2 t2_1
+                     Output: t2_1.b, t2_1.c
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+               Hash Cond: (t2_2.b = t1_1.a)
+               ->  Seq Scan on public.prt2_p3 t2_2
+                     Output: t2_2.b, t2_2.c
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+-- right outer join
+EXPLAIN (VERBOSE, 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.b % 25 = 0 ORDER BY t1.a, t2.b;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: (t1.a = t2.b)
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t2.b, t2.c
+                           ->  Seq Scan on public.prt2_p1 t2
+                                 Output: t2.b, t2.c
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Hash Cond: (t1_2.a = t2_1.b)
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c
+                           ->  Seq Scan on public.prt2_p2 t2_1
+                                 Output: t2_1.b, t2_1.c
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Hash Cond: (t1_1.a = t2_2.b)
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c
+                           ->  Seq Scan on public.prt2_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+                                 Filter: ((t2_2.b % 25) = 0)
+(36 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 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)
+
+-- full outer join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+   Sort Key: prt1_p1.a, prt2_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+               Hash Cond: (prt1_p1.a = prt2_p1.b)
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c
+                     Filter: ((prt1_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p1.b, prt2_p1.c
+                     ->  Seq Scan on public.prt2_p1
+                           Output: prt2_p1.b, prt2_p1.c
+                           Filter: ((prt2_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt1_p2.a = prt2_p2.b)
+               ->  Seq Scan on public.prt1_p2
+                     Output: prt1_p2.a, prt1_p2.c
+                     Filter: ((prt1_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p2.b, prt2_p2.c
+                     ->  Seq Scan on public.prt2_p2
+                           Output: prt2_p2.b, prt2_p2.c
+                           Filter: ((prt2_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c
+               Hash Cond: (prt1_p3.a = prt2_p3.b)
+               ->  Seq Scan on public.prt1_p3
+                     Output: prt1_p3.a, prt1_p3.c
+                     Filter: ((prt1_p3.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p3.b, prt2_p3.c
+                     ->  Seq Scan on public.prt2_p3
+                           Output: prt2_p3.b, prt2_p3.c
+                           Filter: ((prt2_p3.b % 25) = 0)
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 t2 WHERE t2.b % 25 = 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)
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+   Sort Key: prt1_p1.a, prt2_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+               Hash Cond: (prt1_p1.a = prt2_p1.b)
+               Filter: (((50) = prt1_p1.b) OR ((75) = prt2_p1.b))
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c, prt1_p1.b, 50
+                     Filter: ((prt1_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p1.b, prt2_p1.c, (75)
+                     ->  Seq Scan on public.prt2_p1
+                           Output: prt2_p1.b, prt2_p1.c, 75
+                           Filter: ((prt2_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt1_p2.a = prt2_p2.b)
+               Filter: (((50) = prt1_p2.b) OR ((75) = prt2_p2.b))
+               ->  Seq Scan on public.prt1_p2
+                     Output: prt1_p2.a, prt1_p2.c, prt1_p2.b, 50
+                     Filter: ((prt1_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p2.b, prt2_p2.c, (75)
+                     ->  Seq Scan on public.prt2_p2
+                           Output: prt2_p2.b, prt2_p2.c, 75
+                           Filter: ((prt2_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c
+               Hash Cond: (prt1_p3.a = prt2_p3.b)
+               Filter: (((50) = prt1_p3.b) OR ((75) = prt2_p3.b))
+               ->  Seq Scan on public.prt1_p3
+                     Output: prt1_p3.a, prt1_p3.c, prt1_p3.b, 50
+                     Filter: ((prt1_p3.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p3.b, prt2_p3.c, (75)
+                     ->  Seq Scan on public.prt2_p3
+                           Output: prt2_p3.b, prt2_p3.c, 75
+                           Filter: ((prt2_p3.b % 25) = 0)
+(40 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+ a  |  c   | b  |  c   
+----+------+----+------
+ 50 | 0050 |    | 
+    |      | 75 | 0075
+(2 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+ a  |  c   | b  |  c   
+----+------+----+------
+ 50 | 0050 |    | 
+    |      | 75 | 0075
+(2 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, (25), prt2_p1.b, prt2_p1.c, (50)
+   Sort Key: prt1_p1.a, prt2_p1.b
+   ->  Result
+         Output: prt1_p1.a, prt1_p1.c, (25), prt2_p1.b, prt2_p1.c, (50)
+         ->  Append
+               ->  Hash Full Join
+                     Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, (25), (50)
+                     Hash Cond: (prt1_p1.a = prt2_p1.b)
+                     ->  Seq Scan on public.prt1_p1
+                           Output: prt1_p1.a, prt1_p1.c, 25
+                           Filter: ((prt1_p1.a % 25) = 0)
+                     ->  Hash
+                           Output: prt2_p1.b, prt2_p1.c, (50)
+                           ->  Seq Scan on public.prt2_p1
+                                 Output: prt2_p1.b, prt2_p1.c, 50
+                                 Filter: ((prt2_p1.b % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c, (25), (50)
+                     Hash Cond: (prt1_p2.a = prt2_p2.b)
+                     ->  Seq Scan on public.prt1_p2
+                           Output: prt1_p2.a, prt1_p2.c, 25
+                           Filter: ((prt1_p2.a % 25) = 0)
+                     ->  Hash
+                           Output: prt2_p2.b, prt2_p2.c, (50)
+                           ->  Seq Scan on public.prt2_p2
+                                 Output: prt2_p2.b, prt2_p2.c, 50
+                                 Filter: ((prt2_p2.b % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c, (25), (50)
+                     Hash Cond: (prt1_p3.a = prt2_p3.b)
+                     ->  Seq Scan on public.prt1_p3
+                           Output: prt1_p3.a, prt1_p3.c, 25
+                           Filter: ((prt1_p3.a % 25) = 0)
+                     ->  Hash
+                           Output: prt2_p3.b, prt2_p3.c, (50)
+                           ->  Seq Scan on public.prt2_p3
+                                 Output: prt2_p3.b, prt2_p3.c, 50
+                                 Filter: ((prt2_p3.b % 25) = 0)
+(39 rows)
+
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 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)
+
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM uprt2 WHERE uprt2.b % 25 = 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)
+
+-- Join with pruned partitions from joining relations
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on public.prt2_p2 t2
+                     Output: t2.b, t2.c
+                     Filter: (t2.b > 250)
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.prt1_p2 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a < 450) AND ((t1.a % 25) = 0))
+(15 rows)
+
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+(1 row)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a < 450 AND t2.b > 250 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+(1 row)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                    QUERY PLAN                                    
+----------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, b, c
+   Sort Key: prt1_p1.a, b
+   ->  Append
+         ->  Nested Loop Left Join
+               Output: prt1_p1.a, prt1_p1.c, b, c
+               Join Filter: (prt1_p1.a = b)
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c
+                     Filter: ((prt1_p1.a < 450) AND ((prt1_p1.a % 25) = 0))
+               ->  Result
+                     Output: b, c
+                     One-Time Filter: false
+         ->  Hash Right Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt2_p2.b = prt1_p2.a)
+               ->  Seq Scan on public.prt2_p2
+                     Output: prt2_p2.b, prt2_p2.c
+                     Filter: (prt2_p2.b > 250)
+               ->  Hash
+                     Output: prt1_p2.a, prt1_p2.c
+                     ->  Seq Scan on public.prt1_p2
+                           Output: prt1_p2.a, prt1_p2.c
+                           Filter: ((prt1_p2.a < 450) AND ((prt1_p2.a % 25) = 0))
+(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.a % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.a % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                       QUERY PLAN                                       
+----------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+   Sort Key: prt1_p2.a, prt2_p2.b
+   ->  Result
+         Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: prt2_p2.b, prt2_p2.c, prt1_p2.a, prt1_p2.c
+                     Hash Cond: (prt1_p2.a = prt2_p2.b)
+                     ->  Seq Scan on public.prt1_p2
+                           Output: prt1_p2.a, prt1_p2.c
+                           Filter: (prt1_p2.a < 450)
+                     ->  Hash
+                           Output: prt2_p2.b, prt2_p2.c
+                           ->  Seq Scan on public.prt2_p2
+                                 Output: prt2_p2.b, prt2_p2.c
+                                 Filter: ((prt2_p2.b > 250) AND ((prt2_p2.a % 25) = 0))
+               ->  Nested Loop Left Join
+                     Output: prt2_p3.b, prt2_p3.c, a, c
+                     Join Filter: (a = prt2_p3.b)
+                     ->  Seq Scan on public.prt2_p3
+                           Output: prt2_p3.b, prt2_p3.c
+                           Filter: ((prt2_p3.b > 250) AND ((prt2_p3.a % 25) = 0))
+                     ->  Result
+                           Output: a, c
+                           One-Time Filter: false
+(26 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+     |      | 375 | 0375
+     |      | 450 | 0450
+     |      | 525 | 0525
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+     |      | 375 | 0375
+     |      | 450 | 0450
+     |      | 525 | 0525
+(4 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+                                    QUERY PLAN                                    
+----------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, b, c
+   Sort Key: prt1_p1.a, b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_p1.a, prt1_p1.c, b, c
+               Hash Cond: (prt1_p1.a = b)
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c
+                     Filter: ((prt1_p1.a < 450) AND ((prt1_p1.a % 25) = 0))
+               ->  Hash
+                     Output: b, c
+                     ->  Result
+                           Output: b, c
+                           One-Time Filter: false
+         ->  Hash Full Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt1_p2.a = prt2_p2.b)
+               ->  Seq Scan on public.prt1_p2
+                     Output: prt1_p2.a, prt1_p2.c
+                     Filter: ((prt1_p2.a < 450) AND ((prt1_p2.a % 25) = 0))
+               ->  Hash
+                     Output: prt2_p2.b, prt2_p2.c
+                     ->  Seq Scan on public.prt2_p2
+                           Output: prt2_p2.b, prt2_p2.c
+                           Filter: ((prt2_p2.b > 250) AND ((prt2_p2.b % 25) = 0))
+         ->  Hash Full Join
+               Output: a, c, prt2_p3.b, prt2_p3.c
+               Hash Cond: (prt2_p3.b = a)
+               ->  Seq Scan on public.prt2_p3
+                     Output: prt2_p3.b, prt2_p3.c
+                     Filter: ((prt2_p3.b > 250) AND ((prt2_p3.b % 25) = 0))
+               ->  Hash
+                     Output: a, c
+                     ->  Result
+                           Output: a, c
+                           One-Time Filter: false
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b 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)
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+                     QUERY PLAN                     
+----------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Output: t1.a, t1.b, t1.c
+         Merge Cond: (t1.a = t1_3.b)
+         ->  Sort
+               Output: t1.a, t1.b, t1.c
+               Sort Key: t1.a
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+         ->  Sort
+               Output: t1_3.b
+               Sort Key: t1_3.b
+               ->  Seq Scan on public.prt2_p1 t1_3
+                     Output: t1_3.b
+                     Filter: ((t1_3.b % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_2.a, t1_2.b, t1_2.c
+         Merge Cond: (t1_2.a = t1_4.b)
+         ->  Sort
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Sort Key: t1_2.a
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+         ->  Sort
+               Output: t1_4.b
+               Sort Key: t1_4.b
+               ->  Seq Scan on public.prt2_p2 t1_4
+                     Output: t1_4.b
+                     Filter: ((t1_4.b % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_1.a, t1_1.b, t1_1.c
+         Merge Cond: (t1_1.a = t1_5.b)
+         ->  Sort
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Sort Key: t1_1.a
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+         ->  Sort
+               Output: t1_5.b
+               Sort Key: t1_5.b
+               ->  Seq Scan on public.prt2_p3 t1_5
+                     Output: t1_5.b
+                     Filter: ((t1_5.b % 25) = 0)
+(47 rows)
+
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+-- lateral reference
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+                                             QUERY PLAN                                              
+-----------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   Sort Key: t1.a
+   ->  Result
+         Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+         ->  Append
+               ->  Nested Loop Left Join
+                     Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a))
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.b, t1.c
+                           Filter: ((t1.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+                           Hash Cond: (t3.b = t2.a)
+                           ->  Seq Scan on public.prt2_p1 t3
+                                 Output: t3.a, t3.b
+                           ->  Hash
+                                 Output: t2.a
+                                 ->  Seq Scan on public.prt1_p1 t2
+                                       Output: t2.a
+                                       Filter: (t1.a = t2.a)
+               ->  Nested Loop Left Join
+                     Output: t1_2.a, t1_2.b, t1_2.c, t2_2.a, t3_1.a, (LEAST(t1_2.a, t2_2.a, t3_1.a))
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.b, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2_2.a, t3_1.a, LEAST(t1_2.a, t2_2.a, t3_1.a)
+                           Hash Cond: (t3_1.b = t2_2.a)
+                           ->  Seq Scan on public.prt2_p2 t3_1
+                                 Output: t3_1.a, t3_1.b
+                           ->  Hash
+                                 Output: t2_2.a
+                                 ->  Seq Scan on public.prt1_p2 t2_2
+                                       Output: t2_2.a
+                                       Filter: (t1_2.a = t2_2.a)
+               ->  Nested Loop Left Join
+                     Output: t1_1.a, t1_1.b, t1_1.c, t2_1.a, t3_2.a, (LEAST(t1_1.a, t2_1.a, t3_2.a))
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.b, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2_1.a, t3_2.a, LEAST(t1_1.a, t2_1.a, t3_2.a)
+                           Hash Cond: (t3_2.b = t2_1.a)
+                           ->  Seq Scan on public.prt2_p3 t3_2
+                                 Output: t3_2.a, t3_2.b
+                           ->  Hash
+                                 Output: t2_1.a
+                                 ->  Seq Scan on public.prt1_p3 t2_1
+                                       Output: t2_1.a
+                                       Filter: (t1_1.a = t2_1.a)
+(51 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   Sort Key: t1.a
+   ->  Nested Loop Left Join
+         Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+         ->  Append
+               ->  Seq Scan on public.prt1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Seq Scan on public.prt1_p1 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Seq Scan on public.prt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Seq Scan on public.prt1_p2 t1_3
+                     Output: t1_3.a, t1_3.b, t1_3.c
+                     Filter: ((t1_3.a % 25) = 0)
+         ->  Append
+               ->  Hash Join
+                     Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+                     Hash Cond: (t3.b = t2.a)
+                     ->  Seq Scan on public.prt2_p1 t3
+                           Output: t3.a, t3.b
+                     ->  Hash
+                           Output: t2.a
+                           ->  Seq Scan on public.prt1_p1 t2
+                                 Output: t2.a
+                                 Filter: (t1.b = t2.a)
+               ->  Hash Join
+                     Output: t2_2.a, t3_1.a, LEAST(t1.a, t2_2.a, t3_1.a)
+                     Hash Cond: (t3_1.b = t2_2.a)
+                     ->  Seq Scan on public.prt2_p2 t3_1
+                           Output: t3_1.a, t3_1.b
+                     ->  Hash
+                           Output: t2_2.a
+                           ->  Seq Scan on public.prt1_p2 t2_2
+                                 Output: t2_2.a
+                                 Filter: (t1.b = t2_2.a)
+               ->  Hash Join
+                     Output: t2_1.a, t3_2.a, LEAST(t1.a, t2_1.a, t3_2.a)
+                     Hash Cond: (t3_2.b = t2_1.a)
+                     ->  Seq Scan on public.prt2_p3 t3_2
+                           Output: t3_2.a, t3_2.b
+                     ->  Hash
+                           Output: t2_1.a
+                           ->  Seq Scan on public.prt1_p3 t2_1
+                                 Output: t2_1.a
+                                 Filter: (t1.b = t2_1.a)
+(49 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+--
+-- partitioned by expression
+--
+CREATE TABLE prt1_e (a int, b int, c varchar) PARTITION BY RANGE(((a + b)/2));
+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);
+INSERT INTO prt1_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_e AS SELECT * FROM prt1_e;
+CREATE TABLE prt2_e (a int, b int, c varchar) PARTITION BY RANGE(((b + a)/2));
+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);
+INSERT INTO prt2_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_e;
+ANALYZE prt2_e_p1;
+ANALYZE prt2_e_p2;
+ANALYZE prt2_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_e AS SELECT * FROM prt2_e;
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                  QUERY PLAN                                  
+------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_e_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_e_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
+               ->  Seq Scan on public.prt2_e_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_e_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+(34 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.a % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1, uprt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.a % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                  QUERY PLAN                                  
+------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_e_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_e_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
+               ->  Seq Scan on public.prt2_e_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_e_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1 LEFT JOIN uprt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on public.prt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = t1.a)
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_2.a)
+                     ->  Seq Scan on public.prt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                           ->  Hash Join
+                                 Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = t1_2.a)
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_1.a)
+                     ->  Seq Scan on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                           ->  Hash Join
+                                 Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = t1_1.a)
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+(57 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM uprt1 t1, uprt2 t2, uprt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on public.prt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = t1.a)
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_2.a)
+                     ->  Seq Scan on public.prt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = t1_2.a)
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_1.a)
+                     ->  Seq Scan on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = t1_1.a)
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+(57 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, 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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: (((t3.a + t3.b) / 2) = t2.b)
+                     ->  Seq Scan on public.prt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = t1.a)
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: (((t3_1.a + t3_1.b) / 2) = t2_1.b)
+                     ->  Seq Scan on public.prt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = t1_2.a)
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: (((t3_2.a + t3_2.b) / 2) = t2_2.b)
+                     ->  Seq Scan on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = t1_1.a)
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+(57 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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |          | 
+ 100 | 0100 |     |      |          | 
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |          | 
+ 250 | 0250 |     |      |          | 
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |          | 
+ 400 | 0400 |     |      |          | 
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |          | 
+ 550 | 0550 |     |      |          | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |          | 
+ 100 | 0100 |     |      |          | 
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |          | 
+ 250 | 0250 |     |      |          | 
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |          | 
+ 400 | 0400 |     |      |          | 
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |          | 
+ 550 | 0550 |     |      |          | 
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Hash Cond: (t2.b = t1.a)
+                     ->  Seq Scan on public.prt2_p1 t2
+                           Output: t2.b, t2.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Hash Cond: (t1.a = ((t3.a + t3.b) / 2))
+                                 ->  Seq Scan on public.prt1_p1 t1
+                                       Output: t1.a, t1.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                     Hash Cond: (t2_1.b = t1_2.a)
+                     ->  Seq Scan on public.prt2_p2 t2_1
+                           Output: t2_1.b, t2_1.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                                 Hash Cond: (t1_2.a = ((t3_1.a + t3_1.b) / 2))
+                                 ->  Seq Scan on public.prt1_p2 t1_2
+                                       Output: t1_2.a, t1_2.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                     Hash Cond: (t2_2.b = t1_1.a)
+                     ->  Seq Scan on public.prt2_p3 t2_2
+                           Output: t2_2.b, t2_2.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                                 Hash Cond: (t1_1.a = ((t3_2.a + t3_2.b) / 2))
+                                 ->  Seq Scan on public.prt1_p3 t1_1
+                                       Output: t1_1.a, t1_1.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: (t1.a = t2.b)
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = ((t3.a + t3.b) / 2))
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Hash Cond: (t1_2.a = t2_1.b)
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = ((t3_1.a + t3_1.b) / 2))
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Hash Cond: (t1_1.a = t2_2.b)
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = ((t3_2.a + t3_2.b) / 2))
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                                  QUERY PLAN                                                   
+---------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, ((prt1_e_p1.a + prt1_e_p1.b)), prt1_e_p1.c
+   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   ->  Result
+         Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, (prt1_e_p1.a + prt1_e_p1.b), prt1_e_p1.c
+         ->  Append
+               ->  Hash Full Join
+                     Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, prt1_e_p1.a, prt1_e_p1.b, prt1_e_p1.c
+                     Hash Cond: (prt1_p1.a = ((prt1_e_p1.a + prt1_e_p1.b) / 2))
+                     ->  Hash Full Join
+                           Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on public.prt1_p1
+                                 Output: prt1_p1.a, prt1_p1.c
+                                 Filter: ((prt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p1.b, prt2_p1.c
+                                 ->  Seq Scan on public.prt2_p1
+                                       Output: prt2_p1.b, prt2_p1.c
+                                       Filter: ((prt2_p1.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p1.a, prt1_e_p1.b, prt1_e_p1.c
+                           ->  Seq Scan on public.prt1_e_p1
+                                 Output: prt1_e_p1.a, prt1_e_p1.b, prt1_e_p1.c
+                                 Filter: ((prt1_e_p1.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c, prt1_e_p2.a, prt1_e_p2.b, prt1_e_p2.c
+                     Hash Cond: (prt1_p2.a = ((prt1_e_p2.a + prt1_e_p2.b) / 2))
+                     ->  Hash Full Join
+                           Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+                           Hash Cond: (prt1_p2.a = prt2_p2.b)
+                           ->  Seq Scan on public.prt1_p2
+                                 Output: prt1_p2.a, prt1_p2.c
+                                 Filter: ((prt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p2.b, prt2_p2.c
+                                 ->  Seq Scan on public.prt2_p2
+                                       Output: prt2_p2.b, prt2_p2.c
+                                       Filter: ((prt2_p2.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p2.a, prt1_e_p2.b, prt1_e_p2.c
+                           ->  Seq Scan on public.prt1_e_p2
+                                 Output: prt1_e_p2.a, prt1_e_p2.b, prt1_e_p2.c
+                                 Filter: ((prt1_e_p2.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c, prt1_e_p3.a, prt1_e_p3.b, prt1_e_p3.c
+                     Hash Cond: (prt1_p3.a = ((prt1_e_p3.a + prt1_e_p3.b) / 2))
+                     ->  Hash Full Join
+                           Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c
+                           Hash Cond: (prt1_p3.a = prt2_p3.b)
+                           ->  Seq Scan on public.prt1_p3
+                                 Output: prt1_p3.a, prt1_p3.c
+                                 Filter: ((prt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p3.b, prt2_p3.c
+                                 ->  Seq Scan on public.prt2_p3
+                                       Output: prt2_p3.b, prt2_p3.c
+                                       Filter: ((prt2_p3.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p3.a, prt1_e_p3.b, prt1_e_p3.c
+                           ->  Seq Scan on public.prt1_e_p3
+                                 Output: prt1_e_p3.a, prt1_e_p3.b, prt1_e_p3.c
+                                 Filter: ((prt1_e_p3.a % 25) = 0)
+(63 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+     |      |  75 | 0075 |          | 
+     |      | 225 | 0225 |          | 
+     |      | 375 | 0375 |          | 
+     |      | 525 | 0525 |          | 
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM uprt1_e WHERE uprt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+     |      |  75 | 0075 |          | 
+     |      | 225 | 0225 |          | 
+     |      | 375 | 0375 |          | 
+     |      | 525 | 0525 |          | 
+(16 rows)
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, 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.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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                                                      
+----------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, (50), prt2_p1.b, (75), ((prt1_e_p1.a + prt1_e_p1.b)), (50)
+   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   ->  Result
+         Output: prt1_p1.a, (50), prt2_p1.b, (75), (prt1_e_p1.a + prt1_e_p1.b), (50)
+         ->  Append
+               ->  Hash Full Join
+                     Output: prt1_p1.a, prt2_p1.b, prt1_e_p1.a, prt1_e_p1.b, (50), (75), (50)
+                     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
+                           Output: prt1_p1.a, prt2_p1.b, (50), (75)
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on public.prt1_p1
+                                 Output: prt1_p1.a, 50
+                                 Filter: ((prt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p1.b, (75)
+                                 ->  Seq Scan on public.prt2_p1
+                                       Output: prt2_p1.b, 75
+                                       Filter: ((prt2_p1.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p1.a, prt1_e_p1.b, (50)
+                           ->  Seq Scan on public.prt1_e_p1
+                                 Output: prt1_e_p1.a, prt1_e_p1.b, 50
+                                 Filter: ((prt1_e_p1.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p2.a, prt2_p2.b, prt1_e_p2.a, prt1_e_p2.b, (50), (75), (50)
+                     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
+                           Output: prt1_p2.a, prt2_p2.b, (50), (75)
+                           Hash Cond: (prt1_p2.a = prt2_p2.b)
+                           ->  Seq Scan on public.prt1_p2
+                                 Output: prt1_p2.a, 50
+                                 Filter: ((prt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p2.b, (75)
+                                 ->  Seq Scan on public.prt2_p2
+                                       Output: prt2_p2.b, 75
+                                       Filter: ((prt2_p2.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p2.a, prt1_e_p2.b, (50)
+                           ->  Seq Scan on public.prt1_e_p2
+                                 Output: prt1_e_p2.a, prt1_e_p2.b, 50
+                                 Filter: ((prt1_e_p2.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p3.a, prt2_p3.b, prt1_e_p3.a, prt1_e_p3.b, (50), (75), (50)
+                     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
+                           Output: prt1_p3.a, prt2_p3.b, (50), (75)
+                           Hash Cond: (prt1_p3.a = prt2_p3.b)
+                           ->  Seq Scan on public.prt1_p3
+                                 Output: prt1_p3.a, 50
+                                 Filter: ((prt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p3.b, (75)
+                                 ->  Seq Scan on public.prt2_p3
+                                       Output: prt2_p3.b, 75
+                                       Filter: ((prt2_p3.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p3.a, prt1_e_p3.b, (50)
+                           ->  Seq Scan on public.prt1_e_p3
+                                 Output: prt1_e_p3.a, prt1_e_p3.b, 50
+                                 Filter: ((prt1_e_p3.a % 25) = 0)
+(66 rows)
+
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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)
+
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM uprt1_e WHERE uprt1_e.a % 25 = 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)
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+                            QUERY PLAN                             
+-------------------------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Output: t1.a, t1.b, t1.c
+         Merge Cond: (t1.a = t1_3.b)
+         ->  Sort
+               Output: t1.a, t1.b, t1.c
+               Sort Key: t1.a
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+         ->  Sort
+               Output: t1_3.b, t2.a, t2.b
+               Sort Key: t1_3.b
+               ->  Hash Join
+                     Output: t1_3.b, t2.a, t2.b
+                     Hash Cond: (((t2.a + t2.b) / 2) = t1_3.b)
+                     ->  Seq Scan on public.prt1_e_p1 t2
+                           Output: t2.a, t2.b
+                     ->  Hash
+                           Output: t1_3.b
+                           ->  Seq Scan on public.prt2_p1 t1_3
+                                 Output: t1_3.b
+                                 Filter: ((t1_3.b % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_2.a, t1_2.b, t1_2.c
+         Merge Cond: (t1_2.a = t1_4.b)
+         ->  Sort
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Sort Key: t1_2.a
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+         ->  Sort
+               Output: t1_4.b, t2_1.a, t2_1.b
+               Sort Key: t1_4.b
+               ->  Hash Join
+                     Output: t1_4.b, t2_1.a, t2_1.b
+                     Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_4.b)
+                     ->  Seq Scan on public.prt1_e_p2 t2_1
+                           Output: t2_1.a, t2_1.b
+                     ->  Hash
+                           Output: t1_4.b
+                           ->  Seq Scan on public.prt2_p2 t1_4
+                                 Output: t1_4.b
+                                 Filter: ((t1_4.b % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_1.a, t1_1.b, t1_1.c
+         Merge Cond: (t1_1.a = t1_5.b)
+         ->  Sort
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Sort Key: t1_1.a
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+         ->  Sort
+               Output: t1_5.b, t2_2.a, t2_2.b
+               Sort Key: t1_5.b
+               ->  Hash Join
+                     Output: t1_5.b, t2_2.a, t2_2.b
+                     Hash Cond: (((t2_2.a + t2_2.b) / 2) = t1_5.b)
+                     ->  Seq Scan on public.prt1_e_p3 t2_2
+                           Output: t2_2.a, t2_2.b
+                     ->  Hash
+                           Output: t1_5.b
+                           ->  Seq Scan on public.prt2_p3 t1_5
+                                 Output: t1_5.b
+                                 Filter: ((t1_5.b % 25) = 0)
+(68 rows)
+
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1, uprt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+                            QUERY PLAN                             
+-------------------------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Output: t1.a, t1.b, t1.c
+         Merge Cond: (t1.a = t1_3.b)
+         ->  Sort
+               Output: t1.a, t1.b, t1.c
+               Sort Key: t1.a
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+         ->  Sort
+               Output: t1_3.b, t1_6.a, t1_6.b
+               Sort Key: t1_3.b
+               ->  Hash Semi Join
+                     Output: t1_3.b, t1_6.a, t1_6.b
+                     Hash Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
+                     ->  Seq Scan on public.prt2_p1 t1_3
+                           Output: t1_3.b
+                     ->  Hash
+                           Output: t1_6.a, t1_6.b
+                           ->  Seq Scan on public.prt1_e_p1 t1_6
+                                 Output: t1_6.a, t1_6.b
+                                 Filter: ((t1_6.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_2.a, t1_2.b, t1_2.c
+         Merge Cond: (t1_2.a = t1_4.b)
+         ->  Sort
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Sort Key: t1_2.a
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+         ->  Sort
+               Output: t1_4.b, t1_7.a, t1_7.b
+               Sort Key: t1_4.b
+               ->  Hash Semi Join
+                     Output: t1_4.b, t1_7.a, t1_7.b
+                     Hash Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
+                     ->  Seq Scan on public.prt2_p2 t1_4
+                           Output: t1_4.b
+                     ->  Hash
+                           Output: t1_7.a, t1_7.b
+                           ->  Seq Scan on public.prt1_e_p2 t1_7
+                                 Output: t1_7.a, t1_7.b
+                                 Filter: ((t1_7.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_1.a, t1_1.b, t1_1.c
+         Merge Cond: (t1_1.a = t1_5.b)
+         ->  Sort
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Sort Key: t1_1.a
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+         ->  Sort
+               Output: t1_5.b, t1_8.a, t1_8.b
+               Sort Key: t1_5.b
+               ->  Hash Semi Join
+                     Output: t1_5.b, t1_8.a, t1_8.b
+                     Hash Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
+                     ->  Seq Scan on public.prt2_p3 t1_5
+                           Output: t1_5.b
+                     ->  Hash
+                           Output: t1_8.a, t1_8.b
+                           ->  Seq Scan on public.prt1_e_p3 t1_8
+                                 Output: t1_8.a, t1_8.b
+                                 Filter: ((t1_8.a % 25) = 0)
+(68 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+-- test merge joins with and without using indexes
+SET enable_hashjoin TO off;
+SET enable_nestloop TO off;
+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 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 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));
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                             QUERY PLAN                                              
+-----------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Merge Cond: (t2.b = t1.a)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           Sort Key: t2.b
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Merge Cond: ((((t3.a + t3.b) / 2)) = t2.b)
+                                 ->  Sort
+                                       Output: t3.a, t3.b, t3.c, (((t3.a + t3.b) / 2))
+                                       Sort Key: (((t3.a + t3.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c, ((t3.a + t3.b) / 2)
+                                             Filter: ((t3.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2.b, t2.c
+                                       Sort Key: t2.b
+                                       ->  Seq Scan on public.prt2_p1 t2
+                                             Output: t2.b, t2.c
+                     ->  Sort
+                           Output: t1.a, t1.c
+                           Sort Key: t1.a
+                           ->  Seq Scan on public.prt1_p1 t1
+                                 Output: t1.a, t1.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Merge Cond: (t2_1.b = t1_2.a)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           Sort Key: t2_1.b
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t2_1.b)
+                                 ->  Sort
+                                       Output: t3_1.a, t3_1.b, t3_1.c, (((t3_1.a + t3_1.b) / 2))
+                                       Sort Key: (((t3_1.a + t3_1.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c, ((t3_1.a + t3_1.b) / 2)
+                                             Filter: ((t3_1.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2_1.b, t2_1.c
+                                       Sort Key: t2_1.b
+                                       ->  Seq Scan on public.prt2_p2 t2_1
+                                             Output: t2_1.b, t2_1.c
+                     ->  Sort
+                           Output: t1_2.a, t1_2.c
+                           Sort Key: t1_2.a
+                           ->  Seq Scan on public.prt1_p2 t1_2
+                                 Output: t1_2.a, t1_2.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Merge Cond: (t2_2.b = t1_1.a)
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           Sort Key: t2_2.b
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t2_2.b)
+                                 ->  Sort
+                                       Output: t3_2.a, t3_2.b, t3_2.c, (((t3_2.a + t3_2.b) / 2))
+                                       Sort Key: (((t3_2.a + t3_2.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c, ((t3_2.a + t3_2.b) / 2)
+                                             Filter: ((t3_2.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2_2.b, t2_2.c
+                                       Sort Key: t2_2.b
+                                       ->  Seq Scan on public.prt2_p3 t2_2
+                                             Output: t2_2.b, t2_2.c
+                     ->  Sort
+                           Output: t1_1.a, t1_1.c
+                           Sort Key: t1_1.a
+                           ->  Seq Scan on public.prt1_p3 t1_1
+                                 Output: t1_1.a, t1_1.c
+(81 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+                                   QUERY PLAN                                    
+---------------------------------------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Output: t1.a, t1.b, t1.c
+         Merge Cond: (t1.a = t1_3.b)
+         ->  Sort
+               Output: t1.a, t1.b, t1.c
+               Sort Key: t1.a
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_3.b, t1_6.a, t1_6.b
+               ->  Merge Semi Join
+                     Output: t1_3.b, t1_6.a, t1_6.b
+                     Merge Cond: (t1_3.b = (((t1_6.a + t1_6.b) / 2)))
+                     ->  Sort
+                           Output: t1_3.b
+                           Sort Key: t1_3.b
+                           ->  Seq Scan on public.prt2_p1 t1_3
+                                 Output: t1_3.b
+                     ->  Sort
+                           Output: t1_6.a, t1_6.b, (((t1_6.a + t1_6.b) / 2))
+                           Sort Key: (((t1_6.a + t1_6.b) / 2))
+                           ->  Seq Scan on public.prt1_e_p1 t1_6
+                                 Output: t1_6.a, t1_6.b, ((t1_6.a + t1_6.b) / 2)
+                                 Filter: ((t1_6.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_2.a, t1_2.b, t1_2.c
+         Merge Cond: (t1_2.a = t1_4.b)
+         ->  Sort
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Sort Key: t1_2.a
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+         ->  Materialize
+               Output: t1_4.b, t1_7.a, t1_7.b
+               ->  Merge Semi Join
+                     Output: t1_4.b, t1_7.a, t1_7.b
+                     Merge Cond: (t1_4.b = (((t1_7.a + t1_7.b) / 2)))
+                     ->  Sort
+                           Output: t1_4.b
+                           Sort Key: t1_4.b
+                           ->  Seq Scan on public.prt2_p2 t1_4
+                                 Output: t1_4.b
+                     ->  Sort
+                           Output: t1_7.a, t1_7.b, (((t1_7.a + t1_7.b) / 2))
+                           Sort Key: (((t1_7.a + t1_7.b) / 2))
+                           ->  Seq Scan on public.prt1_e_p2 t1_7
+                                 Output: t1_7.a, t1_7.b, ((t1_7.a + t1_7.b) / 2)
+                                 Filter: ((t1_7.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_1.a, t1_1.b, t1_1.c
+         Merge Cond: (t1_1.a = t1_5.b)
+         ->  Sort
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Sort Key: t1_1.a
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_5.b, t1_8.a, t1_8.b
+               ->  Merge Semi Join
+                     Output: t1_5.b, t1_8.a, t1_8.b
+                     Merge Cond: (t1_5.b = (((t1_8.a + t1_8.b) / 2)))
+                     ->  Sort
+                           Output: t1_5.b
+                           Sort Key: t1_5.b
+                           ->  Seq Scan on public.prt2_p3 t1_5
+                                 Output: t1_5.b
+                     ->  Sort
+                           Output: t1_8.a, t1_8.b, (((t1_8.a + t1_8.b) / 2))
+                           Sort Key: (((t1_8.a + t1_8.b) / 2))
+                           ->  Seq Scan on public.prt1_e_p3 t1_8
+                                 Output: t1_8.a, t1_8.b, ((t1_8.a + t1_8.b) / 2)
+                                 Filter: ((t1_8.a % 25) = 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                             QUERY PLAN                                              
+-----------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Merge Cond: (t1.a = t2.b)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           Sort Key: t1.a
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Merge Cond: ((((t3.a + t3.b) / 2)) = t1.a)
+                                 ->  Sort
+                                       Output: t3.a, t3.b, t3.c, (((t3.a + t3.b) / 2))
+                                       Sort Key: (((t3.a + t3.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c, ((t3.a + t3.b) / 2)
+                                             Filter: ((t3.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t1.a, t1.c
+                                       Sort Key: t1.a
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                     ->  Sort
+                           Output: t2.b, t2.c
+                           Sort Key: t2.b
+                           ->  Seq Scan on public.prt2_p1 t2
+                                 Output: t2.b, t2.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                     Merge Cond: (t1_2.a = t2_1.b)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                           Sort Key: t1_2.a
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                                 Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t1_2.a)
+                                 ->  Sort
+                                       Output: t3_1.a, t3_1.b, t3_1.c, (((t3_1.a + t3_1.b) / 2))
+                                       Sort Key: (((t3_1.a + t3_1.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c, ((t3_1.a + t3_1.b) / 2)
+                                             Filter: ((t3_1.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t1_2.a, t1_2.c
+                                       Sort Key: t1_2.a
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                     ->  Sort
+                           Output: t2_1.b, t2_1.c
+                           Sort Key: t2_1.b
+                           ->  Seq Scan on public.prt2_p2 t2_1
+                                 Output: t2_1.b, t2_1.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                     Merge Cond: (t1_1.a = t2_2.b)
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                           Sort Key: t1_1.a
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                                 Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t1_1.a)
+                                 ->  Sort
+                                       Output: t3_2.a, t3_2.b, t3_2.c, (((t3_2.a + t3_2.b) / 2))
+                                       Sort Key: (((t3_2.a + t3_2.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c, ((t3_2.a + t3_2.b) / 2)
+                                             Filter: ((t3_2.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t1_1.a, t1_1.c
+                                       Sort Key: t1_1.a
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                     ->  Sort
+                           Output: t2_2.b, t2_2.c
+                           Sort Key: t2_2.b
+                           ->  Seq Scan on public.prt2_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+(81 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SET enable_seqscan TO off;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                          QUERY PLAN                                          
+----------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Merge Cond: (t2.b = t1.a)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           Sort Key: t2.b
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Merge Cond: (((t3.a + t3.b) / 2) = t2.b)
+                                 ->  Index Scan using iprt1_e_p1_ab2 on public.prt1_e_p1 t3
+                                       Output: t3.a, t3.b, t3.c
+                                       Filter: ((t3.a % 25) = 0)
+                                 ->  Index Scan using iprt2_p1_b on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                     ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Merge Cond: (t2_1.b = t1_2.a)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           Sort Key: t2_1.b
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Merge Cond: (((t3_1.a + t3_1.b) / 2) = t2_1.b)
+                                 ->  Index Scan using iprt1_e_p2_ab2 on public.prt1_e_p2 t3_1
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       Filter: ((t3_1.a % 25) = 0)
+                                 ->  Index Scan using iprt2_p2_b on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                     ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+               ->  Merge Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Merge Cond: (t2_2.b = ((t3_2.a + t3_2.b) / 2))
+                     ->  Merge Left Join
+                           Output: t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                           Merge Cond: (t2_2.b = t1_1.a)
+                           ->  Index Scan using iprt2_p3_b on public.prt2_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+                           ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_1
+                                 Output: t1_1.a, t1_1.c
+                     ->  Index Scan using iprt1_e_p3_ab2 on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                           Filter: ((t3_2.a % 25) = 0)
+(51 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+                                    QUERY PLAN                                    
+----------------------------------------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Output: t1.a, t1.b, t1.c
+         Merge Cond: (t1.a = t1_3.b)
+         ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1
+               Output: t1.a, t1.b, t1.c
+               Filter: ((t1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_3.b, t1_6.a, t1_6.b
+               ->  Merge Semi Join
+                     Output: t1_3.b, t1_6.a, t1_6.b
+                     Merge Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
+                     ->  Index Only Scan using iprt2_p1_b on public.prt2_p1 t1_3
+                           Output: t1_3.b
+                     ->  Index Scan using iprt1_e_p1_ab2 on public.prt1_e_p1 t1_6
+                           Output: t1_6.a, t1_6.b
+                           Filter: ((t1_6.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_2.a, t1_2.b, t1_2.c
+         Merge Cond: (t1_2.a = t1_4.b)
+         ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_2
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Filter: ((t1_2.a % 25) = 0)
+         ->  Materialize
+               Output: t1_4.b, t1_7.a, t1_7.b
+               ->  Merge Semi Join
+                     Output: t1_4.b, t1_7.a, t1_7.b
+                     Merge Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
+                     ->  Index Only Scan using iprt2_p2_b on public.prt2_p2 t1_4
+                           Output: t1_4.b
+                     ->  Index Scan using iprt1_e_p2_ab2 on public.prt1_e_p2 t1_7
+                           Output: t1_7.a, t1_7.b
+                           Filter: ((t1_7.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_1.a, t1_1.b, t1_1.c
+         Merge Cond: (t1_1.a = t1_5.b)
+         ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_1
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Filter: ((t1_1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_5.b, t1_8.a, t1_8.b
+               ->  Merge Semi Join
+                     Output: t1_5.b, t1_8.a, t1_8.b
+                     Merge Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
+                     ->  Index Only Scan using iprt2_p3_b on public.prt2_p3 t1_5
+                           Output: t1_5.b
+                     ->  Index Scan using iprt1_e_p3_ab2 on public.prt1_e_p3 t1_8
+                           Output: t1_8.a, t1_8.b
+                           Filter: ((t1_8.a % 25) = 0)
+(50 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                          QUERY PLAN                                          
+----------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Merge Cond: (t1.a = t2.b)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           Sort Key: t1.a
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Merge Cond: (((t3.a + t3.b) / 2) = t1.a)
+                                 ->  Index Scan using iprt1_e_p1_ab2 on public.prt1_e_p1 t3
+                                       Output: t3.a, t3.b, t3.c
+                                       Filter: ((t3.a % 25) = 0)
+                                 ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1
+                                       Output: t1.a, t1.c
+                     ->  Index Scan using iprt2_p1_b on public.prt2_p1 t2
+                           Output: t2.b, t2.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                     Merge Cond: (t1_2.a = t2_1.b)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                           Sort Key: t1_2.a
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                                 Merge Cond: (((t3_1.a + t3_1.b) / 2) = t1_2.a)
+                                 ->  Index Scan using iprt1_e_p2_ab2 on public.prt1_e_p2 t3_1
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       Filter: ((t3_1.a % 25) = 0)
+                                 ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_2
+                                       Output: t1_2.a, t1_2.c
+                     ->  Index Scan using iprt2_p2_b on public.prt2_p2 t2_1
+                           Output: t2_1.b, t2_1.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                     Merge Cond: (t1_1.a = t2_2.b)
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                           Sort Key: t1_1.a
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                                 Merge Cond: (((t3_2.a + t3_2.b) / 2) = t1_1.a)
+                                 ->  Index Scan using iprt1_e_p3_ab2 on public.prt1_e_p3 t3_2
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       Filter: ((t3_2.a % 25) = 0)
+                                 ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_1
+                                       Output: t1_1.a, t1_1.c
+                     ->  Index Scan using iprt2_p3_b on public.prt2_p3 t2_2
+                           Output: t2_2.b, t2_2.c
+(54 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+-- lateral references and parameterized paths
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Result
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   ->  Merge Append
+         Sort Key: t1.a
+         ->  Nested Loop Left Join
+               Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a))
+               ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Merge Join
+                     Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+                     Merge Cond: (t2.a = t3.b)
+                     ->  Index Only Scan using iprt1_p1_a on public.prt1_p1 t2
+                           Output: t2.a
+                           Index Cond: (t2.a = t1.a)
+                     ->  Index Scan using iprt2_p1_b on public.prt2_p1 t3
+                           Output: t3.a, t3.b
+         ->  Nested Loop Left Join
+               Output: t1_2.a, t1_2.b, t1_2.c, t2_2.a, t3_1.a, (LEAST(t1_2.a, t2_2.a, t3_1.a))
+               ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Merge Join
+                     Output: t2_2.a, t3_1.a, LEAST(t1_2.a, t2_2.a, t3_1.a)
+                     Merge Cond: (t2_2.a = t3_1.b)
+                     ->  Index Only Scan using iprt1_p2_a on public.prt1_p2 t2_2
+                           Output: t2_2.a
+                           Index Cond: (t2_2.a = t1_2.a)
+                     ->  Index Scan using iprt2_p2_b on public.prt2_p2 t3_1
+                           Output: t3_1.a, t3_1.b
+         ->  Nested Loop Left Join
+               Output: t1_1.a, t1_1.b, t1_1.c, t2_1.a, t3_2.a, (LEAST(t1_1.a, t2_1.a, t3_2.a))
+               ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Merge Join
+                     Output: t2_1.a, t3_2.a, LEAST(t1_1.a, t2_1.a, t3_2.a)
+                     Merge Cond: (t2_1.a = t3_2.b)
+                     ->  Index Only Scan using iprt1_p3_a on public.prt1_p3 t2_1
+                           Output: t2_1.a
+                           Index Cond: (t2_1.a = t1_1.a)
+                     ->  Index Scan using iprt2_p3_b on public.prt2_p3 t3_2
+                           Output: t3_2.a, t3_2.b
+(43 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
+ Nested Loop Left Join
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   ->  Merge Append
+         Sort Key: t1.a
+         ->  Sort
+               Output: t1.a, t1.b, t1.c
+               Sort Key: t1.a
+               ->  Seq Scan on public.prt1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+         ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1_1
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Filter: ((t1_1.a % 25) = 0)
+         ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_2
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Filter: ((t1_2.a % 25) = 0)
+         ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_3
+               Output: t1_3.a, t1_3.b, t1_3.c
+               Filter: ((t1_3.a % 25) = 0)
+   ->  Append
+         ->  Merge Join
+               Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+               Merge Cond: (t2.a = t3.b)
+               ->  Index Only Scan using iprt1_p1_a on public.prt1_p1 t2
+                     Output: t2.a
+                     Index Cond: (t2.a = t1.b)
+               ->  Index Scan using iprt2_p1_b on public.prt2_p1 t3
+                     Output: t3.a, t3.b
+         ->  Merge Join
+               Output: t2_2.a, t3_1.a, LEAST(t1.a, t2_2.a, t3_1.a)
+               Merge Cond: (t2_2.a = t3_1.b)
+               ->  Index Only Scan using iprt1_p2_a on public.prt1_p2 t2_2
+                     Output: t2_2.a
+                     Index Cond: (t2_2.a = t1.b)
+               ->  Index Scan using iprt2_p2_b on public.prt2_p2 t3_1
+                     Output: t3_1.a, t3_1.b
+         ->  Merge Join
+               Output: t2_1.a, t3_2.a, LEAST(t1.a, t2_1.a, t3_2.a)
+               Merge Cond: (t2_1.a = t3_2.b)
+               ->  Index Only Scan using iprt1_p3_a on public.prt1_p3 t2_1
+                     Output: t2_1.a
+                     Index Cond: (t2_1.a = t1.b)
+               ->  Index Scan using iprt2_p3_b on public.prt2_p3 t3_2
+                     Output: t3_2.a, t3_2.b
+(44 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+RESET enable_hashjoin;
+RESET enable_nestloop;
+RESET enable_seqscan;
+--
+-- partitioned by multiple columns
+--
+CREATE TABLE prt1_m (a int, b int, c varchar) 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, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_m;
+ANALYZE prt1_m_p1;
+ANALYZE prt1_m_p2;
+ANALYZE prt1_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_m AS SELECT * FROM prt1_m;
+CREATE TABLE prt2_m (a int, b int, c varchar) 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, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_m;
+ANALYZE prt2_m_p1;
+ANALYZE prt2_m_p2;
+ANALYZE prt2_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_m AS SELECT * FROM prt2_m;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+                                                 QUERY PLAN                                                 
+------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: ((((t1.a + t1.b) / 2) = t2.b) AND (t1.a = ((t2.b + t2.a) / 2)))
+                     ->  Seq Scan on public.prt1_m_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                     ->  Hash
+                           Output: t2.b, t2.c, t2.a
+                           ->  Seq Scan on public.prt2_m_p1 t2
+                                 Output: t2.b, t2.c, t2.a
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: ((((t1_1.a + t1_1.b) / 2) = t2_1.b) AND (t1_1.a = ((t2_1.b + t2_1.a) / 2)))
+                     ->  Seq Scan on public.prt1_m_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c, t2_1.a
+                           ->  Seq Scan on public.prt2_m_p2 t2_1
+                                 Output: t2_1.b, t2_1.c, t2_1.a
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: ((((t1_2.a + t1_2.b) / 2) = t2_2.b) AND (t1_2.a = ((t2_2.b + t2_2.a) / 2)))
+                     ->  Seq Scan on public.prt1_m_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c, t2_2.a
+                           ->  Seq Scan on public.prt2_m_p3 t2_2
+                                 Output: t2_2.b, t2_2.c, t2_2.a
+                                 Filter: ((t2_2.b % 25) = 0)
+(36 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_m t1 RIGHT JOIN uprt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 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
+   Output: prt1_m_p1.a, prt1_m_p1.c, prt2_m_p1.b, prt2_m_p1.c
+   Sort Key: prt1_m_p1.a, prt2_m_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_m_p1.a, prt1_m_p1.c, prt2_m_p1.b, prt2_m_p1.c
+               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 public.prt1_m_p1
+                     Output: prt1_m_p1.a, prt1_m_p1.c, prt1_m_p1.b
+                     Filter: ((prt1_m_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_m_p1.b, prt2_m_p1.c, prt2_m_p1.a
+                     ->  Seq Scan on public.prt2_m_p1
+                           Output: prt2_m_p1.b, prt2_m_p1.c, prt2_m_p1.a
+                           Filter: ((prt2_m_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_m_p2.a, prt1_m_p2.c, prt2_m_p2.b, prt2_m_p2.c
+               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 public.prt1_m_p2
+                     Output: prt1_m_p2.a, prt1_m_p2.c, prt1_m_p2.b
+                     Filter: ((prt1_m_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_m_p2.b, prt2_m_p2.c, prt2_m_p2.a
+                     ->  Seq Scan on public.prt2_m_p2
+                           Output: prt2_m_p2.b, prt2_m_p2.c, prt2_m_p2.a
+                           Filter: ((prt2_m_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_m_p3.a, prt1_m_p3.c, prt2_m_p3.b, prt2_m_p3.c
+               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 public.prt1_m_p3
+                     Output: prt1_m_p3.a, prt1_m_p3.c, prt1_m_p3.b
+                     Filter: ((prt1_m_p3.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_m_p3.b, prt2_m_p3.c, prt2_m_p3.a
+                     ->  Seq Scan on public.prt2_m_p3
+                           Output: prt2_m_p3.b, prt2_m_p3.c, prt2_m_p3.a
+                           Filter: ((prt2_m_p3.b % 25) = 0)
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 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 | 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_m t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_m t2 WHERE t2.b % 25 = 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 | 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)
+
+--
+-- 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;
+ANALYZE plt1_p1;
+ANALYZE plt1_p2;
+ANALYZE plt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1 AS SELECT * FROM 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;
+ANALYZE plt2_p1;
+ANALYZE plt2_p2;
+ANALYZE plt2_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt2 AS SELECT * FROM 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;
+ANALYZE plt1_e_p1;
+ANALYZE plt1_e_p2;
+ANALYZE plt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1_e AS SELECT * FROM plt1_e;
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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
+   Output: (avg(t1.a)), (avg(t2.b)), (avg((t3.a + t3.b))), t1.c, t2.c, t3.c
+   Sort Key: t1.c, t3.c
+   ->  HashAggregate
+         Output: avg(t1.a), avg(t2.b), avg((t3.a + t3.b)), t1.c, t2.c, t3.c
+         Group Key: t1.c, t2.c, t3.c
+         ->  Result
+               Output: t1.c, t2.c, t3.c, t1.a, t2.b, t3.a, t3.b
+               ->  Append
+                     ->  Hash Join
+                           Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                           Hash Cond: (t1.c = t2.c)
+                           ->  Seq Scan on public.plt1_p1 t1
+                                 Output: t1.a, t1.c
+                           ->  Hash
+                                 Output: t2.b, t2.c, t3.a, t3.b, t3.c
+                                 ->  Hash Join
+                                       Output: t2.b, t2.c, t3.a, t3.b, t3.c
+                                       Hash Cond: (t2.c = ltrim(t3.c, 'A'::text))
+                                       ->  Seq Scan on public.plt2_p1 t2
+                                             Output: t2.b, t2.c
+                                       ->  Hash
+                                             Output: t3.a, t3.b, t3.c
+                                             ->  Seq Scan on public.plt1_e_p1 t3
+                                                   Output: t3.a, t3.b, t3.c
+                     ->  Hash Join
+                           Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                           Hash Cond: (t1_1.c = t2_1.c)
+                           ->  Seq Scan on public.plt1_p2 t1_1
+                                 Output: t1_1.a, t1_1.c
+                           ->  Hash
+                                 Output: t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                                 ->  Hash Join
+                                       Output: t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                                       Hash Cond: (t2_1.c = ltrim(t3_1.c, 'A'::text))
+                                       ->  Seq Scan on public.plt2_p2 t2_1
+                                             Output: t2_1.b, t2_1.c
+                                       ->  Hash
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             ->  Seq Scan on public.plt1_e_p2 t3_1
+                                                   Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash Join
+                           Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                           Hash Cond: (t1_2.c = t2_2.c)
+                           ->  Seq Scan on public.plt1_p3 t1_2
+                                 Output: t1_2.a, t1_2.c
+                           ->  Hash
+                                 Output: t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                                 ->  Hash Join
+                                       Output: t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                                       Hash Cond: (t2_2.c = ltrim(t3_2.c, 'A'::text))
+                                       ->  Seq Scan on public.plt2_p3 t2_2
+                                             Output: t2_2.b, t2_2.c
+                                       ->  Hash
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             ->  Seq Scan on public.plt1_e_p3 t3_2
+                                                   Output: t3_2.a, t3_2.b, t3_2.c
+(57 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)
+
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM uplt1 t1, uplt2 t2, uplt1_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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: ((t3.a = t1.a) AND (ltrim(t3.c, 'A'::text) = t1.c))
+                     ->  Seq Scan on public.plt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                                 ->  Seq Scan on public.plt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.plt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: ((t3_1.a = t1_1.a) AND (ltrim(t3_1.c, 'A'::text) = t1_1.c))
+                     ->  Seq Scan on public.plt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.c = t1_1.c))
+                                 ->  Seq Scan on public.plt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.plt1_p2 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: ((t3_2.a = t1_2.a) AND (ltrim(t3_2.c, 'A'::text) = t1_2.c))
+                     ->  Seq Scan on public.plt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                                 ->  Seq Scan on public.plt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.plt1_p3 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: ((t3.a = t2.b) AND (ltrim(t3.c, 'A'::text) = t2.c))
+                     ->  Seq Scan on public.plt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                                 ->  Seq Scan on public.plt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.plt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: ((t3_1.a = t2_1.b) AND (ltrim(t3_1.c, 'A'::text) = t2_1.c))
+                     ->  Seq Scan on public.plt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.c = t1_1.c))
+                                 ->  Seq Scan on public.plt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.plt1_p2 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: ((t3_2.a = t2_2.b) AND (ltrim(t3_2.c, 'A'::text) = t2_2.c))
+                     ->  Seq Scan on public.plt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                                 ->  Seq Scan on public.plt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.plt1_p3 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |          | 
+ 100 | 0002 |     |      |          | 
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |          | 
+ 250 | 0005 |     |      |          | 
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |          | 
+ 400 | 0008 |     |      |          | 
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |          | 
+ 550 | 0011 |     |      |          | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |          | 
+ 100 | 0002 |     |      |          | 
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |          | 
+ 250 | 0005 |     |      |          | 
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |          | 
+ 400 | 0008 |     |      |          | 
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |          | 
+ 550 | 0011 |     |      |          | 
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Hash Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                     ->  Seq Scan on public.plt2_p1 t2
+                           Output: t2.b, t2.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Hash Cond: ((t1.c = ltrim(t3.c, 'A'::text)) AND (t1.a = t3.a))
+                                 ->  Seq Scan on public.plt1_p1 t1
+                                       Output: t1.a, t1.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.plt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                     Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.c = t1_1.c))
+                     ->  Seq Scan on public.plt2_p2 t2_1
+                           Output: t2_1.b, t2_1.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_1.a, t1_1.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_1.a, t1_1.c
+                                 Hash Cond: ((t1_1.c = ltrim(t3_1.c, 'A'::text)) AND (t1_1.a = t3_1.a))
+                                 ->  Seq Scan on public.plt1_p2 t1_1
+                                       Output: t1_1.a, t1_1.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.plt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                     Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                     ->  Seq Scan on public.plt2_p3 t2_2
+                           Output: t2_2.b, t2_2.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_2.a, t1_2.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_2.a, t1_2.c
+                                 Hash Cond: ((t1_2.c = ltrim(t3_2.c, 'A'::text)) AND (t1_2.a = t3_2.a))
+                                 ->  Seq Scan on public.plt1_p3 t1_2
+                                       Output: t1_2.a, t1_2.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.plt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: ((t1.a = t2.b) AND (t1.c = t2.c))
+                     ->  Seq Scan on public.plt1_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Hash Cond: ((t2.b = t3.a) AND (t2.c = ltrim(t3.c, 'A'::text)))
+                                 ->  Seq Scan on public.plt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.plt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: ((t1_1.a = t2_1.b) AND (t1_1.c = t2_1.c))
+                     ->  Seq Scan on public.plt1_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: ((t2_1.b = t3_1.a) AND (t2_1.c = ltrim(t3_1.c, 'A'::text)))
+                                 ->  Seq Scan on public.plt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.plt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: ((t1_2.a = t2_2.b) AND (t1_2.c = t2_2.c))
+                     ->  Seq Scan on public.plt1_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: ((t2_2.b = t3_2.a) AND (t2_2.c = ltrim(t3_2.c, 'A'::text)))
+                                 ->  Seq Scan on public.plt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.plt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+     |      |     |      |      100 | A0001
+     |      |     |      |      200 | A0002
+     |      |     |      |      400 | A0004
+     |      |     |      |      500 | A0005
+     |      |     |      |      700 | A0007
+     |      |     |      |      800 | A0008
+     |      |     |      |     1000 | A0010
+     |      |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+     |      |     |      |      100 | A0001
+     |      |     |      |      200 | A0002
+     |      |     |      |      400 | A0004
+     |      |     |      |      500 | A0005
+     |      |     |      |      700 | A0007
+     |      |     |      |      800 | A0008
+     |      |     |      |     1000 | A0010
+     |      |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                                  QUERY PLAN                                                   
+---------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, ((plt1_e_p1.a + plt1_e_p1.b)), plt1_e_p1.c
+   Sort Key: plt1_p1.a, plt2_p1.b, ((plt1_e_p1.a + plt1_e_p1.b))
+   ->  Result
+         Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, (plt1_e_p1.a + plt1_e_p1.b), plt1_e_p1.c
+         ->  Append
+               ->  Hash Full Join
+                     Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                     Hash Cond: ((plt1_p1.a = plt1_e_p1.a) AND (plt1_p1.c = ltrim(plt1_e_p1.c, 'A'::text)))
+                     ->  Hash Full Join
+                           Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+                           Hash Cond: ((plt1_p1.a = plt2_p1.b) AND (plt1_p1.c = plt2_p1.c))
+                           ->  Seq Scan on public.plt1_p1
+                                 Output: plt1_p1.a, plt1_p1.c
+                                 Filter: ((plt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p1.b, plt2_p1.c
+                                 ->  Seq Scan on public.plt2_p1
+                                       Output: plt2_p1.b, plt2_p1.c
+                                       Filter: ((plt2_p1.b % 25) = 0)
+                     ->  Hash
+                           Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                           ->  Seq Scan on public.plt1_e_p1
+                                 Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                                 Filter: ((plt1_e_p1.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c, plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                     Hash Cond: ((plt1_p2.a = plt1_e_p2.a) AND (plt1_p2.c = ltrim(plt1_e_p2.c, 'A'::text)))
+                     ->  Hash Full Join
+                           Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+                           Hash Cond: ((plt1_p2.a = plt2_p2.b) AND (plt1_p2.c = plt2_p2.c))
+                           ->  Seq Scan on public.plt1_p2
+                                 Output: plt1_p2.a, plt1_p2.c
+                                 Filter: ((plt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p2.b, plt2_p2.c
+                                 ->  Seq Scan on public.plt2_p2
+                                       Output: plt2_p2.b, plt2_p2.c
+                                       Filter: ((plt2_p2.b % 25) = 0)
+                     ->  Hash
+                           Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                           ->  Seq Scan on public.plt1_e_p2
+                                 Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                                 Filter: ((plt1_e_p2.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c, plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                     Hash Cond: ((plt1_p3.a = plt1_e_p3.a) AND (plt1_p3.c = ltrim(plt1_e_p3.c, 'A'::text)))
+                     ->  Hash Full Join
+                           Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+                           Hash Cond: ((plt1_p3.a = plt2_p3.b) AND (plt1_p3.c = plt2_p3.c))
+                           ->  Seq Scan on public.plt1_p3
+                                 Output: plt1_p3.a, plt1_p3.c
+                                 Filter: ((plt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p3.b, plt2_p3.c
+                                 ->  Seq Scan on public.plt2_p3
+                                       Output: plt2_p3.b, plt2_p3.c
+                                       Filter: ((plt2_p3.b % 25) = 0)
+                     ->  Hash
+                           Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                           ->  Seq Scan on public.plt1_e_p3
+                                 Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                                 Filter: ((plt1_e_p3.a % 25) = 0)
+(63 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.c = t1_3.c)
+               ->  Seq Scan on public.plt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_3.c, t2.c
+                     Hash Cond: (t1_3.c = ltrim(t2.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p1 t1_3
+                           Output: t1_3.c
+                     ->  Hash
+                           Output: t2.c
+                           ->  Seq Scan on public.plt1_e_p1 t2
+                                 Output: t2.c
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.c = t1_4.c)
+               ->  Seq Scan on public.plt1_p2 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_4.c, t2_1.c
+                     Hash Cond: (t1_4.c = ltrim(t2_1.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p2 t1_4
+                           Output: t1_4.c
+                     ->  Hash
+                           Output: t2_1.c
+                           ->  Seq Scan on public.plt1_e_p2 t2_1
+                                 Output: t2_1.c
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.c = t1_5.c)
+               ->  Seq Scan on public.plt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_5.c, t2_2.c
+                     Hash Cond: (t1_5.c = ltrim(t2_2.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p3 t1_5
+                           Output: t1_5.c
+                     ->  Hash
+                           Output: t2_2.c
+                           ->  Seq Scan on public.plt1_e_p3 t2_2
+                                 Output: t2_2.c
+(49 rows)
+
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1, uplt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+                                   QUERY PLAN                                   
+--------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.c = t1_3.c)
+               ->  Seq Scan on public.plt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_3.c, t1_6.c
+                     Hash Cond: (t1_3.c = ltrim(t1_6.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p1 t1_3
+                           Output: t1_3.c
+                     ->  Hash
+                           Output: t1_6.c
+                           ->  HashAggregate
+                                 Output: t1_6.c
+                                 Group Key: ltrim(t1_6.c, 'A'::text)
+                                 ->  Seq Scan on public.plt1_e_p1 t1_6
+                                       Output: t1_6.c, ltrim(t1_6.c, 'A'::text)
+                                       Filter: ((t1_6.a % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.c = t1_4.c)
+               ->  Seq Scan on public.plt1_p2 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_4.c, t1_7.c
+                     Hash Cond: (t1_4.c = ltrim(t1_7.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p2 t1_4
+                           Output: t1_4.c
+                     ->  Hash
+                           Output: t1_7.c
+                           ->  HashAggregate
+                                 Output: t1_7.c
+                                 Group Key: ltrim(t1_7.c, 'A'::text)
+                                 ->  Seq Scan on public.plt1_e_p2 t1_7
+                                       Output: t1_7.c, ltrim(t1_7.c, 'A'::text)
+                                       Filter: ((t1_7.a % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.c = t1_5.c)
+               ->  Seq Scan on public.plt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_5.c, t1_8.c
+                     Hash Cond: (t1_5.c = ltrim(t1_8.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p3 t1_5
+                           Output: t1_5.c
+                     ->  Hash
+                           Output: t1_8.c
+                           ->  HashAggregate
+                                 Output: t1_8.c
+                                 Group Key: ltrim(t1_8.c, 'A'::text)
+                                 ->  Seq Scan on public.plt1_e_p3 t1_8
+                                       Output: t1_8.c, ltrim(t1_8.c, 'A'::text)
+                                       Filter: ((t1_8.a % 25) = 0)
+(61 rows)
+
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+--
+-- negative testcases
+--
+-- joins where one of the relations is proven empty
+EXPLAIN (VERBOSE, 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;
+            QUERY PLAN            
+----------------------------------
+ Result
+   Output: t1.a, t1.c, t2.b, t2.c
+   One-Time Filter: false
+(3 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 LEFT JOIN prt2 t2 ON t1.a = t2.b;
+              QUERY PLAN              
+--------------------------------------
+ Result
+   Output: prt1.a, prt1.c, t2.b, t2.c
+   One-Time Filter: false
+(3 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                     
+---------------------------------------------------
+ Sort
+   Output: a, c, t2.b, t2.c
+   Sort Key: a, t2.b
+   ->  Hash Left Join
+         Output: a, c, t2.b, t2.c
+         Hash Cond: (t2.b = a)
+         ->  Append
+               ->  Seq Scan on public.prt2 t2
+                     Output: t2.b, t2.c
+                     Filter: ((t2.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p1 t2_1
+                     Output: t2_1.b, t2_1.c
+                     Filter: ((t2_1.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p2 t2_2
+                     Output: t2_2.b, t2_2.c
+                     Filter: ((t2_2.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p3 t2_3
+                     Output: t2_3.b, t2_3.c
+                     Filter: ((t2_3.a % 25) = 0)
+         ->  Hash
+               Output: a, c
+               ->  Result
+                     Output: a, c
+                     One-Time Filter: false
+(24 rows)
+
+EXPLAIN (VERBOSE, 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 % 25 = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                     
+---------------------------------------------------
+ Sort
+   Output: a, c, t2.b, t2.c
+   Sort Key: a, t2.b
+   ->  Hash Left Join
+         Output: a, c, t2.b, t2.c
+         Hash Cond: (t2.b = a)
+         ->  Append
+               ->  Seq Scan on public.prt2 t2
+                     Output: t2.b, t2.c
+                     Filter: ((t2.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p1 t2_1
+                     Output: t2_1.b, t2_1.c
+                     Filter: ((t2_1.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p2 t2_2
+                     Output: t2_2.b, t2_2.c
+                     Filter: ((t2_2.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p3 t2_3
+                     Output: t2_3.b, t2_3.c
+                     Filter: ((t2_3.a % 25) = 0)
+         ->  Hash
+               Output: a, c
+               ->  Result
+                     Output: a, c
+                     One-Time Filter: false
+(24 rows)
+
+CREATE TABLE prt1_n (a int, b int, c varchar) PARTITION BY RANGE(c);
+CREATE TABLE prt1_n_p1 PARTITION OF prt1_n FOR VALUES FROM ('0000') TO ('0250');
+CREATE TABLE prt1_n_p2 PARTITION OF prt1_n FOR VALUES FROM ('0250') TO ('0500');
+INSERT INTO prt1_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 499, 2) i;
+ANALYZE prt1_n;
+ANALYZE prt1_n_p1;
+ANALYZE prt1_n_p2;
+CREATE TABLE prt2_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt2_n_p1 PARTITION OF prt2_n FOR VALUES IN ('0000', '0003', '0004', '0010', '0006', '0007');
+CREATE TABLE prt2_n_p2 PARTITION OF prt2_n FOR VALUES IN ('0001', '0005', '0002', '0009', '0008', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt2_n;
+ANALYZE prt2_n_p1;
+ANALYZE prt2_n_p2;
+CREATE TABLE prt3_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt3_n_p1 PARTITION OF prt3_n FOR VALUES IN ('0000', '0004', '0006', '0007');
+CREATE TABLE prt3_n_p2 PARTITION OF prt3_n FOR VALUES IN ('0001', '0002', '0008', '0010');
+CREATE TABLE prt3_n_p3 PARTITION OF prt3_n FOR VALUES IN ('0003', '0005', '0009', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt3_n;
+ANALYZE prt3_n_p1;
+ANALYZE prt3_n_p2;
+ANALYZE prt3_n_p3;
+CREATE TABLE prt4_n (a int, b int, c text) PARTITION BY RANGE(a);
+CREATE TABLE prt4_n_p1 PARTITION OF prt4_n FOR VALUES FROM (0) TO (300);
+CREATE TABLE prt4_n_p2 PARTITION OF prt4_n FOR VALUES FROM (300) TO (500);
+CREATE TABLE prt4_n_p3 PARTITION OF prt4_n FOR VALUES FROM (500) TO (600);
+INSERT INTO prt4_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt4_n;
+ANALYZE prt4_n_p1;
+ANALYZE prt4_n_p2;
+ANALYZE prt4_n_p3;
+-- partition-wise join can not be applied if the partition ranges differ
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Join
+   Hash Cond: (t1.a = t2.a)
+   ->  Append
+         ->  Seq Scan on prt1 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p2 t1_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt4_n t2
+               ->  Seq Scan on prt4_n_p1 t2_1
+               ->  Seq Scan on prt4_n_p2 t2_2
+               ->  Seq Scan on prt4_n_p3 t2_3
+(13 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 FULL JOIN prt4_n t2 ON t1.a = t2.a;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Full Join
+   Hash Cond: (t1.a = t2.a)
+   ->  Append
+         ->  Seq Scan on prt1 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p2 t1_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt4_n t2
+               ->  Seq Scan on prt4_n_p1 t2_1
+               ->  Seq Scan on prt4_n_p2 t2_2
+               ->  Seq Scan on prt4_n_p3 t2_3
+(13 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                        
+---------------------------------------------------------
+ Nested Loop Left Join
+   ->  Append
+         ->  Seq Scan on prt1 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p2 t1_3
+   ->  Append
+         ->  Seq Scan on prt2 t2
+               Filter: (t1.a < b)
+         ->  Index Scan using iprt2_p1_b on prt2_p1 t2_1
+               Index Cond: (t1.a < b)
+         ->  Index Scan using iprt2_p2_b on prt2_p2 t2_2
+               Index Cond: (t1.a < b)
+         ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+               Index Cond: (t1.a < b)
+(15 rows)
+
+-- equi-join with join condition on partial keys does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1, prt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t1.a % 25 = 0;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Join
+   Hash Cond: (((t2.b + t2.a) / 2) = t1.a)
+   ->  Append
+         ->  Seq Scan on prt2_m t2
+         ->  Seq Scan on prt2_m_p1 t2_1
+         ->  Seq Scan on prt2_m_p2 t2_2
+         ->  Seq Scan on prt2_m_p3 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_m t1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p1 t1_1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p2 t1_2
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p3 t1_3
+                     Filter: ((a % 25) = 0)
+(17 rows)
+
+-- equi-join between out-of-order partition key columns does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.a = t2.b WHERE t1.a % 25 = 0;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Right Join
+   Hash Cond: (t2.b = t1.a)
+   ->  Append
+         ->  Seq Scan on prt2_m t2
+         ->  Seq Scan on prt2_m_p1 t2_1
+         ->  Seq Scan on prt2_m_p2 t2_2
+         ->  Seq Scan on prt2_m_p3 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_m t1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p1 t1_1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p2 t1_2
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p3 t1_3
+                     Filter: ((a % 25) = 0)
+(17 rows)
+
+-- equi-join between non-key columns does not qualify for partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.c = t2.c WHERE t1.a % 25 = 0;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Right Join
+   Hash Cond: ((t2.c)::text = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt2_m t2
+         ->  Seq Scan on prt2_m_p1 t2_1
+         ->  Seq Scan on prt2_m_p2 t2_2
+         ->  Seq Scan on prt2_m_p3 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_m t1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p1 t1_1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p2 t1_2
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p3 t1_3
+                     Filter: ((a % 25) = 0)
+(17 rows)
+
+-- partition-wise join can not be applied for a join between list and range
+-- partitioned table
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1, prt2_n t2 WHERE t1.c = t2.c;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Join
+   Hash Cond: (t2.c = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt2_n t2
+         ->  Seq Scan on prt2_n_p1 t2_1
+         ->  Seq Scan on prt2_n_p2 t2_2
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(11 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 LEFT JOIN prt2_n t2 ON (t1.c = t2.c);
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Right Join
+   Hash Cond: (t2.c = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt2_n t2
+         ->  Seq Scan on prt2_n_p1 t2_1
+         ->  Seq Scan on prt2_n_p2 t2_2
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(11 rows)
+
+-- partition-wise join can not be applied between tables with different
+-- partition lists
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 RIGHT JOIN prt1 t2 ON (t1.c = t2.c);
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Left Join
+   Hash Cond: ((t2.c)::text = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt1 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p2 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(12 rows)
+
+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);
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Full Join
+   Hash Cond: ((t2.c)::text = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt1 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p2 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(12 rows)
+
diff --git a/src/test/regress/parallel_schedule b/src/test/regress/parallel_schedule
index edeb2d6..ac38f50 100644
--- a/src/test/regress/parallel_schedule
+++ b/src/test/regress/parallel_schedule
@@ -103,7 +103,7 @@ test: publication subscription
 # ----------
 # Another group of parallel tests
 # ----------
-test: select_views portals_p2 foreign_key cluster dependency guc bitmapops combocid tsearch tsdicts foreign_data window xmlmap functional_deps advisory_lock json jsonb json_encoding indirect_toast equivclass
+test: select_views portals_p2 foreign_key cluster dependency guc bitmapops combocid tsearch tsdicts foreign_data window xmlmap functional_deps advisory_lock json jsonb json_encoding indirect_toast equivclass partition_join
 # ----------
 # Another group of parallel tests
 # NB: temp.sql does a reconnect which transiently uses 2 connections,
diff --git a/src/test/regress/serial_schedule b/src/test/regress/serial_schedule
index 27a46d7..1bf98a4 100644
--- a/src/test/regress/serial_schedule
+++ b/src/test/regress/serial_schedule
@@ -172,3 +172,4 @@ test: with
 test: xml
 test: event_trigger
 test: stats
+test: partition_join
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
new file mode 100644
index 0000000..0322f1e
--- /dev/null
+++ b/src/test/regress/sql/partition_join.sql
@@ -0,0 +1,515 @@
+--
+-- PARTITION_JOIN
+-- Test partition-wise join between partitioned tables
+--
+
+--
+-- partitioned by a single column
+--
+CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+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, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1 AS SELECT * FROM prt1;
+
+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_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, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+CREATE TABLE uprt2 AS SELECT * FROM prt2;
+
+-- inner join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+-- left outer join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+-- right outer join
+EXPLAIN (VERBOSE, 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.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+-- full outer join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+
+-- Join with pruned partitions from joining relations
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a < 450 AND t2.b > 250 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+-- lateral reference
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+--
+-- partitioned by expression
+--
+CREATE TABLE prt1_e (a int, b int, c varchar) PARTITION BY RANGE(((a + b)/2));
+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);
+INSERT INTO prt1_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_e AS SELECT * FROM prt1_e;
+
+CREATE TABLE prt2_e (a int, b int, c varchar) PARTITION BY RANGE(((b + a)/2));
+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);
+INSERT INTO prt2_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_e;
+ANALYZE prt2_e_p1;
+ANALYZE prt2_e_p2;
+ANALYZE prt2_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_e AS SELECT * FROM prt2_e;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1, uprt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1 LEFT JOIN uprt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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.a % 25 = 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM uprt1 t1, uprt2 t2, uprt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM uprt1_e WHERE uprt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, 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.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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;
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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;
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM uprt1_e WHERE uprt1_e.a % 25 = 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;
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1, uprt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+-- test merge joins with and without using indexes
+SET enable_hashjoin TO off;
+SET enable_nestloop TO off;
+
+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 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 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));
+
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+SET enable_seqscan TO off;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+-- lateral references and parameterized paths
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+RESET enable_hashjoin;
+RESET enable_nestloop;
+RESET enable_seqscan;
+
+--
+-- partitioned by multiple columns
+--
+CREATE TABLE prt1_m (a int, b int, c varchar) 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, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_m;
+ANALYZE prt1_m_p1;
+ANALYZE prt1_m_p2;
+ANALYZE prt1_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_m AS SELECT * FROM prt1_m;
+
+CREATE TABLE prt2_m (a int, b int, c varchar) 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, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_m;
+ANALYZE prt2_m_p1;
+ANALYZE prt2_m_p2;
+ANALYZE prt2_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_m AS SELECT * FROM prt2_m;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_m t1 RIGHT JOIN uprt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_m t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_m t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+
+--
+-- 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;
+ANALYZE plt1_p1;
+ANALYZE plt1_p2;
+ANALYZE plt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1 AS SELECT * FROM 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;
+ANALYZE plt2_p1;
+ANALYZE plt2_p2;
+ANALYZE plt2_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt2 AS SELECT * FROM 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;
+ANALYZE plt1_e_p1;
+ANALYZE plt1_e_p2;
+ANALYZE plt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1_e AS SELECT * FROM plt1_e;
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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;
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM uplt1 t1, uplt2 t2, uplt1_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;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1, uplt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 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 ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+--
+-- negative testcases
+--
+
+-- joins where one of the relations is proven empty
+EXPLAIN (VERBOSE, 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;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 LEFT JOIN prt2 t2 ON t1.a = t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, 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 % 25 = 0 ORDER BY t1.a, t2.b;
+
+CREATE TABLE prt1_n (a int, b int, c varchar) PARTITION BY RANGE(c);
+CREATE TABLE prt1_n_p1 PARTITION OF prt1_n FOR VALUES FROM ('0000') TO ('0250');
+CREATE TABLE prt1_n_p2 PARTITION OF prt1_n FOR VALUES FROM ('0250') TO ('0500');
+INSERT INTO prt1_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 499, 2) i;
+ANALYZE prt1_n;
+ANALYZE prt1_n_p1;
+ANALYZE prt1_n_p2;
+
+CREATE TABLE prt2_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt2_n_p1 PARTITION OF prt2_n FOR VALUES IN ('0000', '0003', '0004', '0010', '0006', '0007');
+CREATE TABLE prt2_n_p2 PARTITION OF prt2_n FOR VALUES IN ('0001', '0005', '0002', '0009', '0008', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt2_n;
+ANALYZE prt2_n_p1;
+ANALYZE prt2_n_p2;
+
+CREATE TABLE prt3_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt3_n_p1 PARTITION OF prt3_n FOR VALUES IN ('0000', '0004', '0006', '0007');
+CREATE TABLE prt3_n_p2 PARTITION OF prt3_n FOR VALUES IN ('0001', '0002', '0008', '0010');
+CREATE TABLE prt3_n_p3 PARTITION OF prt3_n FOR VALUES IN ('0003', '0005', '0009', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt3_n;
+ANALYZE prt3_n_p1;
+ANALYZE prt3_n_p2;
+ANALYZE prt3_n_p3;
+
+CREATE TABLE prt4_n (a int, b int, c text) PARTITION BY RANGE(a);
+CREATE TABLE prt4_n_p1 PARTITION OF prt4_n FOR VALUES FROM (0) TO (300);
+CREATE TABLE prt4_n_p2 PARTITION OF prt4_n FOR VALUES FROM (300) TO (500);
+CREATE TABLE prt4_n_p3 PARTITION OF prt4_n FOR VALUES FROM (500) TO (600);
+INSERT INTO prt4_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt4_n;
+ANALYZE prt4_n_p1;
+ANALYZE prt4_n_p2;
+ANALYZE prt4_n_p3;
+
+-- partition-wise join can not be applied if the partition ranges differ
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 FULL JOIN prt4_n t2 ON t1.a = t2.a;
+
+-- 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);
+
+-- equi-join with join condition on partial keys does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1, prt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t1.a % 25 = 0;
+
+-- equi-join between out-of-order partition key columns does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.a = t2.b WHERE t1.a % 25 = 0;
+
+-- equi-join between non-key columns does not qualify for partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.c = t2.c WHERE t1.a % 25 = 0;
+
+-- partition-wise join can not be applied for a join between list and range
+-- partitioned table
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1, prt2_n t2 WHERE t1.c = t2.c;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 LEFT JOIN prt2_n t2 ON (t1.c = t2.c);
+
+-- partition-wise join can not be applied between tables with different
+-- partition lists
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 RIGHT JOIN prt1 t2 ON (t1.c = t2.c);
+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);
-- 
1.7.9.5

0008-Partition-wise-join.patchapplication/octet-stream; name=0008-Partition-wise-join.patchDownload
From f3b9529dce1b519cb44ad02c8c8640a7b8c5b73f Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Tue, 7 Feb 2017 16:04:03 +0530
Subject: [PATCH 08/11] Partition-wise join

Implement partition-wise join for join between single level partitioned tables.
The details of this technique can be found in optimizer/README, where most of
the implementation has been explained.

We obtain clauses applicable to a child-join by translating corresponding
clauses of the parent. Because child-join can be computed by different
combinations of joining child relations, a given clause is required to be
translated multiple times. In order to reduce the memory consumption, we keep a
repository of child-clauses derived from a parent clause and search in that
repository before translating.

Tests for semi-join, those forcing a merge join for child-join and those
testing lateral join will crash with this patch. The tests testing joins with
partition pruning will also fail. Those crashes and failures are because the
existing code does not expect a child-join to appear in certain cases.
Following patches will fix that code.
---
 src/backend/nodes/copyfuncs.c                |    9 +
 src/backend/optimizer/README                 |   53 ++++
 src/backend/optimizer/path/allpaths.c        |  316 ++++++++++++++++---
 src/backend/optimizer/path/costsize.c        |    3 +
 src/backend/optimizer/path/joinpath.c        |   21 +-
 src/backend/optimizer/path/joinrels.c        |  421 ++++++++++++++++++++++++++
 src/backend/optimizer/plan/createplan.c      |  227 +++++++++++++-
 src/backend/optimizer/prep/prepunion.c       |  160 ++++++++++
 src/backend/optimizer/util/pathnode.c        |  113 +++++++
 src/backend/optimizer/util/placeholder.c     |   55 ++++
 src/backend/optimizer/util/relnode.c         |  266 +++++++++++++++-
 src/backend/utils/misc/guc.c                 |   28 ++
 src/include/nodes/nodes.h                    |    1 +
 src/include/nodes/relation.h                 |   57 ++++
 src/include/optimizer/cost.h                 |    5 +
 src/include/optimizer/pathnode.h             |    6 +
 src/include/optimizer/paths.h                |    5 +
 src/include/optimizer/placeholder.h          |    2 +
 src/include/optimizer/prep.h                 |    8 +
 src/test/regress/expected/partition_join.out |    4 +
 src/test/regress/expected/sysviews.out       |   29 +-
 src/test/regress/sql/partition_join.sql      |    5 +
 22 files changed, 1724 insertions(+), 70 deletions(-)

diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 30d733e..72c021e 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -2070,6 +2070,15 @@ _copyRestrictInfo(const RestrictInfo *from)
 	COPY_SCALAR_FIELD(left_bucketsize);
 	COPY_SCALAR_FIELD(right_bucketsize);
 
+	/*
+	 * Do not copy parent_rinfo and child_rinfos because 1. they create a
+	 * circular dependency between child and parent RestrictInfo 2. dropping
+	 * those links just means that we loose some memory optimizations. 3. There
+	 * is a possibility that the child and parent RestrictInfots themselves may
+	 * have got copied and thus the old links may no longer be valid. The
+	 * caller may set up those links itself, if needed.
+	 */
+
 	return newnode;
 }
 
diff --git a/src/backend/optimizer/README b/src/backend/optimizer/README
index fc0fca4..7565ae4 100644
--- a/src/backend/optimizer/README
+++ b/src/backend/optimizer/README
@@ -1076,3 +1076,56 @@ be desirable to postpone the Gather stage until as near to the top of the
 plan as possible.  Expanding the range of cases in which more work can be
 pushed below the Gather (and costing them accurately) is likely to keep us
 busy for a long time to come.
+
+Partition-wise 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 for a given row in a given partition of a given
+partitioned table, its joining row, if exists, should exist only in the
+matching partition of the other partitioned table; no row from non-matching
+partitions in the other partitioned table can join with the given row from the
+first table. This condition allows the join between partitioned table 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
+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 partition tables
+into join between their partitions is called partition-wise join. We will use
+term "partitioned relation" for both partitioned table as well as join between
+partitioned tables which can use partition-wise join technique.
+
+Partitioning properties of a partitioned table are stored in
+PartitionSchemeData structure. Planner maintains a list of canonical partition
+schemes (distinct PartitionSchemeData objects) so that any two partitioned
+relations with same partitioning scheme share the same PartitionSchemeData
+object. This reduces memory consumed by PartitionSchemeData objects and makes
+it easy to compare the partition schemes of joining relations. RelOptInfos of
+partitioned relations hold partition key expressions and the RelOptInfos of
+the partition relations of that relation.
+
+Partition-wise joins are planned in two phases
+
+1. First phase creates the RelOptInfos for joins between matching partitions,
+henceforth referred to as child-joins. The number of paths created for a
+child-join i.e. join between partitions is same as the number of paths created
+for join between parents. That number grows exponentially with the number of
+base relations being joined. The time and memory consumed to create paths for
+each child-join will be proporional to the number of partitions. This will not
+scale well with thousands of partitions. Instead of that we estimate
+partition-wise join cost based on the costs of sampled child-joins. We choose
+child-joins with higher sizes to have realistic estimates. If the number of
+sampled child-joins is same as the number of live child-joins, we create append
+paths as we know costs of all required child-joins. Otherwise we create
+PartitionJoinPaths with cost estimates based on the costs of sampled
+child-joins. While creating append paths or PartitionJoin paths we create paths
+for all the different possible parameterizations and pathkeys available in the
+sampled child-joins.
+
+2. If PartitionJoinPath emerges as the best possible path, we create paths for
+each unsampled child-join. From every child-join we choose the cheapest path
+with same parameterization or pathkeys as the PartitionJoinPath. This path is
+converted into a plan and all the child-join plans are combined using an Append
+or MergeAppend plan as appropriate. We use a fresh memory context for planning
+each unsampled child-join, thus reducing memory consumption.
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 0eb56f3..a024f47 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -15,6 +15,7 @@
 
 #include "postgres.h"
 
+#include "miscadmin.h"
 #include <limits.h>
 #include <math.h>
 
@@ -93,8 +94,8 @@ static void set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 static void set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 						Index rti, RangeTblEntry *rte);
 static void generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
-						   List *live_childrels,
-						   List *all_child_pathkeys);
+						   List *live_childrels, List *all_child_pathkeys,
+						   bool partition_join_path);
 static Path *get_cheapest_parameterized_child_path(PlannerInfo *root,
 									  RelOptInfo *rel,
 									  Relids required_outer);
@@ -128,8 +129,8 @@ static void recurse_push_qual(Node *setOp, Query *topquery,
 static void remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel);
 static int	compute_parallel_worker(RelOptInfo *rel, BlockNumber pages);
 static void add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
-									List *live_childrels);
-
+							   List *live_childrels, bool partition_join_path);
+static int compare_rel_size(const void *rel1_p, const void *rel2_p);
 
 /*
  * make_one_rel
@@ -891,6 +892,12 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		childrel = find_base_rel(root, childRTindex);
 		Assert(childrel->reloptkind == RELOPT_OTHER_MEMBER_REL);
 
+		/* Pass top parent's relids down the inheritance hierarchy. */
+		if (rel->top_parent_relids)
+			childrel->top_parent_relids = rel->top_parent_relids;
+		else
+			childrel->top_parent_relids = bms_copy(rel->relids);
+
 		/*
 		 * Two partitioned tables with the same partitioning scheme, have their
 		 * partition bounds arranged in the same order. The order of partition
@@ -900,10 +907,15 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		 * RelOptInfos. Arranging RelOptInfos of partitions in the same order
 		 * as their OIDs makes it easy to find the RelOptInfos of matching
 		 * partitions for partition-wise join.
+		 *
+		 * For a partitioned tables, individual partitions can participate in
+		 * the pair-wise joins. We need attr_needed data for building
+		 * child-join targetlists.
 		 */
 		if (rel->part_scheme)
 		{
 			int		cnt_parts;
+			AttrNumber		attno;
 
 			for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
 			{
@@ -913,6 +925,38 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 					rel->part_rels[cnt_parts] = childrel;
 				}
 			}
+
+			for (attno = rel->min_attr; attno <= rel->max_attr; attno++)
+			{
+				int	index = attno - rel->min_attr;
+				Relids	attr_needed = bms_copy(rel->attr_needed[index]);
+
+				/*
+				 * System attributes do not need translation. In such a case,
+				 * the attribute numbers of the parent and the child should
+				 * start from the same minimum attribute.
+				 */
+				if (attno <= 0)
+				{
+					Assert(rel->min_attr == childrel->min_attr);
+					childrel->attr_needed[index] = attr_needed;
+				}
+				else
+				{
+					Var *var = list_nth(appinfo->translated_vars,
+										attno - 1);
+					int child_index;
+
+					/*
+					 * Parent Var for a user defined attribute translates to
+					 * child Var.
+					 */
+					Assert(IsA(var, Var));
+
+					child_index = var->varattno - childrel->min_attr;
+					childrel->attr_needed[child_index] = attr_needed;
+				}
+			}
 		}
 
 		/*
@@ -1057,10 +1101,8 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		 * PlaceHolderVars.)  XXX we do not bother to update the cost or width
 		 * fields of childrel->reltarget; not clear if that would be useful.
 		 */
-		childrel->joininfo = (List *)
-			adjust_appendrel_attrs(root,
-								   (Node *) rel->joininfo,
-								   appinfo_list);
+		childrel->joininfo = build_child_clauses(root, rel->joininfo,
+												 appinfo_list);
 		childrel->reltarget->exprs = (List *)
 			adjust_appendrel_attrs(root,
 								   (Node *) rel->reltarget->exprs,
@@ -1079,14 +1121,6 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		childrel->has_eclass_joins = rel->has_eclass_joins;
 
 		/*
-		 * Note: we could compute appropriate attr_needed data for the child's
-		 * variables, by transforming the parent's attr_needed through the
-		 * translated_vars mapping.  However, currently there's no need
-		 * because attr_needed is only examined for base relations not
-		 * otherrels.  So we just leave the child's attr_needed empty.
-		 */
-
-		/*
 		 * If parallelism is allowable for this query in general, see whether
 		 * it's allowable for this childrel in particular.  But if we've
 		 * already decided the appendrel is not parallel-safe as a whole,
@@ -1269,10 +1303,9 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 	}
 
 	/* Add Append/MergeAppend paths to the "append" relation. */
-	add_paths_to_append_rel(root, rel, live_childrels);
+	add_paths_to_append_rel(root, rel, live_childrels, false);
 }
 
-
 /*
  * add_paths_to_append_rel
  *		Generate Append/MergeAppend paths for given "append" relation.
@@ -1282,20 +1315,44 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
  * an append path collecting one path from each non-dummy child with given
  * parameterization or ordering. Similarly it collects partial paths from
  * non-dummy children to create partial append paths.
+ *
+ * When called on partitioned join relation with partition_join_path = true, it
+ * adds PartitionJoinPath instead of Merge/Append path. This path is costed
+ * based on the costs of sampled child-join and is expanded later into
+ * Merge/Append plan.
  */
 static void
 add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
-						List *live_childrels)
+						List *live_childrels, bool partition_join_path)
 {
 	List	   *subpaths = NIL;
 	bool		subpaths_valid = true;
 	List	   *partial_subpaths = NIL;
-	bool		partial_subpaths_valid = true;
+	bool		partial_subpaths_valid;
 	List	   *all_child_pathkeys = NIL;
 	List	   *all_child_outers = NIL;
 	ListCell   *l;
 
 	/*
+	 * While creating PartitionJoinPath, we sample paths from only a few child
+	 * relations. Even if all of sampled children have partial paths, it's not
+	 * guaranteed that all the unsampled children will have partial paths.
+	 * Hence we do not create partial PartitionJoinPaths.
+	 */
+	partial_subpaths_valid = !partition_join_path ? true : false;
+
+	/* An append relation with all its children dummy is dummy. */
+	if (live_childrels == NIL)
+	{
+		/* Mark the relation as dummy, if not already done so. */
+		if (!IS_DUMMY_REL(rel))
+			set_dummy_rel_pathlist(rel);
+
+		/* No more paths need to be added. */
+		return;
+	}
+
+	/*
 	 * For every non-dummy child, remember the cheapest path.  Also, identify
 	 * all pathkeys (orderings) and parameterizations (required_outer sets)
 	 * available for the non-dummy member relations.
@@ -1394,7 +1451,17 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	 * if we have zero or one live subpath due to constraint exclusion.)
 	 */
 	if (subpaths_valid)
-		add_path(rel, (Path *) create_append_path(rel, subpaths, NULL, 0));
+	{
+		Path *path;
+
+		if (partition_join_path)
+			path = (Path *) create_partition_join_path(root, rel, subpaths,
+													   NULL, NIL);
+		else
+			path = (Path *) create_append_path(rel, subpaths, NULL, 0);
+
+		add_path(rel, path);
+	}
 
 	/*
 	 * Consider an append of partial unordered, unparameterized partial paths.
@@ -1405,6 +1472,8 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		ListCell   *lc;
 		int			parallel_workers = 0;
 
+		Assert(!partition_join_path);
+
 		/*
 		 * Decide on the number of workers to request for this append path.
 		 * For now, we just use the maximum value from among the members.  It
@@ -1431,7 +1500,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	 */
 	if (subpaths_valid)
 		generate_mergeappend_paths(root, rel, live_childrels,
-								   all_child_pathkeys);
+								   all_child_pathkeys, partition_join_path);
 
 	/*
 	 * Build Append paths for each parameterization seen among the child rels.
@@ -1472,8 +1541,18 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		}
 
 		if (subpaths_valid)
-			add_path(rel, (Path *)
-					 create_append_path(rel, subpaths, required_outer, 0));
+		{
+			Path *path;
+
+			if (partition_join_path)
+				path = (Path *) create_partition_join_path(root, rel, subpaths,
+														   required_outer, NIL);
+			else
+				path = (Path *) create_append_path(rel, subpaths,
+												   required_outer, 0);
+
+			add_path(rel, path);
+		}
 	}
 }
 
@@ -1499,11 +1578,16 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
  * parameterized mergejoin plans, it might be worth adding support for
  * parameterized MergeAppends to feed such joins.  (See notes in
  * optimizer/README for why that might not ever happen, though.)
+ *
+ * When called on partitioned join relation with partition_join_path = true, it
+ * adds PartitionJoinPath with pathkeys instead of MergeAppend path. This path
+ * is costed based on the costs of sampled child-join and is expanded later
+ * into MergeAppend plan.
  */
 static void
 generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
 						   List *live_childrels,
-						   List *all_child_pathkeys)
+						   List *all_child_pathkeys, bool partition_join_path)
 {
 	ListCell   *lcp;
 
@@ -1514,6 +1598,7 @@ generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
 		List	   *total_subpaths = NIL;
 		bool		startup_neq_total = false;
 		ListCell   *lcr;
+		Path	   *path;
 
 		/* Select the child paths for this ordering... */
 		foreach(lcr, live_childrels)
@@ -1560,18 +1645,29 @@ generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
 				accumulate_append_subpath(total_subpaths, cheapest_total);
 		}
 
-		/* ... and build the MergeAppend paths */
-		add_path(rel, (Path *) create_merge_append_path(root,
-														rel,
-														startup_subpaths,
-														pathkeys,
-														NULL));
+		/* ... and build the paths */
+		if (partition_join_path)
+			path = (Path *) create_partition_join_path(root, rel,
+													   startup_subpaths,
+													   NULL, pathkeys);
+		else
+			path = (Path *) create_merge_append_path(root, rel,
+													 startup_subpaths,
+													 pathkeys, NULL);
+		add_path(rel, path);
+
 		if (startup_neq_total)
-			add_path(rel, (Path *) create_merge_append_path(root,
-															rel,
-															total_subpaths,
-															pathkeys,
-															NULL));
+		{
+			if (partition_join_path)
+				path = (Path *) create_partition_join_path(root, rel,
+														   total_subpaths,
+														   NULL, pathkeys);
+			else
+				path = (Path *) create_merge_append_path(root, rel,
+														 total_subpaths,
+														 pathkeys, NULL);
+			add_path(rel, path);
+		}
 	}
 }
 
@@ -2316,8 +2412,17 @@ standard_join_search(PlannerInfo *root, int levels_needed, List *initial_rels)
 		 * Run generate_gather_paths() for each just-processed joinrel.  We
 		 * could not do this earlier because both regular and partial paths
 		 * can get added to a particular joinrel at multiple times within
-		 * join_search_one_level.  After that, we're done creating paths for
-		 * the joinrel, so run set_cheapest().
+		 * join_search_one_level.
+		 *
+		 * Similarly, create paths for joinrels which used partition-wise join
+		 * technique. generate_partition_wise_join_paths() creates paths for
+		 * only few of the child-joins with highest sizes. Though we calculate
+		 * size of a child-join only once; when it gets created, it may be
+		 * deemed empty while considering various join orders within
+		 * join_search_one_level.
+		 *
+		 * After that, we're done creating paths for the joinrel, so run
+		 * set_cheapest().
 		 */
 		foreach(lc, root->join_rel_level[lev])
 		{
@@ -2326,6 +2431,9 @@ standard_join_search(PlannerInfo *root, int levels_needed, List *initial_rels)
 			/* Create GatherPaths for any useful partial paths for rel */
 			generate_gather_paths(root, rel);
 
+			/* Create paths for partition-wise joins. */
+			generate_partition_wise_join_paths(root, rel);
+
 			/* Find and save the cheapest paths for this rel */
 			set_cheapest(rel);
 
@@ -3011,6 +3119,138 @@ compute_parallel_worker(RelOptInfo *rel, BlockNumber pages)
 	return parallel_workers;
 }
 
+/*
+ * Function to compare estimated sizes of two relations to be used with
+ * qsort(). Remember that this function is used to sort an array of position
+ * pointers in the array of partitions. So, we have to use double indirection.
+ * See more comments in generate_partition_wise_join_paths() where this
+ * function is used.
+ */
+static int
+compare_rel_size(const void *rel1_p, const void *rel2_p)
+{
+	RelOptInfo *rel1 = **(RelOptInfo ***) rel1_p;
+	RelOptInfo *rel2 = **(RelOptInfo ***) rel2_p;
+
+	return (int) (rel1->rows - rel2->rows);
+}
+
+/*
+ * generate_partition_wise_join_paths
+ *
+ * 		Create paths representing partition-wise join for given partitioned
+ * 		join relation.
+ *
+ * The number of paths created for a child-join is same as the number of paths
+ * created for join between parents. That number grows exponentially with the
+ * number of base relations being joined. The time and memory consumed to
+ * create paths for each child-join will be proporional to the number of
+ * partitions. This will not scale well with thousands of partitions. Instead
+ * of that we estimate partition-wise join cost based on the costs of sampled
+ * child-joins. We choose child-joins with higher sizes to have realistic
+ * estimates.
+ *
+ * This must be called after we have considered all joining orders since
+ * certain join orders may allow us to deem a child-join as dummy.
+ */
+void
+generate_partition_wise_join_paths(PlannerInfo *root, RelOptInfo *rel)
+{
+	List   *sampled_children = NIL;
+	int		cnt_parts;
+	int		num_part_to_plan;
+	int		num_parts;
+	bool	partition_join_path = false;
+	int		num_dummy_parts = 0;
+	RelOptInfo	  ***ordered_part_rels;
+	RelOptInfo	   **part_rels;
+
+	/* Handle only join relations. */
+	if (!IS_JOIN_REL(rel))
+		return;
+
+	/*
+	 * If none of the join orders for this relation could use partition-wise
+	 * join technique, the join is not partitioned. Reset the partitioning
+	 * scheme.
+	 */
+	if (!rel->part_rels)
+		rel->part_scheme = NULL;
+
+	/* If the relation is not partitioned or is proven dummy, nothing to do. */
+	if (!rel->part_scheme || IS_DUMMY_REL(rel))
+		return;
+
+	/* Guard against stack overflow due to overly deep partition hierarchy. */
+	check_stack_depth();
+
+	num_parts = rel->part_scheme->nparts;
+	part_rels = rel->part_rels;
+
+	/* Calculate number of child-joins to sample. */
+	num_part_to_plan = num_parts * sample_partition_fraction;
+	if (num_part_to_plan < 1)
+		num_part_to_plan = 1;
+
+	/* Order the child-join relations by their size.
+	 * add_paths_to_child_joinrel() needs the position of the child-join in the
+	 * array of partition relations. So instead of sorting the actual relations
+	 * get their indexes sorted. We use C pointer arithmatic with qsort to do
+	 * this.
+	 */
+	ordered_part_rels = (RelOptInfo ***) palloc(sizeof(RelOptInfo **) *
+															num_parts);
+	for (cnt_parts = 0; cnt_parts < num_parts; cnt_parts++)
+		ordered_part_rels[cnt_parts] = &part_rels[cnt_parts];
+	qsort(ordered_part_rels, num_parts, sizeof(ordered_part_rels[0]),
+		  compare_rel_size);
+
+	/*
+	 * Create paths for the child-joins for required number of largest
+	 * relations. qsort() returns relations ordered in ascending sizes, so
+	 * start from the end of the array.
+	 */
+	for (cnt_parts = num_parts - 1; cnt_parts >= 0; cnt_parts--)
+	{
+		int		child_no = ordered_part_rels[cnt_parts] - part_rels;
+		RelOptInfo *child_rel = part_rels[child_no];
+
+		/* Create paths for this child. */
+		add_paths_to_child_joinrel(root, rel, child_no);
+
+		/* Dummy children will not be scanned, so ingore those. */
+		if (IS_DUMMY_REL(child_rel))
+		{
+			num_dummy_parts++;
+			continue;
+		}
+
+#ifdef OPTIMIZER_DEBUG
+		debug_print_rel(root, rel);
+#endif
+
+		sampled_children = lappend(sampled_children, child_rel);
+
+		if (list_length(sampled_children) >= num_part_to_plan)
+			break;
+	}
+	pfree(ordered_part_rels);
+
+	/*
+	 * If the number of samples is same as the number of live children, an
+	 * append path will do. Otherwise, we will cost the partition-wise join
+	 * based on the sampled children using PartitionJoinPath.
+	 */
+	if (num_part_to_plan < num_parts - num_dummy_parts)
+		partition_join_path = true;
+
+	/* Add paths for partition-wise join based on the sampled children. */
+	add_paths_to_append_rel(root, rel, sampled_children, partition_join_path);
+
+	if (sampled_children)
+		list_free(sampled_children);
+}
+
 
 /*****************************************************************************
  *			DEBUG SUPPORT
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index a43daa7..c720115 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -126,6 +126,9 @@ bool		enable_nestloop = true;
 bool		enable_material = true;
 bool		enable_mergejoin = true;
 bool		enable_hashjoin = true;
+bool		enable_partition_wise_join = true;
+double		partition_wise_plan_weight = DEFAULT_PARTITION_WISE_PLAN_WEIGHT;
+double		sample_partition_fraction = DEFAULT_SAMPLE_PARTITION_FRACTION;
 
 typedef struct
 {
diff --git a/src/backend/optimizer/path/joinpath.c b/src/backend/optimizer/path/joinpath.c
index 2897245..f80fb25 100644
--- a/src/backend/optimizer/path/joinpath.c
+++ b/src/backend/optimizer/path/joinpath.c
@@ -96,6 +96,19 @@ add_paths_to_joinrel(PlannerInfo *root,
 	JoinPathExtraData extra;
 	bool		mergejoin_allowed = true;
 	ListCell   *lc;
+	Relids		joinrelids;
+
+	/*
+	 * PlannerInfo doesn't contain the SpecialJoinInfos created for joins
+	 * between child relations, even if there is a SpecialJoinInfo node for
+	 * the join between the topmost parents. Hence while calculating Relids
+	 * set representing the restriction, consider relids of topmost parent
+	 * of partitions.
+	 */
+	if (joinrel->reloptkind == RELOPT_OTHER_JOINREL)
+		joinrelids = joinrel->top_parent_relids;
+	else
+		joinrelids = joinrel->relids;
 
 	extra.restrictlist = restrictlist;
 	extra.mergeclause_list = NIL;
@@ -149,16 +162,16 @@ add_paths_to_joinrel(PlannerInfo *root,
 		 * join has already been proven legal.)  If the SJ is relevant, it
 		 * presents constraints for joining to anything not in its RHS.
 		 */
-		if (bms_overlap(joinrel->relids, sjinfo2->min_righthand) &&
-			!bms_overlap(joinrel->relids, sjinfo2->min_lefthand))
+		if (bms_overlap(joinrelids, sjinfo2->min_righthand) &&
+			!bms_overlap(joinrelids, sjinfo2->min_lefthand))
 			extra.param_source_rels = bms_join(extra.param_source_rels,
 										   bms_difference(root->all_baserels,
 													sjinfo2->min_righthand));
 
 		/* full joins constrain both sides symmetrically */
 		if (sjinfo2->jointype == JOIN_FULL &&
-			bms_overlap(joinrel->relids, sjinfo2->min_lefthand) &&
-			!bms_overlap(joinrel->relids, sjinfo2->min_righthand))
+			bms_overlap(joinrelids, sjinfo2->min_lefthand) &&
+			!bms_overlap(joinrelids, sjinfo2->min_righthand))
 			extra.param_source_rels = bms_join(extra.param_source_rels,
 										   bms_difference(root->all_baserels,
 													 sjinfo2->min_lefthand));
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 936ee0c..7476e8e 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -14,9 +14,14 @@
  */
 #include "postgres.h"
 
+#include "miscadmin.h"
+#include "nodes/relation.h"
+#include "optimizer/clauses.h"
 #include "optimizer/joininfo.h"
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
+#include "optimizer/prep.h"
+#include "optimizer/cost.h"
 #include "utils/memutils.h"
 
 
@@ -35,6 +40,14 @@ static bool restriction_is_constant_false(List *restrictlist,
 static void populate_joinrel_with_paths(PlannerInfo *root, RelOptInfo *rel1,
 							RelOptInfo *rel2, RelOptInfo *joinrel,
 							SpecialJoinInfo *sjinfo, List *restrictlist);
+static void try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1,
+						  RelOptInfo *rel2, RelOptInfo *joinrel,
+						  SpecialJoinInfo *parent_sjinfo,
+						  List *parent_restrictlist);
+static bool have_partkey_equi_join(RelOptInfo *rel1, RelOptInfo *rel2,
+						   JoinType jointype, List *restrictlist);
+static int match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel);
+static void free_special_join_info(SpecialJoinInfo *sjinfo);
 
 
 /*
@@ -731,6 +744,9 @@ make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 	populate_joinrel_with_paths(root, rel1, rel2, joinrel, sjinfo,
 								restrictlist);
 
+	/* Apply partition-wise join technique, if possible. */
+	try_partition_wise_join(root, rel1, rel2, joinrel, sjinfo, restrictlist);
+
 	bms_free(joinrelids);
 
 	return joinrel;
@@ -1269,3 +1285,408 @@ restriction_is_constant_false(List *restrictlist, bool only_pushed_down)
 	}
 	return false;
 }
+
+/* Free memory used by SpecialJoinInfo. */
+static void
+free_special_join_info(SpecialJoinInfo *sjinfo)
+{
+	bms_free(sjinfo->min_lefthand);
+	bms_free(sjinfo->syn_lefthand);
+	bms_free(sjinfo->syn_righthand);
+	pfree(sjinfo);
+}
+
+/*
+ * Assess whether join between given two partitioned relations can be broken
+ * down into joins between matching partitions; a technique called
+ * "partition-wise join"
+ *
+ * Partition-wise 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.
+ *
+ * Partition-wise join is planned as follows (details: optimizer/README.)
+ *
+ * 1. Create the RelOptInfos for joins between matching partitions i.e
+ * child-joins and estimate sizes of those. This function is responsible for
+ * this phase.
+ *
+ * 2. Add paths representing partition-wise join. The second phase is
+ * implemented by generate_partition_wise_join_paths(). In order to save time
+ * and memory consumed in creating paths for every child-join, we create paths
+ * for only few child-joins.
+ *
+ * 3. Create merge/append plan to combining plans for every child-join,
+ * creating paths for remaining child-joins.
+ *
+ * The RelOptInfo, SpecialJoinInfo and restrictlist for each child join are
+ * obtained by translating the respective parent join structures.
+ */
+static void
+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;
+	PartitionedJoin	   *partitioned_join;
+
+	/* 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)
+		return;
+
+	/*
+	 * If any of the joining parent relations is proven empty, either the join
+	 * will be empty (INNER join) or will have the inner side all nullified. We
+	 * take care of such cases when creating join paths for parent relations.
+	 * Nothing to be done here. Also, nothing to do, if the parent join is
+	 * proven empty.
+	 */
+	if (IS_DUMMY_REL(rel1) || IS_DUMMY_REL(rel2) || IS_DUMMY_REL(joinrel))
+		return;
+
+	/*
+	 * Partitioning scheme in join relation indicates a possibility that the
+	 * join may be partitioned, but it's not necessary that every pair of
+	 * joining relations can use partition-wise join technique. If one of
+	 * joining relations turns out to be unpartitioned, this pair of joining
+	 * relations can not use partition-wise join technique.
+	 */
+	if (!rel1->part_scheme || !rel2->part_scheme)
+		return;
+
+	/*
+	 * If an equi-join condition between the partition keys of the joining
+	 * relations does not exist, this pair of joining relations can not use
+	 * partition-wise technique.
+	 */
+	if (!have_partkey_equi_join(rel1, rel2, parent_sjinfo->jointype,
+								parent_restrictlist))
+		return;
+
+	/*
+	 * 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);
+
+	/* We should have RelOptInfos of the partitions available. */
+	Assert(rel1->part_rels && rel2->part_rels);
+
+	part_scheme = joinrel->part_scheme;
+	nparts = part_scheme->nparts;
+
+	/*
+	 * We do not store information about valid pairs of joining child
+	 * relations. The pair of joining relations for a child-join can be derived
+	 * from valid pairs of joining parent relations. Amongst the valid pairs of
+	 * parent joining relations, only those which result in partitioned join
+	 * matter for partition-wise join. Remember those so that we can use them
+	 * for creating paths for few child-joins in
+	 * generate_partition_wise_join_paths() later.
+	 */
+	partitioned_join = (PartitionedJoin *) palloc(sizeof(PartitionedJoin));
+	partitioned_join->rel1 = rel1;
+	partitioned_join->rel2 = rel2;
+	partitioned_join->sjinfo = copyObject(parent_sjinfo);
+	partitioned_join->restrictlist = parent_restrictlist;
+	joinrel->partitioned_joins = lappend(joinrel->partitioned_joins,
+										 partitioned_join);
+
+	elog(DEBUG3, "join between relations %s and %s is considered for partition-wise join.",
+		 bmsToString(rel1->relids), bmsToString(rel2->relids));
+
+	/* We are done if child RelOptInfos are already created. */
+	if (joinrel->part_rels)
+		return;
+
+	/* Create all the child RelOptInfos. */
+	joinrel->part_rels = (RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
+
+	/*
+	 * Create child join relations for this partitioned join. While doing so,
+	 * we estimate sizes of these child join relations. These estimates are
+	 * used to find the representative child relations used for costing the
+	 * partition-wise join later.
+	 */
+	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	{
+		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
+		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
+		SpecialJoinInfo	*child_sjinfo;
+		List	*child_restrictlist;
+		RelOptInfo *child_joinrel;
+
+		/* We should never try to join two overlapping sets of rels. */
+		Assert(!bms_overlap(child_rel1->relids, child_rel2->relids));
+
+		Assert (!joinrel->part_rels[cnt_parts]);
+
+		child_joinrel = build_child_join_rel(root, child_rel1, child_rel2,
+											 joinrel, parent_sjinfo->jointype);
+
+		joinrel->part_rels[cnt_parts] = child_joinrel;
+
+		/*
+		 * Construct restrictions applicable to the child join from
+		 * those applicable to the parent join.
+		 */
+		child_restrictlist = build_child_clauses(root, parent_restrictlist,
+												 find_appinfos_by_relids(root,
+													   child_joinrel->relids));
+
+		/*
+		 * Construct SpecialJoinInfo from parent join relations's
+		 * SpecialJoinInfo.
+		 */
+		child_sjinfo = build_child_join_sjinfo(root, parent_sjinfo,
+											   child_rel1->relids,
+											   child_rel2->relids);
+
+		/*
+		 * Set estimates of the child-joinrel's size.
+		 */
+		set_joinrel_size_estimates(root, child_joinrel, child_rel1, child_rel2,
+								   child_sjinfo, child_restrictlist);
+
+		/*
+		 * If the child relations themselves are partitioned, try partition-wise join
+		 * recursively.
+		 */
+		try_partition_wise_join(root, child_rel1, child_rel2, child_joinrel,
+								child_sjinfo, child_restrictlist);
+
+		free_special_join_info(child_sjinfo);
+		child_sjinfo = NULL;
+	}
+}
+
+/*
+ * add_paths_to_child_join
+ * 		Add paths to 'child_id'th child of given parent join relation.
+ *
+ * The function creates paths for given child-join by joining corresponding
+ * children of every pair of joining parent relations which produces
+ * partitioned join. Since we create paths only for sampled child-joins, either
+ * of the children being joined may not have paths. In that case, this function
+ * is called recursively to populate paths for those.
+ */
+void
+add_paths_to_child_joinrel(PlannerInfo *root, RelOptInfo *parent_joinrel,
+						   int child_id)
+{
+	ListCell	*lc;
+	RelOptInfo	   *child_joinrel = parent_joinrel->part_rels[child_id];
+
+	Assert(IS_JOIN_REL(parent_joinrel));
+
+	/* If this child relation already has paths, nothing to do. */
+	if (child_joinrel->cheapest_total_path)
+		return;
+
+	/* A dummy relation will have a dummy path as the cheapest path. */
+	Assert(!is_dummy_rel(child_joinrel));
+
+	/*
+	 * For every partitioned join order, calculate paths for the joining
+	 * child relations and then calculate paths for given child.
+	 */
+	foreach (lc, parent_joinrel->partitioned_joins)
+	{
+		PartitionedJoin	   *pj = lfirst(lc);
+		RelOptInfo *rel1 = pj->rel1;
+		RelOptInfo *rel2 = pj->rel2;
+		RelOptInfo *child_rel1 = rel1->part_rels[child_id];
+		RelOptInfo *child_rel2 = rel2->part_rels[child_id];
+		SpecialJoinInfo	   *child_sjinfo;
+		List	   *child_restrictlist;
+
+		/*
+		 * Add paths to joining relation if it is a join itself.
+		 * Paths for child base relations are created in
+		 * set_append_rel_pathlist().
+		 */
+		if (IS_JOIN_REL(pj->rel1))
+			add_paths_to_child_joinrel(root, rel1, child_id);
+
+		if (IS_JOIN_REL(pj->rel2))
+			add_paths_to_child_joinrel(root, rel2, child_id);
+
+		/*
+		 * Construct SpecialJoinInfo from parent join relations's
+		 * SpecialJoinInfo.
+		 */
+		child_sjinfo = build_child_join_sjinfo(root, pj->sjinfo,
+											   child_rel1->relids,
+											   child_rel2->relids);
+
+
+		/*
+		 * Construct restrictions applicable to the child join from
+		 * those applicable to the parent join.
+		 */
+		child_restrictlist = build_child_clauses(root, pj->restrictlist,
+												 find_appinfos_by_relids(root,
+													   child_joinrel->relids));
+
+		/* Add paths for child join. */
+		populate_joinrel_with_paths(root, rel1->part_rels[child_id],
+									rel2->part_rels[child_id], child_joinrel,
+									child_sjinfo, child_restrictlist);
+
+		/* Add partition-wise join paths for partitioned child-joins. */
+		generate_partition_wise_join_paths(root, child_joinrel);
+
+		free_special_join_info(child_sjinfo);
+		child_sjinfo = NULL;
+	}
+
+	set_cheapest(child_joinrel);
+}
+
+/*
+ * Returns true if there exists an equi-join condition for each pair of
+ * partition key from given relations being joined.
+ */
+static bool
+have_partkey_equi_join(RelOptInfo *rel1, RelOptInfo *rel2,
+					 JoinType jointype, List *restrictlist)
+{
+	PartitionScheme	part_scheme = rel1->part_scheme;
+	ListCell	*lc;
+	int		cnt_pks;
+	int		num_pks;
+	bool   *pk_has_clause;
+
+	/*
+	 * This function should be called when the joining relations have same
+	 * partitioning scheme.
+	 */
+	Assert(rel1->part_scheme == rel2->part_scheme);
+	Assert(part_scheme);
+
+	num_pks = part_scheme->partnatts;
+
+	pk_has_clause = (bool *) palloc0(sizeof(bool) * num_pks);
+
+	foreach (lc, restrictlist)
+	{
+		RestrictInfo *rinfo = lfirst(lc);
+		OpExpr		 *opexpr;
+		Expr		 *expr1;
+		Expr		 *expr2;
+		int		ipk1;
+		int		ipk2;
+
+		/* If processing an outer join, only use its own join clauses. */
+		if (IS_OUTER_JOIN(jointype) && rinfo->is_pushed_down)
+			continue;
+
+		/* Skip clauses which can not be used for a join. */
+		if (!rinfo->can_join)
+			continue;
+
+		/* Skip clauses which are not equality conditions. */
+		if (rinfo->hashjoinoperator == InvalidOid && !rinfo->mergeopfamilies)
+			continue;
+
+		opexpr = (OpExpr *) rinfo->clause;
+		Assert(is_opclause(opexpr));
+
+
+		/* Match the operands to the relation. */
+		if (bms_is_subset(rinfo->left_relids, rel1->relids) &&
+			bms_is_subset(rinfo->right_relids, rel2->relids))
+		{
+			expr1 = linitial(opexpr->args);
+			expr2 = lsecond(opexpr->args);
+		}
+		else if (bms_is_subset(rinfo->left_relids, rel2->relids) &&
+				 bms_is_subset(rinfo->right_relids, rel1->relids))
+		{
+			expr1 = lsecond(opexpr->args);
+			expr2 = linitial(opexpr->args);
+		}
+		else
+			continue;
+
+		/* Associate matching clauses with partition keys. */
+		ipk1 = match_expr_to_partition_keys(expr1, rel1);
+		ipk2 = match_expr_to_partition_keys(expr2, rel2);
+
+		/*
+		 * If the clause refers to different partition keys from
+		 * both relations, it can not be used for partition-wise join.
+		 */
+		if (ipk1 != ipk2)
+			continue;
+
+		/*
+		 * The clause allows partition-wise join if only it uses the same
+		 * operator family as that specified by the partition key.
+		 */
+		if (!list_member_oid(rinfo->mergeopfamilies,
+							 part_scheme->partopfamily[ipk1]))
+			continue;
+
+		/* Mark the partition key as having an equi-join clause. */
+		pk_has_clause[ipk1] = true;
+	}
+
+	/* Check whether every partition key has an equi-join condition. */
+	for (cnt_pks = 0; cnt_pks < num_pks; cnt_pks++)
+	{
+		if (!pk_has_clause[cnt_pks])
+		{
+			pfree(pk_has_clause);
+			return false;
+		}
+	}
+
+	pfree(pk_has_clause);
+	return true;
+}
+
+/*
+ * Find the partition key from the given relation matching the given
+ * expression. If found, return the index of the partition key, else return -1.
+ */
+static int
+match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel)
+{
+	int		cnt_pks;
+	int		num_pks;
+
+	/* This function should be called only for partitioned relations. */
+	Assert(rel->part_scheme);
+
+	num_pks = rel->part_scheme->partnatts;
+
+	/*
+	 * Remove the relabel decoration. We can assume that there is at most one
+	 * RelabelType node; eval_const_expressions() simplifies multiple
+	 * RelabelType nodes into one.
+	 */
+	if (IsA(expr, RelabelType))
+		expr = (Expr *) ((RelabelType *) expr)->arg;
+
+	for (cnt_pks = 0; cnt_pks < num_pks; cnt_pks++)
+	{
+		List	 *pkexprs = rel->partexprs[cnt_pks];
+		ListCell *lc;
+
+		foreach(lc, pkexprs)
+		{
+			Expr *pkexpr = lfirst(lc);
+			if (equal(pkexpr, expr))
+				return cnt_pks;
+		}
+	}
+
+	return -1;
+}
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 997bdcf..fe6b7f8 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -42,6 +42,7 @@
 #include "parser/parse_clause.h"
 #include "parser/parsetree.h"
 #include "utils/lsyscache.h"
+#include "utils/memutils.h"
 
 
 /*
@@ -146,6 +147,8 @@ static CustomScan *create_customscan_plan(PlannerInfo *root,
 static NestLoop *create_nestloop_plan(PlannerInfo *root, NestPath *best_path);
 static MergeJoin *create_mergejoin_plan(PlannerInfo *root, MergePath *best_path);
 static HashJoin *create_hashjoin_plan(PlannerInfo *root, HashPath *best_path);
+static Plan *create_partition_join_plan(PlannerInfo *root,
+										PartitionJoinPath *best_path);
 static Node *replace_nestloop_params(PlannerInfo *root, Node *expr);
 static Node *replace_nestloop_params_mutator(Node *node, PlannerInfo *root);
 static void process_subquery_nestloop_params(PlannerInfo *root,
@@ -369,12 +372,20 @@ create_plan_recurse(PlannerInfo *root, Path *best_path, int flags)
 									(JoinPath *) best_path);
 			break;
 		case T_Append:
-			plan = create_append_plan(root,
-									  (AppendPath *) best_path);
+			if (IsA(best_path, PartitionJoinPath))
+				plan = create_partition_join_plan(root,
+											   (PartitionJoinPath *)best_path);
+			else
+				plan = create_append_plan(root,
+										  (AppendPath *) best_path);
 			break;
 		case T_MergeAppend:
-			plan = create_merge_append_plan(root,
-											(MergeAppendPath *) best_path);
+			if (IsA(best_path, PartitionJoinPath))
+				plan = create_partition_join_plan(root,
+											   (PartitionJoinPath *)best_path);
+			else
+				plan = create_merge_append_plan(root,
+												(MergeAppendPath *) best_path);
 			break;
 		case T_Result:
 			if (IsA(best_path, ProjectionPath))
@@ -3982,6 +3993,214 @@ create_hashjoin_plan(PlannerInfo *root,
 	return join_plan;
 }
 
+/*
+ * create_partition_join_plan
+ *		Creates Merge/Append plan consisting of join plans for child-join.
+ *
+ *		Returns a Plan node.
+ */
+static Plan *
+create_partition_join_plan(PlannerInfo *root, PartitionJoinPath *best_path)
+{
+	RelOptInfo *joinrel = best_path->path.parent;
+	int		nparts;
+	int		cnt_parts;
+	List	   *child_plans = NIL;
+	List	   *tlist = build_path_tlist(root, &best_path->path);
+	Plan	   *plan;
+	MemoryContext	child_context;
+	MemoryContext	old_context;
+	List	   *pathkeys = best_path->path.pathkeys;
+	StringInfoData	mem_context_name;
+
+	/* The relation should be a partitioned join relation. */
+	Assert(IS_JOIN_REL(joinrel) && joinrel->part_scheme &&
+		   joinrel->partitioned_joins);
+
+	nparts = joinrel->part_scheme->nparts;
+
+	/* Create MergeAppend plan when result is expected to be ordered. */
+	if (pathkeys)
+	{
+		MergeAppend *node = makeNode(MergeAppend);
+		plan = &node->plan;
+
+		plan->targetlist = tlist;
+
+		/* Compute sorting info, and adjust MergeAppend's tlist as needed. */
+		(void) prepare_sort_from_pathkeys(plan, pathkeys,
+										  best_path->path.parent->relids,
+										  NULL,
+										  true,
+										  &node->numCols,
+										  &node->sortColIdx,
+										  &node->sortOperators,
+										  &node->collations,
+										  &node->nullsFirst);
+	}
+	else
+	{
+		Append  *node = makeNode(Append);
+		plan = &node->plan;
+		plan->targetlist = tlist;
+	}
+
+	/* Fill costs, so that we can cost Sort node, if required. */
+	copy_generic_path_info(plan, (Path *) best_path);
+
+	/*
+	 * Create a new memory context for planning child joins. Since this routine
+	 * may be called recursively for tables with subpartitions, we use
+	 * a unique context name for every level of partition by using the lowest
+	 * relid amongst the base relations being joined.
+	 */
+	initStringInfo(&mem_context_name);
+	appendStringInfo(&mem_context_name, "%s_%d", "ChildJoinContext",
+					 bms_next_member(joinrel->relids, -1));
+	child_context = AllocSetContextCreate(CurrentMemoryContext,
+										  pstrdup(mem_context_name.data),
+										  ALLOCSET_DEFAULT_SIZES);
+	pfree(mem_context_name.data);
+	resetStringInfo(&mem_context_name);
+
+	/*
+	 * Create a paths for all child joins, one child join at a time. The paths
+	 * for every child join are independent i.e. one child does not require
+	 * paths created for the other. In order to avoid accumulating memory
+	 * consumed while creating paths for every child join, we use a fresh
+	 * memory context for every child join.
+	 */
+	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	{
+		RelOptInfo *child_join;
+		Path	   *child_path = NULL;
+		Plan	   *child_plan;
+		int			numsortkeys;
+		AttrNumber *sortColIdx;
+		Oid		   *sortOperators;
+		Oid		   *collations;
+		bool	   *nullsFirst;
+
+		/*
+		 * Create paths for the child join in a separate context, so that we
+		 * can reuse the memory used by those paths.
+		 */
+		old_context = MemoryContextSwitchTo(child_context);
+
+		add_paths_to_child_joinrel(root, joinrel, cnt_parts);
+
+		child_join = joinrel->part_rels[cnt_parts];
+
+
+		/* Skip empty child. */
+		if (IS_DUMMY_REL(child_join))
+		{
+			MemoryContextSwitchTo(old_context);
+			continue;
+		}
+
+#ifdef OPTIMIZER_DEBUG
+		debug_print_rel(root, rel);
+#endif
+
+		/*
+		 * Search for a child path with pathkeys or parameterization
+		 * matching that of the given path.
+		 */
+		child_path = get_cheapest_path_for_pathkeys(child_join->pathlist,
+													best_path->path.pathkeys,
+											 PATH_REQ_OUTER(&best_path->path),
+													TOTAL_COST);
+
+		if (!child_path)
+			elog(ERROR, "Could not find a path with required pathkeys.");
+
+		MemoryContextSwitchTo(old_context);
+
+		/* Create plan for the current child. */
+		child_plan = create_plan_recurse(root, child_path, CP_EXACT_TLIST);
+
+		if (pathkeys)
+		{
+			MergeAppend *node = (MergeAppend *) plan;
+
+			Assert(IsA(node, MergeAppend));
+
+			/* Compute sorting info, and adjust subplan's tlist as needed */
+			child_plan = prepare_sort_from_pathkeys(child_plan, pathkeys,
+												 child_path->parent->relids,
+												 node->sortColIdx,
+												 false,
+												 &numsortkeys,
+												 &sortColIdx,
+												 &sortOperators,
+												 &collations,
+												 &nullsFirst);
+
+			/*
+			 * Check that we got the same sort key information.  We just Assert
+			 * that the sortops match, since those depend only on the pathkeys;
+			 * but it seems like a good idea to check the sort column numbers
+			 * explicitly, to ensure the tlists really do match up.
+			 */
+			Assert(numsortkeys == node->numCols);
+			if (memcmp(sortColIdx, node->sortColIdx,
+					   numsortkeys * sizeof(AttrNumber)) != 0)
+				elog(ERROR, "MergeAppend child's targetlist doesn't match MergeAppend");
+			Assert(memcmp(sortOperators, node->sortOperators,
+						  numsortkeys * sizeof(Oid)) == 0);
+			Assert(memcmp(collations, node->collations,
+						  numsortkeys * sizeof(Oid)) == 0);
+			Assert(memcmp(nullsFirst, node->nullsFirst,
+						  numsortkeys * sizeof(bool)) == 0);
+
+			/* Now, insert a Sort node if subplan isn't sufficiently ordered */
+			if (!pathkeys_contained_in(pathkeys, child_path->pathkeys))
+			{
+				Sort	   *sort = make_sort(child_plan, numsortkeys,
+											 sortColIdx, sortOperators,
+											 collations, nullsFirst);
+				label_sort_with_costsize(root, sort, -1.0);
+				child_plan = (Plan *) sort;
+			}
+		}
+
+		child_plans = lappend(child_plans, child_plan);
+
+		/*
+		 * Reset the child_join memory context to reclaim the memory consumed
+		 * while creating paths.
+		 */
+		MemoryContextResetAndDeleteChildren(child_context);
+	}
+
+	/* Destroy the child context as we do not need it anymore. */
+	Assert(CurrentMemoryContext == old_context);
+	MemoryContextDelete(child_context);
+
+	/* Partitioned relation with all empty children gets a dummy path. */
+	Assert(child_plans != NIL);
+
+	if (IsA(plan, MergeAppend))
+	{
+		MergeAppend *node = (MergeAppend *)plan;
+
+		node->mergeplans = child_plans;
+	}
+	else
+	{
+		Append  *node = (Append *)plan;
+
+		Assert(IsA(plan, Append));
+		node->appendplans = child_plans;
+	}
+
+	/* Complete rest of the plan. */
+	plan->qual = NIL;
+	plan->lefttree = NULL;
+	plan->righttree = NULL;
+	return plan;
+}
 
 /*****************************************************************************
  *
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index 6f41979..676204f 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -2179,3 +2179,163 @@ adjust_appendrel_attrs_multilevel(PlannerInfo *root, Node *node,
 	/* Now translate for this child */
 	return adjust_appendrel_attrs(root, node, list_make1(appinfo));
 }
+
+/*
+ * build_child_restrictinfo
+ *		Returns a RestrictInfo which is derived from the given RestrictInfo by
+ *		applying the parent-child translation specified by the list of
+ *		AppendRelInfos.
+ *
+ * The topmost parent's RestrictInfo maintains a list of child RestrictInfos
+ * derived from it. If a suitable RestrictInfo is found in that list, it is
+ * returned as is. If there is no such child RestrictInfo, we translate the given
+ * RestrictInfo using the given list of AppendRelInfos and stick it in the
+ * topmost parent's list before returning it to the caller.
+ */
+RestrictInfo *
+build_child_restrictinfo(PlannerInfo *root, RestrictInfo *rinfo,
+						 List *append_rel_infos)
+{
+	Relids child_required_relids;
+	ListCell   *lc;
+	RestrictInfo   *parent_rinfo;
+	RestrictInfo   *child_rinfo;
+	MemoryContext	old_context;
+
+	child_required_relids = adjust_relid_set(rinfo->required_relids,
+												append_rel_infos);
+
+
+	/* Nothing to do, if the clause does not need any translation. */
+	if (bms_equal(child_required_relids, rinfo->required_relids))
+	{
+		bms_free(child_required_relids);
+		return rinfo;
+	}
+
+	/*
+	 * Check if we already have the RestrictInfo for the given child in the
+	 * topmost parent's RestrictInfo.
+	 */
+	parent_rinfo = rinfo->parent_rinfo ? rinfo->parent_rinfo : rinfo;
+	foreach (lc, parent_rinfo->child_rinfos)
+	{
+		child_rinfo = lfirst(lc);
+
+		if (bms_equal(child_rinfo->required_relids, child_required_relids))
+		{
+			bms_free(child_required_relids);
+			return child_rinfo;
+		}
+	}
+
+	/*
+	 * We didn't find any child restrictinfo for the given child, translate the
+	 * given RestrictInfo and stick it into the parent's list. The clause
+	 * expression may get used in plan, so create the child RestrictInfo in the
+	 * planner's context.
+	 */
+	old_context = MemoryContextSwitchTo(root->planner_cxt);
+	child_rinfo = (RestrictInfo *) adjust_appendrel_attrs(root, (Node *) rinfo,
+														  append_rel_infos);
+	bms_free(child_required_relids);
+	parent_rinfo->child_rinfos = lappend(parent_rinfo->child_rinfos,
+										 child_rinfo);
+	child_rinfo->parent_rinfo = parent_rinfo;
+
+	MemoryContextSwitchTo(old_context);
+
+	return child_rinfo;
+}
+
+/*
+ * build_child_clauses
+ *		Convenience routine to call build_child_restrictinfo on a list of
+ *		clauses.
+ */
+List *
+build_child_clauses(PlannerInfo *root, List *clauses, List *append_rel_infos)
+{
+	List   *child_clauses = NIL;
+	ListCell   *lc;
+
+	foreach (lc, clauses)
+	{
+		RestrictInfo   *parent_rinfo = lfirst(lc);
+		RestrictInfo   *child_rinfo;
+
+		Assert(IsA(parent_rinfo, RestrictInfo));
+
+		child_rinfo = build_child_restrictinfo(root, parent_rinfo,
+											   append_rel_infos);
+
+		child_clauses = lappend(child_clauses, child_rinfo);
+	}
+
+	return child_clauses;
+}
+
+/*
+ * find_appinfos_by_relids
+ * 		Find AppendRelInfo structures for all relations specified by relids.
+ */
+List *
+find_appinfos_by_relids(PlannerInfo *root, Relids relids)
+{
+	ListCell	*lc;
+	List		*appinfo_list = NIL;
+
+	foreach (lc, root->append_rel_list)
+	{
+		AppendRelInfo *appinfo = lfirst(lc);
+
+		if (bms_is_member(appinfo->child_relid, relids))
+			appinfo_list = lappend(appinfo_list, appinfo);
+	}
+
+	Assert(list_length(appinfo_list) == bms_num_members(relids));
+	return appinfo_list;
+}
+
+/*
+ * Construct the SpecialJoinInfo for a child-join by translating
+ * SpecialJoinInfo for the join between parents. left_relids and right_relids
+ * are the relids of left and right side of the join respectively.
+ */
+SpecialJoinInfo *
+build_child_join_sjinfo(PlannerInfo *root, SpecialJoinInfo *parent_sjinfo,
+						Relids left_relids, Relids right_relids)
+{
+	SpecialJoinInfo *sjinfo = makeNode(SpecialJoinInfo);
+	MemoryContext	old_context;
+	List	   *left_appinfos = find_appinfos_by_relids(root, left_relids);
+	List	   *right_appinfos = find_appinfos_by_relids(root, right_relids);
+
+	memcpy(sjinfo, parent_sjinfo, sizeof(SpecialJoinInfo));
+
+	sjinfo->min_lefthand = adjust_relid_set(sjinfo->min_lefthand,
+											   left_appinfos);
+	sjinfo->min_righthand = adjust_relid_set(sjinfo->min_righthand,
+												right_appinfos);
+	sjinfo->syn_lefthand = adjust_relid_set(sjinfo->syn_lefthand,
+											   left_appinfos);
+	sjinfo->syn_righthand = adjust_relid_set(sjinfo->syn_righthand,
+												right_appinfos);
+
+	/*
+	 * Replace the Var nodes of parent with those of children in expressions.
+	 * This function may be called within a temporary context, but the
+	 * expressions will be shallow-copied into the plan. Hence copy those in
+	 * the planner's context.
+	 */
+	old_context = MemoryContextSwitchTo(root->planner_cxt);
+	sjinfo->semi_rhs_exprs = (List *) adjust_appendrel_attrs(root,
+											   (Node *) sjinfo->semi_rhs_exprs,
+															 right_appinfos);
+	MemoryContextSwitchTo(old_context);
+
+	list_free(left_appinfos);
+	list_free(right_appinfos);
+
+	return sjinfo;
+}
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index f440875..d861a49 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -23,7 +23,9 @@
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
 #include "optimizer/planmain.h"
+#include "optimizer/prep.h"
 #include "optimizer/restrictinfo.h"
+#include "optimizer/tlist.h"
 #include "optimizer/var.h"
 #include "parser/parsetree.h"
 #include "utils/lsyscache.h"
@@ -2154,6 +2156,117 @@ create_hashjoin_path(PlannerInfo *root,
 }
 
 /*
+ * create_partition_join_path
+ * 		Creates a pathnode that represents partition-wise join for given
+ * 		partitioned join relation.
+ *
+ * This function is called when we haven't created paths for all the child
+ * joins. It estimates the number of rows and cost of the PartitionJoinPath
+ * based upon the number of rows and the cost of representative child-joins
+ * paths.
+ */
+PartitionJoinPath *
+create_partition_join_path(PlannerInfo *root, RelOptInfo *rel, List *subpaths,
+						   Bitmapset *required_outer, List *pathkeys)
+{
+	PartitionJoinPath *pathnode = makeNode(PartitionJoinPath);
+	double	subpath_rows = 0;
+	double	subpath_startup_cost = 0;
+	double	subpath_total_cost = 0;
+	double	child_rel_rows = 0;
+	ListCell   *lc;
+
+	Assert(rel->part_scheme);
+
+	pathnode->path.pathtype = pathkeys ? T_MergeAppend : T_Append;
+	pathnode->path.parent = rel;
+	pathnode->path.pathtarget = rel->reltarget;
+	pathnode->path.param_info = get_appendrel_parampathinfo(rel,
+															required_outer);
+	pathnode->path.pathkeys = pathkeys;
+
+	/* No parallel paths here. See more details in add_paths_to_append_rel() */
+	pathnode->path.parallel_aware = false;
+	pathnode->path.parallel_safe = false;
+	pathnode->path.parallel_workers = 0;
+
+	/* Accumulate the number of rows and costs from the given subpaths. */
+	foreach (lc, subpaths)
+	{
+		Path   *subpath = lfirst(lc);
+
+		if (!pathkeys)
+		{
+			/*
+			 * Startup cost of an append relation is the startup cost of the
+			 * first subpath. Assume that the given first child will be the
+			 * first child in the final plan as well.
+			 */
+			if (lc == list_head(subpaths))
+				subpath_startup_cost = subpath->startup_cost;
+			subpath_total_cost += subpath->total_cost;
+		}
+		else if (pathkeys_contained_in(pathkeys, subpath->pathkeys))
+		{
+			/*
+			 * Subpath is adequately ordered, we won't need to sort it. We need
+			 * all the subplans to return their respective first rows, before
+			 * returning a row. So add the startup costs.
+			 */
+			subpath_startup_cost += subpath->startup_cost;
+			subpath_total_cost += subpath->total_cost;
+		}
+		else
+		{
+			/* We'll need to insert a Sort node, so include cost for that */
+			Path		sort_path;		/* dummy for result of cost_sort */
+
+			cost_sort(&sort_path,
+					  root,
+					  pathkeys,
+					  subpath->total_cost,
+					  subpath->parent->tuples,
+					  subpath->pathtarget->width,
+					  0.0,
+					  work_mem,
+					  -1);
+			subpath_startup_cost += sort_path.startup_cost;
+			subpath_total_cost += sort_path.total_cost;
+		}
+
+		subpath_rows += subpath->rows;
+		child_rel_rows += subpath->parent->rows;
+
+	}
+
+	/*
+	 * For a parameterized path, extrapolate the number of rows for the append
+	 * relation by considering the average selectivity of the parameterization
+	 * across the given children.
+	 */
+	if (bms_is_empty(required_outer))
+		pathnode->path.rows = rel->rows;
+	else
+		pathnode->path.rows = rel->rows * (subpath_rows / child_rel_rows);
+
+	/* Extrapolate the total cost to account for yet-to-be planned children. */
+	if (!pathkeys)
+		pathnode->path.startup_cost = subpath_startup_cost;
+	else
+		pathnode->path.startup_cost = (subpath_startup_cost * pathnode->path.rows) / subpath_rows;
+	pathnode->path.total_cost = (subpath_total_cost * pathnode->path.rows) / subpath_rows;
+
+	/*
+	 * Multiply the costs with scaling factor as specified. Used to encourage
+	 * or discourage use of partition-wise join plans.
+	 */
+	pathnode->path.startup_cost *= partition_wise_plan_weight;
+	pathnode->path.total_cost *= partition_wise_plan_weight;
+
+	return pathnode;
+}
+
+/*
  * create_projection_path
  *	  Creates a pathnode that represents performing a projection.
  *
diff --git a/src/backend/optimizer/util/placeholder.c b/src/backend/optimizer/util/placeholder.c
index 698a387..e06bccc 100644
--- a/src/backend/optimizer/util/placeholder.c
+++ b/src/backend/optimizer/util/placeholder.c
@@ -20,6 +20,7 @@
 #include "optimizer/pathnode.h"
 #include "optimizer/placeholder.h"
 #include "optimizer/planmain.h"
+#include "optimizer/prep.h"
 #include "optimizer/var.h"
 #include "utils/lsyscache.h"
 
@@ -414,6 +415,10 @@ add_placeholders_to_joinrel(PlannerInfo *root, RelOptInfo *joinrel,
 	Relids		relids = joinrel->relids;
 	ListCell   *lc;
 
+	/* This function is called only on the parent relations. */
+	Assert(!IS_OTHER_REL(joinrel) && !IS_OTHER_REL(outer_rel) &&
+		   !IS_OTHER_REL(inner_rel));
+
 	foreach(lc, root->placeholder_list)
 	{
 		PlaceHolderInfo *phinfo = (PlaceHolderInfo *) lfirst(lc);
@@ -459,3 +464,53 @@ add_placeholders_to_joinrel(PlannerInfo *root, RelOptInfo *joinrel,
 		}
 	}
 }
+
+/*
+ * add_placeholders_to_child_joinrel
+ *		Translate the PHVs in parent's targetlist and add them to the child's
+ *		targetlist. Also adjust the cost
+ */
+void
+add_placeholders_to_child_joinrel(PlannerInfo *root, RelOptInfo *childrel,
+								  RelOptInfo *parentrel)
+{
+	ListCell  *lc;
+
+	/* This function is called only for join relations. */
+	Assert(IS_JOIN_REL(childrel) && IS_JOIN_REL(parentrel));
+
+	/* Ensure child relations is really what it claims to be. */
+	Assert(IS_OTHER_REL(childrel));
+
+	foreach (lc, parentrel->reltarget->exprs)
+	{
+		PlaceHolderVar *phv = lfirst(lc);
+
+		if (IsA(phv, PlaceHolderVar))
+		{
+			/*
+			 * In case the placeholder Var refers to any of the parent
+			 * relations, translate it to refer to the corresponding child.
+			 */
+			if (bms_overlap(phv->phrels, parentrel->relids) &&
+				childrel->reloptkind == RELOPT_OTHER_JOINREL)
+			{
+				List *append_rel_infos;
+
+				append_rel_infos = find_appinfos_by_relids(root,
+														  childrel->relids);
+				phv = (PlaceHolderVar *) adjust_appendrel_attrs(root,
+															  (Node *) phv,
+														 append_rel_infos);
+			}
+
+			childrel->reltarget->exprs = lappend(childrel->reltarget->exprs,
+												 phv);
+		}
+	}
+
+	/* Adjust the cost and width of child targetlist. */
+	childrel->reltarget->cost.startup = parentrel->reltarget->cost.startup;
+	childrel->reltarget->cost.per_tuple = parentrel->reltarget->cost.per_tuple;
+	childrel->reltarget->width = parentrel->reltarget->width;
+}
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 19982dc..1eed987 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -23,6 +23,7 @@
 #include "optimizer/paths.h"
 #include "optimizer/placeholder.h"
 #include "optimizer/plancat.h"
+#include "optimizer/prep.h"
 #include "optimizer/restrictinfo.h"
 #include "optimizer/tlist.h"
 #include "utils/hsearch.h"
@@ -54,6 +55,9 @@ static void set_foreign_rel_properties(RelOptInfo *joinrel,
 static void add_join_rel(PlannerInfo *root, RelOptInfo *joinrel);
 extern ParamPathInfo *find_param_path_info(RelOptInfo *rel,
 									  Relids required_outer);
+static void build_joinrel_partition_info(RelOptInfo *joinrel,
+						  RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+						  JoinType jointype);
 
 
 /*
@@ -434,6 +438,9 @@ build_join_rel(PlannerInfo *root,
 	RelOptInfo *joinrel;
 	List	   *restrictlist;
 
+	/* This function should be used only for join between parents. */
+	Assert(!IS_OTHER_REL(outer_rel) && !IS_OTHER_REL(inner_rel));
+
 	/*
 	 * See if we already have a joinrel for this set of base rels.
 	 */
@@ -532,6 +539,10 @@ build_join_rel(PlannerInfo *root,
 	if (bms_is_empty(joinrel->direct_lateral_relids))
 		joinrel->direct_lateral_relids = NULL;
 
+	/* Store the partition information. */
+	build_joinrel_partition_info(joinrel, outer_rel, inner_rel,
+								 sjinfo->jointype);
+
 	/*
 	 * Construct restrict and join clause lists for the new joinrel. (The
 	 * caller might or might not need the restrictlist, but I need it anyway
@@ -594,6 +605,126 @@ build_join_rel(PlannerInfo *root,
 	return joinrel;
 }
 
+ /*
+ * build_child_join_rel
+ *		Builds RelOptInfo for joining given two child relations from RelOptInfo
+ *		representing the join between their parents.
+ *
+ * 'outer_rel' and 'inner_rel' are the RelOptInfos of child relations being
+ *		joined.
+ * 'parent_joinrel' is the RelOptInfo representing the join between parent
+ *		relations. Most of the members of new RelOptInfo are produced by
+ *		translating corresponding members of this RelOptInfo.
+ * 'sjinfo': context info for child join
+ * 'restrictlist': list of RestrictInfo nodes that apply to this particular
+ *		pair of joinable relations.
+ * 'join_appinfos': list of AppendRelInfo nodes for base child relations involved
+ *		in this join.
+ */
+RelOptInfo *
+build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
+						 RelOptInfo *inner_rel, RelOptInfo *parent_joinrel,
+						 JoinType jointype)
+{
+	RelOptInfo *joinrel = makeNode(RelOptInfo);
+
+	/* Only joins between other relations land here. */
+	Assert(IS_OTHER_REL(outer_rel) && IS_OTHER_REL(inner_rel));
+
+	joinrel->reloptkind = RELOPT_OTHER_JOINREL;
+	joinrel->relids = bms_union(outer_rel->relids, inner_rel->relids);
+	joinrel->rows = 0;
+	/* cheap startup cost is interesting iff not all tuples to be retrieved */
+	joinrel->consider_startup = (root->tuple_fraction > 0);
+	joinrel->consider_param_startup = false;
+	joinrel->consider_parallel = false;
+	joinrel->reltarget = create_empty_pathtarget();
+	joinrel->pathlist = NIL;
+	joinrel->ppilist = NIL;
+	joinrel->partial_pathlist = NIL;
+	joinrel->cheapest_startup_path = NULL;
+	joinrel->cheapest_total_path = NULL;
+	joinrel->cheapest_unique_path = NULL;
+	joinrel->cheapest_parameterized_paths = NIL;
+	joinrel->direct_lateral_relids = NULL;
+	joinrel->lateral_relids = NULL;
+	joinrel->relid = 0;			/* indicates not a baserel */
+	joinrel->rtekind = RTE_JOIN;
+	joinrel->min_attr = 0;
+	joinrel->max_attr = 0;
+	joinrel->attr_needed = NULL;
+	joinrel->attr_widths = NULL;
+	joinrel->lateral_vars = NIL;
+	joinrel->lateral_referencers = NULL;
+	joinrel->indexlist = NIL;
+	joinrel->pages = 0;
+	joinrel->tuples = 0;
+	joinrel->allvisfrac = 0;
+	joinrel->subroot = NULL;
+	joinrel->subplan_params = NIL;
+	joinrel->serverid = InvalidOid;
+	joinrel->userid = InvalidOid;
+	joinrel->useridiscurrent = false;
+	joinrel->fdwroutine = NULL;
+	joinrel->fdw_private = NULL;
+	joinrel->baserestrictinfo = NIL;
+	joinrel->baserestrictcost.startup = 0;
+	joinrel->baserestrictcost.per_tuple = 0;
+	joinrel->joininfo = NIL;
+	joinrel->has_eclass_joins = false;
+	joinrel->part_scheme = NULL;
+	joinrel->partexprs = NULL;
+	joinrel->top_parent_relids = NULL;
+	joinrel->part_rels = NULL;
+
+	joinrel->top_parent_relids = bms_union(outer_rel->top_parent_relids,
+										   inner_rel->top_parent_relids);
+
+	/* Compute information relevant to foreign relations. */
+	set_foreign_rel_properties(joinrel, outer_rel, inner_rel);
+
+	/* Build targetlist */
+	build_joinrel_tlist(root, joinrel, outer_rel);
+	build_joinrel_tlist(root, joinrel, inner_rel);
+	/* Add placeholder variables. */
+	add_placeholders_to_child_joinrel(root, joinrel, parent_joinrel);
+
+	/* Construct joininfo list. */
+	joinrel->joininfo = build_child_clauses(root, parent_joinrel->joininfo,
+											find_appinfos_by_relids(root,
+															 joinrel->relids));
+
+	/*
+	 * Lateral relids referred in child join will be same as that referred in
+	 * the parent relation. Throw any partial result computed while building
+	 * the targetlist.
+	 */
+	bms_free(joinrel->direct_lateral_relids);
+	bms_free(joinrel->lateral_relids);
+	joinrel->direct_lateral_relids = (Relids) bms_copy(parent_joinrel->direct_lateral_relids);
+	joinrel->lateral_relids = (Relids) bms_copy(parent_joinrel->lateral_relids);
+
+	/*
+	 * If the parent joinrel has pending equivalence classes, so does the
+	 * child.
+	 */
+	joinrel->has_eclass_joins = parent_joinrel->has_eclass_joins;
+
+	/* Is the join between partitions itself partitioned? */
+	build_joinrel_partition_info(joinrel, outer_rel, inner_rel, jointype);
+
+	/* Child joinrel is parallel safe if parent is parallel safe. */
+	joinrel->consider_parallel = parent_joinrel->consider_parallel;
+
+	/* We build the join only once. */
+	Assert(!find_join_rel(root, joinrel->relids));
+
+	/* Add the relation to the PlannerInfo. */
+	add_join_rel(root, joinrel);
+
+	return joinrel;
+}
+
 /*
  * min_join_parameterization
  *
@@ -649,9 +780,15 @@ static void
 build_joinrel_tlist(PlannerInfo *root, RelOptInfo *joinrel,
 					RelOptInfo *input_rel)
 {
-	Relids		relids = joinrel->relids;
+	Relids		relids;
 	ListCell   *vars;
 
+	/* attrs_needed refers to parent relids and not those of a child. */
+	if (joinrel->top_parent_relids)
+		relids = joinrel->top_parent_relids;
+	else
+		relids = joinrel->relids;
+
 	foreach(vars, input_rel->reltarget->exprs)
 	{
 		Var		   *var = (Var *) lfirst(vars);
@@ -667,23 +804,47 @@ build_joinrel_tlist(PlannerInfo *root, RelOptInfo *joinrel,
 
 		/*
 		 * Otherwise, anything in a baserel or joinrel targetlist ought to be
-		 * a Var.  (More general cases can only appear in appendrel child
-		 * rels, which will never be seen here.)
+		 * a Var or ConvertRowtypeExpr introduced while translating parent
+		 * targetlist to that of the child.
 		 */
-		if (!IsA(var, Var))
+		if (IsA(var, Var))
+		{
+			/* Get the Var's original base rel */
+			baserel = find_base_rel(root, var->varno);
+
+			/* Is it still needed above this joinrel? */
+			ndx = var->varattno - baserel->min_attr;
+		}
+		else if (IsA(var, ConvertRowtypeExpr))
+		{
+			ConvertRowtypeExpr *child_expr = (ConvertRowtypeExpr *) var;
+			Var	 *childvar = (Var *) child_expr->arg;
+
+			/*
+			 * Child's whole-row references are converted to that of parent
+			 * using ConvertRowtypeExpr. In this case, the argument to
+			 * ConvertRowtypeExpr is expected to be a whole-row reference of
+			 * the child.
+			 */
+			Assert(IsA(childvar, Var) && childvar->varattno == 0);
+
+			baserel = find_base_rel(root, childvar->varno);
+			ndx = 0 - baserel->min_attr;
+		}
+		else
 			elog(ERROR, "unexpected node type in rel targetlist: %d",
 				 (int) nodeTag(var));
 
-		/* Get the Var's original base rel */
-		baserel = find_base_rel(root, var->varno);
-
-		/* 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 */
 			joinrel->reltarget->exprs = lappend(joinrel->reltarget->exprs, var);
-			/* Vars have cost zero, so no need to adjust reltarget->cost */
+
+			/*
+			 * Vars have cost zero, so no need to adjust reltarget->cost. Even
+			 * if, it's a ConvertRowtypeExpr, it will be computed only for the
+			 * base relation, costing nothing for a join.
+			 */
 			joinrel->reltarget->width += baserel->attr_widths[ndx];
 		}
 	}
@@ -820,6 +981,9 @@ subbuild_joinrel_joinlist(RelOptInfo *joinrel,
 {
 	ListCell   *l;
 
+	/* Expected to be called only for join between parent relations. */
+	Assert(joinrel->reloptkind == RELOPT_JOINREL);
+
 	foreach(l, joininfo_list)
 	{
 		RestrictInfo *rinfo = (RestrictInfo *) lfirst(l);
@@ -1366,3 +1530,85 @@ find_param_path_info(RelOptInfo *rel, Relids required_outer)
 
 	return NULL;
 }
+
+/*
+ * build_joinrel_partition_info
+ *		If the join between given partitioned relations is possibly partitioned
+ *		set the partitioning scheme and partition keys expressions for the
+ *		join.
+ *
+ * If the two relations have same partitioning scheme, their join may be
+ * partitioned and will follow the same partitioning scheme as the joining
+ * relations.
+ */
+static void
+build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
+							 RelOptInfo *inner_rel, JoinType jointype)
+{
+	int		num_pks;
+	int		cnt;
+
+	/* Nothing to do if partition-wise join technique is disabled. */
+	if (!enable_partition_wise_join)
+	{
+		joinrel->part_scheme = NULL;
+		return;
+	}
+
+	/*
+	 * The join is not partitioned, if any of the relations being joined are
+	 * not partitioned or they do not have same partitioning scheme.
+	 */
+	if (!outer_rel->part_scheme || !inner_rel->part_scheme ||
+		outer_rel->part_scheme != inner_rel->part_scheme)
+	{
+		joinrel->part_scheme = NULL;
+		return;
+	}
+
+	/*
+	 * This function will be called only once for each joinrel, hence it should
+	 * not have partition scheme, partition key expressions and array for
+	 * storing child relations set.
+	 */
+	Assert(!joinrel->part_scheme && !joinrel->partexprs &&
+		   !joinrel->part_rels);
+
+	/*
+	 * Join relation is partitioned using same partitioning scheme as the
+	 * joining relations.
+	 */
+	joinrel->part_scheme = outer_rel->part_scheme;
+	num_pks = joinrel->part_scheme->partnatts;
+
+	/*
+	 * Construct partition keys for the join.
+	 *
+	 * An INNER join between two partitioned relations is partition by key
+	 * expressions from both the relations. For tables A and B partitioned by a and b
+	 * respectively, (A INNER JOIN B ON A.a = B.b) is partitioned by both A.a
+	 * and B.b.
+	 *
+	 * An OUTER join like (A LEFT JOIN B ON A.a = B.b) may produce rows with
+	 * B.b NULL. These rows may not fit the partitioning conditions imposed on
+	 * B.b. Hence, strictly speaking, the join is not partitioned by B.b.
+	 * Strictly speaking, partition keys of an OUTER join should include
+	 * partition key expressions from the OUTER side only. Consider a join like
+	 * (A LEFT JOIN B on (A.a = B.b) LEFT JOIN C ON B.b = C.c. If we do not
+	 * include B.b as partition key expression for (AB), it prohibits us from
+	 * using partition-wise join when joining (AB) with C as there is no
+	 * equi-join between partition keys of joining relations. But two NULL
+	 * values are never equal and no two rows from mis-matching partitions can
+	 * join. Hence it's safe to include B.b as partition key expression for
+	 * (AB), even though rows in (AB) are not strictly partitioned by B.b.
+	 */
+	joinrel->partexprs = (List **) palloc0(sizeof(List *) * num_pks);
+	for (cnt = 0; cnt < num_pks; cnt++)
+	{
+		List *pkexpr = list_copy(outer_rel->partexprs[cnt]);
+
+		pkexpr = list_concat(pkexpr,
+							 list_copy(inner_rel->partexprs[cnt]));
+		joinrel->partexprs[cnt] = pkexpr;
+	}
+}
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index de85eca..afd0c23 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -901,6 +901,15 @@ static struct config_bool ConfigureNamesBool[] =
 		true,
 		NULL, NULL, NULL
 	},
+	{
+		{"enable_partition_wise_join", PGC_USERSET, QUERY_TUNING_METHOD,
+			gettext_noop("Enables partition-wise join."),
+			NULL
+		},
+		&enable_partition_wise_join,
+		true,
+		NULL, NULL, NULL
+	},
 
 	{
 		{"geqo", PGC_USERSET, QUERY_TUNING_GEQO,
@@ -2947,6 +2956,25 @@ static struct config_real ConfigureNamesReal[] =
 	},
 
 	{
+		{"partition_wise_plan_weight", PGC_USERSET, QUERY_TUNING_OTHER,
+			gettext_noop("Multiplication factor for partition-wise plan costs."),
+			NULL
+		},
+		&partition_wise_plan_weight,
+		DEFAULT_PARTITION_WISE_PLAN_WEIGHT, 0, DBL_MAX,
+		NULL, NULL, NULL
+	},
+	{
+		{"sample_partition_fraction", PGC_USERSET, QUERY_TUNING_OTHER,
+			gettext_noop("Fraction of partitions to be used as sample for calculating total cost of partition-wise plans."),
+			NULL
+		},
+		&sample_partition_fraction,
+		DEFAULT_SAMPLE_PARTITION_FRACTION, 0, 1,
+		NULL, NULL, NULL
+	},
+
+	{
 		{"bgwriter_lru_multiplier", PGC_SIGHUP, RESOURCES_BGWRITER,
 			gettext_noop("Multiple of the average buffer usage to free per round."),
 			NULL
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index 95dd8ba..292d9a6 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -240,6 +240,7 @@ typedef enum NodeTag
 	T_NestPath,
 	T_MergePath,
 	T_HashPath,
+	T_PartitionJoinPath,
 	T_AppendPath,
 	T_MergeAppendPath,
 	T_ResultPath,
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 4f99184..146d53b 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -391,6 +391,11 @@ typedef struct PartitionSchemeData *PartitionScheme;
  * handling join alias Vars.  Currently this is not needed because all join
  * alias Vars are expanded to non-aliased form during preprocess_expression.
  *
+ * We also have relations representing joins between child relations of
+ * different partitioned tables. These relations are not added to
+ * join_rel_level lists as they are not joined directly by the dynamic
+ * programming algorithm.
+ *
  * There is also a RelOptKind for "upper" relations, which are RelOptInfos
  * that describe post-scan/join processing steps, such as aggregation.
  * Many of the fields in these RelOptInfos are meaningless, but their Path
@@ -512,10 +517,19 @@ typedef enum RelOptKind
 	RELOPT_BASEREL,
 	RELOPT_JOINREL,
 	RELOPT_OTHER_MEMBER_REL,
+	RELOPT_OTHER_JOINREL,
 	RELOPT_UPPER_REL,
 	RELOPT_DEADREL
 } RelOptKind;
 
+#define IS_OTHER_REL(rel) \
+	((rel)->reloptkind == RELOPT_OTHER_MEMBER_REL || \
+	 (rel)->reloptkind == RELOPT_OTHER_JOINREL)
+
+#define IS_JOIN_REL(rel)	\
+	((rel)->reloptkind == RELOPT_JOINREL || \
+	 (rel)->reloptkind == RELOPT_OTHER_JOINREL)
+
 typedef struct RelOptInfo
 {
 	NodeTag		type;
@@ -600,6 +614,14 @@ typedef struct RelOptInfo
 										 * as the number of joining
 										 * relations.
 										 */
+
+	/* For joins between partitioned tables. */
+	List			   *partitioned_joins;	/* List of join orders which yield
+											 * relations partitioned by above
+											 * partition scheme.
+											 */
+	/* Set only for "other" base or "other" join relations. */
+	Relids		top_parent_relids;		/* Relids of topmost parents. */
 } RelOptInfo;
 
 /*
@@ -1540,6 +1562,14 @@ typedef struct LimitPath
 	Node	   *limitCount;		/* COUNT parameter, or NULL if none */
 } LimitPath;
 
+/*
+ * PartitionJoinPath represents partition-wise join between two partitioned
+ * tables.
+ */
+typedef struct PartitionJoinPath
+{
+	Path		path;
+} PartitionJoinPath;
 
 /*
  * Restriction clause info.
@@ -1747,6 +1777,18 @@ typedef struct RestrictInfo
 	/* cache space for hashclause processing; -1 if not yet set */
 	Selectivity left_bucketsize;	/* avg bucketsize of left side */
 	Selectivity right_bucketsize;		/* avg bucketsize of right side */
+
+	/*
+	 * Repository to locate child RestrictInfos derived from parent
+	 * RestrictInfo. Every derived child RestrictInfo points to the parent
+	 * RestrictInfo from which it is derived. Parent RestrictInfo maintains a
+	 * list of all derived child RestrictInfos. So only one of the following
+	 * should be set.
+	 */
+	List	   *child_rinfos;		/* RestrictInfos derived for children. */
+	struct RestrictInfo *parent_rinfo;		/* Parent restrictinfo this
+											 * RestrictInf is derived from.
+											 */
 } RestrictInfo;
 
 /*
@@ -1869,6 +1911,21 @@ typedef struct SpecialJoinInfo
 } SpecialJoinInfo;
 
 /*
+ * Partitioned join information
+ *
+ * Saves information about relations which can be joined partition-wise and
+ * thus produce result which is partitioned by the partition scheme of the
+ * relation.
+ */
+typedef struct PartitionedJoin
+{
+	RelOptInfo	   *rel1;
+	RelOptInfo	   *rel2;
+	SpecialJoinInfo	   *sjinfo;	/* SpecialJoinInfo applicable. */
+	List		   *restrictlist;	/* applicable join clauses. */
+} PartitionedJoin;
+
+/*
  * Append-relation info.
  *
  * When we expand an inheritable table or a UNION-ALL subselect into an
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 0e68264..a13eff1 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -30,6 +30,8 @@
 #define DEFAULT_PARALLEL_SETUP_COST  1000.0
 
 #define DEFAULT_EFFECTIVE_CACHE_SIZE  524288	/* measured in pages */
+#define DEFAULT_PARTITION_WISE_PLAN_WEIGHT 1
+#define DEFAULT_SAMPLE_PARTITION_FRACTION 0.01
 
 typedef enum
 {
@@ -66,7 +68,10 @@ extern bool enable_nestloop;
 extern bool enable_material;
 extern bool enable_mergejoin;
 extern bool enable_hashjoin;
+extern bool enable_partition_wise_join;
 extern int	constraint_exclusion;
+extern double partition_wise_plan_weight;
+extern double sample_partition_fraction;
 
 extern double clamp_row_est(double nrows);
 extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 7b41317..81d637a 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -229,6 +229,9 @@ extern LimitPath *create_limit_path(PlannerInfo *root, RelOptInfo *rel,
 				  Path *subpath,
 				  Node *limitOffset, Node *limitCount,
 				  int64 offset_est, int64 count_est);
+extern PartitionJoinPath *create_partition_join_path(PlannerInfo *root,
+								RelOptInfo *rel, List *subpaths,
+								Bitmapset *required_outer, List *pathkeys);
 
 extern Path *reparameterize_path(PlannerInfo *root, Path *path,
 					Relids required_outer,
@@ -271,5 +274,8 @@ extern ParamPathInfo *get_joinrel_parampathinfo(PlannerInfo *root,
 						  List **restrict_clauses);
 extern ParamPathInfo *get_appendrel_parampathinfo(RelOptInfo *appendrel,
 							Relids required_outer);
+extern RelOptInfo *build_child_join_rel(PlannerInfo *root,
+						 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+						 RelOptInfo *parent_joinrel, JoinType jointype);
 
 #endif   /* PATHNODE_H */
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 81a9be7..7ad19be 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -53,6 +53,8 @@ extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 					 List *initial_rels);
 
 extern void generate_gather_paths(PlannerInfo *root, RelOptInfo *rel);
+extern void generate_partition_wise_join_paths(PlannerInfo *root,
+											   RelOptInfo *rel);
 
 #ifdef OPTIMIZER_DEBUG
 extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
@@ -106,6 +108,9 @@ extern bool have_join_order_restriction(PlannerInfo *root,
 							RelOptInfo *rel1, RelOptInfo *rel2);
 extern bool have_dangerous_phv(PlannerInfo *root,
 				   Relids outer_relids, Relids inner_params);
+extern void add_paths_to_child_joinrel(PlannerInfo *root,
+									   RelOptInfo *parent_joinrel,
+									   int child_id);
 
 /*
  * equivclass.c
diff --git a/src/include/optimizer/placeholder.h b/src/include/optimizer/placeholder.h
index 11e6403..8598268 100644
--- a/src/include/optimizer/placeholder.h
+++ b/src/include/optimizer/placeholder.h
@@ -28,5 +28,7 @@ extern void fix_placeholder_input_needed_levels(PlannerInfo *root);
 extern void add_placeholders_to_base_rels(PlannerInfo *root);
 extern void add_placeholders_to_joinrel(PlannerInfo *root, RelOptInfo *joinrel,
 							RelOptInfo *outer_rel, RelOptInfo *inner_rel);
+extern void add_placeholders_to_child_joinrel(PlannerInfo *root,
+							RelOptInfo *childrel, RelOptInfo *parentrel);
 
 #endif   /* PLACEHOLDER_H */
diff --git a/src/include/optimizer/prep.h b/src/include/optimizer/prep.h
index a02e06a..5832130 100644
--- a/src/include/optimizer/prep.h
+++ b/src/include/optimizer/prep.h
@@ -57,5 +57,13 @@ extern Node *adjust_appendrel_attrs(PlannerInfo *root, Node *node,
 
 extern Node *adjust_appendrel_attrs_multilevel(PlannerInfo *root, Node *node,
 								  RelOptInfo *child_rel);
+extern RestrictInfo *build_child_restrictinfo(PlannerInfo *root,
+								 RestrictInfo *rinfo, List *append_rel_infos);
+extern List *build_child_clauses(PlannerInfo *root, List *clauses,
+								 List *append_rel_infos);
+extern List *find_appinfos_by_relids(PlannerInfo *root, Relids relids);
+extern SpecialJoinInfo *build_child_join_sjinfo(PlannerInfo *root,
+									SpecialJoinInfo *parent_sjinfo,
+									Relids left_relids, Relids right_relids);
 
 #endif   /* PREP_H */
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index 18238fa..79779d6 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -2,6 +2,10 @@
 -- PARTITION_JOIN
 -- Test partition-wise join between partitioned tables
 --
+-- Usually partition-wise join paths are chosen when data is large, which would
+-- take regression tests to run longer. So, weigh partition-wise joins cheaper
+-- to force those even for smaller data.
+SET partition_wise_plan_weight to 0.2;
 --
 -- partitioned by a single column
 --
diff --git a/src/test/regress/expected/sysviews.out b/src/test/regress/expected/sysviews.out
index d48abd7..c6c1405 100644
--- a/src/test/regress/expected/sysviews.out
+++ b/src/test/regress/expected/sysviews.out
@@ -70,20 +70,21 @@ select count(*) >= 0 as ok from pg_prepared_xacts;
 -- This is to record the prevailing planner enable_foo settings during
 -- a regression test run.
 select name, setting from pg_settings where name like 'enable%';
-         name         | setting 
-----------------------+---------
- enable_bitmapscan    | on
- enable_hashagg       | on
- enable_hashjoin      | on
- enable_indexonlyscan | on
- enable_indexscan     | on
- enable_material      | on
- enable_mergejoin     | on
- enable_nestloop      | on
- enable_seqscan       | on
- enable_sort          | on
- enable_tidscan       | on
-(11 rows)
+            name            | setting 
+----------------------------+---------
+ enable_bitmapscan          | on
+ enable_hashagg             | on
+ enable_hashjoin            | on
+ enable_indexonlyscan       | on
+ enable_indexscan           | on
+ enable_material            | on
+ enable_mergejoin           | on
+ enable_nestloop            | on
+ enable_partition_wise_join | on
+ enable_seqscan             | on
+ enable_sort                | on
+ enable_tidscan             | on
+(12 rows)
 
 -- Test that the pg_timezone_names and pg_timezone_abbrevs views are
 -- more-or-less working.  We can't test their contents in any great detail
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index 0322f1e..9b2baeb 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -3,6 +3,11 @@
 -- Test partition-wise join between partitioned tables
 --
 
+-- Usually partition-wise join paths are chosen when data is large, which would
+-- take regression tests to run longer. So, weigh partition-wise joins cheaper
+-- to force those even for smaller data.
+SET partition_wise_plan_weight to 0.2;
+
 --
 -- partitioned by a single column
 --
-- 
1.7.9.5

0009-Adjust-join-related-to-code-to-accept-child-relation.patchapplication/octet-stream; name=0009-Adjust-join-related-to-code-to-accept-child-relation.patchDownload
From 4d823ad9179e98a206de9c2b5a8c8ea30415abb6 Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Tue, 7 Feb 2017 17:16:46 +0530
Subject: [PATCH 09/11] Adjust join related to code to accept child relations.

Existing join related code doesn't expect child relations to be joined. This
patch contains various fixes to change that.

1. Uniqe-ifying joining relations.
=================================
For semi-joins we unique-ify the joining relations, which tries to estimate
nummber of unique values using estimate_num_groups(). This function doesn't
expect a Var from a child relation and contained an assertion to that effect.
With partition-wise joins, we may compute a join between child relations. This
commit changes that assertion to include child relation. The function doesn't
need any change other than that to accomodate child relations.

2. OUTER joins require dummy child relations to have targetlist.
================================================================
We need a targetlist defining nullable columns for an outer join, even if the
relation on the nullable side is deemed to be empty. Prior to partition-wise
join an empty child relation never had a targetlist since it was eliminated
from planning. But with partition-wise join an empty child relation may
participate in an outer join with another non-empty child relation. Hence set
targetlist for a child relation even if it's dummy.

3. prepare_sort_from_pathkeys fixes.
====================================
Before partition-wise join feature were never required to be directly sorted,
let's say for merge joins. With partition-wise join feature, the child
relations will participate directly in the join and also need to be sorted
directly for the purpose of merge join. In order to sort a relation, we use
pathkeys. The expression on which to sort a particular relation is provided by
the equivalence member corresponding to that relation in the equivalence class
referred by the pathkeys. Since the code doesn't expect child relations to
bubble up to the sorting, the function prepare_sort_from_pathkeys() skips any
child members (those set with em_is_child) unless the caller specifically asks
for child relations by passing relids. make_sort_from_pathkeys() calls
prepare_sort_from_pathkeys() to create Sort plan for outer and inner plans
without passing relids of the relation to be sorted. For partition-wise joins
the outer and inner plans produce child relations and thus
prepare_sort_from_pathkeys() does not find equivalence members since it skips
child members for the want of relids. This particular instance can be fixed by
passing outer/inner_path->parent->relids to prepare_sort_from_pathkeys().

All the callers of prepare_sort_from_pathkeys() viz.
create_merge_append_plan(), create_merge_append_plan(),
create_windowagg_plan() except make_sort_from_pathkeys() pass relids to
prepare_sort_from_pathkeys(). make_sort_from_pathkeys() as well passes those
with this patch.

make_sort_from_pathkeys() itself doesn't know the relids of relation being
sorted. It just gets the plan. Hence we need to pass relids to
make_sort_from_pathkeys() and thus change each of its callers to pass relids,
if required.

It has two callers as of now.
1. create_sort_plan(PlannerInfo *root, SortPath *best_path, int flags): does
not handle child relations yet, so doesn't need any change.
2. create_mergejoin_plan(PlannerInfo *root, MergePath *best_path):
It requires this change and the relids can be obtained from the outer and inner
path's parent RelOptInfo.

4. Handling em_is_child cases.
==============================
Right now, when comparing relids for child relations, only exact match is
considered. This is fine as long as em_relids has only a single member in it
and the passed in relids has only a single member in it. But with
partition-wise join, relids can have multiple members and em_relids may not
exactly match the given relids. But we need to find the member which covers
subset of given relids.
---
 src/backend/optimizer/path/allpaths.c   |   41 +++++++++++++++++--------------
 src/backend/optimizer/plan/createplan.c |   28 +++++++++++++--------
 src/backend/utils/adt/selfuncs.c        |    3 ++-
 3 files changed, 42 insertions(+), 30 deletions(-)

diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index a024f47..7e806c1 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -960,11 +960,27 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		}
 
 		/*
-		 * We have to copy the parent's targetlist and quals to the child,
-		 * with appropriate substitution of variables.  However, only the
-		 * baserestrictinfo quals are needed before we can check for
-		 * constraint exclusion; so do that first and then check to see if we
-		 * can disregard this child.
+		 * Copy/Modify targetlist. Even if this child is deemed empty, we need
+		 * its targetlist in case it falls on nullable side in a child-join
+		 * because of partition-wise join.
+		 *
+		 * NB: the resulting childrel->reltarget->exprs may contain arbitrary
+		 * expressions, which otherwise would not occur in a rel's targetlist.
+		 * Code that might be looking at an appendrel child must cope with
+		 * such.  (Normally, a rel's targetlist would only include Vars and
+		 * PlaceHolderVars.)  XXX we do not bother to update the cost or width
+		 * fields of childrel->reltarget; not clear if that would be useful.
+		 */
+		childrel->reltarget->exprs = (List *)
+			adjust_appendrel_attrs(root,
+								   (Node *) rel->reltarget->exprs,
+								   appinfo_list);
+
+		/*
+		 * We have to copy the parent's quals to the child, with appropriate
+		 * substitution of variables.  However, only the baserestrictinfo quals
+		 * are needed before we can check for constraint exclusion; so do that
+		 * first and then check to see if we can disregard this child.
 		 *
 		 * The child rel's targetlist might contain non-Var expressions, which
 		 * means that substitution into the quals could produce opportunities
@@ -1091,22 +1107,9 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 			continue;
 		}
 
-		/*
-		 * CE failed, so finish copying/modifying targetlist and join quals.
-		 *
-		 * NB: the resulting childrel->reltarget->exprs may contain arbitrary
-		 * expressions, which otherwise would not occur in a rel's targetlist.
-		 * Code that might be looking at an appendrel child must cope with
-		 * such.  (Normally, a rel's targetlist would only include Vars and
-		 * PlaceHolderVars.)  XXX we do not bother to update the cost or width
-		 * fields of childrel->reltarget; not clear if that would be useful.
-		 */
+		/* CE failed, so finish copying/modifying targetlist and join quals. */
 		childrel->joininfo = build_child_clauses(root, rel->joininfo,
 												 appinfo_list);
-		childrel->reltarget->exprs = (List *)
-			adjust_appendrel_attrs(root,
-								   (Node *) rel->reltarget->exprs,
-								   appinfo_list);
 
 		/*
 		 * We have to make child entries in the EquivalenceClass data
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index fe6b7f8..d0705dc 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -245,7 +245,8 @@ static Plan *prepare_sort_from_pathkeys(Plan *lefttree, List *pathkeys,
 static EquivalenceMember *find_ec_member_for_tle(EquivalenceClass *ec,
 					   TargetEntry *tle,
 					   Relids relids);
-static Sort *make_sort_from_pathkeys(Plan *lefttree, List *pathkeys);
+static Sort *make_sort_from_pathkeys(Plan *lefttree, List *pathkeys,
+									 Relids relids);
 static Sort *make_sort_from_groupcols(List *groupcls,
 						 AttrNumber *grpColIdx,
 						 Plan *lefttree);
@@ -1555,7 +1556,7 @@ create_sort_plan(PlannerInfo *root, SortPath *best_path, int flags)
 	subplan = create_plan_recurse(root, best_path->subpath,
 								  flags | CP_SMALL_TLIST);
 
-	plan = make_sort_from_pathkeys(subplan, best_path->path.pathkeys);
+	plan = make_sort_from_pathkeys(subplan, best_path->path.pathkeys, NULL);
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
@@ -3572,6 +3573,8 @@ create_mergejoin_plan(PlannerInfo *root,
 	ListCell   *lc;
 	ListCell   *lop;
 	ListCell   *lip;
+	Path	   *outer_path = best_path->jpath.outerjoinpath;
+	Path	   *inner_path = best_path->jpath.innerjoinpath;
 
 	/*
 	 * MergeJoin can project, so we don't have to demand exact tlists from the
@@ -3635,8 +3638,10 @@ create_mergejoin_plan(PlannerInfo *root,
 	 */
 	if (best_path->outersortkeys)
 	{
+		Relids		outer_relids = outer_path->parent->relids;
 		Sort	   *sort = make_sort_from_pathkeys(outer_plan,
-												   best_path->outersortkeys);
+												   best_path->outersortkeys,
+												   outer_relids);
 
 		label_sort_with_costsize(root, sort, -1.0);
 		outer_plan = (Plan *) sort;
@@ -3647,8 +3652,10 @@ create_mergejoin_plan(PlannerInfo *root,
 
 	if (best_path->innersortkeys)
 	{
+		Relids		inner_relids = inner_path->parent->relids;
 		Sort	   *sort = make_sort_from_pathkeys(inner_plan,
-												   best_path->innersortkeys);
+												   best_path->innersortkeys,
+												   inner_relids);
 
 		label_sort_with_costsize(root, sort, -1.0);
 		inner_plan = (Plan *) sort;
@@ -5630,11 +5637,11 @@ prepare_sort_from_pathkeys(Plan *lefttree, List *pathkeys,
 					continue;
 
 				/*
-				 * Ignore child members unless they match the rel being
+				 * Ignore child members unless they belong to the rel being
 				 * sorted.
 				 */
 				if (em->em_is_child &&
-					!bms_equal(em->em_relids, relids))
+					!bms_is_subset(em->em_relids, relids))
 					continue;
 
 				sortexpr = em->em_expr;
@@ -5745,10 +5752,10 @@ find_ec_member_for_tle(EquivalenceClass *ec,
 			continue;
 
 		/*
-		 * Ignore child members unless they match the rel being sorted.
+		 * Ignore child members unless they belong to the rel being sorted.
 		 */
 		if (em->em_is_child &&
-			!bms_equal(em->em_relids, relids))
+			!bms_is_subset(em->em_relids, relids))
 			continue;
 
 		/* Match if same expression (after stripping relabel) */
@@ -5769,9 +5776,10 @@ find_ec_member_for_tle(EquivalenceClass *ec,
  *
  *	  'lefttree' is the node which yields input tuples
  *	  'pathkeys' is the list of pathkeys by which the result is to be sorted
+ *	  'relids' is the set of relations required by prepare_sort_from_pathkeys()
  */
 static Sort *
-make_sort_from_pathkeys(Plan *lefttree, List *pathkeys)
+make_sort_from_pathkeys(Plan *lefttree, List *pathkeys, Relids relids)
 {
 	int			numsortkeys;
 	AttrNumber *sortColIdx;
@@ -5781,7 +5789,7 @@ make_sort_from_pathkeys(Plan *lefttree, List *pathkeys)
 
 	/* Compute sort column info, and adjust lefttree as needed */
 	lefttree = prepare_sort_from_pathkeys(lefttree, pathkeys,
-										  NULL,
+										  relids,
 										  NULL,
 										  false,
 										  &numsortkeys,
diff --git a/src/backend/utils/adt/selfuncs.c b/src/backend/utils/adt/selfuncs.c
index fa32e9e..c833846 100644
--- a/src/backend/utils/adt/selfuncs.c
+++ b/src/backend/utils/adt/selfuncs.c
@@ -3427,7 +3427,8 @@ estimate_num_groups(PlannerInfo *root, List *groupExprs, double input_rows,
 		/*
 		 * Sanity check --- don't divide by zero if empty relation.
 		 */
-		Assert(rel->reloptkind == RELOPT_BASEREL);
+		Assert(rel->reloptkind == RELOPT_BASEREL ||
+			   rel->reloptkind == RELOPT_OTHER_MEMBER_REL);
 		if (rel->tuples > 0)
 		{
 			/*
-- 
1.7.9.5

0010-Parameterized-path-fixes.patchapplication/octet-stream; name=0010-Parameterized-path-fixes.patchDownload
From 3d7d73e983c3d18b3ac87fecc33b22fee0cc3448 Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Tue, 7 Feb 2017 17:25:25 +0530
Subject: [PATCH 10/11] Parameterized path fixes.

We do not create merge or hash join paths when the inner path is parameterized
by the outer and vice-versa. Parameterization information in path refers to the
top-most parent relation. Current tests (PATH_PARAM_BY_REL) to avoid joining
such paths fail while joining child relations; the paths from either child may
be paramterized by other's parent. Modify the tests to consider paths
parameterized by parent as parameterized by any of its child.

If the inner path is parameterized by outer path, we can create a nested loop
join using those two paths with inner relation parameterized by the outer
relation. For LATERAL JOINs this is the only legal way to plan a join. In case
of partitioned joins, the lateral references refer to the topmost parent and
hence inner paths are parameterized by the topmost parent. In such cases, it's
possible to translate the inner path to be parameterized by
the child and create nested loop join. When presented with a pair of child
relation paths, where the inner paths is parameterized by the parent of outer
child, this patch translates the path to be parameterized by the outer child
and creates a nested loop join path.

The function reparameterize_path_by_child() needs to call adjust_relid_set() to
substitute parent relids by child relids in Path::param_info::ppi_req_outer.
Hence "extern"alized that function. Since there is already another
static adjust_relid_set() in rewriteManip.c, renamed this one to
adjust_child_relids().

Also "extern"alized find_param_path_info() required by
reparameterize_path_by_child().
---
 src/backend/optimizer/path/joinpath.c  |   33 +++++-
 src/backend/optimizer/prep/prepunion.c |   42 ++++----
 src/backend/optimizer/util/pathnode.c  |  180 ++++++++++++++++++++++++++++++++
 src/backend/optimizer/util/relnode.c   |    2 -
 src/include/optimizer/pathnode.h       |    4 +
 src/include/optimizer/prep.h           |    1 +
 6 files changed, 237 insertions(+), 25 deletions(-)

diff --git a/src/backend/optimizer/path/joinpath.c b/src/backend/optimizer/path/joinpath.c
index f80fb25..4d4a183 100644
--- a/src/backend/optimizer/path/joinpath.c
+++ b/src/backend/optimizer/path/joinpath.c
@@ -25,9 +25,19 @@
 /* Hook for plugins to get control in add_paths_to_joinrel() */
 set_join_pathlist_hook_type set_join_pathlist_hook = NULL;
 
-#define PATH_PARAM_BY_REL(path, rel)  \
+/*
+ * Paths parameterized by the parent can be considered to be parameterized by
+ * any of its child.
+ */
+#define PATH_PARAM_BY_PARENT(path, rel)	\
+	((path)->param_info && bms_overlap(PATH_REQ_OUTER(path),	\
+									   (rel)->top_parent_relids))
+#define PATH_PARAM_BY_REL_SELF(path, rel)  \
 	((path)->param_info && bms_overlap(PATH_REQ_OUTER(path), (rel)->relids))
 
+#define PATH_PARAM_BY_REL(path, rel)	\
+	(PATH_PARAM_BY_REL_SELF(path, rel) || PATH_PARAM_BY_PARENT(path, rel))
+
 static void sort_inner_and_outer(PlannerInfo *root, RelOptInfo *joinrel,
 					 RelOptInfo *outerrel, RelOptInfo *innerrel,
 					 JoinType jointype, JoinPathExtraData *extra);
@@ -301,6 +311,27 @@ try_nestloop_path(PlannerInfo *root,
 	JoinCostWorkspace workspace;
 
 	/*
+	 * Since result produced by a child is part of the result produced by its
+	 * topmost parent and has same properties, the parameters representing that
+	 * parent may be substituted by values from a child. Hence expressions and
+	 * hence paths using those expressions, parameterized by a parent can be
+	 * said to be parameterized by any of its child.  For a join between child
+	 * relations, if the inner path is parameterized by the parent of the outer
+	 * relation,  create a nestloop join path with inner relation parameterized
+	 * by the outer relation by translating the inner path to be parameterized
+	 * by the outer child relation.
+	 */
+	if (PATH_PARAM_BY_PARENT(inner_path, outer_path->parent))
+	{
+		inner_path = reparameterize_path_by_child(root, inner_path,
+												   outer_path->parent);
+
+		/* If we could not translate the path, don't produce nest loop path. */
+		if (!inner_path)
+			return;
+	}
+
+	/*
 	 * Check to see if proposed path is still parameterized, and reject if the
 	 * parameterization wouldn't be sensible --- unless allow_star_schema_join
 	 * says to allow it anyway.  Also, we must reject if have_dangerous_phv
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index 676204f..d459e95 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -109,7 +109,6 @@ static Node *adjust_appendrel_attrs_mutator(Node *node,
 							   adjust_appendrel_attrs_context *context);
 static List *adjust_inherited_tlist(List *tlist,
 					   AppendRelInfo *context);
-static Relids adjust_relid_set(Relids relids, List *append_rel_infos);
 
 
 /*
@@ -1951,7 +1950,7 @@ adjust_appendrel_attrs_mutator(Node *node,
 														 (void *) context);
 		/* now fix PlaceHolderVar's relid sets */
 		if (phv->phlevelsup == 0)
-			phv->phrels = adjust_relid_set(phv->phrels, context->appinfos);
+			phv->phrels = adjust_child_relids(phv->phrels, context->appinfos);
 		return (Node *) phv;
 	}
 	/* Shouldn't need to handle planner auxiliary nodes here */
@@ -1982,17 +1981,17 @@ adjust_appendrel_attrs_mutator(Node *node,
 			adjust_appendrel_attrs_mutator((Node *) oldinfo->orclause, context);
 
 		/* adjust relid sets too */
-		newinfo->clause_relids = adjust_relid_set(oldinfo->clause_relids,
+		newinfo->clause_relids = adjust_child_relids(oldinfo->clause_relids,
 													 context->appinfos);
-		newinfo->required_relids = adjust_relid_set(oldinfo->required_relids,
+		newinfo->required_relids = adjust_child_relids(oldinfo->required_relids,
 													 context->appinfos);
-		newinfo->outer_relids = adjust_relid_set(oldinfo->outer_relids,
+		newinfo->outer_relids = adjust_child_relids(oldinfo->outer_relids,
 													 context->appinfos);
-		newinfo->nullable_relids = adjust_relid_set(oldinfo->nullable_relids,
+		newinfo->nullable_relids = adjust_child_relids(oldinfo->nullable_relids,
 													   context->appinfos);
-		newinfo->left_relids = adjust_relid_set(oldinfo->left_relids,
+		newinfo->left_relids = adjust_child_relids(oldinfo->left_relids,
 												   context->appinfos);
-		newinfo->right_relids = adjust_relid_set(oldinfo->right_relids,
+		newinfo->right_relids = adjust_child_relids(oldinfo->right_relids,
 													context->appinfos);
 
 		/*
@@ -2026,15 +2025,18 @@ adjust_appendrel_attrs_mutator(Node *node,
 
 /*
  * Replace parent relids by child relids in the copy of given relid set
- * according to the given list of AppendRelInfos. The given relid set is
- * returned as is if it contains no parent in the given list, otherwise, the
- * given relid set is not changed.
+ * according to the given list of AppendRelInfos.
  */
 Relids
-adjust_relid_set(Relids relids, List *append_rel_infos)
+adjust_child_relids(Relids relids, List *append_rel_infos)
 {
 	ListCell   *lc;
-	Bitmapset  *result = NULL;
+
+	/*
+	 * The new relids set may be expected to be in a memory context different
+	 * from the given one. Make a copy here.
+	 */
+	Bitmapset  *result = bms_copy(relids);
 
 	foreach (lc, append_rel_infos)
 	{
@@ -2043,10 +2045,6 @@ adjust_relid_set(Relids relids, List *append_rel_infos)
 		/* Remove parent, add child */
 		if (bms_is_member(appinfo->parent_relid, relids))
 		{
-			/* Make a copy if we are changing the set. */
-			if (!result)
-				result = bms_copy(relids);
-
 			result = bms_del_member(result, appinfo->parent_relid);
 			result = bms_add_member(result, appinfo->child_relid);
 		}
@@ -2202,7 +2200,7 @@ build_child_restrictinfo(PlannerInfo *root, RestrictInfo *rinfo,
 	RestrictInfo   *child_rinfo;
 	MemoryContext	old_context;
 
-	child_required_relids = adjust_relid_set(rinfo->required_relids,
+	child_required_relids = adjust_child_relids(rinfo->required_relids,
 												append_rel_infos);
 
 
@@ -2313,13 +2311,13 @@ build_child_join_sjinfo(PlannerInfo *root, SpecialJoinInfo *parent_sjinfo,
 
 	memcpy(sjinfo, parent_sjinfo, sizeof(SpecialJoinInfo));
 
-	sjinfo->min_lefthand = adjust_relid_set(sjinfo->min_lefthand,
+	sjinfo->min_lefthand = adjust_child_relids(sjinfo->min_lefthand,
 											   left_appinfos);
-	sjinfo->min_righthand = adjust_relid_set(sjinfo->min_righthand,
+	sjinfo->min_righthand = adjust_child_relids(sjinfo->min_righthand,
 												right_appinfos);
-	sjinfo->syn_lefthand = adjust_relid_set(sjinfo->syn_lefthand,
+	sjinfo->syn_lefthand = adjust_child_relids(sjinfo->syn_lefthand,
 											   left_appinfos);
-	sjinfo->syn_righthand = adjust_relid_set(sjinfo->syn_righthand,
+	sjinfo->syn_righthand = adjust_child_relids(sjinfo->syn_righthand,
 												right_appinfos);
 
 	/*
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index d861a49..f322320 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -3388,3 +3388,183 @@ reparameterize_path(PlannerInfo *root, Path *path,
 	}
 	return NULL;
 }
+
+/*
+ * reparameterize_path_by_child
+ * 		Given a path parameterized by the parent of the given relation,
+ * 		translate the path to be parameterized by the given child relation.
+ *
+ * The function creates a new path of the same type as the given path, but
+ * parameterized by the given child relation. If it can not reparameterize the
+ * path as required, it returns NULL.
+ *
+ * The cost, number of rows, width and parallel path properties depend upon
+ * path->parent, which does not change during the translation. Hence those
+ * members are copied as they are.
+ */
+
+Path *
+reparameterize_path_by_child(PlannerInfo *root, Path *path,
+							  RelOptInfo *child_rel)
+{
+	Path	   *new_path;
+	ParamPathInfo   *new_ppi;
+	ParamPathInfo   *old_ppi;
+	List	   *child_aris;
+	Relids		required_outer;
+
+	/*
+	 * If the path is not parameterized by parent of the given relation or it it
+	 * doesn't need reparameterization.
+	 */
+	if (!path->param_info ||
+		!bms_overlap(PATH_REQ_OUTER(path), child_rel->top_parent_relids))
+	return path;
+
+	switch (nodeTag(path))
+	{
+		case T_Path:
+			new_path = makeNode(Path);
+			memcpy(new_path, path, sizeof(Path));
+			break;
+
+		case T_HashPath:
+			new_path = (Path *) makeNode(HashPath);
+			memcpy(new_path, path, sizeof(HashPath));
+			break;
+
+		case T_MergePath:
+			new_path = (Path *) makeNode(MergePath);
+			memcpy(new_path, path, sizeof(MergePath));
+			break;
+
+		case T_NestPath:
+			new_path = (Path *) makeNode(NestPath);
+			memcpy(new_path, path, sizeof(NestPath));
+			break;
+
+		case T_IndexPath:
+			new_path = (Path *) makeNode(IndexPath);
+			memcpy(new_path, path, sizeof(IndexPath));
+			break;
+
+		case T_AppendPath:
+			new_path = (Path *) makeNode(AppendPath);
+			memcpy(new_path, path, sizeof(AppendPath));
+			break;
+
+		/*
+		 * TODO:
+		 * If this method of translation is fine add more path types here.
+		 */
+
+		default:
+			/* Path type unsupported by this function. */
+			return NULL;
+	}
+
+	/*
+	 * Gather AppendRelInfos of the base partition relations in the outer child
+	 * relation. We need those for translating parent path to that of child by
+	 * substituting parent Var nodes and relids with those of children.
+	 */
+	child_aris = find_appinfos_by_relids(root, child_rel->relids);
+
+	/* Adjust the parameterization information. */
+	old_ppi = new_path->param_info;
+	required_outer = adjust_child_relids(old_ppi->ppi_req_outer, child_aris);
+
+	/* If we already have a PPI for this parameterization, just return it */
+	new_ppi = find_param_path_info(new_path->parent, required_outer);
+
+	/* If not build a new one and link it to the list of PPIs. */
+	if (!new_ppi)
+	{
+		new_ppi = makeNode(ParamPathInfo);
+		new_ppi->ppi_req_outer = required_outer;
+		new_ppi->ppi_rows = old_ppi->ppi_rows;
+		new_ppi->ppi_clauses = build_child_clauses(root, old_ppi->ppi_clauses,
+												   child_aris);
+		new_path->parent->ppilist = lappend(new_path->parent->ppilist, new_ppi);
+	}
+	else
+		bms_free(required_outer);
+
+	new_path->param_info = new_ppi;
+
+	/*
+	 * Adjust the path target if the parent of the outer relation is referenced
+	 * in the targetlist. This can happen when only the parent of outer relation is
+	 * laterally referenced in this relation.
+	 */
+	if (bms_overlap(path->parent->lateral_relids, child_rel->top_parent_relids))
+	{
+		MemoryContext	old_context;
+
+		/*
+		 * Allocate the target in planner's context, since they are copies as
+		 * is from path while creating plans.
+		 */
+		old_context = MemoryContextSwitchTo(root->planner_cxt);
+		new_path->pathtarget = copy_pathtarget(new_path->pathtarget);
+		new_path->pathtarget->exprs = (List *) adjust_appendrel_attrs(root,
+											(Node *) new_path->pathtarget->exprs,
+																	child_aris);
+		MemoryContextSwitchTo(old_context);
+	}
+
+	/*
+	 * Change parameterization of subpaths recursively. Also carry out any
+	 * pathtype specific adjustments.
+	 */
+	switch (nodeTag(path))
+	{
+		case T_HashPath:
+		case T_MergePath:
+		case T_NestPath:
+			{
+				JoinPath *jpath = (JoinPath *)new_path;
+
+				jpath->outerjoinpath = reparameterize_path_by_child(root,
+														 jpath->outerjoinpath,
+														 child_rel);
+				jpath->innerjoinpath = reparameterize_path_by_child(root,
+														 jpath->innerjoinpath,
+														 child_rel);
+				jpath->joinrestrictinfo = build_child_clauses(root,
+															  jpath->joinrestrictinfo,
+															  child_aris);
+			}
+			break;
+
+		case T_AppendPath:
+			{
+				AppendPath	*apath = (AppendPath *)new_path;
+				List		*subpaths = NIL;
+				ListCell	*lc;
+
+				foreach (lc, apath->subpaths)
+					subpaths = lappend(subpaths,
+									   reparameterize_path_by_child(root,
+																	lfirst(lc),
+																	child_rel));
+				apath->subpaths = subpaths;
+			}
+
+		case T_IndexPath:
+			{
+				IndexPath *ipath = (IndexPath *)new_path;
+
+				ipath->indexclauses = build_child_clauses(root, ipath->indexclauses,
+														  child_aris);
+				ipath->indexquals = build_child_clauses(root, ipath->indexquals,
+														child_aris);
+			}
+
+		default:
+			/* Nothing to do. */
+			break;
+	}
+
+	return new_path;
+}
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 1eed987..46eea02 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -53,8 +53,6 @@ static List *subbuild_joinrel_joinlist(RelOptInfo *joinrel,
 static void set_foreign_rel_properties(RelOptInfo *joinrel,
 						   RelOptInfo *outer_rel, RelOptInfo *inner_rel);
 static void add_join_rel(PlannerInfo *root, RelOptInfo *joinrel);
-extern ParamPathInfo *find_param_path_info(RelOptInfo *rel,
-									  Relids required_outer);
 static void build_joinrel_partition_info(RelOptInfo *joinrel,
 						  RelOptInfo *outer_rel, RelOptInfo *inner_rel,
 						  JoinType jointype);
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 81d637a..b9f5b11 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -236,6 +236,8 @@ extern PartitionJoinPath *create_partition_join_path(PlannerInfo *root,
 extern Path *reparameterize_path(PlannerInfo *root, Path *path,
 					Relids required_outer,
 					double loop_count);
+extern Path *reparameterize_path_by_child(PlannerInfo *root, Path *path,
+					RelOptInfo *child_rel);
 
 /*
  * prototypes for relnode.c
@@ -277,5 +279,7 @@ extern ParamPathInfo *get_appendrel_parampathinfo(RelOptInfo *appendrel,
 extern RelOptInfo *build_child_join_rel(PlannerInfo *root,
 						 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
 						 RelOptInfo *parent_joinrel, JoinType jointype);
+extern ParamPathInfo *find_param_path_info(RelOptInfo *rel,
+									  Relids required_outer);
 
 #endif   /* PATHNODE_H */
diff --git a/src/include/optimizer/prep.h b/src/include/optimizer/prep.h
index 5832130..0347b37 100644
--- a/src/include/optimizer/prep.h
+++ b/src/include/optimizer/prep.h
@@ -65,5 +65,6 @@ extern List *find_appinfos_by_relids(PlannerInfo *root, Relids relids);
 extern SpecialJoinInfo *build_child_join_sjinfo(PlannerInfo *root,
 									SpecialJoinInfo *parent_sjinfo,
 									Relids left_relids, Relids right_relids);
+extern Relids adjust_child_relids(Relids relids, List *append_rel_infos);
 
 #endif   /* PREP_H */
-- 
1.7.9.5

0011-Use-IS_JOIN_REL-instead-of-RELOPT_JOINREL.patchapplication/octet-stream; name=0011-Use-IS_JOIN_REL-instead-of-RELOPT_JOINREL.patchDownload
From 59f12da00e6878169559def32adf896e38a00aa4 Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Wed, 8 Feb 2017 14:42:15 +0530
Subject: [PATCH 11/11] Use IS_JOIN_REL() instead of RELOPT_JOINREL

FDW code uses RELOPT_JOINREL to check whether a given relation is a join or
not. Partition-wise joins create child-join relations, which are marked as
RELOPT_OTHER_JOINREL. Macro IS_JOIN_REL() returns true for both of those kinds,
replace RELOPT_JOINREL tests with IS_JOIN_REL() test.

Similarly replace RELOPT_OTHER_MEMBER_REL test with IS_OTHER_REL() where we
want to test for child relations of all kinds.
---
 contrib/postgres_fdw/deparse.c      |   10 +++++-----
 contrib/postgres_fdw/postgres_fdw.c |   10 ++++++----
 src/backend/foreign/foreign.c       |    6 +++---
 3 files changed, 14 insertions(+), 12 deletions(-)

diff --git a/contrib/postgres_fdw/deparse.c b/contrib/postgres_fdw/deparse.c
index d2b94aa..a2171d7 100644
--- a/contrib/postgres_fdw/deparse.c
+++ b/contrib/postgres_fdw/deparse.c
@@ -911,7 +911,7 @@ deparseSelectStmtForRel(StringInfo buf, PlannerInfo *root, RelOptInfo *rel,
 	 * We handle relations for foreign tables, joins between those and upper
 	 * relations.
 	 */
-	Assert(rel->reloptkind == RELOPT_JOINREL ||
+	Assert(IS_JOIN_REL(rel) ||
 		   rel->reloptkind == RELOPT_BASEREL ||
 		   rel->reloptkind == RELOPT_OTHER_MEMBER_REL ||
 		   rel->reloptkind == RELOPT_UPPER_REL);
@@ -990,7 +990,7 @@ deparseSelectSql(List *tlist, List **retrieved_attrs, deparse_expr_cxt *context)
 	 */
 	appendStringInfoString(buf, "SELECT ");
 
-	if (foreignrel->reloptkind == RELOPT_JOINREL ||
+	if (IS_JOIN_REL(foreignrel) ||
 		foreignrel->reloptkind == RELOPT_UPPER_REL)
 	{
 		/* For a join relation use the input tlist */
@@ -1030,7 +1030,7 @@ deparseFromExpr(List *quals, deparse_expr_cxt *context)
 
 	/* For upper relations, scanrel must be either a joinrel or a baserel */
 	Assert(context->foreignrel->reloptkind != RELOPT_UPPER_REL ||
-		   scanrel->reloptkind == RELOPT_JOINREL ||
+		   IS_JOIN_REL(scanrel) ||
 		   scanrel->reloptkind == RELOPT_BASEREL);
 
 	/* Construct FROM clause */
@@ -1178,7 +1178,7 @@ deparseLockingClause(deparse_expr_cxt *context)
 			appendStringInfoString(buf, " FOR UPDATE");
 
 			/* Add the relation alias if we are here for a join relation */
-			if (rel->reloptkind == RELOPT_JOINREL)
+			if (IS_JOIN_REL(rel))
 				appendStringInfo(buf, " OF %s%d", REL_ALIAS_PREFIX, relid);
 		}
 		else
@@ -1342,7 +1342,7 @@ deparseFromExprForRel(StringInfo buf, PlannerInfo *root, RelOptInfo *foreignrel,
 {
 	PgFdwRelationInfo *fpinfo = (PgFdwRelationInfo *) foreignrel->fdw_private;
 
-	if (foreignrel->reloptkind == RELOPT_JOINREL)
+	if (IS_JOIN_REL(foreignrel))
 	{
 		RelOptInfo *rel_o = fpinfo->outerrel;
 		RelOptInfo *rel_i = fpinfo->innerrel;
diff --git a/contrib/postgres_fdw/postgres_fdw.c b/contrib/postgres_fdw/postgres_fdw.c
index 5d270b9..2487f26 100644
--- a/contrib/postgres_fdw/postgres_fdw.c
+++ b/contrib/postgres_fdw/postgres_fdw.c
@@ -723,6 +723,8 @@ get_useful_ecs_for_relation(PlannerInfo *root, RelOptInfo *rel)
 	/* If this is a child rel, we must use the topmost parent rel to search. */
 	if (rel->reloptkind == RELOPT_OTHER_MEMBER_REL)
 		relids = find_childrel_top_parent(root, rel)->relids;
+	else if (rel->reloptkind == RELOPT_OTHER_JOINREL)
+		relids = rel->top_parent_relids;
 	else
 		relids = rel->relids;
 
@@ -1181,7 +1183,7 @@ postgresGetForeignPlan(PlannerInfo *root,
 			local_exprs = lappend(local_exprs, rinfo->clause);
 	}
 
-	if (foreignrel->reloptkind == RELOPT_JOINREL ||
+	if (IS_JOIN_REL(foreignrel) ||
 		foreignrel->reloptkind == RELOPT_UPPER_REL)
 	{
 		/* For a join relation, get the conditions from fdw_private structure */
@@ -1247,7 +1249,7 @@ postgresGetForeignPlan(PlannerInfo *root,
 							 remote_conds,
 							 retrieved_attrs,
 							 makeInteger(fpinfo->fetch_size));
-	if (foreignrel->reloptkind == RELOPT_JOINREL ||
+	if (IS_JOIN_REL(foreignrel) ||
 		foreignrel->reloptkind == RELOPT_UPPER_REL)
 		fdw_private = lappend(fdw_private,
 							  makeString(fpinfo->relation_name->data));
@@ -2527,7 +2529,7 @@ estimate_path_cost_size(PlannerInfo *root,
 						   &remote_param_join_conds, &local_param_join_conds);
 
 		/* Build the list of columns to be fetched from the foreign server. */
-		if (foreignrel->reloptkind == RELOPT_JOINREL ||
+		if (IS_JOIN_REL(foreignrel) ||
 			foreignrel->reloptkind == RELOPT_UPPER_REL)
 			fdw_scan_tlist = build_tlist_to_deparse(foreignrel);
 		else
@@ -2609,7 +2611,7 @@ estimate_path_cost_size(PlannerInfo *root,
 			startup_cost = fpinfo->rel_startup_cost;
 			run_cost = fpinfo->rel_total_cost - fpinfo->rel_startup_cost;
 		}
-		else if (foreignrel->reloptkind == RELOPT_JOINREL)
+		else if (IS_JOIN_REL(foreignrel))
 		{
 			PgFdwRelationInfo *fpinfo_i;
 			PgFdwRelationInfo *fpinfo_o;
diff --git a/src/backend/foreign/foreign.c b/src/backend/foreign/foreign.c
index fdb4f71..e8ca7df 100644
--- a/src/backend/foreign/foreign.c
+++ b/src/backend/foreign/foreign.c
@@ -717,7 +717,7 @@ GetExistingLocalJoinPath(RelOptInfo *joinrel)
 {
 	ListCell   *lc;
 
-	Assert(joinrel->reloptkind == RELOPT_JOINREL);
+	Assert(IS_JOIN_REL(joinrel));
 
 	foreach(lc, joinrel->pathlist)
 	{
@@ -782,7 +782,7 @@ GetExistingLocalJoinPath(RelOptInfo *joinrel)
 			ForeignPath *foreign_path;
 
 			foreign_path = (ForeignPath *) joinpath->outerjoinpath;
-			if (foreign_path->path.parent->reloptkind == RELOPT_JOINREL)
+			if (IS_JOIN_REL(foreign_path->path.parent))
 				joinpath->outerjoinpath = foreign_path->fdw_outerpath;
 		}
 
@@ -791,7 +791,7 @@ GetExistingLocalJoinPath(RelOptInfo *joinrel)
 			ForeignPath *foreign_path;
 
 			foreign_path = (ForeignPath *) joinpath->innerjoinpath;
-			if (foreign_path->path.parent->reloptkind == RELOPT_JOINREL)
+			if (IS_JOIN_REL(foreign_path->path.parent))
 				joinpath->innerjoinpath = foreign_path->fdw_outerpath;
 		}
 
-- 
1.7.9.5

#34Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Ashutosh Bapat (#33)
14 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

Here is set of patches with support for partition-wise join between
multi-level partitioned tables.

On Fri, Feb 10, 2017 at 11:19 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Fixed a problem with the way qsort was being used in the earlier set
of patches. Attached PFA the set of patches with that fixed.

This fix is included.

On Thu, Feb 9, 2017 at 4:20 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Per your suggestion I have split the patch into many smaller patches.

0001-Refactor-set_append_rel_pathlist.patch
0002-Refactor-make_join_rel.patch
0003-Refactor-adjust_appendrel_attrs.patch
0004-Refactor-build_join_rel.patch
0005-Add-function-find_param_path_info.patch

These four refactor existing code.

0006-Canonical-partition-scheme.patch
0007-Partition-wise-join-tests.patch -- just tests, they fail
0008-Partition-wise-join.patch -- actual patch implementing
partition-wise join, still some tests fail\

0009-Adjust-join-related-to-code-to-accept-child-relation.patch
0010-Parameterized-path-fixes.patch
0011-Use-IS_JOIN_REL-instead-of-RELOPT_JOINREL.patch

patch to translate partition hierarchy into inheritance hierarchy
without flattening

0012-Multi-level-partitioned-table-expansion.patch

patches for multi-level partition-wise join support

0013-Multi-level-partition-wise-join-tests.patch
0014-Multi-level-partition-wise-join-support.patch

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

Attachments:

0001-Refactor-set_append_rel_pathlist.patchapplication/octet-stream; name=0001-Refactor-set_append_rel_pathlist.patchDownload
From cf978fcadc1985c27783c114b8a367fe795efe9e Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Fri, 3 Feb 2017 14:45:37 +0530
Subject: [PATCH 01/14] Refactor set_append_rel_pathlist.

set_append_rel_pathlist() creates paths for each child relation and then
creates append paths for the "append" relation. Right now only a base relation
can have children, but with partition-wise join and aggregation a join or an
upper relation can have children as well. While the code to add paths to the
child relations differs for base, join and upper child relations, the code to
create append paths can be shared by all the three relations. Hence separating
it into a new function add_paths_to_append_rel() so that it can be re-used for
all kinds of relations.
---
 src/backend/optimizer/path/allpaths.c |   51 ++++++++++++++++++++++++++-------
 1 file changed, 41 insertions(+), 10 deletions(-)

diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 5c18987..d797d6a 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -127,6 +127,8 @@ static void recurse_push_qual(Node *setOp, Query *topquery,
 				  RangeTblEntry *rte, Index rti, Node *qual);
 static void remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel);
 static int	compute_parallel_worker(RelOptInfo *rel, BlockNumber pages);
+static void add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
+									List *live_childrels);
 
 
 /*
@@ -1169,19 +1171,11 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 {
 	int			parentRTindex = rti;
 	List	   *live_childrels = NIL;
-	List	   *subpaths = NIL;
-	bool		subpaths_valid = true;
-	List	   *partial_subpaths = NIL;
-	bool		partial_subpaths_valid = true;
-	List	   *all_child_pathkeys = NIL;
-	List	   *all_child_outers = NIL;
 	ListCell   *l;
 
 	/*
 	 * Generate access paths for each member relation, and remember the
-	 * cheapest path for each one.  Also, identify all pathkeys (orderings)
-	 * and parameterizations (required_outer sets) available for the member
-	 * relations.
+	 * non-dummy children.
 	 */
 	foreach(l, root->append_rel_list)
 	{
@@ -1189,7 +1183,6 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 		int			childRTindex;
 		RangeTblEntry *childRTE;
 		RelOptInfo *childrel;
-		ListCell   *lcp;
 
 		/* append_rel_list contains all append rels; ignore others */
 		if (appinfo->parent_relid != parentRTindex)
@@ -1224,6 +1217,44 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 		 * Child is live, so add it to the live_childrels list for use below.
 		 */
 		live_childrels = lappend(live_childrels, childrel);
+	}
+
+	/* Add Append/MergeAppend paths to the "append" relation. */
+	add_paths_to_append_rel(root, rel, live_childrels);
+}
+
+
+/*
+ * add_paths_to_append_rel
+ *		Generate Append/MergeAppend paths for given "append" relation.
+ *
+ * The function collects all parameterizations and orderings supported by the
+ * non-dummy children. For every such parameterization or ordering, it creates
+ * an append path collecting one path from each non-dummy child with given
+ * parameterization or ordering. Similarly it collects partial paths from
+ * non-dummy children to create partial append paths.
+ */
+static void
+add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
+						List *live_childrels)
+{
+	List	   *subpaths = NIL;
+	bool		subpaths_valid = true;
+	List	   *partial_subpaths = NIL;
+	bool		partial_subpaths_valid = true;
+	List	   *all_child_pathkeys = NIL;
+	List	   *all_child_outers = NIL;
+	ListCell   *l;
+
+	/*
+	 * For every non-dummy child, remember the cheapest path.  Also, identify
+	 * all pathkeys (orderings) and parameterizations (required_outer sets)
+	 * available for the non-dummy member relations.
+	 */
+	foreach (l, live_childrels)
+	{
+		RelOptInfo *childrel = lfirst(l);
+		ListCell   *lcp;
 
 		/*
 		 * If child has an unparameterized cheapest-total path, add that to
-- 
1.7.9.5

0002-Refactor-make_join_rel.patchapplication/octet-stream; name=0002-Refactor-make_join_rel.patchDownload
From 80861ce9a56d639833f015392bb6d4035a9deda2 Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Fri, 3 Feb 2017 15:41:39 +0530
Subject: [PATCH 02/14] Refactor make_join_rel().

The code in make_join_rel() to add paths to join relation for a given pair of
joining relations can be re-used to add paths to a child join relation, which
do not need the other functionality offered by make_join_rel(). Separate this
code into populate_joinrel_with_paths(). This patch does just refactors
make_join_rel() to pave the way for partition-wise join.
---
 src/backend/optimizer/path/joinrels.c |   28 ++++++++++++++++++++++++----
 1 file changed, 24 insertions(+), 4 deletions(-)

diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 6f3c20b..936ee0c 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -32,6 +32,9 @@ static bool is_dummy_rel(RelOptInfo *rel);
 static void mark_dummy_rel(RelOptInfo *rel);
 static bool restriction_is_constant_false(List *restrictlist,
 							  bool only_pushed_down);
+static void populate_joinrel_with_paths(PlannerInfo *root, RelOptInfo *rel1,
+							RelOptInfo *rel2, RelOptInfo *joinrel,
+							SpecialJoinInfo *sjinfo, List *restrictlist);
 
 
 /*
@@ -724,6 +727,27 @@ make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 		return joinrel;
 	}
 
+	/* Add paths to the join relation. */
+	populate_joinrel_with_paths(root, rel1, rel2, joinrel, sjinfo,
+								restrictlist);
+
+	bms_free(joinrelids);
+
+	return joinrel;
+}
+
+/*
+ * populate_joinrel_with_paths
+ *	  Add paths to the given joinrel for given pair of joining relations. The
+ *	  SpecialJoinInfo provides details about the join and the restrictlist
+ *	  contains the join clauses and the other clauses applicable for given pair
+ *	  of the joining relations.
+ */
+static void
+populate_joinrel_with_paths(PlannerInfo *root, RelOptInfo *rel1,
+							RelOptInfo *rel2, RelOptInfo *joinrel,
+							SpecialJoinInfo *sjinfo, List *restrictlist)
+{
 	/*
 	 * Consider paths using each rel as both outer and inner.  Depending on
 	 * the join type, a provably empty outer or inner rel might mean the join
@@ -868,10 +892,6 @@ make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 			elog(ERROR, "unrecognized join type: %d", (int) sjinfo->jointype);
 			break;
 	}
-
-	bms_free(joinrelids);
-
-	return joinrel;
 }
 
 
-- 
1.7.9.5

0003-Refactor-adjust_appendrel_attrs.patchapplication/octet-stream; name=0003-Refactor-adjust_appendrel_attrs.patchDownload
From 74d38c91f9c37f8c6ede2a1a63ae3fec6731a10d Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Fri, 3 Feb 2017 17:22:52 +0530
Subject: [PATCH 03/14] Refactor adjust_appendrel_attrs.

adjust_appendrel_attrs() is used to translate nodes for a parent relation to
those for a child relation by replacing the parent specific nodes like Var
nodes with corresponding nodes specific to the child. Right now this function
works with a single parent-child pair. For partition-wise join and
partition-wise aggregation/grouping, we require to translate nodes for multiple
parent-child pairs. This patch modifies adjust_appendrel_attrs() to work with
multiple parent-child pairs.
---
 src/backend/optimizer/path/allpaths.c   |    7 +-
 src/backend/optimizer/path/equivclass.c |    2 +-
 src/backend/optimizer/plan/planner.c    |    2 +-
 src/backend/optimizer/prep/prepunion.c  |  161 ++++++++++++++++++++++---------
 src/include/optimizer/prep.h            |    2 +-
 5 files changed, 121 insertions(+), 53 deletions(-)

diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index d797d6a..d8fac14 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -862,6 +862,7 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		ListCell   *parentvars;
 		ListCell   *childvars;
 		ListCell   *lc;
+		List	   *appinfo_list = list_make1(appinfo);
 
 		/* append_rel_list contains all append rels; ignore others */
 		if (appinfo->parent_relid != parentRTindex)
@@ -903,7 +904,7 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 			Assert(IsA(rinfo, RestrictInfo));
 			childqual = adjust_appendrel_attrs(root,
 											   (Node *) rinfo->clause,
-											   appinfo);
+											   appinfo_list);
 			childqual = eval_const_expressions(root, childqual);
 			/* check for flat-out constant */
 			if (childqual && IsA(childqual, Const))
@@ -1022,11 +1023,11 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		childrel->joininfo = (List *)
 			adjust_appendrel_attrs(root,
 								   (Node *) rel->joininfo,
-								   appinfo);
+								   appinfo_list);
 		childrel->reltarget->exprs = (List *)
 			adjust_appendrel_attrs(root,
 								   (Node *) rel->reltarget->exprs,
-								   appinfo);
+								   appinfo_list);
 
 		/*
 		 * We have to make child entries in the EquivalenceClass data
diff --git a/src/backend/optimizer/path/equivclass.c b/src/backend/optimizer/path/equivclass.c
index a329dd1..bcce142 100644
--- a/src/backend/optimizer/path/equivclass.c
+++ b/src/backend/optimizer/path/equivclass.c
@@ -2111,7 +2111,7 @@ add_child_rel_equivalences(PlannerInfo *root,
 				child_expr = (Expr *)
 					adjust_appendrel_attrs(root,
 										   (Node *) cur_em->em_expr,
-										   appinfo);
+										   list_make1(appinfo));
 
 				/*
 				 * Transform em_relids to match.  Note we do *not* do
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 881742f..24a48b8 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -1085,7 +1085,7 @@ inheritance_planner(PlannerInfo *root)
 		subroot->parse = (Query *)
 			adjust_appendrel_attrs(root,
 								   (Node *) parse,
-								   appinfo);
+								   list_make1(appinfo));
 
 		/*
 		 * If there are securityQuals attached to the parent, move them to the
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index 06e843d..6f41979 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -55,7 +55,7 @@
 typedef struct
 {
 	PlannerInfo *root;
-	AppendRelInfo *appinfo;
+	List	    *appinfos;
 } adjust_appendrel_attrs_context;
 
 static Path *recurse_set_operations(Node *setOp, PlannerInfo *root,
@@ -107,9 +107,9 @@ static Bitmapset *translate_col_privs(const Bitmapset *parent_privs,
 					List *translated_vars);
 static Node *adjust_appendrel_attrs_mutator(Node *node,
 							   adjust_appendrel_attrs_context *context);
-static Relids adjust_relid_set(Relids relids, Index oldrelid, Index newrelid);
 static List *adjust_inherited_tlist(List *tlist,
 					   AppendRelInfo *context);
+static Relids adjust_relid_set(Relids relids, List *append_rel_infos);
 
 
 /*
@@ -1719,10 +1719,10 @@ translate_col_privs(const Bitmapset *parent_privs,
 
 /*
  * adjust_appendrel_attrs
- *	  Copy the specified query or expression and translate Vars referring
- *	  to the parent rel of the specified AppendRelInfo to refer to the
- *	  child rel instead.  We also update rtindexes appearing outside Vars,
- *	  such as resultRelation and jointree relids.
+ *	  Copy the specified query or expression and translate Vars referring to
+ *	  the parent rels of the specified in the given list of AppendRelInfos to
+ *	  refer to the corresponding child rel instead.  We also update rtindexes
+ *	  appearing outside Vars, such as resultRelation and jointree relids.
  *
  * Note: this is only applied after conversion of sublinks to subplans,
  * so we don't need to cope with recursion into sub-queries.
@@ -1731,13 +1731,20 @@ translate_col_privs(const Bitmapset *parent_privs,
  * maybe we should try to fold the two routines together.
  */
 Node *
-adjust_appendrel_attrs(PlannerInfo *root, Node *node, AppendRelInfo *appinfo)
+adjust_appendrel_attrs(PlannerInfo *root, Node *node, List *appinfos)
 {
 	Node	   *result;
 	adjust_appendrel_attrs_context context;
+	ListCell   *lc;
 
 	context.root = root;
-	context.appinfo = appinfo;
+	context.appinfos = appinfos;
+
+	/*
+	 * Catch a caller who wants to adjust expressions, but doesn't pass any
+	 * AppendRelInfo.
+	 */
+	Assert(appinfos && list_length(appinfos) >= 1);
 
 	/*
 	 * Must be prepared to start with a Query or a bare expression tree.
@@ -1745,20 +1752,28 @@ adjust_appendrel_attrs(PlannerInfo *root, Node *node, AppendRelInfo *appinfo)
 	if (node && IsA(node, Query))
 	{
 		Query	   *newnode;
+		AppendRelInfo *appinfo;
 
 		newnode = query_tree_mutator((Query *) node,
 									 adjust_appendrel_attrs_mutator,
 									 (void *) &context,
 									 QTW_IGNORE_RC_SUBQUERIES);
-		if (newnode->resultRelation == appinfo->parent_relid)
+		foreach (lc, appinfos)
 		{
-			newnode->resultRelation = appinfo->child_relid;
-			/* Fix tlist resnos too, if it's inherited UPDATE */
-			if (newnode->commandType == CMD_UPDATE)
-				newnode->targetList =
-					adjust_inherited_tlist(newnode->targetList,
-										   appinfo);
+			appinfo = lfirst(lc);
+
+			if (newnode->resultRelation == appinfo->parent_relid)
+			{
+				newnode->resultRelation = appinfo->child_relid;
+				/* Fix tlist resnos too, if it's inherited UPDATE */
+				if (newnode->commandType == CMD_UPDATE)
+					newnode->targetList =
+									adjust_inherited_tlist(newnode->targetList,
+														   appinfo);
+				break;
+			}
 		}
+
 		result = (Node *) newnode;
 	}
 	else
@@ -1771,13 +1786,29 @@ static Node *
 adjust_appendrel_attrs_mutator(Node *node,
 							   adjust_appendrel_attrs_context *context)
 {
-	AppendRelInfo *appinfo = context->appinfo;
+	List   *appinfos = context->appinfos;
+	ListCell   *lc;
+
+	/*
+	 * Catch a caller who wants to adjust expressions, but doesn't pass any
+	 * AppendRelInfo.
+	 */
+	Assert(appinfos && list_length(appinfos) >= 1);
 
 	if (node == NULL)
 		return NULL;
 	if (IsA(node, Var))
 	{
 		Var		   *var = (Var *) copyObject(node);
+		AppendRelInfo *appinfo;
+
+		foreach (lc, appinfos)
+		{
+			appinfo = lfirst(lc);
+
+			if (var->varno == appinfo->parent_relid)
+				break;
+		}
 
 		if (var->varlevelsup == 0 &&
 			var->varno == appinfo->parent_relid)
@@ -1860,29 +1891,54 @@ adjust_appendrel_attrs_mutator(Node *node,
 	{
 		CurrentOfExpr *cexpr = (CurrentOfExpr *) copyObject(node);
 
-		if (cexpr->cvarno == appinfo->parent_relid)
-			cexpr->cvarno = appinfo->child_relid;
+		foreach (lc, appinfos)
+		{
+			AppendRelInfo *appinfo = lfirst(lc);
+
+			if (cexpr->cvarno == appinfo->parent_relid)
+			{
+				cexpr->cvarno = appinfo->child_relid;
+				break;
+			}
+		}
 		return (Node *) cexpr;
 	}
 	if (IsA(node, RangeTblRef))
 	{
 		RangeTblRef *rtr = (RangeTblRef *) copyObject(node);
 
-		if (rtr->rtindex == appinfo->parent_relid)
-			rtr->rtindex = appinfo->child_relid;
+		foreach (lc, appinfos)
+		{
+			AppendRelInfo *appinfo = lfirst(lc);
+
+			if (rtr->rtindex == appinfo->parent_relid)
+			{
+				rtr->rtindex = appinfo->child_relid;
+				break;
+			}
+		}
 		return (Node *) rtr;
 	}
 	if (IsA(node, JoinExpr))
 	{
 		/* Copy the JoinExpr node with correct mutation of subnodes */
 		JoinExpr   *j;
+		AppendRelInfo *appinfo;
 
 		j = (JoinExpr *) expression_tree_mutator(node,
 											  adjust_appendrel_attrs_mutator,
 												 (void *) context);
 		/* now fix JoinExpr's rtindex (probably never happens) */
-		if (j->rtindex == appinfo->parent_relid)
-			j->rtindex = appinfo->child_relid;
+		foreach (lc, appinfos)
+		{
+			appinfo = lfirst(lc);
+
+			if (j->rtindex == appinfo->parent_relid)
+			{
+				j->rtindex = appinfo->child_relid;
+				break;
+			}
+		}
 		return (Node *) j;
 	}
 	if (IsA(node, PlaceHolderVar))
@@ -1895,9 +1951,7 @@ adjust_appendrel_attrs_mutator(Node *node,
 														 (void *) context);
 		/* now fix PlaceHolderVar's relid sets */
 		if (phv->phlevelsup == 0)
-			phv->phrels = adjust_relid_set(phv->phrels,
-										   appinfo->parent_relid,
-										   appinfo->child_relid);
+			phv->phrels = adjust_relid_set(phv->phrels, context->appinfos);
 		return (Node *) phv;
 	}
 	/* Shouldn't need to handle planner auxiliary nodes here */
@@ -1929,23 +1983,17 @@ adjust_appendrel_attrs_mutator(Node *node,
 
 		/* adjust relid sets too */
 		newinfo->clause_relids = adjust_relid_set(oldinfo->clause_relids,
-												  appinfo->parent_relid,
-												  appinfo->child_relid);
+													 context->appinfos);
 		newinfo->required_relids = adjust_relid_set(oldinfo->required_relids,
-													appinfo->parent_relid,
-													appinfo->child_relid);
+													 context->appinfos);
 		newinfo->outer_relids = adjust_relid_set(oldinfo->outer_relids,
-												 appinfo->parent_relid,
-												 appinfo->child_relid);
+													 context->appinfos);
 		newinfo->nullable_relids = adjust_relid_set(oldinfo->nullable_relids,
-													appinfo->parent_relid,
-													appinfo->child_relid);
+													   context->appinfos);
 		newinfo->left_relids = adjust_relid_set(oldinfo->left_relids,
-												appinfo->parent_relid,
-												appinfo->child_relid);
+												   context->appinfos);
 		newinfo->right_relids = adjust_relid_set(oldinfo->right_relids,
-												 appinfo->parent_relid,
-												 appinfo->child_relid);
+													context->appinfos);
 
 		/*
 		 * Reset cached derivative fields, since these might need to have
@@ -1977,19 +2025,38 @@ adjust_appendrel_attrs_mutator(Node *node,
 }
 
 /*
- * Substitute newrelid for oldrelid in a Relid set
+ * Replace parent relids by child relids in the copy of given relid set
+ * according to the given list of AppendRelInfos. The given relid set is
+ * returned as is if it contains no parent in the given list, otherwise, the
+ * given relid set is not changed.
  */
-static Relids
-adjust_relid_set(Relids relids, Index oldrelid, Index newrelid)
+Relids
+adjust_relid_set(Relids relids, List *append_rel_infos)
 {
-	if (bms_is_member(oldrelid, relids))
+	ListCell   *lc;
+	Bitmapset  *result = NULL;
+
+	foreach (lc, append_rel_infos)
 	{
-		/* Ensure we have a modifiable copy */
-		relids = bms_copy(relids);
-		/* Remove old, add new */
-		relids = bms_del_member(relids, oldrelid);
-		relids = bms_add_member(relids, newrelid);
+		AppendRelInfo	*appinfo = lfirst(lc);
+
+		/* Remove parent, add child */
+		if (bms_is_member(appinfo->parent_relid, relids))
+		{
+			/* Make a copy if we are changing the set. */
+			if (!result)
+				result = bms_copy(relids);
+
+			result = bms_del_member(result, appinfo->parent_relid);
+			result = bms_add_member(result, appinfo->child_relid);
+		}
 	}
+
+	/* Return new set if we modified the given set. */
+	if (result)
+		return result;
+
+	/* Else return the given relids set as is. */
 	return relids;
 }
 
@@ -2110,5 +2177,5 @@ adjust_appendrel_attrs_multilevel(PlannerInfo *root, Node *node,
 	else
 		Assert(parent_rel->reloptkind == RELOPT_BASEREL);
 	/* Now translate for this child */
-	return adjust_appendrel_attrs(root, node, appinfo);
+	return adjust_appendrel_attrs(root, node, list_make1(appinfo));
 }
diff --git a/src/include/optimizer/prep.h b/src/include/optimizer/prep.h
index 2b20b36..a02e06a 100644
--- a/src/include/optimizer/prep.h
+++ b/src/include/optimizer/prep.h
@@ -53,7 +53,7 @@ extern RelOptInfo *plan_set_operations(PlannerInfo *root);
 extern void expand_inherited_tables(PlannerInfo *root);
 
 extern Node *adjust_appendrel_attrs(PlannerInfo *root, Node *node,
-					   AppendRelInfo *appinfo);
+					   List *appinfos);
 
 extern Node *adjust_appendrel_attrs_multilevel(PlannerInfo *root, Node *node,
 								  RelOptInfo *child_rel);
-- 
1.7.9.5

0004-Refactor-build_join_rel.patchapplication/octet-stream; name=0004-Refactor-build_join_rel.patchDownload
From 5606e2526b997606e8485c00fc34ac1f5ccf2fe9 Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Mon, 6 Feb 2017 10:58:48 +0530
Subject: [PATCH 04/14] Refactor build_join_rel.

Partition-wise joins do not use build_join_rel() to build child-join relations,
but it still requires code to set foreign relation properties as well as code
to add join relation into PlannerInfo. Separate that code into
set_foreign_rel_properties() and add_join_rel() resp. to be called while
building child joins.
---
 src/backend/optimizer/util/relnode.c |  142 ++++++++++++++++++++--------------
 1 file changed, 83 insertions(+), 59 deletions(-)

diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index adc1db9..160ed6d 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -49,6 +49,9 @@ static List *subbuild_joinrel_restrictlist(RelOptInfo *joinrel,
 static List *subbuild_joinrel_joinlist(RelOptInfo *joinrel,
 						  List *joininfo_list,
 						  List *new_joininfo);
+static void set_foreign_rel_properties(RelOptInfo *joinrel,
+						   RelOptInfo *outer_rel, RelOptInfo *inner_rel);
+static void add_join_rel(PlannerInfo *root, RelOptInfo *joinrel);
 
 
 /*
@@ -327,6 +330,82 @@ find_join_rel(PlannerInfo *root, Relids relids)
 }
 
 /*
+ * set_foreign_rel_properties
+ *		Set up foreign-join fields if outer and inner relation are foreign
+ *		tables (or joins) belonging to the same server and assigned to the same
+ *		user to check access permissions as.
+ *
+ * In addition to an exact match of userid, we allow the case where one side
+ * has zero userid (implying current user) and the other side has explicit
+ * userid that happens to equal the current user; but in that case, pushdown of
+ * the join is only valid for the current user.  The useridiscurrent field
+ * records whether we had to make such an assumption for this join or any
+ * sub-join.
+ *
+ * Otherwise these fields are left invalid, so GetForeignJoinPaths will not be
+ * called for the join relation.
+ *
+ */
+static void
+set_foreign_rel_properties(RelOptInfo *joinrel, RelOptInfo *outer_rel,
+						   RelOptInfo *inner_rel)
+{
+	if (OidIsValid(outer_rel->serverid) &&
+		inner_rel->serverid == outer_rel->serverid)
+	{
+		if (inner_rel->userid == outer_rel->userid)
+		{
+			joinrel->serverid = outer_rel->serverid;
+			joinrel->userid = outer_rel->userid;
+			joinrel->useridiscurrent = outer_rel->useridiscurrent || inner_rel->useridiscurrent;
+			joinrel->fdwroutine = outer_rel->fdwroutine;
+		}
+		else if (!OidIsValid(inner_rel->userid) &&
+				 outer_rel->userid == GetUserId())
+		{
+			joinrel->serverid = outer_rel->serverid;
+			joinrel->userid = outer_rel->userid;
+			joinrel->useridiscurrent = true;
+			joinrel->fdwroutine = outer_rel->fdwroutine;
+		}
+		else if (!OidIsValid(outer_rel->userid) &&
+				 inner_rel->userid == GetUserId())
+		{
+			joinrel->serverid = outer_rel->serverid;
+			joinrel->userid = inner_rel->userid;
+			joinrel->useridiscurrent = true;
+			joinrel->fdwroutine = outer_rel->fdwroutine;
+		}
+	}
+}
+
+/*
+ * add_join_rel
+ *		Add given join relation to the list of join relations in the given
+ *		PlannerInfo. Also add it to the auxiliary hashtable if there is one.
+ */
+void
+add_join_rel(PlannerInfo *root, RelOptInfo *joinrel)
+{
+	/* GEQO requires us to append the new joinrel to the end of the list! */
+	root->join_rel_list = lappend(root->join_rel_list, joinrel);
+
+	/* store it into the auxiliary hashtable if there is one. */
+	if (root->join_rel_hash)
+	{
+		JoinHashEntry *hentry;
+		bool		found;
+
+		hentry = (JoinHashEntry *) hash_search(root->join_rel_hash,
+											   &(joinrel->relids),
+											   HASH_ENTER,
+											   &found);
+		Assert(!found);
+		hentry->join_rel = joinrel;
+	}
+}
+
+/*
  * build_join_rel
  *	  Returns relation entry corresponding to the union of two given rels,
  *	  creating a new relation entry if none already exists.
@@ -424,46 +503,8 @@ build_join_rel(PlannerInfo *root,
 	joinrel->joininfo = NIL;
 	joinrel->has_eclass_joins = false;
 
-	/*
-	 * Set up foreign-join fields if outer and inner relation are foreign
-	 * tables (or joins) belonging to the same server and assigned to the same
-	 * user to check access permissions as.  In addition to an exact match of
-	 * userid, we allow the case where one side has zero userid (implying
-	 * current user) and the other side has explicit userid that happens to
-	 * equal the current user; but in that case, pushdown of the join is only
-	 * valid for the current user.  The useridiscurrent field records whether
-	 * we had to make such an assumption for this join or any sub-join.
-	 *
-	 * Otherwise these fields are left invalid, so GetForeignJoinPaths will
-	 * not be called for the join relation.
-	 */
-	if (OidIsValid(outer_rel->serverid) &&
-		inner_rel->serverid == outer_rel->serverid)
-	{
-		if (inner_rel->userid == outer_rel->userid)
-		{
-			joinrel->serverid = outer_rel->serverid;
-			joinrel->userid = outer_rel->userid;
-			joinrel->useridiscurrent = outer_rel->useridiscurrent || inner_rel->useridiscurrent;
-			joinrel->fdwroutine = outer_rel->fdwroutine;
-		}
-		else if (!OidIsValid(inner_rel->userid) &&
-				 outer_rel->userid == GetUserId())
-		{
-			joinrel->serverid = outer_rel->serverid;
-			joinrel->userid = outer_rel->userid;
-			joinrel->useridiscurrent = true;
-			joinrel->fdwroutine = outer_rel->fdwroutine;
-		}
-		else if (!OidIsValid(outer_rel->userid) &&
-				 inner_rel->userid == GetUserId())
-		{
-			joinrel->serverid = outer_rel->serverid;
-			joinrel->userid = inner_rel->userid;
-			joinrel->useridiscurrent = true;
-			joinrel->fdwroutine = outer_rel->fdwroutine;
-		}
-	}
+	/* Compute information relevant to the foreign relations. */
+	set_foreign_rel_properties(joinrel, outer_rel, inner_rel);
 
 	/*
 	 * Create a new tlist containing just the vars that need to be output from
@@ -531,25 +572,8 @@ build_join_rel(PlannerInfo *root,
 		is_parallel_safe(root, (Node *) joinrel->reltarget->exprs))
 		joinrel->consider_parallel = true;
 
-	/*
-	 * Add the joinrel to the query's joinrel list, and store it into the
-	 * auxiliary hashtable if there is one.  NB: GEQO requires us to append
-	 * the new joinrel to the end of the list!
-	 */
-	root->join_rel_list = lappend(root->join_rel_list, joinrel);
-
-	if (root->join_rel_hash)
-	{
-		JoinHashEntry *hentry;
-		bool		found;
-
-		hentry = (JoinHashEntry *) hash_search(root->join_rel_hash,
-											   &(joinrel->relids),
-											   HASH_ENTER,
-											   &found);
-		Assert(!found);
-		hentry->join_rel = joinrel;
-	}
+	/* Add the joinrel to the PlannerInfo. */
+	add_join_rel(root, joinrel);
 
 	/*
 	 * Also, if dynamic-programming join search is active, add the new joinrel
-- 
1.7.9.5

0005-Add-function-find_param_path_info.patchapplication/octet-stream; name=0005-Add-function-find_param_path_info.patchDownload
From ec2984ab4ea387d5e91fbd354209ff45f114b603 Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Mon, 6 Feb 2017 12:14:06 +0530
Subject: [PATCH 05/14] Add function find_param_path_info.

The code to search ParamPathInfo for a set of required outer relations in the
list of ParamPathInfos of a given relation is duplicated in
get_*rel_parampathinfo() functions. Separate this code into
find_param_path_info() and call it from get_*rel_parampathinfo() functions.
---
 src/backend/optimizer/util/relnode.c |   46 ++++++++++++++++++++--------------
 1 file changed, 27 insertions(+), 19 deletions(-)

diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 160ed6d..19982dc 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -52,6 +52,8 @@ static List *subbuild_joinrel_joinlist(RelOptInfo *joinrel,
 static void set_foreign_rel_properties(RelOptInfo *joinrel,
 						   RelOptInfo *outer_rel, RelOptInfo *inner_rel);
 static void add_join_rel(PlannerInfo *root, RelOptInfo *joinrel);
+extern ParamPathInfo *find_param_path_info(RelOptInfo *rel,
+									  Relids required_outer);
 
 
 /*
@@ -1047,12 +1049,8 @@ get_baserel_parampathinfo(PlannerInfo *root, RelOptInfo *baserel,
 	Assert(!bms_overlap(baserel->relids, required_outer));
 
 	/* If we already have a PPI for this parameterization, just return it */
-	foreach(lc, baserel->ppilist)
-	{
-		ppi = (ParamPathInfo *) lfirst(lc);
-		if (bms_equal(ppi->ppi_req_outer, required_outer))
-			return ppi;
-	}
+	if ((ppi = find_param_path_info(baserel, required_outer)))
+		return ppi;
 
 	/*
 	 * Identify all joinclauses that are movable to this base rel given this
@@ -1289,12 +1287,8 @@ get_joinrel_parampathinfo(PlannerInfo *root, RelOptInfo *joinrel,
 	*restrict_clauses = list_concat(pclauses, *restrict_clauses);
 
 	/* If we already have a PPI for this parameterization, just return it */
-	foreach(lc, joinrel->ppilist)
-	{
-		ppi = (ParamPathInfo *) lfirst(lc);
-		if (bms_equal(ppi->ppi_req_outer, required_outer))
-			return ppi;
-	}
+	if ((ppi = find_param_path_info(joinrel, required_outer)))
+		return ppi;
 
 	/* Estimate the number of rows returned by the parameterized join */
 	rows = get_parameterized_joinrel_size(root, joinrel,
@@ -1333,7 +1327,6 @@ ParamPathInfo *
 get_appendrel_parampathinfo(RelOptInfo *appendrel, Relids required_outer)
 {
 	ParamPathInfo *ppi;
-	ListCell   *lc;
 
 	/* Unparameterized paths have no ParamPathInfo */
 	if (bms_is_empty(required_outer))
@@ -1342,12 +1335,8 @@ get_appendrel_parampathinfo(RelOptInfo *appendrel, Relids required_outer)
 	Assert(!bms_overlap(appendrel->relids, required_outer));
 
 	/* If we already have a PPI for this parameterization, just return it */
-	foreach(lc, appendrel->ppilist)
-	{
-		ppi = (ParamPathInfo *) lfirst(lc);
-		if (bms_equal(ppi->ppi_req_outer, required_outer))
-			return ppi;
-	}
+	if ((ppi = find_param_path_info(appendrel, required_outer)))
+		return ppi;
 
 	/* Else build the ParamPathInfo */
 	ppi = makeNode(ParamPathInfo);
@@ -1358,3 +1347,22 @@ get_appendrel_parampathinfo(RelOptInfo *appendrel, Relids required_outer)
 
 	return ppi;
 }
+
+/*
+ * Returns a ParamPathInfo for outer relations specified by required_outer, if
+ * already available in the given rel. Returns NULL otherwise.
+ */
+ParamPathInfo *
+find_param_path_info(RelOptInfo *rel, Relids required_outer)
+{
+	ListCell   *lc;
+
+	foreach(lc, rel->ppilist)
+	{
+		ParamPathInfo  *ppi = (ParamPathInfo *) lfirst(lc);
+		if (bms_equal(ppi->ppi_req_outer, required_outer))
+			return ppi;
+	}
+
+	return NULL;
+}
-- 
1.7.9.5

0006-Canonical-partition-scheme.patchapplication/octet-stream; name=0006-Canonical-partition-scheme.patchDownload
From 53c7c239715824278c2abe19c15abdb1ed3d7d91 Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Tue, 7 Feb 2017 10:47:49 +0530
Subject: [PATCH 06/14] Canonical partition scheme.

For a single level partitioned table, annotate RelOptInfo of a partitioned
table with canonical partition scheme. All partitioned tables, with the same
partitioning scheme share the same canonical partitioning scheme. We store the
RelOptInfo's corresponding to the partitions in RelOptInfo of the partitioned
table. Those are arranged in the same order as the partition bound indices in
the partition scheme.

We do not handle multi-level partitioned tables since inheritance hierarchy
does not retain the partition hierarchy.  All the partitions at any level
appear as children of the top-level partitioned table. Thus making it hard to
associate a partition relation with corresponding partition bounds.

Multi-level partitioned tables will be handled in a separate patch.
---
 src/backend/optimizer/path/allpaths.c |   48 +++++++
 src/backend/optimizer/util/plancat.c  |  232 +++++++++++++++++++++++++++++++++
 src/include/nodes/relation.h          |   51 ++++++++
 3 files changed, 331 insertions(+)

diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index d8fac14..0eb56f3 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -829,6 +829,7 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 	double	   *parent_attrsizes;
 	int			nattrs;
 	ListCell   *l;
+	int			nparts;
 
 	/*
 	 * Initialize to compute size estimates for whole append relation.
@@ -850,6 +851,18 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 	nattrs = rel->max_attr - rel->min_attr + 1;
 	parent_attrsizes = (double *) palloc0(nattrs * sizeof(double));
 
+	/*
+	 * For a partitioned table, allocate an array to hold RelOptInfo's of the
+	 * partitions. It will be filled while handling the children below.
+	 */
+	if (rel->part_scheme)
+	{
+		nparts = rel->part_scheme->nparts;
+		rel->part_rels = (RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
+	}
+	else
+		nparts = 0;
+
 	foreach(l, root->append_rel_list)
 	{
 		AppendRelInfo *appinfo = (AppendRelInfo *) lfirst(l);
@@ -879,6 +892,30 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		Assert(childrel->reloptkind == RELOPT_OTHER_MEMBER_REL);
 
 		/*
+		 * Two partitioned tables with the same partitioning scheme, have their
+		 * partition bounds arranged in the same order. The order of partition
+		 * OIDs in RelOptInfo corresponds to the partition bound order. Thus
+		 * the OIDs of matching partitions from both the tables are placed at
+		 * the same position in the array of partition OIDs in the respective
+		 * RelOptInfos. Arranging RelOptInfos of partitions in the same order
+		 * as their OIDs makes it easy to find the RelOptInfos of matching
+		 * partitions for partition-wise join.
+		 */
+		if (rel->part_scheme)
+		{
+			int		cnt_parts;
+
+			for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+			{
+				if (rel->part_oids[cnt_parts] == childRTE->relid)
+				{
+					Assert(!rel->part_rels[cnt_parts]);
+					rel->part_rels[cnt_parts] = childrel;
+				}
+			}
+		}
+
+		/*
 		 * We have to copy the parent's targetlist and quals to the child,
 		 * with appropriate substitution of variables.  However, only the
 		 * baserestrictinfo quals are needed before we can check for
@@ -1130,6 +1167,17 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		}
 	}
 
+	/* Should have found all the childrels of a partitioned relation. */
+	if (rel->part_scheme)
+	{
+		int		cnt_parts;
+
+		for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+			if (!rel->part_rels[cnt_parts])
+				elog(ERROR, "could not find the RelOptInfo of a partition with oid %u",
+					 rel->part_oids[cnt_parts]);
+	}
+
 	if (has_live_children)
 	{
 		/*
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index 7836e6b..01ba885 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -27,6 +27,7 @@
 #include "catalog/catalog.h"
 #include "catalog/dependency.h"
 #include "catalog/heap.h"
+#include "catalog/pg_inherits_fn.h"
 #include "catalog/partition.h"
 #include "catalog/pg_am.h"
 #include "foreign/fdwapi.h"
@@ -63,6 +64,13 @@ static List *get_relation_constraints(PlannerInfo *root,
 						 bool include_notnull);
 static List *build_index_tlist(PlannerInfo *root, IndexOptInfo *index,
 				  Relation heapRelation);
+static List **build_baserel_partition_key_exprs(Relation relation,
+												Index varno);
+static PartitionScheme find_partition_scheme(struct PlannerInfo *root,
+											 Relation rel);
+static void get_relation_partition_info(PlannerInfo *root, RelOptInfo *rel,
+										Relation relation, bool inhparent);
+
 
 
 /*
@@ -412,6 +420,9 @@ get_relation_info(PlannerInfo *root, Oid relationObjectId, bool inhparent,
 	/* Collect info about relation's foreign keys, if relevant */
 	get_relation_foreign_keys(root, rel, relation, inhparent);
 
+	/* Collect info about relation's partitioning scheme, if any. */
+	get_relation_partition_info(root, rel, relation, inhparent);
+
 	heap_close(relation, NoLock);
 
 	/*
@@ -1716,3 +1727,224 @@ has_row_triggers(PlannerInfo *root, Index rti, CmdType event)
 	heap_close(relation, NoLock);
 	return result;
 }
+
+/*
+ * get_relation_partition_info
+ *
+ * Retrieves partitioning information for a given relation.
+ *
+ * Partitioning scheme, partition key expressions and OIDs of partitions are
+ * added to the given RelOptInfo. A partitioned table can participate in the
+ * query as a simple relation or an inheritance parent. Only the later can have
+ * child relations, and hence partitions. From the point of view of the query
+ * optimizer only such relations are considered to be partitioned. Hence
+ * partitioning information is set only for an inheritance parent.
+ */
+static void
+get_relation_partition_info(PlannerInfo *root, RelOptInfo *rel,
+							Relation relation, bool inhparent)
+{
+	/* No partitioning information for an unpartitioned relation. */
+	if (relation->rd_rel->relkind != RELKIND_PARTITIONED_TABLE ||
+		!inhparent ||
+		!(rel->part_scheme = find_partition_scheme(root, relation)))
+	{
+		rel->partexprs = NULL;
+		rel->part_rels = NULL;
+		rel->part_oids = NULL;
+		return;
+	}
+
+	rel->partexprs = build_baserel_partition_key_exprs(relation, rel->relid);
+	rel->part_oids = RelationGetPartitionDesc(relation)->oids;
+
+	/*
+	 * RelOptInfos of the partitions will be filled in when we build those for
+	 * the child relations.
+	 */
+	rel->part_rels = NULL;
+	return;
+}
+
+/*
+ * find_partition_scheme
+ *
+ * The function returns a canonical partition scheme which exactly matches the
+ * partitioning properties of the given relation if one exists in the of
+ * canonical partitioning schemes maintained in PlannerInfo. If none of the
+ * existing partitioning schemes match, the function creates a canonical
+ * partition scheme and adds it to the list.
+ *
+ * For an unpartitioned table or for a multi-level partitioned table it returns
+ * NULL. See comments in the function for more details.
+ */
+static PartitionScheme
+find_partition_scheme(PlannerInfo *root, Relation relation)
+{
+	PartitionKey	part_key = RelationGetPartitionKey(relation);
+	PartitionDesc	part_desc = RelationGetPartitionDesc(relation);
+	ListCell	   *lc;
+	int		nparts;
+	int		partnatts;
+	int		cnt_pks;
+	int		cnt_parts;
+	PartitionScheme	part_scheme = NULL;
+
+	/* No partition scheme for an unpartitioned relation. */
+	if (!part_desc || !part_key)
+		return NULL;
+
+	nparts = part_desc->nparts;
+	partnatts = part_key->partnatts;
+
+	/*
+	 * For a multi-level partitioned table, we do not retain the partitioning
+	 * hierarchy while expanding RTE for the topmost parent. Thus the number of
+	 * children as per root->append_rel_list does not match the number of
+	 * partitions specified in the partition descriptor and hence the
+	 * partitioning scheme of a multi-partitioned table does not reflect the
+	 * true picture. So for now, treat a multi-partitioned table as not
+	 * partitioned.
+	 */
+	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	{
+		if (has_subclass(part_desc->oids[cnt_parts]))
+			return NULL;
+	}
+
+	/* Search for a matching partition scheme and return if found one. */
+	foreach (lc, root->part_schemes)
+	{
+		part_scheme = lfirst(lc);
+
+		/* Match number of partitions and partitioning strategy. */
+		if (nparts != part_scheme->nparts ||
+			part_key->strategy != part_scheme->strategy ||
+			partnatts != part_scheme->partnatts)
+			continue;
+
+		/* Match the partition key types. */
+		for (cnt_pks = 0; cnt_pks < partnatts; cnt_pks++)
+		{
+			/*
+			 * For types, it suffices to match the type id, mod and collation;
+			 * len, byval and align are depedent on the first two.
+			 */
+			if (part_key->partopfamily[cnt_pks] != part_scheme->partopfamily[cnt_pks] ||
+				part_key->partopcintype[cnt_pks] != part_scheme->partopcintype[cnt_pks] ||
+				part_key->parttypid[cnt_pks] != part_scheme->key_types[cnt_pks] ||
+				part_key->parttypmod[cnt_pks] != part_scheme->key_typmods[cnt_pks] ||
+				part_key->parttypcoll[cnt_pks] != part_scheme->key_collations[cnt_pks])
+				break;
+		}
+
+		/* Some partition key didn't match. Check next partitioning scheme. */
+		if (cnt_pks < partnatts)
+			continue;
+
+		if (!partition_bounds_equal(part_key, part_desc->boundinfo,
+									part_scheme->boundinfo))
+			continue;
+
+		/* Found matching partition scheme. */
+		return part_scheme;
+	}
+
+	/* Did not find matching partition scheme. Create one. */
+	part_scheme = (PartitionScheme) palloc0(sizeof(PartitionSchemeData));
+
+	/* Copy partition bounds/lists. */
+	part_scheme->nparts = part_desc->nparts;
+	part_scheme->strategy = part_key->strategy;
+	part_scheme->boundinfo = part_desc->boundinfo;
+
+	/* Store partition key information. */
+	part_scheme->partnatts = part_key->partnatts;
+
+	part_scheme->partopfamily = (Oid *) palloc(sizeof(Oid) * partnatts);
+	memcpy(part_scheme->partopfamily, part_key->partopfamily,
+		   sizeof(Oid) * partnatts);
+
+	part_scheme->partopcintype = (Oid *) palloc(sizeof(Oid) * partnatts);
+	memcpy(part_scheme->partopcintype, part_key->partopcintype,
+		   sizeof(Oid) * partnatts);
+
+	part_scheme->key_types = (Oid *) palloc(sizeof(Oid) * partnatts);
+	memcpy(part_scheme->key_types, part_key->parttypid,
+		   sizeof(Oid) * partnatts);
+
+	part_scheme->key_typmods = (int32 *) palloc(sizeof(int32) * partnatts);
+	memcpy(part_scheme->key_typmods, part_key->parttypmod,
+		   sizeof(int32) * partnatts);
+
+	part_scheme->key_collations = (Oid *) palloc(sizeof(Oid) * partnatts);
+	memcpy(part_scheme->key_collations, part_key->parttypcoll,
+		   sizeof(Oid) * partnatts);
+
+	/* Add the partitioning scheme to PlannerInfo. */
+	root->part_schemes = lappend(root->part_schemes, part_scheme);
+
+	return part_scheme;
+}
+
+/*
+ * build_baserel_partition_key_exprs
+ *
+ * Collect partition key expressions for a given base relation. The function
+ * converts any single column partition keys into corresponding Var nodes. It
+ * restamps Var nodes in partition key expressions by given varno. The
+ * partition key expressions are returned as an array of single element Lists
+ * to be stored in RelOptInfo of the base relation.
+ */
+static List **
+build_baserel_partition_key_exprs(Relation relation, Index varno)
+{
+	PartitionKey	part_key = RelationGetPartitionKey(relation);
+	int		num_pkexprs;
+	int		cnt_pke;
+	List		  **partexprs;
+	ListCell	   *lc;
+
+	if (!part_key || part_key->partnatts <= 0)
+		return NULL;
+
+	num_pkexprs = part_key->partnatts;
+	partexprs = (List **) palloc(sizeof(List *) * num_pkexprs);
+	lc = list_head(part_key->partexprs);
+
+	for (cnt_pke = 0; cnt_pke < num_pkexprs; cnt_pke++)
+	{
+		AttrNumber attno = part_key->partattrs[cnt_pke];
+		Expr	  *pkexpr;
+
+		if (attno != InvalidAttrNumber)
+		{
+			/* Single column partition key is stored as a Var node. */
+			Form_pg_attribute att_tup;
+
+			if (attno < 0)
+				att_tup = SystemAttributeDefinition(attno,
+												relation->rd_rel->relhasoids);
+			else
+				att_tup = relation->rd_att->attrs[attno - 1];
+
+			pkexpr = (Expr *) makeVar(varno, attno, att_tup->atttypid,
+									  att_tup->atttypmod,
+									  att_tup->attcollation, 0);
+		}
+		else
+		{
+			if (lc == NULL)
+				elog(ERROR, "wrong number of partition key expressions");
+
+			/* Re-stamp the expressions with given varno. */
+			pkexpr = (Expr *) copyObject(lfirst(lc));
+			ChangeVarNodes((Node *) pkexpr, 1, varno, 0);
+			lc = lnext(lc);
+		}
+
+		partexprs[cnt_pke] = list_make1(pkexpr);
+	}
+
+	return partexprs;
+}
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 643be54..4f99184 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 "catalog/partition.h"
 #include "lib/stringinfo.h"
 #include "nodes/params.h"
 #include "nodes/parsenodes.h"
@@ -261,6 +262,9 @@ typedef struct PlannerInfo
 	List	   *distinct_pathkeys;		/* distinctClause pathkeys, if any */
 	List	   *sort_pathkeys;	/* sortClause pathkeys, if any */
 
+	List	   *part_schemes;	/* Canonicalised partition schemes
+								 * used in the query. */
+
 	List	   *initial_rels;	/* RelOptInfos we are now trying to join */
 
 	/* Use fetch_upper_rel() to get any particular upper rel */
@@ -321,6 +325,38 @@ typedef struct PlannerInfo
 	((root)->simple_rte_array ? (root)->simple_rte_array[rti] : \
 	 rt_fetch(rti, (root)->parse->rtable))
 
+/*
+ * Partitioning scheme
+ *		Structure to hold partitioning scheme for a given relation.
+ *
+ * Multiple relations may be partitioned in the same way. The relations
+ * resulting from joining such relations may be partitioned in the same way as
+ * the joining relations. Similarly, relations derived from such relations by
+ * grouping, sorting may be partitioned in the same way as the underlying
+ * scan relations. All such relations partitioned in the same way share the
+ * partitioning scheme.
+ *
+ * PlannerInfo stores a list of distinct "canonical" partitioning schemes.
+ * RelOptInfo of a partitioned relation holds the pointer to "canonical"
+ * partitioning scheme.
+ */
+typedef struct PartitionSchemeData
+{
+	/* Information about partitions */
+	int			nparts;			/* number of partitions */
+	PartitionBoundInfo boundinfo;	/* Partition bounds/lists */
+
+	/* Information about partition keys */
+	char		strategy;		/* partition strategy */
+	int16		partnatts;		/* number of partition attributes */
+	Oid		   *partopfamily;	/* OIDs of operator families */
+	Oid		   *partopcintype;	/* OIDs of opclass declared input data types */
+	Oid		   *key_types;		/* OIDs of partition key data types. */
+	int32	   *key_typmods;	/* typmods of partition keys. */
+	Oid		   *key_collations;	/* OIDs of collations of partition keys. */
+} PartitionSchemeData;
+
+typedef struct PartitionSchemeData *PartitionScheme;
 
 /*----------
  * RelOptInfo
@@ -531,6 +567,7 @@ typedef struct RelOptInfo
 	PlannerInfo *subroot;		/* if subquery */
 	List	   *subplan_params; /* if subquery */
 	int			rel_parallel_workers;	/* wanted number of parallel workers */
+	Oid		   *part_oids;		/* OIDs of partitions */
 
 	/* Information about foreign tables and foreign joins */
 	Oid			serverid;		/* identifies server for the table or join */
@@ -549,6 +586,20 @@ typedef struct RelOptInfo
 	List	   *joininfo;		/* RestrictInfo structures for join clauses
 								 * involving this rel */
 	bool		has_eclass_joins;		/* T means joininfo is incomplete */
+
+	/* For all the partitioned relations. */
+	PartitionScheme	    part_scheme;	/* Partitioning scheme. */
+	struct RelOptInfo **part_rels;		/* Array of RelOptInfos of partitions,
+										 * stored in the same order as bounds
+										 * or lists in PartitionScheme.
+										 */
+	List			  **partexprs;		/* Array of list of partition key
+										 * expressions. For base relations
+										 * these are one element lists. For
+										 * join there may be as many elements
+										 * as the number of joining
+										 * relations.
+										 */
 } RelOptInfo;
 
 /*
-- 
1.7.9.5

0007-Partition-wise-join-tests.patchapplication/octet-stream; name=0007-Partition-wise-join-tests.patchDownload
From 82e4452ff549ede9f34684d363f66291a3bb632b Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Tue, 7 Feb 2017 11:57:30 +0530
Subject: [PATCH 07/14] Partition-wise join tests.

This file does not contain tests for joins between multi-leveled partitioned
tables. Those will be added later.
---
 src/test/regress/expected/partition_join.out | 4114 ++++++++++++++++++++++++++
 src/test/regress/parallel_schedule           |    2 +-
 src/test/regress/serial_schedule             |    1 +
 src/test/regress/sql/partition_join.sql      |  515 ++++
 4 files changed, 4631 insertions(+), 1 deletion(-)
 create mode 100644 src/test/regress/expected/partition_join.out
 create mode 100644 src/test/regress/sql/partition_join.sql

diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
new file mode 100644
index 0000000..18238fa
--- /dev/null
+++ b/src/test/regress/expected/partition_join.out
@@ -0,0 +1,4114 @@
+--
+-- PARTITION_JOIN
+-- Test partition-wise join between partitioned tables
+--
+--
+-- partitioned by a single column
+--
+CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+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, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1 AS SELECT * FROM prt1;
+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_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, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+CREATE TABLE uprt2 AS SELECT * FROM prt2;
+-- inner join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on public.prt2_p1 t2
+                     Output: t2.b, t2.c
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+               Hash Cond: (t2_1.b = t1_2.a)
+               ->  Seq Scan on public.prt2_p2 t2_1
+                     Output: t2_1.b, t2_1.c
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+               Hash Cond: (t2_2.b = t1_1.a)
+               ->  Seq Scan on public.prt2_p3 t2_2
+                     Output: t2_2.b, t2_2.c
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 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)
+
+-- left outer join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on public.prt2_p1 t2
+                     Output: t2.b, t2.c
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+               Hash Cond: (t2_1.b = t1_2.a)
+               ->  Seq Scan on public.prt2_p2 t2_1
+                     Output: t2_1.b, t2_1.c
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+               Hash Cond: (t2_2.b = t1_1.a)
+               ->  Seq Scan on public.prt2_p3 t2_2
+                     Output: t2_2.b, t2_2.c
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+-- right outer join
+EXPLAIN (VERBOSE, 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.b % 25 = 0 ORDER BY t1.a, t2.b;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: (t1.a = t2.b)
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t2.b, t2.c
+                           ->  Seq Scan on public.prt2_p1 t2
+                                 Output: t2.b, t2.c
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Hash Cond: (t1_2.a = t2_1.b)
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c
+                           ->  Seq Scan on public.prt2_p2 t2_1
+                                 Output: t2_1.b, t2_1.c
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Hash Cond: (t1_1.a = t2_2.b)
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c
+                           ->  Seq Scan on public.prt2_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+                                 Filter: ((t2_2.b % 25) = 0)
+(36 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 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)
+
+-- full outer join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+   Sort Key: prt1_p1.a, prt2_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+               Hash Cond: (prt1_p1.a = prt2_p1.b)
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c
+                     Filter: ((prt1_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p1.b, prt2_p1.c
+                     ->  Seq Scan on public.prt2_p1
+                           Output: prt2_p1.b, prt2_p1.c
+                           Filter: ((prt2_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt1_p2.a = prt2_p2.b)
+               ->  Seq Scan on public.prt1_p2
+                     Output: prt1_p2.a, prt1_p2.c
+                     Filter: ((prt1_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p2.b, prt2_p2.c
+                     ->  Seq Scan on public.prt2_p2
+                           Output: prt2_p2.b, prt2_p2.c
+                           Filter: ((prt2_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c
+               Hash Cond: (prt1_p3.a = prt2_p3.b)
+               ->  Seq Scan on public.prt1_p3
+                     Output: prt1_p3.a, prt1_p3.c
+                     Filter: ((prt1_p3.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p3.b, prt2_p3.c
+                     ->  Seq Scan on public.prt2_p3
+                           Output: prt2_p3.b, prt2_p3.c
+                           Filter: ((prt2_p3.b % 25) = 0)
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 t2 WHERE t2.b % 25 = 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)
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+   Sort Key: prt1_p1.a, prt2_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+               Hash Cond: (prt1_p1.a = prt2_p1.b)
+               Filter: (((50) = prt1_p1.b) OR ((75) = prt2_p1.b))
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c, prt1_p1.b, 50
+                     Filter: ((prt1_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p1.b, prt2_p1.c, (75)
+                     ->  Seq Scan on public.prt2_p1
+                           Output: prt2_p1.b, prt2_p1.c, 75
+                           Filter: ((prt2_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt1_p2.a = prt2_p2.b)
+               Filter: (((50) = prt1_p2.b) OR ((75) = prt2_p2.b))
+               ->  Seq Scan on public.prt1_p2
+                     Output: prt1_p2.a, prt1_p2.c, prt1_p2.b, 50
+                     Filter: ((prt1_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p2.b, prt2_p2.c, (75)
+                     ->  Seq Scan on public.prt2_p2
+                           Output: prt2_p2.b, prt2_p2.c, 75
+                           Filter: ((prt2_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c
+               Hash Cond: (prt1_p3.a = prt2_p3.b)
+               Filter: (((50) = prt1_p3.b) OR ((75) = prt2_p3.b))
+               ->  Seq Scan on public.prt1_p3
+                     Output: prt1_p3.a, prt1_p3.c, prt1_p3.b, 50
+                     Filter: ((prt1_p3.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_p3.b, prt2_p3.c, (75)
+                     ->  Seq Scan on public.prt2_p3
+                           Output: prt2_p3.b, prt2_p3.c, 75
+                           Filter: ((prt2_p3.b % 25) = 0)
+(40 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+ a  |  c   | b  |  c   
+----+------+----+------
+ 50 | 0050 |    | 
+    |      | 75 | 0075
+(2 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+ a  |  c   | b  |  c   
+----+------+----+------
+ 50 | 0050 |    | 
+    |      | 75 | 0075
+(2 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, (25), prt2_p1.b, prt2_p1.c, (50)
+   Sort Key: prt1_p1.a, prt2_p1.b
+   ->  Result
+         Output: prt1_p1.a, prt1_p1.c, (25), prt2_p1.b, prt2_p1.c, (50)
+         ->  Append
+               ->  Hash Full Join
+                     Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, (25), (50)
+                     Hash Cond: (prt1_p1.a = prt2_p1.b)
+                     ->  Seq Scan on public.prt1_p1
+                           Output: prt1_p1.a, prt1_p1.c, 25
+                           Filter: ((prt1_p1.a % 25) = 0)
+                     ->  Hash
+                           Output: prt2_p1.b, prt2_p1.c, (50)
+                           ->  Seq Scan on public.prt2_p1
+                                 Output: prt2_p1.b, prt2_p1.c, 50
+                                 Filter: ((prt2_p1.b % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c, (25), (50)
+                     Hash Cond: (prt1_p2.a = prt2_p2.b)
+                     ->  Seq Scan on public.prt1_p2
+                           Output: prt1_p2.a, prt1_p2.c, 25
+                           Filter: ((prt1_p2.a % 25) = 0)
+                     ->  Hash
+                           Output: prt2_p2.b, prt2_p2.c, (50)
+                           ->  Seq Scan on public.prt2_p2
+                                 Output: prt2_p2.b, prt2_p2.c, 50
+                                 Filter: ((prt2_p2.b % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c, (25), (50)
+                     Hash Cond: (prt1_p3.a = prt2_p3.b)
+                     ->  Seq Scan on public.prt1_p3
+                           Output: prt1_p3.a, prt1_p3.c, 25
+                           Filter: ((prt1_p3.a % 25) = 0)
+                     ->  Hash
+                           Output: prt2_p3.b, prt2_p3.c, (50)
+                           ->  Seq Scan on public.prt2_p3
+                                 Output: prt2_p3.b, prt2_p3.c, 50
+                                 Filter: ((prt2_p3.b % 25) = 0)
+(39 rows)
+
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 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)
+
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM uprt2 WHERE uprt2.b % 25 = 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)
+
+-- Join with pruned partitions from joining relations
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on public.prt2_p2 t2
+                     Output: t2.b, t2.c
+                     Filter: (t2.b > 250)
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.prt1_p2 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a < 450) AND ((t1.a % 25) = 0))
+(15 rows)
+
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+(1 row)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a < 450 AND t2.b > 250 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+(1 row)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                    QUERY PLAN                                    
+----------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, b, c
+   Sort Key: prt1_p1.a, b
+   ->  Append
+         ->  Nested Loop Left Join
+               Output: prt1_p1.a, prt1_p1.c, b, c
+               Join Filter: (prt1_p1.a = b)
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c
+                     Filter: ((prt1_p1.a < 450) AND ((prt1_p1.a % 25) = 0))
+               ->  Result
+                     Output: b, c
+                     One-Time Filter: false
+         ->  Hash Right Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt2_p2.b = prt1_p2.a)
+               ->  Seq Scan on public.prt2_p2
+                     Output: prt2_p2.b, prt2_p2.c
+                     Filter: (prt2_p2.b > 250)
+               ->  Hash
+                     Output: prt1_p2.a, prt1_p2.c
+                     ->  Seq Scan on public.prt1_p2
+                           Output: prt1_p2.a, prt1_p2.c
+                           Filter: ((prt1_p2.a < 450) AND ((prt1_p2.a % 25) = 0))
+(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.a % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.a % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                       QUERY PLAN                                       
+----------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+   Sort Key: prt1_p2.a, prt2_p2.b
+   ->  Result
+         Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: prt2_p2.b, prt2_p2.c, prt1_p2.a, prt1_p2.c
+                     Hash Cond: (prt1_p2.a = prt2_p2.b)
+                     ->  Seq Scan on public.prt1_p2
+                           Output: prt1_p2.a, prt1_p2.c
+                           Filter: (prt1_p2.a < 450)
+                     ->  Hash
+                           Output: prt2_p2.b, prt2_p2.c
+                           ->  Seq Scan on public.prt2_p2
+                                 Output: prt2_p2.b, prt2_p2.c
+                                 Filter: ((prt2_p2.b > 250) AND ((prt2_p2.a % 25) = 0))
+               ->  Nested Loop Left Join
+                     Output: prt2_p3.b, prt2_p3.c, a, c
+                     Join Filter: (a = prt2_p3.b)
+                     ->  Seq Scan on public.prt2_p3
+                           Output: prt2_p3.b, prt2_p3.c
+                           Filter: ((prt2_p3.b > 250) AND ((prt2_p3.a % 25) = 0))
+                     ->  Result
+                           Output: a, c
+                           One-Time Filter: false
+(26 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+     |      | 375 | 0375
+     |      | 450 | 0450
+     |      | 525 | 0525
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0300 | 300 | 0300
+     |      | 375 | 0375
+     |      | 450 | 0450
+     |      | 525 | 0525
+(4 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+                                    QUERY PLAN                                    
+----------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, b, c
+   Sort Key: prt1_p1.a, b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_p1.a, prt1_p1.c, b, c
+               Hash Cond: (prt1_p1.a = b)
+               ->  Seq Scan on public.prt1_p1
+                     Output: prt1_p1.a, prt1_p1.c
+                     Filter: ((prt1_p1.a < 450) AND ((prt1_p1.a % 25) = 0))
+               ->  Hash
+                     Output: b, c
+                     ->  Result
+                           Output: b, c
+                           One-Time Filter: false
+         ->  Hash Full Join
+               Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+               Hash Cond: (prt1_p2.a = prt2_p2.b)
+               ->  Seq Scan on public.prt1_p2
+                     Output: prt1_p2.a, prt1_p2.c
+                     Filter: ((prt1_p2.a < 450) AND ((prt1_p2.a % 25) = 0))
+               ->  Hash
+                     Output: prt2_p2.b, prt2_p2.c
+                     ->  Seq Scan on public.prt2_p2
+                           Output: prt2_p2.b, prt2_p2.c
+                           Filter: ((prt2_p2.b > 250) AND ((prt2_p2.b % 25) = 0))
+         ->  Hash Full Join
+               Output: a, c, prt2_p3.b, prt2_p3.c
+               Hash Cond: (prt2_p3.b = a)
+               ->  Seq Scan on public.prt2_p3
+                     Output: prt2_p3.b, prt2_p3.c
+                     Filter: ((prt2_p3.b > 250) AND ((prt2_p3.b % 25) = 0))
+               ->  Hash
+                     Output: a, c
+                     ->  Result
+                           Output: a, c
+                           One-Time Filter: false
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b 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)
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+                     QUERY PLAN                     
+----------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Output: t1.a, t1.b, t1.c
+         Merge Cond: (t1.a = t1_3.b)
+         ->  Sort
+               Output: t1.a, t1.b, t1.c
+               Sort Key: t1.a
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+         ->  Sort
+               Output: t1_3.b
+               Sort Key: t1_3.b
+               ->  Seq Scan on public.prt2_p1 t1_3
+                     Output: t1_3.b
+                     Filter: ((t1_3.b % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_2.a, t1_2.b, t1_2.c
+         Merge Cond: (t1_2.a = t1_4.b)
+         ->  Sort
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Sort Key: t1_2.a
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+         ->  Sort
+               Output: t1_4.b
+               Sort Key: t1_4.b
+               ->  Seq Scan on public.prt2_p2 t1_4
+                     Output: t1_4.b
+                     Filter: ((t1_4.b % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_1.a, t1_1.b, t1_1.c
+         Merge Cond: (t1_1.a = t1_5.b)
+         ->  Sort
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Sort Key: t1_1.a
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+         ->  Sort
+               Output: t1_5.b
+               Sort Key: t1_5.b
+               ->  Seq Scan on public.prt2_p3 t1_5
+                     Output: t1_5.b
+                     Filter: ((t1_5.b % 25) = 0)
+(47 rows)
+
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+-- lateral reference
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+                                             QUERY PLAN                                              
+-----------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   Sort Key: t1.a
+   ->  Result
+         Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+         ->  Append
+               ->  Nested Loop Left Join
+                     Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a))
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.b, t1.c
+                           Filter: ((t1.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+                           Hash Cond: (t3.b = t2.a)
+                           ->  Seq Scan on public.prt2_p1 t3
+                                 Output: t3.a, t3.b
+                           ->  Hash
+                                 Output: t2.a
+                                 ->  Seq Scan on public.prt1_p1 t2
+                                       Output: t2.a
+                                       Filter: (t1.a = t2.a)
+               ->  Nested Loop Left Join
+                     Output: t1_2.a, t1_2.b, t1_2.c, t2_2.a, t3_1.a, (LEAST(t1_2.a, t2_2.a, t3_1.a))
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.b, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2_2.a, t3_1.a, LEAST(t1_2.a, t2_2.a, t3_1.a)
+                           Hash Cond: (t3_1.b = t2_2.a)
+                           ->  Seq Scan on public.prt2_p2 t3_1
+                                 Output: t3_1.a, t3_1.b
+                           ->  Hash
+                                 Output: t2_2.a
+                                 ->  Seq Scan on public.prt1_p2 t2_2
+                                       Output: t2_2.a
+                                       Filter: (t1_2.a = t2_2.a)
+               ->  Nested Loop Left Join
+                     Output: t1_1.a, t1_1.b, t1_1.c, t2_1.a, t3_2.a, (LEAST(t1_1.a, t2_1.a, t3_2.a))
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.b, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2_1.a, t3_2.a, LEAST(t1_1.a, t2_1.a, t3_2.a)
+                           Hash Cond: (t3_2.b = t2_1.a)
+                           ->  Seq Scan on public.prt2_p3 t3_2
+                                 Output: t3_2.a, t3_2.b
+                           ->  Hash
+                                 Output: t2_1.a
+                                 ->  Seq Scan on public.prt1_p3 t2_1
+                                       Output: t2_1.a
+                                       Filter: (t1_1.a = t2_1.a)
+(51 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   Sort Key: t1.a
+   ->  Nested Loop Left Join
+         Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+         ->  Append
+               ->  Seq Scan on public.prt1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Seq Scan on public.prt1_p1 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Seq Scan on public.prt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Seq Scan on public.prt1_p2 t1_3
+                     Output: t1_3.a, t1_3.b, t1_3.c
+                     Filter: ((t1_3.a % 25) = 0)
+         ->  Append
+               ->  Hash Join
+                     Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+                     Hash Cond: (t3.b = t2.a)
+                     ->  Seq Scan on public.prt2_p1 t3
+                           Output: t3.a, t3.b
+                     ->  Hash
+                           Output: t2.a
+                           ->  Seq Scan on public.prt1_p1 t2
+                                 Output: t2.a
+                                 Filter: (t1.b = t2.a)
+               ->  Hash Join
+                     Output: t2_2.a, t3_1.a, LEAST(t1.a, t2_2.a, t3_1.a)
+                     Hash Cond: (t3_1.b = t2_2.a)
+                     ->  Seq Scan on public.prt2_p2 t3_1
+                           Output: t3_1.a, t3_1.b
+                     ->  Hash
+                           Output: t2_2.a
+                           ->  Seq Scan on public.prt1_p2 t2_2
+                                 Output: t2_2.a
+                                 Filter: (t1.b = t2_2.a)
+               ->  Hash Join
+                     Output: t2_1.a, t3_2.a, LEAST(t1.a, t2_1.a, t3_2.a)
+                     Hash Cond: (t3_2.b = t2_1.a)
+                     ->  Seq Scan on public.prt2_p3 t3_2
+                           Output: t3_2.a, t3_2.b
+                     ->  Hash
+                           Output: t2_1.a
+                           ->  Seq Scan on public.prt1_p3 t2_1
+                                 Output: t2_1.a
+                                 Filter: (t1.b = t2_1.a)
+(49 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+--
+-- partitioned by expression
+--
+CREATE TABLE prt1_e (a int, b int, c varchar) PARTITION BY RANGE(((a + b)/2));
+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);
+INSERT INTO prt1_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_e AS SELECT * FROM prt1_e;
+CREATE TABLE prt2_e (a int, b int, c varchar) PARTITION BY RANGE(((b + a)/2));
+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);
+INSERT INTO prt2_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_e;
+ANALYZE prt2_e_p1;
+ANALYZE prt2_e_p2;
+ANALYZE prt2_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_e AS SELECT * FROM prt2_e;
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                  QUERY PLAN                                  
+------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_e_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_e_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
+               ->  Seq Scan on public.prt2_e_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_e_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+(34 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.a % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1, uprt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.a % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                  QUERY PLAN                                  
+------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: (((t2.b + t2.a) / 2) = ((t1.a + t1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p1 t2
+                     Output: t2.b, t2.c, t2.a
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Seq Scan on public.prt1_e_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: (((t2_1.b + t2_1.a) / 2) = ((t1_1.a + t1_1.b) / 2))
+               ->  Seq Scan on public.prt2_e_p2 t2_1
+                     Output: t2_1.b, t2_1.c, t2_1.a
+               ->  Hash
+                     Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Seq Scan on public.prt1_e_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: (((t2_2.b + t2_2.a) / 2) = ((t1_2.a + t1_2.b) / 2))
+               ->  Seq Scan on public.prt2_e_p3 t2_2
+                     Output: t2_2.b, t2_2.c, t2_2.a
+               ->  Hash
+                     Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Seq Scan on public.prt1_e_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1 LEFT JOIN uprt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on public.prt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = t1.a)
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_2.a)
+                     ->  Seq Scan on public.prt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                           ->  Hash Join
+                                 Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = t1_2.a)
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_1.a)
+                     ->  Seq Scan on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                           ->  Hash Join
+                                 Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = t1_1.a)
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+(57 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+(4 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM uprt1 t1, uprt2 t2, uprt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: (((t3.a + t3.b) / 2) = t1.a)
+                     ->  Seq Scan on public.prt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = t1.a)
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: (((t3_1.a + t3_1.b) / 2) = t1_2.a)
+                     ->  Seq Scan on public.prt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = t1_2.a)
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: (((t3_2.a + t3_2.b) / 2) = t1_1.a)
+                     ->  Seq Scan on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = t1_1.a)
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+(57 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, 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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: (((t3.a + t3.b) / 2) = t2.b)
+                     ->  Seq Scan on public.prt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = t1.a)
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: (((t3_1.a + t3_1.b) / 2) = t2_1.b)
+                     ->  Seq Scan on public.prt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = t1_2.a)
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: (((t3_2.a + t3_2.b) / 2) = t2_2.b)
+                     ->  Seq Scan on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = t1_1.a)
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+(57 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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |          | 
+ 100 | 0100 |     |      |          | 
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |          | 
+ 250 | 0250 |     |      |          | 
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |          | 
+ 400 | 0400 |     |      |          | 
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |          | 
+ 550 | 0550 |     |      |          | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |          | 
+ 100 | 0100 |     |      |          | 
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |          | 
+ 250 | 0250 |     |      |          | 
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |          | 
+ 400 | 0400 |     |      |          | 
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |          | 
+ 550 | 0550 |     |      |          | 
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Hash Cond: (t2.b = t1.a)
+                     ->  Seq Scan on public.prt2_p1 t2
+                           Output: t2.b, t2.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Hash Cond: (t1.a = ((t3.a + t3.b) / 2))
+                                 ->  Seq Scan on public.prt1_p1 t1
+                                       Output: t1.a, t1.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                     Hash Cond: (t2_1.b = t1_2.a)
+                     ->  Seq Scan on public.prt2_p2 t2_1
+                           Output: t2_1.b, t2_1.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                                 Hash Cond: (t1_2.a = ((t3_1.a + t3_1.b) / 2))
+                                 ->  Seq Scan on public.prt1_p2 t1_2
+                                       Output: t1_2.a, t1_2.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                     Hash Cond: (t2_2.b = t1_1.a)
+                     ->  Seq Scan on public.prt2_p3 t2_2
+                           Output: t2_2.b, t2_2.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                                 Hash Cond: (t1_1.a = ((t3_2.a + t3_2.b) / 2))
+                                 ->  Seq Scan on public.prt1_p3 t1_1
+                                       Output: t1_1.a, t1_1.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: (t1.a = t2.b)
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Hash Cond: (t2.b = ((t3.a + t3.b) / 2))
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Hash Cond: (t1_2.a = t2_1.b)
+                     ->  Seq Scan on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: (t2_1.b = ((t3_1.a + t3_1.b) / 2))
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Hash Cond: (t1_1.a = t2_2.b)
+                     ->  Seq Scan on public.prt1_p3 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: (t2_2.b = ((t3_2.a + t3_2.b) / 2))
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                                  QUERY PLAN                                                   
+---------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, ((prt1_e_p1.a + prt1_e_p1.b)), prt1_e_p1.c
+   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   ->  Result
+         Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, (prt1_e_p1.a + prt1_e_p1.b), prt1_e_p1.c
+         ->  Append
+               ->  Hash Full Join
+                     Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c, prt1_e_p1.a, prt1_e_p1.b, prt1_e_p1.c
+                     Hash Cond: (prt1_p1.a = ((prt1_e_p1.a + prt1_e_p1.b) / 2))
+                     ->  Hash Full Join
+                           Output: prt1_p1.a, prt1_p1.c, prt2_p1.b, prt2_p1.c
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on public.prt1_p1
+                                 Output: prt1_p1.a, prt1_p1.c
+                                 Filter: ((prt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p1.b, prt2_p1.c
+                                 ->  Seq Scan on public.prt2_p1
+                                       Output: prt2_p1.b, prt2_p1.c
+                                       Filter: ((prt2_p1.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p1.a, prt1_e_p1.b, prt1_e_p1.c
+                           ->  Seq Scan on public.prt1_e_p1
+                                 Output: prt1_e_p1.a, prt1_e_p1.b, prt1_e_p1.c
+                                 Filter: ((prt1_e_p1.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c, prt1_e_p2.a, prt1_e_p2.b, prt1_e_p2.c
+                     Hash Cond: (prt1_p2.a = ((prt1_e_p2.a + prt1_e_p2.b) / 2))
+                     ->  Hash Full Join
+                           Output: prt1_p2.a, prt1_p2.c, prt2_p2.b, prt2_p2.c
+                           Hash Cond: (prt1_p2.a = prt2_p2.b)
+                           ->  Seq Scan on public.prt1_p2
+                                 Output: prt1_p2.a, prt1_p2.c
+                                 Filter: ((prt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p2.b, prt2_p2.c
+                                 ->  Seq Scan on public.prt2_p2
+                                       Output: prt2_p2.b, prt2_p2.c
+                                       Filter: ((prt2_p2.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p2.a, prt1_e_p2.b, prt1_e_p2.c
+                           ->  Seq Scan on public.prt1_e_p2
+                                 Output: prt1_e_p2.a, prt1_e_p2.b, prt1_e_p2.c
+                                 Filter: ((prt1_e_p2.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c, prt1_e_p3.a, prt1_e_p3.b, prt1_e_p3.c
+                     Hash Cond: (prt1_p3.a = ((prt1_e_p3.a + prt1_e_p3.b) / 2))
+                     ->  Hash Full Join
+                           Output: prt1_p3.a, prt1_p3.c, prt2_p3.b, prt2_p3.c
+                           Hash Cond: (prt1_p3.a = prt2_p3.b)
+                           ->  Seq Scan on public.prt1_p3
+                                 Output: prt1_p3.a, prt1_p3.c
+                                 Filter: ((prt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p3.b, prt2_p3.c
+                                 ->  Seq Scan on public.prt2_p3
+                                       Output: prt2_p3.b, prt2_p3.c
+                                       Filter: ((prt2_p3.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p3.a, prt1_e_p3.b, prt1_e_p3.c
+                           ->  Seq Scan on public.prt1_e_p3
+                                 Output: prt1_e_p3.a, prt1_e_p3.b, prt1_e_p3.c
+                                 Filter: ((prt1_e_p3.a % 25) = 0)
+(63 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+     |      |  75 | 0075 |          | 
+     |      | 225 | 0225 |          | 
+     |      | 375 | 0375 |          | 
+     |      | 525 | 0525 |          | 
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM uprt1_e WHERE uprt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+     |      |  75 | 0075 |          | 
+     |      | 225 | 0225 |          | 
+     |      | 375 | 0375 |          | 
+     |      | 525 | 0525 |          | 
+(16 rows)
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, 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.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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                                                      
+----------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_p1.a, (50), prt2_p1.b, (75), ((prt1_e_p1.a + prt1_e_p1.b)), (50)
+   Sort Key: prt1_p1.a, prt2_p1.b, ((prt1_e_p1.a + prt1_e_p1.b))
+   ->  Result
+         Output: prt1_p1.a, (50), prt2_p1.b, (75), (prt1_e_p1.a + prt1_e_p1.b), (50)
+         ->  Append
+               ->  Hash Full Join
+                     Output: prt1_p1.a, prt2_p1.b, prt1_e_p1.a, prt1_e_p1.b, (50), (75), (50)
+                     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
+                           Output: prt1_p1.a, prt2_p1.b, (50), (75)
+                           Hash Cond: (prt1_p1.a = prt2_p1.b)
+                           ->  Seq Scan on public.prt1_p1
+                                 Output: prt1_p1.a, 50
+                                 Filter: ((prt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p1.b, (75)
+                                 ->  Seq Scan on public.prt2_p1
+                                       Output: prt2_p1.b, 75
+                                       Filter: ((prt2_p1.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p1.a, prt1_e_p1.b, (50)
+                           ->  Seq Scan on public.prt1_e_p1
+                                 Output: prt1_e_p1.a, prt1_e_p1.b, 50
+                                 Filter: ((prt1_e_p1.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p2.a, prt2_p2.b, prt1_e_p2.a, prt1_e_p2.b, (50), (75), (50)
+                     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
+                           Output: prt1_p2.a, prt2_p2.b, (50), (75)
+                           Hash Cond: (prt1_p2.a = prt2_p2.b)
+                           ->  Seq Scan on public.prt1_p2
+                                 Output: prt1_p2.a, 50
+                                 Filter: ((prt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p2.b, (75)
+                                 ->  Seq Scan on public.prt2_p2
+                                       Output: prt2_p2.b, 75
+                                       Filter: ((prt2_p2.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p2.a, prt1_e_p2.b, (50)
+                           ->  Seq Scan on public.prt1_e_p2
+                                 Output: prt1_e_p2.a, prt1_e_p2.b, 50
+                                 Filter: ((prt1_e_p2.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: prt1_p3.a, prt2_p3.b, prt1_e_p3.a, prt1_e_p3.b, (50), (75), (50)
+                     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
+                           Output: prt1_p3.a, prt2_p3.b, (50), (75)
+                           Hash Cond: (prt1_p3.a = prt2_p3.b)
+                           ->  Seq Scan on public.prt1_p3
+                                 Output: prt1_p3.a, 50
+                                 Filter: ((prt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: prt2_p3.b, (75)
+                                 ->  Seq Scan on public.prt2_p3
+                                       Output: prt2_p3.b, 75
+                                       Filter: ((prt2_p3.b % 25) = 0)
+                     ->  Hash
+                           Output: prt1_e_p3.a, prt1_e_p3.b, (50)
+                           ->  Seq Scan on public.prt1_e_p3
+                                 Output: prt1_e_p3.a, prt1_e_p3.b, 50
+                                 Filter: ((prt1_e_p3.a % 25) = 0)
+(66 rows)
+
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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)
+
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM uprt1_e WHERE uprt1_e.a % 25 = 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)
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+                            QUERY PLAN                             
+-------------------------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Output: t1.a, t1.b, t1.c
+         Merge Cond: (t1.a = t1_3.b)
+         ->  Sort
+               Output: t1.a, t1.b, t1.c
+               Sort Key: t1.a
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+         ->  Sort
+               Output: t1_3.b, t2.a, t2.b
+               Sort Key: t1_3.b
+               ->  Hash Join
+                     Output: t1_3.b, t2.a, t2.b
+                     Hash Cond: (((t2.a + t2.b) / 2) = t1_3.b)
+                     ->  Seq Scan on public.prt1_e_p1 t2
+                           Output: t2.a, t2.b
+                     ->  Hash
+                           Output: t1_3.b
+                           ->  Seq Scan on public.prt2_p1 t1_3
+                                 Output: t1_3.b
+                                 Filter: ((t1_3.b % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_2.a, t1_2.b, t1_2.c
+         Merge Cond: (t1_2.a = t1_4.b)
+         ->  Sort
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Sort Key: t1_2.a
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+         ->  Sort
+               Output: t1_4.b, t2_1.a, t2_1.b
+               Sort Key: t1_4.b
+               ->  Hash Join
+                     Output: t1_4.b, t2_1.a, t2_1.b
+                     Hash Cond: (((t2_1.a + t2_1.b) / 2) = t1_4.b)
+                     ->  Seq Scan on public.prt1_e_p2 t2_1
+                           Output: t2_1.a, t2_1.b
+                     ->  Hash
+                           Output: t1_4.b
+                           ->  Seq Scan on public.prt2_p2 t1_4
+                                 Output: t1_4.b
+                                 Filter: ((t1_4.b % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_1.a, t1_1.b, t1_1.c
+         Merge Cond: (t1_1.a = t1_5.b)
+         ->  Sort
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Sort Key: t1_1.a
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+         ->  Sort
+               Output: t1_5.b, t2_2.a, t2_2.b
+               Sort Key: t1_5.b
+               ->  Hash Join
+                     Output: t1_5.b, t2_2.a, t2_2.b
+                     Hash Cond: (((t2_2.a + t2_2.b) / 2) = t1_5.b)
+                     ->  Seq Scan on public.prt1_e_p3 t2_2
+                           Output: t2_2.a, t2_2.b
+                     ->  Hash
+                           Output: t1_5.b
+                           ->  Seq Scan on public.prt2_p3 t1_5
+                                 Output: t1_5.b
+                                 Filter: ((t1_5.b % 25) = 0)
+(68 rows)
+
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1, uprt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+                            QUERY PLAN                             
+-------------------------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Output: t1.a, t1.b, t1.c
+         Merge Cond: (t1.a = t1_3.b)
+         ->  Sort
+               Output: t1.a, t1.b, t1.c
+               Sort Key: t1.a
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+         ->  Sort
+               Output: t1_3.b, t1_6.a, t1_6.b
+               Sort Key: t1_3.b
+               ->  Hash Semi Join
+                     Output: t1_3.b, t1_6.a, t1_6.b
+                     Hash Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
+                     ->  Seq Scan on public.prt2_p1 t1_3
+                           Output: t1_3.b
+                     ->  Hash
+                           Output: t1_6.a, t1_6.b
+                           ->  Seq Scan on public.prt1_e_p1 t1_6
+                                 Output: t1_6.a, t1_6.b
+                                 Filter: ((t1_6.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_2.a, t1_2.b, t1_2.c
+         Merge Cond: (t1_2.a = t1_4.b)
+         ->  Sort
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Sort Key: t1_2.a
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+         ->  Sort
+               Output: t1_4.b, t1_7.a, t1_7.b
+               Sort Key: t1_4.b
+               ->  Hash Semi Join
+                     Output: t1_4.b, t1_7.a, t1_7.b
+                     Hash Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
+                     ->  Seq Scan on public.prt2_p2 t1_4
+                           Output: t1_4.b
+                     ->  Hash
+                           Output: t1_7.a, t1_7.b
+                           ->  Seq Scan on public.prt1_e_p2 t1_7
+                                 Output: t1_7.a, t1_7.b
+                                 Filter: ((t1_7.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_1.a, t1_1.b, t1_1.c
+         Merge Cond: (t1_1.a = t1_5.b)
+         ->  Sort
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Sort Key: t1_1.a
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+         ->  Sort
+               Output: t1_5.b, t1_8.a, t1_8.b
+               Sort Key: t1_5.b
+               ->  Hash Semi Join
+                     Output: t1_5.b, t1_8.a, t1_8.b
+                     Hash Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
+                     ->  Seq Scan on public.prt2_p3 t1_5
+                           Output: t1_5.b
+                     ->  Hash
+                           Output: t1_8.a, t1_8.b
+                           ->  Seq Scan on public.prt1_e_p3 t1_8
+                                 Output: t1_8.a, t1_8.b
+                                 Filter: ((t1_8.a % 25) = 0)
+(68 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+-- test merge joins with and without using indexes
+SET enable_hashjoin TO off;
+SET enable_nestloop TO off;
+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 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 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));
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                             QUERY PLAN                                              
+-----------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Merge Cond: (t2.b = t1.a)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           Sort Key: t2.b
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Merge Cond: ((((t3.a + t3.b) / 2)) = t2.b)
+                                 ->  Sort
+                                       Output: t3.a, t3.b, t3.c, (((t3.a + t3.b) / 2))
+                                       Sort Key: (((t3.a + t3.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c, ((t3.a + t3.b) / 2)
+                                             Filter: ((t3.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2.b, t2.c
+                                       Sort Key: t2.b
+                                       ->  Seq Scan on public.prt2_p1 t2
+                                             Output: t2.b, t2.c
+                     ->  Sort
+                           Output: t1.a, t1.c
+                           Sort Key: t1.a
+                           ->  Seq Scan on public.prt1_p1 t1
+                                 Output: t1.a, t1.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Merge Cond: (t2_1.b = t1_2.a)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           Sort Key: t2_1.b
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t2_1.b)
+                                 ->  Sort
+                                       Output: t3_1.a, t3_1.b, t3_1.c, (((t3_1.a + t3_1.b) / 2))
+                                       Sort Key: (((t3_1.a + t3_1.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c, ((t3_1.a + t3_1.b) / 2)
+                                             Filter: ((t3_1.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2_1.b, t2_1.c
+                                       Sort Key: t2_1.b
+                                       ->  Seq Scan on public.prt2_p2 t2_1
+                                             Output: t2_1.b, t2_1.c
+                     ->  Sort
+                           Output: t1_2.a, t1_2.c
+                           Sort Key: t1_2.a
+                           ->  Seq Scan on public.prt1_p2 t1_2
+                                 Output: t1_2.a, t1_2.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Merge Cond: (t2_2.b = t1_1.a)
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           Sort Key: t2_2.b
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t2_2.b)
+                                 ->  Sort
+                                       Output: t3_2.a, t3_2.b, t3_2.c, (((t3_2.a + t3_2.b) / 2))
+                                       Sort Key: (((t3_2.a + t3_2.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c, ((t3_2.a + t3_2.b) / 2)
+                                             Filter: ((t3_2.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t2_2.b, t2_2.c
+                                       Sort Key: t2_2.b
+                                       ->  Seq Scan on public.prt2_p3 t2_2
+                                             Output: t2_2.b, t2_2.c
+                     ->  Sort
+                           Output: t1_1.a, t1_1.c
+                           Sort Key: t1_1.a
+                           ->  Seq Scan on public.prt1_p3 t1_1
+                                 Output: t1_1.a, t1_1.c
+(81 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+                                   QUERY PLAN                                    
+---------------------------------------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Output: t1.a, t1.b, t1.c
+         Merge Cond: (t1.a = t1_3.b)
+         ->  Sort
+               Output: t1.a, t1.b, t1.c
+               Sort Key: t1.a
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_3.b, t1_6.a, t1_6.b
+               ->  Merge Semi Join
+                     Output: t1_3.b, t1_6.a, t1_6.b
+                     Merge Cond: (t1_3.b = (((t1_6.a + t1_6.b) / 2)))
+                     ->  Sort
+                           Output: t1_3.b
+                           Sort Key: t1_3.b
+                           ->  Seq Scan on public.prt2_p1 t1_3
+                                 Output: t1_3.b
+                     ->  Sort
+                           Output: t1_6.a, t1_6.b, (((t1_6.a + t1_6.b) / 2))
+                           Sort Key: (((t1_6.a + t1_6.b) / 2))
+                           ->  Seq Scan on public.prt1_e_p1 t1_6
+                                 Output: t1_6.a, t1_6.b, ((t1_6.a + t1_6.b) / 2)
+                                 Filter: ((t1_6.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_2.a, t1_2.b, t1_2.c
+         Merge Cond: (t1_2.a = t1_4.b)
+         ->  Sort
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Sort Key: t1_2.a
+               ->  Seq Scan on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+         ->  Materialize
+               Output: t1_4.b, t1_7.a, t1_7.b
+               ->  Merge Semi Join
+                     Output: t1_4.b, t1_7.a, t1_7.b
+                     Merge Cond: (t1_4.b = (((t1_7.a + t1_7.b) / 2)))
+                     ->  Sort
+                           Output: t1_4.b
+                           Sort Key: t1_4.b
+                           ->  Seq Scan on public.prt2_p2 t1_4
+                                 Output: t1_4.b
+                     ->  Sort
+                           Output: t1_7.a, t1_7.b, (((t1_7.a + t1_7.b) / 2))
+                           Sort Key: (((t1_7.a + t1_7.b) / 2))
+                           ->  Seq Scan on public.prt1_e_p2 t1_7
+                                 Output: t1_7.a, t1_7.b, ((t1_7.a + t1_7.b) / 2)
+                                 Filter: ((t1_7.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_1.a, t1_1.b, t1_1.c
+         Merge Cond: (t1_1.a = t1_5.b)
+         ->  Sort
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Sort Key: t1_1.a
+               ->  Seq Scan on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_5.b, t1_8.a, t1_8.b
+               ->  Merge Semi Join
+                     Output: t1_5.b, t1_8.a, t1_8.b
+                     Merge Cond: (t1_5.b = (((t1_8.a + t1_8.b) / 2)))
+                     ->  Sort
+                           Output: t1_5.b
+                           Sort Key: t1_5.b
+                           ->  Seq Scan on public.prt2_p3 t1_5
+                                 Output: t1_5.b
+                     ->  Sort
+                           Output: t1_8.a, t1_8.b, (((t1_8.a + t1_8.b) / 2))
+                           Sort Key: (((t1_8.a + t1_8.b) / 2))
+                           ->  Seq Scan on public.prt1_e_p3 t1_8
+                                 Output: t1_8.a, t1_8.b, ((t1_8.a + t1_8.b) / 2)
+                                 Filter: ((t1_8.a % 25) = 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                             QUERY PLAN                                              
+-----------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Merge Cond: (t1.a = t2.b)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           Sort Key: t1.a
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Merge Cond: ((((t3.a + t3.b) / 2)) = t1.a)
+                                 ->  Sort
+                                       Output: t3.a, t3.b, t3.c, (((t3.a + t3.b) / 2))
+                                       Sort Key: (((t3.a + t3.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c, ((t3.a + t3.b) / 2)
+                                             Filter: ((t3.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t1.a, t1.c
+                                       Sort Key: t1.a
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.c
+                     ->  Sort
+                           Output: t2.b, t2.c
+                           Sort Key: t2.b
+                           ->  Seq Scan on public.prt2_p1 t2
+                                 Output: t2.b, t2.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                     Merge Cond: (t1_2.a = t2_1.b)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                           Sort Key: t1_2.a
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                                 Merge Cond: ((((t3_1.a + t3_1.b) / 2)) = t1_2.a)
+                                 ->  Sort
+                                       Output: t3_1.a, t3_1.b, t3_1.c, (((t3_1.a + t3_1.b) / 2))
+                                       Sort Key: (((t3_1.a + t3_1.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c, ((t3_1.a + t3_1.b) / 2)
+                                             Filter: ((t3_1.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t1_2.a, t1_2.c
+                                       Sort Key: t1_2.a
+                                       ->  Seq Scan on public.prt1_p2 t1_2
+                                             Output: t1_2.a, t1_2.c
+                     ->  Sort
+                           Output: t2_1.b, t2_1.c
+                           Sort Key: t2_1.b
+                           ->  Seq Scan on public.prt2_p2 t2_1
+                                 Output: t2_1.b, t2_1.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                     Merge Cond: (t1_1.a = t2_2.b)
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                           Sort Key: t1_1.a
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                                 Merge Cond: ((((t3_2.a + t3_2.b) / 2)) = t1_1.a)
+                                 ->  Sort
+                                       Output: t3_2.a, t3_2.b, t3_2.c, (((t3_2.a + t3_2.b) / 2))
+                                       Sort Key: (((t3_2.a + t3_2.b) / 2))
+                                       ->  Seq Scan on public.prt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c, ((t3_2.a + t3_2.b) / 2)
+                                             Filter: ((t3_2.a % 25) = 0)
+                                 ->  Sort
+                                       Output: t1_1.a, t1_1.c
+                                       Sort Key: t1_1.a
+                                       ->  Seq Scan on public.prt1_p3 t1_1
+                                             Output: t1_1.a, t1_1.c
+                     ->  Sort
+                           Output: t2_2.b, t2_2.c
+                           Sort Key: t2_2.b
+                           ->  Seq Scan on public.prt2_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+(81 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SET enable_seqscan TO off;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                          QUERY PLAN                                          
+----------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Merge Cond: (t2.b = t1.a)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           Sort Key: t2.b
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Merge Cond: (((t3.a + t3.b) / 2) = t2.b)
+                                 ->  Index Scan using iprt1_e_p1_ab2 on public.prt1_e_p1 t3
+                                       Output: t3.a, t3.b, t3.c
+                                       Filter: ((t3.a % 25) = 0)
+                                 ->  Index Scan using iprt2_p1_b on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                     ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_2.a, t1_2.c
+                     Merge Cond: (t2_1.b = t1_2.a)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           Sort Key: t2_1.b
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Merge Cond: (((t3_1.a + t3_1.b) / 2) = t2_1.b)
+                                 ->  Index Scan using iprt1_e_p2_ab2 on public.prt1_e_p2 t3_1
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       Filter: ((t3_1.a % 25) = 0)
+                                 ->  Index Scan using iprt2_p2_b on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                     ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+               ->  Merge Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                     Merge Cond: (t2_2.b = ((t3_2.a + t3_2.b) / 2))
+                     ->  Merge Left Join
+                           Output: t2_2.b, t2_2.c, t1_1.a, t1_1.c
+                           Merge Cond: (t2_2.b = t1_1.a)
+                           ->  Index Scan using iprt2_p3_b on public.prt2_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+                           ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_1
+                                 Output: t1_1.a, t1_1.c
+                     ->  Index Scan using iprt1_e_p3_ab2 on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                           Filter: ((t3_2.a % 25) = 0)
+(51 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+     |      |     |      |      100 | 0050
+     |      |     |      |      200 | 0100
+     |      |     |      |      400 | 0200
+     |      |     |      |      500 | 0250
+     |      |     |      |      700 | 0350
+     |      |     |      |      800 | 0400
+     |      |     |      |     1000 | 0500
+     |      |     |      |     1100 | 0550
+(12 rows)
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+                                    QUERY PLAN                                    
+----------------------------------------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Output: t1.a, t1.b, t1.c
+         Merge Cond: (t1.a = t1_3.b)
+         ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1
+               Output: t1.a, t1.b, t1.c
+               Filter: ((t1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_3.b, t1_6.a, t1_6.b
+               ->  Merge Semi Join
+                     Output: t1_3.b, t1_6.a, t1_6.b
+                     Merge Cond: (t1_3.b = ((t1_6.a + t1_6.b) / 2))
+                     ->  Index Only Scan using iprt2_p1_b on public.prt2_p1 t1_3
+                           Output: t1_3.b
+                     ->  Index Scan using iprt1_e_p1_ab2 on public.prt1_e_p1 t1_6
+                           Output: t1_6.a, t1_6.b
+                           Filter: ((t1_6.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_2.a, t1_2.b, t1_2.c
+         Merge Cond: (t1_2.a = t1_4.b)
+         ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_2
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Filter: ((t1_2.a % 25) = 0)
+         ->  Materialize
+               Output: t1_4.b, t1_7.a, t1_7.b
+               ->  Merge Semi Join
+                     Output: t1_4.b, t1_7.a, t1_7.b
+                     Merge Cond: (t1_4.b = ((t1_7.a + t1_7.b) / 2))
+                     ->  Index Only Scan using iprt2_p2_b on public.prt2_p2 t1_4
+                           Output: t1_4.b
+                     ->  Index Scan using iprt1_e_p2_ab2 on public.prt1_e_p2 t1_7
+                           Output: t1_7.a, t1_7.b
+                           Filter: ((t1_7.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_1.a, t1_1.b, t1_1.c
+         Merge Cond: (t1_1.a = t1_5.b)
+         ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_1
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Filter: ((t1_1.a % 25) = 0)
+         ->  Materialize
+               Output: t1_5.b, t1_8.a, t1_8.b
+               ->  Merge Semi Join
+                     Output: t1_5.b, t1_8.a, t1_8.b
+                     Merge Cond: (t1_5.b = ((t1_8.a + t1_8.b) / 2))
+                     ->  Index Only Scan using iprt2_p3_b on public.prt2_p3 t1_5
+                           Output: t1_5.b
+                     ->  Index Scan using iprt1_e_p3_ab2 on public.prt1_e_p3 t1_8
+                           Output: t1_8.a, t1_8.b
+                           Filter: ((t1_8.a % 25) = 0)
+(50 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0150
+ 300 | 300 | 0300
+ 450 | 450 | 0450
+(4 rows)
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                          QUERY PLAN                                          
+----------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Merge Cond: (t1.a = t2.b)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           Sort Key: t1.a
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Merge Cond: (((t3.a + t3.b) / 2) = t1.a)
+                                 ->  Index Scan using iprt1_e_p1_ab2 on public.prt1_e_p1 t3
+                                       Output: t3.a, t3.b, t3.c
+                                       Filter: ((t3.a % 25) = 0)
+                                 ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1
+                                       Output: t1.a, t1.c
+                     ->  Index Scan using iprt2_p1_b on public.prt2_p1 t2
+                           Output: t2.b, t2.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c, t2_1.b, t2_1.c
+                     Merge Cond: (t1_2.a = t2_1.b)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                           Sort Key: t1_2.a
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_2.a, t1_2.c
+                                 Merge Cond: (((t3_1.a + t3_1.b) / 2) = t1_2.a)
+                                 ->  Index Scan using iprt1_e_p2_ab2 on public.prt1_e_p2 t3_1
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       Filter: ((t3_1.a % 25) = 0)
+                                 ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_2
+                                       Output: t1_2.a, t1_2.c
+                     ->  Index Scan using iprt2_p2_b on public.prt2_p2 t2_1
+                           Output: t2_1.b, t2_1.c
+               ->  Merge Left Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c, t2_2.b, t2_2.c
+                     Merge Cond: (t1_1.a = t2_2.b)
+                     ->  Sort
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                           Sort Key: t1_1.a
+                           ->  Merge Left Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_1.a, t1_1.c
+                                 Merge Cond: (((t3_2.a + t3_2.b) / 2) = t1_1.a)
+                                 ->  Index Scan using iprt1_e_p3_ab2 on public.prt1_e_p3 t3_2
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       Filter: ((t3_2.a % 25) = 0)
+                                 ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_1
+                                       Output: t1_1.a, t1_1.c
+                     ->  Index Scan using iprt2_p3_b on public.prt2_p3 t2_2
+                           Output: t2_2.b, t2_2.c
+(54 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |  c   
+-----+------+-----+------+----------+------
+   0 | 0000 |   0 | 0000 |        0 | 0000
+  50 | 0050 |     |      |      100 | 0050
+ 100 | 0100 |     |      |      200 | 0100
+ 150 | 0150 | 150 | 0150 |      300 | 0150
+ 200 | 0200 |     |      |      400 | 0200
+ 250 | 0250 |     |      |      500 | 0250
+ 300 | 0300 | 300 | 0300 |      600 | 0300
+ 350 | 0350 |     |      |      700 | 0350
+ 400 | 0400 |     |      |      800 | 0400
+ 450 | 0450 | 450 | 0450 |      900 | 0450
+ 500 | 0500 |     |      |     1000 | 0500
+ 550 | 0550 |     |      |     1100 | 0550
+(12 rows)
+
+-- lateral references and parameterized paths
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Result
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   ->  Merge Append
+         Sort Key: t1.a
+         ->  Nested Loop Left Join
+               Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a))
+               ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Merge Join
+                     Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+                     Merge Cond: (t2.a = t3.b)
+                     ->  Index Only Scan using iprt1_p1_a on public.prt1_p1 t2
+                           Output: t2.a
+                           Index Cond: (t2.a = t1.a)
+                     ->  Index Scan using iprt2_p1_b on public.prt2_p1 t3
+                           Output: t3.a, t3.b
+         ->  Nested Loop Left Join
+               Output: t1_2.a, t1_2.b, t1_2.c, t2_2.a, t3_1.a, (LEAST(t1_2.a, t2_2.a, t3_1.a))
+               ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Merge Join
+                     Output: t2_2.a, t3_1.a, LEAST(t1_2.a, t2_2.a, t3_1.a)
+                     Merge Cond: (t2_2.a = t3_1.b)
+                     ->  Index Only Scan using iprt1_p2_a on public.prt1_p2 t2_2
+                           Output: t2_2.a
+                           Index Cond: (t2_2.a = t1_2.a)
+                     ->  Index Scan using iprt2_p2_b on public.prt2_p2 t3_1
+                           Output: t3_1.a, t3_1.b
+         ->  Nested Loop Left Join
+               Output: t1_1.a, t1_1.b, t1_1.c, t2_1.a, t3_2.a, (LEAST(t1_1.a, t2_1.a, t3_2.a))
+               ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Merge Join
+                     Output: t2_1.a, t3_2.a, LEAST(t1_1.a, t2_1.a, t3_2.a)
+                     Merge Cond: (t2_1.a = t3_2.b)
+                     ->  Index Only Scan using iprt1_p3_a on public.prt1_p3 t2_1
+                           Output: t2_1.a
+                           Index Cond: (t2_1.a = t1_1.a)
+                     ->  Index Scan using iprt2_p3_b on public.prt2_p3 t3_2
+                           Output: t3_2.a, t3_2.b
+(43 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
+ Nested Loop Left Join
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   ->  Merge Append
+         Sort Key: t1.a
+         ->  Sort
+               Output: t1.a, t1.b, t1.c
+               Sort Key: t1.a
+               ->  Seq Scan on public.prt1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+         ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1_1
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Filter: ((t1_1.a % 25) = 0)
+         ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_2
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Filter: ((t1_2.a % 25) = 0)
+         ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_3
+               Output: t1_3.a, t1_3.b, t1_3.c
+               Filter: ((t1_3.a % 25) = 0)
+   ->  Append
+         ->  Merge Join
+               Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+               Merge Cond: (t2.a = t3.b)
+               ->  Index Only Scan using iprt1_p1_a on public.prt1_p1 t2
+                     Output: t2.a
+                     Index Cond: (t2.a = t1.b)
+               ->  Index Scan using iprt2_p1_b on public.prt2_p1 t3
+                     Output: t3.a, t3.b
+         ->  Merge Join
+               Output: t2_2.a, t3_1.a, LEAST(t1.a, t2_2.a, t3_1.a)
+               Merge Cond: (t2_2.a = t3_1.b)
+               ->  Index Only Scan using iprt1_p2_a on public.prt1_p2 t2_2
+                     Output: t2_2.a
+                     Index Cond: (t2_2.a = t1.b)
+               ->  Index Scan using iprt2_p2_b on public.prt2_p2 t3_1
+                     Output: t3_1.a, t3_1.b
+         ->  Merge Join
+               Output: t2_1.a, t3_2.a, LEAST(t1.a, t2_1.a, t3_2.a)
+               Merge Cond: (t2_1.a = t3_2.b)
+               ->  Index Only Scan using iprt1_p3_a on public.prt1_p3 t2_1
+                     Output: t2_1.a
+                     Index Cond: (t2_1.a = t1.b)
+               ->  Index Scan using iprt2_p3_b on public.prt2_p3 t3_2
+                     Output: t3_2.a, t3_2.b
+(44 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t3a | least 
+-----+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |     |      
+ 100 | 100 | 0100 |     |     |      
+ 150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |     |      
+ 250 | 250 | 0250 |     |     |      
+ 300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |     |      
+ 400 | 400 | 0400 |     |     |      
+ 450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |     |      
+ 550 | 550 | 0550 |     |     |      
+(12 rows)
+
+RESET enable_hashjoin;
+RESET enable_nestloop;
+RESET enable_seqscan;
+--
+-- partitioned by multiple columns
+--
+CREATE TABLE prt1_m (a int, b int, c varchar) 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, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_m;
+ANALYZE prt1_m_p1;
+ANALYZE prt1_m_p2;
+ANALYZE prt1_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_m AS SELECT * FROM prt1_m;
+CREATE TABLE prt2_m (a int, b int, c varchar) 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, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_m;
+ANALYZE prt2_m_p1;
+ANALYZE prt2_m_p2;
+ANALYZE prt2_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_m AS SELECT * FROM prt2_m;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+                                                 QUERY PLAN                                                 
+------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: ((((t1.a + t1.b) / 2) = t2.b) AND (t1.a = ((t2.b + t2.a) / 2)))
+                     ->  Seq Scan on public.prt1_m_p1 t1
+                           Output: t1.a, t1.c, t1.b
+                     ->  Hash
+                           Output: t2.b, t2.c, t2.a
+                           ->  Seq Scan on public.prt2_m_p1 t2
+                                 Output: t2.b, t2.c, t2.a
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: ((((t1_1.a + t1_1.b) / 2) = t2_1.b) AND (t1_1.a = ((t2_1.b + t2_1.a) / 2)))
+                     ->  Seq Scan on public.prt1_m_p2 t1_1
+                           Output: t1_1.a, t1_1.c, t1_1.b
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c, t2_1.a
+                           ->  Seq Scan on public.prt2_m_p2 t2_1
+                                 Output: t2_1.b, t2_1.c, t2_1.a
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: ((((t1_2.a + t1_2.b) / 2) = t2_2.b) AND (t1_2.a = ((t2_2.b + t2_2.a) / 2)))
+                     ->  Seq Scan on public.prt1_m_p3 t1_2
+                           Output: t1_2.a, t1_2.c, t1_2.b
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c, t2_2.a
+                           ->  Seq Scan on public.prt2_m_p3 t2_2
+                                 Output: t2_2.b, t2_2.c, t2_2.a
+                                 Filter: ((t2_2.b % 25) = 0)
+(36 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_m t1 RIGHT JOIN uprt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 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
+   Output: prt1_m_p1.a, prt1_m_p1.c, prt2_m_p1.b, prt2_m_p1.c
+   Sort Key: prt1_m_p1.a, prt2_m_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_m_p1.a, prt1_m_p1.c, prt2_m_p1.b, prt2_m_p1.c
+               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 public.prt1_m_p1
+                     Output: prt1_m_p1.a, prt1_m_p1.c, prt1_m_p1.b
+                     Filter: ((prt1_m_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_m_p1.b, prt2_m_p1.c, prt2_m_p1.a
+                     ->  Seq Scan on public.prt2_m_p1
+                           Output: prt2_m_p1.b, prt2_m_p1.c, prt2_m_p1.a
+                           Filter: ((prt2_m_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_m_p2.a, prt1_m_p2.c, prt2_m_p2.b, prt2_m_p2.c
+               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 public.prt1_m_p2
+                     Output: prt1_m_p2.a, prt1_m_p2.c, prt1_m_p2.b
+                     Filter: ((prt1_m_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_m_p2.b, prt2_m_p2.c, prt2_m_p2.a
+                     ->  Seq Scan on public.prt2_m_p2
+                           Output: prt2_m_p2.b, prt2_m_p2.c, prt2_m_p2.a
+                           Filter: ((prt2_m_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_m_p3.a, prt1_m_p3.c, prt2_m_p3.b, prt2_m_p3.c
+               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 public.prt1_m_p3
+                     Output: prt1_m_p3.a, prt1_m_p3.c, prt1_m_p3.b
+                     Filter: ((prt1_m_p3.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_m_p3.b, prt2_m_p3.c, prt2_m_p3.a
+                     ->  Seq Scan on public.prt2_m_p3
+                           Output: prt2_m_p3.b, prt2_m_p3.c, prt2_m_p3.a
+                           Filter: ((prt2_m_p3.b % 25) = 0)
+(37 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 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 | 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_m t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_m t2 WHERE t2.b % 25 = 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 | 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)
+
+--
+-- 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;
+ANALYZE plt1_p1;
+ANALYZE plt1_p2;
+ANALYZE plt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1 AS SELECT * FROM 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;
+ANALYZE plt2_p1;
+ANALYZE plt2_p2;
+ANALYZE plt2_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt2 AS SELECT * FROM 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;
+ANALYZE plt1_e_p1;
+ANALYZE plt1_e_p2;
+ANALYZE plt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1_e AS SELECT * FROM plt1_e;
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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
+   Output: (avg(t1.a)), (avg(t2.b)), (avg((t3.a + t3.b))), t1.c, t2.c, t3.c
+   Sort Key: t1.c, t3.c
+   ->  HashAggregate
+         Output: avg(t1.a), avg(t2.b), avg((t3.a + t3.b)), t1.c, t2.c, t3.c
+         Group Key: t1.c, t2.c, t3.c
+         ->  Result
+               Output: t1.c, t2.c, t3.c, t1.a, t2.b, t3.a, t3.b
+               ->  Append
+                     ->  Hash Join
+                           Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                           Hash Cond: (t1.c = t2.c)
+                           ->  Seq Scan on public.plt1_p1 t1
+                                 Output: t1.a, t1.c
+                           ->  Hash
+                                 Output: t2.b, t2.c, t3.a, t3.b, t3.c
+                                 ->  Hash Join
+                                       Output: t2.b, t2.c, t3.a, t3.b, t3.c
+                                       Hash Cond: (t2.c = ltrim(t3.c, 'A'::text))
+                                       ->  Seq Scan on public.plt2_p1 t2
+                                             Output: t2.b, t2.c
+                                       ->  Hash
+                                             Output: t3.a, t3.b, t3.c
+                                             ->  Seq Scan on public.plt1_e_p1 t3
+                                                   Output: t3.a, t3.b, t3.c
+                     ->  Hash Join
+                           Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                           Hash Cond: (t1_1.c = t2_1.c)
+                           ->  Seq Scan on public.plt1_p2 t1_1
+                                 Output: t1_1.a, t1_1.c
+                           ->  Hash
+                                 Output: t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                                 ->  Hash Join
+                                       Output: t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                                       Hash Cond: (t2_1.c = ltrim(t3_1.c, 'A'::text))
+                                       ->  Seq Scan on public.plt2_p2 t2_1
+                                             Output: t2_1.b, t2_1.c
+                                       ->  Hash
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             ->  Seq Scan on public.plt1_e_p2 t3_1
+                                                   Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash Join
+                           Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                           Hash Cond: (t1_2.c = t2_2.c)
+                           ->  Seq Scan on public.plt1_p3 t1_2
+                                 Output: t1_2.a, t1_2.c
+                           ->  Hash
+                                 Output: t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                                 ->  Hash Join
+                                       Output: t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                                       Hash Cond: (t2_2.c = ltrim(t3_2.c, 'A'::text))
+                                       ->  Seq Scan on public.plt2_p3 t2_2
+                                             Output: t2_2.b, t2_2.c
+                                       ->  Hash
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             ->  Seq Scan on public.plt1_e_p3 t3_2
+                                                   Output: t3_2.a, t3_2.b, t3_2.c
+(57 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)
+
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM uplt1 t1, uplt2 t2, uplt1_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)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: ((t3.a = t1.a) AND (ltrim(t3.c, 'A'::text) = t1.c))
+                     ->  Seq Scan on public.plt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                                 ->  Seq Scan on public.plt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.plt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: ((t3_1.a = t1_1.a) AND (ltrim(t3_1.c, 'A'::text) = t1_1.c))
+                     ->  Seq Scan on public.plt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.c = t1_1.c))
+                                 ->  Seq Scan on public.plt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.plt1_p2 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: ((t3_2.a = t1_2.a) AND (ltrim(t3_2.c, 'A'::text) = t1_2.c))
+                     ->  Seq Scan on public.plt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                                 ->  Seq Scan on public.plt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.plt1_p3 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: ((t3.a = t2.b) AND (ltrim(t3.c, 'A'::text) = t2.c))
+                     ->  Seq Scan on public.plt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                                 ->  Seq Scan on public.plt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.plt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: ((t3_1.a = t2_1.b) AND (ltrim(t3_1.c, 'A'::text) = t2_1.c))
+                     ->  Seq Scan on public.plt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.c = t1_1.c))
+                                 ->  Seq Scan on public.plt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.plt1_p2 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: ((t3_2.a = t2_2.b) AND (ltrim(t3_2.c, 'A'::text) = t2_2.c))
+                     ->  Seq Scan on public.plt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                                 ->  Seq Scan on public.plt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.plt1_p3 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |          | 
+ 100 | 0002 |     |      |          | 
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |          | 
+ 250 | 0005 |     |      |          | 
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |          | 
+ 400 | 0008 |     |      |          | 
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |          | 
+ 550 | 0011 |     |      |          | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |          | 
+ 100 | 0002 |     |      |          | 
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |          | 
+ 250 | 0005 |     |      |          | 
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |          | 
+ 400 | 0008 |     |      |          | 
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |          | 
+ 550 | 0011 |     |      |          | 
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t1.a, t1.c, t2.b, t2.c
+                     Hash Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                     ->  Seq Scan on public.plt2_p1 t2
+                           Output: t2.b, t2.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t1.a, t1.c
+                                 Hash Cond: ((t1.c = ltrim(t3.c, 'A'::text)) AND (t1.a = t3.a))
+                                 ->  Seq Scan on public.plt1_p1 t1
+                                       Output: t1.a, t1.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.plt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                     Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.c = t1_1.c))
+                     ->  Seq Scan on public.plt2_p2 t2_1
+                           Output: t2_1.b, t2_1.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t1_1.a, t1_1.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t1_1.a, t1_1.c
+                                 Hash Cond: ((t1_1.c = ltrim(t3_1.c, 'A'::text)) AND (t1_1.a = t3_1.a))
+                                 ->  Seq Scan on public.plt1_p2 t1_1
+                                       Output: t1_1.a, t1_1.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.plt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                     Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                     ->  Seq Scan on public.plt2_p3 t2_2
+                           Output: t2_2.b, t2_2.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t1_2.a, t1_2.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t1_2.a, t1_2.c
+                                 Hash Cond: ((t1_2.c = ltrim(t3_2.c, 'A'::text)) AND (t1_2.a = t3_2.a))
+                                 ->  Seq Scan on public.plt1_p3 t1_2
+                                       Output: t1_2.a, t1_2.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.plt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: ((t1.a = t2.b) AND (t1.c = t2.c))
+                     ->  Seq Scan on public.plt1_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Hash Cond: ((t2.b = t3.a) AND (t2.c = ltrim(t3.c, 'A'::text)))
+                                 ->  Seq Scan on public.plt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t3.a, t3.b, t3.c
+                                       ->  Seq Scan on public.plt1_e_p1 t3
+                                             Output: t3.a, t3.b, t3.c
+                                             Filter: ((t3.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: ((t1_1.a = t2_1.b) AND (t1_1.c = t2_1.c))
+                     ->  Seq Scan on public.plt1_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: ((t2_1.b = t3_1.a) AND (t2_1.c = ltrim(t3_1.c, 'A'::text)))
+                                 ->  Seq Scan on public.plt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       ->  Seq Scan on public.plt1_e_p2 t3_1
+                                             Output: t3_1.a, t3_1.b, t3_1.c
+                                             Filter: ((t3_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: ((t1_2.a = t2_2.b) AND (t1_2.c = t2_2.c))
+                     ->  Seq Scan on public.plt1_p3 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: ((t2_2.b = t3_2.a) AND (t2_2.c = ltrim(t3_2.c, 'A'::text)))
+                                 ->  Seq Scan on public.plt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t3_2.a, t3_2.b, t3_2.c
+                                       ->  Seq Scan on public.plt1_e_p3 t3_2
+                                             Output: t3_2.a, t3_2.b, t3_2.c
+                                             Filter: ((t3_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+     |      |     |      |      100 | A0001
+     |      |     |      |      200 | A0002
+     |      |     |      |      400 | A0004
+     |      |     |      |      500 | A0005
+     |      |     |      |      700 | A0007
+     |      |     |      |      800 | A0008
+     |      |     |      |     1000 | A0010
+     |      |     |      |     1100 | A0011
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+     |      |     |      |      100 | A0001
+     |      |     |      |      200 | A0002
+     |      |     |      |      400 | A0004
+     |      |     |      |      500 | A0005
+     |      |     |      |      700 | A0007
+     |      |     |      |      800 | A0008
+     |      |     |      |     1000 | A0010
+     |      |     |      |     1100 | A0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                                  QUERY PLAN                                                   
+---------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, ((plt1_e_p1.a + plt1_e_p1.b)), plt1_e_p1.c
+   Sort Key: plt1_p1.a, plt2_p1.b, ((plt1_e_p1.a + plt1_e_p1.b))
+   ->  Result
+         Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, (plt1_e_p1.a + plt1_e_p1.b), plt1_e_p1.c
+         ->  Append
+               ->  Hash Full Join
+                     Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                     Hash Cond: ((plt1_p1.a = plt1_e_p1.a) AND (plt1_p1.c = ltrim(plt1_e_p1.c, 'A'::text)))
+                     ->  Hash Full Join
+                           Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+                           Hash Cond: ((plt1_p1.a = plt2_p1.b) AND (plt1_p1.c = plt2_p1.c))
+                           ->  Seq Scan on public.plt1_p1
+                                 Output: plt1_p1.a, plt1_p1.c
+                                 Filter: ((plt1_p1.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p1.b, plt2_p1.c
+                                 ->  Seq Scan on public.plt2_p1
+                                       Output: plt2_p1.b, plt2_p1.c
+                                       Filter: ((plt2_p1.b % 25) = 0)
+                     ->  Hash
+                           Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                           ->  Seq Scan on public.plt1_e_p1
+                                 Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                                 Filter: ((plt1_e_p1.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c, plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                     Hash Cond: ((plt1_p2.a = plt1_e_p2.a) AND (plt1_p2.c = ltrim(plt1_e_p2.c, 'A'::text)))
+                     ->  Hash Full Join
+                           Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+                           Hash Cond: ((plt1_p2.a = plt2_p2.b) AND (plt1_p2.c = plt2_p2.c))
+                           ->  Seq Scan on public.plt1_p2
+                                 Output: plt1_p2.a, plt1_p2.c
+                                 Filter: ((plt1_p2.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p2.b, plt2_p2.c
+                                 ->  Seq Scan on public.plt2_p2
+                                       Output: plt2_p2.b, plt2_p2.c
+                                       Filter: ((plt2_p2.b % 25) = 0)
+                     ->  Hash
+                           Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                           ->  Seq Scan on public.plt1_e_p2
+                                 Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                                 Filter: ((plt1_e_p2.a % 25) = 0)
+               ->  Hash Full Join
+                     Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c, plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                     Hash Cond: ((plt1_p3.a = plt1_e_p3.a) AND (plt1_p3.c = ltrim(plt1_e_p3.c, 'A'::text)))
+                     ->  Hash Full Join
+                           Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+                           Hash Cond: ((plt1_p3.a = plt2_p3.b) AND (plt1_p3.c = plt2_p3.c))
+                           ->  Seq Scan on public.plt1_p3
+                                 Output: plt1_p3.a, plt1_p3.c
+                                 Filter: ((plt1_p3.a % 25) = 0)
+                           ->  Hash
+                                 Output: plt2_p3.b, plt2_p3.c
+                                 ->  Seq Scan on public.plt2_p3
+                                       Output: plt2_p3.b, plt2_p3.c
+                                       Filter: ((plt2_p3.b % 25) = 0)
+                     ->  Hash
+                           Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                           ->  Seq Scan on public.plt1_e_p3
+                                 Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                                 Filter: ((plt1_e_p3.a % 25) = 0)
+(63 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.c = t1_3.c)
+               ->  Seq Scan on public.plt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_3.c, t2.c
+                     Hash Cond: (t1_3.c = ltrim(t2.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p1 t1_3
+                           Output: t1_3.c
+                     ->  Hash
+                           Output: t2.c
+                           ->  Seq Scan on public.plt1_e_p1 t2
+                                 Output: t2.c
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.c = t1_4.c)
+               ->  Seq Scan on public.plt1_p2 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_4.c, t2_1.c
+                     Hash Cond: (t1_4.c = ltrim(t2_1.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p2 t1_4
+                           Output: t1_4.c
+                     ->  Hash
+                           Output: t2_1.c
+                           ->  Seq Scan on public.plt1_e_p2 t2_1
+                                 Output: t2_1.c
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.c = t1_5.c)
+               ->  Seq Scan on public.plt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_5.c, t2_2.c
+                     Hash Cond: (t1_5.c = ltrim(t2_2.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p3 t1_5
+                           Output: t1_5.c
+                     ->  Hash
+                           Output: t2_2.c
+                           ->  Seq Scan on public.plt1_e_p3 t2_2
+                                 Output: t2_2.c
+(49 rows)
+
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1, uplt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+                                   QUERY PLAN                                   
+--------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Nested Loop Semi Join
+               Output: t1.a, t1.b, t1.c
+               Join Filter: (t1.c = t1_3.c)
+               ->  Seq Scan on public.plt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_3.c, t1_6.c
+                     Hash Cond: (t1_3.c = ltrim(t1_6.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p1 t1_3
+                           Output: t1_3.c
+                     ->  Hash
+                           Output: t1_6.c
+                           ->  HashAggregate
+                                 Output: t1_6.c
+                                 Group Key: ltrim(t1_6.c, 'A'::text)
+                                 ->  Seq Scan on public.plt1_e_p1 t1_6
+                                       Output: t1_6.c, ltrim(t1_6.c, 'A'::text)
+                                       Filter: ((t1_6.a % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Join Filter: (t1_1.c = t1_4.c)
+               ->  Seq Scan on public.plt1_p2 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_4.c, t1_7.c
+                     Hash Cond: (t1_4.c = ltrim(t1_7.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p2 t1_4
+                           Output: t1_4.c
+                     ->  Hash
+                           Output: t1_7.c
+                           ->  HashAggregate
+                                 Output: t1_7.c
+                                 Group Key: ltrim(t1_7.c, 'A'::text)
+                                 ->  Seq Scan on public.plt1_e_p2 t1_7
+                                       Output: t1_7.c, ltrim(t1_7.c, 'A'::text)
+                                       Filter: ((t1_7.a % 25) = 0)
+         ->  Nested Loop Semi Join
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Join Filter: (t1_2.c = t1_5.c)
+               ->  Seq Scan on public.plt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Hash Join
+                     Output: t1_5.c, t1_8.c
+                     Hash Cond: (t1_5.c = ltrim(t1_8.c, 'A'::text))
+                     ->  Seq Scan on public.plt2_p3 t1_5
+                           Output: t1_5.c
+                     ->  Hash
+                           Output: t1_8.c
+                           ->  HashAggregate
+                                 Output: t1_8.c
+                                 Group Key: ltrim(t1_8.c, 'A'::text)
+                                 ->  Seq Scan on public.plt1_e_p3 t1_8
+                                       Output: t1_8.c, ltrim(t1_8.c, 'A'::text)
+                                       Filter: ((t1_8.a % 25) = 0)
+(61 rows)
+
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+  50 |  50 | 0001
+ 100 | 100 | 0002
+ 150 | 150 | 0003
+ 200 | 200 | 0004
+ 250 | 250 | 0005
+ 300 | 300 | 0006
+ 350 | 350 | 0007
+ 400 | 400 | 0008
+ 450 | 450 | 0009
+ 500 | 500 | 0010
+ 550 | 550 | 0011
+(12 rows)
+
+--
+-- negative testcases
+--
+-- joins where one of the relations is proven empty
+EXPLAIN (VERBOSE, 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;
+            QUERY PLAN            
+----------------------------------
+ Result
+   Output: t1.a, t1.c, t2.b, t2.c
+   One-Time Filter: false
+(3 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 LEFT JOIN prt2 t2 ON t1.a = t2.b;
+              QUERY PLAN              
+--------------------------------------
+ Result
+   Output: prt1.a, prt1.c, t2.b, t2.c
+   One-Time Filter: false
+(3 rows)
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                     
+---------------------------------------------------
+ Sort
+   Output: a, c, t2.b, t2.c
+   Sort Key: a, t2.b
+   ->  Hash Left Join
+         Output: a, c, t2.b, t2.c
+         Hash Cond: (t2.b = a)
+         ->  Append
+               ->  Seq Scan on public.prt2 t2
+                     Output: t2.b, t2.c
+                     Filter: ((t2.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p1 t2_1
+                     Output: t2_1.b, t2_1.c
+                     Filter: ((t2_1.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p2 t2_2
+                     Output: t2_2.b, t2_2.c
+                     Filter: ((t2_2.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p3 t2_3
+                     Output: t2_3.b, t2_3.c
+                     Filter: ((t2_3.a % 25) = 0)
+         ->  Hash
+               Output: a, c
+               ->  Result
+                     Output: a, c
+                     One-Time Filter: false
+(24 rows)
+
+EXPLAIN (VERBOSE, 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 % 25 = 0 ORDER BY t1.a, t2.b;
+                    QUERY PLAN                     
+---------------------------------------------------
+ Sort
+   Output: a, c, t2.b, t2.c
+   Sort Key: a, t2.b
+   ->  Hash Left Join
+         Output: a, c, t2.b, t2.c
+         Hash Cond: (t2.b = a)
+         ->  Append
+               ->  Seq Scan on public.prt2 t2
+                     Output: t2.b, t2.c
+                     Filter: ((t2.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p1 t2_1
+                     Output: t2_1.b, t2_1.c
+                     Filter: ((t2_1.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p2 t2_2
+                     Output: t2_2.b, t2_2.c
+                     Filter: ((t2_2.a % 25) = 0)
+               ->  Seq Scan on public.prt2_p3 t2_3
+                     Output: t2_3.b, t2_3.c
+                     Filter: ((t2_3.a % 25) = 0)
+         ->  Hash
+               Output: a, c
+               ->  Result
+                     Output: a, c
+                     One-Time Filter: false
+(24 rows)
+
+CREATE TABLE prt1_n (a int, b int, c varchar) PARTITION BY RANGE(c);
+CREATE TABLE prt1_n_p1 PARTITION OF prt1_n FOR VALUES FROM ('0000') TO ('0250');
+CREATE TABLE prt1_n_p2 PARTITION OF prt1_n FOR VALUES FROM ('0250') TO ('0500');
+INSERT INTO prt1_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 499, 2) i;
+ANALYZE prt1_n;
+ANALYZE prt1_n_p1;
+ANALYZE prt1_n_p2;
+CREATE TABLE prt2_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt2_n_p1 PARTITION OF prt2_n FOR VALUES IN ('0000', '0003', '0004', '0010', '0006', '0007');
+CREATE TABLE prt2_n_p2 PARTITION OF prt2_n FOR VALUES IN ('0001', '0005', '0002', '0009', '0008', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt2_n;
+ANALYZE prt2_n_p1;
+ANALYZE prt2_n_p2;
+CREATE TABLE prt3_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt3_n_p1 PARTITION OF prt3_n FOR VALUES IN ('0000', '0004', '0006', '0007');
+CREATE TABLE prt3_n_p2 PARTITION OF prt3_n FOR VALUES IN ('0001', '0002', '0008', '0010');
+CREATE TABLE prt3_n_p3 PARTITION OF prt3_n FOR VALUES IN ('0003', '0005', '0009', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt3_n;
+ANALYZE prt3_n_p1;
+ANALYZE prt3_n_p2;
+ANALYZE prt3_n_p3;
+CREATE TABLE prt4_n (a int, b int, c text) PARTITION BY RANGE(a);
+CREATE TABLE prt4_n_p1 PARTITION OF prt4_n FOR VALUES FROM (0) TO (300);
+CREATE TABLE prt4_n_p2 PARTITION OF prt4_n FOR VALUES FROM (300) TO (500);
+CREATE TABLE prt4_n_p3 PARTITION OF prt4_n FOR VALUES FROM (500) TO (600);
+INSERT INTO prt4_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt4_n;
+ANALYZE prt4_n_p1;
+ANALYZE prt4_n_p2;
+ANALYZE prt4_n_p3;
+-- partition-wise join can not be applied if the partition ranges differ
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Join
+   Hash Cond: (t1.a = t2.a)
+   ->  Append
+         ->  Seq Scan on prt1 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p2 t1_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt4_n t2
+               ->  Seq Scan on prt4_n_p1 t2_1
+               ->  Seq Scan on prt4_n_p2 t2_2
+               ->  Seq Scan on prt4_n_p3 t2_3
+(13 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 FULL JOIN prt4_n t2 ON t1.a = t2.a;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Full Join
+   Hash Cond: (t1.a = t2.a)
+   ->  Append
+         ->  Seq Scan on prt1 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p2 t1_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt4_n t2
+               ->  Seq Scan on prt4_n_p1 t2_1
+               ->  Seq Scan on prt4_n_p2 t2_2
+               ->  Seq Scan on prt4_n_p3 t2_3
+(13 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                        
+---------------------------------------------------------
+ Nested Loop Left Join
+   ->  Append
+         ->  Seq Scan on prt1 t1
+         ->  Seq Scan on prt1_p1 t1_1
+         ->  Seq Scan on prt1_p3 t1_2
+         ->  Seq Scan on prt1_p2 t1_3
+   ->  Append
+         ->  Seq Scan on prt2 t2
+               Filter: (t1.a < b)
+         ->  Index Scan using iprt2_p1_b on prt2_p1 t2_1
+               Index Cond: (t1.a < b)
+         ->  Index Scan using iprt2_p2_b on prt2_p2 t2_2
+               Index Cond: (t1.a < b)
+         ->  Index Scan using iprt2_p3_b on prt2_p3 t2_3
+               Index Cond: (t1.a < b)
+(15 rows)
+
+-- equi-join with join condition on partial keys does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1, prt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t1.a % 25 = 0;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Join
+   Hash Cond: (((t2.b + t2.a) / 2) = t1.a)
+   ->  Append
+         ->  Seq Scan on prt2_m t2
+         ->  Seq Scan on prt2_m_p1 t2_1
+         ->  Seq Scan on prt2_m_p2 t2_2
+         ->  Seq Scan on prt2_m_p3 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_m t1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p1 t1_1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p2 t1_2
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p3 t1_3
+                     Filter: ((a % 25) = 0)
+(17 rows)
+
+-- equi-join between out-of-order partition key columns does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.a = t2.b WHERE t1.a % 25 = 0;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Right Join
+   Hash Cond: (t2.b = t1.a)
+   ->  Append
+         ->  Seq Scan on prt2_m t2
+         ->  Seq Scan on prt2_m_p1 t2_1
+         ->  Seq Scan on prt2_m_p2 t2_2
+         ->  Seq Scan on prt2_m_p3 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_m t1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p1 t1_1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p2 t1_2
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p3 t1_3
+                     Filter: ((a % 25) = 0)
+(17 rows)
+
+-- equi-join between non-key columns does not qualify for partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.c = t2.c WHERE t1.a % 25 = 0;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Right Join
+   Hash Cond: ((t2.c)::text = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt2_m t2
+         ->  Seq Scan on prt2_m_p1 t2_1
+         ->  Seq Scan on prt2_m_p2 t2_2
+         ->  Seq Scan on prt2_m_p3 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_m t1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p1 t1_1
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p2 t1_2
+                     Filter: ((a % 25) = 0)
+               ->  Seq Scan on prt1_m_p3 t1_3
+                     Filter: ((a % 25) = 0)
+(17 rows)
+
+-- partition-wise join can not be applied for a join between list and range
+-- partitioned table
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1, prt2_n t2 WHERE t1.c = t2.c;
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Join
+   Hash Cond: (t2.c = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt2_n t2
+         ->  Seq Scan on prt2_n_p1 t2_1
+         ->  Seq Scan on prt2_n_p2 t2_2
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(11 rows)
+
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 LEFT JOIN prt2_n t2 ON (t1.c = t2.c);
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Right Join
+   Hash Cond: (t2.c = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt2_n t2
+         ->  Seq Scan on prt2_n_p1 t2_1
+         ->  Seq Scan on prt2_n_p2 t2_2
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(11 rows)
+
+-- partition-wise join can not be applied between tables with different
+-- partition lists
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 RIGHT JOIN prt1 t2 ON (t1.c = t2.c);
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Left Join
+   Hash Cond: ((t2.c)::text = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt1 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p2 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(12 rows)
+
+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);
+                  QUERY PLAN                  
+----------------------------------------------
+ Hash Full Join
+   Hash Cond: ((t2.c)::text = (t1.c)::text)
+   ->  Append
+         ->  Seq Scan on prt1 t2
+         ->  Seq Scan on prt1_p1 t2_1
+         ->  Seq Scan on prt1_p3 t2_2
+         ->  Seq Scan on prt1_p2 t2_3
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_n t1
+               ->  Seq Scan on prt1_n_p1 t1_1
+               ->  Seq Scan on prt1_n_p2 t1_2
+(12 rows)
+
diff --git a/src/test/regress/parallel_schedule b/src/test/regress/parallel_schedule
index edeb2d6..ac38f50 100644
--- a/src/test/regress/parallel_schedule
+++ b/src/test/regress/parallel_schedule
@@ -103,7 +103,7 @@ test: publication subscription
 # ----------
 # Another group of parallel tests
 # ----------
-test: select_views portals_p2 foreign_key cluster dependency guc bitmapops combocid tsearch tsdicts foreign_data window xmlmap functional_deps advisory_lock json jsonb json_encoding indirect_toast equivclass
+test: select_views portals_p2 foreign_key cluster dependency guc bitmapops combocid tsearch tsdicts foreign_data window xmlmap functional_deps advisory_lock json jsonb json_encoding indirect_toast equivclass partition_join
 # ----------
 # Another group of parallel tests
 # NB: temp.sql does a reconnect which transiently uses 2 connections,
diff --git a/src/test/regress/serial_schedule b/src/test/regress/serial_schedule
index 27a46d7..1bf98a4 100644
--- a/src/test/regress/serial_schedule
+++ b/src/test/regress/serial_schedule
@@ -172,3 +172,4 @@ test: with
 test: xml
 test: event_trigger
 test: stats
+test: partition_join
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
new file mode 100644
index 0000000..0322f1e
--- /dev/null
+++ b/src/test/regress/sql/partition_join.sql
@@ -0,0 +1,515 @@
+--
+-- PARTITION_JOIN
+-- Test partition-wise join between partitioned tables
+--
+
+--
+-- partitioned by a single column
+--
+CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+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, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1 AS SELECT * FROM prt1;
+
+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_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, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+CREATE TABLE uprt2 AS SELECT * FROM prt2;
+
+-- inner join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+-- left outer join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 LEFT JOIN prt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+-- right outer join
+EXPLAIN (VERBOSE, 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.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+-- full outer join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.b OR t2.phv = t2.b ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t1.phv, t2.b, t2.c, t2.phv FROM (SELECT 25 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 50 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b) ORDER BY t1.a, t2.b;
+
+-- Join with pruned partitions from joining relations
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1 t1, uprt2 t2 WHERE t1.a = t2.b AND t1.a < 450 AND t2.b > 250 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 LEFT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM prt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450) t1 RIGHT JOIN (SELECT * FROM uprt2 WHERE b > 250) t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1 WHERE a < 450 AND a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE b > 250 AND b % 25 = 0) t2 ON t1.a = t2.b ORDER BY t1.a, t2.b;
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b % 25 = 0) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+-- lateral reference
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+--
+-- partitioned by expression
+--
+CREATE TABLE prt1_e (a int, b int, c varchar) PARTITION BY RANGE(((a + b)/2));
+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);
+INSERT INTO prt1_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_e AS SELECT * FROM prt1_e;
+
+CREATE TABLE prt2_e (a int, b int, c varchar) PARTITION BY RANGE(((b + a)/2));
+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);
+INSERT INTO prt2_e SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_e;
+ANALYZE prt2_e_p1;
+ANALYZE prt2_e_p2;
+ANALYZE prt2_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_e AS SELECT * FROM prt2_e;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1, uprt2_e t2 WHERE (t1.a + t1.b)/2 = (t2.b + t2.a)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_e t1 LEFT JOIN prt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_e t1 LEFT JOIN uprt2_e t2 ON (t1.a + t1.b)/2 = (t2.b + t2.a)/2 WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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.a % 25 = 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM uprt1 t1, uprt2 t2, uprt1_e t3 WHERE t1.a = t2.b AND t1.a = (t3.a + t3.b)/2 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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 (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) LEFT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM prt1_e WHERE prt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT * FROM uprt1_e WHERE uprt1_e.a % 25 = 0) t3 ON (t1.a = (t3.a + t3.b)/2) ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+EXPLAIN (VERBOSE, 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.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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;
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM prt1 WHERE prt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM prt2 WHERE prt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM prt1_e WHERE prt1_e.a % 25 = 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;
+SELECT t1.a, t1.phv, t2.b, t2.phv, t3.a + t3.b, t3.phv FROM ((SELECT 50 phv, * FROM uprt1 WHERE uprt1.a % 25 = 0) t1 FULL JOIN (SELECT 75 phv, * FROM uprt2 WHERE uprt2.b % 25 = 0) t2 ON (t1.a = t2.b)) FULL JOIN (SELECT 50 phv, * FROM uprt1_e WHERE uprt1_e.a % 25 = 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;
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1, prt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1, uprt1_e t2 WHERE t1.b % 25 = 0 AND t1.b = (t2.a + t2.b)/2) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+-- test merge joins with and without using indexes
+SET enable_hashjoin TO off;
+SET enable_nestloop TO off;
+
+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 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 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));
+
+ANALYZE prt1;
+ANALYZE prt1_p1;
+ANALYZE prt1_p2;
+ANALYZE prt1_p3;
+ANALYZE prt2;
+ANALYZE prt2_p1;
+ANALYZE prt2_p2;
+ANALYZE prt2_p3;
+ANALYZE prt1_e;
+ANALYZE prt1_e_p1;
+ANALYZE prt1_e_p2;
+ANALYZE prt1_e_p3;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+SET enable_seqscan TO off;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 RIGHT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uprt1 t1 WHERE t1.a IN (SELECT t1.b FROM uprt2 t1 WHERE t1.b IN (SELECT (t1.a + t1.b)/2 FROM uprt1_e t1 WHERE t1.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+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.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uprt1 t1 LEFT JOIN uprt2 t2 ON t1.a = t2.b) RIGHT JOIN uprt1_e t3 ON (t1.a = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+-- lateral references and parameterized paths
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1 t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1 t2 JOIN uprt2 t3 ON (t2.a = t3.b)) ss
+			  ON t1.b = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+
+RESET enable_hashjoin;
+RESET enable_nestloop;
+RESET enable_seqscan;
+
+--
+-- partitioned by multiple columns
+--
+CREATE TABLE prt1_m (a int, b int, c varchar) 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, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_m;
+ANALYZE prt1_m_p1;
+ANALYZE prt1_m_p2;
+ANALYZE prt1_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_m AS SELECT * FROM prt1_m;
+
+CREATE TABLE prt2_m (a int, b int, c varchar) 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, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_m;
+ANALYZE prt2_m_p1;
+ANALYZE prt2_m_p2;
+ANALYZE prt2_m_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_m AS SELECT * FROM prt2_m;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 RIGHT JOIN prt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_m t1 RIGHT JOIN uprt2_m t2 ON t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2 WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_m WHERE prt1_m.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_m WHERE prt2_m.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_m t1 WHERE t1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_m t2 WHERE t2.b % 25 = 0) t2 ON (t1.a = (t2.b + t2.a)/2 AND t2.b = (t1.a + t1.b)/2) ORDER BY t1.a, t2.b;
+
+--
+-- 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;
+ANALYZE plt1_p1;
+ANALYZE plt1_p2;
+ANALYZE plt1_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1 AS SELECT * FROM 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;
+ANALYZE plt2_p1;
+ANALYZE plt2_p2;
+ANALYZE plt2_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt2 AS SELECT * FROM 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;
+ANALYZE plt1_e_p1;
+ANALYZE plt1_e_p2;
+ANALYZE plt1_e_p3;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uplt1_e AS SELECT * FROM plt1_e;
+
+--
+-- N-way join
+--
+EXPLAIN (VERBOSE, 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;
+SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM uplt1 t1, uplt2 t2, uplt1_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;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 LEFT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 RIGHT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (uplt1 t1 RIGHT JOIN uplt2 t2 ON t1.a = t2.b AND t1.c = t2.c) RIGHT JOIN uplt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM uplt1 WHERE uplt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uplt2 WHERE uplt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM uplt1_e WHERE uplt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+-- Semi-join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1, plt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1, uplt1_e t2 WHERE t1.c = ltrim(t2.c, 'A')) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.* FROM plt1 t1 WHERE t1.c IN (SELECT t1.c FROM plt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 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 ltrim(t1.c, 'A') FROM plt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+SELECT t1.* FROM uplt1 t1 WHERE t1.c IN (SELECT t1.c FROM uplt2 t1 WHERE t1.c IN (SELECT ltrim(t1.c, 'A') FROM uplt1_e t1 WHERE t1.a % 25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+--
+-- negative testcases
+--
+
+-- joins where one of the relations is proven empty
+EXPLAIN (VERBOSE, 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;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 LEFT JOIN prt2 t2 ON t1.a = t2.b;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b WHERE t2.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+EXPLAIN (VERBOSE, 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 % 25 = 0 ORDER BY t1.a, t2.b;
+
+CREATE TABLE prt1_n (a int, b int, c varchar) PARTITION BY RANGE(c);
+CREATE TABLE prt1_n_p1 PARTITION OF prt1_n FOR VALUES FROM ('0000') TO ('0250');
+CREATE TABLE prt1_n_p2 PARTITION OF prt1_n FOR VALUES FROM ('0250') TO ('0500');
+INSERT INTO prt1_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 499, 2) i;
+ANALYZE prt1_n;
+ANALYZE prt1_n_p1;
+ANALYZE prt1_n_p2;
+
+CREATE TABLE prt2_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt2_n_p1 PARTITION OF prt2_n FOR VALUES IN ('0000', '0003', '0004', '0010', '0006', '0007');
+CREATE TABLE prt2_n_p2 PARTITION OF prt2_n FOR VALUES IN ('0001', '0005', '0002', '0009', '0008', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt2_n;
+ANALYZE prt2_n_p1;
+ANALYZE prt2_n_p2;
+
+CREATE TABLE prt3_n (a int, b int, c text) PARTITION BY LIST(c);
+CREATE TABLE prt3_n_p1 PARTITION OF prt3_n FOR VALUES IN ('0000', '0004', '0006', '0007');
+CREATE TABLE prt3_n_p2 PARTITION OF prt3_n FOR VALUES IN ('0001', '0002', '0008', '0010');
+CREATE TABLE prt3_n_p3 PARTITION OF prt3_n FOR VALUES IN ('0003', '0005', '0009', '0011');
+INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt3_n;
+ANALYZE prt3_n_p1;
+ANALYZE prt3_n_p2;
+ANALYZE prt3_n_p3;
+
+CREATE TABLE prt4_n (a int, b int, c text) PARTITION BY RANGE(a);
+CREATE TABLE prt4_n_p1 PARTITION OF prt4_n FOR VALUES FROM (0) TO (300);
+CREATE TABLE prt4_n_p2 PARTITION OF prt4_n FOR VALUES FROM (300) TO (500);
+CREATE TABLE prt4_n_p3 PARTITION OF prt4_n FOR VALUES FROM (500) TO (600);
+INSERT INTO prt4_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt4_n;
+ANALYZE prt4_n_p1;
+ANALYZE prt4_n_p2;
+ANALYZE prt4_n_p3;
+
+-- partition-wise join can not be applied if the partition ranges differ
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1 FULL JOIN prt4_n t2 ON t1.a = t2.a;
+
+-- 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);
+
+-- equi-join with join condition on partial keys does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1, prt2_m t2 WHERE t1.a = (t2.b + t2.a)/2 AND t1.a % 25 = 0;
+
+-- equi-join between out-of-order partition key columns does not qualify for
+-- partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.a = t2.b WHERE t1.a % 25 = 0;
+
+-- equi-join between non-key columns does not qualify for partition-wise join
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_m t1 LEFT JOIN prt2_m t2 ON t1.c = t2.c WHERE t1.a % 25 = 0;
+
+-- partition-wise join can not be applied for a join between list and range
+-- partitioned table
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1, prt2_n t2 WHERE t1.c = t2.c;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 LEFT JOIN prt2_n t2 ON (t1.c = t2.c);
+
+-- partition-wise join can not be applied between tables with different
+-- partition lists
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 RIGHT JOIN prt1 t2 ON (t1.c = t2.c);
+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);
-- 
1.7.9.5

0008-Partition-wise-join-implementation.patchapplication/octet-stream; name=0008-Partition-wise-join-implementation.patchDownload
From 65764b810ac0f972dbd9513ff84397c29fea58c5 Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Tue, 7 Feb 2017 16:04:03 +0530
Subject: [PATCH 08/14] Partition-wise join implementation.

Implement partition-wise join for join between single level partitioned tables.
The details of this technique can be found in optimizer/README, where most of
the implementation has been explained.

We obtain clauses applicable to a child-join by translating corresponding
clauses of the parent. Because child-join can be computed by different
combinations of joining child relations, a given clause is required to be
translated multiple times. In order to reduce the memory consumption, we keep a
repository of child-clauses derived from a parent clause and search in that
repository before translating.

Tests for semi-join, those forcing a merge join for child-join and those
testing lateral join will crash with this patch. The tests testing joins with
partition pruning will also fail. Those crashes and failures are because the
existing code does not expect a child-join to appear in certain cases.
Following patches will fix that code.
---
 src/backend/nodes/copyfuncs.c                |    9 +
 src/backend/optimizer/README                 |   53 ++++
 src/backend/optimizer/path/allpaths.c        |  321 +++++++++++++++++---
 src/backend/optimizer/path/costsize.c        |    3 +
 src/backend/optimizer/path/joinpath.c        |   21 +-
 src/backend/optimizer/path/joinrels.c        |  421 ++++++++++++++++++++++++++
 src/backend/optimizer/plan/createplan.c      |  228 +++++++++++++-
 src/backend/optimizer/prep/prepunion.c       |  160 ++++++++++
 src/backend/optimizer/util/pathnode.c        |  113 +++++++
 src/backend/optimizer/util/placeholder.c     |   55 ++++
 src/backend/optimizer/util/relnode.c         |  266 +++++++++++++++-
 src/backend/utils/misc/guc.c                 |   28 ++
 src/include/nodes/nodes.h                    |    1 +
 src/include/nodes/relation.h                 |   57 ++++
 src/include/optimizer/cost.h                 |    5 +
 src/include/optimizer/pathnode.h             |    6 +
 src/include/optimizer/paths.h                |    7 +
 src/include/optimizer/placeholder.h          |    2 +
 src/include/optimizer/prep.h                 |    8 +
 src/test/regress/expected/partition_join.out |    4 +
 src/test/regress/expected/sysviews.out       |   29 +-
 src/test/regress/sql/partition_join.sql      |    5 +
 22 files changed, 1728 insertions(+), 74 deletions(-)

diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 30d733e..72c021e 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -2070,6 +2070,15 @@ _copyRestrictInfo(const RestrictInfo *from)
 	COPY_SCALAR_FIELD(left_bucketsize);
 	COPY_SCALAR_FIELD(right_bucketsize);
 
+	/*
+	 * Do not copy parent_rinfo and child_rinfos because 1. they create a
+	 * circular dependency between child and parent RestrictInfo 2. dropping
+	 * those links just means that we loose some memory optimizations. 3. There
+	 * is a possibility that the child and parent RestrictInfots themselves may
+	 * have got copied and thus the old links may no longer be valid. The
+	 * caller may set up those links itself, if needed.
+	 */
+
 	return newnode;
 }
 
diff --git a/src/backend/optimizer/README b/src/backend/optimizer/README
index fc0fca4..7565ae4 100644
--- a/src/backend/optimizer/README
+++ b/src/backend/optimizer/README
@@ -1076,3 +1076,56 @@ be desirable to postpone the Gather stage until as near to the top of the
 plan as possible.  Expanding the range of cases in which more work can be
 pushed below the Gather (and costing them accurately) is likely to keep us
 busy for a long time to come.
+
+Partition-wise 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 for a given row in a given partition of a given
+partitioned table, its joining row, if exists, should exist only in the
+matching partition of the other partitioned table; no row from non-matching
+partitions in the other partitioned table can join with the given row from the
+first table. This condition allows the join between partitioned table 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
+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 partition tables
+into join between their partitions is called partition-wise join. We will use
+term "partitioned relation" for both partitioned table as well as join between
+partitioned tables which can use partition-wise join technique.
+
+Partitioning properties of a partitioned table are stored in
+PartitionSchemeData structure. Planner maintains a list of canonical partition
+schemes (distinct PartitionSchemeData objects) so that any two partitioned
+relations with same partitioning scheme share the same PartitionSchemeData
+object. This reduces memory consumed by PartitionSchemeData objects and makes
+it easy to compare the partition schemes of joining relations. RelOptInfos of
+partitioned relations hold partition key expressions and the RelOptInfos of
+the partition relations of that relation.
+
+Partition-wise joins are planned in two phases
+
+1. First phase creates the RelOptInfos for joins between matching partitions,
+henceforth referred to as child-joins. The number of paths created for a
+child-join i.e. join between partitions is same as the number of paths created
+for join between parents. That number grows exponentially with the number of
+base relations being joined. The time and memory consumed to create paths for
+each child-join will be proporional to the number of partitions. This will not
+scale well with thousands of partitions. Instead of that we estimate
+partition-wise join cost based on the costs of sampled child-joins. We choose
+child-joins with higher sizes to have realistic estimates. If the number of
+sampled child-joins is same as the number of live child-joins, we create append
+paths as we know costs of all required child-joins. Otherwise we create
+PartitionJoinPaths with cost estimates based on the costs of sampled
+child-joins. While creating append paths or PartitionJoin paths we create paths
+for all the different possible parameterizations and pathkeys available in the
+sampled child-joins.
+
+2. If PartitionJoinPath emerges as the best possible path, we create paths for
+each unsampled child-join. From every child-join we choose the cheapest path
+with same parameterization or pathkeys as the PartitionJoinPath. This path is
+converted into a plan and all the child-join plans are combined using an Append
+or MergeAppend plan as appropriate. We use a fresh memory context for planning
+each unsampled child-join, thus reducing memory consumption.
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 0eb56f3..cebf359 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -15,6 +15,7 @@
 
 #include "postgres.h"
 
+#include "miscadmin.h"
 #include <limits.h>
 #include <math.h>
 
@@ -93,11 +94,8 @@ static void set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 static void set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 						Index rti, RangeTblEntry *rte);
 static void generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
-						   List *live_childrels,
-						   List *all_child_pathkeys);
-static Path *get_cheapest_parameterized_child_path(PlannerInfo *root,
-									  RelOptInfo *rel,
-									  Relids required_outer);
+						   List *live_childrels, List *all_child_pathkeys,
+						   bool partition_join_path);
 static List *accumulate_append_subpath(List *subpaths, Path *path);
 static void set_subquery_pathlist(PlannerInfo *root, RelOptInfo *rel,
 					  Index rti, RangeTblEntry *rte);
@@ -128,8 +126,8 @@ static void recurse_push_qual(Node *setOp, Query *topquery,
 static void remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel);
 static int	compute_parallel_worker(RelOptInfo *rel, BlockNumber pages);
 static void add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
-									List *live_childrels);
-
+							   List *live_childrels, bool partition_join_path);
+static int compare_rel_size(const void *rel1_p, const void *rel2_p);
 
 /*
  * make_one_rel
@@ -891,6 +889,12 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		childrel = find_base_rel(root, childRTindex);
 		Assert(childrel->reloptkind == RELOPT_OTHER_MEMBER_REL);
 
+		/* Pass top parent's relids down the inheritance hierarchy. */
+		if (rel->top_parent_relids)
+			childrel->top_parent_relids = rel->top_parent_relids;
+		else
+			childrel->top_parent_relids = bms_copy(rel->relids);
+
 		/*
 		 * Two partitioned tables with the same partitioning scheme, have their
 		 * partition bounds arranged in the same order. The order of partition
@@ -900,10 +904,15 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		 * RelOptInfos. Arranging RelOptInfos of partitions in the same order
 		 * as their OIDs makes it easy to find the RelOptInfos of matching
 		 * partitions for partition-wise join.
+		 *
+		 * For a partitioned tables, individual partitions can participate in
+		 * the pair-wise joins. We need attr_needed data for building
+		 * child-join targetlists.
 		 */
 		if (rel->part_scheme)
 		{
 			int		cnt_parts;
+			AttrNumber		attno;
 
 			for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
 			{
@@ -913,6 +922,38 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 					rel->part_rels[cnt_parts] = childrel;
 				}
 			}
+
+			for (attno = rel->min_attr; attno <= rel->max_attr; attno++)
+			{
+				int	index = attno - rel->min_attr;
+				Relids	attr_needed = bms_copy(rel->attr_needed[index]);
+
+				/*
+				 * System attributes do not need translation. In such a case,
+				 * the attribute numbers of the parent and the child should
+				 * start from the same minimum attribute.
+				 */
+				if (attno <= 0)
+				{
+					Assert(rel->min_attr == childrel->min_attr);
+					childrel->attr_needed[index] = attr_needed;
+				}
+				else
+				{
+					Var *var = list_nth(appinfo->translated_vars,
+										attno - 1);
+					int child_index;
+
+					/*
+					 * Parent Var for a user defined attribute translates to
+					 * child Var.
+					 */
+					Assert(IsA(var, Var));
+
+					child_index = var->varattno - childrel->min_attr;
+					childrel->attr_needed[child_index] = attr_needed;
+				}
+			}
 		}
 
 		/*
@@ -1057,10 +1098,8 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		 * PlaceHolderVars.)  XXX we do not bother to update the cost or width
 		 * fields of childrel->reltarget; not clear if that would be useful.
 		 */
-		childrel->joininfo = (List *)
-			adjust_appendrel_attrs(root,
-								   (Node *) rel->joininfo,
-								   appinfo_list);
+		childrel->joininfo = build_child_clauses(root, rel->joininfo,
+												 appinfo_list);
 		childrel->reltarget->exprs = (List *)
 			adjust_appendrel_attrs(root,
 								   (Node *) rel->reltarget->exprs,
@@ -1079,14 +1118,6 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		childrel->has_eclass_joins = rel->has_eclass_joins;
 
 		/*
-		 * Note: we could compute appropriate attr_needed data for the child's
-		 * variables, by transforming the parent's attr_needed through the
-		 * translated_vars mapping.  However, currently there's no need
-		 * because attr_needed is only examined for base relations not
-		 * otherrels.  So we just leave the child's attr_needed empty.
-		 */
-
-		/*
 		 * If parallelism is allowable for this query in general, see whether
 		 * it's allowable for this childrel in particular.  But if we've
 		 * already decided the appendrel is not parallel-safe as a whole,
@@ -1269,10 +1300,9 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 	}
 
 	/* Add Append/MergeAppend paths to the "append" relation. */
-	add_paths_to_append_rel(root, rel, live_childrels);
+	add_paths_to_append_rel(root, rel, live_childrels, false);
 }
 
-
 /*
  * add_paths_to_append_rel
  *		Generate Append/MergeAppend paths for given "append" relation.
@@ -1282,20 +1312,44 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
  * an append path collecting one path from each non-dummy child with given
  * parameterization or ordering. Similarly it collects partial paths from
  * non-dummy children to create partial append paths.
+ *
+ * When called on partitioned join relation with partition_join_path = true, it
+ * adds PartitionJoinPath instead of Merge/Append path. This path is costed
+ * based on the costs of sampled child-join and is expanded later into
+ * Merge/Append plan.
  */
 static void
 add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
-						List *live_childrels)
+						List *live_childrels, bool partition_join_path)
 {
 	List	   *subpaths = NIL;
 	bool		subpaths_valid = true;
 	List	   *partial_subpaths = NIL;
-	bool		partial_subpaths_valid = true;
+	bool		partial_subpaths_valid;
 	List	   *all_child_pathkeys = NIL;
 	List	   *all_child_outers = NIL;
 	ListCell   *l;
 
 	/*
+	 * While creating PartitionJoinPath, we sample paths from only a few child
+	 * relations. Even if all of sampled children have partial paths, it's not
+	 * guaranteed that all the unsampled children will have partial paths.
+	 * Hence we do not create partial PartitionJoinPaths.
+	 */
+	partial_subpaths_valid = !partition_join_path ? true : false;
+
+	/* An append relation with all its children dummy is dummy. */
+	if (live_childrels == NIL)
+	{
+		/* Mark the relation as dummy, if not already done so. */
+		if (!IS_DUMMY_REL(rel))
+			set_dummy_rel_pathlist(rel);
+
+		/* No more paths need to be added. */
+		return;
+	}
+
+	/*
 	 * For every non-dummy child, remember the cheapest path.  Also, identify
 	 * all pathkeys (orderings) and parameterizations (required_outer sets)
 	 * available for the non-dummy member relations.
@@ -1394,7 +1448,17 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	 * if we have zero or one live subpath due to constraint exclusion.)
 	 */
 	if (subpaths_valid)
-		add_path(rel, (Path *) create_append_path(rel, subpaths, NULL, 0));
+	{
+		Path *path;
+
+		if (partition_join_path)
+			path = (Path *) create_partition_join_path(root, rel, subpaths,
+													   NULL, NIL);
+		else
+			path = (Path *) create_append_path(rel, subpaths, NULL, 0);
+
+		add_path(rel, path);
+	}
 
 	/*
 	 * Consider an append of partial unordered, unparameterized partial paths.
@@ -1405,6 +1469,8 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		ListCell   *lc;
 		int			parallel_workers = 0;
 
+		Assert(!partition_join_path);
+
 		/*
 		 * Decide on the number of workers to request for this append path.
 		 * For now, we just use the maximum value from among the members.  It
@@ -1431,7 +1497,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	 */
 	if (subpaths_valid)
 		generate_mergeappend_paths(root, rel, live_childrels,
-								   all_child_pathkeys);
+								   all_child_pathkeys, partition_join_path);
 
 	/*
 	 * Build Append paths for each parameterization seen among the child rels.
@@ -1472,8 +1538,18 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		}
 
 		if (subpaths_valid)
-			add_path(rel, (Path *)
-					 create_append_path(rel, subpaths, required_outer, 0));
+		{
+			Path *path;
+
+			if (partition_join_path)
+				path = (Path *) create_partition_join_path(root, rel, subpaths,
+														   required_outer, NIL);
+			else
+				path = (Path *) create_append_path(rel, subpaths,
+												   required_outer, 0);
+
+			add_path(rel, path);
+		}
 	}
 }
 
@@ -1499,11 +1575,16 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
  * parameterized mergejoin plans, it might be worth adding support for
  * parameterized MergeAppends to feed such joins.  (See notes in
  * optimizer/README for why that might not ever happen, though.)
+ *
+ * When called on partitioned join relation with partition_join_path = true, it
+ * adds PartitionJoinPath with pathkeys instead of MergeAppend path. This path
+ * is costed based on the costs of sampled child-join and is expanded later
+ * into MergeAppend plan.
  */
 static void
 generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
 						   List *live_childrels,
-						   List *all_child_pathkeys)
+						   List *all_child_pathkeys, bool partition_join_path)
 {
 	ListCell   *lcp;
 
@@ -1514,6 +1595,7 @@ generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
 		List	   *total_subpaths = NIL;
 		bool		startup_neq_total = false;
 		ListCell   *lcr;
+		Path	   *path;
 
 		/* Select the child paths for this ordering... */
 		foreach(lcr, live_childrels)
@@ -1560,18 +1642,29 @@ generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
 				accumulate_append_subpath(total_subpaths, cheapest_total);
 		}
 
-		/* ... and build the MergeAppend paths */
-		add_path(rel, (Path *) create_merge_append_path(root,
-														rel,
-														startup_subpaths,
-														pathkeys,
-														NULL));
+		/* ... and build the paths */
+		if (partition_join_path)
+			path = (Path *) create_partition_join_path(root, rel,
+													   startup_subpaths,
+													   NULL, pathkeys);
+		else
+			path = (Path *) create_merge_append_path(root, rel,
+													 startup_subpaths,
+													 pathkeys, NULL);
+		add_path(rel, path);
+
 		if (startup_neq_total)
-			add_path(rel, (Path *) create_merge_append_path(root,
-															rel,
-															total_subpaths,
-															pathkeys,
-															NULL));
+		{
+			if (partition_join_path)
+				path = (Path *) create_partition_join_path(root, rel,
+														   total_subpaths,
+														   NULL, pathkeys);
+			else
+				path = (Path *) create_merge_append_path(root, rel,
+														 total_subpaths,
+														 pathkeys, NULL);
+			add_path(rel, path);
+		}
 	}
 }
 
@@ -1582,7 +1675,7 @@ generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
  *
  * Returns NULL if unable to create such a path.
  */
-static Path *
+Path *
 get_cheapest_parameterized_child_path(PlannerInfo *root, RelOptInfo *rel,
 									  Relids required_outer)
 {
@@ -2316,8 +2409,17 @@ standard_join_search(PlannerInfo *root, int levels_needed, List *initial_rels)
 		 * Run generate_gather_paths() for each just-processed joinrel.  We
 		 * could not do this earlier because both regular and partial paths
 		 * can get added to a particular joinrel at multiple times within
-		 * join_search_one_level.  After that, we're done creating paths for
-		 * the joinrel, so run set_cheapest().
+		 * join_search_one_level.
+		 *
+		 * Similarly, create paths for joinrels which used partition-wise join
+		 * technique. generate_partition_wise_join_paths() creates paths for
+		 * only few of the child-joins with highest sizes. Though we calculate
+		 * size of a child-join only once; when it gets created, it may be
+		 * deemed empty while considering various join orders within
+		 * join_search_one_level.
+		 *
+		 * After that, we're done creating paths for the joinrel, so run
+		 * set_cheapest().
 		 */
 		foreach(lc, root->join_rel_level[lev])
 		{
@@ -2326,6 +2428,9 @@ standard_join_search(PlannerInfo *root, int levels_needed, List *initial_rels)
 			/* Create GatherPaths for any useful partial paths for rel */
 			generate_gather_paths(root, rel);
 
+			/* Create paths for partition-wise joins. */
+			generate_partition_wise_join_paths(root, rel);
+
 			/* Find and save the cheapest paths for this rel */
 			set_cheapest(rel);
 
@@ -3011,6 +3116,138 @@ compute_parallel_worker(RelOptInfo *rel, BlockNumber pages)
 	return parallel_workers;
 }
 
+/*
+ * Function to compare estimated sizes of two relations to be used with
+ * qsort(). Remember that this function is used to sort an array of position
+ * pointers in the array of partitions. So, we have to use double indirection.
+ * See more comments in generate_partition_wise_join_paths() where this
+ * function is used.
+ */
+static int
+compare_rel_size(const void *rel1_p, const void *rel2_p)
+{
+	RelOptInfo *rel1 = **(RelOptInfo ***) rel1_p;
+	RelOptInfo *rel2 = **(RelOptInfo ***) rel2_p;
+
+	return (int) (rel1->rows - rel2->rows);
+}
+
+/*
+ * generate_partition_wise_join_paths
+ *
+ * 		Create paths representing partition-wise join for given partitioned
+ * 		join relation.
+ *
+ * The number of paths created for a child-join is same as the number of paths
+ * created for join between parents. That number grows exponentially with the
+ * number of base relations being joined. The time and memory consumed to
+ * create paths for each child-join will be proporional to the number of
+ * partitions. This will not scale well with thousands of partitions. Instead
+ * of that we estimate partition-wise join cost based on the costs of sampled
+ * child-joins. We choose child-joins with higher sizes to have realistic
+ * estimates.
+ *
+ * This must be called after we have considered all joining orders since
+ * certain join orders may allow us to deem a child-join as dummy.
+ */
+void
+generate_partition_wise_join_paths(PlannerInfo *root, RelOptInfo *rel)
+{
+	List   *sampled_children = NIL;
+	int		cnt_parts;
+	int		num_part_to_plan;
+	int		num_parts;
+	bool	partition_join_path = false;
+	int		num_dummy_parts = 0;
+	RelOptInfo	  ***ordered_part_rels;
+	RelOptInfo	   **part_rels;
+
+	/* Handle only join relations. */
+	if (!IS_JOIN_REL(rel))
+		return;
+
+	/*
+	 * If none of the join orders for this relation could use partition-wise
+	 * join technique, the join is not partitioned. Reset the partitioning
+	 * scheme.
+	 */
+	if (!rel->part_rels)
+		rel->part_scheme = NULL;
+
+	/* If the relation is not partitioned or is proven dummy, nothing to do. */
+	if (!rel->part_scheme || IS_DUMMY_REL(rel))
+		return;
+
+	/* Guard against stack overflow due to overly deep partition hierarchy. */
+	check_stack_depth();
+
+	num_parts = rel->part_scheme->nparts;
+	part_rels = rel->part_rels;
+
+	/* Calculate number of child-joins to sample. */
+	num_part_to_plan = num_parts * sample_partition_fraction;
+	if (num_part_to_plan < 1)
+		num_part_to_plan = 1;
+
+	/* Order the child-join relations by their size.
+	 * add_paths_to_child_joinrel() needs the position of the child-join in the
+	 * array of partition relations. So instead of sorting the actual relations
+	 * get their indexes sorted. We use C pointer arithmatic with qsort to do
+	 * this.
+	 */
+	ordered_part_rels = (RelOptInfo ***) palloc(sizeof(RelOptInfo **) *
+															num_parts);
+	for (cnt_parts = 0; cnt_parts < num_parts; cnt_parts++)
+		ordered_part_rels[cnt_parts] = &part_rels[cnt_parts];
+	qsort(ordered_part_rels, num_parts, sizeof(ordered_part_rels[0]),
+		  compare_rel_size);
+
+	/*
+	 * Create paths for the child-joins for required number of largest
+	 * relations. qsort() returns relations ordered in ascending sizes, so
+	 * start from the end of the array.
+	 */
+	for (cnt_parts = num_parts - 1; cnt_parts >= 0; cnt_parts--)
+	{
+		int		child_no = ordered_part_rels[cnt_parts] - part_rels;
+		RelOptInfo *child_rel = part_rels[child_no];
+
+		/* Create paths for this child. */
+		add_paths_to_child_joinrel(root, rel, child_no);
+
+		/* Dummy children will not be scanned, so ingore those. */
+		if (IS_DUMMY_REL(child_rel))
+		{
+			num_dummy_parts++;
+			continue;
+		}
+
+#ifdef OPTIMIZER_DEBUG
+		debug_print_rel(root, rel);
+#endif
+
+		sampled_children = lappend(sampled_children, child_rel);
+
+		if (list_length(sampled_children) >= num_part_to_plan)
+			break;
+	}
+	pfree(ordered_part_rels);
+
+	/*
+	 * If the number of samples is same as the number of live children, an
+	 * append path will do. Otherwise, we will cost the partition-wise join
+	 * based on the sampled children using PartitionJoinPath.
+	 */
+	if (num_part_to_plan < num_parts - num_dummy_parts)
+		partition_join_path = true;
+
+	/* Add paths for partition-wise join based on the sampled children. */
+	add_paths_to_append_rel(root, rel, sampled_children, partition_join_path);
+
+	if (sampled_children)
+		list_free(sampled_children);
+}
+
 
 /*****************************************************************************
  *			DEBUG SUPPORT
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index a43daa7..c720115 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -126,6 +126,9 @@ bool		enable_nestloop = true;
 bool		enable_material = true;
 bool		enable_mergejoin = true;
 bool		enable_hashjoin = true;
+bool		enable_partition_wise_join = true;
+double		partition_wise_plan_weight = DEFAULT_PARTITION_WISE_PLAN_WEIGHT;
+double		sample_partition_fraction = DEFAULT_SAMPLE_PARTITION_FRACTION;
 
 typedef struct
 {
diff --git a/src/backend/optimizer/path/joinpath.c b/src/backend/optimizer/path/joinpath.c
index 2897245..f80fb25 100644
--- a/src/backend/optimizer/path/joinpath.c
+++ b/src/backend/optimizer/path/joinpath.c
@@ -96,6 +96,19 @@ add_paths_to_joinrel(PlannerInfo *root,
 	JoinPathExtraData extra;
 	bool		mergejoin_allowed = true;
 	ListCell   *lc;
+	Relids		joinrelids;
+
+	/*
+	 * PlannerInfo doesn't contain the SpecialJoinInfos created for joins
+	 * between child relations, even if there is a SpecialJoinInfo node for
+	 * the join between the topmost parents. Hence while calculating Relids
+	 * set representing the restriction, consider relids of topmost parent
+	 * of partitions.
+	 */
+	if (joinrel->reloptkind == RELOPT_OTHER_JOINREL)
+		joinrelids = joinrel->top_parent_relids;
+	else
+		joinrelids = joinrel->relids;
 
 	extra.restrictlist = restrictlist;
 	extra.mergeclause_list = NIL;
@@ -149,16 +162,16 @@ add_paths_to_joinrel(PlannerInfo *root,
 		 * join has already been proven legal.)  If the SJ is relevant, it
 		 * presents constraints for joining to anything not in its RHS.
 		 */
-		if (bms_overlap(joinrel->relids, sjinfo2->min_righthand) &&
-			!bms_overlap(joinrel->relids, sjinfo2->min_lefthand))
+		if (bms_overlap(joinrelids, sjinfo2->min_righthand) &&
+			!bms_overlap(joinrelids, sjinfo2->min_lefthand))
 			extra.param_source_rels = bms_join(extra.param_source_rels,
 										   bms_difference(root->all_baserels,
 													sjinfo2->min_righthand));
 
 		/* full joins constrain both sides symmetrically */
 		if (sjinfo2->jointype == JOIN_FULL &&
-			bms_overlap(joinrel->relids, sjinfo2->min_lefthand) &&
-			!bms_overlap(joinrel->relids, sjinfo2->min_righthand))
+			bms_overlap(joinrelids, sjinfo2->min_lefthand) &&
+			!bms_overlap(joinrelids, sjinfo2->min_righthand))
 			extra.param_source_rels = bms_join(extra.param_source_rels,
 										   bms_difference(root->all_baserels,
 													 sjinfo2->min_lefthand));
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 936ee0c..7476e8e 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -14,9 +14,14 @@
  */
 #include "postgres.h"
 
+#include "miscadmin.h"
+#include "nodes/relation.h"
+#include "optimizer/clauses.h"
 #include "optimizer/joininfo.h"
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
+#include "optimizer/prep.h"
+#include "optimizer/cost.h"
 #include "utils/memutils.h"
 
 
@@ -35,6 +40,14 @@ static bool restriction_is_constant_false(List *restrictlist,
 static void populate_joinrel_with_paths(PlannerInfo *root, RelOptInfo *rel1,
 							RelOptInfo *rel2, RelOptInfo *joinrel,
 							SpecialJoinInfo *sjinfo, List *restrictlist);
+static void try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1,
+						  RelOptInfo *rel2, RelOptInfo *joinrel,
+						  SpecialJoinInfo *parent_sjinfo,
+						  List *parent_restrictlist);
+static bool have_partkey_equi_join(RelOptInfo *rel1, RelOptInfo *rel2,
+						   JoinType jointype, List *restrictlist);
+static int match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel);
+static void free_special_join_info(SpecialJoinInfo *sjinfo);
 
 
 /*
@@ -731,6 +744,9 @@ make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 	populate_joinrel_with_paths(root, rel1, rel2, joinrel, sjinfo,
 								restrictlist);
 
+	/* Apply partition-wise join technique, if possible. */
+	try_partition_wise_join(root, rel1, rel2, joinrel, sjinfo, restrictlist);
+
 	bms_free(joinrelids);
 
 	return joinrel;
@@ -1269,3 +1285,408 @@ restriction_is_constant_false(List *restrictlist, bool only_pushed_down)
 	}
 	return false;
 }
+
+/* Free memory used by SpecialJoinInfo. */
+static void
+free_special_join_info(SpecialJoinInfo *sjinfo)
+{
+	bms_free(sjinfo->min_lefthand);
+	bms_free(sjinfo->syn_lefthand);
+	bms_free(sjinfo->syn_righthand);
+	pfree(sjinfo);
+}
+
+/*
+ * Assess whether join between given two partitioned relations can be broken
+ * down into joins between matching partitions; a technique called
+ * "partition-wise join"
+ *
+ * Partition-wise 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.
+ *
+ * Partition-wise join is planned as follows (details: optimizer/README.)
+ *
+ * 1. Create the RelOptInfos for joins between matching partitions i.e
+ * child-joins and estimate sizes of those. This function is responsible for
+ * this phase.
+ *
+ * 2. Add paths representing partition-wise join. The second phase is
+ * implemented by generate_partition_wise_join_paths(). In order to save time
+ * and memory consumed in creating paths for every child-join, we create paths
+ * for only few child-joins.
+ *
+ * 3. Create merge/append plan to combining plans for every child-join,
+ * creating paths for remaining child-joins.
+ *
+ * The RelOptInfo, SpecialJoinInfo and restrictlist for each child join are
+ * obtained by translating the respective parent join structures.
+ */
+static void
+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;
+	PartitionedJoin	   *partitioned_join;
+
+	/* 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)
+		return;
+
+	/*
+	 * If any of the joining parent relations is proven empty, either the join
+	 * will be empty (INNER join) or will have the inner side all nullified. We
+	 * take care of such cases when creating join paths for parent relations.
+	 * Nothing to be done here. Also, nothing to do, if the parent join is
+	 * proven empty.
+	 */
+	if (IS_DUMMY_REL(rel1) || IS_DUMMY_REL(rel2) || IS_DUMMY_REL(joinrel))
+		return;
+
+	/*
+	 * Partitioning scheme in join relation indicates a possibility that the
+	 * join may be partitioned, but it's not necessary that every pair of
+	 * joining relations can use partition-wise join technique. If one of
+	 * joining relations turns out to be unpartitioned, this pair of joining
+	 * relations can not use partition-wise join technique.
+	 */
+	if (!rel1->part_scheme || !rel2->part_scheme)
+		return;
+
+	/*
+	 * If an equi-join condition between the partition keys of the joining
+	 * relations does not exist, this pair of joining relations can not use
+	 * partition-wise technique.
+	 */
+	if (!have_partkey_equi_join(rel1, rel2, parent_sjinfo->jointype,
+								parent_restrictlist))
+		return;
+
+	/*
+	 * 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);
+
+	/* We should have RelOptInfos of the partitions available. */
+	Assert(rel1->part_rels && rel2->part_rels);
+
+	part_scheme = joinrel->part_scheme;
+	nparts = part_scheme->nparts;
+
+	/*
+	 * We do not store information about valid pairs of joining child
+	 * relations. The pair of joining relations for a child-join can be derived
+	 * from valid pairs of joining parent relations. Amongst the valid pairs of
+	 * parent joining relations, only those which result in partitioned join
+	 * matter for partition-wise join. Remember those so that we can use them
+	 * for creating paths for few child-joins in
+	 * generate_partition_wise_join_paths() later.
+	 */
+	partitioned_join = (PartitionedJoin *) palloc(sizeof(PartitionedJoin));
+	partitioned_join->rel1 = rel1;
+	partitioned_join->rel2 = rel2;
+	partitioned_join->sjinfo = copyObject(parent_sjinfo);
+	partitioned_join->restrictlist = parent_restrictlist;
+	joinrel->partitioned_joins = lappend(joinrel->partitioned_joins,
+										 partitioned_join);
+
+	elog(DEBUG3, "join between relations %s and %s is considered for partition-wise join.",
+		 bmsToString(rel1->relids), bmsToString(rel2->relids));
+
+	/* We are done if child RelOptInfos are already created. */
+	if (joinrel->part_rels)
+		return;
+
+	/* Create all the child RelOptInfos. */
+	joinrel->part_rels = (RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts);
+
+	/*
+	 * Create child join relations for this partitioned join. While doing so,
+	 * we estimate sizes of these child join relations. These estimates are
+	 * used to find the representative child relations used for costing the
+	 * partition-wise join later.
+	 */
+	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	{
+		RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts];
+		RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts];
+		SpecialJoinInfo	*child_sjinfo;
+		List	*child_restrictlist;
+		RelOptInfo *child_joinrel;
+
+		/* We should never try to join two overlapping sets of rels. */
+		Assert(!bms_overlap(child_rel1->relids, child_rel2->relids));
+
+		Assert (!joinrel->part_rels[cnt_parts]);
+
+		child_joinrel = build_child_join_rel(root, child_rel1, child_rel2,
+											 joinrel, parent_sjinfo->jointype);
+
+		joinrel->part_rels[cnt_parts] = child_joinrel;
+
+		/*
+		 * Construct restrictions applicable to the child join from
+		 * those applicable to the parent join.
+		 */
+		child_restrictlist = build_child_clauses(root, parent_restrictlist,
+												 find_appinfos_by_relids(root,
+													   child_joinrel->relids));
+
+		/*
+		 * Construct SpecialJoinInfo from parent join relations's
+		 * SpecialJoinInfo.
+		 */
+		child_sjinfo = build_child_join_sjinfo(root, parent_sjinfo,
+											   child_rel1->relids,
+											   child_rel2->relids);
+
+		/*
+		 * Set estimates of the child-joinrel's size.
+		 */
+		set_joinrel_size_estimates(root, child_joinrel, child_rel1, child_rel2,
+								   child_sjinfo, child_restrictlist);
+
+		/*
+		 * If the child relations themselves are partitioned, try partition-wise join
+		 * recursively.
+		 */
+		try_partition_wise_join(root, child_rel1, child_rel2, child_joinrel,
+								child_sjinfo, child_restrictlist);
+
+		free_special_join_info(child_sjinfo);
+		child_sjinfo = NULL;
+	}
+}
+
+/*
+ * add_paths_to_child_join
+ * 		Add paths to 'child_id'th child of given parent join relation.
+ *
+ * The function creates paths for given child-join by joining corresponding
+ * children of every pair of joining parent relations which produces
+ * partitioned join. Since we create paths only for sampled child-joins, either
+ * of the children being joined may not have paths. In that case, this function
+ * is called recursively to populate paths for those.
+ */
+void
+add_paths_to_child_joinrel(PlannerInfo *root, RelOptInfo *parent_joinrel,
+						   int child_id)
+{
+	ListCell	*lc;
+	RelOptInfo	   *child_joinrel = parent_joinrel->part_rels[child_id];
+
+	Assert(IS_JOIN_REL(parent_joinrel));
+
+	/* If this child relation already has paths, nothing to do. */
+	if (child_joinrel->cheapest_total_path)
+		return;
+
+	/* A dummy relation will have a dummy path as the cheapest path. */
+	Assert(!is_dummy_rel(child_joinrel));
+
+	/*
+	 * For every partitioned join order, calculate paths for the joining
+	 * child relations and then calculate paths for given child.
+	 */
+	foreach (lc, parent_joinrel->partitioned_joins)
+	{
+		PartitionedJoin	   *pj = lfirst(lc);
+		RelOptInfo *rel1 = pj->rel1;
+		RelOptInfo *rel2 = pj->rel2;
+		RelOptInfo *child_rel1 = rel1->part_rels[child_id];
+		RelOptInfo *child_rel2 = rel2->part_rels[child_id];
+		SpecialJoinInfo	   *child_sjinfo;
+		List	   *child_restrictlist;
+
+		/*
+		 * Add paths to joining relation if it is a join itself.
+		 * Paths for child base relations are created in
+		 * set_append_rel_pathlist().
+		 */
+		if (IS_JOIN_REL(pj->rel1))
+			add_paths_to_child_joinrel(root, rel1, child_id);
+
+		if (IS_JOIN_REL(pj->rel2))
+			add_paths_to_child_joinrel(root, rel2, child_id);
+
+		/*
+		 * Construct SpecialJoinInfo from parent join relations's
+		 * SpecialJoinInfo.
+		 */
+		child_sjinfo = build_child_join_sjinfo(root, pj->sjinfo,
+											   child_rel1->relids,
+											   child_rel2->relids);
+
+
+		/*
+		 * Construct restrictions applicable to the child join from
+		 * those applicable to the parent join.
+		 */
+		child_restrictlist = build_child_clauses(root, pj->restrictlist,
+												 find_appinfos_by_relids(root,
+													   child_joinrel->relids));
+
+		/* Add paths for child join. */
+		populate_joinrel_with_paths(root, rel1->part_rels[child_id],
+									rel2->part_rels[child_id], child_joinrel,
+									child_sjinfo, child_restrictlist);
+
+		/* Add partition-wise join paths for partitioned child-joins. */
+		generate_partition_wise_join_paths(root, child_joinrel);
+
+		free_special_join_info(child_sjinfo);
+		child_sjinfo = NULL;
+	}
+
+	set_cheapest(child_joinrel);
+}
+
+/*
+ * Returns true if there exists an equi-join condition for each pair of
+ * partition key from given relations being joined.
+ */
+static bool
+have_partkey_equi_join(RelOptInfo *rel1, RelOptInfo *rel2,
+					 JoinType jointype, List *restrictlist)
+{
+	PartitionScheme	part_scheme = rel1->part_scheme;
+	ListCell	*lc;
+	int		cnt_pks;
+	int		num_pks;
+	bool   *pk_has_clause;
+
+	/*
+	 * This function should be called when the joining relations have same
+	 * partitioning scheme.
+	 */
+	Assert(rel1->part_scheme == rel2->part_scheme);
+	Assert(part_scheme);
+
+	num_pks = part_scheme->partnatts;
+
+	pk_has_clause = (bool *) palloc0(sizeof(bool) * num_pks);
+
+	foreach (lc, restrictlist)
+	{
+		RestrictInfo *rinfo = lfirst(lc);
+		OpExpr		 *opexpr;
+		Expr		 *expr1;
+		Expr		 *expr2;
+		int		ipk1;
+		int		ipk2;
+
+		/* If processing an outer join, only use its own join clauses. */
+		if (IS_OUTER_JOIN(jointype) && rinfo->is_pushed_down)
+			continue;
+
+		/* Skip clauses which can not be used for a join. */
+		if (!rinfo->can_join)
+			continue;
+
+		/* Skip clauses which are not equality conditions. */
+		if (rinfo->hashjoinoperator == InvalidOid && !rinfo->mergeopfamilies)
+			continue;
+
+		opexpr = (OpExpr *) rinfo->clause;
+		Assert(is_opclause(opexpr));
+
+
+		/* Match the operands to the relation. */
+		if (bms_is_subset(rinfo->left_relids, rel1->relids) &&
+			bms_is_subset(rinfo->right_relids, rel2->relids))
+		{
+			expr1 = linitial(opexpr->args);
+			expr2 = lsecond(opexpr->args);
+		}
+		else if (bms_is_subset(rinfo->left_relids, rel2->relids) &&
+				 bms_is_subset(rinfo->right_relids, rel1->relids))
+		{
+			expr1 = lsecond(opexpr->args);
+			expr2 = linitial(opexpr->args);
+		}
+		else
+			continue;
+
+		/* Associate matching clauses with partition keys. */
+		ipk1 = match_expr_to_partition_keys(expr1, rel1);
+		ipk2 = match_expr_to_partition_keys(expr2, rel2);
+
+		/*
+		 * If the clause refers to different partition keys from
+		 * both relations, it can not be used for partition-wise join.
+		 */
+		if (ipk1 != ipk2)
+			continue;
+
+		/*
+		 * The clause allows partition-wise join if only it uses the same
+		 * operator family as that specified by the partition key.
+		 */
+		if (!list_member_oid(rinfo->mergeopfamilies,
+							 part_scheme->partopfamily[ipk1]))
+			continue;
+
+		/* Mark the partition key as having an equi-join clause. */
+		pk_has_clause[ipk1] = true;
+	}
+
+	/* Check whether every partition key has an equi-join condition. */
+	for (cnt_pks = 0; cnt_pks < num_pks; cnt_pks++)
+	{
+		if (!pk_has_clause[cnt_pks])
+		{
+			pfree(pk_has_clause);
+			return false;
+		}
+	}
+
+	pfree(pk_has_clause);
+	return true;
+}
+
+/*
+ * Find the partition key from the given relation matching the given
+ * expression. If found, return the index of the partition key, else return -1.
+ */
+static int
+match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel)
+{
+	int		cnt_pks;
+	int		num_pks;
+
+	/* This function should be called only for partitioned relations. */
+	Assert(rel->part_scheme);
+
+	num_pks = rel->part_scheme->partnatts;
+
+	/*
+	 * Remove the relabel decoration. We can assume that there is at most one
+	 * RelabelType node; eval_const_expressions() simplifies multiple
+	 * RelabelType nodes into one.
+	 */
+	if (IsA(expr, RelabelType))
+		expr = (Expr *) ((RelabelType *) expr)->arg;
+
+	for (cnt_pks = 0; cnt_pks < num_pks; cnt_pks++)
+	{
+		List	 *pkexprs = rel->partexprs[cnt_pks];
+		ListCell *lc;
+
+		foreach(lc, pkexprs)
+		{
+			Expr *pkexpr = lfirst(lc);
+			if (equal(pkexpr, expr))
+				return cnt_pks;
+		}
+	}
+
+	return -1;
+}
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 997bdcf..c1f9da2 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -42,6 +42,7 @@
 #include "parser/parse_clause.h"
 #include "parser/parsetree.h"
 #include "utils/lsyscache.h"
+#include "utils/memutils.h"
 
 
 /*
@@ -146,6 +147,8 @@ static CustomScan *create_customscan_plan(PlannerInfo *root,
 static NestLoop *create_nestloop_plan(PlannerInfo *root, NestPath *best_path);
 static MergeJoin *create_mergejoin_plan(PlannerInfo *root, MergePath *best_path);
 static HashJoin *create_hashjoin_plan(PlannerInfo *root, HashPath *best_path);
+static Plan *create_partition_join_plan(PlannerInfo *root,
+										PartitionJoinPath *best_path);
 static Node *replace_nestloop_params(PlannerInfo *root, Node *expr);
 static Node *replace_nestloop_params_mutator(Node *node, PlannerInfo *root);
 static void process_subquery_nestloop_params(PlannerInfo *root,
@@ -369,12 +372,20 @@ create_plan_recurse(PlannerInfo *root, Path *best_path, int flags)
 									(JoinPath *) best_path);
 			break;
 		case T_Append:
-			plan = create_append_plan(root,
-									  (AppendPath *) best_path);
+			if (IsA(best_path, PartitionJoinPath))
+				plan = create_partition_join_plan(root,
+											   (PartitionJoinPath *)best_path);
+			else
+				plan = create_append_plan(root,
+										  (AppendPath *) best_path);
 			break;
 		case T_MergeAppend:
-			plan = create_merge_append_plan(root,
-											(MergeAppendPath *) best_path);
+			if (IsA(best_path, PartitionJoinPath))
+				plan = create_partition_join_plan(root,
+											   (PartitionJoinPath *)best_path);
+			else
+				plan = create_merge_append_plan(root,
+												(MergeAppendPath *) best_path);
 			break;
 		case T_Result:
 			if (IsA(best_path, ProjectionPath))
@@ -3982,6 +3993,215 @@ create_hashjoin_plan(PlannerInfo *root,
 	return join_plan;
 }
 
+/*
+ * create_partition_join_plan
+ *		Creates Merge/Append plan consisting of join plans for child-join.
+ *
+ *		Returns a Plan node.
+ */
+static Plan *
+create_partition_join_plan(PlannerInfo *root, PartitionJoinPath *best_path)
+{
+	RelOptInfo *joinrel = best_path->path.parent;
+	int		nparts;
+	int		cnt_parts;
+	List	   *child_plans = NIL;
+	List	   *tlist = build_path_tlist(root, &best_path->path);
+	Plan	   *plan;
+	MemoryContext	child_context;
+	MemoryContext	old_context;
+	List	   *pathkeys = best_path->path.pathkeys;
+	StringInfoData	mem_context_name;
+
+	/* The relation should be a partitioned join relation. */
+	Assert(IS_JOIN_REL(joinrel) && joinrel->part_scheme &&
+		   joinrel->partitioned_joins);
+
+	nparts = joinrel->part_scheme->nparts;
+
+	/* Create MergeAppend plan when result is expected to be ordered. */
+	if (pathkeys)
+	{
+		MergeAppend *node = makeNode(MergeAppend);
+		plan = &node->plan;
+
+		plan->targetlist = tlist;
+
+		/* Compute sorting info, and adjust MergeAppend's tlist as needed. */
+		(void) prepare_sort_from_pathkeys(plan, pathkeys,
+										  best_path->path.parent->relids,
+										  NULL,
+										  true,
+										  &node->numCols,
+										  &node->sortColIdx,
+										  &node->sortOperators,
+										  &node->collations,
+										  &node->nullsFirst);
+	}
+	else
+	{
+		Append  *node = makeNode(Append);
+		plan = &node->plan;
+		plan->targetlist = tlist;
+	}
+
+	/* Fill costs, so that we can cost Sort node, if required. */
+	copy_generic_path_info(plan, (Path *) best_path);
+
+	/*
+	 * Create a new memory context for planning child joins. Since this routine
+	 * may be called recursively for tables with subpartitions, we use
+	 * a unique context name for every level of partition by using the lowest
+	 * relid amongst the base relations being joined.
+	 */
+	initStringInfo(&mem_context_name);
+	appendStringInfo(&mem_context_name, "%s_%d", "ChildJoinContext",
+					 bms_next_member(joinrel->relids, -1));
+	child_context = AllocSetContextCreate(CurrentMemoryContext,
+										  pstrdup(mem_context_name.data),
+										  ALLOCSET_DEFAULT_SIZES);
+	pfree(mem_context_name.data);
+	resetStringInfo(&mem_context_name);
+
+	/*
+	 * Create a paths for all child joins, one child join at a time. The paths
+	 * for every child join are independent i.e. one child does not require
+	 * paths created for the other. In order to avoid accumulating memory
+	 * consumed while creating paths for every child join, we use a fresh
+	 * memory context for every child join.
+	 */
+	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+	{
+		RelOptInfo *child_join;
+		Path	   *child_path = NULL;
+		Plan	   *child_plan;
+		int			numsortkeys;
+		AttrNumber *sortColIdx;
+		Oid		   *sortOperators;
+		Oid		   *collations;
+		bool	   *nullsFirst;
+
+		/*
+		 * Create paths for the child join in a separate context, so that we
+		 * can reuse the memory used by those paths.
+		 */
+		old_context = MemoryContextSwitchTo(child_context);
+
+		add_paths_to_child_joinrel(root, joinrel, cnt_parts);
+
+		child_join = joinrel->part_rels[cnt_parts];
+
+
+		/* Skip empty child. */
+		if (IS_DUMMY_REL(child_join))
+		{
+			MemoryContextSwitchTo(old_context);
+			continue;
+		}
+
+#ifdef OPTIMIZER_DEBUG
+		debug_print_rel(root, rel);
+#endif
+
+		if (!PATH_REQ_OUTER(&best_path->path))
+			child_path = get_cheapest_path_for_pathkeys(child_join->pathlist,
+													  best_path->path.pathkeys,
+														NULL,
+														TOTAL_COST);
+		else
+			child_path = get_cheapest_parameterized_child_path(root,
+															   child_join,
+											  PATH_REQ_OUTER(&best_path->path));
+
+		if (!child_path)
+			elog(ERROR, "Could not find a child-join path with required pathkeys or parameterization.");
+
+		MemoryContextSwitchTo(old_context);
+
+		/* Create plan for the current child. */
+		child_plan = create_plan_recurse(root, child_path, CP_EXACT_TLIST);
+
+		if (pathkeys)
+		{
+			MergeAppend *node = (MergeAppend *) plan;
+
+			Assert(IsA(node, MergeAppend));
+
+			/* Compute sorting info, and adjust subplan's tlist as needed */
+			child_plan = prepare_sort_from_pathkeys(child_plan, pathkeys,
+												 child_path->parent->relids,
+												 node->sortColIdx,
+												 false,
+												 &numsortkeys,
+												 &sortColIdx,
+												 &sortOperators,
+												 &collations,
+												 &nullsFirst);
+
+			/*
+			 * Check that we got the same sort key information.  We just Assert
+			 * that the sortops match, since those depend only on the pathkeys;
+			 * but it seems like a good idea to check the sort column numbers
+			 * explicitly, to ensure the tlists really do match up.
+			 */
+			Assert(numsortkeys == node->numCols);
+			if (memcmp(sortColIdx, node->sortColIdx,
+					   numsortkeys * sizeof(AttrNumber)) != 0)
+				elog(ERROR, "MergeAppend child's targetlist doesn't match MergeAppend");
+			Assert(memcmp(sortOperators, node->sortOperators,
+						  numsortkeys * sizeof(Oid)) == 0);
+			Assert(memcmp(collations, node->collations,
+						  numsortkeys * sizeof(Oid)) == 0);
+			Assert(memcmp(nullsFirst, node->nullsFirst,
+						  numsortkeys * sizeof(bool)) == 0);
+
+			/* Now, insert a Sort node if subplan isn't sufficiently ordered */
+			if (!pathkeys_contained_in(pathkeys, child_path->pathkeys))
+			{
+				Sort	   *sort = make_sort(child_plan, numsortkeys,
+											 sortColIdx, sortOperators,
+											 collations, nullsFirst);
+				label_sort_with_costsize(root, sort, -1.0);
+				child_plan = (Plan *) sort;
+			}
+		}
+
+		child_plans = lappend(child_plans, child_plan);
+
+		/*
+		 * Reset the child_join memory context to reclaim the memory consumed
+		 * while creating paths.
+		 */
+		MemoryContextResetAndDeleteChildren(child_context);
+	}
+
+	/* Destroy the child context as we do not need it anymore. */
+	Assert(CurrentMemoryContext == old_context);
+	MemoryContextDelete(child_context);
+
+	/* Partitioned relation with all empty children gets a dummy path. */
+	Assert(child_plans != NIL);
+
+	if (IsA(plan, MergeAppend))
+	{
+		MergeAppend *node = (MergeAppend *)plan;
+
+		node->mergeplans = child_plans;
+	}
+	else
+	{
+		Append  *node = (Append *)plan;
+
+		Assert(IsA(plan, Append));
+		node->appendplans = child_plans;
+	}
+
+	/* Complete rest of the plan. */
+	plan->qual = NIL;
+	plan->lefttree = NULL;
+	plan->righttree = NULL;
+	return plan;
+}
 
 /*****************************************************************************
  *
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index 6f41979..676204f 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -2179,3 +2179,163 @@ adjust_appendrel_attrs_multilevel(PlannerInfo *root, Node *node,
 	/* Now translate for this child */
 	return adjust_appendrel_attrs(root, node, list_make1(appinfo));
 }
+
+/*
+ * build_child_restrictinfo
+ *		Returns a RestrictInfo which is derived from the given RestrictInfo by
+ *		applying the parent-child translation specified by the list of
+ *		AppendRelInfos.
+ *
+ * The topmost parent's RestrictInfo maintains a list of child RestrictInfos
+ * derived from it. If a suitable RestrictInfo is found in that list, it is
+ * returned as is. If there is no such child RestrictInfo, we translate the given
+ * RestrictInfo using the given list of AppendRelInfos and stick it in the
+ * topmost parent's list before returning it to the caller.
+ */
+RestrictInfo *
+build_child_restrictinfo(PlannerInfo *root, RestrictInfo *rinfo,
+						 List *append_rel_infos)
+{
+	Relids child_required_relids;
+	ListCell   *lc;
+	RestrictInfo   *parent_rinfo;
+	RestrictInfo   *child_rinfo;
+	MemoryContext	old_context;
+
+	child_required_relids = adjust_relid_set(rinfo->required_relids,
+												append_rel_infos);
+
+
+	/* Nothing to do, if the clause does not need any translation. */
+	if (bms_equal(child_required_relids, rinfo->required_relids))
+	{
+		bms_free(child_required_relids);
+		return rinfo;
+	}
+
+	/*
+	 * Check if we already have the RestrictInfo for the given child in the
+	 * topmost parent's RestrictInfo.
+	 */
+	parent_rinfo = rinfo->parent_rinfo ? rinfo->parent_rinfo : rinfo;
+	foreach (lc, parent_rinfo->child_rinfos)
+	{
+		child_rinfo = lfirst(lc);
+
+		if (bms_equal(child_rinfo->required_relids, child_required_relids))
+		{
+			bms_free(child_required_relids);
+			return child_rinfo;
+		}
+	}
+
+	/*
+	 * We didn't find any child restrictinfo for the given child, translate the
+	 * given RestrictInfo and stick it into the parent's list. The clause
+	 * expression may get used in plan, so create the child RestrictInfo in the
+	 * planner's context.
+	 */
+	old_context = MemoryContextSwitchTo(root->planner_cxt);
+	child_rinfo = (RestrictInfo *) adjust_appendrel_attrs(root, (Node *) rinfo,
+														  append_rel_infos);
+	bms_free(child_required_relids);
+	parent_rinfo->child_rinfos = lappend(parent_rinfo->child_rinfos,
+										 child_rinfo);
+	child_rinfo->parent_rinfo = parent_rinfo;
+
+	MemoryContextSwitchTo(old_context);
+
+	return child_rinfo;
+}
+
+/*
+ * build_child_clauses
+ *		Convenience routine to call build_child_restrictinfo on a list of
+ *		clauses.
+ */
+List *
+build_child_clauses(PlannerInfo *root, List *clauses, List *append_rel_infos)
+{
+	List   *child_clauses = NIL;
+	ListCell   *lc;
+
+	foreach (lc, clauses)
+	{
+		RestrictInfo   *parent_rinfo = lfirst(lc);
+		RestrictInfo   *child_rinfo;
+
+		Assert(IsA(parent_rinfo, RestrictInfo));
+
+		child_rinfo = build_child_restrictinfo(root, parent_rinfo,
+											   append_rel_infos);
+
+		child_clauses = lappend(child_clauses, child_rinfo);
+	}
+
+	return child_clauses;
+}
+
+/*
+ * find_appinfos_by_relids
+ * 		Find AppendRelInfo structures for all relations specified by relids.
+ */
+List *
+find_appinfos_by_relids(PlannerInfo *root, Relids relids)
+{
+	ListCell	*lc;
+	List		*appinfo_list = NIL;
+
+	foreach (lc, root->append_rel_list)
+	{
+		AppendRelInfo *appinfo = lfirst(lc);
+
+		if (bms_is_member(appinfo->child_relid, relids))
+			appinfo_list = lappend(appinfo_list, appinfo);
+	}
+
+	Assert(list_length(appinfo_list) == bms_num_members(relids));
+	return appinfo_list;
+}
+
+/*
+ * Construct the SpecialJoinInfo for a child-join by translating
+ * SpecialJoinInfo for the join between parents. left_relids and right_relids
+ * are the relids of left and right side of the join respectively.
+ */
+SpecialJoinInfo *
+build_child_join_sjinfo(PlannerInfo *root, SpecialJoinInfo *parent_sjinfo,
+						Relids left_relids, Relids right_relids)
+{
+	SpecialJoinInfo *sjinfo = makeNode(SpecialJoinInfo);
+	MemoryContext	old_context;
+	List	   *left_appinfos = find_appinfos_by_relids(root, left_relids);
+	List	   *right_appinfos = find_appinfos_by_relids(root, right_relids);
+
+	memcpy(sjinfo, parent_sjinfo, sizeof(SpecialJoinInfo));
+
+	sjinfo->min_lefthand = adjust_relid_set(sjinfo->min_lefthand,
+											   left_appinfos);
+	sjinfo->min_righthand = adjust_relid_set(sjinfo->min_righthand,
+												right_appinfos);
+	sjinfo->syn_lefthand = adjust_relid_set(sjinfo->syn_lefthand,
+											   left_appinfos);
+	sjinfo->syn_righthand = adjust_relid_set(sjinfo->syn_righthand,
+												right_appinfos);
+
+	/*
+	 * Replace the Var nodes of parent with those of children in expressions.
+	 * This function may be called within a temporary context, but the
+	 * expressions will be shallow-copied into the plan. Hence copy those in
+	 * the planner's context.
+	 */
+	old_context = MemoryContextSwitchTo(root->planner_cxt);
+	sjinfo->semi_rhs_exprs = (List *) adjust_appendrel_attrs(root,
+											   (Node *) sjinfo->semi_rhs_exprs,
+															 right_appinfos);
+	MemoryContextSwitchTo(old_context);
+
+	list_free(left_appinfos);
+	list_free(right_appinfos);
+
+	return sjinfo;
+}
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index f440875..d861a49 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -23,7 +23,9 @@
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
 #include "optimizer/planmain.h"
+#include "optimizer/prep.h"
 #include "optimizer/restrictinfo.h"
+#include "optimizer/tlist.h"
 #include "optimizer/var.h"
 #include "parser/parsetree.h"
 #include "utils/lsyscache.h"
@@ -2154,6 +2156,117 @@ create_hashjoin_path(PlannerInfo *root,
 }
 
 /*
+ * create_partition_join_path
+ * 		Creates a pathnode that represents partition-wise join for given
+ * 		partitioned join relation.
+ *
+ * This function is called when we haven't created paths for all the child
+ * joins. It estimates the number of rows and cost of the PartitionJoinPath
+ * based upon the number of rows and the cost of representative child-joins
+ * paths.
+ */
+PartitionJoinPath *
+create_partition_join_path(PlannerInfo *root, RelOptInfo *rel, List *subpaths,
+						   Bitmapset *required_outer, List *pathkeys)
+{
+	PartitionJoinPath *pathnode = makeNode(PartitionJoinPath);
+	double	subpath_rows = 0;
+	double	subpath_startup_cost = 0;
+	double	subpath_total_cost = 0;
+	double	child_rel_rows = 0;
+	ListCell   *lc;
+
+	Assert(rel->part_scheme);
+
+	pathnode->path.pathtype = pathkeys ? T_MergeAppend : T_Append;
+	pathnode->path.parent = rel;
+	pathnode->path.pathtarget = rel->reltarget;
+	pathnode->path.param_info = get_appendrel_parampathinfo(rel,
+															required_outer);
+	pathnode->path.pathkeys = pathkeys;
+
+	/* No parallel paths here. See more details in add_paths_to_append_rel() */
+	pathnode->path.parallel_aware = false;
+	pathnode->path.parallel_safe = false;
+	pathnode->path.parallel_workers = 0;
+
+	/* Accumulate the number of rows and costs from the given subpaths. */
+	foreach (lc, subpaths)
+	{
+		Path   *subpath = lfirst(lc);
+
+		if (!pathkeys)
+		{
+			/*
+			 * Startup cost of an append relation is the startup cost of the
+			 * first subpath. Assume that the given first child will be the
+			 * first child in the final plan as well.
+			 */
+			if (lc == list_head(subpaths))
+				subpath_startup_cost = subpath->startup_cost;
+			subpath_total_cost += subpath->total_cost;
+		}
+		else if (pathkeys_contained_in(pathkeys, subpath->pathkeys))
+		{
+			/*
+			 * Subpath is adequately ordered, we won't need to sort it. We need
+			 * all the subplans to return their respective first rows, before
+			 * returning a row. So add the startup costs.
+			 */
+			subpath_startup_cost += subpath->startup_cost;
+			subpath_total_cost += subpath->total_cost;
+		}
+		else
+		{
+			/* We'll need to insert a Sort node, so include cost for that */
+			Path		sort_path;		/* dummy for result of cost_sort */
+
+			cost_sort(&sort_path,
+					  root,
+					  pathkeys,
+					  subpath->total_cost,
+					  subpath->parent->tuples,
+					  subpath->pathtarget->width,
+					  0.0,
+					  work_mem,
+					  -1);
+			subpath_startup_cost += sort_path.startup_cost;
+			subpath_total_cost += sort_path.total_cost;
+		}
+
+		subpath_rows += subpath->rows;
+		child_rel_rows += subpath->parent->rows;
+
+	}
+
+	/*
+	 * For a parameterized path, extrapolate the number of rows for the append
+	 * relation by considering the average selectivity of the parameterization
+	 * across the given children.
+	 */
+	if (bms_is_empty(required_outer))
+		pathnode->path.rows = rel->rows;
+	else
+		pathnode->path.rows = rel->rows * (subpath_rows / child_rel_rows);
+
+	/* Extrapolate the total cost to account for yet-to-be planned children. */
+	if (!pathkeys)
+		pathnode->path.startup_cost = subpath_startup_cost;
+	else
+		pathnode->path.startup_cost = (subpath_startup_cost * pathnode->path.rows) / subpath_rows;
+	pathnode->path.total_cost = (subpath_total_cost * pathnode->path.rows) / subpath_rows;
+
+	/*
+	 * Multiply the costs with scaling factor as specified. Used to encourage
+	 * or discourage use of partition-wise join plans.
+	 */
+	pathnode->path.startup_cost *= partition_wise_plan_weight;
+	pathnode->path.total_cost *= partition_wise_plan_weight;
+
+	return pathnode;
+}
+
+/*
  * create_projection_path
  *	  Creates a pathnode that represents performing a projection.
  *
diff --git a/src/backend/optimizer/util/placeholder.c b/src/backend/optimizer/util/placeholder.c
index 698a387..e06bccc 100644
--- a/src/backend/optimizer/util/placeholder.c
+++ b/src/backend/optimizer/util/placeholder.c
@@ -20,6 +20,7 @@
 #include "optimizer/pathnode.h"
 #include "optimizer/placeholder.h"
 #include "optimizer/planmain.h"
+#include "optimizer/prep.h"
 #include "optimizer/var.h"
 #include "utils/lsyscache.h"
 
@@ -414,6 +415,10 @@ add_placeholders_to_joinrel(PlannerInfo *root, RelOptInfo *joinrel,
 	Relids		relids = joinrel->relids;
 	ListCell   *lc;
 
+	/* This function is called only on the parent relations. */
+	Assert(!IS_OTHER_REL(joinrel) && !IS_OTHER_REL(outer_rel) &&
+		   !IS_OTHER_REL(inner_rel));
+
 	foreach(lc, root->placeholder_list)
 	{
 		PlaceHolderInfo *phinfo = (PlaceHolderInfo *) lfirst(lc);
@@ -459,3 +464,53 @@ add_placeholders_to_joinrel(PlannerInfo *root, RelOptInfo *joinrel,
 		}
 	}
 }
+
+/*
+ * add_placeholders_to_child_joinrel
+ *		Translate the PHVs in parent's targetlist and add them to the child's
+ *		targetlist. Also adjust the cost
+ */
+void
+add_placeholders_to_child_joinrel(PlannerInfo *root, RelOptInfo *childrel,
+								  RelOptInfo *parentrel)
+{
+	ListCell  *lc;
+
+	/* This function is called only for join relations. */
+	Assert(IS_JOIN_REL(childrel) && IS_JOIN_REL(parentrel));
+
+	/* Ensure child relations is really what it claims to be. */
+	Assert(IS_OTHER_REL(childrel));
+
+	foreach (lc, parentrel->reltarget->exprs)
+	{
+		PlaceHolderVar *phv = lfirst(lc);
+
+		if (IsA(phv, PlaceHolderVar))
+		{
+			/*
+			 * In case the placeholder Var refers to any of the parent
+			 * relations, translate it to refer to the corresponding child.
+			 */
+			if (bms_overlap(phv->phrels, parentrel->relids) &&
+				childrel->reloptkind == RELOPT_OTHER_JOINREL)
+			{
+				List *append_rel_infos;
+
+				append_rel_infos = find_appinfos_by_relids(root,
+														  childrel->relids);
+				phv = (PlaceHolderVar *) adjust_appendrel_attrs(root,
+															  (Node *) phv,
+														 append_rel_infos);
+			}
+
+			childrel->reltarget->exprs = lappend(childrel->reltarget->exprs,
+												 phv);
+		}
+	}
+
+	/* Adjust the cost and width of child targetlist. */
+	childrel->reltarget->cost.startup = parentrel->reltarget->cost.startup;
+	childrel->reltarget->cost.per_tuple = parentrel->reltarget->cost.per_tuple;
+	childrel->reltarget->width = parentrel->reltarget->width;
+}
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 19982dc..1eed987 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -23,6 +23,7 @@
 #include "optimizer/paths.h"
 #include "optimizer/placeholder.h"
 #include "optimizer/plancat.h"
+#include "optimizer/prep.h"
 #include "optimizer/restrictinfo.h"
 #include "optimizer/tlist.h"
 #include "utils/hsearch.h"
@@ -54,6 +55,9 @@ static void set_foreign_rel_properties(RelOptInfo *joinrel,
 static void add_join_rel(PlannerInfo *root, RelOptInfo *joinrel);
 extern ParamPathInfo *find_param_path_info(RelOptInfo *rel,
 									  Relids required_outer);
+static void build_joinrel_partition_info(RelOptInfo *joinrel,
+						  RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+						  JoinType jointype);
 
 
 /*
@@ -434,6 +438,9 @@ build_join_rel(PlannerInfo *root,
 	RelOptInfo *joinrel;
 	List	   *restrictlist;
 
+	/* This function should be used only for join between parents. */
+	Assert(!IS_OTHER_REL(outer_rel) && !IS_OTHER_REL(inner_rel));
+
 	/*
 	 * See if we already have a joinrel for this set of base rels.
 	 */
@@ -532,6 +539,10 @@ build_join_rel(PlannerInfo *root,
 	if (bms_is_empty(joinrel->direct_lateral_relids))
 		joinrel->direct_lateral_relids = NULL;
 
+	/* Store the partition information. */
+	build_joinrel_partition_info(joinrel, outer_rel, inner_rel,
+								 sjinfo->jointype);
+
 	/*
 	 * Construct restrict and join clause lists for the new joinrel. (The
 	 * caller might or might not need the restrictlist, but I need it anyway
@@ -594,6 +605,126 @@ build_join_rel(PlannerInfo *root,
 	return joinrel;
 }
 
+ /*
+ * build_child_join_rel
+ *		Builds RelOptInfo for joining given two child relations from RelOptInfo
+ *		representing the join between their parents.
+ *
+ * 'outer_rel' and 'inner_rel' are the RelOptInfos of child relations being
+ *		joined.
+ * 'parent_joinrel' is the RelOptInfo representing the join between parent
+ *		relations. Most of the members of new RelOptInfo are produced by
+ *		translating corresponding members of this RelOptInfo.
+ * 'sjinfo': context info for child join
+ * 'restrictlist': list of RestrictInfo nodes that apply to this particular
+ *		pair of joinable relations.
+ * 'join_appinfos': list of AppendRelInfo nodes for base child relations involved
+ *		in this join.
+ */
+RelOptInfo *
+build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
+						 RelOptInfo *inner_rel, RelOptInfo *parent_joinrel,
+						 JoinType jointype)
+{
+	RelOptInfo *joinrel = makeNode(RelOptInfo);
+
+	/* Only joins between other relations land here. */
+	Assert(IS_OTHER_REL(outer_rel) && IS_OTHER_REL(inner_rel));
+
+	joinrel->reloptkind = RELOPT_OTHER_JOINREL;
+	joinrel->relids = bms_union(outer_rel->relids, inner_rel->relids);
+	joinrel->rows = 0;
+	/* cheap startup cost is interesting iff not all tuples to be retrieved */
+	joinrel->consider_startup = (root->tuple_fraction > 0);
+	joinrel->consider_param_startup = false;
+	joinrel->consider_parallel = false;
+	joinrel->reltarget = create_empty_pathtarget();
+	joinrel->pathlist = NIL;
+	joinrel->ppilist = NIL;
+	joinrel->partial_pathlist = NIL;
+	joinrel->cheapest_startup_path = NULL;
+	joinrel->cheapest_total_path = NULL;
+	joinrel->cheapest_unique_path = NULL;
+	joinrel->cheapest_parameterized_paths = NIL;
+	joinrel->direct_lateral_relids = NULL;
+	joinrel->lateral_relids = NULL;
+	joinrel->relid = 0;			/* indicates not a baserel */
+	joinrel->rtekind = RTE_JOIN;
+	joinrel->min_attr = 0;
+	joinrel->max_attr = 0;
+	joinrel->attr_needed = NULL;
+	joinrel->attr_widths = NULL;
+	joinrel->lateral_vars = NIL;
+	joinrel->lateral_referencers = NULL;
+	joinrel->indexlist = NIL;
+	joinrel->pages = 0;
+	joinrel->tuples = 0;
+	joinrel->allvisfrac = 0;
+	joinrel->subroot = NULL;
+	joinrel->subplan_params = NIL;
+	joinrel->serverid = InvalidOid;
+	joinrel->userid = InvalidOid;
+	joinrel->useridiscurrent = false;
+	joinrel->fdwroutine = NULL;
+	joinrel->fdw_private = NULL;
+	joinrel->baserestrictinfo = NIL;
+	joinrel->baserestrictcost.startup = 0;
+	joinrel->baserestrictcost.per_tuple = 0;
+	joinrel->joininfo = NIL;
+	joinrel->has_eclass_joins = false;
+	joinrel->part_scheme = NULL;
+	joinrel->partexprs = NULL;
+	joinrel->top_parent_relids = NULL;
+	joinrel->part_rels = NULL;
+
+	joinrel->top_parent_relids = bms_union(outer_rel->top_parent_relids,
+										   inner_rel->top_parent_relids);
+
+	/* Compute information relevant to foreign relations. */
+	set_foreign_rel_properties(joinrel, outer_rel, inner_rel);
+
+	/* Build targetlist */
+	build_joinrel_tlist(root, joinrel, outer_rel);
+	build_joinrel_tlist(root, joinrel, inner_rel);
+	/* Add placeholder variables. */
+	add_placeholders_to_child_joinrel(root, joinrel, parent_joinrel);
+
+	/* Construct joininfo list. */
+	joinrel->joininfo = build_child_clauses(root, parent_joinrel->joininfo,
+											find_appinfos_by_relids(root,
+															 joinrel->relids));
+
+	/*
+	 * Lateral relids referred in child join will be same as that referred in
+	 * the parent relation. Throw any partial result computed while building
+	 * the targetlist.
+	 */
+	bms_free(joinrel->direct_lateral_relids);
+	bms_free(joinrel->lateral_relids);
+	joinrel->direct_lateral_relids = (Relids) bms_copy(parent_joinrel->direct_lateral_relids);
+	joinrel->lateral_relids = (Relids) bms_copy(parent_joinrel->lateral_relids);
+
+	/*
+	 * If the parent joinrel has pending equivalence classes, so does the
+	 * child.
+	 */
+	joinrel->has_eclass_joins = parent_joinrel->has_eclass_joins;
+
+	/* Is the join between partitions itself partitioned? */
+	build_joinrel_partition_info(joinrel, outer_rel, inner_rel, jointype);
+
+	/* Child joinrel is parallel safe if parent is parallel safe. */
+	joinrel->consider_parallel = parent_joinrel->consider_parallel;
+
+	/* We build the join only once. */
+	Assert(!find_join_rel(root, joinrel->relids));
+
+	/* Add the relation to the PlannerInfo. */
+	add_join_rel(root, joinrel);
+
+	return joinrel;
+}
+
 /*
  * min_join_parameterization
  *
@@ -649,9 +780,15 @@ static void
 build_joinrel_tlist(PlannerInfo *root, RelOptInfo *joinrel,
 					RelOptInfo *input_rel)
 {
-	Relids		relids = joinrel->relids;
+	Relids		relids;
 	ListCell   *vars;
 
+	/* attrs_needed refers to parent relids and not those of a child. */
+	if (joinrel->top_parent_relids)
+		relids = joinrel->top_parent_relids;
+	else
+		relids = joinrel->relids;
+
 	foreach(vars, input_rel->reltarget->exprs)
 	{
 		Var		   *var = (Var *) lfirst(vars);
@@ -667,23 +804,47 @@ build_joinrel_tlist(PlannerInfo *root, RelOptInfo *joinrel,
 
 		/*
 		 * Otherwise, anything in a baserel or joinrel targetlist ought to be
-		 * a Var.  (More general cases can only appear in appendrel child
-		 * rels, which will never be seen here.)
+		 * a Var or ConvertRowtypeExpr introduced while translating parent
+		 * targetlist to that of the child.
 		 */
-		if (!IsA(var, Var))
+		if (IsA(var, Var))
+		{
+			/* Get the Var's original base rel */
+			baserel = find_base_rel(root, var->varno);
+
+			/* Is it still needed above this joinrel? */
+			ndx = var->varattno - baserel->min_attr;
+		}
+		else if (IsA(var, ConvertRowtypeExpr))
+		{
+			ConvertRowtypeExpr *child_expr = (ConvertRowtypeExpr *) var;
+			Var	 *childvar = (Var *) child_expr->arg;
+
+			/*
+			 * Child's whole-row references are converted to that of parent
+			 * using ConvertRowtypeExpr. In this case, the argument to
+			 * ConvertRowtypeExpr is expected to be a whole-row reference of
+			 * the child.
+			 */
+			Assert(IsA(childvar, Var) && childvar->varattno == 0);
+
+			baserel = find_base_rel(root, childvar->varno);
+			ndx = 0 - baserel->min_attr;
+		}
+		else
 			elog(ERROR, "unexpected node type in rel targetlist: %d",
 				 (int) nodeTag(var));
 
-		/* Get the Var's original base rel */
-		baserel = find_base_rel(root, var->varno);
-
-		/* 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 */
 			joinrel->reltarget->exprs = lappend(joinrel->reltarget->exprs, var);
-			/* Vars have cost zero, so no need to adjust reltarget->cost */
+
+			/*
+			 * Vars have cost zero, so no need to adjust reltarget->cost. Even
+			 * if, it's a ConvertRowtypeExpr, it will be computed only for the
+			 * base relation, costing nothing for a join.
+			 */
 			joinrel->reltarget->width += baserel->attr_widths[ndx];
 		}
 	}
@@ -820,6 +981,9 @@ subbuild_joinrel_joinlist(RelOptInfo *joinrel,
 {
 	ListCell   *l;
 
+	/* Expected to be called only for join between parent relations. */
+	Assert(joinrel->reloptkind == RELOPT_JOINREL);
+
 	foreach(l, joininfo_list)
 	{
 		RestrictInfo *rinfo = (RestrictInfo *) lfirst(l);
@@ -1366,3 +1530,85 @@ find_param_path_info(RelOptInfo *rel, Relids required_outer)
 
 	return NULL;
 }
+
+/*
+ * build_joinrel_partition_info
+ *		If the join between given partitioned relations is possibly partitioned
+ *		set the partitioning scheme and partition keys expressions for the
+ *		join.
+ *
+ * If the two relations have same partitioning scheme, their join may be
+ * partitioned and will follow the same partitioning scheme as the joining
+ * relations.
+ */
+static void
+build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
+							 RelOptInfo *inner_rel, JoinType jointype)
+{
+	int		num_pks;
+	int		cnt;
+
+	/* Nothing to do if partition-wise join technique is disabled. */
+	if (!enable_partition_wise_join)
+	{
+		joinrel->part_scheme = NULL;
+		return;
+	}
+
+	/*
+	 * The join is not partitioned, if any of the relations being joined are
+	 * not partitioned or they do not have same partitioning scheme.
+	 */
+	if (!outer_rel->part_scheme || !inner_rel->part_scheme ||
+		outer_rel->part_scheme != inner_rel->part_scheme)
+	{
+		joinrel->part_scheme = NULL;
+		return;
+	}
+
+	/*
+	 * This function will be called only once for each joinrel, hence it should
+	 * not have partition scheme, partition key expressions and array for
+	 * storing child relations set.
+	 */
+	Assert(!joinrel->part_scheme && !joinrel->partexprs &&
+		   !joinrel->part_rels);
+
+	/*
+	 * Join relation is partitioned using same partitioning scheme as the
+	 * joining relations.
+	 */
+	joinrel->part_scheme = outer_rel->part_scheme;
+	num_pks = joinrel->part_scheme->partnatts;
+
+	/*
+	 * Construct partition keys for the join.
+	 *
+	 * An INNER join between two partitioned relations is partition by key
+	 * expressions from both the relations. For tables A and B partitioned by a and b
+	 * respectively, (A INNER JOIN B ON A.a = B.b) is partitioned by both A.a
+	 * and B.b.
+	 *
+	 * An OUTER join like (A LEFT JOIN B ON A.a = B.b) may produce rows with
+	 * B.b NULL. These rows may not fit the partitioning conditions imposed on
+	 * B.b. Hence, strictly speaking, the join is not partitioned by B.b.
+	 * Strictly speaking, partition keys of an OUTER join should include
+	 * partition key expressions from the OUTER side only. Consider a join like
+	 * (A LEFT JOIN B on (A.a = B.b) LEFT JOIN C ON B.b = C.c. If we do not
+	 * include B.b as partition key expression for (AB), it prohibits us from
+	 * using partition-wise join when joining (AB) with C as there is no
+	 * equi-join between partition keys of joining relations. But two NULL
+	 * values are never equal and no two rows from mis-matching partitions can
+	 * join. Hence it's safe to include B.b as partition key expression for
+	 * (AB), even though rows in (AB) are not strictly partitioned by B.b.
+	 */
+	joinrel->partexprs = (List **) palloc0(sizeof(List *) * num_pks);
+	for (cnt = 0; cnt < num_pks; cnt++)
+	{
+		List *pkexpr = list_copy(outer_rel->partexprs[cnt]);
+
+		pkexpr = list_concat(pkexpr,
+							 list_copy(inner_rel->partexprs[cnt]));
+		joinrel->partexprs[cnt] = pkexpr;
+	}
+}
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index de85eca..afd0c23 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -901,6 +901,15 @@ static struct config_bool ConfigureNamesBool[] =
 		true,
 		NULL, NULL, NULL
 	},
+	{
+		{"enable_partition_wise_join", PGC_USERSET, QUERY_TUNING_METHOD,
+			gettext_noop("Enables partition-wise join."),
+			NULL
+		},
+		&enable_partition_wise_join,
+		true,
+		NULL, NULL, NULL
+	},
 
 	{
 		{"geqo", PGC_USERSET, QUERY_TUNING_GEQO,
@@ -2947,6 +2956,25 @@ static struct config_real ConfigureNamesReal[] =
 	},
 
 	{
+		{"partition_wise_plan_weight", PGC_USERSET, QUERY_TUNING_OTHER,
+			gettext_noop("Multiplication factor for partition-wise plan costs."),
+			NULL
+		},
+		&partition_wise_plan_weight,
+		DEFAULT_PARTITION_WISE_PLAN_WEIGHT, 0, DBL_MAX,
+		NULL, NULL, NULL
+	},
+	{
+		{"sample_partition_fraction", PGC_USERSET, QUERY_TUNING_OTHER,
+			gettext_noop("Fraction of partitions to be used as sample for calculating total cost of partition-wise plans."),
+			NULL
+		},
+		&sample_partition_fraction,
+		DEFAULT_SAMPLE_PARTITION_FRACTION, 0, 1,
+		NULL, NULL, NULL
+	},
+
+	{
 		{"bgwriter_lru_multiplier", PGC_SIGHUP, RESOURCES_BGWRITER,
 			gettext_noop("Multiple of the average buffer usage to free per round."),
 			NULL
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index 95dd8ba..292d9a6 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -240,6 +240,7 @@ typedef enum NodeTag
 	T_NestPath,
 	T_MergePath,
 	T_HashPath,
+	T_PartitionJoinPath,
 	T_AppendPath,
 	T_MergeAppendPath,
 	T_ResultPath,
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 4f99184..146d53b 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -391,6 +391,11 @@ typedef struct PartitionSchemeData *PartitionScheme;
  * handling join alias Vars.  Currently this is not needed because all join
  * alias Vars are expanded to non-aliased form during preprocess_expression.
  *
+ * We also have relations representing joins between child relations of
+ * different partitioned tables. These relations are not added to
+ * join_rel_level lists as they are not joined directly by the dynamic
+ * programming algorithm.
+ *
  * There is also a RelOptKind for "upper" relations, which are RelOptInfos
  * that describe post-scan/join processing steps, such as aggregation.
  * Many of the fields in these RelOptInfos are meaningless, but their Path
@@ -512,10 +517,19 @@ typedef enum RelOptKind
 	RELOPT_BASEREL,
 	RELOPT_JOINREL,
 	RELOPT_OTHER_MEMBER_REL,
+	RELOPT_OTHER_JOINREL,
 	RELOPT_UPPER_REL,
 	RELOPT_DEADREL
 } RelOptKind;
 
+#define IS_OTHER_REL(rel) \
+	((rel)->reloptkind == RELOPT_OTHER_MEMBER_REL || \
+	 (rel)->reloptkind == RELOPT_OTHER_JOINREL)
+
+#define IS_JOIN_REL(rel)	\
+	((rel)->reloptkind == RELOPT_JOINREL || \
+	 (rel)->reloptkind == RELOPT_OTHER_JOINREL)
+
 typedef struct RelOptInfo
 {
 	NodeTag		type;
@@ -600,6 +614,14 @@ typedef struct RelOptInfo
 										 * as the number of joining
 										 * relations.
 										 */
+
+	/* For joins between partitioned tables. */
+	List			   *partitioned_joins;	/* List of join orders which yield
+											 * relations partitioned by above
+											 * partition scheme.
+											 */
+	/* Set only for "other" base or "other" join relations. */
+	Relids		top_parent_relids;		/* Relids of topmost parents. */
 } RelOptInfo;
 
 /*
@@ -1540,6 +1562,14 @@ typedef struct LimitPath
 	Node	   *limitCount;		/* COUNT parameter, or NULL if none */
 } LimitPath;
 
+/*
+ * PartitionJoinPath represents partition-wise join between two partitioned
+ * tables.
+ */
+typedef struct PartitionJoinPath
+{
+	Path		path;
+} PartitionJoinPath;
 
 /*
  * Restriction clause info.
@@ -1747,6 +1777,18 @@ typedef struct RestrictInfo
 	/* cache space for hashclause processing; -1 if not yet set */
 	Selectivity left_bucketsize;	/* avg bucketsize of left side */
 	Selectivity right_bucketsize;		/* avg bucketsize of right side */
+
+	/*
+	 * Repository to locate child RestrictInfos derived from parent
+	 * RestrictInfo. Every derived child RestrictInfo points to the parent
+	 * RestrictInfo from which it is derived. Parent RestrictInfo maintains a
+	 * list of all derived child RestrictInfos. So only one of the following
+	 * should be set.
+	 */
+	List	   *child_rinfos;		/* RestrictInfos derived for children. */
+	struct RestrictInfo *parent_rinfo;		/* Parent restrictinfo this
+											 * RestrictInf is derived from.
+											 */
 } RestrictInfo;
 
 /*
@@ -1869,6 +1911,21 @@ typedef struct SpecialJoinInfo
 } SpecialJoinInfo;
 
 /*
+ * Partitioned join information
+ *
+ * Saves information about relations which can be joined partition-wise and
+ * thus produce result which is partitioned by the partition scheme of the
+ * relation.
+ */
+typedef struct PartitionedJoin
+{
+	RelOptInfo	   *rel1;
+	RelOptInfo	   *rel2;
+	SpecialJoinInfo	   *sjinfo;	/* SpecialJoinInfo applicable. */
+	List		   *restrictlist;	/* applicable join clauses. */
+} PartitionedJoin;
+
+/*
  * Append-relation info.
  *
  * When we expand an inheritable table or a UNION-ALL subselect into an
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 0e68264..a13eff1 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -30,6 +30,8 @@
 #define DEFAULT_PARALLEL_SETUP_COST  1000.0
 
 #define DEFAULT_EFFECTIVE_CACHE_SIZE  524288	/* measured in pages */
+#define DEFAULT_PARTITION_WISE_PLAN_WEIGHT 1
+#define DEFAULT_SAMPLE_PARTITION_FRACTION 0.01
 
 typedef enum
 {
@@ -66,7 +68,10 @@ extern bool enable_nestloop;
 extern bool enable_material;
 extern bool enable_mergejoin;
 extern bool enable_hashjoin;
+extern bool enable_partition_wise_join;
 extern int	constraint_exclusion;
+extern double partition_wise_plan_weight;
+extern double sample_partition_fraction;
 
 extern double clamp_row_est(double nrows);
 extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 7b41317..81d637a 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -229,6 +229,9 @@ extern LimitPath *create_limit_path(PlannerInfo *root, RelOptInfo *rel,
 				  Path *subpath,
 				  Node *limitOffset, Node *limitCount,
 				  int64 offset_est, int64 count_est);
+extern PartitionJoinPath *create_partition_join_path(PlannerInfo *root,
+								RelOptInfo *rel, List *subpaths,
+								Bitmapset *required_outer, List *pathkeys);
 
 extern Path *reparameterize_path(PlannerInfo *root, Path *path,
 					Relids required_outer,
@@ -271,5 +274,8 @@ extern ParamPathInfo *get_joinrel_parampathinfo(PlannerInfo *root,
 						  List **restrict_clauses);
 extern ParamPathInfo *get_appendrel_parampathinfo(RelOptInfo *appendrel,
 							Relids required_outer);
+extern RelOptInfo *build_child_join_rel(PlannerInfo *root,
+						 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
+						 RelOptInfo *parent_joinrel, JoinType jointype);
 
 #endif   /* PATHNODE_H */
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 81a9be7..821fe36 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -53,6 +53,10 @@ extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 					 List *initial_rels);
 
 extern void generate_gather_paths(PlannerInfo *root, RelOptInfo *rel);
+extern void generate_partition_wise_join_paths(PlannerInfo *root,
+											   RelOptInfo *rel);
+extern Path *get_cheapest_parameterized_child_path(PlannerInfo *root,
+									   RelOptInfo *rel, Relids required_outer);
 
 #ifdef OPTIMIZER_DEBUG
 extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
@@ -106,6 +110,9 @@ extern bool have_join_order_restriction(PlannerInfo *root,
 							RelOptInfo *rel1, RelOptInfo *rel2);
 extern bool have_dangerous_phv(PlannerInfo *root,
 				   Relids outer_relids, Relids inner_params);
+extern void add_paths_to_child_joinrel(PlannerInfo *root,
+									   RelOptInfo *parent_joinrel,
+									   int child_id);
 
 /*
  * equivclass.c
diff --git a/src/include/optimizer/placeholder.h b/src/include/optimizer/placeholder.h
index 11e6403..8598268 100644
--- a/src/include/optimizer/placeholder.h
+++ b/src/include/optimizer/placeholder.h
@@ -28,5 +28,7 @@ extern void fix_placeholder_input_needed_levels(PlannerInfo *root);
 extern void add_placeholders_to_base_rels(PlannerInfo *root);
 extern void add_placeholders_to_joinrel(PlannerInfo *root, RelOptInfo *joinrel,
 							RelOptInfo *outer_rel, RelOptInfo *inner_rel);
+extern void add_placeholders_to_child_joinrel(PlannerInfo *root,
+							RelOptInfo *childrel, RelOptInfo *parentrel);
 
 #endif   /* PLACEHOLDER_H */
diff --git a/src/include/optimizer/prep.h b/src/include/optimizer/prep.h
index a02e06a..5832130 100644
--- a/src/include/optimizer/prep.h
+++ b/src/include/optimizer/prep.h
@@ -57,5 +57,13 @@ extern Node *adjust_appendrel_attrs(PlannerInfo *root, Node *node,
 
 extern Node *adjust_appendrel_attrs_multilevel(PlannerInfo *root, Node *node,
 								  RelOptInfo *child_rel);
+extern RestrictInfo *build_child_restrictinfo(PlannerInfo *root,
+								 RestrictInfo *rinfo, List *append_rel_infos);
+extern List *build_child_clauses(PlannerInfo *root, List *clauses,
+								 List *append_rel_infos);
+extern List *find_appinfos_by_relids(PlannerInfo *root, Relids relids);
+extern SpecialJoinInfo *build_child_join_sjinfo(PlannerInfo *root,
+									SpecialJoinInfo *parent_sjinfo,
+									Relids left_relids, Relids right_relids);
 
 #endif   /* PREP_H */
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index 18238fa..79779d6 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -2,6 +2,10 @@
 -- PARTITION_JOIN
 -- Test partition-wise join between partitioned tables
 --
+-- Usually partition-wise join paths are chosen when data is large, which would
+-- take regression tests to run longer. So, weigh partition-wise joins cheaper
+-- to force those even for smaller data.
+SET partition_wise_plan_weight to 0.2;
 --
 -- partitioned by a single column
 --
diff --git a/src/test/regress/expected/sysviews.out b/src/test/regress/expected/sysviews.out
index d48abd7..c6c1405 100644
--- a/src/test/regress/expected/sysviews.out
+++ b/src/test/regress/expected/sysviews.out
@@ -70,20 +70,21 @@ select count(*) >= 0 as ok from pg_prepared_xacts;
 -- This is to record the prevailing planner enable_foo settings during
 -- a regression test run.
 select name, setting from pg_settings where name like 'enable%';
-         name         | setting 
-----------------------+---------
- enable_bitmapscan    | on
- enable_hashagg       | on
- enable_hashjoin      | on
- enable_indexonlyscan | on
- enable_indexscan     | on
- enable_material      | on
- enable_mergejoin     | on
- enable_nestloop      | on
- enable_seqscan       | on
- enable_sort          | on
- enable_tidscan       | on
-(11 rows)
+            name            | setting 
+----------------------------+---------
+ enable_bitmapscan          | on
+ enable_hashagg             | on
+ enable_hashjoin            | on
+ enable_indexonlyscan       | on
+ enable_indexscan           | on
+ enable_material            | on
+ enable_mergejoin           | on
+ enable_nestloop            | on
+ enable_partition_wise_join | on
+ enable_seqscan             | on
+ enable_sort                | on
+ enable_tidscan             | on
+(12 rows)
 
 -- Test that the pg_timezone_names and pg_timezone_abbrevs views are
 -- more-or-less working.  We can't test their contents in any great detail
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index 0322f1e..9b2baeb 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -3,6 +3,11 @@
 -- Test partition-wise join between partitioned tables
 --
 
+-- Usually partition-wise join paths are chosen when data is large, which would
+-- take regression tests to run longer. So, weigh partition-wise joins cheaper
+-- to force those even for smaller data.
+SET partition_wise_plan_weight to 0.2;
+
 --
 -- partitioned by a single column
 --
-- 
1.7.9.5

0009-Adjust-join-related-to-code-to-accept-child-relation.patchapplication/octet-stream; name=0009-Adjust-join-related-to-code-to-accept-child-relation.patchDownload
From 92d2901e80ca783ad9a7a9e0fda24684e7433409 Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Tue, 7 Feb 2017 17:16:46 +0530
Subject: [PATCH 09/14] Adjust join related to code to accept child relations.

Existing join related code doesn't expect child relations to be joined. This
patch contains various fixes to change that.

1. Uniqe-ifying joining relations.
=================================
For semi-joins we unique-ify the joining relations, which tries to estimate
nummber of unique values using estimate_num_groups(). This function doesn't
expect a Var from a child relation and contained an assertion to that effect.
With partition-wise joins, we may compute a join between child relations. This
commit changes that assertion to include child relation. The function doesn't
need any change other than that to accomodate child relations.

2. OUTER joins require dummy child relations to have targetlist.
================================================================
We need a targetlist defining nullable columns for an outer join, even if the
relation on the nullable side is deemed to be empty. Prior to partition-wise
join an empty child relation never had a targetlist since it was eliminated
from planning. But with partition-wise join an empty child relation may
participate in an outer join with another non-empty child relation. Hence set
targetlist for a child relation even if it's dummy.

3. prepare_sort_from_pathkeys fixes.
====================================
Before partition-wise join feature were never required to be directly sorted,
let's say for merge joins. With partition-wise join feature, the child
relations will participate directly in the join and also need to be sorted
directly for the purpose of merge join. In order to sort a relation, we use
pathkeys. The expression on which to sort a particular relation is provided by
the equivalence member corresponding to that relation in the equivalence class
referred by the pathkeys. Since the code doesn't expect child relations to
bubble up to the sorting, the function prepare_sort_from_pathkeys() skips any
child members (those set with em_is_child) unless the caller specifically asks
for child relations by passing relids. make_sort_from_pathkeys() calls
prepare_sort_from_pathkeys() to create Sort plan for outer and inner plans
without passing relids of the relation to be sorted. For partition-wise joins
the outer and inner plans produce child relations and thus
prepare_sort_from_pathkeys() does not find equivalence members since it skips
child members for the want of relids. This particular instance can be fixed by
passing outer/inner_path->parent->relids to prepare_sort_from_pathkeys().

All the callers of prepare_sort_from_pathkeys() viz.
create_merge_append_plan(), create_merge_append_plan(),
create_windowagg_plan() except make_sort_from_pathkeys() pass relids to
prepare_sort_from_pathkeys(). make_sort_from_pathkeys() as well passes those
with this patch.

make_sort_from_pathkeys() itself doesn't know the relids of relation being
sorted. It just gets the plan. Hence we need to pass relids to
make_sort_from_pathkeys() and thus change each of its callers to pass relids,
if required.

It has two callers as of now.
1. create_sort_plan(PlannerInfo *root, SortPath *best_path, int flags): does
not handle child relations yet, so doesn't need any change.
2. create_mergejoin_plan(PlannerInfo *root, MergePath *best_path):
It requires this change and the relids can be obtained from the outer and inner
path's parent RelOptInfo.

4. Handling em_is_child cases.
==============================
Right now, when comparing relids for child relations, only exact match is
considered. This is fine as long as em_relids has only a single member in it
and the passed in relids has only a single member in it. But with
partition-wise join, relids can have multiple members and em_relids may not
exactly match the given relids. But we need to find the member which covers
subset of given relids.
---
 src/backend/optimizer/path/allpaths.c   |   41 +++++++++++++++++--------------
 src/backend/optimizer/plan/createplan.c |   28 +++++++++++++--------
 src/backend/utils/adt/selfuncs.c        |    3 ++-
 3 files changed, 42 insertions(+), 30 deletions(-)

diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index cebf359..0401269 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -957,11 +957,27 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		}
 
 		/*
-		 * We have to copy the parent's targetlist and quals to the child,
-		 * with appropriate substitution of variables.  However, only the
-		 * baserestrictinfo quals are needed before we can check for
-		 * constraint exclusion; so do that first and then check to see if we
-		 * can disregard this child.
+		 * Copy/Modify targetlist. Even if this child is deemed empty, we need
+		 * its targetlist in case it falls on nullable side in a child-join
+		 * because of partition-wise join.
+		 *
+		 * NB: the resulting childrel->reltarget->exprs may contain arbitrary
+		 * expressions, which otherwise would not occur in a rel's targetlist.
+		 * Code that might be looking at an appendrel child must cope with
+		 * such.  (Normally, a rel's targetlist would only include Vars and
+		 * PlaceHolderVars.)  XXX we do not bother to update the cost or width
+		 * fields of childrel->reltarget; not clear if that would be useful.
+		 */
+		childrel->reltarget->exprs = (List *)
+			adjust_appendrel_attrs(root,
+								   (Node *) rel->reltarget->exprs,
+								   appinfo_list);
+
+		/*
+		 * We have to copy the parent's quals to the child, with appropriate
+		 * substitution of variables.  However, only the baserestrictinfo quals
+		 * are needed before we can check for constraint exclusion; so do that
+		 * first and then check to see if we can disregard this child.
 		 *
 		 * The child rel's targetlist might contain non-Var expressions, which
 		 * means that substitution into the quals could produce opportunities
@@ -1088,22 +1104,9 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 			continue;
 		}
 
-		/*
-		 * CE failed, so finish copying/modifying targetlist and join quals.
-		 *
-		 * NB: the resulting childrel->reltarget->exprs may contain arbitrary
-		 * expressions, which otherwise would not occur in a rel's targetlist.
-		 * Code that might be looking at an appendrel child must cope with
-		 * such.  (Normally, a rel's targetlist would only include Vars and
-		 * PlaceHolderVars.)  XXX we do not bother to update the cost or width
-		 * fields of childrel->reltarget; not clear if that would be useful.
-		 */
+		/* CE failed, so finish copying/modifying targetlist and join quals. */
 		childrel->joininfo = build_child_clauses(root, rel->joininfo,
 												 appinfo_list);
-		childrel->reltarget->exprs = (List *)
-			adjust_appendrel_attrs(root,
-								   (Node *) rel->reltarget->exprs,
-								   appinfo_list);
 
 		/*
 		 * We have to make child entries in the EquivalenceClass data
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index c1f9da2..3f1f2c1 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -245,7 +245,8 @@ static Plan *prepare_sort_from_pathkeys(Plan *lefttree, List *pathkeys,
 static EquivalenceMember *find_ec_member_for_tle(EquivalenceClass *ec,
 					   TargetEntry *tle,
 					   Relids relids);
-static Sort *make_sort_from_pathkeys(Plan *lefttree, List *pathkeys);
+static Sort *make_sort_from_pathkeys(Plan *lefttree, List *pathkeys,
+									 Relids relids);
 static Sort *make_sort_from_groupcols(List *groupcls,
 						 AttrNumber *grpColIdx,
 						 Plan *lefttree);
@@ -1555,7 +1556,7 @@ create_sort_plan(PlannerInfo *root, SortPath *best_path, int flags)
 	subplan = create_plan_recurse(root, best_path->subpath,
 								  flags | CP_SMALL_TLIST);
 
-	plan = make_sort_from_pathkeys(subplan, best_path->path.pathkeys);
+	plan = make_sort_from_pathkeys(subplan, best_path->path.pathkeys, NULL);
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
@@ -3572,6 +3573,8 @@ create_mergejoin_plan(PlannerInfo *root,
 	ListCell   *lc;
 	ListCell   *lop;
 	ListCell   *lip;
+	Path	   *outer_path = best_path->jpath.outerjoinpath;
+	Path	   *inner_path = best_path->jpath.innerjoinpath;
 
 	/*
 	 * MergeJoin can project, so we don't have to demand exact tlists from the
@@ -3635,8 +3638,10 @@ create_mergejoin_plan(PlannerInfo *root,
 	 */
 	if (best_path->outersortkeys)
 	{
+		Relids		outer_relids = outer_path->parent->relids;
 		Sort	   *sort = make_sort_from_pathkeys(outer_plan,
-												   best_path->outersortkeys);
+												   best_path->outersortkeys,
+												   outer_relids);
 
 		label_sort_with_costsize(root, sort, -1.0);
 		outer_plan = (Plan *) sort;
@@ -3647,8 +3652,10 @@ create_mergejoin_plan(PlannerInfo *root,
 
 	if (best_path->innersortkeys)
 	{
+		Relids		inner_relids = inner_path->parent->relids;
 		Sort	   *sort = make_sort_from_pathkeys(inner_plan,
-												   best_path->innersortkeys);
+												   best_path->innersortkeys,
+												   inner_relids);
 
 		label_sort_with_costsize(root, sort, -1.0);
 		inner_plan = (Plan *) sort;
@@ -5631,11 +5638,11 @@ prepare_sort_from_pathkeys(Plan *lefttree, List *pathkeys,
 					continue;
 
 				/*
-				 * Ignore child members unless they match the rel being
+				 * Ignore child members unless they belong to the rel being
 				 * sorted.
 				 */
 				if (em->em_is_child &&
-					!bms_equal(em->em_relids, relids))
+					!bms_is_subset(em->em_relids, relids))
 					continue;
 
 				sortexpr = em->em_expr;
@@ -5746,10 +5753,10 @@ find_ec_member_for_tle(EquivalenceClass *ec,
 			continue;
 
 		/*
-		 * Ignore child members unless they match the rel being sorted.
+		 * Ignore child members unless they belong to the rel being sorted.
 		 */
 		if (em->em_is_child &&
-			!bms_equal(em->em_relids, relids))
+			!bms_is_subset(em->em_relids, relids))
 			continue;
 
 		/* Match if same expression (after stripping relabel) */
@@ -5770,9 +5777,10 @@ find_ec_member_for_tle(EquivalenceClass *ec,
  *
  *	  'lefttree' is the node which yields input tuples
  *	  'pathkeys' is the list of pathkeys by which the result is to be sorted
+ *	  'relids' is the set of relations required by prepare_sort_from_pathkeys()
  */
 static Sort *
-make_sort_from_pathkeys(Plan *lefttree, List *pathkeys)
+make_sort_from_pathkeys(Plan *lefttree, List *pathkeys, Relids relids)
 {
 	int			numsortkeys;
 	AttrNumber *sortColIdx;
@@ -5782,7 +5790,7 @@ make_sort_from_pathkeys(Plan *lefttree, List *pathkeys)
 
 	/* Compute sort column info, and adjust lefttree as needed */
 	lefttree = prepare_sort_from_pathkeys(lefttree, pathkeys,
-										  NULL,
+										  relids,
 										  NULL,
 										  false,
 										  &numsortkeys,
diff --git a/src/backend/utils/adt/selfuncs.c b/src/backend/utils/adt/selfuncs.c
index fa32e9e..c833846 100644
--- a/src/backend/utils/adt/selfuncs.c
+++ b/src/backend/utils/adt/selfuncs.c
@@ -3427,7 +3427,8 @@ estimate_num_groups(PlannerInfo *root, List *groupExprs, double input_rows,
 		/*
 		 * Sanity check --- don't divide by zero if empty relation.
 		 */
-		Assert(rel->reloptkind == RELOPT_BASEREL);
+		Assert(rel->reloptkind == RELOPT_BASEREL ||
+			   rel->reloptkind == RELOPT_OTHER_MEMBER_REL);
 		if (rel->tuples > 0)
 		{
 			/*
-- 
1.7.9.5

0010-Parameterized-path-fixes.patchapplication/octet-stream; name=0010-Parameterized-path-fixes.patchDownload
From 633152ec443469a5a6611f7005db2b52474a60f7 Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Tue, 7 Feb 2017 17:25:25 +0530
Subject: [PATCH 10/14] Parameterized path fixes.

We do not create merge or hash join paths when the inner path is parameterized
by the outer and vice-versa. Parameterization information in path refers to the
top-most parent relation. Current tests (PATH_PARAM_BY_REL) to avoid joining
such paths fail while joining child relations; the paths from either child may
be paramterized by other's parent. Modify the tests to consider paths
parameterized by parent as parameterized by any of its child.

If the inner path is parameterized by outer path, we can create a nested loop
join using those two paths with inner relation parameterized by the outer
relation. For LATERAL JOINs this is the only legal way to plan a join. In case
of partitioned joins, the lateral references refer to the topmost parent and
hence inner paths are parameterized by the topmost parent. In such cases, it's
possible to translate the inner path to be parameterized by
the child and create nested loop join. When presented with a pair of child
relation paths, where the inner paths is parameterized by the parent of outer
child, this patch translates the path to be parameterized by the outer child
and creates a nested loop join path.

The function reparameterize_path_by_child() needs to call adjust_relid_set() to
substitute parent relids by child relids in Path::param_info::ppi_req_outer.
Hence "extern"alized that function. Since there is already another
static adjust_relid_set() in rewriteManip.c, renamed this one to
adjust_child_relids().

Also "extern"alized find_param_path_info() required by
reparameterize_path_by_child().
---
 src/backend/optimizer/path/joinpath.c  |   33 +++++-
 src/backend/optimizer/prep/prepunion.c |   42 ++++----
 src/backend/optimizer/util/pathnode.c  |  182 ++++++++++++++++++++++++++++++++
 src/backend/optimizer/util/relnode.c   |    2 -
 src/include/optimizer/pathnode.h       |    4 +
 src/include/optimizer/prep.h           |    1 +
 6 files changed, 239 insertions(+), 25 deletions(-)

diff --git a/src/backend/optimizer/path/joinpath.c b/src/backend/optimizer/path/joinpath.c
index f80fb25..4d4a183 100644
--- a/src/backend/optimizer/path/joinpath.c
+++ b/src/backend/optimizer/path/joinpath.c
@@ -25,9 +25,19 @@
 /* Hook for plugins to get control in add_paths_to_joinrel() */
 set_join_pathlist_hook_type set_join_pathlist_hook = NULL;
 
-#define PATH_PARAM_BY_REL(path, rel)  \
+/*
+ * Paths parameterized by the parent can be considered to be parameterized by
+ * any of its child.
+ */
+#define PATH_PARAM_BY_PARENT(path, rel)	\
+	((path)->param_info && bms_overlap(PATH_REQ_OUTER(path),	\
+									   (rel)->top_parent_relids))
+#define PATH_PARAM_BY_REL_SELF(path, rel)  \
 	((path)->param_info && bms_overlap(PATH_REQ_OUTER(path), (rel)->relids))
 
+#define PATH_PARAM_BY_REL(path, rel)	\
+	(PATH_PARAM_BY_REL_SELF(path, rel) || PATH_PARAM_BY_PARENT(path, rel))
+
 static void sort_inner_and_outer(PlannerInfo *root, RelOptInfo *joinrel,
 					 RelOptInfo *outerrel, RelOptInfo *innerrel,
 					 JoinType jointype, JoinPathExtraData *extra);
@@ -301,6 +311,27 @@ try_nestloop_path(PlannerInfo *root,
 	JoinCostWorkspace workspace;
 
 	/*
+	 * Since result produced by a child is part of the result produced by its
+	 * topmost parent and has same properties, the parameters representing that
+	 * parent may be substituted by values from a child. Hence expressions and
+	 * hence paths using those expressions, parameterized by a parent can be
+	 * said to be parameterized by any of its child.  For a join between child
+	 * relations, if the inner path is parameterized by the parent of the outer
+	 * relation,  create a nestloop join path with inner relation parameterized
+	 * by the outer relation by translating the inner path to be parameterized
+	 * by the outer child relation.
+	 */
+	if (PATH_PARAM_BY_PARENT(inner_path, outer_path->parent))
+	{
+		inner_path = reparameterize_path_by_child(root, inner_path,
+												   outer_path->parent);
+
+		/* If we could not translate the path, don't produce nest loop path. */
+		if (!inner_path)
+			return;
+	}
+
+	/*
 	 * Check to see if proposed path is still parameterized, and reject if the
 	 * parameterization wouldn't be sensible --- unless allow_star_schema_join
 	 * says to allow it anyway.  Also, we must reject if have_dangerous_phv
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index 676204f..d459e95 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -109,7 +109,6 @@ static Node *adjust_appendrel_attrs_mutator(Node *node,
 							   adjust_appendrel_attrs_context *context);
 static List *adjust_inherited_tlist(List *tlist,
 					   AppendRelInfo *context);
-static Relids adjust_relid_set(Relids relids, List *append_rel_infos);
 
 
 /*
@@ -1951,7 +1950,7 @@ adjust_appendrel_attrs_mutator(Node *node,
 														 (void *) context);
 		/* now fix PlaceHolderVar's relid sets */
 		if (phv->phlevelsup == 0)
-			phv->phrels = adjust_relid_set(phv->phrels, context->appinfos);
+			phv->phrels = adjust_child_relids(phv->phrels, context->appinfos);
 		return (Node *) phv;
 	}
 	/* Shouldn't need to handle planner auxiliary nodes here */
@@ -1982,17 +1981,17 @@ adjust_appendrel_attrs_mutator(Node *node,
 			adjust_appendrel_attrs_mutator((Node *) oldinfo->orclause, context);
 
 		/* adjust relid sets too */
-		newinfo->clause_relids = adjust_relid_set(oldinfo->clause_relids,
+		newinfo->clause_relids = adjust_child_relids(oldinfo->clause_relids,
 													 context->appinfos);
-		newinfo->required_relids = adjust_relid_set(oldinfo->required_relids,
+		newinfo->required_relids = adjust_child_relids(oldinfo->required_relids,
 													 context->appinfos);
-		newinfo->outer_relids = adjust_relid_set(oldinfo->outer_relids,
+		newinfo->outer_relids = adjust_child_relids(oldinfo->outer_relids,
 													 context->appinfos);
-		newinfo->nullable_relids = adjust_relid_set(oldinfo->nullable_relids,
+		newinfo->nullable_relids = adjust_child_relids(oldinfo->nullable_relids,
 													   context->appinfos);
-		newinfo->left_relids = adjust_relid_set(oldinfo->left_relids,
+		newinfo->left_relids = adjust_child_relids(oldinfo->left_relids,
 												   context->appinfos);
-		newinfo->right_relids = adjust_relid_set(oldinfo->right_relids,
+		newinfo->right_relids = adjust_child_relids(oldinfo->right_relids,
 													context->appinfos);
 
 		/*
@@ -2026,15 +2025,18 @@ adjust_appendrel_attrs_mutator(Node *node,
 
 /*
  * Replace parent relids by child relids in the copy of given relid set
- * according to the given list of AppendRelInfos. The given relid set is
- * returned as is if it contains no parent in the given list, otherwise, the
- * given relid set is not changed.
+ * according to the given list of AppendRelInfos.
  */
 Relids
-adjust_relid_set(Relids relids, List *append_rel_infos)
+adjust_child_relids(Relids relids, List *append_rel_infos)
 {
 	ListCell   *lc;
-	Bitmapset  *result = NULL;
+
+	/*
+	 * The new relids set may be expected to be in a memory context different
+	 * from the given one. Make a copy here.
+	 */
+	Bitmapset  *result = bms_copy(relids);
 
 	foreach (lc, append_rel_infos)
 	{
@@ -2043,10 +2045,6 @@ adjust_relid_set(Relids relids, List *append_rel_infos)
 		/* Remove parent, add child */
 		if (bms_is_member(appinfo->parent_relid, relids))
 		{
-			/* Make a copy if we are changing the set. */
-			if (!result)
-				result = bms_copy(relids);
-
 			result = bms_del_member(result, appinfo->parent_relid);
 			result = bms_add_member(result, appinfo->child_relid);
 		}
@@ -2202,7 +2200,7 @@ build_child_restrictinfo(PlannerInfo *root, RestrictInfo *rinfo,
 	RestrictInfo   *child_rinfo;
 	MemoryContext	old_context;
 
-	child_required_relids = adjust_relid_set(rinfo->required_relids,
+	child_required_relids = adjust_child_relids(rinfo->required_relids,
 												append_rel_infos);
 
 
@@ -2313,13 +2311,13 @@ build_child_join_sjinfo(PlannerInfo *root, SpecialJoinInfo *parent_sjinfo,
 
 	memcpy(sjinfo, parent_sjinfo, sizeof(SpecialJoinInfo));
 
-	sjinfo->min_lefthand = adjust_relid_set(sjinfo->min_lefthand,
+	sjinfo->min_lefthand = adjust_child_relids(sjinfo->min_lefthand,
 											   left_appinfos);
-	sjinfo->min_righthand = adjust_relid_set(sjinfo->min_righthand,
+	sjinfo->min_righthand = adjust_child_relids(sjinfo->min_righthand,
 												right_appinfos);
-	sjinfo->syn_lefthand = adjust_relid_set(sjinfo->syn_lefthand,
+	sjinfo->syn_lefthand = adjust_child_relids(sjinfo->syn_lefthand,
 											   left_appinfos);
-	sjinfo->syn_righthand = adjust_relid_set(sjinfo->syn_righthand,
+	sjinfo->syn_righthand = adjust_child_relids(sjinfo->syn_righthand,
 												right_appinfos);
 
 	/*
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index d861a49..d1ee237 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -3388,3 +3388,185 @@ reparameterize_path(PlannerInfo *root, Path *path,
 	}
 	return NULL;
 }
+
+/*
+ * reparameterize_path_by_child
+ * 		Given a path parameterized by the parent of the given relation,
+ * 		translate the path to be parameterized by the given child relation.
+ *
+ * The function creates a new path of the same type as the given path, but
+ * parameterized by the given child relation. If it can not reparameterize the
+ * path as required, it returns NULL.
+ *
+ * The cost, number of rows, width and parallel path properties depend upon
+ * path->parent, which does not change during the translation. Hence those
+ * members are copied as they are.
+ */
+
+Path *
+reparameterize_path_by_child(PlannerInfo *root, Path *path,
+							  RelOptInfo *child_rel)
+{
+	Path	   *new_path;
+	ParamPathInfo   *new_ppi;
+	ParamPathInfo   *old_ppi;
+	List	   *child_aris;
+	Relids		required_outer;
+
+	/*
+	 * If the path is not parameterized by parent of the given relation or it it
+	 * doesn't need reparameterization.
+	 */
+	if (!path->param_info ||
+		!bms_overlap(PATH_REQ_OUTER(path), child_rel->top_parent_relids))
+	return path;
+
+	switch (nodeTag(path))
+	{
+		case T_Path:
+			new_path = makeNode(Path);
+			memcpy(new_path, path, sizeof(Path));
+			break;
+
+		case T_HashPath:
+			new_path = (Path *) makeNode(HashPath);
+			memcpy(new_path, path, sizeof(HashPath));
+			break;
+
+		case T_MergePath:
+			new_path = (Path *) makeNode(MergePath);
+			memcpy(new_path, path, sizeof(MergePath));
+			break;
+
+		case T_NestPath:
+			new_path = (Path *) makeNode(NestPath);
+			memcpy(new_path, path, sizeof(NestPath));
+			break;
+
+		case T_IndexPath:
+			new_path = (Path *) makeNode(IndexPath);
+			memcpy(new_path, path, sizeof(IndexPath));
+			break;
+
+		case T_AppendPath:
+			new_path = (Path *) makeNode(AppendPath);
+			memcpy(new_path, path, sizeof(AppendPath));
+			break;
+
+		/*
+		 * TODO:
+		 * If this method of translation is fine add more path types here.
+		 */
+
+		default:
+			/* Path type unsupported by this function. */
+			return NULL;
+	}
+
+	/*
+	 * Gather AppendRelInfos of the base partition relations in the outer child
+	 * relation. We need those for translating parent path to that of child by
+	 * substituting parent Var nodes and relids with those of children.
+	 */
+	child_aris = find_appinfos_by_relids(root, child_rel->relids);
+
+	/* Adjust the parameterization information. */
+	old_ppi = new_path->param_info;
+	required_outer = adjust_child_relids(old_ppi->ppi_req_outer, child_aris);
+
+	/* If we already have a PPI for this parameterization, just return it */
+	new_ppi = find_param_path_info(new_path->parent, required_outer);
+
+	/* If not build a new one and link it to the list of PPIs. */
+	if (!new_ppi)
+	{
+		new_ppi = makeNode(ParamPathInfo);
+		new_ppi->ppi_req_outer = required_outer;
+		new_ppi->ppi_rows = old_ppi->ppi_rows;
+		new_ppi->ppi_clauses = build_child_clauses(root, old_ppi->ppi_clauses,
+												   child_aris);
+		new_path->parent->ppilist = lappend(new_path->parent->ppilist, new_ppi);
+	}
+	else
+		bms_free(required_outer);
+
+	new_path->param_info = new_ppi;
+
+	/*
+	 * Adjust the path target if the parent of the outer relation is referenced
+	 * in the targetlist. This can happen when only the parent of outer relation is
+	 * laterally referenced in this relation.
+	 */
+	if (bms_overlap(path->parent->lateral_relids, child_rel->top_parent_relids))
+	{
+		MemoryContext	old_context;
+
+		/*
+		 * Allocate the target in planner's context, since they are copied as
+		 * is from path while creating plans.
+		 */
+		old_context = MemoryContextSwitchTo(root->planner_cxt);
+		new_path->pathtarget = copy_pathtarget(new_path->pathtarget);
+		new_path->pathtarget->exprs = (List *) adjust_appendrel_attrs(root,
+											(Node *) new_path->pathtarget->exprs,
+																	child_aris);
+		MemoryContextSwitchTo(old_context);
+	}
+
+	/*
+	 * Change parameterization of subpaths recursively. Also carry out any
+	 * pathtype specific adjustments.
+	 */
+	switch (nodeTag(path))
+	{
+		case T_HashPath:
+		case T_MergePath:
+		case T_NestPath:
+			{
+				JoinPath *jpath = (JoinPath *)new_path;
+
+				jpath->outerjoinpath = reparameterize_path_by_child(root,
+														 jpath->outerjoinpath,
+														 child_rel);
+				jpath->innerjoinpath = reparameterize_path_by_child(root,
+														 jpath->innerjoinpath,
+														 child_rel);
+				jpath->joinrestrictinfo = build_child_clauses(root,
+															  jpath->joinrestrictinfo,
+															  child_aris);
+			}
+			break;
+
+		case T_AppendPath:
+			{
+				AppendPath	*apath = (AppendPath *)new_path;
+				List		*subpaths = NIL;
+				ListCell	*lc;
+
+				foreach (lc, apath->subpaths)
+					subpaths = lappend(subpaths,
+									   reparameterize_path_by_child(root,
+																	lfirst(lc),
+																	child_rel));
+				apath->subpaths = subpaths;
+			}
+			break;
+
+		case T_IndexPath:
+			{
+				IndexPath *ipath = (IndexPath *)new_path;
+
+				ipath->indexclauses = build_child_clauses(root, ipath->indexclauses,
+														  child_aris);
+				ipath->indexquals = build_child_clauses(root, ipath->indexquals,
+														child_aris);
+			}
+			break;
+
+		default:
+			/* Nothing to do. */
+			break;
+	}
+
+	return new_path;
+}
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 1eed987..46eea02 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -53,8 +53,6 @@ static List *subbuild_joinrel_joinlist(RelOptInfo *joinrel,
 static void set_foreign_rel_properties(RelOptInfo *joinrel,
 						   RelOptInfo *outer_rel, RelOptInfo *inner_rel);
 static void add_join_rel(PlannerInfo *root, RelOptInfo *joinrel);
-extern ParamPathInfo *find_param_path_info(RelOptInfo *rel,
-									  Relids required_outer);
 static void build_joinrel_partition_info(RelOptInfo *joinrel,
 						  RelOptInfo *outer_rel, RelOptInfo *inner_rel,
 						  JoinType jointype);
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 81d637a..b9f5b11 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -236,6 +236,8 @@ extern PartitionJoinPath *create_partition_join_path(PlannerInfo *root,
 extern Path *reparameterize_path(PlannerInfo *root, Path *path,
 					Relids required_outer,
 					double loop_count);
+extern Path *reparameterize_path_by_child(PlannerInfo *root, Path *path,
+					RelOptInfo *child_rel);
 
 /*
  * prototypes for relnode.c
@@ -277,5 +279,7 @@ extern ParamPathInfo *get_appendrel_parampathinfo(RelOptInfo *appendrel,
 extern RelOptInfo *build_child_join_rel(PlannerInfo *root,
 						 RelOptInfo *outer_rel, RelOptInfo *inner_rel,
 						 RelOptInfo *parent_joinrel, JoinType jointype);
+extern ParamPathInfo *find_param_path_info(RelOptInfo *rel,
+									  Relids required_outer);
 
 #endif   /* PATHNODE_H */
diff --git a/src/include/optimizer/prep.h b/src/include/optimizer/prep.h
index 5832130..0347b37 100644
--- a/src/include/optimizer/prep.h
+++ b/src/include/optimizer/prep.h
@@ -65,5 +65,6 @@ extern List *find_appinfos_by_relids(PlannerInfo *root, Relids relids);
 extern SpecialJoinInfo *build_child_join_sjinfo(PlannerInfo *root,
 									SpecialJoinInfo *parent_sjinfo,
 									Relids left_relids, Relids right_relids);
+extern Relids adjust_child_relids(Relids relids, List *append_rel_infos);
 
 #endif   /* PREP_H */
-- 
1.7.9.5

0011-Use-IS_JOIN_REL-instead-of-RELOPT_JOINREL.patchapplication/octet-stream; name=0011-Use-IS_JOIN_REL-instead-of-RELOPT_JOINREL.patchDownload
From bdcf020d6b0708901949476d5ce59622979d566c Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Wed, 8 Feb 2017 14:42:15 +0530
Subject: [PATCH 11/14] Use IS_JOIN_REL() instead of RELOPT_JOINREL

FDW code uses RELOPT_JOINREL to check whether a given relation is a join or
not. Partition-wise joins create child-join relations, which are marked as
RELOPT_OTHER_JOINREL. Macro IS_JOIN_REL() returns true for both of those kinds,
replace RELOPT_JOINREL tests with IS_JOIN_REL() test.

Similarly replace RELOPT_OTHER_MEMBER_REL test with IS_OTHER_REL() where we
want to test for child relations of all kinds.
---
 contrib/postgres_fdw/deparse.c      |   10 +++++-----
 contrib/postgres_fdw/postgres_fdw.c |   10 ++++++----
 src/backend/foreign/foreign.c       |    6 +++---
 3 files changed, 14 insertions(+), 12 deletions(-)

diff --git a/contrib/postgres_fdw/deparse.c b/contrib/postgres_fdw/deparse.c
index d2b94aa..a2171d7 100644
--- a/contrib/postgres_fdw/deparse.c
+++ b/contrib/postgres_fdw/deparse.c
@@ -911,7 +911,7 @@ deparseSelectStmtForRel(StringInfo buf, PlannerInfo *root, RelOptInfo *rel,
 	 * We handle relations for foreign tables, joins between those and upper
 	 * relations.
 	 */
-	Assert(rel->reloptkind == RELOPT_JOINREL ||
+	Assert(IS_JOIN_REL(rel) ||
 		   rel->reloptkind == RELOPT_BASEREL ||
 		   rel->reloptkind == RELOPT_OTHER_MEMBER_REL ||
 		   rel->reloptkind == RELOPT_UPPER_REL);
@@ -990,7 +990,7 @@ deparseSelectSql(List *tlist, List **retrieved_attrs, deparse_expr_cxt *context)
 	 */
 	appendStringInfoString(buf, "SELECT ");
 
-	if (foreignrel->reloptkind == RELOPT_JOINREL ||
+	if (IS_JOIN_REL(foreignrel) ||
 		foreignrel->reloptkind == RELOPT_UPPER_REL)
 	{
 		/* For a join relation use the input tlist */
@@ -1030,7 +1030,7 @@ deparseFromExpr(List *quals, deparse_expr_cxt *context)
 
 	/* For upper relations, scanrel must be either a joinrel or a baserel */
 	Assert(context->foreignrel->reloptkind != RELOPT_UPPER_REL ||
-		   scanrel->reloptkind == RELOPT_JOINREL ||
+		   IS_JOIN_REL(scanrel) ||
 		   scanrel->reloptkind == RELOPT_BASEREL);
 
 	/* Construct FROM clause */
@@ -1178,7 +1178,7 @@ deparseLockingClause(deparse_expr_cxt *context)
 			appendStringInfoString(buf, " FOR UPDATE");
 
 			/* Add the relation alias if we are here for a join relation */
-			if (rel->reloptkind == RELOPT_JOINREL)
+			if (IS_JOIN_REL(rel))
 				appendStringInfo(buf, " OF %s%d", REL_ALIAS_PREFIX, relid);
 		}
 		else
@@ -1342,7 +1342,7 @@ deparseFromExprForRel(StringInfo buf, PlannerInfo *root, RelOptInfo *foreignrel,
 {
 	PgFdwRelationInfo *fpinfo = (PgFdwRelationInfo *) foreignrel->fdw_private;
 
-	if (foreignrel->reloptkind == RELOPT_JOINREL)
+	if (IS_JOIN_REL(foreignrel))
 	{
 		RelOptInfo *rel_o = fpinfo->outerrel;
 		RelOptInfo *rel_i = fpinfo->innerrel;
diff --git a/contrib/postgres_fdw/postgres_fdw.c b/contrib/postgres_fdw/postgres_fdw.c
index 5d270b9..2487f26 100644
--- a/contrib/postgres_fdw/postgres_fdw.c
+++ b/contrib/postgres_fdw/postgres_fdw.c
@@ -723,6 +723,8 @@ get_useful_ecs_for_relation(PlannerInfo *root, RelOptInfo *rel)
 	/* If this is a child rel, we must use the topmost parent rel to search. */
 	if (rel->reloptkind == RELOPT_OTHER_MEMBER_REL)
 		relids = find_childrel_top_parent(root, rel)->relids;
+	else if (rel->reloptkind == RELOPT_OTHER_JOINREL)
+		relids = rel->top_parent_relids;
 	else
 		relids = rel->relids;
 
@@ -1181,7 +1183,7 @@ postgresGetForeignPlan(PlannerInfo *root,
 			local_exprs = lappend(local_exprs, rinfo->clause);
 	}
 
-	if (foreignrel->reloptkind == RELOPT_JOINREL ||
+	if (IS_JOIN_REL(foreignrel) ||
 		foreignrel->reloptkind == RELOPT_UPPER_REL)
 	{
 		/* For a join relation, get the conditions from fdw_private structure */
@@ -1247,7 +1249,7 @@ postgresGetForeignPlan(PlannerInfo *root,
 							 remote_conds,
 							 retrieved_attrs,
 							 makeInteger(fpinfo->fetch_size));
-	if (foreignrel->reloptkind == RELOPT_JOINREL ||
+	if (IS_JOIN_REL(foreignrel) ||
 		foreignrel->reloptkind == RELOPT_UPPER_REL)
 		fdw_private = lappend(fdw_private,
 							  makeString(fpinfo->relation_name->data));
@@ -2527,7 +2529,7 @@ estimate_path_cost_size(PlannerInfo *root,
 						   &remote_param_join_conds, &local_param_join_conds);
 
 		/* Build the list of columns to be fetched from the foreign server. */
-		if (foreignrel->reloptkind == RELOPT_JOINREL ||
+		if (IS_JOIN_REL(foreignrel) ||
 			foreignrel->reloptkind == RELOPT_UPPER_REL)
 			fdw_scan_tlist = build_tlist_to_deparse(foreignrel);
 		else
@@ -2609,7 +2611,7 @@ estimate_path_cost_size(PlannerInfo *root,
 			startup_cost = fpinfo->rel_startup_cost;
 			run_cost = fpinfo->rel_total_cost - fpinfo->rel_startup_cost;
 		}
-		else if (foreignrel->reloptkind == RELOPT_JOINREL)
+		else if (IS_JOIN_REL(foreignrel))
 		{
 			PgFdwRelationInfo *fpinfo_i;
 			PgFdwRelationInfo *fpinfo_o;
diff --git a/src/backend/foreign/foreign.c b/src/backend/foreign/foreign.c
index fdb4f71..e8ca7df 100644
--- a/src/backend/foreign/foreign.c
+++ b/src/backend/foreign/foreign.c
@@ -717,7 +717,7 @@ GetExistingLocalJoinPath(RelOptInfo *joinrel)
 {
 	ListCell   *lc;
 
-	Assert(joinrel->reloptkind == RELOPT_JOINREL);
+	Assert(IS_JOIN_REL(joinrel));
 
 	foreach(lc, joinrel->pathlist)
 	{
@@ -782,7 +782,7 @@ GetExistingLocalJoinPath(RelOptInfo *joinrel)
 			ForeignPath *foreign_path;
 
 			foreign_path = (ForeignPath *) joinpath->outerjoinpath;
-			if (foreign_path->path.parent->reloptkind == RELOPT_JOINREL)
+			if (IS_JOIN_REL(foreign_path->path.parent))
 				joinpath->outerjoinpath = foreign_path->fdw_outerpath;
 		}
 
@@ -791,7 +791,7 @@ GetExistingLocalJoinPath(RelOptInfo *joinrel)
 			ForeignPath *foreign_path;
 
 			foreign_path = (ForeignPath *) joinpath->innerjoinpath;
-			if (foreign_path->path.parent->reloptkind == RELOPT_JOINREL)
+			if (IS_JOIN_REL(foreign_path->path.parent))
 				joinpath->innerjoinpath = foreign_path->fdw_outerpath;
 		}
 
-- 
1.7.9.5

0012-Multi-level-partitioned-table-expansion.patchapplication/octet-stream; name=0012-Multi-level-partitioned-table-expansion.patchDownload
From 389e0adf55085470f3fdbbfad8521f758cc45136 Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Fri, 10 Feb 2017 13:50:14 +0530
Subject: [PATCH 12/14] Multi-level partitioned table expansion.

Construct inheritance hierarchy of a partitioned table to reflect the partition
hierarchy. Propagate lateral join information down the partition hierarchy.
---
 src/backend/optimizer/plan/initsplan.c |   14 +++++-
 src/backend/optimizer/prep/prepunion.c |   78 +++++++++++++++++++++-----------
 src/test/regress/expected/inherit.out  |   38 ++++++++--------
 3 files changed, 83 insertions(+), 47 deletions(-)

diff --git a/src/backend/optimizer/plan/initsplan.c b/src/backend/optimizer/plan/initsplan.c
index c170e96..e302f4f 100644
--- a/src/backend/optimizer/plan/initsplan.c
+++ b/src/backend/optimizer/plan/initsplan.c
@@ -628,7 +628,19 @@ create_lateral_join_info(PlannerInfo *root)
 	{
 		RelOptInfo *brel = root->simple_rel_array[rti];
 
-		if (brel == NULL || brel->reloptkind != RELOPT_BASEREL)
+		if (brel == NULL)
+			continue;
+
+		/*
+		 * If an "other rel" RTE is a "partitioned table", we must propagate
+		 * the lateral info inherited all the way from the root parent to its
+		 * children. That's because the children are not linked directly with
+		 * the root parent via AppendRelInfo's unlike in case of a regular
+		 * inheritance set (see expand_inherited_rtentry()).  Failing to
+		 * do this would result in those children not getting marked with the
+		 * appropriate lateral info.
+		 */
+		if (brel->reloptkind != RELOPT_BASEREL && !brel->part_scheme)
 			continue;
 
 		if (root->simple_rte_array[rti]->inh)
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index d459e95..63b45d6 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -98,7 +98,7 @@ static List *generate_append_tlist(List *colTypes, List *colCollations,
 					  List *refnames_tlist);
 static List *generate_setop_grouplist(SetOperationStmt *op, List *targetlist);
 static void expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte,
-						 Index rti);
+						 Index rti, LOCKMODE lockmode);
 static void make_inh_translation_list(Relation oldrelation,
 						  Relation newrelation,
 						  Index newvarno,
@@ -1319,19 +1319,44 @@ expand_inherited_tables(PlannerInfo *root)
 	Index		nrtes;
 	Index		rti;
 	ListCell   *rl;
+	Query	   *parse = root->parse;
 
 	/*
 	 * expand_inherited_rtentry may add RTEs to parse->rtable; there is no
 	 * need to scan them since they can't have inh=true.  So just scan as far
 	 * as the original end of the rtable list.
 	 */
-	nrtes = list_length(root->parse->rtable);
-	rl = list_head(root->parse->rtable);
+	nrtes = list_length(parse->rtable);
+	rl = list_head(parse->rtable);
 	for (rti = 1; rti <= nrtes; rti++)
 	{
 		RangeTblEntry *rte = (RangeTblEntry *) lfirst(rl);
+		LOCKMODE	lockmode;
+		PlanRowMark	  *oldrc;
 
-		expand_inherited_rtentry(root, rte, rti);
+		/*
+		 * The rewriter should already have obtained an appropriate lock on
+		 * each relation named in the query.  However, for each child relation
+		 * we add to the query, we must obtain an appropriate lock, because
+		 * this will be the first use of those relations in the
+		 * parse/rewrite/plan pipeline.
+		 *
+		 * If the parent relation is the query's result relation, then we need
+		 * RowExclusiveLock.  Otherwise, if it's accessed FOR UPDATE/SHARE, we
+		 * need RowShareLock; otherwise AccessShareLock.  We can't just grab
+		 * AccessShareLock because then the executor would be trying to upgrade
+		 * the lock, leading to possible deadlocks.  (This code should match
+		 * the parser and rewriter.)
+		 */
+		oldrc = get_plan_rowmark(root->rowMarks, rti);
+		if (rti == parse->resultRelation)
+			lockmode = RowExclusiveLock;
+		else if (oldrc && RowMarkRequiresRowShareLock(oldrc->markType))
+			lockmode = RowShareLock;
+		else
+			lockmode = AccessShareLock;
+
+		expand_inherited_rtentry(root, rte, rti, lockmode);
 		rl = lnext(rl);
 	}
 }
@@ -1353,13 +1378,13 @@ expand_inherited_tables(PlannerInfo *root)
  * a parent RTE must always have at least two associated AppendRelInfos.
  */
 static void
-expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
+expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti,
+						 LOCKMODE lockmode)
 {
 	Query	   *parse = root->parse;
 	Oid			parentOID;
 	PlanRowMark *oldrc;
 	Relation	oldrelation;
-	LOCKMODE	lockmode;
 	List	   *inhOIDs;
 	List	   *appinfos;
 	ListCell   *l;
@@ -1383,28 +1408,18 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 	}
 
 	/*
-	 * The rewriter should already have obtained an appropriate lock on each
-	 * relation named in the query.  However, for each child relation we add
-	 * to the query, we must obtain an appropriate lock, because this will be
-	 * the first use of those relations in the parse/rewrite/plan pipeline.
-	 *
-	 * If the parent relation is the query's result relation, then we need
-	 * RowExclusiveLock.  Otherwise, if it's accessed FOR UPDATE/SHARE, we
-	 * need RowShareLock; otherwise AccessShareLock.  We can't just grab
-	 * AccessShareLock because then the executor would be trying to upgrade
-	 * the lock, leading to possible deadlocks.  (This code should match the
-	 * parser and rewriter.)
+	 * Expand partitioned table level-wise to help optimizations like
+	 * partition-wise join which match partitions at every level. Otherwise,
+	 * scan for all members of inheritance set. Acquire needed locks
 	 */
-	oldrc = get_plan_rowmark(root->rowMarks, rti);
-	if (rti == parse->resultRelation)
-		lockmode = RowExclusiveLock;
-	else if (oldrc && RowMarkRequiresRowShareLock(oldrc->markType))
-		lockmode = RowShareLock;
+	if (rte->relkind == RELKIND_PARTITIONED_TABLE)
+	{
+		inhOIDs = list_make1_oid(parentOID);
+		inhOIDs = list_concat(inhOIDs,
+							  find_inheritance_children(parentOID, lockmode));
+	}
 	else
-		lockmode = AccessShareLock;
-
-	/* Scan for all members of inheritance set, acquire needed locks */
-	inhOIDs = find_all_inheritors(parentOID, lockmode, NULL);
+		inhOIDs = find_all_inheritors(parentOID, lockmode, NULL);
 
 	/*
 	 * Check that there's at least one descendant, else treat as no-child
@@ -1418,6 +1433,7 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 		return;
 	}
 
+	oldrc = get_plan_rowmark(root->rowMarks, rti);
 	/*
 	 * If parent relation is selected FOR UPDATE/SHARE, we need to mark its
 	 * PlanRowMark as isParent = true, and generate a new PlanRowMark for each
@@ -1475,7 +1491,12 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 		childrte = copyObject(rte);
 		childrte->relid = childOID;
 		childrte->relkind = newrelation->rd_rel->relkind;
-		childrte->inh = false;
+		/* A partitioned child will need to be expanded further. */
+		if (childOID != parentOID &&
+			childrte->relkind == RELKIND_PARTITIONED_TABLE)
+			childrte->inh = true;
+		else
+			childrte->inh = false;
 		childrte->requiredPerms = 0;
 		childrte->securityQuals = NIL;
 		parse->rtable = lappend(parse->rtable, childrte);
@@ -1539,6 +1560,9 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 		/* Close child relations, but keep locks */
 		if (childOID != parentOID)
 			heap_close(newrelation, NoLock);
+
+		/* Expand partitioned children recursively. */
+		expand_inherited_rtentry(root, childrte, childRTindex, lockmode);
 	}
 
 	heap_close(oldrelation, NoLock);
diff --git a/src/test/regress/expected/inherit.out b/src/test/regress/expected/inherit.out
index a8c8b28..6941045 100644
--- a/src/test/regress/expected/inherit.out
+++ b/src/test/regress/expected/inherit.out
@@ -1694,15 +1694,15 @@ explain (costs off) select * from range_list_parted;
  Append
    ->  Seq Scan on range_list_parted
    ->  Seq Scan on part_1_10
-   ->  Seq Scan on part_10_20
-   ->  Seq Scan on part_21_30
-   ->  Seq Scan on part_40_inf
    ->  Seq Scan on part_1_10_ab
    ->  Seq Scan on part_1_10_cd
+   ->  Seq Scan on part_10_20
    ->  Seq Scan on part_10_20_ab
    ->  Seq Scan on part_10_20_cd
+   ->  Seq Scan on part_21_30
    ->  Seq Scan on part_21_30_ab
    ->  Seq Scan on part_21_30_cd
+   ->  Seq Scan on part_40_inf
    ->  Seq Scan on part_40_inf_ab
    ->  Seq Scan on part_40_inf_cd
    ->  Seq Scan on part_40_inf_null
@@ -1730,18 +1730,18 @@ explain (costs off) select * from range_list_parted where b = 'ab';
          Filter: (b = 'ab'::bpchar)
    ->  Seq Scan on part_1_10
          Filter: (b = 'ab'::bpchar)
-   ->  Seq Scan on part_10_20
-         Filter: (b = 'ab'::bpchar)
-   ->  Seq Scan on part_21_30
-         Filter: (b = 'ab'::bpchar)
-   ->  Seq Scan on part_40_inf
-         Filter: (b = 'ab'::bpchar)
    ->  Seq Scan on part_1_10_ab
          Filter: (b = 'ab'::bpchar)
+   ->  Seq Scan on part_10_20
+         Filter: (b = 'ab'::bpchar)
    ->  Seq Scan on part_10_20_ab
          Filter: (b = 'ab'::bpchar)
+   ->  Seq Scan on part_21_30
+         Filter: (b = 'ab'::bpchar)
    ->  Seq Scan on part_21_30_ab
          Filter: (b = 'ab'::bpchar)
+   ->  Seq Scan on part_40_inf
+         Filter: (b = 'ab'::bpchar)
    ->  Seq Scan on part_40_inf_ab
          Filter: (b = 'ab'::bpchar)
 (19 rows)
@@ -1754,14 +1754,14 @@ explain (costs off) select * from range_list_parted where a between 3 and 23 and
          Filter: ((a >= 3) AND (a <= 23) AND (b = 'ab'::bpchar))
    ->  Seq Scan on part_1_10
          Filter: ((a >= 3) AND (a <= 23) AND (b = 'ab'::bpchar))
-   ->  Seq Scan on part_10_20
-         Filter: ((a >= 3) AND (a <= 23) AND (b = 'ab'::bpchar))
-   ->  Seq Scan on part_21_30
-         Filter: ((a >= 3) AND (a <= 23) AND (b = 'ab'::bpchar))
    ->  Seq Scan on part_1_10_ab
          Filter: ((a >= 3) AND (a <= 23) AND (b = 'ab'::bpchar))
+   ->  Seq Scan on part_10_20
+         Filter: ((a >= 3) AND (a <= 23) AND (b = 'ab'::bpchar))
    ->  Seq Scan on part_10_20_ab
          Filter: ((a >= 3) AND (a <= 23) AND (b = 'ab'::bpchar))
+   ->  Seq Scan on part_21_30
+         Filter: ((a >= 3) AND (a <= 23) AND (b = 'ab'::bpchar))
    ->  Seq Scan on part_21_30_ab
          Filter: ((a >= 3) AND (a <= 23) AND (b = 'ab'::bpchar))
 (15 rows)
@@ -1801,24 +1801,24 @@ explain (costs off) select * from range_list_parted where a is not null and a <
          Filter: ((a IS NOT NULL) AND (a < 67))
    ->  Seq Scan on part_1_10
          Filter: ((a IS NOT NULL) AND (a < 67))
-   ->  Seq Scan on part_10_20
-         Filter: ((a IS NOT NULL) AND (a < 67))
-   ->  Seq Scan on part_21_30
-         Filter: ((a IS NOT NULL) AND (a < 67))
-   ->  Seq Scan on part_40_inf
-         Filter: ((a IS NOT NULL) AND (a < 67))
    ->  Seq Scan on part_1_10_ab
          Filter: ((a IS NOT NULL) AND (a < 67))
    ->  Seq Scan on part_1_10_cd
          Filter: ((a IS NOT NULL) AND (a < 67))
+   ->  Seq Scan on part_10_20
+         Filter: ((a IS NOT NULL) AND (a < 67))
    ->  Seq Scan on part_10_20_ab
          Filter: ((a IS NOT NULL) AND (a < 67))
    ->  Seq Scan on part_10_20_cd
          Filter: ((a IS NOT NULL) AND (a < 67))
+   ->  Seq Scan on part_21_30
+         Filter: ((a IS NOT NULL) AND (a < 67))
    ->  Seq Scan on part_21_30_ab
          Filter: ((a IS NOT NULL) AND (a < 67))
    ->  Seq Scan on part_21_30_cd
          Filter: ((a IS NOT NULL) AND (a < 67))
+   ->  Seq Scan on part_40_inf
+         Filter: ((a IS NOT NULL) AND (a < 67))
    ->  Seq Scan on part_40_inf_ab
          Filter: ((a IS NOT NULL) AND (a < 67))
    ->  Seq Scan on part_40_inf_cd
-- 
1.7.9.5

0013-Multi-level-partition-wise-join-tests.patchapplication/octet-stream; name=0013-Multi-level-partition-wise-join-tests.patchDownload
From 7a458b66cea826cf4a7e4fd7a44598c902787075 Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Fri, 10 Feb 2017 14:18:19 +0530
Subject: [PATCH 13/14] Multi-level partition-wise join tests.

---
 .../expected/multi_level_partition_join.out        |  687 ++++++++++++++++++++
 src/test/regress/parallel_schedule                 |    2 +-
 src/test/regress/serial_schedule                   |    1 +
 .../regress/sql/multi_level_partition_join.sql     |   95 +++
 4 files changed, 784 insertions(+), 1 deletion(-)
 create mode 100644 src/test/regress/expected/multi_level_partition_join.out
 create mode 100644 src/test/regress/sql/multi_level_partition_join.sql

diff --git a/src/test/regress/expected/multi_level_partition_join.out b/src/test/regress/expected/multi_level_partition_join.out
new file mode 100644
index 0000000..97b1e79
--- /dev/null
+++ b/src/test/regress/expected/multi_level_partition_join.out
@@ -0,0 +1,687 @@
+-- Encourage partition-wise join plans.
+SET partition_wise_plan_weight TO 0.1;
+--
+-- multi-leveled partitions
+--
+CREATE TABLE prt1_l (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_l_p1 PARTITION OF prt1_l FOR VALUES FROM (0) TO (250);
+CREATE TABLE prt1_l_p2 PARTITION OF prt1_l FOR VALUES FROM (250) TO (500) PARTITION BY RANGE (c);
+CREATE TABLE prt1_l_p2_p1 PARTITION OF prt1_l_p2 FOR VALUES FROM ('0250') TO ('0400');
+CREATE TABLE prt1_l_p2_p2 PARTITION OF prt1_l_p2 FOR VALUES FROM ('0400') TO ('0500');
+CREATE TABLE prt1_l_p3 PARTITION OF prt1_l FOR VALUES FROM (500) TO (600) PARTITION BY RANGE (b);
+CREATE TABLE prt1_l_p3_p1 PARTITION OF prt1_l_p3 FOR VALUES FROM (500) TO (550);
+CREATE TABLE prt1_l_p3_p2 PARTITION OF prt1_l_p3 FOR VALUES FROM (550) TO (600);
+INSERT INTO prt1_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_l;
+ANALYZE prt1_l_p1;
+ANALYZE prt1_l_p2;
+ANALYZE prt1_l_p2_p1;
+ANALYZE prt1_l_p2_p2;
+ANALYZE prt1_l_p3;
+ANALYZE prt1_l_p3_p1;
+ANALYZE prt1_l_p3_p2;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_l AS SELECT * FROM prt1_l;
+CREATE TABLE prt2_l (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_l_p1 PARTITION OF prt2_l FOR VALUES FROM (0) TO (250);
+CREATE TABLE prt2_l_p2 PARTITION OF prt2_l FOR VALUES FROM (250) TO (500) PARTITION BY RANGE (c);
+CREATE TABLE prt2_l_p2_p1 PARTITION OF prt2_l_p2 FOR VALUES FROM ('0250') TO ('0400');
+CREATE TABLE prt2_l_p2_p2 PARTITION OF prt2_l_p2 FOR VALUES FROM ('0400') TO ('0500');
+CREATE TABLE prt2_l_p3 PARTITION OF prt2_l FOR VALUES FROM (500) TO (600) PARTITION BY RANGE (a);
+CREATE TABLE prt2_l_p3_p1 PARTITION OF prt2_l_p3 FOR VALUES FROM (500) TO (525);
+CREATE TABLE prt2_l_p3_p2 PARTITION OF prt2_l_p3 FOR VALUES FROM (525) TO (600);
+INSERT INTO prt2_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_l;
+ANALYZE prt2_l_p1;
+ANALYZE prt2_l_p2;
+ANALYZE prt2_l_p2_p1;
+ANALYZE prt2_l_p2_p2;
+ANALYZE prt2_l_p3;
+ANALYZE prt2_l_p3_p1;
+ANALYZE prt2_l_p3_p2;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_l AS SELECT * FROM prt2_l;
+-- inner join
+EXPLAIN (VERBOSE, 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.c = t2.c AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((t2.b = t1.a) AND ((t2.c)::text = (t1.c)::text))
+               ->  Seq Scan on public.prt2_l_p1 t2
+                     Output: t2.b, t2.c
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.prt1_l_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((t2_1.b = t1_1.a) AND ((t2_1.c)::text = (t1_1.c)::text))
+               ->  Seq Scan on public.prt2_l_p2_p1 t2_1
+                     Output: t2_1.b, t2_1.c
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.prt1_l_p2_p1 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((t2_2.b = t1_2.a) AND ((t2_2.c)::text = (t1_2.c)::text))
+               ->  Seq Scan on public.prt2_l_p2_p2 t2_2
+                     Output: t2_2.b, t2_2.c
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.prt1_l_p2_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_3.a, t1_3.c, t2_3.b, t2_3.c
+               Hash Cond: ((t2_3.b = t1_3.a) AND ((t2_3.c)::text = (t1_3.c)::text))
+               ->  Append
+                     ->  Seq Scan on public.prt2_l_p3 t2_3
+                           Output: t2_3.b, t2_3.c
+                     ->  Seq Scan on public.prt2_l_p3_p1 t2_4
+                           Output: t2_4.b, t2_4.c
+                     ->  Seq Scan on public.prt2_l_p3_p2 t2_5
+                           Output: t2_5.b, t2_5.c
+               ->  Hash
+                     Output: t1_3.a, t1_3.c
+                     ->  Append
+                           ->  Seq Scan on public.prt1_l_p3 t1_3
+                                 Output: t1_3.a, t1_3.c
+                                 Filter: ((t1_3.a % 25) = 0)
+                           ->  Seq Scan on public.prt1_l_p3_p1 t1_4
+                                 Output: t1_4.a, t1_4.c
+                                 Filter: ((t1_4.a % 25) = 0)
+                           ->  Seq Scan on public.prt1_l_p3_p2 t1_5
+                                 Output: t1_5.a, t1_5.c
+                                 Filter: ((t1_5.a % 25) = 0)
+(56 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1, prt2_l t2 WHERE t1.a = t2.b AND t1.c = t2.c AND t1.a % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1, uprt2_l t2 WHERE t1.a = t2.b AND t1.c = t2.c AND t1.a % 25 = 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)
+
+-- left join
+EXPLAIN (VERBOSE, 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((t2.b = t1.a) AND ((t2.c)::text = (t1.c)::text))
+               ->  Seq Scan on public.prt2_l_p1 t2
+                     Output: t2.b, t2.c
+               ->  Hash
+                     Output: t1.a, t1.c
+                     ->  Seq Scan on public.prt1_l_p1 t1
+                           Output: t1.a, t1.c
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+               Hash Cond: ((t2_1.b = t1_1.a) AND ((t2_1.c)::text = (t1_1.c)::text))
+               ->  Seq Scan on public.prt2_l_p2_p1 t2_1
+                     Output: t2_1.b, t2_1.c
+               ->  Hash
+                     Output: t1_1.a, t1_1.c
+                     ->  Seq Scan on public.prt1_l_p2_p1 t1_1
+                           Output: t1_1.a, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+               Hash Cond: ((t2_2.b = t1_2.a) AND ((t2_2.c)::text = (t1_2.c)::text))
+               ->  Seq Scan on public.prt2_l_p2_p2 t2_2
+                     Output: t2_2.b, t2_2.c
+               ->  Hash
+                     Output: t1_2.a, t1_2.c
+                     ->  Seq Scan on public.prt1_l_p2_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+         ->  Hash Right Join
+               Output: t1_3.a, t1_3.c, t2_3.b, t2_3.c
+               Hash Cond: ((t2_3.b = t1_3.a) AND ((t2_3.c)::text = (t1_3.c)::text))
+               ->  Append
+                     ->  Seq Scan on public.prt2_l_p3 t2_3
+                           Output: t2_3.b, t2_3.c
+                     ->  Seq Scan on public.prt2_l_p3_p1 t2_4
+                           Output: t2_4.b, t2_4.c
+                     ->  Seq Scan on public.prt2_l_p3_p2 t2_5
+                           Output: t2_5.b, t2_5.c
+               ->  Hash
+                     Output: t1_3.a, t1_3.c
+                     ->  Append
+                           ->  Seq Scan on public.prt1_l_p3 t1_3
+                                 Output: t1_3.a, t1_3.c
+                                 Filter: ((t1_3.a % 25) = 0)
+                           ->  Seq Scan on public.prt1_l_p3_p1 t1_4
+                                 Output: t1_4.a, t1_4.c
+                                 Filter: ((t1_4.a % 25) = 0)
+                           ->  Seq Scan on public.prt1_l_p3_p2 t1_5
+                                 Output: t1_5.a, t1_5.c
+                                 Filter: ((t1_5.a % 25) = 0)
+(56 rows)
+
+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.a % 25 = 0 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 |     | 
+(12 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 LEFT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t1.a % 25 = 0 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 |     | 
+(12 rows)
+
+-- right join
+EXPLAIN (VERBOSE, 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.b % 25 = 0 ORDER BY t1.a, t2.b;
+                                        QUERY PLAN                                        
+------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a, t2.b
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t2.b, t2.c, t1.a, t1.c
+                     Hash Cond: ((t1.a = t2.b) AND ((t1.c)::text = (t2.c)::text))
+                     ->  Seq Scan on public.prt1_l_p1 t1
+                           Output: t1.a, t1.c
+                     ->  Hash
+                           Output: t2.b, t2.c
+                           ->  Seq Scan on public.prt2_l_p1 t2
+                                 Output: t2.b, t2.c
+                                 Filter: ((t2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Hash Cond: ((t1_1.a = t2_1.b) AND ((t1_1.c)::text = (t2_1.c)::text))
+                     ->  Seq Scan on public.prt1_l_p2_p1 t1_1
+                           Output: t1_1.a, t1_1.c
+                     ->  Hash
+                           Output: t2_1.b, t2_1.c
+                           ->  Seq Scan on public.prt2_l_p2_p1 t2_1
+                                 Output: t2_1.b, t2_1.c
+                                 Filter: ((t2_1.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Hash Cond: ((t1_2.a = t2_2.b) AND ((t1_2.c)::text = (t2_2.c)::text))
+                     ->  Seq Scan on public.prt1_l_p2_p2 t1_2
+                           Output: t1_2.a, t1_2.c
+                     ->  Hash
+                           Output: t2_2.b, t2_2.c
+                           ->  Seq Scan on public.prt2_l_p2_p2 t2_2
+                                 Output: t2_2.b, t2_2.c
+                                 Filter: ((t2_2.b % 25) = 0)
+               ->  Hash Right Join
+                     Output: t2_3.b, t2_3.c, t1_3.a, t1_3.c
+                     Hash Cond: ((t1_3.a = t2_3.b) AND ((t1_3.c)::text = (t2_3.c)::text))
+                     ->  Append
+                           ->  Seq Scan on public.prt1_l_p3 t1_3
+                                 Output: t1_3.a, t1_3.c
+                           ->  Seq Scan on public.prt1_l_p3_p1 t1_4
+                                 Output: t1_4.a, t1_4.c
+                           ->  Seq Scan on public.prt1_l_p3_p2 t1_5
+                                 Output: t1_5.a, t1_5.c
+                     ->  Hash
+                           Output: t2_3.b, t2_3.c
+                           ->  Append
+                                 ->  Seq Scan on public.prt2_l_p3 t2_3
+                                       Output: t2_3.b, t2_3.c
+                                       Filter: ((t2_3.b % 25) = 0)
+                                 ->  Seq Scan on public.prt2_l_p3_p1 t2_4
+                                       Output: t2_4.b, t2_4.c
+                                       Filter: ((t2_4.b % 25) = 0)
+                                 ->  Seq Scan on public.prt2_l_p3_p2 t2_5
+                                       Output: t2_5.b, t2_5.c
+                                       Filter: ((t2_5.b % 25) = 0)
+(58 rows)
+
+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.b % 25 = 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 RIGHT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 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)
+
+-- full join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) ORDER BY t1.a, t2.b;
+                                                     QUERY PLAN                                                     
+--------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: prt1_l_p1.a, prt1_l_p1.c, prt2_l_p1.b, prt2_l_p1.c
+   Sort Key: prt1_l_p1.a, prt2_l_p1.b
+   ->  Append
+         ->  Hash Full Join
+               Output: prt1_l_p1.a, prt1_l_p1.c, prt2_l_p1.b, prt2_l_p1.c
+               Hash Cond: ((prt1_l_p1.a = prt2_l_p1.b) AND ((prt1_l_p1.c)::text = (prt2_l_p1.c)::text))
+               ->  Seq Scan on public.prt1_l_p1
+                     Output: prt1_l_p1.a, prt1_l_p1.c
+                     Filter: ((prt1_l_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p1.b, prt2_l_p1.c
+                     ->  Seq Scan on public.prt2_l_p1
+                           Output: prt2_l_p1.b, prt2_l_p1.c
+                           Filter: ((prt2_l_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_l_p2_p1.a, prt1_l_p2_p1.c, prt2_l_p2_p1.b, prt2_l_p2_p1.c
+               Hash Cond: ((prt1_l_p2_p1.a = prt2_l_p2_p1.b) AND ((prt1_l_p2_p1.c)::text = (prt2_l_p2_p1.c)::text))
+               ->  Seq Scan on public.prt1_l_p2_p1
+                     Output: prt1_l_p2_p1.a, prt1_l_p2_p1.c
+                     Filter: ((prt1_l_p2_p1.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p2_p1.b, prt2_l_p2_p1.c
+                     ->  Seq Scan on public.prt2_l_p2_p1
+                           Output: prt2_l_p2_p1.b, prt2_l_p2_p1.c
+                           Filter: ((prt2_l_p2_p1.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_l_p2_p2.a, prt1_l_p2_p2.c, prt2_l_p2_p2.b, prt2_l_p2_p2.c
+               Hash Cond: ((prt1_l_p2_p2.a = prt2_l_p2_p2.b) AND ((prt1_l_p2_p2.c)::text = (prt2_l_p2_p2.c)::text))
+               ->  Seq Scan on public.prt1_l_p2_p2
+                     Output: prt1_l_p2_p2.a, prt1_l_p2_p2.c
+                     Filter: ((prt1_l_p2_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p2_p2.b, prt2_l_p2_p2.c
+                     ->  Seq Scan on public.prt2_l_p2_p2
+                           Output: prt2_l_p2_p2.b, prt2_l_p2_p2.c
+                           Filter: ((prt2_l_p2_p2.b % 25) = 0)
+         ->  Hash Full Join
+               Output: prt1_l_p3.a, prt1_l_p3.c, prt2_l_p3.b, prt2_l_p3.c
+               Hash Cond: ((prt1_l_p3.a = prt2_l_p3.b) AND ((prt1_l_p3.c)::text = (prt2_l_p3.c)::text))
+               ->  Append
+                     ->  Seq Scan on public.prt1_l_p3
+                           Output: prt1_l_p3.a, prt1_l_p3.c
+                           Filter: ((prt1_l_p3.a % 25) = 0)
+                     ->  Seq Scan on public.prt1_l_p3_p1
+                           Output: prt1_l_p3_p1.a, prt1_l_p3_p1.c
+                           Filter: ((prt1_l_p3_p1.a % 25) = 0)
+                     ->  Seq Scan on public.prt1_l_p3_p2
+                           Output: prt1_l_p3_p2.a, prt1_l_p3_p2.c
+                           Filter: ((prt1_l_p3_p2.a % 25) = 0)
+               ->  Hash
+                     Output: prt2_l_p3.b, prt2_l_p3.c
+                     ->  Append
+                           ->  Seq Scan on public.prt2_l_p3
+                                 Output: prt2_l_p3.b, prt2_l_p3.c
+                                 Filter: ((prt2_l_p3.b % 25) = 0)
+                           ->  Seq Scan on public.prt2_l_p3_p1
+                                 Output: prt2_l_p3_p1.b, prt2_l_p3_p1.c
+                                 Filter: ((prt2_l_p3_p1.b % 25) = 0)
+                           ->  Seq Scan on public.prt2_l_p3_p2
+                                 Output: prt2_l_p3_p2.b, prt2_l_p3_p2.c
+                                 Filter: ((prt2_l_p3_p2.b % 25) = 0)
+(62 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) 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)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM uprt1_l WHERE uprt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_l WHERE uprt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) 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)
+
+-- lateral partition-wise join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) 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.a % 25 = 0 ORDER BY t1.a;
+                                                     QUERY PLAN                                                      
+---------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c, t2.a, t2.c, t2.b, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   Sort Key: t1.a
+   ->  Result
+         Output: t1.a, t1.b, t1.c, t2.a, t2.c, t2.b, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+         ->  Append
+               ->  Nested Loop Left Join
+                     Output: t1.a, t1.b, t1.c, t2.a, t2.c, t2.b, t3.a, (LEAST(t1.a, t2.a, t3.a))
+                     ->  Seq Scan on public.prt1_l_p1 t1
+                           Output: t1.a, t1.b, t1.c
+                           Filter: ((t1.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2.a, t2.c, t2.b, t3.a, LEAST(t1.a, t2.a, t3.a)
+                           Hash Cond: ((t3.b = t2.a) AND ((t3.c)::text = (t2.c)::text))
+                           ->  Seq Scan on public.prt2_l_p1 t3
+                                 Output: t3.a, t3.b, t3.c
+                           ->  Hash
+                                 Output: t2.a, t2.c, t2.b
+                                 ->  Seq Scan on public.prt1_l_p1 t2
+                                       Output: t2.a, t2.c, t2.b
+                                       Filter: ((t1.a = t2.a) AND ((t1.c)::text = (t2.c)::text))
+               ->  Nested Loop Left Join
+                     Output: t1_1.a, t1_1.b, t1_1.c, t2_1.a, t2_1.c, t2_1.b, t3_1.a, (LEAST(t1_1.a, t2_1.a, t3_1.a))
+                     ->  Seq Scan on public.prt1_l_p2_p1 t1_1
+                           Output: t1_1.a, t1_1.b, t1_1.c
+                           Filter: ((t1_1.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2_1.a, t2_1.c, t2_1.b, t3_1.a, LEAST(t1_1.a, t2_1.a, t3_1.a)
+                           Hash Cond: ((t3_1.b = t2_1.a) AND ((t3_1.c)::text = (t2_1.c)::text))
+                           ->  Seq Scan on public.prt2_l_p2_p1 t3_1
+                                 Output: t3_1.a, t3_1.b, t3_1.c
+                           ->  Hash
+                                 Output: t2_1.a, t2_1.c, t2_1.b
+                                 ->  Seq Scan on public.prt1_l_p2_p1 t2_1
+                                       Output: t2_1.a, t2_1.c, t2_1.b
+                                       Filter: ((t1_1.a = t2_1.a) AND ((t1_1.c)::text = (t2_1.c)::text))
+               ->  Nested Loop Left Join
+                     Output: t1_2.a, t1_2.b, t1_2.c, t2_2.a, t2_2.c, t2_2.b, t3_2.a, (LEAST(t1_2.a, t2_2.a, t3_2.a))
+                     ->  Seq Scan on public.prt1_l_p2_p2 t1_2
+                           Output: t1_2.a, t1_2.b, t1_2.c
+                           Filter: ((t1_2.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2_2.a, t2_2.c, t2_2.b, t3_2.a, LEAST(t1_2.a, t2_2.a, t3_2.a)
+                           Hash Cond: ((t3_2.b = t2_2.a) AND ((t3_2.c)::text = (t2_2.c)::text))
+                           ->  Seq Scan on public.prt2_l_p2_p2 t3_2
+                                 Output: t3_2.a, t3_2.b, t3_2.c
+                           ->  Hash
+                                 Output: t2_2.a, t2_2.c, t2_2.b
+                                 ->  Seq Scan on public.prt1_l_p2_p2 t2_2
+                                       Output: t2_2.a, t2_2.c, t2_2.b
+                                       Filter: ((t1_2.a = t2_2.a) AND ((t1_2.c)::text = (t2_2.c)::text))
+               ->  Nested Loop Left Join
+                     Output: t1_3.a, t1_3.b, t1_3.c, t2_3.a, t2_3.c, t2_3.b, t3_3.a, (LEAST(t1_3.a, t2_3.a, t3_3.a))
+                     ->  Append
+                           ->  Seq Scan on public.prt1_l_p3 t1_3
+                                 Output: t1_3.a, t1_3.b, t1_3.c
+                                 Filter: ((t1_3.a % 25) = 0)
+                           ->  Seq Scan on public.prt1_l_p3_p1 t1_4
+                                 Output: t1_4.a, t1_4.b, t1_4.c
+                                 Filter: ((t1_4.a % 25) = 0)
+                           ->  Seq Scan on public.prt1_l_p3_p2 t1_5
+                                 Output: t1_5.a, t1_5.b, t1_5.c
+                                 Filter: ((t1_5.a % 25) = 0)
+                     ->  Hash Join
+                           Output: t2_3.a, t2_3.c, t2_3.b, t3_3.a, LEAST(t1_3.a, t2_3.a, t3_3.a)
+                           Hash Cond: ((t3_3.b = t2_3.a) AND ((t3_3.c)::text = (t2_3.c)::text))
+                           ->  Append
+                                 ->  Seq Scan on public.prt2_l_p3 t3_3
+                                       Output: t3_3.a, t3_3.b, t3_3.c
+                                 ->  Seq Scan on public.prt2_l_p3_p1 t3_4
+                                       Output: t3_4.a, t3_4.b, t3_4.c
+                                 ->  Seq Scan on public.prt2_l_p3_p2 t3_5
+                                       Output: t3_5.a, t3_5.b, t3_5.c
+                           ->  Hash
+                                 Output: t2_3.a, t2_3.c, t2_3.b
+                                 ->  Append
+                                       ->  Seq Scan on public.prt1_l_p3 t2_3
+                                             Output: t2_3.a, t2_3.c, t2_3.b
+                                             Filter: ((t1_3.a = t2_3.a) AND ((t1_3.c)::text = (t2_3.c)::text))
+                                       ->  Seq Scan on public.prt1_l_p3_p1 t2_4
+                                             Output: t2_4.a, t2_4.c, t2_4.b
+                                             Filter: ((t1_3.a = t2_4.a) AND ((t1_3.c)::text = (t2_4.c)::text))
+                                       ->  Seq Scan on public.prt1_l_p3_p2 t2_5
+                                             Output: t2_5.a, t2_5.c, t2_5.b
+                                             Filter: ((t1_3.a = t2_5.a) AND ((t1_3.c)::text = (t2_5.c)::text))
+(85 rows)
+
+SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) 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.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t2c  | t2b | t3a | least 
+-----+-----+------+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |      |     |     |      
+ 100 | 100 | 0100 |     |      |     |     |      
+ 150 | 150 | 0150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |      |     |     |      
+ 250 | 250 | 0250 |     |      |     |     |      
+ 300 | 300 | 0300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |      |     |     |      
+ 400 | 400 | 0400 |     |      |     |     |      
+ 450 | 450 | 0450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |      |     |     |      
+ 550 | 550 | 0550 |     |      |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1_l t2 JOIN uprt2_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.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t2c  | t2b | t3a | least 
+-----+-----+------+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |      |     |     |      
+ 100 | 100 | 0100 |     |      |     |     |      
+ 150 | 150 | 0150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |      |     |     |      
+ 250 | 250 | 0250 |     |      |     |     |      
+ 300 | 300 | 0300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |      |     |     |      
+ 400 | 400 | 0400 |     |      |     |     |      
+ 450 | 450 | 0450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |      |     |     |      
+ 550 | 550 | 0550 |     |      |     |     |      
+(12 rows)
+
+-- lateral references with clauses without equi-join on partition key
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.c = t3.c)) ss
+			  ON t1.b = ss.t2a AND t1.c = ss.t2c WHERE t1.a % 25 = 0 ORDER BY t1.a;
+                                             QUERY PLAN                                              
+-----------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t1.c, t2.a, t2.c, t2.b, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   Sort Key: t1.a
+   ->  Nested Loop Left Join
+         Output: t1.a, t1.b, t1.c, t2.a, t2.c, t2.b, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+         ->  Append
+               ->  Seq Scan on public.prt1_l t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+               ->  Seq Scan on public.prt1_l_p1 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+               ->  Seq Scan on public.prt1_l_p2 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+               ->  Seq Scan on public.prt1_l_p2_p1 t1_3
+                     Output: t1_3.a, t1_3.b, t1_3.c
+                     Filter: ((t1_3.a % 25) = 0)
+               ->  Seq Scan on public.prt1_l_p2_p2 t1_4
+                     Output: t1_4.a, t1_4.b, t1_4.c
+                     Filter: ((t1_4.a % 25) = 0)
+               ->  Seq Scan on public.prt1_l_p3 t1_5
+                     Output: t1_5.a, t1_5.b, t1_5.c
+                     Filter: ((t1_5.a % 25) = 0)
+               ->  Seq Scan on public.prt1_l_p3_p1 t1_6
+                     Output: t1_6.a, t1_6.b, t1_6.c
+                     Filter: ((t1_6.a % 25) = 0)
+               ->  Seq Scan on public.prt1_l_p3_p2 t1_7
+                     Output: t1_7.a, t1_7.b, t1_7.c
+                     Filter: ((t1_7.a % 25) = 0)
+         ->  Append
+               ->  Hash Join
+                     Output: t2.a, t2.c, t2.b, t3.a, LEAST(t1.a, t2.a, t3.a)
+                     Hash Cond: ((t3.b = t2.a) AND ((t3.c)::text = (t2.c)::text))
+                     ->  Seq Scan on public.prt2_l_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t2.a, t2.c, t2.b
+                           ->  Seq Scan on public.prt1_l_p1 t2
+                                 Output: t2.a, t2.c, t2.b
+                                 Filter: ((t1.b = t2.a) AND ((t1.c)::text = (t2.c)::text))
+               ->  Hash Join
+                     Output: t2_1.a, t2_1.c, t2_1.b, t3_1.a, LEAST(t1.a, t2_1.a, t3_1.a)
+                     Hash Cond: ((t3_1.b = t2_1.a) AND ((t3_1.c)::text = (t2_1.c)::text))
+                     ->  Seq Scan on public.prt2_l_p2_p1 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t2_1.a, t2_1.c, t2_1.b
+                           ->  Seq Scan on public.prt1_l_p2_p1 t2_1
+                                 Output: t2_1.a, t2_1.c, t2_1.b
+                                 Filter: ((t1.b = t2_1.a) AND ((t1.c)::text = (t2_1.c)::text))
+               ->  Hash Join
+                     Output: t2_2.a, t2_2.c, t2_2.b, t3_2.a, LEAST(t1.a, t2_2.a, t3_2.a)
+                     Hash Cond: ((t3_2.b = t2_2.a) AND ((t3_2.c)::text = (t2_2.c)::text))
+                     ->  Seq Scan on public.prt2_l_p2_p2 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t2_2.a, t2_2.c, t2_2.b
+                           ->  Seq Scan on public.prt1_l_p2_p2 t2_2
+                                 Output: t2_2.a, t2_2.c, t2_2.b
+                                 Filter: ((t1.b = t2_2.a) AND ((t1.c)::text = (t2_2.c)::text))
+               ->  Hash Join
+                     Output: t2_3.a, t2_3.c, t2_3.b, t3_3.a, LEAST(t1.a, t2_3.a, t3_3.a)
+                     Hash Cond: ((t3_3.b = t2_3.a) AND ((t3_3.c)::text = (t2_3.c)::text))
+                     ->  Append
+                           ->  Seq Scan on public.prt2_l_p3 t3_3
+                                 Output: t3_3.a, t3_3.b, t3_3.c
+                           ->  Seq Scan on public.prt2_l_p3_p1 t3_4
+                                 Output: t3_4.a, t3_4.b, t3_4.c
+                           ->  Seq Scan on public.prt2_l_p3_p2 t3_5
+                                 Output: t3_5.a, t3_5.b, t3_5.c
+                     ->  Hash
+                           Output: t2_3.a, t2_3.c, t2_3.b
+                           ->  Append
+                                 ->  Seq Scan on public.prt1_l_p3 t2_3
+                                       Output: t2_3.a, t2_3.c, t2_3.b
+                                       Filter: ((t1.b = t2_3.a) AND ((t1.c)::text = (t2_3.c)::text))
+                                 ->  Seq Scan on public.prt1_l_p3_p1 t2_4
+                                       Output: t2_4.a, t2_4.c, t2_4.b
+                                       Filter: ((t1.b = t2_4.a) AND ((t1.c)::text = (t2_4.c)::text))
+                                 ->  Seq Scan on public.prt1_l_p3_p2 t2_5
+                                       Output: t2_5.a, t2_5.c, t2_5.b
+                                       Filter: ((t1.b = t2_5.a) AND ((t1.c)::text = (t2_5.c)::text))
+(83 rows)
+
+SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.c = t3.c)) ss
+			  ON t1.b = ss.t2a AND t1.c = ss.t2c WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t2c  | t2b | t3a | least 
+-----+-----+------+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |      |     |     |      
+ 100 | 100 | 0100 |     |      |     |     |      
+ 150 | 150 | 0150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |      |     |     |      
+ 250 | 250 | 0250 |     |      |     |     |      
+ 300 | 300 | 0300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |      |     |     |      
+ 400 | 400 | 0400 |     |      |     |     |      
+ 450 | 450 | 0450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |      |     |     |      
+ 550 | 550 | 0550 |     |      |     |     |      
+(12 rows)
+
+SELECT * FROM uprt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1_l t2 JOIN uprt2_l t3 ON (t2.a = t3.b AND t2.c = t3.c)) ss
+			  ON t1.b = ss.t2a AND t1.c = ss.t2c WHERE t1.a % 25 = 0 ORDER BY t1.a;
+  a  |  b  |  c   | t2a | t2c  | t2b | t3a | least 
+-----+-----+------+-----+------+-----+-----+-------
+   0 |   0 | 0000 |   0 | 0000 |   0 |   0 |     0
+  50 |  50 | 0050 |     |      |     |     |      
+ 100 | 100 | 0100 |     |      |     |     |      
+ 150 | 150 | 0150 | 150 | 0150 | 150 | 150 |   150
+ 200 | 200 | 0200 |     |      |     |     |      
+ 250 | 250 | 0250 |     |      |     |     |      
+ 300 | 300 | 0300 | 300 | 0300 | 300 | 300 |   300
+ 350 | 350 | 0350 |     |      |     |     |      
+ 400 | 400 | 0400 |     |      |     |     |      
+ 450 | 450 | 0450 | 450 | 0450 | 450 | 450 |   450
+ 500 | 500 | 0500 |     |      |     |     |      
+ 550 | 550 | 0550 |     |      |     |     |      
+(12 rows)
+
diff --git a/src/test/regress/parallel_schedule b/src/test/regress/parallel_schedule
index ac38f50..8d28203 100644
--- a/src/test/regress/parallel_schedule
+++ b/src/test/regress/parallel_schedule
@@ -103,7 +103,7 @@ test: publication subscription
 # ----------
 # Another group of parallel tests
 # ----------
-test: select_views portals_p2 foreign_key cluster dependency guc bitmapops combocid tsearch tsdicts foreign_data window xmlmap functional_deps advisory_lock json jsonb json_encoding indirect_toast equivclass partition_join
+test: select_views portals_p2 foreign_key cluster dependency guc bitmapops combocid tsearch tsdicts foreign_data window xmlmap functional_deps advisory_lock json jsonb json_encoding indirect_toast equivclass partition_join multi_level_partition_join
 # ----------
 # Another group of parallel tests
 # NB: temp.sql does a reconnect which transiently uses 2 connections,
diff --git a/src/test/regress/serial_schedule b/src/test/regress/serial_schedule
index 1bf98a4..14df5ee 100644
--- a/src/test/regress/serial_schedule
+++ b/src/test/regress/serial_schedule
@@ -173,3 +173,4 @@ test: xml
 test: event_trigger
 test: stats
 test: partition_join
+test: multi_level_partition_join
diff --git a/src/test/regress/sql/multi_level_partition_join.sql b/src/test/regress/sql/multi_level_partition_join.sql
new file mode 100644
index 0000000..821c095
--- /dev/null
+++ b/src/test/regress/sql/multi_level_partition_join.sql
@@ -0,0 +1,95 @@
+-- Encourage partition-wise join plans.
+SET partition_wise_plan_weight TO 0.1;
+
+--
+-- multi-leveled partitions
+--
+CREATE TABLE prt1_l (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE prt1_l_p1 PARTITION OF prt1_l FOR VALUES FROM (0) TO (250);
+CREATE TABLE prt1_l_p2 PARTITION OF prt1_l FOR VALUES FROM (250) TO (500) PARTITION BY RANGE (c);
+CREATE TABLE prt1_l_p2_p1 PARTITION OF prt1_l_p2 FOR VALUES FROM ('0250') TO ('0400');
+CREATE TABLE prt1_l_p2_p2 PARTITION OF prt1_l_p2 FOR VALUES FROM ('0400') TO ('0500');
+CREATE TABLE prt1_l_p3 PARTITION OF prt1_l FOR VALUES FROM (500) TO (600) PARTITION BY RANGE (b);
+CREATE TABLE prt1_l_p3_p1 PARTITION OF prt1_l_p3 FOR VALUES FROM (500) TO (550);
+CREATE TABLE prt1_l_p3_p2 PARTITION OF prt1_l_p3 FOR VALUES FROM (550) TO (600);
+INSERT INTO prt1_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+ANALYZE prt1_l;
+ANALYZE prt1_l_p1;
+ANALYZE prt1_l_p2;
+ANALYZE prt1_l_p2_p1;
+ANALYZE prt1_l_p2_p2;
+ANALYZE prt1_l_p3;
+ANALYZE prt1_l_p3_p1;
+ANALYZE prt1_l_p3_p2;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt1_l AS SELECT * FROM prt1_l;
+
+CREATE TABLE prt2_l (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE prt2_l_p1 PARTITION OF prt2_l FOR VALUES FROM (0) TO (250);
+CREATE TABLE prt2_l_p2 PARTITION OF prt2_l FOR VALUES FROM (250) TO (500) PARTITION BY RANGE (c);
+CREATE TABLE prt2_l_p2_p1 PARTITION OF prt2_l_p2 FOR VALUES FROM ('0250') TO ('0400');
+CREATE TABLE prt2_l_p2_p2 PARTITION OF prt2_l_p2 FOR VALUES FROM ('0400') TO ('0500');
+CREATE TABLE prt2_l_p3 PARTITION OF prt2_l FOR VALUES FROM (500) TO (600) PARTITION BY RANGE (a);
+CREATE TABLE prt2_l_p3_p1 PARTITION OF prt2_l_p3 FOR VALUES FROM (500) TO (525);
+CREATE TABLE prt2_l_p3_p2 PARTITION OF prt2_l_p3 FOR VALUES FROM (525) TO (600);
+INSERT INTO prt2_l SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 3) i;
+ANALYZE prt2_l;
+ANALYZE prt2_l_p1;
+ANALYZE prt2_l_p2;
+ANALYZE prt2_l_p2_p1;
+ANALYZE prt2_l_p2_p2;
+ANALYZE prt2_l_p3;
+ANALYZE prt2_l_p3_p1;
+ANALYZE prt2_l_p3_p2;
+-- TODO: This table is created only for testing the results. Remove once
+-- results are tested.
+CREATE TABLE uprt2_l AS SELECT * FROM prt2_l;
+
+-- inner join
+EXPLAIN (VERBOSE, 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.c = t2.c AND t1.a % 25 = 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.c = t2.c AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1, uprt2_l t2 WHERE t1.a = t2.b AND t1.c = t2.c AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+-- left join
+EXPLAIN (VERBOSE, 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.a % 25 = 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.a % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 LEFT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
+-- right join
+EXPLAIN (VERBOSE, 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.b % 25 = 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.b % 25 = 0 ORDER BY t1.a, t2.b;
+SELECT t1.a, t1.c, t2.b, t2.c FROM uprt1_l t1 RIGHT JOIN uprt2_l t2 ON t1.a = t2.b AND t1.c = t2.c WHERE t2.b % 25 = 0 ORDER BY t1.a, t2.b;
+
+-- full join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE prt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 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.a % 25 = 0) t1 FULL JOIN (SELECT * FROM prt2_l WHERE prt2_l.b % 25 = 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 uprt1_l WHERE uprt1_l.a % 25 = 0) t1 FULL JOIN (SELECT * FROM uprt2_l WHERE uprt2_l.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c) ORDER BY t1.a, t2.b;
+
+-- lateral partition-wise join
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) 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.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) 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.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1_l t2 JOIN uprt2_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.a % 25 = 0 ORDER BY t1.a;
+
+-- lateral references with clauses without equi-join on partition key
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.c = t3.c)) ss
+			  ON t1.b = ss.t2a AND t1.c = ss.t2c WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1_l t2 JOIN prt2_l t3 ON (t2.a = t3.b AND t2.c = t3.c)) ss
+			  ON t1.b = ss.t2a AND t1.c = ss.t2c WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM uprt1_l t1 LEFT JOIN LATERAL
+			  (SELECT t2.a AS t2a, t2.c AS t2c, t2.b AS t2b, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM uprt1_l t2 JOIN uprt2_l t3 ON (t2.a = t3.b AND t2.c = t3.c)) ss
+			  ON t1.b = ss.t2a AND t1.c = ss.t2c WHERE t1.a % 25 = 0 ORDER BY t1.a;
-- 
1.7.9.5

0014-Multi-level-partition-wise-join-support.patchapplication/octet-stream; name=0014-Multi-level-partition-wise-join-support.patchDownload
From fb2e5be3de7e1917bce3b1fdd76058aeac13a41f Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Fri, 10 Feb 2017 14:08:16 +0530
Subject: [PATCH 14/14] Multi-level partition-wise join support.

Annotate RelOptInfos of partitioned table by partitioning scheme at all levels
of a partitioning hierarchy.

Also, while creating Merge/Append plans from PartitionJoinPaths flattern
Merge/Append subplans arising from multi-level partitioning hierarchy.

The path to be reparameterized by a child refers to the parameters from the
topmost parent. This parent can be multiple levels away from the given child.
Hence use multi-level expression adjustment routines in
reparameterize_path_by_child().
---
 src/backend/optimizer/plan/createplan.c |   27 ++++-
 src/backend/optimizer/prep/prepunion.c  |   36 +++++--
 src/backend/optimizer/util/pathnode.c   |  175 +++++++++++++++++++++++++++----
 src/backend/optimizer/util/plancat.c    |   16 ---
 src/include/nodes/relation.h            |    6 ++
 5 files changed, 217 insertions(+), 43 deletions(-)

diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 3f1f2c1..f756be6 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -30,6 +30,7 @@
 #include "optimizer/clauses.h"
 #include "optimizer/cost.h"
 #include "optimizer/paths.h"
+#include "optimizer/pathnode.h"
 #include "optimizer/placeholder.h"
 #include "optimizer/plancat.h"
 #include "optimizer/planmain.h"
@@ -4115,6 +4116,18 @@ create_partition_join_plan(PlannerInfo *root, PartitionJoinPath *best_path)
 													  best_path->path.pathkeys,
 														NULL,
 														TOTAL_COST);
+		else if (best_path->reparamterized_by)
+		{
+			child_path = get_cheapest_parameterized_child_path(root,
+															   child_join,
+													best_path->old_param);
+
+			if (!child_path)
+				elog(ERROR, "Could not find a child-join path with required pathkeys or parameterization.");
+
+			child_path = reparameterize_path_by_child(root, child_path,
+												 best_path->reparamterized_by);
+		}
 		else
 			child_path = get_cheapest_parameterized_child_path(root,
 															   child_join,
@@ -4173,7 +4186,19 @@ create_partition_join_plan(PlannerInfo *root, PartitionJoinPath *best_path)
 			}
 		}
 
-		child_plans = lappend(child_plans, child_plan);
+		/* Flatten Merge/Append plans. */
+		if (IsA(child_plan, Append))
+		{
+			Append	   *append = (Append *) child_plan;
+			child_plans = list_concat(child_plans, append->appendplans);
+		}
+		else if (IsA(child_plan, MergeAppend))
+		{
+			MergeAppend	   *mappend = (MergeAppend *) child_plan;
+			child_plans = list_concat(child_plans, mappend->mergeplans);
+		}
+		else
+			child_plans = lappend(child_plans, child_plan);
 
 		/*
 		 * Reset the child_join memory context to reclaim the memory consumed
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index 63b45d6..9399050 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -2183,23 +2183,47 @@ adjust_inherited_tlist(List *tlist, AppendRelInfo *context)
  * adjust_appendrel_attrs_multilevel
  *	  Apply Var translations from a toplevel appendrel parent down to a child.
  *
- * In some cases we need to translate expressions referencing a baserel
+ * In some cases we need to translate expressions referencing a parent relation
  * to reference an appendrel child that's multiple levels removed from it.
  */
 Node *
 adjust_appendrel_attrs_multilevel(PlannerInfo *root, Node *node,
 								  RelOptInfo *child_rel)
 {
-	AppendRelInfo *appinfo = find_childrel_appendrelinfo(root, child_rel);
-	RelOptInfo *parent_rel = find_base_rel(root, appinfo->parent_relid);
+	List	   *appinfos = find_appinfos_by_relids(root, child_rel->relids);
+	RelOptInfo *parent_rel;
+
+	if (child_rel->reloptkind == RELOPT_OTHER_MEMBER_REL)
+	{
+		AppendRelInfo *appinfo = linitial(appinfos);
+		parent_rel = find_base_rel(root, appinfo->parent_relid);
+	}
+	else
+	{
+		Relids		parent_relids = NULL;
+		ListCell   *lc;
+
+		/* Construct relids set for the immediate parent of the given child. */
+		foreach (lc, appinfos)
+		{
+			AppendRelInfo   *appinfo = lfirst(lc);
+
+			parent_relids = bms_add_member(parent_relids, appinfo->parent_relid);
+		}
+
+		parent_rel = find_join_rel(root, parent_relids);
+		bms_free(parent_relids);
+	}
 
 	/* If parent is also a child, first recurse to apply its translations */
-	if (parent_rel->reloptkind == RELOPT_OTHER_MEMBER_REL)
+	if (IS_OTHER_REL(parent_rel))
 		node = adjust_appendrel_attrs_multilevel(root, node, parent_rel);
 	else
-		Assert(parent_rel->reloptkind == RELOPT_BASEREL);
+		Assert(parent_rel->reloptkind == RELOPT_BASEREL ||
+			   parent_rel->reloptkind == RELOPT_JOINREL);
+
 	/* Now translate for this child */
-	return adjust_appendrel_attrs(root, node, list_make1(appinfo));
+	return adjust_appendrel_attrs(root, node, appinfos);
 }
 
 /*
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index d1ee237..a32ef1a 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -48,6 +48,11 @@ typedef enum
 #define STD_FUZZ_FACTOR 1.01
 
 static List *translate_sub_tlist(List *tlist, int relid);
+static Relids adjust_child_relids_multilevel(PlannerInfo *root, Relids relids,
+											 Relids child_relids,
+											 Relids top_parent_relids);
+static List *build_child_clauses_multilevel(PlannerInfo *root, List *clauses,
+								Relids child_relids, Relids top_parent_relids);
 
 
 /*****************************************************************************
@@ -3410,7 +3415,6 @@ reparameterize_path_by_child(PlannerInfo *root, Path *path,
 	Path	   *new_path;
 	ParamPathInfo   *new_ppi;
 	ParamPathInfo   *old_ppi;
-	List	   *child_aris;
 	Relids		required_outer;
 
 	/*
@@ -3453,6 +3457,11 @@ reparameterize_path_by_child(PlannerInfo *root, Path *path,
 			memcpy(new_path, path, sizeof(AppendPath));
 			break;
 
+		case T_PartitionJoinPath:
+			new_path = (Path *) makeNode(PartitionJoinPath);
+			memcpy(new_path, path, sizeof(PartitionJoinPath));
+			break;
+
 		/*
 		 * TODO:
 		 * If this method of translation is fine add more path types here.
@@ -3464,15 +3473,15 @@ reparameterize_path_by_child(PlannerInfo *root, Path *path,
 	}
 
 	/*
-	 * Gather AppendRelInfos of the base partition relations in the outer child
-	 * relation. We need those for translating parent path to that of child by
-	 * substituting parent Var nodes and relids with those of children.
+	 * Adjust the parameterization information, which refers to the topmost
+	 * parent. The topmost parent can be multiple levels away from the given
+	 * child, hence use multi-level expression adjustment routines.
 	 */
-	child_aris = find_appinfos_by_relids(root, child_rel->relids);
-
-	/* Adjust the parameterization information. */
 	old_ppi = new_path->param_info;
-	required_outer = adjust_child_relids(old_ppi->ppi_req_outer, child_aris);
+	required_outer = adjust_child_relids_multilevel(root,
+													old_ppi->ppi_req_outer,
+													child_rel->relids,
+												 child_rel->top_parent_relids);
 
 	/* If we already have a PPI for this parameterization, just return it */
 	new_ppi = find_param_path_info(new_path->parent, required_outer);
@@ -3483,8 +3492,10 @@ reparameterize_path_by_child(PlannerInfo *root, Path *path,
 		new_ppi = makeNode(ParamPathInfo);
 		new_ppi->ppi_req_outer = required_outer;
 		new_ppi->ppi_rows = old_ppi->ppi_rows;
-		new_ppi->ppi_clauses = build_child_clauses(root, old_ppi->ppi_clauses,
-												   child_aris);
+		new_ppi->ppi_clauses = build_child_clauses_multilevel(root,
+														  old_ppi->ppi_clauses,
+															 child_rel->relids,
+												 child_rel->top_parent_relids);
 		new_path->parent->ppilist = lappend(new_path->parent->ppilist, new_ppi);
 	}
 	else
@@ -3500,6 +3511,7 @@ reparameterize_path_by_child(PlannerInfo *root, Path *path,
 	if (bms_overlap(path->parent->lateral_relids, child_rel->top_parent_relids))
 	{
 		MemoryContext	old_context;
+		List	   *exprs;
 
 		/*
 		 * Allocate the target in planner's context, since they are copied as
@@ -3507,9 +3519,12 @@ reparameterize_path_by_child(PlannerInfo *root, Path *path,
 		 */
 		old_context = MemoryContextSwitchTo(root->planner_cxt);
 		new_path->pathtarget = copy_pathtarget(new_path->pathtarget);
-		new_path->pathtarget->exprs = (List *) adjust_appendrel_attrs(root,
-											(Node *) new_path->pathtarget->exprs,
-																	child_aris);
+		exprs = new_path->pathtarget->exprs;
+		exprs = (List *) adjust_appendrel_attrs_multilevel(root,
+														   (Node *) exprs,
+														   child_rel);
+		new_path->pathtarget->exprs = exprs;
+
 		MemoryContextSwitchTo(old_context);
 	}
 
@@ -3531,9 +3546,10 @@ reparameterize_path_by_child(PlannerInfo *root, Path *path,
 				jpath->innerjoinpath = reparameterize_path_by_child(root,
 														 jpath->innerjoinpath,
 														 child_rel);
-				jpath->joinrestrictinfo = build_child_clauses(root,
-															  jpath->joinrestrictinfo,
-															  child_aris);
+				jpath->joinrestrictinfo = build_child_clauses_multilevel(root,
+													  jpath->joinrestrictinfo,
+															child_rel->relids,
+												child_rel->top_parent_relids);
 			}
 			break;
 
@@ -3556,10 +3572,30 @@ reparameterize_path_by_child(PlannerInfo *root, Path *path,
 			{
 				IndexPath *ipath = (IndexPath *)new_path;
 
-				ipath->indexclauses = build_child_clauses(root, ipath->indexclauses,
-														  child_aris);
-				ipath->indexquals = build_child_clauses(root, ipath->indexquals,
-														child_aris);
+				ipath->indexclauses = build_child_clauses_multilevel(root,
+														   ipath->indexclauses,
+															 child_rel->relids,
+												 child_rel->top_parent_relids);
+
+				ipath->indexquals = build_child_clauses_multilevel(root,
+															 ipath->indexquals,
+															 child_rel->relids,
+												 child_rel->top_parent_relids);
+			}
+			break;
+
+		case T_PartitionJoinPath:
+			{
+				PartitionJoinPath *pjpath = (PartitionJoinPath *)new_path;
+
+				/*
+				 * The paths for the child-joins covered by PartitionJoinPath will
+				 * be created later. If this PartitionJoinPath was reparameterized
+				 * by child of another relations, we will need that relation to
+				 * reparamterize the child-join paths.
+				 */
+				pjpath->old_param = PATH_REQ_OUTER(path); 
+				pjpath->reparamterized_by = child_rel;
 			}
 			break;
 
@@ -3570,3 +3606,102 @@ reparameterize_path_by_child(PlannerInfo *root, Path *path,
 
 	return new_path;
 }
+
+/*
+ * Replace any relid present in top_parent_relids with its child in
+ * child_relids. Members of child_relids can be multiple levels below top
+ * parent in the partition hierarchy.
+ */
+Relids
+adjust_child_relids_multilevel(PlannerInfo *root, Relids relids,
+							   Relids child_relids, Relids top_parent_relids)
+{
+	List	   *appinfos;
+	Relids		parent_relids = NULL;
+	ListCell   *lc;
+	Relids		result;
+	Relids		tmp_result = NULL;
+
+	/*
+	 * If the given relids set doesn't contain any of the top parent relids,
+	 * it will remain unchanged.
+	 */
+	if (!bms_overlap(relids, top_parent_relids))
+		return relids;
+
+	appinfos = find_appinfos_by_relids(root, child_relids);
+
+	/* Construct relids set for the immediate parent of the given child. */
+	foreach (lc, appinfos)
+	{
+		AppendRelInfo   *appinfo = lfirst(lc);
+
+		parent_relids = bms_add_member(parent_relids, appinfo->parent_relid);
+	}
+
+	/* Recurse if immediate parent is not the top parent. */
+	if (!bms_equal(parent_relids, top_parent_relids))
+	{
+		tmp_result = adjust_child_relids_multilevel(root, relids,
+													parent_relids,
+													top_parent_relids);
+		relids = tmp_result;
+	}
+
+	result = adjust_child_relids(relids, appinfos);
+
+	/* Free memory consumed by any immediate result. */
+	if (tmp_result)
+		bms_free(tmp_result);
+
+	bms_free(parent_relids);
+	list_free(appinfos);
+	return result;
+}
+
+/*
+ * build_child_clauses_multilevel
+ *		Similar to build_child_clauses but used when the child relation
+ *		represented by child_relids is
+ *		deeper down in the partition hierarchy reprepresented by parent
+ *		relation with relids top_parent_relids.
+ */
+List *
+build_child_clauses_multilevel(PlannerInfo *root, List *clauses,
+							   Relids child_relids, Relids top_parent_relids)
+{
+	List	   *appinfos;
+	Relids		parent_relids = NULL;
+	ListCell   *lc;
+	List	   *result;
+	List	   *tmp_result = NIL;
+
+	appinfos = find_appinfos_by_relids(root, child_relids);
+
+	/* Construct relids set for the immediate parent of the given child. */
+	foreach (lc, appinfos)
+	{
+		AppendRelInfo   *appinfo = lfirst(lc);
+
+		parent_relids = bms_add_member(parent_relids, appinfo->parent_relid);
+	}
+
+	/* Recurse if immediate parent is not the top parent. */
+	if (!bms_equal(parent_relids, top_parent_relids))
+	{
+		clauses = build_child_clauses_multilevel(root, clauses, parent_relids,
+												  top_parent_relids);
+		tmp_result = clauses;
+	}
+
+	result = build_child_clauses(root, clauses, appinfos);
+
+	/* Free memory consumed by any itermediate list. */
+	if (tmp_result)
+		list_free(tmp_result);
+
+	bms_free(parent_relids);
+	list_free(appinfos);
+
+	return result;
+}
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index 01ba885..c5e0ebf 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -1787,7 +1787,6 @@ find_partition_scheme(PlannerInfo *root, Relation relation)
 	int		nparts;
 	int		partnatts;
 	int		cnt_pks;
-	int		cnt_parts;
 	PartitionScheme	part_scheme = NULL;
 
 	/* No partition scheme for an unpartitioned relation. */
@@ -1797,21 +1796,6 @@ find_partition_scheme(PlannerInfo *root, Relation relation)
 	nparts = part_desc->nparts;
 	partnatts = part_key->partnatts;
 
-	/*
-	 * For a multi-level partitioned table, we do not retain the partitioning
-	 * hierarchy while expanding RTE for the topmost parent. Thus the number of
-	 * children as per root->append_rel_list does not match the number of
-	 * partitions specified in the partition descriptor and hence the
-	 * partitioning scheme of a multi-partitioned table does not reflect the
-	 * true picture. So for now, treat a multi-partitioned table as not
-	 * partitioned.
-	 */
-	for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
-	{
-		if (has_subclass(part_desc->oids[cnt_parts]))
-			return NULL;
-	}
-
 	/* Search for a matching partition scheme and return if found one. */
 	foreach (lc, root->part_schemes)
 	{
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 146d53b..f1ae325 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -1569,6 +1569,12 @@ typedef struct LimitPath
 typedef struct PartitionJoinPath
 {
 	Path		path;
+
+	/* If this path is result of reparameterization by child. */
+	Relids		old_param;			/* Older parameterization. */
+	RelOptInfo *reparamterized_by;	/* Child relation by which this paths was
+									 * reparamterized.
+									 */
 } PartitionJoinPath;
 
 /*
-- 
1.7.9.5

#35Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#30)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Mon, Feb 6, 2017 at 3:34 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

PartitionScheme is shared across multiple relations, join or base,
partitioned similarly. Obviously it can't and does not need to point
partition bound informations (which should all be same) of all those
base relations. O the the face of it, it looks weird that it points to
only one of them, mostly the one which it encounters first. But, since
it's going to be the same partition bound information, it doesn't
matter which one. So, I think, we can point of any one of those. Do
you agree?

Yes.

The fact that set_append_rel_size needs to reopen the relation to
extract a few more bits of information is not desirable. You need to
fish this information through in some other way; for example, you
could have get_relation_info() stash the needed bits in the
RelOptInfo.

I considered this option and discarded it, since not all partitioned
relations will have OIDs for partitions e.g. partitioned joins will
not have OIDs for their partitions. But now that I think of it, we
should probably store those OIDs just for the base relation and leave
them unused for non-base relations just like other base relation
specific fields in RelOptInfo.

Right.

FRACTION_PARTS_TO_PLAN seems like it should be a GUC.

+1. Will take care of this. Does "representative_partitions_fraction"
or "sample_partition_fraction" look like a good GUC name? Any other
suggestions?

I like the second one.

--
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

#36Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Ashutosh Bapat (#26)
5 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

2. If the PartitionJoinPath emerges as the best path, we create paths
for each of the remaining child-joins. Then we collect paths with
properties same as the given PartitionJoinPath, one from each
child-join. These paths are converted into plans and a Merge/Append
plan is created combing these plans. The paths and plans for
child-join are created in a temporary memory context. The final plan
for each child-join is copied into planner's context and the temporary
memory context is reset.

Robert and I discussed this in more detail. Path creation code may
allocate objects other than paths. postgres_fdw, for example,
allocates character array to hold the name of relation being
pushed-down. When the temporary context gets zapped after creating
paths for a given child-join, those other objects also gets thrown
away. Attached patch has implemented the idea that came out of the
discussion.

We create a memory context for holding paths at the time of creating
PlannerGlobal and save it in PlannerGlobal. The patch introduces a new
macro makePathNode() which allocates the memory for given type of path
from this context. Every create_*_path function has been changed to
use this macro instead of makeNode(). In standard_planner(), at the
end of planning we destroy the memory context freeing all the paths
allocated. While creating a plan node, planner copies everything
required by the plan from the path, so the path is not needed any
more. So, freeing corresponding memory should not have any adverse
effects.

Most of the create_*_path() functions accept root as an argument, thus
the temporary path context is available through root->glob everywhere.
An exception is create_append_path() which does not accept root as an
argument. The patch changes create_append_path() and its callers like
set_dummy_rel_pathlist(), mark_dummy_rel() to accept root as an
argument. Ideally paths are not required after creating plan, so we
should be
able to free the context right after the call to create_plan(). But we
need dummy paths while creating flat rtable in
set_plan_references()->add_rtes_to_flat_rtable(). We used to So free
the path context at the end of planning cycle. Now that we are
allocating all the paths in a different memory context, it doesn't
make sense to switch context in mark_dummy_rel().

0001 patch implements the idea described above.
0002 patch adds instrumentation to measure memory consumed in
standard_planner() call.
0003 patch adds a GUC zap_paths to enable/disable destroying path context.
The last two patches are for testing only.

Attached also find the SQL script and its output showing the memory
saved. For a 5 way self-join of pg_class, the total memory consumed in
standard_planner() is 760K without patch and with patch it comes down
to 713K, saving 47K memory otherwise occupied by paths. It looks like
something useful even without partition-wise joins.

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

Attachments:

0001-Free-up-memory-consumed-by-the-paths.patchapplication/octet-stream; name=0001-Free-up-memory-consumed-by-the-paths.patchDownload
From d62d4a40dc52ab05416446d19eaf53125d33aa4b Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Wed, 1 Mar 2017 11:07:28 +0530
Subject: [PATCH 1/3] Free up memory consumed by the paths.

Optimizer creates many paths while planning the query. Only one path
tree gets converted to the plan. The memory consumed by paths even
after the plan is created. For a simple query, it means that the
memory remains occupied till the end of the execution. Since plan node
copies the relevant information from the corresponding path, paths are
not needed after plan is created. We can free up that memory.

While creating global planner structure (PlannerGlob) we allocated a
separate memory context for creating paths. Every create_*_path()
function allocates path node in this context. The context is freed at
the end of planning cycle after creating PlannedStmt.

Ideally paths are not required after creating plan, so we should be
able to free the context right after the call to create_plan(). But we
need dummy paths while creating flat rtable in
set_plan_references()->add_rtes_to_flat_rtable(). There might be other
corner cases where we need paths. So free the path context at the end
of planning cycle.
---
 src/backend/optimizer/path/allpaths.c  |   25 ++++-----
 src/backend/optimizer/path/joinpath.c  |    2 +-
 src/backend/optimizer/path/joinrels.c  |   22 ++++----
 src/backend/optimizer/plan/planner.c   |   16 +++++-
 src/backend/optimizer/prep/prepunion.c |    4 +-
 src/backend/optimizer/util/pathnode.c  |   91 +++++++++++++++++++-------------
 src/include/nodes/relation.h           |    2 +
 src/include/optimizer/pathnode.h       |   10 ++--
 src/include/optimizer/paths.h          |    2 +-
 9 files changed, 104 insertions(+), 70 deletions(-)

diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 633b5c1..e22817e 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -325,7 +325,7 @@ set_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		 * we don't have a convention for marking a rel as dummy except by
 		 * assigning a dummy path to it.
 		 */
-		set_dummy_rel_pathlist(rel);
+		set_dummy_rel_pathlist(root, rel);
 	}
 	else if (rte->inh)
 	{
@@ -769,7 +769,7 @@ set_tablesample_rel_pathlist(PlannerInfo *root, RelOptInfo *rel, RangeTblEntry *
 		 bms_membership(root->all_baserels) != BMS_SINGLETON) &&
 	 !(GetTsmRoutine(rte->tablesample->tsmhandler)->repeatable_across_scans))
 	{
-		path = (Path *) create_material_path(rel, path);
+		path = (Path *) create_material_path(root, rel, path);
 	}
 
 	add_path(rel, path);
@@ -993,7 +993,7 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 			 * Some restriction clause reduced to constant FALSE or NULL after
 			 * substitution, so this child need not be scanned.
 			 */
-			set_dummy_rel_pathlist(childrel);
+			set_dummy_rel_pathlist(root, childrel);
 			continue;
 		}
 
@@ -1003,7 +1003,7 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 			 * This child need not be scanned, so we can omit it from the
 			 * appendrel.
 			 */
-			set_dummy_rel_pathlist(childrel);
+			set_dummy_rel_pathlist(root, childrel);
 			continue;
 		}
 
@@ -1153,7 +1153,7 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		 * appendrel dummy.  We must do this in this phase so that the rel's
 		 * dummy-ness is visible when we generate paths for other rels.
 		 */
-		set_dummy_rel_pathlist(rel);
+		set_dummy_rel_pathlist(root, rel);
 	}
 
 	pfree(parent_attrsizes);
@@ -1314,7 +1314,8 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 	 * if we have zero or one live subpath due to constraint exclusion.)
 	 */
 	if (subpaths_valid)
-		add_path(rel, (Path *) create_append_path(rel, subpaths, NULL, 0));
+		add_path(rel, (Path *) create_append_path(root, rel, subpaths, NULL,
+												  0));
 
 	/*
 	 * Consider an append of partial unordered, unparameterized partial paths.
@@ -1340,7 +1341,7 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 		Assert(parallel_workers > 0);
 
 		/* Generate a partial append path. */
-		appendpath = create_append_path(rel, partial_subpaths, NULL,
+		appendpath = create_append_path(root, rel, partial_subpaths, NULL,
 										parallel_workers);
 		add_partial_path(rel, (Path *) appendpath);
 	}
@@ -1392,8 +1393,8 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 		}
 
 		if (subpaths_valid)
-			add_path(rel, (Path *)
-					 create_append_path(rel, subpaths, required_outer, 0));
+			add_path(rel, (Path *) create_append_path(root, rel, subpaths,
+													  required_outer, 0));
 	}
 }
 
@@ -1613,7 +1614,7 @@ accumulate_append_subpath(List *subpaths, Path *path)
  * This is exported because inheritance_planner() has need for it.
  */
 void
-set_dummy_rel_pathlist(RelOptInfo *rel)
+set_dummy_rel_pathlist(PlannerInfo *root, RelOptInfo *rel)
 {
 	/* Set dummy size estimates --- we leave attr_widths[] as zeroes */
 	rel->rows = 0;
@@ -1623,7 +1624,7 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
 	rel->pathlist = NIL;
 	rel->partial_pathlist = NIL;
 
-	add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0));
+	add_path(rel, (Path *) create_append_path(root, rel, NIL, NULL, 0));
 
 	/*
 	 * We set the cheapest path immediately, to ensure that IS_DUMMY_REL()
@@ -1808,7 +1809,7 @@ set_subquery_pathlist(PlannerInfo *root, RelOptInfo *rel,
 
 	if (IS_DUMMY_REL(sub_final_rel))
 	{
-		set_dummy_rel_pathlist(rel);
+		set_dummy_rel_pathlist(root, rel);
 		return;
 	}
 
diff --git a/src/backend/optimizer/path/joinpath.c b/src/backend/optimizer/path/joinpath.c
index 2897245..2a0f6d9 100644
--- a/src/backend/optimizer/path/joinpath.c
+++ b/src/backend/optimizer/path/joinpath.c
@@ -1124,7 +1124,7 @@ match_unsorted_outer(PlannerInfo *root,
 		if (enable_material && inner_cheapest_total != NULL &&
 			!ExecMaterializesOutput(inner_cheapest_total->pathtype))
 			matpath = (Path *)
-				create_material_path(innerrel, inner_cheapest_total);
+				create_material_path(root, innerrel, inner_cheapest_total);
 	}
 
 	foreach(lc1, outerrel->pathlist)
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 0d00683..ff493b3 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -29,7 +29,7 @@ static void make_rels_by_clauseless_joins(PlannerInfo *root,
 static bool has_join_restriction(PlannerInfo *root, RelOptInfo *rel);
 static bool has_legal_joinclause(PlannerInfo *root, RelOptInfo *rel);
 static bool is_dummy_rel(RelOptInfo *rel);
-static void mark_dummy_rel(RelOptInfo *rel);
+static void mark_dummy_rel(PlannerInfo *root, RelOptInfo *rel);
 static bool restriction_is_constant_false(List *restrictlist,
 							  bool only_pushed_down);
 
@@ -748,7 +748,7 @@ make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 			if (is_dummy_rel(rel1) || is_dummy_rel(rel2) ||
 				restriction_is_constant_false(restrictlist, false))
 			{
-				mark_dummy_rel(joinrel);
+				mark_dummy_rel(root, joinrel);
 				break;
 			}
 			add_paths_to_joinrel(root, joinrel, rel1, rel2,
@@ -762,12 +762,12 @@ make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 			if (is_dummy_rel(rel1) ||
 				restriction_is_constant_false(restrictlist, true))
 			{
-				mark_dummy_rel(joinrel);
+				mark_dummy_rel(root, joinrel);
 				break;
 			}
 			if (restriction_is_constant_false(restrictlist, false) &&
 				bms_is_subset(rel2->relids, sjinfo->syn_righthand))
-				mark_dummy_rel(rel2);
+				mark_dummy_rel(root, rel2);
 			add_paths_to_joinrel(root, joinrel, rel1, rel2,
 								 JOIN_LEFT, sjinfo,
 								 restrictlist);
@@ -779,7 +779,7 @@ make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 			if ((is_dummy_rel(rel1) && is_dummy_rel(rel2)) ||
 				restriction_is_constant_false(restrictlist, true))
 			{
-				mark_dummy_rel(joinrel);
+				mark_dummy_rel(root, joinrel);
 				break;
 			}
 			add_paths_to_joinrel(root, joinrel, rel1, rel2,
@@ -815,7 +815,7 @@ make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 				if (is_dummy_rel(rel1) || is_dummy_rel(rel2) ||
 					restriction_is_constant_false(restrictlist, false))
 				{
-					mark_dummy_rel(joinrel);
+					mark_dummy_rel(root, joinrel);
 					break;
 				}
 				add_paths_to_joinrel(root, joinrel, rel1, rel2,
@@ -838,7 +838,7 @@ make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 				if (is_dummy_rel(rel1) || is_dummy_rel(rel2) ||
 					restriction_is_constant_false(restrictlist, false))
 				{
-					mark_dummy_rel(joinrel);
+					mark_dummy_rel(root, joinrel);
 					break;
 				}
 				add_paths_to_joinrel(root, joinrel, rel1, rel2,
@@ -853,12 +853,12 @@ make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 			if (is_dummy_rel(rel1) ||
 				restriction_is_constant_false(restrictlist, true))
 			{
-				mark_dummy_rel(joinrel);
+				mark_dummy_rel(root, joinrel);
 				break;
 			}
 			if (restriction_is_constant_false(restrictlist, false) &&
 				bms_is_subset(rel2->relids, sjinfo->syn_righthand))
-				mark_dummy_rel(rel2);
+				mark_dummy_rel(root, rel2);
 			add_paths_to_joinrel(root, joinrel, rel1, rel2,
 								 JOIN_ANTI, sjinfo,
 								 restrictlist);
@@ -1178,7 +1178,7 @@ is_dummy_rel(RelOptInfo *rel)
  * context the given RelOptInfo is in.
  */
 static void
-mark_dummy_rel(RelOptInfo *rel)
+mark_dummy_rel(PlannerInfo *root, RelOptInfo *rel)
 {
 	MemoryContext oldcontext;
 
@@ -1197,7 +1197,7 @@ mark_dummy_rel(RelOptInfo *rel)
 	rel->partial_pathlist = NIL;
 
 	/* Set up the dummy path */
-	add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0));
+	add_path(rel, (Path *) create_append_path(root, rel, NIL, NULL, 0));
 
 	/* Set or update cheapest_total_path and related fields */
 	set_cheapest(rel);
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index ca0ae78..b74e887 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -54,6 +54,7 @@
 #include "utils/rel.h"
 #include "utils/selfuncs.h"
 #include "utils/lsyscache.h"
+#include "utils/memutils.h"
 #include "utils/syscache.h"
 
 
@@ -218,6 +219,9 @@ standard_planner(Query *parse, int cursorOptions, ParamListInfo boundParams)
 	glob->lastPlanNodeId = 0;
 	glob->transientPlan = false;
 	glob->dependsOnRole = false;
+	glob->path_cxt = AllocSetContextCreate(CurrentMemoryContext,
+										   "Path creation temporary context",
+										   ALLOCSET_DEFAULT_SIZES);
 
 	/*
 	 * Assess whether it's feasible to use parallel mode for this query. We
@@ -414,6 +418,13 @@ standard_planner(Query *parse, int cursorOptions, ParamListInfo boundParams)
 	result->stmt_location = parse->stmt_location;
 	result->stmt_len = parse->stmt_len;
 
+	/*
+	 * We do not need paths any more, blow those away.
+	 * TODO: probably we should also set the pathlists to NIL.
+	 */
+	MemoryContextResetAndDeleteChildren(glob->path_cxt);
+	glob->path_cxt = NULL;
+
 	return result;
 }
 
@@ -1302,7 +1313,7 @@ inheritance_planner(PlannerInfo *root)
 	 */
 	if (subpaths == NIL)
 	{
-		set_dummy_rel_pathlist(final_rel);
+		set_dummy_rel_pathlist(root, final_rel);
 		return;
 	}
 
@@ -3334,7 +3345,7 @@ create_grouping_paths(PlannerInfo *root,
 				paths = lappend(paths, path);
 			}
 			path = (Path *)
-				create_append_path(grouped_rel,
+				create_append_path(root, grouped_rel,
 								   paths,
 								   NULL,
 								   0);
@@ -5261,6 +5272,7 @@ plan_cluster_use_sort(Oid tableOid, Oid indexOid)
 	query->commandType = CMD_SELECT;
 
 	glob = makeNode(PlannerGlobal);
+	glob->path_cxt = CurrentMemoryContext;
 
 	root = makeNode(PlannerInfo);
 	root->parse = query;
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index 1389db1..492f8df 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -566,7 +566,7 @@ generate_union_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NULL, 0);
+	path = (Path *) create_append_path(root, result_rel, pathlist, NULL, 0);
 
 	/* We have to manually jam the right tlist into the path; ick */
 	path->pathtarget = create_pathtarget(root, tlist);
@@ -678,7 +678,7 @@ generate_nonunion_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NULL, 0);
+	path = (Path *) create_append_path(root, result_rel, pathlist, NULL, 0);
 
 	/* We have to manually jam the right tlist into the path; ick */
 	path->pathtarget = create_pathtarget(root, tlist);
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 3248296..b91b25e 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -928,6 +928,18 @@ add_partial_path_precheck(RelOptInfo *parent_rel, Cost total_cost,
 /*****************************************************************************
  *		PATH NODE CREATION ROUTINES
  *****************************************************************************/
+#define makePathNode(root, _type_)		((_type_ *) newPath((root), sizeof(_type_),T_##_type_))
+
+static Path *
+newPath(PlannerInfo *root, size_t size, NodeTag tag)
+{
+	Path	   *result;
+
+	Assert(sizeof(Path) <= size);
+	result = MemoryContextAllocZero(root->glob->path_cxt, size);
+	result->type = tag;
+	return result;
+}
 
 /*
  * create_seqscan_path
@@ -938,7 +950,7 @@ Path *
 create_seqscan_path(PlannerInfo *root, RelOptInfo *rel,
 					Relids required_outer, int parallel_workers)
 {
-	Path	   *pathnode = makeNode(Path);
+	Path	   *pathnode = makePathNode(root, Path);
 
 	pathnode->pathtype = T_SeqScan;
 	pathnode->parent = rel;
@@ -962,7 +974,7 @@ create_seqscan_path(PlannerInfo *root, RelOptInfo *rel,
 Path *
 create_samplescan_path(PlannerInfo *root, RelOptInfo *rel, Relids required_outer)
 {
-	Path	   *pathnode = makeNode(Path);
+	Path	   *pathnode = makePathNode(root, Path);
 
 	pathnode->pathtype = T_SampleScan;
 	pathnode->parent = rel;
@@ -1018,7 +1030,7 @@ create_index_path(PlannerInfo *root,
 				  double loop_count,
 				  bool partial_path)
 {
-	IndexPath  *pathnode = makeNode(IndexPath);
+	IndexPath  *pathnode = makePathNode(root, IndexPath);
 	RelOptInfo *rel = index->rel;
 	List	   *indexquals,
 			   *indexqualcols;
@@ -1070,7 +1082,7 @@ create_bitmap_heap_path(PlannerInfo *root,
 						Relids required_outer,
 						double loop_count)
 {
-	BitmapHeapPath *pathnode = makeNode(BitmapHeapPath);
+	BitmapHeapPath *pathnode = makePathNode(root, BitmapHeapPath);
 
 	pathnode->path.pathtype = T_BitmapHeapScan;
 	pathnode->path.parent = rel;
@@ -1100,7 +1112,7 @@ create_bitmap_and_path(PlannerInfo *root,
 					   RelOptInfo *rel,
 					   List *bitmapquals)
 {
-	BitmapAndPath *pathnode = makeNode(BitmapAndPath);
+	BitmapAndPath *pathnode = makePathNode(root, BitmapAndPath);
 
 	pathnode->path.pathtype = T_BitmapAnd;
 	pathnode->path.parent = rel;
@@ -1136,7 +1148,7 @@ create_bitmap_or_path(PlannerInfo *root,
 					  RelOptInfo *rel,
 					  List *bitmapquals)
 {
-	BitmapOrPath *pathnode = makeNode(BitmapOrPath);
+	BitmapOrPath *pathnode = makePathNode(root, BitmapOrPath);
 
 	pathnode->path.pathtype = T_BitmapOr;
 	pathnode->path.parent = rel;
@@ -1171,7 +1183,7 @@ TidPath *
 create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
 					Relids required_outer)
 {
-	TidPath    *pathnode = makeNode(TidPath);
+	TidPath    *pathnode = makePathNode(root, TidPath);
 
 	pathnode->path.pathtype = T_TidScan;
 	pathnode->path.parent = rel;
@@ -1199,10 +1211,13 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
  * Note that we must handle subpaths = NIL, representing a dummy access path.
  */
 AppendPath *
-create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
+create_append_path(PlannerInfo *root,
+				   RelOptInfo *rel,
+				   List *subpaths,
+				   Relids required_outer,
 				   int parallel_workers)
 {
-	AppendPath *pathnode = makeNode(AppendPath);
+	AppendPath *pathnode = makePathNode(root, AppendPath);
 	ListCell   *l;
 
 	pathnode->path.pathtype = T_Append;
@@ -1259,7 +1274,7 @@ create_merge_append_path(PlannerInfo *root,
 						 List *pathkeys,
 						 Relids required_outer)
 {
-	MergeAppendPath *pathnode = makeNode(MergeAppendPath);
+	MergeAppendPath *pathnode = makePathNode(root, MergeAppendPath);
 	Cost		input_startup_cost;
 	Cost		input_total_cost;
 	ListCell   *l;
@@ -1346,7 +1361,7 @@ ResultPath *
 create_result_path(PlannerInfo *root, RelOptInfo *rel,
 				   PathTarget *target, List *resconstantqual)
 {
-	ResultPath *pathnode = makeNode(ResultPath);
+	ResultPath *pathnode = makePathNode(root, ResultPath);
 
 	pathnode->path.pathtype = T_Result;
 	pathnode->path.parent = rel;
@@ -1382,9 +1397,9 @@ create_result_path(PlannerInfo *root, RelOptInfo *rel,
  *	  pathnode.
  */
 MaterialPath *
-create_material_path(RelOptInfo *rel, Path *subpath)
+create_material_path(PlannerInfo *root, RelOptInfo *rel, Path *subpath)
 {
-	MaterialPath *pathnode = makeNode(MaterialPath);
+	MaterialPath *pathnode = makePathNode(root, MaterialPath);
 
 	Assert(subpath->parent == rel);
 
@@ -1451,7 +1466,7 @@ create_unique_path(PlannerInfo *root, RelOptInfo *rel, Path *subpath,
 	 */
 	oldcontext = MemoryContextSwitchTo(root->planner_cxt);
 
-	pathnode = makeNode(UniquePath);
+	pathnode = makePathNode(root, UniquePath);
 
 	pathnode->path.pathtype = T_Unique;
 	pathnode->path.parent = rel;
@@ -1667,7 +1682,7 @@ GatherPath *
 create_gather_path(PlannerInfo *root, RelOptInfo *rel, Path *subpath,
 				   PathTarget *target, Relids required_outer, double *rows)
 {
-	GatherPath *pathnode = makeNode(GatherPath);
+	GatherPath *pathnode = makePathNode(root, GatherPath);
 
 	Assert(subpath->parallel_safe);
 
@@ -1705,7 +1720,7 @@ SubqueryScanPath *
 create_subqueryscan_path(PlannerInfo *root, RelOptInfo *rel, Path *subpath,
 						 List *pathkeys, Relids required_outer)
 {
-	SubqueryScanPath *pathnode = makeNode(SubqueryScanPath);
+	SubqueryScanPath *pathnode = makePathNode(root, SubqueryScanPath);
 
 	pathnode->path.pathtype = T_SubqueryScan;
 	pathnode->path.parent = rel;
@@ -1733,7 +1748,7 @@ Path *
 create_functionscan_path(PlannerInfo *root, RelOptInfo *rel,
 						 List *pathkeys, Relids required_outer)
 {
-	Path	   *pathnode = makeNode(Path);
+	Path	   *pathnode = makePathNode(root, Path);
 
 	pathnode->pathtype = T_FunctionScan;
 	pathnode->parent = rel;
@@ -1759,7 +1774,7 @@ Path *
 create_valuesscan_path(PlannerInfo *root, RelOptInfo *rel,
 					   Relids required_outer)
 {
-	Path	   *pathnode = makeNode(Path);
+	Path	   *pathnode = makePathNode(root, Path);
 
 	pathnode->pathtype = T_ValuesScan;
 	pathnode->parent = rel;
@@ -1784,7 +1799,7 @@ create_valuesscan_path(PlannerInfo *root, RelOptInfo *rel,
 Path *
 create_ctescan_path(PlannerInfo *root, RelOptInfo *rel, Relids required_outer)
 {
-	Path	   *pathnode = makeNode(Path);
+	Path	   *pathnode = makePathNode(root, Path);
 
 	pathnode->pathtype = T_CteScan;
 	pathnode->parent = rel;
@@ -1810,7 +1825,7 @@ Path *
 create_worktablescan_path(PlannerInfo *root, RelOptInfo *rel,
 						  Relids required_outer)
 {
-	Path	   *pathnode = makeNode(Path);
+	Path	   *pathnode = makePathNode(root, Path);
 
 	pathnode->pathtype = T_WorkTableScan;
 	pathnode->parent = rel;
@@ -1849,7 +1864,7 @@ create_foreignscan_path(PlannerInfo *root, RelOptInfo *rel,
 						Path *fdw_outerpath,
 						List *fdw_private)
 {
-	ForeignPath *pathnode = makeNode(ForeignPath);
+	ForeignPath *pathnode = makePathNode(root, ForeignPath);
 
 	pathnode->path.pathtype = T_ForeignScan;
 	pathnode->path.parent = rel;
@@ -1955,7 +1970,7 @@ create_nestloop_path(PlannerInfo *root,
 					 List *pathkeys,
 					 Relids required_outer)
 {
-	NestPath   *pathnode = makeNode(NestPath);
+	NestPath   *pathnode = makePathNode(root, NestPath);
 	Relids		inner_req_outer = PATH_REQ_OUTER(inner_path);
 
 	/*
@@ -2045,7 +2060,7 @@ create_mergejoin_path(PlannerInfo *root,
 					  List *outersortkeys,
 					  List *innersortkeys)
 {
-	MergePath  *pathnode = makeNode(MergePath);
+	MergePath  *pathnode = makePathNode(root, MergePath);
 
 	pathnode->jpath.path.pathtype = T_MergeJoin;
 	pathnode->jpath.path.parent = joinrel;
@@ -2107,7 +2122,7 @@ create_hashjoin_path(PlannerInfo *root,
 					 Relids required_outer,
 					 List *hashclauses)
 {
-	HashPath   *pathnode = makeNode(HashPath);
+	HashPath   *pathnode = makePathNode(root, HashPath);
 
 	pathnode->jpath.path.pathtype = T_HashJoin;
 	pathnode->jpath.path.parent = joinrel;
@@ -2164,7 +2179,7 @@ create_projection_path(PlannerInfo *root,
 					   Path *subpath,
 					   PathTarget *target)
 {
-	ProjectionPath *pathnode = makeNode(ProjectionPath);
+	ProjectionPath *pathnode = makePathNode(root, ProjectionPath);
 	PathTarget *oldtarget = subpath->pathtarget;
 
 	pathnode->path.pathtype = T_Result;
@@ -2331,7 +2346,7 @@ create_set_projection_path(PlannerInfo *root,
 						   Path *subpath,
 						   PathTarget *target)
 {
-	ProjectSetPath *pathnode = makeNode(ProjectSetPath);
+	ProjectSetPath *pathnode = makePathNode(root, ProjectSetPath);
 	double		tlist_rows;
 	ListCell   *lc;
 
@@ -2399,7 +2414,7 @@ create_sort_path(PlannerInfo *root,
 				 List *pathkeys,
 				 double limit_tuples)
 {
-	SortPath   *pathnode = makeNode(SortPath);
+	SortPath   *pathnode = makePathNode(root, SortPath);
 
 	pathnode->path.pathtype = T_Sort;
 	pathnode->path.parent = rel;
@@ -2445,7 +2460,7 @@ create_group_path(PlannerInfo *root,
 				  List *qual,
 				  double numGroups)
 {
-	GroupPath  *pathnode = makeNode(GroupPath);
+	GroupPath  *pathnode = makePathNode(root, GroupPath);
 
 	pathnode->path.pathtype = T_Group;
 	pathnode->path.parent = rel;
@@ -2501,7 +2516,7 @@ create_upper_unique_path(PlannerInfo *root,
 						 int numCols,
 						 double numGroups)
 {
-	UpperUniquePath *pathnode = makeNode(UpperUniquePath);
+	UpperUniquePath *pathnode = makePathNode(root, UpperUniquePath);
 
 	pathnode->path.pathtype = T_Unique;
 	pathnode->path.parent = rel;
@@ -2558,7 +2573,7 @@ create_agg_path(PlannerInfo *root,
 				const AggClauseCosts *aggcosts,
 				double numGroups)
 {
-	AggPath    *pathnode = makeNode(AggPath);
+	AggPath    *pathnode = makePathNode(root, AggPath);
 
 	pathnode->path.pathtype = T_Agg;
 	pathnode->path.parent = rel;
@@ -2623,7 +2638,7 @@ create_groupingsets_path(PlannerInfo *root,
 						 const AggClauseCosts *agg_costs,
 						 double numGroups)
 {
-	GroupingSetsPath *pathnode = makeNode(GroupingSetsPath);
+	GroupingSetsPath *pathnode = makePathNode(root, GroupingSetsPath);
 	int			numGroupCols;
 
 	/* The topmost generated Plan node will be an Agg */
@@ -2736,7 +2751,7 @@ create_minmaxagg_path(PlannerInfo *root,
 					  List *mmaggregates,
 					  List *quals)
 {
-	MinMaxAggPath *pathnode = makeNode(MinMaxAggPath);
+	MinMaxAggPath *pathnode = makePathNode(root, MinMaxAggPath);
 	Cost		initplan_cost;
 	ListCell   *lc;
 
@@ -2797,7 +2812,7 @@ create_windowagg_path(PlannerInfo *root,
 					  WindowClause *winclause,
 					  List *winpathkeys)
 {
-	WindowAggPath *pathnode = makeNode(WindowAggPath);
+	WindowAggPath *pathnode = makePathNode(root, WindowAggPath);
 
 	pathnode->path.pathtype = T_WindowAgg;
 	pathnode->path.parent = rel;
@@ -2864,7 +2879,7 @@ create_setop_path(PlannerInfo *root,
 				  double numGroups,
 				  double outputRows)
 {
-	SetOpPath  *pathnode = makeNode(SetOpPath);
+	SetOpPath  *pathnode = makePathNode(root, SetOpPath);
 
 	pathnode->path.pathtype = T_SetOp;
 	pathnode->path.parent = rel;
@@ -2924,7 +2939,7 @@ create_recursiveunion_path(PlannerInfo *root,
 						   int wtParam,
 						   double numGroups)
 {
-	RecursiveUnionPath *pathnode = makeNode(RecursiveUnionPath);
+	RecursiveUnionPath *pathnode = makePathNode(root, RecursiveUnionPath);
 
 	pathnode->path.pathtype = T_RecursiveUnion;
 	pathnode->path.parent = rel;
@@ -2963,7 +2978,7 @@ LockRowsPath *
 create_lockrows_path(PlannerInfo *root, RelOptInfo *rel,
 					 Path *subpath, List *rowMarks, int epqParam)
 {
-	LockRowsPath *pathnode = makeNode(LockRowsPath);
+	LockRowsPath *pathnode = makePathNode(root, LockRowsPath);
 
 	pathnode->path.pathtype = T_LockRows;
 	pathnode->path.parent = rel;
@@ -3025,7 +3040,7 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
 						List *rowMarks, OnConflictExpr *onconflict,
 						int epqParam)
 {
-	ModifyTablePath *pathnode = makeNode(ModifyTablePath);
+	ModifyTablePath *pathnode = makePathNode(root, ModifyTablePath);
 	double		total_size;
 	ListCell   *lc;
 
@@ -3120,7 +3135,7 @@ create_limit_path(PlannerInfo *root, RelOptInfo *rel,
 				  Node *limitOffset, Node *limitCount,
 				  int64 offset_est, int64 count_est)
 {
-	LimitPath  *pathnode = makeNode(LimitPath);
+	LimitPath  *pathnode = makePathNode(root, LimitPath);
 
 	pathnode->path.pathtype = T_Limit;
 	pathnode->path.parent = rel;
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index f7ac6f6..aa2e9b3 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -128,6 +128,8 @@ typedef struct PlannerGlobal
 	bool		parallelModeNeeded;		/* parallel mode actually required? */
 
 	char		maxParallelHazard;		/* worst PROPARALLEL hazard level */
+
+	MemoryContext path_cxt;		/* Temporary context for holding paths. */
 } PlannerGlobal;
 
 /* macro for fetching the Plan associated with a SubPlan node */
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 53cad24..18591df 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -62,8 +62,11 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
 					  List *bitmapquals);
 extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
 					List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel, List *subpaths,
-				   Relids required_outer, int parallel_workers);
+extern AppendPath *create_append_path(PlannerInfo *root,
+				   RelOptInfo *rel,
+				   List *subpaths,
+				   Relids required_outer,
+				   int parallel_workers);
 extern MergeAppendPath *create_merge_append_path(PlannerInfo *root,
 						 RelOptInfo *rel,
 						 List *subpaths,
@@ -71,7 +74,8 @@ extern MergeAppendPath *create_merge_append_path(PlannerInfo *root,
 						 Relids required_outer);
 extern ResultPath *create_result_path(PlannerInfo *root, RelOptInfo *rel,
 				   PathTarget *target, List *resconstantqual);
-extern MaterialPath *create_material_path(RelOptInfo *rel, Path *subpath);
+extern MaterialPath *create_material_path(PlannerInfo *root, RelOptInfo *rel,
+							Path *subpath);
 extern UniquePath *create_unique_path(PlannerInfo *root, RelOptInfo *rel,
 				   Path *subpath, SpecialJoinInfo *sjinfo);
 extern GatherPath *create_gather_path(PlannerInfo *root,
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index ebda308..733e40d 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -49,7 +49,7 @@ extern PGDLLIMPORT join_search_hook_type join_search_hook;
 
 
 extern RelOptInfo *make_one_rel(PlannerInfo *root, List *joinlist);
-extern void set_dummy_rel_pathlist(RelOptInfo *rel);
+extern void set_dummy_rel_pathlist(PlannerInfo *root, RelOptInfo *rel);
 extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 					 List *initial_rels);
 
-- 
1.7.9.5

0002-Patch-to-measure-memory-used-in-CurrentMemoryContext.patchapplication/octet-stream; name=0002-Patch-to-measure-memory-used-in-CurrentMemoryContext.patchDownload
From 4f5bde7a4902e3cd2ef2a1dd4cfec2d93c695966 Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Wed, 1 Mar 2017 12:12:11 +0530
Subject: [PATCH 2/3] Patch to measure memory used in CurrentMemoryContext in
 standard_planner().

---
 src/backend/optimizer/plan/planner.c |    5 +++++
 src/backend/utils/mmgr/mcxt.c        |   26 ++++++++++++++++++++++++++
 src/include/utils/memutils.h         |    6 ++++++
 3 files changed, 37 insertions(+)

diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index b74e887..410750d 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -195,6 +195,9 @@ standard_planner(Query *parse, int cursorOptions, ParamListInfo boundParams)
 	Plan	   *top_plan;
 	ListCell   *lp,
 			   *lr;
+	MemoryContextCounters mem_start;
+
+	MemoryContextFuncStatsStart(CurrentMemoryContext, &mem_start, __FUNCTION__);
 
 	/*
 	 * Set up global state for this planner invocation.  This data is needed
@@ -425,6 +428,8 @@ standard_planner(Query *parse, int cursorOptions, ParamListInfo boundParams)
 	MemoryContextResetAndDeleteChildren(glob->path_cxt);
 	glob->path_cxt = NULL;
 
+	MemoryContextFuncStatsEnd(CurrentMemoryContext, &mem_start, __FUNCTION__);
+
 	return result;
 }
 
diff --git a/src/backend/utils/mmgr/mcxt.c b/src/backend/utils/mmgr/mcxt.c
index 6ad0bb4..94b25ac 100644
--- a/src/backend/utils/mmgr/mcxt.c
+++ b/src/backend/utils/mmgr/mcxt.c
@@ -577,6 +577,32 @@ MemoryContextStatsInternal(MemoryContext context, int level,
 	}
 }
 
+void
+MemoryContextFuncStatsStart(MemoryContext context,
+							MemoryContextCounters *start_counts,
+							const char *label)
+{
+	memset(start_counts, 0, sizeof(*start_counts));
+	MemoryContextStatsInternal(context, 0, false, 100, start_counts);
+}
+
+void
+MemoryContextFuncStatsEnd(MemoryContext context,
+						  MemoryContextCounters *start_counts,
+						  const char *label)
+{
+	MemoryContextCounters end_counts;
+	Size	start_used_space = start_counts->totalspace - start_counts->freespace;
+	Size	end_used_space;
+
+	memset(&end_counts, 0, sizeof(end_counts));
+	MemoryContextStatsInternal(context, 0, false, 100, &end_counts);
+	end_used_space = end_counts.totalspace - end_counts.freespace;
+
+	elog(NOTICE, "%s,%s,%zu,%zu,%ld", label, context->name,
+		 start_used_space, end_used_space, end_used_space - start_used_space);
+}
+
 /*
  * MemoryContextCheck
  *		Check all chunks in the named context.
diff --git a/src/include/utils/memutils.h b/src/include/utils/memutils.h
index 1d1035e..7d605b1 100644
--- a/src/include/utils/memutils.h
+++ b/src/include/utils/memutils.h
@@ -122,6 +122,12 @@ extern MemoryContext MemoryContextCreate(NodeTag tag, Size size,
 					MemoryContextMethods *methods,
 					MemoryContext parent,
 					const char *name);
+extern void MemoryContextFuncStatsStart(MemoryContext context,
+										MemoryContextCounters *start_counts,
+										const char *func_label);
+extern void MemoryContextFuncStatsEnd(MemoryContext context,
+									  MemoryContextCounters *start_counts,
+									  const char *func_label);
 
 
 /*
-- 
1.7.9.5

0003-GUC-zap_path-to-enable-freeing-memory-consumed-by-pa.patchapplication/octet-stream; name=0003-GUC-zap_path-to-enable-freeing-memory-consumed-by-pa.patchDownload
From 760d328696a460deb418c8fa34b80e53fa530129 Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Wed, 1 Mar 2017 13:29:51 +0530
Subject: [PATCH 3/3] GUC zap_path to enable freeing memory consumed by paths.

For measuring the memory saved, add a GUC zap_path to enable/disable
freeing memory occupied by paths.
---
 src/backend/optimizer/path/costsize.c |    1 +
 src/backend/optimizer/plan/planner.c  |   15 +++++++++------
 src/backend/utils/misc/guc.c          |   10 ++++++++++
 src/include/optimizer/cost.h          |    1 +
 4 files changed, 21 insertions(+), 6 deletions(-)

diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index c138f57..f4dfa00 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -126,6 +126,7 @@ bool		enable_nestloop = true;
 bool		enable_material = true;
 bool		enable_mergejoin = true;
 bool		enable_hashjoin = true;
+bool		zap_paths = true;
 
 typedef struct
 {
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 410750d..c7b085f 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -421,12 +421,15 @@ standard_planner(Query *parse, int cursorOptions, ParamListInfo boundParams)
 	result->stmt_location = parse->stmt_location;
 	result->stmt_len = parse->stmt_len;
 
-	/*
-	 * We do not need paths any more, blow those away.
-	 * TODO: probably we should also set the pathlists to NIL.
-	 */
-	MemoryContextResetAndDeleteChildren(glob->path_cxt);
-	glob->path_cxt = NULL;
+	if (zap_paths)
+	{
+		/*
+		 * We do not need paths any more, blow those away.
+		 * TODO: probably we should also set the pathlists to NIL.
+		 */
+		MemoryContextResetAndDeleteChildren(glob->path_cxt);
+		glob->path_cxt = NULL;
+	}
 
 	MemoryContextFuncStatsEnd(CurrentMemoryContext, &mem_start, __FUNCTION__);
 
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 2477138..630d3f7 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -912,6 +912,16 @@ static struct config_bool ConfigureNamesBool[] =
 		true,
 		NULL, NULL, NULL
 	},
+
+	{
+		{"zap_paths", PGC_USERSET, QUERY_TUNING_GEQO,
+			gettext_noop("Free up the memory used by paths at the end of planning."),
+		},
+		&zap_paths,
+		true,
+		NULL, NULL, NULL
+	},
+
 	{
 		/* Not for general use --- used by SET SESSION AUTHORIZATION */
 		{"is_superuser", PGC_INTERNAL, UNGROUPED,
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 72200fa..fc882ad 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -67,6 +67,7 @@ extern bool enable_material;
 extern bool enable_mergejoin;
 extern bool enable_hashjoin;
 extern int	constraint_exclusion;
+extern bool zap_paths;
 
 extern double clamp_row_est(double nrows);
 extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
-- 
1.7.9.5

path_memory.outapplication/octet-stream; name=path_memory.outDownload
path_memory.sqlapplication/octet-stream; name=path_memory.sqlDownload
#37Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Ashutosh Bapat (#36)
3 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

Updated 0001 patch with some more comments. Attaching all the patches
for quick access.

On Wed, Mar 1, 2017 at 2:26 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

2. If the PartitionJoinPath emerges as the best path, we create paths
for each of the remaining child-joins. Then we collect paths with
properties same as the given PartitionJoinPath, one from each
child-join. These paths are converted into plans and a Merge/Append
plan is created combing these plans. The paths and plans for
child-join are created in a temporary memory context. The final plan
for each child-join is copied into planner's context and the temporary
memory context is reset.

Robert and I discussed this in more detail. Path creation code may
allocate objects other than paths. postgres_fdw, for example,
allocates character array to hold the name of relation being
pushed-down. When the temporary context gets zapped after creating
paths for a given child-join, those other objects also gets thrown
away. Attached patch has implemented the idea that came out of the
discussion.

We create a memory context for holding paths at the time of creating
PlannerGlobal and save it in PlannerGlobal. The patch introduces a new
macro makePathNode() which allocates the memory for given type of path
from this context. Every create_*_path function has been changed to
use this macro instead of makeNode(). In standard_planner(), at the
end of planning we destroy the memory context freeing all the paths
allocated. While creating a plan node, planner copies everything
required by the plan from the path, so the path is not needed any
more. So, freeing corresponding memory should not have any adverse
effects.

Most of the create_*_path() functions accept root as an argument, thus
the temporary path context is available through root->glob everywhere.
An exception is create_append_path() which does not accept root as an
argument. The patch changes create_append_path() and its callers like
set_dummy_rel_pathlist(), mark_dummy_rel() to accept root as an
argument. Ideally paths are not required after creating plan, so we
should be
able to free the context right after the call to create_plan(). But we
need dummy paths while creating flat rtable in
set_plan_references()->add_rtes_to_flat_rtable(). We used to So free
the path context at the end of planning cycle. Now that we are
allocating all the paths in a different memory context, it doesn't
make sense to switch context in mark_dummy_rel().

0001 patch implements the idea described above.
0002 patch adds instrumentation to measure memory consumed in
standard_planner() call.
0003 patch adds a GUC zap_paths to enable/disable destroying path context.
The last two patches are for testing only.

Attached also find the SQL script and its output showing the memory
saved. For a 5 way self-join of pg_class, the total memory consumed in
standard_planner() is 760K without patch and with patch it comes down
to 713K, saving 47K memory otherwise occupied by paths. It looks like
something useful even without partition-wise joins.

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

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

Attachments:

0001-Free-up-memory-consumed-by-the-paths.patchapplication/octet-stream; name=0001-Free-up-memory-consumed-by-the-paths.patchDownload
From 02bca46338c9623735b466ccda52265fece6cafe Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Wed, 1 Mar 2017 11:07:28 +0530
Subject: [PATCH 1/3] Free up memory consumed by the paths.

Optimizer creates many paths while planning the query. Only one path
tree gets converted to the plan. Paths continue to occupy memory even
after the plan is created. For an un-PREPAREd query, it means that the
memory remains occupied till the end of the execution. Since plan node
copies the relevant information from the corresponding path, paths are
not needed after plan is created. We can free up that memory.

While creating global planner structure (PlannerGlob) we allocate a
separate memory context for creating paths. Every create_*_path()
function allocates path node in this context. The context is freed at
the end of planning cycle after creating PlannedStmt.

Ideally paths are not required after creating plan, so we should be
able to free the context right after the call to create_plan(). But we
need dummy paths while creating flat rtable in
set_plan_references()->add_rtes_to_flat_rtable(). There might be other
corner cases where we need paths. So free the path context at the end
of planning cycle.

Now that we have separate memory context to hold paths, we don't need
to switch contexts in make_dummy_rel().
---
 src/backend/optimizer/path/allpaths.c  |   25 ++++----
 src/backend/optimizer/path/joinpath.c  |    2 +-
 src/backend/optimizer/path/joinrels.c  |   37 ++++--------
 src/backend/optimizer/plan/planner.c   |   16 ++++-
 src/backend/optimizer/prep/prepunion.c |    4 +-
 src/backend/optimizer/util/pathnode.c  |  100 ++++++++++++++++++++------------
 src/include/nodes/relation.h           |    2 +
 src/include/optimizer/pathnode.h       |   10 +++-
 src/include/optimizer/paths.h          |    2 +-
 9 files changed, 113 insertions(+), 85 deletions(-)

diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 633b5c1..e22817e 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -325,7 +325,7 @@ set_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		 * we don't have a convention for marking a rel as dummy except by
 		 * assigning a dummy path to it.
 		 */
-		set_dummy_rel_pathlist(rel);
+		set_dummy_rel_pathlist(root, rel);
 	}
 	else if (rte->inh)
 	{
@@ -769,7 +769,7 @@ set_tablesample_rel_pathlist(PlannerInfo *root, RelOptInfo *rel, RangeTblEntry *
 		 bms_membership(root->all_baserels) != BMS_SINGLETON) &&
 	 !(GetTsmRoutine(rte->tablesample->tsmhandler)->repeatable_across_scans))
 	{
-		path = (Path *) create_material_path(rel, path);
+		path = (Path *) create_material_path(root, rel, path);
 	}
 
 	add_path(rel, path);
@@ -993,7 +993,7 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 			 * Some restriction clause reduced to constant FALSE or NULL after
 			 * substitution, so this child need not be scanned.
 			 */
-			set_dummy_rel_pathlist(childrel);
+			set_dummy_rel_pathlist(root, childrel);
 			continue;
 		}
 
@@ -1003,7 +1003,7 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 			 * This child need not be scanned, so we can omit it from the
 			 * appendrel.
 			 */
-			set_dummy_rel_pathlist(childrel);
+			set_dummy_rel_pathlist(root, childrel);
 			continue;
 		}
 
@@ -1153,7 +1153,7 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		 * appendrel dummy.  We must do this in this phase so that the rel's
 		 * dummy-ness is visible when we generate paths for other rels.
 		 */
-		set_dummy_rel_pathlist(rel);
+		set_dummy_rel_pathlist(root, rel);
 	}
 
 	pfree(parent_attrsizes);
@@ -1314,7 +1314,8 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 	 * if we have zero or one live subpath due to constraint exclusion.)
 	 */
 	if (subpaths_valid)
-		add_path(rel, (Path *) create_append_path(rel, subpaths, NULL, 0));
+		add_path(rel, (Path *) create_append_path(root, rel, subpaths, NULL,
+												  0));
 
 	/*
 	 * Consider an append of partial unordered, unparameterized partial paths.
@@ -1340,7 +1341,7 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 		Assert(parallel_workers > 0);
 
 		/* Generate a partial append path. */
-		appendpath = create_append_path(rel, partial_subpaths, NULL,
+		appendpath = create_append_path(root, rel, partial_subpaths, NULL,
 										parallel_workers);
 		add_partial_path(rel, (Path *) appendpath);
 	}
@@ -1392,8 +1393,8 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 		}
 
 		if (subpaths_valid)
-			add_path(rel, (Path *)
-					 create_append_path(rel, subpaths, required_outer, 0));
+			add_path(rel, (Path *) create_append_path(root, rel, subpaths,
+													  required_outer, 0));
 	}
 }
 
@@ -1613,7 +1614,7 @@ accumulate_append_subpath(List *subpaths, Path *path)
  * This is exported because inheritance_planner() has need for it.
  */
 void
-set_dummy_rel_pathlist(RelOptInfo *rel)
+set_dummy_rel_pathlist(PlannerInfo *root, RelOptInfo *rel)
 {
 	/* Set dummy size estimates --- we leave attr_widths[] as zeroes */
 	rel->rows = 0;
@@ -1623,7 +1624,7 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
 	rel->pathlist = NIL;
 	rel->partial_pathlist = NIL;
 
-	add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0));
+	add_path(rel, (Path *) create_append_path(root, rel, NIL, NULL, 0));
 
 	/*
 	 * We set the cheapest path immediately, to ensure that IS_DUMMY_REL()
@@ -1808,7 +1809,7 @@ set_subquery_pathlist(PlannerInfo *root, RelOptInfo *rel,
 
 	if (IS_DUMMY_REL(sub_final_rel))
 	{
-		set_dummy_rel_pathlist(rel);
+		set_dummy_rel_pathlist(root, rel);
 		return;
 	}
 
diff --git a/src/backend/optimizer/path/joinpath.c b/src/backend/optimizer/path/joinpath.c
index 2897245..2a0f6d9 100644
--- a/src/backend/optimizer/path/joinpath.c
+++ b/src/backend/optimizer/path/joinpath.c
@@ -1124,7 +1124,7 @@ match_unsorted_outer(PlannerInfo *root,
 		if (enable_material && inner_cheapest_total != NULL &&
 			!ExecMaterializesOutput(inner_cheapest_total->pathtype))
 			matpath = (Path *)
-				create_material_path(innerrel, inner_cheapest_total);
+				create_material_path(root, innerrel, inner_cheapest_total);
 	}
 
 	foreach(lc1, outerrel->pathlist)
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 0d00683..f0c8b62 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -29,7 +29,7 @@ static void make_rels_by_clauseless_joins(PlannerInfo *root,
 static bool has_join_restriction(PlannerInfo *root, RelOptInfo *rel);
 static bool has_legal_joinclause(PlannerInfo *root, RelOptInfo *rel);
 static bool is_dummy_rel(RelOptInfo *rel);
-static void mark_dummy_rel(RelOptInfo *rel);
+static void mark_dummy_rel(PlannerInfo *root, RelOptInfo *rel);
 static bool restriction_is_constant_false(List *restrictlist,
 							  bool only_pushed_down);
 
@@ -748,7 +748,7 @@ make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 			if (is_dummy_rel(rel1) || is_dummy_rel(rel2) ||
 				restriction_is_constant_false(restrictlist, false))
 			{
-				mark_dummy_rel(joinrel);
+				mark_dummy_rel(root, joinrel);
 				break;
 			}
 			add_paths_to_joinrel(root, joinrel, rel1, rel2,
@@ -762,12 +762,12 @@ make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 			if (is_dummy_rel(rel1) ||
 				restriction_is_constant_false(restrictlist, true))
 			{
-				mark_dummy_rel(joinrel);
+				mark_dummy_rel(root, joinrel);
 				break;
 			}
 			if (restriction_is_constant_false(restrictlist, false) &&
 				bms_is_subset(rel2->relids, sjinfo->syn_righthand))
-				mark_dummy_rel(rel2);
+				mark_dummy_rel(root, rel2);
 			add_paths_to_joinrel(root, joinrel, rel1, rel2,
 								 JOIN_LEFT, sjinfo,
 								 restrictlist);
@@ -779,7 +779,7 @@ make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 			if ((is_dummy_rel(rel1) && is_dummy_rel(rel2)) ||
 				restriction_is_constant_false(restrictlist, true))
 			{
-				mark_dummy_rel(joinrel);
+				mark_dummy_rel(root, joinrel);
 				break;
 			}
 			add_paths_to_joinrel(root, joinrel, rel1, rel2,
@@ -815,7 +815,7 @@ make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 				if (is_dummy_rel(rel1) || is_dummy_rel(rel2) ||
 					restriction_is_constant_false(restrictlist, false))
 				{
-					mark_dummy_rel(joinrel);
+					mark_dummy_rel(root, joinrel);
 					break;
 				}
 				add_paths_to_joinrel(root, joinrel, rel1, rel2,
@@ -838,7 +838,7 @@ make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 				if (is_dummy_rel(rel1) || is_dummy_rel(rel2) ||
 					restriction_is_constant_false(restrictlist, false))
 				{
-					mark_dummy_rel(joinrel);
+					mark_dummy_rel(root, joinrel);
 					break;
 				}
 				add_paths_to_joinrel(root, joinrel, rel1, rel2,
@@ -853,12 +853,12 @@ make_join_rel(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2)
 			if (is_dummy_rel(rel1) ||
 				restriction_is_constant_false(restrictlist, true))
 			{
-				mark_dummy_rel(joinrel);
+				mark_dummy_rel(root, joinrel);
 				break;
 			}
 			if (restriction_is_constant_false(restrictlist, false) &&
 				bms_is_subset(rel2->relids, sjinfo->syn_righthand))
-				mark_dummy_rel(rel2);
+				mark_dummy_rel(root, rel2);
 			add_paths_to_joinrel(root, joinrel, rel1, rel2,
 								 JOIN_ANTI, sjinfo,
 								 restrictlist);
@@ -1168,27 +1168,14 @@ is_dummy_rel(RelOptInfo *rel)
  * During GEQO planning, this can get invoked more than once on the same
  * baserel struct, so it's worth checking to see if the rel is already marked
  * dummy.
- *
- * Also, when called during GEQO join planning, we are in a short-lived
- * memory context.  We must make sure that the dummy path attached to a
- * baserel survives the GEQO cycle, else the baserel is trashed for future
- * GEQO cycles.  On the other hand, when we are marking a joinrel during GEQO,
- * we don't want the dummy path to clutter the main planning context.  Upshot
- * is that the best solution is to explicitly make the dummy path in the same
- * context the given RelOptInfo is in.
  */
 static void
-mark_dummy_rel(RelOptInfo *rel)
+mark_dummy_rel(PlannerInfo *root, RelOptInfo *rel)
 {
-	MemoryContext oldcontext;
-
 	/* Already marked? */
 	if (is_dummy_rel(rel))
 		return;
 
-	/* No, so choose correct context to make the dummy path in */
-	oldcontext = MemoryContextSwitchTo(GetMemoryChunkContext(rel));
-
 	/* Set dummy size estimate */
 	rel->rows = 0;
 
@@ -1197,12 +1184,10 @@ mark_dummy_rel(RelOptInfo *rel)
 	rel->partial_pathlist = NIL;
 
 	/* Set up the dummy path */
-	add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0));
+	add_path(rel, (Path *) create_append_path(root, rel, NIL, NULL, 0));
 
 	/* Set or update cheapest_total_path and related fields */
 	set_cheapest(rel);
-
-	MemoryContextSwitchTo(oldcontext);
 }
 
 
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index ca0ae78..b74e887 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -54,6 +54,7 @@
 #include "utils/rel.h"
 #include "utils/selfuncs.h"
 #include "utils/lsyscache.h"
+#include "utils/memutils.h"
 #include "utils/syscache.h"
 
 
@@ -218,6 +219,9 @@ standard_planner(Query *parse, int cursorOptions, ParamListInfo boundParams)
 	glob->lastPlanNodeId = 0;
 	glob->transientPlan = false;
 	glob->dependsOnRole = false;
+	glob->path_cxt = AllocSetContextCreate(CurrentMemoryContext,
+										   "Path creation temporary context",
+										   ALLOCSET_DEFAULT_SIZES);
 
 	/*
 	 * Assess whether it's feasible to use parallel mode for this query. We
@@ -414,6 +418,13 @@ standard_planner(Query *parse, int cursorOptions, ParamListInfo boundParams)
 	result->stmt_location = parse->stmt_location;
 	result->stmt_len = parse->stmt_len;
 
+	/*
+	 * We do not need paths any more, blow those away.
+	 * TODO: probably we should also set the pathlists to NIL.
+	 */
+	MemoryContextResetAndDeleteChildren(glob->path_cxt);
+	glob->path_cxt = NULL;
+
 	return result;
 }
 
@@ -1302,7 +1313,7 @@ inheritance_planner(PlannerInfo *root)
 	 */
 	if (subpaths == NIL)
 	{
-		set_dummy_rel_pathlist(final_rel);
+		set_dummy_rel_pathlist(root, final_rel);
 		return;
 	}
 
@@ -3334,7 +3345,7 @@ create_grouping_paths(PlannerInfo *root,
 				paths = lappend(paths, path);
 			}
 			path = (Path *)
-				create_append_path(grouped_rel,
+				create_append_path(root, grouped_rel,
 								   paths,
 								   NULL,
 								   0);
@@ -5261,6 +5272,7 @@ plan_cluster_use_sort(Oid tableOid, Oid indexOid)
 	query->commandType = CMD_SELECT;
 
 	glob = makeNode(PlannerGlobal);
+	glob->path_cxt = CurrentMemoryContext;
 
 	root = makeNode(PlannerInfo);
 	root->parse = query;
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index 1389db1..492f8df 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -566,7 +566,7 @@ generate_union_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NULL, 0);
+	path = (Path *) create_append_path(root, result_rel, pathlist, NULL, 0);
 
 	/* We have to manually jam the right tlist into the path; ick */
 	path->pathtarget = create_pathtarget(root, tlist);
@@ -678,7 +678,7 @@ generate_nonunion_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NULL, 0);
+	path = (Path *) create_append_path(root, result_rel, pathlist, NULL, 0);
 
 	/* We have to manually jam the right tlist into the path; ick */
 	path->pathtarget = create_pathtarget(root, tlist);
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 3248296..3f1184c 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -930,6 +930,27 @@ add_partial_path_precheck(RelOptInfo *parent_rel, Cost total_cost,
  *****************************************************************************/
 
 /*
+ * The memory consumed by paths remains occupied after the plan is created.
+ * For an un-PREPAREd query, it means that the paths continue to occupy the
+ * memory till the end of the execution. Since plan node copies the relevant
+ * information from the corresponding path, paths are not needed after plan is
+ * created. Following macro creates specified Path node in a temporary memory
+ * context allocated while creating PlannerGlobal.  This memory context is
+ * reset at the end of planning to free up any memory consumed by paths.
+ */
+#define makePathNode(root, _type_)		((_type_ *) newPath((root), sizeof(_type_),T_##_type_))
+static Path *
+newPath(PlannerInfo *root, size_t size, NodeTag tag)
+{
+	Path	   *result;
+
+	Assert(sizeof(Path) <= size);
+	result = MemoryContextAllocZero(root->glob->path_cxt, size);
+	result->type = tag;
+	return result;
+}
+
+/*
  * create_seqscan_path
  *	  Creates a path corresponding to a sequential scan, returning the
  *	  pathnode.
@@ -938,7 +959,7 @@ Path *
 create_seqscan_path(PlannerInfo *root, RelOptInfo *rel,
 					Relids required_outer, int parallel_workers)
 {
-	Path	   *pathnode = makeNode(Path);
+	Path	   *pathnode = makePathNode(root, Path);
 
 	pathnode->pathtype = T_SeqScan;
 	pathnode->parent = rel;
@@ -962,7 +983,7 @@ create_seqscan_path(PlannerInfo *root, RelOptInfo *rel,
 Path *
 create_samplescan_path(PlannerInfo *root, RelOptInfo *rel, Relids required_outer)
 {
-	Path	   *pathnode = makeNode(Path);
+	Path	   *pathnode = makePathNode(root, Path);
 
 	pathnode->pathtype = T_SampleScan;
 	pathnode->parent = rel;
@@ -1018,7 +1039,7 @@ create_index_path(PlannerInfo *root,
 				  double loop_count,
 				  bool partial_path)
 {
-	IndexPath  *pathnode = makeNode(IndexPath);
+	IndexPath  *pathnode = makePathNode(root, IndexPath);
 	RelOptInfo *rel = index->rel;
 	List	   *indexquals,
 			   *indexqualcols;
@@ -1070,7 +1091,7 @@ create_bitmap_heap_path(PlannerInfo *root,
 						Relids required_outer,
 						double loop_count)
 {
-	BitmapHeapPath *pathnode = makeNode(BitmapHeapPath);
+	BitmapHeapPath *pathnode = makePathNode(root, BitmapHeapPath);
 
 	pathnode->path.pathtype = T_BitmapHeapScan;
 	pathnode->path.parent = rel;
@@ -1100,7 +1121,7 @@ create_bitmap_and_path(PlannerInfo *root,
 					   RelOptInfo *rel,
 					   List *bitmapquals)
 {
-	BitmapAndPath *pathnode = makeNode(BitmapAndPath);
+	BitmapAndPath *pathnode = makePathNode(root, BitmapAndPath);
 
 	pathnode->path.pathtype = T_BitmapAnd;
 	pathnode->path.parent = rel;
@@ -1136,7 +1157,7 @@ create_bitmap_or_path(PlannerInfo *root,
 					  RelOptInfo *rel,
 					  List *bitmapquals)
 {
-	BitmapOrPath *pathnode = makeNode(BitmapOrPath);
+	BitmapOrPath *pathnode = makePathNode(root, BitmapOrPath);
 
 	pathnode->path.pathtype = T_BitmapOr;
 	pathnode->path.parent = rel;
@@ -1171,7 +1192,7 @@ TidPath *
 create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
 					Relids required_outer)
 {
-	TidPath    *pathnode = makeNode(TidPath);
+	TidPath    *pathnode = makePathNode(root, TidPath);
 
 	pathnode->path.pathtype = T_TidScan;
 	pathnode->path.parent = rel;
@@ -1199,10 +1220,13 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
  * Note that we must handle subpaths = NIL, representing a dummy access path.
  */
 AppendPath *
-create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
+create_append_path(PlannerInfo *root,
+				   RelOptInfo *rel,
+				   List *subpaths,
+				   Relids required_outer,
 				   int parallel_workers)
 {
-	AppendPath *pathnode = makeNode(AppendPath);
+	AppendPath *pathnode = makePathNode(root, AppendPath);
 	ListCell   *l;
 
 	pathnode->path.pathtype = T_Append;
@@ -1259,7 +1283,7 @@ create_merge_append_path(PlannerInfo *root,
 						 List *pathkeys,
 						 Relids required_outer)
 {
-	MergeAppendPath *pathnode = makeNode(MergeAppendPath);
+	MergeAppendPath *pathnode = makePathNode(root, MergeAppendPath);
 	Cost		input_startup_cost;
 	Cost		input_total_cost;
 	ListCell   *l;
@@ -1346,7 +1370,7 @@ ResultPath *
 create_result_path(PlannerInfo *root, RelOptInfo *rel,
 				   PathTarget *target, List *resconstantqual)
 {
-	ResultPath *pathnode = makeNode(ResultPath);
+	ResultPath *pathnode = makePathNode(root, ResultPath);
 
 	pathnode->path.pathtype = T_Result;
 	pathnode->path.parent = rel;
@@ -1382,9 +1406,9 @@ create_result_path(PlannerInfo *root, RelOptInfo *rel,
  *	  pathnode.
  */
 MaterialPath *
-create_material_path(RelOptInfo *rel, Path *subpath)
+create_material_path(PlannerInfo *root, RelOptInfo *rel, Path *subpath)
 {
-	MaterialPath *pathnode = makeNode(MaterialPath);
+	MaterialPath *pathnode = makePathNode(root, MaterialPath);
 
 	Assert(subpath->parent == rel);
 
@@ -1451,7 +1475,7 @@ create_unique_path(PlannerInfo *root, RelOptInfo *rel, Path *subpath,
 	 */
 	oldcontext = MemoryContextSwitchTo(root->planner_cxt);
 
-	pathnode = makeNode(UniquePath);
+	pathnode = makePathNode(root, UniquePath);
 
 	pathnode->path.pathtype = T_Unique;
 	pathnode->path.parent = rel;
@@ -1667,7 +1691,7 @@ GatherPath *
 create_gather_path(PlannerInfo *root, RelOptInfo *rel, Path *subpath,
 				   PathTarget *target, Relids required_outer, double *rows)
 {
-	GatherPath *pathnode = makeNode(GatherPath);
+	GatherPath *pathnode = makePathNode(root, GatherPath);
 
 	Assert(subpath->parallel_safe);
 
@@ -1705,7 +1729,7 @@ SubqueryScanPath *
 create_subqueryscan_path(PlannerInfo *root, RelOptInfo *rel, Path *subpath,
 						 List *pathkeys, Relids required_outer)
 {
-	SubqueryScanPath *pathnode = makeNode(SubqueryScanPath);
+	SubqueryScanPath *pathnode = makePathNode(root, SubqueryScanPath);
 
 	pathnode->path.pathtype = T_SubqueryScan;
 	pathnode->path.parent = rel;
@@ -1733,7 +1757,7 @@ Path *
 create_functionscan_path(PlannerInfo *root, RelOptInfo *rel,
 						 List *pathkeys, Relids required_outer)
 {
-	Path	   *pathnode = makeNode(Path);
+	Path	   *pathnode = makePathNode(root, Path);
 
 	pathnode->pathtype = T_FunctionScan;
 	pathnode->parent = rel;
@@ -1759,7 +1783,7 @@ Path *
 create_valuesscan_path(PlannerInfo *root, RelOptInfo *rel,
 					   Relids required_outer)
 {
-	Path	   *pathnode = makeNode(Path);
+	Path	   *pathnode = makePathNode(root, Path);
 
 	pathnode->pathtype = T_ValuesScan;
 	pathnode->parent = rel;
@@ -1784,7 +1808,7 @@ create_valuesscan_path(PlannerInfo *root, RelOptInfo *rel,
 Path *
 create_ctescan_path(PlannerInfo *root, RelOptInfo *rel, Relids required_outer)
 {
-	Path	   *pathnode = makeNode(Path);
+	Path	   *pathnode = makePathNode(root, Path);
 
 	pathnode->pathtype = T_CteScan;
 	pathnode->parent = rel;
@@ -1810,7 +1834,7 @@ Path *
 create_worktablescan_path(PlannerInfo *root, RelOptInfo *rel,
 						  Relids required_outer)
 {
-	Path	   *pathnode = makeNode(Path);
+	Path	   *pathnode = makePathNode(root, Path);
 
 	pathnode->pathtype = T_WorkTableScan;
 	pathnode->parent = rel;
@@ -1849,7 +1873,7 @@ create_foreignscan_path(PlannerInfo *root, RelOptInfo *rel,
 						Path *fdw_outerpath,
 						List *fdw_private)
 {
-	ForeignPath *pathnode = makeNode(ForeignPath);
+	ForeignPath *pathnode = makePathNode(root, ForeignPath);
 
 	pathnode->path.pathtype = T_ForeignScan;
 	pathnode->path.parent = rel;
@@ -1955,7 +1979,7 @@ create_nestloop_path(PlannerInfo *root,
 					 List *pathkeys,
 					 Relids required_outer)
 {
-	NestPath   *pathnode = makeNode(NestPath);
+	NestPath   *pathnode = makePathNode(root, NestPath);
 	Relids		inner_req_outer = PATH_REQ_OUTER(inner_path);
 
 	/*
@@ -2045,7 +2069,7 @@ create_mergejoin_path(PlannerInfo *root,
 					  List *outersortkeys,
 					  List *innersortkeys)
 {
-	MergePath  *pathnode = makeNode(MergePath);
+	MergePath  *pathnode = makePathNode(root, MergePath);
 
 	pathnode->jpath.path.pathtype = T_MergeJoin;
 	pathnode->jpath.path.parent = joinrel;
@@ -2107,7 +2131,7 @@ create_hashjoin_path(PlannerInfo *root,
 					 Relids required_outer,
 					 List *hashclauses)
 {
-	HashPath   *pathnode = makeNode(HashPath);
+	HashPath   *pathnode = makePathNode(root, HashPath);
 
 	pathnode->jpath.path.pathtype = T_HashJoin;
 	pathnode->jpath.path.parent = joinrel;
@@ -2164,7 +2188,7 @@ create_projection_path(PlannerInfo *root,
 					   Path *subpath,
 					   PathTarget *target)
 {
-	ProjectionPath *pathnode = makeNode(ProjectionPath);
+	ProjectionPath *pathnode = makePathNode(root, ProjectionPath);
 	PathTarget *oldtarget = subpath->pathtarget;
 
 	pathnode->path.pathtype = T_Result;
@@ -2331,7 +2355,7 @@ create_set_projection_path(PlannerInfo *root,
 						   Path *subpath,
 						   PathTarget *target)
 {
-	ProjectSetPath *pathnode = makeNode(ProjectSetPath);
+	ProjectSetPath *pathnode = makePathNode(root, ProjectSetPath);
 	double		tlist_rows;
 	ListCell   *lc;
 
@@ -2399,7 +2423,7 @@ create_sort_path(PlannerInfo *root,
 				 List *pathkeys,
 				 double limit_tuples)
 {
-	SortPath   *pathnode = makeNode(SortPath);
+	SortPath   *pathnode = makePathNode(root, SortPath);
 
 	pathnode->path.pathtype = T_Sort;
 	pathnode->path.parent = rel;
@@ -2445,7 +2469,7 @@ create_group_path(PlannerInfo *root,
 				  List *qual,
 				  double numGroups)
 {
-	GroupPath  *pathnode = makeNode(GroupPath);
+	GroupPath  *pathnode = makePathNode(root, GroupPath);
 
 	pathnode->path.pathtype = T_Group;
 	pathnode->path.parent = rel;
@@ -2501,7 +2525,7 @@ create_upper_unique_path(PlannerInfo *root,
 						 int numCols,
 						 double numGroups)
 {
-	UpperUniquePath *pathnode = makeNode(UpperUniquePath);
+	UpperUniquePath *pathnode = makePathNode(root, UpperUniquePath);
 
 	pathnode->path.pathtype = T_Unique;
 	pathnode->path.parent = rel;
@@ -2558,7 +2582,7 @@ create_agg_path(PlannerInfo *root,
 				const AggClauseCosts *aggcosts,
 				double numGroups)
 {
-	AggPath    *pathnode = makeNode(AggPath);
+	AggPath    *pathnode = makePathNode(root, AggPath);
 
 	pathnode->path.pathtype = T_Agg;
 	pathnode->path.parent = rel;
@@ -2623,7 +2647,7 @@ create_groupingsets_path(PlannerInfo *root,
 						 const AggClauseCosts *agg_costs,
 						 double numGroups)
 {
-	GroupingSetsPath *pathnode = makeNode(GroupingSetsPath);
+	GroupingSetsPath *pathnode = makePathNode(root, GroupingSetsPath);
 	int			numGroupCols;
 
 	/* The topmost generated Plan node will be an Agg */
@@ -2736,7 +2760,7 @@ create_minmaxagg_path(PlannerInfo *root,
 					  List *mmaggregates,
 					  List *quals)
 {
-	MinMaxAggPath *pathnode = makeNode(MinMaxAggPath);
+	MinMaxAggPath *pathnode = makePathNode(root, MinMaxAggPath);
 	Cost		initplan_cost;
 	ListCell   *lc;
 
@@ -2797,7 +2821,7 @@ create_windowagg_path(PlannerInfo *root,
 					  WindowClause *winclause,
 					  List *winpathkeys)
 {
-	WindowAggPath *pathnode = makeNode(WindowAggPath);
+	WindowAggPath *pathnode = makePathNode(root, WindowAggPath);
 
 	pathnode->path.pathtype = T_WindowAgg;
 	pathnode->path.parent = rel;
@@ -2864,7 +2888,7 @@ create_setop_path(PlannerInfo *root,
 				  double numGroups,
 				  double outputRows)
 {
-	SetOpPath  *pathnode = makeNode(SetOpPath);
+	SetOpPath  *pathnode = makePathNode(root, SetOpPath);
 
 	pathnode->path.pathtype = T_SetOp;
 	pathnode->path.parent = rel;
@@ -2924,7 +2948,7 @@ create_recursiveunion_path(PlannerInfo *root,
 						   int wtParam,
 						   double numGroups)
 {
-	RecursiveUnionPath *pathnode = makeNode(RecursiveUnionPath);
+	RecursiveUnionPath *pathnode = makePathNode(root, RecursiveUnionPath);
 
 	pathnode->path.pathtype = T_RecursiveUnion;
 	pathnode->path.parent = rel;
@@ -2963,7 +2987,7 @@ LockRowsPath *
 create_lockrows_path(PlannerInfo *root, RelOptInfo *rel,
 					 Path *subpath, List *rowMarks, int epqParam)
 {
-	LockRowsPath *pathnode = makeNode(LockRowsPath);
+	LockRowsPath *pathnode = makePathNode(root, LockRowsPath);
 
 	pathnode->path.pathtype = T_LockRows;
 	pathnode->path.parent = rel;
@@ -3025,7 +3049,7 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
 						List *rowMarks, OnConflictExpr *onconflict,
 						int epqParam)
 {
-	ModifyTablePath *pathnode = makeNode(ModifyTablePath);
+	ModifyTablePath *pathnode = makePathNode(root, ModifyTablePath);
 	double		total_size;
 	ListCell   *lc;
 
@@ -3120,7 +3144,7 @@ create_limit_path(PlannerInfo *root, RelOptInfo *rel,
 				  Node *limitOffset, Node *limitCount,
 				  int64 offset_est, int64 count_est)
 {
-	LimitPath  *pathnode = makeNode(LimitPath);
+	LimitPath  *pathnode = makePathNode(root, LimitPath);
 
 	pathnode->path.pathtype = T_Limit;
 	pathnode->path.parent = rel;
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index f7ac6f6..aa2e9b3 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -128,6 +128,8 @@ typedef struct PlannerGlobal
 	bool		parallelModeNeeded;		/* parallel mode actually required? */
 
 	char		maxParallelHazard;		/* worst PROPARALLEL hazard level */
+
+	MemoryContext path_cxt;		/* Temporary context for holding paths. */
 } PlannerGlobal;
 
 /* macro for fetching the Plan associated with a SubPlan node */
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 53cad24..18591df 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -62,8 +62,11 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
 					  List *bitmapquals);
 extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
 					List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel, List *subpaths,
-				   Relids required_outer, int parallel_workers);
+extern AppendPath *create_append_path(PlannerInfo *root,
+				   RelOptInfo *rel,
+				   List *subpaths,
+				   Relids required_outer,
+				   int parallel_workers);
 extern MergeAppendPath *create_merge_append_path(PlannerInfo *root,
 						 RelOptInfo *rel,
 						 List *subpaths,
@@ -71,7 +74,8 @@ extern MergeAppendPath *create_merge_append_path(PlannerInfo *root,
 						 Relids required_outer);
 extern ResultPath *create_result_path(PlannerInfo *root, RelOptInfo *rel,
 				   PathTarget *target, List *resconstantqual);
-extern MaterialPath *create_material_path(RelOptInfo *rel, Path *subpath);
+extern MaterialPath *create_material_path(PlannerInfo *root, RelOptInfo *rel,
+							Path *subpath);
 extern UniquePath *create_unique_path(PlannerInfo *root, RelOptInfo *rel,
 				   Path *subpath, SpecialJoinInfo *sjinfo);
 extern GatherPath *create_gather_path(PlannerInfo *root,
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index ebda308..733e40d 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -49,7 +49,7 @@ extern PGDLLIMPORT join_search_hook_type join_search_hook;
 
 
 extern RelOptInfo *make_one_rel(PlannerInfo *root, List *joinlist);
-extern void set_dummy_rel_pathlist(RelOptInfo *rel);
+extern void set_dummy_rel_pathlist(PlannerInfo *root, RelOptInfo *rel);
 extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 					 List *initial_rels);
 
-- 
1.7.9.5

0002-Patch-to-measure-memory-used-in-CurrentMemoryContext.patchapplication/octet-stream; name=0002-Patch-to-measure-memory-used-in-CurrentMemoryContext.patchDownload
From f01a67fe92eb5046d208cfa11578fe753562d8e0 Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Wed, 1 Mar 2017 12:12:11 +0530
Subject: [PATCH 2/3] Patch to measure memory used in CurrentMemoryContext in
 standard_planner().

---
 src/backend/optimizer/plan/planner.c |    5 +++++
 src/backend/utils/mmgr/mcxt.c        |   26 ++++++++++++++++++++++++++
 src/include/utils/memutils.h         |    6 ++++++
 3 files changed, 37 insertions(+)

diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index b74e887..410750d 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -195,6 +195,9 @@ standard_planner(Query *parse, int cursorOptions, ParamListInfo boundParams)
 	Plan	   *top_plan;
 	ListCell   *lp,
 			   *lr;
+	MemoryContextCounters mem_start;
+
+	MemoryContextFuncStatsStart(CurrentMemoryContext, &mem_start, __FUNCTION__);
 
 	/*
 	 * Set up global state for this planner invocation.  This data is needed
@@ -425,6 +428,8 @@ standard_planner(Query *parse, int cursorOptions, ParamListInfo boundParams)
 	MemoryContextResetAndDeleteChildren(glob->path_cxt);
 	glob->path_cxt = NULL;
 
+	MemoryContextFuncStatsEnd(CurrentMemoryContext, &mem_start, __FUNCTION__);
+
 	return result;
 }
 
diff --git a/src/backend/utils/mmgr/mcxt.c b/src/backend/utils/mmgr/mcxt.c
index 6ad0bb4..94b25ac 100644
--- a/src/backend/utils/mmgr/mcxt.c
+++ b/src/backend/utils/mmgr/mcxt.c
@@ -577,6 +577,32 @@ MemoryContextStatsInternal(MemoryContext context, int level,
 	}
 }
 
+void
+MemoryContextFuncStatsStart(MemoryContext context,
+							MemoryContextCounters *start_counts,
+							const char *label)
+{
+	memset(start_counts, 0, sizeof(*start_counts));
+	MemoryContextStatsInternal(context, 0, false, 100, start_counts);
+}
+
+void
+MemoryContextFuncStatsEnd(MemoryContext context,
+						  MemoryContextCounters *start_counts,
+						  const char *label)
+{
+	MemoryContextCounters end_counts;
+	Size	start_used_space = start_counts->totalspace - start_counts->freespace;
+	Size	end_used_space;
+
+	memset(&end_counts, 0, sizeof(end_counts));
+	MemoryContextStatsInternal(context, 0, false, 100, &end_counts);
+	end_used_space = end_counts.totalspace - end_counts.freespace;
+
+	elog(NOTICE, "%s,%s,%zu,%zu,%ld", label, context->name,
+		 start_used_space, end_used_space, end_used_space - start_used_space);
+}
+
 /*
  * MemoryContextCheck
  *		Check all chunks in the named context.
diff --git a/src/include/utils/memutils.h b/src/include/utils/memutils.h
index 1d1035e..7d605b1 100644
--- a/src/include/utils/memutils.h
+++ b/src/include/utils/memutils.h
@@ -122,6 +122,12 @@ extern MemoryContext MemoryContextCreate(NodeTag tag, Size size,
 					MemoryContextMethods *methods,
 					MemoryContext parent,
 					const char *name);
+extern void MemoryContextFuncStatsStart(MemoryContext context,
+										MemoryContextCounters *start_counts,
+										const char *func_label);
+extern void MemoryContextFuncStatsEnd(MemoryContext context,
+									  MemoryContextCounters *start_counts,
+									  const char *func_label);
 
 
 /*
-- 
1.7.9.5

0003-GUC-zap_path-to-enable-freeing-memory-consumed-by-pa.patchapplication/octet-stream; name=0003-GUC-zap_path-to-enable-freeing-memory-consumed-by-pa.patchDownload
From 9c863c57c585303f2ba7b5ce4d892c723210b098 Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Wed, 1 Mar 2017 13:29:51 +0530
Subject: [PATCH 3/3] GUC zap_path to enable freeing memory consumed by paths.

For measuring the memory saved, add a GUC zap_path to enable/disable
freeing memory occupied by paths.
---
 src/backend/optimizer/path/costsize.c |    1 +
 src/backend/optimizer/plan/planner.c  |   15 +++++++++------
 src/backend/utils/misc/guc.c          |   10 ++++++++++
 src/include/optimizer/cost.h          |    1 +
 4 files changed, 21 insertions(+), 6 deletions(-)

diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index c138f57..f4dfa00 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -126,6 +126,7 @@ bool		enable_nestloop = true;
 bool		enable_material = true;
 bool		enable_mergejoin = true;
 bool		enable_hashjoin = true;
+bool		zap_paths = true;
 
 typedef struct
 {
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 410750d..c7b085f 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -421,12 +421,15 @@ standard_planner(Query *parse, int cursorOptions, ParamListInfo boundParams)
 	result->stmt_location = parse->stmt_location;
 	result->stmt_len = parse->stmt_len;
 
-	/*
-	 * We do not need paths any more, blow those away.
-	 * TODO: probably we should also set the pathlists to NIL.
-	 */
-	MemoryContextResetAndDeleteChildren(glob->path_cxt);
-	glob->path_cxt = NULL;
+	if (zap_paths)
+	{
+		/*
+		 * We do not need paths any more, blow those away.
+		 * TODO: probably we should also set the pathlists to NIL.
+		 */
+		MemoryContextResetAndDeleteChildren(glob->path_cxt);
+		glob->path_cxt = NULL;
+	}
 
 	MemoryContextFuncStatsEnd(CurrentMemoryContext, &mem_start, __FUNCTION__);
 
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 2477138..630d3f7 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -912,6 +912,16 @@ static struct config_bool ConfigureNamesBool[] =
 		true,
 		NULL, NULL, NULL
 	},
+
+	{
+		{"zap_paths", PGC_USERSET, QUERY_TUNING_GEQO,
+			gettext_noop("Free up the memory used by paths at the end of planning."),
+		},
+		&zap_paths,
+		true,
+		NULL, NULL, NULL
+	},
+
 	{
 		/* Not for general use --- used by SET SESSION AUTHORIZATION */
 		{"is_superuser", PGC_INTERNAL, UNGROUPED,
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 72200fa..fc882ad 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -67,6 +67,7 @@ extern bool enable_material;
 extern bool enable_mergejoin;
 extern bool enable_hashjoin;
 extern int	constraint_exclusion;
+extern bool zap_paths;
 
 extern double clamp_row_est(double nrows);
 extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
-- 
1.7.9.5

#38Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#36)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Mar 1, 2017 at 3:56 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

2. If the PartitionJoinPath emerges as the best path, we create paths
for each of the remaining child-joins. Then we collect paths with
properties same as the given PartitionJoinPath, one from each
child-join. These paths are converted into plans and a Merge/Append
plan is created combing these plans. The paths and plans for
child-join are created in a temporary memory context. The final plan
for each child-join is copied into planner's context and the temporary
memory context is reset.

Robert and I discussed this in more detail. Path creation code may
allocate objects other than paths. postgres_fdw, for example,
allocates character array to hold the name of relation being
pushed-down. When the temporary context gets zapped after creating
paths for a given child-join, those other objects also gets thrown
away. Attached patch has implemented the idea that came out of the
discussion.

We create a memory context for holding paths at the time of creating
PlannerGlobal and save it in PlannerGlobal. The patch introduces a new
macro makePathNode() which allocates the memory for given type of path
from this context. Every create_*_path function has been changed to
use this macro instead of makeNode(). In standard_planner(), at the
end of planning we destroy the memory context freeing all the paths
allocated. While creating a plan node, planner copies everything
required by the plan from the path, so the path is not needed any
more. So, freeing corresponding memory should not have any adverse
effects.

Most of the create_*_path() functions accept root as an argument, thus
the temporary path context is available through root->glob everywhere.
An exception is create_append_path() which does not accept root as an
argument. The patch changes create_append_path() and its callers like
set_dummy_rel_pathlist(), mark_dummy_rel() to accept root as an
argument. Ideally paths are not required after creating plan, so we
should be
able to free the context right after the call to create_plan(). But we
need dummy paths while creating flat rtable in
set_plan_references()->add_rtes_to_flat_rtable(). We used to So free
the path context at the end of planning cycle. Now that we are
allocating all the paths in a different memory context, it doesn't
make sense to switch context in mark_dummy_rel().

0001 patch implements the idea described above.
0002 patch adds instrumentation to measure memory consumed in
standard_planner() call.
0003 patch adds a GUC zap_paths to enable/disable destroying path context.
The last two patches are for testing only.

Attached also find the SQL script and its output showing the memory
saved. For a 5 way self-join of pg_class, the total memory consumed in
standard_planner() is 760K without patch and with patch it comes down
to 713K, saving 47K memory otherwise occupied by paths. It looks like
something useful even without partition-wise joins.

Of course, that's not a lot, but the savings will be a lot better for
partition-wise joins. Do you have a set of patches for that feature
that apply on top of 0001?

--
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

#39Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#38)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

PFA the zip containing all the patches rebased on
56018bf26eec1a0b4bf20303c98065a8eb1b0c5d and contain the patch to free
memory consumed by paths using a separate path context.

There are some more changes wrt earlier set of patches
1. Since we don't need a separate context for planning for each
child_join, changed code in create_partition_join_plan() to not do
that. The function collects all child_join paths into merge/append
path and calls create_plan_recurse() on that path instead of
converting each child_join path to plan one at a time.

2. Changed optimizer/README and some comments referring to temporary
memory context, since we do not use that anymore.

3. reparameterize_path_by_child() is fixed to translate the merge and
hash clause in Hash/Merge path.

On Thu, Mar 9, 2017 at 6:44 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Wed, Mar 1, 2017 at 3:56 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

2. If the PartitionJoinPath emerges as the best path, we create paths
for each of the remaining child-joins. Then we collect paths with
properties same as the given PartitionJoinPath, one from each
child-join. These paths are converted into plans and a Merge/Append
plan is created combing these plans. The paths and plans for
child-join are created in a temporary memory context. The final plan
for each child-join is copied into planner's context and the temporary
memory context is reset.

Robert and I discussed this in more detail. Path creation code may
allocate objects other than paths. postgres_fdw, for example,
allocates character array to hold the name of relation being
pushed-down. When the temporary context gets zapped after creating
paths for a given child-join, those other objects also gets thrown
away. Attached patch has implemented the idea that came out of the
discussion.

We create a memory context for holding paths at the time of creating
PlannerGlobal and save it in PlannerGlobal. The patch introduces a new
macro makePathNode() which allocates the memory for given type of path
from this context. Every create_*_path function has been changed to
use this macro instead of makeNode(). In standard_planner(), at the
end of planning we destroy the memory context freeing all the paths
allocated. While creating a plan node, planner copies everything
required by the plan from the path, so the path is not needed any
more. So, freeing corresponding memory should not have any adverse
effects.

Most of the create_*_path() functions accept root as an argument, thus
the temporary path context is available through root->glob everywhere.
An exception is create_append_path() which does not accept root as an
argument. The patch changes create_append_path() and its callers like
set_dummy_rel_pathlist(), mark_dummy_rel() to accept root as an
argument. Ideally paths are not required after creating plan, so we
should be
able to free the context right after the call to create_plan(). But we
need dummy paths while creating flat rtable in
set_plan_references()->add_rtes_to_flat_rtable(). We used to So free
the path context at the end of planning cycle. Now that we are
allocating all the paths in a different memory context, it doesn't
make sense to switch context in mark_dummy_rel().

0001 patch implements the idea described above.
0002 patch adds instrumentation to measure memory consumed in
standard_planner() call.
0003 patch adds a GUC zap_paths to enable/disable destroying path context.
The last two patches are for testing only.

Attached also find the SQL script and its output showing the memory
saved. For a 5 way self-join of pg_class, the total memory consumed in
standard_planner() is 760K without patch and with patch it comes down
to 713K, saving 47K memory otherwise occupied by paths. It looks like
something useful even without partition-wise joins.

Of course, that's not a lot, but the savings will be a lot better for
partition-wise joins. Do you have a set of patches for that feature
that apply on top of 0001?

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

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

Attachments:

pg_dp_join_patches_v2.zipapplication/zip; name=pg_dp_join_patches_v2.zipDownload
PK
��jJpg_dp_join_patches_v2/UX���X���X�PK��jJEpg_dp_join_patches_v2/0001-Free-up-memory-consumed-by-the-paths.patchUX���X���X��]{s����[����D�H��v��V��NI�W��uI�P 0�.���������)�&W�-J@wc�7=�===����s<�����g}(��p8���j9�RN��hi��n�V������
m&��g�O����y
b���pG~���[34���%��'�E2�N(����7/;��H>��'���4����3}�l����d4�����=}z{��j'��o�&���b#7~� ,������ADk)���p��\m#g��]�
$</�{���~��@����xw��S,�������{,����d�c>� ��D>?��=���������o�������+ eN��d�k?p)�������^�����p"�az!����Q"�1�Nr�A���t���_��_�G�M�q<K=��m��|�
�*���lz�p��lLd+T
�k�A �������S����x~$<)mh�o�oA���x�����4D���\i�<" +�����@���|�
t�����<@��Pn��T!�_"}�$�>�s��6�������*�,�t"5$z
@�6;a"v n	~\��A��N�T������m���7�&����a*�@�;A�j�����G������c.]�:9��/p����4�2QI��v����vp �o6�	�>|���=����$�\�@�~����K��
�"��r�����j�R
��&8��J|"R@��-j�nr=��I��SQ�������Y�@������
 ��]������������J�Nl�O� ���.t����;"���i}�&����9A�'=���!�>��������x�C�'�2���F�O��'5>
�;QsY�W�2o-S ��-���X��qC� ��N�^�������~����sc[� sx�$s5X��+��P�����1h��~O�Jn$b�,�����
�L����(�i��q�'�ca����n�c;�Q��;0��I;�Y���8�-���>�&��`:�Cs�O�cT��O�"����?��H��@��.���!2B�HL�[p2�(���'��Z]�n�#�Q6�h���u=vK`�7f�	����K��2Y��%�`W�3�;�	3��s��K4'���62���\'������q�}�S��_�?�8��+q������]��33��(`�*��f����@�kT���{�E`���;�Mh�a[� \;[j Xc�5�`Z�t��^��.o�����\��^�����x�����6�\�0;=�=�5~	7���+0�0��%�m��?�||�����p��y�M�
�2pL��!ub���$����=|G��#�G�G���B��nA����X7��&�N,��C�d����FR�>��9���>\\�#N���2����3r�f�����a^��pa��Hg����S�mV�-JN�W�
!e�4���u��m�c�`��o�4��$M$��A��m����?H�7l���MF`Y!��s��Y�j����C��w��a�1�g't0H�h�C<�$�����R�%(J�'�B������-���E"&sRy
��W��c��
o��{�q���u0���d]l^���`���p�#�.�9��h�Gh|6]��"6ES6S��A"�GS�'�]��b2[WF��~	����1g�C����d��,1!"��U�����{o$���S��$j��c5�XM��*���:8
14B����~�	|�x	��n@?��Dg��Q�'+��K�u�
��7*P����7�RD6�.�%�g������f�>VNk<��Q����j�$�
��_������2E�������D�	ZVn�*���g:�X#M��7��{��}x���I�=B�#�����O������C�s����d����%d��6��H��}���k�ci8�����C	m��������j��
�F�
e.0�C�J������
]���0TF�Dd�@�r�}�?Tvy:R���l�������wo/�g�3�/�5{�]��I}�E-��[���)`k���py��l6�v@���CG"��S�����>\^��q}~q�e4�������`t��{���8��Q���X�4��*^.�/q�m���lE��1�S�U[����=�R[�&���������-
�>���'\�9d�����9����J�A�VS`7��H����������:8���i]*&���hG�U�<�����������Jl�+�K$��Jvb��Z{$�DiZ�C��z�p+���4He���%�)
��1��`�Z��T�)I�4H���#��8�#3r,�� DJc�`�
V����J�R�K�w����[g���w�����q+� �	s��]�~����F����7-���L���t������,^,��\0�q���n�������6��P5&mZ��\�y��( ��]��/�|U�����~z(���?�q(���8���������������7R�_Y��h�]�1�S���*�&�>�(����Q�����x�L�&�$�v�r�C���;y������-���v+�B6�C<e��������w������q���~���lFv�?���T�	:����o�=O�|sm������i��&-�u����#��}�#r��l�.��9
������i���}��ih�t��99?�ic�����@������p�V��^��a��|>��$�>/�=-	�(E�I��6�*[��'�C(�a-��L�E��XO i�P%�1�d�0�����&q�!���#�/N���qF�8�R#k9B�k>V�T�bb�H�1n��b�Cn�g�������E����-������ ����5T���!�J�h�jS�lcq���Bh�#�{���t+��m+
�C�#y��=�����[n%E1XH�+Iq���"�dXb����wi�����=t���F��@'�)(.�;��w��7O���$sk�N���s�������g�U�k����������l��l<���#�
��|����CUPfE����(���F����PU���{���:�>�����5���Lh�����������
��4���tg�*�+t�z����v�4��6"U��l<%`��X9��Uj3�	��;��z�_6��h�&�J��j�6�$U�����F���3�U��(�1ST$Em��{Sq�v�:�[Ub$��r(*d����J����n�Zh���q�.�y��Zn�S(���!�Q���Be�<��t��L[��IJ���������\	�
�\hJ,����R�3�;�/]3��.��m��V�M�������nQ����%*ox�]���n�����Ef��8A{�������nF����*���&W<����������pzqquvs~k�:}�����y�������x�T�H����4y���7%{�e�0�{�D$�4Dcm�C4���F��1��*v#�MdP-,��B�����;�r�T�8F �������h8�y�(����K��%�{�b��zu�Lli.��f��0�~�S��Z��`�����Z��g��j<y�%�aQ���Z�nP���	������HS[���*��E�q����y�}�:������<������#�&j�J9�����q(_�#�5\�D]v4�j�5��N�1MyE�����+lU�:����PnC�Ri�^�< 3=ZL�P'�bT��E".0Q
�:�r ����'�7r,W��N�����7���[4Wg��������[6�_pc4��w�3���U��!�p�x.��	d�����l���������F������B_��U+G[��j��4�\�5Ij���'��Xk�=\m�*\�R|q�o{�R�S3��'�/�%ll����e����+f,p�L���,
��o#Dq��T<�Q�GQ���t��G��y�IZN���S��\8�'�N����4\�%3�5����|����d�5
���3 ����������W�S3���n��4���f��%5M�2-j�bD�+~�\T*�2�BP��dS��4Jg~�KQ\�X>m����u�c�a���sk�<wv��#a�8;��PqN
�k����������d$����D�y2��;/�=�o��<=�n���4����-3�sY\r�H������
y �V�ttH5-��2w�p
,9�k~�uk����om	��$g�� �����:�{ppx�?���=�M�G	���[��o���t��-X�8��H�#�0]��[�b�����"FNGI|Ui�j�t�_�2�L[N��r�/2����b��+��B�#*h_-D���bS�E�*c��',1'[����gJg��HY^-1�	+�HU�"3Zu|�:*����H"ru����T0W5�u����5��w
-qq�Sq�	�tp��bCS���*)���� aV�,���?�@���(���}�	�)��Y�G����Q)�JgT%CTb�/B���@m�q��p�(@H�����EV�1�\������������-�%���H� 
�%L�|��46�B���)���
�.�a�Z��Y�&(�4Fi�PZ:�����l�����������v��w$�{���Y�X��`��V�nG��b
��1TiQ���r�+4�`�?�[w�(�e�h�;�|T3|�$�^S%x{;���%l#>��M�U���^�jAk��*�S��a�,en��U����p-�?�u��e��F?��3N��35T!�<�����]m	��kf@�~b��!�Dk����WCJ�F]$���'�g>E�����@n1�����Z��0�MmW���@W���X.�4+;�Q�t�+dU���:ct\�E���U�U��)��w�J����R
�%�*0�W<��N���j���M�\�>�Fw{1U~PA�T��C�`m��.��*L�KX����G���#VAAoi�_��%*`�,�X^��C2���(�.�z\��A1���9�N2+@bR���Va�x��}B���b�q�~]'{!-��u�m�S�G#VZ���I�U��N�p���aje��})r���%A���&U�|�������Ef�d!����/��=����]�xV���=��O�t/��jQ��6&�j�M�|m����fP
������i�L�5�l���7&��K���.����t�Z�a���|�����
�%p�5���14W2���:=?[�uC���9���5U8�:�F'���*��T��^��B��j����.�)Dm^��J^�������������c�R+�1��|6�D�����>��p�_�S?�A��(�����E	N+����;��e����V�R�p���<i��;A��r�]i�b�sm1��:wO3D����g
��m���gx��5?�����&�fE�}V]Q�6��pL��>�#7O�e-k�KlR���Q���h�y�V%�	��P96�25��p��-�������?Wn���Q�m������T�Dk[:3�
=k}
������f~m�]�_�f�}=u9������T���D��>�'W
�
��Ri)��VL`��p����I�r(�"'�}W��*�O�5�p����
���>��Y�Mn�\��4e�+����?����z�U�a���V��e� � X��u'��?�<i�4 �������Mz�h[(k�l��lH�i�`��U��	j�����s\�����d�����4A��N�����Jc��:'2��P��l���}^������]�i;G8Q�~RU��oK���dZy���7oPB���_�{z�W���%L�|�@!����y2M��d��[,cmL��3���������5�|@�Y���A������������\��O���u���-������3rm�O��a9A�:�R��f����vA=m	��k����];tfdy���h�;��,u�W�j�2�P�+��0��a�����Y��Z57�B��C��V�Q��*������n��0)r����K����n)�\��)R�u���}>�#V���_��^j�6�
��5�����/����hRu��'�x�������
'�z���������.��8���l��
"��UL�'��)�r����E���N���if*�Sf�7=S�v�t���E)���oU����7|zO�u�a�������P���*����
H�
��P#pF�Q�@�))�����7����#:8��\c������}�Nm��e/�W)h4�gj	�d����K���O���4���e�o�;-��F�i�B�����h ��T��� �YJ���������S�p�x��J��vV���G"xP���;����X���m�8�R�j�J�^R(���6�D�8�0�]����2�v����w���z}�n�Di��8��=�x��Z�B�~�]:+TQj������!�<�P� ^HYlfz�u�����9��j��{ur��H�__X6�NN]O��j8&�t.�����Z���Z�l��A����bm�R��(�C���5{k/��|Gg����'rM+�:Iv���R���{P��W��7%��`����zz}zqq~!�tW��3��z��U�h��p�p[>�L��������
6��b����uD���!#k���#
K�0�-��>8x�w$�����@���X�%Q�0������x:�HY�
�Bv����N��@�����F~2�D���u�������V1>�2O=�T�W����������Ph�Gpk+��+��>�{�oP����ih2:;�dO)}������e��c��+7�R%���K�~���L#�%LO)������6�I��P~L�X>��R�b������.�/01��*��|�M�"���9�c���*�������6,#g��r��$������c�_S}��������W�����U3�����?�������r������d�+�XmQ�:��+������3�l�j�:��>����u�J;
$B�OM�6�0���+B.����6�
�I���PK'Jq��qPK��jJApg_dp_join_patches_v2/0002-Refactor-set_append_rel_pathlist.patchUX���X���X��W�r�6}�b�'��H�%�r�L�4n<�[�������$(2��:i�����Rw�#����`��J����$>���4g�i��OGg'�8=�	?��d��q�O�������hta�x4��+4s��h����jf�G��G	�?��pU�R�,�R�x�����+UpW<Ach:_�''g��&'#�}�|������]~x�F��x�����F*���X]s���f���Jm"�;0�@�8��@Cr��YZ@Z�U���Se"Sp�u����]8������n�ya@�;���	�|9��L@�n���� i���@����*�C���,]l>W|�����?^���*���4�����7�6�Tf���u��Nx�h��<���J9�����
����M�(���.��$NU�M�P|����%).����W(E
��A�����e�h������S��"nl�*6X~��G�?�"� �5�^�h��~BcCY�rQ~�jHn�����R�&1���=���i���g�c�Ese}��A�2^q7<���0�c�l��x����0k�'��f�K��8g<I���t<&������3��ip
>�9�aO�V��&��pu?�]�>7���J�p������Fi���
U����:�v��u����
����z!�+�e}m���� s�������A54[�+S7F��U�����%z�d�_7w�p�UL��}R�K�����2U%��|�)p�9���� �"���8�c�n|�s��Pb+�wlN����������Ho���\�z�����#��T"e��u��[V!VO���o��L���!���-a��-����@W�����%.����6_�|���/����\����/�5�,�b��a"����g5�c����R����2�2�c(sG��,�Re\!w����,+���B��e��������F���nYY�Y�S:���F�>�T��Y,���Y����)�~UT�t������kK����
uz�<����V�
�-R:����/�����r�6�9��I�Q�V:��0�w{��Q�?�r.��&V��W���E%:��~"SX�?<������9=��w	�����R5l@�Iw�w1��>�iRY ��V�.r�lF;M_����AW*�{{>~"�����n��j�/��m<{.]����T�<%;T�[���M�S��7As����H�����#��>>`
�L*�
/��������]�1�d]K�,{w"���7\M��"�&-v���������:����W��,D_,
m9w�X����-����B�^�)����;D���4��[f�g��in����������
��ma��]���	��a��hS��mas���~U][�l���t����uH���Z��v$k�����E����w�g�P�j�Z^*�Y�_������m��QSC#�r�~��
�4m����w�:��r/���i4��PKE���PK��jJ7pg_dp_join_patches_v2/0003-Refactor-make_join_rel.patchUX���X���X��UMo�6=��b�@��lI��[,����-Z�"��(JYLdR��n�����8���9�R'�A�p����E��0	1N��4	E$�9�'K1O9Ox4��/g8�3�E+��������8#vAaV�����&��y�-|���(q�sT���E2J��=��-����>L�
F���j�&g���I����S��_�~�����q4�
.1���l�-�o�T�
��`��u��j� ���.����3���T\F�8l�*r���9�6{')W� m5�r�C��B���>�i���-(t	���
�Z�����(i����Iq�~^mtX�UIIK]���u&���������]��C�� 4���%l-���K%���=�5�P^+���Z�T7bA00U:Nxz�J�ui�V�����wh�(�� ^���'hBE�����s�A�C<��V
�}o�����cBf�FZ���!I�����;da�L��h��Pds�(���+��������Lb}�-��+D��.@�����]��K,���G�i�z�-{���R���n�y-�RXu�Z[^�$���J67sM�S��re�/��t���>�|��m�����M�b-����s���������Y�C�.���U����O��z�I�>���8q�a��E<u�Y�?^����[@��G[���tP)i���z��><W7��2����^���&o�yd������j���l�:���))Lg9�=�m����k�uv�#'�����7�s��p��N���v����wR��	�\x�k�D W�?��E7�������������V���eY��'$]���x�1�!|g���C��?��R��G��H��U����"��c���
�����'G�t5�>aIZ���j#����}zJ]����tOc�(nB4��r�[�"=����I]��~��<sb�Nj�����r����HL�F���'�+�F���Oo�������:��
�T�o�AX���9��4<4�D��F���h���PK��DA��	PK��jJ@pg_dp_join_patches_v2/0004-Refactor-adjust_appendrel_attrs.patchUX���X���X��Zms���,�
��;��dQ�,+n2���^:yu����$!�6E�H������Y|��i_��'�(p�X�.v��:��l2=�����
N��T�c1����Cg�����x8�]��b��	��l88�k�y�^�����{�W\��p������E$E�J�T�n���/�_����$����.���3y6>Yw0
��k�Zx������g�����3��s��8a��^�r�W+��g\�$�[���R��L�L&<JC����)�C��	�e��e�9����8���B�x���yr/���\��{�^0<�:n��'��y���$�*�|��_��H<pS���Ep�� �C#&0_G�n������Y
6a&@O���A�g�!9�e@�zw0�������j���*Wjn�WI�^����f��:H�.�-��V��6�O�����x��}��v���������`�^�g�4�]����a���2��H1����!}�}�1�o�g����&y��<�N��p_��G�#��Ax��XGP1uB��a����O�"/\�����`���c6B��[��J�6s��T$d�����l<b��n�u,���^�*��6�����
"_|e�9�;��=>9c��GGd��#ZPM�Q���N&c��u�k������f�{):�?jC���1;H�X��3���4-"�-�j�
Ry&�Z>�^y�����Jy��7�wjuuS�~C�#�B4����fK~#��y�brx�J2+r/�$�E�k^RlJOYp�X�1�H���Cb�Y���B�N��������Ke�����9���Sg����z������/�		@d�LOv:JM-��?�<�"vD5�"�����{D"v�a���0��m��d�B�[_(�nE��p]C�K.M�r���	MG��[�F��9�d/^+K��#3hS([C��##��
6;#����3�m1����l�F����4oL�M�
*������2����+}����!yr%d��;�N���>EL�����[�[�i�l�U�P�C��~N)��"��%�s�G�JB����&���p��N��z�p���#m�����BM�:t���F�����rVQ�$�������E�?�oc��Puo����_*����+4�}K8�� ����������j���3�>\��1�(�xh�U���2c��s��	��O������������J���3�L�5�h
"��q�3C�ipe�t����K��0���H�����&K|�����fNk�I����O���Rr�{?��:��@�bi^42w�T~����h:����~"N���Q#��Y�5z���>&P�khe1y���3d��'-�/�p���0�KeLd(�l�O���Q�H(��D@�Y��R�9V�|���D	�J��re�	�(W{?�l�%���;���p������*�K�BWv�WA?����M�*����Ro3?��,���%���1�x;��\�(N���*��<|���J�F���.t�2U�"}�M��d6{��+V	:�����S�K���vf�c��V�I��9B������8��(R��������a'���`Z�����R1���P��c��Y��WQ����hX I*�KU�o�Lj.��+��]h�,��8W���G~4�|���@)����@�����Q�q�FCN20 �4��h�a��Q�s�4E�~����\�Dc����^�������tS]�"RA�H*:JneU�vf�2�m���2���/B�����QxO�C�!�K
O�E�G�0A��0�n��3%]rW0��
�D?I,\��<�E��c%1��k�l �7�f4���<��n ����
��x���I�yL�P����BqA$h��"J!(�t��z;���*��&��V�i����#��uD�
{me�m�y�nl1��4>�Q�>�f3<U�E��L��H�] zu	���m9�<�p���e��-vii��]��~,R���3<���*����C|��?��B\��Wr��w����1�K�.�|G��Wq����"'$2������\�f�(B�)s��#�X�����'� ����T9O4Y%���xzO��R?�k q���`�B���S�rF��A@�Y �����m���G�4���.�����������/�>}9�xs�� ���0"R}]�j�s��n6uh���`3k���J�Md���������d�����W����H��f�2�?�,�#���_^^����Z�Yx�r��|	KS8{��LSwt-�����2c���e�Y,�p$�z�l�����n�	�����[���ruI�]�EI�
�����.|�F?#���K��J�����+S���D�I�O���\)�<��B��8�2��B
0d����\Ym!K�e������`8��/o�v�-�:���SCS�g��"8��R��1�=~�m#`��Ru��
��=���-��7����G?��K��Mg�r�P��O�+b2��{5��<�d<%��:'��=>��3w�����:!��z���v�j�������:!r<$5��������v���&k*c)
7���&V�.�7�_��n~A��nx!�����t�2K�m�Q��CW=��J�?3IC	Ki*hC���vs�
��}a$��4��*HW�(�I�ZQ���w}���B��~U^���>���U�Y��!�C�
�,�u���6��""S~{��.wQ�Fb	[���5^#u^7u����|y[#qJ��L���v���P�x�W�"��N������������Jc�VE��l���e�Hz��������g���������;�t�;�J�7��#o�'�����'3T�Y�}��b�t�E�����a�Q������4e�������m��v�nh�V2
 �1 o�(=;����we�����"�M���vh<��\V��H]��I�j"v��H��H��\&�k�������:��F����e������&2��j�4���
�M��_#?.���b�����A���>E�nZn����&
"O���-���<���z:9�d��`xd��l�mo���L�|^����n+��*u1';��c�O�(�>~�����)��/���9_ �L�L�3$F�v������tOR���@�)0�Lh�_�]���*#���������N[���A�Bf��V{��0�q���5��O<��{�e:`{�tE�.F�:�m���H"��g�����:����%�4��t���u���+���/h���X���F�S��B�����1��m]}�`&��'��fU��{65��m�,h���
�
$�^zoo���L�.��;�1�����N��P.�������>"
����������a
�+=R����n��h��w#X~SO��A�
�m�:S�4w���J-f�j�b�DfGI��y�
JL�'�@*J�5��Y��S�-����--D����@���G��Q�_�,4����+y�������/g�^~>��Jz�t�yW:��8�>s��e��wH��T6{���q������������[��p����}>��1�����AqAg'���3R�rF��C���E�>(U��1���0y�WU{��
uJy�"���y�}wZr�^�v��e����<��s�VVe(&����'�ilY�PK���Rd3PK��jJ8pg_dp_join_patches_v2/0005-Refactor-build_join_rel.patchUX���X���X��Xkw�6�,�����d�zZ�lor�f��=m�&i��������"T���m��{�$�a[N�S��"s����_=�� :�}�!����E��X�����Q/����H���N���Q����S�K�n����)�g�"���~3���E����g2�����(h�z������)����^�`��uO����5��A�{_e�����x����No�?z'c��PP�$�*��=��0���N�9t�diJuN��
���p���3��Y���TNY������BI��dN�6R]��hf�L�
��%�� "�����J1�6i*�E�6�Y���	Bi��"���:�~��GD1x�7�:k[%�"IdDsx)=��J/��.Jm��}�2v~�i���\M���������C�B�;�Ss�����Q�bh�"�^��E�|��j���l�hxB�L����8&��D�Eg/{���y*��gD"�Oz�v8
��a�����{�����:�=#���uDM<O_����*�� +�UJ��b�r��<�t��L��rN
:(��;������R��V�Ybk6���bE��k�������\��������.PX��6��,�������:0Z�� �j@�����G��A�5��v����0U��y�G���%Z�n�O�j�Q ���F��+�O�d���:�*	������E����Q��am�H':��3���O$eb��4W%��2���W�����p"�O$�Pf����!D���zx�r�mIe�����KS���1WU�p���Pb��*j�|"��N�[EV7MP	qHt)Eu5�%����I�	W�G�a����g�
Un�J[#'b6���F��Q�<#[��������l�y
/, ��R�N��J`�y#���D0�LP�b�n��22���X���������b�M*�3\���	�7���X��Y�����
����(��D�����E��e��e�O�x+�	��f�1p�R����k�i���v��~��cC]X��'�������"���/���.�U����l)���'Oh���iE^+�j]�
�����~U��Rr��x��
�s�*�vH�&�|�-����q47�R8�k��q+��2ArqtUb��iyl���Ed�o�r��d�rS����v��b�3��K,�4}�X��^�������98������-;s
��\�g��)>O2��AK Q|V�������H����[�e�,]�f}}p�s@?�����
P�fc��kU KT�,��)/}~d-��*�������A�w��hEe���������}c��soha�e9�>�
/�h��v���Z-�E9Kjn��7���q&�	'��-+���?�/s��Z[K^��5~����w�����5�<��E�Qe���2��������wD�qw��f�dv����-K�Q��)/����� @�I	,6i6�=)s&�`Yt
��!����u�:��FL^7���)
J�������?�U��#c&�#cwe~B�@�c��#��@`��Z��b�l6Kw���>\�f�p�Efo ������uX�Fg7����6>[�������:$Gi�������������QC��QJ��d���N���X�0���/u��:������������Q������R>T,vP������<���w)3u�"�J�K����5+i�-�����������
��s
y��Y�����C�Os����>��V\	����J��Px�)6zj��p�o��h�����uMec~����3���������J�ra.%�z����T	?��^c�h�]��t���*Y���\�+[;��l�z:���'����]��������->���`'!.��
��� ����7!�XR%�����ob�vB�<�7��������{�K����������Et���
}��K#�S{��Q�q����:���o#'<�'��>n�����PK�1����PK��jJBpg_dp_join_patches_v2/0006-Add-function-find_param_path_info.patchUX���X���X��Wms�F�����7���A�����z�M��|�t��t�s���$����	0o��N:�b�ew��}v�|]�Kp�`��]����H��~��Q�����/F~���x����#p�����8.�F5�R����~���'��!�/EV��(�qhG��{�+1!��Z��U���L�!���w��:�Q5�?�W�^�
�����UCRgQ%T"�xV��/�]-f2Kr��wQ�rP�����i�{np$y	�*�(��Z�"��F�8L9)W 3���R�}{*�p�����" �u���Q��E5��r�0D��uA�Y\���5n�U9;�J�,���)�
J�6X��e��������^^Tr)?��WW2��x����������{��#���\�x#�D�mf�w�HE3�m3�$�nw��y�,<�Y��J|�(����7�#�6��3��3�1��\�����=���p�*�A��������i<��eF��_"��F���fbg�h���*�1Qs�$r��tj����
��pK?���eMh����e�5������Kk�4���)���W�6�*L�	�d%�9I�]
���<����g>a��^G��u��z`���4A'��ywT_qa-D��R�[��p�f�J�)/��r�E�qY��d�^n���R��|��Not��U�q	���u����F����1rX
���(�<Zid��zQH"M����1\�q�6��,�[i��p�L� o3O
��0�����Tk����/�Rd4FO�t�#}mJ���&Z�Y���L�:q)Jy���p��K�|��T�pB�C�v�si�a�7vFxA���M��#F���SU�����PfQ�a{7���G;�[b�1~����\b�/
ms
��e��1����Z��A/��
'�E
Zz��f���-��&��lk��k�,�{�t�:$�JyQ��pL�]m�X��4��@�:�tv�X�4���}��1V
���*b�f��^��q����A�)�~�T<����mu+�?5���{����J�K�k�.��h�-���1���
+���'R�a�y��T���1��s��c��^P������\"��lz��d��m��W\��8i.�Ow{k����[�q����Ul��1���g�k�s�|���8����}n�-�9��]|������y�U��3���0���Ts��=�}��PK7[��QPK��jJ;pg_dp_join_patches_v2/0007-Canonical-partition-scheme.patchUX���X���X��;ks�8���_���j$����x&7���6�S����\J���5Ej	��.������K���gY��n���i�b����X������N���������i������h<����I�>�5������&������br�g�\����g�g�����Wg"]���|�'���7<��].<v���9������W�glt4u?��
?{�~�tqw���N���/���I�<bk�fan�_��t�o��q&��>,���%��y{�8N2@����f�]���%X����\�	�%���dQ��S��`��D����R&�<vJ�&���L���[���d~��B��8��YB -���Bgi�P:`w�D
�8�4��=K-~I`�,/g�$�qb��������A���l��8���,<h9#�g���i�q�{����r�
���R�q�c���7`t:v���x�����k��*FA*b��,Y�!���%[�dx��0�t����`9�HF��P�T:�
%0��n�lB i.4��@���}A����{pp�e2��s�?�8&�,\��'�!�Xy����;c�������UyF�u��3X��&��Y�����QFy �q94�,��A,���X6e����$��N�@�)�bo����b�����y���7��,��J���O�������B��������<{�_e�������	�Rd3��8��f�>�A�"%}�K�$�\
��y^�u�$��g�@:��x��@��pg�N'�g��}(�K��������2�`�p�a{�:��@Bs}���!d&$�(�L�f���+����S������9B
������%B������>#�U^���b���EQ��z�Q�,��~����GF(w��a
�����X�t+[�`&j��3�����0OZ������k.�d3 (C<���q�p��_}��W������aV1�\�;l��qG�@��GX?�`���#���_�"����v�?q���o������������U��h�2�E�9������������u.$����;p�e/�_~a�W�o>��n��]��>\}�
>�Y_���<��M�l����)�������Uu;<7:-��A��J
���M5R(<��E��[���pp��=V��J^����D��I����x���8�H��&I��K�E�e��G��b]'�q	���NR�O�����c��1�%i��V .T�LYH��`�1�g�5U�U��2+��53�KOn�>%uc����v:���	�Z�4	����%�4����4 �b������q@�]����c�"=�}��?������Z1Pd�[#�`���f<�0��W�#+��������aH�T�r�|�,�I��9<Gq���]���,�XG��2�@:��4	DR��|�B �s�F�m(����r@�j<>$�8��������&������e���!>-Hu���
�e-*�3�J�$�aA��v��	0�������[����>�	��B�:k�k���KO�"k�x��0f��\�sz3����|FHZ	��b��4�O��cH�F������@����T���	J$����e��Cy`8�8�P�/�F��+b�2a)����C���N��l�����6~?���j`x���CDJU<����4/��F�lL��h������wf�AQH��I1��3�#J���9���wFV�I��q��7M��0r�\"#����6ze���e��Alg��:�=��iC�h����@��j�|xe$z����X��-�����#jj��
��>� :�)5g�Tn�O�)���q8&�8���3��2m(�����J�uP�����]&���ep�2��'
P\���� �<��,�N	=�U�H�C�K�v��o�Do�c�u6O��few�G��b���}�?�m�I�d|�M�C�LG�dr����8M63p���"m�I�A���*���!Q}������,OQXde���������i���`�
'qN�R���4��J�d����(�5CR���?��e�T4`�# �e�, �S�T���12R2A�W.�-F�Xt]�#QT����T�Sg>`7:hb0���	$�j�H��Y
��&�XwV���g����aE��oS����e��2��!��)�Ret�������B�Q���h%aK&���W���
Zw��R���R�<~:�G�STJY%�/(	����7�O�w�w�7����.~{��G�_X���ZL	qb���D9��
>-re������o.d4�`�WA�7A�~����8����]XfWsv�-2��o��-���k��"jSw����
w�t��
���0��X���c�d��+�I��\r�a��YC��E�ZVq=����z������@�(����$c���aa9R����@�DD6�O����{�\��*f:�9`�H:ba+y�v(�=����6���SA%S�e
?�����6�4��m�++#����xA��h ��}h(W+P[�:�=B]aR�mL$+��"v����������c����C��>�Y�`J�J��b�2�:�������������P^TO�+~�r�����w����N�j���p#�5�/
z�g/����s���Z���[���,�!�V'�)x�'}C�t��*h�^���8X\�.�n�����n�@�cb����8_�����":�0`?����^8��6�q�/��EXa��B&��:��2qM���$���b\]W�{�E�$\�o���
P���'0<hkv�����B�#����7S��x]�s?�R���V]T��,��#cAK��V+4�S���U��XR����"��@8�'-�em47~��T"��+���,�l�W��&{"�h]{�u�|%�#�N�fDm�p�(-���#�����E������� {��B\Qx@A(l�y��{�P��C�u �z�/�K�P7g!$��$ �����y;7`"�����8���fN~i-�8����X�Iz����,Y/�*���k��4r�>K3�S����:�W����A;,�F����L}�y(<������tEw��p^����ZU%0��'m���j����57�J� ���o=�
P���L`-�
��zE����[g*�mg����-��VKf����^2�oB���%E��),��^%�]�V������6D��k�bS��!zRY��Z��F���A�yX�m�^�m[�?SgUY����	�g�?��y�B{�n���*E�Vb����Vh^��n��Im��*,5�:�f����Z�?��U����

&b4N'u�����!l�W�l��fa��>�����T�"j	��� 2(%��RQ��0VS�|�>��6To�h\���������z���uZ����|�-���D?
������[���*.o��y�Tz��c��+"pxE�WkYa����|������R���S��n���E��	�'�
��5��+u��]�e>�������G���{��TTI _A�A[V��TP��=/����R��H@���r������&���c�@S��Y5�1�x ��K(��gK��J m������sG�c���>^dY�Q�X�t��h��=U�/I^0d��Dx
��1�aP/%)�s��Q�_�+]�u2�Z2 ���
�96I�_�,_�V�����u:z
�y+3��0���hs��rC�X��lk�rc��MJ����k�E��1�*�t SQ�	
����x�P<C#*|������8n���1�AA�7�D?U(k���	����j�^��O�V��$��Z7�)YD}�uaG����u}������&������cc�i�j���a���6S�Zf���#����`�V���3��G�}��s{�sX��M�O�2����������
�hSi�B�G�0ud�ST��Px�4{���E��}8�������HF�g$
�Cl���<�P��|%���P>Xw�LT����P�`
��z��n�&w����5�l�����<��x.�%��m����,��.X�Ov�5�J������q���Z�)�����*J��Js\�B�[N�M�J����z?%N6,K��u��ThR3���nK@�;�!�KP�����z��P���5�@���X�:������>y.=�+������Ti����bmC���_�+���J�$T{��`����9��l�������N�3������7�������
/���+�Rw��oU�jW��A������CH�/���
NU��Ka�W���!��)���@�@^�v��8W=�i���Gr�@��
}�����t�/`����nT�tP�X�:��7e�@��1
90���	pgF���ie|�x�8�)�o���(D�h�5�r��Dt��8���mJ�v�T���+����*�r���v�l��
�<DfQ���vB�2�,�S�,�����:[|"��g*U����{+�F����2��5���Cd�&���x�Dca�B]j���[P�TI��
�pTA��VJ��.#U�������]�u��I�0����Ei�D�S�h��I�(�{ZF�^���j<�?]V�(�"GS�"G�g���>�l�)�{�|�>@�0|WV���Kw�"�s�4Y����!��?6I� R��
��H���� D���r����u�1-��}�8������j����`������T�������t5�O�?�OO���)�[~*���A��_�Z�XU����)T����"�0~L�G�jd�:�`l��t��S����^��[	���1�3�`��ba�����&�p�u*�Ow���n1�d�jw$5�����(gj8�o��rR����S�XY�� 52:z��C[C�Q��'Y2
�M��n@�.��(%�Eij(�u,}�������$3���@�c���^Z��+�
&��L+uk���~8�-$]�;��G���PKAje��?PK��jJ:pg_dp_join_patches_v2/0008-Partition-wise-join-tests.patchUX���X���X��}�s�6����_��������%�3��['�d��c������z��d�VV����\����/� ��e��S��F��h�?h��t����h8:���l:}7<����yx�����vx��C?���*xD�#tp���C��`����y�N��8Z���;����_�����~�[�����z7������;?�Q�]������#���������={;W���`�G����������������y4_-������|�� �������y�n�����-W�����fA��5{"D� �K�/���"�,���nv"��}t}�"�|�/�!����|R�������
������up���m��#�;�y�Q��2�Wq��D��p�v�	�G���E������&&�����F�a����S�sC��P9'�����1�����C4���w����!"�
�������2n�d���^��zX�hxpp8Y���\������-����G�k�lS��;��WC�����&���������;�����7����x��!��-�oC{�k��[���.�K��|ry��������.>��[��l��BJ;���)9�����+�Q8_����V��a��}yzr}��O�;;E��h�z>��h���2C��5Q�u?g}�+�<9�����?4&�C!��GN���%����/�W����O�w�G����*�	��AB��@G���l0:cN������5�tN��'�N�N��F���&T(=���������sBw�2X���>����{7@�������������)_��
wp����+���.S+��������r���O����Qt ���~#y<�>$�,`��������"�����*���&UP��Th
5�T������6=@���U�uU��*�~��( .().()�JZP!��5�/I��y��=����N>���?N/���:��/�����?�wv
�p���3��O��Y�����:���~zy�A����������������K��	e�������9��6�5��c"�^��+�"�c2��������_y&vk���<>������������AAbb��_-o������~!#-�*�]��%Q;�O��~��#��q�����BU�i2�Yd��h�U�bx�8_�)�Z/S�>���CKM0/�\YkM�\V�	��hV�f��e�aVDe��rc�vb]��g
[��0��V��l�Du&I��$:���c��&���cET�c7��(���=��i;V����j�y#�^}	����v|����L�/�k%y�H_XEHF:�`nc���2�/��!�x,���_�/#��_�/D0vr�3���I2dT�n#D<���6����5��b�ldd*�S��K���
:�������������=�n����5��O9��j���k��=����1������^�w����c9?�Y�L[?��x~O�?�YF2m�`�T�}���<�X��b�1���q������c��������n���G��&2q�Z������=�d<���OU/P����A��Lz��-X|���l`�X�2���4��X�F�<	���C�Ir�m�Zb��or�����	%�voz|i����l�JT��k�����hZ����l��hB��'������&�
��*Q�
�,�'IT���������Vc2�#z9�r���/�_�E����x�g���1}�w�6HZ�b��5�CS���m�X4;����F�g�O[��/gg������9�y��H�,�A������X34Nz����l��P������ �m��~�
]
Yr$&����!M�=l#��+^���G�7D���p���j�TU�+�\Pi�h�h�8�9����U���L=�!R�0�H=dV
�C���2R�
�TU�+�\Pi�hC���2�r��r�<#�������3R�UC��T�
�TU�+�\Pi��2�>��m����J��J����Q����ST's(L�\����N�^��o�{?B�e���j�G�+�0���G�H�.�@�%
���q���F�P��_���a��V(Z!J�!J�74���=�8�]""�y+���5B��uLI��?&70���m��m��=��mV&�|����/Q����$���5m� *�����Y28=�P�zT=l�z.^e��F��s2�U����V���njT�U��W=�5�h�z�9������Z�^��8��xh%�!�"z!��x�<1���]4�F�� ��I�� ������5`�?H�$5.���>�C��7Y2s���_P�S����t�k��k�FI�(�'�(*�;��
�E�19x,9��N�IE������e\�(�a�@*����1�$�X,U��?qR�J�%g���0����Ux��kx�T�&\.�I�L�	�T��?qR�J�%g���0����Ux��kx�T�\ &�;�^qz�]� �y����2�Y�
���s"����C��[��\I�K�@@��m.q.���o�*:Z�<�.�4��s����9-yNF�K�����[�|���<�����fu�v,��gx�r��=e����4W���[Wt�o�����c�_Tw��c�0>�������!���;��V�U�:X����7A�k(����+S��[$���h��`�%^�������b2^������w��m>���|�iy��ua;:��[���fs��������������mq�@�X�ji}P`����D�]�� �6gs����$�"��m\��V�GtF��qD�{qgVDr�l��d� f���C���I�,����C��t�/�zelb��VM�'8Yya�Z5�W6�d�'���+�f]k4�N\x�b�G��77������fw7/���M��-�/�������XIX4[�s�fK�z���I������QH������br��u���&����i��B�^��{������t��L,A6���0���wq�N������)������9@�^�X�
�z2�%�K2j�����j^?��LX�_!���AZ;Tk�����k7t{Q�m`�z��BU�+������Q�pT�Q ��'|(��Q�AY��J(%�1�m[x��{Q�@<�/s�
p�:��B���m�P2��Yo��0P6Q����NR��*+mnl����O�v��X�!��Zn�\=���`��)��)��1��=����C
M�oA����F��D%������+��n�zR�O��.������N}��G�����;�p��8���:b2(�'�w1�n�l\>O��sT������`�����S��	�6QC�����O�]��I�VNF�*SQ Vl�:��6[r���L�������PG���-D����
�������evgl�D�%b��?�[>[r���%W)��joh�Tum!�@�����������_6~Sn�������v��g����V<~�5�g����Z��:`�����C%�N�O/O�vv��������+r���K�x��"����4�����cq�q�/�H�sl�Ia���������y�������T�:N�G�O/i�����\]�2�*����CC�H����y���w�Xud��+'��d�����*��E ;�@4�(�#�,��_�@�l���m��r�#��</�2�A��xm*34>���`�����]�z4<�������+�K�m�g��z�K������P���[��Jed����%1|���t�����I�T���|}L�J
��v\��a�<	�.?�����4Lm�6�r�K����[�s=f���RDW���Z�<IH�2��y8�y�]�������2c�[�y�cnS�ar���9q,�'���o��5z�a��{A���<�I��=���:��-������5�(�K����	0�dS���K:�"��
�����,0�"gS��K�6�S1���L�O�34J6���$�����7���L�/M����A�}*��J��.w}��[��9�5m���G�6�WU��7_iFM�]��&�
}����'�N4`
O s���5��5��~��j������%�W�3F�,�u�p>���,�=������a(��r����OV���Y�5��d����M���,DC���.�a�|�6~���a�p�� ����4l�����{a�F��>�f�������H\��W��=����S����wg|��I@����������g=���}�������OD��b^���x�����h���� =:��������������da�A]_�m����%����"����hQ��!�������}:'��g�1��jB��#�|����7��������I��A�;���w��hN����������
"F��y�����.����y�"v�"�0[>=V��Wt�Z�(#��gt��.����a�9 �h�����|z#CM����$��,>Xe�&���6%���
����
��R�R4+H�B�p}E�
��UE���a@��*�(�L#;����4��t���v���("yQ�]Jw�N�4�K-�Cb�bKq��k�����}k{Rc����~���I���v�N�R����q������CxF�lX^O�tLZW����ir�ne.���������'�����w�e�8�\x�VW�1��J�;�T����#-���4fp�\
� ����3�~7����+C�$�.�8EJkh�,�a�U���!5���tb�������n�'��O�������n��Q,�,�d���O�r�f���I����_���`�n����Fm�
K��R�f��
�������y7<�q�6Fm~��0���QO�J���%Cmw���;�m:���E|� z���<���e�������Af1�����eh)�f�Z����������F��X�z}�Z�X�@���BK�g ����2�N�SS:�
X,e�.���{��W/��]O������E�:-�6��b
�RJ&+�G�m+�(+<��������_v�� ��j��Sw���A���Stj�^���V�n�����Row.�'������M��]z��S�:�������!4�O�$,l�
�����\�mNm�m����9\��u[[P"��l�k�<)� T����.�!w����ul� �fl����3�S) ��n>�M~�.���l����i �^%���r�L��y�t-�{h0���}w�0=�m��������o���$�+F�'��l���s$���|�����{k��I�R�?%�c|�^�X	�lXW)��
���uL�K�u,zu��������;���xxr�xl�B��'z\�B�tpKm�=��Sn��Y��^����'hd���l�Sh�f��Q���"^��l�hVj������0F�g?��*����ex�����y��3��8�J�}�]��w=��A/�m+���?��A�mC���9���A��� OVDyv�gyv��d�G��A��I�s�<��)�`�?V�5y�����fH��k��hV�3y�����f9(���H��!]����������2��u��k�,�o`��Y�e�02c\��%)�@��k�<I�O:s3�����yN0��ObT�%�A�[`�'�/���W���CP��
/s����bVX�e�7�L���6<�6a�3����-�%������6\n=��5�x��~���2���n8�vV/���l���d��u�I��(@�k�K����c/�Ao��z�{k��$����nE=7��>P��S����XT�Av��]���3�B�������
U>����8U�
�;��N�-�[N7��Z,Ob�8p���p�p����[ B��i�Yo������O ��
p�,�@;�T�LC��jl��{�	g�5�n,�><�xv+������f����K����=��
	�(��*�J�SE^	N��+�Iy%��"����a����!�
,j�v}��C��������'��~:F�v������V|����&<����W���h��y�2��!MA�R��i��j[� @7��H����4�G�p���?Ll�7��_U�W.$�\J���\_R�o�L����ba�!��*p*~������de2GE_�R������-%��NF���*G�-�H�:��^D�|	���n��	L�I7/Ii[{��\�$	���� Q�s���!��jU��=*5��~����?����h��X+,��,������W%�R�~����~a�~� z9��yk�%W��~U��WYw�_��K�������T��[�_.�+/��w�_���T�_�;7��//�^�����=�~yb�<�Vf�9eeJ����W��s-�eR_K��	���e�L�����`�*&�U�����t��������T���f�\:W^\�����rg����R5�w��F<_%�pj���#Q�����E�%KlG)�,����8=���������M�X� ��	��4"�dE���tf�3#�|
���=��!�2���r��['��.����p�Z�h�Da<�=�_��a����������Pt��[�h�(
���D0���������9���Fn*&�4
��:6"U��;F8�1V�YdB"���o���z�����Z�����z�9���v�M���}y��h����Omx�K�����&O�Mt�����������l|^����q�g�(���q��V��hu:
d��&��WeYz���J����3qv4~3f%�5/���2]j}�pp!w6|5'��9����Z��)�U��J���=��������[����[
���������96G����.�*��l�j��A�AE��i
���V
�*Ug����vN�oc�uM�ZSu5�&����f&�
_qN�R�P��B�TO3���z�Q���
�	������KA����]O�)����*��(V�+Y����8��(J����U�9�NqD���#6�a������Q�t��Th����������D�RDCA#������l-�'NZ-�M��26��	'?��@�1�mg2L����Wc�S��}S���.��2�b/�[�A	<���ug��y29a���/1�I�-�-���I��h�*�'��%A����K���;��l^���aK���{�
�4�y�N����S��d
lm�t�t���dd�I+J�IR^l�YK7s�����7��(�z�O�W)���~�5)�'�����`�T�B����o[b�Ey�����8���]G�*�'I�F5:r����sD����vl���K�IRe��n�.+��b���g{����'�����&e�T����x������v��*�m;2$O3�q�Y&��E�<I6V:r��ru�����>)�O
�vJ���v�P6O�XSC��xF%��M�g��_���uqE�	������S,A�d�������y�Mz���a��?���BMW�`��m��t?3�,�6����iu<I#��e[]�*�S����{������=,�[��������()���Cl��M
�I����W����V�6p+�J�^]]�*�S�������E�$��Q�^�y����������Z��E�6)�'�W��^}����Z����4�.m{uu���O�{B��W����n��@�]�
{3��s�W������� ����t�8Y��/o��U�8�/��|y��T��(X�U]�{�5�C���no?H?.	��j���������)�������5�Y�_z~_����>�'��4��71�&����T����>�'��4��#0��8�����z�.�^�Oa�)l���>��O������O��|%|���O��d�"�@�F7n����N�����
-��y�g�!�Yp���������������RB��gCr��$�(�<I��Ag�������T��� +�ts�����3|�����m~����NMg�m<Y����� eY;�Qe�x�@��;�ff����n��'��'�
�G2���?�$Z%g��Yo�������iBUk�5s4�p��l��Z�6$�@w|V�@�YM�z��,u�'��T`���jb�:0�}<UN���q���N�����R_
���(��QW�2�=�o�`.�������PSk��j+�e���P���O��NJ��j�X/����`��<Y���
�
�P�WC����r��D�3
����#��)��������S<ow�[���K��R14��^I�t��&�Z�������0~�'�d�����%U6�f.o��yR���8iM�t3�|������B��Y�k[O�z�����v����z�]�����-�pm'�e���H0[@$xSf�jqet|���,o�����T�������t�l�V����f�$��l���m:������Z.]�_(|3��m�)4�@/U+�e�|�����[�U�?:�B���-Z+�4Uqv��y��|k�.��.	��2����[��7�1���e���f�d
]s����v������6�,�F���!�N5���,��n�]���R`�a��V,��m&��`�B�����<�4�\�Y
mH�T�9�V'y��MFk�;���u��B�����\6
-���j�����fB�[Vl!��[��C�K����u����L%��C�mu�'��T`�v�sMl_
�-������P�m��"��;�����h$t��\��.�u����-��������������25���X7���m
�5[�d,7�D���#�d���>q5��t�6&Pw�������n5V��h���>h5Q��c�m�`���"��{��-�X�	�r18Y���-��-����z��E��E0�1��2�r~w�-{�X�������h��Nrfuz�zW�r�E����]	l��v��F��5���R��7��Vh�����u���
]����!���[��'�\awW��(���z�@�l����p�\|5G�;C�	��Z��p����o���\y����G����v
����lN�V�Rs����pyX��J����;�=����h���N9sBd��6;OV�j������n�r�j�����2
,t���f�2�a�F�U�e�FD�53Y4�htH����[���L��j�2������X����&���/v�E�P�r�^��J���������VbU�:��Xq���.^�N��w3�b��[y��V�t���.�u�]ho-�L��@��6X�Y"y�����I��	���>4�M�>�eg'���'g;��|�
��h:v8�"W?���G�,?�XG�L�c_$���_�z/C�P&���:�IIn]'v�'�
[����@�3u!�%�������u/3T�Q-���M-�G�9�����j�X2n��'�{5�F�p�H�#���k(�A##?P�)N��z��h�^/Za�b!�%�g����55�5I���u;�-�l���K�GR��Z��,�R��i}��zU�h��k�j�ZiU�����Ba�n?�I�,$�~��7�g�~`�)�x�A��M?�DS*�~0L�A
X�4r��M��A���&� :n��hT�W�H_�X
���rI�z�/��nA����/�fn��M\�����B���)�EH�;5�gN����3����_��f������q�R�i
��~��!�����95�Mj\�,��AH���X�q��>�}���$�fAh�Q@�p��>=�T��n�p�^6�n�,lp��C_�2��F��t
&�}���^L���j�2@�4�T)s�Q�J<��%`D�O�CR��P��nb�6�x�Rb�B��o������{�F~����m!kO^���2^���_:�����]��@�}��T�����#o���%�/��T�'����B,�u4���%�F�_�C���������������+t}���)�{@=�����-���g=���}�������O�7-�<9���G=�
)b�H� I:�����������������$J�`@��t'����-ElF1��������R���ft�C���S�t~uzy�>����S5��jBe�#�|����7�U�����hK��AHe1~�n@�y���������L9-� "�a���T�0���{t}?QD��u�S�bEf��5�����Ot�5����e����L��QJ~@�:`7��t�D�6:�2jc`[=���D	}���	�}1���]1�� Es=���T����C���X�C�!�?�b����X�+j4*��Go���|��]6��k����t#1y�s2��?�Ew�L����hc����e�,��6��,��*�t�����w��&�6��3���U��H;t3=���%%�{E�.���!����4�sK��u2��$e�|Ow�C��C
��I�4�.[�a�3������t']I8�vxN5h���k{7�W���^R/�I�f�I�bu@H�V�<{���D�b���~��W/q,�N���%Z�#��[�&k=�����uUW;K��/Y���w�Usg#x��K�_=��p%�v��4�[J�d0Ka���vA��v���v�E�	@.GcV��X�c����x�����?3������,�,�����O�����I���������o���X|�|����G�ZA�M94b����i���.AP68�����3N5!��:G*�����w���/�S����IY���A�B)��� �@C~nW`$��4�e�Q���TCQ������J���y@��,T�WF#�:���@�b��Xl,,��E��11��X�Y,�,��,�t:�]t�j��T)+�:k�6�Bg��8����P�@4�4����X���'rEa�:�):�I:�iu��t�t�s�����R1���l��c�:�6���Qg��aM��Sxd�a<��h��	0K�nrP����;�6�N0��n4P� �;�Vt��v�EF�1�Q��������v
ZP�$|��`�l�s1#)������P��(ca�<
���I��OW��Yq�y�����gJT�z�Z����z�u����7��T���0�7N�8����U'�5t:G��8�w��0aN)���o�<�c���|Uf���
w�d����8O������s�9\_�0�s ];�����t�t7�sj���|-�.�~2�����<��`�_Q��#����8	�t��=2*xs��gc�@I
i�H���i��4�0m;cIhH�Kd����1�9	
7��3x:�Id�OV�D�����_�o������~��
��#f{���������oK7�Y ?
���� m�����3N2c#P�#����S�e3����/?'#P� ql*�b�����m��������5E�E��h�b���C�0�����;��@��^
@e���z?J,@���)���������"�)���30���X��������DTL��g3C��"*�����EV�&(�]e"�h�	r���$5f������;�dc��Vp����oR_+V�,�U�y�:�=�{.���Q�ee�+�l&��j�0���5oj�I.��04�O��H�.5�����q `u�7g,jI9���dX�"Q�M'���}"��o\�����zc�����`:��I.��t4�O�������M�q�gu�7h:�H9���tX4a�fo\��|!�5a8���U��g�E�A*$�������f<�?P-\s������$��:�����C-�a~���m3Ho����h��BiS��|kn#OK��/�6�c�o�a����G����G,c�Az��XK{�_(m:���|kn�C-���Bi�3�=�oO�'GZ�G����?�`G=9��>�/�6=-w�����ii��/��;:3
�=��dx��=.'�Ji��a=j��p�_(�!��D=�g�=��{����������J�3�E��y/�v6����(�x���kJ���'��7�m����~��B������c�]M����y�m������y�&<T�'I��v�����#�_c����n��
�In�)4M2�U�j-���r�����,g6���N�6�
Ov'-��'�s f!�uV$��l�{1��l�L8��b.&��`LD��NJ�7�Z�����'�j�#����O�6�O���X����TC"���nH�O�����!)�f����`�TC��<�����*J�USZ�SYu����Z^�����t�h��5x�F��pl|�qQ��!qYM��d���3�,3S������e�*(�zE��,3��Rbc0�8��p;w���t�ef@\J��|�ef����3g�T6�.�K�����������`T1s�������[l�����u��u��'c��/�Q�������|+F=\�]����(���+�jw�-��qwp�A];8�����6�ep0���6�k�8�u����S�c50��k���V�QY�&���h��k���COY���y����h���MG�2�V<���\��~���n������u�s`$6��AB;��D*m�T���$7�z�����3�h�5}+[��	�t��\��i~����<+�m�mL�z�������,hi.L$�����O�	���F9���Q��XB���g
�Rk��6v�it+�����h��
RlF�Z����@b�������mm��9�
�%$���10uv�ic?�F7���v!�O�l?D�\//+p�	 d�����u=�u��V�f�eV��g9�������P�7E]��U|S	"��%���4F-���	v-�'��T�_?��@�lnlV��M��"zY+�0�A�&��&������������-�
��;�Y�LC`�vl����B�q�i�.6�'�0�f�����g�'����
h!�Wt������������
������������,Cp�eT�"���Ud�����2PV�Y��*2��6+�q]���r<k������r������I'����g]8{�Sv�~)e��s�&�����S�p6��h��9T;�9p;�Q�H���Az�3Q�������c�w�UE��BR�����q+��%���F�/&�
������O1I�SV(�������^�y#o�X
��(%WVUN	�hF��E,�s�����<��?y�J �u����e���G��]ZG.x���T�
b��-���&l��3<)zTj����y�yo�y[c��a�VX��!��*X>�X>X>S���3��������������^�_>�bm���:X>��`���Y�>/I?9�h����I�|�5���-����3,�;3<-_��9Z>/�^�����=��yb�<�V���e�
��S,�X>���Z��D���,��X>O�|^=�g2�_VY�W1o���d�,d����5q�|�\��Z>�5���-���&Z,�;3<-��i�C�6���K�&w���)vY��h,�}r_���XnM>/�wT�o����z����M,���������9�dt���H]��:���:��:{{d��0���y4�n��&37�}�}�3�$��dvF������m��YWMY�N53M.eV|*����#�y��\�y�Qp��V�G�@��YZ�����8���	�A��e��17<9�m�ek�@�8���:��%�TY�?�j�h���p��V`�����QJq.*+�O�A��������U�4��/��v6S�Q�n��#I���.��62U�QS�l�&�<L�,J^[��:���Sm�sA���>���\xm�O�{��Y��8�6T����M�e&���:�S����8&[-�7z��[1x��Lr�v7+��2g*w�2�)w�2�(��2�'w�2''wu2W&�a2�%w[2�$wN2�#�:2#w42G"w'�A��?o�Lq�Nq�P�7T���&_���
]�)Y��K�d��||_�c�r�S�>�pr��e�����UVg�����:������]M������k�N�T��BB��@+1��O���P�������9�99���g�|��Y�����������{7���K�>�z�=c<I
|�h��|�&�I�>v���s��/��L��wW��	���M�n&�c��},�n{�x���i3l�������-�V-wM�ZS�J���%5���D��<�� �~��!�r�b|��[�h,�+��<D����`�����k���I�]3m�X�����}(~�
��&��#,�38��b�]��� �����Vz^#]�0 %�o�u�\L,H�hx
-D/@B:�����F�d1���"8qPr�Jb���.����U��3�EQ�9t�0��gl8x�v�)���4�������d�)4�����Vm�'�O��>��y�_o�����?�e	�MYm�#�TGJlK�	,y	Kz�NbI�T@��V������$��j�����<.��ZU�x��Vg�:��X��/OO�O���wg���L�����2�]f�������>�������'bi�
�<9���7��P}��_|L���D�89���*�zo��E������Ko��)EF%�8>�?�_�^^��N�L�|@�E�	�d�||����)F�.Xk?
&a��a�`�F��
�!��'�'g��3��p����:bH���xI�g���a�cX�X�}Iq���R6���u��?L�G`�`�E4e
Z�������z��=��
k��������psa��0�\X�\^��������kd�<�<�C�Y�3U���Li{J��{�����������V�y�F������q��2j��yZJ��)Q����V�uAZ���!��������4�Hm���#�)GiS���G�)���e�Bt��\Eh ��q1n��#`Yn���wA�n����Z��������)���	}y�I�"����Cn���WE�$�:[a
����`�e3�O[�0�v�������������tN���|��i���B�~�A�l���� ��+��EihF�`s���K,3���kX_�AL���_�C}}�dupd��HR�� �G�M�1����T��?-o�?8�8�/��<�8&���V�����(�h�K�<�
�+�7K�0�P_��=��+�v(���Y�������b~������\��M�)=�Q�p�m�����8������zKG�l�����W�2����u�e����7f%f�1��qj�6.���I�+R���
�4�7}���hou��Z�k�]�������wfx��I:�r���w��FG��v�V��K���:��;h����vP�j��#��`�}�:��r�}���[���GR�^����������?]5;�2)�(`�}����u:���sBH�.��9�sK���9��H�=������q����������N��cn�R�LT��>g,#��r5�6��r$��I�R��Zw+�sh��C`?-���xc3�-�wV1i��i5L?w�i�Si5qho�nN�mo���-]��v���0|KL��X�I8�n�E���v��z��������������.	9�v�F�����kV�����������Ay����(���l=�_v��!AA���>�����J[���F)T����`����y�%D��u�/B*g22�w�	�qqHF��&�Z,g_�]<C�y��?�C�?LW�9�
#���,
3*7~����f���� ���x9���B7D���y�Z�,V���BR]�g��NH����/o�k�q���@�:��g?$/�V��m=a��K{���|yx�_p��G���rIJD_���j��	�d������d�(�@����Fn���K������������g7���qE�R�hz��������#��\Fy������3i��c��Hb��ww�:�F~�)��,��}�V~�5�9�sg|A�s"���M���K�������a�D��&��vI^SZ	��PA������)[ ��M�;���&�QiaR7����fQ1��+�J:]Fj��4��j�
F_
�Um���RC(V
��FJt���"������6oU_-���V�,���������~���Kg���s����c�T)����
i�s��.���g�q����L���}E�1���*.���:c;%��Q�P���a0����:Ee�T����J�q%j���8]5l����)";cVH������7�ZE�7��b���g����c�_P�~G���r�,4�R�6\j\,6�].��l�a,��Nl�
�V��A��mn)P�h��W*>��
d������:�<�L����L7�-1J9V��+�L���{Ep(��xS�����?�7�7�`���q������8���W@~������w+2�@��CJ����M��9>@����m~4V��j����2%��?&70�QSA^c%c���S5c��q�uQiV�(�AZ�$\h=��/jOK�������o��%�������uL���	��z���+����������J������D�P���yJ��iy�
���#�~�0�*ok����8�e9n�������i��O^�:���l�:LoBu��q�=9�|��b}�����;�LV����}����N����T���O��z;��)B����%s������ �Gs6�GR�z;;�>�<9�����oP�5�>'W�J��K�x��"���
�4� b'����%'�����i��h+,��0�'%U�4?���Hr����mU�������d��l�JZVI�c��x�I`7�MM��i�-�����W`�w�����=��hULq�g�����@�D�KO1A$�L��@1��FWS�:U���a{U������#)@UiUES�W5��,(�"����P��F�p�j���l��A*������.��t��}�A���G,0��I�-�.o
$\����4���b(n��lr������
�[5:l�����������z��z�sg�8�0���%N���^qD�N��Q_���T�S��7�	k�-�b�a?�W��W��U=�����z[Q�fT���<��)n����m����Q>%~�����}���e�V��6�~��&���
�leZR��	���R�B�y�2���=��$�()�H<��i����V�����r�$���]N���PK���5���<�x��Hn�^����:P�/�!� 4��1j��M4�f��r�Pe��\	�6M0\�c�\��m�;��$�l��MF�	�T�^��1
����7��b����k�����D��.H�O�-�k$����t�w�Q�����K:����=�,���no?H?.	�= �1�R�t������n_�
J��B���|X���y�|������|����|����|<�b�Oq�#�A�Yv�h�)�>�M����<�����hd#�FCQ:|������K���������p7�**��1��-~�F����{����t���5��F_����TdK_�����!)t7����4��QEto���.v��n���n=��V�n=����)�7P��P�z?�fJ&>��h������'>Xn�<@�+������W�
�v���R���I��
�v���f���IW�
�n��C�
7�����p���f���f���&
���RAP��&bXA����aM�k"��������
`����	5k4'����*Q�e������9f{�4�u�V3��c��}��B���0����%_J�}���F�/�v�6C9�8T�l��H������8f:�����=���"�`�a����OW��Y������D����!or����_��=�U_�����8����NGW}�j�&t���q���M�uK)�/��t�8�[�a�B�����x�>��o�x��"N���:�tJ����}h��ip����������:u����R�\�8�E��Z	������j+��Y����Tt��=�bys�����=uWC	l���mk,��V������
�����,�f�6p�`?�j��f���2��~�?��!q��G:��|�	��L��`�� /�-������0��	J�w!>C?^^��s2��8�/�J����Z�Y5�������EaoM}��������0����������q���}U�����hw��&D��uW���{;+��v�\���n����7k�jjZG�%w��mY��F����}}��
�umB�[���6�X���8�_�\�B��\���X������M��v�D��!�,c+a��4���K��7#��K�J��L�y�	J��&��n��������o3�)7q�N� ?m��&_��0SE�Ph�B7��O���6X1��8�����?s�yF7�?��5��������mY'�Rp�q��,Q��}�����������0����S�l�����'�������"�������K��A`&�Q��U����7�\
�{s����\B
W)2*	���Aq��?��7��.7���%� ���[��3
px��nR��'�eA��`Y�I���MN��XF�p���6�W��<����5�7���Pi�����v1�VC�S��S4��6�4��6�z�`d�I�?$��#��%!��<�����	3���Uq$�����4�Hm���#�1Gyc�<���C���+���M�?>.��
��q]����]����m��5sTL�p�Fk���I9���w������
���aF���f��M�M��K@F����|
��Ur �/��~Z��5�U��$�����/c�fEZ�V���_�oY4�N0�j�baB����������zG{�����&X�rO�-m����2�A�Y���U��|]������/����:��;�<�����4KU�~�\#eS����)b�\
�7C-����e���Ms��dBL�5
����]S���H�[S���p�h���xg��PKX9��EJ��PK��jJCpg_dp_join_patches_v2/0009-Partition-wise-join-implementation.patchUX���X���X��}iwG��g�W��oZ��x�m�PeqV��T�gz��
@�,@�U)N�����ge���;o��(�*+����;^�T%����x8JNO�Azz�����A�x8L���0t�{�u>SW�\u�U��=��z�N���|�����"/o��d�,����=���5�-�b^de:�������d�~���iK����������;�j�s��i\-������o�����T�����/�]R,�E��v���5�f*��'��I�E�����<�|��� ]���L���f��Iz�N��H-��$-����T��E�MJ����6+�"������&
�.g#`��"�f��{�/����h����H�4/�yc��!���0���|�d�7���T��t���dY��J��I6�A�E�5��&�]���Z�������E���|6���{{�S����i:��.�l�|����l<N��<d31��/�3�^��_�`d7�]���`����eV������
���"����)����,Fi�-���0�1O�i^<��f�r:�~`YS�1�LE:��l�-`\<=i���1��L^-�-+�����-n��k�������\���$$*�iF���/�Q���$0���A7g��Gj�X l��P$n}�M&jX��_��kp���m�h������II
o��X���dR�j����	6���/�f9l��n���R�h��C�Y�@��#�#"�|K�8LF���s�"�L�{�C3HK�8�DKM�������*��� ~Lg��<.����a��
��P���������_�g�=���}	���K&�������������p�K������N����/�g���Y�~���A��:p�_-�I2�)�&��!�n�T����D�����QG��|��&4#�o2������I2Lo�	~��6�p�����~~��e��(\�`��m�7�����rXEY�A�	�.�
'�Q*�N?����4�tU�3Mr#_�t����/�t9k�T�a�>��
�V?;Z�Yd���������
>�����nn'�3����7@�=�m�h��>�v�\�g�?+��,�/����=��w�o��'x�@��X ���&����-���]���&�.�I�v�
�����M�zo5e�~��f������d�m�G���p��N���I����k{��U����A���*x��6�a��l�����&���>RO�b�n��go��G�����������7'�x�,�W���g�VEvs6k�l�=���z����C��_d�1��t����A�m�D�Y!�	�	�f�p9�{r��a����p��A!@aO���jT��9�r���I6�X�_�� �8�S�)�������m����Y[�/Q�
`�����$[<0d�V���tZ��;@�i���n��T���e�H��%5Vp�e����
h] �y�L��K�nax���t��so����T6V�4!��_�5��l�bY���=b������g��6V��1v�����N�G��x�����J<���Qn;�������2+����9��������8���#R�93h�-�\D���K���O��R��	���@�C�4�r����f�&O�#����"+�%0� *����#i����DTQ�p�,�HL��&�G�GGl��l���| �9!� �����Nc'"��
��*�5v�}1oq���7��N"r�~E�\�=��3�\:1pn�8ZfmP��� Q��g���="Z�0��<H�sX�#�i�J$Pb�-(���|�Re�
!�UdA���A�C���Aw���W����`�y���Ty�/�8��*�������4zZ9!F��3$8F��f���3�R��������<�����K_����a	P�4`0����;�*K�������*�/J�e�e��/)=�pm"���`�ov����8�?����2F��h�g�P]���[*;`u)�'�5?��hR�2���Y��;oJ�MF1uP[���X
W7�5-����Pz����E���>���9�v���"�"��P���I�0���6'�
�O��B$��+����d���b9�;N�;�ft��B&�k@y�D#�������^;%�)UsD*��B�z�IyWn��(�=v���1��G�~���;R��2g"�ac�{�b�Q�k�X��b���L���V�&���=s���?���L�
������$l����5�5��B���Y����GD����u&���v��
n�-�����/�H�#�b��H�����k5vn��E=lp���B��I�h��.����"���E�z�WYc��:em��8e��Nx"t�*\�"��\,�|wS����\�{��p�b���ZA��Du���X���c,*�a\��4A"agL$
(.���.�X
_�N0����/xn D�d�������w� a���`�m��M<v��x
g���Ej/�0G�����(X�������[����f���&`m�$eC3�8��Hg��0��mI�d-��Q@���niWg�E�yX��7������x��7h����li���V���X����t@W?
Y����Q�	����8��h��D�HK�E\���%�V`C���v	J�v��*%-y�7g�"���p���X����B�.�K�T9GF#��i2�]4v���'���%��2x9_���78�����)�fX���+��1f�
|�,��;��A���9��y
6@����7�iC�v���)����x������s�$�H�!�U�n����C���cyApW��ep&P}��}�*����h
��o��?M����?���}���;=hu�j���u�"v�����34-�|��@��H���/�����E���I:i5V��"1���wI{MP*�L.f��'�p��r��I*`G���L��Q����R�r�O�N�	u��P}��������ve@t����
���w[:r��N�E��
T��F�������cg�dj��r���������9�.`���t��<���r@S���t��1����/�U�Z�+t��;��;��I��t�,���J��o�d�|�&�'0�����w�z���4F�WE�j�Ca.�q�l�r����vp�\�M�O?�L<������9v����_[�5Z�c�
l��qfh}f����k�p0�>�u�����fZ�=i(���>���)Y�����^���}�R������~Pc�
����B�'���5]�U�:/�$�����?��� ?R?���_�z�������������O�x&����'p(��4���?��L����31�*[$ N�D��>�I�����U��}��'��kv`�0��" ������M=y�����hI���e��lZ���[��bC@Y�o�~'�M�����'K�T����4��V�X�=��W`
����������EXOX�+����
���y�R�r�r���pl�L�2�������'�S� �iu�Z0�Qv����6�U���0�#���-6��Nf�#u����
�,E��j�
�,-���C���"��|�W��4�=���<����v���U:H;�+w���u�!hk1����@�4|��)�������g;;��EX�;"����?�J$��������t�L�3n�;��;�z41}��'��r�)~�'�<��>����t����1��|���v�xl�������/D�s21n�v_=��=%��K�Glw$�1H��<��%�B	�fZ���^D���������*|.�baT�|$a��t������R�����vx�2�[�[:�,-�O
�f_UW�wC?�/~��6��%)��;��i4�3`����s����M,������A��M,"
�l��A\�w���-�^��Ak��<�3�k�����%�u�<�����<o�`[�h�b��32�+���~�YT�z�R;�V-8���|�*���C�I�k}-�E�����S.��J���E�^��mAp,�#��Zs�2�}6A�!��t��.�Ca�z��	9�i��)n�x91tw�]6V������M\d2B��L���X6��Ha����o#(�Gf�����c��Q�����?��\�f�vD�t�NS�����f�%�s
|�"�~.�o����x����,��� |~fPy�0�M�N��B"L�XW���
u}\
$8��0�KN�����.��m�/o�+(�F�B�J���Q�\��@PhRG+�ct���C@h����d�m�)��"V�We���!�q#C��r\r?��s��3��H���C�!��_G|I� ������LR+�j�#��Qd������S5�Fe���o� �iF
}p`�O�<�����T���"MF@��o�������-�1��I>I�{'=&���X�F��3�z�� ��F������C�#����1����
}h������_��8�[x[<nv��Z	�mm�$
����f�)�:AY���V�V�w�:8��e�1�\�:���!�������DZitY-�SQ?�D�n������d:�W�>�-t?%#+	���
z*��g>�o\kp(���?����s������<��H��X�[�m-#j���h/������tU�i���$ba32��o����I�Np�.�u�m�S5���(�$���gj�
e6��K�*�s�
oe�rkK?����l����`@?��+���������ZS;�H#R�M��1ZM�����_`�� ��1�������=csl�HA26������t,��k��B�Sv���<[�DFN
b�	�R��U����^�M��c��[�����.vb�L��g��"^&����'�>_fLj���/�lC��Gm��`c��.���I��	u�@a��|a�[��e�������?��������m>ZH�	�O������=d��W�Q��h�Y���&y��}E���LE���LS2#W���V�O�&8BW����U5���	��=k+k��c�MVn$cs�����"<_��cR�u[]4�}��Q����_i��m6K��.h	;-~�p�[{8Y�cK�'�:,�j�I��#l��d{���=dn`yk_y���+W�OBH�$fFOk�t�qD�2��\}�0"����.$�4>:/����������������=|��F��9Z/h9#,�����3����q�{J6�`~O����aHR)i3�4��Gd�O:bBI�
V��������u
�P)���ra	��e��z�j9H��@6�DJ3�h8x8�E��s���B����;��n��MNh���a�0����>"f��Y��Ug@u�)�/T��'���?���e����<9�V�����HQt����%���9������!_�������0��`���9��/�g������tu�������.%]�������G_��L�Q�x��o2�j�qE�p���#tH��|�DX"p��������	I5�R���A*,xn�o%��GA��������)`yysW��t(�7��qQ��q�+EE�NZ�UTR�Je����i*n,1��hio���9=���:'|�|�t����IZ�0�����?K�SS�5`)�����G�\$|��UW)jL����tsMK�-�S���{�	�+�/Y��c�8`a�[��o�2@�j}���j����5��T�)�T�I�O3�j��X�����T	��;�=��Lo�%�A�L�@vUn��T���)�y�����c���ul{��F�v���������M	��*N�3�!�[
W��"���gq3&p�El��38���[k������S�,�N;���+V��'�IICBu�r���D�%
"--�|B�'��U_�Z{�n�{��]H���G����RF7%��q�S��zEM%
*����H�YFz9�D�j9�+~���J{�
7(9���|���0[8���]�-����r��`x������GQiEzC���U�R���h���B�"��U�2z)y�cV�-�����!������%�)k�������)���������F�Bs[�jz�����U	����;t3���i�-yg7��lm������c8��FU����]o(��r������dH�@���0�>�
�34��P���5�M,~$�JPFi:��X\	T��wM��Rb�_2���&l�����
0�:�Y�
�(�D�h -0��o���~�#f�8�O�9'��g�o���L��s$4�����Ym�����&������R���6�^_i.�y�~�����
��V;���Mwa���]���:��A�A<)��.��s�t�?���B���MZ�l��P(v��=�h_,gC�1�:�j$!fx<�s���MD%(�����n����
c�	�N_lNz��H(!%��Y�a�&�����w������J<MK���6��N$~i���#f�+�����i�p(�
��s��1����;��[����o�����9������Q{�B0�	��&������I��=����=�(��hI���cX����7�5�5�\{����V�f�����E�:�����~]�+��]|M��3�P���1�|���^�����H�ktw�G��<���5<Am�+�����|�h~�Z���]�	���SF���r���<N���r�E�[�Z�����Q��u1�LX����8]���!E��$k� ��.lt��N���-J���w<=��'�F�`�Ei�$�*Pmg�_�����{t
D�abl���J#h��d+�K��I���GLJ�P���{{��7��w�&��D��,uS�h,p/}��LRM`���,-�:�ib��*�����d�����@3*�����
��Y*�7������n.�;����6��?oG� ���*������i	,�Jn0��y��G=c<bb4��'�����8�0�	�~���(���,f��������3��f�v�=���C�]���1Sx|��i��q��q&�Z��O�K�����-'������A��('���X� b	0G��������,g"�	Z��s��U��:�C��V��L[q5�#��g�(�~���gbO�3�HB�pyL�G�k��UF������
�A
V�S�����X����t����l2���Z��������?��c���W�n��ZJ&S����6��T%.s������,�I^'������G��u�EW$%�*�)�e#�%f��~���p���xe?����c�����^���.�/��Pp����U�M����1�BK���/�+����[K�+�cX��������������b��?c�A�;���fBL�Y�\�;;��N�={Z�����(����/__��������~���t�������L}q��c�O����f�����x��3_��O��
\"�'��!�eE<e������H����������m��������:���}x?��+$<s*md�=�������?N��Q1c�I��r�X��x�t��9����� �^�U,�s���VP'D8D8Py-�6����8~`�N����������DqRs��Nb�IN���d���������a��qN���(N[p 5�C�L���du��<Xdd?p��qM���M���Y1���
�z-"��m�Z+�����}J�)?��/�?����;}y}��M�������W�����O/�H�L������W�����/�5�x��Ht9
�6E���� �m'H�K����N��t�OO�������i���4]���tO+.O����]Q[��]|Z	��K�7�`��O��4Fp�7wt�������[��=�@�GH��j��qT����i��)6Of*�����0�
�c���y.�9��0�3q�gs��J���(\Z����,�r��Xh��Nx�L����U��#��2��0��j��m��h�m�����+aH[��rH���?�L#�R�KCR�����N����[��F���Ttt�U�~��$#>����8�$�Kfr]���_���n���Hf�@��Ah����*����H�O��`�����o{�M��o��h[����$���c�s�v[su�o���S�������J��~�/�a�EK��P�5M\�=�[��:�0m��=����|����t����x)ZOg;�d�����4E4%m�M�a ��xi��"]��{�=��-���u&���gm����������C�����������vre'��h|����������|�i�+[���:��` <&�#d���w���L��������y�{�<����k��hP����:�GeJ<�Rl%��/�bp���^@��K������0C�=@fA���5p�� ��<����lYum�%��C���V�E��!���O�vJpG�$�!/��s��p�����5�+�B�m�����h"��>j�x�1W�j;S8�k�����YAt�dR}LRA��YJ��k^�c�_���y�bq��ua���Y��|sQ�<��1���9>8`����i������.H�W���
t�'l�u��]4d:X>%�����7�mNa���1�u�~@*�FU��!�����+7y.�g�p��m��N[����4�-r}.����u������^����^���B>$����]O��~&R�a�,	?
���X��a��g83�@�������N��KY�*��w��^��T�X����>�j�D��E������� qf��^�&�e�g�;j5}'��@
�D��W	�N
����^;jI���4��.�R���J ��mk}����|��+y��
����������Y����.�-�Ro'�k;��u�9��R�X�B���3�)���q*;�}�Z$��%6�H���+�zlb�����?�]�V�~M(����}O��������<'�^�(%��Yvg�����XD8�1�&������r�|�@����><Osv�D+/����OG8�x�����;
����qTh��+���^X�#~:�]�*��A�"�K�?9�T�q��[�����9�y����{z��Y�t"
�;#)���i�}6��,.�;`�����Ba\����qF����+���A�Y���3#�nK��Yt������]_u:�t�c�$��S��zw�w��F;wY����!y��K�9�<�O��p;�DKR��Y�!I!_3E�������W�fUp��3������8���qJ]��:���Pp��	���_ |������S�N^8�`/1����g.��O�n�5r���{�*��0���]��H	�_� �T'#^��Q*����L�H��!������5�V|��!�?�z�D�\���m�<�I���F����r����c����b���m��D�^c�f�l�Q�)�[\L%���"�&B����8�\K��8�V<�E����|aj:��xu��#�;����M���k�q��(9�����N�<��UAO���n`��4���@� �2I�'^
�1y�t���v$� ���S3�1���*NbA��!�v�����=n��7`b������[*����Lm��i*Bx�;�\�J���������r�I~�$����z�	{���%���u�����H������t�k>Z���y��DGne����%|�-kb�mG���TI�����8~�xXV�T�8��y*nc���JlT�����3r]�Y�\��O����k�a�2��\�:E�]�WG���FxM��)�'b`K�F�oAp��(�����y��3��^�������oK>��N-(������H�F����5<�$)X�DW�D��X����)x��F|u�)��fW�Z�����' *rX���7� ��Q
;�:������8s�M���t�+���W���������.\�*%���"��� ��S�S	��������% u�.Q@���dGn�0B�|�e� ^�"��3�[��~GV%To7�J���?.��PK\���++�o���&*�6�y�O��^�n���/=|��4��zY�
��o x�m�w-���j�AO*#�t������9����"j��\d	������j�G����3�t�5v�s'���(��A�_W�_��-1�V5����=^he%��	�����5��J��p����>��U�\kE����:����-���ENef}����2I���9rfd$ N+�\���_I���$�,��.I
T��N6	�8<��?.#��@��>����[�l�F�+��j*VN]������[�w�
�o>�w1	\	)�54�h8�[����
�����w��i��������	�s��kz���D^���8��g]��wYi������a��6Da�h���F�RG�43�~�P�'%��x���'hv4����5*���+|�0��_Y������z�5��:��9\��Y��Z��^�&`}�;�#��JE�l�~���^��6�XlZ�� �����C������o��(������M�{�e�������g@�U��?����� ��[������s���z�cg�%���i���N-M�g�6���,���H�QX����qV���v�&����T�qYQ���R��������89��jz	4��1��A/��o'G����W;�m���\Th��u�Eh�8�0^�.�1����$A���#����5���Xl�
2���"�:�i��-�b��L�������ddz����%m�f�:IZ��2os��*}xlN�uZ������9: "�����H�#|��<a>��1��z�,���L��NJ�s�$1}��1�}��������p-�'M�H"�+��E�|��Y���l�������:Q��\r��Y�Z��}���!�)V,Fs����@��1���3�(��F8a=����d�M�����w��w����L�3GX�|������{���b�L
N�J��0���� ���X=;t��k�gP�A������Qy�W&MxJ�9��Lb����1�6s�
�iE�PABTR��W�YN6�Gw�<j�h�Q?U3�L�M��Ty������r����Y8���J_f?O���a��z���*3&�E:��
:�c��%����bU�����������
gSs���k;���"c|R6f����D���u�EC��?���,�<��$YH������"�����Q3N<,X������p~���|�D���$t�
�wx$Z���w ��T���t�3�Y3�����z�P�(/�MU]C�G�I�ci9V*ZJ���������L����-���U��|"��r�/�M/�I�c��������_�� H%t�����wK�^�;�B�����>J��n,��G;8n���"����(G]9�1��^����pa�
�����4�B�k�N�d^��3{a$%&�3�qG�P�����uH�E����E�>�HJ�q7�dw�#�,��fG�|�5�9�f�
�W���p�_r�X2�.k��#g�U���NZW��N=?�r����5�I�g%Y���t��9tP7�%�$�=�rQ����aK�'u:��h�n��A�{�nOX��2����2�R���)%w��88b�����u��:@/��qd`�-V���
�����	
�{�����|+K����t[,��G�'�C9�M�9��uW��m���Lx�-�E>�B��D����!�,�Hw��
���|n����I�q��/���>�|�\�I�Q	��rS��v�"}S�%GD--��/���M�X��Z�U����~�\��H	��U���h�'@�gC(}�d�66����@v�q*�(�PU�
��q�X���v�O�X(�0�b"�F�kk������FR`�M����[�-�&
�r�����eJ��xI�����+O(��c����Dg�������m���b,d���pN}�5����lv�Ut*����/B[P�6���U�����_7���'����el��W��-K@�O���������s���=?*U��z����_���e�2{����x58�g��3TA��<a�K7�/20!n�|)��s�o5I����[!w��m����?=�XI��a� �f,�~qT������FWd���_��%=
������_�����w��6��v�%'c������>��E|��������0��^��u,G��k�b�$���\2e��b��il4n��o����A1�l��K��Nl��t�U�����������8/�eM�I�hG;� ��D5��s�L�-WP�gq�����S��Q;b��-�;D�ZH\����������W����1�!Y��XEj���C,Q�Np���u�k4�j)\Ni�w�tv�xx���sW&������En��L���zRi3+�������eK4�	�ZM����|mB���:��Qg'����@0�(/�&Wv���$�>[�~p+_�Q�����rR17�� �:8���^K���_����/8��j�\�x��k(F������[��3���@��
�fM��%�G>����mq�j3���~��+3�v�"�/ Gw���E3�2������8s�~F%�����J[�������V�~ ������~m�.���&�f4-��]���������.��]��p.��s2�n�!�~,g�b+�cAC���'����v�p�I{'��tc!�����1i���'������s�A��&;��<�bQ��>YdT�^�'�*Z��{�^!
�Ws�9��xw���`sP�1�6���r����O�8kA������
�����5<0,��z��	�SvyV&�����������=MX3��������KS���P�
�D��f����*[��h�2[�7�mhL�|���r.r�$`�[�7%+�b�d���f�2�Bm��7�x�,�v�������\ |}d�jj�TV����]��#�la\���Rh���7i���S�����b��\�u%w<+�a6Q�����d���u�R6����
�h��5eXz�r�i����w����y�#J�]��e@�+��n��:��lb��e�pN�s���?�o��0��ld�8E�gvK��YB������0�������9��������W;c����������g�=�s���v
1/s�_c����;C���*[G�8��n$�l��k��������l����i�='}{�c[���C�\
9�|���
�cm���uN��CtRTwahSMU���8���xL�x��:'����y��
����u.	HH���e�-o�$n{��-�|������M6�U�����lv��3N�*��!!5k����(���-beU&���D���l���Q�Tw;a���4��R�9����M��G�6n.
Z�/3]p�j����,$D�E�4��b�-�.���D�����}�PwV�����`��5�{���K�f�C<���W�6����B$w��r��m��F��Y��T]�A<4�kS���G���y�����W}�9��]i��������*�J�>����$8��r���`�z��-������n���m���X���S6��#>9�`�W"��d�|x\X5�M�.�\�p<L9%��8����q1�,���m�Y��dF:!�	�Ur����������C=�Y����wk2�������0)��:v�[h�\a}�U�=���=0<���x���X8�?4u���p��X8O�6�l5&Y��UH��"4K�3��u|�0�Q������`W�w�}���4���o>����G�t����\��5�'YW�-S�Y��k8������=%9�l����nv:���k�"n�F���X���%��a>��/�|5iQY
���d�:��n���a���4��e_��I���q���?���h�#�)jkRU������\0&�v#'A���o�Z���.8p�$����><���qo%hi�4lK*Pt��������������nm����.�Y���jA�%�������R�����=hu��Q���|;D7���#�j�����kT����&!T<:���z/�v��e���S#c��n��o�s�%���n����gQ��1V=BX2��v���������9S5S�J�&"�0P�Z'��n���Z4,�����i��&s��$��4����Tdt�����S�8�q�Zi2�>-���^Q���OKo�f�H#x�d��.�z������XC��}�7��q#<��������>Vd]hu�����%������W�����=2fj��F���!�(&�-��=N?|��P�/���LlM��m�:��
���{�K���u
�d�Q�����x���r�4F��P�(}�l �z�;'
eNa����g}f��u���`K2��F�V&a_AK�kA��3}�8��������k75�W��*�QbI&����'��{2��L��m���f��A����c`����m�����(�m$�^�����c������b�$�)��n3B=��I�Y����%E�H�����3
G_Y�-y?.r'x*+������-��i(����{��
g����;����g,�����1M��\����������1s>�k9We��'��k��I�sq�k�|��|����/��&��4L:������b3��#���v�4��%��?�g#w�v����uM�����o��T{��3_T���nlw����������F/����^������������Bk������5g:;�)i��
���y�z���#1�dX�e����q�
x�$yI6��U�*{���%|���#��)��m���W�:�^�G;+���6��r���pp��RS����I���P�0����_�$���Q�~�"K�
�w�#�xlb3���v0�5�E>n_"�A2� ���!�['����>H"�t+�x����(+���p+���p�rp���k���R�x����� 4G��1��!��5���D�L���7��������	N|�:}��TTG�'���q��v���ps��l���kN�|�d�N%J�s'~������('�D�p��a��1es��T��+Bh�jk�V$>�W�\� ������{Y1�U/C�7t��]���T7�ZHm87���=�Z�M��0Bd���b4����W���A�rx�>�G���o�/TO�w�
0�\`��������_J)\�n�+�=%3�m�^���b��m�&���
��&]5�u��������<���"��D��K]��%��=��jb���)z1+�E5�0U6����H�1��$���__��b��<�V���m��h��Y`�������:���n�(>��&.�]�(�6H��&��M�����"�[^E��=�k��n_7�������08���.��H/����u�G�A��r�/�!C]�R���C��\3���0�0|c����tu\2���l����F��<�&E2q�>y!E2��vI��M��I9���C�yq����T����L�Zh<�: ����Q��&�(�L������{�m������dc��Z����b�:"C�j~��e��zST�Q��,�m�b1?mM:$n���k��!����yv3�����Zdi�%�����1��d�����`��S�4��p�
n@����$���;4~8Y�um
����_E'J�BP|5a���0}����|V�����X>������v�5���q8P���a's�%�!�����z��c�1�(=r9�4���h�F���
�������K��x���!V\�l�tU��)F�5�E�����l<��Sz$�����B_QE2?
���'��\��&��A�*rzE������Q�k���F
{�e���z��y}�2��PS���/&R@*�����u�/���=�'�i���G��V�w����5�97����j8,#<�7�{F5�#�f+>��1��W�+�[q�c5�>1������cZ��f�<�Qiy��O��Cpr���s/M�cmr&�z��������c �G�1�MUG���Dh�U}}�D��B��0"?�^{) `�����h�^�+y~��\EZ'�c�]W�������A�Yv!�7�'�&;�������(UE�����;���ioy,&��AY�����F����=��i���[kU��ki_[��-^-�a0/���v}&x2��\+s���{����JQQ!�,h��/
����BZ�7O�@w��'��[l3���"Sdp��M�dH��v zE�b�_�N[<�V��9��?.���h����X���e=���b��|`C��V�[�W���<��W���
���Vr�41q�ZG.�x�����f<X�lI�g������c�:����p}zC���V[N���(Oy�E�����(z����1��H>�_��>^��(6JzM���i�%El��*P��!;wT���X�;7iY���d&����TyU.x�b��Q_��E�q��d�s{-���}V���U�xt���"������UZ���-NDSun�@��Y��UQ�M��v�	nS`R���@����"��I{���u\z�����9^���0��G�a����V*m��G�t�\����Mz��HM)�{E��B�_)k����v�)�^��)��}��^��t���b�!��'�^���s=��jq������!������|e�gia�BXXN��K�CT���V'�q�����	w�6�O�o!��Z`oH.W���2^��.��e1���1��F�u�D������m#��^������a C3����_���V�9�v��"�
�
��,=�J��@����e���&���!t��p|as".������s�.7'���Z��*�L&w���2,�1SJ���	�	_�'7���kQic��� �]1'��;:�Fr��O����H�P��I�L���P�8X?t�(J�kC��2w�u�
�.P;st�i�����n�{(�*F�?�j�����f@1��7���X�w�6���H����9��Z��%i:X�ew�x	�����[��m����br�cN��,P��*�I
�p�H���&�T�#�GG'����9���}���&�MJF$<oQ�G7�^
�_�B��":7��a �%*I�����D����R���A��_��������`��v�F�	�����x�a��:���R/R3���qx�8q���)�~"��[|���-k7�����Nnb�gB�[�w�
M��5-e�����?5��?�7��e7���������[����C(����c�u3[B��f�W]��2�����=�F���Gl��]��Vz��c��UW��ld$���*k��Cb�0��?�Sy��x��%���$�E>�H�Rg�{f?����R�Q
�������YNSb��"XXb�&�$�-�=����Nl�R��s�0��aTCu�~�l�?�\q�0��z&u�W:����/��rf������d���}��e�Hn����M����4JT�Gn����Y�gWo~zv?��(�@2��j�x�0m�/����/Nz����?���R�\�x������Vx�8��mhepI����"��y�������U���H��*|��o��k��Ed0���=F���	�TkZ$2b���:q1^��
	O�.��T�3�nu��b���%��h/���^�'�.��NOzb�_����+<~_�=t�4|��O]��h����=-�u7jw���@����;>��V�!i�1�r��G����=���N�������y��a���[M!r��s9��O��|���/yYf��W���B��[��Ykm��>7��`�6"Z���E�f����:iV�|�R����&h����ZY|�V���O
���9qh�X�t;k,e���:3XX�T'~w�'�4{"�V�������	3+Q�����d���5��k4���{�s��RrP;�GY���DK�Yd�T��b\{��s:�����|�(L�����;G?���Y�|�����}��s�i���9�S[�� �-qk�4��]AZ&{��Q��w��A��t����/��x��9�RG��u���OD{n=����@��G��XuQ���5���5�(A���7-��w�.[����lV�V7�?8���7o.��!��
ro x����|Q�.�w�RG	eKuN�����%�x���9������2T�����o�y;�z�l���h��aQo�A0w�)�s�dS������xxq�	�ap���t)�c)��='!����F���p��!4�$sMK���Fl%������\��8m3IuU�6�
��vA|����W���g��8#P�6�!j$����eh�
������3\|\�����r��B����FIY;LB����mb�a�n�Vc^�*�F
��x��C�l�!u������rLY����,x�J�����b�K&K��YD	SE���E��L�r���t4��df��N�DB�������<\��|�nQ���f�2��A�:���� IR�8�N���7�N�.s�_J/��1��
6��������%-;f�������[�����_��.���r�w�V������~����a������+��WT��+MP�9����
t�r%�����>�F�/��
���Sx�7X��,�H�~����eKo���s��Z��O���.�_]\����x���o.���}q���s�8l�q�?��y����o�h?��/���@��X?z�S��Le!(-9���M�[�r�?]�����<:�����^�2c�T�����,�����>�v���/-�����=D?8�8R�Z�ma���%��~��������������o�������{u�]\����i��O_�_��u�f�Vo���vN���x�]���i�/G���M}��h2AgO������k���Y��Ix�vu����g�^�?����uW����������N�e_*�fi!�u[��H�Wo?�vq�����/�q�T����:3�`��DX.�}F�4��l�v<�"(��)���-�Z�o��E������89�����A2������X���!> �r��,[(�c!�n/��q�������i��2�C�O]�7
�@����2~��!����,��[j����e��N�O��a�=�?�G��[I�	U�<��
���uWS�Y�+��'�D=	���3dN��@%��&jt�J=c76J���:X�A�C�L@�X������1�����r��^c��������`�����N�r�u�p����I��2��3�����jN���j���$"��H�u���o��q�d,�"1�?�/D��*�tR�d�0K���UpE~S$SN�0�����NY��X�#��)^Z�^��H--��Gn,�QqVCV,2���s�Y����{����$�/�9�[B8���M��$&��z�h��Y���t��#p�4An��4�L��J�qpI�C�v����i���b�����
t��e����y���/^?e�s�1gr�������������kC�������!��������CWM�ueH����#��#G���D�#���(_�oH\����B�70�N:�	��{��D�����{���6���7��}��bO�v_�U������'��3�K������z%�$�L�Y�J9rx�@����A�c�:T&��{<�+����G���m]����Z===��S�x!S�{�l�/����SpIIn�j���G\�+c���U6��z�d����;���������|j�l��A�6ZUe��S��r��KZ���I�b���MuO:'&>��g �V[��RN1'�h���0a^��#����x��D����q1J���y"
l�0+�-]?���L-E��d��r�nQ�����j���e�T�������.�8'��9�
�v;/3�(l�c�D�n�_��xu�����t�H��9Z�J����n��������F^�����ah�#����b�0r���w����A�b���vt�t��M)A��r�{���N���!As������������0''$0tO{��^��
r�#�����T��C�Y�l�����
�������t���
��6H5��d&4�vYZ�9�`��z���"��Dg
u.�u*���0@�7�t���{�8X:z'���
j8`�f6�z��5Z	l'R`��A�rA���Ia�B���T�T�)�,I�Y�@�~6��P
��O����$��s�W���sJ��Y��gs6��J(���E��&��n��u�����~��W�P�*MH���d���3aW���`���W��Qd����;x��/^\<����E����������P��w�;9��v�q�-
��]�tZ��R�j�Z-&���(]��Lb*�1���$���[�$�/Km�
�-�~%���9��oA�"��������C2T��.l/
�g�^���oZ��"����7�c��>���<�I���-�q���?N@�F���O[�)\��0�N�P�����b�m�K�i�}2����79U�U'�i�z�#�S��,�aA���Z�e7N/�\i\��6O9�A|;��r�l�����p;[������m�*U��ge��v��&������]D���D��..7vVP$
:�
=9i���K�&�
&nw|�����\����������3����.�ic������^��o��@o���y_���.��0Xj\r���/��}~��W���������q�;=<F�q��?�nJ2�
�j�e�Pr��J�	k'U�$i��N�.
�0��R�v��C���I��N&��Ap��:�|�����8�c��j{�9�#Y�=����}[H}u?�^�<H3�12o�]_��}����?��<J��f%�+�]^�L��z�
�����#������V��� w+]��N�:%�����e~{W_�L]��5�����!zxe&�e�������
���l���Y(�B$���(���������t���Ag������
O��4�k��]b'l<s|�}r����f6UDdH?b��:��?���U��>b�������d
j�=�L�5�c�����g/��z�c����	�#��p��v��v��d����l�o`%�q�����gw�A1fCk��F���S���[�-���kZP��|u�g���FE�j�������v����/�*kU�sn^.������vh����xWs��S{Ez���=����8mT�
>����'���1��������Q��m����?!�O")s�
����4�����\��{T-`�]c��P.)�t�s��D���-��#t)�0+Xq��H^�����#�<o�?%����,gj�#�jo,�{����\i��(k
U�� p��D]v9���8�vc���z�
�t��3�6.Y��$�50���tF�6D�������r-��
��N�'�J�''��Q�)zy��!��Q���Z���Gu�%-B�����B`?��_������
��N\C(��8XVe�]��_�T�j�q��	7J�!8I�-�)���,�8����'�	l�N����y|��U�?z�������o��0d�,���g���!��C�brsc���A��#A�uy������L������O�|��L3�U�aU'gm�w ��w�n����;����!�vjv)����V9���7,�����v�b-�-�m�4h�og��������ZD�����1>"�Z����h��H���f���q�5�/ �}���r�&���R���7��ew�b�����S[�c����i
�(QVS��'��A��REJ(����&��
�b9�R���~�7������7H���&�j����dR"��~3��d'v>���V�n��}�>l4�PKa���%VgEPK��jJUpg_dp_join_patches_v2/0010-Adjust-join-related-to-code-to-accept-child-relation.patchUX���X���X��Zio�H�l���/�eK��C����:Hq��X,�lZ�)���3��}_U7�V3�DvW��� _������I�7�����:R=�9�{N��=���s|tr8<��8��`�zg�
z���d��"�y�M���89��1��1��EE�J�4��7��x������3�-T���Z�A��������1���z�M1�����������
�{������\d9}���R��GyLn�)�t\W%9�� ��� �2����YDw�;e��,z��������Lu�d��l�����	6G�`�W'
�"#?��d�;q�;�3qr����(��:��Pr��
����^�)ejtxwF3EhB�e�m�M�����JAS�mE�t:V)��!	����)��FX7�K�"�v[Zr���e���,�4�~sR����
�y���r�L�r\��H�>����A>���M����iD0F��y�iR�
��!�*�)-\�	��An�����"7,���y�Z5R��Ci�jO�h�)k������p��M~����%��/E���`��U^7q�����N�!�w/y��~W��h�%O��s�"�q�����>�b�;����B���gUf]Ba��,�J�,�RS� 5M��>"TR�4oeK���d��E86�>8���8(����
�iqT[��I�D,�M�
�X�Xk���Y��
6�^Z�BSt"�h�"c��<f*���bO,U��3�����T�5��4�h#04�Wp������j�|����T����s�������b&���B�\fH�e� �vk���pyL[�E��j�aHM�WGQ�o%N����5o�%�Z����H�g�j�lz+����0D���Lg,����,3�n@���y.B'�M	�O��+��������>���
S%���)�&q�q��PX�r77�!�J�U�
i-h�����w��f��1l����2���Q���]p�E�}�d���>@K��n>a���u���(�F����B��U'��<~�������\�"'��It���������&�9Y��������!���-�"|���r���a,He��_���{UJ�Xy;�W�K9D�������@$�L�������)�[�[?3'�Y�R����rG���<,`$�^jGd��\�^�%����Fo�6��(�>"��(���O��I���$Q�7b������^�o��3����@�HM���X`�d�h�M��pY'(��c�#.���z���zA������>�g�c7�\t�`!�t��(���F~�1����k�*�]t�(�Q�RI9���s[V�r�^�
�*�@J�4�B�Y\mpDesIm�%���>r�W����i/���-1�\��E�0���"2B�.k���,������@��T�B	hsM�t)���\��vZg,��;��6:�:��g�,�:Z��"���D*b�>��C�3P�!l��J�_�1hg�����n��������8:�Wv)x��9.G�MR�G\#�eC�	`���a�\�MG�,��Pq���*�"C�U�I���@[���Bk:��j�B|Z�y��5���5�����,�gjY�����E��:��y�aMWn�,��b��N?G��t:e��;�=�'���?��l�.��?3�%����a���:sk������\�78�����yf]���K�%4C��T�T��Z@5`��k���������}��A�jN��iY��|�:�;�0���*����`�����7t������9�~�w|x����D����_~�������T�9.��e���G��B�:�qE�m�vv�gt�����z?�P��$)+G��E�7a��R��.KB�������J�\6��"x
��2�&�No�7�e�D���]�(/�i}�����*B7+3]\���3���y�0�#7���/4L��R#I.@%��dW�;�Y���,UwN���imk_�_BQ����K���U��;�����2�
%N|
%�s���H��b�}��Q&>��X�N�{��e�6k��:3�#c��
9��`���K�n23Z�
�0�������j�$]��;�H�)4v�"�|���S����T���K���8�g�x2��ul7Z�rUwU�5��@D��>N��-�W�h��+�<�9�
�����!���[D���
n����'F$#�X�8��,�*��@���Z��B�
����^37�f�/B��.l��9��c��Z�p��I��R����4�D��G�(H����d���G��u�za���n�Q!df,�Ee����5haH=���Ee�G0cZ����+����I��e��������e(M���0s�$Z�Z��8S�2�I����
(m9����~{0��~�?l��dZ�a���P/�$�i$��+���'U3����
B�; �Y�|����ck:�� X����$�y�<�u��:�u��
�5�P?��#����E�������=�[����[D���P�yj;����P[Vk���eg���3��G;�<$����4^������+My�����l�T�}u4��<^���@^Z�`68�����'�d9�L�#!��
�� T>�T��u���C���k���q�6R�Hwh#��Q��%��)��9X�V�=��7n}�������2��[7T�,��' �(v���M�sB738�w�UaH8�}�uz�&�q���V��g�I�:>:f{�s����	�>8�TF��%=��\���A�jo�%/g"5����>�������������7���C|���sg��v��?M�M�}��@���NAS�+�G>����	�]��VMQ6�!���i�':��{�l��K�l��X�'K�����p����Kb����,��]f����g�����zkI��+���������_E�d&��G�%<{j*� �pI2����Q�I*��N���w�NgB����"1���
���q��ugGk�����V��^J8Q�<�Z�n�;J����e�EO6/m/�l�Q�G4�\�2w\N�����U�,�h5�^Kd�y����%�^���iN��m�e��C�5�v��,���|�Y���2��F��-]6KS�4K��j���A��.�Ln�}��������E�:�?�Q6��\���'��n�n��o��)L�����o�)*�����3c����Hq�\�1OQ�l�2v����G�k�Q�p��f2;��1�I��cf���?0�������U�#����Sfi��c�)�4�w�aI%t����9���v���h�{	���j1��F�St�0���������^��s�t��>���^�Aw�H�R`�@���2
�m�������2~(_��:}Y�|�c_�z�H�g#�����~K`�CXf��P�Z�,���,��k�����������������.��0<�����B�Q9t�C��;�;�M0�~Z�|�_�,���:3~�P���o2@�C
���ER��e������K�fj����>���������.1��;��������s���J"����eR�N���)���76W�E���J�(e~� �;J��Vp5��q�@�i�Y(�K_/�W�8�����L������;a�y�r�������G�����������W7W��k�mW���{���n�\}]_]���>�����@E/�'����g�Y}{h��G��PK��!�q)-PK��jJ9pg_dp_join_patches_v2/0011-Parameterized-path-fixes.patchUX���X���X��[ys�������IU���i?�����S�"+�J���C608,+k��1�� !'*�E====}��g�2����b!��'��B���B����|�>�^.�r����r5���q$>����������t��6��U���8��_�����q�����LF�LvI�J5���S�W7���M.G�\��+`�g�����O�����1_}�^�H��������lv<;���M������/��F���2�8�oR����Lx��)�V&�R����H�SD4 w�l#EE2a.A
�
����H�7������/2I��)���� Z��Vf��\��bd�d�n���������'�y���L�Y*���W�W�Wo���cy���Y�_���E�����F-f�!�(�~+<��/&H�B5���A��������x�WX{�$?�J�	��Y%,'����WLK�C����_�L�0"1`�^��	�E���G�N
���v�"��*���C�����l� �]��?@�4��Q}�b��r�^�-���yq}�F����w��
��L��"��.l�{���� �K�2�AP7��&�d)yG�1�id��nd���?*�_;���&�
�*K�k^�=��!+B�@� �v0N�`J�1q�E��
�[��p��-
�E���#J
�����b\�y/\�$h��1v�4BM�b�����N�n��s�[#a�����3ZVe��\[j\_��o`�:�<ZD"M�K$Y�����?�gor�P���<�����_�2������R����	g��A����#�r�Q	>�`��%�c��"����
EpCZj����E����d�wP���}�Z
N���xM����$��[7
voG�z�7l�F��<>�@qWaWE[������pQ0�(���*�8���i��\�����x�[ M���������b�C�i3�y���h���F��,���f�b_���S5�����@}��!�B	�������7�h����5���8��	%�o7���a��)�*�:���
_����Id-�����q�Yu�s�L�W�N�//=2Y��N����M�g''h]gt@��g}�L����K1��g���3G�Wq�Y$����R��oe�)3K����}��t��K�	v}t{!3zB��<e����N��O����y���������$��6a��������8���?�b�����=:ss@v��>�E
���/����@�^��O�8&��'���� X��&]������ #3���0�>r?������`3��jZ~|��eEW��E�2�=S��pX�o�i�
&�!�Pd��/	����@raO�Y ����%q��`�Z��2|����2X�5��8�,C�F�����L������f���Ys���	�`�-N>�;�����V.zh�=�7�U�(��M���0����M7P��2�:���A�`�8`��M*�V?�4�hn�sS\a���I������T�\I:���Wd�b�$�G��1�1q�A<j���3��
�<����`/�X��Ip��YQf����m�V�l��H����X�!<s�\A�Od5��
��kB�z��~�;���t�&�	�����J:�{q
9�0����a���2�[X?�9��f���~�\x
�c]�}�`Z�O��"�Y�D0Y�;M�*P��7���*N�DsA�#�kliDn�Hl(cf�j~��A���d��&�<��06���	�e�m���D��R���4��'�=�q`��I���d5���_��G���y��m�tB7U��
4*�
��zS9��M&�l>?[�:��*���J�A|6���!�:����;���H�yw�N�K7��t���(N�L���Hh�-� �G�X���
��b� �B��!��K����I��;�-�-%���y�uS��Q�9i����
�(��JD��XI��S���	j��f���������m���
H��|�����b��dEx��d��������|'�<��3���B��)�x=�2���J���m<��� ���P����S�7���RF�z���1jn�57���TP�
�-���f�'S���M9@^H
;�k?�/t�T���S�����������3��J�M{SLeQR�?����6>�-6n�)����H\��ene�,u������Ah�����E��6���a�B:�"n�����Q��5^��A�C���Hm��72h���!(h�n:IjV��E��Z\l�6J!�p�b��T���hv���F)K���;L#������wT�_dT�0g�E����Op���L���Xy8�����`��8q6�H�R�7�Z����6JkY�P�<#>��0�"HDn�I���{VnX<Hv��W����3l2��c��`��������/A�uw�J�����]!����A�����EzU�B)	����`�Jl�6���a"\<�P.U����HN�[�3Vmd-�����Q@l� m_5H���1T�F�C�]P,�#��G�):��	C��N����/�F��S�PB*8����L��Cu����D�������������/<�![��(<Qx�++�C����)�X�����4Y��Q�=��mc
����y/������R��U���<�����{_o�����z�d��`6������u>g���8�FhOO}�$����}��X���� 
.f`�P�b6�R�.��������b���M��{����z$l
�B��F�k���O� fZ��M�l�CU�oi����G0��V27�Q~�"bAX���C��k�9�7
��wS�Ig��<�QB�Eg5��Nj��2vUc��~5�p��K�����Q�����������l���z���R����R����������O)�5t�B�����*���u������"�z���rw�C#�@[��U\���}�����_,�u�n��1c%5����M
���\���>=6K��#z�5GX���]��8��K
����2�1k�c�E����:>r�|/e��_�>_b�.��yuF���&?�&�~�h�Q��qa|�@'	��s>_#�0�a��H�9C64g��C}�@�E2���;�8���1�LaR|�q�����w�z%�p� ����M��
���$��F��]xj����(��T4�,-6�(�Tl�>y��
������B�-��:@(�b����L#��{����	�.$n�t��vPoD�-�[|��f��o�,T�C���7*�A��r~�Y�T��^���q�>[��ir=��q�o����R�:�QR���@�����:IQ������(Fu��n����R�$�A�.
�����{�8��N"U�4�E��J�����>S��f�������n&`���N��I��������r�Bm�+=��$�v�(�w�8�tfE���2>9���qTH��.]R��/:��P��5��Zw��+{���D�&�A����0XEp
R�Z���H\��P��#��&���O+�L�cj�H3�2��Y����i��ITW�d F/j~V���C�NB��d�v�
aBmWfbyL����}Ldc���ZN\R) 5�K�xn
��������}I@��s2H�$�J�ZW�;��z�+���M��B�x1w.��8�j�& H�N���D)�b):79��C�;UEit�_UZ��@f+��)�(����z�l�b�^�o�;YF2�^N�y"B�j����	�1��="���f�H�M��d��t	���t<7�+w���R�\		�����*�0\v����
-��G@���������
��\<
M�t���w�^3�c:��~��a�Q
zB�e���DV7�f�� ���S�A�a��eh�������8����k�Z0�����W���I
=�����$����}�D�K��[W�������N��B���������^��OM��Fi���X�d���j��DUU�&Nu��h{l	@���+X���������X33!Z�4��C����aI�\`!r������YV#�V�AO?�v�-�xhY�oJ�Z���Cem0S��j��Q�����^�Q��k,^��z����"��|��Ye�:e�"�w-Q�B���#W��|X_(u/zGE{"��~c����*������h����Mgq��\:l���~�u��0h���Z�)A*���~%W�Q����8
���g���p4]�T}=:5��9���v���ks�fhUTS���e�G�"�T��	[�g����Q�n�W���D��{}u�Z�]�������z�B��0�����BR|�fos� �'
g�����L.������D�dt�@�$������B���&#5�-6:u��~�}���������(`���N3�u��"����[Q#�1_(���f��i:�w������L��R&���Qcir ����	*��%����MQg_��CTcC,x�@9"b�jv���`��$�~/V���8�L�������Dl/��v������d�s�_xP�!���|dhz�Z
F8��N�����K?���9}���s�yh���4��<6�w�Kf17���b�������]��m�f�2�(�Hd��f�M�_�u�C���c~`�V�v�F�'aa�%,8������G4����b�
�Q��_�:����b1�-���tqr�Z�wrLf��)�-��
�?3��I[��9B���PG�Q��J��/�e���`
J%�yg�����i����e���W7���`����|r99u�PKL�e��CPK��jJJpg_dp_join_patches_v2/0012-Use-IS_JOIN_REL-instead-of-RELOPT_JOINREL.patchUX���X���X��Xmo�8�l�����K��'��ij�f�\�8�p8�D��������^��o��l�I��_�AV�p8|��v$�9�}F~�����m6Y��F�w�~����^�o5{}��R���}h6O�h5��5B5�p���2�g��-���3�����������s�N����OL�S���u8��Q�v;�������f���&K�w�S������p[
����s��x��^���^Wk���,�,����E��e�>�ArX�<�Y�A0���f\����OA�����8�{��	+��c&dLk�
���r���8	m%^h���<f��9<�[����_�w�9��@d;w\.!��C�	�39���Y��?�i��-�	���\���q��VZp,k�����O�Q�
��|����V�W�&�o������U�d�Bc���D[�X�m[��T��o,�\N��(\5B�`"�N��+�u�pL?������-o;��r4|<�4l��<���X��A���	'��tJlu;�6.����5�iA��'��e�q�mOc	�q��������A�1�a-����7k�:�����9>�=p�z���fu���D��(w<�NPQ:�J��eT�q���5q$�L��n����nA�0�q^b1�8$���	*���`�D�}�\,����
�����EN~���}����$�����Q���u\H��7�h*$��z>^L�F�!�'q��=��c-\;�n4�[���-$��#�H&8�A ��$����R �f���Q��e��������_U��w�������Q�`�T��^	8�(���]�~P��/mi��������i�1TKLJ�����t�m�7xR%"8�?�,9`�W�,��<`t�/Q������������r?���F�}�{��g+R�c_�/��l-�~e&�xq,R�����
	#/�����7�_g*�^*��^�V�F�w�y���~��X��q��kr�$f9 W\UeU��gX��JE� �5��	�	Pm���oG�S�S��N
������7����U'K���R�|�_<�����i)8�������7���M�5��������,b�C����Y������FoO��)c�v*�G�i��l�}'N�='�cBE	.�;}��u[���n��7����:'���{E������HM��v{N�$S.=�h�x<@�Lx����xM�������x���)�T�+r��s4	�����u�9g"�9:�������I�DN�CaOO���<}PU�N�*�(}�T�����W���aj����:$aWt�n������I[Gk��p�����3>P�#����H$:�TK�x]�c�tU������:Rq	���wi)s�.*K��%�N_����N�A����oMo�M��9{�WX��\T��q��<�����J�
WJ��+�;�Z\�����9��C&Y����nK����_z�ubX�b�K������@�����x~���](�9�����
1F�t���d9��qFm�r���H�xY`k���������z�������t�K4�NaB�LI/u���z��rH�x���!�dB.J�T
)������L���D��{��o�:������
�L�r���g�����������G�����������h�������~U�]�z�	L��G$��5E��Jc$D�\��C��=z�\�$��<J����<�������)&��n��IP��^�*q^Wu������B'e��m�
��aP��fvkCMGK��,>����~}G���LxQY�xx�
����R����z7hH����R�m�8�n��JH1z-��;�kY�PK����eKPK��jJHpg_dp_join_patches_v2/0013-Multi-level-partitioned-table-expansion.patchUX���X���X��Y[s"�~�_���,�`�c�]q|������yH�(1#����D��l����40\�1�.����������������f��U�C����^�u�hV�
^m�z�Z��~��<��T�����U7�b��\F*���%L�gf��=z��G��D�������L�c�A	�*\�JC�n��Y=vpPm����Q�/��c����s�vW����e��	�WB��
TG��z!�-a���r>GR��� �\�E�A��`q�
*��!.
j�g���exq��q��`!�������h��q4/`�t9�8N��*=�������
��?\T��E� 
�������B���O��(���Y�#3k���>F��RU\�
�7���5[9)���3y��u�!��
X��������,K�h��Cn�b>��>8�s��U�4Co��� ��7�5�W�5��Z��j�jj��F�,���y������8��j���m�x�c�tm�t)L�&��qq���I��*�!�/��=��>Q��'0�O���3��w����`�?����!��rA
f�)��~{���;g��J��U��Sx����t>���b�`y�n�yq��h�O��^�D��	n5|�1�%���+$b���>�`�a8�
�2VA#$)Nea���l}b*z!��N�hv+����e���(��=6�\�J;'B�� z�%�@`��j��(�����y����#��=�������1\BS�f2a�d�Fr�--����B�
bR(��,@��q[V��2h�q<
}�/��hM5�efO��g��2A����F� ���E�e*2v��W���)�����F^�Jo���b6*@�$p>`��u����(`���Bqa������n��1����V�(j#,&
h�o|�
���B��3G�0V]bs��yq��$\�`�~��N&����.����"6��H(��W��'�g���W�	u��'5��q�.���`Z��5�����a%x�$���W4/=i�o�W0p���6T����/��W���0���C��I�����i�z_P�ucB�yNm�F�vG|�����}_������[w�D���� _.�As�\M�Zh.����wJod��<8#$s�6�b��w��<%}�f����u�@M��>1�$��s�Z��dA��(6�"ND����U� �-|� �E�W���)VR��)�����&����e�3h,�� B���&z��j
+�TY�`Hh��.����Y
��Zc89���k���<jA���a
����Ph��S0~����Y�^
v�SXh-B��M��q�K�\�R=��/���(�=�����@�06C�4Qw���8C��T�B,G���j�S,�TM��&
T��5b8� �B��M*��7�,��/��qQ���S�t������g�g��Qf�*�4�O�4e���39^[F&���,��v��u�Ep�	���mF�9�t�B`��@�T�b�&�)�PgD;"\YI���o^8��W~��As��x*���J�yXy�e��������U�����+2���������:�8��Z��W���p}�g�,��K�����'l�AVS��(AI�\
�]bX�!I,e@x���z%jS���<�4���e+/���O��\��)/�Q]�\W�c*z,�A��C��S"�K���sS��E_i)<D��(��A����V�2�3�9#�(�W�1s`*q���l	�%
���g��]��Y��y��&�f���)������I�h?	�4�Y�u�4�E6�q�B����/�e�$U�r}�yg��)6�{�U��-��J('���s�������o.�)C�3���a���4�W�SO�Uq�(U��!�R�/�;L�K5<���*F���(���:�L���#/�cD�/-��dD��������]R���H���d��+9�IE��2���D��gZ#��C�u�A���P�F:z�����	�C�u8]��f��6���1��D��#�����Kp�m�/�a�I��pQF;�L���:kSL��7�Ug�����U�w'���r��P+���S}�����e����s�������v���"����������nSQa���������T��No?�1(9��^��$fr��Li
gs=����'x�.J���0���F���R�c!W)Z2w��x���q�b��j��9�~���A���V,����Qt�J�(v�MMk�G�C�b���{�k�	�F"���������|�K�[���;=����SK�h�@>��@���&��.+������y������+��vKnm��3����^�L���;�SS�����O����H�����l���%&u�d&�(JY(�=%�����@�������_�#A�Y������]�4���������p������#���,�GC��bH��.�KL����o#���w7�z��=������)A*�hk�f��A�<������E8����PX~��p���6��\T8K��+do�.&Z-N�X������H�,������,��}z�����s�)�3f�2g.]����]o�q����n��~�\n����������fw����Q�+��&��}0qR�]�b�(����#����B��G��v�Q1�d�����dI$���+G�_7��[�;k{��������o�nT���kw�D�<�����[?��7��GlZ��q���q	3b�����o\��6x��FahB�E?��a�>��X��]{��#�}<�
��u���
i��q/���+��[��"hw�1���eX�n%d+��*���O�,�vW�X�}���nwU�����<8_�Sp���,Z61
4���] dX��1��]�qM�Y�X���)��p~w	�����%�w������Z`-��i�5���%�����a��������Ml"���Q��-���[����;A��V@�Z��Y�����w�U�Tw8l��
�vB�.�6ry���;��>�v�����
wb��4�L�����7Q�.*��������W1;����y�L�8��n�Un�����PK<�K&�-PK��jJFpg_dp_join_patches_v2/0014-Multi-level-partition-wise-join-tests.patchUX���X���X��=ks�8�����T���=�&�����ng&{�������[[.J�1nIw������%�t�a*eHG��t�.<��j���vf���+�������2��)Z��c]=Q��;w7x
�	(�9��(j���9����������pI�_�U���g��ZM��o��F�������*p��������s�]S:7��wl������7?��G��x:�=p�#v`mx���j�D��������?�t�A�����&`�u�@{.X�����v�a��'���	r���yD8����}���1���{l����'�Ap8����g%}��*F�l��w��!�7��t`����v������VNNG`����=�a��L���^LA��ka"'�x4��@��r����t,���;;��):�l����O��YR:����q�m8<;Y����� Y��h:N��a��_����W�P�S}��sH�OV�z����1V�o&����^_�K�'l��0��2T�������v��of��|����	��@]8�5�2}XF;
�����������rz
������I������X�������t���~����l��J���E�"��� Q��(<\�k���K��@!CDc(#��_���.�����jr,a����E,Y���D+H/��&d��A�@������pyM����L�&o�@n=�/pTI�����w�v	'�;��qX����U�������&��������`�u�p�s��	��[k�s|Z��|�vz�{���X�E���������������{�+���0��#&�L� ���0���aAja���M���#~nU2��<������w��<��:$th'����CW�y ������yp�?:T�<�^Q�D��y������<P�y ��@�������8�w�e�8��Q��"�e�������_^C������f��7���
��E��y�@}�k�_4\�_3����~"��?��&�������[ :6�c�H''���NfT���H/�.g>�������7 �\K5�Lh#���^�P���:$�.>�X����S����k��2t��?�� ����@��@��z���s�v�\�����<�)�w~�������^o�����X�u�tls��J$��k1�	�U6t���'��Z�d�E�����:�n��=���]�C.b���2�WJ��e�]���P�]�"a	y!�`*�R��E�eH3qG�B��mY�/v�<��E��c;YH��"�j
$#��egN4�<b�TK�uy�g�X����|�����<�Y�Z<�K^���IH^K$��$�%�UI~�&$��xVH���r9yr��>h$;�(q���L�u��xD}'�(F	����"/	�A$&^�p�n9��DS����k���w�>q��;�N�:�������
���]�<��'�~�~���M`��� ���3��c�B�u$�'�����h��+k�D���\E��]z@�5���0�����Y}:�6h}�	W��9�}J���^�r7���"4�Wb):c������/pA*%�����!_��TN�����!_��TN��%j���K�,��u�D����%�s/Z^|A�vR��
�G�d�ZKU�g�5�G��(j�����:-j�����&�<l����|{=j��`wU�pm��r���$f��f3�����jh��Ra���
�"����'���%3��ha����uq~J�*�f:ZX��.ER�?���b.&�J��~U�d�A�-s�O�M0Qi���}_n�\��E�T��p*W�|�@Me)
��Q"~�q�h�j*��;��@_n�]���N��C�<��R�i�uu�,�"G�T�f���-�;��w2������;Z�;ZNw�����IR
>�%��.�_j�[����P)\	y��w���mv���Vn��I�
�	���UP3j������`�5z���~�00�����9-������Dg����+�+dW��E}��>z�G�}���r�S;��O�p:N[q�.���X@�A&�������*�v_��K2��z�-Q���A��V?����3�"�x��M��r$=0���,T��A1(S����h�UT�J�}
��n��m���d��#3���o�� VL�E�
����,M�j4��ZE4*#z�19H�W����
j�
��������*�DZ�vSD$��|.Hk$j[)%�^�T����$�M5E������E2y�y5E\5E"5mXD��+��.�QSaeJ^2�j*��*6�
6�08�% �������UP-�je�N^F/�m����\5=���R�rD^>T.�%�Z�p.�
��0�
���b���|-��i�ST�
u�*�����n���F(M"������[Bs�������S'���V �n�!*L;�W��,���3����$��M��
�
��t1���aN��:��1���A2�xS��C�����B���|2_u���;��f@���7b�g�-D�$���F�9����}�������(��"���D�Q��t���QK���?$xem���(r|�0��a�+���fl��"O��
����f��8#���E�g��l��.��r��mv���Ap��k���d��6��f��F�
�ei]��<WDR�.:��S
3_#?H�}��XR]��U^X��C�����.V�,�(�/�T�k�r�,vF#�
JW�a����V�)\&E�q�����L�������3�����=nP�6r�#�E���y9�JW����gQ��Y�QAjJle1�V��S���p�����{�:��-am[��%����;8��Bt�T���F�,6�������H���(n��.� *��)��D)�JW�A��A�f���Sb++v�z�Z����� 8<o�AH�(��Q�m)����V��� ���eR�|����T[�W�fFr���`�Y+�|V/���94|F/�-=L,���c/��R�����_��	Z�GRb[,���k~$\��T�����[�
F�
F�4��m�I������V���n�=��[Z��.����C-�������5{�>+j?����Q��Q��QK��sc�<�����7��z�
�%n����s����[i�}&"��0
Y�?)������W��I��/y���&�i��A�K��T'����Nm��t`Y�B~���d��4b�8Q^��d,�S�:���lZ���,'�VrR�e}
�L��(F ����<K��r�������&������X~X0�po�{�/�����m?S���[����}x��{0#����=�>��;ttB�T:���7W����5Mk>QM���0T�_��A�vW�����(h�~Am�lg�<gU��,��(�R'�"�0E�o��l%�U�5{��l�p�L\��ns�g��qW�w�w�w,�����1��T�w�w�w"��I�Kz�^�&���*���i�*���Gu��&_��N��uF����d�V�+�\
�|�3���R������K��U��X��s]G��,��Yj(�T��|��3�Y�Ra�����[��U/�XI�s
O��bu�D
e���g��17�������2�-��w�����m�����z�����dv[���_+3�����s�-eoyt������m=[�R��G�^Fw1;�}���b���>C��gh�����������J����[���#�3��G�����
�p�,|�[��a)��c�,��c�:�5�C�L���*��h�!:/5V���Pn�W�`�*Z��9A����4�e��p����������x����;�
���B2��:"�>v�,m��������L����}�Z��?���>���������p����c��}0���k�>6<���-��
�x"Lu����C��m�2)�� p}0�G�w���5���}B.�Y�������G1\��h���t�O�����_7}�:��p�#Q��M4��5���������b�0<��� O�6��3V��W��X�����WX��=�(��B��
U��G���!�M�V��l���Y�������(�@"�N�/��������������06W����i��#qg&,m��:+��6��k�����hO�j*g:c����V���Y=&e��W��g:�/+���=���k����������zi���w�SP��Q��P0T��v��of2����U��%����>�-��	���
�?�g���%�����l|���k�~�B������"~1�����_&7��w�E�K�	BHH�`��-W���!����:����h�:�8(���^�����ep59�0NPp�"�,�CY�����Q2�Q��N ^^�Lfs��&�br��������J�%�\��v{/E��
{F���a���A?;���M���W��+�r�����4���x���1����������$N��cz���M�����}��_z%w:r��/n��0��#&�L}�.���Yl
RKVpd���&1S3��>������w��>���$th'����CW�} ������}p�?�T�>�^Q�DZ�"vhw��m�Tt��>�} ��@|��8���zY����������k����;.i�) |�#>�����A�A!|�*pZ@��WO��s�_&����_�Gl�,T>�������|�t|�o��������)�������)��^(�����`��2�gG���z��_PK5��+���PK��jJHpg_dp_join_patches_v2/0015-Multi-level-partition-wise-join-support.patchUX���X���X��=�s���?��5o���H�H��OdG��'[�����t8 x��V����ow��?�I�)w{{{�}{��co��t��r�-���ry���~6����`�-&��H74m9d��-��;6<e��s����`��0�����{��b�����?��?��o������\���M{�����������l8`�|��p�|0}>����X�n����=g��p~��
�O���w��g���_���x�g����g?�������v�~�u����n��z���.m����_0���9�?���=s��p�1�kFc�-�iD��,���z�����e+��7<l��;���|���������H�>�?�o���t��"}��\t3�~��o�HG�n����b����9ZcX��)�t�j��y�G6�����g�6���;X�>�[Y��a��E���&/����1���O|+��������������vw]�	3?�]o��{��.����f8���qFP��~�������'s���;�w�����	�=���o�o���~��8�7,�	-*�?���70oNo��>�[�\lp���gw���	����)��#�\�4����p�,&��mm��~�OpL���������_DA�E�s��nVo�<���~����/N�7f����g(�}�	��v:e������l	����{��2}<Fr�����q�t�|���^��ZX�%���?)�N�r�Z�v�b�!������&�3�%8��F��eGl�LK���7�wv���	�[��e�O�*�{3����!R{���z��)[���d������'|�mE�sh"2��� 	�r����5�����5�!������0-�� ����sn��F$f�v���DU�1�=0b+EM-�����aQ;�gH�p���=w+�M������6��w����s�����[��&���p�����r|��
���9c�]o�
��M�=c���qAf����&a�u���1��`
|�h-};�
�r��2Y?a�����t��f;XJh����YG���oX�Il�[��'4��-���-�3, �~����Q�	$�Q�SJg$�H���[J�q��C>�@o�g��i��FV��H�F����5������K�w�U������*�e`������^��a���^p'�
����l4�9����z�
]m�#�2����&%	�=�I#~Bp��%j���{Ir8���_���_]����������#P�$C�\����|��?v����q�g_�z�uV/���
�;�ZA�&0P?�f'�����H5�W�V$|]6S~�7P�@~�@p��q����<#��B;bp����77�7]�������	=
	h�|8�����J�����A�9_�G����U���rWKG���KK^{�X��	�N��4]����;�b����z*�\ �ZhxE�\u����.��!���O`��DE]B^���b�M{�	�L%�1�yE���e�G<�������x��9�~��O���^�x�A���xc�3`3��AlE���DtJ�����#OF�_��7��i��y`�;�"��by�e���qg/P����������p���+B��t�D���=����e�>�[����o��rl �7\*8�r� �|����!����`s����$��c2"	t��m�9��[�Q�����L$�<�@��`l�U*"t�f��^+��E�F!<����Ql�a<aI1!@����[��:� ���
���*�!�+[ZR	���4}L\��Z���q4P��s���������@�>�"�squ��nv}���f����+��gh���gHZ�X�}L��;��r�E�C���T�p���Bc!�^����6�aC���9{�8�@<����1�6������^&�F��.�0W��6�]�����E���P4k30��Y�7��X, ����������)��dECBG ��8�����|���K%\2�n�����45�����H�a<Q{���T`;
}n�+x�@��mG�G�Y�2H?(� +J|���BD_��^ ~D����/��v�����r���EU&�\�k�����6����m��|�m�&T�"��8��Y�"�W����v���������v�OK��qHy�D�-9�S�������q�|��v���������};�����]�����a��`�D�d��C�.����RO�Z��B��+(oxV`eG�F"q��N�����YBCD&5�#�\{�-
�d[e�4���`����D�a,�}��"q����)~��#�����=���7�rg������B/A�	7h��o ��ef��R��iMF�QO�����I���6��
D���9�_�TA9�E�Cu���&UI��L:�4w�mD��OP���X��J\�s����$z%_�Q����|��b
�E8=���
�1�q�*��x\���^F��k��
���/��\��xF�����������v����<��K������1�e�?
`��\y���0�6{(T�E�b 4��J�s���-C�3�G��������m2�
E)�U^T�\��S�������0��@�r�{��4}��eGR���>o����R�
%��T-��`����g?��>�8%�\�J8m��_�P�V	��bJ��"`��5������Op�>|x:_qw�_�Q���n��UP�������.�R
��l�����F�(�/T��#�G�1�*Fr�L��M�����]2x�|m)������d���{�����0L��t���/4X>K���$*���6]&g�g	E�C<4j�;�D�1:��������'�~�(^��:�:p�r�����~LO�jPW�A�x+�p�9���wb7T<i�5��|{�{IC�w ��Nna��H=uX��***dT�0�v���f�.�L�P]�E��8*.��8}��F�q�AW;m@m� ���!�:Jo��w)X�d�h�/D{�q���d�'S�%x��(iM��R�"v��%����[	�z$6��\�n���pWL�,]�����h#�����r����1V��<|�	�/A�U��3��
���3I�(s���l�b���dw�n�����$�(��o|�����(��/O�j]���/I�\�7H�_��QhP��f���z���tT
$�Y��)���c���.�M�-�}B��H�����.?
�N��\��3)cIS���_x��<�6�x���y��d���%�5bO�(e��C��I�T�U�
|�f��_���?����#8��>9	��Z�8sYh
����c�H�`JG��W@�pe~��6�m��L�N�����+��H��#6)1���V������e������n.�2������-�X�}�z�n�_$��������:'p8�V]0����D�`��@�NuYX�'�I�/\T�%�S��p�J���% jB�v��� ������7j���7��~JF&����cZ��m7��g,5���Qo���
-��s�_��JB�3�'�f7�5�xM���	i��`�M�����=�g�����yM��R� �@��B��5vl���"�x"7����>�����#���#�w���P�7�A��?��s��;������o�&1�:1��R��"�ck��4�N_ �m����S��b����0u�B$Bb�!��QTa�K�c����G�~w��i@B�
�
�B:�
��"�����]��#9?U�{"������Xk�����Su&��`+�
�ZX|D@���.�Q��+�,8��]���l�T� �?�4��40]}�z�/ol_����L�����&{�E!�6b���������nzi@L�JP)*0;����	�����j+[�J�pU��K������Q�f��t<��&�~1��|R�&T\4�x��|SE���%���������6c���zGG[�N�b�����)OLX��G�-
��>A�uKCSw�F���_7�(}���.�N9�nw�����AW�������3 H�.�k�^�tH�7��I���=^��~�3H?��0�(ra�F.5����X�%g��O;c��}����@�&���]z���&],��;��_��H��K���c�?O�m�\��	�������"i������hArOGa(���D� \�d!Z_������(���	�����&�l�P����i�B����?Bj�
�?x���?1�"����t�*fRp+���2����VX�������:J�I�%d���'���#��"+5��#*5���Z���#���+V^MWl�%d���y�����p�����X7�g�yR!fafRX�3=@U;�Z(��+���*Z�7���8�GUaGa���Y�H���l~��L�Z�]^i�|�.��������i��+A'�Le,������������O�\�kOJ-����"�&)�"�~4�L�<���������L���R��dCH�F�
��DC��O����>�G�C{�?N;���`�#b�C$���U�I��$�b�(���u4�i�%�����������f������kE������l�X�+q*`'x���<�cV�s���]�S���!?=���������Wx����S=3�'����8w�����w��������^\]��C?����a�{�*�=i����pu��=k�������E������e�����%o�7��	���s�4�C�v��������u�D
�`���%����
��L��d�7�^��W�G�^`�2~8"z������3���R�[���Z����.b|\<~m��B���q���n�=�Xpj�m���G�����P��8�i�i�����?�[L��i���k�����0������m~i�A�=g�v��d�����,0��	�
��!g�8�=��f�|mo46�E���H>6;���qG�\���N'4�xO#��	����O�"(�d4.Xn�1P���DQB8�+�.�%�+�s#������i�����1`���*S���J��jr�����B�����"����G;L�4��X��P//U
AjK�Fry��hf]LT�1���<�*'������K+)����*��L'���i�+V�X6����=�i�g9��rp]r�.EO�I������GxK������D�������m+���\�q.�����_.e�1��Ou^l�O���7��~�|�����)F�Cm*������S��K����]]\�1<3�P��}.����e0������`��N��p4�O����a8yc���/Ae�*���RY�/���"�,'a_�l�|	0��VV_��/af�����-��lN�lNO���}�R+��k>i���cNMa��i�1���>s
:b���������I7|�D��j�rO�UAk���6�sh(�FbwS���`���K�Q�VEmE'�)*�����"Y�E�QC�n9�>j�����a<���c�p��<��y���y���R;���k�;�y�'�:D#+��
,J��1��9A\�S��i9V<�n9�<�����ZdU����F��J9�%��\|[E����r3���U"��1��O��c��UO{VQ=����_�S=e�W\�u���s�� w��66�4��"�RJ������<��<�GxT��h":�������uqC��,�.��L����XR�\�.��r�cv�=���Y���I#O����1����.��yF�N�r�^7������ ��Q�D����+�c�������9�v�2t��#W��9��2�]W�49]�����;%��#'�7�_��7s�&�q���I���������YIY�$-���~������B��t/���D]��}��t�Ji���b�����{Q�x�3����!����_� `��BF����-K�S��Ed�V���'��@&*���T�Tu�o�B��t�Q���1��h)*@�TIZ�jZ�����Q�Q>�R�H����	��*)�
UT���J*C�"2}kf�TZ�U���l<g�|����WS��
A]eZ�\�@ *+�REu�iyI�PrxYF9A�4�����J��Du���*��pa��O�=)z�$RF���;S�t�����`��BbS���`�b@I&�L�kO� g4�+��~���p�y��D��p*�z�]�ru~wqs~E����Iz���o�����IRq���e3����z4�.v�zT�X�f�tA"�#��$��PR:�u%jaV�u���JNzR��3��>Sn��&�>kv)�he������ +���8��kRi��u4���������A�x��U]^F�
��ceT���2��$��1��������������#Qe6����y���P�$��U]<�Vif��?CYK�zE�q���z��U@�\����^��sZ���^��(�l���^�<1j���|*WV0�FI�y	��vuR�@�*x"����G��6{�!>������$���4"�
U��u*k���F%�:b#�iq%(A����X��f���rC���@t�<������g��O�4has)�K�JV��T��i�J�p��EDz�����+����6���X��eS�Mv�����j��>O�T<�&��U��V:���9���NE-m�P��^��1]s=�SQO��+�@	jR��a�Z�Pr���6�t�V0�8u��&5�a�_����	*����N�/�R�~���$�XtOV���R���&��RA��.��W����q�u��������
��$�!��N�����/�L����y������Hs~h�(���O�=a����Z��Z��}����M�����`���o� }�`����M��u�4):# UN�T�������'L4�0��*�~���c����xJ��d�a<��R���U�b��g���!g���u��4��s���k��
���;m��?s�@<7D�	��:��?Q�E~�H/qr2��r26X6R6L�1�8��!��������c�a����Z�PK��Yu���PK
��jJ@�Apg_dp_join_patches_v2/UX���X���XPK��jJ'Jq��qE@��Dpg_dp_join_patches_v2/0001-Free-up-memory-consumed-by-the-paths.patchUX���X���XPK��jJE���A@��Mpg_dp_join_patches_v2/0002-Refactor-set_append_rel_pathlist.patchUX���X���XPK��jJ��DA��	7@���"pg_dp_join_patches_v2/0003-Refactor-make_join_rel.patchUX���X���XPK��jJ���Rd3@@��'pg_dp_join_patches_v2/0004-Refactor-adjust_appendrel_attrs.patchUX���X���XPK��jJ�1����8@���5pg_dp_join_patches_v2/0005-Refactor-build_join_rel.patchUX���X���XPK��jJ7[��QB@��>pg_dp_join_patches_v2/0006-Add-function-find_param_path_info.patchUX���X���XPK��jJAje��?;@��^Cpg_dp_join_patches_v2/0007-Canonical-partition-scheme.patchUX���X���XPK��jJX9��EJ��:@���Wpg_dp_join_patches_v2/0008-Partition-wise-join-tests.patchUX���X���XPK��jJa���%VgEC@��_�pg_dp_join_patches_v2/0009-Partition-wise-join-implementation.patchUX���X���XPK��jJ��!�q)-U@���pg_dp_join_patches_v2/0010-Adjust-join-related-to-code-to-accept-child-relation.patchUX���X���XPK��jJL�e��C9@��	pg_dp_join_patches_v2/0011-Parameterized-path-fixes.patchUX���X���XPK��jJ����eKJ@��_pg_dp_join_patches_v2/0012-Use-IS_JOIN_REL-instead-of-RELOPT_JOINREL.patchUX���X���XPK��jJ<�K&�-H@��L#pg_dp_join_patches_v2/0013-Multi-level-partitioned-table-expansion.patchUX���X���XPK��jJ5��+���F@���/pg_dp_join_patches_v2/0014-Multi-level-partition-wise-join-tests.patchUX���X���XPK��jJ��Yu���H@��BCpg_dp_join_patches_v2/0015-Multi-level-partition-wise-join-support.patchUX���X���XPK��_
#40Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#39)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Mar 10, 2017 at 5:43 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

PFA the zip containing all the patches rebased on
56018bf26eec1a0b4bf20303c98065a8eb1b0c5d and contain the patch to free
memory consumed by paths using a separate path context.

Some very high-level thoughts based on a look through these patches:

In 0001, you've removed a comment about how GEQO needs special
handling, but it doesn't look as if you've made any compensating
change elsewhere. That seems unlikely to be correct. If GEQO needs
some paths to survive longer than others, how can it be right for this
code to create them all in the same context? Incidentally,
geqo_eval() seems to be an existing precedent for the idea of throwing
away paths and RelOptInfos, so we might want to use similar code for
partitionwise join.

0002 and 0003 look OK.

Probably 0004 is OK too, although that seems to be adding some
overhead to existing callers for the benefit of new ones. Might be
insignificant, though.

0005 looks OK, except that add_join_rel's definition is missing a
"static" qualifier. That's not just cosmetic; based on previous
expereince, this will break the BF.

0006 seems to be unnecessary; the new function isn't used in later patches.

Haven't looked at 0007 yet.

0008 is, as previously mentioned, more than we probably want to commit.

Haven't looked at 0009 yet.

0010 - 0012 seem to be various fixes which would need to be done
before or along with 0009, rather than afterward, so I am confused
about the ordering of those patches in the patch series.

The commit message for 0013 is a bit unclear about what it's doing,
although I can guess, a bit, based on the commit message for 0007.

--
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

#41Robert Haas
robertmhaas@gmail.com
In reply to: Robert Haas (#40)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Mon, Mar 13, 2017 at 3:24 PM, Robert Haas <robertmhaas@gmail.com> wrote:

Haven't looked at 0007 yet.

+               if (rel->part_scheme)
+               {
+                       int             cnt_parts;
+
+                       for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+                       {
+                               if (rel->part_oids[cnt_parts] ==
childRTE->relid)
+                               {
+                                       Assert(!rel->part_rels[cnt_parts]);
+                                       rel->part_rels[cnt_parts] = childrel;
+                               }
+                       }
+               }

It's not very appealing to use an O(n^2) algorithm here. I wonder if
we could arrange things so that inheritance expansion expands
partitions in the right order, and then we could just match them up
one-to-one. This would probably require an alternate version of
find_all_inheritors() that expand_inherited_rtentry() would call only
for partitioned tables. Failing that, another idea would be to use
qsort() or qsort_arg() to put the partitions in the right order.

+       if (relation->rd_rel->relkind != RELKIND_PARTITIONED_TABLE ||
+               !inhparent ||
+               !(rel->part_scheme = find_partition_scheme(root, relation)))

Maybe just don't call this function in the first place in the
!inhparent case, instead of passing down an argument that must always
be true.

+               /* Match the partition key types. */
+               for (cnt_pks = 0; cnt_pks < partnatts; cnt_pks++)
+               {
+                       /*
+                        * For types, it suffices to match the type
id, mod and collation;
+                        * len, byval and align are depedent on the first two.
+                        */
+                       if (part_key->partopfamily[cnt_pks] !=
part_scheme->partopfamily[cnt_pks] ||
+                               part_key->partopcintype[cnt_pks] !=
part_scheme->partopcintype[cnt_pks] ||
+                               part_key->parttypid[cnt_pks] !=
part_scheme->key_types[cnt_pks] ||
+                               part_key->parttypmod[cnt_pks] !=
part_scheme->key_typmods[cnt_pks] ||
+                               part_key->parttypcoll[cnt_pks] !=
part_scheme->key_collations[cnt_pks])
+                               break;
+               }

I think memcmp() might be better than a for-loop.

Overall this one looks pretty good and straightforward. Of course, I
haven't looked at the main act (0009) yet.

--
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

#42Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Robert Haas (#41)
Re: Partition-wise join for join between (declaratively) partitioned tables

On 2017/03/14 9:17, Robert Haas wrote:

On Mon, Mar 13, 2017 at 3:24 PM, Robert Haas <robertmhaas@gmail.com> wrote:

Haven't looked at 0007 yet.

Overall this one looks pretty good and straightforward.

In the following code of find_partition_scheme():

+	/* Did not find matching partition scheme. Create one. */
+	part_scheme = (PartitionScheme) palloc0(sizeof(PartitionSchemeData));
+
+	/* Copy partition bounds/lists. */
+	part_scheme->nparts = part_desc->nparts;
+	part_scheme->strategy = part_key->strategy;
+	part_scheme->boundinfo = part_desc->boundinfo;
+
+	/* Store partition key information. */
+	part_scheme->partnatts = part_key->partnatts;
+
+	part_scheme->partopfamily = (Oid *) palloc(sizeof(Oid) * partnatts);
+	memcpy(part_scheme->partopfamily, part_key->partopfamily,
+		   sizeof(Oid) * partnatts);
+
+	part_scheme->partopcintype = (Oid *) palloc(sizeof(Oid) * partnatts);
+	memcpy(part_scheme->partopcintype, part_key->partopcintype,
+		   sizeof(Oid) * partnatts);
+
+	part_scheme->key_types = (Oid *) palloc(sizeof(Oid) * partnatts);
+	memcpy(part_scheme->key_types, part_key->parttypid,
+		   sizeof(Oid) * partnatts);
+
+	part_scheme->key_typmods = (int32 *) palloc(sizeof(int32) * partnatts);
+	memcpy(part_scheme->key_typmods, part_key->parttypmod,
+		   sizeof(int32) * partnatts);
+
+	part_scheme->key_collations = (Oid *) palloc(sizeof(Oid) * partnatts);
+	memcpy(part_scheme->key_collations, part_key->parttypcoll,
+		   sizeof(Oid) * partnatts);

Couldn't we avoid the memcpy() on individual members of part_key? After
all, RelationData.rd_partkey is guarded just like rd_partdesc by
relcache.c in face of invalidations (see keep_partkey logic in
RelationClearRelation).

Thanks,
Amit

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

#43Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#40)
Re: Partition-wise join for join between (declaratively) partitioned tables

Thanks for the review.

Some very high-level thoughts based on a look through these patches:

In 0001, you've removed a comment about how GEQO needs special
handling, but it doesn't look as if you've made any compensating
change elsewhere. That seems unlikely to be correct. If GEQO needs
some paths to survive longer than others, how can it be right for this
code to create them all in the same context?

Thanks for pointing that out. I have replaced the code and the
comments back. There was another issue that the temporary paths
created by geqo will not be freed when geqo moves one genetic string
to next genetic string (or what it calls as tour: an list of relation
to be joined in a given order). To fix this, we need to set the path
context to the temporary context of geqo inside geqo_eval() before
calling gimme_tree() and reset it later. That way the temporary paths
are also created in the temporary memory context of geqo. Fixed in the
patch.

Incidentally,
geqo_eval() seems to be an existing precedent for the idea of throwing
away paths and RelOptInfos, so we might want to use similar code for
partitionwise join.

There are some differences in what geqo does and what partition-wise
needs to do. geqo tries many joining orders each one in a separate
temporary context. The way geqo slices the work, every slice produces
a full plan. For partition-wise join I do not see a way to slice the
work such that the whole path and corresponding RelOptInfos come from
the same slice. So, we can't use the same method as GEQO.

It's worth noticing that paths are created twice for the cheapest
joining order that it finds: once in the trial phase and second time
when the final plan is created. The second time, the paths,
RelOptInfos, expressions used by the final plan are in the context as
the plan.

0002 and 0003 look OK.

Probably 0004 is OK too, although that seems to be adding some
overhead to existing callers for the benefit of new ones. Might be
insignificant, though.

Yes, the overhead is to add and extract the appinfo from a list when
there is only one appinfo. We may optimize it by passing appinfo
directly when there's only one and pass list when there are more, but
that is complicating the code unnecessarily. The overhead seems to be
worth the cost to keep the code simpler.

0005 looks OK, except that add_join_rel's definition is missing a
"static" qualifier. That's not just cosmetic; based on previous
expereince, this will break the BF.

Thanks for pointing it out. Done.

0006 seems to be unnecessary; the new function isn't used in later patches.

It's required by 0011 - reparameterize_path_by_child(). BTW, I need to
know whether reparameterize_path_by_child() looks good, so that I can
complete it by adding support for all kinds of path in that function.

Haven't looked at 0007 yet.

0008 is, as previously mentioned, more than we probably want to commit.

I agree, and I will work on that.

Haven't looked at 0009 yet.

0010 - 0012 seem to be various fixes which would need to be done
before or along with 0009, rather than afterward, so I am confused
about the ordering of those patches in the patch series.

They are needed only when we have 0009. But when those are clubbed
with 0009, it makes 0009 review difficult as the code for those fixes
mixes with the code for partition-wise support. So, I have separated
those out into patches categorized by functionality. Reviewer may then
apply 0009 and see what failures each of the changes in 0010-0012
fixes, if required. They need to be committed along-with 0009.

The commit message for 0013 is a bit unclear about what it's doing,
although I can guess, a bit, based on the commit message for 0007.

This is preparatory patch for 0015 which supports partition-wise join
for multi-level partitioned tables. We have discussed about
partition-wise join support for multi-level partitioned tables in [1]/messages/by-id/CAFjFpRceMmx26653XFAYvc5KVQcrzcKScVFqZdbXV=kB8Akkqg@mail.gmail.com -- Best Wishes, Ashutosh Bapat EnterpriseDB Corporation The Postgres Database Company.
We may decide to postpone patches 0013-0015 to v11, if this gets too
much for v10.

[1]: /messages/by-id/CAFjFpRceMmx26653XFAYvc5KVQcrzcKScVFqZdbXV=kB8Akkqg@mail.gmail.com -- Best Wishes, Ashutosh Bapat EnterpriseDB Corporation The Postgres Database Company
--
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

#44Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#41)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Mar 14, 2017 at 5:47 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Mon, Mar 13, 2017 at 3:24 PM, Robert Haas <robertmhaas@gmail.com> wrote:

Haven't looked at 0007 yet.

+               if (rel->part_scheme)
+               {
+                       int             cnt_parts;
+
+                       for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++)
+                       {
+                               if (rel->part_oids[cnt_parts] ==
childRTE->relid)
+                               {
+                                       Assert(!rel->part_rels[cnt_parts]);
+                                       rel->part_rels[cnt_parts] = childrel;
+                               }
+                       }
+               }

It's not very appealing to use an O(n^2) algorithm here. I wonder if
we could arrange things so that inheritance expansion expands
partitions in the right order, and then we could just match them up
one-to-one. This would probably require an alternate version of
find_all_inheritors() that expand_inherited_rtentry() would call only
for partitioned tables.

That seems a much better solution, but
1. Right now when we expand a multi-level partitioned table, we
include indirect partitions as direct children in inheritance
hierachy. part_rels array OTOH should correspond to the partitioning
scheme and should hold RelOptInfos of direct partitions. 0013 patch
fixes that to include only direct partitions as direct children
preserving partitioning hierarchy in the inheritance hierarchy. That
patch right now uses find_inheritance_children() to get Oids of direct
partitions, but instead it could return rd_partdesc->oids in the form
of list; OIDs ordered same as the array. Once we do that, we should
expect the appinfos to appear in the same order as the
rd_partdesc->oids and so RelOptInfo::part_oids. We just need to make
sure that the order is preserved and assign part_rels as they appear
in that loop.

One would argue that we preserve the OIDs only for single-level
partitioned tables, but in expand_inheritance_rtentry(), if we want to
detect whether a relation is single-level partitioned or multi-level,
we need to look up its direct partitions to see if they are further
partitioned. That will look a bit ugly and will not be necessary once
we have 0013. In case we decide to defer multi-level partitioned table
changes to v11 and based on the progress in [1]/messages/by-id/2b0d42f2-3a53-763b-c9c2-47139e4b1c2e@lab.ntt.co.jp -- Best Wishes, Ashutosh Bapat EnterpriseDB Corporation The Postgres Database Company, I will work on fixing
the order in which appinfos are created for single-level partitioned
tables.

Failing that, another idea would be to use
qsort() or qsort_arg() to put the partitions in the right order.

I didn't get this. I could not find documentation for qsort_arg(). Can
you please elaborate? I guess, if we fix expand_inheritance_rtentry()
we don't need this. It looks like we will change
expand_inheritance_rtentry() anyway.

+       if (relation->rd_rel->relkind != RELKIND_PARTITIONED_TABLE ||
+               !inhparent ||
+               !(rel->part_scheme = find_partition_scheme(root, relation)))

Maybe just don't call this function in the first place in the
!inhparent case, instead of passing down an argument that must always
be true.

The function serves a single place to re/set partitioning information.
It would set the partitioning information if the above three
conditions are met. Otherwise it would nullify that information. If we
decide not to call this function when !inhparent, we will need to
nullify the partitioning information outside of this function as well
as inside this function, duplicating the code.

+               /* Match the partition key types. */
+               for (cnt_pks = 0; cnt_pks < partnatts; cnt_pks++)
+               {
+                       /*
+                        * For types, it suffices to match the type
id, mod and collation;
+                        * len, byval and align are depedent on the first two.
+                        */
+                       if (part_key->partopfamily[cnt_pks] !=
part_scheme->partopfamily[cnt_pks] ||
+                               part_key->partopcintype[cnt_pks] !=
part_scheme->partopcintype[cnt_pks] ||
+                               part_key->parttypid[cnt_pks] !=
part_scheme->key_types[cnt_pks] ||
+                               part_key->parttypmod[cnt_pks] !=
part_scheme->key_typmods[cnt_pks] ||
+                               part_key->parttypcoll[cnt_pks] !=
part_scheme->key_collations[cnt_pks])
+                               break;
+               }

I think memcmp() might be better than a for-loop.

Done.

PFA patches.

[1]: /messages/by-id/2b0d42f2-3a53-763b-c9c2-47139e4b1c2e@lab.ntt.co.jp -- Best Wishes, Ashutosh Bapat EnterpriseDB Corporation The Postgres Database Company
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company

Attachments:

pg_dp_join_patches_v3.zipapplication/zip; name=pg_dp_join_patches_v3.zipDownload
PK
�nJpatches/UX���X���X�PK�nJ7patches/0001-Free-up-memory-consumed-by-the-paths.patchUX���X���X��]{s����[���U�(Q�7���,{��,G�o��J�@`Hb4ZV6���3��j�M�.�]Q�3���������R��}���`��;F����8��v���}k2����|q'W��N��'����xb^��h���h!��VV,�`���S�����e�
�H:��,_7�X�|!��nKqm� 
d����^Ww�N�n=�I����g��?��qj�lR��J,�2����z)1}�B
x�"j77��]����C	��������x\��z������>�e��7��$��4b|�\�>��cxN�s��->�8���Z������'�:�Y�
kl���&9m�6\Z�'n/>��^8���pc�a�����P"C��\�Nr�A���t�;"���_�����x���z�8�a�U(=���c��� \Z� f�x��PF��w#�����B���X�R:���V�~����~p`x~� Z�>��������(�v��8����AS<Jay^`������
��Db�%��D2�����{Rm3�L�z�l���M��R#�'�h�i�mqO��3�:v:7R�oT�~�����-��.^���#��O	�1
���nPM��-�A��h�lXS���@�S���"���*I�@i����~���p��ev�l>�`������|L�P�d(A�����k�qL�"�&r������Kj�T���&�8��I|"R@��-j��R=�����)��7�Ph�]��F�Q�����@/�� �rz"I/\�����=�0�
&G�i�UM��>+�j�o���A������899i�(�O���9
��:��O�0azzm[�C���8>>.cA��0�,����#��I���������'�����<iDO�~������G����X���X�����_�>0{5'�jx4n_��I��c��y���)t��{!��,C z�"�`PN�x��SR��3��`��?�8���a��7[b8��t���h�����9�����Mk6\��_�t`[���n��`�b����f�~fP���?�Q��-��9����v����QqK������`�?�����5���8�&�q����I���[�����q�zd}��3�{+=>�j���� �fn��(�������I/^R�&����h���<G��rF.+�4>�.XQ���B`�0���Z2��� �1_�,����S��1���HL���L�#���Oks0`]Z���0'�)W��6G�:�qs�>ZI��A����$�R`���1�&�C77MG�M�27+4�k�	����C��A����Y���'TA����Mw>���gB�%���M�p���A��4��uC@�sC@7�6�8�����W0JQl�_��'������K�@��J�H�*j��oh1(;y��X��a[O�>�W����WW�X�,x�`Vs���il��_7����kl ��Q��i��_+�V�v
%�V��"�� ���Bm���bu�}���=���sw��&-;h�3�����}p���Z����2[���&�;���v{8t&kZ���%UY�<-�F�;�8����&�CE����@G#S���[���Wp>/q��,��
��C#�H����-W0)��+�����I��&��yI�TI���H�Zv4QaJ�S�J���9�0�I��v�*���h�E`�CC����'��k%n���O�?]<��L��	�9�a�pWj���SP_tM��+����yw�������������D}s�N����6�����T��h	&��d�� "���e�A���&w0%�_�?��z��T����:��1��x4�N�~�fK,�+�{_}*���
�yo�D����+`�����,\E�1X��f��F���������E��-&ZO�����x��d����+)\S��Aha��V|��eJI���V�-J���!e����u�����^%c�!X��GuRCK����
 ����_H�7l����%F�R����*��ZB��#%��O0>���g7r)�[H�D�K������:�B��-��+\��,\��V��L�����9.�qc�q�K3ypO�9Lz.\��F���D����x����p����jh�GdB\�:M�&k`�fJ�=!�Z4�[��d�����0���?��9\a0 ��\GR��ER���e'��k?�&�������D�w��0��t�X
z<V�~���9V:�{�?��,���������Yk�(��w(��K�u�
��7P����7�RR�h�KlIx���`��&��C�����w���f�5��X!���n�Vo��4!v�a��e�EZ������e�F��1������5�e����K=K�>��U�Q�$����!���0q�9�6ys��`N��9�
��0;.��G����N���:����8.!�=�*���V���]��KZ�c|��,;��5]�+Wh;�F ";� �#P������B�o$s�N��8M�+������D�3�A������3����y1i�	.�|_Hk����x�R��:�G��A���|���������&�9��	�q7,�.�F��Sgp�1 ���>�c�����?�l�v�u����t��1�S�UG�:���nw�1
:���)I;��)ZNe��4@�	�`�s�Gd�������E�^��j
���Zo�8���7�@�����7_�}��0,�Y��u|X��s*~Z�&�kxXn����p�DI��dkC]���)@�{�A����g-A�eBs/
R;����u��q����F���K���Z�hQ���O����6y���������+DXMG&
�
UJqO��C?c�4�f�Y�?����9)
�V<C�%�b������w�PI}\A��R���L��6g�]e��i��C��S��0����'_@9�>�� ��S�5Zjl]��F��A������]��
��g����)k�3��J�������$4��)L����L�H���8�J�����c:���.����	�����>|�(�{P�9�d�`I�k�bZ�z�q���*��u����=f������U�P����H�����yu��^7.}3�O���������"5A��5�����9�olT����>��=��iXw�Z�4�
��a�#g�����b;���P��q���������N�0`7��<|T�����?z��s�����B�w��I����Qn�x��������vt�N�y�8�E����$�����JTc*���	gw�O=�}��/� Re�v�`�L%lS;���
J:k����w2V7k�A����YQ��_��
�%���,�1���9���A-@��?;N�H���)G���_�v!4�A����
t<*����sT��V�>5<�e|��o�e3j%+�E��Y��NUt���3i�{���#�V�"+�2S�%��;*�Q;8
��*d�b�%�K��n_�����Q�M/z�l���.��~)���j���v����V��)�c��&��gL�b�>��+����a�x`� &�Ck�k~�X�`�;t)�����YQ��K�.���^Z~�J��y���#Q���6�dr��`K����
T]��9����u�����l�	!�������]��m����������|s������yw����\Bw�"*��]_7�}$f��Mb��������+�q{������%
Q���a���}�1���`�F�26�Qx%H\���-z�E*}������;�� t���2����<B���Y������77/�*���{JU�����m����3����-�����Q?���0�B�B���c�6����ejc�g������-����E����,��j��B���{o!M���@d�>�Q{���1Q^f����9�,��s
:b�������l��8�*��vh�XI;��L�|��eHEj�r2�r�B��7���d��Z+�i
,R�eh�D5qk���u��~i	���
o���=ym�%L�������7������=�/����f�d�5���%E��p�R��47���W�2�������5z��35����;;�Z�6/�����i�Hh�k�t��ID�5�nV�P
+G��%
{c�����%���
Z�9Yv]n|RVL������uc���
����U�R����>�����P�d��5>�D�9���������	�T�aV�%������jQz]8�M��h�
?��3 �������c@eS/G�f<�����n[C��N���T5���t��G�+~t�%W�c���h�~�,�Tq��1����G_��)�9�AT�"������:�Z~���qV=���-�w'';�;+
�����������$/u��G�_b�����������V*���8}"rs�%wv3w\�k�r'fU��HKr�2s�U�2]�8�-zOX�s$j2{��g��uf�����E+��G$;$�f��$�L4[��w������_�`�c�[�C1fL`��
�t������"FNG:
>����P�I���H'�6�n���_d%
ZyvX(rD����h��l
��������%�d�:4�\���,��F���L$�*����O�zh��D��������������X��D[�~����x�,�I28�y���R%�eT9�S�B9$L�b�������e���B���
��d���-�G%+���K�(��_�5��F��z�No���\oY�Y�����o��`>n�'w�#�/�(��.�bs����a������8��hk�c������ LR���5;�"���`�&�JS7^Z+���X�aup��u���aWh}O����E�ei�l9��l+V;�]	C��������!�n!��vK���)�ep���X�f�3�GU��HR��1������k���C�>b��-w�V
Z5f7�.��"�X	K!`�zx��5�u��������1����:�l)3Q�@���e�^�'�*hs�%\�0�����3�&�9������j�Gu
���$'�K�\a�������mc�����ly����`�\����@q��8.�U0�����q�m�f(z���U�W��E���'�� ��T������u ��V3��j4�p����1TK�wb������S�`�������5y���1����	�_�cVAa�^�R?/I��Y��:<7R�4ni���]*�_��A1���9��� QW���U��^gr��� ���a��_�z/����`��VMR��7b��]���:���-����47�%�U�����g�T��)
V�*���g�q��B����%^��%����]p���Y�.�}����f;�F�����g��7���y�>R�6U��!���L\O{G]^��*8~ga�97��t�W XnJ�j�r��0J��-+4�O�V����0��;����D�F���h�a���a,�r�sZ��5E8n;�J'���"��T��N��Bv�j����N�)8c\�����{Njj?d���ab���X������9u
Q�lyko��QY8������ Qy��d����"���M�����u�	����Q�R�p���<k��;A�#t��R�I���d��(�at��3D���yd2��m�[��gx��-?�����*�fY�]V]Q�6��N�"�ng���|���,�O����[�J�������	�
�P)6�25��p��-�g�?��6o>�_�������#��10F�����tfa2
z.��d�Q7���fzm�]._�&�}9u>�O�e����e��8�]}YN�j��E��#�n��&0��mu����h�CY|V�\��+}�p� ���b�2<���`�FY7�
|����*[^�����o��[�,
�j�������nSw���u�I�������n���t�W&����:`���4J�&
f[1���v��8����V���k�nLN�-�4����+4���@w��f��-T��U3R��5O9q���U��s�e�E���DL��wI���J�,���
����+�r�&i�]�mMT�<��Fv2�W+,c�L�lf�za�y����%�|D�I���A��1�����������;f�����-�������\��#ql�G���@\j����}�.(���p������3"�s����S���`���JU�����\�������5-O���*������=�[uNU�����`��RB]��P��������>�S���K����(������S�6fl��P���)/�g|�����I��j����J�|��cO6X�����vWW7s:�d��G�w��:��H��b��Q�{%�#��(�3c%8��)N�}��������Nr�(L�]��m����{�n�3mW���}����B�og������u���e�	%��r?�T=ze����1��S��%6�V���W��tj�n'{�BA���4SM@t�\�U`?�~p+7�..c|�w�4�������
"����O4
�l�`NS��,9hs��@����4(\;��\�"p����yO������?�\;���
a����L�[�kj��*�F��������Z&��<V���^������M�5��9��'���#��l��[��y��P�E@d����mx����VPd���������y�L��TK����[�EJ��eZy[��8�Yg�n����G�GCJ$'BJI��������#g�[����y�{M��'���������\������n�k@���B|������V��<!L��7�n���.����
�����z���5�$�~��"����taS~���%�8QG$�g&c{�����R+������&�H&+H-U��o���$Q����Q��n[����p*�l�C!�>��*�O�@��d��l��u��������b|fe�zH�>�$3��q��.�zF��-�|+�x�T���Ai��]�P=:[�d�)}��������_=1`����Q��!�R���(-��l	�s����!�Nw��������P�+�� �
\��h�����+�R�NW��*"�k���9�X�n4s<�
��Q}��0����}z�`_��T�?�{suuy�������)�Vh/�E<�������R�����,�e�{�XmQ�:���^�Z�}��fVq�{�jA6��Hu�H;
$"�OM�6�0���+".�;
�=jO��F�PK�1v��vPK�nJ3patches/0002-Refactor-set_append_rel_pathlist.patchUX���X���X��W�r�6}�b�'����d�v�L\7n<�[������$(2��:i�����Rw�#����`��J��Qf�i|���p4I�)��,�M��$;K&)�NCx#*��k�������0r���9\���B�����Y�����W��Z��q���s�g��9\���\����h|>>9M�
OF����?�D����/n/_Ax<�N����D	���k^�3��z��B��q���H������p����E��e��TY���y�t����]8O�����n�y��� ���L�����
rv��?�+�F�}�s�-uA��{
>�����s��m\v���i��\��c��{^���#�������.|�	gi�e\�\)o�����[7�����sP9�<��)K�I���0R��U�9"�C��)4E
�����F���U3-��OYQRX���u+��`�Q>9�TT����u|�w@�d���Z���Cr3�}�]��_p�{����@��������q��(.���;� 
!�%���q{��9F����'~�:���d���,����8>���xL<���9>����G�g�\�s
8�4����(RPM����aV7*�}nX����������
�'�T|m�[��#��w��f-�ns�:���|Yi����� �����A�O��-��5��l�ht�h�,RT����U&�'s���C���dU�e�O�c�s�^��FF�������/N?�#ztz�Eg{t:������"�����i���V�3B}�����p����O��������({�n|v�J��h�����)Y9;da{��%�Q�����j�7qb��&{�k�5��x����K��I����,�"^#o�+V�&B��J�sVs<f���-*\�JxP�xEf�t�E_��K�N5����a�S(��,���sS ���6��+J#kuJg�������
+m�����!~�~
�_z@�?_+<*�����Cx�M�P�&���N{m����"��mQ@;������*�*a�����45l���Wi��n�U?*��P�+���d���qQ���������`��-F����Q����$<�K���jX����(=�b[}l���@&D]-�}`
��v�����7'��T3��v\�D8/����0|���_���x�\����;��6yJv��5�����o��z�eC�P���jO�d������D�%^pU����I=��c��(Y��D��o>�"��#ERM�����cg�37${��T���.@\I���}q04���^c
�����
%�xI�: Z��EW��rho������|�����dw��:�W�?(l�����was'l�c�m��Mas�������Uum���_����V��!��/j�Z���Ut[Z��qF+O��%��C��	�kY!�f-���b���ArFM
M���s���nk�3,l����)��;Q0
����PKC>"���PK�nJ)patches/0003-Refactor-make_join_rel.patchUX���X���X��U�n�6}6�b�@��X��n��vw��h��$oEaP��b"��H%u��{����v���/u�g���^Tz�$K���,J����q��x���t�!��dg�E+��Fs��U�q���Y���V���%����a����,Ve%
�$L��=��-����>��
F�G��d�/�����]��
�v�~�p�������7����VW������R�+,�g!c�9B��T�V��@xr��d�VRq����w��EV�3�<�����\A���T�IHi.q���\�9����$���*�j�:/^H���VdOv'�9�yE��aiXW%%-uYS���|������;vM�a���h��6����,����|��By�t �{jaS]�� ``�t������������=���0�� ^���'hB� �u/�j���xB������X`��2� 6��
I�6?&���a�MF�q���M�e�(�M&���fdT�����C0��x��ZKW�`k]�4kQo���Q�X|)��*�0�������NKA���}����Ja��km}x9@��Bc+���5%N�s��]g�0��Y�<�������*v��69�����x��vk���g^����v����JL%/~�����H����&��������k
���]sG�-��w�3G[���tP)i���z��><W7���`H~�Q�i���<2]�{fzG�{dI�f�U���������6Q�����:;����Z����9�b8�b�l�E;njY�;)H�Z.<��}"��V��W����NGH���i�I���P�����,d���C�]<�����f�y���`��tC����N�R���kfm\����qm�����@���#P�B�OX�:g��H{���D��R�D�u�-���� ��M�����H����fR��?��[,�������Bo��//�\���&S�Q���	�
��|���|��X��Y��I
�I*����,xa���DI�����y����
PKC�r��	PK�nJ2patches/0004-Refactor-adjust_appendrel_attrs.patchUX���X���X��Zms���,�
��;��dQ�,+n2���^:yu����$!�6E�H������Y|��i_��'�(p�X�.v��:��ltr4�N�#o.������w�9���O���b<��wq�>�s&l0x����`�X���{�.�2N�_q������K���*	R��}/^��~�R<c���fG��p��������x��������v��'���|yy�+���?���sO�	���:�3�Z��OD8�R&i����hwX��5�`2f2�QB��H���O ,C.�Y��\��0��E��[����{At��Bd�����y��ap#�o<����@.�'�HWq�S_�"�F�����.����1��:�ht�.Nn7�R�	3zZ��>{
��,�����uD�G�Uk�WW��Rs;�J��
Lmv'0�?�A"v�m�e�
��1|�g�$4��[�e�c~���@���feel7g#�z=���w�r�\����?Er���<�;�{���?����.4�[/�)uR]����<R�H� 
��[��:����8��n��e�_y��5~�+��D�yB���GW���3tX�"!���n�f�#��P��^��`�9�����a3����� ��W6�s���~�����1��`p<�9��hA5�G��g�;���c���	CC*2���������
�&��� �ci#��V�����X��6H��Ch�@{�-O���+��[������M-�
9�0���3��-��p��M,���+����8�e��yI�)=e�Uc���"I��A0g��-;�
|��s��..��:JkS���N����Z�eJn�~��l'$��2	<E��(5����X���!��8������A�%zrX�������
�n}���Eq��t
q�W,�4U�EO7'4a&o!�����x�,�J���M�lQ����4�7����;���hl;�x8��?g5
~�^P�yc�m�GhP	�L�������]�[5^����+!{/H��w���)b�uH����.�����H�`���2,��sJ)<�'�(�0�K�8�V���W�4��
�������	g4li+���j����88�7�����v���B%���&��d(2�9}�7u��x�d&�0�R1���]��[�q��'(_�$P��d@�T�PO���1�`��IF�<�C#�"��f���9'���~��{��Y��h����*N9�]��� B�	�<13��WVM�.=��
�Z����K^�i���X8_�o����O�$z�K����@��+%G��3����
�K�a!��E#s���N����G�������q2������j����d�1���XC+�������!��=i�A�6?�a^*c"C�f�|2N���FBQ�v J�R|��f����G�%Jx0T*f�+��H�F��C��ae�-�G���=��K_�n5�W�\�����
�����m�VI�Ld@�z����dywF(�<�����1��Dqb�@�W����{5T�0Z��v�C�����m�u$���_�J��������]�^���3�s���LW����������h�.@����-���;qNlg��4>���!t���;�������
��G��C�5@�IR!^���?`Rs��^N��BSf�����GtP<������TJ�}�&����W>
����K5:r������E�
�t�z���)���lG��B@P&B������m�hfUm���0��*��
�DR�QZp+���3��n;�(����rx��}�/��{b��\Bhx�-j<b�	"��At���)�����U$�Ib����!,��#+�	�P]{d��Y7G�9�u�����
Y�{W��t�C��=�2����wp](.�M�QD)��|Vo �R�Be=�d���>�]qt����Ta��L��a9O`����!�u��0��g�l��������i���D�.!�3�-�����t����LU���.-M���Z��EJ^��r�G��SE�}�v�OS�����U���J.��{��9x)���hx�
!��rY��Df{������K��Eh1e�;���u��=<���t��y��*yM����{r����\��(��5�0���3�Ur���@Yu?|���o��'&?����t����_��8�]��>y��������!�.����JT{��\u��C�F�$�Y;�lVZ@-h� �N����|%m���2/(�F*��6����)e9a_>������������K�K�X����_f���k��P�%����;`��.���bI�� ����e�'T�v[OPm��T���M��K���-J��h�����t��7����7�_Z/T�f�d�\�"�&�L��x2���J��������qvR��!����V��jY�-��'�<���y���m)�I�~��"<��?��P����[m�~P���Sgo���	m�l>��1~�>����\��o:[����~�^����o��'�)9t�9���hw����J�g�����;/����o�2���������Ok���C6���������(�X�Z���Xe���$~Y������]�����^j���,���FA�]�<`�*���$
%,Y���

v���7Tz���-�\��8{� ]���'�ZhE����qR����aTyM���@�"ZWfWR�l-+�����3��s@�L`�L��U��E�!�%l��B��x��y�������=�m��)�Z3�z��v?C-��_QT�D':��#��9���G���������������jqU-*	K��Q�;�7��J�w���	v�o��#F��OZu3�Of(�����(��,����.A_
�����]?@i�����=$�9=������&�P�d@zc@��Qzv��E0����{��E*T�&)���x.����7��F[���D����:^K�4�LX�`#5y�u"J7���Q��A�C1�Md.��Di"ni����F~\"��9�Vq+G���}=}&�����
��)MD�:M[�{�y�5e�trl;�R��pd������������vS�5�V4�U�bNv�E�\�0Q�
|�����R�{_�os�@(��>gH*�����c�#�����a�:S`:����:P�.��/TF��C[_����:"n������	����	a����#B�ku�x���:0��t��b���]��t:�v/Z��D�3���}�quX[�K"i�G�:Q���7WZ)�i_� 5_�$,���t����P7 �M�cHQ�����L4�O6a��BM�lj��3�zY��WE"f7HJ�����)����]�wtc�k�����[�\0�K
c��}D�m7���qS��W*z�v{L?��[�j��F���0��-�$��u��i�@���Z�T���4
������0&$��O��T��k(�����::�[Z��=�[Z����16��B���d[�,��YhV]��W�U!����>^�^��|�G������t�KqZ|�^��f��>�>
�l�6���6����;;/c�{_g�����=:���`(�|���N���$�Z����s��<$m
\���R�Q��y9�7}U�'��P��7�)����w�w��!��ePi��[Q@Q;�m*��<<�neU�Q�br�YN������PK����Qd3PK�nJ*patches/0005-Refactor-build_join_rel.patchUX���X���X��Xkw�6�,�����d�z���MN�l���M�$��={t ��P�
�V�M�{��D=l��s��T`�<0���K�g�����ODop���p�>w��������v���F9�_tB���:'�n��_����%`��"���N�������-���1&�L��Q���@��z��<c��K9�;�����?�z{�k{���Gdg�������������H�64�U�>j�������V�LeJ'���gR
5%:���e��Q0�����`������TFi�������J�LeF�6RM�B47z.a������xaHJH%~�$��2�t��,�C����x�u�B[?Zk�#"	��L�7����2�����X%���R��}�����>�$l�y�f����r��h[�}!���T���o�x���H$6h��/)[��V��
N)��t~��BE���]��g|�2O%���HD��i��<������k������<�y��g����6����)�k�a��Y���x�������� �1]}'�7������
���E�b���q�#���M�b�\Q;����+�B�%�*�/��<�s��ol+><��u#�Kg�����&H���ZGv/z�!oF��i����.L�|^�Q��k�V��[�����{�|����DJ��SY�����K�0r)�p21�QO�(y���p�c�L��|gSI���!�U)��8��y�:hx4���� �iJ0{� �C�&��^&t[RYF$$?���Ld��t�U�(��$�����Z��J���V��MSTB]HQU�����D�1hv�3��x+nE��y��Y�������%,����"vF� ��V��(�0�A�<��z���2v�T
��k��r���&�9T���[k����.#��;`�b&>! 9��Xb�������jk��M�v9��m����
C�����YdgM,#v�Z��T��2{�2�'`��Fq3��8v�R�����i���q��~��cC]X��'���T���2�������.�UX���l%���'Oh���iE^+�jS�
����~���Sr��x���w�:��H�'�|��W�[�Q�0�R8���q+��2Frqt�b��iyl���Ed�o�rVk�d�2�����n��c�7��K,v4}�X��^�������8�D]�d�����w�M����t�g�_��c�����?�X	s��aj{$oQf��-����t}�pu�uD?�����&���c���X�LT�,��)�\d
��*���K���AU��6h�h���w ��w�CC��s.oia�UU�v�
/�hGS��6�Z��J��$t�T�$��nI���T
L��[5�ex\]�d��v���x�j�����v��U5k�E��������sm,.��o�5!�7��<T�&��&I�9�<G����N�#�^�;���y�.�*0X.l�l!b{�N"����)^f��A�;h�C��[�����_�u���������S���b����9?�H��1���px 0@j-�C�Zv��;���J{3�u8���7�Y��M�v��:�oc����a}�-QZFqPAj�c�w�Z^���@f���Y�������_q2��/����,|f���A�� �{1K�@f���������~f��,*{��w��WGk����������y%���~�����k��*Zz�T~�Dh�����<��,��l_���'���9��9��W����&�u%��|����v=�u���I��v]Q�������TD���i�5��*��L���B/1)��J����k�
m�K����}Y%K���\����a�C���PO�s���?����vp�^�������%�E1���[����!��&�[K��x[z�~�]B��N�W����������q�r���vu��.����1S��S:1b6����g��s�^��r��}�:���is�yPK@+����PK�nJ4patches/0006-Add-function-find_param_path_info.patchUX���X���X��W�r�F}�~�F�ER���3v�z�7�����h����E2\R����,%Y7O�N:
mS�����*���D&}>����'B���/<9�a8�#���<���@��3�����
������U���3/x?����h|!�J�E��������iu`W2Be��������
�{[Gw2����������7���?�RH�,��JT&f/���|��$w{7��BB������0�MS�s�[ �K��TA�@)?������0��\����K�*m����4��V-e��A�E�btP�<������
H�H����Rdq�B�s�7�UV���s(�31OSP$��l�n��@�q/����^^Tj�>��WW*��x�����sp���<|������V
���46-���?!S�Lt��	�$���"v�;	Ot�6����`���x��h�P��y�0$�O����Sm^\@w8,|���B�pC���:�j���]�2�}��72���;�E�����D��T��lfR[�V�g�[���m/B���3����n��k��,�Y����F���jW��4���'4���X�(}�5�vk��J�M=�|�H?������c��������q��A	|���E�Ro�����KY���V������k��z�N�^OK����0�^�>c����(�>�N��]�)eUc��:=�mQ�y<�������PD�6�?�_�p�^���&�Dn�1��m*��A�<�P�h��~l��CR�"�JZ�`~H��=��u��)����h�0d��<u2�R��ZK���#����G���&�d�:����\��`�5���'F���1b�����*U\������0����U��8���c���'�:��~��h��PBV/"<��|���* x�Ffy+n8I�(R�2��
v�l��qd5��d������������^ �Q����Cbm�h��y��1xD�{�W����������&6��]���n���P)�4+�������6^���x���S�|#/�������X������k(�.i��L������k���:��5�L�>�� ����*���`6�6o���,\�������JT�v�8��oM��*5�Is���������@���=~WqI����.`�t�y�y���_���Cw�i�Hf�[�v�U������W����z����_�R
�wG��0�7PKk��!�QPK�nJ-patches/0007-Canonical-partition-scheme.patchUX���X���X��;ks�8���_���j%����8���\\��)�{�a.��H���"�i����_w�A�%�f�3�,�n4����4Y3�<���x�����?
|z*��q N�t|t|��G��}Lb�El�������6�'�w�;��<K����7<c���>\��_E��t��R�����t��L�b����1{'@O���_����p6�~��~��������=�&�_���8�y�6<��,���k1�v�%)�L��]$X$�3K,��p�8N2 ����z�]���%KX����\�
���Q�������l%��k��3�2���O������<d��iY�t��&������$p~�HC�0���4nv:d��D
�4�4��l�Z��4�\Y^�I�� ��������A���l��8���,�o9#�g���i�q�{������$0A�"�����7dt:v����Q	G�o6����U��
��d�F�*�l����!�(�����L��% 2������iU�(�Qw�e��Bs��^�L���������.��?Zp�^��(�d�:�?��`�j��?��g?c'l����Y�6�`.�N�vY���g���I ��0d�b��8�8d��2���+E�����Y)R:������r�������y�]<o^Z|�<>+8/�&j|tp��}.�.���X����L�#6��c���(�q0��%,�}���EJ
��&I��*���.�I"����x���,K�<��0�:�NL����Pf��Gg����B�`�h�a{�*��@B{&}���!d&$lDQ�%���*UW�[E�}�h�G����mT��������7�(����X}�	���X�S�b{}��(J�qO���]%K=��GF�d��7M.��%���>"��+�nel��F
�Uf��|�Ik=�y0Q�d-D�l�e���d=�9B�+�����W���3c��9i�U�_�!���4w��>�E��#���0D	�F�(@D�W��cx��o���@�0�;�v��9`Q(`Z�]��>��a*�^�I�NI�N���l����s.���4��C�(3��������p��v~}���f����o���Jw���A�n�f��DMA���@@�jPU���u��2^��������VC��U�ZX�����K
g,��FoTrs`0�\$z�z�t�s��n�9�'�P�-)ur�`��a$���Ah@��@���?H���L�Y���'L:p��dLpI��D1�����#S�~��B������P�DUn�e���D�`����S���OI�X����N��`@05�
�$��v�W�t����K��0P��j�&/�F�AvZg�_�u�����A��T�$g������"�y4�
F,l��;��1S��<��L(<
��C�@���[��eYN����9�+����-h���8"�������!H ������|/=���(�=������|�dr2�&������b_VI����$��:46|���p���+����Q��%`|�z�77�7{���0)h�����Y�\���_zZY��7�1��������k�H�3B�J(�������h�<
��d��,��	H+�v����(��c�H�����C���p�q8����^6�eW��c������6ws����2nYo�m�n���1���.-�-��8�;U�(��3��d�g�=0!s�
slw�B���]DA yZ$I�4�9f�G��/r��9���T�v_���o���0r�]"#����G��hn�6��s�m���d�iC�h����@����|xe$z����X�\�%�����Z:��gDQ}���V�m�):��@�Q4&$��w��Q"��������T��
��<J��E1>0�\��*�+�Fa�+���c����e�)���������l���v��7F�@��c�u6O��few�G��b���C���m�I�xr���C���t<��@{�&�98��;�6��� �y�b�>n Q}@�O#� %�S�G<���y��;v��4�Fe���I���� .)���;Y����
gG�P�������D�n�+���H�,��zjB�
=��
FF*@&h��E���1V]��H�.�S\���3�k41�	������S�@�G�Y	��&�XxV���g����QE��oS����U��2��!��)�Rit��#f��zx��(M�_�na�J&�;��l���;��T�S���,����D��R@I�J��~�������������������>\�?���(��SB���F�-QN�o�O�\�����
#��GP�M�7�C��a�Y���`5g��"�z�VH�������*�6u�J�Y)qWJG��[� �9�r���~�:&���]QLje���k��2,@,�������p��W��
+�q?��LZHk|J������H�����`�$?Y*������>�U�tTs����t��V�a(�=��`!���SA%S��
~�1Num�1h,|��&VVF����yA��h ��}h(�kP[�:K=B]cR`;&�[W	��f�n6xe������w��!�D��K0�P�3w1j�Z�_�RE\o��TW�|[(/����a
�x�&��
S:e��u�n��<�T�(���{ax�/����Vg�"�&l7d&e�6ku�����=�-��>���,���5p��\u�nn/��_�N �11��p ��*y*h�h1����3�F��&8���zc���kq\��4�dt\&��W���0U*����4>��[P�`���#��	�%N�`h�����	b� }���E���|����~���9�U����NY*���@0�VXv
��U��3�[�~�E��88�'-�em4�f*��#����R�
(�'�ao��dK��k/�.�[0:2�jf@��g����L;��a/��\T�T�_��dK��}-��z�+�d���0z��X��
X�n��d�}-u�2$zl����c
5�1c"�$�7�"���q��f�?�����
>�9��v�(7��(_��Ps�eoK��:��Y�[�)gg��$��u�0�����dQ��N�x�
�+R���+��F+�oy��}��/�cQ U��ut�@���u���k��6�r��>��[He�d�MD?:���Q[bZVi��L���Y���!�6u�i<o\R�[�*�,>�Z,dD���C%i5W�nK5��`�hQlT7�{2��9�)'�e�-Q��.����n��R�$~(�xkK��n������,�����QP �J���_od1�Q[;��GM'U���JjV���B� ����
4�"�JQq^�Y��D7���{e>�S,]���H������ �R�P��o�	e9�&�B���7�2;��[��$���^U�j
Jv�3���i����1�n)@��#��d�W�l��B�{������>v"��,K?�p����^����z�W*d]0d��xl��U���0(�;w�����=��R����q�HaCwt�*\��0�k��sK� ��	.:z
l�������,~+�Mr����j,IB��9��m`� �HJ��1$��EE�>�|5�*�t SQ��l�Gz��C��	�
!BYu��q�0<`��n?�}|�LD�S5����6M�6d*U����7x�7b��$���Q��d�y�9��1T��W�P�4.�Z'����^?��A���c��z�Z�b��db�"��2+�F��&=�v�K�������}�����a��?���x8�g����i�S�
����N�^�!�94��UD`������8+�����G2��~���J�<
��C�7!GyP�K�Z�
I�|�n�OU�}zt���|���%��R�TT��\����������%]�^�>)F�"�@d��`=>���8+}���P������6�RY�68�\"�O)�q���Z���)-,��[u7?N�,K�U�.���>�?��/ ��������ywB�z'�<cM3��z<����e6=�f'O�K���>d����*�U��_�m�w@���b������Dj���f	I�c�MA�l�i�"����"/O��b�7�n���{�{�u��J7�t1p�U�jW��A^����CD���O�������@��8fH�x���j R�^�0W0.T;�.M�
��H���H���
�HuY��m[�$|���tP��K��c���o�3@!�T�wf���V������n����St�P�d�]5�r��$t��8����^9�Wu���$�2�{_9���#M��~����Yd���N(U�%z� �eN������1��`609r"m{`6T�X���c
���q����-?��u[�����
��4����
l6��Q�
$[%P��.XP4U������z���P�aV������B��M�Ga��2��*����}��e%�"/r8#/rxt�^_��qf�N��'���^p��]].�x�|��*�8cM����r8�c���"U��r���,�4��Y
BD�����������e����}��]-V�:��?�4g�X�&8P4�LC������8r�M�O���S9�*�Pw�����Ul�O��4�)�	��$zPo�@V�C
��;(������������y��I0�	�*��:�����{U�b�tC_*s�Y%���XI�k,��	�QQ��pZ_(���!m5���������\N��gX�:�5�J�L�q�%�PA�$wC�v��T%v�_���������F�;�d��|hp�~ �K��y�ik�����E�_�O�������u'��������PK�7���<PK�nJ,patches/0008-Partition-wise-join-tests.patchUX���X���X��}�s�6����_��������%�3��['�d��c������z��d�VV����\����/� ��e��S��F��h�?h��4��
�f���h�o���c���c?����7��7�lL~9D?���*xD�#tp���C��`����y�N��8Z���;����_�����~�[�����z7������;?�Q�]������#���������={;W���`�G����������������y4_-������|�� �������y�n�����-W�����fA��5{"D� �K�/���"�,���nv"��}t}�"�|�/�!����|R�������
������up���m��#�;�y�Q��2�Wq��D��p�v�	�G���E������&&�����F�a����S�sC��P9'�����1�����C4���w����!"�
�������2n�d���^��zX�hxpp8Y���\������-����G�k�lS��;��WC�����&���������;�����7����x��!��-�oC{�k��[���.�K��|ry��������.>��[��l��BJ;���)9�����+�Q8_����V��a��}yzr}��O�;;E��h�z>��h���2C��5Q�u?g}�+�<9�����?4&�C!��GN���%����/�W����O�w�G����*�	��AB��@G���l0:cN������5�tN��'�N�N��F���&T(=���������sBw�2X���>����{7@�������������)_��
wp����+���.S+��������r���O����Qt ���~#y<�>$�,`��������"�����*���&UP��Th
5�T������6=@���U�uU��*�~��( .().()�JZP!��5�/I��y��=����N>���?N/���:��/�����?�wv
�p���3��O��Y�����:���~zy�A����������������K��	e�������9��6�5��c"�^��+�"�c2��������_y&vk���<>������������AAbb��_-o������~!#-�*�]��%Q;�O��~��#��q�����BU�i2�Yd��h�U�bx�8_�)�Z/S�>���CKM0/�\YkM�\V�	��hV�f��e�aVDe��rc�vb]��g
[��0��V��l�Du&I��$:���c��&���cET�c7��(���=��i;V����j�y#�^}	����v|����L�/�k%y�H_XEHF:�`nc���2�/��!�x,���_�/#��_�/D0vr�3���I2dT�n#D<���6����5��b�ldd*�S��K���
:�������������=�n����5��O9��j���k��=����1������^�w����c9?�Y�L[?��x~O�?�YF2m�`�T�}���<�X��b�1���q������c��������n���G��&2q�Z������=�d<���OU/P����A��Lz��-X|���l`�X�2���4��X�F�<	���C�Ir�m�Zb��or�����	%�voz|i����l�JT��k�����hZ����l��hB��'������&�
��*Q�
�,�'IT���������Vc2�#z9�r���/�_�E����x�g���1}�w�6HZ�b��5�CS���m�X4;����F�g�O[��/gg������9�y��H�,�A������X34Nz����l��P������ �m��~�
]
Yr$&����!M�=l#��+^���G�7D���p���j�TU�+�\Pi�h�h�8�9����U���L=�!R�0�H=dV
�C���2R�
�TU�+�\Pi�hC���2�r��r�<#�������3R�UC��T�
�TU�+�\Pi��2�>��m����J��J����Q����ST's(L�\����N�^��o�{?B�e���j�G�+�0���G�H�.�@�%
���q���F�P��_���a��V(Z!J�!J�74���=�8�]""�y+���5B��uLI��?&70���m��m��=��mV&�|����/Q����$���5m� *�����Y28=�P�zT=l�z.^e��F��s2�U����V���njT�U��W=�5�h�z�9������Z�^��8��xh%�!�"z!��x�<1���]4�F�� ��I�� ������5`�?H�$5.���>�C��7Y2s���_P�S����t�k��k�FI�(�'�(*�;��
�E�19x,9��N�IE������e\�(�a�@*����1�$�X,U��?qR�J�%g���0����Ux��kx�T�&\.�I�L�	�T��?qR�J�%g���0����Ux��kx�T�\ &�;�^qz�]� �y����2�Y�
���s"����C��[��\I�K�@@��m.q.���o�*:Z�<�.�4��s����9-yNF�K�����[�|���<�����fu�v,��gx�r��=e����4W���[Wt�o�����c�_Tw��c�0>�������!���;��V�U�:X����7A�k(����+S��[$���h��`�%^�������b2^������w��m>���|�iy��ua;:��[���fs��������������mq�@�X�ji}P`����D�]�� �6gs����$�"��m\��V�GtF��qD�{qgVDr�l��d� f���C���I�,����C��t�/�zelb��VM�'8Yya�Z5�W6�d�'���+�f]k4�N\x�b�G��77������fw7/���M��-�/�������XIX4[�s�fK�z���I������QH������br��u���&����i��B�^��{������t��L,A6���0���wq�N������)������9@�^�X�
�z2�%�K2j�����j^?��LX�_!���AZ;Tk�����k7t{Q�m`�z��BU�+������Q�pT�Q ��'|(��Q�AY��J(%�1�m[x��{Q�@<�/s�
p�:��B���m�P2��Yo��0P6Q����NR��*+mnl����O�v��X�!��Zn�\=���`��)��)��1��=����C
M�oA����F��D%������+��n�zR�O��.������N}��G�����;�p��8���:b2(�'�w1�n�l\>O��sT������`�����S��	�6QC�����O�]��I�VNF�*SQ Vl�:��6[r���L�������PG���-D����
�������evgl�D�%b��?�[>[r���%W)��joh�Tum!�@�����������_6~Sn�������v��g����V<~�5�g����Z��:`�����C%�N�O/O�vv��������+r���K�x��"����4�����cq�q�/�H�sl�Ia���������y�������T�:N�G�O/i�����\]�2�*����CC�H����y���w�Xud��+'��d�����*��E ;�@4�(�#�,��_�@�l���m��r�#��</�2�A��xm*34>���`�����]�z4<�������+�K�m�g��z�K������P���[��Jed����%1|���t�����I�T���|}L�J
��v\��a�<	�.?�����4Lm�6�r�K����[�s=f���RDW���Z�<IH�2��y8�y�]�������2c�[�y�cnS�ar���9q,�'���o��5z�a��{A���<�I��=���:��-������5�(�K����	0�dS���K:�"��
�����,0�"gS��K�6�S1���L�O�34J6���$�����7���L�/M����A�}*��J��.w}��[��9�5m���G�6�WU��7_iFM�]��&�
}����'�N4`
O s���5��5��~��j������%�W�3F�,�u�p>���,�=������a(��r����OV���Y�5��d����M���,DC���.�a�|�6~���a�p�� ����4l�����{a�F��>�f�������H\��W��=����S����wg|��I@����������g=���}�������OD��b^���x�����h���� =:��������������da�A]_�m����%����"����hQ��!�������}:'��g�1��jB��#�|����7��������I��A�;���w��hN����������
"F��y�����.����y�"v�"�0[>=V��Wt�Z�(#��gt��.����a�9 �h�����|z#CM����$��,>Xe�&���6%���
����
��R�R4+H�B�p}E�
��UE���a@��*�(�L#;����4��t���v���("yQ�]Jw�N�4�K-�Cb�bKq��k�����}k{Rc����~���I���v�N�R����q������CxF�lX^O�tLZW����ir�ne.���������'�����w�e�8�\x�VW�1��J�;�T����#-���4fp�\
� ����3�~7����+C�$�.�8EJkh�,�a�U���!5���tb�������n�'��O�������n��Q,�,�d���O�r�f���I����_���`�n����Fm�
K��R�f��
�������y7<�q�6Fm~��0���QO�J���%Cmw���;�m:���E|� z���<���e�������Af1�����eh)�f�Z����������F��X�z}�Z�X�@���BK�g ����2�N�SS:�
X,e�.���{��W/��]O������E�:-�6��b
�RJ&+�G�m+�(+<��������_v�� ��j��Sw���A���Stj�^���V�n�����Row.�'������M��]z��S�:�������!4�O�$,l�
�����\�mNm�m����9\��u[[P"��l�k�<)� T����.�!w����ul� �fl����3�S) ��n>�M~�.���l����i �^%���r�L��y�t-�{h0���}w�0=�m��������o���$�+F�'��l���s$���|�����{k��I�R�?%�c|�^�X	�lXW)��
���uL�K�u,zu��������;���xxr�xl�B��'z\�B�tpKm�=��Sn��Y��^����'hd���l�Sh�f��Q���"^��l�hVj������0F�g?��*����ex�����y��3��8�J�}�]��w=��A/�m+���?��A�mC���9���A��� OVDyv�gyv��d�G��A��I�s�<��)�`�?V�5y�����fH��k��hV�3y�����f9(���H��!]����������2��u��k�,�o`��Y�e�02c\��%)�@��k�<I�O:s3�����yN0��ObT�%�A�[`�'�/���W���CP��
/s����bVX�e�7�L���6<�6a�3����-�%������6\n=��5�x��~���2���n8�vV/���l���d��u�I��(@�k�K����c/�Ao��z�{k��$����nE=7��>P��S����XT�Av��]���3�B�������
U>����8U�
�;��N�-�[N7��Z,Ob�8p���p�p����[ B��i�Yo������O ��
p�,�@;�T�LC��jl��{�	g�5�n,�><�xv+������f����K����=��
	�(��*�J�SE^	N��+�Iy%��"����a����!�
,j�v}��C��������'��~:F�v������V|����&<����W���h��y�2��!MA�R��i��j[� @7��H����4�G�p���?Ll�7��_U�W.$�\J���\_R�o�L����ba�!��*p*~������de2GE_�R������-%��NF���*G�-�H�:��^D�|	���n��	L�I7/Ii[{��\�$	���� Q�s���!��jU��=*5��~����?����h��X+,��,������W%�R�~����~a�~� z9��yk�%W��~U��WYw�_��K�������T��[�_.�+/��w�_���T�_�;7��//�^�����=�~yb�<�Vf�9eeJ����W��s-�eR_K��	���e�L�����`�*&�U�����t��������T���f�\:W^\�����rg����R5�w��F<_%�pj���#Q�����E�%KlG)�,����8=���������M�X� ��	��4"�dE���tf�3#�|
���=��!�2���r��['��.����p�Z�h�Da<�=�_��a����������Pt��[�h�(
���D0���������9���Fn*&�4
��:6"U��;F8�1V�YdB"���o���z�����Z�����z�9���v�M���}y��h����Omx�K�����&O�Mt�����������l|^����q�g�(���q��V��hu:
d��&��WeYz���J����3qv4~3f%�5/���2]j}�pp!w6|5'��9����Z��)�U��J���=��������[����[
���������96G����.�*��l�j��A�AE��i
���V
�*Ug����vN�oc�uM�ZSu5�&����f&�
_qN�R�P��B�TO3���z�Q���
�	������KA����]O�)����*��(V�+Y����8��(J����U�9�NqD���#6�a������Q�t��Th����������D�RDCA#������l-�'NZ-�M��26��	'?��@�1�mg2L����Wc�S��}S���.��2�b/�[�A	<���ug��y29a���/1�I�-�-���I��h�*�'��%A����K���;��l^���aK���{�
�4�y�N����S��d
lm�t�t���dd�I+J�IR^l�YK7s�����7��(�z�O�W)���~�5)�'�����`�T�B����o[b�Ey�����8���]G�*�'I�F5:r����sD����vl���K�IRe��n�.+��b���g{����'�����&e�T����x������v��*�m;2$O3�q�Y&��E�<I6V:r��ru�����>)�O
�vJ���v�P6O�XSC��xF%��M�g��_���uqE�	������S,A�d�������y�Mz���a��?���BMW�`��m��t?3�,�6����iu<I#��e[]�*�S����{������=,�[��������()���Cl��M
�I����W����V�6p+�J�^]]�*�S�������E�$��Q�^�y����������Z��E�6)�'�W��^}����Z����4�.m{uu���O�{B��W����n��@�]�
{3��s�W������� ����t�8Y��/o��U�8�/��|y��T��(X�U]�{�5�C���no?H?.	��j���������)�������5�Y�_z~_����>�'��4��71�&����T����>�'��4��#0��8�����z�.�^�Oa�)l���>��O������O��|%|���O��d�"�@�F7n����N�����
-��y�g�!�Yp���������������RB��gCr��$�(�<I��Ag�������T��� +�ts�����3|�����m~����NMg�m<Y����� eY;�Qe�x�@��;�ff����n��'��'�
�G2���?�$Z%g��Yo�������iBUk�5s4�p��l��Z�6$�@w|V�@�YM�z��,u�'��T`���jb�:0�}<UN���q���N�����R_
���(��QW�2�=�o�`.�������PSk��j+�e���P���O��NJ��j�X/����`��<Y���
�
�P�WC����r��D�3
����#��)��������S<ow�[���K��R14��^I�t��&�Z�������0~�'�d�����%U6�f.o��yR���8iM�t3�|������B��Y�k[O�z�����v����z�]�����-�pm'�e���H0[@$xSf�jqet|���,o�����T�������t�l�V����f�$��l���m:������Z.]�_(|3��m�)4�@/U+�e�|�����[�U�?:�B���-Z+�4Uqv��y��|k�.��.	��2����[��7�1���e���f�d
]s����v������6�,�F���!�N5���,��n�]���R`�a��V,��m&��`�B�����<�4�\�Y
mH�T�9�V'y��MFk�;���u��B�����\6
-���j�����fB�[Vl!��[��C�K����u����L%��C�mu�'��T`�v�sMl_
�-������P�m��"��;�����h$t��\��.�u����-��������������25���X7���m
�5[�d,7�D���#�d���>q5��t�6&Pw�������n5V��h���>h5Q��c�m�`���"��{��-�X�	�r18Y���-��-����z��E��E0�1��2�r~w�-{�X�������h��Nrfuz�zW�r�E����]	l��v��F��5���R��7��Vh�����u���
]����!���[��'�\awW��(���z�@�l����p�\|5G�;C�	��Z��p����o���\y����G����v
����lN�V�Rs����pyX��J����;�=����h���N9sBd��6;OV�j������n�r�j�����2
,t���f�2�a�F�U�e�FD�53Y4�htH����[���L��j�2������X����&���/v�E�P�r�^��J���������VbU�:��Xq���.^�N��w3�b��[y��V�t���.�u�]ho-�L��@��6X�Y"y�����I��	���>4�M�>�eg'���'g;��|�
��h:v8�"W?���G�,?�XG�L�c_$���_�z/C�P&���:�IIn]'v�'�
[����@�3u!�%�������u/3T�Q-���M-�G�9�����j�X2n��'�{5�F�p�H�#���k(�A##?P�)N��z��h�^/Za�b!�%�g����55�5I���u;�-�l���K�GR��Z��,�R��i}��zU�h��k�j�ZiU�����Ba�n?�I�,$�~��7�g�~`�)�x�A��M?�DS*�~0L�A
X�4r��M��A���&� :n��hT�W�H_�X
���rI�z�/��nA����/�fn��M\�����B���)�EH�;5�gN����3����_��f������q�R�i
��~��!�����95�Mj\�,��AH���X�q��>�}���$�fAh�Q@�p��>=�T��n�p�^6�n�,lp��C_�2��F��t
&�}���^L���j�2@�4�T)s�Q�J<��%`D�O�CR��P��nb�6�x�Rb�B��o������{�F~����m!kO^���2^���_:�����]��@�}��T�����#o���%�/��T�'����B,�u4���%�F�_�C���������������+t}���)�{@=�����-���g=���}�������O�7-�<9���G=�
)b�H� I:�����������������$J�`@��t'����-ElF1��������R���ft�C���S�t~uzy�>����S5��jBe�#�|����7�U�����hK��AHe1~�n@�y���������L9-� "�a���T�0���{t}?QD��u�S�bEf��5�����Ot�5����e����L��QJ~@�:`7��t�D�6:�2jc`[=���D	}���	�}1���]1�� Es=���T����C���X�C�!�?�b����X�+j4*��Go���|��]6��k����t#1y�s2��?�Ew�L����hc����e�,��6��,��*�t�����w��&�6��3���U��H;t3=���%%�{E�.���!����4�sK��u2��$e�|Ow�C��C
��I�4�.[�a�3������t']I8�vxN5h���k{7�W���^R/�I�f�I�bu@H�V�<{���D�b���~��W/q,�N���%Z�#��[�&k=�����uUW;K��/Y���w�Usg#x��K�_=��p%�v��4�[J�d0Ka���vA��v���v�E�	@.GcV��X�c����x�����?3������,�,�����O�����I���������o���X|�|����G�ZA�M94b����i���.AP68�����3N5!��:G*�����w���/�S����IY���A�B)��� �@C~nW`$��4�e�Q���TCQ������J���y@��,T�WF#�:���@�b��Xl,,��E��11��X�Y,�,��,�t:�]t�j��T)+�:k�6�Bg��8����P�@4�4����X���'rEa�:�):�I:�iu��t�t�s�����R1���l��c�:�6���Qg��aM��Sxd�a<��h��	0K�nrP����;�6�N0��n4P� �;�Vt��v�EF�1�Q��������v
ZP�$|��`�l�s1#)������P��(ca�<
���I��OW��Yq�y�����gJT�z�Z����z�u����7��T���0�7N�8����U'�5t:G��8�w��0aN)���o�<�c���|Uf���
w�d����8O������s�9\_�0�s ];�����t�t7�sj���|-�.�~2�����<��`�_Q��#����8	�t��=2*xs��gc�@I
i�H���i��4�0m;cIhH�Kd����1�9	
7��3x:�Id�OV�D�����_�o������~��
��#f{���������oK7�Y ?
���� m�����3N2c#P�#����S�e3����/?'#P� ql*�b�����m��������5E�E��h�b���C�0�����;��@��^
@e���z?J,@���)���������"�)���30���X��������DTL��g3C��"*�����EV�&(�]e"�h�	r���$5f������;�dc��Vp����oR_+V�,�U�y�:�=�{.���Q�ee�+�l&��j�0���5oj�I.��04�O��H�.5�����q `u�7g,jI9���dX�"Q�M'���}"��o\�����zc�����`:��I.��t4�O�������M�q�gu�7h:�H9���tX4a�fo\��|!�5a8���U��g�E�A*$�������f<�?P-\s������$��:�����C-�a~���m3Ho����h��BiS��|kn#OK��/�6�c�o�a����G����G,c�Az��XK{�_(m:���|kn�C-���Bi�3�=�oO�'GZ�G����?�`G=9��>�/�6=-w�����ii��/��;:3
�=��dx��=.'�Ji��a=j��p�_(�!��D=�g�=��{����������J�3�E��y/�v6����(�x���kJ���'��7�m����~��B������c�]M����y�m������y�&<T�'I��v�����#�_c����n��
�In�)4M2�U�j-���r�����,g6���N�6�
Ov'-��'�s f!�uV$��l�{1��l�L8��b.&��`LD��NJ�7�Z�����'�j�#����O�6�O���X����TC"���nH�O�����!)�f����`�TC��<�����*J�USZ�SYu����Z^�����t�h��5x�F��pl|�qQ��!qYM��d���3�,3S������e�*(�zE��,3��Rbc0�8��p;w���t�ef@\J��|�ef����3g�T6�.�K�����������`T1s�������[l�����u��u��'c��/�Q�������|+F=\�]����(���+�jw�-��qwp�A];8�����6�ep0���6�k�8�u����S�c50��k���V�QY�&���h��k���COY���y����h���MG�2�V<���\��~���n������u�s`$6��AB;��D*m�T���$7�z�����3�h�5}+[��	�t��\��i~����<+�m�mL�z�������,hi.L$�����O�	���F9���Q��XB���g
�Rk��6v�it+�����h��
RlF�Z����@b�������mm��9�
�%$���10uv�ic?�F7���v!�O�l?D�\//+p�	 d�����u=�u��V�f�eV��g9�������P�7E]��U|S	"��%���4F-���	v-�'��T�_?��@�lnlV��M��"zY+�0�A�&��&������������-�
��;�Y�LC`�vl����B�q�i�.6�'�0�f�����g�'����
h!�Wt������������
������������,Cp�eT�"���Ud�����2PV�Y��*2��6+�q]���r<k������r������I'����g]8{�Sv�~)e��s�&�����S�p6��h��9T;�9p;�Q�H���Az�3Q�������c�w�UE��BR�����q+��%���F�/&�
������O1I�SV(�������^�y#o�X
��(%WVUN	�hF��E,�s�����<��?y�J �u����e���G��]ZG.x���T�
b��-���&l��3<)zTj����y�yo�y[c��a�VX��!��*X>�X>X>S���3��������������^�_>�bm���:X>��`���Y�>/I?9�h����I�|�5���-����3,�;3<-_��9Z>/�^�����=��yb�<�V���e�
��S,�X>���Z��D���,��X>O�|^=�g2�_VY�W1o���d�,d����5q�|�\��Z>�5���-���&Z,�;3<-��i�C�6���K�&w���)vY��h,�}r_���XnM>/�wT�o����z����M,���������9�dt���H]��:���:��:{{d��0���y4�n��&37�}�}�3�$��dvF������m��YWMY�N53M.eV|*����#�y��\�y�Qp��V�G�@��YZ�����8���	�A��e��17<9�m�ek�@�8���:��%�TY�?�j�h���p��V`�����QJq.*+�O�A��������U�4��/��v6S�Q�n��#I���.��62U�QS�l�&�<L�,J^[��:���Sm�sA���>���\xm�O�{��Y��8�6T����M�e&���:�S����8&[-�7z��[1x��Lr�v7+��2g*w�2�)w�2�(��2�'w�2''wu2W&�a2�%w[2�$wN2�#�:2#w42G"w'�A��?o�Lq�Nq�P�7T���&_���
]�)Y��K�d��||_�c�r�S�>�pr��e�����UVg�����:������]M������k�N�T��BB��@+1��O���P�������9�99���g�|��Y�����������{7���K�>�z�=c<I
|�h��|�&�I�>v���s��/��L��wW��	���M�n&�c��},�n{�x���i3l�������-�V-wM�ZS�J���%5���D��<�� �~��!�r�b|��[�h,�+��<D����`�����k���I�]3m�X�����}(~�
��&��#,�38��b�]��� �����Vz^#]�0 %�o�u�\L,H�hx
-D/@B:�����F�d1���"8qPr�Jb���.����U��3�EQ�9t�0��gl8x�v�)���4�������d�)4�����Vm�'�O��>��y�_o�����?�e	�MYm�#�TGJlK�	,y	Kz�NbI�T@��V������$��j�����<.��ZU�x��Vg�:��X��/OO�O���wg���L�����2�]f�������>�������'bi�
�<9���7��P}��_|L���D�89���*�zo��E������Ko��)EF%�8>�?�_�^^��N�L�|@�E�	�d�||����)F�.Xk?
&a��a�`�F��
�!��'�'g��3��p����:bH���xI�g���a�cX�X�}Iq���R6���u��?L�G`�`�E4e
Z�������z��=��
k��������psa��0�\X�\^��������kd�<�<�C�Y�3U���Li{J��{�����������V�y�F������q��2j��yZJ��)Q����V�uAZ���!��������4�Hm���#�)GiS���G�)���e�Bt��\Eh ��q1n��#`Yn���wA�n����Z��������)���	}y�I�"����Cn���WE�$�:[a
����`�e3�O[�0�v�������������tN���|��i���B�~�A�l���� ��+��EihF�`s���K,3���kX_�AL���_�C}}�dupd��HR�� �G�M�1����T��?-o�?8�8�/��<�8&���V�����(�h�K�<�
�+�7K�0�P_��=��+�v(���Y�������b~������\��M�)=�Q�p�m�����8������zKG�l�����W�2����u�e����7f%f�1��qj�6.���I�+R���
�4�7}���hou��Z�k�]�������wfx��I:�r���w��FG��v�V��K���:��;h����vP�j��#��`�}�:��r�}���[���GR�^����������?]5;�2)�(`�}����u:���sBH�.��9�sK���9��H�=������q����������N��cn�R�LT��>g,#��r5�6��r$��I�R��Zw+�sh��C`?-���xc3�-�wV1i��i5L?w�i�Si5qho�nN�mo���-]��v���0|KL��X�I8�n�E���v��z��������������.	9�v�F�����kV�����������Ay����(���l=�_v��!AA���>�����J[���F)T����`����y�%D��u�/B*g22�w�	�qqHF��&�Z,g_�]<C�y��?�C�?LW�9�
#���,
3*7~����f���� ���x9���B7D���y�Z�,V���BR]�g��NH����/o�k�q���@�:��g?$/�V��m=a��K{���|yx�_p��G���rIJD_���j��	�d������d�(�@����Fn���K������������g7���qE�R�hz��������#��\Fy������3i��c��Hb��ww�:�F~�)��,��}�V~�5�9�sg|A�s"���M���K�������a�D��&��vI^SZ	��PA������)[ ��M�;���&�QiaR7����fQ1��+�J:]Fj��4��j�
F_
�Um���RC(V
��FJt���"������6oU_-���V�,���������~���Kg���s����c�T)����
i�s��.���g�q����L���}E�1���*.���:c;%��Q�P���a0����:Ee�T����J�q%j���8]5l����)";cVH������7�ZE�7��b���g����c�_P�~G���r�,4�R�6\j\,6�].��l�a,��Nl�
�V��A��mn)P�h��W*>��
d������:�<�L����L7�-1J9V��+�L���{Ep(��xS�����?�7�7�`���q������8���W@~������w+2�@��CJ����M��9>@����m~4V��j����2%��?&70�QSA^c%c���S5c��q�uQiV�(�AZ�$\h=��/jOK�������o��%�������uL���	��z���+����������J������D�P���yJ��iy�
���#�~�0�*ok����8�e9n�������i��O^�:���l�:LoBu��q�=9�|��b}�����;�LV����}����N����T���O��z;��)B����%s������ �Gs6�GR�z;;�>�<9�����oP�5�>'W�J��K�x��"���
�4� b'����%'�����i��h+,��0�'%U�4?���Hr����mU�������d��l�JZVI�c��x�I`7�MM��i�-�����W`�w�����=��hULq�g�����@�D�KO1A$�L��@1��FWS�:U���a{U������#)@UiUES�W5��,(�"����P��F�p�j���l��A*������.��t��}�A���G,0��I�-�.o
$\����4���b(n��lr������
�[5:l�����������z��z�sg�8�0���%N���^qD�N��Q_���T�S��7�	k�-�b�a?�W��W��U=�����z[Q�fT���<��)n����m����Q>%~�����}���e�V��6�~��&���
�leZR��	���R�B�y�2���=��$�()�H<��i����V�����r�$���]N���PK���5���<�x��Hn�^����:P�/�!� 4��1j��M4�f��r�Pe��\	�6M0\�c�\��m�;��$�l��MF�	�T�^��1
����7��b����k�����D��.H�O�-�k$����t�w�Q�����K:����=�,���no?H?.	�= �1�R�t������n_�
J��B���|X���y�|������|����|����|<�b�Oq�#�A�Yv�h�)�>�M����<�����hd#�FCQ:|������K���������p7�**��1��-~�F����{����t���5��F_����TdK_�����!)t7����4��QEto���.v��n���n=��V�n=����)�7P��P�z?�fJ&>��h������'>Xn�<@�+������W�
�v���R���I��
�v���f���IW�
�n��C�
7�����p���f���f���&
���RAP��&bXA����aM�k"��������
`����	5k4'����*Q�e������9f{�4�u�V3��c��}��B���0����%_J�}���F�/�v�6C9�8T�l��H������8f:�����=���"�`�a����OW��Y������D����!or����_��=�U_�����8����NGW}�j�&t���q���M�uK)�/��t�8�[�a�B�����x�>��o�x��"N���:�tJ����}h��ip����������:u����R�\�8�E��Z	������j+��Y����Tt��=�bys�����=uWC	l���mk,��V������
�����,�f�6p�`?�j��f���2��~�?��!q��G:��|�	��L��`�� /�-������0��	J�w!>C?^^��s2��8�/�J����Z�Y5�������EaoM}��������0����������q���}U�����hw��&D��uW���{;+��v�\���n����7k�jjZG�%w��mY��F����}}��
�umB�[���6�X���8�_�\�B��\���X������M��v�D��!�,c+a��4���K��7#��K�J��L�y�	J��&��n��������o3�)7q�N� ?m��&_��0SE�Ph�B7��O���6X1��8�����?s�yF7�?��5��������mY'�Rp�q��,Q��}�����������0����S�l�����'�������"�������K��A`&�Q��U����7�\
�{s����\B
W)2*	���Aq��?��7��.7���%� ���[��3
px��nR��'�eA��`Y�I���MN��XF�p���6�W��<����5�7���Pi�����v1�VC�S��S4��6�4��6�z�`d�I�?$��#��%!��<�����	3���Uq$�����4�Hm���#�1Gyc�<���C���+���M�?>.��
��q]����]����m��5sTL�p�Fk���I9���w������
���aF���f��M�M��K@F����|
��Ur �/��~Z��5�U��$�����/c�fEZ�V���_�oY4�N0�j�baB����������zG{�����&X�rO�-m����2�A�Y���U��|]������/����:��;�<�����4KU�~�\#eS����)b�\
�7C-����e���Ms��dBL�5
����]S���H�[S���p�h���xg��PKF��EJ��PK�nJ5patches/0009-Partition-wise-join-implementation.patchUX���X���X��}iwG��g�W��oZ��x�m�PeqV��T�gz��
@�,@�U�(N�����ge����y��G�UY�Wdd�����j0LG�Q���t��Q�������ht���ON�;��z���U:W�c��|O��^��m�0����v���[�4�'��D�n��Mg���Y���a>���<Y����e�R��E:`�{�}��������wW����p�������g/U�t�{��z��l����;��~������I:�~|�n4.�5�4��2Hwi:Se6���j�~J'��t��`�����m�F�"�&���jq��j�og�o�T
`�����|�M��J�����__���mZ�j�����h���mRJ�y>I2�f�s��t:S�I�,�R%��$���"W��f��.��^-�dVN��F
��H�y>�_
�=�=�)B�m�4�s���i�\�
�Q6��A����`��~��t��Z0���S���`����eV������
���"����)����,Fi�-���0�1O�i^���f�r:�~`YS�1�LE:��l�-`\<=i���1�|"��CK`��4)���s��d����+����-�i�(	��t�������D(�� 	��q��Ya��Z7[�:��[�e��p����-���m[!.�w�)~RR���[5/��-*������8F���������Y`����<���d�/;�H����8�����6)����'�������4�>�R�vcww���b�7H���ho���a>�/g��=T����j��,<w�W��aO��u_�8o����u:�/�{����_e��!��� ����g�����Y��vVw���1�~�=���Wo����E
���j���9x���3D?s�-}|�Q�;_.�	�1��L���6�x���|�_���?\���%���+~�;:
�0�<~[�M�r�w�VQ�ww���f��r�
����m�#�0]�L����4�cg��K;]D��/�a���i�B����V�"}v��3g#o��z+>�������(��
P�r�i[:�3������/��������S������m�Z����I�<
w��b	,�dv��Zx�N���@h��m�lu��$����^�jw�&Z����V�od�Q�Y
��^2����nox��Z�stp��{]
���������^���D�v<�c���lE6\\��y���r�'x�o7�����������������W���t����+Q���h�"��
�5v�����D=�������/2�oP�������6^��������b���=9J��
�lxo8P����'�{m5*��Y9���$�},��KX�i

���Mr|]�S�6��`������(R��EyYf�l�-�2]+��l�m:-��'@�ir��n�O�������;Kj�����=,+��@��S2�F4.Y.���
�+��Z��Yg�w�R�X��t��2~���C�U��e1�w�i��������
�0X�ZN�x���v;%��'��z *M�<t;�G-����C:C���l�x|��V.�<�3��H3���)�X�s
/u|��>I�J]|�'$/P�	q���!sww��<E��./>�����P�������I$gFQE%����#1��F��d�W��������8��P��l�p�;����W6�����9����]��4�38���yZ�9�
p�w Z��s����M3DL�h��@%b��D=����V(��h���� ��a����E+�@�������N�1J�Y*��W��ve��~��C�^�33�A����R�m���L�|KJ����i�������IF���j��J�_��Od(�Rg��7�
/}��
hh6�%@��l���j_,���X,Q��FP�w8�z ���(���y�	������p����&Z������CB�7V�<�k�4����\���@u-��n����������c"�I�N��;f���\�)�6��Am�s�b)\�0����G�B��~D'j�����tB����%g:����BLCe��'y�J� ����$6�?�"/����@����E����H8����=��������bKg{i���T����!'�]�
\S�dvO�oc�`2YB��;[�H
N���H���Q�G��5bu����2M>Z-T���r��u���:8z03�60J��sb����v�(���
U,(�kf��
0 YXk#�]��#s�y�6�1�x����� "I���-+#�jT{�#�����N�h��E:N�&��E�8[N��0���_e�G���w����;�����p��`�ri0����M�w�lpU@H��
c`�]��e7c�j]:��2�bV"�����q5v����1�(��H���c)|�;����W ���M�8�����FS��]���&#�M P��67u���V��)�=|�7��l�����?�`�"���BnAC&�a�bH���A��
�@p��p"����X�%i���6G
'��A�]}�a�H`U����"sBkc�9g����;�J���3sZ���b�"��]�4d�@k�Gyl'H3��L�;��7",\�ZpIpqK@�)�j�/��HPz�[�EP)i�K�9�i�h�CuN�rV���w��O�sd4��0�&s�Ec��q�y���Y��*���������/���m�,X0�����a6��W�R8�sA>�k\�s�Y��`���M{�6Th�	�A;�r�������>N�:�I���BZ%��mI�=D-�<�w�Q�[g�g��G���/P�����
��<���z~���p��������V��vN['(b�h��Oy���E�V�z�RS�%�<y�h9�"<I'��* 8S$f����.i/�	J�����l���Un7)@������i}7�r��TJ�B.�	��>�.��a��u_���N��X}���aKG����������������:�q���L�Z���<w�_��R;g��p��.�����\hj�^��nyY Y�p������R�p�N�~�}��;��|��E��Q���m�L�o��F�v�R��#SO���(��H_-w(��3.��MY�@�������)��������� {�;��!�����Z�E=F���g��g),Ya�6M���Z�z�9i�����ROHP�O�@��uH~����=��/�_[[zJ�5�����	-$B��_�%AZ���MM������<�#����������o�_^����x���g�j��{��,IC�L�c<�t�I
@>3`��E���@����}���[[�X5�_��jz�M�fVis�� ��na0-�Z������H���	[v��F!���:��|0)6����w��il}�yR���N.��H��ou;�5�c1�|���	���iL8_����p����Po/��,�+G(G���V�,��1�n�z�?�R�V���e����������fsdWQ���&���p�n�|�Z�������B�PA��%��x����X$p�/�*~��������S���w��@gigt�.�����7$mm� f9x�h�����3e����1�lg�����zGDz�[�'_�D���R���N���cx�m�~G1CO�&�5��}\�3�����'���8���x\;����vp���-w?CR�������}N&�-���{�����`�������0�)���T�Q(!�L��@�3����|�[�X�o�U,����$L8�.���u�=Y��^qX���^F}+xK��������������n������������$�z�	~�@��X����h��@�C���z��fPtg�H1hv��/1v��`i��;&�����;�9o	|�6���e�,��0�6��X����
-v�z��^���UN�w&�
b���e���Z_KpE���%9�����R��_Q���1`[����������@�gH�,�����P�����BNlZ�E��A��8^N��u���c$6��o���*/�y�"��@�&R��m�������!���C�q}f-�� :��+�Y��Q-]��T���j��m	���z�����������?�&e?�)�>����T�%�y���2���#�����qC]�	�b&L�����Smh7��nq[���
J@���!���R/�;T}W�,���
��6���?��(+�v�~N�D���U��?ft���P��W��O��|J��{]��������/�#�$^�s��SrEAk&�y5��~�(�nc��H��G#�2Z���F��4��>80��������T���"MF�@��o�������-�1��I>I�{'=&���X�F���R=SX���G#�
�yzA���!����Z��d�I��>�TE��y�W-5N����]#��@�w[[?���~v��l��NPV�����U���z�q�3W��6}H�vt��g�)�V]F��T��,��[�b)��a&Y�����g�O��J�-tE����t������J7�����i����wj 1C�)�>����(��Ba[��Z?�$��=g3}�9]�h�@����X��r��"�y���Kt]�u���D�o�'�*����@�PC����uiT���o._���_nm��}r���8s���vB�>������SUkjGiD*���34FB���9��L[#�#�s:��;8�cla`�M�)H��������;v-�V�~��n�p�g����AL;�Q
����
���K��;|lp~��B��X����N��ic��L�Yd���7�������L�I
7�E�m(����2l,���E��:)>�#��!(���/�}���]U�w�W��^�������G�:������T���*2��M>Kp��$����(b���8���b�b@q��iJf��U���I��G�*0�gX����45�/��gm`em�v���m�dl������AY�����sL����n��&���}4
b���+-r��f)��-a��o.�}��`'Ktli�dP��[�#2	Pz�Mb2�l/��S���
,/b�+o>�z���I�����i
0�.8�(]F���/F�}t8���d��G���0�t��|a�2����p����`c2�[>G�e�-g��p��y&���1�uO�f{������9I*%m��f������IGL�#�Q���91����N!*���Q.� ��T� 1C�X-I��f�Ii
�����{|NQ��\����{�{�����	�;Z0���z���G�,�;+���B>E������^W�������T��'G |����)��zUC�d�6>�>���v�!�K�����}c����R3G���,�PU�2�����r�w�B����^����V������0�O���A�QM1��;r~��������K�#�#�B�~1!��U����b2B%��������((z���8,/on�*����s��BY1.*<#.~�����I+���J�_�O��?M��%F`7-��:?3��7�Q���������""�Bf�����g�o}j��,�TX6Y �����O���*E��s���nn�ii��r�Q�p�4pE�%�r|�,��q�w�mV�U����X-#W���f��j4e�
"7)�i�U�+V��4�*A�yg��V����d;h`������m���?�1������p�>���m�Y�� ��tZ��z>��)��Z��<���p�}-B���7�a�W[��}=�����Z;���o�����dqt��6�]�B�=iMJ��3'�,qPi�h���>����R����v��#�B
5�?"�H���D�"0��()F�;�&�+j2(QP)�-�E��2���%"�P��]���U���V�A�qw^���,�������Rl�Dn�����d7�=�J+��/����5�G�`�'�1��B��K�C��lt���i-P�5^�F,Y�8��HY�g���N��L	hd�����L6����U����e,(�J���D����i�
L�m��;�q��`kK\pup���!e4����&�xC�.����hnN&C�6�P?��%��l���t����	mb�#��U�2J��6��J��]4@�;h����H�����7ac����l�����l@G�$�Gi�i,�����q��0���{��9���<c|���wf����#�!/'��j+N\u6)���,��Z�����Js������X���h������mz����.��~��iu:��I�ty������|����/n��e<��@�;���D�b9r������V#	1��y�;�l" *Ai��o%���p��?e��U�N�w�
`s��EB	�(�E���;@�0�|��� ���et���T�iZr��n���w"�KS6t�1�\i=%�/NK�� @�T�����aN_����A�.��.�NO�x�����Y�}��}����M�6y�vw��M������d�DY�XFK����"�h]���Q�A�� ������<��5��
�u/���������]	���k^�����������{U
{%p�������|�yF�_kx���We`|��N��0H�/��X}��������/3�#x������� �*��1!#���5�b6
�����q��C���
H�z)@
]��H����=[�P;�xzhEOP)�*����8I�U���n�bs������*@����7.��F��}�V�������A�����-������oH7f��Mtu�V�Y����X�^��f�����A�Y,Z�u���oUD/���t�Q
���fTx�2?7dS�T-n�_Z�Q�\����C�PBD��������������%��*��T�I��X��������b:w�|��'�w��O��tB��E�jWWg��?����y�L#�w�2;t�dL���v4O��s��EB,���k��?�.�bp�����4��vn�{	'�t���gbe��%�!����������|'h>�=�W�k�@Q�GX*2m����D��m��3\�
���=U�4#	��1%��bWe�..s�*�5X!O5G�?l�b�c����1�?����ZLdh�OP+����{�]2�^��e�j)�L���/��wS���m�*"X(��t'y!�d��^.*i"X�B�i]e�����������6��:��aD
�����gr���7?�{��k<����3B�U+7jWy7���K�D
-u>��X���bkn-��8��aY�v|�����[
FL�C�	Xt����]�4���	1�gvp�����818��i�g���t�����|}�����/�~�	_�����?/23���? J�y>&g���f�J/��)��|��?Ig7p���l#R������K\����"M��k��c�r
w�Q�2j��f����l��������E�xO�:W�89/G���s$�k�=b2�������h�Z��z���x�W������[A���@������-����-:1����wo�_o��I��2;�m'9Q����~r�nwz'����9QHks�8m9���h��2��������`���\�����5%��6)oW�f�$��7����k�%k�\|�FxJ�w)�����_�8�������������7��/�.��^��o�?��v �2������^]8�^�?������~�"��4xh�ipnd�Z�4�� M/���;�v�9>=�6F�Z�q��t)��=��<��2�wEmi2w�yQ$�"/��l$�A�>)�0�����a�[�'�VpnwXl`�a(!���y:�Qq�����O�\�<����svVJx�T�*X8�y������T��(�����)"*I���pi��W��y�3,c��3:�2q^O��Wy���������H��>������/��;��n��8�!m=3�!��r�<3�{8H�/
I-CX{t�z;��N�So%���S��TWE��Iz�L0�X�����$��S2[�������*vkN�@2�g�r�(B��/��Pa�L@}2oW��}���lJD~��F���$'��6�s�������|�����>Wv� �W2���|YS-Z"4���i���)������i�m}����N�4>���m��K�z:��&s<w�x�O�)�)i�mr
��K��^�����l�l�31���I�[�����[�tg��������v��N���������l����-m�q���0��2>4���<,��t*�p�?y��^'�m�s�� +���s����Q��[I�����)y�A�P{��s�s��3��2C��YPc/y
��*��7����)[�G]��v��E������~�y��<����;I�h���>��o6\�t��xM������q��<�qD�A;�F,�������5�Fe�������i�&�kV�@(�T�T�do�����X=����t�F�X�/y]X���h�l-�\<�(}L�0v�X���o��e������U�lt]�	[n�ud
��O��b>w���r�S�-?A�l���
�E�mH#ao:�
�M@��i{��m�NN[���`����>���c��mv���j/w�AP/��Tj!��}x���'�K�Tl5K�O�k�,���������2����m��ze��w�R�Jy���W���U:V�"?����8���G<}d���"H���)������xY����ZM�G�Il?�B<��!�UB��B��/���Z��'(�p���T���H�Du�Z�Ei�1_{�
A�!����c���:t��r���$��a���������t����)��t��e
t�q�����a��I�{��1g��
'��Xs'���d�U�_��w�n����>)x���(���*J	�e�}2���SS,���}
�Qa�X�M9�z��i {f�~�'�9;\"���b@���#�M���h���Jv�8*�D���|/�G�?���rA��X�`�%�����[*�8_��k��^^����|���=���,n:������H��4�>gl��G@��c�0.`Ed�8#��x�o�� ���@���Z�%��,�����D	������F��1W�^���W�;���}���,N�F��<��%�w�'w��L�%)R�,�������aT���p�+r�*8�o��T���XzQxg�8��.
EX�i(8�M���H��/>�=�,�*�a��N�)�KLil��������y�\������4'�)F�k��:R���W3�-���a|��wq�3�<1r����n�j
����`����(Q8���y[5e�����:#"��������}���h�r�i'�����?��ETnJ�S��3���������+�0��19���g��7C#_��N+$^�c���?2�@���y\r9J�*��n&��7��`U�S�.�X��;���&�0��L��W�jL�+]�f��6���� ����p�������X���9CH�����a��A���
��`*�|~��Jw5=:S��g����;.�����o�p�����t��4�ma��y����a	�_�~�u��C�G����u~���.����V��g^y)��[��zzd	�j��Xa�C��=UR�e#�0��8�V�U<5vb����`�7>G<���\��A�%�F9�S$n ��Fe����'2W�N��D����C&��^����rFJ�������F��/��.���c�|�L����{�����@������SJ9a`��9��Q�iw@{
�(I
�0�U%Q�"�9�|�d��@�{������=;���	����n��#H�lT�N��>�d�?��jSk�5���������mpp�3��.�W�J��& �H��%����TBs�j��v�i	H��K���$��� LDD�P$_v������H���������U	������9��K�8�W�o��
����"������Mz^�w�W�[���K_=.�w��F�j��z�]��f��nP����/���2��~N�4=���Z? Y�>�w'���Q3�`�L7�f����I;o1J�k���U���dK��U���},�F�ZY	ha�%+x��D�r��/ aapo���`U2�Z�:i����=�E�b���y�S�Y_�o���L��zq��	��
2W�oF��W�2%/�0��u�BRU���M-t�����)����E�n��V8����
u����SWc��������t�������]LWB
t

'��������`'�����]&i�����gs��l��\�����3�W�(.N�A�YW)�]V�<��h�n�f�
Q��<Z6%E$���>��Q*���_;��I�lA9b��	Z��j�Ea�J���
(��W�Gc�{�}��}����=���>�^��!�k���w�y�7Ti���-�/������:�Mk��~SSq�5W\]��m���=�����Ix�������{�^��c��Z�-���A����>�����8v�Z���&z������z6oc>�����D�u�y�g��o�o�i�O/�L���Y�(�����ZX���s8���@�s��,��_�vrt	j�|���l�E�&l_�_�F���e����M�k?b����\�:q���f� ��k@.b�3�F��2+���jm/���IF�Z?��Y��o������*#��6�LA�����[�l�Pp@NA�����"��|�N��P?����ch�����W�R	������$9�!�M�N������8�����x�4�$2O�r	XT�G.���\}����]�^�5�m�%�l��n�5��7�92�b�b4���MQ:�t�@s*px9#���l��C!o�|>N��$K���x7�x��l��D1�q�%�����n�7�z-��T��$���	���/b����C���V`zT������'~�a���T��s��$vL��n1���V�	$�@%��,z�Y��d�ytW��v�v�S5���T��|H������[/)7��������e����x(H6����Y�2cY�c)���?�`]�Q�Yk+V�����
,�-h����p65�_����M�-2�'ec��J�:�Z'Y4��=���B��^J����{8��(�����<5�������JZ��
���:�gJ���MB���~�G��1�{r?N)]KG=C�5����7
���b��T��5T}4�D>���c����G)�~q.;*{������r:y^u�'�.'�B��"�$>��I{��;N��
�TB�������0p)���Z�������r�}q���VY�/2>���r��C�i����j,6��q�M}�Ns�/���K@��\�?�FRb�:�'q�E��0P
���Qg�� �P�@�n :]��c�t�4w�0Jvp�>����ivD��]���h6]���x�|W�%��5P ��f���9rF]E��,a�H��u ������*Lk@,Z���xV�%\p��H���@u�X�I2�c-��XM��x�Q��K�F��8����F��X+#
+�).Ux��Rp7����'fJ���[����}G��b�[����l��p�=
�������!�L��h|4�pR��C��4�s<(QGqu����.������\��+�/O${��a"���t������*����<������������u��`�.7�=kwj �7� _�p�A��R-@��
O�T���e��U]u������G�������YU�o��{�D~6��<AfQjc�i�
dG��`�B
UE�`l��Lzjg����r1
�,!&ba{`��v�F�{<In$6��������2i��*7��\��{
��d~Ai(�����RM�0����Jt[�n}9������)�B�������X3����f]E��Kl)�"��ms`�O]��yz:�u�.�{r���^�V���<��t�t{���?n�:m���R%�����~Y���L]V+ ��(;	�W�y�9C$��&�t��"����Hq��C�I�7�
���m��HV�����p��J����6cI������M�=0�"KtT�J(�QP������e�����~�����#,9k���T
��6/���M$����%V�1�&���c9Bo^k#'y�4���)#�N�=Oc�q��~[WU����f�^���vb�t�{��gV�WT\]54��y9.k�M�F;����$%����cd2o���=��>�V_�g���K-lA�!��B�jL�o��_������4 e�)��=���*R���b��t�k�(}��_��TK�r"H�����+�����/��2� ==.r#�d�4���H�yX!U|t��L/[���Kp�j���Nm�k��68��:;���~��0@y�4��Dg%������[�����|F��������������Z�������&H�x�	Tc(����V�XC1"���}�R����B�W 7k��-i?����uh��T��
�{H��X����s9}9�����P4�*S���j+=:��3��*�E� ��3V��m�V�]4��z8�Qn�~��kv���4q6�i�g��=v��~uyu�*��s����au#���c9C[��n�8=������N�;�n�A�T���I3�=>��/�o�h��
b=6�it�I����"��P��?��T����
i����������g�����i�)M���5/�~R�Y����X��n��P������a�?���L�����2�$�W

^����4@�i���q�����'�|\���b��r�h�&z�6cloV��,&@{�����<�h�@c
��N�s��K$����)Y��&���4��Pj��1�g��k�'�/����#WS���j�� E����g�����BKt��Ic���~�_��-���(��Y9���n�D&�h,�����)>�?l�^@���)��3���������35���Q���z�-�_��v��M�!�g��((��s��[���xK���md#��)�=�[B��j��Tl������o�'�#���1��}{�3V{/.�l8��x���?G�~o��2��5_�{��04Ai���q����F��������m����F�K�����q��g:�5-�h1����#��W�P���`8�JA,X��\:D'Eu�6�T��z������!�g��sb8l�w���]�^���T�����Y6���O��W������HM�a�d��Q�a����f�M8�����*R���z���bj�"VFQe�1�H�,�k�f\\�Lu��AH|NC[.��������[y�j������2��<�V�X��BB$_�]@��.����kXATa����{��ug���9
F�_���N��i�<����x%m�����(Dr��x+'���fZod��eH!N��C��65��x�)���@/Iz����(������ij������3-oJ�#n)���
F���}�RZn�[(���q?�F��]Ox0e���0���M�6|%RA��N�����U#�T��u-���S�9��[#���������Of��Q�0�[%�O��H��m�AO0�s��X�LKp�&�(��~L�>��n�c��������W\���x�����7K>����}S����k����oc�Vc�E�]��z+B�t?3yX��7
�_)��	vx������]��7\G?r������:.��>��Bl��F^�Qt<�$7���(��t`�N�t��y^$\�q�5��}u��4��-y ��yF|���I��jx�|$�y�t�o�
�UO�Y��-���O��f���7�qlD�A�MP[���u�]e��1��9	����x����D	�%I�;j�;�d��D	BZ�
��
��v��i�,�-=�f��[��?���o�B���Z���u��n~�T�#�,�Z��q��-1������H���u�`�&"��I�N0����K���fYv#���H��$��7���`I������/�G���DT�|�U�����r.��"0t�m��T�������*T����%�����
������{�-��k@=	�@:M�1�!/�����!�T55D��V���O�&���WTDt9��R���Y:��H^1Yw�����c��j;���g��M}o��b�6'g��k��Y�_~��80c�&�����6bi8b���#�o"�{H+��|Gr"G��_a���K�v*[S�q[�N+�B�����ny]�2o��./>������p��.���"��J(d�^��I�E�SX'�~�Y�Y�jE��4���t�Q���I�W�R�Z����L�<�6F��q#27n��M��U`��r�X�	��%�-�	���L�(8ks�&��Y%q��9���1�9k�#D`���>6Jh����/���d3��e��=�o��(���P����ERi�+��xI��+���}@�;�L��WvK����	��
7>�?�gK�h�5�%��*{��� E��Nt�w�K�.!,�cL�.+"zf$����e*s��u�Z�U8��m���g��\D\��2���_�i���{����
��@����xk��L#/��#��$�}m����w��A��� �B]�}�+�����'��x��������m|p��f�����K!m+����������<���Z�����enF���NkJ�J��kL�<a��J���X2,���PX�����v��$���*U��KI���e�Y���6cO�+o�a/�������@��|9�su8��w���b���U�n�`�q����f����(^?Q��������<6�A�M;���"�
�/�� �M�����GE�V$�n:��G<dX�n����RX�U�l8x9�F��5|���P)l�z��}ju���V�i�X����Jm"B��k���Bio����'>o���m**����d����N;G��p�����^��5'i�C�|�%������_����."�	%�'xr��vL�\;�5�s���(�����Oj��)(�;7e-���^CL"C����
�}y��0����R�
<6zO�VbAS�G�-��/��Mn�k@���'u��������A�p�������w�A��*~�y��{��R
���Jd�C�Lx��������m[��#�`C��	DW
o����t�l�<%���$��i4��R�3�CI7�F��0��X��r�^��eQM"L������g�u2I����W��m:���2pt�?��z�-��-��S��Ln��(>��&.�]�(�6H��&��M�����"�[^E��=�k��n_7�������08���.��H/����u�G�A��r�/�!C]�R���C��\3��f����^W��:.��_6�B��h��i�`�"���s���"��z�$I��E���Z\�������VB^�O�jX�Y-4�Lz� c|��-�4���'G#�U�t���F����<�m*��#2D����Y���7EE�LQ����)����C��������-��g73N�[`^�E��^�H���K����	&�?I�
����D�X�M���C���@Y��P�����Ut�T+U�W��o
����pz�g����~��)���aGX��������v�1GYb�K�>�Qq0�g�=�w���#�cHc���f�m�
Z������ +�d	����b����KWQ��bX3\��X
���L9�G�mL��,�U$��`!~I}2Y���hB�D�"�W�X�]�je����j���Z�Yz�7�����)���
5%_�\�b"���[�z�P������]r��vJ�~�9ju{G��ZX3�s��+���2�S|S�gT�82n��s��1�~�����g<V��c����>��l���������(���<'��z<���?��!g��G�y_���8^;rq�?3�Tu�kP�L��[����@t�-$�#������26��[���u������U�u��1f�ue-,	P���j�e�{#|�l��������h�RU���|��+~c�[�I{AaP��G�*7�Q-�0wO�j��<F��Z� ��Z��V�|�W9D��)#�]�	�6/����E���q���RTT�:Z39���5m�������,�a�	�����r�����4e@!�$��^��G�������!�}�>������9�7�%3V�{cY�7%'�X+#�Pb�U�������/<�r��<�{��6��"ML�����!^�o�6�V6[R�����x���s5�4\��������S&l"J���G�b���p}A�����(�k�/������}:���^���r���"�\v(s���;*PHp�������ZZe2�����TyU.x�b��Q_��E��	�&��Z'����.�/����N�+E	o1�'TiE^3�8M�Y�MFguCWE�7eV,�&�M�I-KS�:g7�PdV&h���3�q���Sf�x	zV�����A6�Z��u����r��;�6��dFjJ�+���JY��L�����H1�"�Na/���>�:7�Cm.��5=�������qWW�K/e^i�u�|�+���

`a9�WL.u�Q���[�lX���+�'�i��>���j��!��\��g�x���X������"��5�my�F?���4Z{�"�J
w���p��GP5��Z����k�0**�����*-�8�)"�'�9����c��5BH���M���,�r�.�����p�j���(3��5�����L)y�f&�&|��W�E������vI��<���X���>����#�B�&139�C��`���(!�
%^����G(t�@���a�u�v�O���a�n��~��x������b�o��7v7���mB�?2�F��s�1��?K�t�$r�&�*��ye���rc'6�J;�)j����	Y��!U����=�����M,�G\��NZ�}�s�9l�z�/�M����Hx��8�nD-�8���iDtn���@�KT"����a��
�m���Q���&d�-)
>74��,3��P9dWqa�8�]u\W/<�^�f������q����R��DH���Rs�[�n����������$+���8��akZ������j<iJ:n�1&V�n�[k���Y/�w�O�P����L�f���g��!����e�	DE��{��>t
	��,W�z�����6����;��>HF]U �b��&a<��$,����;���K`��I��|��:��$��~z�}��T��%.��Wq������!D���@M`I�[J{!�����:�[�da������4����~0���a���L��	�t���
^�������Q�~1�a�^�����,�	�u�Z_'7�i����f�g�A�gWo~zvpx�u �OE
5]�h�6����o�'=������y�\�l.`��n�lp�y+�f���6�2�$��_P�C�<�u�WZ����\u$��i������G������"2�l����Gj��Q�5-1��C����l����]�av*���W���j�������d��g�����^�f�'=1�/���������J>���.Q4�bT�������Z
\ [F���u+�����p9���c�x��w['��n�HP�<@|���b����9l��W�'Sc�Zd����,�������J�b��-HI���6SL��o0Y��R��V��Z3�YKl�4+N>P�B��Q��cBS�,>x+d��'�g����8�}�F���5��ZSY�,�G��;��N�=P+���txK�������rE��R�I�
�O�5�P[�=����h)9��������d�[���,�D*�\1����9�
\�cl�s��|������_�,k���{�������4���������k���5d��� -�=���y��;�� ]`:gNt���ya<R�R��u�:�����'�=���U�x���#�E������B�p�x� �l��G�;[�-�imm6�H�����������7����|�7���Q��B�(T��;�m
�����:'lx�u�z<hH�����R�s*�����7����
=m��mP4h����� �;����O��)�����u��8����0��A�{:����������GnA[#WeS�A�`�����q�#��OnZ�z��A������~`�l;� >v�����IW��	@g��(I��5��^��2�`�j�Yb��.>.��Y{H9��R!�I{�b���&�~���6������1/MU#�n<���O6����gr�uRc9�,gE�l�J%�SLrG1�}J&K��YD	SE���E��L�r���t4��df��N�DB���CVPo.cJ>h���ad3^]� mjVHb�$�Z\'O����b'J�9	�/�����O����B��hUB�
�D��������j����-��?������Z��yY9��Y+���k�R?�v����n�O���^�U��J�eN�,�u]�\I�t���O������"}7k��
V�(�K*�_b�d�����G�����z��������W�-��.��G�����7?�__\�|��7[t����|�|t1��)B����9�<��?�����Tv"S�AJK�t�����O���%*��D��~�W��',�{x$�l�����]9(�K�-�"�s��;����z[���f���F��_�8�������������7��/�.��^��o�?��1wZ���W���]�����{{����l)^h�67|Z���#;�cS��;�L����_mh|5����kV�v��]��~���Y�����/�f�U�E�E�;��+2$��b����YZ�j]�|x�������]\�������k\,U�7&����0X�*��@��"M1"��F�J/B�j�-�g���[!g�wB/{��qr8j�N���dP����-����C|@>��Y�PP�BT)�^T�:��)]�u��e������'n�0�7�e��C.���Y@�����}-�8���������{�x�nu+�4�j��U��t��j
A4sE���d��'��327c���V�d���D�n[�g��F�2SK4��tH�	���0�91Fp�t�Z���k���S5Z��UR���^N��NQ�`�0	�V&�|��_B���X����V����B<�D�1i�.��M�:N��e^$&�y�{���]�C�N����g�4�
��o�d��&79���)��+}�t8�K��������% r��
�e3*��j�J�E0��a�aN ��X7x�6[��D�E:�|Kz��)���d{P�M�8K1\��N�~`�a�&�����&�iVP�5.�`(��a���=��;]��b���s��.^��QT;O�c������p�"�L�|���;��<{~q�~m������0Dt�����3B�o��I��	�cC]~��d����Hr��v���
���p6Y����#C�2au�Z��}rA8�sO������f���o�^�i�����U��9�$�q&u)��<CX�$����0�X)G������U=ht[���v�`��s�!v����B3�����WW�����q�/d
|o���%V�ss
.)��S�������ce�����]��!��>�tG�����\��O-�
�4��F�����:s��V.�u�OkT}\ �1B�u���I����g��d�j+�^�)��-1}&��y|D�y�����y�5. F�]T9OD��f���g��r���(P��WN�-�|<X���@��|��F�G&=��*8=�!�IbojC����&���!�����2^A�,�+(�8R!f���Ryi���)��p*��q����+>��t�H��))C��#��&9��k#�=3AP��+���]8�gS�EP,��E��i��~�|aH�8a��*m#�b��(��		���q�cr�\�,xt�$U~�c�%[�|cg�6�����h/c�`��
R���&�	M�]�V{�.���^p7���b-�YC�Ke�JG8�0P��'����>���Irs�����
��s�V����nP�\P���?sR����k-UD3�p�?Kj�3P����"�~���=>�	���r9���RztV/���
���~�k��F�����k�G�t|pr�_/�U T��J�(3�<�L�:z!���������wTY� :��������/�r�v���E���?/�:��NNp�����D\���p�"��(�T���V�	7�8JW$@.���cLEt*�$%�VE+�9��R���{�j�_	�b�u���[��� h�AD#j!�����xG��������Wg����*����m��M��>�g�)g��2hKx���������������z
W��7���+T)��y��n���G��i�n�����MN�d��r������TG��%�cXP�k9��x��S��$W�9��S��D���p���G�0���@Gp7`3���<�,��`��J�G~��L����8c������<����<8���������
��DA���''�C4t�`�����������=���9����ttY}��U���%6ml������k��
s�M���9��s�e��K��K@��������/�/�J��pT���B2z���H2������MIF\�]mC��Jns�_I0a��j��$�}�������B*���v(��a�2�=���?�3���S'��C@��b�G|,�7^m�>���x$��sR�o�����K�)cf6F����������?�x��'�G�`y���D�`e��+����C�R!�U��x�[���
��n���v��PG�dv�����o?��W&S��n�7��-e�^��u�;��+�'~C:`:m�j�����p#�����?�n+�.�nzt�������w���Sg%��Z�-F��	��g����>G����M���Xt�������w}U�������(�=0��A-��I�&}l�2{u�����W��rl|�Q6�z$�������Q�n����������D4nBw�1b���;(�lhM����r{��pw����u-@K*�����z@�zKiT$�������|Q	a��
�����P�:���B��a����k�V����w5gk;�W�7h?��ai{���FE��A�5�?�����������goc��,n�	QI��l���U����]��b���j��;���rI)�c�s�$���l��KA�Y����D���<\$Q�y��)q�\Og9S�yT{cq��X�%g�J�GYS�2'��?&���)'=���;W�+T(������q�*��%��y�,�3j�!�����,�+�"��P����dp|�Dzr�����jby���;�U��z�Q�jQ�"4�l�1,4��X[��E�������	��5��o��eUv�~J2�+�j�ZM6�s4a��F)> '	�1�����%g!�@w�3�����1��/���J�G���0�p�c��������"t�`���s4��w�[LnnL��;:h�w� h�.#��P�wZ�i���w3X�I��c���o���;������n�<������vcG9�y;�o�N�.�[��*������Z��V��n^�����-��-�����65����X��<��Q �G�X�M`<)���LU<8��������x��U��d0�S*�s��P��n^��?��tj+t,�Q<^0�a%�j�va�����T�H	�5A��$���Q�"�[�U��������>�Iq��V�]P��LJdQb_��b&������C
h��������F��PK��J�%VgEPK�nJGpatches/0010-Adjust-join-related-to-code-to-accept-child-relation.patchUX���X���X��ZiS�X��~��/	[^���j��J�B�
�R55���'� K��8t���9����+$���m����z�"�N�:�uG������a_����a��c�@<wx|BWqD�*���z�S���^�z
2�t�M�<����I�����������\�Id���n<{i����)��M#z�& ���i������{���u]L>+7?��|<��xC�^�?�/�{��,��qQ�BP�(���=�����$'w����Qf[��� ���vq�l�b�E�sR�����N�lS�M7� � �;��(w����A\d����w�Nt~�N���6�_T'��K�����C��8�L����h���B�� �J�M�i6�4�\)(`�����&*��7D AX`Q�1�r����i\$�nKKN~�L�T�e���oNJ>����Ar"��T��B��Tn�+V)���=���8���;s8���(�9� 4K�\�1�D�s���k;a�,��92}���A��\`q;�V��FJ��/�C�)�4e�~�,���5.8����7���D���R��q�����W�����<��>�Kr��i!d�S����Cg��8,f����A��N��6��Pv���Y��Y�PX�?�R8���L�'H��������/-Z�k�0Y��A�M��%���.�i�d��`D���a:�e��+68��#�k��w���������H;A�����F1��������K���L;|���$U`D��8��,�M��������J�T�V�r���*UF��9K�y����=1�k[�b.3$�2SM��M�b�<�-�"�[����0����#���7�'�b���7��U�g�xCR�I���r56��SO��V��B@��3�
�@�KU�O7�Zm�<������'i�a����bnX�@Y����u�t�8�8�J(��h������*M�����ZB����fM�	l����2���Q���]p�E�]�d���>@K��n>e���u���8������B��U'��<~����w���\�"'��It��g������6�9Y��������!���-�"|���r���a,He��_���{]J�X{;�W�+9D�������@$�L�������)�[�[?s'�Y�R����rG���<,`$�^jGd��\�^�%����Fo���;(�>"��*���O��K���$Q�7f������^�o��s����@�HM���X`�d�j�m��pY'(��c�#.���z���fA������.���c7�\t�`!�t��(���F~�1����k�*�]t�(�Q�RI9���s[V�r�^�
�*�@J�4�B�y\mpDesIm�%���>r�W����i/���-1�\��E�0���"2B�6k���,����w�@��T�R	hsM�t)���\��%vZ�,��;��6:�:��g�,��:Z��5"���D*b�>��C�3P�!l��J�_�1hg����v��������8:��v)x��9.G�MR�G\#�eC�	`���a�\����,��Pq�n�*�"C�U�I���@��W���tn7�.���� 	k�sk&�7�#Y")���2_�v%���uD	D�*���.�5Xfe���'�~����t,�R�;q�; X7N��m��]�f�KD���>�/�u�6��w��&�BopLki-R)� ����w�PKh������`���j������'��e������	��r���P��>u:��aN�q��<n���o4RG������W��������!���'Z���O�����z�>L�����_f)�a�:d��WT��hg��Gw��������
%K��rd�X4zv�/���$��n*��IP�$i�e�(�����/�n���&�s3a0�[FMd���i������9���"t�2���;U��������9rsXpi�B����� 5��T��Kv������0q/�Ru��^	����������=����8]V]p����J�.��P���P2�1�K0��Tn,�����e�c�L����YZ�m����S�?2�.����H������&3������M�
����fJ���������Bc�-R�7�/8U�OOO�h&��,��;��'3i_�v��W%pW��ZS�
�AD���t��B{���/q�r����Q��A��M"��E�����F,��xb�G2����#���b�T���5~!��Pq��k�5si6�"4�������;m��w���+���<O�]AY $���{-ZKvq-�qD�Y���>����B�`�
ZT��������� ZTF|3��I���21���E��N_���xh�^��Lq/(��1H���U��3�-c�$(z������J�����������O��f?�
��L2�F����!�{R5�\1��� ��3R�u,���0��c�=�u��:O�`��A���!X���S!X�	��IQ��
8�M
\����
9�����[��ed�<j;Am�;���j�j�����l�L@Zy�cF���h��'�;�*�W���e��JS�j2��m�(����G��+���+���~{D��q,H���t9"��2x�B��pJ�I[��
_1�*����hc��u�6FJ���]Q��rK��uo���0�=�a}��'�1�n�]��-��MC��p�~2�b`g�O�v�<'t�03���Y�t��Z��ir�o�o��E�p���������tDO0�����2�=+��/�B���T{;/y9�!�4-����W����o�����@�����`�_8������i"mz���wZ}��
�
\Y<f��}�$�O�j��j���
q0B�pD��<������f����&�����+q�r)HX������!��Z��"��e����z��n�������2��w,m?���UIf�\yX���f2	�	�$�������2������}��t&D��m�/�K����_�:^wv���\��j�-����`e�����f����i� �F�^4�d������e~D���*s����k�^u���6P���D��g��aZ���1��@��d���Y5�8�F�hg������w���y�,��m5���U�4��A��<�7���D��e�_p�gr����[���H���6��q�9X���9���^?<��f�&.�6	�!`����������rl���=36��d���������-c7Y��z��q�	g�\a&��#��3������w�JK'V}������O��ic�-�x�!��+*�+������-��������G#�K�w� V���6������~�D<��=���2����S���NG����{D"-��:.��QXm�nO�����}�JE����������SE�<���U/��[[�2s��r���`�m��`��T_[�g�(�D��`�f���u�\���tD�<�������0/�I��o��������eI������Ju�l�jj�n�.��u!�P���V�-\��0SK��5�����}^5yp��������&��x��o���Ym��.���pp�&�O�������/jtf�2�@)[���q����)���N�BY]�z���Q�F=fd���Uo���	�������<��?�<;�O��>|��:���W]�<v5����m^����������+|�#��
T��z���p����G��=���PK:�2Hs)-PK�nJ+patches/0011-Parameterized-path-fixes.patchUX���X���X��[ys�������IU���E��e�����Y�V��W,��A��aYY����`9Q�,
�������=��I������-�3�d�\������������[���S��.=_��#�Q���\L�����O�3�%�y$.�m���V����L��������g2�d�O�T�����:���|$�s9���\3`=;4?�b8=]L�������G�?\^?%f������n��$p�%}��b|���q~���Eg�K$L!v2��"N�D�� �����Hd[)�(�	s	R�m�v�wD��@��/���I�NL1�,��&Nv�3�L��E##'���]�f8
���D>����d2�R����Z}���|�����o���>-"�n�4��j17aEA(�[��_~1A����q�eE�s�����k���c$�9U�N��`�*a9A&���`�X�Cj�nE����`������zs���\��z$n���Ho�+"��8�;��y����q
R��z��M�h��)��)7�%�����W�o�_��~J���A��Qx'By��v�w��}���-�uS���a�,@��w4�F&0��FF�L��� ��7
l��"���T������I�"
��a�4
���7JQ�����u}����Y��;�����>�)����nA��@cW`I#���*F�hl�����G<G�U0�\,--<�eUF`��������L�Fm���E$���B���nE���!}�&7����l�U*�>:?��:�9���0��pv�D��=�)W���>��\�:&�+��?���W�7��f[�Z��� MFz��!,�G��������$l���M�L�u�`?�F�w��y��qAj$�����w�qU�M�+^�3��� a�;���3��&����>��?��Y���i�q�h�<!�	!1���Q0#��G�/�F��i����,i�(���l>U��9�����b.���a���j�x+����a�XcA\���3��Pb�v�	6?����2�������e(��x�8�D6b<�	 ~w��u7:�D~��|	�e2Y��O����M�g''h]gt@��g}�L���������x���G�U	 �7�t��a���8D?q}�,5]e�
9�F�]�^�����0O��Uv��-����oo�<v�3��/�g���>�M�g ��9��#g(�(b���e!:�|�����]=�qQ�f����w��d=������2��?�I�w�*�{�IW/��z�7��L?������O!��x)*�m���V_�yY���a�������5=�/��o�i�
&�!�Pd�������@raO�Y ����%q��`�Z�W2|����2X�5��8�,C�F���rt&���r4?���1qb�<0�_`��X~������tt�?���Zo��s �<t��~�)��r#Bh�
�`�����x)X9�cl�
���0M!��n��@��,e�9��a. W�N�f���4	�3�z}L\k�p����*u�6�{� ��(Fd��!bV��!;!x��8��66��!����Y� ��
����
��kB�z��~�;���t�&�	�����J:�q
9�0����a���2�[X?�9��f���~�\x
�c]�}�`Z�O��"�Y�D0Y�;M�*P������*N�DsA�#�kliDn�Hl(cf�j~��A���d��&�<��06�oW	�U�m���D��R���4��'��q`��I���d5���_��G���y��o�tB7U��
4*�
��zS9��M&�l>?[�:��*���J�A|6]����aW��@Iq�����'p���eI���@'}&B�Y$4��Q���#�,V���N1[A!e�����%}���$�������X�����V�����Gzj��T���%"�{�������<Am����VP�2��@z ���<7�WI�������]����S����`����P~�a����'b:p�0�z�R�45�GZ��SX�^����U���F�`!�:����������j���E���������
��M�j2T��xK.f����?<|S�!/����5�����y*G����cf�Veq����D���p����)��()������6>�-6n�)����H\��ene�,u������Ah�����E��6���a�B:�"n�����Q��5^�����r�u����%nd�,��C<P�$��v��$�p#�����jm�B��0�f���������a�R��s%��F��*Rw7�Smp|�Q��1��'>�inV3�cb��t"���
�?���"�K-�k�;�(Q�eUB������6��� ��'�"�YM�a� �Qo�^ 8�D���8:���-����2�zh������*���"Ww��:u�7�U�
�$`����=*���R��e�p�pBU�T��*�#9o��X���`..��F����}� aK��P�y[�=��u
@���p2���''MtC8A_�r�ha�N�B	�P�K��2�+��;e���"���f*����lQ� �D���XO�z�6�b�2�R�s�d]ZFa�XlN��5L��~���Og������	5y�%��Q������{��0*Z��l3|�S�%�|����q6�L�����^|����{XX����
�-Eip1k^�
�)~����t��6~�
�g�Xm2�����=��l�#aS�*�x��0X�L5~�)0�"oRf���xK��,L=�1P����9��k���
Z�4�X��Y�Q����M:S�����,:��uR�EX���+L�Q5�;4\�gnm}�
�n��N}w3�LfS��6�.��*�C��*-u�O�g#��{>����	l
���n W���V�C��S�>���D�����������mqWq�w��3�ZU�����y��� ����275�rWs�g�����,
������aM��w��d/U(8�����b��Q�I�N�[����%�I�0���~q� |�D��8*��/K���X�HW�y�!F�������$/��|�F�8���#}��������=:,b�y�T�������`���cz�����>h|��@�C��+q��)>f��Hen��m���'�F5����S�7�E4��@!g����ei��D��b�����o����?},��r
���,���/��1"]�7��8���B�zE�Lh��@t���Xk���V�B=4�Hy��D�(�����I�����n���E
�&��:	P���J,E1��%u��?X=��q��1?>X�bT'1J�v9�<� ��N����p}�J���c�$ReH�X������G�3�1`m��O��L��f�PZ�t
�4-������/7.����c�I�i7��|���LgVd\\/��C�G� ���%U���C�^\�,��uW���g�+O�oR$��9^�1�U� ����{�t���
�I�1��n������D:���4 .c{���<������DuUIb����gu�K>T��$4�KVjW�&�ve&����,���D60��]���%��Q��D����>|x��������:'!��I��$�uU�h�����K ��T[+��q�� ���V�n����H�J�"-���s�c[8��SQU�F��Q�ld���Y���r�������,6�U�����e$������'"��Fm
���S���!BNXa6��@��YM6�M� �1,?@�s�rl*���� -o]���"	3�%`�����r�{t�������9���@^�����MWIL|g�U1�=��q���=��JAO�#�*�����F�����$49u
?h9�`���8>=��b�|Bb��:�s�S��5��
U"�<I��w���$�����/p�(t�;b����t��;�	1[�P6���y���c��I�����(�����Q_V�t���j���NUm�-�^�E�Sw��b�z>kf&D�&�Z�cHt]4?��#��,D��:�T�6�j��*3���X�n�%-K �m�S+��Cz��
fja�@Y�=
����^�K4*{��X�N��|X_������#�lcR�l[���%�Zh��w��u����E���aO���o�wx���@e�S4Z����3��,��K��}���O�	�4�a�V��"%H�?>>��J=�t\<G�V���9n���j��G��1��3w���um����j�R���W�h]����2a���U�X!����j}\���Z|���Z���W���aV�P�A�Z�w>SH���l�d�T��r6�L��z}q��D�dt��BI��	��������MFj�[lt�:?����W�w��o�Q0�7�f����E 1_��F�c�P��m���t
����kS���L�����2��@��
;,.TjKF[�E����������X��*rD�
"���������(��X���|2q�'��e���A&��$��U���d����C�g	�� ����/��p����O�)��~���s����|���yxr�|j����.�����V����^![��m�f�2�(�Hd��f�M�_�u�C���c~`�V�v�F�'aa�%,8������G4����b�
�Q��_�:��G���l1�L��������c2��N�$h1g�h0���a/�H��������:R�j�W�_}1.�{	��Tr�wF������\Y_���{u��z���6��M�'�����PK��x���CPK�nJ<patches/0012-Use-IS_JOIN_REL-instead-of-RELOPT_JOINREL.patchUX���X���X��Xmo�8�l�����K�d'��k�4�oS$#N���([�,y%������fH����N���aE��g�ya�y6����������vpt���]��q�os7�����6��:Ka���t��O�pm�1������Bd���9�O��-����T�|��}+�fg�&�	|�az0�>*C�N����8Gph�mc����@��������q[���S��r�}����w;��7 N�YY8q3����O�~�ArX��Z�A0��=,�\Ly&�W�B�&b�s\���,N!��4�X.bZ3�x�V@�s�*�����������`
,�0c�=��6�����mi��,�����\,r<B�Q����)�TL3<�>N��i�|���?� /D�wlj��0��,NX��[
�a����8��V����������X�T�R��h�� �Y�(�-�4M=��<�[����^.[!����V���ul8��|~[��V��mE�|��4l��<���X&��1�=�!�$N0e����t�m<�W�6p���'\M�
��(���Xk�
�=�� t�~�1�b��uB��>�t��gx����{f�q�]8TN��1�.c1�,��I}����ee�/�&���<�y��&���\��4
���; �a�+� �h'�`>���A�s�������9����K~��v^�_��b���l.�j���vT|�f�����M
Z�Z
I���������O�Q�`���H	7N�[��t�����~c���&������cB�������y�(C�����Cx�p�]��������jpqo�(@����m/�vT�^�.A��o������IY��U��d�����&�Db��h:v[���5�T�����K�����%1�9��f��s���/d'-H�}����&�Vo�=��?�A���T���������[+�[�I��^�T `x{s
A��%�N�'QW���,@��)]�I��n����>�>��
�h�$?��PraE��������,X��dV�I���A�>�8�Ld{i��~)~	�4I�w~uy>�F�����M�$�.��w����+8�W����$��L��&l���d.o5Zf���O,6��Z�0v}_�5z}�h�N��Ui<2M�h�e�;q��)��"*Rp���3����R���n�����vz��=~A�����oIM�n���
="���,Z$P0������E�����ZO��ek����M2���Wd���K��I�����<�Z*������)I�DA�CaOO���<uP]�N�J�(}�T�����W���ar����*$a[t���_�Q���V�Z��_�z*�4���d�H��%2'���$��,^W�X%]�1���e�I��\AB����]Z����"_����p�����I?h�,��������i�2c��k����2mD\S����
��g@�VEn���l�J�N���W*}��l����	��}�{�J�q���K/��N1E,!��<��j�1���#Bh/�����e1G�_P�!���_�A�,f�<������$��/l�����%��~z��^�)�*��<������S_��<]��G�nt����9._l�tH!X.s)X���P�K��Z�H��@��s����*��X���
�L�|����{�����������G�����e�^��a�Y�wkP�~��,���K5��������"�#��	Q�t�i�F��<<I0>�@uk�A��x���'uy��a���z4a���J�W�B^�'�[5��B��������AQ>�s��nlhH�hI������AP���`���G�	�*+��C����R���W��i_� j�?C*����
s^)F��X]�o��PK��eKPK�nJ:patches/0013-Multi-level-partitioned-table-expansion.patchUX���X���X��Y[s��~�_���,� ���+�/W��c�:��j��XH���,��������1����\zz����g����V�3�n��z����l�������5��f��7��/IO<���c���[�F1�p.�C��>��R��3���.���c�E*B��n�Og�K��1\��n�y��l�y�_?v[�W�o��O��_�W����y��������a�B'�_y)*Ta��F����[���EK%���0�s*��!L��1$=`KD��E�4�>�(NfV�A$){�m@�_�E�W��N/F�!HF��������8E���u�������*��pQK#��8T���>|�
}V�<�_��vof�Y�g�1���&���R����y�����P�������oB/�����gT�����\��ei�\��!<���)�A����<�
XmC3t7X���n���F�Zm`H��
��A�E��x�"nt�u�	��f�r{��m���c�x6L<
������b.n�>�$Q�"����G���tt��)P�s&�Aq�<&��&���E�P{P2�O����[����9�/T��?������C�����
��������'�
�!?)��{�}�'�A���!����<v� ���`�P���P*H3�X9�<d��8�E���1�����L@XB:!���� �F6W��g���.��Pr=+��1J����B�1�a�~N��*_C�i�^G��;P�0������p	M!��a�������$��x�-zB�
b\*���,D��q[VF��2h�Q2��/��hM�OdnO��g��0A����F� ���E�g*2v�����_)�����F^����N1��w8��i�:gh��F0�i+�87�R@�6M�Z������F0/j-�&
h�o|�
���!B��3G�0V����d��$��S.+0y�H�H'Y�vF��NLP1pi$������LR�Y$�T/���}J](�I
ph�f*&0���}M�`u�.pX)	t��
�F�+DOZ���\`o�
��������D��2H>����p��S��1����5Y7!d���fh�Iw�GK������8�h/�M�M��Z-���4���T��
1Z@�L�q��F&���s�3"2�oC.�zG|!�3��oY��@�^�4�X#K"I=q��<!^@D^�#(�D�	H��Ze2$���1\�QA�}%���b%�������\OazH|Z>�&"|c��o�GM���"OU����6��Z�����_��8�c�)�(��,X1lo���Qs��@k8$0�0��B>���#���M3�B����\k�^(0HEd�>��B�����1}A�(�D�����r��)�&��C��x�d_�a9��W��b!�j�d>Q�Z������1�N`R��)d1&~IFX��
���x��&��$�(8U2�<-�2�T�d	}��)3��������04����leiU�S��k�(���4L9&_n3�����;���r�:61O�:c���JB�^}�����[�
���p8��P0U�T�0���-{}��?\�w�jO��?^���$
V��G�H�	$�$8�&]����E����`]+gnh�\2n����8a������)J
fj8����I)C�{�m�+Q�R���c�g���$?_yi��`��r�L��@xa���s�HFT�X6:2�]��O'D��eS�f D����Rx��!QF,�,J��C�"e�%f�sF�Q6//[c��L���9�|C��������������Mr�����$wxDoLrD�H����"�����(2�Q��2�	��lY-�$�6����;��\�Q������nY,V�@9��4^��o�vsIM9��!�B���d�$}�zB��{D�r�
��~9_x`�a���Yi���m�1
��F�����f�gyIg#�}i�&$#j��4��������tD���]$#gW���M*rv����$"]$8�aI��+
�%W��6����u�<J��������L6������F�I�$��������(]�Sl�}���\H����*�Yg*�6�Y�bR��1�:�U�����;�.�0�S��Z�H�f����_o�.�����M���������|{E��W��9��6]SQi������'��T��No/�2(9��.n�s�ZG�4���pp@�<m��b�Ow��EA�������;�9S�ur�8{��~�g��P?�i�� C�J��Ft�(�S�g�8��&�����e�f����4�WC��	w�F�L��|���%����R����i}��%�4G(��S��zE�Xv��Z�G3S��h�w�OWn��V����g�_��$��_w��L��5j��FQ�-�f�
���9�x�%&u��&�(JY$�=%�����@�������_{CA�Y���f��]�$���������p������#�����GC=x�b@����KL����oC���w7�z��=�������@&�lk��f[�n��^i�>h�O���{ZBa�5T��y:%3X�sQ��,�|�W���]B�Z����,�On���Y4�����Yf3������s�4S$�4�_eN5\�6����f����a{?h����v���j,^��.izg��XE�R���0�&�"��K~"e�^�r$��UHP���_���`O����<��M�D�]�t��u���z�Y�[�k���\���������D^?��{��[=�����G�[��q���v	3b������]���x���QdB��~����}�.�����.�G��x�0��0�3�1�����n�cmW~%7�A�n/c����n�H�FyU�+8�{�&y0m�����T����X�O�Yp�.���+fY� ��4�"��w��a%�FSwSW�
�g�b%g��,���%���Sl����o��v,�j��@���VD��x;�w��5���&V���6��4��F	+����n�A �����Zej
c�g��7Opw�1W
���pK���B������[���a��a��DO���#l��z�^�Nw��B�Q���^�Nw����R����d���p]t���vu�X�?PK�b��%�-PK�nJ8patches/0014-Multi-level-partition-wise-join-tests.patchUX���X���X��=ks�8���_q�nm&@@�����j���do:�f��nmQ���~$�U��W�m�@��G:�n\)���y[:�����{�#m��^����%�4X��Z_��6\����G:���p�w�B�w���z��%sc��s-��h;����������[�;�p�j�����5�i.��K�hC��xI����E��E�=U�5n���Xw/��?��o"������g�F�����f��kX��#�[�\��N���t:
�v�g�������{�s�z.�=�S��\�?�_
���5���3:���76v�3G3M�����31��?������mC������O���3�ry�y��|k���w�v�Wm���%�~N��E�+lbz������
��&r���H2WX>Y���X����.hge��e���-~d����0�+�����1<Q��~8�z�� r�2������{p�oS}��qJ�O�����s�agj[b���yt}A�/���#66w.�����������*���
��&����7���naBS#���a��tx�l�jv~��W���
��
f��'M���b��5�^��/�3�u|���.g�����(�M�����$
��R{=����C���t�#C��c��=�\.��VW������q��;�d)B�%c 5C���<�ju���v2���
9�{;�������?�ZP%m�O.�S�%�d�6x�m���Ec��k�:��� ��7����byp�p�s��	��[+�s|���|�nz�;�W[�E����
���	������v�����;]��{��F:f�����u��nJj^���m���}~��2*�<�����w��<��J:$t���@B�!�+�<�ypY��<��	��uH��(`"��<Pu��8�v��<�y ��@|��8���z^����gcl	o���q:�����W7��u2{3������v~K�|��s��w�6���/�.���J�b;�����?Mf�
��z����������=���MfT���H/�.g>������7 ��H5�L�#��Z��P�z��#�.>�X����S���w;�]���'"�A�;�W��������.��dr�+��b�`������?��J��[�`�[�numKtv��4�nt+q�@~{����d�A��-h(F2�,��L%���.
���wC�nQ�n���"rH����>W��-��"!�E?%�E	#_�#1g�.-�8C�����/����������Q0,
$��B���US 1��-;v����'_�Y��K>����G����G����Qe�+��J y%`�"!y%���������'������Y!R�����
������`�A�����#���j%�)F����#/	�A$&^�p�o	�QDS���gk���w�>q��;�N�:u���������M�l��!�~�~���u`��� ��3��cA�4��H������]t@�(�q��E���o�����A!^y1fy�6�����k���'\O.���)1xz��������^����e<��L�_��TJ��eiC^��TN�����!/{q*'����p�%j�q�z\�J�^���5/��_;�T�
�G�d�Z��K�.���%�#�	J�/����l?�����@B;P��U��������k������_��if��<j�
��~�����Ku��pd�*8P�`�
b��^�g�d�����#�����)i���hj1���0�,k�����+%��yQ�
��[�v��`�RF'������9��P;(�H?��
���(��!������D*�Rqt�k*��;��@[n����N���C(;�Q�y�Eu'+�"G�T����������;��A[n���%�%�;JZw2�$_���y����-ja�a�������e��o	�T���8)RA85��j�S#�P����FMQ�T_��<���KN�3��+b��l������Bv�+��G9���}T��YL8W:�#*G��O�k�4���4y��ba����/������x_rp+��f�b�*Q[��Jb{���x�^�!���@oG�P�F��L@�g�Wq\RU�L���.�%f�1xD+b0�9nl��D7�LR���3m�*���4I,$����k�(��Jcrv�I/�+J	����E���qU��USD$��|.Hk$�[)%��������$�N5E)������E2I�I5E\5E"5-YD��+��.QSaeJR2�j*��<6�6�08�%1 �����WP%�jn�N^L/�C�T��T5=���R�rD^>{�.�%�Z�pN�
��0����b�!�|!����S���R�kh%���7)J����"u���\!-7�2�I4����)�xH�J������E��x������p�t��~����:y	]�
*���F���Gu��#}�L#�T���P2x���P�z<�������������������@��A`�5	%h��f&��ii�vo���=10J�@_��%�k��5�F��q��p��t	^q`g$��)�Lq��
�e��i��2���eC�z2��7�E�/�Z~�Y*>�x�K~�\-��[N��
{\[���#�2�
��,��(^A�����7�g��H�_��EG�w2a&kD�i��Q��C�kC"��~(9>���
�E)�)�Jt�W.�e@e4�-�t)�=-o����eXD�i����(O���m:�L/?C��ui#?�����gs.�qE�>|~%��UJ�F��s*��9�}!���x^�)_�	#;P�9-<�� �
�eX)TXj�����R��G"j������u������/Fv���d\!�B����:�����U�U�*1�B.<�����!]�\F�v��r.Z]^�AD�����7(��b��a!��H�
b����]��'�P����^J�Csh��"^�kz�X�}a�����
_�������R�D��X(��)Wb��&#Z�bo�L����"i0R�J��m%P%P%W���,9z,I��E]�u��j�#�8��+�(NqV~h���������1����i�Q���n�7��jM�P���&��|������X&BKe���T�_�?X�_��b$%�v���z������/��S}��_l�8�Or��Y}R�A�A?@ �@��3�����$cF�t��������d�8~��������e��d''��1���$D�}x�6�>�5&������{G�?�~F���W��X������[;�h�w�����+{��gt�w���(�t�?��o��a�Yk��6|���g�a��>cI�p� �(X7 _1 Q�����"��"y��wV��T��H�E2a,��
�	�J��Dk�R��"�X�8�h	]*�Z$�*�>���<��<��\����1����
��r�
�^����4�W��P���HU��
�}��|�O=�F��7���0�W,��
�'�}����T�K����(����r]G��,��Y
(�T��|���Y�RaQ����[��U/�XIQ�����Y~1�r��2J���3�O���YG�|p�Lp�9�
�����2��_#W,;ZS�@��nM9]��+Y�_0�[{���-��A�s��gkk����V��Ngg��8;[C���9fh������=�������k�+c��Ngc���9�~�b�=�����s��l�4��p�;��LK�F
c��au��Fk��]����_R�;D���j�������5t�=�=�SGN�.�4McYd�[:�m��A�b�Er4�Z��ac[��5��0HN�C��`�������,��L�������v����9d�+L�x������p����s@����n��u�f�wd�2t��CYi�����#|�7I7X{[�����m�`8��iaZ���C��?K�� �Z+c���)��E�M�?���8Q�}��Q����=�5&~ j�"��f��!y|��:LXY� k�%����A����c��k~V�� h@�v�9��`����0�$����R�W�.Vu��+L�`�����V��� �@"�F�/�����66l�'W�<������a��L��	�G[n��?�� b��V8������u��P_��T���~8�z�)�g�����v�0�=R	Ya��h�gk�{/�L�r]:���XK�/���#f�y�O�������0��C�������lBf�0����M���&�%������f����l~5�"�22���o~�4��\(4x5�^��/�3�u|���6x�N�E�n�i���DA0X�v����x�t���N������N"N��r��e��j\E�%����%K�B�(�rT���ATc��W7����n����`�b���k-��6��'�����K�
�b[s��z:�4{mPG�6���f|���B,N.p�!�9AS~k�w�A�!c�O�M/`~g8�jK���?��5Z�dm��������wPr�#w.���3|o=`�H��w�����u��nJj�
�,�x�$�c�bFE�W��}�����RI����UrH�>�pe���.K����#��@���Y��A��(b������}���@��8������q��>���$p(p�A��v����qI�O�����q��T_^�)p���im�_=}���(L>���_�Gl_*���D���x�]:����_vZ�c�G�ds���wVf��A�gJ49�-�gU��l������6�PK8s�����PK�nJ:patches/0015-Multi-level-partition-wise-join-support.patchUX���X���X��=ks�����_1����I�IQv���W���-G�&��R,��%.������y���P��2�E�LOOO������1MS;���5����\���\_�g�t�/�������u>'�\���=�����9�G��x��0����9��!����?�~d��7�	���l������^v�5��\y��L����
`O������99��q��`~O��9������o�dv:����;l{�����
/��u��<�����{�F�����En��f�u��O�u���H`�[J�����<���%0c�%l,�=�x��M=��6�8��w���p�G����=�����"����d��� ��	0�`x�5������&�f�"���"A��
����.1)�(�4v���&S6������z>�X�L�
������~��`5G�~c���X���N{�<�1>�!�`�n�}C���Oc�?�=��0b��?���{�(I}b;���8����dP{�Qw8v��Y��a��;u������)�=������"��_��������lQ)~�������o�qp`r��������e�;��f�>YL�{x����-#H������v��aEOqL��������d_��0�x�8������;�F�S`z�:e��d���|�">�P:kg��_^]� kG��p�j@������{'}P@�Y�-�O��nwe��@����;���umgE"�	]��l4Z���s�%8�O���UG��L+���7dx~6��9�[��m���
*�K�i���Hl���>�8�C�I�y�=
O�v��z��Dd�C
&@2�������'���2�l�k ����C����=M�U4"c����;�
0�uCl��i�e��w0,jg�LI��x��cM�	|�2vl�
r����t��4k������j
��?9;�"�_����f��S?X�l�B�j�!���`G|�Y��=�Q�pp|0�`k������Z�T��/���0�����y��>q\�����`��-�u�N�\H�N�=�?�Y�����w���h�x����G���F�~%��2"���l)��9]O�L���k�Z��7R�
5G�5��>�q��7]�_B�����x��W�/C�U����'��o
�n���zEm��+z�3��@��T�����d<��(0�.��z��YR�r��3�K�'T>Z�^$m��L�6���k����kT�����{���2d2��
_
�H������?�G'O��(�l��z�~�mp���
2.�������
S
���
�1��?���$<��w4#�g���]�`�N��C�����v@�z�^��H�0d!��'><����@%I�A\�(t�n�W!�1���|���	�����E��eK�x�H��q��,�L5]����;�b���t�*�| ��kxE�|u����.��!���O`�����z.�������Z���L%�1�y��W1�("W�'
�a�lbLG#�\���^�x$A��dc�3`3m�Al��L����Xw��e�<n~�wi\���L������ ���A�g�_�x���
�`l�_��>q|�����c�����HV�'�������~��������!��D�C1�������<��Ps���	��H`)c2F�*�9��x�l���d�����Ff$��@��`l��TF�,��;��V�9�\x �����b�x�2���6�ox�uN&|%�
���*�a6W��WL% '�������G����I4P��s'��Q?�{�$��������������������w��<C��lz.��4���mOb�U��@��Y$1�-��
n	wU�XH��t�E�m-a�P3'��b��} �!kP���]	m���]��F��.��6���B�}���������=P4[+4��Y�;i�V��L��b����d�&4���cyWs�Q�K�����.[?TN��4��������x,��02b\
S��X�s'^�3J��vx4��*�������2>�~q)��.�.?F�������88����n�����x�~RT5gb�������sr��������N������CPd�0�9}V��H���z��p'&cce���h���g�J�hR�7�l����������qOWtM�s�AP_m����o�W���������7�d2O�(��@�x��e��#�����U�\�Py���(:
5��;Zv�W�)
��y@��5�[� ������YH�A8��y����G)�L�y��0��Sd��c����X��!�7�`��{;��m��0�8�p��H)��{d_M.�5����l�<>~^t$	�6;��7%������S������
~G_M��xW�t3h�{���OPt��DiT.��9�����zs%_�Q����	>�|
�E8;Lf�
�~9z������f/���/�t��<u���.{�x��������ha����H3�������8&�L�#:���j��fU�C�"p>��G����������	�#o������Il2�
E)�U_��\���S�i���O��H�9�CqZ�>��%�X`@��j2�s)T��Z���	0�wo��yc�'NI8��RN["��3T�U�@���� ���dcr�z$�#���F�W����/�(����n���)����1�� �u��|��1O�Q�^��"����D��Yf2al�m�O���%�g�W����X(�N���	��af�1d���q@�F!��>��"�%Y�
�N��_�m3 b�2K�c�C����L�����L�9��@H�~������>�glC��p���,cz����y#L�	��"M,w�wC��^V�~�w�7��l��s�/��bL����>)�`SEEE�*1(kW�in&!��t5�uRF;����p���m�7���V��=��������U��� Y����	���=�l+��\��Br�d5M�KM�����z+�nLk�o���V��[,����pQ:���h#������j���16��>�G���&��%�[�������|���xI6��nYo��;b7P�����$�(���$���i���	���N�z]���/M�B��H�_���kP���>��6?�t������)���#��T�������>���M�����V��N��^��3�bI3���_x����6�xw���>�QA"Y�B����t2����Gi��K�]��a�3"��p�'�������'�����)
��1skG%�X0���'+�#8�"?�����6aFf�q'+�.F��J7��HL��v$�����^���\h�������������/o{�������G���+�"���P]��4�8���v���!Dl��&�
�K
���.��x>�`���*��\�Z���DMH������&�d��,�I��Q��(������H�)5�'lqx�{�Z����4~F�{�s�6��S.�Mm�*	i��T���R�l��5U��K���U6mW.���X�X�	,��0�>A�U�r�(����#*�y*�Tl0�N�[%�������=���]�>R��mP��Og���N��9������$Z��c��+�����P4M�����6�`UR��{d	J$_����l1���%(����G1D���#r����4 E9p�v������QP
�����XiG(b~�.�D�I�sg��-VH�Y�Yu&|Y�O�DV-*>b@��p���S�����(��]e�U�*�0P1h�r axEi`���z\���/�y�x&b��Q\������
�^x#�/������]���T�
����kjB�'%1Qm�k�H��
3�a�2��T\q=�l&�`�B��z4:[�zzV�&T^6��x�����WD�},Q��O��8�Fl�sGR�t��!�	:!��-�9F�
#��rX:o�[wlt�\`�4"K��/��N1�+�+s�o�>u�`�	��|����A������e4�p������I��y���� �X�<c�����
0��4����aE����?�
g�[������M��=���s@e��C���`4�H��K����B��a�3k�'����Z�����!G"z�>`���x�8C9b�Us��H�����ZA
��k[ �tD�\F���S0
1�F"C����z!{���_���D�H������������cU(O��)�����G����>��	��ed�y����"p���"�5����5��Z��#2�1��W^M����td��_��ux�`2�g��h��|e�ub.�H�6a�v��X����
�h���n�M*3�O����N��:}��X��7���&\�H�W�:�����,Fn��Z��J�I9V����`Y���AyiM���y.�;H��V��K`�_�y�WB���,~/8c���n�F�����3N��CH�F�
��\C��O��)�~�O'G�3":#�uJ9�IA����������U�I�����(���M�4���5�����������v���������oE���r����p0`�x(��<?`V���f������9�g����\[��8�y��GL��;+ifuD�n_�^^����xu}I{/��[rqG�.�/_���q{�����p�=�-g�{�%�������{���������y}swGn��@��dd�,B��L�i��'0`0�1������K�
s)��\�������7�m�����~{yK^�M������vD���w��#0����w�OquO*
^]���|��xp��������!x����	�%�Cy#�?5�@y}H�\��&��%`C$w����?�[�q���$�'���-l~eoA�='�^��}d������,��)��\�!��8���T7�������-���3��V��s<���k�x�����OC�i�~4G���.�Q%F���%��1�b�+J	g|��E��zewne��AW^�"�����#�u,Xo5�����x�����X%�t��T�4�>Zl}���h�I���)��"��K�B��R�1���@4��&*�X�JG
[U�F��T��U�����r��Wl����F�,[�uMnK��4����u1�.8L��g�$I��%o�1��������R"c��N������UZD],��H��3A�Y�/���������O;/�fg�+��D���~�����
_U�������a|
���Y�������\_^�<6�P��}!����U0�����5g������t� ���E��p��~�_��8U~�A��^_B��DhYM�����`�������_����1Mr[
�'�\�X��8�l6��*��F����)�95m�1�����
���Y��1�n=��}��7
����;I�P��"�!h#����b-�����nj���
��Pwi"*�������/EeINO���e^t54Ts����;a_P�1�$��8f�fP��,����g��]D��.�#���6�q�S[�|j��@�����������^���J1��-��g?���'WU����VUK�jad����^Q����u��(��6�XUP�:9�)O���"�l]��g���I��e3�S�y�[g��:'1lI
���k����,"O!����D���9����Q=��z�GS���`[�Z���X7"Y�r�b���)��e�����-�,a7{�E��7��4�$��>���"�4�%V1�	�Y��>l�m��T93����o�$F�����������}�F!g_L�W��#����(~����D�u2mN��;r���*�;��	��%�?��To��$��K��
4U}6��+�j-i�������/�@��^�{�������;5�SWJK,�����/�������1`���Tlr|�&�!�����
9�c�N�B,Y�O���!�[s�>b��m�#���6�S�R�m�I%���D�ja���e`�iS%i	�i	n�8�V�a��H��#-��V$�p���*�QIZ�*�
����m�
�RiyWEf������m*�&�_O��*M�i�r����L+=4U��E$C��e���D
Jnv(CD
0��	*��X�*���
<E�����J�g�7e��^9e�FTR�	0YT#��v���3A9�4�)&��Z�3���rF���)N8M�y��)D�,�T)!�,�����������	'���������$�7HY]4����~�&<�����6���Y:��H���D'JJ���@-�����R���T(��L����[������]*.Z1� �;T�J��//��{�T�E��:y�����@��#o6��.�"~�W���*��_JK�M�������eea�{���M^���\���*�ju��(�7�"	�������*���3�����P���8��w&Z%�+$\���5]���b�,�F�F�dK7����IPCW�'�P^iPZ��%��Tb���I�Y�P����s�E"������N�g"z��������TY�7����jU��H��T���b���faa�"S��rC��e :�r����?C�Y�z��--Z�B�������j11mR	M���H/Y�5te~�
Oaa3h����Y6��dW�h����x��DN��h��YUi��N�=�Sq����T4�V->�M
�5��:���R���1����5+��%��Y��Vo%�Q�S�A�0�1��XeQ���'��f�|��_���0]5����3m<h=Y
Xz^�N)j�z�8;�
R���9,~��!�����!U�;��M��,��X�S���;�]�1�%��x?�w$����<"�Kc���1����1{�4�&�ZI�5J��?�W/�~�Z5��
��~��	�7-&n�H�4=��$M��3�R��L
�Q�-��<x��A+	�F���	�9�-����ZJ��-*���^G)�~�xr6=��$��b�"W��P�aX�R@y���f��3�)�C��p��a�C����"-�Ez�C��),��I��e�!eK�d#��!B�Q9�!�C����F��Y���PK��h���PK
�nJ@�Apatches/UX���X���XPK�nJ�1v��v7@��6patches/0001-Free-up-memory-consumed-by-the-paths.patchUX���X���XPK�nJC>"���3@��*patches/0002-Refactor-set_append_rel_pathlist.patchUX���X���XPK�nJC�r��	)@��u#patches/0003-Refactor-make_join_rel.patchUX���X���XPK�nJ����Qd32@���'patches/0004-Refactor-adjust_appendrel_attrs.patchUX���X���XPK�nJ@+����*@���6patches/0005-Refactor-build_join_rel.patchUX���X���XPK�nJk��!�Q4@���>patches/0006-Add-function-find_param_path_info.patchUX���X���XPK�nJ�7���<-@���Cpatches/0007-Canonical-partition-scheme.patchUX���X���XPK�nJF��EJ��,@��tWpatches/0008-Partition-wise-join-tests.patchUX���X���XPK�nJ��J�%VgE5@��#�patches/0009-Partition-wise-join-implementation.patchUX���X���XPK�nJ:�2Hs)-G@����patches/0010-Adjust-join-related-to-code-to-accept-child-relation.patchUX���X���XPK�nJ��x���C+@���patches/0011-Parameterized-path-fixes.patchUX���X���XPK�nJ��eK<@���patches/0012-Use-IS_JOIN_REL-instead-of-RELOPT_JOINREL.patchUX���X���XPK�nJ�b��%�-:@���"patches/0013-Multi-level-partitioned-table-expansion.patchUX���X���XPK�nJ8s�����8@��x/patches/0014-Multi-level-partition-wise-join-tests.patchUX���X���XPK�nJ��h���:@���Bpatches/0015-Multi-level-partition-wise-join-support.patchUX���X���XPK��^
#45Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Amit Langote (#42)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Mar 14, 2017 at 6:28 AM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

On 2017/03/14 9:17, Robert Haas wrote:

On Mon, Mar 13, 2017 at 3:24 PM, Robert Haas <robertmhaas@gmail.com> wrote:

Haven't looked at 0007 yet.

Overall this one looks pretty good and straightforward.

In the following code of find_partition_scheme():

+       /* Did not find matching partition scheme. Create one. */
+       part_scheme = (PartitionScheme) palloc0(sizeof(PartitionSchemeData));
+
+       /* Copy partition bounds/lists. */
+       part_scheme->nparts = part_desc->nparts;
+       part_scheme->strategy = part_key->strategy;
+       part_scheme->boundinfo = part_desc->boundinfo;
+
+       /* Store partition key information. */
+       part_scheme->partnatts = part_key->partnatts;
+
+       part_scheme->partopfamily = (Oid *) palloc(sizeof(Oid) * partnatts);
+       memcpy(part_scheme->partopfamily, part_key->partopfamily,
+                  sizeof(Oid) * partnatts);
+
+       part_scheme->partopcintype = (Oid *) palloc(sizeof(Oid) * partnatts);
+       memcpy(part_scheme->partopcintype, part_key->partopcintype,
+                  sizeof(Oid) * partnatts);
+
+       part_scheme->key_types = (Oid *) palloc(sizeof(Oid) * partnatts);
+       memcpy(part_scheme->key_types, part_key->parttypid,
+                  sizeof(Oid) * partnatts);
+
+       part_scheme->key_typmods = (int32 *) palloc(sizeof(int32) * partnatts);
+       memcpy(part_scheme->key_typmods, part_key->parttypmod,
+                  sizeof(int32) * partnatts);
+
+       part_scheme->key_collations = (Oid *) palloc(sizeof(Oid) * partnatts);
+       memcpy(part_scheme->key_collations, part_key->parttypcoll,
+                  sizeof(Oid) * partnatts);

Couldn't we avoid the memcpy() on individual members of part_key? After
all, RelationData.rd_partkey is guarded just like rd_partdesc by
relcache.c in face of invalidations (see keep_partkey logic in
RelationClearRelation).

This suggestion looks good to me. Incorporated in the latest set of patches.

--
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

#46Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#43)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Mar 14, 2017 at 8:04 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

In 0001, you've removed a comment about how GEQO needs special
handling, but it doesn't look as if you've made any compensating
change elsewhere. That seems unlikely to be correct. If GEQO needs
some paths to survive longer than others, how can it be right for this
code to create them all in the same context?

Thanks for pointing that out. I have replaced the code and the
comments back. There was another issue that the temporary paths
created by geqo will not be freed when geqo moves one genetic string
to next genetic string (or what it calls as tour: an list of relation
to be joined in a given order). To fix this, we need to set the path
context to the temporary context of geqo inside geqo_eval() before
calling gimme_tree() and reset it later. That way the temporary paths
are also created in the temporary memory context of geqo. Fixed in the
patch.

Yeah, that looks better.

Incidentally,
geqo_eval() seems to be an existing precedent for the idea of throwing
away paths and RelOptInfos, so we might want to use similar code for
partitionwise join.

There are some differences in what geqo does and what partition-wise
needs to do. geqo tries many joining orders each one in a separate
temporary context. The way geqo slices the work, every slice produces
a full plan. For partition-wise join I do not see a way to slice the
work such that the whole path and corresponding RelOptInfos come from
the same slice. So, we can't use the same method as GEQO.

What I was thinking about was the use of this technique for getting
rid of joinrels:

root->join_rel_list = list_truncate(root->join_rel_list,
savelength);
root->join_rel_hash = savehash;

makePathNode() serves to segregate paths into a separate memory
context that can then be destroyed, but as you point out, the path
lists are still hanging around, and so are the RelOptInfo nodes. It
seems to me we could do a lot better using this technique. Suppose we
jigger things so that the List objects created by add_path go into
path_cxt, and so that RelOptInfo nodes also go into path_cxt. Then
when we blow up path_cxt we won't have dangling pointers in the
RelOptInfo objects any more because the RelOptInfos themselves will be
gone. The only problem is that the join_rel_list (and join_rel_hash
if it exists) will be corrupt, but we can fix that using the technique
demonstrated above.

Of course, that supposes that 0009 can manage to postpone creating
non-sampled child joinrels until create_partition_join_plan(), which
it currently doesn't. In fact, unless I'm missing something, 0009
hasn't been even slightly adapted to take advantage of the
infrastructure in 0001; it doesn't seem to reset the path_cxt or
anything. That seems like a fairly major omission.

Incidentally, I committed 0002, 0003, and 0005 as a single commit with
a few tweaks; I think you may need to do a bit of rebasing.

--
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

#47Robert Haas
robertmhaas@gmail.com
In reply to: Robert Haas (#46)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Mar 14, 2017 at 8:33 PM, Robert Haas <robertmhaas@gmail.com> wrote:

Of course, that supposes that 0009 can manage to postpone creating
non-sampled child joinrels until create_partition_join_plan(), which
it currently doesn't. In fact, unless I'm missing something, 0009
hasn't been even slightly adapted to take advantage of the
infrastructure in 0001; it doesn't seem to reset the path_cxt or
anything. That seems like a fairly major omission.

Some other comments on 0009:

Documentation changes for the new GUCs are missing.

+between the partition keys of the joining tables. The equi-join between
+partition keys implies that for a given row in a given partition of a given
+partitioned table, its joining row, if exists, should exist only in the
+matching partition of the other partitioned table; no row from non-matching

There could be more than one. I'd write: 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.

+#include "miscadmin.h"
#include <limits.h>
#include <math.h>

Added in wrong place.

+                                * System attributes do not need
translation. In such a case,
+                                * the attribute numbers of the parent
and the child should
+                                * start from the same minimum attribute.

I would delete the second sentence and add an Assert() to that effect instead.

+ /* Pass top parent's relids down the inheritance hierarchy. */

Why?

+ for (attno = rel->min_attr; attno <=
rel->max_attr; attno++)

Add add a comment explaining why we need to do this.

-       add_paths_to_append_rel(root, rel, live_childrels);
+       add_paths_to_append_rel(root, rel, live_childrels, false);
 }

-

No need to remove blank line.

+ * When called on partitioned join relation with partition_join_path = true, it
+ * adds PartitionJoinPath instead of Merge/Append path. This path is costed
+ * based on the costs of sampled child-join and is expanded later into
+ * Merge/Append plan.

I'm not a big fan of the Merge/Append terminology here. If somebody
adds another kind of append-path someday, then all of these comments
will have to be updated. I think this can be phrased more
generically.

        /*
+        * While creating PartitionJoinPath, we sample paths from only
a few child
+        * relations. Even if all of sampled children have partial
paths, it's not
+        * guaranteed that all the unsampled children will have partial paths.
+        * Hence we do not create partial PartitionJoinPaths.
+        */

Very sad. I guess if we had parallel append available, we could maybe
dodge this problem, but for now I suppose we're stuck with it.

+       /*
+        * Partitioning scheme in join relation indicates a possibility that the
+        * join may be partitioned, but it's not necessary that every pair of
+        * joining relations can use partition-wise join technique. If one of
+        * joining relations turns out to be unpartitioned, this pair of joining
+        * relations can not use partition-wise join technique.
+        */
+       if (!rel1->part_scheme || !rel2->part_scheme)
+               return;

How can this happen? If rel->part_scheme != NULL, doesn't that imply
that every rel covered by the joinrel is partitioned that way, and
therefore this condition must necessarily hold?

In general, I think it's better style to write explicit tests against
NULL or NIL than to just write if (blahptr).

+ partitioned_join->sjinfo = copyObject(parent_sjinfo);

Why do we need to copy it?

+       /*
+        * Remove the relabel decoration. We can assume that there is
at most one
+        * RelabelType node; eval_const_expressions() simplifies multiple
+        * RelabelType nodes into one.
+        */
+       if (IsA(expr, RelabelType))
+               expr = (Expr *) ((RelabelType *) expr)->arg;

Still, instead of assuming this, you could just s/if/while/, and then
you wouldn't need the assumption any more. Also, consider castNode().

partition_wise_plan_weight may be useful for testing, but I don't
think it should be present in the final patch.

This is not a full review; I ran out of mental energy before I got to
the end. (Sorry.)

--
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

#48Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#46)
Re: Partition-wise join for join between (declaratively) partitioned tables

There are some differences in what geqo does and what partition-wise
needs to do. geqo tries many joining orders each one in a separate
temporary context. The way geqo slices the work, every slice produces
a full plan. For partition-wise join I do not see a way to slice the
work such that the whole path and corresponding RelOptInfos come from
the same slice. So, we can't use the same method as GEQO.

What I was thinking about was the use of this technique for getting
rid of joinrels:

root->join_rel_list = list_truncate(root->join_rel_list,
savelength);
root->join_rel_hash = savehash;

makePathNode() serves to segregate paths into a separate memory
context that can then be destroyed, but as you point out, the path
lists are still hanging around, and so are the RelOptInfo nodes. It
seems to me we could do a lot better using this technique. Suppose we
jigger things so that the List objects created by add_path go into
path_cxt, and so that RelOptInfo nodes also go into path_cxt. Then
when we blow up path_cxt we won't have dangling pointers in the
RelOptInfo objects any more because the RelOptInfos themselves will be
gone. The only problem is that the join_rel_list (and join_rel_hash
if it exists) will be corrupt, but we can fix that using the technique
demonstrated above.

Of course, that supposes that 0009 can manage to postpone creating
non-sampled child joinrels until create_partition_join_plan(), which
it currently doesn't.

Right. We need the child-join's RelOptInfos to estimate sizes, so that
we could sample the largest ones. So postponing it looks difficult.

In fact, unless I'm missing something, 0009
hasn't been even slightly adapted to take advantage of the
infrastructure in 0001; it doesn't seem to reset the path_cxt or
anything. That seems like a fairly major omission.

The path_cxt reset introduced by 0001 recycles memory used by all the
paths, including paths created for the children. But that happens only
after all the planning has completed. I thought that's what we
discussed to be done. We could create a separate path context for
every top-level child-join. That will require either copying the
cheapest path-tree into root->glob->path_cxt memory context OR will
require it to be converted to a plan immediately. The first will
require spending CPU cycles and memory in copying path-tree. The later
requires almost all the create_*_append_plan() code to be duplicated
in create_partition_join_plan() which is ugly. In an earlier version
of this patch I had that code, which I got rid of in the latest set of
patches. Between those two the first looks better.

Incidentally, I committed 0002, 0003, and 0005 as a single commit with
a few tweaks; I think you may need to do a bit of rebasing.

Thanks. I will have fewer patches to rebase now :).

--
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

#49Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#47)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Mar 15, 2017 at 6:51 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Tue, Mar 14, 2017 at 8:33 PM, Robert Haas <robertmhaas@gmail.com> wrote:

Of course, that supposes that 0009 can manage to postpone creating
non-sampled child joinrels until create_partition_join_plan(), which
it currently doesn't. In fact, unless I'm missing something, 0009
hasn't been even slightly adapted to take advantage of the
infrastructure in 0001; it doesn't seem to reset the path_cxt or
anything. That seems like a fairly major omission.

Some other comments on 0009:

Documentation changes for the new GUCs are missing.

Done. The description might need more massaging, but I will work on
that once we have fixed their names and usage. I think
sample_partition_fraction and partition_wise_plan_weight, if retained,
will be applicable to other partition-wise planning like
partition-wise aggregates. So we will need more generic description
there.

+between the partition keys of the joining tables. The equi-join between
+partition keys implies that for a given row in a given partition of a given
+partitioned table, its joining row, if exists, should exist only in the
+matching partition of the other partitioned table; no row from non-matching

There could be more than one. I'd write: 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.

Done. I think it's important to emphasize the the joining partners can
not be in other partitions. So, added that sentence after your
suggested sentence.

+#include "miscadmin.h"
#include <limits.h>
#include <math.h>

Added in wrong place.

Done.

+                                * System attributes do not need
translation. In such a case,
+                                * the attribute numbers of the parent
and the child should
+                                * start from the same minimum attribute.

I would delete the second sentence and add an Assert() to that effect instead.

The assertion is there just few lines down. Please let me know if that
suffices. Deleted the second sentence.

+ /* Pass top parent's relids down the inheritance hierarchy. */

Why?

That is required for a multi-level partitioned table.
top_parent_relids are used for translating expressions of the top
parent to that of child table.

+ for (attno = rel->min_attr; attno <=
rel->max_attr; attno++)

Add add a comment explaining why we need to do this.

The comment is there just few lines above. I have moved it just above
this for loop.

-       add_paths_to_append_rel(root, rel, live_childrels);
+       add_paths_to_append_rel(root, rel, live_childrels, false);
}

-

No need to remove blank line.

Sorry. That was added by my patch to refactor
set_append_rel_pathlist(). I have added a patch in the series to
remove that line.

+ * When called on partitioned join relation with partition_join_path = true, it
+ * adds PartitionJoinPath instead of Merge/Append path. This path is costed
+ * based on the costs of sampled child-join and is expanded later into
+ * Merge/Append plan.

I'm not a big fan of the Merge/Append terminology here. If somebody
adds another kind of append-path someday, then all of these comments
will have to be updated. I think this can be phrased more
generically.

Reworded as
+ * When partition_join_path is true, the caller intends to add a
+ * PartitionJoinPath costed based on the sampled child-joins passed as
+ * live_childrels.

Also added an assertion to make sure the partition_join_path is true
only for join relations.

/*
+        * While creating PartitionJoinPath, we sample paths from only
a few child
+        * relations. Even if all of sampled children have partial
paths, it's not
+        * guaranteed that all the unsampled children will have partial paths.
+        * Hence we do not create partial PartitionJoinPaths.
+        */

Very sad. I guess if we had parallel append available, we could maybe
dodge this problem, but for now I suppose we're stuck with it.

Really sad. Is there a way to look at the relation (without any
partial paths yet) and see whether the relation will have partial
paths or not. Even if we don't have actual partial paths but know that
there will be at least one added in the future, we will be able to fix
this problem.

+       /*
+        * Partitioning scheme in join relation indicates a possibility that the
+        * join may be partitioned, but it's not necessary that every pair of
+        * joining relations can use partition-wise join technique. If one of
+        * joining relations turns out to be unpartitioned, this pair of joining
+        * relations can not use partition-wise join technique.
+        */
+       if (!rel1->part_scheme || !rel2->part_scheme)
+               return;

How can this happen? If rel->part_scheme != NULL, doesn't that imply
that every rel covered by the joinrel is partitioned that way, and
therefore this condition must necessarily hold?

I don't remember exactly, but this was added considering a more
generic partition-wise join. But then we would have more changes when
we support that. So, turned this into an assertion.

In general, I think it's better style to write explicit tests against
NULL or NIL than to just write if (blahptr).

PG code uses both the styles. Take for example
src/backend/rewrite/rewriteManip.c or any file, both styles are being
used. I find this style useful, when I want to code, say "if this
relation does not have a partitioning scheme" rather than "if this
relation have NULL partitioning scheme". Although I don't have
objections changing it as per your suggestion.

+ partitioned_join->sjinfo = copyObject(parent_sjinfo);

Why do we need to copy it?

sjinfo in make_join_rel() may be from root->join_info_list or it could
be one made up locally in that function. The one made up in that
function would go away with that function, whereas we need it much
later to create paths for child-joins. So, I thought it's better to
copy it. But now I have changed to code to pass NULL for a made-up
sjinfo. In such a case, the child-join's sjinfo is also made up. This
required some refactoring to separate out the making-up code. So,
there's new refactoring patch.

+       /*
+        * Remove the relabel decoration. We can assume that there is
at most one
+        * RelabelType node; eval_const_expressions() simplifies multiple
+        * RelabelType nodes into one.
+        */
+       if (IsA(expr, RelabelType))
+               expr = (Expr *) ((RelabelType *) expr)->arg;

Still, instead of assuming this, you could just s/if/while/, and then
you wouldn't need the assumption any more. Also, consider castNode().

Done.

partition_wise_plan_weight may be useful for testing, but I don't
think it should be present in the final patch.

partition_join test needs it so that it can work with smaller dataset
and complete faster. For smaller data sets the partition-wise join
paths come out to be costlier than other kinds and are never chosen.
By setting partition_wise_plan_weight I can force partition-wise join
to be chosen. An alternate solution would be to use
sample_partition_fraction = 1.0, but then we will never test delayed
planning for unsampled child-joins. I also think that users will find
partition_wise_plan_weight useful when estimates based on samples are
unrealistic. Obviously, in a longer run we should be able to provide
better estimates.

Apart from this, I have also removed recursive calls to
try_partition_wise_join() and generate_partition_wise_join_paths()
from 0009 and places them in the 0014 patch. Those are required for
multi-level partitioned tables, which are not supported in 0009.

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

Attachments:

pg_dp_join_patches_v4.zipapplication/zip; name=pg_dp_join_patches_v4.zipDownload
PK
�oJpatches/UXp9�Xp9�X�PK�oJ:patches/0001-Remove-extra-line-between-in-allpaths.c.patchUXp9�Xp9�X���M��0����x����I�|4MA���+�B�cO���%�/��[�J�,Y3~�{f.;���,�,�"�E��b�K����<��\p�$���R����~#�\�K/�����;�o�V����<�a9�K2����{�e���
{/-���"�������"��&<K8��/�r|�Y}y�\D"�����zr�DSBI����6�M�Sn�P1�2�l��{Ae�5��A=��l[2z�Q����w�cH��g��~����-���T�������.�"�d\�<�������"V!����TTJu�5"��zW��.����/�%�y��nj+�f_*hj����c�t]U�M� �����8VMOH���q��P�8&�gi:|���l2���u�D �d6�1��3x�������>������M�d4�Zw�P3e��K���x������f?~������G����n4�"�?���l���38{!��r�����4|	���&������(���3��PKs\����PK�oJ5patches/0002-Function-to-fill-inner-join-sjinfo.patchUXp9�Xp9�X��V]o�6}�}���%��?��>��H�����a(�*b*KI'����]Jvk+Nb��
�$�=<<���,�0��<�&��C?I�p�]��7M�(����i���mY�5V���ug�|���%�����V�T��+��'�i;�i�c�QVR(L"'.�g�W�qb�oo��h�f�7s�`�����W��z�����wp��7�.WE�1�%�"�AJ�+E�Ni��������2��`����o?'0��^�F�9K���5�0O�kx0a"�����%�Lq&��\1}'t���h�����@����#I���X�P�I��o�I����0�u��w�Y����[���	��,Gu�S�}	��t���%"M��o��?�Ot��"�O��F^Lg�x�����ph86#���������E����\�]�9HTZ�zC���d���a�s��+�4t���Z=��~_���Z����&a� t
��>�f�ksueY�|��]�7=�{��yZ#�>v��m�n��<f|��Y�������5����y2"��"��1�6N��{c>e��wk�����Q�qolt_���P!M8J�v�J
�:
9x�\]���F�m7���
��+foG�3����o��z�@K��c��l*�Y��G8)n�����4��7Ky�n����ojD�IY����t�I�r-�[sF���B�K.
��
�i�9��n7I������b��|��j�$u�p)��a�%����n��%%���&�b~� �/~����O�}k������m�|����d��� 0���z����D��lj�����%�����v�2�3i�rIX�T�	�F��kr�BWA;����<�|���o=�{��[/��b�����^��f�'0@������	~��f�F$������^��5=��1��(��48p�!]��<g�L�c�PK�����U
PK�oJ7patches/0003-Free-up-memory-consumed-by-the-paths.patchUXp9�Xp9�X��]{s����[���U�(R	����,{��,G�o��J�@`Hb4���~����I���&W��(�����t���@o}o#�s:L��7�0&��R���|>�������x��f��s�����T��/�?�����[�B��(����������������
����@Z���m^���|!��vW����A��_h�CMt��a�u-~�f�B�������?<��lR�h+6r������ �HK,E����z���6�7���/L_���1�G�-��������"�O��{��u����V��Z�0��|�~�	=~0��7C��$��L3�>������2�@�$�@5�����/�R��9�pzsnqC��A����sK������:=����C��k	o��~�f�4��vM�d��d���+ *_:�����v���1�A,Q5�������k!F�GW�g���z�p��������"���E��>���H#@�(}�x��YA�Gf�R~�K���-�0�3��hrk��k�
��
�K*��A���=���G:^=l���kR�c��P
��b�=qG��3�:v:�2�'*e>��?F&i���
7!@paIx�c��@L}�)��������k/r@�e�X8�ur�i�o���H5�4PI��v������p �m6�	�<|����!=�v��$��\J_�~�����e�0E=�t�����A�6���4�M�q�M#��D���Q[���fzFw�����A�}w��k�X�^R�^n��o�D�^�i{n�ax[L�0i����'.\V��Z��gQ�c#�'�j4�u||��o�,�:r���d%?y�C����L��f���T���cn
b�F��9�y����.��	0��9�z�O�����t�f8��`�M�d���~D.�������C��I��7��)�w�����X:�%O�98�.�x�E�/F(������0��u<A�)�a�%�F0Dk�]I��6�i�?����vWL���.�[-T|q|�?b�4T�EC���Z��X�Mc�7{����c���d4B�n��������8���3�s*�kr7v�����yaW�����B/����	p����k����%��m��9�yT����-��L��)\���"_�gxo���C��|�����]xZtpp�2����K����E]��05~E��8�X��������
VT�^�Zl�Al�����c�KpL��7K�\� ���;��9 0��v@����	�zQ�i=n��^b����a;�j��F��^�<n��[i�K�sX�d"P
30�I�?ts�4P�b��Y�f��r�<�v�P|(�b#���4���*\�����P��tt�����)����!����@~��j�����H�	�L�E��Q
B���:�N���/
�+Q �����:��bPv��o�js�v��}�^��//��@Y�4���m��x0��y\|�<��H5��.�&�RX	����P"he�(�{O�(��HA*VW�g�*��y	j��7����<gon��\�y�l���
�|10���:e���9�4��Y���hb����,|�Uc��A':�0��q��L;���hda��1�<�`ka�
��%X�����A4ph���W~1�&���b��r�0
����4/ib�
A#�>5iP��K��>�����/�?�c0Kq���k�]���L�Lg#�?0d�!��:2/�P�����9wC��#��b����k{������Eg����_�����]��]�o���M�7��dxln<��\�=�4�0��#}X.@P1��
���@`�����)!�
���y���.�N��mp_�#����'F=�$��'.av�����1���O�����>?Y;Q�n�
�
X���5W`"�������=��=��h?yA�	�E�a����SKVX�Y���5%����=�G:[����hn��fhQr�	�o)c7P�
�vw�}�W��|�66��8�K����
 �	����lo�D�E�$,�q.����&j	
\���>�������Xo-]�=.�[8�B#J�x
!��~nq��5p�a�#2A<'�c�?gc<})���J/)`&����9��@��0���X��������p����$TC�8��j#6yS4S��)A,�KS�+�m��	bR[WF���,������$e(Y$��XvB��7q�E.&�6m��){�$*��c��>����n��P�8�F�>K��~p70�z���4J���Ja��rUC�����g�M��L4Z�[���KF>m��=�h8%�G��1A�ej�m/WH2��[��;N{�;H?U�*�2�.�����e�F�����O�L&��H��6��y��}x�����I�=B�%b���/���%�B���!~�9N���+`K`0�����RI6��g[��
�X�v�SA�(U���-�`9�E�* �7�4��!�Y0v��?�tV��v4�@Dv�A(�=��}e�'S��TK�[�>)q1�/._��S���5{�]��I}�E-��ZOp�� ���Z[�����l��<v��H7��������WW����_���xLh���M�E�(zR�O���{������$��W���
�z��j&!]�rL��z����hd�z�,��~��jF���j��SSC�.�2�z�d���OlO�K��x��)�@��z�:���p�W[�����b���:
�QxX��s*|��6�kxXa����t�DI��d���f��W@�a�0���9�
�-g�O� �A�w,�Ni�h2���z��laM�g<2�i��E
��G����6�P��W�/u^!�j:�Q@P�R�{�y�]���uf��rW�#W0�(�[�1v�����_���}-u����M���C3����4�*{��&�=,�s0�Q���nyn�������B����>��4w�U��>���(�Z(v
Pnd�V[���xU����z�rH	�����ld
������]n|JC
X���������*�<+9�����Qw����OG����~������O��`�X0%)<�0���������u�67�1��n�v�4����9�������/����o���eof��r�nPt��kjH������t�����I7���?����b�6c1�{0mjU��p�5�����lLNB}��I�q���.:����t����A��6���Q���	���Y�YM��}�����=�q���`�����-�Fe/�P:u@q�G����_i*���m�����S�P���z�{OUlx^�JU�0����a����]nP�9X��o��������{u������j��G$pV���3o�������������u�&Ruoi���;�����M��|�d���J���t,���DWL������~j�e@%8�������Jo�5m6����p03e��F^Pmv#OJ�zT�3�����N?�o�P�J,{�ow��������<���w;����_��f8��
��>�Q�`�9�Z�o%)�?c��>�i�������!���!����Zx�k����c!c,���x���B�p[�D��{��-��\�7�#�����:��7Pu��g4S�"7Ps���B|K����=w�Z��"����������������;���/���$8��T�M;�v��@,���`�q�$�����-��V2~�4D#�#)�o_���GL� ��:"
��+\~�C/wH�K6�����������O�a�����������K�����b�{c�<f*�a��dw!6�T����O�n��O]�
���q��{�W�v�Vq�^����|����L�����������X�|��J	/.��D�{��#��;�Tt6�.�p8�p��Y�^���Ue����j�p�+���dC6�����Y�F���9����w�^�Tdf(�!w���,������l��
:���
��u��:ng^���/]����\�F0M���������{���:�z���_����q@����QMp���\�,XZhZ"(S�Is��}��&�*]^�P9��p6��^o4��3k���E���"1����TV��:���{����,xIy��(U���s��eb��8+��q��qU^>�Cf���x�!�!*�����UM����]�d�17>)E�9�^bS_
��g��_���c�{R��^S-��'�18��
��z��7�s�xt�j�����xz�Y��11���j2����&|��N�)r�-SU���t�������*��Q�s)��A�G[[��k�;�1�0��dd��]v�;[�����'��UO(9��5���4��������$���iE�yZ��9/��o�f:9���������������,.>E��)��,�����)[�4>�������O�F[�u+�����,	�����Y�c���>88<�_�
�������$��T�����w��v�a����yK�c(F�L�����+�3�Bd���(��;pQ�#� I[����D��M�\������DA��E���1|5�w~����HTF�',�&[����gJg���SV-a�XY�Dr�Z����&��&1�%R��\�|���e�h\�}�G�l+�)���N�$f1�������

uL����t�^(���i!
���W~��~�,P���Wh��>*1��g������a�sO�!J1��G�o`z�`8�AH�����EV���������od������X$�	��&Y��IQ���Z��F�>&+h��	�$���]3='�Q����Cia�c�c����*�r���+0�
��I� ����<
�������]����+e(�0��Q3�3>���h��n%~02��p!�����w���z�I�*�R�R�f�}C�T�� >�������V	Z=f��>��"�XK)`	s3����pQ]4�5W����
�=�$��8[�LT8P�`��*{��t5��R�}�e�����\�����jH	�NU��:9��E����%_n1�u���P�41�MmW{��@W�z\�\�hVz��<�iuJd���N�1��lQ��,C�c�����].���>�T
yA�r���$��ig��1w�F���������������
:����i��6����C�@����z�V�"�7O��6
uX�a���~�^�(����\��^������`�T�i�><��q:��S8���q�	NZ�i��ep��	���9�}�u��4�bCX`���?����"VX������N� ��0�2��������� V�\�Ju>C�
_�R��<3U��f�k�W��u2n�.x4�o�D�
k�gv��lg5�0Q���&T���GjUQ=(9�}�����w��l:V��;S��������jcP�PK�#��Qz�mY��}���n��B��9�R��6����t��3o�.�
�g��������j��n���+R�{Y8����Y��
;�3��������='5�4d���ab���X���l����P�l8�����*��������f��<�l>�y�gAQ����&���f�������6(U)\8��a�5��� ��r�]i�|�sm>���=���"�vi=09��6
�m}�3<I����Wm�3�WC�<�>���u�?��M�O���+������6�|�V�R�A4��B����T
q��P�\�a8���������?����o���m���320�B�m��d4�\�����n������-w�zY�������>��W��=�+q�	����\)T�fF��#M�����&��eMb�CY|x6��R�W�}�����[��l������Q���������I�K]��4l-��c����^y6��lE�_ �
��]Rw�������I�q�{[�h��t��,7���&`���4J�&f[9���v��8G���N����N�hFN�-�4����+5��~���C=�p5^�����5O5q���U�6s�ce��e���DL�wI�h�%$Ki�Rc�������.`���
�6De��i2�O�h��2���N2c�KF�<'�����#
Ns.� ��}��vE|���6��gm2����U=Z������6�'��0��Wp��S�c��y����T��wk`�j���,OG�G�S�vW`���JU�����Z�����
�5�H���:������:n�U�rN����K	�"W�����^��dv���"%Xg����q�j���^�K��f��\aYS�^v���v��/�&���Y�Wk��!k�=�`�C��/�]I]��������8��v-��	"?%��8z���R���_a������	�S�T���oz&��&�\E��B4
�s��p��ox�n��4�LE�5n1���B5u1T������8�>��b����%���?�L=zm����!��S��6�&���
�w����^�|���G�Y������T��K��G��V&�..c|���i>��7B��+���Rn?�@(��-(�9KA���-0�5&o�����p0,$/<�^>�[����A�k��s��m��_�>��\S}���P��P���m���q�a,���e��c5p��j�)�7=~�~����yx���A<�^/�n�e.���(��HxD���^��W�~/�
��563���YM]���^�D��:�sX����,jo�����d���zcm2��t�hH���DH%	W���=
�E��_K.�_���C���5}�,��|Og^�[s�'r
3�:qv�_�R����{8�|P?7|%���a�}���pzszyy~)�tW����z��7K�H�[H�]�_lZ{����^��_��{�'N�	�Y��\����R#<����'Q��LV�Z���g��D��p:4�����������
�U�R2:}6U����5���+�E�O���-���������y�!���������zl�8��v|������
�S����y�������SH����=��i���z��!�e��k5�T%��dK�~���T#�%L�)�J����&����P|L�X6��R�b������/01�R)��l�M�"���9�c���*k�����^oj����5��A�({<����T~M����7��W�o���D�������{:��s��[F��Q���z��z��-�SGbc}�W�6|Gw��u\�^�Z�����n�ICcG�D�����f&\l~�C��u��5�M{�����_PK�y��vPK�oJ2patches/0004-Refactor-adjust_appendrel_attrs.patchUXp9�Xp9�X��Zms���,�
��;��dQ�,)n2���^:yu����$!�6E�H������Y|��i_��'�(p�X�.v��:�����������c�=�^*N�!��d2��pw����8b���9c��?S����w��`���Lk�����d��w���?�H�d�����/_X�p)���I`�c�Z�`����`�l4`����o}^�������?��<����G���B��'��q�z��_�D�'"�q)��gY�_�Y��5�`2f2�QB��H���O ,C.�Y��\��0��E��[����{At��Bd�����y��ap#�o<����@.�'�HWq�S_�"�F�����.����1��:�ht�.Nn7�R�	3�Z��{
��,�����uD�G�Uk�WW��Rs;�J��
Lmv'0�?�A"v�m�e�
��1|�c�$4��[�e�c~���@���feel7g#w�]���w�r�\�����?Er��G<�;�y���?���.4�[/�)uR]���<R�H� 
��[��:����8'�l��f�_y��5~�+��D�yB���GW���3pX�"!�����F����
��e�>s��^���f�u��YA����y'�������3��?��MG�������3�N�#��u�k������f�{):|��|�cv����g�+iZDh[��z��L�!�|����'�i�����oB�������F��h`�}���F8�M,���!+����8�e��yI�)=e�Uc���"I���A0g��-;�
|��s��..���JkS���L����Z�eJnw�&}y��D��$�a����RZ�c�C(bG�Pc(��J<x�H��,���F���n��Phg���-��[��k�#�b���Z.z�9��3y����8G���ke�Trdm
e�b�d����fgD��vp��#�A���	=�9[�Q������S< ��p�T_�{�]I����U��X��<������~'��l�"�^������-�����`
6�*J(�!�R
?���Cy�������l%!��xeJi��`��N��z�p��������X[�&8�C��|����LiG9�(T�nqZmBMN�"����1S����Nfb	�/�Jl���%g~�y���KEX�K�TPM�_��{.F��d�C<4��*��i�
����3��yv�=�u���_e���UR��>e���FS!��{���M�+��k��U^��?�Er��%��4Y�{,���7sZ���H����@�B�F���#��d�y�����K������p��k�����`>���Vb2�����j����d����HC+�������!��=i�A�6?�a^*c"C�f�|2N���FBQ�v J�R|��f����G�%Jx0T*f�+��H�F��C��ae�-�G���=��K_�n5�W�\�����
�����mz��|���,�6�S������P"y�C��c��5������������j�T?`���B�*�Q.���i���fo��*A~�;w�wIz=;��,�Q�*3I\=gbC��2v'��E�s�Z���2����N�5h|�)C8��[]v�����{La�0��
k����B�TU����"��2�*�����y�sU���xd�G���/�����M�1d1[�|�!�jt0�$�M_�fV��<'IST�������L�0��Y���
������H7�a��Up+"�������VVUmg�1*3�v(�Q*�o)�"��L���_���>�������[�x�D
��F�<S�%w#��H�����	�CX��G:V���v�6�=�n���sL��'�����C�"^�d�{�e�3�/���P\	�;��RJ'���@��*��z�����y�<��H�s��^[�j��r��:C���`�����Os�Ak/��-Ri�^B6gj[�3��!��n/A���G�]Z�pa�����hE���5��n{������A�����\��m��9s�R�,����Bp#���	���<?e��3��Y)��b�\w8$����{0���t��y��*yM��k�{r����\��(��5�0���3�Ur���@Yu?|�����OL~4NSy���������pq>�8�}�������7��	B�]#"�����<w��fS�&�JI6����Yi��)��;�<�z2��,���:������R����p�}�����sr�n�6/^.�/�bi
g�~�i���%rB���B�sSf��=[��/?�%��B�����P94�i=A���Su�z6U�.��k�(�����������o�z��b|i�P	����re�pC��2���dd����S,�?��)��*�0C�}Q������t[&��Oy�c����m[�R��H�>54Ex(+�3~(��#����6f��!U�������:�|hqc��|D������t�(w�-�t�"&}(P�W#���'�)9t��L��h�;g�pe���uB�?����������i�e�EuB�xHj��5���!�\Q�M�T�Rn,j-LuL�2�]lo�,@U���N�.��B�w/5�He�J�^�����z�I��f���,�T������*=���H�yi��U��tQ��L-�������8)���u�0��&k�} l����+)B6��tYl���m�9 E&0D���U��E�!�%l��B��x��y�������=�m��)�Z3�z��v?C-��_QT�D':��C��9����G���������������jqU-*	K��Q�;�7��J�w���	v�o��#F��OZu3�Of(�����(��,����.A_
�����]?@i�������N��h;f�	4T+����|��im�����z��jb�
��IJd;4��l.+�M����$j5��}���R$MD.�5�H��G�h����|�F�_�2{P�P�e�Kt5Q���D��&t�����{N�U��Q���m_O��"��-��@�tJ���FS���^coMY=�����?��{6����o�x�q>��TrM��i����e�1�'Li��Fg�x����G���/J&D���
#���������')�lX���N&������Kq���u����W��r������� `!�{�f��zB���?����F�'�����2���X�"9(F���m���H"��g�����:����%�4��t���u���+���/h���X���t����P7 �M�cHQ�����L4�O6a��BM�lj��3�zY��WE"f7HJ�����)����]�wtc�k�����[�\0�K
c��}D�m7���qS��W*z�v{L?��[�j��F���0��-�$��u��i�@���Z�T���4
������0&$��O��T��i(�����::�[Z��=�[Z����16��B���d[�,��YhV]��W�U!����>^�^��|�G�������t�KqZ|�^��f��>�>
�l�6���6����;;/c�{_��:w�w�Oz=���	�����AqAg'���s����2I[@���Ti���l^���M_U���+�)�
x�|;:�]��iy���T�q��AP��c�J�(��[Y�y���e������e�PK�%�BQd3PK�oJ4patches/0005-Add-function-find_param_path_info.patchUXp9�Xp9�X��W�n�F}�~���)����[v�5��B.OE!,���E2�����wf)��Q�
-Q�����93������d<�|0��1���Eq�L��(����OD�9�%�#��3����|v�j��R�k]�9��K��'���/D�EUVR�$r�b����Z��V�p�U����yC��A�co��N����^�{�+x���	�Ii��Z"�T����_�[�g2O��wsq��(��xS�4�=�����K�*����H��(3N���\�L*�oG���r)��HI]f2F�g�B�:���$��jo\P.E5�����-s]�c��$��y����RO����.U����D���R���,�^�e�C�
�1�!�����y��#=���HF\���������f��f,�i
��-b����D'mc+�	�<������0&�{�0���1��T���c�����n�1UGQ-�dvW���C�D&���~/�abg�������	Qs�$�6���VEI����y0w����$��L�<o��?�Z�I���i�i_#R��U�2�*l�	��{%�9J�m
��Z?�L��g>a��^�������`���k�N��8������J��w�����B����2^Z������r��5�Z�\�p/�g�����0�^�>c���(%?�N��]����1rX
���(�<�[Y���zYJ"M����1����6d���[Y��p�L�"o3�,2�0�����Tk����/�Rd5F��t�}mJ���&Z�Y���L�:q)�x���p��+�b����pB�C�v�si�aD0�F��#�E���UOHO�+���9Pfq�c{����;�[b�0~��X�Tb�/
ms
��E��1����Z���,o�
'�E
Zf��a���
��&��l���3B��1�t��$3JyY��pH�mm�8��4�uO�*�tv�X�,���}��1V
��b*b�f��^���:�j�u�o���>�7��h[���O�u���^A[!>��2����a��1{�A��k����_�����)��
�o�
��m�	f��`����,(����}�z�J�T6=a��25����K.3s�4�����������
�X���*.��������5�9o?b��K��a�.7���u�.��U�_q�����2Z�Q�p�^���������PK	;�QPK�oJ-patches/0006-Canonical-partition-scheme.patchUXp9�Xp9�X��;ks�8���_���j%����8���\\��)�{�a.��H���"�i����_w�A�%�f�3�,�n4����4Y���T��,������,��b�������88����l9^���I���
�������������b�r�g�\����g�5�������3�n�P�`1�����[��W�6;f�����������S6���/��_��^��?��^�g�����+��q�>����Y��@��Wb-����$e��0�����pf��e|N�I$�]o��x��d	��]���a�b~;Jv5���Z���|-�vP��2����S������0-K��nA��$��4r����iaH��}���N��v�H��&��<����RK_��+���"�c���@\���=�?H���xg�y���-g$�}�V"
3��B���_=v�& �Td\�X�����N��P?*����Fp���
� �aP�l��CV����=2<��0�t����`9�DF��P�T:�
%0��n�lC��Bh.�����xA��������e2�G���8%�,\��'��X�x����c���
�O��<��&�����e���.k�Q��,�<�8	��W�� ���[��_E�����6+EJ����� \.���=o������@�o�88Y,����`1>��`��G���b��n����_�����;b�8f�@�l�bs`�\���g8�X���{i�d���{0���N�� 2���X�9���3�������ev!@�`~t��	(�8�������b�J$�7`�79BfB��@%[�-��PuE�U�������yt�MN��F��/�a������}�=�����>���u=�*��
����4�=�U��3�}d�1@F(��b
Y����#2
����V�F�*l��]eV�A
P��`�����eM�BD�vHPFxnK���#����}x�~�q^93�����Y� �Er���Is���^�>��	�C���a�DDp��<�����9���s��o��������U��h�2�E�������t|���N�:��OO�8��2��/������o����/o�/?����tG���6i��N��xp���U�;\7z-��A��J
���m5T(\�����[���pp�k�F%7V�E�����A8���Hq"�
���R'w�xF�!l�T�����49�K�d�u��}�T���O����K\X�l�P=2e!���-��_I�LT�V]V��L����.=E�=����__k�t�
S� M�@�n�~EI'
���$
���6j��lDd��u��X�H��I��O�MrF��
�(��G#�`���f<�0��w�#+����PH��0�	�*h�e�_��$ap����B\�>�B��z,�#B��,8l\`��)|(����B��c[
��������L�'N&'co�'�?���/y+�e���|�OKR]�Cc�gYK�
����(�X��Q"���wyss}���>���=�lM�5�5�����e��J�qc3]+.�9=�����K�D>#$���m�ay�H�f����p(Nf��/��V��G+SQ&��(��/�c������<�p#�9\�l
������	+�784�m��:���e���j�����+c���]<Z[�	qw��QGg�y���0*f{`B�F�>��,��3�
���@��H��i�s�8�0()^� Cs:�9Y�&���a��4Y�a(�*�DF�/(�%4�2O��2m~/���&�=���(��!"�������E5P���H��]���f�(�K���5�t4��2���N�Y����St���F�hLH6��e�DL	�aaA���uT1y�T��b|`��UfW���Wg%=&���34��S"F��s�=En����K�v�o�D�
?j���l���*����)z��O��������<�������<����x����8M�sp�ww"m�I�A���:�}�@�����FAJ��(,2�2x@����w��i���[��8']+A\RyN%v�d�s�����
)U�������W*0��BY(����"z����T�L�����#c���7�(�]���T�Sg>d�:hb0���	$�j�H���z�M����=�3
�B�5�;���Y��"5���8�eh�C�S�����G����B�Q���l�4��LZw�W���
Zw�]����	)�Y?�#��)*�,������������������O�o����}�d?~ �/,Q�A-��8���X[��\���2X��>42F0�+�����oP���3-����3��j���Ef��������SUDm�N��R����@]��A�s�0+��uL�]Q�����@�%�6��5dX�X���e���ho�`�V|�~�E������$7,���=E�
� "�I~�TVgW�s}02�������]-!�����0�P*{�a�&<Bny���J�/�4�@c����c�X��wM:��8�(?�������@<�!�"�P�����u�z����6�vL$+��"v���l��vM�1����C��>�Y�`J�Jg�b�2�:�������������P^TO�+~�r��M�;�t��O�X�d9x��~Q�������_�9����EnM�n�L8�2�m���;�C{�[�M}<U1X���k�`a�����^'t�n�@�cbj��@��T�T�l�b��ugf;���Mps���pE�����i����LS��;]a�T(��u�i|.)b�� ���G*@]�K�l���m���
A�X��8�M��s�/��K�sD�ZeQU��Tj�,�2`,X���H
p�Jlgh�����|�qpOZ���h:
~��T�G�+�@P�O6�������)Z�^h]*�`td�)����-������vD��^�Q��n�,�hW3���K�Z����W6�f��a�����!����]#���Z�eH��l���(j�c��D'*H0�#n�E�I��������}|�s����Qn��Q�\u������u(4��$S"����I�;R�V#`dU�x���
�z�`�9V�lzW^��V��������_`��@�8B'$�������������am��z�}l	�B���&��"��~:t��7F������&�a����Z][C6�m>�h�
x������U8jY|�X�88��K��J�j���
jz���5����,nj�d�ksSN,��[�r�]�	]�Q�fI� P���������q�%YR�_�?��@��
��9���b��vR��N�|}���VK�A��-� &h�E$0����J�T��n��7��|*�X�����2?s+�A����|=���rt/L�go&evZ=�*�1H�1
&��6�����6?fJOs�rc��c��R��G�=����*�N+����	���k�}�D��Y�~R�&'���}�����T��`�d���
8����GaPw���oW7*{*q���Q'�|;������U���a��<�7��BA�k\t����G����Y�V��&�jM�9�X���s}���Ar����cH����}6�j�U,D�@�����t���S���= B:���0����ax���~0��n����jM��m��!l�T����Io��o�>�I2=�u��y�"���s/c��W�H��i\��N��'4��~��;�����4c����������E�eV��:�Mz
����|�1=?����K�����S8=:��p���'��z��@��o�B��C�+rh:�����9�����qV4�����d�m�v#�vy5��xo
B���"
���l�B�`�h��N���@uS1��Y�K�o��;���c� [��?#A5�/"�K���}R��Ej��$u�z|�k�qV����h�}�m���Pmp�,
D��R��R�k�Ti?SZX����n~�lY�>���x]L�I}�`Y_@&3�!�KP�������N$�y��f���x,�)�lz��N�<���}�(��7U��0��X�����+{��o+��ue��0�T������E0��;�/D^�
{��)n��b�K�n�����n`�b��?�`��0�%��t����(���
NU��+a��p����-�@���@a�`\�v�]������#�g�B�!����x/��I��E�.����z'��3b��Ng�B�7�L���?�1��//C��o}�������j����I2�tUq����r�����SITe|��r��eG�lQ��y;m���q��P�4(K�A*�"<>�^�c�	D�c�L��l`r�6D���l���\�A�^���js�[~��4�,4�y%�Ai�S%]�l��*�H�J���]��h�X=�{�M��X5 :L�
��,!.j�=8%�0?������e��U��Q����JE^�pF^���T��R���f�R�O���������\R���/(�U�q��&+��0�p��6I�E������AYi������-G/\Ws{�\u��
�Z��u m#h&"����Lp�hv��������	q������b���r.hT���.k[��*�.�B%i.9Rd�I���J��P���wP:l���9�W!�ek�a�=��`� U,�u�o�	��B����*�T��J��}��V�X�+:�������8P�I�C�jN�%Je}+`����P���<udk(�"J�T�$K����I����R��J"�D�$Jw)tK�����w:����:���@$��V��J��?�i��*%����i�I�N�����a���PK�����<PK�oJ,patches/0007-Partition-wise-join-tests.patchUXp9�Xp9�X��}�s�6����_��������%�3��['�d��c������z��d�VV����\����/� ��e��S��F��h�?h��t8���=<��i�����
>�yw3�o��Mo��`��#��Z���
����{������GB�=:	��h����G?B����S��o�2
���y�L�g������G�{tt�>SB������{h�`��\����Y����O���;:8z;�?������|���B(��V�%��0
�wv���!��/t�
B�\Eh�ZF~������M��K,�C���{��s�n�cJ<�����"�������e�X���sC�-H��;{{{;(\��R�o���:��������mFmB��_�������&�!�/�b������.O�?�B|��z��O%�
T\��B������3:b����_�74:���7X�G��n@��!���{{�4[Dt�au�����hd%?3p
vvn���ho�n!��U�M���,�/\
6w�����X��c����:�����.��b������
�H��n��o;�D/��������O�����t�n]���}(�B@����������G�|yGj:[-��%�������)�>���=��!��D/����}��Dm���E��������������<��9����'g��^���?��A]_�h�x&T�	��lB�����9�O�W�����9����:=;����_��P����7�j���	��`M�~B�t�h����>��'�'g���s�|%b+���;��?\�L�(kzD>�w�V��Wf?�b�6G�}��r�I��e����L��QJ~@�:`7��cV���T��k�TA8�S�)�hR!l�Bjz������*tTUTT�U!��P��/h����������+iA�0�B��h�$uf�mg���>;�t�z�8�����t������"�����M(D�}@���_�?eg�f��\��������){}�r�����GxLn���N/�^&�I.��������WD�<�d@�V���z�ZG���8z���
�.�B���|������"t��,o���'>�_L�
����~��y�zL���'�������wt5��D��c<]�g�i'��F�?z�
UQ�j���g�����V
�����|A�Dj�L��T��-5��Hre�5rYM &��YY�Q���Y��V��q��uti?��5lA��x[�>T���$�n��<����i����>�Q��e����PG�v�����X]OR;������z�%$7����7C�2��$����"}a= �X������p���K��������!��x6~�����%��&��Q�"������������������(OA�/m��6�\����n�c��Sw����T0�-�����>��>��O����_������J����{}x ��
0�����g�2m����=9��g����
R!�e,��,c�vo���lB��]�no�R(���VF���~��Q������d��j
:�z^2�T������?U�@�����2�%��`��s���b��TW����c�E�$�Z�Q&���ikq�90��m��n�&�������e����*Q5��Y0O�0��iM������	i���Z������P6@7�D�7P�`�$MPmB�;tl7�V�fX����|��h��9��`�~!��?���g������ Yh��Qr�MA���b����W�fH�~��>m������.>������'Z ���������b��8�Q^���A
@%�~T����<�f��6t)hd�����_F�4M��
�D��xEC�j��9c��j��RUQ���si@������z�\�8W��V�/fd2��H��d#��Y5T-�*��H=*�RUQ���si@���
���������Y�������V�H=dV
��kP=*�RUQ���si@�������j8�e#�+m+M>���F�C;WNQ���0�r�S#+:�{�jG����0��yt����]�����?�"����(������k�Cq��'|��[�h�(
��(]���FD�x�y�lw������o���1%�������F��='�-�ht��Y�����G��Da�_���6�R��9Gp��6�Pf����]B��aP�����x��G�k��tW=wW�Z�����Q=T=�^�\<������������k�zWvt��W<������n����N��i��l�,�{v�Pq��<c'���D��#R{���G� ������v�\�-O�d���R~AO�Gl<:���}�A%U�h�L����hf4l0f�����l[;&�/�j�C�q
o�D�U��CL��H�d���b�T=��I�*=��mk���"��W�u�2��
�S=�p!�|'�3�'�S=��I�*=��mk���"��W�u�2��
�S�r�����z���w����=���gi6�����H�
�#��o��s%I.q�N�����K����h��,�\���%��`t��9m.q����o�5:Z���.�zK������+��)�������g~�\���n]���<d;����]w|Q�qo����l��:^�����n�HCZ�VM�`�����/���Wz��L�jn��&c��9��9��xLvn��X���x�R6����p���nw���}�����xv8nm����)&G��z��{��V����bI���A�i�/�otY����@���f"��H���s�q!:[���r�!%c@L���Y�����i�\��2�J&�����A������i��9
Z5U��dy���k�h^�(�]����z�XHL�Et��\@;q����v�:{�D�l:��3���x�l�4�{�h��k���oCc%a�l����,!�)��&��N��Gu �>�sg���y��q�����fo�m�
5{q����f@�n��q�3��0��f���:m��G���Ps�s�K��zb�+P�A���H/���&z����-x�4k0ae �~��~�i�P�q�n�������E������UE��8C'�KGU�Q�D�h�.��e�P�G	e�*��d4���m��S�Ea����=*���T#u�#(��C�P�[d�Q��@�Dm���:I���`�d���qkc�O<���u�c���R{Dh�s���J/��w���{{�(?���v��)4A��E`k�r!���b�^7������IY?��@pk�vx>*;�eZ��bH����������7f�H����������q�<��QQW*����vg&OU�$\��D
]G��"�^<v1V&�Zy8Y�LEI�X������l�	 ��3A�j?�R"(CUpy���a�v�J4�6>� �J������������n�l�	 �Z�h\�DP�����S������>����~��M���
�	W�!W�X������G��h�����k4�ST$�8;�>�<9�����oP.3j�O����J/���/��#6�L�c_$��=�=�d#y��=DP&��NC����0�g�y���OR{S��8Uy><���wvzru������~�
5#���G�K��b6�eb��5k��|�\��!��3�,�
�����������~��Y6�Ys�Y��c�����24���������y�Td�vu�A����Gr��#��/%����z���/UN�"�RWBe��o�B*��E/KO�����*�e.�gw&MR�B��
�1m*5L�q�s���$������{��0�I��=/����n=����*�K]���2k��$!��������Uv�*?��;���n��A��M����F&���p�����������7�9.���'�����_��P� '3���%�l��.���3'����Mqc�/����70�"S��,����Mu��/��t$N�225>)��(�W<��,OG����.�25�4��[�M�� G+= ��@����lEx����il�
�j��(^U5��0|�5mtv���d*�	�[X��h:��5l<��Y��L���c�I��Pf':;b���_=����<����h��D�DLO/S�&��(s ���>Y�N�g��hvc��^*6�V�h|
1�BP�����a��U�C38�5�4~�l��+�_�����:���!]��@�}�6��^��#qu�_�#�x��/OIW@�'���
�']/>_F�j�2C�����_���'����?&�yyr��i�Gr��i�-��?���8��cZ���K����_N���}t}A��>�R�f��*J�HTK�3�EI0Z��������k����K�I��|@�E�	V�||��'���$��.Xk?
&a��a�`����
��9!|r~r��?S�
7��{�����/~�x����!��q���l��X-_��j�� ��V��}����.�����d��_�R���!�5�Wd'� ��8�`�Q��+��(��S4+�W4+H�J�0�h �
E���+(V
��E�O�hX�h�2��TKJH����J:�e�o���E�w)��8���.���m�-�a�v��?g0�����I���R��m�w&����-:
JA���������c���K�ayq<�.�1i] NcV��������z�bKO�����6.s�%�=�x�s��[]����*i{��SU,O��#�� �����ir5����O�������w����P���)���<��V��N��(������g����^�
�$�?^��W{����G� ���Y��>$��R&�&w�~��?���!�3�q7,��K��B7,��+n�U���y�i8���c��,hrbG=+�k��P�u�5��k�d��|����M�����dv�M��/��������������j����nOG�K�gc����j	b}��
M,Q���;2� B`8@DN!L��6`��Y� &3�	�_���vQ<�:�Z)�����s�5tJ)������@���dJn�������<��i�N�A/w��FN���{��Z��e//ZK���H��^�
�6��w�	��KLm�P����k�l��8C<�.���*,$;�R�or5�8�X��oz�p�
�mmA�����E����tP�z>��,��y��:������[34�O��`ol���6�}��O����"~X��� 8z��B�i2i��M@��i�4����Z������������r��k�
�_�L�a����qw_���~���o��I��z�&}J���P��1{�c%�a]��*���1I/�����#Jnc��c�� ����	�������q����=,�
�l�O�x�f-vzI�O��Y��^p��%N�Y��^F��;` �xI"N�5��Y��^���w�0�r��l���n'��A��D#�M�j���8*}�v=��;P���������'�t�gy�
y�4��v�gyv���<Y��A���A��]pt�gyv�'M��l��|�
�X
����j2� M��A�&�Y���fr���L�o{ ���t=d�{�C���~������a���T���vf������q1;��Dm:$���$	>��E���gb�9�x�>�P
�X)o�%��D{P^�~A}7��#���Yaa���D2�Z���$��	���2�����3�W�>X��p��P��X�I������X���h�Y]��
#���i�^�e�'�^��}�m/,�>g����0��r�������d/
��m���[�@�.N�_^L��`Q���zv]��B��������7T��C��b�TT�*��\�;]�o	8�Hl�k�<�u����|v�i��jl��{��	g�5�n,��><�xv+�����S�2
�jx�	�"t�Q�& ���p����������"8�N�N��/�
�g��c7$��z��+AOy%8�"�'U�� ���DT�W�}*����*��-t����=�>�
t�+�t�Q�}�
?�rv�{X�9�?G����/X_��R�=7�a�\���4�J���S;�mI��O�"I?���\���~d o�0�����U�_���r)�jD��s}IE�y2��?�����h�����e��SL�����}�K]��������:�rO�A��"��<�{9�%�����g&0i'��$=�m�:r��$0N�Z`V�D���W����U-fxR�,��4G�������������b��X+3�<+S�_X�_��K-�eR_K���U���������\�U��^e��~YH7/I�;�/k.xR�n�~�t���2���~�3�S�~���`g���xy����D������Z�����)�/O�_������I}-��'�/���2�7.������(VYw�_��K�O8�/k.xR�����r�\yqes������K����W�|�x�V��]$���DU~>�e�,��|�|�R�����*zcNo\��6m@b���V&$lH\�����;3�����h�5lF����~���<�G��roI��P���:��j������8XEk���������:@�}�n���4�R�,
nlg��?��S���4�4,����HTy��	���XI>d�	�>�n����y����je���Z`�	���2�]6������q�r�>���/�z��<�7��p�
ts���Z��y-hj�[�A���|��[��[Y\J���4\��s�D�^�e��f�*%�������Y����������\t��2\��U��������t��k
���V
�*Ug����oN�oc�nM�Z3n5�&����f��
_�Sk��������9�M
��5|��Z5|�T�
��O�9����5a�jM��0|����k����s4|�9�H�C��
�bP=��'��G��78j$��R�dH/AS�Vpt=���7�� +�XU�d�J*�0'�(M#s3Wu�h:�go������ol0`���F%�&R��"����#��eJ
�@'�?�;���PL�8i��7�
�
��n�%���������0U���_�vL��M�+[�'����SlYL%��.&��a������5��N��$&���4��'1;���<�����N�/+W��b��y9���-A4c��aC(X�
�I:���vNaK�)����E��V���u'�(�'Iy�mg-����@�Z_����l�?�_�����]��L����n���S��2W�m���U,�vdH�f��D�Fv��@�$
����^#S`��#go��}G./�'I�������Xr����w��	�����~\�#���S�#�o���.;wd����X����<����f��C�$i�X����c���N/�W���(>5X�)!��IC�<1bM
)#��H'6O��'~�j��&��N�.�O����{�bto��6��'�������5]��yR����������������$�|H��mu���O�wB���kZ4O�����n�C\"��s�������Z�E�6)�'�W��^}����Z����4r(m{uu���O]zB��W����nG5zu�o�;N����ky����`��^=�{�q���k�j�������U�b>��	
�^mZ4O��	t�7��<w��F\�������0B�����d����aVq��p��C��M�GR_�`IWuM�I�������� ��$���c�����'��D�?��7�'���g���}M>,���|��������d���_zS]>,���|������������Gz�����{�?�����S���'?ur~r��?y�����U��w<�����\ B-��	~N�;�H��6��n���
���g��&�������C#[[�J	���
�2�X��$I������ZRkR)Z�J�����_�����Q�.c�����>��:5����d�7
,6r��e
�,F�����/�|��1w.�����3�+�m���>�HT��h���:f�����:�	U�����������j�k����L��Y5^Ug5i���
��I��zS���s��������T9�S`��AwV\:��;�K}5�c�Is���G]I���������Wp"�:CM�������-�WCZ�?}k�;)����bY�Po�W��N�d��
�6t6�CMl_
�����9�5�4,��w<��p�,���J�v�O���n]/]J��@z%1�I���jaJk���~����|�m�/���T�(����j�I��K��5���8��Kj+���of��m=���j(
�����b�
(v�[ nh�L�����y���#�l��M���������7�����B�R�2[����>�-�5[]��G��-�xt'�e����X0[@�{Sf�j�te����,������T������F�t�"l�V!������B��h���T���G�������P�.�$�d�l�+g#o����S���B�q��Y(tI��Z�.����3�F�x��k�;���b��T�n�w
�BK�u���X�4>��P��[�6��'�P���s�f](�!5S	<�Ph[����7��\��A{-���r�`(�Xg���nK�s�	�n1X��Phsny2�.�:Wk��BR3��s���I��zS���a��5�}5����~8���&C��	_��(�����{���yr	���P��^@���
��C���v��6
R���X��b�|��B�5��l
T������V����}������S�U��@�)#�~��{���X�S��r����DQ�k�!������k��[���Xc,'���d	��t���r���Jt�`��z��������b�z�vF(���]�:������]m���a��Fw%��B�1��.���GK�����[�=8�[����w{+t}�"o��x�no��|s�-�]�6�`����!�=+�7��r��aj�9'��j����o[�����r��kZs��V�M4�Kc��9�[UK�NS��U�
`�+�_����4��n���;��	��Z��p<Y��e�{t��w�U����..���(���S��u�`\���W����d�l��!�VCn���3]�����Z����2�c!�:���l8F�[���_�B1��z�b+��bp�N�Z�UQ�X+b�=�G�xm:������i�n��k[y�-^��P��jt��=��31����`,gA���
z���C@'i�'�����6U������\�^����~��7(�������\��S�/���0bm@3
��}�8_Y���Ca����0�'%I�u����6|l}�GR���0{��~�zg�'W���P%?0D�S�6�h}�<#�Kg��c���&�������"9�\�&�� ���@��8�Z��g���{y�h��������a�N����h(�$qR�F��������[c,-I�sk���hJ%c��)*�U��������_h��y@T�";K��
Q�Q���&I�����Ab���-��5���9�6�@M�d��0�(`i?���
7��Y����@����>�Q^�#}A#`5L�O�%�����?0t��sd_������g4qm2T��
��^��!���x�95�kj�<�f����9�w3���K��)d+���_�D�R�_���7�q-�l$�!��b��]���V��g�L�I,��F��~T��X�S���Qp�z�d�����u$�}���q��ETP(����2Vx1u0+s��i����`S���F�*�(���<�IiC1BOt��m�X�%FH�Y�����o��7����:�����=y�_��xA�������w}��m���Sa[�{?�����?(����\�R��l����x���<�����~E�"�?.�{B����S�����wg�<\��|4_F���2C�����_���'����?��@��������X{4��"�?�$��2��cZ���K����_N��(������{D7TO�����J�k@�NK�3���aT��CN������5�tN~H�NL�|@�E�	�e�||��'���$V�.Xk?�-��!���������'�'g��3��p�����{S������<DU D>���O������("�F�>�}��,�#�G����s@2�F)����=��+��I�\������m����%���'����bw=������zR5�C\_��bU1���C��z�5z�+Sc������+�)�z�]�v���qf���������`��X�i{`0M���������,���O�\n�8�%+k�N������o�H�V	"����(�����Q��Fs������-)���p�^����=�A@5�5(�'�L�lI4���v��^jR��t%���9��I�Cl���_��G�zI
��&E���'E��!ZU��A#x���K���ng\����_8�/,��h����n}�� ���K3c�U]�<,9d�V�d)V�U�W���X�I�/�~��C��0�1�,o)Y���,��+;�%;�e;��IK$@��Y9Gc���/�����g��3c���>�;�j�l��k�'k��>Ir��&�_�O[��/gg����b�Y�Eh�~�=k�7�����s#��::�9@���J,�2��8�������g������c�X�O):r'&id!�c��
Q��[:��
��]��|d"P�Dx�yFC�R1E�S�2+a0���	l��n�P�_���0�&�md���b����9�����bEg���X��X�Y�,v�����R1�������U@����f�B�dz���zbcy"_��1������'����YO�Y�Y�Eg���J���b����Y����G�5[DX�@4��N��������
;'�,���A�#L�����:=�<���@��L���[l����= �{�@�G��
~3�w
�)hAA��
:�����<��h6O�C���y�(�#R&��>]]�f���R�'�)Qa��j�6c<@��%���h��S�����8����NGW�,��=L�%��������9�O����������U�-�R*�y��r*��<9������aH�p}�����t�tC:���9����Q�����s��s��t�:�����1�~E���zH�kL�$0W�E��?������ ��QM$)�#�f�mk(�T����%�!�.�������$(�,g�����'�	h?Yem>���E�1���2����wl4������>�����[�-�,g��4�'���#/��x8���@y�����N��M��������@��������o����C�{/k"�^��y��=��J�)�prw�����JzM(L�����U�(����dn���k�.�� �$�*�� *�c�K�g�3�Q1�ST���6��|���ZTY��Pv��\��&�����8�5�b���g�x����Z�Mg�S�I}�X1lR�DW����t��P���t��F�r��������f�~�a���j��9�-'�lW��<�&"i��H�7���������%����ay�Dm6�L�z��@g�q�b:pj:��!"�j����+'�lg��<)�C_�7�1�������#����ay��
��qq���x���\V�.��Iq���n+������h�@I�p�m4:�����t�&r4o�������o�!� �=������MU~����<-m/�P�t��a�1�e2���J���a�mH&c-�q~���,���������M�0�`�=��ii�J��<�i��XK�8�P�������6z���.�P����(��������P���+�
38��d����a~������l{��8��qg�;�.W�3�*���iw��H����s���p��:�)=x<���(����)�������*o;,�iw4��k��y���2�z�WR8w��g��Pmx�$�>���������>#���Q*�'�ixl��4�LViC��hf��i��>���T��;��+<���dk��R��|��l�Y�4��Yp>���o`Z�U3�\0O����1MF:)Y�Pj����r���a<,�L���?!� C<��cmH���R
�t:�bH0dH��!q>-��N���D����yR
I:��.���h(�VMMh9Oe�1$�3�jy��gd���m������pl����D9 F��eE4
3<�a�"fX����DL)S�O����ef����)����K�����,3��D�M���a��q)�#��Q��!k"2V�|�efP�X���/,��23�K��
����30k,�Q��Y�&l{[\�n�}���u�W��������G1�;8���[�����pp
w���;8����`�}8X�������u���W
��<������;8����j���=�g�N����T��AL*#ZGe��D�2� T��9�)'Z=e����:���]Gz6i��[���Zs)^[�-O���on��UXP�����xX04	������S����C�}�.��h�����lO'����r����]h�G���	�1	�9p���B�V����0�X���6<)&�j���>�F}Sb	}:��5J��j������tD����7H�A�.th���k`�2��j`*��QNj�l�70���3Z��������l�pG20��?���L�.pQ�����'���3>�������Z����Y�����N6w`��B�u�
T�M%�X�b���j���6o$`��X�@��S�
����;��	�Y
/6����e!�8�4k���V�d3`��B��3��fn�����+.��f�2
��j��	�WD/���Y#�����$��7Z\��4s47���_A�q676?/o{[�/+��
G�B{R0�2�"��Ud�Q���2�V�Y�*2�@Yyf������@�u������]�//���7��g�&��~4>��u��9\rL=�����iG�y�2W�N�����M��P�����G�"I?����TD�Gxg�[�����W�I)���F��8��T��'�c�X�x�6�
��_�:>�$yOY��D/*�#{]
���)c)����\YU9%���
�|���/�*k�x���5+�XT�-c��}^���vi��I'S-0+��N���+�,�U�Z����YP�i������m�E���Za�V�0_V�`��b�0`�Lq�Z��D���+���zE~�$����+�`�T���u'�g!��$��������'����,�K���+��p�|���T�|��h���xyo����D������Z���*X>O�|`�Lg�kY>Y�Z>O�|�b�<��y�,��4~Ye,_���������}^�~����Ys��j���,�K���+�hq�|���T�|�����xo/m3���.���e�������}5�c�q4����Qy�qBox���
��7�`pb+�&'n������;�#u�������l�����a��|�7���N������0[ �}�=����3��y���Z'�E7g]5ee8;��<4��Y��{//���y
s-��yF�
:[�Y�.giqb�O���j'�;=~��������������g/�`��xSe�������o�e3Z�Q�oG)����l>�����o�2�V��PxK�xr�}�L�GY����$5.���(2��T�GM)�����0q�(ym�N���6O�!�u�6�[�si��%_<����f���,(�P���:7a������<nL�kZf��l�����^n���W|2��U����������\��q���#����	������\������m����9������\���������J��A3�Y;�]C��P�zJ�|I34Dt��d��/��U�_��}��e�
O-�X�������{�~VY�ez'ww�����ww5y�~\��G��:�P�v
	U#Q���c<I�C�w�g;��pr��|�����YVg��c��"�������L�G.��H�����$)�Q�����K�'����g�=S_���2=��]E�'�w7�����]z������IR��B���9hb��R��^X�d�5ekMI+������?����0��N���}���!�oQt�u� O��O�=�W��%
���Vs&]w��i@?`	�RcR����+(g�����0��D��e�w=2�x�/B�Z�y�,tM����u�)rQ09� a��5��	��S����t�����0@��*��VgK�\���Ve_��Ey��Y�d�����Yp[���O�P�oZ���������K�j�[���0>1^k�����~��\"C&���%\6ad�e��R)�9,T'��%,��:�%�R�[ j,X���k���,:��GOf�����jU����Z�����b���<=�>E�'���2�3Y�����h@<(v����zv��������O����!6����������B�1�~�1%������������������/�g�/��R�������S�t~uzyM\Pr;y21����&��=��������`��(���z����{7��xN����������
"���!����%��}�����a�cYJ�%��N�J��:��az�0��-����5Lh��+N����qR�j+�m�����������@sa]sy������+o��Q�xp�hh�f9�TAK�3��)M��)��S����K�Z��B6?7�>$���#��p{�h)[�DI0Z�Z��i�SZ������ZFV��#�)G@S����M����u4�����y0����r�i�����<�As��e���_�!����ka�_��B�/8��l\n;('��&y�(c$��
���Z�^9� ��l�)88��F�����>m
���A�.�f@���V{Cs�9�sL�O�7�y����=��5�kr��_�P�{���idF��=�g�/A������a}M1q�Za�������5""I�b��h<M7�H��J�S�c���	��d�p��C���LS�Z����v��,��.�
��4`����p,���C}�G�H�Z�����N�fE.�������Wt�Z��s��7y����Gel���]n�������|z�-����/"^��<��2�Q�QP��/$���� ����A��,z�&m�,HYG�7<�t����������j}��w!�{l�+oP���y�'�������[5"��[��.I��������.��A���>�@d��a������iD��wXCo�vRI{1#�/o8�'#OD�t����|�,��i�}b������K��	!��B�l��-�>��v#]��g���f�;��j�S�N:v|�;M���1K-3Q!���E��dB�����b����'�KRke�1�T���j�� ��[�����/���Y������0��E�aO�a������9��
���t���up���-1y�b,&��>���d��34���w������f�xxK���F;D#������5+�oC{�op�v������O��Z��p��?�/;�������	y�kt�V��nQZ�*�q�a�f���<����:�!�3�������8$�ntP-����.���<z�W�!q��������������|��?��iy����xX���m���z��	�"���<\��N����o!�.�3e'���
C��7�5�8Z�d�N��������P���0������{O�<<���/8D�#R�j�$%�/�sZ5���s2DX|EqH�`�d�su���E#7����dI�~w|4������������[*M�R��~u4x�_��(�Wtx��|&-�~>IL����.X�7���2S��B<�/�����8s�,�/�vN���	Rq	����a|;������.�kJ+!�*����`<1e����	}�[�� 8*-L��s��,#*f�~%ZI��H-9����A��T���!�����c0Zj�JT�H)��Z�C���������
����V5�j��Y]_�p�]��C��t	�l�t�{�\pl�*%�|�!�z��e���L�!�8�����	"6y��H1f�?�B���Xgl�DS8
���U"F�V7>@������=\W�<�D�������yU5�@dg��)����s�F\��&�V���������~yL������q�W����Qj���K������������4����i�-��bc����1(���-*m�J����a������sUgS��C�	������%F)���u�����t�%�o�X�7z��#�&���=�_�6<��
�O��}�nE���@~HI3�iP7������������Vm�5!��P�D����f7j*�k�d,�2v�f,�3n��.*��2H����g��E�i��u�qT\@������W7���)���0!�]��`�ze��?�v��6<b_��������j�;O�7-�T�xSv$����S�m��y���,�M��1�<�:�`����`Y�9z��T���M�N�=��''����T�o{:�|x���y@����{�]������
����VoG�5�B(Xp\�d��Q��d��b�f�a�(B�_og'���'g;��|�
�������\�^x��|Y~����fD��x�8����R��T:���#`������������^In�B�T���*={���0��U��TI�*��"r�U1	�&����]4���}8!�7�
��#x�.�R��v����)n�L����p���x���"&�$��	V(��Z��jj_�jV5l�jV5�Ru$�j ��h����[��@�CQ�����NT�x��?H�y�e6��:`�A=hq��F��8i����M��k�q�x��6�C��q��C����_��z�FC'��]6Rb_4@� ��W~@O �X�y�l�����i;z�+���� ��9��
�aJ���=Ab��E^L=����J�����������Uo+����5[�'W=e��q��M���"6���o�W����2�������������"8�T��LK
�4���RJT�9�@���{����%����W"�XW����5p[��������)�#�jI���/��<��GO9�
�K�6�C��5���4Fm;����l0Q
����+���	��s�K����
sg�$��m0��H2!�
�+�"Faq��f�V�Z�qm~������!X����t�Ew��Vq���N7���_\�^�`IG`�{?�g�e\_������%!���d?fS���8�o��<��a��AI{]H�����yb>O�����=��O�R_�O�_��GTL�)�x�7h<����<�����S���'?��V�ld�h(J�/���<w|�9����[��&b^EE�t"F����!��R�u��WR��5��F����}����l�kZ62$��f������6���M7����N�z��_���'|�*��'���<����jV�'�L����x����C����
��r���B�X66@�����T|�W*3:����Y|��,T3:�j���^|hc��C�?����^|��^|��^��Da�"y^*j"v�Dk"H�\1�� UcM��5Z��P���5?�&b�&�D�Y"�@%����2�T�=�l�����jc��xc,��a]tz��q����KI���@������N�F`(g�*��^�U�v���'�L';�x��>4\D���<,��{����7+���a�����Bz��7��C.^r�����`���VCw��'W�\�����Z
����Qr=N��)�n)�e���G~9laQ[X�Ov�A�-�O�XD�	VZ��A���:��
];�������:�y 6Z�����Qj�����H�\�@��#A�� �T
b��=x���.����G^,oN�>���g����j(��U��m�%�������T7_!�C��������&���Sm������V�����;6� �2�H����4�0B��3����E�����qAv�=��>A)�n�#�g����_~N�1rA�G��CW��qV�8�F��z�5�(��-���t�4;YV~�8��_�6����T4.�4��j���>�N���������<^�vogE������u�-"�����bM^MM�����n��-+���5����oX�_AUc��M�x�����|���g����Z�\�k�6q���b�r���� �(��"$�el%�X�f��8cI��fZwi�B�y�)1�3A	�D]�-�5��5�p�mb&6�&���������KXf�(
�\�&������+���g��2����g1���B�'�c��>XS\���M ��A
n?�W��%
��>T�[4����2&�Yq��mp��S�dq��S�Ud>_6�{1�si1�$3*p���|�&����wo���K(B�� EF%�8>8(N��'�C�F���e�T��dP�" }�9|F�\�M�������,(p,K?IZ9��i6��n����j7�7�W�`#�����*
s\2C��.��jh{J��{�����������V�y�� 	��$9�5`��$D��c�Gp{������#aFp���*���2�����9s4�(oLq����y��4BtE�r�i�����<�As>��r����Bt3��
����`��	���hM���!)G4�N� �^{�\����;�h�Z����	�i}	��7�����a���J��=�Ok�������d[Q��c��H�������-�F�	�^-X,�@����p�b�RQ��hou��Z�kY����M�R=U�A?h5�"����>��KRR��E���TgpG�3�SX��b�j�o�k�l����4E�U�K��f����R��l���i��L������P��k�Q��xk�!5��������PK8�vEJ��PK�oJ5patches/0008-Partition-wise-join-implementation.patchUXp9�Xp9�X���kwG�(���i�A���J�,��kH��}{}p
@�,@�UiN����+����=�wW������Wdd��u�OTw�;�FIw���t;����^�s�����Az�>u��i����Su��T�H��?��U���l�0?���f1���"�%s�c"�������yZ���L��� �<�|�����"m�#�:�0�9����C{Om���������`��������oT�x�������l����;��~����&�q:�~|���<��,�|��C?����T���z��qz����P���8-[�W7���$�*��MV�y:��f�-R5H�]L�
���y6��;-v/�N_�;k����H�$/������CV7I	 `(���q�A�0��S�������E��*�������J��&wx.�j^$�r��j�EZ���~�����S�"��R/�>w��\`�f�9��f�Q�l��~6�,��:��U���UFv���z�
V�H[d.i�7�P��d1�g3�Q6��V����0-�%|��4�I:��{��\Lf�,k�>����f���2�c�GO`�`�j�8�������2M��
���&����)�J�.,l�UZ�KB�2�d�lM�2/�.H�-�tsV{���s��q�d�����X
8C����q
�����
����?)����[5+�-*�����8F��������Y�`�7���R�h��C�i>G��#�#"�lK�8HF���s�:��;�C3HK�(�����������`a����O�^���C���a�����&�-��z2��e�-�Y����;8T��gS�W@�>����FW�����b:([����D���?�aGU���j��&�1�������H�?4�%�@yJ�����N����g]����;�����'���	��'��A���5D��ikx��Q���<��[���[�:�y6�)!*x��t��Z��*���@��{����_��������p������$+���Agy���=���a*�N�n������g��G���9��_��"v�|��}T���~v�������������������v�?Cjkx
���e�w
���yj��9�����������>����S�}W�6{�G�t��h+�?�^��&� `����w~c{�������`gksY��s����[K����6��0�]�������������8����}��K�n�-����v�;�M@���V�������EJ�M�p
e��,R2I��~�|��������]��s��~�)������y
�~r��n���nn�s������I�I�u���
~�~/��8���g��Y�n)6�}}�\��a��������!�Q�����y?��i/��-�6���G�G��'	Ll���+��E��Y�w5��n�7���]k;�hUg4�c�pK�Y��a�������R!K<e�����u���0%��"�� ��y��k���zCqc;��a�!F��V�C�����(H�����8���i�{t��	n�vs�������W��z}���f�~svq���z2h����:"e���sDFE2��E�V�Vu����R��9`3@����%������xp>x,28"��6�2K
�,�$ ���T�b����OJ5I�0gd;�HDN$a�|�����(�X�����<��t�V�E��-\�$��T�E�ns6i��$�"t�R����>DC�O��y��"�im�3�
g���A6Re��!���W]H���0��[�R/�-�"�U�����[�����e7��p�w����f������f=iVwt�?�s����������������M�����e���:�6~dpY����>z��GB�9� ��,-Fp�"8�E���I�9��io:���9^����R�����\Q8e�S����E��
R&=��\E�P�h���[C\��W��gRk`,����W�*���o�S��
���^��l0<���:u�_����%l�B�"t�����p�,8�$������*l��.U�0w&�MCf�zv:��?������y��%�.��������/�����\-�_�^����h�>��ZG���Q��KQ�!�(��
�����>���o�Y����cXd���t�7���+��z��Y`,7^~������������go_5��h��/�WRam=��*��6�7v��_�~�����P��v�'����{i���s�QH��x�pY1X{�2�UD�7�6k������-5vf��x�m�M?���,�X����w�"��en�#Q*$ri�}+R����?��2�g������a���M�t|��fhD\�y��N��z��/p<l&�0��5P�>�lH���J�cx���H-f���9�;�6R�4�������Q�����;�4��/���g��s�/}-'b4h��~�5<�&'{�'��z *M�<t� ��m��`�Ce�2+�6m>&�����w9'�5�gc�o��]g����g�yVx��>�O��>K��C�T��E�������7j�V����,T@-���q~���A$gc��Jd`��o!�����#��L1�rS��=i�B
��������`����N�����o������u�38��� r���v���)L:�N��4C����pe��qN�:����D�L"�14�a;���Zcf��1�"2;b@������h��R!p~���F��(e]��V�^��q�:P�R�S"��v���:#���8��?\cf��'��+U\3eV����fe���9_��5��;+�n�(�J������hw@����,��I/�,���� 0.F��D:��F�G����<��V���T+1Y��5T�V��:���hR�z�f�-��-q}��c�p�)������z�����^�G��}��>���c"��1f�1{���f=|ZD�74*��T�������a����hc�*�'��X���������;��	����"t:H��������1$��`�b=��Qn{ d������b�m����Ty����"���6�(�>[�K�$�������l��iR���0��[����8�������Rw�n��5k����!=�5�4i�?��$���u��d�#�?fr�s�0~|����G�^���h{�#'���}��.4�
.�QZ��ER:&z����l~SJ�C�Q�P[�~����4:��'I����nnW�l�����2�w��	�/�y��!��~��}g3���~H���L�X���c���a\��i�D���HT_k]�c���}ERy	�8e"%��%����'A&L�������c�Z����=,]����?#������BD5���1��[��sn@��zT{��$W-I�Pr���niW�����y!�c����}@&����s��B���f�i���k�-�n��t@W?
Y����Q�	����8���PV�jL�D'"���8��6�2��"A]���"��\�J�9�i�h�CuN�bZ��:N��hs��F4�2�d���[��q�y���Y��*�7Z%�"L$t?'zJ*J�|`��
;#ca�W���pp��|0�w�<���"H�F(2X��:�f��S'�D�����'{�V����w����!�erh��4���o�,���M<��\���zwv�C(�u����� x��r��tw4�Kf���/�%N���X��L+�bo������0�������������~����7;�}r�$�c�aE6�2������!lo���"��M��'�����*��m��s�w@vh���U|F��'����r��N��i���L{�V�;��R�-r�O�&����kV�= 
����	��Eyr
+�z�9����228�uW7P�2Zo�^��`��'S;���%Z��]Ln��Yw3,&����].�45~/�A��,�,���	�U�Z�+�^�k�A��Q��",
40,���o�d�x���O`dfM�_���d��h������t��0\8��.���5�v������lH�ye�!�d(������{��P��R�� ��/C�a�Z,5a�6��������i���O�b��� Y���^�6�k	���i��X
=����� ���g���-����7@4���{��3�*�a��u����e���1���$������%�;����������������W��t�m�Y�����,�E)p���>b�?r'%�Zy�"�
P>w�J��8v)S�)�{S0���nJLg���Z��������.�bV*+�;���#f����kV!�2�@c<?��c���|&b�c$�D�I�D!S`��y*Hp���{-Z1I`���i�C������"����V5jaP ��'+��\@��A��tw�`�?�j�������'��H5�XU��E����D����E@v�5 �'%Y��lc�O�Tb����������U���te�������xO���F2�Os��m����J��| )�`���A�@�qH�W
��	�J�(�G*k�
��g������N	�6j\�m?U����q�>��&LT
P�����/x��)��:�3��i�O���{'�x��;d
�(}J�r�/�������N��4���f����e�
�:�f���0d�d����sJ&S@�y�c�������"��y�o>�D����l��;�H��|������*~��~���>�
��
%F��������c���PH��w� ����w����r��Ih�������V��p����#a��L����*o��������c������������&�zr=#�Ao
�G����=��&V��?z�;fP���TD1K��U�s�-�[�0�;z������Y���T�s�F���y��/�yy���6�����
gd0Wh����������v�-[p:�_�+��v��a���i����(^o(`�\��������j3A�>�1@��!j��[Y��p~#�FY:fM�����~��-c
��J�|1���h16l���llW���v��d��(2q�7.b�y��E��R8���lq�2:�X"�K�&w�v����,��s����4��i��i�`��<'�fg�gl��<v��R�#�1��>���9X���0A�y]�f��0���Zb��j(K�14�h�����7E��&�I�K>����ZJ���P����@P�{��x���D����r;n?�����L�N���[��<���QX�R�9��]���M��{]����������|.���C�7 ��\��Y��������I�gu��	���/$�[0 �q"�jDP�f����b1��~�>F�a$�"M������An�	=��2��,~w��S���q�yH��9DRX��*�������lk��}d��L���>�4UD�����j����%ne���Q��qH�nc�'Q�Z����Tf#��%O��?�y��L�e�OX����k7��y�q�S�Ng6z@�+t.�g$����Gnn�h�+f�!���������q�!���"PAOE:�,���k`C/�'���Gtq$��=�N-������H~�O"V1�_�p��is����	��=-����������_���T(P�Gx��5\ ����U�_nl��=������O�/e�z!��� m/m��T����F�i h�
��j����#.�8@�?����cB���������_�����������j����\AF��2�9����qz������!m��N���"E�wE�F�
�z��f�'!��"0�q�BdW]��O�68�U��q��G��j����&�z���!z��r:��vL��p���(��'
�;�'<u$�A�����i��D���r>�w/R�2o9*��G^}z���I�D#�' >�77���v��c]Ff9:
�b���_�R
�"���������Q1��q:.s���rF�W������-�]844|3�p%�k:��#���;��������~���bL����������"8�&���XR����:���j�p�|�$�H��AE�m[��=�>���V����������6��5kL���qDi;"��Ye�������V!�����h���e2��i��ay_��d��4�Z+O��,��r�}`1���!mb���R��|�������H�$�����{�Q�wR[��K.��7��+��5Sb��4T����Q#!8��H�t���]�8�������b"qT�
i�� ��E���,����:G�?���-�W�z�W4����6h�98���aOi����0�%�S����X!�I�:8��G`�h�j�U
��e[��D����_B�DkY!`�)�L�6(����f���+�Y�z��e(]�#���8	:���9@i�����0��&�G���C������C��-fiO��U8����� ����#T�2M����P���ssw#�i<
J)�	�����-�oZ[�(�����1�����g"*fs�-#b�|.�Mf�t=/A�3����&"R.�����)Ej�+J�����1�K�J�����oX�%+F`�O��:{jF�o���_=�>]#@RB��4-D ��h��4�����i����-�G$e$|��U�)�E����tsuJ�%�S���{��+Z.Y��#�8`a����?fe�^������4B/
�����VT�T�I�O3OO)V���4M�J�}�����gz�,�X�����r�E�b���0�Uv�:����wV��5k��!��N�����7lJx��q*��(Bl(\y_�.F����f"<L8�j����fp��+�������,O����Xy�r��SM*j)bY---���1���U_��\{�n�{7�]H��{�����RF7&��q����zEMe/.�n�=��iFJ6�DX@�XL��_'h������
J�;����\2z���68�iY�a�WZ�)/(���w�H�)a�������$8��$y��c�������'�8�L��:����Gk�J����[����E�*:��5w���)]61��XL�.�Q�����=�Kib�$�J�����EJ�����&b�n��+&���ETh��pX������x��
E�Q�`�����e@�@�"�0�>������
�Y��GrO��0M'���+��r�����9_H�/�Kf�����1 ��z�(��fY6��p�U�T�4�����'&q��(��x�y���pd_^2�q��f�&�.!��.'Z����nZ����,����j
��5�2����L�����QW;l�3o���)�g�������N��O�nq����FH�}x� \h|���������M`*��y�\�l��{
mSVq�&�����i`�?
J���~+A~��K�h��5e�;q�jasR�U�]%h�
�[�#+l�e�4*w�b*Sg)�������)1`�����+h����\O*S!��nlF~]p��6�y��w��z����'�`�V�wk�w�=|!����n��kg�9�A�gd�A�'�RA���e�&��,RzG;����C`�G�"�<-�5���'w����������	���[������������W�'p����� �yF��kX��re��|��{��F�y	]�����?��7����~�[���
.S�L�O��	�
�������������X�!���$k�P ���o���N{�V-:��H�=�����>�)}`�Ei�.�QmC�_�)���{t
$�ab{���J#h��d�����YD},������L��O�9�R�n�%�.�JaQ�y`���i�Yg9M�V���7~�����@3*�����
��Y*I�`�_�.�Hjn�OC�P�"~��A�c���;�����p�*���5sdQ��X����
��a:s�[�8/�;��#�a:�Y�$U�K�3���S��f�v�����C�]���1Sx|��U�����k��U�V1��c�NM��+pn�{���Z��gb��H)��(����<��v&��-U���x����5}����V���Hd��p!�7x\~���/5#	��1!��bWe7%b�8U�
*�B�j�4����G��-�c�����Z�dh��P+��#���=�.�E�|��}5�L��E��-R��J�q�uV��BI��=I���lA��r^�6�B�]��*�Tt�RN,G�
�BQ���8��0"����~�3�������^��o-�/��P�����Q�M����4E����@���fE[�mk�w�v���������5�"pP���:�O�5��������G������>;����Y���{v�������z���$�PU_z�7D��'���Y��_��p���Q������.���-D����^���[��lT�i�pmUx_N�����wD����Dml�>��M#�����o��1���_����T��>G���# -^�=�:`���?��.��K�|�x��	T^��M���?:�A'F]~�����������]K���v�Zh0<������$�_+�miej!�-5�n�!�&�����EF�tq1�_O������Y����\s� ��""�����r�=L�m_��>������������������g��oO������\9�j�H�����o�x�/N_��D����r6I���4872q�@�N'O;���v����NN���r �D�-!M�R�?�����(#)���&sg����2M���
n0H�#�@�7����:yC���������Oc���$�v9K��8*�uZ�4����n6�~����L������X��T�����������uED%�)s.-��jBIK@y�3,���3:��q{b@����d]���A&�AZm���|���m��Ot;��!i��Q*�g����iD����=iHj���#t����c�{K���0������"����d�����\�'P����2�����g��E�$�{� ��� 4�xs����`�=I���,\��b�m�c�)��M2n���'�����s���[���|�����>�v�!�W����|QR-Z"4���i���)���9i�M���N��E��c�/��B���v���z?��d�"����M\��h��-^x�H��� d��b��b{��1���Mz���}�?��uH��AWv�SW^��N�������������^�y��-�����-����>�cIBF���`iIJ�X�r;L�hlL�K�J(�rD��:I@m��]�����+y�
�J�zp�k:�x5���*��������|e�;��a��):���{�_�^WA��Y>c?N������Kt,b�FM��v�0�C>�	������I*IC���pd�A_6j!*����x��N�>.W������x;��2�}e"���|lC^*��x+�����h"�����x�1W�j;S8�+�-��Q�Wn�CJ
����C0K����3x����XjGo#P���.����<��x��4;]�}��mv��`n����Yk7++�W���c��6�z
�>�+!X@�
��l6�W1w�P���s>�q�X{D>1l�����!\`E��.���w���>�2(����qg29V��[�[&E5{�d������d\�N7I�.fl�p\Q�{��Ep�r��h�Hl]	�G��dX�U�����hZ�me��U��kcQ�F�{��GT����?��l�(�nM�d�"�28��w�<w!�d�-�[)gH-9$���E�������L!*k���o����>��bj����C=L����zt������Ej
b���C�r�x�Hq�>o�H����=e�g}"���U?��GJ������(�1�Z�D}�L�����1�2<��.�S���7������y���_;
o�Foo��1
Cl�Rg��"��%Hdi-X^�(��ZV}��J,����(B�W�����^L����^�����E�;�S��/5�����VY���4l���9IP�������������uZ�t�U�O�BI��D�i<���$���B?HN�KN�����.��x���8V���KY�b�')�"~�X�+E�G&Y��>�1���"��nb��o�����F&��s�{�{v�����U�����[��������D��5e��?5����w�I��p��o�G_�5�g���`'*�E�s�2g��"��A|]����b���3����o�q|����BncnR���h���.��B���n�`�%����B�*�8���k+��V�8����^oqbE���sTO���T�:�.��l���0��_�-
��-���L�0Z���\ygU�����%�V��Ew�N&���
��=���l�������_��w����$��4M2!��V���8Q-<�����k��mY�2�)��C�dx����[;����������\U�/�E�
iL	g���r�$Y-rfj,.��bp��U�\����#�X�(�������h�`�2����I�E��H}7����SyCn�t��hi�����/nx����C��RK>�A�?e�DL\&���Tsj��&��}�t<S|���h��j���/��?��]T�P�~�n��A��d\��^�xB�+IQ��J<�p��u�
4��k:�PSu�IV�g����M3����g@�v��j�&���\�A���%��9�6\��x���0��L�D"�L�7]VM�^+>�<AV5�o�@`��P���=�����a7�Nm�.7��h��Oi��yF�������(��F^d�
_��K��u�x����'KY,�w@����`�:�y��� �^��s�w��@�j�����G������G���d��#�"&yKo��[�^�5�P����j�k�|v�0n���nVZ���-r���W���$7��a�2�H[�:U]p]����*F6L���)�'`�iG/vAp�����8�^��Z�9��9�����{�w�T�!����XL�<G�5�P�k��)e�BU�N1�/�v<��O9���������vf�2��mg=Qs������I��,� �VdO��\V�`��R>����f�.q�N�U
���r�a(��M�^!3Y��@
��i�q:�����-c��]�	�fG�*{�h�)��8�����Pvc���k������Nw[}����f"C��L�e�/~Q�#��4�[���u�L�������qiU��0�Z���]�@��=��%X`F�����(��k%�e����y����I���B�BU������)v`��D{�+������\/w�c)p1|<��8��[�����?+	"�����-�o�+�X����y���1d�+���gEN��}
���Y���59�#��Z������h_
E��\���������J��(Mv7V�j��Hc�k������Eol�E��~�Pg��-�p�j.C'�}Y7�'��e�G4����-Nl��������}qR%��$�!V�;?��M�0������h�{����	��{I���fd�G&��9�C������I0btC!�(�����������$�}���F����4���m���	�8�������}��g!����x�<��B���~�{�7��]��
U	(nes��f�,������&i�8����l6u ��Vt��@�~���:�f����y�j�P�U��/c{���?���?��������|�F52��X�[��[E��;�Z����7���(�o/���N:UU01'1�k�p_g��&h���7�%v^����@�������7#sK�5������i���&��AV��&����1pM�ahq�D�as��u���1/O{d�@�0���6�U�����e������1��h)��?|���z����3��ie?�S�-f���t2��C*=���A������l�������+~qND\m8'\�7IL8m��A7">?�����>]I�S�(3���
`E=�� ,[Fs�9�i�"_i��v�d��Kv93w�Y��~���K�g�{K�)J:l��H�9��<����C6�	���cK>%�l��etD���U��T�Of���!�����7�CO����y1U48�t������� ��>�X=;t��ie�����,�������Q/�W�OxJ�?9��Lb�yR\�������=�"M��%����g�����k���t�c������G
�����0�bo���d�7;G�]��N����P�"��Q�W��KT���"Imr4�r��Q�X3��Z����M�AS���������6a���`���c����:��!t��YO��B�n$��?�p��Q�>�[�6t#N<,�W�����?p~���|�����$t�
�wx$Z���w ���C�b�j�~e�Vu�F�Q^���>�F��&���<r,B���(��/��e�[���W�5[N��Q���|"��r����,�I�c��������s�����8�a������Hr��gv�TD��N'�Q��#_�z~��aGh�����*~/&eX.lP��vQD���%���A�	/���������?�D^������}�$�+�6��M��3Q|�
~
j�zo]����<7� ��l=�aNB_�	�?���t��/jp=-&G�`8��`��u�F��t�UHT$dI�h�RBG�O����a�u���;>J�
��Z<ZiM9���G��R����cR��@�U��:3��vh�-�������L	
�������U�XYR�������b�5a:.���l�9���:�\b��#�����\��|r�L��0�G�q��D����A:{��3���sL�������}�����a�%���?�@�o�A�����E�Z��E����^� ��4���1���K�)�����������	�x���;�3�T�48fi%*t���8+�t���q�������s��m�z��qb��8������a�sKy�a2�a�u��_��@�J,y��`��a���a�}�������r����QU
���#������br���.��N���R��Gb
����?uU���)�WM4��������B���)�e	��i����t:������{~$���R��/+��I�K���NE3v�d/8qp"�"1K��c}u��np�OgB��� R����j>x��I�B�r~�b�e���u$(0A����|�A�AV2;���{*S{������o�}�>�^��]��+%�@�	�INl��jThGm�@`�0R�����+.���	�U���<�=�V���u�X�V�F�f$�c}1�7"(f�-Vx��Wa�S�����W�_Q��tu�B��3�I-Jj��6�H*����IF&��+(�4���n��)p���1����J,��u����m�����J������Q����4yx�!��:'b��2%;�+L�2��4:3^:�B�?�n���+���S�s��`:}���b�Ra��q��|���)���g���T����4E.3)X#H8���2)��z�1���h�\����Lz:��S��KD;�#k
��l���=�:8���2I;I��X���^i��X�iq���B�� ����M���L��^t�l	r������#�J��P�n8O���pa_B�?�"E�=���
��-�j�������2�:����{��dPC/�c�����������A�����r{����\���$���MS���;����������+W�+���?'��Z�1r���Ql�r,h(����xt���j������n,�T56&f��X����<-��(|	8}�>oFY�����1���O��#I���]��'���8�:���0HA�����1���9!��_�;�aVl�!)��v��D�]���Ie���a(�x.	l$�QIpch����z|�k�_�;<+�R��TF_��wM�K�����D��K��P

��x�f����*���.�EF�O>\�f��������?�4I�x�8yHV��0��o��X��@(k�E��
����������c�A���7��!UY5�����@�N1�s�����B������v:�=�+^.�\�m[GQr���*�t���/L��X�c�/e�\h��������Q*�������7r�"��WJt p����{�O��2��*���+�pmb��`
��pN��
��[p�o��0���g�E�g�Q��YB��lr�#��a0w��=��8��}��p����c�E�
G���{�����b~��3���u@w�v4��V�"� qTwP��c����1�����s3�;�7n��L����-��r���|!�|�A�A�g��B����Si/r7�HS��3,X��������������@��{�����85�!���C`�=rq	R���*o������b�Y�6\<��tm>��Z~��]C<�"e�c�e��GwG��o�k��7�����L�:$�fu����:�G����Eaj���]���M��6j��n8������6��]�W�Gj�������As�����y@���r��L��;��e,�f���i�Q����y�-_D����~�h4�!~���:�b�Y�O�T�J��Ue �����gN�0�����O��B��k�����5�u�u�x"��Uf�nW�se���!.��R|���b(	�����o#��G�qSi��n���[l��d�_8�����
'���`�7� �H��j�?��|S��1��/�V�m���r�ma����~b�rz0?U��l7���VI&��37)�zk���\v5�:���IoJ���Sr�
�l�b����������G��r��<���z��7���\B�R*X,vH�5�����!�+ZrcB\
�<3�)�Mg��%���M��5����$��qCx����[����-���������>�}�h����F�
�"����\�s�m����	�G�F$�X����M�O-��i�{�l=�n�e��v���v�������!j��� ��.$�L��0]�F����@���NX���b�����M9p'��H���f#�8`Mn���|�w�td��R��m��/Tg��M�H>.uw�5�R ���v{�I�+n���������p���1�Dr]f����b�}E����y��_�FY�Z��f=9L�vg�j������f=��L��%�4�^n��'�^��t��ue����.�^���jA�&�?��-�T�CrW�����Q�a�M1�!1�,s�O����Al���f��0i��q&&���w=�k�����K�_S17	��}H�f�`�/IK���q�R��L��X5K����l����Im��\����q	�Ef4��T���0�A�����������d�5����E�;�XX������������/-"nRkQF��E����+*B���h)�M��!��HX^1����=C��Dj,��_|/J���M}@�9E8D_|��<�A�?�3�
��`�9z��X�1r�������������Gr,G�C�/���Z$�<������U��Y!�^r�2�3�T��LFk�����i����q:,&c���eh���tY�~�$n����)�S|��4��z]�i0�%��B���/����v�!=N���y-��\Lt!37n�za��U`���:�1;�&�-�	�:4�(�ks�&��Y%���9���1�9k�#D`���>6FS���'�@�dS��i��=�o��(&���c����R5W����"0�X��,�?��L��WvK.���	��
7+?�?�gS���5o&�}���U���w'������KK�s�������>	�������3�C]��pUN������F`9{6��(.��=Z{�r���Q�{�$�P�|�*.@*6��Kq
xk7Nc_[r���.�#h���$\���Ov$��~���Zk�����}��wc����:zn}��m�RH���@���U��&W��uZk�nz9�Q�s�j�����}��<a2kU�|�X2(����X��I���\o��*U��KI���e�Y���6cW�+o=���������@�|1�su88�w��)c�����n�`�q����f����(^?Q��������<6�A�u;���fR��y���d6Fd%��OsGK}����V��aA�aV�#R���9�.�	�~
�<�*������O�B�p��#Ml���
H�:"dZ`f{��,������Np�>W�;���Fk�w|�j����`0X_��[�����4�&Y�]q(|��c����WTu�;�s����A��6e��WT��+bjCcjk�W$>��X�\�p���		�xy'1U?�b�7t���&���-�TQ�|���f7zO�Vb�TX����z!��fQy��I�/'�����<�����
~��2c��h�0��<?��k)%���y%���d&������X|l[.�#�`C��)_�
o����t�l��,��&���i4��R�X�CI7PK��0��
�����M�EQ�}O���Xw��9#O�l"A��-F��c	�����s���f��h�����������o�����rk���n2���>NM+���L
�������C7������n{����
������)��� U���x%;�[
�������
_�2$��$���Y�
��F��0�0|c����tu3���l���2�Fa$���2�,�>y!E2���vI��r��I�0���������.��T����L�Zh<�: ��A�(�x[�i��<�����V+i�'���|���'�M�uH������~��MQ�F1S�l7e��5�����2�yy#0S)��z����6���K�iy�c�c�v�>7�� �� i|D�>�����I>#5wh�pRK8U�C��;F����m�T_M���5L�?+��9�Ur�n�����8�	o�a�:2cT��:$�I�e�m�8�������V=c����_���Es�O4�o���z�U���X�%Kh�G��`�-R�:KWQ��bX3���[	e����B&�6&\��j��	�������RL4���U��+Zz�.t5�5][Nv<R�[-�4����R�����p����]�0�RV��!���s~�����.��M;!t?l6;��uw-Z|}[E��5�^���=���q�e�C���A�+�U�86�_�c�~lv�1-�c����������D�'j�!8�����WX���9�]>b��������1��+)�������K�����sK
�<�����$�aD~����0^�69�t
���1�����H�DQ|���zdX�����Q(V�.�F���d�;�M������Z��||�e�1��2�UmX����5nTK)�����*hR���e]1~���>?��B�r��i�g�'����9��f�=���s����4h��/�����c�7O�@w��'����f%�M9�r��ZM�dH��	@��,��N����x��!�s�yoT$|�?�����X���e=���b��|`���V�Y�W������W���
������41���\�:��M��������n�'�`���0�3N��-�
a�[��)�	����������8\��)���w.
�������|���>�FI�Iv\:����-�]��<`��
kq�:-+���V��x|��x�*��EY�{1�Kn��`#n1p�l�u�pbk��������a
�t�Wd������J+��������m�0���*
�)�b��0Uq
LjY����9���"�2fc/������2������F���4����J���Q+��s�24�Io�)�)Ex�h���Ke
�3����>"���0����������)Y�hs�XdT�E&�Z���s=��jq������!�����|e�����,,������"�Y���b�8�c�<v�nZ�~�B�Q@����\�d��3`�D�]L=M�b
��c�U�����]!�?���4Zy�"�J
wnS�����P���];wMFB��R��P���_ E�X�2��	~��]�e�4_�����2*7�����jn���|�v�U����yqeX�c��<C3D�
On��
�����gAl�$bN��wt����Q�H����V��[������tq�~��Q�����
d����#�]�v����<Q�G��f� Po��~��x�O�H�D�7�?7w���omB�?2�F����1��?K�t�)r�&�*��*z������vl��v�S���9�8��23���_����;R"1��%��+���q��������#_����}��	������W�{�4-���
t�|�JD�u1�0:����T�jP��)�fh�-��
74�%u�m�	�����x����:�F�R/R3���qx/9�E~���

��-��\�����f�4��<B�[�w�
M��5Me�����?5��?�s7���e����������[����C(��s�k�u3B��9f6W]�������=L��t
	��4W;z�����6����;��>Hv]�"�b��&a<��$,����;���K`��q��|��:�2(��~z�}�mU��4&.i�Wq������!D���@M`I�[��!�������[�d�������4����g�L�U��}=����+���Aw���r15sf�A�D�_�s�~P�>|$7�j�q���W�5b%��#����y��������eH��S�BM���
���z���I5�6�~�A�}1k�+������Sv>��^3��DmZ\�����!k�:�;-rbUa�:@��
�A�z��#���Z��bL6��v��#�z�(����c��N\�W,�IC���g;5����[]v��~���d=��S�g���q��f�����}����
����]%
�t�S��?�o1�sO�j���]{-.�M#j��������R�eL�����z<v�:�c�`�r$�q >GXj1�V�1��6�����[�������e���^
sV
3/liQbd���b���~����h
�2D���b����b��Yq���,�����Q�	l�������s��n�s�����2vVX�jMeuf�����&�DO:iE@�8����
Y(3iV�N�=�K�&�L�|��[�I�B*��?��8W-%���y�����,|K��F��A�+��{]�y).�16�9
Sw>���������p�5�a!��g����4��������l���5d��� -�=���y��;�� ]`�p������ya<R�|\��u�:�����'�=���U�x���#�E������
B�p�x� �l��G�;[a/�i������=�V7��?8������.���|	�7���|����|Q���w[R'��O	^x����7%h�f�j����e���{u�J`�^��[��(�`X�[���
�������x{��*�^�@��~\���=�@�:U�kl���#���v��	��t
0�\�T�8r�[	�'�M{=W� N�LR]V�
��c��E;q�����W��	@g��(I��5��^��2�`�j�Yb��%.>.�3��5���w�h��=F�QR��P�q�b��f�����u�����c7���'�pH�K9�:��S`��y6^��&9�#���n��Bdx�Q[Q�1�C������1�LM� �Z��S!�PA*�y���������
*�x��WF��6H[���$I��y����B������s���A=O��&���#�����/Z�~�>��.�e�,���v���:qz�z�������^��f?+���A�[=|-^�����=LZ�A���9ZR�����^i���I�E�N���+	�n�]��y>F?/��=���x�X���/�H�(��%����>��45���^�>]�]\�]5�}:��{������?���]����7[t��M�|�xt6��)BZ�$��P����c��<���T�����?]���K��i��R��p����5����qR�2_�#Yf�=�e}h��A�_Zl)%�{�~p�q���q��5K\?4z�������W���W�W���~>�<�}|{
?������n�W/�����m�f�Vo���vN���x�]���i�/G���M}��h2FgO������k���Y��Ix�vy����3g�^_���h�:�V���P�}WdHB���q������%l��� ��]~�t��������/��p�T���:3B��DX.�}F�4��l�=�"(��)����
\�n��E�	���G�����?I���~�^����2���>���?d�BA���p{Q�����t��em���)I������Qp��V����t`��"$��b5�R����,�`���w�N�9�;8^�����t�P	����:w5� ���$��U2O����S27c���V�d���D�nK����F�2S\M4��t���A��O	���#��:k-�p��5V@���.H
��*�����'[�6�H0l�o+�y>��/���P����J��}�a.J"���4[�y�&����;��<���B�.��K'�f���d�
XW��E2��
��h���u}��>�b���z�/T�"�|�D.�a�lF�X
YI���3(2�)du�Q��f���h?Og�o	�@���Ez��l���ie)�kp�����1L��}X��$3�
����%�}�>��4;'R`��1T��~q
7����}"�j�	{��;{����B����O>}����g��N_�����3����6=�v��P�t��������?R�?�q��L$9�X9��u����n8�,�x�����M�0;���~�>� ��'Z�l�i}����7y/v�j��XE��X�{��8+�T�
��>EXo%����0�X)G������U=ht[���V�`��s�!v����B3�����WW�����q�/d
�0����7��9����Dvq�;G��-~�M2�^C8�9>|������>���)��8diP��VU����)�\��W�?�Q�q�*��c����'���Q�3�]���z)��n�)4�qU��G�t�w��>?��kQ��n��E��D��aV[��`�R�1�j������E���r��y �On��}d�������#�� ����n���y�����
R�'RG���f�:��edWP�q��b�W �
����ts�2]7y��bpK��!�T?��2�9��l�3��6�3���*�>�����z6�X��Jf��6+;���������6�/�_�����9�5�1&7�����GO���b��d��o����f���>��cLl@?�~�����$S�)7��j����8�?Y�%:k�s���P���J����4�������w���.��Flf.��\���v"���,��p�O�f,��XK�L%���������gS���"�M������s
�J��s�A�g;���l�rV	e����~�������Z�N:�?>���*��_�		���>�L�:z!����������T\[� ��6������g/���z�{y���Y����)u�����;�y���
��p�"��(�T���V�	7;8JW$@.���#LEx"�$%�VE+�9��R���{�j�_	�b�u���[��� h�AD#j!�����xG���������g����*����m��Mf�>�g�!���2hKx������9���������zW������KT)��y��n�����Gi�j��{���uN�d��r������T���%�cXP�k9��x��S��$���9��S��D?�Fp�7�G�0���@��p7`3���<�,��`��J�G~��L����8c�����X?����<8���������
��NA������4t�`�����������=���9����ttY}��e���%6ml������k��5s�M���9�9
�2��K��K@��������zo�JY�pT���:EU2����Vk�NF����B���tYTu����ljb�L|����:5
i����P��*=v��%�g9��t�8�}}���H14�;�E]q��<z������*�G��{<5����+�T��:�Q��l�l���W�������:{��'��iq�a��:Yce��+�;�q��,����#��-B�O����w*}w��:&����eovs[_�]��e�����!~x�%V%�������
�\�l�_So���_P���%��u�\=yRfk��r��������N'=�o��spr�=\bD�����y-���[���s����Fn�8���@l����T���u��W~^�_���#������S��r{0�jk��Vo���/��|x�
���&�+�Gz�9�v7m�����^��������	]�G�a�wg�A1�sk�F���Si
�}\�M���+Z*P��|}Ym9�e�OE�'jK2����W�{v�^���.lUh���]��f�����V�������5���-�k4P����]_�j��������w��Gp��N��N�����6@{����?����n������%�]��|�a�jb��6���O��rX�>�4L�����:D������	fw|8O>�R�Z;l�l�`�b��9r;�W��?���~KN�����P�O�2'�OT������imn_�]-�� �v�����KVq�.�2��x1�R�5Q��/o���\�dnCA������1���qzp���^�U��5F�:j��~�����]IM�x���c�i��g1� ���?p�'��'��J�=^�u�Ez�d��"�H�n��h#A��R�LNbbJkSk�` �|hd��S6b������?~����z�����u���6?=`���0��}�� +�Py�\_���wt�"�A��D]F������5��Y#�����<���+��Lo�wX6�Y[���y�]��%G���m���v�����]
�j;�UN���
���������Xk[�
Z��m�mj������A��G�b-|tp[4��d�P�B3e������7��	o�O�~�N���QhB�:�;y��OT#����s
F�x���}�(m*���nz��Q#5W\�o����G1�o)WA{��u�V����'i��sX5wAMk�Y���� ��L��?��a(��f�u�:iln��PK@�7�q[�ZPK�oJGpatches/0009-Adjust-join-related-to-code-to-accept-child-relation.patchUXp9�Xp9�X��Zio�H�l���/�eK�i�v��$^�"N��s��@�M�1ErxD����W��C������"���|u�o�xNC�=Q���d8L��?���pxr<r�����q��?>VtGt�����;��h����7 sF��Y����^;���?������T��4I�LyS���/�9�:��B�iLo��@�?>������w4�Y����r�3���W7o�w����K��/E���8�(U!(y������o�uU��;BO��(�-��[��At��S�y����9�o	\��T�J�)���Y�Y��as�;V|u� .2��oJ��3'�?3'��}�~���U'��K�����C�M�R��A�wg�PT�f!Dq�Z����,�yh�0��VT��S�R�"� ,����B�n�u��4.�l��%'��\�^*�2Js�'%���Y� 9�W�
*wp!�T*��+�����m��3J�����Fca��{�'E�p�r���R������x���>j�� r�.���w�uQ#%���������F��v�x{P��������Z"���R��q���f�W�����<��>�Kr�_i!d�S����Cg��8,�����A��N��6��Pv���Y��Y�PX�?�R8���\�'H�����O��/-[�k�0Y��A�M�����.�i�d��`D���a:�e��+�8��#�k��w���������H;A�����V1��������K���L;|fhS�*0�&Y��m�fw������z�p���_9y�{�*�k����<����=1�k[�b.3$�2WM��M�b�<�-�"�[��/�0����#���7�'�b���7��U�g�xCR�I���r56��SO��V��B@��3�
�@�KU�O7�Zm�<������'i�a����bnX�@Y����u�t�8�8�J(��h������*M�����ZB����fM�)l����2���Q���]p�E�]�d���>@K��~>c���u���$�&����B��U'��<~����w���\�"'��It���������.�9Y��������!���-�"��72������X��rM���3���&n��v�p�r���g�2���H����/��Sl�b�2~N��h��%�7�"��x��yX�H����������K�9�;/��8���
�P��hz��_�?lKt"A?q�DE�����j��{���/��[s�#5�v?c���i�qv���e�� J�
��������N��L�~�wQ�(��i��SY���CA�U5���I���]U����;D1�Z�J�f���������V�WYR���/�j�#"@(�Kjc-9^���J�E~Li�m��O���(j��6��:�Y�LTeq��M/\�{*H7�JWJ@�k��Kq`o���W���:cI���6����ae=e����R�l�I^�%Rk�Y�����RG�)*U~
���=\GX���Y�����kn���|j��m�L�����A�6I]q���
'L��b�1r=7���B�a��
���V�'	V=j_]Z���T���"��$�A�������d��8S��|�\��(*��%��k��pc�`��S��p�%V���(K���q��`�8A���e�w/���.���G}:\�t�>[������\�78�����yf]���K�%4C�a*4�B���{m
8��.-�?l{h&Ps��N�B������"�9���b��u,������<����^oD�^�x4b�?�D�y��?�D��������{0&\���,�O2l��,u���
�����"��X��=: �~���aIRV���Fo�n�{�"�,	%�����z(Ip��>��)t#��,��@�:���L��Q�n�vy��d��yN������,tq���{��b�3|���@��\���0�kh?H�$�����])��%#L��T�:�W�/���C�Eu�bOf,u#N�U\�h����� 7�8�5�b����"����9*;D��2S�:��i6��a�����������*��R�in:/����hEf4��4�z�{C�����t�r�". =���u�T�
�KNU���S1�K.A>���y��LD����hu�U	�Ui��T�u���8�s���p��K����h�6dG�z��C|�n���ol7�45����ttc��s������j��_-7T\2�Zz��T�M��z����F����E;h��=G&��Je'��l_A��DQ���E�.�5��	����:�����vB��XC�����[���z3D���`�� 	;�W&U������0��M���P�+��a�4I����]q��e��EoP�rV��{��`@��h����L+{�~�E�d:�$sq	<C2��j��b6_A�w��2�XN72�alM�P{�<�u��:O�`�'C��!X�B�����&��p�s�����r:4�GK+��V�Ud�<j;Am�;���j�j�V���l�L@Zy�cF���h��'�;�*�����e��JS��G����m��G������5Z;���f��~{L��u"H���t9":�1x�B��pJ�I[��
_1�*����h�Nt�6AJ����_S��rK��uo���0�=`}��g�1�n�]��-��mC��p�~2�b`o�O�u�<'t�03���Y��
'~(�No��"�y������8�]����lw\z���D>�Q�yI�!�3e����y���H
)�i����4��z��������	���!�����3�_�i��M�M~~��@���VAS�+�'>����	�}��VMQ6�!�����1�{�#��c/��q��M����W�d�R����q	C����_D^��,���}��y�>�[�c-	�~eT��X�4~�����$��(��gO�e$.IfY5������C|����w�NgB����"1���
���q��uoOk�����V��^J8Q�<���n�;J���me�EOv/m��l�Q�G4�\�2w\NM����U�,�h5�^Kd�y����%�^�k����m�u��Cm5�v��,���|�Y������N�,-]7KS�4K��f��dvt�v��]�Nn�}��������F�:.?�Q6��\���'�������&2�S��wW������s�g����1�g��r��<E)�u��Mb�%o]�EE�*W����_�H��)��������w�J+'V}�������2K��;L�HC|��TBW���O�3_za�����F����~A��-6:�
����{?d"�j���y^�y�)o�p��{�{�="�H�E���(��I�'�K����|����em���CML��"e���O���V�-�a���J9ku~�����^��-��3J���b^0f�r���t6ax�?�
��d0�������0/�I��o��������eI������Ju�|�jj�n�.��u)�P�����-]��0S+��5����o|^5}p���{��w:=���?�����x#��Vx�2�ZF�6�|K-����
}Q�3��yJ���8H�N�x�\M!��(�w�x�������&�24�1#�~��z���N���n�_�C����#��s�|�������W���S�$�]M�%�AD��~�y{�yruy�_���G������3��:d���}jY��PK�D�|)-PK�oJ+patches/0010-Parameterized-path-fixes.patchUXp9�Xp9�X��[{s����[�8�L�d=��eG�M&>9I�N^u����{GC����"U������w	�!�i=�X&���b����Uo������g����=�{�z6=��ycw�X��|6����b.����$wbr!����OL����
�<��f���F����L����y��se2�%A*}o�����_�L>�{9������\<����������i�}���������/^���tr������[	�I_��X_e:r����c��X%�[��H'b"��q��T�md$��A���)�6x;�=��{x ����Joe��#S7@?A������,�j�������p�N:��!����}BO2�f���z_/?^^]�[�������=d����O��'��6j1k7aEA(�[����|��	-DQ��eE�u�O���^a�1���*iG�~�f��� �
���`�X�Cj�nI����`�����{�>f������N��is�"��*���C�����l� �]�� o�F��:On{�0�+�����/�.���|x��������^���s����]��d��x���t`�0Y K�M���	�&���J��Qy���>�l��PY*�]�b��$y
��`�8M/�4c�F)�T�
���s8H���E����R�o��v�L1.�l�	�1p����_��i�w���z���x�T0��/-�wF�����kKm�+���5�Z��-"�&�%�,��%������y7�a(\��>�����_�2��������1 �=���>���i�%F%����?������|��+���
i��B�y$>H���AmnK�q�RXp����6���%A&��Q���w������ 5�d��i�����,�:��%��t����`E� a�;���3�&�S�]}����`��)�����t+!�	!f3����Q0#���P_+u6�Q&�gAH3F�/�d������A~�>���S�#�{_�VK��?4�L��s�|���eB����n$���������N���Gs��P��a�q ���px@�:mg��n"�
w|�X�����{g���a|~v�.�vF4�z����p:,D��,����8=����H9`J�
~#3L�Y��O\�'OM�Y�D�`H���S����	��)����w���x*�����'�p��%�)*@��i��	�N�������c)fY��:��f�O�������~�X����kC�������Y����$�m������a����.?�
22�h��G�"��3�}K^�
6�^�<�����o_�t%~X$-C>�80uE������Fa��R�	Ef���0�.	$�����R�\�I�����y%����)�5_���2�l@�c6Y�E6]��=hvL��5L�������K�s!��O�������\�b�Ib�RS�F
���k(�W�Wb��A���8`��M*��;?�4�hn�sS\a�f)�8�y��{���t$87���.��I(��1�c�c�Z�xP���-��R7h����+�2�����`����������
�mq���8�gn��	��B���c����}�����[�i���"M@�G;�c�*� ��5�Q���}�!k�y�� C�q�O����f���~����c]�}�`Z�O�=�"��>�`��w���@����Tq*%��)pb?��!��-�h&������|�������d�GL���
��0�[B$L��j#�.%Rf������F <���������[����V.}�>I�O���m+tSF��@�D����|����h�-�����)�:p���'���B���9�p�M��'
���p�-�t�,I��=�I��u����a$���D}�������Dp�c���u�%}���$�������P���sS�$��s����RQ�O{������	ii1Gm����VP�2�IOz�}�{�����J��C�cw�RC�"<L	C������	����N<}*�=g3�70*�HSQ��z�e>������xX���l�B��'`>�N9^|��]�'e��p��ac�5w�57���T��4�&y4L�&�&���F9B�K
��5
���*t����S�cf���8Fa�"��c<\i��6�T%��cL,�4��}��l���$�F��-s#�z�+�(7��B�����,����<P�h�.��6��hm��]�U�[�<������-q-�zi-���&�����&a��kY4x���Vkm������|0��>|�&��(e	@=Wr�i�xU������7��������Z��Op���L���Xy8���"g��8q6�H�R�7�����2JkY�P�<���0�"HDn�I���kV#.X<Hv��W������s��c���������+t�_�l��p�p��C��
)N��=x�6.��*%����Q�����Z5,C�����K��Beq$G���Om�-����Q��X@A��*����c8��6���W5�%�g��d��l���G
�	�
�{���ktj+�
`c��'��������DQ������������_x[C��� �H�>Y���u(mH!�"�e������������X�lk�,����`��~+����P��Y�2��W>�x~��!FE�1��b��q�w��/�9;1������S�'I#:9����M\���p� 
�0��@���~(���t��&~���!���22�@1(?������7V�B���M����Z7\�����ma��|�9�����[��gaZ����s-���9aY��BL,��n	���S�Ig�X?�VB�Ek5��Vj��2�Uc��a5�r��M����rR�S��|����h4��j��`S�t�^S������|q�~O��kj�uA���\O�3�:���%�����C�I"�t�L����^�Rp���
��Tk�����G�B+Xh�n����c�J*�����\�.��3��~z,�\�G�ik��*��n�4�����S2���U��7�c�F����%�D>��2T���/�/1��.��y5&�v���K+�h��W�qa|eAw��Q7�D	���{|�/�>;��?X�>���<�WSyl��5u�a�lL���C��hdA/wA�;���N�D�$H�1�LQ*��i����I��Q�M-S��[�����@yY��b���O������o�����_��l�G�Bic�]�m6]�H��
3�o8��H\/�O�,���}_�i�[�[���V�y5�@�����D�(=��_�]�����nj���G��&��Z	�7�o[,E>��u��V?X=��9q�o0?>X�|T+1
�f9���`A�a�$1��E��k��,�=��H�!5bQ�����_?<V�)�A�h��}�UEB��F�
xt���N��I���Q9�������%���{Q8���t���I�3+2������&�e#�?���j�oth�P�y#fq���Jf��u3=�IuA����c^'��A
5���Fj���F�`���������&]t�D�	���BG�cF�e�.q��VWCMg�R���0��+3�<��`f��J(���f���T����
o^�}�������T��i!��I��$�uU�*��CX�^�%�o:�+����h�0��������� m:2�%O��h��ic[8�w��*)�ZJ��L��l���*%_!�Y{PO�/Yl��j��m�"�i��p��Sr]�b��f ���N��!',�0�DB �3V��j�%HgDk��sl��5'�RSI�}�*b�H�Lp	X�:<lh��Mj��\6�3G�!��\<
M_��b�;[����R�y��m�m���RMsY<���U�@�>�.&t9u��k��`��-��=����|�b;��:^����4��	+��}�B�G�$�M��6��0��6�8���������T+�l!�Z@Y��~��cY���g�&�G=�h���Y������UGRY���S��8�=��������p�cQ�>+n&D�:�J�cHt�?�/4��s,D��Mz�:?���Un��������?�<��7��Z��-�Cim0S��r��P�����^�q������5n����En
�H�?��&&U��E~o{DQ-wN\���au�T����1��x����/;t����tG����;7���#s�`��o^�	�A�f @�9�����,I	R��O+�tU:����@��xV���7@�K��W�S������w���ue�����;�*m���������"a��x*HxD�2�������Q��������c�U���y��#bPc-�[�)$�7w�s��+.�j1��F���y��Z�(��^(�x�0�
�>�o62R���N��@��#��������,`����C�
u��<���������[�����x�M7����&�25�i����]>�=F�H����y;B��b�h�4/ST�Wu���^#PE��XA���B�w�D��#���������f�����L
�=H�_�B��O��AaC�g��"��������p����O�+��~���u���*�s���yxsRk��h�-��\��R����6�-����~��1�8���(C��s�B���J#D���=�����Z�����"��$,���G���W��
�����__._c>j�A����^���G��d��-��]x��V���L�s>G�9W�ZJw�H�������B)F����3F���6���0�Ns����~����
~����W/?�3�����b����PK��9E��CPK�oJ<patches/0011-Use-IS_JOIN_REL-instead-of-RELOPT_JOINREL.patchUXp9�Xp9�X��XmS�8�������@��&�W��&W:pd�����G�����SYn������$'N�������V������}�N���a8�8#���r����7�n�����Y'��p�L�8�j�?��Z��G5Gp�Mr�fx�fL���|7|�~���LD�F�NO�wL�#���:t��}T����Q�=r:������a���<�?�7g��q�N�/��q8z����{�DI&9!N\
n�"�����>A����gk� S&<�����	�`}�	3����>�Q���T6`���h���}�Z��x!T����
Y�G���0e����2�ru��w]���K�t�n��\�R�F�?�����x�m��Y�|���� �d�w�j���e
�i3�
k
�a����8�������������D�J�Uh,� �Yk��m�z&�"���4�c�3o��!�1��F���uZ�K?������-o���24}��$l��<'�X���>�=��(��8��%cb��&�q��T����!����k�F���8���[n�o�,~���k4��8!FJ���n�[���.��y��C����p���v���������!*J���(?�a�$�BM��4�u@���43<�[P��w@���XAL1N�|�n���r�15�q��Z�BAC}7-�r��_��b���t&�j���zT|�f�����M5Z�JI���������M�^=a���@���[[�-j���/q�"����1u�::��������������;�!F�;Y3B�!�I����WU����w�;��h ����mO�v��^�.@��o���K[�;��,r�"mc�G�p�Y�I)V�4�NkO�i�%�T�z���K��-���|E�r��F��i��|<2����e�������7�4���G���>��He�}�;H������� PtP8����H��W����P��t�z\�~��x|���IZ�LP�_]�������c���i*.,����e�t�sU�U=�"�U*��[A���8N�j���^X}������E�z���Cop����Q'K���R�|�_<���s��*8�x��?������M�5��������"_C����Y�����+�F�O��)c��*�G�i��4��'��J������>����*e�8��=g���u�]<h{_�����IM�v{�D�1���(�=�`*�����E�����ZO��Uk����M*��W��)��K��I����3L:�m�����I�DF�CaOO���<}PU�N�*�(}�T�����W���aj����:$a]t�n�_WG��m�h-|����8M�?��;�4`���t@�3I�4���<�IWe��?��!W��ad�Az��2����e-��Qp�����I?h�4���������i�2e��k���j�6F\/���5��g@�RFn���l�J�N��W*|�%l����IV3}��q�n�����MQ'��� ��T`l��Wz%6����JSz}�(�H��9�b�j��E�q>��qJm�r���H�xY`k����)�����\O)WQ�����5�h��������^>�x�p�������C2���gJ�T
)������'	�T�����Y�w�~����,wm�t�y�e���-�Z������uS�;���t��rw�=�Y-��5�b�YF�y��h���4��Ca\_PD}�P !���n"5�������1��N�n�<H��/?��.N11Lv�`R��:,�W���Z��z���_�\�Z��*(��Ay����
5e-���<^<����7��g������#�P(�l���$�U��ACz��oG�3����y ]1���b�ZN��q��X�PK����`KPK�oJ:patches/0012-Multi-level-partitioned-table-expansion.patchUXp9�Xp9�X��Y[s"�~�_���,,0����V_*�xm�&uR)J�3�03������O����j��v��.�Vw_����G��;�N����V��}|��Z�~������k
X���%���'���V;���^���ks�r8V��O,a
~`�����y��HD ���x��s��)~�"(�[�k�Gi(�m��N�&�jG�Z�i���{�~8�^�n��6��/�PN���&T��8�>(�9�o	�$�T���8�J�=A4�"P,�8.���S����P1>qiPC>��������������8�p�A,F�F�O�E��+y�q� �W�3��G~5NT0
�����,�Q�$=U<��*��<K�D�#����>6��?��c�).UU�g����
q���Z�U��-�1�7���Ar	��E��/�q
!��=�B�X�f|r���y?�q�����;����A���z�Y��k�v�_�P���M���k�q���������rJ����x�c��l��(Lz&��qq���I��*�!���=��>Q��/0������2%!�aS�	������8�C������3����������_��Z�;������n����+|,�K�Suc��#Dc~�/�{�}�'�A�D�!����<v� ���+`�P�	��X*HR�X9�<d��8������)����L@XB:!���� o�>6W�;d���>��Xr=+��J�����1Na�aF��*_�I�^G��;P�8
�����p	M!��q������$����f[�	�+�i�X�\��z�mY~���I'�8�Q�D��5�0��=�~��R4w����C��XBfA�����_��{�����wfy�'�!q��l������E����f(�D����K-�����nT*n������(`Y�V
XL���o|�
���B��3G�0V=bs��yq��&\�a�~��N&����.����"6��H(�����'�g���W�	u��'5��q�.���`Z��5�����aex�$���W4/=i�o�W0p��r�q�����+c�e�|Y�{��EO	L��4=�/���1!�<�6C���#>Y�mt���LDqY{�m�5���C�I�\���8�����\.B���;�����ypFH��u��4G��/DyJ��� 9������}bdI$��"��������+QlE��:�1]��@�[�8E��>*��Dw�3��8r�S��)L��O��g"�X�A�4��M��I�V���\�����]Rk/���2;N��pr"L9�7+��:jI���a
����Rh�g`�Ho��<S�;�),�!�T�&��8��q�[)����q��O�����g G��j���dy�w�!@u*a!�#��x5�+P�&Of�X�1�yC3!��&q��Bc��x���(YN��)j6�J�J��S������(�N�r��gi�2c������c��M�LW�VU;U��j��8H��c��6#��[�S!0�} ��c1`��Q�3���$t��7/��+������	�<�Se@%�<�����������y��������J�`EqOC��D�B�J�s-`�����[k�>��r��f�%�V��{x��� +���O`��$���.
1���$�2 ���F��)t�k{�xL��������gQ[)f�
���y�'�	=�
���� A�!��j������)Q�����mH�Q� ��G���H�u��|&�(���1w`*y���l	�#
��g��]��Y��y��&��f�mP�kw���I�h?	�4�Y�u�4�E6�I�B���_,�e�$Ur}�yg��)6����*����b%�^K�������h�7�����9���0�ejJ�k���'���G�*��Q������&
���:��f
��xm�o}o�zq��t�1����lB2���J����-)mOG$��;D2r�����"�`��9H"�E�3���!����Xse�o#�{\w���!�:�.
���d3\_N�j`���K"O���_�����%8�6��������h[�����u��nC��)fu����CZu^�������
9�h��m����������p�������]]���?�^�o����a��p@�M����0�j[V,����c�`[g	�w����K/n�3�ZG�4���pp@�<�e��j��w��EA�����-�;�%S�ur�8{1�~5d��P?�y�� C�J��Ft�(�S�g;��&�5�N�EY��4w/Mb9��XD�~���8��"���r�C}���vg'uZ@�}jI����#���^�$��a�����4/Z��[tq�6;��[?��L��#�'�{������i�Fm��(j�%�b�u�6��^z�I��2�I��RJnO�p�@��4�i<�u?�����XPlV2w��ztW:u�2s{���+��-�p���a?K��P��2kK�S*zs��'c�����pB�f�(�G��
*����Y��;j������t�.��%�_c/�'s2�-~1�C���G�
��@���V���:�e���� R2�i8�r2Kmf���zz��f���������+����t��m��4o5;�~�V�4�#�������K����>VGQ�V+�GF���QH�w�����6(Z�D���
	�}��#Gq�<��%>8���_&K"���_;J�����Z���[���t��^cKw�F?n_��%����/m�o�|�����m	���t��%��mK�f'��u;���i;$��
�F�����a�P������H�Y���V�|���'PH;ON����]��x�I�A���yd�/��u'!;a�U)����>M�`�_c������U�z�.��|]N��V��hA��4�$��w��a%�&SwCW�u�g�b���Q���K���p�}[Qz��Q���|�6���o\���>��@�P������6�x���2 >���u�}x'����
(Sk�>��y�����j�������"n/��#o ���9��~��ho��&��"q/F�K�������z����+������zs���[+�������:�V��N�(��?PK
��'�-PK�oJ8patches/0013-Multi-level-partition-wise-join-tests.patchUXp9�Xp9�X��=ks�8���_q�nm&@@������t7���t�%�������Q�gL��tW��_I�������t�J�������9���c�C���0:=����Cl�-��0��^�7x5X.����`��5��
������?@�^�qA������=���7�F������`����#��1]�\t
��o�w�G���1����YPhvp�u�?����{��;1�s���������O����{����E���L����G
~��5x���n���t��vO��
C�'������o{�Y���{n����`k��'l�����'�nY�k���o�n��Ao�8���7������m�V5��������q����+�l����5���s��-*X���f���!TPpo/	�S��j���UVLV��4oo��Y��'U��E���5y��L��0�K����������A8Y����oY
*�}~�:�v��m���_7�)����}G_�1l,}M���f<�����9�4$�����z�� z�Y������x4�l��j���-h�T��6,���CU�i�����rv9��7��tt�����~B�o�������d
���~���t���C����R��@pX���p��!B�/|g���(��7���s�
�����B�Z\���	�T���|)Kp�@Z���"�Zu
���f<���5=�{3������?��3%m��.�3�����VdM����E��k�vv���]��~�W���	��9$:'i*n�E��p��l�nr�;�O_P�E�	�����	nm��5��
�;�+Q�w�0%�����!�@w�@��������������2*+dT�y���*�<��y!UtH�<�^�I�����@b�!dI��:$u��y -��"�P��@�;��<P�y ��@������8$v�y��8��Q������������]^C������f������
��E��y8������������F�+���H���4��y7�+���w@plD�:��FO�`2}7�2%G��(�������_����r���1����zc;Gy�{�.����MX[�?�)h�Ou�0�l�z������.��\^RJn�[{�<�f�����k����2Z�����+M/<l�v�1�n��1�5�Q���4����C�m��c�]aK����p���s�Ly�`�0-�R��������UA��p�y(U�_��B���\���t���|��HB��8�9ueQ'RM��})�F�����K���P����lzY5��|��'*Iq|�%������,y��J�x�`KI�-y�C���}XA�8�<NIg%��$�sRct8+dC�q9�<��B4Pq�8�cD�����Z8���$#�\���� 
�R�%���`���G<������� �]�R�u���S�N������4�w-�{S;�~t��_��_��w�n�=���>u�Y��
�:���z����`�+�7����^��]|@�5(�+?�,���]}Z���}�	W���}B���\�r7���"4�Wa):����4��� U��aY�����,U��aq*g��^�����D-9�a���uX���J���D}��A��/�N=-����%Y�����g�5�G��(h�����:������� 
[���-�^�h)��>���NH�?��%��e�����������*Z>�TGXJG������� !�	���L@e:�m;B�x]��������f��K�����/����R��_5�c�`K��#s�LT��Dru�V[5gtci�G6\���1�7PRYr�!j�(�_�=�����Ju�N"2�V[qguE���-�P�u�|��AY����Q���s�`K��S��D��������t�tgu'7LS�e/��w��\���V���*�;n�_�lOl�[�.q�]�'E)'��8 W@��~j|j��}J�hj�������^�}�iq��`E�vC��3��W�W�.{}�n-���>�r����)Q9���x:��-��8HS��P(�� �j1�_�ru|�/J��	�B�h�(Fk� Jj+QImO^�o���6D�8��A&�� ����lC4�*��j�����%���6�jEF4�M����&�WI
��2R%���a�&��D1j|-"����D����
�2�@�ClQ	�CY�CBD2-D�)"R�E1�5����\/b*`C
�X�R�����PFMQ��"�4���"��"��V,���UWS����2%-u5U�zPj\�����'���
�����$�%��*%j�S��C�JI�u�lP��Wk��9s7�%`H�+���C�Rf�R��(T��^!TT��{!��v�I�P�B�
�[���Ji���W!�L�!�-CN��CT:�vZ���,��3����D��M���e��O��_R��6�u<�cR��d:����
��������W��x:�j��w��M��?F7t����B[�I����,����6���x�{j`�N��@���5
��M�[��!jq8�u���
�3
E���}�8l~�"���mQY�����y5���[��Z���,�M����\����������?�l{W��L��������(_A����6�g��H�_� DG�	wra�kD�i��Qe�C�kCb��K~������b��"��K�+��2`o4�-�t��=�n����ETD������(��6�h�G����q�������b\���9���2F=?�R���J�1�����\1'N�q�	�����{�2��5al;��#����{8��BtU���z�,6������A�����3�nQWw����"�s��+� P� R3�JU�1����\�.N�q�)����8���0
�E��s���=D\����|��O=Q���yzU�h�G���
�-��������B�Z@����]���R�K;�\��U�R�=Z�����+=[c�8�R(�����"Z�bo\����C����T���l�P�q���P���,z�H���]�u��j�#��8����(NyV�~ha��������1����i�Q���nh!7��ZM��������|������D&B�d��dT�_#8X�_�Nq��i;Q�Np=N�I�z�������>A�/�K��&���>�����"�L 
��9�yEA�1�O6�(>�!��� 9�3N���.��dr���A�@2�)Hy���@	Qy��M��y��g�|?c��������`���~�x�!��!k���hzw`X����+{��ov�w���(�t�?��o��a�Yk��6|���g�a(�>cI�t� �(Y7�^1�P�P���2��2y��w^��R��L�E1a,�T
��	�*��dk�J��2�X�8�l	])�Z&���>-��4��4��T
��}����7���
��������ij�J��"�	j���B*YT�Mj��>�T�Qg��I�H�l���b����W�<a�K-U/��]����\EI��-{��1]�R����2*�[�W[<a�E-�j+����\�B��{�P��1�/WN�PF��z��	s�5=����#\cx��o�y�J����+���J�Jf������q���������){+�{�Gw��m������"��<���Y�ggk��24��W������?����mi��B��2=�O\�8����8d��=���nY����Y���B���^���_�)��v��g���L�N��y�����j��~
�~��p��	��<���<����p�
;h��@"�H�Fk��#����}�0����	���"�70����t�e3�[�!�j=��|��]:E�%a�
�6>��7`az�����`���0���Dw�;�_���n�,uO�G�T�>�[���k���[s
�}�`���in����K�e?�;���Ks�
�t����K�?���q�}�Q����}�3�y�j�!��f\��yr���,X��"{��������v��k��g}^�� l@�v�������0�$�Bm�_�q���t���Xa�����O��Z �
b�m��Rn��hs�"Nt��t���\�Z�#��}�N����D��j��#��T���D
���n���3�3�dIN��eI�Y<&cq���n�i����fL5�w��6U�.����e��������<�&���ju8X�J��B�qW7h�v:��A���\Eo]��N��k�����L]wZ��h:��]�Y�[MG�?��z�!<��O."��)�:��e|�z��bH7�4A
	�Ab 8,z��
MC>�_6��@GlRs�u�&Gyp��K�rX!\-.Vc	�w*c�B>��%8g -G�X�D-�:�xy}3�������
W.f��y��)i��{t����Q�Y�5qt����#n�����
�&<�]�������sHtN�T����!�5�����fw����N��c8Dg�M�����}��_v�w:z�b/nq�0%����|�.��Cx��f����bOM�>&+fT�}���*�>��}!UtH�>�^�I�����@b�!dI���$u��} -��"��E�>������uH�>��} ��@b�����z^��$����2��������;.Y�) �|�#<.������O��?A8+�����O�����g�:������B��}������K�������N�~����|�|����rxP����|��/S�q6��a���5�PK ������PK�oJ:patches/0014-Multi-level-partition-wise-join-support.patchUXp9�Xp9�X��=�s���?Kz�7�bI�H���6S'�4�s��v��v:J�,&�:���m����M�n�:�d$���~c��rw�0&���uf����o
����a������M������Ks������������O�3��A��y�.����y+���[>��%~�f���������x�b���6���[����r���+{��`���tp�N�#���=�>�s�)����������`����������^�����������}�9[����������v��P��^_��7���c�e��^0���m6{o:�{��W��f0k�f4������l�����W8���u������m�����������]����zl��/��0~o�+���|������3���!D��I"A���
��wlf3������1���a���x��]���G4������ov���`5{�n�x��[[�O8�a��Z��l=�������������f��&b�����L���x��
Bww���1g���0��MgS��j���n��<w~:����z����l��{�mOa-�����v�?Lv������%�L�z��K�����z��z������D����N��>}���f��-��	�IQ 0���Ga�y�1�������}��y�@6Y��b�����Y_/�7udt�e�<$��z�S`BP9���8sJ�9
Dl�<��Tz�sI:��������
<`{k{o/:�<{�����I�� ��^��N��l.���x���9-���b���vaa�r8����������s@�6���	S-<�����q�>���@+;s���
g�S����%Z��������h[�E�������Q�8��������� c�
*P�p(�A,����Acq��B�waZ�&A�d?xj��n�(d�j�~��
T�����
DQ�C�.~��h�$F8h���vJ�M������6��w����w��o'
�<l�4�R���������7��x`e����S�4).6��9�f��e�w�i`�[�0u����4��R�9�/�'��Ep��r���f���mwlK	��4![�*:Ck%�8�,t����V]�d�V�������x-��aIl�n��
��������iZ�^o�����ya��@��J[������r�o��8]��|$c�d�c������w�5zb���o��gD����~�f��o�l�'->"g(q���6�T���t��������	��I���e5�W�1	a�-�M(3>��$[��u��m�a�*�������x���P�~:S�?P����&�8c~1���<�u�>j7"+,����-v�|^$��2������F��C�����b�9���H�/����|�U���������g��1��Q;H��hHt�������!28��
_i���"J��@=a��mO=.v�/I�����1�{���J��h����
�B5Xa���n�a��t�`>5k����5��n9�\���x�ws��E!�3�v������	���$���9|�Mqq���`�!�;�,������O���b�����H�jc�.6�zB0_6A:�O������"�D,�M������X,�Cc0���l9�M
�1X�E�5��9�����h�_�h�}v��<���.�aCg5���I��kkn�v���f��<����@����sg
����G��t�FC��$��l{2�^;��(�B8B����L9kq��L��	?B6'��C5Ge,0{hc����� h����&�;�XAfG`4�65������0u�����,�{ ?x�,���O��d����6�^��[�77�7�����E��@���|����LF���I�q�.�\#^f*MU-l�b��%��E���%��p��_����=10��1�j������^0�W��m��EE�<u���������@.|��$*�{������\,��|����n���k�(�M<^�|DS��:<�P����l>6{��5��b	�(�L�m�|f�����p�0O=
|.��Z7���C'�=a��<DtF�<���>&*xG��p�g��~*<&�sf<���^>Q�����f�)��R{����?�����le�����pB��O�y�@�)
A�S�A�L���OTXA�-@}�i@*c_$4���	�A���N/�����j�IR����T����"�0`��b �X�����o|����,�����H:�D�L�/ ��h��a+��=z�6���l�Mw ��Z�Hq��uqm��J�8��L0�XS�m���H(�4g-���n-N��l��P4��2��'=lk�b����$
�2Y{��� �	����q=�Ik]b��'����8�\��>i���o(`&�$6�#U�/b/�6���ut����j�#N|i�������?�"�"#�V��
�����w�r�w��=G�����B�b]�$F3�}>D�$>��3�P��hz3�pn|I-������CAk:[��'x����%��1F�� n����>Z?�R_�/�	��������������|n�O6��.�XY>�6��&��u�c���L	M�8��eZXJ�T<� &�'�sc����}W�,lj`���-�g��z2�D��21��8R0��h���x���q������9a�[c�]_'�G��9�h[l�`2LG
�m�j�U6B�����V\��x1s	�������E��������y�N���3F�Z�RM��G;���T$)u�y����BP�PBv�g�	���:NW�13\��h)�dt�����{���	�n�2o.����M��^_�L�^�}_p/�������[S�#h�.�?b�<o$���dRM�6����������
#�@��~E���0A�f�Xmk����hN"3NZ�"��~���(��\S��h��V<�{
�6E�(��u�P��jP�F|�(s}qq{��Kvb��7qpHn�?^�y'������@Us&vdC�V:�
����H�m�v�E��p�l�i��F��1��e���X^h��!���V"m� d�w�f�ym/C���8�+%h
�z;���z�zI{�14��1��V�	��[r_rW��95����	la�o��{���6|����;��vd��$ ��{��E��s�%�r��knN��5��E�}������G$�Dn�Io,!�����r��|�h)�����E�����z��Q�t}�O�
�@����9t�cEvqOw���y'g��*a���=��5��f[�!��7@1���Xr@��4�����pT����cD��
�=�����$�[�Ihm�&qRb%b�3��X�Z%lD�!�W$�%V7��������a�l��9��u�zDb������i��V-�L��18����utd)���v����.��d����Q;vd���pdj6j<C��B�,{/0x���E�<�[��|u�#R�
8������{�L��7����	�v���rBIq����k�P���G��1�����ZH>�L
)�Q���k��Y��Q#�S���hy��I�����[�����#sb
�F!e��D����9�|��,Z[�Kfo�&�5��l�9��~�����6}u�����
z�J�V�D-����xsd"WW��J�|m<;/�����*�)�5Bj��:/^E1����92���;s?9q�U��t�/����g���n@|"��p�$>��z|��
�Lxp���������l�z��@]by��l$��q�j`��A���k������)�9U�G�W��(��")P�**(�+-At�>t����\b��O��
��-tY6F�^b"3��>�uj|���`��pn�G��Lo�����e�z���i3��)��nPz)�B�g��qh�Ek�x2�,3�6di{/m�<��i��z������j��@7�U6�c.�!T�2��������~����8���W����/J��VN~I_���R^C�������J�V+�(��JJ�#�@�=x
�����$rW�������"�b�z�J�t�����-.��������4�0"��?�� 2�.��S����S���G������hB���YP��
2\���zL������"�&�v�yY_2��|���@��X�L��	��
���+1L�0/���{4��K�
I��a�Tj���%O<�N<t6��/0�`��g�0��m�x�g�Z�����?��-#ru(��Yv�0��ogM0c7
�����C��<�>/}�L�K��1S�eT�A�]�0/��a�'���P�QO�h'pT\�q�Y�"�
�c\��j4>r,(��a~���h�%K�o��'���tq�`;0�)�x��R�$5��KI��!��z��nLk�W[et��EW��be��4������8HO�%VK�G����~p�I;��8uW!'Oq�6HL�T��|IY
�($X���G�Q��������I��7����8�7��i����\�|����?��5��'5v�cF���A�����KGg��3G.�rOY8���Hu	m�8m	�c�-B��8XkX|j#�x�gR��&N�v��$�ymr�n$��~��8�d��^��H8����qbv��gN�|�|a�T)'�����p�c�rg&^�������h�:Z^��4!������m��L�N*�Y+��c�.�P�L���'���9���������������?M�?�]���a	����A�[�����bBuM&4��4;��5W��W4�,u��l9�&����q�X��KjN�����}^�����L�b�L��Bp<�fa�{XU*M?m�����H,�8e���G�{j���#a�LP���-���V�����/g\������8���)��j�tW"�d�?�3ILL����;��W�'�����u���(��}N���������?7��q�}�P�6g�T�]g"��3��q2�*@���)��������T3
�gX�����!z�O���|���tg����M`�oitj.0	T�u���MP�Xxj����[�{���~�������9�����������)����AF�|?�� ����w�ZQ#�l�9�����_��v�E7w�������@�
a�*CP�ZP5�`^���R�D*�^#�6�?_%�	�3��"K�^�
9�u�>-�Bl�ku���U��p1��h��rm���%�����=v�kA����z_��
�"����]W�=!�V�����g,��[&X$�wr��nT��:neyX9���Z
k�@�=���v��2����tA7w�����\4����B]t�V!�`YB�����|�fzE�M(��.�td��f���N�[�3st���@7N��NL���&�%�#���o�"Q�!��+P�>:����X\����,��K�O�H�b,'x�����s�"�^a��s�(�b�����A����2�H���2���A��n���Q��D��EO{,�S�"�D}|*_�_�8�~.8ci�-F������� �3���o��qf g�����l�KC{}�J�pD���5�.1����N'>�r� ���r�����"K����wo���������������g����u�3V�_��7����3uq�����9��1���G�e�n��d ���g��|N�I-�*;�0*�-	-V�Y��S����L�O��y����M�y	��_��C:����
�Pz`��R������u�K3|P`���.��Kig��
I��g{<I�-�����J�C)����8���>����K<�}w������oL�����^^]��C����-�����}x ��.������w��������������]�z����zV?�G����>���:bx���������Az��.�}O����m��a�=Hv}���
{�W9�w/�M�����7e0�w��7��q5O

^]��$��x3\���\�����������x#�Q�����usM3�(��������;p�������8��~���_��f6��Au>e�V��md������!3�1�
��!��8�=��X7�r�]����C@�D"q{�~�T�-hq��F��N<�g�h�S�F]�#�F���9��1�b�=�	��R��[p��;��j���]�FJY��i3�[M~�25=^�1�!VI(��+U�\C[#�>�qRe>�R��zq�RRY���KD3�Ja���Y�t��g�U1���\U�_�XA���LY��g:~i+fd�X1
�`�L�Kr[w���27����0S,����Dy���ej�eFy�Ty���}���Q�?��ZDS,�(H��#A�Q�/������i��V�/�Fc�'��D��E��
��3�5~�U5���
q����C�@b�����]]��c���j��2I���-�,���5������`R89+����N�`2�[P�S�gT*��-��E���$�[��N�o��oiE�-��f��]�4�m(|�`�|D����|�Y��+�������O,;�4�!���qV!�T?�`NA'm��u������t�Gu���R(�x�\����qQ1���Qc�w7�j�h�����e��>��KQY��c�s�d�ED

:�{�q'�wT`������Y�r0�$�t���cg�+�K�.#��1�wj����u�ZV��Xry#��s�T)�LA�r�x��b~ytU
�����FtU3#�p����r\,�-#�Yqw������*��I�N~�'~e�d��'�=���JL���/������/m��"���&��G%���&	���<���w"qzJ�(���GM�G�(���s�����J���.nH���6�r	�c�	����-�(b7[��0`�T�5�d��9���Uv�U�SBv���������+gF\89�%��F$_}���8��j_����-���X�����vU����1��u�A��e���A���
�1rB�c.�?��L��8D�>\RM���u�IAY-%-���9~����H�B��t���{��&s�N����2"�eD������t�xZ&m�>�b����l�ps�l*�t�D�h�X2�2T�#C����}�>��tG2Q�m��b���|�JV�K&�"�U��a%FK�PR�J2"T3"�dDp��8�a�|d$����G+��UR��$�n�T��yd�V�$���� ���xNg�:�i�/�Lk����X�D�@�V����*��"�������b�Vj"%5;� "����LU,L���
�"f\�D	����1�e�<7����J
1&�J$6�nM9��3�c�n��rFgf���/!���NC:�p2�h�N�BO�k\�.�.o.��}
,�������-A?�!H� eM�����H�<�&����J���
=Kgr�S���$BIi3���Y1��^*9�N�2���Z�J����\���������{���D������H������J;�?
:&��`�Q�Tuy�����UQ��R�X4TI�
.�c����=�l��z�����*�a�V�"���R$!�VM~?X[��%����������ND+�r���A�^3�9C;gaV�7
'��1o�,O��2?I��J��
��(�4/�CV�NJ�B��$�_��(�z7D��w�>���6�ZD����JeM���`]Gd��"����]�0���d1�(���Z Yb���(��,��+��W>�R����\&�JYXC����i�J�p��EDf��D�a*��Tx�@�Z,l����&��D��N9`�k�Gr*E}���P+wz����#�Ty���������R����������c���)����V�Y�G(�J�H�u�z���Z�:���I�NX�*�Z����R�Y�q~!�"��t�h@'I�[�=Y�{^�L)j�z�8;J���x�_3u�#�bu�#�mj���
������/;�N�%��7������{G�J=)�#��Zsvl3/(�O�=b����Z��Z��}����M����`���o!}Sc����M��U�4	:C�J'gJ��Bny���#�
jITJ�?HX������=%PS2�8�Q)V��2J�������������*af�d�R����������4������#�����A��E���"��!��!�g�A��e�!eM�d#��!B�a9�/!�]����Io�l�?PK:���N $�PK
�oJ@�Apatches/UXp9�Xp9�XPK�oJs\����:@��6patches/0001-Remove-extra-line-between-in-allpaths.c.patchUXp9�Xp9�XPK�oJ�����U
5@���patches/0002-Function-to-fill-inner-join-sjinfo.patchUXp9�Xp9�XPK�oJ�y��v7@���patches/0003-Free-up-memory-consumed-by-the-paths.patchUXp9�Xp9�XPK�oJ�%�BQd32@���#patches/0004-Refactor-adjust_appendrel_attrs.patchUXp9�Xp9�XPK�oJ	;�Q4@���2patches/0005-Add-function-find_param_path_info.patchUXp9�Xp9�XPK�oJ�����<-@���7patches/0006-Canonical-partition-scheme.patchUXp9�Xp9�XPK�oJ8�vEJ��,@��aKpatches/0007-Partition-wise-join-tests.patchUXp9�Xp9�XPK�oJ@�7�q[�Z5@���patches/0008-Partition-wise-join-implementation.patchUXp9�Xp9�XPK�oJ�D�|)-G@����patches/0009-Adjust-join-related-to-code-to-accept-child-relation.patchUXp9�Xp9�XPK�oJ��9E��C+@���patches/0010-Parameterized-path-fixes.patchUXp9�Xp9�XPK�oJ����`K<@��Bpatches/0011-Use-IS_JOIN_REL-instead-of-RELOPT_JOINREL.patchUXp9�Xp9�XPK�oJ
��'�-:@��patches/0012-Multi-level-partitioned-table-expansion.patchUXp9�Xp9�XPK�oJ ������8@���(patches/0013-Multi-level-partition-wise-join-tests.patchUXp9�Xp9�XPK�oJ:���N $�:@���;patches/0014-Multi-level-partition-wise-join-support.patchUXp9�Xp9�XPKb�\
#50Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#48)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Mar 15, 2017 at 8:49 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Of course, that supposes that 0009 can manage to postpone creating
non-sampled child joinrels until create_partition_join_plan(), which
it currently doesn't.

Right. We need the child-join's RelOptInfos to estimate sizes, so that
we could sample the largest ones. So postponing it looks difficult.

You have a point.

In fact, unless I'm missing something, 0009
hasn't been even slightly adapted to take advantage of the
infrastructure in 0001; it doesn't seem to reset the path_cxt or
anything. That seems like a fairly major omission.

The path_cxt reset introduced by 0001 recycles memory used by all the
paths, including paths created for the children. But that happens only
after all the planning has completed. I thought that's what we
discussed to be done. We could create a separate path context for
every top-level child-join.

I don't think we need to create a new context for each top-level
child-join, but I think we should create a context to be used across
all top-level child-joins and then reset it after planning each one.
I thought the whole point here was that NOT doing that caused the
memory usage for partitionwise join to get out of control. Am I
confused?

--
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

#51Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#49)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Mar 15, 2017 at 8:55 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Sorry. That was added by my patch to refactor
set_append_rel_pathlist(). I have added a patch in the series to
remove that line.

It's not worth an extra commit just to change what isn't broken.
Let's just leave it alone.

Very sad. I guess if we had parallel append available, we could maybe
dodge this problem, but for now I suppose we're stuck with it.

Really sad. Is there a way to look at the relation (without any
partial paths yet) and see whether the relation will have partial
paths or not. Even if we don't have actual partial paths but know that
there will be at least one added in the future, we will be able to fix
this problem.

I don't think so. If we know that rel->consider_parallel will end up
true for a plain table, we should always get a parallel sequential
scan path at least, but if there are foreign tables involved, then
nothing is guaranteed.

partition_wise_plan_weight may be useful for testing, but I don't
think it should be present in the final patch.

partition_join test needs it so that it can work with smaller dataset
and complete faster. For smaller data sets the partition-wise join
paths come out to be costlier than other kinds and are never chosen.
By setting partition_wise_plan_weight I can force partition-wise join
to be chosen. An alternate solution would be to use
sample_partition_fraction = 1.0, but then we will never test delayed
planning for unsampled child-joins. I also think that users will find
partition_wise_plan_weight useful when estimates based on samples are
unrealistic. Obviously, in a longer run we should be able to provide
better estimates.

I still don't like it -- we have no other similar knob.

--
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

#52Robert Haas
robertmhaas@gmail.com
In reply to: Robert Haas (#51)
Re: Partition-wise join for join between (declaratively) partitioned tables

So I am looking at this part of 0008:

+       /*
+        * Do not copy parent_rinfo and child_rinfos because 1. they create a
+        * circular dependency between child and parent RestrictInfo 2. dropping
+        * those links just means that we loose some memory
optimizations. 3. There
+        * is a possibility that the child and parent RestrictInfots
themselves may
+        * have got copied and thus the old links may no longer be valid. The
+        * caller may set up those links itself, if needed.
+        */

I don't think that it's very clear whether or not this is safe. I
experimented with making _copyRestrictInfo PANIC, which,
interestingly, does not affect the core regression tests at all, but
does trip on this bit from the postgres_fdw tests:

-- subquery using stable function (can't be sent to remote)
PREPARE st2(int) AS SELECT * FROM ft1 t1 WHERE t1.c1 < $2 AND t1.c3 IN
(SELECT c3 FROM ft2 t2 WHERE c1 > $1 AND date(c4) =
'1970-01-17'::date) ORDER BY c1;
EXPLAIN (VERBOSE, COSTS OFF) EXECUTE st2(10, 20);

I'm not sure why this particular case is affected when so many others
are not, and the comment doesn't help me very much in figuring it out.

Why do we need this cache in the RestrictInfo, anyway? Aside from the
comment above, I looked at the comment in the RestrictInfo struct, and
I looked at the comment in build_child_restrictinfo, and I looked at
the comment in build_child_clauses, and I looked at the place where
build_child_clauses is called in set_append_rel_size, and none of
those places explain why we need this cache. I would assume we'd need
a separate translation of the RestrictInfo for every separate
child-join, so how does the cache help?

Maybe the answer is that build_child_clauses() is also called from
try_partition_wise_join() and add_paths_to_child_joinrel(), and those
three call sights all end up producing the same set of translated
RestrictInfos. But if that's the case, somehow it seems like we ought
to be producing these in one place where we can get convenient access
to them from each child join, rather than having to search through
this cache to find it. It's a pretty inefficient cache: it takes O(n)
time to search it, I think, where n is the number of partitions. And
you do O(n) searches. So it's an O(n^2) algorithm, which is a little
unfortunate. Can't we affix the translated RestrictInfos someplace
where they can be found more efficiently?

Yet another thing that the comments don't explain is why the existing
adjust_appendrel_attrs call needs to be replaced with
build_child_clauses.

So I feel, overall, that the point of all of this is not explained well at all.

...Robert

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

#53Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#50)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Mar 16, 2017 at 12:30 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Wed, Mar 15, 2017 at 8:49 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Of course, that supposes that 0009 can manage to postpone creating
non-sampled child joinrels until create_partition_join_plan(), which
it currently doesn't.

Right. We need the child-join's RelOptInfos to estimate sizes, so that
we could sample the largest ones. So postponing it looks difficult.

You have a point.

In fact, unless I'm missing something, 0009
hasn't been even slightly adapted to take advantage of the
infrastructure in 0001; it doesn't seem to reset the path_cxt or
anything. That seems like a fairly major omission.

The path_cxt reset introduced by 0001 recycles memory used by all the
paths, including paths created for the children. But that happens only
after all the planning has completed. I thought that's what we
discussed to be done. We could create a separate path context for
every top-level child-join.

I don't think we need to create a new context for each top-level
child-join, but I think we should create a context to be used across
all top-level child-joins and then reset it after planning each one.

Sorry, that's what I meant by creating a new context for each
top-level child-join. So, we need to copy the required path tree
before resetting the context. I am fine doing that but read on.

I thought the whole point here was that NOT doing that caused the
memory usage for partitionwise join to get out of control. Am I
confused?

We took a few steps to reduce the memory footprint of partition-wise
join in [1]/messages/by-id/CAFjFpRcZ_M3-JxoiDkdoPS+-9Cok4ux9Si+4drcRL-62af=jWw@mail.gmail.com and [2]/messages/by-id/CAFjFpRe66z+w9+dnAkWGiaB1CU2CUQsLGsqzHzYBoA=KJFf+PQ@mail.gmail.com. According to the numbers reported in [1]/messages/by-id/CAFjFpRcZ_M3-JxoiDkdoPS+-9Cok4ux9Si+4drcRL-62af=jWw@mail.gmail.com and then
in [2]/messages/by-id/CAFjFpRe66z+w9+dnAkWGiaB1CU2CUQsLGsqzHzYBoA=KJFf+PQ@mail.gmail.com, if the total memory consumed by a planner is 44MB (memory
consumed by paths 150K) for a 5-way non-parition-wise join between
tables with 1000 partitions, partition-wise join consumed 192MB which
is 4.4 times the non-partitino-wise case. The earlier implementation
of blowing away a memory context after each top-level child-join, just
got rid of the paths created for that child-join. The total memory
consumed by paths created for all the child-joins was about 150MB.
Remember that we can not get rid of memory consumed by expressions,
RelOptInfos, RestrictInfos etc. since their pointers will be copied
into the plan nodes.

With changes in 0001, what happens is we accumulate 150MB till the end
of the planning and get rid of it after we have created a plan. So,
till the plan is created we are consuming approx. 192MB + 150MB =
342MB memory and are getting rid of 150MB memory after we have created
the plan. I am not sure whether consuming extra 150MB or for that
matter 342MB in a setup with a thousand partitions is "going out of
control". (342MB is approx. 7.8 time 44MB; not 1000 times, and not
even 10 times). But if you think that we should throw away unused
paths after planning every top-level child-join I am fine with it.

[1]: /messages/by-id/CAFjFpRcZ_M3-JxoiDkdoPS+-9Cok4ux9Si+4drcRL-62af=jWw@mail.gmail.com
[2]: /messages/by-id/CAFjFpRe66z+w9+dnAkWGiaB1CU2CUQsLGsqzHzYBoA=KJFf+PQ@mail.gmail.com

--
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

#54Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#51)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Mar 16, 2017 at 12:35 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Wed, Mar 15, 2017 at 8:55 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Sorry. That was added by my patch to refactor
set_append_rel_pathlist(). I have added a patch in the series to
remove that line.

It's not worth an extra commit just to change what isn't broken.
Let's just leave it alone.

Ok. Removed that patch from the set of patches.

Very sad. I guess if we had parallel append available, we could maybe
dodge this problem, but for now I suppose we're stuck with it.

Really sad. Is there a way to look at the relation (without any
partial paths yet) and see whether the relation will have partial
paths or not. Even if we don't have actual partial paths but know that
there will be at least one added in the future, we will be able to fix
this problem.

I don't think so. If we know that rel->consider_parallel will end up
true for a plain table, we should always get a parallel sequential
scan path at least, but if there are foreign tables involved, then
nothing is guaranteed.

Ok.

partition_wise_plan_weight may be useful for testing, but I don't
think it should be present in the final patch.

partition_join test needs it so that it can work with smaller dataset
and complete faster. For smaller data sets the partition-wise join
paths come out to be costlier than other kinds and are never chosen.
By setting partition_wise_plan_weight I can force partition-wise join
to be chosen. An alternate solution would be to use
sample_partition_fraction = 1.0, but then we will never test delayed
planning for unsampled child-joins. I also think that users will find
partition_wise_plan_weight useful when estimates based on samples are
unrealistic. Obviously, in a longer run we should be able to provide
better estimates.

I still don't like it -- we have no other similar knob.

We have another cost-skewing GUC, disable_cost, which adds a huge cost
to anything that needs to be disabled. This is different in the sense
that it multiplies the cost.

Well, in that case, we can replace it with force_partition_wise_plan
(on/off) for the sake of regression, to test with smaller data. Even
then we will need to adjust the costs, so that partition-wise join
plan comes out to be the cheapest. Probably we will need set
partition-wise join plan costs to very low or even 0 when
force_partition_wise_plan is set to on. Does that look good? Any other
ideas?

--
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

#55Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#52)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Mar 16, 2017 at 7:10 AM, Robert Haas <robertmhaas@gmail.com> wrote:

So I am looking at this part of 0008:

+       /*
+        * Do not copy parent_rinfo and child_rinfos because 1. they create a
+        * circular dependency between child and parent RestrictInfo 2. dropping
+        * those links just means that we loose some memory
optimizations. 3. There
+        * is a possibility that the child and parent RestrictInfots
themselves may
+        * have got copied and thus the old links may no longer be valid. The
+        * caller may set up those links itself, if needed.
+        */

I don't think that it's very clear whether or not this is safe. I
experimented with making _copyRestrictInfo PANIC,

I am not able to understand how to make _copyRestrictInfo PANIC. Can
you please share the patch or compiler flags or settings? I will look
at the case below once I have that.

which,
interestingly, does not affect the core regression tests at all, but
does trip on this bit from the postgres_fdw tests:

-- subquery using stable function (can't be sent to remote)
PREPARE st2(int) AS SELECT * FROM ft1 t1 WHERE t1.c1 < $2 AND t1.c3 IN
(SELECT c3 FROM ft2 t2 WHERE c1 > $1 AND date(c4) =
'1970-01-17'::date) ORDER BY c1;
EXPLAIN (VERBOSE, COSTS OFF) EXECUTE st2(10, 20);

I'm not sure why this particular case is affected when so many others
are not, and the comment doesn't help me very much in figuring it out.

Why do we need this cache in the RestrictInfo, anyway? Aside from the
comment above, I looked at the comment in the RestrictInfo struct, and
I looked at the comment in build_child_restrictinfo, and I looked at
the comment in build_child_clauses, and I looked at the place where
build_child_clauses is called in set_append_rel_size, and none of
those places explain why we need this cache. I would assume we'd need
a separate translation of the RestrictInfo for every separate
child-join, so how does the cache help?

Maybe the answer is that build_child_clauses() is also called from
try_partition_wise_join() and add_paths_to_child_joinrel(), and those
three call sights all end up producing the same set of translated
RestrictInfos. But if that's the case, somehow it seems like we ought
to be producing these in one place where we can get convenient access
to them from each child join, rather than having to search through
this cache to find it.

I had explained this briefly in [1]/messages/by-id/CAFjFpRe66z+w9+dnAkWGiaB1CU2CUQsLGsqzHzYBoA=KJFf+PQ@mail.gmail.com -- Best Wishes, Ashutosh Bapat EnterpriseDB Corporation The Postgres Database Company. But forgot to add it as comments.

There are multiple means by which a RestrictInfo gets translated
multiple times for the same child.

1. Consider a join A J (B J C on B.b = C.c) ON (A.a = B.b) the clause
A.a = B.b is part of the restrictlist for orders (AB)C and A(BC) (and
(AC)B depending upon the type of join). So, the clause gets translated
twice once for each of those join orders.

2. In the above example, A.a = B.b is part of joininfo list (if it
happens to be an outer join) of A, B and BC. So, it should be part of
joininfo list of children of A, B and BC. But the RestrictInfo which
is part of joininfo of B and BC looks exactly same.

Similarly param_info->clauses get translated multiple times each time
with a different set of required_outer.

In order to avoid multiple translations and spend memory in each
translation it's better to cache the result and retrieve it.

Updated prologue of build_child_restrictinfo with this explanation.

It's a pretty inefficient cache: it takes O(n)
time to search it, I think, where n is the number of partitions.

Above explanation shows that it's worse than that.

And
you do O(n) searches. So it's an O(n^2) algorithm, which is a little
unfortunate. Can't we affix the translated RestrictInfos someplace
where they can be found more efficiently?

Would a hash similar to root->join_rel_hash help? That will reduce the
searches to O(1). I have added a separate patch (0008) for using
hashtable to store child restrictinfos. If that patch looks good to
you, I will merge it with the main patch supporting partition-wise
join.

Yet another thing that the comments don't explain is why the existing
adjust_appendrel_attrs call needs to be replaced with
build_child_clauses.

The call to adjust_appendrel_attrs() used to translate joininfo for
child has been replaced by build_child_clauses to take advantage of
the RestrictInfo cache. As explained above a clause which is part of
joininfo of a child, is also part of joininfo of the child-join in
which it participates except the child-joins covering the clause. So,
a cached copy of that RestrictInfo helps. I have added a patch (0010)
to use build_child_clause() only for partitioned tables and use
adjust_appendrel_attrs() for non-partitioned case. If this change
looks good, I will merge it with the main patch.

So I feel, overall, that the point of all of this is not explained well at all.

Sorry for that. I should have added the explanation in the comments.
Corrected this in this set of patches.

[1]: /messages/by-id/CAFjFpRe66z+w9+dnAkWGiaB1CU2CUQsLGsqzHzYBoA=KJFf+PQ@mail.gmail.com -- Best Wishes, Ashutosh Bapat EnterpriseDB Corporation The Postgres Database Company
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company

Attachments:

pg_dp_join_patches_v5.zipapplication/zip; name=pg_dp_join_patches_v5.zipDownload
#56Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#53)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Mar 16, 2017 at 6:48 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

I thought the whole point here was that NOT doing that caused the
memory usage for partitionwise join to get out of control. Am I
confused?

We took a few steps to reduce the memory footprint of partition-wise
join in [1] and [2]. According to the numbers reported in [1] and then
in [2], if the total memory consumed by a planner is 44MB (memory
consumed by paths 150K) for a 5-way non-parition-wise join between
tables with 1000 partitions, partition-wise join consumed 192MB which
is 4.4 times the non-partitino-wise case. The earlier implementation
of blowing away a memory context after each top-level child-join, just
got rid of the paths created for that child-join. The total memory
consumed by paths created for all the child-joins was about 150MB.
Remember that we can not get rid of memory consumed by expressions,
RelOptInfos, RestrictInfos etc. since their pointers will be copied
into the plan nodes.

All right, I propose that we revise our plan for attacking this
problem. The code in this patch that proposes to reduce memory
utilization is very complicated and it's likely to cause us to miss
this release altogether if we keep hacking on it. So, I propose that
you refactor this patch series so that the first big patch is
partition-wise join without any of the optimizations that save memory
- essentially the sample_partition_fraction = 1 case with all
memory-saving optimizations removed. If it's only there to save
memory, rip it out. Also, change the default value of
enable_partition_wise_join to false and document that turning it on
may cause a large increase in planner memory utilization, and that's
why it's not enabled by default.

If we get that committed, then we can have follow-on patches that add
the incremental path creation stuff and other memory-saving features,
and then at the end we can flip the default from "off" to "on".
Probably that last part will slip beyond v10 since we're only two
weeks from the end of the release cycle, but I think that's still
better than having everything slip. Let's also put the multi-level
partition-wise join stuff ahead of the memory-saving stuff, because
being able to do only a single-level of partition-wise join is a
fairly unimpressive feature; I'm not sure this is really even
committable without that.

I realize in some sense that I'm telling you to go and undo all of the
work that you just did based on what I told you before, but I think
we've actually made some pretty good progress here: it's now clear
that there are viable strategies for getting the memory usage down to
an acceptable level, and we've got draft patches for those strategies.
So committing the core feature without immediately including that work
can't be regarded as breaking everything hopelessly; rather, it now
looks (I think, anyway) like a reasonable intermediate step towards
the eventual goal.

--
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

#57Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#55)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Mar 16, 2017 at 7:19 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

On Thu, Mar 16, 2017 at 7:10 AM, Robert Haas <robertmhaas@gmail.com> wrote:

So I am looking at this part of 0008:

+       /*
+        * Do not copy parent_rinfo and child_rinfos because 1. they create a
+        * circular dependency between child and parent RestrictInfo 2. dropping
+        * those links just means that we loose some memory
optimizations. 3. There
+        * is a possibility that the child and parent RestrictInfots
themselves may
+        * have got copied and thus the old links may no longer be valid. The
+        * caller may set up those links itself, if needed.
+        */

I don't think that it's very clear whether or not this is safe. I
experimented with making _copyRestrictInfo PANIC,

I am not able to understand how to make _copyRestrictInfo PANIC. Can
you please share the patch or compiler flags or settings? I will look
at the case below once I have that.

I just put elog(PANIC, "_copyRestrictInfo") into the function.

--
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

#58Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#56)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Mar 16, 2017 at 8:35 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Thu, Mar 16, 2017 at 6:48 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

I thought the whole point here was that NOT doing that caused the
memory usage for partitionwise join to get out of control. Am I
confused?

We took a few steps to reduce the memory footprint of partition-wise
join in [1] and [2]. According to the numbers reported in [1] and then
in [2], if the total memory consumed by a planner is 44MB (memory
consumed by paths 150K) for a 5-way non-parition-wise join between
tables with 1000 partitions, partition-wise join consumed 192MB which
is 4.4 times the non-partitino-wise case. The earlier implementation
of blowing away a memory context after each top-level child-join, just
got rid of the paths created for that child-join. The total memory
consumed by paths created for all the child-joins was about 150MB.
Remember that we can not get rid of memory consumed by expressions,
RelOptInfos, RestrictInfos etc. since their pointers will be copied
into the plan nodes.

All right, I propose that we revise our plan for attacking this
problem. The code in this patch that proposes to reduce memory
utilization is very complicated and it's likely to cause us to miss
this release altogether if we keep hacking on it. So, I propose that
you refactor this patch series so that the first big patch is
partition-wise join without any of the optimizations that save memory
- essentially the sample_partition_fraction = 1 case with all
memory-saving optimizations removed. If it's only there to save
memory, rip it out. Also, change the default value of
enable_partition_wise_join to false and document that turning it on
may cause a large increase in planner memory utilization, and that's
why it's not enabled by default.

If we get that committed, then we can have follow-on patches that add
the incremental path creation stuff and other memory-saving features,
and then at the end we can flip the default from "off" to "on".
Probably that last part will slip beyond v10 since we're only two
weeks from the end of the release cycle, but I think that's still
better than having everything slip. Let's also put the multi-level
partition-wise join stuff ahead of the memory-saving stuff, because
being able to do only a single-level of partition-wise join is a
fairly unimpressive feature; I'm not sure this is really even
committable without that.

I realize in some sense that I'm telling you to go and undo all of the
work that you just did based on what I told you before, but I think
we've actually made some pretty good progress here: it's now clear
that there are viable strategies for getting the memory usage down to
an acceptable level, and we've got draft patches for those strategies.
So committing the core feature without immediately including that work
can't be regarded as breaking everything hopelessly; rather, it now
looks (I think, anyway) like a reasonable intermediate step towards
the eventual goal.

Here's the set of patches with all the memory saving stuff removed.
It's now bare partition-wise joins. I have tried to eliminate all
memory saving stuff carefully, except few bms_free() and list_free()
which fit the functions they were part of and mostly were present in
my earlier versions of patches. But I might have missed some. Also, I
have corrected any indentation/white space mistakes introduced by
editing patches with +/-, but I might have missed some. Please let me
know.

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

Attachments:

pg_dp_join_patches_v6.zipapplication/zip; name=pg_dp_join_patches_v6.zipDownload
PK
�~qJpatches/UX���X���X�PK�~qJ2patches/0001-Refactor-adjust_appendrel_attrs.patchUX���X���X��Zms���,�
��;��dQ�,+n2���^:yu����"!�6E�H������Y|��i_��'�(p�X�.v��:�V���&�#g1����d:<:{���������t1���|��E!�,������������c��g�e�Le�,�+��������9��Y�R���O�7��������{�6��bf`�L�
���C��G�s:��|��������l�:�?��XpWF1��u��_�E��"�q)��oY�_�;�OX
!����y����'� �l�c���Ks��%�Q"�kw�^�v~��u�]?�br)���Z���w
����7[���/����X$�(���~�w#��M��g��R�����"
]�������l�L���s����^Cr4K�:��`v�!��g����U,�����(]����f�G��b��Vi �u ����>�$�����"�C���4-5++c�9+���Y,���9wo��0ZK��S��ly��������7��	���B��u�P'�e��K�C��X
���E}�!TL���9vXw�_/�����
RO������Wa�~]�K^	�f��a~���������G���
��e�>��]����f��7����_��h����~�dq��s���#2g�-����?��z'��}���5ahHD�z3����!�����Q$m����Z��k����<A-h���qrZ�<u���=����E�!��f�sF_��N����19<`%�������5/)6%���
#,�Q$N��!1�,��{�e'V��~xnV���2RGim�C]��3��Zk�L���o���� �D���;�����)��B����?(����tX�'�%�(o[�*Y����J�[D�<�A�G|��K�\�tsB�fr���Qqq�8��Re�Drdm
e�b�d����fgD��vp��� F�����-�(��{AY���)���A%\3�W�^fW��v�o�x!>$����� u'�I�:�����!}���`K~�r"-X�
��b�pH����)��@��8���<.y�([IH{#^��DZ>N=�����+��a�H[��`��P�����a���=o����U*I�8�6�&'C��������T]�M��X��+��v��o	��d��|��@V��aT�@=}��G�K�Q&F��|��0v��Bf�?O��i��L\-Y��h����:NMU�0�h�C�q�3C�ipe�$����K��0��T��O_��O�%����J����|�$��_���'jd^)9���Av�Gm ]�K�2/���w*�FhL��N���s'�d�Z�Y�5z���>&P�khe1y��X0d����t�a�S��2&2�n6�'�d�(o$�m"��,�W�l�+Q>}D\��C%bF�2Z��k��=��6�~�+O�)y2}Q��|_�r�����*�3���I[%�2�Y�m��:�����D�83�o�\W)������������j�T?`���B�*�Q.�W���Hf�7��"��}/��;��$���ag�?�(i��$��3�!t�
����]�"�9z-[kYv�����5h|�)C8��[=v���E�{La�0��
k����B�TU���"��2�*�����E�sU���xd�G���/���{�M�1dK�
����K5:r���&��E�
�$E=�I�U�E�#Bs! (c!�i�V�{i��c4��6�Mu�u���Py"��()��UU��y����{���[��9<��>���=�h�|!!4<�51��G?�Q���t�]��*�$�pu�r������r��=�
�l�����vF�n�0|��� +~?W`(YFi@��'Y=C�������i�#�(��t�����[J�R����,�[����#��4:��*�����y ,�	���Q@0�����FY�����T1��2)�"�vA���%ds���8sy������������	����"���������T�m_����� (u��������^<g�^
w��;�B�n�\9!�����,|p6'hV�"��2�=:"����{x����>L��D�U��^�����-�S�'Q( �)ja>)g4���U���~�P!m�F�GL~4NSy���������pq>�8�}�������7��	B�]#"�����<w��fS�&�JI6�v�����Z�A��L�y=�{J�xy�e
^P�
U��l)}�S�r8��|����99h�U���V��@�4��w��4uG�9��KP!��)3v��-]�����~�����O�������n���U=�*W�d��[����py�ok8����o�z��b|i�P	����re�pC��2������+��X��S��UHa����V[�+�-d��L��4��z�������L�P�>54Ex(+�3~(��#����6f��!U���������|hqc��|�������t�(w�-�4]����o��'�)9t�9����������N���X��;7����o�2���������Ok���C6���������(�X�Z���Xe���$~Y������]���(����#�Y*m{�����y�&U���IJX�HSA����o����[ ���q�VA��EW2���2O�����������������E��"�:���ZV�e���gL��������8��9*�A,fK5���k�����w�]��!ok$N���I������jW���Z�:���>=R��� ;�y4@il��?��::�,I���P����������|s���z���`'QI�>b�M���P7�d���>����^���n���������^���JSV�O�!�����vm��6��j%��������#�-��y�PV��[M,R��4I�l��s��e%���5��D�&b����Q*E�D�2a]����}�
� @�n�5������rb!��\����D�&�66�����D��s���V�J/m���L���h��"�S��]u0�����xk�����v������=��a�=�}3�3=��9�'��)�V4�U�bNv�E�\�0Q�
|�����R���8
���|�P2!2}�Wq����F�=IQg�|u��t2�=u�v]��_� �����F�;muD�6��4[���|���G�V�0�>�O�u`�������\��bD�����{�*�$��y�����#���z_I�<L�X_��?�\i���}A��|�����6��>B���6]�!Em��A3�L?��5�
5���Q��h�eAk_U���h )�
�{{���g�v1����)�UNw�n�r��.5����i�����Gn�M
k`^!�����0�t�oE�������xz�T�l��������Vj1SU��4 2;J2l��D��Pbb<�RQ���\�������oi
���ni!�����@�
�?p>�m����g�Yu�.Z�T=��/��~�x9{���9U�;������/�iu��{]/����@�4��������@�6�����=���\��������������:;tv��k9#u-g��<$m
\���R�Q��y9�7}U�'��Q��7�)����w�w��!��ePi��[Q@Q;�m*��<<�neU�Q�br�YN������PK&f=(Pd3PK�~qJ4patches/0002-Add-function-find_param_path_info.patchUX���X���X��W�n�F}�~���)����[v�5��B.OE!,���E2�����wf)��Q�
-Q�����93������h4���?xi��0��$�Gb2NC�|4�~+rx+J�G�yg�����
�������Ps���\�O|5v#_�\����I����{��8#��JD�U����yC��A�co��N����^�{�+xA�����$���c-T*�dV��/���3���������H���U<�)m���k�iQ�%
E
��X�J$P��'�
dz.X&����B��[��F���.3��	��[�g\n@F�h�7.(�"����Q����.�1�P��	�<�@jH)�'�`�n����^��"OzE��B~U��2��x�����sp���<|������V$#�DelZv���D3�m3��4�n����Ix���1���C����uG�p��=o������y�������1�Pi�A7�������Y2�+dN��_"��Fd���0��Zt�ZxY���9C�
y��Lj����
��p�<���eCh�wv�e�7��w���Kk�4�����B�*Y�F���f�������n�]&jS�3�0�_�c��{����}o�5A'�GywP_qa%D��T�[��h�f�RT/��r�Eep9��d�^�S���J���|��N�M�1Ump	���M�p��F�TB�9��N�u[�q��,v`c�,%������m�����v�TV��,F\�M�`�7��g
M���
�CwH��B�[I��)��GS�����6%w[_-|�,�cA�@���g�VBa�����E��Qfz�	!��N���4�0"�x
#�������G���'������l��(�����[�j�����-1V�b��y*1����9��"������^���j�`�����d�"-���0`g��
GV��M�
����!���[:N����,�L8$�6�6{�gZ�G���Y:;^
,
�Nob�>�����Qy���N��L/J�`d��:�7������yy��n�����e�{����\CvIs�0er���� ^�5f�Y��a�`���x��7fU�6p���\��Xxce����=W��e*���k��}k��%��9N�K������|���
\�����{����I�����1o�%���0t��v�d���i_�����j�q�n���(L8�/��|w�N�cPK�y���QPK�~qJ-patches/0003-Canonical-partition-scheme.patchUX���X���X��;ks�8���_���j%����8���\\��)�{�a.��H���"�i����_w�A�%�f��-���~7�.M�l�`z����4�����Sq0��`6��'�������}Lb�El�������?�����;����U�%r�~����\.���"�D�IC)���O�o�oy&^��\x����'�W��N�`|8w���	?{�~�|~{���g���+��q�>����Y��@��Wb-����$e��0�����pf��e|N�I$�]o��x��d	��]���a�b~;Jv5���Z���|-�vP��2����S������0-K��nA��$��4r����iaH��}���N��v�H��&��<����RK_��+���"�c���@\���=�?H���xg�y���-g$�}�V"
3��W�Hy���AT*2�i,P�yCF�Sa(��pt�f#8��_�Q���0(K6m�!�r�����RJ�\��Q�Q"#No�,�N��G	����/�����tD �f"^P75����w�L�����"F�&����t3V#E�*�>��;8a���,��h��� x�p�t2��Z<8�(�(N!G�!��?H���!c��!��_�(zl6>��J��1��n7�K��G�G����y�� ��;Y,�r6���G�`��G���b��n����_�����;b|9f�@�l�bs`�\���g8�X���{i�d���{0���N�� 2���X�9���3������?ev!@�`~t��	(�8�����cW1H%��0���!3!as ��-�oW	��"�*�P��hF�<:�&'n�j����f���F�������Mp�U^���b���EQ��{��*Y��>2b�� #��ir1�,AL���X�v+c�`6j��2�����[�OZ������&k!�d;$(#<�%����\Y�><���8
���o�I��b��\�;l��qG�����~�/�%�c���_�"�����q�?q������9���E��i!��*#}-�Tf������:�������Y�\���i���(3��������p��v~}���f����o���Jw���A�n�f��DMA���@@�jPU���u��2^��������VC��U�ZX�����K
g,��FoTrs`0�\$z�z�t�s��n�9�'�P�-)ur�`��a$���Ah@��@���?H���L�Y���'L:p��dLpI��D1�����#S�~��B������P�DUn�e���D�`����S���OI�X����N��`@05�
�$��v�W�t����K��0P��j�&/�F�AvZg�_�u�����B�m�3B��Sh�@��<A#6�������)��sYi&��B��!M UA�-���,'	�s|���������cIB
d�a���$uH�C���@���R����l�dr@>q29{�?���M�[�/�$�C|Z��r>�Z
T8�g��FI�����v��	0�w��������i�4�aek��A���/=-��U����	�Zq	��A����^
$�!i%�n�
��t@zp4[����Cq2�O���@����T���1J$�Eq�����<0�g`�_���M��@����2a%������\�r��[��Cm���ue50��G�`�7!�NU<����4/��F�lL��h������xf�AQH�I1�s��%e��dhN�;'+���W8l��&k4�\e��H�%��F�Q�)�[�����\|���g�}��?D�8q�6����*�^��+�3��,sIpp<���N�@�F�Y�AT��9��r}�N�h4����d�`:�NQ6J���p��JYWA�GI�H ����Zev��(L@qepV�c`�`L<�8C��:%b�\<W�St��������a7��H����v���I������(��WL��|H����<�CON�x�������xh��d;�{w'�&�T��0�]����
$���i��dy��"�(�~`0�~��q�����U8�s���%��w�Tb'K6>7@�������R������{�����)���@�\OM(R��~���H����H12���z����{�K<u�Cv��&3!A��@r���h;+����*���9��,[S�#���m*R���*�S\��<� _1eQ*��{�L�Z/��)���M#�Q��u�M{��u���J�������1 =����R�(	~AI���>��>�������t�v~{���K����e�bJ�����%���m�i!�+������Cs!�a���*�	�u�O;��  K:sa����[dV��
�[��78UE���TI1+%�J��u�9Q����Y��d��+�I��\r�a��YC��e�ZVq=���
V�@a�7�g�]�Ii�OI�1q����)�StY �""��'Keuv5_0�#����j����X�J#��Gl�#��~*�d����A�4�`����M8���x��������3;;/Y
���/
�z
jkYg�G�kLjl�D�b�*!bw�����l����.;$���u	��t�.F-S�x�,U���LNu������zJ����'nr��0�S6Z��&���sM��b?~��W���9�lu�.rk�vCf�Q��l�V�)h�#�l�������<]�U�����8��u�RcP��|�����f�����h�;3�a$mm���[�7���(���LL�MF�e��z���
S�B1���N�sI�	�
}8R��_�d�m��� V���z�?X��n����|�G\��#Z�*�����R{`,`	�c�j�e�@j�[U�`;C��7�^�����x��]�F�Q��g���>�\X�(���|��f�L�DL���B�R��#�N�fDm�p�(-���#������EuKe�E��A�$]��b��7��H6K��G��������A���R�(C��f+-h|0�@Q#3� 8aPA��q�/B�NZ0�@�m6�X|����c�c=o7�rc�����5X���x�C��u�%�qvF�MR��Z�#��8�K�m���w�����"e�����Zo�2��g�����;R��:!YGg
�]7
��6k�-����cK���T6N�D���C}�1�%f�e�6
��Hm�����m�QG�V���%��5��Q��� ��B��A�_�?T�Vse��TP������FeqS�'�^���rbYF��[�B�N����*5K��2�����>�V���0/���������T8����FC����x�tR�����f��[*t��lY��0A�/"���U��JOt�/h��W�S9��E�������[��,�
�����P��{a�(�8{x3)����U)�A" �i0!��qP����d��1Sz����;����$�8��M�|%xP��vZ)��N 
^�@�c'�>����
7��8����+��~�B�%C&+���V��_�<
��s�m}��Q�S�+U�:���6tG�����w�,��[
!�Mp��S`s_e&��f�[n���5��TcI�1�ms�ERZ�!$.*�����!V����b��?`�=�O�g�H�T������[��{ht�����e"���45��i��!S���&�����d'�����Z�%�����!���_�"���qA�:����z�B����&���{�J�o&�A��Y�7��g4�)���_�u����w�[/�.v�N��08Z������dvrxR���(:��S�vHwEMg>@X�?����=6���7�}8������n$��.�B���MA�QT��R��mCR(����	�)8��n*_@<�w���TrU},d+��g$���E�sI�W�O��H-��.X�Ov�5�J���1-��E��6�RY�68�\"�O)�q���Z���)-,��[u7?N�,K�U�.���>�?��/ ��������ywB�z'�<cM3��z<����e6=�f'O�K���>d����*�U��_�m�w@���b������Dj���f	I�c�MA�l�i�"����"/O��b�7�n���{�{�u��J7�t1p�U�jW��A^����CD���O�������@��8fH�x���j R�^�0W0.T;�.M�
��H���H���
�HuY��m[�$��E�.���/�N�����;�
90��2-�3��h��2�ll�u3��?E��H�eP�,�N�A���S/�~���U7/�J�*�����sn,;�d��7��i�E�K��R�AY��RY����:[L ��g*f�#�!��fCE��::����W��
���X]���e����+� 
J�*���f8U��@�UE~��ES�����l�����a*Ufe	qQ;(��))����$|�=-�l���L����?]V�(�"�3�"�G��`��8�Y�����}�b/8`��.�T<x���sq����Jet9��&��H��t:(K �bzV�Q���������cno���q`�aW����`���@�6���	�.��5��8pJ�?9����Xo���Zp�����j��
6���PI�K����C=�o�@V�C
��;(������������y��I0�	�*��:��o�	��B����*~��-f�����JZ]c��L�P��r������RN��Vs�_�T������	���SG��R)��I5N�d*H��nH�.��$�N��D�.��c�r�;�Q�N'��50_��D��*|^i����2�tQ����9 m� �b���xx:<�v�PK�;9}�<PK�~qJ,patches/0004-Partition-wise-join-tests.patchUX���X���X��}}s�6�����@��S#=�=(�����:�';�9v�v�������i[YYrDr&��x�6H$e�&j��(�����������|;�#?x7�����}���F����?����7����j���G4<B��?��;	���$���Ux������O��O����(X?��ap3���������u��L	1Bz8|?>z�����w�sOf�{�>�����`�v8��g���j���PD���Ka���\��Ct;_�f�h���l���4�]��!��� X��x����`����xp���E����)��|� !����[�
��w���vP���������u�o�?	�����������#�'���6���pv���vy����h�#�s|*yn�����*��V��x���A#��!������f�F���
�������2nH���������t�au�����hd�~f�����oo����<B�[�n�Ze�Y_�
l���7�DX��������z{|~���������oh�`@�f8`�������$���'����?]�O����9�uM��b.�P��
 �)9�����+�Q8_����V��a��}yzr}��O�;;E��h�z>��h���2C��5�u?g}�+�<9�����?4&�C!��GN���%����/�W����O�w�G����*�	��AB��@G���l0:cN������5�tN��'�N�N��F���&�Qz����?Q1y�����e�&r?�:����{7@�������������)_I�������.�]�V�u="�����r���O*���Qt "��~#y<�>$�,`����������1���U���k�TAmpl'BS��0$B�V�0����:EQ������+B��{_�8EqAHqAH1��aP�0��|I��������|v����qz�����}qu}E�cg7�
���;#�������I~�V'����O/O�#�[��������cr�]\�pzI�2�Lr����/���"���&��rL����:b�_��cLFVpuY����+��n������`y#����;���b���kP����W������[�d����z���f���z����l?U�k�8�G�CZ�*
Y�4�,�WY4���U1<}�/����Z/�>���COM0/�\YoM���&r�����(
�^����~+�����:��f���Aa<��H�����L��$2���c�{{��X��X��y?
ut�G�=k�����$����=o���/!������\�b�i�%y�$/����H��mL��_�c��_�/��e>����e���K��4�]��Y����e��f�F�x~�m����kD]�L���TlO��_���l0�����G�p������W7��`v[=f��
}��}VC�f_K
��O��/�}�����@��`���1��e�����{r��2�ik�B��X�?�Y���;�����.��.�P����6/?��w#������d��j
:�z^2�T������?U�@�����2�%��`��s���b��TW����c�E�$�ZS�(��j���8�����>gc�oJN�������K�f����,�'Q����$�����O��T�'������$�
��*Q�
�,�'IT���������Vcm�G>�r4f���_0f������������3c�L��m�,�D�(��
�� ���x�hvl�+F3$�D?
�L����_��x���sb��)�FY��b�aq�q�fh��(���� 
�-�~T����<�f��vt)hd�����_F�tM��
�D��xEC�j��9c��j�[���X_Q���J�G+@ES��y��\<p�Z�����"����Cf�P<����^F�Q�*U���:�T�?��xxy�{�xx9k��xx�xx��������xx
�GE�TU�+�\Pi��2�>��m����J��J����Q����SD's(L�\����N�^��o�{?B�e���j�G�+�0���G�H�.�@�%
���q���F�P��_���a��V(Z!J�!J�74���&{��<p��������o�k������?&70���m��m��=��mV&�|����/Q����$���5m� �
�u(�dpz�.�F�0(z�^�\��������d�����i-z��������g/z.k��F��s`�E������+;:p�+���J�CT7D�Br'D�4�x6b��=�h(Ms�B{�N
-��&�w
X}���U�:��N������M����,5���z�����]�����QR5���$�J��6aF�cQaFLK����aRl���:�6��
��aU�T�!&�k$b2�I�f�=��I�*=��mk���"��W�u�m\��'z6�Bp�N�gOh'z&����Uz,9��N�IE,E�������7PO����|'�+N��+D?�1�8X�<K�!��~ND�U�a��|k\�+Ir��8w��%���\�-`^EGK�g����]x.���8��%��hs�s4<�|������ot��[��N���\�OY�|��,W>�����vtSE������N;u|Q�����a|6�q/��CtK�w�!�t��u����
o���Q�+��W&b5�HJ�1�����K�&;�gG�_�d�r)��}����|�����>���vt<;���KMs��������������mq�@&X�ji}P`�c�E"��.Kv�s��9�L��$�"��m\��V�GtF��qD�{qgVDr�l��d� f���C��-�dY{��� ���_���4����*Op�<����j4�l�.JO�W�W,�L�Et��\@:q����v�:{�D�l:��3���x�l�4�{��������oCc%a�l������}[���'���:�J���U�������mM�i3�7��M���8��f[3�O7��8��X�l��aP
3��Z�6��#a�S�9���%s@�z��W�����-i�$�����Z������,��+��+���6Hk�j��u#7}��
�^T��?�bZ��PUD��3tb�tpT%UkA����	_
ez�`P�o�JIFcLk�=�^&������N5�a!Nt��x(�}��7Jv(��M��S�Rr{U0V2�������'��������Cr�="����z�m%�������=b��{v����� ��� �5c����BP1k��'�L����"��n�#'&J�h8��"�d�ecnx29�#g�'zj��_��TO��2 �g�^����7QC�%�l;[��x���X��t���dd?Tg�T��	_@�������[��Z�L�__P�Q��?�6-�j7/.t��c�m�X!��n��8Mk����Y-�W��pl�����~�I�;�[�V��V��X�������F�_�up�v����J�<�������lg��o�Ay�Q�zrE�T3���G�,?���w@3
��}�8����������@a����p��T�`>���<x����2%������\]�2�%����cC�p����y��1w��td�����O�y2��6+2�GaGhM{�����9��a�*�e�a�u����jTl$�P��|�Wfh|��S�<�+m���A�k*�}���2]JnK?�i���*U.��l�@���n��T�FZ������U�������"����m�����l\t��p�W�54�k���}�e�K�������eS�})Mg����Zk��H�md�y8��J����<�sv��X�V4oX_���A&���p�l!���xXp�^���:�R.�_��e��%pr2�/�_r�&#�)��o>;�Q(�76��N`��\�25>�Lk��TG9����F�t0�!S��,��Mq��/�p$��
��-S�������l���
r����q��l�<slk�4�eh5�bm��so�����hV��z�)�r��I'����16�Y��LB��c�Y��EFf�&;b���_=����=B���h��D�DLO��v]CQ�@�U|���������� �5�T�z=�d�� b��8� u�+������fpk����W���ak��u^�C6:�t�M�������H\��W�X;����S�
����3���$�k���h@D�]f�������>�������'"�D0/O�<��H�]4�������^|L��xq��qr���U��G�t��-%lF�����H�jiyf�(	F����t~uzy�>��{�3�����h5���#�|��j���R��`�G�$�� ������}4'�O�O�~�g�_�iF��y�p���.����y�"v�!�0[>=V��Wt�Z�(#��ft�G����a�9 �h�����|z�!CM��6��� ��8�`�Q������4�4H�J�0,h �
A���+V
��A�O(hX#h�2��KJH����E@:�e�o"��E�w��8���N���m�-�a�v��?g0�����I���R��m�w&����-�JA��������c���K�ayq<�@���+�i��4�V�2�vS�W��I�����e����gt.�q�+�1��J�;�T��Zo,9��N�����>Q��a��	��(^�'�v�)RZC
��V Z%RP:R�X�J'Fz��>9z�6|Z��v�zY�I��;�B��]�59�@��a��7�Z���_���`��n����F}�
K����d����+n�U���yki8���Y��,hrbG=u*�k�_O�u�u����d��|�����x����dv�M��/���������������j�Z�k���}����>�W��VK���Thb�����Q��"r
aJ�������1�yO��et�����d7���HQ�E���[��SJ�����z�
4�
Ov����\������\M�:u����9E����[|j��Z^�h-i�s�<�Z�
�v��w�)��KLm�PC���k�l��8C<�.���*,$;�R�or5�8�X��oz�p�
�mmAI��e\��I���z�|�wY��F6u�c��k�l��8C<���������i��?���������W	)���&�fi�(�FK�L��y�8LOj;�x4� G���`������I/+�c��I�g0_����������l����O	�e�:V
c6��l��
���uL�K�u,��w���������A<<9A<6v�z�=�v��{:������m�)�O���N/I���4���~��)4K���(�yd/I�	��?4+����A��#^����p����2<�~��h��IW�^cG�W`���Z��N�:
zi���'�t�gy�
y�� ��v�gyv���<Y��A���A��]pt�gyv�'M��l��|�
�X
����j2� M��A�&�Y���fr���L�o{ �NC:
�z
yb�����w�R�:���`�
�7���,��z�1.f����@��k�<I
�(s3�����yN0��ObT�%�A�[`�'�/���W�V�!�o���9`duq1+,�R�MZ�UKP��d�0��UP�����s�j�p.���k<��B�qW^k{@�7�;���Yad����e��u�I��(@�k�K����c/�Ao��z�{k��$����nE=7��>P��S����XT� �P�N�:z*�*n*�~,�P��p���SQ� 0�s��t�"�%�t#����� ���[���M�j�	�"t���& ���p��X��������"�N��4����&����G���pV�[3���b@�3��g��>���8m8�*�t+`�-�s���P���"�=U�������T�W��*�JQy^����r�����i�C�={j:���O��~�O���/gg\�����9�n�sX��`��/�D57�a�\���4�J���S;�mI��O�M�~�%��>2�+��@��a��c�o�UE~�BR����a+��%���:�/&�
������O1�^JV&sT��/uY�k�Z�Rr��d��=�r�����c�x�E���*������]��%�!mk������d��
���\yqe���X�b�'E��JIs�_8W��3�������b�����L�~a�~UB.���I}-��W	��3����_ru�W�zz�u'�e��yIz���~Ys��j�pk��E�����z���OE�e�s����r�r}����D������Z�����)�/O�_������I}-��'�/���2�7.������(VYw�_�����p�_�\���/�5���\yqes������K����W�|�x�V��]$���DU~>�e�,��|�|�R�����*zcNo\��6m@b���V&$lH\�����;3�����h�5lF����~���<�G��roI��P���:��j������8XEk���������:@�}�n���4�R�,
nlg��?��S���4�5,�������|�1�Q���|�"k$b�P��No��-~�6W+�_4��CO0����������/�U���`��
�|��������.\��U�����<�����kAS3�2�L����ow�neq)�V��p����$Z��,KO6{V��y!�L�����Y�����>U�"K�����"������4�S4|Xk�4��j��Vu6|O>��d�66�����5�V��mr����kf�����;���
�;��3i�dP��yZ��)�U������{�i;'�����&_����o�stu
_3�s���8'W�w(�S�P�����@=�(���G�x^B����� h��Jn��������UdE���,\iBd�d�idn���M'8�����0���
L�q��d2��T�����������D�PDCA"������l-kO��Z��nT5dl�
7�C�n��Sa+y_�t- ���yF�
:[�Y������D���P�^L����>0���dw��O>�<����O��&&�[5R6$��I9pI�v�|�Z���xa�����.�qLF�zhk\8O��<���"
�L�����n�2����t{K���1���Ts��/����4����yRU4���g%j:Fe�|K�Y��X�*�������do���f��$���Qe�w�jw!)(k��[+k��(�|���{t�m���;�4����y*(k�F�(�T�cTv��1�E���n�:���L��a��y��q�(ko|����i�J������!��q;i��'F��ac�yF%��6O��O���]7Vl0!wUv�}�� �L�?���j��y�C^�pr�����F�]O��q^�K����0������t{�6LU�R��Pry%�����j>�	g�����'G	;G-h�y<)�;�5�(���4��E�p�4��a�*�����������$�G�jn���1N�?N�9nAs���I��q�����7�����������
SU���'�\4��
�$�<V�t�7��<w�����g�^{{("o���D���EX���}X�������7�AH��K�:krOT�>��/��������^��M�����'���Y8�o4O<�������}M>,���|��������d���_zS]>,���|���������?���F���w��
�Oa��<�)O~��������+��p�x�%.�"�@�0Z�p��Xw��\�mha\)��"�������vg���,OGO�_�j�#t��|�Y�5�!OR���3zP��IjM*��� +�t���Ggsg������m~@���N]g#m<Y�F��FN��������W<��T�T2c�\w��6)f���{sS^�#Qp�bR����N]*x��%Tu@�Z3G3WJ�����Ek�G+
t�`�xT��z��,e�'+m*0���_5�|�>��W}���`p`��N�����R_
���(���U�0�=�o��-�������BSk��j+�e���P���O��N<��j�X�.����`'�<YiS����xs��������$�j�g�s�;vG�	S�1R%y�c�x��(�N��K�Ci34�C�'�&�������q&�4!�<dS,{}��f���8j��������-�(���A�'����hMht3�|u��4�6���PH�m=��[��)��o*�[_+�)�U�����"|Zq]<���2
��D�2d��l��p;�-��p�OG�����2[U�R����o(���B��je�����l��
d��lQ��f�<v\q]0���2�R�D��2^��l���;�-��x�O������2[U��R����o(���B��je��[�`�������*��u!�]����w��8;��<SQ�E_
��B���L�
r�l�-X����`���Ph3y2�.�9Qk��B;��y�B�HO�Q`�fH�S
�,F+K%���p�`(�Xg���K�s�	�n1X��Phsny2�.�:Wk��BR3m��
m+�<YiS���a��5�|����~8�!�
�BK�u�����4>��P��[�6��'�P���s�f](�!5�x����2���6��\�WCk-�����l2z��������n�G�7	�+�K t�B�
����
��C���v��6
R���X��b�|��B�5��l
T������V����}������S�U��@�)#�~��{���X�S�i�l�A���n�Cn�3�o����3Fo��XN0�7��%p�;����
K��*�YX��-C.�w���g�%�i��0PdG��u�3��;�����/�����J`���c�5r].�������q��B{pH�.�u���V�t�"o�C<o��OO�����PQ0�	�����,��;�b��j�0UmN`�-�rW����(�-\���sm���5���lwW��|�Zo��	��Zj�pn��"o�\����xG6��+�D�U-�6^v���T-mv8���`��n�r�j�[���e$���e�,t���f�2�a�F�U�e�FD�53Y4�htH����[c}�KX�qVX_y\�{,XG�u�
��t�����](F�X�^l%v��������VbU�:��Xq���.^�N��w3�b��[y��V�t��:�T��
������h��`9B�/o����:I���~��CC�T��s-;;�>�<9�����oPE�������R�/���0b�6�����H���,�i/C�P&��z���$	�v�]��L�J	g�A�n9[��!�.�������wvzru���Q�N��.S�=�� z��SAl1�M��Z��7"a�qO)�����A(���g�0}.��)���Zi�����D23Y���'���Vy��z��)����������KOd����o�=�B��V��F��n�\����������0���C�r(���[u
�A���B=�'������=���0��I�"���hj���K���`dp�6����5��4�R2��$��$d+:E�_�D�H����+����l�$�e.Y�i����}4����������S�������A��#����
��t:�:��}�i��w>|��w>|�������u>�����}�����-{������������w:��@�:py*�:{�����7�������\���]j?Pyb��:�G���x����!^D��E�x�I��}yJ�
]�|wv���P�G�e4 r�.3��_���u�|ry����"��'�?������/,N��$I������>^\����rz�����n��#�1�r����(ftU�]�uZ�����zH�r����N/���s�C�tb���/ZMh[���7�Z�&�Jw�2X����<i[����`�9!|r~r��?SN7H�0p�c"B���=����(�����:�)x�����ED���'�����
��e����L��QJ~@�:`7�J��IK�\������m����!���'����bw9������rR5�C\_��bU1 ��C��r�5r�+Sc�(���i�)F"�M�v��.�e�k�}�����`��X|h{�-M�g���k�W��,���O�\
�8�%?�N��{t�l��Hg*	"m ��(c�����z�7N�s<��8Z'3�bK��.��S�P�dK��y �Te5(\d���s�J���������Mrb[A�&�J=9��+>�U)��<)�XUU+�:���D�b���|���V��+�N���%Z�#��;��&k=�����uU�K����/�����j%��Z���$_���y��5�c4�Y}Q��"�Y
++v��v��v�k��H�r9�r���/�_�E����x�g���1}�wl�e1�gq�iO�i�}��t']��0f����_��x����������D>z��o��S
�FL�utt	r���9m���l�;�T�/�s�H�)�zg����bQ>��������@��*D��o��4��vF���@	wTyFC��f�
^L��X������'��/��B�e4B���4�h#[,e���E���14&e+2�%��Z����b@f���V�U�6C����Vl#`)�j�_e��_���4�4%���Y���'rEa�2�)2�I2�ie��d�d�s������fHY1�����,eV�������G]��i29���#+�YGvN�Y�p
~�#L�����K�`}\_� �;�J�
:SC;���p��(P�o\�N@;-h�Ag0C6��������I�}(�cA��0oD���������8���A���3%*L=S)�f��^��:����
]u�\Cw��'W�\��������	���z��;|S�0�����t�1r�PU�*����
w�d��6Aq��WA�\S�0$s���aH�@�v2�!��Z�n@��(
BU�Z�9\�9�d2�A��yT[�N����GB=$�5&q��"Z�zdT���
����&����@3	��5�h*a�v����D��������$(�,g�����'i�~��$�|�����~c�=�e�����h�?�1���}�7-O�|��Y��i�
N�iF^>���p��r���kS���&f����_~NF�rA��T��������!���ui�^��y+z�?��J�)�prw�����JzI(�������U�(����dn���k�.�� �$�*�� *�S�Kv��Q1A)���`�E���f�**,/�26A���&r��{�6Z8OR�`�9��z����M���kw��F�&��b��K�][5/]'��g�`�E��h�QV����F`�����!0�+�Y���v;�e�����I6I��F�����0�����E�V�y(1���f��t�Gs�t���+������ ���f-����$��l:�a�'�t��^B���8����4uZ9���tX��`�fo\��|!�5a8o�?�n����8�THv��=����x��$Z��6C���I:�u9���Z���Bi�7�f�����������"?����F����_(m:|�0��m2���J���a�m�M�Z���Bi�Y4�[sji�J����|{99��>�/�6=�x38������q~���a�#�o�m�NK�]~�����Q��1,'���p9�WJfp��P���a~������l{��8��qg�;�.W�3�*���iw��H����s���p �:N(=;���(.��)�������*�B;,�iw4�����y&��2�z�WR8w�����Pmx�$�>���������>#���S*�'�kxl��5�LViG��hf��i��>���T��;��+<�lF��nk��R�A����h�Y�4��Y0�k�
L�j&��I1�ab0&��H'%�;J�USSX��W5�����yy�'dd�'����*X����TC��r����
	�	v7$�N�����5�5$����t�!t]6�U�Qj����r���cHLgF���O�6�O���CS]��#6:��^�x3�r,@����hfx&�E���g���R�����%/��PA�+Re������Yf���������,3�R�G ��,3C�Dd���8�����uI_Xz�ef���]('g�f`�X�����M>�j[\T��^��N�:�{e
�\��d��"�����n��b���5����`�R:8�����`q��28wpp����_),n�\���`��vp����[���	:�?VSy�1��h��k���V�Py�����h9����^��S$�G;Ez6��e��xrE���-��'���7��}��*,(���Hl<���v^UI���S����C�}�.��h�����lO'����r����]h�G���	�1	�9p���B�V����0i�V
GmxRLH��5�A}������t=k�Z����SM�[����DCn�b�0b]��
.�50-�����O�����F9���Q��XB��`hSgW�6��it���l2����3A���E����B����4�����q�b�������rF��;�Ya��o��\����D,J�KHO5�i�Z��70�Z,O �����~~���������`{E��V�a�5��M�I+L��yc!���I37�AsZ�w`�T���f5���+���p���\l�OZa����-��O����B�� ���������-��Be�#Y�=)�MW�Y��*2��ZEfU��,e�e��<��}UdnmV���ps�r<k�r��e�\+ps�x�h�$���Q�L���p�1��j�K)SE�y�2W�N�����M��P�����G�M�~�
�#�i���![���o����(�\HJ��\5:n����"�<���������aU�T�2��)&�{�
%* M�����u)�7���[��-J��U�S*���p��,�b�����O^��Ee��06���K����.�#<I
��T7��RG-���&l��3<)rTJ������\�q��X�|X�ke�e�
�+���G�e�L����a��a��a��U��/�D��|re,���T��d�,�>/I?9�h����I�|�5����yqe��������B�-��+��k����'Z>O��'��p���P��y����g:�_����������)��,�W���L��U���U����;Y>���K���8Z>k.xR-����sQ��������OE��JZ�����v��6��]��k�]V~?�}��W�a<�;G��K���'��j����i{'��8�`r��mN,���:��tVG�tVg�Vgo���{�1��t�M�d���A���|0CO�N�@fg��?p>k����u�����T3�P��Rf��B���8"��)�����A7�l�zd������>��S�=�P���]s����f[��L���(�)P�M���F�/�	��hFy��
��"�B��Tt�
n����ZtKC�-���i�a3!ea��b<���{�"���F��<jJ�������E�k�t�<�y�
q.�s����Z�K�-���t\�5qgA���<���	��ll*������e6��V����A��V�|�'��]��������]��e���1�=���������]����}��a����-����������������	m����tS��S�uT���������0CCD@W|J@Vj�R<Y��5���X������5�:yY��g�g��Y�wrw���(�ywW�����*~���kT*�N!�jd ���a�'i@z����l��N���O��3u>��,�s��*�<A�}��v�>r��#I���I��F����K�'�}�������/^Vg���vW��	����������vK�m�O�"m��A��:��%���'��+[�JZ��=��&p���1y��0����q�������E�Z����4��z�9X���1�Zk5g��k&M�K���B��/XA9`8�� t��y'_,����C��[��J�kd�k����3������6^C����1z�F�\����si8q��R��Z�3�[���-�[m�tF�(�w�%Lf�����qJ��$
%���:�/Ym

px�W�s�6��'�k
|����7��Kd�������&�����X��JlK�	,y	Kz�NbI�T@��V������$��j�����<.��ZU�x��Vg�:��X��/OO�O���wg���L�����2�]f�������>�������'bi�
�<9���7��Py��_|L���D�89���*�zo��E������Ko��)EF%�8>�?�_�^^��N�L�|@�E�	�d�||����)F�.Xk?
&a��a�`�F��
�!��'�'g��3��p�4pu�P������?�tu
�;����D_R��IU��\G�:L�&�#�G0�#���	�qr���]r=N�B}��}�v|`�]c����]�.��.�vwypwy��52��
���-��(hi{��=���=EW{jW{@W{iW��<�F���������d��=�����
R�$�q	�b����)�CNK��Q-#������#�+G@W�����C��:�SPk��<���pa���4@���b��9G���h�/����oo��0��{�F�/8��l\n;('��&y�(c$��
���Z�^9� ��l�)88��F�����>�
���A�.�f@���V{Cs�9�sL�O�7'y����=��5�kr��_�P�{����dF��=�g�/A������a}I1q.Za�������5""I�b��h<I7�H��J�S�c���	��d�p��C���LS�Z����v��,�Q�Zxb0BW@o8��0W�/�����J�
�/�:���P�=�����v���s=��<dSz��2����.��oqqJ�i>=�?���@�
u��ze@�u���(�U���J��cd�� m\�U��W����i�7}���hou��Z�k�]����a�
���gxV���Nomd��l����.��U)��4��e[��Z��D6*f_;�5N#��\a
�U��>�{1#�/o8�'#OD�t�T�e>R�Q�T}����u�si�����((g:��t��U#]�T3z�a3��V5�)~'�?�
����e���������|�"XF2!f�j�m\0�H�������2�V
��L4��~Z�����f�Z`�,b����J�~������0j�����������[����:�[a���<��pv���M
2���G��t����`z|�%!������vvww����������]~!7�#��@6���4�������HP������5�c��V�(-�Q
��8�0X�h�y|	��j����3�������8$�ntP)����.���<z�W�!q��������������|��?��iy����xX���m���z��	�"���<\��N����o!�.�3e'���
C��7�5�8Z�d�N��������P���0������{O�<<���/8D�#R�j�$%�/�sZ5���s2DX|EqH�`�d�su�
���~�v)Y������~�������U��JD�Yj6�WG�����r������3���c����$Z����uz3��(0�[*����[���`��e���I�>�n���x����a|;X���	>�]������rhCH3
��1i�l�X4��`v��G��I��cN��eD���D*�t�%��x�B8�]�*}5CT��WxFK
�X)��)��[�t(b����|�U!|���FZ
�8���.����y�;�.!����}����R����/6�Q��>�V��:�G^�X0A�&���V�Y�O�Pq!�(��	�����mm�������)
#@�"l�"�Q��/���a��@^UL1���CB�?$>����*�����x���8+.�_k�u�Wq�x�W����Qj���K���������`�V:�b}�����p�1PnE������v�~���Z��A�l�������)��!�������%F)���u�����t�%�o�X�7z��#�&���=�_�6<��
�O��}�nE���@~HI3�iP6������������V}�u!��P��m���
�n���X�X�e�T�X�g�vE]D��#6� -Jj\�=��/JOK�������o'�%�������uL���	��z���+����������J������D�P���yJ��iy�
���#�~�0�
ok����8�e9n����������O^�:���l':LoBt��q�=9�|��b}�����; LV����}����N����Tho1��k�v�[S.�"��5K�@�kA�,�l���H���������lg��o�A9����\�+�/���/��#6�L���/�"���T���J��>�q��\�p��T������ �
V(�j�U�g/��&�����"iY%]PD��j�!&��d6]4������G#�^�}|Do�P���N�@7�U1�M�i>Rbn�/<]�i	0f�U�1�5������E
���E
�TI�H�*����y��eA��P5����5��Qk^d������
���2�Lw0�����
|�#q{�4�s���@����m<@c���!���8��!�{_��I��U�����.)�/ ~���W~@O ���<w6�S	�\zZ����G������IM��0�^}SM�Xs��S��������l��a��=���z4#z�V��EOYpk�c�����W�F���������U����Z��mX��M,��K����$�O��/�D1�a�\��w{b���cK9D��F���V,5W�J{mk��?��)}��SnG�3��G5�_v�y�1�2�r&����m�\
�u���'�7���vn���`�:@Ya5W��M��.o��C�6��u`��g���&#��|*|�LC���i����Z1kU����eF�Nw��`}��'���5�~X�Q��;���n~quz��%�M����m�q}�V����������M�~:�������t7��/�%�u!��b>�����<M��^���R>yK}}>yg|M>Q1��8�����,�j4��V��&Oy�S��TC[e4��A��(��jj������j{V�n�{���y���a���t�Ja���F_IM��h��^�/���b*���Qh�`�����C@~���mT��nF�������[�x�[O��E�[O��{y
����)���O������"�?f�5�����6P�
�x���ll���c�]���T(ft�5d�]���Y�ft��d�[��������
2~ ��'\���Y���Y�h)���E��T�D�.��D���$bXA�����K"����p�)J"~BI�I��$6�D��J�cY�e|)� {��>Mm���� ���Xlg$���� ��>�g���h�iv���[;A���A�t�zY�V��e�?v�3��p�����pv����W���������Z@�uP����6���!or����_��=�U_�����8����NGW}�j�&t���q���M�uK)�/��t�8�[�a�B�����x�>h�����E��a%u���hm�������:��
]+���b#u�(o)���,
q.�4��	�:d�	��K� V���������"Z�z�����
�/�]{��8���\EK��X��hi�Lu��v��?��r�U�����|�mX��������|���Yf�B�Q�&FH3qf��w��H�XX6.������W!>C?^^��s2��8�/�J����Z�Y5�������EaoM}��������0���a�K�_KE�bMc���������d�M�?�]m� ��Y�v����n�t��d-=�T����i��P��V�e�;�f�[���%�T5�����\�
m���W�/, p�~�sU��Z���B\=��X�\mb6�H���m��el�����q��K�o�A�.�_(1�3%�y&a��k���s��.�M�����;q������|	��LEC���D_?5���`�p\��^w~4��!��\���~L���k�K���	d�"H�����s�D��c�u��
~����ZTF�D^"+N9�
�
�,��~��������x/p.�"��dFW]���$sQ4���Cr	E(\���$���2��{(�HT�,��J�|�
ZZ�b�Q��3�v���{>��-
���O�VNlr����0��@����������w����<��4��J����'������������=��=������{(#CH����0�L[���1�#��AJ��A��0#�AZG��j�ZMg����9:s�w���c��<�{!��}���4@���b��9�e���_�!����kaPc0G�	���hM���!)G4�N�A�4�b�B��1w��l���'hz� ���o�r�
���k�*9��P?�Q���Jf�muDY��1t�"�J+�{�/��,E�0�j�baB���������6�*��V�{��M���=]���Z���<��fU��V�g�u��CJ�b�_�Iufwty03=��i,��f���D��(�YMS�
Pp)(o�
Z6W�F<(�+:7��2�	�v�i�x=�e��j�/G2��bH
w��G����;;�PK����I��PK�~qJ5patches/0005-Partition-wise-join-implementation.patchUX���X���X��}iw���g�W���@������R��E''���i4
@G
4�� �����.�vW����ID����v����u�-D|yz<��$^�/�����x|qy��_�'�g����d�?�'�]���J�����[���[����y1_�Y1/�UT��"�wo��Y.K������q/��Z��R~+^�IA��r�������?���������2.�?}|~���8>;����L�$[�H��,Y�d�J�:����j���*P}����X�wR.E�,g�����m!'����,z���YFIZ�l*�yR�R��e��Z�8BPv�����d��S�G�W�_������,��B.�C��`���1�/"����b<O�	U���a��0�R�}����sXx�����`��yO��K�]7�&Ul���U�^b!���"S�-6�1lY�8��:�?r	���5T(��%�$�A�M`�2���J!��`az�|�LD�����?b��������~��iv�ph��1M�/��B�}>�@��l;��C|��K�d�7�"@_[������Ezg�i2���"������NK`�q���Qu���N��]sK���(JS�Y�b�e���Z��p�`��:���(bs��B���l�6w��r��C\�yC��k����8	�;�8�K��v]&)
h�x�O}�<un�F��g�D��%6;SS��f�wH�?6��2uQ-�">��c�7�K=�d���<����xF��AS���q���E�z3���z�����zw[�9�2�4lh�4�S��"��Zq��JN��!V��uY�-���/nyWP��	���@�����|�.���\C@@��LN����gt���AW�^��J�px�jM��T� �G��m�X�J��E��'�������0^������)��
p[���`��������RH?a��������`�p���q��}X�h!����\/F��2�����L���Zj�;J�j*%0��m��G��{����w��H&O��<�%B=�������\N�h,���\mdw����?S��w0��3�f	Sz���FHuD���~%���,��_"m����w�o-"��N=�7��/R���i8���zo0 �O��v8��)�h0�J�b��;���L���6.�2T�,��
\��.����i�NK����g�t
���ycq1������M�Q�_
�C����db���}�5g����o,VG~O�������I?��3����U������] n������Hr:����fZ�,\x�2y���RD�]"	U�F���3�-B	,E�z�q_V��"S��s��X�XN�A��I<�X��,��Xz���H	,�;�6r�xy*��1��� ��g�G�Y���.Zb�.��I�D�a�dA3��N�m����Z���S�e �,���> �he�g�+������U'�\&�X ����D��Cl>�B��fQ����y�lsX�{%9��h�X���kA���N�1��	�	��=��c%��Z�Yr]��W(�-�E/�X���"�1�Yw�*c�����f8��?\c�x<Y����#�K`����\��������&�~�=l�����`a�@#�Y*�.�r@{Z�����[��,Pn]<l�[���!6���
kW�yt�M$|L� A��\�XlP������
����\F����B����E��x\�	&��z�o��Z�����}g�Z�����c@�:B����h���-%
�(�=��b�>1����r��q
P���������B�����u\�s8;�������p��&Ya���Na�l/�NA`
����!.E���4F��(2E'��D�K�:�x[g���b����"�a��=2F���c�B�Er�8]����0ZR�,���Ip����W��J�o�O��S�`���L?����i�5��gg����PG�	��I������P�Z1��*#a���
n�o`l����I(@�D��Tl�;����<p����\.c	�a��%�9"&j�
G
�s�&R�����N	M	�m�r�f !�y)����E}D�����'>I�kzF�o^H�o�jX�����{���
o����E���l9����:����R�z�5;��U�S;W���G�}����^��R�'��.�c�&&�Z���)��������R���Y���V����So��Ph\\)���ly4��E�;��5�d��o�J<���2.����k�t
����������uu=zu����ij��g����\��G>����7�:B_[�S<!�q�ee�93�E�����{^���M���?��p�M�������������w/�|;x�1�>z�:�����OP���4����v���|/+P����aBA���������a�WO&T�>�}E��3o��ZJ��g!�p���>Bl0Y��I���+���/@�Y�]�)���)������6	Y��~=������g���.��0)��8��p�I���U�n��o��P��QQ�\���^�i�A,��,�9N�%���'p@�x~�oY�������#�5�?G
�.�����8�"[(���=�9��0�����L���I���"C����@�c��{x2��#,	�M��6�,����>��"\�[���;@/�Q�����lc�OP���
�k��������s�{��2�^�e����2C:�@����J�� )�`���A�@�qH�W
��	�J�(�Fjk�
��g������N	�Vj\�=~"����%������*T��rH����3�ctJ�o��8��tp�L�����7��{��@��7�O�����
YB���&�S?>]"=��d��@�]�?��Y��U���M��R�����'(v�<��!"M���3Y���Q�EQ�&����D4�fh��=��;�=��~�
���v��a���jwX��������=��'�3@j~�X|�/JT�A�d�F�b����V���T(FY����L�����*o���j	�����?���s������-���D������@U��\�dUP �3��5��A���G4�d[�uN�%v���5Lu��R���YF�����Qh�3���e|[<o�`�X�����32�+�8|��Y��z�R;�6-8�����J��A����	Z�~���f$�b5���NM�{����J��tB
\O�������'�'��H3�����=�{�@�������\p�<��3��4��������o�;y+ag�u��P1���G�!B�O���$nG��e�u��P�J�VgXA ������������T������\���C8j�/����!3�#����-p4��$��&|b����\����!����=f@���jD����N�/�%j���#��F���?��S'�	����n�p�hJ:�F��������y�����n���8�I�O�F�D6<��H&�G�P�?���q���!J�.<PC�^��(�hFV�	���;�=poQ���<Q.$0���3n��&��D;P�+��r��F
ZD�;�7
&���JE�� g�u���-�i����,+�h�Y�\�#Z��i)sR�uq�rt~Z*��c�s�����.�*sX*�$�G+Y��-B�{�BU��V�w�n�t�;)���.��h-��/2����V��G����lMK���w^��:+���=E�`f;�����
	�;��'��g%9�q���i��^���&d]^�������"*�a/���G{*D�VLw�$�@�����tOLc
� ���@/=���m��ZX����m i��,�m��&_����|I���!]��:ij8B�������g���B�^��1	�
�!��:_�
@���O� ���}A�U����:��������*~�l�������]�]�,��0�LR��NG���7t���g� G���D8�.��i�E2����$���X��42ZaM��5:��GM=�4 �-�t��x���v�e:VB<Na�F��]���������=��x!=6�����o�g�U�6C��$�y�i��o��^��
F,
	�a�c����Yl�O;�0�����o��Zx�_���&$8d�~��D������fF�$���+�p����T�������p��'���7�C
�]&Y/8����
X�W?�{��M+-�_��-�%�����)z{L������q����q��4�7�<"��\�(��E��w�y�li���W�` �f6!!f�����F�U�b����tX�tO�JH����C�h0�Wv0x����z5-�r�v/#�fV���5�6��z���R��T�yw��K�
�h5��5�xV��w��0%S_�)�����+���r�1]S<#�VF`�$Gr,	���ud�e��XX���:�m����� .a��b��X�M��Q�9r�]�������a�K�����@f=z��������V|�����7;�5���F[O�5���I_�z��q|��w�k:���5���p���Cr���toO9.0��� x���c,	aZ�n���bf9 E��x5�2��RKRT��"F��h���GN������M����t��]��z2�y�H[���K{�'�3�����!%}X�'����_})��<B$��jc^i��#R��Sm�4/����X�(��S}����(�1`p��Q�	]��V2|�Q������u�`8Z-�
=�-L�]m���Ua
�{� �q�`�emk���$�@��lJ�+J���x�g"K�}VF�2Ob]��q�	�\�|���,�>���u��y;4�#i���f-|Z�n�����0��gF.SO���S��'NQT�������}D��c�� �5,%2�Z!5&C�k��Ei�@h>��_3��6Z��VAQ�E��}���Bqz �2�����P���'#�2�f���U��p�y�����(���Y?�6��f����r����n�������P��}E�s1*�uK�1"4��n���V�xT�1_mVt��^��%��t�o����O������sW��~��������q
dl��*��^�����+��+������6�a{�����MH��AWv��[/O����������E�w)���?��V�v��U]"~��K }�h�A1BF�����E�Y�_��K�����N����?�-g�7�+���
���9�ip��U1�rA��j��mp��D�V�Bi<�$��!7����~�SN��`/�Y�RFJ�j��]�<O�`�"`<�����U�{�����@v�7J���D�1�Q�>�Y
�����
�j�����������V�'I}��P�����o�����Z�n�W��LZ�j��
�����>$T�6��^��k����35n[�'`X��<���t�Y��0���Qs}��3�����P��[V�	��3BWn���Rb�D�
>��XD{��tO�(�Ol���� ;�Fmfx�m�H�_����i�l����X�J��Z��XG�	��
	#��	��HT7�6�xIb/}� )����x��%�� ���,�o�7xO���r'������������U��	�q��+
I����2�.K`*�#�^������:���F�P�{�(���'K'�K���
\���~0il ���ov��^��d�R�n'E����$��Q	q�p:�V	����$c7{Zd�~���-'SAJr��8���zD;������.�)��u�OeCt_sH�S&
\,k��:j���pm�a��z�������IOV�Y:
D�R�v�t8��4���/��ul>��Z��	���=��A;��\s�;�n����x��i��6����u?<��fu'����Z��B[�<��{����0t��zo7��ldK�o��L�����=�~��iG:"/k��*M�ok���m�H��x��*A�UDw�t�E�2!r�10���:������k*>@S��� ��Z2�,�i�L����E���-F�
:��)���GZ���?��Y��2�#!�S�Y��h���0 w|�a��?Y(��j��������1��ti!�i������S1N�x
O�J�� 9mdd�RPL�y�rB���Y�1��M��1Z>jr[����sd��IRE��J����K������}�F��6����/=�e�a��n��5"�M j��%0�R��y��Jf.M��R��A����T�����������!M
�������;����FD���#i�|��O�?�u!4Z�������Z?���Ny���`q��������BY]B�������>&,gjbJ��V���.�s��b<��*�����]��Yc�{����(A���kG��gX.ORx|�������M���@^�
)P,����5�1.��:�zc>;��R%|y����<���+1U�L*v/�O$���B���V�C�N>u���@�5?��v��zF��*���j���jh�+
�5V;6br�}x���S�%����p�(�(E�����-�V�'�8xJ{�n�P`��;O{��)�=�����N��S_�X��K�.�a��Q�W�(��#;V�'���T�R�9j��
����.u��}��$������r��%P��4�k;T�~�z�t
 �;����S0�RNyS�����%��W�V�U�H��
�!�8/F�^�L(�WP��Bg�h����2GU���!7�?v�]�����\U�]�����=�8K���rR�$�g+�dAqF��4*?*��hGe���G,���$���}���J��^<K��]���� �'4<K1U�[�^��\y�6��U��O.����6���j�&� ��3IY���/}��-!b�r>��,��"���@Ma����;j�Y�PU ��e�K����={���j%>W���oR�����n]��V��!�|.�o��Ipf{ru���c��;HwM��nQ������5����o�H�i)�L%~&f���>T���`�43��k3�b�h����
�+x�d+4��G�	?�k_L�q��������T�=��(�h+[�JS�eOs�6��������2���~�Q�ImC�I�a����<�v�4�s����M'��|)��9e�v�|����t��jH>�e�Iyo��7E��vj��^��.o���C2�	���sT��F���G���X�;�x�g���!����JY��������)��p���@5w>�@�b=�kK�3F���d>�ylF�&n;�����)�s������(���&��:��T��Y�f;�bP�	�;���q�c��h���G���g1)�����m��j#%��Y8���F;-����CA��K��f��VNx�DR�
���*�3��V����3)�'>��p�nD��7O�����>o�`Uf��U`�KHW8�9
X:�*�!�"�h��d�yT���AC���9P0�D;L<,�U��T���p~G4u�?����V�}��Gc�wr?�e���D��1v��|_���9��X-������l�J>�������G�*���Y��j���&\u:y^M�T�.'�Zg9
p�VEc���x�2�b��Q�"�����{�����j��
����a���&������Q��_d|������k��)VCj[��U+���#�2a7cE&@�s�nd'pJa��5����ol}_���(QVo;U���g�����FN}o���><7z0r�g�?���
�s$x���v��"���U���)s��97��t���Y}�����RQ�f���pp��M�����t'?�*����������	�<��{H�3"	������=�{��Hd����.*���Y��&�ECt9yz��u�r�v�V�I��I{N=�����^�:�w	2$M�X�mu�4VaC>*=s��L��,���
��P���]�&U�2�b�pT�h��mr�ZI�h[r�2B��~�
�K�zUt�v2�������G9����S�]�F�M���z�@����,�GA�	�����Kg���&p:5mn�i�����{���]J�{�o5/�V�I'@�}@�z���ZgZ
w<X��,�*�T90(N�X��>:^	w�A!`C��ok
�?����8���#�6)���3�yc���['�V��!�O�	U���W�x����L��6��bE����_��6|�����v�Wq����5?{B�ssQ4�c^T�3~J�D6��t�>�BZ�Et��)My..V��q�k�������.n.i��S�I�;��UB������,nj��kHW��@TwV�h�N.�Q>/F�{�IG8&�0����w�	����-6U�1����U_�jb�*�gv��p0���������.�Y�&��Z�����3�{YI��5�.�R�L.�5&���"���@��F�Q5�4����
?J��`�MZ�1�BT���{��m����������b�����q���	�������&�~�����`C}+{��Q�
~�?���MX���������g��M{C��EH��oR�|�y���{�Y����K")���M�TY�=������=8`��l�Y+������P�����-k��<i�������������o_(��7_VA���Xko��H�����P��8��M�j��K|e���xT0([��V�������*m����t��}�G]�r/�#��{l�K�@kJ�Y��x#�;��v�,@(��pqx:�%�����i�,��
��-F��CE3:��K������Yw)��j~�$������o�?lr��������v�hm��I���y�v5,K�qk�������#�"�C>GG��y&���(p�-�����9�G�b�+�&�T�>!��iq
���s���>��h��,���u_��]g���a���h�Tu��!X{��`�#/D���I9���.IRq
�>)�������
����Vz��
������L�Fh<�& T� �<7o�n��L5��
/N�'�����
��=�W��'�U_~N|�f~l#��3G���3V
q����9'��|��[;;3�;M02z���_����<?!������5`V�S���s�K����+y����t�kI,���jdY'����1�F[��M{�YS
|=d�-5L�?+��9�jNS�����3��O.y;�����@5/�*7?3�f���}���mf�=�w�K��Q)+%�x6���n���������:�S����	����c%u��t�~�B�l �{2R/�k�e)�\��yT���j�!~��������hu�2)L|�Rg��~�tm9�t��h��R��t��|V��H���\����>!���R���R���������y�.	�������p�]��s�Y�d��/�/������,�h�*gLvq����2mU����������#Z�Gfc���Syv��N�Hi����~�����k���f$�wY���������hz�%���?4�)��	��yd�kIM����&8?��e]����hP_����,�6��B�	e��Pvn��F������>i6�����q&CM�Z�f�-G����
5�!���A;���R��'|5��Q����Z,�n�Dd����A����;U��p2w����_4���������TjWYL�6�����B�n�+��r�
����@&��g���� �T���5�1o���E1=�h���!T�+j>�������?6��+��8��b��`-#`����J<���!`�gw�VISID9�G���zR���d`T,�R��XmM��������������anf ���kP	a�@�����B��J{VA����8�\�)����|�m�z��V��KT%���q��rJ}��*��ny
	���3�Em����&���I��VT��x�B�CJw��i�F�b�Y�l�>*���[��d��q:���^�$�b"�[Ti�[����[���8�\�UQ�U�vx�>g?��6+��*=&��Cxm6��0��G���t���V�)����z���'L�Qz�����y�`�Q��=+�*�R(��:X�vh�u�S{��\�n���{�r���������%���2/��_ar��P���L�����UC�A�y�&������,��S���o'jWr�����[�0�A�gWW7�������LTbc�A>�Q���j��&Y�I�n����YR��w����<��+|;�:�������;��#�!+��/�>��)'��Rp�k��b������o����|i�O�[��S��n-��P3��ys~�0��&Xb���E��~�kQ�c��7���&lv�+�7t����Q�P���V�(7}e�r$C�"���7F��o�+�@j�����0�������/�-5�K�h���{~r�d	����w��W@�R��'������#�b`����4��\�q(Sn�N2�f�@?����&�8���,P������t���3��H�,�G���_t'�sq|�=����b7������}>I�7'�:-����\F�[,Y&�Oz���B����U=>��aa�7����:0O��) 9c9��_n�Dx�!�"um���T*��;�<�DH���Rs�`���=7�'�$�$�O����2�k��j+��^�a%�
n2K��0Z{���z���t�~it�u8{��'����K4������{XNBo��~+O����fv��v'��EE���j���t�B5��������>/�K����!�9F:�gob��\���;�)��>�z0pLET�7�`kL����e�~
�@���K���?B��l�\��UZ�:�j�v�_�m}���L;`��������v��|��\�����uW���f�����g���}+�8�W7�h�q=��7�1����z~t|���8=�8<�:���i�xs2���;��g'�C���a�_��];��BA����a]�W��h�C+�K���>d�#��?e����%7v��}G�����j`=qE��&�v�s��Fa�Z$2b���2w���n��.
I��A7��,����e���S��%��h/�X<��/�����D��c����t����U]*
�t�S���ZsZ�`��<�����*��]#����Tm����T\N��|H��N��`wr8hp2p�]+S���I�a���i]|�=yY�>�����,�����h
���n���dml�����t�6KJ�������8��Of��j���l�&�=<x+d����3�U�>��c�E����d.����!�F����*�9�;��$)P��%�m~��x\n��^�k���4�z�]J,��R��\D�wA��f�:+�F\�du��|~�U|��Q�"�eC;��l�oX>���\�3��D�/�kD���L9���e�GX5?��=��H����}��(��x��!���e5
fO&_����z���m�����?�BZ�R�6lZ	p�l�~�i |��`����"N�)�<_
�
�z�4��wW�\x�tF>+���;�=�G�)��6���x}�T�`����j�x/C����^+��7>�n��@
�E�����SZ&�{�|���W�o�]���M��1"�AA9�@�:_�b���d�nA�F	�g��f�����P&�32��z0N�LR|���`EF	��EP�x�i��'��g���!p�1P��!=���eh��mg�m�K\|\���e/�$w:m^K��f�R���$�o?q@l3��<c���I�c����G����F0��R�|~gq��CP{����u*!^`�&9� t�k%��.��L(�s>�a���8=�GKK�t�'	��.�����<\Fzf���2���,yet�0��M�Y#����z�8m��2���!&aH%YH%Q����@��Q��&m��Z�Pm���:��e�,����d5C}�T�z��:{�o����`I��q���Z���O�r<8�z���tp_6{�����kUP�����?<]���`���lD_���\����x�~�@1
S��OCW�Q������~3����9��������������>���������]�����7[t�-�l���Z�����n�N(d�O�c�rr��js� �E����/��q���u�PG�������b�g������^��
N��(���DcD��KB	���#J����i89'=�����	��0,?5�JDi�-=!^�+
:W"���E���X�2c�Mf���c�Y��O�S1�V[���8T���T�#'���P�������)����������{u��@Z8�����&����O ��m�	e�����{�B�Dl=��S��&��h��,��,�����S/���d��a�yT��C%
���j%�}7������kA�H9
��	�v��!�[Oo2�)8�R����O�`���,�3F8������D�k~s����q�M��#�
I.0����t������ii��.g��*����^���_����^�����`�Q$!�P���?�������������	���CD�+�3������6��1���
�Xl$�7����L4(��:�������N|��v�c��22S�9=IZ'O.����V���VI��#
zGZ��i���r����Ym����,��
�u��gs����
���4�x�T_)��j��8v]���^O/O/''���F���V�voH�7���T���f���y�6��+�����u2�p)P�9>� �cTp�����p��,�B��Pu&���4Z�F�hc�����D�;�>p}�l��%�U���O����X��&�]���i��j�xq�=�q�lb%o���'�8Bi�J�O�~1������Es�vc����\}���=���S=��w����W��]7W4�����B5���y�����
�����d�����T/�^o�6���bX�z
��.(*���,|�U�����.�?��ndQ�
=�G��m6\��d��A+
�TQ:b`���h��������6^QW;�a��^���2���C�n��{�_p��kJ�D'r���G7_A������p����	8�@���	������^K=�[!���$Z�d����j~�_��^s
�\v|����C�_�6>o�:I�� V�r�����s��Rg������'��������b�3���x��L���{�M�3M��~�#v$c	���W\��Z�D|es���0�A
�Q�*�|M�s��L����0u�����Wo>��
���T��4#+|���5P����E�@5��h\�(�1��t0������%�m��A�k
���U�}��L�P-���)>�Kb�d�{����#�����+�������#�kud�;R��QSX�ku����yt2�Qv��'u�������m�����`��~]���y��=	���Fe�FQ36��ku��|�va�C�g�gE(ZKL��J���tu�A�!W-J�����k����-����������}q���b������b<��k.����i ����m�W7tx��EH.SIJ���l?>���0V��zRY(�gj2��h��MWMJ�/���\�����3��e�eh<G}c�T�'j�������Qg�����t��@w���Jl'}����������Q���W�9�_��0����E�������ke(�G����ZF�(#A�u(�C��i�B��*����i��B�
����V���9k���@J�����������7���K����������P
��j��������?�J
;�5�M
���6T#D2��tj��8�cB���n�v�Da��
i������B�����u���xt�D���
�,�\f�!��
��O=�����
��)���$�<*�g("��T-�r����>�Hr����O�i_�zgg���4�����2h�Mv0���t���.��y�!�9�j�{��e�����PK�?�d,7��PK�~qJGpatches/0006-Adjust-join-related-to-code-to-accept-child-relation.patchUX���X���X��Zio�H�l���/�eK�a��	��:Hq��X,�lY�y
�(�������IQ�����H"���|u�o�8���
�7��t���;:��7r<�?9��F����tN�����nTB�u:���:���d��"�y�����89��1��	��IE�J����7��8|e����9��I#z�& ���ywx��qgp��n�������O��o�3lw����K���%�#JUJ�1�����q]���������8�l����g��-��m^���yN�[�W�2���m���v�g�rg����_�������7%�����������M?G����OJ��������7qJ�
���h���B�� �N�I��6���\)( ��VT��D�O
HXTdL�\7���]Iv������\�^*�2Js�'�)����Ar"��T��B��Tn�+V��Tl�W?�Q��&�n��4"�(t@(L�\�1�D�s��5�k;aq����>j�p?r�.���w�uQ#%�?�������F��v�8�=(o��l�����g-n�^�)��x��u3�+|�I�T�y������WEZ�Y��T{NT�3	�z!x��� \}��t��W(����Y��Y�PX�?��R8��T����$��B%�K�V���8L��zP�cS�c�sx���r�;������7L'b�lz]`���z$|������9���*4E'�N��&26�U�c�r���(v��R�~�<�N�9�)IQ�,N�1�6C�{�`g?��^+��6�`������J���q��x��y�@����V������
�i��i[��4�YDxk��s?����H?��M��	�X���M�dU��!��i�l���?�������y+Qr! S�Kk ��*���P�6k���I���'i�a����bnX�@Y��P	��q
�Iy}%.�h�����MU�
i-h�����w��fM�	l����2���Q���]��A���d���>@K��a>c���u��p�gcY�@�	�R���F���������\�"'��It��C�~gL�z���S��v��d������G����b9pg��0��\�/m���)��[l<���+��"��Y��Llv `���u���)�[�[?s'�Y�R���k�rG���<,`$�^jGdk�\�^����WFo�v��(,|D4�S�����
:��;I�"o�z=l4i��r�Z����������r����T���8�����NP%
��G\_E�(:����<S��
��(���i��r������E������I�$K^�����vE��G-K%�3c�mYe�ezM��VYR�����j�#"@(�Kjc-9^���J�E����8�����MP�
�M�-"#p�������P�^�p
T�nJ����6�tu������5�Xa�q���3mh�3���z&��������!���K�"���
>&9���aST���v�xa}��f9��;��Q���Y���n"e��7����I���k��l�8a��8�A���pl�����.�(2��[��$lX	���E]hm@�f]�R��E��I�Q>w�$�&p$K$��ZV������DQ9tQ������7F
�YY1��	�_�a�Z-���mO���tl�@[6~������_����������Vzp���p�W��Ni#�e*E�Y���6�j	���1:�9�P
�����qr�]Zvx�>���@�)Z
��tJ��r���O���Y���F�3��L&��N�ig0�n�3��Y���hA3{���O�:
�]��?{#�%���R0�8��M��r\QA�����Y��#�up@G��O7�<,I���1`Q�M��/Pd�%�Dt�P�]O�%I}.��G<�n�2�&�No�97�e�D&��]��(/�i}�����*B7+s]\�3��3���y�0�#7��/4L�z��F�\�J�s��wX��&��Y����+������������'3�E#NWU\�����6� 7�8���b��l�E*7+�sTv�2�1d&�u
��l-��Y�����CPU��?����^qw�����h`���������fJ���������Bc�-R�7�/9�B����Q(��,��{��'3i_�v�������@��d� ��qr���p��K����h�6dG�z�C|�n���ol7�41����ttc��ss����*���
�Zn��d�j�5si6�E`����6zI��Y��/<pd:�T6v�<�d���(
�5h#�����9f6^X�|�]z�N��khQ�;0cZR�c��hQ����$a'����*��}R;}f<3���zxJ��^PfIc�DkY��g*[�8IP���-g�n�3l��t���z�_�^^9`���P/�,��e��+��'e3��8b�
���c9��lG��5C�	 ��d�zk=
����ZOa�
��{�A��������g��j�G��m�������k�����i}���3T�p����z��3/�Z�{.y\H���W��V�o�]WW����Lu��C��8����^5���U��jF�^��1�
��h>]����vLc��@Ma8����[���B5�^�6�����r��m#���@�)�H������b�+�����v��n#u���h������I�n	8�?�Z�p��'�:@�q�������
B���
���4���w����e�p�$�a��������IO0��H�2�}Y��_���	��qM���+^�D��"���^~�\_�?�}���V�u`�o3�9~������oi�����7���<�S�����1���3&�}B�P+���(��'��YsH�'�����}ga8�=�����/V����%?a0;0G2������_D^��,���}����}rw���$������c����H��'H�������=�xH�^���j`�U7����c|���|��t�D*>��/�K������:^�����%-��:�����E�%����!���Hc������'��6�V��(�#�f�a�;.��\�HE�C�o4�v�!�-x����!�^���4���6��Y����E;ce��K��,��cfYfm�Y�����.��e��f����
��ic�Nn�k��������E�O.?[<���x���OT������S�:2����w[a�R!*�����3c�L�l���.��G+e�n��B��������H8C�
3����i8:��9OK\��Ri������Tz<�[f���c�)�4�w�aM%t���L)s��W�)?�y	���0��`��F�N�y�:g?d"u���y^�y�)�q�����Q"�H�E���(��I�+3M�b���|����em���cML��"e�,�Uo���;��2s��r�j�`�m�`��R_[�'g�(�DEX0f�r���t6a8���B�z=]!�����K�����zI7�����d�T�����I�T�_�2���H�rI���y�ztL���.M� S+�������o|�5yt���'�3g���vot���N�w������I���IS$�R�oz�sC_T���d�R�����4�kWS�'��E7�� ���|~����z���_��^�+�F��w0�y��{~��%}�z�����������d����_�D�}����W���W�����0`.{4P�+���)>��EV��g����PKy���t>-PK�~qJ+patches/0007-Parameterized-path-fixes.patchUX���X���X��;�s���?����J�a��d��&M:�����7mG����"U���n��;�(��~�L,�����J���'����h��-��Z��N}u:�M�'����`|�����$��FLN�x|A��t<�x������<K�J�$72������L��4�*X��d���Yf�B�J��|+S�_&����|���gc�S�����B�������b|z8��[|��\+@�W��2�����~Q"HD�d�O�k��(��b<��I�-nW*�J�0�U�XB
?����@��k���W�j9r��Y

�e���wF��%@�,AD^�l��DgH�o#�yJO2�3-� �������o�?��������_�0 !������2�,e�Da��[��oAA@?!A4�^�� 5`ky�-o�
�'��6����p�*a>�'������*�CFRYS-��1�X�`�7��lB|�=��@�*����[��(I6�;���l�h��6��7��:��S��+
�
|�������+���o��V�c�:��D�n$�C��~6p'�,#��Zy 5�BD���z@��Ja5���}���� ��q|�[!��2-�]�0���y2���`�D�p)���X#Ku����6
���0X�M�5����I����-$`S�B�9V��i�
��������#�6���t;�rR�V���W�p�kX��c��H��u� �����u{�CM2��>�:���`�U����p_h����!,�`��������H�1+��M�3'9F�k��?�o�[����
r�ey$>����Am2��ZJ���>~�!�{���z+�p3��{�l�y��$�y���@q��N��-�8��<�;
(�Dy����W�����������&��"�!��L��]1��>|v/��_��|Z4=�E�	�Y�8	�!4��}���@u� ��0��@�$%���-8��@��X0�'`�Ha����������J��t�=(�'"P�����AY���&��u���vp���&��?9�MG#u2�j2���������oM��s1��E������=qx ^'��������}�2,�Y�D#2�K�<K��	>}p�P=��QQ�|��n�v�O���_]=��7�c� ^��j"�\2�NO�y���/([���Xa!3�Z���\�vY��P�f����w�g`������2��?�I,�z�@��
�H_�m���P�~@���������Y�hz�]�����^^���J<�%�CAX�!��������>����R�������9�2�S�\�S��+P^��ti�d�j����o2~g�}�~$����lr>�7����D�&V�=D�
^@��K�~�	���~C/����)�&�:�4	r����.r�@��t$�*��n�p� 4x9.�`nS�U+�[4��p�)3�f�@w��Rp��UF�2���t$�.�o�Nc����q��m���:������J�pWoG����;XX��*h��"d���V�{�?+{�
������������U�������x_k�����������D��B�!B�4�1��sW	)}oOC�"�N���l��r�A��������
s��gF�$������T�X`�����q��,Oc ���B�
���������R�.S��A�-!�W�?�0L���1�M}~�� �m*�m�<�F[�(��C&L��_���B�����"M0����p�����>s
����J~U�{�4��|��������;���o������h4:9?���i����jo�S�$>�NE�`7������4�r�Ls�e���sJ�.a�Yt���A��_�/����4;�0�Ml1�y�MP�_�wK�\v�[\[J�C��#���v1�9��%M�����Q!�~��t~<&-�����#4c��J}��(ncp�e�M@����$�~��4�+���zH�L�J��U���H������!P0o`��L������0|�X���pT6t��6Y����5��7�����F5��1Da�@`o�k2�F�L�jR����$g���lr6�/7�=��`	�i�,I�H�Z
\��12�TY� �`�X�2^n4�d��T���}H*�������fs����6�`�<�D���������.`��f��V�<��8�"	��
�5�*�;53��������H-�6\;pU�4s[�!�h��Zq������`���1Qr
���xF�f:�>��@�s~Uq�����C��JAo�������r�����/4��O�cQ�)��r�,�N�7rv�A��N��hf:9�/�p�#u��6��3��w���F��)��'?6C^�V"�V�4���con�d|�?R�c��^��y������A�����Y���W������UEP�2�
�`��;�t�T����
����]8�����VQ�V#.k��
`���j�!��F3�i�����w�=jpf�s�8��Lf��b9�������i���KS���dy�~N�8�m��7%�����X���-����u��:��P���Cn1L)R;�Q������.��5��t�d�=�e�����Zj+OyF�����E�s,,UU��P���Y�.N��PS��5�*���vQ��z���1Mn��1K{Q�"��b(�K��$.��!V��J�r��������
�C?����Bv��O6!(�5x���8����y��S�{�_� w�[���8��������	=����������=�uB�Tdj|�}�MWO��;>���`�E��]�/��?�f>�*6�-6L�~(Ll������~���P���� �f1w��xH���t-oQ1#��zq=�S^��5d��������[�M��7w]501�Ae��
�����*��^5R���J���x7��x6��v��x	���;����-h+��n�o����]��"^B���C���K�V���p��Z�MK&�Z1R[��%L��~a�S��yb��e�T�9���d�u��B5��R#�r�����a���f�|�I���DD\\���{��p����"�zWe��mR\���H�	�J�/�TNeF�e�pt����Q������%�F�����9+�!S3��Y?m���	Y$\f��~���r2p��bu�K�UN�n���:�_�%�	�r�~�� X����|�H%�|Pc����4[>�a���Gl���L�A����<[]�o�T��.�*I�!����6}^�A�Ea�	��j8�G��z��R�JQ�J����t���4:&��uX����tx�
��P6
�������;��Y�C����4g�@(H�OE�<lY���X�:*����(�T�n�M�\�z8�lNb����L�^K��u�[6_�.�5��b#AF��v�+�/������}.r�h.PxI�����Z����� �S���_�6�VFAO�����O�M`�N`���A��<���=���Ss�j�y�����	����R����<��9*6M�/���[�
�l�����S����R�Z�J���E66w���8�����m5�G=g����Y����FX�C���E�m�rW������0P������4��s���T��s�6�A�H]����l\tQ���Ks_����[����+����s���9��=������jK�J|�*������*x�_[������I��K�B�]h&�om�<F��q�@Z������p�sPd�����+��ft�,��*4qJb^����,�]mL�q���>r������a�3`����=B#G�K��~}�_��m��q�T�V������d0��a+��g[�r��'��#��4�B[r�S�M�i���+q�(�[�M�=�&-mwcm{]�|�b�-]}��.�&���"���}1�"G��Z�`��M��G	�9���N����Q����G���/JD�_��xzq<��>��^�b��P�Qs��~�
������a�z��*��������X\�����1�<�+�l�
U�'O����i��@O��HDu}���u1�C��������cU��he��n�g6���5#Fl���-?�� �{A������8��F�|vr
��-�������`t|t�w���z���@g����i���'p[��3 �G������� ���)�j������G���4�g4O�D��!�%<p��P5-L�N���3si�9D`O��+by�r�����z��Es���8p_���
�������o�<�GPt���������5��6�p�3`��	��I0;�������?SG���
JA�9�����'�p�B�k�������A�u���U�j���v�����+x�^����\~�=�m����;���NG��c��PK*�`��?PK�~qJ<patches/0008-Use-IS_JOIN_REL-instead-of-RELOPT_JOINREL.patchUX���X���X��Xms�H��~E_�vlH������[��3e��Vm]�F�����F���������6��rq%�h������_&C��������[~�w��Q�s���\��tZN�e-v��t�
�ic������n��XCTs��,�i6��l�$�����O��x"�X�(��������I~�yX�>���P��9��'N[�v�����ct~w���M��|�8\���7���nW�DI&9!�N����"����>C����g[� Sf<������`}�	3������Q���T6`���h�^�}�Z��x!T����
Y�G���0g����2�rs����0��,������#��9LR~*gtS9K���(	��%�"ftA������J
�G�(f"��h����q(4��U�	���-��d�$D�$��XA6�8�7,��-�L"E�7i&��g�$\6C�`"����7�����~l��m��[�v��e"h�,��I�Dy4M��8Lm�#�{,h�$��8��%Sb��!�q��T=���!����k�F�	��4���{n���,���wk4����#��:�t��gXx����{����{p��0�c�] �r.����qu����e2I��'u�,I�P"MeP�f!���T���0�%VS�@2������\rFMt��b��V�R�P�M���g��*��g�+]H��}���Y��t9,pS��RA<����x`��}G/��i4���S����r�����za���:������cR
������Bx��#������$\{W���|�f<�\��4
�hU��N;�h�w����F��K[���,r�*mc�G�p�E�I)V�4�N���4�O�d���g��=���|E�r��F��y��|<2����e�������7�4���g���c4>[���w�x�fk)v+3A���p���c�
$oo�!�y�@����z\�~��xz���;IZ�MP���������c���y*.�����e�t\rU�U�<�"�U*��{A���8N�j��^X}3�_�_�7uR��_]�����`x�{�,�B�K���~�8��v�Up��1�,S�)�	��k4��[���YD4����5(�+��]�Wl�^�4j�S�*lU�L�6�gi�}'N�>%�#BE	�/�[}�3�uS��qL�m��e|���>���o��Q��$�&=�S�^��>�d���a6�c�(�
� {u��8����b��z�:E��vE���b�&�����0�L��Ny�#k�v�@R!���P�S��3�0OT����J/J�"�(t�%���yu���>�+��I�]��������;:Z_��SOE�&�����q�X�D: ���Z���j���2����L����+H�02� �KK�tQ����`�(8�n�G���a�J������f��^��{~��h�E�H.���E�b�3�])#��Ji�t%u'SC�+��6��Y�$��>���
w�[tN������B��L*0v�m��z%6��������BQ��D�sJ7�����2H�|N����,�|a+�X�����+
S�_��8���R������[,�x;��	�3%�|��F�9�����!�d�	�/�`�RB-/�S+"O���D��{����*��X���
�L�|��E�{�����������wY�������8m��,��5�b�[F�y�G4�N`< �0��(�>b(���rA7��n���s���� t�f$Fp��N�~R��&�Y0��AVz���y]-��t����/�N.�
��Z���<G�����2��LY|�����

���3�Ye���s(7�RJQ�����!=V�����3����y �0���b�ZN��8nt-�PK9-�cKPK�~qJ:patches/0009-Multi-level-partitioned-table-expansion.patchUX���X���X��Z[s"�~�_���,,wc�]q|������yH����03I,K����[��p5�P���.�Vw_��^�x��{����|�h��������N��Z~_�m�G�����V;��Q����(���`�b9��Y�|f������x��HD ���x��,�?�k��^�k�Ci(��<>��[P�4k��Q�/��c����{��:�z�O�2
U���+!aB*�#��b�����HbK%���#���SD.�"�� ��	o0��l�����5�����
<�8a��
�K�������b�h4��8�0[��w'Rx��^x�W�D��.�I��jJ�S���@V���Y�'�k���������qf#Oq���?.em���~����hy�����&���K�,z�~��h��YJ�2����������A���(`�-���r`>�|�
0�1�[�J��n�z
��a�E��z�<nt�u�	��f����S��x�c��6L\
������".n�>�8V�<�����G�'�t���)P�s&�ar�\&���&��$y'��P0�O����[����9�/T��?������C�����
����T����H����K�^�D��	n5|�1�%���+$b���>�a�a8�
�2VA#$)Nea���l}b*z!��N�hv+�������(��=6�\�J;'B�� z�%�@`��j��(�����y����#��=�������1\BS�f2a�d�Fr�--��t��P�����
�5P�g�������t�B�K�<ZS
b�����+Es�L��h;�+�%dvd���]1����7G
��#�`���]�
��S�fC���,`�����,r�Z(.�p�9l4�F���������EQ)`q0Q@��@��T�<�
��9"�����rU�����aw�pY���E�:����3��vb����K#���_�`���"%z�'���B�Oj�C�$]P1��� �k���w����HI���h6*^&z���h�`���6T����/��W���0���C��IW	L��4���d���a�S�������6:`�W&����Xo�;D�%��j�/���9N��r-4����d��;�72�G���9~q1��;�Q���~3�BB�:��&�|�YI���k��	�� �JA'��AzL�*C��>N����
�+���+)�\��_D�z
��G�����4�s!�`|=jR5�y��W1$�ap����<zV�Bf���NN���g��a��"�G-[9����T���'������T�e��`�9���"��@`����}��4�u+�c<��<�Q�����}-��c3TMu�,���3��N%,�r����=�J���l�@��]#�3ob
 d��"S�bL���e ���:EMgYIX	Pp�x6yVeV�RN�4MSf;=���uad��I�����j�j_W�QtI�pL��f�Y�cKw*���Lu,lb�2
uF�#����.����#|���4�=�p���2`��da���7Z���~�<�^U�~9�"YI�(�i�Z���SIp�L�p��q�`
�g�zV���l�d���qp��de11�	�����p�E!����R�w��W�6�.y��`OC�I^�����4j+�L��������\����l td	�!�N�P;,���MA��}���mC��XY�>��GXE��K� ���l^\�������A.�%��4P���9�w�fr���?����z��\�������S��)@#��X�KC1LQd`���!d�����ZVHRu.���w�b#_�����nY.V�@9��4��|�o�vsIM��.
�\���d�"}�zB��{D�r�
��~�Xx`�a���Yn�����a��������T/�>���0F����MHF��Yi.YY�%����$q�HF��r���T��-9{ID�Hp�5��<�]W+��m��w��x��=$Z�����1�l��k�� @
��~I�I�;1�+_Q������!���m7���T�m��6��c^u�H��+Y�ywR]Za.�
�b���8�����]������������=����|��.s��:l�]LE�)W��b~�GS�:MH��`���7���I��j-����z���O��]���`�?��V�*�B�R�l�L��������#���)sB�(gu��+=�X�����N��Q�h��4���)�5���e�I,'����O��7g�_��\.y�o�W����N��O-)[8�9�`��������� ��">����Ek��]\�[�z�����L��#�'�{������i�Fm��(j�%�|�u�6�}7���nb��$�E)%��d8�#rS�4�������k$(6+�;�T=�+��:R��=_V�Z��|8DX���h��\	���~�)�9�m������V���!r3w���#eHm�x��h�����
��"�^z�J(,�F
^8Ofd�b.��%�G�
��@���V���:Ve���� R2'i8�r2Kmf���zz��f���������K����t��n\zi���z�Z���X��/_��.ivg��XE�Z�\?�0�&�B��^,e�~�r$��UHP���_���`O����<��M�D�]�r��u���kygmo�ml�n�����V�~����%����/m�o�|�����i	���t��%��MK�f'��q;���i;$��
��!��K��n��5=����H?������\!�)���v�k��+����-�vw���]�u�VB����R^�I�}�d���*���Set��b�>]�����B�X1��!V�_^��w��a%��SwSW�
��*�
�N�Y���K����8��-)�\�(��X��k��O���7.�vt�m���M���}mbi�u�Vn�����>���wk��5�Q�5J�<��}�\5Lu������ n'��"o w��>��~��hg��:�� q'F�I�������z����+������z}���+�������:_��+��A>�PKC-�#�-PK�~qJ8patches/0010-Multi-level-partition-wise-join-tests.patchUX���X���X��=ks�8�����T���=�[;���jw�3���x������-`%a�7���j~�JB��%�t�a*eH���t�.<�NG=����}4��L�Bg�f�����~�C�n��{����5�G����?@�^�qA��������������k��Wx`o��>^�]�}�[���s���6�{p�M��������38��Z�q�1�Vp��y<�iz����m���8�;�6��lw�y"��w�^A����6�N��n�X0xy�@{.X�����v�����'�������g�P'�����8����u��Cv��� 8�z�������7�&de�����
���|��|���}����^�a4:{E(����q�(������i5��0Q<�KL���(
W
X1[������N���8�J���6V���2�J�^-�28���.
�t���
��?��6�� z���W���{p�oS}��qL�OV�I����c��H8����f	���
�)��@�1�����6x3�����__M`���M��R�3�Y�hx����<��/���kx����?N�F�!����h>���_Lg������
\�����kQ��H�I!!5H���z"Z�i�QH���w����Qo�#���R/��`q�z\MM$L�P&S�B*-���GM*D=A:�xy}3���������L�&o�@n<�/p�H�����wt�I2Pwx�=���Ac��k�~v���_��~�WD��	"�9$:'i*n����!h!2�����s���>�S�B_�w�|�[�c�L{u�=������0��#&�,� �`x�u��nFk.��M$��Cyn�U2*�<������wH�<��*:$uh/����CW�y �����y�?�:T�<��Q"D(�y������<P�y ��@������8$v�e��8�PP��"�e�������_^C������f��7���
��E�Nz���5���"��k�_+����v���?��&�������[ <��c�H''{0������sl�!W0������d�F���S!��Zo\/`$O7�zC]b��&�-��6b�:��x�L�MO�D��w��+
����]-���dz�+��b�`����y�?�J������0J�
~7��"6
����V7��H��-�rJ#�� l��4��'�A?��������u:�nd�-j��
]p��U�7��P��-��"��E?���>F��2	�j�%]Y�I�TSwHq_Jq�����b�#�q��q�P��hTS 	�gGv�D%�#FO���HW�|B �5�U�c�������8Z�k^���4�E��R������4�sR4�R�jR�gO
�AU��q�Ff��*F�c��	���;WsE���E�0�*E[rK�M6b��}{�%��>����AJ��N��w����uN���
h�3�7�!x��O��<����[�v������>u�D�#
�:����r����p�+k�w��2�����kPJV���6��������A��O��\���S,��:W�	��1�
K��}w��|�R%���9����j�?,N����j�?,QK�;,Q��K��U�������8�y������W(<�$�Z�^��t����(��MP�|Q����t�A�d��-]����n��M��fU\['��9��KR3�l��G�Q����?U��Ra(��
�"�����$���93��h�C�x]��������f��K���O��/����R��_5�i�%n��p�LT��Dru�V[5glcY�G6\���1�6P�Xr�!j�(�_��nI�aA��m'h������"�I��l���:b>m�����R�8Q
���=�R��~'1h��=���E���lG��Nn�$���^r�������eGXq*������~��=�Mn�q������(�$��
����B�O�M^�O�=�MS?��y�����gr
W�t7�������Bv�+Z�G����}t��YN9�vJGT����Nn7�SW�)z~�+(<H��ZT��\]�
����}�AB�-n��h�DIm�#*��9�+�Mx���D��?�Bud�2YQ�m�F^�qAM-7�W������&��H����	��n~��p�`�.cU�-�&yRXH��&�"�Q�{� �^�^V78�Q���P�J��� �Y!����-���l��n�T�z�P�j@.IH�f�2
B3E��d���f��f�dfZ��fW�LR(c�����f��TE�Eb@b��@$	 �������4P-Ija�N^�.�]����\5=!JU-	%����
��
a�r4g�ZI�R���b�!C|�a�R��(4��^!TTl�{��v�I�P�B�
�[���Jy���Wa�L�!�-�N��CT:�6,W��,��3����������
�2��I����1nR��9�����H$3�75��/T�~//��'��U�������mt�1�!{�>;�c!<�CB��4����0+m��������t
D��h�]����4���I�����%t%�;�P��Ba����W��	�fl��BO��
����f��:#��j��g��l��.��r�`��
�k�� �r�5\�G2Uj�'��z�|��[Gn*��e����AH�D<���L���i��Qe�C�kCb��K~h�����b��"���H���2`o2�-ct��=�>���fTD����D�dT�����c:���.�@���t�A.2Kq�a�/�\������(���V� 5f���S+W��%����~@$���A����(
�C��s���=D\!jF�����H��F��Zx>vq3#����[��DA|1������\��J99����R�q�la��V�bWK�I�9�@��8���(
�C��s���=D\�kFE����H�>G���"�*H43���u���Ra���zx�T����=Dl�kz�XJ}i��k��_��G����J����X(�U)������Z�bo�J�������8`�
��*�7���Vh��n�;V�[���.����C5���v�����(NyQ�~haG.����&i��1xi�|�aG:������$
=_zJ�S}'�v�D$2F&+�'���Z��I5z���N���w��qBO���D�d�M�	~�]��6(H�����}N@2�(H#��	���$cN�l�Q|�q�iIAr2O���.��dr���' ���<s��9PABT�G�n��s^c�IF�&3�7���9}c���(^��-����><��=X������~��;�stBU:���7W�`~��J�
�^��buE1�,c���%k����_J[&�Y&�Y����T�����(&�eQ�J!�2aZ��l�^)[&���-�+�Y��V���b����!�o�F�p_��rG���8}#N�H����_�����������[TjT��H��I���LMCQ��8_Z)S�X��*��y�������U��oA.gW�?�\��!�<����M1� sV;�G��c�,�7@������U����V/5��q�����S���U�8���4z�z�h�Y��nUj&D,������TY�Z��i+�Z����
U0�����D\��TN?��"K+T�1�X��X��b��k��j��!�r��V�r����>.����w�q.G��Qe.Gy\������+���;�9��>G�5�c�0�c�p�/��[�t�����N�'������
�p�,|�/7S�Q�^-�8���K|���4���p0h�W��8>>V���t�=�=��pGN�.�|jf�<��1}�������d9���{p���5���a��L�N��������|X�^`8>������V�?�G���O�h��4�W�G��X`����v�>X���Z��uO�K�
�-���D��>���t�����|�����X>���}\8����v���~���Ww@`��8p���	�G�1|?N�.�g��n��a�������2�W����d
��]��K�����V�x��)����m�
�����x��o�D{��?3M�(<�j���]�oz�(����l36G��FF �
�H���i��j<��{�I?0��pZa��
���9�wf��V����Opk5<���DCy�������2A�,���j�8Ry��"��{D��3����vLV�I������.����f	��Y8�Sb��CX+��2B:���a�a�7�	��|��*z�4
2��6�+���t�^~����K2'#3�����I�h� �B�q��E�b:�_�W�Ln��_z-Ju�L���$
��"7[����($�R�;�����(�uD�Gyp��K�2X��WS	�7�������D�A���Q�
CQO�N ^^�Lfs��&�8�|�b��_�.��6��G���=��;�����s�o��������&�������Q�s�HAp��I��[k�sbZ�������������-�qW�^D������'����}q���~p1id���i�0<�:�e7��h�F�z�X�1Y5���Ch~H�>Py����C���@R���rH�>�pU���H�����#��@���y%�A�Ur���w���@Y���	��$vH�>��}��uH�>w�M&�^��o���H�O��������K�_�	���!�B����u�Y?��EQ���N��~D��"��}�������������6�~�� �|�|����rxP�����|�s_�����wG����h�PK�&���PK�~qJ:patches/0011-Multi-level-partition-wise-join-support.patchUX���X���X��ks�6���+�����E=�����$�t������t:�-&|�������$AR/?�4�%����w���$q����7]L���Y��t:Y���{���f�����=c��]�
�l4zN��x4��o�sv.�Y�5��ox��������Q*�M�K��C'_�_�T<go���{#l�������sk�z��d������I��_/��z�,���~c� ���*��I��~
n=����f'���>��8����>m�w�K{�,�����`�]9�G7L:k
�� `��l�L^[�"����������tN"x�Dr#N�7�l�H2%�c�(��������ve��l5���#M���N��Z��t�����% y(`
�?�8��������D���h0M+���x�2�g��!�^�Ra�?�������wv��Zn�~+"G�L�
#��&R'���3���=�3������UV����[�Nw�m&������I�I�@��=]��:����5a����`&�`Y����H���I8g9�9{^{��"$�]��0]2s.��#�qx��sf��#'�\Qcc�f��dl��9���%h�G7��OA5R$h������l�\50�����y����tr�~����~��oLL\o4�
���[x.D��24�cWl�8�^���`2��}U��d0t#"��
S��
��
���Uv.@���=K�8��	FD�m����e�����){A����N��R���6k�Z��g��u����1qR	~�w�r��� �#�Pt4u��=$V����z����<!cx�r#c���8b)h]��{���j������|��S�G��ZNA �����dx�^��p���b&�b>.f��������t��X4'krz�?e=z_�9���6%aCVcd�G�g�j#��v��P
����g'�`v�_� )_=��W�_����V��XQ������+w��s��]hsw����4O?A�I���d!���P�a|�	|����l*��?`bRr�Z�����o�_�0�I�R.�:X�K���X���K07&��u��b �l$[�D9�?��;�U����J�m�D��3�\��B)���a��G�m�@a"�,�PP������.��BM����<K��'�.��_m���&�����k�yz���t������3H����:�����=�N0A���-[�U�B��u�Kzc�M�{{���D��Pv��h�R@Q%���D���k�ZP+��R�P����v��HL*�P)�6��Pd�vU���[\�P�J_�h��=�4�l��E+Z����tCL��A��OL\xC�v���R��(h0���S�������������T�8�>o���f�FK����kU����H!�}5=$[�D�
X��\
P��P�OD�3�H��Cz�����;;����H���BjJ���MD�%9Y��\L��>���c����*tB!�+���!d�C���M���^c����;k�	�>�I@�Q�-�c������Od
�4I
�t���B��N�q��b�"�_9��W���l�0_T�8��F������v�;�NJ5u��������)�b+�(����NM����ID�����e�=2x@)Q���]I�.J�L��Z�x��3�zf�U��������a���)���FPVc�7��P��~��O��_.^�_�f�+{�����`���vC���BP���;�`���
�����������>���*�X�e����'��ta������Q)�>�Y�k�a��@�H�����"���iYapw���9����j>p4bUxr�p�[LWydHr`x�`�B��%��Tq��*6�����g6L�E(1=
Qm������������e#]Bp���*��)q
E���h
�-#����0�WV)��=\��hM�e������AI���0������t�j�����g#
�_*����(��(Q�u�X��x�`����z���������~�7��S���GP�@�����#d���"��U#�rD� �(������{��'K�:�l4��KAoI[{R�(�w���y ���g���@Q������=���������tF=�>L�E5b�����'d(9P|0�A[/��r?z����^#}$�{Nb�Irp����}��(u|�MF����>����vqTd@i3��;m���G`0|���Q�O��5��;-��
c��6]�j�"��:B[P6fo�������S����MS_�Y�Dm�J�g����d�-�&�N
���HDX�E3�(��G�b�����m�ZB�N=��Q:��s0���UZN�H���{�^������$5x�i�*��tw���@��{���Q}������K
N�ylo<���YXKO8��>O��6O��!&Pw/X�����{O���*?���k� ~#��-�=�A@�O�TK7��g1���
JwQ��3L�R�-~���q������N�����C���S,�u�?`T$��������.R���
&"W%�-�b{V��x��F������=?��b�hx�fS!��-a����{T:����U�VS�HX�]��!5���C$!}�k(��G���Wa*�|��?�U�t%�~��/�T��[������89Z>�xJ�4,����<��(��-�[
���0����D�tEpW����Nv,U�j�`Jp[3��Zo����"���we�T�q���u�����J�W%a�s*4*�:��dJ
R�cn��:�#u2���a�U�
��GECU�8Lj��:l|��\���YU!��g�Hx�t���lD]>l~��X�,:j�����W���l\��]��UW6-Am.��)rQ����;?�Q[e���5B�X@�<}�}(������������E�����������Q�sv��t�� '������6�niR�'������ej�0�=���D�n9{�T��m�5���k"��k�d�i4�
���h�O��f�~x|Dj�>+*|���/�������6LM�j��$@���cw��::6=��e(v�
O�������A�w�$�5�k��a~�E_r����-��^�u�^�8:�Z��S>�����a]�
�}@�x��U�c��I-����@������$]<I�������C����d�p4�]�z�
O+l������.�f�	���P7���<6�ie��-�z���2�U1����5�y����P�T�h�#��`
�m�k���`�H��nm��p?�EVL��[=�����.9X~��t����p�9coz����������-N�����y��-/M��[;�������N&������`�
�QZ|�oh�6#4R�:������2n�GE,o�-
��C�wF�{ke3��b}���-�}y��+�������e�w�r�E��;rZ�:�Y�9���{c}g���~K��n�����T�<r�ty��8Ak^���B�2<�b����p9��a�1xi\���^�!�B�:�mx
:%����|�6oZ���B]���_Tq���_S(�5�Q��/NZ�J���*.�k�W1�%�o�����k���	c� ��4��%��)��?:c���3m"�X���ix�M��4+�������������R����#�_�'Wm@�����V]G�U����;�����*��)��#w����X��������p8�>�o��;��p'��#`�����2���o0ht*�=����@�8{:�����G���?K3����i%_����Q��.cRr��.�8������[
�����r8k��PK{��.7PK
�~qJ@�Apatches/UX���X���XPK�~qJ&f=(Pd32@��6patches/0001-Refactor-adjust_appendrel_attrs.patchUX���X���XPK�~qJ�y���Q4@���patches/0002-Add-function-find_param_path_info.patchUX���X���XPK�~qJ�;9}�<-@��+patches/0003-Canonical-partition-scheme.patchUX���X���XPK�~qJ����I��,@���'patches/0004-Partition-wise-join-tests.patchUX���X���XPK�~qJ�?�d,7��5@���qpatches/0005-Partition-wise-join-implementation.patchUX���X���XPK�~qJy���t>-G@����patches/0006-Adjust-join-related-to-code-to-accept-child-relation.patchUX���X���XPK�~qJ*�`��?+@���patches/0007-Parameterized-path-fixes.patchUX���X���XPK�~qJ9-�cK<@��j�patches/0008-Use-IS_JOIN_REL-instead-of-RELOPT_JOINREL.patchUX���X���XPK�~qJC-�#�-:@��G�patches/0009-Multi-level-partitioned-table-expansion.patchUX���X���XPK�~qJ�&���8@����patches/0010-Multi-level-partition-wise-join-tests.patchUX���X���XPK�~qJ{��.7:@��:�patches/0011-Multi-level-partition-wise-join-support.patchUX���X���XPK�
#59Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Ashutosh Bapat (#58)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

Here's the set of patches with all the memory saving stuff removed.
It's now bare partition-wise joins. I have tried to eliminate all
memory saving stuff carefully, except few bms_free() and list_free()
which fit the functions they were part of and mostly were present in
my earlier versions of patches. But I might have missed some. Also, I
have corrected any indentation/white space mistakes introduced by
editing patches with +/-, but I might have missed some. Please let me
know.

Rajkumar offlist reported two issues with earlier set of patches.
1. 0008 conflicted with latest changes in postgres_fdw/deparse.c.

2. In the earlier set of patches part_scheme of a join relation was
being set when joining relations had same part_scheme even if there
was no equi-join between partition keys. The idea being that
rel->part_scheme and rel->part_rels together tell whether a relation
is partitioned or not. At a later stage if none of the joining pairs
resulted in partitioned join, part_rels would be NULL and then we
would reset part_scheme as well. But this logic not required. For the
exact partition scheme matching, that we are using, if one pair of
joining relation has an equi-join on partition keys, and both of those
have exactly same partitioning scheme, all other pairs of joining
relations would have an equi-join on partition keys and also exactly
same partitioning scheme. So, we can set part_scheme only by looking
at the first pair of joining relation while building the child-join.

This set of patches fixes both of those things.

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

Attachments:

pg_dp_join_patches_v7.zipapplication/zip; name=pg_dp_join_patches_v7.zipDownload
PK
#�qJpatches/UX���X���X�PK#�qJ2patches/0001-Refactor-adjust_appendrel_attrs.patchUX���X���X��Z[w��~�e��(Y����&g��MO���������E�"�$d�m���
^u1�M{��H8f3�x��K���tt�:�w������������t2=����1���8b���9��?S����w��`���Lk�����d��w���?�H�d�����/_X�p)���I`�!{-\0kg�l0x6�N<�[������3���//�~e}��q��.��{2N�������JD~"��2I{����A�)[C&c&�!�`Q�����![��2��2��el�E�
��[�_�u���
�DWL.D�=]	/��a7���K����yq��tG>��/�n$���z�"�ZH�����#�F�����p�,�0���\� �����2�N�;�]�A�x�[�v~u��+5���$^���fw��c$b����P�PX��=vIB��[�>��N;
�hZjVV�vsV0r���X�xG.�n��(^�`�S$Glq��������7��	���B���B�R'�e��K�#��D
���E}�#��:��s�����n�����}Q��[��_I�1�!t�-xt%|�9�Q*�oz�i�l<d��n��-���n�*��5�����
"_|e��;��^�d>����9���hD�l:��4���Y�d2��Y��&
��\oF�����Gm�7�<f�IKq&����E���Z��A*�DB���+oy���_)O��&�N��nj�o�`�Y�����l�o�s`�����������#�aQ������S\E1�k�(�����s��������g?<7+��R���6u�������i��^��vo��		@d�LO��JM-��?�<�"vD5�"�������D���=9,aDy��V�
�v��PJ�"������8�+�\��������0������s��n�V�J%Gf��P�(���GF��lvF�mm��v#�pL�j����,@����A%\3�W�^fW��v�o�x!>$O���� u��I�:�����!}�C��`~�r"-X�
���pH����)��PD�8���|.y�([IH{#^��DZ>L}���\��h�8�Vx=k+����q(p�o4q��)�(g�J�-N�M���Pd�s�6�o�U���L,a��bZ�
�B�����2OP�pI�+~�0�
�����c�c�%�(��by��F>PE;�P!3��'����s&s.F�F��2�k�*��SS�0�7���'@�����n\Y5]����p(L�i-���/y����c�|����ZF>E�D��/��52����� ���6�.E��X���]G�;�_#4&G7�],x����A3hUg���ub2��@Ug������J�b����������
���0/�1��t�Q>'�Fy#��o;%f)�Je3�X���#�%<*3���J$\�\�!h�������]ybO��������*�K�BWv�WA?����MT�/���f~�3Y���J$�3ch�v�u�Q��9P�U�?y�^
����5�]�Pe:��E�:0�:������X%���~G�N�.I�gg����9JZe&���Ll]��B���d4C�Hy�^��ZV��8'���i��bJ��C�V�bng��^ES�#L����a�$�/U���0��H�f��Jt�)�x^�\�#:(Y��l�����>r`Y��+��GqH��9���`������U�F=�I�U�E�#Bs! (!�izV�{i��c4��6�Mu�u��Hy"��(-��UU��y����{���[��9<��>�G�=�i�|.!4<�51��G� �Q���t�]��*�$�pu��������r��]�
�l���a�v��n�0x��� K~�*0�.�uH��'Y�1=C������i�#�(��t�����[J�R����,�[����#��4:��*�����y ,�	���Q@0�����FY�����T1��2-�"�v@���!ds���8�x�����������	vQ���H��VT���^s�������i
�\��
q�_�E��f/�3�/����
_!�7R.����l��S>8s	���-��uG#�X����'� ����T9O4Y%���xmzO��R?�k q���`�B���S�rF��A@�Y ������8�����i*o?]�>{���.�gg��_^}�r~���3A��aD������.W�lj��Q)	�fvz6+-�4E�}'��g^O���6^^_��b#�c�A�@�����/yyyN�m�f���%�%P,M���/3M���DN��Txn���gK���g���y��C�����*�f;�'���z�nU����%�v�%�c4\���Z������\�T�/�*A�Z2_�LnY&YR<��r�`�����x�8�
)L��u_�j�re��,������F��X���}�����$R�O
M�����J��(��������Y?�CH���7dw���N�Z���Q��z.��7�-��Cq?]��I
T������������L��x�;g�pe���uB�?����������i�e�EuB�xHj��5���!�\Q�M�T�Rn,j-LuL�2�]lo�,@U���N�.��B�w/5�He�J�^�����z�I��f���,�T������*=���H�yi��U��tQ��L-�������8)���u�0��&k�} l����+)B6��tYl���m�9 E&0D���U��E�!�%l��B��x��y�������=�m��)�Z3�z��v?C-��_QT�D':��#��9����G���������������jqU-*	K��Q�;�7��J�w���	v�o��#F��OZu3�Of(�����(��,����.A_
�����]?@i�����= �9=������&�P�d@zc@��Qzv��E0����{��E*T�&)���x.����7��F[���D����:^K�4�LX�`#5y�u"J7���Q��A�C1�Md.��Di"ni����F~\"��9�Vq+G���}=}&�����
��)MD�:M[�{�y�5e�trl;�R��`d�l�mo���L�|^����n+��*u1';��c�O�(�>~�����)��/���9_ �L�L�3$F�v������tOR���@�)0�Lh�_�]���*#���������N[���A�Bf��V{��0�q���1�:O<��{�e:`{�tErP�����v/Z��D�3���}�quX[�K"i�G�:Q���7WZ)�i_� 5_�$,��N
��n@z������u�!��h��l��U������Kg�������*D�n4��z���MSl�3e������*�;}�B�`v���e��4��nf�#7���50�T�H���~�����Z��`�Ma<�[
*H�U�L�����K+����]�i�%��a*LH(11�@�(��P.gAoOut����S{L��m�{cl(��8��FY~�����~��
�B��o?|���z���*��a�]�����:������f�} }R��m�k�m�o��wv^�v��6Wu����{=���1�����AqAg'���3T�r���C���E�>(U��1���0y�WU{��
uJy�"���y�}wZr�n�v��e����<��s�VVe(&w���&�iolY�PK�N�^Pd3PK#�qJ4patches/0002-Add-function-find_param_path_info.patchUX���X���X��W�n�F}�~���)���j�Mj���
�<���.�u(���r7����$�fD-R4�Da/3sf������� ��x4�Gr<
���?����e���C��_���,��2�71<�g��fWzQ��^��<�%���c7���LKY���R�n�-_�W������e��P�L�������{[�w2*'������_�:��'\	q�F�BP�J�,�_��\�Tg.c��LH(3�����i�{np�Y�J�b(��JR@V!P&��kP)�����=�f8��J�[PD�'*B���,g-\�AF�h5�.h�"��%�]�����2c��CI�
�x��*!�4�a���m��:!�>�Tt��TK�Y��TI�S4�F��0'�������'�O�R8����i�M�1��z��dL�8�v{��y�,<�Y��J~���~�u���@�{=r�L{�<���%�������p�K�A�������R���e*���/��z#����0��^t�Fx�)A��!I���3��"�)�R?#�0�nw�����i���t��}�\�`i��&���"E���J��Q�-?�����Z'�������@WBo�qf�F���L�}�7r�l��{u���!G���W\XQ��4���!\�Y��E�sk��~Q\�!�Y�����%���\<���p����3��
.�R����U��B�F���a�e�G+��Z�sE�i��#~8�����$Vr+�nS1X�
b���BFf�c
���j��V���C������n�x��I���WG_�!x,�������WZj!����V<LL7!$;�i�=��jFc�fD0=1bSt��;����e��r�Av��Y�����|=������k��?16�<����F[��j�iL�sv���C5|0�;q�I2E����|a��e��#���6��F�n�"���[:N��<��L8&�.��{�gZ�G�z�Y:;^J,
�Vmc�>�����Qiv��I��L/���:�j�u�[O�;x*�o��������������vB|v
%�%�����	b�����|���}g������%R�a��T���1��s��c��Y�������\�2R��{��Tl�m��W\%�8�/�Owwk����[�p����U\��0���g�k�s�~���8����}n�
�9��]|������}�U�a2���0���T���;v���
PKn��|�QPK#�qJ-patches/0003-Canonical-partition-scheme.patchUX���X���X��;ks�8���_���j%K�[~���x���$N9��s)DB��%H+�$����/�53NlI��h���wI�a���d��������t>��f����x>��f�du<>�O��8b_�����h�����h4n�0���\gi,��7��){����?�*�T$�$��_�x��������D��wb	��x�jv�jv�z��t���-�%��������{6������Gqx<d[��Am�[�������q&��.,��%|��e8y�)��nDx�M��U��,t��i.���y�(�yX�B���t-����3�2���5O��R�$�<`��ii��s��&�'���8�q~HC� *��0nv:`��X
�4�$��l�Z����\Y\��q�D?��j����~���l�#8���48l8#�g�	��I�r|�D�o���c����rMc���0:�;@@y������[�iW�:�DD�Ai�m"Y�I�����w��P��R�^����rz�d�pZ%>J`���|�����\�#�0���y�A�����d�
����?��i�	�O$C���0�W9��������O��,
��6���)��e���.k�Q��,���(������ ��C������&Q����6+EB���u�m?X����>o����k�@�o�d5[��t0�K�gGs0Q�����\�m������+;<�M�G��/�H�.PL#�[HX����� ���������Y��3�p������i��Q�j�"z��?2� ]0?<k��B�S��������	�
��m������9E�V`�w�T]�oa��y4�}����?�Q��b8|���h��P"������2/`]G��tA��0�F
��v���n�CF�������B� "s���f��`���Q�r5`W�UtP���'��d��DY��a��!���uh�q�,p��_}��W���7��aV>��C.�6i���]���EX?�`����Q����;a����q�?q������:���E��i��J#]��D�������:��������Y�\���h���(3�������p��vq}���f����o���Jw���A���z��DuA@���� ����c��F�e�8��]�!]_�-�
�����B�}9��"X�c������*`����V��1����<u)N�R��[R��n���H2��������b��&�q���NR�O�(t��1�I����{�b�K�
�G�,��x����+��������J����������c������k����^�`j�%�q������(��7�������FM^�����4�`����?����T�$g������"�}4�
F,l��;��1S��g<��L(�
��$�T-�������$����sW�+��;��>���j K[� ��C
�w$���h���~�@x�(d�������d��I�lb�K��}Y�8���T�����YVR��a?���J�DD�G~�H����\��\���K���IA��������g���e�5J�qc3]k.�9=�����K�D>#$-��M�aq�HgG����h0'S�d�zN@Z�/-ME���D�_�6���y����u�~Y7��]y�
��oq�W��-t�'��a�=�������P��h��w|� �T���NI�R�}�aT���,�6,<��i�g*kt��i�!�8 �QbPR��@�t��Ru�}��V�i�F�P�Uv��4Pbh4e���e��^<.��m";���Q�CD��b�g��*j��9�����2/e5�rQ0��jj�rhd�eD����Z+������F�h��$�����Q ���� ���T�������T��b|`��UjW���Wg%�L���h�Y�@����*;����Uw��9�G_�<z��a��<I�U����Rt�I���w���'y���i
�x:�OF<��I�[����I�N*
b@X�]l���-$���i���Y��"�0�~`0�~��q������8�s�����w�T"'K6>�@�������R������{%��q�)���@�\MM(R��^���H����H12���f����{�
<u�v��&3!A��@r����v�B/�	2�U�sJ�Y v�xGT8��T�&3o����yA�b��P���� 5^ 1JS�W��F��Ik��*���Ak��+U�?!����c@z�?E��P�����_}z��|~s{u{u�����������$��%�<���'��kK������B W�?�����BF��{�Tx�����v��A�@�t��2X����H����� ��op���M���b�J�����4s��c������Kj��h�����s���,����z4���z���o�KA�(����$c���aa9R����@�DD6��W���k�`�F~3���$���<F
e�",�D��[^x�������
?�����6�4���]�++#����� d}4�{��4��
��e���n0������+���^�r���]�v�B��xl�l���%�������L�����T��29��C����)}�OXC����q���V��i��l#�5�/�����^^us3����Y���	�
�	GY��J]|������`]OU�����8XX��g7���	��������Zt8e�%�<4[4���E}���#ik��b�1Xy�5?.dblS:.�T+�NW�*�qU�t�+��-(H�o���
P����;04h[����D�>���"�wS���\fK/�Rv�*WYTU�(��cK�V),�R�������`��uB�t�����6����=3�G�����D!����M�7{g�%b���Z��-�r53 *���Diiy�Q|��|T&�[*�/���%���o���@�]�M>��X��
X�n��d�]-u�2$zd����c
��1c"��%$�7�"��Tq����?�����
>�9��vS+7��(_��P�eoKI�u(4��$S ���6I�;R�F#`dU�x���
�z�`�9V�h:%W^�������w���_`�"G�8D'$�������������ae��z�}l	�B���&��"��~:t��7F���4��&�f����Z]YC6�i>�h�
x^�$���U8jY|���X�88����J�*���
j:���5��X�,nj�d�ksSN,��[�r�]�	]�Q�fq� P��������6Q�%i\�_�?� _��
��9���|��fR���N�|}���RK�A��-� &h�E(0����J�T��n���7��|*�X������2?s+�A����l����bt/L��o&�vZ5�*�1H�15&��6������7?fJGs�rc��c�������=��^��K�N+����	���k�}�D��i�|R�&����}�����T��`�d���
8�������w���oV7*{*q���Q'�|[������U���a�n�f[sK!'��	.Zz
l���L���,~+�Mr�����-IB��9��m`� �H
��1$��EE�!5�*�t SQ��l�Cz�W��7{$@*�te�a����{ht�����e"����5�wI��&S)��&�����d'�����Z������!���_�<���qA�:���w:�B��J���L3V�]+Y$��L�^�Zf���#����`�.~�������o����;�;������xz2?�v����)���]������"��Xw_���Y����g8�~�4���d���a����)9���4X�7�iH
��u�}r4�;G�M���g�.���J��������t�����<�y&����I1b��"��������Y�[�6��������J� s�����@D�)�9.u��VK��3��y����G������������G���d2�b�u�8�N�[�D���`�iZ@\��2���L'�����s�P\�����~P��
s��5
����W��1��H�PW�,!isL�)H��?m�������B�e��Wl����-V�t�v/�N�X�.���
V�
^"(�K�|��b��i��TU\�H���)O���C�E�����j��%q�X}�{�,R�3$�0R]�Ec��!	�jQB����K��c�3b��Ng�B�7�L���?�1��/k/��o��O�)B!�yw�$����d����T�{��_��������-|��k��4���
�v�f���R;�PiP��)�T�Ex<8�V��*���������m�4���PQc���5�6����~��8V�i,hXh2��J*H����J���YGU<K�l�@�_�`A�T�z`�>����j@t�
B�YY@��
{pJ

`~�	�yO�(;(=<S7��O�4���|J^d~�n���3��
-?�-�(�����rI��gK���P����TFg�����]���D�o�A����,�g����^:��>�������v�X!kA��F��L�E�bc	/��@��2
]�����S���Q2z�������uP����m�v��`C�<
����H�M=����V
d�:�`-���ja�G��-h�
�-��7�1�3c��bn���X��p�*�,�n�����bV���_��������(gj8�_(���!m5��%Je}K`����P���8uhk(�"J�T�$����I���B��L"�DI��R�:�.W��%�t�X�u��1��HT/-�����	~J�
U
|a?���.����y���PK����<PK#�qJ,patches/0004-Partition-wise-join-tests.patchUX���X���X��}}s�6�����@��S#=�=(���fo�����;�q����+%��ve����V>���/� ��e��S��F�n�h6�t3�n�nN�C����p�n|��z��'Cr�=��f'�	�y�B��G4:F�������h�!�-:
��h����G?B����3��/�*
6��E�����?���G���:�}f�!=};9�������}�g�������rz��_�p�v4��-��zu��PD�\/V(
�(<����_��v���:�j��z�it���'B4�/A�B�2Z,���2�A�)��f/�g� <D��kR���rIB��
��$����p3K���w� ��?�����)-�pG�4�L��6����p~���vy����h�#6|*yn������*��V��d���Cc��!������f��GC�H��h���H7�e���A�7i:���	�h8<�����\������-:8�[D�k�m3��{��C�����&���K����#���SBoo��oW�r�G�����DkF&[���>�K��rzu��������.?^�[��l.���
!��*��c��?��"�����|��V<�Wg��g�����3���F����h�/s������s��GW�?����{���q$d���	��B;=����pu�3�
�����d����P�:GClB����L8��������r�=������k���zJ�G>���3�7}N�.X"�S��A������}� dO/N�������4[�.���s������ejEY�#��+�
Z��_����I�E�"�I�'1�W���s@2�F)����=�*����~J��?yM�*�
��Dhu�D������v=@�(�U�uE��"�~�� .).)�BZ!��5Z�H��y��?��_�O?^��������t6@?\~��D�Co?����;'�������I~�V'���_����#�;��^�����rs�.�~<��r�P&�kN������#���&��rL��O�M�
����������,4/�_�W���:�3B�����F���W2�C��ZY��� 11B?�W7��k�����BFZ���7�i������x�\�S%����3~�<�����H�Q�"{�E#�X�����HI��2��S��;���"����t��j
1!��������5�����Rn��N��K�a��a�����������4��i"� J?��7��>�Q��e����PG�~�����X]OR?�����h����������#v�e_��J�r����IF:�`nc���2��/��!�x,��_�/��1��/��0v�g
���!��ep!��5>�A�g�u3a##S�=�~iC���@�jqwu�c�z�c�_�����]���f7�)g�Y
}�}-5��G�==��_(=������P��`���1��e�����{r�1�2�ik�B��D�?�Y&��;�����.��.�P����6�>��W#������d��j
:�z^1�T������?U�@�����2�%��`��s���b��TW����c�E�$�ZS�(��j���8�����>gc��JN�������K�f����,�'Q����$�����O��T��������$�
��*Q�
�,�'IT���������wVcm�G>�r<a�O�_0f����������L�3�L��m�,�D�(��
�� ���x�lvl�+F3$�D?
�L����_��y���sb��)�FY��b�aq�q�fh��(���� 
�-�~T����<�f��vt)hd�����_F�tM��
�D��xEC�j��9c��j�[���X_Q���J�G+@ES��y��\<p�Z�����"����Cf�P<����^F�Q�*U���:�T�?��xxy�{�xx9k��xx�xx��������xx
�GE�TU�+�\Pi��2�>�������J��J����Q����SD's(L�\����N�^��o�?B�e���zu@�+�0���G�H�.�@�
��oq���F�P��_���a���(Z#J�!J�74���&{��<p��������o�k������?&70���m��m��=��]V&�|����/Q����$���5m� �
�u(�dpz�.�F�0(z�^�\��������d�����i-z��������g/z.k��F��s`�E������+;:��xh%�!�"z!��x�<1���]4��9c�=c����_N����>b��*I��h'�E}hy�&Kf�@����x
=b{����.t��x
�(�E�dE�G�0�a���0#&�%g���0��xU{r��J���A*����1�$r�X����$z�K����aRK���:�6��
�=�p!�|'�3�'�=��I�*=��mk���"��W�u�m\��'z�
bR�����������N,S���~K?'"�*��0�^�5)��$��	T�;���Rx.�0���%���rI�.<�|kR������d���9�K����h��7�\�-iV'��b�|��,W>�S�+��IsUN;����x�v�m���:�(u<8`�0>���W���!���;��V�U�&X����7A��(����3��[$���h��`�%^�������b2^������w��m>���|�Iy��ua;:�MZ�����9��h�`z#Z�vvo�����\ ,i��>(0�1�"��F�%;��
��o&m]�R�r�6.D���#:�[�8�d����3+"�;�R@2m��[@�!���I������C��t�/�zelb��VM�'8Yya�Z5�W6�d�'���+j��"��h. �����v;���=n"n6�au����n=^�W���=[t_��_{������h��r�fK�z���I������QH������br��u���&����i��B�^��{������t��L,A6���0���wq�N������)������9�Y��f�+P�A����^�Q�M�p�EU;��i�`�����B�m��C�������u��n/�y�X1-vX�*"u�:�_:8���� 
D[t��/�2=J0(��P	�$�1��k��r/
�`�e�QTg���':�r[<���C�%;�M��l��k)��*+mnm����O�v��X�!��Zn�\=���`��	��	��1������S
M�oA����F��D!������CfBP�MT��V7�����j4��oU���17<���3�3=5w�/�*���r����F���B������C��-�_<��c�vy���h:���B����/�qV�[u����Y-����/�����e�����?�����p���v7�h����������s�
j81�n��J�l����
|��{�-{��i+�k�x��kN�a��/�&�
6;��hD��H�p~z}vuz����7�����i=�D�T3���G�,?���w@3
��}�8����������@a�������T�`>���<x����2%������~��e�K�C�?�����b�	�%�T1c�2y����/���dl�mVd�5���� ��(�#E�sd-�U6�F�6�:9�Q���(H@��%������p��y*xW� O7
��T<��}e����~v�(��U�\����Y[�����,�,=���7�L�9���/IE8�����c�����a�<��7jh��0�I�����m�g7�s=����R�������<��J��F�p�y�*[��yv��x����h�������L��c�<�B������� �Eu��\�����J��df_����MFS��|v��P�)nl�%��w�f1dj|&������r�%Y����`�C��'>�Y%���g_�%�H�C�I[���[kK��	5�hEbI�N:�	x����il�
�j��*^U5��2|�5=��f)���S.�^��N�CUbl6���%��������xU���NMv�-��z��1�g{�r����g������M������(�.�d;���[���A&kx���zX���A4�q@A��V��m�W�����	�1.�*A��������lt:��@�:@;�� �?�������v<�WgD�����|�i@�d/V������g3��7}��������D��`^�^�t�����h����� =:�������
������OI�����/���C-%lF�����H�jiyf�(	F����x����}� ��g����zJ�G>���3��7���V����ilA������}� �O/N�������
���=�������?^~���!������|��X��_��z�� ��v��}�mx�������d��_�R���A�59T�4NZ������UF�l/h3"h�N�0,h�^�0,h �*A�����*
�4� hX4
?��a����4�,)!���~Lq�l���� �5�	t�;��NC;�";$�i ������������'uzK
�w)��t�k��3(q�{�`�z[X��:�g.�����d!����1+��Z��\�O�_���i_O�{������q<����������*�{��SU,Ofh���X|38K�R�j�T�w���'��xe���e�Hi
)��cT�h�HA�dH�by*��yv��l����i�����e�'�[�,Mwm���O������(k�&w�~��?�]��!�3�q7,m��24��n8X��l8��Wq�C��Y�����g�����=������~=1��q����������/�W�7���j����}6}��hd�+;n�;��l�E;|h��u{Rs�y#�l,_��[-A,@��SS��%�3dGCFD���)�)��,�2k�d�=A�����.�'���`^�"E�U�sn��N)%k+�G�m+�(+<����kra�/;VR�gr5����rw�o:����Wn���[jyY�����E�$k}(\�*���<�.1��C
�[�`�B��d�@������XK1������`�������+X��%���-p-�'��c����9�e1��=����-Z�[`�5C��T
�&�<������@�s�^���i �^%���r�L��y�t-�{d0���}7t���v^�h�A��t����k��#��^V6���9���`�B_�M�=��W���O)���1>��t��lXW��
�S�����Xhu��-�5p�Q�;���xxr�xl�B��'z\�B�tpKm�=��S���Y��^���!�4���~��)4K���8�yd/I�	��?4+����0�y�/'��v��n�vr
���� 1o�U���Q���� �V����N�^��*���?��A�mC��!�9���A��� OVDyv�gyv��d�G��A��I�s�<��)�`�?V�5y�����fH��k��hV�3y�����f9(���H���NCv^C������_]����|-���
��3K��Ff����$%-��C�Z,OR�'�\���y&�`�������p�u��X�I�K�'�U��s���eY]\�
����i��f�'�&Lqf���e���������������O���o��W��^��
Gs����eV���0h�v�E]xR�E>�������s���s�.���k<I�����VQ��������������m4�.��S�N���
������7T��C��b�T�*��\�;]�o	8�Jl�k�<�u����|v�i��jl��{��	g�5�n-��><�xv+�����S�2
�jx�	�"t�Q�& ���p����������"8�N�Nw�/�	�g��c7$��z��+AOy%8�"�'U�� ���DT�W�}*����*��-t��P�C�^�Z�N{�������S5����9���s8�����/���K)Q��y�2��!MA�R��i��j[� @7�i����i����J?2�7~X�����U�_���r)�jD��s}IE�y2��?�����h�����e��SL�����}�K]��������:�rO�A��"��<�{9�%�����g&0i��yIzH��3t��'��8���$�(W^\"n+V���I���R������������b��X+3�<+S�_X�_��K-�eR_K���U���������\�U��^e��~Y�n^�xw�_�\���/���rQ���2���~�3�S�~���`g��\�\����=�~yb�<�Vf�9eeJ����W��s-�eR_K��	���e�L�����`�*&�U����E��%�'��5<���k�~�(W^\�����rg����R%�w��F<_%�pj���#Q���'�G�%KlG)�,����8=��������FGM�X� ��	��4"�dE���tf�3#�|
�����!�����j�:X'��-���	�p�^�h�Ba<�-6_��a��{��������	Pt��[�h�(
���D0���������9���Fn*&�t
��:6"-�<_w�pTc�$���>�n����y�������M�&����d��l*�7�����I�r�>���/�z��<�7��p�
ts��'Z��y-hj�;�A��Q>������,.���t.��s�D�^�e��f�*[4/��������1+�������\d��2\��U��������t��k
���V
�������g�������0|�f�j�mN��5|���9�v��p@W�ug�Ws&
�*>Ok�4��j��Vu6|O>m�d��6_����5UW��ms����kfr�����J ��z*�A�4����������KH}��!�M�[	�M��T���8�����bU���+M�����"4���@\����G��b#�d�St\4*���&:.��n<�Qj:Q&�H�tz�#��[��'����[U
�c�
������L�
E��&]H(�y�Qp����G�w�@��81���7T�Sph���17<�����O2/����Ae�S/����V������yR\�]8_���80�F����u��-�Km������t:�v@����(3�F(�:�����ck�,����/�q�w+�g����&*M��9y�TM7c��Y���Q�-���lGi/�J�)����2��(�Y�<I�8nTY��Gf�]H
���='��Z�7�%_@�4��d�{��&M*�9y�
���Q��;�����@��C�6B1����N�++�=s���p�$y�(����@��nz��'E��y�ny`�^���Qij�i�qI����g���j��L��]�}v��"�%������$u^-���W;�%��������k����y���R�(0E�$Lv)k�+���^��
SU��w'�\^�vl�$���t��9�m9���q��q�k^O���s�=N5���5v-�C7�5m��J��<����vl�$	�q���y��lL��OvNZ�\�2xR4w�k�I��'
h���h�)�i�i�TU*��	%��c�'I8O���&��-{,��;�i���������=���y�d���a�q��p��C��M�{�6�F����������K���}/��"���}��������i���-k�3e�_z~_����>�'��4���0�%����L����>�'��4���~��p�#�����}������S��)O}���:�8=��?x����U��w<�����\ B-��	~N�;�H����0�Xzg�����yp��3~xa������K�����d>�����'��z�=���$�&�MkV	�c�y������xT�e�Z�6?�GSS����6�,t��b#'AY���bT�+��I]�w*�1w.���C�3n~���)�}��(�F1)u�zC�.
<t��: K�����+�e����������c�j���@M��^�2���6m��/��X�lO�+�>��R08���K�WCs�s���t�?mn�_��*I���7t��jN�BUg��5s}5������j�Ck��h
t'��y5X,o���j��I�����hCg�9����Pm�x�^
����	���������)������1R<ow[�C��������M���VSZ�K�8@�h�
2�)��>]h�v�e�\���P�tU��esy� ���Yd�&4�G�:�YJ�DHo)$���B��j�n�7�������@oS�T>���.���HjW"�;2na�LB����yl���c�lq�M����o��M����n[O��V�2[�E����k2ek��(wc�U;���.���l�D�W�W;/oa�L�����y`����lA�M���pw��Mb���o[O��V�2[�-U�����
d��lB�����.����;MU��}p��(�����B�K�yA���p6�,��x`0u�h(��<��B����5�B��k�<C�m��'�(�X3$��v���uC�
�k0Z
�3���������B���B(�9�<��B�F��5�B�
����s���I�����h�0`��X��[hQ?���C���:��]
t[��L(t���-�B�s��y(ti��Z�.���i<�Ph[���J�
��v����������u\6
�K��NDqvG���������%�S�N�^�
e[R��o!}�e;Ru��ej��i�n>La��hg�*L�Xn��n��G��>{�}�j�)�*mL���X?a�=��j�����r����DQ�k�!������k��[���Xc,'�������ll���%�[��,,������!W���]�3���UF(���]�:������]m���a��Fw%��B�1��.���GK�����[�=8�[����w{+t:T���!���[��'�\awW��(���z�@�l���p�Z~5G��6'0���	��Z��p����o���\y����G6��+�D�T-�7^�oU-5w87mW�7�X��~Mk�#����j����_/�vBd��6;RVq0��[���Z�V-wqI�wq"�:��Y��eX�Qx�q�`�L�6�-a5��X���Vm\���W����t�d�1"���n��h�Q.��[��0o7������X���"V�cx���&��������&�V����7���N�:��BhIgb�%���&X������o���N��;�����6U��\��O���N�������C�t�p��\�D)��|Y~1E�L�c_$���_����z(��r��CR��[���X`��B���� �H������X���N���;?;�t���Q�N��.S�=�� z��SAl1�m��Z���"a�qO)�����A(���g�0}.��)���Zi�����D23Y���'���Vy��z��)����������KOd����o�=�B��V��F��n�\����������0���C�r(���[u��A���B=�'����F�=���0��I�"���hj��K���`dp�6����5��4�R2��$��$d+:E�_�D�H����+����l�$�e.Y�i����}4����������S�������A��#����
��t:�:��}�Y��w>|��w>|�������u>�����}���[��-{������������w:��@�:pu&�:{����$o�����/���T�'������"�M����Fg_�C���������������>����G�=����h@��]��������>�������G�
D��N/~:�Q��G#^X�L�=H��/�g/?�%}��B;=���S�3�q?�h�Gdc@�DK�Q��0������T=3�F��~;�T?^|:��F/�����Z��h=�m�#�|��j���*��`�G�'7� �m1y�n@�M���������rZ�A����{��������E�"*@�|�o��k:0�]oPD���}��m���]���4���o�@ps��n����������V{�B�
!y�_�!v�C�!H�\1,� Uc9����+�!V�r�9�O(�X#��25V��������b$"��j��`�\��6��J^��6��e�������z�-�������p�(�-��$����3]������m�G������p�� !��L��0VI�N��q����1�sK��u2�)����2�955N�4(�'�LUV���Ef
;W��(�N��mm@�J�4�!��~����#\��^��"������e�QU��Q@����X�I�.�/M����n�
��|�T��,_��J0�^�3�~����/��	XWu1���,[��[�:��V2����<I�����wd�Pc7FC��%k-2����bO��aO�
aO�6 �� ��	+�x���1��\�_<��W|f����gz'V]�}w��d�V�'�ANw�%��c�i�����w|�Y,>K�]�O��g- ��1�pn��]GG� (������3N%!��:G*�����w���/��S����IY���A�B)��� �@C~n_`$���qG�gT1�*m���������J���y;@��,T�WF#�:M���6��Rf��YX��CcbPf�"�X�Y��Y,�,d��l�\ui3���l�6�2�v���Qf���:M��AS2�����|y"�P(��"��$��Vf=If=@f=���O*m��c����Rf��8���~�u�&��)<��0��p�a�������:2������������u��01��d��35�C-z@0
�����r�f������&�t3d���EI�l�4<�"8�D��Q�{�L���t�����<HY�|�D��g*�o������\��:���U��5tG	�Ir������:Y��{��9N�'	��7�	sJ�.K��#7U��2[�[�p�If�i��y�>�5eC2����d�k's�9�����dN�� T������O&s�9�G����+
~$�C�]c����.����GFoN��l�h"I!
�4�0m[C���mg,	
ItI��?�h�jN��
�r����z�6�'�L"�_����������~��
��#f[�����������6�Y ?
���� ������sN2g#P�|sm�|���tu��/�T.H�J�X ���}�8=����� ����"�"oE�G1PI�!EN��6�S ^I/	�}���y�i��%�\`������@pM�ex���DU�De~jp���a��Rb"*&(E���l�L���L�WE��EV�&(�]�D.�t��F�I��:G1T�~���&�������d#y��Z�b��`�������a�3j�����}��(+�M�e#0M��P������ysP����]
CCl�$����R#�Kh�LV�{��V+�<���Cj��d:��9�:�}���S�Qoa�XV�LG�v��v6����b:��/�q�a�Y]�-��:���Pb:,�A�a�7)�\���0���U����E�A*$�������f<>*�����'�}�r��b�D�'�����(�P���6���'`ut�q~�����`�5������J��1�7�p���������'c�Azj����$�P�t�����FGZ�G����Gl{0��FN������M�%��5rr��}�_(mz���[s���~�_(�wtf�{��h��=.��Ji�N`9i�r4�/���]��m�3�g�=��{g���t�]����"���i;��v�}^�ZB�	��bg������6E�wC�zw!qR��\h��1�.����t��<���\�[o�J
���|�z�
����g���tR���1�guUwJC�$w
���&��*���LY9MVY�G�3�jYu'Vb�'���s�m
@�Y�9����M�:+��s6�-���i�V��s�<)�b:J�T4��dyG��jj
�y�����<�7/���l��d�sx^kC����jH�]����!��!����`�iV�����f����`�TC��<�����*:J�USZ�SYu����Z^�����t�h��5x�F����k|oQN�!qYM��d���3�,3S������e�*(�zE��,3��Rb0�$��p;w���t�ef@\J��|�ef����3�d�T6�.�K��������,���`T1s����[m�����[�)\�p�L�����R]�������-���V�z8������lQJW0�>,��[���`��vp�+���m���`���lP�~5pp��3A���j`*�� &�����|M"Q�
*����-���|��v���h�H�F�v�O��5��������������p]����
���Cc����*i�6}*�by�;c�o����m�����-��u�{U�{�4�MC�����6�6&B=n�a�pa]��
�4&-�����O�	���F9���Q��XB���g
�Rk��6v�it+�����h��
RlF�Z�������50u�I50��('5r6�KH��c`��j��~6�n�#��B����~&�V�(^^V��@�����u��4�U�Y~��|�Y��t`s6+,�MQ���@�T��E)v	��F0�Q��`�V�]��	D�=����/:�������bl��^��3L�F�	>i�I66o-��><i�<hn@A����l�*�����`{E��n�a�5��M�I+L��yk����I37�AsZ�w`s3`����w��B��p$+�!�)��*2�\EfU��,�j�e��"���gV�������
t\n�T�g�TN��,�kn��M��~4>���p�.9�T�~)e��9O�@��r��y8�@�i����������I���Az�3m��#�3d�5[�����W�I)���F��8��T��'�C�\�x�6�
��_f:>�$yOY�D������.�F��q+��E)���rJ@E3�.b����_U������kV���[�f��yIz����u��'��8����X��e^\�d��X�b�'E��JIs�|8W�k;�������b�a��P��a��a����H�,�I[�Z>,X>�X>,X�����(��O����S��
��,�E��%�''-�5<���f�\�2/�l>����3�S��UH����r�rm����D������Z���*X>O�|`�Lg�kY>����|�`�<��y����Y>�i���:X��y{�u'�g��yI�YG�g�O���Z�|.j�W6��`�����h�TI�y���^�e0��]~M���������j>�'r�h�y)���|���h�������7�`pb+�&'n������;�#�Ngu�Kgu�iu�0�aq�O���N������0[ �>��f�I��������g��������2��jf
�\���L���Gd�<��z�"��������`���81���pj�GS�����cnxr��l��t��	p��e0�J����B�����7����(������\DV6�i��U���_[�ni(�%_<9�>l&��,L�P���b�]�5��T��M	�����0q�(ym�N���6��!�q�6��X�si��%_<����f!��,(�P�'�87a�Y�ML�y��8����1�j���;��������d����Y��9S�K��L���9F�G��?��99����2��9,����%�s�9�������9�;�
2��y�n��~���j�����4��eh���O	�Jm_�'�x����:���Z��F�#'/�=�L���:��N��6��6��j�<��Y��\�u�j@��)$T�D��:��$
H�t�9��8��������{��gY�ez��]E�'H����n���.�},i�=c<I|��vw�x	�D�O�|q��3����,�s��*�<A�}��v�>q��I���I��B���9hb��R��^X�d�uek]I+}p�V�.>�6&/��a��0���`�C����>@�`I�X��!z��?+<<F_k��LTw��i@?`	�RcR����+(g�����0��D��Upp�x2�x�/C�Z�y�,tM����M�yS�v`1@�F�h!z��?�@�h�K��y.
'���\*�Xkup���es�m_���������=c������;N)=���$���W��%�M�o��qn�z���`x�����W��&p����Z�p�����=KuZ��a��:�%/aI�I,i�
h�Qb�*V]?�d�Y-<~2���evV��O����Vg���������>�������
�|�XE�A��}�7�{�G��^]��H,
�!W�?����� *�y��)��W�o����}J���{����~&��-ElJ�QI(N�����OgW��%��'����zJ+�#�|�9a��V����ilA�����5�B�������H�,� 
��@1����x���?~�������V�/)������M�c~�����1�#�MY���$����.��$e������z;�t��.�v������]�]^yw������GC{�t�I&
Z��)mO��wO�������^���=h��a�s3�C�8���!�g�p��T�m�%�hMJh{�5IiqZJo�kYMW���]9�r�v��z�7���Z_a��/D��u�f����-8��Fu��fq{l�A
��6|�A<e�r�A9�/�0�SD#��U`��4������!Tg+L���p4�l��io���u�5b]�t��A������c"|���=��7^���9��������j���b�(
�I'3l��<}	��`f�|
�K:��s��{������IJ�d@��H�	� @"FT�������M�;'���Z$�t���
?V�#`����*����z�����B}YD�HV-V
U(��	�������o��\�~E���N�����!�����
���m��S�O;��I���t�V�[��x�+�b��XG]FA�j��xgVB`v�#�i�������� e���X����oG�����&���B��6��+oP���<��(�r���wzk#�Fd;�U�vE��JI����.���l�Z�G �Q�(��)��p�}�
k��R%�����E!u�Q?�y"�g������������( �����+���]FA9���spN��������A����O�;���]0���4-;��,�`�D��s�*�	1+Wl��)GR��T.eH��q��R8Gf�9����o
�7�#��{g��.�V��s��=��Q��d��6����5o7��&������e������&^hf�io����?�Mf���w���MpB����{D"������7+�/A��78F��Bn�G�M��Z��p�Y<�/{� A@��S�>�6���J[���F)T8���`����E�%D��M�/C��dd,�VS��.�����M@� X����x�f���\?��	~���2:6F�7�yfTn��'/������aIC��jN��/��n�����p��:]���B�Iu���;%��o���Yl(���'u���/����^X�����.�}�������a���C>"��W+R"�r��U#�N� C��W�$FI:W7��/���k��%��������������y�n�D4��fczu|4��^��8�+:<�K>�L?�IKL����.��7���2S���y~[��`]q��[_���4���&H������������(`M��&��vE^S����
5I3
&�	i�l�X4��`v��G��I��cN��eD���D*�t�%��x�B8�]�*}5CT��WxFK��X)��)��[�t(b����|�U!|���FZ��8���/�E����w>]B8�>��^���BI�_lH���Ct<�?�u�s��$?�`��M�*����~B����Xgl'D38
���"F�Vw>@�(����=\W�<.D�������yU1�@dg��	~����F���6�V���������~y�����_�Q��_�~�F��[.5.�.�+'�[�����{f���@�}cP0��[
T4�5����y��)>��������0k�g�o&����f��g&G?��"8�Lg��c���������D0`��8�|M_�|���+ ?�w�}@\��5V J�!%A|���A�������s�O����>�*B����H������) ����P�����P������4+Gl�AZ��0��{v�_����[WG�D�NpK��eq����b/�����
�W�m��eo�Ol�#����3�K��|��y���jq��N�7eG"�8*a>��<�g�ql�r�4�[����
��
�u����Nt����$��X{r��,�M���'C��w@���D������jy����b>����(w�\E�k���Z?
6��Y��<0���������������7�|�r���9�D�T.���G�,?��`x@3
"vr�@��^rR)NO*�������ri�CRR�K��K��$7X�X�Q�V���H�^�H���n��e�tA9�����v��t��>����>��x���q@�jz;���V�7}��H�iP�DK@�D�t�%��	V(��Z��jj_'j5l/j5�Ru$(j ��h����[��@�CQ�����ND�x��?H>�68.��l2�u�`�z��.����q�d����	�.����]�'��[�(�\|��&q�V��N[�l���h��A��^�q=�`c����,N%r�i��v��W���#2@��s$5&��z�M5Ab��G^L=����J�����������Eo'����5[�'=e����y�&{;^�S�7��K��W�Z�k�z�e�{�5�.�f+��>M�n��m���i sQ����E�R�-����W�Z��\q+�����r�$���]N���PO���5���<�x��Hn�^�wis9(P�/�#� 4��3j��mt�v��r�Pe��\	�6K0\�c��5�9�2w��I�����$����2
!b�!?ofk��U����
:����]���LZt�H�aG���t������k��lz���l���K���}/��"���x���lJ����g��v���1L}a7(i�)�a}>O��i�����G���[����;�k�����?����g�W�Y����6y�S�����*���
E���WS���/=�W��bv����D�����N���0�-�U
c��5�Jj��F��h�}�o�S�}�B�C����l����l����t3��]���'���[�z��-��z����3o~��O�f�~���,L|����1��9�-O|���y�*W(>�+�ec�� �J�x�"@1���! ���x�"@5���& ����6V/>n��?<��������EKIV/�'����$bwI��$��%��R5�D\_�����,HQ�J"�H"N$��%rT��
.�KA�s���ij���f09���b;� ��@�a8P�)>K��D��7H�����	���P����"���.���d��d�c�8�F��8����%�r�?~����/�%tX%*��h#�a���!/���uD�C0]�E��;J�M�+N��tt�W���QB�8��$���^����}KG�#����,�-,a�';��6A�-�O�XF�	VR��A���:��
];�������:�y 6R�����"Qj�����L�\�@��#A��� �T
b��=Kx���.����G^,oN�>���g����j(��U��m�%�������T7_!m����1�)�Y�]�+�����e���'�L����wlHA�e���!$�ia�4g&�yg��t��ea���`{:��:A)p���s������$�� q�#����D^�8�E�U#�n=���\f����W:a��,+?Q����������T4.�4��j���>�N���
�c���x���mG�[���I��H���m�������o	��je[p�k����oY�_AUc��M�x�����|���g��=W�P���e�X,��������&f{;�tM��!y[�V��7������v������<Wb���&����[0k87k.����Ll�M��7�O�=����:�DQ4���m��S�k-
V��9���Up�G��b�����O��d
�}���4��@��!��~��?+<<F_�|��wh$~��Ee$L�%���3�����������>���|��l��b��.b�IfT�p�U1��M2EC�����K(B�� EF%�8�Se�I�P��.�tU��������>��g��&E��|�Q[�	���$����4��at7��Km�y�;��;�+���Qypih���9)�!Oh�k5�=���=Eg{jg{@g{ig��<P����C-9�%`��$D��c��p������H�1�� ��#a��L��3�jg����9�;S\�q�e�=�]��ZGh ��q�n������h�����,no��0�1�����`�&yy���#
|�� �^{�Z����;�h�^�,��	�Y}	��7o�_��5G��{���(g�mU%3��:���K��Y�^���-���[��k�z�`��!b\����i��K�zG�����&����.[�v-�Se��V�*��]�����d�!%u1�/����:��;�<�����4KU�~�\"eS����)b�
��7C-�+m#�����`��X;�4@���wM5��#oM1��{����w�����PK��j�I��PK#�qJ5patches/0005-Partition-wise-join-implementation.patchUX���X���X��}iw���g�W��@I���J�,�i���\?���L`��i&���:��J��O"����[uu��*K����w1�//.��H����B����������t�)��+�Y�Eo(����������w�*�o�4����:*�����3���$W���Y������g��Q!����� �E@�_z������NE�{6�6>o��q����������{v���,>FY�I�:����i��r��K�(��N��F�@�i��/cY�I�y��-�X�[��-�D�x!�N�f.�DQ��E:�<�E!��*�e#E!(�YM�M�E�L�!��O�W/�]��nd^��q.��1��0i�����X�l�1�'�	U���a��a-��]�X�8���_�9��>�w���uSl�S����Xg��h��b
���8F�-s~�d�G&a�q��
�\6��	w��4����B�_���0=g>E*��Z�����`e�g��Z�J����d�c�&�B/�_V�j2�>���vZ3����0���o|D�2�7�I���g��I������������S[
�m�Q�E�&'����C���0��I�X���;-{���R����#Na��O@����_�f}h��;l���?��E���_=�L���I�������W�vS$�
���z�~��l�r�.&2S-������#������G(��2����&v�x3��3HV�b3�'�i��3*���:4�����#�����Rmf�x^mv��Y^����mi���������P��� �.�
����'L������"v'��������6�w95��
����,�]�'g�����7��h5�����v�@:�]�l��>�;j!���Q�1I�Sq|<K������qmQ#YM��b|>������$������u����H5��m�o���?���y��
��~��������a�p����"�����F�R��w�l�Y���U���S��h�5�&|�������/�Q�����k�����"�<=��x,W���M�2;�fr���r�������b5���g
��f�|���`J������h~_����q�.d����HE`�'�=�\�&�WG���&���,��'����������@��kpS��$9���7�S����������������s�t&rm��A,��t:�	v�PN�����
bFKc�6���/90�!l�E�21@U������Q������#?���8:�t.�x���C��o���~k<�����
��^�����IF�X�5�Lk������C�\D�#oVp[�L�!�*�(�53��A(��"O\0��5���Wb��(V'����y��2��.��(�@m��VRW�����0^�
�RDq��G.����G�E����b����O��"�f��f
b���F+ ��
�9��5ZW�S�� �,� �}@��b�8KaW���Fx��N��L2�D���p���8}���e����	�� ����Jx�y���|EG��R��Rcd�� ����J@+r������`�P[I�^
���.E�c����u��a;U�Mq�0<l��(�x� ��]Gl�������w�Y���M���{X�}�����4�:F ��{]�����4Zw�= w�4������y�$����clP��2�.���H��XB�"��V�� ;QYo���a�����`=�l��Y{�������1,L:pu�_%��$@��:5Zz�	���FU��-����;P��(�(���z���/K��mJ:�Qh(@��'b���3�K�2*"(�6q����|�+vV2�]�d�i�
:����4Z9��EsBj���RR�G"O�X�i.-�A�m�-w&�9NJ�P����Gd��a�K	�=�t�Y�"b�hI�����&�
.��^�
k���?��N��������x���X�?yv�(I��4(�P1��d�?�%��C��2�f����������x�d��I�p��#��;{�'��h��*�P6X2�!b��)w4Q<�h2!}���������P��r*���h������x���G4No��;��D��g�1����������m?m���\���qK-h]�!j�V�C;p ���J��+5��[��XVA��s�z�y�7:=?�t�`�������eH���r]�&��(G��C`$�J c8f�tv����~E@�:���@��R)��Lf����.Z'X������&K��~#V����nVq,�_9�S]���7o�����O�������v�=k�.E��|r��\#��l�u|MEN����Y�m����h7���U�*�&a|;~�����7O��O�oa�7�a���=����	H����q���@<7>A�w�R>�Z���k_���@Yz2F�	I��;b"j~���_=�PA���%1���-g�k)�����9��Z���td�:&����[���p�J�9w���S���`_0���$`d ��u����'��Q^:�:`R�;�q�s�L� �7�P�2��!������<'��I�#� 	l�X�S��8�4�b�����}G�a9���C:6����)Hp�0�G���p�t��vC�$������
tR�r1$@L��)�&����m���!�`HHM:V�qa��=w��<��'�R.r���2������6����p�F|������pa�a1wzpU���r,����(V)�,B�?�eJ���9��
�z�D4�$|1�p<��`j��T��~���&��=p���,�K�� �a���5���a��^���V�0U�j	�S@��L����S���Z��#�H���$\.���_�G�?;�zy��>�������%��+"8���&�s�L6����3N���%YGI���$_�$!_���7�}�b��"�t��Q6���;J����dn7��7�(�&_����'�?��G���qC[���E|�y|����q�OD��	���S���� 5?y,>�����V2��x1IIcC+U�Y����H�f&��q��k����@�����C��c�������������D�x|�<%�y�*�M@�P�59A��z�{fP�<�
�,��Vu��m����m�G�{�RhFkj�Q��F7�kZw����WMl+�};pFs����=��Zo_j���S���|Yi}�o�������_#"vt���$Y�F|���)/���S���C�r���������,A11x�I���?�L/,g�h���7*.��k�7\1����n>���87b��#���N�J��x�!(T��.�:���S �G��0l�j�:�hI�F%j���3� �G�B|Nqu�����T������\���C8i�/����!3�#����-q4��$��&|b����\���
�!����=f@���jD�����!�o
����G�+1�h���N]��&�~�����=��)�#I��m���������a1p��(���r�lx��L��T��-����q���!Z�]x���i�W���YMF$�7�h�����9F�[��Dy�����g�������^�p@y4�����)th����4�GS(	*�����"���,�d�&�d�X�D��������Z�`����H	������i�TX��iN��#�.����`���o�e^4����U��Z����!���$W��&+��Tc���<�^x��[Y#���:��-���{�C��`�W[�U>���E�����hHH�i,?��?+�������O�^��?s��uy�{����f������n>�S!��b��'����l�{�`��xzi���Bm���f�oI�nf�m�6����]�%��_���'S��HS��ue�,���=��*��T��I�o��,6�J����|By%@�����������D%<\��TUr]e�g� H%=����d�T�Ie�R6t:Jwl��S��?CP�9���h~@$����MK,���`'$I���b8����
kFe��!�<j��d���nA�k���������-�����n���pY���E�����/�id�X�����M�~
�8C������$�����?x�����$P0bH��2��k�H�����*��0�,�@����@|'��gY�#~�M�8^����������&���Z EnI!`��c�B�"�Y�#����"��
�@�0�l���
�G���y����8�B����D�AR��}�|����Vf����zf�a��8�{uL���N�A,_��b���aV�>!m��``/�`�����<���z�+$%�b���������z��)���T�yT����S�����,	�,�[�@�)�/��S����-���z�:�xF>$n|F\�q��$T���#�-�(���<|��=3v��^2��Fl�uLU���4�#Y��jx�1���L�f��R����
��������?~�t��e�qL�j����e�����d���w��^��e�������%��?D��I-M�eo�Q�q=�OG�:���p���3��Q!�F������Z���0��EC�s�=r�������h"����I�s!���`��i�9ui�zd��=b��3�������y��_�,"������W����w�8��"��z� g+&��O6�I�P���#dV�
M�����>Z���Ve�nb��V��J�q���G=$pe�-�$��\I�u�B��2%>�%P������ �e�B�`�	.��Uf�"Kb]��a�	���`�,���l���)�Z��vk���h����l�i��i�j�����7����9-=��OL%��x���HU$����|��!�^t����V��Bk���T��.�,Zt��f%?�Y� %o�YL�_t?���j3	"&3��N��	U>��l��c� PO�Z�n�8���Z��:��9��G��o���7�7^�)�=��K���>[5}n�P�W���<�d��#BC(��*�p��oA�����dU�j6+
P�
O�zg��Fy�:���1<w����K�h
��+��@��xi��")���o��=��-���u&F���Mz���}�/on�=y���q������+�{v�;?��tz�tz�����lU���i�H�)ZS�c����H��?H�sT��U��Te�~��F����e���M��
6�mK���H�E�ypUT�\�/����_�{}����#�`�X����������������PTkdy�J)�������<���[���(�L�;"W���?oO@����(A�
�� F!��dE�{��+*zF2B�&��:]#�$�U>B=�������T���j���^f�3i��ewN( �C.���P��p[zu���_�b���mE��a���@�:y����9);!� h^�g�����2�����r����z}:W����T���mxa���Kg�j5��QZ�T��"�n�;m�lhq�u;�{@��(;��G�{�*6E���w�	�.Q�!�{�R��S��;�)��1��t���R���u�H�
��mT��u�zvc]W\����N�	�Nu��r	��T��;(w��w�B��.&�K���
���4��"<�@��%��w��Z;@�*e�@m�kk�/�j�B(��VF�J�e�����|�&�@��8 ���ibH�P'�QG�� �jk�pr�����1��D����<��������V��]	�T�F���92�zm���	�td����!tL��M��NH�Y
��
V��������j�4 �����5k�snW���'�����):�%H����a0���������2��j���"����]�����w'������<�X)��������������5t������lf������7t��
��	�'��Z����&�]H�o���8r�4<��X2��d�PPL����rB�]%+�V�yF��o���[������{'�U)�J��������Xi�|�Kt����i�8�)���l(D0(K�����G8�cG9�(=#>�* et��T����`@�+^����Ch+�f�(�v�&����b�������L�Bh#*3
�T��<h�������h������[�#��VQ�t3>Bg���
+8Z�j+m�
��I1���d����w�YD�#����_��Vz��?�!!��n��������b�����)��y�3F:�����	)�����5���4�����'����F�]����r�$q0�&����Wz�ZBo��8�]P��Sv@P�����*i�c���E1���S�m���d91P�d�_*lD;�$:��w����0�m.=����	P��/���H�]��8P��|Q
�B�����`�F�k�}���|��B?aTD��i��]�5#9F�����AO�k������x���24���d�B2�Q>�/Hlb�G#���N�y�k�N����<�c���qb(��N@�(�=�
j=u`f�U���#����2�N�b���+,�v��\��,Pu�%�����@|A��������X��sk�G[(Q��]\���P#^A�U^���*>�p��r1��&�f��e�)�� Cf|G�^��[���+��\�����2���x��s��SQ�S�Ug�'�o�N�:�#�\���|�^��ri>�K!*P�P�U�@*N���|"�$'��=S�X�W]V����>%RIs��;�:Q(HMJ�E�R�U���/��N�M��%��Y&-��>KW�Mq��O�����r���c'�M*3���_�GYK�\8�!��r�����B!��h����
#^e��Xj���i����Z�c�P]������RTY9�L,CU}(/y�n���i�?*P��b�)>���kT�#iO������~�U���(�>&�/=���=���a%*EQ�)�n��>�8mte@u%e��"?�x��J�������-�}ek�]��$k
_�D<�?��2�?����7��MG����2����Iqo	�7E��vF�i2���n���C2�	���SK��F��qG���X�;�x�g��������ZY���-���)�d_�����;d�b���g���f�<IK{������a�LxJA���&q�,�f���s�kBU(�������WfA"���m�G��4��j�Q����iL�+��6x��I�X��w�:����G����P�"�����Y�k���"��@��=;��`]�V��SRO|H9����
4�o�
�M��W}�����r!�i�����h�m�7�������A(�L�%/+�����\���G`�~3L<,�U��T���p~{�u��T���V9�|���Gc�wr?����s�@]�1���z_�����LX%$���
�b���������G�*���5�Z�j���F?u:y^u�T�.�J��	��V1bt��x�2�b3�Q������{�����B�j����q�dzI����m����_d|������k1���a��*|�	��T��a7c�A&@�3�nd�]
�7��������7����q�(!7��m��3Q|�
~
j#�~��7�fJ���Z3[���8M�9<	�^e;�\_��*zZL��9h���}t:���+���u�+U�.��p2�:�i|z9�����W��A�\��zC�H���C�O�VF$���2o��}��u��ww6Q@)�}R�
o���{��^�GO'� wX�h��mT�$��+����m-�:�����K�!i"�Rl�U�'
�~Q���Q:T�`��<V=/���n�6�'��������F�
n���JBD������-�s[�]����������&<z�Cp,HIx0y���jdq���-���X(���O�}B�M([/K8\:�!8�7��I=3�Ml]��.�����^�[���h�i�������PO��X�LK�n��!��_$]%�*G�)���G�+���"(l���M���O@(�E�?�s����m�]w��q�Xv6���G�����7f�a�GUG(���U���dh�n3�����X���������
�����c����U�9|G�����s�Y����1��4v)?B`"R�@:�r;!-�2�wU����|��k`�850z���xSXC����x���e��8]'�O[G�P��,��J�4T`�}GAug��f��2e�|����R�cH	���G�Mpg�@�
^��`Su��^�Y���:�TOqf���N�l|�?�
������1f���n]e���R���	R��`Z��Q�����0� ������Iz�<�4n�Q�$�}/D�{�������p�*8����NGv��q�WHr�n$��P������!x�>�����V��#ZN9���w��uM8��������zp`�����P ���6�?����d�F��l��Y�����^	�&F����U~���������Y+�����^S�����-kV��<i��������������/����/�� {VR�uppc�G��������i�����4������ <����W��o=������f��vM_'zs�C?Z���yyk1��c^��-kP��?��C��qXo��UBq�q���)������^D�Re���m1�t�(��1�]:~f�<$�:�K�0���:a�^P�~��a�����(5���8Dk��[VG�������[5,K�rk�������]���C>Gw��y&�����Q�:�(s~����X��
vI>e�OodZ�B�5�\���q�9��y`�+����z�yy��y�h3�+�``��$�����dR��O�K��'|�O
���b�fC�����l���;*-�vX�Z-4�L*}o���5��G��������s��{����x/�i	�N��VU|�9������l�^�X��<X)��3�6�D�X��=#n����<4A��J��A��>��	Y^�m�w��!������[a�U��|.\�����������s������I29cV��e5�vA�&�
�r���7|����2���,�Wq�*E�����\��Q��-3���k���p~����0�q�<IEWZ�,{,�.f�r���VJN�k|��+2�]������_*R'��F��.��PI"�,�.�pH���2���-S�/��N��~.��P�h����mEI���`��2Q��)B���Z�_R�oG[
@��Z��^:�w��@��]�r�0'�J^9�2_�K��/���i-s�d���X%�Q9�� �6���b?�[��Y�G����)��0y�/�x�4�z���&�d��X��<7#%�Ks6$���\�;4�j�������M�|������&�X��;�������\��S��_���Y��/�Q |�:a$���G�����y�p`o�O��u���h���G^�V����c��K��i+�Kq�=nA{�����M���tA���t�M�b�5p��� [lE�:u��\��������xS�$K���vs�,�ve�����T�$bz�1"�7O��@�C&x��QSh����<`%?3�<3p-������:I@4^���DYIA�"���i���Lt�����>����(\�"k)��)�kDI��3���?,���N�:�+�(�hKc��TO��C���T,�B��ZmCiv�[�����T�����F�>�88c\��TB�F�z^&�k��I��T%��
����5��}��1���^*��-�QR1I�[�I/KT����J����xX�;3�WF�Ze2��m�����1�P�P�2�@h����@��������.��(����a�N�2�W�S�	����W�b�+Kf�V)i*��+���*3o���vu�~�HeV��,Tzl-A����]0��G���Z1��V������z
�LHz��E���x���U��=����"
�2��.�=x��������k�fW���/��3J��j�"����.��_aX/�'V�V���q�esX9�[&��W������`N���o�i�q�������0;>���tuu�W���
�v�y�L�r�����x��e���
T�y##K�w�Yy��GP���Z����0a����8�^����G�r��9g%��(�*QB��-���v�������\����N��N���6���F���S2L-�����rQ������U�<QQO'5����Y%�%�zq���4����Y]��O���x����]T���$�}C�����(B�,�Nk����"���'�?����Vj����H,����������y,��I������\	�
��`�&�xw�V�N���c�=dm�s[�����I����a�� �\������[&�x6����,P���e���1���*S0�Q�����@�.z����WB~�n���8���"�N�MNZ���q��>�W��_i�7�h�C��zO�L��m��rQF���h������O�P^
�������Nw���*��;$�B�Q+��vU}���� �'�� ����-�����c���j���A�SH_�%�d�U��l��[Y��O�P��#WZ�t������B�a��R��u��t�I�J�o�J?�������d������P�#���@�&���P�����z��<F�����J�=�zz�}�
�\W��A�
��������
��/���~st�.��K���?B��c�\�
��Q�:�j�)�_�m}��c{`������Jw7~pr�5u����Omq�Y�9��
���{�������P�,�	���j�D3�4����9~��Q�d>��?����6�t�NJu��}���������a�����2��C������\USYu���C+�K���uTdz$5�?d�w��:%�!��|���=�_
�#�)���$��z�4p�(>YKqF�2�[��=�5��
�4$�4�xNs��V�}EZO������|b����SB��K��9��������Y��4|��O]�u�N�������v�*�p�������h������rz��!%>C�Wj/O��7o3��<��mr���L�_�C��P���wk0�\Q�@2�����w#*&k�����	d���[B�����f��~z���'�5����[�X��e�M��c��n(������P.|����WN������%qU�u}�rD�I��.�K�Z��������0�7i�T���������8�0�
����u���FO=�|O���J]���5�k�d���ug"W5�=���L�������3�k�����%��.�L�I,U�H���.�h�����F��&�q��~"WM;L����75����-}�J*��	p=�{�EA������&4s��q����%����	%\�M�Y����y�f�e����E%�V0�39�?6��j�))X�����V��Bo�_Mf��M#���j��C���r2'����
�V������JnP�/������Q�v$�+������x���$zW�
��n^D�����n���*�]���{q��`��w�G�mP4�������N��x�����>�^��	w����������
��R��\c}<'�$����]�;?e�JL�������� �s���W�%a�����\m[�
Tz��>��p��V���8�(I���^���2��
������.>.�S��Sv�;��Q]Q��+Ey�0	��W�������'c��1���#���}�O6��@��������Q�l���WiG<��a8r�A�6Zl���u}��D)w��E�
X&�q�M�8ZY����Ih$�G�>d���2������<�d�+�����C�
I,,Wj{����d�1�jJYjJj�2L��
6�j~���rt���f,�l�x ����R4)}�i�V�^K��d��+��bFq:���y�����A|YFQP
��TAY���"=��l-�J���&���|A�o2�n��9|�	V�d�UE*0<
ma�E�A[o�?�������������?}��i�������F7?���������^���d�^��U�����+��T�sx�-�	��|[?*'���2��Z5�����%�:����C�������S�UW�P���\��k���Y�����,J�VA�\J��^Q������Ya������O���� Z�"�����/�;
=���U���^�e������ �D7'��������bN���
q���9HGN�3�3�x�W���Sl�%�y���5������|�&��a/L#Mr�	�@h{��S/8�p�b�n��Zvh��S�'��h������Y-��xo1N��/Y�"X�V�S���8�T�!a>���2;t�<���j'r�E��O�%�xJ����==���~��F��
���g�L���;G��&��y�����1,e�71>�hRw���!G���c�H��7ToQk��p�8��`V	>���{��m�(%�����]�{�~�H B>�N�?~�u����W/�����	�@�CDo@�Y����BWM�uk<����H�nd�P��0dp�s
���[&.v�Y]��m)p�@=eD�Rx�9�aZ%O.����V����!)UsDA����1��Cr�?d�����]U����@�8�i��rss�4o(�
��^.�gMK��\�S�]F��^��������`X�T To�J��!m��Tq��_��}f�$Z��B�T�.�Y���@u��z��-P!�XH�A��t������B���f��r=F�����
��������Cj��VQ{7�a��{y.�g��l�d��9�(����}&Z�����R���q�"�B�I��O�~b�(�Ub��>�����~G�>����u
�����{F5!@���n�b�
��P�� N��!>^��~�����5��=13��������T/�Ng:���]�/N�{ d������E/�.�+���j��Fwi�EfU��:�2��"��&�E"R,�
��a/��XF���^!G�D#�:p0������Z�4���(�H�x������������':��������)i~]���^o�Q�H���M�4l{���A��;���:�D����M>Z�ok��n��v��-���wh������&���{�[�WJ�uK^� ��s���������~�'�O��N����_����{p��&_J|'y|��4���9�G#��,nN����o�W6W�����>���[��5)�	q{0��j�'Uo��W/�_x�����
���G*V�u�������dpTe��"W��f���d�#DjBJv�cn��A��
���U	�N|u�d��}������L�{�F�*~��X�y���L�PCt���N�5K�D8j
K���0�2:��5v���7�b��-����h#��!�������7��$N�nd^�)���la�F�>����
� x�#��E��I�S�4Z��o��a�3HO"`�8��2'��}��rE�����>�M�]�s���Z{~1^�5Oz��4��tP�����:���=���^���$m�fU4�g���E=u����35��H��7]�^)5Z,�<3�O�������i��������H�!S�bR�A���*B����#�$Vb��#�����4�������`����Z�7��F��5�(F��e�f�R�:�h63}��=e� �(�.ez(�2�	�4��@�
~���P�\�bz����y���e��r�f�O6�#�hkh��Mj��Ry�Z��r:o�7,T����v�B5�-����T���`
oSC5��
��L�1��*>��������[��(��B!��4]��!��c��5�����y�!�e���4;F��@���#�r�xT0�a��Yd<@��%s��
�e;n���E���O!����������d�sv6<�O,�>�j.���d�#s�JH�/����6o=�>�Z�^g����5�PK}b��6E�PK#�qJGpatches/0006-Adjust-join-related-to-code-to-accept-child-relation.patchUX���X���X��Zio�H�l���/�eK�a���M0��A��� ��b!PdSbL�Q<3���Uu��N+3�DvW��� ��������������?��������=�����v�?��&��V%�P�s!�Q���Zo@��.�i����^;���?�����T��4I�Lyc��g��9����B5i@o��@�;���]����sz��n������O�wWo�s��v�K���"��KD���<�crcO����*������q���u�-�� �,��m^���yN�[�W�2���m���nd�r���N�_�4�����������&�g��8�k��Q��j�C���{����&N)S����3�+*@��8H�Sm�|��<
4W

�Al+*f��J)�
HXTdL�\7���$��$;lh��/"���J�����I�g�rV4HN�����\�2��-p�J"���b[���7�vk��XE3��fI�+�"��|�T�fpm',��1G��Z:<����,o��j]�H	��yc�=e�����/�����{6}�������~/��k<l����>����p�=��1'�U��F�<�k���0t�!X��b|�����T�n��
e>{�U�u	�U��*��xJ�tx"��,�l��PI����-�&�e����X�������NF*fA�Qm�&��X6�.�b�cm=�f�z7H��z�B��i'�@s��*�1S�Uc\�zb�� �i�������
��Q���E����z0���_X�NUu�+'/po�Retm��4���n>3���*�2C@�,3��[��-��c��,"����y�TWxudU�&P��Y��X��Y����oH�4�q6Pn��M�`��S����(��)����5�R�e��
�V�5�E��S���4��0�h�`17�O��x�L	��q
�Iy}%.�h������*M�����VB��Wb�����H��Z��F��B��.x���>H2�EK����0�����:V�ld#Y�@�	�R����F?�_���bC�����$����3�~gL�z���S��v��d�����D����b9pg��0��\�/m���)��[l<���+��"��i��Llv `&��u���)�[�[?s'�Y�R���k�rG���<,`$�^jGdk�\�^����WFo�v��(,|D4�S�����
:��9I�"o�z=l4i��r�Z���dbn |����g,0U2�4�.wu��D�������WQ=�N�d;� �T�Wa���1��V.:V��U��;�\U#?��d���5W���C��e��`f��-�l�L�i~�u %X�B!�<�68"����6����P�8���]��t��q������h��Vl�o�3��*�Clz�z�=(PA�)U�R�\��]�{+77�b���Kj�������+��(�����"g�H�z.��X��*�������MYP��p���u��9�Ls�?w\s�G����
��D���o���m�����i�Pq�X)6q� �s��!�~ T�IX%Pd� �J=I��h�}�����������a$�D�������,�gjY��I��]��r�"��y�aMWn�,��b��N�D�j�Ze��;�=�'���?��l�6��?3�%����K���������m���	�����FZ�T�<����m�%��!�c*t*�s-�0~�5����������}�A3��S.��=��Vk����S�x�u,���:�;�n�����\5�n�s�����=��f�>����5�6�0��K0��`�Q����e�����E���?�GV������n(yX���#c�����[�^��.KB�������J�\6��"x
��2�&�No�97�e�D���]�(/�i}�����*B7+s]\�S��3���y�0�#7���/4L��R#I.@%��dW�;�y���,U'�J����u,�����M���e���u��;j�����2�
%N|5%�s���H��b�}��Q&>��X�N�{�
�e�6k?��0�#c��
9��`���+�n23Z�
�0����Z�?�LI�^9w��Sh��E���%�Z���S1�I.A>���y��LD����hu�U	�Ui��T�u���8�q���p��K����h�6dG�z��C|�n���ol7�46����ttc��s������j��_-7T\2�Zz��X�M��z����F/��=�v����LBG��FN���� �,�?EA��m$���8"G�����o�K��	!0c
-*~flAC�q�x-*#>�;��$��_�TE��Oj�/��gF<4]/Ci���Y�$�Z�*w����1N�Eh@i�Y����5{}:�v{=�//�0�AT�e�i����5
�������T�1��	���c9��lG��5C�	 ��d�zk=
����ZOa�
��{�A��������g��j�G��m�������k�����i}���3T�p����z��3/�^�{�x\H���W��V�o�]WW���|�����mwG�]g��y����ym5#\�?h��?��r4�.GGDG;�1zA�|N�&i����)�PM�7�M;��m���n�F�3�<T�k�?Rn�q����{�#������H���+��e,y�m��[N�O@�V!��I����nf&p��0���.q��B�t�&Wq����������v�=g{�s��6�	fA8IV��/Kz��9!S:����z���HP����O������Gw����I ��mF0�/����us�m"M�����Z�GM�,1�>c�'������
qr66�������#o�Y�f���q'�����+q�v)HX������!��&��W�2�ki�b �i��]�c-	�~e~��Xz7~&��	��$��|��gO�d<$c/��Y5����{P�1>��w�Sw:Y"����%�Wp�O�[�Z[fX���[��KY��"���\����aGY�1�h[Y{Q���K�k+�b��]3����X#�o���!�7�@
���<sF���U/�iN��4���6��Y����E;ce��K��,��cfYfm�Y�����.��e��f������t��t�1m�����z���Wt##���I�������s�����}�D�x���M\>�� C�<�)���-5C�X�?{fl���,).��5��J���n���|y��-*�P��Lf'�2F:�s������������Uo�#��x���R7�1�S�i��0��J�F?K�)sfK�(:>?�y	���0��`��F�N�9������Gt��^��s�+@�����Aw�H�R`�@���2
�m���L���2~(����Y[��X�z�H�&�Gx��,����'��\{���Z?Xx[�?Xj�����%��Q1+��^9�u]:�0t�\!�z=]!�����+�����zI7�����d�T�����I�T�_�2���H�rI���y�ztL���.�N���k�j�ln��7>�?��t����9��m�78w{����F"����eR��{������^����:�k�G��-�1�D�(��Z�����y���,�.}���w�(C�32���W�J��Q/���r�������|I����t7z}y{���&�w5����m_����������k|�#��
T��:���pj�����>���PK���`t>-PK#�qJ+patches/0007-Parameterized-path-fixes.patchUX���X���X��;�o���?��lt%��[��&�7�4)�������(rd1�H�C�q7���3��"e��	A,�o�}����$�������;�&g+��t��OW���������l1?����s�6��'��31_�?1�'�+@s!.�&Kc�?�;7v�����\F�LvI���y��������*	����6�2�^��/�K|:;���g����.�_�����������[	(��J_��X��9�/R����Tx�b+�)�Dl�'�9"Z���FF"�HD�LK�����Y�H���F��xr�U&��l�i

�u�l�wF��5@�4FDN���X�H�oC�YBOR�R%� �������o�?�������p���OB���2o��Y�A�4o���9����hz��D�m�;g%Yx�W�=F���v^�kV	�	<y�<���S�������+7k"b��f}�M�/���[)<72�vE�T�;���)TG��p}�������m��z�tuy��������7�@	�1P�w"�7.���C��B��%���`�U���X J�UBdd��od�I�_�����@����,�)���)�&~���)` H�J�P����T�
�[��p�=
�E����V�/{�D*L>��lA��@�* ��5���j������~�s���� r.��#�A*+0k�l��pE~W��U�,�H�D�X��\�-	_�8������p���J����_*�v1�!�W
<���G���<��w`���� �K�J�}��,I����|�����
I�t���<��II��67�}�ZJ�J�>^��{��|�F�n�
�������(�^I��z����.CWY[��dyHw(P�� a�;���3��x�z_d���4�hr���h;2�'�w!�l&��i^��,Buy�hz
��	ei�(��&4�����A|�:��xS���/+��F����~�0�Lp��J��nt#!����`{%�8�~��\�2��`�s( k1�����Yc�������&'��h$�cWN�b2�ON��m):���T�?���`!���d!�?w���x�_����P
��L�\�Ib���O^��i�D�`H���c��L�	���
�[�w;��N<��~u�������x�b���r�:=!�������8�lq8�b���hj=r}@t�e��B����//�][�u��N�����&��'���e�W���E���o���j��Zl>��]�8|uo���D��5�jZ~zy���+�h�9aq����g����4
��Kq��2M�����������]����;J�8��F�e�~��;���k�#qD��
(8f��2�l�L� <�:M������f��8��v.��[�
�\t�{;�o\����B�l���<�_���Mu��5����k�����P���`��M�V-w~l�t7����26��k�����n�I��kNG������&M���y���=�<����!���4E�~�
jy��������!B�o1�l�������.a�rK_n��o�[�y�����X����5����A@���
���H8,D"�M@��_1wp���4(��k�>�:O T(�j��)�@�0�{}f����H!;p�K��&�(���L�$b��� ��@��H��XI���1N���pI�
�D�(@�#�U7��(r��&#��`��"hT��
��%d�d�����TH�r��H���$��-�?�����B�j��q�����Ogj��|m��T���F�����w�8���F��|:�����*��
N��d���>��c������H7��n� $�n�&j��(N��]g^,�9Va���8�_�c
�+hvrjA��b����.�����$�P���8��G�3��b�s.�C��!���B��+iq:&--NP[���1
�R�>�	K���:�&�hz�uB?��4�+���zH�L�J��M(��Pe;�����!P�o`��L������0|���M8J�{��,�VGO�|��x�ic�w�����!�w�57�����	���a
�I�'�	��|�_n�{�s��(��Y�x��)9�M����lS�q���E"y�x��`�mrR%H���!)�	O�k�f�kv�m8���ynDT��������6`��z����<��(C�Yv+�e��3��������P��6\[pe�k�s[�!�h�lZqjV�E������Z�(9���a<�J3OQi���	���HC�y��	�rF%���3|dut�u
h�V�
���B�'��(��[w�,�N�7rf�A��N��hf:9�/�p�!u��6��3��w���Nz�)��'?�C^�V"3V�4�4�co��d|�?R�c��^n�y�����mA�����Y���W�/��(UIP�2�
�9`���LZHJ�T�z�Iu�N�6��b��Z�(Z���Rc	��c[5V�V�����T��M{�
��������d6��������T1��Taij5�O�G��t�s���x]�1�����
����X�^m�	����j�w�-�)yj�9�����?t��8�f��.�l�a��X3BC3Y������U����q`�KUYs4`|x����!<��E(����]Q�]I:{L�[<k|���^�Pk?�
_b��.�r�f������~,��*�,1��|�������y�k+�>���]�a
��#3Nw��hN������9����j8N�p��#�1�B�(��5��.�}m�y�E��17	>��w����~���N_0�"�m����h�� ��������eI�5S�r��������?���pa��,���������F����^\/���,h��}�~���u�����Cy�����T��x��r��E�5�����R���
������[<�hI;�mE��n��N��%q��v�L�
����s�V��f�<4kI�nE�of�������d���#�%5�P��4q�����;e H=�'6�OY�H����\��}��Sh��Xj�X���/�.l`X�c>����E*���$55�3x�^�*�A;����t��4�T&)�P����O���]J�2#���c8:{�n�(E�^�^��s����i�n����p=��Y?m���	$\d��~����2p��bu�K��N�jn�i��:�_�%�	�r�~5O
����>Q��A>�1�o�������7�#6}x���� P��u]�����iJ�PS���V^�W������� ���m5�����[5�J�f%/hkV��:�N
UQ�UZ�*,tuVV:<���A(��^�d���r�f���pV��P�d�"�+
R�S2{��a�(V'������=�:��[k�:W6N=��XJ����&x�%X7���W��k�:����HP����]������t���b���1��^�b2D����n��z������F���
��V�Sl�����n���7�gx��8�������T��\a^��f/g�i ��=�DzY���Ft�
�M��������B7�p����nA���T������v�����,<�����lZ��Q���0�gV%9�������jQ�i���=)1@���/�0�g���4	�C���p'E�l@�&�2��|�Q����.�2�u��K�~�=�c���uc�#Xr��a�D����X�i@���t�$���(�mm����/@���x�/��b�T��D�%oB�0�k�����;G���x�/>
7:Gy|*������iF��2��B']�k�7.iVkSt,l�0�l�A���>|�u���5�G���p	\��O+����:�������z��w����5l�{��l�Y���dn�Hf:�����4��n?��_~%�Vyr��)�-�J'�"��n������Zl�����������5~�'x�B�/zSd��[��$�i�(!�������;�N\�X���/
D�_�xzq:��>�o_�b��P�Q}��~�
������a�z��*�������7,�}�~�����B�w�*���s[xW7��!�'h�$�<�>����������yue�T����|��������Tc-ck���w��^c'�T��6�|������>=^���t>����8;����H
>FG�'s�?�?����]�:���GP��4����X��8u���Z�0/�8�+�t���3�{������<�����i�GF�T1=?��zc�H�����]��Ds��1]z���&T�\�,����oj��B�����jL	��#�f���
�{����k�Em����10�����������7�'��>A��O�1��y)��M����+����4�=����c]K�����h>�b9o�k6�"��������_~`g#9���h1:u��PK`�u��?PK#�qJ<patches/0008-Use-IS_JOIN_REL-instead-of-RELOPT_JOINREL.patchUX���X���X��Wms�F��~�63�����x�$�u���L;��hN���DN�������	������������g_o(�9t���s���zA'�E�(�s����a������~���,�1_���v�X���v�!�9��|V�,��;�`�g��	��-O%�<
�0��Z������0�!P�v��]����a��i[�"����~���������sc���������z�4��E�M7�G���Z���/f�"��d���w��q9�L��<�&c�9�q��,N!V�IFL��h��Q�B�� '���K	y�����0g��G�r�(r}���M��W,��m��B�R&�� �3�T�2��.N��i	�HX��s��2��R��X�8��	���%@+vu~��R�J��jIZ������U���Bc
���Dk�X�m[��T�8h-�\N��I�lE|�D���������~l��c�<[=v���"l,��i�By<M���L����X��I�`��3�N)Z�.E���aw<�x������(�L������ca���B��=�Q����q&�
���_w���;,�e�=o��}�5�p��0�1Z����a&nxR��tz�N2�IF	KS.����2�d�^H����58�|�1Jx%*(R�@��n��
�\rLF�x���-j%�Q�-������:R�S��-$��y��_�X�%w5-�P��P�a<.��������� ��p��h��'�-n�k�_���1���e�9w \�X �������?����t��#�I)�������h��21����)n��D]����tF<��q��^��Q�^��b������P������qR�'�Q��a`�T���}�?�GBi�(�u�:���<)�exl:-2�F~(tM����#&~��DB�V�b�}
��^E����na&�����h8��P����
��a3%������Y��_f!
��W��]h����F��������
��7�q���������gQ��a]L���,�([r�oUk�d�w���^�j�{Am����%@��t/��������&	��./�����|x�k�4�#�S�T���C��u�
N�>����u�a�S����hyc�0<���7P�
�����;|��>��C��k�)��J�J���8-j�����NvQ��'4��Y�����eZ1�����X��Z����x����q�8��2�p���cBM���!2f�/����7tF_�����oWsZ��F�5�o�,�-��`�O<4����p����;����J�9��}u=n�x��/�k�I_%�O�=d�}��]]����K����YWt���@�9���yY��NsE/���*�d!KT��]3��]4���>���������������j�[��I�b:��LGo�J��>
Q������K���a�I�������1GS���_`��rQ��K������x1�?�]�"����n�$e����I�o����}a}l�Y��u������Q&���!��T`�����Jl���������&�m��]A��"��Uz�YR�����@���dD�V�p����c�����G=�S�U4M�j�Fod�������]��d��{�u����E�%�X(F*����$ukM��e&�g�`��@/��?T!��P��J��pB�}7C���w����t��{�������\�Lz�-���w��6���_N}X��1�0�/)�~�Ta@��
�<9z����{�$��I��2��0�{�t��gqy��a����z6a%�b�b^we��S������6�L�V%E������q���#�}�����d5�W�k=��A�������:��G��(c�z!hH��q�/'����j���
u�)f��:}���Y��PKz��,wPK#�qJ:patches/0009-Multi-level-partitioned-table-expansion.patchUX���X���X��Z[s��~�_���,,�v����+^��I��TJ5H�Q,$efX�\������c�]V�KOOw_�{-�!�Z�C�uY;p���m��.o�-������`�V�s��K�O�=�F�D��f���Q�	���H%r?��)����Z�����"��A��'���K��	\��
n�y��l�ur�8q�Pi���Q�O������{�34:u����":��#H�P�
���X/����,��R+/�X*1�����P���0�`�L �[!B% x?��A
�l@q:�"I�3n"�,�?�0�u��2
A2������)�~����$U�0���z������T��_ ��g�,�S�5�i�f������qf#Oq���?.em���A�����hy��������K�,~�A�[h��Y�*�*�� �7
N�X�~�9T��[�����b����o6��Z�qt��51���6Y~�5�������Gp[��T�������0�(L<
��N����g�$�D��P��yt�*���g@��g��{��1!��w4��E(:�B���|w�������;���~A��wg�xu{���~:���r��<U7�$Va<���
��?�7|�DMm�@�>�c�
B�X���Us���4���C���C�@�SY��1�@���^��%��(��
�a`s
��>J�q��$���N����(�_p� �����4���5dp���u��e��(|��+�����ELX)y��\AIrKK�7��'� &�r�p�BT��ee	*�&'�(@�9��T�D��D�y�J��!�/�m�
b)�]�y�"c�L=�������a�eO�>���Pz'�������P��9m-Z
8�6[~�V�.�w��(`Q�F
XL�9&���*�nC���g��':`:�<as���I���\Va�~�D�N&����.����b6��H(��W������HF�^������P����$�TL`0-�������]��*<R���+���W����7�+�@e�
��������D��2L�����p��S��1����5Y7!d���fh�iw��+������8�j/�-�C^��6�r����j*�B�- Og��Sz#�_p�9������#����7�,�` T��h,��%������z�/ "���s"���t�2o��	.��`���
�����UO	�It��0	}$>-��@>�1��7��&USX��jzpCBwI�^��g5(�v�i�1�D�
p�VY)ry����P�	L�8�=��������*�Y�^
v�SZh-C���L�gq^��\�R=&�/���(�}�����@�06C�4Qw���8C��T�",G���j�S,�TM��'
T��5b8� �B�	L*���,�����qQ���S�t�����Jf�g��Qf�*�,�O�4e���39^[F&���,��v��u�Ep��)���mF�9�t�B`��P�T�b�&�)�PgL;"\YI���o~4��W~��As��x����J��Xy�e��������U�����+2���������:�$��Z��W���p}�g�,��K�����'l�AVS��(EI�\
�]aX�!i"eHx��z%jS���|�,����+/��L��V��)/�Qy������@G�����������l`���(z�WZ
��6$�(�e�E�#�k�U����r>�b������90��8h�E����s��`�.��,A��"�g��A���(�����~:h$��vi(�)�lT�3�L|�u0_V�I��������#Wl+{�U��-��J('����a�����o.�)G�3�/daZ��4�W�SO�Uq�(U��!�R�/�;L��M<���F���(���:�L���#/�cD�/-��dD��������]R���H���d��+9�IE��2���D��gZ#��C�u�A���P�F:z����G)�C�u8]��f��6B���1��D��#�����Kp�m�/�a�I��pQC;�L���:kSL��7�Ug�����U�w'���r��P+���3}�������p�������]]z���n�����r�0gv8����a**M������$��*��i�@���3%gx���Lb.Wk���� p6�����"���^����n��((S9r��UsG�`��N�g/��L��G9�d_�c��b��Ew�t��G�����{\=�,�j����&���j$b[?�����~��Wp����u_I���:-��>��|������{tZ�j����R��xnJ�������mw������L��#����{������Y�Fm�(j�%�|�u�6�/���nb��$�E)�$��d8�#rS�4�������k$(6k�;�L=�+��:R��=_V�Z��|8DX���h�\	���~�)�9�u������V���!r3w���#U��m�x��h�6���
��"�^z�J(,�F
^8Ogd�b.����O�J��@�K�V���:Ve���� R2�&Y8�r2�lf���zz��f���������K����t��n\vi���{�F��X�,_��.ivg��XEG�F�=�0�&�"��K~"e�~�r$��UHP���_���`O���|x��,�$�4~�(�����������57t7]������7��a��y�+��[?��7��GlZ��q���q	3b�����o\��7x��GQdC�uHg�J��[(bMOw�=$�����T+`>�a�g�(e�''������J<n%b���]�,�w�a�����0���WpRy�&y0������T����X�O�yp�.��v+fY� �j���K��.2����c�n�����Y�X���3h�������3l�����o��v,�k��@����D��x;����
���&����6��4��F	+���G�&��>���wk��5�Q�5J�<��}�\5Lu���!�A�N�E�&@�"o}~�������u<�A�N�����	`w�{��&
�E�Wcw�{���*fG7V4�����u���:��b��PKw�K�$�-PK#�qJ8patches/0010-Multi-level-partition-wise-join-tests.patchUX���X���X��=ks�8�����T���=�[;���jw�3���x������-`%a�7���j~�JB��%�t�a*eH���t�.<�FC[��B�6����?0�i�Z��L_.�6�~f���]�
^C��9�����9���M�����X�����&=~�W�����������5�>��nC��$�����z�?�����7�wl������7?��'�����	����k���vW�'~w���n���t��vO��5��'����\l{.h���	�oO�p,��{�	Ep��;��	�c8�[�x�q0d�?�������=����{jBV���;y�����g:0�pk;���X��eF�S�W���o��2`�L���V,E����D���`����p����h,��[�t�������Y�kc���(��4�� ��m���0O�}�� �,���j�8
��}�z�^���65�W���de���HX;�������0k�K�1�0�B�m���0��!�a�7��x>������^�_8�4�)m0����G�k�����r~9�����l|���i�~BY��������t���~����l��������R�DA0Xz�'�����^�|�Q��:
q�=�\!��������D�$A�
e"1�(�"�r�9z���A�����7��.��9����j�f��C�wA��I�=�xGG.�$u�W�#�jA4���6�ggm@-�	������ET�� R�C�s��������B �y>};=����C�<�'��KpW�G�u=v��Ww�c >���3��>b������Y��f���R�D��>��VPY%���Ch|H�<Py����C���@R���rH�<�pU�;�H�����#��@���y%�A$�r���yh;�e�8$pH�<��y ��@b��^�y ��E��+"[6�iO������54��^Oo&mx3���=_������]�M-���&���^�\lG����l���_Y_�e�c+:6�/�tr������9�Fzr�����Lf�a�Z�9B�����F�t�7d�%kB�����a#v��7��z�W����Od1g�����(�1@o����M�����-�v�j���C@�4)<~�j������pc+b�����mu�[I�$��R-�4b���oIC9�|��s�LQ�]�Y�3�F��������G��Z%{��5K�2�.R�]�3�]��a*y!�`��\��U�H5u���W�nI�/��<�h�<'�1��F5��|vd'NT�<b�k>�tu�'RY��P�|1�pKi��y�����5.>MA�Z�y-�y-�y�H�;7!5A�Y!E�&�|���q4P�8�{`d���b�9F}��0��s5W�	P$
�R�%����d#�<����[��N�������y�|���]���,��v=C{S��>����c�o���lgn�����SL=���#Y�>: W�:;w������p�+��Y�����d�Ik�mK��>|��������>%�bO�s����a�����w��aA���� U��aY�#�/{Y�����T.�/{q�����$��5	��D=,Q�h�m������_��zzx���K�p�����Jj(l���Q��a�uZ�^K��Mi�� a����z�DO�n�Q��uB�����$5��6�y��]��S��O/����x��@�!�+HHHBy��3P��f�m1�����)il��hf1���)�$k����L*��EQ�=��[�v
7�D��M$W�m�Us�6�uPP�}d�(?\s�iC%�%7���R�eO����T��v�����;k;(���o��:P~�#������N^ E�������-e;��w����3k;Zd;Z�v�����Ib>�%��.�_j�[v���b�
���(������Z��,��RN�Mq@���A��(�$������3�4������^�}�iq&�pELw#����+�+d����}��>z�G�}����j�tD���
��t�v�8u�A��������D��E5~���U����(��$��v��1���A��V>����3�"�D��m��J$>����,T��A6(����h�U��r�}�Kn��m���h������&�WI
���2V%���a�'��D1jr-b�1�N����eu�CE�����������!�	"������X
���6JE������$��i�(� �1S����H&
;m�Hh�Hf��h�p��T �2f*�LIkF�LU�^$T ��D�R��jI�hI�����$�%�R�5JMh��U��T��P"���
�p9��*Gs�n��t�	 u�<,�2��	 5z�B����BE����m����)��p��u+X����xv�$��2��M<�A��i�r0_�"�P<S_��'�=_h�_��.cN��-nJ�&e���9�1i��A2�xSC��B�����B���|2_5����;��f@���7���>�:$4�L#�l���6��X�{
1J�@���y��5
��N�[����8��]BWr�3
E�/�}�8l~���n���,�tl�����o���C1��VX{����^�R\.W�p+���f��+�]�}$S��a2��7�W���ut���l\�H�K��H�#�N.�t��&�U�:��6$f+��T����	�-.V(,�����T�k�q�,�&#�2F��a�����8Q<hFE����HTLFEyZx~;������=nIW��G ������\�A=?�R��j�
Rcf�9�r��Z��H
��D2���)_���p�1�=�-<��A��fT)�+,�DYlTy���c73��/P��%]�A�c�K;�\��
���@��H�*U��V�j�*v����Ad^��P�Q���p�1�=��.��A���fT��>�D�sT�-���D3#��YW0o�,�
>���M�zh��C�f�������v���k��{��K�9A��H@�l���Z�Bq-!�Hj*�U*���{k|�h|�h�F�`[�R��q����k��_�f)�cE���(�����<tP�m�����j���E��v�1��pij�� _���F��v��si�\zM��������<�w�j�LD"ad�rR.����W��K���(y'�'�$i��A�K��T� ����Nm��t`^�L~P|��$��4b�0�(H2���f�' ��$'s���JA�2�O&�)>p��NA�3O�$D�}D�6�>�5&�d�o2Cs������0������b�,��������o������0G'DQ���?~s5���q�T����/V�P#|�2�zA^1P�V@�J@������e2�er�EQ��K��J���b�X��-�U�Q������eB�*�o���R��LlU1�=,�o��r��j�
���)wTL���7���������=��j-,����E�F%��*�TO��4�L����2����.�W�+��L^%/���2qv������`r�C�Y���2g��y�1�8f��R}�ZI\�\���=j�R���X�
�
<U}m^��#{�J�g�'�����V�fB�b����I���5���q��qY*=�PS��j:J�� �K���J.�T�B���E=�E�.�����v�r.���ae.�y\����k�}q���r��U�r���(�����`
Q�C���#��s_c~�0�c�0�������N���8�=�$�~p��;������������r�`05�����y������1@���
��y%\���c5|�^A����#8w��r��f�c���<{M�?�Hv����
��w��Y�{��t���}�`+X<�������������a�n�����d�KL�xe}����<kw���>��e/!���Y�d�����BY�OD��|xpH7���,���,\���������Z��>a����wA��K{u��Q���+��}����t�"|����6N`/�H�(�z��~M���a���;�t1!�jE���{�r�+�����,���7 ��v�H��g��0��������
��������,��6c#p4jkd�� �D�
�/��������������W��(�@h��yg�,o��+��6Q����J4�w/����qj��!��?��6�#�g1N*�^�G�>��|Yi�de���k)`�{����0k��J��#a>%���?��"(#�����6x3��� �������@� C/h���Y�H������l~9�$s22�����4��B(487�^D�/�3�u|���������T7�$A
	�A� ,r��
MK�BB/E����Ni�B\Gt
q�WH�.����yp55�0IPpC�HL9
�H�Dz�5�0D������d6��kr����-v����i��{t����#���+�^P?��f�
��YPlx|=���_�;'���������&:'����`�O�N�a~o�0_A��<l�"wE�E�.��/����o���]����F��^���^v3Z��od�'�%1�U3*�>������wH�>��*�$uh/����CW�} �����}�?��T�>��Q"DZ!wh�����uH�>��} ��@b�����zY��$�q�A�d��������4��I�<�����EP�I��y�*dZ�Y�W7��s�_%����_�GT�,R>��q����|�||�oK=�l�����������[(��^(�|���0>�
��~w�=����PK�v^���PK#�qJ:patches/0011-Multi-level-partition-wise-join-support.patchUX���X���X��ks�6���+�����E=�����$�t������t:�-&|�������$��~�iK�}�b�������SknOFs�qG��rj�����3�����^,������(dWb�����?6���@����M�Dr�~�[���s�}���?�0�6��p�C;
^�_�D<gob���{#��
p[��������F���}��?;y�~�8���-�����}H�����������xQ8���s��L��m'�v�<��b����M��n$Y������[��^x����`��}Fk�6��e��'b��;\��Q��n<@g��'�A�7��|����>%sQ������l`��	�)l���t��q�b��A��]o<L6,��Z�X$����9��\�"���
�i��vm�H&��9d�O�/��!�'��������.G�n��"T�
�oEh�JQbD|��BJ��q�s*���Q
��ya���
�[��V�����A���>Ys�H�$�&^��	���.�]m��d�����	hO10�MW���,��4.`�YNa�����s�	#Gd4L���K���~l��|��������SGT�nX�"Y0c�\0p	����p��SP�1�����}6�0G�B
�����.�y��;����qpm/t�7&f���r8<���tQe4�N� �]�
�<z�~`�	�>�*z_2�!x_���]r�Qi�VD�*;��H���Q����#���Y���2���W��� ���W;Lh)��Y��{���3v�8�Aq�bL��� ��g'��&�9M��v	��_�Ap��@h`�O���������&���H
Zf{.���Z������._���Q$���Rqv�9��W��6'KL]{b
�S1�l>>��L���Es�&�E����}����wMJ8�������2d���V��1����&
(���Nz��)~U��|�Dk_�~=�oZ�bE�;k��z��
2�@��v�1�m�Ek'{�<�q&�S����M������lJ�:��lG���I�}:�����q��o�_�0�I�R&�*X�����X���K0'"��uNs1�U6�mD����K���J�^RHQ%�R��G�}��X�Dc��W�u�0W����"�F;�%>�(=e{��K��PS���s&��z�����������Q����w:v0KOw��x8�����1���jo@��e�G�LP�>;E�Vu�*�!x]����vS����q�>d�2�]%+��Z�PT@	��;5�a������+���T�^"��1��!T��KN(��������[\�P�J��h��=�4�l�� �-A�wx�!��� J�'&��&�n��LF)�\4	�N���W������gz�OR*��ms�7G�`�R�xL�]�����BR$Z�����-q��#�dr.���o(��"�i�K�!=F�����:�����}��dB�K����E��Y��LL��>���c����*tB!�K"��!d��x�M��Enm��������>�H@�Q�-�c����}��e�4I5�t�����N�q��|�"�_��W���l�0_��8��F��r��p������p'�����bu����g�@��
"
a�#�S��a"obQF�/�~�z�PHT!7dWP@���t�9�����������dn��#���aX�#oJ�<�����=A1��B��`���W���Y����"� 4�"(��P+���}�r�B,���}���N���z�T��@E�R
��m,�2c�P���Y�0�MN�����C����5�0��o L$O�W�sO���,��0
8�;�O��PfU
58�*�r<�c8�-���2$�0<��M	����	���(�_�ODhc��5L�E(1=
Qms������������e-]Bp���(�x	q
E���h
���dlM�++���������"@RP��F����p���Ais\8T�5�����������/�]K39��1J�gA��a/^0��~��^}�~��r�����
�v�3�x!�8�K!��Y@xi���l�C�H��-�'J������^����N2
��=r!����'Pq��{�Y��2N}F<�
EaNa��R���Z]��=�N�a�#��XT#���{B���C��";+��	����5�Gb��$���d`!�����G��R�G�d��Z���cH�:K7��$J��9��i�f�?����v�r�xb�����h�`pW;�����Pia���j@Y���/�+��-|/������>��8��(�����
k����[�L`�
�g���b�f<Q���!�{{����.���z����tVZ�`2�E����^�,���^���BA�OIj�2��U`�����@��{���Q}������KN�y��x���������1}�*�}m�*,CL��^��������5�*?��&� ~#�-�=�A@�O�TK�^���D���(��&N)���^��(b�tTlM�c
i���!�q�e)��:�0*���Cf��LF��{�������A�=�J|<�P	#��P������b��7�`����t-a����{T:����U�VSeHX�]��!5���C�}�k(��G���Wa*�|��?�U�t%�y��/�T��[������89Z>�xJ�4,����<��(��-�[
��z0����D�tEpW����Nv,U��	�`J��ST��E�@u�����b����H�����Lb�a%���0�9%��S2%��1�Y��:�}\�0����R���!��H&��\6�ES.k�������Sl$�`����[6�.6�n]�d��������g}Y6���.�l���*���6���5��a�L�����-����i��y,�X�>�>�����lr����C������W���bS���(�9��t�Q�?��m�V�H�	�[�T���%-q�e���a��hU-"�`���m�bu ��tMd�t����4�F�A7���)X�l4���H��gE��������~�
�<6a�C��o��U�������H�(C�;Vx�����w�w��� �m��0�+.���<���go��/��k?�����7Z��|�Y����� W����c���+<M�hQ��zV4��l�t�$d�.��H�����v.�]+<����5�}_t�5�N�
���\_�yl���z�[l���Te��b�G/�k� 5=����PQ�#�Sc
�m�k��[c�H��n5rM���"K�x�������M�,��s:�O��p��cw�8���#:���A����t�}z������������K����d��M+;��
6L�����W���i3B#��S��Q�_��-�&Y~T��V��R� �=�xg���V4S m*�����B���O���1�K�i^|�,g_���#������5o���W���7�w�`�������F��o+�_@5�C�@������U��.��0��,�
[~����Q����.�K���,���i���)�����	�~�
�{�����r����m��B	��{���X����U��.Tq)^k��H.at��^c'NN+�=��n,!�Nn����+���i)�z�.L��m����Y�t
�\��aZ������.E<��P>2��5P~r%��k(��>`�uTPuO\�N�;��k��������Rm0k1r�O�{��8[��1g��g����Aw�` ��0fM�
.��!<��F����O��rD��������y�AZ���43��V��q>�*)p���2&%��y�B���]\��X��p0`mk�.��v��PK���d.7PK
#�qJ@�Apatches/UX���X���XPK#�qJ�N�^Pd32@��6patches/0001-Refactor-adjust_appendrel_attrs.patchUX���X���XPK#�qJn��|�Q4@���patches/0002-Add-function-find_param_path_info.patchUX���X���XPK#�qJ����<-@��)patches/0003-Canonical-partition-scheme.patchUX���X���XPK#�qJ��j�I��,@���'patches/0004-Partition-wise-join-tests.patchUX���X���XPK#�qJ}b��6E�5@���qpatches/0005-Partition-wise-join-implementation.patchUX���X���XPK#�qJ���`t>-G@��>�patches/0006-Adjust-join-related-to-code-to-accept-child-relation.patchUX���X���XPK#�qJ`�u��?+@��7�patches/0007-Parameterized-path-fixes.patchUX���X���XPK#�qJz��,w<@��'�patches/0008-Use-IS_JOIN_REL-instead-of-RELOPT_JOINREL.patchUX���X���XPK#�qJw�K�$�-:@���patches/0009-Multi-level-partitioned-table-expansion.patchUX���X���XPK#�qJ�v^���8@����patches/0010-Multi-level-partition-wise-join-tests.patchUX���X���XPK#�qJ���d.7:@���patches/0011-Multi-level-partition-wise-join-support.patchUX���X���XPK�
#60Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#59)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Mar 17, 2017 at 9:15 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

This set of patches fixes both of those things.

0001 changes the purpose of a function and then 0007 renames it. It
would be better to include the renaming in 0001 so that you're not
taking multiple whacks at the same function in the same patch series.
I believe it would also be best to include 0011's changes to
adjust_appendrel_attrs_multilevel in 0001.

0002 should either add find_param_path_info() to the relevant header
file as extern from the beginning, or it should declare and define it
as static and then 0007 can remove those markings. It makes no sense
to declare it as extern but put the prototype in the .c file.

0004 still needs to be pared down. If you want to get something
committed this release cycle, you have to get these details taken care
of, uh, more or less immediately. Actually, preferably, several weeks
ago. You're welcome to maintain your own test suite locally but what
you submit should be what you are proposing for commit -- or if not,
then you should separate the part proposed for commit and the part
included for dev testing into two different patches.

In 0005's README, the part about planning partition-wise joins in two
phases needs to be removed. This patch also contains a small change
to partition_join.sql that belongs in 0004.

0008 removes direct tests against RELOPT_JOINREL almost everywhere,
but it overlooks the new ones added to postgres_fdw.c by
b30fb56b07a885f3476fe05920249f4832ca8da5. It should be updated to
cover those as well, I suspect. The commit message claims that it
will "Similarly replace RELOPT_OTHER_MEMBER_REL test with
IS_OTHER_REL() where we want to test for child relations of all kinds,
but in fact it makes exactly zero such substitutions.

While I was studying what you did with reparameterize_path_by_child(),
I started to wonder whether reparameterize_path() doesn't need to
start handling join paths. I think it only handles scan paths right
now because that's the only thing that can appear under an appendrel
created by inheritance expansion, but you're changing that. Maybe
it's not critical -- I think the worst consequences of missing some
handling there is that we won't consider a parameterized path in some
case where it would be advantageous to do so. Still, you might want
to investigate a bit.

--
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

#61Rafia Sabih
rafia.sabih@enterprisedb.com
In reply to: Robert Haas (#60)
2 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Sat, Mar 18, 2017 at 5:40 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Fri, Mar 17, 2017 at 9:15 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

This set of patches fixes both of those things.

0001 changes the purpose of a function and then 0007 renames it. It
would be better to include the renaming in 0001 so that you're not
taking multiple whacks at the same function in the same patch series.
I believe it would also be best to include 0011's changes to
adjust_appendrel_attrs_multilevel in 0001.

0002 should either add find_param_path_info() to the relevant header
file as extern from the beginning, or it should declare and define it
as static and then 0007 can remove those markings. It makes no sense
to declare it as extern but put the prototype in the .c file.

0004 still needs to be pared down. If you want to get something
committed this release cycle, you have to get these details taken care
of, uh, more or less immediately. Actually, preferably, several weeks
ago. You're welcome to maintain your own test suite locally but what
you submit should be what you are proposing for commit -- or if not,
then you should separate the part proposed for commit and the part
included for dev testing into two different patches.

In 0005's README, the part about planning partition-wise joins in two
phases needs to be removed. This patch also contains a small change
to partition_join.sql that belongs in 0004.

0008 removes direct tests against RELOPT_JOINREL almost everywhere,
but it overlooks the new ones added to postgres_fdw.c by
b30fb56b07a885f3476fe05920249f4832ca8da5. It should be updated to
cover those as well, I suspect. The commit message claims that it
will "Similarly replace RELOPT_OTHER_MEMBER_REL test with
IS_OTHER_REL() where we want to test for child relations of all kinds,
but in fact it makes exactly zero such substitutions.

While I was studying what you did with reparameterize_path_by_child(),
I started to wonder whether reparameterize_path() doesn't need to
start handling join paths. I think it only handles scan paths right
now because that's the only thing that can appear under an appendrel
created by inheritance expansion, but you're changing that. Maybe
it's not critical -- I think the worst consequences of missing some
handling there is that we won't consider a parameterized path in some
case where it would be advantageous to do so. Still, you might want
to investigate a bit.

I was trying to play around with this patch and came across following
case when without the patch query completes in 9 secs and with it in
15 secs. Theoretically, I tried to capture the case when each
partition is having good amount of rows in output and each has to
build their own hash, in that case the cost of building so many hashes
comes to be more costly than having an append and then join. Thought
it might be helpful to consider this case in better designing of the
algorithm. Please feel free to point out if I missed something.

Test details:
commit: b4ff8609dbad541d287b332846442b076a25a6df
Please find the attached .sql file for the complete schema and data
and .out file for the result of explain analyse with and without
patch.

--
Regards,
Rafia Sabih
EnterpriseDB: http://www.enterprisedb.com/

Attachments:

pwj_regress_test.outapplication/octet-stream; name=pwj_regress_test.outDownload
test_case_pwj.sqlapplication/octet-stream; name=test_case_pwj.sqlDownload
#62Robert Haas
robertmhaas@gmail.com
In reply to: Rafia Sabih (#61)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Sun, Mar 19, 2017 at 12:15 AM, Rafia Sabih
<rafia.sabih@enterprisedb.com> wrote:

I was trying to play around with this patch and came across following
case when without the patch query completes in 9 secs and with it in
15 secs. Theoretically, I tried to capture the case when each
partition is having good amount of rows in output and each has to
build their own hash, in that case the cost of building so many hashes
comes to be more costly than having an append and then join. Thought
it might be helpful to consider this case in better designing of the
algorithm. Please feel free to point out if I missed something.

In the non-partitionwise plan, the query planner correctly chooses to
hash the same table (prt2) and probe from the large table (prt). In
the partition-wise plan, it generally does the opposite. There is a
mix of merge joins and hash joins, but of the 15 children that picked
merge joins, 14 of them hashed the larger partition (in each case,
from prt) and probed from the smaller one (in each case, from prt2),
which seems like an odd strategy. So I think the problem is not that
building lots of hash tables is slower than building just one, but
rather that for some reason it's choosing the wrong table to hash.

--
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

#63Robert Haas
robertmhaas@gmail.com
In reply to: Robert Haas (#60)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Mar 17, 2017 at 8:10 PM, Robert Haas <robertmhaas@gmail.com> wrote:

While I was studying what you did with reparameterize_path_by_child(),
I started to wonder whether reparameterize_path() doesn't need to
start handling join paths. I think it only handles scan paths right
now because that's the only thing that can appear under an appendrel
created by inheritance expansion, but you're changing that. Maybe
it's not critical -- I think the worst consequences of missing some
handling there is that we won't consider a parameterized path in some
case where it would be advantageous to do so. Still, you might want
to investigate a bit.

I spent a fair amount of time this weekend musing over
reparameterize_path_by_child(). I think a key question for this patch
- as you already pointed out - is whether we're happy with that
approach. When we discover that we want to perform a partitionwise
parameterized nestloop, and therefore that we need the paths for each
inner appendrel to get their input values from the corresponding outer
appendrel members rather than from the outer parent, we've got two
choices. The first is to do what the patch actually does, which is to
build a new path tree for the nestloop inner path parameterized by the
appropriate childrel. The second is to use the existing paths, which
are parameterized by the parent rel, and then somehow allow make that
work. For example, you can imagine that create_plan_recurse() could
pass down a list of parameterized nestloops above the current point in
the path tree, and a parent-child mapping for each, and then we could
try to substitute everything while actually generating the plan
instead of creating paths sooner. Which is better?

It would be nice to hear opinions from anyone else who cares, but
after some thought I think the approach you've picked is probably
better, because it's more like what we do already. We have existing
precedent for reparameterizing a path, but none for allowing a Var for
one relation (the parent) to in effect refer to another relation (the
child).

That having been said, having try_nestloop_path() perform the
reparameterization at the very top of the function seems quite
undesirable. You're creating a new path there before you know whether
it's going to be rejected by the invalid-parameterization test and
also before you know whether initial_cost_nestloop is going to reject
it. It would be much better if you could find a way to postpone the
reparameterization until after those steps, and only do it if you're
going to try add_path().

--
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

#64Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#60)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Sat, Mar 18, 2017 at 5:40 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Fri, Mar 17, 2017 at 9:15 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

This set of patches fixes both of those things.

0001 changes the purpose of a function and then 0007 renames it. It
would be better to include the renaming in 0001 so that you're not
taking multiple whacks at the same function in the same patch series.

adjust_relid_set was renamed as adjust_child_relids() post
"extern"alising. I think, this comment is about that function. Done.

I believe it would also be best to include 0011's changes to
adjust_appendrel_attrs_multilevel in 0001.

The function needs to repeat the "adjustment" process for every
"other" relation (join or base) that it encounters, by testing using
OTHER_BASE_REL or OTHER_JOINREL in short IS_OTHER_REL(). The last
macros are added by the partition-wise join implementation patch 0005.
It doesn't make sense to add that macro in 0001 OR modify that
function twice, once in 0001 and then after 0005. So, I will leave it
to be part of 0011, where the changes are actually needed.

0002 should either add find_param_path_info() to the relevant header
file as extern from the beginning, or it should declare and define it
as static and then 0007 can remove those markings. It makes no sense
to declare it as extern but put the prototype in the .c file.

Done, added find_param_path_info() as an extern definition to start
with. I have also squashed 0001 and 0002 together, since they are both
refactoring patches and from your next mail about
reparameterize_path_by_child(), it seems that we are going to accept
the approach in that patch.

0004 still needs to be pared down. If you want to get something
committed this release cycle, you have to get these details taken care
of, uh, more or less immediately. Actually, preferably, several weeks
ago. You're welcome to maintain your own test suite locally but what
you submit should be what you are proposing for commit -- or if not,
then you should separate the part proposed for commit and the part
included for dev testing into two different patches.

Done. Now SQL file has 325 lines and output has 1697 lines as against
515 and 4085 lines resp. earlier.

In 0005's README, the part about planning partition-wise joins in two
phases needs to be removed.

Done.

This patch also contains a small change
to partition_join.sql that belongs in 0004.

The reason I added the test patch prior to implementation was 1. for
me to make sure the tests that the queries run without the
optimization and the results they produce to catch any issues with
partitioning implementation. That would help someone looking at those
patches as well. 2. Once partitioning implementation patch was
applied, once could see the purpose of changes in two follow on
patches. Now that that purpose has served, I have reordered the
patches so that test patch comes after the implementation and follow
on fixes. If you still want to run the test before or after any of
those patches, you could apply the patch separately.

0008 removes direct tests against RELOPT_JOINREL almost everywhere,
but it overlooks the new ones added to postgres_fdw.c by
b30fb56b07a885f3476fe05920249f4832ca8da5. It should be updated to
cover those as well, I suspect.

Done.

deparseSubqueryTargetList() and some other functions are excluding
"other" base relation from the assertions. I guess, that's a problem.
Will submit a separate patch to fix this.

The commit message claims that it
will "Similarly replace RELOPT_OTHER_MEMBER_REL test with
IS_OTHER_REL() where we want to test for child relations of all kinds,
but in fact it makes exactly zero such substitutions.

The relevant changes have been covered by other commits. Removed this
line from the commit message.

While I was studying what you did with reparameterize_path_by_child(),
I started to wonder whether reparameterize_path() doesn't need to
start handling join paths. I think it only handles scan paths right
now because that's the only thing that can appear under an appendrel
created by inheritance expansion, but you're changing that. Maybe
it's not critical -- I think the worst consequences of missing some
handling there is that we won't consider a parameterized path in some
case where it would be advantageous to do so. Still, you might want
to investigate a bit.

Yes, we need to update reparameterize_path() for child-joins. A path
for child base relation gets reparameterized, if there exists a path
with that parameterization in at least one other child. The
parameterization bubbles up the join tree from base relations. So, if
a child required to be reparameterized, probably all its joins require
reparameterization, since that parameterization would bubble up the
child-join tree in which some other child participates. But as you
said it's an optimization and not a correctness issue. The function
get_cheapest_parameterized_child_path() returns NULL, if it can not
find or create a path (by reparameterization) with required
parameterization. Its caller add_paths_to_append_rel() is capable of
handling NULL values by not creating append paths with that
paramterization. If the "append" relation requires minimum
parameterization, all its children will create that minimum
parameterization, hence do not require to reparameterize path. So,
there isn't any correctness issue there.

There are two ways to fix it,

1. when we create a reparameterized path add it to the list of paths,
thus the parameterization bubbles up the join tree. But then we will
be changing the path list after set_cheapest() has been called OR may
be throwing out paths which other paths refer to. That's not
desirable. May be we can save this path in another list and create
join paths using this path instead of reparameterizing existing join
paths.
2. Add code to reparameterize_path() to handle join paths, and I think
all kinds of paths since we might have trickle the parameterization
down the joining paths which could be almost anything including
sort_paths, unique_paths etc. That looks like a significant effort. I
think, we should attack it separately after the stock partition-wise
join has been committed.

--
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

#65Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#63)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Mon, Mar 20, 2017 at 8:21 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Fri, Mar 17, 2017 at 8:10 PM, Robert Haas <robertmhaas@gmail.com> wrote:

While I was studying what you did with reparameterize_path_by_child(),
I started to wonder whether reparameterize_path() doesn't need to
start handling join paths. I think it only handles scan paths right
now because that's the only thing that can appear under an appendrel
created by inheritance expansion, but you're changing that. Maybe
it's not critical -- I think the worst consequences of missing some
handling there is that we won't consider a parameterized path in some
case where it would be advantageous to do so. Still, you might want
to investigate a bit.

I spent a fair amount of time this weekend musing over
reparameterize_path_by_child(). I think a key question for this patch
- as you already pointed out - is whether we're happy with that
approach. When we discover that we want to perform a partitionwise
parameterized nestloop, and therefore that we need the paths for each
inner appendrel to get their input values from the corresponding outer
appendrel members rather than from the outer parent, we've got two
choices. The first is to do what the patch actually does, which is to
build a new path tree for the nestloop inner path parameterized by the
appropriate childrel. The second is to use the existing paths, which
are parameterized by the parent rel, and then somehow allow make that
work. For example, you can imagine that create_plan_recurse() could
pass down a list of parameterized nestloops above the current point in
the path tree, and a parent-child mapping for each, and then we could
try to substitute everything while actually generating the plan
instead of creating paths sooner. Which is better?

It would be nice to hear opinions from anyone else who cares, but
after some thought I think the approach you've picked is probably
better, because it's more like what we do already. We have existing
precedent for reparameterizing a path, but none for allowing a Var for
one relation (the parent) to in effect refer to another relation (the
child).

Right. If we could use parent Vars to indicate parent Var or child Var
depending upon the context, a lot of memory issues would be solved; we
wouldn't need to translate a single expression. But I think that's not
straight forward. I have been thinking about some kind of polymorphic
Var node, but it seems a lot more invasive change. Although, if we
could get something like that, we would save a huge memory. :)

That having been said, having try_nestloop_path() perform the
reparameterization at the very top of the function seems quite
undesirable. You're creating a new path there before you know whether
it's going to be rejected by the invalid-parameterization test and
also before you know whether initial_cost_nestloop is going to reject
it. It would be much better if you could find a way to postpone the
reparameterization until after those steps, and only do it if you're
going to try add_path().

Hmm. I think we can do that by refactoring
calc_nestloop_required_outer(), allow_star_schema_join() and
have_dangerous_phv() to use relids instead of paths. If the checks
pass for a join between parents, those should pass for joins between
children. Done in the attached set of patches.

try_nestloop_path has few new variables. Among those innerrelids and
outerrelids indicate the relids to be used by the parameterization
checks (see patch for details). They are not relids of inner and outer
relations resp. but kind of effective relids to be used. But I
couldn't come up with better names which convey proper meaning and
still are short enough. effective_innerrelids is mouthful.

I am wondering whether we need to change
calc_non_nestloop_required_outer() similar to
calc_nestloop_required_outer() just to keep their signatures in sync.

Should I work on completing reparamterized_path_by_child() to support
all kinds of paths?

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

Attachments:

pg_dp_join_patches_v8.zipapplication/zip; name=pg_dp_join_patches_v8.zipDownload
PK
#�tJpatches/UXz��Xz��X�PK#�tJGpatches/0001-Refactor-adjust_appendrel_attrs-and-add-find_param_p.patchUXz��Xz��X��[ms�F��,��I]UB� �W����8���Ub����a��C60(Y���~O�^	������&	�����{��:6b�:�O���x2��h8?_NW��h���3g6���?F����V������������x�wq��O����[�������p��t�=;��4~�b�L�]S��k�2�L�
��&C��OF}��n�I��3���W7����l0�_q-W����O�(^X����Pz+��������c���;��Z|������V�g�C�H�@����y T��##��)�a9c��W����A$�c{�zN�t���[��]�V�k����vW��Q{�g)�j����w����0��6�����<`��z���]���M,`��m������%"��������=���v����=�$>�Ol4J����P����n�`�RS�K�����:�mv^�n=i�M��

��k�	��j�	hY�*#A���B�!�I+�+��+��h�Y$c��8"�A[SF��
����"�;��u���I�qv[�����[/Ni����JSQ�0/<�X�L��C��u�u��<O��X�-7���v���B�li�����`��2<�g�G�Q�B�����B\��=+�A<dxh�g�����'i0�;��A2�bH�_7�����]�|"��H��t�����sdi�<j-�_B|���A�#F��)��U���+S&�y$C�V��6�t&�Iu��6��Jt����u�L��fp�L~��r�G��l����A�>��4�������;�NL����7`�������������A���m��H�4������	��T9�;+������|������uB����h���B��bc}���~�
Brv*r43����
�S���D�\��~o����8=#�J$8�/���u�w/����a!��k��9�����k'�"R����U+$,�C�f�v��t�\�}gy`D���98�[o����E�w� k�"X��N�fj)����5��_`rQ�����������gv�+8�.������"����q&�a��o+9���9�~���?&����i�Xb��d������<����^���,|�V�p�}I���ye�O!S�!}�S�7)��G2X����BJ0TT��,O���� &+�{�B�;������Z��g��-m[���������-@���9��[���VV��$�PZ%BN�"�_��S(���r�oi�7����p��$���pC9),~#0;�H����o�]�b,��S�h��l�V0-��p8;w���`���x�H�EDE_U~j�C���#���{�\h�}��T���~s\B�������&�x���	���fE��x� 	�]r���p���ql��;I�����B�<�Zn��F�.g���/j�F������'��t�l$�2��B/��'�Tu&*�2D����\	D���������C0�X��P��f>	'�����E}��(0��K�2C��Q�R�D����be���*�������)*��R����)�F
�
�On���T���E(�GI�wQ��|H'Y�4�S������D�8=�"�f����P��j�L�I�rY�����z�k���@v\�����<�)��~�]'���=_��W�����c�1Y���;!D�� �Q(QL3s��0�HD�7��|C�9����dt�)���l�z'��;;"$sY���q�
�WH�Z�����T��H�����+���8�&����-���h���'��C���#q��ZD�1D��H35�hg��E�F;/�N6h-�����Z�=����A�W
Y��>;t���:�����[�U�����Lc�a��X�b��X�Lm��_�{���Z+������*#DX "����9]�o
���@r9e��aj*��pdj�My7�
���.��gO���oFd7��9|���������%Z;�d�@���
���P]0��%-)6���bG�*�[�+���s+;�$B�����IG���N����2U#Gir:{�=%�q��G*������j�,��F�S���:��\�>��{sP��5��%��";g�P�%�O'�i��
�P�
����3|�R=%�)�HH�o�uz�-^��*q��������P�,"C'����a�%%S9/B����xL�����L��R}����
��k�sR��d�@�#N��5 ���rA�J�v���b�F<�����u��Zi
O���m��/o�]_.�/>������������&�*��W{��\q{�M�Fm#����g�"g@'�%��H3�i�:Lm��v�N�����S�J7�!i!>^������{RZ�l6p�7�;i	���P�m������g�B5�J���cf���
�����l������X�9y*oydS��"���Y��w�*K�%t�����T�+X�/�*��/5W��H7��,�$(��b���^��#��G3o�8u����/K�P����"��>h�.��������,�]���s
��g$e�s���������������A��9��:�3n��}�?8b�9����E~�'����-!����^�p�f�&sR��`6���q}��Qe���.$��Vj��N��J���,�����9�QM��[��������E����nLj�Mu�����mH��A�����n���\e����L'f��
 ~����L���6�4�0'���6�U��P���g�@pMK��)�t��
;lhY����P:
������I;b3o]�0�y%y���e!��m�����c��"!9D���6����'�5�o��5b�������(�1mkDNN��Q�P�>U�j[���Z�*�)�>�����yr�����l�$��7��1��]��U�B�R��Ox����b�qM>]A-P��:7��9&���+����DV�����o���Sw��B����z��O�:��o,��pj�$�r�Em�(���TC~�MA�O��Oz	T��0�m�H�����V�D��
�����&�-@��'(i�jBR	�4�I�N���\M������A
��w�G�Hd%�<������+����&�C]��'�-�9nY�Ug\�:���u��L�d{f������z��I<r�Z]��kO�K���Vjr�de�<r�jE��VAW�K�r���kI[��:�F�w����I���s���m>�����&�4���-���� C��#s����U�uA���2��xG���l�����G:"���%�,:���?����k�������UH�����9D����QGVG���X�������"�����x�-��C��i�;�2U�uB�|H��'�8����,�����%sn��:�5:Q�J�J��C��-;\�r���D����[��
A�]�yu��H%3����4�\��V���s.�h��7kY��<�R����F��������s-������D�������?���45F��s������P�����������(��Y^.�������R,�{��Z��{a��md��;%7���
q���.+�H���*����WKc�"�W���f)8��?��M����pr�����.�e�"9I�h/�Hj��C�5�&bG�S%V���>WR���������|r�^�rqv�x,�in�Y����g�T�]_����f���������u��N��U����&�9����
{��G��o}m�oU����z*��^�sk9�M���t6>w��&�V%D�:�J��g5�v|�r���F��rGzMM	�'��s��\�~h��:�@�Q�L�[��6�^���Q��4��?N�b��K�G�s-B<+�(��g������f$�����i�,����N.K#�Nu�|�q2�z��3s0�.�I_1�^�YZ����`�,A"�iK���u�Y��~�6�A�DXz�N���y=?�!�hWWo2�c�$F����GS�:�-���Y����3�����x�����X�v�{�Y\����� ������_��gV��|����&�i��=|y����=	o�4w��/I�.�j*�����&�����4�)?M��F�}���,�����9{���_�X$���@�2����m�=��bh����0��b�K��.J��
��^pe���!�YK�&U������5������$�J��h4sw�h8S��%F�4�zE��kQ
c��4�[��9[���|��+���F�t)�:���:���mLy]I�j���2���<3�o��J����9���v����
QR�i�I�b��(9����X�Oi�dnR?��v8��Z�e��p���'B$�~DR�{g���n����[�g���}�U���\�g*��@�K��26+����D�t�5u�����'$��E���j�H�����y��vJ����X�	�S��|wvyD���������HZ�~����H��� �n�1�n��:���c�Ldy�**S��ig`�Ab]���H���@��sq������~�\�B��D=�������Z-��a9:��&��5��Fj�T	�����t��^g���3�N�/���9����*�ek���^;�C5���^89���Mj��F��U+!���M)x�nF^�RV�;>9=�.������}����R�-1�io����PK+_/�PEPK#�tJGpatches/0002-Refactor-calc_nestloop_required_outer-and-allow_star.patchUXz��Xz��X��Wmo�6�������%K~��l+�m-Z�oK[��0�D�\%R��i��wG����n�K��{{�H?oTE�����x��4����<��<'�(�,O��Y����ZIx�k�b���qF�s����z���������w���'\����<O�LU������:Dx��!vtv�'�A8����6��g��|����/ ���/����j ce�H�M�T�4��V4<OT��z}`2����Z&��&���W,�[	����a�5����Jj`u]�@����>i����C�Vq�_I�*<�
��BC�0��x�K��r�������)�<�^Jm8�I�fZ�����Y`�,TSY�!����e�6~����t��fAnA*��o
�!,9H��ZTo�j,U�	�����+�
�!�m4���}o�p!��U���K%��\f$���9�o\�+LW��_b;wx��{��l�����H�FT�oF$1���!��_�Y6��#Z��U�*�+�(B-��4���6�{Vi���fVu&P��kd�W<B4�p�wwo�����������������~A�������g4��i>��s���l:�����������'����c�}���yb��e���{W2)y��N���8a�
�,����N�*Ul�=�	]X�whN�����,	R�K}o�^�\1��%�w������W�u''[+�3�t{�\>�=y���������t�L���_��kL��[DNF�������c�r��.#86���kUC=u��E�����l��2�O�oE��7�G��L���nS�KGn��sdg<F���lO�}X�E:���,�4��!M�0p3"j�p����\\I�1iK��g����Y���%/�Q��5\�����.�z���R��]q7�9~a��mZ����������z��"&8R|{1�<@�=��!��/+�N�t����&��6��~���Vo7|\�Mi�����Bt�r�r���k��5o4���'O5M��������%�{��mY�vt|�F������mz��6���[m��+1��z�����d�F�����k-�������<�;��C�
��`Y��V��9���[6A���Wa������s����q��B�An=�K�^��G'�R�toW`��X������r���
���8*�5k���;�`)�g<F�����W�������������}���O��zwx=���Y��	gEA��|�9X�L��b�v��e���X�?ct��)�H��������|������Wg�]��;�h�
��+OV����������c���S�aA�-���)�-��Z"����8�3��PK��?�PK#�tJ-patches/0003-Canonical-partition-scheme.patchUXz��Xz��X��;ks�8���_���j%����d;����8�&q���~�K� ���H-AZ�%����x|���qbK"��F��
�K�5;����rqx<]�����l�M����8>�Nf����d�O�c�/b���l4zE���h4��0���\�Y"W�7��{����?�*�L��4�"X�d����g�������;�`z<zut��������Q�K�����W�������h2���'q���mx��Y�I%�b���KR����H�H<g�X���q�d@���&���	K�������lf+���d�Q
����J0���kgez)�+�
;�	I1y��	��D-�t�M2?IS!7I��,!��da\�gi��t�nW�i�i��;�,��%i �����-�<��A���-���������pv�GYx�rF��o%�0��~������$0A�"�����7`t:v����Q	G�o6����U��
��d�F�*�l����!�(�����L��% 2��F�R��*|�������
aK��HG/`&�u�W����A���.�/�`�l�p��H�0c5�Q��r����������Y
7�`.;�����2�1��@��a�`���1;dL�2���+E�MF��Y)R:��~��
����������x��.���N�GK��|!��l
&j4;:B�>cv�l����N�����c���(�q0��%,�}���EJ
��&I��*���.�I"����x���,K�<��0�:�NL����Pf��Gg�}3�'����a{�*��@B{&}���!d&$lDQ�%���*UW�[E�}�&����7>�sU{`� ��7k��6���0V����Y���)V��>hP%���a��������##�����o�\B!K�9}DF3V����(X�����*:������z2�`���Z�(���m�z�s�8W�����>N�+g��s�0��w�!���4w��>�E��#���0D	�F�(@D�W��cx�o���@�0�;�v��9`Q(`Z�o���H�E�0�Y/�$f��N���7�91��K4?=�w��e7�_~a7��?���o�_��?^~�
^�Y_���<��m�l����)���(S
��?v�n�Z����������j�P�*Y+����s���,�8���Jn���DoUo�p.>��3��D"J�%�N�,�8�$C�>
��(�9�ir���:�4 ���B���	.I��(���p�zd�B��[����0��������Y�h��Q]z�p{�)�+>����<��'�hi�w;�+J:i���%i@��f�Q�e#�� ��?�3�/��Ez�Oz��?�6�!G�)�b��n�������NdhL���9��4
OC!���&������|Y����9>px��
q��dZ��$�����up�i�:���|�B ��F�m)�
��r@�j<>"�8��������&������U���!>-Iu��
�e-*�3�J�$�aADT;�G����������^��?L
���5u� ��?����E�*���a�t���� ���Z/����J����i: =�M�'��` N&��b����hW<Z��2yx�	Q��?t(������e�h PvE�?�LX	�������n��?9_�-���������2����e���q�*�ptB������b�&dn�a����RH<3���($O�$���9�����2�E24�����j��+��M�5�B��Kd���[B��(��-����q.�mR����>�b�"�`�8�?H_TQ����D���k`����$88PSK'P @#�,�� �����J��>E��4E�hL�qt8�NQ6J���p��JYWA�GI�H ����Zev��(L@qepV�c`�`L<�8C��:%b�\<W�St��������a7��H����v���I������(��WL��|H����<�C�O�	x����;�a�=N������H�tRi�<v�n7��> ���C���)
���P���<��;�q�s#�2�V�$�I�J��F��S��,���t�3��f��BJU��g"p���
��Ct�P�r=5�H����## ��"}�����M$�r�).U��������l�c`B��Z �����^nd,<�@����PlM���"p���HM���NqZ��|��E�4:`�3Aj=�Pb���/[7�0�%�����l�����]����	)�Y?�#��)*�,������������������O�o����}�d?~ �/,Q�A-��8���X[��\���2X��>42F0�+�����o��igZ��dIg.,��������[!}��������*)f��])��nA�0� �a:V�1��7�vE��bR+-�\{�xn��ab����U\������U/PX���he�BZ�S�dL��8,,G��]H4���&��RY�]�������b���v���#���C��Q�������
*�:�(o���!p�;h�Ac�+�5����0�����B�GC����@C�^��Z�Y�����1����J���oV�f�W�k��Y���M����S
U:s�����{�*�zS&��z��ByQ=���	k��79�T��)�?�cu�m�����E��?���~a�\�:k�5a�!3�(�`�Y���4��n	6��T�`�~�������{vs{I����u���1�E�q�^P�SA�Es�Y�^4����0��6�q�-��eX]��B&��&��2qL���t��R�W�M����������>�uM�/q�C��eLN+A�c=�,�|7�K��e��#.e��j�EUu�R�=0����`���k 5��*]����
�X/�M��Q<i�.k��(��3SydI.�L�m@Y>�{�w&["�h]{�u�����Q�P3���8K��Vg��g{�G�������]� [�.�k����^�$�%_���W�j�T�
vSw� C�k�S�!�#��4>S�����0� �����!X'-`���69�X|����c�c=o7�rc�����5X���x�C��u�%�qvF�MR��Z�#��8�K�m���w�����"e�����Zo�2��g�����;R��!:!YGg
�]7
��6k�-����cK���T6N�D���C}�1�%f�e�6
��Hm�����m�QG�V���%��5��Q��� ��B��A�_�?T�Vse��TP������FeqS�'�^���rbYF��[�B�N����*5K��2�����>�V���0/���������T8����FC����x�tR�����f��[*t��lY��0A�/"���U��JOt�/h��W�S9��E���)�3���$X*����=�,'@��$Q�q��fRf��s�R�D@�`B�k���YMA�n�c��47�!7v\=��-H�q��#���J������R(}/�@>�v���N$}�e�'n�qR��W�]/�J��K�LV`�� �S��xp�n��vu���W��u2��) l���Y����Y�1�
B^��������<�L������4!Wk*����$$�c�������C"H\Td��WC�b!J2����{�7�:�����!�U�1l���������w�D?U3hj�o�aC�R5\/Mz�'#�N��q���KQ�w�CxC5�zE
�O���u�>�	��������=�����,�L&f/�X-��o��h�S0m������Y���^Z]�����4�-G���t4��LO���VE��u
���tWdj:�����9�����qV4����e�m�v#�vy5��xo
B���"
���l�B�`�h?���N��PuS1��Y�K�o��;���c� [��?#A5�/"�K���}R��Ej��$u�z|�k�qV����h��.~���:�B��i��(QJi�K]���R��LiaQ�����q�eY�����u1E&����e}��<��/E@�>����;���k�����L�X&�Sor����(��CFAm�9��\�9����~�_�+��x�XI���+k����9��$����6�(�a|��|!��T�+�Mq��+]�w��X��tK��Q�v�	/����x>DD���4lp�*.\	����c���Xn)�"��
s�B�C��$�,���>�����T��{���uH��ZT��j��R������������
*��;3��fL+�����@7�[��St�P�d�]5�r��$t��8����^9�Wu���$�2�{_9���#M��~����Yd���N(U�%z� �eN������1��`60��
��=0*j,�a���������o����:��-M�_^IiPxUI�6����g��(�k,(�*V��'��z���P�aV������B��M�Ga��2��*������e%�"/2��������qf�N��'���^p��Y].�x�|��*�8cM����r8x�M�{�*�m9�tP�@����!�v}��K������2W���>B��+dH����	��3l,�(�]��k�?q���82�GO���S9�*�Pw�����Ul�O��4�)�	��$zP�J��P���wP:l���9�W!�ek�a�=��`� U,�u����U����
U�R�[�*?������"]����L
�����=�����D��oel52�*���m
�RD)�j�d�4T�:����]*4UI���/��]
]����wt���N2k`�48F?���U����5�OeZ��J�/��s@�bA�������t0�v�PK���
�<PK#�tJ5patches/0004-Partition-wise-join-implementation.patchUXz��Xz��X��}iw���g�W��@WX�&�DY�h�NN����(�5�pw�4������]
�J��g"�]U��[��^��l!����dz�fgq<���i<����Q��gS�����B�����\���������~�z`��������<ZE��>R�&�������*O
9���l���2*�w�U�t��(h��`��i������O�������e\~'~�xy�����
?��Q^&e�-���{�,E�X�rEX�k���b�>�r�e"�;)��H��T�T�����SQF�T����SYFIZ�l&�����o��/k)�A��r*l�*�E��}��|��
t-�����H��.��
l������B�sg��M�N���0�n�k���y�r��$ME������
��>��	zI�������-3W���K,$PQZdb���8F�-~_��G.a�q��
��l�_� ���\TYf����`az�|�LD�����?b��������~��iv�ph��1��_��/�+z9�B��l;��C����t
�7�"@_[�����Ezg�Y2������o�?��NK`�I���Qu�km�����L��(JS�Y�b�hp2�F�8�
���4���).	�b��Igs?�(����bPi��fx�\�?����������p]&)�f�(a'<l�07I�X�d�T��
���������v�su�Jq�H��h��E�H=Y��z*���{fT��c���D���!���D��Y�����&��
59�������pC���\�����J��6GLu����/nyW��u��z��.����-�r���HR	V�hw�N�T��?�Z�d6�����Q#��4���T�*&�����y�M���d&����	�
p[�V�`��O���lp�=�9��v��o�{X@\v��D�FE�t�c��p��?[���~��{&��v-���H�K	���r���k~_[��~���>�C�D��>��;b_���YMd�t���M�1VS�g
��f�x��,aJ������h_����I��2����@��"����8�E���g��f��e���#
�tvT�
��i�� �?�
��\L���;�Xu���R��
�w602������3S9��i)���A��,��`�=;�#o,.ft�����	3
�K�Ut�3��AP��������w�7^���������,�Nz=)et\9����X���~�=�IG��W$9_`�Vp'���e^	H������\/�~�m��AU�Qf+fye�0�D�� ��+`
�[��9�Q�N�2ww7I|#��wY�E��j
%0hx=w$md1��R AL�c<>2�?@�K�/�+����d]��d]���4�f��f
�p��V' 3-����:��hP�e ,�p�> h��g�+�D�{���U'��Lr�@��Y[����g����e�2��)�� ����J$�y����B'��R�[�Jcd�"�\���J@*
������a�P�YJ�^
�X%.E�c��n�U��a;U��p�0<l��(JxB��]Gl������dw�Y���M���{��}�����4�8F m�tQx]��(���:w�= w�4oW�@�.x�$.��ClP�U2��7�-6
 ��1��e�s�c�@!����+�^Or}�z4j�
=~��]+�-p}cX$�t�����JI,*.��uju�r�fO���[t'S��[P��(�(�7�z���/�V�lJ:�Qh(@��'b���3���2*#(��q	2yO|�vV2�]�d�i�:�����:�)D{J�;��2R����X�i.-+�m�-w&���>�[��1
��t� .�{���~�7�����n\�2*������[����T;�=�I�V���,�����Fy�?X��@q��Y�#k���XpD(a����;RU#4��z�A�*:9;�x���0(UH�8�j]dY�'(����������'�c��4���c<$���~C
8Hq���L�����.Z%X���(�G�����5��2.����+�t��������7�{�i�����?��F��.��Q��2c�,���ZN���c�����E���,+����v[bo��@��MZ4�v�����:O��OWoa���a���=����	H`���q�poO<�����W�
e�����"��+Pi'x+�F�@��Q�|�(�WO�P�>�yI��3o�>���g!�p���>B����D�����#1 w*���+����N����|����[R������>8&�x�[u� ��&����	���� u1�P�2f�Bh�C5?��O�?�e��p��f�=�=%K���{b)n8�y|s�oXX���P�1�9V��0ta�
V
�p�l�XCC�$�f��%
tR2�	�R�I������w���6��������6��8�x����D��p�l)�B�a�(�q��o���=?A!������������0,�N�.�2�^Ld����2C:�@���L	4�*;�@R������h�����"�GL
4�Q����$@v��5��]	��5��6�x�"����%������*T��rH����3�ctJ>]_�3p��������{<��Q��M��>�O�����wd�����N��t���&���g�K�	0?��!�(��	1KI�W�c�
~�"oKB���4]�o��e���@��E]d)�Fv�Fe����1��D������W�;nh�6����3�OU��
8���(��;�*�s�~"�?��G�����D}�J&�R�UDjZ��R�e=G�63�>�[\3T�`
z_�^+�_T-~��0��%���[��)�g�ey��NU�~Ps��1T)��
���x`;�c�@��ilU�9���-kt@X�Q��,����e�������=���_�7�e��Jv�����\���3�G���������i����n8_����]4��h�����6cIV�1<uj���|��� �PO�N�$����x�l�'�:��Z��i��3Q�b�@���:Y������<[������spC 6�{B���������9�B�+*,�}�������a��)
���P�XTHG��3� �Gfe!>g�:G�b*���g�u!�����Z���2qv���U"M��
��H���	��8�x.�h��fH`8%�(����D���:�S"��u���;�x%������p���7��������a�H�H�TvYX>>;�^���?�N	���F���l\Hd�Ch�d�<\
�p]�5C��%�QJw�����^�R�������>����q��C�%���P.0���3n�����D[P�+��r��FZD�;�7
&������G����"��X�e�B;G�P�l�eLgy�pa�i-`���R��i��n����D������#G4T]vQ��R!%�8Z��lh*�#�*��6��J����P���k<�Tc���<l_x��[Y#����!5-���y�C��`TW[�U>���4�
���g4$��4�������������O�Aw��9M�����e�G3E���^�����T�(����I��@�+���,���A2���@/=�W�m��ZX����m �n�,�m��&���O�
����9���u8��p��C���w1`��1���s��'b��C.�u��x-#��A^	Pk��`5�����u.Q���'UU����)�	U�z^�)�����_����t���D@�N���:�G������4-�H�7%;�H:�$�y'��>_S���P������)34 �-�t��x���v�e:VB<Na����]����[�������,��������T@�����3��x���
J���<���7�Q�JL#����/���������M?���b���<�w��}V�>����������m�@.�>iR�
]���(su�RWD2�v`��a
�RDs44�H:�/&����?]��N���+GK���#�Z�eL�S�aH�b�Q���R>[�-6����=�mG������4-i�S���0���8:,��8��
R��"�mPB��"7�/��G��%^/p�dd��
�s��>�^��2C����7�
��T�`��VS9Y��%a��+�HC2c��S�2�h{����X�W�����+L���(��6���2�����l�"���._ul��:kz������mE�aw45-�e�H�v���vL��,��]+�1��x����_( �m������>]�lYt��7��l=mYG������~|1�w�,:��Z��$�������	�e���r[�=r\OA���Q)F_|��p9�7��Qc�~]�����PsH-IU�Nix�����y����e��=��y4���N�����;��i�9ui�d��b��3�������y��_�<"����Pq�+m�pL�;i�fW�]<�����
�=�'�o�$�?�As���
�G�&t{~^�X�8*�UY��Y�`8Z1�*5��G";`����Ua
�<�(�q%Q�3�[���p$�@�^�2dJ�#�t��9�\
>��e��8�.�\$S6���
Y�'��2{�MSf�����]��I���6�8hc���p]�o�!==3��y*�?��J�=q���XU$������;@�����������*�	���#L�<J{B����,X�����,�
��/��`��c5���Kd'���*�^6B�!s��'N-Z�+���Q-=dm���
����o���7�7N���h�%�v[���>7v�(�+�`��l��Rs�
���h����Gr��O�U
@d����5@�7<��!fk�z�l'k�����_,$�)ts�Dj\@��������W?�}���+��+������6�a{����ys�����������^�\O]������\�y2�������lU���Iw���)��E�h}���cp�9����Su�2����e"R\����L�fYc���B��?�5
��<�*�K.�^
�w
G��Pt�O��Q`b,PuB�|����~�SN��!/�Y�R�J�i��]4-O�`�"`<�����U�{�����@v�7N���D�-�Q�>�Y�����
���1� ���������V�'I}��Q����j`�����Z�m�W��LZ�r��
�����>$T�6��^��k����35n[�'`X��<P�N�)c��EN�N���W�Y<�� ���Lg��x�r��I���t��1`�5�-�3���H�����/�j�{��B�V�4�l�;]�l�
q�M;�}@��(;��G�{�*6E���w�	F(]���������m�N|��j�6�"K�ZY
<+�o��	�q����H��T22.K���#+��^����%����FwP�{������'K��K������4��"<�@5�%��w��Z;@�*e�@mQhk�/�jT�����D�*�KR�_�	��n�L������v'6�'�QO�����jk�q�����	�6�R5�O<��������&:2E3�,�p����]=
��7�iC]�7�(6-�Hz�j�!sH�JC�:b����kVbm����}��f!1�H�n�k#��28
c{��k�r����!8OD7�3aSt�K�MP��0>`V��/]����)���#��EZ�?�
�Fw�=��N6����7�y&�J����cE�i�9"�'k��*}�os��\
��A=o���R��O�!����Kv��<�$b7q�t�hx:��%�d��r��2�3�,����*VF�*��n�����t��M�N��R�� �AKA����P�����q������=W�AS(D0(K�����G8�@VG9�(=#>�* et��T����`@�+^����Ch+�f�(Vv�:^���b�������L�Bh#j3
�T��<h�������h������[�#���Q�t3>B����
+8Z�jKm�
��I1���d�:��}�����G�;����J����!�}BT]g�a��s51�,w�ECSd���g�t�XE1�R��{�;kL�
i��!���Li)����O���>>�I�`�m4�e3��0�����}q.d���������=f���?*Ub����1�b8w%f���iM�rb����_*lD;�$:��w����0�M.����	��n(�V�H�]���S��|Q
�B�����`�F���}���|��D?aTD��i��]�5#9F�����AO�k������x�i�64���tR�B2�Q>�/Hlb�G#���N�yTh�N����<�c���qb(��N@�(�=�
j=u`f�Uv��#���2��b���+,�v��\��,P}�����@|A��������X��sk�G[(Q��]\���Pc^A�U^���:>�p��v1��&�f��e����!Cn|G�^��[���+��\�����2���x��s��SQ�S�Ug�'��6N�>��\���b�Ze�ri>l��(P�P�U�@jN���|"�� ����X�W]V����%R�Y��;�:�#HMJ�E�R�����.��N�M��%��Y&���>KW�Mq��O�����r���c'�M*3��V_�GYK�\8�!��r�����B!��h�����#^g��Xj���i��U�Z�c	�P]������R�Y9�L,CU}(/y�n���i�?*P�U:��VL
>�P��=[�J�>�O�aP�2d�����������b����<8��T��.��������|p�u��������*i2FR������wI�;���4|a��(�L
�`��.;g��n�6�t��J���:J���o������d ��Y�B���d��C!��l5�	&}
��w��w�J�D1C	a	���rCu[���SR-��88'�[5w>���b=FQ�
�c��y���6#���6�;����09nCM��Y��u��b���P�!z!��g1���D��1�8��1�i4O���Sa����WF+m�6q3)��E�,u�w�e������ E����W���*�)E")T�PO{v����L�
J3������r��uh��<8��������U��B�;��������b?�	p�j�8P �%JK^U�U�
�$/$f���v�xX��~W�Z�?���D'���������-��*��~���u�~��hcC�������ZH�W���|)�'G@K��RU��9�+��z��y��~�t����&j\���`!�b��$��6e����</����z���j
A��+�>������%A����;f�f~��aGH��;����),����V��qG(,2C��L@�	�����w)���k"/�gK����n�QQ���v�v��D�
+�5����2���)}0xn�O�j�l=~r�4�H�$�{���sE|Q���i19R���sn�����;�����Q�RQ��������'����lx��P�
j��^�29�
���#�~���rH�2&��$��y�U,�3��+d����*A����Vx�&�������;:����D�Am��&��]R��4nk�����D�]�I!�b[�&=Q�����\������K�d���y���0v3��(<�����X=6�op�V��T"������Pm���B���^��L�rns���Q�� %��mWs���iS���v2b��.'>�� �6���,�p����0�Ng��M6�u�[�b��N��������i�PoG�������������Cb��H�J*U�S
v���W��9EP�����
���P��j.���S�T������lm���\�E]Wo�D��9bE;T�����Cct�/��0u+�\�^����p�>`�l�Ixu�N���}�
��a�#�(w|*�]�I�Mt*N'?k/�1\D�����]���b,��EO����Kk�w��R0�����\wg����iK	p�����@)��
��y�UwV�h�N.�q~S�5G�p��p�*a�uS	�	���ZC
[l�.c���O�>��.U��(��tx~�z����dr��V���I����#�6��*io�&`!\
X�y��F4[k��C*�Hk��6�&���X���G	�H��"��Jw�{�M���VUp6���/�{=�?���N��`����Nx@�O��j���������[�+�n9�����iw�7��v�["$��#���1�WB�BA�"$����$�[�����+�c��P���K�u���cU�b_+�F�����n_OY���g�����@�}7�Ek��<����t�2���I�����/����/�� {S���wm$I�����R�,j�����4��%>��#<*������o=�����f���M_'}s�C?r���{9l1�d^�G
P��E��C��qX���er���C�)��5���N�d�����m1�t�.��1�]:|f>$�:�KY1W7�M�-j���+�6�!��h}�O
jJo7q���9����������*X�2��}�3\
?�j5�&��]'����v���F������9�G�ba�+�&U�>!��i�
u��s���>��g������u_��]����a���h�Tu��!X���$�#/D���)��?�%I�+>�'��W�^��?p^�JO6BZ���R�����'��J���������������|�����]�';�e�V��VU|�������l�N�X���<X)��3�v�I
��{J����IhJ1S�zC��}���������]�-5?=���p�h��]��'�����c_K�����e��r:&��1�*7��f�M5����=o����ge8=�Y(���TU�d=9>%~�x�[dG����,W��cV��O6[�����$5�i3������a���J[)9y��w'w���z5��\��L����G��O����J�e���a��@zg���l,n���a�vp�W�s��BD���o#J�G�,��wOr�����2���~;�k
\8��ruD�x�?�M�uU����tS(y!���|�s,q#!��66�����]��b`�G��P~��0�+>2���n�Gf�c�'?�:r��!8~����z�����8��<b�������.+t�PV�s)�\���E���ZhSb�@��� :w��c���L8v�w�:�sMO��~���9d
.?5FA�e���P^b�F������>i������q&7yM8Z�f�-�7���
����9;���P���p��q��
�3M�%t��=��k�.q�B����tU����S#g���I�������X���JAQ�I�v5`��j�ELo��c���:O��)���<=����J~�yf�J����3<I@4^���JA�%���Y���L������>����(\s$k)}�)lL	��3���?,�L�N�*i*�(�xCc��TO�C��6T,�R��XmM)
��[�����T��������>�9/8c\
��TB�F�zj&�k��I�U��K������}��1��w_j�.�QR1I��I�L�����J����x��;sY�F�Zm2iz�x�jE�z��,�=��(�l�- .����j�,���B��<���]�
x�� �S����*P��be��,�*MEsEW�We�-�]�����$���8��
C�!�PYU�
F���U�TK �9�J��9s��P/�8��II/}��^�o�7
�gb�]5|@D�_�q��E�C�s������7n�L�U��|F�]]�`d��Q����+��X���!^<4f�j��#x�dU�-u�7���N�����i�q�������0S���tuu�g��SS���y�L�r��������e���
T�z##K�w�&Yu��G�|�o�Z����0a����8�_�����G�r�g(��(�*QA������������i�B����Q��N���6���F���V2L-�����jQ��������<QPG
����[%�%�zq���4n���Y]�hN��zL���]T���"�}C�����(B�-�Nkc	���"���'�?���Vj����`,�������u�/F��,��I��������	���`�2�pw�N�N���c�=dom�s[�����a���|c�<!�\�����|�&�x6���,P�������a���*S0�Q�gg����\��'���+vS=��Y���qt�lr����}�����9���x�b3��(D�j~��rd���{t��{�r���G����d%��P~�rl���4(�u�C05P����!Q�p
��Z��������o�5Q>�&��E�����?�K��U�������@��<�'KX���fs���:�:�B?$��j�=u'%����K4������{XNCOR�~+/R��fv��v'��`E���j�����B5��������/�C��v�!�fF��7Ub�������{@�)��>��Vp�G���_B��1�~5�}�kXX�k,pQ��D��
�:o ;x��T8����aTCM���l���������^W������c���O�>|��������o�$����������fm8�T�:�#�q���]������z2�����GX�%���+t����3��g'D��t���zE��)W�,_�p��g������GcZ\��/��"�#�q�!���k��)��'H�{X�\~���j`=qE�&�u�k��F��Z�3��1��@�1�n��.
I=A7�������e_��S��%�i/�XGA��/���y4R��������']�����U_O�']��%�P���4�U/9_i���g�h|��p>�����l*.gp|Fa|�S��q�v��h�~ps8�^�s��$������.��^4�]
UZ�{�+$V�����|7��`���Zk��@_��%�YW~iV����Q	=�Pq�_u���uZ��$��f����l�l8��7m����;:���{^BW%X7�/G��&�������b?�-��[n-���QO+K		�<�F�s
#��K���^'���h�������h<���,���L����]�&uU��!�����j���[����Lz]2����l��R%��l���2��������h��o�G�!r5�����}��t+���G����7�����^�JB�{kBsC�8�1y��C@�d�C�R?��K���?��{���(���L�q�����
&}&������^�m:+R����J��B���k����ia�����#BM9$x��QMl�D<�����R�y���S�
���wu0���j���xE�s<z��p�P������
�����a�Q�]��TCE�Z~x/.{�������m�5��z���)7�=M�H������p�(o)A�
����N %�/T1��s2K\����y�3v�4�T,hW(
p��"�?�]�~^�i�I�����@�����Sy(n}�q8����������..C��0@m;Kl�^����>/z1e���I��\��RT4�P�}����	���d�Y��&���r���O��F0H�B�|~�q��*Y���x�*���6G1��F�Z�hX�G�N�r�9[�0��ER�������K:���F�~��CVPo.#�QY������,yet�:��M�Y#����zCmo����5&kM%cM%[��O���W��&w��ZUTm�N�:��e�-�
/�d5R}�T�&�/:����*��bI��q-��Z�����dx�z����q<j����O����:�H?<[���`�f�|Li*_���\����x�~�@1�V�
OCW�Qs������faa�+>��b����O�����~�����������a��������qsY���x�e�����o���puBy {�6����o�~���� �V
�s.�6����������:�G����v5+�8��/N�����txrZG�F%� J�%��`�(�?il����������'d��|���@�H�b�6�'���Badr�������z�,H&���h����1�w����-�tM*��p>������(��U�[��[~�s�f�z���:�o 7�ICl��H��h��'��:�t�)�.XL�-�X�
�t*)��~-���y6�������s�����Y����b
�U��F%���j%�}W��yM�v� X$���y��dEy������n���+tnX#�y>������s�Y"1n��
+|}�BFx���&u��r�Q�!��tN��]j��.g�l�*���px��v��$t��#���w���	D�G�i�����������%���;#!�`��
�9K�UZ���M�n�'�cC����l�7����@A��~���n8����-�����}
:��L������Y=�
>���t5$�j�(���!F3�O����w���+�j�v���8�`��nnn��
�W�@������i��W���2E��`������hz����~������������*n��u3��L�D��*�J����b�Q9\
T�_r��*�	5���n�Q`�x��^�:�l��,q-Vc`����>.��=.\_/�9�qImr;=�
��t����T(�l�>���^�����?F�&V�?�Ss|,���pj�qS�{|5��3U��w�	j�v�0��A�����t>�	�C|��~������k$.;�d�!�����<��{���`r���bc�*
O/(L��{�`!�E����o|��_d^7�����!c�>0h�Jh2\��E!� �2I�E����a�-rDe@4��J���ilDH�UX���d�W'\�o���_����-�S9Y�1 ������7fI��=:����U���)��M_7=�z��B���i���<[���m#|a�9�n�E>�9R����`wjc��O��/zh.A(=�-�����qw�!4G��:���<;��z��|Ww�����H��|�m�q�+��W7�`���,d��88�s
Y�^9_�\��������n�b��$O2$���$Qj��T�i�^��z���K /;_6�H�	��hDF�P�{��`4���;���h\E]4�t��d�cC�I�%e���r<��M�1:'-��4LS�p6"���W�^��jl��9u�Kn���{�'��c����
(\�=���LG�o���/nyW���*D�^Q�������h<��g�u���E�*#��������r\*d*I�^����:S�$�}Q�!g q��c�F@�w��R�Q���x���%�o��fY��cT�J{Gp"��L	��|�F���2B��jhd���+������GOZ�B�G����0\����P�g�
�_#[�2G���������j)e� ���.ez(�2-Z�`�:P���O�ljW�_Lo�2|��Y[���Vm��:u��
���I��]�nU'�UN�����j���T�n^�����5��Vj����mj������k�f���! >&�j����h�V�-�2�E�B?����l	�A�{!:_���N!���OBYd�<��CT%��zm0�G%�.Sf�eIji�U�Q���ZdB�����w�j�_PKK@l��5��PK#�tJGpatches/0005-Adjust-join-related-to-code-to-accept-child-relation.patchUXz��Xz��X��Zio�H�l���/�eK�aY��M0��A��� ��b!Pd�bL�Q<3���Uu��N+3�DvW��� �����������������;P�I_���v���3�|��#�Q	u�������t���9��lZ�q6��N���O���'��'�*M� S��v��+�_N����PM�51���������;�'���|Qn~N��tq{��:��n��t�})����AD�
A��<&7�:�����iz�~G�mYW��,�������U=�I}Kp��^�:Q�My6�N���X����	����q��|S���:���:9����s��Z��Pr��5�^>�g��S��,h�����
�,�(R�T�4�`3O���f����l�R�}C�S(���n|��E�6������K�YFi�����[9+$'�J]A�.d�J��b%��}P��_�|J���|�5���`,����B���NCNT>W*Z3������#�G-DnX����n�.j������1��2�HS���b����=�>�|{�YK�[�A
��56y���
r�;��p�=��1'�U��F�<�k���0t&!X��b|�����T�n��
e>{�U�u	�U��*��xJ�tx"��,�l��PI����-�&�e����X�������NF*fA�Qm�&��X6�.�b�cm=�f�z7H��z�B��i'�@s��*�1S�Uc\�zb�� �i�������
��q���E����z0���_X�NUu���7W�2�6�Y��w7��P^�
s�! Y���h��m���1Mq�Z��<C�+�:2�*|(q�,�n�y�,Y�zf�7$E��8(����w�b�)�A�
C�\�p�R�q��2��T����"t��)��IEx4y���'PV�a�~�8�$�<��
D����n���|��BZZq{#!��+�Y�b�[$�x�Lp�mT!�v<lPv$�����et�OY�p}+j6���i ��P�f�	Ch#������}f�!Wy��	t`]�A��s��3��d��yN��b�as2h��@gw���Ff����{RY���6q�����-6�����Z�u��xD&6;	0`���e���������,Z�g��5o��#^�p0w/�#��E.a���y���+�7�;�]>"��)�����K���$Q�7f�6���^�o��s����@�HM���X`�d�i�]��pY'(��c�#.���z���vA������>���c7�\t�`!�t�w(���F~�1����k�*�]t�(�Q�RI9���s[V�r�^�
�*�@J�4�B�y\mpDesIm�%���>q�W�����(���)1�	\��E�0���"2B�.k���,������@��T�J	hsMWw)���\��v�,��;��6:�:��g�,��:Z��
"���D*b�>��c�3Pj6eA��_�1hg�����n��������8:�n�&R&�\}s\� n��.��FL���&�J�������
Y���0p��UE�r����
+������
����]
�Y�A.@��]0��	�Iq�������+QT]D�@4�2�����Q�eVVL8{���hX�V��,u������[�����F��gf�D���w�x���������=\0�z�3�Hk�J�a�v�����ZB3$L�N@{N�T���&�{�\t���O:h&Ps��V�B����j�!�9��T1�o���o�N�N���mu�*W�����}���'Z�������F��f�����p	�/��?��u�,�WT������G��j����
%K��rd�X�zv���eI(�4Tt���@I���&�QO�A_f!����m<�f�`����p����%3��sR\E�fe��w��{F���#Or��������i]C�Aj$��d?��Jq�5/a�^����I�|!�m��K(�}{2cY4�tUu���Z�+}`�rC�_M�@ �\.�|.R��Xi���C���!3Q�S��fh�����M�����B�!���������VdF;L�7}0��O5S��W������n�j�A~�����T�f�K���8�g�x2��ul7Z�qUwU�5��@Dt�!Ng�-47�h��+�<�8�
�����!���D����
n�M��'F$#�X�8��<�*��@������B�
����^37�f�/B��6���K:|��5x��#��q�����iv(� ��DQ���A�.�5��1����:�����vB��XC�����[���z3E����`�� 	;�W&U������0��M���P�)��a�4I����]q��e��EoP�rV�v;�f�O��n��������?�
���2�Z����!�{R6�`1��� ��3ARv,����4��c�=����ZOa������AX�� �UAX{o7����R���x^
��1����W�[^�w�}���#�����v���V��[o�`�_y�c��W�y�%�	vw����J�����JS3��Tw��v�qT�u����h����V3���������L�-G��rtDt�c������j��ny�b�Tz���#������m�<3�Cu���#�������`�:��������-��9[��G�&-�%��dj�����yx��afG��*�'~(�N���2�y����q���3�7>GMi��`�#�de��������2������W����Eh����4���x�~|�����r���fs����]7��&��?�o��y|������c��gL����VZcAQ6�!N�Qs@�'�����}ga8�=�����/V����� a0;0G2������_D^��,���}����}rw���$������c����H��'H�������=5�����$�g��N�n��Ae��D���N��d�T|X�_$f�_��?9nu�hm�a�KZ(ou*/e�(Krmw7C�e���me�E
Ov/m����Q�Gt�\�2w\b�����V�,�h5�NCd[��]#LCV�0�9�k�����4�f��j���Y�?,�.�2��e���fYZ�n��?h����rz:8������c�������Z/���FF��ww�����-�#^�����}7C7q���N��h����_����cmx�����?&�l���.��G+e�n��B��������H8C�
3����i0<��9>���������Uo�#��x���R7�1�S�i��0��J�Z?K�)sfK�(:>?�y	���0��`��F�N�9�������Gt��^��s�+@�����Aw�H�R`�@���2
�m���L���2y(����Y[��X�z�H�&�Gx��,����'��\{���Z?Xx[�?Xj�����%��Q1+��^9�u]:�0v�\!�{=]!�����K�����zI7�����d�T�����I�T�_�2���H�rI���y�ztL���.�N���k�j�ln��7>��<��t����9��m�7<s{����F"����eR��{������^����:�+�G��-�1�D�8��Z�����y���,�.}���w�(C�32���W�J��Q/���r�������|I����t;~}qs���&�w5����m_����������k|�#��
T��:���pj������>���PK��t>-PK#�tJ+patches/0006-Parameterized-path-fixes.patchUXz��Xz��X��ks�H���+��*�c�q'!�����d+�G]�]�F�8�%�FJ�>��u��H#YJ{.
;��~O�Fo�t����{�����H?������h&���?�|o������K�I���1��N������7�����e��r�~�k���s�����_�$�:���i��^8?�\�"V��w<�/��7;=�����xvx0s>�������^�e�������<�+(��D����-��BN��`a��4gA&�[��Z�4cK��}I��6Hv�	���EI"2�%��m�v�
��<`<	�m����$��l�<E�"�V��B��@�<EDN��'�T�H��1�O��"�'���dC��������;��y__��M�����kG�R��QE�0�,�����|F�hh4���A��V|��B	o�
���Djn�`�0Z(�(>���G���tj�CZR�P-.�H,X��
����!�_d\v��`O��9K�1���t����������Rc�q��m:����Po��.��^<�`?�?�%,��H*�$��X\s0��~�1p��,Sv�r)�������%@Fd���F$����� �Z�>�,�R�d��.L}7�C^�LQ�,�J�� �O$���
���4�R��E����V�g<{�LH��lQ��@�* ���5k�![�C9:��)�NW�
v���h�<RY!@�V�&�W���3�L&�`����H��t�G+@��"�.�d��`����|�wy��$
���?;�vv}�(	�"
N���9a}v!5x0yr-@u��{�y)2
'���e�,�POF��a�&���r���~?8'JB����`6�����x����/fG�9��/E������l����1��w�^�t��{��7B=���t�~s-r�y���yz��^�z��y8b;�`	��Z�#�{K������Xc���/��L��E��� �*�c�q��;���@��cr���	^o�8�wnD���(���-������l�
��`4yA<����c����Bj��Z��������"��wi�������4O����(��Q?�&����7
]��f��Pf����������CY��r��� �6�{T���4��B�9J���i���F�E�~��5���2���(H�\:{s������
���c@�h�M�w@��T6d��"��7���g��
���4��k����zj��T��	��D����&
���L,������o<6��q`Q���T�������b��N<�:���Wb���oY�����_4��b; @e�[���
������Q�|N��M�C���,�!4����������R&+j��*T��&v�R��B�h���xm���e\�5��h+���oS	���"n������wm2���CU�Z���+JPef)�s�K��G�h���\*�2���A�����w��p����~���|���������*�b�c�9�����Lp*��|���P�f������<��]g������A�e}yh����0�8���W�e���Zi��d��c�]�A��c���#;'�a\&�BUk��L�,!��$���qz�A��<,��S�1$�Ya����y��/���B,����zy�6������M+���8�!��e�����=�>�R���(��Kl_R��4���C}T�:T��`a7�m�
��M�<P�|z�|�q�^S&�)�%/�^I����iI��V����H,jL)I��V��}�(�
Y�9F��t�����������
�E4h6�8�a�0E%l$�nz��-5�L��P���D���jSY��"W����z�����0��"�">I)�8��*���������Y���d������)�
���N�cm%��e��T���V��e��D�(�l��~�L��JC�����B{t
�x���]���t�:��%e<�D�\
��h��YLAw�c=(�)	L��� jP��LK�����h��t�]�,���TA�N����Nm�OO=���:6Y��IK�R��0Ge?_�D���j�>��GfOr{@�f��`�?v� ��GV�V�e"/����O�C��M*��a�"�&�{�HG��H��Bs(���7��L��.���tzp������L�����B�2����{���g8Xh���t��0�!������`z������[��L�#�\|U4�m'�o��p4N:b"lx�EPL��lw
�f�=�L�iWX���EN����s������9���@z&����)�K2@]���b�lf�2��$!��"��/�"��!+�iR�5g���Xa�������L�-�0
� �7����7���:�(
n����8c�h����������l����R�"&q���1�F�(�����Z� p���#	�������7�
v[��L������h=
�f�?D�BgZ�D���P�����Qib3~Re���GP�:;Fa&@h��	�<h���_+DII�����=%�@
[�q	�(�S�4�C���@�&�������>����4��\.[)��Q�4���������p�'���
���%$���
h/�%p7�s�{=i����W������{���{�����!�W|�V�uO&�z1����L�t]������MB��eR��B9�$,C���5�r��&��]�G(��S]��L	�Hd�^�Ynr""�wj
�ZU1�����?q�;V����qX��H��v��E��}�6�;�)�,T�T������������\��gE�3�v��gc���.��Q��$\E�*�S���7���B��v+��k;�P\�������u������:��D�+Y@�4���pPe�C�H��_�1�f�Q=g>�+�/�����$���@�#�uz6�R\S���k�P�5"����t�Lu_�N����	��p���[9�R�f�LhkV��*u:5TCi��5J�&,Tu8��)�mC1/�@�!�)d��h�	�{<�3�W���9�=5Jg�]Tc��U���s��������-K�0.3���Pw����3���sV�sV`�<�<���h����c���,:�($���
=�|=r����[���+����h)2D>n�����W��3�����6��ZA����x��a������|}�g�~y��4���a^��f+f�i ���]&�"�P���);�q�������[u�����"�Q�[�n���^%q�Xm�"[�;5
�kMU[|1�F�hd�>
��R%9�y%����j��i��������������Q���*;��IY>;����
��U9���_��qYEQ8�g����J�����y�}��8G��������t�Jb�4���_n� ~aV`u����.x�������m�W����$z)y�o���=��z<��F�����4����9�<���^�8��A��P�	�q-\���f�6���fj�������xe2F�I�^d�r4����g�+���j�O�NdTl=����	V��
���Zp��,{������A+/��''�zo�����Kl�/�[s�����F>"��nP�����L6��5����<��,L�ov��E7E�,n�`���� �����qL���~x2����Iueq/�9?�{k��w��W��.:W���:1>����G���2�H�������n�M3�aZ���t����2{��7���g��^���Eh����`[
��2l��M��'�C��PK���;��1PK#�tJ,patches/0007-Partition-wise-join-tests.patchUXz��Xz��X��=ks�6��W�U[W#�e[$%?&��8��x��s���{WW*J�lnh�CR�xk~����	��,{��1Eh�
���h���4;�
����xnNG����a�f��pzd���x:�&�e�����C4�g��9�3�=:	���"�G���v������	}��������do�x�������v�����	F@����{k�v#k��YN����{�������3����_m?tBg��>��_�C!�`����w4w\�f o����8	�/|�#@>a����:�.��]<C�1p<��������_�r��%��=��t.i�����������S��>��q��?I��l?�6�e�-�!�����C�#�x�����{<[������D;P=�;6�+�g ������9y�d���j:d��������O�h�.�i������Bf���������	���b��1�ZT7���9�9���sBd�k�m�����'���jvo�� �]{{��|p<�)�����-]�C�[���hw�'���3�����C��G��\����_]��vu~�^��)�g:Q�#2}�>��;��3�������=�<r�m����3'�f{����[�Y�q��}�{������j����H����|�x��OOnO�������
��	��}4��)�l��;�^�	��?����O�]��
c�hd_�q�gW���'�������_Pw�C�W�k�(�
2�r8S��3�p�/oN�o��%y�r��^��p��>�.�)]����/���8�;�a�t�1;p������t�r�������P[S���&�S�&�7Jo�"8y:��o6��U����
A3��z`g���^��_��W7�7��g=��X���g���)�k�M��)/�1Hh�yM��������eAXjtr�#�D��h���<���� �T�F������"R�K�+m��E��Y�!+��@�������W��!t����Y�B���D0Ac����}�a����.k��WHH���O�fj{�HE1��&�����AbcOhH2���q��A�����Qj�j6tlDM����,2.V�VsMV��v�A]�6�%�Z��Z
���O���nm3�o��m�<D�7���k��$���7����~K��%�[�@�y2~Kb�)^����y��\����.N�n��L�� ��b7||�Q%������+����FL�f}�f�������o0�_(<:�����o�c����bz�'1E��q���-1��'���c8i{��G<�H��5La���9�_�������C>��Q��_s���Q�sCaNH?U��F�e�Q#�5�W�R����x�#�@���N)t2�CK���1�j����e��1�3D�Hf1u�tM����� ��<C���p��9���&�Bn�/�c��x���=����u[�6�Q��d�iCf�����o���|f�/K�^D|a<�@#�im�WI�8����B�	t%Y�f�t�JYY����d����,Ik����B%���GU�i=V=�:z�@���r�t���d�aC�VMjOM�~�������0��`8@ONx����KW5��!���.e��a�x(XN>-���|��|=����O>F�=0�[��A
��WA�O�S�������`&�_�����g�'�����^h����T*�U_�|
�:2v�����Rl�!�G=�V��!$6A��J!��@[~M��a-A��Y�'/�dE�T���7���L��9��1B(O��Y\���878��y�f+����]K��ZX�P���fZ����/��ClB��
�������M(����_-"Wa�Gd�(mB�,���jY^�|]����|v{�#ov&�u�y�J���
�=S<�W����������KSeM<��
0�`�fY*�l�S��FE��hR����x*�`n����T�W�Q"�c1Uj�(K�+�R���8U�|A�1n�x��^vO@��t�]+p�;�c�f���y�-�{���c���/3�^�h��~��|=8�#T��E���#y�Y���bV�
���jq���.����[���	��q;f������;cF[���]� $���b��.��%�A�h��tViu��������u���[�'�m7�����>��S�����������U���m�C����kP����?u�k��tI�8���bv>*�'Y��������K�cQ���/��yC\&��P���hT�5�Hj��>n�!����+��XYe�5`���)^|���g��Gt��5��E���?8�J.����H�!?[4J9 �.K�
�lv�+�*
D���X�{���\c�����]��Xk�6Xm�j�*s=t����x�1��y��e�����,�����`���P���B�d�[�Pe��L�)	�E�����}����|<�>���|4��'UN/NnO�O.:;���P�)�On����V�`��A�F�>M��{Y�fv��Wa�n�|?PD���@����u�R�HEj�)����j�)iojegi��	;{	�l.X�����	|�)�t�V>�%�������%�Bs���?6_J���Mx�pEC�$hUl�jLBSuo��r����`����Zd4c2��J
��U���d�Tw���(���u��6�74����/����E����"�
C���/�'�2�0�S���s�r�N�����
��-B�z6�t����T�y&��2��"4��:z.YNXK"���
���q�`���L��_��7iE�{�����W5wT��������$3_��D4���������Z��j�:���Y�����%1�����B�j�"p�� DY�����*�	d+Vm�*
�
�����uc���t�A�������b0�+)F;�+#�B��b�2X@�XVE�Q�����cFH����������ic����iB����U�9m�h��6M��I�b�P�3^x���67��Y����p���i��_j�7��RW�UL�C&V�Kt��}��>������`�x��g�q(��X���c#d\%-[����^��<2�s	��2���hi1#�PR��R'��M��%.T2	�r��~��V�W�C�J�0M�=Hv
�Xe�>0�>����8)�3�`1RG��
��>��%-A2�76��-�[Q�G�	��5�N�����/��	:,OV:�!y���v���D�Y�"^�RCt�S��
���k�,a�,a�,Qclh�L��%�cI+5D'.gK,��"]��B���X�I��DF����O��������+�����"Y�i#��T���(H_q��,��(xu��x�����SDD���������I��+������S�����x���������
�����p
�b+����)�5a��P��P��"�FH}��������'Y��of�YS��,���a���e����US[J�����A���������z��la8���s�L�M�j���	Q�j��\P�TV�r5�t��Z_�hi[AB��J�5MF\T�j����o;�e��"�V*N)J�5���1E��B�7�g
e��5�P��R�V�*�i
it}�����P�=�e���Ca/L��i{}��V����k�7��1�-�u�4�f�}�)����FZ�1k��������RD/��;bt���@���\�k/���<�������a�Sc��n4��*�J��*�
z�p������(z[Y\�rnU�����N{3ne(�<z��M��`���#�k�K������n�x���R
���	��)�uIq+�u�s��t#y���>^�n`���"�����uK`�J[Q��-��-)�K�[9�+���{ D=��uuK��x�	��CK�so���������1If���3���2������D��|���G���C�����Sk������P,���mm��fN�z3��*���#_�I��'��~����_�zs�lST���BY$���u�����J�����Y�f��4����kV��`����Y�1��"��������yPu�K�����k�;l�T��EW	:�.�R�8)�~Er�V�y+���5�+��Y���xS����rm����lQ��Jvf�m��wS�8����	w(�%-p+�p#���?��c�r�Q�GM�CA6i�;X)G���U	h�����XIoB���,tA�n`{3�c��2�Gk:�����[�=��0�'���hh��|����#�]�G��Q���O���hn����k�$��l:S�����$���8�df��D����3���l:K��O�c{b
3(��������e�s�*��|.K��
%��^�z��������L���y�X"A�t�������E��
��������W�u�|X����ky"AKD�RGS524�����5+!md]���_B\-&���JlJs�E�_�3�
��������*�X��EO�h�Uez������t����(���r���j�j��R�U[�����^3�b�Fz_����K�x��r�p���Uu�
('�8�i`�H_���ZCaM���h7	nX�2���\���^�+i[(GGc�e4������{b�/��&@�}��J�.�SF�ip3�h��V��SR���������T;�>E�POQ�E6����yi���j!��+��,�Nu�����V�q�ju��Z-j���3U������V��Mwdnw�nwow���8�8�O�p���[_����[U����sN�i�h-�B��6�+7Zkik�<�9���o���q��m�z�ZVJd��a9y?��*�-��W-`��]\i�^rX�b�����,�m����e�����E����E�j���	V_���������F�����Z���TZk�x��
��'u6���S�6+`���z� �y�z���Uy%��"�V@$����Q��|5��,��JxJEl�����j������V����QD�*���|<�>��8`m�~�Tg�7�O�}P�Jl�!����!�%�����'�tr*�R��MG��^���m]����T�~�.O�
����BDq)�@J�m?�e�iK�-%����X��`����y��������,*�
7����qe���A������9&�v���8��������%�k���~�9x����/�'b�D��D�D���q�dE�-����&��Rh\V�ni2@L��e2c"�I���$�Z&YQ.��E(+�I����$�Y�L�������][k+`�P�*-��
X��~;�f�����=�l����!`]�!6�����k��G�}�f;{���	��G$��3zX����b�-�OE��=��"
��@;���}���6E�mzo�=�������9�
���O.:��}6���|@�,H:��y���������O.~;��l��>
�{{���7��O�M5�	���OI'�j�AM�0����C=��9��E���C�;��>�.��]����/�����;�a�)%}���>]�q�19�������w�M�Fr@(1a���Ig��!����IBU&���tV�t&#���KT��I*�	�� ����tg�sg�������G���w,�Nj��z��R��l����6(n����^�Q7��n���\+���/�K�����
�B�Q��o��D���v2�LW�3��>p����L������KJJ��]�����M�����t@�����@z�����n��n	H��H��[�--��|�V��rO0iPaS���
LI��'�����"��`��@�6;	����x�^��x-]�g�O����d_�-���y�b��3�y��A�E 	e���G�CT���R�(����0cA`�aNZ�8���N�r���d��I�fdN�����H�Fr�����
"B�p�2���Ew3�����d	���at?��k�+H�,'�S���������
	fB3�����#�	��E0���u�E��:d�N�3g�J:��S�
}��K��w'����C���B��`�a�vQvR�^7%0���/_Z��,RhX6!>z���=5�����O�3�>s��|���^���bn�3����J�Y'J6(�C
\�:���Z��v�)�Lt�E!p��6�)�����_#�A,(+T_V��cU�+��F�"
JtP*�������pD��?���������'�~��I��!�f$7�b��X+��7@��5&�����!�A�W����-�V���V��}���Cz��-\����5)&��h�D�Z��n5��f�5���?����@�f�L�p����� Y��[�q�T8�������S���U.���=�5gs�7�]�p�k4<������^r9��
)l#�Q�t 1�
��#�92�fz��i�0�zK^#K
�Jo6�M���	�d(�=Lo6
�`���!����G��������5:��>Ho6
�a���$|r(�}��(l�aWp(��#)���Fa�@C�����X
�8�Q����{��1��62�z���
�`>1���0��m7�D'��c�������#�6U���_|����s���������O���$\�dd��^����2�/��\yx��y���}n�O�mqi8ja�������7�`q�$��������r_������"�,h������H�B��:ZIL��"�\%�X�����q��$&�J�����xR9��"�/i�.�����v���@o�E�5k:�13`O�K}$==G;`)�A��"���_��k��0�tA��"A�Jq4x��pC���z�8�X�c`_EOo��?�+�� /F�,�P��
����*�4�jL&�UN���-��F�G%kz�����#��K��I8T$a��
�b�p
$��;�a����A��+�U�+��Pd�h`��c8����������5��E�&�����g���n�P�����������K
���F���H��G_�u�����T�z+�v�1/l����p��>@
�95SJ�<Yr��&KKd]�,2Z������@������	�&>�<��^�����Z�6ap����G���W/�7Xj�T��}��RKP����h����0��Q��b�b�P�	VTydt������f���J��vYwB��9h���]g�L�!e��Y�?$VT�37��^��]�/;�t���q*3>kze���.�yv��u�l�xB��l]��v��xP4���p�r��g�v�8�t�x�[,���|w���/�����u����;3l�{�|���������V�G��n�m��z��Uu��w��WeC;(sV�q��hCN��������+����*��6:6��@��2�K!b�7��{@'�2�j���	]6r/n�U=��
�5d	3]���A=N)�T��q?�F��y�oV�
7b/1�mh>��),�/������L$F%oj�in�R�J��z�7u����vw�2J��t�������n����]��`z�gK��B�����>��&������h�������C8B������Zy?�]c`����4�{������9��`9	���H:��(�����t�{����M.�9����Nsv9��x�?;�)@�?�����LD����L�t����S.����-�h�����1 2��d1u�d`��L�3g�w�2(3;��h��O���dC��7����1� {��	���]LG�
Hs��	�;&�.f������8\��TT�<um��5(J�LN�C������������5���GJDO�m�+����3Z$����4���_;���w��D}k:����w4<��=��[y ���O����A�"����i��sW'�M(���	&�����a?~�v�0X�,�����)���);~Bs� �a1�1�8#�����G���n�?�{d��b.�"j�4�-��}��x�$��c:�W������PXCI6�sp,�)���G,9y&]sn/�������f��	���m��YCP� ����;e��lD��Ip1���fd1�����.KK�A�����8��
�JX����� D#V�oE���`���.�
� �<�8@�/m����&����BM��`Ru^�p�\oi<]],���P��^��:�,���g�x�x��r����6�e��S}�
��]��O��D��D%Q���L�1�Q�e�����r�5�S��i4���pT�����3��7n�=���1��c�+B����%(d����9yG�x��_m��|~-�%���b#������h����f�s���\����������������x�<�K���I�Dk�l��
w�eo=R��,PY��; ��d�FuY��V��V?�$�9f��IRV��>�<�=]g���k��\^��l�����*	}�OWO�X}P�K���Z��<�������M��������<��G4,��_q�����T3�%��Lp`�B	�
3\!Ba;�%`��O����	R����	S�Tm ���2�6�t n$�E\����TV���V�No�/���@���I_�c�/;<�����Ye:P���v��9�l���)��x�j�P��T��v��D�z��M����a�e�L��r�L�v�a��))3�1`�V�}�C����;���^���1�j0��/D���_�i��,"�l<��$�I	�7���vk�he���NQJ���	�m[ye�h����(� �m��?��X��^��E��3�R����>�1��������g�
nMB:�Zy:��VI��vj��GH'����77K�q����I�j����5r��|.S%���e��
^Q9�P���k��[��U4D���f�����l����H��^�d����O���oEx_�?���*���WI��~m��S����%��oO^�'��g����mO�}���n��}u��nO���O����{�x]�5m�1J����Wx����8�c{�����=��
��=�c���xE�y����x�� J�[
���b���������w�w�7�t�PKK~��#B�PK#�tJ<patches/0008-Use-IS_JOIN_REL-instead-of-RELOPT_JOINREL.patchUXz��Xz��X��XmS�H�l��N�v�KX��6\�������Um]�F���"K�h�.���{fd�6��}��+a$MO����3�:��� �Ao��F�o�����n����^���p��p�g����:T���j����I>)d�O��1	g�����=O%3�<�� �[�����+���!���v;�������Vw�e]��<��������O�������s��������������8�%g!d�������G�~��,�P�<_��A0��-�O��p��OA���e�s��{��	+��#&dLs�=���rG��Y���"/9�Md`����[�-s����N���8p����&�,n!E�!������r����q�MK�Y��& �e�1�X�J�e�����R���,��X�������������}��nv��]��X^}Ajyk-�E�������J��qZ���j�^c�>Dq�J&,����|.���~9��'\���Q�=�%��-��[,��?��x��;��\���d��N�������l���{��A��z�f������a&�xR�FF��,�2���	���)����2�$��I��'Mj����c����*�2J�|�i�����ch��uP�f\hVs�z����INz���}���$C�w�V-��7k����(.j�$�jhO��pr}j�l#��vqz��g��<i���R��j�H/f\S���~�vd�C�5K0�xy�)���=��@��;zLJ�7�I:J�f�
c.iY���?H<��@�y���#��=���*�%�5��v.�/-��18�k���������L��,A#3��/x��RT�����k��F�<`t8�����c��hi"���u��B���ao?!��u	[����9Pye��b����&�k>���
	������DS��]5��@�<������oB���4\�TW��-/��������[s*R:��!�E0��,'+��*��bt�2�tN��E�F���AZ[5]T}����/��C�C��I�����kotu:<�W�N���������s��Up����������'Y�`��&_�V�$X�SyN�X�8KF���*�O[�s�J�n��`4����=�`���������>&�������|z\��Ka��1hR������1'C3�i|�G&Kg���aiz��	m�^O�7�yT\����v/��x���������@����WT�����'���Y�Ci�>N2<��~����37	�����t;�[>��JZ������	�I�Wx*DC�*��s���w������,�����<���,R��uJ�&W�=�&!�����;�%�B������HL�g�)����Y�0�
9+$vI1Mu��}�������5�~A�e
�K���Xi��(���L����[��n��g��/V�lh2V(�5z�n�,���=��"�x����J���w
�	�$�yV��j�A9�X���t+����8��i�����`b<esa�f�Zi��1`�����?y���7����y_�>������#�����^�tE���y���������|��pj��*	�Gt�J�T��
]�+_Q\&u�h���f<V�7�jT��1� ��P`�M�J�W��
�&.D��������N�����:�,}^�������3,��\��R�e0���?�����v����V*_+")�LP�T��K����!V-
� ���
w
�<��y�[�	b�K��1%�h�GWbSOUDZ��6����A#�PP�!�����t��)�`ZV
�������+��w\8�#y������jJ��ZDO](��|�N�a@�L)X��h��i�*����ci/��"�����:�v��"}�PfRUf8ao�@�/��P�x���TE��[��7`����7_d���M���N���;s�}?�.���t��L���6�^��a<}@#B�>'��]i�Q�&ts����5�y���$���Lk�Q�?r�W�Rv��.s��l�l^g%P��R��B%�i��e�r��^������
u8������|=��gT�k]����If����!W\ZJ!EQQ����T�����BS,�~ ]:�3!E��\����e�	PKE$q�]pPK#�tJ:patches/0009-Multi-level-partitioned-table-expansion.patchUXz��Xz��X��Z[s"�~�_���,,wc�]q|������yH����03I,K����[��p5�P���.�Vw_��^�x�?b��7Z�~�����Y�y�>���y��;��>|�#x�	��P��?�����ks�r0R���,a
>3�^���O<R\$"���U�xx��d����P��5��4�]o���-(���������1��p���j�j��'|�*pB����0���A�^��KX$����_��Tb�)��b��ap��7�@��B��A�~�K�����tfD��g���%X�A���c1d4�x��-]�;��)�j�y/<��q��a��$dQ5�%����w ��g�,��5�h�fV���Z�8�����TU�����6��s�j�V�G
���L�l~�A�%x=s�GM4����,�bZm�y�M�S������y���f�m90D>����V������^C�v�j���^3��~��~��Y+��D��;�-��!�
����0)<����|�O"�U1��o�����2=�a~
����`����&�	�&���<�C����>��S�����zw���~A����xu{��u>���b��<U7��8RA4�'��W?�7|�DMbm�@	?�c�
�X���es���$���C�H�CIA�SY��1�@���^��%��(��
�A�cs��>J�q��$���N����0�^p	?����d4���5`p�$�u��e��0x��+x�����LX)Y��\AArKK�;��+� &�b�p�T��ee�1*�&���G�9��T�Xf�D�y�J��!�/�m�
b	�]Y�"cWL5����?�?�a�eWz>���Pz'�����s�f(nE����-vM�Q��:�w��(`Q�F
XL�9"���*�n���g��':`:�\bs��yq��$\�a�~��N&����.����"6��H(��W��'���G�^������P����8ITL`0-�����]��2<R���+��������7�+�@i�
�����������2�}�����p��U��1����5Y7&d���fh�iw��+������(.k/���QxI?�Z��%;h����\��"��<���N��L~���g�d��F\Lr���B���������P��h�	0,���%������z�/ "�D�q"���t�2o��	.��`���
N����UO1�Et��0	}$>-��@c<��7��&USX��*zpCBwI�n��g5(dv�j�1��D�p��VZ)ry������	L8�~B�	�O����J�Y�^
v�SXh-B��M��q�K�\�R=��/���(�=�����@�06C�4Qw���8C��T�B,G���j�S,�TM��&
T��5b8� �B��M*��7�,��/��qQ���S�t������g�g��Qf�*�4�O�4e���39^[F&���,��v��u�Ep�	���mF�9�t�B`��@�T�b�&�)�PgD;"\YI���o^8��W~��As��x*���J�yXy�e��������U�����+2���������:�8��Z��W���p}�g�,��K�����'l�AVS��(AI�\
�]bX�!I,e@x���z%jS���<�4���e+/���O��R��)/�Q.y������@Gf����������l`���(z�WZ
�6$�(�e�E�#�{�U����r�H1���Uk���_��"[�oI�����|�o�� �_��I��Y�7)������I�h?	�4�Y�u�4�E6�q�B����/�e�$U�r}�yg��)6����*O���b%�^K�������h�7��������0�ejJ�+���'���G�*���Q������&
������F���(���:�L���#/�cD�/-��dD��������]R���H���d��+9�IE��2���D��gZ#��C�u�A���P�F:z�����	�C�u8]��f��6���1��D��#�����Kp�m�/�a�I��pQA;�L���:kSL��7�Ug�����U�w'���r��P+���S}�������p�������]]����o�����r�0gz8��f��TT�r�-+��yq�1U������fJ�p�����L����)MA�l�P�O�E@1�����huQP�r,�*E���p�o��=�^8�_
�2'��rV7 ����j�E�8��T������Ic��Q���X�i�^���r����l���#pf�E>_������}%�NO����������#�F�)�i��I,��J-���)i^��k��U���������D�>"{q2����NM��vk�>��["�wPgos��w�KL�&��L�=P��Pr{J��9"7��N���{��-��G�b���3M����i�#����eu_��em���G�5�Y0��������Y[���R����F8��nn���{"7sG9?R�TP�������a������.��������k����dFf��/��z�Y�z$���	���h�8��cU����2 %�p���+'��f�������i�H�h����i�tm��Ow�������N���U*-��u������fw����Q�����#�`�(�����RQV�-G"��_�E���u�Q1�d�����dI$���+G�_7�n��w�������F�mn�n����k�X"o����f������+���0{\+@wo\�����kv2��C�
��C�Q�Pl���>��X��]{��#�}<�
��u���
i��q/���+��[��"hw�1���eX�n%d+��*�����IL��b,�>U�@��*�A��e���)�;��,Zb5A��%}xVrj�1u7uE����R�����E8��|�|������;�������}-�������x�oG��������Z^��&6��[�(`���Q�]n���;A��V@�Z��Y�����w�U�Tw8l��
�vB�.�6ry���;��>�v�����
wb��4�L�����7Q�.*��������W1;����y�L�8���J������PKiY�8%�-PK#�tJ:patches/0010-Multi-level-partition-wise-join-support.patchUXz��Xz��X��Z�s�6�,����J������L����'����NG����"U���k����$A�z�����,<��ow���5������������3{2��f��wg}{�\N����Bv+6���~�����m�2/��\�I$W�'��	�����%��Q���7�/���9�����'�R�v�c�����?y��v<�[����p��������2�j����4H�n >��mx����� �>F~�d��Dq����0�X�����&��H��+f	�%|�|,���Ig%�����A�h-����V��y��q�@F��������+�����F�.�<��C�_gT~��y��������t��q�R���:@�e2t����X��`���|-`
�� 8��������X�����a�.�mI�YG2�>�f���|�0��H�x���Z[��?���?�P-�c���R)J��/�XH	�0�~Le�F�q��*����e(�b�� ���|�����/8�0�T�-Fk5[(�Cb^+���=���M���U�fK�<�������d��v���s���'`�4�$���O��Sp0��=���|����;�6�3��{P������)3�����i�F@|��+2Fsf�U����� �`O�9�� uEE���CY��]�d+���l�?�"&S6��M�+��-�O�`�{?a��8-����/L���y�7�GCo��OF#��cW�@�G�����;�vg���=g�t/B��w�v�:���v�l^�-`�#;��(��	-"hY��'��0�i�,���$��?�oN��R��3�Y�F�����n���i��\�F�c'�H�!�9O�3��������Q�`� ��tPrp���G1@�*�������������^]�gqs��Y�G�4N:���hw2v�^���;�b�9C��������r�;���1�����
���OI�Xg�#���jtH���T��,)?�0x{�$��$�e�BG����-��qu��U��l%bel�.0�����"�V	:����v�Qp��h8�m-E�Y`����^�N�c����P@�&��]*���m�=�����t�G��L��^���-�������*3���7w��;���w�_WI�u��`��A6D����=[g
�}W6o���_���NT^�C�C�R�b�`��
�>����e� �Zh(��G�����~��DL.z�����A
jv����"�p)"��4\|e���������V�Ff�T��
5��Z�����[�hY�?a2j���I������}�^����{w�'�,�a^��3@������7�Z�n����V9�^H��E��{�x�9�
yY����P��}C�&k���:E�Q7�n1�-�r}q�7�LI��i�bc��q���;S���Zd�17�~\�
����~�J��ILY1(�`�z-\�(-^d*G'�$��Y�f�tX���&m�z@nS�x<?�	L�,m���]w��]�,u�KP����g���0��<P�`�(�W���#�Au�z��%;���i�[��������)�j+�(����fE�������XG���e��$��B������r]����v�D��-F$�g�Ul-�<�M�M��C1MQ�����
!��X��z�����E�0�����I!W����r� ����`P_�������r�?Af�R.�ml�2(V����HnrF�fS��������G|e"{�\0
4�G�/�9������9:�J�r:��p�W�)A���Ex!]^C��%�&TQN��6:]��O/8[�$X���h�[
�f�0�2�����^n�Kg_W��OHRH��C���p�4�&"��X#�++%�E-a�?!��Hu]��[S�tKEZ��2Bh�������0|||��2vp=;j�#J�gA��	a/_2(�>\�->����Y\]\�_��#���!��.AF��xi���l�C��v:�K�olwG[��W��opu��Q4���k���@�5*��3�6dNF2��P��a^]�=*����e����y�������:�
_D�=���@�`h�J/���=��O�����+'���"98�����:������Z9���1�h���Q��y��v�i[f�?�����r����z���������p��#w���e<u(S���v���3��l���6}O0��{.^����8Yap�_���x��g��'�����}�7�u���P�-pJQ����1$;Q
��������{r�F����e�lAr�Ohq9���b 0J��0GW����lE-��3'�������@���*���$��!�V�d���rj���
�@�Y����PF�b�L{
��wF�����
"G��G��(J�5O����mO�)
��'��@�P0R���R�.�8=Z?��W�4,�m
x|Mg>gn�/���4�����58�E�0w��V����;����RPe����
��E5�J#v}}Y����;������a%���0�sJoK�>�P65)��:s�����xrlaL�d�YB=
�����a��U�F��e�A�Uy�����LW����m������Jn�TM���&�A����5 �{Uc�T�C����r�|L�i���k�G)���;v�c:�������]����2_��<�����J]�����'b���X���J�i�N�h��(Z�u�Z�KUz��h��=��p�{��D��{���}�W������Q�C<�;���l������`7�O���#R��Qq��@��(�x��n�N�(m�**�+%u_�K����D��a�+|�RKB�����5>�a�V\������X� �s��{c����X���'k����vm��'�"3W2�
�\��O���r�)v��m�E�P>���hzDU���4��4]4n�zm�@�~���N���s]�Z��*[������N���>
�M���B�6�%�����Le��bJF?�5�$3=�����0Q
�#��%^F}���6���G�? v�Vj���<���G=U��<���
��)���l��z��3�F��OzrB�z����f:��9}�9OV"�� �Q���x^:�@�����4d�a	=����,L���O����Z��Vo�c��o��G*z��/���8�a
�M%�|��(�|�-dW2f�\!���B�\�l�'���hE0:AP�5oP����:X��HP����|�K��'��0d�<t����"��~}��+#�x>�"OQ���q�����}e�V�Z
_1gk�8�::9����p|����Tb�o2��Ox`H�����P+�m?�����')H%q
�B��[����F��J�dMN+����>XBJ������3V���i)���L��:�� �Y�t	X.e�p�j:�%d�E���>��~u�>��jk(��s���Q�������)�qt�Y�R�7��T�F�|g����1qn�m>�&:���1q'�wA �:c������B/�U\�ux�yFM����_
_�b��s��\
M#�mQ)��3Fj>��io���������?M;����q��{�����_�\\/��N�2��M{�����PK�Y!*T�2PK#�tJ8patches/0011-Multi-level-partition-wise-join-tests.patchUXz��Xz��X��=ks�8���_q�nm&@@��d�n5�Mf�7f	3{g��(c��3��I�����$���d����W
��t�:��9�/��X�!c�8;C�7
����>�vo�]k�7I����VK�!k���9��:�n���9��w��W�=�6���k�����"K��k���b��V��5}r���n.��B�������y��;��n6�������N����I��x�q|���G���t}��W�����%����v��j�j�n�O��5C'�����m{�X��j������v�X���s����%w.��
�t
����b�Ho��-Io�����W����l��w������������v�����#�&�`/)��W?nPe��8��[�X.������P=u���.�����V���oo����}0O����]kK��Eg�f/�|k���t����dAO���Q=����+hu�8�6���zU;��GKsN�����c.�H8�M��f1���
�1������q��#��4x3
�#�__�`����u�����y����t���
�y8�^N/����7�����R(�u7�||�O����/�����A��`4���QBBz���td�B�R�P���� :�PTG��N�C���+�^��p�,�XO$\\_%���R$8���G���#�B���M�pyM�	voFW�7S�7���f�H�����wl�RIrPwdI\��f�A��i�qv���^�~�WH��	*�9$;�h*o�e��p���������>����/`�t>�����L{y�=������0!�GBY�C�t	�@���6B���P����
*�dT�yH���*�<��yH!�tH�<�^�)�����@r�!I���:�t(�y #�B2�P��@�;��<P�y ��@�������8$w�e�R8�@P������������^^C������f��7���
��E�Mz8��6������=��V�+���P�>��4��x7�+����@pl��&��AOv`<y;�0#�h/J�d6�������7��\k5gB�~����\��<���
]t�������c���j�
`�^��"�7;�]����R�h �o���rq�:��_y��?e5��}���W��8l4v�1�n�{���%�QXo��m��[����R��4d;��oEC5�b��w3�LQ�]�]�s����`��(���@9Z�����j�����%�;���;�G�(P�L%�H���k�:.�r�(�*).�1��z���y$�"�y1Nfcz�j$������(�y����|����	������|����<�[�X��B�X�kh���	����q��wnBz��B�RO�����>����'0����m���h��^# ftgj.�=`H4&^�h�o��U#�8���g[��=A��������{�l������,+�v#E{����z���������w�n�=sw1�OpthC���a���^�� �E�
��;�.:�WzA�`Pa�
�j����]}:���|�	W��)�}L�E_gJ7&��"4�Wc):������4��/pA�����4C _��TO����Z _��TO��%jAt�%j�a�zX�j�^���/� X;u��
���d�Z��I�U.�P�%���	J�V/�p�'���&�$l��2��d{#hb$`�����:���A�_��Yf��<*�
L.����g��@��<Wp����$�$�����	�LGS��������42��t4��\�D�5}�\L%�������H�-v���b�R�&������9e��:��>��
���8��!������D+��'�`�[*Um;��@So����N���C(;�q�i�Em'+����V�fO�������;��ASo����N�N�Nf�$���^r�������EGX~*������~��=��o�q���eqV��p
n�r9����F#���&+�������n���<���sN�s�+b���H_A�_������}�������)g#�N���A=%��j���q����e�	��W�i�����@��}Q�/=�)�!���a�=�(��xD%�=gxE��/��Ht`5�Y�N��tP&
 ��
���8.��e&�JR�3�<j1�7�#6���/� �L�U�*�E��8O�|2����4�bz/���s����R�@)�C)jQ�Ci�CRD*+D�"��E��-Um��R��C�%"���LQJA(e�(Esn�Lv�L��L��LK�l����D
E�TY�������H=O(G ��$�23�q�����8���;1x1���F���\5=J]-I%����r��a�b4�������x�a���"������S��u�*�����o���@(M#�V����[�r��d����S$���b�h�!	*L�+��B������g�?���BS�bw)s�$lqS�7	k���`�qsd�������jO�W�^
�����v��w���6}6����)����!�E3L�9��|n�M������b�L����E�v���
�<AZ���6�+>��"�
��P6�n�n���,�t|�P�
o���C1��FP{���&^��_.W	�`���������j
W��L�����,��(^A7���6�g��D�_� %G!�t2a&kD��4�����!��![���
?�`'`6�X!�"?%R���������`K]J�EjO���X��<,"�wf�b2,�������53��/P��%]{���LS\t�gK.SpE}��,J<?�K�F��s�b��8�_(�g�2���
���VDQ�������� �
�yX)**,�XYlXy����������(n���� r����D��2W�A��A$f���#fs+vq��]�/��s8��+q�5�Ql;�������{8���w���O3V���YrUkf��?�
�-�����g���\�Z��g���Y���J�;�L��4�B�=\�%���K=1[a�8.S(�c����Z�bo\��������F�`�)�����0�k��n�;��[���.����C��v���+�(NqQ~haG=!���F�"i����{i|�aG���!�aT$
#[FB�Sc'�v�D�2f*+�'���Z����bv���L���w��QBO�����t�M�I~�]��6(If�I��]A@<�(I#f�	���$cF�t�Q~����$9��'�VJR�Y}R�L�AOOvJR�x��$!��#{����1�$#�����?���>������/����di�l�,��x��}c����-��96!
+�����q���R�
�^��buy1�,cP��k��4��_J[$�Y$�����������h&�UQ�R!�"aZ��j�^*[$��V-�K�Y��V5���|��������G__��r��
}A�@����_������������WjT��H��I���HMC^�4?_Z*S�Y��+��y�������u��oA.g��?�\��������M3��rV;��@`��,�7@���U�������V/1��q�����S�����8����4z�z�p�Y��nej&d,f�����DY�Z��i+�8��B�a�*�B�e�Q2.{Y\j��tr������d,Y,U�(�5�^��e_p�\�Ks����_��\���;\����4��,.���`��+��r��W�#���������;���-�Zw����Z'>�������N��k:qf�uO��\�Q�^.�8��O����5{�G3�~�W�6���v||����+hu;�9��`GO�.�bjf�<���{�k��A
�b�.z4\��{������au!�Kuh�=���=�������M�c����K�����,g��),�C����6�m��\��X������{�t�{�_���m�,L��'*��|xph7��,-����(\�������3ge��{�]�3��3�t���w@a�.��_1����cz^�������������C��X/c����<���{+B	J�j��������v��g���|�vG P�^3g$z����q�j"F�i��j����4rGa�b���841�t��F UhM�K����q��;��'=�d2�Uj����{�D��+K�[n���<��M���Zl%��l�v�<��;}.��y<YnG)�|�L��f�
�yfP�������S2$����C���&wS �Y*k����Z�����p���'�,"�^���dD��0��
��u�=�I�J|g�#���������������j2��qT7?H���@�||�O����/����N�Q]��%$�������VF+�-5
���#6�9
p�)�Q\)�
���kd��z"��`��*���(�"���=b�0d������h2��kzN�+�-v����3�:�����Gb!rG��5}2c~�x�N���&����zx���B*wNP)H�!�9ESyk,;'��t0O�o��0��=�����X.1Y��jI�E�.��/����o��xm����#��,|7-��Kx�h�����ROK�>&�fT�}H���*�>��}H!�tH�>�^�)�����@r�!I�����t(�} #�B2��*B�>������vH�>��} ��@r��$���zY���	�A�t��V�����,�)�<!�s���EP����y�*dV�Y�W7��s�_%����_�GT�,R>��q����|�||�oK=�l�����������[(��^(�|���0>�
��n{�>k��PK����PK
#�tJ@�Apatches/UXz��Xz��XPK#�tJ+_/�PEG@��6patches/0001-Refactor-adjust_appendrel_attrs-and-add-find_param_p.patchUXz��Xz��XPK#�tJ��?�G@��patches/0002-Refactor-calc_nestloop_required_outer-and-allow_star.patchUXz��Xz��XPK#�tJ���
�<-@���patches/0003-Canonical-partition-scheme.patchUXz��Xz��XPK#�tJK@l��5��5@��X,patches/0004-Partition-wise-join-implementation.patchUXz��Xz��XPK#�tJ��t>-G@��Wbpatches/0005-Adjust-join-related-to-code-to-accept-child-relation.patchUXz��Xz��XPK#�tJ���;��1+@��Pqpatches/0006-Parameterized-path-fixes.patchUXz��Xz��XPK#�tJK~��#B�,@����patches/0007-Partition-wise-join-tests.patchUXz��Xz��XPK#�tJE$q�]p<@��s�patches/0008-Use-IS_JOIN_REL-instead-of-RELOPT_JOINREL.patchUXz��Xz��XPK#�tJiY�8%�-:@��J�patches/0009-Multi-level-partitioned-table-expansion.patchUXz��Xz��XPK#�tJ�Y!*T�2:@����patches/0010-Multi-level-partition-wise-join-support.patchUXz��Xz��XPK#�tJ����8@����patches/0011-Multi-level-partition-wise-join-tests.patchUXz��Xz��XPK6�
#66Rafia Sabih
rafia.sabih@enterprisedb.com
In reply to: Robert Haas (#63)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Mon, Mar 20, 2017 at 8:21 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Fri, Mar 17, 2017 at 8:10 PM, Robert Haas <robertmhaas@gmail.com> wrote:

While I was studying what you did with reparameterize_path_by_child(),
I started to wonder whether reparameterize_path() doesn't need to
start handling join paths. I think it only handles scan paths right
now because that's the only thing that can appear under an appendrel
created by inheritance expansion, but you're changing that. Maybe
it's not critical -- I think the worst consequences of missing some
handling there is that we won't consider a parameterized path in some
case where it would be advantageous to do so. Still, you might want
to investigate a bit.

I spent a fair amount of time this weekend musing over
reparameterize_path_by_child(). I think a key question for this patch
- as you already pointed out - is whether we're happy with that
approach. When we discover that we want to perform a partitionwise
parameterized nestloop, and therefore that we need the paths for each
inner appendrel to get their input values from the corresponding outer
appendrel members rather than from the outer parent, we've got two
choices. The first is to do what the patch actually does, which is to
build a new path tree for the nestloop inner path parameterized by the
appropriate childrel. The second is to use the existing paths, which
are parameterized by the parent rel, and then somehow allow make that
work. For example, you can imagine that create_plan_recurse() could
pass down a list of parameterized nestloops above the current point in
the path tree, and a parent-child mapping for each, and then we could
try to substitute everything while actually generating the plan
instead of creating paths sooner. Which is better?

It would be nice to hear opinions from anyone else who cares, but
after some thought I think the approach you've picked is probably
better, because it's more like what we do already. We have existing
precedent for reparameterizing a path, but none for allowing a Var for
one relation (the parent) to in effect refer to another relation (the
child).

That having been said, having try_nestloop_path() perform the
reparameterization at the very top of the function seems quite
undesirable. You're creating a new path there before you know whether
it's going to be rejected by the invalid-parameterization test and
also before you know whether initial_cost_nestloop is going to reject
it. It would be much better if you could find a way to postpone the
reparameterization until after those steps, and only do it if you're
going to try add_path().

On a further testing of this patch I find another case when it is
showing regression, the time taken with patch is around 160 secs and
without it is 125 secs.
Another minor thing to note that is planning time is almost twice with
this patch, though I understand that this is for scenarios with really
big 'big data' so this may not be a serious issue in such cases, but
it'd be good if we can keep an eye on this that it doesn't exceed the
computational bounds for a really large number of tables..
Please find the attached .out file to check the output I witnessed and
let me know if anymore information is required
Schema and data was similar to the preciously shared schema with the
addition of more data for this case, parameter settings used were:
work_mem = 1GB
random_page_cost = seq_page_cost = 0.1

--
Regards,
Rafia Sabih
EnterpriseDB: http://www.enterprisedb.com/

Attachments:

pwj_regress_2.outapplication/octet-stream; name=pwj_regress_2.outDownload
#67Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#64)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Mon, Mar 20, 2017 at 9:44 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

I believe it would also be best to include 0011's changes to
adjust_appendrel_attrs_multilevel in 0001.

The function needs to repeat the "adjustment" process for every
"other" relation (join or base) that it encounters, by testing using
OTHER_BASE_REL or OTHER_JOINREL in short IS_OTHER_REL(). The last
macros are added by the partition-wise join implementation patch 0005.
It doesn't make sense to add that macro in 0001 OR modify that
function twice, once in 0001 and then after 0005. So, I will leave it
to be part of 0011, where the changes are actually needed.

Hmm. I would kind of like to move the IS_JOIN_REL() and
IS_OTHER_REL() stuff to the front of the series. In other words, I
propose that we add those macros first, each testing for only the one
kind of RelOptInfo that exists today, and change all the code to use
them. Then, when we add child joinrels, we can modify the macros at
the same time. The problem with doing it the way you have it is that
those changes will have to be squashed into the main partitionwise
join commit, because otherwise stuff will be broken. Doing it the
other way around lets us commit that bit separately.

Done. Now SQL file has 325 lines and output has 1697 lines as against
515 and 4085 lines resp. earlier.

Sounds reasonable.

Now that that purpose has served, I have reordered the
patches so that test patch comes after the implementation and follow
on fixes.

Sounds good.

There are two ways to fix it,

1. when we create a reparameterized path add it to the list of paths,
thus the parameterization bubbles up the join tree. But then we will
be changing the path list after set_cheapest() has been called OR may
be throwing out paths which other paths refer to. That's not
desirable. May be we can save this path in another list and create
join paths using this path instead of reparameterizing existing join
paths.
2. Add code to reparameterize_path() to handle join paths, and I think
all kinds of paths since we might have trickle the parameterization
down the joining paths which could be almost anything including
sort_paths, unique_paths etc. That looks like a significant effort. I
think, we should attack it separately after the stock partition-wise
join has been committed.

I don't understand #1. #2 sounds like what I was expecting. I agree
it can be postponed.

--
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

#68Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#65)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Mon, Mar 20, 2017 at 9:44 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Right. If we could use parent Vars to indicate parent Var or child Var
depending upon the context, a lot of memory issues would be solved; we
wouldn't need to translate a single expression. But I think that's not
straight forward. I have been thinking about some kind of polymorphic
Var node, but it seems a lot more invasive change. Although, if we
could get something like that, we would save a huge memory. :)

Yes, that's why I'm interested in exploring that approach once the
basic framework is in place here.

I am wondering whether we need to change
calc_non_nestloop_required_outer() similar to
calc_nestloop_required_outer() just to keep their signatures in sync.

I haven't looked at the patch, but I don't think you need to worry about that.

Should I work on completing reparamterized_path_by_child() to support
all kinds of paths?

Yes, or at the very least all scans, like reparameterize_path() already does.

--
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

#69Robert Haas
robertmhaas@gmail.com
In reply to: Rafia Sabih (#66)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Mon, Mar 20, 2017 at 12:07 PM, Rafia Sabih
<rafia.sabih@enterprisedb.com> wrote:

On a further testing of this patch I find another case when it is
showing regression, the time taken with patch is around 160 secs and
without it is 125 secs.

This is basically the same problem as before; the partitionwise case
is doing the hash joins with the sides flipped from the optimal
strategy. I bet that's a bug in the code rather than a problem with
the concept.

Another minor thing to note that is planning time is almost twice with
this patch, though I understand that this is for scenarios with really
big 'big data' so this may not be a serious issue in such cases, but
it'd be good if we can keep an eye on this that it doesn't exceed the
computational bounds for a really large number of tables..

Yes, this is definitely going to use significant additional planning
time and memory. There are several possible strategies for improving
that situation, but I think we need to get the basics in place first.
That's why the proposal is now to have this turned off by default.
People joining really big tables that happen to be equipartitioned are
likely to want to turn it on, though, even before those optimizations
are done.

--
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

#70Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Rafia Sabih (#66)
Re: Partition-wise join for join between (declaratively) partitioned tables

On a further testing of this patch I find another case when it is
showing regression, the time taken with patch is around 160 secs and
without it is 125 secs.
Another minor thing to note that is planning time is almost twice with
this patch, though I understand that this is for scenarios with really
big 'big data' so this may not be a serious issue in such cases, but
it'd be good if we can keep an eye on this that it doesn't exceed the
computational bounds for a really large number of tables.

Right, planning time would be proportional to the number of partitions
at least in the first version. We may improve upon it later.

Please find the attached .out file to check the output I witnessed and
let me know if anymore information is required
Schema and data was similar to the preciously shared schema with the
addition of more data for this case, parameter settings used were:
work_mem = 1GB
random_page_cost = seq_page_cost = 0.1

The patch does not introduce any new costing model. It costs the
partition-wise join as sum of costs of joins between partitions. The
method to create the paths for joins between partitions is same as
creating the paths for joins between regular tables and then the
method to collect paths across partition-wise joins is same as
collecting paths across child base relations. So, there is a large
chance that the costing for joins between partitions might have a
problem which is showing up here. There may be some special handling
for regular tables versus child tables that may be the root cause. But
I have not seen that kind of code till now.

Can you please provide the outputs of individual partition-joins? If
the plans for joins between partitions are same as the ones chosen for
partition-wise joins, we may need to fix the existing join cost
models.

--
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

#71Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#67)
Re: Partition-wise join for join between (declaratively) partitioned tables

Hmm. I would kind of like to move the IS_JOIN_REL() and
IS_OTHER_REL() stuff to the front of the series. In other words, I
propose that we add those macros first, each testing for only the one
kind of RelOptInfo that exists today, and change all the code to use
them. Then, when we add child joinrels, we can modify the macros at
the same time. The problem with doing it the way you have it is that
those changes will have to be squashed into the main partitionwise
join commit, because otherwise stuff will be broken. Doing it the
other way around lets us commit that bit separately.

I can provide a patch with adjust_appendrel_attrs_multilevel() changed
to child-joins, which can be applied before multi-level
partitioin-wise support patch but after partition-wise implementation
patch. You may consider applying that patch separately before
multi-level partition-wise support, in case we see that multi-level
partition-wise join support can be committed. Does that sound good?
That way we save changing those macros twice.

--
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

#72Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#71)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Mon, Mar 20, 2017 at 12:52 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Hmm. I would kind of like to move the IS_JOIN_REL() and
IS_OTHER_REL() stuff to the front of the series. In other words, I
propose that we add those macros first, each testing for only the one
kind of RelOptInfo that exists today, and change all the code to use
them. Then, when we add child joinrels, we can modify the macros at
the same time. The problem with doing it the way you have it is that
those changes will have to be squashed into the main partitionwise
join commit, because otherwise stuff will be broken. Doing it the
other way around lets us commit that bit separately.

I can provide a patch with adjust_appendrel_attrs_multilevel() changed
to child-joins, which can be applied before multi-level
partitioin-wise support patch but after partition-wise implementation
patch. You may consider applying that patch separately before
multi-level partition-wise support, in case we see that multi-level
partition-wise join support can be committed. Does that sound good?
That way we save changing those macros twice.

That seems different than what I suggested and I'm not sure what the
reason is for the difference?

--
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

#73Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#72)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Mon, Mar 20, 2017 at 10:26 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Mon, Mar 20, 2017 at 12:52 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Hmm. I would kind of like to move the IS_JOIN_REL() and
IS_OTHER_REL() stuff to the front of the series. In other words, I
propose that we add those macros first, each testing for only the one
kind of RelOptInfo that exists today, and change all the code to use
them. Then, when we add child joinrels, we can modify the macros at
the same time. The problem with doing it the way you have it is that
those changes will have to be squashed into the main partitionwise
join commit, because otherwise stuff will be broken. Doing it the
other way around lets us commit that bit separately.

I can provide a patch with adjust_appendrel_attrs_multilevel() changed
to child-joins, which can be applied before multi-level
partitioin-wise support patch but after partition-wise implementation
patch. You may consider applying that patch separately before
multi-level partition-wise support, in case we see that multi-level
partition-wise join support can be committed. Does that sound good?
That way we save changing those macros twice.

That seems different than what I suggested and I'm not sure what the
reason is for the difference?

The patch adding macros IS_JOIN_REL() and IS_OTHER_REL() and changing
the code to use it will look quite odd by itself. We are not changing
all the instances of RELOPT_JOINREL or RELOPT_OTHER_MEMBER_REL to use
those. There is code which needs to check those kinds, instead of "all
join rels" or "all other rels" resp. So the patch will add those
macros, change only few places to use those macros, which are intended
to be changed while applying partition-wise join support for single
level partitioned table.

--
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

#74Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#73)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Mon, Mar 20, 2017 at 1:19 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

That seems different than what I suggested and I'm not sure what the
reason is for the difference?

The patch adding macros IS_JOIN_REL() and IS_OTHER_REL() and changing
the code to use it will look quite odd by itself. We are not changing
all the instances of RELOPT_JOINREL or RELOPT_OTHER_MEMBER_REL to use
those. There is code which needs to check those kinds, instead of "all
join rels" or "all other rels" resp. So the patch will add those
macros, change only few places to use those macros, which are intended
to be changed while applying partition-wise join support for single
level partitioned table.

Hmm. You might be right, but I'm not convinced.

--
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

#75Rajkumar Raghuwanshi
rajkumar.raghuwanshi@enterprisedb.com
In reply to: Robert Haas (#74)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Mon, Mar 20, 2017 at 1:19 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

I have created some test to cover partition wise joins with
postgres_fdw, also verified make check.
patch attached.

Thanks & Regards,
Rajkumar Raghuwanshi
QMG, EnterpriseDB Corporation

Attachments:

postgres_fdw_pwj.patchtext/x-patch; charset=US-ASCII; name=postgres_fdw_pwj.patchDownload
diff --git a/contrib/postgres_fdw/expected/postgres_fdw.out b/contrib/postgres_fdw/expected/postgres_fdw.out
index 059c5c3..f0b1a32 100644
--- a/contrib/postgres_fdw/expected/postgres_fdw.out
+++ b/contrib/postgres_fdw/expected/postgres_fdw.out
@@ -7181,3 +7181,149 @@ AND ftoptions @> array['fetch_size=60000'];
 (1 row)
 
 ROLLBACK;
+-- ===================================================================
+-- test partition-wise-joins
+-- ===================================================================
+SET enable_partition_wise_join=on;
+--range partition
+CREATE TABLE fprt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE fprt1_p1 (a int, b int, c text);
+CREATE TABLE fprt1_p2 (a int, b int, c text);
+CREATE FOREIGN TABLE ftprt1_p1 PARTITION OF fprt1 FOR VALUES FROM (0) TO (250)
+SERVER loopback OPTIONS (TABLE_NAME 'fprt1_p1');
+CREATE FOREIGN TABLE ftprt1_p2 PARTITION OF fprt1 FOR VALUES FROM (250) TO (500)
+SERVER loopback OPTIONS (TABLE_NAME 'fprt1_p2');
+INSERT INTO fprt1_p1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 249, 2) i;
+INSERT INTO fprt1_p2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(250, 499, 2) i;
+ANALYZE fprt1;
+CREATE TABLE fprt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE fprt2_p1 (a int, b int, c text);
+CREATE TABLE fprt2_p2 (a int, b int, c text);
+CREATE FOREIGN TABLE ftprt2_p1 PARTITION OF fprt2 FOR VALUES FROM (0) TO (250)
+SERVER loopback OPTIONS (TABLE_NAME 'fprt2_p1');
+CREATE FOREIGN TABLE ftprt2_p2 PARTITION OF fprt2 FOR VALUES FROM (250) TO (500)
+SERVER loopback OPTIONS (TABLE_NAME 'fprt2_p2');
+INSERT INTO fprt2_p1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 249, 3) i;
+INSERT INTO fprt2_p2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(250, 499, 3) i;
+ANALYZE fprt2;
+-- inner join three tables, all join qualified
+EXPLAIN (COSTS OFF)
+SELECT t1.a,t2.b,t3.c FROM fprt1 t1 INNER JOIN fprt2 t2 ON (t1.a = t2.b) INNER JOIN fprt1 t3 ON (t2.b = t3.a) WHERE t1.a % 25 =0 ORDER BY 1,2,3;
+                                                     QUERY PLAN                                                     
+--------------------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t3.c
+   ->  Append
+         ->  Foreign Scan
+               Relations: ((public.ftprt1_p1 t1) INNER JOIN (public.ftprt2_p1 t2)) INNER JOIN (public.ftprt1_p1 t3)
+         ->  Foreign Scan
+               Relations: ((public.ftprt1_p2 t1) INNER JOIN (public.ftprt2_p2 t2)) INNER JOIN (public.ftprt1_p2 t3)
+(7 rows)
+
+SELECT t1.a,t2.b,t3.c FROM fprt1 t1 INNER JOIN fprt2 t2 ON (t1.a = t2.b) INNER JOIN fprt1 t3 ON (t2.b = t3.a) WHERE t1.a % 25 =0 ORDER BY 1,2,3;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0003
+ 250 | 250 | 0005
+ 400 | 400 | 0008
+(4 rows)
+
+-- left outer join + nullable clasue
+EXPLAIN (COSTS OFF)
+SELECT t1.a,t2.b,t2.c FROM fprt1 t1 LEFT JOIN (SELECT * FROM fprt2 WHERE a < 10) t2 ON (t1.a = t2.b and t1.b = t2.a) WHERE t1.a < 10 ORDER BY 1,2,3;
+                                 QUERY PLAN                                  
+-----------------------------------------------------------------------------
+ Merge Append
+   Sort Key: t1.a, ftprt2_p1.b, ftprt2_p1.c
+   ->  Foreign Scan
+         Relations: (public.ftprt1_p1 t1) LEFT JOIN (public.ftprt2_p1 fprt2)
+(4 rows)
+
+SELECT t1.a,t2.b,t2.c FROM fprt1 t1 LEFT JOIN (SELECT * FROM fprt2 WHERE a < 10) t2 ON (t1.a = t2.b and t1.b = t2.a) WHERE t1.a < 10 ORDER BY 1,2,3;
+ a | b |  c   
+---+---+------
+ 0 | 0 | 0000
+ 2 |   | 
+ 4 |   | 
+ 6 | 6 | 0000
+ 8 |   | 
+(5 rows)
+
+-- full outer join + right outer join
+EXPLAIN (COSTS OFF)
+SELECT t1.a,t2.b,t3.c FROM fprt1 t1 FULL JOIN fprt2 t2 ON (t1.a = t2.b and t1.b = t2.a) RIGHT JOIN fprt1 t3 ON (t2.a = t3.b and t2.a = t3.b) WHERE t1.a % 25 =0 ORDER BY 1,2,3;
+                                         QUERY PLAN                                          
+---------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t3.c
+   ->  Hash Join
+         Hash Cond: (t3.b = t1.b)
+         ->  Append
+               ->  Seq Scan on fprt1 t3
+               ->  Foreign Scan on ftprt1_p1 t3_1
+               ->  Foreign Scan on ftprt1_p2 t3_2
+         ->  Hash
+               ->  Append
+                     ->  Foreign Scan
+                           Relations: (public.ftprt1_p1 t1) INNER JOIN (public.ftprt2_p1 t2)
+                     ->  Foreign Scan
+                           Relations: (public.ftprt1_p2 t1) INNER JOIN (public.ftprt2_p2 t2)
+(14 rows)
+
+SELECT t1.a,t2.b,t3.c FROM fprt1 t1 FULL JOIN fprt2 t2 ON (t1.a = t2.b and t1.b = t2.a) RIGHT JOIN fprt1 t3 ON (t2.a = t3.b and t2.a = t3.b) WHERE t1.a % 25 =0 ORDER BY 1,2,3;
+  a  |  b  |  c   
+-----+-----+------
+   0 |   0 | 0000
+ 150 | 150 | 0003
+ 250 | 250 | 0005
+ 400 | 400 | 0008
+(4 rows)
+
+-- with whole-row reference
+EXPLAIN (COSTS OFF)
+SELECT t1,t2 FROM fprt1 t1 JOIN fprt2 t2 ON (t1.a = t2.b and t1.b = t2.a) WHERE t1.a % 25 =0 ORDER BY 1,2;
+                                   QUERY PLAN                                    
+---------------------------------------------------------------------------------
+ Sort
+   Sort Key: ((t1.*)::fprt1), ((t2.*)::fprt2)
+   ->  Append
+         ->  Foreign Scan
+               Relations: (public.ftprt1_p1 t1) INNER JOIN (public.ftprt2_p1 t2)
+         ->  Foreign Scan
+               Relations: (public.ftprt1_p2 t1) INNER JOIN (public.ftprt2_p2 t2)
+(7 rows)
+
+SELECT t1,t2 FROM fprt1 t1 JOIN fprt2 t2 ON (t1.a = t2.b and t1.b = t2.a) WHERE t1.a % 25 =0 ORDER BY 1,2;
+       t1       |       t2       
+----------------+----------------
+ (0,0,0000)     | (0,0,0000)
+ (150,150,0003) | (150,150,0003)
+ (250,250,0005) | (250,250,0005)
+ (400,400,0008) | (400,400,0008)
+(4 rows)
+
+-- join with lateral reference
+EXPLAIN (COSTS OFF)
+SELECT t1.a,t1.b FROM fprt1 t1, LATERAL (SELECT t2.a, t2.b FROM fprt2 t2 WHERE t1.a = t2.b AND t1.b = t2.a) q WHERE t1.a%25 = 0 ORDER BY 1,2;
+                                   QUERY PLAN                                    
+---------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t1.b
+   ->  Append
+         ->  Foreign Scan
+               Relations: (public.ftprt1_p1 t1) INNER JOIN (public.ftprt2_p1 t2)
+         ->  Foreign Scan
+               Relations: (public.ftprt1_p2 t1) INNER JOIN (public.ftprt2_p2 t2)
+(7 rows)
+
+SELECT t1.a,t1.b FROM fprt1 t1, LATERAL (SELECT t2.a, t2.b FROM fprt2 t2 WHERE t1.a = t2.b AND t1.b = t2.a) q WHERE t1.a%25 = 0 ORDER BY 1,2;
+  a  |  b  
+-----+-----
+   0 |   0
+ 150 | 150
+ 250 | 250
+ 400 | 400
+(4 rows)
+
+RESET enable_partition_wise_join;
diff --git a/contrib/postgres_fdw/sql/postgres_fdw.sql b/contrib/postgres_fdw/sql/postgres_fdw.sql
index 8f3edc1..2ac21e6 100644
--- a/contrib/postgres_fdw/sql/postgres_fdw.sql
+++ b/contrib/postgres_fdw/sql/postgres_fdw.sql
@@ -1706,3 +1706,58 @@ WHERE ftrelid = 'table30000'::regclass
 AND ftoptions @> array['fetch_size=60000'];
 
 ROLLBACK;
+
+-- ===================================================================
+-- test partition-wise-joins
+-- ===================================================================
+SET enable_partition_wise_join=on;
+
+--range partition
+CREATE TABLE fprt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
+CREATE TABLE fprt1_p1 (a int, b int, c text);
+CREATE TABLE fprt1_p2 (a int, b int, c text);
+CREATE FOREIGN TABLE ftprt1_p1 PARTITION OF fprt1 FOR VALUES FROM (0) TO (250)
+SERVER loopback OPTIONS (TABLE_NAME 'fprt1_p1');
+CREATE FOREIGN TABLE ftprt1_p2 PARTITION OF fprt1 FOR VALUES FROM (250) TO (500)
+SERVER loopback OPTIONS (TABLE_NAME 'fprt1_p2');
+INSERT INTO fprt1_p1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 249, 2) i;
+INSERT INTO fprt1_p2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(250, 499, 2) i;
+ANALYZE fprt1;
+
+CREATE TABLE fprt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
+CREATE TABLE fprt2_p1 (a int, b int, c text);
+CREATE TABLE fprt2_p2 (a int, b int, c text);
+CREATE FOREIGN TABLE ftprt2_p1 PARTITION OF fprt2 FOR VALUES FROM (0) TO (250)
+SERVER loopback OPTIONS (TABLE_NAME 'fprt2_p1');
+CREATE FOREIGN TABLE ftprt2_p2 PARTITION OF fprt2 FOR VALUES FROM (250) TO (500)
+SERVER loopback OPTIONS (TABLE_NAME 'fprt2_p2');
+INSERT INTO fprt2_p1 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 249, 3) i;
+INSERT INTO fprt2_p2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(250, 499, 3) i;
+ANALYZE fprt2;
+
+-- inner join three tables, all join qualified
+EXPLAIN (COSTS OFF)
+SELECT t1.a,t2.b,t3.c FROM fprt1 t1 INNER JOIN fprt2 t2 ON (t1.a = t2.b) INNER JOIN fprt1 t3 ON (t2.b = t3.a) WHERE t1.a % 25 =0 ORDER BY 1,2,3;
+SELECT t1.a,t2.b,t3.c FROM fprt1 t1 INNER JOIN fprt2 t2 ON (t1.a = t2.b) INNER JOIN fprt1 t3 ON (t2.b = t3.a) WHERE t1.a % 25 =0 ORDER BY 1,2,3;
+
+-- left outer join + nullable clasue
+EXPLAIN (COSTS OFF)
+SELECT t1.a,t2.b,t2.c FROM fprt1 t1 LEFT JOIN (SELECT * FROM fprt2 WHERE a < 10) t2 ON (t1.a = t2.b and t1.b = t2.a) WHERE t1.a < 10 ORDER BY 1,2,3;
+SELECT t1.a,t2.b,t2.c FROM fprt1 t1 LEFT JOIN (SELECT * FROM fprt2 WHERE a < 10) t2 ON (t1.a = t2.b and t1.b = t2.a) WHERE t1.a < 10 ORDER BY 1,2,3;
+
+-- full outer join + right outer join
+EXPLAIN (COSTS OFF)
+SELECT t1.a,t2.b,t3.c FROM fprt1 t1 FULL JOIN fprt2 t2 ON (t1.a = t2.b and t1.b = t2.a) RIGHT JOIN fprt1 t3 ON (t2.a = t3.b and t2.a = t3.b) WHERE t1.a % 25 =0 ORDER BY 1,2,3;
+SELECT t1.a,t2.b,t3.c FROM fprt1 t1 FULL JOIN fprt2 t2 ON (t1.a = t2.b and t1.b = t2.a) RIGHT JOIN fprt1 t3 ON (t2.a = t3.b and t2.a = t3.b) WHERE t1.a % 25 =0 ORDER BY 1,2,3;
+
+-- with whole-row reference
+EXPLAIN (COSTS OFF)
+SELECT t1,t2 FROM fprt1 t1 JOIN fprt2 t2 ON (t1.a = t2.b and t1.b = t2.a) WHERE t1.a % 25 =0 ORDER BY 1,2;
+SELECT t1,t2 FROM fprt1 t1 JOIN fprt2 t2 ON (t1.a = t2.b and t1.b = t2.a) WHERE t1.a % 25 =0 ORDER BY 1,2;
+
+-- join with lateral reference
+EXPLAIN (COSTS OFF)
+SELECT t1.a,t1.b FROM fprt1 t1, LATERAL (SELECT t2.a, t2.b FROM fprt2 t2 WHERE t1.a = t2.b AND t1.b = t2.a) q WHERE t1.a%25 = 0 ORDER BY 1,2;
+SELECT t1.a,t1.b FROM fprt1 t1, LATERAL (SELECT t2.a, t2.b FROM fprt2 t2 WHERE t1.a = t2.b AND t1.b = t2.a) q WHERE t1.a%25 = 0 ORDER BY 1,2;
+
+RESET enable_partition_wise_join;
#76Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Ashutosh Bapat (#70)
2 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Mon, Mar 20, 2017 at 10:17 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

On a further testing of this patch I find another case when it is
showing regression, the time taken with patch is around 160 secs and
without it is 125 secs.
Another minor thing to note that is planning time is almost twice with
this patch, though I understand that this is for scenarios with really
big 'big data' so this may not be a serious issue in such cases, but
it'd be good if we can keep an eye on this that it doesn't exceed the
computational bounds for a really large number of tables.

Right, planning time would be proportional to the number of partitions
at least in the first version. We may improve upon it later.

Please find the attached .out file to check the output I witnessed and
let me know if anymore information is required
Schema and data was similar to the preciously shared schema with the
addition of more data for this case, parameter settings used were:
work_mem = 1GB
random_page_cost = seq_page_cost = 0.1

this doesn't look good. Why do you set both these costs to the same value?

The patch does not introduce any new costing model. It costs the
partition-wise join as sum of costs of joins between partitions. The
method to create the paths for joins between partitions is same as
creating the paths for joins between regular tables and then the
method to collect paths across partition-wise joins is same as
collecting paths across child base relations. So, there is a large
chance that the costing for joins between partitions might have a
problem which is showing up here. There may be some special handling
for regular tables versus child tables that may be the root cause. But
I have not seen that kind of code till now.

Can you please provide the outputs of individual partition-joins? If
the plans for joins between partitions are same as the ones chosen for
partition-wise joins, we may need to fix the existing join cost
models.

Offlist, Rafia shared the outputs of joins between partitions and join
between partitioned table. The joins between partitions look similar
to those pick up by the partition-wise join. So, it looks that some
costing error in regular joins is resulting in an costing error in
partition-wise join as suspected. Attached the SQL and the output.

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

Attachments:

queries.outapplication/octet-stream; name=queries.outDownload
queries.sqlapplication/octet-stream; name=queries.sqlDownload
#77Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#74)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Mon, Mar 20, 2017 at 11:33 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Mon, Mar 20, 2017 at 1:19 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

That seems different than what I suggested and I'm not sure what the
reason is for the difference?

The patch adding macros IS_JOIN_REL() and IS_OTHER_REL() and changing
the code to use it will look quite odd by itself. We are not changing
all the instances of RELOPT_JOINREL or RELOPT_OTHER_MEMBER_REL to use
those. There is code which needs to check those kinds, instead of "all
join rels" or "all other rels" resp. So the patch will add those
macros, change only few places to use those macros, which are intended
to be changed while applying partition-wise join support for single
level partitioned table.

Hmm. You might be right, but I'm not convinced.

Ok. changed as per your request in the latest set of patches.

There are some more changes as follows
1. In the earlier patch set the changes related to
calc_nestloop_required_outer() and related functions were spread
across multiple patches. That was unintentional. This patch set has
all those changes in a single patch.

2. Rajkumar reported a crash offlist. When one of the joining
multi-level partitioned relations is empty, an assertion in
try_partition_wise_join() Assert(rel1->part_rels && rel2->part_rels);
failed since it didn't find part_rels for a subpartition. The problem
here is set_append_rel_size() does not call set_rel_size() and hence
set_append_rel_size() if a child is found to be empty, a scenario
described in [1]CAFjFpRcdrdsCRDbBu0J2pxwWbhb_sDWQUTVznBy_4XGr-p3+wA@mail.gmail.com, subject "Asymmetry between parent and child wrt "false" quals". It's the later one which sets the part_rels for a
partitioned relation and hence the subpartitions do not get part_rels
since set_append_rel_size() is never called for those. Generally, if a
partitioned relation is found to be empty before we set part_rels, we
may not want to spend time in creating/collecting child RelOptInfos,
since they will be empty anyway. If part_rels isn't present,
part_scheme doesn't make sense. So an empty partitioned table without
any partitions can be treated as unpartitioned. So, I have fixed
set_dummy_rel_pathlist() and mark_dummy_rel(), the functions setting a
relation empty, to reset partition scheme when those conditions are
met. This fix is included as a separate patch. Let me know if this
looks good to you.

3. I am in the process of completing reparameterize_paths_by_child()
by adding all possible paths. I have restructured the function to look
better and have one switch case instead of two. Also added more path
types including ForeignPath, for which I have added a FDW hook, with
documentation, for handling fdw_private. Please let me know if this
looks good to you. I am thinking of similar hook for CustomPath. I
will continue to add more path types to
reparameterize_path_by_child().

I am wondering whether we should bring 0007 he patche adjusting code
to work with child-joins before 0006, partition-wise join. 0006 needs
it, but 0007 doesn't depend upon 0006. Will that be any better?

[1]: CAFjFpRcdrdsCRDbBu0J2pxwWbhb_sDWQUTVznBy_4XGr-p3+wA@mail.gmail.com, subject "Asymmetry between parent and child wrt "false" quals"
subject "Asymmetry between parent and child wrt "false" quals"

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

Attachments:

pg_dp_join_patches_v9.zipapplication/zip; name=pg_dp_join_patches_v9.zipDownload
PK
��uJpatches/UX��X��X�PK��uJ3patches/0001-Add-IS_JOIN_REL-and-IS_OTHER_REL.patchUX��X��X��Yms���,���;w�l�"e��i<q���{l_��N�����"��6����%E/N?4�$�D,���3�y6���^p�g�^/��3<��3�#�9����{6��R��Sp�`���tm�1.��)�Q)�"�7l������G�_�T�|��<��&g�;&�)��m�%���vz������#�l�������q{��������'�\����\}p�.��-`�|r�����zd����A~�>�4FI`��<+v��0�����)N��,�"�90��r�0#*q��� #������Gq��9/�	W�J�Y�$�q�OS���9^���"�C���x2M�P�Dq�'e����%9P��#���Cny\�8�>g\X����@~�����a��]ee�t���Ya�8*����=`U#I!�GR�������'X�
?b��GEK$^2��;�D�$�Y�*�p+f��q$�h��s��+��R�P�����"����cY�M�<
P�	���LD��G���i�Y"���4+������p�������:#8�>f��p��y�qP���"�;�?�lT����Z�+���3��v�N�.:�u��Gz��5E7Rvhc�"\��E�G�68x���.��Ls`��x;���	�`�����<txo�mz=�~�*������<����p�|�w�Q;_������;4�� ��J��)l�m������a�e�
Hv3�	O�4�Szz$����j ��`�eL�HQ��O��DG��9K��f�� �4q�<����P
��+���}���_��W����Gj�&4����9���|v��|���
�}��v{��[/�Y�k�]��f��I�:.��,K0#�E�}.y��
j
����
T��$.G}���c.�X���?	��#�<�4BSmO�����/XT�7`'�<x��t�s�q+���������4�0}.Y�N�7]�8��?�F����s��������g$L03����HNK��y����DW��fo�mP������)�E_��D^GY���vf����
�����K_����{�F�U��w�5��@�:�����nnC��J�"��oM�^���~�}w�pq��"���j���`I���l�����aVYgA(�E�A���A�Xu]T��&{%��%�T����{~}u~���]\^��M����)*���C�y<�K8�Z��������s����l_X���}A��]���tg������}�j�����-�9�+@���Q0?j&~{�*!�0�H�����9C	�Z�s�������4)���/���v'M3�i}�ol��d����$���))m6��O��$�yV\��s�
������m���9����Q���z��;�W ���g%���y�d(��q����p��&axl���v��|X+��#��e9�r�����J3��������?�U~��������*��mM�(V�=�& �����#��_ �fU/�-S�Yd�/��bV�RLqxB��I��;���M��bK�6�O+�BlS�\��S������_����^7�-+������1_���2d�P����T��V|T����ApP6i��YsU����	7Q�Y{�V��jVt���T��yw��]=�l/�\�~������%U�d�j������U���g�I�a�	�R�����A�L�'~������tr�Gn��S���n��[�J�iM%���=+�*��)�p�,�r���Y�,q�+>���'���2!���`ls��z,�l��R��?+�^���`t�7e��.��?�B��
ze��@���Y5kc)x���[���iC=�R�T4��r�Eky�������=H���K�sw��?�>� ����JB���n}K�����[�"�%�
s;���?P��d�<-��fs�J��v�����Wn��}]k�����G�������f;Ue����+/�4x��N�q}M�+��-:D��t���:|����{\���Wm�n�4���7^2U�T�Nm���%_%�W�B*4�J���|���U@�y)�����h�Y:�����y�m���/��t��R��h�����z#�j�� ���QJ)�B�*�m��#!�����r����8{B��'6�g��n�,a��Ah�#
G>?��g��Ib#	��7��8������)x<-'z�
�
jx�]�����^�&[���)_�_Y	�r�w�1��x-����jazc��~i��_��> l�h:�	�c
��o�PK�C���PK��uJGpatches/0002-Refactor-adjust_appendrel_attrs-adjust_appendrel_att.patchUX��X��X��<�r�F���WL�T%�R����vEv���b[����S[,��A�@�J�?} !��'�D"�����wO�^��F������g������x0t�����\Y+�;����hdO�&���������b����W��8���8������b�7K}�.�����e�
�H:��l��X�|"^��)���\0��<����{�������'����_���i�o1�+���PX��]/��V�N(���a�l�F���f���'o��l	�w`�c���;��Z����
���aT-���X"��v-�@���G�[��##��X�#`��@d�"�^�k�s����Fn=�v�'���V����h��,�?�P}�s��aa(�m�;8�_��hU��u���Y�0�"*;���#�.?3<�-@^�B���7��W�;<�]�����O��#���s��&�7����0�m�)�$�����
�D�m��QW\��>��Zlv(�j^��-��!��q.����0���D����@v�p1�ak���\z�0Z���V�"��V��'^;�=mK���$D��$���*����yb)6La����E6@B�p�����"i�@�+T�+�'���7�q��WJ7dN�������H��?l��m=����q#��	n�����7��.�-x	�C �]?�r�'��-�����5�l
���"
���e.��������'�<�um!��������)H�[��"�DS��x�O?|�"5��)�;�j����/���:�_�H�)�	;Cat&x.Os}��9��Y���_�y��9���^'��(������b���#����)�C�HO��N�0�F�D�s,�N��qYo�#���p�r�nw���j(����h�rSwE�X{����dl���D�PGe#�J-"���b�!�=	� 6��z���z"=p������/%��'lo�0zZ|E���g?5�������\��?#��X�e����9=�4��G*)�D=������F���+���<g��H��)+?�&&��j����=�O��j����y�7CD�Eq��4��"25�j��,Q��h
��"6��);i��7���it��2��K_QKP����Z:��\���v��j"��^K�3Y�X�{A�-�O���l�H���)^������MP�6|8���Z�t��K�}��(�@AB��st��it/'z)�3��Gl��k:����wB��1h��Oq"�%���%��
�����,�\��.���-_��[��l���x�Hei��`�8�nwi��?���9XGmmn4�+��>�A�� �����$Q��2y82
�����
@^�]��`����lC�i��%�8��DH�10��X@�$cv�x@�������>���|(S���0���`0=sf�n���hU��y@Y��vjFC��G����g����p�j�[�g�Kl���dx�x�'8uT��SM�\�.�?'	�d���������{'I����m�eX��zQ���0���������t�,A���r<���X^u������1U�1�V������+�}g���h������|��A��$6Jb��*�� H~��g&f���	0*������k)	�����(1?,���	���n�7�X[L�N�L��|(to�&9y}�
��m"�������D�8��W���� T{�\!������?�j���M��H���T�������+ �:�w_��w
{�������)'N����d8
f��?5�=`�Q�C|C�:�2������e��'=V!P���A0��~�g���T+B�y� ����<���a�H�IK����r��@��'����C:�����d!X`�\�Q3�hg����F;/�'���nq(�bx�h/j�n�<5�}r���I)$G�(���J�+z!��VA��d��}�������#xwh����zY�
���\�3���$�+t)d��A����`��>���JA>O� o ��*�����f03��@dc�/)|����C^�#.�?��;] ��K�����DHv�T�
^��h��hm�N�'�UZ��p���e�F������R{��.��L����jy�q3�Wxeo���aX�6�"/��kQa��n
R�%����&���O'�J���gH9�v��)�T�z�$�#A�~���-�����P�"%��Y��
Yd�3�qR)��K,1���*S"��J,���g��B|O�����$����^��`4�+��:A�cU�B1��#K��]�n;
jE�u�<��sC���� ������������o��/���_��x1����N��
E��sV�Y*r��P7
���@W4=��5�RdW�`���.X*y�~������,���H���x����
h�Q��l6`��!��-�|���G�8�Or�O&��J*��)��L����n��~�����#H�n<��4�.q9�*�n�D?��zU�4e[�~��_Q�������$�K�U�@7��,��)N�����^��#����7z���6��u��!=*�y����HM0������b^�����O���<CP�g�B�?�
y���X���������A��>`���SnX��~��}���+{}:�OF�-�I�j�&���g(���t:3��j�Y!�D����[q��N�o��eT�{�I`9�a���0����Q�F���.����j�L���R��$^b��b>�����7��L����������L�	�?8�9�����Kjb�q�.�5������P�3^�sMS��-�t�R:j��-K������Y`�>�*?!��9�lf��f1�DY?��E�Z!�v�]0+)4 �H������ZB������k����S]���]�����BG��zX���O�r��-_CR-Cvtl�g#��l4KS���m��TO�C���H��9,f�zC��g�UE�uE<�gA� ��*6,]Y�����EN���W@�T���1k���E��Pa����kA���Ng@7b�l01#��b�Im%+�O��1��P���Oz	8����RX�����@����R�����F��F���#�tP�A�0��\�N�(Q/W|�����"P
d����!������}<������+�R�:�����G�=�>�Y9�U�\�<������L�b{j������|��J<p�JY/_kO�����Jr�b9an?p�rA��VNV�[*ur��������'����[Xo%���s���m�z�7��z�[UR�M��>D(��`h��������a��b7�a�:[�6�����,>�D���7�
����%���>;���l	3$���fZ
�eC�������(jA.�.�'	�"����������A��p�;�2�a��M:Dh���8��T�,�����%3n��m�]&|X��J����t(^\n�����������*+^5���`F<Se!��	@qm.KC�s�G��N��Y�R}�h	X8�I�
���M��Sg.���7��	iU6��F�$fP���U������$:C1iAA,Y	�����rg��E�?�
��YL2�t��0$��`V7)#�b?0�T�+e�>�9/�
����Y����9���y���s�(H�?��T���.N8��f;��E�M�@$'�
��Ie4 Jp!2]N�R��)Hj�������}ucH{��=�hQ�_q�B�/�����=VX�#d�������Nb����.���;|H���hl@~������d}ZA�O��O�}��0�oY�oT�rD$I�0�=QX��'��xm��Iz=��`����0I��p�]�b���L��m�5���|�d��x$=��Vod&Ph��E2}K!���Z���6w:�(����6F�#j�
��gX���_�]]/�]���/�\�y��j��5�� }���T$���oSi�����P�U�E(wWr���N����������:���z�9������H�ER��zv�~�;rl�V
���E:�z��AV�a/�&v�h��A����*�������y���$����H:S��T���[dB]���&�B*���<@����A��g�V7T/�q��t�B��irz�I�pE��������M�Irt�?�]�U�0;��=�����=�$�$��P;������EbB����T*���5�p_ya���U���O�T����-��c�=���2�wD�h��-o�`��x�xW��eY0�}�ZHN�'���=+p \���|��(`�1M�q��l!���Y3�(u����(d�$z�I�6�"�M����U���a����ZN��Q�;����3�N�w��v��Pj��E:�9�Z'X���r�C��mE��o
Gu�^�i�0I`�6
�{CVy�
�a;veT19�����jR�����X3}Ut)�#8r{J-����Y���|OJ���������O��}.�PE��GS�?�V�q����pU\��{djRO��(����'M�:��"bi��=L�<Pz�>I���.3�KxI���N6��������+�AV#]T�
F�Sx����j�r���Y����0� gc�������E(_�Km���,M��O����y��/sW�d�Q���Q:A�4�T������&f��K�`�c�L��D$J� ���	k,ts�`��W�m�zl����Y�P8��#!f]�Z��i��
����.�J����q>����sU,���HrCRy�������$F�����S��R �A@�`'/X��c�
��.��+���Om>���mn�(?(�z���U� 7��K���GCV���,S����R����2���u
�H\[���Fa�I�`�z��c9%�Y�K�xfbS���`���"+�1�z^������WC
#����`W/�a(��n-���g��������!I �V�]+���|.������eS�	y<�\����d�]��$'�u��Kb���cyt������=����V�^�=��@����0* OG�X��5���	&�8�S����z��)XZ�������2 c�squ~����_.�Qs�����Jv�k%�����u���������T8(<�R�!���O���x��,0�x{v��v�bJ������$�ULL���z�4�Ir���Bz�[�����|�:}&�J������.�����������w'�Ywl�PK5�kY�*OPK��uJGpatches/0003-Refactor-calc_nestloop_required_outer-and-allow_star.patchUX��X��X��W[s��~����m�"u�l��$M�I��i����3D�j��@+n����E]�43��/�������\��2#6e�e�gQv����9����,�/&�e���~�%��+��W���^��+x�W��z?����5�������pU)�y�Y<�~d�_�����������B�-�,�OC��z�'O�������_B8G�?��Xb����I\rmr)�X���B�4�5��+SX��u�
S�NV�`��R�����������eb�,5���� ��0Q'7p�f��b����Q��<$�k\3+
K��k��E��Y����%'b�i��*��,%��iMK�D�
�(�L����H���I�+����O]��[�A��m��k��U�G��Pr4�J�6R���T7�&��dUHmH���
Z��������
�J�[G�\���K^&D���9�-k�6�.��]��;x��{�U2^�����XVF�/��D1���C��'�L��s���������*e�)Q~�)�$�S��o�V`/����D�)d"�3���<�����0�����]�s�>�'���,����a�^>��e�?��E8��A���<�B���y��=����>}
�d1E8��%�����nl��[��m����WXp��4#�t��7�u	�.���������#���qy�
����U��$��.�xg��������hg�2Z$�'��r��`�Y����e���_�_~���aG��OG�l����B���E��`0>�?8�W�[�	����GF���+]+�j*%oE�i�e���
���?���(���=J�cd`�:
g�s8����u�m2xoD�F�N���1����1�������}z�nG�_*��YiC���������O�:�!��������g��t��5����K���c|�� t��9�8Z�m`�+Im�hA`���n`	���b��37Y��MF���(�dG*�.�5��n�
Yn��5$�u�����F��6c[
h��������x�S	]ut�������_�mPz�kac[SK����1�����>-�[�rV�hF�?�lr�"��V	��Pb�I���iC���Q�'���e_�[�����u\�n�]��$�.fmh�^��Yy�}���#�~�a|@H�N$�P�B�;�[L�5oK���Zq�m�(6������{9�g{tMw�H�b1��&��b������u�>-Dty>�����x�q�����	+ta�o�>��t�P\������I�&��T���i�����i����Rw��}3�Q	�L���>��q��r{�������nt�Y�~���������,������6nvvD/�z?�4o�Z�w{�|�*'�.d�e����������I2v���'d~�r��U	dB"���~��*fq9ZP�\�x�z��\��NEF�%�k�]��6�� �!�@�������k_��3A�v���%{���&F�\b��]��w���6��t������X/�[�+�;e���P��x�)����_S�yQ��M8�\�c������������$
dO9��0�E:K���]�>*d��G�l��6����u8(p���]R}r7��
���s��������UJ��&;���%
�}m5`�|���?�����'���4��}�b����q����Ep�=�PK�[���DPK��uJ-patches/0004-Canonical-partition-scheme.patchUX��X��X��;ks�8���_���j%[�%K~���x���$N9��s)DB��%H+�$���|�����#���~7�.KW�X����t~t8?:���T����������8��|:^��i���5�����F�q��y�����S�d��5��k�����W��"[g��|��7��<��m!���s���W��W�S�;:���_���D��b�>��x�F����Wv��4��5��(��6,�J��wi�8�Qr���!��<�<I�H`7"�^�W�"e�:s�4�m�|��v��<n@!ja�L����@�^���g�NiBRN���<U�%��,H�L�u��8?O	��!Y�T�������2��!M<�xr�����40W���<-�F�@\]�������<	���"����3�x��xK�E9�����x,�a
T&r�i,Q�yCF�Sa(O�pt�z-8�*XFq���0(O�m�!�
�V���2J�\�4�P�Q"cN(Y�N��G	����/��4��tD �f"^P�`9����u����9�E���<ZE�'�}����q��r����	��?m��<���1�a.;�����2.B�������!�%�?H���!c��7X�(�dt
��"�c�����0Z,�����v��������b���p���O��D���S��s1va7�����l�dz08b��q�����4	g�������p��H�w�4������A�u����gw@<�
�x�g@��p���N'�g��C$���������B�`����v�UR�c���L���C�LH���d���e
�����0T�<��>����������1��Y�o�Q���G�������*/`]O����A��8
F=
s�v�.��~����EF(��b
Y����#2
�2�v+c�`�6j��r������OZ������&k.�t3$(�xn���#����}x�~�q^93�����Y� �Er���Is���^�>��	�Cx�G0G""x��������9���s��o�����E��U��xe2���������tt8�����u�%����;p�e7�_~a7��?���o�_��>^~�
>�Y_���<��M�l����)��?PeT�lq������+5�����P�tU�Vh�o!R��Yp��yn���ToUo�p.��s��D*J�%�N�,�8�$C�=
��(�9�ir{�d�u��}�L���O����\X�l�P=2e!���
��_i��LT�V]V��Lt���.=E�=�����__k�t���L
��4�B����%�4����4 
�b������8 �.�O����n����>���j������Z1Pd��FP�����yv'r4&`�C����V�	�@C!����������|Y^���9>px��
q��tZ6`iB
d�a���$uH�C�N�A����P�����l�x<%�8���?���M�[�/���C|Z��r>�Z
T:�g��FI��������	0�w������{��0)h�����Y�\��,^�,�V�7nc&�k�%8�13��z)��g���P�-6����tz4Y����Cq2	N����@����T���c�H�����C���p�s8�}�9\�l
��H���	K��8�[Z��t�'g��e�=����_U�P��dn�:�A���G1���� ������	�m�`���\6�"
��<Mc�q�@0�"���</@�ft�3�RM�}��V�i�F�P�Uv��4_Pb=4�2O��2mv/g��:�=���(��!"�&�����E5P���H��]E��f�(�K���5�;��gDQ}�����m�):��@�Q4�c����xp����F�uTZ�k*e]�ULJ��E
1>0�\��*�+�Fa�+���&��Or4������������l���v��7F�@O�c�v6O��few���r���C��m�I�x|:���<���<��Y������Y�N*
b@��]����5$���i���E��"�8�~`0�~��q�����U8�s�����w�T'K6>7@�������R������{e����)���@�\OM(R��A���H�����12���j����{J�
�:�!��A���� xL 9UDh;K����*���9��,S�#���m*R�E�,�S\��<� _1e��F��=b&H��I���E����g����M{��uw�+U��>!�	+��c@z�?E��P�����_}z;�|~s{u{u�����������$��%�<���'6�kK������B W�?�����BF��{�*�	�v���v��A�@�t��2X��������2� �{op���M���bVJ�����4s��c���]�iW��,&�2�r�����f
 EhY��h8��+X���x��vQ&-�5>�$c���aa9��StY �""���eu�5_0�#����j����D�J#^����r�� T2ux�o���!p�;h�Ac�+�5����0�����B�GC�����@C�Z��Z�Y��
���1����J���mV�f���5m�,�����&��f]�)�*���Q����z�*�zS&��z�ByY=���
k��79�T���������6r�\S��������U�4s.[���������p�e0�����
���n	6��T�`�~�������{vs{I����U
���1�E�I��S�SA�Es�Y�^4����0��6�q�-��ET]��B&f�:��2qL���t��R�W�M���������F�uM�/I�C��eLN+A�c=�,�|7�K��e1b.e��j�EUu|��Xe�X�Za�5��V�.���n�
����(��d��h:
A��T�G�+^�
(����7{g�%b���Z�����:���v�Y���:��(>C�>��-���v5�lI����*X�{�H�����A�R+�M�5����NQ�D��VZ�`L��F<f�/@p����;��_�`��`��:����`	 �o@��u����4��u2���.�L`��R2p
����-�x���6I}Gj�j��j;��,����ovA��cE|�����Zo�2��������;%R��}tB����->�n\m�f[����cK�����'�l"����>���3��J���ez���jum
������M+�y��R��V��e�y�b!�� ������\�-�����5����,nj�d�ksSN�e�-Q��.����n��R�4y(�xkK��n��G�yI�V�W��((t���aw�������������*�D_%5���2�Clg���	�~L��8��,Uz�~a��=�O~������?R�gne�/H�T6T�f�{B�'@��$Q�qv�fRn��s+/�A" �i0!��qP����d��1Sz����;����$�8��M�l)xX��v�J�'��/�]���I��y�I��@t���B��~��R!��!�X�c+����Ga	��[���]��������F����@
��{V����_��X�[
%!�Mp��S`s_e.V�f�[n���5��TcI�1�ms�E�-���Y{l���X���LE1������@������!�U�1l�c��=�������(~�f���d)��L�j�^��O�F���$��Z7j�{Q�w�CxC5�ze
�O���u�>�	��������f���V�D|3���b��J�QG<�qO��]|�������o��:�:�;���$���pz|������oPv�[�P/������|�*"��u��{l��Mop��2���~���J�<���C�7!�?�H��|%���P>X7���t���@uS1��Y�K�o��;��`� _��?#A5�/b^H���}R��Ej��4s�z|�m�qV����h�}�m���Tmp�,D��R��R�k�Ti?SZX��7�n~�nX�=���x]L�I}�`Y_@&3+ ��P�������N$E��f���x,�:����`r����(��CFAm���L�9����~�_�+��x�XI���+k����9��$����6�(�a|��|!��L�+�Mq��+]�w��X��t������U����
��EG<"��;>
�����)�3��!��[J���`{��\�8W���,-�k@r�@�E
}�DPF����hl�:$�������_������;�
90��2-�3��h��2�ll�u3�����S�B$��2�	H�+���U��l?������N%Q���[��97�i�E�����2���v�WiP��)�T�Ex8�N��*���������m�����PQc����5�6����~��8V�i,jYh2%�Ai�S%]k�l��*�H�J���]��h�\=�{����j@t�
B�Y�!.k���B��M�Ga��2�v*����=��e�F�9�PXrx<V��T�8�Y�k��b�>@�0|W�K*������8�X�d�2:��������6t:�%�f1=�A��]�r����u�1���U�$������
Y��0��fB(�Kx��f�i��O�?�Kj��x3�)�[~*���A���������
�
(T���#E6Q������
uH�:x����� ]��~�[�6o�c:	f<��R��Q'���p�*T,�n��Ken1�3~������"]����L
���/'Ei�9��(���,����rBe?C����T�(>��Iz����I���m��T%v�_���������F�;�d��|hp�~ �K��y�ik���4������9 m� �b���xx:<�v�PK)/�R�<PK��uJGpatches/0005-Treat-dummy-partitioned-relation-with-partitions-as-.patchUX��X��X��V[o�6~�~�Y_f��,���dk��]�IW��0�AQ��D*�a?~�P��tmj�q������;����`�a:��&��'I.�\����8��Y:]fK�g8�3��+��E���$���=�9�c[4����J8�Yt�0��j�uU+�yJ�z����!$1������89L&��Q:���&������/�~�(���pY#9����*Q;����C���W�S���X6�Fo��Ap�u��x�kt��7t.l�oQC�KS#�!Xt�9��U�D/�����O���'��
�D�md��8�b����
������Z!��������9���[����P�>	/�N��yR����a�������0F�Q���L��jl*�K�����[�EY�jC	�^���=���(M������J`�J� ��1B�R,����A?r�\�htM����v���x"C1��a8M��I6�8�f�)WcW�%����#��h>����qt&�lV
5�bL,l��N�TY����-���~����5!@h*!��L�#dI%Jx���X�CEl�c�T	�HQ1y@ho��{�N<����2�-�,"��8�X��:;�}A��0^��9%8���~����63|�`���zR����7�#��+A/����b��]a,z{\�:�3��K!��B]r�������Y�R4�yL�PNh���Zc��CA�4��H�����5*�0�D-&�i�	
u���p����������P�G�������'�?m�)x��__��1��h�`7@-$����hsG41��b����`�d{?�]q��O����(9�:%���=��>�I
������56��k!��9x��w�[3�Y�n����i<����Df�l��t������%���<xm���_�?�q��;�9��O'���,���`�<2�������W�-����4PZC#�2����/�T�|hy�����h�
����?�V�O>�lQ��W��y���:��-���s
�(H�I
����'P�s� .���-��_iR|��i���P%��uM�6�5�B��g��Q�����m�����������(}���)p��� ��A��?PKuv��Z�
PK��uJ5patches/0006-Partition-wise-join-implementation.patchUX��X��X��}kw����g�WL���2)��D=����-�����J�zr��@pH�	��M����y�r�{�ZK���y�����*��btO&����xx1�����<;�Mr2M���~4<����,Wbp&����b��Z��w���Y�Yq#�G���G������\�2_�I!��^�-��^F��N���.�|�
~�����Nt�o}^O�.��;������E��h0�Y|��2)�lyx ���d)��*��(��^��F�@�Y��/Y�I�E���R��V�����2������o���2J�Bd3Q�$�(e|�L~YKG
���S`�U�,��������wW0�kY�u\�Er��wLV`�<��E$2�;C�o�tJUE��v�@X;�5���k�%i*�~-ox|���MO��K�]7�&Ul���U�^b!���"3�-6�1lY�8��:�?r	���5T(odK��pq6�����2+��u��s�Sf"Z�$�<��V~F���,M�;�C3��i��
�~!tX������f�i�`��_���S���}��rxx�B`GE��/��8[��y��~��.D��X�_�rzT��Z8I��F0���(M�g��U��	v���C�!�`��O`������/���t6���r������V�k������A�^S�u��4�%"����y��$�by��S�C+l2��ol��=�2��N!��h���|+8.��=�d���<��������S�~;l��@��&f=o�&'��~7�d����7N���&����&H�`��@��#�9rzT����+z��TM�	�a��@T��a��s9���� H�#X���9���*���A�5Mf3qx8O��5��IcQ+YN��br|v19=��������X����$E���6����������{*:�s8�����������x��8����>��h!�"��\/���2�����L���Zj�?J���.u��6��#����:���,���}8�r�P}M7���5��4����>W�jc�6�j�|�=�b��Y���57�R����_�g�,Ke����HE`�'�=�1�&�SG���:��,��G����������@��+�(��&��7"0)�������ol`cW��������
�G<I8 ���
h�M�.�\F���o
�Z��r������)��.��Yd0:`	�S��c����	�R��`�S���L���
`��n���#w��8t�Y0`4������p�8T5B J0Y��pY&p]�k���gi"so�
/	��<|c�N:�S�G��B��C`�T�"�T�XB���l6�Y\9���=������#��������q�E��QUm���P��������dc�����?���^OJ��8��4y#��� m��N�pe��>��	n_��D_�Wpi#z1+�[^�k�8����q��X��V,��Y,E�z@~]�!�a��9�N�6����$���R�e�%��5���?K`������2�_"�c�o2�?@�K�/�+����q]��d]��a�D�a3<Wt���wZ��HW�������\��[y����HR<3���d�I����
���U'��Lr�@�����U	-n��>A)
��_�,�!��R=�m�p�$F�-�tr-(C/+�QVI����������j]�	���
���t����5P��T���)\e��S5�'�����k�����	]|B������dw�Y���Md����a�B���e�-K%�^j9
 =��]t���-�|(��6��K��TT	
kW����H��XA�2������[QYo�U\�^O�����h��z�f�-�V�[����H0��=�dy��@����N��^�}��	�Q!p��dJ�p����C�C��Y�����
�I�8

(CQ�DLrZ�b!_Fe$��Y�pv>2�+��M��4y	����^Z����=%�
�F�2�����X�i.-�~�m�-w&���>�[��1
��t[NNqv�}�0Zf��/�Ip������J�o�O��S�`��'�~X�o��k�'��N�`9|f�y�0 ��4_z��#U-O�P��������������c`�vaP��6q*����OP����[i�?(�\����i6?Z��xH"8����p��J)�o���G��]�J��cQKM@0�F�A����q,�_9�3`�����o�����O��W����v�uQ�w�fl��X!�1*&��1l�imuf!��<����"�E������,Po�&%'|;|���m���O����0�����0�wW����v�Ddh}��u��'�k��j�k���S���E2�W�,�N�VF�-1����"ja���_=�BA���%q���-�PhV����L ��Z��Z34K�'������4 .� ^���.��p�����KfP1���������1�4����K�1�7!uGdM8Tm���Y���1c�@[��1*
b��X.�GH��6��q�)Y��Kq�������x��BG��F���`������%�Gk�q�E�P��!{
_�Ia�:)��q��$GK@�Z���l�d��d��GXlx��Um�X�g�]}"OE��	��i!w�0Y�8[�����������
�k��������U�s�{�e��'y~o/*�e�t����h(Uv���@����
�!	_E4�&<�h(5��9��I�*�B�-j"�:%�kX�mp
���D�?�K�A���!LU�ZA���'S�g����|��Rg��/��)2	�����������
@�S����N?~G��;<�����O�H�m2]�|6���g��#+�����M����|e>����)��$��a�H���F�\�)i�Z�E��ndGo$QM��cO�^�~u���vhC;{��0��T�;���z������=��'�3@j~�X|�/J�A�d�.%^E���*�,�Q�s$l3������5C�
�`���@��!������G��w�a���_�\<����|6^�7m@�P�59C�����z�fP�<�1
�,��Vu��m���F�����&�kj�Q�N@7�kZ���#��ml+�}�sFs����=��Zo^j���S���|Y3z1����`p�������k3�d���S���������G��d�tJR)�?�'�Vy����O�5���fza9E{@ �
����5���������
���qn��yO���������:GP�}E��#�w��S ����0l�2EA8Z�CC�t�+����_�����Q��J%z��{]�����1���!��L�2�h�HS�>�GC�5���j�'�^)��%�6��NI*JDlj��F�h��N���|]�j�N>^�aDi.��=��8�M���#l>��=,�)�"I��.��g'�s���d{-��F���l\Hd�Ch�d��
�p]�5C��e��&��P?��j�r<'�������]F������-�L������U��q�5�x��������r9�U#
	-"�����hJ�s��=�Z�~�Q[��2Y�lZcV6�2��<c����0��Y)s��tq�r�,Z����
U�]����THI���}�E�p�\��|���.w(+fN�%��m2lQ�a%#2����V��G���nbMK���w^��:+���=E�`f;�����
	�;��'��g%9�q:W���E��?s��uy�{�6��f��E����S��QZ1�����jV��=Y0�)�d��t�|^��Oka	�c�.,}tf�m�69���P28;����L]�cM
�h<�y�z���#X�<�KP}"&���>��\�KQ��2�	����/V��{{�Z�5�h�w���S(��<�9A�
V��� �7����K�=��������)��T'@����? ������%���dO$I���b8�����kj��Qjtr96�?eF���������3Z��#�L�J��)��X]�K����y�<_]�A_���|����c#�1�
��p�Xo3�YAI��7�����=�U	�I�`�2�X�e�;���Q?U���a�^,��C����N3����G����q������m���'M���k����a�]
��Hf�L?��R�h���Ig�E�����k2���)�<r�hI}!uS���	�x*W �^�>j��Y��gK��ft^=����H38>1���!�{`�[�|G�E;�YXA
�T��
JH�����A�&��:�������@@��,WY!{�������R\fH��R��Qa[�JL�j*'k4�$���o�?iHf�zJS�m�{Wx+�
�@pv������Fvr"�"��x�mYD9 ��������c@�oM�4���1����:�����������VW����I���R�ke8&���;�k�����������-����F���,��`x1�^o8����xg��i�e��K"���
�qF�PY���)���7 ��D_<�b� L���w���5���5:^���5g���T�������-�����q���Z��v>���������l4<�y�H[���K{4 �������pt�B���~-���C�-����1	��qT�E\�t�@�"VL*(�T�l�!����:{.��0���yb}����Ve�nj]V�V��J�������
��
�|UI���������EF|8K�L��C2%�e�R�`�	.��Uf�2Ob]�|.�)���`�,���}G����j]7����B�5�f��m�q������&���Czzf��T�>1�{�����H,'a��Yw�h{���B[-(�UBR�3G��y���f%?�Y� %o�%YL�_��kF��9P�$���X";��&T�������@E<qj��]Y8�<��j�!�hsn�]>�~������q*g��@�.�����4���;@y_Q\��l��Rs�
���h����Gr��O�U
@d����5@�7<��!fk�z�l'k�����_,$�)���H�h c[�4�P������o�}�}��:�]z�&=lo���?on�=y���s�������+��8��LNz����<>�<��V�v��U]"~'� }'hMA�-BF�#���$E�Q�.W��S��~��N����G�-g�7�+���
���9@�ip	��U�xrA��j��kx�E��D����U'd�'�O��<�TjR�BQ����+e�4����E����n!���
0I��\�����=mdgz�M�K�����s����nO�`��s���h8�l�|��W���;k���J�������z��{��E+��9������CB�n�����q��~m��9S��y�u�����2F�Y�����M�}��c,"�t���(W���Y�O�JvZ��R�=�	/��?�a���B���7* ��a�J�������3C�u��o��n�Np��:�^��MQ*t�]q����k��,eo<�x[xoJ#L��$�,HoHa��9�U����
�*���0$#�����������r���Zo�A(���w4� �;!�t��z9�F5<?�O�qAL�Ht����) ���^���(����
��P��w��^A��2H��' ���*�JA0���o�����2�l`O�����*l�q�	2���aCXUs,�S�p\,k�:j�)SD�Y��Gm����z��n��F���^-�5?��U��H�������a/�����he�cA��Bb@�X��G��e�]pD��������[�Op&��n�X�f��� ��6��a���6C;a�zd;Sh�G��K�~���0t��z�6��l.c��o��c����=������sD>P���U�/������~#�z��5c+�>'d�D�j�~�l8�O��X'N9��4<�X2�gd�PPL�I�rJ�^��V�y��o��[������{'�m)�V�������Yi�|�Kt}���i����^��*(�%�Y����#�ZP��(u�c����G�e�Ft0�t����pL�!�J�%3t+;q5s�g1QYhy}iz&t!����*�BD�4Blh��^�`�����O�-\���V�(Mz�skg��-R��6�����$��m�zw��w�YD�#���(S��V:��o�>!�����������b�����)��y	�3F:P���})�����5���4����k&����v�]����r�$q0�6����Wz�ZAo��8�]P�TuFP��k��)�1k����]��r�gBS1�0?���K�h7�D����9T_���` ������
��������{�1.��i4T����z����O��-���S"#��+�`$G*�RQ�1(��)��Y��]�$����4���kWHf:����IS����e$
�<*�o��D�A�����81�cl'�J����:0��*�L��V�NN�Xg1>B��
�@���9��v�>��NI� ��c��xFSX�R�xS�����E��V��U����
� �*/F�^�LP�WP��Bg�����2��T���!7$v�]����H�e�Vu�pS�e|T���9�(*�����
�����w'Q���G�T�P�^��\966���K��*o 5^	��jg>�HR���Y_���.+_�A)���,��u�@i��J�N����J��k44f8��fi�*l��\�+�U��{�/�J�_*���7)��Z[}A�e3�����`<��c[wZ#��g�+�j,x��nb�u�:���Weg��%PCu�z3F
JKQg��31oU��|�;��[�Y��p@]V���Z1)��B�<��l��y��?��A����"�c��2���#��;\���d�Ri5�����F�TT��Q6B�7�������m��L���W��&1��K��������4)8��S���|����t��j
(��/�(M�{K�)��K�N��x�fy-��)NX�\[��,Z$��)8"�M�*�a�+=}�%�%�V�N�
�m���N�� ����8n}��� ���E=;4��5���X�o ����v�h�`�S
���
5��gQ>/��G��uX�B����$v���6&w��`�<�h��<U3�L�M�bR\}�������Z��p�����=?O���a�FC^�_�\����B=��QV�2�*4����2x�Cj��ET���T�l�����V�/��H��FD�n��q�&t�U���2k��,Q���z�:�o�1y!1������r\��J���	���?:Qe�����3��#Z�U���X�[������0��}�7��2a���/�*��	
�R�O��������spWdq����������y55RM�"�,�+�%&�BZ���F��m��MjE�^(%�?��F�^�T��UW}Tu�K�~�_��@��E��-!�.�p�������Z����l��T$v3d
$<w�Fv��@{3���8�-}c���GE�r���|?�7����6r�{�x#@o������>��5������T@�#����u����Em�����H!����1�G�c�j�Fw�JE��O&��A��������`'/�*��nz���f6R.X�yB��M��9�h��h	��t+�r9�	+�=��Md]a��(�Vsb��6���
��+��	F���!�&���=�f����E�t�����c[�\s+�meN�5m�vAoP��aG���.���[�LK��73���H��*Z4�J�F��W��9EP���������PBj�?��r����M<w�|zc���['��~�~��v�5�F;�0BUk�Xo>4F��qi��S������uQ��<	��&��v��Ww�$e�'~.������������e����"��t�q�B��Et�j�(�96-Vps�Qt������VY7�'EnN(�)���q�J(	�V���!9��H
�2(���E_ug5Lf��"�7�X_���Gw[��C���8�5��e���R0V������,M�b������hxv�z�h4;���rW!m���u9��H����_��.����[������G��!�r��mT��
���q��0 ���EL �����;�7���0���<:>?��d�t�;���mG�:�A����k�n���6�l�oe`�8����7O���o������59�+k����V��B�"$��\�$�Y���o�+�a�G�pzm�#��WB���*k�3�_#��r(>�V���8k�l4��T���5(��F�h�Z��'���.��?9=����}�B���\{�g�Z{{�:�����)T�,NNfs�Uz��5;?*������o=��@Qz�H�o��u�/�>��$x����������<���n�=�H�����]-�u.��2O����b5�i�,�C��-F��CF$3:��K���\Ob����qus[�����lCqW�m\}����������n���.Y�G-SsG�k�����6x�;�j�i5�1�08�7h!�.����)���3I2���1t/��Z�M
�2}Bx#���,i���
}������{]��:~��N��
��8'�Hs������,�� y!u�LA�0>h.IRn��>)J������m����Vz��
������L�Fh<�& T� �<7/n��L5�z~r:=��z3�m8���/����d����O�/���md�wb�����p��J!��Y��`�e8�qk���ug�1-�z0��}��s
������]c�4?����x�h��<��'����c_���D`���e��r:&���*7�ud�
5���=o����ge8=�Y(���;S	[<9?}���Q7�-3���k��Ur~z��2k�<IMo��,{,�6f�����VJ��j���&2�=�1�)�'�TXF�#G���>����DvY:��`��@._����n�rv`�vp�W�s��^�AD��Ag#J�6�,���>������J��2<;�k
\8%Or�����������8Q��(���od��9����^2�X�f�|i[1�Z:�j��Z@m����~D��#����1���Py���;�Hi��Z��,��n��WynFJx�:FD(����w.hz� ��%4�+1�����Z�;IM����wZEI�m��/}BMO��~��s-S/p��R]fN�%���F�f�{�����fm��.��i8��D�5�hy������i6�K[t\�0��p�C��n�m4��*x�4��m�������5
�b+b~�93m����K�8�%��6$Y�W����c	�++E�'������
0�y��N9h#��S*E�yz����$?b�<3G����p��`��|$ /��A�������P��,���~&���LKf�l{�=��	q����.��3�l���qI������U�TQ�����:e��g��z?�m�Xf�����b���[�����T����W������@3���n�A%�i���wEb��	���FTA����8\_�)���w.�m�z��V�?�Q%���q���@}��*�lKy
����3�Em����&���I��VT�'x�B�C�p��i�F�b�Y�[�����<)���8���@���
J1?�-*���-V����R�T�7Wt~Uf��������x�����Q�0��V���T��`o�ZEO�b�s��_3����#,������5=��}� a{&��U�D*�e,];4�:�w]��y��M��Q�_��g����F/�]i���^
I��������j��[���i�H+���G|QL;;u�X�y@�hT��{[?y���OU�Q��*�����j���;���V7Pi�]�\,����S��jA���j��ko\���|�S2"�~��vA��o���:��D���:���E:+��G����*�_�9����P���s~E�p���Nb��E�cb��kT���\�t+�_1�K������i�Hb�%��F���K�k�0u�����qE ��(��a�Q�������+#�#�:<��2*#N}~D��2������iP(%=����u��/��H&���I�y������]_��p_�N�N�{�c�:�{E3��ri���VE)���)�X'D3���<(����9n
�X�02+���D9!�z��L���p�#���ywx,:�����������TO�q�d��&�<��hear#|8�d^��K/�	�~�j_����9�7d��:0o~��) 9��9����<
xG#�%�	m���
T*?ewH�e�����=������{7�O��6�o?G�7�-�L���y��,��&��,O��F����\/�2M����O.��iO�I	&�����M8�B)�������������~��]}����A%��1�8$6h��PM�#a�8����$�w���������{����.��k
����~��]�������1��~��kXX�_,pQ��D�w��:��:x�:T8�K��aTC5���l���������^W�����NcV��O�>|�������2�S:n��q�w���}u��6��}��8'������A�G=����s���#����@R:��M�AO�sWG�8oZ� ��^�U�����%�-�A�������X�V��;�*���H:��<#�Z�pI�C����:S�{$�XO\Q��If�y�(p�(UKqF�2�[��9���Y��4$� �xN��9xM���"����K���^>�^�x}�)
���J�[�'���?�
����:z>���.Ao��\���iy��J�\U8�F���m�s&����_Sq9���3J;B����jms�hp}p���R����$������.������	U��{�+$��/~�l7��`�6�ZS��@�V��#�YWY}iV���SQ��Pq
WY���uZ�$��D����f�l5e@7m�l��*:���{^�N%X7g�F��&*������'�-����]-�g�QOJ����<�*�3�"���}��^'
��h�������h<����L	Sv�L��y;���T��!o����j���[Sv��<;z]2����l��Re��~�huF�Ex�(�]4��7��������a��M��iH�Vo��WQ��N���e�/B���'�5��!n��\��!�]���I��@�����������Z�Yn�������>)�����VwQ7�T�H��2�*�g����!k���
���5
�	�!X3F5y������K�����O��\���`�5o�B���r�x��'^�(8i�%a�s����	�tC���}���|���^\�"X����Au�
j(*���&��S�'�{�|�����W��.�1P�R�|=0�AA9�@J�^�b���d����	��g�[i��@��P�3�D�����
��6���m+X��Ao���rP��>�p��%���p�]\�VYa��v������e}*^�b�&r���+J�3c��h&�~�����7�3��MF��1-t����`8���:���2kUrX����U��1f�b����������Y(�.s��a���8e�����K:���F�~��CVPo.#�FX������,yetR2��M�Y#��h�z�:ml�R���$&3I%+I%#��3���W��&?��ZUTm��:��e�-�
/�d5R}�T�&�/:M��c*��QI��q-��Z�b(Nfr2<�z�����8�CQP���UAY�b�"=��l-�J���%�1�"|A��s�n��9|�	V�d�MD*0<
]a�E�AWo�?��������������>}������������?���������^���e�.��e����WK��T���-�	���}�<*'���6��Z5�����%�8����C���U_JS��T�Q�lM�y�78�F��Q%X�h��(q|A(�?D	��Z�b
g��ZW>>!S&�H�<BE�i=!^�k�#���E����X'Mc�cA2�nND�@g��N���m!�k�P1�sN��,WdgD����_�&���K��6��^y�)\��L�Y�^�F��FS>����1��M9�l�b�n��Zvh��S�����h������y-�h-��^�,X�"X�V��R���8�7*��0��W+���b��]i��"��'�|&#+�C|����ss}�\�s��@��y.��p��#���A�o����2���3y;@��x#q��c�
���sy%\2�t1��U��/��^���/JI�|a/�wW���3��������u]�����K��%~wFB�?�����F��h�����v\pu6C�?0�AC����L,�?��}*6�����	(_7�*�����k�^�)�������E�����Xk'�U�T�92�Wp�����>����=�����d�u����B�+`����_���[�PVok����hVn�j������d0��������8���B�Z�U�����O���*Z����6��+�����u��p���<���
v�BB��}�2�%�d���4[cM�F��8|t?j��K��C������+i\R[E�NO`��#M���4��e#�M��T�����Ht�������S`�N2� ��Z����������[Un�#�Z���]�Z7$m0;4$5E���gB�/�_����j����8YlA��`�p4�g�q�7;L��rWl,v@�B�����S����s��rk�	�wY�E�uK�:��2���&��&�E�Q��
��a6��\D��_����FTD#��p0������F�4���(�Hfxu�M���������r�':���� �����b2��P;{tB��|����S:
��Vnz�7��l��SG�h9�y�.����F��sp���|l9^��C[_Z���T�jl�����\�P
�[rD��;w7	C�qtk��?����q�w>�8��Zw9��������FGA�2K~uc��D��-Kx��S=��@	����m��������*�~M�&CB�L���|M�������������eC�;����u"`)O���.�����1����U8b1�u/�H	��������&sS���k���&���O�����)V8��+|u�b�[5d�,���c>���S��������+�f�r���#�lxT����+zp��l��Prtz>9;����������;����Cez���D��S!SI����l?>������zi
ou�K=������&��t�p=�	.���$��x+��4�2�~x�T~'j�� �1o��&5�e�&o����tG*2����G��=i
���~3�p�C�u�oC���.��1�li�y�V��f4����e����������T��X�i��@�>��U�]!1�����.gm��NX�]{0$���������I��]�nU'�UN�����j���T�n^�����5��Vj����mj�����yk�f���! >&�j����h�V�-�r�E��?����l	�Ak�.:_���N!���#BYd�<��C�?���z
	0�G%VIS.�eI�lTp�Qj��<�	���z�Q��PK��5^�PK��uJGpatches/0007-Adjust-join-related-to-code-to-accept-child-relation.patchUX��X��X��[Ys�F�~~E�<�"�}��+�G)k�[Y�$U��P
�A!�iMr����������r�"�}�,����?��BL�Jzj�������Lf��3�g�{R���h"g3�.���JD*z�+�'�^��	d��������A�����?���q���*�U��A�<�����?e���}�Zb*~R��tz��\�&��7�����M������/�_��i�?��x��Vd��-"���<����=E��uU��!=�>���cY7_�,���L���*���B}I���\��(�����2b���2���2
�"~�E�x�AFs�� s,������w������������S��L-�6���R�4&���6��X>`3O��������Q��}C�Q(��g���H����\�E��Ri�Q�����	VrC�BF^�+�\�A���_�+R�P�*�� ��Kz�^4,	��B>��")r�U�����JE[�v��E�sdz�����
@`}:�V��F��*�C�)�4e
�x{P�:����7��Dx�{�����u�30$���C����/Jh!jd��|���C��`=�x��9C�)3��3���8�20�
��gT
�xJ-�{���"��:�#\%�G�V���X��m"(��)���9��b�\,e&T,����b&��H��xU`�`�]X�x�$dN=t�
MQFh�"�o�����1���\�Tm�?�4(��aG$�#���4�I4=<�'pN���JaU�S��y�wK�*�k��x�v��I�kY�".38$��PM��#��p�L����[+�/�0u�WKQ�8��0�5�5o�%�O���H�k#�����[X1�c���%d
��T�q��2�tT����"�����~�����p�,����(�hX(�n��nGy_
WD����n����*M��������{Rb���!�-��V&��6�"�~�7D�$�EK/����y�@������d6�i ��P�fU�!���������cf�!7y��	t`]�A��|���B�>�<%�T����4@��	nA�/z�Y$�l�@#A*��q���w�4���u<^�n��?Gd"�#"!��
�l8�vv+�g)��D+�����v��KF��<����^j�e��\�^����Fo����Aa���A�?��X��6;�-�DE�Mz=o���w��%�/�|n^�}����3XT24�!�J*�8
�D!���0��'�E't��@�g*�+���e	L�
�������(���F~�>I���k��h�!�QP�RII���r[V�r�^�
�*�@J������� Y����X��gC}�������E3����>�\�����l���P�����"W�^��pO
T�nJ�n������"����;��N��$5rg��FgZ������������B8y=cO���O*�����:�MIP��W����u��)�?���,��F)��gw�R&���"]� j��.��FL���&��"�1R=��
Y�S���a�@�!��*��nCJ'�[����������E�A��(��b�z�bI������T��Q�]y�heX�������p({�k4,k��0��9 c���lp��z{�&��m�7~����8�KC�TX��hb13���2��G!.S�h�{'��|����w����=�oQ��
�����e�����{�v�mq��N�.b��������~
����)�/f^mJ��]G����8A�������xC�O�)�>�l���~��C���1�ez�K���:�"��+��Ka�X����
���"TS^K�&�����/���=5;?h7,T��/��9j�=�p��V_�D�}5��t��^o�;���MF#���5,(��:?�(�����5�9x�q��M��~�E�P_��%,q�m�;�sW%Sfx�������������sj��y�rK?�?.>������~����?��.��`��3>����d��y�
f����w��>�������k��R���"Ou��3�MX9��tM9u?��)fsbr
�%vd�U������Z+9J��~�`�.�A���[�;��et�G?����
e�3H���'�����1�@1�	�qI�8��`���!p��Z�jz��*I,�i-��X*����Y���������L���������a���b�^�\�N����5��B�u�z�^���OZ�a�t�]������������6�`�q��tb�7U��tS�Q��:m,�j6��H��9�`�b��������i��7�,�
n�_��������� ������Q��{!����`0��G���N���|��P�z����{��D�o�%m}���685�Q�Q�57��a��������[kK�
��}��e�S�����t�aAI����;>:��2���^�`]��q/�R5��W�
���!�)^CQ]S�����M�g[��m���e�,I
%j�jJFd��N>m��6��foT�.
G���;p;Jl�������v2*���n����i��T���>Q���M���n]u��&����E
_��E���5P����3����.��G���"��N1���P�\�n+4���H'�������Z;V4|1��#�K�c2�#W��CtM�����_%�F$�c�#bQ$|��A]�2�*��@�z7`�����*������9��F�5�z��q�F��9a��A�L���]���y�f:N���(H�l��d���y�M:D�����QC����-*~E��-��1�h���x$f������v���1�A_��p���{JE;��5�A�e�r��Tnp�I�y!4�}�����#�������_�+g�Q�l�L��e^� ��w�x�����|���H�[����O"X���c�}���n!��]BX�����wa����U�����u�Z�����=q,���a�$�>f#��O���c���"���3T�pb3��F����}s�u�:�xM�[v�'��[+{k����f����l4�tzR��+O���h���FS���7��������m�h8;�B��pJ���n�H1�j*}�7��&)��������:�R~S�%�`]��r�=�&��^}����p�fm>Dk�;8,%��@�V!�m�)-�W]�8�����aV9�x��Z��4y������u�fOf��${�s��d����D�����%=������)��Vs�/h8��%��?�w�^�}k�����gGn����`�_[�~w����DZ���o���B��+h*py�M���"�yL�\+��������tF!��eO{��'7X�����^o>���GAB
 v�a�|J�A5qcy�-�@�����Ow����yz�~���_1�nt��m:Np�p�4���Z�a�p6���%���h�]�=�+u������$+x������3��j;e���3d.��YXY|%c?��*�������Z<9<��5�.F��5S
K�Q�eS}��vYz�B���,��g��:�4x��1�x�M3�}�i�����k
��,�����,��1���v�,kC��R����y�Y����w���i��1%�3��^F?��W[��������[6�I�������������T' s��������c��������,2[Q9]�1�l�0v����OC���"��+�df�7c����|�b<�,��W�J+V�������_2K��Lq�!��[*����/2�X�Pw.}��@�%�K����A|����AkM{�o2mu�&��Y�^�h��/�R����'�Q"�H�E��J/��q��{����T�
��[��x��i=U������Iu��.��GD�u7�Y�����u����F}m�?(�D90�2S��W��\���
��U���`�+�o���O�6w�9������i}T���
�51{r�*������a�k�Kp���yDu�Il�[{��0S�~�����?�ye�9:�<Erfr<u:���;�]���w�n�w��e4���?����v�E�����#P�t��k��R+�����Q���i�������\�24�1E&}}���]�N��=��5����Q7�#_q�y]s}��J�=�0f(����L;�����PK����;�3PK��uJ+patches/0008-Parameterized-path-fixes.patchUX��X��X��;ms�����_����$���kr9��rI������L��h ��P�BP����� A��uN?t��$�D,�}��:K�b�K��F����������|v��(T���t<
�<��i"��B�����9���h�{
h��s=[������B���4�S���Jr�-�H�`:��������s��4������=?=?����`�]/�w����\���|#F���������\���T `���_�x�G%�T$i.�L�b��[%�L��&q�F	M��a������De�%�����MW�.���I �#_��U���%C�0(J�4����2S!@�<ED^�.��T��0Fc������J�Z�a�o&W���N~���������i�&����Kf6�(�E�����oA����6b�Q�*�����7U�y�W�{� �iC��D!����|@�����q���kqH&�D("@Dd��.��dBt�<p�'��eb�-E�T��������|�j��!���7-cy��N!{��k�����W�/���_�&���H3t�D�n%�C��?���$���B�V��#D�X��G`��l���J�G�A�g�*��7C0�eZ�X�f����"$
���@����4V�b&�$��c��5<�b�Bc12q$b��mo�)��!`YH�e���s�[���fu2t�v��{��p�XT0�\lM��aW�`��������T�<.��?���T��DX��8+
��;&�#��eO/>/�L����}<�?�����C���~�������z����t�Gs�Z6DB�H2Q����S�?�M�`1MM�@����������E�/5����6A��"��t�cQ��RMK��1�A�:��*���!�R��\`���G"P�{���x8�P����c������^��L�����@���8��F���(�G�{������G����z��;>���m�>$���������Ujy���Cn������B��x]�J�D��-�������r��O����/��������&?Y���L�����kP0E��,��9q��-<��Ob��~�`�w�~Z�D�l_���p�#��4�{��:������|P��E�h��'���s]`[�8��A�.��U���[�x��b�d/��_S==H9�F��/�]��#�g�/�g��Y��"��B�R4DR�6#�p�^D[F|�~Ao�	��Z��()�7b�T;2A��j>U��%�`�-�<���1���
�
�R]b��>A�'���v8z5Bdp���D.���1���F��l'��|�KT�,�����km]��������	k|����T�������vs�������BR������0<<�~k7L��v�EO6>�=]�����'�{�M�~ kZ���FU��H	F�gi�
%��d�'y:A� �vG�
!^����8��d�&�j�6�����^^�W��T��y-�ns��N@�z�!8#�'���e��NU�!�e5�M�n=�����d��W�n�Z@X�M:�3Za�Y����t�')8�X.�����/�����{4����c�������%
t���`���������I�4�-K���q��|y���Q
�3��]y��P�.�	���>5�����O�����Gb��KB�#�8�?��CVs0>��O1����kb��e�?�
/!�|L�Oz!�2x��@�[.1�H5`�����@�u`"��KZK�>��Z-��X���J���(�~+!�M8��Cn	=!��A�]S�r�w����y��q�DW�"_P�Hh�ju�hO�������R�Q�;H(���y���}���VX&�7�)h�Q��2[%�R�7Q"�X�I�*.�e�:��[i��<�����jE�h�*��P��lR�5h=�Em�T@��K�j^|�n]�Q� P
��|9S��s�����,�
��� ��T2��r�Q��)l~����I�e��.� �.���4���	TBP��q�0/�M�?SsIA��i��K0S�d� W!�c�Rge��C���I��T�z�����::������w0���x��?
��|�����z��<�����r���V�W�[�����(&�N��uDJ�4dj(�`���1
n#	�:�AQ������"ps�H���+��)��
HN���=)Y��4e���>�������R���,����Y&��)���$����3��q��E�hS�b��+L�����]0/Pk����|���6s�N�E�E�����W6c~o����@���v��;t��+E<N��Vx5�qz��LVF��J��[�H�d�"H�el:�:% X=��B^eQ�o�����w���I~���k�r�dY����������I5M���Pk�����Z��-Z�����.w;��	6H�5�_����7N>��e*_f	���o��Z��h�XH�[�=�Y���*��6[$�;�t�N����gJ�#yNO��9
G'��tuL�J�:,�t���	Ut0�;�m9-��b�m����jc��
H��gJ�p���+s�D=��J���qT� �9��{��!��0�5�xS��U��\���I��De���6�!�V�O����6jl��`Y��m���f�����/r{���B���n��+[�t����G��8.��W9g�gl�n��3�_����$#Y��S�>�1C �$ ��X���Ed����X.��X�:.<����R�cH�E�`���Q��Gq�����`|�����\�#����h^w��}DA��Sj�xz�_q)[u��<��������� p4��(���A�j���0���;��.�z%p��b�.�����v������PD@�1d������X�#�;��� �1�2����JuYM+���k�QaLKz��g�mb�����V�T��5��S�
g*_��0�y��RwCs��+�Yg��t\|N�O��S��QI[h:���������[
�G���45t:�h/���gQ�lO9+�:��������t����R��@^�"����������Ek�)Fb����{Uv����=]��
�N*	�N��R���B/���o�CI����!3���,e#%ix�+n&������l2=�}�]
���_�d20��S�'������J�����S�Z����r?�K���m>������I�*��m�28�g=�����x9��2~"1�FZ���X!&���-�K�E�S�������5yU���t�Yj<f�V�iId����Fqw;UkX����V�,n��-=���$��!3�%6
/U�G�{��<�R��*���r[�:[�rv[��J4���dI�Y�M�b(��H�?f�a���ep�B^�*�v�V8.�"E�cC�6%�����oF��w	T:��I\�,������0���;]�7��,���B����]������M�<{y���F[����N���Ud�����9��%2.��-?������	jPsZ����Q����ArS p�HUv`��]��\��m�1��w������Vxud���A�D�9�h���\K�fe�4n�~���	Q���S��;p���Pa���:����+������O��9�u������+(�~2K�hB�S\.���1�2w��g����P�3f,m�
�kFP��b�f�
�m`uT,n���{�����&�_��t'�	w��7R��TOm���5���k�Q����2�Y�%:�Ev�['�O��&L�Pk���r�o"��iw�5��,K���)��S�l@�k�?������_S9Tm@�5U�.)�[�0���������e�*~1����O���5��(]��*�/����kT�������
~�j�ESS������6)������5U)�����Vk�d�ej��vmq���[�>D�DcU*+���H[������\V�'��e]�=��$�����-r��<(.������s��Z����YPw�<���:-�A �if��Vp_�Ky��e����|�����I�����s��Hx+��]:���8'���@ixI��1'"�� �pWOB�U!l�40w���������2^+�� �����o��{w|�v�L�9���z���IoW�C�s��B.AX�O����g�(E(;�(����������s(cH)��%i��W�,���@��F>�v�T�M��Y���l�����O����"g]
���X���0=ak4�����]a�T�Q�M�l5�N�����b.h�2�K�Q�p�^�}��^��g����#�K����^�h���)�*������1�w�\������3B�(��W��X�G���e���~���I�����v��������^nA�9l��.�x�L����9�\�n6�q�}yx���I�L�o�lDP^��B/��{aG�Z^��0������^+�N��}1�g��1������#q-C�b�|��t���J|	�7'n���$�V�#9]KkL{o��Z3Yu
izc���6� &��V�:x0/�����&��}K#�~ s)2�z|�����7���t�j��*U��\�w�D�r�����Mr� ����WT��5 ��M��^S,��5';E���- Io��]�`?������Y+I��@����GA������` �r<mv�H��A#���3R�C/��Q�<�9�5�1>�x�m�N
"����q���lj���%�d���~������;o>=n��,�S��Cz�56wAV���d�lp�y�PK���
�JCPK��uJ,patches/0009-Partition-wise-join-tests.patchUX��X��X��=ks�6��W�U[W#�e["%��d��$v�=������]]�(	���(I������_ $@R��k�I�t7��F�q�-h�?����h��N�������F�c4��c286��9�e��[���#���g�����[��{t�?��������}k������=�����t��������n��h����#������{���zC���]M����{�������3�����C�Z^`���"�������A�u�`�hn;���G�2@��XQ4_z,��&8x��E������<C�p<k�����{X�r�l�!��5��ti�w����o!��R�������?I���0�6�e,W���G'#�'�x��8���<[��%���@{P=|���+��GPq�''[s�*�fC�t�p�������Y��� �]���~{��%��j�s{��BS���r�Q��;��nB�5��s��o�:���D+y��O�S�l���A:����w2�(���|���E�[���h��%���e���ck�0&��~=��������������#`>Ou��G���} w��g(w==��4y&��[�S!�g�O3�Z{�gw����	4����%
��&.*]�B������Kg�py�n�N�������g����m��h�oS���wz����������V�����J|}��_����^�vv��o�A�^�]��1�I��*P���QO�P�C���9������;tqE���g�g?�!�K��1EJ��|w���w���#�kL���{]4<9!��l��������k�����j�������
���'OGN���u�o��o��B�L2���;�����W������-��y��;Vp�?������5&����$�wyM��������eAXjtz�#�D�����o~<���!�T�F������"R�+�+m��E��]z+��@�������W��!t����Y�B���D0Ac����}�a����6k���KH���O�vj��HE������AbcO��d������T�������Tl��6ul�7���8_�J�5X�lpD��uM����k��+5�m���|����wk��~S�n��!��aX{$!���=�/�!{����|1Y��%���'���� f�������EM������Q�)�$�
�hL!v��G^�Q������R��ok�To��7n6��?�zC�����*.�=��t�5xzCLo�$�[>.�<�)��$�t'
bC1��'
��}C�=��M��7?��D{������F}Tm���o�O������Oec���#AY~�Hk��%���l��_�H2��z(�SC
�L��R8��@�d��<j"~ksL�Q5�YL5m��4�4;m�����wz
Y9�a��a�/���b�<f>����<�ca�����i���K���6�gJ<�P���vy����g$��T�����-T1����&p���sJ�R)4NCW�Ei6I.������H���l����z�z�!T2��zT��&�c��c���|���,FO�hM6[6t�������Gt[���?��,�����t���&[5�<��L�0�l�����������g,��1��a<`��%]�0���h��TM0����[��	f2���?6 �~6!}�~_���>��R��]�5��0�'#c�.�}�+���z4�v"��!��b����N
)��	b����kdF��W<�������}�'#�D�~<M���v��<y�f�q��Z"����L�y��IF/`w-�+Y`�C
�*L�I	���G�	UL�k0N�#�tn�6�`z�V~��\�\��S��	�X���Edq-�u��n����u������YP��+�J/����L�_
��~�*�����/M�6�T�+�, �%�	d��O%��M2X�IA�*mb���W��AK4?�Re_	F����T���(Ub�(J�.�TY������[��=>��]t��v�����������>�����qV���zi���m���x�Q��2.�*�^��Fdq2�3H��6o���V��+5��������l��q,�����34�]/�S�J���
�/��GtI].��F�����H�KwE=($�v��k���0�CC���\�.�Le�C��������V�^��6��c��A�o7���>�`+�%)��
����(���	��G���&/9���E�c2��G�-q�hCu/�Q��d"��>����Rw
/E�be��V�����x�A������M�� �J=D�[����\�"(ci�$E~�h�p@:]��%
��6RW (U����c
��_r�u�_h��r�c�1��j�<PKW���+�5��S[���P����bjxy�oe��a%�I&�MUV�����_t�	�>/�7JQo9��s�aw��G�~������������?�	%��������l�&yp��ld��D]���n�7�|&�����~XX"��)�\�-T�T���������H�f����Fv�f������`���u��J�����IGi�SZB_y3��3�D_�O�&��fK�B�h��o.i���-Y�Ih��m�W^�Xk����R��FDFpY��!|b6KP\�����=~�e�����&�����x�����5�_V?P�^AbH��%�XU��o�;t&YF��"�:�_��Eh\��n1YV��B��V��\���r@G�$�k�C�����!�="4��l��	Z����&�H~/�5=������/;:6���b��d�k@���F�=�U<SU�y����BmUG>uA6-�����$&Vk�CY(\BT�
�V
�(�WB^�X�=�����X�!V��]������c�Y�Nb{h�94:pTUFy%�h�y�`�VRY�Q��*�7J�4�u�������X2�0m}�0m@H�q`A����b���

����2�U�JF��^���B22p?�=.��7M� �K��&W�� ���r��R���:tH����b�V��7Lo6�l;
��Izit�#��+���`�8��6���G�Y.a6W�C-
c�%Jj1Z��c���������J&A]�?Y�a�*�J�U�=�	��N�nL�h����G�'�f��5,F�h�m�Q��7�$%H�������|+
��(5A=R�����4�����1A����L�?$9\�.����:+��KVj�N|��Ba9��%��%�%*�
���j�Dq,i�����l�%V���T2U��i���H_�_��<U��R�t��T$�(m(Q�ZQ�#�+J{�����v� m���;b������wT�>	U{%�YTsQx�����o��VZY[v�PA}v�R�[��&�c;��:������Q�
�O����5���$����?kJ�����6�?���l�[��jhK�����6�E�[�N���8��'����[�n����iCR
y �*!
�S�����lR�f��ZX��a[-m+H��R��������B�S]�mf�,VQd�Z�)E���4�1��RH�����,�6�
{V����CE2� �n�=�Q�������{(���_�!moN2���;q{���V�!F��O����������t�H�4�b����!�����V��Evu�C��F���RU��q�E����G \;Uwb��!,y���T��^T�B� �Pp�^�.7��
����E0Jw��5���q��7������Ga[����&�}O12�F���^s����F��
��J��,� �R^��v^W<g )M7�g=���
����
����^7�+�ey�y����������QIi�Bd��9^W��m�����=0�:�Nl)����9��g�^���2��/������J���j�Q|��?ta������@���
��b�E�n�p���7pbU�����U���bM��|=��c��6H�bT��d����V��"9���;�d5��Vb�����6�7�A��.\�ncl����Y�-���m�o���[]e5��m<�a;����(��H��t�����I��+�[w��[v��I_�%��
@E������+K$���x�����(V�3�m�=��c�p�(��Q�PK�V�h+���?��c�p�q��u�CA6i�;X)����Q	h�����XHoB��c� t7����\h���5mw���>E��.�C?��L4��r>�F�������0�����g�@v87���D��:�tF:�!Og����ttrOR3e{"Kg���tf:�)I�'��51������F���2���\f6�)�J���D/�M�u�YU|nJx&������|,��y:hr�_Dw��|���HJO_@y��y��l>,nm���<��%"t��)��hAM�����6��uTM�/ ��JEu%6��7"���U)V���XuP�U��J������P4��2�A�U���z���Xu�bU@�h�U��X�PS���
hPz��T�����
1k�
��~CB����_�Y��
�|�*;�����0`��VSi���&eqM��7,Z��w.J�P��������1�2�QU��C�V�=���� ��Z}�u���)��4���
��M���)�E��TJ�\Iz��v��t��(�"W�W��4nrf��j���]D�<��v��fJ+P�_5�GV���U���P��W�f����=4v;��w;�w;��Q�Z���'�v�xm��/�hi����^�����|4
n��VH����5�������mrn��l�E�8p��|�d#+%2Z�������Z���W��0������/9ld�AFm�����`�@��^Jm���l
@N���D��^��
��b����_�^_!f���t�,�Uk*��~<�l���*�*�)��%0�mg���<[
=�G���~RhK���r[+�y�zB��Eu-<�"6�@�Pn���AL��;���jY+�CS�A�c�S��
��g-0�Y���S����{�;��u�{���C�����I(��r:�T�n�Q�����t[���>5U����Sy����n����Q\J���r��� ���EZR}KG	1+78�9LX�(mt�d��
4;+)�
�f��Q�A������ ��@����R��IN�{CSI�������5�_V?���L��C���c1N"���A,�I��T������PZn�Ho	4.�I�$ ���2�1�$�Z�gI-�,/���"���$�Y�Kc�,N�|�����N�
������(`��v,�f�K��o@���Jv�N���n���d�G�5��C�>v��=Z^`��%��-VN`?:q������#�N���=,h���K�������M,��~=����� ��p����Ogm����=�
H�I�$��yT���
����og������a|��Q��F�����1���ru)��PM5�1��>q�W�g7w���|s����������M~�;����w��t�]�Y��gc��bxr��2v6&����K:OpY�n$�&�Q�tL:�:��t Te��Ig�Hg0��{����O����p�16�;�;��<��=����_�cQxR������*g^uO�A�p��^V�j���Y��a�b��$��.5U
O6�
�G
�OV�}��-��K�3Y-NA����&��&�g;-�gw���-� o���
)�
��tC�
<������!�� ��nJ�n
H7��ZH��d����`��������$n
Nr��AvE�;�.�3��mv�����2?���Z�������F���`_�-���y�|�!�3�y�����@��G������i�'���i;a����?���|yq{����N��D��M�����1Q���f�����	�������0���D7+%K��pF��8�o�]NR�`9��
����%�6��I0"�Q�`D0���` \-�
,��8,rd�!s�p��:�T��X��N���6������N��cvRH�����E1�Ia�z����Np��|i���H�Q`���ha�jf�e����gT~�����M��.�I���g��1%�L�N�lPb�8�u��?0���;RN����B�<�m�S����o����XPZ���m����WX#
�E��T
=����=��T+���G!�!���O$�$�-�$�?"$Ed�HnN���V
Wo��1kL�Q��gCd�X�t�EY[4�: ��w�T%�8"<�Q��F�[�X�
?lRD�q�YP8	jQC,P���2T����n�bq�2���GP�b�xQ��l�F�R�_�^��O�[�V���^g�h�����A/s��%���z/�N#{�2���R���Fa���W������6��M{G���52����Fa�Y���m0NR���Fa��\A���P
{��(lj�1�zK^�#)���Fa�`&\oS�'#)�Qr��ih�\���O������M	�zK^�)���Fa�P\�!�'��v?u;�w
������_�������7��'�<`�������#;U}d���[~�.�������i�O��w���S�q.����������2�/��\�x��^���>����6�4�0ikKGeo��k��Uhq�F��HH�/���TvXK�pp�oF[�K!Vv� &CvI�j,\��h�8ZK�s-uf�q�e<��\`����A6���e�]�u�7���E��5��������>��������n�n�g�/Qp��wt
X:!Bn� D%�8���<gu���y�=��j,�1����������iY�#X�|�t���l~u��f�&�	��,&�@	�&�v	�~��kz!�����!��K��I8P$a��
�b�p
$��	;�`����^��+���+��PdPk`��c0�����������5��e�&Y����g���n�P�Y�=���������K
���F���H��G_�ug������z+�v�1�m�����p��>@
�95SJ�<Yr��&KId��)Xd���#/�w������"3L|"y�'������K��(�7l����-��$_�/_x����*���:8]�����:z��B-�-�`~a#�n�0p���* ,���
h��B?L:���+�v(�a�	��d��]���r��3u��u�z}o�����gfr�r��$_v�i��G�Tf|���4K]���(u�N�����������+���	������Mx�~1�v$�����\������aO��
������wf���"���$$������*����L�uI�+���������vP����^I����q�aU�S	�W�a�U�I-$tl�E�ef�\��o�f'v�N��4��7y�l�^\���
]�k���j=��z �R��d��~�����g��|�W������mM�	gLaA���l)��]�D�/���g�7Y�A��	d3��8�P�C���6%}ozHWE=|O7q������?}�����D!Q�f�l�&������79��c�E�h0h�P*�����V���h��3�#��o���=�&�=eNf�_M��g?���3J1(y:%���(m�l��r��b���L�}�����6~����,��x&���1)��O&94���;}F��)����z\>�D�\L�S�LF>����>��|�+�2����l���X8$���)m���	\Y��������r�;��O�K�L��1)t9c,wf{������r���c�Xo�A�P:frzz_}��<,�O�,D
[�.)==��iD��m2~;�h��$
P�x����<����I���|pL�����������oe�HzV6�W���I��
�~E��V��P0��?L�����^��� f�YJ���930R@5S�\���6A�b9��8C��������>C��>t�4%�\ET��#h��D=��1�6��1�tf�����)�!��ol��X�3��
Yr�L���Z9u���>�0;�M�@����`I�yEww���Y�����b���b���3#]���}�/#p0�������YC�0F����$1� /]
�A(Y�p��_��
�u"}��� �di�&���^�|���x4��\M���}�<<gt4Y*C����&����>i1M�03��`���8��Z��Rq�
����*ct���H����	j
�"3�h>��
�i��g"So\�{B-��1��a�+B���%�%(d�����>y�=����[m�a6���~O����	J�����^�����H
3��s�x	.�\[�p������kn���=P���N�
%��{����o�����-T}��H{!�V]���U)��O>I�u��cm����x��7OiO�i����t3�G���;o�!��JB_����*���R��� ���x4��zk�y�d�<����Gf�,�h��
�#�����4�����@C	�2�P����P��i	6��S'>/6E��1��c��!�HA����������kIb!��.�8��3a�U����k�G<��h�W������*>��if��;m��nO5E��!�B
�3��Z$T�2��/E&���>s�����l�uY4���.��v�d;J�@J��qX��r�")��Nx%��1�C��
�#�b#��d�6 �H4OB!��BG���4���,Z�tY�S�R��5d�M���^�$�����
u��?�� ��K���Cp#U����7��� ��c�}]\�x��L]��IH'C+O'��*I��N-8���]��t��fI:n�[#�#}A-{Y�F&���e��2��L1W�+*��
v�t��t'�����_x�lz����]����`�[���B�,0�d���V%�����[�{�d����:u}w��Wr�����W|��Wy���n����~
���Ww��$���$��Y����7���uX���8z�G���x�C<v�xl�1��<��A��<��(�Wt���;������
������o+���MJJn�F'�V��PK�D���#B�PK��uJ:patches/0010-Multi-level-partitioned-table-expansion.patchUX��X��X��Z[s��~�_���,,`�K����+^��I��TJ5H�Q,$efX�\������c�]V�KOOw_�{-�!4M��6Z��y���t�q[�V������1g=�h�#�����Sp;�h��?8���Q�	���H%r?��)����Z�����"��A��'���K��	\��
n�y��l�ur�8q�Pi���Q�O������{�3���?��(R���<��	�0�y��"�[�b�-�b�"��#_A��}��&���>�"T��#\����3k� ��=�6 �/�"�3	c\���!��$�x^�l�Z�q�"H��{��qPOR�������a*IO5��
}��<�_��vofu�����3�y�KU�Yp)�hC�<��l�d�@�k������F\�?`�3�p�BCH.h��T)W����G�48�b1�}p��P�oi����a�o�k3��l�j�F���51~��6Y~�5�������Gp[�j*��{�����0�(L<
��N����g�$�D��P��yt�*���g@��g��{��1!��w4��E(:�B���|w�������;���~A��wg�xu{���~:���r��<U7�$Va<���
��?�7|�DMm�@�>�c�
B�X���Us���4���C���C�@�SY��1�@���^��%��(��
�a`s
��>J�q��$���N����(�_p� �����4���5dp���u��e��(|��+�����ELX)y��\AIrKK�7��'� &�r�p�BT��ee	*�&'�(@�9��T�D��D�y�J��!�/�m�
b)�]�y�"c�L=�������a�eO�>���Pz'�������P��9m-Z
�����Z���7{�p+
X���o|�
���!B��3G�0V����d��$�NR.�0}�H�H'Y�vF��NLP1ri$������LR�Y$�T/���}J](�I
qh�f*&0��}M�`}�.qX)	t{�
�F��DOZ���\����j�_�����
��&_�c�^8i�)�I�����B����2�sf34��;���F���D�T���{L^��6�r����j*�B�- Og��Sz#�_p�9������#����7�,�` T��h,��%������z�/ "���s"���t�2o��	.��`���
�����UO	�It��0	}$>-��@>�1��7��&USX��jzpCBwI�^��g5(�v�i�1�D�
p�VY)ry����P�	L�8�=��������*�Y�^
v�SZh-C���L�gq^��\�R=&�/���(�}�����@�06C�4Qw���8C��T�",G���j�S,�TM��'
T��5b8� �B�	L*���,�����qQ���S�t�����Jf�g��Qf�*�,�O�4e���39^[F&���,��v��u�Ep��)���mF�9�t�B`��P�T�b�&�)�PgL;"\YI���o~4��W~��As��x����J��Xy�e��������U�����+2���������:�$��Z��W���p}�g�,��K�����'l�AVS��(EI�\
�]aX�!i"eHx��z%jS���|�,����+/��L��V��)/�Qy������@G�����������l`���(z�WZ
��6$�(�e�E�#�k�U����r>�b������90��8h�E����s��`�.��,A��"�g��A���(�u����I�h?	�4�Y�u�4�E6�q�B&>�:�/�e�$��r}�yg��+6�����N���b��^K���0���h�7�������0-�jJ�+���'���G�*���Q������&
�Z�&��v��he�n}o�zq���t�1����lB2�v�Js����.)mNG$���G2r������"go���K"�E�3�V�!����Xqe�o#�{\w���!�:�.
���d3\_�!j`���K"O���_�����%8�6�������h[�����u��nC��)�u����EZu^�������
s9�h���m���������{8��to���.���O�W��t�`�3;�a��0��\m���q~�L�l�4A �����3���f&1���pdJS8���?��vPEL/G�q�l7Z]����J���#\0�['�����W��	����
2����Z`1nD��;U:p���mj�X�=�Rk����^�XAp5���p�o�l���+�\�H������I��w�ZR�p@s����=:�W5�ewAX��|<7%���|���r��n�m��~&���O����u�pj���[�6h��i���:�����]bR7�Ln����E��S2���)
t�l���m��?����i���NC����/��
-kK>�?"����y4��.����B�������:���ws�G�����;����*d���v<hk�6�����E4����PX~��p���6��\T8K=����7�p����?u����[wA�dM�px�d���>=v����9��9
�W�3
���_�����8{i~�>n���Z���f�|i�������cuu��{�adLET{��D*�j���H$>��������9�#���,������2YIvi��Q��
�sEgmo�kn�n�^kCw�A?n^��y�?(V6��~>uo^A������Z�{�f��%^�������o������b������a�P�����{H�Y���V�|���'P�:ONz���]��x�J�A���Yd�.��u+![a�U)����>M�`�]c���2��U�z�.��|]N�=��e���	�//���@���Sc����+���g�b%���U���K���p�}[Rz��Q������}�Zo\�����6@_�X����&�x�%��r >jt�M�}x'����
(Sk�>k��y�����j����!�A�N�E�&@�"o}~�������u<�A�N�����	`w�{��&
�E�Wcw�{���*fG7V4�����u��uj���b��PKPT��%�-PK��uJ:patches/0011-Multi-level-partition-wise-join-support.patchUX��X��X��Z�o�F�,�S���"��V�u��������(
aI�,&�p�8�\����]>�����;�BkE������cg�e�a�gj�g#o��g$\o���r���h:�Nl��p�p-7`O���������/��S8W�4��
~���������ob_I���h}�|.���v�\����m?�O��G������u���n�~�<�y�
�O��op�����F����Q��C��6�CP�f���l��a��Rp%��6��p)���,�A"�@�s_4��-(w%�D�A���a+_�"vW��X��.��|$��R$4�B����|����@�
���������K��Ts��V�������(���.`o����Jbg��$GB,q�XK\��
�"p�]��2V4.�9f�D�Q[m��J�w������pEH���
o�%��=s�$����,�W2t%�JV$�7�T
E��MU�&�q��*�j��VeX�t�~�d[m������b���;T�I�I�5�Oh�	n&}+��7�g�������g�pF��2�"�IC�g����&���#�b�Dy2[bo��U?��!�\�9�gO�6�� �dM&k��1`�M`��W��"��^�c�C%cB�ju�]M����
�v����%�z�' N��-��q�O~��7������c����~2<�]��z=z�����p`�������V�h�	��X����� ��U�K�
D�	��Q�tK�
[d�nB���(�3������?��n��R���&4;���8���>���eW|!���e���q������i�C������^����;})zd��%�/*B��F1��U�$����%�^_/��|q������VA�T�h�Q�n!��G��d�7�g�49r8s,����t>;N%J��4��d��~}
��S����#@do���D���5d;���8x{z�$��$�U�&��x��)��xsM������`%c��8�\��h}Sa�w�����@E��_x������������>H
���j�����X�i�0F9����O�5����g�$:e��nu6>7�;
��P�_�����;p���FCw2p��`]'u����A6$?���[�E�B�S�+���oP=�DpNDC�C����n�����y���D�|�r�x��,t�b�@w����(#�)saa.�v(5�����/�qd��RL��G�����r��a�ia���X�L~�����C
���)���o����|���e�K�f���9�FUy���?�ysj&=���q^��g����rK��,tkA�������YH�=�T�_1�D�/P����2KL3=J@Rh7nb��ih����)�|���"� �@lZ�����&��2I��-�w�&f�6��~SNjp\�D-����J��M��RL&���Zz>^�xQY9&�fAp��
Z����b�������bo��`��*�i���-&��[��
Z��|)��y;;����W�VQ���<x��pQ�D��7�}*����7����s�[S7���X��^�T���`�P�!�Z5�� �2��e�:����W�����
�j�%�0tI�K�Tn�K�=��PD�}e�
30��xh��,�6I6�����������!F~?A1~�|~~��
%������kq�TA|@� ��{t�����b����K�����x��e��$�����#��HVg�t*�e����	9=&��)����|����)3�{B����xG��j<�a���/v���������'����w�E�c���C��J4���S?��O�2�����X���k�i�a
�L�Dz���a����U�\�����,Mg������%�����N1���"T�����c�yN������Q�rZ��m?f����	�\�Bl1J���zg����Y4�tA6%��l���E�<��	J��JM�G��t�����'�������9{l<��yMf�[�	�y����������.iqW��|
���ro��-�
D�$)���v��ufJ��H��(W��G��S�s�w�4����:
P����
V�T���$�����23�#$>�2/��@�T��,��?�_���S���h�1;N�f�f��@:B��������=���^�}|��S�/~6M�0J������_�����P�G��ax�G�`�
�5�Z�dn�ab�i
�W��.iM�O�9<~���+a�
�=�������fC}!��5��[�#q��&�1�`bc���6y)J��d�{��]Z�(@
���������v�,��vW��bs1�)����/����&������Y��h�;�m���������tOy���������:r���
�*��,U�uh�
(��v��y���wEaEWY�����A��
��P1���fS���Z�jRs��{��jneM�3;���`2�5�e���7�*�m�fTM��Uk������bWu���]ee�#����?tj�d���%�[�p�;4�[p��t�H���5�CH��1������k�~?:4r�VZ���E%�Xy���U��z�P^#/��exY�d����*�,����a�P���/q4����C���h���qp�Le�#��9(�y����Z�yXw�GF��aQ=�����9\����"��
�����y�,G_PD�	=\C������lN%d��|$s���[_���n��(�>�ld7�hWa�j��(�O��&��������+���y�F��?�+4q���T�5C(d	��+?5(����hO����&����!��1{��I���
�s�����~�`��6]�}I~5��xm�Y NE�f�^2�����GD�����������q�Q_�o( L�>�RS�������;+]����KZ��Ew��N���J����'%��&�
�LH��*��+M~�����3g��@�IA*�S\�)�D��X/atGoE�H�L���-gx�������?����` R�u:m���.���P�:��j��a�)��j��CW�=�c=m'��0�MC���X���.���n��S�����X��e��%�T:�m��������'���-&}��O�����>q/���B�`2�������G~x�C�����^_P�b]�E@o{�p'��T�2��v�,el[TL���2���eqI���������C���{p���/�@r�����Wl>=h����[�f�?PK?�����.PK��uJ8patches/0012-Multi-level-partition-wise-join-tests.patchUX��X��X��=ks�8���_q�nm&@@��d�n5�Mf�7f	3{g��(c��3��I�����$���d����W
��t�:��9�/�����>���z�i��,�g&F}��� ��.���o�3x�Z�
YCw��9���tk�9���������6}�_S����Y��]��G���z�[����s�p�&t;pA�����wO��gp�1p�v���N,����p��'���.���8��r�#q`m�����e�����W�|��^�Vk�Z5h��'����!���s�{��=g�S{��Al�O�p,�j���	Cp��;�x�	�c:�Y�d�q��?������G\���+zSjV���;Y�����gp�{pk;���\��E�S������7�2`A����F
,�PE��jA��:�^o�M���g�V[����j��>�'e��y���%y����R�������s~��bf�p� �'������A���:�w��@_��S���9���F��1�t$���@x�X�����|wC~`�8��h��E������L���F�v�����IM�o�<�Y�h�����<�L/���kx�L��?��f�)���k>��_�'������
\L����i0����(!!=H�et:2Z�n�Q(�e�wu(���Q����g��R���a	�F\�'.	���\�B)�����#V
C���N!^^��&S�����7����)���W3f�u����;6r�$9�;�$.uW3���W�4�8;kj�M��W��+�r������S4����sr8�@�t�v|�{��{
�O���Z:�v��{������It�{m����#��,�!�`��w �vJk!��M(��yn�V2*�<�����wH�<��J:�th/����C
W�y ����$�yH�?R:��<��Q!D
(�y����q(�<��y ��@r����I��;���)�G (
�^R���M�x��?_
/����h�z|3j����������&=��m�M�k�_���_+����f�a�?MF�������[ 8��c���';0��M��l�%W2������h�PF���3!T�QZoV��Io���.����MX[�?�1h�O��0\��r�����.��\_)V4�7��j�8�z�����k�������>���+uO6;�e7�=�X���(�7s������G
�m�VS��a�����H1Y���F��(�.l����tC�n0�n���L��	���\�s5��r����uS����a(y��`��L��UH9uw���lq����<h���'3�1=G5�|zd�N��<���k>�t}��RZ����|>�`Kh��y,�b�y,��54�C����qZ�8O�;7!=A�Y!C�'�l���	����{{`��o�b4Fc�3��35���	0$�B�������S�p���-a����]mJ�{U���S�M��n�e��C����n��]=y���c�o���|7������'�:�![G��}x@�t
~��z���`�+��Y����z�d��	k�mK��>r�W�������>��"��3���a�����w��aA���� ���aY�!�/{Y�����T-�/{q����� ��5��D=,Q�h�l������_��:Fp��cK�`���$�*j(h���Q�%a�u8h���{A�^�rH�F��41��]Trm�����/I�,��g�G&�?�Tr��KU��pd�+8P}�`�b��^�g�t��)l[��uqrJ�v:�Z�f.E"�?��>g.��J��~^�d������M1Q)c��}So����YhTd�p�WD|�@Ac���q�~�{��-�����Xd����N�
m'�[�����8�������H��D+t�'�`K��s��X�����L�m'l�m'3Lq�y/�_v��R���#,?���w�x�@�����������8+ZA87��nz�s��Sp�����HqS7NKf�{}�9���\�1�
�g`�� ���]�
���>F��`}�����j�pD����i��v�8U�A��������X���4~���U����(����vQ�0���A��V<����3���d��m��I$:����,L��A:(����h�U��2�}%��o��m���p�������I
s��*V���a�'��D>zr�ce1�N���9�e}�C)E�����������!�	"������\
���6JM����������Ti�(� �2S��9�H&	;i�Hj�He�%�h�p��T"�"f��LIjF�Lu��'�#�
�KDR����vp�@q�������]�]��R�|����������V��y��V1�Sw\���TA����vH_hX��T�)r�?��W�[�^��]oR ��D+�E���a�J^2n�e�)�h�wK�S4�������|!��C�Lu�3��p�|��n����9m��)h���5n�x0��9��L�MM��5����E����dxU;����`k�>�o�����X������bz>��&���y�{1J�@p`����v��f��x� -
�{^����F���a_(�]7�y7s[Tx:�l�_��7�������F#�=K�g/w�/��{��T�]����j��+�H�Nm��d�[o���g�����qi"�/q����B:�0�5"�A��cT����������B�X�0�_��[����F�|��Y�MF���.��"���Gq�xp��;3V1����vL������{���=���@�).:��%�)�"�>|~%����
R#fs�9q�bN�/��3�����E�W+�(�vuG�E��pQ��<��f�,6������AD�����nI�w9�����"Sr��+� P� 3�RU�����X�.��J�9�D��8���(
�C��s���=DT�;�|E���}A�,�*�53���u���Ba^��zxx�U-��3z�x���ab%��{��g|!�.�oN��	����P�)�1y�R�Q�V�7.S�����b�`��U���lca�X�?�5�r7K�k��mEQ�lG]������o;|g��{��(
?��#���FOH�W�4z���=�4
>��#
CH��0*���-
#!�����V;d"b�3����q���`�~1;�YJ��d�;��(��H�m�_����$	��.qj�����������  �@��3�yEI�1�O:�(?���xZR����d+%���>�\���'�';%)�<AT�U���������|���I
��a��~F�������\rK\���O��cn<q���G�o�����������8�?]�C���g�fx������1(�������UE�/�-��,����jg�XJeW��U4����@��m�0�f�J�N/�-���}������Eb��q�~>}}A__������/}\��|�����o�G�`����YdTia�~���+5*Xd�Q��[xR��!/e��/-�)�,���p���f�X��EB�:y	�� ��k�T.ms_`��j���gP9���y 0�\����J��H��U��Q�����8��o�����k��TI�KV=C=Y8�,�v�252�tXEEL���P�A����K��e���FL�j���(��,.��O:��B�
esL2�,��X��}����/��.����gq���Kn����.����rP��A��d~p0����D�9�C����|�����?��������Z�;���s��x��K�\�y'k�5�83��'��C`���f/��u��r�n��=�#�^���Q���U;>>��������q��'X�s153y���=�������d=.W�=q��]m����
���:��q���m���z���������%��r������
��!���w���`�Wk���|e��=b��=�/l���P�o���	><8��n���tf�����V��������=�.����E�Z��;�0l�Q����b	�G�1=/J���g��n��a����!���R������t
O�m�����c�\Rb���f\������w>O�#
(j��3=��f�8L5���w��i��Fwn��0
D1������@��E#�*�&���^R����q���o2��*��e���s����%�-�WmI����jxX-�
��	�v�g����>���<�,7���g>N&�N�C�<3�|yi�hi�)��R�����f��)�,�5F�tL-sC~jE8PNH�Bl/h�f2�sA�__�o���I����s%����
t�m�����rzI�dtf5^�8����PXp j>��_�'�������x�K�����I����@pX�f+�����^�|�������(��z\K�5��b=�pI0p}�H�jJ��DF��R2�F�t
���f4���5='��������i��{t����#��#K��>�1?G�z�	��YPl
xx=���_!�;'�$�������5���C�:�����s�����W�,����f���"z����?�}�����C�6L�����F	��\�%�Y�SZ�ot�'�%q�V3*�>�����wH�>��J��th/����C
W�} ����$�}H�?R���>��Q!DV�vh��w�J�$qH�>��} ��@�������@
������l:iv+�~c��E|���_���_��"(��H�<
A2+���������/��O�u�/�#�_)���8���`�]>>���^�Y�cn�dK���v�-���b/�f>H�[��
��Z�=h���Z��PK����PK
��uJ@�Apatches/UX��X��XPK��uJ�C���3@��6patches/0001-Add-IS_JOIN_REL-and-IS_OTHER_REL.patchUX��X��XPK��uJ5�kY�*OG@���	patches/0002-Refactor-adjust_appendrel_attrs-adjust_appendrel_att.patchUX��X��XPK��uJ�[���DG@���patches/0003-Refactor-calc_nestloop_required_outer-and-allow_star.patchUX��X��XPK��uJ)/�R�<-@���%patches/0004-Canonical-partition-scheme.patchUX��X��XPK��uJuv��Z�
G@��q9patches/0005-Treat-dummy-partitioned-relation-with-partitions-as-.patchUX��X��XPK��uJ��5^�5@��P>patches/0006-Partition-wise-join-implementation.patchUX��X��XPK��uJ����;�3G@��Btpatches/0007-Adjust-join-related-to-code-to-accept-child-relation.patchUX��X��XPK��uJ���
�JC+@���patches/0008-Parameterized-path-fixes.patchUX��X��XPK��uJ�D���#B�,@���patches/0009-Partition-wise-join-tests.patchUX��X��XPK��uJPT��%�-:@����patches/0010-Multi-level-partitioned-table-expansion.patchUX��X��XPK��uJ?�����.:@����patches/0011-Multi-level-partition-wise-join-support.patchUX��X��XPK��uJ����8@����patches/0012-Multi-level-partition-wise-join-tests.patchUX��X��XPK

���
#78Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Rajkumar Raghuwanshi (#75)
Re: Partition-wise join for join between (declaratively) partitioned tables

Thanks Rajkumar. Added those in the latest set of patches.

On Tue, Mar 21, 2017 at 3:52 PM, Rajkumar Raghuwanshi
<rajkumar.raghuwanshi@enterprisedb.com> wrote:

On Mon, Mar 20, 2017 at 1:19 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

I have created some test to cover partition wise joins with
postgres_fdw, also verified make check.
patch attached.

Thanks & Regards,
Rajkumar Raghuwanshi
QMG, EnterpriseDB Corporation

--
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

#79Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#76)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Mar 21, 2017 at 7:41 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

On Mon, Mar 20, 2017 at 10:17 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

On a further testing of this patch I find another case when it is
showing regression, the time taken with patch is around 160 secs and
without it is 125 secs.
Another minor thing to note that is planning time is almost twice with
this patch, though I understand that this is for scenarios with really
big 'big data' so this may not be a serious issue in such cases, but
it'd be good if we can keep an eye on this that it doesn't exceed the
computational bounds for a really large number of tables.

Right, planning time would be proportional to the number of partitions
at least in the first version. We may improve upon it later.

Please find the attached .out file to check the output I witnessed and
let me know if anymore information is required
Schema and data was similar to the preciously shared schema with the
addition of more data for this case, parameter settings used were:
work_mem = 1GB
random_page_cost = seq_page_cost = 0.1

this doesn't look good. Why do you set both these costs to the same value?

That's a perfectly reasonable configuration if the data is in memory
on a medium with fast random access, like an SSD.

--
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

#80Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Ashutosh Bapat (#77)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

Here's the set of patches rebased on latest head, which also has a
commit to eliminate scans on partitioned tables. This change has
caused problems with multi-level partitioned tables, that I have not
fixed in this patch set. Also a couple of partition-wise join plans
for single-level partitioned tables have changed to non-partition-wise
joins. I haven't fixed those as well.

I have added a separate patch to fix add_paths_to_append_rel() to
collect partitioned_rels list for join relations. Please let me know
if this looks good. I think it needs to be merged into some other
patch, but I am not sure which. Probably we should just treat it as
another refactoring patch.

On Tue, Mar 21, 2017 at 5:16 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

On Mon, Mar 20, 2017 at 11:33 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Mon, Mar 20, 2017 at 1:19 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

That seems different than what I suggested and I'm not sure what the
reason is for the difference?

The patch adding macros IS_JOIN_REL() and IS_OTHER_REL() and changing
the code to use it will look quite odd by itself. We are not changing
all the instances of RELOPT_JOINREL or RELOPT_OTHER_MEMBER_REL to use
those. There is code which needs to check those kinds, instead of "all
join rels" or "all other rels" resp. So the patch will add those
macros, change only few places to use those macros, which are intended
to be changed while applying partition-wise join support for single
level partitioned table.

Hmm. You might be right, but I'm not convinced.

Ok. changed as per your request in the latest set of patches.

There are some more changes as follows
1. In the earlier patch set the changes related to
calc_nestloop_required_outer() and related functions were spread
across multiple patches. That was unintentional. This patch set has
all those changes in a single patch.

2. Rajkumar reported a crash offlist. When one of the joining
multi-level partitioned relations is empty, an assertion in
try_partition_wise_join() Assert(rel1->part_rels && rel2->part_rels);
failed since it didn't find part_rels for a subpartition. The problem
here is set_append_rel_size() does not call set_rel_size() and hence
set_append_rel_size() if a child is found to be empty, a scenario
described in [1]. It's the later one which sets the part_rels for a
partitioned relation and hence the subpartitions do not get part_rels
since set_append_rel_size() is never called for those. Generally, if a
partitioned relation is found to be empty before we set part_rels, we
may not want to spend time in creating/collecting child RelOptInfos,
since they will be empty anyway. If part_rels isn't present,
part_scheme doesn't make sense. So an empty partitioned table without
any partitions can be treated as unpartitioned. So, I have fixed
set_dummy_rel_pathlist() and mark_dummy_rel(), the functions setting a
relation empty, to reset partition scheme when those conditions are
met. This fix is included as a separate patch. Let me know if this
looks good to you.

3. I am in the process of completing reparameterize_paths_by_child()
by adding all possible paths. I have restructured the function to look
better and have one switch case instead of two. Also added more path
types including ForeignPath, for which I have added a FDW hook, with
documentation, for handling fdw_private. Please let me know if this
looks good to you. I am thinking of similar hook for CustomPath. I
will continue to add more path types to
reparameterize_path_by_child().

I am wondering whether we should bring 0007 he patche adjusting code
to work with child-joins before 0006, partition-wise join. 0006 needs
it, but 0007 doesn't depend upon 0006. Will that be any better?

[1] CAFjFpRcdrdsCRDbBu0J2pxwWbhb_sDWQUTVznBy_4XGr-p3+wA@mail.gmail.com,
subject "Asymmetry between parent and child wrt "false" quals"

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

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

Attachments:

pg_dp_join_patches_v11.zipapplication/zip; name=pg_dp_join_patches_v11.zipDownload
PK
�uJpatches/UXq^�Xq^�X�PK�uJ3patches/0001-Add-IS_JOIN_REL-and-IS_OTHER_REL.patchUXq^�Xq^�X��Yms���,���;w�l��.���I���N��}mg:H�"�TH�r���.J��8��T��X�>��\f�F�r��;�y��������z�=����x�����!����t�m��D6�p���H����'�[.�~���Y��w-/�������7�7a��Ef�����:�vN��n��/�����������@�n�������������������'7�\��G��r���������#O"�����|/���{��db�C.B��I�� �1���1�*R":J���^�~c���|f�p9/�Gq.�4C)�~���(}&":d�9����OQ�H�%^\��(/[�U�8��K=����������q���s����y�a���aZ�>IW"!�m������2,��	��5�4!�$U��,��|�e��B�LP|T�@�s)�3N��H���	�f�,����Mp�@��+�
�(�'R	>�?�p�l��p�Y�L�XD1�1	a��A�%��m��\L��N��[>G]sny ?_��=���c��-��?�u\�3��2���B�y4I���R�:��h�i%h��UZ�
�z�'�1p
PSt#e�&�(���LZ�~�h�=��\=01��(�4'��������0�������,k���F`���^���{�������k0��qs'j�z��������4�C�����$H���&��,Ix&gi*��d73����i@
�1���'~�+AA�����vS�D��T�����
t�L�Z0����a��s�Kw�3�'�	U^�����:��~5NJ�jD��mB��.�������g�������P<��o�����R��nw�d�n�s\��r�"�<Mc��N^���}i��FC��0l}�M)	B�K�Qy`��:V'��Oe����2�)
P�F;���*�K%�5��v	�7jk�]��\�8Qy�x�e��������'s�'�(U5����]������{���)fFY�"]�iiC�>1���jW����-*��T�P1���k����(�����L�7Y\��Y�	���y^���j��-��^��_��T����]h��U�Y&��.��\���o���.��Tdt�Q�^�,������UQ��1HK�,	e������� Hk��K�*�7d/���������&1t������������7I���1�A��Px(8����S��`���|�:�4������������k�I��_%�)�<4����Y���A�a`�t8N���z�5��
����
��=���u3|_	[^�I�r������iw�4���WP��fcEv,�N�����f�9���qG"]d���8g�p�����������r]�xG}���������?K9����8E��;����+�7	�n{<j��I�_�a�,;�,���ep��YcT4�*I�w/��?d���lV�CX�wN�r����@���94��X���?�r/d��K~�8�����HD�g�J�d��Yi0H1��	�/������/����[�>���M���Ok�����J�|���mY�;d�.;`�X��c�X������A���$�3�)���lR�L)�����yn�*p��+��O�������V������zV�^n�*4��!�#=!�K�J���Y����A����>(�4\�����C����O�
��	��0���<��|�#�]�"������JR'�{�*��q6�������'�N�+q�+>�������1"���`�s��z,�l��R��?+�^�(�`t�7E�.��?H��rze��@���Y9kc)x���[����C=�R�T4�8r�Ek��������=H���+�sg��?�>� 9����IB���NuK�Z���[�"�%�
s7���?P��d�<-��f{�J��n�W���Wn��}]k�����G��l����Z����2�i���
��`�xB'������C��^m���P>MXo�=.�q��6]7P���v�/��[�q�2�,������j!u�Bry�Bh�� ����AQ��g4��hH�h�<�6�g����A�z^�`a��D|�x��Y9to�Bq�(�I�U����tlKH��?�4�\��@�"���n���������oo4b��e�n��r��%�2X&��$d�����	-v�@|�q���b�����0�������;�j���m����\����u���.�Cg�d���������7���vK�U�����&�����`���[}��PK�����PK�uJGpatches/0002-Refactor-adjust_appendrel_attrs-adjust_appendrel_att.patchUXq^�Xq^�X��<ks�F���_1��JH��e�]�y����ey�������M���`$${��J$����w���u������t������Ck2p�#{>�����@.C98��N��0�FO�`�����`h�0O�Y��&a�/�����������_� ��&�b��}'\?7~��D��<S��ki0=<y2=��Dw0��[��t�'�.��_����p���J.-'	#a���q��6���V�Dq�c5����x���~�#���q�!�^�.6Vd��g�Zx�2lw��Q����`�H[��HB�DV��o����w `Y����V��q�*��z��<�5���=�����70N����t���(���E�X��z�%+�	�H��0pq.������W��+�f���;|��l����0�����)
=�_��/^��8�pV��.>�^�d3J����H����o�p���������^$k(g�7U��}q��b��8+�]����y	��xW����XI#��",:E&^�����0�b���V22�h�+�_�hM
n�[�X&"\
��x����-id+|�,�&��O2�(�w��[
$l���b��������e����������>��O�o�#��+�-��m� �/�qDK�&�������8��'�F&�#�2�/j/+o.�}�[���@x{A�jO��X@/K4�uk�F��3D9���|.���[{���1N8x�;�;B�?�q"��� �n��qM���[�dD�4���~ld�CSF���V���u�� m�@'���L�\����uei�4e%�����s>:�1����N��X��
n�k��p��eD�jw;����+}�z����
��:n�F��8rW~�P��I���KW������������/^��|>1����D�PGe#�J-b���d�!�=��01����7�z"}p���o���$��Glo�(~Z~E����<5�������\��?#�X[_����t9>�4�	�G*)�D?�M�������xK���=g��H���)+uML��NGS W�t4�n���b��E|��'����N���"����| D���5hp���~����#"��
����+�	��|AD�@E�Z�h
����rqL�������&g%�/d��`�{`��� >aV/�%~�Y e��x��;���`21Au��a����
|�=G����-�Q���\3������^M�J��h��������#���^�c�d=���H�S��uK�VE)�EDYB��s�]�/G�B&\+�[����k������N]w����#��QcK[�u��F������P����	��IW-���#�����[���E��F���	h�6��x^a�S9H%�d�s����T�Jr1f_�w!]���� L��C#(�2�l(S�wlWZ�e�oM'Kw8i��"���/%;5����7<�p?�=G.��]�W�����/A�
,������U��4Q�=�4�b����$�^�%/���<o�X`��4qd?jB���a%��E#v���Z��*�����x.����d�l��eP{�^�l�R�@?_�0Dr���\
��['1�@��?�s�4B���Q���ou�A�kB<�11s�N���Q�\��!W�\KI<|�1E��a�{B��f�n�K/Y[L���L��|(�n�69y}�
��m*�������D�8��W����0R{�\!������?�j=���*5������%3��WH�s��������;N�?�1A-SN�>�h�%9�q�����93N�3�C|C�:�2W���#��U��'=V!P���A0��~�����TKA�y� ���"���a�H
��J��%�r��@��'����C:�����d!X`�=�Q3�x������[?�J+	���HJ�����Q���(`Ej���P:�RH�FqM'�(V�B2��p"����'\a�����G�>��Zb�$��2��{�
�3�IW�R�������q��q}������C�@��Wz3;5�c����=Zo��uoS�������G\�!~�'w �@8/��
�A�� ����2z5.�"i��[�8���>�2O��t�;�J�����0�������8��}\B����������>��2�������^]�E^Q1�����
�K�F[SMa�O7�1J�+�gH����1�T�z�$�#A�~��������P�"%��Y���Xd�3DIZ)���K�LiV�)��%Dw~j���B�H�����4����^��`��+��:A�c]�B1��#K��]en;jE�u����C����"�����~�����o��_�/�^->~z�������Gt�\wP(bF\�j�2�+��:�i�m$���;����� �h3�=�p�R�k����Ml@>4`�%��" ���_���Q@{��.�p��|
q'n���_<���6��2�TbP�TL��0�*�/� �N���;H���A�n�����M��IT5u�$�!��r�)�
2��?C��$�_LL��/SWA!��lY���#01�Zl���E���=N]�P�z�K���U��<�_�id&����~�0/�m���jLn�!(��3|!�A���S^,�@���N�iRg����1m�T)7��{?���>k�����>��'�����$^�`�d��Q�������N�}f�(M_m#�~�7'����WQY���&��8�5����z�v�G�U���8jV�1�%3�U�J	v^�Dx��*����]���\���=�H'!���e
L��Y�����X�C�#Mm�W���C��r���k��o)��0��Q'�hy��m|�����pT����dsk]�0�q%Z���e!��
1�c��YBH��1DJ��&
m��L7#�����5$��������!ik��&k��j(i���rG���ZF�������&���<=Lyp����E~Q@<m��7�[ �����y�
�w��V��5�t���4����remX\X|7�bR�9sR^�>R�C/�� 6�)�B������)�:������9���h0�tOR[���3��L�0R'�:��^�4Cz��g� ==���h�Tad�������QE���TMP*�--�j�S�J���}`y����`��`���4��E_.�&�h�J�4����`3g#���HFukV�b�)W1O��e�&:S����<p�j}�^��\�V������r�����\�XA��\�ZPk�U����*�\�`�Jb���lp��-��b���_��T��k���<�-�T��M��)����?��}SGs=@����/�b�:[�6�����,>�D���7�
����%��}��wu��F�Y3��r nw����A� �����Gx�CG@Tq�{� {T��x������&�"��ED�vu�c��ynv�����]�[%|%X��J�����(^�������-�v�(�t��W��D*=��TY��rP^������6��+j��T<Zy2mCd�y[a�T��������������j�)}�
3��hl�*����Yak_����� ��\��eE�3�����[�,&@��f���
0����q���o���1q���������,����UKsw<�7���X
��yK�F�T'\MY��R�"��J ��D�����
%	��>�X|��
$���G�h�}u��
4C������bO/�v_G��G�{���G�X'�K��

�����]�Q�w��-��0\��X1^���_����4����.���a�|����t��H��a`{��mO�K������zg�0u�e�xI�������'�z����������������x�[���@�����-���jj�f������6kK����i7�$_`M�2\�����z����������������Q_C{���wl9ME��n�.���LJ��U_�]�rwJ��]�*�����\�c����#���Ny��t=�!�)�g����#��a�`��\�C��^�e�
nb����D��k.#)�������MC������3�I5�P��E&��:�nB)���Z����
��+x�A�juK�"6?L.Dj@�������AD_�}<G�8�y�$���$ !�����;�������>1�sKc�^�[��+����Y$&${xN�F?�]���UT�I\��XP��K�Z����:�@���:�P}G$��A������
�����w��]T������ty�����{�U-�+F���>���y�BIi��5c�R����B&N����hs(rk��]�+X�z]��g�}2�N����d���&
�%@���KC��{����\k�`�����E��mE��o
Gu�^�Y�0I`�6
�wCVy���a;�d\39�����jR�����Z3}Ut)�#8p{J-����Y���|�*��^��_��+����\������57�#lE���x	WE��{���&!�����2
akz�V���,#���������w������"7����Y���f�B6��x	[ce6�jd��
���q
����D�N�^�rk!��\&$������v���1����r�m^���)w�����#/\�e���,9��tcJ'(����jT|�:����>c��X"F�y.��=H"4}���\�����E���bg��W0�}�H��T0�y��o�C��E�B$ ����}�6��t��{��8�I@aH&t3�9�q��h<����h�w\J�4h�K��l�Yc��%�"|E���i�����c����eT��Q���z�`q��U��d��3�r��A��4����'����W� ���u+kv��T��98�S�����TLOMl
���~�T�%=F\���#�w�jH�o���~}���3%:����aW�����5����7�	D�
t�5u�/$z�B���l�<�������}��vK����N�nE,��x���~>�M��?�MC������������<��S��_�HT�>$���pN*C��e�`Y	�.n�O�����	�������w�=_�A�i"��Zj�E��X������}G������X0��"�C(���2��QB�P���o��^��YLI��������������}�^��{i.X][��{Q����Y���T���OXsP<��E�]��:�d�q3����ij�PK����*OPK�uJGpatches/0003-Refactor-calc_nestloop_required_outer-and-allow_star.patchUXq^�Xq^�X��W�s�F�l����I,Y�����@i�P�@����,��#�N�;����������Pf��X������{��, ����)K��t6I�y2�����O���i��NY���,�� �C^�L�0����x����z	?����5������pU)�y�Y<�~b�_����_�����.B����
�w��#O��������!�������g,1RA��$.�6��U��_�P<�e���G�����rk�T��%/X�Q�rxx��%���L���VU�
$\&J@��Jn�,9TL���\�7#j��G�Dy�kf�a�p{-���H�9k��S���@�<
��6��DV1�iI�h�Y�E�IUX�I�45Ixez0����@w�>�$�� �����VJ��Z��Z�����
V�����
�
	C���A[������"Y�W�y�(���2x����]4=G�a��f#�E��Kt}�b��}�J��\�2���B����(��yz��)�t.}��R�[Q�Ly#%����8E��u�{�-��E�0���3�L�\c�����X��!�axr4��Ky�����R�e�����m����y�L�'T7OO� X D�h
Q�����������Y����O���h't?���4�����u����YYr�����f�����.A��X^��2�[�;d��0.oP![�������#��]z�T��m-ZF�$���P�[:+����o�-~�����;�tx:��gm������-"���1��1�0�#L����<�0�u'\�Z1TcP)y-RLK-�fm����I�nE����Qj#�ix::��iQl��Q7k�&�wF����t���/���I�t�FV��t;2����g-�
=��#[��[��thC��������O�iekz��=
���[A>�$�S�q�����W������#����<a_��Sgn���9���St4��@������@.��M+d�"hf�t�D�y���5/��m5��N���2���L%t��b���� �A	�����MM-q/G(�t��������`8��(t,���Y���d����X�Z%<�B��'�����
]�BF]w����}��y�ROS��q��v�v�������_x���d�>�Y[O��v��!V|`8� HB��n0o1IW�-��j���j0��d��^`0�w���ad_���5�=K��_A8�N��Kw�K:��������"�{�D$��I �I���R'����it�m��J�9Bq]��Z�&���L�R�K��Y�X8n��|z��J�v�%�0���� `cr���������������fm�M �v�+:���.��v�����4s��D�T��j}����-�Y����5��g��vXr�O�'�����J����vx�I�V%�	��n��Q�������
���T/����������ym�+��A3�@|�:t�c1�b�ku&�@����d�z<�����t�Ku����v��v��\����Ey��`�q���9J�	��8e����K
4/*s�'�+tl����u�-�>x\|������Y��<�&��mC�A!k�>Hf�.
m��-
)�pP����J���&�nZ���+�>7f5k/b�]���FMvr�K���j���
��?���w�'�o�4��}�b6���q����yp�<�_PK�t=�DPK�uJ-patches/0004-Canonical-partition-scheme.patchUXq^�Xq^�X��;ks�8���_���j%����-;����8�&q���~�K� ���H-AZ�%����x|���qbK"��F��
�K�5����L���x2==<�M��/�����/���t����$f_��Mfl<~E���x<��0���\�Y"W�7��{����?�*�L��4�"X�d����g�����f��X0=�:��:<e���t���/�%��������{6>M�_���8�y�6<��,���k1�v�%)�L��]$X$�3K,��p�8N2 ����z�]���%KX����\�
���Q�������l%��k��3�2���O������<d��iY�t��&������$p~�HC�0���4nv:d��D
�4�4��l�Z��4�\Y^�I�� ��������A���l��8���,�o9#�g���i�q|�
E�S��
���R�qMc���2:�
;@@y�����7�iW�*��T��AY�i#Y�K�����w��P��R&~����qz�d�tZ>J`���|������\�#�0�����aw��d����q0J6Y��O�#���(�W9�����	���eyF�M�3X��&c���G�����@��$rd2\1��dM2�l�~������S��)co��v�p�d��wp|���.�7�-��`�����������"����y6�_e�'G��
�e�����4��������p�HI���$�<W��`��e� �Ad:��=��s�e)�g0�Y����~���B�t����;0PqL�����b�J$�7`�79BfB��@%[�-��PuE�U�������9�z��?�Q��b����h�l_c�y�&��*/`]O����A��(��=
s�v�,��~1@F���4��B� &s���f�d���Q�
5dW�UtP��-�'��d��DY��Q������h�q�,p��_}��W���7��aV1��C.�6i���}���GX?�`��1��Q����z���p��8��8�a�w����s��P���]��>��a*�^�Q�N��N��Co:�sb�9�h~z���!��n `���n.?\��_�������������%3x������;CS���?P�T��p������+5�����P�pU�Vh�o!�R��Yp����X<����:�\|��g ��D*�zKJ��-X�qI��1|P�#P�s���<.�u�i@*�	S��?<\�v/Qpa��B������� �%a<T3Q�[uY)�2�>X3������SR7V||���y6L
��4	����%�4����4 �b������q@�]����c�"=�'����j������Z1Pd7�FP�����xz'24&`����GV�	������aHHU�r�|�,�I��8<Gq���}��-�XG�PYp�:��4	DR�P�c!���@������G9 [5��O�LNN������&������U���!>-Iu��
�e-*�3�J�$�aADT;�G����������^��?L
���5u� ��?����E�*���a�t���� ���Z/����J����i: =<�.O����8��'��s�
�]�he*���%��8v��P�30�������@����l��|�C����n��?9_�-���������2����e���q��S����7���e �>�����	�m��`���L6�"
��"I"�q�A0�<���x����p�d����
��~�d������i>����h<�<Es����x��o�T�, �O����@#'���U�@�s�+#�v�~����`.	����	��>� :��;5g�Rn�O�)��F�8��lL�S��1m$\����R�UP��QR�.���A�28�V�]�7
P\���� <��,�N�=�U��}g��.�s���1*x�����y�4�(�s?J���>%��o3O�����79�������4��������I'�1 �c���q���9)Y����<��������s�97(lN��t�qIi�8����
��MP838j�*,�T.p&w�^� h<�0DG@
e�, �S�T��n02R2A�w.�G������D�(w���RO���]����L�F<&���"=��J��6A���
� `�(<����*g}���d��
���5!�WLY�J�C�1���%Fi
��u�cT2i��_e��6h��w����'�0fy�tH� �����J�_P���Oo���on�n��?]�������������D�����f7bm�rr}|Z��`��|��\�h�p�<�
o��A����8����\X�9������B�D�
NU��;UR�J��R:u��a�A��t��c�10�vE��bR+-�\{�xn��ab����U\������U/PX���he�BZ�S�dL��8,,G��]H4���&��RY�]�������b���Cv���#���C��Q�������
*�:�(o���!p�;h�Ac�+�5����0�����B�GC�����@C�^��Z�Y�����1����J��4+w��+�5m�,�����&��f]�)�*���Q����=Kq�)�S]=�m�����G��5����w*L��������6r�\S��������U�0s.[���������p�e0����w
���H���x�b�l?O����r�=���$N�~�:�T������8_/������9�,`/����vI[��8����2,���q!�p��q�8�^�w��T�P������\R�nAA�xC�T��&��8���A��&'�� ���q���%��2_�����V����:e��Xe�X�Za�5��V�.���n�
�����(��d���t����<��$V&J�6�,�l���;�-S�����Tn����S��Q[`�%JK�3���3����rQ�RY~��f�-I���X��M�l��������c�C*`���F�!���)�����J�)P������NT�`|G�����0P�{��,>d�
h�������F��NF�r���,{[J<����:��L�8;��&��H�[���Um�%��6t��;�]Pp�X��u�U\y�7Z�3��[�~���b������b�����_������e���%��B*�'�l"����>���3��J���ez���jum
������M+�y��B��V��e�y�b!�� �/�*I��2t[*���K�@�b������I��AL9�,#n���w!D'tusD��%��@��[[�fp+_�e��dI�~��p��AW*v��z#�!���I]<j:��M�UR�Z�-:�v�,n�@�����T���*�R�'��4��+���b�"w�G������	�����|sO(�	��0Iz�=����i����� ��4���8�zVSP����)=�Mc��W��vK�va��&{�<�x`;�J�'���]���I�gY�I��@t���B��~��R!��!�X�c+�����A��[���]��������F����@
��{V�����y�o�-����&���)��/�2�s���7M���Js��$	��������")-���Y�l���X���LE1��������3{$@*�te�a[�����=���`���2�O����4C���T
�K������}��dz\�F���E��]�^�P��^�B����k���Oh�{=|��w�c�i���k%��7���� V���u�3��L��/�:cz~�;���V;�u�^L�������$8�^���(:��S�vDwE�Lg>@X�?����=6���7�}8������n$��.�B���MA�QT��R��mCR(���D������b���z�X�J%wPQ��rA�BF�j_D<�tze��1P���I����d�Z���-ZC�b�]�nC-u��j���eQ ������P]��J�������Uw��d���Q_���b�L�#���2�y�_��z}�w'��w"��#0�4- ��c���LN�������������Js��k��e����-b`%�����YB���jS�2�����y�����Sa��6���[�t���^b����
,]��G��&�DP��.:�����������p%,��;�R �b�����(����K�|�<�{�,R�3$�0R]�Ec��!	�jQA����K��c�3b��Ng�B�7�L���?�1��//C��o��O�)B!�Ew�$����d������{��_��������-|����4���
�v�f���R;�TiP��)�T�Ex|8�N��*���������m�����PQc���5�6����~��8V�i,lYh2��J*H����J�6��GU<+�l�@�_�`A�T�zh�>=p�c���0�*���������Z��l>
���Q�Wyx�nD���.+iy���)z����K�N���:�����{��gu�����^P�����5MV*�3`����6I�E������AYi������-G/\Ws{�\u��
�Z��u m#h&"����Lp�hv�������#��ytz���b���r.hT���.k[��*�.�B%i.9Rd�I����Y�)X��t:���ks��Bt�����
zL'��'0@�X�#�$V��'��
�����2��U2~�+iu�E�2�C1*��N�J9)zH[���RY���jd.'T�3,O�J��R&�8��i� u��!q�Th��;�_���������F�;�d��|hp�~ �K��y�ik�����E�_�O�������u'���tx���?PK��D �<PK�uJGpatches/0005-In-add_paths_to_append_rel-get-partitioned_rels-for-.patchUXq^�Xq^�X��Vas�F��~�N:��AB��3!��:�!|�t4'�d)'��H�������1����j�����w��*��������.���y.��� a�����x���h���������u��z��Y��&2��B����L�/�~wBz���UYe������;���6�=>�
���;��g�>�]��Z�6�g�3��j��������L���S�"`e�ET<o���+(Y�2��k����,�\d�,��D����
�}��ay&��lA�83�0
g��o *D��
�Z{����J��&s�$��(�X�m[ ����OL�[�*[g��K�uY��,��o������L���F���x7|�2P���M���"��EP�
�C��\B�2q��<�����_����1��y�[�gI�}�)`����ge"�� ����>s�a21/�u��q}���q��o����{n���5������s&�N��PX�|Q������K,��|�W��0�]�B�%q�
��P�
*�V�n�:��T*H���&�����2���W��%�D��O���a9��m6\M���j��O�����i�j��	9��Y\����OS|lav#z����C�]s�f���4d���3ltO`�-Dw{s���bc`S��t)|"����A���J��g������h���k����C^��|b�9�P����U��4#��:�u1��#�|i�&�@F)_sM��������<���	���{�d��c�B��f��1�%�5_���2�	��K�������@��%7GI�CY��F<p��	��$�(L�0����@u��+����GT%���*��G�i�G�����jS	�Tnr�KpKM����58b����Xj*H���^��
4K����������8
��L���
�m�[��)k�{��!�������:�,D�E����:�FDzk��K������4����H�	��R�'[���v�����mC�����m�����[n�	���+7��]e���Z������4qh�qRF�'Y%Q!v�.\� ��i�f��Ff�Z��k��-���m�Zm�a�i��;[y����ji��U�L�{D���Z�hJb���$�{S0��������q���������g�|�
9�����C^Z��a��0�oxJ0�VVx��xY�2(�/G#>#����	!��',�\o�&!�l$d�����U[P�����B�k�E0���
��H�g��N�'�a������|>]����sF���[��PK�y}�q
PK�uJGpatches/0006-Treat-dummy-partitioned-relation-with-partitions-as-.patchUXq^�Xq^�X��V[o�6~�~�Y_f��,���xk��]�IW��0�AQG�D*�a?~�P��tmj�q������;����@Fi�">�y�`z%�H��,�g��I9����IgF�V�!��I��{2��c[4����J8�Yt�0��k���V�4�f�*x'.���!$1������8Y$��t�h:���&���-��O��o?@4��?��Fr�5��T�v�)�1�K�?�N�bscA��%���UW����Z������E
)��F�C���s,�������rv����
�R���p��6�RX�R1h�����Mk���
uN���INZ��q�tF�����v0��:�O�����|����l�t�$�ae�+��h���8������Ra=��cQ���P��@�Ct�g�>�)����Z	��D���!�S����W�7�A��F�k*��\��\@%�{������0�E*f	BE�����Jfg��_�(�O�3�5���������RT���mR���*�`���!|��y����~aM��G�.�YR�^xl���P�,CB;RTL��;����0m`��k=���N.�����~_e?����_N	��6���`����)����^�T�7���
��;�J�J��L�G��0�=.z������U|�.�n������Y�R4�yL�PNh���Zc��CA�4��H�����5*�01�[LL�&(��/���Yr�J�BoS��B�-�W��>����9���F}����u��0�3H2�����Dj���9����~X��Xm��7���Qr|uJ���{"�|D=�B�u��k
l(���B�w�>g���#bk=K�������4"�S9=��w[��;"�d=�����-����+  H�[v'<�q�~2�
��q��
&����� 
���.��la�7G����J&��F	�F~��r�CK�/v��+��:��Z�>y1���-���C�%�ut/[�{���Q�D-
�81����O���|A�tM�[��'����H	� C����5�w���|	���)�G��p�����_�[#�K��]wQ4���mS���qA���p�PK���\�
PK�uJ5patches/0007-Partition-wise-join-implementation.patchUXq^�Xq^�X��}iw���g�W��@W��
%Q�r�]�NN���`�'`��i&���:��J��O"����[uu��*��txq6����������=���h ����H��OF�y_�����\���������a�?h�0����f]f��x��R|��{��OrY�|�'��Nzq�x�z��;�*O��]�4�e0������Xt�'�~��z�w����>^^�x-�gG����c��I�d��;)��%K�,V�\@G�Z�7��X�������N��(��<�"��2�-�T��$�E�u}#�T�Q�"���&)D)��e��Z�8BPv��
���d��C�G��._���]��,��B.�C��`���1�/"����b|��S�*"L�[%��)�a�\�.IS����ky�����oz�^R��)6)�b��U���	T����l�9��`������9��K�v��By#[���;��),WU�Y)��+�F��3�2�j%a���X����3*p�_ei��!��,xL��W�����^N��G�6�Nk�?��7���M��P�����;*r��|���r��{������p!:�����(�"���������Mc#���Q���������;r���n�g�����0��NqI��dM:���F�L�H����H+�5��������� ������LR��Nw�<]n�F�������699Q#��fk{�e���B(��")���:Vp\z��2N�Sy��3#�M��6v���6!�M�z��M�77)�n����&��8M��~��2�M���J��2GLs����/nyW��u����a��@T��a��s9���� H�#X���9����T����Vk��f��p�I;j����V���_�dtv19=������t2�~���I��-@�M���'q8:�wOE�~O|�|�7�=, .;�h"N��x��1ZH�H��-��qq���?r�=�`������R$���K]��r��5����w?�d�t���\"�C�D�
�/~��,�&2}����������3_|�X<�n�0�g�Mh�TG��/�W��$�RA�u` �FX�I||�"�����3�{����2�����c:;�����km�����,J.�I����L
�:l�r)�G������haa��'�z�O }�Z`����;�Q�.����V�����7f#`i
���z�X�T���x<��hD�8,����>S#/��7�������]�%y�h$����&�"U��L���;\�	\����r`�Y���[a�K�44�������Q���4���G:���&U �c�,��{W�<dq�n'xt;�Ho:�����7t\u�3C-�GTU�~,5��x�x#/6�X�h�,�������R�/F3�&o�tyc�����i�,��G�<��+���+��
.mD/fea��2�`
�9�12�Pk���%��#���H\(��+�1�8���=�����3pw��7�[��,��D�~�g	��;�6]����BDq��M���i�~��}� 2������{:��H3l������N���\���Z�K_r+�2��I�g&t��l6�3��w@!��������I.(p013�*�����'(E<���E8�XS��A��a������ec��N�e�e�1�*	�q�}�������B��<A���^�����b��6�����s �����v����`x��p���z20��O�Xp�b8���0+c�4��L��6l_h�x�L��e�����B-G��������������u��&y})�b��*A!c`�
q�b�	+HPf8�:�b+*�-����	\;_���B����E��x\�	&�����,������w������O�=4�#n��L�nA�R�p���c�50��#Z�<[�4 �G��e(J���IN���P,�����4���Gf�`w`%#�5��@V�&/�SX:�K�S�B���[�(�CF���@���{"���e�a������Dq����"�a��=2F���c��)���/F��U�7	.p�~�X��-��u�u����$����r�a
���Y���,��,4/V���#B��Kov���i�*�4�M����������a��R���S��E�ex��$�{�JK�A��b?��L���j>�C�!���7���WJ�|�d�<�M��U��[�Z�h���7b�b�l���`)���)�{,>|�~����^}��z��4���qy���h��X!�1*&��1l�imuf!��<����"�E������,Po�&%'|;|���m���O����0�����0�wW����v�Ddh}��u��'�k��j�k���S���E2�W�,�N�VF�-1����"ja���_=�BA���%q���-�PhV����L ��Z��Z34K�'������4 .� ^���.��p�����KfP1���������1�4����K�1�7!uGdM8Tm���Y���1c�@[��1*
b��X.�GH��6��q�)Y��Kq�������x��BG��F���`������%�Gk�q�E�P��!{
_�Ia�:)��q��$GK@�Z���l�d��d��GXlx��Um�X�g�]}"OE��	��i!w�0Y�8[�����������
�k��������U�s�{�e��'y~o/*�e�t����h(Uv���@����
�!	_E4�&<�h(5��9��I�*�B�-j"�:%�kX�mp
���D�?�K�A���!LU�ZA���'S�g����|��Rg��/��	2	������x��vha� �)}�u��#���WDp���K��6��]>�EX�3N���YEI��&H�YJB�2�o��y[Z�0D��z�|.����-�"Ka7��7�(�&_����'�?��G���qC;���=E|�y|��V�q�OD���	U��S���� 5?z,>�%���V2Y��"R�J�p�
�(�9����y�z����k0��z������Xy���j�;�0w���/Q.��?OI>/��6 t�
�������J�Um�?��3(���b�Lc������nY��j�z]V�55�(Y'���5
�{�����o��6������9#��B��g���E}�7/�m������p���]�������kD��X��$����:5��U>�}�>*w�'[�S�Jq�a<y���~R�a��4���(�1m T\h����\p�M���lVP��spC 6�{B���������9�B�+*,�����}5���a��)
���P�"��gXA ����B|�pu��J�T*�C���B.V�}�!��Xe���E�D��)8R��f�W>��
H�\.������pJRQ2 bSm5"�D��u
�D���U�w��J#JsM����	o�7�`���a�H�H�Tv���?����|<���Z>��)���������9 j���-j�'�-Mt��~Z��(�xN��1	��������9D�[	�
��s���1�6Nk��z��	d�+��r��FZD�;�7
&�����}{����"��X�e�J����l�eLgy�pa�i-`���R��i��n��Y����#�.���a���h_�@�P��PU����]�PV���'J�A�d����2JFd�/<��������u����x�	����uV0��-{�*��vwb��3Rw�O���Jr��48a��i�{��9M�����e�G3E���^�����T�(����I��@�+���,���A2���^:z>�P�������1o�>:���M�J'�P2��u�HO������c4��]��{�H�,T��%�>���prY�����k���J�Z��Q��=o�s��^4��RU�)��h��� Q���]����j�����PKG��Mt���g� {�a���p`]lA��d~S�'���Ob1�w����5�O��5:�����2�AR��H����W�-`��[�c%���n���%pY���E�����/�nd�X�{����M�~
�8C������$�����?x�����$P0bH��2��k�H���*����0]/@����@|���gU�#~�M�8^�����6����&U���@���B�0W�.�pE$�h���@)E4GCS��3�"`R����5���y�r�$���:���_�a<�+�T/f5_�,�����b3:���c�v$��O����=0��
�X��������,� |*B�%$��^�� r��Ry�]Y��W  HF����=Go�����@).3�Ik�z����K�
�h5��5YVY����4$3_=�)S����+���z�z 8��������j#;9�S�{^����,����U����1����w�K�����VT�qGS�b\f�diW�+��a�$��r)��2H�G������Z����?~�������E��{����S��h0��d�7������E�V��S�D�!
��qF�PY���)���7 ��D_<�b� L���w���5���5:^���5g���T�������-�����q���Z��v>����������d8�y�H[����j2��R
_�3�������y��_�<"����Pq�+m�pL�;i�fW�]<�����
�=�'�o�$�?�As�����G�&t{~^�X�8*�UY��Z�U����Rs��w){�!���$_UE;�$�z��uk���(�k��@�L�D��T6�#C�K�g`�����G�%��d�f94X!�����|�Q��)�Z��vk���h����l�i��i�j�����7����y�<��OL%��8Eay�*�IX{z� �������V��Bk���T���r���Y��,H��hISE�W������j3	"&3��N��	U>��l��C� PO�Z�jW8���Zz�:��`����b��ovo���=��K���>;
}n�P�W��"[���#BC(��*�p��oA�����fU�1j6k
P�
O�zg��Z�^:���1<w�(��hJ>�,R���/
<T$�����[�G_�E_����h��I�[�������tOte������z��>;�O�Gg�����`8z��� �te��D���@�����[���G
��1H���]�:U�*����`3"�U�,[�Do�5V��g*��s�X��(�������~��`}���;���X����U'd�'�O��<�TjR�BQ����+e�4����E����n!���
0I��\�����=mdgz�M�K�����s����nO�`��s���h8�l�|��W���;k���J�������z��{��E+��9������CB�n�����q��~m��9S��y�u�����2F�Y�����M�}��c,"�t��F�+���^���6��&���{f^)@���Y��Z
soT@(��j����u�K�g&�������> Q�����#u��z��T�������H� dF,eo<�x[xoJ#L��$�,HoHa��9�U����
�*���0$#�����������r���Zo�A(���w4� �;!�t��z9�N(�jx~�����<�X5�(��w�ZS@z+e�@umQh��/��������8&�V�e2��cO@Vu�U���`��y��7>0=��e8����s�#�U�R�"�d0!?�����X*�����X��u�DS��p��C��*��w������vU?�Z�k8~�'�6:1 �*m%��M��^H�Y����V��n����"��A��W�$����l?����s�v��L<q�x�����9/A6Am���Ym�v�t��&v��,�0�+�j�.��a�J��&m�G�\��/����*3{{�E���e��|�����t_��-`?s��F�<�k�0VN}N�V�F�Z�.� p$���N�8r��ix:�%�d$�����2��-���*G�6�p�
���t��M�N��R�� �AKA�'��V�����t�������X�T(P0(K�����G8��VGQ�(@#>�*8et��T����`@�+�����Ch�,�Kf�(Vv�:j���b�������L�Bh#j3
�T���i�������h�)����[�#���Q��4>B����
+8Z�jKm�
��II���d�u��w�YD�#���(S��V:��o�>!�����������b�����)��y	�3F:P���})�����5���4����k&����v�]����r�$q0�6����Wz�ZAo��8�]P�TuFP��k��)�1k����]��r�gBS1�0?���K�h7�D����9T_���` ������
��������{�1.��i4T����z����O��-���S"#��+�`$G*�RQ�1(��)��Y��]�$����4���kWHf:����IS����e$
�<*�o��D�A�����81�cl'�J����:0��*�L��V�NN�Xg1>B��
�@���9��v�>��NI� ��c��xFSX�R�xS�����E��V��U����
� �*/F�^�LP�WP��Bg�����2��T���!7$v�]����H�e�Vu�pS�e|T���9�(*�����
�����w'Q���G�T�P�^��\966���K��*o 5^	��jg>�HR���Y_���.+_�A)���,��u�@i��J�N����J��k44f8��fi�*l��\�+�U��{�/�J�_*���7)��Z[}A�e3�����`<��c[wZ#��g�+�j,x��nb�u�:���Weg��%PCu�z3F
JKQg��31oU��|�;��[�Y��p@]V���Z1)��B�<��l��y��?��A����"�c��2���#��;\���d�Ri5�����F�TT��Q6B�7�������m��L���W��&1��K��������4)8��S���|����t��j
(��/�(M�{K�)��K�N��x�fy-��)NX�\[��,Z$��)8"�M�*�a�+=}�%�%�V�N�
�m���N�� ����8n}��� ���E=;4��5���X�o ����v�h�`�S
���
5��gQ>/��G��uX�B����$v���6&w��`�<�h��<U3�L�M�bR\}�������Z��p�����=?O���a�FC^�_�\����B=��QV�2�*4����2x�Cj��ET���T�l�����V�/��H��FD�n��q�&t�U���2k��,Q���z�:�o�1y!1������r\��J���	���?:Qe�����3��#Z�U���X�[������0��}�7��2a���/�*��	
�R�O��������spWdq����������y55RM�"�,�+�%&�BZ���F��m��MjE�^(%�?��F�^�T��UW}Tu�K�~�_��@��E��-!�.�p�������Z����l��T$v3d
$<w�Fv��@{3���8�-}c���GE�r���|?�7����6r�{�x#@o������>��5������T@�#����u����Em�����H!����1�G�c�j�Fw�JE��w2<;�{��d"g�h'/�*��nz���f6<%_�yL��M��9�h��h	��t+�r9�	+�=��Md]a��(�Vsb��6���
��+��	F���!�&���=�f����E�t�����c[�\s+�meN�5m�vAoP��aG���.���[�LK��73���H��*Z4�J�F��W��9EP���������PBj�?��r����M<w�|zc���['��~�~��v�5�F;�0BUk�Xo>4F��qi��S������uQ��<	��&��v��Ww�$e�'~.������������e����"��t�q�B��Et�j�(�96-Vps�Qt������VY7�'EnN(�)���q�J(	�V���!9��H
�2(���E_ug5Lf��"�7�X_���Gw[��C���8�5��e���R0V������,M�b�����g�QMz��"�F�].�*�Mwq�.]����/*9N��;=\
X�I���[����B �`#������cA��%`@�?>-���@*�y��v6o�[Ua���y4:?��d�t�;���mG�:�A����k�n���6�l�oe`�8����71���o������59�+k����V��B�"$��\�$�Y���o�+�a�G�pzm�#��WB���*k�3�_#��r(>�V���8k�l4��T���5(��F�h�Z��'���\ �|z�=��������d�0�����u~1�����R��Y����04�����kv~T0([�'.1	�z�����R�l����_�}��I�r/a	#�u���y0�%��#z<��n��u-�Z�\Ve8���A� j6�(Y��j}[�6���HftLy�����zw)����*��Mq��������n���
?��9(���!Z�/&\�j)�Z�����$��e)3nm��w.����jVcZap�o�B^] O��SZo7<Zg�d��Ic�^�_�`�Te���F�E+TY�������=�����}u��w�����qN���JU��AEYDA<�B�.���a|�\����}R�`���v����d#��P)��a�j��x2M@��A�yn^2���j:����tz:��f�T��bg��������/?%�|3?�����������+�8sf�n8���|O�[;91��L� �R&�a��x�!�p��kl����a�W��O-0�W ��A���t�ky��laU����\N���U�#Y�����U�����
�0}����4Ev�|g*a�����xt�[dG����,W��cV���-6@����$5�i3������aJ��J[)9S�qh&���$�4>����RaE�1j�����$���q#\r���\��p��������������"Z�p
:Q���9g�L4x���������W�����\P��)y���������`t���V�D9��P�B������X���{�Llc��%��m��j�l���ja��F{d������?2��O~�Cu0�y�Cp���#���k�_�p&�y��_��)�]V���.��R����9�<�W�����('B��k@t�,$5��G�i%!�}S\��	5]<���-*��L��I�JQt�91���G���q�Eg�G�O��u��$F���gQ����m��r+:��/m�q)n����-h�:�
�I@�W.��=��G@���C,���(d����A���a��.0�p���d�_�n��%�����TjW�L�6,���	<:����gO����I<H
�����%����
b��s�9����h�(�v4���JBC���<�[����c3-�}����H�*&�
�2R�tS�����g�%mX�f�r�VISID������zR�������b��
��jk�Y�^o��V�+R��
�_SFN�W������������&l*Qy�R��p}E��j��`����Z��`Gm�TL��i��������-�A($6��\�Q��V�L��&�ZQ���!uE(�1
��q��dn�������PO,�lz���"4H(������
3�XY20�JES��\�UD�U�yv`��s��j�2�F���+[A:Rn��=j=��a��Ra�\�/�3 ��`2���N,������������oW
Q���y�t����\g�uu��A7��F~��QrWW;�tTv!��
�z5$��_�K������o��j�%"��z��g�E1� ��u�|c��w�Q���m���J�?U�F�g�(GKj|����PF�Z�@��w1r��{7�OUw�y���v�up���q�F��N��p�Mg�;�u�&�uPp.Z��bL�P����/�����.���������^C�vk������:�q�U�}�%��Q���r9j��X��jX,�������q#�u��*Es:/��a��]��2�����DF��F�[�FZ�/��d���P�0X����8��%+��Rk���A���T�'���sv~aE2I|D�Nz��{���O�@�����`�;�*t*u����!�+����KK���*J��M9�:Y �Y��A��7L�qk����Y��Xy7�%�	q�xme:Tv`��NO������F��3_����`����;(:M2y�i��>��2F�pZ�2�~��^������1�Gs^5:n�:D9u`���MS@r2s�?Ay��F�K��:�!8�T~���(��<�G���{���Y���n$�\�mF�"~��o�'�Z�����m��Yj7M }Y���%,���k��^ne��?B��\j�����LL�E�%�p�R����=,���U����(>3��J��
��J��c2B5pHl�����G�Bq6\���I<z%�Y3#M��1��/��]���b[b�*����	�y;_c�7�>$;����@�X���������#t^�u��u�p�������jB�����OY+�fy����������BW�>}������3e��t����������fm8�T�:�#�qN��]������z�����GX��%���(ti��������2q���A�m����E�K�[��bu5e�����.	w�TP���t���yF��z�����
$��+t�6�H~5���� ;�����Q��Q<�����e,���s�cu��uiH�A����us��V�E:O�������|b���>R�����z��������}��������|���k�e����<���U���kT��f8gB��l�5�3��Q��4�M��67��7[/+�<��Mr����_��:��P%m�wk0�BQ�@������v�)&ks�5��	�h��:B�u���f��~:����p5����[�X���A��M��>(�m��V�Pt����WM��i�����T�us�jD�iR�"�K�����q���
���2yi����n�q����r8�,�
����u����F���>���?\���1eg������sk�N5��vnM���v���5eg�����%���X�fI,UV@����V�a�^�g�r�E��~�8�x�����o������Ja��>z�����n]�=�B T�I~�Y���q����%����	\�M��y��<�E���_�A/�j���)}��,`��qu�NE��T�)����z{�r�6lZ�p�l��P�@��5cT��8��<,np�o����T��j����Y�V-����(G�GO{�����]6<��x}�0L7d�+��j��wA���e/�z��T�
@����B�[oR�;������}��zu��*%���@��	�T��*��xNfI�����0Kz����
�
E�1CK���k�����0m3I�������Ay{*W����3�!p�;P��!�	��eh��mg�m�\|\���E/�l"w:1����?3V���a��/��=�w�L03��d�1Z��B��>����_�����-�V%�u�^�=�c���!1��(]+
��(��R�2g�F�H��P��8ZZ��#�Ih$�G�=d���2�k��
*;y��WF'%3H���5�X�V�'���V/U��Nb2�T��T2��9C0��lql����UE�6���3�X����@V#�'N�hR������1�B����|�b)��*��x&'����;�G����z�D�
�����gkQW,�,��)��}���p����O�(&cn"R��i�
�/j�z;���,,�w��^��|����uW���W��6������?��]]����7n.KtQ/�l���Z�����n�N(d���1P99��o��t�������/��q�p�?�u�PG���R����bVs>=�N{��(�F�_G�F%� J��M�������Jo�>�||B�>L�jy��(j�zB�`�(tF&W��J{9��N�����d���;��#|��9��BL���b,��;Y������_%�:ML���<�m6�7���S����T���0�4���<|���cNY�r���T�B����K��M���"�Y��g�<Z,(�Z:N��Y��E�L�^��H[�q�oT�!a�_�V2�w�<���j'
�E��O�'�LFV����9=����,J�B��5�����\���;G��%����w��1,d�71>�g�v����F�0���9=;w�W�%�L#mX%������������B~w��9;1#�9(;M~��Q�5�^^]��_[�wg$d �]=Oi�J�6��Hj�Wg3���~4����������s1����b�a���m����u���9�z��&�u���g������q�z�5�����f*��+8r]R�J|��������wW2���O�c��������@ss��o(���5�j�W4+7N�X�j���b2�z����b::k��j��Z�
�=�^�W����T�����^�~~<\����K���UT�#�n���p��,�&��Pu��Ch�4Z������Q[}\"�z���^6_I���*j�d<�������$��e#�M��T��V����&VBN�Et<;���pjjyS�{�3{��nU��w�DjB�v�k����������z�	�C|��~������k$.;�d�!�����<;�{��h09��]�����'t����s��rk�	�wY�E�uK�:��2���&��&�E�Q��
��a6��\D��_����FTD#��p0������F�4���(�Hfxu�M���������r�':���� �����b2��P;�<:�^C�U���)��Mo+7��zP�B���i���<[���m#|a�9�n�G>��R���/-wjc�[5�O���Eh.A(�-9"���������8������q����\��s��}�F������n�s���]�%��1cv�d��%������P��v�����?�_���mwKC�&e�!!n&uSC���M������o_y������H@�
�b)����.��t�*�`#�q��G����#R�!,cz��0,���9*��)�I�����C����a��_���
_����V��=�sA���wo���u,�t���J��G���f�H;��m"���4�$�+�������s@�x:���Fu���E�*�����p :����
�JR��e���x�� g_�Kkx�K\��]�_x�4���+���Mp��7'I��[���Y������#8�PC�(��yk�5�a-#4y������;R�a%�/>�����I�P���X�������c~j��u���adK[����2�4����]-#@����e�L�R���L#����i��B�
����V��w9k���p�����!���
n�?o��M�4�Ru�:��r:��7,T����v�B5�-l���R���`
oSC5��
��Xc6�U|�1!T���F{0RX!��H��)���.�?�%L�Y���|�&�;� BG�de���0�Q�,P�|�	4$�(�LlX%M�T�%��Q�=G�M��@&Dk�;�]�NZ��PK<l��x5^�PK�uJGpatches/0008-Adjust-join-related-to-code-to-accept-child-relation.patchUXq^�Xq^�X��[Ys�F�~~E�<�"E���I�b{������LRuk
��A"�iMr��������v�tU.Q�O��;[w��4^��d8��a��f��������h<��n���'}g0z��xG�^&b0��5�C3~�k�2[y�-�+;�s�[7����r�&i�I�1�x������k�P������b =�^&������/��}��&��Z���������7���x��Vd��-"���<����=I����$�"=�>���4��/A��|}&O�b�E�r!�$Xps.Qu$O��)Af@,w��Qn��N����|�<�]���,�KL�s�.���Tr@�
���1~�S��e����XIQ�f�D������E6�4P\I(`	���X.����D AX`P��r��q�<��$;o)��_D.Q/�fh����v*|����AaG^�+����,�)�W�$!}TL�� _���Kz�]4,	#��~�eR���!���������� ����Rk�����t�-�E�$��T�7��S")�
~�2����������Dx�{�����u�30d�s���	(9�_�PB4�
O�
9Q����8,���� \}��tG��Pv������K(���P)��I�T�	��$2�G�JJ���l�5��DP�eS�c�s���B�X���a�"�j�a��vDb��U�{��wI`���n��9��Z��)D����)�H�1���`���b�� �)P3��HR	F���in�hZ<�'pN���JbU�S����V2�Z����<����� &��1BI\fpH�e)!���)��p�L����5�WA����%���oJ�0��o�%�O���H�k#�����VL=���0D��`,�B\*�L#]�j�����6%����g��'�'��!}"�2���������G�W������9�
��4_�)�V�V���C�zOJl�S8��E��W�7�FU������$��h��t�8��~@_��\ZAf���N�*V�0�62��}{B�~�2�&�9�t�<���pF���1O�:���{2'
�rt�[E��^d��7,�H��rM\��2���������+�����EqD&2;"�L�����������[�?+;�I�R���h�����)s0b���a�z,��}A<Gy���e�� ��DP�1��>(������j��[v����H����8��������=��p�)����L�s�6�uQ��o�}r��U��:�O �3���?F����iQG�BF	�;�TU#?)�$�����U����(�e�$m���r[V�r�^��*�@J���B�U\M�YeRI������>R���]�����q�a���D�AQ�v�[xFh���5�� W�^��pOT�nJ�n��&�tMH�c�����`�uM�j�3eh�3���z��bG��\��!��������'~HrbJ��$(W�u8��xa|
����O���(����qpG�� e"��/�KDm����i�PQ�D��d�0A����&K8`*6�yX%Pd� 7J=����I������;M�s!�,�<H�:���5���HK�5����<������h�Q�i�fMUn�,3����	���0�1��8�����{����g@�]�Q���P��EhI3��*
�Ra�T������o�H��L~�u����������W�O\��E��+�?�;�����������b8���=M�tE��'����=S�_����,u{��>"��o�_d+r��
}f<�������]��Ka��"�T��
gb'�u*E�Y����u�bY-=r	*�3�PMix1�Pr��:�@����0���A�e�Z�}���Q��#
w�0�a�EL�?���i^^��c��~2�M��a@	����G�\�������������o,*�-�K.������a��^[���C��(��2���w&u\�F�d��S�#`��X��\��s������������S���qqv+������'����W��SO�������==l,�j�0�~�"g?yZ�������3���)����9�lNLN�w��LU'
�������Z�Q@���0�p�����b�q���-{��>J���L��K�4�C�u��	~�A���l�$PL/�	��$e�#X 5*���V���F����JpZ��d	��jc�u�v�ZVAu���f���t�}��S�0xda1K-t�x'~�n���!�:e=�/v�X�'�kX%]�t��*����x:-�8��������d�����
g�����()T�6�pu5uH��9�`�b��������i��7�,�n�_��������� ���m��Q��{a�Y���`�h*j�.A;��5�TQ��+����;��e�p%��+���+mp*"T����knN1C�g�����Wm��T+�.��H�K��b7Omj;�I�m$���B���8���r>{	�uU2B�� K��N��U��&���x
E�t���6���m9��9�����$���Mk(��b:m���B�`��Q�l�d������Qbk������u��Q���tkv$�M���L����Po��	��u�	����5|������@U��Sg8b��R]����#DBa�b��A���J�V(*YA�N����%�muv���b��G��U�����\�&�5��%����Jv�H GG��H�Bm�8��U�U���n�
�0-7����+�so�"kh��������s*�D������vV�2���4Sq���`EA�vK�$�>��#-�!������:Bv���hQ�+b��h�I�G�Ee�#1�@��v��������I����������KSZJ��d��i�(-+���r�3N�8����+�����3���`A~�ky���"*�u��6���[4�No��1X�`���{K���5��o�$��1M�;���wa�����'�u�[�~���BX�d4��*�F�S��@^#��caO�
{%��1��{j��c������������6�5�d��#�����t�5n	��>�^��X�[�n��5�;��WWW�9�����T3o�:X5o��7Qo~A��e9z/��p�>pv�������e��5��	5T�Nm*��M���6���Cy����tK���j'�{m�o�������|���w.pXJ���L�B8��SZ����q�i�Q���rB�+�/�N�i�:��/��u�ZzOf2�"{&�����'XI�O+�����#rB&u�[�����D�P���S��h��{����������{���3�^~m
�m6����t��������"�5��T��`����D����RZ�������o��s�<����M�xM�svo6����� !;�0m>�a����������X�ok����]��m=O����/�6�+���N�c�M�	�.�f����K>��C��Yu��T7���|N;��W����i��$+x������3��j;�V��2�u�,�,���n��Nc:��e��O�l�l�Q�Gt�T�wTbYT�pE�\�^t5�~�e�y���"L�G�h��'�4��W�f�,��fQ`����h�����3�:k��6t�,M1��,5���2O&�;6O/u�Wr;���e���p�u7���\��Q��iHR����/N�����&�	��������%��G�?��m�7g�Y���r��Pf����|u�w�1g�\a&=��i#QF��\���v��.�6V�z�o1H����d��9�1��DC|��T"���_d�r�B������/�^B]_�X-��m4��;W0�t0�&�V�h��������?��*u:��~R%i�X$�q9���j������a:O�������������SEJ��N�{�t���="b��Q��o,���o,�7�kC�A%����X��f�r���t�n8pG4��������'_�;��w�ef����?��AI����=9v����C�y��t#w	�[�2��.:��qk�|;���������w^p����\����4���;�����w�n�w��e4�����k�]��`G_���ny?�lA���V����]�{;
�Z6m�� �����K^�F=����oW���	�������&9u�>��z�+N?o�`�#9P������e��0���2���PK�>/9:�3PK�uJ+patches/0009-Parameterized-path-fixes.patchUXq^�Xq^�X��;ms�����_����$���e;���|��q'/n�6�i;��,:��}z.���}H��d]��i=�-��b�w�L��x29
��?z��f'�g���������OF������d$��M�k���b4zJ��x4:�^���B/�,���\�L<���`��Tq��Uj�~�<�~��z�X{�F��)������������&G#�:��)?{*�quq���=�K\�T.��OX�y������D��8���*XC,Uz�D���$��0�	Z<,T,��a�����:����@��>�U�^�Z\2d��x��K��Q�j�"K��%��2�.L�����y���L�L�6����������O�����(�}��0��t�/�f�2�`Ga�����[P,����h�
�Aj��r��o���� �w�P;y��Y�tM> �A�U��8�Ne��8$cXb.B@Dd��.�����"y�pO<(����[��(IV�;���l�h��!��7-cy��N!{��K���7/�_�~w���C��q���� �F��"�N�X��Zy�kX,�b��!�eT
�IoT�+������@�����ic����&zH��(  ��	&Z��H����5�T�
��6���5
������a���U�4:��e![��>��
l�����������N�l,?��cQ�L�r�5]X��]9.�I+���
W�P��� ^���E3�)`	c�l�Px4�|��y�0y�,�x�9��S�
���{����N���2�4F���(~>����n���8&�,\���!:D��b�fL�p�����-D45N���Fc3k��c?�5����6F��*,�t�Q��RMK/�1�O D
���o���l��*%��v������;�m.����1�����a/���fJ��d0���?�K�h���c��#�=`�ck�����OG=`��xv
�6H����k��m��u�*u�<�|�!��
Q���g
!v_�.~�o"~�V�[���T�v�'������g��"u����U\����am&�S�
y��5(����U����(������O�����`�w�~Z?G�l_���p	�#�$�z��:����v��W��U�h������s]`[�8��A�2�c��U���[�x��b�d/��_S==H8����/�]<�B��������Y��"��Bz)"�J�b�f��-#>V��7���L-�Q��1[�� �K��)��T0Ks?�a�sG�F�2���B��T����O���l=���^���:���
�0f*�,�HS��:]��Du���������u����!���wX�BE+H��*^j?g>�����/����`prz<?>	�~k?L����EO6����.�>|^���*I>

�5����Qo!R�ei�B�  ��i�L#��C�*�'4�6:]�i�^�-c�����_�W�����F���D�����CpF�@\�)��>0/��"Cf�jJ��f2��Mu���/��8����V�t�������?��ROpB�\qa���_���
9,��h��=l#��9��)o�K�Ng=������/k�_L���XX�Xz�����c���j���/�������vM��,��!���-7�xb�F=�_����I�����Io|�Q��\���Vx1�C�~�+	����j�r��D�+����$"��\�Z�$�jY���"�oV�.�E���[	�m��>vrK�))"������������+7��`&����zGB��W��E{�FPT��j�b��@2@Q���.�E�CE��2Q�MAC�����*������Nr ^Qq�����3n$���X����E�T�M��~g��l@��A.j���,^jU���tp���*.�jd����,���u���	T��o)��QT�[���Ta���\DM�,��$�����
T�\�c��c���<���a
^,�j���� #�rM�`���A�B\D:������du��+�d�t��g_59������w4���x��?
��,_M}�_=�y�d�s�i9
�~Tk�+��)IcE�WY'��:$%V25@���e���E��	i����w`���\*.&��ts
yi��Sn{pOJh0�E�3��,���{��4�d�&6�2��L	=e'�6���A�����-Z��2��8\a��}���y���(\������FT��Svb,��(Z�U���{�}VT�L����f\)�q:G������3��f�2Z�T
��rG�&cBr,#��	��i��*��}�")�fH�s^o0������� �!�I��*!�����\�;��T�d@
K��&�{��S����A���t{��en��;�	�����7������Tey��o��i`�e���������u�#�%kn���h�E�GIW��m+�jp���O��y8��f
��}J�:�]%]�J�3���*�?�������Q^���y�^��1PM$��R�p�[�����H�Fz�}	��8�U�
M���NNzG��NN{G�c�h
*��ob�`5�:�h��2Q�l���j���5~v��E�k4X6-eW����D�(a�����#6�������,��V:�3����h�!�%4�*�L����m��y�`�k@u�d��m<��3�b�"/�Td�_Dv(���|�����q�9E�/D� CR/ZyjcGKpc�)F.���	���B`sp��lO����y�����N����u��l�����
F����������D���
�V[X���������v�#(���W�vN�����d^���E�C�L��~O�8B;����qb���0!�=�4�T�������&�����|6�&���a��P]0n�J
]�j=�M�p����3
����*�p�4w��2�u6�'I����!;�������S���=<����@|P�iNS�Ng�����,J��	!g�X������7��N4WRT�Q�Y$\2�T�Z��[��`�6�HL>�0�p���7����'�uU�I�A��UZ��:\����y�[�P�aG�l�Ln�-K�HI���)*�S2���L�_qW�f��?�� ������I`�x*�Re�+��2��V+dm����d�r��'7��s7�U%���^'5�����y�o"�S.�/$��H!��+ $����r�Eyi��~
��\�Rr�!���8�-�K�����3-��?��(�n�j`��x�����m�P���Yt[8dF�A��F������� ���CJ�Qe�^���V��V������:�!Y�u�hS��)$�O��<Lu��2
�P!/bn;O+_"E�cC�.%�����oF�6w	T:�vI\�-�w����0���;��7��,���B����_����	7M�<Y���F[����N���Ud�����9���2*��-?������	jPsZ����Q����ArS p�HUv`��]��\��]�1��w���S��5^�`b}5Qo"���>��Y ������nBT�4A�?�r.%T��<G��_����iQT���O��9:w9��)�(��
�bA����j5�}��hB�W�.���3�2w�������`�
3j,}�J�kH���VI��`P�2�X��#����L��	T��^�5�C�^I��P-<�e$w�VYP�6'��jL%��3�Pqgi,���o�Dlop|��4a���`�����L���[��!X*��_R8U����k:�o���5��C��_S������{�}j��|���s����4�?^�����E��R���mh�F��{��[`�����w��[45������o�����(�|�P��q����h�LF\V�vk�W�
�U�S�K4V���{�����l+P��e������{wg����@���C.;����w?�{j>��@��f���-O|���NSd�e�����
���R����@�ee,�!�9�s�<.�[>:s��	o���KG������*
/�<���b�> ��I	�*d��s���Bx���{5�`��_F��`�"������E��s���*����I7A���T�v!���~���v��%+�YN�6�yg�R��#
����yA��Q�@�8�6���;_�V��U��Xk
T�i�Ck�Su�����U��6!���^���J�+r��@(�����
���F��Kzx6O�j7��I������W�T�
P�x)=�7�Wo'��o���/s&�H��%�A����.��J�6���"g���A/��eh����=*��� 3V�����uf����kZ>i7�m*������/��B/w �6
�d�i��s�������o��v��:'�<>����t>?�m���������^�L���	�v��4!�@|����������TG�x����%Ix.�ze�P\��r�m�Z:
�R�{e���7��E��k�����%�5�}�w������4�QVwp�^ ���E�x<���������_�>��c?��)j=��m�	�W��r5~R���W.���C|y�����&9�m�Wf��*}����dM�1�����"������7�7�K���[��B��-��$����U��� ��}2������I�#hD������CR�C/��Q�2�8�5�>�z�m�N
"��E7������;$9�t���~��������7�7�U�d	��sz�52w!AV���t�d0���PK"�j*�jCPK�uJ,patches/0010-Partition-wise-join-tests.patchUXq^�Xq^�X��=ks�6���_�������-���H&7Nb'�:v�v�vokKEI��
-iHj���|$@$%�1S��������7#���F����mM�Q��h0@'�I�7M&������|��t�@�sJ����6.0�Sp�?.�����v����#����[x��&�������v�N�����@#��vO�G�V�u�V�q����S��_����	t;�]���W�������1D���3�����t|0u\&s���<��,��"`:��>�{���<-���w�'��IcG�#���q��yv\�l��	pq<\�����#�P�jl��U��������t�|����o��y���������F���|��t!����M��+��z�7A��E��vz�����I�i����`��U�/	<�v]���#�,1)��4@��/y�G�c+���J�@�-�>�9}��a`@���G{��&m���!�G���{�6�f���|o��0�=��
�����3�����@��Fc�L�`��	�}h�0#��l������?����q�;�Lx��

�u�v|������6�Yz�B��g�?�i0_PR���������`�8��?��A_���u4���?c�5�����wg����������C����tr�����{�fd"&�A*��~>�=��������}wu�/����Y:h����'������g�����7���������vK
d��p�.�~U��B��w�������&tw~u��=p���`>$�i:��N������.no~h�<<Z�d:@~�����9tX:$e����xqs{~��u�� �G������(����o�w��f�p��Vc�+�����w>_������W��h��e8��x��],1m=�4�m���<���]�-��VQ��V�H�h���R����_��B�������_H��w���#(ch��#)B�M��LJ3��M-9�����*bS(eSX��jl
�l*i�6�J��'L��0d,�gug�)�w�B����;���Wg����������E�L���A��n�`���1� �����k���o�����&y	���V�~�b�����u`����t~{N+X�y�7�?�����a��]e������������q��p_��^@�L>����SF
��1�a�[�4�p�!/�r��3�g��",��4����u����A�^B::��.�@X������Y{�GD�����dl���#�w���w���C�	�����Q�O������A�F���Q�C�����1�H/��\4
_�	b���K�0vm��g	�A����}H���_�0��
���zK�h`�&���.���Tn�0�V2���x�a�r_�����Q*3������7�6��6���0���t$C�1��q�|�� )t��#��g'x��s����CS����p���� `*w����lf�f�`�@>���m��2��M��l�r�5U���)���1�����d��k�?|�B(��+^���Y�Z�W�)A���i	�)���6�G6ZBx�Y���FG-������%�����6��
���U<�����.��x���@��\����-�������M��X�$N^��eL^���Jq��������0K�
[��lh>�Zh2��'8�Y�`)�K��<1w�:b������A���\g�	��j�5���z����~��V,����H�3���o�L�3��o�P����k�Z�6�U�n�Yg]���nXs�����Q��=��1.@'
�a���N�X��J��}:$�N��UF��N��U�6
����0e�x�&�������(|-��;��������?������t��`�����A�F�E@�����V`�6��������i��	�V��T�.���P,(D��p����u�KJ�o��?����(�\����V���2d�)���[��'����N��)���om�[�g�4
�w��`�7�
��pB�V4~�
!�R
h�����Sm���_�d���g��T+�/��g��D���Z����H��`>�96�����2,��JsM������a�hj�m���klw�;KXd�md��BZ�;�vh@]�t�&�Z��Rn���b��am�1��E\����!J��-�n�b�f��-�R3Qy^��%N�����7v~d
��%o������+���]3�~���y��a_��	�(�~'��1]6]v;�S�\Yy(�#��[=/[��%���z"l��;D����}V�/�nv�0�{�����o/�I���c�)h����kO�����zjb��	�OEs����3��5��J���'{%3�H>�VC	Y:(R�bb�����b+��Q����crW�
�Q�b��iBXj��w�d��/d���>�����B�����;V��>{�O�i��t���,)
% �
���-B���������B>rT��`Z�b�s7iM%��l\BJ�1p����,J^S�BYYl����u�kV����9�H�P���
��Ub�D�����G��
�Q����&�-66u�������Ct[������4?�l����8��� �L;����}\"�xT��?�����<�G�#�0'�A�>�����j�����vi���I�o����a�������Iz����W���236�s�}�(:���G��a'B�
C)���wRH>�-)�-�%AF�?c{'v^�I�>���H>)�&Z��e�[���B{����q��z"�fpf��,�V��W������,����]%���u�]�3���2��'�Y�6*������ZD��n���%P���U,���"2����W7�v���y����YP��#�J/<���T����~X��AV�W��M���HbP���R������&,���,��X)������%�T����DB�l��@�W*1V��JQ�����8.�-I�q���)IqJ|��x���D^~C�C����R���]�m4x��M���3w��:������zQ��(����� ����+�Z)#��p��4�[���1���Dq�b��F�kFN[)��]#?����|�W$�R:@�l�4������)�J2j��������
���#���>���L�Z��T��4���J�?Td�)��������<)a+2%)���E~=��'<��y�-�7y�I�`.�����?3oH�D����gE����%��c�H?(�m���F[��/���'�/{E?�7u\�:*Q�x������B�WA�E���fz��q�x�(d��H_� �Q "N�
��rb�
����,
��:"�9�K[�=Q+������w�=��-/�|
(���^
e1��
"�����2Z��������rm�A�����}�fiXR�I��0�Ic����B�pv��,Y<�b�/.�����z�6K^���&�8��Q�|?�,�|��T���*D:�PUeHc�0u��Y��������:�&l���V��&�%�����hy>�5t�7S�+��+�����cl��2��6
C�F����E�*��U��Pwo��r���Xa����Rd��n������*@qe2Z������uH��Vv���qi��3i%�k���}R�C�H&��_��,E3�|�i�9����
:U,%��_"���� ��Eh�y$P���(	#��KTr�p�IO����	��B�'B��$�$�b�V�6%�Q-���lM������@�[���:��n�['�TwCv�j,}�����8yY3�6m�-y��$W���j�#���* D[�!�m�?�3�N���?��<k��S��u��R��El�Z��%O���Xq5�	Y�4��RQRP
K��U� 1(y����C$MejL�����
�����i#�T��UJ)�����i#I�JiS%^�*
�$#13������, v�Io�I���29���C���vn������:pH��{v�]�b�T3�����7Gl3�y�Iz<:�B�%PRcVD�L�EK�#�4�P�(E4GK�YQ���3�������i�Y�E*A]�?�/a~)��|�U�=�	����!���V^t�{��Gk[4�z��x
M�����h��w�$5(����=�F��f����R�����M�1��S��@��Y�����s���2�g����Z1I$YCh�{��*YF,E�(17�c&���I��:b�@�&��pg/�d�Fo�2���>��f����������rTdD������D�+kEe�WT��#��j��Y�p��R��n�%B�s��S$�&����7"�E-��V5��(���m��W�e'�Ih�*�U`�� W'{l�8W�=vB]B�&�!�)|��-��v���p�F7�l(M*\������n:�o�oK�
Opt;�D?7�[���q�����r��n'��!M!��
I�gs���uL�g#�*�N��^&����a-mkH��R��������B�S_��g��WQT�J�)M���4�6�7=(�����-����K����CG2-!���=�YC�7�gm��6������6C�^�d��w������:_CLQ���p��v3������i#i����V���H
3��[)�i���]��%��HU�\�k/��\�YH��u'�)��o����i����+$���i�S �n��G��iau����E��v�:�]��9������+v���D{�i�P7h�R��Wx�x
�W(�yJy*y]Q��y]�@��6���i����K�X9����
�n�x����V��uK���������5�lHj3=�!�������l}�^W�����wb�k���d��%9{Ar2g&7_�{��J���j�Q"��7v:�QtDC��xB��&apP�4,�v��l�Y��\�C��W�jTz���X1�r%�'�i
l��b�[�T��d
-�/������g�7��KD�����������i����i7�16?�eI��f�t�����}Vo��5��$J[X�1�����\�_7���S6s��!���B�W(����mv�����K9����hS�@��\Z"�6��[����X�����{JqG/����;��p��XRw�Z�6�;���S�;�	wG�q\�;4d����r\�;j������z.��&���DS���lB?��X��Lg6A � ���C>b���Hj��t����v��3�\^�p���k������R��|9�.g��,E9�8G�J���A�T���r��[0��
+(�������z���:oY��,�->�T*{�,�������s]�3n�/t�q��B�f�d��Eg���t
��t����_�K�E�a~o�+��X�V�����2���D*�[7l���e��z2|q��T)�k�)y{-���X����D���U{�X5W9(��5(-f
Em����XUN��k��U��J���X�_�Us5�R�Z�e��(������7!fm����oH(��"�W��6��a��XE��H���@>�	L2��k�����&EyM��$���jm8����hC���6S���d�h���!O5Zj�D<s��&��r�U�]��8�q���T6��hZ�VOE+���BJgj2S����C9EI��������a�+�T)�^e�UIt���wM�Me*��Z���������R�������j��4�p��x��x������s�G}2�b����v�4�Z��5;y.�������V�h�i#��1�i$����/�.g|�E����b�Z<%*Z��r����#�@!�EO}�CM�������g���b��\z����e���b�E�@M�l�D��^�$V����������D���%��Y���T�j�|����'e6���S�6`H�����B"�t3����J�IG�-�"�w��J����0�L,�+�)���DeV�]D^n�Y-wV�Jy]���.��yh6F>����=�	��OD��~�*�����_�y���}�K\���)�(��5�0WnHY�]N����X���R
��R�Gmr�`_�~3 a��R�1�����D}K.�v:M2uV���PE�������Ry��������QC��R�x�KKQ[4��-n�I+�_�����Z�$�d���K��c�I!C�+r,8)�'�XVX����K
�)��D$����HG�_*����BJJ�1�H! q����^ R�E	4&)$�����+�;*F;�&�vr�6�5��m�\c����k;�r����kvr��\s{.I(�u�y�K"�<c�^PK���'p�3�p�^���
����O��\�+�����0����,hc���1�d{�G�k�_�n�/�/�h�\~{v��y�n�el��/� ���{s�tqs�~v���]h���I����'��QB�zc8*&W��N	�����P���A���;��������p�r���`>$�l���.~!��]8'=����P�s�Op�?9i����@�$i?4%���lB7��%PN:X�tPN:)D}�A9��P�I�����R���x���.���\��'1��,%5����IG��5�		��0��\1
�%s��>�%�������u��%)�'��h�e��d���r5�0�T	K�~���*��M��=���o��$�b�n�xR��ax�A�{9��"��t�9g� ���C%���t(A:4Bz
��J =���2�[)�[�-%�-�����UH�Ez~��2�.���o���p�����z�;^�xW���u��\�$�����o��[�����C^K�����N����B�����;V��>{�O�iv�C<N�������Uj^�	�r���3cF`�AJZ����o��r���d��I�r2'��wTT�B#�Y����],B���ed�n��ax?Q�MK�
���Qx?��}���	��|H��Z�K���;+���&�kF0(#���`
K�6.����0�p'u*��S�
<�����wg�$V7���0;������T�t�"� ��6��@w�;��sm���������Ov0~$�`�M�z��~�'����2�j��D�
?r!|�T2�C��DS��4P��?�����
�����@`<�l�c����o����X/T��l����W�Fvw(�A�z�����\/������e�tWD��boDR��@���y���h}�U����3�v&����t�����Un*(���UG���=e�>�O>�!=����&��]��?������zT�n�(�0b������]!�m]L �CX/h�#�QML;���1�["�'x�\�8�|,�x[E�2{������a����H����w,{.��"{���Q_�����Mn6�H����}iwT�ar#�������x,%l+��d��vC(�IO	���l��H�p�W��'7��k,y���@	{��l�Z����R����Qr#�I"����=�+a'7�l�����xN��O��}Blp�v��|���`w�������|�U��n7����v����G�!��S<"�!�&��,��'4�i������D�x��x�-�����.�rSxVg����{��s3C������S���xm�����Iwh_Z:;������@I�#v�;e�}�o�����fA�nqMi��b��h94����w��*����V`�T��1]�K���J�=��z�_�q���_��C7��
�O�������Q�3��5�s�B���Q`���b�����B�J!R(!�~��5��7����T��3iL���2
A�73s�����BY�,.PZ���R�a0��WR���Y��`��`�����vK�v�n
��9^�v6(S�J���5HhII��$a-�EON��l"	#S-y�kON1)$����*ae)$��G�J(��4�)���p��������pF@��<#���u�8LyL\��={@>�8�)��QIG%��|�^������������+jH�>���43a��q�$��T�E�+Y��<F��q��nW�Iq����9@��>+��BPq�<#4���w�R��C��J<c�oX>�'���\���(�0���)mA��k����zIK5sqU�@��I����.�r�!�;��p��R�9�~�����]g�Bb�Tc���>��=�15CK�E��{�;}7��eS"���p�������4�)����<��{�����9�k_v�3X�WT���-
�e�?����	��"����A�ZW�h���^e�:��Bb�������}�(Y���6t��(�	wJD_w#f�F��CF�
��E��IRA�>g���G�L2�f�Gmn��	/����&�*�����L1T�#�n��������� 0��" �du`�E�����<$nO\���M�����\1��kr[79�����p�t�{�C�::���;rH���.r���M�.#�B
�*���Q���Lvudg���k`
j������������X�#��n�y�,�#����/G��s�K��
;����4�~>Q5l�[��}�G.�Oz��b���<�Y9�!����W0A��h6~�19����/|,Q=��c��>">8|�8c���B����G�l2<��50]����;�p}`O>9��{�����?>�.�3������(f�#-�$��>�]�����%S��J�e�}��)���8�?�L���l>�����C��m������>.AX��#���������T�plMN���t���4��J�����m�q�o�d\���~��>�O��s�=�;�,E��|t�()d-��l��3�:�O�	�p�0��,���u���p��%���Du�������b�|���5�>��kqK|�p��e����4����v���<4���
H4���������u��17/��2UN#3g��H\`��H ,P���y�d���}m8��i���FXAf"J��s���TD�PS
H��J��1'�#I7d�5U����`(.���,�1��d���mB&~��2�
�Q��YK���Z)Z<�������Q����Z��)���@�:`�|4J���C�I��N?)���j/%��$X�Y����@T�H��x�#*��%�^�LB}.�d���O"z,�<��,A`�' �4�&��������vi�����/�H�����>��AE�x��4 0�:��� �f�b��,.W4��r��9�0Xx�=�&��������M�x��z��oh~�8���4/�+�@������I�F�E"ls�+��6��+D}��x���';j�n����v:���������G'M,>������5�V��0<�f���V�f�o����z�'R������$p��R��)�����i
HE&8i�4�3\&1Z=I���jN�8M��)�4��SG
��@*��V���:udy�Q%I,��8�%�z*��N�q5}a���'">yyd�����������f��@�#}�A��4�V$��,��?H��GB�JSa���ZdZ���[��
�.�f:�-���j�&���!�P����Vi���!�B���5"|��S�G�*��;B�!�b~I�i(��D��(�l&$1$��H��������KK���*,�!�����&I���y�a�3����z�a��-�$��TI�~#J�|:�Z�1:�-�1��.�8���(O!�I('�~�.'�(��,+���]��r��UE9f��#��< ����]#�L�u���oY�[���T���P��k�;��MtD���S.��o7nv�f7n�u���S��-xB�'�O[<�Ol�Se��{�d��}{5��;��9�yw��>���<�y�{�;|r���?�����@��@�;���9��x[g�l��-o�\�7x@����W8�`w����u�;�`N;��w�a����/]���8��PoS���J���Dq�������A���PKs�W/'��PK�uJ:patches/0011-Multi-level-partitioned-table-expansion.patchUXq^�Xq^�X��Y�s�6���>���	I��4��4�\�&t���G�EpclW�������lc^BB���%�V����j���^�t;�����mo���
��������w�i;�.|Ibx�)8}h����i���%�9�S9��D��'�2?�������<V\�"�<��~2>��3���R�8m��JC�N�h�}���������O��#���tp�38N���_&�
����L�P�I�P��8�o)�%�iV�gI,���
�x�E�X�s�\0��f����|���F|>�Z�l��HR����?���$�q�a"��fC�L�E���U��� ����������*�s�J#��8T��5}��*���*�S�1���fU���Z���F��R�\����Vf�f2Q`�8,�U��0���y`�A
 ����Z�nA�q���W�A8�m?�
X����^9���^���N��������A���{=������_���?���m[}h�?h$|���p3x���Q��E1W�>�$Q�*T���+�<�M����h�>��8����\&���&��s�v��f&��o�����@��	~��Oh���/�o��O���^m�.�/+~�0����=�>�'|�+��>$i�}����c��J}�`�a<�
�<T29y��A����,�����`HEd�!�
M4{&��Q���01�Q��}6�\��B��0~�-�P ��LC5*iT��k��4M���r���Q��QW�n���e1�I)s��
j�gt��]�p1���M�K�Z�x�LF��2h�i2��/��hO5Jd�Lt�G��3A����A2A,%����Pd��o���)������F^v�?�cN�-C��.`X��P,s����41�����a�5��������(`Y�F
X�Lpx@�?�Q*L>\��9F<���rU��kf�O��,�����,�"�D����.�11�l���PG�������H&��������P����$�7TL ���}M��y��p���^tA�����V�J{1�4����g�|�=��(���I����'NZ�J`r7�q��0@M�M(2���fh�b8��5�F��D�X��N�9$
o�/��r����z*�B+�- ?�����D&?��s�+"2��.fz���(�I_?��B������bdI$��b\k=?/ "���s"���t�2o��\�Q��}%�c������\/a�H|Z~'MD��H3�o�GM���"O5��BBOIo����jT+�8�1`WD�Oq<3��V���%a�����
q����	?��#=5�L�v\S[z[�h
���s�Wr�����d�yQ����g����3!G16��"Q���w\!@�t*a�#��x5�)U�'����s����"��u��8�E�EL��L����|���U�$��*�/��F�u�XyB/�4e�c�39^[&&���,3U������(���4L9&_�e�y����T��\u,��\0

�t"��L����Md��_�i�\��S�Y��2����>V�h���{����tp�z������I����F���I.	N��bw��"X���`^&gij�\2n����7��de13�	LR�,�p�E�*��&R����Q����5��C�G~������@m�^*St@`xa�r�s�H�T�dl L��R$�:���� B��<�:7B��+-�Ghe�2(��{���HYv��d��b������;0��<i�EY	�J��g%����*�U��,��u-�KI�@_������ �[��da�.
�0E���j�`���Oa,���I�Z��j���(���dYE��Z�T�r�Ki�r������^��N����RMC�pM�0�����R��;dT*��r��u�A����W���s�9X������f�{yIg#�}i)KHF��Yi!e�^��6�#�d��"���E�nR���Ld�$�"�.j�5y�.tP�i�.��O���x�BvI�NM#�Xl��v�t�F�b\y~gF|��+#J���m������B��:M���T�m��6EQ��1���H��Y�~wR]�a!�
�b��f�n�/W7������jpu{sq�N�� ���������.�����VpuVV,����g���-�0�3(9��7s��\��#S����\�8��	^�cz����huQP�r"�:E-�#\2�[���g#���FL��G9�0�0|����� 8�z�t��[����^���)�u�M�e�I�"���8����o��y���p����������N����T.���3r��n��&����Z��K�����H�+�w�g9�����__��$���_u(NM����
�Eo�iq�u����7ob�0�Li��P�"��[2�.�)
t�m��}[|Na�Y����Q���:R�������jK>\�"<s�%��P��Sd���%�T����	.����k=c�B�fz���rA��v��;�A�w��w��YT4��%�_O��s2�
~1��g����Z�7	�j���c]�/���H�,��px�f��,�v?����{�)�K�[�s
W��/�d��n^�4�w��<8h6����u��M��%�{�/��(�w���F�/�=4����'RQV�3�D���
	B�����(�1����'�/�%�d�����],@7O��j��	m������7��gl�����
[���[������g��%����dS�
�����(�M
���}�����X1S'�C�����o����#������9:�Rk��K�o�F����*#�SeH���a�}�,�e9�C�K�3\��-�`�����eKX1��j�������������c��������c�=�n�q�����C<�]b���G����C�O=���X��{���S@�����<��������P��}���7K�
~[i�m�����7�u_���"wz��Y}K72��d���<�:�~���W��PK�x.��(PK�uJ:patches/0012-Multi-level-partition-wise-join-support.patchUXq^�Xq^�X��Z�s�6�,���'E-�i���n�����v����h@��P�J�q|������zXu����j�$�����,�:����:���|&��h&F���H��N��?����p$�	�q7r�����F���~�d���Zei�V�������r��k�2�$���c�����R��9Q�#�	� m�~>�=N�7������y/��9�xuq��
��{�\faB�Q��I�A
��<���T���Ij��Q�8\���M�6�c�_����	%8�� ����Z�H�"������m�D$���&U���U���D��:\��N�\l62�`�H�O�����9��J��P����>Fe�&PD���w�[�U�nW��18�=�Z���PpY���z���DQ����n�MF��x��UJm����e��g�6(�����m��e��������)Y�D~�$R)��>S���'q�����K[�aI�-��%��t��`0h�J�G�P}'�&
��59��'����,����1�zG��q�#z�A�+�PE��#�����2Q��|���MjP���`rEJ���=��� r���
���V��U7�peEt'�>,�DJ&�H��u�0��'C�_����>�z� N�[-��~�O~�;�
aY�s*&��p8�L���F�=��_�`<��3��������S����%9�%A���:W���<K�8�W\��a�
�_�������V�����T?�Q�S�������Z'����v�H��8��B�;���g'D�!4�O��v���_f���nAO��R���-�K�#^T�x���1�XI�G>t��,_~y������NI�T�j�q�n!�gG��b���g��8������FCw���G��B�Q8U����=����_{DxJ��]�p���7�>+Q�j
�J��v���:M7
�v	h^�J�b^\~w��<�*X�D/6��P�0:_��3�u$&P�<�
����Q)�^9�������GI�RVg���vY�!�H7K��#'�b��Sy�v��Q�w�O�SU�Q,�Vg��6���@�5{���h`-�C��g�������`�$u�����=�����)[�e���S�k����w�x&8'�.K���j]��Zbt��BDz P>A9z<�`�:a1S�;��(H��a�T��0[;�a�Wm���82��i*&e�#�|UAc5k�1�����E,y
&���+j*�����Ti��t�m�~���eK�f���1�V]y_����{wf����q\�VG������t�Y��%lf�.�Z��f"%S�,RE�L}�@u��~���z.����n8�$rM#��$(7�qS���Y�e�Q&��tr%mk�K�Md�%9[��\M(��6���)'58nx
���/�g�I��UY)&�`�z-��6/F����H� 8B
w���C��LK���\�M���s��Jq�aik�I;���k��N�����t��������VQ[�@P��/��X�s&��������s�K�4����h���L���d�T�!�:
�V �:��e�:����W�����J�j���0tI�>R���S-��PD�mU�J30��xh��,�6I6��7���&�����!F� E1��zyq�
�
%������k��TA|D�"��t�?�D1�����n���Z|��2�Tw�G���k_$k�3t4v���*���?:*/lR:?�}y�1^L�s�f������=���A�xF�(��'v�����g�	RK�3�q�l���6(��F7�,hH~��DzK����rG ��bM�����q�c�9�205����od5��r��U
�efib83�������(�$�����^<�
u����C�,r
�gMLe�������h�1���<Np��'��`g��PB]����48/l�$����K�)��g�P�@,����NP��Pj2�[m���:�(��NFs�� �����>��-��~�H���w�W�o^|w�C������!�����s�x7�����NH��;��)%�"M�\6�D`+NU�a��u�EP�������(A�K4���Q�����"�G�n���i!�a�y���5f*��+;�>�i�N�C���d��0�>�
;��0t��.e�`��^U-*������|�7`��Dc���=3����z\	�z�i��'Oq�����Y���5�'�f�?����� ����0�����
%�������A{X��2��S} ��5��[�q��f�!�`6�$�3lm�R�.�q��3����2oP����f�y���Q|������`XS*O��'���;M�u��������W*_4.V�5��	��-����!�c7��Yw�7�s�m��g�F��M[�CuW��X���Tn�/+��WWoM
R��������_�j5��@���fP��;5�#Ts�j����d������/K��0D��S+^h�6�g���V4�����+���U�z>!�>m�c���A����Q���<��P3���O���9X=�4���NkH��.�����C#�j��#�:_T�HDX��Z�>�G
E0�R��W�NW�8�:����|����[��28�)��Uu}( ��&r�����n`*a�0����{���Q���pX�C�.#7k�s�n��,O����[�b������"rA��r����OTB�_�Gr�m�������FQ�����V~�v��v��R<�S�����VL]YO���*���Y��[�����B!K�9_��AY�5UF{�X�G6��8��k�k8�T�����9��A>�k
F:o����$��[}�����T4O�� �*N��}D���<*+_��*��/�e|�������Zq�G`F��I���a<{x:�e�0]tW��|��,���qRblR^P�d�TH������{�Zq�7;�0o
�����$�yqo��M������tWD��M���r^�'J�����3(���]���a������j�2#�R�
�b�4�{^�.{��v`�~l���F�i�U1u��E|����;�g�����.������ao�G��D�l�/\�[���5�����}�^�/����l��B����'�x�C�����n_P�b��EHw{uw'��T�2��v��dl[TL��}3��J�<$�y�V�c]��i��]8ag�t 9������6��mkn-�i��PKsr[~��.PK�uJ8patches/0013-Multi-level-partition-wise-join-tests.patchUXq^�Xq^�X��\{s�H�{�]uuX�f6��VH�;��rWW��[YY"����|�������@��i�1���i��3r�{�v/d�\"����t����)�J�R�P����(RWS��m���@�I��?@�$�F�M�����;x��T���������#��1\�/��}�[����>��	�#���(o���/�r�$K��z��h^��z0{�d|.������-�| &�T�3<��Z�=��
<�zn�Vk�Z5h�������!��=��s���r��������;2�%5p��	8w��C\���QM�2t�;��M��3;!8k%P��1�Z��j#���{of��>�+]�w`i���N�n�����aQ�[?k��������F
4����{['���n������c���ZM7�Kh�n
������2i�"�pSj�����;��n#��#����\'���i�h?�#��ShIM	��&���Okg���RT�$WX��E]�l:���BlY5�f7�9k�c�Y�o]��[����|2��0<a�x����JM�k�B\4��:����LfW���kx�&����u��K"��JU��G7��`�f8���������)�*�����IJ��Z��}��AO$*����D�H�w�D�S�r^>_%�/.	G����A�H�
	��d�#N#��R�r���'3������N��������y��i��|2z�<�"�Y��8t�����u�	J����2\�o�h��;*����d�@�}%�D"��>�R}��(�'���	������D���R}��R�8�����2��	��O��@�'�DbX���E����_�W���7���*=j���K����dg��Q{�������'7u=�z9�9��)`p��3e-(��W@
�)����p�z��F���	!����7��[��_���~���:����~����>o���
`�ZK5��|I�q��wALO���m�}��9��r~
���5�}�|���:��/6;���)ySM���/l�/%Vf�S�O������e#����9����U��>$�(s9��0H`3�mQEtC9��!
���C����r!��H0�"@�#�8D8��.�!f��D�mv���S�tXf�R��� ���h���%d�������r�����:V��\�����U����_��?���i��[/��,B����@��
/���@�`^Y�m�>��j��-P�:[�hHc��w����������J�,LC����hL��fb��y�(�:,?x�S�;�)�)��S��}�=�����_$E<a�XT$�Y��M��������������*,��h������N��h�xA��WD%��.�H����@�����
�=:D���68V�V2|�wmz�Y%e�+2��#:�l�a3��B��yy+�dd�3���M�w�����M+C7�c���x��c�l���pL3L���x�����\03��}�L	EL��#`�8����V���\�ENU���]��������GA�(!?'J�2*��eU��n��������������]..�B�?A�?���,h�.�"hFS������i�J��:?GzSt�(����Go�c����(LK�nm��P�0���qh�(�F�cF�ec�M��0�i��#f!�����!�������������-��^>����e��U�EqlQ"�(
aTd�X8g>>�(5J��A]6W�a�XPM�j����>x�U��/�\HDq��O����-���zQ>�qE��YA�����f�#�
���H��>\j��\��� w����b��W�|��9}b<�
'�q����~��=zl"L�U�M�|�����i5����m����_��F��L,��Op�����f6�4�u}��/����z�-��)�RR������zIO�WN�_�#:�m{c��JFn�x����#>��>�8�������R���x���T�E&3ql��]EMY��	�a��".��)���G)���Dj�C+8����2K��|�<��a ���$.�����@"�E�E��V�VCpN�c ��]�1�c ��r�v�����]IX�q�i6�q����U�+K��Md/
�B����sz����&U��i�,��#2@�� J+[z����-�����u��,N�|�w>5����Z���(,��[�I��HRr$��6a��6��3�nR)		��%rk�hIH�d���/��W ��IH�d�y��$NM<��\����OB�'C��%���hb���B�W �LJH)e�9���S:M����,v�I�@b�����Ue"w��^������f���J��b�"��B�p�����`$}X����,��z�j��b��B)ZX�wG�u��
�%b��.1��`��h���A\X����.���C�[k�'����Q�:as0��Op��`ax���^����[3t�\��G�^uC��
]�Tx�������d�\[k�j�)_T�������ik�;�6���<�Bm��n��0��g����_4Su]�n�Q;����{�T���g}Z�_��ot�P���a[U��jG�\�X��	�.�����67i�xb|����U	6��.{*�l�/{�~���xa�I���q��lb�����z ������m��q��[�7]Oe���h���Z�FEIm����&<�e�Rl�2C��(�cl�s�����i�J�������O[�����'�o`���������G�o�?��"U~����v���9Lf��HjV5�}����Z��PK����
�YPK
�uJ@�Apatches/UXq^�Xq^�XPK�uJ�����3@��6patches/0001-Add-IS_JOIN_REL-and-IS_OTHER_REL.patchUXq^�Xq^�XPK�uJ����*OG@���	patches/0002-Refactor-adjust_appendrel_attrs-adjust_appendrel_att.patchUXq^�Xq^�XPK�uJ�t=�DG@���patches/0003-Refactor-calc_nestloop_required_outer-and-allow_star.patchUXq^�Xq^�XPK�uJ��D �<-@���%patches/0004-Canonical-partition-scheme.patchUXq^�Xq^�XPK�uJ�y}�q
G@��l9patches/0005-In-add_paths_to_append_rel-get-partitioned_rels-for-.patchUXq^�Xq^�XPK�uJ���\�
G@���>patches/0006-Treat-dummy-partitioned-relation-with-partitions-as-.patchUXq^�Xq^�XPK�uJ<l��x5^�5@���Cpatches/0007-Partition-wise-join-implementation.patchUXq^�Xq^�XPK�uJ�>/9:�3G@���ypatches/0008-Adjust-join-related-to-code-to-accept-child-relation.patchUXq^�Xq^�XPK�uJ"�j*�jC+@����patches/0009-Parameterized-path-fixes.patchUXq^�Xq^�XPK�uJs�W/'��,@����patches/0010-Partition-wise-join-tests.patchUXq^�Xq^�XPK�uJ�x.��(:@���patches/0011-Multi-level-partitioned-table-expansion.patchUXq^�Xq^�XPK�uJsr[~��.:@��c�patches/0012-Multi-level-partition-wise-join-support.patchUXq^�Xq^�XPK�uJ����
�Y8@��q�patches/0013-Multi-level-partition-wise-join-tests.patchUXq^�Xq^�XPK/��
#81Rafia Sabih
rafia.sabih@enterprisedb.com
In reply to: Ashutosh Bapat (#80)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Mar 21, 2017 at 10:40 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Here's the set of patches rebased on latest head, which also has a
commit to eliminate scans on partitioned tables. This change has
caused problems with multi-level partitioned tables, that I have not
fixed in this patch set. Also a couple of partition-wise join plans
for single-level partitioned tables have changed to non-partition-wise
joins. I haven't fixed those as well.

I have added a separate patch to fix add_paths_to_append_rel() to
collect partitioned_rels list for join relations. Please let me know
if this looks good. I think it needs to be merged into some other
patch, but I am not sure which. Probably we should just treat it as
another refactoring patch.

On Tue, Mar 21, 2017 at 5:16 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

On Mon, Mar 20, 2017 at 11:33 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Mon, Mar 20, 2017 at 1:19 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

That seems different than what I suggested and I'm not sure what the
reason is for the difference?

The patch adding macros IS_JOIN_REL() and IS_OTHER_REL() and changing
the code to use it will look quite odd by itself. We are not changing
all the instances of RELOPT_JOINREL or RELOPT_OTHER_MEMBER_REL to use
those. There is code which needs to check those kinds, instead of "all
join rels" or "all other rels" resp. So the patch will add those
macros, change only few places to use those macros, which are intended
to be changed while applying partition-wise join support for single
level partitioned table.

Hmm. You might be right, but I'm not convinced.

Ok. changed as per your request in the latest set of patches.

There are some more changes as follows
1. In the earlier patch set the changes related to
calc_nestloop_required_outer() and related functions were spread
across multiple patches. That was unintentional. This patch set has
all those changes in a single patch.

2. Rajkumar reported a crash offlist. When one of the joining
multi-level partitioned relations is empty, an assertion in
try_partition_wise_join() Assert(rel1->part_rels && rel2->part_rels);
failed since it didn't find part_rels for a subpartition. The problem
here is set_append_rel_size() does not call set_rel_size() and hence
set_append_rel_size() if a child is found to be empty, a scenario
described in [1]. It's the later one which sets the part_rels for a
partitioned relation and hence the subpartitions do not get part_rels
since set_append_rel_size() is never called for those. Generally, if a
partitioned relation is found to be empty before we set part_rels, we
may not want to spend time in creating/collecting child RelOptInfos,
since they will be empty anyway. If part_rels isn't present,
part_scheme doesn't make sense. So an empty partitioned table without
any partitions can be treated as unpartitioned. So, I have fixed
set_dummy_rel_pathlist() and mark_dummy_rel(), the functions setting a
relation empty, to reset partition scheme when those conditions are
met. This fix is included as a separate patch. Let me know if this
looks good to you.

3. I am in the process of completing reparameterize_paths_by_child()
by adding all possible paths. I have restructured the function to look
better and have one switch case instead of two. Also added more path
types including ForeignPath, for which I have added a FDW hook, with
documentation, for handling fdw_private. Please let me know if this
looks good to you. I am thinking of similar hook for CustomPath. I
will continue to add more path types to
reparameterize_path_by_child().

I am wondering whether we should bring 0007 he patche adjusting code
to work with child-joins before 0006, partition-wise join. 0006 needs
it, but 0007 doesn't depend upon 0006. Will that be any better?

[1] CAFjFpRcdrdsCRDbBu0J2pxwWbhb_sDWQUTVznBy_4XGr-p3+wA@mail.gmail.com,
subject "Asymmetry between parent and child wrt "false" quals"

In an attempt to test the geqo side of this patch, I reduced
geqo_threshold to 6 and set enable_partitionwise_join to to true and
tried following query, which crashed,

explain select * from prt, prt2, prt3, prt32, prt4, prt42 where prt.a
= prt2.b and prt3.a = prt32.b and prt4.a = prt42.b and prt2.a > 1000
order by prt.a desc;

Stack-trace for the crash is as follows,

Program received signal SIGSEGV, Segmentation fault.
0x00000000007a43d1 in find_param_path_info (rel=0x2d3fe30,
required_outer=0x2ff6d30) at relnode.c:1534
1534 if (bms_equal(ppi->ppi_req_outer, required_outer))
(gdb) bt
#0 0x00000000007a43d1 in find_param_path_info (rel=0x2d3fe30,
required_outer=0x2ff6d30) at relnode.c:1534
#1 0x000000000079b8bb in reparameterize_path_by_child
(root=0x2df7550, path=0x2f6dec0, child_rel=0x2d4a860) at
pathnode.c:3455
#2 0x000000000075be30 in try_nestloop_path (root=0x2df7550,
joinrel=0x2ff51b0, outer_path=0x2f96540, inner_path=0x2f6dec0,
pathkeys=0x0,
jointype=JOIN_INNER, extra=0x7fffe6b4e130) at joinpath.c:344
#3 0x000000000075d55b in match_unsorted_outer (root=0x2df7550,
joinrel=0x2ff51b0, outerrel=0x2d4a860, innerrel=0x2d3fe30,
jointype=JOIN_INNER,
extra=0x7fffe6b4e130) at joinpath.c:1389
#4 0x000000000075bc5f in add_paths_to_joinrel (root=0x2df7550,
joinrel=0x2ff51b0, outerrel=0x2d4a860, innerrel=0x2d3fe30,
jointype=JOIN_INNER,
sjinfo=0x3076bc8, restrictlist=0x3077168) at joinpath.c:234
#5 0x000000000075f1d5 in populate_joinrel_with_paths (root=0x2df7550,
rel1=0x2d3fe30, rel2=0x2d4a860, joinrel=0x2ff51b0, sjinfo=0x3076bc8,
restrictlist=0x3077168) at joinrels.c:793
#6 0x0000000000760107 in try_partition_wise_join (root=0x2df7550,
rel1=0x2d3f6d8, rel2=0x2d4a1a0, joinrel=0x30752f0,
parent_sjinfo=0x7fffe6b4e2d0,
parent_restrictlist=0x3075768) at joinrels.c:1401
#7 0x000000000075f0e6 in make_join_rel (root=0x2df7550,
rel1=0x2d3f6d8, rel2=0x2d4a1a0) at joinrels.c:744
#8 0x0000000000742053 in merge_clump (root=0x2df7550,
clumps=0x3075270, new_clump=0x30752a8, force=0 '\000') at
geqo_eval.c:260
#9 0x0000000000741f1c in gimme_tree (root=0x2df7550, tour=0x2ff2430,
num_gene=6) at geqo_eval.c:199
#10 0x0000000000741df5 in geqo_eval (root=0x2df7550, tour=0x2ff2430,
num_gene=6) at geqo_eval.c:102
#11 0x000000000074288a in random_init_pool (root=0x2df7550,
pool=0x2ff23d0) at geqo_pool.c:109
#12 0x00000000007422a6 in geqo (root=0x2df7550, number_of_rels=6,
initial_rels=0x2ff22d0) at geqo_main.c:114
#13 0x0000000000747f19 in make_rel_from_joinlist (root=0x2df7550,
joinlist=0x2dce940) at allpaths.c:2333
#14 0x0000000000744e7e in make_one_rel (root=0x2df7550,
joinlist=0x2dce940) at allpaths.c:182
#15 0x0000000000772df9 in query_planner (root=0x2df7550,
tlist=0x2dec2c0, qp_callback=0x777ce1 <standard_qp_callback>,
qp_extra=0x7fffe6b4e700)
at planmain.c:254

Please let me know if any more information is required on this.
--
Regards,
Rafia Sabih
EnterpriseDB: http://www.enterprisedb.com/

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

#82Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Rafia Sabih (#81)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

In an attempt to test the geqo side of this patch, I reduced
geqo_threshold to 6 and set enable_partitionwise_join to to true and
tried following query, which crashed,

explain select * from prt, prt2, prt3, prt32, prt4, prt42 where prt.a
= prt2.b and prt3.a = prt32.b and prt4.a = prt42.b and prt2.a > 1000
order by prt.a desc;

Stack-trace for the crash is as follows,

Nice catch. When reparameterize_path_by_child() may be running in a
temporary memory context while running in GEQO mode. It may add a new
PPI to base relation all in the temporary context. In the next GEQO
cycle, the ppilist will be clobbered since the temporary context is
reset for each geqo cycle. The fix is to allocate PPI in the same
memory context as the RelOptInfo similar to mark_dummy_rel().

I also found another problem. In geqo, we never call
generate_partition_wise_join_paths() which set cheapest paths for each
child-join. Because of this cheapest_*_paths are never set for those
rels, thus segfaulting in functions like sort_inner_and_outer() which
use those.

Here's patch fixing both the issues. Please let me know if it fixes
the issues you are seeing.
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company

Attachments:

geqo_fix.patchapplication/octet-stream; name=geqo_fix.patchDownload
diff --git a/src/backend/optimizer/geqo/geqo_eval.c b/src/backend/optimizer/geqo/geqo_eval.c
index b5cab0c..e5883a2 100644
--- a/src/backend/optimizer/geqo/geqo_eval.c
+++ b/src/backend/optimizer/geqo/geqo_eval.c
@@ -267,6 +267,9 @@ merge_clump(PlannerInfo *root, List *clumps, Clump *new_clump, bool force)
 				/* Create GatherPaths for any useful partial paths for rel */
 				generate_gather_paths(root, joinrel);
 
+				/* Create "append" paths for partitioned joins. */
+				generate_partition_wise_join_paths(root, joinrel);
+
 				/* Find and save the cheapest paths for this joinrel */
 				set_cheapest(joinrel);
 
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 64c4f6e..df7d452 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -31,6 +31,7 @@
 #include "parser/parsetree.h"
 #include "foreign/fdwapi.h"
 #include "utils/lsyscache.h"
+#include "utils/memutils.h"
 #include "utils/selfuncs.h"
 
 
@@ -3731,16 +3732,33 @@ reparameterize_path_by_child(PlannerInfo *root, Path *path,
 	/* If we already have a PPI for this parameterization, just return it */
 	new_ppi = find_param_path_info(new_path->parent, required_outer);
 
-	/* If not build a new one and link it to the list of PPIs. */
+	/*
+	 * If not build a new one and link it to the list of PPIs. When called
+	 * during GEQO join planning, we are in a short-lived memory context.  We
+	 * must make sure that the new PPI and its contents attached to a baserel
+	 * survives the GEQO cycle, else the baserel is trashed for future GEQO
+	 * cycles.  On the other hand, when we are adding new PPI to a joinrel
+	 * during GEQO, we don't want that to clutter the main planning context.
+	 * Upshot is that the best solution is to explicitly allocate new PPI in
+	 * the same context the given RelOptInfo is in.
+	 */
 	if (!new_ppi)
 	{
+		MemoryContext oldcontext;
+		RelOptInfo   *rel = path->parent;
+
+		/* No, so choose correct context to make the dummy path in */
+		oldcontext = MemoryContextSwitchTo(GetMemoryChunkContext(rel));
+
 		new_ppi = makeNode(ParamPathInfo);
-		new_ppi->ppi_req_outer = required_outer;
+		new_ppi->ppi_req_outer = bms_copy(required_outer);
 		new_ppi->ppi_rows = old_ppi->ppi_rows;
 		new_ppi->ppi_clauses = (List *) adjust_appendrel_attrs_multilevel(root,
 												 (Node *) old_ppi->ppi_clauses,
 																	child_rel);
-		new_path->parent->ppilist = lappend(new_path->parent->ppilist, new_ppi);
+		new_path->parent->ppilist = lappend(rel->ppilist, new_ppi);
+
+		MemoryContextSwitchTo(oldcontext);
 	}
 	else
 		bms_free(required_outer);
#83Rafia Sabih
rafia.sabih@enterprisedb.com
In reply to: Ashutosh Bapat (#82)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Mar 22, 2017 at 3:19 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

In an attempt to test the geqo side of this patch, I reduced
geqo_threshold to 6 and set enable_partitionwise_join to to true and
tried following query, which crashed,

explain select * from prt, prt2, prt3, prt32, prt4, prt42 where prt.a
= prt2.b and prt3.a = prt32.b and prt4.a = prt42.b and prt2.a > 1000
order by prt.a desc;

Stack-trace for the crash is as follows,

Nice catch. When reparameterize_path_by_child() may be running in a
temporary memory context while running in GEQO mode. It may add a new
PPI to base relation all in the temporary context. In the next GEQO
cycle, the ppilist will be clobbered since the temporary context is
reset for each geqo cycle. The fix is to allocate PPI in the same
memory context as the RelOptInfo similar to mark_dummy_rel().

I also found another problem. In geqo, we never call
generate_partition_wise_join_paths() which set cheapest paths for each
child-join. Because of this cheapest_*_paths are never set for those
rels, thus segfaulting in functions like sort_inner_and_outer() which
use those.

Here's patch fixing both the issues. Please let me know if it fixes
the issues you are seeing.

I tested the applied patch, it is fixing the reported issue.

--
Regards,
Rafia Sabih
EnterpriseDB: http://www.enterprisedb.com/

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

#84Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Ashutosh Bapat (#82)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

Here's set of updated patches rebased on
1148e22a82edc96172fc78855da392b6f0015c88.

I have fixed all the issues reported till now.

I have also completed reparameterize_path_by_child() for all the
required paths. There's no TODO there now. :) The function has grown
quite long now and might take some time to review. Given the size, I
am wondering whether we should separate that fix from the main
partition-wise join fix. That will make reviewing that function
easier, allowing a careful review. Here's the idea how that can be
done. As explained in the commit of 0009, the function is required in
case of lateral joins between partitioned relations. For a A LATERAL
JOIN B, B is the minimum required parameterization by A. Hence
children of A i.e. A1, A2 ... all require their paths to be
parameterized by B. When that comes to partition-wise joins, A1
requires its paths to be parameterized by B1 (matching partition from
B). Otherwise we can not create paths for A1B1. This means that we
require to reparameterize all A1's paths to be reparameterized by B1
using function reparameterize_paths_by_child(). So the function needs
to support reparameterization of all the paths; we do not know which
of those have survived add_path(). But if we disable partition-wise
join for lateral joins i.e. when direct_lateral_relids of one relation
contains the any subset of the relids in the other relation, we do not
need reparameterize_path_by_child(). Please let me know if this
strategy will help to make review and commit easier.

After the commit,
commit d3cc37f1d801a6b5cad9bf179274a8d767f1ee50
Author: Robert Haas <rhaas@postgresql.org>
Date: Tue Mar 21 09:48:04 2017 -0400

Don't scan partitioned tables.

We do not create any AppendRelInfos and hence RelOptInfos for the
partitioned tables. My approach to attach multi-level partitioned join
was to store RelOptInfos of immediate partitions in part_rels of
RelOptInfo of a partitioned table, thus maintaining a tree of
RelOptInfos reflecting partitioning tree. This allows to add append
paths to intermediate RelOptInfos, flattening them as we go up the
partitioning hierarchy. With no RelOptInfos for intermediate
partitions, we can support multi-level partition-wise join only in
limited cases when the partitioning hierarchy of the joining table
exactly matches. Please refer [1]/messages/by-id/CAFjFpRceMmx26653XFAYvc5KVQcrzcKScVFqZdbXV=kB8Akkqg@mail.gmail.com for some more discussion.

I think we need the RelOptInfos for the partitions, which are
partitioned to hold the "append" paths containing paths from their
children and to match the partitions in partition-wise join. Similar
hierarchy will be created for partitioned joins, with partitioned join
partitions. So, I have not changed the multi-level partition-wise join
support patches. After applying 0011-0013 the multi-level partitioning
tests fail with error "could not find the RelOptInfo of a partition
with oid", since it does not find the RelOptInfos of partitions which
are partitioned.

[1]: /messages/by-id/CAFjFpRceMmx26653XFAYvc5KVQcrzcKScVFqZdbXV=kB8Akkqg@mail.gmail.com

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

Attachments:

pg_dp_join_patches_v12.zipapplication/zip; name=pg_dp_join_patches_v12.zipDownload
#85Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Rafia Sabih (#83)
Re: Partition-wise join for join between (declaratively) partitioned tables

I tested the applied patch, it is fixing the reported issue.

Thanks for the confirmation Rafia. I have included the fix in the
latest set of patches.
--
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

#86Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#84)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Mar 22, 2017 at 8:46 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

I have also completed reparameterize_path_by_child() for all the
required paths. There's no TODO there now. :) The function has grown
quite long now and might take some time to review. Given the size, I
am wondering whether we should separate that fix from the main
partition-wise join fix. That will make reviewing that function
easier, allowing a careful review. Here's the idea how that can be
done. As explained in the commit of 0009, the function is required in
case of lateral joins between partitioned relations. For a A LATERAL
JOIN B, B is the minimum required parameterization by A. Hence
children of A i.e. A1, A2 ... all require their paths to be
parameterized by B. When that comes to partition-wise joins, A1
requires its paths to be parameterized by B1 (matching partition from
B). Otherwise we can not create paths for A1B1. This means that we
require to reparameterize all A1's paths to be reparameterized by B1
using function reparameterize_paths_by_child(). So the function needs
to support reparameterization of all the paths; we do not know which
of those have survived add_path(). But if we disable partition-wise
join for lateral joins i.e. when direct_lateral_relids of one relation
contains the any subset of the relids in the other relation, we do not
need reparameterize_path_by_child(). Please let me know if this
strategy will help to make review and commit easier.

In my testing last week, reparameterize_path_by_child() was essential
for nested loops to work properly, even without LATERAL. Without it,
the parameterized path ends up containing vars that reference the
parent varno instead of the child varno. That confused later planner
stages so that those Vars did not get replaced with Param during
replace_nestloop_params(), eventually resulting in a crash at
execution time. Based on that experiment, I think we could consider
having reparameterize_path_by_child() handle only scan paths as
reparameterize_path() does, and just give up on plans like this:

Append
-> Left Join
-> Scan on a
-> Inner Join
-> Index Scan on b
-> Index Scan on c
[repeat for each partition]

But I doubt we can get by without it altogether.

--
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

#87Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#86)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Mar 22, 2017 at 6:32 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Wed, Mar 22, 2017 at 8:46 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

I have also completed reparameterize_path_by_child() for all the
required paths. There's no TODO there now. :) The function has grown
quite long now and might take some time to review. Given the size, I
am wondering whether we should separate that fix from the main
partition-wise join fix. That will make reviewing that function
easier, allowing a careful review. Here's the idea how that can be
done. As explained in the commit of 0009, the function is required in
case of lateral joins between partitioned relations. For a A LATERAL
JOIN B, B is the minimum required parameterization by A. Hence
children of A i.e. A1, A2 ... all require their paths to be
parameterized by B. When that comes to partition-wise joins, A1
requires its paths to be parameterized by B1 (matching partition from
B). Otherwise we can not create paths for A1B1. This means that we
require to reparameterize all A1's paths to be reparameterized by B1
using function reparameterize_paths_by_child(). So the function needs
to support reparameterization of all the paths; we do not know which
of those have survived add_path(). But if we disable partition-wise
join for lateral joins i.e. when direct_lateral_relids of one relation
contains the any subset of the relids in the other relation, we do not
need reparameterize_path_by_child(). Please let me know if this
strategy will help to make review and commit easier.

In my testing last week, reparameterize_path_by_child() was essential
for nested loops to work properly, even without LATERAL. Without it,
the parameterized path ends up containing vars that reference the
parent varno instead of the child varno. That confused later planner
stages so that those Vars did not get replaced with Param during
replace_nestloop_params(), eventually resulting in a crash at
execution time.

I half-described the solution. Sorry. Along-with disabling
partition-wise lateral joins, we have to disable nested loop
child-joins where inner child is parameterized by the parent of the
outer one. We will still have nestloop join between parents where
inner relation is parameterized by the outer and every child of inner
is parameterized by the outer. But we won't create nest loop joins
where inner child is parameterized by the outer child, where we
require reparameterize_path_by_child. We will loose this optimization
only till we get reparameterize_path_by_child() committed. Basically,
in try_nestloop_path() (in the patch 0009), if
(PATH_PARAM_BY_PARENT(inner_path, outer_path->parent)), give up
creating nest loop path. That shouldn't create any problems.

Did you experiment with this change in try_nestloop_path()? Can you
please share the testcase? I will take a look at it.

Based on that experiment, I think we could consider
having reparameterize_path_by_child() handle only scan paths as
reparameterize_path() does, and just give up on plans like this:

Append
-> Left Join
-> Scan on a
-> Inner Join
-> Index Scan on b
-> Index Scan on c
[repeat for each partition]

I am assuming that a, b and c are partitions of A, B and C resp. which
are being joined and both or one of the scans on b and c are
parameteried by a or scan of c is parameterized by b.

I don't think we will get away by supporting just scan paths, since
the inner side of lateral join can be any paths not just scan path. Or
you are suggesting that we disable partition-wise lateral join and
support reparameterization of only scan paths?

--
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

#88Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#87)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Mar 22, 2017 at 9:59 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

In my testing last week, reparameterize_path_by_child() was essential
for nested loops to work properly, even without LATERAL. Without it,
the parameterized path ends up containing vars that reference the
parent varno instead of the child varno. That confused later planner
stages so that those Vars did not get replaced with Param during
replace_nestloop_params(), eventually resulting in a crash at
execution time.

I half-described the solution. Sorry. Along-with disabling
partition-wise lateral joins, we have to disable nested loop
child-joins where inner child is parameterized by the parent of the
outer one. We will still have nestloop join between parents where
inner relation is parameterized by the outer and every child of inner
is parameterized by the outer. But we won't create nest loop joins
where inner child is parameterized by the outer child, where we
require reparameterize_path_by_child. We will loose this optimization
only till we get reparameterize_path_by_child() committed. Basically,
in try_nestloop_path() (in the patch 0009), if
(PATH_PARAM_BY_PARENT(inner_path, outer_path->parent)), give up
creating nest loop path. That shouldn't create any problems.

Did you experiment with this change in try_nestloop_path()? Can you
please share the testcase? I will take a look at it.

I didn't save the test case. It was basically just forcing a
partitionwise nestloop join between two equipartitioned tables, with
the calls to adjust_appendrel_attrs() ripped out of
reparameterize_path_by_child(), just to see what would break.

Based on that experiment, I think we could consider
having reparameterize_path_by_child() handle only scan paths as
reparameterize_path() does, and just give up on plans like this:

Append
-> Left Join
-> Scan on a
-> Inner Join
-> Index Scan on b
-> Index Scan on c
[repeat for each partition]

I am assuming that a, b and c are partitions of A, B and C resp. which
are being joined and both or one of the scans on b and c are
parameteried by a or scan of c is parameterized by b.

Yes.

I don't think we will get away by supporting just scan paths, since
the inner side of lateral join can be any paths not just scan path. Or
you are suggesting that we disable partition-wise lateral join and
support reparameterization of only scan paths?

I think if you can do a straight-up partitionwise nested loop between
two tables A and B, that's pretty bad. But if there are more complex
cases that involve parameterizing entire join trees which aren't
covered, that's less bad. Parallel query almost entirely punts on
LATERAL right now, and nobody's complained yet. I'm sure that'll need
to get fixed someday, but not today.

--
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

#89Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#84)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Mar 22, 2017 at 8:46 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Here's set of updated patches rebased on
1148e22a82edc96172fc78855da392b6f0015c88.

I have fixed all the issues reported till now.

I don't understand why patch 0001 ends up changing every existing test
for RELOPT_JOINREL anywhere in the source tree to use IS_JOIN_REL(),
yet none of the existing tests for RELOPT_OTHER_MEMBER_REL end up
getting changed to use IS_OTHER_REL(). That's very surprising. Some
of those tests are essentially checking for something that is going to
have a scan plan rather than a join or upper plan, and those tests
probably don't need to be modified; for example, the test in
set_rel_consider_parallel() is obviously of this type. But others are
testing whether we've got some kind of child rel, and those seem like
they might need work. Going through a few specific examples:

- generate_join_implied_equalities_for_ecs() assumes that any child
rel is an other member rel.
- generate_join_implied_equalities_broken() assumes that any child rel
is an other member rel.
- generate_implied_equalities_for_column() set is_child_rel on the
assumption that only an other member rel can be a child rel.
- eclass_useful_for_merging() assumes that the only kind of child rel
is an other member rel.
- find_childrel_appendrelinfo() assumes that any child rel is an other
member rel.
- find_childrel_top_parent() and find_childrel_parents() assume that
children must be other member rels and their parents must be baserels.
- adjust_appendrel_attrs_multilevel() assumes that children must be
other member rels and their parents must be baserels.

It's possible that, for various reasons, none of these code paths
would ever be reachable by a child join, but it doesn't look likely to
me. And even if that's true, some comment updates are probably
needed, and maybe some renaming of functions too.

In postgres_fdw, get_useful_ecs_for_relation() assumes that any child
rel is an other member rel. I'm not sure if we're hoping that
partitionwise join will work with postgres_fdw's join pushdown out of
the chute, but clearly this would need to be adjusted to have any
chance of being right.

Some that seem OK:

- set_rel_consider_parallel() is fine.
- set_append_rel_size() is only going to be called for baserels or
their children, so it's fine.
- relation_excluded_by_constraints() is only intended to be called on
baserels or their children, so it's fine.
- check_index_predicates() is only intended to be called on baserels
or their children, so it's fine.
- query_planner() loops over baserels and their children, so it's fine.

Perhaps we could introduce an IS_BASEREL_OR_CHILD() test that could be
used in some of these places, just for symmetry. The point is that
there are really three questions here: (1) is it some kind of baserel
(parent or child)? (2) is it some kind of joinrel (parent or child)?
and (3) is it some kind of child (baserel or join)? Right now, both
#2 and #3 are tested by just comparing against
RELOPT_OTHER_MEMBER_REL, but they become different tests as soon as we
add child joinrels. The goal of 0001, IMV, ought to be to try to
figure out which of #1, #2, and #3 is being checked in each case and
make that clear via use of an appropriate macro. (If is-other-baserel
is the real test, then fine, but I bet that's a rare case.)

--
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

#90Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#88)
Re: Partition-wise join for join between (declaratively) partitioned tables

I don't think we will get away by supporting just scan paths, since
the inner side of lateral join can be any paths not just scan path. Or
you are suggesting that we disable partition-wise lateral join and
support reparameterization of only scan paths?

I think if you can do a straight-up partitionwise nested loop between
two tables A and B, that's pretty bad.

Ok.

But if there are more complex
cases that involve parameterizing entire join trees which aren't
covered, that's less bad. Parallel query almost entirely punts on
LATERAL right now, and nobody's complained yet. I'm sure that'll need
to get fixed someday, but not today.

Ok.

I am suggesting this possibility in case we run of time to review and
commit reparameterize_path_by_child() entirely. If we can do that, I
will be happy.

In case, we have to include a stripped down version of
reparameterize_path_by_child(), with which I am fine too, we will need
to disable LATERAL joins, so that we don't end up with an error "could
not devise a query plan for the given query".

--
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

#91Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#89)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Mar 23, 2017 at 1:17 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Wed, Mar 22, 2017 at 8:46 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Here's set of updated patches rebased on
1148e22a82edc96172fc78855da392b6f0015c88.

I have fixed all the issues reported till now.

I don't understand why patch 0001 ends up changing every existing test
for RELOPT_JOINREL anywhere in the source tree to use IS_JOIN_REL(),
yet none of the existing tests for RELOPT_OTHER_MEMBER_REL end up
getting changed to use IS_OTHER_REL(). That's very surprising. Some
of those tests are essentially checking for something that is going to
have a scan plan rather than a join or upper plan, and those tests
probably don't need to be modified; for example, the test in
set_rel_consider_parallel() is obviously of this type. But others are
testing whether we've got some kind of child rel, and those seem like
they might need work. Going through a few specific examples:

- generate_join_implied_equalities_for_ecs() assumes that any child
rel is an other member rel.
- generate_join_implied_equalities_broken() assumes that any child rel
is an other member rel.

Fixed those.

- generate_implied_equalities_for_column() set is_child_rel on the
assumption that only an other member rel can be a child rel.

This function is called for indexes, which are not defined on the
join relations. So, we shouldn't worry about child-joins here. I have
added an assertion in there to make sure that that function gets
called only for base and "other" member rels.

- eclass_useful_for_merging() assumes that the only kind of child rel
is an other member rel.

This was being fixed in a later patch which had many small fixes for
handling child-joins. But now I have moved that fix into 0001.

- find_childrel_appendrelinfo() assumes that any child rel is an other
member rel.

The function is called for "other" member relation only. For joins we
use find_appinfos_by_relids() We could replace
find_childrel_appendrelinfo() with find_appinfos_by_relids(), which
does same thing as find_childrel_appendrelinfo() for a relids set. But
find_appinfos_by_relids() returns a list of AppendRelInfos, hence
using it instead of find_childrel_appendrelinfo() will spend some
memory and CPU cycles in creating a one element list and then
extracting that element out of the list. So, I have not replaced
usages of find_childrel_appendrelinfo() with
find_appinfos_by_relids(). This also simplifies changes to
get_useful_ecs_for_relation().

- find_childrel_top_parent() and find_childrel_parents() assume that
children must be other member rels and their parents must be baserels.

For partition-wise join implementation we save relids of topmost
parent in RelOptInfo of child. We can directly use that instead of
calling find_childrel_top_parent(). So, in 0001 I am adding
top_parent_relids in RelOptInfo and getting rid of
find_childrel_top_parent(). This also fixes
get_useful_ecs_for_relation() in a better way. find_childrel_parents()
is called only for simple relations not joins, since it's callers are
called only for simple relations. I have added an assertion to that
effect.

- adjust_appendrel_attrs_multilevel() assumes that children must be
other member rels and their parents must be baserels.

It was being fixed in a later patch. In the attached patch set 0001
changes it to use IS_OTHER_REL().

It's possible that, for various reasons, none of these code paths
would ever be reachable by a child join, but it doesn't look likely to
me. And even if that's true, some comment updates are probably
needed, and maybe some renaming of functions too.

Now commit messages of 0001 explains which instances of
RELOPT_OTHER_MEMBER_REL and RELOPT_BASEREL have been changed, and
which have been retained and why. Also, added assertions wherever
necessary.

In postgres_fdw, get_useful_ecs_for_relation() assumes that any child
rel is an other member rel. I'm not sure if we're hoping that
partitionwise join will work with postgres_fdw's join pushdown out of
the chute, but clearly this would need to be adjusted to have any
chance of being right.

Fixed this as explained above.

Some that seem OK:

- set_rel_consider_parallel() is fine.
- set_append_rel_size() is only going to be called for baserels or
their children, so it's fine.
- relation_excluded_by_constraints() is only intended to be called on
baserels or their children, so it's fine.
- check_index_predicates() is only intended to be called on baserels
or their children, so it's fine.
- query_planner() loops over baserels and their children, so it's fine.

Right.

Perhaps we could introduce an IS_BASEREL_OR_CHILD() test that could be
used in some of these places, just for symmetry.

I was wondering about this as well. Although, I though it better not
to touch base relations in partition-wise join. But now, I have added
that macro and adjusted corresponding tests in the code. See 0001.

You may actually want to squash 0001 and 0002 into a single patch. But
for now, I have left those as separate.

The point is that
there are really three questions here: (1) is it some kind of baserel
(parent or child)? (2) is it some kind of joinrel (parent or child)?
and (3) is it some kind of child (baserel or join)? Right now, both
#2 and #3 are tested by just comparing against
RELOPT_OTHER_MEMBER_REL, but they become different tests as soon as we
add child joinrels. The goal of 0001, IMV, ought to be to try to
figure out which of #1, #2, and #3 is being checked in each case and
make that clear via use of an appropriate macro. (If is-other-baserel
is the real test, then fine, but I bet that's a rare case.)

Agreed. I have gone through all the cases, and fixed the necessary
ones as explained above and in the commit messages of 0001.

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

Attachments:

pg_dp_join_patches_v13.zipapplication/zip; name=pg_dp_join_patches_v13.zipDownload
#92Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Ashutosh Bapat (#91)
Re: Partition-wise join for join between (declaratively) partitioned tables

Hi Ashutosh,

On 2017/03/23 21:48, Ashutosh Bapat wrote:

I have fixed all the issues reported till now.

In patch 0007, the following code in have_partkey_equi_join() looks
potentially unsafe:

/*
* If the clause refers to different partition keys from
* both relations, it can not be used for partition-wise join.
*/
if (ipk1 != ipk2)
continue;

/*
* The clause allows partition-wise join if only it uses the same
* operator family as that specified by the partition key.
*/
if (!list_member_oid(rinfo->mergeopfamilies,
part_scheme->partopfamily[ipk1]))
continue;

What if ipk1 and ipk2 both turn out to be -1? Accessing
part_schem->partopfamily[ipk1] would be incorrect, no?

Thanks,
Amit

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

#93Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Ashutosh Bapat (#91)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

Hi Ashutosh,

On 2017/03/23 21:48, Ashutosh Bapat wrote:

I have fixed all the issues reported till now.

I've tried to fix your 0012 patch (Multi-level partitioned table
expansion) considering your message earlier on this thread [1]/messages/by-id/CAFjFpRefs5ZMnxQ2vP9v5zOtWtNPuiMYc01sb1SWjCOB1CT=uQ@mail.gmail.com.
Especially the fact that no AppendRelInfo and RelOptInfo are allocated for
partitioned child tables as of commit d3cc37f1d [2]https://git.postgresql.org/gitweb/?p=postgresql.git;a=commit;h=d3cc37f1d. I've fixed
expand_inherited_rtentry() such that AppendRelInfo *is* allocated for a
partitioned child RTEs whose rte->inh is set to true. Such an RTE is
recursively expanded with that RTE the parent.

Also as I mentioned elsewhere [3]/messages/by-id/744d20fe-fc7b-f89e-8d06-6496ec537b86@lab.ntt.co.jp, the multi-level inheritance expansion
of partitioned table will break update/delete for partitioned table, which
is because inheritance_planner() is not ready to handle inheritance sets
structured that way. I tried to refactor inheritance_planner() such that
its core logic can be recursively invoked for partitioned child RTEs. The
resulting child paths and other auxiliary information related to planning
across the hierarchy are maintained in one place using a struct to hold
the same in a few flat lists. The refactoring didn't break any existing
tests and a couple of new tests are added to check that it indeed works
for multi-level partitioned tables expanded using new multi-level structure.

There is some test failure in 0014 (Multi-level partition-wise join
tests), probably because of the changes I made to 0012, which I didn't get
time to check why, although I've checked using an example that multi-level
join planning still works, so it's not completely broken either.

Thanks,
Amit

[1]: /messages/by-id/CAFjFpRefs5ZMnxQ2vP9v5zOtWtNPuiMYc01sb1SWjCOB1CT=uQ@mail.gmail.com
/messages/by-id/CAFjFpRefs5ZMnxQ2vP9v5zOtWtNPuiMYc01sb1SWjCOB1CT=uQ@mail.gmail.com

[2]: https://git.postgresql.org/gitweb/?p=postgresql.git;a=commit;h=d3cc37f1d

[3]: /messages/by-id/744d20fe-fc7b-f89e-8d06-6496ec537b86@lab.ntt.co.jp
/messages/by-id/744d20fe-fc7b-f89e-8d06-6496ec537b86@lab.ntt.co.jp

Attachments:

0012-Multi-level-partitioned-table-expansion.patchtext/x-diff; name=0012-Multi-level-partitioned-table-expansion.patchDownload
From 7051b9cb4760908e23e64969988b58fcb466868e Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Fri, 10 Feb 2017 13:50:14 +0530
Subject: [PATCH 12/14] Multi-level partitioned table expansion.

Construct inheritance hierarchy of a partitioned table to reflect the
partition hierarchy.

Commit d3cc37f1d got rid of AppendRelInfos for *all* partitioned child
child tables.  But we'd need one for recursively expandded partitioned
children.  In its absence, some of the later planning stages would not be
able to access the partitions below the first level, because they are not
directly linked to the root parent.  Fix that by allocating one.  That
means such children also get a RelOptInfo, which also wasn't allocated
previously.

Expanding partitioned table inheritance this new way means we need to
teach a few places how to traverse the hierarchy, such as propagating
lateral join information from the root table down the partition
hierarchy.

Further, inheritance_planner() needs refactoring to enable recursive
invocation of its core logic.  While the query_planner() can handle the
new partition hierarchical structure just fine because of the way the
relevant code works, inheritance_planner() would miss any child tables
below the first level, because it can't do recursion.  Refactor it so
that it can.
---
 src/backend/optimizer/plan/initsplan.c |  14 +-
 src/backend/optimizer/plan/planner.c   | 290 +++++++++++++++++++++------------
 src/backend/optimizer/prep/prepunion.c |  69 ++++++--
 src/test/regress/expected/inherit.out  |  22 +++
 src/test/regress/sql/inherit.sql       |  17 ++
 5 files changed, 292 insertions(+), 120 deletions(-)

diff --git a/src/backend/optimizer/plan/initsplan.c b/src/backend/optimizer/plan/initsplan.c
index b4ac224a7a..20774b29fe 100644
--- a/src/backend/optimizer/plan/initsplan.c
+++ b/src/backend/optimizer/plan/initsplan.c
@@ -630,7 +630,19 @@ create_lateral_join_info(PlannerInfo *root)
 	{
 		RelOptInfo *brel = root->simple_rel_array[rti];
 
-		if (brel == NULL || brel->reloptkind != RELOPT_BASEREL)
+		if (brel == NULL)
+			continue;
+
+		/*
+		 * If an "other rel" RTE is a "partitioned table", we must propagate
+		 * the lateral info inherited all the way from the root parent to its
+		 * children. That's because the children are not linked directly with
+		 * the root parent via AppendRelInfo's unlike in case of a regular
+		 * inheritance set (see expand_inherited_rtentry()).  Failing to
+		 * do this would result in those children not getting marked with the
+		 * appropriate lateral info.
+		 */
+		if (brel->reloptkind != RELOPT_BASEREL && !brel->part_scheme)
 			continue;
 
 		if (root->simple_rte_array[rti]->inh)
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 4487683196..59cb559b9e 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -91,10 +91,28 @@ typedef struct
 	List	   *groupClause;	/* overrides parse->groupClause */
 } standard_qp_extra;
 
+/* Result of a given invocation of inheritance_planner_guts() */
+typedef struct
+{
+	Index 	nominalRelation;
+	List   *partitioned_rels;
+	List   *resultRelations;
+	List   *subpaths;
+	List   *subroots;
+	List   *withCheckOptionLists;
+	List   *returningLists;
+	List   *final_rtable;
+	List   *init_plans;
+	int		save_rel_array_size;
+	RelOptInfo **save_rel_array;
+} inheritance_planner_result;
+
 /* Local functions */
 static Node *preprocess_expression(PlannerInfo *root, Node *expr, int kind);
 static void preprocess_qual_conditions(PlannerInfo *root, Node *jtnode);
 static void inheritance_planner(PlannerInfo *root);
+static void inheritance_planner_guts(PlannerInfo *root,
+						 inheritance_planner_result *inhpres);
 static void grouping_planner(PlannerInfo *root, bool inheritance_update,
 				 double tuple_fraction);
 static void preprocess_rowmarks(PlannerInfo *root);
@@ -994,25 +1012,105 @@ preprocess_phv_expression(PlannerInfo *root, Expr *expr)
 static void
 inheritance_planner(PlannerInfo *root)
 {
+	inheritance_planner_result inhpres;
+	Query	   *parse = root->parse;
+	RelOptInfo *final_rel;
+	Index		rti;
+	int			final_rtable_len;
+	ListCell   *lc;
+	List	   *rowMarks;
+
+	/*
+	 * Away we go... Although the inheritance hierarchy to be processed might
+	 * be represented in a non-flat manner, some of the elements needed to
+	 * create the final ModifyTable path are always returned in a flat list
+	 * structure.
+	 */
+	memset(&inhpres, 0, sizeof(inhpres));
+	inheritance_planner_guts(root, &inhpres);
+
+	/* Result path must go into outer query's FINAL upperrel */
+	final_rel = fetch_upper_rel(root, UPPERREL_FINAL, NULL);
+
+	/*
+	 * We don't currently worry about setting final_rel's consider_parallel
+	 * flag in this case, nor about allowing FDWs or create_upper_paths_hook
+	 * to get control here.
+	 */
+
+	/*
+	 * If we managed to exclude every child rel, return a dummy plan; it
+	 * doesn't even need a ModifyTable node.
+	 */
+	if (inhpres.subpaths == NIL)
+	{
+		set_dummy_rel_pathlist(final_rel);
+		return;
+	}
+
+	/*
+	 * Put back the final adjusted rtable into the master copy of the Query.
+	 * (We mustn't do this if we found no non-excluded children.)
+	 */
+	parse->rtable = inhpres.final_rtable;
+	root->simple_rel_array_size = inhpres.save_rel_array_size;
+	root->simple_rel_array = inhpres.save_rel_array;
+	/* Must reconstruct master's simple_rte_array, too */
+	final_rtable_len = list_length(inhpres.final_rtable);
+	root->simple_rte_array = (RangeTblEntry **)
+								palloc0((final_rtable_len + 1) *
+											sizeof(RangeTblEntry *));
+	rti = 1;
+	foreach(lc, inhpres.final_rtable)
+	{
+		RangeTblEntry *rte = (RangeTblEntry *) lfirst(lc);
+
+		root->simple_rte_array[rti++] = rte;
+	}
+
+	/*
+	 * If there was a FOR [KEY] UPDATE/SHARE clause, the LockRows node will
+	 * have dealt with fetching non-locked marked rows, else we need to have
+	 * ModifyTable do that.
+	 */
+	if (parse->rowMarks)
+		rowMarks = NIL;
+	else
+		rowMarks = root->rowMarks;
+
+	/* Create Path representing a ModifyTable to do the UPDATE/DELETE work */
+	add_path(final_rel, (Path *)
+			 create_modifytable_path(root, final_rel,
+									 parse->commandType,
+									 parse->canSetTag,
+									 inhpres.nominalRelation,
+									 inhpres.partitioned_rels,
+									 inhpres.resultRelations,
+									 inhpres.subpaths,
+									 inhpres.subroots,
+									 inhpres.withCheckOptionLists,
+									 inhpres.returningLists,
+									 rowMarks,
+									 NULL,
+									 SS_assign_special_param(root)));
+}
+
+/*
+ * inheritance_planner_guts
+ *	  Recursive guts of inheritance_planner
+ */
+static void
+inheritance_planner_guts(PlannerInfo *root,
+						 inheritance_planner_result *inhpres)
+{
 	Query	   *parse = root->parse;
 	int			parentRTindex = parse->resultRelation;
 	Bitmapset  *subqueryRTindexes;
 	Bitmapset  *modifiableARIindexes;
-	int			nominalRelation = -1;
-	List	   *final_rtable = NIL;
-	int			save_rel_array_size = 0;
-	RelOptInfo **save_rel_array = NULL;
-	List	   *subpaths = NIL;
-	List	   *subroots = NIL;
-	List	   *resultRelations = NIL;
-	List	   *withCheckOptionLists = NIL;
-	List	   *returningLists = NIL;
-	List	   *rowMarks;
-	RelOptInfo *final_rel;
+	bool		nominalRelationSet = false;
 	ListCell   *lc;
 	Index		rti;
 	RangeTblEntry *parent_rte;
-	List		  *partitioned_rels = NIL;
 
 	Assert(parse->commandType != CMD_INSERT);
 
@@ -1080,7 +1178,10 @@ inheritance_planner(PlannerInfo *root)
 	 */
 	parent_rte = rt_fetch(parentRTindex, root->parse->rtable);
 	if (parent_rte->relkind == RELKIND_PARTITIONED_TABLE)
-		nominalRelation = parentRTindex;
+	{
+		inhpres->nominalRelation = parentRTindex;
+		nominalRelationSet = true;
+	}
 
 	/*
 	 * And now we can get on with generating a plan for each child table.
@@ -1089,6 +1190,7 @@ inheritance_planner(PlannerInfo *root)
 	{
 		AppendRelInfo *appinfo = (AppendRelInfo *) lfirst(lc);
 		PlannerInfo *subroot;
+		Index	childRTindex = appinfo->child_relid;
 		RangeTblEntry *child_rte;
 		RelOptInfo *sub_final_rel;
 		Path	   *subpath;
@@ -1120,7 +1222,7 @@ inheritance_planner(PlannerInfo *root)
 		 * child rel (they've already been transformed properly for that).
 		 */
 		parent_rte = rt_fetch(parentRTindex, subroot->parse->rtable);
-		child_rte = rt_fetch(appinfo->child_relid, subroot->parse->rtable);
+		child_rte = rt_fetch(childRTindex, subroot->parse->rtable);
 		child_rte->securityQuals = parent_rte->securityQuals;
 		parent_rte->securityQuals = NIL;
 
@@ -1165,7 +1267,8 @@ inheritance_planner(PlannerInfo *root)
 		 * These won't be referenced, so there's no need to make them very
 		 * valid-looking.
 		 */
-		while (list_length(subroot->parse->rtable) < list_length(final_rtable))
+		while (list_length(subroot->parse->rtable) <
+										list_length(inhpres->final_rtable))
 			subroot->parse->rtable = lappend(subroot->parse->rtable,
 											 makeNode(RangeTblEntry));
 
@@ -1177,7 +1280,7 @@ inheritance_planner(PlannerInfo *root)
 		 * since subquery RTEs couldn't contain any references to the target
 		 * rel.
 		 */
-		if (final_rtable != NIL && subqueryRTindexes != NULL)
+		if (inhpres->final_rtable != NIL && subqueryRTindexes != NULL)
 		{
 			ListCell   *lr;
 
@@ -1222,6 +1325,47 @@ inheritance_planner(PlannerInfo *root)
 			}
 		}
 
+		/*
+		 * Recurse for a partitioned child table.  We shouldn't be planning
+		 * a partitioned RTE as a child member, which is what the code after
+		 * this block does.
+		 */
+		if (child_rte->inh)
+		{
+			inheritance_planner_result	child_inhpres;
+
+			Assert(child_rte->relkind == RELKIND_PARTITIONED_TABLE);
+
+			/* During the recursive invocation, this child is the parent. */
+			subroot->parse->resultRelation = childRTindex;
+			memset(&child_inhpres, 0, sizeof(child_inhpres));
+			inheritance_planner_guts(subroot, &child_inhpres);
+
+			inhpres->partitioned_rels = list_concat(inhpres->partitioned_rels,
+											child_inhpres.partitioned_rels);
+			inhpres->resultRelations = list_concat(inhpres->resultRelations,
+											child_inhpres.resultRelations);
+			inhpres->subpaths = list_concat(inhpres->subpaths,
+											child_inhpres.subpaths);
+			inhpres->subroots = list_concat(inhpres->subroots,
+											child_inhpres.subroots);
+			inhpres->withCheckOptionLists =
+									list_concat(inhpres->withCheckOptionLists,
+										child_inhpres.withCheckOptionLists);
+			inhpres->returningLists = list_concat(inhpres->returningLists,
+											child_inhpres.returningLists);
+			if (child_inhpres.final_rtable != NIL)
+				inhpres->final_rtable = child_inhpres.final_rtable;
+			if (child_inhpres.init_plans != NIL)
+				inhpres->init_plans = child_inhpres.init_plans;
+			if (child_inhpres.save_rel_array_size != 0)
+			{
+				inhpres->save_rel_array_size = child_inhpres.save_rel_array_size;
+				inhpres->save_rel_array = child_inhpres.save_rel_array;
+			}
+			continue;
+		}
+
 		/* There shouldn't be any OJ info to translate, as yet */
 		Assert(subroot->join_info_list == NIL);
 		/* and we haven't created PlaceHolderInfos, either */
@@ -1253,8 +1397,11 @@ inheritance_planner(PlannerInfo *root)
 		 * the duplicate child RTE added for the parent does not appear
 		 * anywhere else in the plan tree.
 		 */
-		if (nominalRelation < 0)
-			nominalRelation = appinfo->child_relid;
+		if (!nominalRelationSet)
+		{
+			inhpres->nominalRelation = childRTindex;
+			nominalRelationSet = true;
+		}
 
 		/*
 		 * Select cheapest path in case there's more than one.  We always run
@@ -1277,12 +1424,12 @@ inheritance_planner(PlannerInfo *root)
 		 * becomes the initial contents of final_rtable; otherwise, append
 		 * just its modified subquery RTEs to final_rtable.
 		 */
-		if (final_rtable == NIL)
-			final_rtable = subroot->parse->rtable;
+		if (inhpres->final_rtable == NIL)
+			inhpres->final_rtable = subroot->parse->rtable;
 		else
-			final_rtable = list_concat(final_rtable,
-									   list_copy_tail(subroot->parse->rtable,
-												 list_length(final_rtable)));
+			inhpres->final_rtable = list_concat(inhpres->final_rtable,
+										list_copy_tail(subroot->parse->rtable,
+										 list_length(inhpres->final_rtable)));
 
 		/*
 		 * We need to collect all the RelOptInfos from all child plans into
@@ -1291,109 +1438,48 @@ inheritance_planner(PlannerInfo *root)
 		 * have to propagate forward the RelOptInfos that were already built
 		 * in previous children.
 		 */
-		Assert(subroot->simple_rel_array_size >= save_rel_array_size);
-		for (rti = 1; rti < save_rel_array_size; rti++)
+		Assert(subroot->simple_rel_array_size >= inhpres->save_rel_array_size);
+		for (rti = 1; rti < inhpres->save_rel_array_size; rti++)
 		{
-			RelOptInfo *brel = save_rel_array[rti];
+			RelOptInfo *brel = inhpres->save_rel_array[rti];
 
 			if (brel)
 				subroot->simple_rel_array[rti] = brel;
 		}
-		save_rel_array_size = subroot->simple_rel_array_size;
-		save_rel_array = subroot->simple_rel_array;
+		inhpres->save_rel_array_size = subroot->simple_rel_array_size;
+		inhpres->save_rel_array = subroot->simple_rel_array;
 
 		/* Make sure any initplans from this rel get into the outer list */
-		root->init_plans = subroot->init_plans;
+		inhpres->init_plans = subroot->init_plans;
 
 		/* Build list of sub-paths */
-		subpaths = lappend(subpaths, subpath);
+		inhpres->subpaths = lappend(inhpres->subpaths, subpath);
 
 		/* Build list of modified subroots, too */
-		subroots = lappend(subroots, subroot);
+		inhpres->subroots = lappend(inhpres->subroots, subroot);
 
 		/* Build list of target-relation RT indexes */
-		resultRelations = lappend_int(resultRelations, appinfo->child_relid);
+		inhpres->resultRelations = lappend_int(inhpres->resultRelations,
+											   childRTindex);
 
 		/* Build lists of per-relation WCO and RETURNING targetlists */
 		if (parse->withCheckOptions)
-			withCheckOptionLists = lappend(withCheckOptionLists,
-										   subroot->parse->withCheckOptions);
+			inhpres->withCheckOptionLists =
+										lappend(inhpres->withCheckOptionLists,
+											subroot->parse->withCheckOptions);
 		if (parse->returningList)
-			returningLists = lappend(returningLists,
-									 subroot->parse->returningList);
-
+			inhpres->returningLists = lappend(inhpres->returningLists,
+											  subroot->parse->returningList);
 		Assert(!parse->onConflict);
 	}
 
 	if (parent_rte->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		partitioned_rels = get_partitioned_child_rels(root, parentRTindex);
+		inhpres->partitioned_rels = get_partitioned_child_rels(root,
+															parentRTindex);
 		/* The root partitioned table is included as a child rel */
-		Assert(list_length(partitioned_rels) >= 1);
-	}
-
-	/* Result path must go into outer query's FINAL upperrel */
-	final_rel = fetch_upper_rel(root, UPPERREL_FINAL, NULL);
-
-	/*
-	 * We don't currently worry about setting final_rel's consider_parallel
-	 * flag in this case, nor about allowing FDWs or create_upper_paths_hook
-	 * to get control here.
-	 */
-
-	/*
-	 * If we managed to exclude every child rel, return a dummy plan; it
-	 * doesn't even need a ModifyTable node.
-	 */
-	if (subpaths == NIL)
-	{
-		set_dummy_rel_pathlist(final_rel);
-		return;
+		Assert(list_length(inhpres->partitioned_rels) >= 1);
 	}
-
-	/*
-	 * Put back the final adjusted rtable into the master copy of the Query.
-	 * (We mustn't do this if we found no non-excluded children.)
-	 */
-	parse->rtable = final_rtable;
-	root->simple_rel_array_size = save_rel_array_size;
-	root->simple_rel_array = save_rel_array;
-	/* Must reconstruct master's simple_rte_array, too */
-	root->simple_rte_array = (RangeTblEntry **)
-		palloc0((list_length(final_rtable) + 1) * sizeof(RangeTblEntry *));
-	rti = 1;
-	foreach(lc, final_rtable)
-	{
-		RangeTblEntry *rte = (RangeTblEntry *) lfirst(lc);
-
-		root->simple_rte_array[rti++] = rte;
-	}
-
-	/*
-	 * If there was a FOR [KEY] UPDATE/SHARE clause, the LockRows node will
-	 * have dealt with fetching non-locked marked rows, else we need to have
-	 * ModifyTable do that.
-	 */
-	if (parse->rowMarks)
-		rowMarks = NIL;
-	else
-		rowMarks = root->rowMarks;
-
-	/* Create Path representing a ModifyTable to do the UPDATE/DELETE work */
-	add_path(final_rel, (Path *)
-			 create_modifytable_path(root, final_rel,
-									 parse->commandType,
-									 parse->canSetTag,
-									 nominalRelation,
-									 partitioned_rels,
-									 resultRelations,
-									 subpaths,
-									 subroots,
-									 withCheckOptionLists,
-									 returningLists,
-									 rowMarks,
-									 NULL,
-									 SS_assign_special_param(root)));
 }
 
 /*--------------------
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index 2bbef3acbd..cb7d9b0c43 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -97,7 +97,7 @@ static List *generate_append_tlist(List *colTypes, List *colCollations,
 					  List *input_tlists,
 					  List *refnames_tlist);
 static List *generate_setop_grouplist(SetOperationStmt *op, List *targetlist);
-static void expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte,
+static List *expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte,
 						 Index rti);
 static void make_inh_translation_list(Relation oldrelation,
 						  Relation newrelation,
@@ -1319,19 +1319,22 @@ expand_inherited_tables(PlannerInfo *root)
 	Index		nrtes;
 	Index		rti;
 	ListCell   *rl;
+	Query	   *parse = root->parse;
 
 	/*
 	 * expand_inherited_rtentry may add RTEs to parse->rtable; there is no
 	 * need to scan them since they can't have inh=true.  So just scan as far
 	 * as the original end of the rtable list.
 	 */
-	nrtes = list_length(root->parse->rtable);
-	rl = list_head(root->parse->rtable);
+	nrtes = list_length(parse->rtable);
+	rl = list_head(parse->rtable);
 	for (rti = 1; rti <= nrtes; rti++)
 	{
 		RangeTblEntry *rte = (RangeTblEntry *) lfirst(rl);
+		List		  *appinfos;
 
-		expand_inherited_rtentry(root, rte, rti);
+		appinfos = expand_inherited_rtentry(root, rte, rti);
+		root->append_rel_list = list_concat(root->append_rel_list, appinfos);
 		rl = lnext(rl);
 	}
 }
@@ -1351,8 +1354,10 @@ expand_inherited_tables(PlannerInfo *root)
  *
  * A childless table is never considered to be an inheritance set; therefore
  * a parent RTE must always have at least two associated AppendRelInfos.
+ *
+ * Returns a list of AppendRelInfos, or NIL.
  */
-static void
+static List*
 expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 {
 	Query	   *parse = root->parse;
@@ -1369,12 +1374,12 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 
 	/* Does RT entry allow inheritance? */
 	if (!rte->inh)
-		return;
+		return NIL;
 	/* Ignore any already-expanded UNION ALL nodes */
 	if (rte->rtekind != RTE_RELATION)
 	{
 		Assert(rte->rtekind == RTE_SUBQUERY);
-		return;
+		return NIL;
 	}
 	/* Fast path for common case of childless table */
 	parentOID = rte->relid;
@@ -1382,7 +1387,7 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 	{
 		/* Clear flag before returning */
 		rte->inh = false;
-		return;
+		return NIL;
 	}
 
 	/*
@@ -1406,8 +1411,19 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 	else
 		lockmode = AccessShareLock;
 
-	/* Scan for all members of inheritance set, acquire needed locks */
-	inhOIDs = find_all_inheritors(parentOID, lockmode, NULL);
+	/*
+	 * Expand partitioned table level-wise to help optimizations like
+	 * partition-wise join which match partitions at every level. Otherwise,
+	 * scan for all members of inheritance set. Acquire needed locks
+	 */
+	if (rte->relkind == RELKIND_PARTITIONED_TABLE)
+	{
+		inhOIDs = list_make1_oid(parentOID);
+		inhOIDs = list_concat(inhOIDs,
+							  find_inheritance_children(parentOID, lockmode));
+	}
+	else
+		inhOIDs = find_all_inheritors(parentOID, lockmode, NULL);
 
 	/*
 	 * Check that there's at least one descendant, else treat as no-child
@@ -1418,7 +1434,7 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 	{
 		/* Clear flag before returning */
 		rte->inh = false;
-		return;
+		return NIL;
 	}
 
 	/*
@@ -1446,6 +1462,12 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 		Index		childRTindex;
 		AppendRelInfo *appinfo;
 
+		/*
+		 * If this child is a partitioned table, this contains AppendRelInfos
+		 * for its own children.
+		 */
+		List		  *myappinfos;
+
 		/* Open rel if needed; we already have required locks */
 		if (childOID != parentOID)
 			newrelation = heap_open(childOID, NoLock);
@@ -1479,7 +1501,12 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 		childrte = copyObject(rte);
 		childrte->relid = childOID;
 		childrte->relkind = newrelation->rd_rel->relkind;
-		childrte->inh = false;
+		/* A partitioned child will need to be expanded further. */
+		if (childOID != parentOID &&
+			childrte->relkind == RELKIND_PARTITIONED_TABLE)
+			childrte->inh = true;
+		else
+			childrte->inh = false;
 		childrte->requiredPerms = 0;
 		childrte->securityQuals = NIL;
 		parse->rtable = lappend(parse->rtable, childrte);
@@ -1487,9 +1514,9 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 
 		/*
 		 * Build an AppendRelInfo for this parent and child, unless the child
-		 * is a partitioned table.
+		 * RTE simply duplicates the parent *partitioned* table.
 		 */
-		if (childrte->relkind != RELKIND_PARTITIONED_TABLE)
+		if (childrte->relkind != RELKIND_PARTITIONED_TABLE || childrte->inh)
 		{
 			need_append = true;
 			appinfo = makeNode(AppendRelInfo);
@@ -1559,6 +1586,14 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 		/* Close child relations, but keep locks */
 		if (childOID != parentOID)
 			heap_close(newrelation, NoLock);
+
+		/* Expand partitioned children recursively. */
+		if (childrte->inh)
+		{
+			myappinfos = expand_inherited_rtentry(root, childrte,
+												  childRTindex);
+			appinfos = list_concat(appinfos, myappinfos);
+		}
 	}
 
 	heap_close(oldrelation, NoLock);
@@ -1574,7 +1609,7 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 	{
 		/* Clear flag before returning */
 		rte->inh = false;
-		return;
+		return NIL;
 	}
 
 	/*
@@ -1595,8 +1630,8 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 		root->pcinfo_list = lappend(root->pcinfo_list, pcinfo);
 	}
 
-	/* Otherwise, OK to add to root->append_rel_list */
-	root->append_rel_list = list_concat(root->append_rel_list, appinfos);
+	/* The following will be concatenated to root->append_rel_list. */
+	return appinfos;
 }
 
 /*
diff --git a/src/test/regress/expected/inherit.out b/src/test/regress/expected/inherit.out
index 6163ed8117..7a969f2f10 100644
--- a/src/test/regress/expected/inherit.out
+++ b/src/test/regress/expected/inherit.out
@@ -625,6 +625,28 @@ select tableoid::regclass::text as relname, parted_tab.* from parted_tab order b
 (3 rows)
 
 drop table parted_tab;
+-- Check UPDATE with *multi-level partitioned* inherited target
+create table mlparted_tab (a int, b char, c text) partition by list (a);
+create table mlparted_tab_part1 partition of mlparted_tab for values in (1);
+create table mlparted_tab_part2 partition of mlparted_tab for values in (2) partition by list (b);
+create table mlparted_tab_part3 partition of mlparted_tab for values in (3);
+create table mlparted_tab_part2a partition of mlparted_tab_part2 for values in ('a');
+create table mlparted_tab_part2b partition of mlparted_tab_part2 for values in ('b');
+insert into mlparted_tab values (1, 'a'), (2, 'a'), (2, 'b'), (3, 'a');
+update mlparted_tab mlp set c = 'xxx'
+from
+  (select a from some_tab union all select a+1 from some_tab) ss (a)
+where (mlp.a = ss.a and mlp.b = 'b') or mlp.a = 3;
+select tableoid::regclass::text as relname, mlparted_tab.* from mlparted_tab order by 1,2;
+       relname       | a | b |  c  
+---------------------+---+---+-----
+ mlparted_tab_part1  | 1 | a | 
+ mlparted_tab_part2a | 2 | a | 
+ mlparted_tab_part2b | 2 | b | xxx
+ mlparted_tab_part3  | 3 | a | xxx
+(4 rows)
+
+drop table mlparted_tab;
 drop table some_tab cascade;
 NOTICE:  drop cascades to table some_tab_child
 /* Test multiple inheritance of column defaults */
diff --git a/src/test/regress/sql/inherit.sql b/src/test/regress/sql/inherit.sql
index d43b75c4a7..b814a4c471 100644
--- a/src/test/regress/sql/inherit.sql
+++ b/src/test/regress/sql/inherit.sql
@@ -154,6 +154,23 @@ where parted_tab.a = ss.a;
 select tableoid::regclass::text as relname, parted_tab.* from parted_tab order by 1,2;
 
 drop table parted_tab;
+
+-- Check UPDATE with *multi-level partitioned* inherited target
+create table mlparted_tab (a int, b char, c text) partition by list (a);
+create table mlparted_tab_part1 partition of mlparted_tab for values in (1);
+create table mlparted_tab_part2 partition of mlparted_tab for values in (2) partition by list (b);
+create table mlparted_tab_part3 partition of mlparted_tab for values in (3);
+create table mlparted_tab_part2a partition of mlparted_tab_part2 for values in ('a');
+create table mlparted_tab_part2b partition of mlparted_tab_part2 for values in ('b');
+insert into mlparted_tab values (1, 'a'), (2, 'a'), (2, 'b'), (3, 'a');
+
+update mlparted_tab mlp set c = 'xxx'
+from
+  (select a from some_tab union all select a+1 from some_tab) ss (a)
+where (mlp.a = ss.a and mlp.b = 'b') or mlp.a = 3;
+select tableoid::regclass::text as relname, mlparted_tab.* from mlparted_tab order by 1,2;
+
+drop table mlparted_tab;
 drop table some_tab cascade;
 
 /* Test multiple inheritance of column defaults */
-- 
2.11.0

#94Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Amit Langote (#92)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Mar 24, 2017 at 1:57 PM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

Hi Ashutosh,

On 2017/03/23 21:48, Ashutosh Bapat wrote:

I have fixed all the issues reported till now.

In patch 0007, the following code in have_partkey_equi_join() looks
potentially unsafe:

/*
* If the clause refers to different partition keys from
* both relations, it can not be used for partition-wise join.
*/
if (ipk1 != ipk2)
continue;

/*
* The clause allows partition-wise join if only it uses the same
* operator family as that specified by the partition key.
*/
if (!list_member_oid(rinfo->mergeopfamilies,
part_scheme->partopfamily[ipk1]))
continue;

What if ipk1 and ipk2 both turn out to be -1? Accessing
part_schem->partopfamily[ipk1] would be incorrect, no?

Thanks for the report. Surprising this should have crashed sometime,
but didn't ever. Neither it showed wrong output for queries where
partition keys were not part of equi-joins. The reason being
partopfamily[-1] had 0 in it, which when tested again
list_member_oid(rinfo->mergeopfamilies, ..) returned false. Attached
patches fix this code.

Also, I have fixed few grammar mistakes, typos, renamed variables in
PartitionSchemeData to match those in PartitionKey. I have squashed
the patches introducing IS_JOIN_REL, IS_OTHER_REL, IS_SIMPLE_REL into
one.

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

Attachments:

pg_dp_join_patches_v14.zipapplication/zip; name=pg_dp_join_patches_v14.zipDownload
#95Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Amit Langote (#93)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Mar 24, 2017 at 4:18 PM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

Hi Ashutosh,

On 2017/03/23 21:48, Ashutosh Bapat wrote:

I have fixed all the issues reported till now.

I've tried to fix your 0012 patch (Multi-level partitioned table
expansion) considering your message earlier on this thread [1].
Especially the fact that no AppendRelInfo and RelOptInfo are allocated for
partitioned child tables as of commit d3cc37f1d [2]. I've fixed
expand_inherited_rtentry() such that AppendRelInfo *is* allocated for a
partitioned child RTEs whose rte->inh is set to true. Such an RTE is
recursively expanded with that RTE the parent.

Also as I mentioned elsewhere [3], the multi-level inheritance expansion
of partitioned table will break update/delete for partitioned table, which
is because inheritance_planner() is not ready to handle inheritance sets
structured that way. I tried to refactor inheritance_planner() such that
its core logic can be recursively invoked for partitioned child RTEs. The
resulting child paths and other auxiliary information related to planning
across the hierarchy are maintained in one place using a struct to hold
the same in a few flat lists. The refactoring didn't break any existing
tests and a couple of new tests are added to check that it indeed works
for multi-level partitioned tables expanded using new multi-level structure.

There is some test failure in 0014 (Multi-level partition-wise join
tests), probably because of the changes I made to 0012, which I didn't get
time to check why, although I've checked using an example that multi-level
join planning still works, so it's not completely broken either.

I have gone through the patch, and it looks good to me. Here's the set
of patches with this patch included. Fixed the testcase failures.
Rebased the patchset on de4da168d57de812bb30d359394b7913635d21a9.

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

Attachments:

pg_dp_join_patches_v15.zipapplication/zip; name=pg_dp_join_patches_v15.zipDownload
PK
R�{Jpatches/UXt�Xt�X�PKR�{JGpatches/0001-Add-RelOptInfo-top_parent_relids-IS_OTHER_REL-IS_SIM.patchUXt�Xt�X��<ks�F���_1��&�DR|��}�Fv��v�X%�����B�1��%+��~����I9[w��sb�==����M���S�����'���=��;�������d�����\8�}���������b������8W�M��x���T����\����R���@I��y���������v����H�w��������P��'��u�q�^�\�������Ep:��8�}q-���2Z������^;	�b'2|��7����/����w�\��zwA����>\��?,��I���_�4�u4Y����x�7k�N��8�pBS���|��x��H��V�J����IO��H���E�j-#?�������{�B�`6YoR��E�{�<��J�)"	���(	S�����{��T��r�!>���A��,���O�F'��ps����y�����I�o<��f_@8����.�����xN�D�T� �m�
��9n�#�� ���i7�A(����C����������'Dz���}!�
+X�C��)h
=�:K�e����,�,��>�Rz���R�	��.�����	����f�Y�����\�����?J��X�P��T�N�8�a�I�
����E2!*�V}{���V�?"���o���{yHS;����W��"9z�
q8	�&������%��
��puk�>�����n}��������vG%����*�XLgG)���l ��byt���@�6�+��-����A�m/7��}�a�� k���N�x��j����G�9J	'�e>������H<�G�y:]�\�(K9����A3a*>I�K�H%�~����u�;���-}��}���r��#}W��h����x��AP9�4~ L�
��Ep����#���t��o))3�)	�b��Q�Z
��,�qYoP�HsDAs,3mI�P ��a�F���l&�#��r<���?�zu9%�D�%_��u���5��,6h���#6��)J>� 	��)�D��d��|�2��1j+K�5b�����Rb]����U������ls���V�
 ��b��!V�����a���4Xu��sA+��s��n�2}��_X�
f�E�
(��������#j3��Z,qeB�)a-���w(\u$_�C."��=�C<�5���I��olH��	0�!�����e�&>���3Gqzl�|Z*��+s��	��f�.X{�l���!�UZ�Z�QD�	"�9�����z��.�ww���k�K���8��D��s�TE*�H�&�@?U1_�vZ�atV��F�C�����5DX]�����cn�p��DeZ�������kfq<�����4r�*�E��`9��9��bac���l��D���G�F��x���q7��v����4���7�p�)��S(`�����@��$R�{�&�0�6�&YM*c�"��iF�W�YK�2s�:|�
B�FK`��V��� ��1�M����*F�#�`�o5��r��q����5c�V� Iel
������0��.
�"H@x�NPs�Q8������1h��W���Aq�X��_����eY�����M)�Y��Y=���qp�j�sh�P�9�� �)���L!����$;j7^��crP��x
�	��<�\�L�9������F0���AJ.]'b��!�G�[' N�<�ZU7��B2:��������Li
��X3*���L���K�����<���.�����c��U�S+�b��&�(}������E�� �`��AP����s�QY�`�r>IH�
%�c4d�5V���-�zx����i��G���c�|b$��.L����4���G�bj\wn�E�O��#O�'yV��f'��}J��4�J����*�52Oq�y��
;"���{�
e���de�kfCO(cc��u$��W�
��
�A�����
<�Fg@�U�Q��;&*;KX��"��|���l.�4X7�H	F��iO+dD*�?nK��1��$�t��,@Ji�O�gjJ6)�Z���gxW��D1�S������:�R����@��I���
G��2���(c����I��v-!h���c������N���<���
���D����4�)����`���YB%���x����j��g!�?[~,^C�
�������S��4��03>��q� ��L�����O�0�����d(��h|�P�g&����y���?�����^5>�����/�b��#���fm��EpD�:~z�d�:�r6U0�9�)��N�x`Y�*xZ&�IF*�L^�Y�t `����S�Ou�t��ZWW�'`���t��/t�,�����N��Qw���<1����z����|_����5o��}�<�~+������SL�V��7�*}'��oR����nq�5��$��N��{l�X�%���H�<�CKf�f�~�3A�p��8vJ(T�|��S����W�t_�?�aH�}��e5���7�75����[�<sb�)f�������UX��i>^]���K���c�#At�DH�����a�~EE��&��!��a8~2�fu��9lcuP�[�!�
���|8
�!�!r�4QZL�5G�4.��;P�"pY��Rq?O���$�A��^�9���_���c��p0':
����T�y:�Y��"d�k����f������6i�3�����x�����3�h��6�0��X9&:��
z���L����j�ZH���<:����2�[��?�Q��u�}��������x{��'N�]`/���br�f��p���n[Hv�G���������
�g�y%���VX�)�����e�QX�&��,��Az�,p����(�p�e;�:UTR"^�RL1��$�'�9.�$W_f���A�Q_�����Z�J��v�������u�C������bP�w�L��%~���2��9����/4����,��M�?B�.��*����}����Y�N��%{Jd��?�vq_F��YV�_������I������`����(�����L���0
XeP3����x�[���F�{���rQ�)G���[�`�j�<T�r�G�&>��
0�TT��a���\���e�<��as�j�9_�0:-��d��R�)T2��0�<J���b0��{=>���s@�P��#q��D%�B�F?�0,�@<������n����2��A��� 4�����2Q�~�UP6*�[��oR�]�1+�$�����	��J���/�Hxm���_�.*�P`99��������
�\��G�l�k��T0��~����B/Qn�������H;7�����T#�jX���v������P�1>Xj��l��*s^���[�@�M�3"8�``� �m��O7x9���Ima�/�F5�_���US|�i�`>x:�[-v|��|!����U�?�C%(:�zjh�i�����+��-Do�G~��X
-8BA,��no��'��&�/!'�X�N%G���	.����K�H�d�E$�US,������*R.������hM:;5K�
�����W>���6��p2"����{�4�%6��Jy����|�I���+��f.��Y��7L�B�w�K����i�,b����0Sx{{/���O0���"V�%��p����V�u�Q��E)��q���/b��2��i`!�EW�E���MT70�S���Fw7��<��C%�}�"(Y���
��� 7,l�����u�6q���d��9�o����������1��LM�
�����}����W �b���Q����,1����e<2���F����5;�G���R��dj�FO\0����x���V��9w��#Bou_A� ��EVj+>�K�qD�%�F`�,��N}�G����T��1I)�8������-�yHZB�@�6����E��q�n89pf�^o�x���h�����G�X���h��~���V�(|��gC��u��
����c�������|��ZoUm��l�P�-�O�������C`���R�s'�VJ�`�0�f-��S�x�D=������������������������o�5i��K��j�iW��c~�u.��"���\@�b�������h(:1�>���!�H��|�I�$�K���w���"�����-�=������1%��i@XX�7'���1%��?5�`C�N�(���Rw�������/^?f�P�|��R�-����8���N����{���Q���/u3�������uCA�����nL���q[�'������#��/drv����|��(%6u��.Vu��������������Gc;�}���M]z�?���`�5����/{����1�_�\T����_U�t*=U�7���"����X^I��D����k�Z��#Y����������2 z���
V���f�	��N`
�E�%����E1���v6�Au��bu�	�.�_h�/���HC�_��v�.SH��;�fr��I���6E+�Q�:Tu���h�z�vx�^��I��X\����;��lo<���)E�[c����K��C���!�nJ�Q	�D�%~��H�Zb�8�TA_�5^a���v/<"*�Eb^��8���F�y�,���?�V�.�������w|�'�����2���9��K����^��k�VkKr�ve����7��%w�� DO���U����ylwXu�0�J[�F�<*�O�:�*�JmuD�����(Vv��5U�Z���5���j�7�y3D�]��$�J&wXZkP���qVx�w��2�f����;���op.�-w��|���;��^O����@* �

cI���9�8~��7�59l��(9)%fU�
�?��z{8+��s��/���h�=��C�����.���3���	wS�f25f}���/�J�m#+��������O@�}_�K�+�v
Pe,��![+]O���5KK���9�<ts}C�<zAa�r�y�����F��������l��yyk���X�f�����-�O���	xh?��	W����.kq���NU��KQ��F�����[��ZH��l:��^o��L��s��TA����`�%�#�O�O��?��a����9��t�p�I�e��!�e��������$��Or�$��D�,��P�/��O�3���&��������	nG*)���������$����$�aZ*�w:���N���"�@�d�2��F��2y�:�Z#~GD�?�����S�D�,z{��(=h�^���?��������dH��A��]�C_:���������X�����w�L�!��>��`lBvG���q_�p�'�L��"qLVXvf�.�.>{`���]D��'��F�U�Hy�D��L���r�D�ivx���$^���4������������o���M��pe�I���,���.&M�5�l���Q���v!����D��7��!���O��!��h$.�[�C|������ ���+(�8Q�F|��9�[��Te���A�G��s��M�d�|*n�&
���q.���6f%�p��+�z?��K�t>Wk_�����]�����5���a�H2
��^�C����L��5nksj[�g��U��\��m���>>%�c,�3��u�^����������}K�u��wZ-���~,���n�jo�k�l��S�^��Q��������vc���~��mH?6�NX�=y�Y:w���z��7sz����������������#{|F=o����Rx�Pc�A�/���#v~c������ms�3\�c����V�Q@l��Q"oC����n�)J�7�w�{C�5�,|%����jS��C���������/�E�7-�r>��i#����C����5?�}d%F+��+Aj��������%~/�&��(^�u��v�+����x!��M����R��8�u"1�7Z��'�����(��q���O���I0�_������.��Z>���������4f�
,����d�0�����ir6�2�T��-�cxr��Z��d2��P���Io1��s������>{��v����[�j����!oY�Ko:�w%�2���	����\hf��;��/�h�k�zA����:���@�:�l4 �g���C�s������x0������1��M���
�qw>)���[k�tkS��o^kjS�����L4Y�l�K�Q��O�3�������9�z%_����U����7���L4<g�\��rQ���x�+�Ao���&���PK}�'`��UPKR�{JGpatches/0002-Refactor-adjust_appendrel_attrs-adjust_appendrel_att.patchUXt�Xt�X��<ks�F���_1��JH�HP|�>�,;�Z[�-�����+E� ��dm��~� !��W"��LOO���G��p-F�t4���`0[��S�t)'��h:�X���:99���\��a >��N�`����`04^�'�,^m�0^����N���{��_� ��&�b�.�N�~n�j'��xy�����5����z2�Dw0
����g�$O��\�]�z#��p���J.m'	#a���q2�7����v�Dq�c5�����x���~�#���q�!�^��7vd��g��{�2lw��Q����`�H[��HB�Dv��o����w `Y����v��q�*��z��<�5���{��m�n`�L���xK�Q�}����#���KV�F��7a��\~�M�U%��W��*�!w�Q�.wa���n1�Sz4�xQ_���q�����]|��f���77�����D�v@Mq'a��o�H�P� 2o�����z���9qVb��C���m����5�3���FFEXt&�L<_����"X���Y�[��8�����5)��ne b��p)~��S�������I�@$�H"P?��b���y�/R a�V��5Pd
� D��(+�>�-�vt�D���}��Xx�']nq�lC�x1�+� 
��4����������=72���I}Q{Yys������ =D�����V{�m��x�X���[���0z��!��9^����q�I���/��c�����#���S��?�v��v��h��o�o�#�-�`
���
��M��S����S���mA���N(�9
'�&�</p��+K��9+����������������rVvp#]SX�0<������bf	W���:�Fb)z���}����f�@�]�U�mg0����;<�����`rr���tE��x�/Do6�@}�5��Q	4S�&�/YdHk��0LL����M��H�i����+	J!���[;���_���|�;O�.?j�we3��������9l�7�@����3
wB0E�Q�Z�/�D�O�w��3B4��8:F�R�0{�w����g��_]�:D�����=M��j����}��#D��I�94��!2��j�omQ��h
��"���-;���7���i���r�v+_Q+P���?Z:�+�\���v��j"��YI���%��X{A�mP���|��H���)^���a8�������1~�>^�*���}���Fy
r�H_��N�{5�+)��?;g�{���ze��A���#�O)V�-�~TX��Ef	����y���
]�p��nle�o���G*K;�.��d��-k0�4��Xmma4��5Zh8�ox����C�*s��B�,���(d�9�V�o*Ep��\���kZ�
u��W��TRI Y���4~-`��\@���]Q��.l2�#��H��L-
�����/g��tq28�Lq�h/��C�N
O�w�y���#�j�.����v���/A�
,������U��4Q�=�4�b����$�^�%/���<o��`��4sd?jB���a%��E#v���Z��*���?�K��,���`���eP{�^�lS�@?_�0Dr���\
��['1�@��?���4B���Q���ou�A�kB<�13s�N���Q����!Y�dKI<|�1E��a�;%I�_r������
fsGY*�CBy�q���>HY�6�S�d�tw�Q"J�Z����daP�=`���?J�^
AT�`��Z��M��H���V�������+$����@��w
{��������)'N�S����8
f�t<5!q�����~C|C�:�2W���#��U��'=V!P���A0��~�����TKA�y� ���"���a�H
��j��%�v��@��'����C:�����d#Xa�=�Q3�x������[?�JK	���HJ�����Q���(`Ej�)9�
��**x���*�(0��e��buo���4�C=i�� l$p���p����q��A ����!8 
�B�����3�M�_,�"�%�c���e���;*<����X��irjG�H��<V���}��x&^�[���,C�LH��+@>/��
���A �a�e�j|zE�Y���H]��}8e���������3�1=9`Y3����1�P���>�6�L��1���`�e��o��`�������M�:��H	d�$?���� �HG�P�(E�H���!h��g��S��"��R��M���w��gb�b'
��[\d�;b�W%i��CjOl���J3+�L���f�p��{:6-�s�)>y�P_D&��'�}z.F�?+n�~�����o��_���^�?~z�������Gt�\�hanZ0*�2^5�\!��r����B��V�
wk��0�\�s	,Z���
r�h����)K/�%�+�.=�>G���J�p����5D���Wo������@��I����&d���?u�f0��^}����Bw[��t��XR���������yv������\j"��?CI�$�_L����L�G����K���@9��52�S-�V���^~oH���.'�
�a���4EOR*��U�/o�I��� X�>��[�y�l���?Uc�f�fB��B���/D�#�l��x��&��96Mb��M�X�n������=�Ax@�5�����>�N�����$^�`�d��P���Skf�O�}W�(M_m#�~��0'-���WQY���&��8�5����z�v�G��Mq��pcTKf��l���R��RU�+<H�^�Wr����"���Rx��)0�gxR�cIC5�4E�!C\5;��}�p�Y�����7)9tA���[���'3�,�_���(�����@6��s�_3�.:&vN�#�B��UHh������"Eb����M.�d��H�h-���!9?7��?D�I[#t4Yk�UCI�\-g�S8�
$�2bGGv�t42��Cn2}�Uo=��E~Q�9<�Ow7�[ �����y�
�Y������5�u���4����remX\X|7�bR�9sR^�>RB/�� 6��%�������
�&�:��n�����ND�!�=���gX��La���u�����i��@),���	wz�T��l�����x�	w����J{���T[Z���V����	>�����2����7bYi���>�\&M����Vi�I��6�F��
���������S�b�\i��Mt�z��y�r��P�VQ%�P��W��#�����+%�z��0w�N���p� ��-U:���������G����[Xo%������������y�[.��;��S�k�=��}Sgf=@��v'^���cu�G�M�� ��|��.OD��g�g>H��}~(���IfH�gm� ��������c���5��/&HE���P�-o���Q��s�dr'v�t8��vI�n��;����Gn��t�jt�n���`e�i�X~���xq����_�2j��2�N��x��O���L��0�/'���J
q�yo#:E�.*[u��%`q��'�6D��vO�+I�����FH���.��6��'�0�*�����h������%�9�iob�J���]VT;3�/*��o���f��;�R!)L/G��i�����_w)�yYa@����������-P�4w�#�����@A�����j�Ou����5�)�.�m�"=�S`��X*��Q�����s�����O�@r���2�!6��B*8�������=M
�M�����c�5=��:1���`/74XdpR��wt�F]���j��@�p
�c�xaFf~%?��
�|�.���'x1|��:~��e "Iz��}���=�.%+�O��+���=�I�%�S�WF��(��gWZ�m&��N��E�����o�c�vJ_$��r�����Y�?���j��-Y�2>�n�p�|�5��pu��������7�W���o_�/x���G}
���H���4�����T�3)�7;T}Uv��Ut�����U��S3�T���N7Gs����k�z�CJST3
�/u{�
����b=�L��Q/���
6���.���<�\FR�w�}S�����r1�J)Cg
��j���m�L��u�����:�G%� ���Ttn�1Y+���z��iz���H�ok�����sT.�C6I��'1}p���_�S�0U��E��z�,�E�4���.r_�����u�E�&�sT�
5jF�Z�������jA�������F�C�k�YG�;p�[���w0���-�n����q�|��EUL�}�ZdM�Z�e�uj�X1�l�0������[Vt%�'��=J�l�
�8
B����ASu�2^]ksq�j��������}[����i�P]����4���-��?gZ]�],��4��w�MQ����K3����l�F!�n���76D'��k&g%�uV*^,�P�Sk����!��n'����K`v�"�Q�%�������"r�Se3B�+-T��Fokb-l��fLt�����E�[P��zJ�B���3=i����7��e�s��0��A���4;����M.�%a��/�Y�Pg�&^��X�
����Bl0z���w<�,Q�S���ZH����I	8�;���>�f�`(B����6/Z����;��4���}�.�2oyO�e�+�1e�C@��!	q���	��DX�K�5���*��$B�',��}�y��
B!�slp���)vF�{C������MC��`h��;d�[�*t?���J��>mLO�&�^��8�I@aH&t��9�q��h����������i�<�)
�.E�����K6E���U��@�OAq�t[�%*�������z�`���U��5d�
Or��A��4����'����W� ���udv��T���4VE�����T��&�v���r�����2���y�
c�����(��r������|�D���=:�*�~�&��c�4�([�n���n��dB�A�Z�M�'� sqPV����nIV����P����������!*�w�������r9>q��~/�<~�;�����9]�E�@�q�C��<��24�_�a
�U2����D`�_�����\\�]�y�������&��
��]�:M��`1�����'����X0��"�C(���0��w�
%V`x�v�������T����,����M�w��P���������>���TM1����\N��=��4�]�����Kk7c���O�c��?PK��q���NPKR�{JGpatches/0003-Refactor-calc_nestloop_required_outer-and-allow_star.patchUXt�Xt�X��W[s��~����m�"������i3��4M�t�p�������$x�����.���K�f�p4�{�^�]����^NX���|�����(Z����b����t6_N��	�,J�
+��W��ay/H�<S�Z��X�4|���`i��b�QV�+L�A"�'��L���:"1�3�t#���*�����$�~��b����o�����8���0c��'q�J�BT����\b���
O���,��:V��X%+,X�����$���+TY]&��R�������8�Q@�+��IV ��1C
"�� ^^��^)X2�^K.1�)1g�s*�h#����TYj�*��Y%��W�D^fB�od�l��$X���~��.��*�\����M�����%��V���������a�lAT�P�#���![�����W<Y�W��u���^b�bM��lY����t�m��]�����}�%���%7X�cQi^��P�
��l�<	|��	�u.s�Rk�[Q�H��E��w��L�:��~��{N�YN��@�sT�q����J��l����axv22u�b����y)�2��k��������<^����-��4
��E8�hQ��S���j�h{���)����(:�3s����4�����uK���YY�|E��R=�N��{MY���r*/X
�,�=2���(.oI!�Z��w��a�K'�Y������Rt��h-���&�na0����@h�2~����/��������q����x6��O\��O�"�x0����+�-������GF���+]+��TR����R���Yk(�6�4�nEzk��'�}�lP'�t4��I���-�6�m���IG��mdL+�� >'L�@p�*F0�n������J;y�B���a���G}���{�ov���m���ca:@��G^���%BO�1��V�:����8�
mP�+a���N�'����5�D}�"n:s�5.���(�R������
T��r~�6����@;����$��S�W���l3��@�:a{l����g;��UGg�]h.�^,���%��6�5���1��#�=��|C����e�bq�2gU�f�QKf����L0�B
�oT?:�6t�
u�q�>Y���85=M�Z���v���M��b���p	�e\� +���zv�����RQE�F	�$XyGyKI���T�+�J��j00��$b/0�t��0r������~yy��� �\���{_�����5]�!���@D"�N1M���K���P@������a�U�k(j���ZQ2���M�HT4n��|z��J���G%(���� `kr8�����������/���fm�M �v�+:3gQ\����q���z)����2S�6��}�����rb���B6\v��������Oc�:�+q���v�c��kY�1!�]_?itsq>Z����������*�v*cD]��v���nC���
r =��9t�c1�b�+U����(�s�^�t��	5��~����#3�
�6���5��#�=����I�J�N�=s(�����4e����kh,*}�	�1���~���;�}�<��,I�Q���)�p��&�|��
���Q2�vQh��nQh���l}�P�%�W!w����,];��1�Y{�	��Z%�-i����X���V%���9my�}���M���7$f+���)��(X�����PKu�w�DPKR�{J-patches/0004-Canonical-partition-scheme.patchUXt�Xt�X��ks�����+�t:�,Y��w^=���<�%���p�h �XS�J�V�$����>d��y�X"���b����;�����x1	�����x�8?:?��#19<��"<O��i�>�5������c��x�}`���,�T.��|�s�����9~�A$���Y$E8��U�5��3vS�!;eo����������s6�����D�?c�}�������&���%O�$
x��<��<�d�+1�v���LF�m,X,��3K�,��p�$Is �]���:�J)K������l�K��dq
9T��`��D����R&�<vJ�r���
��T-��t�M� �2!�i��<%��dQR��7nv:b7�T
�4�,��-l�Z��,�\�/g��Hpbqu�a{2@�2�;[�$���8��[�H��-E�?/#��,Xn�a
T&r�i,Q�y#F�Sa(O�J8�|��v,�8�Db���6��U�d+~��pG�].eD(X�(�1�J�����Q�~���M[��E:"x3/�[�u����Lf���w"	�u�����f,x�_9
�7������eE�����
�M��vY���g�8H�P������$�x��q��7X�(�X��f���{�~�F�����#�����q�� ��'Gb~*F����d>?5>9:B�>cv�h�?�������6�?�H��PL�p��IX�����z/K�|����vY'L�pv�����y��s�����$����d~)@�`~��;0PqL9�����J@*y����I_ |�		�Q�l�x�LA��VFj�����t89�cU{`/����Be��{���6�}Vy�z�Ul��i0�i�{��t�g����2b��P���� �	��-2
�2�v+c�`�6j��r������OZ������&k.�t3"(xn���#����}x�~�q^93�_���Y� |"�lw������a/
Ta}�_�!<�c��<X��!��������9_��sA��B�"�tUF�,^D��{1 G1;����O����?&f���8���L|�B��|%gA��*���}�8J`��o6i�Yw���`gO�?�4�������e2gWjH�^W���O�ZL����\H
g,��BW��80	u�S�U�u�8��@7�@��T(���.�[���0�1ct/4�rG�U���y��u�i@*��2�|?F;9\�j/�a��
d����)�h��o :f�I�d�f��*��de��@������S�5V~}�U�y6L
��4��N��^�Tf����q����\Ht��'�q@�Q�����1m���������6�!GZ1Pd�[#���`^s���-	��������fB1�PH��*d)�)h�e1G�$ap�����BP�S���,�4�	�2��u�Y�:���|'B@*�-�P������dr2EK5���'������b��i���� ��:.6|�������p*����Q��%��n{o��?\�����A�V�������x:���Z%��0���%������8_��@"�V��������� 8;�`4
O�����D�@����T���)J$����e�q<0���@�-�6��eW$��e�R�5
�C���N��l������6~;���1�����3�%�i���3��	���t��c��df�\}��g.����� .3:��&E��a��4Y��_�,yf�`t�N����UMD	.+A
<P�de���e��Nl�_����@�:����$u�4�>�:��^����r��X���=������G�QQ}���1�'��Qqz%pb��$ZG�s������e�6���Qi�?PE�*�bRn�.SH�i��C������CD�	���C���=Or4������Y�������������cZ��'[�bv���4��F��q*E���>}�w}����==�t|����p:&EG�������[�5��RP�
��*���!��G�#�p'/2 Y�9<�@�����;�q�s-�2�V�$��VJ8����{J�d���t�[��f��BJU'Gjby�|�	��Cht,�P�
�v=������c,j��"�b�����:e���x�@u�#�A�_fB^��1�!���-9��,�^n�l�_��B���HlL
��"p�K��O��Nz�����Wa��3Aj=�Hb���/Z7�0<3��Yv�a#�|U��>}@
V$G����o)������Kv���?����}������������������o���'�(���B���Z�-Q��o�X������w�����TU�Tx>��v��A�%
HkX���C�V�_X��W8U���|UIV+��J
�u��Q��a�������+kR��\r�a��YC���E�ZVq=��.
�@a���]��i��'��b�YX���]H4�(�����:�z8X5#����j�����D�� #^���r�� Tyux�o���!p�`h�1i�+�6����0����B�GC��G��@C�Z��Z�Y��
���:����J��4+w�������Y����&��f]�)�*=w����!|z�*�zSm��|�z{Y����
K��7��T���������6r�XS�����o���U�4s.[���������p�e0����w
����,6�U�
`���.�����	w}��8��~��cP��b5����fk����h._3��$mm���[�\F��,����L��uN�e��z��i.S�C1���N�tA�I��8R��_�t�m��� V���z��Y�j
���b�\��#Z�z���R�=0�����`��k 5��jf����
�-X/L��Q<i����4&��sS�d?�\X��m@��l�-�[�-S��=���wrtd�)����-������vD��^�Q��n��_��9dK��}%V�j���@�^�Uo�u�vH�`7u�	2���:E=6[iA�1����?���$����`��`��*
����w 	 �o�B#��I��X����e��-C����8��G���o����������p�������`��9������W+��y��}��/��Q"U<@$���q����f����am����>���L�yp2�&������-1-���7,�#��H�kk�i�|�����m�Z�|n�{���86����C%g5��J3���g�6*���=������}�pKSn�!:!����,M��;^���[�*�`>����Y�����nd8����Z�C���:�j:�
N�UR�Z�-:����,/�@x�����r6�^��]�/��|�S+]��~O���U�?!�RYP��������	�<����{M���]`
�\����K��`/�k��2%�
�����4&�Xp����� i�1��x����a���i^}'����p���N�|���{d�IZ�]v�M/�L��7y��:�ZAw~���8
K�������F�N%�T�5�d�n?��-]���x�>��bmn9���0aEGO��}��\�.���4!Wk*m*+�^%��T]�P�Y�`r�x��1��FE�>�|6�*�x WQ�	��Gj3T�24�$@n������]�8z��c��>a���r�O�
�:��,;���T��S����_�}2�dy�I����C}�u�e������>�K�
�{����?tw���d�4_����%�����E�eV��:�Mz
���y:cy��;��7^�;�u���p1����������N+����:�:`�t���t�CT�e����~�vY�4�AGx �(k��������=�{Wp���4X�W�mH
��u{~zB�OOt�1 �����J+�B*�X$����������1/$��^��(���@d��`=>���tUt�FP�����/F��Yj68�B�e!�J)�q����\��?WJX�7�b�nX�m�{x�L�I��_��/ ���e���y�B]�'�"cM3��z<��)���|xx����(��C.A����L9gmC�����7^AV�=���%$-`��"����
7�P�w:���"�~nS����J7���%V����w�p��*T�+Lp��/]���!"�n�������p),��'���a�����*��s����b
��$��Q��wH`�j,^��f�C��QA��}u���1/��������
*��73��fL+���v�H��[_*�C(D��)���d9�w�:]U�z�o��WN�U��p���x�^�W����H�-�1c�m��;.����D�r,���u:��@T9����&'n�m�����(�X�k3~\mn5`��cM�����&��WR����k
�
�pT����t\0~0U���N��X �s�YYEL���[a|o�<
�P�Wy�/G���&U���c������z������:^�Ost�5�}���ORq��o7�T���5MV���_H���&��D�\���B����,�g5��:�-�.�[�sa���OB���Z��u c���#
E�c/	���@��2=\�����U����p:~���X�G�[�4��S��Q���C�S{M�����	,Mx�6-{��P�����+��=P|���Cy��}��V����
:��z�)�\��5�%���l!����2������'��g�O=01e�R�I5�S�
R'{��Q�D��~=��?�%(Z]�N���!L��1t�0R/����>��\*���%_�w���uLH������|t���PK�]��<PKR�{JGpatches/0005-In-add_paths_to_append_rel-get-partitioned_rels-for-.patchUXt�Xt�X��Vmo�6�l��Ctvl����.E��E��v���0�DEjeJ�&����;RI�:��E0d�t�����!?�������l����1o���7����;�>��a0�O����o�{��q]���0�0��F�2�SV0����	����E�JN���Z�������t<��JDC�N������t�]���	��P�����Gp�m����(�9���}V\D~��z.����*Ui.��J����/y*���^Mt�+� �+���
�R�8+9f*������i8�E@pa.Bth��&�U��7�&9�EI��m�Y����_1�v^�t�~�e��k�,�Y:!����m����7�K�����v��e�Rf���E(	������8���0a��G-�y�
�K=�z���>D<���nXV��1��e����zp�����_c��� 9N�����u�q}���q������u�n�j��1��gEza���8��2�U�<[����eA���<���	�g�o�eI���l*Ty������k��N�����c������[���L4l�
��1��_�����	,��������r5[���{59=�6��	���������?�|��c���kO$��o��$����!�!O�a�}+l!�����X�qH%T����@��M�I�X���3..UR�1��=Gk��A^��������C /a���I�-��->����������k�5��2L��k�5}�'}��A�H��>�$�o�@���6����-9��:���aL�\*-��$�y�w�4�9H���O5��[%L��������<`� az�W���,
�Q���d�=R�=U��0�t����|�\mJ���M�p	n�	p�����jK
B���<�K�\�f	7���]���k6Nt�������
���W���^mu���@����9�F"��A]�l�x#B�����%\]q.v;Mj(l)�l�<��w��-i����:�`��r�of
�6���c��-3��cq�����.S���v�B�Dy�Q�84�[;*B������r���J��U�uw#�~%�U��T��n��}�Zm�a�n��;[q���jh��U�B�{B���Z�hJb���$�;S0�������������G�q�1��:N���n���>����E����������_�)]���(��Q���C.�_$�|�F|F��	B>$�N����F}LB���H��a�V���r�7=������t���+\�q^��:�_P��/�B/�'��t���,�:c�oY�PK���_
q
PKR�{JGpatches/0006-Treat-dummy-partitioned-relation-with-partitions-as-.patchUXt�Xt�X��W]o�6}�~�]_j��,���dk�.]�IW$��0E]Gl$R����w/%[v�������L��~�s���L8����LW��d2;I�%.���X��d~4����B���h���D����$���-�9��6����(J��������P;��JYL�P��e�F8<���0��RT��|�����t6�A4�F�u�|B�N���o��A4�����B
��E����r�hL��\�W�W.�N,@��L� 8_���<eaej��.-��(K���<.��{}H
Z���y�m����|��Rk�:����o'i*��mL*�����F?w��G�Vt���[Z��(s�>�{d> {��t�dJV�CN�2-5i'�����zZ��E���s*��~FMyR~��~��/�;�+c�����Pyu����l-3���|����c��"8�k�
��$�A���J�����#pl����C����7EPvp�RM`���2��`4`+9N���������������i�����?���a��}��'�4��G�k�7�����$�-�C���U���A�v �y��� U��F��X1�/�d?��Z�_ ��"���Q�����R��GG��}#T��Q_��Q�8:�a���"�R�EM#�5�m���������z�!/��!\�7�D��KB���`�I%rx�q����=�*,$�5�"j�����}/I�@�E����E�[���7//]���oVW?]@eel?dg��>�w�M����
�[J���wNz�/�{�M�*�T���s���7�
����r�%P�^��{n�1Ey��������J�R��M����~��rBK\���+c�W��f�,��q��J[���1�Y��H�6��<�������t@9�����^������mJ^t�7���`L���4�4C�v^U43~��i�i������W�>mw!��#��������4�i��j��}�Z�aEg@i��!D~��s�@i��[j�U�vv�j�E$��$�<^�d��jly��jl�z�'s�~6P����'�S�Z����k�����k��|0��Li���(|,mf�wG��,`�	.��F��{��r�C��G���zwm�y����UH7����H���H���/�k���D���02��a3F�u
<[w���|@|uu��V#V��T�/3C5����~�vC4�4s�	����pg�"O80y��|�~�g���^�2�1����Y�����kk	��<�?���+WTA���A.��pPKL�?��PKR�{J5patches/0007-Partition-wise-join-implementation.patchUXt�Xt�X��}iw���g�W��@	.�rCK��\k�Z�����3�D��fl��WK���r��Nb����[uu��]�-���i:��Fy5�������|�=M�����sy����*[��r-z�����'��n����F\�M�s�m�NJ�8QwF��_����:O9u���i�yR�o�wy�F�����/��7��7�3��N�����?���F�������{q�;�I�M�2-�lu| �?�t%��z!��Q��F��� ����,�_F���r%�t5[H�������Q&��,:�s)&�L�E!��(�i!J9����7R�`7�����L���d~�������Y�u\�ez���LV`�|��D,e>s�8���	U	���a��a�,��m�X�q��s��x�8��z�M�IAf�b�oVXH��E��)���	�,h�}����i��
T(��!I��q6�����*+��e
��s�Sf"Y�%�<�1�9�,�L
\����"�E84Y����/��B���L����l;��C|��n2���E �����;)r��l�8g�i:���"����w%Z�����d�I�&'��W��/�Mm���9����d��F�R�6����$��,�6���c?�Mn�q�bG�����m��&���`�\.�@��W�@+����rM�����ku�gxu�6e������t����&�d,��b"sh�M5���`��'�qo����B ��2-�'��X�q1J\�����b3�'�_�3f������ocG��ic����,��or��I�wCM[�8;0w���4����T7Ar+5RV�0������������mJ���p
�4��(Y���I[\u/��@�`��f��-�pS,$8>j4&�t*��g)���Z"3�-j����E�'�tp��t����`:�n���i��
@�m���q|z~v��A?���_�C<��e�+P�IQ<9b�,%�<�OW����[��8���f�]C������.@�����q~_-��~���!�c�B��>M�+�P����"���C�64��XmH��*��1�b��Y����7�R�|\����Q�-d����HE`�'��e������f���f�����c:;�����km�����<M.&i���
\
�:l�j%���+���]jaa��#�>��'	��X-��R�m���T���0�X��1����X�n���e�?��fLL�c\4"�X
,�{b2����m�0�yY��n%�8�d
$����,�"U��L����\�)\�w�Es`�Y��[a�|�44�����|��rg�i�1��t"�	M�@,!N�i6�v,��x���V������vt�s�r������ZD�����Xj(�A�h�V�m��X����;'g����{u:�i�VL��VA���^������]��#��"�������F�b����(�$6�q��#�
U�F��Y��
:�X ��u�"��C���9�sX�xs<��t<��R�f�'%��
�pK����4���`_�"��������"����>I�17EC�6��D,2l�����8�V���2�����/����{�t�g&v��07�3���@!V�������4K�P��Z����x�S���y�2R�	�� ���wJ��y���J'��2�2h��MJt�% 8<���P�:K����W(���E/�Xn�����|0�H�:c�����f8��?\c$���L������%�-���-�l"S��5��<,S�dU*i���P�Q�i�n�;<@�ni��@S��I�_���� �=(d��]!��glA �cb�e�s�b�@!v����������	���f+����[t
�[���a�`��{
�
R3��:5Zz�	�G�FU��-����;P��(�(�c
�z���/��(
H:�Il(@��'b���=0K�<)�<�&����Y0�X�w
h"���+�����h��	)c�5����j�8E�����HctiYYDo+�l�3Q�qR�P�:l��G�(8"���rr��#u���2k�|�M���.���z`�*�#����u�r5������Q����g���b��LJ���;RQ�1
aE����d���8���h�j�R��lK��%J��;C��T�C������i\0�0���-~@�~H�@6��O�p%o�E[ kG�R	l�����C���-��0�Jy�A���qI�u������@���A1�_��4�)�g(�Hq������R\!���3���$T���p�zK �A#0pd���������$���'']���IXPxL�RM^h���[���9��%&HP��famW`�u���BMbl5�uH�S�~�
.'�N���;Md������zX�0���}h�R�(q8���f'����H~g~XS��qP�(��V'��m�N��eQ'�L�p=��X��4r�Y��h)��;�t
��x����W/�����������]�O���h]���=<��,���C��!j�c��D�"mD�($�Y���S�D_ALDh��O�hY����I5�3!�1�c:!��J�F������NI�w(K�G���Q�d��c���!�W�V���|N�3y�1Z�
���A�a8X�`A1�b��;bsH�����GJ��HJ�DVp�jJ8�;o
f�%s��Ar�������1������K�1��uG7�p.�i�P�*�[�e�g��D���.a�!���_�RX�����X\<"jz]��kR�$e���Z���������=QP��n��pP�qP���P��p��X��T�q?4��I��Cg���q/DlL<�JM�k@�������X1�o��fT���Q9������x��
��>�(�y�T�l�7���E���q�g��<oB��G\�w���������4�����������iX[@�S�ye�!�$�5��+I�W�C�
~��A�����D�o|i� E���tX��l��$�����x��=�������~�
m1�y��s.OT���{G���;���t�~$�?���[���(Q�����|�d�!�J�p�
%����L����?�.����k0��z����������G��w�A������\<��yB��pU����)�*0j.r2�*E[��z�{fP�<�!
�,�	G��"�[V�W��
�5�zM�2JV���[w��_�Ml+�};pFs��O�?z������mN���?`H	
�P�k�zg�@L����'�P��p�O����#3�^g�?t��f1!�?�'��y��L���{��-X�T���o T\j�7s�
\9����-8���qn��YG����,ag��A��u�P>���@h?�#�,��u��P��|�N���@�S���}���O��.�by���.�r]�v�Xq�����sp��8�d��W>��H1��6`��sI�`��
V�jD�����!���(�������,@H������	�����}=��"���/��d�V�Zo�����v�o�/��d5I�	��M��1�	�+��i�E�JQ2���H���F/G(��e2�����'�L������2#7�k��h�K��hd!,�3\6�Fyr���hJ���W�b�~�S[ne�^��y�T7�����Z�`����I�������k%#6^G&Q]�Q%��R!)�������PUy�5�m�P&e�e�B;p�z��2
]�v/<���������W��{Ox��������0�)*3��q[ �?�!!y�������m���K���b��x1>�
M��M�lHE�ff���K�VU�i0u:5}9��Qs�i\��M���h���;�"��~��� �����
�>+�fx�l�#@�^~�1�
7:��&_� ^���pga%w�V�������JS��.[*=�A�
V��� �Y.��Q���-�N�-[a������T'@��N6��n������%��l^����SO�0�c�-�
��Q�50�)��j��l4`�g��=n7�l:r�t7	�pk7T��
���? �WUz���2_,�����|�����p�Xo3
%D��������y�z����H�`�2�^N����������t�p_"2L6�%}�Cr8��I���{�o�	����^�c�@�+�Bs�
Hq(S���F��Ff��L�o���my%��O&�O�ob�!?�{���\;��9�@x;���`j�OC�0��5A�����v�����l��W��1l;����O����=2���
�X�n��"���,����2(�������:���B
�@����Y!�p�H�������.�I�z��L[�Jk���j"G��������?iH�><��M�{[x+�
t�pv����`�9�S�{^����,����U����1�/~p��&lwU�U�U������#Q��jx�1���\�f��������P@@�Z�����o�}������o5,�z��u�������9�w����Y���,�.���d+�	Hhx=8P�wK�9A��	��x:�b��������o)f��c`t��W#r3B�) �$���ad����u�=r������������v:�q2���#��=r��:�l���G]�)�?�,G�W(D����2O��B�o����6{8$��4�j���x�^4�Y������'���9�j��1�?��A�	���� �'��,�M�W0��
YV�9V�������0��$��+���O������%P������ �E�R�`�	.��U��2O�8��-��n"�����|�\��)�ZW��N|B��'�,�o�o>����+�@���6��bg���j�x]�n�!=3r>z"�?�J�=���CU�XN����v����?���Z>&Z��F��g�p!g��s$4+������@dMV��)�_�rG�*�0P�$�j�X";��&Ty��������@<qj��,p�Dut�u�9

s`����_c����o��Sn{�Y�H�]}�j�������(.�E���Rs�
�4k�h����Gr��O�5@d����4�5�92�`��B�l�v�R�]!���R"�������/
<T$�|����}�}��:�]��&�oo���?on�=����s8������+;_^�>\���q"�r�!�ue��D���W@��7��'�c��cy�����BO����x>"���-g�7�j+���-���9�npl��U�rI��j������E��L�����N��OX9�
y��mG�E�&�B46)E������<���[���(�LZ8"W���?oO@����0E��
� F!��dk�{��+*z��1D�o��:[#�$�U>D�������/(��u�Z�T��LZ����
�����.&T�7��^�=�k�W��35n[�'`X��<�2E)CtMFN�N#�7�Y<��(���Lg����r��Y�Mb�+m�kM*K���6�0R~���o;V���@(��z����N�B�L�q�u;�{@"�����#u��z��T�������H� d�NY��z�����2^�F��-IdU���"-#Q�(�(SH <J��y> � �z��
YpK%��Y9z�_g.iR:�]
%tq�����B`6D����W�X*�y���q��M�jt���#�G ���m�2�(�=���0M8�y/����dR���H�n��a�������9Hm|Nd%���k�
n�ir������aM\[}0�S�r\,k%�9j�8�p��c��*	�7�+G\���p��^%�8��������J���d�a����OZ�XI�����%�s��Hy�Lz���2t���9�+�
����KkW�V��d#���0����uh�LW�l��
m����r�7��+c���l�-��4����2�x�,8����v����&��������u��i������Xu�>%K&�\+}8��d�����K��vR����#P]����8T�b������W`��J5�<��m�_M�?��to%;3E�"��T�'+]�O~�����m���5�����$8�Q_^z�S�*v���z4�c�B%v@��J���������~�;�V�8j��b�"�~��,&*�-�/MO��7�2��y@uQ���F�-Mj���v���)�W`by�*J��G��Z��aEG�Tm���q�1���Y�lb�mq��uQ��w��9��8��z�"�B�C��5SSlu;,��"���p>�H�u2f�C��totg���!M4^�p���"-��;w=p�zx���t�hd�����C����b_��.�q�]���&0���@��'�-q�a���������������(��1���B ��	��w���@M��x���]�>��L���v��P]�H�c#��S�W`h>MX�1*+O���R������@*G��������f��vM��tP��S��}!��(��g&cO�����H�yPh�O��r�<�c���qb���N@�(�>�j]vdf�U���#����H�:u�� R�n��9��jmg���ltJ>�����_�����������D�*�p�2FJ
U�i_y1b���dB����;�D��fD�9B$WA������v�7���(��Vu�p[�e|T,��9��*�������������k'Q���G�TLQ�Y��\y;�%U�Ki�7�
��Sv�3�H$)�Aa��;V�U�����D�T�s��Y��`�Z�F����_z%m�4Cf8��fi46}f�����i����a%�O��'�M�2�����eGYTB�1wyl�lk����b�LqES����u��nQ����Bv�X5T��7c���U&?��PU��^���u����
�e���C�7kT�#i�������~�U���(�>��O=���=���'*Q�)�V��~�8m������ ���o>~P�'MC�H�`����5�R�1�����/l���I���e��+�
���v�Pk@9��$���������@;���
�w/W�T�M:�	����K��&�soEG���X�;�x�g��������ZY���-��^)y�^�����;d�b���g�F��f�<K{D�����a�LxJ� �&q�4�g���s��@#U("=�����WfA����m�G��4��j��T������U
8��n~/DdO������4�8�Zi��G�����u�3|&0�����x0�Q���(w.E�iM
N�kP
�sR��A ���g@��kc2�E�`i#A{��^�������}��Ri�b�DRR��)l�9��t���JD�P��q:M��>D�p�_�7�Rb��*���4�V�PU��G\���N�D��O����U�;�g���d�����%l=���r���Pn�z#�(oX�f��j(>�H�O����'h)�Q�J?9�dMfb��"#�R
�W]#�D-���S�D8[��1JRc3e�&;��I�T�x������h(��� ����[P���_����e��X���*��0�W�1U"zj8��B��<A�e��xl�f��n����������cM�liFB��qR�(�7��m�l]D�C��F�����><7J1�gi?���
�s$x���
��"v��U���)s��97��t��X�+P��TT���������t.zRN�{��������7�>@	�yF���3q++������VX���f� ��#����B��[�hZ��q�p�^V7	PL�8���	B�M,/,��;<"���������q��&�.��%m�^�i�V�77Zuz��>�i�V����z�i)Y�G�'I#�r�5�I�h<�J�3�
����v��' ����]n?��M��N��Ao,;�{��#�J9�s����(�F�jEA�����63~�{ta��D�^����p��c�l�IxuwN��1��g$��a���o�K*�Z�O��I8�N��NL_�L�\eea ��nbNH�hm1���X��~���9����l�R�6������0K�(e{PaH�u��U|����t�����x����J?����6��q�.+�a��M��`�g{]��{=u�bPO]�y)G���L��g{�������a]�������
��~�K}���A�����6�(���@:�Z��0�)����
?J���S���+��R��{�=��}#��
��.����NGv�G��x���F�:�A���v+Ya����G�l�ned�8���cN)�R���������WF�c`��ba�"&���$�Yo�����07�PbmG$��WBZ��*k���_#��r�v���q���h�Eyt�b�*P|�\�h�J��_��ep�����Y{p���������c�={�:8������}�7���3�������^q��
e�v�5f.�[O�FQ�^�����5}�������C�eYa�4��.�4/O����-E��0�-���O7�b�����`��!��G�Z�TdT��m:�������.?5r=���R����s(��Rq�������jx�D�%��9(���!Z�0�[�j)������d��e)3nm���y����C���08�s4���#l��|��n���9?'��#x���`�2}Bx#���,i����������,���t�_����z��0�I�R_)t��!�"Jc��"P������$����B������9/n�G[!����J���V�'S�J������������#p���{1��wv!����������/?'�|;?�����'���3�V�q����9s ��|��
���<b.#��6�`���by�1���j~
6z}m�a>���p"OY�kO�����	VU��&��dH�Y��*7�e�
�a`��7|����2���,pVq�	b-�N������v�5k��r8x��J�����B�M�'��M��e�%��SFYT�J�i]��59%&	�qm��D|jJE�p��=�3n_)Id����F��$�O����)��k&�������"Z�t
:[Q�x�9g�L�8��������������\P��)����W����{�������ap+��J^��]2_�-���^Wr�d���X%O��O,F� �f��-����cx�a��!�@��((��^kd�}h��A����f��WY�CW�����+y�����_�B�����=��u�����d�|�U����qz�,�t���7�<�7�2+�7Fa�e��2P�\��D���4*>i�����A������]�V���g��E9x�-:.��s����Vg��6�������3u�t��=��k�.q�B����ot�O=�o6�]�Yr�zc��ul�9Z�`��RTTz��L�6Z���	<:����gO����I<H:���)��D����g(s�A$�xQL:jh%�9���y���S���fZ2�d���xU���e��4���!��'��K����M�������R�4�y6��8�.���CM�2+���6J���Z��
W��O��q9�4
���1�ww
*!L#E=�2�k��I�J�������
��}��1��I*��"�QR1I�[�I�T����J�G��xX�;3YTF�Ze2�����T)*6#<d���e8F��4`#>b�Y?`�>
��<-�{��8���@����
J1��gT\E�[�,��[%�Tt�Wt~Uf���E�����0�����Q�0��\��T��`Do�JEO�b�s�T�����z��L�Bz��E���x���U��=����#"
�2��.�=x��4�����������z�%wu����KGeC�/0���r�0\.���1�?0M�NKDZ�Ua7��8*��g�A����#��2LO����*]���al���DeHu��R�-��is$����\���J;�b�bI�n��
w�~���n�Up���t�F��N��p��o�;�u�&>�Pp]��bLNG�m8���J:��������B.��������^C�vk�������s?�q
��cc��5����7'5����[
���zq���4�%��YE���A��H����]T��q �}E�����(b��Hk������}
����k?���Vj����!����|��D�W�vo��"v'���}����+ rt}a���=|ZA��{�c�:�{E3��ri���VE)���)�X'9E=����+���9n
���03+���%9!�z���L���p�#��E��?����}v�k �`7�{q���}�E�v&�?-Z��\�_y+Y����3~B4_��W=yF�hN�F�
Y�e�r(�MS@r2s<"�ZA���F�K��:�!8�T��n�(��<�G���{���Y����%�\�G�"~b���������m��{Yj7M }Y��(K����\/�2M���������@�I)��������8�B)���V��K�����D?�������d#��r{���X�#��@�&���P��
��,�~=iw�����9p�=�+oz�}��
E�V��A�
~�/�c���6��N?f���,,�/�(�v$��{���y�s*��0��0���Pq�������Y^kF/�+������1����wo����fe�L�Pq��q�7���Cu��&�#��!�!��g��9?��:>�%�{���{a�_V������mZ������:��yE����^�R~8��)�n���U��U���:�2�$���PAEvG���K�q��9Nrb7��w��	��#���:����L:m�G2#'��R�kuJ�2�[�J�d����4$���xN0�9xu���"�'��K���^>�^�x}_����U_�lF���?�
��������O���K���>�f4�[���>W�g��|��x*�����k*.�wzF�Z�A��}�7�Z�� j\�����<6��el�z9�J|��|B�K�����
E�"9=���6�
�0�����T�2�:�f�u�VV_��z�Y^TI:T�Y�DV�o�b��{�0���}P���������M��/���i���eU�u}zmD�IZ�"��$Z�h�q����<5�0�i�\���q���|8�-��<4��t.���F���C��I���~g{&��IM�3���L�jX{���I�A��Ntg&����G�KF�]��N��T�
1�>Z]��z�9�y�~�M|����"D��v��a[��j�=��[��*��	p��{��@������&4s��q����%����	\�M��y�����E���_�A/�*Y������,b��qU�N�`E�W��X�?;�=5�ak6-�P�w6G@�i$O��1���
w�aq���x�������6E��:f
��PG���W�<z���(8��5a��^7/��#���L�E�Z
�.h�����$�B�vFG��(�PT{����SZ*��"�$��n����*%���@��	���*��xN�i���i�0y{����
mE�0CK���m�����0m3I������&�Ay{*W����3�!p�;P��!=d���eh�
������.>.���3�l"�:_����?3VJ��a�7��=�{�o��0a����3��#���}�O6��(�g���������_����#���a8r�A�s��(
��(��R�2g�F�L�����de���,&���ev�����HO(�sTv�0�����f��5+$1�V�����V/���Nb2�YI��$n�>&[\@�M~����(l��:��e�-��p��:q*E��'�����1������6�J,EX�b(�W������]�&�A}E@5v�Re��=�?<[���`���lH����\�����>�+�b2�&"�����E�A[o������a[��������w�o>���|�y������_��~�����7�y�f�D��*������L*�9<����@�����
�V�KA���p&�������y����:�'��nj���u�s�,�#���MNO�."��,J�VA�8� J��^Q�q�>���I������Gd��d-��P��BmZG�g�������{Qi/Gr������ �D7'������-�bN���
q����0�N�+�3�x�w�OeSl�%�s���5O����6A�K=K�����H��d��'��:�L��n[���[(��`�T����*Y�cV��,O�KJ����^���g,S�'t)�Vi��dH�7���]1�������`���3�S|�#+�c|���tS��\�s��@��Y.g�p��#�OS�A������qK��M�/��� �b��8���	�A_�3��.g�i�*�o�������_�����^��n^}�N�H b�N��o����������kC����i�� i�z���m7�5��I����[�vl��[CO�L�����`�����86���Wa<jV�T|*��:R�uQ7����N���c����q`���k�u�����.�����d#������w$ ��F�[��I��gp���:��Ur5��;yquv59����*�WZ�
�P��z�^E*S<zo��R�^z�Xg��:/���aE,$T��W���Y�-FTC��d�/��z�=�5��r��w��es��.���s��O�S`P/����|���9@�m�S�������?��M�M�=t�:���p**yS�{�4w��lU����BjD����i�����P�T
�'�x�
��������<��@��'N;�0��L��������F]�/6{�b��ppI����K�}Ve�5����,�$���O�LER��AUB��"�(�Y����QZ.�����z�s#*��T8�H]�� a-BW�`
#�)�-p��|���������N�h3�v`���e2��@���ztB=��Y�?�'t�=]�Tq����SG�d5�y�)����Z��sp�t�|l9V%���OC�*c�Z4vO��/X<r8:�r&'��x���&`�?�n-u�{=y~�=��hpu�?�$e�g+�j�utI����Q��L�_���!;P�d��]����<$�]���������������C�$]�!!n&mSM���M�����o~x�e���Rw�#+|Kyvq�v���%�����lE4�����a�_���h�x�����!�m27E���|
mR{:���P,��ah�b�s9���g-��UaA����\P{��=�?o����A�h�I.g��:���'�]�9��EN�B��J�/G����Ioz1=����,r�Q������=����T��$��fU6���}����I=��������}}�a���Rt��zf�[.�L���Vj�i�e���B��N"��)�A�S��LjX�����t= ���cX�m���??x�8���`���-��P�g��e#[�2G^������f����4�2B4�Q��2=��L��1�F(���`�Y���+����J�(���[V�	�5{}2g����?o��Mj��R�U��V9�����o[]
�y��������3Z���X
kc5"���Nmb@|L���������*�L[$�{K�IM��_�
&��,v[t�&���B�#'2���ry����{(�>u`J&6���<*���������&�U{ ���\t�:�F��PK�e���5��PKR�{JGpatches/0008-Adjust-join-related-to-code-to-accept-child-relation.patchUXt�Xt�X��Z�s�H�9�+z�8��#��c�2P���jk���Z�����������{�:|���f�-u�~���u�D�/�h0���>���?��7�8����O�rz���g���X|V��E����a�?�^���x���"����Le!�)�ogJ�Rq��4r�M7��[?�B]��Ru�X�VS���r0�<����I��\N�(������������'�/�/e^�/I�L����"n�)�����B�� ��� �s��^}�"�g�;y���<~Z�-���{��T�6�9�f��r��2���2�2~�M�zw.����G�k������8��{����^'��U��;%J�,�(R�T�b1�f��+D���(��L$�!	����(T�&X7�eI���-����%���,�4)~����������JWP���<W�W�$�|T�����T�&��p��Q"�w �e�p
r���R������8

c�\�tx�a	X�N�����b��*�&P{FDcMY�_%���������W��D��gdp?X�n����W���f���{x�}N��Z�Y�)_{N\����`=	�<��9�@�3��+���qVm`�%V�����)��D�(-������e+[lM��V=(�����9���A�X�\�0�����b7LC�X�xYb���z$|���n��9��F���������#�(�1W��b��zb���x�k����8"�Q�<��	�6C�[ugg?��^*��6��W�(qo�2etm��2���n�	bBy]+T�e��$Y"��[;b[W�t�YXx���E������ ����D�y��X��Y����mH�,Mp6P���oa��S������`�)����5�2����
���5�e(�����4K�"<1����'P��!R�n��n��E_�
-w{���4_e�����~��`�����i9��-S}�V&��6�p�vD~�9�������������:VT4	�	�� ��P�fU�!�������@�6����<C�:0�.� t$o�pF���w1O�:S�o����@:����8�7��[$���@F���k��#(soJi��!��Jw-���b^�#���	�~���8�v+v��g!��D�������������0��{����r1{�s�9.��Fo�v�{Ph|�5�S���_��
:����4U�7!�v�b��j�ZKr637>\Sn�3X�2�4�.w�T�1
�D��@����+�E't��@P�*������E���Mk�*X��\�-
j����tL��U�Z�mW�!�q��TR��Pn�k[.��Z�_gH	��PH�H�
�E�P���Z|<�#%|����DeIRt9�?�+q4EQ�v����P���%���P�^�pw
T�n*�����tm������{Z��N��$5r���FgZ���L�%SSGs��B8y=�HE��O*�%-�
(�6%A��o������S0���,��F)�����
��E���o��������j��j�(a��8L@���hb��0I�5��V�'R��k_SZ���V;�QA6 J�6LRoG�XR��e%�fv���MD1D�*���.�5Xn����'�~��e��%���M�{�%):��/�-��x���q�����q��g/�m�7�igLp���\l�e�!$Bf7>�u�i������c@����G�`w,Tn�/l{�|${��5�o���o����K%G�OF�������S���'Z�r�S�I��'��N����%���80�$��M(�,���k����Y�>zG�}p ��8���#��@�w~�� ��D�\�w�]CEw0)��4�"C ����y�A
�V�I��x��O�:�\E�T$��y2�>N�n<�Pp���%�0�	
�L(#��%��Wru=����`���T~+E���b��{A�����
H!�c��k(��>�x^�4��U��V;Z�+�t�*`
%Jb-%M?������&a���n��7��T�����2l��^^�\�!����)����:���Ix�;L�7�3��^��q��.��S:L\��4� ��T�>==����0In�7���B�:��FTa�]�]M%/Q�q�!�"���N4|��U���F`Cq��7I��Bw�����
nDM��&A����$�8r�"�j��@��V���+�����__K���r����QO�>v���8|G�uh������V�DE�2"���(H��.�5��	��se��^�Q;!df��Em�`��0����{��6�=��-����]�,��v�*�h�C���2�"E};��� ���V���T�I���-c@�T�*����;<���I�������?�KuUe��e�_���=.�iH���Y���������4,���j�a��A��(f?���a�#A�]CXoo7h����6��g�����`Ol����������_�>F����������Km�q0�/?�1��W����Fv�{��ke���uu������{v�8�S9�r��y����ym5!��t��c�8gt+�H��8�1Y�B��pJu��nu�b�T�^�\G��M�;�-�yfR��pM�G��<��a��������n}�-�n����<b<�65�-%�G �������n�&p��0��P���J��Y�^'�[�[s{�1�	m4� {F�~���G���H$Y��>���/�G��\���^�9-'"m@aZhw�?N>���������7���!�����3�_�m�&�~}��@!�d��������'D����VZ�����'��aw$��y�#o���&����������+I�v)HI�����!��%�����ki_3\����6�HK���_���+����o$��'8N�4����S�zx���<��oZu��3�����A���������?KL.����S��x=8��2��g�Q�����:
V?���n�9J���me���'��v�V����#�f�a�;*�&T�pE�C�nt�N���5<SF���UW�4��6�x�`����j�Y�3�f����Af1d�3�2k;���t�,m1��,
���rv6�t������1?����V/���F���������]�3�����}�t�x���MR=�m��y�R���/�[*B���?ybl��i�O��������c7^��zV�u�1g�\a&��~#QF@����	��J+'����������mc����0��J�{�\����^78�>=��y	���.��!��F���{��'�e"u�#���*�����_��NG��w��D"-��:��UQXo�v�g��%��]��D����������SM�<�h��o��k;��s��r����������^��-�7e����eT0f�r���t6a8�S�|6>�
�{�����6o�q���z	����5����
=�339����]hxh@�����K�G�������K�s�r�IK�8���;��e�PK�����
�*PKR�{J+patches/0009-Parameterized-path-fixes.patchUXt�Xt�X��<is����_���&�xH�D���������������b�3 9�p��H�&���`s���Uok�����h���Fc�'�R������W�tv~>��������|t>��O�:=:�N��xG�F����88xL��������<�z���^���J���4��S����R���@+:���E�g����`�k���}x�x|��x$z����M6}����������/�������+������(_�1~Sz�n�S��E��K�!�*�+'b8��q�-�*�B� �T�P
���u�3x d����S�;�h9t��i
�Y�,�g���i���i�,c��4@h
��P<�z�*�j�����\]^_������g��B���O��y[g��,f&�V���
����{Z��M��Ab�-�C{�x������|�
�C����$�'����7����Y����dS�D�
����6�^�l��{%<YvKb�+��U���i$G��5`}[����4�����]0�9�����g����_��|DX����Q� B5�����Y���$��K@Tj��U�di� sT�	L�Mr�"Oi�h$��\�l/��C2-��y1���I"�w��X�`*�1��F��\��i}�6�b�Be1<q8b��uo�(���g^H�$(��p��G}M�h�F�`A'�4���3�`$,9_��5c����k�e�+����T�~�
���>��Q6T�<j^< �������C��,�h�1��e�����N���
�.b�&��^x�';�(:c��W��`0h����u���p�a��!~���lv����"?^���L�q��H	"�G�g0���t	i �Cc��h���D^��j���y���U0\��'��'����TE$���T�_^#��p����	P9Th�d4W s����O����^��}@��v��L�db#������M�S%�t<J���$l�'����-��@�ms����N�@N�;:��v|?���3n�(w��7)
����}��dE��|�����K��_������?zP���G4F<s���k�3M��4�U��/}��c�=�1�=Y%����8sx�
>��v����Y��OO�6t����yz	
,��8N���:����6�\���*e0du&`�r= [9���m�E�MFm+9�I���F(��59�T�Qb������W!��mW9�O�/��2�F[Z2�U	����1c��3����>V����)���Q��a����	��T���-H��I��L����(7��?!��2��x2}���F������E���
�"f"�4��S���u��B0���9 �k0{�o���j[�Q�CT>a��0W��
W�������n��[?c�<9����������_�[�Ab�[_�d�q�\����9X����/�����\�l�Eq�5(Q��!
���S=I�	BFt�bo���jG�,�$}X�5m����W��*���=����ao�������1{��Ta����T�n:kV�����y�kF><{s�`��Zz�\�te�����ROb0B�\qts��o����#���4���v��������%�u�� ������Z��F���O,6L]��v���cE�����������BN(����a���[j���X��\�=rr	X������^����?:�]�G���
��fx
{��8��W��{�	���"C�<`Z@	*q
X^�Lr����;�Z-�X���r��iArO����k��>vt��8hD�+V�u��u�W,��H4-������_% :S�pEQ�b�� ����U���2��!�����yf�o��P��O�*�k���m"G������&Y)Dg��q��<f�����j3�)D�nP����Zod=�Em��4�R�<�:�8�V��N I�t��K�j�%x�Ia�M����d����>Q���1�Q�.��>�B?�.�mE�PJ���E	A�����X2��B-%m�L��`��@�"z��CA�.�w���8x,����\A8�'�����)F$GgG���'�*cs�4D<�l5��~���i������0��A=�~�oM��W�~G6�v�Job��SC��g�`F)8��;���P@��j�@��"����LJ)��6�9��'�d�X�a6yH�����a�L�P�~Ab�,o���S6n����7�9����e��c1��C�MP��)7���Z�����L�:���"d��E�4 ���O�8ZV���f�&��8cJ����^ye��^�6���,$R(��:�d:��c�4���L��FN������,�\l��v��Yn���b�a����$�j=�n��:�;�W�x@
S�
�*�k9�|3-7�r~�MH���ONG ��/����7�?��%*��M~�Ot[-vt�$$����LY1{%KG���B�jfn]pU�gB��|>>�
�j��FG�.!]�������Sg��:��RD�1�;�hhO�� .��9���g����Wmu����6^(3���v����EJ�G��������i�����u�dRi��4�I�*���OKc��z?�@AP��Pn]�%}h�U8�����}|0���}|0�c�6(rSV���Y���/6�F��Q�&�@Z�_(��!9���7�n6�)4��(�&h���{�p��
"d�)�`3Ce[4�R����h��}��U�8I�&�M2���`�kb@y�JE�M�%�=��>B�|B/Uh���/�W���*��y�y�#��[?��t�Kw�g��!���
y"��}|X����|�G6������{7�-��Mos^���������.o'O�^��x��,G���od8'�Za;ta\��
|�����K�z��C'�����a=���]��p�5��Z�����-���
�`��m�'`�j��(��>��K����:��u������.�g�28��#�����r��M.r6q����o�'�r��IbY�e(�,�Ig����
!9�]�������X�=���>�P�S�hv�gN5���Z-
�q,G�����w���;G�����Y �� n! � �>���������!��^�\�A�12�We`3�U3V::(��^)/��5�V����V����FrF�]�NP����u�������7�s
����5
���<���m�O�^PM}���(dfk,����8��S��h>_0Nd�k���5�����e\oB�c&��D�����"D������E>x3�~
��\�PrU�X�I�M�����q��I�3��R+�o���F�:�2�J'��h�/j;�.#
�L�H�h�x�B�>�=�!�$�(4o^�ZKXY'���.}_��z�6�*q�����5
����,Ht
����PB0d1�8OK4��G�e��&A����?�(�W	X:�6�\������s�0�_�x\�8��������������
��m����d�LN�]��W%��>+�����vs�$�d�)��b��'B���|�;71�L5�6���ppZ��a�Q�RW��6��t�9e%
:�Qg��l+G��d2[y�m����9��}����M���$�ZMm��5	�j��)_$L�����
.���Q��Gt��#��" ��V�8I��J'���"�������w��t�=x��|E%�rj���-1Wb�-��s����0�%C�19l�"�_c�!�`���YC�<�J��`����a����]�,���<���[�����\�6b�-���� >��m������b�� �o�NkB���&���+�P��{^![7�y6�	��8���WA�iw��@�I�	&A>'Fs��5`�$R{�������d��}��,7�����|TE��b��X:3�K��9�li]X7�Z}I���j�/����B����o��>��U�/�+��-����������P�6�Er_`�����.�'���$�x�����Z-�^��mC���q�R�RM�$�i���sY���z���K,��nk�/o�,�,+�qgv��f�$*��y��Y~M-K�������
>U���Q��8�����m�$?�W�o���3�&DJS���]v�_��cV�'��b/�D���H1���}zd�N3$�&X���G25�_b��8��5����|g�.g�Q��W����H�|gz4�?�6��F5�/��)E=D����b�]�w$�.�|�MA���X����3�X��~S� k	��"Q|O���/���4���hQ�e���>�|��)���8T]�w�
��.��U\S�p��yM1R���JO�VW.��U�+;�TsF
��TG��q��L�vf����������|f�!2a���&�&UU�;����^���Y/�G�����������+�g�V��Wj%�x�
B��c`����ER�`����`P*(�Q&�K2�����w@����F�-'A����hll.�X
a���U�>r�~�h2�\�*Cq��0X\]����E�����{���g	uS�c4kR]D�SZUdU��-Bm[�a�{����
��$=�8�y��5�����.���2$S������5�X~����H�@b�txt��1jb���t(�;S�A��j
��^0 �������r�������%U����8&��������P�����w���."�� OfY����D�`���)�!=�{�}~s�Y��}��E�P2��5Rq���^��Fh����,E��y���fN-��
���.��X��c`��8��C�ni�@�:<��f���A9��4�S����k5��XF$�������(��H���%��s��b���P��m��E��a�ELk�z������T����U�b<�]�	0u�*T�����=�g��_V��{o��>)h��-E�\I����5tl����1�-��6��Y�U���SA�c�J���d��}[v����h��c�L��kt9w�Q�5�^���8C4�����\<
M6�-�����.0���P��{.Wy���*��O:M���.>?���s��as.����m�)s��~5b�KF���E�O�N�Co|:��N�!�(�?��B��c��0������k�����s��]��U�4�C��R;]�L����)���v��p+����E,������v��$Etz�����%�hU%��eU3�\�M�q��%h�s�������_'a,{���q��T���_�e^���@F'�����s��+&����d���u�����F��m�lP��06	Y�br���k*��_� �
j�
m�s;s~kO�.���8��4�5h���}���`�y����wv0�
/���n:�������&�)��_�AY�X�i��7���S�9�%c�j�l��z}��)�3������g��|��k�������.��gjvt~��������x#�:����L�P�zSW���A�YS������V�b�u9��G%@��~���q����_��6�'^��n�WI���.l|��\*^���������PK0��%<8SPKR�{J,patches/0010-Partition-wise-join-tests.patchUXt�Xt�X��=�s�6�?Wf��D:��J��4}uS�M�u�l���ws��$����I��M���~�\�I��c�1EX�`w�X�sWt:���N'��d0$��=��'69��V�������)�u�D7d�zC������v{�w�:��7���G?�k�G���sg���'K��kw����3Y-�k�h���nH
�;2��(�^�l0<����;�������d����v~��g������l��������BD�^���'��u�����fs����xh���d���0	��\��Cc�?�D������8d�X������!^���h9s����=��om�K<w�{���ZMl��b�+��s�7�M:������;%?�?���d*���6>��PS�(rz9}��Es���hF���z��%���-���Y�^0�E�����8#orO�J���*�0:8rz����|A���n��Q�yF|�QP�c�����;2m��)��2qYn�y�����r{n�(�]B���)�L�=�����nD�1��f���n�#���a�����������x�;��`2������g����X7������a���My��{�B�����h����T����������r?���!o���z���
��!w�@��@�./8������7�/��/����R_)7��,�@4����9��jI[���������py�fk����MY�o���M�'�����o����o��B?�]�_�t��[ �����|���Q��������E��n?��n../���y����F�2�����F����I�B��?����������p@�&M����O�a��t���Oc�A�}��x��U�T?lG���\���o����E4�-D+�@�������9��zlO~G~c�oP�O/��� �U��Wm���u�bE�H�k�`�+���W���a���z�
����GW��^D���W��������g���+16c�1��XfS�J�������aSfS\�ZEl�A6���)���dS�z�l
TV0��8��ea,X����%�L ����Rb�����]���B��nnoh������)��:v���q��:�@�;�����h�~�@�
T��)f�,z�X�V:�h�D�+Kdu����/�/xa�O��.�p�#�G�P���m*s��_/���h��Jeg���E�z�r}�d���|>�`�����!t�^��4�����%�;��L�ek����L9C��z3v��N<G�i0I/)��tR� �V]��E������>�!��<�k?��������l��{�Q������\��7`�/}c�W�?������w���K_�P��c����!3Q�5 �r�8l|@��6D{��d^^��
�$�s����TVo�F�r�
�g�,S���p`��@����>G��x����]?A�L��e�A^W������RW
�	��^w!�����I������D'��$h�y�����~��C��dF\��v�6�$L�7�x�F�n`���	�.�dM�s��L�fm�G/pk���h���0��M1u��y(hq}	_`%3d^P(T�����@�_��t�j�l�j������X�����Nkm��Ml<��"3�����Zn	��>��9�E	�mtI����h�c(n�'F�h�K�Y�$�|L$�#zF�:�`i{���/�	����5�2fR�LJ�r���7���0��
[,�60��,2��:�t�g}K�B������7�r#0���
Xd����O)A^q
�����3��1Y�k���%�s4���_��O�L�b(~1?yC��0�_����m?k��F�]��<��=����c��46J�X��4�p�	t���:��5����\�9��OGl�	T���V�mw�J���U���3Y4���W�Y�A�b���uX��]p�B���[�'�h�M���8>�{h:�X�i�H��"	b����d-(Py���W�f��$a���em���	�p�IbI��<,��������������y����V�uR�t8���Y�t�H�C����Rei�]EWJ�^E&'�?�4��������n)���5tbR�t8���Y�t�H��{�YBc�d'b��)(��
'�pO�6d�Ja��7��1��������m���q����v����23��$d`n�Gn�M���L��*,���4�k:7w�t���1Z�i[�nCG�L�a��2e+Y��������P�b
���/�T���[�w�����C���a�*����B�����?03�H&n�C��H�q1�&t�$3�����"�������z�J��>�S��_��Yo��y�D����8<p_��.{]��r��"=��yX���-����N|}�r�
�9�@$H��=,
�.��:G���?��E{�����4\�{;h����8~j`��	���1C\���0R�5��J���'{�#�@���M)l21CK��_1�j��	�ed�1�3D�(f1}�41.5��;����8���p���_0�_�-��y�l���3`y�'��1�8N-�Fj��P���L�-F�w����KM�g����SkU�`�f�Y�^���;F�B)4NA��EY6E.������HX]f�d�HZ
=8�pU�+��]%�N�X	�X���=�2����}��f���5�>5)���7����i�-���W����a�:B�X��[�+:_"o3��!�g�r-���wB?����#�n��Bd{(\��>USL��?�K�=�L*��G�
D����O����_���������~��xdl�E�7Q���C_-	��a(B(0�Ac�H!��@[~J��b?�����OZ�I�>���o�Y?��s~3a���<y3��n�-�p�383�y�f[���#�]s��RX�P#��f\���'.��CnB������,��M������."�a7Gd��lB�,���v�_�t]���"<��y�3�����Wj�^z�Q���)�dU� UR�EY�_�*i
��W�Y@K6�RIf�J~5��T�d��2U�� R��s�
�l~P�J���1��bE^��X��*6\�����_�}kw���#�k�XHW�V0_�!7�s���o9}�#{���*~����F�G������h�P]�2,�j�^��-Ndy2c3H��5o���V���4��1FM�m#w���=`�Y1d������;cB[)��]�����j�F�����|4�x:����t��S�B2jW���$����
���G�!I�}.R�2�!j��SM��qM�P��A�X�
�1e����;_j����"����1I���E�~h�:�U�K.������q'f�q�H[b8�����;��Kx���I�#��x�0�k�DQ���M�����FD���	���i��3��S��[�O�.���@�uO�����v�P;L���o�
Y����,��I�&�"�k ��D����@W���P��
���nv�����,���`I�W0cn;���1��k�)��,��CW���
��w�2^~��P����y1�1~(��J�L���������I�x�1>oK�YD�~b�9A�8���o��������/,�����~9�n�8R	�8�?!
9�y��� �
��������\%MG=���i����zF3eo�e�k��u=yI�v+E��I9��O:Z���z������������c�n��2��0cxsAC�$h��LB���J\Y�bkd��YYJ)2�����WAC��l��2-������:�eM+����qi���%�k���~�������!�+�e�p�?�0�c���S�R�N�������-C�������E��!p�@IeP�eh��3}e���Z��M��F
`��M�L�7�d�V�4%�q�W���MV�C�	i��mf�+#�k�����X4�$��b�~8���y{��"C���d���)B�u����0�35��Q"?,����
|�+���`����iYdH���V$�J����P"�,��jC���XD���Sb�@��rIm��#�]b���C��!�E@z�$8bn��T,wD���2���[G���%`�\Z�q�\��� R���P���`�,����>��q4b��i��r�t.���J���\���#�5��1�[�=�l�=�Rq^�{@XE�x�sO�7����76�U�$�~����E�#L�)�n| d9��Olq��V��hjEF;rd�CGSB:b����P��jzj�cJ��F�m�%B/��Q��Q	��s���n���(�%�������RcLJ���,�e�R�`p��0a��(�����A�.��|������c��
��V���7|y,X��`0/�:-lwT�n���)�V���RJ,��m!J�/^�����)��P�C!��BU)��x��8#X�k��M�yf\V�(���.�T��:�N����U#�d�/C;`�S� �%y%GB)���"��J�J~��t�B���@[$�%�y%�s�F^�J��
��O��B����[�����Xb�/�B�����
�\.L{�!��iO���W�<��Z�����3{KN=� Eb��;��R��>Mi�I�H�o�D�Vi��<(4 ��h9��Jf��WU[��KeC������L%Nq3g@�+��r,jegla V*�c��&�04��}���v�bX��G-���=�%IesLB����������B�>�{!�)��[����R�b�;�����4�X�o��s�������XJU,���a7��7�%o'�g3�W�"�z��s���L�j ��	Yk�����TV�r�#��N�6up7����}l�q�%�����e�����
8��H9�#t9k\r)���k�Wm$O{ByF�~|�����%�k ��1y�F������m���b��s��_��aw���^I"G�O������-���l���4"r����7���1o<��>���7!��+���������`uc
��o�  !�bq����l���*�
�\.��
���f���A����F�
qQ���������F������%�����Rj�������c��
�4�c������m��5�P�K3�R�F��x����m^�Wx�J��%1_��(��������m��5���K3=�"����zS��y�
��}K�s����%�s��8Cd�L���Dfy�g��q����R�-?�~J�S-���Z;�;�a(�Y�k6v�b� V����k��M"��(�c�a�a�pt~w��;�W����]m�a����h2iJ�Lx4��ST;������y���$r\J3� TR~fc?��bA$g��	�iD�4����/\��i�+���2�3�u������j��t�
�:���w����E�yq�
^�k���w�L���v]���&�x1�D���b�������"�n[�y�8�[���H��2*o1���e�E����V�Bu$���Z����q�3���*���V�7�g���[�������d�R����i���
;i�`����z���*z�R�#1���V��1C��uR��4��z��h=w�P��|�W�R����;�4=&�O; Z������-"K�X2������H�����qIs:�}/���E?]��.����WN��h��D:h`_dW��$���P�L^@y���
ZkV�;�#���+��H�T���{� ����#"+�V$�VB���VV=7��FL��&j�)�����������
�����Z�V��H.��5��f�sm��26�����Z������ ���`\��{%����R�Z��o&�+��xv�����F��7(_f����U[4��J(F��MF�����g<�#}��*k
)�2b�$�y`��pj��(�>d�:�k|!m3��hL����jp<�R�'�a��5���WYwUd���,�M����@��d��T\���
)�)�L�3�S��%U��r�*��Fu�L�R�����B���1n�Le*�+����z�~U}���C��U��*�si@��'/���0~�a����������p���������k� ��������A#v�T��O�Yg������F�Hyb�>���	i#��5��j'<���T��,j:�F:#f��bH��14| H:\�t&Q�t&U�t�:��#d��	��
6���E����v�d^����<�!��)["[����������./_��-(����A�H��A��9QB�W2,Dl^��P`Dn����=����J�������x��iN���N�e��SH����XB:����^zM��u����V
���tK�tKB� �2B��V�]��[hUH��}�SR�������-X���(��+G��Vx`�Vd^�5�����_0�_�-��y�l���3`y����9M�Y�C������K����#�i;e�����?���|����9����X ��N4!s2.�EE*4�������"$7-#+��x������
nZJV�=���IP������ �r��Kj9*�\�`"�N0�kF0�kD0\���
��2����\�O�UQ�����I��W�����Sf�]>K�l��<U�6����J�f�4}&*�������OG
]�#����O��:��<R/O
�?��i��?���LsR�D:_�	��q��u��0��L�y�E�P��)�<?A?]��o�� �*�/����Z�-�U��4(�AU�>��22Rb|�G$�����8$+&��/�S��/�JsyH���`������!��j�cp���C��d��sJ���~�H�q"���4)$��W/h��"��&�:&�D\�I�(_�	4���5�c����'��-��	�����
��L�Z�_�^g�O�[�V���^����T������.V��5��@���yh/�@��uO	��l6���
���9*�8�1��w��z+^#K	��o6�1\o�a��������f�B0\A���@	{�lf���z+^�c%����`��M\oK�'C%�a|c��~�>\���ON��O���6��z+^�S%����`�2\��'��
v/q;gw������_�z����eC���������N����AG.ql_�M���]}"KDk�s�����(:r5���^�'��������|	������$2��l�#r��������mi��T=��.b@�Cv�>a�r�������"�,i���'p��^G���at�����A�� �J�Z��@�6�t!.%��*mo*�������p��pz��d���'�M�vJ���pi��,�p(���2�&����[U��D������n�/r��)�����O�S��2��[�-���x�BY���^R)\0�_��p��Y��`�d�����B��]��B�I�*Y;����m�Hz�$�@�5I8DC���4�CI��j���}�b $���*ae)�Jz��W���sr$]������d�\�hL��^;s2Es�<�K����#��g3���:���#�(o��t1������i�~�
R�1Z��UswQ��>@�0���>���
%���%�6+�z�M���\���[������������?�L�����W�a%^���bg�z%�x���b3-U}��~p&X��g�f���e��T9��R:'��
H�b.��2��R���q4]���������>3=U��!F����~�Fh�Y4
�q���T�8���h���T���f4�q�0���`/X�(t��sW�+��]V����D�	'���?\�W����n��u'J�+�7�&�G�A���S�K��
:;i������DN{�C�YK��;%�����z���z���������T�����|GE�^�z�2��D^���g��*}%����%�J
]`����!W7O��R�@b��P���,}g��:��"i����<$oO�����a�^0�ji�5�����d����t:���[::r��;rD��v����=�n���s����N�n��9�g���^�{��7(��j����������Ct n��r����3�p��m������C�J0}:�����w��~5Ca1b�[*����L���9y��z����1<�Q����#.09�� hJe�r��m&h<��z���D��&s:�{�����t>�8�����w/��������l���v������=�4�V������������?c�wD]M��b9����������>M��#�3kP<���Z��W?������}t��k#Z�j��%���9k���9o��h��$	�=�]��<�8�w��}�t��x8�t���M��}+
D����x���-�����+6�4�����O��bb����;��/=��#K��=�#9'Ts0ecI�lN�XMI�#���q���w�������hJ>-����\CT���hj�SL�-�G�=#6g�W��~���1\G�i8���������]sfo�y�M��0�H�V�m`�yC�����a��T1�l���QL#�s�a@#�r,P��)�A�����8�S@GZ�K;#��bT��������*��O9R�����.�I2Nf�W�@�\ L)���)��������n��P*�X"!UJ'�z����"]��"(�iUk��������4������6~]h(���\��Qg�-.������3��F��Y,�����u F���~�I��	�"�~���^����-s�f�f"���	����qC��T�g.��p�Y����%�!�
_w+&;1��*��iu��h}��]m�A:���~O��U�1E)����EE�x��4 0�
�+J�p�3m1�E�[��O�=��yT��
�n"M�C3w��430D;�v#�|��AE����q�I�T�����UJ��ROA�C���$�g5�6�u�g���%��{����;��b���Q=V�B�b���q���`�3U��w]R�����q��7����l���
��mf�=X��!�������}�2'u� �}p��K�������uhI�����k�S%�8��0��de;���������=]r��N��8?������^�<�������0s
HE�C0��V��0F�N�1���1�Nc(?H#<��sa�\V:�%��v���l�?Q�}���6���������\��������u�BNb]$}���7y5}h�����d0F����7��];j����e����S��:�TJ�DRQ�����������E�l�g�s�)�R�����arni:Q��\+�>�&@Rd�xj�����&DW	�~!�#b'�x����'�����d!I a�f���n6��4��*5_��Sy�:�{-<��i=�����1<�/��B��S�E�{
1?����������/����_�����u/�o=���^��zr�o������r�8s�Y4�i������O1�����"=��z��X�/���A���x�{��	E|~~���y��� �s
����=�Pj��F�3��v���PK���8_%�tPKR�{J:patches/0011-Multi-level-partitioned-table-expansion.patchUXt�Xt�X��<ks�����_��C"Y���_q�['qn}����L�L���HHbC�*I��m���>|Ir���3s<�-����.x�K!m�����/����?M��#����G����4�}�6��\�������?1������8��$�������������'$2ZE^,�i�	�����D>�#�+�c�����������F���z��t�����g7/���`��x�����g���%^���tEbO})�������l��8��N"�`!#/�G��'#;rw"�	�D�H�|Z$�Ld=	�r�%�9��h6p�<LD���l���{%��`�bFb�����X����&��q��/����O\Hh��s$�u{��;^���[��d .�%������vE.%N
�"|�{$V�^0qb�e,n�5�!��OeSc�v��+(�~xKg^'��������%��v$X��`�	L���O���:F! �+`���/��f
}?t�g���7����@O�������q(�2��/W	"�+n����v<I4H��*���p�H�sB"T���*������N�Dn%�&	�	�'f�p
���3�?���qM�Wa�b�+{NKmMl_�z@9XG����f(�)�xrnx�i�4���:��Q�\���-�V�&#o�NF�|��r�cM/�Ld�)� /9!����D�D���������AD����x+����ZY*���5p��F�<�X1�P"	<f	L���a�)�[������;a�Vs��$���a�Pc"Da�R��1P9�����e5E9�S�����o�����qN�^��O=G�-��Egs�h/����/:U?V�S/�+��p%<���p�D��~$���>h�W��+��BPF�o8�~]�?��5|�C�=���
�Z��\���O�@�XF�OZ�vW�}�����n�����e�����������_�tl;��������������1u�1�������'a��]@��x�D��t�u�u@�=��5��
�����#Sxbo
�!�������r��	<��Qd��8���)�V���D�?�>�y#��[�w�9��I��G�������7�g��������������������{�[��0���!D5��� �n�hQ[�P��?tQ�.Qh�(��Z��@�h�����S
���lUP����2�5y���0(l���JM���,���|��������>�	
���$���)(�u����b�|w�.q%0Bt�j��D����Z�pC6L��@�!�1�Ek����${��#\.��*�W���<������&�7s�x�X<�fH�I�,�R"���"�;�3,@���s@C{g������6S�?<>�S[�z�Q4�Nw��V���������s�;<F�O�V��3e5^����K|	Fz��'���r��W���b���s�O\;����=0T��qs���J��x�bN��$��t.������0����>09�o��@ba��@����w���W�U��BX�(>C��=Cn}���'�w���������1-�Z�c��2�9��~����{�6��k%>y��P�M���l8�:���N<G�CWf�s��=����W5�6��$E�}�B�B�a�s
k�s�g���G��"�*��l�`�[:1{�'�?P��Z F�sK��vb�������^���.��(�5E8kT=��&���E,G���J�����J��������Z|�Bs�P"�y;�����i����kQ�0���+��
%�`A�CA����DK��ZE4�Ryj��7�e�0^J�=��Oh�(����S���`�����o0{��8�����d�jBy��S)~%F�E�����Z1��
*2k�	#"%%�/���1�K� v�T�oC0Sw7?�F#7��a�e�r���X>��!��O����]�%�T���KW�aR���YKC����$i�7�3�a|j�@�#Wk����{��(�������7b
M���)�5��L&�bB��������W`�'���b��1j��
�*t���
���)��P���`�����'}�H����:X]�c��`���^��x�\k�1���"?1�����D�.�)���7���sN`�/���	�c�#�I&7P�]/�w�b9/�����l�D�L�;�;��3N@IQ��M%9��{�\7k��6A�j�H$Vi����������i1X�v1��E:��5K;F6q�����<_�������4am��*��)���;�N�Kj�K���L����r�b�j�yu��~�,;oQ^ H���
`�����sR�*@�����U��vy�0�n]a�|3��1(��mh���~�U�#��e-�G��8�+��a�~��f�Z���$F��@0�����O)�4C�J���t~G���2�^�E�a4���J�����~����|�����s���u�O��t��$_��J�,����6hE
�H���@Nlb��&�N�����F��0(S����$/�����k���� �����7���u/��Gm�Z5�{^��$]�_��W�o�!��O�v]������U�����2_Q��Y7���Z�.Ai�7UT���k�����K�j����M1��lTA*�h[������U�I�4�0%�D�7�mh�����GqL��t<�'��$B�IzIRPP�����/��Ty^�kBCl���������`Sl�AUp�����
G��4C�)��_x	��a�����
��|bq9���"mc�1��Z|����n�2�;W�#h��BL����(�+�G0_�V�*�@E�*>�drqU�TUY�g%��.@�����`A��Q�:��j:�����1���bL@���
[/���\��>�������A����a��8�l�(�L�hd�$�6!���qs��x������(���,�3���r��������������W���o����-snn�Sm��(Z�w�PM<����F��&��x�������ds�0e+����N�P3�mz
����!��d��)c����={���2�*�7yWz�`*)#$0G�L3e�G�s�:�%0�U�yZ���C^p ������`�r|rO��9q�[Dwl�|����{�2��Id1nD�.sBA���v�� [���
qe�Mqa��B�0�0&y6tFw�J�>z���k���)x�w����{*u��:2�N���{��f!��(��3���'"%d��IL��rP��'�S,��
�g��m�c�hpK;�-3H�A��1������M*$�y<����a�e�(������B�`
4��3}?���\��}ew<��Q^n�aJc���	����Pp�2�����Tp��M2������p�����;�Yg$�H�
���h�0G�@�v��{b
��m"����/v�hF����x���?U����r]q�"G����)&��fw�po�61Z�g��{v���b`H������<h���`��!��N�����-1��d�uw�_���ca�sA]�:#xiqU��z���A�L�J�7�\��d�/s/8�P�
'��"0]f����0Xn���[5��(���dV���rC�Y���8���W�T���m*`���v)���Mm����
P���b���U��+�-5�����2[��R5P�/SjVe��5����ucd��5#
���;�U��"W����U�:��
�t#�-���f���Y�'�)*	4����E%\��X��E�p!
{�J��0-�� �|���G�r01HF�Rg�x�U�?����S�U��l;���v���;���@��W��5�J����Q����'�EE!�*����%$Q��S���%��E�� ���`U&I��!e3�������G��gc<�R�Hq����z���W2V�b�^G{�K�w�X'Pu��]�u����8"������J'\�X�h���}r
i�1����*���0��~��B��X��,�y���6m�&�R��������5�PIu:�0L�R�s1����K���/t����m�6,��`����hU6L��M���8�N��j���i��?fN�����,�s	�`�������n�9���;��J�-0���������Pu����l���u4B�k��]D���
�z��90]�q.�eK��	��cUcz��g�<�&C��U5��^�4(>CF����S����5��e���Ur���]7u�S����3�v6#���qS��\>�����yd%~�-f_b�nG_�;�*V�bJ�a64�e�Z�,��y����\�������z�/�(�4�hjq��c�1G�V�0C�O���]�!���z"���C��[)���dO�&���*v)7F��YqN���ru#t�D���h�/&�b,X�&��na&A�Y�\�+e����������W�.���Z>�f����.D`1�5[J��Q���z�(��o��(q��`����D���qQ�?�<�!�����0&X���B��R7��e\����G�E�������o��l�*�9�u�q*.��-��P��o���
���!ez�xt'U�d�U=]����\)���������{J��o/�����C��YW�g���Y�
s��>k�>5<�B�t�bN��t�2K������S�)���������?����M>�em)���#�u*�?e�������K�,�8�6�TUz��2��*��\e^�"O1�7T�Y����"�*���T�YW�g�V��rY����yV�~�P����U�Y�+��;��"��%�*�*^gy��R%^�caW�xYrL�����({<����D����*���\Yg<�ZY'�����*~v:�V<�[{b��P�S;H9;8�������wv:�V���Z�1&}N���D�}�*@:C���v@Ap<B�A��9�����~�)�n�K/X��^z�Y`/e����9�9�)
W|$��,|����0�N��4\��da�<�PQ����c9e��M�$y����
��+����?�;�X��h�i����Y
P��y��VI�<���C�f������|�@/����=�n �w8qd�����p��i�;�BV�	�o�61��Qm�i��|������zB\��a�.`�f�k��&��#oN�H����O����?�S8PW���B�nM�N%�r��r�SE���`:f���*�����:���c��G����	K��b~��������S�
@�����+��I�3�E-�B��Jv��7:������������d��\��d�!JI1��f���v�	����p/�F�}E�'�$8�"�
�����h�2kMk������,zx:#�o��P
�������9�+�N�h(E<���j��sx�;u�����{�i���po��F�j�@�$�q��v>�B�|���O�D�� T9d�b���V�]\�go��'g�9����������3L�d�Q��>���^�����_�(4?��<����BE=�>f�]}P�h�����%�}a&�1WI������������q&bJ2���*��I�U�o��+Z���;������AAq	L��%Q����t��b|�
�y���j���r��yT���?�^$�qW�R��Hs��N�^F����]����M�m�]K�2����]m�����/UYo�K�a���9]���{K"E��*;Q�����9��w�J�Z�J�h�=��Y��B'Y ��P�&�,��.�X��Y�R0U��X�k��
ob5��};yM�B4�������@?9`.� Qq{�a,�PAh�-h,9�c���h��+�c>�0>��F�;h��`j�8��j)ac�V�~�kD�L9.�zhF7b����^|V�z}�����5Y��!b�:�T���'%�$|�2F���G�4���#��a��$���.x�*H}G�������?�Gh��!7S��t�!j
��AfumL���U���crW���i�2���Y���\M�MDX;�7���u�E�������<�-�<��hK��������^F������W�h�r�z�,_�#4��k�7�H�`0�>�y����6-X��psU�������$�x���������3������#P��.+%4K�sG��t�|!Y����Q������CG��UJ*e$|���J�|����zp�g������2<��\"����D|�ruMG��Ax-3���7u'[���^f\*Z�����&��k�-�RAB����MgG?��l����TF�@���*���#>�s�?��u+N(�����;U����Y�B�]W��t�����������.���������������z����� Pv�n.����o��R��r���������!�U���?9<�U\E�R�����.����?|]�u���!"y�@8��O�&�y��1��S���j����-{"�����6EkD{ud��(\��/k
��r��{b�������9���,����+���7���qC�+�=�N���q#���Z-�����~X�f������%V��`;������3�nc���fl��SK*}b?����`���������T���+p]�`���>������0�����2z����'�ry�#��N.�f��[����QJA7��M�<3`�f��[�`����	1�A+��8$�S�������x�k��[��;1�OqU����w����w�:B4;U�V��oY�Bn�@A�j0��M
���bT��(#����J��'d(��iN-��t���]t�]�\�<*��z�'us]8�Bw>���R_��M��
��2�����\����x�r��/4Q���GN�7=��q���F 5����=�1����p�v���`@������6F1������ma�ccN�G�^�ozG���A��PK���qZ�dPKR�{J:patches/0012-Multi-level-partition-wise-join-support.patchUXt�Xt�X��Z�o�F�,�S���"����F}y\r�[�v�P��$��T�d_�����.�zXq��Z�>fgf���y��+���{�XLNN&3[L�'�D���G��c���h>��8�k��������C���<�s���X-�ob-R�V���C���Q*�u(�9����/E*��>������������z��x������M��/��7/��=:����E�� ��dk��A������8�@e�u��V�}Eq�K��\�o#?V��,�A*�P�s_6�-(w)WD�a���a�@&"q���X��>�-$�&R�4�B&���|����PD
|��eN����H����"E=E�9*s�4��#�<}�M���2t����.!����H)V������\d��._&���<�C�h1jK��*V)��nZp������>��F������gn�D�6�$#���FF��L��$��:�J�( ���JWD>�3T�Tm����n��/�l�k��AT�9����;��i�����=���oe�������;���8��9���^9�?Y�b�$�3����]�P��9�bO��������~z�$W�D���3��"7�<��ZB�a%�@m���XW��"��^N�!��L�����ar��nt�m/�}\�6HA�Q�a����x<�m�����p8�L���F������`<���3���	`����SB�1�����Q�:��8�|<K�8�W�������o����P_`/|��O�T��Q�K�_O����ZG������G^�8��A�;���gGD�!4�O��v���_f����A'w�Qt��(�K�#^T�x��t/�XI^g>t�^/^�tq������NI�T�j�q�!����b�{7�g�t<<v��-k��O����p�PzN��'{b�}t3�m�Oir�m���2���G��0���z���a���$p�0Pi��6����/��5�w{x�c���L�f��sE��M���C[a�C���9 ���&9j[)~�)eu�>io�%��t�Dax
1hr�����+�����It�
4����l���� P7��N|5����7t-k$��7��u��^\7�G���$O�l��
Ou����>�C��3��
Y�UW�
��Wc�"���R	J��a�����Y��EA��H����i������j��t��a|GK1)����
�	c����m��(b�K0��^QS5����J����n��N&-�\R4+��N�i����~����S3��D���B�:���W[��z�[K��v]��N�BJ��Y����I&�|���
��<'��\*I��p�I��fd�IP,�&�����R���LB���J��N���4Kr�4���Pmj���t����)�Z.3�@��&��Ve��L����:����rL���)�%tB�9�2miK!p6��"��$*�i���-&��[��
:��b)��E;;��s	����0��<xv�pQ�D��7�c&����7����s�[�4������\�T���d�T�>�:
�V �:��e�*����W����J�j���0tI�>R���%��My("��*[��X<4`Df�$��R��W�j�t�@��s��#��?]�<�y��L��D�����<b� >�S��=�X��|���dh�R���u%>`b�q*���Bh���+�5������@�o>�
�MJ��gs,3&�Q���=����8�%d�E<�a���/v���������'�������c���C��n+hJ>fA"��eB�#���bM&}{LrM�O!����H/4���wl�	������#T5P�������v��Z>��8����)F{�DD*�W@���71��"�@{��T�(\9����H��3����g.z����
%����L����H�,�x� ��{�����h������&���Z����������`6�s`zl<�yKf�[�	�y���f�������7o:�����D����;8c�wC�)Ij�����;t��Rj*�41����Q6"U�$�^����B��9�O��D�H����>��G@��V����g��� q�qc�����_��S���p�?A�NOf���c�}.�#�ag/�m�j�6J���:T��W}85�� ��D�#���+<����~/F���'�_b�kc���t����]��k�
g��5�B��]+|=���k������O{����eIK��_����:������}�pl�8S8���)�!����c�E�S�6b�
��,
>.�����x�����
�_s����=3�=���h�H}m�|N���T�c)��'+sD�������Y\��.��@S*��%V��IM�u���#*��]h�	��|���R����0\���q)g���u�m��~;����os;�j��8�7T����(�-]����i�n
./���g]�Q�C*A��l����a5��S�=�A7�UMus;��h;����/�l�u ��N��J�������������;�\���6v�����������A�&kL?a���m���~���Sm���� �u���/�(����� u�71Y���r���x�"R��Y���q�h�uy������b%",������@%���L=D� ���G�eKg��n��F����pu�4�!��&$r�����b7`�,�0�9��q����Q������?�%�(����3��l���c�r<�9�~G(=��P����tL����I��M�������,j�����
��?�]E)�������+�Mf�-E��yZ��_���J�G�\l��q�����%���ZmR^���f{>�G6�>=��Z�������r"�>�{]�b��8���$�_8���m*m��Y\@��L�(20���qy��pu��xe�,��������!�5���K`z�����Y��=>��3g�;��mt
>U� ��	Rb�S�P�����P�Q����\������-LbC��%�4�p]<gT�t�^������H��L����o0D�S�wx
��o�@�l���8�7r������Cg�:h�i*�~�.{���`�~l���Z�i�]1oKu���8:��qt��QS�W+0�;_1M������YF���6>��>��lhY��;q\{�'�$P���C���O�7��T����]�������Ji�kB���[w2
Hei���B���mP1	�fq��#�~Y'���.�/��C����9bg�t 9������6��m��:����PK��_���0PKR�{J8patches/0013-Multi-level-partition-wise-join-tests.patchUXt�Xt�X��[�s�8���o����bK�v���m�h�zw���������6I3��I�����&��;�d��������{����o` ���"g�@V�9V��r�Vg��F��Z���������`N� @����$��	%3��{��l�^�[����A��b�W����u����f���x�zd�h�,���(5J[V���P>�����|��L4o��0Z�y2>���������In�	[������%�m�����m4:�N���)���d�~z�z.y�����u��;��;�R�����v��:d��=�tT��]���;�@���.N:9�]�jA���'�*�~_�"m��}�^|�X&qA�V�
����g`XT��m���h�NL���N��C�H���:PR_Q�*�D���z5��^C��1<PO����]��P�Ui�N�R����%yuv&3#�S���Z;�l�q8���W����4�6��W�'���RWT�<W���E]�d>^��bdY3NW�9;�##�I�D.F����7x3�cX�^O��u<yiBS����a��4�UjhN>�f�����%�����/����1��r+��_MB����s4�8��dv��R���D=IH	���HpZ=IJ�:��/���9�e�c|..�����6�����b���(���q9���~>b�\�Ya����q99�(������x����G����t�f4��W@�6x���e�����=��OqC,��Yj�&f�6�6�����Z`P�����BasT@U]!W��
��+�|W��T���PMW@BWS,v��
�(v���	]�q$t�BW@1W����
8�
��
�E�����/;������
��yO�m�i�(���..�o���9Ur�b�����]�������_5_���=�*�!����l���O��.�����%�����H�i*xN���|�>��R���G�}n;��'�C.��`��K�����)6�J�������-}M��O�\+�����/0�T��Ol��PncFY�V� �{r���k**�VM-�r��R�i�2P ��H��~����c��B�QF�P(��1���!�/�g~�F�#�t�q0��z?>���4FZ$o�d�o�sp�;����S�����j_	��`2MT��m��_���F�DoQ���R�OT�o�x%���D��hf������1
���~���J� 
�9�i~]�����L��#CZ�H�+S"�����wx�H�=+���C�|���&�9�����X�d��c��S�,k�"�5�)���f�?q��v{P
��	A����%Bo�N#�!��S%��T/B?b�y��
�(�*��R�}�T�Qn{�K[����J.ma�@�k�����Q"�p�>P�1���]M@��Q�T�7��R������;�KM��������,
�~M����1��S���t�yO��g15EY�e�Kc"���*�^��;�r��}��A�(&�/����� ��i��(
���D�(��8�ba	��FAo>T��(��A�%4JR*�����6�\3_z�`�-oup�|��=�
N�A(x#��`a�^�O�,���w�y\�����HD�������N>N���f��x7Z���`�[G�G�R=WJ��L���T�4�a��J��L�]^��8����'��A|6�V[���{h�����K1����P�4�(P$���=9�\����D�V�n�|rpKnE=�8m�8�r��rq	���S��Xs[.�������x)!�W����H����p�Y�?���rI���>�RUs\�^� �3����h�8y���M�g��9�����J`�~��>*�VL����6����r1�%����`��CkF�5��d\7-Z��n���|��x[�j�Z7K}��T��W{s����aj�[���[���/M^���xI�U�k�������(��;��w�"y�(������0,��}���!�����i�l ���_~q��S���MN|@�0�*��~�[�2c�?�2��z��Q�2Ph���~�P���->���XF�b�$,e,=�e�Q�����Qf?��6�+a5o*�R�t�������fB�bPe��/	����e
�����a<��M��Y�>g�j��]E+`F�]5���PU���_���O��b�����^�=ZE�W��[�Q��N�T�z�y��%�����+�@���iX�`���q|�_��RY%�C�b�X)�^e��z�:S|4{27��e�Qp��L]Kk�w�h����:<�Im�/�Q�lCv�S�p{'��>`]����LCS������9��U��-~imd��5q`���-�k����S:>y��D����saK�����Yzm;��X���=h��&5��E-bi���i�2�uko]�����:x.a���]74��S�UO�;
�}_oL�
�;Kcz����uA�o
�v������]�.���uI��:;CDi�����D���[�T]�Ge'n�@|J��e\�������mB*�mYT��6���j��<�v.m� h�q�xb��{�_��^���*����.����xa���������z �)��m�R�-:4���'|�z*�0p�\+,��>������e��#��T]*w|�����^�1�?]>��|t������������������O���9�M����6����H�����wG��>��A��!w��n���?PK���p�
�PPK
R�{J@�Apatches/UXt�Xt�XPKR�{J}�'`��UG@��6patches/0001-Add-RelOptInfo-top_parent_relids-IS_OTHER_REL-IS_SIM.patchUXt�Xt�XPKR�{J��q���NG@���patches/0002-Refactor-adjust_appendrel_attrs-adjust_appendrel_att.patchUXt�Xt�XPKR�{Ju�w�DG@���.patches/0003-Refactor-calc_nestloop_required_outer-and-allow_star.patchUXt�Xt�XPKR�{J�]��<-@���5patches/0004-Canonical-partition-scheme.patchUXt�Xt�XPKR�{J���_
q
G@��3Ipatches/0005-In-add_paths_to_append_rel-get-partitioned_rels-for-.patchUXt�Xt�XPKR�{JL�?��G@���Npatches/0006-Treat-dummy-partitioned-relation-with-partitions-as-.patchUXt�Xt�XPKR�{J�e���5��5@��ITpatches/0007-Partition-wise-join-implementation.patchUXt�Xt�XPKR�{J�����
�*G@����patches/0008-Adjust-join-related-to-code-to-accept-child-relation.patchUXt�Xt�XPKR�{J0��%<8S+@����patches/0009-Parameterized-path-fixes.patchUXt�Xt�XPKR�{J���8_%�t,@��p�patches/0010-Partition-wise-join-tests.patchUXt�Xt�XPKR�{J���qZ�d:@��9�patches/0011-Multi-level-partitioned-table-expansion.patchUXt�Xt�XPKR�{J��_���0:@���patches/0012-Multi-level-partition-wise-join-support.patchUXt�Xt�XPKR�{J���p�
�P8@��k�patches/0013-Multi-level-partition-wise-join-tests.patchUXt�Xt�XPKC�	
#96Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#95)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Mon, Mar 27, 2017 at 8:36 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

I have gone through the patch, and it looks good to me. Here's the set
of patches with this patch included. Fixed the testcase failures.
Rebased the patchset on de4da168d57de812bb30d359394b7913635d21a9.

This version of 0001 looks much better to me, but I still have some concerns.

I think we should also introduce IS_UPPER_REL() at the same time, for
symmetry and because partitionwise aggregate will need it, and use it
in place of direct tests against RELOPT_UPPER_REL.

I think it would make sense to change the test in deparseFromExpr() to
check for IS_JOIN_REL() || IS_SIMPLE_REL(). There's no obvious reason
why that shouldn't be OK, and it would remove the last direct test
against RELOPT_JOINREL in the tree, and it will probably need to be
changed for partitionwise aggregate anyway.

Could set_append_rel_size Assert(IS_SIMPLE_REL(rel))? I notice that
you did this in some other places such as
generate_implied_equalities_for_column(), and I like that. If for
some reason that's not going to work, then it's doubtful whether
Assert(childrel->reloptkind == RELOPT_OTHER_MEMBER_REL) is going to
survive either.

Similarly, I think relation_excluded_by_constraints() would also
benefit from Assert(IS_SIMPLE_REL(rel)).

Why not set top_parent_relids earlier, when actually creating the
RelOptInfo? I think you could just change build_simple_rel() so that
instead of passing RelOptKind reloptkind, you instead pass RelOptInfo
*parent. I think postponing that work until set_append_rel_size()
just introduces possible bugs resulting from it not being set early
enough.

Apart from the above, I think 0001 is in good shape.

Regarding 0002, I think the parts that involve factoring out
find_param_path_info() are uncontroversial. Regarding the changes to
adjust_appendrel_attrs(), my main question is whether we wouldn't be
better off using an array representation rather than a List
representation. In other words, this function could take PlannerInfo
*root, Node *node, int nappinfos, AppendRelInfo **appinfos. Existing
callers doing adjust_appendrel_attrs(root, whatever, appinfo) could
just do adjust_appendrel_attrs(root, whatever, 1, &appinfo), not
needing to allocate. To make this work, adjust_child_relids() and
find_appinfos_by_relids() would need to be adjusted to use a similar
argument-passing convention. I suspect this makes iterating over the
AppendRelInfos mildly faster, too, apart from the memory savings.

--
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

#97Robert Haas
robertmhaas@gmail.com
In reply to: Robert Haas (#96)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Mar 28, 2017 at 12:54 PM, Robert Haas <robertmhaas@gmail.com> wrote:

Regarding 0002, I think the parts that involve factoring out
find_param_path_info() are uncontroversial. Regarding the changes to
adjust_appendrel_attrs(), my main question is whether we wouldn't be
better off using an array representation rather than a List
representation. In other words, this function could take PlannerInfo
*root, Node *node, int nappinfos, AppendRelInfo **appinfos. Existing
callers doing adjust_appendrel_attrs(root, whatever, appinfo) could
just do adjust_appendrel_attrs(root, whatever, 1, &appinfo), not
needing to allocate. To make this work, adjust_child_relids() and
find_appinfos_by_relids() would need to be adjusted to use a similar
argument-passing convention. I suspect this makes iterating over the
AppendRelInfos mildly faster, too, apart from the memory savings.

Still regarding 0002, looking at adjust_appendrel_attrs_multilevel,
could we have a common code path for the baserel and joinrel cases?
It seems like perhaps you could just loop over root->append_rel_list.
For each appinfo, if (bms_is_member(appinfo->child_relid,
child_rel->relids)) bms_add_member(parent_relids,
appinfo->parent_relid).

This implementation would have some loss of efficiency in the
single-rel case because we'd scan all of the AppendRelInfos in the
list even if there's only one relid. But you could fix that by
writing it like this:

foreach (lc, root->append_rel_list)
{
if (bms_is_member(appinfo->child_relid, child_rel->relids))
{
bms_add_member(parent_relids, appinfo->parent_relid);
if (child_rel->reloptkind == RELOPT_OTHER_MEMBER_REL)
break; /* only one relid to find, and we've found it */
}
}
Assert(bms_num_members(child_rel->relids) == bms_num_members(parent_relids));

That seems pretty slick. It is just as fast as the current
implementation for the single-rel case. It allocates no memory
(unlike what you've got now). And it handles the joinrel case using
essentially the same code as the simple rel case.

In 0003, it seems that it would be more consistent with what you did
elsewhere if the last argument to allow_star_schema_join were named
inner_paramrels rather than innerparams. Other than that, I don't see
anything to complain about.

In 0004:

+                                       Assert(!rel->part_rels[cnt_parts]);
+                                       rel->part_rels[cnt_parts] = childrel;

break here?

+static void
+get_relation_partition_info(PlannerInfo *root, RelOptInfo *rel,
+                                                       Relation
relation, bool inhparent)
+{
+       /* No partitioning information for an unpartitioned relation. */
+       if (relation->rd_rel->relkind != RELKIND_PARTITIONED_TABLE ||
+               !inhparent ||

I still think the inhparent check should be moved to the caller.

In 0005:

+ *             Returns a list of the RT indexes of the partitioned
child relations
+ *             with any of joining relations' rti as the root parent RT index.

I found this wording confusing. Maybe: Build and return a list
containing the RTI of every partitioned relation which is a child of
some rel included in the join.

+ * Note: Only call this function on joins between partitioned tables.

Or what, the boogeyman will come and get you?

(In other words, I don't think that's a very informative comment.)

I don't think 0011 is likely to be acceptable in current form. I
can't imagine that we just went to the trouble of getting rid of
AppendRelInfos for child partitioned rels only to turn around and put
them back again. If you just need the parent-child mappings, you can
get that from the PartitionedChildRelInfo list.

Unfortunately, I don't think we're likely to be able to get this whole
patch series into a committable form in the next few days, but I'd
like to keep reviewing it and working with you on it; there's always
next cycle.

--
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

#98Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#96)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Mar 28, 2017 at 10:24 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Mon, Mar 27, 2017 at 8:36 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

I have gone through the patch, and it looks good to me. Here's the set
of patches with this patch included. Fixed the testcase failures.
Rebased the patchset on de4da168d57de812bb30d359394b7913635d21a9.

This version of 0001 looks much better to me, but I still have some concerns.

I think we should also introduce IS_UPPER_REL() at the same time, for
symmetry and because partitionwise aggregate will need it, and use it
in place of direct tests against RELOPT_UPPER_REL.

Ok. Done. I introduced IS_JOIN_REL and IS_OTHER_REL only to simplify
the tests for child-joins. But now we have grown this patch with
IS_SIMPLE_REL() and IS_UPPER_REL(). That has introduced changes
unrelated to partition-wise join. But I am happy with the way the code
looks now with all IS_*_REL() macros. If we delay this commit, some
more usages of bare RELOPT_* would creep in the code. To avoid that,
we may want to commit these changes in v10.

I think it would make sense to change the test in deparseFromExpr() to
check for IS_JOIN_REL() || IS_SIMPLE_REL(). There's no obvious reason
why that shouldn't be OK, and it would remove the last direct test
against RELOPT_JOINREL in the tree, and it will probably need to be
changed for partitionwise aggregate anyway.

Done. However, we need another assertion to make sure than an "other"
upper rel has an "other" rel as scanrel. That can be added when
partition-wise aggregate, which would introduce "other" upper rels, is
implemented.

Could set_append_rel_size Assert(IS_SIMPLE_REL(rel))? I notice that
you did this in some other places such as
generate_implied_equalities_for_column(), and I like that. If for
some reason that's not going to work, then it's doubtful whether
Assert(childrel->reloptkind == RELOPT_OTHER_MEMBER_REL) is going to
survive either.

Done. Also modified prologue of that function to explicitly say simple
"append relation" since we can have join "append relations" and upper
"append relations" with partition-wise operations.

Similarly, I think relation_excluded_by_constraints() would also
benefit from Assert(IS_SIMPLE_REL(rel)).

Done.

Why not set top_parent_relids earlier, when actually creating the
RelOptInfo? I think you could just change build_simple_rel() so that
instead of passing RelOptKind reloptkind, you instead pass RelOptInfo
*parent. I think postponing that work until set_append_rel_size()
just introduces possible bugs resulting from it not being set early
enough.

Done.

Apart from the above, I think 0001 is in good shape.

Regarding 0002, I think the parts that involve factoring out
find_param_path_info() are uncontroversial. Regarding the changes to
adjust_appendrel_attrs(), my main question is whether we wouldn't be
better off using an array representation rather than a List
representation. In other words, this function could take PlannerInfo
*root, Node *node, int nappinfos, AppendRelInfo **appinfos. Existing
callers doing adjust_appendrel_attrs(root, whatever, appinfo) could
just do adjust_appendrel_attrs(root, whatever, 1, &appinfo), not
needing to allocate. To make this work, adjust_child_relids() and
find_appinfos_by_relids() would need to be adjusted to use a similar
argument-passing convention. I suspect this makes iterating over the
AppendRelInfos mildly faster, too, apart from the memory savings.

Done.

--
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

#99Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#97)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Mar 29, 2017 at 8:39 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Tue, Mar 28, 2017 at 12:54 PM, Robert Haas <robertmhaas@gmail.com> wrote:

Regarding 0002, I think the parts that involve factoring out
find_param_path_info() are uncontroversial. Regarding the changes to
adjust_appendrel_attrs(), my main question is whether we wouldn't be
better off using an array representation rather than a List
representation. In other words, this function could take PlannerInfo
*root, Node *node, int nappinfos, AppendRelInfo **appinfos. Existing
callers doing adjust_appendrel_attrs(root, whatever, appinfo) could
just do adjust_appendrel_attrs(root, whatever, 1, &appinfo), not
needing to allocate. To make this work, adjust_child_relids() and
find_appinfos_by_relids() would need to be adjusted to use a similar
argument-passing convention. I suspect this makes iterating over the
AppendRelInfos mildly faster, too, apart from the memory savings.

Still regarding 0002, looking at adjust_appendrel_attrs_multilevel,
could we have a common code path for the baserel and joinrel cases?
It seems like perhaps you could just loop over root->append_rel_list.
For each appinfo, if (bms_is_member(appinfo->child_relid,
child_rel->relids)) bms_add_member(parent_relids,
appinfo->parent_relid).

This implementation would have some loss of efficiency in the
single-rel case because we'd scan all of the AppendRelInfos in the
list even if there's only one relid. But you could fix that by
writing it like this:

foreach (lc, root->append_rel_list)
{
if (bms_is_member(appinfo->child_relid, child_rel->relids))
{
bms_add_member(parent_relids, appinfo->parent_relid);
if (child_rel->reloptkind == RELOPT_OTHER_MEMBER_REL)
break; /* only one relid to find, and we've found it */
}
}
Assert(bms_num_members(child_rel->relids) == bms_num_members(parent_relids));

That seems pretty slick. It is just as fast as the current
implementation for the single-rel case. It allocates no memory
(unlike what you've got now). And it handles the joinrel case using
essentially the same code as the simple rel case.

I got rid of those differences completely by using trick similar to
adjust_child_relids_multilevel(), which uses top_parent_relids instead
of rel->reloptkind to decide whether we have reached the top parent or
not. Those can trickle down from the topmost caller to any depth in
recursion. This also avoids any call to find_*_rel(), which was the
main reason why we had different code paths for base and join
relation.

In 0003, it seems that it would be more consistent with what you did
elsewhere if the last argument to allow_star_schema_join were named
inner_paramrels rather than innerparams. Other than that, I don't see
anything to complain about.

I had used the same name as the local variable declared with the same
purpose. But this change looks good. Done.

In 0004:

+                                       Assert(!rel->part_rels[cnt_parts]);
+                                       rel->part_rels[cnt_parts] = childrel;

break here?

Right, done.

+static void
+get_relation_partition_info(PlannerInfo *root, RelOptInfo *rel,
+                                                       Relation
relation, bool inhparent)
+{
+       /* No partitioning information for an unpartitioned relation. */
+       if (relation->rd_rel->relkind != RELKIND_PARTITIONED_TABLE ||
+               !inhparent ||

I still think the inhparent check should be moved to the caller.

Done.

In 0005:

+ *             Returns a list of the RT indexes of the partitioned
child relations
+ *             with any of joining relations' rti as the root parent RT index.

I found this wording confusing. Maybe: Build and return a list
containing the RTI of every partitioned relation which is a child of
some rel included in the join.

This is better. Thanks. Done.

+ * Note: Only call this function on joins between partitioned tables.

Or what, the boogeyman will come and get you?

(In other words, I don't think that's a very informative comment.)

I mimicked the prologue of earlier function. I guess, similar comment
in the prologue of earlier function is written because if we use
something other than a partitioned table there, the assertion at the
end of that function would trip. Similarly, for this function, the
assertion at the end of the function will trip, if we use it for
something other than a join relation.

PFA patches rebased on f90d23d0c51895e0d7db7910538e85d3d38691f0.

I don't think 0011 is likely to be acceptable in current form. I
can't imagine that we just went to the trouble of getting rid of
AppendRelInfos for child partitioned rels only to turn around and put
them back again. If you just need the parent-child mappings, you can
get that from the PartitionedChildRelInfo list.

I will reply to this separately.

Unfortunately, I don't think we're likely to be able to get this whole
patch series into a committable form in the next few days, but I'd
like to keep reviewing it and working with you on it; there's always
next cycle.

Thanks for all your efforts in reviewing the patches and for your
excellent suggestions to improve the patches.

As I have stated earlier, it will help if we can get 0001 committed,
may be 0002. 0004 introduces the concept of partitioning scheme which
seems to be vital for partititon-wise aggregation, partition pruning
and may be sorting optimization discussed in [1]. If we are able to
commit it in this commitfest, the patches for those optimizations can
take advantage of partitioning scheme. I understand it's really close
to the end of this commitfest and we may not be able to commit even
those patches.

https://www.mail-archive.com/pgsql-hackers@postgresql.org/msg308742.html

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

Attachments:

pg_dp_join_patches_v16.zipapplication/zip; name=pg_dp_join_patches_v16.zipDownload
PK
QU~Jpatches/UX���X���X�PKQU~JGpatches/0001-Add-RelOptInfo-top_parent_relids-IS_OTHER_REL-IS_SIM.patchUX���X���X��=�w���?���� )���v*;r���U��t_���"b`0�d5����� @RNv�o��%������^'�Jt��;����|�����[���yg0�x��\�������6���\��Xt:��?��t��k���P�M��x���T���������2Je�N%�y��W/�o�T>������[7��`�n��������a���l�?J/}&�z}���`�g��������2|�N��E��Y�gk7�Yf�_5������?\����|�t������7���������������q���]IXn�w7
`�L������|�7k�n��8�pz���|�����H����b�
^�S%x]m!�"�J��B���� ���?
������g�I�y�(B���!
VR�KI�8��FI�:P�;�����F8UD\�+�e�8o��G���g�����q��b�%��&���`�4�"�0��0���c�9P��
C�(�jD��f�Qg�����V��
D��(G�N8����a(�R��o�]�^�qC�W�R[�@rw��u(W0���8b)��Lw^)�[��a=�Rz���R�	��6�����	���f�I�����\���y�?H��8��4KR9s��}h
�@BOE�o�(�	����a��?\*����Dk@���
��yS$; ����w��"8��a0	�&����]7"TKF���������K����0���|��9��@�z��P�M���*%Z�Y^��hu��<z��J���+��L�X�K5��yq�YEG�MF�hM~���tS��Q2��kd@�
�.����L~��
,k6��`���]I�t+	!aKG��X ��Ol�-������D�2����}�.U\
��a�I
_5+	���
p�7����(��~�Q�a����f���6��mo����e�
+��i�N	u(�"x�i���j�3�l<H���
L��/+����2q�2��sH}�u��:�L[�D$&-�xl#+���
;��8d�������,�Z�8%�D�#_��/J���rX&R���H"��FA�.S5������a�d,�#�E��$t9�8?,n%����P�>^U1�
�66x\���Q�[�X��
� ��VX�0n
v���Za�c�|�j.�{	��VmZ��!Y��}��XQ�x��w��ZXR�4����W9�7$�*�/����L�I8�9���A����4�.?�@�K�)�[���\����3�Dqz��|�*��,b��	��f��4r6�hJ���)�?X�@�"��+���.����M�-�����u�%X�$��M"+I0���A��S)ji��Q��v	{���	`�FN�F�E r�@L�R�s$�:A���d�V�3j�D>UD�!��<("��J[�����C.����8�D����<�C����M�g���ah��n8C��m
���Yg��r�a����H?r�c
�	�\����*�q�T�����p�fQl8hv+#�#y�aF�VX%��,��J�v�cq�+�n�1Q�9�B�S��� �NIC�i�l%9uF�S4��@Wx ��F�I#Ie+�)1�7�Rys��� �M���BB�G~C��6MF�YX:*`+){md�0������L��&1��*�y�=��G�D]�b�
yUH��������c����N�\�{]��8���m�1��~2h�����|	r$� �A'8l�)�/�c�������t����U��E�@;$�#Y��z�5�6r�\K��cOkZ����)����bE�l�����}����\�b����$g�~-�P���D�*?q�K���g�M�(��4��?��'�z�E���Z2�|5C����cU���<���Ia��_��j������T�e�,���,!���4��W��U|��C���=lY9������2�]\�j�m��p��T���
�Y8K4��nWAd���-�5lO����|��
n��XS(�������2��%����%���{Iv�$����D�$���4�|�:���o%EQLJL#`�(����"E�G��
��&�p�,��dfi<C�� j�ZFn0?[�*�T�����'������8�~Z�����/z8�7�q��l�z����������/F{;�kp����3��3�=~*�L���to/��w^�*������Rwv�����C��.`���������~�DA�t���?+�2�����	�~�, ���M���,l v�{f5Mv�5�.�Q���"�2�Zt�M�[X�Q����gJ�(�TN`��j�'Tg������	�SXB�s�Jh���|��� j^�g6E�3@���NG� ��0�B�Z�����h�nQ������������������t������h0@��wv����^�z�~8�?�A�2I%�y����I�i�
q��s9>)����O�G���6	j&�S�����I�o��`�qV����b���a��c
m!^�J�!�z� �&k�`gN�,��6Z/���'��x��h��K>vN�y�s�}��a��m�g������\,��i����Ls��sh&���e/�#�G�1B�?��
���~�����vN�V�����WJ�$�`V~|v"�G��.0u!���:����f$�6Q"U�������Ys��^F~_U�Ow���������X�������E�c�*r�H6�pGh�q���x���x�0�.�]5�Y)�k1L{��(�r�h?�y����:��>����c �<NS�`�v�l���q�R�B��Q��PJ�e��j%|K
���	�n��6�<��#�L�d�����@�;�I1�H��oM�-
���	,E���P|5���e,��	�[l&�����%�P��i��I��z����?��$�R���4����'G�9�P/�*"�`�R��z�#�0m0)^���>�6��^�ZQ���@��(ECwjzQPm��C�0<��qNx�R���Ny�~�=9P�7��T�x���a� "������d�a���8����e�|E�������7��>0?Q�hy3vP�L#�7�9��J� �%�U0�� ��f��sV������wZ)�P��b+Y��a�|
?H`��CQ }�$2"�gC������z������&X�������,����wo�����8UL��<ydM������.�.�����x-^��Eb��-���~������.,	����:��8�v����Sx�v�8���q�a�tV�)�5@ ,�@I�i�(�1�j��x�������Tn����l[zZ:pK\���F��u���t�!1MC��I!_��0�5�������!�(��+���;�/IX�M����
Z/��\�o�&0�Vht`�&���%��-k[��H�y7�r���=T�n����,��e�R�h���6i��*��ys\!�I`�|�l��"����n���e���>��|�}��z'!4�M����x�����'��|F��S8'a70&�@�4��9��V@K[�{���-���v��w������'Mpv����fv�����_����4q��E��O$���IQ�m��0�r�f7�Pj���d���
��6��X[���2�dZ��-�����N	=@��\5���-�B�Y^
�2�1u�t�%��N�F�}�^�'�R��B`�������/���:	��7�5��p��t%�������2���O�,>��6�r`0�&E��+�~
[�_e�N��1&7�V8��Wo��A��L���P��;�[����M���Mt$�/X�BVh���~/������,�<����#zlU1�6�$�
l��EueKj��I��;��D"��MTA�6� ��J�6��c��@N����N�|�C�T��`������NM���J��;��N��.�~�_.��#�\j�L6�MQ(�G����@���p&=��0��;�]=j�(�1*��W]����m���N61�b�8����n�-��x�[��L�dq��
n��'�g**/Ta:9��1��+�Ib�Z������cb�|����/V����aY��2��~�����ww�j�^m���V2�I���y=�M��l��hH���<g��&����l�<2�Z���x9���Q����e��V*�g
������x�7������B�����@�������cM�CaV2����������L�B^T`�>�=�Mb0%3B������7�~��K���5��*4�]�Q�O���}e9�R�����b%1Q���D��=���j���������n�*�C"��=�V���>��A���������,E���������#l�o��
�']��x+��b\H,���!���674l����zjm�v��
9�����]�����@�
���$�d�&0�pV)Y�#��5Z(��%��!�5���CC��/1�W��e��*��F&d�2#lfRM@�w2�:�Kp�H$�*,6�<A)������<��T������{�}po�YS���g�+ZO#�DU
�8%/^�v������	�B�j���4:����.�Mz_�R��E�����������5�P�O(��:Ux��>4�}8��@�H�6�����yJ��8�'N�:��i��
s����+��1D���.���o�E7iw�|�{�
��p8��kV�|Q*��=;E���;��86&(�K}��
��"\
%:��q���f�
g�8��b��V1���gf����tl���20�e��!cI��0�U�pL��W:l��gn�?o��~����k3�����6����Z��R�AJ������� ���H�9��r��fu~X;-7�����q�-��p�]�������G}[��a���~��w<�hx�M�2< �����,|���#��}��Oy��N���P����A��Pv�@�(H�>���W����EM���Q%�����x�S��CT�[=���~���	A�t���H�)�*���0��(M�Y*�q������1�%N(7@������*w����O���=�q��~�Z����~�}n>9�������34���8��q��+
d=U���R}��t�����J����t���L�d�k���.���N�F�A�����;</�X����tjG�� 
�A��1yOU �_��6�h��EV�`~���[���M}��n��s<�,�
c�\�������v���qPX���#0���������L��B�� b�#�MA���\6k:XC�=��s�Y��z��(��o�At��.�kMV�5A/��1���N���06��Q�Gx�Y6�Aq�|�vt�n�]�?��|��3��k`J�4)�EA<\�S]��MN���P�SGnJ��6UK�N�1��������<���k��x�������f*�$x��.?�K�9t�'6`�t����A�������B������-$��P��_s�B�D����n��.��^d#���O�����9?;��6>�G�9*����.x?��l7���������>��i��V��=�G��E�E��eE������r�� ��f�����r���!
�2F)��Vd.4�n�J���TiV:R��4b"��>3B����<0'��$������p �p2O��t&���-k���
��Z`�jY@�4��fw
��O�j.XG�v����1�~w��������^�`S�i��`�%j�����I<^s����arbJ�O�����
F*@RTi��nX?�����$����g�<9[y���T�t-P���XAgga}^��8���L�"��R���c �����#� "�k'm��2�k;�g�m��
��Q�|]�Z��� M��V�Y-n��$���������;�<C�VL��`�/X��mE�P�
�.����x��������A��b�rM�{^�\M�n��"�`���}�v'
X
Ml�����o���z���g�������pepGyp}?��@��C��E�c��gY����;7�����*,���� m�C����8����]1<���*�>�����1�Vv@]�r��t���E�ulz'��v�}o8�;C@�����a���H{1m�e�������s�>�]�_���d������Py|N.�r���l�Ro���*����3}1��������(n���K�����S K?�����xn���w�����sa
r�<�����5���N{��������d�(���nJ��������x?�B0r���{80y��Npy��0~��/j
X���_���G59cn[@��<�a�n%���b8����D*�t23g�	��1��+�_��r�9��{j1Pj��.���������@����P;1_n�v,��$�����V���i�[WU�;|~	/C$��6���zr�!����fN�8��#/xRz��&>W:��\`�p���Tx�Q�Y��NH�$�5���-jBr"���vi�|�oH��X�r�D�
�{��������a�VU���6����q5��� �w)Q�%s����vv�9���2��|hZ�"H�"E*7�~M@3��?��v`�G�A`
b�i{�S�2�t�I��iy�y���y��#�(���vI�RS�K����������Kk��3;��!��;��
5F�}S�M����n��n��� �qn�{y2(�cJ}A���g��f��(e*��1�����������e+NZ(@D����v�����:�qH�%�GLBIv��Ntf�4	���x2��������r,����P���bJIN��d���E5������D�ta�%��A���O)��Na�������4���>��O�����E���X���D��9Xs���>\��r���$������s[����ZM�,V�U�n�EkXm���k�MmB�YI���"���,t���L��(SU�)�=�s�\���k���U����)E�(��X�F���� WV�f�`�$����-1@�|�>BG%��1^=���|Y�����K���5������kx8�����b���x���9���x,��x��PI'���	~������0�f������T��-^>�����{��p����jfsM(�M����`�L��GUap��~g�[g�ag$[/^?N�W?�F��]��yE�w���Kn����!y�U'b��z�����#��UA���U���l�,%���P���������zS�6		|:�N��|m�������
��i�������fo/�rnw����U=�������]������R���1����Fy�Y!j]p��x��T���)����o?�7����]�,p�K�v��zP�CW����IyQIQ�+�t[^�e%���D��.|��ah��4�4QB���L���N|0��S�4R>
���4N��R1�~P����ME��j�&
,�v�Y�+�H��m���S�/<�������_��������d�vN��pK�����j�Zb[�i��1�w���� �*L���7���zgvmL�V��'�QyY^��u���>	J&\�|��t�\eN����am�u 5�`�Xayc9��,���W�U�������dVUa���~�T�j��M��W�Z��Ty��|o�������k�G��p����G��l�F��8	8�sp�/���Lm���_��I�o�MAt����W��qz��cf���V`0��q����>�P�g�n�`���������E[�/D�:�Q>n���������i�#��t4�{�
������	�r0!�?zti�d��*-K	����������#~�^s�%�IxUu��[r��-��U@����70���o������yG����
���VD�ps�+������5�;�W����|�������vNig�wOZ��{k�b%��i�@��������:z���F��q7���rm�y�X��
|�
e�Syx?��>:����e�����KL���k4�63O��.���ZD��x,jeJf�<��������14�m���2j�X����_,��kS�Vj�������ML�w�����n�'�ag^Q��s�m	V����U�����p0e���JC��X��U�����=�G�����q�8�]�GU;�U�������w
���44���!����_�3|�[�Z
�)�C�5���/�k�!�!�t-l�bK�����L<�-R��� ���O���/�������"��[�WD�x�
�b��E��(�%�n{������OPK��a;� vPKQU~JGpatches/0002-Refactor-adjust_appendrel_attrs-adjust_appendrel_att.patchUX���X���X��ks�����+.���H����+OY��I,�����"l`P����ww���	�NO"������woO��p����?������r0�
���d���r���m���!���������^�9���^�o�0��Y��&a�b?�;a���������G������:�����N�s�:�L6`�������e=Z��z����#w�����g7�oX�:���f|i;I1������������?��$�M��x5_o�����Mf.�s
��w��#{
?����a��5����KD����Y�$����}� ty����u����Wa��kg�����]<���o;^p�x:=�p�[z��{�8�����{/YNE<����s��l���j]6�nW	�����6pp����O� ?E�Cs��u�k�'��������lF��}{�[���mn7�d������x�"��l���nV^�g���;�q5/a����>��q�V���(����"�s���0�����xd���3�_�hE
n�;��',\2?��S�������I�@$�H"P?��b���{���!a�V��5Pd
� D�7(+�>�-�vt�F���}����f;"N�,��J���	�b�� 
��4����������=�<��p�����Bys������ =D����(W{�m��x	[���Z���0:�����9^��'��q�I���o��c���������F��
R����'����)����G�H�U����m�luh�5�v����Wo�vtB��Q811WL������Y��b�aN�r���Ng����1���������&�N����G��F�i���\�s���4�K������zl\�B����������vm>�=��{���	�M�
 c�U��u���f�_��Q��R�&6�������0LLp���&�O������,F6.��F�����$��l��Qd�~��>;�r��
N��oQ)[MF�;/A���FG&�,,��_�M�,Y������V's�d'���b��@9������L��N���68��1�����/��C�dLv���MbC�w24Q8�_�
Z�t^�m�-�?�
m5	�z�����S�^N�2��X�����fv^"��o�]�`���[��������;�`k��}���&=��`������3�N���B3�{W)
�h�p��^�;����;�m04XM�6��u`M�j��	��-��9G�c,�qw�Q�!P��D!�.�_ 5*���������|v�S��1(�MTPFR�w����O$�I.45��]����A��5�9*�A����A@f�C�2 	7s.��O
[��m����#���-$0��a����l�e..�ij���!�.��T��
������|
��&���o���*FJv��;�5�2�%�������2�]�\&-�PK��P�R��aR�����p2�v�d���Z�����>��?��h~ ���<�'�����nJC�%�H�,M�|��|B��n�;a�`�)��KHV����4�R��N�]������ ��#����y	kA2�����u�X�L���]���@bg�����K���@Pp�@</yxv
R�$�!pu���N��X��|QK^�qv��J�]��	����`:�%�EP{��8�v8D+?Qd
�<l��������w������InK��I��Q�&���0���c�UvB��N��1f���$``��[�tY�	��W���S�����&���3�G/Y��b[Y	��D0����� ��mjj�����;T�D��@�����0�{�$)���A{)�Z�������
��"#���tu���.���s�����
�bw�4k��Sz���s�6 ]�y�#����-	�
�jJ^N$��Z��y�y ���0Pxa!��s*	�E�_����B�r+!������Y�y�tK0Y�S���.�IU�$�l�qqy�c��pO6�����sj�o��c��
�#�$�\2�k��5`7�N�8%G��XYE����#�A�F�x��iU�=�����p�$�����aT�\����0�p���������@�|�|���g�P~������tO�",�����QQ,S;�,��R�FS�?@E�L��	Y�
��U������2��������
R IC@�lY�.�S��K�^���E������R������Q�QS8:��ZY,_e�W�RG�K���&���c��Lrg��������,�������kc�sNup�j� }�++;HH��.��_l:��m0���I�R�������r�_����(�0?�� Pq���k�'Q�,�K����)�5.������QL-���L�[���^qLZ��	�&�G�8��$� q��8���#�
�Du����{��.Ds`�
m����@�I������u~���W����|�����>\�./�cp �YELx5�w�	E�Y���mb������^0��7EL�A�M����W��/�e��K�aE����Q��8 7�2/��|�,�`�_�AV$r}SN�^����0wt����-2�3Qe���T�+�M����s��)��O�����~O����/ v��K�M3��[�rz���Sh���������G�G �VkJ��L�
�(�c3�Ig:�F���$	�����
��U�V�9��R� U>�dx�������ad���N&3� uo?�����_�E���C��s�B|iA������X�����h����L��al��Z�F���#��"�������do7�d$i�[(�tbZSP���2�'�^�B�����Lu��K'-d�������&��9�5���j��D�=����Z1���/���4vR��Z/�K-��
3<2�Y�3��UBy�SB����eL$��\����Tg�GuQ��"W��@����s�u���o)��4��N�H��S�����lX��{
�����l ��u=�-��hW���ZD��h�����(%E�����M.�$����h��)�B$��������WAE���YS�>�K�P?���$o0Lr������FA������,W�Yz*���fu�ZinN�S�����n
Ty]��,��^��++�bm���K�q�n'�U�%@M�	t|O�����{�D~���g��lHE�}����`�hdZ'�m����tf�a
�#����
T*"Y�Tr���>�������R���Qq�k�tPJ[���T[<����Q
��)�>����k�2����x�bYa�E�/�:�(�
������`Wn-�Q~KF�k�f���KO�Z;u�b��u����*,���Tn��5�%]���#s�J�*GfG����jV9F�r���L
\���B|4�����f�a���w�����x:{��ek�Z��g��N���^h�����s����Q�v���E�x����	05G���xzg��et�x<����z�t]<�'t�]�a��yI�����`\q��{FX8��"��I�FxG�rT�$��� ��'�u�,�d!�ga;�I'�mw��M���K?u���r�={�������e�Y���t�?��P��X�s���<j�X�M�8�S��(����Z�0X�`�� ���mD����i��h�����.�P��7���N��qj��Ml�C��MuA��NS��;�����W�4C3��5�����!����(��*od2��~V[P
 ���&�������q��7e�0c�.�E��
�HI��)����asw��GJeD�(H������)/����;��N��Bw�{��\�
�\��GW��|�BXM��9����mY��9+�w_�LUk��6�G�~	���~���XpYl0��28��s�{�|&���v4��@�p
�c�x��g�&oQ*�Y�"J����'��U�
���$i/6G2;P�$t)Y�x`�]�%����
�xI�O�F����d��q���g����6d���,y��b|�@��6�M��f��z�Av?w�TL��R@�F�.�V�w��yQp�^�>d�q�<B�l��t�
�|M����O�2����*_�\d��j)L����H�\�D;�����P��$g���sI&�n)���*���v�������a�S�%R���h�z���|����pu�AP\��Q�����WD�AR������r���:��Q4AF"*:����$s�a�I5�q��}��kRE�+wE�KG����V��..����5� pu}3����|�z�16��+�WH���H'ur{��f�����i�� ��S�e�y��P�#�lH��A�������H]e��B�)�b�~~7UiN�;�Y�$��4CO=(��R���"��d�+�Y����	��=MS	qdN�A?��hW����b���[*��v#�s{��;��vj~�"Zj�@���q�jY��]��C����e�}��*ZX`���5|�d���������2*��'�i$�'�V~Y���X�u��n��2V��
���4�7�1N$;���D�S��"L�����u2-
�b�%�����;n��\��m�b6����/� �E��R���c�H�^��1��(]�^xe|��_���8h8EQ���:W'
W��.0�������:�����h���\�(�wi�0��LX�����(Lk�����
���P��25��/�������BJ�w�7^o��w�����:�0��J�Kp������<�-Y�	���Uz�~V���{h��2Y�%�E��*S�fm��[t�v�^VL�2�(\�?�9	�']))�)D��i���m�"bY�+F�q� ��CZ������+��a��or6&�GY�0^"���wd���A������	|G�Z�vS3t�����k�$��o������E(�2xk�EKY�Rq��r<�!����-��,���28����Q%���@B\���E���DX���k��%"U�'I��OX������S*
b��cC�%�lg�+�?�`���+�"C+�hgb�o&��y�%�(�n\\n��|J�
������g������t_�+� �yX��*E������t:|I�b�&i�!�wL�AHTQ=��2���M���W��#�<���o��R����?Ve�?�)$��CX������D0�z����o�'��|����o,yU �&t��#�w��P�O	�5ul]gIRt�;���M���u�51l��}���h��������cy�#2M6e���A������HV�YM����]���;?��Cd?�������zC>�.wc��@��}�0��O�����������y.R����z��)X�MW7�/��2 c�sv}v���������:�!��S%���ka��Q��{c>�Nk���W$�J���~�G�4hUr`x����q�YLI������)������}w���F�^qT�urD�#���}�3��|��	�����U��G�r�=���(�_�K������u��k�q�	F�;�N�C��/PK��
��SPKQU~JGpatches/0003-Refactor-calc_nestloop_required_outer-and-allow_star.patchUX���X���X��Xms�6�l�
�����,�N�d[�]�^������v;-�1��H:n���$�X�_��w��� <x������h�f3�.&l�bL�#�,�&i��tr>�O�I?J�`	������(���9����z�2R/�[V2_��s8����0�J�5f�0����;f��Z����)z!���eDS�Eg�(�e5�Ss	��y������x�;��K�T��<Mj�KY&
�\q�Y"W�]���`y.��6L%:]b��?$��0�-Q#,V"5\

�,�[HQ��pz��n�%S�@���bV�"�����5��0gt�Z\a�3R^����b�V�*b/�6�2+V2�����YR�XHU8�����*��4-���UA��!��fi�6���}����A 9���F�Z�kX[��dYHm�12}w��v?�^gp��%}+5���\�����
�l^f�����n�o�����`0@�t8g�5�l(K��������7a
`D(�z�������3'd���8����E��2l������	gN�t���9jBW�QYL�	�6��I��0G���$2�X�`p�
���>�\�E�h�i4�da8���	"���|2��~�����>y��t��g�s�g@_�,K\�#]��or&��T$p��4��tH�7��b�S��\�� ��1�����6�S�$n��%nU��I����Ve`�<N�;��"�	����&�~��q�nQ�7@��"y����O����m���W
��v���y��W���_$�:�S����|I|a��>*��T����3c�
J%oxF������ilLm�;SV��f��j�,C'6$�����:�7�rVvG�7Cw_|��n��X��6��Ge���"E�wx�M�w�����!e��	 
���8����7��~<�(6�v��lv�B�o�T���~F�;?j��'��%#�\����I���x{�Y��o`��m�� {�S	o�y����_�c�< �
�=�o�pLos������n��Fwx)m����	�D�y���i� x�y�*��x��'�����m��$j����_��!R��
��~�7S��3?,i<AQ�g�������YvM(Q��NN��W=�\��-B+}5On�T�9�����
:�n�Q���e���b��H�\�g���G��K]�B��p����N}�n0�lWr��ry�m��
pw	z#s����z�����|���u�Y�TTQ���F
,��%�H�X�
�E�$u����B!�C9}�<��M-�j�����y�a���8~�<��tl�i�Z��/���!�7��
����|��J���CDM����)^K{{R�W��b�}G�K��T�
����������#O���h��L?����OB���>
��i��>�;�c7:��]�iQ��/1�L/�
���q=Z�p���^�y�~��=`�������kV�����4&{d��n2�S^�'��[���e�G����#�u!��m{��W�l�����������bv;�ub%h8=�f��@�aX
�����j�(���iW/����������6�k�)��.���Gz��m���r��p?fpq�g���Z�P*�]�i�x���W6�n������u��m��;�}�<�W���(:���"#B��\A����Q1�8r���8���A	��=K{P}sW��Zdk>?fUk�8q�[+�����:�<���s��@�	�"�#��O��;4��}Mf��y��N`�p^�gA�/PK���]�:PKQU~J-patches/0004-Canonical-partition-scheme.patchUX���X���X��;ks�8���_�dkk�X����dr�q��k�8e{n�j6��H���"�i����_w�A�%�f�\qd�F��������gS�<���t19�f������`�\M��G�`y����$fWb���l4zI��d4w����D��,�k��������p��q&�mJ,�~�y�}�3��]�b��;�d�z<z9;|9;f{��t�����~��������=���/���I�<b[�fam�_��v����q&��&,w����2�5y'��.Et����U��Lt`����l���%�I������Z0�7�k!�2=��5O�iZ��X��	����I�'i*�6���BiqH��}���N��z�H��&��<����TK_��+���2�cB_�Z���=�?H���yg7y���-g$�}��"
3��C���_?t�U*2�i,��pCF�Sa(��pt�v+8��_�Q���0(K�m�!�r�6���RJ�\��Q�Q"#N(Y*�V���q7_�C��Rh.�����uA��������e2����qp�l�p��Hb}��?��g�c�r��mZ����6�����i��#;��g'>�L�B�cl�4<�}��G��a�p������Yg�V!2�_���t4�H�������n�Vl���<�?���uA�W��b2����j4��l�h1���<u�.������3�?���9�8d�@��C��8�I���'��,��4I��k^���:A���u��/@��l{<�RD _�pg�N'�g��C(�S������@�E9������wu����3���CV��q��?����������(c?�������D>����X}�cc���C��S`/��HQ�������!L����������i�4��BLV���:�V2���(\������;7��=�)��d��RY��Qr?$,x+���#FO�><�������7������C.�6i���}��B�G\?�p���#��X�_���
����#�>�|���	=�U��@We���U�����(f��1���x6�������|����
��OD����wq�����x����g}�Jf�����f�M�������0���nd�G�e�9����1]���F�����B��9�B.Xc�N���$[a��L�V��1���<s)N$R-�������%��d��;�;�:!� M��2Yg�F�B�0U�A�qO����+�X�l�P=2�X�u��d��$��
U�U��2+���53�KOo�>%uc���Z;�g{{�S#�,������~AI'
��>#
5��6j��lDd����d�[d`�`*n���A��P{�b���%(��#�`���f<�Z���O�#+���@c!]+�&�����R�9�$np�w���B��>���-�$�hA�d��Bfi��
��Ca��@��- ��$(�-
��x>A�5��?���M�u��u��'">�H���
�e--����	N�Q,�T�R;�I�X���������=�i�(4(eek��A��_��-l�U��O���Zs	��Nx�[�����^��e0s�V�`4���d�xJ�YA�+����LNQ"��.�����<�p�s�~�4�]�-k��8�W��x:'��*n�o�m�����1���������!�yq�{^��A����������I�RHK3������e���9zd�����*0)3�'��J.�g������^�D���^	�&Y�����n��k��2�N�����3N��
�+UA�u�+����~��.Q�����w��h��dF�BM��]X��*d��I�f�c��d1�P�j�MD\�������9d��$��D��X��`��!>����Q�!����i��c����s�)�a�deO�ABb��X��C�,��<~pJ�������������D�^A���C���H�����`
���hN��y��{��onD���J+P3`����a+�����G��'�S�G<���������s)�2X��P�I7J(W����O*th|n:����3�e!�����4��[JKa�F1:�RKZ�v=��@����+l���"}�(��m$�
�).������@Bf��1�!���-9�����n�m�e�8�����P��z QE��kT����u���:t!$�)�R�u��������B�A���j�4�8(������q��}S�)yD�b������2
b)���%�?����|ry}~}~����w}���3��;��qB��.�}P{��UU�O���.���{+4���+���0��Ef����E����j�&�����J���*�}}N8���[��#V���He�BZ�+�*�<X:�:U�%h��t�6YNVJ�v�20g�_U�p�r��W���V��P*�q�&����9b&J�(o��
/����� ������YY0Q6vg#1��!��~��a��������R�X7����dE�+qRw�Y����Q�-�����/��f����i;%�V���I���M����o��E���7��O���SaJ�l��J�+����S���b���g�W�Uq��UN;����
�����f���S�X>������<.���d0j\5�.������I �7��:^8��%�6[|
���~�l����	�U`�.\�E��8.dbn3:.����s��JI�b\]7�>���V�
�X7��H�kB|��{04h[v����B�}[�����o������~���9�U-P��HY*u�k?e�X�Z��5����D����
��Y/�M��Q<i�.k�����W�h�>�\X�(��d�d3lu������u����r+C�
�B���;�LQZZ��#������EuKe�E��A� ]�7b�o���H�+�	��A}U;��5����Ib_K������,��1��1c�B�'Q���6I�
8P]�6���E|�o�W�����Qf��Q~\ur<��bK��u&���[�(Qf!�mR��T��9�6�A^�&oC����)f�)[�^��������<������X�/U<@$��Y���?���������z�}l	��=e��dSMD?6�����M1-���7L�#��H�ks�i<�g�x�6u�i
>��=	�ZdD����8]wa��T0����@���
���t���6�0���|�e�������
UJ��w�/?�r��|��|$K*����c���@����7[Ya��N��A�Ie_�����jN�����eq��M���P����J�[��\�|����rj�+����S�fnY�OH�T�o��--YN|n�I������:��*����\���K��`��k��2�
��i^n,�zLwC
���sF<Io-xPq��@�
'|�/�]���?�dY�I�@t���B�}~��P���!�Z�W+����;�A������oW6*�)a�z�Q&�v;����n��R���"Q�yY�5=����&��h��������L~+�A��jN�/��r�o`j����
��Ji<�������g�/�Z�C�*�^��]�Hm�Tf�������"�kG��t��@��>�Y.���RAS+�>M�6$(U���d5x��b��$Y�H��K�P�w�Ax{Au|zE�����:�>�����.�������V�X|5���9a��J�QG<�qO��m���3����[�����mg�[�S~�X�&��D��������������C�9����c�)����"Mu6�PU�\_���S�b�Z������4��A�z����x��W�������k�*B��]q��r5�Z��k�����2������2��6
mU� �W���;�T�-.�o���2��x�i�,P��-�C�\q�`����(�`���W���<����m�9��������*�i����P�g���i�Rhb��=���%�<�����xM5[/Y/wkU\,K�x:N��|6uUlEP�`+������e���:����Z\�����@a�vw������(j@�B ��AEL��6$�
��U���(�,J�1U�Y����DM����>���5��������%]�_��=��$�LR��'��Q$Y(���d��
��S��e�t�0����R�T95�:
>xDZ�J������z�$N�Y�>�WYP���:��.V"��^IIv��p7B��A$�yqA��|<����e:9L�=�%|}H���������bmC���_�K�/4��J"�GW����X��#�Z[�����"��T���M	��~?�T����q�PiM�n����E�&�CT�.��UV���7��*a[��1vW�����Y`
���*Z���R��o�$���="})�=�+R�/���$|����.����zk��e/c8���L�o���4cZ�7vB��fH�mE��HW@M@��;I��*N�*g/�;NN��x����/�����@�v�_��F�,�j8����=F�*�:>�^�c+|D�c�L9�l`�p;�m{`6���\�A�^[�����
zw��.[&�r\y&E@�4���!u�G� 0�"]�ES�s��C������5;B����03�S��]�&n�8~4Ie_�({Qy�������W���X������X�jUY��f�R�*�%:����]]s���|�7m=n�bM�J_t]�G�q���"U.��H8����P
���*-�.�[���8��>R�7�L�A������zc��:�@�z6�+��'�����x<��a�"����{���y-�������%�t�NIV
K�p-{��P_����ci��Wd
_*�,���/��mQ�tE�nS�l{��T�A�fk��7|���L��F��Oe?�2���);��d�T��J�]Y�)�����m�D��~��n&��a�]'�B�&J��@��V����T�������p�����;���n��PKM�n���?PKQU~JGpatches/0005-In-add_paths_to_append_rel-get-partitioned_rels-for-.patchUX���X���X��Vao�6�l��Cvl��e9q�uR���A�o� P��)A��f[���H%v��s�����������r������g��0`'c�x���xp�z)��������J�x�1������u=����DfkU��Y����w'��w\(^Wu.y9q�zk�g���r�{0@tV#b��{:����k}ZG�y�N������#�A������$!"g2Te����$�y���
WP�Z�*/�V	iY[����Y���xD�$��w��\*.j����pfd!4`�B7F�B\�������e�q�M��I�Q��l��@�q?b���T��T����u������Y:1��������L���5F����v��e�r��E(���`�C�\B�1q��=�����_���A	/�y�;���i
�}�+`����g�"���$u�8I��A�%��E>$��B6��;�=��<,\�04��H��3��pT����5/�j,��Y���j!�G;ey���t�&��Q1���@��j�-|@�f��Z�)Wq��`�-�r\&��}����?�����\O/����W���l9[�������r���&�ABN�bW��|�i����oDo=�<vh�m���0���M�:�V���B�q�GA����Kh*6&�15��Q��'���M
\����}Vx{��m=��.�?�?�o/'�@^��?��;\�;|@3����_�9>���&k�
e���$<k�!*nH
���C�H��>�$�/�@��N�����-9��*���aL�U\*-����e�w�<�=H���5��[#L���g�3����� az�W���,�����d�=R�=U��0�t����|�\�k���u�p	��	p�����Z�k*C&�53�*n���m��
{��Y�-`���~��8�Z��o���J�������6���x^�)d!
,�s�@�t-b
�?r�q��s��aRCa+�V�!%�K�nEsm��y���a;0��o�����[aD	��5�0R�����}Z��6y^P�84��:�b/�����2�d����S��uWc���,6	7%;��[�j�1���1��m������������"����������z��I�w�`��]�w�v�b��)�/�FR�16�g�����8����H�>/R���@��$���L�U56j^�e����K����H�S�`B����������g�IH4�k�CY�",�f�T.������< �n��:��s �zR���/�V���d>�^�����s���X��PK�u�i
PKQU~JGpatches/0006-Treat-dummy-partitioned-relation-with-partitions-as-.patchUX���X���X��W]o�6}�~�]_j��,���dk�.]�IW4��0E]El$R����w/%[v�������L��~�s���M	Ir�q�G�X���,]�+%q�Q6��d�''pi4\a�
���`Eq�����K�7��~�p����a����U�,�I(M�<x%��u�c��p)j�F����l~���(Z����I>�t������go ZN���p]#I����J�N9e4�Pc!�+�)��'�
��;�a�g @��HAY�L����E�U�:]���U�`�A�8��(������r!�O�Sj�[g����n���\���F=~8\pg�S	�pt-�{��^�Z�UeU���1�!������P��rr��h�I{i|�a��R��
���-�S	m��j����M��X�R�s�&�>�Cq\��Ao����y,�0��M�>���]k��X��*�����o���)=��/����E
\���I@��	$�tH5q��Bx��K��d���4��r�����>�S�>������;�Q���k��<��Zm��(�2U #I�L��ME�}��!�@��z0A��&�j��>.��qv�?���).��1���hyt��yl���{t�/`��'�%���ZD{B��lh����m�3�P��a��cxG8��08���
=�4�����.qR��x\=���B��+�v-��B{w/}t�K�6Pb�`ma`A���������_��qo�����.��6v��`��;�&IEG��%7��;'����D�G�M�*������s���7����[��s�#P�^��{j�1[E�B���o�5w�>��25�&(Ec��4-�����
�5�$�,�;�$Yf�>�v�Y�Z�,�bX"}��B�tB��o�a����xo�/���)y�����)�7�������hf��!����������C��C�;G���#Q9i����-����r�������c��j8����~�j���W���u���\�H5�<^�l�h����M�����O��l1�l�&O8$����|'<�����Y4�y�`.?<&� ��/��4Q�P��8���Y��	\�!�>1�BI�����_���V���u1������NL �2���#-���"e+�����cd-�+��,��q���5�l�C����5�fZMX
�Ri���
�L��k��������&��j�����<�����\�y���+z!���~F�����N��m$��������@\Q%��c�Fq�
O�E�PK�p^K�PKQU~J5patches/0007-Partition-wise-join-implementation.patchUX���X���X��}iw���g�W��@w��
%��r�]INN����&0��@4c���Zz����|�������k���|!�������~��/.O{g��x0���h2����W�����R�����\���������A�����|#�������x���J|����#��/rY�bUd�LG�q�x��<��7��"� ��I���~����7�S������?�G����������^���q��'�.)������-��������\@G	t��_�b�>��e$�[)������s�Y�m��*�e���8�"�U��K�OD5�JQ��l����b� (�^����*[d���������o`@eY��q)���0y���1�/�����x��S�*L��+�����E2����|.�,<�Z�x|���YW��+�]7�&%U�7s�b��B���\L`���H�eI�����($L{���B5�����;�),WU�y%�/+�F��3�*�j%a����,`e�gR�Z����-�����4�~�^�:��u�B���vZ3����g0�4���E �����}v\���b~<���l���E��o����h���%�Or���^k3��6�����������;V���hl�3;N�s�YRl�?=�~�8�
�9 �i�-}�"�/����7���
9�\
����5�+�������kuz������lNcY"
�����M��X��y*h�M�.��7�����h��ggL�����O�c��(q�G�-��u*���;f���������Q!���C��Y����ts������mh����i2���#630$��RS e�1)��w��L���|]�&'���hp�'�@���T�q58
V��.[���8;��b.�����~�M&��h�
<n$2����l��_D2>9������������z���H�6�����/G'��'��sp&�����oaq��
�yR����&	7����z1,���o��zOD+�n_����s�#*	\���9)�#P��������Yd��8Gr�P�|�NW������y2���\mh�
����=Q���0���f	Sz���FHuD���n%���|.(��
D�(�?�o��Y$0��:z"xo��hf��=�pLg���`@��v��p��S��`��)D���;^������6.��xP
��������.���3`�x�p@���2�-$���L*u��J����s������)��/�MZ�0:�!�SU���d�9�E#������S��\����������/w=�p���`��$+ !4�f�q�j�@�`�F��������,���H�
����6����t`��;�;M���-y���$4���8�'�d���r�!�{t�����Gz��-��%�������j=����c��8�������bE�'��d��v�R�������0]�Xis�wq�9m��C�<��+���+��
.mD/�}a��*Ib
�<�1r�PkT��E�OG�D��.P�_V�c�qX��{�o�g�v��g�[�����d�n	��;�]��.�1�79�;D��g��W�'	2����uyG�5���������������>\��������W�� �-�9���qS����aW�
��XB��g&�B,PBabfhUF�[���P�x��5�|H�R��A��a������ec)�N�e�e�����8K@px�X	hy��u�!�+`�P^]J�^
�XuIE�`���U��a;U��q�0<l��HZ=����'T,�Y1�K~[��1[�D��j�/�y<X����R�b�u�������M��������u��&)�a�@���1�v��%��i����T9����v����z���z��'�G@�����7ko�5o��;�E�IG�a`($+H�T\8�����}@�=4�#n���-��:��D�E��k`��'F�xE�Bi@�!NbC�PV@?�����X��I��i���w�����J&�k@��0M^B��t���vI`J�JI�Q���Tk��(sE'�wD�K��"�x[g���r����"��~�{d��#�����S��k6��Y��n\���(&p���[����T;�]�^������I�5��gg���>���X
�?�eR�/�]���Z��i+*et6NF�q�{>���h�j�R��l���%J��?E��T�C�����i\0�����#^!J?�b ����x�����#��#E��noo����o�J`l�je�� ��p\2f�a@Y*S��R:(�+��=��?�C��)P;�L!�E��]�<gM�HBi`	�Y�wB4G6�I�j^����H�9�`z2������T����cb��zh�B�x��:�]��-1	@�N�s3h�C���|J��`{��lB�������K�.����*M�c����6�zX�P�1��sx�/��4Q�`�k�O�W�!�		���������q*�.�'�m����mQ)����F��C"9Y/�e����)��4(���������y?|~����ijW����'�W����������w�:D�q�8[�_���xB�D��b�5�D�@�P������-{{�%��Z�����x�T<~,����1�~|c~}��)��o�iGt���8}�o5��!��V� �%�9R�,Q!���<��)3�L�\�[Az��9IS�BY����Z���B��Z���"��OD�O/K���"�@�y�]�)���)��W�*���	�����������.��\5p��]�����"��Y�p�"�0�\�h^�jXo�x@w��|�,+��eET�|D�����7�g��K�j�#������1PC	��{��S��{�����!�������PC�!�nhX[�6����5��N���3����Z��h�	���b�8h������K�G�<K�M���~*�{Ct5K�����mvur�9����#Y���;��]�\�t|G������^���,]����q��P`�d��"�q)	��b����)
$-�a
�X�
L�K����Q�CY�K�Sm�St�a�h��Zb��VO~q������S�������Q�����*��	=�E9��;�s��G�1|����
qP+�+@�4'~�V���R*	�0��Nf2=���������9����Xy���j�;���R���-)�����c���j��������tU��jC��5��A����4�dF@@�J�[V��\�B���zM�2JV���[w��c_��-l+�}�sFs�GO�=��Zo^j���S��XRI�{������P�[x�d��CI��!<uj����h����7�-�G�yJ�\O���E���'�&��@sS����=�;�@���������W��|=e{�b�|��+�q��E~+?K����@P��FM��V��)������|9G
D� T#� �a�3� �6��-��������s�xZ=s��XUw�:Rl� �/��E5�����������Ol6R�"���!��L�n�e����F>�4}_����t]�t+3�0�9\,���q��@�
�GX���=*�	iI>�%���;���N@�� 4��e�)���[��1�	�+��iF��PN�CP���_��%�C��L�3��#�%��TK�0���)3r��R�u@�_�G#qX�����n���5�GS)�(t�r[����b=��������fZzq�g7���{=�dAj�nW�N`K��:B����
��
I�6���H6������G�p�200��(�Z���l�Q�"��}��oe5�`}]���%�y�;�vl��p��a@Q�����;���	
	�;��G��'G6b�����C1�d���R��p�"R����i(�R��UkL��M_�;��\��W�m��i�y���{��3��;��Q��PS�!Zke�
�`@�5�aD�<��O}"����F�ZK����6�,�����jTuo���B�j�$\q)p�e��g� HT�*�udm ;�27*C����	�e�8�������\���[�
X[���"��*v��t�I�t,��U(0*�<\[���M�l���~{���`��\2�MB<���
�5������+���Z
���]������P���S��p����mF��� �z�a���/�����C$���%�s{�:7g7'
?����z��uH����n��BG����q<����l�u�^�rF�iI�0UKh�hd��t���
�L��W!�1�$`R��(�5y=��s�u��Q�����NuS�a*W �^�>j~�Y��'K��ft^=����H�r8>c<M�#Z����j.b���6�t.���W����P���"7�su��Yh�<�� 9�
�Rj�����+c�\
@��R�����.�6(��U*GkT���r������4d��9S���]��u�J���]>�]a��g�%��@n�T���#�-�����<|��=3v������	[amE�`w45-�U�H�v�:�vLb����U+�1��x�����>P@@�Z���w������q����he���q�j������=���&';��H[m[N]�(���	Hh���S~x�@A��1��x:�b��������m(f��#bt��W#r3�m	�Z����0�7���u�=r�6�����h0�<?���d�^�F���i�9uYCH��'��w��a���+"Sz�KU$�i!�7T��J�=��N�F�Y�U<C��,��I@����~�����3g�]�|Pi�@������G��*�q��f8Z!��4��J�����&9��vP�
$9��"����84��?�(W)c��!���3��Q�G�1����!��������MSf��
��]�I���6�:r�C�IF�o�����0��gF�H����#S��g<Gy�*�IX{~��#�^�����V���BkU��T����4�w�f%?�U������7�����T���ce��Y�Kd'���*�_|0��s��'N-Y���������n�@Aa�����kl������r�m5�i���vC��Cy�����e�.�Rs�
���h����Gr��O�5@d�-���k��������dc���bx�JQ�-�������@��xi��")��^��{�[���L�v�~�t��u�����MH��^Wv���[/O�g��./������?����li�+[�%�w����|���<v�t�����=��������_�D�)}�l9�I�X���m�P6��yM���C�
���W��]��N �'���(1�w,PuBz����n�S�Km;�^(�5���I):������Q�B�G�`����u��y{��������h�1
��[3�[��X�P�3���!z�;�pV�
�$���!��w�\�mA�������z��=d��-4��P@t�\w1�b������8^S�6`���q��<��E���)r9��2rRvB�����!F���e:c'}�+.N{+�\i�NkR[
�g6���������X-�j�{#X�T��<�o�:
�3O�]7����p#Pv������Ul�R����&�=
����7�z�-�����p�@Y��7���HL"�*JD����It���2�������B�J�f����������K��N{�B	]���a>��
���1��K��
./;�B�{����n�H�H��l��-Km��%G6L!�i�+#hAh��q{���nm� �����.r2��Y��+�H���#�\tsb>F�^����F����Z	���(!�$�H��J��M=�����j+\��W=���ueh�����e�6�t8��4�A�V�:V��,%�w���:2^-���dv��0w���m�3��y�����U��u5<�%fu����2�P�R��<��G���M�������=�|��1�mc���3�L{{��������<��I��4c�u.b]smFZ��k�2V]}����&�Z�Y(p$�������t��xT��"����6��d�tj!Sr�
��Z�������������'���dg�(�A�#���*d����/�!������C�4��F��P�`P��1��K�pj1���Q�&|�U,��V[������!6V\��Oy���`G��Qv�c$�����De������P��F�f;�.�!����Iz���C�s?%��
L,/[Gi���]X+�7��h��-�17n=&r>m�M��#���"�����3�g�F��BT(}�?��/�jb����ESd�����@�J��qH������1�7����_3Q��tt���Qa�8�^�l�$p�z����V��s!�5�����"�����@Y�Yc���%�2L�3>���� �4�_*fD;�d:���t�����Mn}����-��n ��#l�{�1.���l4T�$����m���(���51�PCa��n�B�G�O�!���%��r����x+���.$����T^��x�d��V}m��;y{��(=2el��s����o��q=7����#�6shVFy>>3��B��\\�`��R5�(��c;mF�2F���T�q��j��"5V���a���\���<�2�yIy����LB����O?0*!�d&ttWm�SP��M
��z��Z"�5 ����Nw����?�*&�Wj�^������0�\m�?�u�<���p�Fl�/hX���Me��R/�,�:�R�[�ux�'��4N�>�C�����r�Z��r�>j��*Pa8o 5��������S����~�H�nD_�Bi%����<�nwi�UW���zw�T>��i8����E�P��9������C��k��?��7i���\}B� e�	�|�r0��e��G��b��Qf���Y5>���7q��E��W�yf�c	tO�����V�R�9E�L"CU}(/}�n���i>�?*P��������.�D<_�m�>�O��_�2��G������#�aC��2 ��J�N;�,��Fu3jC��(3����������e�	5L�����F�61��S�������E),�9`���|����t����&(
���d�Uw�0xSTh�8�1��������R��
y���I��0�WtD���U���Wz&��JK�����*J����A���91�����A^ ��p�zvh�j�����B�y*��^������QXjGO�b�x�������*���Pb�Yj� �u�y�7��q�v�S5��c�����	�����xG7�"�
Za
�GL�8v�	@`�M�4_^�C�iQ�ou��3|&0����x0Q���(�1E�iMJ�N�kP	��R�(@�����@���$��(��*F�v\]�P_=8���hg�2��T��k��qR�TwW�����.�$��l�)�@�<���"���������i��_������?�=��h����a��z>�F����w5s?����?C
�1
��}�F~�Q��2��U�|2��J�����R��T�~r�	@^5EF|S�"<��F��Z�3�N'd�p�V)d4���f�������H�!�s�3�VS�bWAd[�=��=l��7�1�4���[U:]aB�"���
5p�a�{y�"�"WA���D�n�������R��cM�liFB��qRV(�����l]D�C��Fb����><7�7rEgi?���
�s$x���
��"v��U���)s��97��t���>e��HTT��''Wg�'�n:���<�������V&���%�=��gg����I"'�<f�e�v8�T(jb�V����;���A� ���R�������M�a���0Y�}lb���q`t�����E5#4H�
��j�h�f]���T�����i
s�a}�t��*,�Ji���Lm���v���GTk���60��]K�^��JP�!b���*����j_������V���_�V�^k�\�h�����h���X��������7%����KS7PC���FP��<e���2��u�^�?8eG����� B�Y��=kHSy�r~��D��q��vcJ�Er�j�(�y�-V�p"^�������fq7��N�P�H�8�����i�P-1����@)��
���J�������\d�bV���p,q�5#���on�`��a�#�6Q��1!jn=2������SR�h��8PO:cH����>���'�{��K9������Wg�;��B����u�9A?�6�{dd��H�x)�0f���mm�pK���B -n#��I�K�CI���\4�b^��c`��t���^���nU����29��������rw��mG�:��D����������6��2�W�r������N�gb&���N6N(��=c�b�b��"&��D�$�Z'����q1%��b�mY%E�WBv��*#u�jd8]N��~��F�Z
�J"8}�;,xA���)&�U+�����]�S��������B����.�1��k��}�9�h����N�	�l�K��LO���Je�v�5&��+X=	RV^�����6}���x���C�%�a�T��M�2g��nn=BD��vo��n���������O5���l�'�O����s���`
7��w����f*T�����Q����A~����}�%Z��#(���k����^�k����B��U�c�,���yj���M�E\�*���}_5\?�W@
*1C_�p�<��NI��X|������}\c�|�X�Z������2&'�7)�44��B��a���0� �s�� ��C��(�H�0Z�_�z�6�6��m���+*��fX�Z#4�L*u%w/vR/�n��L5��D&��}��{�����N�����a���������md�wb��I������b�8�i3N��e8�3����8f��.��Q�I��h_
��v���6�{u��K�C�F��-1�*Y`��@����I�<�}a��h�*d:$f,�eE�v	����0v��a��Y��i���96a���C�g��Ev�
k�r8x�S:� ��f3�x�(��nf�=6xL�{Q.%g�5.����|���c?��/��Z�+	�/�,�a����K�y6�@����
���-S�(�gdhu?Ty��'D�b���6���=t0�b�hp�1gJo�+���/(��c��ySr-gQ��dG9��'���jG�8��$�|���S,q��d76�5de��k[1�Z��0���� �6���b?����Y������w	Cn�<�7R<P�������	�"�O�����RG		�X�Q-�����(r|��lJtrb$}V�D��BR�a,{����r�nY7m���)����y~�I\z��"�������<b7����/>W< |�<���/uq&AyM2Z~���9��m�����7b"����Z����$"\�*z�4���m�������5�����m�0������j���iM��s<y�/tL�/�M�L��wW��B���a\������7/*�AG&�+��SbN���0���A%�A�6Gw�q�����CI@K^��SZaC�4���I���D������l����&^�+GkA��)*lH9������?,ps��E���$����
�uT=)���~x�P��+rc�5�8�^o��F�U%��3vu\
)���g����^�J�H�A�����'l*�U�<E%�q�xC��j_Ka����Z���Km�TLB��i�������rwQ($L6��T��Q��V��|�9+�����Y�{(B��Q 6
�����@�w���o�l/�J�PG�0N���p �AB)f�����H1c��<#�p�������.����^��k^�s-ydln~|H�����h�V9��m�Go�ZEO;B�s���Z��3���#Z�<����5��a��a{&V�U�G*�%$,];4�:���]��y�M��Q����U��j'/����og�w�P���1����0 �6�j���
4��HE���I��?#
�oH����=�����N]����a�|
U���W����$R��y3M�r����#���p3���F*m���u&�y
-���4�����9����^u��#�����
b����G�	���1]�E�h���?��S��Q��=�J9��^����S�x��U����'�cL�9e�*,�6��g�Q��6
7(t��ia�:������C���R�Kd���9���2�KB�������+"�Fp�M7][����2�9�s���NFO���������a��C�L"�����/����Y�O������=?%C�"G� ��;��)��	�$��6�Jv�V|��pZ*��Q�0�}���I>��	��L�J��ar����(�������oIA���cl)���#�1q~��N`kO{��_����������<:w7�9j	�>��W��_���1=�(D�5����vd��t~t��q�Yb�:�4� S2����$���h$���������1�J����"E+}��E��]����{����5I��-�'.��or�1��_�
_|/+��
�//�)E�i������������\���K��2���E�%q������=,��K����!L?�������d#��r��p�X�C���@�&���P�X�y4�>#=Yx����9s�=�+~z�}7�5]���E�~t1�c�/,O��dP?V�15,,��(�v$��������s/��N��0���Qq���cV���Y^kF/�+������1�����o�w��zi�L�GP�x���o���u���C��1�"��w�w9?G�:>���{���{a�d�������nZ�����:B���@�?�+z5M��P<����E?zW�v��P�����pgC=;I�/Y��]��V���R
��C't�����od0���A��0
��*����Up1���&O����+t�9y��kZ]vYi?�^�����uG���������/���z�_��������k�Rj����A��]�gi�*a��9+p�U�� O���?[�P���O�9]����s5����������!��1�7t�`�������W{Q�waT�����Td,�O�>?k���(����k
�/#��m�Xgek�Yq���aG�	�S���MT{|�V0����g
�����e����V��o�4Y�\;:��|^�X%Y7�OG�H��^���W��7�
q�-�}���QOVKi?�"�=�3#��K���N����h���J���h<���>�w7���7�G�����*V
k�|�[S�2���nM���^����}W�'�l,U6J���2����%��F��&�rt�~"WC;L����W
����-}�������d�?��-TjS~O\�������f���MJ��
.��@��H�l~����X������Q+�J�|�"��Q�V�zM������������a���{g�3��Fr4��9$�:��u7�^��o���<�� pW#�~-��|���,��v�w8
��wM�����%�y�0����h]��"�-����V�iwtn��@
E����(�;�����O�xu���x�(-A>#�����N ��/U1��s2�"\�M����s��4�T�kG(
p���	����e�"����$��z�z>��[�c��#���8�����t	����..CV���%�E�p�qY�g�1er��������RR6�P�u����	��Y6�dq�R�(5j��B��>���������D.�� wz��^�]�C���!1���|��4���\"J���-bm�"+�b�N����K:���F2T�}VPo.#��Y�P������2:O�A�&���� H���P�Z���&3��
d�	����Z0{�lr���uEa��Y],�l����H��S)��>����o��P��e��x��"8�b�!������h{��r���5���Z�U�������-�J���d�!e�|F���n��)|�V�d�E:0<
a�E�AGo�����AG������7�?�|�������?�x������7_�}�7�:��y�j�,�f�P���!�P'���u����_���\�Bh���S�s�q������N%�>��S��T�3�����3��^�����WG�F%� J�\�!J��~Q�R�����a�����Gd��s-��P��BuZW�g�b�N����{Q�/Gr����� �D7'��������bN����8T��4�C'�Q�������)����/������uV_�������Q��i�In���	�6�9���3�,��J�e�X:�t/�[&�l���"��bAI��S������,�ej�F2��*����	��z����+��t�V;�	)H��>����Q	=(��-+�B��5�����BN��kG��dC8�b��2���v4�J@��('�V��S������i��p�8���V>���{��c�(���}�_��~���Hbn�N����u���7����}��3��Q �������w�1���c���G��/�3����O�-z7�����hm��F�6���?�l��@���0Q���Y.��?O�����5u�vz@���k}��w[��v3��+B��%.{XQ��x����wRX�S�\%W�����W�W��E��W�P��jU�����.�t�(Jb}���<E��m��W�iL�@��u��x)�>] ���T�0,*w�c�p��,��� HU'���3�'��d�Pj��K�	b�.����Ni\R[E�]����B�`�F�����9@6m�S���u�D\9�����~�Y������.m4&�?t�:p�!�	��5�vS���Dc�TS����1r"R�w�_�y��f�)��8YnA��`��l"/.�����?�������X*<<���������L�2����m^|�E��N&�)����*��p������t@�(��j�W��g����3L��vx��!��c0����y^�~��p��+c�DS9ZO1����v{k�L�,���'�P�xV����N��7��^��=��~�(M�SY��r��}n�/a0�M����#c���Bx�6���c������#�/�
�3�*w�;w79D�qtk��������I�{yvu98�d%�g#�j�utI����Q��L�_�h�!�Y���2`����<��=��������K������C�$}�!!n&�TC���M���������y�����"�H@�Z��v�
��d�\N"i�lD4�B<�Y�x�3��QT!����:��rcU4�0�)��F�y����5Na����=�c�����$����,�o�5`I����ST1k����������B����
s��/G�@��ir1�h~weqh���M���/�����J9��^/���C�})��>���������}���Q��(1�nQ��-%�o��m&y���/����$B
��$���Qi���L�v����2�nI����P��{���G�GB�G����0\��m��P�g�U_#_�2G����@�L�����t3�2B��Q��2=��LK�1�Fv��-a��y���+����Z>����[V�	����M�#������n��p����r:o�7,V����v�b5�-l��4��og��������jD�z�1��*>��������[��?QX!��H�#�(���.���%L�R���|MF#�;� BGae��(/�P�,P|�
��(TLlX�L�X�)�QS=E�
$�
h�	���^t��g���PKxq>6_�PKQU~JGpatches/0008-Adjust-join-related-to-code-to-accept-child-relation.patchUX���X���X��Zis�X���;_ N,y��$TC
��������)�,=�"�Z&�3�}���i�C'��m����z�"�����x6�T��\���?=��w��3�]�����?���z�&�Ie4��px%��x8Y�@�����L�9�p3������3��?��Ty����g��������������^����hr5:�:=�������T��(���x���5
/����s�KU��%
�UJ>�)y�����<���������4)�z�-,�0�Y�)��T�������e�3%�����yXX��csR�!V|u�0�

�oJ�{s7�?s���#�~I���w5��a��}��4�B����Z(�@��8H�S��b��25W

�!��Tq<S9��!	�
���)���X7���*+{Zr
��c���,�4�~us
���
������r�B�r\��H�8���rN���|�^�iD0FQ���P�U��)b��*J%k�v��8,�9
}���a�E\`y;�V��&J�����B�9M4e�~i��P�;��/�_~����U���`��M^7w�����F��w/�����H�!K�
��$U����Q��>�R�;����B�a�g5f]Ba��"�J�,�R�O��������/-[�k�0Y��A	�������*i���0�jK�0����r�E�[k���5K�{a���K[Uh�n�� �Mdz���B�V�qQ����j��q��>s�P�+0��E��Sm
������~~a�P8Um�A����������q��x>�{etG����V������
�i�vh[����YDx���EE�Uxsd�4�&P�FE��X��Y����o��<Kq6P����7�b�+�A�
C�\�Tp�Z�q�*
��T����*r�����,O�"|��Y�
�(+�+�_/�A7K�������������^PZ��\HkAn?I@�z�Jl���q�V�>^+�h5��{T��Y��h��t�sV?\_����a1�5=��*���Qm�1B�u��-,6�*�9�L�}��[8cZH����d�+��OlN
q ��na���(,�wVX`cA�u��!���R����s�y��[�!���W���f"fB�_w��N������Y�I���z����v�P���.�a#q�Z;"�@���g�sR����8�������hz��_�?Kt*A?u�L%���z����{����.��s�#5�v?c���i�qv���e�� J�
������QtB'�	�e���	��$]��i��q������E������I�$K^���j�vE��G�K%�3S�mEc�ez}+��)��
)i�� ��%���/���	_�o� ��<M����pEG3��`~������w%��8T�������Z�+%��5]��8��r��W����bI���6����ac=e����R�l�I^�%Rk�QE?g%�����,�T�-�vqa}o�%��;��Q���iqp`��2�����qq�$uy�5b^7T�0V�M� ��\<5d�����u5	2,�ZO6��k_[Z���U��q�a� ���LroG�DR��ee�nB.�j�C����U�5]]x)j��*�gO8�
��m����\�6H3tl�@[6������������W�����������	��_�FZ�!2��a�O�CN��*���K4�����P���
��;�O��~�,hW}��{��#O9��L����i4����"�=���{��Od_NN�#��?��%��N9�����M0�F,���o����,����e�����!O>����G�w
���+T�u}'��7Tt������k v8DB���@���o�u����4�����*��"3��ss��\��c�+o��[�
�����]Fe$���:���\]an$)ud?��Jq���a�~X�����I!�c��k(j�.�e`�v���ii���U��~��g���'�\O\qr���'u�'�Cf�<��
�2l���_\�dP0"��C�)����[���Id�;�B�7�3��f�I+�.�
�s>L=��5� ��T�>}=��X�S�����Y����F�1�pW�QWS)*5D���<�����D��8^=����l� �<�:��T���#�����np#hf�5)����.	�����0�*���
?Z^���
�����t�A��c�����[���\kN�M����CAY $���G=�Hvy��O�@�����:�{���v"��X��~�[����2�����@�� �:�W����YvI��u�������e(���:��%�A�e�rO|�n��,C[%@���TF��Y|J���x�������?L*��N2v'�\��!��R������ ���.����iDG�1���C0�A�~���B0�A���n��;��.��g����}�G[Q�&��f��}�����;����O]��*�rm��+�j���e;�������^��Z���p]]i*�K���=�	&������`^���d^[��6>��'t��m%�H�c#��s� R��T��q���C���w��:�l���n���2�2R�k�?R^�o0�g��K�����s��nu���h�������n	8}?�Nyp��'�:@ziT������	Az��WZ�7yv�Fo�o��e�p�����%�{t>��Gz���D��I����BvDF(�kj���x9����B�{�a�����o������Y�>0����t��t�������/o�ha<��(h*�d��q����>!t���k)�f6�����N��<����Tg�G\���``����4[�f�f��H&X������+w�����-o�'w�}�%�_��L���J��O7R~�@���[������X=2��\^4�7����Tv����h����������H�.!���r��x=8��2c���*ou�.E�(KWmw7C���b������'����Vv���#Zf�`�;.��\�H=�C�o����'��<sB���UO�i�'�4��w�f�,�P[����1����eC�>�,���,KK���������,gg������.NM�Wr����d���H���&�nr�1W��;��Z_?9�w3t���o��0Y����}K�(;c�G����6����b�^c����g�&�_O��.��"��+�dv�/c$���c|�4�������Mg�#i�x����5�1�S�i��0��J��~*P��K/�?��i	���.��c��F��y�&��O~�D<��#���:���S^��FGw�w��D"���:���Q�l�fW����]��D����������SC�<�h�5���K;��2s��r��������^��-��3JR��*���[9�u]:�0��/�B>����
�G���]���$���Hzi�n�~�������DNB���t�ZZ���.��u)�P����-]
��P+�u���l�F���t�,��PKw�*�
�*PKQU~J+patches/0009-Parameterized-path-fixes.patchUX���X���X��<ks�8���_����H�����d\��$�����=����R�$$1�H� �����~$���<�j�n]�-@��/t7|�&KqL�������'��zpzrr|t�I�?�u&e�w(^'��V+�"���?1����>0���^�Y��'���x"�����Tq��Ujxc?Y�w��z�P�F��)���{���N�`��`�{�{���=�yyq����;��?�oq)S�T2��`!f�oJ���wJ���L���9�R�s%�T,'�>	c���B�"[(��JJ������S�$�B���
}5�U��c
��@�0�%��~f���Ao�%��%��2�ND�������<�'���=X���������O�1�z��� �m��0�wu�/�bf2�`Ea�l���[PL�������z���t[����x������|�
�c�W��$�'���00��
��Y����dS�D�
����C<!���<wJ�2���"��W�$�.�;�H�l�h��.��6Mci����}�d�s��W7��.^���}���C����^Dj.�����;Il����������A*���F�0��F�����H�Y�2�]`7�dZ@[�b��	�"D
�������)�1��F��\�6�9G{�D���81dx���J�F�0/$@S��u�K���u4t�t��y��`�U0�\,M��fU�	`��������D�"*��?��F�"(aL��2���=��`#o�dK2Z|��Tcqi0�i��dy�x��$@��r�>�)��B����8	��;��`9�]����2�t�t��E����=tv���8�MVY��9�]\�.R�����S0����D4���1��4f��c?��DS�<F��*/�H�cQ����]�[�b�r�?�����P]8�'���)�h2�+����3��V)����?�����~�Vu&F�y2�����f`m�~������c	{�d&a[�;><D&?�$}h�������I'��]��$w�s��5����:C�A����"����������	|��+}a��#�}d�G���="�����'`N#u�u�z��XW��J����OvyLw`@0F�'�4���(�gO^�'��\U0�C=���)���%�{0O/A��N�$�P�XG��<���+�]e���"Pnd�#��o�����g�����m%�6�[@�%��6#���>J�[�<��]�*��(����@���9�Q�jK�@���������4f}6���r{���7�R4!�Z��0.�7l�X;<A��j�)��$�`��~��������
��b]B��>E�����V0�5Dd�,�T�V`�0S�e�F��t���e�	U���^��3�x�mW������	�|���-T���b��3��v����^p�y��?;����������%�
��~�������"I>�J�u��`�Q��]�ei�@�L<��,�"D`D�/vva�P=�v����M���Z�&~o~}�
�bw��@��Ch��=�v��_��.#�#.I6�@�{�p�Y��|7�����A;Z����� �������f��+#��VxK=M�Er������M��
�4��`�r�C��s0GS^
��0���:|�L��g���h%>%�C1��0u�#��Q��:�a�}A�_��O)t���b�B�Z�X������u���!'��
I9������L���S��qztx���0�_a�����K�z%!<���@�;.2��6���������� w.q-;��3p���P���,'��$������6��c�A��=%��F�]�r�}w�]��q��q�DS�![PO�h������A��;(�B�E�4$���R�����]E6�3�~;��&C�U1_&�m9"���X��7�+!:{ ��Hb��13��W'��^)M��vg��4z#��/j���������������\��!��O
� �V\���&����\�JFQ�oC��S�8�|5�2��<
��2�V�	�n�y���Q��M���S�/�R�&�����
��t-��;y?�Rzg	������i����z�Z���::�����`x0�Se,`a���g���>����<K2�����p?�{=,��)q�����&�N\�uHB�4xj�� �M�(�2
����
)�VOa���T�\t��I)��8��{���,���<�=��,��[��d
e�$V�����	=e#�f��~C����-Z��<��8Ta��}�r�q��j.��j�4�C��Q*�A�P�G�
^�����e���m&�a�n��3��8��Z�U�P��j3i�B"�Ro�#M�3
�9��I���:�hi�~���e�b�K�sV�r����-�|�&iV��	pS���@������j��m�Ua\����iY�iU��nB���Mr:��� ���-�q��.UY��h�;������g=8 !9n}�Hf����X:Zl�n��3s���Z?��gG���Xy�dr�o��!m
��})9uJ����	Et]�������e����������y�[�V7MX\b�G2�J�o
Y�X�qd�L����
��=Z�V��6�kM3��,U���z��2)�w#}%�
���Z����ZE3K\`
o��{�����{GCL���En�����2Pa�����26J�d�C����6$�Z�a�`�4���3���E���s���A��3e�af�l��<#X�O�&.��
\�<���l��$	��&T���T���Z����N �8 ��HE����"�Y.�������g<
���H'��qA����1qa���'B�8���e~�
�0L�{|d3�Zw�q�y@@��m��v��T6���������K�o^���h�����DP�'l�>l�K�A�������z	\�����a�05�'��#��O;���B������`Wak���
���W-��EW��"z�]�P��y����U;1��3���s�T�r:P��)D�&�����������k�X�`�:�E����x�BHNm�F�{qz��&�wO*�9�/�����AG�1$�V�{���k66pF�]�&��#�����Dt.�[�H-H}�)m$�F�b �osi�W%�k�y�L��}U�L�A��U�*��W�g�o�C�����!5��s�2���Q<A7S����M6>-����
����0�����������	K�i����l��%��3��z|J������x�B������u�����m�|�e���tZq!��bbY��ZX:w�og�Oa���J��6�o��o�jWk��>�s�Zq}[�>
4����S9y\G�bQ�"�����d	�6��*�����w�NR��B�������M��	��������n��G����L[#���h���Tg0��#����Bs�9|$Y6Xlt:L0k���Rs����m���<��a97
���'M��|�O�_0�����l����������*���Tp��=�|U�yk0�
��{n7g�@�XF�2�)V��p" A�j���cq�0�TSi#�9�!�elv[�+s��,`������SfP���%p�h�C�(�"�Lg+�������"R�O�3<c���	�2�dU�����&!�@m�;����P�`�����CDsdT�@��|��������*I3,�����)&o��-������m���+�j�m!_e�_E��Z�kK�����K��rU�U��!���n��1��l0�U��!v�~%Cl��DC��J����X
���E�Z|��]c�y]�KR��W��JC�n���Bo��
�z����7Jg
���~����N�D�=/�����<����8���WA�i{��@gi�	&A>'Fs��5`�$R{�������-d�}��,7�����bTM��r��X:3�K��9�li]X7�Z}I���j�/����B����o��>��u���+���/�����#�xm�����j
�/��m2O�EoH~���SO9u:��,�[��.q��V�����IZ�&c���3��\�#�X�3���_�TY�kXV���l����ITa�������ZV8T���M��|�����[q
o%�2jh���I~�3������gPM�*��[����������O���N��"�K�r�E+�*��?�fHM�f���d����[p�P�)���&��]���,Z�����������hjm@��j<_6iS�z�@_G���am��H���u��
��x_�$SY�g��0�+�<� k	��"qrG���/���4���hQ�e���.�|��)���8T]�w�
�����U\Q�p�EM1R���JO�VW-��U�);�Ts�
��TG�5q�L�vf�����8����bf�!2e���&�&UU��{����^
��Y?��������-x4���/�g�V��Wj%�d�
#��cb�����c�u8G��c0��
�����jmu�; �JL
H����� �j�t46��l,�0``�*�g���f4A.S��xpo,./_r���zi���=W�������)���5�.�����*�jE���-��1������d�!I���Nk�r�u�(/��6������x���ooM�5�_�#����.�)��4E!�]�e������-��T}����u�H�,b�$C���G���;,���tI�8���)�Ioa6��#��{?RC�w6�������H!0��Y���8�!�8X�xk�zH�����\b�)�)a�%�Luo�TD>�����Zj"�(�P�p��t�YP�A���f��C$k�u��]|�-���R��5��_3(���fpjv�9|�F���dQw������%1��;����{c��Q���
�k���Iz���4,���i�Y����R��U��xw2�jZ���K=��^S����;��S�������{
��2�'�T��H�� �}<������/Y�7�������B1k�j���`*HtS��c��L��oh��_�@-��b��)�c |�.���� 
���bZ�U�g���0����������E�:�����f��-�/��H��@%w��I���#������G�tn�>l.D��t��9e�6�F�}�����\�8�����d<��Nf��yb-�����.t�����=������k�����s�}��U��$�y�����p�Z��m�M��\����m���s�x��8��{'���E�%���u	D;Zu	i{YG����p�e\�p	�<�3� �/���I�^u3|2��.E���m����-��1rub���0����|/���~�~��g������xk[�htB����
1��z�5���/`���F�6���9�`O�.���8����5�{��1{�������}�to"��F�D��tB�9#���mS������������5lk)���6s�J���L������-�STgZ��k���Nc�Vk�r���O��]��wz2;Q��w��'���o��yk7~��>
�Coj�
9 �2�8bj�"|��ZU�*@��.g������1Hr�2NW�$�������o6��4�T6>�l.�����������PK����98SPKQU~J,patches/0010-Partition-wise-join-tests.patchUX���X���X��=ks�6���_�������,��_����x��:�������-%A�6�cDj���x��A$%�1S$��������b�F=��Z����19�v-rL�5�Z�����7��{�~]��-Y"�u�����]����9G���_x�g���['x����d���r5��x�-f�5~t|r�����N�;2��(h�:����]t������z�o2���?��{�3��G��/������t1?|����9���{�F��a����%h� �/|4Z�}'L�&����!�	�������.�D\2n,C�d�|g����������L���7�6`E�p�{���Z���b���_o0?v���
^��tx��tD�X�V�����X��B��9P��>�r&����D}���jt��u�",��(.l��U��2Z�~�	_<
�q]����))����������Q�rZ`x���G-��FA}�9{xh��������3�e�b���A��Cc�
��y���^Jn4[�	e��q�g�F=p#��t2A���S9G�34���������������G#�d����q�����{txb��)����mD_^\��&�b�I���9������&�=�����.�^��u5-�Z<R�5�����.���u����M���a�Ku�C>��W���"s6
b�`qy���������Dw?\]��r�[��P���h(n#��YQ�_��o7w����F?��\\�t�tZ �����z���Q��������C���>��n/�.���i���/�2��Q��F����I�B�n>������ho���xM��������q��������&��ps������~��}���I��.��zy+�hv[�V��n5������
r�����>�����&�^sgF��O��hMi�"��#e���
�����jUkU���W���� >����2� ���������$O?�Vd_a�y��A�2�bU"�N�f$\��0�����.bS�)��Mq56� ���fS�����I����,c��tT��)e��aEH %6.������k�|�����6�]�
�����q�>�������I�}M����W���O1�d�������hF[$�_Y"������|ys�CB���t���i>���6n��	���?������.��!n�m��b��&��/��� ���8�������	����+2��Z���g�vC���D�s�l.�Cw:��s������I u
�n�U\T\X,��<a��G�&c;\�������9��\�`��K_Y}� ��76}��_���^�=����)�H/�\2�_�������4roM�G	�A������:A�?�)p�!}Ke��h������x�q�2�)��y�<�s�����>�����|!}P���u����C�|(u����9�q��O}A���?���8Nt}I"�������.9����L���G���$	S��M.��E����
t�4Y���:?�Y������d'(��j*LorSL]�c
Z\_�X��
�.��jP��+������Z�����`�/�<������Z��cO!��Lu|~���[BW���a�aQB}]Q����*������.���aV!�0����^P�$X�����
'E��$q&E����.�R���n.�M�5��z���
�����GV������gk��Ah9�`0=3[']n���1��L�q�cJ�W\�����|E����5�l����%��f����Y����������_��x�j��l�Y�]7�l������Xu�#���Q���<
����N��t������h�=�J���|:b�N�����n�KU�6w���{�`����n������j�C����Z�������z:z@��hL&�����C���2�;EI� _GE%kA�:�������p���H&	[u���](k�Pl(L��p�����u�HKJ�9`�OG��gu���?^�Ml
������*���tX��N��E�P
��4�*K���Z(��P��*�0a8Y����'Ew���Mw[Aw���0�{��*���tX��N��E��3��$�;S�HA�U8�3xr�	 �T
�o����QWm������x���T-������<��)�V&!sK>r+l����`RWa	 ��\����K�#��$��b�H��u:d
k���Xd��d��b^��v`@]�5�&����R
nb��o�[;����D���U��+y���-�?�-~`f|�L���"���bB9L�<I4f4��EE4F�����������}Y#�~���q��n_�����~7px��@�.���V9�b����<,�V���Ho��'��[9�������/��%�MK��sq�#����~��=�Yb�G��������]?50d���S��!.i�H����%T������d��P���6���%w�����}���20����j��>j�����vY���������9���`���[��-���<}����4O�yc�v�Z����'��
��Z����zu�5%����U�5������j���5��w��Rh�" ��%��l�\(+�U�����t����zp=���W�=�JL�������c?zTe����{F���
]{5�>5)��67�Z��������+����0r!,��-�����~\��g��Z�'�3f���~Z�?���[
e0���p���TM1�|��.��3��F��(a?�>�>/0a�O�K���?���������D���}}�oI�"����01�
0��B�1l��
0���~��;�'�����}b�'%���~4M'��f��=y�f$q�["�gpf��,�6��'������,���F\%&���yO\�#���2��
'�Y87*��3��+�YDn�n���)P���Y,]��"2����g7Ex��1�&g2VgI���*��*��s�S~����A�������2U� R����
�l&P���"����5!�`�&e���A��_�,���J�~%%b:fS���T��"/Ul�S���D���5���<�=4a!]�Z�t~�V����x^~������8gU����6<��&V�G#w��:��`W{��$oq"���A����+�Z)#��p�5�[�������%f��jw����	m��rwM<���W��]1�K����4���6R��]^O
��]�����Mg$*`����$M��H]H�����-�O5
N�5�B=Q�c*���]���n}��yV�VT��:��$���]����KTuP.���o��S��\����#m��<��2Z��..��}S�&q�4>�����E��^7f's�
�0k&����m���N-o>-�0����=�?����B�0);c����d6=��n���'&)�`�8��P#����]q6B��6�:�M����cY�������,`��q��!Fc��k�)��,��CW�o
z��y�[@/��d(�x�o����������P��f��PUS���<�����,"N
?����i|��7H
D�.n�]��6}p���M?����E�p������<��R������uhI��x�����UU�4f��{=���7���5]�����$K��"����v�'�EYe	��>/q�gp�i�:�����.���=���\�1	���*��n�WV��+lEV�R��8$#��U�11�(�LF[wc����kY����~As\Z���j����/�(r� 1$z��h�(\�O+���Xw4�T���=��ur+�c����a��niQ*z�&PRTr�t�LOY���=~(�Q��!C>��
)���?O�o��b�=@���oD��k����H���%��$)�;	;f���.�.o�^6E��-�:�#rj��Fs��|�/�L��y��s��(n������>*�@��=;oZR�fb���R�j��&��q�����"�+������9�b�\Rb�H~�B�!�A���`��H{�@@�#���O�r�,l*S���u�[p�Z�����%`�[RQ U��*���l,�.�s�G#6{�P~�Fx*N��:��t.[���; `�Zc�R�qT��a�����a�!����U��:��x�Aq9m�xc�Z�N��'N�\\8�t���B��o������hu��Vd�#G:t4%��#6
�AM	������1�$�n�fQ� ������@I�1'I��m-!��\���-�-5��$�z\��_&*��]&/�r�����D�b����@	:�:�8��0��H`���x���B�u��B���f�A�}k�j=?.���J��4��%�X�,����
0����+T��a�Z�3����X���7a�e�������"JE��c��hZJ^5�IV�2�f9�Z�W�q$�"�<)".����NQ*�h�E�]���W�:��i�1�4m� x�4�+D�/���O��!�%��B�,���la�@P�����B]����y5�s���<�
+;�����R$N���,U�H����������vJ�j�����BB_������d�0 z�W�E[�T6D?�����,�T��a01sv��B�,���P��b��9&a�kbCAS\�g��hW(��Xx�2���C_�T6�$�}M�a([�Ki0�.����R�b�;���].�)V���>J��5����>7��x�[�����T�xx;����f���4�l&��V�����>7�������"�Q����)�@�Ne�)W?����$lSw��	8���&�YB��j���Y&�z����������=B�3��%�k ���1y�F��'���a���'{)ki\r)���k��WmdO�yy�F�*6���yy/������;u$��W<�:p�w7b�����a#������V��H����	�����V�����(._<���?O����%�����TD��:������������+|�s���60�����������0
W������v�4(
7u���_�(!��F�7��R�Wx�xK�W(h�y������(n���y�B\�i��4z���e��M�z�����v��m��
MGi^�A^����(n���y�E\��ai�l�����6��Up���j�{/�?���M��a�3a/�I�	�=����ArS'Jm#��0�b�M(q�j����B���	
C��B_�a��;���dL\;�l)&E3
3��������w|U���V��2�U4�M&M��	��rs�j�T��r?�_��D�Ki&�J��l��[,���r2�>�(�������+?8�t�;]�byf���z�:�W
\����^��@��x�8��9/�7��awM{x��N��X3����^ ��U/�C���`^�����=\_�M�?O�u�����[F�-��7[��%H��mD(TG�N^�U����~�;��K���In�x��z��J��0_��<
oL�+�N���NBv:��N"X=�>���d�
�^���D�!��b�~�P�!C�Va(
9�/"Z�m2Ti-_�U���m&��$�'�����fduOx�	�a��9s,<P�coY��d�Z�8�9]��}|�����W��G��+�LB�S"4��/�+nV��G(`&/���e��5+��������jy$a*D��=UjTh���H+�W
+�ldY+����C\#&UZ�����d�j��l�U{	V���XU-~+�V$�b�dt3��6VU�XUL�v���X��`U@0��U���mERv)V�A�7���ro<;�Y�Y;m�|��/3��M���-�uo%#V�&#PN�q�3����\M�����E1n�<�hm8�luW	�D�5����rL4&SF�e�8�h��
���g�������*2Kqh����TV�ro2�G*���O����d���)F�r��*|q�~�K�:g&P)W_e�U!m�c�7
n������F]q=Q��>S�������l��4��_����;��;������X�y�z���VP3���t�$8�SS33;h�i���jf�I53��vXSx�H)O���'�#8��#m�<�f�]@���G�����YM��Hg����SI�8#���dI�����!�����j�W't����2��/��f��`3+�������Kr���'>{?eKd���49W�}���T���6�Ee�}�?����=(96'J�J�������*��M9�A���gt��Y)B�5��yoZ>�)5]��)�,�q
�XB:V"KH����K��VB��65#�N!���n+�nKH���FHW�J7��|�
���/vJ��8"@1�U0���V��V�`�V��
����L�k�F�7�v�'����zK�E;��/��Y��u�9��9�yh�X!w�����3���@��K�1#0��?%-_���k��r�+d��M����_qQ�
��f���[T����ed\+���8������d���Ip?
�k��H�,'1���C�
�	�!�����@�f��@�F�5,��P���!�
O�u}��\u]5�5�|���`up�u���g�������J��]8U	�����D��C_���_�����rD|4s���C���G������{>����'s��	cnB*I�����:nx���#���0 ��� �A<%��G����-���B��E[�5Uk7��*]>�E:�j����CFFJ�������"q��d�D6�s��Zi.�5@���b��s>DVX-s
N}��bh���wpN��������5N��2��&��X�2���.r�jb�crJ������@���m3�<p�L�}������ \�o��PY/D������u��H�%m���ujm:L�j�{�n�b�+^��)�^�]����I|m)a[���f�W�����Q�����f�����x�l%l;�1�l�p�1�q�S���7��
�p�k'}%�~|c�������x���������7�p�m��(a��7��G���)��T	�4�1�lA���5:S�>�o�������|buU������l��}�O,e�����`[���|o+z| +B��o����q_�M��r��D�������k��;Qt�j�9�V�'��������|	�����Mg$2�O�#r��������mi��T���.b@�Cv�>a�r�������"�,i���'p��^G���at�����A�� �J�Z��@�6�t!.%�[��7�a�x}m��{NN���P8�D�Ix�NIQy��\���v4�g����_���b���1>�9�9�b�����E��>e=�;���pj�X�!��s�%�/U(������K*�� f���.�0�2l�v>zZh�a�+`[)t������N�
�v��gOAB$aO���C4��L�4���6����)B�����V�B ���,@�^��MA�'G����q���H�M�����9��;%c4�c���r���C��dBV�������)�[�3]���e�1pZ�m�Tz�nq��]T���0�����(��B�sv�A���J�%�$��mE_��"�"<?Q�p\��y�GB�	���|�*�#��V�V��R�D/=WXl�����
�kP��������L��*��V*@��PUi�]����C�w^
�r��<��:��~�q����g����c�:�`����
:�&C�<�����7���:�*}�����#�����+�.r��}E�����P@�R��=���X�����b5&+y�7�m�����`�����a��)h�q����N�-�%��!�����E��"�N��q�c^T���bX/q������3�9���������@/�'4��ks�L�xY��d5YU�D_���]@�8���Y^
H��
�����/�LVQ\$-wYO�C������#Y�H���8���6+����O�t��FGl��hE����#�?��w���x�4�H��r��lr�v�N���t'�_�n���kP
j��888�+��������'�@����-�Cw:��-�[��j�d
�_(�P���v���s�������n���G\2�����C���w\�����dz?p��]{tAc�(K����~=B��?s���G%��p1����#l
������`������{>^<�?f.��&�����q�������X}������m.�3�����(������_8t�aR����xtdxa
����B�����s�0[v����wD[���D��0eM���7�����=�� �G�������T��o�.>��:��5qH���o��(zV:�W'�m��+z�����5��7�`��|������=Y�/=��#K��=�#9'Ts0ecN�dJ;[�I�#���q���u{�~����hL>�����\CT���hj�3L�-�G�=6g�Ww�~���1\G�i8��������~�]s��]�y����0�H�V�m`�yC�����f��T1����QL#�s�a@#�r,P��)�A�����8�S@GZ�K;#��bT���������*��O9R�����.�I2Nf�W�@�\ L)���)��������n��P*�X"!UJ'�z����"]��"(�iUk������m�4�����-�~]h(���\��Qg�-+����g�a����X��	�W�@��A��4
����E��,����/'�[�),&���D>.V�A<:�1�����>S���hz�=f��N��a	T��_0���@��@P�������E��O��h;����/���X�R�����_T����Hs��0��	8�3\dq���������/�G�r�f�M��xh�Z�������H9��xPQl��x�'��*�e3�/x�R;���DP��96&I�X�7�z����!u	o��>�����D���X�Cd�yTOU�P���l�KG\��5*X�L�0�]WT����j|��7(�+�~uqK�N[��g6}p���|�h�Dm��L�I t����R"f>���l�yZRq�}����\�6L73k9Z8.�F�)����n���5�Se#�;������M/w�[������0s
HE�C0��V��0F�N�1���!�Nc(?H#<��sa�\v:�-��v���l�?Q�}���6���������\��������u�BNb]$}���7y5}h�����d0F����7��];j����e����S��:�TJ�DRQ�����������E�b�c�s�)�R�����arni:Q��\+�>�&@Rd�xj�����&DW	������R<Mc�����@Hr��$��y��fk����V��������<f��������d�Z����Sp��Ks��!��]����J���x����*��|�������������������?�K���"��b��P���W��Gz~�H��X�;�y��D{��{��x��(�����c�~A8_R�����E�R�%7��I���o4�PK	�[�]%�tPKQU~J:patches/0011-Multi-level-partitioned-table-expansion.patchUX���X���X��<is�8���_����d�a��;�qg�;��vWjj�K���E�I*��'�}���%�����j\�-�������7Q��t"�f�g'��hv|������`����t�����	�vN��07j-�'b0xF��h06��g�"^n�0^��r-�����3��$*ZG^��Y�	W/��e���7���#�NF
`����`,:��x����~WN�L����������p8�M�����������/��@�"�3_	�u-�����Wa'��I�,U�%2p�Xz*�����\�
I("5�ah�,U3m��$����w�8�����0��"���Z�����y�y���������M����}!^nq���"P��Q�H9�(�`���>���0�@\�Kb!g���vE�N
�"|�{$��/X�8���ps`�3�4�����z����������D�����X��{!#�����N`��|F���1
a 	\�}�}���43����#�,
^����J=E�q���Z��X��X��N�]q������q�41 C�H}��M�#�.	�8P��6�$K/R��{��SL�$l&J�pb
��/�D��:#�EE���k��
�u�����$�H_�z@9XG���ms�c<97��4i�L�f���k/bJ�VQ�M������.�����X��`
�6�KN$����Q>xalTto�w$���_7oe��X+K����.�{����4�"�J���d��$\xF����1�<`[�[��;$f��!
���6����������zMG��L:�A��u����V�!���w����	!�G����^
��ct6���^�S/Rk��	p%<��3ap�D��a���!h�W�j��CPF�o4�~]�?��5|�C�=�c��-d�P.���P/V��V����@�@e~�k7As����[���������U_���GG#����:����GGH���lR��/������?�3O�H��O�����h}dV@�&P����'�od
O�@:�sR���Z�j
��2���?���7E��hxs������_���������&��"�<��o?|���������f���6�0��Q��~?<���@\�I
�O!�!�$�'q}{)@�J�SI���Eu�B`4��pRk�bD��NO5D^G�UA�h��2����il�*����2�*5]w^��fd����yC�7��}F�U�$,6��4���`�Z��>�;M�8�!�o��h"�����0��
�;Dp�p�d���p=`#���d�����Ve@r�h�<�s�}~h�;���O�Ry;K�R��6�����1l�����
�T�VAL�i�?>;>;�����\9���+�����5E�N�C���wt���������j� �q��
��4	�N���t%���ob��5�s
�O]�HDj��qq��8�93^����&��l"�s��	��"L5�p��&'` ���H,��hT;���������73��g��g�������`����B�#[z3�c�����>��Sk�w�Z�3�1���Z� �^��'/5���
���o�]g@3:��������u�B������B�������=�D���Sh_B�:,`l`� t.���O�[�T�[���rG'f���d>�g
�PK�Hqn)��N\�0�s�7k�%]����g��gI��5NY�r������h�MP��g ��w�z�e�!B����
�{��)�$�E��0�����oh�b
�V-Z�?7*��-��S�����R��J�� ��yBCF����cv$���t��|�E������m� �U����)���0
X,�5����ydUP�A�����0"R�Q2��+hS���bgM�H�.3uK�j4r#�s�(�����a�)���
�J���Z?k�t�&�(���0��kHJ��|�s&9�Och~�j-��|�{>q����z�Vl������9������YN�=>�������5��)u����'�'�Z1���
'���(�p�F�=�t��o���@����)�!�����V�i8��!�7�?�k��gL�f���*���&�@iF
k�����*��������=F&��0�dr���juO)�s�B�*�F(���=�9�A5�q:H��}c*���"���X��Hd��	2W+E"�J�'�������N�����Hf�,�N��d�l���{#�/x����x;i<��<��U"Y�xs3w�������<�s���E#^����z��������Yv���@(����E�����&U�0?.�e�jy��
`������v�_bP�H��� 3\�V�4jG���Z���'p�W@��PC�8#�l�|�[I���`��m�R.Nk����������'Qe��"��0��h���k���]��7�A�/n/o�zq})��.�)x`������b_��%�Z�J�V���T��T�&FJ:`��\[e%������_&yA6���\���_�4�(�a��x�6�#j������&���j���|{	�8��xr�uIod:�+ZV��Q�+�|EX�g�,z�N����BTQ�Z7*����K�j����M1��lTA*�[������U�I�4�0%���7�mh����T�����Z9���.��m�^��|�!� ��f4u�W������<�f����`S��Aup�����-G��
C�)��_z	��a�����
��|bq9���*m��c�F�
�m��������\a���l	1��r�d���~E|Z�� -������U
RU��08+at:��gJ��3D��0KL�f��4*rf�����1�*++Ll�z�zz������B�����G�	&q���Q���&I�mJ&��������j���(GY�������������W�W�_���^�|{���o�ss#�;�E��b����H�l�/h"AQ9�wh:qo=nJ6w�SR[.D mt����o���w��C$�C�O�\�V���2�����]��������I3�������<��8V�N�y1�CL-y���������	3��C1���1n-��}��V���=��:&�b���]�(�
�6��7�>CA����5���
,����_��-`:5`l�l�.�����}����o$)
[�r����/�����u6`z�������R�JQ4e3��CODJ�^���B-����g�S����	h�6��H�4������ K��erAy�e�&R�E<����a�%I�kF��?�P!z0�U���
FD��`��{���������w)���Pp�26���@Kj8��6���\�'�c��R�p�����
y��f@j4o��Y��P;�'������~�6�G��Y.���A`j)^�b�O����\W�����a��j��?���4���MA��<Qf���3)�S���m]ulU�d_���r��^�%��,�����
Wu,�zk#��Sg�/-��J^OY�r�#��TI��������e�g��C���AW���L9���#5�m���c�F���2�,�*���Cn�<�#�����������Ml6��.����MvuTa�qWL]��y�b����`u1z�^vK3Z���eZ���]�&��U��n�l7�f�A~~'�
zU�
�x�?CU��;!�o�w��,���#4+��9E#����qQ	��1?t�:�C����V��>LKs��@&�����L�����+>bU�_C�e���D�*c`��'��v�G�Qw8|���
���}k�^';G���O���B�U����#$Q������%�RE�� �����L�TGC�f>)G�y�Y�����xT���ro��:K%�*��b�^�x�+�w�X'�u���]�M����8!�����C)7SN�R����#}r
i�1����*���0��~��B��X��,�y����m�&�R��������5�RIu:�0L�R�s�����K���/L����m�6,��`����hU�L��M���9�N��j���y��?eN�����,�s	�`�����4�����r���{����v[`i���;���Pu����l���M4Bjj��]D���
�z��0]�q.�e���	��KUcz��g�=�6C��U5��^�4(>CF���������5��e���Ur���]7u�3����3�v�#���q[��\>�����}d-~�f_b�nG_�;�.V�bJ�a64�e�Z�,��y����\�������f�/7(�4�h�� ���c�,��a����4���C�o�Dl+����U�����=q��+�����X}�g�9�^d���[ar$�V���G1ic�J��0���3���R����X�([��W�����������������[���,f��fK���:����^4���o6J�3�,%C*&�GG.8d\��O=�b�i�/�$�
4�����m�m����h��n��`a�f'������dp��~����u�m3$�;����?E�iH��(C�I:Y�F]O��(�7W�8��0$�l&�������������J�z�W���-���7h���K�z�����IJ�{=��=J�zv�^���y�="�������~������{��v�v��U���:����L}Gi_q��%z=�8�6��Uz��2��U��r�y��<�\�Q�����y�O=JE^��*�z�y�\�*�z���^�~�R����#U���W��w�E��KTU�U��*����J�|�����������cQ�x�w��b��U�Y���z���N�ksx������Z�n���BC}N�u�NN��~�������Z���j����9;��o�
_u ��:�u;� 8�������G��L��
���v7���7	w/���<�+�k�pNa`J�5��iXSYQ�$+h��d�0u�}����e����~��e�<��B��
�O����@a�����X,�I��4y�����(�����^s�oD�;�aD���:� }u�O����\�g�
D�'���:|�q�#Mt�S��2��
!2�"�R4���:
����������A_���3��l��vMp���a�-��I0���i� }]��c����DE�i�T��i��YnUV.}��:>L�,��gE��"�GP�zL������Z1a�@Q��@L_�A�wZS��[w�t}e�4I�C|Fc����Q��M��dD[x���
}����/.�K���K���s/m`O�knG���I���{�0��+bp<�����]���G���[k�XN��de��3�|�.�j�6�q���n��FX�*GK)����U�L�����O&z��1�A�J�����[���Y�H���3���
��a�?�ja^-�P���~H�'����������[���<g^�����Bp�I�� �#�\������������(4?�o<�7�l��A3��>(r�U���5�%�}a&�����N����������q&bF2���:�iH�U����+Z����w��#}���.���"E+������f	����,����U�����n���<@�9l�H���Z���)�*��� ��0�[)-��L';������*�etOw�)�S�Zh�Tg��.
��v��tU��$D��
U2�����s���>�������$9�:{�N�@��
��&3L�YL�]���O�<�`����&�RE���f7�~���h�yk
(R�7��~r�\� �q{�a,�PA��[�Xr�g$�G���_�����W�����;h��`j�8��j)ac�V�~gjD�L9.�z
hN7b����^|V�z}����35Y��!b�:�T���'%R$|�2V���'�4���+��a��4���.x�*HsG��)��O����i��!7S��2C�������&�h�5�;p��������e�%�\I�)�42�����v�o������?���w��������E[F?����f��*Z�|�.���XF���6�f�*ae\s��Hy2�t����wx��U^���DT��h\i�]�~����z���3���7��#P��>+%�K�sG�L�|!Y��OvQ������CXG��uJ*e$|���J�|��i�:9�����t�G������s�(��m�Y��C4�6���\N����lU�Wz=�u�hI�+3d�|wn`�TK	�F.�����+�����S]
v����''|T�x8��VL��)�8����r����9+S(��
����S���������X�lJu
��L���L����C�}LFm�y�
(�Q7��f�7�R��9������~��]t@V����'���l>*��wC�R����E��	�;�����\M�"�g����8~�,Q_�[�aV��.tV��Em�F����)Zc��#��F�ZG}YS���������U����d���\�\�D�W�5���
m��x�h:I�:�����{Nh�$�Q-$�Z��P�	
��oV��p7����F�3��c���=f,���%�>�O�;{0���+d�d!�4��5�
�A0��4�Oc~��r�<�Bws:���~���i��\����������0�B�g�R0
:�|���X����-�/�:!�?h�����9bJ�4�]{O{-F@s��"z/���9��~t��_	�gx��#D�S���X�{=�T!7a��U5����3��1���q��BmZGZ��'d)��yN-��td�H���n�^]>�L?����.�t�;o�H	��u�&q����fW�%� Wo�m)��\��M����xv2q�����H9;,JH�1)6c�������h�v���b@������6F3���o�o�ma�mc�D�^�o�'������PK�]��\�dPKQU~J:patches/0012-Multi-level-partition-wise-join-support.patchUX���X���X��[{o����hI�<��i%BT?jV"HJ� ���x�������������{�!ZV����>fgf�����8Z�x��&�;Nd�
������|8Mg������LLGp�p-�`�@������������38W�4���*�"�o��n����2Ld��}%���F�����gD��d�B����m�Y��?�N<�7���;�&�����������_�"
���2���?���w���]�����:���<�(���J?��7�"R-�Y��D�	����oA�K�� ����x��[�2�����u�n�#97�"�2������=X"T� �_fT��_�d���@$���:G�s=M���"O_mS�6��,%v&KH"�K�%�+�k��B�Qd.��a�B���%<�C�h1jK��*R	��nZp������>��F������gn�D���(C���Z���T��$��:�J�( �w�JVD>�RT�TM���I���&�j[�^����s��G�G��_������f���\�����;��8��)���^9�_i�b�$�3�U������|y2[�d��J������	�cz�*���
RO����P���|O`�8W��"��^���P����Z�vF3�d uC��lz�b�k��	���vi~�8��'?�t.��dbY��=��&�U2,]��:=x�����p0u'���`������L���CPb���%�"�Gq%��������
�k�����wl�����
^J�z��f��8>�s���yyXDq�2�
P���|�<�O�bW��V��=���C���P.I��Q��6���,#%y]��/����y�����������>���p��A*O�S���nl1��Ix#��������]L�S���p*�%8�#{�E���������v��-�.+Q�h���T�|�9=s�n�*i�z�����E��&�n�MY�U����l���a���0�;ch�L��u�7��G�,ja�$�Bm+%�����,��%�m�C0�n+�MFLnb��Ry
�v��A�w�O�SV�Q,lWg��&���@]�1;�Uh`=q�#Ox�%���d6<�uR{q]L���O�����N�B�S�+����."�P���:���"�}���������z%1�a�!�{M	(��d] �2��\����0I�epE�@"e�,L�Vs��0,w�J~�2��h)&e%Z|Y�g9}�0������8���a�%����xmz�����h7;���]z&k�-i4�r�&�hT��������~��������n-
�0�Q<�h�/L_��f	%�>R��,��@
��G�,i�T_�I�mqH���f���b,�&@Q���J9�Gb������6�:�Igli�3�`�;'�N���z����5��������8u����IH���O�#jTV�I�Y(��B�"G�S�?�-l�:��PC@��M���W��������9+6�V���Q`O�����=���3�����p�[l�vN~HEP_x����(>`z�����+�:7�?��1j+�(�����%���W�D%W�B4����M���y�_�TI�?s��������}S�5��x���H����#�0s%)��
<s���R�Cd�?A)~�|q~��5%��R����oq���C|�x����G�����?Q~0O�H����x�Ij�i)��������Xg����������b"��!��Ni_,���#�h8�����>'�����D���#F�"}�Hs�;�{��H�z2��kk_����^�-n7�8�k�R���K��Rd��P�������\��S�e`j$��(��
[d\�FT�x�B�c|������k���e#,J�CI�2�����E�}�d�_v�C���4gML�s�+���1��cvzY��G/���S���P���w�[~I�EOdS]����X�Sy��$8���dx�ZK�_�.���,�;6������t���#����V�@��z{~�<���g����u�;����H4�{{gl�n R%I�-;�3�.P�CG$Il��C��l�����������|Ws^�,��F1�3+��RmLy��P�(~�Sc����Jzr)�����p��;��;��c�{�mc�����p<�v������L8�N����t����(m�*_5�J��>O��3�h"�QS2�+|U�y�;�W'��S������
���Y�+zXzT�yJv�����m�
_����Z�hj�n4����OY����P�������j���Wes�}6wb8K:�����R�z������T���053%
�]�|&�1��<z��W��t/B&!�i?�%C~y����!�������|U�����6�������Y\��.k���CS*j=���|g|M�u���#�>�Kyh�K"��l��Y���@_��qej�����m��nl����{�;����b��=��W���./���]�S�C*~��l���[�a5��U�=�A��eM�3;�����&O�I����<I�r��-������H�
Tk���Y�����a����G�����"'w���������>����Un��Q�Aq��)�Ga"?%�w~�.o"�XW7e���OT�����x������'���_��3���E�?&�U���A�����L�L����~�@>����mX��a��w�yQ���@{o�Q�����=d*�>�5�B)�4Ln�x��;���
7����?�!?#�5z��s�a��d8����-l1��?"��~C��r����'��O���`ajft���JQy!S!<�A����J]g��m�W�"2�guN>&������B����D����B�RP=@�l�x�1��t<�l�}<z
���YI�'��;+��E�����D�(Bg����I���KY�)�B��zQ
U3��8��4#�E���������`�08�z,Sz��h������l'}������9&���9�$�]n������ �m��c���z&����*�R�7�vk�mV��la�H7)H%q���.����:b�����I���)�b�!�A���r
'��.�0_�����7ztz����1	P�U�V�>�T3�n�[�Bz��z�N�%a@�����X5������N�;�c�:q�H��]��%�T:Vn�������Y-�p �}�x���M���@�w!G0�v����'�0;���z�����M���H\�E@�g�������BSJ��F�6����Y�ij��(
e�r2�vY���C���U���*��0��zy��f��A��N��5n6�PKf��
�2PKQU~J8patches/0013-Multi-level-partition-wise-join-tests.patchUX���X���X��[�s�8���o����bK&!��Ni������z77��q���m�ff���d��
��;�d��������{����o�b���^�q_������r_W�5��h�d�/�3�/�{����|�4��$In�)����g�7�Z���M
��%��"�G��c�D_v5����[�#;Fd	�dI�Q��2���N�����3�����p����T����[�3:&�#&lT�3<��:��<|�
<�zn���t:
�v������!��-���=����ko=���Ju�S�:�)cp���C\���QM�t��oM��;� 8���v�c�}��2�����}1����w�{g�����0���j���|���"��n��EGtbVovZ
�BG
nm�����%�BIt���W���t:k����8����a�{e\��a��+�^�]r./�}�!�������f���!�_������)����z�8��G��������T-�
'��o#��q�0����9N�6r1�G�]����h8�|�z2�����*5%�
K�����PCsZ�a8�_�/����'��~5����T9��z_O������������ZL�&�IBJ�%F���IRJ���lMM�A .��sy���(^�����Es�PDE,������g(�eP�#����,8�`�C��	�������s
}h:MFo�@���k�g/�]6Y]������i���"��lb&nSjC���
��0�N>�'6GT�rqE������w�\J�\	]�t$t1�bW@�����~W�����PW@BW�+ts������3���+��aX��<m��UM��;��Z�m���&����2���������z6�Q%�-�����U����+�.�U���������MG���{���-�/i]�������d@�6���$]~�����P���M��}@��g��q!�?�w�0���G����
������;�b���������������2
��3M������6f�h��
��'06L�6���j��r!z.��fP,��X�t(��X�Z8J*�e$���~#_�b��zV�Wi�q8���#��������V�x�q0�"y�%�|]�c����'�jU�(�����J0��i��5�hC���6��7r/� z�*�
�b}�
}���(�>M%��F3��<��Q0����5MW"iH��O��Zp��/e�X��*G:_��nj�o���D"�qX�#�5x��G�4�A���F��x$����BeQq��	N�8v\�B����J�M�A5��'�#�qJ��!�F�C�9B�J6Bc�^�~�@s��
�(�*��R�}�T�Qn{�K[����J.ma�@�k�����Q"�p�)��^����
 ���d*��X)G��cJ����&I�Pf��Kr��`7��bpa|��W�)��uB���'[v��������"D��1n��g/���K9p�>����B���pPq������PI�S��CqE�F�0��x���7�l^�Dh/�%)��eKft�X��/=b����:8�~>�n�=�
N�A(x#��`a�^�O���sV[�<,�im~H�?"~�Oc[lp�'l��P�����
��Y0����D��+%�c&NUS�]�0���vv��.�/Bc�X�X�q� >G
�-�}R����^}a�����f�Z](QM�(a�j������.J��r�E"p�n7�>9�%���.N[.��\,�\\�r��T�8���K"r���/%d�c�J9�7��7������/������Tj�j�o����{���|4N'/^���)�L8�w50p^	l��0��A��IT����f��_.&�������qh���f��l���E+��R�����o�V�V�f�O�����jo�]�#:Ll{�!�`�������/����{M�{Vz�b��P�pgp����]!��2;>~�!�U���'�c�����Q��/�(����;����{�=�AT�M�K�*�X��q���4�z��B�H�������6n�u�c�w����������G}\r�gFa�I�,����M�X���K!T��,�"�zW����	e�Y@�Ms�$g�5��
~��H�6���r�9�W���2Z32������B�\���8oe|�}[-���w��J��*:������vz�Z�#�CN.i��#`�hU^���N��R+�����
<��z(iJ+%�J�X�*��U�#���������9G/3����}u%������e����c���K���6d�:e	��������6��ih*��w�t5���J���/�
-��!�{�{!�0t�C�'��h��� �.l��Y5]6K�l�kk�y��������E,��[
��w�n���}��5C�%�������;*���pOA�����I��jkiL�\P�.�������������K�e�%�.(S[gg�(
�a{6�����q����B��l�����O	�����Z���S��M��@��-�
�7��Q-� �g>���M
�!��OL�r���T����J%�Y�����*�=^�&"��t3���mL=������6�)��X��>t=�a�k�������m��2���Bz�.�;�^�g��z��1�?]>��|t������������������O���9�M����6����H�����w��>��A��!w���^��PKSB��
�PPK
QU~J@�Apatches/UX���X���XPKQU~J��a;� vG@��6patches/0001-Add-RelOptInfo-top_parent_relids-IS_OTHER_REL-IS_SIM.patchUX���X���XPKQU~J��
��SG@���!patches/0002-Refactor-adjust_appendrel_attrs-adjust_appendrel_att.patchUX���X���XPKQU~J���]�:G@��8patches/0003-Refactor-calc_nestloop_required_outer-and-allow_star.patchUX���X���XPKQU~JM�n���?-@��e?patches/0004-Canonical-partition-scheme.patchUX���X���XPKQU~J�u�i
G@���Spatches/0005-In-add_paths_to_append_rel-get-partitioned_rels-for-.patchUX���X���XPKQU~J�p^K�G@��Ypatches/0006-Treat-dummy-partitioned-relation-with-partitions-as-.patchUX���X���XPKQU~Jxq>6_�5@���^patches/0007-Partition-wise-join-implementation.patchUX���X���XPKQU~Jw�*�
�*G@��U�patches/0008-Adjust-join-related-to-code-to-accept-child-relation.patchUX���X���XPKQU~J����98S+@����patches/0009-Parameterized-path-fixes.patchUX���X���XPKQU~J	�[�]%�t,@��%�patches/0010-Partition-wise-join-tests.patchUX���X���XPKQU~J�]��\�d:@����patches/0011-Multi-level-partitioned-table-expansion.patchUX���X���XPKQU~Jf��
�2:@����patches/0012-Multi-level-partition-wise-join-support.patchUX���X���XPKQU~JSB��
�P8@��<	patches/0013-Multi-level-partition-wise-join-tests.patchUX���X���XPKCm
#100Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#97)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Mar 29, 2017 at 8:39 AM, Robert Haas <robertmhaas@gmail.com> wrote:
I don't think 0011 is likely to be acceptable in current form. I
can't imagine that we just went to the trouble of getting rid of
AppendRelInfos for child partitioned rels only to turn around and put
them back again. If you just need the parent-child mappings, you can
get that from the PartitionedChildRelInfo list.

Please refer to my earlier mails on this subject [1]/messages/by-id/CAFjFpRefs5ZMnxQ2vP9v5zOtWtNPuiMYc01sb1SWjCOB1CT=uQ@mail.gmail.com, [2]. For
multi-level partition-wise join, we need RelOptInfo of a partitioned
table to contain RelOptInfo of its immediate partitions. I have not
seen any counter arguments not to create RelOptInfos for intermediate
partitioned tables. We create child RelOptInfos only for entries in
root->append_rel_list i.e. only for those relations which have an
AppendRelInfo. Since we are not creating AppendRelInfos for
partitioned partitions, we do not create RelOptInfos for those. So, to
me it looks like we have to either have AppendRelInfos for partitioned
partitions or create RelOptInfos by traversing some other list like
PartitionedChildRelInfo list. It looks odd to walk
root->append_rel_list as well as this new list for creating
RelOptInfos. But for a moment, we assume that we have to walk this
other list. But then that other list is also lossy. It stores only the
topmost parent of any of the partitioned partitions and not the
immediate parent as required to add RelOptInfos of immediate children
to the RelOptInfo of a parent.

Coming back to the point of PartitionedChildRelInfo list as a way to
maintain parent - child relationship. All the code assumes that the
parent-child relationship is stored in AppendRelInfo linked as
root->append_rel_list and walks that list to find children of a given
parent of parent/s of a given child. We will have to modify all those
places to traverse two lists instead of one. Some of those even return
AppendRelInfo structure, and now they some times return an
AppendRelInfo and sometimes PartitionedChildRelInfo. That looks ugly.

Consider a case where P has partitions p1 and p2, which in turn have
partitions p11, p12 and p21, p22 resp. Another partitioned table Q has
partitions q1, q2. q1 is further partitioned into q11, q12 but q2 is
not partitioned. The partitioning scheme of P and Q matches. Also,
partitioning scheme of p1 and q1 matches. So, a partition-wise join
between P and Q would look like P J Q = append (p11 J q11, p12 J q12,
p2 J q2), p2 J q2 being append(p21, p22) J q2. When constructing the
restrictlist (and other clauses) for p2 J q2 we need to translate the
restrictlist applicable for P J Q. This translation requires
AppendRelInfo of p2 which does not exist today. We can not use
PartitionedChildRelInfo because it doesn't have information about how
to translate Vars of P to those of p2.

I don't see a way to avoid creating AppendRelInfos for partitioned partitions.

[1]: /messages/by-id/CAFjFpRefs5ZMnxQ2vP9v5zOtWtNPuiMYc01sb1SWjCOB1CT=uQ@mail.gmail.com

--
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

#101Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Ashutosh Bapat (#100)
Re: Partition-wise join for join between (declaratively) partitioned tables

On 2017/03/30 18:35, Ashutosh Bapat wrote:

On Wed, Mar 29, 2017 at 8:39 AM, Robert Haas <robertmhaas@gmail.com> wrote:
I don't think 0011 is likely to be acceptable in current form. I
can't imagine that we just went to the trouble of getting rid of
AppendRelInfos for child partitioned rels only to turn around and put
them back again. If you just need the parent-child mappings, you can
get that from the PartitionedChildRelInfo list.

Please refer to my earlier mails on this subject [1], [2]. For
multi-level partition-wise join, we need RelOptInfo of a partitioned
table to contain RelOptInfo of its immediate partitions. I have not
seen any counter arguments not to create RelOptInfos for intermediate
partitioned tables. We create child RelOptInfos only for entries in
root->append_rel_list i.e. only for those relations which have an
AppendRelInfo. Since we are not creating AppendRelInfos for
partitioned partitions, we do not create RelOptInfos for those. So, to
me it looks like we have to either have AppendRelInfos for partitioned
partitions or create RelOptInfos by traversing some other list like
PartitionedChildRelInfo list. It looks odd to walk
root->append_rel_list as well as this new list for creating
RelOptInfos. But for a moment, we assume that we have to walk this
other list. But then that other list is also lossy. It stores only the
topmost parent of any of the partitioned partitions and not the
immediate parent as required to add RelOptInfos of immediate children
to the RelOptInfo of a parent.

So, because we want to create an Append path for each partitioned table in
a tree separately, we'll need RelOptInfo for each one, which in turn
requires an AppendRelInfo. Note that we do that only for those
partitioned child RTEs that have inh set to true, so that all the later
stages will treat it as the parent rel to create an Append path for.
There would still be partitioned child RTEs with inh set to false for
which, just like before, no AppendRelInfos and RelOptInfos are created;
they get added as the only member of partitioned_rels in the
PartitionedChildRelInfo of each partitioned table. Finally, when the
Append path for the root parent is created, its subpaths list will contain
paths of leaf partitions of all levels and its partitioned_rels list
should contain the RT indexes of partitioned tables of all levels.

If we have the following partition tree:

A
/ | \
B C D
/ \
E F

The following RTEs will be created, in that order. RTEs with inh=true are
shown with suffix _i. RTEs that get an AppendRelInfo (& a RelOptInfo) are
shown with suffix _a.

A_i_a
A
B_a
C_i_a
C
E_a
F_a
D_a

Thanks,
Amit

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

#102Robert Haas
robertmhaas@gmail.com
In reply to: Amit Langote (#101)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Mar 30, 2017 at 6:32 AM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

So, because we want to create an Append path for each partitioned table in
a tree separately, we'll need RelOptInfo for each one, which in turn
requires an AppendRelInfo.

Hmm. It's no more desirable to have an Append inside of another
Append with partitionwise join than it is in general. If we've got A
partitioned into A1, A2, A3 and similarly B partitioned into B1, B2,
and B3, and then A1 and B1 are further partitioned into A1a, A1b, B1a,
B1b, then a partitionwise join between the tables ought to end up
looking like this:

Append
-> Join (A1a, B1a)
-> Join (A1b, B1b)
-> Join (A2, B2)
-> Join (A3, B3)

So here we don't actually end up with an append-path for A1-B1 here
anywhere. But you might need that in more complex cases, I guess,
because suppose you now join this to C with partitions C1, C2, C3; but
C1 is not sub-partitioned. Then you might end up with a plan like:

Append
-> Join
-> Append
-> Join (A1a, B1a)
-> Join (A1b, B1b)
-> Scan C1
-> Join ((A2, B2), C2)
-> Join ((A3, B3), C3)

So maybe you're right.

--
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

#103Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#98)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Mar 30, 2017 at 1:14 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Done.

Ashutosh and I spent several hours discussing this patch set today.
I'm starting to become concerned about the fact that 0004 makes the
partition bounds part of the PartitionScheme, because that means you
can't do a partition-wise join between two tables that have any
difference at all in the partition bounds. It might be possible in
the future to introduce a notion of a compatible partition scheme, so
that you could say, OK, well, these two partition schemes are not
quite the same, but they are compatible, and we'll make a new
partition scheme for whatever results from reconciling them.

What I think *may* be better is to consider the partition bound
information as a property of the RelOptInfo rather than the
PartitionScheme. For example, suppose we're joining A with partitions
A1, A2, and A4 against B with partitions B1, B2, and B3 and C with
partitions C1, C2, and C5. With the current approach, we end up with
a PartitionScheme for each baserel and, not in this patch set but
maybe eventually, a separate PartitionScheme for each of (A B), (A C),
(B C), and (A B C). That seems pretty unsatisfying. If we consider
the PartitionScheme to only include the question of whether we're
doing a join on the partition keys, then if the join includes WHERE
a.key = b.key AND b.key = c.key, we can say that they all have the
same PartitionScheme up front. Then, each RelOptInfo can have a
separate list of bounds, like this:

A: 1, 2, 4
B: 1, 2, 3
C: 1, 2, 5
A B: 1, 2, 3, 4
A C: 1, 2, 4, 5
B C: 1, 2, 3, 5
A B C: 1, 2, 3, 4, 5

Or if it's an inner join, then instead of taking the union at each
level, we can take the intersection, because any partition without a
match on the other side of the join, then that partition can't produce
any rows and doesn't need to be scanned. In that case, the
RelOptInfos for (A B), (A C), (B, C), and (A, B, C) will all end up
with a bound list of 1, 2.

A related question (that I did not discuss with Ashutosh, but occurred
to me later) is whether the PartitionScheme ought to worry about
cross-type joins. For instance, if A is partitioned on an int4 column
and B is partitioned on an int8 column, and they are joined on their
respective partitioning columns, can't we still do a partition-wise
join? We do need to require that the operator family of the operator
actually used in the query, the operator family used to partition the
inner table, and the operator family used to partition the other table
all match; and the collation used for the comparison in the query, the
collation used to partition the outer table, and the collation used to
partition the inner table must all match. But it doesn't seem
necessary to require an exact type or typmod match. In many ways this
seems a whole lot like the what we test when building equivalence
classes (cf. process_equivalence) although I'm not sure that we can
leverage that in any useful way.

--
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

#104Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#103)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Apr 4, 2017 at 2:37 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Thu, Mar 30, 2017 at 1:14 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Done.

Ashutosh and I spent several hours discussing this patch set today.
I'm starting to become concerned about the fact that 0004 makes the
partition bounds part of the PartitionScheme, because that means you
can't do a partition-wise join between two tables that have any
difference at all in the partition bounds. It might be possible in
the future to introduce a notion of a compatible partition scheme, so
that you could say, OK, well, these two partition schemes are not
quite the same, but they are compatible, and we'll make a new
partition scheme for whatever results from reconciling them.

What I think *may* be better is to consider the partition bound
information as a property of the RelOptInfo rather than the
PartitionScheme. For example, suppose we're joining A with partitions
A1, A2, and A4 against B with partitions B1, B2, and B3 and C with
partitions C1, C2, and C5. With the current approach, we end up with
a PartitionScheme for each baserel and, not in this patch set but
maybe eventually, a separate PartitionScheme for each of (A B), (A C),
(B C), and (A B C). That seems pretty unsatisfying. If we consider
the PartitionScheme to only include the question of whether we're
doing a join on the partition keys, then if the join includes WHERE
a.key = b.key AND b.key = c.key, we can say that they all have the
same PartitionScheme up front. Then, each RelOptInfo can have a
separate list of bounds, like this:

A: 1, 2, 4
B: 1, 2, 3
C: 1, 2, 5
A B: 1, 2, 3, 4
A C: 1, 2, 4, 5
B C: 1, 2, 3, 5
A B C: 1, 2, 3, 4, 5

Or if it's an inner join, then instead of taking the union at each
level, we can take the intersection, because any partition without a
match on the other side of the join, then that partition can't produce
any rows and doesn't need to be scanned. In that case, the
RelOptInfos for (A B), (A C), (B, C), and (A, B, C) will all end up
with a bound list of 1, 2.

I have separated partition bounds from partition scheme. The patch adds
build_joinrel_partition_bounds() to calculate the bounds of the join
relation and the pairs of matching partitions from the joining relation.
For now the function just check whether both the relations have same bounds
and returns the bounds of the first one. But in future, we will expand this
function to merge partition bounds from the joining relation and return
pairs of matching partitions which when joined form the partitions of the
join according to the merged partition bounds.

Also, moved the code to collect partition RelOptInfos from
set_append_rel_size() to build_simple_rel(), so everything related to
partitioning gets set in that function for a base relation.

I think, we should rename partition scheme as PartitionKeyOptInfo and club
partition bounds, nparts and part_rels as PartitionDescOptInfo. But I
haven't done that in this patch yet.

A related question (that I did not discuss with Ashutosh, but occurred
to me later) is whether the PartitionScheme ought to worry about
cross-type joins. For instance, if A is partitioned on an int4 column
and B is partitioned on an int8 column, and they are joined on their
respective partitioning columns, can't we still do a partition-wise
join? We do need to require that the operator family of the operator
actually used in the query, the operator family used to partition the
inner table, and the operator family used to partition the other table
all match; and the collation used for the comparison in the query, the
collation used to partition the outer table, and the collation used to
partition the inner table must all match. But it doesn't seem
necessary to require an exact type or typmod match. In many ways this
seems a whole lot like the what we test when building equivalence
classes (cf. process_equivalence) although I'm not sure that we can
leverage that in any useful way.

Yes, I agree. For an inner join, the partition key types need to "shrink"
and for outer join they need to be "widened". I don't know if there is a
way to know "wider" or "shorter" of two given types. We might have to
implement a method to merge partition keys to produce partition key of the
join, which may be different from either of the partition keys. So,
after-all we may have to abandon the idea of canonical partition scheme. I
haven't included this change in the attached set of patches.

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

Attachments:

pg_dp_join_patches_v17.zipapplication/zip; name=pg_dp_join_patches_v17.zipDownload
PK
QU~Jpatches/UXz��X���X�PKQU~JGpatches/0001-Add-RelOptInfo-top_parent_relids-IS_OTHER_REL-IS_SIM.patchUX���X���X��=�w���?���� )���v*;r���U��t_���"b`0�d5����� @RNv�o��%������^'�Jt��;����|�����[���yg0�x��\�������6���\��Xt:��?��t��k���P�M��x���T���������2Je�N%�y��W/�o�T>������[7��`�n��������a���l�?J/}&�z}���`�g��������2|�N��E��Y�gk7�Yf�_5������?\����|�t������7���������������q���]IXn�w7
`�L������|�7k�n��8�pz���|�����H����b�
^�S%x]m!�"�J��B���� ���?
������g�I�y�(B���!
VR�KI�8��FI�:P�;�����F8UD\�+�e�8o��G���g�����q��b�%��&���`�4�"�0��0���c�9P��
C�(�jD��f�Qg�����V��
D��(G�N8����a(�R��o�]�^�qC�W�R[�@rw��u(W0���8b)��Lw^)�[��a=�Rz���R�	��6�����	���f�I�����\���y�?H��8��4KR9s��}h
�@BOE�o�(�	����a��?\*����Dk@���
��yS$; ����w��"8��a0	�&����]7"TKF���������K����0���|��9��@�z��P�M���*%Z�Y^��hu��<z��J���+��L�X�K5��yq�YEG�MF�hM~���tS��Q2��kd@�
�.����L~��
,k6��`���]I�t+	!aKG��X ��Ol�-������D�2����}�.U\
��a�I
_5+	���
p�7����(��~�Q�a����f���6��mo����e�
+��i�N	u(�"x�i���j�3�l<H���
L��/+����2q�2��sH}�u��:�L[�D$&-�xl#+���
;��8d�������,�Z�8%�D�#_��/J���rX&R���H"��FA�.S5������a�d,�#�E��$t9�8?,n%����P�>^U1�
�66x\���Q�[�X��
� ��VX�0n
v���Za�c�|�j.�{	��VmZ��!Y��}��XQ�x��w��ZXR�4����W9�7$�*�/����L�I8�9���A����4�.?�@�K�)�[���\����3�Dqz��|�*��,b��	��f��4r6�hJ���)�?X�@�"��+���.����M�-�����u�%X�$��M"+I0���A��S)ji��Q��v	{���	`�FN�F�E r�@L�R�s$�:A���d�V�3j�D>UD�!��<("��J[�����C.����8�D����<�C����M�g���ah��n8C��m
���Yg��r�a����H?r�c
�	�\����*�q�T�����p�fQl8hv+#�#y�aF�VX%��,��J�v�cq�+�n�1Q�9�B�S��� �NIC�i�l%9uF�S4��@Wx ��F�I#Ie+�)1�7�Rys��� �M���BB�G~C��6MF�YX:*`+){md�0������L��&1��*�y�=��G�D]�b�
yUH��������c����N�\�{]��8���m�1��~2h�����|	r$� �A'8l�)�/�c�������t����U��E�@;$�#Y��z�5�6r�\K��cOkZ����)����bE�l�����}����\�b����$g�~-�P���D�*?q�K���g�M�(��4��?��'�z�E���Z2�|5C����cU���<���Ia��_��j������T�e�,���,!���4��W��U|��C���=lY9������2�]\�j�m��p��T���
�Y8K4��nWAd���-�5lO����|��
n��XS(�������2��%����%���{Iv�$����D�$���4�|�:���o%EQLJL#`�(����"E�G��
��&�p�,��dfi<C�� j�ZFn0?[�*�T�����'������8�~Z�����/z8�7�q��l�z����������/F{;�kp����3��3�=~*�L���to/��w^�*������Rwv�����C��.`���������~�DA�t���?+�2�����	�~�, ���M���,l v�{f5Mv�5�.�Q���"�2�Zt�M�[X�Q����gJ�(�TN`��j�'Tg������	�SXB�s�Jh���|��� j^�g6E�3@���NG� ��0�B�Z�����h�nQ������������������t������h0@��wv����^�z�~8�?�A�2I%�y����I�i�
q��s9>)����O�G���6	j&�S�����I�o��`�qV����b���a��c
m!^�J�!�z� �&k�`gN�,��6Z/���'��x��h��K>vN�y�s�}��a��m�g������\,��i����Ls��sh&���e/�#�G�1B�?��
���~�����vN�V�����WJ�$�`V~|v"�G��.0u!���:����f$�6Q"U�������Ys��^F~_U�Ow���������X�������E�c�*r�H6�pGh�q���x���x�0�.�]5�Y)�k1L{��(�r�h?�y����:��>����c �<NS�`�v�l���q�R�B��Q��PJ�e��j%|K
���	�n��6�<��#�L�d�����@�;�I1�H��oM�-
���	,E���P|5���e,��	�[l&�����%�P��i��I��z����?��$�R���4����'G�9�P/�*"�`�R��z�#�0m0)^���>�6��^�ZQ���@��(ECwjzQPm��C�0<��qNx�R���Ny�~�=9P�7��T�x���a� "������d�a���8����e�|E�������7��>0?Q�hy3vP�L#�7�9��J� �%�U0�� ��f��sV������wZ)�P��b+Y��a�|
?H`��CQ }�$2"�gC������z������&X�������,����wo�����8UL��<ydM������.�.�����x-^��Eb��-���~������.,	����:��8�v����Sx�v�8���q�a�tV�)�5@ ,�@I�i�(�1�j��x�������Tn����l[zZ:pK\���F��u���t�!1MC��I!_��0�5�������!�(��+���;�/IX�M����
Z/��\�o�&0�Vht`�&���%��-k[��H�y7�r���=T�n����,��e�R�h���6i��*��ys\!�I`�|�l��"����n���e���>��|�}��z'!4�M����x�����'��|F��S8'a70&�@�4��9��V@K[�{���-���v��w������'Mpv����fv�����_����4q��E��O$���IQ�m��0�r�f7�Pj���d���
��6��X[���2�dZ��-�����N	=@��\5���-�B�Y^
�2�1u�t�%��N�F�}�^�'�R��B`�������/���:	��7�5��p��t%�������2���O�,>��6�r`0�&E��+�~
[�_e�N��1&7�V8��Wo��A��L���P��;�[����M���Mt$�/X�BVh���~/������,�<����#zlU1�6�$�
l��EueKj��I��;��D"��MTA�6� ��J�6��c��@N����N�|�C�T��`������NM���J��;��N��.�~�_.��#�\j�L6�MQ(�G����@���p&=��0��;�]=j�(�1*��W]����m���N61�b�8����n�-��x�[��L�dq��
n��'�g**/Ta:9��1��+�Ib�Z������cb�|����/V����aY��2��~�����ww�j�^m���V2�I���y=�M��l��hH���<g��&����l�<2�Z���x9���Q����e��V*�g
������x�7������B�����@�������cM�CaV2����������L�B^T`�>�=�Mb0%3B������7�~��K���5��*4�]�Q�O���}e9�R�����b%1Q���D��=���j���������n�*�C"��=�V���>��A���������,E���������#l�o��
�']��x+��b\H,���!���674l����zjm�v��
9�����]�����@�
���$�d�&0�pV)Y�#��5Z(��%��!�5���CC��/1�W��e��*��F&d�2#lfRM@�w2�:�Kp�H$�*,6�<A)������<��T������{�}po�YS���g�+ZO#�DU
�8%/^�v������	�B�j���4:����.�Mz_�R��E�����������5�P�O(��:Ux��>4�}8��@�H�6�����yJ��8�'N�:��i��
s����+��1D���.���o�E7iw�|�{�
��p8��kV�|Q*��=;E���;��86&(�K}��
��"\
%:��q���f�
g�8��b��V1���gf����tl���20�e��!cI��0�U�pL��W:l��gn�?o��~����k3�����6����Z��R�AJ������� ���H�9��r��fu~X;-7�����q�-��p�]�������G}[��a���~��w<�hx�M�2< �����,|���#��}��Oy��N���P����A��Pv�@�(H�>���W����EM���Q%�����x�S��CT�[=���~���	A�t���H�)�*���0��(M�Y*�q������1�%N(7@������*w����O���=�q��~�Z����~�}n>9�������34���8��q��+
d=U���R}��t�����J����t���L�d�k���.���N�F�A�����;</�X����tjG�� 
�A��1yOU �_��6�h��EV�`~���[���M}��n��s<�,�
c�\�������v���qPX���#0���������L��B�� b�#�MA���\6k:XC�=��s�Y��z��(��o�At��.�kMV�5A/��1���N���06��Q�Gx�Y6�Aq�|�vt�n�]�?��|��3��k`J�4)�EA<\�S]��MN���P�SGnJ��6UK�N�1��������<���k��x�������f*�$x��.?�K�9t�'6`�t����A�������B������-$��P��_s�B�D����n��.��^d#���O�����9?;��6>�G�9*����.x?��l7���������>��i��V��=�G��E�E��eE������r�� ��f�����r���!
�2F)��Vd.4�n�J���TiV:R��4b"��>3B����<0'��$������p �p2O��t&���-k���
��Z`�jY@�4��fw
��O�j.XG�v����1�~w��������^�`S�i��`�%j�����I<^s����arbJ�O�����
F*@RTi��nX?�����$����g�<9[y���T�t-P���XAgga}^��8���L�"��R���c �����#� "�k'm��2�k;�g�m��
��Q�|]�Z��� M��V�Y-n��$���������;�<C�VL��`�/X��mE�P�
�.����x��������A��b�rM�{^�\M�n��"�`���}�v'
X
Ml�����o���z���g�������pepGyp}?��@��C��E�c��gY����;7�����*,���� m�C����8����]1<���*�>�����1�Vv@]�r��t���E�ulz'��v�}o8�;C@�����a���H{1m�e�������s�>�]�_���d������Py|N.�r���l�Ro���*����3}1��������(n���K�����S K?�����xn���w�����sa
r�<�����5���N{��������d�(���nJ��������x?�B0r���{80y��Npy��0~��/j
X���_���G59cn[@��<�a�n%���b8����D*�t23g�	��1��+�_��r�9��{j1Pj��.���������@����P;1_n�v,��$�����V���i�[WU�;|~	/C$��6���zr�!����fN�8��#/xRz��&>W:��\`�p���Tx�Q�Y��NH�$�5���-jBr"���vi�|�oH��X�r�D�
�{��������a�VU���6����q5��� �w)Q�%s����vv�9���2��|hZ�"H�"E*7�~M@3��?��v`�G�A`
b�i{�S�2�t�I��iy�y���y��#�(���vI�RS�K����������Kk��3;��!��;��
5F�}S�M����n��n��� �qn�{y2(�cJ}A���g��f��(e*��1�����������e+NZ(@D����v�����:�qH�%�GLBIv��Ntf�4	���x2��������r,����P���bJIN��d���E5������D�ta�%��A���O)��Na�������4���>��O�����E���X���D��9Xs���>\��r���$������s[����ZM�,V�U�n�EkXm���k�MmB�YI���"���,t���L��(SU�)�=�s�\���k���U����)E�(��X�F���� WV�f�`�$����-1@�|�>BG%��1^=���|Y�����K���5������kx8�����b���x���9���x,��x��PI'���	~������0�f������T��-^>�����{��p����jfsM(�M����`�L��GUap��~g�[g�ag$[/^?N�W?�F��]��yE�w���Kn����!y�U'b��z�����#��UA���U���l�,%���P���������zS�6		|:�N��|m�������
��i�������fo/�rnw����U=�������]������R���1����Fy�Y!j]p��x��T���)����o?�7����]�,p�K�v��zP�CW����IyQIQ�+�t[^�e%���D��.|��ah��4�4QB���L���N|0��S�4R>
���4N��R1�~P����ME��j�&
,�v�Y�+�H��m���S�/<�������_��������d�vN��pK�����j�Zb[�i��1�w���� �*L���7���zgvmL�V��'�QyY^��u���>	J&\�|��t�\eN����am�u 5�`�Xayc9��,���W�U�������dVUa���~�T�j��M��W�Z��Ty��|o�������k�G��p����G��l�F��8	8�sp�/���Lm���_��I�o�MAt����W��qz��cf���V`0��q����>�P�g�n�`���������E[�/D�:�Q>n���������i�#��t4�{�
������	�r0!�?zti�d��*-K	����������#~�^s�%�IxUu��[r��-��U@����70���o������yG����
���VD�ps�+������5�;�W����|�������vNig�wOZ��{k�b%��i�@��������:z���F��q7���rm�y�X��
|�
e�Syx?��>:����e�����KL���k4�63O��.���ZD��x,jeJf�<��������14�m���2j�X����_,��kS�Vj�������ML�w�����n�'�ag^Q��s�m	V����U�����p0e���JC��X��U�����=�G�����q�8�]�GU;�U�������w
���44���!����_�3|�[�Z
�)�C�5���/�k�!�!�t-l�bK�����L<�-R��� ���O���/�������"��[�WD�x�
�b��E��(�%�n{������OPK��a;� vPK
���J	__MACOSX/UX���X���X�PK
���J__MACOSX/patches/UX���X���X�PKQU~JR__MACOSX/patches/._0001-Add-RelOptInfo-top_parent_relids-IS_OTHER_REL-IS_SIM.patchUX���X���X�c`cg`b`�MLV�V�P��'�b ��2CB��L��@l���!.�����XP���WX�X��W����P�o```hmj�jl�lnn��Y���_a���ljba�����b�k�hb�k�b��k�lhjf�jbh`b�PKM��>��PKQU~JGpatches/0002-Refactor-adjust_appendrel_attrs-adjust_appendrel_att.patchUX���X���X��ks�����+.���H����+OY��I,�����"l`P����ww���	�NO"������woO��p����?������r0�
���d���r���m���!���������^�9���^�o�0��Y��&a�b?�;a���������G������:�����N�s�:�L6`�������e=Z��z����#w�����g7�oX�:���f|i;I1������������?��$�M��x5_o�����Mf.�s
��w��#{
?����a��5����KD����Y�$����}� ty����u����Wa��kg�����]<���o;^p�x:=�p�[z��{�8�����{/YNE<����s��l���j]6�nW	�����6pp����O� ?E�Cs��u�k�'��������lF��}{�[���mn7�d������x�"��l���nV^�g���;�q5/a����>��q�V���(����"�s���0�����xd���3�_�hE
n�;��',\2?��S�������I�@$�H"P?��b���{���!a�V��5Pd
� D�7(+�>�-�vt�F���}����f;"N�,��J���	�b�� 
��4����������=�<��p�����Bys������ =D����(W{�m��x	[���Z���0:�����9^��'��q�I���o��c���������F��
R����'����)����G�H�U����m�luh�5�v����Wo�vtB��Q811WL������Y��b�aN�r���Ng����1���������&�N����G��F�i���\�s���4�K������zl\�B����������vm>�=��{���	�M�
 c�U��u���f�_��Q��R�&6�������0LLp���&�O������,F6.��F�����$��l��Qd�~��>;�r��
N��oQ)[MF�;/A���FG&�,,��_�M�,Y������V's�d'���b��@9������L��N���68��1�����/��C�dLv���MbC�w24Q8�_�
Z�t^�m�-�?�
m5	�z�����S�^N�2��X�����fv^"��o�]�`���[��������;�`k��}���&=��`������3�N���B3�{W)
�h�p��^�;����;�m04XM�6��u`M�j��	��-��9G�c,�qw�Q�!P��D!�.�_ 5*���������|v�S��1(�MTPFR�w����O$�I.45��]����A��5�9*�A����A@f�C�2 	7s.��O
[��m����#���-$0��a����l�e..�ij���!�.��T��
������|
��&���o���*FJv��;�5�2�%�������2�]�\&-�PK��P�R��aR�����p2�v�d���Z�����>��?��h~ ���<�'�����nJC�%�H�,M�|��|B��n�;a�`�)��KHV����4�R��N�]������ ��#����y	kA2�����u�X�L���]���@bg�����K���@Pp�@</yxv
R�$�!pu���N��X��|QK^�qv��J�]��	����`:�%�EP{��8�v8D+?Qd
�<l��������w������InK��I��Q�&���0���c�UvB��N��1f���$``��[�tY�	��W���S�����&���3�G/Y��b[Y	��D0����� ��mjj�����;T�D��@�����0�{�$)���A{)�Z�������
��"#���tu���.���s�����
�bw�4k��Sz���s�6 ]�y�#����-	�
�jJ^N$��Z��y�y ���0Pxa!��s*	�E�_����B�r+!������Y�y�tK0Y�S���.�IU�$�l�qqy�c��pO6�����sj�o��c��
�#�$�\2�k��5`7�N�8%G��XYE����#�A�F�x��iU�=�����p�$�����aT�\����0�p���������@�|�|���g�P~������tO�",�����QQ,S;�,��R�FS�?@E�L��	Y�
��U������2��������
R IC@�lY�.�S��K�^���E������R������Q�QS8:��ZY,_e�W�RG�K���&���c��Lrg��������,�������kc�sNup�j� }�++;HH��.��_l:��m0���I�R�������r�_����(�0?�� Pq���k�'Q�,�K����)�5.������QL-���L�[���^qLZ��	�&�G�8��$� q��8���#�
�Du����{��.Ds`�
m����@�I������u~���W����|�����>\�./�cp �YELx5�w�	E�Y���mb������^0��7EL�A�M����W��/�e��K�aE����Q��8 7�2/��|�,�`�_�AV$r}SN�^����0wt����-2�3Qe���T�+�M����s��)��O�����~O����/ v��K�M3��[�rz���Sh���������G�G �VkJ��L�
�(�c3�Ig:�F���$	�����
��U�V�9��R� U>�dx�������ad���N&3� uo?�����_�E���C��s�B|iA������X�����h����L��al��Z�F���#��"�������do7�d$i�[(�tbZSP���2�'�^�B�����Lu��K'-d�������&��9�5���j��D�=����Z1���/���4vR��Z/�K-��
3<2�Y�3��UBy�SB����eL$��\����Tg�GuQ��"W��@����s�u���o)��4��N�H��S�����lX��{
�����l ��u=�-��hW���ZD��h�����(%E�����M.�$����h��)�B$��������WAE���YS�>�K�P?���$o0Lr������FA������,W�Yz*���fu�ZinN�S�����n
Ty]��,��^��++�bm���K�q�n'�U�%@M�	t|O�����{�D~���g��lHE�}����`�hdZ'�m����tf�a
�#����
T*"Y�Tr���>�������R���Qq�k�tPJ[���T[<����Q
��)�>����k�2����x�bYa�E�/�:�(�
������`Wn-�Q~KF�k�f���KO�Z;u�b��u����*,���Tn��5�%]���#s�J�*GfG����jV9F�r���L
\���B|4�����f�a���w�����x:{��ek�Z��g��N���^h�����s����Q�v���E�x����	05G���xzg��et�x<����z�t]<�'t�]�a��yI�����`\q��{FX8��"��I�FxG�rT�$��� ��'�u�,�d!�ga;�I'�mw��M���K?u���r�={�������e�Y���t�?��P��X�s���<j�X�M�8�S��(����Z�0X�`�� ���mD����i��h�����.�P��7���N��qj��Ml�C��MuA��NS��;�����W�4C3��5�����!����(��*od2��~V[P
 ���&�������q��7e�0c�.�E��
�HI��)����asw��GJeD�(H������)/����;��N��Bw�{��\�
�\��GW��|�BXM��9����mY��9+�w_�LUk��6�G�~	���~���XpYl0��28��s�{�|&���v4��@�p
�c�x��g�&oQ*�Y�"J����'��U�
���$i/6G2;P�$t)Y�x`�]�%����
�xI�O�F����d��q���g����6d���,y��b|�@��6�M��f��z�Av?w�TL��R@�F�.�V�w��yQp�^�>d�q�<B�l��t�
�|M����O�2����*_�\d��j)L����H�\�D;�����P��$g���sI&�n)���*���v�������a�S�%R���h�z���|����pu�AP\��Q�����WD�AR������r���:��Q4AF"*:����$s�a�I5�q��}��kRE�+wE�KG����V��..����5� pu}3����|�z�16��+�WH���H'ur{��f�����i�� ��S�e�y��P�#�lH��A�������H]e��B�)�b�~~7UiN�;�Y�$��4CO=(��R���"��d�+�Y����	��=MS	qdN�A?��hW����b���[*��v#�s{��;��vj~�"Zj�@���q�jY��]��C����e�}��*ZX`���5|�d���������2*��'�i$�'�V~Y���X�u��n��2V��
���4�7�1N$;���D�S��"L�����u2-
�b�%�����;n��\��m�b6����/� �E��R���c�H�^��1��(]�^xe|��_���8h8EQ���:W'
W��.0�������:�����h���\�(�wi�0��LX�����(Lk�����
���P��25��/�������BJ�w�7^o��w�����:�0��J�Kp������<�-Y�	���Uz�~V���{h��2Y�%�E��*S�fm��[t�v�^VL�2�(\�?�9	�']))�)D��i���m�"bY�+F�q� ��CZ������+��a��or6&�GY�0^"���wd���A������	|G�Z�vS3t�����k�$��o������E(�2xk�EKY�Rq��r<�!����-��,���28����Q%���@B\���E���DX���k��%"U�'I��OX������S*
b��cC�%�lg�+�?�`���+�"C+�hgb�o&��y�%�(�n\\n��|J�
������g������t_�+� �yX��*E������t:|I�b�&i�!�wL�AHTQ=��2���M���W��#�<���o��R����?Ve�?�)$��CX������D0�z����o�'��|����o,yU �&t��#�w��P�O	�5ul]gIRt�;���M���u�51l��}���h��������cy�#2M6e���A������HV�YM����]���;?��Cd?�������zC>�.wc��@��}�0��O�����������y.R����z��)X�MW7�/��2 c�sv}v���������:�!��S%���ka��Q��{c>�Nk���W$�J���~�G�4hUr`x����q�YLI������)������}w���F�^qT�urD�#���}�3��|��	�����U��G�r�=���(�_�K������u��k�q�	F�;�N�C��/PK��
��SPKQU~JR__MACOSX/patches/._0002-Refactor-adjust_appendrel_attrs-adjust_appendrel_att.patchUX���X���X�c`cg`b`�MLV�V�P��'�b ��2CB��L��@l���!.�����XP���WX�X��W����P�o```hmj�jl�lnn��Y���_a���ljba�����b�k�hb�k�b��k�lhjf�jbh`b�PKM��>��PKQU~JGpatches/0003-Refactor-calc_nestloop_required_outer-and-allow_star.patchUX���X���X��Xms�6�l�
�����,�N�d[�]�^������v;-�1��H:n���$�X�_��w��� <x������h�f3�.&l�bL�#�,�&i��tr>�O�I?J�`	������(���9����z�2R/�[V2_��s8����0�J�5f�0����;f��Z����)z!���eDS�Eg�(�e5�Ss	��y������x�;��K�T��<Mj�KY&
�\q�Y"W�]���`y.��6L%:]b��?$��0�-Q#,V"5\

�,�[HQ��pz��n�%S�@���bV�"�����5��0gt�Z\a�3R^����b�V�*b/�6�2+V2�����YR�XHU8�����*��4-���UA��!��fi�6���}����A 9���F�Z�kX[��dYHm�12}w��v?�^gp��%}+5���\�����
�l^f�����n�o�����`0@�t8g�5�l(K��������7a
`D(�z�������3'd���8����E��2l������	gN�t���9jBW�QYL�	�6��I��0G���$2�X�`p�
���>�\�E�h�i4�da8���	"���|2��~�����>y��t��g�s�g@_�,K\�#]��or&��T$p��4��tH�7��b�S��\�� ��1�����6�S�$n��%nU��I����Ve`�<N�;��"�	����&�~��q�nQ�7@��"y����O����m���W
��v���y��W���_$�:�S����|I|a��>*��T����3c�
J%oxF������ilLm�;SV��f��j�,C'6$�����:�7�rVvG�7Cw_|��n��X��6��Ge���"E�wx�M�w�����!e��	 
���8����7��~<�(6�v��lv�B�o�T���~F�;?j��'��%#�\����I���x{�Y��o`��m�� {�S	o�y����_�c�< �
�=�o�pLos������n��Fwx)m����	�D�y���i� x�y�*��x��'�����m��$j����_��!R��
��~�7S��3?,i<AQ�g�������YvM(Q��NN��W=�\��-B+}5On�T�9�����
:�n�Q���e���b��H�\�g���G��K]�B��p����N}�n0�lWr��ry�m��
pw	z#s����z�����|���u�Y�TTQ���F
,��%�H�X�
�E�$u����B!�C9}�<��M-�j�����y�a���8~�<��tl�i�Z��/���!�7��
����|��J���CDM����)^K{{R�W��b�}G�K��T�
����������#O���h��L?����OB���>
��i��>�;�c7:��]�iQ��/1�L/�
���q=Z�p���^�y�~��=`�������kV�����4&{d��n2�S^�'��[���e�G����#�u!��m{��W�l�����������bv;�ub%h8=�f��@�aX
�����j�(���iW/����������6�k�)��.���Gz��m���r��p?fpq�g���Z�P*�]�i�x���W6�n������u��m��;�}�<�W���(:���"#B��\A����Q1�8r���8���A	��=K{P}sW��Zdk>?fUk�8q�[+�����:�<���s��@�	�"�#��O��;4��}Mf��y��N`�p^�gA�/PK���]�:PKQU~JR__MACOSX/patches/._0003-Refactor-calc_nestloop_required_outer-and-allow_star.patchUX���X���X�c`cg`b`�MLV�V�P��'�b ��2CB��L��@l���!.�����XP���WX�X��W����P�o```hmj�jl�lnn��Y���_a���ljba�����b�k�hb�k�b��k�lhjf�jbh`b�PKM��>��PKQU~J-patches/0004-Canonical-partition-scheme.patchUX���X���X��;ks�8���_�dkk�X����dr�q��k�8e{n�j6��H���"�i����_w�A�%�f�\qd�F��������gS�<���t19�f������`�\M��G�`y����$fWb���l4zI��d4w����D��,�k��������p��q&�mJ,�~�y�}�3��]�b��;�d�z<z9;|9;f{��t�����~��������=���/���I�<b[�fam�_��v����q&��&,w����2�5y'��.Et����U��Lt`����l���%�I������Z0�7�k!�2=��5O�iZ��X��	����I�'i*�6���BiqH��}���N��z�H��&��<����TK_��+���2�cB_�Z���=�?H���yg7y���-g$�}��"
3��C���_?t�U*2�i,��pCF�Sa(��pt�v+8��_�Q���0(K�m�!�r�6���RJ�\��Q�Q"#N(Y*�V���q7_�C��Rh.�����uA��������e2����qp�l�p��Hb}��?��g�c�r��mZ����6�����i��#;��g'>�L�B�cl�4<�}��G��a�p������Yg�V!2�_���t4�H�������n�Vl���<�?���uA�W��b2����j4��l�h1���<u�.������3�?���9�8d�@��C��8�I���'��,��4I��k^���:A���u��/@��l{<�RD _�pg�N'�g��C(�S������@�E9������wu����3���CV��q��?����������(c?�������D>����X}�cc���C��S`/��HQ�������!L����������i�4��BLV���:�V2���(\������;7��=�)��d��RY��Qr?$,x+���#FO�><�������7������C.�6i���}��B�G\?�p���#��X�_���
����#�>�|���	=�U��@We���U�����(f��1���x6�������|����
��OD����wq�����x����g}�Jf�����f�M�������0���nd�G�e�9����1]���F�����B��9�B.Xc�N���$[a��L�V��1���<s)N$R-�������%��d��;�;�:!� M��2Yg�F�B�0U�A�qO����+�X�l�P=2�X�u��d��$��
U�U��2+���53�KOo�>%uc���Z;�g{{�S#�,������~AI'
��>#
5��6j��lDd����d�[d`�`*n���A��P{�b���%(��#�`���f<�Z���O�#+���@c!]+�&�����R�9�$np�w���B��>���-�$�hA�d��Bfi��
��Ca��@��- ��$(�-
��x>A�5��?���M�u��u��'">�H���
�e--����	N�Q,�T�R;�I�X���������=�i�(4(eek��A��_��-l�U��O���Zs	��Nx�[�����^��e0s�V�`4���d�xJ�YA�+����LNQ"��.�����<�p�s�~�4�]�-k��8�W��x:'��*n�o�m�����1���������!�yq�{^��A����������I�RHK3������e���9zd�����*0)3�'��J.�g������^�D���^	�&Y�����n��k��2�N�����3N��
�+UA�u�+����~��.Q�����w��h��dF�BM��]X��*d��I�f�c��d1�P�j�MD\�������9d��$��D��X��`��!>����Q�!����i��c����s�)�a�deO�ABb��X��C�,��<~pJ�������������D�^A���C���H�����`
���hN��y��{��onD���J+P3`����a+�����G��'�S�G<���������s)�2X��P�I7J(W����O*th|n:����3�e!�����4��[JKa�F1:�RKZ�v=��@����+l���"}�(��m$�
�).������@Bf��1�!���-9�����n�m�e�8�����P��z QE��kT����u���:t!$�)�R�u��������B�A���j�4�8(������q��}S�)yD�b������2
b)���%�?����|ry}~}~����w}���3��;��qB��.�}P{��UU�O���.���{+4���+���0��Ef����E����j�&�����J���*�}}N8���[��#V���He�BZ�+�*�<X:�:U�%h��t�6YNVJ�v�20g�_U�p�r��W���V��P*�q�&����9b&J�(o��
/����� ������YY0Q6vg#1��!��~��a��������R�X7����dE�+qRw�Y����Q�-�����/��f����i;%�V���I���M����o��E���7��O���SaJ�l��J�+����S���b���g�W�Uq��UN;����
�����f���S�X>������<.���d0j\5�.������I �7��:^8��%�6[|
���~�l����	�U`�.\�E��8.dbn3:.����s��JI�b\]7�>���V�
�X7��H�kB|��{04h[v����B�}[�����o������~���9�U-P��HY*u�k?e�X�Z��5����D����
��Y/�M��Q<i�.k�����W�h�>�\X�(��d�d3lu������u����r+C�
�B���;�LQZZ��#������EuKe�E��A� ]�7b�o���H�+�	��A}U;��5����Ib_K������,��1��1c�B�'Q���6I�
8P]�6���E|�o�W�����Qf��Q~\ur<��bK��u&���[�(Qf!�mR��T��9�6�A^�&oC����)f�)[�^��������<������X�/U<@$��Y���?���������z�}l	��=e��dSMD?6�����M1-���7L�#��H�ks�i<�g�x�6u�i
>��=	�ZdD����8]wa��T0����@���
���t���6�0���|�e�������
UJ��w�/?�r��|��|$K*����c���@����7[Ya��N��A�Ie_�����jN�����eq��M���P����J�[��\�|����rj�+����S�fnY�OH�T�o��--YN|n�I������:��*����\���K��`��k��2�
��i^n,�zLwC
���sF<Io-xPq��@�
'|�/�]���?�dY�I�@t���B�}~��P���!�Z�W+����;�A������oW6*�)a�z�Q&�v;����n��R���"Q�yY�5=����&��h��������L~+�A��jN�/��r�o`j����
��Ji<�������g�/�Z�C�*�^��]�Hm�Tf�������"�kG��t��@��>�Y.���RAS+�>M�6$(U���d5x��b��$Y�H��K�P�w�Ax{Au|zE�����:�>�����.�������V�X|5���9a��J�QG<�qO��m���3����[�����mg�[�S~�X�&��D��������������C�9����c�)����"Mu6�PU�\_���S�b�Z������4��A�z����x��W�������k�*B��]q��r5�Z��k�����2������2��6
mU� �W���;�T�-.�o���2��x�i�,P��-�C�\q�`����(�`���W���<����m�9��������*�i����P�g���i�Rhb��=���%�<�����xM5[/Y/wkU\,K�x:N��|6uUlEP�`+������e���:����Z\�����@a�vw������(j@�B ��AEL��6$�
��U���(�,J�1U�Y����DM����>���5��������%]�_��=��$�LR��'��Q$Y(���d��
��S��e�t�0����R�T95�:
>xDZ�J������z�$N�Y�>�WYP���:��.V"��^IIv��p7B��A$�yqA��|<����e:9L�=�%|}H���������bmC���_�K�/4��J"�GW����X��#�Z[�����"��T���M	��~?�T����q�PiM�n����E�&�CT�.��UV���7��*a[��1vW�����Y`
���*Z���R��o�$���="})�=�+R�/���$|����.����zk��e/c8���L�o���4cZ�7vB��fH�mE��HW@M@��;I��*N�*g/�;NN��x����/�����@�v�_��F�,�j8����=F�*�:>�^�c+|D�c�L9�l`�p;�m{`6���\�A�^[�����
zw��.[&�r\y&E@�4���!u�G� 0�"]�ES�s��C������5;B����03�S��]�&n�8~4Ie_�({Qy�������W���X������X�jUY��f�R�*�%:����]]s���|�7m=n�bM�J_t]�G�q���"U.��H8����P
���*-�.�[���8��>R�7�L�A������zc��:�@�z6�+��'�����x<��a�"����{���y-�������%�t�NIV
K�p-{��P_����ci��Wd
_*�,���/��mQ�tE�nS�l{��T�A�fk��7|���L��F��Oe?�2���);��d�T��J�]Y�)�����m�D��~��n&��a�]'�B�&J��@��V����T�������p�����;���n��PKM�n���?PKQU~J8__MACOSX/patches/._0004-Canonical-partition-scheme.patchUX���X���X�c`cg`b`�MLV�V�P��'�b ��2CB��L��@l���!.�����XP���WX�X��W����P�o```hmj�jl�lnn��Y���_a���ljba�����b�k�hb�k�b��k�lhjf�jbh`b�PKM��>��PKQU~JGpatches/0005-In-add_paths_to_append_rel-get-partitioned_rels-for-.patchUX���X���X��Vao�6�l��Cvl��e9q�uR���A�o� P��)A��f[���H%v��s�����������r������g��0`'c�x���xp�z)��������J�x�1������u=����DfkU��Y����w'��w\(^Wu.y9q�zk�g���r�{0@tV#b��{:����k}ZG�y�N������#�A������$!"g2Te����$�y���
WP�Z�*/�V	iY[����Y���xD�$��w��\*.j����pfd!4`�B7F�B\�������e�q�M��I�Q��l��@�q?b���T��T����u������Y:1��������L���5F����v��e�r��E(���`�C�\B�1q��=�����_���A	/�y�;���i
�}�+`����g�"���$u�8I��A�%��E>$��B6��;�=��<,\�04��H��3��pT����5/�j,��Y���j!�G;ey���t�&��Q1���@��j�-|@�f��Z�)Wq��`�-�r\&��}����?�����\O/����W���l9[�������r���&�ABN�bW��|�i����oDo=�<vh�m���0���M�:�V���B�q�GA����Kh*6&�15��Q��'���M
\����}Vx{��m=��.�?�?�o/'�@^��?��;\�;|@3����_�9>���&k�
e���$<k�!*nH
���C�H��>�$�/�@��N�����-9��*���aL�U\*-����e�w�<�=H���5��[#L���g�3����� az�W���,�����d�=R�=U��0�t����|�\�k���u�p	��	p�����Z�k*C&�53�*n���m��
{��Y�-`���~��8�Z��o���J�������6���x^�)d!
,�s�@�t-b
�?r�q��s��aRCa+�V�!%�K�nEsm��y���a;0��o�����[aD	��5�0R�����}Z��6y^P�84��:�b/�����2�d����S��uWc���,6	7%;��[�j�1���1��m������������"����������z��I�w�`��]�w�v�b��)�/�FR�16�g�����8����H�>/R���@��$���L�U56j^�e����K����H�S�`B����������g�IH4�k�CY�",�f�T.������< �n��:��s �zR���/�V���d>�^�����s���X��PK�u�i
PKQU~JR__MACOSX/patches/._0005-In-add_paths_to_append_rel-get-partitioned_rels-for-.patchUX���X���X�c`cg`b`�MLV�V�P��'�b ��2CB��L��@l���!.�����XP���WX�X��W����P�o```hmj�jl�lnn��Y���_a���ljba�����b�k�hb�k�b��k�lhjf�jbh`b�PKM��>��PKQU~JGpatches/0006-Treat-dummy-partitioned-relation-with-partitions-as-.patchUX���X���X��W]o�6}�~�]_j��,���dk�.]�IW4��0E]El$R����w/%[v�������L��~�s���M	Ir�q�G�X���,]�+%q�Q6��d�''pi4\a�
���`Eq�����K�7��~�p����a����U�,�I(M�<x%��u�c��p)j�F����l~���(Z����I>�t������go ZN���p]#I����J�N9e4�Pc!�+�)��'�
��;�a�g @��HAY�L����E�U�:]���U�`�A�8��(������r!�O�Sj�[g����n���\���F=~8\pg�S	�pt-�{��^�Z�UeU���1�!������P��rr��h�I{i|�a��R��
���-�S	m��j����M��X�R�s�&�>�Cq\��Ao����y,�0��M�>���]k��X��*�����o���)=��/����E
\���I@��	$�tH5q��Bx��K��d���4��r�����>�S�>������;�Q���k��<��Zm��(�2U #I�L��ME�}��!�@��z0A��&�j��>.��qv�?���).��1���hyt��yl���{t�/`��'�%���ZD{B��lh����m�3�P��a��cxG8��08���
=�4�����.qR��x\=���B��+�v-��B{w/}t�K�6Pb�`ma`A���������_��qo�����.��6v��`��;�&IEG��%7��;'����D�G�M�*������s���7����[��s�#P�^��{j�1[E�B���o�5w�>��25�&(Ec��4-�����
�5�$�,�;�$Yf�>�v�Y�Z�,�bX"}��B�tB��o�a����xo�/���)y�����)�7�������hf��!����������C��C�;G���#Q9i����-����r�������c��j8����~�j���W���u���\�H5�<^�l�h����M�����O��l1�l�&O8$����|'<�����Y4�y�`.?<&� ��/��4Q�P��8���Y��	\�!�>1�BI�����_���V���u1������NL �2���#-���"e+�����cd-�+��,��q���5�l�C����5�fZMX
�Ri���
�L��k��������&��j�����<�����\�y���+z!���~F�����N��m$��������@\Q%��c�Fq�
O�E�PK�p^K�PKQU~JR__MACOSX/patches/._0006-Treat-dummy-partitioned-relation-with-partitions-as-.patchUX���X���X�c`cg`b`�MLV�V�P��'�b ��2CB��L��@l���!.�����XP���WX�X��W����P�o```hmj�jl�lnn��Y���_a���ljba�����b�k�hb�k�b��k�lhjf�jbh`b�PKM��>��PKQU~J5patches/0007-Partition-wise-join-implementation.patchUX���X���X��}iw���g�W��@w��
%��r�]INN����&0��@4c���Zz����|�������k���|!�������~��/.O{g��x0���h2����W�����R�����\���������A�����|#�������x���J|����#��/rY�bUd�LG�q�x��<��7��"� ��I���~����7�S������?�G����������^���q��'�.)������-��������\@G	t��_�b�>��e$�[)������s�Y�m��*�e���8�"�U��K�OD5�JQ��l����b� (�^����*[d���������o`@eY��q)���0y���1�/�����x��S�*L��+�����E2����|.�,<�Z�x|���YW��+�]7�&%U�7s�b��B���\L`���H�eI�����($L{���B5�����;�),WU�y%�/+�F��3�*�j%a����,`e�gR�Z����-�����4�~�^�:��u�B���vZ3����g0�4���E �����}v\���b~<���l���E��o����h���%�Or���^k3��6�����������;V���hl�3;N�s�YRl�?=�~�8�
�9 �i�-}�"�/����7���
9�\
����5�+�������kuz������lNcY"
�����M��X��y*h�M�.��7�����h��ggL�����O�c��(q�G�-��u*���;f���������Q!���C��Y����ts������mh����i2���#630$��RS e�1)��w��L���|]�&'���hp�'�@���T�q58
V��.[���8;��b.�����~�M&��h�
<n$2����l��_D2>9������������z���H�6�����/G'��'��sp&�����oaq��
�yR����&	7����z1,���o��zOD+�n_����s�#*	\���9)�#P��������Yd��8Gr�P�|�NW������y2���\mh�
����=Q���0���f	Sz���FHuD���n%���|.(��
D�(�?�o��Y$0��:z"xo��hf��=�pLg���`@��v��p��S��`��)D���;^������6.��xP
��������.���3`�x�p@���2�-$���L*u��J����s������)��/�MZ�0:�!�SU���d�9�E#������S��\����������/w=�p���`��$+ !4�f�q�j�@�`�F��������,���H�
����6����t`��;�;M���-y���$4���8�'�d���r�!�{t�����Gz��-��%�������j=����c��8�������bE�'��d��v�R�������0]�Xis�wq�9m��C�<��+���+��
.mD/�}a��*Ib
�<�1r�PkT��E�OG�D��.P�_V�c�qX��{�o�g�v��g�[�����d�n	��;�]��.�1�79�;D��g��W�'	2����uyG�5���������������>\��������W�� �-�9���qS����aW�
��XB��g&�B,PBabfhUF�[���P�x��5�|H�R��A��a������ec)�N�e�e�����8K@px�X	hy��u�!�+`�P^]J�^
�XuIE�`���U��a;U��q�0<l��HZ=����'T,�Y1�K~[��1[�D��j�/�y<X����R�b�u�������M��������u��&)�a�@���1�v��%��i����T9����v����z���z��'�G@�����7ko�5o��;�E�IG�a`($+H�T\8�����}@�=4�#n���-��:��D�E��k`��'F�xE�Bi@�!NbC�PV@?�����X��I��i���w�����J&�k@��0M^B��t���vI`J�JI�Q���Tk��(sE'�wD�K��"�x[g���r����"��~�{d��#�����S��k6��Y��n\���(&p���[����T;�]�^������I�5��gg���>���X
�?�eR�/�]���Z��i+*et6NF�q�{>���h�j�R��l���%J��?E��T�C�����i\0�����#^!J?�b ����x�����#��#E��noo����o�J`l�je�� ��p\2f�a@Y*S��R:(�+��=��?�C��)P;�L!�E��]�<gM�HBi`	�Y�wB4G6�I�j^����H�9�`z2������T����cb��zh�B�x��:�]��-1	@�N�s3h�C���|J��`{��lB�������K�.����*M�c����6�zX�P�1��sx�/��4Q�`�k�O�W�!�		���������q*�.�'�m����mQ)����F��C"9Y/�e����)��4(���������y?|~����ijW����'�W����������w�:D�q�8[�_���xB�D��b�5�D�@�P������-{{�%��Z�����x�T<~,����1�~|c~}��)��o�iGt���8}�o5��!��V� �%�9R�,Q!���<��)3�L�\�[Az��9IS�BY����Z���B��Z���"��OD�O/K���"�@�y�]�)���)��W�*���	�����������.��\5p��]�����"��Y�p�"�0�\�h^�jXo�x@w��|�,+��eET�|D�����7�g��K�j�#������1PC	��{��S��{�����!�������PC�!�nhX[�6����5��N���3����Z��h�	���b�8h������K�G�<K�M���~*�{Ct5K�����mvur�9����#Y���;��]�\�t|G������^���,]����q��P`�d��"�q)	��b����)
$-�a
�X�
L�K����Q�CY�K�Sm�St�a�h��Zb��VO~q������S�������Q�����*��	=�E9��;�s��G�1|����
qP+�+@�4'~�V���R*	�0��Nf2=���������9����Xy���j�;���R���-)�����c���j��������tU��jC��5��A����4�dF@@�J�[V��\�B���zM�2JV���[w��c_��-l+�}�sFs�GO�=��Zo^j���S��XRI�{������P�[x�d��CI��!<uj����h����7�-�G�yJ�\O���E���'�&��@sS����=�;�@���������W��|=e{�b�|��+�q��E~+?K����@P��FM��V��)������|9G
D� T#� �a�3� �6��-��������s�xZ=s��XUw�:Rl� �/��E5�����������Ol6R�"���!��L�n�e����F>�4}_����t]�t+3�0�9\,���q��@�
�GX���=*�	iI>�%���;���N@�� 4��e�)���[��1�	�+��iF��PN�CP���_��%�C��L�3��#�%��TK�0���)3r��R�u@�_�G#qX�����n���5�GS)�(t�r[����b=��������fZzq�g7���{=�dAj�nW�N`K��:B����
��
I�6���H6������G�p�200��(�Z���l�Q�"��}��oe5�`}]���%�y�;�vl��p��a@Q�����;���	
	�;��G��'G6b�����C1�d���R��p�"R����i(�R��UkL��M_�;��\��W�m��i�y���{��3��;��Q��PS�!Zke�
�`@�5�aD�<��O}"����F�ZK����6�,�����jTuo���B�j�$\q)p�e��g� HT�*�udm ;�27*C����	�e�8�������\���[�
X[���"��*v��t�I�t,��U(0*�<\[���M�l���~{���`��\2�MB<���
�5������+���Z
���]������P���S��p����mF��� �z�a���/�����C$���%�s{�:7g7'
?����z��uH����n��BG����q<����l�u�^�rF�iI�0UKh�hd��t���
�L��W!�1�$`R��(�5y=��s�u��Q�����NuS�a*W �^�>j~�Y��'K��ft^=����H�r8>c<M�#Z����j.b���6�t.���W����P���"7�su��Yh�<�� 9�
�Rj�����+c�\
@��R�����.�6(��U*GkT���r������4d��9S���]��u�J���]>�]a��g�%��@n�T���#�-�����<|��=3v������	[amE�`w45-�U�H�v�:�vLb����U+�1��x�����>P@@�Z���w������q����he���q�j������=���&';��H[m[N]�(���	Hh���S~x�@A��1��x:�b��������m(f��#bt��W#r3�m	�Z����0�7���u�=r�6�����h0�<?���d�^�F���i�9uYCH��'��w��a���+"Sz�KU$�i!�7T��J�=��N�F�Y�U<C��,��I@����~�����3g�]�|Pi�@������G��*�q��f8Z!��4��J�����&9��vP�
$9��"����84��?�(W)c��!���3��Q�G�1����!��������MSf��
��]�I���6�:r�C�IF�o�����0��gF�H����#S��g<Gy�*�IX{~��#�^�����V���BkU��T����4�w�f%?�U������7�����T���ce��Y�Kd'���*�_|0��s��'N-Y���������n�@Aa�����kl������r�m5�i���vC��Cy�����e�.�Rs�
���h����Gr��O�5@d�-���k��������dc���bx�JQ�-�������@��xi��")��^��{�[���L�v�~�t��u�����MH��^Wv���[/O�g��./������?����li�+[�%�w����|���<v�t�����=��������_�D�)}�l9�I�X���m�P6��yM���C�
���W��]��N �'���(1�w,PuBz����n�S�Km;�^(�5���I):������Q�B�G�`����u��y{��������h�1
��[3�[��X�P�3���!z�;�pV�
�$���!��w�\�mA�������z��=d��-4��P@t�\w1�b������8^S�6`���q��<��E���)r9��2rRvB�����!F���e:c'}�+.N{+�\i�NkR[
�g6���������X-�j�{#X�T��<�o�:
�3O�]7����p#Pv������Ul�R����&�=
����7�z�-�����p�@Y��7���HL"�*JD����It���2�������B�J�f����������K��N{�B	]���a>��
���1��K��
./;�B�{����n�H�H��l��-Km��%G6L!�i�+#hAh��q{���nm� �����.r2��Y��+�H���#�\tsb>F�^����F����Z	���(!�$�H��J��M=�����j+\��W=���ueh�����e�6�t8��4�A�V�:V��,%�w���:2^-���dv��0w���m�3��y�����U��u5<�%fu����2�P�R��<��G���M�������=�|��1�mc���3�L{{��������<��I��4c�u.b]smFZ��k�2V]}����&�Z�Y(p$�������t��xT��"����6��d�tj!Sr�
��Z�������������'���dg�(�A�#���*d����/�!������C�4��F��P�`P��1��K�pj1���Q�&|�U,��V[������!6V\��Oy���`G��Qv�c$�����De������P��F�f;�.�!����Iz���C�s?%��
L,/[Gi���]X+�7��h��-�17n=&r>m�M��#���"�����3�g�F��BT(}�?��/�jb����ESd�����@�J��qH������1�7����_3Q��tt���Qa�8�^�l�$p�z����V��s!�5�����"�����@Y�Yc���%�2L�3>���� �4�_*fD;�d:���t�����Mn}����-��n ��#l�{�1.���l4T�$����m���(���51�PCa��n�B�G�O�!���%��r����x+���.$����T^��x�d��V}m��;y{��(=2el��s����o��q=7����#�6shVFy>>3��B��\\�`��R5�(��c;mF�2F���T�q��j��"5V���a���\���<�2�yIy����LB����O?0*!�d&ttWm�SP��M
��z��Z"�5 ����Nw����?�*&�Wj�^������0�\m�?�u�<���p�Fl�/hX���Me��R/�,�:�R�[�ux�'��4N�>�C�����r�Z��r�>j��*Pa8o 5��������S����~�H�nD_�Bi%����<�nwi�UW���zw�T>��i8����E�P��9������C��k��?��7i���\}B� e�	�|�r0��e��G��b��Qf���Y5>���7q��E��W�yf�c	tO�����V�R�9E�L"CU}(/}�n���i>�?*P��������.�D<_�m�>�O��_�2��G������#�aC��2 ��J�N;�,��Fu3jC��(3����������e�	5L�����F�61��S�������E),�9`���|����t����&(
���d�Uw�0xSTh�8�1��������R��
y���I��0�WtD���U���Wz&��JK�����*J����A���91�����A^ ��p�zvh�j�����B�y*��^������QXjGO�b�x�������*���Pb�Yj� �u�y�7��q�v�S5��c�����	�����xG7�"�
Za
�GL�8v�	@`�M�4_^�C�iQ�ou��3|&0����x0Q���(�1E�iMJ�N�kP	��R�(@�����@���$��(��*F�v\]�P_=8���hg�2��T��k��qR�TwW�����.�$��l�)�@�<���"���������i��_������?�=��h����a��z>�F����w5s?����?C
�1
��}�F~�Q��2��U�|2��J�����R��T�~r�	@^5EF|S�"<��F��Z�3�N'd�p�V)d4���f�������H�!�s�3�VS�bWAd[�=��=l��7�1�4���[U:]aB�"���
5p�a�{y�"�"WA���D�n�������R��cM�liFB��qRV(�����l]D�C��Fb����><7�7rEgi?���
�s$x���
��"v��U���)s��97��t���>e��HTT��''Wg�'�n:���<�������V&���%�=��gg����I"'�<f�e�v8�T(jb�V����;���A� ���R�������M�a���0Y�}lb���q`t�����E5#4H�
��j�h�f]���T�����i
s�a}�t��*,�Ji���Lm���v���GTk���60��]K�^��JP�!b���*����j_������V���_�V�^k�\�h�����h���X��������7%����KS7PC���FP��<e���2��u�^�?8eG����� B�Y��=kHSy�r~��D��q��vcJ�Er�j�(�y�-V�p"^�������fq7��N�P�H�8�����i�P-1����@)��
���J�������\d�bV���p,q�5#���on�`��a�#�6Q��1!jn=2������SR�h��8PO:cH����>���'�{��K9������Wg�;��B����u�9A?�6�{dd��H�x)�0f���mm�pK���B -n#��I�K�CI���\4�b^��c`��t���^���nU����29��������rw��mG�:��D����������6��2�W�r������N�gb&���N6N(��=c�b�b��"&��D�$�Z'����q1%��b�mY%E�WBv��*#u�jd8]N��~��F�Z
�J"8}�;,xA���)&�U+�����]�S��������B����.�1��k��}�9�h����N�	�l�K��LO���Je�v�5&��+X=	RV^�����6}���x���C�%�a�T��M�2g��nn=BD��vo��n���������O5���l�'�O����s���`
7��w����f*T�����Q����A~����}�%Z��#(���k����^�k����B��U�c�,���yj���M�E\�*���}_5\?�W@
*1C_�p�<��NI��X|������}\c�|�X�Z������2&'�7)�44��B��a���0� �s�� ��C��(�H�0Z�_�z�6�6��m���+*��fX�Z#4�L*u%w/vR/�n��L5��D&��}��{�����N�����a���������md�wb��I������b�8�i3N��e8�3����8f��.��Q�I��h_
��v���6�{u��K�C�F��-1�*Y`��@����I�<�}a��h�*d:$f,�eE�v	����0v��a��Y��i���96a���C�g��Ev�
k�r8x�S:� ��f3�x�(��nf�=6xL�{Q.%g�5.����|���c?��/��Z�+	�/�,�a����K�y6�@����
���-S�(�gdhu?Ty��'D�b���6���=t0�b�hp�1gJo�+���/(��c��ySr-gQ��dG9��'���jG�8��$�|���S,q��d76�5de��k[1�Z��0���� �6���b?����Y������w	Cn�<�7R<P�������	�"�O�����RG		�X�Q-�����(r|��lJtrb$}V�D��BR�a,{����r�nY7m���)����y~�I\z��"�������<b7����/>W< |�<���/uq&AyM2Z~���9��m�����7b"����Z����$"\�*z�4���m�������5�����m�0������j���iM��s<y�/tL�/�M�L��wW��B���a\������7/*�AG&�+��SbN���0���A%�A�6Gw�q�����CI@K^��SZaC�4���I���D������l����&^�+GkA��)*lH9������?,ps��E���$����
�uT=)���~x�P��+rc�5�8�^o��F�U%��3vu\
)���g����^�J�H�A�����'l*�U�<E%�q�xC��j_Ka����Z���Km�TLB��i�������rwQ($L6��T��Q��V��|�9+�����Y�{(B��Q 6
�����@�w���o�l/�J�PG�0N���p �AB)f�����H1c��<#�p�������.����^��k^�s-ydln~|H�����h�V9��m�Go�ZEO;B�s���Z��3���#Z�<����5��a��a{&V�U�G*�%$,];4�:���]��y�M��Q����U��j'/����og�w�P���1����0 �6�j���
4��HE���I��?#
�oH����=�����N]����a�|
U���W����$R��y3M�r����#���p3���F*m���u&�y
-���4�����9����^u��#�����
b����G�	���1]�E�h���?��S��Q��=�J9��^����S�x��U����'�cL�9e�*,�6��g�Q��6
7(t��ia�:������C���R�Kd���9���2�KB�������+"�Fp�M7][����2�9�s���NFO���������a��C�L"�����/����Y�O������=?%C�"G� ��;��)��	�$��6�Jv�V|��pZ*��Q�0�}���I>��	��L�J��ar����(�������oIA���cl)���#�1q~��N`kO{��_����������<:w7�9j	�>��W��_���1=�(D�5����vd��t~t��q�Yb�:�4� S2����$���h$���������1�J����"E+}��E��]����{����5I��-�'.��or�1��_�
_|/+��
�//�)E�i������������\���K��2���E�%q������=,��K����!L?�������d#��r��p�X�C���@�&���P�X�y4�>#=Yx����9s�=�+~z�}7�5]���E�~t1�c�/,O��dP?V�15,,��(�v$��������s/��N��0���Qq���cV���Y^kF/�+������1�����o�w��zi�L�GP�x���o���u���C��1�"��w�w9?G�:>���{���{a�d�������nZ�����:B���@�?�+z5M��P<����E?zW�v��P�����pgC=;I�/Y��]��V���R
��C't�����od0���A��0
��*����Up1���&O����+t�9y��kZ]vYi?�^�����uG���������/���z�_��������k�Rj����A��]�gi�*a��9+p�U�� O���?[�P���O�9]����s5����������!��1�7t�`�������W{Q�waT�����Td,�O�>?k���(����k
�/#��m�Xgek�Yq���aG�	�S���MT{|�V0����g
�����e����V��o�4Y�\;:��|^�X%Y7�OG�H��^���W��7�
q�-�}���QOVKi?�"�=�3#��K���N����h���J���h<���>�w7���7�G�����*V
k�|�[S�2���nM���^����}W�'�l,U6J���2����%��F��&�rt�~"WC;L����W
����-}�������d�?��-TjS~O\�������f���MJ��
.��@��H�l~����X������Q+�J�|�"��Q�V�zM������������a���{g�3��Fr4��9$�:��u7�^��o���<�� pW#�~-��|���,��v�w8
��wM�����%�y�0����h]��"�-����V�iwtn��@
E����(�;�����O�xu���x�(-A>#�����N ��/U1��s2�"\�M����s��4�T�kG(
p���	����e�"����$��z�z>��[�c��#���8�����t	����..CV���%�E�p�qY�g�1er��������RR6�P�u����	��Y6�dq�R�(5j��B��>���������D.�� wz��^�]�C���!1���|��4���\"J���-bm�"+�b�N����K:���F2T�}VPo.#��Y�P������2:O�A�&���� H���P�Z���&3��
d�	����Z0{�lr���uEa��Y],�l����H��S)��>����o��P��e��x��"8�b�!������h{��r���5���Z�U�������-�J���d�!e�|F���n��)|�V�d�E:0<
a�E�AGo�����AG������7�?�|�������?�x������7_�}�7�:��y�j�,�f�P���!�P'���u����_���\�Bh���S�s�q������N%�>��S��T�3�����3��^�����WG�F%� J�\�!J��~Q�R�����a�����Gd��s-��P��BuZW�g�b�N����{Q�/Gr����� �D7'��������bN����8T��4�C'�Q�������)����/������uV_�������Q��i�In���	�6�9���3�,��J�e�X:�t/�[&�l���"��bAI��S������,�ej�F2��*����	��z����+��t�V;�	)H��>����Q	=(��-+�B��5�����BN��kG��dC8�b��2���v4�J@��('�V��S������i��p�8���V>���{��c�(���}�_��~���Hbn�N����u���7����}��3��Q �������w�1���c���G��/�3����O�-z7�����hm��F�6���?�l��@���0Q���Y.��?O�����5u�vz@���k}��w[��v3��+B��%.{XQ��x����wRX�S�\%W�����W�W��E��W�P��jU�����.�t�(Jb}���<E��m��W�iL�@��u��x)�>] ���T�0,*w�c�p��,��� HU'���3�'��d�Pj��K�	b�.����Ni\R[E�]����B�`�F�����9@6m�S���u�D\9�����~�Y������.m4&�?t�:p�!�	��5�vS���Dc�TS����1r"R�w�_�y��f�)��8YnA��`��l"/.�����?�������X*<<���������L�2����m^|�E��N&�)����*��p������t@�(��j�W��g����3L��vx��!��c0����y^�~��p��+c�DS9ZO1����v{k�L�,���'�P�xV����N��7��^��=��~�(M�SY��r��}n�/a0�M����#c���Bx�6���c������#�/�
�3�*w�;w79D�qtk��������I�{yvu98�d%�g#�j�utI����Q��L�_�h�!�Y���2`����<��=��������K������C�$}�!!n&�TC���M���������y�����"�H@�Z��v�
��d�\N"i�lD4�B<�Y�x�3��QT!����:��rcU4�0�)��F�y����5Na����=�c�����$����,�o�5`I����ST1k����������B����
s��/G�@��ir1�h~weqh���M���/�����J9��^/���C�})��>���������}���Q��(1�nQ��-%�o��m&y���/����$B
��$���Qi���L�v����2�nI����P��{���G�GB�G����0\��m��P�g�U_#_�2G����@�L�����t3�2B��Q��2=��LK�1�Fv��-a��y���+����Z>����[V�	����M�#������n��p����r:o�7,V����v�b5�-l��4��og��������jD�z�1��*>��������[��?QX!��H�#�(���.���%L�R���|MF#�;� BGae��(/�P�,P|�
��(TLlX�L�X�)�QS=E�
$�
h�	���^t��g���PKxq>6_�PKQU~J@__MACOSX/patches/._0007-Partition-wise-join-implementation.patchUX���X���X�c`cg`b`�MLV�V�P��'�b ��2CB��L��@l���!.�����XP���WX�X��W����P�o```hmj�jl�lnn��Y���_a���ljba�����b�k�hb�k�b��k�lhjf�jbh`b�PKM��>��PKQU~JGpatches/0008-Adjust-join-related-to-code-to-accept-child-relation.patchUX���X���X��Zis�X���;_ N,y��$TC
��������)�,=�"�Z&�3�}���i�C'��m����z�"�����x6�T��\���?=��w��3�]�����?���z�&�Ie4��px%��x8Y�@�����L�9�p3������3��?��Ty����g��������������^����hr5:�:=�������T��(���x���5
/����s�KU��%
�UJ>�)y�����<���������4)�z�-,�0�Y�)��T�������e�3%�����yXX��csR�!V|u�0�

�oJ�{s7�?s���#�~I���w5��a��}��4�B����Z(�@��8H�S��b��25W

�!��Tq<S9��!	�
���)���X7���*+{Zr
��c���,�4�~us
���
������r�B�r\��H�8���rN���|�^�iD0FQ���P�U��)b��*J%k�v��8,�9
}���a�E\`y;�V��&J�����B�9M4e�~i��P�;��/�_~����U���`��M^7w�����F��w/�����H�!K�
��$U����Q��>�R�;����B�a�g5f]Ba��"�J�,�R�O��������/-[�k�0Y��A	�������*i���0�jK�0����r�E�[k���5K�{a���K[Uh�n�� �Mdz���B�V�qQ����j��q��>s�P�+0��E��Sm
������~~a�P8Um�A����������q��x>�{etG����V������
�i�vh[����YDx���EE�Uxsd�4�&P�FE��X��Y����o��<Kq6P����7�b�+�A�
C�\�Tp�Z�q�*
��T����*r�����,O�"|��Y�
�(+�+�_/�A7K�������������^PZ��\HkAn?I@�z�Jl���q�V�>^+�h5��{T��Y��h��t�sV?\_����a1�5=��*���Qm�1B�u��-,6�*�9�L�}��[8cZH����d�+��OlN
q ��na���(,�wVX`cA�u��!���R����s�y��[�!���W���f"fB�_w��N������Y�I���z����v�P���.�a#q�Z;"�@���g�sR����8�������hz��_�?Kt*A?u�L%���z����{����.��s�#5�v?c���i�qv���e�� J�
������QtB'�	�e���	��$]��i��q������E������I�$K^���j�vE��G�K%�3S�mEc�ez}+��)��
)i�� ��%���/���	_�o� ��<M����pEG3��`~������w%��8T�������Z�+%��5]��8��r��W����bI���6����ac=e����R�l�I^�%Rk�QE?g%�����,�T�-�vqa}o�%��;��Q���iqp`��2�����qq�$uy�5b^7T�0V�M� ��\<5d�����u5	2,�ZO6��k_[Z���U��q�a� ���LroG�DR��ee�nB.�j�C����U�5]]x)j��*�gO8�
��m����\�6H3tl�@[6������������W�����������	��_�FZ�!2��a�O�CN��*���K4�����P���
��;�O��~�,hW}��{��#O9��L����i4����"�=���{��Od_NN�#��?��%��N9�����M0�F,���o����,����e�����!O>����G�w
���+T�u}'��7Tt������k v8DB���@���o�u����4�����*��"3��ss��\��c�+o��[�
�����]Fe$���:���\]an$)ud?��Jq���a�~X�����I!�c��k(j�.�e`�v���ii���U��~��g���'�\O\qr���'u�'�Cf�<��
�2l���_\�dP0"��C�)����[���Id�;�B�7�3��f�I+�.�
�s>L=��5� ��T�>}=��X�S�����Y����F�1�pW�QWS)*5D���<�����D��8^=����l� �<�:��T���#�����np#hf�5)����.	�����0�*���
?Z^���
�����t�A��c�����[���\kN�M����CAY $���G=�Hvy��O�@�����:�{���v"��X��~�[����2�����@�� �:�W����YvI��u�������e(���:��%�A�e�rO|�n��,C[%@���TF��Y|J���x�������?L*��N2v'�\��!��R������ ���.����iDG�1���C0�A�~���B0�A���n��;��.��g����}�G[Q�&��f��}�����;����O]��*�rm��+�j���e;�������^��Z���p]]i*�K���=�	&������`^���d^[��6>��'t��m%�H�c#��s� R��T��q���C���w��:�l���n���2�2R�k�?R^�o0�g��K�����s��nu���h�������n	8}?�Nyp��'�:@ziT������	Az��WZ�7yv�Fo�o��e�p�����%�{t>��Gz���D��I����BvDF(�kj���x9����B�{�a�����o������Y�>0����t��t�������/o�ha<��(h*�d��q����>!t���k)�f6�����N��<����Tg�G\���``����4[�f�f��H&X������+w�����-o�'w�}�%�_��L���J��O7R~�@���[������X=2��\^4�7����Tv����h����������H�.!���r��x=8��2c���*ou�.E�(KWmw7C���b������'����Vv���#Zf�`�;.��\�H=�C�o����'��<sB���UO�i�'�4��w�f�,�P[����1����eC�>�,���,KK���������,gg������.NM�Wr����d���H���&�nr�1W��;��Z_?9�w3t���o��0Y����}K�(;c�G����6����b�^c����g�&�_O��.��"��+�dv�/c$���c|�4�������Mg�#i�x����5�1�S�i��0��J��~*P��K/�?��i	���.��c��F��y�&��O~�D<��#���:���S^��FGw�w��D"���:���Q�l�fW����]��D����������SC�<�h�5���K;��2s��r��������^��-��3JR��*���[9�u]:�0��/�B>����
�G���]���$���Hzi�n�~�������DNB���t�ZZ���.��u)�P����-]
��P+�u���l�F���t�,��PKw�*�
�*PKQU~JR__MACOSX/patches/._0008-Adjust-join-related-to-code-to-accept-child-relation.patchUX���X���X�c`cg`b`�MLV�V�P��'�b ��2CB��L��@l���!.�����XP���WX�X��W����P�o```hmj�jl�lnn��Y���_a���ljba�����b�k�hb�k�b��k�lhjf�jbh`b�PKM��>��PKQU~J+patches/0009-Parameterized-path-fixes.patchUX���X���X��<ks�8���_����H�����d\��$�����=����R�$$1�H� �����~$���<�j�n]�-@��/t7|�&KqL�������'��zpzrr|t�I�?�u&e�w(^'��V+�"���?1����>0���^�Y��'���x"�����Tq��Ujxc?Y�w��z�P�F��)���{���N�`��`�{�{���=�yyq����;��?�oq)S�T2��`!f�oJ���wJ���L���9�R�s%�T,'�>	c���B�"[(��JJ������S�$�B���
}5�U��c
��@�0�%��~f���Ao�%��%��2�ND�������<�'���=X���������O�1�z��� �m��0�wu�/�bf2�`Ea�l���[PL�������z���t[����x������|�
�c�W��$�'���00��
��Y����dS�D�
����C<!���<wJ�2���"��W�$�.�;�H�l�h��.��6Mci����}�d�s��W7��.^���}���C����^Dj.�����;Il����������A*���F�0��F�����H�Y�2�]`7�dZ@[�b��	�"D
�������)�1��F��\�6�9G{�D���81dx���J�F�0/$@S��u�K���u4t�t��y��`�U0�\,M��fU�	`��������D�"*��?��F�"(aL��2���=��`#o�dK2Z|��Tcqi0�i��dy�x��$@��r�>�)��B����8	��;��`9�]����2�t�t��E����=tv���8�MVY��9�]\�.R�����S0����D4���1��4f��c?��DS�<F��*/�H�cQ����]�[�b�r�?�����P]8�'���)�h2�+����3��V)����?�����~�Vu&F�y2�����f`m�~������c	{�d&a[�;><D&?�$}h�������I'��]��$w�s��5����:C�A����"����������	|��+}a��#�}d�G���="�����'`N#u�u�z��XW��J����OvyLw`@0F�'�4���(�gO^�'��\U0�C=���)���%�{0O/A��N�$�P�XG��<���+�]e���"Pnd�#��o�����g�����m%�6�[@�%��6#���>J�[�<��]�*��(����@���9�Q�jK�@���������4f}6���r{���7�R4!�Z��0.�7l�X;<A��j�)��$�`��~��������
��b]B��>E�����V0�5Dd�,�T�V`�0S�e�F��t���e�	U���^��3�x�mW������	�|���-T���b��3��v����^p�y��?;����������%�
��~�������"I>�J�u��`�Q��]�ei�@�L<��,�"D`D�/vva�P=�v����M���Z�&~o~}�
�bw��@��Ch��=�v��_��.#�#.I6�@�{�p�Y��|7�����A;Z����� �������f��+#��VxK=M�Er������M��
�4��`�r�C��s0GS^
��0���:|�L��g���h%>%�C1��0u�#��Q��:�a�}A�_��O)t���b�B�Z�X������u���!'��
I9������L���S��qztx���0�_a�����K�z%!<���@�;.2��6���������� w.q-;��3p���P���,'��$������6��c�A��=%��F�]�r�}w�]��q��q�DS�![PO�h������A��;(�B�E�4$���R�����]E6�3�~;��&C�U1_&�m9"���X��7�+!:{ ��Hb��13��W'��^)M��vg��4z#��/j���������������\��!��O
� �V\���&����\�JFQ�oC��S�8�|5�2��<
��2�V�	�n�y���Q��M���S�/�R�&�����
��t-��;y?�Rzg	������i����z�Z���::�����`x0�Se,`a���g���>����<K2�����p?�{=,��)q�����&�N\�uHB�4xj�� �M�(�2
����
)�VOa���T�\t��I)��8��{���,���<�=��,��[��d
e�$V�����	=e#�f��~C����-Z��<��8Ta��}�r�q��j.��j�4�C��Q*�A�P�G�
^�����e���m&�a�n��3��8��Z�U�P��j3i�B"�Ro�#M�3
�9��I���:�hi�~���e�b�K�sV�r����-�|�&iV��	pS���@������j��m�Ua\����iY�iU��nB���Mr:��� ���-�q��.UY��h�;������g=8 !9n}�Hf����X:Zl�n��3s���Z?��gG���Xy�dr�o��!m
��})9uJ����	Et]�������e����������y�[�V7MX\b�G2�J�o
Y�X�qd�L����
��=Z�V��6�kM3��,U���z��2)�w#}%�
���Z����ZE3K\`
o��{�����{GCL���En�����2Pa�����26J�d�C����6$�Z�a�`�4���3���E���s���A��3e�af�l��<#X�O�&.��
\�<���l��$	��&T���T���Z����N �8 ��HE����"�Y.�������g<
���H'��qA����1qa���'B�8���e~�
�0L�{|d3�Zw�q�y@@��m��v��T6���������K�o^���h�����DP�'l�>l�K�A�������z	\�����a�05�'��#��O;���B������`Wak���
���W-��EW��"z�]�P��y����U;1��3���s�T�r:P��)D�&�����������k�X�`�:�E����x�BHNm�F�{qz��&�wO*�9�/�����AG�1$�V�{���k66pF�]�&��#�����Dt.�[�H-H}�)m$�F�b �osi�W%�k�y�L��}U�L�A��U�*��W�g�o�C�����!5��s�2���Q<A7S����M6>-����
����0�����������	K�i����l��%��3��z|J������x�B������u�����m�|�e���tZq!��bbY��ZX:w�og�Oa���J��6�o��o�jWk��>�s�Zq}[�>
4����S9y\G�bQ�"�����d	�6��*�����w�NR��B�������M��	��������n��G����L[#���h���Tg0��#����Bs�9|$Y6Xlt:L0k���Rs����m���<��a97
���'M��|�O�_0�����l����������*���Tp��=�|U�yk0�
��{n7g�@�XF�2�)V��p" A�j���cq�0�TSi#�9�!�elv[�+s��,`������SfP���%p�h�C�(�"�Lg+�������"R�O�3<c���	�2�dU�����&!�@m�;����P�`�����CDsdT�@��|��������*I3,�����)&o��-������m���+�j�m!_e�_E��Z�kK�����K��rU�U��!���n��1��l0�U��!v�~%Cl��DC��J����X
���E�Z|��]c�y]�KR��W��JC�n���Bo��
�z����7Jg
���~����N�D�=/�����<����8���WA�i{��@gi�	&A>'Fs��5`�$R{�������-d�}��,7�����bTM��r��X:3�K��9�li]X7�Z}I���j�/����B����o��>��u���+���/�����#�xm�����j
�/��m2O�EoH~���SO9u:��,�[��.q��V�����IZ�&c���3��\�#�X�3���_�TY�kXV���l����ITa�������ZV8T���M��|�����[q
o%�2jh���I~�3������gPM�*��[����������O���N��"�K�r�E+�*��?�fHM�f���d����[p�P�)���&��]���,Z�����������hjm@��j<_6iS�z�@_G���am��H���u��
��x_�$SY�g��0�+�<� k	��"qrG���/���4���hQ�e���.�|��)���8T]�w�
�����U\Q�p�EM1R���JO�VW-��U�);�Ts�
��TG�5q�L�vf�����8����bf�!2e���&�&UU��{����^
��Y?��������-x4���/�g�V��Wj%�d�
#��cb�����c�u8G��c0��
�����jmu�; �JL
H����� �j�t46��l,�0``�*�g���f4A.S��xpo,./_r���zi���=W�������)���5�.�����*�jE���-��1������d�!I���Nk�r�u�(/��6������x���ooM�5�_�#����.�)��4E!�]�e������-��T}����u�H�,b�$C���G���;,���tI�8���)�Ioa6��#��{?RC�w6�������H!0��Y���8�!�8X�xk�zH�����\b�)�)a�%�Luo�TD>�����Zj"�(�P�p��t�YP�A���f��C$k�u��]|�-���R��5��_3(���fpjv�9|�F���dQw������%1��;����{c��Q���
�k���Iz���4,���i�Y����R��U��xw2�jZ���K=��^S����;��S�������{
��2�'�T��H�� �}<������/Y�7�������B1k�j���`*HtS��c��L��oh��_�@-��b��)�c |�.���� 
���bZ�U�g���0����������E�:�����f��-�/��H��@%w��I���#������G�tn�>l.D��t��9e�6�F�}�����\�8�����d<��Nf��yb-�����.t�����=������k�����s�}��U��$�y�����p�Z��m�M��\����m���s�x��8��{'���E�%���u	D;Zu	i{YG����p�e\�p	�<�3� �/���I�^u3|2��.E���m����-��1rub���0����|/���~�~��g������xk[�htB����
1��z�5���/`���F�6���9�`O�.���8����5�{��1{�������}�to"��F�D��tB�9#���mS������������5lk)���6s�J���L������-�STgZ��k���Nc�Vk�r���O��]��wz2;Q��w��'���o��yk7~��>
�Coj�
9 �2�8bj�"|��ZU�*@��.g������1Hr�2NW�$�������o6��4�T6>�l.�����������PK����98SPKQU~J6__MACOSX/patches/._0009-Parameterized-path-fixes.patchUX���X���X�c`cg`b`�MLV�V�P��'�b ��2CB��L��@l���!.�����XP���WX�X��W����P�o```hmj�jl�lnn��Y���_a���ljba�����b�k�hb�k�b��k�lhjf�jbh`b�PKM��>��PKQU~J,patches/0010-Partition-wise-join-tests.patchUX���X���X��=ks�6���_�������,��_����x��:�������-%A�6�cDj���x��A$%�1S$��������b�F=��Z����19�v-rL�5�Z�����7��{�~]��-Y"�u�����]����9G���_x�g���['x����d���r5��x�-f�5~t|r�����N�;2��(h�:����]t������z�o2���?��{�3��G��/������t1?|����9���{�F��a����%h� �/|4Z�}'L�&����!�	�������.�D\2n,C�d�|g����������L���7�6`E�p�{���Z���b���_o0?v���
^��tx��tD�X�V�����X��B��9P��>�r&����D}���jt��u�",��(.l��U��2Z�~�	_<
�q]����))����������Q�rZ`x���G-��FA}�9{xh��������3�e�b���A��Cc�
��y���^Jn4[�	e��q�g�F=p#��t2A���S9G�34���������������G#�d����q�����{txb��)����mD_^\��&�b�I���9������&�=�����.�^��u5-�Z<R�5�����.���u����M���a�Ku�C>��W���"s6
b�`qy���������Dw?\]��r�[��P���h(n#��YQ�_��o7w����F?��\\�t�tZ �����z���Q��������C���>��n/�.���i���/�2��Q��F����I�B�n>������ho���xM��������q��������&��ps������~��}���I��.��zy+�hv[�V��n5������
r�����>�����&�^sgF��O��hMi�"��#e���
�����jUkU���W���� >����2� ���������$O?�Vd_a�y��A�2�bU"�N�f$\��0�����.bS�)��Mq56� ���fS�����I����,c��tT��)e��aEH %6.������k�|�����6�]�
�����q�>�������I�}M����W���O1�d�������hF[$�_Y"������|ys�CB���t���i>���6n��	���?������.��!n�m��b��&��/��� ���8�������	����+2��Z���g�vC���D�s�l.�Cw:��s������I u
�n�U\T\X,��<a��G�&c;\�������9��\�`��K_Y}� ��76}��_���^�=����)�H/�\2�_�������4roM�G	�A������:A�?�)p�!}Ke��h������x�q�2�)��y�<�s�����>�����|!}P���u����C�|(u����9�q��O}A���?���8Nt}I"�������.9����L���G���$	S��M.��E����
t�4Y���:?�Y������d'(��j*LorSL]�c
Z\_�X��
�.��jP��+������Z�����`�/�<������Z��cO!��Lu|~���[BW���a�aQB}]Q����*������.���aV!�0����^P�$X�����
'E��$q&E����.�R���n.�M�5��z���
�����GV������gk��Ah9�`0=3[']n���1��L�q�cJ�W\�����|E����5�l����%��f����Y����������_��x�j��l�Y�]7�l������Xu�#���Q���<
����N��t������h�=�J���|:b�N�����n�KU�6w���{�`����n������j�C����Z�������z:z@��hL&�����C���2�;EI� _GE%kA�:�������p���H&	[u���](k�Pl(L��p�����u�HKJ�9`�OG��gu���?^�Ml
������*���tX��N��E�P
��4�*K���Z(��P��*�0a8Y����'Ew���Mw[Aw���0�{��*���tX��N��E��3��$�;S�HA�U8�3xr�	 �T
�o����QWm������x���T-������<��)�V&!sK>r+l����`RWa	 ��\����K�#��$��b�H��u:d
k���Xd��d��b^��v`@]�5�&����R
nb��o�[;����D���U��+y���-�?�-~`f|�L���"���bB9L�<I4f4��EE4F�����������}Y#�~���q��n_�����~7px��@�.���V9�b����<,�V���Ho��'��[9�������/��%�MK��sq�#����~��=�Yb�G��������]?50d���S��!.i�H����%T������d��P���6���%w�����}���20����j��>j�����vY���������9���`���[��-���<}����4O�yc�v�Z����'��
��Z����zu�5%����U�5������j���5��w��Rh�" ��%��l�\(+�U�����t����zp=���W�=�JL�������c?zTe����{F���
]{5�>5)��67�Z��������+����0r!,��-�����~\��g��Z�'�3f���~Z�?���[
e0���p���TM1�|��.��3��F��(a?�>�>/0a�O�K���?���������D���}}�oI�"����01�
0��B�1l��
0���~��;�'�����}b�'%���~4M'��f��=y�f$q�["�gpf��,�6��'������,���F\%&���yO\�#���2��
'�Y87*��3��+�YDn�n���)P���Y,]��"2����g7Ex��1�&g2VgI���*��*��s�S~����A�������2U� R����
�l&P���"����5!�`�&e���A��_�,���J�~%%b:fS���T��"/Ul�S���D���5���<�=4a!]�Z�t~�V����x^~������8gU����6<��&V�G#w��:��`W{��$oq"���A����+�Z)#��p�5�[�������%f��jw����	m��rwM<���W��]1�K����4���6R��]^O
��]�����Mg$*`����$M��H]H�����-�O5
N�5�B=Q�c*���]���n}��yV�VT��:��$���]����KTuP.���o��S��\����#m��<��2Z��..��}S�&q�4>�����E��^7f's�
�0k&����m���N-o>-�0����=�?����B�0);c����d6=��n���'&)�`�8��P#����]q6B��6�:�M����cY�������,`��q��!Fc��k�)��,��CW�o
z��y�[@/��d(�x�o����������P��f��PUS���<�����,"N
?����i|��7H
D�.n�]��6}p���M?����E�p������<��R������uhI��x�����UU�4f��{=���7���5]�����$K��"����v�'�EYe	��>/q�gp�i�:�����.���=���\�1	���*��n�WV��+lEV�R��8$#��U�11�(�LF[wc����kY����~As\Z���j����/�(r� 1$z��h�(\�O+���Xw4�T���=��ur+�c����a��niQ*z�&PRTr�t�LOY���=~(�Q��!C>��
)���?O�o��b�=@���oD��k����H���%��$)�;	;f���.�.o�^6E��-�:�#rj��Fs��|�/�L��y��s��(n������>*�@��=;oZR�fb���R�j��&��q�����"�+������9�b�\Rb�H~�B�!�A���`��H{�@@�#���O�r�,l*S���u�[p�Z�����%`�[RQ U��*���l,�.�s�G#6{�P~�Fx*N��:��t.[���; `�Zc�R�qT��a�����a�!����U��:��x�Aq9m�xc�Z�N��'N�\\8�t���B��o������hu��Vd�#G:t4%��#6
�AM	������1�$�n�fQ� ������@I�1'I��m-!��\���-�-5��$�z\��_&*��]&/�r�����D�b����@	:�:�8��0��H`���x���B�u��B���f�A�}k�j=?.���J��4��%�X�,����
0����+T��a�Z�3����X���7a�e�������"JE��c��hZJ^5�IV�2�f9�Z�W�q$�"�<)".����NQ*�h�E�]���W�:��i�1�4m� x�4�+D�/���O��!�%��B�,���la�@P�����B]����y5�s���<�
+;�����R$N���,U�H����������vJ�j�����BB_������d�0 z�W�E[�T6D?�����,�T��a01sv��B�,���P��b��9&a�kbCAS\�g��hW(��Xx�2���C_�T6�$�}M�a([�Ki0�.����R�b�;���].�)V���>J��5����>7��x�[�����T�xx;����f���4�l&��V�����>7�������"�Q����)�@�Ne�)W?����$lSw��	8���&�YB��j���Y&�z����������=B�3��%�k ���1y�F��'���a���'{)ki\r)���k��WmdO�yy�F�*6���yy/������;u$��W<�:p�w7b�����a#������V��H����	�����V�����(._<���?O����%�����TD��:������������+|�s���60�����������0
W������v�4(
7u���_�(!��F�7��R�Wx�xK�W(h�y������(n���y�B\�i��4z���e��M�z�����v��m��
MGi^�A^����(n���y�E\��ai�l�����6��Up���j�{/�?���M��a�3a/�I�	�=����ArS'Jm#��0�b�M(q�j����B���	
C��B_�a��;���dL\;�l)&E3
3��������w|U���V��2�U4�M&M��	��rs�j�T��r?�_��D�Ki&�J��l��[,���r2�>�(�������+?8�t�;]�byf���z�:�W
\����^��@��x�8��9/�7��awM{x��N��X3����^ ��U/�C���`^�����=\_�M�?O�u�����[F�-��7[��%H��mD(TG�N^�U����~�;��K���In�x��z��J��0_��<
oL�+�N���NBv:��N"X=�>���d�
�^���D�!��b�~�P�!C�Va(
9�/"Z�m2Ti-_�U���m&��$�'�����fduOx�	�a��9s,<P�coY��d�Z�8�9]��}|�����W��G��+�LB�S"4��/�+nV��G(`&/���e��5+��������jy$a*D��=UjTh���H+�W
+�ldY+����C\#&UZ�����d�j��l�U{	V���XU-~+�V$�b�dt3��6VU�XUL�v���X��`U@0��U���mERv)V�A�7���ro<;�Y�Y;m�|��/3��M���-�uo%#V�&#PN�q�3����\M�����E1n�<�hm8�luW	�D�5����rL4&SF�e�8�h��
���g�������*2Kqh����TV�ro2�G*���O����d���)F�r��*|q�~�K�:g&P)W_e�U!m�c�7
n������F]q=Q��>S�������l��4��_����;��;������X�y�z���VP3���t�$8�SS33;h�i���jf�I53��vXSx�H)O���'�#8��#m�<�f�]@���G�����YM��Hg����SI�8#���dI�����!�����j�W't����2��/��f��`3+�������Kr���'>{?eKd���49W�}���T���6�Ee�}�?����=(96'J�J�������*��M9�A���gt��Y)B�5��yoZ>�)5]��)�,�q
�XB:V"KH����K��VB��65#�N!���n+�nKH���FHW�J7��|�
���/vJ��8"@1�U0���V��V�`�V��
����L�k�F�7�v�'����zK�E;��/��Y��u�9��9�yh�X!w�����3���@��K�1#0��?%-_���k��r�+d��M����_qQ�
��f���[T����ed\+���8������d���Ip?
�k��H�,'1���C�
�	�!�����@�f��@�F�5,��P���!�
O�u}��\u]5�5�|���`up�u���g�������J��]8U	�����D��C_���_�����rD|4s���C���G������{>����'s��	cnB*I�����:nx���#���0 ��� �A<%��G����-���B��E[�5Uk7��*]>�E:�j����CFFJ�������"q��d�D6�s��Zi.�5@���b��s>DVX-s
N}��bh���wpN��������5N��2��&��X�2���.r�jb�crJ������@���m3�<p�L�}������ \�o��PY/D������u��H�%m���ujm:L�j�{�n�b�+^��)�^�]����I|m)a[���f�W�����Q�����f�����x�l%l;�1�l�p�1�q�S���7��
�p�k'}%�~|c�������x���������7�p�m��(a��7��G���)��T	�4�1�lA���5:S�>�o�������|buU������l��}�O,e�����`[���|o+z| +B��o����q_�M��r��D�������k��;Qt�j�9�V�'��������|	�����Mg$2�O�#r��������mi��T���.b@�Cv�>a�r�������"�,i���'p��^G���at�����A�� �J�Z��@�6�t!.%�[��7�a�x}m��{NN���P8�D�Ix�NIQy��\���v4�g����_���b���1>�9�9�b�����E��>e=�;���pj�X�!��s�%�/U(������K*�� f���.�0�2l�v>zZh�a�+`[)t������N�
�v��gOAB$aO���C4��L�4���6����)B�����V�B ���,@�^��MA�'G����q���H�M�����9��;%c4�c���r���C��dBV�������)�[�3]���e�1pZ�m�Tz�nq��]T���0�����(��B�sv�A���J�%�$��mE_��"�"<?Q�p\��y�GB�	���|�*�#��V�V��R�D/=WXl�����
�kP��������L��*��V*@��PUi�]����C�w^
�r��<��:��~�q����g����c�:�`����
:�&C�<�����7���:�*}�����#�����+�.r��}E�����P@�R��=���X�����b5&+y�7�m�����`�����a��)h�q����N�-�%��!�����E��"�N��q�c^T���bX/q������3�9���������@/�'4��ks�L�xY��d5YU�D_���]@�8���Y^
H��
�����/�LVQ\$-wYO�C������#Y�H���8���6+����O�t��FGl��hE����#�?��w���x�4�H��r��lr�v�N���t'�_�n���kP
j��888�+��������'�@����-�Cw:��-�[��j�d
�_(�P���v���s�������n���G\2�����C���w\�����dz?p��]{tAc�(K����~=B��?s���G%��p1����#l
������`������{>^<�?f.��&�����q�������X}������m.�3�����(������_8t�aR����xtdxa
����B�����s�0[v����wD[���D��0eM���7�����=�� �G�������T��o�.>��:��5qH���o��(zV:�W'�m��+z�����5��7�`��|������=Y�/=��#K��=�#9'Ts0ecN�dJ;[�I�#���q���u{�~����hL>�����\CT���hj�3L�-�G�=6g�Ww�~���1\G�i8��������~�]s��]�y����0�H�V�m`�yC�����f��T1����QL#�s�a@#�r,P��)�A�����8�S@GZ�K;#��bT���������*��O9R�����.�I2Nf�W�@�\ L)���)��������n��P*�X"!UJ'�z����"]��"(�iUk������m�4�����-�~]h(���\��Qg�-+����g�a����X��	�W�@��A��4
����E��,����/'�[�),&���D>.V�A<:�1�����>S���hz�=f��N��a	T��_0���@��@P�������E��O��h;����/���X�R�����_T����Hs��0��	8�3\dq���������/�G�r�f�M��xh�Z�������H9��xPQl��x�'��*�e3�/x�R;���DP��96&I�X�7�z����!u	o��>�����D���X�Cd�yTOU�P���l�KG\��5*X�L�0�]WT����j|��7(�+�~uqK�N[��g6}p���|�h�Dm��L�I t����R"f>���l�yZRq�}����\�6L73k9Z8.�F�)����n���5�Se#�;������M/w�[������0s
HE�C0��V��0F�N�1���!�Nc(?H#<��sa�\v:�-��v���l�?Q�}���6���������\��������u�BNb]$}���7y5}h�����d0F����7��];j����e����S��:�TJ�DRQ�����������E�b�c�s�)�R�����arni:Q��\+�>�&@Rd�xj�����&DW	������R<Mc�����@Hr��$��y��fk����V��������<f��������d�Z����Sp��Ks��!��]����J���x����*��|�������������������?�K���"��b��P���W��Gz~�H��X�;�y��D{��{��x��(�����c�~A8_R�����E�R�%7��I���o4�PK	�[�]%�tPKQU~J7__MACOSX/patches/._0010-Partition-wise-join-tests.patchUX���X���X�c`cg`b`�MLV�V�P��'�b ��2CB��L��@l���!.�����XP���WX�X��W����P�o```hmj�jl�lnn��Y���_a���ljba�����b�k�hb�k�b��k�lhjf�jbh`b�PKM��>��PKQU~J:patches/0011-Multi-level-partitioned-table-expansion.patchUX���X���X��<is�8���_����d�a��;�qg�;��vWjj�K���E�I*��'�}���%�����j\�-�������7Q��t"�f�g'��hv|������`����t�����	�vN��07j-�'b0xF��h06��g�"^n�0^��r-�����3��$*ZG^��Y�	W/��e���7���#�NF
`����`,:��x����~WN�L����������p8�M�����������/��@�"�3_	�u-�����Wa'��I�,U�%2p�Xz*�����\�
I("5�ah�,U3m��$����w�8�����0��"���Z�����y�y���������M����}!^nq���"P��Q�H9�(�`���>���0�@\�Kb!g���vE�N
�"|�{$��/X�8���ps`�3�4�����z����������D�����X��{!#�����N`��|F���1
a 	\�}�}���43����#�,
^����J=E�q���Z��X��X��N�]q������q�41 C�H}��M�#�.	�8P��6�$K/R��{��SL�$l&J�pb
��/�D��:#�EE���k��
�u�����$�H_�z@9XG���ms�c<97��4i�L�f���k/bJ�VQ�M������.�����X��`
�6�KN$����Q>xalTto�w$���_7oe��X+K����.�{����4�"�J���d��$\xF����1�<`[�[��;$f��!
���6����������zMG��L:�A��u����V�!���w����	!�G����^
��ct6���^�S/Rk��	p%<��3ap�D��a���!h�W�j��CPF�o4�~]�?��5|�C�=�c��-d�P.���P/V��V����@�@e~�k7As����[���������U_���GG#����:����GGH���lR��/������?�3O�H��O�����h}dV@�&P����'�od
O�@:�sR���Z�j
��2���?���7E��hxs������_���������&��"�<��o?|���������f���6�0��Q��~?<���@\�I
�O!�!�$�'q}{)@�J�SI���Eu�B`4��pRk�bD��NO5D^G�UA�h��2����il�*����2�*5]w^��fd����yC�7��}F�U�$,6��4���`�Z��>�;M�8�!�o��h"�����0��
�;Dp�p�d���p=`#���d�����Ve@r�h�<�s�}~h�;���O�Ry;K�R��6�����1l�����
�T�VAL�i�?>;>;�����\9���+�����5E�N�C���wt���������j� �q��
��4	�N���t%���ob��5�s
�O]�HDj��qq��8�93^����&��l"�s��	��"L5�p��&'` ���H,��hT;���������73��g��g�������`����B�#[z3�c�����>��Sk�w�Z�3�1���Z� �^��'/5���
���o�]g@3:��������u�B������B�������=�D���Sh_B�:,`l`� t.���O�[�T�[���rG'f���d>�g
�PK�Hqn)��N\�0�s�7k�%]����g��gI��5NY�r������h�MP��g ��w�z�e�!B����
�{��)�$�E��0�����oh�b
�V-Z�?7*��-��S�����R��J�� ��yBCF����cv$���t��|�E������m� �U����)���0
X,�5����ydUP�A�����0"R�Q2��+hS���bgM�H�.3uK�j4r#�s�(�����a�)���
�J���Z?k�t�&�(���0��kHJ��|�s&9�Och~�j-��|�{>q����z�Vl������9������YN�=>�������5��)u����'�'�Z1���
'���(�p�F�=�t��o���@����)�!�����V�i8��!�7�?�k��gL�f���*���&�@iF
k�����*��������=F&��0�dr���juO)�s�B�*�F(���=�9�A5�q:H��}c*���"���X��Hd��	2W+E"�J�'�������N�����Hf�,�N��d�l���{#�/x����x;i<��<��U"Y�xs3w�������<�s���E#^����z��������Yv���@(����E�����&U�0?.�e�jy��
`������v�_bP�H��� 3\�V�4jG���Z���'p�W@��PC�8#�l�|�[I���`��m�R.Nk����������'Qe��"��0��h���k���]��7�A�/n/o�zq})��.�)x`������b_��%�Z�J�V���T��T�&FJ:`��\[e%������_&yA6���\���_�4�(�a��x�6�#j������&���j���|{	�8��xr�uIod:�+ZV��Q�+�|EX�g�,z�N����BTQ�Z7*����K�j����M1��lTA*�[������U�I�4�0%���7�mh����T�����Z9���.��m�^��|�!� ��f4u�W������<�f����`S��Aup�����-G��
C�)��_z	��a�����
��|bq9���*m��c�F�
�m��������\a���l	1��r�d���~E|Z�� -������U
RU��08+at:��gJ��3D��0KL�f��4*rf�����1�*++Ll�z�zz������B�����G�	&q���Q���&I�mJ&��������j���(GY�������������W�W�_���^�|{���o�ss#�;�E��b����H�l�/h"AQ9�wh:qo=nJ6w�SR[.D mt����o���w��C$�C�O�\�V���2�����]��������I3�������<��8V�N�y1�CL-y���������	3��C1���1n-��}��V���=��:&�b���]�(�
�6��7�>CA����5���
,����_��-`:5`l�l�.�����}����o$)
[�r����/�����u6`z�������R�JQ4e3��CODJ�^���B-����g�S����	h�6��H�4������ K��erAy�e�&R�E<����a�%I�kF��?�P!z0�U���
FD��`��{���������w)���Pp�26���@Kj8��6���\�'�c��R�p�����
y��f@j4o��Y��P;�'������~�6�G��Y.���A`j)^�b�O����\W�����a��j��?���4���MA��<Qf���3)�S���m]ulU�d_���r��^�%��,�����
Wu,�zk#��Sg�/-��J^OY�r�#��TI��������e�g��C���AW���L9���#5�m���c�F���2�,�*���Cn�<�#�����������Ml6��.����MvuTa�qWL]��y�b����`u1z�^vK3Z���eZ���]�&��U��n�l7�f�A~~'�
zU�
�x�?CU��;!�o�w��,���#4+��9E#����qQ	��1?t�:�C����V��>LKs��@&�����L�����+>bU�_C�e���D�*c`��'��v�G�Qw8|���
���}k�^';G���O���B�U����#$Q������%�RE�� �����L�TGC�f>)G�y�Y�����xT���ro��:K%�*��b�^�x�+�w�X'�u���]�M����8!�����C)7SN�R����#}r
i�1����*���0��~��B��X��,�y����m�&�R��������5�RIu:�0L�R�s�����K���/L����m�6,��`����hU�L��M���9�N��j���y��?eN�����,�s	�`�����4�����r���{����v[`i���;���Pu����l���M4Bjj��]D���
�z��0]�q.�e���	��KUcz��g�=�6C��U5��^�4(>CF���������5��e���Ur���]7u�3����3�v�#���q[��\>�����}d-~�f_b�nG_�;�.V�bJ�a64�e�Z�,��y����\�������f�/7(�4�h�� ���c�,��a����4���C�o�Dl+����U�����=q��+�����X}�g�9�^d���[ar$�V���G1ic�J��0���3���R����X�([��W�����������������[���,f��fK���:����^4���o6J�3�,%C*&�GG.8d\��O=�b�i�/�$�
4�����m�m����h��n��`a�f'������dp��~����u�m3$�;����?E�iH��(C�I:Y�F]O��(�7W�8��0$�l&�������������J�z�W���-���7h���K�z�����IJ�{=��=J�zv�^���y�="�������~������{��v�v��U���:����L}Gi_q��%z=�8�6��Uz��2��U��r�y��<�\�Q�����y�O=JE^��*�z�y�\�*�z���^�~�R����#U���W��w�E��KTU�U��*����J�|�����������cQ�x�w��b��U�Y���z���N�ksx������Z�n���BC}N�u�NN��~�������Z���j����9;��o�
_u ��:�u;� 8�������G��L��
���v7���7	w/���<�+�k�pNa`J�5��iXSYQ�$+h��d�0u�}����e����~��e�<��B��
�O����@a�����X,�I��4y�����(�����^s�oD�;�aD���:� }u�O����\�g�
D�'���:|�q�#Mt�S��2��
!2�"�R4���:
����������A_���3��l��vMp���a�-��I0���i� }]��c����DE�i�T��i��YnUV.}��:>L�,��gE��"�GP�zL������Z1a�@Q��@L_�A�wZS��[w�t}e�4I�C|Fc����Q��M��dD[x���
}����/.�K���K���s/m`O�knG���I���{�0��+bp<�����]���G���[k�XN��de��3�|�.�j�6�q���n��FX�*GK)����U�L�����O&z��1�A�J�����[���Y�H���3���
��a�?�ja^-�P���~H�'����������[���<g^�����Bp�I�� �#�\������������(4?�o<�7�l��A3��>(r�U���5�%�}a&�����N����������q&bF2���:�iH�U����+Z����w��#}���.���"E+������f	����,����U�����n���<@�9l�H���Z���)�*��� ��0�[)-��L';������*�etOw�)�S�Zh�Tg��.
��v��tU��$D��
U2�����s���>�������$9�:{�N�@��
��&3L�YL�]���O�<�`����&�RE���f7�~���h�yk
(R�7��~r�\� �q{�a,�PA��[�Xr�g$�G���_�����W�����;h��`j�8��j)ac�V�~gjD�L9.�z
hN7b����^|V�z}����35Y��!b�:�T���'%R$|�2V���'�4���+��a��4���.x�*HsG��)��O����i��!7S��2C�������&�h�5�;p��������e�%�\I�)�42�����v�o������?���w��������E[F?����f��*Z�|�.���XF���6�f�*ae\s��Hy2�t����wx��U^���DT��h\i�]�~����z���3���7��#P��>+%�K�sG�L�|!Y��OvQ������CXG��uJ*e$|���J�|��i�:9�����t�G������s�(��m�Y��C4�6���\N����lU�Wz=�u�hI�+3d�|wn`�TK	�F.�����+�����S]
v����''|T�x8��VL��)�8����r����9+S(��
����S���������X�lJu
��L���L����C�}LFm�y�
(�Q7��f�7�R��9������~��]t@V����'���l>*��wC�R����E��	�;�����\M�"�g����8~�,Q_�[�aV��.tV��Em�F����)Zc��#��F�ZG}YS���������U����d���\�\�D�W�5���
m��x�h:I�:�����{Nh�$�Q-$�Z��P�	
��oV��p7����F�3��c���=f,���%�>�O�;{0���+d�d!�4��5�
�A0��4�Oc~��r�<�Bws:���~���i��\����������0�B�g�R0
:�|���X����-�/�:!�?h�����9bJ�4�]{O{-F@s��"z/���9��~t��_	�gx��#D�S���X�{=�T!7a��U5����3��1���q��BmZGZ��'d)��yN-��td�H���n�^]>�L?����.�t�;o�H	��u�&q����fW�%� Wo�m)��\��M����xv2q�����H9;,JH�1)6c�������h�v���b@������6F3���o�o�ma�mc�D�^�o�'������PK�]��\�dPKQU~JE__MACOSX/patches/._0011-Multi-level-partitioned-table-expansion.patchUX���X���X�c`cg`b`�MLV�V�P��'�b ��2CB��L��@l���!.�����XP���WX�X��W����P�o```hmj�jl�lnn��Y���_a���ljba�����b�k�hb�k�b��k�lhjf�jbh`b�PKM��>��PKQU~J:patches/0012-Multi-level-partition-wise-join-support.patchUX���X���X��[{o����hI�<��i%BT?jV"HJ� ���x�������������{�!ZV����>fgf�����8Z�x��&�;Nd�
������|8Mg������LLGp�p-�`�@������������38W�4���*�"�o��n����2Ld��}%���F�����gD��d�B����m�Y��?�N<�7���;�&�����������_�"
���2���?���w���]�����:���<�(���J?��7�"R-�Y��D�	����oA�K�� ����x��[�2�����u�n�#97�"�2������=X"T� �_fT��_�d���@$���:G�s=M���"O_mS�6��,%v&KH"�K�%�+�k��B�Qd.��a�B���%<�C�h1jK��*R	��nZp������>��F������gn�D���(C���Z���T��$��:�J�( �w�JVD>�RT�TM���I���&�j[�^����s��G�G��_������f���\�����;��8��)���^9�_i�b�$�3�U������|y2[�d��J������	�cz�*���
RO����P���|O`�8W��"��^���P����Z�vF3�d uC��lz�b�k��	���vi~�8��'?�t.��dbY��=��&�U2,]��:=x�����p0u'���`������L���CPb���%�"�Gq%��������
�k�����wl�����
^J�z��f��8>�s���yyXDq�2�
P���|�<�O�bW��V��=���C���P.I��Q��6���,#%y]��/����y�����������>���p��A*O�S���nl1��Ix#��������]L�S���p*�%8�#{�E���������v��-�.+Q�h���T�|�9=s�n�*i�z�����E��&�n�MY�U����l���a���0�;ch�L��u�7��G�,ja�$�Bm+%�����,��%�m�C0�n+�MFLnb��Ry
�v��A�w�O�SV�Q,lWg��&���@]�1;�Uh`=q�#Ox�%���d6<�uR{q]L���O�����N�B�S�+����."�P���:���"�}���������z%1�a�!�{M	(��d] �2��\����0I�epE�@"e�,L�Vs��0,w�J~�2��h)&e%Z|Y�g9}�0������8���a�%����xmz�����h7;���]z&k�-i4�r�&�hT��������~��������n-
�0�Q<�h�/L_��f	%�>R��,��@
��G�,i�T_�I�mqH���f���b,�&@Q���J9�Gb������6�:�Igli�3�`�;'�N���z����5��������8u����IH���O�#jTV�I�Y(��B�"G�S�?�-l�:��PC@��M���W��������9+6�V���Q`O�����=���3�����p�[l�vN~HEP_x����(>`z�����+�:7�?��1j+�(�����%���W�D%W�B4����M���y�_�TI�?s��������}S�5��x���H����#�0s%)��
<s���R�Cd�?A)~�|q~��5%��R����oq���C|�x����G�����?Q~0O�H����x�Ij�i)��������Xg����������b"��!��Ni_,���#�h8�����>'�����D���#F�"}�Hs�;�{��H�z2��kk_����^�-n7�8�k�R���K��Rd��P�������\��S�e`j$��(��
[d\�FT�x�B�c|������k���e#,J�CI�2�����E�}�d�_v�C���4gML�s�+���1��cvzY��G/���S���P���w�[~I�EOdS]����X�Sy��$8���dx�ZK�_�.���,�;6������t���#����V�@��z{~�<���g����u�;����H4�{{gl�n R%I�-;�3�.P�CG$Il��C��l�����������|Ws^�,��F1�3+��RmLy��P�(~�Sc����Jzr)�����p��;��;��c�{�mc�����p<�v������L8�N����t����(m�*_5�J��>O��3�h"�QS2�+|U�y�;�W'��S������
���Y�+zXzT�yJv�����m�
_����Z�hj�n4����OY����P�������j���Wes�}6wb8K:�����R�z������T���053%
�]�|&�1��<z��W��t/B&!�i?�%C~y����!�������|U�����6�������Y\��.k���CS*j=���|g|M�u���#�>�Kyh�K"��l��Y���@_��qej�����m��nl����{�;����b��=��W���./���]�S�C*~��l���[�a5��U�=�A��eM�3;�����&O�I����<I�r��-������H�
Tk���Y�����a����G�����"'w���������>����Un��Q�Aq��)�Ga"?%�w~�.o"�XW7e���OT�����x������'���_��3���E�?&�U���A�����L�L����~�@>����mX��a��w�yQ���@{o�Q�����=d*�>�5�B)�4Ln�x��;���
7����?�!?#�5z��s�a��d8����-l1��?"��~C��r����'��O���`ajft���JQy!S!<�A����J]g��m�W�"2�guN>&������B����D����B�RP=@�l�x�1��t<�l�}<z
���YI�'��;+��E�����D�(Bg����I���KY�)�B��zQ
U3��8��4#�E���������`�08�z,Sz��h������l'}������9&���9�$�]n������ �m��c���z&����*�R�7�vk�mV��la�H7)H%q���.����:b�����I���)�b�!�A���r
'��.�0_�����7ztz����1	P�U�V�>�T3�n�[�Bz��z�N�%a@�����X5������N�;�c�:q�H��]��%�T:Vn�������Y-�p �}�x���M���@�w!G0�v����'�0;���z�����M���H\�E@�g�������BSJ��F�6����Y�ij��(
e�r2�vY���C���U���*��0��zy��f��A��N��5n6�PKf��
�2PKQU~JE__MACOSX/patches/._0012-Multi-level-partition-wise-join-support.patchUX���X���X�c`cg`b`�MLV�V�P��'�b ��2CB��L��@l���!.�����XP���WX�X��W����P�o```hmj�jl�lnn��Y���_a���ljba�����b�k�hb�k�b��k�lhjf�jbh`b�PKM��>��PKQU~J8patches/0013-Multi-level-partition-wise-join-tests.patchUX���X���X��[�s�8���o����bK&!��Ni������z77��q���m�ff���d��
��;�d��������{����o�b���^�q_������r_W�5��h�d�/�3�/�{����|�4��$In�)����g�7�Z���M
��%��"�G��c�D_v5����[�#;Fd	�dI�Q��2���N�����3�����p����T����[�3:&�#&lT�3<��:��<|�
<�zn���t:
�v������!��-���=����ko=���Ju�S�:�)cp���C\���QM�t��oM��;� 8���v�c�}��2�����}1����w�{g�����0���j���|���"��n��EGtbVovZ
�BG
nm�����%�BIt���W���t:k����8����a�{e\��a��+�^�]r./�}�!�������f���!�_������)����z�8��G��������T-�
'��o#��q�0����9N�6r1�G�]����h8�|�z2�����*5%�
K�����PCsZ�a8�_�/����'��~5����T9��z_O������������ZL�&�IBJ�%F���IRJ���lMM�A .��sy���(^�����Es�PDE,������g(�eP�#����,8�`�C��	�������s
}h:MFo�@���k�g/�]6Y]������i���"��lb&nSjC���
��0�N>�'6GT�rqE������w�\J�\	]�t$t1�bW@�����~W�����PW@BW�+ts������3���+��aX��<m��UM��;��Z�m���&����2���������z6�Q%�-�����U����+�.�U���������MG���{���-�/i]�������d@�6���$]~�����P���M��}@��g��q!�?�w�0���G����
������;�b���������������2
��3M������6f�h��
��'06L�6���j��r!z.��fP,��X�t(��X�Z8J*�e$���~#_�b��zV�Wi�q8���#��������V�x�q0�"y�%�|]�c����'�jU�(�����J0��i��5�hC���6��7r/� z�*�
�b}�
}���(�>M%��F3��<��Q0����5MW"iH��O��Zp��/e�X��*G:_��nj�o���D"�qX�#�5x��G�4�A���F��x$����BeQq��	N�8v\�B����J�M�A5��'�#�qJ��!�F�C�9B�J6Bc�^�~�@s��
�(�*��R�}�T�Qn{�K[����J.ma�@�k�����Q"�p�)��^����
 ���d*��X)G��cJ����&I�Pf��Kr��`7��bpa|��W�)��uB���'[v��������"D��1n��g/���K9p�>����B���pPq������PI�S��CqE�F�0��x���7�l^�Dh/�%)��eKft�X��/=b����:8�~>�n�=�
N�A(x#��`a�^�O���sV[�<,�im~H�?"~�Oc[lp�'l��P�����
��Y0����D��+%�c&NUS�]�0���vv��.�/Bc�X�X�q� >G
�-�}R����^}a�����f�Z](QM�(a�j������.J��r�E"p�n7�>9�%���.N[.��\,�\\�r��T�8���K"r���/%d�c�J9�7��7������/������Tj�j�o����{���|4N'/^���)�L8�w50p^	l��0��A��IT����f��_.&�������qh���f��l���E+��R�����o�V�V�f�O�����jo�]�#:Ll{�!�`�������/����{M�{Vz�b��P�pgp����]!��2;>~�!�U���'�c�����Q��/�(����;����{�=�AT�M�K�*�X��q���4�z��B�H�������6n�u�c�w����������G}\r�gFa�I�,����M�X���K!T��,�"�zW����	e�Y@�Ms�$g�5��
~��H�6���r�9�W���2Z32������B�\���8oe|�}[-���w��J��*:������vz�Z�#�CN.i��#`�hU^���N��R+�����
<��z(iJ+%�J�X�*��U�#���������9G/3����}u%������e����c���K���6d�:e	��������6��ih*��w�t5���J���/�
-��!�{�{!�0t�C�'��h��� �.l��Y5]6K�l�kk�y��������E,��[
��w�n���}��5C�%�������;*���pOA�����I��jkiL�\P�.�������������K�e�%�.(S[gg�(
�a{6�����q����B��l�����O	�����Z���S��M��@��-�
�7��Q-� �g>���M
�!��OL�r���T����J%�Y�����*�=^�&"��t3���mL=������6�)��X��>t=�a�k�������m��2���Bz�.�;�^�g��z��1�?]>��|t������������������O���9�M����6����H�����w��>��A��!w���^��PKSB��
�PPKQU~JC__MACOSX/patches/._0013-Multi-level-partition-wise-join-tests.patchUX���X���X�c`cg`b`�MLV�V�P��'�b ��2CB��L��@l���!.�����XP���WX�X��W����P�o```hmj�jl�lnn��Y���_a���ljba�����b�k�hb�k�b��k�lhjf�jbh`b�PKM��>��PKQU~J__MACOSX/._patchesUXz��X���X�c`cg`b`�MLV�V�P��'�b ��2CB��L��@l���!.�����XP���WX�X��W����P�o```hmj�jl�lnn��Y���_a���ljba�����b�k�hb�k�b��k�lhjf�jbh`b�PKM��>��PK
QU~J@�Apatches/UXz��X���XPKQU~J��a;� vG@��6patches/0001-Add-RelOptInfo-top_parent_relids-IS_OTHER_REL-IS_SIM.patchUX���X���XPK
���J	@�A�!__MACOSX/UX���X���XPK
���J@�A�!__MACOSX/patches/UX���X���XPKQU~JM��>��R@��"__MACOSX/patches/._0001-Add-RelOptInfo-top_parent_relids-IS_OTHER_REL-IS_SIM.patchUX���X���XPKQU~J��
��SG@��#patches/0002-Refactor-adjust_appendrel_attrs-adjust_appendrel_att.patchUX���X���XPKQU~JM��>��R@���9__MACOSX/patches/._0002-Refactor-adjust_appendrel_attrs-adjust_appendrel_att.patchUX���X���XPKQU~J���]�:G@���:patches/0003-Refactor-calc_nestloop_required_outer-and-allow_star.patchUX���X���XPKQU~JM��>��R@��B__MACOSX/patches/._0003-Refactor-calc_nestloop_required_outer-and-allow_star.patchUX���X���XPKQU~JM�n���?-@��#Cpatches/0004-Canonical-partition-scheme.patchUX���X���XPKQU~JM��>��8@��PW__MACOSX/patches/._0004-Canonical-partition-scheme.patchUX���X���XPKQU~J�u�i
G@��NXpatches/0005-In-add_paths_to_append_rel-get-partitioned_rels-for-.patchUX���X���XPKQU~JM��>��R@���]__MACOSX/patches/._0005-In-add_paths_to_append_rel-get-partitioned_rels-for-.patchUX���X���XPKQU~J�p^K�G@���^patches/0006-Treat-dummy-partitioned-relation-with-partitions-as-.patchUX���X���XPKQU~JM��>��R@��xd__MACOSX/patches/._0006-Treat-dummy-partitioned-relation-with-partitions-as-.patchUX���X���XPKQU~Jxq>6_�5@���epatches/0007-Partition-wise-join-implementation.patchUX���X���XPKQU~JM��>��@@��A�__MACOSX/patches/._0007-Partition-wise-join-implementation.patchUX���X���XPKQU~Jw�*�
�*G@��G�patches/0008-Adjust-join-related-to-code-to-accept-child-relation.patchUX���X���XPKQU~JM��>��R@��u�__MACOSX/patches/._0008-Adjust-join-related-to-code-to-accept-child-relation.patchUX���X���XPKQU~J����98S+@����patches/0009-Parameterized-path-fixes.patchUX���X���XPKQU~JM��>��6@��/�__MACOSX/patches/._0009-Parameterized-path-fixes.patchUX���X���XPKQU~J	�[�]%�t,@��+�patches/0010-Partition-wise-join-tests.patchUX���X���XPKQU~JM��>��7@����__MACOSX/patches/._0010-Partition-wise-join-tests.patchUX���X���XPKQU~J�]��\�d:@����patches/0011-Multi-level-partitioned-table-expansion.patchUX���X���XPKQU~JM��>��E@���__MACOSX/patches/._0011-Multi-level-partitioned-table-expansion.patchUX���X���XPKQU~Jf��
�2:@���patches/0012-Multi-level-partition-wise-join-support.patchUX���X���XPKQU~JM��>��E@��J__MACOSX/patches/._0012-Multi-level-partition-wise-join-support.patchUX���X���XPKQU~JSB��
�P8@��Upatches/0013-Multi-level-partition-wise-join-tests.patchUX���X���XPKQU~JM��>��C@���!__MACOSX/patches/._0013-Multi-level-partition-wise-join-tests.patchUX���X���XPKQU~JM��>��@���"__MACOSX/._patchesUXz��X���XPK�
g#
#105Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Ashutosh Bapat (#104)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

Somehow I sent the old patch set again. Here's the real v17.

On Tue, Apr 4, 2017 at 7:52 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

On Tue, Apr 4, 2017 at 2:37 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Thu, Mar 30, 2017 at 1:14 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Done.

Ashutosh and I spent several hours discussing this patch set today.
I'm starting to become concerned about the fact that 0004 makes the
partition bounds part of the PartitionScheme, because that means you
can't do a partition-wise join between two tables that have any
difference at all in the partition bounds. It might be possible in
the future to introduce a notion of a compatible partition scheme, so
that you could say, OK, well, these two partition schemes are not
quite the same, but they are compatible, and we'll make a new
partition scheme for whatever results from reconciling them.

What I think *may* be better is to consider the partition bound
information as a property of the RelOptInfo rather than the
PartitionScheme. For example, suppose we're joining A with partitions
A1, A2, and A4 against B with partitions B1, B2, and B3 and C with
partitions C1, C2, and C5. With the current approach, we end up with
a PartitionScheme for each baserel and, not in this patch set but
maybe eventually, a separate PartitionScheme for each of (A B), (A C),
(B C), and (A B C). That seems pretty unsatisfying. If we consider
the PartitionScheme to only include the question of whether we're
doing a join on the partition keys, then if the join includes WHERE
a.key = b.key AND b.key = c.key, we can say that they all have the
same PartitionScheme up front. Then, each RelOptInfo can have a
separate list of bounds, like this:

A: 1, 2, 4
B: 1, 2, 3
C: 1, 2, 5
A B: 1, 2, 3, 4
A C: 1, 2, 4, 5
B C: 1, 2, 3, 5
A B C: 1, 2, 3, 4, 5

Or if it's an inner join, then instead of taking the union at each
level, we can take the intersection, because any partition without a
match on the other side of the join, then that partition can't produce
any rows and doesn't need to be scanned. In that case, the
RelOptInfos for (A B), (A C), (B, C), and (A, B, C) will all end up
with a bound list of 1, 2.

I have separated partition bounds from partition scheme. The patch adds
build_joinrel_partition_bounds() to calculate the bounds of the join
relation and the pairs of matching partitions from the joining relation. For
now the function just check whether both the relations have same bounds and
returns the bounds of the first one. But in future, we will expand this
function to merge partition bounds from the joining relation and return
pairs of matching partitions which when joined form the partitions of the
join according to the merged partition bounds.

Also, moved the code to collect partition RelOptInfos from
set_append_rel_size() to build_simple_rel(), so everything related to
partitioning gets set in that function for a base relation.

I think, we should rename partition scheme as PartitionKeyOptInfo and club
partition bounds, nparts and part_rels as PartitionDescOptInfo. But I
haven't done that in this patch yet.

A related question (that I did not discuss with Ashutosh, but occurred
to me later) is whether the PartitionScheme ought to worry about
cross-type joins. For instance, if A is partitioned on an int4 column
and B is partitioned on an int8 column, and they are joined on their
respective partitioning columns, can't we still do a partition-wise
join? We do need to require that the operator family of the operator
actually used in the query, the operator family used to partition the
inner table, and the operator family used to partition the other table
all match; and the collation used for the comparison in the query, the
collation used to partition the outer table, and the collation used to
partition the inner table must all match. But it doesn't seem
necessary to require an exact type or typmod match. In many ways this
seems a whole lot like the what we test when building equivalence
classes (cf. process_equivalence) although I'm not sure that we can
leverage that in any useful way.

Yes, I agree. For an inner join, the partition key types need to "shrink"
and for outer join they need to be "widened". I don't know if there is a way
to know "wider" or "shorter" of two given types. We might have to implement
a method to merge partition keys to produce partition key of the join, which
may be different from either of the partition keys. So, after-all we may
have to abandon the idea of canonical partition scheme. I haven't included
this change in the attached set of patches.

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

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

Attachments:

pg_dp_join_patches_v17.zipapplication/zip; name=pg_dp_join_patches_v17.zipDownload
#106Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#104)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Apr 4, 2017 at 10:22 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Yes, I agree. For an inner join, the partition key types need to "shrink"
and for outer join they need to be "widened". I don't know if there is a way
to know "wider" or "shorter" of two given types. We might have to implement
a method to merge partition keys to produce partition key of the join, which
may be different from either of the partition keys. So, after-all we may
have to abandon the idea of canonical partition scheme. I haven't included
this change in the attached set of patches.

I think this is why you need to regard the partitioning scheme as
something more like an equivalence class - possibly the partitioning
scheme should actually contain (or be?) an equivalence class. Suppose
this is the query:

SELECT * FROM i4 INNER JOIN i8 ON i4.x = i8.x;

...where i4 (x) is an int4 partitioning key and i8 (x) is an int8
partitioning key. It's meaningless to ask whether the result of the
join is partitioned by int4 or int8. It's partitioned by the
equivalence class that contains both i4.x and i8.x. If the result of
this join where joined to another table on either of those two
columns, a second partition-wise join would be theoretically possible.
If you insist on knowing the type of the partitioning scheme, rather
than just the opfamily, you've boxed yourself into a corner from which
there's no good escape.

--
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

#107Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#106)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Apr 5, 2017 at 8:39 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Tue, Apr 4, 2017 at 10:22 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Yes, I agree. For an inner join, the partition key types need to "shrink"
and for outer join they need to be "widened". I don't know if there is a way
to know "wider" or "shorter" of two given types. We might have to implement
a method to merge partition keys to produce partition key of the join, which
may be different from either of the partition keys. So, after-all we may
have to abandon the idea of canonical partition scheme. I haven't included
this change in the attached set of patches.

I think this is why you need to regard the partitioning scheme as
something more like an equivalence class - possibly the partitioning
scheme should actually contain (or be?) an equivalence class. Suppose
this is the query:

SELECT * FROM i4 INNER JOIN i8 ON i4.x = i8.x;

...where i4 (x) is an int4 partitioning key and i8 (x) is an int8
partitioning key. It's meaningless to ask whether the result of the
join is partitioned by int4 or int8. It's partitioned by the
equivalence class that contains both i4.x and i8.x. If the result of
this join where joined to another table on either of those two
columns, a second partition-wise join would be theoretically possible.
If you insist on knowing the type of the partitioning scheme, rather
than just the opfamily, you've boxed yourself into a corner from which
there's no good escape.

Only inner join conditions have equivalence classes associated with
those. Outer join conditions create single element equivalence
classes. So, we can not associate equivalence classes as they are with
partition scheme. If we could do that, it makes life much easier since
checking whether equi-join between all partition keys exist, is simply
looking up equivalence classes that cover joining relations and find
em_member corresponding to partition keys.

It looks like we should only keep strategy, partnatts, partopfamily
and parttypcoll in PartitionScheme. A partition-wise join between two
relations would be possible if all those match. When matching
partition bounds of joining relations, we should rely on partopfamily
to give us comparison function based on the types of partition keys
being joined. In that context it looks like all the partition bound
comparision functions which accept partition key were not written
keeping this use case in mind. They will need to be rewritten to
accept strategy, partnatts, partopfamily and parttypcoll.

There's a relevant comment in 0006, build_joinrel_partition_info()
(probably that name needs to change, but I will do that once we have
settled on design)
+   /*
+    * Construct partition keys for the join.
+    *
+    * An INNER join between two partitioned relations is partition by key
+    * expressions from both the relations. For tables A and B
partitioned by a and b
+    * respectively, (A INNER JOIN B ON A.a = B.b) is partitioned by both A.a
+    * and B.b.
+    *
+    * An OUTER join like (A LEFT JOIN B ON A.a = B.b) may produce rows with
+    * B.b NULL. These rows may not fit the partitioning conditions imposed on
+    * B.b. Hence, strictly speaking, the join is not partitioned by B.b.
+    * Strictly speaking, partition keys of an OUTER join should include
+    * partition key expressions from the OUTER side only. Consider a join like
+    * (A LEFT JOIN B on (A.a = B.b) LEFT JOIN C ON B.b = C.c. If we do not
+    * include B.b as partition key expression for (AB), it prohibits us from
+    * using partition-wise join when joining (AB) with C as there is no
+    * equi-join between partition keys of joining relations. But two NULL
+    * values are never equal and no two rows from mis-matching partitions can
+    * join. Hence it's safe to include B.b as partition key expression for
+    * (AB), even though rows in (AB) are not strictly partitioned by B.b.
+    */

I think that also needs to be reviewed carefully. Partition-wise joins
may be happy including partition keys from all sides, but
partition-wise aggregates may not be, esp. when pushing complete
aggregation down to partitions. In that case, rows with NULL partition
key, which falls on nullable side of join, will be spread across
multiple partitions. Proabably, we should separate nullable and
non-nullable partition key expressions.

--
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

#108Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#107)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Apr 5, 2017 at 2:42 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Only inner join conditions have equivalence classes associated with
those. Outer join conditions create single element equivalence
classes. So, we can not associate equivalence classes as they are with
partition scheme. If we could do that, it makes life much easier since
checking whether equi-join between all partition keys exist, is simply
looking up equivalence classes that cover joining relations and find
em_member corresponding to partition keys.

OK.

It looks like we should only keep strategy, partnatts, partopfamily
and parttypcoll in PartitionScheme. A partition-wise join between two
relations would be possible if all those match.

Yes, I think so. Conceivably you could even exclude partnatts and
strategy, since there's nothing preventing a partitionwise join
between a list-partitioned table and a range-partitioned table, or
between a table range-partitioned on (a) and another range-partitioned
on (a, b), but there is probably not much benefit in trying to cover
such cases. I think it's reasonable to tell users that this is only
going to work when the partitioning strategy is the same and the join
conditions include all of the partitioning columns on both sides.

There's a relevant comment in 0006, build_joinrel_partition_info()
(probably that name needs to change, but I will do that once we have
settled on design)
+   /*
+    * Construct partition keys for the join.
+    *
+    * An INNER join between two partitioned relations is partition by key
+    * expressions from both the relations. For tables A and B
partitioned by a and b
+    * respectively, (A INNER JOIN B ON A.a = B.b) is partitioned by both A.a
+    * and B.b.
+    *
+    * An OUTER join like (A LEFT JOIN B ON A.a = B.b) may produce rows with
+    * B.b NULL. These rows may not fit the partitioning conditions imposed on
+    * B.b. Hence, strictly speaking, the join is not partitioned by B.b.
+    * Strictly speaking, partition keys of an OUTER join should include
+    * partition key expressions from the OUTER side only. Consider a join like
+    * (A LEFT JOIN B on (A.a = B.b) LEFT JOIN C ON B.b = C.c. If we do not
+    * include B.b as partition key expression for (AB), it prohibits us from
+    * using partition-wise join when joining (AB) with C as there is no
+    * equi-join between partition keys of joining relations. But two NULL
+    * values are never equal and no two rows from mis-matching partitions can
+    * join. Hence it's safe to include B.b as partition key expression for
+    * (AB), even though rows in (AB) are not strictly partitioned by B.b.
+    */

I think that also needs to be reviewed carefully.

The following passage from src/backend/optimizer/README seems highly relevant:

===
The planner's treatment of outer join reordering is based on the following
identities:

1. (A leftjoin B on (Pab)) innerjoin C on (Pac)
= (A innerjoin C on (Pac)) leftjoin B on (Pab)

where Pac is a predicate referencing A and C, etc (in this case, clearly
Pac cannot reference B, or the transformation is nonsensical).

2. (A leftjoin B on (Pab)) leftjoin C on (Pac)
= (A leftjoin C on (Pac)) leftjoin B on (Pab)

3. (A leftjoin B on (Pab)) leftjoin C on (Pbc)
= A leftjoin (B leftjoin C on (Pbc)) on (Pab)

Identity 3 only holds if predicate Pbc must fail for all-null B rows
(that is, Pbc is strict for at least one column of B). If Pbc is not
strict, the first form might produce some rows with nonnull C columns
where the second form would make those entries null.
===

In other words, I think your statement that null is never equal to
null is a bit imprecise. Somebody could certainly create an operator
that is named "=" which returns true in that case, and then they could
say, hey, two nulls are equal (when you use that operator). The
argument needs to be made in terms of the formal properties of the
operator. The relevant logic is in have_partkey_equi_join:

+               /* Skip clauses which are not equality conditions. */
+               if (rinfo->hashjoinoperator == InvalidOid &&
!rinfo->mergeopfamilies)
+                       continue;

Actually, I think the hashjoinoperator test is formally and
practically unnecessary here; lower down there is a test to see
whether the partitioning scheme's operator family is a member of
rinfo->mergeopfamilies, which will certainly fail if we got through
this test with rinfo->mergeopfamilies == NIL just on the strength of
rinfo->hashjoinoperator != InvalidOid. So you can just bail out if
rinfo->mergeopfamilies == NIL. But the underlying point here is that
the only thing you really know about the function is that it's got to
be a strategy-3 operator in some btree opclass; if that guarantees
strictness, then so be it -- but I wasn't able to find anything in the
code or documentation off-hand that supports that contention, so we
might need to think a bit more about why (or if) this is guaranteed to
be true.

Partition-wise joins
may be happy including partition keys from all sides, but
partition-wise aggregates may not be, esp. when pushing complete
aggregation down to partitions. In that case, rows with NULL partition
key, which falls on nullable side of join, will be spread across
multiple partitions. Proabably, we should separate nullable and
non-nullable partition key expressions.

I don't think I understand quite what you're getting at here. Can you
spell this out in more detail? To push an aggregate down to
partitions, you need the grouping key to match the applicable
partition key, and the partition key shouldn't allow nulls in more
than one place. Now I think your point may be that outer join
semantics could let them creep in there, e.g. SELECT b.x, sum(a.y)
FROM a LEFT JOIN b ON a.x = b.x GROUP BY 1 -- which would indeed be a
good test case for partitionwise aggregate. I'd be inclined to think
that we should just give up on partitionwise aggregate in such cases;
it's not worth trying to optimize such a weird query, at least IMHO.
(Does this sort of case ever happen with joins? I think not, as long
as the join operator is strict.)

I spent some time thinking about this patch set today and I don't see
that there's much point in committing any more of this to v10. I
think that 0001 and 0002 are probably committable or very close at
this point. However, 0001 is adding more complexity than I think is
warranted until we're actually ready to commit the feature that uses
it, and 0002 is so small that committing isn't really going to smooth
future development much. 0003-0009 are essentially all one big patch
that will have to be committed together.

--
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

#109Tom Lane
tgl@sss.pgh.pa.us
In reply to: Robert Haas (#108)
Re: Partition-wise join for join between (declaratively) partitioned tables

Robert Haas <robertmhaas@gmail.com> writes:

... But the underlying point here is that
the only thing you really know about the function is that it's got to
be a strategy-3 operator in some btree opclass; if that guarantees
strictness, then so be it -- but I wasn't able to find anything in the
code or documentation off-hand that supports that contention, so we
might need to think a bit more about why (or if) this is guaranteed to
be true.

FWIW, I do not think that follows. If you want to check that the
function is strict, check that explicitly.

It's very likely that in practice, all such functions are indeed strict,
but we don't have an assumption about that wired into the planner.

regards, tom lane

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

#110Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#108)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Apr 6, 2017 at 6:37 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Wed, Apr 5, 2017 at 2:42 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Only inner join conditions have equivalence classes associated with
those. Outer join conditions create single element equivalence
classes. So, we can not associate equivalence classes as they are with
partition scheme. If we could do that, it makes life much easier since
checking whether equi-join between all partition keys exist, is simply
looking up equivalence classes that cover joining relations and find
em_member corresponding to partition keys.

OK.

It looks like we should only keep strategy, partnatts, partopfamily
and parttypcoll in PartitionScheme. A partition-wise join between two
relations would be possible if all those match.

Yes, I think so. Conceivably you could even exclude partnatts and
strategy, since there's nothing preventing a partitionwise join
between a list-partitioned table and a range-partitioned table, or
between a table range-partitioned on (a) and another range-partitioned
on (a, b), but there is probably not much benefit in trying to cover
such cases. I think it's reasonable to tell users that this is only
going to work when the partitioning strategy is the same and the join
conditions include all of the partitioning columns on both sides.

There's a relevant comment in 0006, build_joinrel_partition_info()
(probably that name needs to change, but I will do that once we have
settled on design)
+   /*
+    * Construct partition keys for the join.
+    *
+    * An INNER join between two partitioned relations is partition by key
+    * expressions from both the relations. For tables A and B
partitioned by a and b
+    * respectively, (A INNER JOIN B ON A.a = B.b) is partitioned by both A.a
+    * and B.b.
+    *
+    * An OUTER join like (A LEFT JOIN B ON A.a = B.b) may produce rows with
+    * B.b NULL. These rows may not fit the partitioning conditions imposed on
+    * B.b. Hence, strictly speaking, the join is not partitioned by B.b.
+    * Strictly speaking, partition keys of an OUTER join should include
+    * partition key expressions from the OUTER side only. Consider a join like
+    * (A LEFT JOIN B on (A.a = B.b) LEFT JOIN C ON B.b = C.c. If we do not
+    * include B.b as partition key expression for (AB), it prohibits us from
+    * using partition-wise join when joining (AB) with C as there is no
+    * equi-join between partition keys of joining relations. But two NULL
+    * values are never equal and no two rows from mis-matching partitions can
+    * join. Hence it's safe to include B.b as partition key expression for
+    * (AB), even though rows in (AB) are not strictly partitioned by B.b.
+    */

I think that also needs to be reviewed carefully.

The following passage from src/backend/optimizer/README seems highly relevant:

===
The planner's treatment of outer join reordering is based on the following
identities:

1. (A leftjoin B on (Pab)) innerjoin C on (Pac)
= (A innerjoin C on (Pac)) leftjoin B on (Pab)

where Pac is a predicate referencing A and C, etc (in this case, clearly
Pac cannot reference B, or the transformation is nonsensical).

2. (A leftjoin B on (Pab)) leftjoin C on (Pac)
= (A leftjoin C on (Pac)) leftjoin B on (Pab)

3. (A leftjoin B on (Pab)) leftjoin C on (Pbc)
= A leftjoin (B leftjoin C on (Pbc)) on (Pab)

Identity 3 only holds if predicate Pbc must fail for all-null B rows
(that is, Pbc is strict for at least one column of B). If Pbc is not
strict, the first form might produce some rows with nonnull C columns
where the second form would make those entries null.
===

In other words, I think your statement that null is never equal to
null is a bit imprecise. Somebody could certainly create an operator
that is named "=" which returns true in that case, and then they could
say, hey, two nulls are equal (when you use that operator). The
argument needs to be made in terms of the formal properties of the
operator. The relevant logic is in have_partkey_equi_join:

+               /* Skip clauses which are not equality conditions. */
+               if (rinfo->hashjoinoperator == InvalidOid &&
!rinfo->mergeopfamilies)
+                       continue;

Actually, I think the hashjoinoperator test is formally and
practically unnecessary here; lower down there is a test to see
whether the partitioning scheme's operator family is a member of
rinfo->mergeopfamilies, which will certainly fail if we got through
this test with rinfo->mergeopfamilies == NIL just on the strength of
rinfo->hashjoinoperator != InvalidOid. So you can just bail out if
rinfo->mergeopfamilies == NIL. But the underlying point here is that
the only thing you really know about the function is that it's got to
be a strategy-3 operator in some btree opclass; if that guarantees
strictness, then so be it -- but I wasn't able to find anything in the
code or documentation off-hand that supports that contention, so we
might need to think a bit more about why (or if) this is guaranteed to
be true.

I need more time to think about this. Will get back to this soon.

Partition-wise joins
may be happy including partition keys from all sides, but
partition-wise aggregates may not be, esp. when pushing complete
aggregation down to partitions. In that case, rows with NULL partition
key, which falls on nullable side of join, will be spread across
multiple partitions. Proabably, we should separate nullable and
non-nullable partition key expressions.

I don't think I understand quite what you're getting at here. Can you
spell this out in more detail? To push an aggregate down to
partitions, you need the grouping key to match the applicable
partition key, and the partition key shouldn't allow nulls in more
than one place. Now I think your point may be that outer join
semantics could let them creep in there, e.g. SELECT b.x, sum(a.y)
FROM a LEFT JOIN b ON a.x = b.x GROUP BY 1 -- which would indeed be a
good test case for partitionwise aggregate. I'd be inclined to think
that we should just give up on partitionwise aggregate in such cases;
it's not worth trying to optimize such a weird query, at least IMHO.
(Does this sort of case ever happen with joins? I think not, as long
as the join operator is strict.)

Yes, this is the case, I am thinking about. No, it doesn't happen with join.

I spent some time thinking about this patch set today and I don't see
that there's much point in committing any more of this to v10. I
think that 0001 and 0002 are probably committable or very close at
this point. However, 0001 is adding more complexity than I think is
warranted until we're actually ready to commit the feature that uses
it, and 0002 is so small that committing isn't really going to smooth
future development much. 0003-0009 are essentially all one big patch
that will have to be committed together.

Ok. Thanks.

--
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

#111Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#106)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Apr 5, 2017 at 8:39 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Tue, Apr 4, 2017 at 10:22 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Yes, I agree. For an inner join, the partition key types need to "shrink"
and for outer join they need to be "widened". I don't know if there is a way
to know "wider" or "shorter" of two given types. We might have to implement
a method to merge partition keys to produce partition key of the join, which
may be different from either of the partition keys. So, after-all we may
have to abandon the idea of canonical partition scheme. I haven't included
this change in the attached set of patches.

I think this is why you need to regard the partitioning scheme as
something more like an equivalence class - possibly the partitioning
scheme should actually contain (or be?) an equivalence class. Suppose
this is the query:

SELECT * FROM i4 INNER JOIN i8 ON i4.x = i8.x;

...where i4 (x) is an int4 partitioning key and i8 (x) is an int8
partitioning key. It's meaningless to ask whether the result of the
join is partitioned by int4 or int8. It's partitioned by the
equivalence class that contains both i4.x and i8.x. If the result of
this join where joined to another table on either of those two
columns, a second partition-wise join would be theoretically possible.
If you insist on knowing the type of the partitioning scheme, rather
than just the opfamily, you've boxed yourself into a corner from which
there's no good escape.

When we merge partition bounds from two relations with different
partition key types, the merged partition bounds need to have some
information abound the way those constants look like e.g. their
length, structure etc. That's the reason we need to store partition
key types of merged partitioning scheme. Consider a three way join (i4
JOIN i8 ON i4.x = i8.x) JOIN i2 ON (i2.x = i.x). When we compare
partition bounds of i4 and i8, we use operators for int4 and int8. The
join i4 JOIN i8 will get partition bounds by merging those of i4 and
i8. When we come to join with i2, we need to know which operators to
use for comparing the partition bounds of the join with those of i2.

So, if the partition key types of the joining relations differ (but
they have matching partitioning schemes per strategy, natts and
operator family) the partition bounds of the join are converted to the
wider type among the partition key types of the joining tree.
Actually, as I am explained earlier we could choose a wider outer type
for an OUTER join and shorter type for inner join. This type is used
as partition key type of the join. In the above case join between i4
and i8 have its partition bounds converted to i8 (or i4) and then when
it is joined with i2 the partition bounds of the join are converted to
i8 (or i2).

--
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

#112Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#111)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Apr 18, 2017 at 6:55 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

When we merge partition bounds from two relations with different
partition key types, the merged partition bounds need to have some
information abound the way those constants look like e.g. their
length, structure etc. That's the reason we need to store partition
key types of merged partitioning scheme. Consider a three way join (i4
JOIN i8 ON i4.x = i8.x) JOIN i2 ON (i2.x = i.x). When we compare
partition bounds of i4 and i8, we use operators for int4 and int8. The
join i4 JOIN i8 will get partition bounds by merging those of i4 and
i8. When we come to join with i2, we need to know which operators to
use for comparing the partition bounds of the join with those of i2.

So, if the partition key types of the joining relations differ (but
they have matching partitioning schemes per strategy, natts and
operator family) the partition bounds of the join are converted to the
wider type among the partition key types of the joining tree.
Actually, as I am explained earlier we could choose a wider outer type
for an OUTER join and shorter type for inner join. This type is used
as partition key type of the join. In the above case join between i4
and i8 have its partition bounds converted to i8 (or i4) and then when
it is joined with i2 the partition bounds of the join are converted to
i8 (or i2).

I don't understand why you think that partition-wise join needs any
new logic here; if this were a non-partitionwise join, we'd similarly
need to use the correct operator, but the existing code handles that
just fine. If the join is performed partition-wise, it should use the
same operators that would have been used by a non-partitionwise join
between the same tables.

I think the choice of operator depends only on the column types, and
that the "width" of those types has nothing to do with it. For
example, if the user writes .WHERE A.x = B.x AND B.x = C.x, the
operator for an A/B join or a B/C join will be the one that appears in
the query; parse analysis will have identified which specific operator
is meant based on the types of the columns. If the optimizer
subsequently decides to reorder the joins and perform the A/C join
first, it will go hunt down the operator with the same strategy number
in the same operator family that takes the type of A.x on one side and
the type of C.x on the other side. No problem. A partition-wise join
between A and C will use that same operator; again, no problem.

Your example involves joining the output of a join between i4 and i8
against i2, so it seems there is some ambiguity about what the input
type should be. But, again, the planner already copes with this
problem. In fact, the join is performed either using i4.x or i8.x --
I don't know what happens, or whether it depends on other details of
the query or the plan -- and the operator which can accept that value
on one side and i2.x on the other side is the one that gets used.

--
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

#113Tom Lane
tgl@sss.pgh.pa.us
In reply to: Robert Haas (#112)
Re: Partition-wise join for join between (declaratively) partitioned tables

Robert Haas <robertmhaas@gmail.com> writes:

I don't understand why you think that partition-wise join needs any
new logic here; if this were a non-partitionwise join, we'd similarly
need to use the correct operator, but the existing code handles that
just fine. If the join is performed partition-wise, it should use the
same operators that would have been used by a non-partitionwise join
between the same tables.

More to the point, the appropriate operator was chosen by parse analysis.
The planner has *zero* flexibility as to which operator is involved.

BTW, I remain totally mystified as to what people think the semantics of
partitioning ought to be. Child columns can have a different type from
parent columns? Really? Why is this even under discussion? We don't
allow that in old-school inheritance, and I cannot imagine a rational
argument why partitioning should allow it.

regards, tom lane

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

#114Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Tom Lane (#113)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Apr 20, 2017 at 11:32 AM, Tom Lane <tgl@sss.pgh.pa.us> wrote:

BTW, I remain totally mystified as to what people think the semantics of
partitioning ought to be. Child columns can have a different type from
parent columns? Really? Why is this even under discussion? We don't
allow that in old-school inheritance, and I cannot imagine a rational
argument why partitioning should allow it.

No, we aren't doing that. We are discussing here how to represent
partition bounds of top level join and all the intermediate joins
between A, B and C which are partitioned tables with different
partition key types. We are not discussing the column types of
children, join or simple.

--
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

#115Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#112)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Apr 20, 2017 at 10:42 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Tue, Apr 18, 2017 at 6:55 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

When we merge partition bounds from two relations with different
partition key types, the merged partition bounds need to have some
information abound the way those constants look like e.g. their
length, structure etc. That's the reason we need to store partition
key types of merged partitioning scheme. Consider a three way join (i4
JOIN i8 ON i4.x = i8.x) JOIN i2 ON (i2.x = i.x). When we compare
partition bounds of i4 and i8, we use operators for int4 and int8. The
join i4 JOIN i8 will get partition bounds by merging those of i4 and
i8. When we come to join with i2, we need to know which operators to
use for comparing the partition bounds of the join with those of i2.

So, if the partition key types of the joining relations differ (but
they have matching partitioning schemes per strategy, natts and
operator family) the partition bounds of the join are converted to the
wider type among the partition key types of the joining tree.
Actually, as I am explained earlier we could choose a wider outer type
for an OUTER join and shorter type for inner join. This type is used
as partition key type of the join. In the above case join between i4
and i8 have its partition bounds converted to i8 (or i4) and then when
it is joined with i2 the partition bounds of the join are converted to
i8 (or i2).

I don't understand why you think that partition-wise join needs any
new logic here; if this were a non-partitionwise join, we'd similarly
need to use the correct operator, but the existing code handles that
just fine. If the join is performed partition-wise, it should use the
same operators that would have been used by a non-partitionwise join
between the same tables.

I think the choice of operator depends only on the column types, and
that the "width" of those types has nothing to do with it. For
example, if the user writes .WHERE A.x = B.x AND B.x = C.x, the
operator for an A/B join or a B/C join will be the one that appears in
the query; parse analysis will have identified which specific operator
is meant based on the types of the columns. If the optimizer
subsequently decides to reorder the joins and perform the A/C join
first, it will go hunt down the operator with the same strategy number
in the same operator family that takes the type of A.x on one side and
the type of C.x on the other side. No problem. A partition-wise join
between A and C will use that same operator; again, no problem.

Your example involves joining the output of a join between i4 and i8
against i2, so it seems there is some ambiguity about what the input
type should be. But, again, the planner already copes with this
problem. In fact, the join is performed either using i4.x or i8.x --
I don't know what happens, or whether it depends on other details of
the query or the plan -- and the operator which can accept that value
on one side and i2.x on the other side is the one that gets used.

I think you are confusing join condition application and partition
bounds of a join relation. What you have described above is how
operators are chosen to apply join conditions - it picks up the
correct operator from the operator family based on the column types
being used in join condition. That it can do because the columns being
joined are both present the relations being joined, irrespective of
which pair of relations is being joined. In your example, A.x, B.x and
C.x are all present on one of the sides of join irrespective of
whether the join is executed as (AB)C, A(BC) or (AC)B.

But the problem we are trying to solve here about partition bounds of
the join relation: what should be the partition bounds of AB, BC or
AC? When we compare partition bounds of and intermediate join with
other intermediate join (e.g. AB with those of C) what operator should
be used? You seem to be suggesting that we keep as many sets of
partition bounds as there are base relations participating in the join
and then use appropriate partition bounds based on the columns in the
join conditions, so that we can use the same operator as used in the
join condition. That doesn't seem to be a good option since the
partition bounds will all have same values, only differing in their
binary representation because of differences in data types. I am of
the opinion that we save a single set of partition bounds. We have to
then associate a data type with bounds to know binary representation
of partition bound datums. That datatype would be one of the partition
key types of joining relations. I may be wrong in using term "wider"
since its associated with the length of binary reprentation. But we
need some logic to coalesce the two data types based on the type of
join and key type on the outer side.

--
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

#116Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Ashutosh Bapat (#115)
Re: Partition-wise join for join between (declaratively) partitioned tables

On 2017/04/20 15:45, Ashutosh Bapat wrote:

On Thu, Apr 20, 2017 at 10:42 AM, Robert Haas <robertmhaas@gmail.com> wrote:

I don't understand why you think that partition-wise join needs any
new logic here; if this were a non-partitionwise join, we'd similarly
need to use the correct operator, but the existing code handles that
just fine. If the join is performed partition-wise, it should use the
same operators that would have been used by a non-partitionwise join
between the same tables.

I think the choice of operator depends only on the column types, and
that the "width" of those types has nothing to do with it. For
example, if the user writes .WHERE A.x = B.x AND B.x = C.x, the
operator for an A/B join or a B/C join will be the one that appears in
the query; parse analysis will have identified which specific operator
is meant based on the types of the columns. If the optimizer
subsequently decides to reorder the joins and perform the A/C join
first, it will go hunt down the operator with the same strategy number
in the same operator family that takes the type of A.x on one side and
the type of C.x on the other side. No problem. A partition-wise join
between A and C will use that same operator; again, no problem.

Your example involves joining the output of a join between i4 and i8
against i2, so it seems there is some ambiguity about what the input
type should be. But, again, the planner already copes with this
problem. In fact, the join is performed either using i4.x or i8.x --
I don't know what happens, or whether it depends on other details of
the query or the plan -- and the operator which can accept that value
on one side and i2.x on the other side is the one that gets used.

I think you are confusing join condition application and partition
bounds of a join relation. What you have described above is how
operators are chosen to apply join conditions - it picks up the
correct operator from the operator family based on the column types
being used in join condition. That it can do because the columns being
joined are both present the relations being joined, irrespective of
which pair of relations is being joined. In your example, A.x, B.x and
C.x are all present on one of the sides of join irrespective of
whether the join is executed as (AB)C, A(BC) or (AC)B.

But the problem we are trying to solve here about partition bounds of
the join relation: what should be the partition bounds of AB, BC or
AC? When we compare partition bounds of and intermediate join with
other intermediate join (e.g. AB with those of C) what operator should
be used? You seem to be suggesting that we keep as many sets of
partition bounds as there are base relations participating in the join
and then use appropriate partition bounds based on the columns in the
join conditions, so that we can use the same operator as used in the
join condition. That doesn't seem to be a good option since the
partition bounds will all have same values, only differing in their
binary representation because of differences in data types. I am of
the opinion that we save a single set of partition bounds. We have to
then associate a data type with bounds to know binary representation
of partition bound datums. That datatype would be one of the partition
key types of joining relations. I may be wrong in using term "wider"
since its associated with the length of binary reprentation. But we
need some logic to coalesce the two data types based on the type of
join and key type on the outer side.

FWIW, I think that using any one of the partition bounds of the baserels
being partitionwise-joined should suffice as the partition bound of any
combination of joins involving two or more of those baserels, as long as
the partitioning operator of each of the baserels is in the same operator
family (I guess that *is* checked somewhere in the partitionwise join
consideration flow). IOW, partopfamily[] of all of the baserels should
match and then the join clause operators involved should belong to the
same respective operator families.

ISTM, the question here is about how to derive the partitioning properties
of joinrels from those of the baserels involved. Even if the join
conditions refer to columns of different types on two sides, as long as
the partitioning and joining is known to occur using operators of
compatible semantics, I don't understand what more needs to be considered
or done. Although, I haven't studied things in enough detail to say
anything confidently about whether join being INNER or OUTER has any
bearing on the semantics of the partitioning of the joinrels in question.
IIUC, using partitioning properties to apply partitionwise join technique
at successive join levels will be affected by the OUTER considerations
similar to how they affect what levels a give EquivalenceClass clause
could be applied without causing any semantics violations. As already
mentioned upthread, it would be a good idea to have some integration of
the partitioning considerations with the equivalence class mechanism (how
ForeignKeyOptInfo contains links to ECs comes to mind).

By the way, looking at match_expr_to_partition_keys() in your latest
patch, I wonder why not use an approach similar to calling
is_indexable_operator() that is used in match_clause_to_indexcol()? Note
that is_indexable_operator() simply checks if clause->opno is in the index
key's operator family, as returned by op_in_opfamily(). Instead I see the
following:

/*
* The clause allows partition-wise join if only it uses the same
* operator family as that specified by the partition key.
*/
if (!list_member_oid(rinfo->mergeopfamilies,
part_scheme->partopfamily[ipk1]))
continue;

But maybe I am missing something.

Thanks,
Amit

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

#117Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Amit Langote (#116)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Apr 20, 2017 at 3:35 PM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

On 2017/04/20 15:45, Ashutosh Bapat wrote:

On Thu, Apr 20, 2017 at 10:42 AM, Robert Haas <robertmhaas@gmail.com> wrote:

I don't understand why you think that partition-wise join needs any
new logic here; if this were a non-partitionwise join, we'd similarly
need to use the correct operator, but the existing code handles that
just fine. If the join is performed partition-wise, it should use the
same operators that would have been used by a non-partitionwise join
between the same tables.

I think the choice of operator depends only on the column types, and
that the "width" of those types has nothing to do with it. For
example, if the user writes .WHERE A.x = B.x AND B.x = C.x, the
operator for an A/B join or a B/C join will be the one that appears in
the query; parse analysis will have identified which specific operator
is meant based on the types of the columns. If the optimizer
subsequently decides to reorder the joins and perform the A/C join
first, it will go hunt down the operator with the same strategy number
in the same operator family that takes the type of A.x on one side and
the type of C.x on the other side. No problem. A partition-wise join
between A and C will use that same operator; again, no problem.

Your example involves joining the output of a join between i4 and i8
against i2, so it seems there is some ambiguity about what the input
type should be. But, again, the planner already copes with this
problem. In fact, the join is performed either using i4.x or i8.x --
I don't know what happens, or whether it depends on other details of
the query or the plan -- and the operator which can accept that value
on one side and i2.x on the other side is the one that gets used.

I think you are confusing join condition application and partition
bounds of a join relation. What you have described above is how
operators are chosen to apply join conditions - it picks up the
correct operator from the operator family based on the column types
being used in join condition. That it can do because the columns being
joined are both present the relations being joined, irrespective of
which pair of relations is being joined. In your example, A.x, B.x and
C.x are all present on one of the sides of join irrespective of
whether the join is executed as (AB)C, A(BC) or (AC)B.

But the problem we are trying to solve here about partition bounds of
the join relation: what should be the partition bounds of AB, BC or
AC? When we compare partition bounds of and intermediate join with
other intermediate join (e.g. AB with those of C) what operator should
be used? You seem to be suggesting that we keep as many sets of
partition bounds as there are base relations participating in the join
and then use appropriate partition bounds based on the columns in the
join conditions, so that we can use the same operator as used in the
join condition. That doesn't seem to be a good option since the
partition bounds will all have same values, only differing in their
binary representation because of differences in data types. I am of
the opinion that we save a single set of partition bounds. We have to
then associate a data type with bounds to know binary representation
of partition bound datums. That datatype would be one of the partition
key types of joining relations. I may be wrong in using term "wider"
since its associated with the length of binary reprentation. But we
need some logic to coalesce the two data types based on the type of
join and key type on the outer side.

FWIW, I think that using any one of the partition bounds of the baserels
being partitionwise-joined should suffice as the partition bound of any
combination of joins involving two or more of those baserels, as long as
the partitioning operator of each of the baserels is in the same operator
family (I guess that *is* checked somewhere in the partitionwise join
consideration flow). IOW, partopfamily[] of all of the baserels should
match and then the join clause operators involved should belong to the
same respective operator families.

The partition bounds of different base rels may be different and we
have to compare them. Even we say that we join two tables with same
partition bounds using partitio-wise join, we need to make sure that
those partition bounds are indeed same, thus requiring to compare. And
to compare any datum we need to know its type.

ISTM, the question here is about how to derive the partitioning properties
of joinrels from those of the baserels involved. Even if the join
conditions refer to columns of different types on two sides, as long as
the partitioning and joining is known to occur using operators of
compatible semantics, I don't understand what more needs to be considered
or done. Although, I haven't studied things in enough detail to say
anything confidently about whether join being INNER or OUTER has any
bearing on the semantics of the partitioning of the joinrels in question.
IIUC, using partitioning properties to apply partitionwise join technique
at successive join levels will be affected by the OUTER considerations
similar to how they affect what levels a give EquivalenceClass clause
could be applied without causing any semantics violations. As already
mentioned upthread, it would be a good idea to have some integration of
the partitioning considerations with the equivalence class mechanism (how
ForeignKeyOptInfo contains links to ECs comes to mind).

This has been already discussed. I have showed earlier why equivalence
classes are not useful in this case.

--
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

#118Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#115)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Apr 20, 2017 at 8:45 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

I think you are confusing join condition application and partition
bounds of a join relation.

You're right, I misunderstood what you were talking about.

But the problem we are trying to solve here about partition bounds of
the join relation: what should be the partition bounds of AB, BC or
AC? When we compare partition bounds of and intermediate join with
other intermediate join (e.g. AB with those of C) what operator should
be used? You seem to be suggesting that we keep as many sets of
partition bounds as there are base relations participating in the join
and then use appropriate partition bounds based on the columns in the
join conditions, so that we can use the same operator as used in the
join condition. That doesn't seem to be a good option since the
partition bounds will all have same values, only differing in their
binary representation because of differences in data types.

Well, actually, I think it is a good option, as I wrote in
/messages/by-id/CA+TgmoY-LiJ+_S7OijNU_r2y=dhSj539WTqA7CaYJ-hcEcCdZg@mail.gmail.com

In that email, my principal concern was allowing partition-wise join
to succeed even with slightly different sets of partition boundaries
on the two sides of the join; in particular, if we've got A with A1 ..
A10 and B with B1 .. B10 and the DBA adds A11, I don't want
performance to tank until the DBA gets around to adding B11. Removing
the partition bounds from the PartitionScheme and storing them
per-RelOptInfo fixes that problem; the fact that it also solves this
problem of what happens when we have different data types on the two
sides looks to me like a second reason to go that way.

And there's a third reason, too, which is that the opfamily mechanism
doesn't currently provide any mechanism for reasoning about which data
types are "wider" or "narrower" in the way that you want. In general,
there's not even a reason why such a relationship has to exist;
consider two data types t1 and t2 with opclasses t1_ops and t2_ops
that are part of the same opfamily t_ops, and suppose that t1 can
represent any positive integer and t2 can represent any even integer,
or in general that each data type can represent some but not all of
the values that can be represented by the other data type. In such a
case, neither would be "wider" than the other in the sense that you
need; you essentially want to find a data type within the opfamily to
which all values of any of the types involved in the query can be cast
without error, but there is nothing today which requires such a data
type to exist, and no way to identify which one it is. In practice,
for all of the built-in opfamilies that have more than one opclass,
such a data type always exists but is not always unique -- in
particular, datetime_ops contains date_ops, timestamptz_ops, and
timestamp_ops, and either of the latter two is a plausible choice for
the "widest" data type of the three. But there's no way to figure
that out from the opfamily or opclass information we have today.

In theory, it would be possible to modify the opfamily machinery so
that every opfamily designates an optional ordering of types from
"narrowest" to "widest", such that saying t1 is-narrower-than t2 is a
guarantee that every value of type t1 can be cast without error to a
value of type t2. But I think that's a bad plan. It means that every
opfamily created by either the core code or some extension now needs
to worry about annotating the opclass with this new information, and
we have to add to core the SQL syntax and supporting code to make that
work. If it were implementing a valuable feature which could not
practically be implemented without extending the opfamily machinery,
then I guess that's what we'd have to suck it up and incur that
complexity, but in this case it does not appear necessary. Storing
the partition bounds per-RelOptInfo makes this problem -- and a few
others -- go away.

--
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

#119Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#118)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Apr 21, 2017 at 1:34 AM, Robert Haas <robertmhaas@gmail.com> wrote:

You seem to be suggesting that we keep as many sets of
partition bounds as there are base relations participating in the join
and then use appropriate partition bounds based on the columns in the
join conditions, so that we can use the same operator as used in the
join condition. That doesn't seem to be a good option since the
partition bounds will all have same values, only differing in their
binary representation because of differences in data types.

Well, actually, I think it is a good option, as I wrote in
/messages/by-id/CA+TgmoY-LiJ+_S7OijNU_r2y=dhSj539WTqA7CaYJ-hcEcCdZg@mail.gmail.com

I guess, you are now confusing between partition bounds for a join
relation and partition bounds of base relation. Above paragraph is
about partition bounds of a join relation. I have already agreed that
we need to store partition bounds in RelOptInfo. For base relation
this is trivial; its RelOptInfo has to store partition bounds as
stored in the partition descriptor of corresponding partitioned table.
I am talking about partition bounds of a join relation. See below for
more explanation.

In that email, my principal concern was allowing partition-wise join
to succeed even with slightly different sets of partition boundaries
on the two sides of the join; in particular, if we've got A with A1 ..
A10 and B with B1 .. B10 and the DBA adds A11, I don't want
performance to tank until the DBA gets around to adding B11. Removing
the partition bounds from the PartitionScheme and storing them
per-RelOptInfo fixes that problem;

We have an agreement on this.

the fact that it also solves this
problem of what happens when we have different data types on the two
sides looks to me like a second reason to go that way.

I don't see how is that fixed. For a join relation we need to come up
with one set of partition bounds by merging partition bounds of the
joining relation and in order to understand how to interpret the
datums in the partition bounds, we need to associate data types. The
question is which data type we should use if the relations being
joined have different data types associated with their respective
partition bounds.

Or are you saying that we don't need to associate data type with
merged partition bounds? In that case, I don't know how do we compare
the partition bounds of two relations?

In your example, A has partition key of type int8, has bound datums
X1.. X10. B has partition key of type int4 and has bounds datums X1 ..
X11. C has partition key type int2 and bound datums X1 .. X12. The
binary representation of X's is going to differ between A, B and C
although each Xk for A, B and C is equal, wherever exists. Join
between A and B will have merged bound datums X1 .. X10 (and X11
depending upon the join type). In order to match bounds of AB with C,
we need to know the data type of bounds of AB, so that we can choose
appropriate equality operator. The question is what should we choose
as data type of partition bounds of AB, int8 or int4. This is
different from applying join conditions between AB and C, which can
choose the right opfamily operator based on the join conditions.

And there's a third reason, too, which is that the opfamily mechanism
doesn't currently provide any mechanism for reasoning about which data
types are "wider" or "narrower" in the way that you want. In general,
there's not even a reason why such a relationship has to exist;
consider two data types t1 and t2 with opclasses t1_ops and t2_ops
that are part of the same opfamily t_ops, and suppose that t1 can
represent any positive integer and t2 can represent any even integer,
or in general that each data type can represent some but not all of
the values that can be represented by the other data type. In such a
case, neither would be "wider" than the other in the sense that you
need; you essentially want to find a data type within the opfamily to
which all values of any of the types involved in the query can be cast
without error, but there is nothing today which requires such a data
type to exist, and no way to identify which one it is. In practice,
for all of the built-in opfamilies that have more than one opclass,
such a data type always exists but is not always unique -- in
particular, datetime_ops contains date_ops, timestamptz_ops, and
timestamp_ops, and either of the latter two is a plausible choice for
the "widest" data type of the three. But there's no way to figure
that out from the opfamily or opclass information we have today.

In theory, it would be possible to modify the opfamily machinery so
that every opfamily designates an optional ordering of types from
"narrowest" to "widest", such that saying t1 is-narrower-than t2 is a
guarantee that every value of type t1 can be cast without error to a
value of type t2. But I think that's a bad plan. It means that every
opfamily created by either the core code or some extension now needs
to worry about annotating the opclass with this new information, and
we have to add to core the SQL syntax and supporting code to make that
work. If it were implementing a valuable feature which could not
practically be implemented without extending the opfamily machinery,
then I guess that's what we'd have to suck it up and incur that
complexity, but in this case it does not appear necessary. Storing
the partition bounds per-RelOptInfo makes this problem -- and a few
others -- go away.

This seems to suggest that we can not come up with merged bounds for
join if the partition key types of joining relations differ.

--
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

#120Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Ashutosh Bapat (#119)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

Here's an updated patch set

0001-Refactor-adjust_appendrel_attrs-adjust_appendrel_att.patch
0002-Refactor-calc_nestloop_required_outer-and-allow_star.patch
These are same as earlier patch set.

0003-Refactor-partition_bounds_equal-to-be-used-without-P.patch
0004-Modify-bound-comparision-functions-to-accept-members.patch
These are new patches to refactor partition bound comparison functions
without passing partition key directly. Changes in the first patch are
being used in this set but the second patch will be useful for more
generic bound matching.

0005-Multi-level-partitioned-table-expansion.patch
This is same as old set with minor changes. I have moved it ahead of
the other patches as we discussed offline.

0006-Canonical-partition-scheme.patch
Partition bounds are no more part of partition scheme. They appear in
RelOptInfo. We are discussing the data type handling of partition
bounds for join relation. So, I still have partopcintype in partition
scheme. There is one change though. From
ComputePartitionAttrs()->GetDefaultOpClass()/ResolveOpClass(), I
gather that partopcintype is the type used for comparison of partition
bounds instead of parttypid. When they are different they are binary
compatible. So, I have saved partopcintype in PartitionScheme instead
of parttypid and parttypmod.

0007-Canonical-partitioning-scheme-for-multi-level-partit.patch
What was earlier mult-level partition-wise join support patch is now
broken into set of patches and goes with corresponding patch for
single-level partition-wise join patch. The idea is if we agree on
changes for multi-level partitioning support and want to commit it
before partition-wise join support, we can squash those pairs into
one. This also associates the multi-level support changes with
corresponding changes for single-level support. That might be easier
to review.

0008-In-add_paths_to_append_rel-get-partitioned_rels-for-.patch
No changes in this patch.

0009-Partition-wise-join-implementation.patch
The patch adds build_joinrel_partition_bounds() to match the partition
bounds of the relations being joined. This function is called from
try_partition_wise_join(). This function is also responsible for
creating the pairs of matching partitions. When we come to support
partition-wise joins for unequal number of partitions, this function
would change without changing rest of the code.

0010-Multi-level-partition-wise-join-implementation.patch
multi-level support

0011-Adjust-join-related-to-code-to-accept-child-relation.patch
No changes to this patch.

0012-Fix-ConvertRowtypeExpr-refs-in-join-targetlist-and-q.patch
Fixes a crash with mult-level partitioning reported by Rajkumar. Fixes
set_plan_refs code for nested ConvertRowtypeExprs corresponding to
multiple levels of partitions.

0013-Parameterized-path-fixes.patch
No changes to this patch.

0014-Reparameterize-path-across-multiple-levels-of-partit.patch
Multi-level support changes for 0013

0015-Partition-wise-join-tests.patch
0016-Multi-level-partition-wise-join-tests.patch
Added the testcases reported by Rajkumar.

On Fri, Apr 21, 2017 at 12:11 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

On Fri, Apr 21, 2017 at 1:34 AM, Robert Haas <robertmhaas@gmail.com> wrote:

You seem to be suggesting that we keep as many sets of
partition bounds as there are base relations participating in the join
and then use appropriate partition bounds based on the columns in the
join conditions, so that we can use the same operator as used in the
join condition. That doesn't seem to be a good option since the
partition bounds will all have same values, only differing in their
binary representation because of differences in data types.

Well, actually, I think it is a good option, as I wrote in
/messages/by-id/CA+TgmoY-LiJ+_S7OijNU_r2y=dhSj539WTqA7CaYJ-hcEcCdZg@mail.gmail.com

I guess, you are now confusing between partition bounds for a join
relation and partition bounds of base relation. Above paragraph is
about partition bounds of a join relation. I have already agreed that
we need to store partition bounds in RelOptInfo. For base relation
this is trivial; its RelOptInfo has to store partition bounds as
stored in the partition descriptor of corresponding partitioned table.
I am talking about partition bounds of a join relation. See below for
more explanation.

In that email, my principal concern was allowing partition-wise join
to succeed even with slightly different sets of partition boundaries
on the two sides of the join; in particular, if we've got A with A1 ..
A10 and B with B1 .. B10 and the DBA adds A11, I don't want
performance to tank until the DBA gets around to adding B11. Removing
the partition bounds from the PartitionScheme and storing them
per-RelOptInfo fixes that problem;

We have an agreement on this.

the fact that it also solves this
problem of what happens when we have different data types on the two
sides looks to me like a second reason to go that way.

I don't see how is that fixed. For a join relation we need to come up
with one set of partition bounds by merging partition bounds of the
joining relation and in order to understand how to interpret the
datums in the partition bounds, we need to associate data types. The
question is which data type we should use if the relations being
joined have different data types associated with their respective
partition bounds.

Or are you saying that we don't need to associate data type with
merged partition bounds? In that case, I don't know how do we compare
the partition bounds of two relations?

In your example, A has partition key of type int8, has bound datums
X1.. X10. B has partition key of type int4 and has bounds datums X1 ..
X11. C has partition key type int2 and bound datums X1 .. X12. The
binary representation of X's is going to differ between A, B and C
although each Xk for A, B and C is equal, wherever exists. Join
between A and B will have merged bound datums X1 .. X10 (and X11
depending upon the join type). In order to match bounds of AB with C,
we need to know the data type of bounds of AB, so that we can choose
appropriate equality operator. The question is what should we choose
as data type of partition bounds of AB, int8 or int4. This is
different from applying join conditions between AB and C, which can
choose the right opfamily operator based on the join conditions.

And there's a third reason, too, which is that the opfamily mechanism
doesn't currently provide any mechanism for reasoning about which data
types are "wider" or "narrower" in the way that you want. In general,
there's not even a reason why such a relationship has to exist;
consider two data types t1 and t2 with opclasses t1_ops and t2_ops
that are part of the same opfamily t_ops, and suppose that t1 can
represent any positive integer and t2 can represent any even integer,
or in general that each data type can represent some but not all of
the values that can be represented by the other data type. In such a
case, neither would be "wider" than the other in the sense that you
need; you essentially want to find a data type within the opfamily to
which all values of any of the types involved in the query can be cast
without error, but there is nothing today which requires such a data
type to exist, and no way to identify which one it is. In practice,
for all of the built-in opfamilies that have more than one opclass,
such a data type always exists but is not always unique -- in
particular, datetime_ops contains date_ops, timestamptz_ops, and
timestamp_ops, and either of the latter two is a plausible choice for
the "widest" data type of the three. But there's no way to figure
that out from the opfamily or opclass information we have today.

In theory, it would be possible to modify the opfamily machinery so
that every opfamily designates an optional ordering of types from
"narrowest" to "widest", such that saying t1 is-narrower-than t2 is a
guarantee that every value of type t1 can be cast without error to a
value of type t2. But I think that's a bad plan. It means that every
opfamily created by either the core code or some extension now needs
to worry about annotating the opclass with this new information, and
we have to add to core the SQL syntax and supporting code to make that
work. If it were implementing a valuable feature which could not
practically be implemented without extending the opfamily machinery,
then I guess that's what we'd have to suck it up and incur that
complexity, but in this case it does not appear necessary. Storing
the partition bounds per-RelOptInfo makes this problem -- and a few
others -- go away.

This seems to suggest that we can not come up with merged bounds for
join if the partition key types of joining relations differ.

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

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

Attachments:

pg_dp_join_patches_v18.zipapplication/zip; name=pg_dp_join_patches_v18.zipDownload
PK
|��Jpatches/UX��X��X�PK|��JGpatches/0001-Refactor-adjust_appendrel_attrs-adjust_appendrel_att.patchUX��X��X��<ms�6���_��3�����jI�9S7q���M���ps��H�bB�*I������@)�I�\3�-���b���X�UmXo��F��p<�9�h�[M���NW�q�9���7����5
�{�e�	����l����W��)�H��4J��gg�������]���x��x�	��]7�<�^:)�^�����_2@��<����{����#w�����7/^�^���/6�+�M��9��]�.����^�����q�h�V���f�~��x�h2'����������
�L�?\E�f���&�CkLsk�fi���	������	[�
L�a0�I}`2�������<+{�|�m��~xp\
O���W�+Q�'���������-7�c�l�����E6�f�b�.����@��!��]�>�������%�(P$th,|��.{�����Q�{�;��!��*<wnoc~K�;���������:��1���El��M�t���O�s���&�`�<��%L���:���.��[g�8Qe���a�}��\B���5���{��+�������!Kx�����[�����lf�O�*QG��i������g����o�#`|�A]�qm	wb��5��5���&�7[I�c�g�dM@�����8�H�d��v��w�f�d��t��%����
��u��~^���n?L#�����]x��l�N�j+���t:Kb�t���@
��6�7��y|�N�N�.c���v���� ����Ip
8!�yL���\	���(V����.�������@�N�O��9	�3&��a~�;�KC�L�#�a�4��jD�Ok��M`X��	o�g����NxL�j��6�
��.t��>b�:�[�sZO��zp�������r�r�]g����3����F#���3Z��������F}�)~�0G�#�K-��CF���(�a{
�nS�������\$���Ur��Q�,Ic�%�f�A���m���js�9X��5��Q���pw����?���	&�,L�6_�m�$Y���9��Q��t'IH�b��x��G��D.ln�^��h6��)A�+���x��!c�6{���&���;���������f�/����q����/�C[M"��
���4������9U�#ub�������;QWD�T"���d-�����`��}�q1L�L������.���yN��v�?h�:�tg��79�u��i�;�Qm�a�:�2h���`�^�~OQWo9h%l�T��>�QO��V��\�_��y}9_�/i��J�D�c� ��p��\�B�!��JS#���)�O�Diz���b:�id�	�4�4�.D�(�?�l1Owq(��G_���~���C��!L\���/���%�<U���r�K��k`����x�7`ZBj������212����n ��-��]��D�A�����(m��Z�e�:��m�)����p��z���?Y�2+�A�2A����3����dQ~�����x_�M��b��1i���d�7���'�>K�v1�� l�^B�E������v������X=E�������,t����u��X�L���]=��@bo����V��1���%�����5H����8�3ynC:�{��o��Z�Z��+��(u��/�t<�vg�p8�;Q��"0��x�^
~��Z,}�r��};7�����=�}6����=
����P%�������/U~B��NI�	f�.��$`�[E[��(�
�nmV�OE�6�=�7��(<x�����,��� `~Hc�.iPP��M��*W+v�l���EEM�s�*�8�b�L�r�t�C�%?�l9�\�`9�<2��!����]��E�����{��Q/V'�I�kL1��8}Vd���42!���D�K�@���0�bh�`��L�:�����d��/"�y�@%���w��p����JL�o�y�~ab6W^p�MV���5���$����"��z8=��(�pM������i�D���5s�U	���\s.���������F���-0�f���(��r�Dc0-�������Ituyx�{�A�H�0�y*jM�_8	��YaT����J`
�|�~���g�P������lO�E.x6�wwGE�P���  �JC����
i��_mH@��yXR����]��@ZV~!�� `�R�I8bd��wY��m�$���]�[���j:`�v -�;�l#j��Q X+��+B���������Ez �Q;�z,'��I�Y������;��|8��m�r^P���&h����	SR8�
��A�����'�v[Lz��A`2���g�����PF�u9���Y_^w�_qJP(�8�X�5�'q��)���%�f��A�S�=��(���=��L�{���^FUx<�����'���6H�����7�p"_����.p�������y800@w��!�'R5���n�����7o�������?��p9��|����gI1�5\�y�X�gin��U�,�{���_55��6e2�S�_9G>E�	����W�gFq������y�����eq�p���"����r����
������\h�����*�.�2_��oJ�N��M3���%��d�n�|��'_�mY������;O������X�X���$>Ae�,6jM*��,� D)\8�i_n�SH��=-I���K������
 �:���jO+O��3*�#����d2�
Z���/��H�8��L%3w�^�����i*�c}"����@��^Uf��c[k
_u���0:b�����>�J��n�Hz�F��Z��f�#f`�����w�
�'.�e��+Q�tU!�����LmW4�1��!`���#��x�^�j���&������I��k��Oy8��xdz��|�������e<;(&�H��H��/
�&�����^SD��\@��m#���Y���R)�i�����1���=���(�z:6�%Q�� 6��fT[�e���g��V+���5iQ�"����6�����]��4W��A!��c]���G�c
��E��}����{�5r(������b���hh�f0�A
v ;���I���\�g�J�����X�6�9!N��jO��5P�uEo��Hc�@Kg��c�����q�~'�U�%`U�>����^MZC���n"?f�������?��pV�zrfF(��?<^�G�3�A���Q,At��J!�JM K�FQ��G�NKd�Me@������2@'H���!�[<��C�I
��)�=:`q��T��p��k���G�'���9\a�:�������8��T��e>����t����A4w��cV���YVa:V��r'�>jY�$��eNu��c�T�U91{��������)2���Xbj�Rn\�c�������0�x���v�;��8�@<����].[�7�U?���D�j���>Ef��=|th�E�v����e������	05G��a<�s��2n�x<����~�|]>�'t�]�a��yI���!r\�`<q��FX8��"�'�I�FxG�rT�$�p�0��G��wi�������Z��?	�HU��<�2O��#��e�u�n�%�v�zp�']��:8/7��}��%�9c3���P��sYU+����s�a������2yq��P�e����F�Z��9� ���:�^�P�_�TqL�X�2��C��5C3��#5��J�U��
���}q����ld2��~VGp
0���&RMumD�U�{��!MY?�D��d��+���"e}�?�`la�^���RQuR���%��u��>�f��J���/���)�^	��BC"�'"���f��>^S�x�x���3{2��������m,Z,B�~����~��~,���,8�-��r{^tO����3�����
xm@����l��[@��~�������	|�e~��59b����H�����-�k��+]�$c��<�?-�)�h7>|���8����L��y@�TC�������-�G��n�j���o�����s�S�O����Zm�U�����a����8n��@�:��F��������"�>��/�������E��I����u��l���n��G���{>����,��w.�E�-%��_�=V�.r��}���$�9`j��@��8
QoP��O57��N7����t�;Q�*�h�`&�BJP�����]��uP��t�6�HLE'�tt��n},?���"����|M�H~���p����]������h�~�F��^�,~�x	�^x��j�����'y0�Q���.lA�~�~pi�(-H-��vs�(<���>�F<�����~����+d�i2�Z!��w�� ���5'����M��a�a�be)]u�G��`�����a���	��#!U��82'�`Ej���R�}�b��=��F���k�����r���H-�E��g�8Q����������o����	\i=,��CJ��k3qW��o�aU�5�����5�_g>�l�#���*X��;z�{����q#�Q�D��NFpN�?7%)�ta����\'s���/&\���I�q�v�,�&n��������Z���u,
9��m���"7O��G���U����/$��B����E
�����:Q�R\u����&���7���]��xoZ��D��KP�21��X{,����d�\�P��m��)�#/������H/4����!�x���zW18�z���
�y�T�g�������C���Q���p���Z�W�������rW&K�D�(�QeJ\�M��/n�{��b��IG����%�A�=��R��B8��|��,���b��w��=�b���U��]F�����f�(��O��+���lv0u��:�p�����4]'ws2����	$�� 9�!CP�����b�R�*.���Sn��7_y����9C�%q`�PG;+dlT	�" ���>C�3��YOh�`:�5B��4B�',v�e�����J����:u��6���cC���W�����.p���vp!���E�Ig�(J��[d}��I�L����|&mq	z8������#-B����b�Z���.M^�3�K�c5����b��B���H��l��a�lS���*V=
�����q��n5�_r���v�X\����Ia�I�b��� `�\,�Oh�xfc��6���;kE^E��	�(�&(��?.T�SB�tM�[��Y�T����|�<�d�lN[�s���@�V\S���t,OuD�c��LpL:�]���W�j���*�0:��%������<
"c��d�N&�nw����w��D�����f?�c��_T�/�cL�q�HJC��ig`Y7]U�����:� >g�7���}y�x��SG=�u���j1Xz��Y��{>�����DpP%�r��?�g�6Urx����q�ELI��u���)������Cw���F�_q��u���q]�������}�����k.K�*|��t9���s�����rW���n~y-,,��w'�YwlY�PKvQ���SPK|��JGpatches/0002-Refactor-calc_nestloop_required_outer-and-allow_star.patchUX��X��X��Xms�6�,����V�E��%��]3I�d���]�N>��p �P���M�{w���;��q<�v��g�]��V%0v���y2O�f,��'gY�]\�y|g�i��O��O����� 9�8�t0��$x�f.��Y�V�%��*f�������.-�����(S���gf�%Y�x�4�����2��s8�g�q�K=��g������g/!�������`�U2Vd���JU������y�j�nxL���P��X�S�-y�������(>.����ef��XU��qm������<�]r��f%G��wF���
�������Zq��������sN
���(x%��,'��CKJ�v�Ar�t��Fde��,���y��O_�%��D�I�%�m��+�����`�Art�Y
�V������"��TU*c���;
z���>�3���U������W�K.3�����7���M���/�����00:�Yv�e>V�����1I����&��LeO�}�Y��(�9�r�XJ�X��BfE���Ni	�!M�9M�9��(�A�1y�s,�)�H��� ����A.�+a��v����!s�f��Y�DQ�&�E�@�g�)E��;x����	���t��'�u�����8��/"`��`Rr�
���Rv�1���a�
�1�+U���6�=����!;&q�o-	RuKG�I���)R�8m-���'(b��m������E?��������~�������?::��Pg�s��Wl��/�N��1��cxe�G��x��G�P�vf�����F��M�J�hmA*K1��w�H��F�h�O������ZK�K���V
;GuCw�}��N�RI^V�6%�sM��_�s��u���;�o�}��%�@^����Q|���	����u�������6�������}�L�	I�T�s�������z{�Y����FM��p{���i����CZ�i���C��#��=���pHo}JN����S{1��^)j�$��$_tQL��������D|���Q2�DOg_�h�#W���-���B�kjh���������P������<��]ol�����B	Cupr�f�����o�mhz�kyrs�V���T�ao,`��G�����l5s�fejT�3�:�
�<��/\�:2����|q�Kv�����V�I�����[���&������B� /����zr��{��g
������9;��T�q� ]��T�/�%r�k��^��^b0��g������'��3�|�1��(N��Ng�g���3}Y"���b���qM�9�R�����&crQ@��1�n����nO����P��wd�D0!(�����9�������|��3V����k�9ha�$���.����Qi���9���C�7���f��M� �e{�gzA7pO\>��hu#��#{�����{�&��c����������L4t����d����O����81�������\�Tu��w������/g�8�����[�
�����%��C��6�(��!����[����1c�

N�f!�g�vz6>$;���\��9�L��T���]b7����%���!�
��I�[��������S��__��D��w�NrW��}�'n�
����
x~�HC��E����(b3~:����}�����9�N&�:zIbB
������71w����E����cV��Z ���J���M�a�IC|�Z
��0��'r<�.����C����hf#����� ����hPK����:PK|��JGpatches/0003-Refactor-partition_bounds_equal-to-be-used-without-P.patchUX��X��X��W[S�F~f��K�I��e(��&��)�����d��je/��+� ���������[Z��w�sv}�g
�~������A�����v�?�#��|��� ��{�������v���Q����9��Y1.uV���rM?�����[�j�OsU�(�D6y��s-�iXJ��t6�a���8w�����o���������_�}��{��J�\�,�)���*Ko��L��F�U��V'�Q(��;zPz�����G������2���D�����r��5m���x>*'���J�I�����{�&�b�X����L�u����J��e�f���^�9����V���M"+\���H�c$��p%|��3�����������c��x�=R�bD���EZh�����%M�,D&3���+m��B��6b�d��`L�$�����Y��R�2��v@�2��!����
'y:�I�����+(������jYk:��J�dq��[S�T�� �M
k~�����!Zq������u]FE.Z!w2�Z�k�d����'�������\wK��*)��r�`��@�*u�R%��2�O8/�V
��lc��d���*��m��f��x����[g,RqL�;��RV�KL���N�uz�����y�A�^�k�{�C2{xz��������9���UU�w�J���Qj��W�����i�vH�|�$�g�ZOY���[�9�a��

b^�\��Aa����Y�3�����+���'�s��G����4����'�;��h�t������_X�;:0�*c�'��{p`�[��N�}B����t�,��Z�Yg�m�M��Z
:�ln���
8�6��r�Fcm'&�F�����2Y�T}������W�t���O�y���3@�����d�~�Q�u9�(>T3������2������
c��m,�|�f��l������o`Agc��]�`�#A�yJ�m���z��I���m���N1G�B����Psc`���?�1A�����6�����l�d��F�����M]�M
������������������l����7Wg�?���H���j ]#^��
&��,W,L ��;&}��P5N���=��D|�B��;�lO����~��5U��G����k����1�	�����,x��,-]��U��m��,�m��`�u�p�b���Aue`9���,��V���}�?m�&�L�m1��3@��D��n�e������9���%���2���f���2x�H���o�"���:��l/)jl-���~�������M������u�e�3�����x�����^<!T���'�
;NK���
[0b���T>�l�Y��2� L����B[�C�Y$�ae����g��^�6������2��!���w���PK���$y�PK|��JGpatches/0004-Modify-bound-comparision-functions-to-accept-members.patchUX��X��X��X�o���Y�+�����H�-��8~�����v{(�@�c)�L�
I�g����.��D��E�$w���3�7^�&��OT�b�2����9����L���f���i��0�g��[�u�2��)�*]��9���m��+x�o�^��s������X�	��YF���o�3=bs��Y2�38�(
e���@���(��"�n�_����������2��������O`�[���Q���[���&���4�&�5[,%�m���C���$�"]�I�\��\o�-97��A�,��)K���dA� `_�N�,�d��%��	�C��t����9���:W�������p���H�������F���{<��98������0N���L��>eJ'F+��A�`�9�b�x&:�������=I*�'u�]	����y�<�o�����~z��	� B�S����`�����-	Y�8^��e���`1���nK��*6t�K'�_��Q�Br<���!3tK�����0����C:t�.����������eu~�:��NA{�< �X�lu�yC�0�:�@J^K]}C|���^��l�{l�� H�
������^��A�X	��PF$�.��#p�n�!�4��i��RW�=#��\U�+<�1��mS���]UY���J��	O�"T�b�.=��6
��
a	M�KL��+,*��K��dl�JWZ���8���
dU�f`,�Th�����F�����Q��fZ=����#g����?��a�U���s'���������A����p�=|�Z���B�`1����	�Rd�L�oT6B����b�{P���:��P�Q�2W��_x�qQ����<@�1����m�p���o1�W[�m�p<\(86OU�n��,���kQ�b�M���2���
3��D�)I���
��d�,�����P��"�}��,~L�[�r���;���1(-��}t����>�Y���9��+,������7��������M
�F��������������N>\�-���t~z�8;��������j���`.�����Z����aY�C�����^����x(O0N�c�/!���4!"����4������#�14���I!�����(����Q���!��hx�����P3,�}�z�.�[J�:Y�"�
6����*�P��o������L���<���R*�Vd�8!�>���
D�������$��}M�X�t^^LT%��xy�SSEH�5v�4v�4v����n]�����8@�5\�d�?�S��tp\9^C�8��P,,/+���W:7��< c��Y��!����@��6^��q��+����EI$�3S����:��s��`.�Y��|r>��psr��|qvr���<��8�5�������=�����9*'L�@
t��]��E��f�������"��-��(����u��My�Na�o�Ca>����L��U�c�~��H�l���BN���^C�`�(Z�\�(4������,�����Jm^����q�M@+�>G�v�� �`���
�S��at-��I�.1������+��n���X�U�D��HV���>�@�r�����^
1����J8�Y�U����J���&�R0~�RMN@��p;g���w"<)?	���q5�����<���J-����DNj6S�R7|<��������������.o����TJ�T��Rbb��VJ�Ve^)e�4 o���v3�-���]
�����T���jD��PV�ER��qqI�JAWwE|���,_%��yS�lw�~�D�4��6����1K,u;c�9N�j�i�ro������ta�Ngn�?����Z3f�R�(/?���Fe�R����2��c�@�)�F-
^g��O@�����o�h��d���?b����~�da*���$.��/��q@�����+���ltL���?wKr��X�x8(�5����[�G�kL��t}�����4�A;*�+UV�$�d��d��K�����x�c�`�5����+�W��I�Mz��H��PK��FL�PK|��J:patches/0005-Multi-level-partitioned-table-expansion.patchUX��X��X��<ks�����_�vg�e�,�q�['q��&qj;����h(��P�JRq�m���@�/IN���3����p��98��(\
�����x4��r��
�������X���{��G����P�	q-W�$z�'�?1����W��8��$����������N��e��hy�t�]'\>k���D������;h{������;�������I��_�����I��������x�/?I_��(�/�+{�K!?�� �/�f�E�I�v�y�8R,<�����L� �PDr���"Y�f�!I��K/��q�G��+�a""�E�g���+�_�0�0������r��6�'�w�x�N��|��@B�G�#�����
���\Z
`&���������v�q���(�����X�vx�\��=���
����?�M��qd��t�:��-}�yQ��B>;�:��p'�H"�����X���!
�B�@W�j_y��+0��~��	�6�7��\J�����B�]B�8s�������zQ���
0���'/\�>������o�J��b �-����[��I�f"m�N����%��gd�#$������*
W����$����=��r��hi��PDS�����6���i2��u�Q�����-��-<F���$�p��V����^���K�A^rB ��=�����c-�;�c��^���x+����^Y*���5p��F�<�X1�P"	<f	,���a�1��������;a�Vs��$���a�Pc"Da�R��1P9��v��e5E9S���~�o����pM^������K�?���!j/0@���I�T�����"�����K�
��%2N"9�@�@���J�@���6�q��������w��h�GG��@;�K������J����A_�@f�b�5Au�f����j�`GOw�����BN��h`w����'���^o<!Uw��	H�}�V��3m�y$~��)��
|��*�Diw�}S�
 �6���j>I�V�����o�<<k@f���j8��l�0�`C;�q��I�������6�I�a��&��&��Z���Q9�}�b{M�@������*��H�Y�bo���$~bG�}7�ar��n��?���L���)t0���q��6�0Y�f��It��Y�7G���hx3��)����_��%�o���|N�=W��/��L��]���{8eq(}l8a*-aU���>��������������9`zl`������6h�zS����&T,@�I
�0U�y���-����h����7�0zm�S{�%cE�,�<;��u�{���Ze�o�k���$�`�[�T��;I��4D������=�
����l��H����s&�06����������p;d���B�G�9��.�����)��#bdH�it����������������/'7g�_���l���f��������"�B�DO��;%6��d������sg������1��T���G�nw`�Fcy��J�mU�YWR���Nl,�wp��u�+g�C�}{Adb?�%xN�$�8������
|�(\��_&�������i���@\F=�
���oUvc&�u�/�|TXK�O �a��K/�}"�j�5
�i($`l�1��Q��x=�XmQ��������b!����~.L~F�������m$m�z�cp��v�L�����_*��[F�+�"�Ct?g`9�4c�x�x���>zLQ����8��l�:�;�Aw4(�{�)�O!���?��w\��z��'h������u�]n��U������B$�1R\[���?;���
EvH�4@9�)�\���E6Q���,b9J��+���> ��}�bO��e�����6FL��
�;��)�$�E��0��/\5�@���B�A���O��h4@���0eo��Ta��>:�������Qx��_��
�)`���y�cv�]q���\��4��W���J��+14�/�5�����!dUP�AX3PM`x)���K�S��7b�SEv�I�&3uwCA-j4rSl���/*'=����!�Ai��FC��K`��#E�����@���������1�<�$���-��\�9:k�/����\�Wo�^�5X��IZSJk`��L����������w�W�CLhx��<)?`*�]u�1C�-��;�����J�z�I� �\�8�I�3$@���(p����#�/�������*T����,��#�J8���M�P���X;x����=���������#����@iw�\�Q���\���RD�2�t����8}'E��6���_�o�r��Mh&2K����"�X����_������N������XY�SS*���cd'\�i� }��� ����#���u��>�5�77s��������'~�5S�h�7Dn��{^=�v�)���5��.cX���v�;����
&A����d����^y�0�n���=mC�<L;�Z���m��.�	��>)�c���d����W�Vsa�%���<�S��f��R����D�����/����W�W���������g7��?�]�N/t�O��x��$_[�J�,����6hE
�H���@Nlb$�2�N�����F6�0(S����$/����������dw�ZGE�(^�
z��<�j���F�E�,�
;/�_�C��IM^����72��-��K+�%�e�����a�	�]'\��to ��l��k����\�f�B�Q��yTv*� �}���k�����*:�Y����h��>�
�}�����(��I���g�d�D�=�^��|#� `c��*�.�cMh�}r~��w��l�m>�
������(��f�<���s/��38[��������@,�!��]]�},5g�aV����k�F-�zp�9�.=��!�������z������ =������URUe���0
�L�gJ��3D��0KL�f��4*rz� ��x(�*++���x�rr������B���q���Ao�I��kF�@4�E�y���i���cr�v������(Y������U���O�W�h=�a@5��;R&��HPG�-�N<�C�����K���,"�N����8L�*����AL��c����,���W+�X��Rh��*02SI!�9�V�)#8
�#�y.�)p�jN�i������+�7�_���3���Is����?��Q+X{��ep��b<���(�
�6��7{]��l�_+��YX4��9�
���k����0\��+E��%w��mR����N��/�T���u|��:�w��M���D��h�f����������1�Z�A]�)O��*(�l@#��!(�yJD��-��� �Y��\(�
�k�(34���g������#������B�`
4��e�~������W�W�����:��
Sk��,H$6��������XI����|o�	�@����xL<z$J���@#��G�n����a�2d�B�<�tF����;�d���r%��
SK�B#���H����9'E�c)�SL�q!��a�:b�h���)�1���29��7Cy��/����J�d9X����N�^�����KmF��q�Q�3B��V\Qy��,`9��T��z��enOf�2����j�P8�V�y��6SN���H����v[�������D)������t��"������	
��r����8��S[�
u�K�ol�S�]U�*g�S&��^�X!n�!X]�^���S�����|�R�*kW���`U���#;�����P��?	��^��4=����T�q�V���l����4�.�	��yNQ�H����.Z��� ���Z�;Q��U�8��i��)�|����G�r01HF�Rg�x���?����S�U��l;���l�G�>V:���@��W����J�����]����'�EE'�*����%$Q��S%��%��E�� ���`U&I��!e3�+G�y�Y�����xT���v�%[;i�d���t=����X�
�N���?d�z��	7�?@������Rn*�p)cu�	&���5�s�p&r�/�\����6��

��b�,g���y�������'LH��j�v�Sb�h4C%���:��LJ�W��T�fS�:+_�n��	x�~mX���a��M��lXTe��E�qm�����G{�V��X8�Ol�K���C�%z������)F�v������3��J�-����������P�����l���u4B���SD���
����0]�q.��\��f�2��=&�d(���f�4��V���\Q�����V���,-S������F��i���<�\a���y���������_7a=��3+�o0����5P��P�bX/��fC�Sv�� ������9p���������W�|��N�����<�sdi3��m�����r�b��^�i�8����V�kq9�������C���1�zU���"��\��#Q�*�����*����Qa��!�f�4��������Q��/.���:�y������sov��s�B��Ps��iP�Y���bi�����l
6K���E�����S��r�K9	c���-�-nuSx[�uq�,�N��a0��������c*���"�\71?���k�r�	�
x9���"�4�L������
��T����|��+e��[��C6~~K	���%|<5�x�>��J���>��Ac��^�g�^���'Q(��Y��;��Yf�^���y�="�^����Y�X���S<��g��-�~�y��A���,S_Q�W�bs��e����JO�W�YYe�����W�)����<+s7�T�Y�z��<��*����<S.KyV�"�J�*��r~��<ksEz�W��DUE^EsV�g4�*���*Fc�1�;7������?n��*��3>re��ake� ��`����������c������3�t���3re�{jEPo�;c���O��'����������-nsB��3��E��< ����^�Z'<���Y`/e���������+��C��\QYQ\'K���b�0u�y���vg����~��e�<��BS�
�o����@a������,�I��4y��V��(��@�f����:�f�2�lvi��Au�O����\�g�
D�7���:|���#Mt�S��2��!��"�R4���:
zH�_d 33=�����!g�i������	'�����+��=%�� A����b����DE�����[��]	����\�T�u|*������r-"u%��TY�X�$����D��	�5���q!]_�'M������jc�~Q�3��p|�jC�#:b�/��K��K���s/`o�+nG�������)`T�W��xM��+��89�+�O	u��
�������3b����m���9������oT��R���[���>�=>�����\�*��V�n�&
dM"��3gt��
��<~r� �����!����z�����8{��<�8���D���o�'��a�%+���2��)��~�����W�.E���}�%���*��1��i�"G���/9��,�3�N�\v�]��$ov���3S����T	NM��*}.�^�F}}�?@��������A���������^f1�b��vTU5�q�[��*�u�k/���+�V�m�
��9Bw'Ho#��VJ�����n�������l��d��6Ep�_	����7���0����������
���l�����������A��a�h-c%w��I���^��,zr�	���i���,��,O)�*�i���T�����������F!�������r��0v���=�0}� ��i:���H�F����xD������`k���F�
����i���6fh����F������W�f�L��m`��ge�������>]�!Z@9q"�SL��3�Q"I�g(#a�����N����P�c�5@:,���0U:��@��h��������V�r�1�zI/L��0�d&P�6�B���b��
>���6g�)��K<>f��Spid4�/a�?��-�9Q��X��X^mZ���S�~
�aVy'�e�w�r���2��>4�W�
(���F��������\�����7W���*�+-���[��4Y�Z\9S)�]]l1%~��RB�4<w�n_�������i����7�ra\M@V)����{v�+��������Q�_�;9��G������(����Q��}4�6���\N����oU�W�����kI�+.3d�|{�a�TK	�F.�7���#���tE�2���WA���z��n��7��b<R���;U����Y�B��#����������?���.����q������>��������/� Pv�n-����o��R��r��C�������?J���=�O�'�A9��R�����_����~�.��h���<y ���'O���u�f���Be���\��}a��HL��5��:�jn�T��u��9z�31>��_V���>m@������Lz��h�x�
����@'	����L���Z-������qX�f������%V�V;�����+�n�c���Vl��S[*}l?����`���������T�V�#p�`��)�6�o�����?��O�����??n����m���r!5�=�FC���R
�C����'�3Vk���Ef��X���J��)N	k����0����!��^�����D�N�;�S��S��w�m���Wv�h��@����Hr�
ZU�66u���_ FU�!�2T@�Ok��xB��1G���RJI���E��������'����|S77��.���
^)!��*<�����_/���
=���l[�/^W(�B���h8=:t���qd��-������w5b�y���0s�YO1��(�`_��0�X��o���9�-�Uz����uO�����PK�o����fPK|��J-patches/0006-Canonical-partition-scheme.patchUX��X��X��[{s7��{�)`o�.)>DRo��Y����#�d��UY��Y
gx��^��~�����T{9U"y@������wy�f���x���tq|$�p�������,���O���S1�G�c�s��Ob�&gl<~A���x<��2/��\m�L��|�������_���&�����0[�y����m����wb���d��������O���O���DX�`�}��{�#�NN?��<��8�	�����x��J����y���3��D�D|N/������iV�V$6�u��X���N��eq�ba���*i�B��b%��k��=�3=������4I����N=��i$��E���qJt����:g#��������Kfy.�&K#���|��M��M1bw�L
���9O�%���,�l����eub�<6;�_�2;�V<�`/�����-�*q�C�D���E��p��Dt@R�(������1����A�y�S�������}���(��u�m��C�l%[�{l�+��)�p)�0FUt��L8�Ci��+y��~�/���4Z��`"z��`��
��6<����yx8���H��lS���D~�-��p����c��26�����?{����,�;9+�i>`w�m$��<4"����q�c�Mx�1� \���� EN[���l�"��<�u:Q�X��p	���$�����j����t|�G�_�'���������:������N�������W�-I���$[�������Hl`
����+�7���7m�3m�r�H[�[�mk_����M�+�e�3�{��p��������d
�f�'d��;X�bf�g��9��Bv?��R��w;���t���-����qV$��a��f�$�3�	����T��23�����d�q��>(��%��GJ�<���"�YQ��������Od�����}�y�9�m8������Wa����k�;��g�/Y1O�1�j�~UFv�&Y�;<�0v��������x:FE:���Q�j���|�N�\���gY��	�u�����������v$D���x�2�-��������x��wVv�$v[��i����o@"���r���[�Ea����p�a%h�0��-����g���'`�6dr>��Xr>�&�G�'+.gy�0+�x�y�!+�d����^Gw��`0��G'4lsT	�^|���O��8�r����
�f���k����X��Y���D��mYH�K��o$"�����(Bt������@�u��T�
�
����oE�C<��{������R�)=�i����'`A$��_A�U���rVB��1<z(������~���c�����e8�����8�T1@R%����G�G��(�n^�S���u�H���m����s����V�8o��`�)d�Y�����:u=�$q�=��)����X.}�y5�>�p��W$��#�xy�����7��W�w�w�n����]���-�����K#�}}x��%�X���#���W��J"��ZQ�2"OK�@���w�Anr�^/��v�F�����\:#�8�y�f�-����$w��1���X�i��R��h��n�Q�����-�i�U��������*����#Y��CH�C<���������}�������d�-���w���
�_���Sa��H�f�]>-X��&��s����@#�i��%�'F<Y��wM���1��p��@������_���O��z
���r�T����������@�N�����~����)�����9A�7�_}[���H��� 	�(G�6O�����y�����%��
���=x��|V���W����qo�\�=�]��t�O0�$v��Q���6`6S��^�Q�l��A���*�q{��$��(��.*P���z�Q|����@�5m�=�gR�03�'f?d�Mt�$7"�q��*����������A������\�6�|����%&�7aK�kFrI�p4�[���D�� z�z���Vya����||������}�/4T�A��r;.e�Q-�$��A�U��he��X�'�:`<����r�&s/0�����T�I8�;���l���,���I��
��3����f��;��j����$��v��g����N��1��U{�%<��x�������,8�xl��:\o��eg�_��nP��6)0	�PZ�]8���*)����YJ�4!xH��q��'LtB8�6LcZ�6I������j���VP�	O���9��}�^jp��W�n%���D�d����Vi���*�����7//�y����c��3��nk@��7ZWbZFij�[�i���������Qm�~o��U���qb��<�I���+tM*pu�����]m�;�?z��8�$��w�	nj):��=(���_F�0��p ��:�'@�	���6��c����]D����	#s;��������_~�R%��a�(&1R��������D \&|�����wo�{7_N>����N��I�27�'�h�x�������A��0@<���=�����g�"T�{i�n��|�w%�KW��xD��k��IZo��;9[	U������{�@%xx����tU���
�t���os����gd+x�P���D(NEb�u��'qT'��������SJY)Mg��D�@>,h9C���o#�bVl7����&f�,��Nb}e��hr5�r�`u�K��	� ��Y�a��>�Q�5d���[%CT�*����Xu��f�ve`I� ���)�6��=�E������/��?�Q���P�Q$���������=�3��%�l�[�
����Cr��y�N�����z��
��N]TtK��7�5����n�w���(*60-W�tmd���`�@c��x��s�-�t�^���Hg<��'���w�{oKm7}CE'b|>M�Qtt~������7�eW��8��J�3����

*��6��T��7�
���F_�0���� ���u����y\`�����n���?.M�e`�*�ys�������c�{��]pPm?EYMzT����,���u��C��o�X@�#Rb�@��.�X$���9��(�R~��0uU>�c��NO�G�NP�`	���x�P2
�e������9gW�]�n+EH��/������*��b?�� �hAG���JK���pY�i%-/��������i��������c�����8**,]7�4!Z���7�;�]�����'Q���h?1��l���/��3��.����:����$���S?���j!���'�r��:7A�������X��9����td W5i�*����j*��a�3���51����/h�wNZ��Zr���/�����=��V���L^T)�/o��O}l_�Vd�8NW�i2v��4�����s�����:x:�$<D�V8����T���X|�
�y�x�Y���HF��-_7�D�V���i���JCp�Ca���������}>�p���3�d�����L�����s�$0��6�qj�mn�������^���=�w�w}A0�{�>�o�~}3��V�6�H�uR���ECM�JY[�ck6���A�������Tu�W�JUT
88�E��Y�����$O��B�>7.��n���NYGB5����������P[�?P��e��A��	P��
@.�����m��}���<�U��������l��R5HY��%�!M�-���+Ea+�(�ao}�z���x>��(���q5��Z:�����?�dF������~+���v!xB���`FxBX�p���Q�em#C8��(���*��W�"�����|�Uk0��e[�*��(������tp���	V`����`Gu��Dtk��+L�]"1�_'|+��xe��0]Hja��,w	�x|�ol��"�m�4�~5�_!�R��DR���TG����%8xBVq���e`{P��i����RzTj�&��/�����7�<%�����R�jsb1�&<g�
���Mi[��UM��.A�^���`mM������O<a�(��.U��!�-	��z(���j5|������,��p�#�pT��V�!��U��<���w
��&���`^B`�s/��a�����+a�8����a����
�	���+3�U��2������{B�Oa������.�����:�%���2�>��;�z���8��):��2��n��lc����1>o������os�R T"YV���f9���6]5��E���� � �We��'b���J���[7B���z6�*��qX�e/�p�9@`��X�@mq�@s��$�����n�����n�T�����.$�d��v�J��H�m�I7@M���5�
�(����������C�,vPy�K������K�R5����&����a����~��[�f��V���Xq�a��A���H� ��:���)����)
�JC�����F/& ,����{D����XGY�x�h	(���di�:V���.�0^���|O���U�&�o��}=N!�`�n��!u[��>��Wr�	����r��O�~pr�������M��.^�{�d�p�R����	�0���?��q"H��M��]����T�4~�U�O+w��7��EU��6j����5��t�������TzL��.�h�� U:��Zz.C#�n��E@Fo�O�k�#�)�����w���g�w���9���Y���n�@��?��)(}��:�D����#:��!��E�����D���t�wJ'Y�[Cq	-���������tj
=�y����2M�����H!B?��@��2=S�j�-I�bB�?:L���+z�,am�����}�@����AP;_��/:#��{U�J|*cB���:a��*��+�@:
nmd�M�_m�^V��]�]Y>���995�&F})��!T�b,��K>���A�b?��X��&���"������z |V�#��ReR�#���A��[,tT)�Vp��Jw�aT@aP�qVzh�~��ts��/��� �!�p�:����bt���/PK������?PK|��JGpatches/0007-Canonical-partitioning-scheme-for-multi-level-partit.patchUX��X��X��U�r�6}�b��)R�$KI<v�����x"�)�����DC������. ��e�c�.\`�����RW�D�q���y>�d�,���4Ny���t1��$������!���
�(��%�,����L?��[x�wr�:��E]ki0KCQmN�nq	7E3$8�5�v�,'�2�� �N"�j��P�%|�>�����(�}��*%/���J+IR�`D�����iJ+������0��
c�JU�bxr���������Sp�w���2D.
��P����so!y{R��A��h1J���*U����Qce9�K�)
x�q����rY"���[��$KeP;��A��6�}N���X&���VZ��W��J�I��7�y/p�s<IfCE�$q�����yvA|2'S��xth��R��c#V��j��u����.�k�S���B_W���ZB�|g�i���Z�{x�����P����e���������N��:W��,�1�_
)��t��2N��NR�����Pv��e�np���?��H��F}�X��|8[\A�^��#�4���}~���V=68����h��V��`������4�'8=��A���}�t������
h!P*�K��'���*$j�Eqw�;�Vs������g_gg�zS��i��n���M����&##�j:w�Ny]��T�uI
�����h���y�����Q�\Rb�9�+�������@%��a��*o��d��4�%m�=��
���Z�!�*_U�
�iq��{���0��#�C��l��Goa/��E�g�A��������fm�T�����*�������3:be�����X���v�]qE{6)��;��W
����p���n)���L�qcAa��p.�)c�PK��C��PK|��JGpatches/0008-In-add_paths_to_append_rel-get-partitioned_rels-for-.patchUX��X��X��Vmo�6�l��Cvl����.E��A��v���0�t���� �m�-�}GR��:��E0d�t�����!/�|���aw�b�����q��������� �F���r��w�s��9�k]�1LD���H��L���z������(S�Q`����u�$�r�-�\��JB#l�;�:��.4�~��>��/�c��j�<������E>!'�����y����p�
V�T�9Gm��_����e����7����A�o���B�pV"e2��)�B��,rc*�;s�#'���4�bY&���9`�!J�V���@�a'`�WJ��2]�a�Q�uX��,��w���n���7�%E�po��s����:��J�RP��
�B�f( L���=R.����7-�C���v���4����I%��aS��R�-��3�Gc�GC�"\��z��CG����G����7�\*\�02��H��3�pT��l�5f�BV�Z�.��j���NY��|v��\��� �rY�Q�D�Z�=�S�j~)�e��)���n)-����|����?|%�������������z9[�������^NV�TH��V��j��N>O��A��M��g�����a�f�c����SgX���ZH}��Q�,�RU�F�5��S p�Q������������cV
x.�������r1��^O�y
3*���������	}o�������5Y+X_�	�P��������R�

U�@�
���$�!u�y;Z�Vwf��y��K-��+PH-�6�$y��v�4�;H����O5��[%L�{��3���@{u������;�H����TiO�=#L3�8y74�%�u��S��$-��jZ�&�����j���*C�U)j8f�U�@%3��Z+���jY��
����=I�D��C}S��Wz�����a[����:�,xF�BzN�y�����(�#���ZIi��<��wI��h��}�:�P�(�x�����p��=}��(�X�H�2#U�}aJ��bYu�V��M�g*MZ�[;*B�����2���
��S�uWS���,V	W%��k��j�1���1�l������������"��������)�������L�L���o��*��SS�_�>�]���������}�����}^J��V�t*A�[:�s��%6J,�<D!�"��#��NL��Qr\�y�����$���@_����Z��*������<"�n��:E�s �+zR���/�U���d>�^����
�k�����PK�^�F	i
PK|��J5patches/0009-Partition-wise-join-implementation.patchUX��X��X��}iw���g�W��Aw����hKkI����N(�pw�c��������8���LD����v�����"_���p0����X^���E�xz!/���g�irzq!������x�g��\���������~���|-��������6Y%��&Q����_dV�bU����{�|���<�����"�"�WI����������q�?=���������k�����^������g�6)��J���@��i&��j!��Q��v��� V�����_����2e��,�X�[��-�TT�x!�^��\����tQ�|&�yZ�JN�Y��Z�I���:�
���t��SG�������}�eUR��\���{��%6+&��H�R7�'�t1��"�T�] ���5,���K1)`���j������{�^Q��)6)�b��U��u��*Y����l�9��`�������QH��$YC�j.��S�L�),WU����
������E�ZI�y�c"XY��������E~�ph��1��O��B���N����l;��C|��3�t
�7��"@
_|�����,�7���$�f�M��������-��8�|���(D�Z����67������d���6�B46������������
�w���$4�?MSh��7\N��6B���`�B.�O�F�k�W��������5�ZW����!*���'�M�D���T�
���o��j��V����t�Yb��h������D5w�G\����d���#�����u,=1�������G�X�71�8���lnR��P��
M�u�;M��N��b��V�Yy�$JN����6�we/_W��1�3��Y
���$��S���g@�J _��e�`����B����v{��f���&
x�HQ��E�4��OB���������L/����vr��j�6`�&���8<>;9����9<������7����x��")�'{@J���{g�i�^���l���S���D���|s������/�I�����}����"�>��#p(3�z�St�P���B��X.��q���6�j#��T���,�O������	�����7��J>��B&P@���QV���L`b;u�T������rs����������?�Z��D�����GS�iZ��x�O���e�xX
��������.{����N�$����+�e6]J���T�n���+5+���1�KS�]_���at�A���XO���s��F$K�!�qO-@fr5��+��V :/�������#�c�_�����<��S���Q��@0�Y���|�4���,�6�'LC�l���@��^�w8*���[�H�r���J��#���``�����,��=��}�7�R�R�;������#��m?��sP<Z��So,V4y6��&�I�'��_�|�����U�6��g�cq�?�H���}eZ}4_������/ #lyY% G����4F>�b�*_�#�td�@�)��E��C���^tX�8s<w�t2��R���G%��5��nK����t���X_�"�L�����>��"����>J�0�e[���=�D,rl���������AD,�pG��k\��^u����Lxfb�M�r�"�]*p"��U'�\��X�|��������S�����/k���bM���6�E�W&����eP:����AcmR�� *���c%���Z��I_{��j&(z)�r
�m,���@
W9c��T
6�IG����1�#i�DfB�P���f�p.�]iV�lid���=l�����`��'Y�d���B-G	��}p���rwK3�%�����M�}&^b�@���1�v��'��4�@���	��Z�b;�RlEe�V�{=�k�#�#�I�V������������"��#�00��R2{�:��r�f����[t'D��l)Q8DQ�1�����-^��P�t���P�2��O�$��{`(��yR%�y�ug�-�`�;��	��D +L�3�����>(	L):SR�k�!'�Z�/�\����HctiYUDo+�l�3Q�qR�P�:lp��QpD�����gG��
�e�*���8p?�	\�t���:�:�fO���7��e6������Q����g����b��L*���;RS51
aE���O'	�����X�OO���(5P���Zed[�g'(Q���	2-�����\uN���GE�W]���#*���G���]���I%�u�V����O�J`l�j��� �L���������xtDm�FG)���xz[��G�J'R��n2��9B��"D?{�y�z���
��0�@o	�>h�l���������s������k�/7+G�C�Z/�c��y�z��h�I�p�^�A��z]m�7��?�����&��)\�����t��.��l'v<��	�������'�s@����RD��	P�E~s���������j��1	J����h6�KV)�B�	&�%\�~#V�!����I-��sJg �7o?�|������_���4���)��rp�=�C\�j�h;�=��8v�-=�/r�E,!�?"�1���A�@ 
M����F��V��D�ZG�;|
��N�1O��w�?��Go>��!��~�-��o�g|��b����},����UU�I8n�������(J��;b���A��4�M�k��e������R�I5���BLM&Q�����7�}�w11k�4
�J�xV�&�}V�Y����3M@@��9����{,��7�{���~p@+���|G�����0��`��i�r'�D���h���y3�������-
���
�C����\����8�T�h�.NC�p��d�<n=p�B�`w,N���j-[T+�_T-~�rQJ��������yB��(��8;)����z.r:�*eW������`x�#�Y2so!����N��%< ��yXm���,�dA7'���g��������`�Xi��[��j>��,�k�y�h������0XAs�3�����5C���t��m���-�J�:�*(���N#_�x/��1���Z����J /�;y+aQ�lAj��%�=���H�C:���r$I�>b�p�� /�!M�u���;d ��)1��
�$�t����RA2s�"�N��Z]dV�W��|}�O��?��x0n������Z����P ����81��)Y�-E�����(�o #��s\���c!	����.�rU���Q�1�����(��-q4$�����	�X��3M�.`��sI���
���;4�W�zV�o���w���T0�0u�{ ��7����0{��{X&3��I��]�l�/�{?F�fx��Wpx�b��%s�1<F�E��Z9���$��$�r_���Z/G�C���e�8n�!�I })�`���[O���5U|9��S��YH9P�\6�_��$8�J�������_@
�\/�t�`�?����3r9�3�aDk���U� A���U��D&#6}R�^	i�,�m|���n��U��Z���e`��=Q�]����=<,�A��}��oe5�`}]���%�y�;�vl��p�}Abfg1�
����4$$�4�����	�%9����K���b�x1�����>7)�!5)��NC�,!�.|j0M2��v�@]��_>7��Z�$����l������i��G�
���!�Nx���#@����'�S�nt!�u�� ^�������Jn_)�FU[-
��'�#�j����5��� Q�z�]�>�4�J!�T��vD'����tx���G� {r';op7p`]lA���f^�s��SOfJ@�2_Z��2`k`�3��l�\
��6OaR; �v$d��n���n������? ��"Pk�U�Ui����b��#<��2���n�bQ��(�<������G}��5
�(���L��e �d��n�\��������a�^.��CR����n���,~�M��1ZgPf�O0��?�z���c���Q��� ����Z>K#
���:���bn7�5OA�3�~��D$7�!��K�|0�b�����d�{��\9��9�Sx����a&Ga4�+��T/f�5�����cTE��> ���\M��e��!����n
����7�b����:��.r� �w��O���J���=��A�'B�h��3d�����I���R��=V�^7Zp[�ZK��w���T����VS9^����2R
�4#�1���.��t��J��;&)R�*��:��T��6� H��T�$�#�-��\��<|��=9v��\���	0�E��:�h2[���5�ju�?<��T�B�N����6`��_H#�m�x����o�}�Y/�8�n���zJ/|:<;9��{����b|~��^���U/��%���
�A�Z0Z-���D�m�i��,��#(Fc	aZ�.���bf���\�8�jD�LT�RK��
F6D���;���
�q�l=�G�dx:�z����d��{�@��GN]�#Vp���%n�w��<�"���+"�z��*2RJ�
U���fG$��6Tm����,�Dq�<m�>�w}��?��s���i��J�C��@�O8*�[Y��Z�:������s<�'{� �a�_���d�z�o�2'�8$�@�^�2��K���y��2$�|��y��t����C�������|?7��)�Z��Vd�������l�i������$���7��P!G��3#U�����T"��,Px�����^�whs���g������:�1��U\��d����|�WA�z�e�;�����T>���`5	��!�9Nd'���*�^�7B�!sh$ �,Yu�����~Y\�1>z�Al�z�{���q�}��D�n�����������J��2_�9F��P:
U�le���d`}$�,uX�D�lj��_�t���X���Pf%���-�sW��~��������7.���m���CE��w?�����/��/�^gb�K��I���n_��7�	�����q�^�\O�U���T�z��3��'�ue+H;]��.����W�	�>�'�\7:�bD��-��M��x�Nd����n������
6NfC����fi��xpU��\�/AK.\���$ ��Z���
7�_!?+����~�n"��������Pk^j������M�Z�����j�0��v����F�x_&-Y�^����'�m��Lo���<Cs�g��w�~������;b��;������W��I�#�P����l`�J�~]�H7�CN��,mr����;����I+�
�@����5�kV\����0<ath#��:"�f'����k�,aa��� ��}����5xC�j DtN�^��N����	8+����!b�UD��(e���t�01����6�6�u7!��y��a�1��-4�h�kB)bV�E<r����9d"'H�hB��!��$	�)g,#��6eiP��\���9���s�H���D��4����1���k�S�r�NR����Q=V^���u�d��*`t�#�!�^S40���v��R�o#����W�1�L��/��K�����^�tO���MUh)t��]y�+!��2����,�1���y5k�/��:!hA���������
8c/��{����`��p"�HzD<�iV�s�������)��!�9���)�.�5�l5��)z1�r��|
_��1�p��6A������x�OO��/r-�\�kYcn:�bZ� JB�����?�j�N-)���d���;���]��;�n����x
���$6����Q<��g�9`
�����H�R�<������W_�����o����H���5
�������
����Y�U~���s��������5,v���qFl���������o]4
C���w����z��[���0&'c/.����/�LG�9G9����L")|�e	W-�'��c�=���R�k8D*z*Q��u�.���m����Ev3�2��)���_���.�������"g
��x:X�]P��hESPL����rJ���)�����q���CW��^�R�z����\t����N����0����3u���7�� �s�!aU�����GB�\�0�;�8Jj��P{�0����9��X���������C$��Ev�Lj1��d�R��%�������cX@ZBlh��#����'����[�!�������	C�S������QX����c*���!�b�b�d�K����Vm�V�k�Ex�I'.�4b����_yBaH��sI�h�5&��Bf7p��.�qw�"�	���r�������h0"���r��t��}�{/g����7�3��O�&���-'*�?���hf7
��0�
����b�p�rd_%v'��F�s���t�ib�a�w5a�c�r�D���h&���w�P��_�G"3������.�������>`���!a��os���0SA?|oFR�G���p�����>'o������?dt�[���c�������?t}�~�*�$Zq�Vr��n��|���v��`\�e�'��80QB^�2������
]T�����Iy/YZ�a�*�0�����H6�@U��?�0�c��}�JZ[�]4���jSQFJG�+$���C����3�zn���#`�0�fe�������	%�#r�1��K�8h������-p�t����5��,�o��"5V���a�������E:��R������%P��V�k;T�~`�W��}�0\ad.(�Im�75�[�5�Um�3e~e�nv�+�=o�V�)�����j�	d����8��m{��t�����N����
�����r&*�n���Q�R���ig�c�fx�%�^��:�����6�B}��="�B)��j�*���)�)IF!�&p�����@�4���w�����c��lRw��m�|D�T2����^	5$���i�Q�]�XE"Z�GV�b���^rz��'���W9�R�b��c�������~^}DGAene\����~���F�����	 mG\��h�=��U	� �7Im�E]�S��>�\��3���O�V�4�M�����2T��������4���+�
��JoVh��$_���>�O�aP�2��O��8�����������J�T`*N�����gL�3�~�o~����:���`�d��3�D�1��c�������<�)���lV��<P�@m:���j�(�0R����4���@�u�j�,S����7	��7�EP�(Cae���(���5�$����W*�:�0�H�/���(U���l�f,�pL�	T=)h�K2��$cLtj10�R���o�������������.�����jy���,wn�WJc$y��02��\��������w�����H_�����������m�0����J���j��7��:$R��&R����aT�<�am��a���h��F�T�<�GT��$=���@.'T�E|<�*��]��[��fi|C����[A����7�-5�F��p���u�J�`�0�%Q^��&�5)9�!�A%��_4����=��65��������K���Fh���������<�vV*M_�w`�}����t��)���JeV��� ;Ig��,���i������t��S}����;�Z�?���e"�U|�������|--�&cO���`
��n��@*���yrG��ZP�G7���	(�X�ON0-�?JU�g�����USd�w�PD����H5Q��J��*�Z���*7c�0�zSFPl�6R�S�I<���\���l�UW�Vu�����_�������z��-:��0[Y����+X��B��>B�n��|����)�,�{es�%�1���(M�����f�:`oP/�q�v3Vt��KP���  !@o�����Qw�m��/��F�+ ���I���@���	\EO���B0
�sc�/8��D�[2��tQu+�vWW6)m����=Z��gP�gR `E����z��;N�n,�
���Pm�N
��:��
�1�������Z�
A�����HJ���@8��6����<�E�Y/w���f�X����i����k���)*�#6���~���df�x[|�3D�D���M��q/p-"��d�<
�4B-w�����Yd5(�0�J�c���;��-*;j�L6����y�T��j�������Y:4��Z��O�|���)�������4e�v���{u����x������xnT�3�{�O0�������y|�,<��k���5sek�l9��9"�������{����Tb�Y�������e�
#?������b���1�|�*a���1w�k�\='��`K~T
sN! ~���O	��t��� ��
@��N��?�_���@E��&_�i������Z0�,7y�R�H�0�b�'��7�-�����PH�8�L���Vje%}�W��sh�=u�g���H3Z����3�R�B�FV#���Sk��&LS����+��3�/��|������RJ@���&>@�F�N���V��
t�1:Z��?%��G����a�*Z��P6��*��O��K����0
���lo+ �n���xmlD���
�9E�?�RVQ7�M��Y�'!�<�*6.$2�J���H�M��^������[�G�<;:X_��_;��r�hv�r'�������T�i<��d�(��	�������c��LN�r��������ni�CPY��/9��W����)����Fd�%Sm����0���y���
�CD$C���8��|�}�k{}�s=8�k��4��I�������
���X�k���������Id�vt��C�����;-��6�7���$=FP������zs�������S�����?.SK����;:����PA8XN��n����r
�������	l��Y:<���`��K��!n�<f��f�.��X��;�?5[oJ.�-`�.M���%�W�4:��~�)�U��)#�]������R��B"$�����������_�3�!����{�Xh\1��q����}�\�E�o�`<bW����|w�
�B}9'K��$_����8��3��<�8P�����|�;+B����tT����q~�s�p���`k�i�f���u��c��X�=�DiD�w���b0j��V�,
>%��h:����{6(�D���0(�k�M|BPO�'�����������
.f��'!�M�IX�s?c$1�{�j�%	-�������������3��Q����6	���P���G(M��e��y��g�7o�[Ua���Er|q����N�����H�U'<�����n�9x����o���VF3���N�^�	�|���Yv�[Rh5��x����	�jY�D��g�B#S^n���h`���9c9��j�]���r��2���F������)A=���V#v����-���Y����y�i_N/1p�����B��P���~�~�����Z������/���cG����
_�t�����U64���b^V^^����6}�L���V�9^�cFL�a�)#�a1�M^��A����m��uV�F�p�9~3�.���H��z��?"��U�[Q���~l��
j�B�y���0it��7aV���*��.��U���o��o���/3z�U��_����E�G��3�y��;^B�9\�*����x'\?�{�H&Tb���p�</��^vrs2��ux;��~N�E���%������^��R>8���JaX��:�9I�=�A��.�Vs��R�������W�^�u��J�7BZ���r3,S�O�	����;)���������S��N�/.z�����t w��,��|���x�3��7�`Y���IR�1�x�R�g6m��x`���8���n-�&~P��q����g������U����������$KL���YQ2G\{�l�n�Fa����W�����]�{�������F������f�����?H�u�'�<����Q7�-3���5O���xipc�����K��-��nf�=6xL�;�*\J~b���8a�y�D"c:c|&Y��@-���rg����DI�,�[5E�0�T��n��H����v�*�����V,]���(i�D��X&��D,N�[�J@�tI��9�
�7%Yw�|H�zq�_���a5~�E�Nbq����%n�������
YY���X-�����~(�0��k=4������f��;?�E8
��!8Qr��2����p.�z�����s3������@�e����sA����*�������YQ�;IM�����&+]�w�FsMO��~_��b2��M����<@�W��]�?�����xH��y�;_����v5�(�����
��l:m's)n�D��-h�:�
�ID�T��iJ����!�W#w�kj��Q+�a)��D�M��nv�x��(���_4���K�]a(*=j�)L�V�5�y	,�P)(�0��LVB�$aP�F+�#9l��X���@���E*-yQL:���
*�55����x�m%���-�������[�ZF���o�,2��\��qI���/f}�l�*m*I�������!=)~$�	9{�+Vy�@n���<�����J|���R= ���I5B�@�72���n�A%�i�����'t�6	��<@����8|��S��/�2���q����6J*&!s�4����r�UP9��(&q�F��Q��V��bq��x�jE�z��,�=���(�l�- .���6�}{�N�������T/�F�lzW�$6H(�W�nQ3)fl��gdn�@���\��e�U���va��s�="���6/.8�R�P��T�����G�������Vju�j��zm�-�{"xm(��f�� �o;l�������@C������`�^��/W)��m����z��*wu����KG����;`(jZ���CA���6�j���
4��$8�+�&����(tA�nN ���N�:���xa�V9yV�T���G	x���S�S�a?��@9�Z��f�n��Gku#��^��:�����I'���4�����9����^uF�"�!;�*5���a�/��Z�CY���.�"B�����W�O�Q����J�p���?�}JO��b�5R���a������C�E�&�����5��������u=-�Z��~�8�yd7Rjw��vS4g���@�|I�]TV�I �= �i�(�tS.�5�F\*c�#p;A���d�T�.�rqB��Z;�<G��g=�0D%���y�;8
��m��>�������9�1W��NA��Myl��dgh��[������	���|���fNH�!��g�6L����� �������������S�6���(�����������{r��=�`7�%0����z�~�����f_�w��|�^��
P��'h��	�y��f�M� �5?y@�
�%v�Y�4� S2�������qLI�3�����qx�A��]~�D�b��>j�"���r�Z��J�s�1�.�
�{��{���0�zm���j��^��@	���Ho(��6h��^n�������O��HG�IO]�B�aI���4��?t��@����e�8��l�kc���k���+�N6�*-������!�i�j�	���zH<c��|!�53B9�BO�g�����Ip6��)n�V��K�	�X�C�!��%z��b�gl�*<Q3���r�}������F���V��fy���������	����{��+����3=#Jord�s�������������1�,<���~��9T�w?=��qx�u���
$}����i�h�~�?w�f-�8���W]�8K+�Y#w�z�1���{�4����%����z.2v�*��������R�oe��
����48�_��9�3
&�a����g�~�(�����e��*�E6�OtiH��R���4u��5�.��<�^�����uG������������z�_��������k�Rj�k����a"�&���U��%�sV�*�5��A4�%��������������Yw�?���������}>��c�s�&��mm/gu�Og����d_T�wk0�2�Mu�PBV�Z�)��FbpY[q���e��X�������4+�q��o��<�X��JN6����d��k&�F�����fce�E�����X���H��9(�O�x��(a��19D�iZ���{����z4r\i\�v_��A��������h�{�<+����{����N:�������h<[se�*�[|������.���~z�Y�)mZy�Uhf�85H����/�����"��!��|���:_����#�F;+H(�;������
��E)7�K�$�,�V��M�(>y��T��$/�0����P���=��9=y�&�!������%��\d�P�[ ����p�>�����:��9�=���&8���|C;L��������qM����:I�E��Z�s��2� )�rNG�z�*K�h�,��0eto\B��H�UM|�����FJG�_���N]#�;�S7�
�����y)�u�5���:S������&�5�$�!XkV@]��.����������s��l������]�%W=���a�����W�
�z����a���^��+5Td���������
}�����pj�P�z����S�n��"�(�������:��\~�2`PP�9��)(U�~Lo�FxT�`is���
_��$�p8a%n��y��8m3I���6�
~��YE��y��<U;�&eC��@Iz��h�����2�`�j�Yb[����x���W��R\�r��JI�8LB�����$�����1�r_������1���_�l�,�3u�����%e8������k^��5)��t.)-!0>k��c].�U�{�[��iyK6�:d�	*^>����Bl������{��;�V�\}���������Fo��	�k�5����P�M���lLN�� ����S%�M6��&�N�eL'��{���|�
����f7�;������Y���������e�R4D~��)��1��cM�&G7�I-�',V�>���d6�z�'��g��p��z�O�
*8.9"x:u�����en{F������[����V0e2)N�xu���M]����5��{]���g��_�{��+����w}�������^]x��9������z�����w��}�=��}nA�@�0��U���<����\����_�n�/��q�_o�u�P��|��4%���v5k��������������HX#��U%�/.%��`�(�gn*g'����L���*�����@�iUZb@����'�3��#�*-�m�t�c9����O���D����l<s������/��1~�d�$�&�4�NU�h��\�d�N�B
m�(������J�\)��4�:���M�cr4�G�T�����R|�,��G?���e:a�Y���rI2����j�d��`�_�4�Di��5�H���+�����\����hq��&E:����a	7�m��$LY���������M"r��B�rT=�Tb|�5P)|��H���u-M��14�����(8�d@��u%\H�d1$����^�����k�(�����_]������<�|��&?�}���o������m��3��*���1��v[N9J�7�-���s��v
�'~�b6,�	�E�X7h{�M�P�����������5�n�e&��y
g�f�v������=���k^^������4C���!�/\��b���C���U��i-���bu���y���&	�w����^
@���U�H>
�;g��"o��Q<��)fx��bo<O��yR��K���Qr�d�B�_���0!�E�b�'�%^u8��L�5��M�r5���:�c�p�;l��������fYN���I���������.[���}N5
V��wO���t61�����]�,y�f�1ul,u������:p�U�������)�n�-�1��)��O�L;	���/^�y~=z�sG�,� di�qx:���^o|~v��v"$�6T,��u��p�0��+�qS���GY����db���"r���9�Gq�*�L���j�����[<�|l!*��j8�H]�O]�O�=Z�UK�_�q��t�W\</_���x�us�S9^�`��F�09m���ap��	"�d����y�<j��)�XnC�<v��~�h�d7����h5�m�/a0�M
������Z&���t�\R�E�h��m��/���/"O6��B�c��A(U�-�%{��(��c�����tB��<;�cB����Y$��68i�W�n���.<&�6j�~r�rF���1K����Xd�v<������g� �]7!�~I2CU�L6��d�����g�/���(���e�iF*�ll�\�_^���y!���
����`cO��G�X2\�q�h��
y��8����f
����%
����3l;W����[���&O3Y�]�m��
h�Q!oP�t��O����6�we_=�XQa������(�d:�������(�C;T&?�~w8�cH�a���^gU���x�.�	�K�����X�ROG�P�?(%�w����6�B�{/�������,����*�Ri�N"��)�E����HjXY�6t���bo�������>n
���~3�p�c����=�K�cyf�I�V�2brsc��H[)#A�!u)�C	��@�i��HY���U�])1�����wgm��
NX��3�mk�} ����7��a���:�o���A|�b5�mk�a7/V�����@��vFkx���om�F��������@��	�>>�5:�c��i��t���78�p)��L
U���|M�c�;� BG^�e��0/Q�,P*|�	��(VLlX�L}�������Ay��
h�	���{���v��PK+�{��:��PK|��JApatches/0010-Multi-level-partition-wise-join-implementation.patchUX��X��X��Xio�F�,���X �,��a]v�����w
���(aH-&��3��m��>�������Bb�3�=�{?dIDg#o"��t2�NDo4r����!��H1_�����>�������?�z}�����Z�:Q�E�B�O�|�~~#c-�4���M���;��9�-���e����g���y�G��h��>��W��s������������_�:u�
��)�t���G���IS����T��e�aH:{�/��L<g�f�D����e�v�`M/T�ee��3@�3�IF�a;l���u�x�+Ix_s���"�y&��Ss�o�a,����

RR���i(\^�I���[E�<�Z=?.du�����_�|�z!��|F?BE�E��r�����l1�sJ7[Z j��!�m?�L�&��9��������U(�g��$��,y��>L�pU�FBX���o7J�.��I�P��n�k���SG��d��&����2;�s;�����]��	�!��9J_WSp�����?y�����F���g���8�q�I����Jy�A���'7��������@�B�E|/�M��
��U���'CY<w[���O��}�I�`�8:+�=�D��L�|���x�{c��������B@����
u����3���������P� ����N�$�����R]���eQ�/�o����N?�����O7�F��ra��n4NO���v2��T��.2���2�����e�Da������f���]EeA��c�j'�����.*\�T�H�4z�����4�����������|I��&1�>�G���}vEW����}��������u��SM��p��2��g�I��|#E����������c�X|���T'�� �&���Y��r�b�u�Q�.�K���<M�L��a��n&�����^4X3���@[�t\���tBWE
_�'���EJ���^�������R`����A���o@�_�=�~W�d9������U���6?���Q��{��AX��7Sg������7<�fn�:��Mb���?:��i�G�l��g���R<uP04�"M9�x�LS�M��j���#�68�[3�`���IkO,(���$0���@/(��DCB�f�t-#Gff8�o�����!��2��q��/j`�E����F��a���~a��]jq["�����Z�o��e����8sn�H���Rk4���3������}�|���(��5r�����]T�U!�G�����K����]B �|���o}����[�Ae2b�<.'�ls9�����y�14i���v�Z�L�<��*���|���z�;��7j�Q�j�V�?O�(���~sGg'�(�^��]���e���9��+,�]��i�[f��fo � �����/��[G���7�L�4��;�S������|�5e�������#�7NN�'�����g^���W�f��u����/
��a[�
�!������m�|�$*��4F���O��q�|���[��Ere��9�����:�\��-5�6-����8�D�0�������^�~I����_� �L�����������<��E"��;��'.����~fRx���H
?x�i��Q
d���c$�ad���*�1�x����)����x:��Q_��P����8�o:��7H�
����1�c���.�/�������'�0��[���j�����b/�.�jx<d��+������s��� 9�����}���� �R���6v�B��,GN3"����,)��������a�'�$+�XaY,�@�oN0�J��}��)�e�a�HW+){^��\-����G,I\��eq{Yg�����{9�P��;�;iQ%����Gf��XQ)��p� �n�+��V��E�^-%w_?��*�Y����3;�Mq�6
��R��p�'�����K���U�b��n9M{w���{�r|/	��x�Q'���	�=���
#�)oU">P�C��E�X]~���N���%�l���r8
��7�,�_��Kia����
���@�k�!ts��f����v���{f�,�PK��42PK|��JGpatches/0011-Adjust-join-related-to-code-to-accept-child-relation.patchUX��X��X��Zis�X���;_ N,/qb'a�2P04tOWMM�d����Z0����s�}O��:)�c����z�"����F��l���;9���;t���l�����x0\
��3����%1}R)
'4\�?:��+������,�|A/��)���������T\�,��\y���D��8��������^����pr=_���tp1X��������?�����?���{_���/IS�BP��H�M<������ w���$q����������N��%*����8p}/S�)���}^���6��`�W'�2'?��d��p�9�Y8����8�]��Wq��-����c�J2�U��;���4K!���&�.-�,�@s���b[qE3�Q�"� ,����B�n�u�y��i~����_�.S��f�9�����n��i����t�;���*�[���D��A�g�+(�:����%�FcaE�EiY(�"���b�T�apm',����#�G��nX�V��n�)j������	��1�XS���D��mq��������D��{dp?X�n��-���!'��"���%�9��i!Zd�S����Cg��$,#���� \}g�t��W(;�������K(���P)��S*���RQZ���B%�K�V���8L��{P�c3�c�sx���
Z:9�0�����7LC'f�z�������H��%�� es���*4E'�N��62=z���\V�qQ����j��q��>3�Q�)0��y�Sm
�����a~a�P8Um������������q��x�v����	�u�P1�9�e�D�n��]1\�g�����AR[���A\��@���vc��f���3C�!-�4��@�����3O��V��B@��3V
�@��T�O7�Zo�<���5����Y�a����bnX�@Y��H	��I�i{}6D����n���|�eBZZs�IB�{Pb�f���L��Z��F��F��.x���6Hs�EK�����X����:VT4
���� ��P�f�	Ch#����;��mn�!�y��)t`]�A������B���<'�L��}bs2h��@gw����b9pg��0��\�/{��{[J��z;�W�9D������@$�L�����7�)�[�[?K'.X�J���[�
G���<,`$�^iGd��\����y����g�=���M��k�G���J�O�4U�7e�w���^�o	�%Kg>77>RS��3�j��g��:\�	
�D��@����OQ=�N�d7���U��a'��1���.:S��U���\U#?��d����T5��	�G��U��`f��-�m�J�k~�u %XZ@!�2�78"��qIm�%���>p�W���O�$K��+1�\��E�0���"2Bg�w�EU�
������S��tS�t��qM�v)�����k�t�YR#w�
mt�uX[�DY23u�9[ D��c�T�}T��i�l@���)*U~����_GX���`�sg�57Jqt>
n�.R&�\}s\� n��.��F����&�J��������,��Pq��a�@�!����$a�J���5u����m�K!�a�a�|n�$�&p$K$��ZV�kpaW���D�@4�2����MP��V^�8{��WhX�m[�gn���@�~��c��������{.����t��c����7�kLp���%m�e�$Bf7>�u�|��Bw\��) ��
��G�`w,Tn�O�=G>r���5;l���o������p0R����������x���<���~"����;����lB�SV�����
%���v-::��E��?���#:!�q�9��GZU���V��.�{���*�$:����`Rip	���@(uS��\�:�-�N���x��O�:�\E�Td��yN�}�+�x,u��.�{�Ha�3����<�\f?����a?��$������[)���#L��L��������S�E��%��K���^�m����JO���P�$�R2�����)�Nn�Z����dzc������?�@�m����k�r�! ���H1����q���1��[`�Y�fw�P����!�`��eRBzN�������W�����'\I^@n
���y�)KL����h5�
��4�j*y�����}�E\�w��h���6�:�
���^'!r
��u�~��7�����aM�^G�3�Iq��e����
u��E�O��*.}-�fn&��_�F=}��c��t��E;���#G���ZeS�(��XA��� �I���]];��#P���<�N�^�r���26����w@��0����{����=��,H�N~K�_�3�.�}��2���s3��Hq['���1H���U��/U�_���_���*���pp9�����p88���������zR%��dn^���=��yF�/�Y��H�T���F�4"MO�1���C0�A�~���B0�F���n��;��6��G����}�G;Q�"��f��C�����;����OU��:�rm��+�j���e3�������A��F���p]_i
��5V�I�7����+���y����yc5����YwB��q)�V��t96b:�3V�B��nJuI��
?1�Z*}���n������k�,3-Bu����V��~�������[�>�~P�j���l�/���������i�G|���)������=����8�}�u:���$|�}kn�2��$��/lo|��#=�PG"�� �iE��!;"#�J�5�{�g�����Dh����0��������o�|�,�l���`�_9������D�����o
�0O�
�
\Y<e�9~�$�Ok�u���
1��'�1���RG��C-��q�sK���'�W�t�R����`	C�GK�/"��e����f��m��m����2��g"�?�H�EIo�ny�W���"����Jry^O���.G�����p����������H�.!���r��x=:��2S���(o}�.E�(K�V�v7C���b�v����'��v7V����#Zf�`�;.��\�H=�C�ot��AGdkx����#�����i��O��4�f��i���Yv?��.�2��e���fYY�i��?h����f�_Nd4v1�:7M�_���z���Wt#���y����S��d{V���~4rz�f�&���	�!`��T����-�plM�=26��,�����j�yFRe����,�~=(��x���3�0������.g����	��.��N�;�1H����d��9�1���@C|�6TB��C�r'Zy������OK���ovA���mt6�^��F�?d"t�	��q^�y�)��p����;�O"��H�e
W��(��I�+M�����za���em���SML��&e�|4���WR��=�X���Z9k�?Xx[�?Xj�����%.�qU�������.��0]q�|�)L*��C�?tc^������"����5����
?�39	����}hxh@���H����C�3`�\�r�w�\�]{��'N���&����e�PK>Hd�
�*PK|��JGpatches/0012-Fix-ConvertRowtypeExpr-refs-in-join-targetlist-and-q.patchUX��X��X��Xko�F�,���/��/���l�(��1�<���.���C�	�a8�d���}��!%J�g[%"��������T��?�q0�Fc�'�b<�;������`x�����wH�TL�dB�1���������y�c��Y�)=��"�C��S��L��L�4���v=5?q~�<f�6
�4IR�{08��������w�����c������%
����z����2�.�2�M��
Rh
c������ZfQ�3����\D��8JD
}~����HvR�dI���$�h�,��)S$vl�`�Ht�,��������KW3��<�T��4�-�R��������H.dd���Y����,��H��m�h;"Id�cK��c���7R��0!l[��&�h)�z'���0�&�[�eaQ��]�-�f�J��>6�m��R��M�j/D`H�J����\[��"3<�$���}Q�������<�~�	P�������1R���;��8�2s�][]oP��0B�%�.�	��*v"�k�	(��1<�*Zp��y�)��.�4,m��*��V�5�5�]:�K�7t�SP%|_;�����t�p�r�}�	�����P�7�(*���\��
v�����K�)�#�j!A���<�L��|��c����-��up
D��R��+��9��00�-5�X������Ql^�.��Y������	������r�A���!�Lz�FD����B�w1�"h��U -U��FeQ����������y�����	� �2���t����
�#���,	�#�^��Gq]�z����G���x�p��'�k��i|�-5\��Xo5Pt��H��z��8~���t�CT�����>j����A�����t8
��)j~����?r?�=v�g���?h����#�O:_��4�2�ja��j�8��{R��������B@������lA�HC�P�Q��Z��+���ZmS�m$<�RE�L���y[���-i�B���V>o����@Vj�f�����?o.f�
.�W��u��������_��������Wg���]�a���T��A��X��k��8d���=5����*�A+�.��&a�OP��"�D�dp����k�Jemh[|�bJ3�j�4�[�d��d��2�S.�[���@�U���>���Z���1�+�D���*����G�����C��_�q������CC�Q��4GK�Np�K��l����a��9����J�^T���?��z�
W�[��"� ���0��=�p'x�M�����OQ^M���|J�&�
�F����}�(�kw���Kr��g��n����t�'���do�hB�1,��(E�`�(��>��}��i�+��W�?������q���m��� ���+��������G�8*�n�mV�l&2����|*#_s}��������<b��
t"�0��i�7R&Hy�V�@��Z2�����6�V��I�
�,bCXpu�y��1HE�U�nC��uZ�<�fBc�_^]�\'M�V��a�Ve��9���s�W��c&2�.q/��?��|���S�}���0W&���]z�������h��H��u���U��%(u���5��U��
�8���c.�.jqX������7 ���(�vs����t8<0$
������Vp�='qar�9��6�����i��'�&�z��<I"$3M�P$�D�pr�<P����*!�
�L]6Z���{�B�"����df����s������qs�/�b,�:'�h�Z��3L���r�Mm7��X.�=<�%�����w��R����yvb�Lp<���|{>o~�:�t�O.y��-vY-l�)I^�i_��vw�s�~�����NDu�D��~�'���ex^�i�V�bn'�J�&���k�� Ku#P�bBir���V�5��=�����
�R����Z?�����I����I�=����Lc�H�bW��_���y�C����:����9H������n����O�nN����������Q{�7�}�����Nd��a���u�
n�JY"X����q��/T��]�����k���qA�Kk�����V{���5��[��s�E�uk�37�e�f<�+��"��l%c!��|`�
Zu�My�V
��Wu��yea��H[���/�k�|ofG[m6.�wW^s[���u����I�i}�Y�iq�B��KM��l*g�5������������u|�����
���p0�j�09?a"6W?������v��/o�Y�Bt��_����\���m���t������=�������=���*�3a���"�~���n;t*rf
��o���P����x���i�qx-������"�b�����:����s:r�q��{�8�PKn���I	
PK|��J+patches/0013-Parameterized-path-fixes.patchUX��X��X��<is�����_1��Mx��$K���������������b
�	h E����>f�v��}��$����k�=x��<:����������,8;
���N<�;�����H���u��k�����c�O��t����B��<�s��\�\<��������$W�2��
���.��?�\=F�##^�>�������G�b��p�{]x���?�}yq���8x�wp���Rfr�d�*��\��/JO��wJ�H�\���5�Be3%�L�'�>�����\%"�+%��J�������iHZ@��I n#_�oU���EC�(J�4[��2S!�y���y�/R��2@��i�D<-2j������~_L//�.^O�����W}!o�(�MD���n6�(�E�����_A����6bF#�U
����]O��-]a�)�Fl'�� 
�$�'���(0������l��dK�"@���e��'���G�N	_&��R$����t�%v���S
X������e,�V�)y���d�������W��o_�"��H��$���I`��G�,c�N[&�% *����a�<B��9�G�Q�&�Q��45KW�@&�s��L�k�L}6�CR�HQ�
p0�:�bE+f2��R�+������Q,Q�,�'G���-3��0�	����t���F��Y
�*,�D���#�#f��-�[��f���c�jo�.p�}Q��
������q�I���J�ec����9�r���l�-��fIB���"�T0�3@��:O��7`;O�){���l)@Pt�&i�&��x�������-�=@��&�[����`�������1�{�2��f����CJ!|�p|
����9@��&:4�����l�cxQ��E���h*�%��\F�yiq"�q1��~�UB"_������{$T���CT�Z4�������S���po��@���q�����x�L�m����X�_D�))���D���a(�X�?9:B&o���n[����������I��'�������9���r��~���f�o�H�'{(�����s�~t���~�(���>����u��
xpNs�0��:G��b=�d�ksX�����'{<�;4 ���e��`Ni�3��W�M���f�G?�?E���s��%(�di���u�{��m��R��e�`��L� ��@�&r8�-[X$>�MFm+9pH���A(���9���QR>�g�-��W!��W�%�O����2�V[Z2�U
���Uc��g`����'lV����)���EI��a����	��PO�$A�����?w��_Tv���
r�)b<����s�1j "��`��r�����<�4��f;
�.�L��eH���a�m��6t���OX�;(�o��%x��e5+����vg��Yx���&��}O�(�n�v��l��h�OFgb�������x��%��3��(�38�A��,�Q��&��i�N"0���=8/TN-��::��i~�Tk��w����^�U����=�����.!��;�c$��Ta�����J7�5��wCp��75lG�<{s�`��:=j���i�)�2��������P,��\=������H��M� �=�8>s4���y������4�~��y�V��Q�8��
K��a;r�����}�Q
gg�
���~J��������r���-5�����M����\6"�xxz^������)���<:���nX����S8C��������o �����JP�S���`B�W�V��88��X����_���H�G�_��aL��`��FOI��a7D��m��g��v^�q�3�t�4�"�~�d��y��	����x�A�p�T)C������.�c��S�L�C�����������Fr"^P����=�C��k�������W���AA��N`VF#��/j��r�Y�����������
.�H��s^
W+.���R�m�\.p%���������A��w�wi�79+��R�"�Hb�8N��`�����j!�d`Z��Kc������
u)����c^�4��
�i=]�o�V��`Drtp8zx�	��X��>M��XN}�_#�=Os���ZMC�������8�gE�Wdi'��:"!V<5d@P�&
f��s���:��(��La���P�\t��I)���8��{���,���<,<�����[�d
e�$V�����	���p�o�����l�-ZF�<��8Ta��}�r�y��Z	\0���i\�T=�T��,��(����*#��G�*3�L���!gL)�q�W������1�s�f�2Z�D
��RG�Lg�s,c���)
�%��H�*�m�<-�f�lg��r��:G-�|�&iV�	pS���@������jY�m�Ua�������-<���Q7!�M��?9m�p~���F�_9�c����4����
�tl������>[$�d���,m�tw����u�Uc�
�N����t2������]B�&�M!]s,%�N)5u:zD]W������E�},�d2��s"�}^$���a��M�&W���,����AC�vV)�����g#L�����x+�F�����#��j���>��EJ��X�A�S�w���u�Vqh�L1��!GG����En�����2Pa�����26J�dh@����6$�Z�a�`����M��Rjq�ZV�}�p��
"d�)�`3Ce[�9��}y4q��h��8I�&�M2��#������6��J
�^��;|�%	�8V��~���@a�K�4)���	E�\)�'��qA���<1qc���'B�8����ta����d3�!Zw����"������u��_q)�[x���f���%��7/z�r4#Q���DP�'��>��A����r��F	���������0�����������;��qSh��R#{h�s����o�.(a	��������Z4]w��'Jr�\"����9�KM�X�=�nU\��At����,���(�<8���_��hsJ�����j{^�������e��������FMm�FQ}�`���M�}�9t���x�x����Nr����4���ZL0��a�3��23�x&)Z%O��@`pE�J2@�A!��e>*1.�CHn��KQ��I�T�ty_�fI��v�'����K�Ga�[+R!�aw`�H�n��9�,%����)��c�
��n\_����<8��#��qI
���cU��+���bU��V�.TN2�u"�nr��_������ha|��s-D
Nj����iF���^|�R�G�&��VL	������3���<�������|!�/�\�p��%�|���2�>�"�c�����w
�VH�
�������hTnj;�.�`
�L�HV�:(%OU���w���pG�z���Z��U�������hb<3�$�6a���N_Ba���#�L�0��L�5c���Z���>�,,6	:=�0{�����.K��&���y�6[�r����U��z�O�_0|u��3<���7��z�61�k���m���#���G�Q��*�!�����@�<�X�����y�����Q2��'�]*��8?���*_�+^f�ld���J��v�����m/�2F���O��iL����T���T�,?X�A������}��$D��N�r����,S�7>�F4GF��<1\@�����X.�,��?+H�r�����*>yuPs�;����U����)7�;[b.^��[,���U�Ve4k��gr��H��C@�������i�B��`����e����]�,3��H���k��Z{S�+R����5�O��n���Bo�
�z��w�7J�+B�5��&���+�P��{�![��y6�
��8���/�@
���3	�g�&�L����5�������;�k���A����h�I�����"��V����Y��2�W���u?�*T	�G�4����y���Aq^H=�Qo�P<���ASkj�;g�����4q��;��7���T8�����.�0~F��5Usg�L�u:���\n���z���V��%��EZ�!&-d��~�����m�K,���t��o�,�
,+�ugv�Z�!�$���ig�,�$�55�je�����&��1j���:f$�-r;>�?s�5N�*���	Q��6b�0����$�X���U��(R��fY��/B����a������c����P��7~x����Yf��6s�V��h���f_F�g;��=(k�e6*�l�J�Z,F�2�.f�u}G"�����
���5��LUQ���z.|��"���f�\$�U�J���Y84������XU@�w��78���D����jC�k���J��+*�n���,�F�X�T��
�E ��;eg�"�Da����UJX��,����Q��2j�2Z�1	h�T�g�*Ng�gWWo�F�A.�����EXv��`�<����/��V���Tj)�t��b��ca	���(�u4C���c0���
�6�,��z�y��1�NLiH����� �z)4��l,1``�2���<x���6l)��E�2��
����K����_1`D�=�I B�iG�����fH��^m�T���2`������^A�S��?P'R�8J>����X�b8k�r����LA�O������c�4Q�2����� g�8����B-RT�4��/�D�w����b`�h1��A2�;$bG��83�/pHb�(��K*P�uM�Pv�q�"a����	��R�3�m/|�Fa�A��E����D�`����s"��[�#~������E�P2��+�"����;�/f��������YH��%���K h��1D��\��\tp��`�������\1��E	�r��i�t�)'���4+��H��G���O
��N��e�
���,�����������M��H>���u�H�+e�b�x�-3����@y��X��[U������G������cEN��W[����
f��Wq��<��������7������i��5l
m�6w:f�v`�����td�4�_�A.�zcL��[d |�^t�G��(��M��iy���!�����j-^���[�<�rNW�����n-�����2_d��%N��^�0G�=���X�<��_@��S0��SS���=��ot�6v�[''�<:
M&���0|��"����&k����c�dgt�<.p���vo��2��N�/6�����48/�3��T�_�P���_k�x����W��X;�����A�#:C��To�s1H;ZMi{3JS�����3�n�~�����p��;�f�>��Kq��Z�����m���!�!=4�nJnx�B@��I����?^�n��X7���l�"��-V!�MB��:��������v���]Hz]�J�;��9�u$��qHI�\oZ^��m`�������6�t�P��@�U3�2E��$���U`����\�PW;:�Fm=����g�@����%�5u�Rc����h�Z�����}��k����-o����$��B2y��W���z��Y�y�02�d(��^����@�E�sn�T������U6��g���?�L��/mg�x?����i��\����7���e��)��678�W����������PK�uzN�TPK|��JGpatches/0014-Reparameterize-path-across-multiple-levels-of-partit.patchUX��X��X��Ymo�F���|� ��y'�F�^u�H�5�K[UU���o��d�N.}�����BA��:��yv��g���V&S`��� h��a�
'�.�'�v����^0�G�;�xn���$��|�Z�C��v��Zo���0SI��l�|���
������r&E���'�#���!\���0p<����n���9����uZ������:����/�������>��l�T��G��2IS�f���C�ox�B2��_*�D������@L�@�0��1���~(��Z�I"�`���U�v�r��T�M���x4��S���L��I��I�>�
C�����wY��D`~��K�����S.)��Z�aKq���,g8��
Xs�x���,�3�]w�u������x�n���v��mG�p�[���58�nod�����M��<bq��I<I�@&�����r@f��V�U8����}��z�_�9k��(0}N�x6�|���;8�A�j�\�Ci�"�u&$�I�A���U�R���'�y�op'Pa.p�B�(X�q�5`z�8��R�����P���rm������>���L��sg�v(���K(��!z,��D�����u��t��h��b��Va���J�8����`B��4b���L�4�R9[���I ���������[A0z�������w��1~���4 U��1�&������|�k��+���}���Ofw�55������-�����h�t�}1>L���2m��f.�Q�v�$�a�><[
x���`Z�Bx�==������1n�����s�;d�{�#����RNWOE���,��REl����1SJ�T[���z��[}��#/.��{�����M���K�������M�:c����*�����Jfc�	�l1����+kI����p�����M�(;����������+�W��S������}��UA^%"������
�CZ��S(3w#�S%��hY����A6�j)3�I�j��z}�e���}m�_���JKL
�fy��%����O����#��Y}h�����;t���������dl�$�������;G�5�����5���z���m������������t�Q�Y��i����z�?����������$�l��F�\@������R)�,�>=�QqL].�c�y�h>���kr�WM���u*l�e�m(���p\y]N��6�&�2H��\,:����b��J���;�P�y^�0P�B�p?1��v�OS
��nJC��J8<mh�&�W�p��U�}
�8�yQN��oh��x�*��0�;�vm���x_����%e�>�=�g�[����x��E���BvC����S]E�"��rY
e����E-���N�UKK�q�Pf������7�3lw��/��O�6�j��kR=���9:y�tL���J�+^�-m����6��,w8,���XMk��������C�h��Ad&\]n�!'+y��Oy�&���>�
��	��o�����l�%�u���{��Lf�	TIE�
�ou9
�#*�J����F
j�qF�����O�z����L1�9E�z����:��B��z��������6 �CfE��)�=9m�+!���Ob����-8�o^�|b�L0�;f�w����6�Q�gY�PK��t�n�PK|��J,patches/0015-Partition-wise-join-tests.patchUX��X��X��=ks�6���_��;;���,��e;Mo��i�u�^��w���h(	���!������x��	��,;fS��������w�j,4��x8������{�LO'�;�:'���>>>������Z�����g��}���9���&\��w���[7������e���?�t�����~tCtn7���c`�e�9�3����o�l��F��������?�9���~s�p�W���{5_�a�k�n����=�+��*��2t�,`��i��Q���,6^8?��'��ikGS�c=p{���<�=�t��p|\���HX+Z5q��]����G�h6}��[�m�||��t��X�^����[mB�b�(J=�N��%�c\0�'G�]G>"����#�xZ!)h
HA�#�������Q0�G�

�!
JJ����rQIK��o��I�iAx
A��cJ����;4����I��}��i`���|ov0�}��,VS�~�x00B�y'Z��|6��w��G�36,��/��0����������9u�6�5nCc�����:�b��o�'�_�f�jM�
|�p}����W3N�G��?��q?������Wx�Z������o���u�����C����L�C��4W���-@K��R� Q
y�Z����8����?\^���-�v1I�]0f�	��������������W�������.�nG
d��p.��%������_�\\���W�R@7�oo��K�W#������w��w��!��w�~wh�|<[Fd9@A���S���r��tH�NS�Q�}��x��U��0�G���"\���o����aU���G@wZ������x��z�N~~#�o@�����] �*���.�����b����k� �����W���A���v�:��R����_/�B�������_4�����$(#hF�c)B�L�*�f�l�Lm9�����.#S(%S���zd
�d*i�6�JK��'�~��0d$�W���L$���ERb�����]�����nnop��u��MG<�zn7��q7�{�A6;�����p�~���2T��1f��xH�N6.h�L�W������?_\_����t��>�p�#.�����]wTy������W������}�Y�!�2���>��� �����|�F�)���n����z&�2��k��"���v{�{�I/��
��%d��/@���i�=��=��=���=$�T�?�'a�T���������u}�L��d��I�C���8��I�~�q���}����I'#�#xm��,X|����xY��s�
�^%`���n�����9�#��[,�w$��rJ*��"NI�j+�������h�'s#������N}n�r3�I�p������b*S)��pNgA:� �S_�'��c�:N3�$���3�a������q2���h9)�0]"9�*\n
� [Y����,��$�O�6���;ggL5���$��6'A�k�2=��`]�#�=_��$�\����~:�4���W��#��C~H�?@���B���Z��#���r���7:k�%����,Y��w�%V��/�������t�b��
	����'20�M�H�����'��PR"x
'/j�2&/\�R�8��/�M�5����-V�72���}���p��G�,�V`	��'F`k��F`�v��
�
�,��7��yE5p��O��|tGd��ef+L���H�b&~1?	3�����P��
���T�-z�V��l��evY���w��9�
�����({k������>}��z��,��J��}:"�N��UF��~��U�.u���{B`��n7"DL�n�����}-�����������i=����g0E3w��`��< ��2��$�0��IU|+0P��Y�W�f�d�a�	�C*l���r(�"`P8XoT��:p�%#�r�OG�t�:���/���F.X-c�	H�A>T���|6��C����Rmi^*����r(��*�0�p��#��������~�q��N�5G�D�H�A>T���|6���{�YB�I���r��H
J8T)�98�m"�8���[l��b�U[m���?��=���*�
����rz���\'����A��n�iVWa	�@�L����wsKGd��
F��F�V���Q$SX#��YB"�|#+u��w��xlG�k�74�6��R����[�w1����C���!����/x�0����?�+�B��,{�_�|D���,�>y5��~x��t��h��U�+����^�5
����a�U��z~w�� {�����B+��pnU\t�S��CI�b\���\1��nPS�l�F�x�F��B^��6�2���9[��p�]�������Rk�]��o�RQZ�Ba9}���r$�K�}��J�*���a��%������� ��vZ*q���1?����0
>���t'8i)�t�drb�m
<����2���:�~���ZtA>��"�V�����^u])�p���*�-,��Q��d��B�Q1�<n����edL1�<�j
y�j��h���>�}����C�:�_ ���W�����eV�!e�'��m<���������|�D��w���n�q�a�#7��m��Z�����f;�kX����(50$�1�ZfRL1�
ek3}J��XI+�r��.�
�����L5��C��������=m80b6{�t5��@t���G��cJV_LUr4��C1t�
�����f�m1tk �VsJ��w����������=�jl�-
B�����0�nA���2ug�/A�� �3���-���;�?���(@�nEvk	� �����1�������c&S�����luc<'��Q�	�$}�\��p��_����mb0x�4�.;8y�t/"����0�b`�E
)��-��m���$�(��<���~��/���OF���~��9�������v��u7���s�����c�%=<����'V�����T`�i
�|�=�+���*��-'�Y��](`o��o��0��#��*�P�����."�[�mK��)"(<����d���2���*���S���)&9yU?���� ��+s���KL��T�D3�*�`6`��$'oBP�M
�\�����$�,�����M��8�s���\���(Wj��se��$G%���8�'������d�`��~|,|7������_I�w�������c
�������;Bu�P�������w� ���p��5����V���4��aPM�m#w���cjW��a���y���i+%��+�xu�\����x�J'(]M����F���f������?���v�@I3��lI*pn�H�L]�Le�K��������v�i�6��c��A%�;�j�V�UR�
E�!����<������TwQ���Rl�Ns�������+m��<��*Z�����}[�&i�,>SNa��RFQ���M$����F��!��^U����Y`�i������#����6���.j�I�C���A����w��=�Q�(��L������]q�����
���nOkd�XV��n�%�+��1w����n��jJ������[����b��PF�<�s���w�2Z�S	e
o�]/U
Ef���%��E�n���d����Q����7�|�X9���������x�
�vwr�,�����X�F�[
���T`���\/T�t������0���3�)gS#1�u����#X��4ZRa!%�)f:Z���,�s^��rp�i���5�![K����B������0&h���=�P��{�r����F�	e-����Xpn[�����F[�`!{���Y���xF<.+^|!�$m���O*rH�dbH�%��z�x�?�0�S�Y�Ag�e��K�^�G)$:�M��*����/����q�J.B��(����/���jH�k���������M�����7��
�n�d�z(��6��KLtU$B����^���g�����������6�e�E�N���"����o����������>��6$6�<`���@E������2+��6��L��U�
�D}X.E5�1�^��(A�:RZ�c=��%�!F6X.t��D<�=�+X�_��@@��M��*����YdW���q�v��t�4�/��8�/�F��A*��JaI��`��{%e��Q�>���M������RP���-e�����#$��kL=cL=��z��z�9�@9�H!�G
�R�VY��������������[;���0���p1�b��L��)Mxb������omS+2��a5�����Y����b������v$�G�<�e<�!�
(i0�0�T���%��Q�J�}������p�(R��Y�������a��MQ*�]>����]$�|����������&���Eg'����`����/4���
������-�V1��Z*l���NN�W/���f���](�C!��bU)^���@�QF���f��:�fZW���#�VQ)�f{�z���� ���/C;`�R�AR+�J���R�]$E�uU����i�B�6�A�j�=
�J^�64�Jh��+m[7�>q����7����5�1����g��LG`��F�Z��=�����}����<��[�����3{G�
:� 3��K���,4�L����������nF��T����BK���@�W�,�a@������|���~s����T�d�V��P�y�2Y�eC-���,�JewL��7D��&{�Oy��T+���e�����$���I(����P�d��`V_��G`/�>�&w��6��T�S�t�/��l(M*���� ��%=����Meh,�*����������_'��f��jG<zY����4�3�7*��9���9%^h�?��&��Gr���m��n��$W��d�:+��u
:6K�i��!�40p���rv7��L�i���\���?�
y����0A��{%kiZs%���a��#�Oc����4��M�=Vxy|�E�}vw.����$bT_�-3�� �����7�,i�FHl����&BR=�M�'�'���&FW}�@���������t��������( b!�Rq����j���&
)/�9��!���$u�D����0Jw��:\���u�4(
7u��U��� �F�7h�RlW5hfh
�W*hgiJi*i]Q��i]���6�(�Y�����(���m�0_�����l��JMGYZ���n+i]Q��i]����6��.����W1�m�����=��:����Xb;5�;D��G�^�^&��	�3�����d�FZ~`1��](1�o�HK��j� �b��e��b_�qt�e'�`U��={�m�(&es3�H����;���*��O�j��[G@�q#f����h��ST'�d
�.���5�$
\Js� TR~�`����H�)'����iN�-�_���NW�vg�Tl�l�^/^'�4@����}.)J��;Z,/jN��6h1��Y/i��u*����j����l������h6���6�u�����y�z|n._���x�T�f���	R�r[
����'�KY=����d�q�]z=��o���w��"�����d�J�4H�i���9i�`����v���jz�R�#���V�����Nb�:�CPrT3^D���$��Z���v�?�Nx�H��?�,��h���`��%q,�c�#�$�j6{-���%=������E?-�������+
N�S|,�l1c����y�O��X��I}�c����	����)��"�����N�����v��n��BW-#�RZ������$�zw�T!�GZ��i��r�!i���5$��HKe��#-�5���BUH��HK"F�#�b�W[5$-}i����6�I�|��^�����~�c��
X�U���tl*;h#���JV
��H��Z�l�,�FY�
�.��'�ZN#���S���kC����m�
�����*�:�Xy���"�W����j�U�]��<<�q���T6��l2<K����O�#���Lu2�Sd��U!|����/���LR��Z{�mW�u)��b�5y7�
�=������U}NU�7���j�*��h;R��'/����}9e��S��������x���������kF�������en�����������A.,������-��-�	����.P������"��������M.��nr�IYHo1:��\B~)�t�`������B�:(:)T�����Nv����5*���?d���,���pG^���K������G���|9K�S��x�<��q�����29BN��\�>A'8�r5���J����+d��b����w0I������}����Z�u+��"f��C%���t(A:4Bz�=�ZH����a�����m%�m�����U���"��B�B���g����G���U0��%W��J�Z]�+Gr5H�����"�Z���C������!��������/��2)���sw���V>��A(h>!����L��01���f����7��I^����(u���I�����_v����EH9�����kE���
���
nVJV�=����I�^�UNR&�r��+j9�����A����e&�k6`P6`R�F��n������4,�%�����F���������W���S��]�O��������m:E%��*a�M����w�����+�!4v�M,�prO|�i�8�Z�"5��tG�x��>17��s9�d"�1��w�x�3^lN|c&��<��#(��������?���Ht+��
�mm�T�����t��:������i1>�#����I3T�(*d��^����*��!�(;ig2�zvF���e���'b_�:R$�]S!��$�s�4����x��.��?��%��M�5DZW�T��=�7�+����&q
�f�@��G
A�_�������8����KYr.}"�x[E�1K��M��H�����<�rE2�����yl/��i���m�/�,$���$��vG�/��K�nE2�����E`.���P���� }����7�$�p�(a;���&�[�nE28V�>N_698d��m+�d��=L_6992�7p���%���E`��y���T	�4}���'o�#����mq�s�&��
t�tb)��e�/�����;��=�f����>������_}BK���s�����(�v���Q���%<��ie���|X����%����H�u���p���;�/��g�a�EL���`�O�L�/���Tv�"�,h�R�O�}�nv��<��}�{W��)��\(�j���
���()��u��
���k/��B��C��6��
��5	O�M!*������ds�B�;.�^`���|�^�]U�1>�(s4�B�P"�N��7����}�*w���'N�S��2
A���4�Ba���
e]2�@i]�-��m���J
W�1��0��a���a��v[�vl+����]�v�)S����c�-���F�2\�c8�-$�g�-wu�GL
�����JX����=f%#2���)�c@��we?|�,( �p�
�w���h
�LyL�}wy�0��f�oFu$�(��@=E|��y�h=��&� ���G\5O���HfRY4g�_e�N���'��,K�IQ�����
���L��PT��e���]��>��Q��3R����R�D�o=��l�����������=���e��(1u�4�U�����
�����;�H��B��GiLWx�%���������GO5��M��"1����Z�,��By���-�/
�m��G�uU�dg	M�G�L�5=:V=*[1��]���7����D���=1SXm����p�O�/r�����'Q����$7!=��\�,1�L��`���f�f��gH�D��3�z�u��HtR"��2c�����b�,q�0������2�9w�����9���Y��4��ks�L�xYg��5YU6n�40��)��Su��
-����0��^�2Q"��3*���G�!�x�V�]K���h%���\p+S��O���8<���%�(�'Gd���Gw�������7
&�h��kdj��rk�Nf���^��8h:��������GP������^}�������	��Xo��|B}[@��&_Z��GP��9�\�X'�������a��2�yh�>��C�+?t�����h~�Q������)"#�����n3�y�p��u�%��x5���>@d���	;`G�L����{9]=�?.f������Fn���y��?����w��w����#\�jJ
���'-W.f:D��4�����C��7�Rh�����Y�{�G���.����K\#x����a���z�}�g� �@�����EL�n���d��V����s���������Y�lt^
O�6�W�5H��5��3��#��11
������ t���2�R�����X�������%z�9F�b5E1�"������d�w)�������%���D��}��.<�M��|��3"<�&�
��8�3\'�i8Z������F$9������x!����R`�.nH�@����p��yC��b���9��F
���FrG�c�r�N�H_�|�MG�� 
�#��e������������/��*�o�Q�/pa�xJ$9���W�@��@����|�Jv�$p$a��wkU�M��F*�Y" U�'�z����,�u-x�qS���������6v�����6a��.S��W:��5�&6)U�	X����pb�ZE�Ou�#��\
����QH��R�����b�����Q�.�KA���2F���t(��H6U�I�{T�J�%xm#���
�?�YL�3��pG�'W,v����D`7  ������������ne�
�ly#�%�a�;c�R�c�iBM�x��4`
>��� ���b��<.�4��r��%��X���$:Nf�jAhbZH�D�F���F�J�j}G#P=��S�-�9wA��9K�fQ�c��)9=�o�:���#M	o�����[��D���W�Kd�MT�U/S���������ia:�.�2{}~�:���o@�Vr�����.0O������ ���K2uC��y 
q��B#Rb8���d=\S��}����T�m��f�r�r=LP������=��l��*�pq�9_p�8`zI�{I�9?�M�P,	0��Tf2��S�*1��5i2F�h<��=c�Y�gJ�l)�S�����R��:$7/��Op_n���4�(�)�r���~�������GQ��4#�N��Q��d/���#��m��R��a$I
���U��I��"/wMO3���f>���B����z$4��(�_?��-���#{f=���[��Ur?L�-C'A.�?[�=�DHJ�obC-N���obtU ���x����A)e�Pf����8�\&$1$l�������F��+��Tc�p�zg*�V'������g5��~���K|�����4�$:b~���E�_�E�/W]>��.���.�w�^��z��n�\���.�z�z���z���-v���������Gq��bD�'�%��#�x~���'Q�_�<?�8�/���,������?���_A���(�y��{A�p�-�7����V��PK�Y-i�%%tPK|��J8patches/0016-Multi-level-partition-wise-join-tests.patchUX��X��X��\mo�8����7�#��{ou[{�=7�����C��D]Er%9i���GR��JY��l[��P"�3g8�����oa=������@���U��[k���;G=�L>��~���5��-��
�g IC�H���������z�{���������]��Kby��8�K�UW�oi�Q=2��c�A�`BV��-+C�|(�D�a�1��>��������?����w[�3:&�#&lT�3<��:��<|�
<�zn���t:
�v������!��-���=���K��ko=���INi��h�l�S�\;�uO)�4)AW�!��$�.�'���.q��o�[�_�]��f�4���s_�u��I\�nT���m�	���:����
��IX��i5@s�)��uB'J�+JV����#�_�FC7�k�t�
������v��E�9�qQ���/�:x�v�H�<`J�:�;������p��/_BGj����LC_�l�P�cK]Qf�Lac�5���x�7��e�8mX\��l�O�'y��#�����l<Z�a1z5�����	M��������Nu��9-x?�-.W���#�F����j��\*��k~5	�O�f�����x���;hJ-�s�$!%T�#�i�$)���b���&A�.;6��%4_0�x�v�_Q�e[@��".�������g�+�B�p�d\DN�A9J'PF��r>�-(<�T �|<�^�3�P�
��dz�du��7y����S�&q�*�d3q�Rz�AP:��r4�����PUS�����n
(�r)�3$4T�����M��B.
�M!w�BBS@uL	MAD��P�;0������@��a�.�����V5A��hg�l�|��m�a�(����..�����9r�b�����]��~5���+�������P@����gc�
~�=`t���W�.����x�f2 LP�s�.����x�(w��&��>������q&�?�w�0���S�_F�
�����[b�o����k���������2
�9�sM������6f�h��
��'01L�7���	j��r)r.��fP,��X�d(��X�Z8r*7�H
���"F��x������x�f�3��:�8�i\��i��g3-�7^2�7�����>�R��M����U�e4XL.�D2��w�a�^�{A�CT�W��U�%�G��i*,���d�qgv����x����h�!H]z|�_�"�k��x)��b�@�V������w3�����J$�����Z��G�x�J��T[��g,�G�A�0Y�)T��p������c�E(��O����NTC�~@P�������i��>��=t�d=4��y�Gt4G������2.%��g�^��G����m���FT����<�M%B
�[��b����ok��-
6�r��rT<f���^j��e�O�$W�h
v+h���h��1���NHW�d�nS�w��1[���0&�-������}!�A��#<
�(D1�}�G8{Q,KEQD=%�8D'P�i����G�0
z����EI��/!���R�]�df������#:��9��a��v8��Wp�
B�9}��b}���8g�5����f����D�3�W�0��&w�a:��6�
����?��`����:0<J���R�<f�T5���!�]�P*�3az���El�p!��W��a���OJ0�����7��h����VJ�F��D�����'G�`�G��E��"�U�M;�On�T����i�����E��Khn�L�n��i�$"GQ)�RLF1�c}���9�u�dV�Oxx�XR�'���TA��g����9��h1�����~�v��#����@�y%�y��TL+&Q]��f�uo{|��%����`��CmF�6��d\7`-���n���|~�8�[5Z��>Qx*N��N�]�#:Lm{Sv#� E�������G�K2W�����Jg)��(��������Y�!�e2>~�!�U�g����1����L�QK�;�(q����
��%9�q��U��~�k�2��?C3�{���f�P3+D�|�P�X���8�Q:C���f��f����?���p�zj���c�VB�6QcqR�B���,L�G�+�mi����4�J��/	����e����0�����lW�#5����0#�~5���PQ�/�%�+���dO�v�y;�]%���h��a��S�;�S-��CNni��)`�hW^aTY�K�p�?���+��T�CI��d�R��U����r���/0��
]m��N��n��G����N���C��w��X:O�wM,��>���r�}}2��{�6\��%l%x�6�N���J<�R�I�*�T�G]����� 7��+�t�-����j��i}t��������9/g�/c�G�5�v�.+�����f�
�����m�^e�%�>��@O�.C�lW���,��v�j��a��b�Ik#��n��������p����KL�y�;�������&lm;��������-
��	�Nbip��`ex��������+[3t�\�^��G��<wGEW=��J�n���[�v��������t]P�;�����ik��'���~V�wI�u���a8�c�f��|�w���.$_o�|������Z�q�jH+����&�a�|��E����I���k�3`��&�����KL�!�.RM+�Ik���nw�jH��=V�&"��t3n��RS�%�@:���_��E��1����t=�a�k�L����"O����(������E+���c������s
�lx�`���|����
��lx�dC���o��
O:�7�:�W��7�F������h������F�sx#�����r4��Yw��5�PKA��7STPK
|��J@�Apatches/UX��X��XPK|��JvQ���SG@��6patches/0001-Refactor-adjust_appendrel_attrs-adjust_appendrel_att.patchUX��X��XPK|��J����:G@���patches/0002-Refactor-calc_nestloop_required_outer-and-allow_star.patchUX��X��XPK|��J���$y�G@��patches/0003-Refactor-partition_bounds_equal-to-be-used-without-P.patchUX��X��XPK|��J��FL�G@��$patches/0004-Modify-bound-comparision-functions-to-accept-members.patchUX��X��XPK|��J�o����f:@���,patches/0005-Multi-level-partitioned-table-expansion.patchUX��X��XPK|��J������?-@���Hpatches/0006-Canonical-partition-scheme.patchUX��X��XPK|��J��C��G@��]patches/0007-Canonical-partitioning-scheme-for-multi-level-partit.patchUX��X��XPK|��J�^�F	i
G@��&apatches/0008-In-add_paths_to_append_rel-get-partitioned_rels-for-.patchUX��X��XPK|��J+�{��:��5@���fpatches/0009-Partition-wise-join-implementation.patchUX��X��XPK|��J��42A@����patches/0010-Multi-level-partition-wise-join-implementation.patchUX��X��XPK|��J>Hd�
�*G@��]�patches/0011-Adjust-join-related-to-code-to-accept-child-relation.patchUX��X��XPK|��Jn���I	
G@����patches/0012-Fix-ConvertRowtypeExpr-refs-in-join-targetlist-and-q.patchUX��X��XPK|��J�uzN�T+@��W�patches/0013-Parameterized-path-fixes.patchUX��X��XPK|��J��t�n�G@���patches/0014-Reparameterize-path-across-multiple-levels-of-partit.patchUX��X��XPK|��J�Y-i�%%t,@���patches/0015-Partition-wise-join-tests.patchUX��X��XPK|��JA��7ST8@��Rpatches/0016-Multi-level-partition-wise-join-tests.patchUX��X��XPK��
#121Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#119)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Apr 21, 2017 at 8:41 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

I don't see how is that fixed. For a join relation we need to come up
with one set of partition bounds by merging partition bounds of the
joining relation and in order to understand how to interpret the
datums in the partition bounds, we need to associate data types. The
question is which data type we should use if the relations being
joined have different data types associated with their respective
partition bounds.

Or are you saying that we don't need to associate data type with
merged partition bounds? In that case, I don't know how do we compare
the partition bounds of two relations?

Well, since there is no guarantee that a datatype exists which can be
used to "merge" the partition bounds in the sense that you are
describing, and even if there is one we have no opfamily
infrastructure to find out which one it is, I think it would be smart
to try to set things up so that we don't need to do that. I believe
that's probably possible.

In your example, A has partition key of type int8, has bound datums
X1.. X10. B has partition key of type int4 and has bounds datums X1 ..
X11. C has partition key type int2 and bound datums X1 .. X12.

OK, sure.

The binary representation of X's is going to differ between A, B and C
although each Xk for A, B and C is equal, wherever exists.

Agreed.

Join
between A and B will have merged bound datums X1 .. X10 (and X11
depending upon the join type). In order to match bounds of AB with C,
we need to know the data type of bounds of AB, so that we can choose
appropriate equality operator. The question is what should we choose
as data type of partition bounds of AB, int8 or int4. This is
different from applying join conditions between AB and C, which can
choose the right opfamily operator based on the join conditions.

Well, the join is actually being performed either on A.keycol =
C.keycol or on B.keycol = C.keycol, right? It has to be one or the
other; there's no "merged" join column in any relation's targetlist,
but only columns derived from the various baserels. So let's use that
set of bounds for the matching. It makes sense to use the set of
bounds for the matching that corresponds to the column actually being
joined, I think.

It's late here and I'm tired, but it seems like it should be possible
to relate the child joinrels of the AB join back to the child joinrels
of either A or B. (AB)1 .. (AB)10 related back to A1 .. A10 and B1 ..
B10. (AB)11 relates back to B11 but, of course not to A11, which
doesn't exist. If the join is INNER, (AB)11 is a dummy rel anyway and
actually we should probably see whether we can omit it altogether. If
the join is an outer join of some kind, there's an interesting case
where the user wrote A LEFT JOIN B or B RIGHT JOIN A so that A is not
on the nullable side of the join; in that case, too, (AB)11 is dummy
or nonexistent. Otherwise, assuming A is nullable, (AB)11 maps only
to B11 and not to A11. But that's absolutely right: if the join to C
uses A.keycol, either the join operator is strict and (AB)11 won't
match anything anyway, or it's not and partition-wise join is illegal
because A.keycol in (AB)11 can include not only values from X11 but
also nulls.

So, it seems to me that what you can do is loop over the childrels on
the outer side of the join. For each one, you've got a join clause
that relates the outer rel to the inner rel, and that join clause
mentions some baserel which is contained in the joinrel. So drill
down through the childrel to the corresponding partition of the
baserel and get those bounds. Then if you do the same thing for the
inner childrels, you've now got two lists of bounds, and the type on
the left matches the outer side of the join and the type on the right
matches the inner side of the join and the opfamily of the operator in
the join clause gives you a comparison operator that relates those two
types, and now you can match them up.

(We should also keep in mind the case where there are multiple columns
in the partition key.)

This seems to suggest that we can not come up with merged bounds for
join if the partition key types of joining relations differ.

Yes, I think that would be difficult.

--
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

#122Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#121)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Sat, Apr 22, 2017 at 3:52 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Fri, Apr 21, 2017 at 8:41 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

I don't see how is that fixed. For a join relation we need to come up
with one set of partition bounds by merging partition bounds of the
joining relation and in order to understand how to interpret the
datums in the partition bounds, we need to associate data types. The
question is which data type we should use if the relations being
joined have different data types associated with their respective
partition bounds.

Or are you saying that we don't need to associate data type with
merged partition bounds? In that case, I don't know how do we compare
the partition bounds of two relations?

Well, since there is no guarantee that a datatype exists which can be
used to "merge" the partition bounds in the sense that you are
describing, and even if there is one we have no opfamily
infrastructure to find out which one it is, I think it would be smart
to try to set things up so that we don't need to do that. I believe
that's probably possible.

In your example, A has partition key of type int8, has bound datums
X1.. X10. B has partition key of type int4 and has bounds datums X1 ..
X11. C has partition key type int2 and bound datums X1 .. X12.

OK, sure.

The binary representation of X's is going to differ between A, B and C
although each Xk for A, B and C is equal, wherever exists.

Agreed.

Join
between A and B will have merged bound datums X1 .. X10 (and X11
depending upon the join type). In order to match bounds of AB with C,
we need to know the data type of bounds of AB, so that we can choose
appropriate equality operator. The question is what should we choose
as data type of partition bounds of AB, int8 or int4. This is
different from applying join conditions between AB and C, which can
choose the right opfamily operator based on the join conditions.

Well, the join is actually being performed either on A.keycol =
C.keycol or on B.keycol = C.keycol, right? It has to be one or the
other; there's no "merged" join column in any relation's targetlist,
but only columns derived from the various baserels. So let's use that
set of bounds for the matching. It makes sense to use the set of
bounds for the matching that corresponds to the column actually being
joined, I think.

It's late here and I'm tired, but it seems like it should be possible
to relate the child joinrels of the AB join back to the child joinrels
of either A or B. (AB)1 .. (AB)10 related back to A1 .. A10 and B1 ..
B10. (AB)11 relates back to B11 but, of course not to A11, which
doesn't exist. If the join is INNER, (AB)11 is a dummy rel anyway and
actually we should probably see whether we can omit it altogether. If
the join is an outer join of some kind, there's an interesting case
where the user wrote A LEFT JOIN B or B RIGHT JOIN A so that A is not
on the nullable side of the join; in that case, too, (AB)11 is dummy
or nonexistent. Otherwise, assuming A is nullable, (AB)11 maps only
to B11 and not to A11. But that's absolutely right: if the join to C
uses A.keycol, either the join operator is strict and (AB)11 won't
match anything anyway, or it's not and partition-wise join is illegal
because A.keycol in (AB)11 can include not only values from X11 but
also nulls.

So, it seems to me that what you can do is loop over the childrels on
the outer side of the join. For each one, you've got a join clause
that relates the outer rel to the inner rel, and that join clause
mentions some baserel which is contained in the joinrel. So drill
down through the childrel to the corresponding partition of the
baserel and get those bounds. Then if you do the same thing for the
inner childrels, you've now got two lists of bounds, and the type on
the left matches the outer side of the join and the type on the right
matches the inner side of the join and the opfamily of the operator in
the join clause gives you a comparison operator that relates those two
types, and now you can match them up.

This assumes that datums in partition bounds have one to one mapping
with the partitions, which isn't true for list partitions. For list
partitions we have multiple datums corresponding to the items listed
associated with a given partition. So, simply looping over the
partitions of outer relations doesn't work; in fact there are two
outer relations for a full outer join, so we have to loop over both of
them together in a merge join fashion.

Consider A join B where A has partitions A1 (a, b, c), A2(e, f), A3(g,
h) and B has partitions B1 (a, b), B2 (c, d, e), B3(f, g, h). If we
just look at the partitions, we won't recognize that list item c is
repeated in A1B1 and A2B2. That can be recognized only when we loop
over the datums of A and B trying to match the partitions. We will see
that for a, b A1 and B1 match but for c A1 and B1 do not match,
instead A1 and B2 match. In one to one partition matching we will bail
out here.

I think, we have to find the base relations whose partition bounds
should be used for comparison looking at the equi-join conditions and
then compare those partition bounds to come up with the partition
bounds of join relation. That won't work straight forward either when
their are partitions missing on either sides of the join, I guess.
Needs a careful thought.

(We should also keep in mind the case where there are multiple columns
in the partition key.)

Yes. This is tricky. Consider A partitioned by (a1, a2) B partitioned
by (b1, b2) and C partitioned by (c1, c2). If the query is A join B on
(A.a1 = B.a1 and A.a2 = B.b2) join C on (C.c1 = A.a1 and C.c2 = B.b2),
we need to fetch partition bound values for a1 from A's partition
bounds and those for b1 from B's partition bounds. Create combined
partition bounds from those and then compare the combined bounds with
those of C.

After saying all that, I think we have a precedence of merged join
columns with merged data types. Consider
create table t1(a int2, b int);
create table t2 (a int4, b int);
explain verbose select * from t1 join t2 using(a);
QUERY PLAN
-------------------------------------------------------------------------
Merge Join (cost=327.25..745.35 rows=27120 width=12)
Output: t2.a, t1.b, t2.b
Merge Cond: (t2.a = t1.a)
-> Sort (cost=158.51..164.16 rows=2260 width=8)
Output: t2.a, t2.b
Sort Key: t2.a
-> Seq Scan on public.t2 (cost=0.00..32.60 rows=2260 width=8)
Output: t2.a, t2.b
-> Sort (cost=168.75..174.75 rows=2400 width=6)
Output: t1.b, t1.a
Sort Key: t1.a
-> Seq Scan on public.t1 (cost=0.00..34.00 rows=2400 width=6)
Output: t1.b, t1.a
(13 rows)

When using clause is used the columns specified by using clause from
the joining relations are merged into a single column. Here it has
used a "wider" type column t2.a as the merged column for t1.a and
t2.a. The logic is in buildMergedJoinVar().

Probably we want to build merged partition bounds for a join relation
where partition keys of the joining relations are different using a
single data type provided by the same logic as buildMergedJoinVar()
and attach those to the join relation.

[1]: http://www.mail-archive.com/pgsql-hackers@postgresql.org/msg312629.html

--
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

#123Rajkumar Raghuwanshi
rajkumar.raghuwanshi@enterprisedb.com
In reply to: Ashutosh Bapat (#122)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Apr 21, 2017 at 7:59 PM, Ashutosh Bapat <
ashutosh.bapat@enterprisedb.com> wrote:

Here's an updated patch set

Hi,

I have applied v18 patches and got a crash in m-way joins when partition
ranges differ, below are steps to reproduce this.

CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
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, to_char(i, 'FM0000') FROM generate_series(0,
599, 2) i;
ANALYZE prt1;

CREATE TABLE prt4_n (a int, b int, c text) PARTITION BY RANGE(a);
CREATE TABLE prt4_n_p1 PARTITION OF prt4_n FOR VALUES FROM (0) TO (300);
CREATE TABLE prt4_n_p2 PARTITION OF prt4_n FOR VALUES FROM (300) TO (500);
CREATE TABLE prt4_n_p3 PARTITION OF prt4_n FOR VALUES FROM (500) TO (600);
INSERT INTO prt4_n SELECT i, i, to_char(i, 'FM0000') FROM
generate_series(0, 599, 2) i;
ANALYZE prt4_n;

SET enable_partition_wise_join = on ;
EXPLAIN (COSTS OFF)
SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2, prt1 t3 WHERE t1.a =
t2.a AND t2.a = t3.a;
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.
!>

Thanks & Regards,
Rajkumar Raghuwanshi
QMG, EnterpriseDB Corporation

#124Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Rajkumar Raghuwanshi (#123)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Mon, Apr 24, 2017 at 5:02 PM, Rajkumar Raghuwanshi
<rajkumar.raghuwanshi@enterprisedb.com> wrote:

On Fri, Apr 21, 2017 at 7:59 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Here's an updated patch set

Hi,

I have applied v18 patches and got a crash in m-way joins when partition
ranges differ, below are steps to reproduce this.

CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
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, to_char(i, 'FM0000') FROM generate_series(0,
599, 2) i;
ANALYZE prt1;

CREATE TABLE prt4_n (a int, b int, c text) PARTITION BY RANGE(a);
CREATE TABLE prt4_n_p1 PARTITION OF prt4_n FOR VALUES FROM (0) TO (300);
CREATE TABLE prt4_n_p2 PARTITION OF prt4_n FOR VALUES FROM (300) TO (500);
CREATE TABLE prt4_n_p3 PARTITION OF prt4_n FOR VALUES FROM (500) TO (600);
INSERT INTO prt4_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0,
599, 2) i;
ANALYZE prt4_n;

SET enable_partition_wise_join = on ;
EXPLAIN (COSTS OFF)
SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2, prt1 t3 WHERE t1.a =
t2.a AND t2.a = t3.a;
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.
!>

Thanks Rajkumar for the report. When two relations with same partition
scheme but different partition bounds are joined, their join relation
has partition scheme set, but not partition bounds since we do not
have logic to merge such partition bounds. When this join relation is
joined further with other relation with same partition scheme, the
code assumed that the join relation had partition bounds set. So the
corresponding assertion failed. Instead, we should treat this
condition same as the case of joining relations with different
partition bounds and not use partition wise join for this join. This
case may be further improved in the next set of patches by trying to
merge partition bounds so that partition-wise join can be applied.
Here's set of patches which fixes the issue.

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

Attachments:

pg_dp_join_patches_v19.tar.gzapplication/x-gzip; name=pg_dp_join_patches_v19.tar.gzDownload
#125Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#122)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Mon, Apr 24, 2017 at 7:06 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

This assumes that datums in partition bounds have one to one mapping
with the partitions, which isn't true for list partitions. For list
partitions we have multiple datums corresponding to the items listed
associated with a given partition. So, simply looping over the
partitions of outer relations doesn't work; in fact there are two
outer relations for a full outer join, so we have to loop over both of
them together in a merge join fashion.

Maybe so, but my point is that it can be done with the original types,
without converting anything to a different type.

When using clause is used the columns specified by using clause from
the joining relations are merged into a single column. Here it has
used a "wider" type column t2.a as the merged column for t1.a and
t2.a. The logic is in buildMergedJoinVar().

That relies on select_common_type(), which can error out if it can't
find a common type. That's OK for the current uses of that function,
because if it fails it means that the query is invalid. But it's not
OK for what you want here, because it's not OK to error out due to
inability to do a partition-wise join when a non-partition-wise join
would have worked. Also, note that all select_common_type() is really
doing is looking for the type within the type category that is marked
typispreferred, or else checking which direction has an implicit cast.
Neither of those things guarantee the property you want here, namely
that the "common" type is in the same opfamily and can store every
value of any of the input types without loss of precision. So I don't
think you can rely on that.

I'm going to say this one more time: I really, really, really think
you need to avoid trying to convert the partition bounds to a common
type. I said before that the infrastructure to do that is not present
in our type system, and I'm pretty sure that statement is 100%
correct. The fact that you can find other cases where we do something
sorta like that but in a different case with different requirements
doesn't make that false.

--
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

#126Tom Lane
tgl@sss.pgh.pa.us
In reply to: Robert Haas (#125)
Re: Partition-wise join for join between (declaratively) partitioned tables

Robert Haas <robertmhaas@gmail.com> writes:

I'm going to say this one more time: I really, really, really think
you need to avoid trying to convert the partition bounds to a common
type. I said before that the infrastructure to do that is not present
in our type system, and I'm pretty sure that statement is 100%
correct. The fact that you can find other cases where we do something
sorta like that but in a different case with different requirements
doesn't make that false.

It's not just a matter of lack of infrastructure: the very attempt is
flawed, because in some cases there simply isn't a supertype that can
hold all values of both types. An easy counterexample is float8 vs
numeric: you can't convert float8 'Infinity' to numeric, but also there
are values of numeric that can't be converted to float8 without overflow
and/or loss of precision.

The whole business of precision loss makes things very touchy for almost
anything involving float and a non-float type, actually.

What I'm going to ask one more time, though, is why we are even discussing
this. Surely the partition bounds of a partitioned table must all be of
the same type already. If there is a case where they are not, that is
a bug we had better close off before v10 ships, not a feature that we
need to write a lot of code to accommodate.

regards, tom lane

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

#127Robert Haas
robertmhaas@gmail.com
In reply to: Tom Lane (#126)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Apr 26, 2017 at 12:19 PM, Tom Lane <tgl@sss.pgh.pa.us> wrote:

What I'm going to ask one more time, though, is why we are even discussing
this. Surely the partition bounds of a partitioned table must all be of
the same type already. If there is a case where they are not, that is
a bug we had better close off before v10 ships, not a feature that we
need to write a lot of code to accommodate.

This question was answered before, by Ashutosh.

/messages/by-id/CAFjFpRfaKSO4YZjVv7jkcMEMVgDcnqc4yhqVWhO5gczB5mW8eQ@mail.gmail.com

Since you either didn't read his answer, or else didn't understand it
and didn't bother asking for clarification, I'll try to be more blunt:
of course all of the partition bounds of a single partitioned table
have to be of the same type. We're not talking about that, because no
kidding. This thread is about the possibility -- in a future release
-- of implementing a join between two different partitioned tables by
joining each pair of matching partitions. To do that, you need the
tables to be compatibly partitioned, which requires that the
partitioning columns use the same opfamily for each partitioning
column but not necessarily that the types be the same. Making
partition-wise join work in the case where the partitioning columns
are of different types within an opfamily (like int4 vs. int8) is
giving Ashutosh a bit of trouble. So this is about a cross-type join,
not multiple types within a single partitioning hierarchy, as you
might also gather from the subject line of this thread.

--
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

#128Tom Lane
tgl@sss.pgh.pa.us
In reply to: Robert Haas (#127)
Re: Partition-wise join for join between (declaratively) partitioned tables

Robert Haas <robertmhaas@gmail.com> writes:

So this is about a cross-type join,
not multiple types within a single partitioning hierarchy, as you
might also gather from the subject line of this thread.

OK, but I still don't understand why any type conversion is needed
in such a case. The existing join estimators don't try to do that,
for the good and sufficient reasons you and I have already mentioned.
They just apply the given cross-type join operator, and whatever
cross-type selectivity estimator might be associated with it, and
possibly other cross-type operators obtained from the same btree
opfamily.

The minute you get into trying to do any type conversion that is not
mandated by the semantics of the query as written, you're going to
have problems.

regards, tom lane

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

#129Robert Haas
robertmhaas@gmail.com
In reply to: Tom Lane (#128)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Apr 26, 2017 at 12:41 PM, Tom Lane <tgl@sss.pgh.pa.us> wrote:

Robert Haas <robertmhaas@gmail.com> writes:

So this is about a cross-type join,
not multiple types within a single partitioning hierarchy, as you
might also gather from the subject line of this thread.

OK, but I still don't understand why any type conversion is needed
in such a case. The existing join estimators don't try to do that,
for the good and sufficient reasons you and I have already mentioned.
They just apply the given cross-type join operator, and whatever
cross-type selectivity estimator might be associated with it, and
possibly other cross-type operators obtained from the same btree
opfamily.

The minute you get into trying to do any type conversion that is not
mandated by the semantics of the query as written, you're going to
have problems.

There is no daylight whatsoever between us on this issue.

--
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

#130Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#125)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Apr 26, 2017 at 9:30 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Mon, Apr 24, 2017 at 7:06 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

This assumes that datums in partition bounds have one to one mapping
with the partitions, which isn't true for list partitions. For list
partitions we have multiple datums corresponding to the items listed
associated with a given partition. So, simply looping over the
partitions of outer relations doesn't work; in fact there are two
outer relations for a full outer join, so we have to loop over both of
them together in a merge join fashion.

Maybe so, but my point is that it can be done with the original types,
without converting anything to a different type.

Theoretically, I agree with this. But practically the implementation
is lot more complex than what you have described in the earlier mails.
I am afraid, that the patch with those changes will be a lot harder to
review and commit. Later in this mail, I will try to explain some of
the complexities.

I'm going to say this one more time: I really, really, really think
you need to avoid trying to convert the partition bounds to a common
type. I said before that the infrastructure to do that is not present
in our type system, and I'm pretty sure that statement is 100%
correct. The fact that you can find other cases where we do something
sorta like that but in a different case with different requirements
doesn't make that false.

Ok. Thanks for the explanation.

The current design and implementation is for a restricted case where
the partition bounds, partition key types and numbers match exactly.
We want to commit an implementation which is reasonably extensible and
doesn't require a lot of changes when we add more capabilities. Some
of the extensions we discussed are as follows:
1. Partition-wise join when the existing partitions have matching
bounds/lists but there can be extra partitions on either side of the
join (between base relations or join relations) without a matching
partition on the other side.\
2. Partition-wise join when the partition bounds/lists do not match
exactly but there is 1:1 or 1:0 or 0:1 mapping between the partitions
which can contribute to the final result. E.g. A (0-100, 100 - 150,
200-300), B (0-50, 125-200, 300-400)
3. Partition-wise join when the partition key types do not match, but
there's a single opfamily being used for partitioning.
4. Partition-wise join where 1:m or m:n mapping exists between
partitions of the joining relations.

First one is clearly something that we will need. We may add it in the
first commit or next commit, but it will be needed pretty soon (v11?).
To me 2nd is more important than the 3rd one. You may have a different
view. We will expect 3rd optimization to work with all the prior
optimizations. I am restricting myself from thinking about 4th one
since that requires ganging together multiple RelOptInfos as a single
RelOptInfo while joining, something we don't have infrastruture for.

In case of first goal, supporting INNER joins and OUTER joins where
the partitions are missing on the OUTER side but not inner side are
easier. In those cases we just drop those partitions and corresponding
bounds/lists from the join. For a FULL OUTER join, where both sides
act as OUTER as well as INNER, we will need exact mapping between the
partitions. For supporting OUTER joins where partitions on the INNER
sides can be missing, we need to create some "dummy" relations
representing the missing partitions so that we have OUTER rows with
NULL inner side. This requires giving those dummy relations some
relids and thus in case of base relations we may need to inject some
dummy children. This may mean that we have to expand simple_rel_array
as part of outer join, may or may not require adding new
AppendRelInfos and so on. We are basically breaking an assumption that
base relations can not be introduced while planning joins and that
might require some rework in the existing infrastructure. There might
be other ways to introduce dummy relations during join planning, but I
haven't really thought through the problem.

The third goal requires that the partition bounds be compared based on
the partition keys present in the equi-join. While matching the
partitions to be joined, the partition bounds corresponding the base
relation whose partition keys appear in the equi-join are used for
comparison using support function corresponding to the data types of
partition keys. This requires us to associate the partitions of a join
with the bounds of base relation. E.g. A(A1, A2) with bounds (X1, X3)
(notice missing X2), B (B1, B2) bounds (X1, X2), C (C1, C2, C3) bounds
(X1, X2, X3) and the join is A LJ B on A.a = B.b LJ C on B.b = C.c
assuming strict operators this can be executed as (AB)C or A(BC). AB
will have partitions A1B1, A2B3 since there is no matching bound of A
for B2 and A is outer relation. A1B1 is associated with bound X1 of A
and C both. A2B3 is associated with bound of X3, which happens to be
2nd bound of A but third of B. When we join (AB) with C, we should
notice that C1 goes with A1B1, C2 doesn't have any matching partition
in AB and C3 goes with A2B3. If we compare bounds of B with C without
any transformation we will know C2 matches B2, but we need to look at
the children of AB to realize that B2 isn't present in any of the
children and thus C2 should not be joined with any partition of AB.
That usually looks a quadratic order operation on the number of
partitions. The complexity can be reduced by maintaining as many
partition bounds as the number of base relations participating in the
join (an idea, I have floated earlier [1]http://www.mail-archive.com/pgsql-hackers@postgresql.org/msg312916.html) I don't elaborate it here
to avoid digression. There's also the complexity of an N-way join with
multiple partition keys and joins on partition keys from different
relations as discussed in [1]http://www.mail-archive.com/pgsql-hackers@postgresql.org/msg312916.html. There may be more involved cases, that
I haven't thought about. In short, implementation for 1st and 3rd
optimization together looks fairly complex.

Add to this the 2nd optimization and it becomes still more complex.

In order to keep the patches manageable to implement review and
commit, I am proposing following approach.

1. Implement first optimization on top of the current patches, which
enforces that the partition key datatypes of the joining relations
match. I am right now working on that patch. Do this for INNER join
and OUTER join where partitions are missing on the OUTER side and not
INNER side.
As a side note, the existing partition bound comparison functions are
tied to PartitionKey structure and require complete set of bounds from
partitioned relation. Both of those are not applicable anymore,
PartitionKey structure is not available for join and we have to
compare individual bounds in case of join as against one probe with a
complete set. This refactoring did eat some time.

2. Implement support for OUTER join where partitions can be missing
from either side.

3. Implement support for partition-wise join with different partition key types.

All those implementation will be different patches on top of v18 patches.

Given the complexities involved in 2 and 3, I am not sure which order
I should attack them. I don't have any estimates as to how much time
each of those are going to require. May be a couple of months, but I
am not sure.

Obviously we have to wait till the first commitfest to commit the
first version of the patch. So, based on the status at time, we can
decide what goes in the first commit of this feature and adjust the
patch set accordingly.

Thoughts/comments?

[1]: http://www.mail-archive.com/pgsql-hackers@postgresql.org/msg312916.html

--
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

#131Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#130)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Apr 27, 2017 at 3:41 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

The third goal requires that the partition bounds be compared based on
the partition keys present in the equi-join. While matching the
partitions to be joined, the partition bounds corresponding the base
relation whose partition keys appear in the equi-join are used for
comparison using support function corresponding to the data types of
partition keys. This requires us to associate the partitions of a join
with the bounds of base relation. E.g. A(A1, A2) with bounds (X1, X3)
(notice missing X2), B (B1, B2) bounds (X1, X2), C (C1, C2, C3) bounds
(X1, X2, X3) and the join is A LJ B on A.a = B.b LJ C on B.b = C.c
assuming strict operators this can be executed as (AB)C or A(BC). AB
will have partitions A1B1, A2B3 since there is no matching bound of A
for B2 and A is outer relation. A1B1 is associated with bound X1 of A
and C both. A2B3 is associated with bound of X3, which happens to be
2nd bound of A but third of B. When we join (AB) with C, we should
notice that C1 goes with A1B1, C2 doesn't have any matching partition
in AB and C3 goes with A2B3. If we compare bounds of B with C without
any transformation we will know C2 matches B2, but we need to look at
the children of AB to realize that B2 isn't present in any of the
children and thus C2 should not be joined with any partition of AB.

Sure.

That usually looks a quadratic order operation on the number of
partitions.

Now that I don't buy. Certainly, for range partitions, given a list
of ranges of length M and another of length N, this can be done in
O(M+N) time by merge-joining the lists of bounds. You pointed out
upthread that for list partitions, things are a bit complicated
because a single list partition can contain multiple values which are
not necessarily contiguous, but I think that this can still be done in
O(M+N) time. Sort all of the bounds, associating each one to a
partition, and do a merge pass; whenever two bounds match, match the
two corresponding partitions, but if one of those partitions is
already matched to some other partition, then fail.

For example, consider A1 FOR VALUES IN (1,3,5), A2 FOR VALUES IN
(2,4,6), B1 FOR VALUES IN (1,6), B2 FOR VALUES IN (2,4). The sorted
bounds for A are 1,2,3,4,5,6; for B, 1,2,4,6. The first value in both
lists is a 1, so the corresponding partitions A1 and B1 are matched.
The second value in both lists is a 2, so the corresponding partitions
A2 and B2 are matched. Then we hit a 3 on the A side that has no
match on the B side, but that's fine; we don't need to do anything.
If the partition on the A side never got a mapping at any point during
this merge pass, we'd eventually need to match it to a dummy partition
(unless this is an inner join) but it's already mapped to B1 so no
problem. Then we hit a 4 which says that A2 must match B2, which is
consistent with what we already determine; no problem. Then we hit
another value that only exists on the A side, which is fine just as
before. Finally we hit a 6 on each side, which means that A2 must
match B1, which is inconsistent with the existing mappings so we give
up; no partitionwise join is possible here.

The complexity can be reduced by maintaining as many
partition bounds as the number of base relations participating in the
join (an idea, I have floated earlier [1]) I don't elaborate it here
to avoid digression. There's also the complexity of an N-way join with
multiple partition keys and joins on partition keys from different
relations as discussed in [1]. There may be more involved cases, that
I haven't thought about. In short, implementation for 1st and 3rd
optimization together looks fairly complex.

I spent some time thinking about this today and I think I see how we
could make it work: keep a single set of bounds for each join
relation, but record the type of each bound. For example, suppose we
are full joining relation i2, with an int2 partition column, which has
partitions i2a from 0 to 10000 and i2b from 20000 to 30000, to
relation i4, with an int4 partition column, which has partitions i4a
from 5000 to 15000 and i4b from 25000 to 35000. We end up with a
joinrel with 2 partitions. The first goes from 0 (stored as an int2)
to 15000 (stored as an int4) and the second goes from 20000 (stored as
an int2) to 35000 (stored as an int4). If we subsequently need to
merge these bounds with yet another relation at a higher join level,
we can use the opfamily (which is common) to dig out the right
cross-type operator for each comparison we may need to perform, based
on the precise types of the datums being compared. Of course, we
might not find an appropriate cross-type operator in some cases,
because an opfamily isn't required to provide that, so then we'd have
to fail gracefully somehow, but that could be done.

Having said that I think we could make this work, I'm starting to
agree with you that it will add more complexity than it's worth.
Needing to keep track of the type of every partition bound
individually seems like a real nuisance, and it's not likely to win
very often because, realistically, people should and generally will
use the same type for the partitioning column in all of the relevant
tables. So I'm going to revise my position and say it's fine to just
give up on partitionwise join unless the types match exactly, but I
still think we should try to cover the cases where the bounds don't
match exactly but only 1:1 or 1:0 or 0:1 mappings are needed (iow,
optimizations 1 and 2 from your list of 4). I agree that ganging
partitions (optimization 4 from your list) is not something to tackle
right now.

--
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

#132Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#131)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Apr 28, 2017 at 1:32 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Thu, Apr 27, 2017 at 3:41 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

The third goal requires that the partition bounds be compared based on
the partition keys present in the equi-join. While matching the
partitions to be joined, the partition bounds corresponding the base
relation whose partition keys appear in the equi-join are used for
comparison using support function corresponding to the data types of
partition keys. This requires us to associate the partitions of a join
with the bounds of base relation. E.g. A(A1, A2) with bounds (X1, X3)
(notice missing X2), B (B1, B2) bounds (X1, X2), C (C1, C2, C3) bounds
(X1, X2, X3) and the join is A LJ B on A.a = B.b LJ C on B.b = C.c
assuming strict operators this can be executed as (AB)C or A(BC). AB
will have partitions A1B1, A2B3 since there is no matching bound of A
for B2 and A is outer relation. A1B1 is associated with bound X1 of A
and C both. A2B3 is associated with bound of X3, which happens to be
2nd bound of A but third of B. When we join (AB) with C, we should
notice that C1 goes with A1B1, C2 doesn't have any matching partition
in AB and C3 goes with A2B3. If we compare bounds of B with C without
any transformation we will know C2 matches B2, but we need to look at
the children of AB to realize that B2 isn't present in any of the
children and thus C2 should not be joined with any partition of AB.

Sure.

That usually looks a quadratic order operation on the number of
partitions.

Now that I don't buy. Certainly, for range partitions, given a list
of ranges of length M and another of length N, this can be done in
O(M+N) time by merge-joining the lists of bounds. You pointed out
upthread that for list partitions, things are a bit complicated
because a single list partition can contain multiple values which are
not necessarily contiguous, but I think that this can still be done in
O(M+N) time. Sort all of the bounds, associating each one to a
partition, and do a merge pass; whenever two bounds match, match the
two corresponding partitions, but if one of those partitions is
already matched to some other partition, then fail.

For example, consider A1 FOR VALUES IN (1,3,5), A2 FOR VALUES IN
(2,4,6), B1 FOR VALUES IN (1,6), B2 FOR VALUES IN (2,4). The sorted
bounds for A are 1,2,3,4,5,6; for B, 1,2,4,6. The first value in both
lists is a 1, so the corresponding partitions A1 and B1 are matched.
The second value in both lists is a 2, so the corresponding partitions
A2 and B2 are matched. Then we hit a 3 on the A side that has no
match on the B side, but that's fine; we don't need to do anything.
If the partition on the A side never got a mapping at any point during
this merge pass, we'd eventually need to match it to a dummy partition
(unless this is an inner join) but it's already mapped to B1 so no
problem. Then we hit a 4 which says that A2 must match B2, which is
consistent with what we already determine; no problem. Then we hit
another value that only exists on the A side, which is fine just as
before. Finally we hit a 6 on each side, which means that A2 must
match B1, which is inconsistent with the existing mappings so we give
up; no partitionwise join is possible here.

For two-way join this works and is fairly straight-forward. I am
assuming that A an B are base relations and not joins. But making it
work for N-way join is the challenge. I don't see your example
describing that. But I think, given your revised position below, we
don't need to get this right at this point. Remember, that the
paragraph was about 3rd goal, which according to your revised position
is now deferred.

Having said that I think we could make this work, I'm starting to
agree with you that it will add more complexity than it's worth.
Needing to keep track of the type of every partition bound
individually seems like a real nuisance, and it's not likely to win
very often because, realistically, people should and generally will
use the same type for the partitioning column in all of the relevant
tables. So I'm going to revise my position and say it's fine to just
give up on partitionwise join unless the types match exactly, but I
still think we should try to cover the cases where the bounds don't
match exactly but only 1:1 or 1:0 or 0:1 mappings are needed (iow,
optimizations 1 and 2 from your list of 4). I agree that ganging
partitions (optimization 4 from your list) is not something to tackle
right now.

Good. I will have a more enjoyable vacation now.

Do you still want the patition key type to be out of partition scheme?
Keeping it there means we match it only once and save it only at a
single place. Otherwise, it will have to be stored in RelOptInfo of
the partitioned table and match it 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

#133Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#132)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Apr 28, 2017 at 1:18 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

For two-way join this works and is fairly straight-forward. I am
assuming that A an B are base relations and not joins. But making it
work for N-way join is the challenge.

I don't think it's much different, is it? Anyway, I'm going to
protest if your algorithm for merging bounds takes any more than
linear time, regardless of what else we decide.

Having said that I think we could make this work, I'm starting to
agree with you that it will add more complexity than it's worth.
Needing to keep track of the type of every partition bound
individually seems like a real nuisance, and it's not likely to win
very often because, realistically, people should and generally will
use the same type for the partitioning column in all of the relevant
tables. So I'm going to revise my position and say it's fine to just
give up on partitionwise join unless the types match exactly, but I
still think we should try to cover the cases where the bounds don't
match exactly but only 1:1 or 1:0 or 0:1 mappings are needed (iow,
optimizations 1 and 2 from your list of 4). I agree that ganging
partitions (optimization 4 from your list) is not something to tackle
right now.

Good. I will have a more enjoyable vacation now.

Phew, what a relief. :-)

Do you still want the patition key type to be out of partition scheme?
Keeping it there means we match it only once and save it only at a
single place. Otherwise, it will have to be stored in RelOptInfo of
the partitioned table and match it for every pair of joining
relations.

The only reason for removing things from the PartitionScheme was if
they didn't need to be consistent across all tables. Deciding that
the type is one of the things that has to match means deciding it
should be in the PartitionScheme, not the RelOptInfo.

--
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

#134Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#108)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Apr 6, 2017 at 6:37 AM, Robert Haas <robertmhaas@gmail.com> wrote:

There's a relevant comment in 0006, build_joinrel_partition_info()
(probably that name needs to change, but I will do that once we have
settled on design)
+   /*
+    * Construct partition keys for the join.
+    *
+    * An INNER join between two partitioned relations is partition by key
+    * expressions from both the relations. For tables A and B
partitioned by a and b
+    * respectively, (A INNER JOIN B ON A.a = B.b) is partitioned by both A.a
+    * and B.b.
+    *
+    * An OUTER join like (A LEFT JOIN B ON A.a = B.b) may produce rows with
+    * B.b NULL. These rows may not fit the partitioning conditions imposed on
+    * B.b. Hence, strictly speaking, the join is not partitioned by B.b.
+    * Strictly speaking, partition keys of an OUTER join should include
+    * partition key expressions from the OUTER side only. Consider a join like
+    * (A LEFT JOIN B on (A.a = B.b) LEFT JOIN C ON B.b = C.c. If we do not
+    * include B.b as partition key expression for (AB), it prohibits us from
+    * using partition-wise join when joining (AB) with C as there is no
+    * equi-join between partition keys of joining relations. But two NULL
+    * values are never equal and no two rows from mis-matching partitions can
+    * join. Hence it's safe to include B.b as partition key expression for
+    * (AB), even though rows in (AB) are not strictly partitioned by B.b.
+    */

I think that also needs to be reviewed carefully.

The following passage from src/backend/optimizer/README seems highly relevant:

===
The planner's treatment of outer join reordering is based on the following
identities:

1. (A leftjoin B on (Pab)) innerjoin C on (Pac)
= (A innerjoin C on (Pac)) leftjoin B on (Pab)

where Pac is a predicate referencing A and C, etc (in this case, clearly
Pac cannot reference B, or the transformation is nonsensical).

2. (A leftjoin B on (Pab)) leftjoin C on (Pac)
= (A leftjoin C on (Pac)) leftjoin B on (Pab)

3. (A leftjoin B on (Pab)) leftjoin C on (Pbc)
= A leftjoin (B leftjoin C on (Pbc)) on (Pab)

Identity 3 only holds if predicate Pbc must fail for all-null B rows
(that is, Pbc is strict for at least one column of B). If Pbc is not
strict, the first form might produce some rows with nonnull C columns
where the second form would make those entries null.
===

In other words, I think your statement that null is never equal to
null is a bit imprecise. Somebody could certainly create an operator
that is named "=" which returns true in that case, and then they could
say, hey, two nulls are equal (when you use that operator). The
argument needs to be made in terms of the formal properties of the
operator.

[.. some portion clipped .. ]

The relevant logic is in have_partkey_equi_join:

+               /* Skip clauses which are not equality conditions. */
+               if (rinfo->hashjoinoperator == InvalidOid &&
!rinfo->mergeopfamilies)
+                       continue;

Actually, I think the hashjoinoperator test is formally and
practically unnecessary here; lower down there is a test to see
whether the partitioning scheme's operator family is a member of
rinfo->mergeopfamilies, which will certainly fail if we got through
this test with rinfo->mergeopfamilies == NIL just on the strength of
rinfo->hashjoinoperator != InvalidOid. So you can just bail out if
rinfo->mergeopfamilies == NIL. But the underlying point here is that
the only thing you really know about the function is that it's got to
be a strategy-3 operator in some btree opclass; if that guarantees
strictness, then so be it -- but I wasn't able to find anything in the
code or documentation off-hand that supports that contention, so we
might need to think a bit more about why (or if) this is guaranteed to
be true.

Partition-wise joins
may be happy including partition keys from all sides, but
partition-wise aggregates may not be, esp. when pushing complete
aggregation down to partitions. In that case, rows with NULL partition
key, which falls on nullable side of join, will be spread across
multiple partitions. Proabably, we should separate nullable and
non-nullable partition key expressions.

I don't think I understand quite what you're getting at here. Can you
spell this out in more detail? To push an aggregate down to
partitions, you need the grouping key to match the applicable
partition key, and the partition key shouldn't allow nulls in more
than one place. Now I think your point may be that outer join
semantics could let them creep in there, e.g. SELECT b.x, sum(a.y)
FROM a LEFT JOIN b ON a.x = b.x GROUP BY 1 -- which would indeed be a
good test case for partitionwise aggregate. I'd be inclined to think
that we should just give up on partitionwise aggregate in such cases;
it's not worth trying to optimize such a weird query, at least IMHO.
(Does this sort of case ever happen with joins? I think not, as long
as the join operator is strict.)

I am revisiting NULL equality in the context of merging partition
bounds. In [1]https://www.postgresql.org/docs/devel/static/functions-comparison.html paragraphs following

--
Do not write expression = NULL because NULL is not “equal to” NULL.
(The null value represents an unknown value, and it is not known
whether two unknown values are equal.)

--

seem to indicate that an equality operator should never return true
for two NULL values since it would never know whether two NULL
(unknown) values are same or not. In a paragraph above, Robert stated
that

In other words, I think your statement that null is never equal to
null is a bit imprecise. Somebody could certainly create an operator
that is named "=" which returns true in that case, and then they could
say, hey, two nulls are equal (when you use that operator). The
argument needs to be made in terms of the formal properties of the
operator.

But in case a user has written an = operator which returns true for
two NULL values, per description in [1]https://www.postgresql.org/docs/devel/static/functions-comparison.html, that comparison operator is
flawed and
using that operator is going to result in SQL-standard-incompliant
behaviour. I have tried to preserve all the relevant portions of
discussion in this mail. Am I missing something?

[1]: https://www.postgresql.org/docs/devel/static/functions-comparison.html

--
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

#135Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#134)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, May 18, 2017 at 4:38 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

But in case a user has written an = operator which returns true for
two NULL values, per description in [1], that comparison operator is
flawed and
using that operator is going to result in SQL-standard-incompliant
behaviour. I have tried to preserve all the relevant portions of
discussion in this mail. Am I missing something?

Yes. You're confusing friendly advice about how to write good SQL
with internals documentation about how the system actually works. The
documentation we have about how operator classes and index methods and
so forth actually work under the hood is in
https://www.postgresql.org/docs/devel/static/xindex.html -- as a
developer, that's what you should be looking at.

--
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

#136Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#133)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Sat, Apr 29, 2017 at 12:37 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Fri, Apr 28, 2017 at 1:18 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

For two-way join this works and is fairly straight-forward. I am
assuming that A an B are base relations and not joins. But making it
work for N-way join is the challenge.

I don't think it's much different, is it? Anyway, I'm going to
protest if your algorithm for merging bounds takes any more than
linear time, regardless of what else we decide.

Having said that I think we could make this work, I'm starting to
agree with you that it will add more complexity than it's worth.
Needing to keep track of the type of every partition bound
individually seems like a real nuisance, and it's not likely to win
very often because, realistically, people should and generally will
use the same type for the partitioning column in all of the relevant
tables. So I'm going to revise my position and say it's fine to just
give up on partitionwise join unless the types match exactly, but I
still think we should try to cover the cases where the bounds don't
match exactly but only 1:1 or 1:0 or 0:1 mappings are needed (iow,
optimizations 1 and 2 from your list of 4). I agree that ganging
partitions (optimization 4 from your list) is not something to tackle
right now.

Good. I will have a more enjoyable vacation now.

Phew, what a relief. :-)

Do you still want the patition key type to be out of partition scheme?
Keeping it there means we match it only once and save it only at a
single place. Otherwise, it will have to be stored in RelOptInfo of
the partitioned table and match it for every pair of joining
relations.

The only reason for removing things from the PartitionScheme was if
they didn't need to be consistent across all tables. Deciding that
the type is one of the things that has to match means deciding it
should be in the PartitionScheme, not the RelOptInfo.

Here's set of patches rebased on latest head.

I spent some time trying to implement partition-wise join when
partition bounds do not match exactly but there's 1:1, 1:0 or 0:1
mapping between partitions. A WIP patch 0017 is included in the set
for the same. The patch is not complete, it doesn't support range
partitions and needs some bugs to be fixed for list partitions. Also
because of the way it crafts partition bounds for a join, it leaks
memory consumed by partition bounds for every pair of joining
relations. I will work on fixing those issues. That patch is pretty
large now. So, I think we will have to commit it separately on top of
basic partition-wise join implementation. But you will see that it has
minimal changes to the basic partition-wise join code.

I rewrote code handling partition keys on the nullable side of the
join. Now we store partition keys from nullable and non-nullable
relations separately. The partition keys from nullable relations are
matched only when the equality operator is strict. This is explained
in details the comments in match_expr_to_partition_keys() and
build_joinrel_partition_info().

Also please note that since last patch set I have paired the
multi-level partition-wise join support patches with single-level
partition-wise join patches providing corresponding functionality.

[1]: /messages/by-id/CAFjFpRd9ebX225KhuvYXQRBuk9NrVJfPzHqGPGqpze+qvH0xmw@mail.gmail.com

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

Attachments:

pg_dp_join_patches_v20.tar.gzapplication/x-gzip; name=pg_dp_join_patches_v20.tar.gzDownload
���"Y�=iw�F��*�����	i�N��/�-����WRf6;o6E� @��$�}��q_<$9�����D���uwC^���`��WOX$,��N��P���q�JVeY�%I���$Y��W�?%RqY��|e�u���~��?�����D�]����K2�����E��|/z���+����9A��c��a�Ch��6x0�j�_��WY�
�OA�����{K���6f����H�����,Edk�����
�T�����5[�<I:��@�-l�c�D4��L�
�kL�����e.2���6��-o�����1�,�]���_[��U�X����R�z=��Y�1������l>����<�q+n��[Ap+x�v�=�vo[���������aczoX���;0��
X9P�V+��~���� ����i�g���[����G��v�h�������Y,�Bi�s���{�f]�4�����F�>�`�
3�0Q���Z��9�z�v��v�L������L}*�M���46c�2
��������z�o�����������q�9��2���{�������Q�Q�n��w��������R7���6������d��\1G�={��0L�y�
�,88���������ih�����k"I����3��gp6�p1�����]���T��E�I]o���E�8&q #�aD����U��P�NA��k���v���=�0rD	b�f�RAJp
��y;'
p��G���e��p�!���7�������Z���LN	)%��i�3��#��v�-���t�Wpg>Q;�9�#�L7^����;�C���T]���R�w��d�;�.����{&X��>��B�q�r�/CF��;X1���,�r������"f�]Y#fi�����������S��rz>�1^]���@E���� >�AD	��[{/���s0"q��Bx�s�3�n�62��0���H�:����=W�,�y\��"4�|W`�+�lqE+������am:;.;��r�>���2�$Dl�����%>�xPID��$|tp�c�@K��.���S������(N��@���c���1������F5����TA*�Jsf4�F�,}�
i[v����43UF2kf��X��F��Be��c��/�������9����e{���d�������	U�f��$v�(�#9�Xr��H���]dwa�?������D���3��E������6TG�����0��[R�\y(U=Jr
Y������w��=�o���`F�nd��(b���+?B�� ��\�x~N���<]�Q�(�@�}�j�2��`�H��CJl9�w�Z �s�;��)�VK/$A�>��
qB�2]��R��3�aNy�$�!@qZta�\��1R�8�BPp����=J����!!a�lnU�k��p���d"�}��'�@)��]�X�At��0c�	���,%y����������u�b�c`:���{X�)�������-���&]���
�Gy�8&�y����m	���<�:���������>�N[��)�v.�+D=�U�9�.��q���Q�����a����{8�����J�q�?�M�m��C�~�Xp;[���X��eK����!�%���Ckq���[�*�=M��2������p��a��M\�}d>����A�%�m}�1��p#	:��D3�v~v��'�:���=M��F��FZ����I����{������)5@~6�Ql�&�og�9,��,-ZD��n���q��w$Z��	����8�������}��*�����i�;s{�l��F�E�����i
�uA�C�����@�������_��?E���SK���@����5����|8�������9�u6���S���Y�������������K#~�F�`%��;@J������@d�V���`�����U�  ��4��I^�o���pe���kdrp���uz������+N#�g�z������gr��xe��;2�c]���xY�v8�'��6
G����|� S(�����@��B�@���	A�z�\����~;����{�~����:J���k��h�f��N(Bj��P�K����F�p]���P����s.WaT��_b�pq�c�+����9qC���9��v�v��Cc�PU�S[��7N��>�'��7�G���pur����k������������q3�����F�� !M2c.���M�O���T�vW �jd�3���x
��
2�1+��S.m�9�9vf@g��������I@��&�4����v��|��������L?��=�0�!���;��7�k`5Bo���]�]�O��k�5�5�8��a���q
��):��2����_C)p=�7[/��qV�g��W����b��%��8��{����o���j�VG��E�i�k#m:��%K�UU�J1�5��|WRKy(uQ+�o��R�U(&~;�w'�c�,@��7�-�l������5���������R<f
3I{��CW`��f ���*;�oxKneS��.}�1����td��"�1_���<���LYx��������'Z���p���� k"��4Fb�c# ^1����p.f��r��B��(�2��$}\�K�|EhF����,�����f��9�bm���8�WV��Z������_�e������nu�����p�����J�B����b�%��e���]Z�X~��+k�&�i��G����������.M�e��#U
M�H+�������Q�_�;w��It6|��=���Z|4��k�rC������n>
D"�*c0#�p.�h�.�]������3H i�WO��l	�2����_���w�����+�
w:��?H�����+���:0
�Q�����~&#�X��%u�K�?���%�����25�4�S1��`�T������x ���:��F������y��pk�>>nM��nmU��Vk��^HQ}�1	��������
���c-z� {\U�m��P���b~E��k��dBoy#i�|1 A���~�-�_�>�����9�tu��@��
H���{4P�*3�C��\���s����L��(�	����t�}�-^c`�<�1��{tl����0����(6iy�K�������);7������;�(������a$,W���yv	�(���<Jq�,N/��-��l�.��*���x�<�?����W�SL�8}����7BK� <��{�.3Ya����2Z�8��Oy��Q>�t:�����E
���8���|���r�����1)�������xN����7��IQ(��^r��]�i]a�p��,w�/&��X����������h'�?"�o�B����������Q��b����-����)��_����[��p�q���][*�z�&�X@���7.R;�f�}�9����E�����i(����zj9f�3������d��:~�����"=!��x ��]��$�KJ&������q9}q&h�v�$DK%1��gO�E���<��-yw����y���a���W(������Z��������$�����&mU��4�+��p4�t0���h�h#k<��O����6:-��i����cy\�����o��9�V����-�f���#�������1���'��B�:��xC�9)nMp��[
������ @��5Y���%������f��h4��b)��N�j�9<�@���j��T<����F�������3W^������uUx���y�{��ew���,%r�����(���c�����#w�8�v��������r�8��o[���
h�(��LY���`�[,u����������xLh�SCtxuu2A��i��e��������T|Y�����w 9��~�����'m��4P����~y?����O�<��������:�s[��ZH�v���6������
z�3��Ksg�9���7��t���o� V�]Q� ������OT���w���)j���n��6���A8�1�;��>��A�`U������W7�v���EG?�3�/���!�N�������A�R)f��.��1���q����������X���(T�U�ewQP�E��~�������((��P�Y�jQ���fQ��^J�((���R+
u�EA��B$�����DA���f��J�-O��^���#H�al\t����������E�����5.��C����}�K�&�J�?-�^$����*��7'W'|��4&��yJ�"\^�>����cD����?<�����k�@�27�u���=?�H����s�xU�%DF!�&U�A�&�T�������1�9^pp�RGs�>�5]�G�>�;�R�� �P�Vd�C���������7���*
9Z�Q���!�"�a���a�B���c#����	Q�q�(���i>��+�s��v�i5�i�QwZ�vZ�}�i����t��R�|[������O�jM�Z����C�F#e�{�+��#'���"����_���RfL��-Zv����R��5s�<�<l������RT��h�+�g�g+%�^�+[��c��@��������]���p����=NV^eu���}z��=v>_��==�2��!�b�E�z3�'q��c�K��g�G�BE��T�F����-��R(gzS
�E
��(��BJ�B��~�	
�#[n�Za�P��K��qm��W*�+���=���V	��{�U���`�e%�j�\
���:����<��r6`g���������;J�Fl�?E�k�t
Z����%�|G�b����[���K�����ro�1���I�V�^�f���v�\�pS��PQ��������4{8����<��5N����SQ�QQ-R��J�xr�a4���_���F�[����<�r)�n[������X��S������.Z�b��D-r�E��g��G9?g�v���9�����F��������x~.��<L�����h��
��y�{�+��J������.U��M�a���)��� 43��L��q�!�������T��n����+M�&-��}I���TT����'Y�ON�����J��J%q�:���nJ~r�n��z4��E�U+9W��\u��5S���gZ.O�Gw���+ ��X�N>��RIf=^2+�=�/)��s�ogU]|lt/���L�'7'W�s��'$��=��5~���������=���039ovix7��bnb%v����(]��Y�L�d� B-�P���xe9��4�uw�V��R?�{Z���;�w�������VpN�
)�'8�+����-�\Et^�=�*f)6�(jWg7g6fj����M�G����Y0k���cl�CO1�s������;�U����.7��T7,�b�l�(Y�0J|a��������q�sFNC���]n&q�nX��p��bQr���8CyJ����kg���b3�������6i����������)�t�|���jq�p�.IsQr��^������������p���o���4��	}Z��Zp)�������"��rU&Z��������(�af�rUi�]�T�F���!��T}�Q�s(;�C-��J%Z��o5��0���:��Z�VZD+��J�y}�u����l�\�=c�����IN�H�d.q�K��x��c�<�F��}���7b�m}����ik�H�Mkn��I���wPv�i��}�]����qX�T�#D�tY��^����M'`���w������+^8.u�^_V-���j���dm4���}7��y������I��P|a
9��z����<����?\�-�d��&�����[�[�����O#�@a@O�������Xy~��
���a���o�D�f�����=��-����\y���e����8������m��(0xH�vi����
E��_^?x��D��@47�w��szP��N���I��T_��u�|��0�]��@V�p<X��0]�@�f����i:��Lb�t���J���|y������r�7	���{��|�����%�x5���P|�&���k�O���f����1]P�Jh�*�|6��6�U�*����jr���F���.����0�C�\M�v��N�5'� �#	�7)9�a3��E�a�G��@����
�=��Y�����~��r�G7l��E���"6��i#vI�p�J���`��
;���k8l��~O��{���6�%a��)~O��{�I��O���W���&�71������(��]���G����'�~_g1�u����~�����mn��;�.&�]�D��&F��<�o���t����n������?���j������I�<:9I��a�(=l�&�n������/V�1R�!9~�����;�����RyF����1]Zu�Ru�)��k�U�+�l�^�c��6�2G�R9�����z�m���	��J^�B��J�o���$P]��BY�&�[[�������)��Sc�J��(+�9|�gg5��O�W},����7�f�����5���X�@<��@|\]�oV��KUH�>���7��IV_�o��<%��\�<`�4(�$�~�M��ZaI�!��N�\����~A��7t�C�Y��8r��n{c�M�K���`6f
�PP�D����U:a�D�R^��n������^��?�d��A�1�������|j��v�>���y��������:9a��kE����P"�Sh+UP
���0n���H�r	����!�w����i_��C��m�D�Gq3j�����p��.���j����\�1��
��z��{��/����U�J{�aI��M���s�6n�m�����v�Y���t;Iv�3:9�������g�����~�EZ�=��Z�'I�������N���D�f�,_-�"��\��{S�$_/�6��4J��2i�uT���W�|���1�|$57�z��J�l�g�hq]z�j�zq�:p���-y��-��D�����{�V��+�b��]�:�J�+h(h=u��G�@%�������!��x6F�NZ�.�I���j��z�<���G�����.���=RWR=x�����)�zA�
�L�eH
k"������[eRjs�W��U����sD.�}��Q�_�Z�����@	�T��c�����<����.~��u3���~��2�k!�A��O#��r�00����.�d��b���j������\��z,�:�?��_�Y���,�Y���R������Y����EW�2�����0
z�P���J�h�����^���QIRS�������L�������P��K���z����XV��uu��t�
�Rx�:w������)]�+�}�X�Hj����Ft��O���A���[�k���ueW�uMC�����Y/�0SM���\)���F�s�����G�Pq��35�����������@�^6;��e����E�y���Be�9\�pL�5�'JkxT�(H��������!4�u0�9�</�mA��1��r�g@3�\q�����*�RT4?�\����1����}����U��S���R��v��>�,�tx���h����
���{��{`R�������Z�jc�A�L��l���I��h�N������
-���
mS��OZ�jC�'7���[XK��(b�d:�ST��~��5�(�yb9c
�H������P�3��=H��F�F�@��?Dz���`�}Tf���a2���4�KR����A�[�Ud0�l0_
��r�a�@���+��v�����)�����nS�PJ�$�Ty���������%����P*r����&<v����}��=v>�?��?��A�?�is��it"0���3:�|���\�5T�\Vh�{,h��a��.d�)�{
��g���|Y� )��M������x4h4N���d�&Kl���M�lR�C�w���1#����;��N�_�_3�����6Q��+@���(p
s��!�J>��"#���{ls�z����������S��]��F�`5_Z�O��%+�1����~V_������|�������GI��P�������k�#�O39n���I����v����vc8&��h��kw��/e�Yg�q�?�G��
���d������Z����U	�Vl9�?f����U�5~��CJ��s�kd�0w<�
�]��r��C:���7Jp9�����AqT�2L�q����4*o����A�f�,06���(�g�9���)s� �S�>�����}JL�l~��Z�~���l�j�yf�Z�d�Y�[�_�x�pB��cj5/���H��z.�$����G~l2�6*�s�s�PhT*��A�A>��=/�-R���E�L��f}�ep3�W,D x��Ts[�K@�-,���*���d�a���b�M�#d�s���~�E�Gp&���S���t���vh�i*|�j��������OG�fgt��q��$-�b0�RR�2Q��F��x�k��<�A@���Vu�'�x1V4�s��g$h`���k>��\�U��cR����P9�KA�C�j<�$���#���!�hVv�t�5%����E'R���s{���q�\,�v�������@
���z�o�+�aWc�T�]�D�1t�}@�%b��k����)y��u
�c�J�E*�~��u�H��`%�>���BC
����I#�?�J
��zz��	�Z��2���l����;(OG���(��`���A������I��$]���A6����"����K6%l�����:^�n�P����2��7�Ep�/�x�-ET��f��Y���P�|4E�P����E������]^�^�}��������p�w����eH��I�%�	-S�H������wh��@Iv�>l5���u����L���4$�f�$�J?��C��������k@;�J�f�^?��$ta�>�1�#��#S�i���������O�UO��
.�:�m�Ul�N|�Y+�e�x����CIw�o��F��O���m�QTj�Z!������2i����sPd�[�@Z���zPxLr�(>���]��e�q�.����3)�o����x�8��M�m�����P�!�o��Ba���f�M/fc��5a�N��YoQ
��k��}@m�}3�]e+�����I�g4/���������u��S��0�;8�������v�(������~����H��n����������G���(=l��'C�u��'i3�������r\SIG�JE����f��[���}����J�������8r��i���l�i4��l�`O�pA��A�h��+�J�@����g��d����W�Wr���k2p����"C�c����7�Q����
 �g���s�������(����R9\�}C"KU����s����\����W�AK6HE�t��������<��k#b�P�o����O��A�d2��0L
^^�Uw��~�|=��N��,�\)�}��YU��I��D@�
���a<_��;V9�E������
Y01s�Mz�x 7��j^Ye)v�F��|�ruM�D�8�C���XS�Y�9���M����w������]�y�xp��M��Tq���z�yOj�$z�����j/�Ic�rI4aq��������z~�7��|O�F���]�x4��c8�\���`�VH� �g�Uq��Q-����/��e�c<���������lA\8�0�����Wb���(��A;���m7�����z"��D��B]��Od.��;9i����,]wqK?<	����&��P��1jD�%�~��������z�9�z�''��J���>��<\�R������;�$E�1�w�N��!����ac����������w�����<_��6���u��0r�U�$|}KI��s�}I���_�x�o�X�[�2�����K�%'�VK�����B�`����262�M�(��BvC+��+�p���AC+m/].��^�SV\���!�������p��w�������Ze��`��W�y �u�:
���>9=�`q�&�r���������O�����n?e�u�?=��t���U��=uqy�&�#������"���H&p�$��+���b��������M�*�,;�W������[�/z}����n���^>�S�;����f��ZO��@q�dt����0z���"�!npu��t<�N`�|��%!~zd1����p>�X2M����HWJ��K��s�uI���SX?���I-�f<��g��g�{�g$��0h��Qqy� %�EO����'S�jr*R�":K��N���3Y���F�h�iU:h
��F���w�������d��3���L��
+mBA���$���X���z������9��KP3��x�C~��e�����VE6�w�^��� c!g�sZ���|:����A2�]/���`P�����^����}�
�!>x��4<C������.P�	�Qx3�r"s��I���9���=�����?���cjt=��*��sd?�	�2��n<�rL�9�>=��sF��������K��"���=@:��)���^�!��[�]a�[>b���~�:{��F]���c3�_:01l���8(�h��8�I�\#�-��hhu?�#����:�.|��6r,p�;-8���8P���9�Eb�}o	*;�e�K�XVq�O(\����-�B��A�V�I��D��0�i�={�If��l�L���@?�.���Wp��~T�)2�{\���:��k��56D�+l����l�eX����3DU ����>��^�3�b]�E�1�9	�, f>E��I�Ws��n/%�"\����;��H�t��o��6�d��j����/5��A1����0�?.�R:i�7_��c"&h�i|��]!��"��$Yk�����K���x��$G���F��l5���{�;���o�.�����5f@���H@��Y/���Dk���>
}�p@���9#����Br�2$X�+�����c�	�_n�����
���#���w=��gP+�� {�rA��z+��"d��Yz�Z��'X@�_:��5�n��!����<.���X$Lq�*��������d��$�����J�U�QOt-a��7���GpPn���-�2-*��,�l�US�C����w0_��)�<^U��Qh���i�Fh����Y�uZv����0���M��%�Ars�*����~����+</��W�����!��1Ds�������W��jlz��!j��H���,���j�����4z�c���d3Lzt�@,���	���P���������YeE��@�%��(�b������/?H���w��^�)V/�'p`�/�79�/
�cH�9����."mH:S1�4�,�a"�\p+Zm,`P�)����5r�-�Q����3�3�(�����(��3���"57����(����WV����3�Q����(<��f�T��
zi:%�+I0���9(�d����������t�.[]�W�K�j��mJ�F�m�	l�KJ�/c�I�0\1�k���{�w�����^
�����s���S��}:�:P���`x��S@	���%����|��/�����������u;�,�?���46�����dK�r�3dz���c����s��.}���[����.m�gZ���Al1M����et�+�����DZ��8
���XC���
g�Z��vK
���*�ut�
��pB
�G���.V$V�{��y����
�s��������*��7�$]o=�r�6�\������{DX������zs��{\�b���7O���K��:%��N4�!�=�L��G�o.\@�>���1�5d�[�d�HZ'������1u���������L9e���4RK�L��F`���+v�s�
t�s����{�E���1Bv��"F��r����~MY��:�QhP�����C!'���Z���u�D��>�����q�g�����R����?�S"����=��_�R�
��I��u���y�.)����h�fH�.�K"��~N��0����o���C
����=�\�M���^WH)Y,��'�xBq�w��&��4���80����P���"�G��*P_T���^�%��Gq�(Hi�
l��XO�a>'^�mNm p��m�k�c5� ���cC�]�� ��w1/�b
B���&��W����������|P���'��
UK!C{i���)�9Ird>V�1u7*���h�i� )5�T[59���!���q����������8�v�����`�]m�'�T��32W6KJ��y�#4��r��\zsr����6���	AM��4
&G8<"#p�`���v~����\i����t��
�g�c�4�����=�����&[�
e��o6�	a��
P��m2u�y�yq���d��d��~�-uo���eBfEk��r�����a��%=8
B��%4=&�~�e��Lf�+Q9w+���l��3��.�H2Nr�wI����������16���p"
s�B�
=4�F���������+����'��P�EW�o��!_|�J��
��w�qS�:v������p���:�!�q������3U�C�d��%.��8���i�P�<3>�
�_!.��
��$$��E	��4�d����QaT��,�Q�B�9I)O���/�����z��l
��-z�oZ�C����t[���N����i��5���C�������"W-�'�^X����;+����
���8�]@9Im]���R��6�YB�rTF���0I�K��W6*��g�������T$���g�rMx�lT��w��8������F:��A����b0��k�<�o�%
�`��W���[��t��~��u�.���i�{��M�FB��7��5��DKB��X����A��X���O��W�!b����J�����M�$#W��M_9��@6�#���w�G5���3��j��9���+)D�a1���a��#�3��q��M�<�t��a9��=��S�L�@��8^��G�1��"M����q���������-��E[�R�*ww
4����}���U�������`�������@�[��mQ�\�����(���
?)
6���Q�>��#y�f5	d���x���VC90��q7.A�Y�N��0dK;����!F����������q2m�Nv���+g������ .������b�����{�*h!�A���	��E�S����RT��`�"no��nm�kvg%�/��|�l>��a����J��Q�x1������~n��!8A�[��7���1c��q''G�(�t�Gq����	��1$pg�?�}����{�5��s���?��^��}����3�Ow�����'��Q(����wp���n{1�)Es�=�`������?��O�Oh��5]�-n~QR�G����L}�[_��f�����W*X���*���[������{�	r}�7^���[���N}o�������z������W/z����a�7�����W���G��w��sx��7^����s^������������X+���p��6�P���y�^u���^u�tH=�����r�=:<��f��MF�R=�L}p"��������O"�pNx~7�O��x����'tfA~9�-�+���n��0�O�3%p�c��t��p������j
M�=+� �s�J#;c9���f\$��O�&N
�#�����O����{���|F��tj���>��M�4�n�kV���5�@�$	����
�z>q��������3���m�����'��-�9���W!��D�@��*��������{_��BOO�|�P���5��	�q�t��$9�|9�"V
N������
i��_�&�(s�ZNt��,�4;��,�
���"Z�'��1���k�������5br�".=&R?mq�����s�������>������ig�O21)��g�0!�]��+~�w9:R�����	�����E#I��
3W���`;r�-�&?���(�+�t����ju3L���1*�4BTvJ*f�-�����-k(��$�k����CT>��:D��n�#�?�M�����b�����,����zq��4� �n����,��I��D�d�A�y~5��Y�!�
e��^��y�N_�$N:��Y���Ll���J<E��u�e��k�������?}wv�����g��T�P�� �9�iB�v����?g���Dh��:i�����S��6������������|�(8�Y��-k���hl��	^��{$��Q(���D�R��w���(_1��y��G5��L��Z7�q=^f:�A����k�*[�l2���/����������&f��_E�e������M:��h�1O�0;I��i/H��#uc��o�7�t����V��Z\/��JN�c/�I7rr��I��F�����e|�V"�]q��Z���}]E���}����h&�>9P�+S�a*Z�.R��Ir�
�X��f�:�����`��t�yO���`;G�_���1{�Lix�����6��g$��e�<����R�����9�x�k����SRvD4���JC>\�Q��m���,3:|1R�^_h�|<5$�8b
l:������S^ ���:'M��n+�E����dT�����H5�x{j�&h�5�Wp������6����\�9�E���3�[c�����s��:�Pd�[(bq��iK��������6[Ns�����C2�
:hm4��p�d�����r7��>�������6
�t�W*!/Wd
S[�$�LW��9=�
�h2)~o�+�����=����d����V��c>�|hB8a	����2���m�����VS�j�����`w[��O��1��lE�1�>�w�tD����r,}�|o1���s�[��<�`)�v)\�L����=O�w�[��v��=:��u���xd�{|Hd�0i����M*;	��}^��y��6[7��]M���O��e���?RB�!�)q�cL�T��������|L�Z�
�"1��R�2
Lp_��J
/h��'1���Nt����u����tO�fQ���U�oo�Y5)3(��Dt9�Eu���#A)>=�J4��/u~$'��j<`�6�D���l��Um���n��r���6����� �z���`��m��
F1r:��N�qTS4h}�\�K�I
���]�e�V5E4*�D>#-�;tTs{��C:YsA�jk;�dwhIt���}�w������rp2�����`�w�'����]��I�j��p5XA��>���g���0���9bt���~�_9@,�(L������1�>!��t����d_g�V9��U�^)z'��oi|��.a��R�
��g�����>�������V-y������������[y�T� f�8���R����@�5H�E��A_�O��X�6�� ������C|�/{d��� �����o.���=R�L�s���	+](��%���Z��Q:���35�F)� Vo��f��������e'G�F��fi��h��)�L�f�]u�o���6�#��jG+�g�c�7�/��0��0��7��w��O��EW������:����9u�����p��������X���=l%���S;9<��������?uO�Y;fG��I�}xt�����q7=:�ZY�ut8����_���4��Z�G���V����	���U����Z���QS%�0�K?[�`r�|<����f6oTP�1�0<TJCc��lp=���DZ}�n
`�;�����WgX�:�P*
�M�t�jR|�?u��c�H|,5����o#����A�a,a�u�]8�A����?�Or.�`���r=3���4�����&E�YN����%��������1w@�ZLx;+=t��R����p���
k��(��J��F�T��\�+��!�1X�m7���|�,��{��bP����@'�;��N����������)�SV�����Mr������9�����k)u�J?A������?��B�j'��Py� �����S����;\O��~��'���o�G�|��$�JG��:�r��rT����aP���hR>Q��F\S�)��=�Ao�Y4�����|���r���8<�Z��EXd�/�u��+�{�F��r?�~Za?���4��O:�?��nt5��O�-�v>I6|���X\?�����M�5�x��a���5�Z�����U����U)e���UB��:N���a������-��U��Qu�>���~re���Q_����<@���{�����s�)VB�og��:���j�����WQ��x��C���d5y�~��}���'����f]}Q�[����vc��&O�q3��,6��T��j5�W0��S���������~��Y�b��,�����HE`�.�+`�)Ll���*������������c:{X�
��i�� �?�
�B����7��d��RM�H�Q-,�lo������I�	��yh���)�zB-�������r��9_24��/k��2����-g��1c�~��a��{h2�0��c������r1n�������S$��GD	&��30�3�-#e�|Xp�� ��+lXP��f�0������=��pTn-4
?6�����	�%�O>��F��
�+=dq��A���#��������;���SC-�GT�m?��sPv��,L�_M
��A�i4�,k��G�	�M�@��k���X��S{���K����N)�xUk��	��!���Z�!Q��RX�tG��8�
�C��v�CW����0��Z��0t���H�|tY<�`�t0@��Mn����tB���/�yE���������3<Wt���,S9�H�y�2R98�>t��a�'xfb�MD��r��uWI�(�8��x�N�^9Y�������v)t�b��T�� 9����m�#X�Z6��'��2�2�E�q&I|PII���=ly]�*�����7��b��)^�oR�
�K ��yP?W�2cE����1�s�aGt&t�	���;��fe��F6����a���6��k����{]�r�@z*X)���r\�V������_���@!�Y;��O#�|L��$��� W[QYo��w�^������4-�
=~��]����L&nek{������l*�������G��4�x��X�yB�p���)�(�c
�z����t����G�I��/	����C1��c�3[�J���X��m�:��v���AN`rU�JX�XsR���������HctiYeDoPq��r�x0�"�a��{d��#2\,''�)m6��Y��=n\���8]��-��u�u*����l>�O��5
���r��\�b`�@,8"���<i����AeLC���������������.,J�&f�����������8I��}��y�I�2���������2��.������G���J����y���5f�������T��I�5����O>��C����s�E�~6�s���~���p�zK �A#0pd������\7:	@l���'z�bG�J��"�z$`������� 	@G���]
Ik����BIj�(�@�Y,��\���G�8����
�;���6��m	��;���N��9����A�m��J�"J��fG��t1��R0N�z���F9����Wa��7o/�_��������������qn'��Z������������������`�(N~�N}�'��Ga~4?��2(���T~s�-���������@�
��s{|8.�����5	�{{�j5�����N��B�{���p�a<\��.����`�����+������U&%(���I`�7%&�
$s>%Xr��}�YN�oh����0������=V��+�<��}.	5mR�����B��p��cg�=wBOT�S~U���_	�&�<�I��+�����������8���$&?>����$��7���`w,N���o�46V��~(_p�k'c:�3H�>���z�r*��4��4����5n�Aa�a�y��-�}oQZ)�nY���D=��������H�s����
�������i�zWA�}�o{��`������gQ\��K�@���?{��m�]���2f�����s��o��b�D%�A;�X]��{����k��]��Xr���c\k BO�C]]��|�A���j��t��^�y/#�z�i�����?�aG��hu��O�y�fDw�Q:d��~.���k����DZ]dV���^�r�%@���`\��i��u��>�'F����N	��������@N��I�E`u(��$#�3�����e_3��,�����R���)�z��OB�	c��x&%�&5��}5I{"��i���;t�����R_�W�����C^�p���7!�����c=�z��2��x2O��h��I��j#s����h6L�Cf,����1�,���e�j&�eX�y2���^�����u�ps�������^�0r��t.<�����M�����
�M�p,��j0R���]�Yl�/�U/8TO��\<#��<=`z�N��
	5��%�s�"6}RD�D)�@�X�R!)��������/��;�i�����	��s���&qk�(@��n_x��[Y
)X_���l�w����[g�������t�RP8�������H�~�.�J�,��p�"R�B��4�;	�E�S�)�A5����J��.|n�=��I6���j�N@T#y�M^��3�e5��i�H�)�q�^��'��O�g����!l�$���2	��<������f���$j`UO�����_�������Z������9k�.8Yu�u6E5_]��9"�SOfJ@Gt66�H���5�9KQ6�
)����v@����=�,	
������������'-����0O,`��D��t��b����8C,��9����L�8������(������L�j%,�m 9��}2�s�B�95�����s��fs����cy[��~

�����@)W��������"9�A�kCn#oX����~��v������<�n�Z�S	/G��X���w���/)�������we���Yca��
u�+�{��X��7�k�2�o�GM�U�^������`�K� ����@���h>~f��`�~��`�N�%��Y9S����9i�RRe��JY��Y�5���b�tsN�_�W?�`��v0x�8�S����T&l+����������W��B�oz��T����]��-�-Qm�W���F��x�r�P����|Ht,��\�W�c�K	��t�r.�+LG���dL��yK�\J�ud�3����U���n�����5/e2�>�E���;�����jW����a���-��ZxG�:F�|������V�����7�;k����:_�N4��a�u<<�l��Q����iv m�4;m�<g4J�
*W���):�#�drn����B�W�����\�������\N���B�<�l��������f�q�l;��$�Z��n�qtx�9��G��{������I�?8��w�D���%�<�^"�{��j���3��P�+m��G�����Vi?���=�~�9w:������;N���b����g��VC��h�:��5
ar�qv\Y��K@o������Xe���A�l�*J���|��y�2�H�{�'&�R���!�����b|����������QI���6�1fM^H��IF��D�CaHC�L�v�>6�{8�N�A�-izZ�Z�XX���nh�Eq�������Q`�]��������?(���}�Dtc]VL�+�Ntx��	a�{]I:z����g��� �����tQ
8���>a��e�k���1&$�b��'���5y�"�n�����������J.��|�^2�1"4b�K�x��d`{$�,i�T/�	Y�
����8��@p����l'�����*��N3DS�����d�/
<T$����K�G�E����h���Iw�[�����-C��w��%�g�������������h���n�����H;]�����a�����>N�����cD�Z^j�?��6�{���-M��=S��������� /���c�G�@\�F���K�%�����$ ��Z�H�|\�^��'7Q���B��	��n/��?(�������
f��5L�.���E0t�0�G���,Wl[���I~[ ;����?C�g��Wu��.XR�*�z���C���b�@,�<BU�;k�Z���M����y�<�$0�����P�t�\���i�zuv�i_�p}2n��'`x��<��F��=tGD�N��g��
`,#&e��������|}S�5<~Gm��x�I8�&9m��i�
��w�&�1���7�9FY(sWk�D��XL����fE*L�@}Rl�
��5�rIb�{a��B �%/b����""N�n#A�����yf���Zk���������I15)�m\��Ng�X�����j�EK
���$���<}�I?��P��%H�)�%9Njm65��v�P,��/+�5��:���6��2(�F�����h�D�H��� ����B0�E��1�D� � ��L�
����,��:�:��}��(	^)�X!x��e�ll5����y#�u��-<*�S�7�Z
m:,��#
�Q:�,4[�Kf�d�(��M�I#�K
��R���Jr�np��e��e2�����}�j���V�^p&�Z��Q���
�K^�y����#*��``4�O_m"rm��������U�X�v����Ze �����vm~��Y�!9lY�WM�h��0b�,�n�f��hW�Rc"�R�n%�}*��X����1�`�����*~��������r�����3��x�=KQ��D�1!W,��Vz��5`��#��\�����w��	o�Z�O���z'5��yqc<������h�j��0�0�x:��]PYJ4"	T3�D;�lH�|�m��Z�q1�����*�.�z�}��a��O���%<�� h5.�g�G�Q~M�b�a�K��B���#gAQAI�ID]�����L�*����
�j���L����(�E�*���Gzd����\��
��2������\���G�6��c��9]��R��9����_J�
�$�k~�hQ%
$�1n^o8yS����
�--D�Z��k�.a�!������na�,�0tD�t]Q�c��������?k&�l�����$��F���U������I������0���0��*��5u�C;�C��l�p��W����GcE������U��Z��Z�*1�,p����7}��C�b���a�����W{�s�����k
P������,����p����@�������B�U5<WD����Q
t��D��O]��]�����X)����%��n��i�L�9�[�m��k�R�]\nJ�K�Gfc����|�7�)�e
��+F;6z���gX$�)Iy�'�un�"�*��0\�q�9`X�s#l7G +K4����Z$�����u�6 �����+LE�1��lWHaM.�%�.���aqN�5�����PqkJ=3�pB�Hgm6�������T��[B�E�2�Evd�FF���-*�H�M�5+���#�1�k�HV�!����W|���[;T�~`c��>�2��3��$��M
�V��7�Hf��cs��A����C<�p���J	�~�3'D2W�����c��n_8}#�4l�b��w�m�X��s�u�<*�~r�6�qff�k�hG��W�V{�*y�k��P\�}�HJ�d�^,�K��(fX7	F�������F��U�������c����+��GR�"Z���0n.,�yB�o��D��A�DzR,��B!L��=�FF��y*�<�(��i'�=q�rtL�����r��x�]���i���M(�.�riZ�~#.%"GA�)s��� ���2	��|�O�������0)��,(>&���J��8^��P���C�
���h�f��5��4��C��
O����[�_�=�?��m�,h�^�9B�Kq���#(�!����|w)�gUCQ3�d�9�}�H��������)q�������f����
��#x�P�Ey���W��`����m���1&��v��������h�FLX��B����~�����*��r��"=�d��M�4�O���q�3C�BUS�"6f��I����b`8eB!��� a9��D�����X�����z�����s��2*!%z�{1)����
���P��0������U�3��r�s�u�����|<5�H�W�NQ��&l��4�ps����,��,H��a���(��F�T�<�gRp�
�}2u���b:Y��cG��"�.G�D��@n�`��j���� �D0��)��0U�|�U�q�I��P��*�T��o���v�[�\�3b���)�D�3��
w3rEH��k����'
'�yb�vV�t/���H0@����'$�I�"-i�R�;��boQ*�\����S��8�wjqXiu�W\��qDD������aK>�������cM����	��H�� ���yQGy�B@�{W���s���#G�DK��R��P�,�fB\|�	!%<����YWbx1��i��j�������;�bS�5�$R"����^Kt���{]I�si[oF�1�����
��s��[V�"�372�t��\dV��1���7���x.��\�a`7}!�p�,���C1(=�Yl��5��i��:��*;Mk��-����1�N%�QdB��c��l=�p�R��BgX*��L��"�����i��z�������9]<~^,�������a�&��`��]����L��"�z����y�����M��������J��	k�q��&v����j�9����:���\_%�)�}y����!�KVo��������=��#D&�>�O}5)o�
����,J	Z�H�{���&G�~�P|���f76�Q�y|t���j����|��N��?�]�}��t{�G#5Z���U���_��<eu$�NDh'����Eb��r s|�#���#^8�*ui"�������B�}�&���,�]U�����i�MH����;-�aK|����AVJ�(��5H�MMO]�T�;Q�C~����,n������5V8X�vG��~���a�����|�_�&��'�z��������J{7�s��y3:q�"��`*�� �����1V;ti���`^�����;O�m�	SF0�N�k��S�X5���!B�y	�0����k�������������3F�DG�Mi���z���3J�w9����urr�7�|F�5I���|1��'�TKGK6A�v�M�<�tgu�fe������{�E���P9��8����8�����	v���"b���'�+�-�ahA	�w{{$	������B�}��0$�L��A	���	A;aONZ��!��8J:�V{�<�!�M�I��3�a\��$�-�1A������Z~k�`�kc~�q�b�XJA|H���@r7�X�E��H�������y#����'�i����8<ju����ID�mG�9�A�X�f!��]������`Cq+�Q�8�N�����h�|v�[�������q�	���L*�y�)H�2��"%
t�����=���7d���;9U�b����GOV�4��U�m�����������	�^��i_�'��9�����
�����A�������NK;���?�,����X@5�\��9��m��-�,�m���]n���M_'k�\S�<�����S�69�E|�*������������u+=���e:Ir��\��{4CL��T��4�����s��n~o��������T�9os��Q����A~������K��5�����o\k���y��k5?����]k��{V��e�s��c�"�����Ad�E��O�&�	��F�p�<��N���<6:�9/`5\��x���Nq����{j���i��?)�n��B�3�~���j� �s3�Hc��]�#1_�>)�%_���f�����p%@�I��iV
�'S�����;)�"�y"�Lx�v'KGI���FI�����[@[�0�Tx�C��7�`Y���A��<�x�Q�g6�������%����7��	Jq4%�<o;{����j�Gw� 0t��7�0�8�b�4n@������sdVh�s��(�=D��2��3Dl��N��������ge�;��X4{�+1H��m�5�y���K���p��'&7� u�f�$9 �d\]� {l�6�r��*<�8���d�x@����`
8,V'1��%��W�0�T���C�>vY:�Qe�H)|���f�;q(+�C��}��j�!�-���u#J�a�,��Ok����z�����j�8�
�7%�t�����k���a>N��m'1�s����7n���(�&mYY2$��X!U����\!a����o�>]�����7&�'+A1����y�G��/�������VLn9}znFX}5�u��b�,�qA����c}>T����d�YQ�;I&�Xv���12��e�3�����^W9��r��3U,�L/��,F%x�nP1�|�N��F��y�;_���L��d4�0�|����k����������=�rvKn��p�.��=S�JIS��X^��%�����Z���H��D�6�����O��91!��h6�3����PTz���L�V�5�y�`zP�����Z+-��A�xl���������"N^��]bb�������S{V��t��yo�L���������l���?X�)VA��2��P0h��9�o��tR
����Z����T:���c�<\O�K<!7�x��|% 76[SV���hI3���?�$]�s��`�C�����0��noAo�H�A��t�6)� �r���pyF��j��1�R\���/W%�&!s�4�~Tq��*H��e
	���s���Q�A+Lf2�0��^��>�X��
%>F��4`#>b���������k�U����RO+zG��%�+�@b����)�j�"�[�����$Pv477tu~Sf���EX�����cs�c����_|i��
�"�c�f���K�������x
m����T�s���/���
��������yW�y��/C9`�����s]�U�:��l	��f����rWW���u��;+��������p9#�`#�����@W
a�~#/�xk5��\.���6Xr\�8m?
�n�)���"��0(#�'+]j(���:����_�0�c�e�8Zhi���G������j�O�#(g�C-��J9Q��D`<�2��t�K�]j�>c%���X'c��Y�W�����Q)��\��d�z���/�v��-��z��
�Lu��a*|�(c��;\�Bm<|X�����K����_/>���K)��DV�����HT{�/	��b���D>�h%�n��P1�9"�s���>PF���n�	
�c��a��C�]�d��I�D�k�n�q����v�����e<"G� �f���6q�	)�7Mv����@-���(n�?��v��sB���N�46L����j��,P��,�N���R��*�O�(#��8���6l��Q�s�+F?b7���n�=]�<!�g+{����]��@���T�Y��+�FKmZ�ls�[:n�8�,�SI�6M�,�����OByz���o�z�:�!#h���� Q�x��Z����~��s�F��k���[�/��_�����������V�}H�|9��(NmV��w������&�
��;i�Dy����}��"@�O��l��-����he3��J�����$u��[��8$6����O�#a�8�]^�S)�L���������3�����;Rp�|q�
�����)����Zc�]]��4
X���0pQ�m?����R��=S�7����O������H����'OX��fy_3z\in��4�Y��..�\��������3QN��G�?���fQU8����2�BL�20�]�O�N�GJn�����px�u��HI�v����X��)�Z�r�0���T�T<z(Hm���EW�-�C�~*�mhepI��?�&����,��l9'�Z�����,}W���H~4�&u�i0�QM�$OW&5e�E�2e���<�n��
Ij@����!1�lu��������������"^�'Mr�m5�:�
���8��;<~a�=t:5���O^� ��M���4j�0���8���*�w!����������9�N�P
���I-i&�T;�g��d��X;���n���w>*�|������d��u[0�\�X$!�-$o#J�@����km�����6S�����iV����$�t�8��I����u�������������Yn��U�1��%��zu����<F |�Wvt�J"�3�q�
C/�zyML�������i�rKX��b?�h�%�G���2�>���K�e�aou"����u�7�F���"�<�����^�o�	$��
g��4��l�������K�e'��H!�h��Igby�'�����*��� �p�R\V;��</]Qy�w�������Y��Q�}y��S������RT��Vw���:%*����}*�%Rd!�{q�v�P�[�}tR�p���=����08�������6
h��1��)L�c
;�������qm��������oC�����PR? 6S������9Q��
�#�`�>�\��������S�;jb�k�3-�����.���p>S�Q�O�s����
��E������C�v{�x����^�t�A!{�E�~���~�����3�����)�.I�NQ#�5�&��${E��[�!d�k>u�>�Etu�L���W����p�Z3���W#j�FW������������:��� ���a^^�D�/[����J%�w�R�G���7�/$#{�4a3�)6�Hj$�����������Og����������������+�Pvrj����� z'i~O	���x�E��������2T��7��i#������@�d���zk���SfR��d�>�>^����w���x���B
��PH��ry�k��Y�fEnd������T�H�Y
b�U�
�EF�m&���
���%9� ��DbT���!E����i�� <�r��]\��0@�:Kl_=���e}��5�P�F���^�E�Q���P�z��>�w`���}�Y��uv����V��X��L�3!!\I~6������Z��Yf����
M���KJ�w�Zt�l��o����;b���t�G2U�-�h�d��C�������P�'���/��52����c��\�"���5����{��D<�������Q��m��F��*��]&�,���'�����d�2v��
��3�T����G��%�b��W�x�``(gQ2�������o�����\c�Yt��S]�P���4��L(��M���C{_��5��u);����#b�7���.����[Q1�e�e2*�$�U*R��t���C)Z���g��$��c"��X��d2	Y�������l���;������o�19����w�}El����d��'FYV�������a?4��n�ta�����y��r��M��z��M@��(��h7'�e�huG[i�l�m�����>
�;{u���l�?�#E��>jAKO���a��(�p�a��b��K�x���i8xo�;
��S8���w�y���L/�G�/��l����6Vf����d%KRfkf�:��������~<������
Qj5���wIlK����p��9,x��x����b�	�[�U���V�l���'��BD+P�}o*J8S����=N��E�����!#�����P�����������8���������t |�S%Ykx�n4:�h0J:��
�I
M���"#3����D��r4��Q^�g��z���+������h��2Pd�)�_�e�4_��{��)8zo�y������������wg�]~����7�Wg���y���*[a,go6�/���f�c��s�{����
L���|���5���\��������~�o�7gzS+����)���fl��]e�u.�,���v���=*�D)��M(S��Rd��VQ	"PS��5�y�J����cr>���m��0��;J=�� 2��s���~6�X�A`�s�h/:�0 �O��.�T
�����\����l'�W47�=��Eb�0�gv��B~�>��Q���+��$�CE�>^jQ��	����q��	�������)����I���,��l�_����t�����v���x*�?��J(�� -����l}k-��$��c,=�W������N=�|�-����dC�WW������r��q���\�"���q�,Eq�576�K��985E���C�)[�A��g����=T�>��{��f�������o_�����w�<�"s�O�{�V�5����>�_+�gg$���=����q%����+L���1k��>���aAy�����Nk��b@�����_�M�DMe����l��@�$��a������)Bk�����&j�n�����/�
��mqc�!m]��G.yp1����g^j�N
���U�~����c'�9�o��n���h:���A��2{��hE�S����z�VRAm��(�"pQ�Y��;���eKL�������g�w�H�LnaT���m�&C��e�������Y���(O�!�>{�S�ad���s6&��4,�8I���(�'��n�S
�/���3)@���M��@_ D3-|����iN��Y�di���oR�#kgN�����uf���X����I�u�4�F��r��-�eCN3|�3�
F�����he���q���,(r��&L�:�]-��H�.#���tb�pyo����k@������u& �ED$	��d�M��s�'�%2�D���p�F5�`�N���c���p��
�<w�2�_�tIm]����7j4�~�l���u�M��4��t'�ZW��g����@js��/�<7M�;��$o�/��M:p?�1�N%�.g��l���l�L-M�S����14v��0oO/�}���Y�[��;�d�!s��i�:�dX_�5JN�]�1�s�C`���?�;X8���jt���/�g��3��LL���M�
M�_9�Gy�$���t����4]H
�-�u>"���hZt������U�8�v�P���#����;u����{~��w���f��f��0�f�=�[-�dh�="A��|i���i�K�����Y�R]y?����0�]e��:�-�?��W�*�S�V��9�N	wh�.�������'�X�\5_:Z{�!+�9j%��w�1�����C�O��JN~X�N��hwO���H��mp6��%�F�t{��h���N�����ob�K����<��C��}��B��S�����%����0n�����4^�^�x��<}v��������|�P��r$���oY�(;:9|k'��h��1����M�e��:��v�{F�B����.t&����Ds���b)'��~�	Z�A���{�,�&��������(F�C����%X���p�]�J����=�o���&o������`N���t�j0l��F�"������)B�YKZ�P�Z�M2���g���}��i���=��-��E�& Z����`���B����G�q�}H���-���s��Q�������.�J{j��r�JZ�,E%�|hdqD6b�����\�+��^��0�p�c��oCF����H���TWx��`zue����(��=����;=���^c0��y7�������wy�����n�\��
������U[	9��W����!�Jv)����V9��7,�����v�b-�-,m�4h�og��������ZD{l1�&>����k������jV(�-�H���������`2hfe��4�������m"��|����:nJ`�O
�8����P�����@�[�B�
d���J�q�8it+����?������a����_��R������2z���S���y}5���A�X�E�� ������lu��O����n�[��j5��f��v�a����j�����J����j���d���/�ahg�Q���i�x�
;����9LN��$;l��I6jv�+`A�e�:R��#���V�#u*k��N+�Wz�}���2�V���������s ����k�u������v���T�n�Yy���
X�G��oO/�}������c+�W������k��j�Dm.�-���m����B�z�����Dm���sr[/����T7Xf�*�v�O!�%	K��|��s�%�������s�G����9���}���dl��-�����J�f�o.E����Z\>h�8|�i�R���&0��;� ]����C3���@����
OE��Wnp�j%���	�Z�]������[puM2~P���{������B���n08L��A��>�8m�Qt��"E^��nb���qc�SD=I���g�����\���x=@�`Y�%>�R���E\����et.�;l��
)���!|�N������������8=|���z <K���U3�+6�:�!��U/�WK#O���q�(�����b0���4���a�A��E���l_;K��V��2�e���'2Bx+��|����k�n��Q�e���
�lG���<�����U����9m��<�����\l16����}���c�u��y?���{����a]r��CE�C����fi�k����-���
�a��tVJ�$
;�:.��M�N��[���&�:�����5��"��.��^O�?����Hr9,�e�>��'_�)!�H �O�!�?
6��n��QO�ib/�������"l�:����k�?:��l�-@�g~��f�����gh�<������{o����3��{q��<\�k�r�_U�7���d�,�C�B�m�
�����J:T$��2Z��rw/�t�+ '!�<b���>!��
�XB����A5���=�W}<J{�A�b��3d�8Q�d�*G�~���n���-�_r��kW���M�(�W�p����W� �B;���g�)�+-����������Ej�r�=���(Bx���vR����|<3��4����.B;&��lEG�b�	�������'-�����hp�-��hO��c���������1�fa�09q����{&)l�Ng������.��9*�D��9��=?��a���IW���u�&k�8}��Y����w�zD����iu[?�[�>�h��� �<�_Rk�>��0����lu��[����t2I�bU�9��]=�;�����/�G�+���z�\J+������x�NZ����v��_��k�������PO�&�x�CR�X�����
o!N�in�eR����c���8(�q���
D';�������������X���w��n;�I��P���[k������<�ot�j[���E�����`�����%�Nm�*�|&@��k�,>���5�W��������	a�Mw�]����/���M
�6��w���g��������#��O8T��PEOUS���fx��s����rW=��AU��>���.t���sRk%x���Z����G}a.%�������A��� ��PAw)K�_��)s����5U*.������-#p�d�P�]���;��m8h���1�Z��?(g��e���e����4s�X�Y�lW�*Db�����	�(��I(�|
c?&%�������tI����K�{��	~��e{LY�����-z�'�4�v��[tp��G��:L�n`u2W�z����J{�Jl�Y'a/TS�~����������_'������������'���~Ga��/���p���l��������������d�?lw��V;K;@
O������N��s��[Y����!��������E�7��$y�M���-2�kd-�c�������_�v|t�����7*�����Xf2���^�eN1�4���f}L������y�G=:��z����>M�Xu��$��_��_����@*o]�#0&H�*�+&1�)&h���!�9���|���;I=�	~��xx����/��8P��\��y��r<PZR�,��a���{	��xV��V*��:��)��.s
'��N)����RVR�����|����oAzJ]Q1=�s�
��[w�D]��V�bJC�3E�
���a��%I��
�PY �\F^$�c�53��eh&T���K��|�$5�5����bBe�)-&�2>(F1�[�m ����\�	���+�cOg�b�8yU��Q�}�Y���������	����O�� �����I:{(�����)��P%a�>I�G%7��&3a��G5C�*��?�Qk\_�\�_p��,�T�Es="'��z%�V�t���T������K��� �v����=:jv"���!���R2�IN���4���6��k�pH�v���n��V>����O6v�@�:Sfx��`�:�B�^W6�y���/��p$p�=�:y�r�r�wJ�.v@6�S����_�������_*S�
�8�>�6[���n:��|�V�(�hig�n+�-�_���y�cnG�,V0���~�(=���{S��|�b��/����Vwy���P����#��*�������f�qt�����\s�`m%^k�)LP�r@?I�`�)
y�q6�I�eX�^9��h8*<�f�q�����r�_P�Z�y��Z���i
��,� �P����JDf�;Sa=P�9
�ei����� �D�b��_]�P��s*
�G�9�������=b����M�hM{���������G��%2��cKx���"[�x7?��W����������L�U�9���$���N����L�M9��}Rk5)�^N�x����A����pe!TN	T���2�����{#�q?�f��&PV�(}���tI�e�������*I�2w���)�r,�Q��$�oC�z���]��FF��Ap�7��R����o�����kW���5G?z*-�M��N�����4����Ng��0�����0���X6��}H��Z&cagQeO���|&
�����������)������y:���9��z\��<Fx�fQSz�,��yML[b����xI�b�?��r�Ms�%@o5��[�|>�IEp����[a�
I���	��B�����q�C��"�z�q��K���C�:�=�n�og�<;'d
����e��G�A{n��x+���\�/���#����/R���h������0A�
Q�L�������Pm����%�\����[�����,rX&�]/�����TK@u�(T"�VP:�'�6"��^-�L6���}����� �j�^=v~��k����0������jnX=��nX=�TXC%I]��yd��R\t�L�JG��*��
�|���=M
~G~ �93G����-��������b3����yC�q����/-�k��	����}b��L�
Pb�J���������I!
�v�a�U)�,��^����B�����$�8(!J�"gA�0��/a��uo�=�����F����t�:{Y��f�����;��s�yX����)��[b�zu�bD�����E8��p���vz���V�P���:;��OTK/��0�0v����k��h�[i���4�IU_�K�4p�4~�t���u�l��c��>'����m��������{�I�$� ��&�S^�&E��<��ny]�����xM���r�����O�����y����w������17���;�O�!�����'��;7b.��D�<��t���g��������cJ������@r<�k`^�$��se������ Il�7��|:�M��`F�^=�qk�h��X�D���WJ���'��8�9�����'�r�]�`�cW�*���Lt��m����yA����xPau	O���Hs��?�����'r���j��#$�����~*��P�l�f���
Rv�E��E��m5+4�J���K��l�:�!JE������R��s��,1�?h_�����Ft��g����#��RIww�h*��@`D�����|��w������9��zA�V��+b�N�� �4Oj�	��N������%�O�,�&��Xs9����yl�����# 4�FM��~�r�w�]�;A�u��y��;b'�ou�\'�)�����?��F�H8�{Z�Fy�6n|�[[v�o���I;����]�����a�?<�N����[b#��&�x�Rt����'l�4_���F�F��o������>W��HWwgg=��Q���5i\��%z������� 'h����}a
q9��+B���@��o��%�>'��-�yo1C�����/L�
:��4���GpA;��?����D[�Fe����Ku�]s����tOO�:P�u	�oAi#g�����%&u;�0^a�����%@��	����7�6Ibcu>�t��x2N����F�XY���#�{��J��;n���f�������8��(!H!L�� ��!.��#���������a��������Ik�������z�1��0$7����x������@;���a<A�_j}�L���f��{�e$�����w3�8
?�Fl?YM+;\A(�6j�6q���.�*>���
�S��)N���������w�}0�FRz���)��_f}�'+>"�p�u0;�%~�J��;_�c�|����0�?fg�!��Q\��&���5����>� �+��a�E��q���3k����a8���|5^���^,�������-�xSM�
Rp
����[k����$=�\b|�@�h�a�h#���!�Q�\�T!o�F8���:��"�����k�YST9��k��U�J�(N�b�����fI������tC��o��l������,���v�c�Y+�6��'�U���.����
���[f*�)�lBa���P-�^_j�;�s������u�z�O���2S�bz��k�zAk�"������(�W�O+����������_�?k���n���PS�
Q�;�4�������T�����iY"�>.~�-l�m����D�:�
����K���)�>C����f�N3���g*K�V>1��vL=,o�V�\IZ�6UH��Y;:r��&�2��
n,�,�#K���b������]pE[G�������$����TO���N�t������8�}��Ho�c�N�r��
������:Gb�O;'>K���~nC�X%������W�����������EX����]u��.%�E�]���(�I;����[7�A���^������������y\�\�;d��I�@H:����)�������Aq���9���P�`����s�A1TG�!x
"O��p�8.�l���$��*��E-�6�v����^o�a���%1R���"���B�:d���"��#E|:�N�
�n��q��:��|��C*s�.�G��t��#�$r;F��Kn��(�>���I#������S����oK�K�m�>Dg���2��s�������yd��}����&58S��2���C�������-9��no�.�d���)����S��:'R��q�$p(f����|�B���25�%G:���Pj�06����!��^}���2�Sv�r�t9�g��<��mM~�"�<p]B:k�D��V��]2���6���aU��(K1U�dPS+��:����a�������h;?�g{4�w��;8��2j��
�`��o8"�Q��\�d�����������t���:�D���;Q���c�
��K}�Z���uj>��
���@�,G���w�/����t��	'+5���y��� ��,��o�Cn����q��v�Ng�������)�L$T�4aS��"���\�Mj��O�.�u�c2�a@b���*��Ko���������ne	�^�9�PT7V��l��c�;J�+'�p��9�g�@F+�z"t��[� ���UJL
H
�I���k
���"���p}x��o�������a����\Fb��3^��+������:S�:+N�o�'����G����	��$��y����C���^�����U�����U,�sEn��:��&ay$��Kl0�����	"% ,!�C~��C�3��|M�[����N��'S���G��T���7H����5Uh��"������f�����r ���#S�b��LW��&�|sQ��e�"�l��0�kb����y��ns4�(o�����^C���I(�_V6��dm���3���Q�y8����!l���vU����������|�YF������:���O���;�!Tf�A�$��&@@
�?Y]-��=�������E��Xa��<pW�co��\H.�
�5�t]g[��N�uXS�!�+y�Od[��Gg!8���)�|\��K:n�������V\=�(\=pUK�G�A���������i"<|��'�A�����p�
���*���*�r���W�<X��0�g ��ws�f�����_�1��E�^~������~���
�0���w������j����������[k�g?-RXp�
�t��C�������GWY'-v5�:��i3��|������V���6���#lt�B��q�j����i�����m��o]~����>���R�u��-�_ �A��V���������_O��V���q�s4&�A��(mu���t�j7�� M��_(��+����z�����G���NY�V���-H?���R���J��a\{u��0<�
Y\"u,�������2D6�����&�=�0du��O�0$;�s��$Y�u��
���t��B{vCI��������������U���������t�H��
�(�����EC�$�o����ER����R'��M��J_i��0wb�������T��qRF��M��+�d�	Eq^i����tF~_h=��
����'4.�|M�b��"���k���;�U���^0�.f*�����f����1{_���m_�^�]��Txs�:7��5�{�dW)lGJY�O�a�P���XW�$Z�E^�d���5��i����A.a>�kr��K����d��i<�E��Q���b��c�l3��`W�]��G�f��,��'���2|�g]=�0u�?f�5yo+���������0�(v8�G���om
\;�����Y9$��f���g�&�*���'t��g��D3z-Q��
%��%�(������:%s4A~��Xy r��Cv�qG����%RL��|K2�����W��Cl����\��w�`c�Z\	Z�]�=����s�����$��L����"V�$8�t1vS��d�~{
sy����P>HA����6HZkS���g��B������w�����G�>\��n����(����X<�����GM�u���D�wp�����w@�����V��tx(��
�Y�O������������������u��Q��=�o�W@N'�S<Q:
k�I�~���A��@���Z,�W�
*]���',���;����o�!m�	:��s�k����b,[88lO)�����V0��S��/��:J�R��H�&�[���_M�t]Yk�W�2e`QZ�a��B^�1c��30��/��	;ib$�}�!H.%��G��	��]8��W�����:��8;2��'����u+ ���F-H\��1��;5udm�*�
��|/���=m��k��S��������x�v�{�����gm;c�?���b�J�a��s�[����N[��R+EK`�C����9��b�������q��r>!�����4���Z!X��Rs�"��d$�x=��X���?�J����9��_en��R����������#������n�"/p����~9{}��l�W���;�NI���:\���8�S��w�Hs�}l�8��=��e��/,S�����Z�����Xm������l[�}<��[�jy���S������Ft�e[����#B���h�5����1g���!�����^10��!�c��F���������$_c�E!��8���i�����6���L�7/����K���R����/�� �}������Z_r�w�bv���wz^������j�9���!V�2@�����
\��@		01�E�Ss�d\CD�<���`����HVE}�)�B�Ca"�P����\{"���!=�|��dw�=�rlQ �KP�0����c.�6}dZJ����������".9���3*i@�Z�c]���V����d��WQ_f���op_T���7:�@�8�^��HB�C=���P�����4�K����-'��6h��+�&�m(uI�3e#�Z�{C�@��{��D�:M�H:p���;�*���>5(�z��1&�>_�WTG"w?C��������Q���()lS��`������P�.�0������j Tp�jN3e�k�{L.�9e�+�R�X�C�I���N��	!�|A������H�)	W��fE�`������cn�Y�� ��Un���&+.yM0��4�
��(Yu�WSF�( 4�Y��]5�z�l�/a A|[�qBJ���
/���A���0�
"J!���IE��s�ND
����OI��(���\�
���(�c��Z��w}�mK���A�/r1Z=�����J� 
hZ���G]������du�a� ���`:���������h����U�q���e@����:�����h�6�C�vZ�K:�V��h��~�$m����@������d���:&G����N����A:��g��{�sV���s�o��������F;���	)}iU���#�����X0�}Bq��?�����$`O��2�����������I~�Sf��S~;���K��<����*��mwr.kw�I
��0r�cZ5F|>�������z�m�
�
��6(���0s�"=�dS���� #�s�
7X��G�L�!��iQ�iv�I��o_����	:f��(��t$~��;�O�Y)���x�PfTd��Y}���a�Z.�^�g�_M�ud�u�/��z�oH'&O��,�Y����b��!��[|�5����V9�x�lA�Cz�z]����/O/{����;��[���D��q�Y�W5iw�]_N��F��w�����5N���	E(
�|���������4�����=���Xxw�~���������}6���yA��wV��Z�A4(�@G.�������)i)4�h8�E�/6j��?��;��v���������Ew&�MJf� �P�FR�1,;���\#���^6�L�����.�)K�	��Bi_2������ea��U���#
�����{X�u ��f����205�
Cj�Y<����+T�>��a�"��2��7��hhW�=<#��*�����������:{���XX��<U���}����G��n��Ml��z�R���9�i�g2�E`�w�����U����T�����)�<��������+�G{��3���;��,]v����7�9����k��_����V�hn�
KP�[���gl����/��lX�F�c�,�H<�<������T��,]��������������a�2f�}i�e9
����'��U�����c��pYF�	��^!s�YP�8K��m��t�=�,�r~k8�v|^�q�/����^���>�a�/���-����G�DmC�(�����y���"�w��N'���*�/��������\J6��{�
b	���ST�x����^2�M4r���9��
:�G�i����32�r��_e'f����dnq`�G�?��A���������I���n'3I���F	�����n[4G���y����5z/�1E�b1_���I���t���>~O�>�\lT�I��<���_��^�����������J<p���e��G���5��b,�VN���iB�<�L�XFqGB�%�t��3�#���H������<�k������g���uGp%�7
=��E�r������|U@j�q` ��q��EN
��o<"B�5��hZ��Uq@���2��`�GHJ���Kp��Y������"�B��0	���@���0l�b��*@1]�C�v���;�3��T��*�	�)Z;�2������pp�M����n�7��6O����l94����Cs�`7F~���c.����]�al#(��U�����fz-���!��grT�����D�!��}i���[���������i��o����Q����vf�]��D��8�y��s���}E�i���O�0w���N;����tR8E��;�g�w�Mn���X5�|(�;��������?��|�}��7���]����������!��aD=tV�d3R�/w#�w�76��>V�6����o�+=����oW���v^��P����x������d1Z��s�������"�r����T��8�2Y�Hy��:���w����k��o�K4E�L�����C���qM��I�����k%r:��>��a��w������c�?5���*,'��`�0���&�_��+W�$��Uq�l�B�g��L��;7���g�0�����L�zZ�e�]�<�_�l"J�M�`�1�W��������a������1Vm�q=F�>H"j=K�||��gGF������O�$_����Cf"�A����J��}W���L�X&�8M
h)���$<�G��~�4������y6�P{������1���g��[�	��!��sI����I��9��qGGg5xT��a�����d�1��l:��HJ}/�/��>f����#e������������pIr�g�-b��������^�4���`��F�&��Sz���������>aH�L]�?':^�^3i�q���JI�������T�8l��J�2��L�+DG�g�:�iV�A}��]q^Y�>z��s ����	�������1�������e�����N�IHB5AI�~��������#�p`Q}�I`��a����w��t9�~�����z�^�V5��@�Y)M>i[��/L��+���	xVl�iNNn������-�D����>�)��������+z��zR��(I�hM����![�%�n~��*��c�h�M�!]���@8��U����|S��H�8C��:�[�/����D�<]wN�<��
0S�d�[��
�}�,��K��'�X3��7��-�8z�
�?{�C��
�C]�_K��� �tFG���{4���C�C�P�I�KA&����
��&�c��"�P������E�����s�;M�����t������^\�)=����;����Vp�?X�p8z8�R�;;����H,3J�Cq���R����H�����a�y�S%�X��T�,�X`n3�m�!7�!�j"A7f7����|6� |X���^��Y��x�}�!��r�B#��	�b�x����KLt	��!r�ygB���om��9�RrI�RI����@�@��L�V��7�M��# �d �Q��0��0�0����<���-�{c���;�~�C��U��������S�=�������^a����B���~�l� �?>j��l��
�q��~	�W�[��	>)�a+Re��a|T$����h�/��5��S|�`��8�m��1�i;�^,��9&>KSHgk����}���7���T�9�.�7�-g����E���Ye��e��pe��!7�����n���=�������5�Q����?���v�y�u;���10�'GG�'��d�e��I�������j������z�$���e��������E����`+�HU���Y9ED���������8ed��\g��J�bWh�.��D�����Lg�Hs5E�2��!B]��Jj�R	�w=�s9��$)�
�wV�eM4%Dv���g�8�������I1�S���Ff����~���D��{$L<��P��S6����������R�n�����8=�8����T,��D�g�*"�]<1���������#�^��j$C���|���r�gw���2�}�O>��q=��#��K����#����(�4���K�P�\`��!p��GX���k
vsmZ�{��YH������d����������S���t�+���1=9O8���c�1��&�WY8C���F����n�y��iQ��4�KeM%����p'����BZ+N^�B�c��`fu�ElW(�g�LC��G\#v8o�Z�}��:0��zB����$te�$�-IW��CJ�0�%��������E9hm
���0m�d�-&��i4�9�f��w��u�Xzr�v�����oJ���?��F�2,��1��)���d��	��P�DW���I�{{x���5|�rnj�/������d�-�|�����t_�������(8���
)�<m���^�s�`���7�Q�*�>�����/������g�������^������f����-���pk� �����>!EZ'��;~����x�`��Lw	�&���X��T��6��I���2�zd��q�	,5����t�>[V��$�R^��]���?����\�9�����{��L�'��0�!�=�-�:zL$4f&��JL
z�����	�/_����?�������eT�o�?#��G�r�u����2c�1I�.�d��9��Q�e����d2�f/��
���1e�y~j�����i�����V��Q�r��+u�6�'�')�����gY��6�><�J��]���p	���}VlOyh��!3
o�OY�DJ��������|vE�\�tqR�O	��{M�-d��Y�9#%y���@�2"@�r��3%yx���e�	��jNQ����A��0�2���d ��e�x�_�������|Pk!��S6��;'x�3+�#�����K�����D��2���g2��0!e��^7�5[>]�u�;����Li���q��G�K����d�P�����|�4zM�	bh��N��o���N����D��'�q��k��r��t
�3�����v0�'������_'R�q<[�I$
wMe�P��Z��QA�%�Z��2Dtq���tq����������*��mL�C��$�����C)�/����7�e7����������K�k�V�@=`O���>�y�bK)�z>�0.b8/�_s�5�K��~.,g4,Y����n���8���g����roa��Tv��v�X���0z*��pL.��| M%A��&p�]e�j>��L��+#:��#�sv�����1^�Et�9v�|�A��9����<��^�U]$U%�oh�]�t5����O��`��tr�<����a�]kw����D�lH{z�:�!��[{��*9t`���07��nH�Mm�I���#V3������PbZk���/C?B�
W�������A�Q`�t��$�����u�ME%F8<�1j_�KJ�#SO���Vs3���n���J������\u>'��|b�+sd���7{U"��C}a����W�[:`���e������������`_�O����O��������Oe-��Z=y���9�xv�;����������w�r9���������|j��"c�3c������3��0s��5����mA$������l�3l�+�t�R�X���l����~���&��Xjk`�G�d����?���t����{t�����k�#�o���IY���A�m��'��I�x4
�����v2v��������G%��&�5��M�Z�g�o��Du�MWy�x������2�����	k�Ut�U��`��n.�NM)m��g�~�����B�r�� ����
�![�?�� [������j=�����B���5�"�����S�N��L�/6a}���+��@29�nG�T{
����nV$��;����t5v��n6������Q`K����P�R�Z��p���]]���L:�������DA�A��&�F#I��a��kdk��k�cu@?I6�[��~|����SJ"0���G
b�
��Y���-�S+�
C�����p��V2jc	+/��){������9��*u^���Ap��}S=ET�k�8�)|��Le���x�5{C���S�L��.�>�����d�l�X��H��K����-�W���}���m��}��!�7re2;�;]���s9��;���T�[�������]��EK��^}lY���|Z���Z
���{Xku`��'w�h7�<Y�(��AR�E����f�)�x'9����8�BZ�1��.���9����{�pu�T�}�����
4S+�������j���xA�H�����r@FCn�����`]��:�r`5G��X����-q���)?�,(�4�����J�c��97c}�� ���N��x[9*RQ�����-�5���e�N
�M�q�����R����C�!�F�$"���)D���9JO���B�I�~�Z�X��*c>�'*��?b�Y�(�W�O��������2F<V����|��n��'��,\�l�qJ�
�m�Z)M&x&\�����0��U�)S������k��m�Z$��q8�B����3���E��������1:9����X��O���1�2���M��X���t)����h0��A
Em)(?���s�vs�E���x
F�i�'�C�[��w70�:������!c�Rf�g����1�[��B8p���&��n'w���iH��6��b����� 6��N{]�b1]�VB�O����$&������(�\�(�<.�
����e$B�>�4�p�]�����	���7�}�����0�������������o�o=U!�|��[�M��V�������[��V�(��w[����~����g������Qr|�<��;�� i7[��a��mw����q����/�YM��W��Q��t�,��������m$5����j��fM5����b�f��Y�����CK.�X�1�t��U�$me�X��)2���;�~���^���f<��z6�u6��t:^qyt,����K
|�-+��vh�g�1{p���
��sO8Im,�#$yaM�yd
�wdJ+�,��A��K<�e���a�O�Sl]�>�Y����5���*�c���=�5(�t��V8�J
N)a��(S�������N�S����B���1�*E�+��@�X����J"�f�,;��Xk��nO�t&I�:Y:�C�\kJ:*
m[,���lD5$"0��}��j�M��!�/��:��t���5�������%�A�z\�~P)��I�U�KS��^J����Y�$
*�����"�P;^5��,� 0e:	K����"Q�>��n��LM�Tb����=z��9J�s��)l{N#o�>��|_Y����\��v9�S��ev'�D`�^�Bc��X�ZL��\��X�(�1_ %`�yTPb��}����B%9�A�C�.<�y�ACe����z	�;"l�m��p[�b����
�����@��P���������L%AE��aG��m���"�V�N��^q�����%[o	 ��#�a���~B��<�/�i��M7��������������0i������e�����\]�QT��Z������n9A`@F�������`�v>[�t�����J�u|��F��c\���K�j�?f��}v��i��klO��K�E/��;�D?�qh���25��Pt�t'V,��q�[� r��L�
Qv�2oX�78��|����f���J�h
[;~��%�Gh���<�����
<x�)^��q>��i��"��]�)�"�+�1�1�t�e��t��G����GW<��Z$���9��=���AG���p������r���2��[b�.���������l���h����u^��NI*sZO&�I��0�^:������v���$xv5��IR��-}�����V��9��$���g����F��'��O��2���0E�G��Z����bP�Q����Kt!����o�g�@b�&���E�������4}#���^T��y��}!����6�N��#��UM=���`EzE�������	�S9�u�}��I�J-�}@91���,p����#H����3� !b���$D�>J�[���a&�q��}����uC�
����^\�_��y�{wyqzy��_z���9bZ��=T��Wv����A1,f�Wx�v�GI�@�hc�������W����/��mH�����B|��������y�N&�3�����&�b�o�J���m|+s�s�0��N�{���o�L����=s���K�������\@�[�w����w��#';w�?vG"s���+�x��pi	�������%���&n�{��j^��Ct��,����\��K�������c<T��9	`A;�P_�I�	8]<��lF��5i��bp��%I6+��9��:H�B���d�[��U���J�������B�xi`:�r��,�HD��������b����#
�x���0��,�&�&�^x��{;�1��G��[>����������3��t(��.��4�7�4_��	*����	I�w&@M��FNng������f�j��z���b�KG[n����.	�7gJ�^.�8x������v�>h�����h������}���{��%�y-"��n��v^
��r�����,H��P�#�i�
V�����^�G\�� ����yJ�0�U��m�WU28�%j������x�#M�h��cP[�_�;��H��<�v�h�
]���7��<R��
��"D����������t3=�<���t,
����#��{.�b����d�I���`����p���ZRP!\�S]u�J����������C0��i�!�j��G\n��FoZ�@w�H��n�$^Gw���ZFj�($���IH���-���'��^������C�{ &=���{�8�I��=J���:M���!u�>�������4���r97��y���+�P��r��tr�=��K��vjU�Qg�a����K��<C���Y:U+�v��?���?��:�&V����$��[������^�5�`l�=S4�����|�9,T	���:8��(���\q�4�95�����8��d);�b��X��d*�A[w�}I�_�]w�����������C{{1�����Sck���f�2zg�����9�����
G�]
 ���%^��Zh6
���YOOf��wc+���V ���X�D��{��T���4!��E���6�I���9����M�g����,���v�=�L_������9�w������52!�	<��x����������3�OyL����3i�'���9���8�l��6HB����v��]C=6�����������<���{�U/������;�t���mpV��	����E)�9:��c�������BMd�5{+��T����]X�c��^�R��>$5�B=#����n�&���)�����mS�F���@����&S��-f�g������vEp�Yk�LYK������N�
���=�����J����L|S��xs��Gi��O/����������sy�K�J�ZR�s�qB���e��V�����5�1%�4NH�!s�z�0[/�4�n���������4s*{�b�>�%v��c��2w����+�+���Eq�^��[&>Q�R�����L�Yk��+�X
t,Bq�i���s����=�6�o�(����2�,h=��Jz5���2���A�������P&V��-���99�B���:�h�u8��N�=��sL1f�z�?���5���O�v�����tr��������fOn��,��\��N'�|���&�7O�s�V\��P��"�6	��/�c�%C���a��Ul< d�Eu����]��WT2f���H%{�D:J�"W�9,M2
�����'��������=F�ht�����s�=]��r�CV��Tfm�T^<T5���� ��5$=�=7���I&���b���<�F��*�8|�%�m2�w�X������\��]�wG��pC�*D���e���m)�[�UJ�&[�����u�b�|7J�Q[f@~��%��,D��(���rk|"�|x��8�Z�CuY�(�L�#��'	� �z7#y�����yF��V"F5-�O�lL�G�R~w
�x��c��5���kC6oa��Y��v��������$*�'��t%g�k������mO������j��(sb�K{�}`H�4r�/���V�
FR����,r}�V��0�������k��s-���(�-3%�$��]Na���G*n���2���Hd��vJ4���|c����*����6tU�m�������8�1���s)E,7+��i�V�-gs�E��Q�V�U��\���_FZ�Q�+�3By�P��;����B�����yd�/�L[6�S����|��W�B\wQ�v.�*?�K���a��Aw��>n%{��i���=�R�dL' 1
o)��7��4��c$F?��/���M������p�������
�� m��V�7�S���Z�P��#��(��%xi,�-t]C3�x6X���H����:G����b��P��X �
������k��_�iq���Z�kl^���c�
.P�Pfn��y������h�Ot�h��m�hDd����av��

f�����&�k��x:@L��e4�i��
�T���k���&��i�i�AS�57~�sCe���������
�`�7j\�
p���,V���qS*����U.���~��L��e�a<_��r�&��y�����er�Y_"�4P������?v�Np�>�`64�������N7m4&�NZ��Wsm�
��G�s��� ���*>���Y��k��5b[Y<G;��m��U�v?
�����I{��Nzx�b[�?�������;ly�4HF���\-;��!�w�p��Z�����<��{�"���U�u�.���jd�4�L�<J�-*�	s4a�$��O�X�t���\������>��&���@B	�6~��F��	;������)�����eSdO�[�2�7��d���X��c��!��\>���!�8X�wb�j^��5�����'R�s�����
^g~gl��k�u�5(8��4�.s2;si���Nqa����R/���pe`E�/s����Yp���H��q:�UT0�5�;%O�$J����`O*�b��u�^�7IdrS���eH~Z��O�D�)Q�
#ay�3:q9^�&WI&uai�u����/��R���p��g�-��r�tD�6�+�eqL�@�z�(�Ti
�AV��9�t���F9C#&]q���7��5Y��9�$����<��M�N�N=��<Q%����0)khL��
w������(E�V�	��c!���P����uTa�)��uH��h��V-��c/
B������
8���:����1��X���L��r�i �d<�R�6��"t���*68"m`�ZG�4����P\;�$.����w
��$�q��{t�:�8�$�^Z���L����t'����Cq3!��LH���aB��n:�����1�#VO��g��q}�y��t"|�&d����%{Q4g��l���Jr9����)\�e[|��n�ep��'���{�������8�&k���g,XQ
���d���u��G�\��V��M���B>���y�J�m���Fw(7\?h5������
�.��w����I��8�#��#�^��:�5�f_=0�g0o����:�;F����d`<�LkW��WJ��l�������Q/)�')Ty�Z�?���
B����������F*��(]hj��@���QUR!vB���]xA�}a���vq��(_���]�M�6�x��kw�d���n��
O�W6����d[��w��z�`tk�;n�k��t;Iv�h�$��~z��zi��z��m����cu@?[-]���c�Y����
O��}!c��wp���	
�&�NO/���S�/@y�����T]*�H4����^}V�7w�;�����5�v�����U���������:]9�1�v[��P��h�pM��	}����S�5�����O�H
�����Zr���Mj��_d�xr1��Q���x%���C.��������{h��L����t[e�w�"��8YzT&����_�'u1�(]��wgG��L���e"�V'3y�+�%�o����������9_���G��a
���h^�U��������9�Z0��Kp`!z��:O�/5$�����|��t"��R�rS��#�OS����tX�?7�����{��K��-�p=����!����4�r(��}�+�O=�$�|c���7�Z�a�}���l*dk���e#
m�9�b��L�U�7z�x���Id!bM���4���0
���~x3�l]'�J�aK���:��*b�B������vM���������!���3�bK!1[�=b�.����
��U�#�hU_%n}3����,��V5�JF���I�����Q��'��_y�\�'~=����
L1*RY�����Y�p*��g����.>&����
��4"��v��X|x�����+a��w5��c�*J6����6�S��0H�M�oO�R�����r���X�,zjf~��f,�V��c�4.2���
������~�(��*|�]�-MwTp��b�����U��c/���=�QjAB��Z)�^a�iF���+
4;�����^�� �J8�4@Z���(�����7b}���.���D���7C�W����{9�����$7J��X�H;PS�V�Ay�Y�V6�kB��;B���$-��NS���2l�:��I�j�U�uXp����D5�8*���n�P��ol�h�s��p���J&�}����k���C���Wv&�/i�R�#]�A.�����%�\�FE��s��\_ �f(^��H�cGZ1����7T�SpM��}�n��d��{u�a�D�ir�;��?�,�e���Xk�|_����|A��~�C�2�����=.��v.��4Uk6V'6�D
���G'�F#d�q���$���"�mJ�0*1�'�Lg��^{��xz���%]���q��
i�{�U �3�-����M���U8k�[�G)G����E�����!F�z(�
�Q�te���^��2������V��z6�qMv��8G���kf3|������m���������s�"��g/{������|�CCJ�������f7��}�\r�T�����N��K8uS�j�D�U1�m��b!=7;$;4`�C�!�x��1��;Y�'�U�<ab(����J:vs}�����y���L����u����Q�v��7��8Z�����hCd��LrM��R�J��YCJQ�D��'�BI�;+���9���%�����kE�!J^�9�p'7!:{:�DS�I���H�A�j��
�y��<�KJ!	_��b�OrX����sv���|�g�0�-<Y�����ep�<��?��#6�i�Fg��h��3VG�����u?�i����yb3.m`�Ce��)r��R��
Y��������g��g�{@d�*C8u��	�}xLv���
R��K��\�_�>��	(=�K���3��e
�Hx�����!j�g�.c���1��C����0���	����V����*�u�*��c9�~��;���1K�2%��������{�Hb[�K���9<�7�Q'k7����1����V��'G��:��z5���}�������Z�&��9��VC�F��[�2�A�NDY�����R����
{��C/��K���M�+�$����������}��(u����EDH��� |T
�`�{�-����R���kKA����a;�6���l8h�vXywi�t�M*��%4~w�����Z�i
�;��&�>�o7Ru�?���@��x�JU�������H���Z%����]*����8x>��N����`�8~�N)2`<���M)���+���W��_�<��[�E�q�0�Q��gg��g^���v��=T/�\�?������������QU���s;����R���g�������������
���
���g@&�n�q�#��:<9��d_�������<��x��x�;�7�����ew�z���u�&��`|t��o_�rT��yw��E�>��j�j�� ������]�@��X���s�w���0���C%]���7��.���C��i��Y��\+�;g��~� ������N~����}�.������M����k���������FQ���F�[W�3����+,^�����4�	�������������|���i��@9�:�$Ba�4��j��
����_�*�CY��|
��|�\���� �������?���'�{�Z��wh��###��c�e��v&�Od�`k��{"���k��^��@��7�i#�L����:�~[���UrG����w�TX*"*���.�f���	/�E��L=����ZRw�N;��z������?	��U[Q ���<�b������t�S;�Q�y�U��Z��*Y�6�I��'}����c���WC��)�A�G�K�X�j�Z�������i��H�0y�RS���z����H�_b�-�OB$����-?;���eA��4�����Kv��8]������%�o��P���c��l�w�['�����Cw�qx�Q��aR�y�mnU2@���q9<��C�3��	*OT)��!���&wF�R%1d����&1�����~�xO���y�%p�����}_D�����L�v�o��?�!?�G���_���y��3I�X�qA��%W��'e�Z��U���������,J���O>9;B����J��(Q���+��������������\$e�Z
#r��al�N��L�ab��y�d��q7�[z�0��b�C��pt�t[�hqI��/�����,�W�����W��WG:s����{�h��D��'�����1�|��c���7��v���e��7��E&�Y��,ss�h�D�:�r������g�����~#[�ht����W��od����&�J�"��I������7'���J��i�8���oW���M������)�=��_"���)��#fj�_n9G��l�X�V62#�4W��{�m������j5N�J��~����G^hY�����t6����n����A�>�d�;��u�m�?QL:��k�[�f���KZ�v��T������:_�K��-����y^:�m��I��@�ttr�o��GG���8=lv��q��{<<L[�������Q[���]�P�#�l>��W	�f�<R��F���E}�������+����b9��a�1�O�V��(�H]b��#�"�0�j>�=����&�A�����l�z���������y��u��2��k����
�@��I�&��lRT���;���O��M���T���BmY��w�a��}�&���1�� #x���f�)u�F1�����bm	���B������%�"���B$��n
]T
]`�fS��%��zq�<h<��i�+�x�&���2(Y��NS/��\�O��M9^���I�S��\��L�AN7�gXCcE�/�������GY��|Q68\�5:����+8�%&���y>�����q�g��v%����z�y]�����*�V���%�K��g��N��!�Cs�p�����&�`����l#�S�O��]��U�P/Q�Z�����
�l��W�h�5�n�`8nn���D
�I���+;]�5�i�i?���px�l4:���a:�9���ti������U�#�_�UQ�.k��i�x����{�q��lp[��:K����jq��#��F��������^:-G;��x��~Za3���1���D�L%�'���s���K/�*��H�x����_h{A3?��N*�%�
	(�m\3��I�h��{6:�&�hjo=P2d��Cv�g�=t�"�>��v�G}H���>o��@<_\U.��"��8t����a>*o1����A����Z���($�w{��6�t�&"R'�A"���A`(�n�����!��O��ZP����>�O&�r��L�,���On�S���1�.��A�l�
F�����rB���o�4gK�^��W��3{�~EF%)s�<���v���/���6���2��Z[��[I�u��=�N��r~����*[�2�J���g�!�����c���N�����-
�a�� �E#����6�)#��kn7�ml,/;�9Ib�������r2K.���������i�lEM���x���9m@c_�4'��[�4�x2���z��)���s:p�y�M������sS���2^`�N��8������������)����$��e�e�
�-�L�J7K�a"�?*��_fi��������gY>��Y��Pi���&[y����<��[�v��3g[���.�=�6�=F�����x����������E5Q�p��9p�b��\����Z?l��s�+��N+/g�=��J��t�Z�c���QQ7s���t�O�	�Sx����!�'�t"k����jt+��n��%�N�0e��� r-��r�6��O��AE(�M��[(F���K�(���D�A����N-=F=xB�&Q��[(����M�2�=P��A���MZ��"v����,6
�\I�V��@��z�Q���=B��f`�R�$V��U�0N�6�?f�L��"��&h�Q�O�l&��I�/�^��7��$���$�)���PQ��R�/����.��k�7���t�LH78^`�[i��vDs����+&!�Q�HE ~)+5..�$��Q�sv� F�4'T�5���rNS��x��O%�;RZr�5���6���G9&�����n.�r�X��3�y�B�%;K
E^���t�Q���[`P��D-�TS���Pb���
0�\������AI�;e-t*�"�����I��U��&q����`�[�
q@S�b��@
��R,(�3H�;���+OkOc���+?m����c��/P�r����c>��K���`.�9zaK��K�gT�����H,1D��4�������Q:Onk�^�+f%�JVOa��xd8���JI7��g������t��hj['��NJV[����"����{�\��h����N���g�[����z�e�5���Z�����d�[z����$P��/J:,}A���Tk<���������!G�5�����d�?�6W������{S)���U�)��B�����M]�t�|�E\l�Ok����+���g�&����W�<���)|#�����u����#��^��t8,�#�����h�����S-����WW��oW��*9@��|�ru=�������4�����h�3)G=��l������"'2�(}��T��t��W�9��+�$]2�/�
��e&l���q���2~�K���p�m�h���sDm��}�Mq�\]�gr�����/���� C9"�i+�e%	�T����J��E���c3��M���a� ~L��-H��	U���apg�f3�>sXY��+�y�������5�u0���0x���}�����1�}lk�1�%�g@u�C��G�t�iYI���)�BL�����1�#����d�@la?i�{w�������y�*u�������x�0���^���1�b��<��2�1�x
=pU0tf]�cS�]��I  ���;R���7�7I�j�7��UD�c�]����]\����{7�9�A+���{�'�����D&���Fl�����wq���-���V���xF�mX��H��U�A��JM�jJ�UW��C6�~�����hN��~}q�ZU����7��<;U������a'�t:�F������+$-�s��c/e����MbJ/�I�*���/�P�D�L�g$�M��^�
���
o���__�^���c�������K�����{�:�5#zs-��:�fT������!i:z%���%W��n���0lT����~������f&��#.=�3�nI��4�A����m��z���������]�t��|�L���.�~^kI>S��3n*���0=?�_ �
p��}�6�qV_Xf��Zl���er6���������c��Cq��Q2z~I�����,���Krv��x���\8�oe������e���n��#��I������'�
{R��4��a9��v{�W�N�M�x�.��w�����^��@����1�P%,���������t�(�]�3��1�D`�?s���S�z8�y�h���������l��9+���F�=0�Q<kC��:m����k8(�,&��:���b�s�a����������[5��Ni�H�n��$/�����)�O�,��@�_�-
z��GI}'�x���h��is�c���sH}@nmK���k��������,m��(��9������� ������;[�\T�������Kl��3�ws�n[n��*�y@T���p�.5�xE�d���(zy����Q8�n��<��	��(��	U��jK�\�n*�"{e^��XN#�bz��rB%Y��}~,�����wYp��Y������gR/�U{/����P��fj��D���?�O�U_g��$�NZy����������".�4��d�6�Y2���.(&�.4!��K�AW��Q������S��Z��t0�������iV���u2��P	<���%
>M�y��<c���&��~xT�r�����N����fR��1XQT>&�E?~6I�99�_�ET7Z �|������o����s����MTF�L����*[��� /1����HA�N�T<f�^}7R�|~���$T��Y�0��|�ep?����fW���U��4D,��T�~���Nh[��f�s�u_��d�98�����_���#���KIC��o����.P�i��q���"����w4�5��T�0�t����J'�2����'�cen�[f�Z7���_��t
���o;l`�X�������c�D4�Z��[����s�c�����5\�����P�o�
��4�i:���!aK��&�&��������s}�������>�H��0��zN5�%�4���z!���k��]������|A�7�
�d�\nj������Q!�,{�q�\j��'�>���+d5�.�eA�0�S*����+�JD;]u��=����m�)�$�b!��g)��4��}H�@�dw���C�f���=��+J9s]v3wXYB%6���&d6��1���<�����<]���K�-C��&h�J1Kx��=���sv_�"�o���H������$���s���p^tL
�U >�S,6������b�F&L���gl�t!;3�l� �i�ul��1��18�����p������_k�;E/)z7.�P���/�ap��C�pl���A?���!�
wG<�dc
g��]aN�����R����Pw�����;7�V��������`�5��������?�|���^�)Z��n�Z[�B��������R��*��5��C4@����Y|���M�q�tQXYI��A�X���n$|6.n��]��(%��[�$��g�b,#�w�:�s,�%�8����I��Jx�[��������~{u��xsic��=�����.�@�z_��sM�	��!�o��z=�H���)�I:�"��#��Z�����k8w)�*@���-z�����
�S��N�t�JPj��@n�O�)~�]DSM�
���2������j�V�E�����8�}w@
�n�����h`d����;���y>��(r�����<�H���,��b&�z ���m�c����5f���Q��	�@�/�gK����^6�� j��U���j���t���y��k�����R�G���!Sg{@�>3?�E"����/������r ��IB+����7,=�XH��{d���p�4��*��M���LJ'<��u�\{/�"�q���:�9<_&�!k
s�;���2�"��������\XF�������	h��o�y�T���<����L��`�~�����8�����d$A�.]z@;�_�Ak��R���K��As�����D$��J>
��Q���K�KW%�W>������$p�Y

z
����O�I`�_a�gEt\��nN#��������)����.e��j^���D}�bU���Fc�=3���?I��9
��&G������(v�9H���q��f��x�=�7��n7I�����$�ns�m�Z���uK���Z��������>$�#kB��T�t0��%}��8 ��z$�V4�F_^��
W~4�
��x��A$��e�6dS�	Q���?f���V� �V�l�G��r]���k���@����D]b�9�
d�1pBYH���f0Y�
!�v=t�$Ee^����`AY��,�-��3
V�8,Y+d�Q�"�W��8��F�Q���r�%��[S�:���g��,�6��	C�+�����:�&�s�s�L�dI�e{�IC�3���7����PD����|wyv!l���1F[�n����D' SW���C�b�6$Y�'��Ud��c���RC�����]��	�����lp�����0Z�H�	�-s��]��n���0��V"��F.)��
w�	p	3<�BCJpZ,]�� Vi��k���&<���!�3F,W#�H;�����G��ct�#��i7�z����d�S3v3&8��E��l���8k0�RR�� ��k-��7o�s}{D6�U&�2O9���e�n��3���Y����w��n�V\v}&��������T1��L�X/��FW3�������)l�TE�fB����r�������yz����xH�\�a=��6L3"��|k����v?$),�([.]}��;�Md;^l����v���|�Fq��(X�W����"J*��.WU�dr��io����>�; {��0@Q��o�/����+�����h:�s�����G����\����5�����D�A���\�E^�y���B��������Fh��g��1�����D��M�rK8i��}���285��t�;���,r���-�����^��Cv���i�Lq�-l�D0-���"a�}��?���Q�
��=�.�u�����w��X��Mzu%/��OY�g8ae��qs6�k�lQ�<'n�OE\)i�9U_9��*�&#s�I�.�)hjP�����Q�j���~s+�\�]7����D7LEU�J�<���������*���o��9fI��B�?Hi
0����[�H����3�F��������h�^���h�*N�����f+���%
X��t.J��Y�h^a�`8��p�2��7Z�L��z������G�t���������P�c����S�(q���|;Q�_]�8<�QU�!�5��S�	�&�\'\����L�O0������C< (&��\�pC�W�NtlHs��w�/�D�s�]vb��I�I����Ms���7~�����=Q�ER����`<X^��}1�x06%�������������9���Rx����&p~��XEaUT���4�6O��o3�VY�N�N\�~wp�n4F�$v�v�
!m

�����{Xk�����/���:`D�N)����J�4��g"�)4�	�������?��a�����@a	f�NKJB��a�B����5X};�A�@�3��������#9�q�X��aFap�
�#���m�M�g'kd�3�e~��O41��#��e(-��� ��8_fW�\�	)�Vr/�g�P_���/�B�:3���]I���+��s�	�Od�F�p����3-�I1U^9�{{���e^�T��~����M{�3���?e_V�����?��]�
 ��I��$X���xn�����
�=���9d
����w�?�1tEZ�u�k�rM�=�Ll�lnYO�T�5�&�FW�Q�E��6N���g��|w
�i�+�����Nf��(k� ��z��$���������$�~L�LM(����O�h=qS}l�#��)i5{f�(�W�x:���P���m����cst�47R��(�w %�B@m'[����-$c�Pt�G7];�������������%���u�/��Q�vD�%��	-���{h�������y����t�A��l���i����L��d��=��|H0��kB8��DHIo�_7����4�@����(X��P����`��F������
{�38�.���w'��P=����R����m|jW�������~�[/���TH}Q���K�Q��Y��3v�Dw�]���^���-u�L��e��Fcxx�nw�����,s�5�t�5,�
?����H���
jn0�F�oYV�.&��dWw���[(�e6����1K��$��A6���{I�.1���y%^8,�!&J��_|P�2�<'%����d��T�: +�`>�������A�dU���g����'��tG��a��~��s�gP�`���a�&�#�%fV�u�`�m�)6G .9!X �>{�{������%V����q^�	������n�'�q�&-H�{T%u<���A�K��*/���HS���!����C>y�*����4<�/
��N���&w>��a����h����y�Y�r�;zk�ssN�"�s���1GG��t,�!��lJz�_���Kk�x���cX���q�%��v�9RUg�4cD	��8q�g��L�$��������a1w�M� �8I������E��i�dH)w�~�9X2X=�n���#�/j@5MJ��g�L�����S���5�&oM�{��)n!T���V��r����E�l�h��kZ�w�-v��m�uI5�=<ND�������&��',��u�iC|+�5@��Z���4r����6s[.�P@l,�oNZB��M�qt��_~){�E��2d�u����Z��������X��d�hd�iJ���f�IG'D���c9<wf���d�1b���������
[��F�a
K"�}�e}�_�j:B�jKP�`�.����/?nuk'�E�I�Q[���
���x�G�'�������-����Z/0��|�O���z/��J�V;L����������������`b������?���|$����w1[O5F��r�O���c��[����7��y9���l�$KKI2k(�S�5rtT��'�6���i��R���n������G�t#x�����a����������z��_e�)�_7�'G��f'���i����k�������?�4���a:H���q�8m������yx�'����W��n=�=j7�������6�X��Z�����-������l�|�����pyQ�b��j�:��KgQ�*rO�E��Kr���X�4_��
o���c[���Wu����`4��0��A�����#x`Kcv�J*p��.3l�;��P�#�?lu�����x&�c��k�
�D%J������%�\�~'_��S�y��73���C%�=�Z'G��A�e��=A�N�#b=���q7�I�wD��?�ogx|������8�v�]_�{G����G��%�c,�G?[	������*����������iE�'�x�����jK-�7��\�y����g����^��
�pp��������wg�*�q@�A��0��S<�8;�<S\.m�X�Z 9�>����c�VY���L�5��_����o���~Hop^��Q�,k����_�;��T��/�X@��^�=�T���j���T�������m��zq���"&��3��T�E�9����qz�	����:'�L������o^����<���y!k
M��O_~w����6�A�p�����hQo����T����e��z������tc�a8�>���mx�N��q����8@�{}���0�A��>}���'[�
�(>Jb���
M�QL|4M�%a�������8�&�M���4��i���4�44M�h��h`�u�.~&��EN�������H�$�S���������U���w��`b/��|���Z���J������t����~
�������V�����������67�����H�����]�Qg�?T�UO����s��P����0�1������_���G}^	\M�0WT���I����o���^��*�������/$!�;������*�G�Z]�����a�������T��A��?�x�m�I��'��T���B��o�S������E;��uZ�&	�_���d�t�Q%�G�u�Q�����1�H����{��N@�$����T"),�����;���m��
��+���#��p-^�$XS���(�	�/B6�ss�`Q�?>���}g�
'��@�����������%�?^s:�%t������_���mtl��'�|Ko������7r��������R���u�Q�����N�/p�G��S����f5|����/y������.�����C����$��PxP@������>��pq��xu��[��@�!�]j�=�p���x�Q�A�����	M'��Y$s������K.N_�K\��w(�!v���V����F�������p�0������I���8N��tyL��t�J��]�mV�=�A1���bcmE�m����N�qt�6�����(�
��h{r=j��~vQ�2Z��6�>I�mR�<z���8G��t����_QI����w5�?���wE����zE������7�A-�osT�u������������;�-����2�n�i�q���S�6�l��q�������C$:N�����T��5A���k��"!���z�����(=�i/�k��m�k6�������d�9~�����|��TJT�
r��t;�_��<f�s�Qf{+����
2��8�.p<�%���8��k�-�(�������j�RE���oQ�v��.)o�v��������W����(���u�C)��63��?8!��^�������}����e��oQ�v��.)o�v������g�.I������pA���z���m2�*��nV�~�R�\k�Y_�m�_�?�Z��7����#�3��BC�.�����v���Mn�&<IT�U�/������������w�hOx�V/�}��"�� ?j�	��+NX�m����	��mS�5�j��k��c���r�?��qw������6;�������l��}�����S��]���s'J�v��}�l��+ql]B��A������#��v&����������D��m�D��$[���e)!���0;��_gi6��MKSB�?yiv �����$1��_������f�������� ����~%��E���}*Z]�����fb�2nI�<�������� y�|��k��W���c��}R�����>��Nt�W]�26�j��V��IK�w{)}2��.���v'��_���uq����U�X����s�Y���.]A�(x)�mY6���0��_si4����1��OX��w�.K��~m"�[�1h����G]����Iz?�7m��]����t����wo����g�6�m �
������/�{�2js�a����1���oPK$�G��������H���W��>D^��DJ~$�����mek6�JrU�ME��GJ���g�j~��I`7	����j�
����x���Z���Q�`��������~�L`G��M���D +�zB=���+@��f�M�D
z"{�D@Vg5zz��i�9����]*�O����H1���p��Ew�������P���~?���X#6�S���c��1e������v�������B�0� 
+�[�(q�i��<�6���#M8���x���5=��0XJq0�|�&�z�����{��{'���
��M�������p�'Y�3�o���x��be�Q����Ql�!�o�����pd!� �Cw\H5�#���_#�������2?&���>�c�7r���ie��)J���uC�u_�D�yX����c�#988���rO����K
/�fQ��������w�Gu���;"k�F������"r
pwD�n�h*v1���Edu+����A�4V^u'�m��y~��[%��	���qY�o��oP���TUK��0X��{KS���[��
�����TK��0X��{���)%�q,�U((��*�Uo������{�*��y���q�H���7=���c�����3��&���Zip@��?�Wn��6�!q�k�{�!?g��oftvf���������T�Au�[���9�����]Q�.�Y9�dT�����I�%�����)xO�B�	�V���Ui������TR����*��}�L�
X�~hI�0n08^..(�2�%j��SK��MK�P��m�CI}��T~w��O�{]����(:i����GOj3rR�E��8�Z]�YLr�����������ra��~y���I����b�p�k�F����&����_�#"�vM�>#w|V�wZ�x������.�_�f+���7���1t����������O�����BE�.F*����Y��}YU��b�F���a8q0,�0sw8���������'Nk�������8�C�4�NF��U�{@�t0��46v0^�2���X�c/�!���^�Z�������U^�nA{�r����RZ����o��S���&q��mEp���C])�C�?!���h���Y&*+�,�"5����B��x�T0��=\����f�lj%��Y�]W��i��hI���N��cu(��0��������\��k#�Y���I_������o��C��]�R�+v6�
�L��x
c(���o����7������2�C{����9Nh�3��/����������-�
���y@�@��s�!;������?�x��R2�M�^�m3Y���0�Ta�uhZ~��6�Y���@u��5th�f���^������M�Y����`���$�%=��OUt>!/����8��}G�c
�������~��]����h��
����t��,�{f�bk����e�~��=���p<�:���h,����e��Qm)];�U�3�-�D}X�E��=�^=�����b�X��|Ik���
�3]b8��C��%�c�:��A�����H<�I���
w���:<���&`t_R�s|���A����`��!X5�^�7;�����}�+6�AQ~a"��*4�
m����"�+5&�$c�:S��PO�QOSO�N=!L= ��H� �����7�/X������56i�4������X����S�����r�N�����mk;k.����i�4��tx�4����I�!���	������S�1��x���@�����I#�J�~��%tGK���!�i�}�aq:�0a��(��~�����-��_AL����a��~>Q���v<&��38��n�=E�
���4�"3����P�>tr�#�d��)��P�����^RT��8��2�q_�lr���0��|��*�tQ�W��6�N�"K��"���/G=`�R� ���R�#�������./Z�	�]T�z��Z�P���H+e��Q5
�
_Wz�m�c����K����/������gR{f���w�V���wo,�:���f�Z��ze
�t������7��+��!�����5��������������z�{s~�A����E���1��:�]���X��hG���;�hm�'/;�W��y�<Y�fgP�Fl%��0�-�#����������a�'�y�s�hw\B��D��%/���9Sw�,�Qp}�!���������ny	����$r�-/������^��r���"G��r;���@g>�r�?v�~�qq��g`�g�]1Tc��'d�sJOha?��6������v5pw����>:��N6����Fg��v�����c[)g��b-j�b[p��ZZr����0G����K[Z����0�zD��6������m��)��r
�r��v������JI�G����	\�=����Vq���Q�i������$��c�<��>������BR\l����}�m���' }?����`��+�}#�W5���(�r
��:P��M�������$����Ww;o��eA�N���5��
F�2��C�PI�=\5�����P#�ZF������u�����e��6�(�&z�G�>B��i}�}�=�i�W�:2i=i=Bi�n��n�����5��������*m���W�Q����~(���x�i���/(��(j��3���Bu�7B!����O(q+�Z_�I��l�0B�`f��l*<b���/$�R���$L�8�:���.Go7�c��B��m�~~�����v����
�b����`SPC��~���7�
��R0��/9���gD*��\��"
�;��v@��'4���v�u"�3�5������������s�t�y�X^�����E9]M/��f����nM�����Z4C���o0-���J��Q�^�]�?��������{����b�Y�	a�� �)������^h=��7�3������sk�����S������v�mxg<�9�
r���t���,X�vl}H;�IN
�j��@e��y�� �;IPwM���j����s���[�f}q��������q���[2�e�yLX���b�� YQ���B{�.
���������)5���4�g��5��H��FCa����-f��n~[����!�J�������j'�{�)����u���&�������}+l����p"*T�gEV�kK�j�+���F
i�c3��fx��NG����p������#-�5�YN!�+����H�����@I��n����90���`5�ZO�������U�.l`q�^��t|�s���
������n���h��@u�6��w��N+�~�x�Q)m��|����q� \	-DwB��[����2���c0���^��Xt���$�]���6��lr����0��v�K5��N�s�������7���4isg����m�����uq��M������*/�W�w�J��}^��V���7m^=�y�v^�{����p���q_�||�%iY�l��&��K��tK����l-K8[���A���������s82�F?@��,��-��L���L.��2�hyR�`��S���/��$����C�CB��:������J��l�FEj�T'���d��w}Q\^�o��
�0����5����Y*JS_qQ�2�X��/��9�a�����{������;��_�� ���.�v��
$�|!����}����Z���`_���C
�!��PCz =tB���e#���qi����HCz�"=��H�����Jw��j
-�t(��nI-��3X��-X���(\\��#X�X��Ak�D��:����$�r1�D������3����7�\E�`,��M�.��&�l�t�]A�NJ���0o�n
n��O���������\'������e�"a�%��		�5yd�P��k(��\�KF��8��z'�;���)S0�|MxO)�@:6���������p�,���4`af�6j�{(M��,�����n��,�=�}��[@���s����4}�'����:E�<�(l�ij�%��|im����|7�`og�-6�������G��������*�\�+�i9fR�c&�9fR��mcf\�<c�'(
��?�������o�:��L���Z�ukw%��&�C��y>h�:26�d#R<B,E�<E���
������<�����u� ���cKd����!���G��d��1�ug���k�-���y�E���O���u~�G-��U����3y(�E9h�M�0	�%�,&z�Z"y^�����N� 
.����v��L����Rq�m�M��h�����(�r�v0������R_r����(�aq���B2�Ho���`���Ba��1����"vT\(l��p�����=*.6�����N�(�qq����&����nP�7�����C����[�mq�����n���;�]q�����n7r;�Ga�
�����v�a:0�C���^)l��c�N�����=,�g�����&![�7����M��[;<�6��d$���g��SaN����5:T��^����2��"���J._,�\�>��,�c��x4,zXt��������:���=�����A�}��&�����Y��A�O fl��U�c8��t8�rL��r%k�=2��^PJ6ro�!��|�����C��`E������g4Sfu�r3]w��pU��Jn�-=U��"dhBdP��`PV��/�m�F�9�
��p�pX� ��+�#��8�@������#��c���J�X�x"x��aY�=�������8%��F�����Ca��r�A4��<�#h!��l�:�G�
�YVy�@X��,0"�F2#6�)���E��t3\���4	���t�<.<G��x��d��b>O6���������:�s=�#�u�1�1t��A���.���E��}�����9���B/Wx�A���J�as�L#R���%�W�����G��g��%#��4j��U7R^dUw������^~�>i7B�d��O��;���v����r���79b������}����H�u:�br��[����3O}�H�&F�o���d��w$���K��S��q�`�jb�Bj�S�����?u9g�A)�T�\��*����U`T���tzg�P*�SF�����ut�||����3����X�	1Y���O�����K�e{Z��`���������s_��.��l��C��k���i��l��+�L�c��M�=rSX?o/�����!��;�t��$2�>�`��Af���P1w�+�84�u�,������t�0��sn�&<���n������b�t��W���z�d��1o��r�����J�]r;�c�-����,��������_!���,��w����t�����3��F���5@��h��h���F��Q��z_T��6��F���7"�����CH�U����|.�.��K2e\�@cv��UF���R� �x�1_5���A�e=��Nx���<
��J��"����9���<��|\���6�kj�q�I���5��8M�t��>$�iL-^:[0M�p~7��]]E�q�0�����htFF������������r8������������i��1� {�f�����8�=P��zK��"`?2��<��p�t��K>K�d���,�OY���l�4�x&�3Y<�&L�H�3��	�d5�<>���b����OO���lA�h�PCr}X��<��oc�v����.S�Y0^�h?�tB�fA��������t=�#���t��7e�OH����[=,6���uL�*2���8#,��u�X�&L������;�c�t�}L�����jEj>}X����6[F8�<g��0/P�~�f~�s��zv���5���Ytu5�GIx[3�L ��2_c�����yE.�b^Q!�L�MFP���LL����c��7�m��	���|L���P@-�<[%���� v�~H$�y������D�����!y�^�m�\E��? h���7M��\n�1�{&����T�l��S�<���~��B��IN?��9���-5�Xd���+-�R��9�H�]j~���X��8�gy�A��1� l�Y��6hpA(`�A���iF:��`Kf5�y?��������!U{OO����gY���v��HJ�n#��)��]G�B1G��"t6���t�F�(�T��QC���~�������T�H�<�&MmA]j��Sc����xdzh�~�
4��f���u�\���
LE6��(X�s����:�����
S�in����3m�BZ��afN%�6����?���)��*b�����=�������.��J��(���"�QF�6*,~|N6��,��^]�$��&�v=��z\S������ |��|x�����K�m�c�{'�������S�R�P����4�z;�0��%����",��
e\�h~����)���0�i�L��\����f�d4U-�����o~`����fa"��4j�n�#Y��g|�m����sz5-�5�m�g�����{��CSy^Z���sLKd}��C��)"�n��V`2ZX���D�������o��&(�JM���F�q�g��?"�#M�l�6�>}��eq���*��gi�O�FTq�*+�$��HM�������k���Muz���$�%=���r����l���v�:*��
[���&
vc�W�SQ`Hu*C0�+�FmX�����8aJ�i�%�(�0n|�_�6_E�W�������ew������]�~Bw�	a�!���A�a�������&i�i
��=1�B13a[8e4��v��|�Tc%�[-��6�~���������r�����A��a����GZ��G���q���O�#G�=�������B+y*�s+E'A%�N���z�H��o����0��7]�
��b/
�b�!�����I�9����{��n[�����u�\���
"��bo��-r�,YbsRV�[�9s������������n�>�7�*:�����F��d���Q��|�_eF���]���scv�1_]v�.?�W�����Sb�D'^W
�#J.������]"�$b�R1I*�.�	$c��1Y:�W���P��O/����h�
b��R��S�Cy�8�'>��}6��������[W����t�+]�JW����t�+]�JW����t�+]�JW����t�+]�JW������)�
#137Rafia Sabih
rafia.sabih@enterprisedb.com
In reply to: Ashutosh Bapat (#136)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Mon, May 22, 2017 at 12:02 PM, Ashutosh Bapat <
ashutosh.bapat@enterprisedb.com> wrote:

Here's set of patches rebased on latest head.

In an attempt to test this set of patches, I found that not all of the
patches could be applied on latest head-- commit
08aed6604de2e6a9f4d499818d7c641cbf5eb9f7
Might be in need of rebasing.

--
Regards,
Rafia Sabih
EnterpriseDB: http://www.enterprisedb.com/

#138Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Rafia Sabih (#137)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Jun 30, 2017 at 2:53 PM, Rafia Sabih
<rafia.sabih@enterprisedb.com> wrote:

On Mon, May 22, 2017 at 12:02 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Here's set of patches rebased on latest head.

In an attempt to test this set of patches, I found that not all of the
patches could be applied on latest head-- commit
08aed6604de2e6a9f4d499818d7c641cbf5eb9f7
Might be in need of rebasing.

Thanks Rafia for your interest. I have started rebasing the patches on
the latest head. I am expecting it to take some time. Will update the
thread with the patches once I am done rebasing them.

--
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

#139Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Ashutosh Bapat (#138)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Jul 4, 2017 at 10:02 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

On Fri, Jun 30, 2017 at 2:53 PM, Rafia Sabih
<rafia.sabih@enterprisedb.com> wrote:

On Mon, May 22, 2017 at 12:02 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Here's set of patches rebased on latest head.

In an attempt to test this set of patches, I found that not all of the
patches could be applied on latest head-- commit
08aed6604de2e6a9f4d499818d7c641cbf5eb9f7
Might be in need of rebasing.

Thanks Rafia for your interest. I have started rebasing the patches on
the latest head. I am expecting it to take some time. Will update the
thread with the patches once I am done rebasing them.

Here are patches rebased.

As mentioned in my previous mail [1]/messages/by-id/CAFjFpRdF8GpmSjjn0fm85cMW2iz+r3MQJQ_HC0eDATzWSv5buw@mail.gmail.com, the last two patches are not
complete but are included, so that the reviewer can see the changes we
will have to make when we go towards more general partition-wise join.
Please use patches upto 0015, which implement 1:1 partition mapping
for benchmarking and testing.

[1]: /messages/by-id/CAFjFpRdF8GpmSjjn0fm85cMW2iz+r3MQJQ_HC0eDATzWSv5buw@mail.gmail.com

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

Attachments:

pg_dp_join_patches_v21.tar.gzapplication/x-gzip; name=pg_dp_join_patches_v21.tar.gzDownload
�UcY�]{S������S�M����6��0��=��K��7���R.��Y[������o���Y#��0EaK���c���#ya����~�����������[���|�h����UM�����o}�C�q
����\�����w����/*��Gk2��o�,��4��|a�Sgj��+����i�V����m��|Dl��ZX����^���}e?)u_o��[������Omkz�`|�jM��U���e���h]�������D�;T���eB�,@iC�u��@EVh3G��y��
�>�YsD���%�U�LFMT�7��%G0����?x3l�V���HUa��k�J��F����cgp�;�U~����b�
���kWV�����h���V���TN�������9��Z=v�����T�.M��l6��Mm2����}M��8�N��
�7kjN�k�?31M���tF`b���U�;z����qK��6B��T�C�C$S<k���cs�1���*�9��h�y�fR�M�x��X+c�D*6�P�pe�l��z>�f��T�f��hT����#c�������3�z7dRs���K#]*��e���0����LM�����
Tk���?��U*��+h4��.�y���jT��������5�{��aK#��$��G��K�&�����P4�����>�O���N�p>u\��&&������~��Y�������
T��BQ�(�1������2���'�}TJ������.���+�E�O3���`�`f��l��M�*���*
��	m0hW����<�p�A2�����U�Wv�{���L�u�����}f^Y�MO:���<6�)��N���5�������3O(�����h���Z�O�=�P�W}"���bF�����FT�0�n�#�c�W�#�z��j�B�����X�MS���s�TN����su�}XWWA�Z��_������DQvDD/�<r�a�o��]���N[m��pVok��C:i!��n����3���#3��#������V�.aN�6�5��^���&�7s��J5�,o6�U
�|�����������WS\���������_q��{�
g6�&o��Ctm�����e=���kh���Wu�xS��8m8���jL[��%�)MR�ELl��l����.{�������������'����O�����������I����lpvyQ�n�i�\W�EA��&�b�2��/9v��W%UQ�m�Q������A��B���PK��u���0�k:���^Cu�*��[�����V�<���s#^��g�&�g����i�H���~%D���+]v���U��dQ��&�C����/��}.)�����Z�{g�������G�	�f!>W�hc�<>�Y��^���
���Y�����Yr���MU����*8�mk�\e\��P����+�����=<��C��t�t
����U�y��^�9�8�p�w��6�#���	S<����w7��
87l���g��]���?�0	�od
��a�?l�[X����P����[5p1���L=Fu��w��~���X�37��S��t;����u_A3��x��f:�Z�d�[%�UE��-d1*$�R��(h�l�W��_��p��V����e �	,'g�	HL)O�(t������1d�qc����jF����x��t�>��r0��b���d���{�"��O$��'�k3�P��kq�����RW�����uE<7�2H�����yk��	!����(���-qk;��J2!�@�f!����P��1��Kn�x��~�����VD��r0��4<t}O�3�k5���&]p��No�P��?�u��?����#&�5���6*�U�|�(gV��Y������-�s]t}��O�5.?���F��|~���a]Q�Q����^�����	�]�J�c��~��	'��?�Y����&�K:����>�W���,��v�vA������HN� ���q�Q,G?��!���&A�/HW������X�Zi����P�����a�]XC1i?rex3���If��0�o=���K��D�����1��W9��r
���+�)��h���d�g��X�k
w�4UKc��W�9&��W�s�����Y�7S���`�W	"�:w��%�C�j��+)�2��L6L`������F*J��<l���_��|)A�������z.q\�V7�G��q7uH�f*����e�q|�������=}�1����m�%�����'cM�&�J�-��u��Q{l\��F{����7��������H��>������>���
�l�V���������G��
����z6�]��T����k"� ����M!��vJ���	�g�
�������	��]�����t{�\����I���`Q� �>�g���S**H�3`��5a��3��=MLgzK�~���-���C�\���	����go���3�o�[�tp�aa������w���05�l6w��v��:�?)g���N��\��O��ima����)�a��bt2:8M��t��@����|�eP�W��u3j*��}N�1/����%�@w���5	���d�sF����lR(H����(�&��*b��;��{����DU�lk�������l��P�O��Z����!U��o+;���E-��n�J37��(�K�
�Y���@��q��;eh;���u�i�p8#�������fe����uq�'����.������Z}��f��+����Z(q'��`�������������������)G0E"�q6!����d�����>9J�^x;e���L�o,������4�?3�k��,��t��t"f�E��8oR��,�s��|�#S:7.�K�b,������;���N��H]�9�"��������tl1]�M2�ym���!����,j����i
^�����0��;�"�v�4T������*�a�"����y��jPgh���B�U�<�
�*?u�?u�p����V�.MUUoe�������v�[Y��2�P2X;:o�����
����;������u��ZC��*~���#�������MO8�O�Sm�A?<�)�Sl�s�9���8e�Cd��u��������%���
�4�j*�%&���Q��:Iw D��&������2�|F�/0��7 �Hq�c���l���CH3\-�B#��T�
�EUH�^�JZ��v�UQ�NZS*���BW���~L�R
�.)����*W�%R�*��MM3�+;�����sv����w��n�k�G�����Zp��Q����_m���q��@��W���D��x�jC����T������p����2�B��PaP��S��3 ��U*�{D��o�.#��s:!�A�*�.����J�T�
@g� �$6z�:G��u�d����5tl��#�2��fw��*�^���)F������U3���MF0���AeT\I%[�\e�J������R�]5T��mXKc���P�]�'��P����H��Ve��m���z���J��%fI)��vI9J��K\���5��+�������B2R��pX�5�K�4W$/��0KJSc��J3�?^�L�fU����;*92X`b�c�k���$�@[X�b��������Un������j�#:@���2��;q���\�`y��!�wO@=�P�������	}�e	�C��}�A��I� )���!)z�
���$���YA��.W��������(,����J��*��&�����l;Xh��������%��!8 v�'�����h�����\L���H��Hi�����B{g�?H�5�Z���u���!i��\c#���B@]�[LS��=���ZbA�����F�<��YX�����0}�^y�-��q,����H<�;�r�.)y��^�-����AgAX	����������60�d����u94���`.�1l���U�]zy���}������P��K�����O9�x��h���������
~��h�m�����{���sL9��E%D������6�y���"|!��1'�F%|����x�>5�/.���	�������(���q=�R@9|M�����Dt��
zeH� E�_�f�����L����ZDm�-�"�&��	�g��X�����P0n~kN�QVI���j�$jDZ^2l&T!\��RJ��VRI2��I�I)Ic$��f��t��q�[7�$Z�v-R-"M�R-�$Z%���$I���hT��H�U�k������v��{b�%������%�x8�s�R�����j�K��xQ������t��yDi1-�A��a����I�>��x�/�����|�/}�'^��Kr\[���O�l�'C�-nn���]dQ��B�ohk��2B���*;���='G.�U�����O6����^K��8�`�X���\�:PF<��X�p>���$f(�*T@����3��E��������YX������P�R
X����Q��si�R���ma\rn�Z�<�g���I�1��%�[��=Hb� ��I��N/�O�C�G������6�8����P����>/��'x�"� �e��:�C�1^���jkE����Ukt�������Z��2�)b����]�
��Z`
��C�l^�9X�F	��z&2$d"�x��R��)��2K)��a"���!2c)��9~H��k8k+`��EbJ)��^�b
(���R�\&"��"3��
���$���s�����	�p9w�V?o�����D������$N-�PXt?yJ_�����IX��d���[x��)A�%��d.&�V"7�k%O��y����y��Z�<��<%��d����d�J�Jd~"9.���@�jE�U���PP+75Y�Kx������b�/��U�c�����.l������/�>�+��g�b����bk����P��� �����b	�'��5�Q(���B���t ~����r��5��K�;��V���2k�Y��;�;f�
���)���5+�v������0Y�e�X(�!"��qh���=+9{U.�����pN_�Sb�JH@���;�E�����AbvEv�/�E�	��W1Ic0����+����`��x��|h4/�����
e�W�������������8�����Q^���K��Z�u&Onb��V� [v�mx����VsB�2[���{������@S���u�S�j��9�����FP��dQ7��!����\�7��q.������rkgz�|���_�-����L�0!��� �:�K3�Qq�qX?����	8��p6�J�"I�
�3�����YO`�l��1+Ng?]����sGk�,���b��b��#,f�%r����3wq��=T��J���:%��e������d�/v���A&���U�
����`����A��F��)��Q:XU��'8]�����c�%��|�!T!���xz��h�T7�6��m;�}�G6c�����x���$f3a\p+�Dp,��Q,�_d�F��������.^��}`@�����V��W� ^d~�A�&��4�{�X��az�p�������v��b������*��R/�b���C�4$��������nONZ{���.v����%i���z�'�2�	���m1��N�Q��IK_����p�c/�*������
�o*x�i��_�x��p��39�u+����M��M�/�J���D��������Wv��� ���:}��V������l
��Juz�o\���X����	�����,����������)>^
'���p���p�u_u��w;�A5D���Z�������{�%'�T&/S������?����"�"*��"���$�%+�IlX���e���4�9(���l�z7sF��&BS-�	--5E�w)J����������)�,U���!ShT������U�W)��v�sx;����%���M�-�M<U��xi�s\��Of�c�Tl����1=�S e�O�^e�L�0A�W��'��4A�_.m�e$�}JqO����-l�	>�dST��]	�O+�+%Z8q�b��FL������R�^Q�@&U}����|��_�'�����F�������A�?�p� h�����(~�Z��
�����8�5�����>tx�?
��� -�l��y�T����O�B����'[�O�
�D��&J9��b^��x��'a�<�5�t��P�k��G
e���G�l=���N����8�w���I�����t��m �P(���J���}�Av� WLC��(�%�������
{}�����		����b�W<^p�.��|�x����*�F#z:+����8���.�:���1��>K85�:����a�o�?vz���*0*f�s��[�b��v�Z���=�2�/O�nN/{����7��#y���OQq����=b)�������=�P�����Ol0H�ds��4|�v���g��[��@�(R�&�z@��e"��������A��8�~����s�a9a�z�W����
X^(��)����W���w�5��>�}�os7F����|��Ohd=�3��rY�&�r�.W�r�;�g�W��Y~�������_���}���5�r�����u������������U�1[����-�e7w�lS��(6��K����6�*o�~)���kO���M��^X��Vj�-m�j�T�\�rF�V�611Z�g����r=�isr1Z�g4m^F��3Z�0Z.3Z�a�\�h�2Zn`4R�(�
�X�5�'��@`��x���Z�IL��D���D
k��ai��4+��
qHn�@��������&��VE;
_���-��������cH.��^�8��K$	B�������~����<g;��c���V}��Pi��f>�}/5� ���R����d�`�+�4���M�7����c���)J��
�p��h�����&X�3��P���J&�
p�
�p;Z�#Xl>���K�xX�g��-�d��f1�@6j��,ok���V�8�����G!�z:���&yD�!�#���h��' �fq�,�������0
^:��|�
���d���m����lj�z����o	H�.������96���s3���Q�N��;G�������kj������52~-��7��%�p�������W�<}�w�H�G0#6�!�D��1g��+R/P�����5��(����iux+m����eJ*�n.���w�G��$?�a��V�Q�>�7t�3�htD-���R���U�{w[�u4���$I�l���7d�]G���+��F���o�h�`?|t�H��M�1)����m^��<)�lR��I����S���wR��[W�Bh�$I��
���
�_����>�h3)p
mR���NIV��7R���_%4 	�g�6�[lo���8��60OP���=K����={
���g_��2����o���;��.�����O{Y�|,>A{�;�[��c(�'��������C�<�s����60����
$)!>�W	��1��#5��p��) �?��������L	ogP����,!AM�	�Gus/��.o�E}VN0�`�������'�a�������#���E�Troh���}����GZ���e}�r1m	�_���������v�C9`�+����S���6��4���W4�A	
Mhh�h�t��LnBC�������"���,��	
�������&44Kh�ACc������?�8N�\<�������E�n�<�M�\4�����������FH!{?B��z��_C�mE�3�
�)��}?L�Gk����2?���]j&���$�=�*��ZE�����p�c�Kl	�w@I��RD�/�`?G��"�e^��Di�N����$N���4t�)���ZL�����0|g���O�aM}�'
4���0�8�v�;q����-$IG#Wh���'��8�������|g��$LJ �����X�I�m���x��1�F���hBi��R���n�H���Tyt�b\��OhHt��0��x��K���S�s7���F�Te�Q��E�Z��B�9/@.��3����E���T�u4a�r}"a��S)��"w�|G!�� S���h{H���]a��R�0�h��^��{��A|?|A4,���Y�r<+ M�g�����9��#��""�x�3PUBL}Q�4��:�1�	��SO�����
�c4�xyIF��^�����^��6�X=���FnC����Jb��B��p��
�kI:�����I��3z��=������� �l�/�7?ePX,��l��
{Qn.I��$���v����>�r��?�&:���*u�X]1�����kZ_kv�����-'���N�jx�Q$�- �L=P?���k��$+���"B��f$w�y�m�XS�,�e�V��$q[�����,o�A��������r�V9_+�^�)Ld�0�9�V���Y�Md��!�j"����Ld���2�A�G7%Y1��������,�l"bMq?`"��$u"�������f��.��.����kU�������&�B���f�V���Y�MdE��b^��(`"s��e��&2�nJ��'2oBH�'����,d�5�9�
Y8=$���o���5�$���@Z�����_�n��p���:��sy��#_��
�_��oD������'�-7�l���-7�l[N"[a��i$M#i1<yy�"{3Y������K��[�XV��d�(�����\��w�u6_f�����r�[��0��5~�v�����*���r]f����]���&k�E9���|
W�_~��3�K������Vu
~2T6�ZTz��������$������2�#g�c�c���]�V����c/���f%��N�E3����iCo��(f"��?G&I���r���oB>�ydT3���t&��0�x5F��0u���8�G���\i������-=�a���M!����FaL�����]X�B8�s�Y� �2f<����l=��g@Cq���r����`��wf��4�2���wi��0�1����Z�rR�������Mg$Rg��8��t�[5x��^��3S^������K�]�0��.pT�u�=TPa=�;������.@�C�����e����R����G�W����lw��2��i��mtT�1�$�����j�����;�
��9?��<�����
��c3`SgV3�f�qDv��{r�;��
�gN{��3��r��@�8
4��y�q9�����mjVF��w�zS~�.���q��8kq|Bo7�R=�Z��E00A$�;���!�y�'p w�Et�:�2���{�Ra�	@;	MSe�!2�$I8y�(C�.j���D
,��`<��*��i��9����	2a�O�C�E7��S
�j��?���j�A2l9Hp����L�&F$M���1R�2H�c2H�!D<I��O��cT����0�A�V����h
�t�Lk���d�r�����e�$0�H$��V]�OIv"��	Nb�/�f�~��u�<�
�!J�X��ghi�-�y�u�lsu�&�
�=Gd���:�GyE���PM����S�)�M!��7�C��*O��)]�OA ��#��)e]��{#RKE���.{���gJ`���-�$a!<5qDR��T*�3��Q���#f���36b����V�G|G�����^Xe�7b���$p�Y��t�+�3�������Q3b^������( ��w��k��)�U6~#Z:IG����1������{�4�������lS���m��\f��Ew�L���w�M�]������S��\�6���%�v2�R�N�Rv�>���y-�\V/�<_�|suq~}Q5������?��1H~�&���r����\�?W�C��xJ�C?zS�s9�\n~���+�v<��5��B���<�����\~+��U�o�[���/��_��@�Y��\��+�xEz W2��Lr%��P����RCOI2K�hR������������${Y����R��9���M��� \��!�� X�E��$��z��J2�2rM�M���p�M���g��������7��Ce�z���H�
�����������H�@�����&0�!3x�3l��~08|No}�������6�{�h��^w�L�:/H�Y�|��������U^r�L�t*P�������x�%I^J!4�9H��\�g��kz���8�����5"�qE�>�c�h{|���]�����S�2��Pr�B�m��d�m�z��������x���~lI���R�w�U@e|��H��F�� ��	�����8kF:U��l:U�<�N�Jc��Jc�5C�1�Tp3/�WT;WE�8{����<������*����(h�=e��~9��[U.��+�s��k��58��e]��� ��}s���W�k�������}3���ZZ�y�|����MWfS��0`B9;��}�N��" .�r��)B���f�0{bwF���^�M���N����~��jhi�O'j�a�K����+
�" �N0f_��=�����#n���_��"�n�B��������W�k�����SD$Gw����ZZ�S��5~��B�K^���^���^^��p�cV%���]k��e�F�XL�m�k;����]��$�w��iA�I��mS�����a8�<��Bb{��J�����Gp,��Vp�z�p3��g��=�:���SK�����,�,y�&����������4�2��Bw�Q����F��\����������C���k������:2�5K����63���Z��4�2�kDg�Q��M�����'�q/�	��������
��MC(
�G0��H���k�����RX%���v`4K3���fo��(�`4oQ(7"���njl�vF'S����)i���}k!�+6�����w����Hm\�s�"�h�z�8%�� 7O�{KM	a�����'� ��1LSC��V�0W��_�%N�',�;*��^_�!��}���Z#c!�pTco��%D�^p�Y�������~������i�EN���r<��y6��G�1�����x��������7��~���2o�l����������=�qT^�}�{�#|���d[3na��
���	�7K{,��������{��n���w���f������8G0o��:���7wh�n��Y��{H�����)���A��ky>�$A\<�����A���r�
��}�vM �����5�.^�5��A+�����=��=�^��W�� T�
��z]9�["l�������T�V���R��FZ���������]&Om�I����PB-����bR-��E�*o�U���k|���d5�-�[��)����5W���f�]�__\�_=����5X5� �?T�:(�����('�
h}�P�<�3�����e��~�zMk��z�mU�l��
#6�Z���\o�������$b�/�|h;��������^=+�YU�8������G���0��1@�n7\a��r�f�5�����6V+�9����*R�D�V��%��65����7P�����7z���7f�6�Vb5`�3�!3:d���\P?
j]�'������������S�$���A`?��t34�B*��x���H�����#*�g�>1}ZC����x_�`��w�S4*����A��SU�3�x%��J4����E4�it�+0��`�����`P��T%���2^M0(MnD50�	�c:�Zv9�dl�!�Mc �����N���4���Z�Ko��k�/i�{��z�P�xH�������z�3I�O��#�x�Hc��1pu�E��2Y�{�n���/+=lQ)a�u��5�'l]���6���nYim�����b3�_��iX���\]T�*�>����T��z�l���+��_����j�e�:�~>��~{��
�_������4����>���Y"Y�y�����_e�~~���u9�W�
X��x�����c����ZuWu�1��k����:%��}����:{���A��S����m���-{���~�	��gtV�\.��d�zr5/��-F�^��G�����������Q�\��Xs-�\+0�T��������:� ��?f�r���;$�����M�lH��|)�����6�*o�~)��f%����&��P~�Vj�-m�jnV\�3r_>���NVo��7��a�����9��0���6W0����P�0��0��a�����07�!�L���<Ggjl-t��r#��c	���[;Y8 qk(6�M�c����(������F�nV���v%���&DI���7�k�f{�4�	`�����8D����v���t�wC��������H������o,�v&���-N�Nu��kp��%.���^� ��j�����h�p����-��)w�,��&`x���"jF%�V���� ��1�;��L�T����rY��,����(p�cQM�S��5��
n�Mm�������A���Q	�U|V$	\&�e���-`?�#������^?�#�w��������N{�B����s6���9�S}�w
�N��3"���;=x�u�V�g��i�i����N
�]B~p��1��>^^��W���w�\�S��y3�q,�O-�>�V�1%�JT������~��#Nh~����{�n�������nj��O�|����.����_L� ����s��F>�r2�s�J���������|=���&m�?L�@��Jmh��N�
F7O�����y�r�b�|���\��\����������s�@�u���-��s�c�'��}���\����M����s��
����X�H����������s�s
����rf�������M��+��#�s���b��i��D5��K�8h5�����i��[U�i�H����il�G�}�V�Ic<?�����z5��1z��#k\C��i��81hbP�A���r���t1_o�lK]���XT����7��!�/�~���T���5�j�)g�F\�[���.���|N������|4�����+S��A���)���~����)Fu�����'d����r�~Ju�J�����&Pm��
r�-y.��\���r�i����\�s�|�x.��s�'G���S��\����x.��\�x�)������z_���~��q	g��f5��UR���g�7��v��
�0I��iR���DYj�J}�~�e����U�e��m�����Yj����f��M��'���w/~�����������K��}�g��g�?�x�L���������������7E#�����K�dD���hP���������X/�
w<0~8�K�^�������Ww�!FI�������` ������s���&3'(!�ue����n{/��{J�[%R#���T�yo��a,�W�����j�^����Y3��1nP��c5 7�UHE��:}��C�E,/6j�$	�������?�/���M����{����^�{U[bh�6�[8�7�~
C���Yt���-�c6�?��o9[�����J,;t��DI�\A{����.��s�mM��Y�UC74���(��)
�C��#D';�x����9��T���U��9�mS�e�!q� I�CD������u7���6
��`�C<��P��F�G�xW7�����r}&|��%%�]��m������'RB�.g������93p��39i/�y�/�7Z�z����:������X~���p9+�y��	����<��������By��cs=����MF��G���
=�����1������L.�t>93�}�|�����C=�C��4����By��G�z�
��W��_5_(�W�h��{�����)��u��Q�zGz>��`�|�������m=�o��}���4���I���w��������<�wsf�B9+j�;����\[�7�soH�#��p�N��[�4�E�������_�y�����v�-��{S4���/��~��/`�.�Rhhaa^�~�T Ib��O�w����r-�Y���X�8���S.+��55$A
7���m\��PG��s�l-a�����V��a��Ep�$I��x@g�1?w0�����ZE�n��Z�"���6�E��F��$X���3J<��<����$��8���%o5�lA�
������(��$�(L�z�6�9�J�UD[W������w�}S^�mD�H����Y����:5���X���5z9���,�7#�4��NG�p��p^?��E�3�8��0�?5�a�0FYf#����a��������a������9��c�����_�cm��`��^�cX������	�5�5�p*���Mv	���U����m��K��
�CA��$������8!�	)�)n;���vz[B�R�WJB��)�C���<!�	)W7!�O)��H���<!�	)W7!�O	)�\Y>��������"�Uu���~.&HUg������ASM�vT�~n�4��M�`��!�����,EW�.I��wB?A�	&��#����D0x��^Y�K�*�X�����\��\8�^��B� �p�*4�M'am"���T,`r��A���� ��0b��.��7 ���������G:#��eN��D���3K�8������g:��(i?j�/F��`D���h��4m	 I�iqr�s"	QfO�4$m3��	��Q���A|��f�@����J�����rw|tY8a4��4������������'��	��HPqO��]��{J��<����M0������qh�$i��l0 ����8���ph�)������\c��\�!��V����q{H2X
����N?
���C����C����OE5'��l���-���88����@��rP�,Ft�i�*':}X����b^����e����9��":�xX�����aCs<,bh��%X��pw@��*�E���#��!�=�!�	�S�:����
�c4j�xyI��^���F#�^.X�tC���3�
M�R�����Z��(��n����*j����N�FM���Xx�u�fk~ad��)�b!,gkvn���rsI*�$A���D��!����15��'A�����)��j:�����S�[!�-����Jr�T����(:�
�����
���n���<�������o�6���6#����ni38��v_�jEI��N~��f��������)0�k���u���KS`����R�)��S`�M��4��������xO�b}�@rP4z4S��n8zB�<��M�!��)�a����!I��8�0(�a_4�]�S`����kW�M��4�)��j
����XpS`!M�7��@���V��)�a����=��)�jP	��	!I�����������L�����N�2��N��PpB��gN�O��Y�6��^������9��
������op"?����o������q�����<�l�Ig�fa��Y��J��.g�/*I�v���X����x���n�\��}��f���t��1�H����b2g��Y@�U���T��S�]_�RQ��3���\o�O����=� ��i-�O��Ff��`��D���]LIq�a�`���@W��a���M���q
b�n��NG�J*1�w9����O]��&��0���������������C����@��m6�r�0G�68��a���R�������������B�����II�H��<�zT�R�z$�u�������h|�~�;s�����t��^�uI]t�����^���h[�6��T���(K�f��G���hA����7�.�h4����-�^�($���� ���hF�TA�t����m���;�[%����A
��X/��$5+�Y���kn��k
��A�~�p�-[�t3����gw�J� ����>�l^:�����q����Wq�U@7S��l����T���?�:4����14Fth����{�����
���~U6~C�U*IB��T���t�}f����n�������u%������<�����Z�f�u��o!S�:��K�*�����m����\To�U����~u�K������o���������G�� {���G.)�z��bV�p�����_�/���e=!�L��j�"��E��!yg��A�4n����]�v+�IE����FCTbo���4�@�r�%s��������}[\�
�Q���z�{PRk�KR������I�F��Od	���� �����a�8Y �������.�4A�JUV��+8�@����r,�*�Z(;Y�o�GL�F>x�Y�qhK���/���&���4�\]�__d�����#~��z�l���+���_&�����8��������j�WC��������km.������2���U����/>�I��<*���=����s��9�\h�����w.��+��L�������m�������g?�D��~.��j�)��r5/���~6|�����y���������R�P5��Z��c�k{rz����o?\��=��{.�Z#H�WUE�����~J���=��{�P���5��9�~E�_������}���	��F������5�����h�]���
{w
A�S�����@���5+�.�yR��z��.��.4]]�����F��L�];��?�Mj'd>+NN�9����	e��Y�R{W����uJ��zs�j�5t�P����+�����|�"���6s���u�.1��!�ay���e6��_��O���?���j��\��O���r���-@�H�G�a`�}�*�!M�*��4y�~����X����R�X�Xo4����\�7��lz� ��g�eo����xDT�(����8E�����v���V�����.+��%�U�Y�)8�lW�l~-�%7����m��3��+a��/�����Q-\��Ry�����gj)x��T~%�m����lN���fd�����`��l*����s>{�*��0h��$����7_2��q)��'�����j�F�����b~�[vs�2��vb|[[q����N���d��S�Z�v��	�eBb=R�k�TUW����h�9�i=�(�&\�u�P&���
�e�V?S����0z��p�����yq��T��"#;�w>��u��&=�� #H���t��ygY��k��~����:B�N�c�����A�3��G5b�Z%�LY��z�M���"fV���Le�Q�l�5}a�������ft����1ntD`����M3>�������'H�G��t��;>M���5��
[.7bFx�h	o.=��D"����:�p���� �#�b�n������>,G����`�������f�bQ�R.����J�g�����������\�^��/iMv8'�w�&'�Em�����\:PzQ�7�������������Du����Z1�M"X1�M25al��R������1t9��XT���*S����e�`k�_�=]���s(H�`�~���?g�_�k". ����� ���rA4s��� j���\���fW���/��F�z[X6�����k[���2�}7��8�9yh�/�O�����'mN��Gn�<�V�Gn�<��B&�\?yh��=yh'��8y������-9C�#R���G�~�(4>W����J�<r����'�\3y���#��#7L���l���me���n����XXw�`�c����"��-��9,���[�����$Y*��$8�MP���h��������z�
�4kA6�mwA��)dWPA���kd
@�4��!���
�j�30�~���m0|�Gc�R���@��"d�IumJ� �s�>S\C,�47��\B��H�X�� b}#}b�?4�����|G��!��%�7��6��Qb[�^}�Q����~���6t�;��W[m���>�����B�c���e�q��[Wcv�i�<f���U[����A-�����%��IR$����k��,@���������w��R/�gI�$������-{;���v�@��R�'pR]����\�F�8�@���P.������-���x�����wi�2�
�
�
��� EZ�����Z�j�P�j��P�z�lP@
=T�
����y�P���*��|E1
���E��*�5b'���ph,���x�����^��<
���y�U����Q��b�F��������_#!oC7�F���6u�!u32���5�.,E��a	���6�p��g]���c%�	X�uG\"�T��K���j�KR��BwC���X�lcj�����(�$!z�������>LK�8d�xe�p����v%�����^O�wT�;�$�$)�T|���>L��9(g$�3������;��}9��Pf�����`�9����0�N���qP!pP!s�hjp������F@�|M�<�iI3���R�J`�(��B6DT��T�����;�����U9.�l$��W_�z$p��C�wz/C��w����=�S)�7h�K$�^���B�A���x������c�[�B�G
��%l`N:@��{YTu�c�o��Q�6?d�A���SCf�����O-��� �q�U\���$5�N\��4��L�������%��@"v�kZ@�s[�[�i���@g�\��\b�\�6�`�\��\���������9R�`��c�&��-�;Z"B��V�2�����G����D���A�[d�\����Y�]"�����if�:gVH�Z"w�D��5��e�YiB)����X�1���Zhg��4�����g�1�X���|�Dr���?����A.��#�eM�m�������:���X#��M��/,f%�B�^���(���!�s��}u�Z��V���'3nn�&vL���#��2�f���N�����wy~}qu~y���C���������2>�A����DA+���d��\�G��7��V(8����55�������w�����V^�������pB{ 8���D������&�����e�wyq���WO(r����I8��k����t�P�X�j�Hf�����4��9���X���B���
d�S4�S~~O�l�w��j9��������j���K�K�
x�b}�5%$I�'����`�p�������zFM8�6�1U����������VS��������x����lo_.��+����B��
?���"k�&]�G�|�#E���Jo�:���)'�)s��p�y�y��z3S4�M���qA�������3����m�5S�l�d� ����������������B���f
�3i$�HR�U�m_g�3E�9e~��p�9�2�b,*�cv�\M��z��{��r�p\5VC�Li���ie+�x��L�vGA�Me���!���*�� ��-B�i�u����uas��}�Uw^�
d��q��T>)����a^��
����4��6��6�0R��
2�wv�J��a����;�%���S�Y�K�J�!Fg���oL4CM��}V���Fc����K�iGc���#Y|�?��$�1Y�!�"�����lG��((��Icn��C�Oc���:��1���+��R�jiL\�wt��h	�������6��4�-����~�s��U�����f��vM�m7����\Zv����ZOwg�2Uwn�~0K��y@�t���7���\��|�U�!� ��&o�o>`[{�-T�iv�,��AT���~��l�A��Xf� �I,�1V�%z���@�0�h��hG�.&����a�T���	��k*)Z�����6��6��*j��H�\o��5P��ha���5&���Uth��8k"�$�0�N�;	tz���c���Oc?�}����&�_����y�*���#����������w�V�������o������E5�����XT�|���������j{v��X=����J
��@E��3|�i�S�U��!��N��y�'���<G�;����y����>�y�������3�(W��p�k�v���7'��it|�+�^� ����Ns�x��WI�������^�"�����9^
���S�|�� ��Oy�����0���.�@�'t��w,x5e|5�}��N*�b�$:\m�sz`�p@�$�w�pfzN�J��=B�k\����{�X�"q��`�j-y(�,�� i�c�3;�sj����@�sP��U+7�
������ \��
��q�{�_��]�&)�m`�a�����v�I��.�pM�_����T���8"F���H ����S-j9������|���=����S+!���S�'qM�2�z�����\���������b���oQ5IQ����Z5-��0��]$���A�:u�F	�Y�K>Vk���+QZ{\�_�Z������=���z2�|1������|��5�M����/��
�����yTn�����B�����E�t�=Z��H�%���/����%��

����UT��~�F�K�<bhlN�Ij�����X�����!CF2��V7�$~�q��(���d�o��GIG0�������Mf�������O����G�p��_���M�� �
�]��b�i%p�fE��(�Q�[��!p��HRHO;�]�Tz���weP�P,���jZ�65�~v�g0�7��&��6T�0��y�6
�DG�he��j��{W��cJ+f`�b%�3$u"�4��F~�l{�{��������:m?^��3
��1�4*��-'�#b�� �����a��SD��~���)�������f����qJ=Al/7������[��`, $�,F��U�������?��o�6,6��.6����'����������D����>B�W��'L8o(�]]���������C�QA�jpi�1}���� CLg���C�����(���h��G�����6�]��Y�NN^5X�#d
�}�-k��%z���H��]����= ��;hX�_�����������v@���6"�hq������0�+����Z�l�������a$���_�X(;B9N���$��%g�5(���H�rh���=�A9NN^jPU�a(��@�r��z�C\JoSNG������W]��n��i��)|0
�����tu����l`�	��-t������&6�9�
G0�<7H�
���u[�h�-�Ixr�>����Pr|�3>i3{���[��h�'-E��I�;��k�m���8�w�^�J�����3��KQ	za!�����kbvM�`�z;��CpQ���O�f,����x��T'�S�o������m��*!�����(������T�V�MW�:c��7b6�l����
���&<�~cH�����1���;�y���M��
�
iN����f�\b��c��u������"�R�4��98n�@M��HA���4{��P����P�S�I!�P��P�B�@��Y(R�H����C���;��`�$�#������;I�m���/]d&6l�q}3a��f����e��������U�^����l�����?���7W��+�����n��_�u�}��6w�#��Q�lS~j���-f�c6"&�|����nXTmx���~�]5��>`�6��U��W�?�L�����z�g�c�6�nw8V���������*?O6������U�cA�C��p�f������+��K�����4�+�-I��I�����&w�o�C������I���:�����z�	eV�Z_/�����;1t�H��oJ
��#BX�����A8�����������cc{�V�'O�@�>��=��))[�HI`�A�C0a|��"� �"f���}����+���]����G�7%��	!��{U7��"gW���S���r�g�X5����|{�I�V�����A^����)$XbS���lzPP���u�����)WE�6>����P:C�Z������$4_g#C����!������gH�7<�h�������V�����A���	�i*��,Mi*�~*����j4+�F}���\��$	UI��##v�V�#�-����P��k�#�X�����\�4%=,��(��������Y����tDS^��_?��r��D��0������{$0���[1�~`�m�p��@�=F��� ��<-|r���,i!����-^1FxA_2��a�����H�Vu���X��H4hFX�X5�hA����HKhF��X5HiA����HbhF8�X5�iA����Hy�����.#���"����2��q��Y�]?���K��;��x��C���-/^`�8$�����Sv?Ym���|�����^=0_~�V�b�/Gu0����e��_g��g����3�=��9�~�����_8��{H�'�:1N�����)G�M��������}ul�E�X��/�<H�
6F�^��M������`4���p[d��"�{@��C��7&�}!L���kv^��g�_4"�h8��}���=���7]���`�'T��(�������r��NP��c�Gs��v��$����fW$�;�sv��+��#����8;� �C���`��/�����NymX�=�>���y�4�yFg��m�fX���mN�������k�Ga��9��b G��\~;$�q:�sk��dI
]�b�NFcW��4�3��$E@��j���\vY�X��)���T$[M���L�Ih��pd2g�69�$R	��e��z~[��s3Y���L%�"���1�~������$q�6N�����b����/���o�����g�<�N�7��7ot*���=Y�~k$����M@�%��/y���p @�\��dY�~���j�� NR�cDb%5|F������K���{3i�S���;8x���`���%��v����d-1�^9	��k��=h�)Gr�'^�I�Z9������G}<Fg�69*'��KA��3��H�F	�R*�6	y=Y%4�Z����{�MI�PU����P�q�o-Vv-:�<���� ��I�_����\{X��#���'���H�/*���YgI���:�=�iRa�R��8�M�V����+���&�,����b��B/��������pG9~2��'�$��{>9q2�^�L/����n�&�y���{��O������/^T�������8=��<1s��s-��%����?1��~��v��Ni�u�)�Q���:��r�)?e�ww���e���'Ix����}���B��������	8�,��Z�P8u�<m���Y�L��:�� ��qu]?M�����T=�+Z���8�j�r�j�S����h8v0B���7�A�{ ���}#9�J�=�t�79��px��Ts\O���m����l�>b;�5��4;����`�q:�&_���QN������y�p��=^�On��p��b��c�����342�����h}R��������/*�hbt�B�{@���Q�|�_����!��S��q��b��9p3����zj=�q�b=��y����
����Sc����	��*��4�JIJ�U"U(m�RJe��WM���?��MVO� AM0v)�M�����D��^E�O�#�O���O����O�#����8�s����^m���t�Z����$���iv��y������Ma~�N\~�N8�e���'R�@�2
�]K-������~.�~=�e�z=K@8���c����8 �\	��H��B�'OprQ�,6�$��P&2O���VO�����5�y�!���O�z���!+��#�)����l�H���hN�F�5Ga���~Es����8l��GQ�s�������:�.L��8b�X���DD��s���/|X�8C�Bi�v�7�\M{P\������mc�m�t
������e�4��;�Z�J�y�yz����Z�K'�8���y�#@x��f��e%q�Y�Q���[;��!�OdK��e�$��4v���O����	�2?�������)
R~��Y���M������,�$�����W�A��[�9T�=����f��r/����r���;S���{f/����d4E'ue��l��h���nNM���/8Oz������_�9�����F��i����t�O-��������%2c�X*�i�e��X���ZR�'���R8%��!HBA��+�
K�p�V�0�&�CW:������;���U��6`�;P~8+�|G�<+�<�N����cEY�
��bKc)��r�Y�q��'+:�P��V�V�j�����#
��H�7���+V,"���S��`Vt��������;���U��6`�Kz�����R`2�tk�d��s
Gs���r�����B)8����K�����/^|��	������S6Yg��~1������d]]���T���&����(��lV]_�
��m����)W�rV��O��f��m�}��9���������������J�F����M�QT?�d�!c��
>���<op��
2~I���q�^���
����UIv��������?�������P�]u�M��/�Y�����uO�AEF����bj�
}$�q�Zf�0
���]qs��P��Ut)�}�$����<��kC�X���1�� �f�
�T�
v|D��3��<�l��}���D����� ��p��T��n��v���6�/�����1�b�";�?#|2��}�1��N���$�2�E���	��9cQ��z�N��
R�Yc7��z=�����3�r������7� ^�C/�j��,�$~�I}����X�������_���jCh�t�_5`�DI�~�
��t�k�����#5�����[r��H�u�m�~���_5:3��`�_Gj���B�Uc9#��`��RY]���}�"�|���n��o������]�*o�~)�A>#X:�������8�����0��ih����C�kpoB����i���$.����E��*
?.������&0�KF����������k#��oJn�d� �C�,C���;��n����M�]��S@�tNIOL"#u�K8��+k�}��|� Ie"�".�4QQ��9I��v�����4lDf��P�:���|��<�k� %Y���:��0�UdE_�k����W�mY�+��^=Vte�v���z�X������(�UcEO�k����)��FRIOw$eo�Y��|�%[�-_,���d�(�����\��w�u6_f������~�_�����n�����\���;4Q������m�O�������y��_��)a����uFeC��-~���K��#�l$��sn���Z�;��H!��
mV����_���n7Y5-d,<^�.H/���S<c�V���j���������]c���1�'@6R��F��f7����K�?��=�#4nt��o-IK<I�F�Ji��J9I�f��Q����^M)�Z�J��v���8�v��k]y�qqk#�
9����a����	H��W`t���.�Z@���m��,K3�$w�	��3��$I{���%�����f� �o�������:��]<�N��'�y9�xp]�#�����|`=	A\�s��u�An�$�:�u�u����xp�\O�s�t����������\�c���u����g��Q�h2k{��gj	����i�a�D{L��.�p��3l��c=�EK����}�I��#�cU���	`E��O�_�J�;��@�=�Y}�c�K'�S�KH����^4�1:�m9��q��c�mt��
��M�������r����I���,?�?���l��T~���l|�������E���������6��i����P�u���s�����0<�����rs���\n�����%k/G!�I��%��^.��C�+��
���w���/2K?+��+�r��H�J&��I�dR�\�PSj�)If.��r�����J���y�>���o.+^ 7��f�jx��d�����a����8R�L9����Wp��H3%�������V&L\��J��E��jhu���S5�g����1Y	�����kT�3�t?���%�����d�6����M�#�W�iR��fKtn��K���f�u<N4�-��I������'
t�/�����������g����u��(�h�zr=���IA��R����aK����B��}���Y��v��{�E�g�8�������6����������E�;�� �i����Z�/w�b���$y������W%p������������!S��0f��0�be���P���x�.I�����k���P)��|����Sq�������u�q+�6����(�uGs��]���@j_w|N�h�����$e������rd�����uG[[3aa�N.�;����7��x�i�m��O�{�����$
>�`���M��]w�i
���{3��^��a8;������-�����ac���/4��:�z����	*:Wx������*:v�N5g��\�qzU?,������s;p?�D���7�j����8�'#��g��g��_6As�N�p�����S���V3��#n��\rOl�ub������ ��2m�7����������P]X?mk��
��B�;����9�Dl'��%s�XH�9������E��$�9L {k������)��(Q�K%N�,W������*��2q�x#�]�@���q�O���r�3�����K��O-�G!>x�QGu#)h�R���	a6���	�&�.��u#�s�W�9�59�vN���0�6Y����Vz�~�,[v��?�eY:/o���Y�saY��e9�-�2�sY�4���,f�����~��=��':��T���e��a�����w`vT7��vh*u_����$yY�&�6��u#�sk(W�e�:9+k��yN]�s����e�
wAm`�7hYD�Y�-[}����,�:��,���,��\��e�����2���e1�������d���?������],�
�����E������]�[����l$��r`61��G�I���J}|�e�����^����\h�e�,�n)j+�A����%�@�d5�����X�w����B�C�a�8�e}��9��\!����h���ka��E��Q}X4�zY����J���X��>,�������7�u�9r�1�9��
������op"??�<����32��DH�N�u��c|.�����/�Q?f7���F|1>GG����`�����l�1���S��(1Ot>%��j���{��b������)>��n����X����x��#T������A�B&d\�-;�0�����j�MP����r��$�a���*E�Z�oUO��-j��!0�.�*O��9�v�(yi��u��M?���dv���a�i���-y�`�t�?��MLC:1��'��41�u41
}&&���m�U&~c�\�oU���InQ���+�v�*O��9&��v�Y��	r�������Q��	~����,tb�����'���^�'��41��hb�LL���$�L����L���'&�E�'&�@�}�<���01�l71���	g����J��� �������:R�:R�:R�:R�:R��9R��9R��9R��9R��9R��1����?wu�6�'��+;��:����'
���]�������^V�-��Lq-��#��>h"��!���J�)�L������v���+�����4�8��!!�����
�xxa
�m������)c��NS(�&�b�������-��y��M��gpC{K�~aaM������;$�i6�3A���F�h
3���H��wf`����x���asb��6��������Wg��L!��i{����[��i1����!?�pZ)�]rf�LC��J�n/�oJ�R��.E�K��1�����s
OA�"������U�s�u�.���;���w)���R��'���~�jEO��Z�����Z�S���B��8.��)n^����7��������Z�S(�V��|�Z��)�^G��B�nV�r���������_��W{�*��K�N�u���B�ur'O�n|ft�1��/��=�y{�7[�������6��b�����m�uE���Y=�xo1�dE`���k�q���o��
�����vL8���(s�MP���
��H�����^��[�x[a��\q���Vp�W"V
����H�)�Q�X������l1���d����rU.g�:�,?e������n���D�6_�@M��|��/g�L��?��X���W��G����?d
�����C�=���Q �A9Yo_` 
�<�3�����&�A���^����1tg���*I�R
��y�������N��]�8��r��miUZ9���./�?\���f�z#[�4�g�� � ���M���ww��M��(oJ6�i�*����n���Z�(���
���w���YD�u��p��j7��5Q���_�[v�:����	%R��pc�et>YT��Y0'��RP5����U4T
�$r����x��a�	�H��?��hU�|$���*.����|K�����66���tr�ym���m������P��X0���6���Y����
��	�j��/�����(��/�F������/�����(��/��������/�>���(��/�������/#�~����E��2_g����-[��6���V���/e5Dge�)���}�[��������F�6
�4��yhu��M��t�2M���t����i]"i��
�-I�+�HH�W�M�N�c��u�Q����� V��Cbba����5�^ilg�����a*?��a���	�QP����w!u
�x�u�MX��x����Q!�A��:�sW;AZ�����NE�����W9�S�q*��^�T�O���(�zuS�=����z��ME�������W5�so@��
�����N��w�������+��t}Y��E�%������Ca�e�y���B��l�.��-��:��K�B���S���������V5��W��]f�����Va�E�O���J��A�{���%LG��@��*.����;���|q=�-ka�f�X�U�iL���h��U������_�s%B3�7j���P�I;�jV91����`7(���D������#�7�>w��<
����I���/����G�=t-*_40&%�FN�Z��Z$��I���A�D<-��m'k[0�q~��l�X����x� �����uW��m��m���W#V��B�'~�>b���A��Z�y�M��=�/���N���.�X���.
���b����(������E�7�]l�T�X1�$�>��;�t�I��Zh��i��Ni�M���[gcN�|���e������;-{�p�����K��[�Xn�fe���U�^0j���S��+<����n'�(�[��W�2�|����M�(�r�)?W�?��]����/�z��6���E��?��������<��8Zh�$g g�Y\-�;'�������s�G�
�T�����B��9���������3����iU���c�:H:!�7:A�EaFd'X�?���u6"���9�0�!k�SK���;�(I���JB����Yp��p�>��=0;&qP�l��cvD��<�\���{r�&G�fk�'�^��AMx5�{�i�
���Y!�
�{�C>\tV�w�W������\"<�h�������%z��J5����5Wih�
�
�
�>("�A��B��B����
K8U%�$IfTm��z������'�g��7B��,Z9��l�'�������Rm��
y�����A�S.�)���k�,��N��O��OEh?���2���Ku��:��S��So��"I���W������'���{
5�_>b�@���	��h,X�A��w���
��J�!�+�b�����@��
l�M|�#��
Av�W�%���������{!!�����HMs�!<��mH���m���7��s���]�����Ot�_�3�#n��TKq��}@Gt���#�'�1V�a��L&����L����7
J�J9����?�G�	$�N5NH4[������iC�R��_�v���%�����7�Z���^�%�s���5���v�yM'zmg�Q�'i�!��j��f�~
7����(����8c{=�5��l����{zz��c8�r��Q}����] cS��[��\o�pGwP�E�z\>���X�fQ���]GX7\D����n�Wj��mI'�sO��w����.'����'��-�ez�����gABCij�(;���#��MGX���}E@>S�If�M>>B9X���d��h�B��
��~'���9�OM�F�
-����-�2�2�3�9�����.�]�k�u�v{J��NYp�d!*����=R��FN��V�:���8SM�3�U<�ry���q���,����4�r�8��q���,��Y�b�,�V��H��8������\��g`L{z��M��q�+*m1j���4� �L9�� �*�~T�8����Y�M:����s�Y�j��=��t��P����A�f��f�ggg�o�!����7��!���C�dC#Zhx*Z�w��E���n:�@]b�B�K�Pm�\���+��*�+�����Z�D��kR��j���F]������V��T�"�+����Ms���6���|tE�B..@,x��t9��9Rsy������#5��TE�V�l�[EY�@�p���-j���#54!�9�3�H2:��F��|��bv�W���j�����
�� ��M��=����D4gh�i���]��K�\��W�Z���]������%l���7X�\}��Q64D��H��<�4!�fL�0m�?�t��.��k?U�E��}��;t��p��C7+W_��
)3w�n��G����h����e��T��x=�`��n1�G[�y��R6H����-$6hk�����wj�0���n�n9�n���V}<��jw6T�oo*�2|E���k�0��U�.v���9�8�
���NC���U6`�n;}<@��~#�b�=*��i;�?�Wkm,��m�l/��*��#GV�������~��(����8V���Oo��<9�9�u��ME�)����OA����o�@��wL����9����xT)l!�Yf���y$�H@�\#']��j�c��C��2H�����<#�3q�;H<��E�$��i�Z{o�~X���$����Q
�����5��<5���%���q8<a~,�����_c�)���x���Ddc8�#�����HSRA�����;
T��
6%�l7%A�&I`���)����u��O��$�ksJ*(;��1���j��1�5%��MI����U����L���{sJb<����W�d��C��^����P�s	uG���N\8p�u(��&���5��v�?�� �m�}���G��4���;����a���7��yu��D�4��$�AE�E���vwH�>4�~��Y'M��.�64;���A�t\Ts^������O�����0;��S�}Q�����Q�"�H%���R�o��[���h��9JbAI4k����'�F]�#qHIR�0�:1��/����9_f�m�Q_g���{�E?x�Z����#�Iz�T��OJ.���S���d��"6��9-������;�����	~���)��8%�2��I�Lr������=r�b��J����]w�lUNp����>M6�l]n���Uv[�����r�-���������o�.��/����\^������M��rS�$�k��2Y��LV����W�o��V�X��������Z�	Z
���������U����/>`S�3�����.���|N�i
�l%
E����r�=r�(7���N���R7��9�~��*��Ur{��
�������cm�S���~I[|�o��f5��������76Pa'������9r��r|�����uu��=k:������n�co���I��?aV>&��rY�&�r�.W�r����z�G�������������\�Bw�x9�\�^�����.t��9$�/2���9�F���]��g�j��[.~�s![��a��m%Z�#����&+�P�����yK���C&�B�Ye��c���~V���Us������qV5�h�Us��jh��jn�U��Y57�����j��U
�yV���j8���Y���uV�����2�����kf�\?���Y5����~V���j��Us���?���fU�����������4�[�J�������e��+t���u���GwP����#��?W�I���3���7�"������d=u��u� �Hx*��<":�.�����NP�����^9���'�3�=�X�
�
cRw�M�Mptt�!|k���3[�^��O�M4�T<���5��D;�D���oE�nB���(p�A�O�D����'�-'3��N�c���������G	�#J�y7����Dtox��K���e�=��N��7�w����C_�q
�]�8b�
�������]�6�r=�M]��5��.������������C
��J�O�5�W�|�:��6�>G��kuM�w���=�|��o?n��iA?uq��Nu8����(w���6���c�GB�,���6E�`�(O��"�pr���9�}lf����wA�>�gx$��:�r`��HF��>Q��+'���
���p��}��}���
7��	�������������'�����!��!�}C�}C���2�
��7ts��� ����q#:����D�I����7b�7��o��k�o$s���}#7��	��8���n���}{�4������S"�Z,�� S#�����U#������9{d�_5�e{��~�y)\5AkU����=tD���Q\�������?OE�v�����9�R��I�5��X`��I�5��{�����{%z���{\x}��5{*b��N�G!�L������{�I���"?��r�vD��~����z�L/z�?�Q�j����J�w���S<y��0�����*|P�[9VT��{�V]�^F��FI�H/��H���$f�� ����O5��
��'�
i�h��zuZ�[�C]�C#t�@�{5qg��m��E�e�A�m;i��6���(�hZ���a|�\����Dc.���#�-�\7�ryp��������F��sp�����.SGp��3����������8��hZ��4���<�s4�'�����+�W.��l�����2�������;C[\�#��e|4hpAN�vw����b��\zF�~�.���1����i���K}�K4�R�+W.�\�n����+��\�i�nt�2>:��+��#�f�\qW.=#H�[
.���=�V�2�6�
i�,�	��&��c�*�����\��r���0�\�{�*�+�����e9(��#�f�"���g�w���)�M�q��\����Dc.��r��U�V�B^�
��U�V.��qpAW.�����rY:�k&��w��3������[�Z��i5.S��u�g�z��}�����G�`_4��@�C#�V�����3��,[��<O$�pmD�X��|�����'�o���bRM��E��7���q�@��?���/��/��/������sB1~��'t��t��h����zlQN����7hC�Xp.�JdP�����TY�1��1P�c W�@���q�^�
5v�����6�(����H#������wg
~D�_�5�^���0��\)���������^�����7c���rdB7���p~�#^f�ww��3^���`� MJ'�5���s!��ch$CY����Z\`��E-P����>e�y�e�V)7����d)5(q�,hS����$j��:�*���b$bH�[����9��;�8�xx�a���<���sz=�"��z=+q�g�v��)�%��<G�]�}g){�Y��g�bfr>�L�"��B��B�0�����/w�S�sX���k�
^p
��\���X3�uOD����:����(��Z�t�N�)�#"��s)�NA�73]s{Fo���Z������t�}�3����m�5���y	�E3��hF���Y��t�����G�g:�����
KZ�5e3��[��t���E�2dn�xLf��&S��0nj1�5����6��^���N�X�4��3z��L����>��7y�t������L�4Q>rV�"g�B��B�����Rf:�V
��%7&do����i��aM��L���X3�u�rD����:����d-��Z�t���i��{����=� �����=����t-v����g:��Q����x���������L7d;f�����(m��F�Gi�<�pp����aI����`��tk���1��HR���M��A�\�d
��M-f�f����l����\�TW���L�����-g��m�5���N�fn$�{��6����f���?������n$��1" ���7����D-) 58mX�"�);��4�k���j��(��� s�@�cb@H7��G����T��K��.1��~G��?g��}�nB��I������6<���'����xCi|���|
�R������J_q���W�'NV�����K���/��hi<���|
.\��W���K_q��8}i�
S�9�����f�m�v�������v�(`jQ'�����hrM�YrM����P�O�(�XrM�����{��rSrM����)%��vU�)9�&PY�����^;����@�����jJ�t�@UnJ���BVr5�����*7%��.�<��'�c��J�i�I�N�vv>�do&�r��:�|��w���b1�.���z�*����r����z;%��
���_��o'�(�[�jb�R������P�d��(g����h����>�_~���$}6����(=Y~�8�����~�+6�p�HENF�x������tb��~��G���Y�����gE>"wQP����F'��y~�|?������X>���8�l=�����'vJe�o��&��V�U5�Zy���p�'`j�`tb�0��n��J��k8�������o���s�������������DS��_
`'u{��T��]c�U�����,3�2L��&lq�-��3;��w��w����=k4�N�S���v�#W'���u���]�����:iO��S�~
;�����@3�:�1��S��Qb���H��I9;jxf2�$%_����~'h"&j/�b�!`���	l>d4�~�~�]��_�����[��
cxc+0�6�;�{l��?��O����vY���d��o0�x�����^=�xcU.&�2>���UF�D�5�k�>F��5�������_1�]����Q�.��}G�t���A$�K�r����F��C-^�6#j�7�In���EN�{���p�n]
^�,��te-K4��S��v��7�	�����1mxWX0u�M ������B]��s9nn�S�MB]��:����Z9�\+!�
[Rgu�����;����:'�!�O�������
&z���5)���Gvd���~�k1��c�&-6V��KxMrMM�x5	�85c��5)45)"������ ��PS�a���:>��4�&#MMFrMzC������A��O��
P�u|
��kh������W�q��8��l��39�b�;��.��z�����}W��J:u�j��y:�h2�^�zE��oN�H6�0ftEx�n&M�w�
2�]C|�@!��"�T�f5�����d�5*�����8���+3��V�Vv��m��I�5�������\�q��?Pn(���k�~5�6����k-����L,�@�,[�<x���V
<������d�+�ta���Q_t�W3�'�s���W�I�IKN./9�v��MK�pC�������AKN[��������%G��k3,��&b���&��I%W�(K��IP�O��[�ry��uK]./u�v��MK�pC��	���:��AK�[��T$6t��c.u�:�.�����Gj0���4M�:�Q�ul�:[��u��q�I���VD��Vu�f�(7����ujn�G�����&6t���Yd�����a�0|�Q����dwl��S\�#�:�[�l�� x����S��B^�
�ZW��:��q�A�:�������3�f�u��:�p����;����L�B`u�Z�;}�e�z��~�9E���<
UO������G�����U�F�#��\�On3��v�=�mGAab��z����������vf@�z��_�h�������6��kEDi_,����"C��A�%�+T��I�
�?�
�r�/��m�$��US��*�p�Q���&t�I;��$�FX�*vF(N4nMF>�e�s�����u�6��m���r�T�������-����(uF'�6�XhB���c�H2n�6�[��r�����,vD�q����Mo��io��M����6��4���3�H24��[�x��5Y-�5�q���V]oy�0g���������vq�f��:��qn�4����1m$�v��mw���Vi1����B^o�z[��m�]ou�c�.��0�����r���F���?n[��M���wLI��4�qk55�M4nM�N���s����U�[�D��Y�@��u��0���N�r7��YW���d��[�M���wLI��K�q����N���hE���wP�`qp��;B�o��('�N�>���%���a����wa�����e�85ZV��#��|���w_�����x�$u�U�I�[�FQ�];���]Engf
0�N]��^����O���} |�5��L���[��V��f����O#���5�qM{\]M
3���]l�}����,��?#��9��]�ms����v6��g�a��.�'a����y3��O������&��N�hg�0n��]l�}�����Z/��
�T�oX�[�:<��3�v9�m�'�m��oy�;&����Q.���m��^����&�|��YM������z�G) v�/{�q,����$�N� �n�-�����-�#uJ��d-��qF�E�b����I��u���N�L+f<�����I:���x5�x�����&y� ����z|>Y���tz�-5��"��r�/������OB��BN5� ����g�d������Sv��{x�����/���8��s��-�8�����A���8�E�������z����1S�'�q^���__������WQ�zsf�����Cy�N���8�,s��M<�3��q��3Sv���Q���������u`:��6>�b�I����v���?@�|�r�t��B3���bYn��Zn4�Z�m�Z�m��Z�����V��Y�C�JG'�d��F}'��|������������<��!
=���u}���/�
�E������9g7�:�~U��2+o�7�����e�G.�z�/T�pq�'��_�/���e=#�L�j��p|��'��2t�4��m���jv��
�>L���i�.�N
�e��|p+=��z7V��m�|�Y���mS�8Bb��/���E�Wmh;$�6q
Oq��G���p��p����������Az�����-I$�MW���S�����I�����$�.�'���'��3��������J^���'��QR���q 4�id
��u���^e�ey�]4Qv1����!wQ��L/��jrQ2T����:&(�<a��m��������7f�8BJBx����p3�I��&1	����q}~�2���M�'|w����&D���'���e�X�M��Oh�F��o��\d7�E����	�P�����z���e�����f2]T"F�z_�|Y�W_s�}��OV�92tf����|���@��7W��$����\^���������������f�������/?^|�F�A?��\r@.���1_��J�������2Yl�F���O��<�?Q��������?�����>�z��������Y>�6|�������\
�
�m�\hym��\�EadX������o���;Y~�n�V�D?��M����^�L�����lv�\WN������N���5���*��\��M5����n+Uw��P�-JbUL�eU�Kw�����_��_�9��y�lu
S�
�u���d��Z�$i��T��dN��]��s�VKnQ������������j��d�Z�������)��g�lNhM�tj0����������X1�J]%��Y[��U����
R� �wN-�#n�!{��:{W	;������sZ��i��8-g�V)��y-�5����k�RA$����[���fz����}E��~CUb�M�V��.}��������E�����r,H���/���\�}I7��N����m��2�3����C���%�Q�C���r����x�c�>����p?��
�I��f��B�������"s<�5�dd-3�J!�p�uh/r�n��N�{�d;Vd���Zp������<�
����.>
���e������_�}�Feo����lv����l.�������W5��Hs��orn�wt���{�ru��g��u�B������}�J�qV�!m��gK$I�b���1�����&�j�y������]��>����M��Z�Q+�����X�T<��������T�"���'�EC[��]�����I��r��<�Vc�L'�!�4�v,o����Y�zi��vz8�r�����mt?���xf��]��V�m�#��8�:����/��F8b�>����&}��E��,Gld�'�P�G��BAy�
u��V�4���h�'����nG?1M#DS<�����&&z�LT�sK�9����wwK�J�vEX�LB+�?��������
Ge���t��T��!��C�(�c�
�N�UM��/���*�2��
������{R�C�>�6*�m�A�E��CL���le�=�����,��
��Y��Wm���C��$�O�o�j9/o���mz�Bo��G��&?v?a��^~�����`\���_���T?+MPW�~�B'�k�	���U�/���W��q|WnS_��/oP�n�n�Uo(I��UQ�5��p�z�Dq����C��N���&�WkvI�b��#�{D�����zo��_^|xs��Q�H��>j�����#��

52��@P�/�^�D�N�/�
��S*���NTy�6T�	D�%�@�T��f������&
�WUr�q���
�p�UCp� �3N��;�L�Ls#�A��IAg]=#�������N�Y
����U����b��o������xE�[��``9&���c�/
��T8'*�U�����yA�����
uZ�
�{)���V
�jD]%���J8�
\��Vz*!�6xT�J��`2z.�7J]�]��l��	�����z;�}�����g��1�&�l=/�!~L.e���?^�_�����k2�������������I]h$�
����"���R�Bk����7#��gO����:
6��ho�c�UO�Xu�pQ8������Hid/�#���d���Vv�Myr�j[��@���!I�(��*&uI>s�\��s
���|v@�5��YN��)kS�g ���2H���v�:�ig"p��1����sc��"_O=�x�"�f�]C@u�@u�:���^ZW!������M�a���Z�3�D���ni����e�N
D�&Il\�,n�L>�)�r�GJs��0�l�kG��t�H����B�����:[���V9d��!q��"��u4R�.�3R�n���H��#�|��\���R4��f�m
�H��)�0R��#�h�)j�CFJa)�nF���qF���$85-FJa)�r��(7�wI�(�q����+F��NL��4��x�)��A�"�|��&u�d�0D	���yr_���@������D;�g'{$g����HSP��[�b_4����z�O��Y?�r�g|u5�����eHsP�W�%�hDs!��F��Fq��u�m#l����1��Xcl����o����[d&��>��r��!M��}Ba���b3�_�����_
��b������n�tv�]nz<&�B�G��z+g%�|E�b�e���c�	l�h^�����1l|�<��������T�=���G���2d9T�������G����:0��b��Z�Uw.���+H�����8���}~SB%1E��,��]��i���|n�-�%�a��~��T~��/���c��lN�=��Q��'�0��m��[)0W�+������
0G	�
��
�A<lq�v;j�4}%�_����L�n������]����n�\�����������6j��������8|L���u	�r3� ������7���\�b_����1 #�����t"W`����t�r	[���r�Hm�j�aEGtl����j��G������t�=V7�aWn������}�c����>�Y��d�<6@I�-&�V�x
��=O2�F'r�X�6�8��~6� >�ZC;|�E����5���%�m<#�@�xI)C'���&\�;;Q�*�&��l�^�(Rvhm��i�a��lW[�[���mA��/U�T����n[�;�x^5�2[W��7D�M��y�W/F�b������z��+7�����\��
����#�Q.[VC%��jH�UY������Ef7.�c��0Mf���Y��Q'��,[�E����[
�����@,�z��n�k�_���
:L� �6�����/6��FQ4��&��o��n�;f��l'�^*)y��D��$?�������Um<@�����i��m�)�<I:�#PwJg�0
Ey��s�"T�w���I�c-/���W�@5����.N��pl�$���e(3����P�c������!�1����{�e�N�����
��S
n��;9�����|�M�>�	q��C���y@>OF�E��|��
Mp>@�	q� �����Ih"�Dp�a�����`���,�M<�'�i�����-���T������v�@'U W��jr["a�*�!a�We��4�c����E��E&��7wp ��������u��wT��S��M�H9�������P���%A����G���I���jT��
bH�q9��SH���s?�� ����ZW��L?���Hr$R�W
V���g@xprCV�$���������O��<���8h��JFW
��x��87x�p$����;F+G�aAJ��-}��,f?f�]���,DW���O�P�Tb����M�>�\>�G���QL1�$Y�����.Ut7�# ����N����)j�gPNIG�����tI���^r��\�hC��dM� ����F��&�\���aN5�4�"�4Y����!r�
��0~�i���p�X��L�	�bA}�G,z�w�w����g���g���l���<���9��WX�nJ��s��d������{�
y�D��DG�pWS%D��3/Bu��`w�g�m��	��H*�j,% _�}$�
��1OX_�M�l���)���$^�j��e����
�@�=L�b�F~��d�%��(��A�Zxc����xE�=��=��jBt�	A�Dw�Ur���#�X
�L`;��F�/�$�c��9u;@5��-�s��*�.`���0�Z��q���!���f��4Mm�!���
�C�����sb�T�dU2�s��97�k1E6'��t
�a#��B���M�e��\C�.��l�S�bd7�md(���@I��v���kc��~d���Fv3��G���l#����v��a�Ss�$,�gN�c���{e=d��!*=nO�q��C���w��d�:�~��Z�@�3�Og�!:`f9�Tb*�#f�%h��4���i��A��3J��������e��Am��A�����b�9P��	�O��=�0�/9JGS�����6�	�Zg��-������>������ ���������@Q($&�8��,��/���\#��E>�}=n�c����`��p�#pP&e����L^�����W��m����u�NK_�G����O����������c�S;��T�O�
h�z��uA/o?��k�44��^�J�g/��q����6���m�$��	/^�O�P�!��l�.�8J�����n�����x�������l&�����V���7����K��2�e^���a��B�:��������\}�_�W��}|(��j4_.i������;?����w�5~���CNdM>uy���<*��*�<*�]��u]k��B�5�J��j��~���#I#����%�#��q��`�`�mBL�".�c-z	
�I��N�v�gloV��I;�C�#�x���p��#��WR�7��aQ���b����l�m�"����F��o������?��������A��n��J����F��?���(���<�3�iS����<����*���]�!��4��(��D�~�Z��������R�{\�_fw�^v��A���#�id�F�����`�������_�m�����!�h�P�~������W�"�,,����~^~P�u�WW�L{�1��4�	w��[-B��.?��~������!=l����.���V��s�Vg=�>rn���"\�6
�8l�����/S���<Y;=i
;"�o�����M�G��^����]�,�2������d-%[�b~[�g�����������
���~@�u[s(�>��E�����}6��}�x	��R���+�H�2R�=�rc>���NdI������i�H�<�4�����N�0O���s�TcSSXD&��#��I,MbiK-�&�4��JJb���$��S�Wb�
I2�%���b�`�D��C��m�jkz��[�Q�64,�Q0�����U�����j�$Qs���S[ue�"�w��|�n��w�����I�����,�*A�m����u ��
�LM
,�$���@�[�.��CH��H�,��O�#J&�����M��+F9;b���+��>W���|Tk~�<3T�Q,��Hyf$�#�`[N6[�K��>>'����G5,~��:9�@%U�������D5�W�%�dR4��
�1�p�r��=��z���c��V+�/��8z1��"L�I�S��r�

��R�"a�l�o�W�z��	�g>����hG��^g]8�u�������|_�t@v�M������d�:���*��{���c@n��yU����y��j��eu3�l�ej���u0��n"o�F����e��� |��3���}��������:*v)L�m��'!`��������&��x��UQ��,�X��L��2v��&���h�K_t��I4��h��7I4�������rg�d�����\�prT.ru?�NwQ��*������u{�"(����O�����M���b7UCpJ+���}������{�<��d�y�B�g�DkA|�������q�*����#	����^�-C�=��Y$��h]�;:xybD����=���B[�����2xGe�/����k���j����q��F.��N�Kkm��2�<��^���6���p	/����Q���F��4��5y�\��}�r�3i����G[��DS~B�_�)>���y���>�
+�y��0����V��D�������H.�n������	,�>pb4-��,��Gop.1�=r����/�(�>�r�W�29M��w�zp���X-e���|��/�i�01���0��$�p�*����	nE��g��9t���� 5���N��	���nbP#A��O�}Q�x��Q���8�3=�
��xo�!�b�rp'���#��W�e;dS�>�03I �~�mF���D�
�h'��M�$�"��=��e#(�h#��gA��v�5�$M,�X=LG�e������e�p��6�|{����bcI�`�-��hcw��~��.F�i�X�/���@o��"�{��Bh�Z�j��]����y��5��c5���.���On�}��3:��)^\�)i�Vz]�[]0|I�H��U�6��t��H/�s�WG����)9�L'}	D���%����Im��k�)rG'�	J�����Q��d".�O� �q��D�%Q���F��$Bs�r}��91����?����#9?��m
%X`7�.za��$�K�"���.�r����.�#���'-�
q��H]	����;�mW9�d7��&�e��$L��L)Y`��
�,0j��&�,0�%U*�*��l�L;�f��p����\� ������=��,��Hl������Q��$IQ*x��U@
�/�4f����z�������-IsT�p4�`>��� )����f��QT���&�����:��V����c�r/eg/��Cm�s {�C]��e�'3��?�(9$��C��@�gO����<����S�5{�\�F�����:�M:��W�H�Rt��|(�)NIe�U���L����GJ{UJ-�
�Q��~%�KW�G�xY�L�3�2y��5�v��C��������S��B���i5��7���^���M��R�7}J��d�Jj��d�2W9)Lp��Jazd����������D�rV���Uj�YES��TJ��#Q}��*��d���lV�f�T05%������P��y>A����u`��1.;Y��+�q�#)Y��+)a`*��\��>���+���,X� +>i]�i�����\GZ)")8I�I
NRp����8e�����$�(8,&>B��k�����9����K����u���J��d�_����j3����n&�E������}�K�!����e��9��5]�k^�"�Vb�(0:����ha��+��T�ctO�5���|��������|<'O��c���X����*D�u�U
I��_?o�q�=�7�$�	����I�poMA=����'�b�U@��Ix�����=Y4�$�	����I���S����.���V���'���$�d!Kb��(��l��0)�H;���_	N�����uNh)�N�����L3�	6qeG�|2-�f�s_V~����A������Ro��e��bD�P(�rH�#R���^�m�����V����QIv��ce���N������km�k'�����Ws>�����o�s��UK�.���}��0a��I�k��A�����SaxAO~����j���>.@�6��L�.�� p^��5ij�Z�MDt�)�08�m�K����f��S��e<5q��b�,��:*�����mZ�Gt�����;�1�H�mLp�a��wb�5<M����.S�k�q����������'	&��to�A����$�����8���r��yj���t���N��v�y��UVd�� 3P-����@���:���0����3�!d�vm�����>%�U�$Y�W#���ZM�Y����I�g�����9���i���H7	���O���wv��Wt-0�6U
�8Bt+����=��S��;�,P ���o{j��ul�v������#���Z4\�Ds��0m��j������n�6�����N)�r����(d��b %�~�Pk���� @����_�n;;��}o9�G��D�I�y�7|���v��1�	v�-���������X0^�V����f��!�x-=���������ds������O���Z�����������g�Zk+J��6��1���h���hr�.JC�l����|[��Vi����k5Eu���DR3s��bv��?o�����_����qv��{U�
w���S�0�%�����
�V`�z;�8�#I���igKa�{���#/���D�����qX����)�QIbF�X�f��/��v��J ��@����O~���I`���.�������ke)��w{������������/#�����7��W��
���A������v�F'k������_�?�Z�\�����j�/j�3J!����s�U��.�r<Kk�SDP�c0��D�KLUS�sTCe)�
0�=��R�0�T.+�UkjH�u�jj�W �YM��0
�Qb��@�o�e`Y./��L�H�w�ij�q!^M��
�Q ��@�o�qPq]./��D�H�{��$�w��$����d���$fCUN&7
U\&7rUj&7Qt�v-�w\�#jkNQ��V��8|�Gii�;o����^\]0}����z�WW���Dj�^��g'�}F��\�*.�$�����-����:�������bm���#��n'��x�]h�'P:�n��p���Xdn��vm�Xz�3�h)>0����~���iGN<�����#��_&�T
�'@�^ooq#�%ni�&7%�&k���,��01m�c��Rl�}4U��Z����U�y�)uh6~�~�{������g#��qo��~P�;���}v�tm���lG	~�-wb
x�@kG	~��vV����e?�4,�V��b./���,/�"�(����1J��7G#�����IE�zS����rSe��������2�{o�D�P29�1���o^��f�9/�xR?���]���������s�4�^�90`{����@Ww��>&��]A���<�g��P�6h$i�m����h\1�_�_^�s�Kz-�p7w�l��q��d�.}��]��6����W���VK��f�Bv�����~q���������tkFdf��'������j^_�WN	J��+I���$����$+p�D�H��%8]��!��
�V�m�.�"�B���;h(��\����XW����,�����
�+[9kEIa��CS
�n����v�h�hu�-�U`5�]�6�
���2�"�K��Y���]����m����������d�g���������@$")����@����@H5J
���I�x�
'@>E
���*���j�n������^#c��Kd^���eo>^}���*
�r����wU��o���_^���J�����-T��";z���Yg~��������������w�]]\�z��b��R����o�vjFX���>ru�m�?�=�.������������w�{]64+�T(�5��F�Y�|�MV�_?~<��"�_�^�s���J]�>(c��W�����U�]�����������7����5b���?kzWd�J{^e�������0u2s��{5Y~.��#f����
���������d��o�3����
����������x-���Xr�";��+�����dich�do�&������t��h����*����p��V������$[��,��M���9~���Y���k:Y3��?����Zy:7?��OW�������^�YS�����g��g���w�5~�3������%79�%L5Pwh�?�W�)T{N-��h������J ���Xd��CW�7<�h�k(V9�M�����m�*l�^�����B�T���|���O�d��@a`������
D�^q�Vk#�;�7C)�db`�=������
'Ti�^~f����y�������~�!Z���I#����lD~�Uw��3�����N���R�p��w;z��[M_lSg8;�n����V�^�Q�Z"��+�C9y�@�z\�T�|$�jN�a<�HQ���'}�
zY�~�O�2�@�	��iy�9�(�����2���W���U�%e����2�AP%�)��
�
�n�PC�Y@lH���.9�cq+���,>	B$�a$�4{yd@�--"���u=�ly3������Uh��H(dl���*=�d�)'�q_x�e����],,5��%��/^,��Mv�����$�d������&�)�OrJ����$?=�	�m�H�������O&�O~���z'��j�����R2M��Q�L�T��0��)��$L�)	SI��&L���q�f�X��4%�d��A`����K�� �p����Q�"F%)�N�u�B���P-B����}��H�WJr�R�(�U^'He�xgnR��Ht��.�W�@���s��P�:���K��z`$�j�_G�s~u���u�|�F�����U��Ka{�r���&��.�|�zg�0�,�\/R�E8�s\��2�vS���o%�t=9�zS���6Wf��L9�<��'�(u�[-B���~�����-�[������[S��]��R��7-�nqHN���-�nq�S��)�]�����}�����Q/�t�k~��]��]Aov"ZQ�F�8�1[�����S��H��_���:���f<�
�2�A�(y6�&��Ub�r��`G%n�5+�k:��T^����ji;[�x��B����O����r�!v�F����4�Y��\v�I�M��Z(��U�N�k�8O;�����;�w��W�����+�[�B�JZ�F��z�X�;�w��_��7&�r�81�Y�^�����D2��#���`�Gl�Y�y�;$�����n�y�9H�A�]�w�Od��cKx|�m!���z�C;�j�Md�ry���|����E^<GC^���5���+$�>"AXoDB�	��O��
��{*G������R���u�G-F�>��1�A�B����4�~*:Ov(����V���\����L�5b@b��?����������%��V_;��I���^P���=Z��-��~�V���JO5���V:��J�-�����uNeG�/�+ZK$�Jrd�!92:��F���"GF?�#����%n��W�u�����Q�5k5ys}���s��^�5����M��B�p����O_�
 �.9�q]n5B�~mM}U�U{��i�I���Oh��>g����4�a��&C�����K-��Q)%[
��+h='"���<O!���i��$J���V��cu����(
�<�a�R���	���`�������]�D��P�G�p�����%�\LK�EO^��a'V�9 /kS���3���"�O��:�x�8���9��O�j�?��1�����8 r����s�����
�b�
�[Ml��C4�
�#PS��H�\�C0`2��������GU�cp�"b�7�&�8t�v<B7<��:�^�"/wm����^&I��>K�����h(���c�&����z�n{
"�O��-������W��}+�E�$�d�������V8�Xv�C�����G�wrz��c.M����,���:0O�������9Ki"�O�T�|n��^�kp���Vi'4��F�)[�-��I�N����$jk2J����$j'Q;{z�����!��T�����D#t+�����u�������$v�Lb�&�$v;�Mbw���'v?�|q��N�����aW4�����[42��I�Nb���$vk2Jb���$v'�;{zb����!��T����4z���t�b9e9�(�=���p+�P1�aCI��L��+�G���G�(��4=�vg99���R�4�ZY����t2����/Q~_������H��p�z8R#<�����z������������OU�p���@�VR���%��MeR�Nj���6$��F�TDjyL����
��;��T���3�%�!�
YR�T&��X��6�hNjCRl�<A��!���$u>���~���ug���������oNI�H
����@@�L
D(Y|J
DR L�%�LM[b�E��$�v�F����b'�Df���e'%")BJJDR"8
�����D$%�N������,�I�������6x��xu�7�"�i��SR��`y<�:��J����TyIe0S�Ve�?q�1���i\�%E���#r>���y�+>&���M���.���6st�n��Le����W_�Y����������1�@�OH�k��bo���Q�����X��+@t�� ���9m���3*�������keTH��J��L�YT]\?k�v,P�J9W���V;���!2o��,�����{�ECd�	�������M���2a�o�@� ���,c�W#��� ;o����zb�j�p�d
�EC�+\�Lv!���_*��^,����;L��	7�����A-�XE���N*b"�����R�C';���jA�*N�?t�~��[f��	X��X�Cu�9�����"P��"cYY�HQ���p\ObH����E$3�j��[�o+�V-�:u`�>�j�h�Q�:�U��d�$�����,�����s���8^�yL���h*T���B�(0��HIy#�T-W�k��.Ys��5?�V:I�8����C.�F�@�F�"�K���8��x"a&z�|
�0��z�f��^�f��Q0�+�C���aC���/���9�y8��?�P>��au���Zx�������vP���������Jc�T�X\U�/E�[B�R:�>t��-��t�(�#:G6;H�3��
8~.(?�|F���`�CL��Q5�T��[�j��*<����jD�2w�@�HVTGCX��������"[+���m����n�n[+�������l�����77X����~0o�Wl��{�S']����`|�X2I���nkT7o5V-?��c_n����}��vYTj���e�����"�F*Ryq|���a��t�r����$��(H�
�P�]CyWR�S.u�pL�����h��ty/�����|��T.��SK	]�>����SK	]�>�<��^{<�_�W�z]��k����*q�6��.gH8_K�/��(���r5����vP"�K.�U9fd���������<c���P���tth��>�%?&xk�t����\I�Nv�QCitO�����_����*�8;��*���f��_�c���������,�=���i�z�X$��N9.��������`��7bD0\+�����f�V�	/�L���Q��eY���xyqGmw���\��I�q"�j���K��pqy��Z�cW�w{������������/#�����������?y���:4�������F� iyMY�1_~�����g&��������t��F!��AT��g��]�<a\E�th.0d��}t��a�*���C?55H�1�1�)8����B��b��x��Amzt�}f/%@��qW��F>�t*�f)��HB�
*�eC	+X���
Y���X�(�����3��[�1'�OV��o��#�K���h���^\]���.;���������/Y!������k$a����ZF�,���K9������h����$	g��n���M���N&V�g�����=��������6������vy�ZIi�:�
4.V�� vk��FXKB�L��	f
�����rI��M`V�`�T^t�
�7�ZmZa�\�*��1n3�06�Yn'�,���\���i�8��������C�V�ZT�M��D&����AT�Z��~���lxN�������:��}��R�4�D��h�����1�f���������)�_�����~_l�A����j��D-q�����<��(��i�&�~�A��;�l'4i ����a���>T��pq��O'|:��Z|z��E����%��	Q�Bw"8j�&"*�h'��D$%�c�HMXPbQ8�3��h���t���9�~U�O���7�,z[.71?_]�|^I�����K��!k�Nf}���t�F��or�+�Y
���$�]��������j,�O��*kV��T�>�
^��3n�&O1B�4�wYb���t�Sd������������(����i�~m�����j&`��E1nOg����#A�E����{�>���bt��@�*P���I��v����/�//��9�������U���V��b�]���a�t%�����W���V���"�w�\���a�n!��f�	���^��9���W��EZ�$.��$����7+v10�o�2��z���Ai��C��e�F�8�������5�Z~tPn�=���f��@�q�n����(��{��3�P�x���F��(hq
*�=*���}PZo�������lTQ����l�_����������$�%9/�yI�s�X��������+���w�J*_��mWk#�/?�}�Ao.���f���x����^k0��m��(�����_���K�i!mii=�M�VZ���|[^������,(���7WB����7�o��Cw�77��qvuq�������oH s�7����L-�c�����G��]����2{���]����/��������Y����u�{�4�7�d�,����/�R�?���x��!%�n2�1����U�U��Iw��C����>3���?���
���?k�F��J%[e�������8@2���o5Y~.��*=_n�������X�r��o���l�}������?�����/��y�Z�K&���t��Nw.�����}/�|"/�}Lu�z\�;@ax����'	w��]�/��S�.���{�q�p�}��Y����Z�
@x3�lUN��;�L�u�����v��/�_���bG��9~�����*	~A��M����D�7_+o��7r����5[b�f�\�9��{N{Nq�M~O���	���<k^��_h4�F�[�q9^����)5�MN���A�%�b��q�v"��z���;���J�G�Hz����20`��J�z���MI�i�PX8,E�(4�`GMyk�
G�U1�*��"M���b�TMC#M>��"dB�����r��J;�'��/�������2V[��%**�xTZ�P��P9�����:_�'��R�.��(�7�D%��=��U�
T�:y~�y��������tr����!������y58�p�����m�h_4up�����
Z�����\/��.H9��`y�-����Iw,��&W��s��^1�h���+��1E����l���L���u�a�~4�u�u5
���
Y����c1�j�g�`9�1��1����(L6Mj/	�_P��+R1\.��@1�[�M���"���}�=<:��C��xQ�����s������H�a�4�@�|#5Op�DD-�"jA7t�Vaq��*tT�,������*,��P���
�=������; *Z�z�EA�'+��[@�;H��R�v�m�*Zy��a:�L��p��K�Ri6N@�ciR
!�q}�p���%�����	��9W>�p(i���so�k�H���O-�/Gy��4d��<�\�7�G�+N�:�*9���i�x,��Mv��<����DJw/����y�b���h:JKM��CS!�n��4e�c7|��k$9� �'��G1����� .1��jb~S�`c���A'�IJ�����:XXk����6��C8
�%c�'c���2��U5�����2n8g��ehm��25�<gm��Q�i�����5���Z�����������+�Z&H��f&@CjM;���`�$��V�
�X�R���u���&�q������S���KG��wAx�Nu
�3|�2�~�sY��M5�jk�Q��H`��bNu.c��z�9�S�t��w4�z�.��!�T�}���O����h<�bU�(�2�)���j
�j�=Dt�1X��y���U�H��{":u N�_�G��������J>���Nd��79��\�dF
&���\Y��j ������F�V�a3����L))�7�H	�I�Mo��������@J�
�Z	�I�Mox����o��Z������.{�x#�����Lv@��
���@�
��k��+<����@�u�ZW��F�im��a�Fs����|>�g'�0�|'����w�-nq�������C#QBRH��0���</`A�R(P��r���e�o���$��~��[s�wx3q x��DIa(��A���%����;�y����8s��5K&y�9���~��u��w�q����P�� v$�H
;�.��9H����`|]�|���r���K	���M_6��g�Q��Cr�:�y%�6|����8����`\�P�0��X��TU������"y#�"L�D���j��gr%u�[�<3T�Qv�!�Nzf$�#!y�r���&�(����L!q4�G�����0(;rd��D����f���)x����{.t�(��=���"��X�^�~��:`�
��q�8q>�w/k��K�o��G=F�Kuk����h1������K�l!T�EH�)�]����j��G�����,'��[n������_`����@jR~��7��G_�!�����
���[�����ys�m���-|����P���v�����e��{-��Z��AQ"V��9�|��``FI��Q�����S0��y�S3W���[��/=x��8a]�C����L�3&��6�������P7����A����	�Q]�a��1�:���0!���P�8O
H��nr���PS����G~O�/�!L��QaP�OS|���Q������#O��+�6�����f/�:0E<��``������2[����y��B{�4H'��F��-A����^\
Q}����h��	9���(
v,"����k�T�X�z�pK w
�F��aJ,0��8%��#�HU1������M��6E�$U1'�����D��
��%���|��[�7w�A���whg�M�r����iJ���G>�m������{���|�����NQ ����n���o^����|��o���o��|�Ex��%�}!���}q�q�U��W
s_5x�}u�[�{�5xw��!�8���u������\����5j�iv���&��s��Y��i�_�F:����������Oh�>e�^�Q1H����If*�
��/�*2����3�I���xa�	9��C`�&�L,�J�C6��1�1AX`B�D+,X\(8]�fc"��(����$h�P/>A�����$��%��r��6@�RuTG������2�������h�g��<CV
�)��ddN�x����D����q7�����hRixvw;�/'���r�.�Vw�5��������,q������3������V����D(��������M�pfs���(��)�C��7FF��#�����r���Zl�������Ju�x�)�f�B���m�)�f����(���T8.�fl�K�fW�mq(��\P!�p8�t���Q�$�"?z�D:��$X��x�	����J�!��o�)@b
�!�x�
1�����*���mg���&^[Z
�pT�,,F��)��AW��M��v^���t`�N�����b>x����<
��6r_�x{*C���F�(Tz[�}����#��Q
iu�T�+<\E��Y��&�\�X8������|�>�����\�A41a�>[l�|��2���R&���J�k�O��?�02H�j��
U�nw��:C�'�5!��j��B��1�1� ��QG?R�y���
JeF]��B=�EO*X�����I.}'��AEAR�-���	4N��-�s��{��������%�8��V*ll��%l|6��A� ��A
<����A|�}���;v�w�����C�J5@�rlzwH��,�=��#���&����2z��S��&�����v>%��,��KPcCB��)��&�1�f*�h�r/�SI���r*��|U��?�U>��;�U.iq�H)!��W5���mM��r���j�UM���W5���uNr�#������ �!��L�����U=h�!w���=�|U]]D�3t��a��<��v��QY��I��J��L� (��,�`a���?*,�	%N(q J�H,��	4N�q h�f,�j	CN�1��!?a'�t�,��t����t����3{�{��	E{��:P�V��t({��>��c������p}�����j���}��
;A������s��5��W������c��]�v�����TG���b?(r���TP!N�$xB���^��;s��{�|{Wc?AR�j��M�a�P�).��T��Is0�	�����O�����x��X�=����8��� �����{�����0�h}�@h����]����%NN���@�'9-J�LN��i19-&�EP���IN�{�'�����!���-9'����h�"9-&�E8��i19-���X]����o9���<5�O�����"C���������}1����+<�/ZRr_<L����}�^J��N���br_����}1){��w����&G��r:�f��P�������3crfL����193&g������3�%�������3crfL����%gFOb
���3���'���@��G����7]��:��I!Ov�FJ	�N�wB������~'�;��	�����������~'��U��~{ch��~'��B�E��8~�(�o�O���^%[��o�d���_��n���l��]v?Ym��y�N��tQf���}�E����	��y���t�9���L����PjM�������aI�x��Z�j�P�1�`p�; P^ I�B�=R1�U/����5��<�k������g����X|oJ������������������ ��\*=��[�����?*��K4�X�F
��������m��������N�O���6k����0�����\^��D�:�R5r�&��d�L�>0��A�O.��U����.��SH�SH�c������2M��K������O��O�u��a'L� ��u
�?T*��? mHw��+Z����R'����f���<���N������S��x��%�P�$�'�~�e�d�I��d�����/�y����u���	�_c��BK�}�<p���LU/�T�>��x,/����Qq�<_{d5L�����}�d�UE��Z���6��FdO�Q�?����[��~7��������h"bK���D���d��f�t����TOrA"�+@q���P�be���������)e+E��|���\4)I]E������vh-��B?�����O[G��������h�g���/���������l]x0~-�-�-�U���0�"�k�S�	"�O�*�r�"$������)E��FJ5|$M5@j�0eI������Hq��@}R���Di�!�R�]�P���t��9��H����I>z�B)B���e�2��S�x*�O�l�P�7"�q�b����
���p�1�1m`��O%h�J,/� ��X�������
t#�C��fm����L(�����B�)%��b��=�`��B�(!g��4
%�
�L�����b��=�-b��B^QB^QB���;}�c�?���AH��~S��
Ov`���~i�b���+�-�
*���{���H�G�� �:�M��q$sL
	+�eA(l�A�5>�1�,8�E�������^@�
����`{A,z��M��Q�m�-���@oS�h����S{(0X!{�B��0�0���=��
!��!�����	�BnA0�$8Y�:%B����{�r%�f��,"�`,Tw�Q����R�0`ZS�Z���
�T��V*ak*\�
B��n*���
����
Y��B��r�{#����N�'m�w��_�W�z]i�k���[o���f��!�~-=���������ds��R���r���?��k�4&����9y��u��G��V��)�T��;�o�<T�r#i��K&]�^`�����������K&Ti�����ro'���b@{�������L��+Wy"���������������e0>�n�������j�{�\���&�����o�Q����g�������������4���g?\��	��fQ���a_l�0E�����7:�[u���/?��z��3@7�gR���'��I���Z��O����v�W�$=��%�-����I� �;�
�7!���m�7^������4�Ma������&~�����b�cV�_�=1����'�p�eeIO�Ki:u�y-KJZ�����Bn�Cn��*Cn�����
�e�����NV��o�m�Gm��<�����v����z�WW���D�N�>���d���k�XN�dVb{�O��;���
-��<lh0�h��cPmg�,_PMf�lZP-e��U:
�8�k �JB:0��v�:<���4�����=���e�K��P79T����GD=���kU��.��'k���,z"Y�4I�K�U����J{��?�9��yU~�lJ�b�_������6�sw�y\��@K������[�J��d���[����V2lyT,�t�������Xdg��T�������	�� };����9���n�F��_���������o����������+uv���|�$��|�%��6^3�z�7<~��rHZ���f�y���5�T5��AHoj�d��?�z��7���&Yo���J�#��p��4��~�:���,I��������s\�kQ���[e�/H��-&�u�Y��/�sT������_+��%p5���j����?����&�NR"S�f�	��C��9���Wk!�~CH�$id>���N/��=�$��2����������4�&Kl���\�U�d=p�����HV��L�����?�!�h>����b��}D�B���
���{|^D]�B���9I���P�`�3&z��1��!��]h$1�xExt�'jp��O���S�>��}��&����%����t8���t8K���t8[�wh�sR���I-���j�"?������^#���K��ts3��|��P��J~������\E��o���_^�������.�}�Zh<U�����d�����7�o��Cw�77��qvuq������1oH[#W�o��[��!�'�����h�����Yue��5�����?_������bT�&�z
����u���|�UZ����������\�n8���na�1��k��U�nU'P'�>���>�:��g�������h`]��Y�"Cm�U�|*o��j6�b������d��#w��r��N������(��&�fPu�7�O��������������\^��C�`�D,�f��,#P��Q(�R��:���DXA�XB�����I�Nd�	�[��yg�*'d��l&���`\�v����/���b��4g�=�/��4����(��������`�o�V��mo��7���}�f�>�w�\��m�h�Y�1������!��^���c��w�5.��(��C����������������$rsh�$a�cD��MeV�gL���TQB���J`�&����{30�{��0_W�����`xP��FmDSnS���3��q{����t�t^my�r���WW!��icj��5W�4!�
c����t2��4<m##m�����b5�,��S*��sT�{Aq� ���8W�zR� m��CQp0���(�2�>�S�m�����-"Q��(N�0��m�'nr�6�{P;���
�'�Tb#�,�����a��xrr�(!��|�8����|D���w�;�;�rg��#������>XA����?��C��M�����Q�`�Q<�'b����F-�+���VXN
+��r���.�
@�T6�-��������,�g��AD�>�H����r��&I>+>���6��~V�{��#�&�����R���FAk;��j������3�wQ����X��T��Uo}��`�J3E�4��6�T�|��]�HM��[��^D��E&�W\��NKB`��B��0����
�N�`9-	��
!�&�\p�PB8%�~��~;��
�9"�g�����A��w�2�8N�s�����������D�H	dR���VB&��+~�����d��.�%�E�+��%�F�K���d �.��d?�Ey�������w%���7��=b��RGz��C+��8��h�=c��j������A�z��5���n�� 
�;���6o�f����LMAT���M01a}=�F���*�SC��h@Q�!)$S/K��?� q��.�8����t���W��������f�Y��q�#1�|W��7�{h��J����"),�W ��J�?�%s?���9��lf�������l����m�7X�?��Ap�A�MD� .��E<�V���`�_��_0�/�/d�/���[�Gbp��j���q�J� ��"),�` ��P�?�/s��q����!�����C���f���_�Hk���U����P/�� ���H�A���r��B�Hj�����s��q�H�����#3�����$��7��	��~�\i�!��H�;�����@������������;�W88n��M��$�
o�b�#����R�o�	o�r_�7F�����o��rBk�9o�Y�I~�<2U��/��=`SrnL����197�+���s�����197jHL�����FbL����<vG���b�wz�DT�\'�GL�YJ���G{N����19:&G�N�!)9:&GGP)��1K����19:&G������crtt������K��O��1!����Q8:f��f��,"G��s5E3mm���Ptgm�X�"�����7G��a��O������8��]0��\�����F{�Sml/�{J��%/�%�	H�*���F���T?H��
cc`k-��0-$�	�>bh�-��0�5��B�pxX����o+CE6.����Z��o��c�������=�����X���)�uy;���;�y��P�u����}e���%�c\���w���M�����}������[4�N��\���bcSp����@����p�cxP�������5zM9]<M�����C=K`�0up�h�������D@�q23����K�y����U�e���crp����Sv
{���)9�`������%�LH�6�Es�����W��0K��*��Q���"�h����F9e�e��6P��AN�=����		a�F��(ss��F���*��������*"���7I������>,���R�g�v�5fBBX�A?�,:d,:�Yt�Xt(��PfQ�����C`QO��N��)C�:@�|X���'�n����LH�6X�EG�EG<�����,j��09],����)68ea86h$��E��B��h;7������<\���z�j�j��<��O����� o��g���|�oa
��M�T�W�m��������z7)�����9O`m>�s�_K�/&�A<xW�N�D'(�'�����X8t�=�#�29~�#|�x��c���gSY��$�����A--M5:��B�����L��v����B�<�1�<zx.�$��{��G�?��Q
�6�x����>!�d�I��lM�Q`�J!J�^������W�a8
�J�>��n�������?�������m��?��������i�?��F������;E���x`h�>�bV���?��6�e�5���f=�#���d��A��-|%�uK}�� n]`j���Y�2]T5k�r��+]�$�9��b����)v�����+g��%�a{L3(f�t
D�
A�g��k<��K7�0X�E�,��H-
�}Z�9�.�
�m[D��F#���*"=���
M^�IP��-T��W_� �/(���1���14D|/@�m���^#~"=G�)�:�T����K'��]|����t_�f"�S��rWQk�3J��?�SB*R	{L�^�:5��I'~&��c����^.�O��\~Bs�9��d�o����M�T_EOrW�R�J)��hR��W��g�Uh�����C�HzQ3�������;�P�=ZPwk�3�.�a=��VL��
u���6����B&,A��
V�{�0�����i
����J��a<��(��"�����V	[��j����];@vyAEcV�ZUkA��(�[�lX�]�
���k��
�$��*�	q�&�����8R�^�A-��Q�`gl���}2�������������n���d3�[�����n���6��o����$KcD8;�9�P��6������wqp���?���:�i*8���Q���Ds�>_>IH�y
VRf�B
��~��c�q�]jU����>��Z�k�� ��P�J�M?M������J��:�Z. ���v8����'�I�U�������/��_Pf)��,S��G�|�1��}F��������G����r8N��p���H�S[�Q8��o��FW��[	�'9�e3�t�#�Q������'aj�;��r:��L��+(�N���f���{�R���:�
��7�'�!�����7�'�6���G���:��&��G5�V���l�7V `����0�N'��Nz[�i��05;��
:��g((�_P�)������5s��k���U����d[���itJ�P*Cb����	��;�S	����x�6�t�N�2_�^m�a����GSM�p%Zx�aD��p����B���������/�����aT��dQ�l�������%V�t��1���1$C�
��!���!�z%cH2���F2�$c�G��1$C�1$C�1T�dq�`�s2�X�L��d�P��1�����h�{$�E�f������E���[:J�0�H�0�#��L��dI�w��a$F�V#F�a��f�0�#�0�#�0�Y2��i0�9F�T&�H2�X���0r�x�c��(]�L#f���v���eD.��:���#��#�0bn�dI��dq�+F�a�m5�a$F<j�#�0�#�0�#��%���C��a�Je2�$�����#�
�>F���#�,"w	�2�Y ����� ]�[KG���0�T-)+�VS����0�qec�PD4��"ZMXr-W�K��G�B�i�&��|����G�������:�0�Z��f�����.���;r�"��\o*�.��Eu�U��{
��{6A�E(<T�|js�����8j5`:�R��V�;���y�j���wV�o5����iEJ5�������F�@mR�1����3����T
��.�Z}��a���#KY����gu�p{Q���p=
��H��]�>��H6��6�t��veQ\VLkM�g�����$� �=�����o�d0c{��5f� ���3O#��J����i����8H��������0����,��,������s��������G�D��\�(l�@�]������	���@M��x?P�>%�=�	xO��J~����wC5�.�r�M)�	x7V#�	xO�{�U���������t},��CG�9l.h��N�r6x�
'	x�����'�=�*�	xO�{�
�H����	x7��'��X��'�=�	xW	J����j{h�����M�����.����f_J��%��� ��'>A�*�	�O|��
�H���	�7��'�X��'>A�	�W	J|�|�	����E�I^���
��+��'\������'>��	�O0|����H0|����'>��	�O0�Z��'>��	�O0�cD�t}T�{�	x���l��N
ISA��M	tO�{�����@��'��P��K��@wSJ�{���H�{���@w���w�'�m����v������O��_��/�/���5��[��&��z��OV��f^5�f2]��z~{_}�gA0<j�?"6�#?;�Q�.B����zO����?��FF�a�#��`�����q�p��"�Z��S������/}>��o~���
�
����N�_L���'	��|��jD��P�^�E~��E9��H����}Wo����1��d�H`���	`55���!E�����%�s�z3������ ��������_���>��}.�}.�}���y��/�m_Hm_Hm_��}��x�R�R�~m_X��-F�/xFo����0�*;E"���\&n���'���K�q�����VD�U�\��\���h�O�,���>�T�\�������2m��SH�SH�c�<M�)���e"5�'���K����B����TA���*#��@.Q��B�+zQ���3�����e����t�9���q��e���X�"��n���~T.T!

$	����ll��p:C�0{�yz�����{�y��x���Z�T��Nk������y���,�������'Y���c,����K:v����I:��>I�>p�����:w��R����z_������@��P�Ar�7H7��S�x�yG�[�O�S�)O�;����j�mi���~s��3����7O�-I���o��q5����y�����Q��������K�aR����z�To��D������$�T����^T�������!�M4v�Aq��fq��{
I��-��9w��*���)���y8�^{����{��x��\��=0j�|v�@+��F�X��M�:�1�2J����h*��[Z1�=3J��v�h*����b�]4J����h*��Q�T�����>`M�3\�m��^m�Ur����4�� ��)	,t���sR�3Z�3Z�����u�����KZ���b/C+�����UH�^����{Z�W��?�� Y��	C'L:	�N�[�a�^-���^��g��vi\��W���:cF��5����u��3fd�_���Y���:cF�@�j������QyJ�<��=�J����[�Y�#��tF	:st�v�@#cp�����t�����F6�zE	zE	z�"�U�-����O�'l�=a���(�N�}��Z�SU��r�[D�����juu}:���k�3:�n�V���"���?���.��U�0�w4��x��e���q��t�f+�5j����$����=,��du�h�uR�h:+
��|t1D�rz����l}w��Cj���33
��5�qm-c��A�����`�u`5�����Q
�@R
��n�DhwT�(�V� �[���0
�!����zGU�*C`]LJ��.0��U{��	�����b�Oa7S��v3���T#���c���4����������7�>7��}�S��j6�x�vSL1��78B�v��3�{���w�
����f���7�,�}�#t��P���s��b�����:�	��}�����I�������U�^���k�f�����6��.g^K�/��(���r5�����jB]��}�O��@����)�HM�����ggD�z�@y��=�5����0�9�
R��I�z`/�5�x�t@���^_����kV��T{�S�i��xj(��
�.��(�t�j�|����I��Q����d�p�;1N�
r
S�wP��QA.c�
��TprT�C��E�
���
r7�T`�I�E9�i*0�T�7<1vpHb��WY���������������S��nr��Q}�/�/�'����G��n|���7����������'���T����	)|�n��'�18�n�O|���qJ>��<�n��'������|�_8�Q}�C��!��7������b��Y��4|��2�6qv���q6�-�&�� ��o�E����q��z)����������~���q�����3��9(��}%�d(?e=����������g�<�����_�?�u�R62r���Y*��s<(���r	*�$]�G���]����A5L�l�5�v�N;vY��H����3o{�����r����.��L%S��X
���X
��U�=��
]������k/:G	�o�������J�DA�MG�����1�r�i�~��j�'�{���&���7�o�2�`�y���������[�����o���8�r�����k������VZ&+&sZ�`����"n�lx8����������K�}I�/������%
���q�E�N�Z��k_�XD��E��24(��Y����w��u����Md���Z��J�|�]����Ww�{TT�/�_�>��s@m����Du����auE�eR��!�_������"HJ�(i5O��P���{ �A+���7�?^��'�R�t�L�7�'���ow�^��tLp�$D�FH~�!�!$?����T#�!$?S����b�^�����X�~j��G��b(�zB�x4ov�-�U��v�h���<��nP��Uy?��]o*m��\n��������"���7_"-�C�la��qS7��o^��f �-�r��n!d#�
6�.�����k\�j$'~�d0��8M0���L�!o������2��x��T�	���<f0�}��=O��k%B�n�Sa��ZA�����T���V�!$m�B.D�>
�C+�`zmR�b)��"<�K
1�d��K���Y_Rv�DGji�2+(�������)��UA9��%%N4H!���
���/)�p�esx���!�Zz��y��0���0x��K��Q�-
kGM�j���/�//��9�������U�����b�]��u�_@�7������*���Ji�m���8��JIU[^���qUq�A� ������s���j^_���N!W�4j
Ivj��I�s��i���`���<����L�b��(Zje�g��^uur�:n�LS����djZ]�B��[��T���<�JWWg(V���i��p�L���3��V5�q�aXE5Sfw�����Y�����;�����b�DWJ�����LM�+(�X��Tp?N�`*m]�B� T��Tp?��`�o]��XA�b���~���T���#��P%ZSAU�~�C�t��Q��1�F��M���"����4�<%t�����_'x%fJ�J�W�����$x%�+	^	�R�W4L�J�W,L��#RF��X��4F�o�]��������������%�����eo>^}���O9�6������������f����!�*�n��=���[������6f�Z=su�\�~��������������w�]]\�z��b��%h'�7���LK�c��������]����2{���f����/���l��
���zM��_g�?I��Mvy�����?^T��^�s���qZ���Q���fUWi��DP������>�:�k�y��Oo��0�z����D�����T������*s!ZhW���r������>�
�X������m�o���������F���������\^�������X2�O����(���'������ C�"w�p�u��E�[0^b�"�=ewo	w_�H!iD�2pf���;������r�����vY�k���]!������>\\���o��?�5{��f�����CUs�6EL����*�����=���E�f�q,�E:B���<x�piu���?���6������*����46D'��MP�������������e�=w�7����i�V�o�����_Qw�?m�|�N&G����)�eU\�U\	Ul�s�����<C������xXl���<<q�6m�M�T}T�	���O�f���bf�����T��jf���M��X�E��jr�HzA��B�r��?G����N��i����Po����B��>p�$�2�$��������&"y~�G|�Xzo-�q�.�A%P�c*���C9�w��TO���B�0�'�h�l1TYB���f�q���p�T%��c|#��}����R��
`��T�\��j�;��}����|�H<�D="�p�H<5�D=B�p�H<U�D=b�pI�&/���_Uby����3�����#>CcQ�l(~��������QR]���1��xM�i1�������HU��gT)���d&E�����K�6.w��L�4�aaz�V;��5<A�r�N{��R�(�G�$� �5�B;aL��\/��D��t�X���@��z�GMy~�R���/p����=Z-�"i��e u��A@��`����^1��S���x��YJQ�q����"����)��$�
�{#�W��6o��������U�\�
�<�!o ��	y�g|y��D����` ��R���2;��
_	��	%�
t�r���:GB�\(O�%����\(]h����P��K!����+��L�����|��b�
�	5"��7m&��Pw�
�?���������x%���.�����_�H�����C�I��b(�"�a �[Bxrp&�?���������E��^�;|)�'��P���fBO���1=ZW�����\�Q�_�0�F-�P�H�_!���%};FU.��������)�Y#g$����f���YB���8O�3�8����}+�����x�3�+�{��|%��B	B�eBo�B?T7��'�q�8ga�]
�'��@�������}$��'�4�|-�)�=b[��)�h$���=a�(��g�J\7�5U����K��_�<(���Bh��8^�3�0������	#��e�
�rqu:s��@\W����ZP���m.���D�F������$�M���E\�����:��C��"�g!r�8k�.��P��iG�8I�*���Z,�R�
�	T�8
T�
%YJ����BF_�"
��Hl3�{D��T\��S\S�Q%���8��2��"ug��{D�	�wGB{EY_��>�s�?�:���8�<8���P[I"�:Q"ghqF���R�E�U������.�q"e�$ JH��#��(tfB���/�B���VlQb��D=N�?q���B�<�
�\
aT��&�)�*Q�g)Q��:Qj�E�8�yPxo(J�#q���D-Y�v�:d�I�(W^i���T����O\m-�~u}l%!�t�lZ:va��^g�O��������sM���RQ�!��C��V����>�J��hhO�t���Y�����Fn������U��D��"��J�eee%�*�C%���ma2u�L�@Y�J����'��?w>�L���W4����"�H�G��>�I[��@��^T��Zx�>kAV�<b
��y$�a��
z���Y/O�g�PI�GL9�>k1E���O���)���>��#�~J�Gj�>��)}1�H���k��H��Z��>�)}1]�<bJ��Y������#wH�G��>kqT�<b����������s��y^T��	;�bZ/�/���'�aM�	Wrd/";=�K0!�$�9�l<��z4n��Hr�m����kw�S�L�2�vHq��q�v���]��8-��6�����d&�D��<c�����1U�pN�DCxp=Z�S�z:s
�U�5�}A{�P����E��w� ��O�����[�X����@�yw��2��$�O����:�
�y2"/���$���1 �|��)b� �'C�����j���!�b|�[�z_+����f��z�^Vx��>��l6�}G���9��*���M�o��~����!�����fX��M�?PZ�1����dM^�	�����T	"w���p;�e�j�z�O���Vf���4'!��>�}KY$��W<= ������g�r��BY�]~*�f����H��k��pN�=�D��x�P":e~5��C�� ��4hc�TWS��`����G|�����~%K�W����$��f��������@�
����\���;O�9�k������;,�S�;�2�s�yCz�g8*_�o���W������_��l��$�P���`p[������h����k�
Ul�q���T$��n@}`�@�����]�3����x�9�j���c�@M�+]����*����e���$������Bbn�}H�E�1�qb)".�
�;������C.R������FA�d���.P.^R��{?�L4�������� 4ia�EK�,%���h���c�vx�W�nP ������9����_�$ot�;p�Hqv�%C�����9H
B~�q X�K�1q�V�$Q��>���}}�9�.
I�)����d)���1�
|��r�3�O���-6J�.��g�XP8CV�$���-�Z���f��,����8�&�0+�&f���B~tz��Z����bcy��}w�!��+F����Bt�Y�+�:����V�I"F8�g��]�X�Aa<"�P�q�	9U���)<�l���2�)]:R����B��FSPi���Rp���
*M6��^
���A���i�K�1����:"�f�u���_J�~�����d3_W���d��R�'K��*����������������������,�1���C�M~���~,��zSG������e��a��F��zM�nQ�����$������+M�/k""���]�(���r*�X��]^����v\z����XS�c���9��j0$�s���0��$����#q��:�(7NUOC�U����
��e����M�f�����B�%�,ja>\���>��l�����L�hU`���efN��-H������V��F- ��Bmh�R��W���u��>�on*-jI��J��r�i�}��
��_V�,�������>����������d�r7��Jn��-��������F�Bs*9!����	w�)o6(}G����9�O�n'��A�"d���������j���~��]��|\�z��	\�E���l�#����$�l=�C�?e�������d�9�q�
��Y�6���T��`�C������K�m�g!�t����p�����.������"�����
'a��9[��r����u��@����t��n�
�����$����L��y��e9��\�2~����~���&m����������t�kn.�����$��������{��	��s$. �BH�c��cP�
��e<z6�����;y��T_����S����;#�N�%/�/	��&���oO[��a�DS}��' ��e'��%���]��:��j��.�������\��F����v��$	���P�X�K�y����r�f
M�	���]n�4=����#M�8��)WB!��Yl��ir����xd�-��U��,=;e��07]$�<DI3�7b`��
&��.P�r�~U���������31�X��mQcs�M��QY^����pm�l���]c�����Y���W���(
A�|u�3:/w��>��Cf>�9S�������A�����AGlJ�O�+�MG
��N[�����x��Y��q�u-���iU�O��t���8��<�x4��+e����X�b<�rl��:5���@cZ��3�Ih<5��z=7����~�R��Ur���F.�A&��L �(Zj�M��������58���<$��G�������#�t�)ap"jg�_�_^�s��=�sr�
&{���|�M��^���<b��*?����O�f�(���K�i�(�)���9�}���}�yu�������A�V������%PYG��?�����g/'E�,{N����J��N��{�����l5���g���[��_�����j]&���d���Zz���,��_�������}*%
���U9�����-�-�����F�O����}����|s;���G�����l^���r��}��?�g�u����f��Z5������E�Z}��d1��]g�O���w������?�������)���?a'����
Q����7Y������b�^��n�(�1��U��|�(��Y�u���V]z�L�2g�X��_�������������U��U�a%8��e��b5Y���r��
�����>�q�;�n������"5�������5��W������d2�OH������QY�x���S�������4����}�?��s?�O������V�dSV�0i`4�N��]��2������5-��v��.#M��|S5���#�+�-������9-��Z/�����&#<F������dz��vzD����]_|�����������t����8�1R3�%l�������5���f��Z��8)���"E�Iv����m�#�i3@�W
�U�������~�R}�I`���J9������v�	g�����n�Vh�'l���,��[~.W�V�a�b5�|��[1��rr_�H~wh����T4dG,��l}�8i������Y�D���e�����%�t�����V
v�'�M����vI{su�����\^}�WU{��W�6��e� ����^A�����/z$)y �y��$c���/?^|�`C�N^��z&��R@r��|NOL���|0Z���|���pqu��}W]�oP�c�G�6wc�(���g?����g�O�sYi�������\�N�����>2i��l���_���e�����+�r�xM������9��R�(�v��d��T���#�A\���j
�����o�������zhF�%z��
F��Zq���!S�!�or��~L4�u[�c����r]�kr�;_�����|df��(o�Da�����9�s�Ms�Ms��
�Z&�����T���2:���������iY���+>E%�l�,���.v��V���H���3o~�h$��B�m�$���sN���inr�L�����SU�~�!w����n�~z�T�g��!-�C]Y����������6'*z��0�n=}�9t�����������1�M�s�]�t��i��a~����Ez�������I%V�=��
����{��:f���h�����s����}��"�2���G�ya0F$���}q��j����������'K��Ye��O�������_j��R�FDlp[x)�`M�3Nrn�r��4��<����*(���)��,���[n�������a>+"���C���
��4�L�#��r��t�@,W6��$�o���%��+^h�cY�SZk4_�a��a��e�<���o
oh�enqvy�i��>��P'��ou��[��3���mj�w{8t��A���A��G�T�./'�p��'�1^$�5��r�X\��v�_����|��IP��i8��#�1�����	��u��#I�����������G�%�}���M�\rJiFZ��b���Sf��u.��~k��k��Y�>"�Z6�ZZ����<��������-%ZQ���Z"��KDwg1�!������l�F5�}
�P-��Z�<�~T�������j��YS��j��.�T<��!�)�T�L��H	��O�m���)^��-��8��@���&W�1��+�v�h7n��K���4������9�a2��Z-��#v�__R��gI��c�>���v�}fR�P�����n/�I�^*Gl��]B��P�9�����L���+���#����Sd{S�]�D����z;1����f�r��g�(��c�]����2���ono.[�����Y����n���L�0�D��NfKx7����-�Z�=����=�x����Z�oU�h�-p���]5���'���v��r7��}+���o��dlLV�"t�x�r��%�V�@�-����X�����X��5&��CC(kxs	�[[�pC��4�ort��[�0��gnc��iIf�s�����Q�}s��g�?��l�
w��x)$D��A�aS���|&��B@��a���<@{<���Y/U��R�T_�{u�?n*e�a����f� �\�,WO)g�F�Vx�\`�����r��rdQ�
��#<{��������<��n��]kC�26�`��(��?�L��|�>^m�j�;?(b��QE�A���o1D�3h��k�O2��k��}�F��������^�>��%):������y�Q�����?��;o=��o�|�',��y�����R��>�L�c"���h�q�{�p���%��Cq��Pz��JL{��X�������iQ�#}�J,�gq��Y������d%���8����0��#2zD��;�^�x/S���D�{����
z11X?&z[���������`}L�97<Z�$&w��A
&7�"wg�f����#������0t��b�O�.��a�� �
������F)���p��~���"�(������De3��(@����ug���s��y}qY�$/��/Y4j\\�Gx�������}��=�o��hS������;�'i��WmJ����#�},�[{��n��e���)��m�0H�-f�;�� H��o�H ��y�{@�kc�'�GJ��a�'�GJ�Ka]Z>�e����3�\;�A��e �\��|)A*%�.�qa���?nO�f����/�G��<<���g���Y���WIm{���l%���L7�
����������\����yWaX)�d��~�B��3��'��������:�fB��.���%��f��	���m��������A�1vG&�c�y�0	=����\Mc~����:r���
|�/�0Ui��IE�~��p�������3�/?������`�p������<���ws����-�u����H���o���O3�u������d����"���K��[��?�T���n��<�[
�1����%D�?Uo��r�Ya,���`����#��"�<�rs!B4,�Ksn@s�ap�-x�o�������c��1�P�M\�n�X������.
f�w4s5������L)'�ZS�h)���X�����������X	�G�U8�"s��+|)|���)���=!�IA�=��?E#
'��[N*�������2� t������otcZ"��]��J^��{��s����G�����UGI��GB�w��A���-��U�-b+��f��T�Q��M�iF����Q��d>�E����c~���rJQ�����}�7�}�+I*6�����`��������`u������k���Ji��,�q��\����Mi��,��],v�]��g��r��gu/4r"�0��A������h&�u��s�������jfJ��}�m-H	�X��MY����
�b6<����i:x�?�c�i���}4��v���d����Ot�*����*�Y���ES�X��0y�9 F�l�&D?��N����h�����0��x������v4�q�WC�ct�?#/UI��������������`h���c��0������Ck�4��7��-���/LN����x>+�{���Mm�K������Uf�N���"v]4�������K����5g4��`<z�ihK���F#��H�l��;p826��
.	.	#2���NGs4��
!�<�h��{�4���:���Q�
tQ,�O�����S]K�*������,�?q�g�<-YI�<cei�)t�2^�|?���q���w�)�6�I>������������G]�����Q3c��������!a�[�$x����T�����t"b���qp7^}e�� E^�����R�=>�����|x[Z��R��`��#H=>���s|���������R��`��4�ZL��E�PfV������ ��#<J�����G�9>�������8>xC���K���#<�D� Sbog(�t�^�`na��1��O��m|�\N`��@(�����w�#���2��Xh7��������#��K�ne���+�����h2�lhA}n��-}�����k������{�W0��5���3�[S�ot.VL�l�[��a���'$�m�C��~H;����h�Z�G"\�i�
�f����g�����Zv' �;0��%P��I��3�2#����������������.D����6P���;�����.�O��1�;h��.�.._�^�]��h���yy�F�|�86�!�Gv6�!�GvHi{��$��)m/�d��<�I���'�C����ad� %�C ";l���|�dd��::ZRq�yw�4��h��!m���J���-�L�0��R�a&���MJ��%]�w�����l6�r9+���q�B���X2$��F��`���P�����������n(��-a�&)%�������w�{2������,�>�	�7�Fw<���)���l�x
&�wck�\�[��n\�7Gn���'�p�����y�9 ��2�k����'���1��X<o�6��7R\@R�a�1��@rp�����A,����3��9q9�eXA�\0	�f�M�?�V����[?�����}_ B_�g�]~�+D�p��7r�&f�*��|��	�����U�X��0#�Yw�p�$V�;��w{{�_*����&���r�`��x��
 �V�6z����:����<���2+d�R�u�~�s[������uQ�]��jr���#�#�MJ47`�Qa����X���/�F���Qn	;���W�:���>J���FJB+[�s�P���FK��nm�fHK]���n3������p��7�A��4K��JDJL�����q7G67�6���mr�5�2�Ze�Wc����!�oq��G���)lN5��C�{��1T�3�jf���Y7�p�lV��5�oA����b��
�o�}�a�uA�k�7�������j��9S����T9�j������Z��1\^��G�cx�tJ� ��b6��'��%��%��J����F�Y:|����\�Vf3p` ��Q�������Cw<�*�`��Qm.s0�-Cf��ox~m3O�[�����f1�����")����_F������ 0v�`�����gO���`bl�2�89�y����U&����<p*�;O�����{X��Y���������-����Z��u��i�a4�!a�|���2�A����������w�����qf�Z�������*����ru�\=�\`�������f�Az����!�L��v�v� o��]���gg����k������vT<�79����_vY?�=�"�,����k����l��=�^�8���~��i�!�f��>d�|H��i�^��!��
���Lqx��{22�S�����@8=N������| �����	'xX�N�i8������W��:���Wk�������������v1y������:��y����R_�>��3��p]���'�2��W�n�>c�X_���X��q��N�|-F,Nfy����K�=.�.��k��`�k�z�jd%�#v������u�!�����a��C�n�����!s�����&�&q�?�[�#������3����	d�){��6@\�U��V`�g-�PR�[9�����B�Y�x�5H��n=0���H<�A�)���=�&4�m�l���R*a��Bj�;��V�;�-��TJI�(A"�8D����]EC~���zw�0�t6������y�`�r�@1����CL��u�!xm�{����-�zK�j�k���M���6��N��Oz�MV�������<���{@�
$)��Al�����nw�RP���;��Q�TfF�����mU���Fv����F��.i�p����P�rg��[��VT����~�e�V$�_m�����d�w�nf��O�6)�����Q�Zw
�;�zC�t����g�C8o*��,�5����=��BG3�:<�f�� �
����*?/�L��N���N!��/�u��
�7��b
m/��:�!��1�Sx��oV�����-f���z������T�T���f��o�����^2sx
T22���"=����S���g!��y[�L���?_)�y��{��M<�8��m��0Hy�o!��`�{�o�N-�AvX_vk/�M����\
�w���j�������-f����\M��&�� �\
V<W��s5���s5���)P���������j�z����y�9W��s5p���s��s5�����\
��j�?W������n�� �\
������BH6XOHf�-%��n%����{�%z�����&�
�*�xv5����B&b����7��n��]�������f"��.�|K���� ���EL1��n`N����v�-j�e��Hp�@[+L���k<�����3�2R�w��^	����<E�����}��+C���A�*<k�M�n��"�O'����"9������
��
l���K
������������|��B��x8��4+�p�����_^��� �\
K�"y�&SW��������S�h�^�&h�+�vnw~������Cl�����������S�y��sp5�5}�nYv������K9��(�nq3e���p���+��l�,������'a���"h����!���u��p�;l)�y���o]����0#����o���9��|�A��s��p���k���n�N\�u{�1����t�w�6�n�u������Q�c��|�`7�mw
�Y��^p"�x@�w���]X,p�F��������ta%�m�z�A�w���Y���[���D���`�X�K��������-�}��������
�c6����f�%T�d��h�QjB�����#/���(��Z-�DP
���h���4J�Fv��j���o�oR����5����y�VRX��B>� �}`��^�gi(�����+4��J�����$�B��%2:������l�b�%����6u��eG�3���H�=�z3�WO����vz����;ok�����/���/Ve�3���1�gLN�N:����/���m�6�c��y~~��7����g9��G�����/_�|kd��Z����;�v�������~�_>Z��D��ng����g'��a���<e��M��6'�^�{7O�A�)MF��aI�h����#����n4��I�n,���w���7=���a�=0_��dc��e�����S������&���D#�5������D"0%�����x�cwed����-w3%��(���C���NJ�M�f���)�Zk�q��������7������Pk�Q���@��������x�R�o1�ib�us��e�p��Yl������z�����K�F�#�p�o��mAz�UP#�u>"M��������� G�K�p"���b�A8G=S��!���x>�����E�rz����P*��H��)�+�j%y��c���@6�����St���n�����F�c���a�+n��KOQ:������Y�3����77��p�q<���Mh59���4������Y���~mNN�o4��K?���y�,����������������T��j��L���/ �s��w�_����9�yr���'>O�|���������h�f��|�(.�m��%t�r�a-tm���qQ�\�r5
�A������V�5=�oR��6z�,>#�[C�or���m�z
�
��q�8L(dlg8�;���J&:�O	��z�5*
���I���k�h��T�O k�����������C��}���1�:�y�Y�O{����HH?�2`u:1,���q���$�/rH��b����MW�dRtk4�y�+,��]z4��D\H�������O��R<Z�����*��-����>S�n������,Q:H/$K�Z�*�.�J�������J������OR��q�<������-����#S:����Yo��p!��<�R�!��\y���j	�{N9��)#I��E���W4�s����Y��_��I��[pnA�L�)�]a�vp��dDPl}=t2�P�A�(�y���+��.lIg ��q��vP����
����G[X�"����BJb�V�C7X��S�� �A��<�kD;�K���qIR�\�q�]ob������<f�W��xc��)C���n����o�v�5���VZc�9o���~-G����K�0�����1c�J[��pyF�|�H����l��a��W0C��c�f��%��.��������������x�>�p�����1�Og<l�G2`���}o���:�fik&M-
�V%&ow@��|�������N������x8���T�(�l}X��6�r�ML�F�o���1�;p�l}X��=$%?z\.;��:�����a�2��(�Y���{qK�V1�j7�&��Q+���5��1�1m�\�E\�z�m\��0��V�W��-\�yVq����qm���kT[O�����s[�[n5��p;3�-����������-x�U{�0�mm�;�r���p�D�-!�k
�kxv���"�%�qo����`
��w�
b~�G(�C����(����d{�$��s V1���^M�AFn���<���Cr�@�!_g����3����z�-<�v��0r�[�����6`�E��i3?3��g������Rd�������{�x������'��Q�w��G��>Z�B�6������"�u�vz�>.?�-��A��>{�+���{uv�r����G���wl8�H��,��r�r`��ixc� ����AnE:>}������������C%@��������3���8sY6;m�����C���S��@�<���9F��s��q\
�D�a��,H3c����y�x��O�1	E��<h��8#1g�]\8�p~���#d��E�yz��FXVlm����Hv�}+Y��l�dg�s���KR��"^���y���)>KL}
�a��d=��M�T��g��V��n;�����q&�p>���N-�/^q���UT���2���	�5��8��~z�7��7p��z����^q��\W�<c���xv��g��
�6�N�w��aJ>L�����w/1�&���f�$<?�����v�fG���?jN�&>�DK9?�QL��s��?��=�U�8{g��&Pd�4!V���?��B����7��~���'��:��<�+)F�������3����o�����?��\w=�~��n��C�N�IvWg���-to�-Q���K��E�s����?!�w������`|�������Z�mq96�R��PL���P����t�A��~��@?H:�c\nOC�]��]��X�����t�����w�6��	�4.��a����u3���`����70�;r����/����9��-2�z��_�Q��m������?Z	�t����U�������ln1 �R�K���A�c<���p�s��2 �0�K\��-�|�wjB����z�
2h*;���LY�n[��mSiNP���~`��
�����4J������7���v�/;��Z�^?����ng��5���rV���Wz� �7'�~�^�C�K��t�z�(�	�f�Y�;���1���9��$��}�Yl0p��y���N#@'�!�)f�P���l:��=��g��Ob��
�/�r�������H� ��An��8��nm!����|D��+���(���Ho�6'tZ�6��x1}����;�-#�e\+c�&|r��G���xw#�e�[Q��>�m6~�
���.������p7�E���D�z)�3�-�Xa�W��vI�`�w������C�-��y�z�kr���~�8z�Z�-��X��`7��]���w�������.��^��m�{'p��up+H���^k��Y�{8�-���cg���r�PX�po@�A�s^C&��]�e�Y:|�K#��������������!c�����L������d�a���]������+�d�+w�c��%a��&3�	��q7���u�hc��nH�l��fr�!N��E�v���6�xG�P���<���G1������	����m&XJbJ�!��'���)Y���fX&���*w=�m�T�������1V:���[��t��b��~E]Y
y�3O ����D�9�m������a���rgC�����8/b�["����/[��C4����t���,���#e�1i�a����?�>E�2VM
ri�_����f�|5�ECC���`�X��
Pw��	9���L�V��z�t�~���xF�����li4f��YuQA>�lm����$K�q�� �(�����p3�z�I@mT�.���I��Z��2,����
��|c�� ���le����e�U2.�\���b�����/c�v��O��a��)'}��x<�����p���v��|�z��<4���Op�P#o��m��NH�e��~@�
gF���m3��&J	7so��P5I�=;$��z��F)�Q�~`�nN��4+��]c�j��W
$��:��wJ�[��*vn��!o3f:7��=��3�
��
�g�-Ht�pGn���]�5�a�!���������/�j�D���['z�@��6��Im)�n����;������!������q������X$��{�l!�W>t��|w�����v;a��������b���#���#�+��l�;R,y%l6��]��wD�l�zy��v��m��wE.u�m8|�� ��]�o��:�����+jw�=�b�60�n����wN����qO������wg�����v����{Q"����x��0��6��p�����)6�R4~���nt+���s��B�R�I+}o�D��x���������FM��A�yqW3�����D�]�{1�[�����|����.�k;���T����/�X�#��}��g������|��V�V��v�W7�;
Y{��U�5�Z�cou�7�5�k��f���uo�7��F�]�(�������{���l#��Bl�=0�'8��`�{=i����+�-��S���o+4�Nc�	r��+���|_F��#��".�wF;{f��h�5�>�s^=�s(���9��2��9g�A?��F�hd2�3�t�9�"@|��E� f��oQ�,�`6���wJ�N��W�I ��	ZDwo;��^�j9��<��Io����e6�<��x��!�Q�h��I�{�h�f��
F\���2q����Ug��$?�k��2�t�����ouu|	�wb��n�R��%�(����E�����*=�n-��(��������%��vd��n��;�s��m��MpH���}~���������v!�*�7��n��l���46U�������;1�����,zF����W4�[�����;�5b�+�(��$)�K#J�_r�l����~��������?Y64�8���k�c���]��4��m�ID�n��1����^�j�1�m`�����.8��b��ws>}��:��z���'+d���'�A���z���f���F��Y������s�w���X�������[�����7P��N�6�]+r6��6�&�yT�f�Y����c5��;k�������nI�<+��v}ePn��M�]���,�����v�c���:Z�1,�����9��.[�ns?�-��U-��_���vu+,���������}�}:j|<�%����r�������,�$N�Kp���m�HqKz�k+@����0�p��]�cX(+T�7Z��b{k�~��n�Y:|�u������v� ��MgIRv�
N���Z�M��A�������@�V�.���;�-��W4��K��t0��
�����( %P�=��\��<�u"w2����p'���h���:����Q���Q�����0��;��
f
H���w�����]m�M�g��#a���d X�T����6�M,�'�<k&��x�!����LFa��~���kX�E�<�+��\�f�L�6�y�.�/>xRa|��Sc��h�<������GH��z���|��i7v���Q.?������y�h��G3�.���pW�;��.��A�����gV.��_��\a�c��������-��{^���h,�!J�G�>��U�����������g�����c�d������}�L
Ln��6���.k���xg�����GU>��j��=6d2iHIg>qO�k��?}���7�!����4;Q��Ad�����|��Y�Y�6Y/����q�]��"gao-�����Kl�`�MH�no�a�BgBd���k���/����kNh�}I�}}!>��`J�����e����m��������^
u��x�v�����M'g�Rr���*q���4�u��]������u��S7�#M3	��0�dN?���$Y� ��us�7g?^����o���7����=���A����I[?��� 
��"�����x`���;w�������NWxO�cG�O��u�K����
�
v������q����n�����F��`<������UD�6�e(�+��{?v���c��C@���?�|��
bI`��B������4���S���X6���������wn��2,�_�w���h4����wc��� E�m ��n�e��c#.�q+�_J�a�
�X1�;����Ck�����B�rT9�4��<|n��/��u?�Vk����/����Kf�����It�e2-O�^\��PGR��|�� Qj�#q��
���G��];jV�o������V���������a�Q����1�c��|6�Sg���=��G�atT;:��G���jx���v��6?���z'h��f������"����V���Y����<e�F���c�^������@�6A��]���/
��Y�Z-���'S�o�V{O������W��?������)��k�'��_�V��V������������m�J�p
�.������AT��?CM��:�u����I]z�*���$�(�p��?����Kd:���#��>G��B��l��I
�(�1;1�J�_��i8�\�T;��$�������y�(F��8��>���K��c�9���i���?�)�R�+��6����g�5��g���q	fA�Qc�X�������*��/��yLC��\���o4�\gH�z�f���Z�����7�O�p���=����g8����M�������O�$���������R���x��Nc=��i��}�Ze4�U����8���"��e�k6X��H�4���#�Ka��G2���M"�+[+��|�����S`sv$P
s������'��.1�Fqg�o��	���������+�}�?�@�SK�'C����FJ����x��=Tb}�"l�SU��/o���������#f���!:�#����j�S8�G�n
��Ouu>\�5��_�2���%d�1_(������7?Y��;� ,�>8�����#��������"y1�r�\`����v>�}�d<�E��M���/�k����1vt�}x{5�����T0�;:�]�|	�����+9��w��_(�����>?-��M;W�B����v�s�4*��(8�:M~�W�M���_�O,o��++7j�Cv��=a�' ]7��x6�wf��	�����|�;������|_�p,�@��W��9u@�.�}�C�����{{��&��6�g5m����<t�)� ���[Y�n�69E���J�~v�z�)���g��<��+�{�)��/��ph���7f0�����^���������go8;���$d-����n�{V`�T��	�U���`��8��ao�������"��H�����5��t<���h�4��?�y#�3�
8�Q2���YMCZ��P�"j�s8�M�����=��&�:��1�-�D��%���=��
���M~
�=&��4���f� ���K���KR$`��}�D�� ���z����=~F�ku���{��fxC�~���N�oh�]���+�0Ix	 �n>g��[b���4��z�8�����8���S����.s��w�u|�������S����n8'�L������s����'�'&���>��^�u�'^'�'�s���p����ST�|�?��3
n���g��:�aCp������xt��1^�@F���&<� �D�~�u9=�7NJ��-���+P����34h/�lm���#���`����2o��/z��1F�P�mX����J�`�p���@�A{���)I}����D~p"@��<&��>��d5b,8M�{v�!P@����:����_w�9�N`{���^39�
��������D�&W�}��q�.�G]�Z\\<_K~���'t���#�sv�I��VS��%Z��C������{��)8��$����Sk8����y}:%���oO8��X_�p���d2���	��	��@�q�%�xF�+y�-��\�Q�	��? u�N�?�_L����78��Oe������PQ4V~��a����<��/���g;6�7�>s��+��e!}�7�J)qC���brH�x{O{����y��O�I��G��Q��#���I/�^�#���a��}@��'��a�Z���@��(Y�B00U�-(����,+b9DQ�M.�7t������`�f�1��3:���Ot�7n���!n�?�jGr���a����"+m�%��gD����y}gHj��Q��������
�^]�%�w�a��T��w����M���82?��c�[�5�F�l��.��b�����.����?�Lx���|HE��~��@�I� i�
�|rf�S�V|rk�s]����2`F��	�����a�C���^��dE�{��&��~e���������G��NK����tSS���P���i����[yuX]��v,Lv�g���1�����@C����7��5�I3`I
��!4��t�<�1
d�|���0�b�xI��� �+�z�����{���|v�������7�_��l�<���3�HW�W�S3����3�!
?O�<�Fi=�\������n�r��!��f�1��jB��9��^�?��Zg������Q����iY��V{<u�V���l\nGe�#��*����������j�����������S
�A�$�5���	kG�F�Q?9�O���j���]�mI�{9�g�P�6�6��6�)��j��Y�YAM7�b&��
����PH�v:��S�fq��B~oO���4.�h�f���>)��7��#J�R�-���jZ������G���;���Q�2�iYw,�S���3�x���&
|Z�;�p�N�JH��\����-H�3�t��(E����-�Z��j�q��:a��*r}��g(�3�� HB8,8?�a:�@��mX"�����($Du������K�, 6�|�V����������0��1�����J�Y��HO�@����XK�6(�u�n���Zu����Bus�V����r�X)��l��oV�E%�N�Qo���;n6�a�N��FR�+�~����������kj�_q�+�W�$h���F@��=G�A7�C��R*Z�B��U�0�ct�����3�W#NK�����cx���A��WT�'���Jv�����h��v��}��K�"�$^;����������a{��h�� �+���yT�'������$g������$��rn�G��D����zl���F��B?���������i�v
I����M<�a�3����Q[)�3~�u��|�:>Gz|�Z��
\����kqK�aD����7V0f|����?��+���y�X��,$�=G)�V�s��[��E���+�1���&jo8E�4��b�Ke4�I����3��x,#�p������
P�N�/_�kh�����������O��tF|����5�t��_�$X���\H�x�A�r��sR�iP?�v+�����V;N?NS�I��)QGT���o����Z�i�\cf���{�VRx/qX"��"@nS�!b��I�4�Xue�[R������Urw�%K�:����,`N<c�%0)�h/*!p�W�������0
�8���F��R�=G58h���1��c��8�r�b��Q�O�q��������E]�use���x���'�r���Bh�{��
)���'`D�ak}n9�,r�r�s��!��P��U�����$����J���%#��!�9.+\@^����VJ�;��'������GAB�S���?��A��Q�/G�S
�N�q������vx���z�����F�~�%����~	�|j��f�i��&:F���"������k��+H��hG��Q4����j 4����h'&��n49M��@�1�:���ose����Hdm�z~v���JGhu2�h��=&-^���bj�4,��9����y��Z��p c��3oN!��2tA��%��w��XPK��d:G� l
m�{|�P��mG�x�?�O�g�Q'�Zh�K(H�5P�ev��9�M&��8GMA
LN����ci�%^8&�K@���.��3W��0^��_^�`�.IK����B��[G�*��p�������������4cnB�D���Q�*�m���@g�+�%q�-�f<�]?	�����Vo0�><�ng����4�c��<�]%�U���:�F�C�7��M����F��c���A�
����
�J3c�A�D�
�]�a��h����SA�F5�6���!�����j���X"L[	�O�u�u�J���K=n�O%�
�*
��v�Fv���F�JS�KpmT	2�p���E
E�+�xBGU�}�����2��$�����$!�����������,��W�K<�vk�G'�J�z�mT6��Z��������M�
���2N���}�`��,�0���F|�l���Q��'F����W+��0h��"N�'��)<���T���|������=P�F�jy��[�e_�Qo�����o���#A14���b�=���t���*8B,���?�Bx}���q��Y)�{N�C>�\�`�6��b���?�����${��~jX����Ov��A��n?���"�����y������U�����7p/�I���>C�$��4"�<EK��hPK���`~���3���g�H&���D��.P7Hd�X�*F`������i����0���q�$�@D����)��!��J|�1x�M8Q9���}�4t�-�)A��iH�
�����h{����r�[�����i�����b��'_�{=���+O,d1���w��-��u����S���*_�P��w��b���l��(9����tF��������Q�t�����8����D��V��b�rcf�x�r4��K��(�=8����9��hhC�(���'��D����7�ys~"|�� ���$���)�0�G����G�����M����gs$E����ZT�p�E��}�92���1���<���7k����n�!��-x8AL��Nm���	����o�	��=�U���g�*�;��v���T[�gU�gz��	xE��O�n�b�-�����`4i^6�Eq����y�TF�@�Z��9�%d��i'�^�I"�:��V$(�|c�>A��N�Q8v�N�`�G��
��4(�Xg;/\L��1�~<���ZR�"��/�0e�|�G��?�A����]p���)|o�A�Z-��������2Tp$=��e�1$��a�I,���BT�K��|�����H��?K/��R��+�ktu���c���sW�o
�)��5�T8���1�=�f	T!�������O8@Q��G(x��o
��f��8�B���rQ���LFd��*e�R�6�|��h46��X��������;*��L��]���j��1���}e`�4V6P�$5B��L�u&rC�}���p@=
;!>�yGs��C�M�hEp���N�������FA	�����!����-(����&�v*��vt�m�"Me
��dd�@�Uk����������������z	��1�����|Qe�/�A�nj�v��	�":t��f> 0���B�C��K����9�Yr�� b������`�"<�*���d����T�� ��6!1��{�������HyEj���,�����F�|������"Z���]��-^�|/��r��+��;��R��0�H�!��]��	\?
���Q�R9�n�����i)3��S1����
����}�4�W-��r�mJ	~dw���R����^�s8����raw��&����p�������x���������g��Wg?��N����Z'��R71DD �G��j}��<������l�P�H����v��M���%.#:���� �*�z�������sv~������g>d
�����'��m�!lxK��f�w������t�#[��s5����������.\H6���uj�	HO`�!���A�w�c�~���F���I����C\��H@�:���[Z��3F�����/�sa�' %mq=5�N�T�/{����sz��Cp[���r���m����$��4�P������1Ee��`,}���+��S�lb�����h��O���FT�X	����}a��s���F��"�A`�S~!�u���v��%�kjp�8D��S�Sw ���@7%���|
�
����:9������
Pp���1�:�a������){��
t��,��!<	���s�[D�`H�Mn$UF���m�M�&���h���N����e;��m�?�%��g��1�y����(��6Hxr\��V�A������d�E����s����qJ������_h�
T+��9I���`�(-���0�>�8zI�������zJ��:���	 j���5<;�)4E@�FO����q��!�)�al�G��"Z@�u��]`d:��5�����.��dvS�����B��(���c�F��r�0rC�!tP���H�
�f���O��bg�x>��3��?G�>E��p�	��
?:}ZC RY�F��uB��%��}R/�����Np(�%����KD&Q�><�5�����>hs�6A���P���-�%(������s	;���GHE!g�a%�@_Eq,(j�!�k��N tA��>$����4���a�����	�?,�+#��D��`2Ebj9�O�xA�{3
�2_�G���yr[	a�{L9������>�.���)��s��hhu�Kh��V��A���(��>��6\�3e��>[q�)1����q@c���W�����yA��A�?�
:�6�Y��y�Y�*��Y5;��R�l�����q�������|��ll+9X�T�(�Y�vR+��$�hK��-�JD��{�s���)H�_��)Q�|f�p������� ����������:Fj$��Dq���IXQ]�H
��B�.�B�;�3�v���{��:�86�-�8NUh���i	G4q�p�����#�������;�����������!�-&��o?�lH�8�Z�����kt��h�����2�Gf�h��w�����B�"�c�uR�*�I��������76�� ���]x-����GZZ�����F_(�Z�SS�sBH�tX/&:B������1\)R������%Lud�G6�����D#����"��������Bm"�'I��3��������u��M��Z���H)�i�	#^��|�R�#�t=�Lv�����Pe��a��0��=��B8�(�-�0��)�Q#_%Q��x�� �,-����k!(T%=��{�QV?&���&+G���?T�_��J6�g����	h�G@>��������X�;�w�h������#��^�����G	~[�.�oS�b�V7j�A`���!����;�5��:���s!��#{ "����>xrxJ��R�A����ou��eN9&AY��Xc��:��d�*���;���[����jhq��tL���QTL�C�~qJ9Z����_>|x~�[JmX�f��u9!�����z�W�t���j/�]/���Rj�,9�aR�E�1��0f�17�E*��Z��Iy/����&r���U R���P��	�!���{���Lh�r����^��J%��t;�F�52ZZ�FFY\�Z@�W
���=$x
����/��=��R���@S�T�g��-�%�dV�^��%(��j��%��O��M+�c����<�*�.�O0*����B����d�N$PmG6,p��m����&�k����p��qpX�*�=F)��iG���RK@UX�1�A��@���F�A��'E���y}2�f�4�j��z������u�R�����*��QC*rf"P����*��#`��H��d�V#�P�����V2�H�K)U��1�� �
�}�!"�2T�S��������5&X)h�NA�rF�H\ 'DJ���/#^A�qI�pRt�)���> 	��H�������M�����K��SwQ�)}fv�.�+�����|��$��!Y�R��\FOS^Af��DE�_�SU|�T+eNZ��8�>�.��$�������0\EsV����e�7�,d�~����P+��
klL�j��Zn���vn�5�=[��n:oQ*'���(
��j���������/n)��-�����p�5q��0Z������*�TE�����N4�K�YeW���������3�����%mp��c�+���!�������&�x+��n��9�6 ���72'�����R-`���5���A$g�+����6!�U�AA��d3x�ac4����S�,[��m��Z6���cZ��pF
t����i ����E��-0�4:��L���#��H�s�j�?�M`��RyU�%)��%�S���48�4YR��,5����
�O�Y��+��;P���-7�T��
����8��V��pm���xBW7p����Yp�E���&����>��U{Y��D5�7(z�:AW�0!��L~�]em�������i��x@�]!`��m�2}��H(�s�a�vrD�x8�\�A������f��m�O�s)����!��e� �(�Ca�i{�t�+|H*�W��1���3H��E��-�����]��Ap��k�qP����gV�IE���(<ff���c���bB	�=���N�ck���j�c��3�f��`�z�[���0��7��-���t��I:O&ok`�e�A��I��	�'<8XSm �h�x�t�\�7&�����4a_5W�Hm!�i����~��J�j�PE�Y_!%���0�����[����'H���\��J���B��z����3�\J�V(~�!Pz}���C�ae�A��
����	x4�]`	��_A�����a��5qixo[u�}���Oh������5��G�W��<[����YJC���R�w�����
�4�}T�a2S��i�x������i����+T�k{ll+ `,��P2w
� ~>�G���9�kL?�f�8Eu&�O�A��Y�'Z,R2/Ii��D44��`�sNV�0r/���
�i^��N��Q."������9�	�@��2:��$�F�a��R����Q���T@}�h����}4K�0N��TbOK7�u���O�t�@�1��$�����������q��6��������9�����Uj�q�8�������������D���n�wZ�����gj#�}��o���:�����%	����KE}���{���g��(���Df��A���^d������4zzp8��F�5�( \^_���m�Z@h�]3�;�^z�LoY���HA�-����0gNJ�hx1�8&e?T)2��7�ok}��������L�k@@�5M6�x}	;�PpX��Zp9~�8r���a���
������������
�AA��:$�XV#���>�����HkUGN��_Q!_/�o-�X#��������S��X6V�����r|�����H��u�}�W��:���>�����X��@��i|��;���<Eh��3����}
6i�Z�����>fOg�~�����TV�����a���?��]���1�s"��D�G2�6������7��p!63�v�B��e��j�:S�	���z�V_��A����bP6kj`�=+��v���]��)r�����yD/�b<XS7��/@��8U`��*j��D>i�
fX����V�ZU��h���\��
���m	�s�N��7�����~h�Xb�)��+�����
��..�B���B�G���TV�O��	3`�H���7b�~FUj�<@J-b�,e��8-��A�##�(fF�
(V���O�6���c� Du������r[A�1�|��pz:O_���Wk]b����MP�b��-�E�z
��oU3�Q��?��d[D��tf���I��EM��z��e���9g��%O*�*��lh"a����6d���U��d5^J�B~��1)�S!�$���u�x0�i�:#��;�9���5�#O��V�?Zv'�U[xK<nZ�Gf�/)��L')�VaJ�/�(�e�������xM	E� ��I2C��M�!���������
��-L���X�fO��\v��{�~��0%y"C�n	P�D=��$9�4��dl�OLYZb'�*eNb�(� >�1������h�WT�a�P ���L&^��4��&��H�������Z�Q`�7�����3�;�4�GMDF����En��>�������9�.#��:&A.3����:%�):���Od�L�����P�k��|#���q#.��Lw�'qv�������'�p���^��h�ah�e������!3������B��R�Xh��X��'#R=zA>:rTPPNX �Q1;�	1>�=��m��1��9��A�blG98�L9&��)�G,j�I�~
x�_�'�ZW��[!�h��rl9��+�,2��G���uk���g-FX��y����Y���55����w��G8�T��i@�-O��[V�{Ab����G-s�c��H����O
�xg����O���71��pg$�g��0���)���i�����
�U��$l�{a��	��y�� ��|�m��0�B*�����V�3����8��rLO,��{�.����7�5{��z�"^�O����K�?���:��80�bx��!����{"�yA�A�>5���5�w����0�I���?��]��`�~L��-��*�GU�A�h_����VKS�H�2����4B8�����}T�lc��Q�(k������A��UL����|��VJ%�A��`�?�)�3w�zG%�
�;lEG+�����%3o7Y�&T��S�fQ�\s���*�l`����*���0d1���
��%�.��S�h�9C���s
�"�t���a��'S��Y�!����>�v���$=UT)���%F����zEdW��Gh2�����`@n[�[���]������~�}��Ji�{��]��}z��
X�+���sq
l��	����+��GJ����7�����zs�������bU�a��������{���n�R��%���@�Z '���o��)mHC�.qb�	9M\C�~N^.N)�f������n"��o�V�s��<�nj�TW����8	�fT*��Q�S���^t��t�p��}pB����$W0�����2
��1�"�-8%�(����G{�sl��l�4l�W����
�l�����^������m�i7���fBET8H��{������D�b��3sZ��M��G"�c�7�S�DN��	{3�Nti�cS���HCJ���������MQ�T��gBL������-T��S�2|��'D��5��m[<��8����8��\�t�� y������M���9��ZS2�Q�f9�Te�3/�[����S6��1eh&����kN���������HF��\��J�>����/t������#c�m�p�	U��
���2'#���P�jk�k�w�����������[��Z�c��J;Y����d�)�V������:nIa]���aO��u�%��#�.�V��[E�
�>@�}�h�r����=�
��TZ��J����#PK����@q����	N9A�����N�Ri5��y���,��-K�K��{��i^����#0��_�+���4�#r����*��&>�n�i~4�8n�5�(�u���-s����.{!���@=�6A�R�:�F�����b$��#`�6���{<0�m�K����	5jHt7 jHU�_�#\�!�CR�)GA��|2�X�/��i�"z��a��F�~����x�'sJ�a������
X�HE�(��x�3���� ���Wv�E\��	8�59/t��uAw�?l4_��a������vq>���2L�7(2�Y`�$���QLM��v
���VVx���[4}���85��V>[��e	�4�
����U��
�Q���'G(-��=2mHz6��������&��
�������H#Tq�	4���E��Y67���-�������J��rcW����'�(���i��.1������K_Sr^I&JDB2X�� ��VG������6
������p�&�c&u���	�����\Yj�\9�x?2�<���uIzwO[�q�'_g��{>()�=#�8w��\K3>.���������aL:�����s�;��������U�
]������}6������B�����[��D���b���$�Z��ZY�4\�%xtrg�V��������I�Q����a����
-��tQA�"M�M�e����N8�U�-������&�;���6�o�xA�X�x��}���0�t��<7E���T!VS*�+���-f���s�~U��[��?n�Lo1����{���|�&�N$�f	�f���M�:e�D}���7�)�0m��O�qP�O��;���h�E�5�?9{Q��BN�*"��8�XH�'5A��-@�0����C����S��'M<��r�;a���[���������CS��������8�e,�N��,�-X���bC��}����F�<<�@���������#W�'�C�
���h�����8�8���5�H��R�����Fd�L<Y2S�BG �DQ�������]���kOP2��A���f�0t~�B�A��	���`�FC�ctK4B�GOM����W�L��ee�F�W$��<�	g66��0�
�tF%�T{wAg���0z��0�N�eI��AO������Lx*��wi��Y�{E�C��R.��r����`J�I^��z�S��_{�n��-�f�e�x��[��kl�l:�k��}��C��p�e�
�rA������U���7�L76`C����L���H`�"�lDi��X&��C�\&��B��p��z�~�-�=�P�X����t3�`U{l���B1���\�_2CxK)�q����*�K���!��:Q�~5���	1�F+���.'E���a���l<Mf�c���������H������F�Y$SJHD�hwq	|���2L���������Lg���g��Fu�:�`�B��K����%F���g��&f�J�KCA��z[A�0w��81j���>�c�@�W���M������P�7
��8b�^�0U��~�*y����"C�w3���������o��-��2)�k��Dzfaq�����l�.J�J��<ur�������n\��Sr�O�Tvb�W[2Qf���K��%����q�$�j��K�g�`Tk�X���d0������rS��aT������U������a
l�l�:ID�f�����L�J�0����/�D�!9q��X���ON4���.��X�����c[����M��Mne�9����Q�`&�1�x���Y(������L&�����F�[��!�-H�7f�5M��(�)����%�*i��>�#H'��l�X���NC	P��
C|�6����k�d��b1�f��:J�����_2��LR{yc&qG��il���X��."�zJy.��9.��v�}UI#�
���Q�C��|"��O��F�8���*���&������=>�m��� Oc�I�LN�W{�K3V)��vS��~�����QJ
jYD]��
2;DC&(�
BX���u8�h�n=I���g_?.�l�G�I���Z�	'^`�;r	�����?�g&^����$��)�i9��d�-X�F���4{�E
kC��NR�8~K�-���a������O'-�8�$�E�@JihG�k5�JA�/��a�qh�EWXM+���P&!AK��u�R�P�L�3�:���!f�f����"m0*�)�0�^�@�,���M42E}59�Sz5���*���\:�!�SX���|��Xp���!�D=E�"LT	),r!����S�$��cV�7��,rS�|c�/F�S����y?��Df!'/j ������'7�������9��[�I:r�J���h�
i�����!�����*RUc��2d�M'��uN�r��GN+����Ec��������QD?>�������{f
�~�����T��`�������0��K@n�����5~��o9p��� Nu1�P�dwv~���������1q&��Q��������
X��3�X�2����8�e����%8_j���M��%6�l�5Pm"����6�0
�;o�?D���$M-
�������7#NI�8�HJ�:5_������$u��'���}>�|������'2pR���pI-�������
5�(i��h:F]��G�%2k��v!����^��
�f����pL�Qt��y���t��
P���+��D�-����%j��A�\�3{-IJ�k�L@��}REc�Z�H$�X�
�_��"����u�(�lY�J���V_�qjI�1l�T����X��
����� �yr\
j� �7�P<i31z/�p�s�-d7��d$Z��Q��;����%��D�iC����%��X���I:m�vP5�V.K���=���#��|����A>5v�-L�e+P!������_�ZJr7p���0���tu�/������L��R.��##���*)�}Q5tJ���A�h�OO:�,
C��8���[qs�T�hM2�>�T5lU���k�^Az��(D({#c�e�$�����J����)������9�w���D�,"�)G|3��v�c�@U���ho�I�an.��Z�z���0�C��������z�;�P$j$	�]�GO����
���AH�&}���Z��F)�A�����T"7":9��$����%(�w�De�wL��O�_���D���+�0l�L����.�C_�)��L���aX<��;���z^�	C�d*� ��V�o��7)��rQy�:���5[O�����������R4a:���v��g|A\��W��]QB��
��F��8�)�1��c'�p�����?`NJzl<2�&���k/�e�W���j��'
$���J����������7xa�������:�f�(�k<��"2��[-/F���kD,w�>7
	�f<|����x6�*���62W^M7�&���{-]�mWT�qOU�@����qz�!�Y@���+Hp�bu�G�d$�����}�����"B�'v���|c��k��im�s���BLb�t�^�{wv�� O�����}�G=���)�-B��"��)���=q����t����b��{������J�a�C���� �`P{�������������|m������O���Uh2�P0,
�����E�S�����C�R�(���|s������	\l��������k�x-YA^���C�)5&c{���z c��1�B��T�<p�K((0��d�$xB�$%�1�T�`8�����X�����B#/�����b��%�����t0��g��������V�0q�O�G�_��~����$��1�x����!���8�$s��&��Rj ��
�\rTAjKx6`@0~���\��0XW��H��1����{�v���%f�oE���B�>;�.��P,�*r�iCK����@��l<$t�#���������bt�&�l$��?��,�f*�h���� ���E.��Bn@	6�afy��Jf�z��S����D
����6�
[�*���:& <;=��`���LQ�^Io8��,��2!Yq�;���V����C\fCW����u�DOb����h�_���!jP�d�'���4�R&��0R��vY�B�D^W�PM��QQ�����D�V�"h�S�g���$�|a�<M�x�7NmU� ��&gh���s��i�*Y�����o��o/H�n|A��L������n=Y#�Q�$R���M�$b~�rI�/(�4:�ch��O���1�/�v-Zk&�?9d���A�-�Cucp�'.{hO9�#5�z�@n>u���N��D�������f���������y�����4����:����F�p>�	��)�I%����6�2��'�����GP��Y��)�N��BG��Pf�s������3������������^v��bO�>$:��	x}�����������@����a!!M^����|�seX�Gf��y����y'��}-B6Q���V*�a�;9:I��h �!QU�5T���e/)�+|W��W-�����O�w��������Zo�V1�1���,J�������=��>����������e���_���������w?���]���W�g[��xR��lDB;Y�v�j��8��������U�J���`R���o�����������/%QBp�O������+�b��^�(�0��Jp|�k�GI�Hm@�Dj�U=��e��V�����'M�+�=v>C�^�9P���]��X�T{IY����R�",��T��}Ac���Ox�d���\C���BP���LM�AY�7��s{�i.'��]p]������Z�l�Lx�9���y�q���4�nQv�e�I�,k�3�8p�	I\�f�2f�����ph�Z�j�����(8�P���������d��q��C�m
(�%�e:�>����r���'��F��xM��S���������>��5���~
���f��cF!�����v��r���6�r��<��3
u#~��d���)��goJ����1�������?��4>�h��/>�������W����	5!�7��/8�������P�).���sO�vL��7�-���Q�P'X��XG�J����q��i��P6�DU���1Dk{���Z�Uev���'��=�w�\����B�mmE�;���`�G�L���Z
���������������������N��vP���F������\�Ir�&_�0�7��8�o8�(����p���dp�
?2F����=��
!d�{I*����*����{q1i�f`��N���"O�.P?*��+J�
�����a���a�1�a�E
j�]2�1Dy K@e��"��r��R��B� ZeZ���-��W�b����%p��}�Q�5wj/����hZ&��tI��8 !��.��i��TZS���Kk���yX�E�����a�$�������f�x>�m�	�h:��B�N��i?�_�[H�����#�u�[N�����L����G�-�qd��a�T}����.Q�;��@�3���t������,����d:�q*Hu�tQ�(�W*��a����g��F���(�A�N��&;��c?��2�ed���6���������*��������^	��\�����i�f���
�HEy#�e�1�0�M�c�����1�'��i���?�{����3�9Q2^���B�FT=lQ���Q5�"c�c�������tl !����Pn������h�����0��BA�'C���H�"��D�i�g�p"r^/pI�p��cV����y��gSjwZ$������y���=��8���������Wg?���=�n��_A(r�1-��P��H�nx��%�aU*o>1�@u$,�d�����Q7]E��<nM�?��O@�|Ar���x��c���3�[��p�q+\�w����t�}�<��}i�%=����'���trmu3k�V4K�m���a�o�Fx��I/������x�Etl�.��3�I��I�D��q��/���[��8���A��?G\��E�X��<A�NT���h�����������y�����
�M�|-��]
�F�r|x|\��r�5��oT��f�����u�����yHQ2���bdn���P8������f��0!A0�Oh&qa�y���.�*�gRpd���'��
��O�:�I|�G��
����%������d��zG=���)�A9
cX�j)���B��8D�x��f����84���G2/��V��[w��x)��3��/S���F��>���Y��mz�1�� �����q����)����bX�����j�j�� >������B������Ucg��@����'	���g��.����J���C��"X~a��wr(�;����T�����~0O|���7�[��x:S�u���.s�k�����V�^���Ur����TF������-��^<_	{	SK�uJ���-a-�������J�z����FlL�����,Q��V0:["��
���(��w<��M0���a������A�����p<���it7x0�OJ8>�G3:lH����G3��x�
�+����Y��	V�U�*'�f�����6>�zG��j��(���ht]���+Xl�>��sx����Q�j~W��j����/�z�^=l��?j�Z�V��D�>s����_��2����I.zO??����Ga�8���ZU�����k�v����Z����^�����E4a�#V�>�����f�y�N��!G���Y�
��AL����V�]���/
��1��]�9�q�~���1�t���y��^e�f�Z�����)���N/_��j�'��_���%#d-��
l�1�hp��	e9B+�1r9��[�k��W:��N��E��������P(��3��Z���
�B���~b����U����:)5��v%�@���5��WL�[����5�!T�5���K��������;D���?;({j�x�����5�p������"?k��!�G�:� )������1P��K��.�Q��I�?~�u#�1,�|
�'Q���%��d�&�k��S;�T�����p��%�$vi�J��q!S����u:}���f������/���z��0N��C�\A��b
h$q���y�����z�-{���Azf��;���rqv�A4>e�������3vy���3��Lg5VA4Pbm��@,$���}���������w?��}o#�	o����?�
n���.��/��w��uCgo�^^�~	����pZ�?iVK���oq�������j�7Q\�%��	�g����k�@���n_L����g�z'�:����{���5/�����/g�E������i0z�����s�63M���w��3b�{$����Q�i4�t�Q��/�/^�kx0.`��=@l������`��8}w����O,����"�(�!O�����h�
�B."�A��M�M�i}�^4
6���zhx��3��h�`�p�&~��$
���T7�����Q$������|xs��+�|qy�'��>����Z%,��J�4�W:4A�����'8����C�� c��N!^�N�������o?���ag��Y�d�����+^���Z)(���B���y�����g�����s�Og8e���Wt�������LH�
���E2���r&t.M���bq2o����#�5h��Uw'?��KP��M�'X4�`�xO�HJw�C����oG���oAWEf��3�?�5����Gu�(��|���U�A��G�"

~6��u��qA;�r�A����(�?�%��}�i�}�Q�����
S���I������ k��=��j�����n}c�;���8�����r,o[[I�G0�]��P�{�w�D�y��u�c���9h��
@��aJ@9X��<n2��;Y�m��v�6�a����>%��� ���&Xt�m��|�[��u{�������A�����]�����hE?����*�pu��[���!�M,a=��Z+�p�a	�A����*�M�=�@�}������u��Kx��S���0 ��7�������h[
������w��iQ�&�i��&qiR��;?��=�!��Rd����j�z�n7���aXm6�����eH�������v�q����9AhAz�����9�:�O�>�FWh�XXNVl���F!����AL|/����A������W(e��Q��P�6��:�����RvKY����Q������T�S��j���!���}z��B_Og��TKU�������C$S��`�G�I�!%��2XS�Q�$_m��}�D5+S�"��u������B��+8����pD�,bk����^Hk�[��il���)�y��(��~�/i�L�(���:���j�&Z���j��rAz��Y�N�$=$�~JkS�^�u!��o%�^�(L;I��������it�u���;@���?,�Z��\�^�n��S9c��9.I��CW�+� uC-���
n�6�e^*E��-�]E��.����������(�y9u�j��7��(�\D���Xn>oQh7E�iI��3E�?��:�-�>��++�;���)j� �dEZ�A�4����	��m-��u������}�4�bP�K�oc[���#�$��U�9H��
!�}��?�K�1>�/�c��Hiu.�I�'�&��	����I�rh�VW�e��C�u��^uvX�@�7;?d��O�#V7����G���V<t���$Z$�,���k&��|;��>��@�rx�
�y6h�A�����3g���j	��������/�����	E�Y�m*jMj)�5��j�k)����������E�-Y���U���2�h���}R���b!
����4u�&L
5e��O8�I�����&�EB�x>�E��:]9��H�U������S���� 
\J��%��X0�w�m�F�Kki���3�M��4�����]�1|���_GM�����	|���7u�SO�iR�&�)[�Fo>l��p�Z���6��A:���b�.�:���������b�;(%g7�f"
�sH�~��X(`P�C6�z.1TK��K����S9%)'�J�	�x���{���W2�I��
�����~�I�3<����e�cG��o)&���Uq! KQ�K�V����h�����*��N���T�n��X
�����I��v�&i�m��k{�j���V_b��ZY�l��^v��mM!�/p!1�\�x��^�
��CbN*����9�&��J+�=��S)�Wb����|O�K� �K����g��	S������X��s5(<v�������	�$	����`	?P!��{!\_����Iq��zv��qi�����C��[_]���_4��/O��	����� ��`�\�-������w�J����>+\�����}r��V�nA8�< ���S�����o���/��W`�2n1w��d�(����[J����5o2�����t��z�`����5���(e��,����2ET�~���e�	�JYb*e?j&Eim�"��R���J��<����l�CZ)��%����,�Y���"��\�R���?���d:�p#���]DW	�n�8�{\�	�_ f�c:*?w-4�C�� ]�Nn�M0CB�����%��E�/3�Aj��,��j+	qS�2�2|Kn!��q���q+Jd��u"~�z3�����(����
d
xV��
����i����.k�x:I�]���(*_����3�������%v���?�n�|���t�!-�m�M���<fU�zz�����
����4�Bq�%]�,O}�GO�d��u��)�"h]RorC���+�?V9�2v��?���4�5\x��by�.�(����z���<|IF����5
�����<<3�;�x���U#�{��/[���J7�H7���1l���h�/��n|l����H�C/i�3e�xE���u�.{��*h0y�\�Fn�Zk	����%T�^w>��-g(�W�4i��Kf��l�q`P��\��0�p}-0`��%���	{�������6�c�4O�\\O��w�/�����Z*5$���q^��fK��KX'o��62�=wp���<k�,�!�N4t�)����������s
=�����U7�.�u7����5�o,����vfY2��s�k�������N��^�o�>B?��� 
�n����hxF������!��r����L�q\�=-��e�q�Hp����88�5��'W��~��9q�HW��KW��2+�������6R{��v��{y�H��_�M�lq{Ye�d�� +\B�D�
�/p(]{	�����I��[[�5�����2r3B;�N������������I'0�b=��|Ew��
�����/���!y��&�C�K1������Z���9��L��8���#��[�\�cs9?�C�U1S^�����Y)��csyK�C�V1�����e��h>��xY�,������������u�� �;Q���@��
EH�]��Y���{�Pu��|�����Y���J�x%Z$gNr��.'}����_c�����
u7��������>jJ\����������m"b�"r��6�p�H�����������D��%=��c��]L�l0v<��D�0D����G���@ -A ��~k�����h���������`�p;���,<�����(4��	(�kkAHh���`����@���@~������<��n��]����a����6��0
{?��cO��oK���{��������c�)n����Y�a��J�����ss!	�$��y$-��l�Y�>��v�tb�tF��.��+G�{��sV�@�Os���
F�n7�xQ�=�4�	��Hk�l0�x$X�������%(�E���e��#��&�_%&�-N�I��V�4�����F��G���	:@��T��������zL6���a�{a���v��)����-���|��
J����u�+D���bseJ^-�W��B�/%����<��������+�b��#���}���B��.V
��	�i����� �$��%��ILI��ZW�9"��z��Z	W����]��x
�|����A\I��K�F=�B�J��7N�.��)����?(S�� ��{!y&�gy6YVj�$�,{����eO�k���[�bRB.�m�}��Xt#<����<��U������I�m��[����ke�o����o������,�#��\r������`��!��e)N�l5�M
���&��l�(��K����Y&����cIB�>�G�$i�������$���)���,��bC��$mI�T���D�.�;A��(���PnW
o���[~����d��[~5����c��J�`i*5E.����u���k~���Q�iq�U���ohx��X*�(�kB��=e�����������D��(�e
����<������>W 3�t��Y���:<��K8f���no�W���W"7��9���gcK�ZB.ei��}�����T��A��e�!G4?��s)�[�v����p/?�����;����M��
����F���\�����cjE��r3���T���N����<��Z�)H�,c���l�>/���nL<�*JL���(
��&������"kH���������`��
O������d5��&�=m5��8�8��cOmuu�
�Xfb�%���	nVk�z��z`!�BB������A*��t�u\��fE��l c<����0���
�j
\�;�^��o��������������q=g&����p\����"J�6���b���s?��wE����������@�2������p`:u2��Pp�2�|G9g�d�]���w�X�Z���(m��� $Z�U�Z�S�(�dQ�������
�����4�
gQ
�~���9:hj����eR����hX�NI���
tu����K"��4"��O8�{�����rZ�r�]�[�?��	\�+���>S�3�5�_,8����7��\^<����pqq���q��
\���������3Eg�]S�E���
���$]����$����b#�\��m�?;�z��F9�e�v1���3�@A�[n+Daz0z��h����<����+k��Qn��������~+�a6���5�5o%tjht:��t�:� �6c���y�����M5�B� �VB��F�c�P�� T:j3VD|���PkPk��o�I��v����q��Z��f|�a4��0�D�d�KZ���)@�z���#^s�{��WM��"�������j�\
�G�|����o��?��4?��������n���x�����1%��o��I/ rs����L�F�Q��F3c)�J���B+�����A�
j5�����V6����\����kP�C�4^>��f��h�j��!#�C�l�77�$j�����Q�s�L�����i9��7�_�Y�*v6��R�M���9����K7f�-8)����\m�����Xic�)����:w;q���
vT�V����M���,��7a���W�=.�61?���xS���dqx�����f���MK�R��n����N�������uXMp��w�}%�%���&��Vo�����,�������L���jIgU����Z������W��V*�l�k�p�	�/�/�[��=?��?�9����YZ��!����A�S��T-���m�%�0[����e���	�d��K��2�gm������7J,5�������jG��(��L.�
er����D�\:#O
��OJ�]�`���K�m1�������{������Fe�F)�3�L����p�����b��?2�^�Y���-��_99K�g]_q��+�q�����R����<ia�=�,�^��/(;!>�+�,r��w`�S[-��B-�������v!�W����U�8@,��@,��K}e��Z@���e�@�;@�[@���n��z})���J��l	m�}y0��+i�)y<���Kg������Jg�����
Jgx�������9t��V���� �7��}S�:�d�&�iB�b�0���b�O���,�g�Q\�q;��	�y}�9������N������:�Q��,��"'�W]|7���IH�.���nM�����>Ik���S�=���c1����-���'@�����V=�\���`������r����Rl`�\����e�m�0K�`��������?,�����#��a ��	d��i��==y?S�^v�z������6u��r�����~���|W->���5�bc�83�\~��k|�D�+�4��U��r�d����13��ae�!rIB(h�p�������_>����$*�7���e��m���&bB�M�|��l`8d|�FD�J��VH�yIo����=����|H�L��q��S<"���
��|b�eI�������_�#�VZ�YYoy������j�E���M�mr��Y!�vG*�WB�9ho	�3�l	rg1�g�!$���b=.�+!��5~_�8��c}����g���nZ����J��@�)�Y�����:���##GM��Zj�5�m�AR�7��Ik;�_�6����)�Y=���������?L�m7��
������fj�M�m����w�cv��������q��w=O�R�>�_�6x�4�l���qj��������)��Ij�'��>��M?���im���S����l�����/k5�m�������}����
�=��� �Q���M��O��E���fu��0'RiG�s����q�'y�,���38����|�FJ��qd����jX�PO���������m"���D�`�M�������c�����������|���G���X*���%�)��r&��{�K�f�.��������VAu�_{��L�Lj�
��"	w SH���h9�5�r[���V`$�'�G�U��E�>CCo���h�Y�&E�Tsu��F�9��k����2l�o�7ZN�B���"�/�]���t��B5��y}%���X}�e�������\`�����v��q��L��4�Li��,��],a�����K(�}�h����;H�ej�o�������zU�����\!o[���gEkl)����0M����1bp��3��X8��Q���y���i8��b���z�t3�#L��<��d)��U���m��'c�L�A�Z�.tq��]����Ca,G��=����+����+���j��i�$�HIC=�,,u����U�����JF�Y�|��{���Jh�H���G1)W�_%�F��]~N������{yo�g���[anr�z�.�1����+�[���n�)���Y���R_��46�l�3�E#�����ka~� m����r��j#�kz������#X=u2g��,}j�\�ku�'+oZ�"@c9���������q�s���o�p�����o�[oC��Ji:�����t�da������.�]���o��.������E�w���6�k�NL���3�LRC���6c��Ka<�����x����
!�o{9�
�&�h�Nv����#b~�8��Y�����!�8iGwZ�������?v�W�crr1r;��u$��F3�!�Y�o�Kb���E����n�����\R����l�G)��VX�s�d326Z�k,���g�z[O���x�6�0�>�D��t=e�.j,t��z_d�xm�������a9<�3'��e���SQ�%�6���K�,b��@���D� <�7�_-�����v���0�<��&��\"�
R/+���B:��Y�|����$�v��}��it�O8��A4h����;D���P�����~x�*��NF��>�h�
���ppp�������k�z���U����=�w����v���'��������-v��+t�����\��
ej>�Qg����>�l2���Ap��3�_�Z�}�1'�X7���F�v5��v6'�I���a{���=G`(����y�c+�pr�k�f_�^�����#�x�1����xz��;�?c>]�����x��.J�F��F<�����Sg��d��&���t�������I%�m@�c�xg�����>_�aj�������y��L�~i���y����"bo5���QtR���I�W]���FRv�[��q�T���5��&� ��+(��>qH�f���U4��Y8S�,���6xbW�����[�0�>�^�v8�FFAa�^��A;�5tO���'#~��B���Z�r0� t�_�����EF�L|ti9U}B��^PsX�����J�o������L���*t+Vv%=��a�1��9x~��T���%����|��6��}A�x��Xz��YF:�����j��^�{u�?���P�rv:��rv�5*g�)"�$,��[��8�>;�eW������b�v<��6�J��l���J%4�P�r�
1���	�r�i�8���n@\���g�}���F�Q@i=��J$�|=DeN^sREd��l4�Co�@��0[�2E1�����K���7�!�!�Nu�I��?���fDg���Fa�W��N���%�.�V�{HP��F1?��Y�mMo�.���W������F`�����1PM�ch���Qw��mV���Sie�5��K�O�{�)K���R<���8q'��c�N�|���E���r�H�rK��>���*���t|��qb����x@�hA9.�}�=�kT�/_`��;�~��v�#qH��s�%!�,�cM����x���s?/=�Mo�><} �g9	�~�I}D.NyW�����oP��_���y
���j��<G�r������9���������
���O�����^:� 8���a�����:�1��x��h	
�fV�h<0Y"����E��;��&�+���g���o>�u���4e��8Dq'*���������y�	����N�Ar���,H��	�;u�����{�����H��
����)j"��&%������@~������<��n��]��	������J��u{�'X?�x[Z,������ER�����(Z��x�S"M;���s����n}��4CG��
fw����Z�{�����pwg�w���*�?���YC[yvow�L��x�����)l�,�r`����!���%��4rE��r���HJZ�\����L}.��B?,�-�����5�D^�6n")$"���})�vGw"K��Ku�T�u�
$�m9���S�1�;F����d
a�z���3���#�_>�����5w/&�|j����������qz�?e���{v�B��T��B�]���t��t�	C)C�W1����g�r���,8�0��C^�;�K���bG2S<���
rS<d�����(?�]�6�������!�����_KX���E��}j���I�Y(�E}& [��'�z9.�����k���|j�����W+k��
����\���
��5��7�+j����k��a3��y��5������n�3�g�������|6�Sg���=�����z�(�N��;<�����Q5j5�����a7��^����S�"����V���Y������T���r�`�K�U�����D���)G�n��_^q��)�����!����7���5���O�MvPm����y��Qg���������Y��I��W���}�lM�������=����s%J�
g����)�Mc��{���N���������s�-dE��}�-<R�/a�������
��|��+}x7�f�C��ll8�*'{c(�Q� �;��,�'�d�3I�xa����ALG�a/\�?Y���w�SP<�"����p>���:���a#�JT1F2�,�K�d��kN����8F���!�w�mT�k������e����#!����8"��������/w��rm�a��
���U�s���XG;�*\�aBY��hFv��<���`w>��"]1��;�]�X<��P0�!��D���	�\�B�tF���N����w�!)WI_�|&�i
~�2�~��Nd�	����EG��M��r�U;
�Y�
�P*�G�N�	�mD��7�
5G�����o0���1U-`���������}�����S��#��c�b���!_�B&�In_^|�i}D�(����XN���9��u<m�9z4+��K.��-�������2���I��"����F`��;�K���/���C��!CV�q�@C�����yz	��8�e��f��}1T
=�"���}�I'�q�_����t���;j������x2���My���'0f>�[���#l/��|�<�Fc��1V?f�J����g�������F���P��)F������@<vK������).G�`���'�q8���~!	3��E%D���I��Z�*~w��v2LhV:fV)�8)qx�WpQB!�g��h=0b~�y��'���6�(����qx5}=���cx�Q���
F;�V�\��(������x�=��?@ylk��_���q��;�����i�����1u#>��#���������|5��|��2�"�����7��{g]�6Jg��VBH������R`T�H9�7:�f��{l����}�g�2�Z��'�A�
dV��pAdSbIyq`0�����N���6W�*����jvE*��R���A����rve\a,��0���o����
g���x�gV�V��_����|0�*�p�J���4B=@\��a�����ff����<V	�*����;8��P�a�����!K9#i?��'t�Wf�����x�>|I�����R*Y�QX�=y\`�1;���
o^E1�G���	;8��j5�#:�Q�ck}v��p�1��� -��D�Wuv�K���������pc*���=��ApY�W��$�o��������h�,�a�svW~�9�d[����RK$�Z���1��(F���Wp�������2H>�����
��<�6KG�cX��7t�}T���F��t�c>�?@���=@$���GF
��?8gT���K�y�3�c�W��M �+��C�p0�8��p�������{����7��H/�D��s����i�������nsU6k���<G�`�p�/^����u\�g��^�Nc�v�rV��s#�������Oo�"���{��y}l���k�������T�������%>��T�_�n�:���m����^_�a
QEO�S4{
'L�G�5/�� x�����H�xr�Y��RF,]�[11y�{��������G!<���G��������Z���H����d����������A��ees$b.���D���,y�^upN��+!�SG��^JS�@TJ��.�����y�D��	�X�Q���K��1�r�Q��x�`B�Sa?�Q���� �Q���|���!o$"�(�/��d��L���c"S���H��9BC�&��� ��d>(���F]�)_G��j��|�RD������Y7��(�V���t�
Bb���FN��J��R%�#����UV��V��T�3*����
���N��m���<�iW>y�~����
����d?"���A+�xh�_�{��lz����q�>�����{&:Zp)���oM����,���sm����o���_z��.s�;%�n/�����a�p��
�,7�x����������v>�17�#GB�)����%�S-����L*mQ���D_�t� �����g�C��?�
�E���Y��w������-:�8����u��_��^Ji��d*��������w�����K� xhsX����1�3��-��/��� �R�R�H������e�}k�A�Vk]�Au:��N���S-`J��_S�2@��N(J�����b���{�p�_��&
o��$�*_���h�JoSS�M�s7b0��`n$�*�q�X��z���?��C�z�w���OA����C���;H>��� bE��0���;f�}!����&3����!���t�gAc3�J�XZ��,�8�X�l����C�6�4I;��t���d�������9���Y��#���T���6���X*�9���!F���E�����2���FX��>���.��\��5����h��k~8�9�A���-�@���,�o�>'�Q��7���!cJ�_De>��<����h�O���pH��
�H��-�����{�oyU���\��:^]�s��j�q���N��`?G��pp�!�|���^r���A/gj	��"b9:BOPtXs��W4`�u�5��j�-�6l��E�0EA�������q��[����$%�X�1�"'���1+�?-��h	\�EL@��i1!kk���#�������qS
�����)�������t��*��T�XF�F�@�m�`z`u�z��Q)�������<�����J��,+%�U)�6������d�|�����a��x>��&E�+�k�M�	g.���<*�%�*��B���	1�g�;	�TE��,%f��-��!�!]��v�T�����t.H��5����@(:�e�q$T�i~T�	���O�X�B�C ��-��%��������	��eTKs��s�����U2@��|���wg�O����xzq���S1IH�!�xJBl>�9������u2�B0��e���4
���,��)
�����c�>G�A��0G+RP#�3ElC5�y{�r��'�Ws]��GP����*X��l ��BV��1���W#�!%3�"Gq�w�cW��w��_5���7�|��� 0�)�jN9��va)�+����2?r��#(R�@��k����(��f:�e&��92��yL&����&��{7�o7t������ww���z����������b���Mn_�A���7�����~���U�2�o�g����[0%H�]��
"	�E|���d�fG��������z���)C*f�X8Q�����!8���&o�JC���P$�%�S>@�x2+�/K���oe�����sU���������q�Y�]'�U��kP�/���QS���:Pypm�E���x�]q"9��Z�kY��X�����>�)A��\��.����lr9��r���b�g�%�`3S�l!���-t��f-Y-{�|�y��|�%SM��dAr'�����>��(���R�q�@Za�Mfq�b�!�B�=��J�����5�����������]s� �A�g�v���s�(_�3���65�����*�C6��zI������$��(���R��U �����[	1��e�.�%J�p*��	
\�B�$��[��4��*&�>uV����'����r��4�d�F���l>UoML	d��K���h_�n'&^I=�����(�c��`)]�	 �p��Fk�4����i�#�(��
�����(TO������~��II�<o9=��wr8hP>O���oF�ng&
9�����d���T"I`f�n�������O)�eF���7�"�
��w��E���������3�p�R�
������l��a���q��i�F�Y�Z	$�>���R��-��+���2!%��p����
���c:c��ma	��r�a��wu���r(�O����5+�<86Jm����f3��X�v�IK����`��������W�U�%P��G�)�^PK�z�I���.3����yq���^xf�mi���*5�hCk}p+�A����Gw�z7MQ�g�[�,�rz�{{~.�y��o����y�5u�C0+`"����I��!),Y�J�!��M�u���;f��b�~
t�
���d�P6�~�T��MEe�` �"��������I	��kv5�SK�����y��,�1��o!��L�O��<6���C'���c�%G,�S��Hx��|�����($��� ��
jAC�C�.�����%Dz�&�����y�!9pt�X��U��g����m�gQ�5L�5r)?-��i���������l}�L��-�Z�%
)���k-[��Jkm�Z�Z�H�_���NJ��[j�j�TN����+���4j��R�k<�T�J��'8�K�o$���D2�la	���������1q%���%q�=��0�JO���dd�[��r��� ���J��bPC�����gl*�c��"HV!����������+YV��m���SL2�=�����E��'�m�Wm>�6�s{��
�U��_��������[���Q�6�KX�%��0�s�����,�EdK������m�zv����C��7����S�W�����l
m'�fV��lMVT�
R4��`�Q�L��������M�)6�#�c�L'�,+�xH��O�>Td=j���xKv����+��>�������]r�!;�YJ<2v��f>?�/oP��[8��7@X~���A��#��S
����������2�+]���8�����W�)�;��D��.���E���!g�/�%���I�e��{xJ����,�~x�C��MxWC����xf��%k�ZZ^PL����	�� �1f����D�A���uh��^�!��^��#!i��z�e8��h���q��>�<"��%�X���bA���-2i~�;|3G���?
������2R�)A�U�
��$��H&0,%\��QU�!3�5h��d��:��IF/�������D[��jX����b�k��\��+��]	�}��5,;s�20���}'H��$C�d@%C�/����7E��6O}'v�3��Z9 I�g���9�J:�����j��c��c�x��K�8fTsb�Y!q0�M�
����	�t��o�����vBJ��TD$cq\_L����q�g�~G,���*&�������Y��=Qds�3c48�F��#u6I�F6��q3-
%�B��X�qpyb)%�O�}�X!��B�#����o�4
;��AG����T��?0���L����~�b���@tP��m#��9�:�����]hT��#M	�R�=�
8�E���J��.����C�@����#�Nq
�,����mm��3rY�3t��;�^��-���0U��7=:p;_9�]m��^�R�6����Iz`�����q��b��z��t���V��y���q��!��C�O��Y��2ad�Z�I��"�/�J���*�p��m�7O+����?�n�����F���
e�/*�5L_|Z��n���u$����N�a�-e	!�G�L^�� y�����O	�O�p��0��o�W��{N�u�h��^�7`�<1���~4�0�d��(l%���F��8���6���	,p��������%�bB
������}���Q%�*wSU�[W���y.P��9�T�����"%���5HR�!�F-�&J���)�	8���O����,x�l�d��1^0�������c��q���
3���k1�C�g.*nF't��PF�!9��*"R��k!�,������Hj����?��M�
�����i=-�[5�,g�����p��S8�7����P�2���	���:#)Wu�a]��o��,������Ig$�x~�&��v)gm�b�Q4��W����/�����d�������Rc!|KE�eh��;�����}�����^����O���Jl���'���1{3������c����I������m��6��~7�/���j�"1���I����z�� �16.;�4A�4���G�`l���L_��'���m����[5-ku�W��lnPH=��\�F ��$``V�#�}����H�1r�}����K���Fl�9�{\�+�.��Owf��KM�k�F�%�������;��GM	�4�r��B{��
Nb�1R�`�\-*@,�����,�*��J�I�|�����3��5�w��s^5m<�I^eA�{��bp:�` �3�DX��.~}�Z/`�����=X<��:�������*��	�����%!kj�'x)I��:%:��Zu;,�)�D�+���j�o~����g~�GI���X�W��o��Q�1)5G�(Jlr��<�B���� OL����3�[�lQT^%%�e]/r�b��dq�����r�mI,n#h��68���<:��Jb��X�4�1�l�\%aV�����H�I��
�y��-)���{|����L�/	^�N?:-��^�t�����r���	��FK���/�����|��u6���9�$��<����!�8gc4��=Q�
>Ke8�F�P�Y.	���M?tc'��-Z��qr�H�?�,Q����X��|_\�{��?�����lE�L�	L�/	�����ODZ�����jZ����'��Tr��K�Rhm\����`��NM�K��x����G:y����v1!��4k�^P�T��GA���#�t��E:Eq����v�bM��^&�E�)���mJZb�dH�ak�������
���1�Y�\��Y�����J������k�����w_���^��>:<<,��(L���M�������7-Pl�>���pQD(��o����@����L��.�pp5���b��S`�b:j(KV�"�1_���Z�A�Q�S��/��c�����6]LEMO7�oQ���c��!b�x8���Z���I3 +��a�I����v�@�?�r�t#:i��*�n\D��}��]�!�����R����,���+�>`WYa��)Hg�������0��	��@�Jey��O�o�	0�	E4!0P�w��������� ��
�>�#i8����44�J�����y�>�������x�H��g��qyDp�X
��1XC��b6�4�J�$j����5�-�}r7e��w���:�Yb�i:�n��S������D��8a��x}�R�g�Z�h���}�"U�d2��L\����;�U�pl���P��}���S�W���h��~�aB���BH��DWZ.���(�92z����n~t���j����V��q��7��!*��cF7�L]�s����p�C_��b��ta�b�27�o��d%!$d�]�L"v1�2Y��]��I�,E�g�?��H��Y�s��Jj��r���mv{G�f���jC�Y�`K������Kr���(������������M�^��	D�']�r�]�@�8��(�eVVs'��q�:Q��t����%l���x�����cj2����$.�Y��I�������7���{�����6��6hE�dkn�-)P6�A��Y�4����g-��-��yJ��rD\�Q��H|q����%��*a	����J�������Wt�������������i�D��������K&�~�^��9�G|���x�w�3���������S�zU�����a7
�8��.Oxs��Z��3�a���/��N/���
���������������_�.�,��}N�"��w^��8;�d���gXC@�_����-0-�?����?���!�]���~C�%v���>o�������?�g��o����
�o=1^��*�q��?�7������7�9�/�_\^��C���PN`\I"Z���[u�������Ap�m��w��������@��w4A�����:;g?�]�7���s�@N�����1�}��� /S�Og��|������
�7��Z&��������N�5��Z(�C��6��_5/��	|G�QYr� �"�/R����os�������0����!�Dh;��/'*L�?0�_�C�����/,���6���}�:x*`e���[?���j1��"�"�{Z�����S+�G���|.����n��N�����Kp���qG�{�"��D�-�6�-R����m�T�Tx����$����� , ������Rs�F9��|�`�����$�
��`��/�?K����a<:��|�[�2�'����a)h.���!$��;}h���1F���V+�kc�
��� �d(%���o��l��*��!EV�n����I��@6�K��H�N����n�w�3�	�v�vA��i/��^M�����>y�2�z�vE;G��X���6���Mi����I_�f�qX�a��a�*y��.2�MC$b
rh~����W��OX)���?Y�������@�����v�C���T�v�C���{�����Li'���v���I����D�4��2�_�!�C~���d�"���V-����0��NA�%����;e0���E~�=:}��?�#���;Y{�`�������G}�=��m/{�/3n�x�m�0���<I\S��\&�6��^��S� 8�&�����:wK+�0�J#8n��
������}���e��
����	�����r�c��R�<���AB��!���D0mbh��1������_>r���$��7���Id�Z��)D(!Z���)�6��Kmt�_��vd	
������4�YDC��5-�-M�*PZ�{M�KkN��o
��\`���\bV�m����=?{N����"]"�2���MO���9�tV��p2�)d#Y�+W������?8���Z�U�*'�f������|��,���Q~��� �
���
��*�$���9�J������f���?����_j�Z�z��4���_Xu����y<���%����q�:�E����G@R���I��>i�z��yT�N�a���V;��QX=�VC��s>����X������h�);0b?�7��f�6��+G�h:�rt��+���E�������9g���t2�_��Z�ip��Ve�f�Z����ufO�?>�^���UO���e�z�qll-^��`���>�
+>�SDx���Qg>��������Q)^�f�qw������R�i}0���X�H����0NgTp�l���'��C~H�7��o��9��!���/Mq]D����n�7p��;W��8~�}���J��=M��>E����3�h�}�L�i8��Z��z<��2ua��^]r�"����|��11�|�|H6�U����2�h���������O�d�0��Q�c^� ��<�:
�ZP�	�d��s|i��hO�W0��\3���8�&��!�>y�vK���8��
h��_�C���.�rGQ3 ��L�v��A�m���J�^�6�v�(jFK��eABS��������c�/�]U>"����?�@xf�h���"LX�U"+�G"�^�$?�}���J��Oit��(!����9G���^������)�e^_�^������F�����d�?��������^������JP�F��A���l/(���@��oO�_�|�!���8�O&����)
�HZ�Nw����17�@��I���� (��*��D��@����0��q�916�~`0�5=��y+R��C*~9Z����z�Q�r���8���nS�x�&�R�Y�wJ	~h@t�0��!���XZ(�}�������Dnt����i<iMd<I�p@������IbE*�mQ�b�A�g�v�liGo�w�M	
���-�P�"� ��p�����|nFxh��lKO�B��N�r���r�G���f*��������d6��DL,]�^]�rR��=�G1�g��{������[�P'��� ���bt�-t�4��H�%A��iE�XzD���@�}H�,~��7���;�N�V��z	AN ��9o�3q�du��&�:6��;�ST�'���a]���Q�W���������]����h�.���W����O`�����C'����g��;����p�&.��l�^S��F���d��y�� 40����8#����4���������d�����/ue@R��������0P�<97����Im1S�R�A��.y������oH�[u,2v�4���>0��|x����U��y��3���WD�kC�^����O���l���;�m;���Q��������H+'���9�o�[8��HK��Dt�F�%������i�!^�
����0�2���������6R`�D���9�hd+��dE8:�}�J"��1Eo�v��Gq�q �e}���Qg<�������-	
'e�H����������`��e#i�w&N��$���_����}
�%���t�}��|P~�	�D����e��0��Lu��EK��d�}�L��>��=3���P�L��V�/8�� ����K�a>D%#vox�	��y����1�j�u���p����C�S����6U(��|-"����^�R��L�~���'H:�'��yrx\j���������i�O��)�&����,����m�e�3�� �Qo����1��������#�����g��nd������D�2
�|#��x���d��G����4�j�A�s������Q�(������a��������atk�G����%����~�
�6���7�|Z�����)�b[��	4��R���xH61��W����T�Jx��!?���N�~��Arg������u���q8<N�q2!��K��y������0�O�A�
�E\�F�G�bl4�bh���b/"2�����LM�����������d2���D�]$_tG#?�C������*w�n!Q~�r�q�DDa;�D������TF�? �"E���hv ����P��T�7DE��D�pN���$G��n�"�
���QW�~p�(V����
3��}�4�~�U,��Z
�2�S��L���	�������O��>$��|��X��2���������0
)N�H?Z�A!�{��9���8N?s&oJ��;�DY1�x^�V��K���@/L���R�I��u���^���?RH��9��Oq_@��ag6�����D�t� 2{�r����'��U���3|��a�hv��=
�n��������6��8�)��!^FA�'��C����c�6=����!��=s����#�IH@�X�������R�l�-�f9��/�W������*b.Ru�T�����,��=<Q�X�g���l��
D��zhb<U ��,
�����}����������
�[��:���u�jX�����I;W��DS��	�0*�(�v�NY��};�3i��[$������(�?�������� �����?B0,��+�;����|t����%������)
Z� #���1�bE�|�����?����B�/�������=Z�6q������(�;��l�F����E�Z2g���JG�KR�1���T�����QP�yj��l�pzq��D��[�$
Z�|�*���-�.���e��g H	9b���q�My��A��u�"t-��!��"�/!a�7GU��y��G�I�q��N��"@�e����S?����|��9#�b��p5�S��y%�>�G�X��}b�������}�;��@E?V���*���������*����0���d�p�
'
9��(�6�mju��gc
������K�����WV�N��q
�f 3d/�6���'l�v����)a/�x('�{,�k,�pT�����s����T����k�S�
-<�uQ<�!!�c��bW���z�B�!�!��lq��>]�D���v5�!�C�<@���cP�$������dL�j>�9-x�����R{{�1$���&\�h�c�$����w����6�C�	�!l���
���x�vz�����(�e��+��7�t�{!�RW������g��&A��@��go�@�B�H���*H�_~��c ��c`�ZZ���Qq(Cf��Ar&Z�4&���!!m�X���9���m�G�&/�g*�tZ%�4�PS�u���G2�����#�4�x<��O�:�(?����B��Vo���$���T�k/`h�u�^��o���t�������E'����@P��@�T� M%���/0Zro��x�_�~�R����zO�b�&���[y��-�c�D�����R��F�W�.kn��H&�u)d:��.O��P��ce��!��p��D��V���l�\Y���I��c~e�`�Y4HD�%��G������ZR���������X���A�T�����,)n5�����Q��8>$���p�����lN&�����iaIfr�A/�:�-|oh�~����2(`���R�����"
(���S�a�XPC��G:{h8e
D��AxE4�R���6����������tN�
4b�xZ���G�1`dZ�L�q��|r)lK ��QxE��1/��@����r�����(��3N�D�R2����F�Nh���-���yg"����2��PkaOx3ATw+ Js
���Qh<8���1P�d`|dS)���a�t�e�����^�1B��K���p��$����!��W��c����sy2U��<��3j�\�����D����_8����
��.-&9ck����/���~r��a��^s����R������(s�-�Q"������;@�xNC��&�O��O� (�9��S2��[	�*�M�����Wa-������3F�2��T��������QKh������-<�����`FL�c�B�\T�82��F~s�vIfg�]���b#�:���6��B���{!���Wf���B�A:����q>d�48�c��B%V�f	����9�fg"5�@���fX�11��x������V�G��2��^JTs�
o���r�oy���D~��������O�"ry��pmZ..Z!�
�F����xq!�q�JS�[�a��r�	DPP���P�������z`6�"���t���/��.1�^j(�C��f�� _�����n��>����kU�,��m����k���f�M]N�*)�h9��Tv��$��{H0��W�&���!��!�}�YUNg�!K@�@T�Zi�A`k��kGZ���QNM��������V {��U����I
�p��	&��G"�E^�K$8������wN�������B�*q�����y�����\��g�"[��"G��Aj,������H�}�T`�����.]��EW�R�2%	�+��c�e�2��
�yr�j������UxM�M��#q��4R&���e��QV��E�t�����a`|���_j
J)4�����������+*���.J���p�
g�b�����rI�.+G,�����4c.OFufT�t)��������xj�;�z�����8oh�NZ��F�x��������uF�0i%�C��a:I��qx�1`NE+�B�2(��2S������LR������-@�q����d>����iv����"%BJO���@BO�Q���}Pm�R��G+l����#c���d�A���1��{M������`�� ~�=o���wRhHl��kAin�[y����8�����XZp[$3/K'�"�1��I�����J,,8���*(9�o�6��Gf%UX[��x��7��B�m`Qt�����C����;���:Z�j�l����eutP&;�k�������������2�l0�xd"o�I(Q�5'S`i�����b%f7���0�C��1�w�n1����du��+�p��$5��4������)�vj���|<�;1Y������N��������������e��v�����rX���J��q���2��9���s��*{���9K�����a��`p������}|�7�%R0�X4�x���->�lfA���a���D�rX�n	�y��'����$�\�Jp=�p���Deh�F
R�L����oy��u�0��y<�����>���>��<�RX��!��/Ejt��A��@:�����d]z�
S������p
@�8��oJ?n`��'�&>\��{���W<�g����M��o�4N4y�dr��~|�?9BS��uN(X-8�k3m$�������DdH����r��1,��5e0��V�u�Ca����.D-������3x�,'rE+h�
�$��kn�����2II�O��t�%?E�3�43����.�a>L��{�b���+#���&@E���� �'�u�^:[�^+���q��������|�t�SX�7�������}�i2�vz��N�5@�'p��K'����RcyVz"+3B8�>��nb$h���sI���?�\D��F���y���n���_C���_�6"y��f�wuI[��%:��/	�|�g�2LV�	
8�1Z��icU_�����*��&������+A�)��F�����,Y1��3K��Z��-d�,�{�|�)"AF�JX���cE�J�NFH`6S�zvH8?��--G�8�Rw���E��fP�&���+�h0����d�$�b�������������8)����/9*�J��1��GE���t���,���VI�P����-���Fr��b>fS���n�N�BFS=���|�4����/��^����Z1�����b1Q)z%�~f�l
��-�q�k��m�����������q�KRc6���E�;�,�6	`�;�	}#J�G/��g;T���T�n,�$��u�7��l����Y�=��v�8J�d�:��$,�Q�0����I�%�M@%���w%�w����6{�5�gS6{�����Sm��Dc_�f���fOt��!a�W&d�a�W6m�|H���
{���fl��k��I���[�U^`��#�UJ�F{j[>��l���a���Y
�<�n�W�vye�.������=^Y�����Nm���9{�r�=��/�x��=^YI 2�����!{�r�=n�m�g�>{<�km�g�L������2A�����U$���"��ePQ���xHvu���vu2�f������A38�@��� hF��vb��@����H��A�T�O|�U�w����oO���x�p�	��^�G����'�M��(FqK�����_��	���08
���5.fC^t<�����y�Q�k�/�Q�|-)#I�g����C�"��NO`���T���UE%)�}+��l������~M��z��� ��W�<(��_�'������i70�p12
�����p�%�����f������* R7����!�px
�����������X��q�pJm�Xv�i�
I)�TZ�i`A*���L�q�����Y�:
�)��M&K9�%��<������u�_���
�e�����q�61HAG��1"�.U��^����e�h�|��Q�ELL����s��~����l����������X�I���#S����v���Pj�@���`Q�P�GB,��c�����G]�/C7���p��'%av��j�l�@�O���9������m�i������7��n���������%��	9Ys���Df����B6�'�O2�6__��Bx,!e��������y��t��'��,R���g�o�/��XJ��s*{��������'�P{x�� 2�:�����������@I��i�N�dww�/nn�h�@�@���6nMH
��\"m��taQ������p�nz���!W4.�9;��[���`�	��y�6����@������o��4�����i��|�t���@Nc<��j-KLG�|*�-
��QE�,\4�0A�
q7D��6|A��r���1����`��^����8T*�I�X�]K:���'^A@��Z����T�L��V-�S3�&����������P���l��5I4��Hk+�(�	������E���q�D�h\��Y�sr�h4��w�7���qT��]��:x�����1�E��i�	�(�D�N9v.{��0=�qZ	��b�7�1*��<f�!m�g ���~$R�n>�0b��\�H�|�5�X�5|����5�]�*�H��qRE|k�����d�����pj�
��M|���t������z�])��{@��<����(R��"0z���:���a�<�]p"&G�����6<�*��0&�:���'c/!���3�
���4���Un����w�p����w�,� i)�p�� K�JF��s�L&3��^����I@���6'4m�-G����mL�\���R�T��hB�	��B&�	�kO/��aAS�j��F������?��Q��t�XJ��1�&��tx�u���)���������T�13G���=��$_���6������d�MbG'���K�Nq�`0E^�1�<>�c�0��[���JL�a# ���b�`�;�y�6Up���P�d64����F��������u�3R)�K�1^jm����(i1��	���_-����b��aL�����e�n�8�V*a#h����,nI�������(]L��x�s���S��|�t}Aj�C��h{!�Z��d������MY���uT������'���E�a"t�D���=NI��"�
��r26=�(�����+D���3��<�� ���hC@����n�f�5�'��?��y�1������[�#n/�����z�����)9�>
�h��t�ml������55Q�X+1A�C������o����m�����G_�|yT8,/0J���'���E
��A�.��8e�Q�p@����L���nix��.�A�*�!������������a�R�f�Q_EU�m���a�p�������e��g��j�5_�^Y��|�����C/u��)6���)!��1k>��%���0��] ���|���)�b�9��ZUH��a/��D%M4EJ ���F���@R/�nq�]{x���Q�������Da��h���6�r�����j��(u�G�
�(j�
�1�3�����������y��)�����TQF�����p4AR	���3eJ�\��T&2D�H����Z?<�:�����C����`������:���Q���W� hC*��n�Vk�N��{���O���o��ND��Z�i��������?)le&�2�V<�l-x��
���x&r���JzE�����STq����q���C��{�����s���6��E�����dj���D���� ���v�� �2s��b���8�D�z�5j��^3w~T������(�i���RBP��C����\���������)y�h�%B���E7����XI��?E3�HQU��z�*�;T�)���Y?;N����b+��l�fF�����D����:26�0�x76hcw�����q*��=��g�Hh����i���,&E!����=Q5EY��C>b�������d�P��c>��M�	���[��`P�����`6�����kNz�������#� 1�k^���p@�8x��A���]�����d���e��0t�0R��@�K�L�L�G,�9��O�����2.I�xj��9�9�,���]&DoI�������L��~v~����:�[����_�@-Nbzt��������U��n��p@Dq�U�r�I��E�o�!��(6O�16b,\���T��z�<���x��i~���l9�L�q\��)m��<�m��8�w������w�����I��l4k��7���q�]oV�G�F�?����k�������O�G)�_������"'b���[�����9��K����@�c<���������Z����������U��ry�����?Y�������T9A�v�jGUH2�=i������ni1����Wo��B�Q���������X		!~t}nQ�Q���!��z�j��������(��!Gw���9��6!s�:���iA����	M������lDD`H}��������QT�nGA�p}��{Q����w�O!��r���A�t������T,��mKP��^0��Y�G�.^�Bj&�KJ��Z����~�{��5 �<Y�U���E��1����!��C�
+�d�t�d@4a���F��W8��c`��rP�E�i�|$"�.���;�T:mJ�
�8>f!k�������q_��������������)�s_���j�]6'���x���H��dh�/��������T������2�HY����a
�:��z�}C�Mc�:`�p�Q_���Z=X�-��
L	)@����Nrs,.�q���f�s����`�X��*9�f��M����`N�w��]�'����9K������u��~P?d`����?��yA0U�l�������+��#w��u�Q<�rFS�<_q���DJ�)�2��{V�5���p/�TR*���wc*��3a�'t'6N6u'= �Sm)��Cja4��W����u����Vq���{�`���=�<�V5�dW>a��'�I�	s�|8av���6�:��7r������p���L=\V^��'��fwF�\g�+G-H�#����seN AeV;Q�[��H�%��|��
s�n��M4�fv��.��y~g��<�!t7��IC����A[)�="N�R�(; ��a%!�T��|�d�L!����FU�L���j�T\�(!�!���N+Dj��p/#](?b,5��\-�	�%��d��-�4yBLn�-yt�)��U���~!�<�� 5.�fx�I�?��
�R�$�&���{��6�$Q�3�+��36HPX)Y~G�)��e�C��=wn�P aak@�������k-(.��������������X�4���sz���K���[�@LS��Q���g��C&5Z��T%��h���O��H�:S�U���C�~�!��A8xn:�=��+^�R��9F�^$J���o��{�����;%*Y���i.W�*>�e�����R�}o�3����e�:��}?4�q-���h�?�����5�P�m��f�_]��U���&_AZH��~��}8���+H���x���+��-�:l84�k��-lz�O�{�^��+6Q���yR�*vh��{S����E2
<���GQW�q�=tE�k9[�)%���g��M��M��
7����
����������4oW�z65e>~��O��k�=��~�>��|����~7�:�^?�7���A���Z���LZQ����Od��f3���M<����0|�
���[b������a���B=�L*���a�R�k@��hbW@��8	���@g��c�29��J'�d<�vD�K�h��!�NBXd�0^m���^�y*��M�l����-%1�R��n9�w��KP"&y9�+��K�h�i�a�����W9��y%�����4�g������y�Q�t�	�����F��X#��h�bF���`+�W��p��\�C�@Bi�0j{��/���j��@�(�17Qf�����6�@4�-Ep*=��B��$hIkV��Q�l<����R�PYC1�(��VP<.="	m�G=����(�����]�%�����Lsp��%�1.R^��h���
�P��	�~�d�n/�Ngp�$�@^�
�9=7
��T����RlYXh$�e�A��;i��]>����
b�����e$�U������FJ���{�*a���_���!������n�u�����KU����2�qn
��)�j�Q	��eebL+�N��V�t�n[�Ocn�j���VN�����F8�����TllRahl�,E��Po������#���q��Y�6�����,yU�������
R{����Z�]U��,pO����$C��9cA1��JYl^�j�W��������;ah��)���P��.������#�jGK�n���&��*@����5���1y35{|�W<l*�pq�wF����*�y�68��8�B����wRH�|�������X'�����acX�Y��WO�Gs�$�FFL�mT�Y8�p��%�a�D�i��k4�''�i7#�NX{Y�S���J`K��`;:�='�ybO�1:1�N����_Q6�}�����C�Vg�Y|~��F2������Kv]�f��LklLW^���]���*���w�����Z�������ih��06T��[��q) �N�[X�%�.��n3����Om��z���Zd��.��A���H�� x�����w�\m1uQ���
;��S��'����j4&�I{<)���*\UnQZR��	�C�wOFJ�fD
��lm"�c�0|�^I�+�b�&P�%�^���Jnktjr�����TkHc�\�w)�s�	��p��E�T)]f�.��E�x���	mD 	L\���XzsJ~�92���������S�|��*]�l|�"�V���:����SP���O��3:��q�9�8�gc���c�rOB��������DR�����U�������RZ�����f��7�.�5���.����fv�TI.��.@o���.��dx
�jeJ���nsPh��1�9��p\B���6�JI�:8g&,�%#���\��p�E�T9�����R�in��F���C��9^�X��OE�7`~I��|�S@DJ1���J=8���F�\����9"����pL���O�A���#'H.s�P	�;r�w��N��8�����*��<p��d�8���
L�A]�	�d7��H8v�+
s�M��S�����p.R������>��n-���TLv�_����m6<�	��H,�>K��R��\�\�
&�j9��FPp�)�+\^��D�<��7�-?���v]���F���7*��I���y����dm�U�8[3�d����D����Zx��YD�#����n���}������(3x5��YV��Ur.��N�������+�����q�s����$N��+n-��A�������hP6V���{�zlB?[��z�{�W�����NL���*+C���Q�H7
�wsd-�	��%K5�q�U�5|����|�XR�R�J?�;g;�k��������@PkR�J�+8�o��(�#�,���N��I��N1h�%6����`)�hy�<u�P�T6�}`���C���)�`���q z�7����m�N�*�] _	m8o�����������_��9g����F�]��tc|�H��O�s[E���;0j�&D|���8f�An�Y������~H�@~�\!��j�c��:�b!J�D>����K��^����K�����UM$���#K���Q����Mi����%�����	�~gd����Y��u��-'�� ��{�
�������w�O2�&��/�����*l�.��v��n[���������������Q�!�y�mg�����]l�.GL���h~��_e��b���� �N?hf������w��39Q�SV�|p;��y�N\�=�����?�����-,�|F�Z��@�|��U���h3� ��KF/��n*0l�9��-�bf^IZ�'��Z��NN:�Z��/	�,I��kV����>u�����,r���J��}HC�ze�Y����{L�^[W��������@#<�z���B��S�R���'bcg��g��0m���g�-�?8I%�l�Q����"�=A����6�,#J����L���	�����������0�_�������:b���+h����$lG��4W4b���������B~u�Y����V2�^Q[���������� ��W�����gI��5���\7���!��>��A��Q��&N���Y��v����P��������T]9�#O�����$���^��,�)�-d!^$���&������S����p��ZyY?�F��p]��-���D��f�Y�c�tc#a�Z'�~-��\�����'{Oq��������MB"�Z$'Y�p���[�}h�������v��wJ�`wH�#���W��W/���]"��2���c/���Y��s�fZ�?^�O��y<����U�k�LO����"��cN�����;ef..��x/���X�������Ojey�e���������.����	���}=�]V��I��O������#gq��3%����r��Y_�1W���yL9@�*���5���1�@�����D�F�k�C���5+�0:0&��z����,��4wu���1���d������/��n��������f�K"^G������
����-@�N0��41�9�{M�n��`�$���b��
I�j/W���)�N-X�4�x}����P�Q1���8�@_��}~'[���:�"O�{�V|�V���3�
yY5��(��ss�����WX�����
�J�Lv��v�K���D!,��%j�x�% �9��rh��	0�$�/���U�L
��E�4z�-�"e�!��R;aGyi����|do����i��
u��6Y�>Q2{�L��<�ra��6�d>�:T�	h���E���X�nJ��$��
*WW^�4N�e�XK���T;f�����=�\��j�r��"�!q�5|+�A����F3��0��~�b�E�;y;�5�y.�:����-���8���F^:�����{L�����F6������	�/���&���1L�n�%RZD��������J��E ����x-a4���a�7��&����3�(a]|�m�^�[�&Z�m</l�f����i��=WU�����`��\����S�������j����g^x��3��&Rt&�S�<���0��j#��>~���
���+�:�Z�������0Tf��bOf�6IB�3w�RbRe��������c����)�80Y��(��M�!�?��i>��N�
~�Y�/PRg���N
l����CA�3A]���q�E[��MgR;���_������:����a��4)�\�=�k�1��\*V��G�j\���?_�{���$�!�)��� ������]d�=��
�����ToI���8�zU���!j��q�r�/�;8x�hpW/V;��x>�B��Lr�+�M�
��A2�>d�+7U��\_���5U�	�����NO��H,�G�v���m_R�`e���r����}�����aq�cd8~�b~�7�2��#�����F��yr�o=������|����z���t��������i�%���N�%�;��5E$|+9���?o��D��WO�0�!:r=a��;�,�{����NH3w�,�9���h���V��\VO������/'pF��T���(�(`^9��	Z2i��7_J@6cFrp �@�uY'�������l��ta�]L���p��7�h�u��YZ��IZLQv��j��#��	�E��<��x�J%�x�}�;��6i5�@i�j���@���#w�4ck
���z�m���Yy"?d����c�$[s���E����),���H��d���2�4I�>��]�b��*�M������Rz�Y�gL�I�)"�Ep�;R�TB�8Rp� q8h7�@�PQ��T�OK��?>a(d�%��4�����=��j��E/UEam�*D
p�hz �v�y=��k����q���<DV���\���H��:��
_0����?/�Sl������Yv�x����<�qB����������F��Vu��!�j��G_A�K�!T��v3��� �uq�P���I
=u��^-l?2U� w�>���2�YN������>�f�:��F�v��hF�u��t��EQ��c����3������7��#�/};�)Bw�]��UG��{��D^�r�8C�,�3�[D�������L�i����i����(��z��u�c�"�s� `������b+_�L� ���-1m�Ml0G�-��`��21��~���`>�}����O�?��)C<)w����ZE�No�htG��vJQ(�.B>�m����,��;G���;t�C%�&k=��l_jP�����j�#v9�z���V=I��W�e���WN�{�4si������
�c���W/��k��W�������8�l��������zt[���p��q��&7>K��������;/�s��z���9>�5��FQ�E�A�;
��A4�O��f�n���`�
N:�*��.�a+�)���a���|�i���S��e�C��C��^Q@��e��%3�/�����It�����^c���Ud��a/���U����b��d���i�ZO"�>�
�"��T���q�~������(�DRTQ�B����i��S��8���G�I���[�`������buei��}:P~�,)����%�X8�lc���?����p�Nc`�Us��+:/��y���k�?���~����������N�^���-y��
���9e��y����z�Wwj���$
��(�U1�����D�dYB�hJ�������U����r�c{i�����'7(F�\��3�Xv7������t������?9r������$�E|(W���?�Kg���A`��(xY��q���
]B���n��U7�����$,v���3�7@:�'��.��Okn2�;�.q�t���\������MT#�wS��r0�g�m'�v�FVC�8*1���;��q�s���-W�\��,+�_�i�3�)�����>���������v���sX���g���v�g��'�(������[|�RP���r\�����)�����RYAf�����<���:^���2�5X�s��o���[w��W���o�iwZ�����]����g�[e�s���<��e�;��D&�_�;�w7��e9%���r���`��x�o5���M3���(���R�R�~
���k�����j'�p}umEC�"R��e,��H�J:��oz�$+���3���H&�������i�t����~m��h�5K1������w�����1��V����{g������oa_LJ�����/a+���`�����������F���Y��j�Ga������I{�����5���Q�S�i5�0����s���<���t���53�Sk����F�p&O���sH5
/�������x��e>!c�6�%��g������-{v�
�u\�g��
i���-Lq�C���6/
���{����������Xo��:���1�>��M(=���-l��2���l��	zl *����1�4�����Oh�~6����K��7��a���5�bWkS��1�?]����(>$��N/���Y`��.����Qa� H+jl��I����S��EG�`�������*���k\c��hs�Sp������pZ�pw��2\�����(�o�������O?o���T����j�m�B
�AB��lz}}{��"���[��.�m�s�.�~�D�u-����E3��1��|��rX��#z),������D.���j���A8�#�����}8=���#������/�gf��a���H�U���������!�	G�6kA�����^��z����o���!�w]
�x
��Bx��f/�LH�[
a�R�������x)��K!��B&�
s�Bx���.�<p�K!������}�B;�BZ
�mP��yj���]�KFM�v���6�QN�'��w���?\|�A�9�����m5��;�����s�d�o[55�m�����)U^R�������-��������L
`(����1O���s�E��Nvu������	�#�S|���G���*3����G�)xT�����j9yT	/	�a� ���-��n ���
3#�le���|�*@p����l]�	:��(�-�0w�),�AE������^���������k$�Bam��c�N���^z�wi�N3�V3�~��n�L��^�f���g�-3��_���|5<!��CTj~fV��Tr��a��L�4�&���N���iu�G_��i7�:����u:V�j��v3���������o.��0[z���{l|�����������;�t<������%�m��J�����S����HZ���?��X��q2lyX�/���=�Sn{\)��?l�tc=���/��}\9[��P��P{��vh����������p�y��;�w����~�t�����|�Y>���/�3�w2a�J0����,��V�����d���~�'
�T�A�Fpgy#�T"��s����d(�?��rQ3��z{p��b��L�������{�������q6TXC1����^�b��w�D�!p���`1TXt���p�b	g/���b1G���Xl+,�,�},JA��\�(�MS���\�^b�=)%��?����[���>�f�GVuY��~����7a(oZ��vn��U�:p���NJ�[��J�#g"yF����8�o�g��6]-�����C��j��>P<r>w�������*Ri1q�e�R��7H��Vh�o�P�����)x��;����}��������FS��}Q��3��4�'Z�����?!rC�a&r�<��U��~v��TE=r�>��3)��G���{WM�n��j9#�.6����u��X�;�X���I��x�,�C}}:Y��s��" �(��P�<����}uqz��-Y��I���#!p�!4�?�����
?�q�l�6kX������p�;�L�rJo+j���8y$�t��P����)���u�*��WJ�L�i������w���j���Ep���������#�������>��R��Q��Pi�k�jr�PX,��������=����1�����~��u�:7��{�Ns�k)9�{��qm*l���b�?A�����e��2q/}p�p-�m{�p�2Jk���PF����SR����rnK�##��8f�{)���S8_��.��b��v�������Yn�,
����?��S�.K���o��'�o��KW�$���~G��`��1��70���b���N���G�i9������������<2'l��ZT�o��i��Q���T~�6�;�����n�p�����)�mt�<���F'�����C������]�����#e
�x��lR���0��k������;� ������������
�����e�,mG6�������+�6(w����=�]���Yf�����_����"���4��8@������9����p�*$���q��A�1��'��~:��09����.�MA��`���z7����'����f���i�X�	�^-)tp�Y���G5�!�
u���x�^���$X�6[@0N��].���@"���3^�o���8Q���:	���h5�0&I����=�����2��p��du���C5=(�.�9��%���p�~M0��3���������{�]�^��~���(I�A�r��=� �7�^������c��X	�U������:#��(��,^n��e�)����=+�����.�"*�O/<�N���������``�@r��_�V��E����cV LhE�l/aj6���x�bR�D�����������-���?lD��
'�hw���6�h����k~�p
O�6<l�K����
O!�B6H����
���w�?D'��<�����7������������a!�_Go��
o����������x���M}���e�l���z�f�G�I�V�1t������N?������S���������(
[�^��mF�I�5[�����z��������NN>Q���0#h�4�Z��M�����'����Z�"j�b�J`��OD���Q�\��Zm�`�Q�o0���L�^'3��jZQ9}8�f����������+`�x���-�mMX?�$�����0B�f!iS�~W��x��z�!�7���;i����!�$5�����	C���Up��|�]��j�XQb���G��������@V�_��X!*����� �1^��0�f��8��a
L�]����;���7��r{p*�T*$��3%=�y*%)S+H�c*��i.��T���G)��B3��3��5���F����������4�����K%��!�M�d�Euy�5���1}wSi�$�WfnO���w�1�nk�����:7#�����P��W�5�W�&_���jnD�RU~B�-0�t]��[z�)q0e����?����q�j��U�j��>�|*1��s�o��+/�J�������z#0�:c�Y]�t�p���n,W[��@ay�"A���i�T�1I���r�r�F�a��W�uJ�B2J�C$�x��#�"%�m��>������*R���z��;�Z�����x���5[�N&	c�q5��~�����5p�_�I'�����Gl'������(����dR�6��)M�2�u\D�9%=���z���zf(^F��������D�@��U"i�1�LL~�&��IR"�Gd��<w{�V&�{r���~����kL*�1�5U>��6��d��v����+� ���j���2�l/;��U��S���<[�\�����M���+,R��x&.�?��888�������o�MD�-��j������M|��_p�;6:����_E��p���f���9U{o1��f��L�����z��>��qA_�T��~e��%9v�`/V�[�[ ��X-����PNpO�u���j*gu	y���$��JRNz�A��{�^<*#������,YX����C�$�p"����$�h��(2C��d�LkY���+ &��Mt���
,��A���Rx�����,��B�$d����GT��],G�mw��s7*��i��D�F�R*���xg��8V���vv�^I��[��9\��]�.��!�.��hpq��j���<�O.9��qx�#g�"==J$�.P�7�}hWL�lbF�c�>a���j�����%���E���"��@ �y�Q���E�����Lg�P�s�(�lNs:���KY�n�@o�`?p�Q��"�Q�D�u��E�-����T���!��wx�H���NI�GO}���A�Dx����N�,�9:������A�TxHa.<S�u+$cT�j5�|xo�K3Q=�s��N���F�Y��3����fv
-��Z���{�j0��������t��Z>l�&� ��9��W\���������m&^'V3���#���)��w������������,��w�����V��N�����N��NkN�^+�Z���w2�l����n�y'������ZYaj�T~��TMr�J6o@^m���c#[�B�9�(��H��3������E�:�f
�J�5�����m(����-U�*���m�	��U:~u����w�5�����^�&�(�+�n|%��F�9j��zkR��������8�r4�b���2�K�W;���>�����]�Q���>��l"L*�88�U#�<��+��H��p7`�������e	�p�D�Z�;"��BvN�#�.���x���Y������Y�����s_1����T���N
=K�4jj��e����tC��$<8[�$]a���X�eH$V&D7�vL�O� �.������o�������?DE�)|�_���d���"�?`��Y�w�t���Rx��X3"h��&F�0��@E�l��@*����P�*�I&u�u����I�w�*�����2rFe���[���/@��&������PqT�����<����W�mI}.'��5w��/D�k��V�c��W�	���v��1��`�h��.ZuI����jL�rr��?��p�6
�m<���S������W]���:�If
��|�J����g��M_���<6��7���H����m����������n+u��.��+��Q�k93MFq���F4���wmW��K�E����x�A�aOK`���n�r\B������-��eq�|CN����	��!� ��o�W0���Z�����]�&A5��[v����<�W�9�����SG��3����A�,�=:�f�Y]BL�<��Y�����u4���5����`�PW+�>=�L�xt���������9�ED�V
`��F��P�YL�!�2���_����'U��g�]}�����o��2�L^�i@�c��;��I13�2�~�Z�=�c����Z$�u���M�e�U��9�~�.	n7��v	�kM3��2 ��A~�^�n��\9�r�y�Q�:M~�<��	 fm������*:$�/2��ZQ�U<
�/���^m���k����k��w�� ��*�����d2�v�
o������ ��`��['��<;
~�[8�J@t�]����kXD��j��i�p������{\��+�7�����w�K������&�%���_ei�@����3`F��9�E �G��yee�n.-����������zv;���Cu���L��W��w�T�T�F���^�����28��RL���4�p�������]R}�
M����s���>����2i���.��!B�1�m5=e_3[�'z?��_��]�r���K��6���;\n*w1T�b�P/m�Ph<Sd�Ph�`_��"�^f
��d��l.@U59����J]���������g2aV�O����o�d()�����R� 0��^C�
f����D���J�L�A����4n� ��<�)l|/�A��6P	2��mJ��d�;�|0|B�[��%���+���
3�L�����&�mt2%m�G�~�M��t�t����Z�.f3��4;i`?���p��f=����"����p?��I-���:>���#�0c�T�c�	������E�����y�K
���|�
�#���&e�X4@z��$�JX�i0(u��F@����w�w�(�"��$^�FNN�'��9��,���@Z���I4�0D����}���2����S�TM��Ag��1fsjsr0��[�"���P
��=��
�������\���>��B[����2��f�(9��t���5�����q�s�����x��W.��a��Z���FG�B,���A�AGl�,���,������H��S����p�q�������td%�a��x�����/�(����(|(a����@.��������j����9�z�~��k4z�~���~;'�wm
�b���Y��b����b�l\}i?g���N�	�=����9����xal����Lr_E��|d�����lI��xm�w@�	�_�M{����N]�t�l��a�{U��"^��U�x>U��I��6E���N
���0cg��������d�B6=r��%���E��
�@J��T�J�q���E<K�E�\����l%�
��{�3�pg����V�����xE�i�Y5�,�V���M��Oh�a�D��)���F9� �����b�k1D����j��U�Y&S�,�g%=w0�mXr0
�E(��;�9b�Z��c�)}C��	�r\���!P��$C�Z9�;���o����~�3��?Va��/��/�7k_��*p��"������T�J8�������!�	g555
hI�Ax"?�*4��p�ox�j=�x��z��a��Fmf����[��r8[F*����l�z�e���%$����;9��u��q�M�J}��_�W����Q%i�D��5�"�"w�}����\�Z.~�o�{G��/�_��9�Y�l��#�)6F?���X��V���6���� 5�n�r�@
�bC@��&C5����s:�y�W�V]���� ���K
�����9*}Eg+Y���t6V�t>�!C��t]������J��C\!��!�����pL�*G��?�4*`�P�2�X>�Oa�����-�Z��s�DN"��44����l�����\��fY���]COd�kC&Y]��A�pfw	hV����;���'�{����$��	��"b�G��������U���[5�n]�I.S���pR(�2�4� 
*��s���'=���z����I��"��]_�s�.8�������#]�;{���(�\iP
�l����1�g���D�e�	�t���M�����
xp:8gJ�ZO���K���"���D5������D�s��h�����~�3�����S�JO9��9���Gf<=��3�*1����gxw|�ux��pp�V��5�K����|����,�u�l;��I�F%�]�1�?lVDK]���={6����L��$&(b���l	$:^Sk���8���f����53��cA��9H�S& �=A9�g����
,�`j�������)�����fi�#��x�@A"�D�[c�l4�S�A����(3��f*DjNJ������%H��9f���K3d� ��{��&.����6?3>�y"B���p�)��n�?��>?�^��g���M��H���4[s&*��"�}�6��>t�y6�/�;��_�����<���%�\��8T����Z$��j��e:�+�X�2�/����l�8���$�|�N8���	��H���6����I�>��@��Vx��<�f��%��'��)E]�\����"�[�T�kyd�}�-AVKOt��A�%������2�S����Wq�J.�E����pJ�,U�*X����ga]=-���ue>��=�������I��2�7��H�*�88��n�tmd{��9d.8G�f�R��J5�T�q%��G6��{����PL���^/1���Y09e�d��3�����3��c����$����Z�sL���rs�(5Y��9����d����g��vs�������U��N�������o��<lG�"��l��Z�[�G������*;����+�]`�JL�a�&�<�.�&���8SKu��Q&��q��4>�Ow�;�L�*�\�q��A��������$��,�����-zn
������h.�v�#�-:(��ruC���g�������x�cUV�b�JYYH�b$Asa�
w�;e�d3{���v$��0�����k�B��Q6�z1j��L���$�F��si9xO�gK��[�j�L�a$���VP)�����zz~���|�[b�������l����/k�?�=\�����x�)��9��o�Z,�������gP���1�;&�X4�^���m���H�@	(R���zp*T��aM�h�"`��z���Tp�l��	��o�*��@��[������� �j�c[~e�BL8�H�����V5v����Rq=�%��vE�1?.��l���4CE2�>'S��~�X��p��MrA
B��hG������3t���x��m�/1�"��r�uLQ��D!�!�a���k.�f}�E{U���V
&�a�kh��&�w���<��)�M���cjl��`pN��-6�U��������:��tJg4�D����%1�N���C������aN��-�#��Q�465������-�$��=Y�op ���a�5t���:�o&�S��TZ��,�F�=�[�	Ij#��h^_X;�v�s7N�w�&\�����d~u���o���n�Q�V���@�J����"o[R�[�2����_kuCNV��g����*7��\�@��-C�F�vP���9~��k��q�t��I<^��X�6�E+�MF��r��9�6�4=S���8����$��v�g �ZHp��@���%]���J���@q��Yb��[��6.��D�[6T&�{�L��)O��A/�>rh�T��YD�����s�E�
<�� ��Z����G�L�t���}����Fc4���NF��\����"������|���^ ���v�(#�6�8
�(8����%V���`�C4�mh����9�	E�^���h~����������'LU����(�=��#+*�W$/
G��@�o&7�E���"�)E��:'-�����f���~��g(�?z�ipw����=`��?e�p�W��S�����B����"�h��[�XYV(�ED��R6��
�[ ���	~�PI3���3�3�����o�����h���5����?D�dB�9!�9�e�1����,�5��O�Z-�X�ba�)���JK��3mRbUnK��274Q���me$�V����{���|OZ�N����a�E�Vx��P��&;�'w,������Y���G�_NV;t�'���j�^���},?+�a�7��
�
3x-.���v����������^�Cq�d�xR���c���4\u��������v������;O�?����Nz�f�����M[����f<n�����3�L�}x����^��Z���#	����=�����6)���4I�%���h'��3�� ��TN��ynM������aK���M,Y�� Z!�7��]�K81���bQ	M�t��g�[�����r���a2�x1�e���jGiu�7U��oIO�j'@��^,w��@T���
���r�����w�
�*�����#N|�+t�����.��t
P������;��Q2D�	
/f�KY�rN��_�0(�*�m���L�8��(�[-V<��I0l���#��@&0�YT�AA�d?��#"�{Y`��&�n>�F0r�[�����t-��u*�W":����x����B������gX*di��rE�z$��GA|�ry=O(J���I����U]!2T*�F��n�����K�MbPl�z�F��-^2,f������qB���B-�af��i70T2���	��84��0��
|^j���� ���)H��\�� ��k�p�'oS����"j�y��L`A�X8�:�u~�Q�L���z���V��I����� �5���T���F�������E�^m&���Ua"�:�U F�`fVhQ��LUW�>����q���:06��,Q���6w�Z��b������$��
scG�P���U��R��������n��6�'��a�|��r��p������@���j�Id�|�hc�UTkm��7ii��cR����L8Y�D������!,���:��5����I���]�_C�������R�Y�o5]r���-��&��Qq�R��r����1��x%�Ax�a�4w����C7`9D�|!����1o���G11#"w����,?5��0�����y���g��"'qZ��'�IB��S�nk����R^��!�2��p��S���k�bq�$!i��H�p�`���?z�\�n�|��D)uE����/�a���0r�w�|�+�CBU�����2�.�Ze6��W��$s6�
Q"��(R�lQ�4Qz
\����kX	�0�������9������_p���L?�x%Fl�Lg��c���e[��s�G*��l�3���=Ye����$�d���t�K+H�4��qNR�a��b�Q97V=t���x���`���%%��c��*��<Y�����7�O�"���\o���l[Qx�eC�z�������k6�I����a����$�M��**�����C\�5+�X4�p�T�W�2S�8/1�`��\��^���1���V:�B�T�s��gL��${� ��U	B?B���?����Va�oSNE�k�&�p�h���7�LK�� �
-n���L
h�o�����e��kYU6s��&��:��9��+��c��a�[�$Q� �gB(|�Yc���E W��$P��[������AM_�6��E�=�4�<T��X	O�>���3�>;�����Y.��d����o����� ��,����f����5 ���adY����a]�LWSV QLW���RdB_Q��C�w^��io�(�eO`�2H�-e�}���3�3<�HL�1�I$��f4�nT<Z;��0:�R�7:��j<��5�����j�����-h_qk�2A�m��J3��+g�(���A6�J����_DxJa��s`�h������o�A��������dTG���'*��P�=���	$����LD.�_p��9�jNy������n�`���]��3����2��e[��+��P�A:�q!�`)f���,#�,HY���,Ni�Q�,Zu�-�c��$�4�V�P��^]i���:�`F�Xfp�[u�[�m��@J%����_1�t�����a�����QX~�d��&���2��&���2K��X`�?[#%�������F�`8�8X��8X�Q8X�q8X��8X��8X]s�g����wI�nZ��	��������\O�s����^������~�[��������]5�0_�����Q��F�_�L������+��%�]wp29	�~��E���B�9U��,�����Br� ;�
�*\`O��6��n��5�D�B�O|�:"��x,��1��p;��)��cEo�p��]L����J�bH*���M���<�I�p�~0Z<�����j�����sc�����q�ax�~���M~3��NI��V�C���[�y�B�D)l�����`g�� �% ��kyTY��`�9����?�z�vx����'�>�y �;m��
{*�F�8�V���2L�����W� �M+�^�x�����w�W]Y���;"{OV������!�f�m��p)P@�����s���9at��aBJ8A2 �Zhd�����C��I� ��hO���Q7Oe��A[����x�$Ss�1����Kr���8yd2U�$��bar\���&`�P�&�,�k/\�����`g�0���e�^�w(\Q��ey���p�F��%�>�{#q�C*�B��������<5�i!���&F=-�7w��oZ��N�S4=-�0�9-�������Y-��
��!�:u�:��>�p���%�$�.s�h�iQ��k����7+uuk���������xB�����+�c
j� ��(7��D���$�-��"��0MR�$�O���'�x�,&y-�S�}&D���A�bO�LF�T���;LC
%/D���g�T�)^����l�5����o��A�rWva�^S�I.,�����ZyQ�;1��o�,	����p���U��)�Tm�2�U�&C�,q�6��I��[s��Q����z;��g+�{
���{�����w���[(�����9������;h�XB��8X�	^���;l�E>��<�����Wp�������,������+ �5����t9��4�'���+��'3��c����_G��
o���r��%���?����V/lz���n�{������M�Z�y�9��a4�D�� ���h08���d:7'�����n>���wNr���]����j�_b�T8��\����:,�o&��-W�wi��)��U�Be��c��d�j��DRS�W����BU��*�����S���nz��a��lW�/�Gnt��a��
U��|�d�#Z H/�j��j-:���b/l6d�zH#e�K���v�����mQ�VY4�b�C��Y�6m��x3c���Y��6��������t�����Q"��?l��er�H��(��t��,�i���V��0V�:�* ���3N��&$�����/D�O�x!��Q,X�H�r������]��&&��D�1${��� m_�����`d_���U�#�?�6�������n����
R��0e�rw�	��U1��O����h��x���4j@��3���W���b_�3�4�P$��.����DP���pp���)pGk/;�z����L�S?�.��6���0Z��#q?���X$�c�RR�N0���JvL�����R9��95)2���qH*�,-5%w�r��/�[�z�7�L��T���K��-��1��P�e	�3�������) ��/��+���>�v��	p������������SpPa w�{n�P��k��
�U�9Su(����V�(U0��g��=�1g�Gu�� ����l>�d�X��p��_'*vA��J��F���t(s:#e�t�~���6
�Do)�C�<B]�M��_�o���r+��Qm�J~�w�!������������
ow�H��������s�6)�ik���Z:Q_E	�n7���x�����7���br�a��Ug{��q���^�����L��1^	c�,g��j�K�&Z4�@Y"N�d������k��%�]����������O��"lQ�dD���}Z��r��V;%�<�l��8SG� N�u��w�=�^���P{zi�V���nL�D��|�28?}���w�c��������N�^�����Jj)���grRx��\�����,y<zxB!��x���j/��W�Y�9&t��P|0y���"$����UJ?�������,��N;izE�����jrEw-�f@��
��W_^��
Ql�7s6�s���Q�!R����8�Sy�Sg������1X`��T���,�*<c&�N������p�� �9�`��I��;��P
����wv�z\�_���0T ������;�R���C�U��]�?�{�s�s���h���/O�g��lV����2q���Y�P�]�9|�4#H����^��.���h��u)_Lt���M�^E	eY�f���G��������U����T��lA������'��e���~�G�����t�����T{�mW����$����]Y�M�z"�tyHk�o-CQ��V�s��wwe����i�9f����z��X�z�S��0������5S�L5(8O@��1:S�?��$m������![ EHUM�0�����M $b���~H�(K&���V-�����u����a^@�1���a8+���b�*�J�0��RE�����[�.��vc*����t�o����[�����Ff��
_������%k��%����Hu���$rK
6dwS�/�Uwa���h1C��T���*u;����g���JN3c�����v��Gh���H���zS��l�5��xy)��G������9P���tF) D��}&D`��<^f_���Gm�z��^��^�h������[��}NcB�oh��.jE�:%�n����&����k�k7\�]�UokF7fF���x���G�H��^��~����ubO��L��^���p��Z��*�EN-���j�&��,�T-y�Y'��u�R��<�k�;�x�W��dFz���o��������h�����8��Y@B=��m���
�A@i�\����z!�����jy�f��=r�(vN	���73��Cw�NO���)Z��
����n���k��y�#����`�QB>+�Mt���?��x���u��F_��
]<��Q*W���Vvt��p��65CQ)x����:��>:B��XV%s��N�I"�����1}�0)fZ��������X�G��F<G��s9�����k�b�4�1�dY���
T?���SxA�\�7Km�tY�R�#���`�=���A�8[��M$B"g_�-"��Y��+�^O:?�l����f�=� �qP��c��D�����I�Rs6���Sr�*]wSV�$��S!� q���,��NB�����w���>���Nr��=q�!czH��$�q��|�Q5�1A~M���it�J��s����=J�@�.����|9��,�0�p6[U��2n���(�{��oXu1��D���5��wr2����5������t��!�
��;Nf��l�����M�,CJTWy��k�@T����p1[t��m�3�������Wy�
[�}��N�y8������d>Wy���<ms3S���5�!.�����:�e^�n"?��,��)Z��1�6-�LK��_j�|o
ja��wj�����4lc���b���KHK�V�Q�����R<�h��r�C�s�|�:*�oi�"x��@k#j�Z(����K�J���;��9��
��~�a��o2�����T���T���Ue:�']^"6a��=:��&����>���a�p�
��Ml�p4���E�a�F�,4H+!�mb�����-KIX����J�N
�-5I����Y�y�m�d*��6���4 ���>o�!	�������>�^��X�d�������D�/J���m��.����l�F
�G�����mI��Nl��nA���5�75��)�����5z�O�Q��X%���J$�G!f�[�kPpzE8R�4��o�q�6PO-��(0�y�Qr��N�@g�c�I�����P���D�rl]%Z�_�����L�&��,����m����-�\h�������w��@qXI�:;rd�r���	�$e'�s��Z�7{�����>|�[V�{�q�!.�u��
�����}�������A-��9�}��m�
����#��7b��$YY���_��ZV�Tr"K$����2�����,���L	`\����(CP��,��y�����**E�*:FMQS�����h
Gi�f�Tk�5fxO�� ����rn8Eq{��s�4����t%��d%V�V�����'|����D��Jt
H�g�d����T��3����7[��S�����1�D��*�Y�&&�!{���~f�k���9i-&d�0�j��H,��k��b�������d�� v}��*�k�L$�k(U�N��R�c�u-M��#��[��v�f���>��P�R����X�St�&��?�(�1l��fC"�C�p1'��?Ua^�9Q��a��%���Zg�wZ�$l�E����^�74���y��,�>f��*Y��*D,���L���Zy���F�#/������W��$K�cN��
��I���#��D�,�J7,a�HTS��1���`���J��z1�I��MD,J�I-{-����b|Ph	]�+�v�Zd�FK��zXx��n2y�Pfr������+T^�j�#���e��R�3S�y����P���L��]�E��j=�?���b]n)gg��p�q4I)���b��W.8!v�x�Z`���)C7@�p��Vk5T&���`�y�U�5�wt�#S����f�h�	�h�Q]���N��_�����\=<��a��/(��%�)�u���kk������i��S�#���%����tqM���'��V�w�_������TX�����I��uN��hN�?R���ezs@�2������w�6�X��|����c���">8�w��M�F�.������a���n���l�C�B��~69��������rT����H���r*�;_��>
NCR��6| \�_�b��g���@�4�)#�7�Kf��u��01��n�V�8���0��o
@n����~�.���s:��'���{(�vPb��C���P�v��W(�9�����yi��*b���LU�Vk������VG���7�F�,�Jq�g��!�A�[��F�!�( FD��.��-b	�?#wF<"`=�E�y�f�
�P���
���`]����%-���������)���d�u��
�t�W(��o\
�+5YWAN�=����:�r��q�A�Hx����h�,q'�e�1�#h�Ge ^��pL^F�'GR���C�w��{�W�P�h��������5���r��B����@��	Z5B�s�e:�~Sc9�;�@�v�3��602��V��O/��r�Vp��G����=`����K�I�c�
���"�*�z)1c�$��J���+L�O�-�JR*���f���������$@f���v���J�����@�#���H����A��n&�~����cK�,����&�$�	�����gF��$Y�I7 _Z�=��\�d��O���R�8�����������#<�~�X!�f�����%{q�'q$^3�!c#j4>.N;J��j)��E�9��R>WY�y���}5u�60b�q�4g"�������x��%e�AF(���J��9Q�J�N��m�
$�eANi���}
��5�c�JC��[%���b�g�Y����D�Q�qV7�n���:}
V��y�<}�>O��������y�<}�>O��������y�<}�>O��������y�<}�>O��������y�<}�������q8
#140Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Ashutosh Bapat (#139)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Mon, Jul 10, 2017 at 3:57 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

On Tue, Jul 4, 2017 at 10:02 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

On Fri, Jun 30, 2017 at 2:53 PM, Rafia Sabih
<rafia.sabih@enterprisedb.com> wrote:

On Mon, May 22, 2017 at 12:02 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Here's set of patches rebased on latest head.

In an attempt to test this set of patches, I found that not all of the
patches could be applied on latest head-- commit
08aed6604de2e6a9f4d499818d7c641cbf5eb9f7
Might be in need of rebasing.

Thanks Rafia for your interest. I have started rebasing the patches on
the latest head. I am expecting it to take some time. Will update the
thread with the patches once I am done rebasing them.

Here are patches rebased.

As mentioned in my previous mail [1], the last two patches are not
complete but are included, so that the reviewer can see the changes we
will have to make when we go towards more general partition-wise join.
Please use patches upto 0015, which implement 1:1 partition mapping
for benchmarking and testing.

[1] /messages/by-id/CAFjFpRdF8GpmSjjn0fm85cMW2iz+r3MQJQ_HC0eDATzWSv5buw@mail.gmail.com

Here's revised patch set with only 0004 revised. That patch deals with
creating multi-level inheritance hierarchy from multi-level partition
hierarchy. The original logic of recursively calling
inheritance_planner()'s guts over the inheritance hierarchy required
that for every such recursion we flatten many lists created by that
code. Recursion also meant that root->append_rel_list is traversed as
many times as the number of partitioned partitions in the hierarchy.
Instead the revised version keep the iterative shape of
inheritance_planner() intact, thus naturally creating flat lists,
iterates over root->append_rel_list only once and is still easy to
read and maintain.

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

Attachments:

pg_dp_join_patches_v22.tar.gzapplication/x-gzip; name=pg_dp_join_patches_v22.tar.gzDownload
�|jhY�]{s���������s6`$!�6�'��w���s�����%`l+����+9�������v���\I����_��c$���b��e����g���K���N?�}���+ES4���+�StEo�}������
��pn<�rn���]�e�,*��Gkb^�7F�7�4��la��cZ���7���i�V����m��lDl��ZX��
xo��!eO�����o)�ZZkO}��`��\���5�����cE����+]�7TC!{z[;��H�dL&�
>Zs��(��j�?PQ���t|�;��<k����d�{�8"�Q1��rb��7^��o��a�J�H��TvZ������od��/?v���]��W��P�!B��kG(���u?�k�
�"���iX{��6����Z��N���`�'��K��fA��M�x�M&����I�'��\QP�3��Xj0b���sJ`b���U�;zs�����q9l��
���D�HL<k���c3�1���*�|z���,�2��@�b;�V��Tl��x�����)�|��J%	�f��hT����#c���'�c�5���n��������F�T@�+����1�&�:h`�b3*�;(P�2%�D�V� W����6]0v�������	��]oz��>T['�v�:��
�M�����.�uE���x
'4������}M��F-FX���j���H�u*�m��{����vm���dh����'�~T��L���h������'���SG$XS�Zw��.~�x����l��J�7{B�U��he'�-�d���1���{U����^�(�����H����>�_Y�MO:��b��\�~alM9G�i����D���b���.���8��Z��i���|�U�H������n�*��UE8L�[����X�����2����<���o��+93V���D�������X#�5">�����p�(;"��H9���7v��r�A��}��pV�����t�B�s�y8'w�TgU��m����RN\X���u	�pj�8��v��b��Mo����"�,o:�U
�l�����������+�.��t}o��o�B�W��"�����M�����@�������P�X����}����oc`	���������.Q]Hi�:.bb��e��n�w���U%�=�&��?�<��.~����/���=O��gg����Zt�M3���
����W,_f��D��K�]�����*�a}vT�!8��8H�P�=�����C�p��a_�afL����U�E��v
FT����A����6�5�x�l�z�\�1D��N�{(!z���_������,xD���"�+M��������{\Rz?7%)�^���5\�����)�f'P���\iHZ;������������|�}>��|�0le�B�?RC�S�S���q�Uv�1M}����[���� �=<��C��t�t
����!�<_G/h>�v�����!l�Gn{�9�xs6R�����pn���3����6]���?�0	��d
��a�?l�[X����P���������t�#�����_�_�p�f�v�t��a'��r����4���Wjl�c��O��U�qXU���B# ��<FA���}��M���h5��=]f�A5a����_b$��'m���H-������ �`��N���0���u.�w�'����CfEt�1��b���d���{4EN��H0�O��������b����RW������"�W�$�is��yk��	!����(���5qk;��J2!�@�f!�Ln�
o����%7h<��~����JW��q=��������=���B������t��;�����A�3���yx~�1A��T��R	���+F9�j��*�W���lA�����U��q����5BO��k_����J�j�]W�I�*}�$�z������`�9'��?�Y����&�K�u�n��^�jL���W�Q������&#9��kd�EFm9�A��eN4	rA��������CX�"H�JC�f�*s������0i?rexS���I���0�o�9�����f�#���r@'�$'���S�5�|E��|��>�b�G�)�A�T-m������11���|��d������1��R�]P_'����-!�P�U#�^�H�0P��o0A�0aS|D���4PQ�����^���Y��q�������ev
��Rk��i������e�q|��c������6��������������Ak_S�������*1ZD�����D���}�>��F���H��>����#}�H��6��
�B����ey�#@�b��g�����J�o��&���?�)�����L�0�p��p��.?����u�����I��=��%?����4�����
��&=C��RQA���cn\�������G���1o	#����7��]���M�m���{����7Cx������;���0����P�;���05�l6w��v��:�?�I�q�iZ�����4m_����~Jr�(��6:��x&��@���?�����|�fP�W����f�T@c��hc^j���J�5��23k�������\;+�#��P�;AQ����o�����p���5��WU��
�r�;������}R�Uo��R8}�}e���h���-43#�����P���,��AOw��v�=3���<�pJn�����fe����uq�'�����\�k����Z}�����5���S-����O0��p��w��cT���pyv�N
��#�"��8��X���z2F�ras��q/��2�a{��7������?�m���5	fa��e
��g:3`Q��C�M*���zn��Ow��s�r�t> ����=���c�I��D{�$�%��Q��DB�;2�c�a�6�������N��Vs~=
���q��ta�yw�E��U�j�P���c�\�]�$�~���A���6�%V���7�� ��9����i��#T[5�4UU����&������ke����C�`������~�7��<���w���0������)U����G:}[�
���pn7�Sm�A?<�)�&6���������!�m���tF{�u�����^D�2�S��Ybl+��$���Gp��#3�ok>�3������fE����`���B��j1�DBSER�BH�^�JZ��v��(h'
���u!�q=��c�KP]��R��t	B�B*�9�1�i������;gP���{w����x������Y��4�������?��	4��z��@K��oXm�\�������\��\���[�ZH��U������H�Ef������[��H�����N�x��
��"��;�x�Y,�|=��#���:E�DG��:���#�2��aw��*�^�����D7PfJ3�vW�`vHR6���Z�Qq%�lYs��(�/���R�]5�b
�6�����k��.���l�pv
ED$�F�2��6��Ne��Jo%A����Uv��%��%.Kz����4�5p��_HFJ��Ki��z���*���fIij�vYi���KB�i��jm��;�����#�&��>F�����+N��
�������^Qtv�?�����-���6��?�dL1�x!c��g����+,�������P�9Z�q~�8�u�>����C��}1����6����bI��T�!9���D2�>+�h����
��wK�Wh{�,����J��4T^_M�Wy5�v�a���Z�~�Wi'��4Bp@�@O��y=�vUQK�r1�z/"}�"��?6�>��;{�AJ����f(HY���1�y���b���I��B�6�����z{��[j�5���bQ�8&d���5����0������2(����y������E��<����-�������Xi|K�#��_�o^�h`�����Y�r�`
�\0\c6���Z�w�q�9
k��K������2�7�Z?�0�!����������WT�]������-���{tzO�����D���"��@����N7k�V�wE�B�_cNN�J����9��}jt_\�5	������Gi
��������k��N��kKDwO��W��
R/�50l�>	�\0�D�Y�E�f�-mvC>�����=��
��o�	<��D���F Q#��b�a3!H��R��,�$�� ���&Y&��H�U�k����n$Z�v-���I�D��D+M
$Ij%A�m$9����=��
��oMY����'f#�DU7U��g�*���tBC}_
������ma���1�}�en��i�������p��/�7���>�_�7��Wy�%��$���E{�m��@�t�}2d����^z�E��/4������=!#Tz�?��/��sr�"��J�x��f><���-<�/}���ty<����� �!�����;�T�TK���}-��s;�`y��0���@M@������"|8�v-�wlp��m�Jy����{&i�$��DnI��L������b;�2������b�����������g|=RB���yY8�9!,�����8����:}��V����[�F������^�e�� ���"&^J�0�e������|?$��5��l��I�g"CB�e/�^J�3(���`��]�
�KA��!I6���
�"���$���^�b��XJ0����.����� s��$��p��j���P
��)���f������3�9�1����&�SK9�O����)~�x���+����^+yJ�{�j+��������Z�S�r�&��d�&�V<��k%O	r:Ym%s<Y�����H����,��ZQFhU�(;��MM�S���g.������:~U��h0k�gl�7�����/�>�+��c�b���[��_���t(dsAQ�C�Q1����zc���(���O����t ~����r��C����U���j�5������3U���a��T�����J<���}0P��L��zY)�~�H�|y�b�J�^���D8���������/l�g�x���=e���D�]1�KA{iB��U��10g$������"X;%�4"����a�h�F��U*�,��=�!����Z�=�[8�+������^g��&�Ml�
R�e���g}�m5'4(�U�����=���h�=y��r�����p�����-(�U2mQ7���e���/�(o�m)�\Z��	fI������P���4-Z2%���1�
�	�H����\��m'����������Zg�1�t(���`�_1����k=�����:���t��E�/�H<w���e�es��
�U���-,��%r����3wq��=T���"6�2w�������S�M�M��bW��d"	Z��Y!�h��$X��ou�f#I�)[A�t���Op��{�=�/���K|�7���P�p!0�^�����h�T73�^����>�#�1 ����9"^/���l��g��5�D�X�7�X�������O�-��#�]�$]�@��a���x�4A�����M*�i��(�-����Ee{!�(���i	��U��R/�b���C�4$���������-'��F�����e�6vIZ�F���������i���[����b��r���6�+"����c�����9��T�b��_�Z��%���g>r��V����M��M�/�J)�D��������Wv��� ���:}��Z��L���l
��Juz�o\���X����	�����,�O��=�gq�S|�N�����}��!f��B�C�������_������a;��2B�y�	?����$�d���'�6�x�7�"*��"���$�%+�IlX���e���4�9Vf���8�n�����M��&PZj<���R����%�#sS[9U�$�y��L�R���K�V�_����S����#z�*Y�O+�T@����S����&:�U���d�?N��o��c?RF�$��E�Pf�d
x%<���
��x�ri,�C y�S�{NTDoaL�i%�����`��J�Z�_)���C}T6����Wd�����"�2�����W�����?)���F�3�������������A���D�S�Rnx���/�����X� ��������QX�'i�dk<5 �����������x8�V�&�D5Q�)�x
�*�����<	x�a����^�b]p=j(+��~����J>��_���|@�X�t|m�`�A7�p�b
��p�$]*��d�r�$1$��b�Xb<���Hn�����L��x�`Q�0��Z�[�/}E���]�b�e������C�r�����}�w�x��^U�w����CB�%4�n��>�pk���������lp�F`�:���*���QmW������on#G���G�����3��{�*R�����q�=�P�����Ol0H�ds��4|�v���g��[��@�(R�&�z@��e"�R��2���f��~w���������A��8�~����s�a9a�z�W����
X^(��)�����W�����5��>�}�os;F����|��'4�����S�,W�M9^��y��������,?��U��o�/��_�>�B���k��ZAN��~����d����l��������	R[o������U���Xn��dgy}�]�_n-���kO���M��^X��Nj�-m�j�T�\�rF�V�611Z�g����r=�isr1Z�g4m^F��3Z�0Z.3Z�a�\�h�=2Zn`4R�(�
�X�5�'��@`��x���Z�IL��D���D
k��ai��4+��
qHn�@��������&��VE;
_���-��������cH.��^�8��K$	B�������~����<e;��c���V}��Pi��f>�}/5� ���R����d�`�+�4���M�7����c��O)J��
�p��h�����&X�3��P���J&�
p�
�p;Z�#Xl>���K�xX�g��-�d��f1�@6j��,ok���V�8�����!�z:���&y@�!�#���h��' �fq�,�������0
^:��|�
���d���m����lj�z�����	H�.������)6���s3���Q�N��;G�������kj������52~-��7��%�p�������W�<}�w�H�G0#6�!�D��1g��+R/P�����5��(����iux+m����eJ*�n.���w�G��$?�a��V�Q�>�7t�3�htD-���R���U�{w[�u4���$I�l���7d�]G���+��F���o�h�`?|t�H��M�1)����m^��<)�lR��I����S���wR��[W�Bh�$I��
���
�_����>�h3)p
mR���NIV��7R���_%4 	d�6�]l�,�
�-p:�m`��{��V�{�`�c��8	�e;O= ��C��v�]�5=�A���?�X|��>w�����P�O ���9�-���Dy��8%���m`���"HRB|��>��cx'�Gj���>�S@B	�A)��� $6��0��	"!AYB���5����^@
�]������`�����}��mO����}
��GZ���������a8�:;a����3������b�N�'��#��
�,���r��W������u�mAi=���h(������P��W����&44��	
�E$44��YBC�ACq	
Mhh���G���V7�!�q���xN���������y:p���h�M�������3��4B�~��3�y�����X��g>8S��~��'��;�e~�����LB�%Ih{:�UD������{��	����J��*�8���_h!�~���E���02���8�N	�	 I��y==h0�S�'����(<�o}��a���#��6����xOh���`jp4/��w�4N�G[H�&�F��`�1&O0l3q�!q}������#I�8�@r;Dw����������n�b�� ?�4��:�!�)�*.?�+��"�0�G���,���4�^�����a�;�
-�$
�����n�	S�������K������s^�\Lug��	 I�vw����h�T��D�Te�R��E�Z��B�9/@.��3�����E���T��4a��0��RY����~��hXP)��*�xV@��
H��Y=r<+�G�gED��lg�������i�1t�c�T���\O����h��pyIF��^�����^��6�X=���FnC����Jb��B��p��
�kI:�����I��3z��������� �l�/�7?ePX,��l��
{Qn.I��$��z�X@TER9J�S�|�u��L����oU\�5
��5;Su�����\�P'u5��(
�B���qD�5pS���V	!Ih3�������6C�)���2W+zH���t�[�D�7� ov�tw�Od9_���G��&�\���L��R_��,�&2��o5xOdb�&2��|�����������|�D�MI�D�w6����8�0���C�:�ACA�MdE3�f`Mw�DV��*�Z�BF�e
Y!LdN3u��R_����&���D1/�j0�9��2�A�G7%Ym��7!$�Y��D2���f���,����L���i�}�P�^ ������J��F�G�W�p���<���/�s��p�7"����z������A�^S���C�-'��0�l�4���4���<�<{�����u��|�9[�.�-+mc2]�Y��nU�����:�/��v��m��=[a�~���e�KY�Z���s�.�O���6Cyd�5���m��������r��%Mh�EiJ��?*U-*��VF�P�BV�z��TM|������������.�R+�Xv��g��m���F'���lt�����vl3j��#���l`�p��7!��<2���M^:�����c<�����:r�Y��SK[�������������l����ljg��0�YFmS�.�h!���,W�q3�[��t�u��3���Ng@�U�g�}0���;��E�[�Z�����E����g�n�n���`{�����3�3`a�
�u:��<��E/h��)/�����b���.��n8��:�����0����`��O�G��!�����A`��2XW�v)�����#����f��Y���`�s��4Q�6:���o�����Y5M~�?��
��9?��<�����
��c3`SgV3�f�qDv��r�{��
�gN{��3��r��@�8
4��y�q9�����mjVF����zS~�.oo�p��8kq|Bo7�R=�Z��E00A$�;���!�y�'p w�Et�:�2���{�Ra�	@;	MSe�!2�$I8y�(C�.j���D
,��`<��*��i��9����	2a�O�C�E7��S
�j��?���j�A2l9Hp����L�&F$M���1R�2H�c2H�!D<I��O��cT����0�A�V����h
�t�Lk���d�r�����e�$0�H$��V]�OIv"��	Nb�/�f�~��u�<�
�!J�X��ghi�-�y�u�lru�&�
�=Gd���:�GyE���PM����S�)�M!��7�C��*O��)]�OA ��'��)e]��{#RKE���.{���gJ`���-�$a!<5qDR��T*�3��Q���#f���36b����V�G|G�����^Xe�7b���$p�Y��x�+�3�<�=���Q3b�������( ��w��k��)�U6~#Z:IG<���1������{�4�������lS����r������.�,?�?n��l��/?e����k�Fm|��K��d�������]v{s�R���2^�y���������j�.�3�S1~<�c���M�
���s�����0<����>�~����r����\�?W�#�x2�k��^�7y��w�[��Vy���*�����_��������K��+W
���@�d�+��J&��
5����d�4����?;��c0��'?�I������'�0�s�Y��R�#A��1C�
A��J!IB��t��d�#d��8�V	-)�|��o��N%�G/��3o�I%�������1<c��Y	�y�5k����M��QM`Cf��g�H�9�`p����L�}-~
qm��H'��3�����u^�\���^_)#Y�3��������T�+��3T=l-�dK���Bh�s�*��.������R��p���AkD����5|���������"6�Y#4�e��������J�
^#��u/�	�M@�Y#<6�rU\#���$�����������$a�h�Al��{>H'Sq��t����t�y6�L�����k�c��8�f^F��v��2�5p� ��-yN���cU>�G�Q�{�: ��r����\,�CWj��9���kp:3�� �[A����v����}ss9���f7bs5��X�r�C'�����&�a��rv�q����9E@\���rS����a�2:������3�2\�����""�}C�������"4N��)���,���W2E@<��`���c{Nl��G����n3E�����{_w��
���}ss��H��PGqs5�����k�L�3������:u���x�p�cV%���]k��e�F�XL�m�k;����]��$�w��iA�I��mS�����a8�<��Bb{��J�����Gp,��Vp�z�p3��g��=�:���SK�����,�,y�&����������4�2��Bw�Q����F��\����������C���k������:2�5K����63���Z��4�2�kDg�Q��M�����'�q/�	�������
��MC(
�0��H���k�����RX%���v`4K3���fo��(�`4oP(7"���njl�vF'S����)i���}k!�+6�����w����Hm\�s�"�h�z�8%�� 7O�{KM	a�����'� ��1LSC��V�0W�O��%N�',�;*��^_�!��}���Z#c!�pTco��%D�^p�Y�������~������q�EN���r<��y6��G�1�����x��������7��n���2o�l����������=�qT^�}�{�#|���d[3na��
���	�7K{,��������{��n���w���f������8G0o��:���7wh�n��Y��{H�����)���A��ky>�$A\<����A���r�
��}�vM �����5�.^�5��A+�����=��=�^��W�� T�
��z]9�["l�������T�V���R��FZ���������]&�m�I����PB-���g�bR-��E�*o�U���k|���d5�RV���S�_��k �*���(�<���:�<z��?�)k�j$A����'tP�E�cQN�<����>�y�g���_���G/�����������$���Fl,���m��T�sy{{'I�&_<��v�g�������zV�70��:q(���[��=7a��c��n��nk����k�+7�=m�V^s�7cU�D�>��K:]mj�7�?j�o��	��/n,�p��o�<m���j��g�Cft������~�
��O�=w?
X?iQg?��F3H&V�O��~2+�fhD�T�Q�\Q/�8#��@T��}b���z�%�����>��Z�0hTN��	�R�=���g��J�A�hr#��A�h���'V`j���;4�M��6p��J��e��`P���j`���t���r���
C`��@{1\��&�7i�I����4�P��J_�x�@�����������O�W��g��������1��@�c������y����j�~)]_Vz��R���4��K�O��|7Ym���������g_����nQf��aM�{uuQ��������S�K��d���_�-��e�NV����q�������7�h�������=����7��8~�������^��^���������{� r��4��W�F��1��c�����>�:c�,�:��)�uJr}�����u��mu��M��y����Q[��?���	��'tV�T.��d�zr5/��-F/^��G�����������Q�\��Xs-�\+0�T����/����:� ��?f�r���[$�����M�lH��|.�����.�*���ZV�J���MV%~������[����
:�2rg��|��M+&� &����o���<�s=js��a��Cm�`>���a��a.�a���\���=�an�CR��y����Z�"��Fb�<�����v�p@��P*l��6�6�#CQ
=64�����������J�&�M��Jm�o��j���i��g�R�q���Q�")0��P��~��!_�b� �R"�,O�� ��,�~|iqr �t�8��
\��~�(qq���
��U]�V�G���p$��hi�M�{g�~7���p���6	P3*�:wDI<�����er�R��/��r�e��e�lE���j�������pop�ojc��H\&6�
R��J������ I�2y.���m�!|�-��5��������>�GFOt����F����|'����K�S�w
��yg����������>��N��Nk���tj8���6�Q���pyI:^}7���~p]zL���� ��T>�Pw��[���t*Q-��k~���z�8��!�Kr��y��������H����K\.#5��A�
���!�|��d1������%|���Y��zD?7M�>���������i���n0�������t�<�:�X����������ss�ss
�����2c[[�Q�\�vO�������0��

����!�|<]O�.���s�s�s#���-9���dm	F
�s1�<9W��G8���Y��A�:57�j����q�jduEc�?����6���v�U
��������*��x~��I�C�j��c�4+DG�������5Jqb���
�Rd�\c�b������N�����Q��o��C2�_�y���&��k
�(S�<���	�*�3�]��!��<1�+��
�h~#�����|e��!�S���~N�<Q��(�I������Q�*��,����+�bMGM���I�>�[�\����=��:�����s�����z�\��dO�*W���s��s���\�����|S&���Y����}]�6L������j��WIO��h|�pn�)�*$�$Yj�I}��e��*�y�M�U�Vm���G�Ys\*43g��;���=�6����2���>�m�{����K ���',
���9�����i3�v����f�'�&����x;#.'3,��A�>�AAvW����}��J�bA�l*�������.�{��;�j�^���%]��3����>��O�j|�����t��M������@��)�o�H��v�SM������x^5`�
~�%z]�^g�����A�>����,�T!q���Y�rm�Dh�����$�O��G������C�6�2���a��*{Q�Um��
��n�d���-(�Jdg�
�?��8��|��h7#��l��?/�n*���"$�s�=6[���/<�y�5��f�V
��`�;<����4h�������}KsH����R��V����M%�<��!�$i%RBs���;�C�4tC�}���Cio�5��]�������������wY
��o�����H	n��
����_���u���l��<0�=h�P�hi�	D�G������/�7b���n���0�]4_(o$��z��\�&Cc�����������6�5_(od`+�t.g���O�/�7:2���]�����Y���F���
|����������6\�^�~�|��_ ������O'����9�Fy�	��d`��A�����S��������g�m����.V'M�r�izW�N���w������������v�sm9�P��!Q��k��;8oo	8�8`M�;ft�7S�I��;w�Y��M�D�'��z�	�����PKA�E��1�y��%R�$��C<?���F.k_���f�
�cy�H�vO����H��	(�P�;D�qI�BEs��
���=�f~XZ��������$���9����L����k��l�j1�xB���b�&�`1��Zx�(�,^����'������f�����
Y+|gK�v;��L�<�0Y������+�Vm]�V�63����MyQ��i"��_g{�2��X�cc�������C��������:1�����y�0�Y�~�T�c��������d������1��Cr��N��1F�r>��|V?�A74S~^?�Q���	?pzQ?�a1�Rr�'��8�������4�%|G�Vn[���M.
�G6�)���z����'����	�hF���m	)NH�_)	)���-lC���'�\��?^��#mC���'�\��?&��se�@�+���j��W��:����� U���j��M5��Q����E��@�z4
��H{����c�2�]A�$��]��J$H�h�`�*�����zei�.���bI�c�[r�ns�l8Tl{%�
�@�����6����r�S��1�}���#���C/��s��4Z�3HkH��W��@BF�9�
C[�,��t�&jx~������0�1"�R��}g@�u;��%�$y�q�����$dD�i<��`���L�&LNGr��f����Q?�+�;�_����e���4���{\#�SZ\�Z�����C'Z"A�=y^v�2�)��\���F6�hf[z'n�������]��L?t���CG��e�d�����3r�u�s=���^Xezg���	 �`9(t�#:�8\�-�'-;+C?�T��k���K2���*���;sHnOI�A��������p�h8�a)���y�W^��e�cC�����aps<,n��E�������`1�������NC�<������\'HtO=���'7L���}����@���y�F��\�����5g4����H��-*>����Q����9
�M�U�$��Y��<��������������x�S��BX���������T�I� ����D��!����15��'A�����)��j:�����S�[!�-����Jr�T����(:�
�����
���n���<�������o�6���6#����ni38��v_�jEI��N~��f��������)0�k���u���KS`����R�)��S`�M��4��������xO�b}�@rP4z4S��n8zB�<��M�!��)�a����!I��8�0(�a_4�]�S`����kW�M��4�)��j
����XpS`!M�7��@���V��)�a����=��)�jP	��	!I�����������L�����N�2��N��PpB��gN����Y�6��^������9��
������op"?����o������q�����<�l�Ig�fa��Y��J��.g�g�*I����?�j����M=������w#����4P�3b��1h���d�vY����8&*������������6)ft�m�������;�Az��Z�����f����Z�?p�� ��L����m�<��!�,:�c�@�1H�"���@[7���>�Tb�r��S�����M<�a[�=I#)(�1����s;���?+����l��a,�m&p:�������u�
=5zE+|]G����]��4���v5�y���v���H��H35n���G����\
v��K�
_�4����&��&�fO%
�Q�j=���jm���j��Q�j����j����j-�Qoj]��hj���[j��QHj���Aj��4�������m�O��Q�wT�J��9���(��^�-tIjVR�44�5��2T��O��-����[�j�fW���*�.�$Av;5u}���t6��1�C�,Hq����f�6�n��U��

W�$	]uh<^��ch���x����W��U@7���l����T����lO��$�.�&��#���'��+;�JT�g��e�i���Z�q�z�&�r]�B��u���rUV3K��&�|.�U�����n�������r��_�6����G��
sS�����M�~�����uN��Y����z�nY>��)�	�f�X�U{Qv/R�;kU��)p;`�
.��J��X��M*:T'�7�{��������c,�g����]-����Pp�������ZCX��,o��'��H�0�}"K����f�����*!�F��F�p��	�U���"�^�Y��me�cyV��B��
~�>b�4�����C[�4�}�&�4i���������"�>�����e��d���_��k��:Y�>OV����W�o��T��W�o����_jsA��<��5��������/?\�����yT���;�w�-<1��Cs���G'$�7o�_\]W
Xu��I��y����Q%{��O^�D��~*��j�)��r5/���~6|�����y���������R�P5��Z��c�k{rz����o�_��=��{.�Z#H�WUE�����~J���=��{�P���5��9�~A��������}���	��F������5�����h�]���
{w
A�S�����@���5+�.�yR�����.��.4]]�����F��L�];��?�Mj'd>+NN�9����	e��Y�R{W����uJ��zs�j�5t�P����+�����|�,���6s�<�m�.1��!�ay���e6��[������?���j��T������r���-@�H�G�a`�}�*�!M�*��4y�~����X����R�X�Xo4����\�7��lz� ��g�eo����xDT�(����8E�����v���V�����6+��%�U�Y�)8�lW�l~+�%7��R��n��Z���������^Rn���QU��eZ����3��Y~,�������S6���xJ32i��M~0�K6��PS�9����[P�4pS�������������������m�Z�q���d1��=��]��X�1�R[q����N���d��s�Z��w��	�eFb=R�k�TUW����h�9�i=�(�&\�u�P&���
�e�V?S����0z��p��<��yv��X��"#;�w>��u��&=�|/#H���t��ygY��k��~����:B�N�c�����A�3��5b�Z%�LY��z�M�	�"fV���Le�Q�l�5}a�������ft����1ntD`����M3>�������'H�G��t��;>M���5��
[.7bFx�h	o.=���D"����:�p���� �#�b�n�������/G����`����_��������\T�M=�P�`�a��%�5�U���$F�_>���pN:�MN(�����53�t�"���n��c�WK�!�������W�bj�D�bj�dj.��$�����cc�rY
����
�U�����B1��<���{��]��P���|���w��?��D\@^��U9A����h���A����Z�����/��"��Q;�
la�`��wR�mi+���T�I�P������e�\?y���G��<�9N�q��[M�q�0��<r���m�����s���;'�J���hh]�Hy7Oy�����\-��+u����G��<r����'��~'��0y�����/�����6���k�ca�qT���MgOn��F ����6o!8�.h 7�d������6A=�S���[�;j�n��
*�� ������F��\A-H4T��)W��$� p��7�%��T���b���d�	�J!�~%���'��)��X�M@��Lq
���t���qe��#	bA�^�����A�5G��;�#`�����������x�Gu�ma;x�uTG��{���[��=D���^m��k��[��-#�V���Cn]��9HV�����nVm%GJ[����J����&I�������KKt�����_���G�%/K���%Q����������>����K��\�Iu}�2#s��\�y�sB� �*�������'p�r��i�\P0.(.(d.(\\��i���JzjU��B��uS�C����A5�P�(����A>@S���d��l4S���#�,����t�����'�U�zB�z���4�W.�mVeh��B|�F��y��k$>OC~���
a�%�����������W����%��%����(���>�u��W��p'`��aq�\Ri���.=P�F�E.I��
�
��Rc���a�W\��\���E��C8�Wh�0-Q��1��U�%�������;�z=Y�	P�����HS�9�S��0
S���q�0��
~nW���\c�� w .@e@�EKH����� NC���:��
�A��A��AVp���!h�������i�4}�p��%�|Ja(��@�
�!P��R�z�"p�Z��g#'nW�����|�\}�w�����=x���=
���F
R�tO����}/��{M���I��C�i���V?|��o	�)R���9��Z�eQ���Y��F}����9�N
=�m��+>���O�|��Wq����28q��?��G2u����^k�D�Z��M�i��m�nU��
��!��r�/r��r����s�s-#�&^�[�c�H}C��2������D�h��kZ��$&��R����N�1o�Ys-��&f
w����3��!|�x��Y!
h������
�I4f-�	��f�B`���Z�3k��Y��Zf�������ci/x�
�����\����L"�L �5y�E�rP�G �O�`.,+O`�|`_�5�F�0���
�z���&��x�
����2h��[������q��1�#��h{���M'G8E�������������������9�-�����	e|����
��>VT�-��z����^�o���Pp.B��kj�-_����8[�)i
��^W���|��vOp�=��v�)7�Mj�2�a�b�����u��P&���1�
�pFa����)J�$9<�������`����ij�s")do�n���C���h�������\�6�7�r:hy��Z#R_���i
���Z�*�� �@kJH�XO�I'����9wM�����pNm�c� ����=6a���5��<����}�����m�3����\��W�
6����~�vWE�L�f�2-�8F"�$�]���um1S4N|S��G=�&��"��+��f�h������XS�)��1�g
k�Y��k���L!�
A��Me!~���_e�U�5D3���c�`g�HD���������-f���r��0�#��sRe��XT��&�����U����n����j�����X�9��6V���#�6%��.����F��-B|q�UX�A:�O[�����
�u�����z
������$��N�6|R&�������[i�!m�!m�a��d(4H��L1�$l���'���TK�9#?���@�p�DC������h������k�����}	�j����c{G���(�IHc��C,E�������QPxC���d)���4�(�;�uJ�cH	��W#��b��������	|k	�������6��4�-����~�s��U�����f��vM�n7����\Zv����ZOwg�2Uwn��7K���G�t���7���\��|�U�!� ��&o�o>`[{�-T�iv�,��AT���~��l�A��Xf� �I,�1V�%z���@�0�h��hG�.&����a�T���	��k*)Z�����6��6��*j��H�\o��5P��ha���5&���Uth��8k"�$�0�N�;	tz���C���Oc?�}����&�_����y�*���#����_���x�����U����������W���E5�����XT�|�������_z������z*�M=W������'��*�����C�=��K��OX5�=�y��wN�_���O�5�}J�:���)��'�Q�$�(����?�hNt���$�PW��rA:g�5��Zq��6Q^e��9��E�+3~;�S��9���]A�������aZ3\`�������Y�j��j��|-�T,%�'Ht��N�������I�����8�����{����X;���f�jE���p�Z�PY�QA���gv����'r�����"p��V,nW�%7A��$Y���
����MR���rmS'���:�-\���J�pM%�����q2D��a�@v�C�Z�r0�m���+��e�1z0�{�VBZG���O���eX�z��_����3��q�hi�Z���j����Q#�jZ��aZ��
H(iC��u�n���#"�|����W����9���|�|i�?z�/��d$��2b�GM�?���#0k 
����5_|<]O�����!����/�/�6|�r{�����K�+�_�-���K�#Z4�Sw��L�;�����!x0����X���!�9���Q%C&��d(�R��������D���%S|K�<�H8����L��l2'�F��|�\�?���X��B6Ulb��m@�R�O+��4+�F��"�����/�M�$��������K���NqW
��N���UoS��g�}�{�Ki�mjC�
�I �7m��Mt�Vv��A��a�w~;��b�-Vr�8CR'BMC�n���7�w��.�~�/�����.�8�`N��@���r�8"��	�O����<9E�������:�N�yNi�
��M�����rs��o��
���v�B���ad�]U0����m������j�b3 ��b�*Y}�{�l�]���I���N��O���#D{�kQ|�����"���5��������9�D�W���+��,��2�t�Z;
��M���NN�fa�~4����i��U?����E��U?�@�P���R�Vo^�����t|��m	�"yq��������^�kq�Ky���{�n��
h#b�W���!�{� �
����x�E�f�jH��;v@��/�����#��d�A9NNr
��Qr��P�r���4(�F�)��g������PE�r�t(�����?����6u��tq4Q��������*�������r����:���MW�l��,����>�BWhQ*I�lb#�C�p��s�t�P�KY�E`�V��2��'��c��p
%�'9��6�Gl��E��V|�2P�/����`��F���^���|����H�N��9������B�����&f�D���c�?�;5����m���@��^����Mu�:�/�Y�������z���Ax1����">	kj�O�o��te�3��y#f@�M]�p{jk�#�7�4��98�@M��H��x�4{��P���Z�|j6�%�9
]��
��Y(R(H�����&���;�$�H�M�

i��u>5�A,������"E=�4��98��@M�����LRI0r<��������F0�JO@��E&�ab���WvXk��]>[V++<^~�[����v����l���s[�~gq�I��/�_��
1��\����ls��<��eQ�6�G����~�lx�F���O>�
��
�>������������J ��������Qz�aC���u�������W����O�V����4Q�}��s,(�#N�,p��R:���b�}I�u4���tE�%I���"I��"����
vH�6�tp>i�qQ��qQW>����C���tv�z'�����MI�z}DBH��r��8���~��vl�c���
�������G�7%�c	!	�9�uf� ,�OZ\B�D\B�,����q�x�<��� 6�h���p|#"!$9q����X�����z�RUY��,�&����N4���j�^�5>�k��8;��Bl�U��M
j3>��P�V�6��(��y}Jg(^���rU�r������ld(^��3��V��)���
?C:��a��j�r��6h"�r3�6M2��� M�Oa��P��ce��/����+���$�J"	�s�`���Ub���#�z��Zt��!z}_����������Z��t�X_?��8����}��h����g��R����_?0��w�Fc�8x`+�L�
�>�����1x������N����%-������+�/����CF�;l�rQP�����_bR�6	���k�&�-(���`	����&)-(��6	@����&�4-(A�)�8��X�ed���2�@D~Y�#SF |9���<k���A���abi�pg���x�=����3�C����vY~��&��|�=�oV�_�T����U����Q���18g��w�*�	��'�}��wN�_<9fa��N�������N�a��t~f��fSk�����zu_�b�4���&�x����W8�F��~����:�.p?�V���&���H��v��+���lDS�/���W�_�����>��#G/�O���ME���o��	y����F��( �i�\n��b����$�]/?	�'9�������m�J��H�;2(���-H���>1=��nv�����S^@Vy��O?Gbj^?�e������@���>q�S�D��p!1��}+L<�9�X�����o�3CxnM��,��KXL��h�j��FrfY��hA�Y-����.�k�1��Z8��d�I#�����!	M��L�l�&g�D*���,�]���u��L�26S�����qL��/0 <�7I�����;����G�������,!����)��D���������
�5wO�3���r0z!D�e���s^z�2�+W�,YV������/�����XI
��x�xio�.h��LZ�v���6*'X3aj�@��]�r2!YKL�WN���7n{��\���r��VNhD����`��QO����M���	��R�r��-R�Q���J�MG^OV	����A!���x�A7TU��04�}\�[��]��1���|0���i��/rr?����H{����y�6����b��C�Ys��cx�T����+i�o�U��$�
@��c�I?��$�F����}g*����d0��Q�����*�����N�L�?�K�l��[�Is���(�a�3'fv3��g�b�e��X����f��9������{��Y����gS��v;�Z�4�������c�k�m�������;����f��$�_IT��GbP!�`��uI}|��P|x�V(�:�C��qK��G�jK��ou��:���&����b�?������BG5W9C���J�a`4�;��
}�����=
G�����l%��z������c8�G�r�9���~��_k��i�S�����v���GlS��8�f����X�(�m�mj�x���	���/�'��G8dj�Y���^e���	�R^_r�>)����w	�[�i41�]��=��A��k��/�`aR������x�8��S1
^����t�o=����K��y�<���f��d�����@�\��Rh�Be���$
��*�*�6r���T��&L���e�&��x��&���^O�kT��U���'���'����'���'����Ws��9���}����r
�M-�j�x����4;����\s�fK	��0?�l'�
?�K'����Hy��g �E������y���v?�v����J=��% ��������[AK��@��o!��'8��o�I�y��(�'�s�|+�'�ibK����<q�j�'^��r��������ltYs�^$��W4'a������\Q��9
��D{�V������D�s�Hl�M&os�M,�PY"�������>�l�!k�4d��j���=(�~`���A�����E:cQ������Qu�M-v%����<�O[�T-�%��H�}�p��� <K\3f|���8���(JN���x��'��f��t�xn;�G
�'NA��o��r�O�Ld���?��,|���c�k���Z�N��O���+����-�
*���y���u3�[�����P�W���)�F�=�D��z2����2RW��O4^�U7�&�K���'=�n��p��/�GSnhj����kUGL:�����S�J����?��l,��4�2`�,��H-���z�\)�J�$� �����%e8d+S^���+�g��[ps������XM��(?�s�#r�s�H'������,�
YX��1��dE����8�����
�wn�w�b5m�^������O���Z�������+X��)�j0+:���dEgs������XM��%���K�?p)0�D�5��?2�����9���}9�UcXl���Lg�%�M�`�g������k��c�1���Uy����qg�t��.V|,�V���jD��M6��������6YTo���r9+�`���z3_�6�����~y~}qu~y��O�S�l�A���}��&���(��r�����G�C�e�78d��$V
~�8[�ia������$�a~���E��O}��PbG����:��a���wyq���'��"�bxUr��	5��>��8T-3s��M�����j(S�*����`�B�gx�H����I���n��T�M�v�@��K;>�u���RT6I�>wm��"�~q��^��z8�n�PK�kc��r^��|���N�f�h�K�������C�>�r`����I��e������C��
��(�n�P�W�����dz����N�f�z9����\���z/	��g��_�O?�����T��B�A�@���aB���_����0i�$h�j��}:��_Gj����wA�U�-9�o��:R�6h���H����q���#5���H�
������o0�~)������.{�m>���?����lYV���6[�_n-�A>#X:�������8�����0��ih����C�kpoB����i���$.����E��*
?,������&0�KF����������k#��oJn�d� �C�,C���;��n����M�]���S@�tNIOL"#u�K8��+k�}��|� Ie"�".�4QQ��9I��v�����4lDf��P�:���|��<�k� %Y���:��0�UdE_�k����W�mY�+��^=Vte�v���z�X������(�UcEO�k����)��FRI�w$e��Y��|�9[�.�-���d�(�����\����u6_f������~�_�2���n�����\���[4Q������m������>��y��_��)a����uFeC��-~���K��#�l$��sn���Z�;��H!��
mV����_���n7Y5-d,<^�.H/���S<c�V���j���������]c���1�'@6R��F��f7����K�?��=�#4nt��o-IK<I�F�Ji��J9I�f��Q����^M)�Z�J��v���8�v��k]y�qqk#�
9����a����	H��W`t���.�Z@���m��,K3�$w�	��3��$I{���%�����f� �o�������:��]<�N��'�y9�xp]�#�����|`=	A\�s��u�An�$�:�u�u����xp�\O�s�t����������\�c���u����g��Q�h2k{��gj	����i�a�D{L��.�p��3l��c=�EK����}�I��#�cU��?`E��O�_�J�;��@�=�Y}�c�K'�S�KH����^4�1:�m9��q��c�mt��
��M������/��S���B	L7Y~��n7�|�������������������.�3�Sem<��4����������r�s�\ax.������|�����/77�K�^�B���lK���\|/��W���{�o�/��_d6�~V�)Wr�J!^���Lr%�\��"V������S���_\g�a���������]v{s#�\V<�@nb��!��D9����)5��9��q�0��>r?�I��8��fJ&�	y+���L��
�����8�����N�U�j��^�k�c��cG+����g0�~&��K�7�	���mXu!�2G2����0�8����&):���k���2x�hX[F��$�]9�wO�_��#�����au3�����y�AQ�!����zf�y�����A����$y%�P��(����*~3
� *��z�8�?p0�!��%:m�E7!�����9��w��A<:+��x��_�t�hM5I����_��J������5D_[3aAk	)$B��s`�>�a"����W��=lm�d]���������';�R�+
��;~����V+`O��A�N�V>m��7!'P(��� >�!�M7�������j�iK5I���9��������������f����\\w�3�o����&���; +�R��u��uI
|
���)>�8*���@���K�f�	�=C�>4pv�9�^��6�]l�,�-����z�?�8��\�iB�'��\�qj"�6G��\�q:����s=��E��t���.����p�6w?�D������������~��u><8i�����Nq�o[�����mri�=�!��uR������s�8�u��~C~��>3Cua�<��%:+�s�/�>�����sV���b!��[vn3����L�0����6jp�r��?���De�.�89�\�R^�������J��TJt!W�uH?I����S��~�K�.�nX>�����
F����
HJ�;&��H:&<������$��O\}��0���9iJ���d�v;BPX�t�a��l������e���)��gY��e9���\�,�pr�e�^���a��rk�I��T�����S]�.�e���S��Q����Q�H
����-|Y�f6��e90��@���$���\}|�e�����^�9uY�5��W�m7������e3dY�l�er��tv�����,��\H�r�[��#R#.��F�Z��;_�[�O���r�t[���v�,{4,�Z��B|����FR�v�n���7��$/�����'�n$���+��]������zy���r�Y���l��<�
��-��N��������j�/6b1�af7�
���E����Yz��,Zr������%���-�ATs��B0D�a���e1#*9K6b!<���`#��7;�k�@������\��+X~g��F4�����o�|o�3�@���e �:i�I�������sF
�\G��e�h�������zT��	�7C������N����t<���h��wL����E�:�/���N�Qp6���Xo�����V��=>�2!��n�9��n��U#o��Z]���%v4W�x(��2}�z��nQ[���v�Wy�5�)�CE!�KC���o�u%�{=

[MC�l�����9,hb���<1
������i�31�l��2�{�2}�z&NLr�zOL^��sWy�5��01�����"ML�3�dv�����ML����
f��~��=1����<1����yG��gbr=�'Ye�7��e�V��81�-�=1ye��[���8����y����lN8��=UZ>�HwQ
�5X
w5���z���zd��z<��z��z���z���z���zt��zL��z$�qw�|���#�!=�M]�aW�/��?Q@��������^�����o)�e�k)��q-�A�\�
���U�LPg�78F5���k&]���.l�����
	��`���o����h�nCg�H��vL��v�B�5f<�g���o�e��^n��<��[2h�[h��t�x�!M���	R��%0z�@S��&`����@zf�3��X��D��{����6���D]E���v�:���g��N��F&��/N�)���x�a��J���3sdr�W�`wp{�|S�`�"��v)�]�`����|m�Sx
j1����,��R����8w!�]�)4�K������=��w���V+z���*������B�u"/�q!`EOq�R��&��y�&n�������B��
�����jEO��:���v����{���/F��*����W)�^�u���f�r���{8yz�t�3�s�9�0|�v�y����x�a��6|�\������.l��+*�����{��'+(���^�����|�@WP4p8������cb�18��w@��;o�B|�mp/�@RN�7o�rE���
�N��k%F��s��j���DMa�R�������e�I5|'�lU���r9+��d�1���&_����J�k�y
�4?����r���Q~y�����<���:�<z��?�)k�-]����E���Y
��zC�i��1�y�o��7y�
�P���Fv��;��wUI:�j�g���E��O�vz�g�*����nK�������uyq���'��6[H�����	=#�����\o*6�����l��GyS�a7H�@�P9��w�bG���U������N�"B����.U�������257�Z�����q��H(�����-����Z.}��99���B��o���rh8&���{^����N3OX�EbU���F���3 i-����Uq���V�[�6�����������k,��X�n���$����>W��������X�U�/L�V#�~��d��Fy�~i4j��F��~ito��F!�~i�t��Fu�~i�y��F��~i4���~)����e�����:�����������m�*���ZVCtVf���O���e��q\m;X[adm��JCk��V�x���I�+��
�Ix�?��%����M��������������y�����;��XG�����!Pb5�� �!F�m!*Z���v�z����b����������;�y�R���|��Q�����(���m��Q����;'p��u**�^�T4P���(�z�S�?���~�NE�������W7�S�m*��^�T4O���(�zUS�;���J� NLi0�d0]]p'�}��?��1^J���HZTZ�|���1v�Z��&���d��&�r]�B����>�+4<��7��-������nUc�nU
�eV~����*���H����CI�4�~�����q6S��Ep�vGY����_�ZX��,�e�m�S;�<�5f��)��5��W��D��L����7�a�����UN���r&�
��;��g%a����M��]�>���qnt���y���<��e]E��
�II����U�����G~�h�1�OK8g���L|���/5V���p-�&��%���f���x�*c[1v1���U��P���������|���u^`��D���n������.�������<�����.
����zG5wQ��g�=�(VL,�����N3�u�z���&8FZg��_Zg�:����������f�m�,���N�^!\*�m���-o������YY��nU���Zf�����
�5F��/�_����D����>���Fyd�5���m�����[���z�}��_���f0^��x��;zV�@R9�G��G�����7��E~�$�;���z������*2��wW�Qz!�T���t<�:�
�v~�UI'd�F'��(������������F���8��?da��bj�O[y'%��"{\I(|r�6��.�gA��f�$�
���|��H��G��+�X|oB����l�����6�	��|5�T���2+�!X{otb������N��J3��#S�K����?�y��Do�Q��T����*
����A!�A!�A��E�>(�}PH}PH}P��Aa����$���m�P�Y��U��$���Fh9�E+'U����0��T�\�MZ!���\u�9��/Hu��:��u��b��e������������"Z?Y�c�r�N�T��~*�~�
UX$�W�
=�q�����yO��C��G���89!�A�K?h�!~4�A�Q��#z��]�<��b�[�-��/v�X�#���j����V�9�/�Y�~�^H�'j�#;R��\m�`g��6w����������kuw��9��/�]�W�����0��R�d��a���	z�{�>2���$!h<��7�d�M�`�RN���z?�O���z��S��V-�a��r����x�����j�v�t�6/�
��k���AI�4�vM}���h�A��^�Y`�I��g��oE���_�
c�w'�5��(��^E�z�55+u�����������xqT�� xh����2���-���G���TxQ����F#V�YTc�l��
��)������Ab�E�I�3B��S���r�?<���~�d�	�Az�������%�Y��P��$�N�:3���e��c��G_���{���C���P����>�Dk<��`�"���K�5,�a��SS����E��@��}��L���x��d���r���p]����Rc�S\�5Y�
�2nv@��`��S;���v�9�T��e���\g�n���8���,7���6�ry���8��q��g����#��1�, ��83q�dm������dS�i���J[�Z��>M!H-SNr$����5�g���@|�n���g���\|���}���<��<�����Yo��&��_����[��y%x�M���@����$�����V��j�B�r���?P�X�P���,T2�z��J���
�
�0s*�V9�9��Ta���d�Q��'�����U�,U���
��ux@���:��$�;]�����&]���x��\��n���H��#UQ�U-[��FQV2�(��F��i�Ze1�H
GMsN��6��N����(������U2y�Z��j��B�97x�x�a���x*��~Zh�i���0�_����1�enW�$����y	���
~W�8`�

Qf(?�>O:MH��7L�����O=�9��n��O�nQ.j�����<)��������B����[��A��.�,#���n�8�g=^����[��V~�2A��
�rn�nEI�
�Z�� =��� ��>�[��[N�����UD������
U�����J��_�v0�?@��@�����`��#Ne������t�h�
���N�����H�XgJ.~�����Z�w�D[5��!�Jw������t�e��X����J��x7#��p?���$�F�=GNe�gGS�u�i=>�S}���[5P���<(n�`�m0U
[�f�wfz�	?k��I�4���9�9��3�c&�#���L�A��<mQ6I<sZ�����vf'I�'wzTo��:s��(OM5 a�@<rjO��f�d<q���f��$4�)$8������>�'�,��TP�+(�������MI��MI��I���~J*(�y���S!>	�Z���
��C�ji5��C�jcMI�aS����~�(��4S�'��������#��?�/��h�:��>��\B���n(��`J��	��aM+��[9��������H���:yx��{����s"�f==�n��H�21�$>����H}C��)����oV�4��	��E���f��0:H����j��b������i��C}<f?~��/�!T�{�#*_�����X*�
"�u+�M�4GbAI,(�fm_ ��7C����+s$)�CJ�F�Q'��s��:����m>����Ux����OQK�p��`�?I��s�I�X8x���{��S��w?��yt�6�{���s��;�����7%��$W&�2��I�<T�r���.^��X�C����n���	�Q�����f���Mvs���T�������\d�yu�>�z���������������8���z�l��T3	��e�NV����q�������7U?Vm|���u%?��f�����w�Y���]e�~~���=6�=A
������~��������V��Pt�Z�-��#G�rS���ty
(u#�������[%��k���*�9�M9�V9�y�������VYlV�/�Y����cs��*�$�r44�6G�U�o������n^�c
A'�y�����`���<:��{�f�c21*��j�)��r5/������}4)������_�������!t���s�e�5}���BwM�CArx�,C�_�k��m���xV�V����w<�u���A��U�U�0����l�"e��;����[��g�,��U��<����g���Y57��y���gUs��Y5����V�1���Y5�Us��j��>���Y��*�g��8����jn�U�9Zg�<��Z(�j.���fV���j��Us����g�\?���Y5��������aV%���ah�x�,�y��1��A���E���
���X��P��B��Z\�[}t������p>�K�s��t��=���� \�/��[PA�S�<Xw������
�#���	��k�����=\}�=���e�P�0&uq7��z���A�AG���y�
8���������D�ME��@��Y3��A����A4ni�V�&���'���M4�+a�x��r2c[O�D;6�x�[�~��{�=�D�w�j�ID��'~���)Y�����$@�}{W?��=�������#��)�
�]��Ul�-�3��5�]O������0	�{����8����d��JQ�|e���Nls�s$��V��{g
\����X���&I��SW��T�3�9��2!p�x ����ac=�<fx$��bk� n�PdF���>Q+�
'��Ym����f�q.p�<�q�s�p�G���(�}p��`�aO��[�rr_������gQ�W��W���ps���P������-���}"N!��N�2���7d�7�o(s���}C7��	��@^��1���<�O�L��}#�}#��F|����F2����7rs���P�������>8���H��0i�O8%2��"��25��I.\5B`9{~�?��GF�U3\V����������U�VU���CG��_�����Xj�x���TDl��=�?�+%���Y�<k�&���Y����7���	��'P�7���@��'��7���Y��"���q�D���>�����7�ti*��k.�iG���w�������w��������}W�I:��qS���/�������cE����k���e�;n�����I<�T��Abf�R����Ts�@�@�V����W����1?�;4"@G��Wwv����X4[�������hH��B����L�7���K}�K4�R�^1b����u�+�W�\�ip��h��:W���`��2u�L>���opiA��jp���A����L�
���c�7Gc}�-�8�ry�O�����O)�K8*�\�3���%<b\�G���iwGx�<-�}���g�w�����M�q��\����Dc.��r�����V�\^�r����V.��qpAW.�����rY:�k&��w��3������[�Z��i5.�h���&�B�
i�(<V�B^�
��U�+W�]�
��U�W��r
\���=:�k&��*b�\zF�~�.���1����i���K}�K4�R�+W!�\�n�*�����\�i�nt�2>:��+��#�f�\qW.=#H�[
.���=�V�2�[g~��
�'��x�{��E#���;4�lU=�N��9������C��^"
�FD�e������<�}��v|�-&�d?YTl~S�J�d�����y~}qu~y��O�S����}�=���c��@C����c�r������A���s�U� c��LlP��������B�2
6����V������U��G�xe���Gyh���F�;k�2������0�V�	��J����./��_���f�������<��#�)�$�+�[�2����������d��yhR:��aPo6���C#�����^���?/j���?��)��;�(��J�� n�v'K�A��eA����%Q���a�W�l�#C���"�d��������}��8��)�������� ��Y��=��[�M�.�5��9
���;K���v>s�0��f
a&"�r�>��<�#�������r�^�m��k��2]������x: �$e8��4���`��M�`Da��b�kvBL�9��KQv
r�����3z��L�bGM=|��������gm;��.g\�Kh,�Q.D3*�hF|�"e�s�4<j=�( 58mX�"�);��4�e�s���(��� s�@�cb0+7�_�qS����>5��Q��z.Et*��������-g���_5������{��6����f:�����
9+"gr�,><�2���Rx�,�1!{xH
N��k�f:M������; �$e8��4���8��M� ka��b�k�\N�����^���a���t����f�k�c��>��w�z�t������L7d\8�g�!���0�
e�4Fi��5�&8J���������
KZ�5e3��[��t��	E�2dn�xL���&S@�0nj1�5���f�vN��R���\of�����n9��n3����t�6s#��3����m�5�����n�8v$�t#�7��g��	�(7&$jI����i��aM��L���X3�5TKD����:B���<�M���o\r��Gw�����jU�9#?���@�p�L:?&�������T��8=i\��/�J�ex�kp��h��4^T��c���J�8q�2�]�?4�X�,}	�GK��ex�kp��h��4�]������K�x�h�����'��4�l�����4�����@S�:�$P��@�hr��hr���8��|�@���hrM�>��+P���hr���@M)9�����M�49�Z�J���@��T����@-d%PSJ�{��rSrM����)%��vU�)9�&w����<��'VrM�N�v����i'{5Y������s��]>[n��tQf���U�^�o��l����)��V�
m���e�KY�ZU��r]f�n�����&k�E9��>?GS6EU����k�U$�����oD���c�����ud]��8]����G*r2r�{5��,��_��������?�����������<+��������7:9������������F���v��I��`��
��$=�S*}�6�}�����z�I����pX<��=S��C���`v{��T��]�����6�~��v�[�fM���5h�mO%�J��j;����_��o����j��u.d����a4a�Cn���\��n��p��X'�Y�!u2���W��#�:��x����XG�:��'�I{�^H�"�S�f\�L�y��v��w�����=G*�N��Q�3�y$)�:%=�Ay0Q{���3�/�H`�!����������4���^l��[�9�����c��c���5|7�����5�&��|�q��������7V�b�/�C��\�a�MtYs�V�c��[��_0�[<����]�=������w�N���DB��-�:�@n$�1���m3�61~���vo�^�t��=�8
�����5��?M�Q��D3J<5Nn�yC��}l*m��w�S��"l��J[> ��j;�����=���!�m�s9[
��S��B��%uV)�j*�CA���s��������`�G[�^��r[�yd�^6^��7�cM=��j�bc��m���$��$�W���S3��
^�BS�"RMZl���
��55��I����9@�k2��d$��7T/��t�������X�� o��V�1~|-�Zq��^����v�:�#(��C��r��>�>�ZP�w����S��������&3��Z�W�]����d�	�aFW��f��{�� 3�5��
DB�."�H�jV�����Iv\�2�]�?}�����2sj�le'+�6��TX���iX~j�uW���������F�Wsk��.;��B�������"$��5��G�k��C�����L��2Mf��u�EG�p5sy8�y����������k�����7�K���}�1>����|�I��KNnYr��6���o"v���aB���Tr����i�E�4�.��.���\����R�k�����	7�K���}�3>����5�NEb3A��<�Rg���Z��^K]+z��][M�`��uZ�V����X�QWp�W�4�
nE�|k�QWh���rC?�
�Z���z��<��nob3AG]a��EF�:�p�
��u���Lv���:�U;��S�������G].�:u�+�����u�i�nGt�3>:��k��1Sl&�Qo����a��M��.����-�Q��u���^&����S�}���P�D<z���~�T�x^_�l4?b�����6��nw��S�&p&�?��l��g�h�hg��7?�m���|_-��oC|�VD����h<�)1$9t]��B���d����#� /�@����a�VL�x�_5eZ��7�.	hB���c�H2n��k�bg���A��d��X�<�m.��\7nsy���q�(��A�8n���<��+����Rgt2n����&t�;��$�fo��uy/��[�i�bG���l��@��V���&\�����?n#�H#n;��$c@������[���b^���<n���7s&N���0n]�-��k����qo�M�3:��k@�,��F�1h�i�v��Zl����-�����������Vg:V���q3���-�7��mT���������.|���dL��VS���A��d����<�m.�[u��M���Uk4�[�z3>���-w���uU��N�m����.+~���d�d����,��X}�V����p�q
�L�#���r����s�+[��{������w�^����Y�!�3P�e���=������qZ{�����1j�WNRG[E����o�������U��vf������5,����(�����ZP#n�9��e�m��*`v;+j��4��L{\�������0�h�a��&�Ga�����3��N������6���-hg
�~F���}�������7#n�9����mR����v&��f�a��6�Ga���p)�����PM���e�����c�Y0l�S�vy�vy�f���c��A�Z���������Zl��������e��X�Z���b7��g�R�7?��O�������������:�b=R�d�J�B�XgY�/v9Q[����Qw�:����b�}�`�������Wc���8L�y`����?M������J��G��Rsk/(��r|��@��$4�)�TC�q����}�O��������1e�l���X>�i�������1W�����=�=d��	�]D�����W���3e|�y�������/��~u�7g�L��-��>�������^����1�����8c�;3e����E��K���[����i��(�����o>�_����G-�N:�>4��M+���F)�u�F#�U�F����F���Z�>�h�H��?�ttBN�=i���q�o�����M t�����f�+i��d]��[(Xg�}.W�/��7Ug��9�Y�qw����Y��n������}6���G �B��	p��e��z���g���b]����/�5����Y���F�����U�Nv�������b6����A��0�n�g0R������s��7�q[��m�GH���e������
m���&�a�).��H�.�n: <�t�C<��q H�}>89 �%�����prJx{;<)����b��q�����$vxf894�@���p2W��W���d@�2J*1�:d�f2���������k��"�,���&�.���|2�.J;��E�ZM.J�jrQ�Q�B�'���M����������OGHIOB8��n&0	�Y��$&!|�B�>��\�#���$����������|��X���+�i�����Z������f�(���U9����8�TOV� �����]��L������A6_V���}gw��f�����l1_o�'�a������������$�~�|~u���M���^�l_���������{l���~a�%�2��P��������[������j������D��s�E�],�������Q�������;m���
����sm���
�k^�������������6}�5}Q�G�u�>{F8����3�������v�L�w�O�d�L����t��o~�f��u%�T��I���Xs�8����E9�T3����K����^ 
�EI��A������c�� ���������k5�=����a����������UP+�d -���Z��)T���\}*��j�-�2���wT56=|W��LW��|�v4�8�U��T��	���N
��R;Z�q�}+�P�����4k���[RyRA�D@�.���~��o�go�]go+a�8��r���`N�=8-�����*e=:����fR��y�T*��v;~��s��L�_����h7����J@����2�����zz�������?r7Q�i���Ev���/���?�_���myY�w��=�C�p��"�d>�s��wY����a,���A����X��3)Y��uR<���5�Qd��f����e�Z)$�B��E��-���}O�l���;�V��wWW��X�{���{�������}���������#���M���������������Y��F�i��M��C����]�@{oC�����	�NR�����3���UI2�J7��u�l`�$�[�8�8�A���� P�6�1T�����gt�{��@[+3jE]�Y���gt3q��B����Z9����hh+��2���=I1�B�����j����<$������mxA�6��B/mY�N�Wn�^�\����RR�LW�[�����v�UgRG��|�����Gl`�'�PQ���O�!����������j"�H��C((�����
���M��4���'�i�h�ga�����C��D���nI0�uV���v�]i#��+�IhE����{���y��������nW�2�a\he!rl[!�)�
�����V\�XF�{�����<�x�C�t����FE�m:��h�r�)^������[�p�e����?������r����)b��U-��E�-��M/Q����������'l����4�4�������4��g��	�
�/�B�u
?�q!���J��e:���z1���m����
�������-�%�p��*��F��UO�(N�V�{����8���s�j�.�]����H��@tRY��������.�?jiv�G-1;>�@$�[A!�Ff��JB��K�(���E�B�B�zJEz��� �*������6���D�J�"�P���c��D���J�8�V�U!�j�{�Isqg�i��inD6�8>)���ga�@Z�t~���<+@�8U��J�]ULU�����Y����v+�`��,��U�u���EA�����
�D��
�Z^�>/h�0{P�NK���o/�;���^�������W	�����jCO%$���Pi<LF���F��������v5��v�\o�����4����1&�d���e3�����������K��yMFS�V>^�����7�>��d^a��R$r�Q
Uh-xZ�f$���^�������Y��������
�c�������.
���a���)��vD<9�������/�)OYm~H�=$�e]\����"�g1�k��rN!�~������|2�iZ2emb
 �DY�P�#P?��W�c;-�L.��5��0|�q`�]����oX����oh��a���C�3U�K��*���i:�Q_Y��|��h���-MqV��l���h�$��������g9%S�c�Hi��B�mr�H��)�0R�@�\S�^^g�=R�*��9$�0Rd���F���qF
�����)�y���y��<R��"����M�)��a#�FJ�v�
A�#E�r�H)l#���H��4�H������H)�#�P�x�&�.)��� N��?p���i�i�&��0e�=h�b@d��������(�_��0O�kp��S�b_4������c'��d��,|1i
�yt��X���R�Z�XO��I?�GX.�����FC� ���i.��J���h.d�HW��(�Y����b�m�b�4���b��>��_�����|�Ef���S�,W(����'���]l�w�2�X����2[L��\;�Mw��n��M��c�){�}��rVB�W�.6_f=�<���v�������������
�*���M����y�OL(C�Cu��
:���}��?}Z��J-fM�Yu�:�J���$1�Z���;�7�7%T�W$���"��E��������]R@(�7������r�;�=v������I��J
�qB	���v���s�2�z��k�s����~�����i���W�K�W�,��xi��$���1������������������~�:��9�k������;��)���4��@_�p.7�
�-���z}c�:|��+�O22��-��M'r�j/I�/��5j��-������VtD�{��l��;y�o�M��Mw��cq#v�F<�~��W>���~�C��O&�c�D��b�k������$�ht"����`��������a�	���5�#��\�z��[Co\"��c0�����2t���h�u��%��Ah���F��M�"Ea��68������v����-�eM�m���\��@����n�o����U#-�u%�aCD�d��g�}�bD.����Z~��7��r����EZ��q��0�<�k����e5T����$�Q����P��WdFq�2`8&
S�d���;�e��u�����%Z���]�����N��m�
�������V�����!
a�W���$	i3\���bC|�aEs�jb:�&��V�Ca����q�5������A/Q
@��H��(�]_
X_�����.�*�6>
�������c=u�q�	#�P�W�;+B5z7���� �0��[�Mq�TsLn~����:�vLb~~]�2C�~����;&+�X���bsLJ.�G_f�th
N|�� 
Q1��&\p���X@-��	q��D:�c��
>�	�@ ���dD^�?��	q��G����	�.A>O���&RJ�!��~���
�������x��fq����/�r�P������N��
��2YM��HX���aH�UY��*MG���F��a�)a��1������i���{�xr�U��T�sS9R�����s�"�#�ekI�F7 q��Qo|�E�&��U����}\N"���s3��'3���36����&��iu ���E��CE�U�U �������%Ifn�x�z����f�`47�:4���U���!^&A!�
�%����=C�����oX�R9xK_5#K����kv:���Ew6�� �:����@1��q�O�����3rm�@L�I�`����K���������+<�p��G���S�Q�c;�)]�h1����?5�4��4Y��7�����@��:�%w�S�%��*M�:!/u�p�w+��m�i?�0�F�=�<S��j���XP�����]�4Cw��Y&3�Y:�>��}> ���a�q��������9�=�5&(��Dn�E�0Q7 ��&��T	�<���P]|*�$��i[%w���)�
�K	��w_���B�}����z&�@�B�|�E
=�"����n2��CsC6�j����_l%�k	�7
)$F���$�2yd^`2y�#���wB�)��D���7�H#�3���Q�K!I���dN�P
�v����J����1��V�q���BGrA$q�v>MSo�u�e�����������7���B�����4|N���ZL���m7]Ck�����$id�u��.�k��![�����MkE��/xdD?��5���Fv��b`����!��=`#���;���]�s����5	����X�q'��^�C��w�JD���m\&���@���2���l�_��4P�L�����u��Y��������l�s=�y6s�?fP�����kGi�����pkgP�,~P��wv��g�i��v?���@�����`����o����m�������������,H�?��p�><P
�I:7,����d�-�H{?F��z_���X���j@�g+���IYs�B>/��<��k`�U8,E�+��`j��������bt�7�n/+��*h��-����;0���~�~�.fA����~�Z;
M*����B��K�u�dw����x�1���i�����G�|��8���5�����"����j�5���*'�9'�I�.7���d��D���|3���~��~���an��P�N�����tn~:W������:�A�)���K`�>���N���%���z������Y�O]���A8�J<��?��f��z]��)����R�a��A���ox��H��~vb�m���!�|��+X�5�a�S����X��G�vRz�B��7������@�u���P��>^�$>\����s��T�M�~Xfa�X�w��!r����������<?��r����s���pC� �4��R?���7�{�O��cJ��2��s�v)�1������xee��3�@�-�p9����V�x<`��9��$�����]���/{��'��z�0���2gAwo��"��4<"��*@[>}m��}H0��6T����2����H��t������T�@]��<��{��%Ml�����V��s��n����44>:v�B[�c,��:w�������Y�������W��BC'���!<v��b@&b�C�NOZ������ri)f����6����%xnm�)F��d{���/YK�V�������l{{�:� ����(�����d��J�O�Q�`q<{��|�D�+������8%@�J>����xO�����+�Y��1%���z)�&�,
���+�5����\&����I1���dK�X��R�I,Mb���Xj%*�����Xz�BG�Lk��f�X�X*b���m[�������@qT�

�gL���tbU7�+ �Z;I��,j��F�V]������<��`�]$�:�l�|%�p~��J�|���2m���y(SD���'�_"o�#P�V9��&z�b<g��������	����l*��Q��X�;��b���gr%��_(��|�6R����1�����i�O��I�����C
��j7��A�*PIA�i��&�r-QM�t�;���z�t�c���\��8�^q�k�7������" �^�k��`R�"@�\��9����B�"�������^5o���� ov���S��YN�B�n���$>����}�3���2��?������;6����f^���bc^0���1mY��g���@����nL����[����?��d�l!_������E�p�5�o v����]
a�5z��A��:r>���	���dkU�m"�%�32�h���(��9��$���]4��CMv-�x�MM�C4�j-�-���6�r~.9>�?\����C�O��]��n�������x�^��1�~���ny��u$2��M���
�@GA�l�7���q�9�/$m���0�Z���D�gh�n���@�(lA�H���k���GD�Px�D�}���'ZW��^�mm-z�-D����+�faj�^�Q��.,��s�����j����������Z����2�!�1������/\�~���$���p;M�cM3W�i��\�L3�������C'����W}���t�j������O��Jm�q$L�v���� �ws��q"��8�[��4x�s���MKa@!������KLx��jez�K#������\��LN�>����/ 'VK�(4���K~3L�$5"�!	0���>�4z@�[�w���n]`��A4FMg�_���E�^71�� ��'��(M<U�(]~w������|~����o�X���|B���������)l�D��$�c��6�]nk"I	m���&]m���e�2��H������mB���D�&�v��#��SD��H�2s8K�Di�=�v�v���$M0��aK���;�s?xZ���4o,��kK�7T���Z~!�v-o��E���u��L��^��b���|ru�H�'7��j�����/������C+��.��.��u�^��W�H:��v��������#�]����[����xw��yy���6����������{��S���I}2���}�8q{�
������g#tU�9q�������������c���b�6
�,��B�0Y]��%i\JVS9V}HV����
Im������j�������������+�_�}I�2Y`�&�N��,0�U�J��R�J��*W���@�_&��j3�A��y��\G.]�s��L����k�h$�S�JIR�(WD���(<U�*�������I=FMH�g@����9*{8�X0�[eA��q��A����*���s��QGeG
�v�jR��1�F������!��6�9�=���g��������m��Q��|���}U����jORx���f�����[.~'dn�hb��&O�+B�D):Va>����2�*�@T&�H��#��*%�������R������Q��c�����
�<f�c;z���YN��[i��jq��x�������o�L��R���d�J��>%KU�T%�Le�T���&8�{�0=0KU���c�q]��e9+\W�*�5����A{�%����>�f�lV2�IyJ6�d�J*������D����<��JR�:0a����,X��������,X����00���e�rR��t���t@�}������CRrx
�#-�����$')8I�I
N�2IJ
NRp����{�n���cSk����|�9�u^��F�p%oo��o���d��o�UM7������/w�}�K�!����e��9��5]�k^�"�Vb�(0:����ha��+��T�ctO�5���|��������|<'O��c���X����*D�u�U
I��_?o�q�=�7�$�	����I�poMA=����'�b�U@��Ix�����=Y4�$�	����I���S����.���V���'���$�d!Kb��(��l��0)�H;���_	N�����uNh)�N�����L3�	6qeG�|2-�f�s_V~����A������Ro��e��bD�P(�rH�#R���^�m�����V����QIv��ce���N������km�k'�����Ws>�����o�s��UK�.���}��0a��I�k��A�����SaxAO~����j���>.@�6��L�.�� p^��5ij�Z�MDt�)�08�m�K����f��S��e<5q��b�,��:*�����mZ�Gt�����;�1�H�mLp�a��wb�5<M����.S�k�q����������'	&��to�A����$�����8���r��yj���t���N��v�y��UVd�� 3P-����@���:���0����3�!d�vm�����>%�U�$Y�W#���ZM�Y����I�g�����9���i���H7	���O���wv��Wt�-0�6U
�8Bt+����=��S��;�,P ���o{j��ul�v������#���Z4\�Ds��0m��j������n�6�����N)�r����(d��b %�~�Pk���� @����_�n;;��}o9�G��D�I�y�7|���v��1�	v�-���������X0^�^��/��v9C��Zz|1���n���ds������O���Z�����������'�Zk+J��6��1���h���hr�.JC�l����|[��Vi����k5Eu���DR3s��bv���l�����_����qv��U�
w���S�0�%�����
�V`�z;�8�#I���igKa�{���#/���D�����qX����)�QIbF�X�f��/��v��J ��@����O~���I`��������������������]�R�{w�C�����F������������B�?p��n��'������*���|�����O��0 �/�5������RG&�k��|U�������T��8(Q�S���P�EJ��l���?L,����@���|]�����HlVS�+LCo��:P�[pX@����0�&���h�rD�WS��CCo�:P��pT\���� �&����A#	��`'��`1F'?��-I@���P����B���\����A]�]K������ST&���w_�QZ���q��oWL�;���������/�Z�����G��=4�������'�n��~���;����5�q�X[8c��Gv���0D:g��	���4&�l�(Y���mf��][6�����=Z������|�p��?�n�w�.D����U�A�	P���/������]���P�5E�k��x��6���Z)��>��~D�s�����4��:4�Z��=�_m�������������v�+��nm�>;S:���N`�����;��N����}t;��������bs�gD1�gyA�a�R�gE�%`�����V����n����/�rSe��������2�Go�D�P�39�1���o^��f�9/�xR?���]��������s�4�^�90`{����@Ww��>$��]A���<�g��P�6h$i�m����h\1?\�_^�{u�Kz)�p7��l��q��d�.}��]��6���������%�g�N!;^P�x��8^��~��E��2����f�������9���W��S��"�J�f�&�2E+�)�
B�3�4R��}	N@z`�:}�Us�������v����J�8���>�UA���,�Fp(3s�F��VA�ZQFRX����T���cqv�%�iZ�}gX��@W������B`���$����|zV��cu���`���������8�������*u')I�H
�#%")(%")R����hR ��	��Q��������l�Z��[�������X����onf��W�+b�J���lk�]���������K��U��U���/���PdGO_]] ���o_]�y����lz���������+��!-��
���n�f�UO��#W�i�����Iue��%m���w?_������Y��BY�)��%�����o�J���������������
wP���A��V]�����2�= u6}�e�U��z��Oo�S_��Y��";T��*�X����<����������S9F1�����P�T?����M�� {�}��I���m���O?}�>����Ky�(^���k��&\�l�g��%KC�${�7�t��=��Se@�^���V��
��{~�*'dv�l&���`8e1_o���GO�S�����6�X���y}KW���K����t�>]	�W�~F{=fMir_/������������9���3Zr.���X�,0�@=�}�e��p_�P�9�?����>dg(�$nb�%
\-����M��Xm��6�V/���%��i�z�BVn$�S5�W��b�?������i�J��7a�z=��Z��@�@��������z�of�*�P�E{��1����i"�2n��!�h'''�T�~�w���qV��G��H~G�#�3:!wJ9����i���JWl5}�M����F�	�?cX�{�G j�~� ���=d�q�R!\���2�9-���"E�$���q+�e��Q?)�\IJ$��/����8��f+3�lJ_��B|�W!��#�*�$A�L�8#�**��EC
�f�!�&&JT��4�]��GO��$��^������I8�a������n�Y�����<O���
LV�!�#��	�F�����q����}����W��w���f����>{�(o6��v�GR�D��S��zf���$?�)�ON2���@�'�#y��S<�w.?�H?<�i����HR�����&J�4��Fy2	SI�J�T�dB�0%�$L%a*�0u+�N����b�g��D��k��=�v��/I�J���	R�"G����:)R��
Q�+C��C���Uz"y^i(�K�xWy� �e����I
��#����T�_�9�.�iC��Ht��.��_�����%~���������Q'�W�#���I��f��,����y��!���#pA�H���q��t�M��=�8r�=����M��~�\��+3����wx��Z���{l����_�G�F�T�o9��v���oM�:w-�H���H����!9A�K�����N�j��vEo7�F�u���G�����Q/rt]k~tq���\hE���h�l]~��X����G���:_o�Y��0h6�	m���1rD�3�Q7������p�;*q��Y��X�9�����N���PK�����X��&�Y~,�f����	��5��T������SLJo��BY��jwr]��y�!�d��\�q�;��:oL�.\�����V��4:��k��B���;��:���1�{��Q���2x��/�'��0P����<b���#�!��`�H��w��#�A��`��
�["kV[���o�?U�K��U�m"�����n���
��/��9�����m���0�X!�	�z jNp&}��n��S92fD��}���4��>j1r�QT�b*�u��0/����S�y�C��\����~(����XP�f��k&���_	��7w���,������DO��w��B� �A�*�o���3��~�Vz���%����U�mY����<�s*;"}!\��X"�� W�#����y���5rdt92��Dd��(q{]�����,x�~���Ys�����������Z�����l�������g}�zU�v�1��r�b�kk�������O���,V~D3d�9���-��
{�>7�t�_j�\�J)��hX/]A�9��t�y
���`L+ F%�P�5h<��O�{�\Gi�^�q
����N�%�'�]��g��$�(���2=���44�],I���`Z�/z�Z�;����yY�����`����x�������I�H����}��W����q�/e���+�T��#�����lP�#�o���j�c�_�iWX��Z�Gj��������|9��Y�pTu>�/"�x�kb��C'l�#t�����s��+��2q�xk��e�$�����X)|\��B��p:�k2j������ ��d�Q�����
�=p����"^4K"N~��
�Mo�#�e�p?���)
p�'�'��}>���d�_�B���4(^}�������&����Hu���������oi��vB#_�a����5Q����D�$j�L��&�$j;�M�v���'j��xq��N��xAy`O4B�m��[�/��I�Nb���$vk2Jb���$v'�;{|b�=��!��T����vE#x�Q���5A#�����$v�Lb�&�$v;�Mbw���'v��xq��N�������'��MG.&�S���B����G��3�7�4��t8����p��p����I�hw��S<��*q �M�������H'��a�(?���Uhi!OJa��xGj��#5�����H����:�������-���/��T����i%�!�
YR�T&��X��6�hNjCRl�<B��^���$x�������OU:p����0�\R���%��MeR�Nj���6$��F�#T�ChyHR��j
��G��X�q���L{�i�����@XO
���@�����@$�D^R ���U �[tyHrhgj��)Y v�Lt`�H\vR"�!��D$%��0)I�HJDR"���L�������=���Kj�g���WGz)b���9%�!������3��d�)�Ie0��T35mU��W������u[R:�8"��������crqK��d��n���j3G�f��tQf������>R"P��:��fD�QBz]����{�?������%���R]�;��^��i����Q��>�7�?e]#(;�B�T�De�������Xk�c�rT��r���Z�AD�
�yC$dy��o����,"��H�tT�o��l�G�	�|���W�d���!>��yCv��
��W#��� �h�.�]�je���^�\L�b	���?`�L����&�j��*�?tR����d��:��eVRVq�����h��F�2]L���`E�B���	^-������V��$@������zC�t�."��VS�
}[����j���c�AW#F{�������0�$%��d�$�<f�G�/�n����=�����c�MF(P�
�f�
G���x �EJ���j��\t�r��5���E��I2���Q
�r�6*20Z ^�0�7��9��	3���k��Q0��4���*�0�=�r�_�j��P�@��&}�,�������A��)�����/��������V�#�R?wM]p��W��r��b�~)���J���9��s�oi�sD���9���!@
�q�l��sA��3�+�b�t��!����U�vT�����U#J��s�F��:����V'�O���Z�
�mkE�Vt�u�Z���p��VdkE��������FE��y3�b�>��:�Re
/����I�7t[��y��j����r�m$��G����R{���.;������7"P��������� �+�C�u
� �F9@rW(��������r�%�c�/}�G�o��{i�V=]���r��X�ZJ������.X�ZJ������������z�*��J_��\�^�S_���r������b�K�������v����]Xr����1#�g��''���	����|��<��;@[��Y-�1�[#�{����J�t�#�J�{zDT������_6Uy����Tx�63&���(����g�da�����H;�S�"I�w�q@<���������#��"�Xa|4�5���Lx9g�>���M-���L����;j���4-�
��@��qV��/.�������k�A\���g���������������������W�~�
$��uh6Mf=�
��.@�����c��x����O.L�#e#-U5��I�Br������A��y8���$��\` �&I���:U���!�~jj�c�c�cp��E#8�0��?�B��������K�����%�|V�T��RV����Tl��8V�2
	�2r	�2Q4��a�cgR��cN���V��9|�G��4!������ =�}v����������_�BeO���>�(H���n��Y*i�rt�o��g,��q�I������_�)�L����C3�G{�K����kmPcMO� ���<� �<�t ,h\�0�]=@�0�0)����B�"�]��ug����S���������d6o�����*��U��c�f��al��
�N�Y�EG�$Y��^q����o��	������&��k�L�?#����������������iu��Cu@��<��ji����z�/���I��R��/���������}A�	]�C���;hJ�����O���d��`b������I��������0����]?��P�����>����Ok�������;���'D�
�!�������D�}�4b��h��Q#5aA�E�@�����Jh�!�>��lD��Uy7�j��T���r���������J2���7_"���Y�u2�c�
���7"/�3�[�\9�jX����$���?/^}��DVc�(}���PY�*��r��l�
N��q�6y���`����|�������s�"��W$���g�^�M��G�u0v�O�@�k�G����E3k�.�Yp{*8�E'�	�/������@�9��
��{���W��tOu�S��pq~y���9����v�����g�z�����o/[D�+��
e��]e��V�g��P\��~��E��2�*B����#�����?rv5������6I\�7I�5�oV�b`*��e���i!��������5��q������
j������{�������$�J����P�
-v/;!f!�T��=���1<�Q��T@{T��-��*���F�7i�_��������8�������*A5�yI�Kr^�����h�$�%9O]9�W&98A� Z�T��7���(F�_/~|��%��^]"3���,{���}U!,��>`�A;�SQ���������K�i!mii=�M�VZ���|[^������,(���WWB}�����7���;���M�8����p��}��7�����o��o����1}���5���|�=�.�����}��������{;�?F�c�k(9~���#M��7Y%����������O�.^6|H	��eL�zu�6D�+xR��"�}�e������O?��FC�������R�V���f��&C.��sh�[M���1�J���>��2�~0��\D���A�4�d�������O?}�>����K�G�V>����h�+��������"Ar��:��dS���P��s���+�I��!{���p�����G�{��e�'�}A��y��;8��x����9[���N6�l]n0t�e�������~��#{��A����V�� ��&k�p����/�7r���
T����-�g3_����u�='�=�8���&�'�w�Q��5�M�/4h#�����`]M����&'�ba� �i�K�8\;�{d=�_��RC%��[$����N�0���%T=�o����4p(,��q�r�����5M�#��TBS�&k�T1\*����&�_Q�2!�E��C9�]%�������Be��kxP�-��
���E<*�M(T}��T���i����V)\�fC��x����K�r����w�<?�<TY���o~�U�y�9�V�����������
���Y�u�6��/�:�g����-�r��@��������Y�<���M���;�VF�+{�)~[�k����h���G�
HG6����&�	F����O?��:m����QEX��Z����h��3Z�������{hv&��&����/(���.qw����-�&}e|JN������DZ���b���������h�^$���?\�e
 P���'�D"�P���A���ZC�*yto`AwT�j��V@���Q�s�
������=�������-����f�D���P-�<h�0L&��8�C��f)�4�'�M��4)����>v8qB��BHsB��~��+�r8��x��9����m$G�������gq�j�I�}*���h�#�'UR�Uu��a<��&��n�M�i�\"�����J���b��QH�%���d����d7|p��2��>��5�F������D�g\�T���M�1��G�1G[���$%�u��vT,�5�^�Fk�!����1����zc�W��N��a~7�3|�2��|[�d��6U�(�4x������Y-}��E
qN�MAG���-�k�3�!���O�|0f�X+�V�o)����L�b�8�T��^���aa�%��e��� <E���>ev?��,���P�5�(Sz$��r1�:���P����)�:��{�W�a�S]�q���>Z��'~N`P4S��R�j���j
Y��b��":��������*f
��Lg�=�:'���#���D�RTt%�WF'2�����g�`2#��
�C�,�
	}5N[�?}�r�q+���LLw|�����]��$�&�7uJ�Mo��K�
 %�Z��$�&�7	�I��A�7�T�N�M���l�=q��H�����d&;�}r���i s
�����W������L �����+�y����A��	h#�����Y>�����E�g�}U���;��8�k����u����(!)$�`w�h�� q)(HG9U�^�2�7��~�W?�a��9w�;
��8<��eF"�����|� I~����l��<_kaK�����%�����v\?H�:���������a�q\;Q$�h��$����|]0�.x��Bp�^t��T�r�&��|�3�(l|�!�^�����Z>iB�x�Q�QU0.m�jFm��P���{��ENB�����?&w�\Qv���3������P�*�(��V'=3�����l9�lW�E�pw|^��8�^��Y�@Uw�92bF"Z����Q
3p��<��OH�=�F�k���_{�S,o/y
?�E�`��8K�8�����v�������#����P��T��iQ�_��%f�*�"���.�zPx5md����h�D��i�-���f����_`����@jR~��7��G_�!�����
���[�����ys�m���-|����P���v�����e���{-��Z��AQ"V��9�|��``FI��Q�����S0��y�S3W���[��/=x��8a]�C����L�3&��6�������P7����A����	�Q]�a��1�:���0!���P�8O
H��nr���PS����G~��/�>L��QaP�OS����Q������#���+�6�����f/�:0E<���``������2�[����y��B{�4H'��F��-A����^\
Q}����h��	9���(
v,"����k�T�X�z�pK w
�F��aJ,0��8%��#�HU1������M��6E�$U1'�����D��
��%���|��[�7w�A���whg�M�r����iJ���G>�m������{���|�����NQ ����n���o^����|��o���o��|�Ex��%�}!���}q�q�U��W
s_5x�}u�[�{�5xw��!�8���m������\����5j�iv���&��s��Y��i�_�F:������������h�>e�^�Q1H����If*�
���/�*2����3�I���xa�	9��C`�&�L,�J�C6��1�1AX`B�D+,X\(8]�fc"��(����$h�P/>A�����$��%��r��6@�RuTG������2�������h�g��<CV
�)��ddN�x����D����q7�����hRixv�e:_N6���]��n�k,�s�%�!�Y����	;��g��Q�w�0����PX��]^����$�������P��S@�qo���kG�y�w���C��a�-�������S��<�~3O��S��<��~Q�mo)�p\p����8�����PP!��Br�p�z+��:I�E~�
�tgI���f:I��+�vCL��S�6�lCLq�b
�
1U4���&�M���T�,��YX�.iS
,	���?���q
���S)����k����|p/&s_y*�m��6��T��}��Q�����Z��sGN}�2��\�"Wx��\�#���M>��p$��#
6��+|>#eo���hb4�(!|8��j�b�e�E�L��
�*����9�~d�P�������[u� OkBX%��,����CZc �A����~�&
�������I�>zR��T��IEw��\(�N������&
Z�h�@c[�	4�@c��]��A�u5K�q���T&��JeK��l6>��'*@(��x6BW%������<��Aw�~��^�[����=�.j�T�����T�Y�{��GJ	IM��	==d�4����M���?5�|J��Y�O5>�����5�S�OM@c�T&��J�^��>�|��T�#���34�|�}w�\����RBX��jB[
��|U�:'_����|U��j�U���� G��O�W5A�	B�Q� d+�{��z�~9B���{(�����Dg�l������y�{�<���<8�L�����dAPp�!&X8�� �9TXJ�P�@���X�h�@�@���X@���0dc�	C~�N���Y ���Y'����{;g�,�D���4��u�x����P�Z�}�����!^��� ����������{O[v������D�k;����'�;����p� ����:�H��W��^���B�H������ z�w�?������~��~���Z�L�S\h�5���,�`��C?�A� ���i��;>�N{@'��q��7A���
����/a6��6�6:���������K��;O�0OrZ�*����brZLN����7t���NN��i�C�;9-ZrNN��i�@ErZLN�p���brZ<(-���6��M�r��y.j��+<%�E�h'�E��=���b@��}Vxr_����x�p�_5���?�����%����h%���bR��@����m	M���
�t���������1)%g������3crfL����193&gFK>�	!)93&g�����]K���� 93&gF����1�h	���	o��7t��B���������~'�;��	�N�wB��m��5!$%�;��	�N���b	��$��	�N����G�~?r��A�\�&�{��J�4�6�|�~�����5�H�7�����n���7���6������/w�}�'���1�&����w�3����C�5��Z����%�]�^h
��B��T��M��@qx�$	
)�H�h\VT�����(z���6��*��S��c��)�[����;�[��ojB��^Zw�t��s��\" oA�k3"��DC.��`a)��Bj�Bj�"�
x;R;R;r;85?y�x���:�����S#Nsy�����K������2y����G��>�T�V)fK���O!�O!��a���SD��4a�O.�'����?��?�bd�IH��0m�h;�)��P������!�uN�h�V�co�J��{�.��_�J����KL;�c��7NA_�5c��C}h�L�d�=���5'Ys�5J�}Zs�$�}��_h����[$���--�����K�JJ<0U��S��T6����G��cD�a�|���0
��j������W�W�k�C��4��A<�FM���SJo��A��F82��B�����-�V+��B�I^��
�E���R=��t����+B���e�_^#��~���!����T(r��$uU�*��j����w
-��~J?m]x0>�~F?�����.<?�������ut���������-����h��N�'�<>��@P�a��L��:�V�yT�)��U�4��I�j��%��*�� ����I�������Jev�B����%'�d2"	��R_'���
�%+V�=��CL5���?��BE������9'cD*`{��������)�>��*����<�cy�{z���*���E����r2�T0v���
!���SJ�Y�� 3�]�V9���QB��4�6�2�B�sJ�sJ�Y�� ��]�VyA	yA	�Rv������0���!���MU+<���~o��M�:��h��7���
�axWr"I�����x6����1)$���e��u��h�x�����bb{�S4l/z���A��m.7��F��];����M
��b~B�O��T�`�8�AV�C�PB`��B� +�@����Vx\6�&x��|��d)������
4��������@��P�qGC�J���iM�k.*�wPa�Z�p���p�*\.��0�V*vk*<d
����{�J�?�����[~�[��u����J��}�)�d7����k�����2��+W����K�GS���vH�� ;�����''���	����A��_����S]R�P�an�P�^���}.�tq{�I�������/�P���C��������E��r�'���j0az�\��H��Ll�+�z"����
�F�����P8���8�{�m�mjp�V��,qi�/./^]+�d��;��U/e�wW?T_�;i�.��:�^_��	��fQ���a_l�0E�����7:�[u���/?��v��@7�gR���'��I���Z��O����v�W�$���%�-����I� �;�
�7!���m�7^������4�Ma������&~�����b�cV�_�=1����'�p�eeIO�Ki:u�i-KJZ�����Bn�Cn��*Cn�����
�e�����NV����m�Gm��{<�����v����{�WW���D�N�>���d���k�XN�dVb{�O��;���
-��<lh0�h��cPmg�,_PMf�lZP-e��U:
�8�k JB:0��v�:<���4�O������y�k��P79T���K��z0m��\]\�O���Y.�D�bi�V� ���%���>��D=r�����4��:��Z��=�_m����������.��+�y+��-�[���'$��a�U�d���X2l�*��aK������X{���^K��^3A�v4�EMs�������w�*�&u�M%�)��*���.~>����?z�%��g�a�P��x��������j�!i!�j�]������R��K!�A�}`�=2�$�M��XL��d�I�+��z��"��}�<��8$������w��qI/E���v��?#�k��l��gd�{����Q��wW���UrcK�<j��y����_�n��&�NR"S�f���C��9���Wk!�~CH�$id>���N/��=��%��2�����������4�&Kl���\�U�d�s�����HV��L�����?�!�h>����b��}@�B���
���{|^D]�B���9I���P�`�3&z��1��!��]h$1�xExp�'jp������S�>��C��&����%����t8���t8K���t8[�wh�sR���I-���j�"?����7o_"���K��ts3�^}�z_��J~��=����\E����W��//IwVtw�lox)4��}�����r�������wo������w�]]\�z��b�������o��nf���1}���5m��|�=�.��������������{��U�j�$Y�!��%��Z��o�Jk��������������
����-�2�?xm��J�����D����G^f]���w?���
��w?k�Cd��������|Y�f\�v���b��,?�c�.9_n��I��Z��qe����.��	��A�����~�p}�����r�,��%�Lv �e�u;
�C�UR'y�a�+HKh:Uv6����9!p���1��V���r��$[���}�.6�E�k�x���7��~GO�%>���t,
��'k���-,�������\���/������+3��
-6�9��Y���$\�+9�rL��v���y�0�u��:4yru8w@Y;��5�Dn���$q��(����j��i0��*�CH^[	l��wy{{gu/���J�s`��U*������h�mJ�^qP6nOS��N����-�A[nS]��*���5mL�v��
�&��a�p��N�����md�
Q�S�X�����J%3`��s/(�u[��_O���
�y(
���Z�����pjc�-��C0s8�E$j����������M����xjG2�������JlD�e\@�C�?�ONN%���g#���������pGxgtB��r�;�;���+(�����6u�C�)�{�9��>�{U�D�RP�����`�Q�|�
�Ia�XX�V����������>����E���5�h��iT?�TNs�$�g�G�� ��?��*�c�0v���Pv�<_j����(hm��S
�#z��pf�.�]������]���o��^@i���S���*�/��+��;s+��k�(� ���$��s�iILQ�t����������!�%!0B!��$�N{B��B�$�o��og` q��B�;G���90�\_����A���	w2�8\�s��~�H)��L
}RB�J�D�v�/�1T��,�T���$�hu��$�hu�T����������(o6��v�W��DT��&�GL�Y�H�ah��Gc���g��]�?}��#"Y5hY���FU����AtGxx��m��@v0s��)�
>u�	&&,�o�{���1Z��aj��
(*1$��c
�eI3�$��g\s��.�X�j��?������c=�<nx$��
���v�\i�r�Q$���
�~Q��G�d�g��8������\�~yv�m�Y#��!`���+��~8h�4���(���r������2�����������������b���H��@���p<\i$�Q$�E�~*���e�2���?d�?�(s����C7��i��;���s?�W�6w@I>�tS�@�Qh�IM2����x�1�	�?��d������������8=!���"�+
2dt@Ivc�� ������=�z?�4`���@�
'��\��2����C��\,w$�t�Z
��3���X�s���HC=������P�@h�� ��;+<����G�������G,bJ����197&�F{��scrn�#5>97&�F
���197�H��������x�],�N���������:K����h���19:&G����	1$%G���*%9:f��19:&G������crtL������c�v���8:&:9:>G�l9�lW�E���|��h�������
+�_dz��c����2,<Q`�)�q��Fs��k��Z{<��h�Bz����vOI������7�P�vq��h����i[�a`ll������DZ� a�GM������P@Z(���x�me������Q��T�m~�b,�\���U�GrZ���a4e�.��C�y��Ne$-T~��7��G_��nI�W��q�]��u��w:d�'��3������8W�z�����"94���Gc�\������#�qx�G
�^SN�O�kG2(�D�P�%L�(Z�g#��.2�v��d=���s�.�}v�cA�&7���\xF����T������dJ8�0�=k�s	2����e���(?C2��|,����JnvT1y�,}�l�QN�f��F���
�4a�S{��v~;fBBX����,���9��>�����m�&1��,��M�)~0e8B���6������Y��s���m�(��y2
,:�Y���a��X����S�h�P��"m��	�������1��
�e�c���#��#�EG2��2LN�z�et�
NF�
��a��2a���,��M�#�wzj�?u���Z7��?�Dw��S�"(0<������h)��[X��q�G��}�����v����MJ�-�7�z���#X�����������z����S+�	��	-��y8��fb���L���8>�e2��T��$����xwP@KS�N`��5��������(00���yL2��2���^j������yT�
����F'�O0q��A1[��A�R���W�� �j��z�B����#����<b;h��#�m��<b[a��#����<b{Z��#�����z��N�0s'����U-{����Mz�d�fe�Y�����1��~�iz_��C�R_�6�[����h�LU����J�'I{�e���)vo��k!�c�
��7�E�m����'��nC���"�O2���@4�i/K :RK�`�a���8D�t��i������-��D�$�{C���C�9mU}���%������}d�-qd
�K�o[�����I��m���7�6q}����s?�~�5��������|��U����R4������T�S��N��t�����+��{+������� ���_}��'�������s�;�W���U�T�RJ&4��hAG�hq��s�!%�w��6�^����!(��c�+�m�����L��uX������B�F1A%�M`�;�P�	KP2k���(�l08qjZz�-��R,w�<
��� st�U�@m��Z�,��z���]^P���V�Z5$
�3�jW;@FCAub&�Z�����;I���eB���>F-97�/���GjP�#5F��[molMw�m��!�;�)���/��r���.���lu�^cr�f���i�n���1"���J��S�Bxhg~��8�i��[rZ�4�����c�d"�9N�/�$$���)�Q!���w?K���8�.�*`��}�w��5Lx�U({%����&�K��b�?%ov�z-�U�S�O}h����*`��T�Q���T�/(�NK��^���f���g
�>�Le����}��#s
��_9�hu8Nax�����(j��	zT��H
�j�������
��w:��(B\��i��05�Lz9��B�q
��y
�Q�X3�t�=�
)EfC���l�����MzCa���^�Sa
�#�Oz�P��y���+�UC6�+0�
��j�y��^A'������I��Lz���3�/(�N�c�F�����W��s����C�[\��}���4:%p(�!���T���n������i�D��S	�c�	z�R�/z��6����b���&l�-��0"�n�dKMs�Wt��Uw�V�[mg��0�����(o6��]��A���S:J��H���!��L��dI�w��1$C�V#C�1��f���!���!��Y2��i0�9C�T&cH2�X���r�p�C���=��"f��j�����"JQ��-�dq�dI�sC&�H2�$���^�0�#m��#�0�Q�dI��dI��d�,F�4��#V*�a$F,Tve9p��!ZF�.I��i�f�X��2"�tX��QJ�GJ��d17d2�$�H2����#�0���0�#5K��dI��dI�P��a�M����0b�2F�a�BeW����{�]D��d���D�,�����
[������#��A�T����[��CV]�E�����1F("M���,���O�|��T!��G����d>��}���hH{EQMfQ-�R�{��\c	JV����9
�|U���d���:�*�Z��~�=� �"*�L>�9	�pzVb�0�F�FN�����[�<r5J�R��V�;+��E�j��"�CZ
��Q�j#W�6)��jp���Vcd��{V�V��s�������,K�����B����Qp��j$��.L��j$�L|��v�
wq��(.+��&�3��E�iD���i����C	2��=���tAf����Lc%�`���milX$}VxB�	j�AmT��uNt����GR������G�D��\�#v�ss�s��a �.���hL��
��M��]N��(��������'�]%?�	xO���	x�Z9�����������'�=�*A	xX��Acm�
�>�����64�{'��v9������J�{����w���'�=��j$�]j���R��n�F�������%��a����7��Pp�{�&s�L�m���]�/%���iJ|��� x���'>A��j$^j���R��o�F��� ���%�{>n���v����$/��e	���~������
�R���`��'>���j$>��Y���`��'^�F���`��'�!"m�
�>(��a��{�o��y'��� �����'�=��	tW�O�{��n�F��VN��)%�=���j$�=��	tO��JP������Xk�x;��	j���'p�o�����y��]��x{�m~���&��|3�r3�.�l=�rW}�gA0<j�o�|��c�E��PE�t�)����?�����QqX�;���#��k\�-�����V��T�=�g��K������D�m���j��o��g&��I�D08_,�Qb6�i���A}�D�$ ��5yh��?��i� $Y0����rXAM�8'u�E�i,p�s��\��,h�w@�0�H}.�}.�}���y����m�Km�Km���}��x�R�R�~m_Dh����������_���g�Q������`0��N�H56"���!�������R}\���> �{�'��'���:1���G��$i�O.�'����?y���L���R�R�X'OS}�h�c�H������R}���P��7<U������0�K�?�PE���^�?��F���q0<�vY*xj8�u�hs"�t�v�f<�B�H��[�0��U�BIn���F��F���;���o�op/���mA�l>l����<�|���{�y��x���6�0��*k�I�=�K:������tlw}����O��\yz����]`v����1�����By`��1P|?To���
�����7��w���V�S��zJ��S��<��i�~[���r�������s4���kK��=���f\�s��y^�?u�j�o�b5����R{��p����0�[�/Qjes���$I;i�s����j��v}{��mP�t�Y\����B�~gK`f��-�J�oi��ri����^'pa�^<���"W(p�Z-��0�����Q*�+FS�z�g��R��}2��y���V�g��R���3��y����x�R���4��iv�(���/���X���u�,�W}�\����3>
a!��cJ�v����T��V��V�,�bg��?�C*��V�9�����=�n�yR��b/h�^�V�Ew���$H:a��	��N������A��W�z��.���]Wi��}5����u��3fd�_���Y���:cF��5����5���j��@#�pT�R*OktO�R������FV�:��Q��\����=��A�)A�)A�]=o��{��
8�^P�^P�^�z�v�@�����	�wO��{�"
��n�@��T��A����*y)��A]]����Z��������v����{�O{`��:u�=�Mn3�@g�s�jco�����d����4%	���p�!4Y�� �r��0��J��f2]������4u2[������b������GhM}E\E[@��!6AP�� �&�hX
��m�D�vT�(��T�,�[���0����0��j@D}G5��H5,���@�Q��X�a�L�p��y�^�n��!����S��v3��La75�Ha7���.7�+�ph�����v�
��M1�|O����6���2���S���
��������7� �=}�#�hA��D���M1�|O���#�`�����g��op��dBu!@���pa�k80d��nU��������������l�3	����_����\M6�+���P��y��k#�����l
3R�''�����Q��.P�}�w�lA
��o6��@w@N-����y�$���}
!^/�����<u�����6�^��{�k+�J{��K87�;���m��%���E��y�����/Y$��N�S����T��Tv~T����y'��P��@��3����4vR�iQA�h�
�,�
O���:�U�)�<2j)�z"+&��)��|�����T���s��s��n�C��Q}�_xA?�
�p���D7�|��G��o����guctB
��G��nN���8�o��Ot#���G��o<�o<'��F��T����/|H?�
�p����tp� 
_o���M��8{w��a�����������##=��;��U/e�wW?T_�;a�.��:�^_��	0��O���r:�wP�OV���S�8��������C�1_~�����'�Q�)e##��z����h;��B��� ��I���<���>T���f�_�i���c�U�Q�t�q:�8���9�a�])W@�i���	J�TR0���`*���`*^��`*����e�j:��?���s���F���*��@>�${@���t�{8��(����G��z��'9�h�ZM~��V| #	�������`��������������3 ,�=�N��FL-�L�i�e�bB0�y��+�v���3��{����;��$����r�/@�a��]��k�XD���%
���e�E�N�W��.C��������zw�Z7�=�k�Dv���e�:��������z�iu��Cu@��<���3�>�F�-n�JT����PW_&5���5�����-������V�$u_����r��z����U�i�)uJ'��~�{���pw��H`�@��KB$o������B�CH~J5�B�C0U(�!8+���K~��u���nx��(��'D(��A�fg��\5Xmw������l�e�[�w������V���M���W?�_]d���K�e���-L�>n
�������]���%W���-�ld��F��^��p��[��d�o�&�'�	f{p���"��9����Tf���v�
3!�`��������!q���0�`��CH��y*�X+�w{�
3���7�D����C��������~`�L��
@�V,�XP�g}I!�S~c)���;�K�.��H-
Sfv��Y8� �5��*(�������)���PA���%N��lO�^#�RK/6�s�6]�ri�;j��a���V
_�������W�����Zqs�����b1[L��������&�������������������D�T����������1�m�����[�#gW��j�gHv����QkH�[P��O��+EM������.������d���X�@�R+s�>�L������q+e����7'S���bu�
��:����T��:C�:neOS���e�_]��X�b�������*��2���=�"�n>��)L�d'Z�R�VoO��`jj]A���
����q2Si�
b������qfS�
�
BcM��4
�*���*��
���s]����v�����7�~o���f�������1�+~���:�+1S�W�����$x%�+	^I�JH����`�W�b�`�W�2�������1���j�U����o��D�.�.�?���,{���}E�����>~dU�������_^�����u����o���?����ky����r!z�����7���;���M�8����p��}��7�K�N�o��;����'����������}����2{���f����/���l��
���zM��_f�?I��Mvy����?^T��}Z�s���qZ���Q���fUWi��DP������>�2�k�z��Oo��0�z����D�����X������*s!ZhW���r������>�
�X������m�o��������F���������^^�������X2�O����(���'�?���� C�"w�p�u��E�[0^b�"�=ewo	w_�H!iD�2pf���;������r�����vY�k���]!�������_\��������-{��f�����CUs��
�"&}��m����]�bR��U3�8�"�KO~<i���P]��hav��W�_����r
�N�&(��e��z�������e�=w�7����i�V�o�����_Qw�?m�|�N&G�~����*��*��*6���L�ZA���U���\<,6�@�A��F��6�&g�>��N�'a3���n13CmxQ`*��X53��U���SG,��zA
59S$��@O!E9p���WTL��?���4�\�f��OA{C�ek8\�O�|x��K���Vc�o�<��#>��,�7���8Ak���(
��1��p�����gJ�'�xbL!{��c4A��,��@�a3���\R�s*����1����>DA�
U��^0��
��.iN��q�>�BNj�t$��q��a�t$��q��a�t$��q��a�$C�WW��*�<������V�PW~���P�X�7��+2z09�b�TWm�6@�n'^SqZ�)����=R���U�����5�I�(0�)������*S,MyX�^���N��b
OP��\�����,J~����d��FQh'�������Vr�HZ��K�a1(�Z������-����wz����,���[R��{	��6������}��k6�R��cN
�9E��OO�_'9�k ���
|�y�$�����������*�k(���{��L�3?��'B.|�!��-g���)o�B��L(�T����K.�'�9�P�By�/���~�B�B{'����\
����^��e����78��7�V��H�9��i3�o��\P����<����_��%�v��~������Fg
���O
�C��������3q��|-�D.P&�g.R&��@h��s�=���R��6Zpx*Pv&�����Bm����������7h���FB�
��L,A���0�rad��<E.�H!�B9#a���6�OD���D���yB����(p��T,]�[��EN��H��_��#��q�J�,z3���)�?a<����8�R�<!����
�O�#q.8Yh��@�k1Oq��ZxO�H�F#qN�	kG!�=���)��b�'�m�\�A�� OB{����\�Q������P�Lq�(xW��������0���"�.�����hG�0nsqM}!��0��,��~ �n�-�(��-�'����2��=�#�YC�w�����H�8:��H�Vq5�ba�V�L��(��S�Z�l(�RB?\7�0�z4yP�Gb�	�#�D��:&����*�*�uO��T���A��8����#r�0O�;�s(���<(�Q���)�A���E���H\���J�(���8C�3�@�X���<(��b���evq�)%QB���D�X�3����~Q��L�b���@�%�qB���� ��!�h �R����5�NaT���8K���8��R��.��)���{CQr�cL�%j���+�!�N�E��zH����"�$N|�jk	���c+	���`������_�:���-�v�kJ���
��,��������W��GC{���J�]��5r�����>� �B$�l1fW�(++(�W�*��fn��3d
��RW
Fg/?������e"�������AG�<bP��YO��g�J���"}����Y���S���#��tP��&��zy�>��J�<b���Y�)�g-~J�GL��>��!}1�S�<bP��y��L���AE�g-^K��@"}����y� H���)���S
��ZX�>�(-}1�C�<b��Y����5�O$d-������b�z^L�a�z	�xY�?�k�M��#sx���?�X�	)�$Q��f�A��q��F�sl�,�'\���ez�i�C������k�?����iqG�q'u�@�%3�x$�<�S%�@>E����s:'�����:�
�����k��������Z���,*���A�|�(�H����r��L>OZ��#f�!�'9~���g��I�h���y� �'��?�������O��<j�������P�do��+�J��Z��W�7�e��}8����n���f�	�{��GOQOWY��l
~S}��O�
Y���7��-�hj�������
�' k�ZNx[�7��J���f�/sYfp������h�jua�/_Asb��c���ER=�|���8m\m}�(W�(������k�n����~�F�����I����	%�S�W��;�
"O�6�HUp5��
��?�<�p��/�_�k�W��~%+��L�n�^����]�� oN���\������6j�������?p�c+S8��7d��y���U��,@��_����H�6:IR�o����HO-�����L~����P��g��MER��&��Q
T>0�)���;�Y�X���#�0���0n86Q�����U�{�a��b�z+�X� �Jb���l ��*$����D[4'�"�"�pn�s�M��	>�"��o�zo�H������%�����D����(�B�vJP�$�r!PR+<�[�������@��t����7go���&��?��$y���G��/��/��AR��C�r]
�)���Z$��0���]&�����;�AvQH�N���'3H�����U�>��i}�F�m�Q�v	��=�$��"�:��$���l��P67��f�
���A5��YA61{�n����s$������g�{����
I�^1
5�"���r�_���46�����L�0��1�=C}�*��R
�q�20d�#L��"E���xH�AdSfF���O���j����D6��J�
����C���WPi�i�R`p��*M6NC^�������(h6���u���Z���w���KYM6�u%��M6�+�x�t��R�K:�\^��&����W��GO���?eY��<�8���o"��E�cQN��:"��O�%.��s��0:��kZXu�zH���U%i]$�]�Pi��{������:pEa}��S���T�����u���������*����i�U�	 ��[|��A�&�/�����yD�q�z��
D�n��?.�<Un�6;&�l�,��`Q��b�����e����e�E��'(3s���hA2���'���,�0j!�jC[�������k���q~sSiQK��U2������s5T����zg�V��Wuo����&�=Cd�H$���Vr��� I�����nD*t1����p��p���f���z}J���\�vN�+BF���-Np��)�V�L����y������'	�!I���Z��O�v?��>��N��8$�s�+�H1�	L���w��o����8����LxHZ��\��c)��Y.�$���!�#!�����1����E�'��q�Ir~������{�l]=&���*�!��������� ���� .�v^c�sY��,�������<e�I[.+������2�!�������o-� �z~f�85o��C�h���������������D�f���p9g#�N��;��|� ��;#����{��KE-����������Xl�T�g��B�l�{�	7z	�~2pb�N��6���j���e�{�q���]�#I@�$��/V�`^u�z:���YCeB��p��9MO��3�H�6}�p��P�k}� o��}G`-4��vKx�BU�:K�C�;�MI<Q��M������	c5�����_��h.��a���L�1Vbj[���k��`T�W����$\[~��ue�Xmk��`�����c8�BP?_]���������G<��}���,�&<y}Pe�m��������q�Q�>>�����?/^}��DV<i\h]Kk�bZ��n2��p3�!O5�����JY/��8��O�@��9��N�il�������m��B�e�^�M���:�_@����@��*��K�@���	h$5���s��f��;ey@~
�4g.�������3���H=�y@J��������^�cj��\d���=�4�d�?�W�?#��?��O�r��3r�Y,��x=�\~�.�l
x�h�}_gggo^�}�����Dq����t8<�z	T����Oa��������I�?�����z�����|6A�^g��t=[���A���3�������v��Z��� �7+���^xF�'���y��:��f�JI@C��vU�?-����g���R��1�h�)����O�Y6�o�L�n��=���v6���u9Y�>W��������d3�~�������E�Z}��d1��]g������������/�?�Uu����	�����|�(��N�����v��l1Y�Qm7s�����*�i>^��V�,�:z�h�.=I&D�3`������X~���������U��V�a%8��y��b5Y���r�������>�q�;�n������"5�������-��W����%�d�����w��'���������?/���i��C#��R��~�������������a��h�2������dCZ7C+���V�eF��4�o�M���K����H��t;������tZj�$�D.���p��~_���	�����q?\�{v}��:{u���=�����m������H����-�n�g���D��}�[Yr���F6��&���?��U����5^5Ve6�|��r��s��'��:[L*��OG�o���G��f�K�����<O�`s�Y`q��T���D���jJ��+�b�����\��n��3++n�h�*�X�����&�;��_g�����%f��o���E��Zd�{5����j4�n�/K����$8\�����������M�����,���du��
������x�� �H�	����^��Q8�?��p���
�:y�.�!�@�K�etB�9=1��C��h�gD�y�����u��mu
�A��y����Q���?���	MEO��������!;^��y�������}d��W���=������U�)Wr�J����w?��Kv��bQ���W�\����+VG����C�T1+'��eW���_������
<6��Nj�-����L��0��M��1�T�m���r_&�u���E�|M>*3j���Qb��-������$�6�6�	�*L��g����8����K|��wc�"��diCK��]]K���/�<`@"�e ����]U}O�`p���1�guuw�x�(��!�Z����YV<��r���m������=�F����oH��P���l��d����������,��[Ok���v;��;gmrt�k����z�&]�(�z�~~�f��4�#�1vR���{���p��)�G������%Z(#�8�0�x�G_�������g�_g��V^�D��gpw�D�s�/m�0�&8�>?���kT���Si�5?n��Z{�T���^�>XA������-5��1���'�Jc?&@J�+0Ab�[�$�l�x{�����A��7�8GC'<�����y�&&�<��M�>*	�-s;�d7w�mz,�qJ+���3��#�x���g�������VY�`����CN���1/�:9�
���o	N�<>���9����������=R1��p1����8��"���m�3�:��}�C�����g+�L��pM�>'��d��<DL�X���{>I�'4�f4����=J.�����n���SJ3��2�{�f��2�X�Cp9��[#D^�����������<�6������5G�
u�o)��<���^"�;�		��N�gc�0���k��j����Q-�������^T��5��buT�%P-pQ-��Z�A��Q-HA�@��z"�DJ�Jn�u�N��^�hq�132��H��r�I
X���F�v�]��7���F���w���[&�K�����:0b���%�Z��9���
�m���a&U	%�=�f������r���%�L
e�3/\����-�"_�<����;E67��E�@x�����X��w*w�z��b�=V�5����,�|������a��k���j��l��w�g��qp&�w2B��1���l	������Qdo���������x�Z�e@kf�k�x����m?I�������a��[I��V��5'cc�2����K.���zX��h!��4>�b�|�mc9�kL�E��P���r�����$�%hl��h��[�0��gnb���$3���{G
a�(��������������V/����:�� � l
�1�Me�U�>��2�h�G��3��J��^*���w����M�L?�XV��,�����j)������
�`��rAz��Y��R�,*�a+Pc�`�r�Z�8��V��V��U�k�)T�Z���A�����}�/����^
v���E�b";��1��8�5�({��p��Af�p�>\����{�|5��k��6�$E���?�v>o0��uvz�'�y�����������s�����\�������pL���c�9NsO,����z(����X�i��K�T�==-�v�Y���,�>��VU;��������^$xD���xp������A�����hx/�~WA/&�cb��Do��11�c��������>��A��G����11H��@`�z\��,���0��w���
q3q���
C��A���4���>���@�_��l��r9�P	g=����)���������NT6�������[w}�:W����W�v�����u@���@z��
�y �����w��C�v��6���h�!��}���{���{$�9�'b��G��Z�_�O�
X����f���B?a������-��f����a�'�4�2�y�}����y�������Y�\*�?u0��s�'�\���������R����=���������h:�
��byt�����Q{�n|�eJOp���=,=�VR�^��t��p���~[��i�Y�%�<K��w����GNf����k$)8���A�i�	
�mj&D��L�\�Kh.P�����=-�� �6�����cwd8��g�����Y��4f����������S}I��J�m�H*�H�������Q&:K�t��Tg�����n�f#���y_v��D7�����a�)�\i�o���3�U����nu�Fp{W\����1�0��g����
�vK>��{�������sTN�5+�E����qW��l/�(�"D�r�0g4gk����l;]�5���<������Bl��v�Z���e��wi0��������8�M�dJ91��BE��H��nc��z�c�B��>b%F�����D�m.D�������& ���u��Z$��x2�
Y4Oo��T�9��?��Ye�A��;'g����.��@���N����/�0���p1�A�&[�����w��.
L�U��O!'Z���z[�VD��J%�*��������J����!)�|F��/�����������[+S��nz�JW��UlR����v<���V[y�j��e/X=�����v�Y��
�h;i_��v�Y���X����5�b��b��j^��Da���d��u��&L�k~�����-�*���0u��z������Q���uw1����{���t�8���)����h�Z��~/���u�4����(f�^�M`b��!��]�y�d�XDC�<8����~�����a�'h��Z��s��_
���m����T%	��J�rFzh��$�:���}�����d���^�Q���,��dX�[?79Mrz����p�=M:����t[��,��2�t�5�!����l.�:$c.9��?d��������Ix��-���j���"9��"������J*� �$��(�6:��pj7�p\��y8n�)�X���N�F�+�E��}���<.�����_�U������^���}?��i�Jj�+K�M����B�s��z%�C�v��Oi�^I���v}.��X��$^�>�"$����XK���F&
	���h?�����=>����}��������(��K�)�����R��,~|�������G�z|+A����n��#�^��?>���z|s�����b
0|-��2���||�w���Q����G�?>������n�� ������
9t]Z|%��&Z����x;C�����b3c�M���}��t���r��By-4\�;��7���B��w}���t���]�wc(3�X^�\n�|E���fM�sK��n����V4]�-���}�z����y���������l�s�b�eK�rFsO�<!��n���C�y���E{�:?��L+T�5���f���8��=@$W��;�����.��L���~8�� ?������n���f!�u����rx�|g?xv�����?9��^
�����������>�W � ���/�:#;���:";��)m/���!������3��':��!�Dv<��;���DvDd��������,2YGGK*�������f��66�
��Y�3�����F>Vj.���~5�I	���k��x]����:C\.fx�2�[�����K������4�j�8�k^���7�5E2�8a�Rb
i�[��}��'�+��������a��~�kt����b�q�q����f�|7��B���%����}��=�w1��|����/C)�FK����Gy��/��������hr+�$��#�
$7Q����������������@�$���PmT���Sii�8n��s`I�
���"�� |�0��g�D�g�y#g�c��b�x�����zomX��]3��u����u�a����}��;�R�O�?6����c�NF�o������&?��f��_�Q_h��Y!C����k��"��N�n��"��L�P�s�N1mR��#�
s5�tH>�"7,|����$��%h��.2_�<c��()	�l��=C�X�o- 2��.�fHK]�����p�~WM�X�kR���DI���~%"%&�h�[���#�C�msl�\oGM���������=��o��-�q�x�h�1���Fw|�3���
w�R��A>�\3��X.���\����	]<���o������o�a��DP���
�0F�t�f��Zhg�T|��;U�����}{{��V9�_����Qk^.�.:�������oI�g���R-�Z�^j���}��;<�v����@��"�����'(��h�S����Z#\f`f���1����(f�\[�����z1�����")����_F������ 0v�`�����gO���`bl�2�89�y����U&����<p*�;O�����{X��Y���������[�Er-�������� �\���x>Cl\��7�D_X���o���W���z8�q��������g��rAz��Y��R.0�
��
�~��~�� �_��n�~�za;��^��^��WK��3��n�UOZ�dRy;*������`N�/�����N�+
�Ph���
���{6�v�Y/�����X�|����i3�T2m>��|��y�@��is�c�~�8�g��
�)�~���pz ���iGo�P>N���t���?<,L��$D�U����qz��s��5����`��`��`��]�_.�b������3���W��7�Le!F�����@��������=�����"�p�����	_�!��i���~��B`���K�����#X5�Z�6YI���9>�~�w�hHK��u8��u(C�u�����d��~9���$.�g�u|1r����ql;�-zN�@����{l��[&[�k�x�"`	� ���)�H!��U��Z�����������$I��q���h2@�������)�jK���c�n$��@�H��4�$���Ct~��|�U4�|M*w��Ng���m�\�w
�
g
����<1��Xk��Z����9o��[V�]�7MFp�me3�����l���M;����<���{@�
$)��Al�����nw�BP���;��Q�TfF�����-U���Fv�^�+#�c���;������F�3E�V��%U1w=���u���9	����:��;�����q���&e����3�a�Z���rXo(��z�_��v�M��]����1�0��w�i��h�Y����w��>E}����J���'9����?�H���h��e�����XC��e�Nv��zL�g���b2avz���������o/� *Av�Y-�@��bv�p��������>�H���t��y*�j�C�j1@�3�� ��D����{$���v<��b[�'��S��[H�$ ���[�Qw�����������`�j�?W����A��,y���jz���j�?WS���s5H=W�%�� �\Mo1�\
��j
T�|���j�������-f���Z��Z�\
�s5�����\
��j�9W������s��s5��s5H9Wq��'~���W�nK����*�k���A���lw&�Nt��/�D���Z�@vz��@�D��]��Z�
[����
j&�����7�_1���Z��c`���Z��D_����o�Pc/�]�$@�;�Ja��_�y�8-_f�)�����3���J���g��)�E��u��D\r%�  T�Yn���p5�~b8Qg-�y&�<V`�+�_/)����9K7�#�����k����p��aV��|��kw��t��A<��E�8M��zU������C�h�^�&hgK�vfwv������Cl�����������S�y��sp5�}��,;�j��u�BN�;
�-n�������}%���������P��~F�K(����9r��X�
w����w��1�VuNj	3bp��9x{}��98��g+zw����{\>^�N�u�w��8����Y�]v���������6(��1�;�8����	���n�#��������D�	�����v:�X���!k��U��r�k�u{���4G��y�����py��R���*��}	�{_k[X�]�y	�u	 �l �
E�:~K���l%��J���68!M %8G^*�)��8Px7�jv���%j%�,g������
���+�
����6�I?=jv1�'}����>���|`A���}�V�.RWp�y�QWh��������qx���Kdt�������9��$���K<k!gm���0��f
A
d��u�������������w�V��/�/_��/_(���g8�/�c6M����t��u_���m�6�c��yqq��7������9��G�����/_�|cd��Z������v�������~�_>Z��D��ne����g'��a���,e��M��:#�^�{7OVA�	MF��aI�h����#��F�N4��I�n����w���7=���a�]0_��dc��E�����S����
��&���D#�6*�����D"0%�����x�cwed����-w3%��0���C���NJ�M.g���)�js�q��������7������Pk�a���@��������x�P�o1�ib�5s�s�e�p��Yl������z����K�Z�#�p�m��MAz�QP#��?"M�
{}������q?G�Ks"���b�~8C=S��!���h6�����E�rz����P*��H��)�+L�%y��c���@6�����St���������z�m���a�Kn�gOQ:������Y�2����7���p�q4���Mh59���4������i���~mNN�o4��K/���y�,����������������T��j��L��w/ �s��w�_
����9�Yr���'>K�|���������h
���|�(.�u���%t�r�a�um���qQ�\�rU
�A������V�5=�oR�r=�����!��9�m!B���Bc��`�!
��6�m���6�SB��^�^��<z|���I���<z(��Zj{��,���=����e_��y��Nz������-7o+���X�NKr�q\��"�����X���g��/��
����
Kji�.��G�9@�8j��$��V��I8���nK��>�����&(q�<K��K��������K�R�n��p(�R%���)���d�,�1f&��t;�G����C����=�^?�m�.���!Yh�0D��Kx�U-!y`�)69e$IR�h���wn@x��4�����A=����gT� �2�%jGKF����s@' �%4�������R<���t���lc�]����j���)�^�����$�mu3t�5?���D���J�c�����D!%a����zKf�8��o�1�"��k[�L���v3�o|}����1o���/�y�'�#h:z�t��\���F�kZ��q*mQ�����e#�����y����_�pM�1\�y��$[�$*��Z��~K�����8��Z����>������=R�(c����#�X����4�4�[������De��>��7k���%.��T�p�����V������6�r�uL�F�o���1�;p�l|X[��9���=.����v�������<K�l<z�^����G�������hF�A��C0/{�-tLiL��%�g��^��k��]#��������K1�*.wm�6<��F����0�8;7u��V���	�3c� ����;<�����{Q���c���(g8ln�L��������gw:�
�X�w��eE�SH��cWC���<Bq��<�D����� ��$�$�{��	��z5�!]��Z���\�[��f��C����e=�g�����rSx���sa����	��#�6`�A��i3?3���������Rd�������{�h������'��Q����G��T-c�����ft��<�y]����������Ol��yP�����J�����^�_���c���6�����".6�����d���"H��za�[������xvqq�� h�P	���<��������0�\�����A���h6��P�&��#"�<��x��0��G�|W��)�h�����wa,+^/�)���MLBv"Z��7�HL�)y>\�8���g�G�]&�[�x�����o%k{������_�|�C
0^d�����7o��<�g�i�O�
L����f����
�`^���<�*~��m�\~-�5��������E���+�u�[E���/�����Xs6���������p��|�M��f/�����u)�3F���W/b���i�^�4y�����Tx�~~��j�7�nw�I�����/�]r`wiVpT����9���MQ���1����#�{�LDWy����Yo�@�}��XQw���K����7g�~���G��:���?�+)F�������3����o������x��\w=�~��N��A�N�IvWg���Mto�-Q���K��E�s��������;S�����1�q��AB��Y����n)�F(�{�C(�T�~�}?�@�Ow��'�1.���!�����l��-����+:����H�;}��k���B�	�g�0H�����Hb�3�hgk���[I�F@���[��h=��/�(��6�M�tw�����~:
dI]�2pkmv�w~6[���v/i���� "����e8��%����4c�7�������`���Z����1�#SV��V�c�T��`�����eC*3��;6��K�!�����[��e7Xi���7�8#��vVxS��*g�8X}��r�xsr���E8����_J�g���`-������j0k3����LM"}�W��'��'\��4�!���bV�;���)�S�y�0[�$
�����|)��H��y��������H�V-8��Gd��������]�t��>���X�G������(���P��xk�q-�y�����������]�t�h��Xmu�[o��%�K�]fK��!�n|�8���X�R^e�<��R/y�������K�����������r�� ��~�8�m-�C],kV��g�.�H���w�t�@��e���
p�Pv���	\5clI7\�k��;}��A��Bp�Zn!.�1��0
�$�'=�5d���UZ���g�4�����Mv' ~7�^W�/sp�f�\d���nN�� �+�����W$�.
��������b���-����'x,_����>^���w<�����!M�����C�, �Z�.Z=��6�xG�P�-=���G1������	�����'XJbJ�!��'���)Y���fX$���*w=�M�T���mLj�+m�O���nb:�X1gr���,����'�]�`e���&n�-�k�6�=\�l�p���E|C����e�pz��|4�6�����`6{�, &-<�`��������USC�@��~Z�|���p�5_
��_�����1X�2V7�����{EB��$)�q��0]-�_�����"&5f'[��6mT\T���[�k�1�c��<�;
-c���n&bY�=)���*�d�Z?��S���AF�E25�QAr��o�4{_B��,�7}���J������=�-��\���2Fm���Mk��r�w��]O������n'/���.�C�!|�g5�Z[�m��ZdH�J�pfT������j��p3�f�kU�$��C���Wh�����������Y!��S���R� �N��A��S����U�{p�yM��1��1���_���1T�<s� �1�����w��0�u��
<G��bg&�P��U�f��.N�F�X�m�o��PJ�,U��v
����C<����Ku����S��-0O�}�@�@8�|������O�v�\���e����a�G�[G�W<�;�w�X�JXo���`
��Y�����n�m��wE.t��9|�� ��]�o��:�����+jw�=�bw�|7����;�������'�~�D`��3��K�Z;s��?����(�w��FL������H�W�[�T���?(?@�N7�R������T.B�J�:��;^�b,�ro��������=���!o"�j6[3va�(���x/f�����`������������hL�]������9�������qFM:���������[�s���_���(d�v5,��+����nu���5�k��z���u������f#��C�M��E����_�l6�w.��V��Sh�����4|_�
�����)a����7�d'���9�����D�/#���qM��;��=��z4�X�9/�9b�y���z�W	��3t�OS#@��2�_����Y��������Z3t��(�U�?��p���Gg���+�$OV�-�����I�D5�~q���7w���2���yG<[���(f���$��:Z�$l��@�5 �F��ug5���.�������5��0�|�]_B��X�����[�|��%�6�H��Y�����w�;s��x�^��w��,��V����wgr.v�m?#�i�~6�91���q�[.�2Ce�������M�����*�x��{'��z��E����������0X!F���R,j�A�bQ�t0���%��V�X��g�i��?����eCcf�c�X�<��\�E�J����Dt��
XY�������������c�(��x7������E���T?Y"��W=�5������6c5P�?0z>�����
������Mu���u���D���d>�����t�w���Z������7������70�c������V�Y�
�fFwKr�Y����(��r�g�6E(wM���;����	���h����z�����w�l���~�[���:Z�/(���e7���>X6I��-�������t��x�Jf�g#��������,�$N�9Kp���M�HqKz�++@����0�p��]�cX(+T�7Z��bsk�~��n�Q:z�u�i{��`�������);����D�����8C��j�~��<'�(P�j����q������yw	�����S��vpz�J�g�^����a�N�N����
w��[��X=������|�u��PZo>���C�Yc�����zg��l	����X7x6�<b����H�yI%��Yn"���}r��b��7���d2
&�+=�]�2/���\q���2Kf�`���-������I��L���kJ�!@��,���[�A�!���i7��������aX%#�������{q<�b
���p�a�a���
qg�������^^�]����/Qg�0��^fX��������=��K�Y4��%��G���*��I�����_u�SPcA������N2�sb
Sb��S�
&7XF��XR���va���~���*�L5Ct��	2�4��3��'��5���>�`�k�[�MG����d� �dl��P>�w���h��|ml��z�K����Y�[Mbo�5��=�j���m9�Q�L�la�q��������\t�	-�/���/��4��L�7�����@���u��|Z�������������q����T[J��\%��&�������^�6@��vs���a�i��$D������� +��y�n������+:��
}q��p����XQ#H�^�o8ik�G���Ga<U�pX��0�{9y���8��W����2��{�h�����z��|P��Y�N���98��f���M��������Fc�-7��(���!�{������6��������k�$�XE��B��I`H����[�J,��}X�F��\�;7��CX��`���g��h��o��dy9zA���@�9��6���F|x�[A��B� [nP������]Zs>N�����i�Q(���������7{���o�D��J�Z=������h��������v�O'Q7>�
Q]r8
���B��C�������d�9:��w��Q1���F�z��j�Z*G��J�O�Z��V��l��O��?�p6����g��{�����qz|z\	*�j���G��Z�v��z�R�G��Q�S�v�[~�]FcV=f��S�?�����L��}rT`�I��[���H��H\�����E�U8��B�%V��x�[�mW�O��i��T�J�r��'?���|8�p���z��(��(
r��<�5���[`���~\.�����ys~��'��Ph�v�x ��mR���
�l8�g}J*����;���
�v8d����!���($[��@R�Jv�N�����7�hN�7�e����p<� �E�����m�&�b=l�����Rb��}��|a�:�Mh��T�����~�
G���o��MA���3_c\��� ����,���A��G�����,�!�@n�C�7��o2$W�@�c}�t-�1ou���k�&����M�y��
O�hJ���u�S�	���o���{��R���h��Nb=��i��~��Ze8���U���O8���"��e�k6X��H�4.��C�Ka���0;��M"7+�+��|�����S`sv(P
s������'��.1�Dq{�k��	[��������+�}�?�@�	SS�'C����DJ����x��=Tb=�"lFU���o�����.���#f���!��#����j�S��E�n��Ou�?\�5��_�C�wK�Z#><�P����Go~��XgTFX�}p���m�G���y�-�
"RE�b~���O�OZa�#��OFc^����	p�O�B���i��c���`���W�[|in@�)���e��P�x��]���~�����u���Or����X�7�D_X��>�N��rp�K��������s�W����_�����;���2���uY<����������p6��lo��c�/-8� ��+�?�:��_����OxU���=�]�Q�W������~��~�;�Zu���,w�S����JF%g?;u=��B��3Qr���=�����F84���1�?~xs����s�Dc�������_���e��~g�
Q���=+0\�j���ZJ��X`��k�7��p���Cm�Y�y$sh�a��x2MK|�������������(�N�����!�NW(o���9��&�w�N���F��Y�����P�K���?�y
�`t���:?�|O�i�Nn���L���%)���>l��_�?gr�����.?#���������F3�!�F?��t��4��.�����E�$�C	7�3��-1����q�t�k�{��?�����)N��x�9��[�:>�J�v���)`��BA7�Q�p�C���9p�I���I��XY�w�����J��9���+�_���)*Q��M��
��b���uzN�!8�p�[Q4�F���N #��KY��v���:��'�M�����(rr`��
I�\��w�i�@�Yd�!o��/z��1F�P�mX����r�`
�p���@�A{���)I}����D~p"@��<&��>��d5b,8M�{��!P@����:����_g�9�N`{���^39�2��������D�&W�}��q�.�G]��\\<_K~���!'t����#�sv�I��VR��Z��C������z��1)8��$����Ss0����y}:%���oO8��X_wq���x����	z�1��@�q�%�xF�+y�-��\�Q�	��? u��o>�����op8�8p���xq�9�/8���h��2�M��3������"�����0��aJ�8������h"1�X�
��G�A�!���=Y���W���9� ����FAW��'��{�� ��N��r��0{OC���$�z���P��``*�^�PB�
	YV�
r*��|�\Mn�*�3Yu2�7LG#d��t���� �no����#��u~�U��6�[��$sE����c������Y}gHj��Q��������
�]_�%�w�Q��T��w���j
���82?��c�[�9�D�l�����b���F�������Lx���|HE��~�}�@�I� i�
�|rf�S�V|rk�s]�k��2`J��	�����a�C�Ge�^��dE�{��&����!���%���+}��#�6���79)����K�����.,�k`������1�X�*��0cc�/�	��
��~c����IL�Kj�p�I����Q�#;�c�u�y��Kr�p��_Q�;|!`��~m�������/����~x���������+q9s���t�pr�>5cl�1�=�����A�3o��C��?/����v�,g�����t:�+&���������u��+��������Pm�fk4v�f:�����Ky^s��<�����r\iT]�����w;���+a'h���z���a��]o�k���I�}\���+���^���Y7��������Z���RC���P��PP�����	-AC��+�-R���v4��Yn������:�K-Z�Y���G
=��&�����uK|4��V��>� .��t2���D3{��L�T�	������4�{�I�V~������R��*����FR���+��2J�b�(y������z�-�Ne����D�'�)��5!���p�"��`��He3��
	Q]'eb����(��_��Uk���?�~�f.-uj�tj���J�i���HO�@����XK�6(����N���Fu����B
us�V�U+�r�D)��l��oV�y%�N�^k����;i4*a�N��FR�+�~������\���kj�_q�+�W�$h���D@��=G�A7�C��R*Z�B��U�0�ct�����3�WCNK���V��cx���~��WT�'�[��Jv�����{h��V��}��K�"{$^;����������a{��h�� �;������O�Mg�d�AI�0���gH(���"l�ZG�"h�s������
�
~
�����_|=�l�*��S8H�L�7��GuP����sGm����i�	�����\��
j�[*pm4��o�-�]-cz����1����_���_8����f� �9J��B�C����-��^I���'06P{s���Rl|����4�����#s&�A��>�"?��Et}�����������8�:������w?��!�"o
>](.��W0	�D?�$R&�z�������q�N:�r���$�VO����f�'jJA��<(������q'��Yp�������G��.�����y���eR0
;V]����*��q`���c����z���%���k	L��Z�J�V+'��!���J5��$e��(�`�q�c�q��"�i8N���\����c�D�S9Fi9���s���.��������x�����r�6�Bh�{��
)���'`D�as}n:�,r�r�s���P��U�����8j���J���%#��_�sr���N��G�i ��4iO��)��6�W�4�����6>(�9������q����S��*�N��a��UC�:���VuC��&�4�6������j=Mt� EE�E����W�,������h�Y��u_h t
0C�NL���hr��%��c�o��_g���G)���8������s0����dD���KZ���m2��iX�s(D�����
�'a_���Y{��B�#e0���QKb�
-�������d��A��4w�l�:�����>���"��v(B��F�P�(k�����Cs> �#P?q�"����Z��F�RK
�pL���" h|9:�G{��a���?���>]��������U|=�?�f�w]����OIi����:A#��u���i��%V(K��[��x8�y���+W�Vg�}xF����	�i���s�x��J��dwu8���X������5����\!����$�K���f�D���R0��A�<!<���3�-�:�j�J�m���Czq����-�1�D��(:��(���D�Z�%��z���J�U<o�*��*������V1����dT������T�`1������$��1�My4�J�Z�{�iB�[��\)n��%�M=YZ���xR�T��O��z��S��m.1�2�����/�5`��Ze���;���N�Y�a?�F���������wO�r/X�[� V��'`���F��O>�x����2�����:���{�0B���$��
�2������C�	�!�����;>���8��Up�X��~�����	��W-�Ba�*R�w���|b�:z�hmf����{"�Q�=I��d������-�����U�^L�EJ�3�dm�(F��)��7�n�5��o�^F��;�#}�6�;HP�IDf-x(�,�$����0�v����Ag4��C���Lh�4��
]�n����vU��.����i���Sa4���I8��x�SD�C!?��dc�`s�r8��+�Vi�t[�R��������a��v�?�[�V�&��5�H;Q7�I��%HO�u��W�X�bn���=�[:k����1��}�U�z�N���oK���Qr8�	�-���^K�]��m��r��������M4���&����r|s���0�������_��^D��Y�y\�O�8��ACE��t4&F&*���������+���81�5&Q?/Nag��@%8�~M>
�75�lB�7>�#)��%������-���K��>����4g1�����YC� ��p� \o���	bZ���ppf3|��N86�o}�M�t����
?>CTQ���D�s�&�"�8�z]��M�+��~buz��h�Po~�n�I�B�/�;W7��K�2
����,� �M;��M����ad�"A����	�fw������Cw*�>�fh���A��:�y�b�'((�1���d��z�N��)��[<,�������B�#�GO��sx+�jv��������BG!���X�:� �1�A�����b=���Ee�Z��������<����O.����FWg��>&:<�pY�v������N�	�o�[#�a�@����,��:|I���}����}!�&��n
���*�:P�(�h��d4@f9�RQ�/ei����F�`3��x,y�z���A�����A����1z��-�W6`Hce5�KR#Dz�T�Qg"7����������51'�;��d�Vw����pN	:p0��k�p�pJ��rR !�I��������i�J�\>jE��F.� �T��(LF�u���_�:(\lr�x0�8N�k|��������		�UF������&o�j�,�C��l�zm[!d?����{�����%�K�"�
�~c0�.����^�H���x�H���"?>`���W�`�����W��J,��O���h�'�,]{(�%�,������������!�������kl!���tbj������08�W��� ��Z�\���2�=8e�	��h���M��7J~���A�7����Gv�y(��>�v>���<�/A v�j�+��
'0x��o����]����p�����w~�|u���?��N���uZ=.�qCDB[p4m�����/����T<��F
���4��0��l�L�/�__\�2���Y���w����W�#x�>�.���!7�_���L! �������Z��M:�
o���t��=4���v� `dkz�f��wz�z��y�%��fz���M6�	4�{�4�/��w�`�����7I�#q�z���<	hX'���cK�{���w�y��|.���� ������� ������{{������������`��('G8�.)�.���fSmmLQ�8'K�oy��J��T�1��D"k��=��S����7V�|(j�N_����q�AG�����wX�����:�hm�����U58Z"y�)��;��[��qZV>
]��huY��y��z��(8�|�t���0F�gl��z��
t��,��.<	���s�[D�`H��o%UF���m�M�&���h��������e;�g-�?�$��g��1�i����(��6HxrR��Z�B������d�I����s�������@�
1��2�V@s��CI��Q,ZRs/a�}�%p��bm������_9t�g@��1��kx�gh��:�02��	�6;�T.C�%P8X��p���E��|?����t��!
��[]D����L-<)�
�^��J�e
r��U��
���e@@�K3"L4D�����>���g�����M��=��6�>'�:��h�hd
l�He9�C0JF�r�,	���Z���'u�#,i�	'"2���1X��w���$��@�S��	b�������mb-A!���]\��I����>B*
9
+y�U����b���BG� ��A��Y��O�k�1A,�	1���c��2�_�A��
�� S$������l�;�`-S�u}4�0o�G�!��������
��1(����1�}�|���*0'h�������hhI�d��^�"����h�?Sf�@�s���l?|!�?�i,����b�B]4/�}���&��~[��5+8<3kQ�6?�f'[Y��M����4nVR�yY��]����b%���*�4�UO������m�3�	Z�hRtop�438�	x�k�:�#�����XN�1��3�������p@97Y�V�H����([�<	�����^���@Uh�p}&��<|��[��������

P�8-���&nyT6rt�X]�1|qu���H���=3���E��������
��>^S��C���N���N��X?S��l����T>�?QHV$pl�N�VE;	�^_6!�=����qds��oD�3~"��HK�5� qy���}��@�pj
uNi���CG��:���9�+E�}�X������l��F_��S�h������^Q$����VX�[�M���C��{9��?�]6���i�{P�����i e?�8a��k�O�Rjd�.��������,�#,S��� ^���FR�C�>
c��$
��1������vv�"�����~�4���$=@�Dbb���R������`!��^�����L���<���G~�p\��}K|����W?�}�wDx��pr7��#!�o�B�e�m�X���D����5X����s������O�R�z.D_�c�cD$#����ONOI0WJ5HQ����������	�$(K�k�c#W�bA���[���pG�qs:2�]
-N�[���)V�$���w��o"N)G\Zv����/�rK�
k�Ly�.'��Q��Qkw��N��]�f����4WJm�%��c�C��(� ���,0��H���_��=)�e����DN�q�
Dj���V4A7���r��a�	-]N�T��jQX.���N�Q��FFKs��(�kTEH���w���O��:���%P��^
u�h��
�����������@���4P-mA~�$W����is`�6|4�^�����	F��0qB#y�����m���8�M��C~��dwM{�2��8�X���(E�8m��V4_j	�
�:&:��^H���4h{b���[6�Of�L�Pm����Y�_6U�w���j���_E�1
bHY�L
5>S�{DlQ�Y�U�$
����J��s)�j�Vc2�G�a����6D��O�j{
�����^���+m�)�Z����HIq���b�E��A�D�,�q��� �H�=.�#��c88~�9e�	�u�%���;����>3�f��M{��h6iG�^���,N)by.��)� 3���"�/�)�*�d
��2'-�c�_�U�ZS�	������`��9+��E�2���v�@?����\�%�i�56&�K��J-��f_;����-tw7���(�w�Q�F�\nuCNgy\������e���C����8E`�y�Xp�@�{�"��U�k'���,�����������Rh���qj����68�����D��pj��~|S�	�SQ���LK��[��O	�
���i���ZC����X?��3��e�qM���*� ���Y����1���b��K�=��m��sZ6���aZ��pF
t����i ����I��M0�4:���Gc��#��&H�s���?�M`��RyU�%)��%�S���48�4YR��,4����
�O�Y��+��;P���-7�T��
����8��V��pm���hLW7p����Yp��E���&����>��U{Y��DU�7(z�:AW�0!���~�]em�������i�0@�]!`��m�2}��H(�s�a�zzL�x8�\�A���z��f��m�O��3)����!��e� �(�Ca�i{�t�+|H*�W��1���3H���E��-��������Ap����IP����gV�IE���(<ff���c���bB	�=���N�ck���j�c��3�f��`�x����0��7��-,��t��I:O&ok`�d�A��I��	�'<8Xm =o�x�t�\�;"�����4a_5W-Km!�i����~��r�j�PE�Y_!%���0(����[����'H�����J���B��z����3�\J�V(~�!Pz}���C�ae�A��
����	x4�]`	��_A�����a��5qixo[u�|���Oh�������V����+IU��m���,��
�w�
U)��;UH����m��>��0���y�
�r�S�dG�����E�*���=6�0��m �;�k?fC[y����5�c3p��:���� w�4���)���4rq"�@C���9'���c����4/�L�J�(��
c{������S zm�j|K#���F)�u[�(���*�>X4���d�>���V'�n���������q���A:�]��MT�lt_�E��X�8�J�F�F{�����@@Xe�*5���BG�b�V��YKs��~��CN7���je�t�3���>���Ax�	��������F�x����"����m�����5
r]���~"�B��D��QO/�^I�^_p==8��`��|.�'hp���w- 4�����<^���FK@� ��N�j��?�3'%]4�@�	���
U����>Gdk�����D&�5  ���&I����mb(8,Ji-8���Q�^�����c�P�����i�Bz�D�9����FqT,���}�N�N�V�5���'�_��������7u���/Dm���vu�����78V�c_�.��^	���.���J�Ug|c��`��X@`0Hw!�"���yg�X6�'h��v�c���O�&mW���u;�����lp��VR�������#l<>�GS���!&|N��H�H&���p����=.�f���k���QR�Yg�$a��R/`������0�2�AXBY�fM
��g%:�NW���;Cn��#<���`Z�g��o�����:�
��PE�����'
U!�K�X��J�Z��4������Ya���-aCrn��P�q?�3��K�[�}��q�T��E[(��U��(��*�����6aLi!=2�F����J��H�E,��L����:���e����a���������w���n�s}�S�Tn+�>&���NO��K�s��j�K�\���1
b]L�����RO<��j�3��\b�'��l�hZ������9	R���sY�4�,|3�����IEV��t�
M$�U����\B�
`>���K�PH�o�:&�s*d��������0�Vg�3y�>������c����G����jo�g�M+��(���%�t��� ���!�C��e��?�<Z7�)��Hd�6If��i3dr]�{z��rPa����6Q���)U���y�u�����$Od�-���'��$����������)KK��@�, �IL��'<F��Z}XM��J:�
�w������&�d4i���3xRM>
��7�X�z��&����H��X�����A�Z�W����8���`dTT�$�e&tt�b[��1E�����l���?���aM�=�?�nxi��t ���I����s#���v��q7� ���#Z(@Z=@9@��$rr������*�'�o����)�h8=���A�^�����vT��@B��raa��j��h��gP���Q�SN���{��Q��h����/FcJ�+V���D4B�j9��u���k���#~z�:�5VS��#,M�������g�����M����W�#�\�
���'������-+�=� 1��s���9��1�i�OU���'n���F�L��f����[V�3���uuB���������Jk��C`�����a��N�<�R�o>�6oJ��K	!S�F���I+�)���
��s��Q9�'���=V�APB���5{�Wz�"^�O����K�7���:��80�bx��!��m��"�yA�A�}j(�as���G�a�� 
D��~��R��4��[��	T�<�,b����"��%f����Pe����i�p����)9����>���!6�LQ�(E�_�3��E��8*l���	��J2�6�d~�Q^g����J^0�w���V\�}�10Kf�n��M�1����a�Q�s
�p����kWf��3�w��9�8�77���0���Oa������E4�"�M�s�~�L��Zd���:
"P�$�m����TQ��6���OF��%�]I$����Sg�>�m9o-~�v5P�?�����{*}�-�A3v�"����6`��$^�B8t�	�5�IJ'���B�HzP)!f�s6�#��o���k&���Z�U��a7��:�����J���lv,qk���
��I���!Qt����	,H�'�4q
��9y}�8�\����!NS��y����F	�R�$��y���"��NA�QkU��r��i���< ��2] ��hi���)~7�
���6QPF�}<F]$�����`����h���`�
��m��-���B9����a�M��8`s2��D9B]���m -�����L��
i��r�����YL�xfNK��)��H����q�����7ao���.m|l*�iH�����Yq��=�-J�j{�L�������9��J"v�\������z�f0�m�'�g�:�'�����U$�v�{C��	}�=1[kJf3��,G����`p��}�@|�{�f������Uv�)������Q�)�����{^)�'w� ��������Wcd,�-�9�Jq����S�ddXV�Ym�x���a{4�a`q�����t��a���G�</2�vA��d�A�9����FXU;�o�S#{�t	g���K�=�V����|8Z�!�no�B� ���2���}����i�!P�,fit�SN�'�Q����*�[G����Q��m)�:q�RC�����:a��q��������Jzk+
���5�-�
%��O�`�
1��M9�y�1-��ko���^������J�t�������\!P<��G�8��n������3�F�!��^�X�P����5���/������!)�������a>�N�������I��E�0]@�f�A�KS�C��9%z0]�E�u�P�����df���	_��F���+;�"�K����G�R�t}���h��.h��-�(��w%#��
|��d�FoPdp��TI�������e��bO��<��zs�7o�2��eqj�9L� |��?�.i'���OM
,��L�~YO�PZ��zd���|�&�cIqM(5H�AY�{�F���hW}�3z�lnl�7[''��i�_(A
�Q�]*�/�|�T��gE����:�NG/}M�y%�(	�`��8�ZX
���:dJ:�4�w:��
�-�T����
��'���0~rd�!s�00���������=m
�!�|�A��Y������,d0���r-���D����C��1���������F����2�V-+t��cV@v��lL2�7=�nj�o^�e7���7G�Lk�jeY�p1�����I[�b��[G���z�|z\=j�an�N74���EM�4}6-�I��"���t=T�����'r�������`�
�ad
b��Y����ot���f,���f��S�XM�,�`�V��}wk�	�T��n�G��A82���CWJ�EFnT�H�4:�0$��Z7=�����^���p��e�>��Ai>���t�����Q�����QDI��9������c!��p�����f��k�t
Lq+�4�h"B����Z2oB�r:J$wD�L]n&J*{c�4��;m����`=e�fo�y\eo�Q��o��e����v����g��\���M7(;���z:6�� ��<Rpz���#�GJ�ra�nH:�3�,d�L
��ED�R��"�v
�r�=A��5�>�M����A�&8*#t�M�
���-�\==5U_2$KO�2�����-3\���0b$���p��
4X�}�����!ZH�������;��%�'=���F�O2��p+��yHzTf9���vK�<�f��S�WCZ�)Y'yu��9�M��~���e��t�����u��oP����y�����i_Y���>�M6��=sJ�N`�W����,0��T�	��~3I�##���x���b��7�)s��wQ'#�I��A��65� B%b�fw����U����c
���+s1Q��-��p�A��&F�@/zS�� �v�D�����&�t���2����ox��-����T4�Yl�a���C�F���"��3�No��M�L(!a\���%�
b�00]`�f�&&bf:�3�Fr;\��.��������/��N��%�F�1s���+	.
t�m��T�������FKL�����a%^q�6��{�
�7B�4�B|��z=�T��{�������e����|�����A���K���<�	[4���Y��^T���	�(�*y����9'BS��f�q�_O�>�R��%^yl�D�y#��/�����3c���(���/E�=�ePq��bxB#���g��g$�M���Q�6�c4�WE�
V�2�)�5�%�$�����2�+-�����q��H�}�c)�?9�p�F���cY�bB���mi���7�7=��-�P�NF�����T�A�:Nf� ^�����S� �#b�	nES��� �>�@�)�4ag������(k��������� �D��j�9c�^�:
%@G@4����_ }���������A��(��Czxtw|���2IM���U��i�����lNLb�G���D�	��TD3���)�)�U9��7�nF�28!;��h4�>��}�K������zF� ��J�h�@����z�1�<�8&92A<8_�=.�X9�<��M�b3�q�
�7kD))�et1�6�|�
���+x`I(�O��l�����0$�/�}�������Svpr��
'�x����T�{v������x�B��&N�H�y%�$Sl�"h5�>���3�-��PXX��t�J��[(oI���
L���|:ia��$�.
�PJC;�^��~\
j|iO�J�#[.��jZ����2		�XJfp�s��dJ���Adu
1+7c�� �i�Q9N��q��g	�$��h��	������a�V��o����ax�������p���D���%z�)2a�JHa���w
.��%9������%������v~��a4`?%=�lZ�w��8Ndr��R��KN.qr��~�jIh��I�����#���>�F��k���2��\�"U5fo>�L���=���)_n4��i������h��  �x���"Y@?����3��}�L!������"�j��1u]��F2z	(�
�����������-�X����GJ����/.�_��7���2&���4 ���O�_:����99�����k@;���Y����TZ����[M���^Z`c��_�&B���!l������C���N���Mj��\�1m3��$�3��4�SC��>�_JLR':2��gc�'+����M�}"�'��I��2\��A����P�����&#$�E{�^"���m������+C���l��-I��Egy��Y0oJ�,������I���[��X��nt����9���������$\��4��V�E��E� ���*���>�\�a�20�����PZi������6I�G�)���-� m����	�'���	�@ }��6��"I�09��Dvc�LFA�5�5��#���n�mL$�6Q:?��Y���Q�	����kU�mE���k��kO:���^�=��Sc����q[Q���?x�\K����$w�/l��MW��r��:iJ�D�-��?2b|)�2���UC���0����Vp�����2�0�I�����7gnA����$��cHU�Vu�
�F����B���2�\&JB���L�4�=1/�I�����xw���4���E�<��o�S�n��(��3��=�6���RKX�����v �X�0��3P�vfm�DB�$�@������`rS�AU���4)���bq�PK��� e:x�8T�J�VD'����������L����������0_�(_�ye���)�����q�2���B�)��0���~c�Z�K�1a��L����#�
U�sJ�E�\T���o�C�����byF��#�y��M��<1��p��F�7��5g��7�P�xC �<+�}J�FL�!����+�i!<F+���������������b���d�z���1D������n0wc�F0wc�FD�5^�����<,��v���*
�O���L.�V��Q/n��]���B����$$�?��&�;�t������W�
F�	5��^KWc��r�SU/��y�s��3E�y���
��X���4��}z�%Go6��������t�&�<��h~Z���)f��8�����_8)�S��9�o��Q#�`�c�P$��*|����iO\p��e)���,�����w��r}_n�'�&7���-k������'g��^:�	�a���@6z��(�-�<��|Q�����;����=J3�,���p��$pg�W[���{�i1�Z,^�DV�)�j���@J���^5&����pu��23�4��

��2Y7	���8	�C�e!��4�@ak�*�q/`�������0o�h�X�z	��>g/�m�5�Bgk�Fz�@!3��3L�~����Q:��������3�D8	:�|�!�����f�b)�6���������p��*��U���
���3!�'5�z7?p4&8����]���sI����CQ����'��N��98���s����0�q��!	���o@�E{�7�E�]�I<[	-�@l5�0��
C�&��&+!Ha2i��$E����P�
d�Y^o����^������6QC�6D��
����������	�N�%.��M(ST��B��9+K2�LH���+#���`���������A]!���j��;���?�d�DHF��2��H��!
���	�&�T����@�P����$T�)iT�E'��/�"�,���Z���3�%�:�[ OS*^��S[)���!���ybDZ��J�� ~������K��_�@�?�!7�q!�[O��j�?����v�9����A���J/�N����S��t�C��]���I�O���n�Fo�P�\���ZN�H��=��O�g@}��&�:h�@�G'�r�� ���B�?�|�w���pc=���������d:Q7���$i�~RI�64��������	�/+���xl��>|��S(�����$�����r��B-�����r2��4k�2{@#B����������d^>s���y}s��.<��|hXCH�>6���8��\������p����z�N�np_��
'A���+�rxtO�O�C6$H�jHA}U��e�K
�
����u#~���g��������px����UpL%��K������M�f�o�?�l�|y~qy~Ub�������W?�{�������������:���ls8����I��a'���S
���m��=���_��@Lj����u��(s�?����R%7��k�H��|#�+�����
�nP.'G�Vx�D��4J���X�#X���U@	8�i*q�����c��3����}�X��5�?�uL��������^,,���L���4��A���N �~�k�1`�
�!��j�������8�����r.����*�l]:���O�'��SYj
�'��MH�&e'�S���T���P=�����un����&c���z����&��I�������pH��8Of���<D���"Z"_���=�AJ�Q<=�93����O�1�;`
u����$����������� @:���>q�(���5V�N{��S��FC�4'sF�n��c����{�?�[��MI??��|~{��{r�����mT���YV={u~���Y`�#!��{����9g��`?3T��>����}N���N�IV����s|3J��Y��Xi���?�<m�������5s�C�hm���T��c����N]�x�'��n�{���\H��-	r�b��,�(�I�b�TJA�s��5~7~=3��p�:�5���=�Y^��;��h��]1�K>I.��K���g@�
'%�������`S�G������:T!�}c/I9�aZ%��Ata/.f ��r��	<rT�����gB��sE)�C�a�P�=��L"f0�HAM�KF3�(d	��U�R�ZSJ�[HD�L������],y��d.��o8J��N�0w�M��3]�17HH����k��*���FEhh����k����B�~�*G����Nk&H���"t@��pL�5'�����	:��&�}o!I�{�k�87�o9
��gl�3q�A���m�C�=S����K<�D��h�v?����gp,��CP�p�3���x�� �E��E���{rR.��G���v��3�Z>��;����N�E���@��q*���k�k�v����
c'3c86�z%��rN��S��5:��r0*x"��`��(��75�QZ������/�a>�]�������Op��D�x>�
�Q��D�rXmDQ����qL�r�������|��;B��6?�&�I��^lLL���'
���2�#� >cp������y=�%��u@c�iX�*b?�Q8��M��i��r"����u�*�T�������^�����;Qkv
�P�-4���wC��^ ��u@�I��8x��U��������0'���s�cG�pxMF��9����>	��mkf��r�Y�.� �|lvf���pQl��f�O�-�=��������4������������[�,%�}#�w�G-����q�'���N���J�Mt���EL��t{_�8&MrL&iI\<�����`(�co���l_����q]��c^�w�1;Q��S���/�7g/�z��?gr�7�7��D�V%8
������I��I���@&�Q�Z�:�Z�]#�_�!Ee�HxSf���
�a�B�����~�r�����?����5�5C��\�H��)?��L�k��>���'�m��}�����r�xst�:>�q��S�w=�N��4�(�a
*����������e6�����vG��lw0XX��l�!^����c��6�L�r}
{(��f5����F�6��;�rl���b��&3���a
C�3��j@�!r��Bd���z���g�C&?�Z~~S��}��oC��$�6S��=��u�w�����o���<�A������Cq�I�����]=�����h��G����F����n�w�[�X
��o�p����B�"����T��2:?�/����li%���J�K�Z��S�^No	kQ}%�������P���El�5bc����f�b�&����a>�P�V_7A���!�h�I�h<
[-~��:��V�It8��������@	�G�hJ�
�s0:�p�z �^w��)?��1�
��q���(�������fg����M?�T�������a��2[��
����z�����J�R9>:�S�V��>�T�T	jG���XeCs�>3�������2������{O??��^?=�6��V���Z��F����$:
;�(�WZ��i�q��rR�2��1�T���Y�W��<egF���c	�N������ ��x����*�G��W��}��f��8f?D-�o�Z}�8~Z���J�V)\�Z���S��gW/b�����/�C���
����x48p���������9��S�k��W:��N��E��������P(��S������2�B���zb����U����:)5�_�v%�@���5j�WL�[����5�!T�6j'�K��������3�G���?;8�������5��)������cE~�01�#D���o@R�)���-c�8+��x7�
(�:�W$���������5��D�f�D������V�v��\>O�v�a�l���+Q���	�L��jP����+���Z��� �<�F�n�8����ryqj�U�q���x����g/��P����l��q��������U���Q//��������o�Yw<�VY1�@����
��8�O�������.���x^���4���7��K�L+��^��<��b��]��
]��9y�z%�?���I���Q)�G?��%�g?\����pDq������}�����ze���}1��_�������T�C�����(���7?�_R��>���������/ ���4Q���%+N�YF������G5'�`4�!G�#0�|�?oxA����p��J��������j��������?���o>�����<�bh��b``�i�V(p���Mk~4
����yhx�4X���i�ES��r��g��������(
�S��3��D���������w������%���p|��O���4
����Vn�iw����
��"T����:�x�
�{�7����t~q��������W���W��C�RP�=��@�?h�R�wKU�)m���z9�Lq������)-���/_0!�7@�.�#����45{���������������^V���J/AM���5�`�x���	h<�c)I�M���o��m�v����
2�/���Q�QANG<��G���T����?:��k���4 f�����Y�������EQ��.��;N��{�<��E�m�B��N���Y�\�9T�����w�+���D}p�idm��X��cy��J�?�9����S�1�'��#�u���7&��A�]o��
S���T�q�	���o,�6�	�[�S���)�f%x���&7���oM����R���!o�>��A*2$�V8w���?�����������o�P���4���J�k�����%���N�[���69�,sE�%��3go� .���Nv���T�Z�_�B��a_��,��}�
&)���8MR��1M���*�=vq�-�{�C0����b}e��L���j�������h�{[��{��e�q��f�ws���t����r�5�<}:����������B�1����^�����{/(�*D��7��P�����u��6��:�����BvKY����a������T�S.W����!���}z��\_Og��TJ�������C$S��`�GWI�!%��2XS�Q�$_m��u�D5+S�<��u������B��k8�G��`H�,bk�����Kk�[��il���)�y��(��~�/i�H�(���:�_��j�&Z���j��rAz��Y�F�$=$�~J+S�^�u.��o%�^�(L;I��������it�u���;@���?,�Z��\�^�f��Q9c��9.I��MW�+� uC����
n�6�e^*E��-�]F��.������7���(�y9v�j��7��(�\F���Xn>oQh7E�iI��3E�?���:�-�>��++�;���)�� �dE��A.5�������6X�LxM�[����p1��%�������H7I��s�oR��BHp_#���?@�O��K��g#RZ��b��I���c4��,tR����`�����u]D1�W��8P�M�.��������
�<h��d��%]��=�V���8,)G����q<o4��vhR��A�� �M1|[8s6l+���������H��9��W�B-�L�PT������^����2���J�\���K��_��k)���n��'%�.��0��`z�AC�j���PC�n�����Q�Kieb]$�����]d�������T_�\�����>UZ8p�
B����X2���y�m4_�cL���h2��<���E�>�v��@����7�����&�
�r���N-Y�AuP�xb��Y���k�Qk�����������|�0C��S�����H��0��4\�!5�b��A5�@d���P-e/H[/�O��D���K�'0��=BS��;O^��:�S3�S����'��l���B���]k����jT���,E�.�j�~�z�T>�����n;���Q��]^�c����o��o��T�a�M������#�Z}�!CP.he]4��Bz�Y��Y4�4�����r1��Cz�k��m@�9��^�����|*-��2N���_�x#$�k�=/��,�O�'���:L�w���'c���������V$��$D
�/�� �@�dC���pm�'B&�e��=�M���}L�GS>y#o}uMw~�B��<q&�Kr&����yr�6��;��f��KI`���,qi���{��yr�SXFt������{7�N!�zs�Y@nB�8 �^��S�����o���p���n)������`�3O��Yz�Q��G��~�H�������$��Z�P)��G,���-&H+e�
�����!��e�RK�"*e?������i��G�PB�c��Pd�����RZp!K����T������zNti\%���q�w�>���@�Z�tT~�Zhp���A�>X����`��7���K���l_fp�T+3X\����
Ne�E���Bn�,75�V���	�D���f4��9��wQ<������B�OS��&7�V�]�N�t�`����atx�D����;�2����CT����������hMZ�;����CIy��:(�t���������4�Bq�%]�,O}�GOZg��U�%�)�"h]RorC����K�?�9�2v��?�7�4�5\x��bq���(V���z���8|IF����5
�����8<3�;k�xW��e#�{��/[���J7�H7���1l���h�;�e�����5�B�^ 6�(g�
�����7��]��]V�`�4�<<����e���b��|s[�Pf/i�^m�0:�2f_	�r���p���qa���J`���!�v�������6�c�4O�\\O��w�������Z*U$���q^�
���X'o���2�9=wp���<+�,�!�N4t�)VO�����������H�jE�
��I����UW���7�i�qe;�,��9��hM�����
���.�7e�5����7u	cnQ4<���taR���h���Xt�y�8.���K�2�8y$8��nr�������gm�����}�+��������j��RO�l��\v�=Ts������R���/�:B���,�x2\B�.!u"t��8����A^�L�$���-�
�hR{Yj���R'B�mm�W^�Z^�c�$��Y���Z��;���~�Q��5�����<�|\��!������qej�_-^���b_�~H	��m����Y	��������!���)��H����Z�������!L��������[S4s\��b���~R~-wW�u{�������HoK �_�"��.i�,Jq��c�:`^�?{s~���H�n%X���3'	��]����Og\��1���a����A�V��yT5%.�Pxpd]�T��6�r9ku�N8�	$vt��T���b���O"z��Lq
������K�;�|��"�|��#��@z � ��q���zJ��f�X�g`�Xk�I���y��ny�w�Z�����5'$4��X0���V�Nlx ?p��
�ZA�Z5�V��e�
oF�0�cO�cO��=�{���'�c���������mk^@���G�7C{������G%��������x�{�<���O���A�ER;�;�c:#�D\���h��=�S�9+Z ��9�F���G��#^��k$�g��)�\$k�5	�n�`�8�t����`q��1"y$���$���$�������h�H��;����h��_0A�������Z�Z�{�sT���y3,p/��Z��4e�^�������{YB�2?��n~���Pl.�L����
0Rh���^�U��x�}���!�}ERlvd]���e�#W�y��R�x��;�|�>k��$���0�)�VWK��>G$Q��"t_K�J��������CA�/?}��+I�{A��W�\)�������?�����e�����q��%���,�&�
B-���e�ru��ieS�|+ZLJ��q����o��n�Gq����������|9���Pz����Cz�/M�-7�C��!�28�4�%w��|�O�Kn�2u"��,p3���(�I����I����d��tekh�Y�:�E2=�	9$4��y�H�vs��	O:M�=��OI�Ng�lkB�iK��
�V'�v��	�/E�-��r�bx;$4��+P��&St����^����P*S�)*p�u\��^_��D�^��M����jE�.xC��e�R�GI^���);�M�MU[']}G&����5p_�n�d�Z@&��\��\U��Gfi~�#���
. ��f���B/'b�=/E8�a�s�������\��P����})i��y)�p
��C�h~������B��
������^~ v����;����M��
����F���\�����cjF��r3���T���N����<��Z�)H�,b�����>/���n�=�*JL���(
��&�W�/��"kH��������`��
O�����ds5��&�=m5��8�8��cOmyu��Xfb����	�V+�z��z`!�\B������A*��t�q\��fE��h c<���e��M��a���\�Y�7Wt��z����T\O�n���3��m�|8.x������6�����X����d�]��(2�4C-��!/���1�����,�N��q#\���*�Q��c��F����m!�6B|���YKx��@��LB��cj����]����^�8:���D��4J�����l,GM��Q�L��r
��)iS��.O����}Id��&�A�Q�	�~x�"^N�`_�����_}�'5?�kt���g�zf�����'?k�B7�F�����0o�G8^��E�]]/���>StF�5���9�Y.�!O��o~\L�yQ\,�����4���a�W��m�s]FmS��M0���6B��7?��^H=��#�}���>��k>�uk�7Bf�+_����B��F�c�N���Sl=�>|��D��l���*��B54B�H�:Y�r�Q��"���&B�@�e���&�����J�ni��C6��
��u��'b����`X����O!���}f���}0�S/�j�/��:����W� �R>*�;���m~������D����g~6}�u���;'E/��)�t|�MNz������e�4��"/5��!U��/ZA����Z�eP�n���x\
��	��t�����^�2]j�	���V�3k�$@��V�@-�je���)�Q+?5���.��f���N��������bV����M��������y�6^�1��m�IA�9��r#M�/���HO�?=o���Y��}���2������m��E8�E-H�	S��������pq���	����:���$���,<5�4S��nZ���>w����t���X�����r������+y/��}.79?���xS���e~\�����f�V�W:�������7U&����Z��Ryf^��{O�?x�>x�n�������x�<���giIZ�7���#L��R�K����l-���'�$,�)7J,5k�������jG&�(���-�O�������2�������2�fYp��<)��?)�w����.�/����K����j��V_:_��
iT���t2�N&���/���Y���p{�f=���$r��,�U}����\���_��J�B�?���I����{���������P2��E���1Nml��/_�������"���^��S��z�=���=��x�,��u�+=o�5���f��
���������&+�,��%�i@����j��5���0f~,����V~&*�]�3G~6(����7~FF���h���R9n�o���/�M����uT�u���D�"0�Yw4a�^<�8�)pGq����92&�i�e�P�o^_^�I��O
���F
������h�_5�]��*'!���4rJ�U�^C|��4�]�JNi�H�s,�O�x���2������$��[�L Xq����/X�[0o��-X�[0o�-X��sy���=$���,���7Kl����������Gg�<R���S&����������Lm{�-�����b����a�~�mm�3�J�]��lN�7`��A���rYlp������	d����TI��1��9f�2�L_
��mL��m$���)R������ H�#������vek7����|�	)4����v�����	�*�RDX"Q�^��!�s����r���M u2N�A��O��\A[�_ ��=��:^ �K�D|���sXj�sde����)��oV������3Z
�J�Dl��T�/�9r�n	�3��	rg1�g�&$���b-.�+!��5~��8��m}����g���nZ����r��@�)�Y�����:���c#�
��jj�U�m�AR�7��Ik;�_�6����)�Y-���������?L��m���
������Fj�
�m����w�cv��������q��w-O�S�>�_�6x�������Ij�'������)��ij����>��
?�T+imW��3����l�����/�U�mW������}��&�
�]�&Q?�R����'�O��E���vy��0'RiG�s�V��q�'y�,���38������7��������u�F������e?�M�W7a�D�3c���0s���/���q����{
>=�0CW�����P>�tW&S\��L
6������]�'��+�7-��Z�����~7�.�C�>E(nC��4���b�k���74&��H�O6`�@����}�����b�RI�����6e�4s���&���e����o��
�~kUEj_>�@o_�*��jjs��Jl�3����P�/C-{���^��=�����-�h;i���v�Y���X��w	�9�PD�`Q��a�w��������_1oK����Mm+�B��L�Y���W:�R����a��~����e8MY+b�x��E�#�Q�'��:�Y���F����0Q������[T���������S��j1:��5�w�\�>��U&�L����p�W���rV�VW���8"%
�$��p�Q�3��U/��F(IgQ���Z����RK��<��oe��\�O|��A�ft�9I���\����u��Y��o�����y�����sSz��f�:��&��o8b��3J}��|���Q���s
���ct�|�����5�#����U�Xx��)zw����`��������D��Ur%��A���i����h&f�;�X��Yg��:�7���~�{��i{l�
�(���P~^#t3�����Ek|:�������w��������[�R�:����K����*1����42I
1f���#/��lz8��&�hb��x����Wh7�E�w���D�����l�b=o�l�(���!h��Ofh�K�|�1_�������D�WD��.N���[dE�$.��34xe��E�r��KrIYG���� Za�������h����x�V�5�m-
���r�������Rk���A��1����}������6"�Zx�B������PJ�Y�'OE��l��G2.!����a4jJa���^�~�lR�wz,�16B�T���� Kr���H�,}�W+
��"g���=�<O�I�	�w<�D�����cP��G�f���:�~�Z9
z()
jG��Q�\>nG�Qt��G�z��`��
����_�a�R+�����S6����6��x�����~��1���3��n8�}�p�.���K�S�����~��6?���1�&������z��&r�Y��1��`e�a��]�������hs�d��{|�����N�M^��J'����vF���A�Wc���
��M�n����^<��6���G���O�i��M������a�7�OG!?.�e����1'���	����,�2�������`\���2��h8�=��7=�?n�'���l�"@�_���@of�.���[���u����Qp��V��-������}uR)����_u���	(���
�?�O���w}M��xN�9�2<����p)|#��,�������:��APX���a�
��
��N����_+������*L%����*�9e���;]YN��E������>� ���%���[�5���?��N/�
���]I���D�t��y��i�C:x0�p���~k=��_��MjpA_��1�v<���(w���k1�&�1���^���?k�T����/���e���v��$	���V>%N��Nq����������O,��M���t,�#�R	��T���BL/gk��i)���|�k��u��c����(2
(�G��C�$c�oF������T@Y{.����������F{A�LPL�!�����>��a�Z�E�t�S]`��G�������2b�QX��!�S���x	��UA?�T:�Q��6`�E�[����"&��A�1��&��p��zT�����t�Y}y6��TZl�
��B�S��7B��)�|�"N��I.����%�z�`$��,���������-���0i<�_�x��u'�� ZP����o��p�*��yG��Jc��s$�qxN�$��%r�H9��o�y���'�.�m������<�E��cR���E�U�91�����W��_C-9��9(��(��7��!��~^��Y0}�������������Kg��;,QR�Q�?�QO�-A���j�&K$����h3�#`gz��q��4�{�����.[�������(nGEz[�T��w9�4�4ayv���?Hnv��9�y<ar'n��|rsOP����IP�ApS��#EM�]�=Bz+�r��\�;��V��V��U�kY1"��}q�QI���n/���O��oK�%�^���5O
�:���E+���cJ�i71x������C�f�hx��l�.�[RSKz�Zq���������R�[%�'t�7#khK�������i[�7�����2����/�ZH���\r_XA#Qd�*�|.���5����?p���\K ��B��BlE�����'���qKI!I��K�6;�-�,�r.�iS���*�p�r��V�``�w��k���������g�K�GX�|X����5w/&�xj����������qz�?d���{v�B��T��B�]���t��t�	C)C�W1����g�r���,8�0��C^�;�K���bG2S<���
rS<d�����(?�]�6�������!�����_KX���E��}����i�Q(�I}����#�4� f��'��99��������8��<���*��C
�AX�����uztT���q�b~����zt��j�Z*G��J�O�������U�M��3�����?���l:�=��������8+W�j��V=:��;�;���F�Z��Z�����D���z���S�2��1�T���Y������L��}rT`�I��[����D�&�	��N��
^^q��)�������Y�7�����VO
vPi�*��Y��Q{���������X��I���1�}�lM�������>����s%J,
�����	�Mb�����������������s�)dE��~�-<R�/a�������
��l���=x7�f�C��td8�*'�#(�Q� �;��,�'�d�3I�xa�n��ALG�a/\�>Y���u�SP<�"����`6���:���a#�rT6F2�,�K�d��������(F���!�w�m��k�����e����C!����8"
�������/w��rm�a��
���u�s���HG;��\�ABY��hFv:�<���`g6��"]1����^�X<��P0�!�zD���1�\�B�t���e�������o
!)WI_�|&�I
~�2�~;�vd�	����EG��M��r�u;
�Y�
�P���N�	�mD��7�
5G�����o0���1U-`���������}�����S��#��#�b�M�_�Bo*�In_^|�i}D�(����HN���9��u<m�9|4-��K.��-�[�����2��I��"����F`��;�K���/���C��!CV�q�@C�����yz	��8�e�����}1�=�"��6�}�I;�r�_����t���;�������h<�
z�Mx���'0f>�[���cl/��l��?��#��m1V;a�Jo���:�g�7�����F���P��!F�����7@<vJ������	.G�`��j��q8����B2f�[�J� ����i����6�t��d���6t��R$�~Z�������B��^��Z`���������!m�a8��%���z�z������2��-�v��(�lQQ���=���Z{�Q/����:����T�O�^�w:��I����n�b�F<|�G�9o!�i_��j"n���3etE��1w1o��0*���l���U���"�9+f����r�n�t�M�%�����E��� /d���?N��f�,�����������`�M���*��1m��U��g���Tp��l�z�-]������X�ma���F��Q�Ng5
���(N�4�c1�Y�OU����&g�i�z���[���5��a��*A�y�8U����wp�	���=
��B�rF�~�'jO�*���1A�O�N%|��`	;	���T
�*��{����cv!���?���b�#�Oj�SvpR9*U�pG�o����0��"��c��9AZR
�x������c���=�Oa�t(^���{�ea^	����Q�GSv|'��I�L������9'�R����Z"������9�D1����c�����A����_lX��V���A��Q:�������3�����$��&C�����<��">�482j@�w��9��@�~Wb�K���+�*Cul�^oj��������^�#�t������&�����F�a;�����p��vO#����lww���Y����9*����r|�&���Z>NQ��w�[��b����/�.����{�$�����c���N����`��b&���~��(������u�����o�?�~���k�*zr���k8a�?�y���K��o�E���[�Z��2b�2����������^XXly����_Ao����Q*M��;�������h��lts����2Q�S9����l�D�E`�\8��t�a�%o���	Wz%�{��x�M)b��J)1������1�H�9a?�b1�Q?c�1� �P�{#
�7�L{���J�"�@�6J�3�
Z�8��D$E�%5��8����Q,Bd��P?�78Gh��������Et0��C:���7Qm��X���{u����"�����*�����T�@H���(����Si_��ud����24���J~F�WZ�S����� �	���B����;��'���a�\��_�T��E$�V7h�
�����'���~�o��h��a������\J`"��[��h�$k�D�\���;���k�-��^�����N�����q�r����lx���rC���
�^����x~q������
e�
����z������~
�E���=��Ie�-�p������#?���|���g����h`�U����q��x�a�y�G~}��������O�K)-��L�8���vv����9~���c+W��="�u���A�����~��A
Rj�{�>�����oa�9��j���9�����	p�s�L�z�g���B�h1��	E)��4P�y�Q��k\�1�D��"�$Z��X-R��m����r�F�~���P�UN2W��y���C�zT�N��tw��!H����{H2}z�'�<�G�h}F��~���/��c����d���?!D�����,hl&T�cK���g�����ZZ`���&iE��������!�[G��5��;G7T=�{D�:�Jp����K�:'�C6�(W��h}B[��{����:���v8��4`�����u���w���8G����������p�e�u���?��{�:_4dB������]D���)��i���W����;��={{>yO�-������K�S��Kp�xUm7�zP�I���(�N7$���t�KX0��L-��VD,FG�	���"����������V��������[$!�S����y����
��H�kJRB���f�B�7������D��&Z�hP�hEL���d��!cmz���a��Bz�l4zJ�~����5]!�
�9�.�����Q"�%�D�8��G����yT
��r2��+�v�����~'�J�wE��
��s)�&�=���h{�uX|3��;�I����=�d�����'�
jI�
���d,i��B���N5U-7K�Y�gK�lHoH�l��-���|k+�R�a��;�)�uYv�~	�Bk���@B����2������ei+`I'�3�w ���.Bej���|m���$7l��`*�|�����S�)zD'�^��}�T�A�{H+����b����d��,�c���BuuY���r��{�G1?�=�+�`m�����Q�_f������T�P��E������F��I��\F&���x"�E���3/H����ql�o�����CH�L���@\�������A{���{��"�>gzHLnJ��S���]X���~x)����y�����,`�Zf���4
;���LFh��Ail�jw���x�����������?S�}{��;X�Z�KmM��l\i�z��a�&��������V��W�]h��j��7���_��-����_���">��h<��#V�P��iJc�^Wb��!	3Z���U��E��P��7w�!eIR(���G�)���<���%f����2����E����?�����`�?x�������������5(���u�)s�h�<8��U�"DjJ����8���Ac-����B�KaH�]���IS��|��ha6�nH�h�]1
����e��)K6�o��:KV�����d��<�Y����&�Y� ������L�R��n��Np)v�@�a���q�b�!�B�=��J�����5����7������]s� �A�g�v���s�(_�3���65�����*�C6��zI�������$��(���R��U ����[	1��E�.�%J�p*��	
\�B�$��[��4��*&�>uV����'��ug}r��4�x�F���t6QoML	d��s���h_�n'&^I=�����(�c��`)]�	 �p��Fk�4����i�#�(��������(TO������~��II�<o9=���r8hP>O���oF�ng*
9�����d���T"I`f�n�������O)�eF���;�"�
��w��E���������3�p�R�
������l��ca���q��i�V�Y�Z	$�>-��R��-������2!%�������-���c:c��-a	��r�ca��wu�u{r(�O����5+�<86Jm����fS��X�v�IK�����a=�������K���
�2.6��(�
��
��3]f>����2=����8�[����PUj\�8�.���V*4���������n��b���&YN������\���/��?��m�Hk�"�`V�DH!�m3���CRX����C ���&�Yg��Q�P��.04H]2%���l���������(@h'D@�-����j�� j��,��E�1��J-Y:c�/�
B�6�D�z�Yl�%?�N8�)���XK6�X������*`��Oa�U#PH|��A6�}���"���]�=�+K���M#����'Br����,.����p-�3�l��0|k�Tk�R~�[��������]r������0�[���KR~���Z� ���������������]���/��z��������$W��ENK����J9+�+���.-��<���^d�0���W�_��|�������1����H+=��+�a��oa���.���f�4+e&�A
�jV�_�����M[S� Q8X.�D�R��;�<�O#t��dY}:���bN1����[30�����]^Y��L����16pV�T~�g��[]�n!��F!��.ai��J��������,��<,�J�3�I���7����o�N�_���>��)��d?�=XY*{�U=XQA6H�,r�IF	3e��F���/4M0��0���)V80����H�!=H?��P���u��-�Q�B;���x�K;\>PP�3t����Xg)����>���@H�D�Augn�d�a�1b�!���O)_s��G,��.N�@�tq�c��~&GL_����3�r�@�RS�K�/��������s$M�UPJ��=(A7+f�����	:6�]%��#��]2H���biyA1a��%�c�L��uR"*v
I��~c��Iz��hzM�R���%��������A���e��|������b���������i����{��0�>~S��H�I,�TY6 �B��"����p�?|FU���@��a��i�tb$��^���"�milZ�amDXCf?���r��~,+w%��uf����t�!h����+�Y� AO�������������<�����Dp[j��$����:(����|��Eb�y#�i��{,%��Q��=f����7�(@7/#~$p"��VR��A�SpB���)��Ru���q}1�_�v�U�9����.����K'K�
����D���.@�����jS���h$���_���0�$Ze�3`�b������`<e��c�p{
�r�D�O����4l��me��J�Rg���HK@��Q0����	�dF�AU��
�����x���w��aUiJ`�
h��o@�)���("_%VB<w���@���
�w�Pv�kXe���x�lkD�������W��q���oYf������`����[������J�v��r�^���O�������^���*'�v���D��Nn�[�)\4�x2MK��	##VO�|!�Pb��V��}�(oc�y��?�x�t��w�����0j���U8���BZ����8��l�DBH�������X(Ky=*�`����K��|`|J8�~����1�PS�b�5��sZ�3CKm������A����	�1$�@�Ga+��f'�M�1���Q+�Na�'��f�� ..��R��,�n?��[X`��*��P���B���58�s����B�FO��)i�%�A�
�4j)5Q��LM)O�i�H<��x"��hj�{dc$�F����m|�7`�����_�J�xd�qe�]�Y���8sQq3�a�=��2z��Q�W�j>���d�E���DDRcf�8wd�!P��d���h���V��C}�����?���n�~
���F8T��\��|:�;^g(���<����'-K�5Cy��k��2��_�	���]�Y�X��A
<�����������V2��jvUsP)�1>�����2����h�Jb�>���xS������Z�}c%��G�_�i�fo�4�^ql�4W� i�Z��jt9�m��������%�U�^�S$f3�B<�x�����c6��eG�F ���v��0���������D�r��M��Rt���e���*7��
����$w�,�r�a$��� >��I>F���/cQQ|�������=��u��}EX���^��N� �z�){M�H����|;{�p��)!�_.��Qho����I��7F*l���E��zt�����@�_i0I�/~Z}zFb}��n@~p�����9��,�y�@�[�Nq�K���/�R������G"B[�`��B���_�<���{:���b"dM����d�/%�Z�D'��T�l��;C�%vZ
��������b�(	2���b���56�� &����E	��
An��GV��8"s�����1zqS�-����������E�V�R�,�y[�!_��-���
Z�:�����G��@Ilc]+��9f�����$������3IyYf?�>C�%�q�zO�����=�
�%����G���{�K�.]<T9Wnr]�=A��h��������@�/��NG�<������4�2���#`�f��'j��g�g���1"k�%{�!��E�N������EK�8NNYi��%��Zk5���ro<#�'�5A�����H��7���%�����1����H��~�SMk8s�����J�{!�b��\
��kv�,^��	~iOPv��H'�X�\�.&���F��
*�r��8hw�y��NCY�H�(.�Q@�|c�.R�I����H7%P4�MIS,�)7h�?�2B��-�����Q�E���g
�F��a����;�L;&Y
��{�<�j�*�����Rp�f���.��4:�l��y��V�����E4p�r���(n	�=����2���	?*h�8�(����du)b��5�H�e!�U:Ei��r>���*<h��T�Q�t#����?��:"��cL>�eh(�4����D�$��o�k�zC)�K7����2M��E����X�ER<X/E_�~��������"�v�e&����q��:�-�;cJ���P�a�T��7��$`�V��c�PD��~������X�K�n���s8��:�)MC3A���+n�����g�����q��	b��l�4.���!y2k�bvX���FZ	�D�7����E�O���p ��7�D>K,2M���l���2X��h�'l0�/��6��U��Rpc������B� ��<A��������
����l��t}����Uz$?Z��_j�����R�,������)�a��^95>��]p"����
/���i�?��ka�J�1����ySW���v|�5����<����)]�E���M��m YI	Yb�=��]��LVA�s�jb�"KQ���"$toV������=>�T�(��n��=n5��nV���*[���Tg�_���T?`DqM������t��m����� z<�(���r
"����E�,���;�������:�P�+��N�`�^�GS]$��`S���v qq�2�H���NG�G��qf�����Q�O�A(�'[s+oI��a�
���r�G'>k�<mi��S�n�c���K
D���7�/���ZK�o���[�/�]+���Z���~�x��� ���X<`L�� ��V\�_1�������y>����Fc�3��a�$(GO�^�U�����?���Q8x��vu���s�������&�xyq~vun���eg����������|~)fyT��s
���������{��?�z��:j��h�������y�O
)�*~5���-�#~��`��x�g��������<�
|�
o��xk�����!.������p<�G�����G�����W�|��|o9�W��V����V������sl�x[���+��h4��$������>���W�����!����a�����q�3F�o��e���������5��Wa��qW�D���_m��i[���B?��a���D��yI>O��8
%r- �z"�h<k�:�H��+���lN��Ha�6���I�r�����C��@�?p,��������k������V�?n��]��~`..���E���
�15#{T����H|i��{.��k������w���.�A��H�9���E*�#4����
�
�@W���S����9���]��@j��(G�t��L4���@�m@x��-0����,�c�V��h���Xo�r�X���jG��1����(c������q8�O�[	�@��aT�4r���s@C���X�����/�}�����YEp�9L�o����9fX�G�Drv���w�����Lp���gO{A����������������{$�9�'b����/�nJ;����N�:4�����$�T���Fe����h"S�C��=��F�Ax��H,����>$
F�R>d����>�����.?����gJ;I�����O"��~&����A������m�'q�J��_GG�pt
z/	~e��)���7(�K����}�Qyw���;�����=�#���o{��~�q{��os������H������2���}��:�����9�y�D��p��[���`H��,�Pa��q#����.������,�O�H����O���wf��mK�������=���6q}m$�iC���El�/���A��vG&�l�y��L"��Z��L!Z_	����L��1^j���Jg�#KhHt��5���vd
�����[�
U��8��b������F����v���T�8��}{~�����k#D�D�eD[#��6�s������xz[�$F�$W��;[�]5p���P-�O��B�O������1�5��,���q������S�?��P��"PR�q���P�GGu��7*�7�Y;
��T�UkA��~\����?��60�����?���l:�S'1��=���H���q�q��V�q�
��TZ�����vz|�:��N�5��s>���U�Y������h�);0b��!�wf���+��h2�p������E�������g�*�l<�_��j�ip��Za�F�R������O�?>�]���UN�TO~a�z�ql-^��`���>�
+>�SDx��&Q{6�{��[����Q.^��Qg������T�i=0���X�H����0NgTp�l������M~Hc�7��n��9��!���/Mq]D����n�7p��;W��8~�}���J��=M�
?E����3�h�O
�I8��Z��f��!d�$��Gm����E\y���F���Q�!��V#��������_f�7�?5vp�Y��
F5Nx���n���?�4�kA�t��Ik���AT��=5_i�l�|��G�|��������)��:��_(��|I�X'�G��p��;����xf����=	�N��)�[�j��rGQ3Z�E�(�ZPC�E�F��[���������p�3[D�H�da���Y�=9��'����k'�W:�6xJ�+D	iE,n��9����z�F4�gFN��.�������o���0��}4\� ��!D��|P�nXkw�N�o+A���*���j���|{N���N��=��e�@:v��x6�&S�%�4�"i9:�)�&����!�'j@��w��`��8�y@�^K�������
l��X�k��
�L�p����<H�^���h}�=�t����i�^��r���T&^��I��8AQ6|7����0��D�3��2�H���"���������H�F�-Y	�F��X��$
��i,�
l�d &Q���F�)V�d]pVn%��V�yW����z����(��s�;��7�f���o��T�,D����/��/{���o���0��=YQ�O��qS�������+'���a���Q��H�����=���u�	2
*FG�B�M3��\d�V���G4n	!���D��w�u+��9O��ohuX���r����8S�NQ��`b�#P���9Ey��Nj!�����m�xUY���]y�\�����m����lOz����0��.1?t"�:\v�M�c���ka���G���5��l���L����G�MB#kP
+�32�=?L"��0Mn1\�l@�h�Qz�IK�RW$�x��I�xU��s�����f3�)u$���w?,n8����d�UP�"cN�������W�>��X��7��@�0���<qE��� ���}I�d��F�m����#��W�aX.��+���rZ�B\�����?>����o�LD�k$�^n>�}<9�fm��)QP4�H	s(��!5�����v��/��F
���t=�l����G���WI�<�5���M��.��(.� ����V'j�&���|�1��%A��l�Q\s�!� R4���l$m���iu�d�4����X�O����'���o��_|!�t8�}�r�2�85�C��r��i;l�={��dk��*�'T�S%���hTsOf���O�0��;�7<������e���b5�
;_�h�����/����O<<m�P���Z
D�NNj]N��C�U;
�}�
$�E�<9��<;��N�.��F�$���l":��)�b	~��]�9#�R�u�.y����L�M��;"�/��~q��|��FA��������s���C�'�a<W�#>���z�����j����g��E���=>=:�Z��q���U:��N�S������I��:�lR�S�������'xZ������������i-4#&h�nK����!��b^u�"�R�+�m�������q�
��9���@���J��a��8)�-��������&f��2�`l>qA+8oqi1������
�,�I�����n��21�
D�2N�*KP,�q����w�|���xL�BS�����:�D���A�)N����d�S����!NZ�A����n�B!.R��9o��A�����:=>l�d��
?FI��
�X�+N(d�$:�I�!�k�`�0�RXp������`b��(���SLUD�=��!YD?�������v����4��LeQHq�D���2�]��P��?��q��3y���y%���Y�G����\�P$�za2 D�jO8
�3����
����B���I4x����|���D��H'����x�6�J���^�H�*?�>�r����L��9���e����c=&At@?hc��!U���#y����2:09K]@�����l��`!��Q�	���G��{4���$?�hwA�K�ce/�}s��D>�G 9�Xyg���%@Tjc*���c�D2_$�K�k��J]�]Q�H�e�R����Y������T����P{V �'>�/CxLl/0�����A��5�JcpnT���*���;�U�P����p
Ji��q;���r����r��M4��q����|7���d�`��7�Q<��o�Az_�P��������v?���~
���	��D�r������8�x�G/�	0X��Q�"�W���)NQ�
��huQ.����1��;<^_���Q��'7!yI�GE�"v�_�����$�W�����Z�(QK&l�P�	BQ����O�����8!���Ct������]��?Q��V%1�Vg��N~�~C��x�o���9���$n�o(�)���3#�����Zv*����k�j�n���<��I��G�I��A��H���U@�f����S/�I��l�bA#�c��p=����ya�2�G�R���b�������} z�^_�^V���*��N������:���� ���d�r�
��9�'x���6U�r��1bdR^�i�%�b����++N(:�V
M3�R'�H	}��G;�J�H
�����D�=��=��5�y8�b�H��8;�i���FP�U��G�nh�����q\	�����#������)xI8��`�o �d�X0���J^H���Jf��������M�zQ-�[��}���1l�H�lD�(vF|5{p��]�6���2��e�����HTQ�x��7E���������J��9���k�1M�������~D�f�Y�������f��c�0�)2�}i����#�a1�������"��L�b�����'������WMrbD��Z9W�*����B�]��S�(�t�<N!��y����8�9C�)�����L1#m'�u�l�:���f7��f��L��x��x[�L���	��5�d�u��>+�g���><�zD� Y&12�6A�T�����*�����3~YRk�j�A��*����H���Qu�wX?��G�0u�}��^�a�(�Z7��H[�f�S��OD��/��l��&�����O��nV'�d�L4���A���PD�������T��2;�`�dB������&�N��L9S����7���
5m`;`U71�L*n��o!4t����";����C�1�?_�k��3$�B�_bf�,��2
~eqF5H���i)�/tH�}�7x,��,�n�	�5��/
lb��%�Nz�z�CO �M!lP���t��))4����j�[��\��a�N���x[S�&��-�# �$�i����16#1�$;@��X:����E�$t��T7���-u\	L���3h�Q����g7�H��`���D�Z�@��h�l�L��R���*�Q_4��k��T3�j@�I"ZdAe#W�M1{���q���f������=�-��ts�Xr����nqi�$n�|~&���& �(�\Y�Hv, M)�����,��W�Gu��+����0,f�Hz:j�f|G��K�|H��t��;���4	�u��j9��������V�vr���DZ�a�(i<�����2��������Bs��Y��Db�$V�,
���p���7���Y�IM>~k�;�R�8��p4/DK�R�A���5BBG� ����C��6�7A�����i\)
����
�wi�*���p&��>���b�[���jP"��I����v'��3K�R�]` f�SJ�y~R�a0�����G^� B�<r�k *����m8�c���o[�"��$|�Z����VuN�p7j����D�e��Qi\k7�� ?&�9�T92w/�>�����4�Q�s�'����U��������~3X�pYt^����"k�����)��X
(	)(�'?)1{��s���Q�2��p�����S=��1��
��`Q&�q��tM��:�K&��Az&�88|q��%�wE�<�TT�1��G�/j�����DD���j
9/3C���E����)�&;�4D&�L#4"r����{����WX�N��������"g�@����`2E����C'������3������nIp��p��[�0Sb��j����O �"���)��)U�(	&"
��
���9Ib�*:���*�
{�Xr�	�!�
�?L��Z�������i�����Yyf���\��YR��T3>���!�w� 5@-S�v�&����n�!sp��]�*����I4��6���ZG�jm;�T�`lypzbI�)S���7Ys����r�Xf���_�w����eo8�M�z�'�� ���y�?~RqD���������=��t����r0��x9C������+g�����`#��A
iL���b�|S����HT�0q�2��d��0��_�R�����2�u�AwO��}�g��-s7��I?!)O�-L	V<9nA�B�����M��� �!E_�|����
k
�PyO�Er��]�X��pBm	zEp��J���)o%�F�8���l����/�p..�����d)��$��Q�?g���|!�m���]�T�����H�&B����@~�����P�4��/�v(Y�����������p���s�;u������:����.�>Z�������?U�M�?���MlGY"��T�������Q��W�����x��a�
�JF��+�R����K�������Je���u�:gca���8s)@i�N*���W��!��9����s����&�6W�?����,>��5��'r���k�y�/D:������B���y���F7Ot�4Rj��s�rp$���*���^����|~�w�x��w�!:�� �����S�(�c$�|���v"��].�iM����c}ES_W������R&Ns��(A�D�v�I��	��(�B���+����'BQ���?@���!�qy�!��?�����-X>`���9=�a�F����A�B�����������H�Iy���)���3CsIF���<6P��Q�a*��AB���	Sr���o�y���F�Ol�������3;��2k*���,�p���+H�j����tq9��U%��n=�*&�'9r���KhB"��E��$�^�D ]
�I�(�w��%?�/B�bC��f��c8:$��9	���������C�r���F����2��SS���U(O�&_����iJ2x�3��,��0@I��'R��������n����F�3�9�,9Z��7������~�C4���~��
��@��}���	o��I�&��Mp�����K�=89	��K��J���b��RR���^=$=��
 �HJ�'��0�`���C�������
5�R<X���Za�(
9����)v�T[����8����i b���,��L�j'
9�1���*g��D��A7��^��w# *����w�+KV3����=�`�n��x���j�����"!��c�k8z�EE�M*�6�����"��@��[HW�S�d_%�+P	�����D����q�2i����i����SD���l:����$��S�2�;z����KL��[A�Xf3Z�.LF�$\ty���B�./P�L��F?�R'Gx�I��{H"U�A8�d���O���J�!�����R�A��&�������^y
����H~w4����D�z�%2E`
oc���7DN2vWv�t�_�'��������I�)��z��}O���-��M��%�t���*��nU�]<}��@O��O!]P��%D��"��c�q�O�h��&�U`��HMt&����tQ�[J����>�S��=6�DH4_8 }�hz�7����@^��G�i[�$�*��F�-�Z�j�sZ�'8�?��Y�&������������Q��G-��������F��r4�Z��6K�y��#{j�h�Zb0����W���3��l�Ak9�
�5��[���/_�<zp�����o�UP��Y��j�g�"��V8���E����r��3<�d�0F���5>�E��9�A���-z��%Hu/<"E}���Z�'�f��\3������!o��oxU���@/��-Q��b������h�������q��5�Y��Z�v������_�~y��Q1���*$@���]A�#v��l�k�N�
!m3�;�w����9��"�F9�t��c������97��H�e�#��N�J]�+��J���������
�p���
�p�<[�^�\}������?�����p8B �p�9�C��}�O��A"BXZ0���_���Q���U��k������_�Q��4���'��S?��:��Q�r��7�aP?9�5���������O�����0��BQ+]	#u3�<6����p8�
�.�R��BZ�j�t�(x��4�2c�/��c��4��RX����LF��������`T�`��,�2c��b���$jG�Z���'�n#w|\������(R�'G�R
|)����ab�A.�%�E���jyD�������!�}�j���[J�����\��1��E��:4��WQ�����M��������������?�����MU��Dl�����F�h��B��6�H��9z�����,��/�������C1)�6Z9|�'��(�v(
��8I^�?�1����\kt�C	���`���)8n�"���Nx���f�"8�b�����=�%fb�+��&�fX����a�CU�sJ$/���)�_��D�\�i�8H1�.2=,3P"6�=6�a�2X}\���%i���&�0L�A��C�����=(,���(�����!����q3���t/��iz3/�>��Xy��{����e8@;A��N�����	�w�4�tD��d�c#��-`�������/0c� �76?t��0lOFq|(��G�����T��#����������I��w���F�U9>9:��T�����n�]?��_7��E�E����v�B�UkH��(J����|t�/�OP�3c��)� �5��"���x�u��w�v��Q"�_"�����,I���d�\�� ������J�\�t8�P��j�mi>���b���Q�z����k�����le�7%$�D �a��)"iu�
��W�������7���;�`rt�I*�A5G3>^�~3m���pyD�	T��b$��'�����l�u�X����O+�M��:��$�~�+�	P+����K��'��j���)����L()�����#��8�����gf
H6@��tU��R����eJ:D�p�dA����tH�*��[��Z�:hQ���AZ'��H��%(��4���3��G%N���qb����){�3�������������7�)�S_���j��<'���x����i	�1�����H�/�i�_C�����k�����q�{����Jg�����M$RJ�`��������dZj����`�X��*9�f������w����'�B��[���H�]���18P@�m�I�����</��Pf�V���k�W��WzhT��F��Q��.���S"���j���=+V}�m��c*)�n���1a����
�T;����Nz@X��B�%�>�hM���/e�@Mk?����A���jk�{�y.�j��.}�2O�X����p���	sc �M����Q�Oi��BO)�z�,�L�O4u���r��:W�U"����u��3�����������D�,Uf�����q-�l��50�Ktqv�3�;c�fY8xR�<�5��8h+e�?������������~�9k��#:�-T!R�F��j�9[UkZv��J��"5jJ8e�.b^�jb���H��&#�!F�]���$������S�*�����������+.�\I�M���y��2��\D���"G��2�*���^[����(|�T�N	�BB/��x�u�_c-	F~�!�/�O-��s�����!�MmuQ������bb��Z��p�������n�\��2OF��T�*�a�pk��7��>w��:���<�:���^�����!_�b���!�_~�M�7W#8*��H�o����N��"*�j��C�"�>E>����_�lQR��	�h0cX�}��b?��tz3��@a4�������s����U	e2:X]�p�!�p��X�V��^�S6���������s����|�J����|����C�U���X�����4�G��k�4�?o�C�?���6N������Rk��DG�j�8:��t�a�V+����a�+��!j	����A����W������r���l���O)�;���r�/
i��9��D�L�l�����~Pp.���8�B�Qo[h���92l�:	�B��h������S�Had��.�?�����l�����Gj%d�G� �|����I95)������ID�D��H�%#�Rx$#��>��������T��D�zV�?�4[(�a�4N�K��Zx�c��,�'����mk s���\��Bsi��a��-��y�@��\��4.�p�Q!(��l��p�z�\����1�/&R@�W�
���fD�
Y��	����wi5���m��V�����1���q��&�=����S8!C����wSP&p��lTi}�>���H�"P	���B�eavjb^�	|�*�0;bI��
@�e�%������1���y�G������!�e����
Fu���/�L��0#Y1��Ej
>f]Z�7��\����A=pm+O�K$6v�15\�]N�Vv+'�~O���qT����R�����)�1��)�/�y"�tD��J)�2*�//GrC��u|V��~�#m����'�D`A�eU~�����bZ&"�r�6����TA�'��3H�v�h���2����� �=�����"�`m��N��t�.���oF=(��P��������x��jK!��3r�R|-�-���`���p=*�O�����f���O���'e�8*�����9�a$�/� �4"�����fI�=�u�*G�����q��������G�U�������T.J ���^�iF�����8�4��=�+�M�/���~:�� �ET��n
�����g)�����Wf�4�����<K����t	�Q$M$.�4�g2zP�����EZ,H��$.�Hc�'"��(���:�o���n1{��g�f4��k���z�������&��V2��
���d���M#���x8�>����f������rY���B6kL/X�AO�{vA��|2W��SO�5�34?�O2��7�n��u\��#%��N����>hP�����N�e������y��Y�����N�n8�$�l���iG����b*�pPtPlU�z��47������w�n�������lBI$M6��8�8����YY���{��4����ds��}3��?���6e���+&���PU(
��S�4���R�*l�6���&-���e@�p�\�>�u��B��[�k%���>~J�r�p�
�9(��exN2,�P)�0��2)�����"�d�W]�+�G��8��E�s�[��&pVB+��m2y[�>J����)�K���� ��YR��\$�����^��n�Y3G�T���pL���O�A��k�GN�\���(dw��0&��Z7q4������@B+�Q�2��7�>����P�*���9�
 :�����:Vi�)���`���
8����e��GRIu����B*&;h�h�l�6[��N$VA��%HT�jP�y�:+�?Y�o)�0�3�n8e��x�J�.�������%$`����$-������&o������'�$L����t����wYI��Et;�L���S�MlPY����M��h��0�=��W+�>��|-v�������Pe�q�����$N� ���� �|�m�@~�$���+�c0��.�K�[4%���W;5�S����^�F="�(P��5�U�����\p,�GGWYw).G�F�/K�_���������Z%�n���&��V�X��cp��(�(�#�,���N��Y��Nq��Jd���1>�b������C�$Y�KH�	�"b���Y"�:���
����&�Y.8E���|-������_��~|���|x�B�1zu���-h��E<�:<��3�-�>�R�z�C������'I���m�&$eVL�4�- I��2A�M��B��H����1��0�poL��#z���!��C����Hzy&G�	'�J��9�����1K����yN$}���>>�+���fEt3�������A�����S������5�&��/g��|(�*l�.���I������X��*��Tt�"�**Fq���Q���Os�fv�e�1�?����3|���������g89���9�C���Q=f2	a%�����v���F��{-7�������-�,�bF�Y�UB�|��UoW�h�!��KN/��n*ycl���-�bfQIZ�g��F�	��z�F�W,	,I��V����>u�����<r���J��]HC�ze�Y��T��S���U�|.��5i4�O�^w������T�!f�X��V�Y�4AL6����Yr��NR�>[sT�����O����0��C'G��%��$�Z�Lcr��6�le�<����Z�oE���_�X�n\A�����!��\����j
���HJ
���:E���������C+���U	���A���R-�������k>���n��C��}�w�^�J�m8��
K�����v�������+�����v6G�Ny��)�IF1k�.8Y�SPX�B�H��-;���sB_��H����;������4J���N
���g� ��6��"83 ���=����4��u���?�;��L��ZTl��Y�"88�"��������9s��rJz&�*���r	�����T�K^Y��^�O�D�U�c��^	A:������J���?�x������J��T�t9�D8V������+������\��Nng���q�e���������)�	�\Y�]r�w'tggv��pY�w&��>~rWV~���u�og*�%q�����c�\g���2��W;�� ��A��h2Ng}�\�=��dEF����[����E3���	�y��?��,MN��f�N|��q#FK�Ea��K"^G�������O�l�~��h'��X;MLG���)�Z�&���e#4���4�D�l�!BU_�S��F��$�h>Fh�F(��8C�=�u�f9��/�����<��`���r�[S�U��*����Z�A=>0�Ke�3J8_aY�XA�
���L�k�|�}v��B���BX��K���-���"&��c`�P@_^�	,�B����Lit�J[.e*��C���rv����0������������c7��C�%���s��"sJ�N1���"�O�N��Zpf}��� ����>8E/k����+/@��e�XK���T;f�����=��*���"�!q�5|+�]����E7��0������y�9��8�X��X�����[�������4��a����(�7In��\�c�����?sg���&���qL�,��)-������O�]�#�(W�@-�1�Z�h��d���i�M%���<Q����m�^^XL� �y^����9����;LUtqK�0���~pe�wvOMC
N#������h�y���X2�H�!L�S�<���0��n#��>~���
���U�F�
D-��������0Tf��B2�[������RbRU���{��Q��_�u�[��@I���&����a�4�
[��
~�y�/PRg������#pA{����M��c�Am�
w4��I��CG~I�v�
(���\>����I��Q��%�yS_3o�JE��|��x���������O�R����BZi��e���3��thN-KL���j�����P��`��F4��~P�|����KD��bx���)���s+�������s��I����>H������N�**����1}C@�a�m�|z����;R����e�H��uV��|k�E-��~82,�v��O�U��5�&^Fswd�=����6O��M�
��<��6_2��B�|��M���v��9�|������)�Dt�;������c%g����
^*QG#�+�'Y��M�������I��yM[�E6!���X�
r�%lI�d�%�����_\��h�&pF��T���(,`^]p�S�d�*"moI@6�Frp F ���Ai���yZ���ti�]L�9������:�&�Q/������������F'�����	�E��n����*���"��Wv,*ym�jx���;�\�Y���G��i�V��
��+�^{M��D~�����NI���3��$R_�SX������>3�e�i�p}��
�F�Up����5H�(��.(n�<���"��6SDx:0�Al����q�"�ZA�p�n��������0��|�P��J�y
_%�1{<��B�^�"�.���U����@��y��l�]�nO������52�KX6�2���`@'���f}�Ap���;b����L�����`���M&��8't{]^<8|�x�i��lU��b�V��d���B��h;��09��*�T�?k�M�no���LF'��O'��Lq�3q�r#��f@���g���N�1�����]@G[�Q��C|6p�;U^t����|��!N���b'�:"��z�Q�y�����4��hg�*��v�w6i��a{t6g��R F�,-F:��~������K��#N(��@���
������{X>����T��ET@��p�
�1�J��`����~8��+�
up������*jQMz�Q����A��D��"��1���^_e��9J_�o�Bw6T��a�����Q�R��W�U�.bWS�g���')Q�Z�k�ip���@l���u���iX*��i�/T]���S�e�_y�?��������z�|�l�J�$��&�bO51������giv������}x��m����_������g��������x2��t�O�N����$��'�n�s���� �o"���O��'�nQ�
�j�h�E��'RPk�@Y�=b&�H�i���F�l5��Z�L[����v8�%�s"���!^rZZ,���2�J�I$��>� )r�I���X5�#�O�B6��0��Q�U�E���c(�:���_:��Nl�N���{��;��)�O�?�����G�����@���h�[Z�0�/1���z����B2k��f8��?����Mt^��Bt�wn���������_\��|����������������4���r:/��Xo���f��]���D�����O���'�%���
�NS�� ��<^^m��~���P<�WF�>���?��|2b��rg�����P#�Kd�
����{����}s��o�)���PL���?�K�JAv� �Pc����8�x�
��,���bq�,����^;�Vq���[ ��S��.���kn���N�jM�u�s�O�kn�	��F���R#`�b�N(�h���FqTaj)<�����T�l��-W�\��,+�/��Z�����j=3��W�kM��di��	;a������������c�4����2�{�/U
*V4���Pw�M��t!o������M�S_y�+e:^���2�5X�s/�o���[��b��j�{v��Vj+��m�Q^��luK�~]��O`x9�N�)����|�����v���d��S��C�PL�OO:�Vx����9A��`��St)R���~
���k|)/�d��pu�����G�\�X���$�t"��!x�&M`��d��!��#|�;��!���+S2��]p�����,������o��_����'����7��x��OY�X���-2-:�{��_�N��	�	>�������0a��������4�F���x<��O:��ng|2���������i[	`zO:�O��O�������E���,�Dk|�n�Z��3�x����C�ix-��n7��nx�{P�26�x_!K|����m�����m����r3�6lH������`�����y�h���<h���V�����zG����Y��@�0oB�!T�n5`����@�Kwd���H������������v�{F{��I��1^I���;5��v��Q�]�aW;��K�~����Q���eS1,#����`���3*,i%B��t>)xqq����H����p�#T�7�����0&Z�|�_����?��#��s�W����
��w�>�������wo�z��\��BHa5H�`��m���_�����
��m[��z��A����w��2���.�
�n1�n5,���h�@�/n�
����t��� �B@x���?����#�����/.�gf��p�
6�����{���ohn�Na�Rp�z����Ax�������������!�w)��5�_
��K!�_
��������q)��K�b�R��B.v/�\�.��.K!,\
E�J�Bh-Y��.�nf)��`��D��4�w3F�2
����M���6�Q��'�Wo�/���|�|y���u�iE
�w����;����tj��0�����s�<����?��z��.~8����P:�#t�c����
�5���Nh�=���G����	u�5�SnV7��O bS��L����d9y�	��a� �y�-x��:@�7|g�F����%���S�� x9����Jtx�Q;2�a�H3X�����1,CI���	��=-l�H$�����)��^���^v�����Lw�Lw�u��2�����L���Lwe���k2���g��}�J�/�j#:�Jn���!lT����F����9��V��7�������7��U���7=�N��S���n�![����{T����e�F1�����o����w���O�}��|����N�����q���\�����h|��	�J~����������n�'�������R�M�9�����������O7���0tw���;��b��z
��>���0��C{���m�m����=�������S^=-�����a.�b���[���Pz�������wBG���l�%k\����;�@H%>I���7��%r�)x\�Y�c��
����E�h������i|O�b�+�[,�d?��E�'�;�l���b���/����K9�v����]z�/��Pa��>_�	�%��X,K>�B���bWa��`��c�T
2���FIm���\�bh'�
#(�I%�.���n������7�?������~���M���������[u����3�J@�;�9u)s�L$��`1�'�%����V�j��'�P��^2���(9��e%��9�}E*-� N��PI���b�!�j}�M��676�;�3�����?��`�5�5��������f�������~q�������
s�!w_u����[Quo��������n�v+P���*���r.F�]l+u������{�X���A�u�,�C}}>Y�3p�sh��\*^�L_?�<�x����M�$�?��8���H�]����8J7D�
����q�i8t�]�f9�w5�L�x�<�T�fN(i��~��OO����J�w�R��xZ����r�-�z|�$��5�KT�����/���?��B�5�=��R��Q�Piwkvj
�PX���������=����>��S��~w�e�
�o�N����ZJ�����A��M��M�]�'(����~w��PQ��!��.�������PFe
1�3�~N���2.�t72R������B|/����J�n%��tU���t[yd�}�(`�9��f�[�����7����zo�9�>H��9���	���c��o`(�/��>��������r��Y�P���9E�'L��ydN��I����F�=�������|�6��������3����]���G�������eq�\�W�����;�u�.0R�p����&q@!�?��	�]�V��iAT�1_O��'V?���zE{�+��y�*�,,lT�u�m'�&�
>(�xZ��O������a�x������e���-b��M#L�To�������s.g
����q��Vk|r2���d#��Sp�9[��Et)�;}���$XmG���#���Q:^�V�0���8���%�����v��}T3���d�i<�����Y|��d���a����r�!�
$bI0��:���6������JVi0N�d�aL�/w��d6���$�D�����&�m1�j:zP4\�s�q�&���<~M1��3���hBYC�l�=�$�c����y���{A�v��=���n��(���O��bE�&I�����r	���Q�Uj/7����L������+�����.�"*�� <�N���i���;V��`
��h�v(�<|��
�	�����%L�zvu��CL
���TX2Y;��;�E��v���������q��
���5��#\�C����
�	����!d�C�	��O���6�Os����M�^���F�s������m�����������}������	��4o�us���e�n�I��	���r���7C��{��?��wz�:����?_�C�'�Qv�n���:�~?jw:���
NO'�^���sv��������a�����������?!��U$��h���C��~"�H���Z��>�j�s�b��x��0��C�v�������r�p!��3Q�9>8*����	F1V����U{�L7q4a�t��#������M��]�������__m�����[K���60k�2�'�f�n��&Y�]�B�Z$�����`�`�~-����I+D��dy�
�����l��������U�����<<;S)��)��R��Brm��(���S)I�:A&Si�Mc��S�fN���	�|v�DJ��KS�r��w:��n���uO���Ry��DJVYT�w;���ipL��L%I�����2��mb6��a�Q�B�C�f�t��qj<���&����k�T���h^���O(���nJr��zK/1%��^������A8�[m>��X�A�g��C%��:v���7h�B�4+�O9��7C�3V��#H�)�Nm7�2�PfxX�
��	��M���b��Ih������6����&�Sb�Q:"��k��)aHaS����w5WT��2���Kd�;m��{}L��������pvrI3��iu�3���>/��L:]E�<o�_8b;�ug�Y`�F�_X
$�z���NMi�����2�,(��u�{-��3C�2B�����4&r_%��m�$2)0�����!I�P�%���4:=����^��@���L*�!�5U>��6�4d�'[L[�B�5e�S�S��e`9}���MJ��R�)��t�-o.�W�M���K��52<�s������~�����:"���4����$:cv_��a�������������x8�]|�l){N��[�f�)S�$�|�^���z\���9�_@�n~�@�!X��d}tDz������	���N�q\���� ���W�d��SI����{�o��Ap2U�g|He��_�<,����!��h8�P|i�q�,b9�!ur2I����,���5��d]�,��3���o�0������'O�)�P,	����,���j��Q���;�F%�5�7��IPJ��O����J��8���W1���eW�q����|�����$B��Z2d)O��K��fv^�����N�������kZ���������/��i5��T��������m�VK�� ��
�����"�LA�M�3T(�9e��6�9�l����}�D��s�8��ty�(o"�c��E�
����T���!���hX�	$��L�����>����J"<s.�O��h������V�=	����R� ���L1b����Q�������d.�D�I�[�v4��0�X��w��������I�M��0�O]
&1^�_����N����
���{z�t����������6?o���~����s>���~v{^��n/|�������~��?�����u��$:NOFg�������������OzaQ�?��;$���h��K�j�KT��x
�p���t��lIM���R"�#��F��.�~��$���n(}����j��4*Pc��Ty��V�b��'���I������7���k�	�I�1�MQ�����Z3�fs��c����e��S�bPi�z9p�Et[��C��N>������`�`�g�p?�Oc>���+�n�H#��
%(�v��
��W�]sB���f8j�d-��bJ!;'�H�M9k467��~<i�%�Y�=�53Q�N\\�v�rj�Y��QS3��"��}����E.���%I�5�����Y�T@�eBt�j���
�m1h��&3�AZ�������?DE�)|7��&i:Cul�?`��Y�w�l5���Rx��X3"h��&F�0��@E�l��@j����P�*�i.u����x�$�;Mmdfh�^��Xw��-u	�� ���B���_(�8*�5@A��3�q���5�t[R��I����N����A ��
,Y���N&v���TF���	t�h�%	�mg�dL�rz��?��p��-�mn���S��cl��(CWe�Z/<�M����U��"�c���L���mx��1L����D������w�B,?�
y�N���-3���������&�8�F�V+����g�+����"tV
O�h���p������O�����Nd����	�q��8|��K����)��!������`6y�Zo���6]�&�4}�_����<�W������SG���}|����G���Z'WP���O�`V;�p:`
��}��!� �9�5*�@�O�:S9�m~���%r6e;G��HC�JBl��a#�l�cH��"6����x�I���mW��.~��;f�,���@Z�����m1f�Qf`�Sk�'|lr�SK�4�cjy��f�kkN�����G@���v�YB��Y����H3v������u:WG��s`4��D�_�#��3���f�b��u��C��En�X+J���%�e�����Z|\|�Z|�����
$XXe��� ��M��^���G���p7
�f�����?����W3vD����f����r]�� ��S�5���5�nnWq���i�H��o�x�BF�����L �o��10�����"
��X�����d7����qP����?��^Z=;���������(|�u`��=�:�oPec�C���lH-�C*�tQM����_z�
��%��pP�t�;����(^�W.-t�C�2"���V�S���r>�����:��B��A���]��O�0n�:*�8*�8*4+;*�:��9*�z:�������p|�yA���������p��[��� `�[5�<��L&�J��	�YK��m�%#y����T�F���k��`f,���M�o�4�TL4*x�H���/"���w��o� wn���NF�#���'�%\�o9�2�n�1�(	,oo��V@'S���=��#�b@����D/�7�����������lW�1����|�l�E�SS
�~�o�F�k�:>���#}?c�T���L�c�Z��`������%���d������EL���y, =_�Y�U%���4�:q�#���v������(�"��$^�FNN�'��9��,��5@V���i4+0D����}���2����3�TM��Ag��qfwjsr0��[�"���P
��=��
�������^���>��B[����2��z�(]@C� M�5���M������N��?&8���"s�������������YPv�_"���2�%f�j%������46�{\����fc+Y	,@X+�?�8l7���>
dX����%��������K�kEt�U�]���i489�w��`|�9i������vN�c���Y�h1K��d1[����Y�������o����r��lz3�G���=,���F������zr���`^�
�:E?Fk��uJ-��f��8}��S1�>����AX�^U~���Gn�4�OraR�7�KQ���@5+<�Y�y�,&|���;y��O��`q���ik�?�R�D%����|\�d�A�Sd�9��9�([����#���L/���j?.�U�ydq3�D��yZ|V�1�U;h�	�)M�;���[����q�(g�����X�}-�HT�`�J��]���c2�";+����v�����h`,r�@A������d��R��[|��
�c&��q����@mG�ej�����:��|��r�����#^�T�G����8��}9���!l���C����R
*�p����.{�|�0�djj�����D~Uh�O�!�����j��v{��#+����<Z������'��p��T4'�c����u4��%$����;]D������W�����������]�J�^��kk�?Fd"w����u?c����������_����wt��Rd�]>B�bc$����!���o���h-�o>	�Q�l���j�b��0������D���������r���
,<�$�[���-'>�sT���V�����l���|
C�D+�2��y��%i�W/��B�C�.sz����U���iT��B���by�@?�U;3d�mlf��k)/�9�p"�`�T6���3]����:�$��8�������L�����>����%�y�K�����f���������'�{���S�2���
�F�eS�o���uU'�L�;�I������,��b�e�Oz`��|9)���4E������]p4[i4�G�z��u!r�,r�A50��A���� �E>rR9�I��&<Z�M�1������*�����)�n=u��i.�����f�`z��=����I�*O��u���@b��<�N9��;���#3�g�O*���,������6���Q#2��&�0}����Ge���n����N�*�X���#�a+x�0-u�����@$j�"�silc����Qn* ��%p���2�Z�e���P�V����~���=���A��29����ww�������Qz���}4%`�b��n�,"
~BY�H#�@�H�+��Nw�<���j<)�����
����2�]����
$f��
�|����2{b�yl�i��N���=��c8���c�{�[O��K/���9��K��{z$H[R��9S�@C��>i\e��<�g�����pq+%�m*/��q:W�x*G����~-���U�2����E�I������*>�X�WuZ�['X�t���f=��50w?��g���JO}�'�<���q�A��J���uix�(���Mu��Gv�W��wa��@��t�P�����>]�CNs��E��*�A������0N���
��R��_Q�v,�����/���-��;���t��������m�UeG������l��:'�����LCy��C����j5�$w��FXpG�����^���%F��?K&�����y��3;�s���9Z8����7S�����Ym��&/`��yf-�/����z��o��w�Z�P�JT�)T������m������h[����SKuk{/�������
���^X������0ql��L�y]�M��=q������L����Ue|����w���U~����9���g�W�{E$UY���#X��[��)h���'SJ���]�,o�������<U#d�K+�=����������������
H���x7�q��qp���<�.��D�B�����]������C�QkLn��l/&�4��H��;�C|�-9T2o]�ed3���@�&_A�|���U�����E#x�]b������������G
�?�9\������x�)W�9��o�z,���l���
gP���q�;&�Xt�����m�c����P��o�� �L�,a����E�@�������\�����U���'�2���?�b���Y6�m#���6"`�iD���<gx0���#ug��@��{�9X-�b0�Rr<������lVn��*��9��
|��!�G������$�� �,��wDj@��Ms>����mp�o�V�3,���^n��)
C��(�7�2�y�c�%��o6��Jw"����cX�#��n�����x7�)�]�����X�!���L�l�0$�C���-/����a(q������R��;�*B��.
����$Gl��iljl1�?��"H�+z����@,����kx�f�.����\��~����@e94b��[NHR;A��y}!``�D�.�8e�QM����p����2��o���v�A�����@�J����"o[R�[�2��f�_+��KV��g�����UnNs���r][���|��-ws��75����2�x(�$��bm�V���L-��l�s����L9�_����|�Ib��K���R��,%2�pI�`5��2-��:P�of��~��:��K�5��N�
��N'��k���#A�3���'��b�{o��}���"����c�e�?-�����GfB��J_���{����Vk4�ONz9�
d#Ff��u�~������@��=��]?����5���(
&.8�R������>��6������(E�^���h~����������'LU����(�=��#/*�W�(
G��@�/'7�E���"��;�2�MNZ����a�$�
��	8������ �LC��W���,���M�E%�:��v|�� Sa��W^$-��g�+�
��(R]��Yau��>7��*i&��~Nt�����O�a4��Zq8�L�����"�r
!����
r	��Q�a?k�1�����<��C��xX`�p���R���z�TX���R��,M�~`{���r�L���������������j|���!_�Vx��P��&;�'{�@�N���Y�}?�4��$[��O�S���o��j���Y�{�N6	.f<�Z���F��c�����,�7�E���x��$�1L~sF�&�nS��������^�����
�������t�htv:���N{��zq/jF�I;:�v���i�l��L�/�q#8	^�#	�{��3x���mS�_9i�NK(��Nb+g�A�U�������TM]F���-���mb��G�Q��f����6�8����J h��"�W��MoU�����]���H����f�z�-��*&o�������N��F�Xn�Z0QaC8�*7�r��u�]��Y���i]@���8�1���/<H% ���O��U+O[�!OPq��e!Cd���b�n)�R�i\��/
J��J�6�: �+��V�%�d���,	���.���t:�	��m�qP��M%��H�=�,���&�v>�F0r�]�����t#�?�,T&H�Dt��)�����r��G���a����;�51�H�	����ry=O)J���i����U]#2T*�V��v�7a��K�MbPl�<e�B�/��3�������qB���B-�af����0T2���)��84��0��
|^i���� ���)H��\�� ��k�p�'oS���nE4j�{�����p�u"��������-V3�~#1�4�u��
���� �j�w����F�]�����?�z��'|���D����V���X�E1�3U]����G��Q�&����O��D
jt���*Y��B�B��mW��a��H����d��r�*Bo���6��p�nW����"+�����$X?��UJW�$��y���s }IG��$2c>�J�1�*�����7ii��CZ����L8Y�D�#sB~�l�e����c����x���	�/����]�q����B^�[M�3>oK����vT������z�cL::�$s/5�����7�~���C��bU��9x#e������u6c����T����S04�v���������i���'uQ�N���%7������C2e1���=���)#��:��iJ�
�M�*r=���l�����Ln�|��D)uM���
�a���0r�w�|��?����*Q	]ulY�K^�6�Za�k�6$s6�
Q*��Z(R�lQ�4Q9z
\���@3��z�YO��Uz��PUc�
��/��xC�N�#�P��I�O���|�tw��He��MBp�8��'�,�MBN��K����2��^�n;�I�7��3@��#j����o(�Gk��0��(��>�H.W��������0�������D����z��f���-��U�g��Zwn�h'A�bl����b���x6B��Tj<V'qQj��(b��V�]S�_1�L
�����;s��z!���X��:�d
�J���3�O����A.�Z�	zm?�Ax�L���J�|�r*"^�3���V+n���dZ9��ighq�,�050��9z2������j�8�-M�kMr<�s��cl����s�(���3�>��0��z�"��+�i(k���l~Jn�`��Fw��:�En�"��D����"V����`��L�����(f?e���09�+f�����7�#|2K��U��(Fj�?}�z�rY���D�jXGW:�5��H�� y����W8�������G�3��h�	�t@�I������Xqfu�'��u`5�DRX�f����Gk�ufC'Xj�F�L���x#�)����P���/����g��,��F{ �4S(a�rvZ��	�Z���^����iQ�E��6�10����O��|�:����/���.�:
�v5Q�(�*�YO Q��Tg"r1���T��Us�������8���p�%sd6��R�i��7����4���CC����$�ed����=XF�Y��,��Y��,��Y��[b�@wI�i^�2T���3���"�cu������(�(������~�� ���I���b:�N�����m+���rKa�i�iZ���s�,��xJ��,�&b�%�xA����vCjM���{�`�{�`�{�`���`�{�`�{�`M��W&k�%��i�M�%R"�R�r=�-�r�fU��������Y�%�>�%�j&/`�d����Q��@�_�L�����^(��%�]��lr�Z'�^gU�3�JE�Li��hSf�"/�}��,8*Q�p�y<���	�tK�n�$*J��y�������������<�g���a�-�wq����+��!����6���T&�#����h���cP�,��d�����s��<���r�ax�z��_M~3��NI����G���[�y�t�4�W���g
���v�4��L��u��b%��/~��������W�/9��/�i�i�m�S��@tG�
y_������z�5��z��y����*���\����W]U���_D��$����
�����&����R���C����!��s��<9����p�d@���@]����5O��xHwE�x��n����Pv�=����{���Ir57��#I���$Nz���G&W�N.�C����������v���XR4��������+��+��B�,/Y|���jS���g��H��J=���uxjz���&;-DP�������������]��v�tZ���i��q�i1}�������b���:�|��N]�N1���0�=�j�'I��e�����|6�W��I�����������xB�����k�c
j� ���k&��]�����p��g ��4IM�%�trF���8��������O�w����O�{:d2J���D�1
�`t4�L����gP��h�AM	!��k�Y��4G��]�?
�w���\ra����7�]'���-�%a#���]�U5�
u5u#U{��n�����5K]��c�x����a�p���%�U��cO��5�(t���(f{g�Z�w�Q8��v�02L[r��,:�2<�t4����%�������������>^tG����!8PP�'�8Z*����	0}0L����[3Q9��d�9���<��g_[�l<��W�������2Y�N��$(
`\��]�)���ua����'�����|��{��NO�����4�u������N�4������3��N����K����N���
���}���j�_
��8���u�S�d��	�v���]Z:qJ�m��QY�z�����s�H*�
��7$�P� ����u�?��)C�Y7=J�Bk��dm�K�����3�?S�j^�o�<D�%3M�2M�G��Q�����Ti��})��V��P�8�/
�*��[|s��Y0���my�g��v=���z|}[�q��l
n��%2���&[�-�SM��D�N��p�d���d�V
�-`��:%5����SFf�		�9�����/�X;����A�6�v*s�p��	���c��sv��/�|J���`�_2��U�#�}L�`M���O���f�wH�v��G��)�s%�b�]���#���
O�q<�VO��������^��./�HzF�&�D����|{I�[N	'����q����W i��L�S?�.������0Z�#q?���X$���RR�0��1%�
;��DACcTNs�ANM��3�y�J0OKM�����K�V2��F���
>��zxi���pk��%]��:C��!��������h���L,�z?z���
$�y���-�w��QXmP8��qW�s���0e��:�������C�T}��t��2�:�nf\ ���j����$�:����,�s.U��T�.�XI����1�eNg���b�"s�A�MC7�[���#�PW`Sl���[b���J�rT[����]u��l�s�C>-����x��uG��m�:8���Q������N��9��u�bl��f=����y@�z�^,&�f
��lw7��������Z2\���"F�0��rf���&w��D�&(K���|���;wr���D`��T��������R��[�=��G�t��b���N	?4��!N���!�c"�m���`�tB��^�����wk�,��_=.�_����0�����W����0�|���s��Rb;a��������5���Y�x���B�����A�^N��rd]W<��|�!tJ����z���D*~V)�X\}Pn��&���:���5����j2![���=���v�����e�DC�F��������*?oTr�%��8��T���17�*�d�F�JYy��%P�g��"����l��&\���%���X0
��I��;��P
����wv�z\�_���0T ������;�R���}�U��]�?�{�����	�!���a_�<T;�_��<��U�)�e��UQ�������\eF��%�k���7�\F��x��R����5	'�e�����������+�-;��i���������%9��Mj
��/O��G�����d���qy�o�m����Xn����]Y�M�z"�tyHk��,GQ���#���wwe��rC��s�t!���&Y[�z�S�0������5S�L5(8O@��1^$�d����A�R3"�^�l�!�5�����������yv|���!
�,�nGtG�n�n&��+�U��Eu��H�����j�E�P�S(��vK1���o��T��]��k�@���GW��#
�d�El���+|��^���|�����ow������I��l���d_����NV�h1C��L���8*u;����g���JA3c�����v��{h���H���zS��l��u���L��������9P���-��R@��c����h\�x�?,|!����x�z��zZ�%S~[��o
���	���eZ�������e�C�0����%�^��v���^��f������u����h�`�����/�����6N�	Z�����k�p��W�5SE�(�%T�SM�������%�s����N^j���E�u�5��j����C/������8����C�Qv;��g^:H�'�����S�>(��KY���Q� �� ?|>J�c����#VD��pH���k���?d����o:E�Uj^��T������t��?�|��@�v�tge��nqC�����3t������h�a�l���=��R�v|���K��S����J�;�|��Y�������r(��vN9�'�V������H9��n�)��X���|X i�sd8���8�x��.�H�bK����@�cK�=����ex���J0�e�]��o�W���\��WK`<���DH��+�E�7tzE"��I����������<�4��@}��~�h�P7�U�c�&z\;J.��Eq�0e�LB9Y0�A
�g�O����$�Y'�+�����#;
�E�$��A���!�>$H��o�y�n�������i�2����)����Q22t!�%�
�3���<f�h��E����3�l�q{��D�i�.���bb�=�&�Q/j�����l��l�5�vZuMQ2h���~���Yr:[�`�g�<m^�!%��<_�&U�uQw��-�)^��m�3��������y�
[��(E�<��pL��!�{�.|����y��v�v�)k8�B���S���ya_�=eA�3����ck����4!M��K����i#lct�^c���y"K�&F��k��x�i	�j;�?5��]�G��RNxr�G�����v-���hmD#_E���}IY	�����xo��B���}����d&�=���<W���}U���	C���M�Ag��N�sS��O[w_�a�p�
���.�W8����v��&
�J�d���u�/�me��2���D�����Q���������'�6H����lc�j�@M*Q���&�`��y}���#}���R&��P�W^��/)(Prf�n3�wE^m|�`7R�8�F6��"�9D��;��0�Q�*kkV �����hNy8�<4����$�E��$�FeT� Q<
q���� .X��#�s���L��}L��@@=��{|����G��:����d��kE)������l�-�/�q���5��Im%�~��`[���x��.4i��oN����|����a�]d�r���	�4�'[`��,��������t��-+�t�������:FDyd��nH��}�5~���EP��cNn|����x��SGb#?�|9 I��<9K�*���B��D�H��e��+���y�����b!��Q������z����^�����5E9L��b���%�*�p�\Sm���0�;R��������s��(�����Q<
w�+�%+��p�"NV��?)�s��(��e�"�8Vb�4}�Nf�"�N�<<���z����8%�K��
�L�E���5h�����pp��e�Z-r����Ko����H,����b�'�'S���MA��t;����3����TY_Z�KU=�]�����(0R��e�j�n���hQ<����%�!��\��'?Eg�n���3�b�69�b6$�=s�I�S��J�%J�z��dI�n��;�9-u6��\����W�
��5x�_<��3�j�<�T"�@I����Zy���F��[���{[�]���$O��N�F���$����hn�[�
u
��0H$����Lh/� +�R7/�^�`��i���E	?�e����Q��-��u�~�NC���h)��]�^�e@�5�"g(39��I�B��[�j�#���(�u���)�<DDr�P���L��]�E�A�s���Y1�Wn)gg��p�q4�(���b��WWp(B�R��Z`���)C7@�p��Vo5T&���`�E�U�5�st�#S����f�h�	���Q_���N��?��Vt��~x����/(��%�)�9�������Fo�fF.u9���|�l	����-]G\4���1�����*`"�f�������j)r��;r�����:�d����y�:�o����*����UYc1f#�����	�6��2 �	���c���n��a6��K!�|7���W[|)���q�#tR�w��������E�YH�l������,~*|{�	�+�rry��b(}Pve�����+����M�t���|kbaF���2"��7�t��M���P�c�A�N�2N���6:;�B���*m
���5YG�8���P�i%`�)��M���h9�9��G�� K�S\��dbHc���(��vH!
�Q�h��Kof�X���:#����^y�f�
�P���
��& ����W3OZ��H=>�4��9��RF,����t�W(��o\
�+5Y� ����1�6�����9e�Ds�.R�!(.#�7K�	Yb�����Q9���8W������T��u��3��J��1�>�P&�XS<K-�0�,�:X9�����#>�Z�3�73�C�C-�nw?6imc#��'�l������[����H��Y�����q�?�q,�!<�y�Y/%n�dwxC�W�t��0���Q�@J��~�q���b��n5	���9����}��PJ\` �k|h$��`tA��n&�~�����K�<����&�,�	�����cF��$Y�?"�#K��>]H����I]U
��9�6C���/f;�����8F'�������dO�$��k�{dlD��������Z��b�����/T�;NG��K��L����x}8���=|��h%^fvI�s���
f@N%�����
I%V�J�6N�� ��O
z����)����X�!�����rnt��3��lgb'�u�y����[3�n�>�y�<|>��������y�<|>��������y�<|>��������y������F�
#141Rafia Sabih
rafia.sabih@enterprisedb.com
In reply to: Ashutosh Bapat (#140)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Jul 14, 2017 at 12:32 PM, Ashutosh Bapat <
ashutosh.bapat@enterprisedb.com> wrote:

Here's revised patch set with only 0004 revised. That patch deals with
creating multi-level inheritance hierarchy from multi-level partition
hierarchy. The original logic of recursively calling
inheritance_planner()'s guts over the inheritance hierarchy required
that for every such recursion we flatten many lists created by that
code. Recursion also meant that root->append_rel_list is traversed as
many times as the number of partitioned partitions in the hierarchy.
Instead the revised version keep the iterative shape of
inheritance_planner() intact, thus naturally creating flat lists,
iterates over root->append_rel_list only once and is still easy to
read and maintain.

On testing this patch for TPC-H (for scale factor 20) benchmark I found a
regression for Q21, on head it was taking some 600 seconds and with this
patch it is taking 3200 seconds. This comparison is on the same partitioned
database, one using the partition wise join patch and other is without it.
The execution time of Q21 on unpartitioned head is some 300 seconds. The
explain analyse output for each of these cases is attached.

This suggests that partitioning is not a suitable strategy for this query,
but then may be partition wise should not be picked for such a case to
aggravate the performance issue.

The details of the setup is as follows,

Server parameter settings,
work_mem - 1GB
effective_cache_size - 8GB
shared_buffers - 8GB
enable_partition_wise_join = on

Partition information:
Type of partitioning - single column range partition
Tables partitioned - Lineitem and orders

Lineitem -
Partition key = l_orderkey
No of partitions = 18

Orders -
Partition key = o_orderkey
No of partitions = 11

Commit id - 42171e2cd23c8307bbe0ec64e901f58e297db1c3

I chose orderkey as the partition key since it is the primary key of orders
and along with l_linenumber it forms the primary key for lineitem.
For the above mentioned settings, there was no other query that used
partitioned wise join.

Please let me know if any more information is required regarding this
experimentation.

--
Regards,
Rafia Sabih
EnterpriseDB: http://www.enterprisedb.com/

Attachments:

part_reg.zipapplication/zip; name=part_reg.zipDownload
#142Robert Haas
robertmhaas@gmail.com
In reply to: Rafia Sabih (#141)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Jul 19, 2017 at 12:24 AM, Rafia Sabih
<rafia.sabih@enterprisedb.com> wrote:

On testing this patch for TPC-H (for scale factor 20) benchmark I found a
regression for Q21, on head it was taking some 600 seconds and with this
patch it is taking 3200 seconds. This comparison is on the same partitioned
database, one using the partition wise join patch and other is without it.
The execution time of Q21 on unpartitioned head is some 300 seconds. The
explain analyse output for each of these cases is attached.

Interesting.

This suggests that partitioning is not a suitable strategy for this query,
but then may be partition wise should not be picked for such a case to
aggravate the performance issue.

In the unpartitioned case, and in the partitioned case on head, the
join order is l1-(nation-supplier)-l2-orders-l3. In the patched case,
the join order changes to l1-l2-supplier-orders-nation-l3. If the
planner used the former join order, it wouldn't be able to do a
partition-wise join at all, so it must think that the l1-l2 join gets
much cheaper when done partitionwise, thus justifying a change in the
overall join order to be able to use partion-wise join. But it
doesn't work out.

I think the problem is that the row count estimates for the child
joins seem to be totally bogus:

-> Hash Semi Join (cost=309300.53..491665.60 rows=1 width=12)
(actual time=10484.422..15945.851 rows=1523493 loops=3)
Hash Cond: (l1.l_orderkey = l2.l_orderkey)
Join Filter: (l2.l_suppkey <> l1.l_suppkey)
Rows Removed by Join Filter: 395116

That's clearly wrong. In the un-partitioned plan, the join to l2
produces about as many rows of output as the number of rows that were
input (998433 vs. 962909); but here, a child join with a million rows
as input is estimated to produce only 1 row of output. I bet the
problem is that the child-join's row count estimate isn't getting
initialized at all, but then something is clamping it to 1 row instead
of 0.

So this looks like a bug in Ashutosh's patch.

--
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

#143Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Robert Haas (#142)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Jul 20, 2017 at 7:00 AM, Robert Haas <robertmhaas@gmail.com> wrote:

I think the problem is that the row count estimates for the child
joins seem to be totally bogus:

-> Hash Semi Join (cost=309300.53..491665.60 rows=1 width=12)
(actual time=10484.422..15945.851 rows=1523493 loops=3)
Hash Cond: (l1.l_orderkey = l2.l_orderkey)
Join Filter: (l2.l_suppkey <> l1.l_suppkey)
Rows Removed by Join Filter: 395116

That's clearly wrong. In the un-partitioned plan, the join to l2
produces about as many rows of output as the number of rows that were
input (998433 vs. 962909); but here, a child join with a million rows
as input is estimated to produce only 1 row of output. I bet the
problem is that the child-join's row count estimate isn't getting
initialized at all, but then something is clamping it to 1 row instead
of 0.

So this looks like a bug in Ashutosh's patch.

Isn't this the same as the issue reported here?

/messages/by-id/CAEepm=270ze2hVxWkJw-5eKzc3AB4C9KpH3L2kih75R5pdSogg@mail.gmail.com

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

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

#144Robert Haas
robertmhaas@gmail.com
In reply to: Thomas Munro (#143)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Jul 19, 2017 at 7:45 PM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

On Thu, Jul 20, 2017 at 7:00 AM, Robert Haas <robertmhaas@gmail.com> wrote:

I think the problem is that the row count estimates for the child
joins seem to be totally bogus:

-> Hash Semi Join (cost=309300.53..491665.60 rows=1 width=12)
(actual time=10484.422..15945.851 rows=1523493 loops=3)
Hash Cond: (l1.l_orderkey = l2.l_orderkey)
Join Filter: (l2.l_suppkey <> l1.l_suppkey)
Rows Removed by Join Filter: 395116

That's clearly wrong. In the un-partitioned plan, the join to l2
produces about as many rows of output as the number of rows that were
input (998433 vs. 962909); but here, a child join with a million rows
as input is estimated to produce only 1 row of output. I bet the
problem is that the child-join's row count estimate isn't getting
initialized at all, but then something is clamping it to 1 row instead
of 0.

So this looks like a bug in Ashutosh's patch.

Isn't this the same as the issue reported here?

/messages/by-id/CAEepm=270ze2hVxWkJw-5eKzc3AB4C9KpH3L2kih75R5pdSogg@mail.gmail.com

Hmm, possibly. But why would that affect the partition-wise join case only?

--
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

#145Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Robert Haas (#144)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Jul 20, 2017 at 2:02 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Wed, Jul 19, 2017 at 7:45 PM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

Isn't this the same as the issue reported here?

/messages/by-id/CAEepm=270ze2hVxWkJw-5eKzc3AB4C9KpH3L2kih75R5pdSogg@mail.gmail.com

Hmm, possibly. But why would that affect the partition-wise join case only?

It doesn't. From Rafia's part_reg.zip we see a bunch of rows=1 that
turn out to be wrong by several orders of magnitude:

21_nopart_head.out: Hash Semi Join (cost=5720107.25..9442574.55
rows=1 width=50)
21_part_head.out: Hash Semi Join (cost=5423094.06..8847638.36
rows=1 width=38)
21_part_patched.out: Hash Semi Join (cost=309300.53..491665.60 rows=1 width=12)

My guess is that the consequences of that bad estimate are sensitive
to arbitrary other parameters moving around, as you can see from the
big jump in execution time I showed in the that message, measured on
unpatched master of the day:

4 workers = 9.5s
3 workers = 39.7s

That's why why both parallel hash join and partition-wise join are
showing regressions on Q21: it's just flip-flopping between various
badly costed plans. Note that even without parallelism, the fix that
Tom Lane suggested gives a much better plan:

/messages/by-id/CAEepm=11BiYUkgXZNzMtYhXh4S3a9DwUP8O+F2_ZPeGzzJFPbw@mail.gmail.com

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

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

#146Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Rafia Sabih (#141)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Jul 19, 2017 at 9:54 AM, Rafia Sabih
<rafia.sabih@enterprisedb.com> wrote:

Partition information:
Type of partitioning - single column range partition
Tables partitioned - Lineitem and orders

Lineitem -
Partition key = l_orderkey
No of partitions = 18

Orders -
Partition key = o_orderkey
No of partitions = 11

The patch set upto 0015 would refuse to join two partitioned relations
using a partition-wise join if they have different number of
partitions. Next patches implement a more advanced partition matching
algorithm only for list partitions. Those next patches would refuse to
apply partition-wise join for range partitioned tables. So, I am
confused as to how come partition-wise join is being chosen even when
the number of partitions differ.

--
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

#147Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Ashutosh Bapat (#146)
Re: Partition-wise join for join between (declaratively) partitioned tables

On 2017/07/20 15:05, Ashutosh Bapat wrote:

On Wed, Jul 19, 2017 at 9:54 AM, Rafia Sabih
<rafia.sabih@enterprisedb.com> wrote:

Partition information:
Type of partitioning - single column range partition
Tables partitioned - Lineitem and orders

Lineitem -
Partition key = l_orderkey
No of partitions = 18

Orders -
Partition key = o_orderkey
No of partitions = 11

The patch set upto 0015 would refuse to join two partitioned relations
using a partition-wise join if they have different number of
partitions. Next patches implement a more advanced partition matching
algorithm only for list partitions. Those next patches would refuse to
apply partition-wise join for range partitioned tables. So, I am
confused as to how come partition-wise join is being chosen even when
the number of partitions differ.

In 21_part_patched.out, I see that lineitem is partitionwise-joined with
itself.

Append

-> Hash Semi Join
Hash Cond: (l1.l_orderkey = l2.l_orderkey)
Join Filter: (l2.l_suppkey <> l1.l_suppkey)
Rows Removed by Join Filter: 395116

-> Parallel Seq Scan on lineitem_001 l1
Filter: (l_receiptdate > l_commitdate)
Rows Removed by Filter: 919654

-> Hash
Buckets: 8388608 Batches: 1 Memory Usage: 358464kB
-> Seq Scan on lineitem_001 l2

Thanks,
Amit

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

#148Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#142)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Jul 20, 2017 at 12:30 AM, Robert Haas <robertmhaas@gmail.com> wrote:

This suggests that partitioning is not a suitable strategy for this query,
but then may be partition wise should not be picked for such a case to
aggravate the performance issue.

In the unpartitioned case, and in the partitioned case on head, the
join order is l1-(nation-supplier)-l2-orders-l3. In the patched case,
the join order changes to l1-l2-supplier-orders-nation-l3. If the
planner used the former join order, it wouldn't be able to do a
partition-wise join at all, so it must think that the l1-l2 join gets
much cheaper when done partitionwise, thus justifying a change in the
overall join order to be able to use partion-wise join. But it
doesn't work out.

I think the problem is that the row count estimates for the child
joins seem to be totally bogus:

-> Hash Semi Join (cost=309300.53..491665.60 rows=1 width=12)
(actual time=10484.422..15945.851 rows=1523493 loops=3)
Hash Cond: (l1.l_orderkey = l2.l_orderkey)
Join Filter: (l2.l_suppkey <> l1.l_suppkey)
Rows Removed by Join Filter: 395116

That's clearly wrong. In the un-partitioned plan, the join to l2
produces about as many rows of output as the number of rows that were
input (998433 vs. 962909); but here, a child join with a million rows
as input is estimated to produce only 1 row of output. I bet the
problem is that the child-join's row count estimate isn't getting
initialized at all, but then something is clamping it to 1 row instead
of 0.

So this looks like a bug in Ashutosh's patch.

The patch does not have any changes to the selectivity estimation. It
might happen that some correction in selectivity estimation for
child-joins is required, but I have not spotted any code in
selectivity estimation that differentiates explicitly between child
and parent Vars and estimates. So, I am more inclined to believe
Thomas's theory. I will try Tom's suggested approach.

I am investigating this case with the setup that Rafia provided.

--
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

#149Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Amit Langote (#147)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Jul 20, 2017 at 11:46 AM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

On 2017/07/20 15:05, Ashutosh Bapat wrote:

On Wed, Jul 19, 2017 at 9:54 AM, Rafia Sabih
<rafia.sabih@enterprisedb.com> wrote:

Partition information:
Type of partitioning - single column range partition
Tables partitioned - Lineitem and orders

Lineitem -
Partition key = l_orderkey
No of partitions = 18

Orders -
Partition key = o_orderkey
No of partitions = 11

The patch set upto 0015 would refuse to join two partitioned relations
using a partition-wise join if they have different number of
partitions. Next patches implement a more advanced partition matching
algorithm only for list partitions. Those next patches would refuse to
apply partition-wise join for range partitioned tables. So, I am
confused as to how come partition-wise join is being chosen even when
the number of partitions differ.

In 21_part_patched.out, I see that lineitem is partitionwise-joined with
itself.

Append

-> Hash Semi Join
Hash Cond: (l1.l_orderkey = l2.l_orderkey)
Join Filter: (l2.l_suppkey <> l1.l_suppkey)
Rows Removed by Join Filter: 395116

-> Parallel Seq Scan on lineitem_001 l1
Filter: (l_receiptdate > l_commitdate)
Rows Removed by Filter: 919654

-> Hash
Buckets: 8388608 Batches: 1 Memory Usage: 358464kB
-> Seq Scan on lineitem_001 l2

Ah, I see now.

We need the same number of partitions in all partitioned tables, for
joins to pick up partition-wise join.

--
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

#150Rafia Sabih
rafia.sabih@enterprisedb.com
In reply to: Thomas Munro (#145)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Jul 20, 2017 at 8:53 AM, Thomas Munro <thomas.munro@enterprisedb.com

wrote:

On Thu, Jul 20, 2017 at 2:02 PM, Robert Haas <robertmhaas@gmail.com>
wrote:

On Wed, Jul 19, 2017 at 7:45 PM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

Isn't this the same as the issue reported here?

/messages/by-id/CAEepm=270ze2hVxWkJw-

5eKzc3AB4C9KpH3L2kih75R5pdSogg%40mail.gmail.com

Hmm, possibly. But why would that affect the partition-wise join case

only?

It doesn't. From Rafia's part_reg.zip we see a bunch of rows=1 that
turn out to be wrong by several orders of magnitude:

21_nopart_head.out: Hash Semi Join (cost=5720107.25..9442574.55
rows=1 width=50)
21_part_head.out: Hash Semi Join (cost=5423094.06..8847638.36
rows=1 width=38)
21_part_patched.out: Hash Semi Join (cost=309300.53..491665.60 rows=1
width=12)

My guess is that the consequences of that bad estimate are sensitive
to arbitrary other parameters moving around, as you can see from the
big jump in execution time I showed in the that message, measured on
unpatched master of the day:

4 workers = 9.5s
3 workers = 39.7s

That's why why both parallel hash join and partition-wise join are
showing regressions on Q21: it's just flip-flopping between various
badly costed plans. Note that even without parallelism, the fix that
Tom Lane suggested gives a much better plan:

/messages/by-id/CAEepm%25
3D11BiYUkgXZNzMtYhXh4S3a9DwUP8O%2BF2_ZPeGzzJFPbw%40mail.gmail.com

Following the discussion at [1]/messages/by-id/CAEepm=3=NHHko3oOzpik+ggLy17AO+px3rGYrg3x_x05+Br9-A@mail.gmail.com, with the patch Thomas posted there, now
Q21 completes in some 160 seconds. The plan is changed for the good but
does not use partition-wise join. The output of explain analyse is
attached.

Not just the join orders but the join strategy itself changed, with the
patch no hash semi join is picked which was consuming most time there,
rather nested loop semi join is in picture now, though the estimates are
still way-off, but the change in join-order made them terrible from
horrible. It appears like this query is performing efficient now
particularly because of worse under-estimated hash-join as compared to
under-estimated nested loop join.

For the hash-semi-join:
-> Hash (cost=3449457.34..3449457.34 rows=119994934 width=8) (actual
time=180858.448..180858.448 rows=119994608 loops=3)
Buckets: 33554432
Batches: 8 Memory Usage: 847911kB

Overall, this doesn't look like a problem of partition-wise join patch
itself.

[1]: /messages/by-id/CAEepm=3=NHHko3oOzpik+ggLy17AO+px3rGYrg3x_x05+Br9-A@mail.gmail.com
/messages/by-id/CAEepm=3=NHHko3oOzpik+ggLy17AO+px3rGYrg3x_x05+Br9-A@mail.gmail.com

--
Regards,
Rafia Sabih
EnterpriseDB: http://www.enterprisedb.com/

Attachments:

Q21_SE_patch.outapplication/octet-stream; name=Q21_SE_patch.outDownload
#151Rafia Sabih
rafia.sabih@enterprisedb.com
In reply to: Ashutosh Bapat (#149)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Jul 20, 2017 at 2:44 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

On Thu, Jul 20, 2017 at 11:46 AM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

On 2017/07/20 15:05, Ashutosh Bapat wrote:

On Wed, Jul 19, 2017 at 9:54 AM, Rafia Sabih
<rafia.sabih@enterprisedb.com> wrote:

Partition information:
Type of partitioning - single column range partition
Tables partitioned - Lineitem and orders

Lineitem -
Partition key = l_orderkey
No of partitions = 18

Orders -
Partition key = o_orderkey
No of partitions = 11

The patch set upto 0015 would refuse to join two partitioned relations
using a partition-wise join if they have different number of
partitions. Next patches implement a more advanced partition matching
algorithm only for list partitions. Those next patches would refuse to
apply partition-wise join for range partitioned tables. So, I am
confused as to how come partition-wise join is being chosen even when
the number of partitions differ.

In 21_part_patched.out, I see that lineitem is partitionwise-joined with
itself.

Append

-> Hash Semi Join
Hash Cond: (l1.l_orderkey = l2.l_orderkey)
Join Filter: (l2.l_suppkey <> l1.l_suppkey)
Rows Removed by Join Filter: 395116

-> Parallel Seq Scan on lineitem_001 l1
Filter: (l_receiptdate > l_commitdate)
Rows Removed by Filter: 919654

-> Hash
Buckets: 8388608 Batches: 1 Memory Usage: 358464kB
-> Seq Scan on lineitem_001 l2

Ah, I see now.

We need the same number of partitions in all partitioned tables, for
joins to pick up partition-wise join.

Oh, I missed this limitation, will modify my setup to have same number
of partitions in the partitioned table with same ranges. So, does this
also mean that a partitioned table will not join with an unpartitioned
table without append of partitions?

--
Regards,
Rafia Sabih
EnterpriseDB: http://www.enterprisedb.com/

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

#152Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Rafia Sabih (#150)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Jul 21, 2017 at 11:42 AM, Rafia Sabih
<rafia.sabih@enterprisedb.com> wrote:

Following the discussion at [1], with the patch Thomas posted there, now Q21
completes in some 160 seconds.

Your earlier reports mentioned unpartitioned case taking 300 seconds,
partitioned case without partition-wise join taking 600 seconds and
with partition-wise join it taking 3200 seconds. My experiements
showed that those have changed to 70s, 160s and 160s resp. This is
with Thomas's patch. Can you please confirm?

The plan is changed for the good but does not
use partition-wise join.

As explained earlier, this is because the tables are not partitioned
similarly. Please try with lineitem and orders partitioned similarly
i.e. same number of partitions and exactly same ranges.

Not just the join orders but the join strategy itself changed, with the
patch no hash semi join is picked which was consuming most time there,
rather nested loop semi join is in picture now, though the estimates are
still way-off, but the change in join-order made them terrible from
horrible. It appears like this query is performing efficient now
particularly because of worse under-estimated hash-join as compared to
under-estimated nested loop join.

Earlier it was using partition-wise join between lineitems (l1, l2,
l3) since it's the same table. Now for some reason the planner doesn't
find joining them to each other a better strategy, instead they are
joined indirectly so we don't see partition-wise join being picked. We
should experiment with orders and lineitems being partitioned
similarly. Can you please provide that result?

For the hash-semi-join:
-> Hash (cost=3449457.34..3449457.34 rows=119994934 width=8) (actual
time=180858.448..180858.448 rows=119994608 loops=3)
Buckets: 33554432
Batches: 8 Memory Usage: 847911kB

Overall, this doesn't look like a problem of partition-wise join patch
itself.

Thanks for confirming it.

--
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

#153Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Rafia Sabih (#151)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Jul 21, 2017 at 11:54 AM, Rafia Sabih
<rafia.sabih@enterprisedb.com> wrote:

So, does this
also mean that a partitioned table will not join with an unpartitioned
table without append of partitions?

Yes. When you join an unpartitioned table with a partitioned table,
the planner will choose to append all the partitions of the
partitioned table and then join with the unpartitioned table.

--
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

#154Rafia Sabih
rafia.sabih@enterprisedb.com
In reply to: Ashutosh Bapat (#153)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Jul 21, 2017 at 12:11 PM, Ashutosh Bapat <
ashutosh.bapat@enterprisedb.com> wrote:

On Fri, Jul 21, 2017 at 11:54 AM, Rafia Sabih
<rafia.sabih@enterprisedb.com> wrote:

So, does this
also mean that a partitioned table will not join with an unpartitioned
table without append of partitions?

Yes. When you join an unpartitioned table with a partitioned table,
the planner will choose to append all the partitions of the
partitioned table and then join with the unpartitioned table.

I tested this set of patches for TPC-H benchmark and came across following
results,
- total 7 queries were using partition-wise join,

- Q4 attains a speedup of around 80% compared to the partitioned setup
without partition-wise join, the main reason being the poor plan choice at
head for partitioned database.
When I tried this query with forced nested-loop join then it completes in
some 45 seconds at head. So, basically when no partition-wise join is
present because of terrible selectivity estimation optimiser picks up a
hash join plan, which results poorly as the estimated number of rows are
two orders of magnitude lesser than actual.
Note that this is not the effect of [1]/messages/by-id/CAEepm=3%25 3DNHHko3oOzpik%2BggLy17AO%2Bpx3rGYrg3x_x05%2BBr9-A%40mail.gmail.com -- Regards, Rafia Sabih EnterpriseDB: http://www.enterprisedb.com/, I tried this without that patch as
well.

- other queries show a good 20-30% improvement in performance. Performance
numbers are as follows,

Query| un_part_head (seconds) | part_head (seconds) | part_patch (seconds) |
3 | 76 |127 | 88 |
4 |17 | 244 | 41 |
5 | 52 | 123 | 84 |
7 | 73 | 134 | 103 |
10 | 67 | 111 | 89 |
12 | 53 | 114 | 99 |
18 | 447 | 709 | 551 |

The experimental settings used were,

Partitioning: Range partitioning on lineitem and orders on l_orderkey and
o_orderkey respectively. The number and range of partitions were kept same
for both the tables.

Server parameters:
work_mem - 1GB
effective_cache_size - 8GB
shared_buffers - 8GB
enable_partition_wise_join = on

TPC-H setup:
scale-factor - 20

Commit id - 42171e2cd23c8307bbe0ec64e901f58e297db1c3, also, the patch at
[1]: /messages/by-id/CAEepm=3%25 3DNHHko3oOzpik%2BggLy17AO%2Bpx3rGYrg3x_x05%2BBr9-A%40mail.gmail.com -- Regards, Rafia Sabih EnterpriseDB: http://www.enterprisedb.com/
Query plans for the above mentioned queries is attached.

[1]: /messages/by-id/CAEepm=3%25 3DNHHko3oOzpik%2BggLy17AO%2Bpx3rGYrg3x_x05%2BBr9-A%40mail.gmail.com -- Regards, Rafia Sabih EnterpriseDB: http://www.enterprisedb.com/
3DNHHko3oOzpik%2BggLy17AO%2Bpx3rGYrg3x_x05%2BBr9-A%40mail.gmail.com
--
Regards,
Rafia Sabih
EnterpriseDB: http://www.enterprisedb.com/

Attachments:

part_perf.zipapplication/zip; name=part_perf.zipDownload
#155Robert Haas
robertmhaas@gmail.com
In reply to: Rafia Sabih (#154)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Jul 25, 2017 at 1:31 AM, Rafia Sabih
<rafia.sabih@enterprisedb.com> wrote:

- other queries show a good 20-30% improvement in performance. Performance
numbers are as follows,

Query| un_part_head (seconds) | part_head (seconds) | part_patch (seconds) |
3 | 76 |127 | 88 |
4 |17 | 244 | 41 |
5 | 52 | 123 | 84 |
7 | 73 | 134 | 103 |
10 | 67 | 111 | 89 |
12 | 53 | 114 | 99 |
18 | 447 | 709 | 551 |

Hmm. This certainly shows that benefit of the patch, although it's
rather sad that we're still slower than if we hadn't partitioned the
data in the first place. Why does partitioning hurt performance so
much?

Maybe things would be better at a higher scale factor.

When reporting results of this sort, it would be good to make a habit
of reporting the number of partitions along with the other details you
included.

--
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

#156Dilip Kumar
dilipbalaut@gmail.com
In reply to: Robert Haas (#155)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Jul 25, 2017 at 8:59 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Tue, Jul 25, 2017 at 1:31 AM, Rafia Sabih
<rafia.sabih@enterprisedb.com> wrote:

- other queries show a good 20-30% improvement in performance. Performance
numbers are as follows,

Query| un_part_head (seconds) | part_head (seconds) | part_patch (seconds) |
3 | 76 |127 | 88 |
4 |17 | 244 | 41 |
5 | 52 | 123 | 84 |
7 | 73 | 134 | 103 |
10 | 67 | 111 | 89 |
12 | 53 | 114 | 99 |
18 | 447 | 709 | 551 |

Hmm. This certainly shows that benefit of the patch, although it's
rather sad that we're still slower than if we hadn't partitioned the
data in the first place. Why does partitioning hurt performance so
much?

I was analysing some of the plans (without partition and with
partition), Seems like one of the reasons of performing worse with the
partitioned table is that we can not use an index on the partitioned
table.

Q4 is taking 17s without partition whereas it's taking 244s with partition.

Now if we analyze the plan

Without partition, it can use parameterize index scan on lineitem
table which is really big in size. But with partition, it has to scan
this table completely.

-> Nested Loop Semi Join
-> Parallel Bitmap Heap Scan on orders
-> Bitmap Index Scan on
idx_orders_orderdate (cost=0.00..24378.88 r
-> Index Scan using idx_lineitem_orderkey on
lineitem (cost=0.57..29.29 rows=105 width=8) (actual
time=0.031..0.031 rows=1 loops=1122364)
Index Cond: (l_orderkey =
orders.o_orderkey)
Filter: (l_commitdate < l_receiptdate)
Rows Removed by Filter: 1

--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com

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

#157Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Dilip Kumar (#156)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Jul 25, 2017 at 9:39 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

On Tue, Jul 25, 2017 at 8:59 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Tue, Jul 25, 2017 at 1:31 AM, Rafia Sabih
<rafia.sabih@enterprisedb.com> wrote:

- other queries show a good 20-30% improvement in performance. Performance
numbers are as follows,

Query| un_part_head (seconds) | part_head (seconds) | part_patch (seconds) |
3 | 76 |127 | 88 |
4 |17 | 244 | 41 |
5 | 52 | 123 | 84 |
7 | 73 | 134 | 103 |
10 | 67 | 111 | 89 |
12 | 53 | 114 | 99 |
18 | 447 | 709 | 551 |

Hmm. This certainly shows that benefit of the patch, although it's
rather sad that we're still slower than if we hadn't partitioned the
data in the first place. Why does partitioning hurt performance so
much?

I was analysing some of the plans (without partition and with
partition), Seems like one of the reasons of performing worse with the
partitioned table is that we can not use an index on the partitioned
table.

Q4 is taking 17s without partition whereas it's taking 244s with partition.

Now if we analyze the plan

Without partition, it can use parameterize index scan on lineitem
table which is really big in size. But with partition, it has to scan
this table completely.

-> Nested Loop Semi Join
-> Parallel Bitmap Heap Scan on orders
-> Bitmap Index Scan on
idx_orders_orderdate (cost=0.00..24378.88 r
-> Index Scan using idx_lineitem_orderkey on
lineitem (cost=0.57..29.29 rows=105 width=8) (actual
time=0.031..0.031 rows=1 loops=1122364)
Index Cond: (l_orderkey =
orders.o_orderkey)
Filter: (l_commitdate < l_receiptdate)
Rows Removed by Filter: 1

If the partitions have the same indexes as the unpartitioned table,
planner manages to create parameterized plans for each partition and
thus parameterized plan for the whole partitioned table. Do we have
same indexes on unpartitioned table and each of the partitions? The
difference between the two cases is the parameterized path on an
unpartitioned table scans only one index whereas that on the
partitioned table scans the indexes on all the partitions. My guess is
the planner thinks those many scans are costlier than hash/merge join
and chooses those strategies over parameterized nest loop join. In
case of partition-wise join, only one index on the inner partition is
involved and thus partition-wise join picks up parameterized nest loop
join. Notice, that this index is much smaller than the index on the
partitioned table, so the index scan will be a bit faster. But only a
bit, since the depth of the index doesn't increase linearly with the
size of index.

Rrun-time partition pruning will improve performance even without
partition-wise join since partition pruning will be able to eliminate
all but one partition and only one index needs to be scanned. If
planner is smart enough to cost that effectively, it will choose
parameterized nest loop join for partitioned table thus improving the
performance similar to unpartitioned case.

--
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

#158Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Rafia Sabih (#154)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Jul 25, 2017 at 11:01 AM, Rafia Sabih
<rafia.sabih@enterprisedb.com> wrote:

Query plans for the above mentioned queries is attached.

Can you please share plans for all the queries, even if they haven't
chosen partition-wise join when run on partitioned tables with
enable_partition_wise_join ON? Also, please include the query in
explain analyze output using -a or -e flats to psql. That way we will
have query and its plan in the same file for ready reference.

--
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

#159Rafia Sabih
rafia.sabih@enterprisedb.com
In reply to: Ashutosh Bapat (#158)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Jul 26, 2017 at 10:58 AM, Ashutosh Bapat <
ashutosh.bapat@enterprisedb.com> wrote:

On Tue, Jul 25, 2017 at 11:01 AM, Rafia Sabih
<rafia.sabih@enterprisedb.com> wrote:

Query plans for the above mentioned queries is attached.

Can you please share plans for all the queries, even if they haven't
chosen partition-wise join when run on partitioned tables with
enable_partition_wise_join ON? Also, please include the query in
explain analyze output using -a or -e flats to psql. That way we will
have query and its plan in the same file for ready reference.

I didn't run the query not using partition-wise join, for now.

--
Regards,
Rafia Sabih
EnterpriseDB: http://www.enterprisedb.com/

#160Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Rafia Sabih (#159)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Jul 26, 2017 at 11:00 AM, Rafia Sabih
<rafia.sabih@enterprisedb.com> wrote:

On Wed, Jul 26, 2017 at 10:58 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

On Tue, Jul 25, 2017 at 11:01 AM, Rafia Sabih
<rafia.sabih@enterprisedb.com> wrote:

Query plans for the above mentioned queries is attached.

Can you please share plans for all the queries, even if they haven't
chosen partition-wise join when run on partitioned tables with
enable_partition_wise_join ON? Also, please include the query in
explain analyze output using -a or -e flats to psql. That way we will
have query and its plan in the same file for ready reference.

I didn't run the query not using partition-wise join, for now.

parse-parse error, sorry. Do you mean, you haven't run the queries
which do not use partition-wise join?

--
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

#161Rafia Sabih
rafia.sabih@enterprisedb.com
In reply to: Ashutosh Bapat (#160)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Jul 26, 2017 at 11:06 AM, Ashutosh Bapat <
ashutosh.bapat@enterprisedb.com> wrote:

On Wed, Jul 26, 2017 at 11:00 AM, Rafia Sabih
<rafia.sabih@enterprisedb.com> wrote:

On Wed, Jul 26, 2017 at 10:58 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

On Tue, Jul 25, 2017 at 11:01 AM, Rafia Sabih
<rafia.sabih@enterprisedb.com> wrote:

Query plans for the above mentioned queries is attached.

Can you please share plans for all the queries, even if they haven't
chosen partition-wise join when run on partitioned tables with
enable_partition_wise_join ON? Also, please include the query in
explain analyze output using -a or -e flats to psql. That way we will
have query and its plan in the same file for ready reference.

I didn't run the query not using partition-wise join, for now.

parse-parse error, sorry. Do you mean, you haven't run the queries
which do not use partition-wise join?

Yes, that's what I mean.

--
Regards,
Rafia Sabih
EnterpriseDB: http://www.enterprisedb.com/

#162Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Rafia Sabih (#161)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Jul 26, 2017 at 11:08 AM, Rafia Sabih
<rafia.sabih@enterprisedb.com> wrote:

On Wed, Jul 26, 2017 at 11:06 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

On Wed, Jul 26, 2017 at 11:00 AM, Rafia Sabih
<rafia.sabih@enterprisedb.com> wrote:

On Wed, Jul 26, 2017 at 10:58 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

On Tue, Jul 25, 2017 at 11:01 AM, Rafia Sabih
<rafia.sabih@enterprisedb.com> wrote:

Query plans for the above mentioned queries is attached.

Can you please share plans for all the queries, even if they haven't
chosen partition-wise join when run on partitioned tables with
enable_partition_wise_join ON? Also, please include the query in
explain analyze output using -a or -e flats to psql. That way we will
have query and its plan in the same file for ready reference.

I didn't run the query not using partition-wise join, for now.

parse-parse error, sorry. Do you mean, you haven't run the queries
which do not use partition-wise join?

Yes, that's what I mean.

Ok. If those queries have equi-join between partitioned tables and are
not picking up partition-wise join, that case needs to be
investigated. Q21 for example has join between three lineitem
instances. Those joins can be executed by partition-wise join. But it
may so happen that optimal join order doesn't join partitioned tables
with each other, thus interleaving partitioned tables with
unpartitioned or differently partitioned tables in join order.
Partition-wise join is not possible then. A different partitioning
scheme may be required there.

--
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

#163Rafia Sabih
rafia.sabih@enterprisedb.com
In reply to: Ashutosh Bapat (#157)
2 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Jul 26, 2017 at 10:38 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

On Tue, Jul 25, 2017 at 9:39 PM, Dilip Kumar <dilipbalaut@gmail.com> wrote:

On Tue, Jul 25, 2017 at 8:59 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Tue, Jul 25, 2017 at 1:31 AM, Rafia Sabih
<rafia.sabih@enterprisedb.com> wrote:

- other queries show a good 20-30% improvement in performance. Performance
numbers are as follows,

Query| un_part_head (seconds) | part_head (seconds) | part_patch (seconds) |
3 | 76 |127 | 88 |
4 |17 | 244 | 41 |
5 | 52 | 123 | 84 |
7 | 73 | 134 | 103 |
10 | 67 | 111 | 89 |
12 | 53 | 114 | 99 |
18 | 447 | 709 | 551 |

Hmm. This certainly shows that benefit of the patch, although it's
rather sad that we're still slower than if we hadn't partitioned the
data in the first place. Why does partitioning hurt performance so
much?

I was analysing some of the plans (without partition and with
partition), Seems like one of the reasons of performing worse with the
partitioned table is that we can not use an index on the partitioned
table.

Q4 is taking 17s without partition whereas it's taking 244s with partition.

Now if we analyze the plan

Without partition, it can use parameterize index scan on lineitem
table which is really big in size. But with partition, it has to scan
this table completely.

-> Nested Loop Semi Join
-> Parallel Bitmap Heap Scan on orders
-> Bitmap Index Scan on
idx_orders_orderdate (cost=0.00..24378.88 r
-> Index Scan using idx_lineitem_orderkey on
lineitem (cost=0.57..29.29 rows=105 width=8) (actual
time=0.031..0.031 rows=1 loops=1122364)
Index Cond: (l_orderkey =
orders.o_orderkey)
Filter: (l_commitdate < l_receiptdate)
Rows Removed by Filter: 1

If the partitions have the same indexes as the unpartitioned table,
planner manages to create parameterized plans for each partition and
thus parameterized plan for the whole partitioned table. Do we have
same indexes on unpartitioned table and each of the partitions? The

Yes both lineitem and orders have same number of partitions viz 17 and
on the same partitioning key (*_orderkey) and same ranges for each
partition. However, I missed creating the index on o_orderdate for the
partitions. But on creating it as well, the plan with bitmap heap scan
is used and it still completes in some 200 seconds, check the attached
file for the query plan.

difference between the two cases is the parameterized path on an
unpartitioned table scans only one index whereas that on the
partitioned table scans the indexes on all the partitions. My guess is
the planner thinks those many scans are costlier than hash/merge join
and chooses those strategies over parameterized nest loop join. In
case of partition-wise join, only one index on the inner partition is
involved and thus partition-wise join picks up parameterized nest loop
join. Notice, that this index is much smaller than the index on the
partitioned table, so the index scan will be a bit faster. But only a
bit, since the depth of the index doesn't increase linearly with the
size of index.

As I have observed, the thing with this query is that selectivity
estimation is too high than actual, now when index scan is chosen for
lineitem being in the inner side of NLJ, the query completes quickly
since the number of actual returned rows is too low. However, in case
we pick seq scan, or lineitem is on the outer side, the query is going
to take a really long time. Now, when Hash-Join is picked in the case
of partitioned database and no partition-wise join is available, seq
scan is preferred instead of index scan and hence the elongated query
execution time.

I tried this query with random_page_cost = 0 and forcing NLJ and the
chosen plan completes the query in 45 seconds, check the attached file
for explain analyse output.

--
Regards,
Rafia Sabih
EnterpriseDB: http://www.enterprisedb.com/

Attachments:

q4_idx_orderdate.outapplication/octet-stream; name=q4_idx_orderdate.outDownload
Q4_low_random_page_cost.outapplication/octet-stream; name=Q4_low_random_page_cost.outDownload
#164Rafia Sabih
rafia.sabih@enterprisedb.com
In reply to: Ashutosh Bapat (#162)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Jul 26, 2017 at 12:02 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Ok. If those queries have equi-join between partitioned tables and are
not picking up partition-wise join, that case needs to be
investigated. Q21 for example has join between three lineitem
instances. Those joins can be executed by partition-wise join. But it
may so happen that optimal join order doesn't join partitioned tables
with each other, thus interleaving partitioned tables with
unpartitioned or differently partitioned tables in join order.
Partition-wise join is not possible then. A different partitioning
scheme may be required there.

Good point, will look into this direction as well.

--
Regards,
Rafia Sabih
EnterpriseDB: http://www.enterprisedb.com/

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

#165Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#140)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Jul 14, 2017 at 3:02 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Here's revised patch set with only 0004 revised. That patch deals with
creating multi-level inheritance hierarchy from multi-level partition
hierarchy. The original logic of recursively calling
inheritance_planner()'s guts over the inheritance hierarchy required
that for every such recursion we flatten many lists created by that
code. Recursion also meant that root->append_rel_list is traversed as
many times as the number of partitioned partitions in the hierarchy.
Instead the revised version keep the iterative shape of
inheritance_planner() intact, thus naturally creating flat lists,
iterates over root->append_rel_list only once and is still easy to
read and maintain.

0001-0003 look basically OK to me, modulo some cosmetic stuff. Regarding 0004:

+        if (brel->reloptkind != RELOPT_BASEREL &&
+            brte->relkind != RELKIND_PARTITIONED_TABLE)

I spent a lot of time staring at this code before I figured out what
was going on here. We're iterating over simple_rel_array, so the
reloptkind must be RELOPT_OTHER_MEMBER_REL if it isn't RELOPT_BASEREL.
But does that guarantee that rtekind is RTE_RELATION such that
brte->relkind will be initialized to a value? I'm not 100% sure. I
think it would be clearer to write this test like this:

Assert(IS_SIMPLE_REL(brel));
if (brel->reloptkind == RELOPT_OTHER_MEMBER_REL &&
(brte->rtekind != RELOPT_BASEREL ||
brte->relkind != RELKIND_PARTITIONED_TABLE))
continue;

Note that the way you wrote the comment is says if it *is* another
REL, not if it's *not* a baserel; it's good if those kinds of little
details match between the code and the comments.

It is not clear to me what the motivation is for the API changes in
expanded_inherited_rtentry. They don't appear to be necessary. If
they are necessary, you need to do a more thorough job updating the
comments. This one, in particular:

* If so, add entries for all the child tables to the query's
* rangetable, and build AppendRelInfo nodes for all the child tables
* and add them to root->append_rel_list. If not, clear the entry's

And the comments could maybe say something like "We return the list of
appinfos rather than directly appending it to append_rel_list because
$REASON."

-         * is a partitioned table.
+         * RTE simply duplicates the parent *partitioned* table.
          */
-        if (childrte->relkind != RELKIND_PARTITIONED_TABLE)
+        if (childrte->relkind != RELKIND_PARTITIONED_TABLE || childrte->inh)

This is another case where it's hard to understand the test from the comments.

+     * In case of multi-level inheritance hierarchy, for every child we require
+     * PlannerInfo of its immediate parent. Hence we save those in a an array

Say why. Also, need to fix "a an".

I'm a little bit surprised that this patch doesn't make any changes to
allpaths.c or relnode.c. It looks to me like we'll generate paths for
the new RTEs that are being added. Are we going to end up with
multiple levels of Append nodes, then? Does the consider the way
consider_parallel is propagated up and down in set_append_rel_size()
and set_append_rel_pathlist() really work with multiple levels? Maybe
this is all fine; I haven't tried to verify it in depth.

Overall I think this is a reasonable direction to go but I'm worried
that there may be bugs lurking -- other code that needs adjusting that
hasn't been found, really.

--
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

#166Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#165)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Mon, Jul 31, 2017 at 8:32 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Fri, Jul 14, 2017 at 3:02 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Here's revised patch set with only 0004 revised. That patch deals with
creating multi-level inheritance hierarchy from multi-level partition
hierarchy. The original logic of recursively calling
inheritance_planner()'s guts over the inheritance hierarchy required
that for every such recursion we flatten many lists created by that
code. Recursion also meant that root->append_rel_list is traversed as
many times as the number of partitioned partitions in the hierarchy.
Instead the revised version keep the iterative shape of
inheritance_planner() intact, thus naturally creating flat lists,
iterates over root->append_rel_list only once and is still easy to
read and maintain.

0001-0003 look basically OK to me, modulo some cosmetic stuff. Regarding 0004:

+        if (brel->reloptkind != RELOPT_BASEREL &&
+            brte->relkind != RELKIND_PARTITIONED_TABLE)

I spent a lot of time staring at this code before I figured out what
was going on here. We're iterating over simple_rel_array, so the
reloptkind must be RELOPT_OTHER_MEMBER_REL if it isn't RELOPT_BASEREL.
But does that guarantee that rtekind is RTE_RELATION such that
brte->relkind will be initialized to a value? I'm not 100% sure.

Comment in RangeTblEntry says
952 /*
953 * Fields valid for a plain relation RTE (else zero):
954 *
... clipped portion for RTE_NAMEDTUPLESTORE related comment

960 Oid relid; /* OID of the relation */
961 char relkind; /* relation kind (see pg_class.relkind) */

This means that relkind will be 0 when rtekind != RTE_RELATION. So,
the condition holds. But code creating an RTE somewhere which is not
in sync with this comment would create a problem. So your suggestion
makes sense.

I
think it would be clearer to write this test like this:

Assert(IS_SIMPLE_REL(brel));
if (brel->reloptkind == RELOPT_OTHER_MEMBER_REL &&
(brte->rtekind != RELOPT_BASEREL ||

Do you mean (brte_>rtekind != RTE_RELATION)?

brte->relkind != RELKIND_PARTITIONED_TABLE))
continue;

Note that the way you wrote the comment is says if it *is* another
REL, not if it's *not* a baserel; it's good if those kinds of little
details match between the code and the comments.

I find the existing comment and code in this part of the function
differ. The comment just above the loop on simple_rel_array[], talks
about changing something in the child, but the very next line skips
child relations and later a loop on append_rel_list makes changes to
appropriate children. I guess, it's done that way to keep the code
working even after we introduce some RELOPTKIND other than BASEREL or
OTHER_MEMBER_REL for a simple rel. But your suggestion makes more
sense. Changed it according to your suggestion.

It is not clear to me what the motivation is for the API changes in
expanded_inherited_rtentry. They don't appear to be necessary.

expand_inherited_rtentry() creates AppendRelInfos for all the children
of a given parent and collects them in a list. The list is appended to
root->append_rel_list at the end of the function. Now that function
needs to do this recursively. This means that for a partitioned
partition table its children's AppendRelInfos will be added to
root->append_rel_list before AppendRelInfo of that partitioned
partition table. inheritance_planner() assumes that the parent's
AppendRelInfo comes before its children in append_rel_list.This
assumption allows it to be simplified greately, retaining its
iterative form. My earlier patches had recursive version of
inheritance_planner(), which is complex. I have comments in this patch
explaining this.

Adding AppendRelInfos to root->append_rel_list as and when they are
created would keep parent AppendRelInfos before those of children. But
that function throws away the AppendRelInfo it created when their are
no real children i.e. in partitioned table's case when has no leaf
partitions. So, we can't do that. Hence, I chose to change the API to
return the list of AppendRelInfos when the given RTE has real
children.

If
they are necessary, you need to do a more thorough job updating the
comments. This one, in particular:

* If so, add entries for all the child tables to the query's
* rangetable, and build AppendRelInfo nodes for all the child tables
* and add them to root->append_rel_list. If not, clear the entry's

Done.

And the comments could maybe say something like "We return the list of
appinfos rather than directly appending it to append_rel_list because
$REASON."

Done. Please check the attached version.

-         * is a partitioned table.
+         * RTE simply duplicates the parent *partitioned* table.
*/
-        if (childrte->relkind != RELKIND_PARTITIONED_TABLE)
+        if (childrte->relkind != RELKIND_PARTITIONED_TABLE || childrte->inh)

This is another case where it's hard to understand the test from the comments.

The current comment says it all, but it very cryptic manner.
1526 /*
1527 * Build an AppendRelInfo for this parent and child,
unless the child
1528 * RTE simply duplicates the parent *partitioned* table.
1529 */

The comment makes sense in the context of this paragraph in the prologue
1364 * Note that the original RTE is considered to represent the whole
1365 * inheritance set. The first of the generated RTEs is an RTE for the same
1366 * table, but with inh = false, to represent the parent table in its role
1367 * as a simple member of the inheritance set.
1368 *

The code avoids creating AppendRelInfos for a child which represents
the parent in its role as a simple member of inheritance set.

I have reworded it as
1526 /*
1527 * Build an AppendRelInfo for this parent and child,
unless the child
1528 * RTE represents the parent as a simple member of inheritance set.
1529 */

+     * In case of multi-level inheritance hierarchy, for every child we require
+     * PlannerInfo of its immediate parent. Hence we save those in a an array

Say why. Also, need to fix "a an".

Done.

I'm a little bit surprised that this patch doesn't make any changes to
allpaths.c or relnode.c.

It looks to me like we'll generate paths for
the new RTEs that are being added. Are we going to end up with
multiple levels of Append nodes, then? Does the consider the way
consider_parallel is propagated up and down in set_append_rel_size()
and set_append_rel_pathlist() really work with multiple levels? Maybe
this is all fine; I haven't tried to verify it in depth.

This has been discussed before, but I can not locate the mail
answering these questions. accumulate_append_subpath() called from
add_paths_to_append_rel() takes care of flattening Merge/Append paths.
The planner code deals with the multi-level inheritance hierarchy
created for subqueries with set operations. There is code in relnode.c
to build the RelOptInfos for such subqueries recursively through using
RangeTblEntry::inh flag. So there are no changes in allpaths.c and
relnode.c. Are you looking for some other changes?

Overall I think this is a reasonable direction to go but I'm worried
that there may be bugs lurking -- other code that needs adjusting that
hasn't been found, really.

Planner code is already aware of such hierarchies except DMLs, which
this patch adjusts. We have fixed issues revealed by mine and
Rajkumar's testing.
What kinds of things you suspect?

--
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

#167Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Ashutosh Bapat (#166)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

Forgot the patch set. Here it is.

On Mon, Jul 31, 2017 at 5:29 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

On Mon, Jul 31, 2017 at 8:32 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Fri, Jul 14, 2017 at 3:02 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Here's revised patch set with only 0004 revised. That patch deals with
creating multi-level inheritance hierarchy from multi-level partition
hierarchy. The original logic of recursively calling
inheritance_planner()'s guts over the inheritance hierarchy required
that for every such recursion we flatten many lists created by that
code. Recursion also meant that root->append_rel_list is traversed as
many times as the number of partitioned partitions in the hierarchy.
Instead the revised version keep the iterative shape of
inheritance_planner() intact, thus naturally creating flat lists,
iterates over root->append_rel_list only once and is still easy to
read and maintain.

0001-0003 look basically OK to me, modulo some cosmetic stuff. Regarding 0004:

+        if (brel->reloptkind != RELOPT_BASEREL &&
+            brte->relkind != RELKIND_PARTITIONED_TABLE)

I spent a lot of time staring at this code before I figured out what
was going on here. We're iterating over simple_rel_array, so the
reloptkind must be RELOPT_OTHER_MEMBER_REL if it isn't RELOPT_BASEREL.
But does that guarantee that rtekind is RTE_RELATION such that
brte->relkind will be initialized to a value? I'm not 100% sure.

Comment in RangeTblEntry says
952 /*
953 * Fields valid for a plain relation RTE (else zero):
954 *
... clipped portion for RTE_NAMEDTUPLESTORE related comment

960 Oid relid; /* OID of the relation */
961 char relkind; /* relation kind (see pg_class.relkind) */

This means that relkind will be 0 when rtekind != RTE_RELATION. So,
the condition holds. But code creating an RTE somewhere which is not
in sync with this comment would create a problem. So your suggestion
makes sense.

I
think it would be clearer to write this test like this:

Assert(IS_SIMPLE_REL(brel));
if (brel->reloptkind == RELOPT_OTHER_MEMBER_REL &&
(brte->rtekind != RELOPT_BASEREL ||

Do you mean (brte_>rtekind != RTE_RELATION)?

brte->relkind != RELKIND_PARTITIONED_TABLE))
continue;

Note that the way you wrote the comment is says if it *is* another
REL, not if it's *not* a baserel; it's good if those kinds of little
details match between the code and the comments.

I find the existing comment and code in this part of the function
differ. The comment just above the loop on simple_rel_array[], talks
about changing something in the child, but the very next line skips
child relations and later a loop on append_rel_list makes changes to
appropriate children. I guess, it's done that way to keep the code
working even after we introduce some RELOPTKIND other than BASEREL or
OTHER_MEMBER_REL for a simple rel. But your suggestion makes more
sense. Changed it according to your suggestion.

It is not clear to me what the motivation is for the API changes in
expanded_inherited_rtentry. They don't appear to be necessary.

expand_inherited_rtentry() creates AppendRelInfos for all the children
of a given parent and collects them in a list. The list is appended to
root->append_rel_list at the end of the function. Now that function
needs to do this recursively. This means that for a partitioned
partition table its children's AppendRelInfos will be added to
root->append_rel_list before AppendRelInfo of that partitioned
partition table. inheritance_planner() assumes that the parent's
AppendRelInfo comes before its children in append_rel_list.This
assumption allows it to be simplified greately, retaining its
iterative form. My earlier patches had recursive version of
inheritance_planner(), which is complex. I have comments in this patch
explaining this.

Adding AppendRelInfos to root->append_rel_list as and when they are
created would keep parent AppendRelInfos before those of children. But
that function throws away the AppendRelInfo it created when their are
no real children i.e. in partitioned table's case when has no leaf
partitions. So, we can't do that. Hence, I chose to change the API to
return the list of AppendRelInfos when the given RTE has real
children.

If
they are necessary, you need to do a more thorough job updating the
comments. This one, in particular:

* If so, add entries for all the child tables to the query's
* rangetable, and build AppendRelInfo nodes for all the child tables
* and add them to root->append_rel_list. If not, clear the entry's

Done.

And the comments could maybe say something like "We return the list of
appinfos rather than directly appending it to append_rel_list because
$REASON."

Done. Please check the attached version.

-         * is a partitioned table.
+         * RTE simply duplicates the parent *partitioned* table.
*/
-        if (childrte->relkind != RELKIND_PARTITIONED_TABLE)
+        if (childrte->relkind != RELKIND_PARTITIONED_TABLE || childrte->inh)

This is another case where it's hard to understand the test from the comments.

The current comment says it all, but it very cryptic manner.
1526 /*
1527 * Build an AppendRelInfo for this parent and child,
unless the child
1528 * RTE simply duplicates the parent *partitioned* table.
1529 */

The comment makes sense in the context of this paragraph in the prologue
1364 * Note that the original RTE is considered to represent the whole
1365 * inheritance set. The first of the generated RTEs is an RTE for the same
1366 * table, but with inh = false, to represent the parent table in its role
1367 * as a simple member of the inheritance set.
1368 *

The code avoids creating AppendRelInfos for a child which represents
the parent in its role as a simple member of inheritance set.

I have reworded it as
1526 /*
1527 * Build an AppendRelInfo for this parent and child,
unless the child
1528 * RTE represents the parent as a simple member of inheritance set.
1529 */

+     * In case of multi-level inheritance hierarchy, for every child we require
+     * PlannerInfo of its immediate parent. Hence we save those in a an array

Say why. Also, need to fix "a an".

Done.

I'm a little bit surprised that this patch doesn't make any changes to
allpaths.c or relnode.c.

It looks to me like we'll generate paths for
the new RTEs that are being added. Are we going to end up with
multiple levels of Append nodes, then? Does the consider the way
consider_parallel is propagated up and down in set_append_rel_size()
and set_append_rel_pathlist() really work with multiple levels? Maybe
this is all fine; I haven't tried to verify it in depth.

This has been discussed before, but I can not locate the mail
answering these questions. accumulate_append_subpath() called from
add_paths_to_append_rel() takes care of flattening Merge/Append paths.
The planner code deals with the multi-level inheritance hierarchy
created for subqueries with set operations. There is code in relnode.c
to build the RelOptInfos for such subqueries recursively through using
RangeTblEntry::inh flag. So there are no changes in allpaths.c and
relnode.c. Are you looking for some other changes?

Overall I think this is a reasonable direction to go but I'm worried
that there may be bugs lurking -- other code that needs adjusting that
hasn't been found, really.

Planner code is already aware of such hierarchies except DMLs, which
this patch adjusts. We have fixed issues revealed by mine and
Rajkumar's testing.
What kinds of things you suspect?

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

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

Attachments:

pg_dp_join_patches_v23.tar.gzapplication/x-gzip; name=pg_dp_join_patches_v23.tar.gzDownload
�m*Y�]}w������S��s�-l�`�&��i6!�������O�="q0���9����H��e,�&���,��h�7��H���t5��4����Ln�����SS����J���~��+ES����Q��W-��������)$�vt�+��q���*�w���UZ�T:�����7����6&�b�[�m����]N�b7��O&d�4d1&��d5�����og�_�*ZB�Z�K��z���f�P[}LZ�����Lo����i�{����k����6�Lg�`.a@V�t��:d�pK��j�����
������^G��V�#27So+��Cax��a~p�XV�������n���*w��8���O���������?�
�A(��P�Zs?��c�?��J�,(���G���d�����l+��rZI������X,��5,2��,�uM��8�veF@��X^c����i��n�9��I�m�w���7�-a��Y�R�v�v��kE�`�1�e�*�r~���,�2��@�b=�Z&:v�T,��xafZ���z��f��G��h4*`[���>�D������s�z/`Rs@wv���LP`F7����dZ�v��M,Feu��0%s�3�J�2�F��p@���g�W�b,��wh�������lO�-�Ppl�i��[�`�$Z��;h(�~]��.��f����0l^]�%�P�F!V?������A1R{S���X �>e�;�]��'22�dd�L��(>��:2���zvh��`ja�	�������� >�jY��J�W{J+��U�	���
J�'��������::V�)�<����G�5���$�P:���|�P�K�q�����u�����f���*$V���2�����sq������p0�:�3Q�t$�`��P���Q��H�|�CRb����*�Yp��i��x�:�2�[���	[�������6���&NnMV�e��'q�}d��(�����c����F�������YW9��������X����hI�F���8�[(F2_��6���^��
�� V36iY�v��o)*��t�S���,V�=���<n����8�1��i��
�y��Wp��[�
�������[@�.��\��N�L
�A��������R���
������I�O�'��XdeZN���_���T�eM�)��'W�����������w?�N������������>�����~�F�x&�)��VhAQAy2c���y �� zv������������ug�~�������9p$Td&��#bf��
��x�M���#�dsQ8D��YI�	��?U����?�#B����!�����{���*j���]���A���@	a�6���cz��c|��k�^���,���
o��^�`�:�����<�\s�X�q��xF��P�a�~��F�1���lZ��Hv�B�G������H�r�����d���sg��q��n�GC���I���4#w��V"��v����+��YZ�?<V�=Vw<VK�9��9�c����������o�h��U
�o�58�=����sV�X�p}GA8Y���]N�����;�d	�����b	�
�����ww7���6��d�1	����I�~a����g�O���k������N��\=A<T����_��$s���
�i&�Bl��t���!4���wj|������{���N�hT���U��Q�(�o�p���o#�'�`8Tc����?�k���:t�\����
�D[Cy����(D8���t�2[o�?���7:=~�0��L��=nx3#	���M@����+R�L
�"�~�"�j���P��N]���
b>�������y����rA�Z������Kj-��E����L��7T�_E�o����7�~���g�F��#���mC�����{����H��mO���Q���<�}����|0<d��@�ZI�jU�S�rj�J�UR�ZI���_y�_U�!����i�o-p���.��q�]W�q�*}�\���������� ��
]U����F�]�%�V0���w�WQ�l� 1!�M����@�����<���:���%��� ]\��;;B_"H�'�$��1�H���(�=H:
���3%3��;��M27��|�.��8388/�&�?��?�_{H:� >Fy���h���dN�?��A�c

S�cM��@N�2_SA �[���	�pc�PH
�@}�=$bC��!�,�z�v�a��vC�	"�������h@Eiv��N��W�k�$�$�������j�p��0/��6h�5��4�C���a����8�������v���:����t���������^OZ���'��>�?P52%�tez����_�n�fh�<���t$R��~�m�v��Z'k�>�� 
D���8D���C@�b�����������\ rU�Q�%!,�D��,rk�;2m������)���xq�|���.���~��W<���;m����
,��o�0h��{JE��{F��~M����.��H4Y��-a�{Q2��2��@}���El�������] �����{�/m������XbW�;�l��pkF����*����p�4���:@��jK����|�%9��B���G6<SwN ���@K�!x��<���xO���������������$��(���3�_xZ�S�F��|�����;�M*����%}I�����������TU'�����)��[��ydK�6�����`W�s8}�]e�������M4=���������l����������[f�����9�E0&n�z��]�u����o�d���8��I:����+w<7&oh�GO5Q�N{?��7����7�9�Q������%��!G0D"�q4!��)��M���}�]����q�8������6�sw�����i�x�[����r=���?����:8nR��M�s��|�#������+b��L:�N�j�I�[$�N	����b� �!�%�[+�J�#s�&�r�8�"Z����_���N���a���.zhn;
Tu?B��L���K�j�D��/�b�U��J�:F+%R���W�� ����W�������vZ�h2�tZ^=���zT�z(�����r��������E�dF��9��2��_�9�@��oj��`������:tp�����__�����%�-���r�7\�N���U(C:E*%���RR�PL��&���	}�0q�6�>"y7��hV$����|��=���d�ZBc��T���RE������4�$�����)�4���~q	�)��)���)�B��G������n��?]�_B��^����WG�u0 ��jtd58JS�����6���$D&��s������%��#V�/O��?@���$����Lq�������U�������H�ef�Jz�p��mL�a$\���E'D�iV����=/��o�W+��F�A3��s?0^'H�h(�XE'�rzU��#�d-U��: �a@m$���7S���;j�������WC����k�d�����D��x:3���"��k������'�����O&����5�-�$G�(.;���+��IK0���RT���r�h���,�CJ������~!	i�z8*��k%_����f��%����e���O1i&u����2�l������`����k�'v�:�����`Q
��(v�?�����-�^��6/�?�dL1��c��g�������l���������
-�(?#�2M~[���_���A���� )��CR�>cHN�/&����$��t�e�����;�>j��^i�s3���W��U^D���mXh��/��E���3������wx�N�������"u_D�tEJCl�}+���{)��_��� e������M�>��Fz&mn

��V�d�h-h��o��&�P���4��qL���5��k�o`�o�*O�i�W����y�G����E�<������J�FgA�5����g�o�6OQ40�d�x���u9k0�,m0l�`-��u��t?��
��)����P��K�����r���[D���a�t;�;����G�������t�3�L�u9�9"���E%D�F�����k�V��"<!��'�F%|����8��>o�CA��zv��g�QZ�������r���4������3*��!��ESt���I�2�
r�j����DH����O@>c$��f��t��~�Gs�� QC��!H����dPM���k8K�$N�$H2��q�I�$�1}U��Z�AA���[��]A���iR �" ���D�I�ZI�h[Ic$��f��t��~�G�v�����l������������X��s��N�b���A������-��:��"�WZ���A��a���+�M���b���.�����|�/=�o�#�l�&}����
��9[���e���z�qY�x����V����Q9�M/Ce/��sr�B��J�8
�d#fw;�X
���_��A�%��5�e��;�u�3qz�NbU!�R,�g�_������X��,����fPP+�2>l���������L9��V)O��$��D���-I��I����Zl'U;�?�Ru#u��wX����]�	@M�������<A�s��
Oa���c��^���jkE����U4�������Z��2�	b����]�
���Z`��C�l��9X�F	��z&2$dZ&�$���9S�2K)��0!�e���`�g����,��HLB)��E,@��s}���2n�L_
0���y�`c���	��^�;XoV?o+/j>S�c+&<+{�$��ZCa��xV'�����.2@z}%�Tt����g	�^�����d�����R��Nz�&����Mf��:/���d�_��*������rL�
W���
W���
W��R�K��T���+���y�/����_�:6��+�+�]FO��A���|��LP����e���&��C!������I,�>���W��_
L�P���n�>���5���q��U:v���jVj�N����B��0'�<��>���bo`go��,����^����"-�F^�����W������z�O�-+����`�x��[��M�S�
��H2�+�$��� M3����`��8iD>4�������
e�W��g���Qy�m6��8�����Q^F��4�;���
���j�/���f�>��jNhPf�t��/�{8�=��y�v�T�c5A�5��Z�_oAI��I����,�������VOK������*�����:jq�/]-�$���1�
�	AO�����4=�*�;���>��v�j9�
��������^����n�B�����`�
���oe{�h#<����n1��b��[XL�K�����W��r�{���+El^e��)�,O��&���������d"1Z��Y!�h��$X���:m�������q2XU��'8\�����c�)��	��&T!\�`<=`48�-�F���^�V������B�-A��l��g��5�D�X�7�����Q���[rG��h����)�R��9�W� �d~�A�&��4�{	�wX��Qr�p�����H;�<�6)!��\e�.u�k'�2�	�#�v�& �W>um9i��*-%8.v�yllJZ�FuJ�������i�l�[����"�u���Im�WDv�cO�:S!i#�9�o*8�4������$����{N�JGt�s���X�����a��H��zq<���/*�;;;���Z���Z�y^��n;l��Buz��\��v��m����+���S�nbK����K|<^������6����	�����`X
,x�F�V
��)��J��SN����LI�J.�����&O��[He�^d!9�d�dm=�
���,�P�$3��,�6����9�VsB5�1����"�����)�sdnj+�J"�������(���]Z���*E��������[T��|Z�����&�*�M�$�9��V�'3��p*��x����)�2�'�>O�2C&���+a�����U����K�`��s���p�Bz�`�Ok�TDU_suW�����J������d��"3��l�'A���I���|� �&�W�I���5���'�5Y"�3����g����a�^�OKx�����
,���k��~A�?^,�C���],�k�K�u�D���\xo�B��a�����������c�DKN����h�S�E^t�I��3{���*��z��QCYy��#G�����^B���$�(k���m�,7�&n�B��P�a���K%{�Av� WL]��(�%F��[����
{}iZ7����
���r��~��2<^�����M_t�ER����00tV����q�%N�=T���'b�}��X:u�����nMn��g���������������L���������i�r�/�v�,����.���)
_��{}����V�w$P(����I�^��2�H��������\���b����?^�z��O����������������5+�������/?\���'���;
���S�	;����5�U��BY��NI^o�����������w��1����1j�^����?�������e��l���\��u����^��g�q6�2>{~���b�)�f�\�5�
r�����%��<_g|�W��lUN��z�\������M����p��%;������r�kY=�\{����l�N��z�wR�niKT��*|���]��3��b����r=������M����r=�i�r0Z���
��r��r
��F����r���D	mX�����?�F{<��w���BMb�=%*Nu%RX��K�
�Y�4W�Cr��v��'tF�g4�~��*�i���Omi���@�}�����@ra��2���^"I�����������m�)����������:HCM4��{�I�����.\%��\������'lJ���F�X�m}JQ�p^(l�����D3��� 6�������"���U2IV��W��{����b�!��]����=dl�&��6�)r�Q��udy[������A@��$>���I4t5�
a�DG�<>q7�sd�'8D�  �>L�Q��0��cV�U��%[]_oSg^gS���g�M~�H@t���N��O���0����g��v����9�e�����^S����8&-���k����W.�/���$H�_�-���s�c�F%8������$r_�9+�^�z�*�.����E�>�HO��[1hK-�x�-SR)us�_m��<*5$������
�����I�����E�#�h1?F�������s�����a\$I��g����!��:�,t_�O5ZL|F�����D�ol*�IA�Dm[�����I!g�B.N
���`�r��������B�$I��T�v�n�v������F�I�k�h�`�~t�H�"�������*�I�#����b�e�ol���h��X���������+{��I�-��y���J�������������������S���>�i��aoI��'�>�)a=�m����@��|�� >�;�>R���	����H�JA����0 ��v��N		������}T7�j0��f_�g�C
����+�o{B�M��kX�>����_�O%��F{����	�}�u������� ��p�58�oi]o�giW�0���2M<��{m�J#����}EC������v��J����&44��	
}Hh(."��	
���P
��HhhBC���>44��yX����4���p�<�M�\D������4�E�l�<�xM�����i����#�����7?�-��V$8��������<q���/��M,���fZ,IB����"��U_�/�WN`0����PzT��)(Et�B��p��,^���yN���tJhMI��0���A�A��"<��`���D�}��m
�wFI��a����{��@��0|-S���yag��q��8�@�4q4r���1qx���`�����m
�wFI�����) �;������F��w�Sn�9��!����N!Uq��\�����>bL�gA.���Z���D'�S��Ghh�$i��.0�=wM�j$LUv��]Z����(����`�;�mOIZ��+L� \G�*�'�*;�B�.-r��w�yrq0��y��'�$-���z.�	S������������@��J9�P)����xV@��
���Y=�?+"B�g;U%��UMc(���C���=��z:,����8F�����K2�����=4���r������5'4r��H��P��'�+�n]K�a�gDM����cf�������o
�fk~a4��)��b!,gkvn���rsI*�$AP�����*���Q���������ebu�\���ih}����Sgd������:����F�h���2�@��#������h���HIB��l�m�N�bMq����Z�C��m���'��y3����s~"��Z�|�`8z]�0���D�D`ZMd��zNd97�Y0��x���{"k0�9����Md
��d��'2oBH�'����,d�5�9`���,����
�o"+�aP4�h���'���U��
2�.S��
a"s��[Md��zNd7�m'2�y�V����aO����<�)�j#���	!I����&��!��0+Ld����Nd2��N��`���j�iM���U�}6�=����C��9~�|Y�+H~�+��o������@�����B���m9�l�Yd���4��i��\�i�����d]��������v�lYi������w�r���.��|����n����
������/�_�����6��u�}��6��#��Q�lS~�5\�}�����/iB�.JSZ�5��P��jQ���2�����x�#n�j��X<��]��}��5W'v1�Z���{0�=��n���7:9�Xg�c�m��c���Ps��$ug��[��	����Q��n���l���q�%�������X�Z�r����V���to�5g�6�tfS;�1�2j�rvaE����g�����y���V����3`n�
�u:��<��a.h��E.����b��.���8vku�H]��g��F6��H��h(��Pn��p,zA3��Ly�f�VV�3�.�wf�8v��P5�Y��P�@�Q|���'r>�@%�@��O#|����K�w�O�A^e�5C0V���%��k$��b��Q��|��g�g��i����?�T��I����\�m(�L��:��)6�#���?�+��T�=s�k��i��Mb�Q�a�[��)LEnSc�
0�.�-���cvy{{��K/�Y��z�Y���0��e-���	")�7��L��p��=��3�-�+(�	8���V�s�
cN�Ih�*�	P&I��;F2dpQK���%�P`�U�a�`��U�$M�>P�6,�uH�	},�I.�Zhh4P���q�]T��a�A�;�7H\e�$0�0� i"tL��:�AR���Ab
!�I��}���:����������5D�h��+�cZ���$���w>e�(�$�	F� �����~J��mLp�}�4{���#�)U�Q��R�<CK�l�����g���[5�o0�9"?���>��(b���j�m����MIo
!�����2Ty�O�j|
Z�0�=��O)�����Z*" �w�s�@?S�l�@h�$	���#�J��Ry��!���Rt1�f���s�v���>�;b������*�-�$�#������_y����9��9t��������mG@E)'�#�]{�LI���1��I8��l���l�X>t������^��e�jF����Oe��Xtw�d��q��d��|�)�/?�_�5j���\�m'����D/e���������e������WW��U��p�������	��Gorlx.������s��9$���9��75=������
����`��i^��.�z��S�������r�[��V!�u������E��]��\�R�W�r%�\�$W2�U.h�)5��$�t��&������+�� �<�	O����.�<!�y�#����Z	�u�Ro��XT
IZ��;�$�!#�����JhI���~�|vz(�<z1��y�O*1T6��|�$�����`�J���Y�$
DmJ��j2�G<�F��������g���k�+h�k3�G:�F��uG�d��������J�
��[�u/g��M��X	\H���ak�'[��5�B���P�u|f=�6��Z��S��
Z#�W$��3�=�����,����!��9e(s
%�,t��V�HV��&�{9N�o:�������z����$y
(��xG^T�w�p��$	kD[
p`�����A:���f�Su,��Su���d�4���4�X3DC��7�2zE�sU����<o�s:o�����<j���S�������U�bA�R;����\���Y�9�
r��7�[`xe�6����Y_�7�����������:8�te65�&�C���W�d�)��,����"4��m�����!vg�n�������on.����7m�����q�Ov�d�MM���)��c���s���`�m?����u�)��V-t����[/oxe�6���8EDrt�:�����>E(^�gZ�� ���5~H���{����`���*y�&�Z���(C6Z��b�m�]���\T�Z�$��k-NrM�wm�J�w����1<���U�o���?�c1��������#���Y]>�x�v����w-�Z����VGf�f��6y�f|8l=p(��n����\�����d$4�w����5�V��8��z�u�]��$�w���Y�Y��M����@�F���n����]#:����dl"4�w�>A�{��N@~�' Gu�mP�omBi=�!T@Z��\�e�����()�l��Y2�I$�p4{�F1�x�y�B����vSc��3:��
���MI[E.�[�)^�Qm]���5&�H%@j�����D��{�)1O�y*��@�["hJ3.�&�\?)x���a�r���*x������/q*<aQ�Q�������
�&�+���!��{�/!B�������W�uo���w������k/r�������4���i??J^�at��h�����{��l���'�i�w���^��y�<`�|(�f���u���9���z�����>e'��q��m���PL��Y��`�=�GW����w��7���N�5�
�����8�y���N���C�v������Cj��7WN���p]��\���($	�b��}�X]ZX����l�����k����it1����\Z��������������z��:lx�����aD�����.�j�r�h��b��5�j������7�2yl�L�F�4��j1��=��j1�,�UyS����\�s��&�����5����|^qU	�oF����������?��OY�U#	��}�=���(��r�����G���c>��_�j_&<z1������f\���U%	��n�0bc�}o��������;I"6�2����s?�]^���������U��C�8����zd��	s/��v�v[+�l^s^���ic���3���"%J�i�0X��jSC�!�Q+~�O�{�~apc���}c�i�n%V�?��2�CfLN�����V��~����i��I��8��<5�A2��~��YI7C#z ������z����(��}���5�k.�����Y}�:�A��rj�N�
�1U��?��W"�D�QZD��Fg�?�Sk
����h
��{LU��,������FT��P?���e��NF�V�4���1p��4i�I�M������j�V�����g%���4^�~z���8���$�?0�7��4�W\������T��K������&_��>_"~�����j3��o���6�=��]l�w�2#>
k�����j\e���� �J_��$�/7��o��,�u��}���������\��FC��W�o��!����)���Km�H�l�}������U����.�S��~�	�����7��O�9���|p�Uw�Q�s-`����\O��S�����/���7o��m:U�����v���W����Oh?����rY�&���y�Fm1z���<�U��o�/��_�R�B���k��Z�Y�"��w��?���1PV�1[����"���v�m*fC���s������w�U�����zhV����l�*�����������fU����;#���^oZ1�1a���s=��|���P�#�s=js�a���s�s
�>���s������st���By/7��)�`��O������Ra��9���R�����Nm��fE��nWb�0ynB�Tj�~��Uk�GLC��8k���C����jI��L�z7�[����Q�`y����d9��K�����A�l��C8G���v�W8�m����2>7��#�>�@Kkn��;��	������H��Q����c *H�y�Nl.�#��w}�\�3.�E.�g+
��XT��vm�e�{��~S3eD�2�!m�2�fTBl�DI��sY�8n�����o���������]d�y?2z�����@�u6�����;y��T_����S����;#�NO�m]����6u��uZk-��S�i���y�z����K����9�n����c�=o9�����������>��S�jY�X�c�o�s�	�Y]�3`����x��],���@��$�\~Xz�r�����"n��=�i�.'C�1������-1��\M�z�� ��i����$/4.����M���`t���������+�)���Z��%���������k87�\����0�:��{r����p���mh��N�
�8������*x�u���[H�[�[9�8��pn��.g&kK0j��������r�?�9��*
����AT��t��P#�+K��v�U���������j���~���mU�4��#�N��W?��qX!:��5����&�Q��&U��x +�H��F������w:/�E���m}S��a������L5Q/�_�h�F�r�i��O�U�����~�����!_��n�w@����~�0�+�
���|���sJ���bTG9�M�F��@��*W��dQ'��\�k:j��N� �A���r���y.���6_?��u<������<'{rT�J?�������r-�����2N���wU����aJ�p]lV�/�J*xr�D���s�N�U!&�R;M����(K�T���o���X�j���?����R���9K��Y���Y��v��������o���`]���_?ai��O��q�l��O�	�3>P��7�>�p6y�u��h��q9�a	���
��b��x����T�eS�����w�����^�Uc���`?�(���P���d8��iU~��T��d�%���l�~\�n�tOi}�Dj����j�6�mt0����yU��X-���\�:kf<6�
�q��fa��
���T����sh��%B��F-�$�r�?�\����e@�����|s7U��`�jKm��t'��oAa��W";�nX�����`���G��-g�\�y�uS�e���� I�+h�������|�y���	v7k�j�������5�As�||��d��[�Cr6���*����u0��m*���9$$Is�(q���C����&������s�g�~J{#���@�����p]V���/���$��R�m~st>��DJ�p��l�\w]�:g�3}&g#���1�A���FK�@O �<�V��w�|����t.g�1���By#�>�����6�6_(otpl�']��������By#[���p9;5�}�|����������'g����/�7:�v�'ph������7_(ot��PO��r��������M�t��|281�=����7�[O�H�'���/��@������'�m=�o����w�:iz��N���w�������n�,^(gE-t'��3�k����z�
�rd^����x{K����h��1������0O2������%�{o�&:<����O����Z
--�� ���/�
$I�����5rY�B�E4kV��G��{�eE��F��$H@��z�� ��K��(�cun��%��6���
��`4�t�.�$i���1��f����\����`�V�Y�����(��4�����{F�g��g?���g�\7���f�m �Z�;�X��%�`����"\��&0G_���h�
�r��Q���o��b��HI���:+�cP�A���>7�F/G��e�f$�5���HNT���1��rF�����~�<���<�� �l�}xT?�!=�~�tZ?�1:���6���a��������a���8L�����a�����3=A���FN�>\���.�;���p����mpi�=�w(H1���X�;>'�8!� �M`G3R�NoKHqB��JIH�> �|haR�'�8!���&���"�|iR�'�8!���&��1!��+�\�?��U�\D�����T�����L�T�\<h����J���/����i �@�3d�?�����
�%)8���B�'�W"A�D�~cP�5���+K�t�\�K�����v�g��b�+Y�@h�^���$�M��;���A�37���~�F�z������:�AZ@�4��"�Hg2��)��h0�bfi���7Q����L��%��A�������;�����-$�3�#N�tN$!#�L�	���mf�6ar:
�5��0��h|�z���]����zX����.'���F��q����b�r��4�$:��	*���� �qO�����@�7�	F3���;q3-�$�]���d���':-;%C?�T��k����1���*���;s<nOI�A��������ph�>�ph�Y������\c��\�!��V���Cr{H2X
����N?W��CG��K�X��������,��:�xXD�����aps<,bh��E����9���P��Htr��4����5�:A�{�Q�t��?�ar�F����d����{h4r����H7���9����4.E�<mQ���������f�Qpo���&a����i�D�O���Xn��F���2@.�r�f���(7���Ma�], ��������N>	:T]h5�L�.U�Q/�\� �
�
�ma��VU����u�E�i��P�u���h@M�,p�L��	p����d5x�����$����f^pK�����#�2W+zH���t�[��7� o�}�tw�O�9_��������\�s���ZM�����sn
��)0��@��~�}�{
�0�8����)������v��)����)0�l
�MqSG�NI�����)�h�A��������V_+`(��Pn
,�)��L�P��VS �-|����i
,�)�h=B����L���L}����MIV�J��MI�Xt6���8�
&`
��$u
���wZ������8s�x~L����������\����1<W������h~�����4�|��&��������m�9g+L:�4�C'�:�W�uv9�<{VIj_����U�-�n�������aHM��z�����A��-&s���4\�1Q��N�,>����-�	H1���o�����]������������n`6k�HM�������g�m+���q�
6`�����A� �����t����Cx��O���en��Cx��@�IIA1�a�<�����<�Y�X�h�fC(Gcq4h3����^o,5��o���+Z��:z�(��������d�������G�+5��G"_G��q���|=�����j�3�\jo�B�y��\7��5A7{*i`��U�Y��UkS�NU�L��TkF�~T�?�Tk9�zS�2�FSk,��R�%�BRk�R���atMT}@7m�~�������U����@	���oq�KR�����	�����)���`|�m��W��U@7���l|Vqw�$	����������y�!gA�~�p7[�t3����oh�J%I����C���}CcD��� ��w��������We�74\��$t�se{�M'�wa6�,��^=�M]�YW�J?{�-�O��������k6Y���2����>����Y��7��s���E��muk���V������r����=:tW�����\@do�{���%%T�sBT�*n����w�����KYO�7��������{�:pH�Y�zP4M��{op�eW�����nR��:�����ou(
��5��c��8���-�ji�����{T�����������ey��?��G� ����YB�l$4�0+G 5p�4NV�6j�6"��K<M��F���
�"Pel+������NV�������m�h������4i�Ic�'�WW�����_//��/��$�/7�J�_����j�y�:�~>��~s�����:��Eov�R�����w�Y���]e�~~���=�t� ���~����+m��1��#���8:!9�y�����R����M:������v�*���|��'J��S�,W�M9^��y�������}4������_�������!4�rMs]�������|��Z�����s���AZ��*R����|��S�}���\�#��4���������=��Un��?�N �5�wW.wW�����]E��*��U��k��B�=��R�<�Y��w������}tu!wu����u�|��4���d����q����nR;!�Yqrb�I�mmN(���������F,�S�����V���+�rW5]9�t��u��g��d��#H��o�u�q�I��M6-����b^~������V���r�}����+N�n��F��>z�P�+W��i�U�&���������:���-�����z��|��z�i�d����8.{�^�#��E�EP�v�)�N��.]������.T���Y��-�����N�`d�zg�[Y.�i����u{��"����tl��r�G�p��J�-����������c����]����9U��S��I�'o���_���������
<����/����
�,8��6�|�D~�� .�,0�go����n'��������4��
��������t�<%��s��w��}O�/�3����^c�������F3=�1L�9}@	4�R
���2���nX-�������
���������������r%��	��A$�+d4���{AR�|
���#8���]�d�#�����wR�4�.��i�<��b�(�d��Z���l��H�1����d�([��e#���?�����6�C�]M�q�#�/
Loz��q�E�w�<A2�D8�p��G���i��
x����GP�r�3��GKx�p��5�W&A�N����t�!�`v���X4�~9���&�su���]l�������l���z��?/	����E�%�0�����d�s�yGhrBY��?
����a��p���;H�Z�
�OT�m��S�$�S�$Ss�&),�,�XC���j�E��n��2U.h�\���!�e�����>���P
��w?��Kv�y�&������	2�.D3G���&����j�]vU~�������Ao`�[��zmK[��}��O��Rg<'-����#��<r����)p����G�j�����9_����'m��'���'��9yT:�%GCc�rD��y���O���j�\��\��G��<r���k&�\?y��;y����4T��|Y�����mTX����Ltl:sxrC��6�EP6����{�1tA���$K��u���	�q��m����S�t[_oPA�a-����.��52���
*hA����p�L���� 1�����P-y��O�#�
&�O�h,P
Y�(�\�8��M	�rn"b�g�k����&���+C�uI :�D�o�B�9���������hu6D���������>�Cl�����:��������=����!z�7�j�-�X���Z���@�plQ��,4�r�j��A�"M���vs�j+9R�:�E]uV"���7I�$t6�u-_Z�����V�z�>�.yY�e�,��$��x ���eoG��A���^
��N������h��#����T1�%���=����.MS���qA!pA!sA���p��HK��W��P�R��W���`�Vo�
����@!���0���B>T%���(f�A�b��HY�`��F��|��58���+����r1o�*CC|��4��P�����x_#�y�k$�m��(�^��n>�nFf��F���(Y>,��F�����+�8�b��;+���K��J��t�����0B-rI��U�n(����mL
�������$D/����B��i������.������75�����2��N���p���$E���A�2��i�2����yFV�s����`��/�q*�,Z@�,>qr���T0**d��M
A��G�����H�/����c 5-i��0P
�@	�2T���*4�*���C�z�>9q�*����;����K�C�.�w�!�#�N�y(P���7R��'�{*��{���kb�V]H:��O��~���{L|Kp@H�H����
�I(�z/��N~����0����6��wj��l��X��%u|�3.��+�<������k\��f=��C�4���Z�$��H�n2`M�wn�w�b5m�^w�l�K}�K��K��,��\�k17�b��s8G���yl�6<�%rGKD(_�*\&1�5��(��5�hv�<�y���k�571k�KD�5�Y5
����S���
ix@K����P��U�L�1k!M(�4��+<f�B�Y��Z�f�0����/<&K{���o�H�����^<�eqd����-��:?�|RsaYyk����I5�����$P��w8e597�sn���� �A+V�
���d��������yvD�[&�l:Y8�)�@�o��|���W��GO���?e�9�h!?_}O(��t,�hH���zlQN���}�z��o��s�/_S�n����x�,p��zMIk`�����.~�+'�{��	O�{(M��mR;���[{����{��2!7���o��3
����LQJ'��e�����d���NS��I!{�u{�-�Z��@�<E�=����t��z������A��_������Nk������WA,�ZSB��zRO�8	�g��k����g��sjcS�^n��	�}?�i5������;���gm;���������"�m��,������+�*���`B�5{�i���0Q$)������h���q��2g>�7���g\A�73E�������z�L���=SX��v^3E�f
AVrm*��,��*���!��)4�k�;�F"�$�]���un1S4��S��I'��*s,����=6a�������w�(��Uc5����
x��V��r����)awt��T6�/n��[p�����P�"����\WP�+�\6�k�Xup�5�@&w�H���2�(L��� ���J�i�i�#5��� C�Azgg��$a��<��,�Z���1E����$bt����Dc0��?�g�]o4&�K�T�v4�;��G�c@	�MB���b)���,��v$�����4�&K9���1F����S�CJo���,�����e~GM�[K������4��i���oy'���c�3G�J|���H4[L�k��v��
d����5��z�;�u���s���Y�]��=�����`�����*�f�c�j���>0y{.x���lA�H��g��
��D��;��8�e��n�2�LbA�A�b,(�>�j��pD�6�G;�u17e�G���,8 O��\SI�:��'������OWQ[F2�z����j]E[E��1�������C[E%�Y9$��	t��I����"���~�i�C���6a8��?����Uy���&���x������~������N���M�>���G�.��T����x��������f5���&�'�O���Pqn�����T�?�g�V9U_�����dX��|��1��sD�s�����|��1�S���~N�<Q�r%yG�Vm���Gs���G'�������9���4���8x���z�*�L����.�_���Y������9����
���������:���dF����WS6�W����h���b)1 f8A���v:�F�N�~�g�����$p�#�����qM�7�U+�H
������ ���
��=�8��=��?���4�0��wX�bq��`�j-������ �W�W�E���l������h�:ig��Il���T��k*�pM�.>��!bD����:����Qm�,�_��,+�����;��:z�?u|�t.����_���eM�����<@�pH+�z�U�e���U������U@BI4�Swk� �%��c��|X����e��e���\�K��	�c�|��'#��C?jZ�����Yi��N����������*x��G���9�,�|Y|Y��K������D0_\!=�Rna�G8_�����i��[EeJ���`�����!������������M�*�21d$C���&���p�Fv%���,��["�QG������e�=`��(8�6*�������Q-\����A�b�=lr�b�xZ	��Ye0�l�����G|�oz$)$������]*��@�v��2�h(�u�^5�z�v?��3��XHnS�mO��i��n�#x���}5z�����1�3�n�����:jjw#�@����{�uQ��}i���p���s���������eMgx����0��)�|d?�]��Q�pJ�sJ3oh\n�8%�� ���S�|s�W�-�D�{F#����I�\�oS�~��W�Ql�\P����ke���oO��v��~��l!�+]��&�������D\���D\����� b5����]����`iww�!�3X��!P��l�
�urR4c��Y�NNN�����,`''/������>����z�=m�F����n�H����4,�/���_�^�s�u�v;�pW@k����]
��{�P��dF�K-B6cWC������|��],��'
�qr�kP���������d�A94�L��?��''�5(�*�0��d�C9N|=�!.�����������fP]�+��GP7l�4e���>��l]n�:d�wd	60�����B�RI��`���#�t�����]��-K�����$<�d�Hl�S(9>����=b��-BF�����"|��u���H��5�6U�R��;t/E%@�w
���|�������j�����51�&z0�E��	�!���~��n3��Z��r<��m���|����?g�|�6��?����y�p~�IXS�|�~���+k�1w�1B6h�bx��S[a�1��u����j��G��@���������4'�Z�S�I.�P��P�:DW���B�B)@�
P�7^���y� 	Drn��hX�hHsG������X��X�b!g M]�,)���u��!�j����g�g�J���������$�6�Uz��.2�
6������Z������ZYq������\����u6_f�����r�;�+N����RV�P������>�f����M�(�r�)?5���f�c6"&�~����nXTmx���~�]5��>`�6��U��W�>�L�����z�g�c�6�nw8V���������*?M6������U�cA�C��p�f������+��K�����4�+�-I��I�����&w�o�C������I���:�����z�	eV�Z_/�����;1t�H��oJ
��#BX�����A8�����������cc{�V�'O�@�>��=��))[�HI`�A�C0a|��"� �"f���}����+���]����G�7%��	!��{U7��"gW���S���r�g�X5����|w�I�V�����A^����)$XbS���lzPP���u�����)WE�6>����P:C�Z������$4_g#C����!������gH�7<�h�������V�����A���	�i*��,Mi*�~*����j4+�F}���\��$	UI��##v�V�#�-����P��k�#�X�����\�4%�/��(��������Y�����tDS^��_?��r��D��0������{$0���[1�~`�m�p��@�=F��� ��<-|t���,i!�?��-^1FxA_2��a�����H�Vu���X��H4hFX�X5�hA����HKhF��X5HiA����HbhF8�X5�iA����Hy�����.#���"����2��q��Y�]����K��;��x��C���-��a�8$�����cv7Ym���|������z`������_��`|��9�����P�OP�=�g�{D�s����1�-�pB(���|N�tb�;6��3S�4�Z{uO�]��������0 5_4y��>l����5�f��%4���ht����0�4�'D�����\�oLd� B�������zO��hD��pD9�x��{���l*����.O��+�
$P�7:�EIM��r���+�����$��z�I�?����Hfw��n3W��G��Aqv&lA�������^v�;N_�����{�}�9S��i,���O�h��������� �������[a��9��b G��\~;$�q:�sk��dI
]�b�NFcW��4�3��$E@��j���\vY�X��)���T$[M���L�Ih��pd2g�69�$R	��e��z�����f�X���JE ��c��~��!�I�8m���������?�5^���f	��'�Ny��&�o<,Do���T��{��)��H���!��,KV����c��@�^��e������%��A������Jj���K�K{��pA+�f�����wp��Q9��	SK,2�B��	�Zb�r��q{��S���O�����rB#Jw%|=��x
��VmrTN�%����g�m�����Tjm8�z�J8h�
�
����������jw�����j�Z��Zt�y�=8��A.L���x�������`�G���OF�3��t_T3o����k�u{�����d_qH�~;��}%�W*�SM�Y�U%��0��^�;Sy��'��.�r�hLw�VI��|t�d2���^�g����M��~�E)����91����=��/�g����6���q&��M��s���z����<��
P��i�:������F%[\��o�����]����f��6k�$��J�J�=�
Qsp�K��w@&�����k�B��!���[Rg=2U{X��~��3��At�4��O��Q��h]�_�8����u�OU���8��x�o�C�0}�o��h8�F>���`+����]O����=r�S�q=��#��ZsO�����d����L>b��5��4�|M'�ZD9m#nS+�;�O�e��x�?�]>�!S��j�E�*�'�O���������I���K��*��H�A���
�2G]�=~��:��\O
��1D��i�R�\����;���@L�]���s�9�F4`6h'Kh�N�Z/��'�B�*�(}$Q(�V�T���K=(���^5a�x/37Y=��5����7�zz^���z=?��<?�\??��;?��\G������^��{����k�mjQW���<8g���	~�����6[J 4��!d;qEh�!\:�<��G�[�H=Yt(�44w-��cv.��������MT�y�,�����/�
��Xr%�"�~Q�<��E}��L��l>@��<a��[A<�N[�����W?�����N����4�`���c��"Y^��9	]�������i��'�����E���'b�KGb�l�0y���mb�����>��ae�hY�!���Psu4�Aq�k��M�-�)��?��=���[�hj�+Y�]��}�J�jQ.I�D�8�#�[����Y��1����9fMFQrZpOl�����>�-5�����s��q>R?q
B&x���3~Rg"��(H-�)�f��O6\�V��t�~���^���oi�PPi���c�������<g�����~�L�7b��� z����������~����:�95i\���<��w��c/|��8�rCS��]�:b�)>�,^�zV��v����es`�t�)�{``1�FjI����J��P�� qi���6,)�![�Z���t]�<K����P��V�V�j����@����������D:�.?�e�W(���-�� +�-fE������ln@�s[�[�i�z4V,��(xV,x"u��ZV,V,�X8X����&OqW�Y�q>�'+:�P��V�V�j���.�mg^���K��$��I������5�Y��)��`��Dg:�/�ob_�>{���&�_�����d�����|��;���uu���c�k��,?U#�\o�Yu}�7�.����z��\��Y	#>����r����7���������������f+
����o4�GQ�X�����=<�6�r.���!�7��%�R����zMk6@���U%�
��~�/�����F�;B�v�)7
���g������=������}H��6��$��j���(�o
v���WC��V��<���?��D:�MMb�G�wS�8��l��+R\*@���#������IR��k[?|���s��|��Iw;xP�Z�]�����T�;��v�6��@�X��/������)��;�O�,��-��&�lP�EY@w;x��:��6H�g�� ��=��v�6����Yw�^�
\����xI��8���"�|���'�Eo�:(?`Jr�"��
}���D�~��I%A�U#(,��I��:RC,h������o�Y~#����A�u�Fr�~�������~�a �_Gjl�W���h~���K	du���w��l�y����-W�g��Z�6����r�kY
��2���o��w�q��t`m���MC+
�}Z]�{2�&]�LS6|&q�=/D/:�WQ�a9�&<7d7�FX2b�$����$��\�GSr��'{�fq���t����n*�r�'��x�sJzb���H]���\Y;�^�CI*��qQ���:%�I����>`O>t��a#2�����,����	\K)�������Y�"+��^CV�e���h�z�XQ�����+��cEU�+������EY�+z�^3V��N1�4��Hz�#){����������v�l�],&�E��_�V�z=�]���2[o�d��p���K����/eukU������>��1��@�mU�lS~��4w��W������H	C��xV7�3*�Um����n\���g#���s�6x��
��&Gr���Wh�:||�R���p����i!c��rvAz��8�����r�~�V;�v����������N��>���w6�.�7�����]������y��q��|kIZ�I�5
VJ��V�I�6�����J����jJ���Tr���P���)t���]�����[�m���~,�^��O@���� ��t��JV�oS$dY��&��M $��� I�d�/)��'�5��}�d?]���Y�'���ur<������������\���I�:h�#?�k�r[$����+8����N�����
xr�c����\�l>$d��:c��� �3�<�@X���F�Y�C�<SKM>L�%�cz�w��
�a�v��.Z-���0�cLz����Bo�A+�'~��� U����$
����3X:i���^B�>�O�����n�1�6����m���V`�m����Gv�~)W���J`���#��v�������l]��GO_]]�__do��p����*k�������^�\�?���+��
�s9_nn.7������|���\��r��<`[�����{9��B|�P�;{~���"�����O��+W
���@�d�+��J&��
5����d���:+�+�$=������������r�o���'�I�}M������#�A�����9Nz�1��4S2�L�[�<oe��Uh�4�\�Q����Vw
��0US�z\����;Zy�FE=�I�3!�X���OxNo�*���9�y��&����a�D�6I��4�^k\���D��2*�$I�Z���{�@g�r-A]N\����p�\���[g�R�&`
�'�3{����(�
�L�$�{()���G�nN�U��i`Qi��[�y�����Y
8/�i�-�	�,/�
�Y�S���Y��H-���r�+Fk�I��]H�z~U�7���!���	ZKH!2�[s`�i�-V��
��ak�'��$
>���>>���\i�����
?�Zq{z�Zw��i�^�	9�BYw4����h�	��u����V�N[�IR����L,G6n���^w��5������)�|����0�f�Y����;~�KR��S�oN���P)�uz����@_��7�Np��������^������b�e�o�>l�������^�RO�S<AE�
�S��9RE���������9N/���CW�Ppqn�������&Z���p��d�~��~�����&h����I.��p�t��~�x�j�6`�m��KC��
�N���p�^D^�C���������3�<������m-�YA�[xx���)��������d.�7'��s�Q�(7�d:�	do��Q���#��1�%*�u�����J����_��_&Po�R����:�C�I��T��b��S]B�p�w���E�(�o0��n$m@R��1!�F��1������n$y�|���5��&g��IS2��&������J/����C�e�N6�'�,K��M�>�r.,���,��eY��c.��F�Z��;_�[�O���r�DG���v�,{4,�Z��B|����FR�M�n���7��$/������n$yn
����,['ge
�2���r�Y���l��.�
��-���!��e�/��[���C���e���BZ�����qY�4���,f�����~��=������T���e��a�������wT7����+u_����$yY�&�>��u#	�w^����l���5���_]�����e�-�Am`�7hYVv�?���F~V�}���3�qVt�>,g���B�C5g��+�>T-�^vm!���h��!��f_/��Q�Y�����t���]��.?GN7�:��\��;�?7��
N���'x����x�qB��(i�I�N�u��E�u��3R��:��,�F;���/�������L������7&�4�w
�%�����D{T��c�v�P\,��}�e�w�������|0��zsh6������m���@���	t��)w#o4�yS�b����.	x�����CQ���[�S~w����f������qN�*
A^2~���~���+���ih�j�8fK�?8��)`A��NLg��i(MLgMLC����0`l����3��[�3qb�[�{b��������q�������tib��i'�{�7v�nb���'m0�����?��iD'����i$ML�;��F>��a�>�*��g.������InQ���+�~�*O��y.LL��ML�es��6�������D�3�j �A�j�����c���#�����������c��#~���|���{��cz��#y������]�
��m����x�.����x�-v|���|��}Kq-S\K����k����@n�v���f��:��1����]�0�J��va#M���mHHms}�|�/^�ECw:;Eb4�c�������	�0��>+l�|�-c��r�������A�_X�B�����`��	h�
�L���(������5�C�D&�3������4'z���d�����4'�*��������D=D�`w��5�0��}qZL��xwH��+�V�`����#����R������R��.E�K�R�&E�`�k{���SP��A�|�f�}���e���1��N�)�]
~'���������Z�S<�V��|���V�"��y!�+z�����5)��{4q�|���V�J�U(=_/�V+z���Qt����=��{�f�}1j��W)����J!����f�4���{����������c������c�3@�����
�������w�����u�hua�l]Q��xp�G(�[�=YA��6��o�/'����������m�t���9����y�ko�{��rr�y��+�8�V�v:W\+1�����UC�v| �h
s�"V=��|�,[L��;Yd���\��Y��&����d5�RV���S�_��k���q�����M&�����t?����������?��OY�&h�:_}�(:�~�jPN��HC7���s~3f��kFP(�j���0�k�Y���J��T�>�~�/�������c?CW!����s[Z�V�D�������=���B�����$M�)/H.����zS������eS~<���
�A�b���8>�����;�d��B�&���urr��0\t���
,n�G�������]����p0�@B�Tg� ��h�O�r�C����B��pxs
�C�1������.���tr�y���(���e5Z�%�Ik�����mt�"��q�����f�7��f^[`�_��p�.f5 i-���@(�����>`�E�}�@���aB���K�'k�4���K�Qk�4j��K�{k�4
��K��k�4���K��k�4J��K��k�4p��K�����.{�m>��Y�<�~��e�Mmn�U�������2��h}�>�.����j����
#k��VZ�<����&dM�^��l�L������.�4m�
���$���W$$����d'����:���G��p���Y�>��0�nQ��p�4�3�k�`U�0��}�0U����(�����;��:�X��{�:��&�Fa<�~h��������g��9��� �SQA�J����uNE�������8�S�p*��^�T�O�����zmSQ=����y�uME�������7 ��P
q�`J�i'����;���d�����R���F�������d��������4��%co6Y����w����\��Y����oY
�E��mu��w�j�.������Va�E�O���J��A�{���%LG��@��*.����;���|v=�R����d�.�n���Y���1��O�7�)����$J�f�o��M���v,N��rbu�3�nPpM��d?+	�]G�o�}�Z�y��s��l_�.^��	
�({�*ZT�h`LJj������H�5?���
@���xZ�9�N��`2���8���"^�k�4A�-9N7��^��T�����F����O�6}�4��������e'z�_t������]v��M�]<���f?5wQp��/�;����o<����F�bb)H}��w�����k���6�1�:����:�����������7�l�gi�uwZ�
�R�o���ly�|��.����w�r�&`�2[o��OWx �1��}��RV�V%B��e��u7�#��Q�lS~�����]������z��6���E��?��������<��8Zh�$g g�Y\-�;'�������S�G�
�T�����B��9���������3����iU���c�:H:!�7:A�EaFd'X�?���u6"���9�0�!k�SK���;�(I���JB����Yp��p�>��=0;&qP�l��cvD��<�\���{r�&G�fk�'�^��AMx5�{�i�
���Y!�
�{�C>\tV�w�W������\"<�h�������%z��J5����5Wih�
�
�
�>("�A��B��B����
K8U%�$IfTm��z������'�g��7B��,Z9��l�'�������Rm��
y�����A�S.�)���k�,��N��O��OEh?���2���Ku��:��S��So��"I���W��?���'���{
5�_>b�@���	��h,X�A�����
��J�!�+�b�����@��
l�M|�#��
Av�V�%����������BB<Q�����jCx;�������,,o�?�^������x�����bg6G�0�����'�����
�G�O�c������,L$	A3����!'+n�;�r��F��9~:���H��j��h�jI
�/���6�����P��K���y�o0��]�F
J"����k��m�>@��N�����O��C<��|+j���n��;Q��Gq��*z�k���X�[/���<���p������<�C�@����;��m��>���������|X�7����sg���n��&M���X����.�N*4�*��2�
��;���]N��= �OX
�[0���V%�-�������$Qvb��	F/�����=���|����$�||�r�&p5���$Z�����	��^*�a)s���r��,Z,�
�[�e�e.g�s&%��%]��l���"�.������
��BT(�q�z�����!�@u�+�q��8g.�x���8�u�,��Y�g�i���q���,W�Y.����8X6��v�qf����+�&k3��6���H
&��M�,WT�b��g�h
Aj�r�#A�U��>�q >K.��t�t.>KwE���F���{.���6���w�����z��61��2��n��}�+�[o:L�Bd�� ��F���T�B� T���]�t����r�*�&f�����[�Wr=T@Wx��S����y��
[�\%��l=	n�u=��g��EtW�m������m$������\\�X�H5�r�s���H�u#5�Gj���2�j���4����F�n7R�O[�*��Gj8
hB�sjg��dtR5�TE�64�������#���U�m��A���(�;{��S�h<����BSM��u�����5�X� �),s��'��w�K�X}�o��;����lh�2C���y�iB
���a�>>l~�!��q]$ps�~�t�rQ���w�>��Hy�nV��P��Rf���Be�_wd��u���>��z����bX����	B�l��sw+ZHl���=�'����a<����>�r���
E�"�&xT���l����,T
�e8����a��a:7�0]��msq*�E�����G�l���v�x�&>��F��:{Pr��v�����X�%���^�U�G><����/�g���
.�V���q�������'y6r�9r*�`=;���SL������������z���H�Aqsm���R�B4���3��H�	��X;�FN�P�����1��e��3)yF�g�:5v�x�i��I�����8�������3;I�?���x[���k�Gyj�	K��S�px��X4#&��S��4S�'��N!��<��<p�G��?�g�����_A�Ow�<%lJz�nJ��M��~E�SRA���U�
�I��"��TPv�UcH�1���WckJz����E���E�5��2>	��<���x���/��~!�D���'��&���� �uC!��p��P�M8GkZQ����� �m�}���G��4���;����~���7��yu��D�4��$�AE�E���vwH�>4�~��Y'M��.�64;���A��t\Ts^������O�����0;��S�}Q�����Q�"�H%���R�o��[���h��9JbAI4k����'�F]�#qHIR�0�:1��/����9_f�m�Q_g���{�E?x�Z����#�Iz�T��OJ.���S���d��"6��9-������;�����	~����)��8%�2��I�Lr������=p�b��J����]w�lUNp����>N6�l]n���U���7�����"[����)(��GO_]]�__d������AW=��d����I��,�u��}���������\����j��7��+���6�4O�{��~��*��������	j�'�}�{H�'O4f�
����rm9�9r��r�f��k@����~��m���*��\[��V���Qn�����)���~?�->���b�����O���76Pa'������9r��r|�����uu��k:������v�co���I����0+���S�,W�M9^��y��UO�^���Iy^�}�������U.T
��f��k.k��s��O�k�
���gZ���]�uo������b���.�����{�0��������Dod�Y(���I����r�>�d!��2W�1f�\?������qV�g��8��s����~V5�J�Y57��y���gUs��Y5����V�<���Y5�Us��j��:��Qf�B�UsyV�5�j��Us���kf�\?���Y5����~V��wV�
�*i��C���e�����\�-z%gW����J]�����:���:e���;�|D_��^������'��l����Z~�o���
�������d�o$<T`u���HX'�@\C�V��������H,k�J�1����&��&8::�����T��-N/����&�m*������o��
�qK���h7���f8���'n��^	������z'�����h����������%���TC�H"�7<����O�2��|e'R����qT�������~�}HQm`����bn��������xJ��]wD�I���|hg�!�]%��V���+c>lpb�S�#i�����;S��F����7I������jL�:�I�YM�	������f�Y��1�#�\[{q��"s0Z�'��Z�V8���j���>6��s�������p�{�3<�}@90��k��{r���
����
�}�}u8�B��B����}������r_l\�}ph�`p
O�q
�tr��q����!����}C���f���O�H�u����@��}"�$`�N�1���7���5�7��od��������r_�_�7�����GO�aH�~�)�a-	�d���@ONr�����C���=2������=�??d�<�����B��@�:���z�(.�V�R��S��"b�������X)y�$���Y,0y�$�����I�L���=�������=.��>D��=1��c'��L&���}m����cHS�\s9N;"�W�[��N=V��c���(P5��pP%����M�)����C���g}aW>����+�f��\��f/#�q�$E��O��B��3cg��OM���cH���H��J��]�:������.��:b ��������w���2�� �����F@�^z4��e�0�y
�\��\�1�������M�\�<�r���M���E�}�9��x�����#�f�\�XT~�K���V�x�fz4��emp���9��m������+�xz6��T~J\�Q������-
.���2>4� 'O�;���i���
.=#H������hZ��4����>�%s���+�W�\�r����kW���r	7���r
\����\3���+����������M�q�F\�4a�PHD��r��U�V�B^�
��U�V���r����h������\3�W{��3��;|p����A����Lc�\��\�1��^�
y�*t+W!�\�v�*L+�p�8��+�����]�,�5�����r�A��jp��\������h�:��o=h�>y������/��d ���g���w
���|�F���@���Q�6"j,{v�W���������l1�&���b��rU�8v ���f?����������������P�����	 �3�:}��[�����&f�
��&����e�`c�:Ul�d���1P�1p����B������
?��+3v�>��#@{�6��Y���Wg����)�:L8?W����wyq���W?4�7f�������y���M�$9\������������>E$3�E,�C��	v
�z��\�-���F�<E�����yQ����O��G��G�U��q;�;YJ
J-����u(�����
e`����$��d s����'�8^�C�<�95O9���^���'�^�J����rn
u����Qp�l�Y��x�����X����0S�3�a��#��ad���a��&�������h�\Cw0���5��f���Q$)�A������C.n2#
��3]�b������\��S���L�����mf��8j��3x��Lgm<k�y�t9�B^Bc��r!�Q!G3�C)3�s��)�Q���p@����i��aM��L���(3�k�`D����:�Y����*��Z�t���)��*��s)�S15M����n9��n�����t��_�3����m�5�1M���U��Y�9��#g���������#g��	��[88@jp���EXSv0�i�5�Lg���Q$)�A�������i.n2Y��3]��r�o�^��z.E+��f�kn���63]��5����c�{��6����f�!��!?�
���n(��1J���7�Q����( 58mX�"�);��4�e�sEL��(��� s�@�cb�>7���qS�����=�7�sz=�"��z3�5�g�v��.t�yM=|����������gm;��n��p��t#��#a���i��?k�MpD@�1!QK
H
N��k�f:M������Z: �$e8��4�����M���@n��8U|����=�KL$�?P����1E����P�`��1i�7�M�����I���})�P�(��8_��T�C�?�����@�4�U����������q��8d�K >ZO-��8_�W�C�������@��4N_��#F�T}w<�l�Yg���4��ig����
�Z��$9�����@�h�@��w5��T����=�@�hr���^9����@�����jJ�t�@UnJ���BVr5�����*7%��j!+9��Rr�kP���hr���@M)9�����M�4�K$O����<��h�v������O;����\g��7������r�XL��2+�����z~�\g�e��N����nh����/�_������>��2�t�&,�G6Y�,�Y���9��)�����_��"I���<|#JO�3�����#������M-�<R���3��yeA/���X��_���Q���~/�E��Y��]�t>�����t��D>�O=��w6:�O��<Nr�[m��� ���R�����	����|�CM�V^>���9��	�&�2=��c��o�����,���������=0k*���A�>h{*�T��W��D��2�~{���FxU{f�s� ����� �	[r �����t�]���E�:i�
����6�:�����t��w�l�:z�)�$=�N���B����0��d:���N�C���pn�X'�9R!uR�����#I��)���?������<��|�!�F�
����?�>x�� ���b����
��
������;�����n�]V�q7Ym��3��n�d��@��*|�N��
#l�����b����d��a����������	����_H��#t:DM� �%l����r#It��/n�����$�{��"��=�	�i�]�.�i�i:���%�Q��qr������cSi��6�+,�:�&a��T����U���7����
��hC���Rp��J����-��:H	�PS�
B�HG���'���vExX=�������j�#;���zF���k�1�U�+gl�%�&��&y���n���
T�����j�bc��m���d���0^MB�g�^���&#�&���x�����Sw�'����:>y�5�z���k�����8�z��~�C��A1���t��@��I��������]%�:n5��<�V4�q/�
�"��7'E$�O3�"<@7����u���!�n �bt	D*U��p�wUM�����j��3��U����P�d+;Y��������L��SS����E�(7�W�5r��[�w���r�E\l&q I��i<�uX+j�lO��`���i�0k����/:Z������9�x������%'���\����%G��_r�G�K����%����L�
_rr���x���V�Kw�J����+L��N�$(���u�-u������.���\�����N��_��G�K�������iu*�	���1�:�p�
�Z�Z�#5���j��K����:�r��������[����QWp+"�[+��B��
��QW��:5����alu{�	:�
��,2
p�Yx���V���kE��`�;�v�)��F���-v��_<�ry��k]!�u�v�+Lk�p�8��k����Q]�L��b3y��xk��G�keo�u��H
&o!0�:e���>�2I=`d?��"�cg���'��S����������f���U.�'��tt�����6���0��a=�g��?��@��@;3 ^���o{4���jA����"��/�D�iL�!������\�$����y�������b�����)�Rg���vI@����F�q#,];#��&#����9nsy���q���6��[E9W
��qP�����^��fT�:��qn,4��x�1m$7{����{9|�ZL[;���}�f�����J��7�J�&uF�q�D�p�m$h���]<h��,�������q����y�3qj�o�q�Zoa�_������{�lJ����
�X��eA��6��A;L�����b���|�m!���n�-������:��j��[���nen����l�Rg���l��&t��;��$c`��������&k����9nsy���-o����Z�a���[���n�n������Rgt2n����&tY�;��$c�%��U��gy'��C�"����;���S�88g�!��pg��w'h��^����{?�����0���u���Y��-+~�����
�����/v^��Qc�r�:�*�(��|�����Vf���
�3�X�._g�a���GaM�>>��qC��I�-�n�eW��YQ���e������=��&��G��.6�>
[h�G�@���t�������9vulA;h��3���h���0uv��q���I���n�bW'o�3q7��.��>
Kf��KA���w�j��7,�-�]���`����������7�<������(�`��VW/���`g>\����/�g����z�����=�8�������g'G�e��l]�g����:%�U�
��8���~������$�����S�y��p�3��hSe�$�����<�a��s�<v���iR=>�,��S:=����[{�@9���S�`R�'�qN!�z�������3��Ld�l�p�);d�=����O����||f��Z�v���At�� SvL��"�P��Pg���o��)���8�cu��/��~�O���C�9�e��n��<�'���b|��^�&���8���);~N�(�XG]��`g�:0��O�D��$�xg};������\>j�w:�H��MlZ��7Jq�7q��6�o��6Zo��j��F�E�,�!h��r2�I��>���|+� �l�m��0$�,?M6�_IC�&�r]�B@�:��s�B}Q���:�l������[U=���/w��������#�@=���|��O��`�-�g��/e=#�L�j��p|��'��2t�4��m���jv��
�>L���i�.�N
�e��|p+=��z7V��m�|�Y���mS�8Bb��/���E�Wmh;$�6q
Oq��G���p��p����������A������-I$�MW���S�����I�����$�.�'���'��3��������J^���'��QR���q 4�id
��u�?�^e�ey�]4Qv1����!wQ��L/��jrQ2T����:&(�<a��m��������7f�8BJBx����p3�I��&1	����q}~�2���M�'|w����&D���G���e�X�M���h�F�����\d7�E����	�P�����z���e��_���d���D�������������8���6sd����g��zS=���.��/H���� �������7�o*}��f���U����.�c�����k.9 �a���/�e%�V����~�,�U�T��'�~���(���baf��O��\�
�g��i��nl��-
�k>�l�\��J.�������|.����s�����?2����3����7����,?f7�+d���|�&{d�G�M���|�{6�]�+	��~M�Xo�x����y|].�����oV�_*Uw��P�-JbUL�gU�Kw�����_��_�9��i�lu
S�
�u���d��Z�$i��T��dN��]��S�VKnQ����w�������j��d�Z�������)��g�lNhM�tj0����������X1�J]%��Y[��U����
R� �wN-�#n�>{��:{[	;������sZ��i��8-g�V)��y-�5����k�RA$����[����gz����}E��~GUb�M�V��.}��������E�����r,H���/���\�}I7��A����m��2�3����C���%�Q�C���r����xc�>����p���
�I��f��B�������"s<�5�dd-3�J!�p�uh/r�n��N�{�d;Vd���Zp_�����<�
����.�
�O�d�������_�}�Feo����lv����l.x������W5��Hs��orn�wt���{�ru��'O�u�B������}�J�qV�!m��gK$I�b���1�����&�j�y������]��>����M��Z�Q+�����X�T<��������T�"���'�EC[��]�����I��r��<�Vc�L'�!�4�v,o����Y�zi��vz8�r�����mt?���xf��C��V�m�#��8�:����/��F8b�>����&}��E��,Gld�'�P�G��BAy�
u��V�4���h�'����nG?1M#DS<�����&&z�LT�sK�9������K�J�vEX�LB+�?��������
Ge���t��X��!��C�(�c�
�N�UM��/���*�2��
������{R�C�>�6*�m�A�E��CL���le�=�����,��
��Y��Wm���C��$�O�o�j9/o���mz�Bo��G��&?v?a��^~�����`\���_���T?+MPW�~�B'�k�	���U�/���W��q|WnS_��/oP�n�n�Uo(I��UQ�5��p�z�Dq����C��^���&�WkvI�b��#�{D�����z���_^�u��Q�H��>j�����#��

52��@P�/�^�D�N�/�
��S*���NTy�6T�	D�%�@�T��f������&
�WUr�q���
�p�UCp� �3N��;�L�Ls#�A��IAg]=#�������N�Y
����U����b��o������xE�[��``9&���c�/
��T8'*�U�����yA�����
uZ�
�{)���V
�jD]%���J8�
\��Vz*!�6xT�J��`2z.�7J]�]��l��	�����z;�}�����g��1�&�l=/�!~L.eo��?\�_�����k2�������������I]h$�
����"���R�Bk����7#��gO����:
6��ho�c�UO�Xu�pQ8������Hid/�#���d���Vv�Myr�j[��@���!I�(��*&uI>s�\��s
���|v@�5��YN��)kS�g ���2H���v�:�ig"p��1����sc��"_O=�x�"�f�}C@u�@u�:���^ZW!������M�a���Z�3�D���ni����e�N
D�&Il\�,n�L>�)�r�GJs��0�l�kG��t�H����B�����:[���V9d��!q��"��u4R�.�3R�n���H��#�|��\���R4��f�m
�H��)�0R��#�h�)j�CFJa)�nF���qF���$85-FJa)�r��(7�wI�(�q����+F��NL��4��x�)��A�"�|��&u�d�0D	���yr_���@������D;�g'{$g����HSP��[�b_4����z�O��Y?�r�g|u5�����eHsP�W�%�hDs!��F��Fq��u�m#l����1��Xcl�����7��L��-2�f��e�B��&��>�����b3�[�����_
��b������n�tv�]nz��M����k���z�"w��2������S4/l}�|�6�g�fhHTY��l����@�#}bB����o�Iv���t���
�T
h1kr-��;�V��$����z|��q��	�)����"�\I�.j���Ut>������@Y�Y~,�f����I��k_6��O���Uj�J����������������G_[���X����� �8L���]��g���K�V&I
7�Idg�f\��}V\n7N���\
�c�����\���\��a�M>�aE���s�nma_������[.\�/x������n��o:�+0hU{I�}���Q�n�m�6T����#:6�vgk����|{oJ~m����+���+7b��������1�e�k��~x2y�$�^+_<�D��'�D��e���O���
l?L�L��	>�"��o�z���6�]�\������\D����(q�B��D6j/m
)
;��������n����
o�-k�m�� x�������Mv��x�w<�i��+a�"�&��<��#r��|����|�Y�����\`.�J��E�i�^��(�-����g5$Y���xgU�Z���"3���1�P��&��m��,����
�n�-��D�����u�n{m d=�\��5
�/
	P��r&I�H���N����(�CU��7Ae��
3lF6�P������z	�R�g�@��G���j��*�6 �v�W���Q�6��o�$��;��3L	���b�9X����T��������l�+\��cr����A8�c���2�������1Y��J�����cRr�=�2{�CSp�T��P���7�����`��hA>O���&�	���8P���N��< �'#�"�A>O�[�&8
����Mw	�yB�$4�R"8
������Ve�m0GlU��&����4���dV~)���T������v�@'U W��j��D���TC����2Ui:���%6�
�L	�L��o��@��M����������������r�t��;!��/[K�00��C�z�S-�4a���
����ra��$��	�~8�An%�����$5�~L���H&,:�*����-� ����0��,I2s3�#�+67{��y���q������&f�2	
qn�.�H���2v�V�|�����[��Y:�~�\���	
6X��(�������46��;�0���0�}�'�|���k��b�H�{�
O]��n�G@�M�%�^��S�>������B��O��F����������@����%@�Q�%�F*M��`/���j.i�EPi��	y�C���[a�n�N�����4����
T�����X��"���;��2�a��I��,��yM
s�������4������1�1A!�&r(������4���J��)g^���S�� 9�L�*��-L�T8�XJ@� ��2&H��c���,��0���/R�AI���w������T{������b+�^K�QH!	0����� !��#��{��{y������(L��8%�����G����v0�Z_
I�.%s�v�j��[���mUb]����a������:�C"����i��xC�/>?��G5�_��������dd���s:n<�b�lNl��Z�F�'�$I#���nw��X;\8������nZ+��.P~�#;� ������)��4���.{��f����F��)v�����<���IX���T�"�;	6��z���CT"z��l�21�2w����ud;�RM���:g��&��Ct��r-��T�G�fK4���i�����1��g4�^;J=�EE��X;��f��
���3�<s��O�� �{�9`�_r���n�O�m~����o[x�m�=�}�]�-���dAH����������PHL�q�aY��_'�m�F��1�|��z�t�z��W
�<[��G��L��c�y���a_3��a)�\^S���68����k��p{Y��WAc
n9�f�v��������p1��^�~����ihR)�\�"�^����%��m����I�~O^�&?��C6��d]�q���?���U���v�U9!�9�L�u���&�O%24n���m5��ce����
s����u*�?/��s����t5�4��Q��PL��h�\���Uw
~�/I]o�k�p
�����|���M
�yT�U�yT6�f����N)��k���
3�h
"��~�#G�F��3nK�G$9��D]�������E\�Z�>���k"��1������*-��$v0��G��J'����fG�S���o����0{���n����E�'��`����!���������78��#�������A����+~�G;P��y�g����#H��x����Ul�+�(C��ioQ���$���B���7�Q�� �������j��~��n=�G����1�>�9�{3�,a_��q�V���k3�C��8�����= �9Oe��E�=XX���5�����J�Z������cl/ib�~-�Z4���]~p+4~�������Cz�cq\����u����z�}��T]wE��m:q�
���_�(2x�vz�vDD��KK1�n-��a�>.�sk�LY0�e&���}}�ZJ����v��g��s���=@���>�$���P�}��Z�+���l�S'�^�$����(�W��|e��{R��|$_���$�)�O5��H�6ydi���_��a���M�2������L��G�'�X���$�Z�MbiK���R+QI,��x���:�dZK�7��b/�R�3��n�"��t?g����mhX<�`*5����]���I��fQ��4
���4E�������[�"I�Ae��+a��XnU����G�i�@��@�"�X>I�y�J��!\0���;��83XV,�$G�L����e�PIW�rv���	WK}�<�+����Byf���X�����H�G����l�h�H�}|Nz/WM�jX�V�	trV�J
�L�'7�k�jJ��K���h<��c3�>��{�����]#8����V=^p�b�_#D��������Y�0E����H�,|��y�� |y�k=�������p���w��m%���������x�i%���Hu�a�U�>��������5��\o��)f��i��f>�6����mw�`:��D���>��9� �XgA��Og��-��[��}�[u&T�R����+NB������ao/M�m>8� [��nY,����De�FM�Q7&������	��h�k���o�h��IWk	l�������s�����2��\���~�����w;T�U�������EP�9�c�u�#��� �	�n����V�:
�g�����k���!�h���$�������/ �=C�/p��U�Ga�G��_���="Z��{ "��H�=���wt����hkk�{n!j��^14S;Xe�"��@_(wa��+X���U��W�\p_� =�����e�y���1�m~u}�^�s?�&�5���i�k����O���*g��!>7���:����&��S|�U�V?h
(�}�Vj��#a��c5�D��;�����\���z����X�}��hZ

Y&N���\b�{�P(��_Q�/.|��2��er���� ��~9�Z�D�����_���ab$�a�
I���U�a�����;�ws�Sm�A0j:��2�/'(����A�)�>�Ei��zG����H��|� 7������|���u���j���_%���}La�$��$�����r[IJ(h����7��h��'�h,K�9��D��4��m���$�4��c�0���"RG����YJ$�H��������E$i���[�e��������->�ycy�^[��b_�x������ky�U,^t�c�gJ��z���������Fr>�q�U���D�g�xq�N�|�ZQ\�u�ou��%�#�jW��(Fr��n�#M���<\)f�2����2��%������{H�&�	L�������'(�����F}L�����>��@���U��DmN�>�������}^4��`v��</�c�����Q(�`��h�������.I��R����y��C��d\oHj���6�Uv u%��~tG����]Y`�����K�p���,0Iw2�d�y�*T���5~��T��x��T������2�T�	�~��K|�:r���SfR|4�^��F#���P:H��G�"�$UD��!�"V5P�H��-0N�O�1jB2?��$M�Q�������*��@x� ����FuP�vw����:*{8�h0�[U3�Rf�	4�����9L�A����u=;��mT�.�� lh������sU�#���=U{���@�7������r�;!s�F��6�x_"%J��
����8%�IW��2YGJ�)�U)i�0^(lG����/]e��e3�~��W��1�����%�r���J��U�u�;��m��\� |fzM�*6%&KUR��)Y���*�]`*���\��0���+���Y�:����w-�Y��2Wi,��fM
�S(�>D�I6�d��IL�S�Y%�UR���lV�&?@�N�Y��W�j��	����d�J���E(��d�J�����L,s����{����`�����u��]���kpi9����$')8I�I
NRp��IRRp����������u�E�ZC�����������5��+y{�m~���&��|3�j��Le�������^*i����(��)���]����F��).T&D_�?���{�����|(��4�OU�\��9y��&�.���T7T!�{�RHj���y�����i&�OHO�\O�{�h
�I]^=i���GO���������i&�OHO\Oj��b�'ty��@�
_=i.�$�'Y+�FIlg���ID"��,���JpR�}$���s�@Kq`w��lf�aL� ��+8���i�7����[P�����������z��/��#j�By�C�����o���eu���p�rH�c,+��\�u���-�_�h�\;��'������g_]k�k��Z��8u�]�����	S-�O^C���v��-����
�z�6��U�D��q��uejvi���R��IP����l� ��Lq���1��mc^���6���b�.�����sf�0�Q����4o��`<�s�0�4f�a�YF2ocZ�3
��s��i��oFw��]������
�L�oU>I0�_�;x��Nf�&I�FV�	���g �SC���tt�'�����?��"#n��j9ge]0-���v��g �&���!S�k�~�G�(���'������j�J�,M�<
��,����@��L�&�G�I@G'~���8����{�h����j`��[v�iD�Z��\`�Ap.}�S�n�c�k?�7��I������s'�3Hg�i�-�P����.@u�����5OFv�H��3o��G!�D)aP���Z�~>h�@-$���t��,�{��>~'�=H�w�����f����IM��l��nU���������j|�n��������/ev{W�&�������M�}�6'����x��������<a�Z[�P�������
$Fc�F/D��vQ�f#�'���B��J��4\��)�C��$���M����eS�����&?����?��o��
���A/1�����mX�{����QIB�VO;{X���s���(y�E�$"�,��������N9��H�0���86���|�����Vi�Jl,W|�o.N[M��������������g������������4��_g�����B����u�e=����N�V�?�����������)i-�)��_�pg�B82Y_k���r�]��x��������`�A�
���� ����.RR`d{
��ab�\V4������R��$��@b���]az�8�H��������\^D��(4���D��(�� B���]z�8@H���������\^D�(4�d��Ip�;I$��1:��i-lI2�����Ln��Ln���Ln���Z��������2Y�&�s�����hw������`��������������: ��=�N��8
�����U\H��=Qu[&�[�u��'��S���G68��N�!��8���O�t���1�f�G��
�0n3$�����L/f��R|`p���������x�iu��Ct!�?O~��O����~��t0����������)�_�\h����	�]�JA����T�#j��O�V����������M�	�j�?w����f���w��^a�pk�����1�}�v�%�����}(�u�%����X�.T������[=#���8������h>+��p(���(�*�&u�M%~)��*���.~>����?z�%��z���!nN|#���7�yY���������?/^}��$U���q��6���C�����k\�!���
��=���<kw�J�A#9�H3o�7���pE����������s\�Kq���]e��h��-&�u�{H��R��	g��]e��V-�>�]p
���R��^����v���-�����0���6����w�������Z���!W�45I�)Z�MIV*8����)�2>�Kp��C������&�]0E$���>�w�PB���}����
�3��?`Y`5�C��6W�
r��2������4�6��c�(�L3���[8��j�27l�l�e$9D�����:���#t�����������f���?�P�;I�H
DR ))I�@))I��j�}E���N�|�D��'TV�f�������o��D��W���~s3�^}�z_�U��8�g[S���������_^���J���ox)T��";z���Yg^x����������f�;��.�?\�}_1�
i)�o��7u;5#�zbL��xM��/�gO��(�'/i#7�����-�^�
��$�zM��/�u�>�|�U�������w���\�l��RW��������*�u�x�)����#/��z����~zs������������W���f���.L��h�^M���1���/7}���8���x�v7m�o����L��mCgo~�����_//^��D��'�\��N6�
e�<�-Y�Z'���I�#���4�*;���D��
}m�,���U9!��d3����)��z���>z��"�wV�����N����[�V�^*O���s��J��R�3��1kJ��z!�������V�v���y�V���s��&��d�����-�����0�j������W�!s8[@	$Yp�,�ph�j��mz`
�j#'���zA@��-Y�M#���r#Y�������O��)��T(L3V�7��C��!��jmz�f(�L��'�;|3SW��*-������v6O�`�q;��A 0D�899i��C�S������N>�wF�;����	�3P�����NcGT�b���m�gg4�Mp������>�QK�{y('�![�+�
�����A�i1�g)�� ����[A/K���IQ�HR"��}!-�>��%7[��XfSr����
��,�VP& �d:�A�V�P!�-j5�
�61Q��%�y�"n%8z��'A���"���f/O��#��E��t���'�-o�y��U`�

Q	�L��0��\�'��3��5������U����F0��t���Ey��n��=��$���"?�3�$?%�INI~r����"?A�-�;�����s��D���O��^�D�Z�?}�7QJ���7��I�J�T��0%��)9%a*	S���CX�w"N�l�=��$��]#?��c�~I�T��O�:9�P��$E�I��W�:\��E�������J#@IX������,#��Mj ~����B���qvyNs�_G��}u�T�B�DW-��Ht��.��������08�*y.lOZN6��d�e��S�&�e��E*�g~���X��n
���������'Vo�]�����\�)��x������.�c�EH���o��<�6���~���;^�}k���kY�@
��F���-�	�7X���-�w�U3��+z��7����\6?������z���Z���#��N�B+���F3f���<D�"�T�=�>���z��z��A�Oh�F��q�#J�����gd����k=�Q�d��$���!4��Wu�%��Z���$�2��5���c�5{�\�N�����0�2�x�v'��bRz���"vU���Z-��a'k���������yc:w�
G��V�P��V�����^k�������l��	��+N�r�������(|�>�������Hw �,���hV�	�0�D��[id��DW���Y����}[H���^����lY�\�nv+_�oh�}��QG��=l{
%�!`�
	��H��Ps�3��t�����1#"D�#���9p��Q�����b�`�P���5�yME������=����}��C!�n���:6�d��X3��|�J�����4��fI$���&z������q�V�}������3��c��t(��%�o������y�S�����I���}H���#GF��#�����O��� "{�D�����m��f�#�k�~��CM^]_��/�����zM�6�d���,���?����H�K�a\�[
��_[S_U~�^�|Z~d��#�!���$n!�l�C����x����RK�rTJ��G�z�
Z��H��;�S7ocZ1*I��A���}�X�#��:J���k����p�0-!=y�z�>k�&�D�?���?����bI��~�������mF���Tf���������N-4N:G�wN����Z���p�#|)c/�\q�z��.>��@e��A|v�V����M������>R;W���n�L��	
x������18}1��_�V:a;�g_�c/]����6�[�t/�$�m\�%��J��X4zl��1^�Q��pD=P�=�'����VlV���+�����Yq����V�oz+Q,����@�NNQ�#�;9=��1�&���J�^@��A��K
Dv���4����F�C����p/��5�}�H���Z��������D�$j'Q�@e�5%Q�In����=>Q�~����v*o���{���h{�H���I�Nbw�
T&�[�Q���&�;������I�8io�rw����+�[������	���$v'��@e�5%��In����=>��~����v*u��~=A\o:r1���f
��d�t8��v��y����T��Q�N���hN��D��M�@�����I�U�)ne��,o��@:�F�E���(��BKyR
�p��{8R=����GjL���EtU��l�uI��*
]��@`�M+�
Im�����2�
�
'�FsR��`���
�"�<$��~��P��������d�������,�
n*��`�pR`4'�!�
6Z��pB�C�:�Wkt?J����3�ter����H�7��@$��xR  t&"�,>%")&��a�������C�C;S#t�H��e�3x@�����!%%")�I�HJDR"�a�fgJ�ne��$x����t^R<{�W�:�H�4��)�Ie�<�T�Ie%�OIeH*����2��i�2��������������A��
�D����
��[��&��v��MV�9:@7�L��2[���U_�Y���[���6#r�2�����>����[�a�?u�,�t�b�
���0H�bN�?g��J,���y�)�A��(��$*uU�O�Z���R���~���"2o��"!�k�~��Dd�Y4DB���}��f�$82�L���./H��%��F���y=�����l��X�!\��ECv���
W+�]Hd����`�K�,���d`�M�C'�0�dP?V������4�E$��T���N,��Z���S���E�4��Y�bV%+�P�uN�j!w������X@� R�}?�����kt������V��JD6�U��N��1�3ET�d��&(�@%%��1�?:|iv�����&��dn�0�@�
U�07��P8
�D��,RR�&U�����K����y�O,��N��=�j������Q1���������A��%��0^�H���$_C=�b��^�l�W����a��
�P3��rB���6�Kd�,�@m�`�"�OaX�f����&>.�������h�8�����&�;C�K����P:������h~K#(�#J�����M�R��cd���� ��x0X��S�sT
)U�����
��.��Q���+P5������:q�m����n�n[+�������l������"[+�-��M�
V�7*����������I�*kx)�7�L�������[�U��������nc i_<�]V�����v�!4��������
�T@��%t�9]�*�k(��0���B9T~�P����K(h�x��>�}+]�Ks���:�=������RB�O-%t���RB�O.�������U�^Wr�����J����l�3$�����_����\M6�+��H����@@U���8#�?9�?'O�5�%�����*��j��	�)�s��.W�@��y�P��#������_����;�����������1�@��F��-���83&Ch�m��F���:I��S���i��5tD0n4���
��������f��9S�ydmj�F�f�%^^�Q����i9W���v����,qI6�/./^]�x�j�n�8{W����]�P}�����?\�u���z��W �0��Cs�iZ0�	m�ot�0��7���������?}ra�8)i��I�Oj��D�~������U$I��A6I�GG����?10�SS���c�cL�(�� ���(F��:��G��g�\��w�-ah��J�lF��*�$���b[6�����QHx����K���1���]�;���usr�d������>2��	y���������������_]�����b@X({�����FA���p���e�RyH������?c��,��pM�pF�n����H�dbE|���=�c]z���]k�k�8xj��������,�a�@�b�q��b���!Hi�%�$�1��`���;��(�$��f
�M�E'��yc���V�5����6�Gc3�U�v���/:�%�����3��kL8To��E5�d]Kd��yD�u��'>?���$�O���������UKCM�����/x�=�LB��<|���5��k�7�����M0��2^-�AS���%�?}Z��&3������6H2w����&
��v4�������W.N����O'|Z�O�T���\��DU=!JT�AG��DD%������D#w��	J,
z�w�UB��A��7g#�����IU����E���ME��W?�W�����������Y�o/�y�������qV�����&	g�y���5&�KF��%t���Uy?��Oe�Wp2�����S��#�����]�X�F�>��m�"��F>��zn��<J����}��_�?��}�/�	X3wQ���S�Y�(:A�H�}t/(������m���@�-��
��{�����������w��1�/����v��?��c��l�%|{A�"]�l({��*{��j=�W���v���-���V��n��x��-����y}��]��I����I��i�x�bS�6,#~�GO�f>���^f�aT����=��nPS��G����.|m�
$W�����|��lh�{�	1c�:�G�io������������o��W��6
�I�p�:�F������f���?�P	�I�Kr^������<G�%9/�y����2��	z�������v�F1B�z����/�����nnf��W��
a����
������������_^�N�	hKK�qm����z�v�����/�eAY�����@��o_]�y����lz���������+�!��m�|S73�TO��#W�i�����Iue��%����w?_����q�12H�^C����i�o��*Y���?^T��}Z�s���CJp�d(c������!�^��*����#/�}f�W�~���5�W�~�0��e�J��>�7�e5rq�d�C��j��T��Uz����u�1`����"�m����7� ��mgeo~�����_//^�>���!�L�G+^�lW��\���	��^��D^ ���$��pw����#�-_1N����_����]�?��3�.�<���n�����	���4�f������w��d�r���/��f�(-���M����
r>\v��$��n6Y3�
�|������uo�2�/�l�=��re���s�9!�9�At�G4�=a�k'�����ym�~��@�v���x�j��79q���H�]����Y��#�q/���@�*��"���Fv�d��Y�G(���6%%��Ca�������-@4��i*V�����4YC���R!4
�4�D���|�	�/���a�(q�0��^��F*sO\���XmY8�P�,|�,�Qi=hB�r�C�P��W�0H�|u�h�J��4������MDX����V�7P
�������b4�{�
��u�y����\�C.~���4n�5.���+��/�}���=�'�7h1�;or�T.� e�$���I�o
V&��\�2�\�cO��z�X��'�@��=o@:��n7G\0	O0��]�}�q�T�i��4�*�*Hd��F0��@�=�A���|�@����C��0�4�M�$�A���H�p����|�o7�C(��Pr��������%��E=�/���F��"�
���,S����<�%�����XX�
Z�����P���{����xVC5�*P�P���[V8N���hTN,��=����o�� 
6K%�m���hA�AK��`2	���	hb,5K!��<8m"��I)�4�����#P�B��&�{�\������S��y�e�m#9��?� ��=���U#L��S1��D{	�8�������K�
��(o6��vsoJL��)�u�\�V������B�]�(Y,5%�M�$����T�����1p���0�x�o� "<�r����|n���M=��9�J��')��SD����`a�	��7�Xk�D40��������2V�p�������S������� ����ZG1�����xW�j�3T/R�s�o
:�F��h� ]#F��
�5�|���1�0�Z�7�Z`}K����e����q��F��OuC/q-�_��):�5��)�����ea�6�����G��#����9������5��N�w�y���<��
���B@��S����v>�s�����U-0��T���Vk��5�����`��
���V1k ed:{�����8	%�WF'2���+�H�2:����L>s�)�oxre�WH���pr�j�����[���db��3����"%�&�7	��Sox�^o)�7�j%�&�7	�I�Mo�y�ju�oj��f�X����DJw\&$3��#�S4�GH��S84$������\dg����>h]x����
b�M@���&���$vT��(�<�����F������A^����#]�D	I!!���G����K�@A:��z�*�q����0��Al��;�i�����a\,3Q$����I�C�d�f��8��Z[��E��,��������A��y$�����C������")�@�@�� I/tD����u�����e�r�/%��~�4|}����AGa�3������xXt���I����B���qiCU�0jcU�RU���/r���X�>��0�����-W���|��o���P�G���:�������e��f��,�����2����
���������3�"'NdD�j���p�AFB"���0��^�������by{q�S�i/��(�g�Y�������M�{/���F�	/�������N[����g,1��Pi!u�`w�����i#=5G�&��POSl��V4[�_�����UV�R���|�Yg=����,�7�V��F�J&t����m����l�3X5��zU��@�_w,��G�k��b��������p�0J����m�`'^�����3���j������x��s��	������pd��1!?�����o����9N4h�r��vL���������/�	9e���yj@��w�;�=��*��m>��{|~��a���
��x���� l�:����5��x�]Q��~,o7{���)�q�G�0��]�v6�q��B�E���+���k�A:��6�^o	�X|t���j���#��E��O����DF�h��`��� �%�^�Z�����X�S7��Sb�	��)�}1G�����M��G�m*��)'����9I��G�%j�m�$.Q�g������B����C;�o��tm�MSbF?��l��w7���n���W�w�:��~w;���|��o������|�����|�v��w(��.���'��������8��j�����c������[���������o�M%E]_��r����Q{M���V4���u�bM����6��]8.��T�o�D��^~Ds`�)�����A��M2� 0P�U�}�~�V����$��	L��%��#;L��X��� �0!ebVb������E��!0�&Za��B��r|0�G�(��$AK�z�	��<�% .A���\�!��"��r8���\P����LMm�E�=+����j��H���$#s���$�`,'��n���A�G�nE��H���/��r���.���lu�^cY�k�(QI����L�?=�Z��h�i/�O��z����
�|�$	g6�����r��:��{cdD];��{�+GM��f�o	���TW�P��n�)��y
�f��m�)�����o{KA���k���iv����
���CH�[��I,��WH�C8K����7��Iz^��b
����!�`b�{�S�m�)��Yp��v6)�l�����
gA��btI�R`ItE�����k`�5�JA6���][��,��{1�����P�m#�����2�(�k�B�������;r������V���@���U�
!��l�����#9��i���\��){��D�F	����V�[-c�(*e�� �h�T�����T����� ���FN�PU�v���3yBX�*!�f)/���A
�Nt�#5i�������Pf�EO*���
P���EO*����B�w��T$5i�����@��rN�1��'���
j��Y��ll�2��V*[��g���@D8QB	�����*�~�W���<�c�{�����=t ��tQ�*��w������3�=RJHj�OM��!���?��m�OM���h�S�O�����56$$����j�h�2�V*��?�����x(���Wu����X����CX�������|U��P����*�9��&_����|U���\� '9�|J��	BN���![��_����r�(��C�W��E$:Cg��w���\���k�)���X��d:�t��$��k1��	�����@�P����]��^�@���]`���0��!sN�#vBN���L��:�N����9�g�'�P������k�P�M���j�m�0�.�����
��M��]�{���<��8w �]���~���<	��1�m��k����@�qD��(��"��
M����X@�'��<�1���3g�Q����w5�$�����f
��BK�A%8�d�0�@��i��=>@O{�'���u�:AP����	n���h������}	����	���6�������_����y
�y���T�������brZUn�������prZLN����i��srZLN�*��brZ������A�h	���	�n��s���sQ��X�X�)�/2D;�/J��h��
O�������%%����K����������$-�/&�E+������j~��nKhrd�o �o��
eL�~��H)93&g������3crfL����193Z�iMI��193&g�����Xrf�$����193Z�~���	DK@x��Mx���y�3��hn�����~'�;��	�N�wB����oK>�	!)��	�N�wB�]K��'1�H�wB�-D?R����g���4a���U������s����m���F���6��fw��f��W���Le�������^x8�~<�Q6�.>��	t�J�I����X6,��Z�BkP��?�nz��$IPH�G*F����%��p�F��xM�yPVW�����M�������:�|S�?�����K��y\��G%r���H�4R;R;��P���������������v�3���`m�A��F�q��+���U�\�F.��0����c��=�/H������J1�X��W}
�
��^�"Z�X�	c}r�>�T��)���#�Nz@2��iD���N���JT��
��szEk�b{#V�D�{u���T��=\b������q
�����C�d�$���L��9����9P�����%9���v�B�t��"��k,�]hi��]bVR�������������Ep<�>#*��k���i�7V�6��������\����Y����5j�g��Rz����5���YW��MDl��Z���L��l�.�]]��I.H�s(��\JV�,{������T�3�l����4��B��&%����U���T��e�Sh����SZ�i�������3Z�-��ut���9������������/���������h]�]Ds-u�=A���W�RS�d���!�0����H������HM�V�,)��U�4�)N��OJ5|�(M5dU*��6��.99'�I���:�G/T(E(Y���YZ`�O�)��*�FD4�V�9#�P��n>�6��
L����P���������{|T�nu(R�����[�	����]vV9���RB��!��r�B�%��b�����a`��B�SB�SB���!�E���B�J�J��{�/|l���|�?i��o�X����{�/mR,�q_`E�E�A�Vx���I�h�W����x<�d�I!ae�,(�
0���G3��G���^X�����a{��Sl/�E@os�	�5���������mj
�b}j�+��a`B`�B��B���!�X!7$�������6��@�-��'KQ�Dh��^xo�Y�d ��V�E����;�0�V*Lk*\�p�P���
���J�� lM�V�B�p�M��0�R�0�[S�!��P��P�pod�#~P������-�.�����\�+�}�U����M�%��.gH�_K�/�����]��lnW^�=��@._�C���q����?9�?'O��N������6e���z��
s��*�Bn$�sp����L��#p]<�����*�7��W��D|vV(b��<��U�	�{�*ODbw�`b�^9���]5U@6����������>����[m{oS�k���d��K�xqy��Z%����q��z)���������I��p���q����O�?0�b�}�bS�)"��'����������|�����O��I?���>�'O����|=�0��3��
$�^�,�n����<H���QT��	7n�������M�m
����e�0�+�'�����J�����D�|=<i��(+Kz�XJ��+OkYR�Z�g6T�r�Pur#WUrcE�l�W-3����8�s�ZM~�lc>�h��������]\]�;�o��;��:���%���pR���'�-=^����pR&��r�����lh���aC�iD{e�j;{e��j2{e��j){e��i ��^yP���,��k�����9|Z�n�]����_������z��_zD���h�V����H}����r�'�K����\�-��������#���O�V�����)���M�	�j�?w�����
�t9l\��[��em�d�J�-=!���[�j%��G��aKW�
[��?Ev��K�����Z
����	����/j��>�mt��Uy7��[o*	�K��T��|u��y������/���>���z��kF^o�f��OV[I�T���?/^}�����_2�
B�����'Yo���Z`��$�M��X�~ �n�F��[�v���!�?\�_^�{u�Kz)
]7��l��]��d�.=#�����v��^�������[�qP�}���v����ru�N6�t���7���r�������Za��Br%I#���uvBx�$��}/A������Ff��.����5YbK�w8���B$��-�d
E�b�`f�n���)!D�io�~`��:��`.V�N���"�
b�Iu�b#�1�+����A|�B#��+��k?Q���}�l�
���*�5�p4%.�p��t���y��t���Y*�t����*�t�C�����
HjY��mWk���?�y���^]"����Y�������U����i����*�__\��[v~yI�����f�x��K��XT�����.����������{���on6����������c���F�d�|S�t3CTO��#W�i�����Iue��%����w?_������bT�&�z
��/�u���|�UZ������w���\�l8���na�1��k��U�nU'P'�>���>�2��g^����7�h`]��Y�"Cm�U�|,o��j6�b������d��#w��r��N������(��&�fPu�7�O�������������^^��C�`�D,�f��,#P��Q(�R��:���DXA�XB�����I/Nd�	�[��yg�*'d��l&���`\��v��/�_����|�i��;z�_(�i~U�cQ�=Y3�o�`�o�T��mo��7���}�f�>�w�\��m�h�Y�1���O�� ��^���c����5.��(��C����������������$rsh�$a�cD��MeV�gL���TQB���J`�&����;30�{��0_W�����`xP��FmDSnS���3��q{����t�t^my�r���WW!��icj��5W�4!�
c����t2��4<m##m�����b5�,��S*��sT�{Aq� ���8W�zR� m��CQp0���(�2�>�S�m�����-"Q��(N�0��m�'nr�6�{P;���
�'�Tb#�,�����a��xrr�(!��|�8����|D���w�;�;�rg��#������>XA����?��C��M�����Q�`�Q��'b����F-�+���VXN
+��r���.�
@�T6�-��������,�g��AD�>�H����r��&I>+>���6��~V�{��#�&�����R���FAk;��j������3�wQ����X��T��Uo}��`�J3E�4��6�T�|��]�HM��[��^D��E&�W\��NKB`��B��0����
�N�`9-	��
!�&�\p�PB8%�~��~;��
�9"�g�����A��w�2�8N�s�����������D�H	dR���VB&��+~�����d��.�%�E�+��%�F�K���d �.��d?�Ey��n����w%���7��=b��RGz��C+��8��h�=c��j������A�z��5���n�� 
�;���6o�f����LMAT���M01a}��F���*�SC��h@Q�!)$S/K��?� q��.�8����t���W��������f�Y��q�#1�|W��7�{h��J����"),�W ��J�?�%s?���9��lf�������l����m�7X�?��Ap�A�MD� .��E<�V���`�_��_0�/�/d�/���[�Gbp��j���q�J� ��"),�` ��P�?�/s��q����!�����C���f���_�Hk���U����P/�� ���H�A���r��B�Hj�����s��q�H�����#3�����$��7��	��~�\i�!��H�;�����@�������������W88n��M��$�
o�b�#����R�o�	o�r��7F�����o��rBk�9o�Y�I~�<2U��/��=`SrnL����197�+���s�����197jHL�����FbL����<tG���b�wz�DT�\'�GL�YJ���G{N����19:&G�N�!)9:&GGP)��1K����19:&G������crtt������K�����1!����A8:f��f��,"G��s5E3mm���Ptgm�X�"�����7G��a��O������8��]0��\�����F{�Sml/�{J��%/�%�	H�*���F���T?H��
cc`k-��0-$�	�>bh�-��0�5��B�pxX����o+CE6.����Z��o��c�������=�����X��)�u�e"�c�w*� i���|�Yg=���?uKj��Z/���:g������!�>Y��)'�h4�F��*������L���`��=�R-����G
d)��k=j��r�x�^;�A!'z�z��(a��D�r=��w�����df �1o���v!����>0������3��L�:��p� Sr����Y��K���m4$(���E�����ca��Ur����E`���e�r�4�4Jm��	���{.���1���>eQ��������U�Eeh�7��UD`QOo�N��)�:�|X�''�l����k����h�~@Yt�Xt������P`����F��?����.��ES�u���hgO��������m�.(����x1	,:�Y���ar�X��/�Slp�0�pl�H��6�	3��g�vn*!��S#�y����0��9�`�y$�#T��A��A��?D��GK�v��^��>�����DW���=g�nRXmi��s4!��6|���� 4�$_L��x����Z�NP<Oh�����p�4{Fer��G������(�����`%I���G5���ZZ�jtk� �'��5�D��9�(/x�c�y��\�I�g�R,�$D���m(-�M5:A}B��������v����B����W����p���}��6���AS�m3���
S��/����R��,�g��w���;����}���j�T�m��&k4+��z�G>~%��~��N�[�J����Z�A�����E� e��j����MW�<I�s,+��M�{S�^��W ���-Jl���fP�>��v�<���x���n�a�N�xY��Z��s$]�!����N#=�F4@�mTD z$���<���i[�*�+��,A.$_P4�#cl�#ch��^"�~�r��F�Hz�lS�m������N�����K���&~�D��������g��i���T������uj\��N�L�])��[qO�\�������s�?�����=t�����������
�R2���D:"�@�+����)��������fW
A9�3�wX�n{�����g�\��z,(����~��6�	*im����:LX��Y���Daf���S��Ko	��b��xP�Q0��EP��K�� j��:f����v�������������!QP����V��2
�3�*T%��IJU.��M�10j��	~q�F�8R�Z�0��5���j{c�h�Cm{M���Oi��~������v�Fg�����4��mOv�d,�q��0(�@TB����C;�����M������b�8���,�G�'Y�q�|�$!��5X=H��
)$��Yr�5��w�U�����^�k��a����BA�+7U�4�^�O��)ax���k��8�j���x�C3��&�Wcx��b�"�_��~A��pZ�L�5����>SX�e*c>���_�S>���8E��q
�#�OmF�P��M��]EjpTl%���(�Up������F���O;���!�`����2�S����S8������s�IoH)2C��7d������l�
��P����
S�~��0h�������X�����X��IoH�W��;��
:�lE���O��0�`�+�������A��p�wk62���/�"��+wW����m=��t��)�C��u�&t�N%LO�&��J��M�#8��|�{>�9�=x�;M5a��h����w�] [j�����+����j;c��Q@o4�Ey�����"�tX��QJ�GJ��d17d2�$cH2�����!������!5K��dI��dI�P��1�M����b�2C�1�BeW����{�ED��d1�EV�O�wdQ�:�n�(%��#%�H2��2F�a$F��J��di[�dI���%�H2�$�H2�$��f�0���P�d�R�#�0b��+������2�tI2��M#7��b7�������R2�8R2�$���!�a$F�a�]�dI����H��d��Y2�$�H2�$�H2��j�#nuN�+��0�#*�2�6���"r�$���%$�Xg��vx��=o��t�.l-�W��R����ZM��:`,�T���1B9�hzp�h5=`��\}*���
��=�$'����c5GC�+�j�0��j9�
���O��HP�R,f��i@f���r�]�$���!V�J�5�S�����PYd���I8��������4J5rZ
�dO�j���Pz�j��Y���(bV�)��jp���Vc��I���V�;���#S5����j����mV��,eYzg����E%����(�V#�>wa��V#�d��d��U����EqY1�5��)�(O� ���pO����J����i�����2�<�dcX(A���mKc�� ����NPj��*w��p���f<�JNtn�u>b':7�:���s���
�wy��}Gc:l&h`�n5�r2���@	x������'�=�*�	xO�{�
�H����	x7��'��X��'�=�	xW	J����k�o��� ���������;	����~7�$���S��������'�=�	x7T#�R+'������wc5�����'�]%(�x?d��������s4��fn�@wn��})�� �NS��� ����'>A�	�7T#A�R+'��� xc5� ��'^%(A��C�	p;$�C ^
&y��/+Hp|'��;�p-nV���`��'>��	��V#��	���`��'>��j5�`��'>��i�o��A��
$���x�Eg�;)$M	t�6%�=��	tO��J~���@wC5�.�r�M)��	t7V#��	tO�{�U���=��@�}�Z;����OP��v?��o>g������n���l��mv7Ym��y����tQf������>��QC}���[~v�,2]�*z�3N����W�a�?����RG�9(�1U�[�l���E$�
����Y?�U_�|.=��&�o��V���?�0��O&���b�����p�H��T
��$r&���C������Mc!����T��
jj�9�C,� Lc�S�K���f`AK�*��7@�s��s��s���#�}o�\j�\j�����m_����������k�"B���/��/��/�����=[��_����o�a0�Uv�D���L���O.�'����W��!�0���>��?��?���T�<Z�X&Ic}r�>�T������e����������:y��SD��Dj�O.�'����?��?����p=0�UF���0\��a�*�V���a4�f�����R�3P���s8G+��9���+4�� :DBu�j�9h��\�BHp��-��0��72�t��1`�~��x�{���l�f��`k���������#-�������Y����WY{O� �1��X��-��t��c���tlW}��}����U�u��[�F��/ �����g�����z��fo�n�����������R��S*4��w��L{����n5�s���wg��ao�\[�����7�j�s-���r��U�|3��w/�������%������}�R(�#�U'I��H��s���7W����C��h�l���������6��;[3s�lqU"hKS6`�K�p��.�:�3���Q���B�{`�j����V�g?�R��]1��u�c>;d������T�c��b>{f������T�c��G���h������TL��F�d}���}��Xg���d������*�V��iA�SX�����,�bg�bg�bg�;�n�yR���b�i���V�yw�������{A+�"�b/�['A��	�N�4t,�t�
�$�Z��e�pa�(����J����qu����k\�1#��Wg������3���qu������V������RyZ�{*�}�m�p4�
G�%��t�"�����F��zN	zN	z�"�y�-��l�������E���[��5��vO��{�&�Q��v�������^
��@��T�KA�����xL/��zgt��T�n��E���~��]����a��hr��:3���W�x#���V&k�p�)I0���{X���$������tV
6���b,���������6���3ugf=Bk�+�*�Z��	�
=i5�D��j�mk5 ���F�<�f�Z
�����Q���A��V"�;�a�C�aQ��*���U�����{]`j��.��c�v��
)�����n���)�f
���F
�	4��w�i\1�C��'���o�}n��{����!�l������b��op���5g<��)����EB�&��o�Y�{�G���C���7�8�=}�#t$�������^��!��w�r���.������7���f��!Hx-=���Rf�w�j��]y�������\�(�DgfS���<9�?'O����v���;�{djP�}�a�rj�@��$1��^�k�z��`����������<���:����^[��P��`]��Q��f�n+<�,)O�/�������"�wb�"���?������\�4�;���� �2M�����n�����
N�
rF�T`d�@oxb8�����	��L���QK��Y1�7N������
����7��7��Ot#r7����(����o����'�1��?�O|�/��>��R�� ?�Otcp��`���)��|�y���?�O|�9�9�D7
�p���D7�|�C��o����g�����9h�z�e�m������l[XM��� �����E����q��z)���������	��p���q����O��|�������|������Q�FUw}`E��H������o��>���L)9X��,�E�9Z�L�H�.�#����.�����&g6���N;N���j�j����i�����Q
c�J�ZxNK��OP
�����V,SqU,S���S���.��T�����������7�}f�Uy��Y%���N��#��a��D9��]?B@����=��D��j�;���I0�=��������W����w~uu���KD�a��iv��5bjqeJN+-��9�p0�Cw]�[6<��L�c\|����%��$����}B
�����]���"z�~-Q���/K,�w��"�t���,��t�������a_�&�S}t-S����G>�.���O���������U�A�9�6ZoqW�:�w�������2�i����Y��g�m$%o���'�@��B�=�����G����>��O�M�S:q�������7��S�G[:&8\"y#$?������B�CP������B��Y����X�C�U�C?5p���F1�P=!B	�
�7;������h�k4��ngC7(�����T��7����\n��������"���7_"-�}�la��qS7��o^��f �-�r��n!d#�
6�.�����k\�j$'~�d0��8M0���L�!o������2�����T�	���<f0�}��=O��k%B�n�Sa��ZA�����T���V�!$m�B.D�>
�C+�`zmR�b)��"<�K
1�d��K���Y_Rv�DGji�2+(�������)��UA9��%%N4H!���
���/)�p�esx���!�Zz��y��0���0x��K��Q�-
kGM�j��.�//��:�������U�����b�]��u��_@�7����~w���[�4���vd��w%���-/og�\UgP'�yl3��������9���Wk=C��F��$�ZC����|��\)jZd�>��Fu)O$��g&S�X5�bE��Z�{��d�W]�\��[)�T���9��VW���V�4��??O������q+{����(S�������CMuvX�VQ�������y�t�i�N�`�'��8���R�z{r�SS�
�}V`5���1�J[W�+Uv5��33��[Wp(V�k*��i0U���H� T��TPU����P>]�k�uLD���{���6��l�>N6�	]�����	^�������$x%�+	^I�J�WB���M���K�������<�v&�Q��mWkt��_/~|��%rtyu���onf��W�+�0�SG��M���#�����~������0DU��������|�^�������\�����.���o_]�y����lz���������+6�!]�v"|�M�!��T=1��\]��}����'�E�����7�W��|���`{�U��V�k�t�2��I��o����?~8�����������
������2�?x4��J� ��w��U����.X����~zs������5l$��v]����f���P�pU��B��,?�c�%g����V�������h�3��f��='o�6z����>\�����%}�����1|B���gF)��<��!VM�2���#���-�������)��xK���D
I#2��+0�}��	��|�����\�n��J^�����
�
T|�=���
��}uu��o��g7���G�����m�1���o���?����"����� �]z���I�����J�D���{_����\o��h��p�7A��-�g��/e-�L�(�����y&O�����7n����^�i��t29�������Uq�Vq%T�A��g��
�
������a�	b�0��5�T��6	8S�Q�'�s�?	���t��j��SY�����f�*�5�:`��P���"�z
)��+���b2\�9�?����6C�}
2�
-[����|����\\b���~�� ��E�	v`��	���	�XC�@QW����s��=[PR=9�;`
��`P��	��Pe	er�I�-���
�P9�T_���@X��!
Zo�J��*���VPmpIs������rpR��#�T�����#����
��#�T���
�%����zwU��L.�������[|�2����P�_����1���j�bv;����b`Mq'�����/�R�5���L�F�O���m\�DU�bi�����5�v�kx�B-�b5��p�`Q�[�$� �5�B;aL��\/��D��t�X���@��z���<�n�GF��[�����g��V�2�������K���)�Wt���s=^�q�R�psj8��)�D~z��:��_����U����'��Fo4��'W�_C!�|��%dB��_��8r��7�<h9���N�{�B.gB	����\r�=���P�"�	�$��0J��8y.�'�Rm}&�
/�����)_����B�FB����M��}+�]��B��l$�)��`��*^-!�K.���/ot��78k(��P|R(���u�������O�k�%r�2�=s�2�WB���	|6��'�����S��3qL��j+��0g�W&���@K&�7�W�eb	B���Q�# {.�)r�8G
y��	cl$�Y~"r��G'��-��L$�F�#��b�B��m-r�0�G��B��=_�3�P��f������Ma�	�A�5N�YXh�B�	a.f�l(�xb�s���B+
"_�y�s����{"G�5�s�pOX;
a���MqM� <�\h��"
�y��s$�����"�|�@�8���g�����@�����\\�N�\��7��t�u�G�8�q��k����'�`!��IvghqFWm�<a���5@�������Y�)�B�-?�.DZ��!�D��������e�E!���eCI��A���P���������<8�L�Q&:�1q��TqT	�{*���-�B���D��C�yB�����CQ��A���\�O��,,�(F��.�V��D�N���Z�Q���=�AQn�'��(��k�H�(	��(�}$J������������f���E�X�/Q��O\���%�F!�BU�$��t
�J�'�YJ����N���v%Nq����Hc�/QK�]���u�.���C�~<?�$q�W[K�_][I�7�p����]�d����=��h��;��\S"x�TTpH7�Pd���=v�����v?�S%���xV2�2o�������'�1A� �f��1��@YY@����P	�6s[�L�!S�P��R0:{������O-�(�?��e�>(�:������z��>�P�������Z��>��-}�fX����7)|����Y/T��S��ZL�>k�S�<bJ��y�������������#gJ�G*�>k�Z�������#AJ�GLA�>��R(}����y�@i�������(��Z�>��!}"!�ham��0l���b����K���" ��gX�n������N��AF�LH$�r5�����4��c[g�=�����-��L�R��m��];�y�n5N�;��;�k2/�	�#��)��*!�)bL���9�^\���T���N�\CdxMd_�^>�z��dQ���5"�SGaDz�7��#�d�y2�z�1��<��S�`F>��NjD�|������<���yG�&�'~�X?���P����5����'�xc�_�V���������/������w���6�Mx��n=z�z��j&fS�����}��h�r���VoyDS���e4�hh=Y��r��r�)?V���]5#|��2�c�v��'F�W��}q�
���T���,��	�+��Yh�j��F��G���,?�_�w������5�u8��O"��<N(�2����!U@�x��F���)uU�m�	�a�#�}q�
\[����+YYme��p3���o���Z�tysru�����'����Q[�����)��[�����!��3�����`�+P~�J���Dz��I�B(~�u0���Dzjy^V��d�������6�8�tn*�ru7�>��R����OI�.����b����Q���q���Z�&������{Hv�[y�����P��g���T!17�>$�����8�y�s��[nj
�H�!�^~���{��D���`D(/)e��V&�p]��D��p���S��%Y���Z�	���t�X���U��F�9{c}�6�/����� ��8R�x�P��u`���_p��RxLA��"I��y��2A�wE�y��B�t�=�~8�A��g|�_��\�L�7�o����Kt�&����%If�f��V��1�Y�7�m��5.�I.�
���Ct�����#��d���>�X�#�E�nH2��Q0�����C�
�N��!��U�g������cW!F�bP��;��!CaBN)�^��C
"�23���J��Tc'���@$��T�l0��D6��J�M�����(FePi�q��dL�� ���HD�@��d]�+������&_�j���+Q�n��\)���Km�b^�i����5�_�<���:�<z��?�)�z����}_}���(��r���q'}2-q��|�#5��q�^���[�Cb���*I�"����JS��������G��+
����J-��z����{���8>�7�T���&vNk�LI����;�7I~���H���#��S��PgU� �t�@�q�8���r���1�e���F`I��Z�����.�4tx,,Z� >�@���|�D�)�=<���f��Q9�P��=�g&h]k�����J�ZR����>�~\g����.���;�����{�O�?�4��z #?@"Y���x�����IjWo~��u#R��9���LE�������7�����S���z��pZ� ]2BNwmq��nOi�JgB?h�.��
>.V=I�I����o|R�����UvP���!����X�E�yL`2������~���%���T��`�C������K�� �Bp�$�O
�	��%]0��E."?)�N���S����&��e��1�>WI
�^������EI>G7q������r�e��e�lE
0���)M�rY�U�U7��
����\�kAI��3{��y;��G77 �H\@����'���'���'`5�x(�l��9�w�����;y�P��wF���s^�_(j�M������$��b3�(��<�O2@d��N��K ����{u�-��Q]��UsWm-���������I�'��~������������(������iz��AG��q�#�S��B\���y���;k�����[���z�Yzv���an�H�y��f�o��t-&L�\�T�^���Gs�G
�]gb��S����\�����b�W5&���+�|�+��j[g�l�}/�5�Q�����gt^��=|<����|�s�e�5����*ko��������$W�����	����y���5&��I�B�ZZC����p�����q�
y��h��W�z����x�"���0�ujLc;7�����gl��xj,��zn�u/�����]7���W�\��>86L@#�@�Q����n5s��)��kp�9syH~
�4G���_gG���R��D��~�8��|��S{��"L�����&��y���y��yU~Z�����+�bQ.������vQfS�CGs��zrv:8�y��wg�a~:���t8<�z	T����Oa��������I�?�����z�����|6A�^g��t=[���A���3�������v��Z��� �7+���^xF�'���y��:��f�JI@C��vU�?-����g���R��1�h�)����O�Y6�o�L�n��=���v6���u9Y�>W��������d3�~�������E�Z}��d1��]g������������/�?�Uu����	�����|�(��N�����v��l1Y�Qm7s�����*�i>^��V�,�:z�h�.=I&D�3`������X~���������U��V�a%8��y��b5Y���r�������>�q�;�n������"5�������-��W����%�t����w��'���������?/���i��C#��R��~�������������a��h�2������dCZ7C+���V�eF��4�o�M���K����H��t;������tZj�$�D.���p��~_���	�����q?\�{v}��:{u���=�����m������H����-�n�g���D��}�[Yr���F6��&���?��U����5^5Ve6�|��r��s��'��:[L*��OG�o���G��f�K�����<O�`s�Y`q��T���D���jJ��+�b�����\��n��3++n�h�*�X�����&�;��_g�����%f��o���E��Zd�{5����j4�n�/K����$8\�����������M�����,���du��
������x�� �H�	����^��Q8�?��p���
�:y�.�!�@�K�etB�9=1��C��h�gD�y�����u��mu
�A��y����Q���?���	MEO��������!;^��y�������}d��W���=������U�)Wr�J����w?��Kv��bQ���W�\����+VG����C�T1+'��eW���_������
<6��Nj�-����L��0��M��1����7�O#I��_�)rz�m#a(@������������;;����W@!1�j
lk����yWVQ�Bn1�FT��w�|���(X���{Zq��N=���� Q�*��6���9($�4H�i@h�@���D�8�!���������A��C�\o3��l�����4�M�CZ,���<xg{�%����/mNT�lda�z�Xs��w�����9k��c\�.�4��4����@�������yz�������I%��=��
����{�.;f���h�����s����}�"�2��~�E�[ya0F$���}q��z����������'��Qa����������_h��R�FDlp[x)�`E�3Nrf�r��4g�<g���2(���)��,���[l�������a>K"��C���
��4OfC��r��t�@,W6���$��������/���,�)�4�o�0��0���C��{�[��Ze�-"�.9
s������6P��%89��L���|7�C�g�dZh��Zz�H�@�����N�4��$���Y���K�����"x;���3	j�i4	��xD7f����1�b!b���x$�k�����lSS��(�`��o{>���N)�(B�XS�!��b��b���p�o�y�#x7�GD^�_K����������7�Y��D3J<�XKD{���,&8$�68�������a�~TG���j���[RxQ���<k��Q��@��E���j���;D� ��j��X)��)�
��:�+{}��]���ls#Y��9&4`%����
4vi<3��f��C�~	o��.��Bb��������kYR���O6���s���T%�4�lh����b����[@�o�35�a���yp=S�����|3��o?����Tv������N�c=�����9��6���Xm�po�'��9�������o��k��e��-�����������l��Xw?�%�C3�G���7��ZRKX�-k������F�������$�r�N�S����o%=#Z�m������X��\.����a!���������9�����1�BY��K��ZB�����y��-�na�$C���n���(���5�)��f��g��J6k��;[y�"~�������)��h6�IV! D�0�c�t�=u����*��z�T�o��:�o7�2��b]X��\�^�f����3r#�+<�U.0���jf�ZJ9��h��@��=��k���
�ZA�Z5�V����Pk	d�VS����f&�]�t��{5����1����"� ������4��5�'���5�p�>\������LdG�Q��`���v�D����(v�����p����	�7J>�.�=��g�s)jv�n&�1C��U��8�=q�`�����8�{(<nc%�=�J,�Sq��T�����>d%���8��,zZU�Ho�K�^l�{q�x�<"��z/R�)��b����k�]��,����-�������Vscb�:&���-H�;�� ���q���t���g���6���A�C7�90�����dX���z�~��3��C���C%�����g��?J Gg�gf:Q�L�>
�;&n�i�u�\�o�\^�����K����7���$f��N�U~��u/��v�������iZ��U����h�X|���V%�[h��>*`I��o�-�f���B��2|��l������u�����X������bX������BX��d�s�����(�����s�K;�;�:_J��J	B�K�@������������7(�����#O��G����q�=(=�UR�^���[Im{��
��a�~�}m'�mf%���,q~�U�~J�9��B?�������2�	:���&0(����i�K2�sI.��@e�2���@��,������|d���I�Xo�=LBOb�f1S��mw�RG�Z��O�%�*��]#�#�o:����g��,9���S�mw���������	x|����vox�;���Dr�}�-[�?�LW�oK����n��E\q���[��?�Tg����7<�-��k���"p���7�Q9��0}�{�?�]y��� �\_�
��B������3���tU���c��vl��
����mk�>��1������f��3[�x6��)��P�
��"E������.�

x����`x]���g1�!������&��\���"k�D�����s4d�`<��rR���j�6f�!�{����m~�������:�Wr���\������,=�dl�o�:L�=�(0�{T	>��h�g��m[-6*����j.o�WO3*&x�J�� �-z���[��gS��o�|L��g���+]I:W�Im���M<��^�.Xm������`�\T�/PJ�UgaN24����}mJ�5g�O�b�k�b�<�]���>�y������u?�}G�0Q��m������T�T2S�����mkNJ��JGm�b����{��Y��^L����K/��F<���)kE,�����zD���l���uz�n41H��9���bt��m�A�c
�R��D/�^G,�mF3k����H���kE�/�~5�>E���3�R�$�+����a�*{����q`�3��
��m�{]�FI�j�@[��a1n-���4��IP�f��Q�p4�D��mi��t�����|S��������(������_0��YsF�;p��'����T+�o�1�>���V<���Cc�*�����0"�l��t4G����q�������Lc�~��;�%�@��~�9�����v��bAWE?s���{���'��l��%+��g�,�7�.R����'��4����>��z%��z���Tz@bb^�x}���T��2:jd�c-��4$Lw>��������
�?��ND, �?������/���+<>R\J��G�����oKKA���t|��Gz�y���|xA2����A���=>�FV�)������J����s��'x�G��#����<�G�?>��=>���#�o�7��ui��|���h�g�C,����������!�61���������	�����\p��|$3�Pf���������_`�kt������cYxEs����B��5-��-�o�����Z�t
�����A�u���1�&2[p&3k*�����i�-}�1�=�����
~P�i��3��Q��H��2�P��,3��B|���~��\Q��$3&��JZ0I�����LfD��Z�>r�2���z�������?W��������q��Wg���{m4\dW��W/�.�/Kl8��_�<�<t���P����#;����#;���h@od���J�����$nd���!�Dv�0�C��!����F>~2��d-��_?��K�yG4���6��f%���F&[�X��X3����&%�����;�uMZz6�q��P����n���kf,pc#[k0�lK���p�y�wt|3���tK��IJ�)��n�Zw�����������6��O�iB�����"w|�U���5[8���	���
!���������|�I/�����d^d�����-���~�	:�h�zC�Z������b~��+��Dq��Gl:b�ch"jO�NN\�gV�jLB�Qa������������%)7hk�������l��3�������/����~<���a$Vv1�Hb�c��	�%��g�mo��K�?���x2Z��;
����*��Z/���[���p�G}��^f�]���oyn�p�;1��.���3=BM�;qD|��I���<*��X�!������Z0�<�������|���5��th<l�$��e;�u`��m���lk�h�!-u9�'�fD��]!4�b=�I
�%i$����������n��n�lfm����s�5�2�Zf�kWc��g�!��8�����y\�����!��c\*��J53�r���b�H��r���7't�|
s�������
�a�u@�k�7�������j��9S����T9�j������Z��1\^j�G�mx��K� ��b6��'��%��%��J��j�z�Q:z�������!��#��6�/���:�aTN���j�p�Y��m2��|��k��axrm�w{~���o,;f��hT�+*	O��Fx����	�o���=}
����������J��{.@�W�(��j���HD�<�b����aivfiV;������[lU��t���+���hr-B����qe����}eq;*��7�_��7�����f��+��U.0���jf�ZJ���7H�70�
��
�~��~)K�1B�������zA�z5�^-Y��o�W=i��I���x�or��9���6�z�{8E�X4B�y�7��W���}f�r<��Fc��!��C���S}�����!����C��%�������
�dd4(�8�
�����pz ����@�@8=N��]N��� 0��p�W��A���M^��o��[�����������v1q�,����:\�{����R_�>��3��p]���g�2��W�n��`�X_���X��q���'|-�,��y����K�=.�.�����`�k�z�hd%�#v������U�!-����0_��i��zpw�[�9DN��|�J�����������s�������9�Yv����
�l�lE���Y��%D�T�VN��; a�vV)�jR/�[z�;2|�$AJ���g���j[2�K��J�-�Z������Nl�#�R�8J��b����EW��_�5��]"�;���.�Esq�)6�)P�nN�"��0b�	^k�����EZoaX�v
X[�4���	��p2��;�m�6����o��S��6��X��Mn��:���IA����l�FuR����V[�,T1K��i{}��\�]�~��g+z���)[���T���7~�e�V�$�_n�����d�w�nf��Ok���nO��(�9j����`��|������!�7�sw
������	�!��)f����a?��Y���*?/�L��N���N ��/�u��
�7��b
m/��:�!��1�Sx��oV�����-f���z�����T�T���f��o�����^2sx
T22���"=����S���e!��y[�L���?_*�y��{��M<�8��m���Oy�o!��`�{�o�F-�AvX_vk/�u����\
6w���j�������-f����\M��:�� �\
�<W��s5���s5���)PY��������j�z����y�k9Wk�s5p���s��s5�����\
��j�?W������n�� �\
������BH�_MHf�-%��n$����{�%z�����:�
�,9vk5���o!��wa�kEL7l�.L��O`+����3�P�e���g�kF�����k��|#����EB��Hv�	�h+���~��q��|�q��CFj@��p��+��:���������qe��T�0H�P�gM��B��M�X$���D��PT$�q�p�xX����Z|I���VVn�,����v�[�J������Y����������]���jX���4���UEVo�[�O
��z����-	�������F�wwX�q'��7�oO��kN��!����<PV�=���������9��(����2�^�8_��L~�S����Bm;�I�/�Z�?�|�mrc�w(��[z��v��[�98�%����f�����>���c��<��u��q�x�;1�������n�6fw�����f�&[�����4����#w'�s���h�P�s�r���x'�z��\���b�4����W���o}���#��tM������e
K��_����5��}�ma�[t��%�%l�[���75��-��'�� G+�R���p4���y����?�@��T��%�Rd���@��]�^���K4T+~��x+�
���'����������������y/���Z-�H]�U��G]�_PV��g���5�G�.��A�����6�����wS/������K4/�8�G�)5�EZ�~��R�z$���s�n�y[e_��~���~
 ��*��������p4er�w��7�}�/�
����������3�t����_�$�={���~���e�k��g�jn��9��_.���	|�h}J���u[���j��o
��9�67	�F���d{1;`��hh<Y9&4��%�?H�AO~���3M:��'a�16��&���l#��qw�|
�Y�!k�Nr0N����*���J�������������V>����������j������`R�p:�*;)i6��	$g�B0�����^�}z�o��_.���kK
FC-�4�%�?-���sB���B]��D�������i�Q�5f��Z��s2�mZj�D>,`k5�L�y�	@�6��FA���M���4y7���Bz��z�>��P/m���N��s�����L�� ����pZ|�O43^�_�����p�C�(�#�����0�������������v3`�N-��Z��.[���O���A����.�Y�3�-<EI��f�Or&g9���F��\f����l��6���
��������=����99I��@������r�U�L`0t�0|S�CnG�����'�j`Suj���2i��?��d�	F��
5$~l�b��g�y����,9���S��~���A4�f�9����Y�G���I�M"�I4����z�E�s��U5,��>�N��X�����IE�m��^|2F<��<[����-�z
�
��q�8L(dlg8�����J&��O	��z{U*
���I&���h��T�G k����������WC��}���1�:�Y�Y��O{����HH?�2`y:1,���q���$�/rH��b����M��fRt+4�~�+,��]x4��D\H�������O��R<Z��'��:��-�����P�n������,Q:H/$KW��*�.�J���z���J������OR��Q�<������-����#S:�l{���uR�s|�d���B.=��V����=�����$I��I\
�+����^������"�@����Y@P%S�`�p�X�-,_F����;�t�$J�6�n�J��[�H:`�����w=4r0�;�����{�z2��X����
�h���8k9.�*���B";w\���!�jl��u,�=������L����m]3e��_��t�����������Rk� �
[t������}��:gp�f�i3���E�g�����J/�����;�14Y8�pa�}_�l����[Nhhl��,�����gk:#��G�Vo(�H
�����J��ca��j����oYbr�Z����D4��u*6����fP��y/7��Zf���2�d�-�1!y��f������q��am{�HJ�~��\�f��#t���g���2�,)�Y���{qC�V1�r7�:��Q+���5��1�1m�\�E\�z�6�mz�w�l���Kb�.�<���u�m\��(��VS{��4������[��'���u����.���s0p^�E���;�y�N�����\3zC��Z������6�b	s��b�QL!Q��]A�/##��y�{�e�;�^�l/�D�x��2&[����`t�k�{n�sEpn�����#���`�1^w��M�������S�R&�������!j����D��6�F~Z�J�i��>�{b������'�����G����U���h�m��6�5�y���,�$~Z4~bK����}��WbE������;%��!f��p��p�Y,&�X��'���86A�����t|����������A��J�8`�_�W%Peg��q��6�w6�\OF�1���7����y��3����8����N�F�t/X�f��cY�z�La�lb��y�"N�qFbN���p�����G��?�=��2�����������*�V��g�(������/W8��E&�:?{����3|��&���D),�j�����0����s�����f���r�Y�L��l�/��Z�_>��X��UT���2���	�5g�8��z�7���p�t�m����K^q��\��<c���>�z�3�gM��
���W�0%�������K���4��)'	/��~�x����Y�Q����S���/�E.�_�i>}������g �������z����&�����g�\B�������O����U���o�gz%��|���}�|����7W��>z��������=H�i0����������M�!
2�ar���H�b��V:`�' d����y���o�`y�P�{Vk�).��[�����
:�����@�O:�����Iz����bH��v�+r}r2x���"�6��N���Z?���f��Y8�y7��G8���3���{G��V�����=���E&Z���K<���Mb�=��]:�g@+��NYR���Z�]����,���K�<�#;��`>wN r	��4�%��r���zw��&$����� ���C������U��4��5Xlp�6h����.��M��Rj�-v�F����~��
V���
���/����6��Y!V_���(����z�/-�������'X��g������L���0S�H_�f���	��	f:��D�x0��B����`
�f�9��?���*,c<<_�e/�r�F^v#��x��F3�<����@No����G ���v#�2��O��4���l�b��/;�ww2�-#�e\Kc�:|r���G���xw#�2�-)V[��?b	��r�����|H��� N�a"���W��O,��K^|;��w0����eo������<k��5�H����5�n[���P��������� ��n��:]7#c��7b�+����wW�X[A�M��Z���B��!o7�;��[��C}`1���	�I�y
�|�v���F���.�d{�;|��	��
�����C���Y/�`5����;�2�����<������K�d�+w�c��%ma��&3�	��9q7���u�hm��nH�l��f��'�����V{��!�Q(�EcK�m��Q�m����1�G�vml�	����bH��Im6fJ�j-�	�1/��]�m1U21i�Z1�J��S2���N1V�����+�A �r�	�rW#X��3���x�Z�
{O�;�=\�5�y�q���~�8���,
���g{n4��)�I3����~�9�b��P1!���_�` �nF��W��W444:lVp���
0�ug�_���9I��oE\��LW���j�g��I����Fc�
A��,����ZL��g+��B��������X�}O
j#��5���O����,v��@`�L
V��e�+M����e#�M_g,���q.�z�m�x�t<W����Q��?m�������]���@���n3��f�����8�����v?��C����7E[;!����(���7�� t��(%����C�$	���Dw8���,pFD�����9�-`iVH�����2A�:H��)t�{��6��sU���C^6f�tfLus��g�k$��-Ht�pGn���]�-�a�!���������/�j�D������Q �g���6�R7K�������cg�����n�R];�����`��`�=P6�+��D�;uD|�S���0W�~�@Yx�|��@���
O�6������.X�;�G�{�<�i�[tl�]�]|kw'�|Gd������������]r�o���&�
�y�v�����o��5����7����Cw�{����>�O|�o J��!�o� ��?��7���7r3���O�����n��s��-�/�������N4�����8���qo/6j:hv*�p���������]X$
��*��Yl5&�"�#���#��x��|S1G�k�>'bi�d��i�ks�Q��#r��io?��V��v�77�;
Y�]
�2���n���[]�5kM���V�����vD����z������e�{}����s��1�MD���������}�'#
����w��7~J�{2�M�&�	bl3A�w|e=1�����}d\S����hg�,������x���s�a^<��^�U�?�������}��L�s��n<�~V��/4��������-J�E�O'=\�v����}�J<	��#A���-'�@�+Q�"�_�?0�������f��g���2�;�-��;Iv���n	�`�5q
H,��i�YMu6��K"�����.cMg.�/��FW��P|'��p+���"_/�FIg�M/��nGV�t+!�]E��\��7���(���#K��p{������m����CZ����CN��E�z�����P����w3,ev/������^e������<e�3�h������l��"��V�����Zd��X�/�(m~�=�B8�b�Yn��Okm�d�����<V��5�#tQx��@#�u;��eC�VV��{=�u�,��1�Bc���X(��2����96�l�1�F5�O����MOx���y�fw��X
T����<r�{�?���`S��.e�;k-���!�O'po��'��l��V��sw�#�M>,��$�
�������u�j��w�z��������\yVb����<����M�]���,�����v�c���:Z�1,�����9��.������������o�d�
����MD��{K�{|�>�>5>����Y���9���a���7����f����e�R��6�J����D�2<&w�����
���n����Z���k�[j���}�s�^�?������l"�A�����)Q`���&�������9��?��t!�n��n���y�]B�����q�Tx���D)���������q�����n?w������=V��p�&{G9�F]�4����<*�g��5 �o��Y�'[D8w�%�
��&���Awwh,��`^R�og��H61w�������������5������JOq����by2W������� �mjs�.�/>xRa|��Sc��h�<������GH��|��6�|��i7vV��qx�|�?0��^���E��h6�r�tXg6��B�e(vy~r��g�?���n�k��C�+Lx�������������"�s@�%4D	�����#���<z���W���X��c0�y������X���������
��&7��Eb��]���������6S
����F�L&
)���'�	��c�<���6��0�z��fg#*9�,��8�B��7*�&�_5���8a r�V��jM��f����v{[{:"[X�U\k��|q�� 86]sB�Kr���9��pS�M��f,;P��D�l]���?��y�g�j�+������(Gm-h:9�����W���%�I��#��z��
�/����isFi�q8	'sz1't�!�J�a^��s(�=}Eg�[��/�����+�c��K�
'mM����(���K�F��y/'��}��3�J�:q\�=y��>���]/���6<+��u�#���b��:1�	�xP�:��h����&���<<��bo��c�����8�x���Wz��a ��(t^H=<	Li�;05`^�e��k�h�|�xx������]�l?��4����,/G/H�}8���f����q+�_H�a�
�X1�;����Ck�����B�|\>-7
��<|��_7;�&`o�����(~R�T���{__���Z�^��Lo������pu����%���o�)�L=��|�[[H�Q����:|W���l����������Q��r��Q��V�f1�c��l:�=����������N���vt\��6j��Z��4����I���j�Q��iU�G��?�.�1��J�)��|	��Sv&`�~9*�$��-���H��H\�����E�U8��B�%V��x�[�mW�O��i��T�J�r��'?���|<�p���z�+�(��(
r��<�5���[`���~\.�����ys~��'�/Ph�v�x ��mR���
�l8�g}J*����;���
�v8d����!���($[��@R�Jv�N�����7�hN�7�e����p<� �E�����m�&�b=l�����Rb��}��|a�6�Mh��T�����~�
G_��o��MA���_c\��� ����,���A��G�����,�!�@n�C�7��o2$W�@�c}�t-�1ou���k�&����M�y��
O�hJ���u�S�	����~�q�=B*�9
���I��]0
�{���>Z�g��p��p��'��P����`����e�f�q)� ��fa��	@�&be�c%_�/7=��c
l���an�����#�d���%f�(nOz-�<a��P��v�����5�`j��chBS���@��76o���J�G^$"��h�������:�����~r�,�8D�|$���T�}����m��3��n��'��������p��n	Yk���J��e4���O������~������>o��e�AD�H^��������I�I+l����h����'�<�	_(8�8-�;c��������|�/�
�`:%v|����*����W�����~��#;<�����I���r+����+�w�itZ/����8j7���/�:L>g>��}�����z�t��������D]O'�����8����7�
p�=��{����A��.�>�
��������q��^0`ov��F������������G�N4�D�s+����&�(���Q���N]�=���L���}{�z�9�����M~|uGF���o�����o��h��|w�����_q@|�LB���!�4��g�KU���ZU�A���zm� ��a�Q|�M2�1�dm8����OF�i��V|@s;����7�9#�0��%�I��6�4����
�-��6���O���~����!��h�1�C�S�Jt�qX��9OA�����<^����c��i��>m�	����i��$E6>���A��B��L��za�?��`��V��z`�wp�h�7���uoHw�A#���^Y�I�K1�p�9�]���?:���J'������	�������p����go�3���l�}��&~� t�9E`
'?d�����$<�?1��������:�q<��:��8q�`����������K������+�>[�7������G�k�����2R��4���8j��������qR
��oyl�\�"'F_�A{��dk�e
lpg���E�������/n�c$
E���/[.��'\P�4�������G�*(j!A�'�1*�c����slOV#���d1�g�p$������uFQ������	�5��(��QIP��HX�1Hd`J`reX�'���~�%������3�������rB�o��>�=gg����h%��\��;D9������i���j)@��//qJp05�)�����SB\��������MW=���}�����7����]" ��gD���
���e�� ��R��������
�����7���7���g1T��@��I��w�����U���F_8�A�������M$�������b19$[��'�}������_g����y�(���@V���x���r�i�0_nf��bh�� ���:@/�6�JV�LE�kJh�!!��XANQ�o���-]p�"�.C&����h�l������3]�����v|����������|�q�d�c��Ry�}��67r<k���I��3
R������AV����k����=���j���R���G�g�x�b�z+4��(�-P��=Y_�����y��	O#u������O��<i$m�PBA�O��r���On�yn��z�ZL)�1����C�"0�|���>H9��r/��d�Y�;D�4���|x%���v���u�&'���|i7�{bs����y���V^V�=�S����cl��E7`�_�!�������5�I3`I
��!4��t�<�1
d�|���0�b�xI��� �+�z�/,����
��9{�����y����o��|s�|yvy%.g���N���f��3&�gtC~�>�y���z����%��}���,9�B5�N�#�a��<�s�������^�o�vxu��8��-�l�f�N��@�v8��C`)�k�������V�+�#W�{\o<���A��I�Z��Q���;�j��O���n��]kt��VTkG�����W3~p�
�o�i��i������P�+1��1�t#)�`G@�P���n���a��9�l��*����z�N�R�V@iV�9��B}�I<�$*e�
���;��+H�Ko8��:�6���*;�uFB>�9*8#�G�i����_��2������a��%om���T�1�J'��R�X*J�r�5��u�FY �"'��Io��=CM�$�3������/��5!R���i�BBT�I�j�:$�bc����i���4�������KC�#���a��l����*�>P�C(�8���
������y:�Q�4�<�PC�\��s����;Q��8[;��Uk^	����J������Q=8�����|m$�q�R@�7H�L_�����A����G��|�H����It<�s�ts�0��+����)��YE�>E���J)	:#x5�����g��=�'���G�}E��|�����d�*��~�F�;iUK�����*Ud��kG�*������a{��h�� �;������O�Mg�d?@I�0���gH(���"l�ZG�"h�s������
�
~
�����_|=�l����O ��+Q����>q����QU)�3~�u��l�&>�y|�*���Z
G��qE�a�����_�6`�� �G����W����-P@�XbG�{�O�����k�&s�2��W"c��	��c�����`�C�������&yP�v��$/�"���S��6���o��xvq��B��WgW�?��yq���sb:$R��������
&���'��B
�SO��c����8K�Z��Z�R.�F�e��)�$������*'���o���Zgi�\cf���K�ZRx/qX"��@nS�!b��I�4�Xue�[Rr����iUrw�%K�:����,`N<c�%0)�h�+!p�[�������I�FA�����U
���
���2�@�^����B�R�q�1����R��w?����CQ���G�a�e<�_���
9z�K���{���@���		0���9��4�a9Q9a�9����V
�d��p���r��c�����b�?��/�99T��l'��#�4�dx��'����T�A#��?A�Q�<���A�O+
j��n�v�	�v��ON��j��:
Z�Q�]	;��`C����^	�|j��F�i��&:A���"��������+H��hE�/Q4�����/4����h'&��N4
9Y��@�1��7��o3e�����dm�zq~���9JGhu2�h���%-^��6�bj�4,��9����y��Z���/c����=oN!��2tA��%��w��XPK��x2C� l
m��|�P��mG�x�?�M��ZQ;�Zh�K(H�5P�uv��9�����8SM@
LN��������/�%�Z_�N�����eG�/��//c0�O��%��Fa!B��#Q_�O8���]��o�~@�SAR�17�A�N���s�6���3G���8��j3No���>|�J�U���i�Q�=�x�����������*��@N��!�������mM�1��1W����:���jo����1� }�T�n�0O���4��j@����Z�d��{���^�~r=k�vL,�-���'�:�&Q�Vc	��7`���F�d��J=�Jlw#e��U�%�1�U8z����"�5X<��*�<�ocL{S��f���^q���V�5W�[od�qSO�V�+�LQo�����r��Z��OlF1�2q����5�5`��Ze���;���N�Y�a?�N���������O�r/X�[� V��'`���F��O>�x����2�����:���{�0B���$���:������G�6	�!�����>������Up�X�������	��W-�Ba�*R����|b�:z�hmf���x"�Q�=I��d������-�����U�^L�EJ�3�dm�(F��)��7�n�5��o�^F��;�#}�6�;HP�IDf-x(�,�$���2�v����Ag4��C����Mh�4��
]�n����vU��.����i���Sa4���I8��x�SD�C!?��dc�`s�r8��+�Vi�t[�R��������a��v�?�[�V�&��5�H;Q7�I��%HO�u��W�X�bn���=�[:k���o1��}�U�z�N���oK���Qr8�	�-���^K�]��m��r��������M4���&����r|s���0�������_��^D��Y�y\�O�8��ACE��t4&F&*���������+���8I�5&i?/Nag��@%8�~M>	�75�lB�7��#)��%������-���K��>��O�4g1�����YC� ��p� \o���	bZ���ppf3|��N86�o}�M�t����
?�@TQ���D�s�&�"�8�z]��M�+��~buz��h�Po~�n�I�B�/�;W7��K�2
����,� �M;��M����ad�"A����	�fw������Cw*�>�fh���A��:�y�b�'((�1���d��z�N��)��[<,�������B�#�GO��Kx+�jv��������BG!���X�:� �1�A�����b=���Ee�Z�������O<����O.����FWg��>&:<�pY�v������;N�	���[#�a�@����,��:|I���}����}!�&��n
���*�:P�(�h��d4@f9�UQ�/ei����F�`3��x,y�z���A�����A����1z��-�W6`Hce5�KR#Dz�T�Qg"7����������51'�;��d�Vw����pN	:p0��k�p�pJ��rR !�I�����ekp���.��Z�q���<H4�E($
��}}<�W�
��?�=����_�#�%|���e�G��D�����O������#���="�����^�W�5����f������%p��|�3s��Uf�F$�lE�@��P���	����+N�E�DG�+R{%�e�'��u4���^���=	�`��h�r��h���{QD�����XW��5���_�D�15��]N��q������IvZ�N�XNK����������^`�"�@�;��n������RJ�#�����]�t;_�q^�� �;~5��H��<�����k�m��>^�y����_4_����O8��*:j�V�Ku����M� ��m����a<O"�QC�; �s0��4[!S�4��W�����w�D�(������@�������[>�������)D�?C<�TX0�I���-u6�N�#c������N�l�B����N�s�33��Dp!�LO�����& =���x��&�1���A�L0�5��&�{$NRqY��#
��WCPpliB�=�A>����a�������\�<t��|�{�xo���s��������x�l�W��'�%���R��l���)*�c��-O>Xi5���gc�Hd��Gc�|*��4���J�E�?�X�;n>��7���#��r>Q�m;^Q��G��C$?<u��|D�tS"N��G����.��c>�]V���'�����3z����m��C���n���?�'!����|��	�����(������I�vZ�P>�1��lg�����d:��z�1f?��@8���	ON*@T�U6��c��A#�l7�@��}N���C���h�!���
�`N�{()�<�EKj�%���^Rl��m67���	,���l��6F�
��lMP'F��A4!��fG�*�e�����k�p����G� c��cr�#D!=s��h0�����'�C�C��wC��l�A\�j��z��~�cF$C����?3YW���a�=�<���)��_�G�#�F��^���-����
�,�{v��uB��%��}Z+U����Np$�%
;��CD&Q�><�5�����>hs�6A���P���M�%(�����3	;���GHE!g�a%����XP��C��"���@����=H2#09�It
0�#�"�e3!��X�WF��1�\Y�d���r�����-v��e
���&�
�(2��� 88���pP��#5�}<�] ���RTY�m���0��
-	����KQ$7�}�m��g��H}�b�]b���/D��>����_
_����������7�oK��f��af-���g��d+��I6V6���J*#7/k�����Y\��`QS]��f��i�t\��P\�My�6A+M��
��f�8 /|
W�xD1~��K�i0F�f������{{(�&K����|��aK�'aYu�"5�����

��/������Y{��������8YT�JP�%}��!�-��F���K�"6�� .��;0I~0i?��7�<��p�!����k�[z^�o��_P��)v�G`���m�7�����g
�����I��h'�����&���� 3.�l��Ct��x�Od�ii�&$. }��|hNM��	!
�a��`��"Xg68#�p�H�O"+SS�0��mC��+�r*��|�+�D[�[�
�{��0�p(�z��#�_��������M�j~����2
���'�Xx
��)PJ�O�etA3��!_��@�Ec�e
�����������H���Ga�|�D���>��������Q��P�|����FY�����HL��[j0�P�~,$�+=�8_(�3�'�q��/�#��b��������w����z~n���b$D�m]��,�M��Z��5����KT�~�����{��X*Z���Kv�|���dzSs���	�)	�J�)
0�2����Z�A8��ei��c��z`��\�"h�qp�P~���:nNG����)zkx�1�*�DQ1���M�)�h�K�.�����Un)�a��)/���F�����r��>=n4�3\����+�6����1"�N��WX���#��~-������~R^=E��*�U�s*[�����{���Lh�rb����Z-
���s�i7����hi�eq��h	_U���6�`*�a�������K��#M=Sq��6Q���Y{h������-���>��6�`�l�����d����8>���&VBHa$3��b;�@�Q��	���&v�����iPfF�Z��i�����S�m���K-Ya��Dq��a��b\��O���b���,�����
4p6�u���*�N��aS-��C���r GA)���H���g�b���-
"=���j�$CA�B[�}#�.�T-TkLF���:���\��X��PoO�;��S�"�+��`���:]�9A�q��9).~�T����0��H����8Nz�d���'��@~�}p�����I����]<u��y��gfw�����mo3�&�H����)E,�e�6�d
�vT$?eP���A�R��E|�3�}��RkJB!�|���"�U�g%&��Z��ps��B��/o���Zb��Xccb�T���bl��s+����Bw����{�R9!��Z�R.��a������o)��-���]�x�Uq��0Z�����:�TM�����^4�K�[eW���������3�����%mp��c�+����������6��+��n�:�6 �6[�2)�����R5`��6;��~$5g�+�L��6!�U�AI��d3x�ac4�%���W�,{h�����lL��� ������"�I7��@�"AU����`�it������GM����Uu���.+����KR>�3K���9y6p�i��\�=Xh�)����,FCWd�w��CS[�n��4=��!�)�qB
�"��R����n`�)�G
������[M�5r��}Q���PE���oP21�t�8�aB�1��>���C[k_$���a����B����� ��d��
��"�H���"�TE����C��b��S� ��gR��	eC��FQ�Q.��t����X��tF���c("��g���;�(P�;4�})��;�K��#:W���T���������S��x� �`�*s���4�f'%�����_���2�g��(�l�6�� �wJM��`��o(5[Xf���+�x�\L����n���>��t_lO�p�&�Bz������BwD�E)�i2��j�Z��B���$}��5��UU�����B.J�i�aP�IS�)>��GW�8�6���AK� s�<
|SSg����PoC.��$��1�"���%��K# !Sbpi����X+���	�M�*wk������^�R%��P�9*=S��5��G�W��<[��S�YZC���R�!w������4�}��a6S��j�������i����,T:l{ldk b,��@Rw
� ~>�����9�oL/�f�8E}&�O[�A��i�'j,R2/Ii��D�4��`�sNV/3�r/���m^��N��Q."������y�	�@t�2:��$�F�a��R8���Q��MUD}0i����}4K�0N��D[bOK9�u��O�t�D2��$���������,��q��6�������9�����Uj���8�������������D���n�w������gj#�}��o���3z�����'	���LE}���{���k��(���Dv��E���^�������4�z�8��F�5�( d^O���m�Z@h^3�;�^y<�LwY���HA�-�����0gNJ�h�1�8&e?T92��;�ok}����������k@@D�5m6�x}
��PpX��Zp9~�8r���a���
�����������8�s
��A��:,�XV#���>����HsUGN��]Q#_+��,�X#��������s���6������jt�����H��u�}�W��:���N����X��@��i}��;���<Ah��3����~
6i�R����>fOg�~�0���Tf�����a���?�^���1�t"��D�G��6����48��x!63�v�\��E����:S'	���z�V_��A����dP6kj`�=+������]��r�����zD/�b<&XS7�0@��8U`��*j��DNi�
vX����V�ZU��h���\��
��m	�s�^��7�����h�Xb�+��+��������..C���B�G���Tf�O��	3`�H���7b�~FUj�<@J-b�,e��8-��~�-C�(fF(X�0�O�6���c� Du�������[A�1�|��pz:Q_���Wk]b�����Q�b��-�E�z
��oU3�Q��?��d�D��tf���I��EM��z��e���9g��%O*2+��lh"a����6d���U��d5^J�B~��1)�S!�$���u��0�m�:#��;�9���5_ O��V�?Zv'�U[xK<nZ�Gf�/)��T')�VaJ���(�e�������xM	E� ��I2C��M�!���������
�OM����X�fO��\v�L|��~��0%y"C�n	P�D=��$9�4��dl�OLY�b'�*eNb�*� >�1�����ri�WT�a�P ���L'&^��4��&����������j�Q`�7�����3�;�4�IM�F����En��>�������9�.#��:&A.3����:%�-:8���e�L���Q�k���Ut�K�F\���O���V������������B������
%��Cf4&Tq?A�x#�%�N��D����OF����|t�������@���vb|�[�UcGs�=�B���rp���2LF�S��:X8�F�&��8|1Sn]����0$�bX����c�\��������������air���ff=�n��nJ�w@��j��ZPUXx�>��<E��lYY�Y�Y ��c��y��QO#}�j�=)p��5rgJG5����t����������z� g���WZ+�V���d�
c&t��i>��X���yP�X\J���6d�LZ)N!To����'��1?�����:�}�`�����c}�T�&^2�)�F����A��V��1�m��������TC)��^��?�
��Q J������
����x`�"��M�"x�Qe����I/1m�4���*���&L#�Nm�L��'���)V��e��F)
�j��.Z��Qa����Hh�T�)�	&��Gx���;r(ym���a+:\q��,�����6�:��Zx4���UG��)����f�]��b� �	C�����hnZ����>����3��j<�`$b�@7-�v�}21/j�6��(�@I��h�I�J�SE���(ZbD?	��Dv%��&c[L9���������-��X@�J��7���G��t���Q��e���W.��e`��x�
I��1'��&)�0o��"�A}������|#a��	�77���J*l-V%�	�{��W����*��_�
�����r�*x�&M���D�1��'&� A�P���5d��������iv���8M=��!�[~%�J��x���ps���"8�D�]=�@��0lu��v.��L�0	�P0D�n4��'�m�����x��HvN	%�w������'C��-� 
[����r&wA�� �8�q��d���r�*�,mG�4@Z��o���P�d�h�4a#=��&�����mS,�������!���o��L�]���T<%*���5������m{|[�&����S��%�ss|s�D&�����	�f�`n�Oj5��w&-N$)�7�� J�8%��.|E� {b����f��Y�.UYG������;��6����+L��;e���S.h1�e���";R��'���R�O0��A��C%�-�!���Xh[>sB�c��=����� �T��������h����R���-�~-�1�@���,B�^dB���}+�F�^sr7%���v����F�����k�D+z��"he �>p�H9B���r��A�
�e�%C�`��%���C��i����� O����q�U.������<���Ru��%��d���N��e\!�o9C|������J�>$gMo�BIm�s�F��GC���_S�r^gL��2��[����,*0������r9j��a�$?X��G�8�o������3�F�!��^�X�P��Dw��TT�=�9Q�<$�]���q���!�'���5P�2�6 )�p@�(��h��7�}i�w�q2�D�K!���.k��U�T���,~�?�AY�"��xe�_�ui��SZ���B��[tG��vA��l@���+jW����%�4z�"���J2m|���.[fg|�.`��`���;�y���n-�S��abE����Y�pIC8���|nRdQ���`���zr���_��#�����x6��H�kB��@���~��<Be��@���[�����esc��b89,M��B	RP�b�
�P�~�����<>+��%f�1p:|���+�D�HHk����h8���!S������Vn��"t���n�>~|���� K
�+���G��']�.H��ik8����v��%%�gd!��.��ki��%:0��T��4Jg�4xV}�u�7���<����jY��w|���Ogc�����Y�YvSc~��R�(�1U,�9�dZ+�V+������N��L�J4]�::=m������Q#s�t�����.*hz7���2I�\�\;������Q�D������|��A<���A,p<k��S��n�~:��e��"�yp��)��l�����nM�9A?�*P����7G���w�J�����j��	�F'F�����	��u����v/	n�S8a��u�h��4�~Qw:�m����kL�r�(�,����UHv�q����O8j��[��Q3�jG���:
��O�x4!��w�-��
�{9%���M�.7%����q�X��6�Y`[��2^�7�D.���7�(F���c�2yt��Z;�	US�3OG��O����I�B=�q�q
)8=�k���#�z�0b7$�����x�d����@B�"�D�OQQ�{9���dr��F_��a������ {�:�&D��p���h.�����/���J����������Z��1�td��aN��>�J������-$�a�fsa��J�����z����g��T���<$=*�����b��\f���)��!-�����:�����M��j��y[:�>���:��w�����<}�t�����������@�&����9�]'0���]Uo��nl*����q�������HE����D�L��}�����;����$E� �P��{��z�;	�f~�*��T�1�bT�����(����R|8���g#T���)�C�Q;t��j�K{b>�fFe_]N��7<G���h*��,6��G��!H#�o�?�K�f�7���&H&���0.����1
e�/��W3b13��L#���qX���u����H��xag�K�_#��9ML������	:���|a*�\Gqb�b�%&�������8����=���o|!>s��b����U����E�2�f�v�L�y��� �[H&�dR���-�����,�H/*�M��]�|���y�����}s�����'��|����<�d���z��DAKx��1��I��B��"���2�8�p�<��`�3�	�3��t������1���"G�s�@���u��L����M{����a
s
�_n��Cr$�����
����h8Q#]H��,H1!yK��4W����������K��l���c*� Z'�P/H�ig����������)Cd[�fn ��k��3Q�Sh�u�5J�U�Zw}�G�N"�o5���joL����� ���m�/�>Y�����bJ���u�B�!=<�;�dRQ��&���*L�4��]����6'&��#�]D"��]*��\�������F�T��D���D4�]�hA�>�%p�qSUN=�Miy%K4{ |�[t=��A�F�� �����f�Rk��P���q���
������kd>v��LP�<��$����`6��N�z�P���^\R�|����);89J��N��|w���=;�Ko��L�N!
�a����k���h�)�`�U����eX(,�
�Z:I%��-��$cr����f
@>��0�p�p�U(��a�q\?.5���G���-]b5�l�fC��m,%38��J
B
R%OI� ��������;F�����`��{����T�_\4�����N��0����7
|p�d�0<Na�B���8c�a��^Ch�z��D��RX�B2�]��hIN����ne�i��`���`�`
�OIO.���4�������������F�����@�x�oe�&���+�C�*��Z6���G/+���U���� Sl:y��sJ���>qZ�S�}��(�8�-���o�H��"������o�3SH��� h�H���kE]��=���^�p�}��|`����$x��3Vmq*����@$����%��l�����01
�2v�S�����U�������W�5�e��4���h*-��R����hb/-�1d�o�j�g�����QH�y�&�_d'ijQ�&��x�����rJ��FR��!����%&��@�����1&��@x`���>��������Kj.d�����h�EI��D��"�=Z/�Y���ALw�����!��P6�����cr����N�,�7�Cn��XWd^	�$Rl�-gD,Q[7
�d�r���kIRJ\�g����
�V+�"���V���*���>�\�a�20�����PZi������6I�G�)���-� m����	�'���	�@ }��6��"I�09��Dvc�LFA�5�5��#���n�mLd�6Q:?��Y���Q�	����kU�mE���k��O:���^�=��Sc����q[Q���?x�\K����$w�/l��MW��r��:iJ�D�-��?2b|)�2���UC���0����Vp�����2�0�I�����7inA����$��cHU�Vu�
�F����B���2�\&JB���L�4�=1/�I�����x���4���E�<��o�S�n��(��3��=�6���RKX�����v �X�0��3P�vfm�DB�$�@������`rS�AU���4)���bq�PK��� g:x�8T�J�VD'����������L����������0_�(_�ye���)�����q�2���B�)��0���g�Z�K�1a��L����#�
U�kJ�E�\T���o�C�����byF��#�y��M��<1��p��F�7��5g��7�P�xC �<+�}J�FL�!����+�i!<F+���������������b���d�z���1D������n0wc�F0wc�FD�5^�����<,��v���*
�O���L.�V��Q/n��]���B����$$�?��&�;�t������W�
F�	5��^KWc��r�SU/��y�c��3E�y���
��X���4��}z�%Go6��������t�&�<��h~Z���)f��8�7���_8)�S��9�o��Q#�`�c�P$��*|����iO\p��e)���,��������r�Xn�'�&7���-k������'g���:�	�a���@6z��(�-�<��|Q�����;����=J3�,�����O�3a���-QS�=���z-�e"+���`5u�T �Fdl�Qd�C�:�Q��J�Nr	fv���OhA������2��J`�Z��5|��0�Th�e�A��Z4@�_��X����6�����5{#�^�����&n�����(���w�kAp��@"�c>�o|ytA3D��G�d�r��^�@J
d8]A��@�*Hm	���������
��8�bO�.R���$�l����QPS��g����QE�9mb�[����������r���7���=��"X���$����� ��F�L�!������0���e�"T`�M(�2�,�7_��S/r|�^QT��!y�P��Xfb�QE�]P��g�����&�)*�!�
���%W&$��u����JV0xu��l�
�����.���Jl�V�����~�@"$#B
J��z�P��Wj��}F�qa��@ �T(y���J���4*��a�]�H�JQ�s
����D��-��)/������z���{{�<1"-�A%k@�_���z������/h����������'kd5��D��x���D�\� �@���F'zM���)�|:�!���Ek�$�'�L�P7h��w�n.��e�	'w��V�����3�>�	Y�H���#��b��LYPu!�V>�;o�t����_�UPg��w2����S�4e?�$@�}�FQ�S�������5���� ��,tdo1	eF�=7��y�P�>�ya����!�������n�e',*���C�<�������);p^������q;�����
!�!���<W�%d�p�x�7h?������"d�I5��J�������
���EPE_E=|Y��B��we�w����/���$z�i��G�����@ hS����$����.]������^6�<��<�*��������W����������?����:���ls8����I��a'���S
���}��=���_��@Lj�������Q��:��JI���t�}"����X���*�G7��A��u[�q%R�(�ZcU�0`��V%�@����	G�
�G������l��c9g�8��1e�^�C*�z��#3�7r_���.�c�:����)W��uf(���?�i�G�cP��
z���F��I��k\W�,�u��&[<�4BNe�)D�t>6!M�����O�bR%�B�L>tB����^��&��I8X���`��&9
N<��	6/�!}7�<��b\2�i[�h�|����)�G��0���\l#oS<�����)�y�����Zd�v���n/��LP3��1�xR�XE;�M8JN	w
9�h�`������B�����o���%�DX�O�������q��S�Q���eY�����+�g��a���G�������A����PUz��BZ�9�'B;	&Y
�����(i�,fu��c�����8���Z[(�c����Y��=Of�S�V����s;u��������M.�AzKs!���$���AoV���X&�eR)
�Ub������g�����tWR�`gyu;(�LS�aSPt�p.�$�d�/U���ND��7�L�dBZD��NTH�K8�M��?�Fs��.�P�2���$�hT�i�P��I�����4N30��
'��Q���Z��	�������u�C�� lOF0I���0�"5�.��<�%��vT�WH�kM)�n!q-3-xh�S��v��Y���
�L���(��;����c49$��tI��8 !��.��i��TZS���Kk���yT�F�����Q�4�������f�x>�m�	�h2��B�N��I/�_�[H�����#�M�[N�����L����G�-�qd��a�T}����.Q�3��@����d������,����d:�q*Hu�tQ�8�������Q����a$����3�a���J�����X��`�t�r����F�fh'a����0v23�c��W��)���h<�~Z��Y/g��'RQ�vYq�2�zS�����pL�
�r������?�:o��hN����c���U�Z���6�(j�E�8&�
9���I��@B>g���~�_F�O�$iK/6&�ib����
N�^���E�����M�X�����h�:�1�4�
N��(�z�����H�		9������@�q*���7��Sy����'{���5��P(��cZ���ZE/�j�: ��EJ<Z��T�|b����H���z�9���N8��&�Y��|Nm�������5��y9��q�gN>5;���V�(��\3������y�u��tKz�	���hu]n���f���h����;���������^�k's�[%�&:����"&p]���f�&9&���$.�}��q_0���duq�/��n������1/��y������)���������?x���3������Z�]��r��������B;j ��R��R�S�
����/���2d$�)3����0�i�p��EO?��9LaB�`���L�������]U.�?�����oO&�5Hu�Hu���6�����e��[9J�9:i��8���v��n'GS�r���RPe��~q��P�2N����ph��Od^6�;,�����/��JH�1f~�_�d9�>�=�K���tG#`�A�q9���ki1S`��i���!�DE5 �9�@!2A|D����g�C&?�Z~~W��}��oC��$�6S��=��u�w�����o���<�A������Cq�I�����]=�����h��G����F����n�w�[�X
��o�p����B�"����T��2:?�/����li%���J�K�Z��S�^No	kQ}%�������P���El�5bc����f�b�&����a>�P�V_7A��?�!�h�I�h<
[-~��:��V�It8��������@	�G�hJ�
�s0:�p�z �^w��)?��1�
��q���(������/fg�x��M?�T��������a��2[��
����z�����Jp\�j�������Q��rA������l}f�����Se6��3����~^���>����*��sRmt�����>
�� ��V*��j�U;��;N)\FcV=f��S�?�j����L���r,a?H��[���#�<�p|������E�?e���'4������MW�O�OkvPi�*��Y��� y���������ZR=��}L�0��B�\�������PQ�#��� "�u
~�>��J8R�c�@��H���z;@�
E��x
dW���RJ��c��ZO�W���*��;c��C� �f�[����C��F����d�����:���F�����=%��Kg�����vp������'jV�S�y��W3���J���!z��}��N	���/!��Y�`����pl@	�!�!I��o�N$������$
6W� ��� S�WO[�Z�\��j��V���l���+!�T�A���5�����b�)0�����d���G���M���\.� LM�
$R8������>��=_��� 93N^���z�<�����.�������9�:���9��'�*+� (�}�!���>SJ`����������������������i�������W����������/�X��GiT8)��4*%���;\�}����;-��� ���DP?�������`��(S?���)��pq����r�S9
���yQ�_go9��.��}�M��+��_@��1Xi����KV��|�#	�G%p�jN��h�#��G`}�h���\��q���b���g���������b���|�G�y��������F� �P�r)X���h�Mk��4��i�64
VC�������FS���4�3p'Q8 ���[gz3�"A%������7�Y�����K>���p|��O���4
����Vn�iw����
��"T����:�x�
�y�7����|~q���eA�=���x=�����T{f��r��������-Q�����r4��������SZ
�W_�`B(o���.�"�����4{���������������^V���J/AM���5�`�x���	h<�c)H�M���o��m�v����
2�/���Q�QANG<��G���T����?:��k��� f������Y�������EQ��.��N��{�<��E�m�B��N���Y�\�9T�����w�+���D}p�idm��X��cy��J�?�9����S�3�'��#�u���7&��A�]o���
S���T�q�	���o,�6�	�[�S���)�f%x���&7���oM����R���!o�>��A*2$�V8w���?�����������o�P���4���J�k�����%���NG[���69�,sE�%��3go� .���Nv���T�Z���B�
���m-(X>�?��LR��Eq���Ic���IU�{��<[��,�`��H����J���Q�s�)�;�n�vm����!	��G����:�������N'"��#��k(>y�t]��ca1Y�%"���b�1��?���^P�M��woD���m7G��B�m��m��;6���������=�9���:�����\>����q�L��N��	:s]<�WS)U8KSB���w@0D2u�a|t��R2�*�5�EN���!_Z�L�P����(0Y� ����P�!���y��
�T�"���~x�������Z�V���v8���N���bqh@�a�x*D`���dT��u���'@�!T��aA�*����r5�\��I�H��.�V&���\���Jr�����v�H�i�E"g�5�/V�����R@�l���d�r�Y.H/W3������	���%���������r]E��#�R����e$����l���a|��������a��&�ys�����e��b������vS�����|8S~����$&.�+)�;���(�� �,-���-5����L�j6X��tM+Z���i1��%����2�H��H1����o�Q�@HF_#;��?@bO��K��gRZ��b��Ir��c�RAw�����{-����h�F�"�9U�l���i�<�gc���O�3T7��$�G����<U��~���9�m<R���!����0L���,����+�$�I��b�L 6z�H�a� ���� qFX���C�~9NFF�jU��xB��
|R���P�@��|�POjT�V�-���C���t�:���6�T�����
9|���T�!�4��:�P
�}�2!-�v�X��,x��'����'v��1D��"��=:��-���p��)�+
�i�G�E�D2/�Ea�>�!��[p��_d��3�
D7��&�������M���b	��d�w��2'����:n`?�
�M�����Q�Z�N��4�N���"��~-7��g���e�?������Z>�����?i-}������sF3���9�6��s~�8�����b��Z�n�K/��,y-�����L��	�eO��R3R���6��Y
@�A}�;c'��5�3L�5*�HG��J�J�b?M��*����R`��~��|�./<��v��w]��7��
*����?�������u1��((����W!a����,�@�\�b�]��!a�5�5�6 h����~�rH�o>���v�j^�W�<�/�a|'�K�(����|l�:�������W,O�\
����o��@�%R`xa,�?1�����`���H�G��vH���K�����]E����5n��!��}��4L��L,H	p��tr�����;P�e�jKI4��,q����\��w.�c_F�N����{���=��rF�)��Uv�8��^b�c����Qo
t��;���SJ���|��w��b��Q:[m=J������Iv[�2yP�d�SK��8��yX���lV=����S)�Q#����e����L6�J��<,Z[� �����uL��B��RZ`�UJd)W���`<�����-����4-*
�6,�����YK�A�'�����l��22HW�W�u��Ph����b}�qE�
�C��!��Uj)Ii�tR��\���-?Vp�r�2n<�k6����f�y�b���4�`}����i}�>L�G���atx��e��*���&D�[�����O�U���*��7������e�;���U;��%�8K�f5�"���e��������k#2������@�f�K�r���x���AV0 (�-��G�*�\Z���O�(	h84<���d��E �!TY���..�W�����}%�?�W��A���m�`[r_F��a.���frm*h'}��S2�t��~8D�r��Y��ZoF�>�;��7/0�X\/ak�]�|+�l������A��j�|zLs�\��4���w"�\������mx���v������[#qY0��������>������U��������C|�$6�
���/�����S<m���}���b���Z���zAg�N�<u�$r�������8A�:O���w�Oa�G�r�6�4��|�h3�tte��I���V����u����|$�]~@�+�tTDWV��KK�
�����r�q-'��073��9��K�M�/�G���Ga<-NE�����k��:*p���X��4��e�S��m1���9�������b���Lf��������+�C
�s5L��e�Y��p��l��s�!�fZ���dg����<��3��Y����X��g�����3��~��bV_�e��6��t}:�9��Vp��g��\�Z�2�9IP���g3G��������0LK�$��	���B�3�6�B8��!�serN!�Ec��Ivg�&�;��@����f�<Ovg��\���R]�TO��^*�����vb����RC��H�%��w#���u�d���I��7�7�ro���@���(�Gq;*�[>������bqU�0�j'a�n0���������O�&�����0��p�����y��H�,��\�!_�s�A��2�C�kf]�I��8�*��%���e��C6e����L��[#k�9�����I�-=���y�=}��s�K+�\�B��(E5x�g{x�g{hA��72�NQ�L��V%��C��P��:E{��U	�����b'k������(;�y�����k�?	���#oe���!�#o�c_K�"�b[������D��:V2
���l���@���|�������<�jn��]������1����a�~\	����+���G]�����y����h�x�+���?�4��?�`�����'�  a�Ph���\{�8�YD��n�k:��Do[�C�hA�=����hm]|��)Zcl�h�n;�B �U��phJH4���#0G���<$V4]�@���$��c�q�#���!Kx,�m�G�{m&
��Iq���#�Y"(��%'Y���,g� ����Kx+������mAt }L���>7%�Z��
�T7h�t��Y�P_���Y�Y��]S���M�{YJ�0/��n~	���������_��i�K*�S�Y�����8��dq�@
y���n;���R����uK������m�g��������H�U�H�OI��H���B����uK���$����gH�d����!��`[��pG�������Q��s��Co����)��R����dAMw�����G.����d����j[#yVL
X����=�o��k��p�r?���5�5�P_�6�/9�������Q�=4��+������u���\r�����}�c?��E	H�l&5G
�6����2�WMm�S��HT�5a��t#}6�IJm.U�!mI�4���	����#����'��L[�:��c�]qOL���h3�s��T&=.��W]Lzu�_��	M����u�)��+��u�0���m|U+�`2Z�YP���Fc��$v�����.^�^D��Fa���E���`<~\HFu�KP��
��O�b��*�G�B�if����r�N��R��V:�}V_k��`
��������R2K��Rt��;��}V_o�[am������Z^����}h�uM4�N%_�#&�/gG��w�?W8f��z$����~n�U�\��)$���\�s	��)���B�������}������$��E��@�����
��5�L�![�k����E{���|"���m�U���p�����bwr�x>��=��u�Os��.���r���p+�u��u`��\r8����A*Z�t�q���_B��hX<�@�%x����B�Y?Z���Yx6Wz��u����T�N�n�h�3���m� x���K��6��AG��k�l���5ByuJ�������b��v �gAd�R~u�W5���w�Y�X���5��`|Y���U���(�J`@���N��t>��-`R�u���z�d�Dz��9pSS��c�K~~��fc9:hj����uQ�V���/wk�y���<���[�m�aK����F�'"�x�:��H\����tj�6�����0N�U��;n�������Q�S�
�����G^���w����������}�������!�2�I��b��KDK�gg�<
��X�+?���m��,i�T��v���~0��Pw���Q�uJ��u%��W���6"�&(��o����������H��-���Ga��4��sm
���q�������h�����X"��*��TZ���Vh�,��]
�~�~�e���'_��<3�-��XF��N$����5cZ�s�����Nr7C�Upp�-�@t�	4o��]�A�${��oO6�&��C�XW,��F��#fV���F��3������m�z�-����	>>�w��B�Q9��Eo�|
��?�L��g~6��������SH���,�w�ukd������]�N#�(�����z!4J��B$�����B�2�
d����!S6����\��S�+���#S��-2�?��L��25d����!S��07� 2�'kW -=�eo�]�vN�v�
���\��E�0:f�9�xi�3��9j����HSG���0/���S�O����l�Bk������|�Y�]�D?����bX�Cb�3���fN~��&��}�o����b��������L���hAo@�ly#�]�DO��=o&X����i\��P�&�
������m���\<�*jQ7�D���[�g�j��)��[|C-w?�l����O��O��z������mx�<���gi�3�����#���Rh��E����Ec���1�dF�<(29B��f���ixZT��4	%��Wc����iU�#&�XZ����6��ov���_�K��Ex*X~��By[��P�����^�`��������B��L����Z1P�>7��������D�~{��o�|�U���������_���?���?�d�Hw�,�����������P���E���1Nmhj��/��0O�0��o~Fw�:]����s�
��s�s���j���sj������Z*�k�k0�s�trC`��zR����xo���r���������k<��L	<�O{��*5O���=O#�.���'�b���b=�����\C�+V��_��)f����{���%��o�e����� J1��M��}syUl'��>I�m�5h;@�G��>*1����:}W+�R�ui��v�������iZ�.5����h�X|���V%(Rh��G��f�&�{O�P~���1N����@����V_���`�v[���`�vZ�`
�r�U��'���|���7f��������o�Gg�<�|_j��n�\v{z�������ALm{�m��E��c����������Z|6��0��af��,�3�|�W�~���S�i�;
��m����t}���/e��=�p�BA�S��n��.>��Q\�vG&!`���m.�Vn�L5�R<`�o��b�!���&r��E^�_�?G#n�*�%�R'�dDY�W�$�H@a�eA����rx,=�-��K-}�|�[^|1%����z w�	{FkB�M��E~[*��B�I:�����������		�.�XK$�F�>`
���=N<o[_�}j���cG�+K���z��|�����~�{n�	�Yv�o���}\Mm����m8H�����w:im����Q��;�1���]�_�6����A��I=�������!��`�Hm����m�M��������}���m�����]K��������
�u��)xr��������N��w�cv�������OA��w��'�JZ�U����m�~<����jUA��dHitPd_n�I���_�C�I��O:f���s4d�`<�]^c),nT�����U�G`�I�-�+�����0:��
"%���8����Q!+f���s��cv��M�����X���1�W����/���yp����{� =	CW�����PF�<!�Lg�\���jz��dk���>��\]��g��J�|��+�����g!�����9B�n2A���
�b���D�^s/��[���
��S$�'��G`����}�x���b�RI����_��G�^O3�z��<0]@��-�-����C��V;�������������6��{�v=���5�2������5?�S��:�>��x�����)m��%=��%�y���s	���u��}������C����T�h{����+�m��2��H}��-e9�������^Lj������"���^�a=b�q���uz�n4Y�0���1Y��Es�}[���8U~~���s}8s:��ub�P�`b���w�
gx���+��juuJ<�#RjPK"��;��}��wy�n6B�h/�"�o�r����Z
���O|�(#R�R���dgR����I�5���5/�����R�wK�M��\������I5Kn��-6����~�Q��������u�_�hh�w���x�#�/����9�p^�b��{MO��7�w��N���%�%���v���QM���k,G�/������8�:#N����m�{�[��M�c�m��@)�O��s����,Z���������+\������G�
W���w��:����T���-��BRy��f��Q�p4�D� �^���U��&�M��lM�����eX�k�{����:���{��g[��49!��<L�>J����B���W���D�DQr��;��4�pj7��+����'prh��

������4�$�=��0�[�������y���<���x�-��5����f���a�|<X|�qp"�I�]K��s�����Y�^��l{a�1�/RR���	�t���TT{��v�{$12��@�,�U� =s�u�W�0�z�����m�W8���vb5�>��+6�b����xU'Z���^�8�<�'�'��2�����	��������}uf���r*�P�E�N��r+�t��#�x>��|s�U888�����;;�Vj�cv@_�Ty���V��F[C�Zq{����f �u���
g���i�e��>v*R�q������^�%f��d�c�3��Q�z�D��?����N+
���z�f��t�G���������{4����wzmrx�V:�4�d��3��������m�G�o�vcv>�������?��|��O�m�NG
;�	�x:
�ql��v?�9���MH~Md��Y��?>�?�r�[�����F�!��}�����q�8!��e��	�(�����j�0{w9E��:>jU:Q�\���q{��rI�Yn1�W����W����0�7_A�g��C�9������|O��B0gY������.�o����a��u{��Q'�0"
C�r�8h�����I'��d���T��@UJ�R�R�`RfUJa��;]Y���E�P����>�k��%���[�5���?��N/�
���	F���D�t��yN�iQ�B"e�-�@l1�0�
�'���d`�_�Fg�EQ�o/�i�cX�
f�p�e����Z��B����W����7gA�*g�K/g�~�r�-bJ�:l��O����e[t�=A�����i��ma��A��R�7���O��,�Z�l]Sz9[eD�L:qP�U�C]�02���'��2w�ZGd���TL�D��/7�~t��kN*���=���fs�u=��9����i8���^�3��{Dxt�+]8��OZ�����X4v78�!^�;0J�K0Y�-� ��P�aib~R��,����	��1-�O5�����S�#���
�������K�������2�:n�U�Mme^_(�9 ��x�������-������z� $��I�����Y���eg2���Q<I����Y(���������T$��o�4��������@M?��p%J,O����n�y��E���j7��
�
<�E��Ke����^���n��!��S��/�Q�����l`�i�?���~���M���+v��o�U9��w"����n�&N�f��rt�"���-��/��������!;���a�2w������x�-A�e<3L�E���9�f �������2�)��z�MM%��K-��T�E'�u�C���@�% ���k��~��"�m�09f��s���i�����l���s��x"�N������8�)-��z����5G��S�@&��:B�w+��9�\;��V��V��U�kY���_D���l)~��[���%�c����"s/�x��'6_[|�P���p���� ����8%�|5=aW~�p�CP����1���������V�&'
����N���RC[%������5��a�SS���vs�1��-[�_���.e�����dF:�H�U����C:L�$2-�_<p���>K �����H���,���[G"c���< ����*_���6>��3���MP�A�+��6��3U0U4"�L�d�R�=F��3��L`�_>���������9,���!��7���!������'���-��!������!�������O��!"%=�&&q����P��{���~�mx�A����HX�����@`����;��������^��o*\�7����VBG�}��o(���P-�O��B�/��}�����4� e��'��;9�����-��s��#�a�W���j��el!��GGu��7*�7���8�W�R�Uk�GA�Z�����Q�/��
��i8a�/a|3��b���3��=���J�W�Q%�7*��z�������i�Q�������I����e4f�cV�<���2fC3O����1���~�0+�����C9��'�:�r{4xQx�)����fVbG�?f}�o�ZZm<m4�A�Q�.g�F��S���gW/f��'��_�P��s��QV}�o��SP�>��c��$��B�<���o�>G�)�����
��h��NQ��~�%��lM����J���
1��p`���h6��e���@�2�!��e:2�l^�S�����A���!��'�d��i�x��n��A�Ft}/\�>[�%���u��DS~�"����`6���:����a#�rT6F2�,�K�d��������(Fy��!�w�m��b=����a����C!����8"
�������/w�oCmeb�0
���u�s�L�H5��\�ABY��hFv:�<���`g6��"6����^�X<��P/���zD���1�V�B�s��/e�����Y+�o
!q_ISK|&�I
~�2�~;�vd�W	����EG��M��q�u;
�Y�
�P��5o'���6"`�D�����t�k�7^�����i�>�ru~�UJ�����xutD�)N�O���������/{�7�$�//���:&�j��P�j$������:�6�4>�p�%KB���-k�uT���p��$�mL
@�c#���������u����������`�!�h���<��_����FS^����	�[a�����N9��u�CN������A��no4�����&����3�H�����;>�����M{�'��p��h P�v�d�����u"��o�a���xj�4h`
,��z�w��fA��/�&������N�����x�Y�m�+!BY���J-���z��N��`�Ym�H�Y�0h�Z����Z
b�Bd�{�#�'7��`\�
io
��4.������aw��C����T0�I����EEH�'��9�k��G���c�X���R�>�z��d��Mk4�3�������1��
����}=F�����������?>����9<>��8�/6�Q:kV�Bj�0������������7���[`,?�[?����4`�8��e �`��"���{C�6�GP<F�p:����V!g��U�+R�U��5��;�t���+�"����^�}�D�N8�
�(�C<��8������g�>U1�/8P��P�PP~k�����4��Y%(y!�U��0�:�
�:
�3b�3Q���z�.�a\0��7f�Q*p��4s����=�!pe�fb������_�h9X�e+�'��=f����^U}�t��Z�`��q�Z	 �s��*�h.�(�&�t6�����
�q�q�C��C]�K��P
Kr��_��'�&+1���_���-\�^d vNn�#Q,F����^F*��(�L��`�	V��~!_��`������l���/o$"��oj.�1�T��(q�D{( ����18A�<C�8^�X����r~UC7Qo��4��Xr2D|�����[^q;�4)�7A�#�������[8\�'�~�D|��(T8p�e�U�r����'j���cn�����J�{�WZ��-�/���J�	~���%��'o)��=y�Y�h� `�(��H��8�2.�/^�08|!f}nLS?��9���j������t��F����������Kl��4o3�3P�w������;���k�-��^��K��N�"��\`�r�����C���
�,7�x����������n6�����#!���k����I:&����0/
��hjAM�
?��b�{/^�6�cD���3	o�Ce�@���%NY��-D��~��L_K����H44�%M�b�+��q�c�������/?��|T�>��Uy<kR�<�]8}gc���~���|�N>���ZA������3L��h���l���
F��2a${=y�� �(�aC������X�CV�Wg-P��4�QZ�>��^�\%��=���q��Cu}���>�C2k������'�B4�V�QFL�J�o9��s�1c�����(���[Vv��Y$!��!���B;#������a��)���uhAc6���������z�	�Ec����Ol'c.=54d%���G���96���pKbZ�N{6A��}����$�`�������(��!o�}F�8������q��"�UyN���y�,�\�
W�����%3b�@�Sn`����Q�r��K�J�K��FQoQ�-�8]u��
�!��1^�(��&��&4�J|_r�����I���M����X�y�t���bi�xC7���R5C����5�A���",}Z��^�rt��U5�����Ir�w&P[b`�?���!��CTXZtP���u�RS���{*��@���&��5k���tKtf���><eg�����'�	c�$W�-�Jm����z�<��=P������\`jO@S����J�F��p����^^?������'	c�2m`��'�fE��`�_M�_}��
*s�3��Q��%�B#)/�S�@Gmi����vi<�hg�^�7����M}'���w���#$�+j�@,<o�6���0ve�� �O��sd'	��^R��%��t�Rk�[����C=}�������H~~�������gC#����)��Q&x�P���@�� �c��4�9���'y�r������3����4������Wq�e �f��C�[��
,��#xb=?}���B���(1I��x��9f]�����_��G{zE8�"�����������	>�a��.x�F%yZ���������/��_���_������=j���Z���"�S������H�-}1d�[r��W�{���<v�I�X�EF����!��EK��������/�C�W9����Nv8��e�����MbC�cy��}|�4*�*�y�fk*��L�n*�K��k�C��3k���j�+Q/][��Pq����g�Z�u�����Y���}*��?�J���YD�
�/5@�H����pz�=0���w)C���=�%�nc��`�C��M�c��w\��Z�n�3DMh�y�mX�2	��$�L��1+���s�����j��*�y�{/���@j�_����TBYC�����H�����D�1o�T5	�B��1�Xj
%���y�%�=����y�p����*��->����B1��F�"{lnH<��/ZQ��y5��lT���[�W���;
\��1��st���@�l
6����z�`��;/�0�	T7 z5�C�~4D����HO�yC����?�P�,C�����������������]�;9=Gk�4��{9/�6��������]�^ �}��-"Z��<-y�7�O1�t�2H��[�����c�BJ]�V��B�����}�u
Aw����!�d�E��������i$��{����l�]H��Y�|���t_x���R�{H3=!������r$b������9�
�#�����r^I���4pg3p����#T ,e��*����SH����pJ}o��)��������:x(����!�l�F��K�@���2����������xqh��@�,| E
gl	�B85��$��B�����F���a'�H�&�hF���v��^��(������x�DA]�"��@h��!���[�e����0O\�x���X���h��!G�*�Bfaj�-"�h���X��}��iT=8x�����]����;U���W�Q��f���_������_�����n�2C��`
(�4|({�8�=
��7��%zW�%;�����/���00����rWp~��B��6�a�V���I)�g������-����-%�������9�F��^i!Ma��z�M35�zv:����"���^�`*��"����f�EI�Um��Tm9�!P�Qc�BA���0=�3�Nj��P<�2[�QN���	���r��S��;U���v8O�����[��n���h��j����e�'Z��v��B����~�=�5W4dB���1ft�?�� ��O|���xi����������e��$�����2�1TA�!SQe��e�5�fqDW$:�=�Vp���o�/<3���>zPw�����^iZc��RP����Vex���#�22-�/K|6�6���A��������b�����.2��5��fk�6��|�?��D��\%�M���~"�;`;�Vk����o6l���oAR*����S�/I�u��K�o��O��:��0
d������~d���S�=�"����t��d���m���`��k��*n��My���^NBN]�>�L���y��R���g����G���
��7f�} ��wf+bC�h��r�����n�	!tB���I�S�'�������Do��������`4��}�O����T�z/�.�����G���	����B��2�ELQ2\�A���P*��)���6�s�9����a�(����m���FX���hQ9yu����B�Hw'��A
�!h�
�q��I��(f�)���DD=R�Z�@�l��	,�����<�+s��:���T�\}d���e�Z�����"�H�<�-O�Hle������f�����h�4�&�4%���8HT��'�B��bt���,�A��j+�D<W>�j]m���<��#��X�JE�4���=�E{��v��������5�N�$o��W��
��!�e$b���#�_U��������J����M����&R��W���Gi�OS�����}��t�������T��$Ui:q��b6����p��3}�8jY#��������5S#�B8�gv��*����'�
��-�-�U��Q�Q�jNL���(�U���M
	|#��}b6(�W��t�r �O��
x�_��%���%������?*�:t��
��X,O��z��u�~U�5_qx��A�//�#]��w�B<kC���_�`QP��+���M��R.��F�V5��Mm�����Y4�5(�X���5T|&�'�}g4z�|s�N�g����{!��2�&(A�Y�>PZ��l�w���*A��e�*���#�����	* �����@�~b�;%!-�p�4lA�<oT�^�s��z�Y���HL�K�����Nm>*
0�a0K���P[3.��*/�L�of]���+A�������M�C��'������sPb�2��qY>x�7��Y��4��Ev�{mr��G!g�D:��B_�8F����x�n� d������+=�g���Pc�D���w-��MI�������s��bI���b��w�7]�)�#�lhOJ�`�o�+vg�3�&�vCQ>	��C�7�$�����2x���.5Eu�K����N>�4�<N�pN��^�&���RS�A����������{=K�}L�N��b�N>���w��z���k��_�����_��m.��}S���y�[�^9/kjr�����v�d(_���$�E���y]l�&������y@�g�}=��|[�w�d�|�25���ac�x����t4���~o���kl/s"� �-�1�+��X�:��y���Q�4C~���qD�J��/D�sJK��W8���@������=�p�K2 ��;�H�r��>L�����$���m�U�K�D\
l9�]
����
9������Zz��V1u�v��;�����B�3�����
&�B�>��H3[�"�����U�B�T�O��JG�).Mf�*SR.cNe)�`\N1,��M�nX�9O);��y!����(�/p����D/�r�/�n4��M��
���\
i�� Vp���HOZ�"8�Pp�?
3���_���l�V�7&^���
�OS(v������cby8>b��R�G�#�T�qFm`v�1YZ1$�������&A�	�����9�������]���A�
6%����x����"��2|�����w�P��Gs`������&�Y��2N�*]��'������s�A�k�9-�wA���Bs:��;��1�$>�8]�a����x�!��5�&����,��7lO0�0����i,�T0#r����D=��v,v�a��M4b
�x)�S���iA�:�H�P�.����pbH���%�D�u�=�����I>3����m�$?���xFV8#�{4g����u�Ia��`as&�\�;e���
���z%�24v���]GS�R&3���3P�&�"�c�Y�~R!`/�b*��-�������������s(����	���e�c��e#'t������b<r�7]�D�zu	c]<�jk&�D�]<�����%��D#������y���mD��T<7�Wq�I�K��$�=^��p0:��:���~H�O���#�R���R���dO�4"��M&'�MK
�D��s��������"i�l�����In
IE�#�lf.���fd_����3��x
5��^���r�r�l,#�%�$ZZZ)`RR.`�CI��+��$!�I�c�6��,��'�z�n� �-�e�Mb@�Pi�T�@�(��"��YM�6z	B�rl�d������:4�����h�
������/��-�H�!���,�)����������g�]��L�ef��h��F��.Rl��}����c^o((��.��=}��`��Q6��?*��35lK����D��jD,"�����!yR��$��F�I�7��D�����<�z���ot���lZ�zv.��mL�~0����S�W�^�/�!����C�`Z({��T�`�z���l��6{���B����y8:C���_��B�:]gA��%���%lb�Q�T����,����������Sg�Pr���d�P�D
�iDSR}�MG���(�DL�0���a�������7������9��#���Pz���^����S�w��z�t!A.5Ei4En�pJVk6��S��}����H�K��5��!A�&�����{d<�K��U]�(*vf&�p�������:�4��I�$�/d���������t/�c*�.�\
z:�4yW 
��(h$��c�`&��=�����U}��J����z]���2#��A�z�'d-zP�`���iHV�����b��s:1B����^����c��milZ�a+�)`���u�����%g��<q&4e`l(j
'H��$C�d@%�1��0�Mo(h�<�����Dp[j-�3
�}�s0��!�7��$���E�zz�yj�����~�x�27��){�%L���t�^���2������0���j�P2����J*"�Av�L��t���P�����$�r	�d�W�4$����� ���qN��6GJ�����9���������FA�V�O�`�X�hm��=4�
0��>{��nO����p��\@/�Y���!)5�(����?8B���ExM#N����D�9�j{�!	8@����I�"�F�0S
��;w�**"t=k ���[���`��F:���PY��5�9��g���g����9v\���[&?�sm�5@j~��x�����&O����	�q�&�Y�S��\��J�v���I�V;���s�����8�^���*'�v��U�<?�����M�T4��x2MK�l�?�����@��7_�1��%�Oza$:*"�F��?��t��w�����0j����p(]�����+O�m�Qv���t4c�;�^�A���X���v�B��x$��j�Q���1*�,�p��:�,0e���U�>'�:3�`�XV�;��M/�@:�[�T��H�S[hv�1�d�i}�R��qRj�7��4qe��=|a��yu�����s�dT	��D���u%�9jpV���_0/
����u�*:�Z����Jf�%O�25��p�*��Q���#�[��.&�MPw
,��-d���e�����,��0����3hg.*�r;��g}d/A<@�	���c��!��9�T�s'NK�8%�vF�o�M���#�6�^����7�n�������W�����a�_\�*%�+����tw:��P��UyX���OZ+��G&m�&��(�MB�� ���������*D��_Q��;|�	�?�K>�{��j����(3r��"��2��mi�J�	��L-�����Z�}'�S4}��+1�����sp�M���9���!��Z��jt9An�V�����E�}�W0(��f$�x���Af]�a>6�A��������3��rt����Mh�;D��CO��R��������+���!
1��.yj��Na9�0���'���p�$�aR���H��|�����������"9�J�a�4X�����N��$(�E���{,5%�����;J�MpCs��,|�;�&laa�#�-Udt��4��Z�`_�4I�/~Z"~zFr~�qu@�p����3N�'������,H�����_����EE&�%��(d���*!���1��`o/���{���b/dM����d�/%�Z+q��^�����0wg(*���N���|��gf<��\<Z��������|k�����I1�8j�@k�S�t$������`�#�XZ*��D��lQT^%%�e]/r�b�z�����
�r�mI,n=h���OK���8���i���X�@��
�}�k�-����i
�F�D�M�f����/�lI�`�"����d�����Q�~tZ��<����CUd����r�FKd����ll���(�����tK�H'�i:�ej��G�2����=XI���C5Fd
�$`o>d��H��4��82h'���4��ei��������7��%�����o��V$����D�����q�p��R�V7��b�i
g����`�}/�Z,y�� Z�����I�wj�����c|��T�����G�(CNib�v�Z��r�sr�����NCYRJ�(.�Q@�|�����M��^&�E�)Q���L�b�d��As�����m�
�D�='�����WZ5��7��o3��wz1D���du4�G>���<�������Rp�f���.��6>�l��}�eW������E�x�r��������=�������h���ZH8���'O
eI�Rxp�J!{e!7�IF�$������
�t5E<�H��E%��}�]�N1��l��5�Y�o�G�)��"���(��q���������C�h�����
�����"H�<�3E_���0������"���e&����q�
<�b�;c2�+��
� �@��f>	�U�&`��?g�T���B�3�s&b�.I�9@�/�PZR��m?��f�`W)W��_�3������3�z���(��|9]��1VC&e�����;��>Y�o��%|
hc�DN�@0+2x���)"�i#������#�CN�`6�\6�����&?Z��F(�eA"�U�d{*�1C��.��#��T�+�	�h�(X�>����	��{$?Z�_�2N��R�,������)�a�f����A�O���<#t5����P�$1����#���dQ7�[�r?�������/�S��'s�5�%������}�
$+	!!K��g���y��*hv�RMLRd)*>��Q�D���Q��'�p�G�O��7���(!����J5�N�6:��V#I�f��	��R�%OOKuv��%9M���x�]��SoG~�7�3~'h��W�x���qG���SA>��^-R����=~���N�i�SU�
:���{�Mu����u#�P�)���@�|�/����������'�W)(Y�%����$������L�+n~=hC��a�&Pt���s;�������5^�Q\���()Q6h���N��j��i�I�l��4��)-n��NJ
D�����/���ZK�o���[�/�]+���Z���^_|x��c��c�'��ec
N�g+.����o�y{x�<���G�1�|�0~��kR���O��U�������(���
�:���9t^m�k���As//������u����k|�^�`�u����K1��Je�S����`��7�/�/������������Q�G���G��U���>5�������bh����A��}��
��?{���Gc��7��7Xx�����%�KnU��������������g9�/?\^]������r('0�$�D��������� x������W�w�hp�I�_�+��`.^�_��.��xvo��V ��k�����]BL��)���oN>kF���^����]-��[|~�][�mM����j��%�W�Kr����hx(9X�k$���Y�����n� ^y~��+
�G�C�A������B���g����"(�X�9��%���4�P�%���p��>\-F��\\�}O����cjF��r��������\�	���s	�%P�a!�h!pO]F��!�~s��c�T2Gh@��n $*����+	�!��s��g�>�
���Q�p��*�hh7��	�����8�������[V�=��`���!c}J���v\
s9kq�2�N�;}�K��G����d�8p���s8C��X�������}���s��rE��9L�o����fX�G�Crn���w�����L0����fO{A����������������{$�9�'b����/�nJ;����N�:4������T���Fe����hS�C��=���E�Av��H,����>$
F�R>d����>�����.?����gJ;I�����O"��~&��I�A������m�'q�j���F`D��NA�%�o��;�?��E~�=:{��?�#���;Yy�`�������G}�=��m/z�/2n�x�m.1���<I\S��\&�6��^��S� X�:�����:wK3�0�
#8n��

������}���e������	���`�j�m���R�����~B��&���40mbh��1q�m����_>
j������7���IdsZk��)C�+Z���)�6��K�u����vd	
�����[���,��u��qK�B��
��^S����q��2��(���.���j'��o����wyvm�H��8�h`kD��p�\vF��`<�-d#Y�+W������?8���Z������F���������
KsL!��)|�!�����h���P�I�@�X7� ����]=nT�o��v��Z�����F���qj�V����Z>�xN�K����8u�����k��nw���Q��:
*��jx�:it����:��O�j�=��v+�'�/�1��J�)��|	��Sv&`�~9B�$��-�����d2�p������E�g$�����*�l<�_��j�ip��Za�F�R������O�?>�]���UN�TO~e�z}Dl-^�;[�K�>��*��SDxh�&Q{6�{�[�T&{�B��p*b`���o*w��B}��G��XH���$��ny��~�&�����EZ���~�)J�>����������`�����|��i4�k�9-B�{�x9~�&����<�:���p�Y	����B\�I�!��x���8�����=��+���C�Q��}����=�?a���j�� ����j�����������J
�&������F���E{j����H����>��6X��Z,<H;%v\g�R�X��/��D��~�r��2 �}K���'A�i��V�V�4�V��_FKsc~eAlPj�N���N�c����T~^�wi��p���"�A�$��V����8��6�>	�ws_;���1��{��E��
3�4b|����}��Q\8�\6_������9C��������7����!�
k�n���b�m%�\�V�r�]
6Bn�q��+�D���1,�Z�b���g��h2�_bJ��NeerS�6����� SL
?j��	��y���y��������}�����|kqW>������D�P����!��������N���i�[;�Wr���T&^��I�q��������( :��;:��%�c_�kC�$o"�{�����,R0���9����ipo��H)�wD�U�Yh[I��W��d����K���4�������
�a���-U<�8=�A2
�����q�`"h�zO�}���`�q�tq3�r!%�A�B����N��kj��nE�=���T����q����I$�� �@�"�)=�qKA��$����[���y�|�@�CE����e��7�>�8w2�:�ENNud
w�{��CrH-$���:h����*+�!��+�;��M�z]�-�I��Q�������%���U�7��eg�������p�&���l�^S��F���d��y�� 40����8#���zA���hr�!\f2����NZ���2 ���Hp�p��s�z���w��6����F� �C�<�aq��u�7$����;p��TD��_>���h�*���	l���60&�P9�a����@�fS���rk{�p������6�rY��|���jb����U���9�����d":#��p�Q����4k/O�l���'���,z�Sx����8����#OCI����F���OV��C���L7��CQ���M�k|B]�)Y����G��L��x���pR6��(��9�0t�4�s'.I��3qZ]$�%��&>#��s8)��
'��������AH4���\��4(N��P����t��N{\���3��3�
�	��T	l�������a�"��HY���
?��3�uY{8��{��W4d=d������y�O�*�ie������Z�Z.�C�U;
���
$��E�<�Tv��=6�]�1��I3�D"�S�����
�ls�C�0&�w�cJn�o316����tD��s���G��^Dk���_�S����!�����\���d���A����+���m|P��mu�:Q������v\���Q�~����O���QX;:�N�M��5�����O��S����?�t��Y1����f�
�m��xR<$��B�n_��)x%�������Z�];�V;�R��:��u���q��?N�qB�J�P��M�@�e��|�0�Up�(�:6rbd]�Q��'X�xay!?���eb��EJ2N�*�P,�q����8��Q��������= 4�o����SH@���d��Q���@*a|+I&a<,�v����HQ�����j���'�E*��"�M�v8#��R>�X���
�������#i?�a+�	�)��$:�I�!�k�`�0�RXp������LUb��(���SLUD�=��!�A?�������v����4�/�7)_���Y�b���uyr�eq�~�L����w^�R��hq1�Q�X�%
�B��3"�t�p\g��������T��I4x����|���D,������x�6�J���^�H�*��>�r����L��9���e����c=��@t@�\c��!U���#y����2:09K]@�����l��`!�L�b��9��7��'��P��IQ�!��t���N����Pb!$G���QU���l�V����|��/��
����'&���G����0�s������+>�j�
F#��'�e��L��S���*aG����N�J���'��/��,�0v����p
J	��q;���r���k5OD�Me*C�����`�
��8)/X,��xO������W/T���&��������c��_��'~����P�[=V^wG����%2~�)
��j���q����$N����&e�|�����?~�#������)�o�`�N{��-b������Qls��w���!P9�5��d&aQ�� �.HE�
�����������~�x��������R�*�6�
���n�C�;J��k}�.���T�.q#~Ga7�D���I[�_(���R�\�&��[�����k?�>�M���
���G}j5cDf/�{�Mv��gC	C��Y?��V�K
 �8�9$�S�5�.�����*�L/V��u��H��`>��
���%�j�V����>�����F�������w���ACk���L��X��\������/��������3���-�:�H��vv�~=��N�>Q*���o���b�kM��Bm����o����	��=k%#��DjH��"�������<~U1q�wj���4��:���|�vCs�|]�JP�X�������G�N�K�NM��BL��������~��.�dfF|
�!�/�m�B��U����� ��0��ET�b��`�W�G������#� iv_��{=����z��$|S��`�{���q�����_�����
.NX��F$+����yO��Z��,�(1�	�z��&(8�3�����[����(�=�����.��U���Pp���V�����?� X��s:���{�6��1/*��'?g�8-���Z��f�������W0{��F��-Z�)�� ��f���mZL�����y�a	gc���tO�I���x0��C�+���Z`�@�\�"VqQ�}�U�p�=h��u��v�������hmYww����m?��DE	�J�!,1�@�)H/d�T5h��S��@�9���!�J���_{���H�McGo�>?���]���2Y��Lfc�Q:�T��[��R9H��lJ�I(�x	��+�M$$L7�|��U��&������P�
�l�/;`U7���=m��om��	<�U\+v&��!���`}����cfH�����(	#�
e"Z���	j�d�?u�<�:~�>��<p���B�I2���A1R���=c��!�S����f���0�P�d�4��bi���]�vP������S�,�p��5k���r���,I,�l.d���H�qUY�C���K'[���H�.���qA���B�i^�jR�Fq"����nN=����3p�00D	�d{4V6a&VI)�@hX���/�Q�@����)�O�<��:(�n8n���5��s�&6c^�=�5��mQ�;���;V%���pH�%q����0�fD�qQ�=����bT=������R������w�"SV���Ga�
�^��$t�"f�&����9��%����w��i*� �j9�r|���60
Z\�!(5�N'�2�+G)�y=��.���P�=?��.�����$K�'��d	^:��
$����������C�g�V�K~0���w ^���e5+����k4�d��.O@=#\�`�� �E����H��R����T���v�{�f8�k�XI�wX1���0h_5(W�D0��*NC�g ����B�@D�������C:`u��H+��0�A�M��z�@��s���p�)�Hf�}��RE��I���E����*��n���m����G�3�L4��>n�!@~L<yP�.
rzcz���RZR� Ur����"5�3�o�Q"���}�}Sd��aP9%���V�D�D
y��OJ��d��'%�@����:|=����T�rbL�q�����V
�����7�����-5�����d}�^�"_���I�]Q3�1��r������Z$��=]&����C�
Dm��e���#�w�B�S��p�p�L�3�}����{Sd�P��%���H,>��X[r�
t����&tQ�*��
R�l�����5m�����u2�[�d��/��l��(��XsA��*4D��H4`r��p�E��b�f���S���D�
y��Tb�r�=y,�������wF�Agb�t�����Dd*����\-QG[�I�,)/[��������[�	 5@
Y�v�&����n�sp����*�����4��6������
?����u
���'�����(/Sq���9�k�� ��ef������	��^������'�q`������c�'G�H8��h�A�]N��h,�]��3d����4q�q����	6��@T���q�(���7�q4D�����#8��o� S�������Z�T��!�a�3�{2��C>��mi�IdL�	|R#bf�L�'��-HXH8���33�"5��k������4�VX����{z.4^����*����jKX�+��T2"6Hy�$q��G�|P"Y�jL�B��p���jN�� 
 =&��
"dbz�g&����N�3?Cv�7���7�?M)v�m2Y�i,��y��3�P�2a!�t��{K���Jxn8��B|�����p��/�C��8����������� 9��%����� �"B}�P��L8����p������8=��&B�8d����Ep����9?���'��Q	w�t63�[�e�J>������V�����k�'�Yj�vi)P6� ����
�L�QH~�8W9���,�c�~��|��A�F��h\�
����d�;z$�ZfYBqj���9`����`���[`T��{5�T�>M1��&
�Q�]KT�.���,_Yy;�M#-�W��0��^�_`������z�H���4�T��T�L�:�����S�G������xA@L����G�iIuA��	��'nm��@k�)iF����%����|��<O������=��g[�O�u��C+I�<�����X7������}T���5;O��vlO�R�u����R�P�2Uh0��u	���
*N��P�#��_-�g�#2?�hi�.�J[Tp.SB��
�a�8Y�9�T�j����+q�������f���b�C��X @)#���t�@�	���q`������o������#sS)[e�S��_~����_��4\������:�@\��������s�>������W(�'C�#������z�!���P4u����^BJs�k�8�D���(+��P���1������O"���!�G�O����3/]�p���O�z���l��$q6���f�I$��i�S�E�R�����H��x���)���3���B<�(D�?�E�1��W���%�4\������V����������<t�YH�*e�$T��"��D�

N���u�6�9�!)�H�r�OM�S\S�'}�����DV?�.�iTB$�$^&��X�����M��=�|�����\�h����7����O+8�F����F�0�#5�tSS���U(O�&_����������g�+Y0<g�QD��a�.-A�]���D~�h�p����hI��H�O�������dc���WL�b-v��KD�-�qh����1g#8*�����?"���D�C�ba�9�Y�~��H{����\W5aFZ! �
��0AI;Y����w�*j�%�m�-��]
�B,�0�����[+���p���+�q\%��S��b���k��A��Rt�OQ4��:��y�����q�
�M�0gR���W�Z��]qf`>w�{4�J|6��/1��%�rxK���,a��:�(�O"�Sc��{��dE�S����]S����o���#�W"����F��%�"��8:��ak�9������]�Y�t�:)c�)7=|b�h%����],�I�_�n����aty���B�*�R�L��F?�R'Gx�����Z8��p�Y��O���J�+n����RzK<���������*"�{=B���������?��ha*��J�v���W@_���T:����J��WN�j�����Q)����J$u��[����e���R��\:��bV�����Oy�`��)����C�������c��'l4����U`����HXt&��`�tQJ�LL�������)qe�Y�Dhe��M��8t�4�K���i��-I��!�QjKhQP5����	��a�wA�:�� k�w���}���Q�1�0�91%��G����n����?9�����(Z�����!h����j��74w�P�l��@�<�fy����G����GT�'!�=��*h��Y��j�g�8��V8�0�E�&��c��<�\�0F��"��0���\���'��-���g0+�����?�������XA�0?����%���-TEk���
�D����+S�^j�,S�����s��c�|fKj%�a�;@���p����SF��sR[YUH�w���|*B�������N�
!�=�;�w�n�s�;E������I��o�V�z���Q�FR.�QWu�W��]!�6P�8X�����q�<�07�]�0w��UN��������g���?���!�����;���|�"��Af�����*v��Z�sO��������j��s|\j��z���m�k�V�v��T����r��t���j������G�Q���������
�B�l8M���z�c"�a5n����<���u�9T�Y�1�AD��El�*�R�N�C�.#H����8h��� 3V�*&
�O�v����Q�z������447>�.����Q�T�t��zd���q�xQ ��Z��&%�o(���a�Z����:}�<W���f���RB��U��6�JS�C!�>s<�t��!4eDh���p
�uj��*���F"����	��d���BH���8"Q����;��������������d�,h�������,���4Db�$�$k���"s�
�)t��C	��/`���c)8z�"`�Vu���LY�\lD�����F���X��R�CGF�V4j�0��!�6��&��P�����n"��n�8H1�.2=,3P.6�26�a�2��\��/%i	��&�L�����P��(M��E���	��������n���ZH
������m��l�<���pH��2ST���N	1|��uyip{���������7�M�^��Z;���������{����q,�~&~��{m$�H)�]�L�:���R���O����
�����z�k�g0)��!�%3�������U�U�t���U�F�e������(�/'w���f�����V��}��������v��^���M&-������Y���;��y�7G�Q����N����~���j����({$2�z��V\>^y�7����A)1��d���;=��r�"yg@��;�����m�����<��b�K�A�7����n�1�w��R���vKz�,e���Z�7C�d.�C�\��t�wxs]#AH-���D&B�(���G�VS���l�o���k|a5r�hv:�;
�[��%�Y�J��N��mK-{�V�/7�1�O7��~�"�����xS��	��D�"����P��a�k�t���-D�q�Z:@�q����	S&{�
	P�~-o��l���y�6��!�sWr�������I/�1���*����9��xR��Q'�ga"����D)�Wui�I��'$q���w��#��IO����������/����p�������{���F�3b3��������? /^�������}m����������v���Jv�������$��x_1b����]����;h��-��"���*%��������=�b
���������!)>��T���c��6�v���K�5x(��P!\'���������U���������S��k��9�s���4�8�S�K.��C��Xv1�\���K!�v9P]��9��=��`S��"X*1g���y�����);5~^�]#�t���m�J�����U-?��9�����u�����w�a�-B����{a0��9�6�������i���s�O�\���J��bQ��i����eH���8J�P|%2�D�l��Q�{9�f���a������t��v����Y�#4x�3�]i�5��K"V�w��/@{I��u/�d-������2bQ������6���i�}��O"���K�l��������R'�%��D��D��k�u�)���rT�P����*`��j
����k9[Rz'*��.���A��NB�E��b�Fw��/�?�������I���)����/z��e��S.3gJ(���c�~�!�����~x���	)o�x�K��%�)*����fj-o�������;%�*Y���i.W�*>�yNk���c��yo�r|�W�w�`�j�����U�7�|��}�\������ft�a��b��4���B���%�������AB\~���������T�@e�(D{4���h?}��t�e��������I�W���y�M��]�d
������QW�q��uE�k�/��jv�3��&G��n�
7F�7�������*�n�OX�����t[�f;������/�!�����k���pF�Qx��&g�N�Q�<�����ps���^�������������a��wK�$�s7?�
l7W<$tg�/97*���j�&�q���s:s��[�/��E9;��GG�� ,����f����������.��N/�%��R���JK�.)���3o'&:bBi�
o�������bM�
r���V�.'F=�DWW��S3=R����V��<���\��DR��Q�R�\��D�fT�|�[�<��d ��p�y�NE�u����9_�/!�&�
������b�~�,��RG�4
/�h1�q84����8
���\>�W*x��X�����^]/(�T����">*��"a/c�w	=���#
�0]T0'�	�V).R^��h���5;�P�qL��*���A���T����Q�� ��s�PKHe�o+��������#�~':�|�
��c��'�$�WF�nQ�t"���3`���������!���t��
�.W�|
��R���J[�~�[�lJ�D�-k?��>�d���wHr�����n9���4��c�'����g��>� &}H������>t$,i��W����(���C9��,������P��:���h����������=m�C�Z�����=cA1�z��}�����	&*��5���s\���T|�
D��N�Na��1l��l�0����&��&@�������1�P��9^�q������Rf)�hV
��mp��8�!��
��;���wi������sQ��*����.����h�����)-Sif�0���k��q�' �7������,�0X;Y�S��JO���C:7+Q����A���F��z	�=�+��W��>�xq�A��d'�.����_�[c��]H�f���jlLW�d��]���*���w�W!��V�3|��3c�+�q���<��e�\����e��]b�"m��x�kN@]�D��/1��8K��U`�i��$�8f>��;Y,7'�G���
;��S�Y���F�V�1n���q���)@���-j��b.���_=[+QF9�kd���r�{Lk>��/�3l4����H��<����b�S�I�lnW�8� �mG�<��Y���=Nr�r�i����/�����&S�GC,�,b��J��$�J�x*Ki���yV���7����O�.����z�)��\h�]��*^+��T���P�������A��Z���$��q	b�)��J�p�LX*KF|U����pJ��Yy�7�����n7g�������&#\*��G����1��?H`>�:s�I��'Q}u�yS�5k�� ����1�6��a����9Ar����H� �#O�ncr��uGcN<iH�"8��t�nF�T�`2�ZM@%��5B�RgG���0�Y��L8��lP�Qg�"�Q9�L�� �Q�^G�-�(ULvlj����m6R�h;�X}
� ����uV��r�	`Pp�	g3Z|�������o6]|T��Q"@�_KOX`m�{K�f����u��)z��J��)m\Ig-I�{�J�����!�j��r��k�&6�,�L6��G����+JV%�d��Ur�O�������B�������l;�iW�;��	Ad���|�����M��e2�7�W��&
��]����hJ�7�T��T��u�Q�i�#����\YE�})��Z�r\e1�|�2]�l,)��T���v��*��u�
6	�"���q��F�(�Kq�(�pf1\w���;�w��f[���/��	�(-����0�&���XBr�H�;����Do�F�u7y��b��)Z�����
��}�y�����./�/T��W����F�]�S����>5����.�W<t����	pjF��q���7�3��b����I��K��%���8�&<����c�;(�n�S���^��~���Pcz��&�^���%G��T)>;�)�Z����*���s"1��F��������%��\��o9�����>8���Z���l%��Q5Y�@�x9;���Tap�����v�: ���c��U|�-(�`E.UT����#o;�O<g3��"[����r����)����b���� �N?hfN������xT��L=X�)+w>�����Q����A��g��bi-qf=��d��jP,_5�~��6���=���}���P`���������b(f���z�<��������u;��@���~�`���	�SGJ��z�#�N�=��4�n�l>K�����c���"����R�^�F���uW���qO�J�b�������NT�dN�_+�%��?q�J�����]/9E�?A����6���+J����dky2�0r�&��S��	�H�f�K����]G`y�q��x��������F�~W�T70w� ��Z/��pv+�s�����J��KY���s�c	�+�A�����$����<8���a{�����n��`T��	���Sa�xVz��.|+:T�����V��U�����)o�1E7�(f��9'w
rY�����s��n�#�%��d�nI�s	P���I�dk������||f�����4��1���H���y�_;8��.���d�(n_0aD.���b�����	��I9/l�e���n�)�2a����Kv��}�]J�M������D����L�����BP��,��F3-��T����Y<����U.�J����t9.E8V������++})���\��Nng���i�e��Ofe�������Ow�,�������3�z���������������Y\��v�D_��+�|��us�:K\����"@��~��S2P<C�q2���x���!���+�0:0�T����Sk�-���u��{�1�fir�T6kt�����1Z�0�,
k�]�:�5�G�-�*d��c�E#8����ib:*wx�L���S`�iB�X6B� �
I�JdM"T�U:e��K����-�Egh�����,gX��^��St�#�,��Vlt�����De+5�X�;��G�r��sF	�VX�5V���.���,�r����Pls������x�' ����sh��	0'��Wu@��P��|�$SS�-�"���gz������4L�_w>��=9r|��B�r����@VW�0s�2&PJ�N1���"�O�N���sf}��� 6U7#}p�[���UW^�4N��4���)�v�0SN��b����n``$l������ ��$Le��N�N�0l��}��y��9�8�X��X�����[�������4��a����(�7^��:�6�>��_-()&��3�M�w��bY�%RZD�����w���G(Q�.�$ZXc��������5_��3J����D	?�b��zyi],0��l�ya�6+�y�g=�w������-`�o������=5
)@8��[��/��}�Ij�p���D�ab�B�����y$U����I�DU`��%�d4�l jat�6m��2�L����
=��E���*�M����N�)���.�b��(�R���<$��t�!������nC�a_�������%��\��C��2A]���q�G[��M�kR;���_������:��s+���iR�8*Z�d=���c�-]�X�6��qE@���a�����'i)O����u!-��ub�"�p���P84��%����j���j���q��?D�h�� g�������w��b�Sn�fVh}��I�g�{���A�=|���t���*UT.��Ec�������������H,�G�v��m_S�`e��o�4�������'��UN����h�{��������~imdL�'W�M�
��l9��m��$��z�����A��)j7K���J.�����'�;��uE$|��*9���?o�R�:�]y<��Ll2��������L���k��.�	i���n��/aK�&
i�U�����e-�z�a��Hu��������8AO&� �N���������.�DPZ��t����)]�0ESwA��p��7g��9�D���@;����|5�sVk�x9��;��������;n�J%4������J^���@i�j���@���#w�4c+
���z�m���Yy"?x����c�$[q���D����),f��H��d��2�4I�>��]�b��*�M������Rz�Y�gL�y�)"<��� �{R�TB�8Qp� q8�7�@�PQ��L��K���?a(d�%��"�����=��j��E�T�"�.���U����@��y��l�]�vG����U
�jd4��l*�e,����N�i���������)�S�b���Yv�x��d�<�sB����������F�e���?��Z���W����kUs��Tn�xcU2���{^��:�N����*�N�;n�NXa��,g�y#��f@���g���N�)�����]@G�Q��C�l��;U���y�����C�"tg��NZuD�=��4E�J�u<�O=�ea@���""U6G�n�|�hL���|8�J��@�DYX�t�g�Z78�/�?���U��8�<�������o�i�0?���[*E��	�!db8���N��|0���w�_~��S�:8xR�l�k�(��������a/���
PH\�����O���|��/�w�Bw6T��a�����Q�R��+��E��)���m���(k-���48t�4si�7/o��
�c�H*�^R�)�2�W���O��~?�l��_��W�:��I�l�V��:�TZ����,���X?,�k���wn��f��z���%>��6���<��[��;�:�Vs2l�������
�N+j6�����y�����$�s>n7w�y`���"�!��1��A�HQ@��e��53�������I�)�VC	�����1�>�k��^"8'2+��������:�!����D�}�
�"��L���q�~������(�DRTQ�B;C�������%pb�����v��%"(���3����M��G�����@���h�[Z�0],0���z����B2k��f0��?W���Mt^��Bt�wf����������_~x������><��5��e�]�9����h�c��+����v�B��C�*>1��(��"+@;M������xq����{u������4��n/�����������e��F���^���z��-����fd��3�������D���A`��(xY��q��^	Y�����VY��1��$,v���3�7@:�%x�]4��;�\�?iu��F����z����}�����������H9��Y��	�����(�JL-��9��tc����{��������eE�e�V0��2[������ �o�)z�,
Q9a'�u�����p<#6R����,��xsS����K���
��2��o�&9]��h�� 3jSH��R�+e:^���2Hk�T�^`��i5�l�����(��iwZ������FyM���-�2�9v}v���<�N�)�����N��
�pYN����\-�*��'�V��5[��'�_�B}�.E*�3�����z�/���e��I2X]��!����`K�2�����N$x7o�$K��wS��wH&�������i�teJF��N����R���_?�����v�O}���V�����3�}�P���
l�I��?����%l���/a���?��������������7��N�uv�v[�d�>;��>ji���3�K�i%��<n�=�[?���K��D��,�D+|�n4��3�x�����ix-��n7��vx�=(�	x����%>BM�l��k����6����r�S6�iTPWj�0�E����h4�ZW�5�z'}Vo�7����>��Y=��c��a���C(��r�v���@8�;�E4��������c�i4za��i����hz�W�����q�n�0�]�M1�*��bA��o)P|HE�_|b*f��b;��.�����
KAZ�Pc+�O�^\^`$/:����5��U��`�_��&�����>��%�|����j����`����{���|w�����x��|�&�6����������� V��L�������(����b�V;���o�(���������9C�����.�<b������M����;q�v�x�!��������x$#}�������i��A�E���ew/���3D�� ����f-������8@S����_����z����x
�K!�)�����t�Rs�Bx�Rs�B>����������{��K!<d)��K!\�R�� �`z�Rhg�BHK�
JtA;O-�|7#�+�p��I�,Wi�
v��?�������w�?��A�<�����i5��;�����q�d�oZ55�M�����U�R�������=��w��_\�L
`(�]������\n����]�������:��oS��Q��r�����Ge�Oo^,��A�0���g
"������x��{�w6��0B����l��V����tg�*M�����d����f�X!cc�7��z!�;��=�m�H$��������^���^v�����L��L��u��2�����Lw��L�e���k2������}�J�/�j#:�Jn���!lT����F����9��V��7�������7��U���7�N��S���n�![���=*x}��C�F1���{�7��#����.�Sf�>w�����x0%�;NY��K8�����?�X�O���������<>�
R�d�Jaep^����Sn�_)��?l���z���y�C��k0�Nc��C;t�������v���������p@Pq?o:����]>�����%���|�;Wz���w}����5( �}���~��@�����`oy#�\"����%��SL2�����q����AS{p��b�S����r���=���b��Dy0�
k(������Xz)��.��N��G�2X�KK8a�����b�]��#D}I,���i,JA��]�(�MS���\�^b�=)%�e?����[�K�q��=�:�,��W��o��7a(oZ�7��:]�N���s&[	(q��S�2'�D���c�qr_��a��l���?x2�e���|G����/+9��)8�+Ri1q�e�R���H��V��o�P�����)����A�L�.�������T��m�PT�il�^��y8
Dk��1[��gDn�Fn�En���}�M�GnIU��!������r�y��.A��j�t���\������^��F�j�%c��w�e��)�����d�V���E@����P�<����~�����k��4)����=|GB��Ch� ���G~��(�m��zmC�E���=v������V�25�q�8H��9�$I�eS>=)V��+�*�~!�4_�w�r�-�z|�\���h.P|_|���qub���Yi-�nE��fp��a�!�
��2�h��m!�������c��j�L�q
(�o�X~��Ns�k)9�;�{��6�?6U2t1�gP�����6��PQ��!���R��w�~(����?e��?'e�����-FFj�q��)����)��Tn��P��@;WEnj�A��G��7��Q���!�v���e	�����a|��V;��q�A����whN���|C�i,���w2>�>�N�������os�N�0�/��9a�O�E5�m��#mS?
�����h#�{�t�fWm��m���h���8:�6:Y\uW�Uw�6���#�:C)S8��O��8���������E+�
��� ��XZO��'�?���zE{�+��<mG6���Mm'�&�>(�xZ��O�������G<��#��n�?L����&&��7����������Br}y5��f��h�{���<yd������.�MA��`���j;�MG�'���z���i�X�	��/(tp�^nW�G5�!R�>�g�h3�4�o�`�\o�8a$gz�|�o�X�c��x1�
���`H����$-�����$1^����t�I4�q���+-���������A�lp1���i�\�f������
�?C�w�.)k(������%�����h%I�^���{�/��e���1���(V�xc� ��r���H�(�*5���m@Y&�@
����������Y�"�
��p��*lOz��'X!��5�.F+��I����cV LhE�l/`j����x�bR�D��Sa�d�@������Q&h�y+:�;�rAJ�i�k�5?G����
6���!d�C����
��_8h�m�_�:����B7�~���w���������m�����������}���������x�6�u}�F�E�lf��j�&�G�qSX�0��a�Za�������C��/�����f;��c�/��G��p2����(����a���=�G��g���3��M����{����y�B���H4("��1���J9�D������ �X���=����5��|��	�u��-������i�0��X����x�_��'�X+��F��"������Ib�\�j#����6u���h�6�`����v�N����x����Y[q<f�u�n����+YCX��KJ�����h0t�t��/dE��2����rq�����lV�������U�����<<;S)��)��R��Brm�9�(���S)I�ZA&Sa�Mc��R��'�R���f>?0��5���F�����)���N��i���q��iH;)YeQ]�n��Zg�)}w3i�$��7�'e�E��t����$�<�C�f�t��	qj<���&����k�T���h^���O(����Kr��#zK/1%��\^�����@8�Zm>��X�A�g��C%��:v����h�B��+�O9��7C�3V��'�NN�2�n��rC��a�(,�Z$(�m6�s��a2&I�Q�SN[C���;����N�9@HF�x�D��s�W��!�M�g�;��\^�c�xT@/�q���*�t1�7Z���f�����0fW���g��}0^'��t���y���p�v1�,����P�"�ta5/�HaC����/C_�E4�S2��T�j���J*#���q{Ib"w ��2����D&&?s���$)�#�Dx���Z��=�k��ht^|�I?�����G�����������PpM���T-@w������I	PU(9e�������������K%�
��j������9�����nF�t�����:"����d�]�bIt��&���/8�����?�A���O�`���z���9U{o1��e��L�����z��>��iA_wT��~e��99v�`���[�[ ��X-���5�PNpO�u���j&gu	y&-~�I�r*Iu8�����F�����ay&
�H�I�%���6q��$N�1_�d-��'1�NN&���p��9�b>�\GW1�����`@���4�>+���jF��	=��%!��<�8�R^�b9j���~�nT�Y�~c1���T�Ii����q�D�OS���{%soI[�p�4�<����H��<K��!�Qr�%C���>��,lf��%������)��@y�R�C��b�f3����9���V��N���.0�M.*���h4D��������,����d2E5���F��y8��ds�]��wz;��#�J�����&��>E�i���n!����W8����
+0�$����T�4��y7(����s���"�e7���Q�������T*<�0�)F���3*|��V>���������ek'@��
���,y�����j=�-��Z�����j0��������t��\�m�&� ��s�s���oh�����?���N\g2�5.�������a������;���������w�q�n7���f�}>���Q����~7������m1��q������c��C�,�0�V*?�p�&�D%��� /�p�����-i���bWJ�q�����������Y�D���
���@�A�6�Fej����Jo/�kzB��A�����_>3��/L�N��O��8�%���e0�h:C��,VoM�[�<<+������G���9^a��C��<�[�vyF	��4���`0�����V�4J�_�E������j�kN�_��GM��e�#R�@)d�d	�	g�F���f���GZg�s�iG�}�L�W��'�z�pi������#B�9r��n�Kx�jA�t�/$���b�!�x����1�?����4q]����@�0nm�Qv
���0��$���:����K�Y�w�l���Rx��X3"h��&F�0��@E�t��@*����P�*���:�\��h<�����623�D���QY,��f���g����1`!x����@����U���8y����-���$h]cq�AI�� �_������^��:�{���G�>t��K0��������|���[7����F�3L)�����W���f�Nx�M�����d{����j�m���R��1L���D���o���0��X��6�u[K^���W8���rf�(v�ht6
')�]!>/��R�o�y��������?^�,���w"����7H�'�3������L�N`��]���
���_C�u�~��K�$�&_?�K�v:������u��v�(�=��?<�:��E �G�X��WP���O�`V9�t:`
��}���w��j%B ��G���n��t����l�v�l�������?!�F(
Y�cHiEl�W�O�"%U�Mi�z����G���20//u�4�����-�b2��������lO����'��I�)�����2�����`�Y���7��h����Y����H3v������u:W���s`�R���G��g�D��:�>-�������c�(�*�����vj��Sk��p�j������g+�`a�}vw�h��n��M����K���l3�o����N��
[	�n��);�\�v
�h�^����.	��o�������
6��8�]���n����q<A	!#�WY�E&��r��QpBz�b��a��t^Y>�
�e%�S���[����V���cGUzp�,l(��|�z��J��������qoP6�����b:�?������/S�
����n8(h:������?va��+�:��u�����l���of��O�~"k�����q��D��iw����BG�`G�`G�ziG�B��"G�BO� t�!� ���#��g?�����YBo�����y�j8�!xr���������r��.JF�b����%��A5������Xv�#����Li���"h��^��
��#���w��o��;7�S�~'C��������.I���x�D����e��7�m#��)ic�m��u1 ��i��vk�}�f�aiv��~n����M�z�Yn�E�SS
�~�o�Z��a��b�����s*��a���h�dQ��ac|^�%���di�
�#���&e�X4@z��$�JX�i0(u�n�@�����)�mcQ�Ez'H�:���pO�)s[Yf��	��
/%f$�4O����
�-�Sk����&��3P5�V*8;@�3��S����)��J+��9�p��C�tB�tt����k���u���#^hs��[F�Z\OA�h($Kz�n�A4D���"� p��z�����"s�����[���V�����4(:��/��u���i�Y�Z���:!-/�
�W+i�����JGV����G<�MA�����C���H	�U%7bp<f�x�����j����9
{�~��k4z�~���~?'�wm����%�!Z����4E��t����~��7��%��|{�~k+����7�Y��y���2�}iy_�y�]O�k|�U�����S�N��N�%����85�������N��[� ��TU~;�����j�&
�0)���na����P�
=����b�W>��g���	��:��*pt�)�
TR^%���%���O�E�\����l%�
��{�3�pg����V����MyE����j�Y���A�M��Oh�a�D��*���F9#�����b�k1D����r��U�Y9&S�,����;o�,9�
��"	K���1Z��W�1x�����r��V9-�_v��h�Q�VN��-�����?^��	w�?Va��/��/�7k_���*p��<������T�J8�������!�	g915�
hI�Ax"?�*4�'�p�ox�j5����^����0t~#��S����[��b8[D*����l�z�y����d�d�r~��hY�{�J��R���W�U8z�sTI�+Qpm
���L�"�7��g���?�7���������]x��|�����	�$6kH�c-�[5�;Z���O�t�$�*7����4��a1�j�q�1$�,t*��o4���{�� ���K
���4�sT���V��G��t���|C�D+�Ct���K�*9�~�
yL!��E�5�c�V9��Q�Q�
�+���#�V���Q��������8�L�$��L�AS�|t��"g�r5��UdIX��_CO���!_�1t�]:��.��.�j��j��"�j��H��O�w��8Zef�}��������w��Nr��3�I������,(o1����'=���z���I��"��]_�3�.8�������#]�}�:�Q���J� AWg#���4r9�I��&<��M�1�L����*�����)�j=u�4��
�E�[3�j��	=e�����W��G�'��:g�� q��|��rn�;���Gf<;��3�,1���.��v|�qx��pp�V����Qu:f��'�=*u]v��,�w�P	�z�d����%�R�r,��?�D
U����66�����-_��-���^vy��iu=Y��W������ ��$�	�s=A]p��;p�����Qz���}4%`�bW�o�?�,���t L$��F�;Et�T5�A�r�{j�B��l��}�r{����>��,���Qi���X|C��e�2����#��rOD�y�N:��������{����}��g�xI�tO�iKJ�7g��h(�<M��B��n����`Op�K8����6�������k<�����F'�	�����@�N���"��\��u>X�����NB�{��k�N��"����������}6���S����=�������}kJQ>��������6��Z)�������s���������\G��uw�9��������k%������d8�W�*��,�~E���������������������Y��2�7�QdV�y���n�tmd{��9x��s3
��/��R�)�j\I������@�����^�8�K��+LN5���!gv��n~�s4w&)��o���S��f�����,��9�f��d��K����vs�=kA�B��(�S(�g�i1��6+�����#[#�����^���Re=�����2@?�k[b�����8�88�*��+{�L-���{����!�������R�3���j���s�#��R�v/��H*���o��B��xsS�DE�O��@w!��Z���A�?��T��=D,�`���f:BSlL�|�-/e�!5�����n���)��$��<�.��D�B�����]������C�QkLn��l/��$�K��;�C|�.8T2o]�ed3���@�&_A�|�g��������Z��v��
�&�:?�?��������q�zD<���+L�����?�N�cy��`��8�W8��x����)9���G����?H�u�]F"J@�
�����3���
k�FguWS���;��jJ��~�Wq4����?���r���0�����@��	��3���������T�a�]&�a��j�����(��!���%0�.>b�r��P������T�K�I<�~���w&� �!gi��#Rz^o��)^���h���4����a���r�uLQ��D!�!�a���k.�g}�AU���L���5v����nG���M�Z���0��Y�d��`�X�!Q=z�N\�hy�e��C�����hTu��$��Tr�|��pi�d�� 9b;UMcSc�A�i�p8A�]��%������=���WiF�������)�g*-XT�C#�I{�	Ij'�74�/��H����,eT�a��{�{r�����o���v�Q�V���@�J����"o[R*��%d�
�.�t��
]�r�<����v�V�9��*�umE��AZ���e?�j���2�x,� ��bm)�879�Z�����0Z����r����!�O��������_-��YJ dZ��.�j�AeZP�u�x��ib��[��v.��D��(��N�����G����90O��b��������b�pA��1�2�����o��#3!]����L�>oM�f�1���~��� @6bd�]w�v��|��	��@��K��O4��qFQp0q�y�XE����$���_���eN(J�=�?g�
��'k
;y�z�TU=)����Q�<|QQRE�b�p�}��r|#�^T.�!2���(3X���k\�L����������=�4�{����{������f8�+���������B���|�x�����Z�,+�"�Hu|f��=�v���Y����_���L�d���aw8�6q8n��~�����2O!$�s�����`�������>mk��c-8d^���'n*-��g�+�������274Q����e$�V��3��z��s�|�[�N���a�E��������Mv�O�,��A���Y�}�ti~9^n��?�N-�|�vW��u��f���r�����^�������?�o��r|_�I,�����>�i�o��U���k�yuH������S����~�!����P��f
��V;nN�z����4������u>l��y������3�����kA?x%�o�q����������r�$��P(��V��T�F�r�+G�skR5uN:�$UWnK�>:�V���7;F�_n�CL��k����[
2z����V����=���`��$�Oim&����!��b�f�*��
��Q��c���v��36���r(7�Z/��=�� ���� -��v�'>��^�����W���4*�i1�	*��c�(d���O�-eQ�9�����A	�P��&Zdz�b���B ��|9�C]��F@�SyD:�f��Gu�(AvS�."RwO#l@7�IE����!��m�r:���:;]�O:�	�+�t}Jg<����\������'X*�i��rEL=�z"EAlh��R=O(J���I����U]!2T*�F��v�7a��%�;m��Q!q�L�����'��bu��nQ��03Q��*UJ��n�g�H>/5���x��`�+*�o0�2�Z'������!$�[��,�^&� q,g��:?��j�fb�U��HL+�p��h�-0�(�|����]���l���n�+��l���^/�c���Ua"��td�@�
�����@�����}��"W�(yu`l�'pY�	5:Z��j� �b������$��
smG�P���U��R������Y	EV����I�z$����lI8y	�P���@���j�IdF|�hc�UTk��o��F������3E�d�9��	��!,���:��5�z��I���]�_C�����@C��_�[M�3���U��V;*�Xf�Xns��1&�M��K���K��Z?d��!��bU��9x#e������u6c����T���S04�v����������i���'U��N���-o��+y��d�|:�zL��SD��u�����"!T�z�I>����e�q��I���$J��+�D_m��
�������K��^�Z0!T%*����-+s���U�+lq�����a�U!J$�VEj�-j�&����5��
4�J��)��d]%��	U���0��wo����WbD��t6I��:�7�o����c��[�I��z�d�-�"G���a!�y���V��i�m�I��s�]rD��x���
E�hm�A�����������r�X�
�,P�S�����'`�J���j�7PLm��(<��!�^�zF.�x�s�F;	�c{53L�5���	RE�R��:1p��R�fE�&���J��Yf*�%&������y5F�����&S���x����)�x�dO?��*A�N��i����f2��U���S���q76q3�������-H;C�[e�>��������1���8h��Pe3�Qnir_������p���:��:g �r�8�@��
c��(��B�&�������
j��.@�]��(��Q�`�q����R�P�Jx"��l��������'�rY&�p�U}�Fu����:���c�H���/Q�T#���B�hU
��Jg��Z��b�$O� ��'RG��:���F��{;�a@�m)����V�Y��IEbr�XM"���)�w����a]���	����A6���V�����VC(|��lN���3XS�	�n�=T�)�0�\9;�@�D
�AX�Q��yZ�~�)��G����Q��r{
�i����g��b�Qb��SB�,��'�(KN�3�~�Q*f��9��s��tp�lgv���92d@)��a�y��e[����P�A:�q!���������0R�e�2�S�e0��V]�aK��.I2��U���fW�@��x���9��0��V���d���3H��w�<��BL'�j�q{���r���.���6��������r��'$�L�k"X����HIo�n7����4�@�V�7V�V�V�7V�'V��Qi2��]R����4�\"%�+�zA.�S��2.������]����}���Q�[����W����L5���0������������s�tI����vg6�a+<o�J	�X�"s�4{Y4)3|����>Av��U��,�l���J��^�t�o��uBn��HRqcn��fW3�?�G��0�����Ay��v��T�ui���'y*����}`�x��cP�,���,�5��g�I)x���H��v�b9{5���v;%y�[���o����
h�Ha{O<�;#�I,�t]�F��J�s_�4x�������_������x�H�N�o���;��Ea+�A|��#���7� �M+���x��^]����W]Y������z�\eMW��]o6)����4�
N='
A���@���&���0$���N�z_�P�y�I�C�+���uc����Q��w[����;�&�����F�$A����\:���N�j�:\,|��u>�	$��	7����1��h6��=�7�q�W�
W���dY^���\���|IG��t��8�1�z"S�	yj:���&;-DP������%������]��v�pZ���i��q���>������I-���u�s�}��n�b�	a8{��O����m=-��l9-[p�T�[@ ���~?�?���9����od�)�y���*���XJtA��J���9(����
�$5��LR����i���������jQ�*����;M�=2SQr"���J0:D&|���3�F4������5����o���VsWv���AS�K.,�o���V�����|����$l�[��+����V���n�j����r�0�����q��O�X=�N���������{�g+������bv�P	��;
gV��F�iK���EgY�g!J�0��>K�������������>^tG����!8PP�'�8Z*����	0}0L����[3Q9��d�9�&�,�S����Y{p#/���7���b�@��1IP�8��;�S���i�������a����K|����;v&��a'��>��ku��(�vz�0>k�Zg_�������9���nv��[S��~E(�R�\�b��)OuX�
Z&��-�t����S�o�l����c��d�|�i"���+|��8�BU��:Z��5�l#�0�'��p��"X;�,����;���z��g*T-U�����h� �d��J�	��@�?���^���!���/%s����#�EA_e�|�o9`�sfU��-���������:Z��o+�#n�M�mY��D�����p��T�8-Q��(\;Yv�(��U�l+��N�
�c���SFf�		�9���"�M1^��v���.|mb�T���B��"�
�?H��,�_����w����g��HG������2�+��A��v�!��3)�;���P�+Q��x���Fx��a�|�F
hw~F]����������	�"�ly�H�SI�[N	'����q�Je'T�@��5�&����"���W�h�����$��b�\�=JI�;���� *���3
�Q9�j����g@�8 ��OKM�����K�V2��F���
��A=�����pk��%]��:����������h���XZ��~���3�H���U�[�;�;����p
*L��.c�V
�0e��:�������C�T����HG���N���w$���8�,	�e�%����K��6Q�>V��,0zt��C��);����H��n�m��M���>x��
l����bv�@���[IT�jkWJ'zW�i*�i�x��%8�u4��h��#}�|�}<���Q������V��:��u�`l��f=����> ���/��f
��l�w7�����#�!�`�:��e�&a�����YM���M*P��3b�����wr���D`��T����������c��{2"����>��^9�|��~i��}����'F:D���L/���	u�ozi�V����&Y"���z\^������c���>�z����������PI-%��LN
Os��^<@;�%�G@(��oT��t��.GVu�c���A'L��g�I���a���u��G���{o2�j��N�^Q3K6�XM.��rjd�Y���&HuY=�����p����!���F�HQ�:�� O�����U�'c���W�T���,�*<c&�N.@ek�580�B5,,A.T���Q�n�����#���X�zg'�a���W"36H<���h�N&��?9x�G�|N{��O���9��y�gHtos�WJ���������?��L��*rV2qr�+��y�D��<����hc?-6]*-&�|M��fY����,�+3��#�Ja���k�*fezD��a���iIN/w�Z������������A�/?\�[y��j�N,7�f�o_��,�&F=q�<���3�Q��~+�������AY{���4�3]���t�Y��p��)��Y�B\vuZ��	I��'���/S�?�]I� w)��T�l�!U5���7���!���T�CDY2���n�"-�L�GW���
������Yq�>�T�h'Pz�9�*b���'�Zw����3������
*��n�F���5��0KGW������,�X+/]R�a�C��O�&�[R�!�[��}���{��D�)��fZ���) P����w,��=�N7�Pr���y�Q���!�3�#����\#9�3��xq%�������#+s�la+t[���Y��3![��f��?,|!�����������DK4&[�*��s*}I�4wQ+:�)�u���~:6�����{��+�������3#nVe<���G�H��/c{�T�W�8�'hmf����|�>�-�Vk��z�SK��SM�x��b�����:�v����u�y^;@��f�q^
^��:�����>��#j���9�e�;,q�����z��m�v�
�A@i�\����z!�����r�)���=bE?	���������C�	�NO���S4_%�
N�]�J?I�J�K���5�����F	�YY��[�����,�����#�7�t�5�xr�9�T���Vvt��p��>5Q)�N!cu�6}r��u1_%s��N�I��x�?��1{�0)fZ�[('��*��b9�H�Y���:���]Ks����%���?�@�cK������ex���JG0��23>�2�?����!>�8���x�c����Wp��_o�&��D�����![97����G<�4��@}��~�h�P7�U�cN�z\;J�����a�*��r�`*��!�:�����Ir�^R���V6�/����\�ubOd������C��,.���7��� .�/��
��+SIU��b(5�%S C�^`�>���c��F�Z8��*�����'��NSt��U��Y4n;Q�
{���Ni�����k��A�C1���m�����<���MxY�����|%f�@T��E��|�$x�f�A���Wo?�<��U���G)��1��c:
8��Sv�s��i]PJ���j������ �
� �=���5���S�?s)Z��1�6-��	iJ�]j6���ja��wj����Y61z�_��� ��������S3K��x���+�D
A���$sT����E�������P�M�����a��I�@���VW��LG������D���.�<����LG����K�&��S�G�����������n��W8n�l�`��+]e�h;����i%D�]l�:��B�����e��V�vR�n��IZ�
E���g��I�
����]���j�HM*Q���&�`�?�>������sc)�m_(�+�o���_

��Z����]�W�)����G���
���mI��N|��nA����U�55�1.�S#
+�m>$Gcc�p�2*u�����clA�A��9�H]�q��9&��o ��Z�=>Q``�����	�����r�G*��R@����l�-�/�q���5��Im%�~��`[���x���.4i��oN����|����a�]d�r���	�$�'�c��,��������t��-+��{I��#.�t������s�������A-���}�]�.;�u�g��F~�~9 I�2���_��ZV�Lr"K$���B�~�
f��O83 3��/��=�j�>��w^+�(����J���N�QS���/�����QE��k�5s5fxG�� �	K�k97���=���a��pW�@^��g+�dE���>����P?"B�c%�PO�G�x����T�#��9�Bo6�$�����<I��$f
��p�=N���{��X�En���bL~	������E=}AV��$�dc�0Y�)�]�ng�5z&���*�Kx	"�����Z�0�F����Y��M9?-�'|�7;��=����[����,�MZ9�QLb�&[��D��b�bN2����R	�D)W��,��n�:g;��J��qV��� ���@�
��bx��Z%<U��6P�i�~�Vkm�������{���e���l�w���QA 5?��8:�Mt�R���t��D5��	�d%V��2`@��&M�*�X���ZN�8d?��Aq�% t���o�	bh�q-d��Kx�"�hP}M&��L��{��u��V�Z���d1~�u]j���E""����pt&��.�,��`�q���Y1�Wn)gg��p�q4�(���b��WWp(B�B��Z`���)C7@�p��Vo5T&���`�y�U�5�st�#S�����h�	���Q]���N��}����Fs������_P�qK�%Rbs��%�5O�����|�i��S�"���%����tqM����c���;��T�D���
*,	�Sac�$R�:�w�4'��uD�2Ss@���@u��(u�6�XlBi����1yF�����M��	�H��Oh�����O��&��p�.�T��t|��_m����rT����I��Ms**��[sg!);�
�K/d�S���N�\�������3@���+#Ln�]��?Nob�Ll�[3�6����7����n7i'��r�J�pb��q�G(���S(�9�����y�[�U��c�L�U�Vk�����|���#���|�
��*���J&�$)n��m���E�:��f:�%0���3��Y,�U*����A��w�A���$X�u�����t:��'��z����/e�1���,����@K'P|��)���s�&�����siL�]�E����2N�H	����%�DzYb�����Q��t����$|r$U.h]�M��s
z5	����a�k(��h�)����p�Ur���L��
�-�������R������1����#P��D���P��,gk�G�z$��,���P����>�q,�<�����7J�;����|��D�����T �<��4C�����o��[M��mh�2|_)2��z�	���'�����1��	�v���+�yu<�	9iB�a-l���,I�@�_��kK��>]J����I]U
��9�6C��N_�v����~�x!�k�}O[���II�����5�;.N;J��j��G�9�?W>WY�8����[���i�x�����������g�V�ef��=9�@a�T"���p���Tbu�n�T I.rJ�IA�8�a|��&t,Vi�8zk���],��/���Itugu�o���[��j>��������y�<|>��������y�<|>��������y�<|>����|�/b�8
#168Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#166)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Mon, Jul 31, 2017 at 7:59 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Adding AppendRelInfos to root->append_rel_list as and when they are
created would keep parent AppendRelInfos before those of children. But
that function throws away the AppendRelInfo it created when their are
no real children i.e. in partitioned table's case when has no leaf
partitions. So, we can't do that. Hence, I chose to change the API to
return the list of AppendRelInfos when the given RTE has real
children.

So, IIUC, the case you're concerned about is when you have a hierarchy
of only partitioned tables, with no plain tables. For example, if B
is a partitioned table and a partition of A, and that's all there is,
A will recurse to B and B will return NIL.

Is it necessary to get rid of the extra AppendRelInfos, or are they
harmless like the duplicate RTE and PlanRowMark nodes?

/*
* If all the children were temp tables or a partitioned parent did not
* have any leaf partitions, pretend it's a non-inheritance situation; we
* don't need Append node in that case. The duplicate RTE we added for
* the parent table is harmless, so we don't bother to get rid of it;
* ditto for the useless PlanRowMark node.
*/
if (!need_append)
{
/* Clear flag before returning */
rte->inh = false;
return;
}

If we do need to get rid of the extra AppendRelInfos, maybe a less
invasive solution would be to change the if (!need_append) case to do
root->append_rel_list = list_truncate(root->append_rel_list,
original_append_rel_length).

The code avoids creating AppendRelInfos for a child which represents
the parent in its role as a simple member of inheritance set.

OK, I suggest we rewrite the whole comment like this: "We need an
AppendRelInfo if paths will be built for the child RTE. If
childrte->inh is true, then we'll always need to generate append paths
for it. If childrte->inh is false, we must scan it if it's not a
partitioned table; but if it is a partitioned table, then it never has
any data of its own and need not be scanned. It does, however, need
to be locked, so note the OID for inclusion in the
PartitionedChildRelInfo we're going to build."

It looks like you also need to update the header comment for
AppendRelInfo itself, in nodes/relation.h.

+ * PlannerInfo for every child is obtained by translating relevant members

Insert "The" at the start of the sentence.

-        subroot->parse = (Query *)
-            adjust_appendrel_attrs(root,
-                                   (Node *) parse,
-                                   appinfo);
+        subroot->parse = (Query *) adjust_appendrel_attrs(parent_root,
+                                                          (Node *)
parent_parse,
+                                                          1, &appinfo);

I suggest that you don't remove the line break after the cast.

+         * If the child is further partitioned, remember it as a parent. Since
+         * partitioned tables do not have any data, we don't need to create
+         * plan for it. We just need its PlannerInfo set up to be used while
+         * planning its children.

Most of this comment is in the singular, but the first half of the
second sentence is plural. Should be "Since a partitioned table does
not have any data...". I might replace the last sentence by "We do,
however, need to remember the PlannerInfo for use when planning its
children."

+-- Check UPDATE with *multi-level partitioned* inherited target

Asterisks seem like overkill.

Since expand_inherited_rtentry() and set_append_rel_size() can now
recurse down to as many levels as there are levels in the inheritance
hierarchy, they should probably have a check_stack_depth() check.

Overall I think this is a reasonable direction to go but I'm worried
that there may be bugs lurking -- other code that needs adjusting that
hasn't been found, really.

Planner code is already aware of such hierarchies except DMLs, which
this patch adjusts. We have fixed issues revealed by mine and
Rajkumar's testing.
What kinds of things you suspect?

I'm not sure exactly. It's just hard with this kind of patch to make
sure you've caught everything.

--
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

#169Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#167)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Mon, Jul 31, 2017 at 9:07 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Forgot the patch set. Here it is.

The commit message for 0005 isn't really accurate given that it
follows 0004. I think you could just flatten 0005 and 0006 into one
patch.

Reviewing those together:

- Existing code does partdesc = RelationGetPartitionDesc(relation) but
this has got it as part_desc. Seems better to be consistent.
Likewise existing variables for PartitionKey are key or partkey, not
part_key.

- get_relation_partition_info has a useless trailing return.

- Instead of adding nparts, boundinfo, and part_oids to RelOptInfo,
how about just adding partdesc? Seems cleaner.

- pkexprs seems like a potentially confusing name, since PK is widely
used to mean "primary key" but here you mean "partition key". Maybe
partkeyexprs.

- build_simple_rel's matching algorithm is O(n^2). We may have talked
about this problem before...

- This patch introduces some bits that are not yet used, like
nullable_pkexprs, or even the code to set the partition scheme for
joinrels. I think perhaps some of that logic should be moved from
0008 to here - e.g. the initial portion of
build_joinrel_partition_info.

There may be more, but I've run out of energy for tonight.

--
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

#170Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#168)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Aug 3, 2017 at 2:10 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Mon, Jul 31, 2017 at 7:59 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Adding AppendRelInfos to root->append_rel_list as and when they are
created would keep parent AppendRelInfos before those of children. But
that function throws away the AppendRelInfo it created when their are
no real children i.e. in partitioned table's case when has no leaf
partitions. So, we can't do that. Hence, I chose to change the API to
return the list of AppendRelInfos when the given RTE has real
children.

So, IIUC, the case you're concerned about is when you have a hierarchy
of only partitioned tables, with no plain tables. For example, if B
is a partitioned table and a partition of A, and that's all there is,
A will recurse to B and B will return NIL.

Is it necessary to get rid of the extra AppendRelInfos, or are they
harmless like the duplicate RTE and PlanRowMark nodes?

Actually there are two sides to this:

If there are no leaf partitions, without the patch two things happen
1. rte->inh is cleared and 2 no appinfo is added to the
root->append_rel_list, even though harmless RTE and PlanRowMark nodes
are created. The first avoids treating the relation as the inheritance
parent and thus avoids creating any child relations and paths, saving
a lot of work. Ultimately set_rel_size() marks such a relation as
dummy
352 else if (rte->relkind == RELKIND_PARTITIONED_TABLE)
353 {
354 /*
355 * A partitioned table without leaf
partitions is marked
356 * as a dummy rel.
357 */
358 set_dummy_rel_pathlist(rel);
359 }

Since root->append_rel_list is traversed for every inheritance parent,
not adding needless AppendRelInfos improves performance and saves
memory, (FWIW or consider a case where there are thousands of
partitioned partitions without any leaf partition.).

My initial thought was to keep both these properties intact. But then
removing such AppendRelInfos would have a problem when such a table is
on the inner side of the join as described in [1]/messages/by-id/CAFjFpRd5+zroxY7UMGTR2M=rjBV4aBOCxQg3+1rBmTPLK5mpDg@mail.gmail.com -- Best Wishes, Ashutosh Bapat EnterpriseDB Corporation The Postgres Database Company. So I wrote the
patch not to do either of those things when there are partitioned
partitions without leaf partitions. So, it looks like you are correct,
we could just go ahead and add those AppendRelInfos directly to
root->append_rel_list.

/*
* If all the children were temp tables or a partitioned parent did not
* have any leaf partitions, pretend it's a non-inheritance situation; we
* don't need Append node in that case. The duplicate RTE we added for
* the parent table is harmless, so we don't bother to get rid of it;
* ditto for the useless PlanRowMark node.
*/
if (!need_append)
{
/* Clear flag before returning */
rte->inh = false;
return;
}

If we do need to get rid of the extra AppendRelInfos, maybe a less
invasive solution would be to change the if (!need_append) case to do
root->append_rel_list = list_truncate(root->append_rel_list,
original_append_rel_length).

We might require this for non-partitioned tables. I will try to
implement it this way in the next set of patches.

The code avoids creating AppendRelInfos for a child which represents
the parent in its role as a simple member of inheritance set.

OK, I suggest we rewrite the whole comment like this: "We need an
AppendRelInfo if paths will be built for the child RTE. If
childrte->inh is true, then we'll always need to generate append paths
for it. If childrte->inh is false, we must scan it if it's not a
partitioned table; but if it is a partitioned table, then it never has
any data of its own and need not be scanned. It does, however, need
to be locked, so note the OID for inclusion in the
PartitionedChildRelInfo we're going to build."

Done.

It looks like you also need to update the header comment for
AppendRelInfo itself, in nodes/relation.h.

Done. Thanks for pointing it out.

+ * PlannerInfo for every child is obtained by translating relevant members

Insert "The" at the start of the sentence.

Done.

-        subroot->parse = (Query *)
-            adjust_appendrel_attrs(root,
-                                   (Node *) parse,
-                                   appinfo);
+        subroot->parse = (Query *) adjust_appendrel_attrs(parent_root,
+                                                          (Node *)
parent_parse,
+                                                          1, &appinfo);

I suggest that you don't remove the line break after the cast.

This is part of 0001 patch, fixed there.

+         * If the child is further partitioned, remember it as a parent. Since
+         * partitioned tables do not have any data, we don't need to create
+         * plan for it. We just need its PlannerInfo set up to be used while
+         * planning its children.

Most of this comment is in the singular, but the first half of the
second sentence is plural. Should be "Since a partitioned table does
not have any data...". I might replace the last sentence by "We do,
however, need to remember the PlannerInfo for use when planning its
children."

Done.

+-- Check UPDATE with *multi-level partitioned* inherited target

Asterisks seem like overkill.

Done.

This style was copied from an existing comment in that file.
-- Check UPDATE with *partitioned* inherited target

Since expand_inherited_rtentry() and set_append_rel_size() can now
recurse down to as many levels as there are levels in the inheritance
hierarchy, they should probably have a check_stack_depth() check.

Done. Even without this patch set_append_rel_size() could recurse down
many levels of inheritance hierarchy (created by set operation
queries) through
set_append_rel_size()->set_rel_size()->set_append_rel_size(). And so
would set_rel_size(). But now it's more prone to that problem.

I will provide updated patches after taking care of your comments
about 0005 and 0006.

[1]: /messages/by-id/CAFjFpRd5+zroxY7UMGTR2M=rjBV4aBOCxQg3+1rBmTPLK5mpDg@mail.gmail.com -- Best Wishes, Ashutosh Bapat EnterpriseDB Corporation The Postgres Database Company
--
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

#171Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Ashutosh Bapat (#170)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Aug 3, 2017 at 9:38 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

On Thu, Aug 3, 2017 at 2:10 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Mon, Jul 31, 2017 at 7:59 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Adding AppendRelInfos to root->append_rel_list as and when they are
created would keep parent AppendRelInfos before those of children. But
that function throws away the AppendRelInfo it created when their are
no real children i.e. in partitioned table's case when has no leaf
partitions. So, we can't do that. Hence, I chose to change the API to
return the list of AppendRelInfos when the given RTE has real
children.

So, IIUC, the case you're concerned about is when you have a hierarchy
of only partitioned tables, with no plain tables. For example, if B
is a partitioned table and a partition of A, and that's all there is,
A will recurse to B and B will return NIL.

Is it necessary to get rid of the extra AppendRelInfos, or are they
harmless like the duplicate RTE and PlanRowMark nodes?

Actually there are two sides to this:

If there are no leaf partitions, without the patch two things happen
1. rte->inh is cleared and 2 no appinfo is added to the
root->append_rel_list, even though harmless RTE and PlanRowMark nodes
are created. The first avoids treating the relation as the inheritance
parent and thus avoids creating any child relations and paths, saving
a lot of work. Ultimately set_rel_size() marks such a relation as
dummy
352 else if (rte->relkind == RELKIND_PARTITIONED_TABLE)
353 {
354 /*
355 * A partitioned table without leaf
partitions is marked
356 * as a dummy rel.
357 */
358 set_dummy_rel_pathlist(rel);
359 }

Since root->append_rel_list is traversed for every inheritance parent,
not adding needless AppendRelInfos improves performance and saves
memory, (FWIW or consider a case where there are thousands of
partitioned partitions without any leaf partition.).

With some testing, I found that this was true once, but not after
declarative partition support. Please check [1]/messages/by-id/CAFjFpReWJr1yTkHU=OqiMBmcYCMoSW3VPR39RBuQ_ovwDFBT5Q@mail.gmail.com.

My initial thought was to keep both these properties intact. But then
removing such AppendRelInfos would have a problem when such a table is
on the inner side of the join as described in [1]. So I wrote the
patch not to do either of those things when there are partitioned
partitions without leaf partitions. So, it looks like you are correct,
we could just go ahead and add those AppendRelInfos directly to
root->append_rel_list.

Irrespective of [1]/messages/by-id/CAFjFpReWJr1yTkHU=OqiMBmcYCMoSW3VPR39RBuQ_ovwDFBT5Q@mail.gmail.com, I have implemented your idea of not changing
signature of expand_inherited_rtentry() with following idea.

If we do need to get rid of the extra AppendRelInfos, maybe a less
invasive solution would be to change the if (!need_append) case to do
root->append_rel_list = list_truncate(root->append_rel_list,
original_append_rel_length).

[1]: /messages/by-id/CAFjFpReWJr1yTkHU=OqiMBmcYCMoSW3VPR39RBuQ_ovwDFBT5Q@mail.gmail.com

--
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

#172Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#169)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Aug 3, 2017 at 7:01 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Mon, Jul 31, 2017 at 9:07 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Forgot the patch set. Here it is.

The commit message for 0005 isn't really accurate given that it
follows 0004. I think you could just flatten 0005 and 0006 into one
patch.

Earlier, there was some doubt about the approach for expanding
multi-level partitioned table's inheritance hierarchy. So, I had
separated all multi-level partition related changes into patches by
themselves, collocating them with their respective single level
partition peers. I thought that would make the reviews easier while
leaving the possibility of committing single-level partition-wise
support before multi-level partition-wise join support. From your
previous replies, it seems that you are fine with the multi-level
partitioned hierarchy expansion, so it may be committed along-with
other patches. So, I have squashed those two patches together.
Similarly I have squashed pairs 0008-0009 and 0012-0013. Those dealt
with similar issues for single-level partitioned and multi-level
partitioned tables.

Reviewing those together:

- Existing code does partdesc = RelationGetPartitionDesc(relation) but
this has got it as part_desc. Seems better to be consistent.
Likewise existing variables for PartitionKey are key or partkey, not
part_key.

Done.

- get_relation_partition_info has a useless trailing return.

Done.

- Instead of adding nparts, boundinfo, and part_oids to RelOptInfo,
how about just adding partdesc? Seems cleaner.

nparts and boundinfo apply to any kind of relation simple, join or
upper but part_oids applies only to simple relations. So, I have split
those members and added them in respective sections. Do you still
think that we should add PartitionDesc as a single member?

Similar to your suggestion of changing name of part_key to partkey,
should we rename part_scheme as partscheme, part_rels as partrels and
part_oids as partoids?

- pkexprs seems like a potentially confusing name, since PK is widely
used to mean "primary key" but here you mean "partition key". Maybe
partkeyexprs.

agreed. Done. PartitionKey structure has member partexprs for
partition keys which are expressions. I have used the same name
instread of pkexprs.

- build_simple_rel's matching algorithm is O(n^2). We may have talked
about this problem before...

If root->append_rel_list has AppendRelInfos in the same order as the
partition bounds, we could reduce this to O(n). That expansion option
is being discussed in [1]/messages/by-id/0118a1f2-84bb-19a7-b906-dec040a206f2@lab.ntt.co.jp. Once we commit it, I will change the code
to make it O(n). Right now, we can not rely on the order of
AppendRelInfos in root->append_rel_list.

- This patch introduces some bits that are not yet used, like
nullable_pkexprs,

We could fix that by adding that member in 0008. IIRC, earlier you had
complained about declaring a structure in one patch and adding members
to it in the subsequent patches, so I just added all members in the
same patch. BTW, I have renamed that member to nullable_partexprs to
be consistent with change to pkexpers.

or even the code to set the partition scheme for
joinrels. I think perhaps some of that logic should be moved from
0008 to here - e.g. the initial portion of
build_joinrel_partition_info.

Setting part_scheme for joinrel should really be part of the patch
which actually implements partition-wise join. That will keep all the
partition-wise join implementation together. 0005 and 0006 really just
introduce PartitionScheme for base relation. I think PartitionScheme
and other partitioning properties for base relation are useful for
something else like partition-wise aggregation on base relation. So,
we may want to commit those two patches separately. If you want, we
could squash the partition scheme and partition-wise join
implementation together.

[1]: /messages/by-id/0118a1f2-84bb-19a7-b906-dec040a206f2@lab.ntt.co.jp

Updated patches attached.
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company

Attachments:

pg_dp_join_patches_v24.tar.gzapplication/x-gzip; name=pg_dp_join_patches_v24.tar.gzDownload
�9{�Y�<is�F��j���G<Ey���8��*[��U*�6��0������{fp�(���^m��0����5=JWK/]�O�x���]����/��g>���=������=�'��|:�8{j������c��)��g�}��uY$�z�����?����q����{���/�L���Y$��x�����0�uz�}��A�^<�Mv�4O�m������+6�#�.���$b����dj;��N'��p&����3[�<w:�q�����e�w"e��
��/�4
s�.��S����yf9���".D�fA.<�r�������.������`��_�����������t���`����~�����_��P�����"a���-5���2���e�?y���L��%"��)��������4Mx�^�A���0�#�����|4�]�x��yK��1~f��h������>��I�r�7s���%���6���@����D@��Q���%��%*s���-a�r�%e�K��_�.B�L3�8��'��w8�x�i^�9<�s��+�
�����)(�,��_�P"�,�7(J�B�*$HG!�-�K9��((�8r�Tk� v�T�=g�mRcA��h�&�����T�$J<`^�
�,�����o%E���2�@�@7}Z��A�x�E�{����D?	&�h�e�����er'����t�K��!=`�#0;�Q��J��K`X5?���,������D�7{[%�Gm)$MX��u5(�e3�Gs�V*���L�A���
R�4�1�e��%���X;�]3h��(N9�f���c�1��c�����h N����Y��R)@g�K3��e�d�IC��� n��9L/��V�<�/Jx��P�V������n�0DBx#�ei��&��b���^f@�B���o�0dM�WKqe����0O�K�$JF5�	i��	�
V���bW ����8�� #S�2jKC&.y�4]�j���yV�t?����
<���
��,iC$���Ip�$�p�iSX�t7'�!X/`�/��@KB+l��� ����tp���\^2��2�,�n<:f�� ��.rI]���5�T_�-�|Ic�����R�*C�F8~�������(�.�@r
<P�.�������g�yt��@}{��D�A
$7:�[����
��tP@
CB*�5��b���\�E�:�<^�n�W��[hB��E3��m��k�I���M��
�$M���C�����Vr��d�H�5w��3���3� �qa>Q|C/��0�`j!N�%�c�O��]��_�4��D���wz���l�j�^����r��!���i�p��+���� �(�7`������z:�'�E.B����d�S�i���	ZE� �������H��]QY��{HQ��C%��f&���J�mx#�+�TJk`H�I�	�H�,���hy�t�"�����dIRhO����A-!8����U~|A�2p��^���n@w�Y�	-���*%3����t�/�REw.�x&yXIO���Qq49=&���7�Sa���"|��0u2EB)���1��o�#���j] �1���Bq�|j;�c��2���O���i��1���N��a��(�0��EK���p4\��r��!���|�m�L`���B��:�l�@<*�"H�����5����"D)�)iE�Vv������"��j2�p��r#/����-�i��3�����`gI
[�O��(�3�/��[.c������N����	��Lj8�����Y/,���"��J����Q��+?:<Z@b�#=0�
��|���
�?;�,��qpW|b������e������g�	���
���U������b`��{4g�D�=r���>+QoXx"���<��`�uvb��y�L����(�;l�F��*�[	��hw���l�Y�!�l�R&u>�50��e?$L�~��Oq�t��*�BEDZ��3����c#w-�h)�|4E��(��
K�I�+�a?�%�����-m4"��L�x�iC
�Z�)��:{�xT/��j���h��F�J9�@oX	�X��`M�~b8�c��IB'�u�G��./e~�c��j�����h��B8�
�$6��TrU&�z��	����T�2U}�2XZw��@=���u�L�
�[JU���.�_�&��
�,1���W#�0@]�4�J^BL����$�0�����"���pi��G��!	���������j�9�X�@�C��$p�Y�����X�����
}I�D����/C��3��#�����vr�p�j.+�-yQd�Y@F��O�Y/��X{�D�����S|w�j��1[����L���P@5;���s���c,�:�?%�U<�VI��z�a�����KA��u�\�c@���d�K����$M!~-c���S����`4a��=��{�<����R<�N�l8�g����3�(�a�&e�-~�!��E��P����$�X0��Y0��X0��X0��Y0�Y0��`g�AC�8�5��<��kX=v��c;�����v�����
��������X��b���/������p��~��
��b���kw�
���b<�G�5t��bl0o��2o�F�6�sv�_�d�
�"]�1;�SV�B����0)\��Dj�����N0[
w)��%x�e��-��W�����v�N`|��[��\
5Q�M���]%�����`����#��N�$������y���%�����\e��$��>���H�r��f�s�5|/�}��,	�����5<�
<#x�\�-�����@���`A��������/��/��]�&6(����o�������l ���/�YA[�\	�T���%������#���"MFA����`�N��\��CZ�6j�3�Y2����$I�)2���rB�+B�� �=�Ko��5��*�����<��^0�T��[BIl$�d@��\-��i�DD5�]�����d��
�)|������n��Q�U�t������Hn���cV3�[Z��71�N�v����R{0e�v���N5n:��$C�FH�1zE�0�ZbdC�����bV�m5�����9�Q��h&#)������)�N�He������NbQ`nK��bi
�K��{����_,��|Ne���|������z#��Od
C���*�L�}�U>6����c��C'o}t��u��?�?�n���F��C%���(_
��u>�����
���H�s�f����L�M	i� sv:=���s��Y��*�#�����%��8�0���@A�A[,a/����r������:X)�BvuY��7����(���^"�";�m���^�`���x0������%�����x���]XM�D����;����U�Oc�]���SJ�z��pU;
���9^E��:��a�3�6N�fwr��=J\��e�
0�f���e�����#c�x.#���!��L�����=^��h�V#+5<�S�8�*����P�P���b��<������#����<{��'�d2��������v����O���|+|v���N$�NR���������g�^��������d������?���������3gv>t\1�;>d�31�X����3�u���h������KBd���bz~1���S���P�����K�2=�t�`o������9Uv����h����*�KrC�|��<����<��C���\A^���y@@����/�\O����� �i���4O���U���,�^��2�A�;�a�kx"w��it�R��;p
U���e\�,r^���[st�6��dA!�v3a�����Lq�E�?�K���:���N��P_�j���������<LVgu�A�l����������Q�A�?���9^��{YW�Li<Lw�#�����C�#��rn�
�����snY��b�N&��}0��}�d3V!��`�*�ZB�]��;P�#�
���8��a�s\�.V��������$	�>���Q(@��c$�U��"���=�8*+���X����'�
����o��SJ��>���@|�����_�J�#(�F`�e�������
���/8���:xvzJLw\w&�Ab+kF�in�l���4:�tVT}$K*����<}�X�0%��9�j�����Hf0u/��*��y��G[�2�8)Wk��p_X
F���-�0=�	~��������*�e5��v�y�o�t_wg��PR�L�E�e8\�)O8��gwf��:G�^�:��D���L�<��F��:J�4�}D"�bu�?fo.�^_]_�~�|w�����_�k����_�c�}Z)��3�\�P	�&�H�'���v[�f�9`���3�[���e�������t�ms�c �������(f�j����2f�$9t����{�xI�"l�6��<Rl�2;4���j���-S�j��9�rk�������4	����4i_��6B��d<�L{lY���|����F�@�F���m4����$��
�*��}���b'[���VY�>&�w�����T�%�{�)3�8����#�������R;,�o�	;��zN���F�fZ�b���yAeF�=1���������c;+�K��N��
r�������cO������vP�)���M���������cO'�d����������?�C���?u��t4������.�����
��b�g��U�Y�6c+CU��\`	h���cO�����.�Bjb_���/�6��k+n�3%,i�s����^��f�
p�.DU���0����Hu�*5����Gu��wj|T�*K%��:����"������`��?���F��GiM�K����&#�[��|l^T�Nqu,���I
��U���u�D}��\�De����t��D�����������
L�;��0!OuYK_��d��]����ik���'Q�^���'bO����1�@m�Td�q+Hx�+@�b���+A4ROou���	n������	�{�V��ye�+�4K\A�*��=����.ER����.��{Z�_��UH��n�0�t=,�A�C?T�"W�-���EI�[%N���W��+�4���?���l��(�n#U �QCx����eUg�x�h�6���,��Mdx�J�;��8�R��r-�wxTLz"�A����/?f��V5yuW��u�I���tq����������%\�����os��$�M�rV���t[P�Ne\�:7j�j5�5���A�
���?|�v�f�,���t�}o���{qB�{ps����&��m��w] /���^V�{gL�U���+<�?��){Y��f�c�V�n����y�@<�����w\����l4rF{��W!�'6v��x���+:u3����4��h����!(24�-�zt�����E���b[���N`K8B��gl���Z��������V8RmMg����!����+���+�u�T��G�-�hc��}��Qs��1	�+�oRi��~�{N����C9�&1�_'v��>��c��R6j��F�6��$N�8!�@��D�)����=RkO�?G��~���n�"��]n��VP�dYz�~���;'�	A��6t;SNi��/�)K+ks����]^����9e4��<Mu���S��\�6l�l�^gk#��z��I`���7(�3j�ad��8V��^�@1)��[x�����
���@V�3���S���]���`�NG������`g��4�.fU�@�w���?u������mI���+~���1iQ4	�:��;�-'��������7�%�y� �h'��o��' HI���v'��F���u����g��
[��4�� A�!oG,�'��n�f�������27����<-t�M�'����w�B�X2[�|w	I���	��Ln,��S�pq�= ������|I`�2�p��R�i��:��������|��R��C&�\9��T���g���1��5�Y
�CWm��sO�����.�f7|�g�G�^���atIf(�?�\������(���)��' ��3� �SK�����a��O
�
fM�L����9�	��u���`d�('��Y48|p`J���q1��u���{��M�����y33(q${R���M @;���r@�� J�A�����%�������=��{�e��T���G������O�$�(���1�LV����-^-6�x������'��2�����7/v�����\}(�Ad�i�Ss-�k&8#�x4G���:tW��u���O@���c�Jv<h)�lX)��p��h��h��OLd9N��S7L-s�D7%�;u��~7�Zr�nS��6�0V����V����r%O�	E�VM�Z�(�x��r����.�8���Z����z_�~xl�d���h�Dp��D�?u/���EP+�
��E=�%���!�����vE��m�P��@���!�oe����d�k�D��9�0�H��Ci)���$C:D����
|��=kn���@����+a
>m8?>N�4|~jf�������9�%�[�]���B�������*fc�V����H� ��$�0E������94hA�_,��MZ�N���Mfv.s�+EbcY���/����c�
,B��=[��d�3���F�
o��-�%]}�=�y�� 	vB��-^��u�]���YW���1���>K����TH����1������V�j�,M�n�
NP�5��H_���^�rD����G����E�?@�/�����ArV�����{�P_��at8���!dX�C?�S%C,f|4r���z3=�o��}$N�f
�%��
�6:,O\�����}�r��v�t�1�|?NU���	������+��p ���$(��^]?��Bi��AJ"����A8����r$����Ak{��Z���AsA�M��)���*��Yi\
T6o���������F�n����v����{����Qy����z�P���Y} 7�a���#������
����s/��,R��I���z�~�f����P_�;j��?�����j��{4��K�/���
�m�RL�\rH�N�:bd��:�8��R�:�3��{#�H�c�9��e��:eB&��Y����x��-4��b�sj�X
���b�>C�Ry%�sXU��9m������������G�o.^��=�#�Ia�y����7~��R�N@h�<��w�NzU�X���7o�����?{���5���O��q���p�,{���f�=�I��E���i�]x���W���u|����-@|����d����tz��9����<}�Y@�����W$����Z��A�<���[���8Y0��W��`�L>b%��FFX�
�s4ZOX@"e���M�G�cx�1F���ZY'�j�����59|?���O����j����+����P
i�w����Z�WI?Jp>6��*Q���x1�����O�9]I�h�Bhj�>�(�~hs,-�{�l���e��\? ���@��X�0(�8_Y8N-Y�(dm��V������sI
1#fLr�U�ja8VQ#&\�,|1��2�v�������P�=���TW/G�j P?�T�8B�t��M����:IV��e�oi�V�f�:����z��������e�q�q�&nI������{��TS���w�=��\�I�~K��m�Ce����'���2oe���g��F���O��o��q5_Z1�H
rL�5|�u�F�e�v)s��y�������������
��{X}Fe��7=�8�\=�>�{��L��zc�D������;���U��+2�	�Gf��~��@.�(��si)���3-p�,�0(���`�2��9)��?��s�=eMb�1�A?�#��YV�nGBcV�PCb�6"�}Z�����{s�s(z��C3�8������6/��^q��*G��
�V��z�!���
}�ko����a`���B&��d��K*}#O��k�5��)#�>M����P�	z�B:���@�6�I�>p�~h���/mBb}ZfP�S�1�s�SN)�__���g��/:
����w�(4k�^�b
�a��g��Z��@�5�Y���.6��x��>����k��K*���P�'����DT����+�Zx��X�+H>���xSS���tB�l�&�5�+���\s��+j���C���@_��7�!����We���#G�
��N�
f��ao(������"S|��U���l��m�F/��0�r�j���R�0����5��S%����K-?�{a���5��{��@,?2�_�oZ����7�[,FP �Fj�S:�E���;�|����u���"g�Z�C{b8%��C����c�FY@���O�x����zCTo/L��L{�'\*��_ R�am8��zG�����T8>*��K.X(R�|����Q�Z���U�7��C
u��p
��W�������|A���Et��-�y|-�e>%yp��.�B��/���k/c��7��`���Qc2��0*�?U�����m�:Gu��Wk�JiW���V�|��P~����nB��Ak�$xp����C�-
rmsx�`�����[c�
�	��#�w��3���(��A����9�Cx���-
�w�RN�2�)�i�8�:|�<�&�/�@,sOT��q�s�#�g���m�@WR�J�p��9�8�l��`���;�d�V�\�(��4J[��c���40��!���.�����q�Pk$��JH��W�:��Vv��.�\^�����a���Ilp���Tc��0��g��0�>6*�=wuT�������q�2�0��kc�[Btz�����,M���X��O����!�6+� Q��
����<�����������h�>O@H=V�x��Wp2�C5
�U@�D��dS�UFE,@�x�b�K���6���JT���3���4�~CJ3Z�dp\���`��!LSB���d�0�*�����N�����wU��KV�*6>g��g'b���x�HYV
+*�G�����%-�}EA�n�ZM�:HKA�=��)�����-�iF������4��x4���}K�
�Y@F��7�9�B���,%�'f�(�6�y��&���[�V*g��9@]�<%�s��������L
Q�*������3w��q{7�[s5/B�j��5�wm����F�Z[����$��E����P�+!�/���bcx����G-_S���f������4G��5E^��c}�C#4W�[^u��&���s�c���\�;���U:ZP�[g���K���3R]`�i ��=�q��w��Z��V,���e`6�N[�Kc5�s�
��&c�����!d�n1+RG|�b����Z������/f z���0��Q�9�Z��9.!�i�2���u���Y�"k��gKy$E�JE�b�m��g���z��N�������w�������D���`�6�g����2*s!\<�}����7�Y|�n���G�$q����w����Z�l=3jb��4���!3:���"�t.e�O\�y[)�*7
F���g]_��`.���:�Y��E6�ns����@�WE��t2F��N^'^�����a�=�3�!��+m�p�;��P]q�6���� �Q'�9�HgCT�@f�9����Bw31�C��O��,����e^p�m����K���E2���*����|��B��+���rj��o���5E+��k��\A\����b"���!_�H+�X���7�	�5���_�<���-����/������|G���f'�s��~�81��������1�=�s��}fc�3�a��]U�0��W���f����a�(��
r��C�%��g3C������������b`(��vP��`&y~0�<?�I�f��Y��`��~0�\?�������U����:;o�D-gF�{#�M��:b����m�42���4_�`�w&��E	�W>��r�j�t�Qy^3�,m�j��3L4��Q9$C�#\U�"5��$�$������MV/!������g��g�N�~e���h��W�u�~x/x�m��M�h��\GH�g�H��\Z��&4�v�L�F]i2�lUr�6�/9���>2��5�x��0����57�$����Gb�j����*�GT��������w�����x#�Y�E���5U��&C� h9|_h����V�����-��J�J�
���-D������L�n�����
������,�e������a!���Q����q&�*;�����pw��[yf�o�\�(��bYx���V&��1��\��/A�#��I�O��Cs�l7��I�������-g-�����:��d�C�2�.��b	�G�^�Sf�{I��u�4o��a2b51���$S2���~"%g{<��M}��n�?�z��J�E�1�������"��kQ�8Z/q�����H>X�G���%���
��[0��9~K]���������+�G���������~rt��}OWF��������O2�����yi�C���:��_�6|�I!�x���X���QN|�#���h��VF*+'g����(�wlu��}nw�&�����/.wC���-�gv���[R��{R|G������@��m�2�-�T��(m�rEZ���WB`����L�c����������Eb'��t<C�~Yq���E�������3����=i��3=.���6��X�����!*k���K��*1���f��;t��_!�B����^�0�@��,��/6�����L���V�}�W����;����a�j;�U�m_gX7������u��,��t��Ad+�g����jgx�~��|�8l�5K
�,���$��$hU��Bro���6r�&��G��?�&n�o�}�Vn9�"�vH�E9M�p��#vu
����R��*=�_se���F����f<����Z�� a�jN��Jn7���B}��4�^$��Z�)q]���"�|�4%�PnS%��oBY_��B��\&�v�kV�5�)����U-�|C4l�����bR�s^s�YEs�t;�P��@qyU�.b�oJ%j@�n���-)�Jvo/%���@�38N�ht������
�+��OB
i�"��*I����C��%R���8��~�
B�aHy������������pe�F��1Y�	%��
'Y�&��zK�S5z(O�c���A���hqv����0d��2SK��&�7�i������|g>��-���2��r2��q�^�H�`ES:�a�DM<�d������f'�%���'W��r(/�)�:C���s����|��8�[�f�-�T�*p��N�U��
%,�z]�L:����^d����U'g�������f�T���#�"�VQ��Oqj�K�V���(�N��V7�������w��&��=�c,R���D��\<��W�Y�+��,���j�C�at9�����y}�7���>�G�G��r[��
�#�9���
��oh�� E�&@�x�
��o=W}X���u����P\e�V�l��F����������A�7`X2'������w\����VA��d�r��BP��d�v�����l��5|�^��B%���{�Z9`������f��Z�U�d�1E�mq�H�(��o�����r5W!�/�JBp�qmF���- P���r����"�.z1�?_)uv��I�J����Nov21��(9����D6���boNqC��9uf����z	���|L;�[m�����*��U�8�]�|T��^�4;?x�����.����7i��|L*��~�����P-�U����V�U�2��a%��q����Y���-3�lF7�h�O�3���kC�ke���4r80�o��
��:�~ �X���c��d��'q:����V��m���<�(�M@���m�1��a]�$ ��'p�`������u�c����qj�s�2n�L%��60��
M�)�d�����fp�C��#QG���b�����|��vH�-Oqn����L����Jq�xk�*����!�&1Ln�G���T^���$�����-�
�*1$0'���fV�P�&�	��r���g5���^���9_,����������;T�+���`�#�oE$L`mM0A���me[�LK��������\L�{���<�i2�>����m�9T���`��C���K��.�!!�&�1!��^2��K(-��:����`$_��a\c���"���C,Wb�U�3�e=���[�nz�k1��fv^�P}�@G�S����P�"W3o���=J���(��������:v�)�s�Q�+O�V�\J&[*���p�����p��'\��������d��z�@�9����tz�ci�H<g���8��`�AFfo�*��+��f8�����*~4��&�����<��n@6���l��$������7{a���D�����uT��������B�������yf�;]�o4������	4P��g��-������Nwv��KW{�f�+)w5C�|��u-m
%�	6-{����^��%��k\��oY4��i��4��*X�8����T�����9\�����N�������pa��|�[�rF
����.���-
h5^�&���p��F�*B7����x��8��M�]x� ��@I��|7hD������[^���q����x����������0]����%������7Y>\������I�<mD�\�
�%>9��O����<<�F��lv:�����#V���K����Hdn7���b��B>[-�����|T���`9����nO>I&����^���C]����8��FG���H/�'���������$���
0�V����rV�s���&e�W����7)�;�^�I���\���^��\�<���g+�H�r��L}/%�}K|���a���D�x�5�H`=T�T�,,^�j�,����/U�'2���z�d!��|��n��>�
����;�`���QL�8xj�#��������
�@)�$r#h7
~�,QC�:�Q��	K2J�$[/��e�S)7�����*�jW~
��?��t��J�����u$x���*�%7c
��At��B �t��s��:�%�[�Es7����
�����'���KP�yUz
.����BJ2��-�%�i�Gv���0��r�����1����'�}����R����7�_�'������T�p���������>\�������:���D5y����L�L�`+�@�iw[y{�6�74p�(�PS��;x(h �Rr�WFJ5��0$��(�rB�"��6��m�|��@AE"��}��������E�`6���I�����N��%@���i�iz1�En�vM�w,����*��`RW�T'�"�-|5��	5���6� �bNhvf�g*�Li��ac��{�0�=-c+]����{��._Bd5��.e��Lz^0�	�AF����8�����d��asZ�h*��~���n�?O��	YM�p%���,�VS-K��V���Q9X�!��RW����B��Q�o���5f�3u����-u}��[<��Z�b_�4I��:*~zFz~����1�2k��If���$+���#(�r`�Z�a a�����q�w�p���wc�#=���5B8-X}�s�+��2����[h�B}iD|����R1�'�����;�v�����U%��$L+?N�%3�Y����=~���.���)���#<^c�Vsg���'�M�U�N�)-�6K^"=n[5UWI]�I9���-����7d_�����=��v�.mn��������%������u��b�7�������&P�5��7�����?A�um�5�PK"�A�{�2>�����w/l���DU���<��J�VO��3V�l�R�R�E���RE�'d�D��w�{�2{"�zaO? ��y����5#g�u���P\��;��JL
���s\���2�(K����>�4�����/P$T�z`����V%�4��F�:�|��~8���Zw������g�.\+���g����s���\�p�~��~�N�HR_8yB���Jy&����0)�S��XMy���Q���ns�)���:*�RzMq�#����	�:Y��m�:��V5�����dnw��!���4��|#�$f���;
��V��j6�q�V��8�d�C��Nf�(��5�����$���[�),���U������W�z`���;��}}�]������@w�����uO.�KI,��#���R�<uT���Q�5Uj< |
���D(s��('��m��6_�E]Ow�oU�����k� ,f�X�[��ZK�9R�J����*�m]����bl���P�X���w�<NX��R��I�!d8�"<84%?�K���%��|E�\-��=
�
MxX^I���~*�+A�(d�~	y�@����P������U��:K���������x�|)�g��-��Ap�4�ky��O��g����A�C�7�b�bm����z������&k)c�`\py-B����s[�:0N�!��v��|Y��x��G�V%R)�O����%n�|��g�{��*�
��UC�"����{l�$�B�p|����QS�R�wp��X�}$)�C��c2/m=����f�H}et3�i�c���Y�C�K��	��12�t9�	��l���P[L
����UA7�k�{?u(ue?��G����6���i.�r���l:oNc������LF�f�mr���|�0\Y����A���p�c*��p�������]����������Q�5���(n�<oQ��-j��R��<�p&Q_��$�<C�.�>m���{�������q�@���b�E���+PE>H�o-��3�+�x��R4���ny�\��}?��L�T�{�3��9w�[o���<����)������d��xu���rEu'��|�7)-Ar�oR������q��iIWr��s)e��a���`��|�(T�%#�`��$qS
j�N��4>��e�2�[�5�-$�������^�=��V#����Q��������_}�c ^��}-�P����O�e+�"\E)\�?�,�H�{x�=���B�C�7��a��0���<����5�?�G���Q�.�qq���3��[����?����^�9m;f����P�x{.�����g�y���fM���*/b���o���_��o�3���7�����=�#�����)���#v%��q�B�uqxrRQM@���7�����hn��F��F[�7��|���Rx��n���O�5G$����y��T"G�������_H�v7�c��+D��FJ��-�����v�b�/q�����juX{�W�.�.�@����g�����)��f�y�@�?�B��5����(�T�����/Z�3��w��[���M�an�����}�\��6��w�6�sl��j�Q0'���%��v� ����?�A�F� 
�����D���Ji".1����R@8x)TX\}�`HyP��U�Zr_���um8��@jua"����������������Z������9�wV�����Z���0����=Q������_i#�L�O��b�6��d������o��Y*$*����W1L}BD�����_�����L�v�K��P�FC��g$�����)���9���I�2�O�S����P�>!�]�>�G���5!nc���}���%����s�z��]r"�P�A�R}�5�]���YB�\;��B\�A�����	�W`���2��&��6������-3�����s���\-����#�}r?g��r�r?G������[������+����C��<��G�0�U�<�67�d�����8���C��k]�t'"��v���>������A�(���~�C�(�T�~�C���{�������'���~����������k� ;��Q�a�.���
_GQ�U?����`���0xI���I����iU�i�O����8	w9��O� ����%�I�(�����o3��|�}�0���I\s�\&�>����s�$�A����W������-��ra!7(TFX*p<�Q.��d� �y����%���:���%�Y���Q=O��y�T����y@B�y`:���c�������cn�����7�'�(��nem�m�uhUk]�V�Z�lu�������BK��{�����/d-���i?��P/������t�d�~[GY�"K���2��}��k5w}�����������n�V�=��Ul����L��J!3R������������>�T+��Q����T���������
�?�[PZ�� �`Z��I�1�X*�TA$?-������1�u?<����Q�i�����?�������jJ����'��X��x)����j����k���7��R��'�� 9��y������9��D����9iG������}�l��Rz�,D�H4����E$w��<�#�M,�B|�`������3��r�&�~c0�>�<�B�#qq���V[��z"{�}��G��Q�-���v��~��g2X=�xwz��;��<lu~�f����h������$��bHI����]`a���s�U�4h��@���x%����}4j
�%��S'6�t�����3%�%�>�0��1^���P
;�t�Bc*���Dd
����1���T�V��q�;�9��C���\���g8��oh�T�XL��?.��Ik9yW��J����2���Xn�5��0V2]&P@�%T��`�����:F��;#Rt�����������
bL��������� D��|<���k]Ch8������x�����hF�����B�L��%��q�],	��
X�~F!BC���l�<	�PT�I�?`��t=����MRy�p�x}�����9|�;��&Oue���c1�
�`�W\9�e�1Y<���S����E|c�1ni<!�����Q��@�����O^�Z�>�/(P�n/�������>���|�!�����~
�W���O���U����Oxi��as���3�F����jN�!�SW�����S�Q�)D���f
R��Hp;�dm,�����`4�h,�M��|�`#��L>B�`>�O��i���.H���8�}���D9��+c,���`� �R�mn�W��(��'Qc�%�+~��#��s���x8��m��O��(�_�O�P�!�F���C����h��d����<���pO�7���~�����"��+a~�>�o�s�4��7���>�����\��``OW���D�C��%�C�������1��Fj��i!��.L�h��4<JZ��F�H2��A��sO�BZmQ��A�r�~$�����7�9qO�\����/W�rZ@�|�-$�_������^����X��t<�>"|���B�o��/���a2o�]�|������0�.����v|xL��i��q�W{��?%���qcYb�������vH��\Y�De�
fx�7i_�������3�t�����O[r��k�Hzr|4��{�uG�v�[
I��
��o�1���A��<]��Gd��N���z;��i9LSo�\���4)x/g/a	��	d��?l79���z�J�h�!���8Q��$��@����c<U�&$Rc�Dml)�8��F��l
9t�_�C~G	w0�Er6��9��U��J���� �����l}�5D�&��	$�������3�O���JY���d��d��q������������3bs�eX�{	,�b���.���E>�\.Aq .k��<��~QG6���6��f�N�t�������=��q��������XY�c	���$^r/6���;�4aq2-�aa���H~�x��R���S]s����R�	�s��VX5,^~��Fr/��J�/����m�
��~&�0�P����Zo/��W������_�����=�oV������?����J&������N�6�yk5��4��B�q���W��`�D���u2dSG��y���qY�ox�B���1��4���I�;��h�����h#�7M)>y�}�
��Uz,()��&���@2QXRC����:�4����._����3HG�W�E��'b��1���#1�C��%��`:��9�uz��j�\�*�LT���i�:��������i�%��'�F����q����|6LaU�E������#�;g7JtD�Jl����i�� l_2��Y5�������N�//�6�O��&b��<��^?���rV�V�\�Z��.E�m`9Q�O��V���`C��}Q��)�Jn1,��dH���j����^�F�j�!�2u���r����3���@�8��!�O��%��,;#"u����Le,��	WN.+3�����W����W�x�NP��U�!�d��9���|���&���P���;�yl4��;�7�j��~���d�N
�1�^��rDh	�!-0j$t��l�+��V��\�D���T�����t=�*�����U�Vf�F����5����!�E�N�R�A�����x
(���$��	f�� 	t]r�T��)�NV���L:\�x��������p
I�Y_UQ���s\`�dp��Z���3P�~��Nr/!����#�.�ro@CL}NK_�`
�!��$A)@t�"A{����DW�> �s�������Q��"�K/11�c!�y�1"������9?�:���w�5��)5��#�#�u�9<i������B)��=xt�(��~L�b�W��32
�]��ty���y���^������3����Z���!cZF?����c�ru������0"�#����V���Jkc�R}2B��,��F���N��������hX�%n����#�o�1a��������M�^�
����������R�E���@qe@��
�,�yJ%�6��Y�L�y>�����=<����.�X@��i	iLhx�
�8H�i���uI*�y��0��0��A�E�.����
{@Ud?�)%�c�vNx���V�L����W���SUP.���2~����u,�M9����������#�)���t�lQ{�}��4Q�	3�J����#cW��i������=p���r(anG:�p^?:]i�n�������;�0�����'(���I��KNg��T47����5�0�dJz2NRB)�&@UE��4k
��_c	7_��T��Q��X�K7y���-:T�w��>qH��������W2�y��Mi����<�|U��A~t� ���� 1���O�����WD�
�}�����/9�9W�W5S�,22�� P0��
��T0Q�Xw���p���e������TO�^���U.�Ev)'St��G��V����%
:<���b�)�v�T6�z7�U��T��u����Y������8k�m��"+�-������v���c
��MR�=��
�U���mbB�x�I���U��>��G����/��|��0� a�Z]�Y3��$��?`�-*���X!Rg�6�KI�H����DiR$W��o�u]���#�#�������X��5�{*FKH��*K�i�x��Vl�#3~]J�g�Ua�5g�U�FS<|��������aNI�K��}�%)�Y��E/J3'��|�A��g�\;uR�	UTS[|�����5��g�('Q�����\
�=�j�������=��{
����F�2:�+��wU���N�#T�oL�����~&�������\I��A�%}�j�^;��L���f��9BYkZ���	��	'n~?�$�����Q���8o���@Q�|�	w��"��n+���\8����C�
=v7*O�)����G/M%��,���T��r��)P�x��D5���GN3&Q������E'�@2'�-`�:��Q�?�SBC�S�v��~�St8�"U�QH,U�I���[|���''~O�&1����r�Dw.��M�sT>��\�C�a��������kH�lB����i!i\rG�Ga��3��A����{4f���*W�(�0����7H��<�z�pS>c\�N��q���(Bw��N��	"�����\2f]���)����c�z���__�}~�7�>H���	����|���3�N��9����'{{@�X��TbH��`{4_����A+����-��f�i�<�������V�����X��:\��B����;^�j�����o'�[�an��X�Z��jc%O�'{%��W�����~�2P
�V����;���e����5��S� P��cmf�z��I�A]����!��e�!���B�$(3��y2Y����DjSm�ws��;�����1	�o���PIM�A�!����6j��@y����p"f�����������NR�A���S���+6W������2��w�^���#Mw��}�y���%h�':��~t1�����;�����9l���798������\&@����1D�=3`��	��&��E?M6��2�� {�+[���)��P����XY(�]!�l����4��L�yX��n�4>)Dl����e��3��er���t��7]��|<�)"x����\�
q,����������':x��	a��Om1�?�Q)0�
� ;[���7�l�������d9��������T���!$s��\��4�G�o�5����|�<m<��T�Mt������kN����2�lz�w�c�@X�	�u����&{��5�����^��|��S'���>X]'���T��b
Y/�tT8�RR;�m5^�x=I�"�<^�W.�X�5���r%�46<��Ju�u��T��������f�� �w���-�>h�T�������j��#�C���D���]�.���^B�V�qD�����Y���9��?�Xg�,�?����lot�R
�������K���Kv��4�� U�*��t�������4���D����Gy�k��m��	������IEt�#��d]f�!C}|owB�����F���=�sI:��<�O�m�V��@nf5[OS�6.��=�����[�<��W�����*�!n[��Z�x�nh�I���([�[��I��t�P}T��5eh�:j�����\?�.���"T�N_�!�?��C?NT)���4Q��/�D�W@i��>����0���3�������@����bF�����>��R��7�a�y��g��mv���H�Z��
�#������
�-��8�e�E!��{y��.e�����BJ�d� 3�6�}�Z�jD�,��eW��F%!W�B�OV�����@��I!��P�u*�@)��(>���i�>�@ ��c�������W���I�i����N<���%K��3�k�suu�1����lm!&��f��b���w����L6��s���e ���
�%���eDX�G����UI��M��}��N�� 8'9�q'�H0�=��h��#�#w�����J
����b����b.��h�����B�<�������H�gY�%%0S�!������n���{��s��Rm��x�WE�����4b
kX�	tqU��h�~�k��������h6�aw���v�-g�i��������Tk�^$���0���|O��`�������������>���������W`b��)6`{f��=���S�'��n�Ov<�\K@S�kU���5<<}!+����3����7rQr�;�XO�H�S�n8��Q*�'���&�De9��l�6�
�t���������%+)�3��-9i�J�	�"�esh�5���}T���\ `_�"��6�(vV��X��P�j
H:W�M�`��8x,�gS��
�^I\[�T��Ap0��P��
���Q��o�vL=Z�����l4�Q���7����SX.��"e1�~Q�,�w$R�{�Hv�a��AFa��$hg�4Ag	��6�I�<�2��_�6�����1\�LjLS��B����=���h>���2��4U�'�3w<�X�0i�!����l�I�k:I��������e}������~�'k<����[T��(8���1���h��q��/���~|�D�����c��F�N��4nZm���.!�����j�3��z1D�s���	 ����~��~e��x��S�C�!�}T����-�IMH_�Ze���U�(�]/M�/�7�����>V
�F�6��^�:�����^�VJr��^'�������c ��c�g�����C�{5��ET���
����P�6���N�M���G�����|�%�����hp������G���sR.9�\���4����-�OD
+���P��\�R;�JN�P��*U*o���@r������S�T��IA�R��
�F��\���*������v��~��~�u��?n��7��;Q����j��_[����s}�������1�L��gz���w�bu�^5y�`�����Z������<�������`��f�{x
;����p�D�d��F'��x�>I��V������V����V����m�)�+b(��j��Te�\���WS(����<��������`���������c%�J��T5@�6�k`�N"9���/�
�B8���,7
��8D�x:��+h�3�R,��$Xb2V��Tw��u
���r�(U�������m��_E���^@���$����%�`�=-�Q���������T������"eH"hrv:��Q�����7�`���=9t:�F�d�F���>�q�VZvHA���N��
5��T��������w@'�7�3��h�O�t-�o���|T���\�k�J8�<	<`��X���S�q<����OYf����C�����I������p�h�����ox��h���!���@����Z}�$e�g]�TA ���P��bzi�����2t���!��<���z�SM7@�]��b ��r�+2���o���w��+<y��O�s�
.?�x�n���!��O+�1����C/��r�Ox�����p�Ig��Op����\<�fG�p	>�Z�E.���S��
B4�%	C�3�H��Pu#9�p���5���-`'��P��A	���)��0���"J���77|H[�Q��s���8x���xH��&�����(�&�����X
S������[U��9���t�F�+��|������=7�d�d��8"x�������7�}������o����E�����o��f>��izY��s*��e8(T�����{5F��.����!��X��������=	t)�sBA��'�!�������6�z�B�#�m�	���g����� @=�P$�������G�_�|�h
_$�!
�T�2�DQ��!��c*�<�l�b:���g_��xyY�����"�T�/��5��9�PL�0���D�KB�����-C��t��_x��?�Z��QY"��)�O��h����!�Q�M����G`i�4�����|m�h)Zk��Q;���G���V!Dq����B�l)d�$Pw����k�1�� ~?�m7!�8+�
� ����c@����=�r�g��-�;�Y����B��EJ���x��dW
��-����,HGZ��9L�4�t�����u8���[�����%�[U)���I�L�C�+���c��G���o3�?�CuE�s��g�}�75����3^�T���������!�,�)����xR��S�B�@���E�a����;�~�B%����1��5L����S�!�4C�N����-z��o�=�=?���u�5�����7<�p8%
��;:��lG���	��xgo����F��<��N�&�E�J82�x�������Bl�Zi#��Z��{�=�T�4E����4��"�`�|L�:Wl�?���Nt*e$y=o��@xy
gX��z5�����9j�/�O/���{��?�����j��*x��|0v;VM�X������j��	�Pe[������:T������6g��
�gv���=�Es����+	������F<b��D�(��(��-��:d�2������u��?���n������M`1��d���K-��Fed	�����H^���GG(4D����p|�q����L&�K��{��N	dX<Ed��cL�P��pi�:�]�L����z����)�2jG�/l�Vn>�%�|6����d.����X?�J�����-��bEO�b'����
�j�]��R�����o�'\��u����3����mEG^��v+:������������q��st<<n
����I��?j��V������$i�������r\m�"������(z����h���C�}��}
�������7'9r@4Y�R�H�O�(\Q��Z�R��?-1�2��[��beyS/:a
�u��h������K���x���(L��Rd���$]�)b�T�Vn��8�0�U�t�B �<�*�R��)��g�l�%v���y%��\&����J��-�r����>���T4�SN���f����`��J�TI�Z':��I��_szV�-���Z�>�����O\�?&3t���	`u���sq�T�$��]
�0�I����:
e�S}�6kbA}`
&�T.P�:L(�7�L�!�SHk�G�+b���ze� U8K��9]�T'�@��\.@c�n��T�XG�)�Q��
 O��R>`@���>����d�\�$o��5���Z*��-+v��?�����gp���lS��������	@�#N$�)�-���JK�DE��)`�����W�xe��[K���U�]�S�`e�L�Zw����\�f�����S��j��p����Q������>w��Q��h(�����egO:��~�~��6�)��V�$`�'RtV������4)���E^�~��C����������2����7���C�dO�����n��������]�����?�xN�jb��F���!��J`���x�k�R�y��,����mg/�<%7SLUT%�<R����v�;�<I�!��Y��P���S�vzK����a��mR���4�T�H�@a�x��	�'P��b��U�X8����-UF��=:l6G''G�n�4�p��H2������3�7jZ��)�����8�@t�l$�wZ�e����!/�s�P(4:�6��586�������_��4%8�,MA=���2���g���W��w��0�vc71Y�U1���&�&�Nd�nEd�o��j�,��`] �4/���QX������,���`��%Sy�������Cz�GL���9� e��3{�GNr�
A����`6_Ab��.��Z�F���~�2�-����V�B
�L�FLT�f��6
F}��Q����2�X����(�B�}��c,��&i!�`f�A���c�k�L��}������<�	�W�O+7<@J�(�\��jr���(#�]�E�ssD��)5�k��>�*9�����)��Y��x�[��D��
y�^�� ���/,T�B����&z��h���U��|Ipj%��x����m�kZwg����tJs%���=7s�<Q�!����.�G7@p	����<����AE�P�<�Z��l�C�a��{G��
,z�*-���rLc6�Tl@I�S32�'���J�4�+%@2�b���z
���S�sW�zpE�v�i\�@�e�j�R��[�nqw,��z*����E�V������@3�ZPge��x��>�m#��O��@�+@<"UT��5(���GT�b�1Y",����8�>��lp����2T����T��H�����[���2�G�
�
'����v9IP�8��'V�8H����&X�b�|�����
d�fw"^��MH`���O.�]^���;p��W��P%}�t��/�	�aZ4����=���5K�]��d��u�^��B�q����N���[��*�cT)J��t%�*��'���IY���*(b-DJ����2�����)�Df���S����I����:CX��5����N,���Y����oJ~��:n�9����	~�B��#��W�.I�N��!gX���3�1
��;�KHS{�}�k~X�����b��iZ%�.)�2n8o���o����y{~�;�{��7��������T-WJSa��yO��jo���@�x)�I�����-<��~IE��r���t��	�'�����p�(?�����p�����|:���Br���g����5��53#

r/OX���p���aL�N�?$�~P5�v����5���v��+n��ax���
�=5�'����8=���(�R^��!7,�3���G!?�q���n[_m��M�w���+,m��0e��<���@�4 ���f�vH���F�kHFEbL2Z���R�iZ.�=C���3���&l�*��!�aV�~�8��i��u3�#<�����#Nd��������D��p��L��	%E����EQG��M9���6������kIYe�'���t��n'��9�8�g�%��r�����:hp�Gr����~�n�|��%m�����p�'�>}&���S��=�*��y22'�z���v�c|V�M���Ed
w�m�+nR�9[{Tv�%�Hv� H�g���RW���R�d�"���!�l��+I��Y@o���'����A��n\F�g����&��\���]i$'5
�))�W�a5�O�u�+�Y�d�Y!�y�X�JM���r�,�{�c����w�zH�t��m��������n;�g�w���}���i��l���(Au�\����fI	�3�Y�u�d�nAn�@����V��<���w���kX����j7��j�l
��EI�����q,�c�h�����>:�G��v���foh�g-w�9_�_�����Z( '>�R���BL��������������r�����+3%��o�.3��SB?�2���k/����'H�Xf>^����$��L�j��Rf&ep<uK!���67j��rhZ���B�6Q;��E�����|2'�N��S��o<Y�)���j�d6�$�X�'�9Yw���
��9\��N��\��+}��u{�:K�'����"$�3�(Y��P�<���8C*[r��*N�>�H���	��3;���[�f*�T(y�����j
>�l�F_Ge������pc��sO�m�����?���4�#�%����(H	f��N,����M�i��F���	�F`D[!��������1EG�q�bn�a-K�[v���>�l�s]�Y�����|*���~=��\�[�����4�8?�]gfA=�@�0�Ke�3J8_aY�X��
(d8K�K�|R�S��D��!���#
�����r��~������B�(	���CyT�>�Z2u��;����Q��	yf���	&JG��it�#|q����8�����dO���
qSEa�s6&PU{�N6�����T��J����Yg�u�}��>~AY����(�����ur��:�f���1��|-��3�8��V.5I�8��	��x�9�8V���(j���,�[�����S�\cy�B�=4�O�1"���RpK,�����~�BA`0"�u�����a�u��)���qL�,N�&�)���	��u�E�Qr��n��5�Y�4��S�E:.�8$��W��D	��b��*yn��kt��@(71������LU�����b������{�L�@�#�F��?�?�y��c��p���D�al��1[O�x�U�5P��
�djfY��B�<�!��jf��D<0�K�YCU�O���e��p(&����m}�� �����h�-
�^��J�kc~=��~j��?����'������o?�v�kO(��c���#k���J�#�'��H/��BEu��"��	f0��P'y�U��UILL�P~����
��?���o�]��9}&�����B��"����d2U��[��u�gf�.���bS�#�-���=����b#��}�s|�����{b��J���s'�jyfb��Dyobw~���������T�rR�|�k���*a�l�$�H�%O#&�wC��/�G�vf���=��<Y>�����p��/?<0$����g�������&z�	7�2�����O�
�i���?�$L&�A���#�}M��O��������A=\3v3I����=(&��G�n���Pg������l�����YQ4�s�d�&�/[|=B����������mB�����a%I�d���8�cL���kcB�>�v���$]�2�K����g��p�Ns�L������N��_���x�yQn3r�D���F�86��Ra?^GE� ^S
��SMu����z�B{��?�#0����7��{��xQ��:VM���PY�^�p��8��^+����������9�[fp�\��>�zt�}t�H
 ��,�Vzy���w`N��1�$q�G�~v9�_���c���D�4h0?��b�?[���l���C�p<���	���N�l
\R��R�s!��N���3.�q�$���
F@�[c������e���.2-�1x�_>b(`�Eq���Oi)�;jN��Xn	E@��?Ap�h|��p�y�����p�.��Y�N
��"��E���08�x5����>0l���+���Q��?�����7/���fv�V�i������sK��U���DX����d��������Y��� <�1�bk������!R��9�����F'H��RX�����9��'W-����0�N����c<F{�k��m�A��2v%0�d`?�U�^R����E��v����,�������{��X^�sy?�]���ol�\es��vN���(j�O��,WY���(����[��}�����'	��j�a'����X� s�?�vGL����M�Qx�R���DF��or��A�w����6��0����-|������s�o4��ft�/�vP��c����'��:���Q������)���^��"����@�r>��b�S�g�m��(k-;��48�v� 4{i���7
�@�1c������r��L��w~��N�fo�'���������g;�:����g}�LF��xvP`\����c��n�u���=������9~0��I<�%7r��$��n�#��Q+:�Hf��5�7G�����}
�M�)����� l�8/�	&����,��E�(7����D��I��>��O�)[[��K�'��+2Q���*�m�H�`0P���j\0��RF�A�(D+��e/qje�a�����%����^���.�U��2�3)�a4��z���4\����W����\���V��r6������l�i����L+l=/@)�d�s�����{{�c���	�]���P��R�x��t1���`:.PQ#���')��=�jH#�"�\����RY��E���g�v���b�4�g������},������Pt���v:�@dg�6�KJ���$���i�D�!V�@���QlJB��Z���9��\R1L��r�D�1�$�:#*p�8� �O��'�C���X���n&�m�aP4�
g����p[1���iF)��������d�8�������9J�`m����p��i���L�TOT@���yaa�9=8A�h=A��#���`},����F]S��O�2�Z�W��������&�i����l
���f,�������kEG��[:���hc"5���N�~(�;\L�O���M�S<a����u��?�C�H���}���j���Z\���B����X%������}���|�o}��g���=eg�{����^�9s^{9�7���
�0��xu�?/�yu�;=??�{����o���	��
Yt�P�p���Q�O`�t�;'��a:Qr�=N�Ir���JE-WT
�J����X����	&p���Y��>��IN��p����$U���KRA3�?��z�Q�V��>�v��I�M���������x����O��/��BP�J����}���;�=\��!�/:mp�{��G�t����
yF �~dc�����������O����J�L%�2����H���E�_w\Qt|��������x!�yY�����:n���L��2C���5H��)�dr0Qm�K[�BJ��K��~2������G�NJR�&<�@��P7��Q���Up����,[?�L���F�V�a��#��BJP_^�3�(�w�l����7��i�B����_���b�y���B�Qf��	����?�	�'K�=;��$
�X&@xR�91�a�
'�9�'�P�h���:s��R!'$YdJ`q����]������y��5EW�/g��%�\��i"���h3�Nn��FD%����M�i�"Hw����^���0������;{l�]O�����b!O"%����keH����Kg�;<!;4��|���xY�*���
Y�r��(���y^��+�2�����,�DYr��F�A��h�TRUy�9���=����g���>=���nJ������K�F��_��~��a��m����"�HlS����d��ldV 3�Z����X^�U4E��(�ON��%�"ZON�k����w$�k�c,���I�9���t��<��}����B5$u�/hHC����@V��S�5b���y��a�]ow�0Y^"�a���L�=��Gr�Pr����a�=N�����mR	�Bzf_R�^Bzk�M�rm�I�m�Y">ta>�UN!�7�[�G�K��mR<���a?Z��
z!�H��Qu�����fm7����ve{Y��U�,t*
)0��f��W1,��������_:�L&a��r'���������-P�+z7e�����������rDbk���K���g���z/_�{���������|9K�`y�q���_�5��������c���������u�����J�\%��W+J���!Ot.��������sl9d��*��X{�?����a������������������a�(�F�q�p8�n'j���f'�w�Q���{�=����������v��Z�<�����)^�R=�:\%���Xr��I��.1�:+;&V��dX!����H�a"y�IJ�����j(lFcl�g��_���,A��5M�c<_J�!r��4Y^�L�%jituMl]Y��P��c�7��w����,��$��pt�)j��1���r������5����a��w�:�C����'P��Euh�{R+��*�����Pe\��3�
��oT^����)�	��P��I8�c�z��c���?S:���t1��s�w4=����Cy���
����� ���lz��o�n�7���s�OD'S�X�~�-$	�����6|;�K<�Z�![�x�?�'QkS�Jv@5Slw�s-�[T���6#�UaJ�>���,�T�W��e����Y��%|�in0�zk�^���q:xx�p?N-��&&�{���W�O�R��-�	���`C�*��I���4[����~bm����V�@u}U����%���Z%���u�q�|JP��a�
����h��5�w�E5Ds�K?��
Tl���q������f�����������e�w����������tB�y���{:x��S��������vOE5�]�w��������'��Z>���|*?|n?���{�$3��`�e[����@8�����=j���z���{O���\��)3�Kz��	����� �<e����_5�#nv���%#4���J
�T����3���|�P��{�MN�����LG�h�g8���S�7��T�������\s��TM_��o����-R���)�\���|a����820�>%������k?���tN�r:�U�������a�C���i-
�8�.������|b:�,J��$���,�h��(�����F2���X����W����Q*�
��P�8b����{"�kzS�����i�d��R�d)��G#��
�+d���~����#]tt���T2����^=��"xD���c�z�J	�'f������
�4G��#ir�
�~W'������p�C�c��T���>Z���6�1��4��R�XK����
0De��<TH��G��.���N�<g����-�������	dt*DS����A��u
1�`��d��JG�$��@���uM�L�4H���N���^s�y�
�s��}
1m��qA��:��Oe���3S�|�O7���x-���j`<���*/�m��UcU�=e`'	h�0�������\�����TY��k������*��0t�M�$-�(*vP�����E�BL�����hW�3��d�Z�����Ay�:�3��K� ����@��TCFoi`����l_h�h�D���7���3�#�����)�f���-�_Q'g��"��>pf��1;����.&zH���Z�Xl&��������<�^cvyh���9[���ao���/:���0t7���$�l]���=���D�,B�}�hU�{��-�i|`�B}�K�xK�������@;�cu��|��du�^���y��M����~�
s�X'F����J���*�)v���3�;
�0Gm[ZWg"�k��A�����4��`"�b�Tj��O#RI`���1s���������'y�K����B�oPP�A�)s6eX��.&���*O���	��F���
����LWE�B�1�[;h���Z����=)O��F�m���d=��Xw����-h+�w|�@�nj�{T�`)E����f9���m�T���?������
^��3�q����Y�}��v5�s�S%��s���.��x�](�n�Y�F�3��g��q�?��Zs���d��A���S7Y����Z�=[���Y�85i
��d���)[T�x�����F�u�w�1������u�h�����I)���B��-`�I��i���������N��!_�
BW94sRa���\+��� ���,|��<�{��O9�P���=�Tv1��j���=�s�Da�Kb�d��������})db�T7c���/��.�eADt�9���%��P��;1af����+6mp{:7�����.b*���X������'�9��QUq(�C���/��%%���krs<�����A��a��k)�N*��z%/�!��r������N��4�C�_*����-`��*�T���W���C����~����W�-�VW&��Wc=[������\���S�����e��G$�[�R�t
�&a%1��s�c�_7���.@���#8�����53�b��� �b�[����0�;����Y����E��n<+���B����{����:���5UY�Y2��+k�w�O�Y'�|P�����V+�(m�-�Q��KP��#����n�"����7�Hb��v�
�"�DI��~�=���1����(�������j9__�z�z����qi�s@�	��	�"p�Zj���'?�S�z$�� �YnIX���v&�{���I�(dV��I�����%�g���� uI,�R�[)����3S�x���eL�31��;[�[}�*���ZJ�r���9����	��2
��x�Y�M�o�'�t�x��������K�F���rl�l/)qZ���C�9�1�O,e`�����Z���������|\>_+��Z
!��p'$rKi(e�s:I���I ����X�L�Y(�.�K�1��kT����6r��0;���d2����$?�#S'A@l�hE�-+p
��O�Y���c��4%��s���Z�ZC��1���a�?}r�����a��������KnR�!�s>���������?������cr�z	��e1( �9\�+����^�������&O�S\Fi���n.��zJ�Csn��
�v��S?BJ��hO�����e6��p�*H _����($
�Y��^��8�q��)�i{o���e�<0��IK�!%�������v���y�SRj�����l&��ET���3���yk�&��U�I��]pY\�	�<�h������
�3|a��M{5W����n�n3{f%�C�"��*G@��7/��>8�]�|=5����w�vy��2X ��l���|����/�`��%�����nE�7D���ld1*U�$(\@6����'���O]�a!
*T��\r+��t���; ���0S]c�>�RC����A�X���|�$���:��|o��
��^�Yp���}w��w��U����E����w��@*�]K�H���O^�4�  ���w�5�%����dNHC>����R~dyK#���g}4��WYz6�gs������s�h�a�����0(k[e5�)�d/z2��_Y%��1D�������_N9���O����g����Ob����T�6U�����_������c��3[��v�����G�b������_�������o���&��eO
j����	)�u���	������[/�0m���6�G����;J��l����b/%C[��J�����`��Vp��um��k	�E��VsK�6����3=���TF�j�"��D�Z���w���_�V$[>��*]�NM��x�?j4����nwM���FM���2�a�����1;�@B�%�e:y���+�j���n�*xM�����N��pb���7�*�I�o!�L;��N���N�F#��@&��=m�#�-�Q��{$�N���u�X�	�����p�A0 �1,��N������T�D�j(J��cbQ8�U��ndP��)r�!�E<�YQMdX�D��
���`����(��'z����oK���<��st`j���dS&y��
E<��:,$��9�a�!��+�I��g$�<��C�u}��
0�`O�l�An��8�G�Ni����v��Ai��qZ����b�+���MS�����'�-��J�G�@)��hTY��nirO5�!"�
4�Kq<�z��������$x�K���J6��@����Z`��W��=�I���`9�H��t!�]
!W���������]�ns?g���@��}��t�^��Bo���4����H��3%=&���e�&U������Y�Q0���=��=%�>)|\O�	�*���
��{�o����������>�����p�����6����a�}����9�f�-J���>L���m����(>>)��������m���@r���@N��o�!��+b4���������x(ao��~����
h����m�=���<��	o)h���OA(y��h�_�I���Kz9I�S,N�:G�.�(�RH��WR��
���	Oiw93���&����,;��XM���2z���n����h;t3N-�.���]�#n��Z��Aj,o�2��9��5���z�?��Jk��b��|���J��ZN��M�[9�~|���,e��d%E�)���JMw����L�Y?��jFS�lap`DoXJE#�4�p��|.��&cW>��\9c���Dt<vGm�Yz��(i)5eT)&�I�Z�h�B��-3I_Sd+<�j�G,�!2����!��		�@`��-*A�U�l�$�9��l�a~C����L#8f�y�����M����2KF�R*��Q�5���.��h���I�d�5*�Q�q�;���)�2\�����3�yHAt��wZ�qTo����4+�Ra�%
y���3��CN��4���e�\�c/�vb���p�S�y�1�A�b����}� �=m$
�02�G;n���=��8@Q�lk9I�U��PdH~8��Q��qb�4k�#C���)z���e��tw��2V����"��i$��-��P�}#��0j���^��_��5�b"��
�g�-�O�e����f���j��-Yz�V@��s��0�}Ny�\�X��j�:����X�r�^$"7�[3�F���d���*����m�g3J=��ueIt�}!s�c�����o��=G[����5_Rn�	b��ZC�"�>�
�����@����<��	l��H�TL�u������J�MyBPg��w$}X�\�)��p��8�n���5�g|� ��*5��6�������*��!�s�����d<W�L�=S�K���j��8u7��(��gA0����o�R�B�#���%)�W:�8T��NT�uZ��V�f�ni|,��c���Z��c��(����=d2�l
_��]Ck��$��3�?h/�v�{�n��H�:l�>f��[7{�h���;�.B�^�}������j.�>b�{<^��`��o����m_������Y�.�s�<�W����T��&�9�oo.�~Wc��8
�f�_�Y@!�1����w� tj������@��j����������lyk��E+����(��NxW�I�c��Vr��($���*�bA����1�����x@��E=���y����9�;�<�;��������[�.��D�I2+���1������0��8$�����_��(g��f��~5���E�]��� �����Zk��8V6��\A�}1/����J�f������+�V���1��E�>l��8"�%H �����������\/��������)�
�C:hV��?yn���;�,�XNZ�v�f�e��G8xr��W�{Y�������;&���<��X{��P��c�����	����OTK84u�q���.a
}��0�c����vXT�����Z�ob��7l�-��/Yh�������O��V����`�XB� `o�n�)���B�
������9m����
����	;RkS1����'�m[�j�VU��p�tRs���t5��O�c�vrh�M�KF^����������������Y�B-uZ
�+��@eg���-(�#dZ[�.��G`U]M�T�u�������Yv.����k�����>&8�b_d�T)����w������#��l��"�e���oj�\�;�-s(d�����T�V���)w������W��l�%��z.�������8o���(M&9��#�Jz����e2X/SyN���Qlc�Y�Dke��h�|QmE��*r�\'%s�!_����F���G���	���������y%4�����_�"d�#���<S���UYI�u�����z�nz���C�2�X���v6_�K$��m"?����,m���F����bg�`iK-���KV^���Z9@�k�3Y�#�K��N4����	���|���[����AqO���������V�Qd�V�����gd�,P��<N�D5X��FQ��F0	��bg�������SU�hP
-�n�S��p�������_�A�yY��']����y����Xx
�W��n���	q���1��
DC�(�&��BN	����-��Z��3�eT^)���,�����\����_�~ �������th��]�,��h��iP�9�U	������|�b��EBm��������Ua�]|����f�i/�N���sp�K C�~"������%�H���-�����YB�a��Pb��l�������a�&dW4�|��a�UD�U����:Z�����YF�R�:�g�q�-�u2U���C��t8��I��9\�����,t8��*W	���1	��Ze�VBT�L��������Y=��R)�u���r�\A����\���=����:�>Y�A�??0��i^�9rB���E��z�JU���"@�-��W�:%��|�'�g���&���7�aRn�e�nu���@.$L�����9����"x�r�
��o����9�������{����}��Vt,��T?�tVJw����f���4cR�z���"Y�h�|��-l1@at��r�q��{B��^L:��+L���zZU3vK9T] i��-��@r@�����|�6��/a����n<Z~���d�\u�0#'7�s����L ��|i_�\ 	��AK�s��(�H�CU]��jZ��X���jD�HZ`j�&�j�&���SW�9[O&:��}��NxBU;�����N�����vd��:5�yW/}A[�	$m�j�F����0D�:�����1gHD����4�Q)`�����6
?�ij�Sr��J��F	���lsv-��3�p`��m�{�)1g����D�>��c\����4s���3{d1Khg��S���"d���X���i��2~M������Y=��#�h���Y*����0��oL�e������j`��>�@������R���@j��Ch��t�d%��
)C?���vBAK�r��x�|O�m`�[U[%��:��Y|�PJ25�sf
�s�b2=�r8���,�L�7v_�y|G����&m�r�jJ�����~��.z\��4�����R,��I`����Va���|a�2�����	�)���")^�
��� ��]�O����-
V������F�[�GO���BE�Z���=����Fzw���2�HP����<@BGI\�:��]oz��M�p��s�3�cGw���iZl�<\���������<H�/��s����c�~���������B����8�o4�I3Dr�`=G��z�][����T�$�p����5�%�:�JS>���������^���+I*t������:M~ovT�h�p�kt�T�-1F"]c:�x����T���=�w����07�h�H�{]����gU�����fc�oB�(r��w�{�����Tv�����OL[b&]/9�>��Y�[���g@��77_d�
O�����^�	C����(�	@��G�������iJ�/����!O�`q]U�=w���&�t)��=����
;�^}�.�t	�
6��hH���W���6(�i�C�BM3��x����nt�u�!u[��{�
�
N �.�d��
���]����K��:��)��T���)U��K,�b�w��<E��[�S�Q�aV��\���*�����k����
����b�u0����[*	'F��*�W�����p����M����*�)����2��_��i��]�Y����O����r���CB����(]Bjb�T�C��h�Y|C���h����p�v,XG����A���u��ek��"��oK�	����`���e��R����������{@/�����{'IC����V��y	�������Z�vS����f�
�d�������t�	���X\B���x`�-�l�ne0M,�����H������7$$�K`T���2jk2�:Ph�k���i[ �B�Qx����A_��C}��Z�a��
�1�g�[�����������w��9�+f���Wn"d�)���v��6�#�q��C=;Np�jo�B���x�������q���lw��)ue�6�)�a��4
�xN������|�u�	����N:\�mKE��-�>�(��N��_���t�[5C��_��5���f�{�R|Q%Q�9��|��\��=���k��������R��H�lM��T�P5d�
�P�y�2�WzZ������v1y�_R9y�&*DB6�x$X�L���\��)��]��k��Nx�����ta�:�$��ZB2(@�SL�jBv�;U��X���%��g�%�����=���]��V���F����`9��`��:z/�Z�i<\]���:i��Y�kp~��)���������qqWy��V���3�,�;Zd^/��(C���R&3��}!o��1O�$�G���F�(����<��h#?g�2O���
�\!�^���������B\=�{Wd4�w��.T������I����n1���a>�N�I���/���Iv������;X�b��m0/�p:������*�N]������Y(�N���K
�m"��m���Y�,��o���F���4�w�����9aUw>������k�Dz�$�r@��@v]E_����r�PA�}�;�Nr��Cm����bji%P�
-9�������e��W�L-R@����L*DIzca��2��$B�H�WB~9�bE5�� ;������&\><�@m���m�	Ej�TU��C;"�M*���I����=�5���5_��6��[i��q�RC2��P�l#PP��`I��sLD��%�����S�1K>�]cl�r�rA����E�eD������h-����/��l�[��%��R�NP���l0&�[��i,nD��3i���:�`����Fs�0NWH��L�H���������6��&��Y�{��C>n9@@�����WC^�E�M�� �5@�m��y����D���".&��=�X�g�	:��KH�t����a�e[
JA���~<����z4'�t���A��B��(H`�>�-�R,�RLr����0'��=�J��3�g
��{�eL7�S�t���\PVb��v^7�M�VZ.��TL���dM0Jo��T+S2��Z����X��W��*5�E���S��j��.��1��v�J=�o������`����J��Y����1
uX�w�W3b�����,W�:gx
�NMY�`����f^��p3���(3.kcf�e������^P.����$���Zf1���q
(�J
>p�B��W �
��!7��D��V6;:-�������k��h�i)��d>B��o���G�I^�!2�
����h7�unS��
`��k������_/)L���7e^}����Y��LCG�%2��	��U�;���Zb�.����H��^(R���1�5�a_����-��m��\W����V�bS4��:����_!�O���
��V�c;2J�����O���8�E��6�]��F��H�rYC
�$��Jh��9��������L�m������Be<�����U��Un�������������]��������+�=�k����?�3�g6�����q�����w�h������D&�a��|�T����h���~��?�l�oU�-
��:k�*M&#;&�?�NU��H8�Q!���Nk�+N�G��������[���L���Bj#�/���}x����hDFg)����s��>]Wv����$O/��S��Ari��#��p������+ ����5��W�������>���!�c'�������$���C�i^�}��7R����~"����z��U������y��}6�I���)$�?(������5���JMr�f1��R�PT�Q;�N��S�+(�ma���Y��xP�.���/������
���lu��Q[n��a����v���S�.������~�J �����6_�9j��zv�������$�4�S��^�@�*a����q'K�YSJ*7�Y�0w�������S��$�%������B�_C��dGc�u�s��:rX�_BR�I-�fQ�
�U�U�W$�3��������d+W]3{Z�D~q�T���z+5�I6U��1�����������d���B��O��Jd�B�C-j0%��FZ���\����T^'g	��$9�L����b@�S���|�A����G�	�P����S�����T���Dv�&�;%�2����b�
�*�^ @E�q
y��y���G`�1d��\��&�s��V�����=WevX����d���1j�3+8
�}UO,�~����%����wP���_�D�g��o����z�w�Vb]:�iQ�G������U�����E|	�����&��d���%j+��M�L;� �����r�y��>%!`����?��m(���Yk|�K
'���Lr����j��A-e�?��~P1wT�@�����f��OH��+y�Xr��'�
�n�`�
�C���v���	.�)zl�+���������;k@����f<�����Y�����Um7��}u�i���S�
�x�[9�4E��.9��?�^�f���q��������l���/�qW3�g�n��z�y��AC���(O7�!���6�n8o]�}l�o��ku�6�G:lv�Q�#!A �~"K���fDJ��d�C��f��"U���[$��%��(+&����2I�<����:K��V="`���v���ee7����jD&ebh0���G�f��Zx��X�N���Q&�������������o�����r��j��T����<�Z�v*���"�T�q��au���W7����'�����U�D{eVt��M���T�<t����T_Ie}�������B���cS��ji��bSJC���w �H��������*���DJ��^�V`�X-P)W�hz�*m��K?���;	�7n�����h�p���=\(u��iks�����5������i�4l�g�=x�����*-/$����$��S��D�@��V*ei9S&;�W  �-H����Nq�-9�Q_%�\���j(���R\
�����gja,?��	��f�� ��1��o��?�$;k�����7'j�u2�'$�Q(F)�i�a��u�f�O�������O.�8����$N�c,�A��B69)�������!�����'s�*���G����������\MN���/���.�~5k�V5k�VI�[�0m�]-{8�r��$�<�R�1�&Q�+��h67s��v�]�Q�t#�].cb���|�cb���2�)h���:0����"�}�cr|l6�=@�9�/����
,i%\�
t?'��k.1q)�^v�.���]':<NKS�Q�L������b���{7��*�f	Ng���7g��d�_��~�:���c�?��|��	��$������_s���$\���)OdP���7��K}����Sc�
f�p�F�Y�x�����7/-�����E�Ul����)a��q��`(�����{����G-�\����E�����&M"u%��l�j��_�z�P1#r����P�)�@w�T�a�@I�F��

f�>�m<���L�A�?f0
�C=P���
<X�2���� ,{�Z 6����O�����~��RF�Q?��.o�M�_���j���]����o�H�� B��W��!��7%"����R����������:]|i���7���d~�o��Hj�����4�q��^H�8������u��0ruz_���z]��
��C��+�mY
t�	F:\�)���xF��������[(%��j�`�Q�@T��	��#Ha1y9G_����PF/`'�����s��>�QQe��BI8���M�!����x�����*\���U�*U�����'R�%]W!$�u�#�������B�e#
W&��z��Ha����l�_~�������
�B�K
Y������0K��~�Y�E)F]W�]�V���%�"/C2�-��&'�����9�����hR����]T�L���&U�^v��/}�B_a��Z���t�{����������7c��*�:oA[l��RI��8eJ�T��
`�<��HBA�Uo3��hf�F���[e����y�P���IR�^�`Ry��g
,�[R��&]<�47��p^yS,�1w�%p>�
;6��.����o����d�p��6 ��c���w
UI,0)�Cf�@"X0%��x
6�%�m�*��B�e|�*i*�Z��C��6�������{T���,x��o��r�B����XY(L�~CH�s$A$��9),\y9����k��_
SjP�)�^��T�k��s8J��Fc�?n�O
2�g:����4A����/�w�u>�����=����^&o$?�~#�C�,���=v@���_�R�+N�_��m��T���Y���g���.����?;�{���7/�|�{}v����D'.�2�f���z�lF������W�/T����W�s�������e���_�.�����������(��Cw~�8��oW�kF����I��/Q�����Ly���m���(k3�j�%�VJB�p������N����1�YC�����"��i�s�!�3
�C��8UJ�~�'U��$B��|�&�����S@$��TIj*�kTOC*�.��J��88�
F�(�kF���(0!����ky��W�����&��9���C�IdE2�ral�2�G������o"A�����,���h��_.�����Gg�&�.<f����?Hm��RN�C��V�B�AFQV��3X��L�����T
�q�����*Y@����#_^.�KB8����0'�0�
>�n6��4�AN�pMu���R���p���@��1������Sk��Q�|��9�G��U�<ao$�	E��>{�
�y�[�|���j��=?;}.�Y�X3!���!$�����
���V�:�L�Fy:���W�*v�s�G���a!4��Z���j��&[Y��{��[�>��mK,T�Z���@�v�)�y��*�P�v��~��xMf���f7v?�����{����`+xRn�&��i��n�*��������^F����Ed�C�����7�6���]A�b]
i�)>BWud#g}<�F��U�W�7���T�l�!�1�.��C9�w�8l��Al�G�?,��zH����k�9�|{u�<�-?5J��
�O���v�D������x��g�Fx~(.�b��A^��Sg�!������!�eY��>-�Z���"���"+���������������}v�8�����{��������	�}~����a{���f��nO�v>��� �of��@qD7�d����*2�A��q����PF�~��D�������2.�)����n�>�!u�Or��b"��6����I�_�� l��g`��Bh�-39�sAj�����ht,����ppr2(�uV'E�g5���'�zW���k��k5�f�<+e7+WS�'��������Z���mv���c7w���m����dR)VE��U�(�T��qnn��|����#���c��^|������w@HK�d�S����y���F#nu�$��E��&��������~l!!T���
���4_~H�Y_>��,��i4
�\����Orb%������1�7J7��:�h,1
?����8Ex���)w8���t.9�����83���|���IN���7��.q��������tK,KE_�Z��j;)-�w���W�����g����$��2CG5���@C�W�������cn����>�v�$�`����=�x��7\O��G�G��w����n^�>�"{�Ct��xj��'�1(u�����G�n���w����<����tR��;�~�%�D�x;���h���W�G���q"�K���g<��FE��9��-s��_"����yl��oS{��?'��yu�������K:S�������������a�i4�q�o����:(�7j�\k���\�*�ax���m�V�&~����D��10�;c������W�S�p����+����+#f����]��$�}�L.A��PY����q�)m�>�_��������q[�����h��Jte��DcLy��G-��0�E�L4�g����x
������\�������D�/X��5�b'_�e�1�I@���5��A��{��{��#���� �k���f1�=�C=!��A#r��O�������~����Ywp����>x����������A@����4���c��P�x��q8d�����NI��>�x7�������wi��-�n�\i�z�V����U[:D�*���qv�����]��j?�U����
�p�-����Pws[��Z��l�lj�����1Z�F����N\L�p��nQm�+���}��~�w�����b�2M�O�i��K��
$t�c�L���`�<[�yP�SC�QP�����
��0{�l�v)P�^��'����e�L�J�q�8it+�?��?r����WI��)�������@6V�,�I,�+��#	����	=v�w�������|�j�:Qt����?5���n�O��9��T�B�I���j��O?���F^��7<>�&�Vw���V����%����~��m��I4����>Y���h6���H�f�y$NF��X���Z�������@j��RR�a�1�O�V�Kj�H\���,9��r#"�"����:m���������?���H�������D��a���t�A�jM@�I�!Z�2U*��q�'(��l�I���E}F)@�������29:*����J���4�<�d���!�i�I��6�q�J�h
��a<���j 	���[?t_�}/����:������������3vr�I-t%�������fp��a]tZ�������_����~���>�U�����W��k����$(��a���O�b	�������	�mQi�Dh��D��bI�)�wr-�_\���/��3)=_�W	�%Zb>h���r�%�>�a���Q�B��nw�=����}���c��8��������Y�G���[�7yP�������Y�yRF���Sug�f(�>b������
']bG>�,�$���U���*-��-
<��������a'��!ce+�$�o���������+��e;E l�|H`M������
�-L�L2�9�����'Y�9�e��[A2wP���� F����Kps���L�<���0�����m&%���
I���R����o$X�Yh������6{�Jl-��\���sw1]��U�t�b�������Rcx	���L�hc��������G��Tl2�������N�O`G��g��_����H�^�/���**e?���@A�r^�![�y���+�v�,.[;�Re��fr�R���M� ��8���.x��Z��L����\m7zX��q�+���k8����
�D3����J�����V8r�.��
L�B�r������hD��V4,i�>6A��@�h�
 �j�r�����%6���	���hXC<���NI������4��p�o�^3�ttv%JF?����h�5���g���/��~A����[��wv���F��$=����~��)�u����;������A��3��ar�����3�����Ig����I�t�v�y4�Ew$����RV�����+��(G�o5Q�wP�5A(Z��-QQ��a!H5+Y@����'I�:��_�^8�����N��
\��� 9�\G7�i�O��w��Xy��#q5_L��cz�/8�5�����
���{��}x1M)dU~Ct���� ���>�?x��7�5l=��S�i|]�s2{W�vtJ���lr�$�#��<��T���������h���C*
��\�<{��������4�����lj�
n7%���l������4���v��{_1��������W����l
#g|��"��*��*F^�;5-T�;�`�K����N�q�A6A����AF����_�|�0<�����0�.y���v:v���t�xXxO�a0�
��@M��X���T����%�M�wg�!������h�X������������*�����R�}��%,b����P��L�R^�6=���������CIF�n�@��0��b��9�1J"'�{H�S{�b�mt�1#yL�[����u(8�%������t�P��f�l����EW��}�!C��.)"s��X�[{��7���HyRar�+�{�k��Ra
�=���8�3`���m�}���Q�V� ������*�v���p���Y���8�w�xp<�F�r��&
�����7�wt��{�P��������JW��_F>�-�~�������?z
(�'�%��'����@�.�#���{J��%}�$O�����K�z;���-����������*���M�������7�(E�d��:�u�����Q)_-	6r��M8NUf@�8�F��Q=��gZ8��m\6C|=��e���>�
$�Y����`�3uL���n�!���]���}��KT.�<Z����mV������A�Z�5�!���������:���AT����:y���I�}����k��� �O����|����m�>m���k��������J&���SV��c��B��i�tKJ!�A��h��F�~+�������-:�B}&��8c��wP��i������fFPJ.������Y��AU�KW��]��z�b"�N�&0#T��"  ��5���(�D�WK��v���AwY�
�r�%�q�����O�
s��=*���$B�xA�����0�����������w���@`���W/<X���d�<r����3p�<���Q��k�I�w����S���w�����5��1rt<��f�%��6��{,�WnA����3y��m���.b)�|R�BC�����",�u~u�T�~c� 56s5�M�3�+�S��`����Z��B5��>��V<]��������Ym��N�����,�X�0���aP���=�@UMk��N]������j��e(��9OPN���3�e8/�#�g6�"�����������oa��MH�_�rQ��2��o�34w�
��U"���3�C����*	r���f4��q"C�,�� �)��a7�|���:�A�OX��,�1���R�j
�s�1�d�a���DL����G6��3������v�ea�����`�������j��S��KY.��>r`�Wu� ��1�,X���X|(8��R������8��8&���\W���+���#����P��#�a�%���8S*j�b�Z���N����JD	l�����duz����<��*\���:1��3�5����!��Q������R�._3d�trG�b�S	�t��|�91���Qx��T�J��E��;H�T�j�����qWH�������A��\G���j�+4-%!+t�����s����
)�@'�w���^����B�V�;!5�ae �na�����6hI:�o

W�]�q@ Gw����H��j��� ����D�R����n���O:���� ��T$H�mQ%��-2.S�(���y���t<k\�����
��BJ�)������9�c+_�2B���1�@�K������ Z�N�`�*;����x���x��H���t��+-�o)1�$�Py��O9m[2�?���
�rS����[�
$w��f��8��"���,�A,��Yi�{{��R�@%TBi:��d�A)��#��������c��g��c;����z�*��H��.Q p�Z_%7�	X���N�O)����L������r�7*NN��l�YM 	�c0h\E�U�/���W�s��j+�������b�7�e����1���TfJ�P�'D���/T�����Xe��N/z���������q�ua��	���AM^������wU��SX�`q]������9E���&0���S���.�k��9,��bx#o{�F	�.��V�`��+��B��>����\�Xb3�r��Fi�R����Y#��/G3�.p����2
��u��j�\����s4�
N�
����f}*_�*L���^w����F�i�X�<������Nn���������������!;��I���i����+��Z���z��]:�U8.�M`��o��_z!��
����� ����
4���� ���\��2wq
�`M��
�	{*p������x�A*��x��*�,z�P��V'k�"��l&���,\��vl�������P��a���Slzs]u�C��f�n��,p�,py@�0�<��k>p8�����r����w��1	g��������i/a<���f�ba|��SN���h�����v����Bq~[>�n9������������K���8{��XJ��
,���8�A2��!L���<�{P��o��9�������e�Xh��j�'r8������n|�;��
.����c+�xE�rU������\<Y�e�EO�f��b�������'�HFtD~�[z����o�pg���;J�S�A;c�����f!��X]��G�]FB"]/��J��f���@Q^fO�V��/��/�P�.�@1���`����`P� �4�@�6r�������������<z�����������[��<�m�q`�r����J��593�
��~����5cv���s�|"���Hj�������x>(��o�t��r���m���E�?m|U]�I�����I9��J�B}��e(�?��4��I8}m3	��s�A��z;��%����aY����������tR
��n-�O�tj#����P��I�wj��
��C����$8����T{�.'o��O�/v�;-q`������U5���G����p\��8������u� <��O�^h�P���ZF�Hy�2g�<����(c
v����<��*��*�zM
��nkp%��G6���������7K9+����)��C�� ���������M�M�������U�4�����fq���Y�_l��Z�y���w��n/��T%e8��g��Y��<�EY�1_�i�o��������z^M����e�my���mj��f����e�f_��m�R���������,
�������)���,l�t;�A\n&%�T�D�_���^^C�&�	�gLn~B�p�U�����d]Tc ��#�V}�T:�X���2��B��{�r��'����!J����u��7vx��W �:�v9�+��O��V/�Q��*;�����-�d gZ��O��@��%�������{��!c���d�7�~qu<[���{�XC@�5IX�D����q=W�>$ _��4_�j��8~&3SaN�����������Pn�+�I
.�i��i�b�*� b���Snr�':aS�?a�e��c����0� 6�<�J�} ��@����
7���qEv��O~���V��M7	�;R�y������������|��'�s 
HTG����\q���/��P�8Nh30�)�ZJR��2�}�aa09�<@r�)���8g�����.��s)4��C<d�$/W��G�7_^S��V�����0�Lf ��a�J-�
p�����J�S_���r4���
���.0�"<���Wd)V�V+F��0f�wP$�-�P����������	5Y��J���
��6d�)����s1��W+��8�-hjhQW�/$@W������t.;��n��<wI��8�.��7���]ky�I�����o�G!��v�{��������{����M��YV�^�o�{�����w1�~�����z���V������}D�M����G���y��5��qe9���-oW�bq���E��,r���2��~\
9xS��i�[% ����e������nt���&�?=^BH�X���+Il�.t� ���:��K����B���'r��2#�<D��?�8����#g������)w��l����U%(]��G��jO���g�TC�?�<jq�T
e���L�"cXL,%!��$W��������*/�f�I��/|��J��FQ�������NAe�L�$��&d��b���ub������@�0T�hT?�����ej5}�a��Qb�VP�~������L6o�U5B����Q��T5���k��K��~�;*�l&S�����x�OK8I�]��	Q�Ut�q��_�$a�r6��
������k����oX�qC�F�Gz��i��=yW��������00*A$��'�f�/��J���������@��oG����h$��5Ly����@#@����� �w����c�9B}.��?��J�{���ET�G
���wR�S)����27!��e��v�u��zG���j����d�*YC/��|p���/�r�+�zE�-��������,-��������Xf��\c��H���8w���n�	�b9_������9p��MW���-���?E��(��KH�70!�.B�+������S+�E^��V������9~0�3��u��a3:<���F��I�i��q�7�a+�G�N���>|����W�)���
EE�Vk��#'yM���T.$�'&R(��,1�h�������j���������G����%����!$�{9KWI<$@��VXB�@t��+��A�Xy3����5��A:�z����z�r����u5�|9���Z��|7j=�Jx��!��)�*���|�����Pg6�L��vW�kl��0m,��e����r�An��V�Z"S��d��������e���Nv,��s��n�58jw�N���.Y�&�Y��v���:����)�UPsZP�E��7�M@�=��8P�b���w��L�����1/�<p������_n�W���1�XY���H/W�!�@����i��
R��Y�8���P:��^��r2��/�dI���������+�]����
�����0�|�L�� =�#�?�=6���P����7
��1
^�aSJL��t!ab�tf����y6dQ��&���6Y��	�v��X:/����"�j6=���a����=�j��Xn������*�8��B����|<u6�q����s���g�.���TO+�����_e��/h�Z=C3�0P�9g|y���]�:iC<Q��e�L4�|�$\������/H���c��������U�n1��%��M��QR�R��_�����u���Q������se8WFg>*��	�7���W`C��4h�hx�>�6�a��h��,i�NN�H!��������&z���J�c�%�K�9�C*Y��2!���Hk�b���g�P��f���S�C>'D
*�2DV)��b>j�����Yz3�5�7�@[,��5:*��lkE�"��e����.p��4�@��wbq4��$N�5g������#��Q4�����7�5��&��k6O�2��qUdo*��z�rAi�6�F�����]A�������Tr��R�i�9%�����.6�+���[��+�N��~�S/Tf�g��X;�x��[��$�p�e�Sl4r�f;��#>�	7�wdXE��I��ht||w��V+��_��eYpRS��r���y��mb������L����4�C�??!7%$���g7[��+����vnd��f�p����4������P�P�����n���x�<�C��9~P�?Eq�����N���99�4��q2l�FG�v��9&����/�I]�I����G��G�f^���*��� ��k�C��B������8!���$����@XE��X*��5���R2�=I�Y��a�������O��d�<m����B��rZ�K9c�#X�Vwu92���e�Rk8w�0 |�:�6s���SYaQ��'�S�J�E;|�b�\�;��\�;�y��v�K���ER��������6(��?��u^����a�[~��3Z'�v�z��I������Kb�����v��L��V����s1ZO��<�r_���(�wMR�>9������� e1V�?���7���*y��'7��~\�a.���Fyv!�������C���G��������8����-������!$�~
�d�%�/kR<=�xy����������o��q-�Io!�y����g^��o����������[����Wg�.������`2���n�.��x�[V/�����d���J1H������OM���G7�=�6��?/��������QK]�u��}���M������=
Qm���t
�79��>;G',���|�^T��y
���
N��n��Q�.�)����:�_�4����`^8�n�x��y����3=�'q������o���\E�QB�h;4�10r�4�k��|D�nM�a4�nM���4
�itkh�M� ��WM�6N�����;��������"��j�@�^R���y����Q}����{��5 ����V#���F��j7�@:���_�y#�����%d���x"����H~��F�-4j7����wg�g8��wu���x{�\~'�P����K
����g�r}ov����J|������
��M��V�������%h�!�q�{;�t�t�ju��O����#�5h���O�$@�-��v���m�O�q>��z�% ������1��}�������]�	�/�\����?���j��"�#R���Q�	���c	�����
�d�bU.�}1[O&@�`���4��2�|u���w��?0-"�P,���z-h�D�����S�h7J�%a�BPt����@�����<��NeN>qq�EG����Xby�9J�s<�Ex�~�����C���4:�ol�����OW�Ir ��M����BrS�e)w�;wp�G�
KP{��D�:<����v��`��wK����r����<�k��Y=�r�!�@�"��������V^\u�\]5x�<� ?D���-���i���b����U���Z�-f�����+)0����@\'�[�N;{c@+�l��V�����c��\�����`�f<��f5m�f.m��=c�g����%�A{@�.6�V�u����!T�%�����`o��`{��v@	������
��>,7�>J�mT�<z�L.��N+�����QI����5�oBM����E�o^Q��Py3�j���Y}�J��9��C����������R���� �A�eJ=�l4��s8<>B���0�����/\z0�j���i��Z����!����H��.���b?Y}J���I�����a�Z����g����8���M�\'��P�,0w�A�O��5s���?���F^;�K�L/�``?Rn��'*����x< ~a�H�,&���
*/�<?�1f��bH<�8�u�Ev�(�]�n��v�)b����Y� ��
����Mlf��,�����|�����w��v���z}�,�����.��E���v�6��6?�6Q��������2?���RW�u�p��A���X���f7_u[���JH�?L�,����l��0���Z�!t�>�U��N���B�9��@O�.���O$�$����k�����E��_%�E/;������c�j7{[�gVzK;����[i5j�5�w|xc���������3���t@HG�&?�e��=5�_��M� ��*����Dh	:6,5t�%����kwSW�E+�,ZT�;P����8���G?�i���$%ugRx�v����Oqn�
��94��)A�&�t�&9���0)A�s`R�����R���^���'���;��]��qr2�������	5ju�I��!Z]�!���iQ������M��z�m�C���M����>;�����:]�
|�5��������"�:��8�>�1#-�_~�]���������u��%��/���T���`�J:��-�G��V�)�+
�i��G�Ep�^\������x9�^�W���r1��D�%��D�AU������]�
$��Q�=��%K;�����.�s���D����7m�����K�t����sE���e�n���{G���������LS�e
�I{�N��@=
�a��B�������o�u�P6H�^J���rNkYx;IgGb�@" �+3�r����:����vy��?'@��H5�lug|d����!E�&�t��[t������;)�������"�����M��n{�p��}��u]j�u�w�I�����~����:t.�g���o���@;��Z��W���\y)�.�"~�V�rP�Rh+[��/[	�����.�.��}Z��8/�gu�<	��G���WN��
�����5��VA�b&�F>>����`6290��0�����a>"�� L�[���fEr"J~oD�f_|��f^��1(����ukW-e���aC��V�@:���4���t����e���N�|BB?;\{f��7�l���w����t{�m��r�^pAy��+����x{�m��r�^py��+��M��O9)~�T���`��@M���j�QF�F��}�������AEV��me����}���rE��V��N��G�����+�����|j�>
��y}�*��V�#G1`�1��(	�Z�AQ+�<P�|����r=K�����]����>��#p������)b��~��SBr����l~E��(�0j]
��U��Y�J����pG�������#��� �N��\��J���hQZ��Z^e�x
n���W:m�QF+�o��n���w�o8��(��v�\���~O�$��&��;�����3S�����*_���
�R��j��O?�i�z��g��,c��f7w�L�k�{;tw{kD�vW5d�T�q9��X���������w�Ar#i�	$w�7���!J
	��=g ��`{(Un�r�.�jW�����+�����}�}�\o���}2�r�{��o��f��	���;�����~���������wcL���]��[<�w���[�f/�b������A����bzlw�R�)t�.���<dO���l�]|�s��>I�����52���=���xm(x��\��VDWIO_^.�K	�
��t=E�Q������'?����Zto�^���y	�*�����������

%�,�������[O�6�������p��k7W�F!���t��&���A�"~.z�OU���/z+G>dnW�F�����(��QpC��vy�<2n�d��	�f
'��8}/S���-��$q���8�XI�����`��
���`"My0#u�iC�T�~������������
�n���`2�=������u ��M)T.�0g����B&��D�m�������(�H�tn�sn���|&���(s�?������������_~�����.�Rr����c:���v�F��M;�5"�g��r��.~��+�������8������
QB�@��R�ya�&[	g�k��|�������R�9�I����)2=E�sr@��S�9j=_����1=ur����Z���s�������5=u���h7��������
��H��e�wp6�l�l� ��`O�t�T���D�V��S,� ��/,���I��,{P{�h�������?m�k�$�goO_��vV%������[�[I�%x�\�`���E�t�6"n���p)��J���P����b��h��(Zu	����.��v^yK�����h�%���\�B��0�4�����W�����~�(�(B6E�g^��n����E0(:
�[����%0�;��$��M�%����W0o.xk��~�aT',N�	��[�PN���C=m���}2����M�q�iPV�]3''���G�~���b����**�U����~e�X�%���[�
�8W�0�D��J��`O��.q%�������
'Z��N���_�C�j�?#���iF����H�3TZ�'l����������5������ B��@M<����xm
]|Z�)�����6��A��@)�ZT��p�)H�v��-f`NX��L�PX�����}�pn
�[����,VR����K���%A��
�r,d�]��v������_�����(s:���.�����O�5'K�G��mf���?��������l�(�`j:�`�Kx������*M�[�Y��}W���ClF�I��)���~������3L�Oy3FN�� ���(����Hq����(�@
u���n;3��Z�����;���=J�����@��T����E��zv@C9�.b�X;*�6g7C��#xc$)��~nI�B�f��&$�������;>�'�g�Tg�?|f� �;!�a�y
����v�*5���]9��������*4�����n�=�f��������-xw���^�JD:�������m��'U����>�x����qH���0�������4�����9�� ���m�@������a�6�XZ����;��-����l�����bK��~�/���F��@���i|>�(��.�{���6��=�H?��57�������d��������������5[��uL�lT����=�����4
j�:��u
���~���a6[��P���NcwqH��OI;_+�;��6�����1��Oy�&���f���wxS�C���o��*Q���aU�����n�N3�N��-�t����^���xho����~���4#�����~����������7�����j��}�7s����������E_q3&�_��4	�����R�������:���R+�\��)%G@��F�D���|��V�����.��
�l�u�E,��m�����2}�d�ES*������.�z��^tF����4��c��u�����g���S���?�5�n��y�XEt������@'���ZGZG�md��	�Bh��u�pw��%�K���M���s��2����V��}�n{h�v�l���G�v���h�3���u��f�m+�x��h�����z+�h��N;_�����l�6��TL�o�i	����%����?+\�K��9�^���b�Qe�#U��|������-7K]��_y�����O��uzL�g;����|��(�3E>�A/j	��R�@~�ds��o���B���tglu]T���l���y�=�������n�6(����h�c�3�ap��cd.�m�M��z�����`���D�'Rp���9��;JG3N�����a*�|��|?�iz��=��b�6����=r�m�B~���g���2��0�C1?V��*�b��m6�e=�:A�����|�d�w����VV;?(�
b4�~cn���Hy���1o�������������=���]����9�[3`�	W�	�~y�/�u):�	�`�n��E��� �
�b��f(��l���0/��|y?6E,��L�[`�x��5�v�p���Y�����_m�X��t7�s���� z�2Z�V�����*�V�x�i��L0B
���|5zW�����[1�;/g��	�����G���ws�O��5n����pz�.D���W�!s�����x������B�2���b����%��pn�b��4wYz��e��EY�3D��B�\�\)D��K���U�vA���Lv�f�T����4�D��\�
x��#S���2��K;m���2���!S���4�%2�gko�Z���*�_2���)�~���~u��g#2�)p$����m-Q(�n3��u(���[/)����7��
X��r�-��w[��M���
��-�E%46%�`�|l6���maa�w����=/-���[/-���	��m
H?�� m���H��=[&�����y	\w;P�&*J�����\�Q(�r�l79P��qf�y�%�|�u���P���@�v?�j����8�?�D�_���q~�6<��+C����S)W��D��y%4���"� �m��1
W��*#@U�.r�iL��4=�~T������1
�����Q�"���t�
N��i���e������"���QQx��=���(�Q�^KoT��F��[LoP�B������Z1���0+���,�>��K.�����\�_m�z?7
g��_+���	��4��Q��G�J��`���5�
~S�P4�+�6��~�o��8�Z=�+�@=l�$l���F�_0��Aws��9r���9r��m�������55n�����v.����0��s�v���\�
�.)���7{����)�??{u|�g��)Q�]�/T[�h�	���b��5�����_B�6�gQ�}����v��a��x<�b4_��8]9��
����&w2L)�pu��W/�_TY�nBZs���N-���>����~��w�~���s��y��~[�_�G��$�_������9���<���G
=8��j�\����!0��_������$d��D7��(�a��7,
mX���6,
mX���6,��
�%
����3�pc=��*X3ow'��xZ�7����Y~��vdw�	�z<us����� ����1��"J������������|1�W�+p��av��"�3�x�Wo�N����i�;-�a�����rU�c�t��?�p�BA�Sd��o��~���{w �p���6�+�J~�u��i0/6��q�����	��.j��M�2^d��"������r�b<D���7�$M�(@��eK�J��jx��=u���;m}�z��y�yIj�{��E���&��
�EMy0���z'$(Q��s#�Kz��"A��owE����]mg�N�9a��7=�<8�l����v�=3�j3�:�����<���s��){���p'G���Vn�-��B�
Ow����;2��o8���s�vn�m���[0
�;��0����1��oF����!�ts���_�7������y,s�>4��o��i�����������/�#:�	vr��8��c��?�Nx�9��In�'��}�����a<i5��nY�N�7��`7�'��s�j�_�w+�R���d���@�o���2��+�'���r�1��d�X]�n�d�]����c��#�HxVf+���w���v�\�������I����n� �+4���������zQV���s��"���I�a�Gv��d�"a���
��HX&�����v���]��2�m-PP�O.>�n��B���\W����O)|f�>z�L���$�n�g�$���/A���;�M�y����v���)����v�
�D���F��.�]`vhL����sz��!#����`
�+Zh�D5��GC�����q�vQw90t'�aD:9Nl:�Y�/�wgg�t��(��<�B��Q��G!��nG!�=
��G!�=
�=�(|�P�|��+�=
�.G!�=
y���:
|����w�����������qO�@
sK0�g�k9����FajO�J	y��1/��k�������f�\k�����������l�� 2s;�1��7��39�]�)������(��,F
���p,�i���#�h���m'��W���-�t[�t�Vw��;]d���w:����v��E�ln��d+��$�9�g�f�E������$�����ig5���M���K�6Q�q^����tpAO�g�i���2�e}O�t=������S��w�Uwm��cWl�
!�)���?��]�7cyx���I��A��\����S��])���B>�X������7��B��z
��>��� ��
��do�v0���z����-�[X`�������|l�������`��������wC}���%��[�5<��68��!�g����,�x�T�uc�w������zwp��b���(������)c�	f&M�����D��K1�6�7�&=�������`���D��F(nL�t(�0Q��m�����B.�L��9���K$���+��I)�.����j;X����,�dU��dx9b'����#6���y����k}S=v���[���	O�`�Ia�Q;XxbO������Y'n�m���2f�4��JL�"����?��D�M�������U��n���k_0�7h�_����|hFA�Fy0�vk��![0�7����w[u������*�����1���v��K`���*=��j9"��6F���M��X��k7����Sf�?����%�d bK�-g��O�4��^>����:�8;?}U�����6����w��0���g��h�?&I��7��y���;G����>YJo+l��F'k"MyjFBI��������X��-��+�����|���=����\�w`���������5�=��Rl�Q��Pi��Fgy�#�h��k!���n��oc�oj����	��7r,���t�������2��Bl�M�+z����q#��?�n�)�p(�N����Z��M��o3Jk�����(��]b��e\�i74R�����I!�lL�|�r���]��[E���r�fa�6Js�q��A�vY��a�9nw4�j�0#������������x#�"��k	�E���dN�4Gb��>�M��H��/�Q�N��H�E_�c�\���	A�������^G�����JUh7�d�e��l���?F'+Ut��YiB����v���mR
��a�%�MS�h�"���|=�fq��~�"��U f7�j�H2��~���w����J8�l�j��C�6F�r.��� V7?J�����Aj����0�4����m���:Gz���h�9���C%���Y8�3�#��=v��l�K�2���Lt['/�s��t����C8\p�]�R��$�+K+?��������g�~k�x��mhoC���a������}\�����������������vp;%�� u@'���!���v��K'�c���M+^:����r���;��a���n�]��P\i��J�/�q2cJ�d��.��e���x4J����	.,���m� ���k"�q.����F��w�f��0��(�&��LNDS�qA��Ps]A��\nGTn�vYD<���Sod�w����e��������i�vB�MJ��9jo�)�m���\����������o_;v������amj�6�������k��c�����[����H~\���\�Is$�Y}J��u�}H����p:>:_Sd�
��M�9j�	�����3x9&?�w�����x��R���6q�Z8X^�}dw�s7�h��=@N���|-��rT��������q1�K��
$q����N�����MMZ�i���J5��L`���*�����,7k����p�;y�+<X0/t��5�I���\�Z�tZ�*�;_�����r�,]B�j�~n\�l�Ym�gC�|��g�%���}�]��������S���������9���	??�>;�Y����baI&W�(��;��4�l�v���n���@������j4�lX��uj�8I`���K	��'����<��b���Q=�m��=��X����m�a�f<�
B��[������9��Qc�0U�t�UD���O��N�)���_��O��Y��J�d`b�0X8t��Hw;S�Lo��:��[�*Yvc�|W��];h 6��_��4�(Y�L�[&!\����r,��a�<���er	7J�'�d�KW�p=ID�D��	I2&I�h�$Q�p�:�t*rK�U���/7�_�"Z�v�H���c!�'��b���XD��~:X��GE���J�u*���/�"�|$�0T#�����O�I2X�>��O�X���x����L���r��u����d6������W�x1_��������<�i~��p<�����0^����
���O���L����#��d������|y����?S�\�O��������
�K��j�x �|Lb���-���I;���o�?��F���D�X�����S|���$�M���\�u*�D��������a5.>_^>]G���0i7��?<n8]~'9g�o�'�����'K����b@��-�P�3�(!�[-����R=LW�J���1������nEh����Y�I�����PS �`�~����I�&���0��p&/�\��@5�M	�z�{"�������&>�p��ZW}�L���Sw�������3J�������
� 
�)|0l�M!��g���B�j.�y
�x���R������9.b�R�!��}��P����~�e��\��Jl�|�s|�������@��-�����.��E���v�6����S2�a�R� ���}������m�H����MT�6��������:�\kS~;�hD�l/:&V���Qe�(��g�$u��W�N��>]�'��d�%��J��F��-a�
Ti*	���J8i�ps��-]��#��T����^���/�|��.��[�6f�`N�t��L2J�O���($��?����<��4����4���W�:$;u9n�q
]H~>�|+�M���X�XG]���@�GY~�ff	Hx��o�����m���������@�YFid�w�����
<���=M���)!'F���,(t���S��k�W����x��z��G�}wSog�U����<]���7����5��\������/s�HU�'Sr���-�3iEk�83isd5��?��?�h�������//��l���z���V��K�O~�7O�0u��7o/�����^�ZZ��Xw��_wl'=XI�����2���v�1�_�������X�
���������S]�=��E�A��@1���"������y<I�AR��r�Zm���!����S|!2/,:��=�����*[��~�aTk��ppt���;	Ez����B%�}mP��7N")DwU{'���GZU
��>���"����Wm��������U��a`p��-Q[���%
cK���*� ����6�[B�e�1���:� 1���x�m��9:n(��4�{s�)�f�%<�fL���P�9�m]�*|iI�O�CMuh�gg��������ZV�_s[��,����Z	���������km�k�(��v���������)�]�*�����ZM�y�����������J�2I�%�F��U��V>Q��y�������h�Z�.�e�[O#���I�'���oU��9�tG��/:��� ���������9���
�{b�,/�A.u\����
�R���F���3��B���?�����n�^Ce��,pv{���)P�M%6^h��<���*9��:�iX�6
hI@=�%u��[��uk�u?JeR�Z�iX��U���'u��o���os�Y�m�[ei�d���(�U��[��*�c����[���VE�lUH�5�m������N5W�����Yd}��)�����r����S����	(��`�F�e�!:	9�o��rr���n��3�������~���������~M���YH'��~�vXN�������V[p������mY��>����,�Z���V[����Tu���%�?������0o'��xZ������O3	�	'�En�#�����1
��r������"	;����vf{*8�$K��b�W����H�~Rt[^,���x�X�u�
C�D*&�P1�*t�i����*�
���o����;�}39o�N����������s�N�J!�n��Lf^=�;u0U�����|�u6���q*���y���G�Q	�(@X>�$�����'4On%�r��'���L�2r�'�� {��B�NC`��&l�P�����5���L����1��4e���n�n\�]�]��~���IQ����s���������!G<���!
�`O��(�0D;�(�0��X|��aBa�a��(�0D��(�0�uWx"�0���:!��$�[�,�b �%�����Z���{��Uv�s��\{&M
Y�[�����L��B��&������\�����$�`�[������J����D�^EE����A��g��|W���v�Bf�G�������������~��p���BJ��Q�wP���p_Xy��P��_���������d�����o����|���>�w�u\�]i5�'�n���?~n�gq�.0Qlo�I����g���A<90x	��I�m3p����:�6���[��j�[�(:�F����Q��}�'���E�?�t/��S�^�W�4wa���F^@��Q�=8�6����������$��A�x4���$&��as ^Kr�>Y���h6���H�f�y$NF��X"��Z������"Is�\����k{9�/��>E��>k�L��LR��S=�K��=*�VV�L�l?|�T�fE�yq�sj��/����IQ�]��D D�"X���j�b�����@��.��V��$�����@�����'�
_e��F��������t�]��?�^��Kv4�v0�[Vs(��=88]�e�]T�������%��(���S}a��|~ {g���P�]�7�Lm1��l����q���Ef@��u���vq�F"�z�\A�p���������h�o�zMx�)���ZU�[(���ahh�t�R�Bn�����r�E�r�v��B�R,3�k���,������A����w��tY����1�kA������]O�V����~��
�v���ii��Z���@������rG]�l��4T8z�!7!^� �0�����Z-��]�nf7���V�n7�����r1����d'�3k2������U����ly���b:�^U��|�o�s��O�����Q&�Z����>���g�`x(���{����]U�
~x ���v��)��~5=����x1=L���=�ZP����O���3�����_�3�Y���������R��\�����*o�O�����=[Eq�Y����k����q\���!��,8���/�
���%l���� ��<�����|�6c�Bc�
�7�r���{?�n[T+��|�Z.7����v6��qo��x�eJ�oPfP��_�j3�g� 
��[��E��{�)n���+��<)�Ws&�>P���g���GV�'�b|3��r�X2������\�f2����:v;�d�n�+�u�0u|^��2�+�
f�,U��=TW���� ����,�����=G\4�� 
������*"��i��H*xV��r9K�'� b@������ro=�P�%�6��O�KK2������\.��~���a�IY�0�E�Am�FM�2
�
�R.����/�_q!d��em���uv6�	���@��A	�� 8L��V�i��8���9��{MK�-���_��f��D^
�]"/�������r=^-�w������~������m�/9�W�dUm��	��7Tza��,�7���\T�2P�0
0���3V�?��p�-�`	F�&����6���@$T���-�7 6q��h���a-<&�"����OD���d?x��l?x������C��"B�8�Q8�--����7�o��>z���l������1�_�gg������5X~�������_=������������}�ha�����%���s��z��N�'�k(��{�"5y
��?�Gu$�����T?��������7�fN�~�]re��������9l��e���w&�L���:����z�]�WHex�]Mz]���S})����h ��R���Os�
C;�B�t�!Xly��2�%p}R��E��z����H,��,P
�k����P��U�������6�CW`$��D�=�a'^��/�v=
�L��������
���A����zX�
TO�*�I���`	��3�4U�Z����T_Q ��� ?w���������.hO�����(7x��B���.Q��t�\W��EhgV$���"2mAF<�(�����62�����sH��k8��� �@�(���,!�����A����������~��	 0��?~E$4{"�H��3�&�����l�3?�-�� 1g���'@s��������.!m���QuS�Lon{\�/o�����k_���	o����@�\^���{H&4&�|D[��e
8�������P����Z�/nuh������G�z�`~.���������0��^��X7�A���+!�si�5��V��T7
T58��=l�k{E�E���ObeP���E�G4A����(��L�����k�@���!���
{��f=�'zOXA��rz�#�~�������*&�`fE��^�������N?�
�\��3�-���52;���c�+��-�4�W�c]-�[������_A��}����_)
����xB��P��p{���| ��dWV������Cuq~�f#
�
,���YI����

�@Y�oO�1�?�R����\en��~��7���'(4�I�|%t"��'�����	ajHw����y�5j/�ap��@����x9��d�B�2k/�)d6�h��P),��|S�QJ�O���!ty�����n�"�V�����']=�c~?����JGo��*��]�Ifd�h8tw~���sSz��Y-����^^���6��;$�=/-�b{����?AI#[�6���o`a�]5���B�!	U?���mj����/t�/����Yw,�W�&��0T((�������8U\gW�M�<7K�Lp��������O5_WMW�>�c�5,/�,����������~[-�z������Ou?|���h�A3���
���B:U�w���W���7��d=�.e����r�����N'�'����P�	K�)�:��8�A��v��1��A'�;[����
Tg#������dX�xQU/O���6[�Z���tC�~`;^d����CtL��%/CX���8#�Z���5��n����nW�
\ly��r��������5]�z���~��
��=�a���K��<�$j�(v�D����r�Bb��9�_������3�nHtr����~�p���U+�UE����'2^����Y����hc� �
��!�����0n�B�osu�2=���
��W��8�7�<��
�%\14`�
�������?9E���pY #����a�&���%�-�+P:�������0;\�.'���f���qqyF(���@$� ������7��9���h�x~�
Hd������7���� \���������{��2�`���D���=����K:���bv-R�7��B�aw@6�Z��+�#B6$0�s�lf����dT�)��M5F���h�[C#>��Mv����1v��~���\�R@���[���\^����_ R��i
@�^�nT�����+���5l�_
�
�z:�Bq������;���r��U��X�q��UT=��G���3����������:�(MY���zT�������Y�����=������v4^</�o�a�x<n������dqc�ai�Jm�V��=����7(�Q.���y�J���_��"��Wi���_�<��%�W����o���7�Yg0�g|�R���Hik��Nm��&��m�"v%�Om��nYr����W
�y�ZY�\�[�@r����{�O5,��Ze,�sd���(0�C�������o�r(	���������	�3cv'bE�:�Z0������J@��E��@���S���7�����������������*��'�jS;w db����k@e!����c�Ozc��DI����M�?���(�]�/^^�|5|5��m�������RF�S���_A��)����;�q+�S�~����t�gt��c��f#��!~���W�����u��1�n�#E'?>���3����L��#J`���+dQ��y��q���Y��7�r���,��z�`���X/W������H�p	�6����2��0�u�]�E#���
]q�WW�:���UhR��Z}��F��������"UD&�9��V\W`�o�0��w�}�6�E���v^�2���>�"G�RG��K�����y(�s��d���T�������1������� f�(�ka�X��>��"o���O���J����DC�������j~+�b��~����(P/w��z;��U6��0\�����~�}���� B�cS*�@��p�B����q��a�c�=�]HA��nP�q������S W��
���un���;(
!JH<7��W12�I�t�Bp���=����;�z�Y@�b����ob���hc>1����
������7Uy%A}��g[� (��}�!u�j�� {��-RD��T���ms�����bC�-�����%�fo����Dl���\}��nf���w
��Y/�����o`��Cy*<�2[�f�Y�J��i�#�2�z�oFN�u�RLK�E�J��S�lB:�kT�o@x�T�Tr"�7�����������)6-*~7��Gr�!���y������������F���YL�=��O�6�]l�����<���Q`ZJ-�X���u�^�r�pnhh�M��
��~v���k3.|���p��
tn���4/RK.�������y�-> ��8��/	���@��rg<�����G����9_.��b�d8���[��#����|��t����<���1G���5�o���w�y��y�c���X�*���*�+,mO����XU�
g�\�I4�R*��_�`�<:�14������J������r��R������!*�D>_��+T638�����c���|0c������w��
nN�L�Tx������%1X2ykO�g���$O�
>aJ&k���m��W�Q�p� ������W�S�����)?��O�9�.�x�@�Qp�j�����D"9Z������j�,���*������Z�>��I��3�5�����A��=,�r�r��LTZ���������Wj�!K����J��(�����a�c4���Rq�������p�z;����"V�)hr�	<�5����L�cS�A����?~[�~�VX���\@����P���R������}�����Y��7
�[i��@rW�#���� ����"}s��k�����F�����!�W9�Q�����"�0�?!+�?1�=���MO�HU]!'���f�c�#��f3D���T�#x������(���W[�^?�Ik��?��
qc����CEk������Q�O������LS��0������..\}hi6�4/����8�M�M�$���
����x�^��gQ��U����O�XkW 6��u{e��5�2Aj1�g�@����|/���Dn�E���~���n�ofD�	���n���0���}��::�s�\0� ��E�y������_<����t�!L�����Qq��(��E������?��`R��rt��<zy5�������(������`ptU_�:������_e?���8�nt�]1���(b�&��.~�w�Q0� �L������?|X�@����>9X�$u?gU}�U�UW/�����p�)��������*�������?�]��#��������S x��o
���f�	�mBc%v�b�����.���k�3��o>�����20�W�G�b�88�(������E����3P >��E�f�@�}U���I��
^�x�-)�[��4�?^4P�_@����B���$��������`>��
V�����\C
�������E����H�X0�#��������M22��q+��E�O"�����E��8���h�������|�Y����>�E�0�['�@��G��A������ T���1��V����#\���]j"���+�
���p�{�����3*�>�vR��o�$�S�<�����������v2�A�����������_�������g�(��a�1���]r7d1�P�p���M]�/�-����a�a�w
���+�4l �}��*��5�u]��p!�HX1�u�|	���E�a	^:�'U�p=�J)��&VC��4:�n �9/��^�E"��~�x��%���
hb.G��Go����
7����r��-��ALuW6��fF�
�Mz�KO��x.����w8B9
�&k�����]����Y
�3@A��n�s������5�����-j���L���p���w��L����
mzE9��e������������g�W��Y�L�KL�c&�X(W�P�Z�[�����gz����ux'�Y��$�������� ��6lXxuX�6������(!�F�,0o���8z���Y�]��v���Q��A}-��P�PAo����Hh[tsby(e��~���}�2��K��t�orMg�$���m��P7J��H%��6Kx�����9�JA��jV<��.�m�����1��2pw�UD��6&�6�2��.G�XG�H�������8�����%|��h�8&��$CM:�A3�@��G�D��h2����:���p�FS���b������
���JLL3�1������8���#����_�/���?�/��e>����g���1~���;f��z����{#�<�{v�~����A�@�%1HJ{-9�Z����a�mL�H�B����nW����G#�,�x��O��B���|�g�����E��|�!�2���
!�A����?�YF\��A$�1#�MC��CZ_���*��g��P14s�D���h|��a�L���M�@��d�L����vI��P�q�7��������U�dc�bz�Zqb�Y:9$��o����rg�M��G~L��E%��~l������s;l�~��qb�i&����Y�n���t-�X
b���`���aSZ������+�������>�p��D?NF�����K��_���K��)�wF��|��2LxfF�"=��
 �^o���2��6u2H����s��;(��7���{�������	�D����N����(��(@�/�����p����'Elb�������	2�,{��Z�+�&��� 7�	�w�C[s}[�+T��j]�$o�=o�$oH�a�u15�(�%����$<��L�Ei�.sbK�8�5��5�BE������h��h��hH+���`��a���Ixj����$��qhk�okp��v�WI��K��3!���PU����;�S���U�nf:�7#���e;�?�A'�����r��>�����-�����;��h��'h]e���,����s"�d��n?}����������z��B�O�L�#y��I��O�\#������:�����v�6�
�1�N5�g�#v� ��z=S���$j0W2`��!zf�O�N�l�����7�?T����� �a��0`w*m+Wl�PRn�GA��=�X�DTLX��QK�������=?O*Mwn���u�������Re����>m7���~�,���
<urS�CSh�;��������P������jQ5N.�W����
�D�6�0
�!j�K[r�=������B90t�C�X5��	����g<�Ec	b@���������-z��-�����^�&!�]}��Ec	b@���������-z��-���h��P6��2���[�c�gH3g1���v��dCA�>�h$�SH.�����m.���s���]Y�-F�����\���l�����m��X;��?R�tte�6].�g�i�Q��X�L���)Wc���������*tf?�7M�4�t|�Ic�b{��.������?L����`�g4P�+|�G�b�4����AT���Q1��^E,��G���?��^3��@|��y��
�"���@�v0�,�L�i������
`;����
���A�X�	l�� �f�&��^����.�hw���g����O��������	pY�	`���U�x�������]5�h�"M�9$��z~9��j�������!��;4j����q��_��N��,�������9����*x4����/���-�1\m�	6���~�s��tq���m���oGBs������HM��
�++�Y�r�%E����s����z��)G�����y�H{h%u�;���+����EM�d=���\!L(Z��,��mi�^*�������S�la���
F��'���$�F����d��>�@�U����E� ���V�N���������{	���C�,v'[��qy��x5P�WZ��jq2�
��E&����kpA28Z����Td>�.�mT�l6q� �Y
��`U��=\�7�A�A>Zo��|�"�_(���������f��/�7���E6F@���5���2��0��l�[�����y3a,�>����������M�jO'�`/���� �\.rp�)�4�m�@q�h����c���h��#D������_�{�����S�' �.;uC��F�A.v�
+�V=�FA�P=��p!M�?-��Nw8��n�6Z#�1��;v�e�&QA�aN�M�M��_�/)���-��[��;�y����U�nu,i�����g����<y�5}W������|)��yU�7h��L}��![x�F;.�8Y�Q��l�&�5��z��d�����p�6������[B�:�~�;?;�p��%x���a��4���f��q��O!v�@��}�'�E6����J�/���O4]b,�sX.�����J$�"e��f��o�2gS��*��s\�����Au�$�Z�����`!�����IEs?O��[)l���s�vC��$����M�����������
����-���og���@B&�c�8)J�����
�1����dg������'�����^�)�����|�`N��u�:���-%E�p��L�A�)�n_qYZ��I��1��d���4�^�;��-�_D�R�T<R����|�p��o�6����	n��B�lP`�����~��gu��9
���Y�/
�v!����_�i��
9�<���O�*"s�T&'��dl�)�Ms ���������������N�,���cd�^���r��8�X�3����
�9�m�/�!�b��R�;�c�arh�;���%�(b$�G����dZ[j�>`�7L�������$x$�<��L7Ms ��.�����``@Y�|e�q�9�\����]���G�W�,�l��VC��s��~*W�����o/��yq���g���,�~��V������?�j~|�����/g����av�F�=��������aY�G�ene�"PY�����>�]\fo��g��|������vV����?������E�*7�x]�f��w��F�^���0���O�����?(}����g��Y�o��|������O�u�Awx�LWU	���b�5�^��
Pcal�������"��n��+�	�����Y	o�Bz��O��'�z)�1����6�V�-W3Z��h����%�-W3��,�����--W0Z�`��-�0nL����u
��t���3�}F5�i��8�I�����J3"i����(�]��7��pa��bUd��So����)���jM�;�!�����U�3��F�\@@�1f�=��0!���Fn+�������������&��Cj��@��]2����pa�'7���4���M��/��?�>B[��6�
���85��%�/�]20�g�y��������~�v�G��X	������`�[�����b��l�tlY���1C�>|q�!vg
�P�4�I4�u�
�8"	��.y|�n6��BOp�
N@�}�
���2`������qK��������M,R���V~�p@����N�����}�����W��t����9������^]����8�=���k��~�W����)'��>Gx��gxGS� J0S1bcpr��}=���zE�uT�T��OM��
'�����Z������qdJ���.�W[�.�J
N~>��SZ�xU�s�$�����E�#�h1?F'�+�]�B��t7�[G�>�J���O+GC��v�Y��B�f�X���8���NN���<4��m�+�|:/
9]r~Qh����[��]����Z%N��@���vC���?���O3�,
LF[N�G�	'#b������j�����.�����8`�hE�'�����%����F�;X�h�WG�L���D�`(������';��Q��9��3�|c}�
�q�=z8��%��K����������|g�H���W��1��#t��p��% �?��������,	�;ogP����,!AM�	�Gus/��)o�A|V���9�M?��gOCs��~����/��S����p�qrD>P:��/��� ��p�18�>P���y�P����}��>�x
[����f�C�A���b!(��	
�
��MhhBC���PTEBC�%44��
4U�����f	
}$hhlu�>`9��v�������7u���M]��i�E�l�2�xM�o��4C���r�h���?�%��V$8��������<q����3^���:�K�$�Z���'�YF�[�(� _�������[B�P�����%�����x�F���tN�KBkp��q^/
:��	�}-
�[]ok�3�pb�
}���=�����k�Z�;�]8:����-8	G#W(���'�Y8�!qu��������[8�@r;Dw����'y���������A~�i
�)t�P��*��x�Pf�D�T1����
S�z-|BC��w����#4�Z�xv�����&L5�*:���]J�����8����Tw�!����hwW���pM����H���T��v)��V��.�� S���h{pR��]a����4a��0��RY����~�hP)K^�����&K^i����#K^=2��!K��@U1�EU�Js����NP��|s=�����8����_���������tc��Y��������mhW<fCI,��K�.��t
I���v	���\3{�4/�O���P���B���"(�WBK6g��(��$������v>wQ}He(e�Ltt���U�:���o�Qqqnih{����Sd������:�j����"��o!D��3�H�nj2���*b !8q}�����8�f�5�Y�r_�jEN�UV~\��f���������,g[���r���:��,�2+�j!si��B�3��k�7��2�E��h�H}�B���MMFL>p!�&'q!�;[�B�XS��X����I^�\CA�-dE3
�fb�p�BV��*�V��������[��f�V�K{=��Y�����y������bO�Z�<:���h#\��	�I\�����)�Tg1+,d���$/d"�����`���j�iM���U��d�F}�	����9��?�
\^a+o��������-��l���-��l[."[n��e$-#i�����<��)��:�m���-����m��y�U_nW�z=[.��l����l���l�`��k��M�k~ZU��S�����l��`Y��ET�Mu�k�����Y��7aA�%

�*��o+#)(j!+�w=�N_F�y���x�����:�����6���y�y�v��d���!o�:Zmc���t���g���;X.�"���'4��jft���d�=�����(�=�?�VG�����K}w�k�niJwfX�vjSIg6����0�fmS�.�h!���,��q3^[��T�y��+���NW@�W�W�}0���;��EZ[�Z�����EX���W�n�n���`{�
����2�+`�]5�u������E/h��)/�
����b���.�
�n���:������3��1|�O��0�� ��������.������
#�"��A�g8��e�5�D1���jc�I��2���`���=�;�
\�svQ�yu)3p
4��f@���f�����������������Mwn��&;b�Q�a�[��	L�nSc�0
�����*;_.o�x��8cul�o7�R=�J��E0g�p
����v��,�d�:��=��;�
��S��X�fZ}�Y*�%9��C��B<D&�:q�n����!���Z2�F/�
�2C&�1��'i��q������	R��'�S�F7��[5���u��o���d�r����M[�8qL0�:I�m�i��;�(d�C�x�����{��Q�0���� caQv�S�U�1�mT�N�Q�I��e��'�	F�$�����~J���6�9���`�3��O���*X�(�c	t���;[������}�����v�����s��� 
_��&�p[�i�jS��B�)@o�w�HQ��c�;m�D������u����H-'�w�����~��b�6@��q�6�cG$��K���1CB�	���u��sBg�I����8�#�3��z��^���1����q�I��x�+�3"�$�t�1�f���3�e����8)'�3��z������1����q�K��S���<|�u/������z�<��9��V�Z����rq����n��z����W��j
��2�����������}���~�����:_�������t�g����?.��_z��&_�����
6_�������_z]������l�B���r��4��^�|��w�[��V��V!�U�o��;=��`�+�Kz�KO
���!�
��Br�@��@AM������M)����N��q��Oh�=��wq�	���i�����Jfx��zC�[�E�'��z��J2�2bK�]��%)��6�����^J"�^D�g��7�h4z>�����H�u1�g#��
��������W59��G��FX��/w_�[��������!����y�W]=���[����%+t�oU���1���
�b'�!q�����l������wR@c<��;�����jwp������="�uE��|���������"]6��#�e��������Fi�
�#��u'�	�u@��#<�2M�#��'�	%��������'n�h�A8\�f&��r�n�b��VC�t���n�Js��Hs�vC�9�Xp3/�WT;�k
�=H�y����}�Xy�O�Q���=e-��~9��{U���CW���9W���q:���NN��\k������1L����em���Q����P�b,��S��X�tE6��;,N�V4n_��=��d�����pGn�D����1;#w�����O���r\""�}��M�����}�P8Q;/f�d�Mu��.K�����Wwl�%��[���D(���,z�jn@����zy�7��������H������f(iq_"$��%�x/���5~���S�����ns���]��k
��J�4�(a����wm'~sQ�k�(������� �$y��i�����a8�<��Db{��F�����Gp,v 0�w-��������.�q�k;�|�����Z���k���b��wm���4�p�~`�P�1�J���/t��������������G�85�|��������Zwjqr��5:2�-K����6����Z��[ipc|���<��8���]�O��^8���
�Q�k�	�[��P�B`
����?�p�����4R
3���h�f	)�����6��h��PnX$'����d��N�v;����l"��M���(�������R�Gj�����D��{�)�/�~)��@�{"hI3.��\�(x����[r���`U�Sq1��i_�Tx����R
 y��!�5bu�W���52B@5�1x_B���!��u��1��(���/Ig��Y�t�?�!K�4�p�t�&/�0:Px�w��c�g�������Lx�M���^�������P�M0��N�=<sG��<�w'�7n���l��-����m�bB���^K��9�8���Y��;��q;;ke[=��?P�p�8G������|6wh:n��Y:�{��+��Sr1��.�L�!$
qrq1����.-,���������k:R�<���qM����a��b����l������^:��.����I���{"l�:�������V���R��FJ�����{a�M�L�)�tP+M�4�ZL����y	6�r����jU-���{[���
�4��
~�|Z;��������������gO�<��J��gI&�R�/��\o�D��L}��![x�n�/�����kRY3����&���Fl,���Zo���/���D��e@���~�;?;�p��W%�BVe'	���z{����&��tP��
W�m��ix�bxe������k�`�z�
�(��T��`I��M
���G��
�1a���5�h=O�u+�n�3!=:����\�8
j]'�7�8
�8)Q�8��F=H�7�q����W�����Q�RR/�8#��T�����S��T����>k�B�Y9�V'4J�MV!��Y�*U�)
+-������H��������5���d���o��j�FYhJ���FO����k��������6��4�b�s�w�4���&��@^�xC5^#}I����s���{G��8=^��[�I�~������@�]���3&z��r���`�~-<_=l�0�9�����.����l3[.��6���l�������O��{sq�Uvy���3��t���l������i��\M?����������o�l\~q���g=����7��8|�,������5���"����_�>��>�`����>�m��[�u9�T0\}8t�R�R�rP����1.���g���w��6Y�f}���r��������,~JV����Z�8��Y��}1z��=g���w���J��t��Y�xV ��������l�m e��UUB%	����*�f����S����������Y~�@�i�J"?d��B/TW/�����k�,��������a�7LXB&o��7W�a�����%��a��Ce��|����B��\��\�����;��\���1-�/������C��
��y�$x��������D��?��|��)���lg7�6���aE�Eu*�F�<!
*����_�����'�1*�����./���E�c2���
��Cn�b�b �P#l�[�lg�g2��q���@��qb�N:q5��!�#�����KDW
T�[)�NB�t���57�������UBG�@$������cNT�����\&F*�����,�\��\��V8���������
�7��6j���e|G� e��	�U|V'*p��L\�z��������4����x������Z���':	
�����d$�����!�R�w
��~g�����i���1��A��Ak���Tj8��6������9x���}|a����G��Ac�8lj���i���.�T�ZV�=�|���rB�ET���;f�<�����fSc ul���<$=0�Fj�Q9��{�B��)Ib��+Yd+cO�>�S���p��4)�����4.'Q�

����A����9;^9KW�R��������s��k97�87Wpn��63��'(5��k9������_�9�M}hC��wjn���;^KW�R��`���B�����B������s�@��93{�R�����M��+��%�s���b�����x5��K���jduEa�?P�7�6
�����Q
R������*��x~�<I�C�j��c�+XGV������Jqb����h�\#�|��p���~����f�l��s����.{S�D=�~������!�?EV�~?�9������rEs���)oD>s��JW�hP��{L�9!�/	����Q��6�m����T��`Q��$=�k:����� �A���r���y.W���\?��U<�,����<'zr�R������w�s������MZ'_���-(}
�k��q�����jv�R����
�7T�r��DX$+�2�.�w���K��l�����.���������Jz�X9+��Y���	�D�~��n~����k�w���GN���#����9�����������
�������M���)�v�]N�H�3�}�����f�x���D�+beS������W���z @g����_�(�������(�p������T����)���n���Zno9T���U<5��}F5���6*��y�d@@��M���u}&x��0c&��:V
r3�Pui��V���Prh��5����Z9N���h|������6�@�i�v�|/s�5���QU���M�N�
������/Ev�����y�������Vo9S�������;t��DN�ZAF����.��{�M]��U�UG74���(��)
ZC��#x';�x����5��T��ij\kH����^C����������5����;\C�ttC�y
���Cio$5��
n��w�c����`�KR�{,��fG��GB��kg������)5p��9)�e�X6��j����9�����
Y~��[�8+�e�,
����<W��P[����e��cs5����OF��G�,�
5����������
�L.�t>9��}�|�����C5�C
��������e��G�j�5��W��_5��W�h��{��������q
?a�jGj>h��`�|�������m��o��}�������I��XvZ�����.���wsg�\�+j������\[�7�{op�#��phN��[�48E�����M7^��&c����,[����h��c_n���r_�Y]WKA�E��1�	���K�
q�G{~2�C�\��[��d����<��b],�����%�pC��J��%�u���6�������`
h��G0v�^W���n�d��k5Y��JlUD�V�]��*�y	mS_�mD�pr����{E�g�W?���W�\����V�@���wE1th�+Jh�8�+
�E���M`��[��l�j����{���~�&��a�UV���,;��m����5z9b��,c#�4��NG�p��9�3#|���A�JE��,�'�<�3#�6Rf����Br�������%�(K>�3#�����/��E1(��Wuf����T2�A���FN%g�GPg���m[y�m���6M�4����/H1���\T;>&�8!� �M`G=R�NoKHqB��jIH�> �lhaR�'�8!���MH��E��0�&�8OHqB��������w�,��
���Y5�"�Uu��������SM�x�TS�����/�&�0�i"����g�.{9F+3K���Sp@����O��@�������bk �<�S���rZ-N�xT[.=m��
�m�d��Ah�z����6�B�0*�cr��A���� ��0b��.�\7\:���58	k�-�pG &#���	���-V�Vp:
}5<��t+Q�~�^�)z���+�C�u���%'q�������dDYi<��`���J�&LNGr���V����P�'`Wrwd?������
�i����f\�04�����<�=	�N8�@��{��lE�Sp=f�8����l:��mq����V���.P�v
2����	���]?����k�����n��U���w�x���4��Be9�����%9��b{"������'����r�u�sIv�-���88�������rP�,Zt�q�*':}��}����+/���2�1�s��<:g��n��<�f��ch��<�f�,�b�������NSgMS��xXS�$�'_uJ&	�'3M������s<����=8��r�Z�������DohW<U����Hw���iw�w��H�7�Wa��N���i�E����.�m�7��7_E�����l,��{Q��I�'����|������P�~���d�C���)�Jt���z�d�w���{!�/�����KR4�.��(Z�
����7P�8�"R�y�w�75
��m !8q}�������fr6�Y������n������4��i�7���K`��*g[����2K`.,��b	t��Z-�.}�����K`�,��c�V�����7`	q �AK�G�75��K�7!8�K`��29��,���%0���%��o�K`�L����E3��l�
�U����J�%���B����Z]��w	,�%����Y��K�������%�b��Z=����hP	\�	�I\�������Tg��,����$/�"�����PpB��gN����Y��d��=y.���?8�|-���oD���z������Yp�^+��Yr����-��l���M����i������9��nf���TW[+���
Z����7����,�:����6Y[t�l�5��p�D��;����q���T�& ������*;_.o��uOcul�o70]�c�&*U���
��g�-,=V�"��2Rl��SY�	���4.LA���������j�]�?}��S9���2�a��'i8�8v��a�����C��%��@�F}6t��a,�v:L`u.ws������7�D�-�u=d��FwmO�p2F�UP���������G<_GZ�Q��\�z��������b���+���r])�k�n�L������m�������e������-��r���e���X���K���>��1MC���������i�����V���##8���e�b@��f%5KA�s��!R@t���8H�B��_}��l������������n���O:����15�dj�)n��������@7U�@1~S�V+N���D��W���#25^�~��������@7��@1~S�V+N�������t�}f����n�5��4��
%l������Xnf�A��kZ��u�4����>U�
�,���l���V���?����j��Zd����k�#zd��7��?r}�	���%�P����b�r��V���_T�/g7U� ^��u���rz�8p�Y�zR4]��yo0�eW������S�]uru�A*�7���t�@�s�%s���W����}{\1�h�{��=�5��$���Z
b,'N
���KQB6$��0+K 5��i��� ���1���.�4�h�&���Wp�*m_i�X�U���N�:�M���|pg�F���
�X�E#-{�h��8;�<�.O�t~�f�x���l������>���r5�T���O/.�^�L���w>�M_+K���d�#-��������r��,:O��������@[x�-1w-�BJ����pvq	0���I��Y���F��?���!
��ZT�rS���jV�{G�l��U��3P��������Rz:B�,W�1W�=�=�����.�����=�{	���AS��F>��s@���������5 e��gN>_������j�r�X};:r��z�rq�r�p���*ZW���<\C��)���){ ����e�e������.��.C]����N:?^&KU?��?T-jGx=+���%���,	����GWY���u��Fs�j���P��b(�����|�<�-W��t��6[W�����Xn�I�����Yu��0�S��V��c���f������[�>�������$�
��|�!��*��4y�~���RX���
�b���7��(�#,��M!$�2kdX]v*\�*�,�GxE�0�j��S����]�`��i������V��`PP��l~�����Z}]��L%��Y���K�z/)�T�(P+k��5���L%oW���v=[|�fD�OHA:-��N��&��+>g���p�

����@��0\��l�)������#���`�����r>���]/W�9�nB�oj+����c��^ ��e+����=���[(v�+���h����l��22�zMu�T���L�d�V���~,��{��f��Ma��<_^?_����Cd�$��'��������d	���n�8�`�,��f��g
9��3��8��ih_�_��yP3�Q�������5��7Y���h_xcSN�U�Y�h�5}��������fv���.3;"0�B������{��@&hh���&�����>��v_&���&��~��� �~��70��Q�;e"��dK�p����N�G��4��G1��[�����#�9�8��l����y������^J�g���0���P��@)j/�������&;T��;BQ,�6�)h�t.�

��r���H����UR�)wx$�w(�����.����.����vIa�hd�B��GF,lnt,(Tz��r�K�����U��3u	.L���?��.��4[cqz�OWU	
��k���B����Z�_d���3t��b����,,dQx!�������Lv�D%;�x.J����G��x���CYR���k�����k}�.�G�^<�]b_<���]<r��tnC���P��w����_<
���\z�����\�x���#W,�z���v��5��(P�l�VD��6����x��-��,��.Bp�� (��b�������M��$J��u�7	�q���m�����t[_o��
�X��}�^QAZ���TT��TA3�*B-� `*���V���C��i���~��h�mg2�����B���L&B�;��C	�2n"���W�t�����?�&�v%G���N��F����#~(����c�����������x�u�m�88�8��^s���9����wzC���2��}����9D�c���e�p�{Wav�I��f���U[����A%�����%���S$����k���Y��Oa�otY��o���Q�yq�$n��a��-G;���4���K��\�Hu}�2#r�����q�3B� �*��vIw��kO.`���M����\Pp\P�\P�� \?tR��F��Q�Q��*U��MunjWo���B�+�B6��a�;(c��*��|I1
���E��*
�5�7���pH,���T����^9��������	�U��I��!_�V����F|~�k���	���Q2��M�|���]��]�d��{]\e8�����T������Y��Gl"�P{��M����j�KP��BwC���X�l��������'.z�����`����(p���,��QI��Jz���k��,����1N�CKp"M�� F��i�"����uFT�s����`��/�q94�	�hINBT���h�}7�N���rP�qP!r�hZp4�;�I}����4�:t}�9R��f6q�0O��9�@��!�
��FE ������FN���u�'#�rA0�!W�;��|��2(`F�)H�0<@����^"�����`��/�������_|��o	p�)�R���>��Z��Q���i���c�|���N
=�m��+6���M�|����)�JjL����h���Z�O]k5�5["�,��N�[�:�;7��U��>��[��l�c�����,��\�+1��b��sw�Tw�3_��Y���z"��D�����Ysa�r�Ys�f��_6����Ys���D�E�3��#|����1wfu�x���-=�~E�0�Dc�BXP
ae-�	Vx�������������������Xp��>�l�Dr���?���A���3�������q���G �M�d.;O`�|`_�5�E�00_�
{��DYM�
���?�Gw��+n9^�z2����mb���,;��-%XM��%��	�F��w~zyvqz~����'Ys�'��O?���0>�B���DA� ��*���}�z��o��s�/0_�nA���x-p��zMHk`������������w�'�=�&��V�N?x�C��������^������C�$����q�;S��q�x@�c��)"�����������S��b��9��t��L�hO���3���n�o�eu��2��E��v���.�/����N,����'�����q���s��:y���qj�S~^On&[I����z���~<�w��;��w>�������}E��`�X�����`�H;��A��Q�'�HD�$��4��N�-V���oB���\�8/R��?oV�&[I���z���96��Ja�<c�y�9])8Y!�=�i��_e��WY0M���J���X+��3i$�p��Um_g�+E�9�~���d�T��`1�&[I���uS�p��;D9�8�j�!��6�{�4���k��H�Mq��� ����}q�_����w.��O[�����
�u�����z�86�y��W\w�H�a���HL���!���R�I�I�#u���!C�Cz''��$a��\
v��a+��)�2��5�7���1����PS��gm�Q�p��Qm�QxL���!�5	)C�z��Ha62�#���_
BkhR���`���ezG�N��	��Ja�2���Z
��U4�o�v2������������4���}�����CSz!�t^n���r�a.
�����5��z�;�u���s���Y�m�������`w�o{�U����Z�2�}`�� \������@57 ���r7��b!X������ ��3���ld0�1����DO��5P��G4k��#_ns���*���	��+�[�����6��6��j�H�\o��5P�jhaj��5&��V5thj�8+"�$�0�N�;	tz���C���Os?�}����M?�����lU��g�!��g.��~8������O�f��l�5��>�W���
R�W�"��P������|����������|**M�WW�\�
����SP�(��T7����V�eH�#����W���[g�e��N��KB���t�+.����]����������/uE�K�{Vi�)�wq�*���U��8��+U�*c��>C[���9��'�	����c�p�i����d��������	�����UP�plP1�3#��j;Y�#�;���~����g���q:#�����qMx6�6+������� ���
��=�8��=#�?���
4����wh�bq�a�i-��������W�W�E���l�����h�ig��Il��
�T��k��pM�*>��!bD����*����ak�/4�_��,������;��:z�?ul��t�@0��7vwg��y�7�>��eX�c;l��I��WE����a��ngv�U���64(z��(A0+6"b��j���x�C��S ����/s�/
�������vR��2b�GE�_���#0k 
����5_�x,]K����S ����/�/�6|is{4��������|)�0�%�/�������zXyeJ>��`����C���4I�Q�%2'�7�db����e7�G���0��O%���,���x�#�$W0;�g/S�)p6����l��~>�K.��u(�~!���A�6 {-F�����^�QvF��"�����7�M�8���4����K��=����*
�E���UoS��W�}�{�K��m�C��I �7m��Mt�^���A��a�w~;����
Vrgq�N������/.����{�uS�}a�6_/q���q����W����eM'�����a��Sx��|���)�������f�P��qJ=��/;������[��`w, $�,F�:U�&}3c�M�����0��b;���JV��^+[�T~{x�A�o'��'�H�����(^R��=�O}t
'�:v&��v%F��
�6v"�����A���`Yk�@a��i7P��Q�l��K��7���l`GG��
|	�iGmsO���Y��tg#��}W�m$h�����kX�������������v�6�]mX�Q�jS�4d|�D����� 3,^*�)}2�w��#�����F��q4P�GG��P�(9�?T�GG#��g
��D�r�T���CQ���
�8��������!�?CMf���_�t=b�
w�vLSvL����`����P���,���0fo�+����6���!m8�J��F:o(����"�D+�hN��K�1����������I��#�|�"dD+>i(��OZ���D��X#�S�,��@������KA�9!g)��B����&��D1��c�����x?���0��Z��r<��m���|����N����'��B:p/��9�Q�&nO���V�Mv��`�7o�l��A��+�������]�����s0�
��#E^��q��y�ByC�h����$X(gX(t8DWP����"�Rp�6����q�+T�;�$K�M�

i��u95�A,d
��h�#E�z��mm��C�W�hw?���I*	F�#����$������w�Il������MI/k�����YQ������Z�g��:�-��v��m��J���p�7���	��T����2�,3XFV�a�tS]9j���
b�C:#��*|���o?}�������Vj�o�|������z�iC���u8���	G@��W��������c��8M_��r��F�5T&��N�<H���/N������.��8�z�T%NJ�V�������DR��&�>����>����7�us��z��N:�H�C�������p�>"!89��b��8`���~���vl�c���
�������G�75�c	��s���AX����=���Y�cc��0=�
6yb"�Al���MM��FDBp��C�
�������zJh,��U�4J���#E2����Vu�Y�]g'�`P�u�J:�.#�6��4�+5i]��r������R:M�J\W����2�4'#M����A�]�YR�3��k2K~u|uf��j�2��6h!�2+�6-"��� -�/a�yf���j�����K���$!K"	���`���b����?�FGD�����p ���0=1MMw��QJ����^c}���lJw��9DS_��_?�/����D��0��A|�H`4���b����0���|�a4���b� ���I�x@����~W�l������E�!��&�:��$@"7����$��MA�A
2����`G�x0�EA<XB
2@&���@J�x��MAC
2�1��� M�x�EAR���k��0�_A��� |�+�	_��q �
��n���yh�X�<���#mbAo�x����!�v�]TW�m���6��z����2��U5/������18�������)���~��gN>_==�a���H��|����i'���kS:�3�@q��W�T=T�z�u�hT�y�N�����+�Q��v?[B�^�F���V�29�d"�g@��C��7:�}!D�?�1;�_����3��#���?����w�m]��7��*�
z�(������r���`�����$��z�Q�?��[]��n����Zi�i|G��	w)�'���X���08~u�
hCP�H��%S�:7�yF'����2Z�����mN�/�������n�`9��#�@��C��qHg�����IR%5��i*�>
�p���$-H:�E�6r�yum�7&U\g�l9)���2�5$����#�Y{�)Y�$�j /-���������r��b3��<@������������q��w'�G��������}�K��?yv�B�$�a��{����9���M�3��
�r0z�EeI��%+=����-J���KV��c�q�#+��3/�^�[��	Z�7��s
�z{ok���P�� ����rR���Q+'A�q���=�H&�z������Q�+�+X����S`��jSr�r�Xy)H9����)�H���J�ME^OV	�����Bx��=��'f������P�q�o-Vv-Z�<���� �	��z���s�����>R���h�2�I�Ee���}�Ys��cx�T����+i���U��$���
��Te?��$�������T^��Ic���?���U���{>:q2�^�L/M��c�j����;Q�=��'fN�lg����fs3{�w$N����_��D���������?�o�3���v;�Z�4�������k�k�]��TW��ry��Y���e���+�*��P*x�r�q�.�@��2.(��V)N,�!K��%u�+S���N��u��:���J�qV�g�%�	L��0�NX���fP����~���
����CW��>�
c�������o�C�H��+o=�������\�#F9U\���?�����4���If�u;���z(Xq�Ns�Wws�A�Sv�6�b�K^�\e���'s���dj�Y����yz���(�Yz��a����������4j���P�3��E�ph�G/�`a��`�����a�0
�������o������b���z���r0��%d�n�u�^�K���P�Fg��Gu��(�J�6r���T�����w�r����HP�]�{S���5*Q�w��#����#����#����#��u����������WSn�����h<��q�o��g�}��n��BR�B�W�����c�|�����3�E��LCs�R��}���v?�v����
#�bN��@;��OAK�t��f��(O�`���^l�I�	��BD���	�OA<�N�{�����W?�����N�����`���k��"Y^���	>V\������m��G�����A�N�7b�
Wb�l�n�6����%"*�+},~a�����JC�K����h������X0H�cl�c�V���?�������F�rG��]��N�u����t����H$��>B�E<�� <\3�|�8G��HJN���x���OdK��e�8��4����O���w�E~�)?�+�^���[���'�����V��v�8~��^!��oi����������0\7�������6�~+��N��F�=������d4E'
e��l��(���7�&�+���'=�.������m�)7$��qH����'��S����g{kg���X5�
w�2�w,��H-�c�|�\�����$�"���T���uXd+]h^����w�:����������V�*���nA��Y1g"gY1g���Ly+��/W��[p��(��3+Zn��dEkw;�;7��U��>��Gc����e��%R�M�d��c�B�����������3+Z���dEkw;�;7��U��>��z�������d��$B�����5�]��(�
���J��Nww_b��������{��WW�UV��Uu;�M�ug��\���W��lU.>�Q�7�<_��sx������U��V�`��l��-��\cN??�<�8=?x�����9Jg���	��|)��yU�7x����A��3��
	����JAY���T��^�_�����A?��a�?�#B��F���Y��X����������%��T��}�PSm����C��0�Q�=l����"��Ky��)�|���tL�h��X���qM>4kV�6�T���#ZG�8[����I:�����I�:/�X��YO'����*��m;�.����"�������<&Z��R�x�g�/�L�!A�bL9g����q�e�����SC`�
�1/�~�z�N��
B�c7��zO=�����3�r:����U�7�\�$��^�����"�~���'�Eo$;(?`Jr�;$}��}��PDP~��I%A�Q#(4��Q��8�C,(?����o�iy#����A�q GrP~������%�a �rl�G���Hy���C
du������l�i�����V_�E���2[U7����S�e��������v^m;�X[nfm��JSk��V��^�'P���4�����y!z�a<@�/�X���]�!��b��t��X�,x�WY��H<�����?�#���a�C�s�w���95��R��-��NN,�9�#QFF�J�K8�P*��}��|��If"�".�T��S���Ri?���CN
6��Qo(y�r�JM.��d'%Y���:��0�UdI_Vk����V�%mY�K��Z=�te�v,��j�X������(�UcIOVk����)��fR�I�w&eo�Y��l�)[,���������r�����r��f�l�����BA����7���i���j]e�p�2�{(��n�+�Cs��>3��q��J��D�@�Q��E�����T�=ROF��9�V��J��CE��2��/�ft�.����
�&Q��e!���r�@x�8�7�3V����~�N�������\>5��s� U�d������.yL�l��2k�P{���|cMJ�qb5
ZK��F�q�����J����jJ�S@*~j;Y�����t���_�����z��)I���^f�O�KuFG��[3d���d��1EL���$���	'�K#p���K�=��u��A��$>O�u9�u4�
�xp��O��r�����D"&�5���"j���5���5G�#���\��`s+J��ub<��r���[�kb�Zs��1@MB�i���q�m�Zy�)����<SKM>Lc�J���Z�b'�y�
;}�&�hI�8A{�B�1��{�W"�
��;��?QN�M�T�Ng>(tf0�/v�`�������}�s�����n�0�6����m����c�m�����O��T��Uo
�0]��B�Xn7�lqU}��`5>x������,{�����fDYO�2�������|9�/��+�|�&_�������zs}�9[o������+��p�{9�^��^��W����;=����X�
�������"d��Br��\*+=PPS)��pa�.�j���' �!��|�-�������M���E5�P��3�1!f�fR3�1SG�c�8���6�L��R��rq�����PZi����>]m��XUa������>9�;�>v���FE=�I�3��Z�,��OhMv>�U�	u$��9�+�_�Cm��c��si���!s����Q��I��B��&:��k�������nm���\43:���:E)w�&�:@M�g��7��#�7h0}�'��$jvuws�l��J�v��{�E�g?pg2Cv�Kt�x�nB"��{uE���uV�=�����r�;Fk�q�UH�z}��7���!���	�Kp%
u�������;t���Y��UHm�[<Y'a����k����(�����;v�a��V;���/������|���oBn����E|�@�4�n��
�}����V�N[�q�����J,F6n����w��������\���&n'|�CGx�}���'�=p��c]��&�D��p��!��F��;�ir���K�f�	��8��}h�l�8�s^��2]��7���ac������*�z��?�TT��(5_�+UT.�(+�XQ������,\������s;�gn"����M���=s��hd�|Rg>�g~���f5�r��������V1��3n��\�r�l�ub�T���� ���Nm�7���P�����f�.����h��
j=B�:�s����Dd'e�s)_I}8�T����E��&�=LN��h5ZR�/��xe��;9�R�R^������J�;�Q)Q����C�q2�T��b��S]C�p�w�����H�0��m8@��80!��70���8��m8y�|b�����g��������M�m�#���^���.?��
'��?�mY�/o�~���sn[��m9Wm�"�s[v��m�/��m�5�8G*gotd���a��G�����H�����m8�����-{3N��XL�c�������P�=���qq��P/���-��m��������Ho���3d[6���d�e���	��g[.�m���B�-KW�F��]:�k[��|[nM?N�����m�q�k������lj1>��G�;jNAG����o������-��|����|v^j���l\��=���_�����Ud�#�A}`�7h[�N��t�:Y���F�b#��ff�,�P��g���\�C�d����>�3��\/�6Q.�7�r�������b�F�Jl�\xD93g#~oNx����������h���s�)op$�s���y�	�7�g d���2�V����UG�/����9#^��~L3f�������Gf�����`�����t����9x�����t<Q��(���OL����E�:|�nf~��H9��.���2��E�����k��o{�/r��qA����n��U�o��Z]���K:dv8�
�P�u�6��=�"��c7�t
���9��	� /
��a|�����+���eh�j��fK<?���-`A��,L'��PX�N:Z��>�-��1XP��������~a{�{a�*��87(��sN������tiar��Nd��l�����~��p�,ta������Y�^:/L#aaz���4�Y�l��I�B����N����&�G�&�����2�:�%�0�l�0��	'����D�/&R�ATa
V�]
�u _#t _t _t _t _�s _�s _�s _�s _�s _��=�'�n�m�Hn�Pv8�u��u��5��lq�+��W�'���[�k��Zrd�G\Ku�D�8�ig�*m�Wg��(F5���k&]���.l����hR��\�<����/��f�5��"1��1Eb8����\X����k�����z�n��nh����/,l�n�S5�?0��	�[
�L���(������5�C�D*�;�����B�&z�3�d��t�M�&����~�����j&�����k���[��i1��b�!-�V8��.M9=G�)�|�v��,�oJ�R��.E�K��1�����s	OA�"������U�s�u�.�������w)��R��G���~�jGO��Z�������S���B��8.��)n^����7����������S(�V��|�Z��)�^G��B�nv�r���������_��W{�*��K�NZu���B�ur%O�n|fT�1�7_�{���44�o���/�_���b�9G�;dk�
���3f�G��b��
������|�|Y9_��9Z�AK[�1>��;��w@��'o�B|�mp/�@RV����E�r��v�����h���R���R;�'�h
s�"V=��|�<��`���lU]W�j1��Y���n�UyS��f�U�o�OkGM�j����M&�����y���������gO�<�6�[���9%��R@�AU�7�/�<d�����!�)F� ��^����1��z���b��}��_���Q�����~�����b��meTZ�����N?\������,F�8)B�e��I|W�7�����[���x�75kN�4b���8>�����;�dmoB�!����rr��0Lt���5,n,��������Y�6��pn����
��A���6:+�`��!�����B��pxs	�C�1������]�qx��a��-D�T.���.����`Np����FV+�)p�����������{�[������U���Jp@ f�R�f��"���Cb�G�>�J�X�����
�Y�����
5[�����
�\�����
�]�����
%_�����
8@�!a��x�=�6�f�(�������f������
L�i�m*8�>fW�E�B;���N�-7��ij����S�k<����zg���Sv����i]"iZ�
��X,N���HH�W�M�V�c��ut�r1��s	T�X��w�u��aw[�	�J`[C�vVuS���wSu�P��EA�o�9��)xD��1x�1nT���@�5�-�#C:����Uzg��v:i��
�V:%
T�sJ
�Z���O��)��j�S�>����|��MI�Tk����V6%�S�kJ��Z�����q��R(��&S�L;�Lg�M��������|f�h��sr3�k�P�kQ},7��x�M�u�������>U+8=��`�V`���K����+0yYus���*�/�I����S�?4��/����ql��Dp5_v��,������V�������0����Uc
���O�R�?@�Q�������zC"��_�i��� ���\�5� ���8,@w�k�yh��Q>��N�}5�x��'<"����|�����9�X�H��<���
@�R<
��M7k0�����2QcD
�.�bir9�d�}\��z]o�Q��+�.Z����R<q��I���:\���t�������?V������C�s�>l�S39W��a}��y�������j+&���1f[�A��u�F��6�&8F�g��_�g�>�������_��f�m�,���A��@\*�m���-�����|����r���kF-��v��t�&��\�M�k~ZU�ZW��%nXFV�a�tS]9�����
�q��s�����`<����{�,�AB9[�b��OL2r������s��^>�!>:l`&@����J����_�O�/���rO�n�rN;������z�#xYbDz�E�e�~��P'#�9g3�lf�E�:����T��w�P�X)����'7o���M���4H6�az��<�p�L����d1?��L���{%��&��fk�'�_��AM8��=�5B��K���`�8z�#M9LtVft�WR�����\ <h�������&z�Y��Br���4���}
a
a
�1("�A�>�0�0���1��JN�$^Q��5K7
�D��c�p=G�h��J���&�#����C�uq��Mn���B�r�Myh���N1������T�T�T��Sm�K��M���\hS�8�8��2,��+�~����X?!�c�S��������9:��A�K������������ �w1���c�m Wl9��&��_ �#;���{Zmf���z��A^H�'j�#3R����`�A�6x�M�6�����1���]���3��W������7�b�qKq��}@GT���#�'�1R�����,L8qA3�����7+n�9�2��F��~:$Y�$�I5FH�[���a��IG�R��_O�v���%������3��]�Z
J ����k��m>��4��u���B6	��*W�-�5��k�a��D�E��)=������ec�m�����>m?�;0^��9p�46�����5n�t���ptV�����*����tU�;�
���E4i�?��z%�}��v�TR����{B=7�K������{��6!5Hm��>S[�,�$?�J�D��]g�y�l:�~�6�����3A�d:�P�����4����O�L�4Mm�Y@�����b0g1���d�b�b9�~P��Z&i�b,g2}Y��+�v���ncO���)\�tY�
e3nv@��a��S9�%��v�9�d��f�p�g�8�r�<��y�+�Y��g�i���<��y���,o1�,�F�O��<3���L�L���g�6���&��u�,�T�b��g�h
Aj�t�#F�e����q��%�>/9&��y������U
�����O�9��{���o��o��q��qv@0���r]	>z�a�+"{<8N&4���'�j����������&�KT�41�������������*��
�TNT��:U���J(�e�Mp{��)U=CS��Dl{�����:�
'�;�����
��:]���x��\���j���L��3UR�e-[��BQ�
P(��f��i�Je1�L
G���9�3�p�:��f��|k��`vP7I��j8����
������(�_,���g�h�������~i�.������|��8�0������n^���+}������N�@C�
�0��VRh��
���a��SAg��"����S%G��������S��w�ae��g��2���*{P�� ���8��(*@�Y�7�w�h��z�����L*e����[����m%��� =���N�]����na��V}<���t������5���p ������tn���tq��p�il��vz�n��gw�����D�i��A�������|����](�V��bH���8��q$X�of����W���<�f��gvycs�ld9sdU����h*�J1��'{���^���k"����:��P�B4�L{2��H� b��Y���v�f���w�����5���g�|&�c�����'�9
G�c��
?�vg'N�7wz4���s��*OE3\��9���v:p<����Iy���3������c��Dd#8�#�����HKRA�� ���
T\�
�$�l�$����~E�KRA���U����"��Tv�5cH�1$��7ckIz��$��E��@U�u��26q��2��Dy������~%�D�R�'w�&���� �uC.�8w��P
���GiZQ����� �}�}���W��4��������n����7��yu��D���$�AE�����vwI�>t�~�{�Y']��)�64;��QA�wt]Ts_�O���O����A1;����!��\B��7QY��E���|��"�N$������N�fH,�!Q��s$Z�fW=)����CB���8t#Q�;]��r����K��������]x����OKw��`��$O�9�Sp�.�*�2�w�����ma��M�����\�dn�����MI�D)��I�Lre�+��\�O[�/�H�d��M����OWU�F�lvUn�l]m���*�������\����Z���<{sqvzy�]����]
F�Wf���$�c�}.W�O��0�������[0�����~���ke!p+hr���������_O�9��LeOa?�g�� �C�98z��03���>6�k*1�(��\W��8UYB��|����o���^����J����D�r]��k���N��K��u��7��Mo�z��P_���
3	�5�,�i(���g������#��;���o�c8{�oGG��B�|�����Z��j��V�j�9F�^���<e��;=��PB��#T���s�c�3u	�:w�z�.��%<�������{t�7�Z���\������{�0�.�~D�zcD%��r�7����0�[2,�2A��*rEcU���j����kW�<pU�����D����WUM��XUs������vU��h^Us�������j�]U��U5������jeU-�U5W�\����U5W���bU���j�^Us����W��nW�\���nm�04o<^��<^�@��i~"Or�����X��P��B��\�4G}T�9_����|���|`M:}J�Pfc�d3D�k���=b�T���=O�^!�|#��N��Q{����u:U�Zh=���3\}�9%����F������mb=�h�����+���y��p�a����i������e�fM���������GQ��P��0�;�N�O�D����'�#'Sz���h�a�QO�o��N�b�G�h�iR�#�������z?%�g���P�o`o���>C���_����6�6�����[fd�ih��&����y�`���_��qH�}W�����&���/[w��������Z]���)pq/#�c����a��Om\5&K*���&��]���_c=���,���&K(�������C��7Z�'��ZVX���j���>���s�����y q�}��d	���7�s�H��:��}�z�!WV�+(�,���,
��
��
=�v�Sg	�������C����>��`L+�
)�
Y�R�r�7�o�������Y��������1�{Pyr8q����F��F,����o
��D���od�>u�P�����`�>w�o����a���pJdX�F=dr$����{���r����i��T�4�	�H������5� �e�E�,��;�K�U��6�������.{zx8'VJ�5��&y�8V�<k�g�������&�G��@��D��G���D�G��=1���N�G.�L�����g������_����v���~|y!_+��>��ww��v{9���]E�&�OVtD!L���>w�
]T�~���c^�UW�����Q�"��&�R�l0�����l�������#�����H�D+�U��������-��@���sw���X4�3#�M7-��'q/
=�^c
���&W.�i.��m�z��e��&WM�\�\�rr������Q�k�\q�j���'�n �n��\���&�����&��u�1�Q�Sh��U_x�~gh�o�e�;N�\������\1�4��������C[�\\���f
�\.7O����i���M.5#��'����1�Q�Sh���c�4�B�w�\��r����;W���r������\�;�6k��r���t�����s�A��jr��\��Q�Sh��Uf!,��@;W!�\�j�*���P�\�n�*�;W��si�M.�E��t���*b�\jF��O.�+�c���5����+�4h��68�\��s���w�B�s����A;�\w.m�����s��&��w�R3��������Z���5�P��u�g�z��}�����G�A?H$�����<�'�R�wN�_HdY@�<�����Qa�3��|�?O|����g�,����u��P;'��7z?���������gO�<��{B~�%� ��3��:�V�l��\o0o"v��m|�9�*Q�9@\&:�SeA�@��@���\����zM������5�,��L��L�Y�����wg
~@�_�5��&��P2~����Y�����e��4E?L������X�2���q���C/����Vy��"��"��I�tv
s�fS����t����e���E-.P���(j�s����������V)v������R�P�hY��4��EIT�/����l�#����<�D�a������}[���	�����������U��yJ~n�6��d��;�Q�.�������}��f�39a��fr.�L!F�a�����=��.M�5,G}����/���`-S
k��x���p����M���l��F�M-V��$��9����e�������yJ~n���8�QS���9���^���g�;��.�\�Jh4�Q�E3*�hFl�"i����<j��a(, t8�X�#�+;X��e�����(��� r�@�c|0+7�_�qS���9>5a�Q��y.Dt*�����<%?�\�B����t����W:c���k���(9����r.rV!F�b�cI+�]+u��%v�����B����=B����N1��V:�����I�"7
T<�N�q�.�Z7�X��3��~s�2'�s!zX��7+]�����f�kqb���}�s>1���;��w^+��r��]���c��J7yS�����M�(mbg�(, t8�X�#�+;X��e��EL��(��� r�@�c|�>7���qS���9�=�7�s�<"��y��5?O��-W��c�5��+�x�\K��Jg�<c�y�t#��#v�Q�q+�H�MmD@6��7�#������p����cq����`�Sk������p����M��!m����M�W��o\r��Cw�RpP��*�;�_&�o�.�=�T~L���&��S�E���p}b>$o(�O�&;*W�,UQ�O)���5`�*�{�:;v���]�_�X
�,u
�GK��������p�_^]
�.u
��K����1��s��Eg+�:����e'-;{��$P��E^H��oJ��49�f�49�z7s@E>I�L���@��O���T����@
d%P]J�{�*sSrM����.%��v��)9�&PY�T���^;����@������K�t�@enJ��]"yb%O�����@��������|�����j��6�|�����v>/'�*������z�\���"[o'xUZ!7��k��M�k~Z���S����K�`�2�r
���@��s4�Kh�����V��OFb����U�����ud]�]�\���YG9���#�� G��2����?��������;�.�.��|@��NI�S�
�����~������x���u�;��xi�uN����F����i�-�Uv��x���p�7`*�`t�)0��n����[�p�E�w�M��~-p�����&�w�����M��~-p�����_&�w�h�@�*���nx�Q��ft����:|b�r�.��nS���|��w���I{K�_���td������M�k��p��&��z!m�4Na�1m�]h��&�%f�m
�7�o����6IwG
Ot������|��w�&�`��1!68��}\#��
$�����A������������c�m wl9��&��1 
�������m���6g�^-o����7V��D��C��\�a�.tYs�V�k��[e����
D(v�����i���u���;B�B��"�_��k�o�Ci8�1���3"61����v�=^du�w�{Bq���M�k����FY��,��8�St�e����p|��6t*,�:�!���D8��B]���9nsn��M����
u6gK��r"�V�P7lI��A�s���P.��T�YQn}R��l�P��U���l~��H���,�s'�����cE;��Z��`���toI�hI�%�����{K
EK�H-iq0bJH��d�h�0^KB�����%#EKFbKzC������A��jo��
`�u|���h������W�u��<���L��S9�`���.��z������}W��J*u�h��z:�(
c^�zI�u?��l6!8L����L�~��d*�����pB��E$�ph������i�W�Lf�����_�^���R���k��^���~�4,��0C�T�s�@�A�~�]-�����~��\i!WX��n�`�d8���#���R�]����:L���-z���u�����jj�$�N:���{�0a���-'Wn9�n��~Po9\��������|�
�������Hqm����������#t��
e�S:	��)|]���\��r�V��[]���r�V������,��N�5h�s��4:������1�:�0�
���Z�#t����[��:�Y�tle[���6�
fc*f]����o�8�
�^7�~P���i��Kk=��<��no|7���B�0���8�<�X+�k���G�0�[9�$W��Nv��[�/�<�rq��{]!�u�r�+t{��v���u�����u��9f���1���ua����w���#t�x�@7����w���$���� s�p��z����T�����������T9w��d�Q�?x�:������`��?�l�?���x�����u&�W�����}�$joc�@N]�`/Qe�'������#l��f�'V��M���X�
F�K:t��N�1m8i��=P�3���A�Vg�3X�<�m.��\5osq���y+)��AU;o�s�y��W�����d�����f���6����u����>o
�-��g�>t�m3��[ih��pF#���`"�`���6��
����.4ou�F�y�s������[�<��8��7�����n�_������;�l
����
�X;t����1m8i�v��mw���Vi0����B�o�~[��m��oU�c�.���n�c������;��
����y;t���1m8i�(�����wu���Y;
&>�y���V�oY5cfU5�������v�������*F'�6�j���6+~����
�������$��X}�Hq���#�4������e��wG���^����{7��5��0������2LYT�-+������
�����/v^��Qm�r�::*�(��������Vf���
�3�X'6_g�i��	�GaM�>��5��L���[��V��f����O#N�t�5�qMg\m]�f�;-�8�(l��_Y�j��x�����vu����k��@���eG�d����{x\��}����ln7)vu�F;g�q3�������dv�������PE���e�����c�Y0l��vy�vy��"���$��hm��w1{[��c*m��3.g���f���Hq�#�n�eO3���wv�cs��d	�l��:[W��AU��z�N�|����0�(�p\�r��:6	qa7������b�]\��k������Wc���8L�}�$�-DCr*A�Y9��We��8ZnE����['�#el�:���VC������}�O�������\�1��l���X��i���	�����7K����=��dB�	q'��<�_v?������3z���u���S���+?�_���*�To�l���[��P����n�z1��F�.m���b|���^?�N�(�\�C��d��:�]��O�x��(�|�c;�9���P.^��;�����J�V,���R\���F\����[����[��
}��j�2��pZ���{���/#�$��7Y���	,���f�w��\W��'���>U+8��k0XUs�n�vFj�U7����������%@5��/�r��O7��_T�/g7U�"]��5�=z�=��L��%��i��������@��vv�����b6����@��0�n�f0�������s��7�1G����GH�{�e����`���K�I����cy8��	��]x;�}���� Hw}?8� �%�����p|Kx{;�)_���b��u�����$vxg8�4\C���p�V������@ri%��n�d��2�����x�w�������<`r�(}�{���y(�p&yk5~(��C�Fu�0�6�����Rx��?c!%!<	�LJB���$�gIw&1	����q~�2�����'|w����&H\yuY�^��������5�V�X�c�]���:]U%�P������`D�!���f�r2���������W_r�y�����:���l>[o@hXxsqvzy��<���.�_?�^\��|X������_d�zz���d���~a,%w(e��2f�u�[@����\���S@5��9G��ss�������<��ph�1�KQw|���������������{v|��x�M��\�+{>�z^��9��EQ���~}�s�o��'�w��Uv�\A�m����	�VNf���k6].�@���q���X3�8����y5���z����9���[�DP�L�g`��_��/�/_�/_�����!�������MF��H2'-���
jf*��U�>V�`��'�R�|�6�./��!�t��&�AS�U^�y�LNhM�Tj0�������{��o�D�������o�oH�InR�]��!�����{����a(�������{pZ�3N�)�e=:�����S��y
7*��v;~��s��,�_���/�4�W�/ ���)x��p����=@������?�uQ�i������X�����
8�������g�� ���	?@���.!c��%�F�a���i�^���b�v]I���o�D���-[�E�x�{�;���L�*����Q�hC{�cw��qa��-��#���`�}������)��N:��hp~�4;�����/_���"�����l����
J
A���/���3���NQ]i7�M��CP���S����8�\����Sj�$�/{�;��i�i�`�~�)���5���c��U�Ix�G��<��Pi;��v��C���w�Z�QuEf8Z�*������b��T�&���K���l`����h�$E����<�Fc�H'�!�4�L�X<�Doa��K���e��������_J����J@��t�;����>�����B��;��l&��3��
�e3�L��f��<{X���_6S�G��f�Ay�u��V�4���h�_&��9nG�"�F��xVm5<�ML�p����sZg��w����d������ 1��;�tCG�#5?-WW�
���?4��.rl[!�*�r�����V\�XF�{�����>�X�C�t�g�FI�m��h���)^���w�?�xa��( 8�V�,k��6����sx������V��������Mz�Do��G���M~�~�&{��@�HA��7;�.A����M�w�#v��B'�{���s�]�m���WT�q�Pn�X�p,�a�n�n�`�(A���U��5�6��]p�8�Z�����I��	���5}�����3"]�QIe�7�O��>�9k�Q�H��>���������
52�����K�����t�r��P����R�^%t�����N�iCE�@d^"����$�`���c��D]�U��q��`�D�;�� 8��'���M����� ��$����u#��IgOMQ�������Vjs���j�M��B;k���>n��4���hv� |Q�(��A�u�rX�
�Z^�1/��n��B^�zC9��^
w&�U��jQW�5a�kW����
H�	���x:��������/��vU�9L��4�~Z=V�2+��z0^4S�?���^�rqz��6��4Ao��E?��?�&�'q��+��r����2R��K���\��5*gG��:
6��ho�c�UO��{�]nz����C�4����b�^~���_vS���� � /zp�5��:����"�g1����qV!����g����~2�mZ"emb
@�������%�@�����vJ�)9.rak��a�������WS�&�����}��B�C*P]��N�k��V�4��+�.>���2Guc
��iZ�R��4�a
�m�L�������]���|�[2�6����An)d�&W�!w�L����B������X��"79d��!q��"��u4S�!�3S�9�LM����g���g���3�h�n�,���"��)7S��3�h��)r�CfJa�)�nf�8�qf���$wjZ��B?S
���Q�C������c�	e�5��6�iR�S���6	���`�M��I�PD	}��y2�#�"@� ���:�;�g'���R��|��(g�-c�RJ�j�k=��'q����R��CU��,6F@0�AJ�R��p-�R�!���E�Q��j�l���v5��i��9���~����C���l��f�e��ZT+�k{I`���|3��W�UU�&�t^n��<f����.��M��C|(y�}��r��t�-����&���ya3�S�����=�4�S5���=)�G���2�%��������������+xaR������ ��^@X�v� N��/�������C�M
@b�<}]85���������ZG�'X���U�%{����G�}���{\�Q��8d��P�������������G_�7G	�	��
�A<lp��;jqti����`/mS�85�<�����q9��Y���X�:Ws����&�f��-w�j��n
��1	+:P�%���po�2&^��?ra�}�R�f�����P���r�j/I�/w4j@�-7�����VTD�{���Z�w���<��_��:`n�
�F<������o�����1�E�k��~x"y��^+_<�D�3O"�Z'r�\�+��8��|�3Al���r&��U/�uk�
�$�x�����%���f#s]��H��t/4iDd��������Ck�O�
[�~���������������|5�ac���rx�^w<����@�C����r��eN����\�8���2[o�u�l��	��{�@q��0
=�k���25�������(�uV���?��Q���3�4�el��fQ<?���qv3��%r�4�5wP'�����`A���u�S����]��_����I��ti�?��
�u�Q��Dw�MP��F����I�th��HJ�BG���R8D�9I~�����x�T��XE��G!�xS�~���GN�)D���H(�������O�,z.�0��{�.q�
T�,n~������mY����4u��������,V~�#���e��(��u����.8�=T���`Sn��;:B����=��H'l,�#�@���;����=�����U(�������v������"RJ�!��.���b�[�%��'�����v^N��j���
`t��?���T]]�UySAa����0���
CU���Q`���A�"�"�!��\����e|���=��U��T�sS9���p=�gE�F(���� ���p�C����Y8)�*�Q*�!����!@��B�Tn&����r+��M�t%���cZ�$F2��!�P}�`mytN������Idn�x�j����z��hvjU4ht%�����C�L�B�k�K�keEy���#�� �b���lF.�S�.�tB�
�+��nj�'@(u2�
1���1���0W��Op�>1#�F1	�4?�d���T����q6�Tz�GN^���X�r
:
qlg�
�Z��K����G
m��6Qrz�!����B�r�M���^��9U<Rh�N��Z��K"���r�w�&p��}���Y0��s�*�	�%��0�B��)�u"�e2E��%��l���
����WH�nj��{�fx���� Wz��(�������&R��	c^t��'��Ap���UrK�-L�T8QXJ�|A|�e�P���#�0���3Y�r��7����.�X=�a��!:4�7d;R�a�������&s+]��B
N3H�
o��2qf8���x��E>?Ub�������J����K����S����'~�2'v��X��}�v��o�3��s���h�8������pbf���4Mk�!������!�YM���1n�?�����9������� ���4]Ck����'af�}��.�k�g!G���o1����6�X^�� 'vf[bk�����B=���^3��C�3{@g�i���a��gn�S3�"��g��c���kGe�z	��~BT z��l�61s�
df?.��������ZAuN�'<u�]��p3�a����������1W��gS��c�j>��Pz�(-�����v�j�E%J{''�y���>M ~�����u����p5u;X~��+Y�:��m�nk��#�l}��9 ������}Cb��CK��}.��j
��C�����������y��9��2I{�Y�ge����}
��
��(Ku�*������5x���5����J��rkp�Y;��t������_���^G�����3_��NCj����y�RG.���o;&��=Mh�*��`�A���\Wk%q��y������_U%��rSf�j�2����
7��l	��Ve�g�o���_0Z����Z���s�rn0���Q��RL��h�X���`89?�������s���!'����>�PwE{��������S
��s���j4���&��������%�#�9��Dm������r���k��JP���^����]�����@�U���P��>V��>\������+��7���Q���b�����9������`��q�!S��)�����78��#h�������A5��<*~�G9P�� �y��&w"H:�x����Ul�+�(���)oQ�)	'���B���o��.A"�q��|���~����z��Y@M#��n���Y��7�B�Uj����
�#��6C�sHn4�m�lx������W�"�l,���
�~V~��y��w�Ly�1��4�	w��sG-B��.f�r�_��t>{;v�F������7t��V��u��W=�s��R]E���:q�v���4�HQ �#�!����Ik�}ST.%�t�58�����u	�G�E���-=��#��K�R�(f��{�<=����.}�K����kp2k��'�?�u����Og>q��/�����8%@�I>���xO�����'�#Q�d1%�W�4��MY2u�OFGr�'�u��L���I,"���)gK�X��R�I,Mb���Xj$*�����Xz��$�����v>��T D�!b�m[�������@q���5,�V0��S�U������I��n���4�l�i�,�Y�����m��$U�������r�$�t?�H[2`|�p�)�I�����/���W�V���	&z�b<gq�����%���~��M���G9;����'��>���R9�5����r$�6����r�l�r���D��`KR{�*F����ou�@� g����$xrP���&�	|��L
�S}`:�5���'L��8�^q�kG7������" ;\����"L:�������Y��
`� �I2�9l��^Q���u���A������@;x��:����k�����D���o�|�����Jl+��������]���K��0{�E���7�
�/�����M�ins�������r�����}�8�k��X8Y�:��7:�=h16�~�:6.v����]
a�5z��F��>rw>���	���/�����DK�wd&�D��QD�S8�I4��/�h�>I4��h�16I4������m�e�����s�����2��\���~���"�w;�.����E����EPts��?���^[�A"����V��
�e����-�k���{!Ik�`�$�������/ �=CY���y������	x�z�����h
1i�E2����+jG/O�����g��Ph�C�0��]m���K��q����G�Z05n�|����	�c��m�]z��>�6����o.�l��w���^�i��J�X����}�'/��qg��|��y���PI4�\��_�~���hP������G��k�jZ�1jw0�k#"���#��Zjs�F`����i)Hd�8=F�q�	'�Z�:=��E�������%�tN�>��\�<.NN��:a�p=��W�2�YqjD�� �0�*����	fG��g��;T�����Cj:]�Lf�F�^71�� ��'��(E<U�(]�t������|v��:�o�X�.�|B����W�u[d]�>�0=IN��^��vy�	')��N�7�dK4E��G4��KFP �D���:���$���c�0����"�G����Y
$�H�������E8)���[�d��������->�ycy�^[��b_�x������ky�U,Vt�c�gR����=�������Fb9���US��H,g$yq���r�ZQ\�U�oU���#�jW��(Fb��n�#E���<\)f�2��d=2��%'�w�/����AMj�3��V�"t��\�����6�cR�t�%�i2F�.ea[���d�
"4#.����1#��{���r�c���b�6
�,��B�0Y]��%iLJV]=V}HV����
Imp'-�
q��H]	�����;�mW1�d7��"�e��$L��t)Y`�
�,0r��&�,0�%U*�*��l�L;�f�������s�t��SfR|4�V�PF#1����t�$��E$I��������@��A�p������M=ZMH�g��-I�4��hb�|n���@�7��>hN?k�A���qn�:ji��QG�����9)e��@��K��a�>����9.g�C]��E�6*NNf.lh������sU�+���=U{���@�����K�~1�������:#]:.�+B�F!:V���)NIeR5���L����gJ{UJ�-�
�U��J�J����D�2��B=g�+d���{���a�g��oo������:����m��\�\�6���,UtIL�����S�T%KUR���L�*}����N�^)L�R����Xu�@�{����+s����lV���=U����@T�d�J6+���<%�U�Y%LN�fu�D����<��JP�:0a9�1�;Y��+�q��)Y��+)a�T&���I}r�{���{d��Y�Q�:M�>$%���:�rpI�I
NRp�����������$')8@��c1��6L�>6�����6������5|����6�-��r��mf���r2�����|��^*��������)y�U�:+2v%���R\�L7&"����z�*�Y��{-�P�&vk�s�n���\<a�u��.��$57T!r��l���,�u~���:�y�M�?]F2gFR+�ES�����5�f��(�z��{��:��,�n���2�3�JwE���4���F� �/������8��(�C�$�����R�v(�}N����sFh)�Np�3���z�A���,��I����}��
�Z����\�;�1���M ����F��
�QN�����
Zo�&Xj_vw#��
W��'3����?�M]��O�p��G�g�I�?����9�|��e����v�����o��=L�rEl���.T��fe�GTh^P���0����:j��9H�d������	���-�-q�Z�����$Rz`p�h�61/N���I�L1d��D�EZ���j���D9�3���<��3tXc��0�*#��-�+
��s�ai�Xo,Fw��]��������7�U�8���"���0G���gi���gdt���@�zB<5��@�E@E'������(
����
T�9�(+�r�hq]�jigybh�Y���"U�6���@qd%�U�8�W-���Zu�Yj���qW���������n�d�H���D9���;9r��Wt�-��6�F!�`G���F�����(����o{j��ul�������G��=�hw����Igv��[h�&���:(p�[��I]�dd��M9�V�|�NT1'%�U�j���_�4�] ��$���t���Y����}$�Nd{'��"����Z����t�[��e�/��j�2�	���`~�d���
�k!�|�k�-o�U�Y��DdoZ���;���n��S\����}��>kme�����R7�6'1A4j!�l�]�l����,���y����+
�%r��Pc�	�f��������6����s5�z����A�]o��D+z���@����hC�8��7��p������ ������V:���ZIDY���6+p���r,������,K�X����-�=��[	��(��l���yw1��\�j�>��������>|�w��/e��?��?�w�g�f?^���+��-���z\��Y�����W��N?~dz@_�P���*�/b��J!�t��k�E����z<kk��DPu67��@�ML�S�sTCe)�	nd{
����R��hT�������iI_9��r
v�i���#4���\�, ��Et��BN�N4M�9�E��S��CCo�9����8Wq]�/��D�p2{��$�w�� ����d���������(dq���R3�aE�h��~�%=��f�r�*�2�����hw���o9�8�������N/.N���Xj�Q��eG;�#{h.q�+OO�a��q���;��a���q�XG8c��Gvn��0�:�	��}4&���(�Y�Y�mV��]�����=���\m�E?����##~\-���.H���3�8W����u�����.m������dM�r��'��q��U���.U��s�����Xn*��^��dr�W���_�����@���x����N��|��)C���N`�����;��x�@kK
~��vV����e?�4$��<������ ���P)������������������7@<��P��g?��e���l��>dbc7'�X���yY�������������r�H���q���9����w���q�����+�s���C���*��d`#���>4��V�;S�g��������^�;��r��?�=n:/�����,���n�~|�}��b}��r���O�����r������T{Bd����:��D_r�4���;�%E('�F��d�VyS��
8��i�H��%8]��!���V�m�.�"�B��C'h�������X7�3��>`Y`3�C��;6W�
r��2������4�6��c�(�,3����pV��t�����*V+�p���/��gu.?v!PG��������W��W�f���_~��;I�H
DR ,))I��))I�Z�uC���F�|�D���T��O��5����?�}�+��C����4{���@RiX�� �mE�Au?�]��Kvz~����Y������k:
Ev����������7�o�������7��������wS\�����C�O�9�$��������>{
����&�������w�{]v4�7(�5����i���P�������������?��� ���&_X%<%�C�N���dy�u5�o�����K���V�.�@{^eW��l�&L��p�^����z���>�C���P!�M����Y��L��eGgo�����?������
���f�����Vy�sZ�p0�N����@
�CI:�N4���hm��Z3Xh������{�)�u�Ap�|��4����\��N0Nr�i��^G����Z���s�rn (^�����6��c=��z��zzH��\�Q>���SRs.���X��1�@��}�d��p_�`��eQ�y�3���:���71��s�.W��(i�c(V9�I�U�D���U�4B���BV�%����Y9��W%&�@a`������5Dh�^Oq�Vk"�;�7E)��D�J{B��7Su��P�G{��6����i,L3����	�hGGG�T���'#������/#������e ������;�=P��b,�i0���7�C�g�G�)D-@�8S�30�i��G��Gr�\�i1�g)�� ����zA-K�Y��(}
8I�@���g��`���������9|W
�����,7�"�H�� (�i�5
��v�PA�^@lH4��69��q+���,6qB��(������I�#����"�K�YgQJ�7�:���*Y��(��B$�t����K �La��@�/�O��"a���]}�|^]o��v�GR�@�~P��a#3I�S�����'+�I~z ����F����x��\~��~������w"I�f?��(%��)g��0���$L��&aJLI�J�T4a�>��;������ISI��3:��<0��$A*	Rw'H�9���QI���$E�_!���Pw-B����}�.�+�%8`IB�*���{�S7��q�;��G�2���<'%����<:�?`���������+����|�
�#���K�x���lW�<��:[��2L �<��T`���k�Lu��Y@b��bGr�_X��O�O�'S�d*]P���'�(U�[mB���|�����n�S��<��[]��]��R��7-�lqIN���-�lq�S������������v�|�7C��5_�M��k��.���;�s��v#�����Y��D�	�{��/��f�����0���X�"G�<u��(1L��x��7�XN�=�Ah����B�mg{c�R���vqU}��/�_1��5\fD�O���2S�ko��BY��jws]��y�!�d�\=p�'��oL�.������V��$:��k�B=��'�����1�{��V���6x��/�_(3 a��{���@��=hV�=P]H`
��������"A
M���	����Yyl	�������^����,[.�����wa_��S8�1�E�^C��0c�PG$
�Pq�3�t���=�#cFD�:Fv�Oq�:�3gE��� ���]g{���N?����=����}�C.]��ul�r
[3��|����~�� fI(����M�$�G-(��
b2:���\i?c+�gh��Y��V�/q�X[�!������	�W48�*��+�����@�<r��9���h�D�"�gH����Q����<��F��������M�B��~���k��%+��B�p����O^�a��!7]�~mM}�~�_�zZ]q�XuWH�w��b��f5�����{����V��rXKE�G�~iZ��H�����7ocZ12I��A��������r-����kh������0-!5y�F��k�$�D�?v��c|LC��E�0>6�%��&�������'/k]���1������j_�Z<h�td��z�'>���p�#�(�/�\q {��.>��@d��A|s��b<��5 �~u#��V��<��� �d����%	xO�����18}1��_���:a']<��su��tE^���p>�z'lc�,,F
�����8����������|�)� 6�N�� �Y�w��?�V��f������[����D���w�wr��;������}6��Pg�_�B<��(^�������"�:�k$;��p[���������@����0j�H���nI�N�v�5T&Q[QP���&Q;������F����N��xAy�H4B�m��[�/��I�Nb���$v+
Jb���$v'�;{|b���C����1|��F��4v#w+�F&�;��I��P��nEAI������$vg�O����~�z;��c~?����7�����f
��d�t9��vW1�nCI;S�.G�68]��Fs�%�4����;��)�t�8'�M���������N���XQ~�%�����B��>���@r��9����A-uv�C��]�t��������.\d\&`�M+�
Im���`�2�
�'�����6$��D�#T�DjyH����
��;i�d���f�KjCR��6��Lj���Imp�9�
Im0�����Z��y�ZC��Q>:|���+�����G�>%")��I�p�3)�d�))I����=5m��]�����GJ��(�!�'$�;)I��RR"��P����D$%")fjv�D�VfyH��h��%��s�X��#�W�O�O����TC��2���T�P���T��2��K*����*���+I��L[`�-)
���M��g�o��\����l��2�-W��@7���y��g7����uR"`���fF�a
B�\1�n���V���-�Mn:M1t���0P����4���3"��y��S�5��sTH����L�E�!�s���-��Q���~����Dd��7D�l�����{'"����!�E:����o&M�!��	�r��u��|���~5B|^�����!�]O�_������!�h�����.�{�KI0��-P[~w�d�����J����~�"P�E%QhX�HFA�����h�'� e��/*	�?h�-��E, KP�sF�j!w����(jE���,@�(���p\ObpR���E$=��k�]�o+�V%�:�`�>�j�h�Q�X�U7L5�@I�W2P������K(�|i01�$����
$�Ps=x�
G9#Q�@(���7�	��?d�������En���1A���:��MTD*�hq�H���A��%��0^�H7=N��z7���"��F{�n7���|�rW3��rB��l7�d�Y�9����D(���nmp6����4��-�qpk��#��?wE[.�j��4v��-���	��H~KCW:������h~K#W:G����#����G�?���|FP�`���!�Ag�����Q5lGZ
tC�P5"T���j$*���X��������"[+���m����n�n[+�������l�����7��ZoTt?�7�+�{��=P��6UV�R0���'Qq�?+T7o5V�?��c_n���}���Ydj����f�P����E��TX�����/��Q� �K��J�
��dv7�$w�rW�]AyWR�U.����@+�K����[��^��QOW����\�'���:g}j)�s���:g}�y<���x��������k4���@������)��B����*[�V�r�\y���b�\�8Te����S\����}��>s��:�@{y@Gw�6J��Z����BJ�<9�*''���<j���R�?������6��������o�)���(����gJea�s���vWO�$��)�f�4z�:"7�F���c�Y�Q����NS�r���=��6�L#J3�/n���@S��-p��;F�Y��_m������7�<���;�������������[����7�����
$��u�l��z\��24���
�d��lq�����G&������t��F �R#�0�SfP�f�0�*q�o.0.�&N����u����C���hAr�I�1��1�W��p/����~G�
jS�k�o�R��7�04��c6�HH#)+���
p��u6�u�6�u�hh7�~���1n=��_�V�W����M�{���_�.��H~����k��������w�f��Yv��C���/�{��qQ����I����)���D��m�&N��7���wM���J&��g2��8�=:���E�Z�h���S;�<g�'i��T ,hl�0�]=��a�`\f	$	�2E��3kx������������*
����VV���X�MK��+X�Y=�m���fwVq�t}�R/NF��W�������c��j+w-��&�Z"���� �p��v?��qv<#�~\-���
�y�����]M�������:�LB��<���5��1��Qp�����1��"^-���?��8��qU},7,
�Z���@�6H2�K��q]c[�h��A��+^}q��O'|:��J|z$�E����$*�����/5r�H���Fl����1j�&$(�(0O�$����3-��C��F�]U�%h�zV��j���|q��)��g���7�C�x�L�s��a;���0{?9�jX���$�������r��sI+}���PY��S��X5x#cL�U�� �()$<��1��?t���O���+Zh�3����Q����3v�O�@���G����U�+�.�Y0g*�A'�	r.��� 9�@�9��
� g��rV���Lq����pvz~���)��5�w\/W���=��r�����m"�;1�
e?���>��g��P�/��^,a��L��`}u3�B�@g���KN����zu�&��������1���]t������=-d�4���O�RC
�z@�����v��r%?Z(7���kS�l�I�S7U����+�����S��x���FL)@)�q*��*��6�}PZ�`������������*7�/����@PMr^�������$�Yz,�yI��w���I���w/z7��]�a��?�������{s�@�����/@���Z�����h�����7�N����B���{L����������<{��k�GiP��go.� ���/��\�}��2��������_.�},|������������,g?�����)x{���]���������x�(��Y����u�Fw���Y�������z�q�����	�u����V@OIG�QA�*��H���>3���?���N���?+���2������z��!H�9������j�����������z�����a�=�����~PV����~�<����k�G�V>����(���N��.�e��w� q�E���|��N��s�`��s�_�W���_��]�/����]�?��'�]�y�����5����P�������CNWU�w�rSf�j�����|3�W���s�bG��z�����?����5u8������7r���
X����-�g3[����u�=K��DzD��%�^;A��?���J�
���������	�RS���P���Z"L ��=��B��{A�U��P���-���od'L�0K�E����[�	)9	����q�rgG'���4��|PU��4C���R]hji������_t/?�C?����|�{�"Q��=q
*c�ea�XB������G���	����C��^�J ����T�BmiD�cOT�o,��oU���y��Uc��7{��`<]��w+�?d�2��'��q�1a�TC�Mc�:�@<2~�q7cq�v �u��\G��,h������u����)�f{��V+�
-8<y����x���u����Jx��2������A��&aT!V�#��_�q�Z��7$�S:<d�������6��@��<�aJ�O����r<��B[�T{n��f��d*�/��X�x��x.�87[��	���f��(��\#QW� r@�0�Z�*t���Y���9QU<�]5��U�,������������rbA��(��d��~���d34��q[W-�<���t0���a�t1��%�,����$B������qP�$B��J�9��|��P�����<��������\[����i��&y�F�T^G�*N�:$+9���)�x���M��n�L�i�\ ���+�
���b��H�%���d����d6|0�����|��k8Y� �7��G>����� 61�Yj�����e#�n��H�-"������Z�%o�����h�\2�|2�zo,��XV�q�:�/��cP��h���o+S;���]�:�9	�8�����yFK��y�B����+n�����	�5b��	��Z��&f=S	c�x���oI����t���8�R��^�K���KG�����E��C>�v���,����5�*Sr%0�s1�:��^�����	����{R��a�K]�q���>Z��&vM�P4�S����H���)���f
�f9{���c`�����M�b�@�Hw��#��q�I>���D,EFW���dt$�-����%Lf$a2��y���>�����a���O;@nn�
6������x������$��N	�I��c�x��x����$�&�7	�I���o�Z��������{�x#�����Lv���)������)����/�Z������>h_q����O+;D�8�w��<�g?PqN�p�����j�z'��ye�>��C���H��b0�;Y4�X��
�#��F�r��p?��32X`k���Io&�b����),e _7H��$�5]�9��J����b��/?#��	_���zWW��:5�bG"
������$��������J.���
���:#h:�����:
��aH���5�������M�P?wz4���45��6u(4�����"'�H����C!&w$=�N��R�\*G>�VHy�R9��8��	yFb9��-��vU����c����Q��;��
Tu�A��#-f��EV�H��0<L���	��gC��������^D����C��O{Q�Y�q��8K�8�����v�������^#����P��T��iQ�_��%f�*�"���.�z��j�������,��S[n�
�����_`�����[R}��7��G^�!�����
���[�����ys�m���n|����P/��v������� �@5�MF^"����|�����%q�F�6C�+L����3���j������K��������1�	���	s�����;S�����8��Ugd_��p�&�#�C�_2��#*�����M���j��w����������i�7*4�����.a{�����[w���G����Mt�c���%[����q$��Bv1��T@��b�9�;�@wx��f���h���� ��GWi/���>bi{Q4H�t9���(
v,"����k�T�X�z�pK �
�Z����X�C��D�PG���u���P���9�������I��{@]�F�H�U��|��]������y����� �^B�i����K>m�����ex���|�����NR\4���N�%����g�0��%��=4�%�������I��J����K�����G
35t�|��1��	\o��-�����������o�
��.�lr9�w���^����%Mx�l�XG�>��������w�����+���"j/�|!=���9��H���w�h�Z�KB�����Xb<��C��� ��X8a�
)�AX�!pp�2J��C`��*$�Ja��B��rl0�G�(��$��������b���8ypq����
�n)�:*�#hI��
��R�������p���U,Cjv
�)� �����0	5��l�;)c�P���G��(��ty3�-��l�X����r�F�:�Q��������6~&�{��h��,�O��z����
�|�$g������b���������~t�#���91�]!�%��D��b<p�)�a�B}��p:�S��<��_~Q�o)�p\0�������;�P!� Braq��(���I�F~�
�t�������&d�;�P�1��@L�t ��b�;?S��)��Y0��v5)�j�u�� 
gA��`tI�R�7������_�@�k�:�:]���Cvm=�����d�k"�B�����&������<
��Vp_�7{���oT������W�����O��4�`SN.G,����H�M9�	[�H:���� �0J�-�J��*cGQ)gqF���5w��*���$T5rj��2?w��ZC�'�5!�����B��!�1� ��AG?��y���
J�����I�:zR�=�������'�P�����*
��hqdfL�q�M%'���m��w��5V�,��	66R�`c#�-a��A��@DX��2P��>`?��g�<�������G��@��2D
�*]�����~���H)!��?5���=M��uk�j�O�D;��j��S�������F{J��	hL@���4����{��$_��C9%�H����
�*�|��*���8x�����������&_U���W5��&_����|U�6'9��hgS�UMr��MT&�H�^���h�!w���#�|UmC��3tv�a���<��v��QY�u��L�����dAPp�!&X8��Nns��0&�8���(q 1�&�8����q�1��%9a���������=��t�{f�D�{f�������N(��`�����ZH��C�k5���Z{�x�U�{1�
��M��]L�;:��<��8w �]���~���<��1��?�k����@�qD��(��"�w
M����X@�'�`<�1���3g�Q����w3�$�k���f
��BK�A%w8�`a��< w��=���=�!� p��i�A9> N{�&�1C6��B6���%�&��&�F�tr�{s�n|���b�)�IN�B#��brZLN��i��q�
�$��=@���brZ��NN�����brZ�P�����;��i19-�+-���6����2��y.*��+<%�E�h'�E��=���b@��}�����hH�}�~��~�H����Rr_������������I��5�CGu$492�5���7���2�~?�p����3crfL����193&g�����
��&������3crf�5,93z�������
D?Rg��% <��&�Z�<��+y�7RJ�wB������~'�;��	�N�����������~'�;����%���M$�;���)�����	|3c�0���*>���l�)�<�~[�g�������ev[�6��������l=��$��9���cXL���A~�����V��kE�o,�-��Z��{P��_&�{�	w�'N!e�Fc�"�%S�{��7XM��(��l��X
��&�������Z�|]�_����s�:j��4�#�Yr�����:�P�P�P��C��������V�O����&k�:��a������b��^m��f�BK4���<f{��G��=��w��/4�v���B�B�7���6>�eB��\hO.��|
y|z�H���#����,v�SP������!�sF�h�V��h�J��{�.��_�)��KL;����7VA_�5c��C}h�L�d�=���5'Ys�5������xI��(�������K(���s=-�����K�RJ�c��S��X5���	�G��cD�a����j��zc5lS�X��������0��6M��d`O�QS>}���[��`7�*�����j,b���D������nCt�O�\h'~ ��T 9�3UHE���E:�voKE?�������pWs*~�S���*�t�O��Z�;v���T~L*?n]x0>q���T~B*?i]x0~�Z�KR�KR������W���"��"��j]�M���ZjU{��c��
�J��"$�����pS�<���I��U��pR�\���,I��U��pR�<���>I��U��U��,T���r����L�%VT��$�P!U!E�2|i��E,�+_�E��Q{Z1gd��B����p�1�1`�uN%��J,/� ��X������W�fP�"E��y���TPv7��!���cB�^�� 1�Y��9!��B���+!t�ej��������d�A��|-���������+[��@����Z�7�Fx�����K�t�XQo�oPA����.���:���(L2�#�cRX+�@a������z��Ydl/,z���DO���P�)��E@os�����N��C+�):��� .�'�����
7X"�� ��q��]	q�%B��aB@b���!��(�.�����`6	p�u��f[�����F��rSiD���k���;C�R���iE�k.��w�`�Zjp��hpA\p
.�i�3�-58�V4xH<�<�����+����z���o��vU��@{_#�~�u��n���b
U���}>��������,W^�=\��\���D|��F�?=��GO��N������&e���j��
u�wU���p����I7W�t�G���y�����U�onr�^��c���0G{�����lo��0�W��X$�7�Ml�+�z,���&�Z��{(���pv+$�p�w�V�����
�dU.~�i����\J�d��;�������������;���o�/��������������~��F�_��n�M�m��Z�v����n�����O��
#�5G�x;���1�
q�S-Kl[4?#/��A�w|oB@�
��2�o���Aak�iF��<�]��?f��
�sK����Y%~���|��5����=�w$����g�,)h-�o6���C!�0��\Ve��(*d��r�2�h)G��?�������|t�F�����o9�8�����k�����������$���o��fJ�����6����D���-���
-��<lhn�^��\����|�j2{e�r�R��Z��@�c���$�{f�2���U��3���q���B� ������]��`������L�]�rusP�=YC����d�R$�.�-WuOJ6+��p��3���W��rS��j�&������R����Z�,6�H��d�2�P2l%����d�J�-[��a��a���jX��-����";#����VKa�V3��v4�EEw:h��C�vU�����H�7�b
�����S������-���!/�t����������?Ym9�=dRM��?{��%j�� �6�l�G��d�I�c��z��7�zc$��Xo�SD
A@}n�E��,I��������ST�k^��^���'(vM��v]yF���������"�� 7��������~8_N�X��M2�������
��\�/9}��Ok!�|CH�8)d>���N��=��$��3�w���)Ac~�:hrM�SwN�&�����@�5YC��!������{�C�l���h��i��n����
�����������s�D��!���wL��m`�� >w���������mo����S�~����&����%N���t8���t8C���t8S�w������e�V��O��5�����?�}�����C����i������U����������?�]��Kvz~����Y�����5�y4����7g����/��\�}��2��������_.�}�y������uO7+�1&Y.�~$�����S���5����?��c�����T�.�z
����s���C��?�r����v~�q�����CI����/����~�8����P`��u���y��Oo/���x���?x�����\U��X����"{��lU.>Vc�.9[l��I���B��0��?���	��A�����~����O�g���%H0�k����D���W�(�@!VI��S�M�� D,!�X:y��WG���-��F�3]U%���M���
��n���l^}�����������z�B���AG��w�������Zz#7����@�������h��zE&��a��f5C�:�?=}
�+�$'OI_.�kT��US��\nCS&��s�u�s/lQ������I�!��S4�i
�Q
FSE���w@`��ry�U/���J�s`��e4%}z��V�h�MJ�Zqv�l��6��[�h9����<m�IuU��.��i�j��5��4]h��
���f<�mxN����A�~��jV�g�UI��sT�{Np� ����i���z�6T��(�3���(�R�>w�6V�R(?3w���D-E�C�twj���E}k�q����0����Hmo��8a��@��~��M�����F	�_�_���/��G���������2���~a����XA��r<�i@�Q�������A���S5��p	��X�Z����WV���ReE]����l�)Z@>����=��_�f}4�Z��O�����xW|o�ml�~��������f�N�:x���m��Q��N�_9H��R������'t���ow�kwak��c4~0j�Y"lOA�c"
>���r�.����C�	����E$�U\��NKB���F'��9�	qP4$Bu�:��$�M��a4	�����@"�Q�����30;��B�;G���;2�s`�����;����;N�s�������;G���#)T�N�OJ�]	)m��PC��"J�]�C0��'�!�F��c�C0��G����e^]o��v�W��@T��&V�GL�Y�H�����M)��,d��g��]�>~��#"5hQ��FY����A:����u8��F3s����LNAT���C01au��Fm��*����87�A�JN!���xY�������dG��=&�<R��f��G�=�s�X��,�k����]�6�����:7�I���(���zr?�t�#\"��U}���OW�q�q������6�W�����w{��s�;h��h'���p
��<�����_P�/X�/(��"�z����M���+������8�F;A��SX��@���B,_��!��!��C��C���"���?�s�:Kk���U�����^�F;����"�4&��y�f@����#��#��G��G��D���d�~u����&�������v2dt@Nfc�� ������=��<z4���~��
G��9���`��o��s������R�o�po�z_�?Z�4�S��=8��q����
r���B���xD���_R�{4�"������scrn47,97&��=R��scrnT����s������I!y������|��`����N�p������R�99:&G����;!�����jI��YrtL����19:&G�������%%G���X�]rt|��	�N����1[�����G�>����f����m����`��E���3���H/���O�aa�qq@v������'�����0����jc{��SRo/(y�,���T�b�8��h����i[�a`ll������DZ� a�GM������P@Z(���x�me����9��R��D�m��b,�\�/��U4Gr�zR��ate��nf!�<�x'"������u�#��[��~�������{��I��?�g�'��3a�����8�z;����$Y4��|�s�\��h2�z�@��b8����^SV�O�kG2������%n�Pu���\�Nh�]�# ��������K�x����S�E�\��spa���S���p�G�d����Y��K���m4$W�)��+$u_���*):��zG�
���W���j�h�>,�@M%���{.�������>�����������,���zo������$���#t���hN��l����k����h�~�������e�!e�!��C�E�.:�E=]>:E�&5�-�a��.)��������m�.WQ�,:�,:�Xt$���/C�t����_F����b�����hm %5��g�vn*!��c-�yQ'��a�ts�!������
^�Y���������u�x5n��z����.��=g�nRXMi��S8!��6|���� 4'_L��x�#\-�;���x�����7�c��-h� �y�������?FYL|��+q��,=��|:���hF'�V�qq��i[��A�30'��y,2������^j������y4��J�G��NP�`2�"��b����(g�J"J�^�N����W��p|���=��m���M��������0��z���{@����A��/��;E���x���}����?�F�e���������s����S�z����R?�6�[����h�LU�������L���J�{S����@�=���g>�%�a{L3(f�pD�A�g��k,��K7�n�N�xY��z�
�i��#��C��n�":
G�0��o��"�#���P����uF��M�9�0�|��yhcKhCC��q�-3p�k��d��1��f�M\��tb���8.�f���5�O��]E�9�E���O�b�����j\��N�L�])F����.�+NA�����N�_��7�{��,�D_�9��x���TTh�)����*�����*�EJn����$���U�S�k��V��������|Wj�s
lm+&�a���b�JZ���gp��d�+�YJ��181jZz�-�f�X�0TyT���������V	[�j���:j����v8��9�Z�kU���DN��f�Z�jG����S����V��W07)=T����W�c`��s��@�zq �8�#`D�kc������#���u�����f�\����j�^#r�n���)�n���9�O���J�0R����3��]\�����)Y-��k�
�b�q��|3���.�M����������A����X}}�Zp���/�uw��5Lx�U(s#�M?���Y9��W��7�n�+��������>4����zp`���(B�����Y
�%K�����2�e��3�A���2��_�>���9��/]�S����d�~i�0
���m���*R���VB�E�`Y��;]�0o!������[���,z!�8�
�<����m�m9�^���"=�!/zC��
�EoH�!��
�E���0�&K��^�A3��o��hF��
~�
��,zC2�f���W�E/�(�?�l���a�^A������<���C����e6~aa�Z������-6���>�@��4Z%pW*Cb��S�&t���J7=���W�T��1�=r�RZ/z/�&����"���"l�-��0��n��HM��Wt��U�V9[�`��2�{0�d^]o�������5��A�(%c�%%cH2��;2C�1$C��J��di��dI���%cH2�$cH2�$c�S��1�N����b�2C�1�@eW���
�>D��8"�,�7��f?��."Uu�����#��#�0���dI��d��+F�a�m3�a$F<Z�#�0�#�0�#N-K�;
�6'����dI��]F�9��-#��$���4r���wck�_��QJ�KJ��d�wd2�$�H2����#�0���0�#-K��dI��dI���%���M����}ks�H��}�
�^���)7	��=���Vw{C-;dyw�.6I��H5I����������������GUfVVU��w��B�;Fz�H�u9F�����~~Dz�?$��Xm��<t���-��n��R�{e+\*���r����U�q�"���pe��DT�
D�u=�5X��I�B��G�����>����G�����bL�0P-'RA��b?j��@�<f�	�W%���(A.<��m�ZI>f�}�@V�P�,<���	��i���U4�t
7F�9��
�2J������j��g
=�H@c���>j���2j���4F�9����
��U������F���Y�4i����.��UQ���h���&\��h�>�>�rU>�E;�EYQ�5%#S#P$�&�R��
���vY�nlCw��-]
 ����I&q��H��2�mI|X�I��gh��Mmz�6bTe�u':��U������s�����s�����s����go�P���1m7df�z
55��������������{��~ox�
���]�Fox����U������h�������{��Pox�-��V���6������j?������B@M��&�����?���7������.���{�{oxW���9*��w��7���w%���7������.��w�������-��bwo���v3�.P_��$/����&�Z?�	�7��&��/����{|o�W����9*�&x��7��&x%�	�7��&��/����7���m���h��U������
�9��P������
�>��7��f�����{3�����������{3|o����"��7��f�����w�����u�����w`��Y�i'�R�����F�����{��~ot�����]�Fot����U������h�F�����{��Pot������nk����~oj���Ss������:���5B������t��f��#Bn��0p����'~V��	���svu��M8�
U���8Eo��X
'�����U\�S��\��v�4��] �8YiK����{6tX�7~����~S�7��u�
����;�M�������a�hF�fbs=��*5�C��E����+w~H[��A�'�F��z�rJ�����9n@rA=�Ez�����W������M�]��.G{��������w9���]3��h�����h�q���h�Y���O{������3���C�dP�~�W���\c��8Wv����L#�60=��)>.����Sd_���ea�3��r��r���0��q��O�"�����q9|����6>9�������S��gm|rR%>.����c>>�8>���`��1���8�S��g�H��*�����G�A#���q$���"��p�������c9��OS��]�\�,b�V%��4����`�v�M��`�g#�[
�!����#�����G���f3�`��V*R�$:�zDZ�(4�����f�"�L���G������u���:v�c����E��:��+OOW�����X�~�/F������c,�~�� �:���)'�ig���V��qP�a,4�g��T�j�m.[
���ye����������A{d��f�.Cv�����2�;	�3�{.G�.�K`��S��y����B����T��w�Q�u��/������V(N�l�"���P�G��R�����"$J��d}kd�d��u��I��ry/�hg������2���E�$F@�\V��F�$CF@�\��1�l]�Lrf��e�H3��1@L;�F@�\.�1IF�����"�K�{��>�4]�I��F�RQ�-�:��eXHG�Q}����*����W1b��"�����u�^����{]����?o� �&F�M�������o��J�B�D%����84�o���RQ/���{Ji�I�j\|����Y�d��su�8Y��\�%N��:Wg������Y�d-�\���=��<��1���uO��wX5�{,�U�^���zU���)�c����:�u��"�^WM�Kd�71@ob����j
�X�����(YwG��;*
�S-�{,��Y��� �d����
o����O���t�w���
�z�_�r|l���;��P������b7�X��+r���8���t��V*o���J��&'����a��f�(���$��,DJ�lf�KlAqQN�R�2�mXL�����������AiM9"E]�-m���2
��GO�.��B��ECG�.@C)��AC-����#����K���s����P
�e��Qr��T
PQ�%qQ)����E��Wk��fQ3��
+~_v�/������nJ���nj�c���$����v�)g����i��	N����ZVP�[�0�Mp����`u��p���	����^@�.��7�-�6~�tR
P���0�Mp����`����J�>�����Q#�!���U�^���5����z�;7�b�M�k��p�{�,���Y���j�CJ�}2NE�@1�Xg���0��<�������:��]7���5H���$�rRD�H��&�����k(���j�}��5C��fe�MU���������R]���%
������k�%e�E?@�H����o�Iwl�"U*(L�����*2&A�����J�I��#�wfT�p3	�Z�8-�T0��i��Hq��6��>��\%�{J-%UOx���8do�o|c�27��Mn�fo�����;an����������`;'?�7�1a;'?�7��vN~�otc:����
�}��s#�&7����p]������������������oL��'�7��vN~�o���Wvr��u$e��������l������>������fd��w��G��3�x����?(��?��n������+yw��lU�Y���'�@���3��c�]^XQ�,���s��^~;���3`�}r���c
�z��� ���I!��d&�R����#���.�����j���r��������"�Z�i��i�����
e�J�yN
�ILP_L�/�b�X_L�����b,�2��3]����>����h/�@	ylD��`Vec�7��^l�q�r�F8L�7�V�?�6�:F�~$�G�1������[11qf�g��rr~��W���18>??����"�������Wj1�	e��V*~rmBzA+�d�CW`�e�����������}�c_�����}�45t+��
�M������+@7�b�F�W��.����B��@y�}�����n��[�2e���4��Q/oW���������1��s�4ZG��HT��k��"J.SL���:n6�v�"���F���I#���c�;x�������*��7�L�$��o2O�W���N��(�P�`)��C���8�>��C����8B}B!b}�>b}�����
;��Sz�X4�vf=��`���	�5l7O�����a<���i���b��t~�������6�/�����R�fCB
��<��e�]?�y��+�NB���F���O�}� ������+~.�#���v��� �u����y*3�����SI\��1@���SI����b���$��T�����Tg`������y*��/U�u@T*��q.�(./|Z>A[q��GA-u��R���S��A�S~m)�@��
9eWhK��Sf�;r��>�ZJ�-E(��!�������RB��9r
�>���99�dz�{z��yL�p����������]������.[�|���������I�GP��Y���V,f���
um�����;?}<w>���&������]W"���t9��q�B�����������$?����^M��Oj�U���5���A��?�t\)�(�l���E��t"23Q�4.!"���L�1������Bt��2	:�Es&jZ���)V�$�����t):�N��'A���D�K��Bt�C	:~����	������y��f?�;�"Q=��B�����u���DMM�c���J����J�"�Au�]	��83#QS'A]�X�`7N�HT��)DPW�� (���ee(wA��
P�&"K�n'm�Kfh\����S��|�����7����������Wz�Jo^��+�y�7��A�7�H��+�y%����C�ho_�:Sb��f�j�U������p���Ss3s�}9��#���rDL����D��tr������2B�2u3zBZ��=9{h��k�����	!�������������`�9?��r~���
������d�z�2~����x����y�.���������������h���3�p�?r�xR����sz|�����O���?����c�S�����5]�I���K������#�	�z���_?\�iz����� �Ek4����|��P�pU��F����%N��/6C�
��$��0c�W1�;/�H���{)��������o�'Gl���6{&�j_�<3reg����5�x�)�Z�4�p��z	/%��&w�w����4��\AX�m�?����:
�N��-������VXl�������'���������87���� �*����b&�pv�Z����=���E2����g���2.E������g�75�����u��F�LQ�Q����"8����|���)����N�������M� �A7b��*c��'�ch�7������
��Y��3�5�>�W%��3���G�d�����A�A��>S
1�>��"���X����R��`�����f,*�)�2�����P�rlI�bC��U9(j����6�}7n���w�4C�Jg�g���i�K2���^`��[|��4K"�����I�#
��L�%3N��J���QT�+.c���\�3Ls�tA5��BLKH�Ar���VbU������$����`L����r�i2�M�U�Q�lUcO���=�K�S-dG\�z�'U_���b��#2����#�
��=x��H<bCq�7M���<Gb��'��;D��C�<)>3F�lb���[�}�F	��'nk�����X���5�.NPT��g�T��L\�J��,���6E�D�O���0X^�J��b
PY9D��^��(�\��h��;Eu�2^�\�TTIn��U�Kn8r����%��
K:3����%�r����A�4,cnX�
	�����X�N����=�4�U	'5�&��r��p�_#��5���/�m�=��7��{�	�+=�����6�	{ozp@��+���7���������W���!{o���
�p�|
Zq�{�)��sA����z��F�A����������!���-�o
0���g4�cp\��t��M@�������tq�8����������&`4'�I��7�����xr�
�?���.������Q�N^���M8��'��WpNOu�`l'.\Q�_�S����<��{c;��3�y
��\�H��f���)��;���hVo�N���F���C�;[Hk��`�O�
�r���b��0�.t�?0��1��0��y�`�w&p��c�r���J�1�k�&\{ ��{�#!FS�&�{`����3~�M��B����_C��<�zN�|x
W��yj8���qh����_.��F�0��p_��C�
��g��.�S�@�3P�������

W�k���z��=p��+���H�j�q���=�p%����p/�������@
 �p��u����ty��SH3�=P&:��\1��
g��!\M�
w�\�F�{ ��u���se}(�1r]���
���x
�w�-'�A�Jdp��+
�����Pn��AzB��q2(	@		�=`��d�t��{���?@38o!]��2������@:���0�V<0�<NZ�p�Y�	�JAY
�uPj�t�'\�{(9N������]�����E��X���O�1q�����
���F��3���t����g�O��_�f����b!*A�X��ETb4���^
y��AW<�J&Cflkd���1)}A�t*��Qcvr�rYY9��T)�mfR�T��Ch^P6�H��#����/-><��������@�{/1eq����}� ��n*�w*<s�� �}�%
4�����A^|�$�N�'�;�����D9��S1��N�O�{/Q�������}�%�'�����������}�%�"�;���T ��Sa���KL���^��s�{�R�}��*����������}�%&J�;G������}c!���mg�0�"/�b���fq�n��(��O���S�dO]^�z4/2�-����~�����i]�,���������9��l�8,���\?$L�V&k�A���:�aF�2�:
 3���x��<eXS����bM����DEym<*�S���,�UTV�����id_#��N|�^fAdb�b3b|�7����d�=K#������)�0����Qj����)}����#W�������#OQ��M��y8���fpHc2h4�x�S����������� �
�1L�6�$��tX�^��FM�`3�T�~in�$i��������J���hH#��r�Y���HY>��~���u���'��17�9sx�Z�0����aN_��.8�dy�����UT?
7�aq<:�w�}��Y���K�R.������w�A�H.�J%�p(�R�B^�Oi.���\5��G��>��5�O���9��$o���T�,*�V�����Kf��s6C�������\��^��T���	�u��b��|j�5��G*>?R�9N�%o�G#O*�����Q������p
�,L*�Z-&�>z�RZ�k�?�5�dg8s��$���V��P�p��VS�zd����J�=�Y���-��l ~�,��<�Y��v*���F_�����p;�1�s�;�\E
�L0�^f���8Q�9�����p+FR��8��r���H�n�
MdJ���qA��J������A�����QMt[��o�����M��?��5���:P�q`Ov`%CY�R���S���]�@��[����8��E?Pf�>���M��s�� ��rO�_�R��a}����\����n�0�6JF��s�2	zE��$���3w�"*���\����6��5E����d����K��P����&s>���cy����rC��V����4���
�����0f�(�����8�����nX���#�b�23�5K��	9��V��m*��2��~s��Dg��Ke��NS��x���K�K���W��x���K%��Ne��x���K�3��"�".���u�F
���}�W�}����b��)���Hm�j^�����O4~�������t�����'�$!�N�g��
����~�����q�!]���]���X�h�Y�����8Bb�~�z��H|<�4���#��P�1���������B���S�����������f�Tp�VV;��V�
�T���P���cV��C��!tUn
U=	t�
��(�1���|�x*O�*F>���Q�'S�E*������.[^i��X&�jU���P��485������������V���mY������j����u=��AZ�"����u��^;wh���z'�;��B�V��od�<8�8���H�nw3�X��+�cI�����\d"������������,A�mlz}A��L���i@��`�9���	�����+��q��]L��|L�z��h�~t
WK�o�GH����~���l�!����/d�LF��w����*��	����m��%iYb�v[�}��B���G���2��Y�YIW��.��\D�|t�q19�H����dz�l�^"���*����~1o���3��~L�n,�e���j,�\�&\�B.cW����^$�IU.�J��$���2H�by������*@A?��g�Cu:w�#���+,�
�8�|���#e������HbXu��jJ�y5���.����w<��'�3��L�;�����������+������OR�Z1����LJ@� �\hn4��d�:�^�pk�����������<������EFG�������E4.%��Ef=\�f���	���\��49��� MJ���)W�������H@���f-<��{���i���(�b�����4E}M��R��	%��;�P���x��yT���=���8�-J|��k8��;�*q	��_�����%^��9��h���{01G�&�O�'��y����x������gG<������U����������W����	�yu;N�~�����r�-�Z�n���_	�d����
}*�hLLw�t����3�{��E� �C��V�3-���t���<K�44�ge�ey���F��X�~�S��7
�RA^�n�
Xt�	��Lt��T�Yn%k��)�c�k�'9syBM�$G����^���3�i�������~|wL���sr��98��o���j�#���q�����G
�Ax����Q8W��I����
�������_��
�p2�C������/����s�#o��yA���$�EW�|��c��ut����� v�:���~/��;�/��A�7N�
ii��p@�����y�m�<��)	x��,W��vq�{����R��1�x�	���m4s���{�a���a�W������_������l�N[��7��
u��y��k�Y�~x��];����z��~.g�;������|_���I��z��o��z�D����_�1��9n��J5����.mh���"����v��.�����7�f�f���:H��l#�y�r|=��Kf�����Q��R~wxZ��#4���{�g9\~�
�M�$RJ<�}g�t6w�������U��"r}J�o�
�����
	�p���>���hvs�I�/�^����������8��.q�������Gw�c�����sq���yw���3�BH����L�K���K�������|MD��� `X<@3�kZ����_����0���[��n�F-}�C_d���N�#	}����L����k����
7e+��R6�,	���m�z��3�"��w�~���C���-��0� �|�Y�cNZp^bE��	;�_p��i�.��C�C���Z��f�lF������{_����*���b3D*/�3s�b�W���`%����������
,Ef��6�kR����������t�������V<�V�������W�b2��Li;�>��_8��5�Flp����%&����O����Y�z uM��u����h�L��b��5{|v|���C!�~"�	W\��wD������\�!%��y���H�E���s%�'���[�/���~�5@���� 3`CL�/9*F���6�o<�]3&��
�+c"���\��KZ�_�����vxf�����De�����9�]�M]�M]���R&r�E��Y�tr���N^Q�Bc`�z��������t�e��N�d��Z�(��^#���r�troK-�9���j8���m$��`l4:&cs�4
�C�v��2,C�!?}9=-r��2�$Nx�Ci�n9�)oH&e>���[f�G�H�0�HC�
,�����X�Xc�r]�����M��{���#�w�Hs�]b]���k�D0�� 1��t�<�}��WS�}��O�
5�F��7�.�P�A�"��eT
���3��2,M�a	2L��q��3}�{jZ�z|J2����7g9'�Fd5XE�R�a��1�������X��
k;�g������LS���E'>�#S�#�07�7�Te��� �8G
��h������&�.��n��[m�f2^�Z��'R��}`f��J�0+ /�3z����l�G�WfE�vU��j�M��C$<�\bl���:-�Y����GbI��P0��Bi�Y��t�n��M�4DI��X����3{0��8�o_�H�#P7���4�A�R�{��%����U����������h�@/�1�(!�� ��b�Y
'��*Vs������+g5iK��RV��UMQ��<��\��\	��Vs[d5W�jn�jv���Q�3�6�lgEj�=�b��@�� �9r\H8�k�Ak=@�K�D2��� J����o�w')�Ti�ca�S@_n�������oI��Ak3g��]	��{j����}���}�D�)����F+A
�VzDv���}���P�.��q��mg�N�a�����:��E�9�u��:��	�q����R3�����Z�<k������\��LM�N��~nK����"z��=�JR�D$^��jE�\Q-��j�6��������)��,I	D���4��F�������6!W�	��
D���Qy�kb8�
&�L��Q�����nW�
��6\J��C���80��g�`��d�s���sC�6�x�4~�<��b����*�KKb��Y��X��eS�?��&9����>�����{���97�s6���K�����������x�e�s��y�s��9��b��G���s��9�}�S<G#*.�+7�_��,/�h�-���y�����[�JeX)d`5��/=L�����[�V��}�w�<	D:��a��u��W�cd�~}"���h�������>;�HG�QY���XR����_|��n1��&;����O���	��%�|���'{�w=15��cm&'1��H�5�U������Na���<C�^�66t������T��g*JZL�Ir���2g�^��(i5m'�&:����:�������d|�b���"{�^��h��L�"zC�*(�D�<'�rN���������js�[�e�����D�D�ENt����h'E��3_w�H
a����AB�3�N��8P����~�D���n������F�(��JyV�~��K�?z�^:�9{�Iz��K���Xv�M�������/3q�
e�u�F����o���O��N��1��������q��4��������ZE��q;����cx����� �l�R@I�/�a��
aak'|`��l(=���Y��$�{�F+s���E�f\'9�C�����1�:^�7G�q.��&q.-p��H�0=��.���nO(���b�p���������D'�U{B�>N��RR\E���b))��l�l���� �9������Y%C �&��m�a	d��~����)�������q0���L�g1����������C�n�N!>s~>���S,���X�9�����R,���Y<�rG��/����iz�5|Ct�Q������?D#����Y\����4��d������MD����q�w��J���X�'�i�V�F����� Vw���0�?T#	�6x�fl*�f����	��/}�(QY�k^���������*����W"$�edD���fd���g�A����\����y�1L�M��nfYs�e�L�x���t��Rp"���=���bP�)+�����X�
�17U�85/�[3�JM�3\\��"7���`�������d"������`����
*���?���2b �Iw��m���6`28�����D�����b��,bP�I&���Ub�����C������!�(W��HZ�[��F����Y>�o�
*]$A�� �-J�o��'-��3fNQ�;s>*��nJ�R;I��M�� �]��$�R9`^������Dk�<�)�s�:�C�-��*������`{�`{�������yR�h2@�/��D����({����|��-y�Q��0��J�*8fRi�U����d0'���d��&x|��iM#g�/��*p���p\;s*��O;+q�����M�bD�
�&�D���h�������=���c/K'�#�
�3[.�������`�-@�oF������I�Jj�yi?�(����@�RG4?$�9�K4��?"?���ha�aAba�L�8���iD�R/�����`��V��I�R�X������V=qa�"|��%���Es~a�}�L�pz0Yz��4KU�E��p)J0���x.���Z
����e��N��`��
:V\�(\�������[�Mj� )�a�5I�x��2U14LU�+g�\���O�j��%p��VG���.������H���v��+���D=U��,�RB �II"���](I��t4�GOIY��\�[Ch(\����$4�!�e��B������n;Y}�����Pa�0\>)����5_>\��!m����*�����*�u�:��+_>�$)^>����\>�������� ��E�P�����Y�n(\������/�|�p%��+_>�v�W�|�����/uyY�R�p�	�a�;o�8R��V@1 Du�@�}����Q�5��8@H7B�p�Z�$������������7��bJ~7@�����V�=�|B������lZz�jE�e��C�� ���;�G�1�*�U\@M�|��f�0~E�)IE�6�#&%���yiV��s�B�*6
���VTcHH�h��G=E����B���I�&QT�$�t�Qr">�#��/9�����W����"�1?+���mg?d�8����1Z9���.:'�/�>�<t�o���"��WY)��\ye�FeW^�A��iAieE�fEIlVv�T'�+;��������beWQ���+;�Qm�>19��&:��+�](�B�RM�m���Eja����L�*$�e%wg�Q�7Pt��ijRill��4�r�um�;W�&��
2��L�#���pZ�����T��!��$)j
eV7���n�w�������MQ�	_e��:F-C�]��
�����e��`T(�����N
�=q{���p����
2��@��w~b�&���(�=!�/�v�g]������Z!���t[��u��.����m8�7�6��&/����� %(S\P��t�<�}���WS�}
.Q��W��B���G=���������[9��e�=������^�w!�\ad�8C���9���M:��;�����A�����c�h+������7`�vZ����������:��������K�k��e�L4�N`Wi�Pm��
5�suDd�4+�j&Y*����H�������b�:JC���,b;�:=1,�#55��$R+���e�����420�56��,i�=��dMV4��o���f=[r���a$���*%�u4m��	Y��5	������i�U���1�
a$1jF�|�ym�`�5�q#)��X�5�jn
�q�}WX�$v]���_A��b	�8��"~��m8����k��e���5	J�����3w���b:��T�������W�4�_��/����,���IRaw���4�E�����2�Co8N��G�����_p�
N`���WV[��/$?!M`�^.��J�H��,�%��j�L���d~��L&W���m��r����c�H�A��A~1��������N�Hz���?�g

������%���6&e*�����/p/R��$�pOK��~h:34�5�#�|������������������6.	K�g\7)r0_\��z�/���g�O���	f��)@�J�9�}�U?���y��\�_W�������u�~]u���vB�S�������\��<��'�O��i�c�������*���,f0���|#������,��py����^|t�77G���q�\>d7��6��6�Q�O�����O��*R�'m�X����-wNdd$�u���{���z���:������^p���'yyXl0}�W�}�n���j���f.�l ����:��}�C��oN���_6���D��8��'�
�z����.�[3�JI���[������������l�8���>��X,���a�[�\.^����2�=f�U�i�Pke��G��3+�z�VC*�g�t(�u���6R��R8�f��R*��>V��sk���"l���'���x��� |X����(0�S��J����4���v�1"6v�i��1>z!��p@O|��\rAA�����z��������h�#���GK�L�Z�;95�Hq�cB������g�=���W4&���l��67��8[%C�'���+bW����%zY��9����iu�X
N���DM�Mf������+��1j53ob�Q��.!i@�X�Q��	����;���&�W-��jU��dFf��n�+�*��U+;5�we�z����0G��.t���HiT�+��i����\�J������G�uc���h�&%�n����P��j�.-p=�|j����C<�:��&&9k��-���������
9u�>
7��0��N8GwS?z^<��-g��_���e'Z�d�A�Fd�X���N�;t�zTt�/�����x`�������{����6�������o^�
�H[�?9��'���J���2�$�����i�Jy� ��� ������1!���^��sW8��+\�$���J~Y(*��[kP���[�h-����Ev;%����u����n}���\W�����\W�-����|]UP����*�U����*�Uu����+_WT�����u�-����uU�b���ZYW=a]u�u�����|]u���+YW]�����UW����u�mw]u����v�G�F�����I[���RU�kp�P��ua����/�Dcg5���7��B��. n�1���]@p�h�5�b�]���R�is���	��@q���Y>`G�o6�Bpc��.�
 n�D�T���z�I���S�#��@�;���WJK�>�	�q���,���PW���f$t��Y�4Q�~�&���$Q�
}Ds9�<�h~���HE�FF.��.zJc��oZ#$�G��=���������-��H�j0�Et�&��>N��f�u�4����eI��m�muk������$���j�|/]K�e�WvA��{���|gqo��h���Q�58�r�^� Nm��v�L�����R�����p���,�\�k����3��t��L������0����������
sT��&��=����	Xk�����cc/k������&$��@�5���Fn'���l�m;A���e��W���z�����������`�\�o��B����?B�f���u��{A�[����8�e���<�i������g���;I�FO0�K���B��h�\�q����j�2NqHV�Hp�����~!��n���'����?�
�g9���p2�b��"'�������$����yz�Y3���y�V�w���w���w�{����$�kvn<�2���J��������GX]�A�p�������Y���5�e]���%���:rR�@�`�����WO�D�����A���;������������g�5�#��kg��8	��t�Ct����A�������|<??9=&�����������g8��_�d��u��3�5������/�?���s�2�I\��*o�d,<����`#C!R���F#�#������q���J�XQd�Z7��A
��_�Kr����`���8v��j����g��j��}����b�I��(`Q��pf,	��q�!��t}����#�-�Vr%a`z�mr	�#���
d��F%zK;(a��&�2��qRI�d�H���8�r=�\d��O/���������[���i����^.�����l��������8�]��E�SV����]�6�98,�M�������4N��=���)�9���Q]��j%5������D&���!��^�E=P��/�^�����>��,�#�g�_8bay�����}*3���1�t�<��wA�(
����CV����x������6�@1mPS3�|�-��uH�I���v3y9c��m��d�b��:��<�Q�e��f��w��c���e�A[[��D�����9��p��b3w��_��%�$x���*�@>f�q�����
����H�#�����MMIS��5��x^�����gA�:��1�s���hG"�Qi�#����R���Yp,6����\�G��$�����&�UP��,g��A2s��g�H2���o1�?�������)���.?1G��&�M1���|f�^a�}&~���x����W�ebF�)>��q>��bz��{/V��;vi������*�Wr��h�a���Wp���eF�t��uT�vToP�����P^N���`�����?@� I��������c�DW�Q���?L���.�	/�G�s`����7D�5p<�B�+qd����u|�u����$�9G����iW|z|){%�����v��bK�����Y�_"���������a8Y;i�@��!h0b�CG��Sm�(�>�Bb 1�������'
n��ID��Ix	Tj�@��d_�T���� j��#@��IXn��� �x��@Q�0Jh�2<(j�O�����:��0%9@����j[(���0��j��I7U	 ��D�V��4r�A�569�zLeG��d����Pja��f��1dbmK_i����X���k�
��������7�[m�Q{7����H7���	��^r~5U�w�&:�4�9p�������4�e�f�w'E�G�;y��a<YF�Xy�OD��a�2jY��r��>%��p�fYO*����W�ER�'1�`�a�� VJ�*��������d�Y��d�������@.�U�zXE�y��il�aj�a���[�@2��[��=2e��M�`5�=��K����<�C����j�yJ�l��/��o�����6�@Y#k��e�9�e*��:69�A,��4�ku��6���Kr[
���(��������l����=�8�Y;�Z����X�:k��'�.[\�8�a���,[���W��`9&a�^c�&f��a��g��W#�	���r���C!���t�k~[2�Gh����g����/@c/E�)�.!D���ze��K0�t�c!����"9�d�D�1�l���h��xy<��������8r�Z�%�u�l��5R��O��ab���7�=Z�����<�G��N�]��CO��^����>�]?���Y��&��t��\��Z�������~�U\<�;�w>���AV�����}X;E������REB3,��e���3��	�d\mLrP����������_B*%�g�_E�U�t(��b�
v����v���}����U�"P��������k,��<�CP��8Yu\Uc`���HP(��:�H����oH����]�������c�����z��g!��l������h�v������O�}�  ��^�p�������w����&�)�����$���m�T���_����wWO�����)�a�p��0;5�_>�G\��(��u}S@�	kF�5�`�v���32q������f���r��Hx��\K�Qw?�\���AkE��W���<���������;RT����� ��������gD��^*�!�i�����#�'?�h����]��=;�	�������^�DI��9����#|=��?8��g?9����p��#<�F2�.��H��#������������O��������n���NFI�G���o�$1����
2����B?�A����v��.e)���_�B�*�`�����@�)�&TTg��R���P�@�	��:�>�I]n�+��{�]A���rI���	����D���f��B!GA�%8R���^�G`#	��p[J���3 �=C�mIZ���.�������8����C-�?�"���-C��z����4X��pb���r���2���f9��	�w*��alZt���y�Yb�n�R�\A�LK����r[�>�����f�U/lp)�[]��-c�Rn�;���~��_~q�JPW/o %pN}�fF�.j���G�\��T7�	�t{������������]�����Zh_>Q��z�+`a�pB����J'S`&�����
-�b��D!���G���B���b+�r������ ����u,w��@�����P<��HGy�
Hf8{F��\g%�x0e����]+�6�xm\�9�FNN�hJL>��6 m��J���s���%�K�6���D=&�&�5���QvS��r
�'Ju���L��msi�������u�m�L�<�m�h�^�K]�
+����e�kdZ�������e���
�U,�$���U��A#LZGq
i�NwzA��7A	���eu90��1Y��piA�P���(��5��N��G]����2d;A�v�m�����x��M��l�������]����"z|Z�&�G�u���Q2�c����\s�$����F��"��l k��|I�|�E
�4_���Un/�^��	[*�bb+����(�41�<��lv��(�1�m�����7Ze&�I���*m�VGM�\Nj��5Vf������r:���b�]UW�Q@w8u
��Ae�������r-M�^R��5�p�-�y��I8���/����S�������j0�.)�bT�a��p3��Ax@^U���rCU�r�/���������}�_d<4Y��Od�:C��@���{j�|K-e��T�J��p��|��j��3��)��;-��A�#��Ky�����_�L�1�dK�e`���xS�eJ��m��e/�3�Of�i��c/r09�������7��J{��,����:f�Wr6����q�t$[������!���&��}�H�Y�GB]��w�Wz]'7��8��IdbX�G.pj����\�l��2�zA ��s���`����n"E����b�Q`�/I�"��)8���vph7h�K�J*����
2�^i����NK������-t�Kxs@���9�F�����K@�����E�)w�;R��]��v�)W�t�GBty�RUU��1�7nN�V�(�6nR5���j��@��a�v��Hl���el��;�ZS����>Qj(���.2�o���F���Phwo�(����q����%b�Qka"��0��V<kC5lI����t�w�6���%m|������-�]�7��N����mI�%�|-�n�����}3��t��	�np��|���a{��]��zjuf�	�Ox���Jd{������{2=����+���*�����l�S���J�>�9�@��d�(}i�D&���`�I������&M���� �Qw5���<-�::�[�E�5YM8�#��^����z���9i�Q���K5�&��%��H���[G������9�f��!�R��f=,e��nA��F��b�V�6���e��.���,O��G�y����nRP&��6uT�-�����b�`M����.V����mQ�x�W�!���*M�	a��"�YIM�m�x���R�e�5�Y���h�Y�z���9�����s��\���f	�p��(+@��R��Ln�����
���4�C�b: l�P6(a�A	7�9������}t+���W*V����_q��Y�)yq� �������� ����
�-��N��s��6���	0`��H@��t���vV�������\m����y�����@�NQO�
��P�k�1��$I�~GF�']%o�
w� 	�xu������!�	W���W���\`�m�"`CC��O��W����u��x������gw2��	�J4,VU��*��w'��^\1�9E�;�����U�
*�O���u|�$�E����a�C.�66�9WNvM[��~�����S#b�Q?WW���z��\��h
dj[����W�:��*�o5V�k�CbDRjD�%�Q�2��z��:
*���������4�s�u�f�y3���~at=������fO�&U���3�J�-=&����jO�&W�����F�=.��$��fO��V�n��3�n9�V����tF��t�I�C�Z��&M(m��U�����N����ZZ���������[\j����
�;��l/)wge�F\]����J��G�o���t�����K2N-�L���)U�5��S�_a
�`���:�P�%���������0�/&�B�A}�
���Vg�^�7Vz��v�����n����r�m�v�:=P�@
�N���k,������WU�u��P��Ctgc�[O����0�3�0������(��e`[K��q(����o�g7���+'{T?�a��m���v����d�<v�h�g,� ������iq����S��E���arA����e�M���!O�C(f�zH�-��a,M��x��i��b:'Wl?.�U2���68�b{�Ar� �����1	:�r���[D������n���7+<v��P�D������`��u��`1[F�
���s��^�/�c�O�������_�����cp] ����<f<���PK����1"��B���/=�4~��2�������o�*����=>�<��Cfx0W�����g�8��<����K��G]������h��#�q;���3d:I %]#�~�@�q�<��G56l�C���.i�4�!s��I��������LGYL�_��@�h��<�0r��E��3O��N��d�������)#.~����#�G��N�~���lC|K��{�QBM�`3 ��ev������L�[G[j��x�ZL	���BB'7Y��>��M�aq$%+��Kl{=o�h��g���t3C�0�i�� 1g�F����m����u�����.��p�&������?��O�3H�"�~F�J��?<�#��&��!	�q����=w�Y�����x�Y�_����N�>�������Cl����� �\�<��ny�v����0\�w�����_���6��6�Z�7��q���3f��,p���_.����xH�`
b��~t�i�w�����x������!��;��4��8X �@2�(4���g��|},��C"�
��	�;���
1�9�$j9#�JwG�7m/Z�=$���_�z���to������O��?�������v������W�
#173Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Ashutosh Bapat (#172)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Aug 8, 2017 at 8:51 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Updated patches attached.

Hi,

I started reviewing this. It's nicely commented, but it's also very
complicated, and it's going to take me several rounds to understand
what all the parts do, but here's some assorted feedback after reading
some parts of the patches, some tinkering and quite a bit of time
spent trying to break it (so far unsuccessfully).

On my computer it took ~1.5 seconds to plan a 1000 partition join,
~7.1 seconds to plan a 2000 partition join, and ~50 seconds to plan a
4000 partition join. I poked around in a profiler a bit and saw that
for the 2000 partition case I spent almost half the time in
create_plan->...->prepare_sort_from_pathkeys->find_ec_member_for_tle,
and about half of that was in bms_is_subset. The other half the time
was in query_planner->make_one_rel which spent 2/3 of its time in
set_rel_size->add_child_rel_equivalences->bms_overlap and the other
1/3 in standard_join_search.

One micro-optimisation opportunity I noticed was in those
bms_is_subset and bms_overlap calls. The Bitmapsets don't tend to
have trailing words but often have hundreds of empty leading words.
If I hack bitmapset.{c,h} so that it tracks first_non_empty_wordnum
and then adjust bms_is_subset and bms_overlap so that they start their
searches at Min(a->first_non_empty_wordnum,
b->first_non_empty_wordnum) then the planning time improves
measurably:

1000 partitions: ~1.5s -> 1.3s
2000 partitions: ~7.1s -> 5.8s
4000 partitions: ~50s -> ~44s

When using list-based partitions, it must be possible to omit the part
of a join key that is implied by the partition because the partition
has only one list value. For example, if I create a two level
hierarchy with one partition per US state and then time-based range
partitions under that, the state part of this merge condition is
redundant:

Merge Cond: ((sales_wy_2017_10.state =
purchases_wy_2017_10.state) AND (sales_wy_2017_10.created =
purchases_wy_2017_10.created))

0003-Refactor-partition_bounds_equal-to-be-used-without-P.patch

-partition_bounds_equal(PartitionKey key,
+partition_bounds_equal(int partnatts, int16 *parttyplen, bool *parttypbyval,
                                           PartitionBoundInfo b1,
PartitionBoundInfo b2)

I wonder is there any value in creating a struct to represent the
common part of PartitionKey and PartitionScheme that functions like
this and others need? Maybe not. Perhaps you didn't want to make
PartitionKey contain a PartitionScheme because then you'd lose the
property that every pointer to PartitionScheme in the system must be a
pointer to an interned (canonical) PartitionScheme, so it's always
safe to compare pointers to test for equality?

0005-Canonical-partition-scheme.patch:

+/*
+ * get_relation_partition_info
+ *
+ * Retrieves partitioning information for a given relation.
+ *
+ * For a partitioned table it sets partitioning scheme, partition key
+ * expressions, number of partitions and OIDs of partitions in the given
+ * RelOptInfo.
+ */
+static void
+get_relation_partition_info(PlannerInfo *root, RelOptInfo *rel,
+                                                       Relation relation)

Would this be better called "set_relation_partition_info"? It doesn't
really "retrieve" the information, it "installs" it.

+{
+       PartitionDesc part_desc;
+
+       /* No partitioning information for an unpartitioned relation. */
+       if (relation->rd_rel->relkind != RELKIND_PARTITIONED_TABLE ||
+               !(rel->part_scheme = find_partition_scheme(root, relation)))
+               return;

Here and elsewhere you use the idiom !(foo = bar), which is perfectly
good C in my book but I understand the project convention is to avoid
implicit pointer->boolean conversion and to prefer expressions like
(foo = bar) != NULL and there is certainly a lot more code like that.

0007-Partition-wise-join-implementation.patch

+ {"enable_partition_wise_join", PGC_USERSET, QUERY_TUNING_METHOD,

This GUC should appear in postgresql.conf.sample.

I'm chewing on 0007. More soon.

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

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

#174Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Thomas Munro (#173)
2 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Aug 10, 2017 at 1:39 AM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

On my computer it took ~1.5 seconds to plan a 1000 partition join,
~7.1 seconds to plan a 2000 partition join, and ~50 seconds to plan a
4000 partition join. I poked around in a profiler a bit and saw that
for the 2000 partition case I spent almost half the time in
create_plan->...->prepare_sort_from_pathkeys->find_ec_member_for_tle,
and about half of that was in bms_is_subset. The other half the time
was in query_planner->make_one_rel which spent 2/3 of its time in
set_rel_size->add_child_rel_equivalences->bms_overlap and the other
1/3 in standard_join_search.

Ashutosh asked me how I did that. Please see attached. I was
explaining simple joins like SELECT * FROM foofoo JOIN barbar USING
(a, b). Here also is the experimental hack I tried when I saw
bitmapset.c eating my CPU.

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

Attachments:

bitmapset-track-leading-empty-space.patchapplication/octet-stream; name=bitmapset-track-leading-empty-space.patchDownload
From d2ea8d719f1981937f3205150b0870d38499fabc Mon Sep 17 00:00:00 2001
From: Thomas Munro <thomas.munro@enterprisedb.com>
Date: Thu, 10 Aug 2017 15:40:30 +1200
Subject: [PATCH] Teach Bitmapset to skip leading zeroes in some cases.

Some of the algorithms in bitmapset.c can avoid scanning most of their words
in the not-uncommon case of singleton or fortunately organized sparse sets,
if we keep track of the first occupied word.

* Quick hack, not for commit! *

Author: Thomas Munro
---
 src/backend/nodes/bitmapset.c | 22 ++++++++++++++++------
 src/include/nodes/bitmapset.h |  1 +
 2 files changed, 17 insertions(+), 6 deletions(-)

diff --git a/src/backend/nodes/bitmapset.c b/src/backend/nodes/bitmapset.c
index bf8545d4378..8bf8271f88f 100644
--- a/src/backend/nodes/bitmapset.c
+++ b/src/backend/nodes/bitmapset.c
@@ -187,6 +187,7 @@ bms_make_singleton(int x)
 	wordnum = WORDNUM(x);
 	bitnum = BITNUM(x);
 	result = (Bitmapset *) palloc0(BITMAPSET_SIZE(wordnum + 1));
+	result->first_non_empty_wordnum = wordnum;
 	result->nwords = wordnum + 1;
 	result->words[wordnum] = ((bitmapword) 1 << bitnum);
 	return result;
@@ -310,6 +311,7 @@ bms_is_subset(const Bitmapset *a, const Bitmapset *b)
 	int			shortlen;
 	int			longlen;
 	int			i;
+	int			start;
 
 	/* Handle cases where either input is NULL */
 	if (a == NULL)
@@ -318,7 +320,8 @@ bms_is_subset(const Bitmapset *a, const Bitmapset *b)
 		return bms_is_empty(a);
 	/* Check common words */
 	shortlen = Min(a->nwords, b->nwords);
-	for (i = 0; i < shortlen; i++)
+	start = Min(a->first_non_empty_wordnum, b->first_non_empty_wordnum);
+	for (i = start; i < shortlen; i++)
 	{
 		if ((a->words[i] & ~b->words[i]) != 0)
 			return false;
@@ -444,13 +447,15 @@ bms_overlap(const Bitmapset *a, const Bitmapset *b)
 {
 	int			shortlen;
 	int			i;
+	int			start;
 
 	/* Handle cases where either input is NULL */
 	if (a == NULL || b == NULL)
 		return false;
 	/* Check words in common */
 	shortlen = Min(a->nwords, b->nwords);
-	for (i = 0; i < shortlen; i++)
+	start = Max(a->first_non_empty_wordnum, b->first_non_empty_wordnum);
+	for (i = start; i < shortlen; i++)
 	{
 		if ((a->words[i] & b->words[i]) != 0)
 			return true;
@@ -611,7 +616,7 @@ bms_num_members(const Bitmapset *a)
 	if (a == NULL)
 		return 0;
 	nwords = a->nwords;
-	for (wordnum = 0; wordnum < nwords; wordnum++)
+	for (wordnum = a->first_non_empty_wordnum; wordnum < nwords; wordnum++)
 	{
 		bitmapword	w = a->words[wordnum];
 
@@ -640,7 +645,7 @@ bms_membership(const Bitmapset *a)
 	if (a == NULL)
 		return BMS_EMPTY_SET;
 	nwords = a->nwords;
-	for (wordnum = 0; wordnum < nwords; wordnum++)
+	for (wordnum = a->first_non_empty_wordnum; wordnum < nwords; wordnum++)
 	{
 		bitmapword	w = a->words[wordnum];
 
@@ -668,7 +673,7 @@ bms_is_empty(const Bitmapset *a)
 	if (a == NULL)
 		return true;
 	nwords = a->nwords;
-	for (wordnum = 0; wordnum < nwords; wordnum++)
+	for (wordnum = a->first_non_empty_wordnum; wordnum < nwords; wordnum++)
 	{
 		bitmapword	w = a->words[wordnum];
 
@@ -707,6 +712,9 @@ bms_add_member(Bitmapset *a, int x)
 	wordnum = WORDNUM(x);
 	bitnum = BITNUM(x);
 
+	if (wordnum < a->first_non_empty_wordnum)
+		a->first_non_empty_wordnum = wordnum;
+
 	/* enlarge the set if necessary */
 	if (wordnum >= a->nwords)
 	{
@@ -779,6 +787,8 @@ bms_add_members(Bitmapset *a, const Bitmapset *b)
 	otherlen = other->nwords;
 	for (i = 0; i < otherlen; i++)
 		result->words[i] |= other->words[i];
+	result->first_non_empty_wordnum =
+		Min(result->first_non_empty_wordnum, other->first_non_empty_wordnum);
 	if (result != a)
 		pfree(a);
 	return result;
@@ -890,7 +900,7 @@ bms_first_member(Bitmapset *a)
 	if (a == NULL)
 		return -1;
 	nwords = a->nwords;
-	for (wordnum = 0; wordnum < nwords; wordnum++)
+	for (wordnum = a->first_non_empty_wordnum; wordnum < nwords; wordnum++)
 	{
 		bitmapword	w = a->words[wordnum];
 
diff --git a/src/include/nodes/bitmapset.h b/src/include/nodes/bitmapset.h
index aa3fb253c27..fd2ed729de5 100644
--- a/src/include/nodes/bitmapset.h
+++ b/src/include/nodes/bitmapset.h
@@ -37,6 +37,7 @@ typedef int32 signedbitmapword; /* must be the matching signed type */
 typedef struct Bitmapset
 {
 	int			nwords;			/* number of words in array */
+	int			first_non_empty_wordnum;		/* first non-zero word */
 	bitmapword	words[FLEXIBLE_ARRAY_MEMBER];	/* really [nwords] */
 } Bitmapset;
 
-- 
2.13.2

make-partitions.shapplication/x-sh; name=make-partitions.shDownload
#175Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Thomas Munro (#174)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Aug 10, 2017 at 9:28 AM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

On Thu, Aug 10, 2017 at 1:39 AM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

On my computer it took ~1.5 seconds to plan a 1000 partition join,
~7.1 seconds to plan a 2000 partition join, and ~50 seconds to plan a
4000 partition join. I poked around in a profiler a bit and saw that
for the 2000 partition case I spent almost half the time in
create_plan->...->prepare_sort_from_pathkeys->find_ec_member_for_tle,
and about half of that was in bms_is_subset. The other half the time
was in query_planner->make_one_rel which spent 2/3 of its time in
set_rel_size->add_child_rel_equivalences->bms_overlap and the other
1/3 in standard_join_search.

Ashutosh asked me how I did that. Please see attached. I was
explaining simple joins like SELECT * FROM foofoo JOIN barbar USING
(a, b). Here also is the experimental hack I tried when I saw
bitmapset.c eating my CPU.

On my machine I observed following planning times
1000 partitions, without partition-wise join, 100ms; with
partition-wise join 500ms
2000 partitions, without partition-wise join, 320ms; with
partition-wise join 2.2s
4000 partitions, without partition-wise join, 1.3ms; with
partition-wise join 17s

So, even without partition-wise join the planning time increases at a
superlinear rate with the number of partitions.

Your patch didn't improve planning time without partition-wise join,
so it's something good to have along-with partition-wise join. Given
that Bitmapsets are used in other parts of code as well, the
optimization may affect those parts as well, esp. the overhead of
maintaining first_non_empty_wordnum.

The comment at the beginning of the file bitmapset.c says
3 * bitmapset.c
4 * PostgreSQL generic bitmap set package
5 *
6 * A bitmap set can represent any set of nonnegative integers, although
7 * it is mainly intended for sets where the maximum value is not large,
8 * say at most a few hundred.

When we created thousands of children, we have certainly crossed the
few hundred threashold. So, there may be other optimizations possible
there. Probably we should leave that out of partition-wise join
patches. Do you think we solving this problem is a prerequisite for
partition-wise join? Or should we propose that patch as a separate
enhancement?

--
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

#176Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Ashutosh Bapat (#175)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Aug 10, 2017 at 6:23 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Your patch didn't improve planning time without partition-wise join,
so it's something good to have along-with partition-wise join. Given
that Bitmapsets are used in other parts of code as well, the
optimization may affect those parts as well, esp. the overhead of
maintaining first_non_empty_wordnum.

Maybe, but if you consider that this container already deals with the
upper bound moving up by reallocating and copying the whole thing,
adjusting an int when the lower bound moves down doesn't seem like
anything to worry about...

The comment at the beginning of the file bitmapset.c says
3 * bitmapset.c
4 * PostgreSQL generic bitmap set package
5 *
6 * A bitmap set can represent any set of nonnegative integers, although
7 * it is mainly intended for sets where the maximum value is not large,
8 * say at most a few hundred.

When we created thousands of children, we have certainly crossed the
few hundred threashold. So, there may be other optimizations possible
there. Probably we should leave that out of partition-wise join
patches.

+1

Do you think we solving this problem is a prerequisite for
partition-wise join? Or should we propose that patch as a separate
enhancement?

No, I'm not proposing anything yet. For now I just wanted to share
this observation about where hot CPU time goes in simple tests, and
since it turned out to be a loop in a loop that I could see an easy to
way to fix for singleton sets and sets with a small range, I couldn't
help trying it... But I'm still trying to understand the bigger
picture. I'll be interested to compare profiles with the ordered
append_rel_list version you have mentioned, to see how that moves the
hot spots.

I guess one very practical question to ask is: can we plan queries
with realistic numbers of partitioned tables and partitions in
reasonable times? Well, it certainly looks very good for hundreds of
partitions so far... My own experience of partitioning with other
RDBMSs has been on that order, 'monthly partitions covering the past
10 years' and similar, but on the other hand it wouldn't be surprising
to learn that people want to go to many thousands, especially for
schemas which just keep adding partitions over time and don't want to
drop them. As for hash partitioning, that seems to be typically done
with numbers like 16, 32 or 64 in other products from what I can
glean. Speculation: perhaps hash partitioning is more motivated by
parallelism than data maintenance and thus somehow anchored to the
ground by core counts; if so no planning performance worries there I
guess (until core counts double quite a few more times).

One nice thing about the planning time is that restrictions on the
partition key cut down planning time; so where I measure ~7 seconds to
plan SELECT * FROM foofoo JOIN barbar USING (a, b) with 2k partitions,
if I add WHERE a > 50 it's ~4 seconds and WHERE a > 99 it's ~0.8s, so
if someone has a keep-adding-more-partitions-over-time model then at
least their prunable current day/week/whatever queries will not suffer
quite so badly. (Yeah my computer seems to be a lot slower than yours
for these tests; clang -O2 no asserts on a mid 2014 MBP with i7 @
2.2Ghz).

Curious: would you consider joins between partitioned tables and
non-partitioned tables where the join is pushed down to be a kind of
"partition-wise join", or something else? If so, would that be a
special case, or just the logical extreme case for
0014-WIP-Partition-wise-join-for-1-1-1-0-0-1-partition-ma.patch, where
one single "partition" on the non-partitioned side maps to all the
partitions on the partitioned size?

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

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

#177Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Ashutosh Bapat (#175)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Aug 9, 2017 at 7:09 PM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

I started reviewing this. It's nicely commented, but it's also very
complicated, and it's going to take me several rounds to understand
what all the parts do, but here's some assorted feedback after reading
some parts of the patches, some tinkering and quite a bit of time
spent trying to break it (so far unsuccessfully).

Thanks for testing the patch. Good to know it has withstood your testing.

On my computer it took ~1.5 seconds to plan a 1000 partition join,
~7.1 seconds to plan a 2000 partition join, and ~50 seconds to plan a
4000 partition join. I poked around in a profiler a bit and saw that
for the 2000 partition case I spent almost half the time in
create_plan->...->prepare_sort_from_pathkeys->find_ec_member_for_tle,
and about half of that was in bms_is_subset. The other half the time
was in query_planner->make_one_rel which spent 2/3 of its time in
set_rel_size->add_child_rel_equivalences->bms_overlap and the other
1/3 in standard_join_search.

Thanks for profiling.

I have separately mailed about bitmapset improvements.

Equivalence classes contain all the expressions which are known to be
equal in EquivalenceClass::ec_members. For a partitioned table, there
will be as many expressions as the number of children. The child
expressions are marked as em_is_child and are looked at only when
child relids are available to the function scanning the members. The
number of equivalence members increases linearly with the number of
partitions, and the number of words in the bitmaps increases linearly
with the number of partitions, effectively the the number of words
scanned increases quadratically. Hence the superlinear increase in
time with the number of partitions. When I took separate profiles with
1000, 2000 and 4000 partitions resp. I see that 15%, 29% and 40% time
spent in bms_is_subset() resp.

I am not sure how much we can do in this patchset to reduce this
problem. Apart from your bitmapset optimization, we could perhaps use
some more efficient data structure other than list to search members
based on the relids OR re-use parent's expressions for child somehow.
I have been thinking about the second option, but never got a chance
to work on it.

When using list-based partitions, it must be possible to omit the part
of a join key that is implied by the partition because the partition
has only one list value. For example, if I create a two level
hierarchy with one partition per US state and then time-based range
partitions under that, the state part of this merge condition is
redundant:

Merge Cond: ((sales_wy_2017_10.state =
purchases_wy_2017_10.state) AND (sales_wy_2017_10.created =
purchases_wy_2017_10.created))

That's a good idea. In fact, we could use a similar trick when the
condition is sales_wy_2017_10.state = 'state'. We can not use the
trick in case of DML or when there are locking clauses, since we need
to evaluate the qual in case the row underneath changes while locking
it. We also can not do this when one of the keys being compared is a
nullable partition key (a concept explained in partition-wise join
implementation patch), since a partition can have also have rows with
NULL values for such partition keys in that partition.

I think the idea has merit, although, I think we should handle it
targetting more generic cases like the one stated above.

0003-Refactor-partition_bounds_equal-to-be-used-without-P.patch

-partition_bounds_equal(PartitionKey key,
+partition_bounds_equal(int partnatts, int16 *parttyplen, bool *parttypbyval,
PartitionBoundInfo b1,
PartitionBoundInfo b2)

I wonder is there any value in creating a struct to represent the
common part of PartitionKey and PartitionScheme that functions like
this and others need? Maybe not. Perhaps you didn't want to make
PartitionKey contain a PartitionScheme because then you'd lose the
property that every pointer to PartitionScheme in the system must be a
pointer to an interned (canonical) PartitionScheme, so it's always
safe to compare pointers to test for equality?

Right. Other reason to keep those two separate, is we might change the
contents of PartitionScheme as we move forward with the reviews. May
be we should revisit it after we have finalised the design.

0005-Canonical-partition-scheme.patch:

+/*
+ * get_relation_partition_info
+ *
+ * Retrieves partitioning information for a given relation.
+ *
+ * For a partitioned table it sets partitioning scheme, partition key
+ * expressions, number of partitions and OIDs of partitions in the given
+ * RelOptInfo.
+ */
+static void
+get_relation_partition_info(PlannerInfo *root, RelOptInfo *rel,
+                                                       Relation relation)

Would this be better called "set_relation_partition_info"? It doesn't
really "retrieve" the information, it "installs" it.

Yes. Done.

+{
+       PartitionDesc part_desc;
+
+       /* No partitioning information for an unpartitioned relation. */
+       if (relation->rd_rel->relkind != RELKIND_PARTITIONED_TABLE ||
+               !(rel->part_scheme = find_partition_scheme(root, relation)))
+               return;

Here and elsewhere you use the idiom !(foo = bar), which is perfectly
good C in my book but I understand the project convention is to avoid
implicit pointer->boolean conversion and to prefer expressions like
(foo = bar) != NULL and there is certainly a lot more code like that.

PG code uses both the styles, search "if (!" in execExpr.c,
createplan.c for example.
I find this style useful, when I want to code, say "if this
relation does not have a partitioning scheme" rather than "if this
relation have NULL partitioning scheme".

0007-Partition-wise-join-implementation.patch

+ {"enable_partition_wise_join", PGC_USERSET, QUERY_TUNING_METHOD,

This GUC should appear in postgresql.conf.sample.

Done.

Attached patches with the comments addressed.
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company

Attachments:

pg_dp_join_patches_v25.tar.gzapplication/x-gzip; name=pg_dp_join_patches_v25.tar.gzDownload
��I�Y�<i��6��j�
�K2��I���+c��������&�R)HBm^�ayv�����x��h'���(�fH��7��I�K?]�K�x������������3�M���MF�o�y`9�������`dM��l��1���y�3��|SI��7����O?���As���1������7���_�7�<���	B_��xHv�@O��=��'��i�~�������p��%���|1�mK�S��'�����������G�p������I����Y36]���i��])��T��E�l���_��Y������DO�o@�.�M)l��.�����^���|4qF���}'���������wl4����0��5�)
eE�PC�[j(#
eZC��a<��E��3i���<��`�c
v�"TW�4���&�
�x�<�x$e�V�GA��
�������5d?����V��n������m��\D�f�����$�����:`�MhY ����6�2�\��d��a	��!�qK�\gI��'��r�*c�k��i���i�j�;d<�5�������`�Lbb�(C�������K|mnAi�0$F����(-�� ]Ql��w.����P���R����KP��t�-vI������3K�R��(��y=*h��?�<#)�W��A������
�:���(BP�#��.%�Q0ID,��JjN\�!wC@=	�pX��q	�;#�L|f+�8�0�V��`)(�/D$�HDiq;d��T2|���A���hXW�bX6~t0m�`��my�DDA�Vm��!���!{Z��=��wI�5��{A���CkVH�<�J�>0C��@sQ
����5k���\*��3di&����X"i�eo��[�p�����UE/V�%���L(^+�����g�o"!��
�2�DZ"�I��}6�����F��� Y����A\�7r%<���7��Q�G�pBZfik�����k�b��A�
��X��)A5��#��<W���j5Y�\�<�E	��f�0���b��/K�	r�^��4�}�>�
k ���D/�L[�,#���
��d���66�-]4�2��Kf6RF����'�,�9�EC. ���I�A���K[�2��4���X*Q�a���`�o�����
��3���F�@t����a�f}y��3�����N�H:�[����
��tP@
CB*�5�r�p����H-z�A��Kog�q����&;x$p3��U��v�$�����ly\ i����7�H(8i%�}������vAtz��9.�'�o� ��8�ZG��I��shH�.���<ME�/��'�v�����%[�^�`>S��3$�U4�!u����m��T~$�C�	<� (r�*�'[��JM+uH��*z
5F��?I�D�������z���:T|f�{[^��
o`�j�*�
0��&�N$QC���hy�k��Ev�v�%I1 �~
X�3�ZBpz��u~zI�2�TM?�5�[P`��,���C���*%3�=���tN/�REw.�x&yXIOYY��8���B��Wd�`k���OD�:�"����`�w���`�)���Y`�
�8d>��q��2�����C�����c�L'T�a��(�0��EK���pt\���@a�
C�������W��F�w4�N�xT�E����uk$17E2�RXS��x����Ek�"��j2����r#/]�=A�[0�4
�g�������.�2����mQ�`/��=���c��w?f������p�	������l�vv�����G�Q��+?9l/ 1�����Dn�3�5�G��8�����]��9S1�f��p>[xc�b�h4�����h�G�����\L������#��q@����y��`��{������3�|���1��dr���TG�h�`��<U��f��u���l�Y�!�B)���50��e�%[L�=c�'���:�sY�""-����q�d�������{�j>��"���a�
K�I�+��U�)J
r:4�o!0[�jD���X��������0��e�S��N���*��3�+�tm�
nb
&�7A����
NL:�p��=��(0��x)����Vc_=�T{A�n<$���`Jbc>�L%We�������*@u.S��(��u�I	��v�x^�I�|K�j~����"�`o
��=��U��I]����F��B:]	%/!�a��U�E�zVTx�����?���
��'�KB�S������~�	�j���X�@�C��$p����_!�
D(c�?"X^(0�]I�%r.%��2T����2z�N^ ig�8�!����b��E���G��a�(������m���K��Vy��8�wZ�z����,*���e�8�X���Y(8G��������)�����JJ��V����Z��	L�����e��tIgI�B�Z�� J�M�-F{��-k4����m+� .�#����M��s�g���c�0�����_D�H�n��n�LC�%�3x0��y0��x0��x0��y0�3y0��`G�AC�8�5��<��kx=v��c{�����v�������������X��b���/������p��~���
�wb���kw�
�'�����p�Z����
�w`�wF��'�������|[Y�����e9 +��&��_��(@
�����&`K�-e���]fY��d��g�����!�?���3�x�F��2WCMTkS}�l_��0�}0Ux���N'�^��f��a^����^����,}������u)X����x���������P��%a{�"�c
��v�\H�f�\�	G MWB� �}�z��������oo����
�>��[k���MQ�f ���/�[A_�\�T���%���/�#@'�i�5D���pf��`���{.�������k��=�>I��GA�@t�X��xE?��#z�-������U��y����\"����pK(i����hk���|�5�"���h����d��AGu�����wr��M���?Q�*A����6+���C�-Uqz�j�uK�������j��)0�(����^����C;#�d���e�^;���P,+M_�kG�D[�3n������GJ4c[�f<��hv�B
�W,R+��?���X����F��XZCw��$��R��/��t6���d:������z#��Od
C���:�L�}�U>6����c��c'o}t��u����?�n���F��/���p�|)0��c�����O��h ��E������0����f�oJH�����|4R�s�0��b�U~�@*>��&�4���q@G
�b�a	{)DV,�Q������P+%�B��. ��&���`�"���$",r�30���y}�uO���$Gf��2�����-�L��z���j%�T�T�)��n���~���8��$�*P���>���Q�����*"w�	g
��n����N`����qi��9�s:+3��������?���g�u�����(��m5��RC�#8U�#S��?>$���I7�.Fqkk�a��/��z��a.:��l�V2�a
g��pb��-���?�����F� �M2�:�X�I��b�+�2��za�����=����F����g�����?�C��3�po�N�#�����Lp*F�{>w���9�m;�g���]A"��'���������P���P�����K�2}�t�`�����[��;Uv���������*����l-��y���ya�@�
��$�F�y���]<$,d�_nr=���RuA�T��i�x5���!
�^W��)
���3]���n�;�g���P��(_�u�� �u�j�1�5G��a`��m"o7�]�����t0ZA�Sz�������T+��F����}��i��x��d}QQG�����j`hO*� �a*0��T�h-���U=�g�M�������a��G��������k�*���7[��p�LSo<�_����[��%cG�02�y���i	�v%�o@�N�+�
��������>]��3i�C��I{����FA�����XH�W1/��������(n��� UO�[����j���*
�5�}l���@|�����_�K�#(�F`�d��!���w���w�/8���:xv~NLw\w&�Ab+kF�in�l���4:�tVT}$K*���=�<}�X�0%��k5�u_�u	$��:�et�?W�<���-��E����NA�J��/Z0az������_@���*�e5��vx�o�t_wg��PR�L�G�e8\�)O8�:�{���:G���:��D�[�\�<��F��:J�4�}D"�b}�?f����\�\�j���������������>�<��R�cg�8\�@MB��O�M&���{��{� �K���p��ph[��J,���=`v����tnb���	~/P�$��/�we�ZHr��[�J��#�%��<��f��y��d2{4���j���S�j���V
�cu������M����&
��F(;c�������t(�Q+��Q�=3-��4�	�J�Y
	r
�����F����T��,���?v����T��-�hWf�����8��j~�jw�J����'�	�9�q���i�����	�5��<������>����6;��7��?K(�	����56�~mV�A���[$��3��:�6#�I�������{���?�?���?�Y@�2s|�O��{s>�.����?���t�8���;�~��w��������������Xc������������X��������.����L	K���ga 2�,�Ye�Q�{�0L����)R]A��J���x�Q]������R���N+����l��) �3�*2c��xj�����&�����J����?@A>6/�k��:�Ry���A��N�w�q��]M.�2��Fk:�R��E�?�������
L����0!OuYK_��d��!��4#.�6���O��K���iO��0�)c�����������W�����i��h�����Qu� i7��u ����!��L7Vi�x�nU��{�+]���;/�]4$��� !�z�'���
N	��a+�zX��l�~��jE�&;�������J��=����W�i��Ik�.~�Q��F�@\���;%]<���x�t�@m��Y�)T���d��w8oq<��9)8�Z`wxTLz"�A��������������+U�&�$��2]���������<�j	W.;����=7��A�������*��F��F�h�����Z
AfM��to��r+���7���T6���-����{p�3��������7�0��dO�
��,��e^]��2@i�S�u�E���XD�w�X��������U���6���l�g����j�Ei7��6Z�&,|���u�������g�fO]>}�N	���3�lW��.��=4X}�6zq���
G*�����:$����vE7��{���]^���Qe�n��xh9�nN���M��2~,���3���},g�Q�(����~�|
g>�1��R6���VD��)�_�VL�8#�@��D�)����=RkO��(]��H���*D.����X[��������Pq�N�B��/t�RNi��/�)K+ks����]^����9��r����)��X�!���
q����1�ZVo0	���EF�Fv1=�c������s�l��g�`�T���l�����J<|�T%c����C�,��7hJ�BWJcFy�w<S���	��
�[�~�sz�c��yA��
��2��,;��������m����K|�^��� ����h�Y]��l����0 ��e �Y�w��>�g0IYv�dMq��������Yk6ON6��;���"mV��grR���V�nU.�V�_d�C+ly>�i��@>D�/'����(3SQ3�\B1��Z�@�IeF�:�&������S��Z���T6��t��
��LN����l���8�}0�C��e�C�t��%q�����mG��%��Xb�IgCoj���(��*��y���S����
���E��gQ(`�q 9���*<��u�����X�Q���m�T����3���p$ t���uyS���)��� ���� zTSK�Zu�V�)~������z��9��sF0�a������1\���hpg-`L-�]���b���^C������7{���fPb�v������@�V�50�-�� JCA��(
�^KBqU=��xwQ�VS�'��!9
RO������'�Q�?�%c v�,����;�Zl�'��e��X�E|�/j��7J�:����^�������h�a�l"�O�"H#"D�����q]��-�M���b�)��2��A�����@@�����"A������.������-b_4w�Xp�F3�*� W��B%�����8����D��gfF�/��O�g)���;��
/�Z�GR�-��X. �do��?��'�+�o��b	�(9[>G��������['u�_��O���F����C��bg�Cfi����[}�����.g�e������qr��}0E
��K��6S$"L$'x;cR��'XS����R�m>t��R*��I�B��8is��D @�a4���3�SK�U�~�H?N
��@�w*�I�������q��U8��.�LE���w'�Q�An�����>�����@�&4��u7������
�#���"�GY�<���D�sX%�V�����27���/�2����)"�r2�p��<�����j��^��5�0_��QS��0���>K�=�#����5�M�C ;�]Y����4�unop���}:[&��V������8/%z��5+3|6��J����.��*�����K�2D@���T�Cz�@�o:���Q?�W����}����z���9��L���X �0y`�S���ZQo�'�!���W���S��.�t�������S�S��k�.�����������>%��G�Jn�)��v�}���g�q��� 
�v�W�]>y�0��u�n	�����h���72Y,����A�����#`��Y~v��������u���w��z��	Q}X5�4z��zS���[�:@�H��'�S�8i���n91I\�(Y���y$T�d���"$��u8���n��>,:�zT,pY2�F	��g��Ef�0����d�K5[�m��4%��v��S���=����wg�z�2���p�I����IcM^;�RMRG�z�Au��W��0"9�x#�H������N�"�5JWN�c#��/#��X����"����-c)�V���B��+T����$M�����w�o�����|�>:}{y��v��HB����y3�o���'e���%��yq�����NzU�X�����g��?������]7�=�]#��I4��~����L�=}Y�[v��w��^��,���K�x7���d���������������TH������;=�x�"i�$<���-������5��y�}RIp�G,�OX�����q�|8WcIE�����BR�������t[�-1�S<&���8w}�6���P��������=���� K��w`������Z<mZ���3�)����8M�]@_ltm$|�2��q�������h�������q?�
b�r+��S���5<����5�_���s��%_���R���Qx"�b��Y�X92`�����IX�NT�����	�Js'��BE.�0I{Q@�,������n����@�~|U@��}�#�n�P�x�C�H���M)@�P7���>q�����+H=-�P+kTe&zG�u�"<Je\�I���t]*����������D*��S�\�$t3w����;F��T������Gq=[X�H���y��K�S� YR�z����� ��%����uT��\�S���&l� �D���d2B��)
�&�Tu@����H r���&#$���x���>vH����0B��kL���U�E+��u2��-�)vh
�qE��.��Y��|wOg���E�`w����7S��_����jpj�Q����1�-������}:��r7��N���t�t���_z���� ��-������0d�AD����V0#��vP�b�}8;����I��ye�Yk��!�����=zohRNE��u��B	���
}�������r}oh,��m��'v��x�k:�r#���m�(1%e�k��������*$n�
��.S#>H�5�n��A�u_�'��������c�\��������gG�(�
^�6�?��H�k�����5L��3�=zh�c �E�@f����C��i �O��QOn�G��W�>�����	�dG��}������
zp�kE�
[���DCl�`���k���!�K�����[]{<��$;�D��J_[����l��|�U_1|w-�Y?������|����b����(�P@8���Md��Z�h���G��}���Ar��v�����]$���-^Fh����	+���#���	������]�[m����q+�n1��� 5&�����[�^�����xs��Ue�'�c0�4���AR�T��c�3!�n���s%�>iE������O���dt���:�$,5��G��)5�>��!��J� ��q����A�_�1�p����G'�����Qk��Y�n���\��rve��]��dUJk�����NE����,��J&�o`�B��&�-�	����T�y-y�.0$��8�h�7��1G�_~�7`��/h�2oQ�r����3
��3����f73]�='8P���d�������ivFS��K��	6{.�����s�S���^S������ne�)�?C������.i|��i��jB0�@���W��R���A���Dd���PGKd�:J@���H���f)�S���=���3��n|���������s��vN�R�;�m�|�V/���V�/�����~�k^�D�5�N�D��N?�����s��x0���:�X��a������xs�'�ci��HId8�p�q�s���W��V����A/)�:3C����=��~���@4�C2�/��"���v��[[-�>������5k���wQ������]\�C�7���j���$x��/j��@����Xk���:��������;\�_�b+:�q��x6��%<���uu0�'�����@}q�����c�h9"�
!�>�@�=E�<���#9pX����I���"��.�1�`�S,RM�<_�&}��u��cj����Z ��-^08�f����`*�����e�9S$vZ�\p
ka�	P�p]|�`*~(,�]?������uMm�W����kH��d��~���Tj��A�0��Z�G�1�z{�J�~�L���U�=a����5�GF����[J@!�!��n7��2&�T�J�g�C�pl4����?]��5�t�@b�Zq��]k�b(T�����Swp�N��E��d�-�(����.w����p��9q!S��������m���-������Z[w���$��K9f8�sO���6&��	E�)�&*�!�Z�"L�,�h���)Y~����q�k
~#�����Z�4�P��W;P���U�Y^{X�L4��e��[�����u�lh ���qPNr��Z���F����e��1`s]c������$^�Q�����V���!�l�)U�gf�2hu@ �	ama�,�w��t,
���2���u���~��o�����3BY��/��e{���<�3P�������p�
���cA�.\��N6Dqf���;#�j!'$�����m������,�I���i��l�i�/�m�zf(~�I��u�B�������L��7\����a$�(����:���qt�����Lv�"�f���)�\�~&R��HF���|I���S�Q����-\i��s�N�[H`���������r���{TD�1�(M��7����;�+t7}8�i���"S��_�^�"�l�!�8I��~�&��N Q�����w�*��r�rIh)7)��&�RpxP�����.�s��Pe�Qod?�|��KiI+��$���1�Wk�xn-���*�����S)�UT�L��-������;�=�B�46�T~k��LGv��\m������iXyx_.L%���k��� \f��j��Lw�����}�����Pi�*x��{�I������k�q�Ql��(6�s��z�E�Ql��(��u[��b�=ZE������H�rf�sb���@TG,������F�}���-�����(��������X�����h�����MQ
?�9����7�T�Ji�*���5,��N��
g�]�<��'�o�){)�=z)���W�����������{��h��ov�� �yc,�h`	���x��#�����	��	�)M�-p���
�����[�~�t����f��;�X�q���:������s9T����F����r������<�0��u\��+q�:��������6�jA+���dA=@8������Q�Yn�`[���y�t������+����������Y��"9M/�?�B�-]i�����b�:Lf+Tw=[��!P��tna��J�E���.H�5� �oG���G,���1��?&�AZW�	��bG0���|>����!�����\(n��u������b6_@
sf����BS�o����<�$V���I2!�|���g�s������EaC�
�3�������l Qx5�������������v:&����
$�e��fHi	E���i�����[��n{4]����KZ~�k�����!~��+F�d��yV�����������4_�S�!A���J@^����B���1��l�,��e�L�<��H~�p�1�����In�J*dUJr*��l�����mL3Gm������m�2;RJ�.)%�=)����]�J����e�u\�K����:�
p�us�+�2�PG��v���-6�����'$��d4E�~Yq����b^�����4���������=�X��2���S�vYm����!�3o��}d�+1���d���u;!)�%�G���"9����!�v���&+I���d4�l+w2�Q��i	��A����<��_��]����W1�����L�\�w���?t�Q,�3�B��>0��	W�k��|���a#�Yj�g�r����\B[��O����6��������=:j���q�n`\�r+�Y����C��/�G���E��+G�D�^�TT�I������N�3����(��c)an�x�E����BU00\����6Jh���'����Y�*����9#)��� �S^��1Q"��&��IYF2!h)�����Y��T���'V�brE���Q�XK�R�����0hG���7y���~���$���+D�����i���M)K^p/$���A}@38N�htf���z�5�+��OB
i�"��*/��h�I���K�K9wg���U��)o�h�_`a2
pkkX���\6&�6���_X�G;��g�t�7:����np���b5�������H�qa���e��D�MA��2��,Uv�|�o�67�e�u���f[�������4I!�"�,���xp��m3X�#o7�N�KL��O�&���^(p5��
4������s��>k�p��M�]|��U�\�%�����)J:�������'�7���Q�xx��0�l��6�sU�h�QD�Ej:��VTu�{+�&�L`�����������)A�����[���s}�����������.=�F��{������y�
��CxdT8/�U��1;���(�� -	m�������a����z����=3����]����8S*�=���T�����_7��>�F-��v�����h����d���n���3^*��)��3��YJ����\$�w�%��A��"���&����� ���Y8qO�@+L� �V>����r������<���-�����M�=U��*�EZ	@��l��'����y���)���j����^����fv�J�m�����L��J�+{	Ie���t0��&K�SZ<����We���]�s�r�R�3dO�N&j�����V��X�R�b��y�������h�N����1�\��G�*������5N�����yo}T�o��7{L?y���r�f������l��j��#�~�����o-o]������:�_��#������993K�s;f����H:��u���7qhv���\n�F�����Z������^A���������d#NPY<Y��x�M3Y��'��7��,$�O�}�!@�z![�M�iK��������i����2#�P#�W��UB�k�����R��+I,zx�j[yxiM��t<*eN9��h����!.'i&I��\�/NomX���>&��8�����i:��������
�R
�9��Q%���t��A���v���p�jZn:��������D�$����?qr�2;`��
{SPY*��ft	I�D#P�d<s[F��M��x:qj����S���	�=�q�2���"h@�i�3*����C���W��#I���dD������|JO���=Jr�|@���GO�=�f>�*Pf_E/X�;����p���=hfgO
�M
�M�v��S
���/r���
)��D�Q���b*��}��c�p
;j���=����dv�2�����P.��h6"�2����^E��2��2e�~�;>��B�����&�?��5X�Q�9�BM����4/t�{���X��Z���B7���S�fZ*=��1^&�F��Z�v��IFQ5]Mz$J%t��v-pe�H��������e�7�z�I�s�0���3����eL�a�;;Q�X���8R���`h�4�����P�R?���l�4��_t)#E���O4��h�[7��*bx���������r��9\�{{���������Lg��|���tJj�2�U�����u���V.�mR3Y��j�r����s�A`3j��6!��i���knU$}:�]s�mf���n9^�s���������0��(��V��_�c�A�����G�%��R�V��,����x;�������kW���'G��I�^��G�H4��v����_v����^�Q���%2���c����-!�-���"A>�Z��x����5K�(�$�f�IT�B���&>��d�A���.�k���}$��������$���0j�"�Q�W�����3�M�>���UE�R�w����������Y��ye��f��l%�$���*^Hv�*�b1x�������d�+��|�P&�[���(Y����-^�bG.����=t�\2�����V�u�9�u���4+�|���1������_}�y�r>��������*K������u�v�AX�GX�C�w�z
z��j�vQ�@�c�J�:/u����G������qr���E���v���+�y��SF���%�Y~�Es7��������Y����ENP���|u�b�\��HI��!����t��n�3�7F�}[�s��`	� F���Ev�o���������7��~��������T�p�������L��\�������:Y
�l=y����L�L�`+�@�w[y{�6�7>p�(�PS����w"�Rr�WFJ5��0$��(�rB�"��6��m�|��@A�x_�}��w��x��EU�7��_I���F��3�%@����i�Iz9���n�vM�w$����*��`RW�|/�<��-|=��	�����6� �Db6pv���*HNi��ac{�{��2���el���s�SW���K����������6a;������]�0�� j���9J���zf����w}B��Y��T�Fk�)�;�+���Z�V���X�
!����f�7����o�`�E��r��|�`��jU�~m�$�?��������/H��D�d�=�&�a�#����������
k
���-����g��E��"3�e���:C�0��`�ax���@���H?��nl����}�}�s�K����j���o7[�f�a��ST��W|�0O�<�8A���ga;L��P�����n��^c�h4;n��xM�\Z�84���-���&��]�S&Z:w��D���j&����0�r�5[������VS�e{��mG���I��y �=K:���d�D����<^��?�8l" �pOd�$g]|?��nk�,k���D,���ji</
�#j��^>�K����my��&����h�L�������L�����O�N��4���$e�D����~@��8��5UkF��k{���%*��}��t �A��8�Kf�P�8Kg�}�5h����|6G�P���7@o�*a�96���[G�#���F��|����u\8Kw�Z���=��D���Te����O������$���'��+����a| ?��I���m�j���wx����xu�v5��Q���k��~�=�O`����p�l��a����~��']� s��P������9d��S�8���j������Vh��R�8<��d
�B���=������Zt|�0��]���~��{��u�]���������u�tpO���le�v�:��W��$���"f)��:*�������*5�:kgVR��r������BR��������;	����c��1u7�3�����_C�%��)u%OQ�u���jj&	~L*��������|}'���� �(U��$@�g)���S�3�9Y\r0���Y�����)h �h���Zr�8%�a�]����D�t��H�{ja/�:B�*�~(�$��Y��} ����M�/�Q���e1��*���������/��8�' X��������2�9k��V�������Up�e��i�B+�m��8�tr�5|��
���i�;�u[	�HA��W<�����q���=�^u%iU�6���4ELf�')v��I�����}{���d��0�0���HR���?�$d^�zN�a�V����f��V�O�B'W�T���cd��r�k�?7>t���=V��B��j�)�)ue/�	He����6���i.�����l:oNc�����gLF�f�mr���|�0\Y����A���p�R`��A?^������]����������Q����^�(n��<oQ��-j��R��<�p&Q_��,�<E�.�xi�����������Q:G���|�E��KPE>I�o-��3�+�x��Gt��ny	�\���0�-M�T�{�3��9w�[t���<����)>�%��d��xu�X�rE5'��|�A)7Ar�/R�
����q���}Qi���'�N���?~�������9^��9g4��>'��%�>��n�}�&�//�x������!;l��w������N�Ej���V��>l���aK6�����!�����,�]������o�E#>�����ZQ#>��^��}6�v���zG����q�%�H�C2RTo4����H�f����H|K�_+��{���C���|!e�A�.�I���R<\��d��c5�������v��#��F���|X�����O����^��^4�����BQ�iBE+���u�<X��J^w���G;-am�@�j8f�������=���}���[G��@\A�k :���=���'�XDU��g�Y��B�e��L�a#3�*���"!I��c���gu�m��>�����l
�8*�h�K���@��N�����������a-��by������"���Hn�-�� T2]$���/J��R([C�Z�]P��OI R��$j��1��1=����x *���~�"���L.���6�s���G�H��l���zb�����)����w
�$g+8`��)�D�� �x��u(*s�SS����Ft��;�X
�3�G
�'�>nK�����o9s����,1
9C�@�V}
��3\H�dA�4'�F!C1$b����-*��������$v �ir��������[�k�"���.CS�����-XEJzN��[@�J���W�>�!�����	�S��:3�'u���5M���V�L���]0�����Uv=k8�:@���%k#��&q`�W��4���6��1&�@k��AF�������P���a�| p0�[s�$&��?^�b���4���-@T�0Cn[y�"
��%
������[)���~!Wq��R�����"���2`$����?t~���&�H��4���N�=��TJ�SFz�H�;��X����R
j�����
�I��uPD�`�|��4���[���_}v$7�L���$��~�muo�&Zm1��F�w+�����.tH7�X��GIsp\�I>��?,�o���
�jrw����������;'iA��K��lQ�~��"����0��7�|/��Ky,��`2�z��u�bp�-��[��p������o����E��pi��G����c2������-��w��Rc)�}�f2����s[8�;W�"Q�sA��=�f~��������g�
u�IeP��-���5d$=9>�K��^�:�v��)��~W�X�7F�y|t2h:��K8��9�	����q���4���U~�J���rv��p���~���`�c����J�h�*�V�^��$}�E���|�<��n/�]-��	���F�F��l
>C���&�##~������-$�Z&��T�?~����!(B�{�KB�������v�9�p&��1N��u6FSL[�x��C�u	o������<#6�Q�E|D�/`1#g��(�U�|49d\q .k��r��ud����qjs���p���k�<B���|�.�jp`�+�/�@�Z�cU���0�dT��6$�t�aa���� ���������R�w���g����J�X�v/>Z������
l;cS���2��D��:�Z�5j���^^	o��|#�g*�� �Y&/q�S�T�$��[%cc�r���c��<�����.j�
���Pp\ ���B{��:�ND ud]`���~��_���~�q4�H���%G�8.M�MGkI�iJ��I��7\�V���������Dy�����_�t���K��a��/��5����y��'b��d�#:��J�Y��
&3��# Y��V�Bf��b�z��B�v��$�&#JY0���0;;+a$���9�p�(�G��0�U���1�B}-G�w�n$��E�l��lWQ`�`��L�lZ�l�nfh{lSXX^�7�?�L$MD[�<�/��������%rV�f������&E�M`> s�����]L������v6S�8��b�_�z��:C��8h���^v����&G����@��THa��h�:*�wb���"b	�1����H��m23{��p��v2Y�@$�C�����B������cT�h�BE��%�)��Z���U���h5��#�X��h��w�oJW&p��X]�'��O�+-ZiEH�	m����q������\�X���*a`��h��X-h�v�s�f�F����5�RH���"n'^����R�3������%�9�f�a���\�z~L�t���&��;����"[|�	�!�����@���:��[���
�9����]��^�����#N"H)����,�����(j���$Am)@�7"A;����DW�> ��S��e6C�j�����Kx&�4:F��S�=x�9���A�?{Z�9B0�8�G���sx�w%,1 ���th�{�����	��^L�b����S2u�}���:L�I�y�T/_�
�[���4`��nib�����6Gl5'���\��4��d)��r���������V)`m�R�OFh���l��h9U7�5����b�q�-�s���z���J���f��X���V�)�K���?�:����C�W
�h=�q�OA\��:%�����g)^��
*q�*�n���<�tv9d���%6�#-�m��W�$�� y��2"�%��I�a�ua(��b����8gQ�=�,'���������1���lM|�(��?eF�7)(�G���\����KJi,\�*�`���DU���t�������']EKp�f��R��U��}W�w������t�t�C	s;�AUy������HA��!�w�a�[%�wNP�k��n3���NV�+m�Mg���:�����T�P��	PU�8��*�4�����)OW}��c
.��9��w� Pu������!���.��_���As7�m�

A�������0���Q#����-�����V�;��(Wa���s���@r�\2�C�c�{�k9EF�����@�X�;e�`�����5�2w���CM+k��N'��=5�\X��� :n���f�<���tI���4+�kJ9p!�M���s�����nT�i7J�S�j�L�
6b1$� nk[�����c�����d��2�l�61!]<��Y����j�#@Q��='�?2� �B��3�j�����E��^��l�/%eSz��&Er���v^�D���:r9R��?��:M�Q�S�MB2�T�XRMw�'�sb�V9B0���d���Ysv�[5��7��%�&��?^X�9%�������`k���f�{�|�A�C�Z���	)��J"��
�_
��h���3l�����������.Ea��T���zc����w��~oP��N�'��r����
��~c��V�*B��.��\J�'a�k_0��!�'�b��=���e.�gvdu�Y'`��E���[�yAB)Gz��jfT��H����44/��i|b�H���
0�O���J"�BT�����Py�N9� m���zi* .���+Sa'��2�C>�x��X5���GN3&Q�����7E'�D2'�-`���Q�3�SBC��o;JQ?�):h�*��Q	�)F����nE�����#9�Gb8���(O�'P��k$�s�vo<�}����!����y5S���x�c
i�}�����0-$�+���(�x�a��?NN��Gc����L��<�~�}xS� P����KU��0�%� �t�V�G���k�vA�,���a0k�@�6�H)�wf��S��w/����������Q�^�MH�����73���(��'��*'�<����|�����V(s�[Y�@�>�z8�gU'y����9><��
X��K:���B����
/H9 �G������0w�qL�+ot������}��\�k�T�	��g��e+�P\��>�i`�>t��c�H�>�������iu�lPj:5ryf�{�!8��P4	���w����oo�)��DW�n�V!V'2q��$���aD]���R�`�BE5�i�<_��M(p"f������*���NR�~���S;�:���+�%�7�����o��N������1n��~�Qy�cP�a!�%\�p�Og��@'��'���s����XR�W��sp�t�v�����e�'�g�p.xR�T4M���.oPD�����lyFnf��@wt�C�B�n4�ae�|n2���ur�����4k�����'����[��Lu~F��IN8����O���������5�v�Gr
NA	H
����g�l�<���[b?�S[��vT
Lw
5��V��g3o=x�F
�'�	��s����6���(��]9��i��&�\k���T[y�x���T��D1������
�!Aeb#�������0M
$'T����^jv��A2p����p��Ajd��I��u|u��(����JG��*%U�c�V�����$*r���c���Y*�+�c�ilx�#]���]��V�����n��� �w���- h�TB�������E��3�#�C���D:���c7y,����x�f���2�m��'����s�KN���Yn.}g�)���&�
�:*�aEwG)�����'����hfS�A��UX��*xe3�?�i|����;9�O���:���b3$��Z����'�Q��b�u�y��]���	���j�f5s:���j2sI:�8F��s��U"4��YNW�����T���v-j<��e�S�z�������0�mk�_K��m��)�"
Oj$�����j�)C3���P/��oe��AtI}������������'����M�(~��r"�+��Rj&�L����3�)������H�W_1�Oj�5�h��NVj?����z���c��m'����#H�Z��
�#��}����
�-��8�e�I��{y��B�����]��z�e��8�Vt�tX��rE�y�P_�]���l�\������g���G�*��`�RB9��(������gG��z����	-���v0Z.g�'Y�	r@�:�8CV��l,9/0������)�@�"��u�����z3,c������M*�\j�Gd�>B���Eu�umj����xdj@Y��+B� k�{G���w2�I��s�	�(�L-'� Jz{G�
F�"{��\�R;������-�
�\������O�%�<�yvk��c����2�v�6g�4CR��W�;�z'w_]�rE����
S���"�yZob1�5,��8�in4m���
���I�X�7��3X�r;��o�4A,8ia��=��N{;O 0��ax��^��$�(�����|�_��������O_�3%L��3��WU��#Q�t|����VZ���2�D�I�,"��g������������s��.�z:@��y�I��RY/1V4I$*��f;��n��3���b�U���%+)�3��%9i�J��b�:�_6�F^���G509����) ����m�" `g��PX+W�&���C�c2��lJ���c)=���J�y
ep�%N���	e�����Q��r��z4���a�Q�G�^��d����Y��"e1�~Q�,�w$R�{�Tv�a�O�.�����������*�m���L��	�,�UD��):t�9��T��r������S`��|<�Id"F�i��GVg�x2�fQ�aO%������];G�-��j�AnO���������HpN?����6����-
���~�V�1Z%f�w�K�:}?.�mo�n{�-���S�,��V�53�I:����Z�Lv������C��f��d/�����+{��X@p/������'��B����^�m�+$�/Q��w��gU�R���+������)���j�����p�kQ��Y�[��AMAN����d��_�z�����#������}�S�����[�A/��=n��f���`�w�m�m&=���H���3�(����|?F���w��/��
j��I��|B�1p��R�^�^s/]�D�����(��,~��y���}����q�/����q0�~a'9W�����?#EX����B@����}�0�.�/�/�����
��.4x+��8�m0�k����wn�1�>{�#���+��t9����r��j����m5[
+�k�O�V����#�������Q;>i&����u���V��x����^/����A�}�����V����f�)���m�(�+b(%��j��Te��/��WS(����<����S~�n2������9��J��W
P�M�����W��SG�2�`f�pFR����KLF*���J�LbK���T�
.z�v��i���{�G���R��
��%\z���G ;c6
=g8<��p�0*Rv�$�&�`��I!�M!�Oo]f�[���~�]�wN��~���#�jE�e�����<�^��X�������i������V���'�Z 9�.z����:H�1�����+O�X#!Vz<�I 2n�����)���q��t�����;i|�p�xM��?�
/����9�y�h����U����l��I�j-H�jV�x�U��r�B�,���SM�@�]��kb ��r�+2�������w��+������x���;���v��L���rc6�_8d���,f��w�
yA�;��tf��4w�8�*��5;j��K�Q��`-r��&7]o�*��
�$M��"ACB7���~��q�A�x�Vp=p�����O}������Q*�T�|s��r�B��[�����E�O�C�74yd�8�.���M.���%���ZW���*�����i�S����|�XV�..�]W�j�X@<���Y�����>�}�������:���-�������h�^U�o�L���n
U3�g���.#�a��G��\���X��������	t)�sBA��'�Ref�-T���K�1l�M��0Tf��rhK���H��U�����G�_�|�h
_$�!
���a����jQ�)���
���t,�	����o��j�gro/��R
���������ds]Y�@�"!���N��4d�M�
���O�������,Q���'�\�c�Q�����"��{�#����0���j5[Y2Z����>jE������ho{�*��(nwS�T��T�-�������A]r�>�{���E��&g�^�����x�R��'�A���L3��v�<+xY�T�0�H�l �xG�dW
��-�*��%�C���1����N������r�P�.Y��U���e<G/%>T��O��F?z�<@\���&��#8TWd;�{���7�|��-B�g���Q�
N�{do!CNYSX��5��*�������������;�(����J�K!�#�Yk��U{��C&i���g�T��8}��Y����o�o�%��������)Qp�(���g;j�>O�F��;{������An�k�0�,:U���FS���b��J���j����i�
�)�&�����c�-�c� ��b��F��Xt�S)#���x�t	���[8�����y�]��Q�pyqzy���t_��|�q�C��U�T5�`lw�����U�?;,���x���$v���O���P�16"��Z��>P��EfOvl��NO�$,#�$GO���)V�hB�C��K�����Bx�3�����K�DXvS���?�����!$;�n^j��5*#K����&>F��<:B�!:>���;�#��v��8f2�]�H<ZMuJ ��)"�T���c���
�F�������
y��WJ\q��-�V��m��S�Q�g~��gbO\�(�����9�G	���`�e}Yl��)T��~9�Z��Y
���Q*��_��������6�UU��pf�����?:������jF�?��~���8>9:n�����A��?>i�����a�������$����|��DK|������i=���4�����>���{B�U]����9 ��B)q$}�'p����n�R��?-1�"��[��beyS/:a
�u��h������K���x���(L��Rd���$��(b�TfVn����0�)U�t�B �<�*�R��)��g���%v���y%��Z$���@�E�v9������ri[*��)��\��B�fU�A0�m�r�	�$g�R�$
��n9=��bBU�\Tc�M�'�F��):s����:�����N*V�E���-�p���R���	�2���B�5��>0V*��z$��x&���	���������l��]�*��b��.R��F��h.���A�P�d�`��S
����A�:��xA)0 y�}�n�AN�#.d�7F�qN��-������;H����MhL�3��@U6)MKeYEA�Y� ��
'����}�W��O��O���It���?k�����$�B�*���)��2n�j��X�DW�U3��B���sK�g���L�����}�N������q|4��d���~���'m�K�G��T��
LO�0.)���K]�c�SL�C�p
`�"�]�b���X���\�fn~��W���	�d��]�+EX�e�}��q@C���	{�`e'y��!��������97p�����&^B�]N)�8lRQ���������_��)�*��g���A����[�$�����(zf�@;�����n1K�6���`��*�
�Q�0g�����c���1f�*M,^xp��*�@�6������a�Q�`8}�%Nk�yK��5-���j�J~b��	 �d6�;-f���r���\#
���
 �.N@�gJ�A�/��A�R�����t����1Sy�+��;Nq�u���������3[�E@'2g7#��7���t��nb�.�x�J��(�����wS������Zt��<\���]�!��#�Sv��]�2@���=�#'9����|��2�-!��C�Y�N����f���V?C��&]��{�j!U��#�s*
u7�y�>C����}�;jCu,V�yjt�����1�Qw���U0���WW�1���&���������<�	�W��K7<@J�O(�\��jr���(#���D�ssD��)5�k��>�*9���O)��Y��x�[��D��
y�^�� ��/,T�B����&z��h���U��|Ipj%��x����l�kZwg����tJs%���=7s�<Q�!����6�G7@p	���������AE�P�<�Z��l�C�a�{G��
,z�*-���rLc6�Tl@I�S32�'���J�4�+%@2�b���z
���S�qW��M�v�i\�@�E�j�R��{�nqw,��z*����C�V������@3�ZPge��x��>�m-��O��@�+@<%UT��5(����T�b�)Y ,����8M?��lp����2T����T��H������[���2�G�
�'�����9IP�8��%V�8H�����X�b|�)����
d�fw"^��MH`���O.�]^���;p��W��P%}�t��/�	�aZ4�������5K�]��d��u�^��B�q����N���[��%*�cT)J����hC�LY���,���T�"%P�S{���n��D"3kU��br���$^s�Gr�!,�����TT'=`�,��N�7%��k7Z���~q�?L���j����.I�N��!�X�����2
��;�KHS��}�k~X�����b��iZ%�.)�2n8o����������8�^��~����������U����AXth������- b:^�e��%��kO��_RQ���1��(]�8a�,��z<���C�'�R�p�}r��M&�z_JNV����.��R�ff��A��9�9N��b�3�i��������&��{���7�������`e�MT7o�6��F�A�8����'6%S���<���t�8��(�:n�Z-K@��m��+�WP�`��-��U�G�uh�d3��4���y�hz
�h�H�IF�?S�<M�%�g(��w���5��
Z%S<>"���g�<m���afr���?��u��,�|B�9{��}�U��j�6��P��\�(�HX�)����"���y-)�l��Ar��v��i�s9G��r����CRC��V�{B
N�H�p��O�m�O�7����������������Q�B���'T�~�H���Z!���y��
�I~ub��L�.�M�M*9gk��N��������l�:���.A��T���������`rH!���J���[f�����y�k]7.�����q��mi�p���4����������������
���Z�d�^!���X�JM���r�,�{c�������zHt��m��������n[�g�w���c���i��l1��$Au�\����fA	�3�Y�u�d�nAn�@����V��<���w���kX���Oj7��j�l
��EI���G�Q,�c�h�����>:�Em��V���f�i�g-w�9[�_�����Z( '>�R���BL��������������r�����+3%��o�.3��SB?�2���k/�����cH�Xf>^����8.�L�j��Rf&ep<uK!���>7jo��rhZ���B��Q;��E�����|2'�N��S���=Y�)���j�d��$�X�'�9Y���
��9\{�N��\��+}��u{�:K\$����"$�3�$Y�OP�<��8A*[r��*N�>�D�����3;n�Z�f*�T(������r>�l�F_Ge������pc��O�m�����?���4�#�%����(�K	f��N,����M�i��F���	�F`D[!��������1EG�qjbf�a-K�[v���>�l�s]�Y�����|*{��^=��\�[��m��$�8?�]gfA=���a,��<g�p�����mP�p��h�������C���G����@G�\P�����*Q|�O��2}2�d��i
�v^k�E�HE��,��L���)���G���7GWqo�?5&���n��
qSEa�s6&PU{�N6�����T��J����Y�e�u�}��>���i�%Q���������u��JQ�c���Zr�g�q�_-]j�R-p u8u��sbq>���EQ��Y���s'�a�����j�V;h^�bD�U6���XR�9�1��(�7���`D��*�?�b�u��)���qL�,N�&�)���1��;u�E�Qr��n��5�Y�4��S�E:.�8 �����D	��b��*ya��kt�5�Pnb6)��g����0�I9����[;�����w����TG���7�����g����%s���>�c��0�H�6wA��7�B���e����������J��(,�gU�s4�b�
�5����H��j�������#����-�(0z�"+����!�X����E���cH�?p`��S'�������!��=� ����&���:��-V����O�`�^"�����E�#�`��E��W�'W%11%C�-F"��K�z����{�������'j����"����d2U��[��u�gf�.���bS�#5.���=����b#��}�s|��������a�l'���&`����x������.���)'�g!#���R�(����;U���Iz�$K�FL���a_����L�#
�G8-y�|�[��#h��1_~xbH\e0+/	%I���u�^n�e�wOl��B���UI�g�*�����]	�'>m��R�����p���$u�'_������)B,�C�]WD������Ak���+��hD!� �rMJ_
��z��sw'���3mm��4q'c
8�J0���z�Gq2�F���!����}(��q�I��eZ�������������6�%�pk�,��Ee��T,�q�����x���K��xE�xM)4�}L5��J��2���	�����������]��y�_�X5��EE���"��������V���������s��Bl��=q9f���%����'5����([�������9y�?�G�����h����'O����2'R�A��3��$�g��������'8���;��)pMH��J)��0:�?����I�0":>4}n��`�n0��f���������|X��9~����Y1�Y*>����x�9i�b	l�%��h�1�i����i��nS�n��!�\�d;5<��X6�6��@������>����a��SFMg�T�3�o^L�_��*����������V$���_��Z��'�q�3�9*����A�Ax>�c����c�sR�H�V��3W�#V� M��NHa��.g��x��\��2��z��w8�Fw�O��u��BN��V��1�������V]xI%�V|��ANf���b-�:,����a�by���wY�����	s��~��>�����;���\ea'��,l�:��N�#������%������P[�cQ��A�0�1�F6�G��Ja0k�r	���y��<I|��^~�
�|����s(w���J����a�^���a����A!FP�1o����o��BG�+�����0;��z���X��2j��PZ�]N��
#�UO���	�������N���d���o^�4�@���7*�^R�)�2�O���{:ud����p����oG?��������RR����"����>|�O�ap��K"�����;���_�����#���������x�����~��h�'����6;�� �7�����F;9~���o@�i6$uYp�&$�m�%�=��9�EQ��b�f��X�W�H�=��>��B�q�/ek��|���:`I&�80TE����
*�B�CT����T�(S�P�h%���%N��9l�Y����H^����m]���
�U&y&E?G�wA��@S��K�S`��]�x�_��2_Ng7u�W���&��L�Y��p�����"�A�1W��z����:f�(����
�UZ)�����O�:PH��5r}�]�x�2{�5��4�(R�$HIN!��c�X���l�ig��+�O�h�i6���x��?���q7�"������a�";��.�)�K�#:����X���G�)	M�j���V�(�pI� I��QL������������xr8�\<-�|�`]������t�0�ng��an8�������nMNc�0�HY�����M|�����m,��Q��hc���}OK�Uf�x�z��h'��{���	b��1��H�u��cq6X�&7
����}��q����-�}t��55YMK'5ugS��L7c)����$'�]�(:J�����LGk��� ����C���b�~b-�l���	{Fv&�����BG*�@��5MT{N�������T'���*�o���X�n�k�e��=�#�<{��);��������������Oh�Y����g�}����������t�������G����P�u@�O*��t2��	���x�d�2L'JN���	4I��UP����J�X����kq��4!��P�8K��M<����	7��	NR%�1��$4#��s��>�����:
�I�MFM*mB���E��,��� �.�~����W����C�~�y����
�|�i�S��]>���DW�31���x�x�����#U\�~���%���t`*Y��F?DGb�� ��.J������������,�'�������������N��(0t��L\���$���QH&u����,�����-�%��-��}��D ek�`���q���E��;P�����4d�z	eM�4Y`�6<b�)�����=��~�����Q�~�N��+��������.�����&4e��`=��S��`!y�T���xA�P�e�'u��S�v�@q���~�Q%���0�1��!rB�E���,h�%`�� ]����:[St��j*]r�����I)�6r��=~aDT��(����F�)�tw�\��%I	S(��kH���fQ��t�i�z@(��$R�>���R�t�IP�� pf���C�!����q���%�2^����.W*��b�z����B/��h��M���C�7��w�����������������f�~�<���������tSr����$_j72��:���{�l���o��@"�M��z�J+ha�Ff2���(����-]@SQT���"���P�\bK���d0�U1�Q���q[����:���I��c��Vy�s���<��*�WCRw���4�TY��du;<�PC������vZ�V�
��%��f|0��4�}�v$G�%'��;�!f�������N�&��/�g�%u�%����4(��������v0�%�C�CX��Y]�Sz4a�D��&�S�S��������DmUwhqJ�M]��v���-iW�6���[��B����|��a��=��!��"�l�;����d�!
�!7q�����[
����wS�yM��=������ G$���
1���|wv���{�����������.�r���4��xq��4kn�9��1�� �y3��k��kn7��h+Er��.^.)�Z��<������m�r��{����������c��������(r��Q�}x����s���w��N����Q�<[�����4�^rrx|���a��������)�|�o����������}�����A���2�_OG�@'L�gu�)�Y�1�j$&�
y7�Fj
���S*>0J��XUCa3rc��8;}���e	*_��i2��R���c�'���g
-QK���kb��Rv��(<S������?e4� )�l���O�P�
�	W�+v����\-|s���!��Bww/��r�,z`�C�8��Z)DU�UD%���(�B$�!o0�`�x���L)��B���<��;��9F�w_;�~��KU��l�K��>'{G�Sz5��h8����E�+��.��G��0�|x��4��tA�����
����.[WR�l
c8,��?��#�t?����������Z�s�7�V�8h��t�Y{#��
�Tj�Zo�Q���;��^�W}
�q����1���F���[?��LFi��j���q�p�/��n.7\�l�O�$������+�SLx������Z-��Y�U@�����k�T�Ss������n�R��w��; �����f28Ho�O��&�C�I�t���_���<:�m��NQ)�\��}��%[��a�����8nG�u?�c����5�dL�6��#T����5��N(CT$|�H�P<z���v����j�BT��U�_��e�L$s��@�m�LT����?���X�����L���y�{~$~�r�X�T����YW7�B3�������r�0�T.�E�'8Cl#�_�����S�>��vu��Q��z$����$�+5�A{�"q�����Gv�MN�����LG�h�g8t��R�7\�T+���>���\����M_h@�����-R���)h~���\r����,��C�D.o���P�F|�$��J�m2��=}��*�B�U��69�!1,3�;���h\�md�|Jd�1�IN)�K2��`����p4CI��bSL�#���r$��[�2Z}YN1��
�e(&����d�=����6���C�r�A2�qQ)`	r�/f�������Y���<�{�H�4�)��{���W/4�Qn��X�^��R����</(z��J��a?nK��iE��W���;(���&\�����7���QE8�E�8�.���lV���Q	� ��s�������t$����F��4f���9���}�"r3[|dYLv�J!u���3���%A�M��>��d|��\�x�?&R�]�`cNo��=8'����>��:���x�Vt9t���wJ���K�����TC��vaQ2D�s���Dd���������?0�� ��:q�`���R�~LnSUoO�,��5]iz�}L�U4<�����s"�(�
��ib��@!&\�J�����Ay�h�tC�v���p!��%T�ZRCL�cI�!��4��D�ys�/�y4Y��P]������!u	����5�z���G�<��?�\��}����F�����H���!�rk�b��@*�����y�{�I��v�l�����AWo����>���=\��F���T\<�UF{
���{��[tA�����[��	���j-�!��U�V�3���"���������)&��x��Iue��4�d����*g)���3U�R�&���$wa�J�t<��TLo�:�K�+Dz�L����=d�I`���HH%�1X�
3��j�
f��a��. �'s5�AA	�����a2����o��;�'h����^r4��b2]1
��d�m������2���JAy2�U����$�%����E�<yAi����������P��1�~b���Fv5���i�N���Tw��_�d��G�����S���.^��.	D��>�tA��{�Ba8v#��7�A�<3�v�3nru��K�]%S�
�t#��9C���e������-�"�3�Ih��&��NI�r��B<�en���q�����z�u���NJx=Fxm�O"���_m��T��wu��5i{T,�J����__\�Z��'Q��$������������R5�����k*-���(������+�_S�E6.{A��`�i4X��B&$Lu3R��ha��x�bxfw�_�*1^���	������m��b�:��s��8��LY+&���x�=�Z=���s�U���Ltu�r���ZRl����&7��������kfm��b��2����r��1����:I��^O������	�����ROuD�y��+@=�/�I��=��s�@��������J�g�S2��zr����qJx��������\B���uPJ��CA���$�V}N�����~v�\>�O:<�IX3�+v=�P/��[����S����l��`��T3���RT��7�yR�*2<�s��aSUL�%3���2�~?�� ����4��(�5�QD������_���.>��sS�a�D%�!R����T�$J6�S�I�������=G�64����������s��w �U�+�}*�H@]$����PF[{ UN=�9� �#a�<u�rK��=�p�f:!>`<1FR�D!�rg��U�*�����2���%�lBix�����N�EZ���1�I�HT�&wD^u�o�}����+)�����d��?%��J�4H��'�7����T��O�!
tQWcW��#�l���Zt�!n�A���m]bWCxl���S<�����C�"�ju��wn��,������J����vR��	wr_"���Rf;1��$�
����^���$�T��b�"�#�p�F�M���l��0�3j\�A&�;(�H��<2u���b���pH�&m�:V�1��W�A_���@W����H�} �������>��	����0W������^��%7)
�������	�����1��o��?Zeq��v�B��������������T/�|YYOu�'�).���g{w���D=%��9��A@�y��!%�K��hhT��"�yn0[$������G���,p�CU��xm���N������"{��l��������WaO�@{IX9�G))�Y��ZG6��	�Z.xx��u��lN���c����v��23�,.�~	y��It�K�����hn�-��)/l�������=��"�?��S,���}�V�������yd��EX�#�����Hg3���]���C�?������L��}����o9�s	I ��bT��IP��2l�3y��o)��B�BT�f3��W,����9&�@���aP���}�������d�4,)�
� $3I���mh���B��'�*����(�����C������������w�?��T|������@��N��Q@��!��+�K0�*pe�<��\��]��<�����F��O5�h�Q��(�r&�f��Ah;�S�����PE��kaP�<��{3��^�d����*K�d� ����I��>�lv<�+
8�7�!�������Ob����T�6U,��o��7���M1z�v�o.�2B��w�/����������o~��$��UW
j�E��	)�u���1�����I�/�0m���6�GU���;��l����b/$C[���%����`��Vp��um��o	�E�v�3K�6����3=����bG�j"��D�Z������]\�V$[���*]�NMZ���;��;���NgM���ZM����a�
��b�1;�@^��e:����+�j���n�.xM�����N��pB���;�(�I��p!�L;��v���J�z=���@B��=��#�-�Q��{$5O���u�X�	���:��p�AL"�1:��������T�|��,J��cbm:�U��ndP�)r �F<�YQidX�D��
�Ka����(��'z����gK���<�&3t`j��$i�&y������2?,$��9�a����+�I���g$�<��C�u���
0�`O�l�An��8���Ni����v��A���qv	��g�bgM���MS��&���6�.��J�G�@)�zhTY��|jr����CD��hZ��xe/GK���+e�I�$0���UK9�8l�'8�������^���{������r���BN�]�{���.�1T$�ow��v��{9c�6�+�a�����(vzC�<���LF��)�1�vU��4��3��U����QW��e�)����I��v2IW���TE�3|7�,������Q[��{l-�j���l�����6��l���w��������$�Z����O����ri��J�����6���k2
�C��������_?��y12�@��X��H���B�����{"�Yn�R� ���Q�&��8N��L����r���X��`%t�&���Q���-��$��������p��5��e�^Yv�?0���Xse�T����=���,��f�ZR]����nG\���<����X��]d�9���7j �P�u��N��8��l�i[u!	0���5�R��(����,��Y�:f�����S�Se����F����,�~<g�������
�����Grif)�\�ie(#N��|$ho���lNW7��x��Z�:��Q�T<j��R
L����"T���&�7�Yf ���<�Vx`)�B�XL�d�	���C�����&o[T�����"�I�sNK�<9�4���#�+$��p����Q�@����+�e����T�m��o��C]��.�zK	�L� kTH�8��{6
b��:RD�����Kg6�����5����jaI��iV��4P�J%a'���4���!����>�^s����y������3��n�S��(�����(���D�P:��.���(&�E����$�Sy�C�!�� �GY���y�����[b8���*�Ux������k�������p�pL=�
eh!���i���Q=�M�B����&!���<3"h���Q^�}��=l�\��V(����n���9���s���k��u*�6��cy
�����%c"�p��5sio[M��|����
��A��1~6���gH�)K�k���%���/���={���k�k���c�6Mu-����6|����$Sr��f&�E�"�P1U������*�6�	A����a5um�H���+��r��T���q�����,�2��f_�����msGp������2��\1A��L�<��6�M�����<�:� 6���*kP�O�5J
��S���d8Z�,l�P�G;Qy�i�Zq�����M����q��.$?��F�8����!�Agk��.l��Z�D'q��Z���@{)�����+u3�dDG�a��0�V����D��D��w:����m��G��hw���'|�G�`�l;�@�-�Z%����U����C���E}.�'�jt�Y�
�8U����%�E�v�a(NC0�Y�rP�q�<��};��(�Ri!"���2�eG��d"A,7&�e{1[���v�
�u&,�}���|�����e�2
�@�����X�A�k�x�d�*�9��@!aQ���#r�fi�G��� ��5>C��?��r����o+�f�L�7��~����8;%���	�m.'��n3��E�'�_���h�o��#�;6����m�+�����9���G8W�c_�>�s=���Y����"���U��`�1a����=�`	�w!��|�%��y,���6v;���h
)����U0�����A��8�;�����]1�qe�������^VgG�,#�!/���� �o;��#?���H��p�!}���Em��M]d\�r���CXC�e$��� l�U��68�V���/�
��skr�KZ�g���W}�S
�%��!=�
K($�-�m4��V����1�0:�-��Ya���=aGjm*�1Pi��D�mk_M����N�N��o.:C���F��)�}�N�f���d�����
�8,�*�]H�M��,�R���2�Uv-���2��2Kr`@�����=V����b�NmX�<��=���[E~�]����"��[��,�
 �w>�ba��2�p�t����
�Vd��Y��Mm�+zg�e�,\�x.����
u0U������t�
��M���t^��B~�3rn����
���d����:�����q��^$��"�7���>�6���@�V�,8�&��Vt_�Z��UR2�Q���j�I`D���z4���`JI��^��WB�*�K��%?!B&?��i�3%��\���pY�:K!��9����?v�<����mg�eQ�R_�&�CKkY����;�h���*+v�	�������d����X����:��|�1r��+��Ds�;��@z�����%����d6���;;�<ifEo5�l�F���U����IT
��n�a��i��+v�p����x8U5������(u
ZGx���8����I����e��z���	������ n0�*����hyk�F�g�a<A���@�Q�0�{�2j��@)��������>����`F���+e38��@�2��0����:��+����@�<CR��������1�17
j4��*��:�2��O_,��H���]�����b6�b0��oX��1���	T�w�x	d�O$��}W����)\��?C�6�;Kh0�JL�@��z���Y:l����j�&��@WW,��(�
T7�XG�p�s4��_�^G�L2n�Z�N����r��G�C9��:���9��r��'�^�*a��0&a_��"�J����>RZ�p�Q�� �G�B*�����P.�K�5�B��,�G;�Q��'�� � ����Z<�K3GN(b3�6�\���� �AH ������D��� ����=���tg�A5L�-��S��n��1���i�1;�6'tT�"U�A�XeVu4�K��w�v{4�b2��x�����������G��
A����h��U�fL*\��=U����T�'�-�Z���Y�O<N�xO�w��Ig\�s�I2UmQ��d�n)��	$��v���H����w���f��%��2x
��G��4�L��f��&3N���1d�-����4A> �"h	r��)~���"YMk�����BV��IL
��BZ����y�j;���XG���	O�jb��������Q<U���,�Y�F3���/`+8���P���t12�hU�}spq6�������w�fS�H,Y94�[�F�G:M-~JX��(�7��m��e�u�Wb����v�?� ���T��(�Bqa�v���f��|f��,f����c
|[�L�����4�C�\&���^���5T1�'���q!�84J�T�]�T��)���V�w�C"Q
�u\��'�P<5x7[
�<�.e�Z��50Y	��B��C��������P�R����.� ��j��V�V	q�\!g����L
�\��:���L��Na�9n�����n����E����I��\��
��=����,'�.�,0�374����y<���U�u��:�[�G��#�Ey�����H����6z)�eAf��:�lC�U����5��u��H���\���UP�����\��H���p��A�@�	������A�(�U�����M���).Ps�B{f}���np��7M�������u0q����[�	�ev|N��`3�xf����X�c�"X~+��P�j8(��
eA���\G3X�Q���g�4����+U"	 ��*isMbI������O��AF�.���W��g,�J�
]`7����_����<���
�#UzK��HW��$^r���nuNj��usy�5$��U�^W�t��YUk�z����C��P�����]�n���;��|�c�����I�KN9���e���������*������Sw��,��r��P��1A9�kB�:���w">v.F�C�R�Kz��o����V�a�]�`��?]��b�o��c���W_�"]����};R�s���h�
�DmG��P��k��^��$�x]bH�������o�HoK/�nz�%C7e������-�N��@J �2�2.cFJ���������6O�n���sTb��d>�l���u�;@{����v��F��~6a��"���D���a���U5��d<��15d�����N�J����L�����fZx{G�z��y������}���|���:J����#�����;Zm��-(!Z��=�$\���a������{�GG��a����H���s���'���nY�o%���6m@�k�l6��������S���I�������D�`^'�@��w��o���1����iG�z=���q��<8��G�9����=��UL�[Lj71/lh5t&"3�5	���UDn:���Z����fZ��}������Pv�w�o����P_*�#X.eC|���������0#(��{x�=�`�����j����E
A-�/�m����}��������'8h��s�r������T��`y�8��~������2m���0�F�`<'�����[��:�_�
h
'���������������['�����m:����
n�/�����l�r��=I)���(���t>�G�z�������������R��H�lM���B�P5d�
�P�y�2�WzZ��k���v1y�����b"!l<,u&MV}��L��}�.g���p'<������6]��N�I>�����S�����Ns!V}�n����p	����v��m�sW� ���5���t-���GN9#X6�G��a�@�nF���	����V���5�����R��r5'�����w5���l5N�;�����E���o�24�N,e2C�I�Z������K�����a�~��[Qi��t���3M��Gu�^��e/������p��F!����k2�;Xo*��uH�Q��HNW=7�~cF�0a�u�$@�GWT��$;��X]���y1��6���
��Q����QN���}����,�C'�`�%��4�A�4�i���s�N��?y�p����X���{z�i����;��vx�u�5V"={�P9 mJ ������/�@�g9g� ���F'���!�6e@GY1��(X��M�~�^��2���Y�) �bb{&����0�`��	Y�`$�K!�M���el��>NQ���.K���KB����"�P*�*P���&vP������L���N������X�^.�4F��[
�!���j�c�((Qr�$w�9&"���a?�LR��q��&7v���A�\���DeerQ�#<1�'Z������Ke'�V6r�����B*e*�I��3+B��d�L���0��0��b>��\)����2!�0�h�i6���ss�
���d�dV�^1���[�6�������kQ}��:H~
���f^�n��-Q�w���I��=�Y�g��*��>����v�3D���R�3���G5�����6��G �s����P|>
��Ov���Tc����?���	�$E��R2t���Y����p���B4���/���X����M�FS��� &S t1Y����;����2����71��|�J��EQ!%���m��-���f+����4�R��p���4�E�_vA3�����c�n���ALC������$8}��<�e������SSE�0���*��n9��,�5�������Yp(pz�E��sq�*I3����Y�x�i����������o��e���$t>����F�T�]�~�r���an�^I�#4I���|�T�5a�"���x��FqC[7�6����p�:����h`mnPq���r�_*}S�e��[��e���4tXR �.���Z�����%6��p�������"�s[�V��+�X�������ue-Kik�.6E�
��.�����k�Kk�=�#������t��3_�+j�9n$��t+�5������&��C��:�j[�����<H�*Y-T�3{����\��K�X���+Km[Q�~^����������������~�����'�l��^��U2.�V`! �
3^������<l�:C���*�m����Q���������A�Xg
^��xh������J	0*�^P�i�8c����r�v^���{���a�i��`�QHMcD��Q��������\m���,E~��=�|������y8�����pj�:H.��qd@6N6�Y2�u��u���&������X}���<�z�����u������w�<m���o�b�F����N���QTkv���#70�����FSI3�:����%�2��6����Z�I��,&S�
��7�#`���	�pj{�eB�-,�U�8���
��������X;�b6��a�a����Q-j��=9�u"W7��nr�x*�eV5���R	�3S9�b�f+�#G��X���]����}�r�����\%�x�4�d�6kJIB�F1��n���>�uj@����d����h,�������h���3��XC�� �+H
�=���`%��V�X&~E�?�������~�q�}J�r�5�w��EN��/�(��R��eS�X��L���y[��Mf=�,4��d��D�*49�b�D3�H�_��k:�B������#='���iQ��Y�ip�^?��>J^�cb�\w�8J��~�|��}��H����.�Dr��V&��XlVAWE���5� ���B O<���0���b�KS���B�p�a����1W;����a�����Y�����B��ufG������������q��Jc����k�������EM<ZM�N`�J�KG�!-
�T���_}�*��]l}_�1��x��I�.Y*�w����h�!��
����T�<�wa��$lq����	�0�1�c��|����9�J�U�VM:=��������*�����s���������R��<V@�����m����[4������]��d1C�D�f�[���p}a�z�l�����}��
k*'A�l�t�+C(aU��9p_]c�������&��V�0MQ��KN={��������|f�>�E9i�s�q���z~��w���������h��n���-=���s��i�����[Wu�[f��������Z���HH��������m}���"tQ���kQ�*�q�-f��y>���*�s�L�)Otk���R&���U��u/����qY�����<P���L
�B��q����l�\K�+�I���7���p��9���M�����<C.�_M����_�gS��NeVX��j�:.�4�Nu����&������yV����h��������������"����+��/6�BBy|U���xd�2P-myS�KiHz1��������oI��I*It�DX��t	6����r�����}���}��C�	�� |�����\��
���U���Q'Y8��6�:J.XXS������AH�&���z*�	�����rAb-�[.O"o8�-��H�
(-a�R��3�a��7{����\������Ub���h*���[J1*���,?�_~���#[��h����s���>�sN���&X\O�yw��Z'�kq�Ar��b�r�ff�c7�������f�F<�����z��8U��T���
���L�J_\�(.�CJ�^<���t[.Y@��'���C�vs59�^,p�k�dd
��Z��\��Z�\��Z%]����=v�������P\�0�H�G\�D�P����5��]s�F%���w����O������C��T������p�xo��������- ���$���,�pk�����p�*�Y��������� ;���G?v���8-M1F}P2�r����9����,v��%8����o�.\�&�����m1�^o���,9h!VIL�)B��?���]�I�R�S�,����wo�i=�0�����N��7�4���=g�������������l�X�"�26IB�O��0��(PB0�_V�f��L�N��W�������E�����&M"u%��l�j��_�z�P1#r����P�(�@w�T�a�@�IkF��

f�!�m<���L�A�?f0
�C=P���u<X�2���� ,{�Z 6�^�����z���@U)#���hd�������z��.j���]�z��o�H�� B��W��!��%7%"����R����������:]|
i���7���d~�o��Hj���'�4�q��^H�8������u��0ruz_���z]��
��C��+�mY
t�	F:\�)���hJ��������[(%��j�`�Q�@T��	��!Ha1y9G_����PF��N��W#r	�F'}T���6���p�O��b]����t�76 '=U�`��XU������R�%]W!$��u�#�������B�e#
W&�����Pa����l.����	�G��KM��%��,����~]����m/�,����+��f+���
w��!���C��zM���~[��K4)�]���.*l���E�*w'�n��>a��0{�n���t�;����������7c��*�:oA[l��RI��8eJ�T��
`�<�OIBA�Uo3��hf�F���[e����y�P��IR�^�`R�z��g
,�[R��&]<�4w��p^yS,�1�%p>�;���!����o���x�p��6 ��c���w
UI,0)�Cf�@"X0%��h6��m�*��D�E|�*i*�Z��C��6��������{T���,x��o��r�B����XY(L�~GH�s$A$Z�9),\y9����k������
R^�\���T�k��}8L��z}�;n�N
2�g:����4A��&��/�w�u>��GW]����Z$o%?�~#�M�,���=v@��	�_�R�)N���m��T���e��g�.k�?8�����o��~�}sv���WD'��%2w������lJ����[�K_��8�������1t	��Y�t!��?]%?�
�������B��,]^I���q ^O�	Da��_�=#�`�D�V�~�9�7�RH��c!�|h�"����X�i�S
R�s������{�
L����6h1�����W��Y�P��T�5��}f#X�r�q�J��_���^3����'���<�'�� �,���@5�	f���������	������|��A��@���{��!��Xg\���.���.�K�TD��(U�7vC�%}UB�Ds��|��������@6��T�k"+�@N*���j�?>xt�FHg$88�+�����.�j���:�%�����d�p<�s(k�1���#��\So���@�?t���N���O/���"�Ls��p�uR���cv2�8F���h%v;��f��4)���v�?�������r��n�U^3]&sZA?r�+yx����7�?�p�@�����i5`�$���k���%�����b�t��h�M����XHmCN`��Jz��f��[���R�9{�
e:�`}������~������gE���	y�u!��7�r����"��9��Tw)a�r�����n�tr-�9,�����l\-�����~���`+���m���QW9���:�5O�[%����Od��IQ�~��������~o[����a�NJ���$?�Z�	\%����zf�����`k��N��e
����bW��	�.�������:��S1�a�T`�0�4����f->
����!����������s<gl/-��@�K�~�
aH��6hF�|��yVt~jL��[�_`IO�������E&q1�V���4\p�����E�K�NID�gI9�-B�����}ZlWawLB�<[���$�kE�������O������X�N��*3��4��7��z����������0���z��8�D�|~3�A���4!"�4��nl)e%�)��d��M�<�Co��)���k�<��o%r
�tdR�4p��'�����B<�n?�U�����f+����1����@�l�N�
������d�� 5MT&��Q4<�b|��?9���:������a��F�#���������x3�b�B���,��'�"�����l��ov����=w��ho����dv/�	m���(�W��wn�������C���c��^~�������@HK�d�S����q���z=nv�$��E��&��������vl!!�$�0���->&�l�L�����@�
.�^Y��i@9�R�x������M��bu\4������BE�Bm�A��;��Rz�R�\R��h��������?���:�����%���
�Q�t�e�0�f���-�@ �E?[�<:Mwu���4�ZS�{�ff�h���8�
.f��;����������]�J[:�I�o�$^|�V�	&t�Ql��fr���/N�{�=�!:��xj��5��9��/u����G�n�������<����wtR��;�~�%�D�x;���p�����K��q"�K���g<�%H�vn�9��
�3��_"����y�+�`S{]!�B��y}����w�_I:S�����������a���;q�k����:(�7j�\k���\��vbxLU�i�\�&~��$Q��10�;u��������S�p�U��+����.F�f����}�.%�=X$`�H�	� �M?������^��
��{G�-�����a@�*�����1�H�5����#i2N�"��.�Ov���'���9~�%��H�"'x�Z},n��;9�.�O�M�����lb<��SV�c?��R����X����5�A���	��(���c�_?�������/�3�s�gO�K���'f

G����"f�������*��;DpS�!��T�wJ���������x6���S���;0�Y���)K������3u��'�g��M���%���[e
���Pw��Z���p�0��;Q��������Z�[j2~�����0�:q1!����E��b�D[,�s �U�B�w6���ZO�^O��T �Cgg�e2[$���>��������r��Dl���i�������@����R�2I&D�Y?���;���>��\�`����u�4��>�GdA`�H��.����R�"��?�`�c�1��a~7�:
�7���w�V��j��������Q��'��XA�Y!�$O�j9K���u��?��}�>���v3:n���$j6�G����q28�N�� 9j�:=�F����h�F�)����nV�����a$��%*����=����I.%�4�����E���>O��J^��cx#7"��!��i����{�N�Q����]��O����^��^4����V�Vw����=�UY����G�RiOg7"u�,�3���u3D40��I�R�v�����)@ +��Q��N�����dH���`v�xLZ��$h��g���������U��������C=Ee��Z�V�J�-)�;S�t#��h7%��5X_uo�&:����{�R�������J^;f~��Q�������(� +Q���������T�I�fx�Gd�X���B���w��������7>�����p�PB�&��]�.��a"�3�8
!���DG��(���h���������7����?�M��>��f�����*�~��*]H\��R1PZ���q�����J�m8�{P�K�8�^-����P4ir�si�i������w{�n
+[Y�P
�=�H0�?�W_��-[)a��C�p�m�T�hHOajavGL^�^�\%@<�
������
�����5N�1�-E�\��K>�f���K�q\���N#	(��vTH�����jFG�]�~#�*@��B������R�)�K-Ubk)�%y&�����.:��]��W��I�e���2�����oR�oG@
rSA���v
��������`6����{�T@���^��WI){!e�
b�Q���:��K��W����q�����LU>5����Z�8�[Pm�9�Fi��Pu��S��`5g��6��j�W���]�P
]<�/@�hp$r��p5��P]=}'������|�T`����>n��G�z]2���pP���}�1zp+�yv��i<(����
UG�lI�h����x	x6��a��,_}7�{h�����f���lK��>x]?E��kG1�>aw���%������l����
��]�?�,�Y��_�S,�K	��r����n�!������V�����a<���q�qt8v����������$�����RV�����#��(G�o6P�wP�5A(Z��5Q���Q!H+Y@����'I�:����^�~����N�x\��Q?��\G��i���g��X	��q9��Of�cz%G8�5����C
��o�����b~X4�����O�B+7o�n���`k�z��f�����*
�r����8����~IBG �y�9�z�w����+�=��+��� ����	8
�����1�h~y�c:i�
���n�(Bi����<yi��������b6
�����g��16�A(�FN�c�I�U*]U�2s�������X��������l���HK�����q��� a�}JR�5%`D]k���d��Q�y�����X;�`�+�=��b:���-H9��]�K��|��JC�aU���������@!p^/-�'#gU	�����.��"Qj����WED2�j�A��6_c�S"�*j%V�{a�RLK��.g��(��D��K���>���b���1�o�S���������G�W�����tQ��mC�"���9�2
B��RD�*/�T���o��I��2����c0�x>��K��<^{�i�q�g�|��[~�����"��A<
i���U��"i�.�
_���^��N������S.v@�DaRR���A�� j0��~K{�AR����K��'3`������/�������q�_b4x�H���T���;����d�Z��q������������~}@����]������8��m�����x}p���E):$����y��&��O�mJ�fI�����n^�Q�R2���s0��h_��QO>���vX������/#�}��K�`gA��v�Q���1�U��Q�Bw$���<��[Q�f��h}���Y����7k�\��vV��6�f8_�HSz�8Q��gw��Y��%!T�Q�&.�s��J%�Z�N�Ul|���=��=���Y��	�O������:�%K+OYYC�E�M*���xp2�w����E�f a}Ok
9V[t=��X�������P[�gVW����fFPJ.fc�����������.1����u1�G����)L�_F ��D@@DkZ��Q(��8������1�����,�*�NK�����53���O�*4
�m3I����������b�.j_��-?Be��v���\�^��Jl=%+��K���������:�B��]�M��[6�=v�
�<������)�@%���S���F�)����k,�Wn@���'�7��l�1��R^�Q�BC��x���7�U~y�T�~c� 56s5�M�9�+eS��l����Z��B���>��V<]���������Y����N����Y��|a�������)[����B^@u��Z� ��!-���@KS�<A�,+N%���H�0���`���6;.�����E��!�^�J^0x�����A6����d*�3L��33�T�T$T������@��$����@�Yp[��y$��cWStp��E7]P��7&�v�B�IPMa]�>�@:%���$bB5��<��4�9�v����
(,MNT��.D������R�H|j>�~)���G{��$�!&������^6�A����g��d�6��k�T�~����d����^��$>,������E�_l���=������B�(��=������N�c3�x��`0saA�KrrVV'&?CwF��P��V#d�9���0�wSJ��k����QLzJ"�����3�~�����mB��t�A]=���tF���H+�w��������e�
��:��V�N�i)	Y���n,V��+���-
I)���b�������������	���+��u��L��AK���}S��J��.r4�K��DX�f�JR�>,O���� �;<:��������j
�R~OE���UBP�<�lZ$�y�A*�x0M������~��~+�����om�����`�����? ��Q�=RA�G�^#q0�	�{���S���I�Nq$������������l����<�����,��?��B%s\�)���[Mp���5P����s�e��Y� �_`��4`����P�T��
*�4�^������Y���E�Z���'��e����C���Z��r%�7�K���W�Lp�j��S���,�8"���"$����\q����5�m��>��W�g��k0K(��U��u������"�P����PE���\wT�2�o���U�
��5����Tt�2���O/�/��������q�5a��	������!7�[���nU��/���������snP�����.�O)��\.��sV���(�F����(]N�-���W%VpU��E7|uA��~��iV��U����1\Pi�FFo_�fH]��Ag-�
d��n������s�h���/��2��
���T�nU����0��e6�2g��y0��{����U��A�'�g��O!��Cv��=��Je��&!G�W�Ir�+���s�w�DV��@7�Q����~����6�������6�\����~Rrmn���),�51�*&����|��O�y��W�I�����B-�[�%���c���'�p"�U��cG���^�z@�N9p�7L���u��[���������������pJ������j����M~bp(����$�	����S�b���i�,r��MT��'�Sv�r��_E����w��=�����Yv�i�������xv��w|Vb�/G����R�d��@VX`�Dn�q��
`�}&����jLK^����M�$e-�>����y!3�y��{����s�����Q�8��Q�PD(�%hY� Z����u^��Wt�l����W�����z��dD��@������	����
wl����:�&���3�0��n�����.p�e$$��|>[,un��<�e��l+����
%��a	3�m�Y.oO��	29�A�5�o#���N��	�;z�����W@���)zl=�'z�����)�iMO�p�[�3�*�jz��>jX3f���y.�����I-y�4��<��%��m�.3Xvv������!�����+3�����4)�mi]��P�������6	��M&A�zN.H�m�G��D���r?,+���Cx��x��NJ!"<����I�Nl�_y
s�#��N���@��s(<Q��'�B���j����
��i��n}`'%��qz]������P��(x�Wk�������q��N����i���W�Z�h)OQ�������el��.�t���Q%��BZ%V����m
�����R1bP�aR3�f)g��Y�9e��������Sw������I�x����6���f�����,�v�9K���8R�;��0"�P��%�����'��t��*�������"+"�+5����c�9<����Q�+�y��� ��-oUb���A
��l�X<�*��+{��WjZW���W���W���^�w2���j���#�"���<����$������<��[�d0aS���-�O�.��S���,��j${�����J���t�U�S!�rOV�Q�Du���0D�;^�������T�jdT�v�.�u�q��k� ��/��p�><��$9�R|�G�/T����
���&��X�����rW��'�4�_�d�N$�[��V#pU�A���4N����F����g23U������H���^N~�_Q�"���2������fC�Y*� b���Snr�':aS�?a�e��c����0� 6�<�J%� ��@����
����qEv=�O~���Vk��u7	�;R�cy�������s�v�d	>����9$�#h��L�8X=���f&U'�)��z�}-%)��?M?���0�N 9���JP�3��N�P����������!2y��������-n)m��KL�L}a�Wt�����0�A%|�|��b�C��_%��/��'9y������qR�Yrs�+��+U����O��;(���B�����5Jq������,NI%��A��A����*]�L������%Nb�Z��s	�%�Da ���>��8�[v'��X|(F��������j�Z�r��|81���Q�(�������n�)����p��z�U���������]���%K~q��~��U%�@'��hQ�c�.ln�A9�0E^��_�n0��!'�Y����],��H�R�E.� �;9 R#���kB!oJx3�~��;�����WT���m�.�y�$���))���`%����N�SP'x�����F1��D��Uf��H6������u�l_]������t���>�xb���+��h]C�9��v��j����C-�_��L���	[d���$�R�a���J���i������Yf��
_�"�'�a���{����]PI)�A6�e�	u:e$��5�14��$��F;���t4�LbY}r�ZM�s���x����������}�:��vU�������CTxF>U
������%��p�n8�����5�	�Td(��� ^��f'	��q2!���a?#���k>��$���#�z ���������������t�i}�W(����?�w�,J)�H�	3�Db�{������@�;��9�zk�<����v��$4I AYh(s�srD�����}�3G���P���]��`Oy����H����N*q*�;R�\�&������x���>P�`�,��$4K����*��j������Z�^�f��b��$7n1K�u5`�c��G&�����%����`,�f%)�����l�=_��38*Dx�k��b���s�?��O��E2�������M�������7iA������G���Y>o5�G��,?���B;����u�G���$n7����������^����?>m��?���a�g���E����#'yMu�^�\J�O,L2�P�!Y`���S�)#+*�2i�Ac7P�9��N�iKD/��CH�w>M�I< @��VXB�@t��K��~2_z3������A:�z����z�r����u9�|9���Z��|7j=�Jx��!��)�*��\�����Pg6�L��vW�kd��0�-��e����r�?n��V4�"S��d��������e��6O�,��u��N��?j���v���*Y�&�[��V���<{���)�UPcZP�E��7��A�=���W�b���w��L������1/�<p��?O�\�!�0c�cn���Yu����!�@e���i��
R��Y�4���P:��Z��r<��/�dI���������+�]����
�]�ca��4����]@z�G�������C�g��[7\��(x1�M)U0�M����������J���iX�Eqo�,�{�dQf<'��6�b���{w��x���p�;��f[nt�d���Sd`yi�fB�WL������
��������F)'����>v�a��s�R=����b�~�
���L@��j����@�������v���
�Dq���2������p�;"7B�9�� ����
H���g�V��� n�@�6�G`�����e�v9�J�\�|N�z��5N�n;S�set������x\�;6�O�6����N�v������������
k�Xn��K��tq:vZB����k0�����������d�VK���|V
��j��	1=�=�sB���/C$�a���.��6�����7sY�}c��r+�]���]�F�VPdH`�!��\���j>���IS
��|'GKH��Vs�na�lz(<�NE�=�qL;1x��Q��hP��F��*��WE���I��P�.��m#Q���	���5Dm�1������M���Ls�)�$8���t�A_al���^ywZ���2x��2c4�/F��������g)�K.��`���5�����O��6x ��0j�Nz`XhE����3,4�`�j6,�����N���v��K� l�e�E�u�g��������t��	�)!��o6���M]I���s-��;4���?6����1�����}�����V�u:~��������������I�����?jE�����~3����;������/WIM�o����O;GO[����MU�1@JE���tI���!�y=�iL~�	�I��M7��*��>�/�T�k����d���^��+��z������\E2p�����!�\9-�����,A+�;���`�@���]>6�;��:j��TVX2�q���Ru��_�!���l����n�����Rp��j���2�s�9���
ny�Ol�r��i�7D���(���I��j��u��y��7��X�
?���>�,������:}�J�������_�x��o��x����)q��e��Gy�W�����_�{��������\����`?.�0`z�|8��j_w
r�@�!�g�#������L\�~��L��eSTc��A��B����R<��<�<�V|�?����wg�x7�Iw.�y}��3�Q�7:����R���|g:�p����������.L�::�4j���op�v������D2S�e�$Vm�m�D�gW���Gw�=�6��?/��wg���UK]�u����a-���:}������rr�5���t����x�������Q���]('+8=�A�QDF9G�x&@8/������E����	v����oO���	F8������������UdE!��6C�^#M��F���G��~��F������M� �F������4
�i`z��40�h�m�����(
K�N-b��	Q��%�?����O�����w.?��}��w|����eT����z�H�S�����r]P�A!�_B�
��>���[�H��F�z�+�����&�UD��!�]����3��E����CC?���������[}^q}%>��\���b�K���$�Oi+^}|��Bp�4���8��{�P:OE�:_���~������i��S���Ek�����O�v>��z�% ������1��=�o�����}]�	�/�\����?���j��"�#R�:�Q����c	�����
�d�dU.�=1]��@D���4��2�|}��%�7bZD�X|-y���Q-����.L���(����AA�9�gN�����~��[;�9��}H���/b�c��=�(�����#����o��?��C��X�� OZo�g�d_>61��L
8R���!�R�lv
���@��d��S�j� ����<���{����su}>��]��/�A����)]���?���yo��U�����o�����t���Z@=M�\l��y����~�S�����,�9����x-�������kx��ibgoh�����j���1�wt�������~��L����q����c����U�g���X����b8h�����*���agp2�j���tRB��@l�V���6(��w�t��!�%����<F	�����O���ie3]��"�=*��P��&�M���P��(��+���/oF�C-��9�/])�9��}��_�t6p6�[�m��`�d;���C��z��G(��O@t,����T��5�HSC���w@0D2}�u���eVz \�%��k����~������U�X�Vp�Y���R�`P_'QPm�U�Le�s7Ap:��x5�R3����#�l���������#���~�2�8�����T�b������r�������C������.��E��Zv��SL��>�������k�����c3��d�(���D����w~����r6@���f�d�v��.�o�����������%��O����!}W����+�;�
r����m4����b�-TB��`�f�o^�����q���fB7��Pk��\�)�������B�����d��4q|Mq�v��hv�����fg��:#�s�T�fw��l�J�iG[8�f+�F��F��o��;�}=r\>�p�=?���[��A�����f��������@%����-A�������P��{mo�*�hE�E��/x*C1]�X:"x���4
5����L
���T�~�)��`����#���"%�����.�$�:�&%(uL�Qr�xP�����A9=��c/C#���C;NNF�f5�=?�F�>��7D�C=D�>2�#ju���i5T���shQ��i���m�g�����A��V�o;�}��wT��f�\'T��'wf����w�����lx��OY'�{r���.K~_�������4�����������f3x^w�H���)L H<���c�50~����(7��9j/q�&��j��&��"o !�����/Y@���'up����&������i�7��7���T��+b��,{�pCxv&�;��']�����B���i��L�?i/C���/PO�tf3W��%�6���
��s(�z/%��g9��,����%1�
 �����s9|N�lK��H�H�<@Z� y����7�3�2t������N�I:r�M�V�
�i��Q��m��J��w�����-�=G8����������;����M�P��^l:�K������l��FL-G�+dj�R���lv?H�}5(�)�����������Sml�@�>-��Z����:L�x��#�\�+'����������U� �?7�b#uv��`�290��0�6���a>"�� L[���fEr"J~oD�f_|��f^��1(����ukW-e���aC��V�@:���4���t����e���V�|BB?[\{f��7����������ts����r�^pAy�~(����xs����r�^py�~(��M��O9)~�T���`��@M���j�QF�F��}�������AEV��me����}���rE��V��N��G�����+�����|j�>
��y}�*��V�#G1`�1��(	�Z�AQ+�<P�|����b5M�����]����>��#p������b��~��SBr����l~E��(�0j]
��U��Y�J�������
�C��@�wwA��4���I��s����X��(���������t�,9��V,���'��=+�z_s3�Q���4��M=�0�I$�L��w4Ue	�g�Jw%I�)T���y&�(�����~v�N�d%�V�Y�L��nn����d�~�����B��j�2�v�r��V{�����9�,��6��Z�~Hnmo�'�C�4�{�@2��&,P����x\��������W"����������'��d2�/�����l��B��IwZG�/��6"`%d�"���]�����	����,0�x\o�k�0���^��@�=��a��;��������S�0]:o y�(��7�H����8�}���9{kd.�{`���P��i
�<�.����6��Z$W�k4��j��c�&�OW����g��vC�{��R���9�������N����*(�����Gg���n<�����C�6_�=���\m��T>��a����1�����>U9�k������]I��
�lT�W6��KG�
����������q"',�)�tN����M�/����c�������c1$ys�&��ut�6?:��4������u9R���r0�
�\��n7w�[����hB��M����27�P�P�llYA��?o
��697��N`�@�v��"�����I����������\������W*23���b����$J����v���,���;�?7�����S�Y���q���}�z�p�L������S�*+D	���K����l%������ur^#GD��I
��&�;��������>O
���|���S������+^j����v^���S�������\ow#�����:(�/#���=?������������Y<N�~R��r���
9�2�]��\`���-�����O�.���r�4��Mv����������/��d<` �z��s�|#����+`L��������F��WY.��V)?���9�mW�t���E�.���y��<��+o�������t��+^���U����=�� ���E��h�����-�����P�E��u7����r'�������$�;W�
��o�=��=�v�	��`e9��)��`��PO�2�b_������d��o�b�Uw���	*:z���������*����|���j�_�9��I0'�F��6���(�+����q%�����A\	��.��6��������w����P�����"�p�Qy~1	{�J+��mr�m<���&��V��3�^��;DT��8��������%=��{���l����z��A��P@u�
���DwX�c������]/���#
�����<�	�b%U�
�$j{X��^_(��~V����i��_�
������_�2������|����o9Y<��n3����1U����4d[E�7S���m�K��D�^i����r����P��b3�V��u��M�[x��M-�a�~��1r2MY�uF9��dE���M6G	��R���Ut�����R�O�m���G���Q:�6���d���v�5(�������X�V`��R��>��b+�;#I����sH�����!	g�m��F����9>����:[��3S���	��Sh��<��W��������W
}�W���'��v��Y5�`U>fn�����w��y���n��CgV��B�~�u"x���v�f�6�"���p\R�:#L)B�~�yk)�.�.�|N�
�~�h�2���0�9r���,V��a�i���a.1w9�d��'���o����,����*���s�/�3���)�n��5�m�*�O�l��y�/���5��@$����y�@p��+jl�B8�_m�0�i��l�h�>�_G�#������7f]C���_��i���m7�;����C��G�S���
o�o���Ge��c�7�S���	�qm�5cn��U�PF�h������Fl�tX�A�|;�������w;]:�~���~��:��m�����[�,��[�q����������G+�����6���*k_�����������~Q�W�����w"M�_3��n�t�p��L�	���R+�\��)%G@��F�\���|��F�����.��
��l�5�E,��m{����2}�d�ES*�������z��^tF���Y7��c��u�����g���S���?�5mo��yZ[Et������@'���ZGZG��e��	�Bh��u�p��%�K��6M���s��2����F���nyh�r�l���G�V�[�h�3���u��f�M+�x��h�����z#�h��v+_�����l�6��TL�g�i	����%����?+\�K��9�^���b�Qe�#U��|��{���M7K]��_y�����G��qzL�g3����|��(�3E>�A/j	��R�@~�ds�����\��Zwglt]T���l���y�=��n������6(����h�c�3�ap]�cd.�M�M�7������`���D�'Rp���9��JG3N�����a*�|��|?�iz��?��b�&����=r�M�B~���g���2��0�C1?V��*�b���7�e=��A�����|�d�����VV;?(�
b4�~gn���H����1o�������������=���]����9�{3`?W�	oy>/��
):��`����E��� ��b��n(��t���0/��|y?6E,��L�`�x��1�v�p���Y�����_m�X��t7�s���� z�2Z�V�����*���x�I��J0B
���|5�������;1�9/���1�����G���ww�O��
n����pz�.D���W�!s�����x������B�2���b����%��pn�b��4wYz��e��EY�3D�7B�\�\)D��K���U��A���Lv�n�T����47D��\�x��#S���2��K[m��:2���!S���4�!2�gk��Z���*�_2���)�~���~u��g#2�.p$����m,Q(�v3��u(���/)����w�7
X��r�-���[��M���	��)�E%46%�`l}l����naa�w����=/-���/-���	��m
H?�� ����H��=&�����y	\�;P�&*J�����\�Q(�r�lw9P��qf�{�%�|�u���P���@mw?�j����8�?�D�_���qq�6<��+C����)W���D��y%4&��"� ����1	W��*#@U�&r�iL��4=�~T������1	�����Qj"���d�
N��I���e������"�m�QQx��=���(�Q�^KoT��F��[L�P�B������Z1���0+���,�>��K.�����\�����2
~���?�VT��Vwi������2�:���kk���h4�+V�9lr��n���q4�z*W��zXgIX����`�u�n��s��9�s��9
�9��k���
kj��-�-��\0�0�`nm�<����X(]R6��o�z�S-S�~~���Z�5S�@�(�_��J+���/P����k�u����
-l8��(��h�Q����L��xJ�p��Q�td�%�i=$?�%�d�R���r���?\V�Y�nLZs���N-���1���k~��w�~7�s��y��~��_�G��$�_������9���<���G
=8��r��_�����������n&I�^���aQh���oX��`��mX��`�mXt�K�W#�g���z��e�f�������*o������XY�2�����x�����
�A��{�c��E����ro��%*�S��b/���0����:]E�g��
�����N��������S�c��y�U9����:� �r�uO=N���/��x��{����lF�ySZ��T�|(�1���e���'���bg<&����!n6y�x����;���r�]L���������4�����
�*A ����E���2�o��%�U~���%���6�JW�pWtO(�5����W����D����.
�F�������([v��MH�;a��
~Y�8������S3?�=�����<j����X��C��_��5+���u����}7�/�I3<Ax�	.'�������t4���y,Z�}��/�n�(<�(
����w����!!
O�`����c~A����
�;��8�������!��w+O�r�>2��o�h�'���������/�����y�<'�}��_��	h�����������i�:���wx��0�4s�e�i~A��lJiP7��"�W����C�E2���O:���S2�d����b�7��cqY���Gd����V$���,���4��M���4q����A�Wh��k�-�v��[��EX�B�1X��\ �o&��ep���~�����o�*L��"a�R�����Bv5��\��@AA?����S4�6m�^�s\��?���U����?N>%�d`����y��2�S�Es�;/��6;��~^��+���#^�'��+`{!��!g�0v���0�����1����C��(��h���k
��V��5[E��� ����!��8�M �8g������- �{�	p�����<
��G!
�`O��(�(D[�(�(��X|��QBa�QR�(�(D��(�(�uWx"�(�9m{B��c2����j��U<})�-�8��o�xs�S��Y�mD<�*%��?���R�E��2&�;��r�qk2���K�+�%�����<��:_cT+��Pv��w�J�C�`V�8)�]f�J��p����d�-�~6-��3^�B����-���{���t�m2�;��p������9�������4�����D�U�e�/3>���7������s�7m�/��DU�ye�;���=���-S�����=����a<O�o:OM���Vu���k�]��+������,�v�E�����^�&����Vp��&�N�{�~!�pS.�xc=�B�3����!��5j�j���7�����[�����Es�7��oa��Vt�j�����Q�}�;�#h���}�r*F�k�	��[Fx
�����_��������#(2X�4��U�����]*j'{`
������|C�����F|O���d+��d4Mn>�s/��[�r�	�����:G��}>�s8k��6�]���D}N(�[[>� 3��0
�.,�Pz�J� w&�8��Ofw��`9�_����UF������Z���$7K�M+����M���s�o}'6',<�����%G�`��,<�/7����f��c��_'�D���l��*1~��c����C!6
ljln&�SW9�N�M�}�p���~!��2����4���g�l����nn��M�M�������{n���Vs[y��*���j���[��\��;����7I�c�����(��O���P���t���q,4��I�oFi���|��3}}zyvq������#Lm ������`��?(�&�����8]"n����'w�\��}���R��q� N��4��	%M�r^6���b������\�gbO�U|[+���wks?����~�"�K����,_K�^G��B�|�������!���8G����������c+P����������.�Rr���~��6�~l��f���k`������aF�0��[��s�k)n��.��(�!�3�fD���q9w���H]3�[@k+��F�1�����R*vh�o��+�����(���A�V�[e~����a<�|��������z�,���?��\��/�%P-WL��I8u��Y�!t�H71�#aZ��G;m�#�}���tE��~$Uw�6#�x-�V#+UY��p�y����:���,�T��sg�	-;Fg�u��
P�9H)����<�>-L%���,�J��T�����=��rV������"�L�z9n��}�k*������&�������X��(�f&Jw
��F�L����$7>w�YPbN��4�9��=�@2H{�Tr"T��`�`����i4�I/��p��2�m��@�i0@/�
��b�p��fIK�o�4w�P,ap��$�k�R���!�
,�u�mh���U�
�R`o����w��qA^Z�;���w����_c[�-l��������C���n[�".���{j5�x����;���:��H�N�-g;��v9&c@p�MgK�KD<��G�@�(]�I����WI*��0Y�O�&Xh �,l�(b�6e��ds��l��d���eJ���YJ���|�>39M��5&C�u=�s�?Q��ZYd�t��.,N������.F(����"Th�J�����
��)
C����������s)[�6_�r�n�->�y��-W��bw���[[v�2�[�Mg��Z�"`�[l!��Mg"�i5���r�&���fy�$S������w&a�t|t4����;�����r�|��q�gp>$?�w�����xw{kR���6q�Z8X^�}dw�s7�i��=@N���|-��rT�������c�q1�I��
$q
o��N�����NLZ�I���J5��L`���*�����,�k�&��p�;y�+<X0/t��5�I���\�Z�tZ�*�;[-�g���b�,\B�j�~n\�l'Ym�gC�|��g�!���}�]_�������S���������9���??�>;�Y����baI&W�(��;��4�t�v�������@������j4�lX��uj�8I`6��K	��'����<��b���Q=�m��=��X����m�a�z<�B��[�����Z9��Qc�0U�t�UD���O��N�)���_��O��Y��J�d`b�0X8t��Hw;S�L���:����*Yvc�|W��][h ���&_��4�(YmM��&!\�b�j$�� ]����H�������x<N����V�D�J4��P��$�A����I%�0��v�"w��X����r������f�U;{��X���S1_���>V�����9�Q�",����#v-�+�H8
5��>����d����O��&��b�S��<���j�E�~�J���A;�L���j���r�g�T�����?�g4M��B���T�{��X�]���F�<���p5��:�qW���x�i�������Q�=�������]9�l����`��/g��b���8����������������c2��?�I$��n6��1���'	n:����V�R�$����5'�q������::�5I�^����Q�p���;�9[~3<YG'��<Y�W��,*�o�����'	��r1��J�a������Y�������[V����$`'���@�����A���
�� �t0�K.������S��9�`��U�d��[]:un��>c&�u��;��l
OB��%{��p��x	U��>����	~�5k��q!b9����b�z�^)�X�RY�Nz��{)�r��xC�h�z��@?��5Y�.eE%6u��9��}u�����uc�qq ����vQ~����E�lo6�����-y�m^� ���~���o�x���4e~f�2F*�N;�����5Q;���	����sTY+�,��2I�g���S������8���dPI�_+xA�%��*M%��PX	'
�n����k�D���>��_��Y6����"?�E^v���,�/�i�� XS��CF)�y��D1��U������Sr[���j��C�d��f�
A"N���'��oe�!���j����Zh�(����,	/���������
6��W^S#8�2�(
�,������V�G�?���|��:%��p1� �e����yj�~��j]�/0T�s�(�O�a�M�l�J������Pb{���0��x��u�5�w�eN���dB.k���s&��h�gb&m���z�G���
p:]�^|���B�5�3]M�r��D��J�����S;L�,���Kq�����ZK3��nx�����+)�y��Yf�������;�?�_��&��w��2��z�����h<����"f{�w@d�5�{�?��I�O��VN����������eH�)����d���WP�Y�-�Do� �
k��ppt���;	EzZ���B%�}�Q��7N")D�U{'���EZU
�{>���"����W-��������U��a`p��-Q[���%
cK���*� ��Z�6�[B�e�;1���:� 1��{x�m��9:�(>�4�{����m3��{H������W�
��������:����x�E��^A[-���u[��,������3��a�E-����Q~i�����w�3��S���U���Q���:��R�[�2+�zB��[��qH:,�42������������?�Ds�tuYMs�zG�M�������1��}�Y�|�e(�G�l��1G�Up��dqEr��w����r�l8|����/���_��_�_p[_�*{g����{��M�*m"��\������W��9���M�:�I@K�A�(��\��$�[���Q*�����M�Z�@���<��<}�dk}�����h�6�*K�%[�UG������(�U��oU��`���*�f�B���}[���,�t���u����"����O]U�<�s�������&M@1��0�/P��q��t�����u{_�Q4�_�4����=4�_�<��kRm��B:v��k���rt����7������uU�n���9�n�e���~7�������c��N.�Q\k%���y�;^.F��$d�Ox�q�N8�-r���-(��M�i�6���fG��I��_~��3�U�Q$Y���x��'DTE����+��b�F���b����*��P1V��q�P�M��C(=V���/��x��{������y��w���?7��2��(;�v(�0��b�)�z���T��?��!��!X'r~���[d������Nq$d�ay�|J���s��<����qXVo�2�������u(�
	�:
�A`�����SH@Us(�c����C2}
�/D`�D8^��q
N���q�VQw90t'�aD��E�����R���s��mD�0D��(|�=mw���my�����c�a���!���!H����ms�����]�a������0�?�lTn
���d`�����[9����V���r��4.dm�!:�~3Q�k=�����n���rM�������}�	\����z/��<
|'x�?�����?�]�����������\��`�.�����
��Q�)
�Gq��Aq�?��}a��~C�~��b���C���1���"��b����~���Q�q9v�Y?���;���������Uw0�D��9x;$��y�/cI/F�x�o�O�:��d�.���y�i����<:l���l5��F�-��?5��Q���h<����U��B�)N�W�Y���u��?�B��h������q#n������i�V�h5�$�{���VS����C2�#�h<�����
t�T�2��7�D���Y���$��b���h���g��W��|*.WIM�o���Lv�l<m=m���F���|X�����O����^��^4:���Bc�E�C+���$s�sq���������?�,!������8�+Y�L�
�lO,�l1��H�c
C�&z�J����{=to��Rd��ll��q�����oT�5��g�'�!��b��s`�������*3Cd_����X�x����!��x�8u�����J��K�r��.�������ff�!&o-��-�~�u=�V��$�_+�J4�@b�O�J�*���d����&
����e*{���k@C�@�+��������d:8���������j9����/��/B�#������E2��	�'�v<���?^
�h�(����.<_u�?j3�K������V�)q@
������ '���n%�~-z��q��A����W�7;��A������#�~mSH�����$�����d�x<�:����G����BN��$�����+yq��RP-��p�������W�x��R�K^p��&?/�����|xRkFP8���k��\%��B����"M�i���4Y Uy�������eo5����%0��O���"NO���g�<���?$Y0A���j�'�}�t�0]�9]��{�D�dl���BV|��W��Z%vh�T��S������%�����\��9Q�������`{��f��HmP�F���}��B�q��"�����E��h$��Z�x�i��;9o��r����f��$=�r���8x"^��P�D���������h	j&j�s���O�T������g�o����|qp��>E�9}�T�v���e��f�j���{]v������9<$���EQj>T�sk��o�)� �)�J4�H ��_1�5)���H=�M���O���?�����H�,�>�>,_I!t�W��J�Z��M��M���9����oUB�!
�m�5��d�
�G�����{MK������LF5_%��d��y9�+�����z������[(�W���77W�_�Jr�����oV�f�Z�����FJ/����,�E��<eEYU�4
(����3Q�?��p�-�	F�&�����0�����"Z����fZ�M���%�o�[K������-b��a�W_:��GO�o�k�l}W�g��J� ����o���K�7�o��=z�O�+����\�\-����6�8����n\�����	���o�����o�*��}kX[�)A�w��a8�����?��z�d��l���{?���&��r�O�Q���x��3�O�|�7��T��S3�X?�������������9|�);�����L"�=�Du�SU;����]!��@w5�u5zSN�e2C7�@�0�<��F�(q��U���/:ds
�b���(�^���J���'j7��go��*���B��
eU� �P��U�����!��5�CW�HV���P3�uL�|d)�~�v�i�d:6���?�{�<�X���77x���WS�J�A��UYu�b--X�t�T?M���}q�?�W����'o�C��H�{S.���Z�����ZQ1���LUYh��%����t�_�vnE�
�)����H�#���\�F���p�r�����i4'(5H&���bV-!U�++������������*�~A1�o5?"$�� ������fOD��wF��:�����x��p$���=�M��T������)�.m*�E����������������_�j��Od���l]�����J�=���h>a-��2�8��G[
��}=U1�J����~��a�j�����\~B����/�� 1�?�nW�y�/bi�I����s�����Z�K}��T�UU5x!5�[�����p��U������c��2��x��3cQ:���������*�
����k+�
���@��=i=<��X,ng'=����&���q���3�[����rf���fg�O~�=7�D�L}���}���%�X��z{�6M���TK���J����S_���}���J\9)�-!���/�=`�����?
��2��]y�)���������(�+|�Dn�%Q"��.)��[)�����?�G��`���2�V��W����������d&u�����#L�$��o ���&���
NN�/%�����*����B�>��$�q&;z�	�X{)N#��G���Iy�r����z�V���0t]]i�d�����
���h���������wh�@�7�\N��]�Igb�^�p���:C����$����Z�;�^D{y�3d�lV��h�M���VnO�p�������[�6�����.��e~��{W�%T�|Bg����?l����m���j���cQ��6�D�B�z���~I@P�q~��d�ss���|��I�����u�t�34�D� �R���S�����E?��_W��z������u?�AE��r�gp��4�u�v���f�w_	�k�z��o�`�P����R�����1)��
��t�|����S��bI3%p�1Gc7����1��3�D�o���1�Y�@u6jyz9�:�'��������{��9-OMV�_ �};Yd���j�G��N?���c="�wG���BNx��������y�p���j�A��Oo�_�<���,3���S����#���wv�I�o��I��"�$k�(v�E����r)Bc��90���(��!��/���J�����5�8�6�+�3x\���\!��+�w���<���p����dO�|�1�6V�TR\E���%L���$"�\��\s+s���UF� �@�{E��E�)��	��aDDe��\F���9��T�����t�{�/�5�@���JmY\U��_��~������r�8_nV_����b�JJD"�Az����z��0���������nx�|���u<�h$�+1]6pv����C����#�-�"1���>~�$j�C��N�a`Y��Ec�����H~��]%^���+�#B7,0�s�nf���
���S��r��������
�������_9����q������_JVJ�s����n9&��)���b��!G��`����
Y1u
[�W��B��N�P������!��T���dF�U���]�Ji��z`n����gx������*�:���5e)�CU����N�f�����t' ���d���-����yN#���q�e������H�Ljc�*���u��D��rfx!�������5�+��oy�*��%��o���3��E��oF��7�Yg	��f|�R���hik��No�S&��m�"~%FO]��oYj�K���W�y�[Y�\�[�@z��Z����jX�8���Y��<�aQ`D�F�/�
������$��A��2�[\�J��;/����Y��(���u�P�,�5�5�*��������nJ�}9�z��=��>�~��#��;��V�,��>��{���Co�����F7���g�?���W�����������4�k�����#�.����UY1���g�Z>�3���	l2�UJ��W�����M�}l1�����$hH�	i�������uI�1�n����N����
���V�g�T5���~�-�34/7��1{�f1���>K~c^7�+�2��+�}���n$��A����_�?+��:��k������
_qBVW��j)@g���xM��f���5��[����U1�x��T��������f1c��Q�O���8�.&��@���F�����I��9,=,p��\�������*�������cUWbR{�[������D�\s|�����5�Hv��7�|��m����vq��??�P��z�����_'_�dX��P_TTU�p��z;��Uv%/��s�^��<�8��W����M���U5���,�8%�I�V��J����|�)�{��U��O'K�����J��*�Z����8���wP"��zn���bt2�������Re{0Q?�����qW����{v{�w�jl�O�W{[vU�������7��J��p9(���#PbS��#�*����7wX
�UZ.���+��
�zoQ�D���(,6w�E�~&j{�����WB�f��Mzw��0��������Tl�E�G
�L���9q��s!���L�����ASu]��	�"�%��ih6%�5.��Jx.u�Tz"�W������������b,Z�� n��:��C�?e����1��i�������I7��Z{����mL��|����Mm����Rf�%Z2����[�s#C�|�E�P����o���6�������{�A�M����Ef��tq�Z3����t��'2�%!���xWp�c�(��|" ����Q������w�#
o��]��c����|��t�����<���1����5��K����:a/�f%&��/�U��S(�WX����7�cU=�+�Ys}\M���pa�\�	V���c�>��#�6$�Q��VKb���wU5��WYIqK|m����	���ZP_����9_@T��D���X�<�������hs�d�u�$����L'-����{D���Y�^��L����7\�t��Cc[�D ���(A8R�Aj�BB
�����������8��O&�J�X��O�����nZ���w\ ��M?��z��&���(�����?�Ho4�������=#o0c>E�Q�����#B��+������&H�\�2]��*m9�	z��[������hYC���T�M`�x>����S�t�f�����S����R��!%Mn;E���D�|��2l�:(L�����~.WD��u�DR���\��Y�1:DhS=��d����f�:�(ho�u��]^����[�i��Z�P`�+�7Wh��F����������L=��*�2����1"�X,YV���
��`�i��x�7�(S]�&����-���]���;*S8R/�����w��Y����^m){��I�XK��	-l��|�&Z�����.'��&�0�M4���p@�C�����5..B}xi��4�������M�MgB����W��J��@7T���
���6�@|f�����k�e��b���*��fDo{Y^��� z3-�Hm%�;L'p�s#�N��tm��A���K%�?��>��.����2���|����go>���)2�-��p8zqR���?�_'���E��?�"��'�W�����'�bXF����������(������lz5��:���#���_e?V	q������!� � 0�6,M����U<��e�I�^^�}�������6�����EXR��sV�/�����yv����e����w��
�t^U������/��C���\��w���I%x��o��Uo��l�D�J�T[-��O�,�vI���r�f�;3B����v��sV��U�	�����.&�y{S�������/��yN,Z7�%�������T����������2
�O
����H:�������- ��u�������m���	���+�^�}a�������/05�x�O'#9b���X��sV�H\���$����X9��hb'xA�D��/�_q����F+��?u}�����Z�����?�m�3L���$���I�t�'����?8�����Dx�������"\���]Rj*���+�
�L�p�{�����3.�}h����_�FH~���VR5�����o���w��b���=�v_"����.�?^fo�>�����i�(J~t)���BQ��^7uq?���h|���:���3�.	^���J����O%[�E�&��k�3Z�?1V�����q[������m�b�#}�����r�!eT��j��F' �
��&��p�t�+�H�0��._�n���g1������������w�QM9�X��R� ��+�\3#}��My�+O��d.|���I� zvM��+'a�$g{��T��T
�Eys�1�Y���XS���9����?���q�5��p2��]�i��qt���C�)Gs��w\�����t?�|���R0k�Is�Is��
�Z�_Kw�p�����O>�������!�o�P�B���?�_�5�}o�����?��6������RB���YP����_I&���?eq]�����R����k�d@E|���]^�����7'N����������HFB�y������Mn���$3�M��F2Z����f`a	��H�~������z5+g���6�x�sR���E�����:"�q�{p�c��Z��g%<�|3G1���PI�X��Z�&e�1��c��Z2��d�p4\�{4m�G�kf�h�u�h��#IMyn��a���l��v4���,����k��#|�
=�2"��K��`�����K�K�3����K���d#���c��e[w��1��!��Fy����=�0E�3@���%1HI{-�F-	C z�E��i� ��Hw��,dd�>�_PyH, ���D|jr�?��$K.�m,
���I��X�Q������#�e$���A�7��4��;�����H����f&@�C3�$��F���s�Z7�m��k3�j�k��%5aCm�Q�Ljv#B:6VS���[��-j%��f��P'��>#���}�D'���^}�K
��B�'X�op;\�~��I��k&����Y�0iP���[��V_~�	��v���u�$���>��ABx��z�a8��D"B�#\��H�%��/���KA�)�wF��z��2Lx�F�!=���J���.qe^�m�t��?9%�X�wP}/o����8��A�r��	�T�C�������(��,@�/��t�{4�VH��">�{`]�C�^��P��ZG���f�hh�
n�hknnkp��v�W�#�I�����I��4YS3�V\r
� &�2����$���57�5�BM������h��h��hH+@LRpLR�0Ib�Z ����1�����W�i7yU��O��=��U��`���?�Nb�Z����v��}3�=Z����7��>���]>��������|;�/&N�(�f��
�.�O���*��e�s"�dU��}����V]$�����s�*���L���~�r� )r���5�A�����|�����P�nt���U�k�EN�S
����+���^�����D�Z��0D�l��������0\��f��S@
Xh�b����v���b�p����vx�i���k&�b��%�Z"�L$�ehN ��yR�s���6�C�\�nN��+�=|��Y���~��gu>tl��LA�SM���������G�j�}��oU��I��^�wj120�1|
�a�C�H[r�=������B9�t#�O��	��d�'��"��'B4� t�0
�!j�-���C�nn��f@�"=	A�E�c@����N�!;D
����z���-���h�	A���^������x��,XL�#��Dx��P0�/>����72d�}������\�#���T�h�1�l3$��h��iLe�vc.��Cr��46SY����K~$X~�qTs5V [��"d��X�X.�i2l*h���O�M�1M�5�=�������K����x��~�!������f���HC!�@�>*0��U���|
|������[7��
n�y�s��I��Qg�eZN� ���Lo���M��}mh�G
j�N`���430i����Ew�D�q���->+_��8xb����.{�������	o��*���w�@X�t���b������eY>����5M��Y��	�4�F]�t
�5!�+����y���n������^o�W����
�~�L�na��j�L��n�������� �U�c��i���	M���6�3�#5e�6����v	
�K�{���%+���sS�-EoK�����J
�;���+����EM�t=���\#LhZStT����Y*��������l����
V�B$���4�A����d��>���U����E� ���V�I�3�������{	���C�<v�Z��qyX���+��N`�$��%��!�vu�� H�@�V<���,����Fu��fS���R�T����r��a���z��������B#P%��?�x��Z�n������^�>��&���P�F��x&pf�b�Kp�t�X=o�����q�W�Q�8���$��l
�A��,�x����@SNh���h����c�����#�������_�w@H�����	�j���!l��QGkP�����*�U��QP.\��? �i�������'p��`�5R�:�c�[��&*�2�&��&��&����+K�%m���rt������"[�����7���E{�������wGO������3�j�}�>�,*�����('�
���(S�x������Nh� [�ie�|�^?�jr[q�&>��
}4ug�����N������}���<�����J��}3Bp�$9���L i��>���"Ip�������%��C�X�����u���tH�>��������4�J%���g��"�n�����6�t,b�� "S=��c����z+�K��z o7��N2�����1���x��N<�h��P�]�A�B��?��q��
$d��'I�P����h%�y�0����8�s����>I�P�z����O�n���9������
�[J�z��;���(S��YZ�'I��1��T���4�^���-�_L�V��<���������z
h��m�{
�/e�4��������_,�`��_�&���z6�
��X1
h�R6IK�����rXxX,����D"��LN2����
S`��@�{1����������N�,���cd����mB�Gs0@���Z{�9A70�k�B"(V@�
���8�	�$8���%�(b$�G����TZ[j�>`�7L��������$x$�<��L7Ms ��.��g��``��2����Dr��8��Bvy��w$"��Dg	��M�b5�1�~��f�'���/g�o/�V\1�����{�*��lz�������<$e4/~��U�����_���t��:(�g�P4�cI9�$�T���z�XV+��zA�z�����e��}���C��y������U�|���hf=�+��rY�&�r�.W�r�;�g�W��Y~�����������3��U7j���g�����w��]~���
�����lUN��z�\|��oW��RcQl�������<�(on)�L���*���M��^X�}.����D�^��n�<�?�M+F��-�3Z��h����%�-�3��,����P-�-�0Z�a��-70iL���U
,��';�@`��x�����IL��D���D
k��ai��4+��
qH~����:�	���-��2��~��S[Z5e5b_�){�8$�\���q&��H��1��1'�)�t�5v[y�4v~��l����<���P�|��^��A(`E��0�W�$� W0/4h*�	��O/��?��>F[�R�6�
�#�85��%�/�]2��g�y��s�L���~�v�G���|	������`�[�����b��l�tlY���1C�>|q�3	�B(�t
]]��DCG$��%�O@���Y�	Q!H�Sa�t�#����7n�V7��4��7��"������	H�������)6��/�{3���Q�n��;G�������k�������52~-����5�p�����s�W�<}�w�H�G0#6�!�D��1g��+R/P����>5��*����iux+���qdJ���.���O�G��$?�a��V�Q�>�'t�3�htD-���R���U�{Nw[�u4���$I�����u4d�]W���+�iF�����h��<|t�H��M��(����m^���(�lQ��E����S���wQ��[W�Bh�$I��
���
�_����>�h�(pmQ���NIV��7R���_%4 	�e�<�]lo���8��60OP������������X�X�W'�L���D�`(�������:��Q���9��s�|Kc}��=������'���)a=�}����B��|�� >�;�>R���	\���H�JA����0 ��v��.		������}T7�j0��f�g�C
����#���0�������H���P?��1�g�'��#�������r1	�����#��
���C9`�+����S��'�6��4���W4�A	
Mhh�h�t��LnBC�������*���,��	
�������&44Kh�#ACc�������q�&_<�.�����"b7u����
�i���5u��A;�I3d�g�=��o�s���Hp�c�3%����y�h��30^���X:�K�$�Z�����YE�[�(� _����`l}�-���"�SP�������h�Y��#�\ ���������a�������Dx�}�@_����������:��e�!�i������a�Z�G���|@�t�p�%�$i�h�

6c�����6������:���C	$�S@tw�!�=���|<�� ���sMS(M��B�B�����C�-	S}��*��.\L����	
�N������jI���]`�{�&�0�H���<
A���]+Q8����Tw�!����hwW��A��&LUnO$LUv*��]Z����(����`�;�mOIZ��+L� \J�
S�+�u/�=������r�P)G^ir��&G^=r��#{^r��T�S_T5��4�~u��������@��&�1�������a�{y��n�{9o��s��Z��

�R��n(���
��S7��%��^�3�.a���kf��������
���|�hp�U��JX����`����T�I����o���C*G)�ej��O��R���s��������gjN]�]�r�����E��E���B���3�H�nj����*b !$	}F�q��8�f�5�9�r_�jEI��N~\��f���������,�[������u��B���i��A������Bf����&��LlQ�B�8�/S��ytpS��\��	!I^�����)�T����pzHR2h(���h�A�L����_�
�U�*X���Na!+���i�n��A�����BV�]� �e[�2�=Y�>h!����&�� p!�&�$y!+:[�B�XS�������C������{Q�C%T�Hk�_}������#<���N8���/�q>�a�W��
Wy#R�H-�7x{�r+��k	�rk���"�V�mZF�2��C�����g����\g��7��������6&�E��_�V�z=�]���2[o������
���������e����6��u�}��6�*#��Q�lS^���>��-g����U�%��3T6�zTz��������$���;�Z�2�#g�k�e���]�V����c����f%��N�E3����iCo��(f"�\>G&I���r���oB>�ydT3���t&�o1�x5F�o1u���:VF���\�������-��a���M%���NGaL�����]X�B8�s�Y� �2f�����l=�
�W@Cu���r����`�Zwf����2��Xwi����1����Z�rR�������Mg$RW�����t�{5x��^�
�3S^�����
�K�]�0��.pT�u�=T�Pae��`�����@�C���'�>�`]���;�'�A^e�5C0V�r�%��k$��b��Q��|��g�g��e�f��oX*����R��Rn�6h���-�Y���������o�	*��95�����Mb�Q�a�[��)LEnSc�
0��/���*{w{{��K/�Y��z�����0��e-���	")�7��L]�t��=��;�-�+(�	8���V�{�
cI�Ih���	P'I��;F2dpQK���%�T`�U�a�d��U�$M�?P�6-�uH�	}�,�I.�Zh�4P���q�]T�N�a�I��7I\u�$0�0�$i"tL��:�IR���Ib
!�I�����:������N����=D�i��+�cZ����$���>e���$�	F�$�����~J��mLp�}�4g���#�)U�Q��R�=CK�l�����g��;�[5��0.�����H�A���M5��Z�N���7�DR��d���<
�/�n��Ra�[�����^vDj���T�m��z�L	�������$l�/L�T*/��c�)���S��63������3������n=�~�V���h�$	qu�<^��c��(/)E/�3f����l��l;c*
H9��1���gJbU�����N��/esfLek�����^>��=��z�,�T+rvS�>��b��%�����v�m����l��*��k���Y�D�N�����^�.?d������U����?��8?�<�������T�O�$_z�cC������|���	��)������|9�/7�+�|�!`��i^���z��S�������r�[��V!�u����_��8H_+��'����Hr��\)$W
�Uh�)5����t��&������+�� �<�/����]\yB*��G���)���3������$�^Ow_If�BFn��ka���.���v���Ry�bZ=���Tbhl���0I�C!1��8���[7\�I���Z_�&0d�x�����/�����t����W��f}�t�������\��-\���2����*�^����nj���8C���O�$�kB)���)�1����z~o@/���xy�G���H�#�w�{�m��X�k �eC�=Bs�P��JnY��m����=�MY�r�P�tT�=��/���=��-I��P
����
h���^I�����6;����t3g�H��X��[uH�t3U�C�G�C�"������y����*��������$�����*�|&�Z����u@����k�U�Z������s��;��tf�uAN��\k������O���r����(n��fhi����)�.7]�MM�����D���;�s���$�}?���;r�%��e,���[�gxc�>����DD����M�����Dh���K��/YfS��4d��x@;��}u��\" >�r���%B��f�0�Ub�������O���.������fhi�/����g<%.y�Ru�^����`���*y�&�Z���(C1Z��b�m�]���\T�Z�$��k-NrK�wm�F�w����1=���5�o���?�c1��������#���Y]>�x�v����w-�Z����VGf�e��6y�f|8l?pN(��n����������d$4�w����5�V��8��z�u�]��$�w���YnY��M����@�G���n�����#:����dl"4�w�>A�{��n@~�7 Gu�m&P�om�Bi
=�)T@Z�_�m�����()�l��Y2�I$�p4{�F1�x�y�B����vSc��3:��
.��MI�D.�[�)^�Q}]���5&�H%@j�����D��{�)1/�y)��@�{"hI3.��\�(x���aZr���*x������/q*<aQ�Y�������
�&�+���!����/!B�������W�uo���w�������,r:����#o�C$o:���b(<��s�1��=;}���	o���r��2�����������3�qT^�s�{w"|;��d[3na>�
8��	�K{],��������g�����vv���f��|�p�8G0�����~6wh;n��Y:�{H�����)���A:�k��B� .�������e_�8������&�Z��?���F�������M_�L�����^:��/����I���=�@^���Bjf+��VN)��Z#�V�z�^X|�)��v�$�JS(M�S���l1�6��"[����\��5���n�����O��*����5W���f��;�<�8{w���o���Ig��	��KQ}Y���O�>��G?�����W�2����l���5�z�~^�$��q����V��/��jp����I���/>���Y������^�*�0��:q(���[��=7a��c��n��nk����k�+7�=m�V^s�7cU�F�>��K:]mj�7d<j�o��	��s\�X��F���y��[�����8t������#pA�4�tu�����4`��E}��d^� ���8
������=R�F�KE�x@��l��
Q���/�Ok��<���/�o�Y������:aP*�l�
���2^�0�MiDq0hM6�����
L�5t��4�)�.��$���W�BSQ
zB�M�_�.'����0�i�9�s��5�i�x���4^yI�
�x��%�w4^�J�=i����x5^oq&I�I�`o�it9.�����'��������e��-*%L~N�}�F����w��f���.+�m�5��.6��E���5������W������O���7���M��[�1�~��f�'���/g�o/�V�������?���f�C������H$k6�~��������_���t��:���+M,�U��G|b,1��X��K������Xj+�.��}{AJ}�����e��}�}�.U�>�os;F}�������,~BW�O��\M6h$W�r��b��Uy4����������3J�Uk���gf����>�1��<_g�@Y����� %��g���lS16��l�����g���/e�iV����dU����R�niOTk�*��������a�7��p���zC7����p>��|�-��������0o����������s
������IcZ_�9:Sck�������K0��'>����GC���~�9��^�R�����nm��aE�Ew*�F�<!J*������=�#��Oc\4wI
�!B^�G����d�K��-����%@��F�,X�|gA�3Yn��iqs �v�8��M\��~�(qq���
��U]�V�G���p$��hi�M��d��4���p���6	�2*��wDI<�����er�R��/��r�e��e�jE���j�������pp�jc��H\&v�
R��J������ I�2y-�/���C>2��[<�k�E��?2z�}�����t$(4P}��p=�#��<��T�/y�P��wF���
<��l��m�{��ZJO���!!_8`����?�{G^}7����pCzL���� ��T>�Pw��[���t*Q-��k����z�8��"�Kr��y���o��1�:>	�W��R*Fj�19���{O9B��+�b��+9d+kO�>WS����~n��c�1��$*��A�;57�`<97��+���y�u���ss�ss�ss#������9�e�������3���+��%�sa�C4�SsC,�-��*x�
�b]#-���F�-�-4�[r������0�:b6yr�<��p��G��A��wjn��S/����������=nUm4��#�)n����i�a[U&���H{�z������iV���q
�3(�Ik�����A�.������|�<>�-u�\�����Q��o��C2��{���7SM���,�Q��yq�lU�g����C�98yb(W6����F�3���L��uC�/h9���%�w�D1���&�? [�wGU��U���^R���5u�jK'M�� o�s�����<��xN[���:�����sy��=9�R��
��
����s������MY�_���]U���nX�%�A����WIO��h|�pi�%��$�"Yj�I}��e�]*�e�-�U�V}���[�Us\*4+g�Y;��[=�>����1���?�u�5����#�?�������9����)��f��������O�M�g�)�vF\NfX%3��}F�����e?\|��/T+�eS����������z ����C���Q�5��:3�?0�����U��bS�O�������I�au��H����V���o�9�����`��U�U��X-���\�:ka<6�M
�u��fa<�
i��V���Trh��5B��F��$a|r<>�Z��?�m@�i�q�|/s75����jk���v'��oAa��W";�nX�����`����v+�[�����B���]!"�A��V��c��������w[�n�h��

���3
kJ�������?�����l
i-U|!m��`
i�Ub��kH2H��Q� 5t����A�]��5�MG74���@�>��F�Q��(��M?��z�<�	�qII~��p����|��DJ�r��l�R�\�8c�S}!�#�����A���F[�@O z<�6�Tv�|����tg�����@e#�>�����>�6�ltql�'=����X���@e#[����8{a,�E���FW&z����>m>P��^������O^�~�|�����C=����+c���T�+d4��=�����T���8C��l=�#=���r0h>P�}p�vk��^�����M�{Z�����].;-�j�iyW�^����x�����t��t�-���7$��yo8v'��-w��������f�������n;�����)�����[o?a�pVj)��0�0/��D��$qt��'7:��e���YAv,o���)����� ��}W��6.i]��h���A���|m��5���h��z\1I��1��c���di+�U�[�v����%�M}Q,�i"	�i�����%�(,~2��!�(�nE�[�(�@���wE�th�+Jh�$�+
�E���M`���[��l�j����{���~�&��a�uV���,�n���6|ol�^�8�1���H(
j���<��f���_d%#`P��SQg��!��%��d����Guf������uf����O�%���1���fj��uf���8L����:3��xXK)�� [�\#�R3�#h�K����:����m�&\�p�l�
RL%16����	)NHqHq������R��b�ZR�H1Z���	)NH1��	)~�H1F���	)NH1��	)~LHq����W��n�����u Uu�� U]��j�����B��T�o�4��M�`&��!�����,EW�.I��wB?A�	&��#����D0x��^Y:�K�*�Z�����\v�\�U�^��B� �p�*4�M'am"���T,`r��A���� ��0b��.�\7 ���
������G�#��eM��D���+K�8�����_g:��(i?j�/F��`D���i��4m	 I^iqr�{"	QVO�4$m���	��Q���A|��f�@�{���J�����rw�tY�a4��4�����7������'��	��HPqO��]��{J��<����M0������qh�$i��l0 ���8���ph�)������\c��\�!��V����q{H2X
����N?���C����C����OE5'��l���-���88����@��rP�,Ft�q�*':}X����b^����e����9Gf�sd7Gfpsd14GfC�g�`1G���h	Un�D�)G��)�<�)�	�S�:�����&�h�~���wd����{h6r����H7���%����4.E�<mQ���������f�Qpo���.a���������?1^c]���oo���X	+�Z�8���\�J7I����bQm}H�(��LMt�I���J�)�Ju���z����T�^�kq����4
��v(�Ns����k/��Fjg�[d�=o�����&�!�[�
$�$��H1�>��[�L��:����Z�C��m�������y3��f�s~	��V�|���_])����k�@(��j	�����sK`.-�9����;�+�K����%P���-����d�[.����$/�ygK`��l�s�:��pzHR�@�.�E3
�f��p�X��*�VC���rK`!-��f	���Z!}���XHK`�-�E�%b���7`	t��e���@��oj�T�@oBH�����%0dr6�9l0K`8=$�K��o�Ek(8!O��3'�����,�~:��>����HC���wj�7��
N�|�������[p�^+��[r�������l��#L����i������Y%�����
�6P[/���
^����7����.�:#���6][L�l�5h��c�����U|����G*j�b�A���My��������=���	���l�����U��+�	") �4,,;VHv�
6`�����A� ��:��t����Cx��O���en��Cx��@�IIA1�a�<�����<�Y�X�h�gC(Gcq4�0����^o�5��o���+Z��:z�(��������d�������G�+5��G"_GZ�q���|=�����j�3�\ko�J�y��\7��5A7{*i`��U�Y��UkS�NU�L��TkF�~T�?�Tk9�zS�2�FSk,��R�%�BRk�R���atMT}�0m�q����~��U���j���z��8�%�YI�����\sg�P]k0~�m��W��5@7���b|vqw�$	����'��Kg��C:5N�7���*n�f�f�[U���p�J�0��Q���U�<���N��A�~�pu?[�t3��*�oj�j%I����T�N���lRY:�-�Fr�����D�~�,[��&��/UW��l�.��O����~�\��je����6��lU.�7n�����nu�K�������VG���]an���� {���K.)�z��bV�p����~X��.�7e� ^O����"��E��!yg��I�t�����]�N+�IE����NCTbo���t�@�s�%s��������}{\3
�Q���z�{PRk�KR��k%���<�)��G���� �A�Y9����q�
@H�Qc�y\�i�h�6���Wp�*c_�X�U��PN����YL�F>��U�qhK��v,�����_4�\��]�g�gzw�g�x��&�|��W���e�LV����q�������o��^M����?��f���� nl��������_���t��.:O��Qi?���+m����Z".��8:!%�}�����R����M��������5�W����?R�p���e��l���\��u���
_���up^|����_��Q�<�:B�,��1��=�=[��wo?^��=��{.��#H�WME����{��~�jG$�����-kD?s���~��utc�����p�������k�+7
W�z�
�p����������4,/kV0�]@�.��uC]�C]h��`C-?+4�0?Y&'�~���[�N�zV��KRG[[*�5�������+�)K�a�E�0�Cy(���j�r������n�����������O�$���&�����n1/��9����j��T�����u��d���$
���0�	��qU���&ZjB��<Z?A�Zi���L�l��g���F�������M�
2�);.{�^�#��E�EP�v�)�I��.]������T�/o���[rX%�U����v�����\r�����u{��"����tl��r�G�p��j�-������������B�����O��*��)-����7�����T�CM��|�nUA�W`��MH���o>g"��Z�O�����j�F�����b~�5��]��X�	1���8����N���d��C�Z��w��	�e@b�R�k�TMW��Ma4SF�aZ��J�	�j@?���lu�jY����Tvo04��)�n7�n��������Cd�$��'��������eI��5�n�8�`�,+�w����r`_g>IqL<���8�����fP�A�d�	3k1_o�����-ba���tQ��E�FX�6�}!.�mf�����q�#�/
Lo�����*d��V^ h"\I���#h�2��<��D�#(l������%�A�H���+� F'[��I�K����i0�
Nb.,{�aFcG�9�:��l���E�K���z)����7���KBk��J�{I,���|�5���t���P��OCkfr�@U�E�;�0�'&�R��zC����C[�j��v�`��v��\��K
KE#�8����8�`Qs�`�B�/��b��yH�X���=��P���|����.?��D\@^��U9A����h���A����Z��g���/��F����l�E��4j[��g3�}w��8��xh�/�/�����mI��Gn\<�V�Gn\<��B�\�xh���xh���x������-%:CW"R���G�~�(4>W����J]<r�����\�x���#���#7,���b���oe���i����XXO�`������"��-��9,���[���(�$Y*��$8�MP���h��.�����z�*�4kE6�mwEm�)dWQA+��U�[d
@�U4��!���
�j�30�~���m0|�Gc�R���@��"d�IuJ� �s��S\C,�67��\B��J�X�� b}#}b�?4�����|G��!��%�7n�6��Qb[8^}�Q����~���t�;��W_m���>�����B�c���e�q��{Wcv�i��f���U[����A-�����%��IR$����k��,@����7����w���(��$J�0���4����Q�
;@ �{)��8���Rfd.`�}�\ �x�A(�R�`\��t�^���N\��4M�
���������C�"-5��H�J�J�R�^�n��q�[�qv(���W�|�W�<�(c
�P�LC����a�Y�"qd�����n�e84���D|jTOhT�\�O{�b�fU���*��4��P,����x_#1?
�5�6�qh��(o�0�0#��]���R�,����k�g��p��J_1V����^�q�%rI���
��@Q�j��$��*t7�J�u�6��	��]pA�zI�Y/�
�^����D�C��GPQ
��JznW���\��d@p'@c@�CKH�"M�� N��4L��r�A�:#+��]�oZ�s��������a-	 I�
��8
�S�d*2Y��������N���n�)��M�HMK��$L��>Q@�
�!B�
��FE ������FN���u��#�zAp�!�B�.�w�%�#�N�e(P���7R��'`x*��{���kb�VCH�|_O��~���{M|Kp@H�H����
�I(�z/��N�����0���6��wj��l��X��%u|�3���+�2������k\��f=R�C�4���Z�%��H�n2`]hwnkw�j5}�^w�l�Kc�K��K��,��\�k17�b��s8G�;��yl�v<�'rGOD�_�+\!1�5��(��5�hv�<�e���k�571k�KD]4�Y5����[���
�x@O����P��W�B�1k!-(����+<V�B^Y��Z�V������/<K�����H�����^<�qf��)�-��?�|R'sa�y[����I-�����$P���p �jrn���@��>��X�Xq+�����7n;&v��o�T��d��hov�{wvy~q����7�|�5�x����c�9������@��)Q�lE�mQN���}�z��o��s�/_S�n����x�,p��zMIk`���yE?���=����'�=����6��~�J�-��w�g/{��2!?� 	gN������N���]M����=��V?'�B����[�1�����e�F{��������nc}�,������-"�u[���p�|�W��X����$�����q���s��9yQ��	���<�
����\�	��~\�j���~<�w��;��w>�������}E��`�X��]��oWpMd������(����a$�HR�Um_'�+E��7e�|�n�9/2���<oV�&���|?.�5����{��v����V�������4�WY��U\YGt�Rh�=�Jv&�DI
�*����b�h�/���:N8'U� X�E���6a?��i�T/x�QN���f��)�
�^#�lc��5�iS���"��ilt_�"���z��d����E�?mA���\W8�.l�������k^�,2>���Z�'e�Q�>��?�C<�����F����C�B��NOCI��9ly"�Y~C��g���-�#�$��3�h~oL4CM����>k�����}�j����c{G����(�IHc��C,E�������RPxC���d����4�(�;�uJ�eH	��W#��b��������	��8`'3+M�m��i�[�Is��������?4c����~��n�K����n�Gg����jd���>}o�h�s���`3��n�mo�
�Y�X��C�A`�L���|���[����Y,w��.�:�<=b�8�����&A�Xc�J����a>��
���|]L��M�����-�b%�4R��w�	�m�km��5�6�����uk�Z����P/kL<0�5�k���P	p�DIja��w���A��8����~������M?�����lU�-��	F�/�?^�9�x���-o�~B7c^e���U��|u��ZAJ�J\,���{�������7�j{r��X�����J*��@E��|�iUR�Q��!��n��e�7���2G�3����e����~A�:��/)��'�U���*�����~knt���$�RW\��@�g�u��Yq��>Q/^e��9��E�+3�8�S��!���=A������eZ3\`�����oY�j��i��|-�T,5�'Ht��N�������I�����8�����g����X;���f�fE���p�Z�PY�QA���gv����'r[����"��kV,n;7�%7A��$Y���
����MR����m� �l�:�-\���J�pM%�����q2D��a�@v�C�Z�r0jm���+��e�1z0W{�VBZG���O���X�z������I�������bm���Q5IQ����Z5=�1��]$���A�;��F	�Y�K>Vk���+QZ{\�_�Z������=���v2�|1�����/�|��5�M����/~�
�����yUn�+���B�����E�t�=Z��H�%���/�����%��

����UT��~F�K�<bhlN�Ij�����X�����!CF2��T?�E���6r*QF5�g��$ru$\�H�^�;S6����j�B~>�K.��u,�~!�*6����6 w-V�����]�Q��F�l�k{��M|�#I!a<�tve�R��"�S��AEC�����e������}����0�@�p��P��x�M�tt������A�{x�]���)���u��,����P�P��r�
��������K���"x�4�4��</����� ��-k�8��g��)ON�#���*���2�Sr�S�uC�r�)q����������Bl�'�����0�x��TL���~���������b����JV��^+[�T~{x ����nd��Y�Z�0��=��N}t
'�:v&��v%F���m�
<��K���1������.W�n�������/�fwr�����/�vr�����/p ���c��X�7/���lD:��������8��A�r�V_o��8��<G���`��w��F��������w���]AfD��"d3�4d��; �����F��q2��''����(9�?��''#
��g
��T�r��������P����8��������!>�!�&
��Au��\�1@����S.>�`�@�}������%�����-t������&6�=�
G0�<7H�
���u[�h�-�Ixr�>����Pr|�3>i�z���[��h�'-E��I�;��k�}���8�����)�R�wN�_�Y�J����^XOCsh��X����?�'5����;�ea���O�{��u*_�#���3b>b��?����y�p~�I�S�r�q���;k]0��7b6�l����
w��&<�ycH�����1�+��;�y���M��
�
iN��.�f�\b��c��u���i��"�R�t��98n�BM��HA���t{��P����P�S�I!�P��P�B�@��Y(R�H����C�+��;��`�$�#����?�Gw�t�T�	�_ ��$6Ll�\��f�.k����g�jg����/w�r���.��|�����o��WW��+���\V?����u�}��6�*#��Q�lS^5�����1��_>aT�X7-�>���K?x��:Kz�R|�*����?��N����6���Y��c���p�h�au��;��iU~�lJAE�W�>�����Q�d��)*�39+��K�����4�+�-I��TI������t�o�C������I���:�����z�
eV��^/�����;1t�H��oj
��#BX�����A8�����������cc{�V�'O�@�>��=���)[�HI`�AnC0a|��"� �"f���}����+���]����G�75��	!��{U7��"gW���S���r�g�X5����<?�$�^�IzU����~�uv
	��T��#�2
j3���P�V�6��(������/�3T�U�M�*Z�)#�G�q:2T�����
�u>C
�!���gH��g6aO���-�l��-�l�R 3YZ
�R��R����l>VV���U\BrE��$TI$!v�����z�yl��O��	U-���=����/L�ELS���{�R-y|:h����a�m���>�@4�u�����b)w�O��
�
���G�1x<���&���d��c��<R������G'�����B�M���+�/��A���w����fP�����bR�6	�-�k�&�-(���`	-���&)-(��6	@-���&�4-(A�)YP�y��
��@~Y "��,��� �g�@�5��� �i�0L��y�J�G�<���������C��j�,����j3�`O����M��*�|�)[��	~����s���p��Z���{�����~�����c[|��f(���|N�tb��6��;S�4�Z{�H�C��������0 5_4y��>l����5�n��%4���ht����0�4�'D�����\�oLd� B�������zO�/��}4���y��O���ME��'�������@ex�C^���].��	������$�]/?	�'9����������v���wdP��
G����}b0z�����N��^���*{�����y��<�SC��@�6�>q������p!1���^X*xr���Qw(�?	f����$5YRC�������� 1���� I����Z4k#��+������k�l*��&�X�R&s�$4Q{82��W��
L����|v9�)����,�el��C�H���r�_`@xHo�8N'�w�1���n���E��YB����<�N�7��7����Zs��r��]#YF��h�,Y}�K�U�z���%���K^�Tc�q�#+��3//��^����E��������F�k&L-��<�UN&�h�i��It\���AcO9��>�RN���	�(���,�8��)0:{�)9P9�@_
RN�!�E�6J@v�R��I����*���7�:(����o:H����������}k��k�!�1����0M�������k������'��;H�/*���YgI���:�=�iRa�R��8�M�V����+���&�,����b�B/�]������pG9~4��G�$��{>:q2�^�L/M��c�j����{Q�=��'fN��f�g����f��oX����f^�9�����������?�o����e��f�S�k]j�iT�������\o�������fYj��I>�$�d�"1�e0����$�>>qd.(��V+N�!O��%u�+S���N�'�:c\D���*�|�����'pU������U�U�P�~�����P`�!���}��a�������7�����[Ow�p�;|���QN5��4g����k�u<�q�#v�Ys�Ns2��
�\���5�k����M���W��;������.�ZlVs,�W�<9{�f}����O������]B�V�EE
B��T(��x�9�o�K4X�4 >�Fj �pC���oe�
��o����Z�d��X�<w�snD�f�v�����X��.}B)�J�2�`��G�Rj�HJ���RY*�U������r��S�HP�]�{S��g5*Q�w����������������u����������W[n���u�h<��s�o���g�}��n��BS�B�W�����c�~�����3�E��LCs�R�>f�v��[���{�D��Gy	���x`�� �%W�� ������\�7��$�<��T���5A�)�'�ibO����<q�j�'^��v��������l�Xs
�^$��W47a������\Q���
��D{�V������F�3�Jl�M&os�M,�PY"�������>�l�!k�4d��j���=(�~`���A�����E:cQ������Qu�M+v%����<�O[�T-�%��H�}�p�|� <K\3f|���8���(JN���x��'��f��v�xn;�G
�'NA��o��r�O�Jd���?��,|���c�[���Z�N��O���+����-�
*���y���u3�[�-��oC���~�T�m��3{1@�n$�):i(#
ek�D�e_
psk��t_~�y���V_�^���q4���6:M�Vu��S|jY�����o�,�����R�NS����b>���:>�����-��A�*��\umXR�C�2���5���y�(�w7���������{����Y1�"�Y1��t�\y+���P��[H@V�{���[?=Y����v��v��V���h�X�Q��X�D�����X�XX�p�b�M������~NOVtv7���������{]��N�����I�[��G�>;�p4w�y�/��jk�#���������}�����{�0M����l��V��b>���e���zX����%[M���Q�7��z��F��dQ�u]�����W��f��m�s��9�����������|�M��A���c����~)�/�r�����g�_C��78d���$V
�z�������z����n���~��G��`D(�#��jW�r�0�~�{w~���'�@EF����bj�
}$�q�Z�0
���]qs��P��Ut)�s�$��������C�X���1�� �f�
���
v}D��3��<�n��s����D����� ��t��<�B-����R�y*��^b?Y��c��
,EN�~F�d�	�c���
�'I�e�SO�s6(��,��9x��:��6H�g�� ��=��~�v�����p���
\����xI�O�8���"�~���'�Eo�:(?`Jr�"��
}���D�~��I%A�Q#(,��I��8RC,h?�����o�Yy#����A�q�Fr�~�������~�a �Gjl�G���hy���C	du�����l�y����-W_�eY�S��lU���RV�|F�t�����|n�W�N'�V�Y�4�������5�7!h���4e�g��B���x�?-������&0�KF����������k#��ojn�d� �C�,C���;��n����M�_���S@�tN�HL"#u�K8�T*��}��|� Ie"�".�4QQ��9I��~�����4lDV��P=:���|��<�k� %Y���:��0�UdE_�k����W�mY�+��^=Vte�v���z�X������(�UcEO�k����)��fR�I�w&eo�Y��|�9[�.�-���d�(�����\����u6_f���L���~�_���\V?�����\���[4Q���*��m�+�Csw��<N�7����0���'`�uFeS�j-���������z:���9�j�Wj��kJ$��o~�6��w��/5�X
7���Z2/g���C~S<c�^;������������Sc���1='@R�NG��a7��y�%�+�/�^f�������%�$^�`�4�i��$�0	=K��D��Y��=�H%O]'Ez}�A�i��n��������L���0�eN��X�0:�o���]-�d��1EB��}���B�������A���b�xr]sn�;7I��p]�q�
p�.\'7����|<��9�H�j�u>���� ���9����� wD������:l�Di<�Nn�'�9N:�}��u��CBVk��1�I�:��A�;���(o4�����3������4��P�=���{8��6������%����
�9��W��^��*��o�"��9�7
R�;��L�������1����&W��R�����4F��-�x� ��r����{[������r9zS�>��)��t�����v���W��l]��GO�\��]�go�w�o��*k�^�����^�/����|��0���zss�9_on�7���������}��;n|/�����{������w�w�K_+��'����Hr��\)$W
��Uh�)5�������Y�DX��s%�a�����^_�?.+^ 7���!��B9����)5��5��q�0��>r��I��8��fJ�	y+���LX�
���_�8�����N�U�j�^���c��cG+�kT�3�t?���%�����d�1����M�#�W�iR��
fKt��K����
<N4�/��I������'Mt�/�������������.�u��(�h�zr=���IA�������aK����B��}������v��{�E�g�8�������6����������E�;�� ^�i����[�/w�c���$y������W%p�����o������T�P��0f��0�bg���P���x�.I���/�k����(��|����Kq�p�����}�q+�6����(�}Gs)�=���@Z�w|n�h�����$e��\����rd�����}G�Z3aa�N.�;����7��x���m��Oi{�����$M>�`���C���w�i
���g3��^��a8;������.�7���ac����4��:�z��?�Tt��85_�+Ut.�8�����s=��U�Y�t�7��v���D�����h�Cw�:N�������|����	���<8i���3�]�8����f�mf�6M�4�����:�	���A��9d��:oo��	��S�����~����y��w��	�r���N�9+J�R���p��8���rsM�{�@��h58Z9R�/S��Q�2^�J��Y�T)�	A�U��e�%�N*%����k�:��$�H��-f�8�5t�xw,�Z��B|����FR�$�m��l$	^L�SN��F���'�=^k�kq���4%�e�m�l{!������0d[��dc���e���)��g[��m9���\�-�pr�m�	^��X`��rk�I��T�����S]�.�e���S��Q�>��Q�H
:���-|[�f6��m9���@���$���\{|�e�����^�9u[�5��W�m������m3d[��e���e���)��g[.�m����B�-+W�F��!���-�v�-���$�H�����8�5�b[��X>����#�����#�J���eof#I����qN>z�H��W���-[ge���W��B�-{��HyPX�
�����/�B'����h_l�b���n���E����Yz��,Zr���jf���e�� �%�f_!��Y4�zY����J���X��fl����	��7�s9���C���wj�7��
N�|����M@��8!k�O�����U'�:�|�W���x���1�����7��1���zV��	��B������N����t<���h��OL����E�:|,o�~��(��-���6��C����Gh��o{�/���L���Gv^�p7�F���7E@-�.J�������U!���N����O��=
���@���2�:�wBE!�KC���o�]u%�{�
[-C�l������,ha����0
������i��0�2��V���=s��M=&�G�&�@���2�:�TX�N�-L��&��v2��gcG�&�}z����i����^�Ftaz	^�F������i��0�2�IV���=s��M}).Lr�z/L^���Ve�u�Kaaz�naz)�N��mD��/&��ATa
V�]
�u�^#t�^t�^t�^t�^�s�^�s�^�s�^�s�^�s�^�c<�'���m�Hn�Pv8�u��u�w�5��lq�+��W�'���[�k��Z
d�G\K}�D�JC�sw�6K���-�Q�e���IWb5�i�5pmCBj����|���.��0�)��S$�����P~M�����Ya��[�����V=�����Z�����?]C�vH@�j�g�%0E	�%�f�	�&29��Y�,��&�5�3���'�����m�5Q�P��u����&�� B��]#n���N�w�td~X��R�4������"��Y$��"��v)�]�`�"�5)b;_������Ej�k5��.�,�]�q�w	M��R�;!��w�&��������������kn���y��q\��S��7�I)n�����k�m���Pz�B��z1���St���������)���5s��Q;�J���4�U
��V����U'���,�N�4����c7_�{���44�o��?,_���b������uE���Y�P���g��"��m�5�8_N�7tE�C;h�
;&�s�~�y�&(�����
$��|�A/W�-p����t��VbD+8�+��Z��@$��(E�z�C��Y��T�w��V�u�*��r�M�W��d5�)����U�o��k��y5_o���&�f�����y�wg��g���~��7Y�&h�:�X}�(:���jPN��HC?����a������P�Z�ie���e�~^��c��}��_���Q���I��]�8���������rp%�e�;?�x���[H�����	=#�)���\o*6~w{{g9����f�i�&C�P9��w�bG���M?�����N�"B����.U�������25��Z�����y��H(�����;m�����.}��99���B��o���rh8&�������!<�7�f^�`!����sY�Vu�g@�Z�9�a��r]����m�o~��V3��Nn3�-��/bS�m���Jp@ f�R�f��"���!b�G�?0�Z�X������Y�����5[������\������]�����%_�����8@��`����=�6����RW_�eYiS��lU���RVStVf���O����|n�W�N'�V�Y�4�������5�6!h���4e�g����u��y`h�n04`�$904��"!i^u65;��}��F�>bd�s	T�X����u��w[����{����^;�:��X��{��:F�&�EA���9��)x���3x�1n4a-
���k@[`G�tT�,8���	\�i��
�W:
T�s*
�^�T�O�����z�S�>����|��ME��k����W6�S�k*��^�T���q��R(��&S�L;�L��M�'���D����e5���,?�7�F������f��{���\�?�r������������-�	�����~������������|m�W�$�t���$D���\�t��
����"��/�#Y�����MY+������M{ajg�G�����?qK��+zN�Dh��F����0i��R�'FPW9����H�����
d�����u�GA�87:���<��_����r���E�����������W�DZ�#	��4H�����%���fm&�������+R�u�O�����q���u��@�����j�*�V(���o�,�e-t��E]�(;���E7��������������<{����
�����D5�P��g�3�(VL,�c���A3�u�F��6�&8F�g��_�g�>�������_��f�m�,���A�� \*�u���-o������YY��nU���Zf�����
O�5F�����e���D����>���Fed�5*��m�+�n�{q��v�}��_���f0^��x��;zV�AR9_F��G�����;��E��$�����x������*2�o.PV� �T���t<���
�v~�UI'd�F'��(�����������NG$s�g�����,�uZM-��b+���$�Rd��	�On�f������Y�l.����y@�������r~4o�����&��������P�5��������^Rf�8+���N�p�Y��	\i�Kd�s��\�=�'?�����g�Z�K-��[���2
����cPD�>�4�4���1p�J�E�����	��e�[�O"O�1o��cE�rR��8o���KM�����
��8��&��-����\jS�&��)�6Y}�
i�
i���q*���e�1�)���Km
�B��P�E�~���P�����'�}�{
5 �|���?''�3(��`�:������S��#z��]�<��b�[�-��/v�X�#���j����V�9�/�^��`/$������`�6�g��
x��M=�R������k�p��9��/�=�'�����0��R�d��a���	z�{�92���$!h&<��7�f�M�`�RN���z?�O�4K��D>��	�f���1��9�hSj<��I�N�Y�d:y��S��5�k��$xP������t��D��,0
�$�?��\�����>���1���qgg���g��������rOO�C�~w`X.�8��s<�dl�a��ck�����#���*��]���U}���*�1w����h�����JM� ��"��B��b�)���.��3O�r�_�|�j����m��*9lI~$4��&����L0�x�t��Xo��7��3��d&����#��5�����@&��f.���H���Re
Kq���k�u2W�h�(?h��-��p��3)��,����`�\iw������Wp]�B����#u�l��Ni���]y�3��9sY%��,��Y��g�<�r�<�M�,���\�g�:�ry��-���i5���{�3��?�L\�uY�y����G�0�o�g����V>�@SR���	B�"�G�E�1/u��y�1�\�KOE�b^�j��=b~z�y(��{u/~��|�M�����3�;�t��J����}����!H��-4<��;�����FW7~�.�\����Y�6�����\A�^a�T@�r�s�5��V5W)@�.[o��c]O��Y�b�b����.���uFIFw>��i! <SM��E�����<Ss�L����kg��L�Z��-����Q��������b�������m$�TM3UQ�
]m1;��d�H��a�b[�ls~�M�/{_�S�h<����BSM��u�����u��� �),k�r&��w�K��|�o��=���	�h��B���u�iB
-��a:>>l~�!��q]$pw�~���rQ�����s����=�\}���+��<���������1���)�P}�����!Z��� l��)�J� )��V��8���{ `O�c?�
�x����}��t{�ZE�M�hZ���P5��9Xh��p ������tn
Ta�8���8�46��n;
=@7���������M|*����u������@�A?Zkc�.�h�f{1�V�N��8�Z�o�����(\�{�<�f��g~y�s�l�8s�T��zv4Y��������H/T���5�JF�p�m���R�B4��x2��H�
��X;�EN�P���l��C��2������<#�3��0�x�i��I���r�8�������;;I������X���k�Wyj�	K���� ��9[4#&��~�i��OB���'� ������d�FZ�
�~e?�U���T�%�e�%	R7I��/Ie?�[T}�'a\��KRA�q���!m��6c���a�%�%lI2g��W����L3e|:�e�%���+?�_���J�6z�CO�CM��%�]AX��B8q�.���4j�p������/G�p���t�>^���X'�R�{����{N�������I�@&f���3ioc���%������j��f�tA����x���F	��uQ�}QB?S?W�>�?s������OYC�E��J"|o�rD��t#�:�^KE?A$������I��H,(�%���$:�f���ue��!%qHI�H4���K�tn�^�|����W}�v�����)j�.�������<�sJ.���S�P�d�.�"6���-�c��i�;����
~�>��)��8%�2��I�Lr������=p�b��J��4�]�lUN������&�I�.7���*������E�K�������*���7�g��������������M��rS�$�c��2Y�>OV��_�..�^������wo?^V��km!h+hr��������_���t�����~���gA?��sp�DSaf��P)zl��Vb�Q"G��Dmq��������+�^��������h�m���h���e�������^YlV����������rcvl%:H["������?�_\V?^~`A�y��������atR�����������\��������\���W��hQ�We��?{��g�*���=3>�5����%������������Y�6�?f�h������U���n���x-d�1L����W�U�1����dE6����4�[:,g3Ib�*sEcU���j����W�<pU�����D����WUC��XUs������qU5�h_Us��j����jn\U��U57�����jeU-�U5�W�\����U5����fU���j�_Us����W��~W�����nm�0�h<^�<^����iP�D���	���X��P��B��Z\�G}t������r>�K|�jM:{��0f���3D�k���=b��z��'���A���TP�yDt�]!�c#a��
q�G�rt��O>g�s"��*��.�F�X<����0��C��:Rw��������hv�x(6kF?=�v7<��=m?��=�2c�Q�D�����fg%�O�GNf��	�h�a�qO�o�N��G�h�iR
�#�������z?%�g���H�o`���Q}�������r�� E����/��m��Ff���h�)]��w=&!�(������w�L�Z)j�����u��mN}��=���v�L��{��~�$	��~���1]�p!5g5Q&�$��9l�g!��YB�,���6E�`�(O��"�pr[��9�}le�����@�>�
g��}@90��k��{r���
����
�}�}u8�B��B����}����YB�/6��>8�p0��'��8�c:�o��o�s��q�P����}C3�
�����}��0��cDPyr�8	����F��F<����o
��d���o��>}�P�������>8���H��a���pJdX�E=dj$���\xj��r����Y����,�)���������A��@�:�Y|;w�v�c�m����S�]����pN��<k�gM��V�<k�g�������&�G��@��D��G���D�g���������Q&��h��6��|20������������<W���E����
TM��TI��"v�t�'/:����Y_8U�/*��+����q/�����Hw�(I���x�Pv5�����|������T ���T�H�Dk����������` ������N�{G,�-���w�Mn�$�E�G�k\�������1�%s�
N��lq�������+�N��4�L^4�w��+�WM]1|r���&���E�7�4� }o5���i��G�k\��&W}�1���������\��s��g���r.er	We��Kw��4��,��e�4� 7O����i���M.=#H��'����hz�+4����1�%s�
��+�w�\�s����kw���s	?'t�2f
�\���2\7�L��;�����&�����M�q�F�\��`�PHD��s��U�v�B��
��U�v���s����5hrY.J��������s�A�>��W���G�k\�1'W.�i.��Km�\��s����w�B�s��K��8��;�1k����\�����gr����� }o5��v�V�hz�+T�o����4d�<��y�����d �B#�V��)�;��/4�lE�|q/�k#���g�{�\��>�};���j��,*6�.W%�c����������������o��&k�	������	���9C���S�lE�mQN����7�@�Xq.�JdP�����TY�9��9P�s W�@���q�^�5v���y�~��Of��}��,@{�6��Y���Wg����)�;L8?W���������e��4�?���������X�2���Ir���/�w��w�;^���`� MJ'�5���s!��c�$CY����Z\`��E-P����9e�y�e�V)w����d)u(q�,hW����$j���*���b$bH�[����9��;�8�xx�a���<���s�<�"��y�*q?���-��P���z�w��]���g�;�5�E���3�0�f
9�FF^��fLAhb�a9��U�vx�utk�nXc�h�c<E�2dn�xL0��&S0�0nj��5'!�����>��(;y��t��3�s����9��z�J>����Y;��w^+]�����X4�\�fT�����E�J�<+h
x�z�#P88@�p���GXWv��i�5�J�:?�Q$)�A������`V.n2�
��+]s|j�����\��T�EM��yFn��������t��_�+����}���1M���U��Y�9��#g���������#g��	9�[88@�p���GXWv��i�5�Jg=��Q$)�A�������i.n2Y��+]s�r�o�^��y.E+��f�k~�����t-N����W:��Q����y���Z�����J7d;V�����(m|^#o�����		P88@�p���GXWv��i�5�J����Q$)�A������ }.n2���+]sP{�ol��y.E�+��f�k~���[�t���k��+�|��H��Jg�<k�y�t#��#~�1�	+�H�McD@>��7����D-) u8�X�#�+;X�4�k���j��(��� s�@�cb@H7��G��������%��]b"9(��z���/S��@�p�L:?&���o������qz��>q�7��'���kp���h��4^T��c���J��8Y���/W,�C�������2d��\��/�.�o�����q��f�
S�9�����V�mZv������_v�(`iQ�����hrM�YrM����P�O�(�-9�&������rU�)9�&PY�����^;����@�����jJ�t�@UnJ���BVr5�����*7%��j!+9��Rr�kP���hr�H�X��!xb%����e'-;;_v�7�u��~�o>g�����v��LeV~�[����v����l���Ui�������7�����U�0}.�e��-X��l�FE��J��s4eKT������[E�>�e�F��,�2���}�#������--�:R���;��uA��]X��_���Q�E����^�����"��((�|J{��c�>��|��z�_�tt,�j�y���+�^�pO�A�#�����Qo�|�����Tm�|	��s��05L0:1zf��N���-\{��]nS��_`��u{a�T�����c��V�����������e*}�n�q�����X��A�m�A@���@��������Mw��m��5�&�-m~m
���k���G�6��u�nS�Mzb��7���)�8�]`���t��w�l��y�)��(�M�{�B���5<5�G���S�Q��������<���0C�q�6d4���������g�?���^l��[�9�����c��c���5|7�����5�&��|�q����M��*��U����@��v;Wma]�\��-��$�G����(v����@h���u���;B�C��"�_��k�o
�4�D�j���Q�`Mr�7/r����P��;u�R��n�M�Q��D3K<5N���@�p|l*��O�S�:"��JG> ���;�����=���!�m�s9[
��S��B��%uV)�j*�CA���s����;���`�G��^��rG�yd�^^��'�cM;��Z��`�����$��$�����S3v�
��B��"RKZ�����5-�kI����9@�[2��d$��7T/��t�������X�� o��V�1~|=�Zq�]�^����v�:�#(�@��r��>�>�ZP�w����S���������0��Z�W�]����d�	�aFW��f��{�� 3�5��
DB�."�H�iV�8���Iv\�2�]�?}�����2sj�be'+�6��TY���iX�5����X���������F�WKk��.;��B�������"$�r4��G�k��C�����:L��2-f���u�MG�p5sy8'�y��T�������k�����?��!�}�1f
�r�z�����-'�l9�G\�ii�7�t��0��H]*��D���N�"}_�V��[]���ry���[]n����[������mu0oM�S��M��.���Yx��V���V����d�V�2��<f����l�7l��.�U&����9�Zy���n����uh�SKk=�`�V�7�����0,�"�g��G�k��{��V�H&�ckg���a�����`k���.�g����^Wh�����	?gt�3f
�u�����)v������Yx��V�&�YN��a���S�����^&����S�s���P�D<z�;�~�T<x^?�4?b�����6���t��S�!p&�?��l��g�h�hg���?�y���|_-��oC|�VD����h��)1$9t]��B���d��P�#; /�@����a�VL�x�_5eZ��7�.	�B���c�H2��{�bg���A��d��X�<�m.��\7osy���y�(��A�8o���<��+�����`t2o����.t�;��$�ao��uy/��[�i�bG����l��@��V��&\�����?o#�H#n;��$c@������[���b^����<o���7s&N���0o]�-��k��<�qo�Mi0:���k@�,��F�1h�i�v��Zl����-�����������Vg:V���y3���-�7��mT��������.|���dL���VS���A��d����<�m.�[u��M���Uk4�[�~3>���-O���uU�N�m����.+~���d�d�����4��X}�V������q
�T�E�-�2��;�t�u�l����c���]�zq�:CfY�,.@������{���i��;����1^9I}���~P���N+�fW����Y�S�����������vjA�x S����y�������������2�qMg\�WW����q�e�`�-��+K����g:ENjoWw������
4��qZvtJ�Q�:���j��x�S���fs�I���7��8��q�e�h�%���������B5M������.�ig��]Ne��Il����E��I����(�b��VW/���`g>\��������ayj=���9�=�8����������#���f������l��L���*Y�ka�Qd����Dmu|���nD����p�3��jSc�$]�������a��s�<v���qRe�O��*�GK���������dR�'�s^@n5� ������d�������Sv��{z�����/����8��s��=�������A���8�E�������F����1S�'�s^���l\_������WQ�zsg������Cy�N����8�.s��]<�3��q��3Sv���Q���������u`���6>�b�I����v�s���\�j��b���C3���bYn��Zn4�Z�m�Z�m��Z�����V��Y�E�JG'�f��F}'��|�����i������4��!=���u�
���������j��������U5R�����|=�{w����\!��^����>n���,�]�o�zE��,����+��2]7N*��e��i������j���N40�0qR�����;5�������F��X��c�}���f=�hT�]���q���Y\����p��5 l���ZI�k��;�M��~�W��6���'��$��7]]Nn	oOb�7����[�X���.������'��Ht^N�*�}�J�S�h.�����@6h&��(�_�A���Z({�,�������9�'����	g�P�V�����<l�1A�P�	��o���h�w.�7�1�3�R����$��	LBx��p0�I���������HoF?����o�5!�&WW�+����2�V��z�����_d��E���V�s(�}5�T9�U?F�{s�m&�E�O��Y�� �/+��K�>����j3G��l�5[���*2,��8?�<'\����9)�_9��|{��b���b��p�������?b�����k)9��a���/�e%�V���|�L��S�j{�����E�],�y�����cT����<�}�����_-�k;>���\��J)�������|.����s�����g������_���|�N�W���
���&������k��|1�|�f��u%�T��I���Xs�8����E9�T+�����Ru�P/����$V����mV
���� ���������K��=��V�0�_��[g��M���H2���Ju�j�*��U�>��`��'�2�|�5�-���C��j�O�)������J5����&��`>)��������0�J]#�
�E[��5����
�� �wN-�#n�1{��2{_	;������sZ��i��8-g�V)��y-�5����k�QA$����[����gy����}A����Ub�N�V��.���@����9�G�.�� ������Kz��7��j����,�'���!>8�~"�d>�K��wY	��p�al���A��6�����+)���mR<���5�Qd��g����e�V)$�F�
�E��m���}O�l���;�V�������������@���'�����|��}�Feo����lv����l������������h����9
��<���1��L�O�0�$�~��9�O�\�$����t�Nc�
��$���0��O�s�����6�J��w����r��hkeF��+2��b�P��&�C�3ZP]� g���
m�O]�V��')�^�!�\TZ��2����<���|/���f�W��=���������_*����J�����7�H[�os�y����w4�Lxs��X���T�v0�/�	m��a9b#�����H<�_>
���#6�B��UmG��r��q;��4B4����Xhk��mb���D���$��:�^�����lW���"�"�SI�=���<�p�Q�H�O���r�y��ZEY��V�u��Bh��~�UW1�Q�^h�,���$���:����Q�l�j-��b���d����/\uY�h���r��j#?�:�')|�}kU��{xFon�Kz��<��/`�c�6�����G�x������p	�Gu^!h�����[�:A��O�M\����'x��w��^���r��r�cy�Buc`t��F(@I*������������ ����j���$��0���Z�G�{��?#��9�T�{������7��?jiv�G=1;>�@$�[A!�Ff�/JB��K�(���E�B�B�zJEz��� �*������6���D�J�"�P���c��D���J�8�V�U!�j�{�IsqgSh��inD6�8>)��kda�@Z�t����<@�8M��J�SULU�m�y�Y����q+�`��L,��U�u���EA�����
�BX�
�Z^�1/��0{P�.K���o/�;���^�������W	�����jCO%$���Pi<LF���Fi��c����v5��v�\o�x���4v��Z1&�d���e3��������Og���fT�5M�[�x��r�����I]h$�
����*���R�Bk���(7#��O���Jf�������7p�1��'s�Bx(\nz����c�4����b�^~����_vS���� � /zH5����I��E��b�u��BF�������~2�mZ2emb
 �DY�P�#P�l������q&A��o���0�.�����7,rof�6�P��
T��������q
������h��Q�X��|��h��5,MuV��m���h�$��������g�%Snc�Li��B�mr�L�r���\�)Z Tn�]/����)j�Cf�G�)�[^G3E�83��vjZ���<S��<�m�)E�~k�����)w�L)��R��)EC��LQ�2S
�L)v3S�!�3S��%��i1S
�L)�+^G�	�KJ?@�'���n�@'m0
"�����L��mR��!���u�ea��6�/��>'>E�(�AK�q)|�8vR�N�%g����HSQ��[�b�����z�O��Yga��;>��
yl��`���2����+�Z4���CT#]�F��w�:'�����j�m�Ks��96�\�~����'���	n�O��\�pH�jm�PX�f����evU��We��l��<v�����n�������P�Q��>�Y	=_���|��h~�5����A�9�5��o���UQ3��j��%�H��P�����o7�&�C�Q:��i�.L*��u�d���+a�
��h�b�>��8l?���PI�_��g����5vZ�":�[ZtIyX���.��/����+i=v������I��J
 ;����h�L�J��	\O=}��M�9J�M�?h��a����Q����q�B��mu��p��Dv�o����g��v���\��?vM����Q[��������cVt�oK8�����eL��1x~�����������P����V�����K85`���fjCU�+:�c�=`w�V��<��������X����r#��
����_v��!���'��qJ"pl1����SH��y�I4:�[�J]����x������������C.R�����7.�l�1�*�KJ:��E4�����W� 4iLd����@���Ck�O�
[�~`��������������|�j\���^g���7�u����������!"l���3��z1";�Gh-����u���'�s�^*p�/"L#��lDq�lE
��F��!)jT�;�"���	�Q������T4�el��fY<?C�hpv��%r�4�58�xv��y� �y����i�yiH�B���+�0I�D�L�v����_wE����n�	����P�a3�i��v
���-tt0J@�=c��<�o7V6V1����������B���~�"���@�)E�a�H(�����P����:=�/�����fK\����_��8������_�������w@����/Vb���X�������{14'>@U���jp.8��	v, ���	q��D:�c��
>�	�@ ����������'��B�PwB�&���{B�$4�R"8
������Ve�m0GlU��&����6���dV���
BU*V�WZ�};Y��*���d5�)���_��0���CU���1`���A�"S�"�c���8��e�|���:���;�x��t��r��}=��E�G(���� ��n@������4�$MXE5�bC1$���D�)$I�f��Nf�[�glj�+IM���@$9�	�)�����@h�3 <89��K���L���������'�hnju4ht%�����C�L�B��K8�keE{����#�� �r���jF�.�3�.�tB�
�+��nj�'@(u*�
1��b�}�&j��	.��#f��(&������^{�.Ut7�# ����N����)j�gPNIG����_��F������y���@����%@�Q�#�F�M��`/���ji�EPm��	y�C����[a�n�N��O���4����
T�����X��w�w��:��2�a��I������-
s�������4������1�1A��&r(������4���J��)g^���S�� 9�L�*��-L�T8�XJ@� ��2&H��c���,��0�r�/R�AI���w������T{������b���J�QH!	0����� !��3��{��{�������(L��8%���}�G�����N0�Z_
I�.%s�v�j��[���cUb[����a������:�C"�����i��xC�/>?��g5�_���i����df���s:o<�b�lNl��Z�f�'�$I3���nw��X;\89�����nz+��.Py�3;� ������)��4���.{��f�����F��)v�����<���EXh��4�"�;	6��z	��}BT"z��l�61�^2w����ud;�RM���:g��&��Ct��r-��TvF�fK4���i�����1��g4�^;J=�EE��X;��fqF�����b�9P��	�O��=�;`�_r���n�O�c~���8n[��m�#�}�C�-���dAH������?��B!1����ee�~�,��i��(�Y��q3���/�y��9��A��=�.��2y����f\��R���
��;mp,}
=F�~<���2k����s��N��S->�7����b�� ���������Zz�*-D^�tG��Kv���+������&�yM��P�!��l�.�8J�����v�������������l&����L���S����f~[�%���~���bn��h����k%wn�������y��z���b
�F����/h��S�#~M�z�^�|�=\r"k���5�Q�WT��Q���]��Z;���1WJ?��Fk��O��x\I��@��-9� ���u�;lb
qk��JP�IJ�S�p���?c{�z���N���*_��+�����7���z%��&~?,
�pV��w��!r�H~���x��7�?��r����{�/b���A�0h��q��C#n���������d)s�N)�1/������xee��3�@�-�p%��?�V�x<���9��$�����]���o{��'��z�4���2gAOo��"��4d�k�#��6C�9$��A*��R��T���Y$���E�zY�����T�����i�=���&6���r��EC�����[�������o���a[|��=pC�~h��;�mu��#���z(��m�����6�j���2E���X@v���Ik�}[T.-�l��8���2�����2e����l���%k)�J���=G�o�]W�>�Re���Z����C)�	�/j,������O��{�K����G��?�G����I����dt"K�<�D����F���#KC�.���D
�D�n:�I565�EdR�<��LbiK�Xj!6��I,URK�D%�����KP�H�i-�^o��K%B�"g��E���~�n	G���a����t��N��FvWk'���EM��(��+�Y�s��{wl��$U�M����`�U	�o�U��0>oe�hRx`�$�K�-x*�*�Xp�D�CR��G�� `Y=�~�Q2a����mB%]1���v'<Q,���'W�Q����g���X����g$�#�`[N6[tJ��9�$���fD5,~��:9�@%U�������D5�O�#�fR4���1�p�r��=��z���c��V;�/���z1��"L�I�S��r�

��R�"a�l�o�W�z��	<g�����hGO�^g]8�m�������~_�
t@N�M������d�:���.{�g���c@n��yQ����y��j��ew3�l��ej���u���n!o�F>��e��� |��3���c��������:*v)L���'!`��������&��_x��UQ��,�X��L��2w��&gh�h�K_t�>I4��h�16I4�������rw�d�����\�rrU.su?�nwQ��*������u{�"(����������u���b7USpJ+�]��}������{����d�y�B�g�DkE|�����-xp�*���#	����^�-C�#��Y$��h]�;:xybD���=���B[�����2xGu�T���k��?j����q��F.��N�Kom��2�<��_���6��<w	/8��w�KzM��~�x��<f���>}�*�tf������g�N�)���W�����{�/����
;���������V��D�������H.�n������	,�1pb4-��,���hp.1�#r���1.�(>�r�W�29M��w�z����X-u���z��/�e��0���0�I���U�a�����;�ww�Sm�C0j:�����E{���F�����4�T��t
��q$�g>z������@��b���N�	5G�����v�>��}af�@��^��vy��$%��N�7�tK�E�G4���FP"�F�����	��hI�X��F��X�Ii�#����,%m���p�����"�4��[�-�2��|���i]����<_�-��P�/D<��j�������*/��1�3%�z}j���j���S\#���x��)gt"�3R��F'R9�	�(.�����`����F��Qm#9�\7��&^����3t�KSr�N�����K��=dP���{�jS��N���]kOm�>&��D\R��A ����*lK�6'V���SI�����w^4��`�{/�}^P=���!vo�P(���(t���%Y]���du1��`��du�����8iIm��6�@�J(���,� �il�����%���4�.�&Y`��dJ��XU�d�Q[� 5�d�� -�RqU�d�e"��6�������u����>���h�1�f��Fb��
�t�$�rE$I����CPE�j�|�,�1[`����c��d~�mI���������UI��7@n4��������87Iu4�p��`n��f ��h�{);{!,r�h��9��zv=��8]8�A�6��E�!����WP*?{��$���(<o�W����r����]��uN�t<�����X��R�R����k�Q��3%������4[/��vJ�J����D����B?g�+d��a{���a�g��oo������:�D�6j\�o�
3�&K[��*)n��,U�R��.0��RenrR��t�����,U���U�	t�;����p]��4�d������T���$�U�Y�$&�)����*�`jJ6�C��R��,|��+I����fc\w�`%VR�"�GR�`%VR��T&���I}���W��Y��AV|��N��I��5���RERp������$')8q�$))8I�I
�Pp�YL|��
���M�!k�������y��=�����6��fw��f��W-�L��2[�o����K�!������9��5C��^�"�Vb�(0:����hc��+������r�5���|���:W��u9��'*��n��r�KJsC"������f����e�H�M7	���d���Q������t��5�v��*�z�$<�m���HM7	���d����]�)v�F�A��H����1��ZH�G��%�bh��v�{M�@$��l��'e�GR��9'�v'	@��v`����2��^>��3��/;�Uk���� 	��A�l�~���x���[1�V(�G=$�)��
Zo�6Xj_vw+��
W���$;����?�M]���n���%�6�������ws>������s��UO�.���}��0a��I�k��A�����SaxAO~����j���f y�QW�f��J8/��Z4-��H-�&	":����6�%I>�a3��)F�2����H1g�Ms�8�v�y���]c��5�1�c�2�y�\i%���k
O��z�0��T��|��8�m@{`"~��I���2���0GPv2�4I��3�:N8W�\�a��W �"�����?os��*���d���Q�Ht9��@W�Z�F^�8�|V �#�L��M�a+P������$��j�;�P�i6+m0�4I�
4�:�8W�B�a�2����8'?�z�'p��{E����	`S���#D��<���<��#����\�����Z�h�~No�U=�.��E���N4g����[h�6��y
\��Ai�k���T��)g�j��B��*R���W���z�t�ZH��e���3X����}$�Nd{�T���{�W-�h/��`g�����\�+x����j~�d���	�k)�b�s�������v�%"#x�i�����&q;<��������'�Yk+J��6��1���h���h��]����H�	,���y����+
�%j��Pc���f�D������T�e��?��������w���z�?�Fa�Kl%:D?/G�����v�q�G�0�Un�Ka�{���#/���D�����qX����)�QI�f�X��f��o��q��J ��@����O~���I`���g��������Ke+��w{������������O%�����7���~�
���A������w�Lk��������g�>q=`�@*_�j
��5���L6�����\oj=��5�)"�>�8(Q�S���P�EJj�l��T>L,����@���|]�����HlVS�+LCo��9P�[pX@����0�&���h�rD�WS��CCo�9P��pT\���� �&����A#	��`'���a1F'?��-I@���*'�
U\&?���L~D�5����uI��9Ee�ZM�r�����hw����>�8g�������..���Dj�Q��f'�yF��\�*.$�����-������?����b���#��a'�x\\��'P:�i��p���Xdn
�Yvm9Xz�3�h+>0����~����GN<�����!���'�T�'@�^oop'�%ni�&w%�&k���,��01m�c��Rl��h������O���dS��l�j�&������R�FB7���;�}�0w�����L��>P;�����[��>�:���������R�O�~�iX����b./���,/�"�(����3J��wG#�����IE�zS��7�rS������UZ��o���C}���7'�x��w�yY�������������t�I�8wO��������tu���C�����{��!y��
�l�Fr��f����Q+���U�����w����^�;���*[F{�l1��K�K�|�:�M8���E�����s���/(u|������~��E��2����f~�������/9{��O��S��"�J�f�&�2E����
B�3�4R��}	N@z`�:}�Us�������v����	J�8���>�MA����X���Pf��������������g�����
���X;J4��9��*������?[��jEI���������.�H]�x�a�E�a{w5������_�����@$")����@����@H-J
���I�x�
'@>F
���*��f������o��F��7��y��z�����cEVix�� �mM����?�|�����;��U��5���w����PdGO�\�#���?�s���{����z�;�.�/�x��b�k�S���w�����aU�1�rq�=��?~�=�����������/������hV'iP�k�=~��������J��������������������
�_x%�zJ[]
>e�G@l��u��������o/S_|��ftEv���UvU^���:����������S9F1�����P�T_����]��A�4�v&y������?�������������
O��f�����y�Z�t0�N����@J�Ciz��h����*��a����V������$[��,��M���)�E����iC�5�������X��Z���s�j�J��R�3:�1kj��z!�������6�v���<C+�h��\sSb]��T�r������}u�B���r�Es�U�dg��$nb�%
\���Q���P�6rr�h�dD���U�4B��!+7��������d1��R)L����4c�}���04���N��F�w�o�R����J{B��73uN�������og�4	fw�}C�����F*@_�_���rZ����/#�����	�e��#����{;z�2[�Xl�`8�n����V�^���D�W�L9�T�L�W�#�)���b�,RTmA"�I��^����IQ�HR"��c!m�>��%7[��XgSs������,�VP& �d:�A�V�P!�-j5�
�61Q��%�y�"n%8z��'A���"���f/O��#��E��t���'�mo�u�\�U`�

Q	�L��2����'��3��5������zK�`�}���g��z��n7{$5I$�E�62�$?%�INI~r����"?A�-�;�����s��D���O��_�D�Z�?}�7QJ��<6J�$L%a*	SI��	M����0���h��!��;������ISI���3GFu\� ����E�:1*IQuR����W��o*�����D���P��"D���:A*���;s��G�C}��?�9..�i	C��Ht���?P����%~�����W����Q'�G�#/��I��f��,����e��!���#pA�H���q��t�M�<�8r�3����M}�~�<��'3����wx��Z���{l�	�7��_�G�F�U�O9�����vlM�:w-�H���H����%9A�K�����N�j��vEo��F�u���K�����Rort_k�tq���\�E�����l]��C,RNE�#�S~��7��G_4���>l��9�����xFV�a�-�����A��Hr��BS-x��[b^���lO�a,S
���.��/����+!v�F����4�Y��\v�I�M��Z(�8T�n�k�9O;���������������u[�B�JZ�F��z�X��������7&�r�81�Y���/�����d,��tR���=b���G�K)L��"Y�Z�=�\$��	�=����5+�-�������*�����6�e���f�����F�y�
dy����Prf����a�	578�1�K7����3"B�1rK}�����9�(*�1�:�c��Tt���<���cn^[��>r��_,�c3M���5��o�q��J�j�D�Z��`�'I�;zA!�o����[p��������kd�C	�t(q�x[�%��=�����@W�8�H*<�����Crdt92z��E��~"GF�3$J�^��n�a5i\��k�\j����%~�|���Q�5����M��B�p����O_�* �.9�qCn5B�~mM}U�U��iy%�b�Z!��3����4�a��-��3=��Z���ZJ�=�KW�zND�8�}�B�y;�
�QI2�`
������/�Q|x\���$���i	���k7*�]�6�&��1�N���0
�gK��� ���������n�n3s@^��:{g ���Ep/��}uj��q�9R�s�m�n���$�c�G{q@��#�k�Hu��%*���p8�������h�6F�����:���`�dv{3�/'4�=������E�o~M,Zq���x�x<�su��tE^&�����L��qc�+��`�P�qN�xMAm��@=�D�l'�[[�Y�w��?�V��fI�����[���pD�����;�E�����8���\����+Y�gu`���5�s��DZr�T�|n��^kp���V� 4��F�)[�-��I�N����$jk
J����$j'Q;{|�����!��T�����D#t+�����u�������$v�Lb���$v;�Mbw���'v��|q��N������P4�����[42��I�Nb���$vk
Jb���$v'�;{|b����!��T����8F���t�b�e9�(�=���r+�P1�~CI��L���.G���.G�(��4=�vg�9���R�4�ZY����t2����/Q~_������H��p�z8R#<�����z������9�����OU�p��L�@�VR���%��MeR�
Nj���6$��F�#T�EjyH����
��;i�T���3�%�!�
YR�T&�����6�hNjCRl�<B��>���$u����~���
g���������oNI�H
�%{R  t&"�,>%")&��a�������C�C;S#t�H��e�3xB�����!%%")�I�HJDR"�a�fgJ�ne��$x����^R<G�W�:�H�4��)�Ie�dO*���2�����2$��D^R���U�O\yH�fg�7lIQ�`����o"�e^�����=y{�m~���&��]��m&�E���7w��R"P��
���9Fh�	�s���O���?��p�,�t�b�
���0H�b��6���3*��y��S�5��*$QH%IT&�"�!�s�����Q���~���"2o��"!�k�~��Dd�Y4DB���}��f�$82�L���./H��%��G���y=�����l��X�!\��ECv���
W+�]Hd����`�[�,���d`�M�E'�0�dP?V������4�E$��T��N���Z���S���E�4��Y�bV%+P�sN�j!w������X@� R�}?�����kt������V��JD6�U��N��1�3ET�d��&(�@%%��1�?:|iv���/�&��d~�a����an�p��BY���L��+��qA�,�9[���X��$�	z����!�o�b S�B��%�|CK��a��0=I��z�s�L3�h��
3��(���f|
�����m����Y���d�`�"�OaX�f����%>.�������h�������&�;6C�K����P:������h~K#(�#J�����M�R��cd���� ��=�`k�i�9���*�pT
�Q�W�sT�(U������h��[��?��VdkE7\���Z�
�mkE�Vt�u�Z���������������^�`t��K�5���k&IV�����[�U��������nc i_<�]V�����v�!4��������
�T@��%t�9]�*�k(��0���B9T~�P����K(h�x��>�}+]�Ks���:�=������RB�O-%t���RB�O.��������U�^Wr����k%N�d���	�k)�b�s�������v�u��]Xr����1#�g��''���	{���|��<��;@[��Y-�1�[#�{����J�t�#�J�{yDT������7U}�����Tx�63&���(����g�da�����H;�S�"I�w�q@<���������#��"�Xa|4�5���Lx9g��E�Z����z��7w�q�iz�8���'��&��]����w�o.E<q5z�w�}�^�z.��>��W����?�9������W �0��Cs�iz0�	}��t�0��7���������g�>�0I\�T��U���5
�M�
G?c�n������Cs�� �$��#�T�����iAr�I�1��1�W��pr�D�~��j��k�o�R��w�)ah�o�^H�A��4�����m�P�
VG!�aCVG.A`#V� J��v#�w�L��c�����j���7}dzI��������s2��fg���;��8���b@X({�����FA���p���e�R������?O���Dk�]�&I��7���?6!R:�X���f�������/B�����&��A�9k}$A�y(�@X*��Xa\�z��a�aRa	,	�2E�&�5<���B�9�%	�6�YE�ySy��*l�k�i�Ur
�p�������g�������zI�����T/�����[�kQM6Y��lF�Q�k]�������9����v{�����y�K��P=����o���I��R��������f��}A�	]�C���_������O�V����G��B�7���I�~�W;�Kal�
�~ �������S'|:��	����#�/b?�w,QU9D�
�B�Q#7Q�F�8i�&")���Fj�������N��Jh:�!�>��lD��Uy7�Z��T��M��T�����/g�d>�[o�D�a���d����K��D^���@�A��8�ay�Wb����;��%&��KF��%t���U}?��Oe�Wp2��[�I.B���F����@,�C�>��m�"��F>��zn�<�����}:�_�?��}���X�vQ��;S�Y�(:A�H�s�,(��z���m�=�@�-��
��g���z�����w�>�9�������U���v��b�]���a�t'������?W���"�ng?_���a�n!��f~����^��/9{��O��E��$./�$����7+v10�o�2��z���Ai��}��e�F�8���9���5�Z~tPn�=���f��@�q�n����(�����3�P�x���F��(�q
*��*��6�}PZ��������TQ���]M6�/�����*A5�yI�Kr^�������$�%9O�9�W&98A� z�4��o�]�Q��?�����k��y��@�����O�a������������/��9;{��ZEH@_Zz������3�3�����{-�(�r����9�P���������_�����qvq~�����_�@�������fi�r�i����������I���5���9�����cd�6f�����=�e��]V��?�t��yU������>��]�
�_x�zJ;����A�x�fy��3#�����o/����
��\V�d�����/����$���V���r�����������/��(�a�� {��d������?�ty��w��y�Z�k&���tv�Nw.7����c/�|"/�sLu�F\�u����#�Z0�b�$�:d�2~~}��e�#�z��e�'����[�,����Z�
@x3�lUN��;�L�u�����v��/�_��3�bG��)~�����*	~A��M����D�?�V��mo��7P�k����|�2s��9����� :�#�|����D����ym����@;�v���x�j��7%q���H�=����Y��#�u/���B�*�^�"���Fv�d��Y�,
�P���e�������pX��Qh��� ���4��bPUME���S�p���F�|"|EE>������0�8v�Ot/_D#
��'��Ae��,K(T>T���^4�P9��r(S�+^�u��O�G�p[�Q����������C�[�?@5�����C��h����*8O�!��J����\���������0k�������E#P���|����X���R������2V&9r�)X���r��hJe�������FO^�v�)z��tds�n��`�`�������l�h�iU�U����/�8��xA��sg����p��6��@���z"U��"��1�<����B)�?�����[:�H��T�{�{<Wz\�-����K�L�7R��HD�*��`aAhWk�BW@%���
,�����Y
��
�@Y�K1
zoY�����P9���Z�z�B{�4��4�,�hw��������$p�'�s���,�����t�0�&��\��.'�@@YRin�����s�S.��6O�>�������h�����z��,NCV�0��O�0z�u$����C����.�6�����d����)1��K�t7�rE[i��W,�:
�v��d���l?4���NSQ;�����{�$���&x3�(����%�sK-@�o�l��62��?I�t�"��UkM�����Zs'���d��d,��X�������u�_�
�����=�V����]�:�9
�8�����yVK��y�B���S�7r��C��1��hH�i�����1�
����[J�|�.S��B7���5�|�cXz��h����R�``�������:����U��_eJ�{.�R�2����s:��A��~K���7�r�".u>�Gk����	��s*V��@��,����!k�Pl�CD��U��7 ��Z��������G�S�$��|$?��X����#���D�[x�3��+��H�dx�����BB_
���V�O�w��h�J5l&��)5%�f)�7	�I�M�x��`�xH	��6+�7	�I�Moxs���#U�~S�7���bOo$R��2�"������a>B�����!���+��E�`p&��^��������Z�!f�t���j���O�@�9�"�3����mD��x�[�����{�6���H��b0�;y4�X���
����G�r��p?	��3rX`k���Io&�b���"),e _7H��$�5[�9��Z�.��f�$/?���)_���zWW��:5�bG"�������$��������Z.��������:#h�����=����g��5x�+�������&����M�����a�������W=^�$��8}�/��N�'���\��+��'�
%�P)G9��:)�H.GB���d�]MQ���e�B�hF�fyU�aPv����h�'2"D5��S8� �?!�\�Qh�	zj�EtO���8�)��u��
�3�,q�|t�^�&���v�B��z�����B]S�b�-DQ�3���B�����[����A��������tYNh�)��n�������6 a�=����2_o�Y��2�C8K���p�Q��	����~�6-�)��V�'�^����������Z@5�CFQ"V��9�|��``FI��Q�����S0��y�S3W���[��/=x��8a]�C����L�3&��6�������P����:#g�j����.�0a�q��B��Sv�L�q����{7���S��b����O�������0����)�_������[Xw���G��u�����v��l�"�p�0�s���ng���-�Z�|���px��f���h���� ��GWi/���>bi{Q4H����nKd���
��Xb��A�u,A=H8�%�;�p���0%�`H����#�HU�=bnS��#�6�s���T���9I���K��;I\����o8w���;�~���L�R�%��0M�}��������rO��_0z�)�D������_��K�y�
�o^���Cc[����o��/X�DT�D� _�6�?j�����c�����N�zKo��N�?�'����7�uy���q���P���n�=�h�;�g���:��i�md0�p\���x���:a��Bk`�WF��*���4����@�W��{���L-�%��L`�m,1A�aB��"��Xa�	)��C��
&dL,�F�A��0�

N������v8JD��� 	Z+��OP7��)-	yp	����
P�A���$��"��L-ejjs�(Z�Ye%���]��FJF?H22'x<AB�r"��N��;T��Qt)�4<��������v�Fg�����\D�zHJ�8v�w����(3��n���<>
�����+��y�$��|�^���r
��+���u��#���5q�[!,�%p~�R]1@yJ{����)@�y
v�����/
��-��[���~G
*$TH.!]%zP7I���^!��.	�/�jB�1x���)�b
�����)��@L�w ���f�uc����������*�U8��%J�%a��W�������`t*]���Cvm=�����d�k"�B�����&������<
��Vp_�7����������W����"O��,�`SN�F,����H�M9�	_�H9�E�� ��0J�-�Z��jcGQ)gqF���5w��j���$T5rj��r?w��:C�'�5!��j��B��!�1� ��AG?R�y���
JeF]��B=�EO*X�����I.}'��AEAR�-���	4N����s��{�n������%�8��V*ll��%l|:��A� ��A
<����A|��<��Aw����{�o��?�P��R�k��CRuw���)%$5��&�������Z�6��&��H��)��f�?��-A�
		jt������4��L@�����O=|$�"����G�U�g�h V����V��=��#���&_���>�5���mN���W5��&_���*�9�	@�D;���j���l�2A�V*��W���r��9�+�P�Uu
���������'����y
Gey<p'�+��;����CL�p��Ans���&�8���(q ��&�8����q����%9a���������=�@:�=�N��=��v��i��F'�i���@�Z-o�����B[#���K��+��A�m�w�aS���6��z���f�8v�_1�7O��v�e�O��Aj+j�Pu�::�}����BS1@� 8!��	!8�O{A�f��~��-�����I���6���B���RkP	'Y,��0�'�~��>Ap����	w|`���N�����o��;d3�!d��z_�l��mmt�M'��w1w���89-v�a���(529-&������A��o�&9-�2����b��wrZ��������������D'����xP(Z�c
m�����:��\�8?V3VxJ���N����{Z'������"����hI�}�0�R�f$�Ex)�/:IK���}�Jb{��������!��;������qgSS��s8RJ����193&g������3crfL���rZBRrfL����193���=�1t@rfL����3c�md�-o�'�<��)%�;��	�N�wB������~'��RNkBHJ�wB����oW���I�����o���~������1M��nG�i�u����2/]�g��n�������d��o�U?m&�E���7w�����������s�_�3�n��C�5��Z����%�]�^h��B��T��M��@qx�$	
)��b4�(�^r�	��(z���T���������X|oJ~�������C�7u!�E/��A:`��T{.����u��*��K4�X�F
�����������������N�O����6k����0�����\Y��D�6�R3r�%��doL�=0��AjO.��U����.��SH�SH�c��������L��K������O��O�u��a'L� ��u
�?T*��? mH���+Z��8�R'����f���d�p�i'�oL���)�k�f�~�M���L��2}��$kN��@��Ok�����b���:�}v���/�Pw���1P2�����S5
?��Oe#���xD}<FT&�k���i�7V�6��������\�2In��lD��5���O)�u	�q��R����K�&"��Z�H�

I&y��6L���K�$$��
G�
�(V��H`|ax�x*��R�R���G�ZP��C���UT��N�~������h�/h�/h�/ZG�O�����Oi�����/���������l]x0~������U���0�"�k�S�	"�O�*�r�"$������)E��FJ3|$M3@j�0eIi����iHq�h@}R���Di�!�R�]�P���t��9��H����I>z�B�B)��e/2��S�x*�W�n�P�7"���b����
���p�1�1`�uN%��J,/� ��X�������
t3�C��fm����L(�����B�J�J�Y�� 3�]�V9���RB��4�6
�2�B�KJ�KJ�Y�� ��]�VyE	yE	�Rv��+[�a _�B-���Vx�����K�t�X�l�oPA������D�>���(l2�#�cRHX/�@a������f�,2�=�����h�^(���`��\n�g�b%��vh%$Ez�D����X��C���
!p���������a�8<�A$V�@�
	���>�l�C�>r��$��R�)�m}�h�7�Me�c�����������2\.48���>����0[���5�\t�`0��48��4x�<<�������xz=�|���r�*��J{_c�~�u�)o���r�T���}1���n���ds��R���r��m<��k�����<a�:Q�#�������y�K��7�
�������%�.n�0����u]�����!�4�`r�^�����0���WN�DPv7
&L���<��M���{�PO�_w�T��v�2
g�Br�Gbwo���M��JV���.������7��(�G�����KY���w����J:����o���/>�`��(�;��/6E�"��z���3]�������W���}��M��T�$�I>yRat�(���a�9��!�U!I��e�m��g��A5H�����M���p[����6(lm2�hS��?�-�c����=1�Dn���U�7lO�'����I#\?@YY�S~�R�N]yZ������fCUy!?�C~�UU��0��B�+���FK;Z��9Y�&_9���.����0y�����������]\���?��; �Te>���d���k�XN�dVbg�O��3���
-��<lh0�h��cPmg�,_PMf�lZP-e��U:
�x��@��t`*�8�Zux8#�i�V��;D"�������nr����M��z0m��\�\iO��OY.�D�bi�V� ���'���w����F����U�i�)u�~�~���_m��~��qah-]W$�V2lY{(��aKOH2l%���Y�����d��5�C�����C����R��^K��^3A�v4�EMw�������w�*�&u�M%����MU�_.��rV������K$9~���.�^������7��'�-���l�iv�o�o~�����_2�
B�����'Yo���Za��$�M��X�~ ���F��[�v���!�����{������(t]����g$v������,w���9*���E������8����d^��������f�L')��z3����!o���=����f��!�T�42I^w'��O����Y���ihdF�X���\�%��}�S��*D��9�rM�P$+Ff&p�f����B4�����0q�>�[!��b���=�/�n`!6���$Q7p(60����X���.4�a�"<��5����d��T����]��&����%����t8���t8K���t8[�wh�sR���I+��f��E~���o��F&�7������,{������U���������*��?�|�����;2���u����^���j=}sq������������/���M�8�8�������1�I_#W������f��r�i����i�����I���5���9����C��&]����_���������������nq�i�����Ci��F�/��\=��V
�
"C�]h��YW<����?��D���_4�!2�v]
�Uy=_V��]f/���&�O��K���>vRd�V}a\F�v�����	��N��������O��_��K�`"�\3��D����(@)VI��S�M"� E,���r��I�Nd�	�[��yg�*'d��l&���`\�f����/���b��4w�=�/��6�j��(��=Y3��S���Zy#�������������x��ze&Z�a��f5��:�?9{��+�$�O�i_����>��f�
����L�
�(�b�^����������!��#[4�Y
�1
FSE�������V-|�no�����e��|]){�:�J�!��O��
����D�ge���1��D+������6�U��Bh+Z���j�k��iBh��
���f<�mx �FF�U?V��j>Y���T
S��8����AP�5;��zQ� m��CQp0���(�2�>�S�o�����-"Q��(N�0��o�7nr�6�{P;���
�'�Tb#�,���/��a��xrr�(!���Kq:"_������2��~���_J=�/�;��/VP�c��m��4jS���9r�#X�)�U�1\JAA*���UFm��,'�be�RYQW� \*��PGL��g�N���� �Y�Q��S9�e�$����p���Ul��0v���3(�@�/�`[�l��S�W
�#z��tf�.�	]������]������^@i���S���*�/�����;s+��k�(� ���$��s�iILQ�t����������!�%!0B!��$�N{B��B�$�O��og` q��B�;G���90�\_����A���	w2�8\�s��~�H)��L
}RB�J�D�v��1T��,�T��>$�h��?$�h����!H�G����eQ^o���f��]����M�p������,����m)��,e��g��]�?}��#"Y5hY���FU����AtGxx��m��@v0���)�
>up&&,���{���1Z��aj��
(*1$��c
�eI3�$��
\s��n�X�j���������c=�<n���w�xs��fn4H���(���zr��t�#\2��U}����V�q.p������6����`X����
����
�m: ���q��/�9�����������q!p!sa�~��:0d����������
�; ���H���/B��X���C��C�������e�����~}����C��?���^p�AhsD���H7�	�~�v��$s��q���������#��Gf���_�%��7��	��~��h�!��H�;N����@���������;��:W88~��]�\|{(���������B|�Tx{(��R��a����������H�w �Vo����^����#S���Rj��	1%�������s��a��197�����s�������m$�tnL
�Cwt��.{�KDu�ur�{�t����Jq�������crt�����crt��������crtL����19:&GGGJ�������������c��l���"r�y�TS4��V
oEw���/2����1sDz�(���8}������5O@�=���a�W!=���R����^P��Y���t�b�8�`�?�A�c��-��006��O��b@"-`�0�#�&�B{
�Xc( -����IH<��2T�`���|��`*�6_j1�J�����*�#�=i���0�2[�7�ak�S�I�_���:��W��[R?��zq\}�=k����������L9�E��4:�E��.66g��@
k>���j5���5�}�8����k��z�iz�H����Y����E������E&��������]r������r,(���f�����>0���[��>�L� �g�v.AfBBX����,�3�WH�����URvT���*&/�E���-4�)�,;�(}X���&rj����o�LH�6�0�E��3�7B�gOy����$&W�E=�I:��G�?�a���0��=��s�1��
�e�!c�!��C��C�E�2�]>L��z�|t�Mj�Z���
�=a�v{�v0fBBX����,:b,:�Yt�Xt$��HfQ�_���B`QO��N��)���A#�>,�@&���E���p��^�0����:�^S��C
��Dw����EP`x�7�Q��h)��[X��q�G��}�����v����MJ�-�7�z���#X���<�������z>���S�	��	-����nA�1`�Q&��/����*kV��Yz4|:���iF'�V�qq��i[��A�S����<��EYd|N/���H�G�<�>����4��'��������i� 
�[)D����iz��j=G!GI��#v���{�N��������;
S�=b�_��G�LK���d��V��r�H�y
��Q�����G��!�l�F�29�����7���w?�4����������
�����(�)�EU�.���I��cY)vo���b�ZH<���q�CnQb��4�b�I�@�;��x������t#
�uZ������0��E�#��"�0��Et��a4��o��"�#��������uN�BMAy�a	r!�"��ycKCC�������5�G2r�����M�M\��tb���;4.�f���5��/���Zs�Q���Y�R�J�c*�"7�q�N:�3�w�so�=�vQ^	
ry������%�~�������N�U��{J�*TK��F�-���-�8{�B;�����N���1\5��v���a������[[��gp�������b�V��(&���	�~W�0a	Jfm�����
'NMk@/�%`V���A�G� �Ad�.�J����^P��e�V����*����Z��DA�bf�Z�jG�h(�N�DX�P�Wp7)=T�L��7����%�&�����H
jq�F����c������5#$��uv{3�/'���r��V��5&��F��&�&)X�#���aP����J#�)�L;�����M������b�����,�W�7Y�q�r�$!��=X�H��
)$�Yr�5��w�U�����^�k��a����BA�7U�8�^�O��*ax���k��8�j���x�C3��.�wcx��b�"�_��~A��pZ�L�-�]��>SX�e)c>���__�S>���8E��q
C�����p���&����"58���/r�*8f��RGx�q����O���w���t�Y�)�WP�)�Fec�\����7��!u��Eo�/zC��
�Eo(/zmn�)Y�_�:��&��G3�V�k�l�76 `����0�N��.zG�i���4;X�
���g((�_P�)������-s��k���U����d[���itJ�P*Cb����	��;�S	����x���t�N��^�^m�a���GSM�p%Zx�aD��p����A�������v,�����eT0�dQ^o�������5��t��1���1$C���!���!�v%cH2��mF2�$c�G��1$C�1$C�1��dq�`hs2�X�L��d�P��1�����h�G$�E�f������E��kX:J�0�H�0�#��L��dI�w��a$F�6#F�a��e�0�#�0�#�0jY2��i0�9F�T&�H2�X���0r�x�C��(C�L#f���v���eD������#��#�0b��dI��dq�+F�a�m3�a$F<Z�#�0�#�0�#��%���C��a�Je2�$�����#�
�>D��<"�,"	�2�Y �^�s�� C�[KG���0�T=)+�VS����0�qec�P.D4��"ZMXr-W�J��G�A�m�&)�|����G�������:�0�Z��f�����.����rP"��\o*�.��Uu�U��{��{�@�E(<T�|js���l�8j3`:����6�����y�f��mwW�o3����iEJ3�������Fn@mR�1����3�m�����.�Z}��a[��+KY����Wu�p{U���p=
��H��]�>��H6��6�r��veQ\VL{M�g�����$� �=�����o�d0c{��5f� ���3O#��J����i����8H��������0����,��,������{��������G�F��^�(l�@�]^�}����	���@M�\��<P��?%�=�	xO��J~����wC3�.�r�M)�	x76#�	xO�{�U����������t}(��}G�9l.h��N�r5��'	x������'�=�*�	xO�{�
�H����	x7��'�����'�=�	xW	J����j�o������M����;.�����\J��%��� ��'>A�*�	�O|��
�H���	�7��'����'>A�	�W	J|�|�	����E�I^���
��+�����[���'>��	�O0|����H0|����'>��	�O0����'>��	�O0�CD��t}P�{�	x���l��N*IKA��M	tO�{�����@��'�����K��@wSJ�{���H�{���@w���w�'�m����vn�����O��_����/���5z�{��:��z��MV��f^u�f2]��z~sW}�� u�����_���L������FO���?������RG�9(�1U�[�l���E$�
����Y?�U_�?��7����`~�����'��&L~����`p�Xk5��b(\/�"��A}�D�$ ��5yh��?��i� $Y0����rXAM�8'
�E�i,p�s��\��,h�w@�0�H}.�}.�}���y����}�K}�K}���}��x�R�R�~}_D�����������_���g�Q�^����`0��N�H5v"W��!�������R{\���= �{�'��'����0���G�"ilO.�'���?>y���,�������XOS{�h�cYH������R{���P��7|� \L���` ���X�������X����`x���T��q����
�8D�2�
�x(���P]�ZaZ?*���������6��$	����@���)�r� )����jdv�7��C�wg�bCQ��n�T��-_L�o�z��P
J@t�"TefeUef�c�b-���6�-���5���o9=�����{��f���V�k���Z���i����x���6+�a�����'Y!�1s��N�N!^�cw:v6>����O�co���r�k���l���'�`�E����=�����
���A���_r��q����:`N�Lh<H��#/�b���B�n9~�:od��&6O��^������8�]��:/����;r�r�{�@���K��6S��q����L��%D*2�]n%|]y���>@l�v�5��*�%��a(�RG�l,3�%�B�-I�Q.���b]�q2n�X�Kt��_D�
���h���1E,O<��X��b�X�	�bq2
�rD��"�'fFB�X����393���+K�@LQ#�a��������f��pM�i2�n�F)��v���",d"��.�B��?o� ��!��!��(bo���A�C�C�]Q��U���/��{��{�������n�
�B�HD����84�n��bQ/���{Zi���U�|��������������������������������������j����B���<`P��=J�A���BV�z�z�z����!�C��������.�weC��
���=�=�}@���pU{:���uw-��,��;��������*�
��@�@�����~b���r�^^����9v���z�g�?�����;�RW��8�cW�1�Pu��u����x�A'��Jwgzpe�JS�`���a1�f�H�)�$���D�(mf��KlA,)g�T�*���M����������b���jD��N1Z�L�id*�)H���h]
����������V /��^DOE�Dh�@C+�DC#���a"�g���������a� d��W
��S"�q1S+�pI�<�H���fh�%V�.�f�v�K����T����4<�1�rS�b����S���n}3�s�a���ZVP��L�h7�(������Ts8�3�M:�9bop��J�?���&��78@�������g���PU
�`�{yva:j$1��������r�&4[_o��6\��Ix-<�������6�U�RMxH��O�)+���u���f�W���a��[���������q��M�@�*� FI
�6�S�2������h�����r�h�5���Ty(��S^[�����+�u��@�L7�r�����r�}�h�:���M7�l���*R���d��*@��~T!�1n%T�*�P�@`d��Q����+A��ZT!g4�z�����6����\G�;Z-%VOD���p��p@?�C���|A���w���'��s?�/��0�;�O��9��>�c�s�}������~�h�C���>���C�I~8�8�����~ _�'����;�����/��0�;�O��9��>���-����P��a��g����ma�-~�2)\LO�'������no���^rz��?������'��O��~5X���td����n���TE�ONp��f��:���Y,��|q��v|w�f����-���q������L
�v�2���^����7��C'��9����]����q��Z]��������.
m�J�yN	W��.�J�L%/b]2�,��d*�ka2��d���/L5��m��D{Q9J�}#�O�*��@?�%z@�������0��Z�ZnM�j�H�I�L��V�w�o%��H0�68������42����w|~~���KE�!e9g��Wl1Mqe��VJ^�6!3����]W`�e��������w���}�|_:����}145����
�u����V�+@���b�Z�W��.c�����Z��jw�X7=�c�Dt���e��)=�e!�^����#��w�}Ecf��i�	���n�;��- F����:	�u�d<��E���F��R���X���/x����������6�J�$��orO�W���x?lQ�1!�%��7B����!t~�B�� ���!t~:�:?�L�:?s�:?b�!����6JL	�	hJ����cg��ld-�FAf���$tbey\���v�A�������?�O?�O��o��kYN�4�R��'/o��a(����ub���f6r���|�$�������4G�3ND���Y&f���rcB=�Tf���S��cc��5�S�cE��z��D���ob��T���XA7��z*��_�|���U���\�Q\�������l��F��-�(�����l���Rx��;���9��(m��v�Ys���V[�*PNg}AI5�H!����s�Ps�%es|���
!���ri�8�av������\�������v��VN_�az|z�����z�����Y�c�bx��7�P��~a���3v��O���/Hi�}>[��l��z!Z�.g��#����xl3�!��9K�������X��I-�J/�ZC��Tk����JQB�z}0�D�R���
��H������(Z22
�lF�W����V��4���i1:#�N��@�9?�H���Ct��=:�y�F�_�����*��8��8��T3iu7P���gQn�j]��H�����R*������#��X�U ����J#8��*�
�Q3#Rc�AS�X�`;�iD�r��"h�D+�U�w�4��A��
P:'"O�vm7Kfh\	o��K��|�����3���:�Jg^��+�y�3�t������W����Wv������`g^yF�hg_y.t��@����U�����g������������/�0b��3G0�>$������/���)e�Ri�&�����;=�ph����;9�b�����\~�t������v����������tHp$�?��,K��+����'6F>r^����W�l4�W>}����m��FQj9���C��'%�?8��g?9�y���[�&<�P���f_xk��H����%�+p{�����N>����K<M�?}V���p�F����/��W�o�+oq�_����b�'�0K�/32�b��a��9?����J�;������N��|���6{&�j_j=3
ig����5�D�)���4�n��"^���z�����@HI�J�����}j����_;�b�H^Cs�&\a���o�c�s�[�����g�v���d�P���:aS����PK��	����("c�cA.������U������R���l�y
���s)�h�)�q�7�BZ����?�o���1w���C||�������	T�#���Sf�7�29��u3z���Q\�9|�6�������<�+ZpL�r��=�A��3	s��c�=�yM>)��'����!>x��T)�#"3��e�?�S�Y�#���(���,�#f�	Y��w�+�M�k�e���}W3T�O�L����>�tIy��/�\^��_4'����/�H�dG,��DK�8A�k(2�
FQ3��bri�(g���25'��N�d���0TN�Q$��������c�����Z2�6�S�)�y9&o2�M�
Y�zcY�lT��V0m��������Ev�19�����1�Kdhn����������1�Klhn�������s$�;fr9_p�!���Q����a�rG�x�\�t7M�6����L���5�.N�QT��g�s��kFG�Z��,���6Y�D�O���0X^9�*��b
P�r�F�yx����k��\@7v(j:WlMs�RQ)��Vf0���H9(6�8�~�Z�_�a�g��xX���=Jm��R�2�E��`q.��u�_o4T�Z_���M�����S�1��b$�������
�o����6O�������&c�[\��|�6�1�����6��|�����7�V�X��O:�o��������;���pN#��*�q��w@�����2�~G�������o��1[��`D��'4�cp\0��L@����	�m�P����0�����������'A�1��0�AJ�'�o���|
(��=]�!���x��X��Pp| {��P`�v�������7�8��	�o�s``l'`V�`8�@���	���3slh� g�1���p%��p�����r2���B��������@3������p��U�ey�`�w�p��c�r���J�!�k�&\{ ��{�#!F�&����1k��=�7��
qO��~y|���s��;����o��������8��.����i[�o��;�.� ��p�y��=�=�y0�G`�
�\���wm�X���}�s9�`�������	�
wC��U
�@�z4��'�@6d)0����	���� X'�f�{�Lt�1�b�=�*@���B���
 �@���	����Pc���?p�+
�����[A"�r���

W5��\��
����2;�� dP��{�A����q	{?������t��~�z?�k�p%���2�j$HkN0��<W)(K��J��.P��� xo%�	�c�����pp�	�(�_k���T`%q�����>9%��t����.��9!����+�,�T�\S,x� (8K�qQd@%�����!�@����2��a�d2d�m�\�b6<yR3#��I��*r��@T*+@�F9�*O��&�`��O(�\)"8{�@]��������I�(NV�BT�>w"S��/��g�
���"|�������N�@�;��/t0b�&�g�=	��F%|�D����)�g,~
�;��-|�D��s'R?�����$|�D�L�s'2	��x-|���[��Nd�>w"]����B�3V����(-|�D��s'2Q
��8*|�DF
�����F���P����b�����(��$gZ�j�������N����K0E:��s���0����]^[�s��Y>&\
��2s��z	�������_��S�aD�6�:v �%3��ey*gN�"����*�p�����*�x�N�b��������b����s�J��Z6��������13"+�M���i2���w;�����)z`F?QS���A?�"�B?��?o'2d����<EO?��`����a�n9�C��A�1�+���=!�z�c3_8=���l+��F?��L���������FM�`3#>�}����q���h3Fo����?QZ�9��EC�	�<������A����sQf����^����T�����hM���>��)}�+^p����9.6�<{TV�(�������|Z�)��G��W��X��)$�)�I��n�6�\]
�P�"�
�����<������>��Otk}�;kZ��J��yq�7�rt,Pf%y��jW��#O�s6G�������X����T����u���|U<4��O��������'�$@��Y���h���yQ��/�_o�LM�� '����"��l�c(e��a�S��KU�Y,&��X<�����c�����]�W0�!�*6To����7��8R>k@4�N%]���d[��	��KsQ�tnfu�u�P��<�b�+��5�((���cf@�p+���q�xXh�u��)�a+4i)��%���DI��	�u1�1.�c�G5�
F�j��s��qm�;��c�_�^���v$g^2T�/�N�A�B�_�<&��,�G��!-zAi�w�PnT|���g�A���^*��p�9������1�
����b��y�F��r�$t)`:��&�,)�9CD�^"s'!�J*���:�^n3H[��A1�#���c�"�}����9�d��w�i,�#�E?;��p��`���4���2�����0&�h�������3���3�G])FN���xX�P6L��&�@����)=�x��P7d�S��#v��T4�xhj��x`j�R��'���Qo����K��H��F����&/	��M'��0�8�3���)�_}���[y>Zl�k$�=z�{�{�,�U�y�����O�����rz~|����?��qz���S{���?�2B_�[o���^�.K\�.��HNa�����3���X�����.��? M�o�H2"d]��`�:0�0�����}�w:=����������~r�
�n%�sJk�����[�y��F��W��-rB�#�M����.U�3���O�L�*��L�D�3f�(�S��X�/.v�:}�u��JC�e���U�_��L�iy�4�$B��xRk�	��)�$h�R�Yz��[&h]*����-��L/C2���f�����o����������#���>~��_�D�x����J.\�Kj���2��M�INX����=��?�I��3��Q���h'P-HW�"�]KTp��Jk�t�A!vq^vx�q���e� �LW+�o�%��Q�t/UvV���G����H	�&#�)��&���JE��)bi�yJZ�XY�]�V�P����wz�W
,�9����t�9�*��+�E�+�������y/���rS���1z�@o��Bf����U��/��S���A[;���s�q���_��L|~��&e�lT0T%B��ABf�k�\f��V
z��������-��<�n���kL���
Rb�9�����a�qxS��	���D��u�/���}g$�3��L�;�w���������{
�%�(7�b��G���lAF9���K �'W!���[�ac\�?��E�{0q�������2���@�"���Yf=\�f�&��d��n'k�bIgP��${DS�
t��?C�		h�;j���1�3�$<~�7�s�p��
�xb�i����T�*&Lh�,���r/~U������x��Y�m���lQq�-�Ceq���*����_p����+Nm��Yp,���=�s3A}>�~��r���Hy��G_���KY:Iz��Pe�m�����P���Mj�'8��vL�>=�rI�D<��h�����)��WB2U}�g��O%�qa�x��7^8�S�'��e�s�Uz�En�U|xi�=,KuW7�����&YJ�q3�*e�U�f.1�����0a���IKu��V�v���<���;���c�m��(���~{�#Wg����}���~:9&�nE�\|�����7���z5�{�����V�z�#v�	?�Z����0��k��v������������l�
���0��Q2�kgoo����7g8��:{�t�����:��<\��Y����j�H
�;����'��x����}�Y�:Q7�����>m�n�W_������V�$��z�\�������wg�H��e����/f���p�\�7�����=\/gs���}o5�G���g�u������o���o��C�^�ky{�jw�x7_�����U������F���k�y�+ �'���|�!�,������_g��^cl7s���j�7~�W���1����g�G����7�4���S_�Y����9���?�Nt�
�����f0�}���o��?.� �M������hR�]��*R\~�����$RJ<<g�t6�����������BR�>����
��O��K�Hg�_�W4���I�ov�X���x_:�~��r�v�;���U��������p.��������!�B
��G\��
�z�9P�i��m��
	"����s,�����5�is��/���we�xh��|gv�\�����2I�k'����G��h&[��
ip������^l)l������_�A�zM�{U��i�}��_���xy���[�����0'�8o�"}����/�I�4G�t�p�"���B�C�l�-��aA;9������o�S�t���M����������MF+a��g?O{�O�6��<��'�0�I���_�L�G:��'��uuu+#�V&���@��k�1�v&���g��K���G�`�y��Y^a�����~�/E�������]4e���j��{��3y�������������
���M��JwF�t>|������=bQ���W�Zq�X���{0W�}-A�Y��p��9��_}����K�26��F�bH�G��l�!�&����D��asUL��e"W5��V��W�#3���&r�2��2Q�������������.eS��\%�l��,E�<��(-����9��������|��$\hn���3�F�6����P�6,�/����������\���P�o��tL��iv�F���X������4�0�������������������nQ�	�7a�+.���X���O�L�Xc�r^������M�����C�w��Hs�[b]��xk��?�d� �A���y���/<�o'����>*�K��h�d7\r�$��D$C���!�gX5�EX����u�J��o�e��hy��)��
����)�9�6"���*\��Zf�Y��q,,`�l��R��Y�QU�iJ��|�R)h��w�K�8�rh��������[#��d��a��:@��:d�+���������V�C����r�`���)g���g�R�p+ /�z����fl��-�WeEPv���j����$<�Xbl���<-^�<-��%�"B!�(�
��g�����7>�d��(��"�����w�����?����F����������A�R��6iK�m�j+���c�S���n(�%|���������g� ��1��j8�����f57?��jVS���I�d5e[Y��Ym$��+���`5W�jn���jX�e�f'm������!F;kB����G�V���\"(��];
Z�m�'TVQ,�����;Q��R���=.������?J���$zQ��f���)%����j��'K���4��'��a�����5Z�y6x����
�T�J{Y�eYV���Z�����J��}����m��rD�0���Y.s_��k�������o�1�����v����q��Mm	1����P�O�����TAJ��+���B��+��^mu������0v�;e;��U#����C��:S/c��������\&P7�h p�7���\�Q�3Qb:���\4��]3�Gm�����]n�$��Y�uz�q���{���Q�yS��Y����Y�7,
/M��3~$�������?��&�t���l\r���>=^����:���������������<������F�s#�s\�b��G���s�����H�������F|�����,�[���k��H|k���*�J"��(�}��`����m���
�����A��@��1�����X�,{���m�/�n���n}����A����*lpM,)I����j��&�ho�SO����.�p�Y��'��;�*�������,�g�LJ`"I
�gk^��r&l�;����E�X�I"���6R�A��h1n'�!�;���u������(�������^|�n(L�
��C������M�bNN����PA%'��9�Us��EsNt���l������
n|�n(NAdNt�DW��.�D;!r��|}�J��@�&�stz���l�O�]�%���� v�>_�8nT�r~��g�g�X��D����J'q5�7*	�X�v������������y�b�Qn��DzE����Jb��1�w��!u��F�iw����]��{]��V�i���������;|%m���f�#�
J~t[$��@r[���I �������Y���=p���NQ�C�n>�S�������t�@�p�(���D?�@�s	"?��NU$�K�P��� f��	�y{t,lV���X^�R���:kO ��)�P*��h���X*��h���`��W������	���!P���������?2�R?x_��H��e�'>C`r���4�O&X-�~T��
!�� .���:'ptZP������/��;���a�D�Dc�h���Q�qd �������4�u
�����WO��C�_
���i���^\���Y�MD2��$z<�n'��pw�����y�d�y^������+n��V��pK�{��^3��TC�6x�blkJ�$��A8�_�^V��D��4=d�+g/1�e�Hi/EHB�01C���er���a=->�3��y�1!3�������&�Zw�6�B�����z$5�1P�1+���,���n������<��ax��y��Wjb���B�'�#����W�.���e&kVD�W�����7�k.*x��/ne^��G��P�M�t�L9�]7]�K>�/-�.;p.�y*N�I+jw���|���Ea��Ww�:�c-��U�-�VX���@>*�ow���J�
�S�������q�OZP�g.��������/����/�!i��&m��/iW�����������6J������i{(���r���_�i{$��&{$�H1��h��{#%�
�9��(9Vs�X��1����y�oeK�� �$�X=���2J��K-�����9V�X1��d0���o�5�i�����r�\������gN���ig�-���s3���W�PB��dR�sX-�<X?@h�
�	���b�t�?B�0;���f�
h�����7I�	�~����c��*�!�/����4��
IRa�����,��bd��`~K�Qt�)����s�`�3����E�^������ru�� ��������<��V3qb�,|��%�I��p~a�=�L�px0Yz��4�KU�E��p)�0+��D.��[s�+�u���.��b"t,�Q<��w
�������\AB�����q��6T1���V��������<EK���VG���.4���3���@���v��k�d=U��*�RA �)I����]hI2J�h�2�#-1T-NReH�n1��tC���P�C�����������n3Q}�����@c���|hBJ�����p���������]>�R���]>���,�z�P�${�P.��v�p3�w�����QC�UIK/���@��0����^>\���*�W�|��.�f�p����/uEY�T�p��A�qp�N9����[A��V���o7�GM��l!S����8$�
��X�l���W0�b|�-����P	K�5
�G#z�+�XPUXz�����V].,=��d7���D������*. �]���0c��"���"{��HI�D~^����������B��f��8�;k�aG��e��$h6I]2Ib�^UD��9k�R8�yq������R�y���D��?j�E����/������g�q���^\�_L/��b�
���$����U���ffW�������Ugv���7!�2����|IIlfvPd'3;����"���`fW���e�����J�����dG���+�m(TBVRE�mZ��,����B_�Je����l��$������#������Lq���U�e�m�n!\!���2��&3M����������P��L�5q��&I�S(��yVgt�����,�ool��Ox:#��1j
��e��8f��)X��F���
�������~9
7_��YB�<8�������{�'$�E��|����?B�=2P+�����~���{�;wKg�tp���M���c����!J+H	�%��:8��_�����D|��K4;���*�*�a�m~ld��5��a����n �w%��7���V�4��8C���9���M:�;��W��A�����c��p+������7`�v����c�����O�&����B�YJU�s�-���
�Jm��S���c!��J��%Qc�DK?�����P��BYG��-�E����NG��HO�n"��J��l�t��7�r��j�Eu��L��u�:3���Dj�|=[:!��IL�%eJ��h�>rl'd!ZX'l6�[8&�����o����*�a�����
����P	�5Vh�H55y������,@�G�������f���E��������7���o����)G3��Z�����;��T�S�f���gQ�2��B^F��h�E���d@H�
���;���-�x���a�������p�������������`���jK�B�������%����"]B@��b�T��\�W���Er������.�?���)�!�Ax,E�o\��[z����
�e�j?���q���Y%��T�����2�x���
�E�@�7E��i�x�
Mk����f}D���7�yj-5:�da%�����c)a���s�FI����Y��E7��������`�8��������s��\�s#����9����������~]�_W�/���A��j����\��k���7����aX�PQV^.*��!���r��,6R��=^E.���{o}Ox������=?.��c�cWi���i�jWi�+�U��*Rv�6,�Y�p�������������}'8u�S'8u�SKw������	N������N�
����{��O��G�7����������9:t���C����.E9�����Tqhq(���+��L?��.�;o3�JI;���p�k������w�������+4�x��f%W����!������/n�kF=T�Y)��t'��������
�Y;�m�#��������a��l����b�U����bm�X��5���d�YZ���D�+R�H��_���L� $e���S����f��|�C\z!���C��:�.�!� ���w�iRHSnJ�S�`I)�A�@i���ZIr'����1�4e#�D��	B�_�1�8�_���+��(6�mnykq6J��+2�2��+C��u`�^V����5�z�i��X5N���Du�Mf��n��IK��2j�3o��hx�� i@�X�Q��	���7;�rQM9�"[��NqeFn������U�'�fv������Z�3aK�y���AJ�Z^����a�|�K=Z�(0__{T#P70beF�h�)����$��c�����s�OMc���k���sM���&��qx�������!U��`3���~�s�kl�G����1�R9���
�ZVU����A�FTU�q���X�:i=j:��S�MkQ.��o1��6yO�\������j��������*��+��������TWQ%����[��k	4J!�E
��-��?��`�-^���+�=G�J7��lZ�oKi�=urNJ#�=u#�B����*�S��|��XW]���V����u�-����uU�b�����U
Ul���v]u���v]�������uUC����]W�����]W�-�����uu$�������u�U���z]u���^W]���*�UW�������f]u��j'43���d\���<����m��J����=�d@1M|Q%2;�n �����r��q��)��o�[�@-�)%�o������3�] Lp9�]u��r!�g�}���1v����6J�Ri�_��q� N�L0�_�L
$���{���s���E\'��GR^J����H�g��Dq�,L�$x��N(���r(yt#P�X�%Q����\(]����[��FH�n}{1#��o�;v��[DQ�j����MjG}���D��hU���6,H�7�*�V���s���x���w����^~�1p����g5���-��s���h5N����&�SY��g�|�SZ�oK�V�|bN����sfS�����Thv����r�5L�����)aJn�����m/8���4��
nD{,����V�'f�a��7��^�(�
�M�.[uH���fcGv��V����3���
/��!p+��	t��������a�D0t��'U!r����s;|u��������t�����l�P9��!��{r����i�s���V�'��%�
l!
H�P�Y!n������2�OpH�W�$8b^��W/QP7�a�n�u�����r�3��x���$nEMx��}g���%����qz��|��c���h��+)��+I�r��F���c�&u��Y!G_|X����:�����K��`�7����6+u�����k�Y�g��f���T
!'�=:������)��<��4p���C�����4�?=�8�p�,�G ~�p�,�'B��N�]w������iS�O����c2����^
c�^a�yzz����7�����<��5��_.>���c��C���:m��,<����`�B!��V�F��#������8g�w�s�(2q�G���
z�/�-5F������������;�b�y��J��p�b�5���k��&���i`8��
��!��t}��'�p+�10-j���(�c��=�Q���J���	q��o��T�6Y�)uv�\�� ��[hw��r~|�?-����&o�W����?����vEa������E-J�J��xX3�uh�#����x|��9�����!�#��`V�#e:�U:���a��&���?������< ��8���1��A��="!���G�^H���&,���b�{�Kef�Q>��.�g��.��{�9d�K^�����,�G��i����f��;m�.�}�����G���'�o�'��07�����#FX�)h&�z��0&Hk\����SM�������M
��-6s�H���\$�Q@������C��S�(	FX��p��t��[ZSc��	ijL����x����{���������v(�F<�1���XJ�:;���s6���K���D�xBQnjBX����j�H �E37�aB�(�h�|�]��<��o�LYG�v���8 �6a������q���0�C�t���z-Z&f�<�C�� ���QL�9Brgo5�����9���)����q�Vy��r�x
�Z�_f�H��X��h������H���^?lW�v���{�$h����`�^�yJ��J4j_��������������G�s�S���o� kp�5������m��a_��7������ i�9��v�O����+�+��'G�`��/L*������[�"�v�\�g�T
�I��{�A�o:���rSG����)�\b��,p�Q���d�M�3�H�2�(�*M����k��W�XA0D���`�?�!o�-0P-�(*F�-QFEO�I���:����6LI=a�����A�a��N9��)7U ��D�V��4j�A
�569�'����^.��W�B���h4�����A��-}�a��Jd=�6��6������+�tl�1F��c�"���.'��{%������������qL�3>-J8H�X6jV:��f�yn����d����n$�]����e��V���!(8����4��*�x��g�|��/��=�����Ra����y�:r�+*L��%)Ke=P�h���
�Q���U�w1��V�r�jf�t��
� �H�u5��R�[��TV���a)����Z~5��0:�a��������&��C����-3~���B�(+d-���6�k�J8��M�dn/u�Z�`6�lEw���V��h2�����y�b(�f�r�f����U-n���-X�Zk����.\��`�3�$Z���W��`9.`�^��"f�a��g��W!�I���r���]!���t�s~[r�Gh����G��gs���/@}/e���W��GM��t�)E���P���Y���S�Z"��o���z��}�"��������8r�ZJ%�u����R�;���ab���7�Z��� �{<�����G^�\�w=��z-����v�j���WE<�����Ir��jha�W������Y�<�no���^rzIB�����.��bbm0����c)#a>,�Y9�,�j�}���<L�pun��L��g�������� P)!���e9WI���,0��K>n����6w�e�����{_
2E�!��L�������10%
��t�W���1������>��X�cu�|$�!Eb�mht�������c����z${������l���8��`k;LgIg�����K�K�EJ7>L�OO?��{�$��#J���s~��S�<�� :gz�[��wy+�S���Ga�j�|�����Q��k�0��&;v�Wk���m���32qON�js{;sN��_�	��k!A#������/Z+(��W���}����8��O_�NHRM����������/�g���+�T��C�����.��{�|�C��G�+t7���$y�����^�D���9����C|?��?8��g?9��V}�[�&#����#S�a{�<����/����� �8h����v��vrJ�8:��~�"��Wx�Y��
�
�7��J{�.e%���_�B�K*�`��U��@�i�&T�G��T���@@���>�>��I^n�+��{�]A��rQ������6����y��@�Q��p)�$�l�W;�	�Pmh��B�A���w���D������[iq�27m)M��PK:���L>�-B��j������ �d���#-d�-���t)����t�������:z�f�	��J�t)2L1�S��me��ZJ���|b�kH���"l	K:���s�������'7(u��J���g����QNKqP~��%9�us�O������W#�'�k'�g��5�5��t�	�����b���x��N.�LJ �i)��v2�B��!��D��
E����\p���������u@Zt#6���6�D�~F��TDZ�kM@Z3��3:�:+�`���2�����f��h�*�y6br�GSa�i)�5i�lW��V�����(p�Tl3+lO4c�v^!O�(E7�2(W�x��P��Z��
@�4�����i���X�&[����4��}�Va���n�\S�~�L������V�#���s���e������:��I�H����i�@o!��&H����!/�5&��LH���	e��-����a� �/�
�VP�b�����!7�c��l������e��m����"f|Z�&�G�q���Q2c�PI�d�9A����w#}yQzw6�5WP1�I���L�Oq��Ul/*���J���0V7��Q�i�{|,&��8�m��I�"���il#Um����*2�$�����4lU�TI��:�*�ce��)���<��+<&��u�L��$KS��x����kH�R7�\a��R�xE���I�R9��l0������9��S���3A�.=�Cl��������T18Cn�K�B~��t�)o��7p���$myA$2y�#s�����5rQK�W��{N�����Q-�x&11�0��(��t2Y!��f���/�9�$��@6�Z����7�\���m��/��>����w�j��^J�<�^*I�L|�dP��
�T2��u�0$��l�B�7�5&�Ql�
��*FUC^u�&��}�H7��GA]E��j���D����#N�tA
���Q���V[�J�R��|B ��S���`�����D�t3[��T5������A��'4�pJF����v"P���Tj�S�%d����A�8����~u#�y����-�7�`r�V�~&�ZL��kN
.����H?���SN���S,	��s��*[�6�X^�9Q�����mHUTR7��A��"���Yf���6C��u�o=j
L�,v�D� ����L��Z"�1j
��L�{�D��^��[�v-��&BC�9����P
�G�n/����u��M	��6>���a�������m'��O�����s�a�NnF����r:m��o���l�W���9���ob�Z�����W�%�9��r�����^�������U������P6�(��D)E_��!i4���2u"wl0��l
���F���}T�B�������c��-�������y8�%��^����zm��9q�Q���K
��o��gb�ML���]Z�Z�9�f��C�������Y�K&��ckv��Z����-;iuaF�zy�.=�����(U��<	eap����������>2��lh4�>��L���W�)�e���4�[pU�IZ!�U����#����-?f��q�kLvV`�|4���]>����=fF�����0�I�l�:��7��Ar)�s&?�������*4�C�b< |�@5(A�A	6�9�Y����]����;%3A����B�9*1�� .N���Ms=[�l��V@n(gt��_(�[;[�$���� ":������Q�Y�'3pG�,r5�k:u`T��+UA�VQG��]S���c$+T=H
����RG�R�T��A����#9:�%C��o������l�jE���T����!!�w^����)��*[���P�'l+���U�Z��������|p���$�n���x8-3x+hP"Gxl���YI���D��~�C�����k'�g,cm?Z�����r����K��b>r&y
Z���3)��5�G���[��������a{��+�y��f���:5*������U��5�c�M���;�(G*����8Ka����L��T
����YV�o�1��L`k��B�n`r5}�csv�Hpc�e�#�����������1R��e7|�nF��\&#Qj�v�4!e�k@�&��E�2�PA��F�C���ZZ����gmiZ#���T��9��������|>+K;��
/,Ud�o�S}���������t�N����T"��>=�T�[3%�&c�li��6�&,�������
�(��@��4W���H�������n���':��'�����S}��d�>�=3t�(z��!�"���'�PZC�U�������K�B�ELgc�[O=#<�+�0ctn���T
�$J�2�����u��D����
����=��u�.d����qaa*���/
������y�X��!��)a�<�������<��J<,�(6�9O��<%�P�����[\�b<M��<E��4e1����Ki��'A����]l/?(Ja<d^4w[*��H�z������"�we7�����o�Yv��P�"���Q�/��|����/f�p�A4�qn��+���?v1=�v�����_�������!����:f<���PK��	��1"d�B���o��<|��2������o�,�|��{|�x^'EfD0W���w�g����<����K���m�oz~��}�T���?q;��^s2��������B �8Q����
��!��M�4X���� 66�r����8�� ���+��w]�'8FN�����^r�q��S� Yoo��'fg��������{B����@���	�����l�G��=�jj�~��k�.������f��:�*PK��k�b������:�I<%w����<)yi%^b��y+D,�83o���R��L�����9�5t=l+
<�S7aQ8��tI��S4���Ow����?9N/ZF�xz���+��}�/���7� ���Hp����,����z�:C?�Ug�~�zR.;��i�o5=D�m(F�f~{�8r�x
���f��{�{|��@�������o�;�����������?����������L�
�t������3/"=����t� ���Ol���������[��>4��8X wO1�(4L����Q~>���"����i���-�x�L����j9'�JwK�o�^�2�$����}���dg�tWwuWwuWwuWwuWwuWwuWwuWwuWwuWwuWwuWwuWwuWwuWwuWwuWwuW���z�|g�
#178Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Thomas Munro (#176)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Aug 10, 2017 at 3:13 PM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

On Thu, Aug 10, 2017 at 6:23 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Your patch didn't improve planning time without partition-wise join,
so it's something good to have along-with partition-wise join. Given
that Bitmapsets are used in other parts of code as well, the
optimization may affect those parts as well, esp. the overhead of
maintaining first_non_empty_wordnum.

Maybe, but if you consider that this container already deals with the
upper bound moving up by reallocating and copying the whole thing,
adjusting an int when the lower bound moves down doesn't seem like
anything to worry about...

Yeah. May be we should check whether that makes any difference to
planning times of TPC-H queries. If it shows any difference.

Do you think we solving this problem is a prerequisite for
partition-wise join? Or should we propose that patch as a separate
enhancement?

No, I'm not proposing anything yet. For now I just wanted to share
this observation about where hot CPU time goes in simple tests, and
since it turned out to be a loop in a loop that I could see an easy to
way to fix for singleton sets and sets with a small range, I couldn't
help trying it... But I'm still trying to understand the bigger
picture. I'll be interested to compare profiles with the ordered
append_rel_list version you have mentioned, to see how that moves the
hot spots.

build_simple_rel() which contains that loop takes only .23% of
planning time. So, I doubt if that's going to change much.
+ 0.23% postgres postgres [.] build_simple_rel

I guess one very practical question to ask is: can we plan queries
with realistic numbers of partitioned tables and partitions in
reasonable times? Well, it certainly looks very good for hundreds of
partitions so far... My own experience of partitioning with other
RDBMSs has been on that order, 'monthly partitions covering the past
10 years' and similar, but on the other hand it wouldn't be surprising
to learn that people want to go to many thousands, especially for
schemas which just keep adding partitions over time and don't want to
drop them. As for hash partitioning, that seems to be typically done
with numbers like 16, 32 or 64 in other products from what I can
glean. Speculation: perhaps hash partitioning is more motivated by
parallelism than data maintenance and thus somehow anchored to the
ground by core counts; if so no planning performance worries there I
guess (until core counts double quite a few more times).

Agreed.

One nice thing about the planning time is that restrictions on the
partition key cut down planning time; so where I measure ~7 seconds to
plan SELECT * FROM foofoo JOIN barbar USING (a, b) with 2k partitions,
if I add WHERE a > 50 it's ~4 seconds and WHERE a > 99 it's ~0.8s, so
if someone has a keep-adding-more-partitions-over-time model then at
least their prunable current day/week/whatever queries will not suffer
quite so badly. (Yeah my computer seems to be a lot slower than yours
for these tests; clang -O2 no asserts on a mid 2014 MBP with i7 @
2.2Ghz).

That's interesting observation. Thanks for sharing it.

Curious: would you consider joins between partitioned tables and
non-partitioned tables where the join is pushed down to be a kind of
"partition-wise join", or something else? If so, would that be a
special case, or just the logical extreme case for
0014-WIP-Partition-wise-join-for-1-1-1-0-0-1-partition-ma.patch, where
one single "partition" on the non-partitioned side maps to all the
partitions on the partitioned size?

Parameterized nest loop joins with partition key as parameters
simulate something like that. Apart from that case, I don't see any
case where such a join would be more efficient compared to the current
method of ganging all partitions and joining them to the unpartitioned
table. But oh wait, that could be useful in sharding, when the
unpartitioned table is replicated and partitioned table is distributed
across shards. So, yes, that's a useful case. I am not sure whether
it's some kind of partition-wise join; it doesn't matter, it looks
useful. Said that, I am not planning to handle it in the near future.

--
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

#179Robert Haas
robertmhaas@gmail.com
In reply to: Thomas Munro (#176)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Aug 10, 2017 at 5:43 AM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

Do you think we solving this problem is a prerequisite for
partition-wise join? Or should we propose that patch as a separate
enhancement?

No, I'm not proposing anything yet. For now I just wanted to share
this observation about where hot CPU time goes in simple tests, and
since it turned out to be a loop in a loop that I could see an easy to
way to fix for singleton sets and sets with a small range, I couldn't
help trying it... But I'm still trying to understand the bigger
picture. I'll be interested to compare profiles with the ordered
append_rel_list version you have mentioned, to see how that moves the
hot spots.

Perhaps this is stating the obvious, but it's often better to optimize
things like this at a higher level, rather than by tinkering with
stuff like Bitmapset. On the other hand, sometimes
micro-optimizations are the way to go, because optimizing
find_ec_member_for_tle(), for example, might involve a much broader
rethink of the planner code than we want to undertake right now.

I guess one very practical question to ask is: can we plan queries
with realistic numbers of partitioned tables and partitions in
reasonable times? Well, it certainly looks very good for hundreds of
partitions so far... My own experience of partitioning with other
RDBMSs has been on that order, 'monthly partitions covering the past
10 years' and similar, but on the other hand it wouldn't be surprising
to learn that people want to go to many thousands, especially for
schemas which just keep adding partitions over time and don't want to
drop them.

I've been thinking that it would be good if this feature - and other
new partitioning features - could scale to about 1000 partitions
without too much trouble. Eventually, it might be nice to scale
higher, but there's not much point in making partition-wise join scale
to 100,000 partitions if we've got some other part of the system that
runs into trouble beyond 250.

Curious: would you consider joins between partitioned tables and
non-partitioned tables where the join is pushed down to be a kind of
"partition-wise join", or something else? If so, would that be a
special case, or just the logical extreme case for
0014-WIP-Partition-wise-join-for-1-1-1-0-0-1-partition-ma.patch, where
one single "partition" on the non-partitioned side maps to all the
partitions on the partitioned size?

I think this is actually a really important case which we've just
excluded from the initial scope because the problem is hard enough
already. But it's quite possible that if you are joining partitioned
tables A and B with unpartitioned table X, the right join order could
be A-X-B; the A-X join might knock out a lot of rows. It's not great
to have to pick between doing the A-B join partitionwise and doing the
A-X join first; you want to get both things. But we can't do
everything at once.

Further down the road, there's more than one way of doing the A-X
join. You could join each partition of A to all of X, which is likely
optimal if for example you are doing a nested loop with an inner index
scan on X. But you could also partition X on the fly using A's
partitioning scheme and then join partitions of A against the
on-the-fly-partitioned version of X. That's likely to be a lot better
for a merge join with an underlying sort on X.

--
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

#180Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#177)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Aug 10, 2017 at 8:00 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Attached patches with the comments addressed.

I have committed 0001-0003 as 480f1f4329f1bf8bfbbcda8ed233851e1b110ad4
and e139f1953f29db245f60a7acb72fccb1e05d2442.

0004 doesn't apply any more, probably due to commit
d57929afc7063431f80a0ac4c510fc39aacd22e6. I think something along
these lines could be separately committed prior to the main patch, and
I think that would be a good idea just to flush out any bugs in this
part independently of the rest. However, I also think that we
probably ought to try to get Amit Langote's changes to this function
to repair the locking order and expand in bound order committed before
proceeding with these changes.

In fact, I think there's a certain amount of conflict between what's
being discussed over there and what you're trying to do here. In that
thread, we propose to move partitioned tables at any level to the
front of the inheritance expansion. Here, however, you want to expand
level by level. I think partitioned-tables-first is the right
approach for the reasons discussed on the other thread; namely, we
want to be able to prune leaf partitions before expanding them, but
that requires us to expand all the non-leaf tables first to maintain a
consistent locking order in all scenarios. So the approach you've
taken in this patch may need to be re-thought somewhat.

--
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

#181Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#180)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Aug 15, 2017 at 10:15 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Thu, Aug 10, 2017 at 8:00 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Attached patches with the comments addressed.

I have committed 0001-0003 as 480f1f4329f1bf8bfbbcda8ed233851e1b110ad4
and e139f1953f29db245f60a7acb72fccb1e05d2442.

Thanks a lot Robert. Some less patches to maintain :).

0004 doesn't apply any more, probably due to commit
d57929afc7063431f80a0ac4c510fc39aacd22e6. I think something along
these lines could be separately committed prior to the main patch, and
I think that would be a good idea just to flush out any bugs in this
part independently of the rest. However, I also think that we
probably ought to try to get Amit Langote's changes to this function
to repair the locking order and expand in bound order committed before
proceeding with these changes.

I am reviewing those changes and contribute to that thread if necessary.

In fact, I think there's a certain amount of conflict between what's
being discussed over there and what you're trying to do here. In that
thread, we propose to move partitioned tables at any level to the
front of the inheritance expansion. Here, however, you want to expand
level by level. I think partitioned-tables-first is the right
approach for the reasons discussed on the other thread; namely, we
want to be able to prune leaf partitions before expanding them, but
that requires us to expand all the non-leaf tables first to maintain a
consistent locking order in all scenarios. So the approach you've
taken in this patch may need to be re-thought somewhat.

There are two ways we can do this
1. In expand_inherited_rtentry(), remember (childRTE and childRTIndex)
or just childRTIndex (using this we can fetch childRTE calling
rtfetch()) of intermediate partitioned tables. Once we are done
expanding immediate children, call expand_inherited_rtentry()
recursively on this list.

2. expand_inherited_tables() scans root->parse->rtable only upto the
end of original range table list. Make it go beyond that end,
expanding any new entries added for intermediate partitions.

FWIW, the first option allows us to keep all AppendRelInfos
corresponding to one partitioned relation together and also expands
the whole partition hierarchy in one go. Second will require minimal
changes to expand_inherited_rtentry(). Both approaches will spend time
scanning same number of RTE; the first will have them in different
lists, and second will have them in root->parse->rtable. I don't see
one being more attractive than the other. Do you have any opinion?

I will submit the rebased patches after reviewing/adjusting Amit's
changes and also the changes in expand_inherited_rtentry() after we
have concluded the approach to be taken.

--
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

#182Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#181)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Aug 16, 2017 at 3:31 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

There are two ways we can do this
1. In expand_inherited_rtentry(), remember (childRTE and childRTIndex)
or just childRTIndex (using this we can fetch childRTE calling
rtfetch()) of intermediate partitioned tables. Once we are done
expanding immediate children, call expand_inherited_rtentry()
recursively on this list.

2. expand_inherited_tables() scans root->parse->rtable only upto the
end of original range table list. Make it go beyond that end,
expanding any new entries added for intermediate partitions.

FWIW, the first option allows us to keep all AppendRelInfos
corresponding to one partitioned relation together and also expands
the whole partition hierarchy in one go. Second will require minimal
changes to expand_inherited_rtentry(). Both approaches will spend time
scanning same number of RTE; the first will have them in different
lists, and second will have them in root->parse->rtable. I don't see
one being more attractive than the other. Do you have any opinion?

I don't like option (2). I'm not sure about option (1). I think
maybe we should have two nested loops in expanded_inherited_rtentry(),
the outer one iterating over partitioned tables (or just the original
parent RTE if partitioning is not involved) and then inner one looping
over individual leaf partitions for each partitioned table. Probably
we'd end up wanting to move at least some of the logic inside the
existing loop into a subroutine.

--
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

#183Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#182)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Aug 16, 2017 at 5:21 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Wed, Aug 16, 2017 at 3:31 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

There are two ways we can do this
1. In expand_inherited_rtentry(), remember (childRTE and childRTIndex)
or just childRTIndex (using this we can fetch childRTE calling
rtfetch()) of intermediate partitioned tables. Once we are done
expanding immediate children, call expand_inherited_rtentry()
recursively on this list.

2. expand_inherited_tables() scans root->parse->rtable only upto the
end of original range table list. Make it go beyond that end,
expanding any new entries added for intermediate partitions.

FWIW, the first option allows us to keep all AppendRelInfos
corresponding to one partitioned relation together and also expands
the whole partition hierarchy in one go. Second will require minimal
changes to expand_inherited_rtentry(). Both approaches will spend time
scanning same number of RTE; the first will have them in different
lists, and second will have them in root->parse->rtable. I don't see
one being more attractive than the other. Do you have any opinion?

I don't like option (2). I'm not sure about option (1). I think
maybe we should have two nested loops in expanded_inherited_rtentry(),
the outer one iterating over partitioned tables (or just the original
parent RTE if partitioning is not involved) and then inner one looping
over individual leaf partitions for each partitioned table. Probably
we'd end up wanting to move at least some of the logic inside the
existing loop into a subroutine.

I originally thought to provide it along-with the changes to
expand_inherited_rtentry(), but that thread is taking longer. Jeevan
Chalke needs rebased patches for his work on aggregate pushdown and
Thomas might need them for further review. So, here they are. The last
two patches in this set implement the advanced partition matching
algorithm. Those patches are here for ready reference. One can observe
that patch doesn't change much of the basic partition-wise join
implementation. I am starting a new thread for discussing the advanced
partition matching algorithm.

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

Attachments:

pg_dp_join_patches_v26.tar.gzapplication/x-gzip; name=pg_dp_join_patches_v26.tar.gzDownload
����Y�\{s�������@�3�dS�����&�N���u�\���l���H�bC�*A�����{	R�c�����zZ?D�8/���Y������4Jfk�K����_�:<��O�p2p��'��O'��p:|2�'�d�DL���4|mt.3!�H����^�����G����G���m�<�����{k��Q��
{�����~]�D�'}���5P���x��'�xxX��p::>��
�?���Y���(��?�����t�����G*XLF����B�.M��Z�P��?1i���sqbx$^K��7�����J%���Y�U8���U����sq��x��lb��������d$����}�������������o��?�������QQA**Jm������M��(Y�,�e(��T&�`y'���
$�Tdj��"_�v1��I�W�(�(F�?7i.�(D�'��J�K�%�T�E��=�{���e�mZQ��B�:��L���uN��B|
����[��@�)���'���jO�*�(<�2�'Q<�2�C:_�4p��n�W����?b&
��j�����H�s%�zG@>�)��U�B�����o����L�LG �;�r���G3�������V rO�t����8	�h�$���6����m�~�M5�A���Y�B��-}��2��I=�8������"ba��a�q�|DQ��E��3 	��;�t�*��%y��X�F�RC-i`LE> e�'�
������cmb���Gx����*����8%,�5���p+n��X)0�R/�v�$[��=����\�a&?���YZ�r��Xg�Z����I�2f���1��q�n�`8o.Lo��H��a����<�3��N�p���4���^UB�Q��P��A
����( 5b���ke�.�+U�S��������J��������$!~�U�2�����5����I�n-�
v��L�Q_@�Uh�1 $�T��K�V���i�}�^��kl]�H��v��sM��?SJ��h��F����p���~���������>����<ZE�������0���@��|�����p��(��o0g�0��v/�
p%�8�+_��}�����m<<o����RA���� ��� 1����KQ~�N�9��9���A��'��#�~��E�
�
�/q1~w����v�GE@CoT��h:�UF������h,BP7���m��h���
�\<L������b*?<����aL�hv<F�z��m8��W��+�������������?����@����v�e���|�cx���<��n��Ex+�>�/�Y2\EIm�^�M�����k��"Tq�������^������:����v4=���pB�|&�.��F�4����mo6���s�?��B[��1�Z�D�y�_��������k/�����=��Ff��n$(8��/R�P�l�
]��78�P��\Y�R}�w� 2
>�h�,T�|���h���N4�g��jvu���������;�vq�� �����Rh����0d�?�,����AJZ'u����,�	����t��-��a�v�wkUS@�iK�0M�O���\�0�y���x�����}�?
��*d�)�igJph�Pb[+� ^�D�O���_��o�1z�t�Z���Zf��#��!��8�7���������?���@�`
:��y|
��g,������D��jE��%^��������w�|�|M_���`���'W��k��O�[A���(����=q�T�F�������>e��t">��d��9e��
����).H��F���	�8X�������R7���Q�tv���)�U�7I!n6���p
nb�*���':Z�p5�GD��;�.F~2�
��b3��VL�F��eq&<������\x��L\/�=#p�2�����5����q�s�D���KE�^�uq�-\	�N�������KR(�srU��5-vr}v���F	3J���~w��������=};�>y}~�E;s�0�{���
���=.F�)�Y�y�V:q��s.�n!^��;���~�a���;@�?�#5=����>�������!8�}���@d >�Q�n����&�������c����^���zgK�������i ,*�����SD�u���MP����b%3M���U
���e�\�9c
I6��R�U{�����nI���geS��i�@+q����c��Z�c�i=,�����?;GU���a%|V�>0b����NRq"��J�����(O����[K�k���Fa�d�X��V��'���y�vv�.�kBV�[�D?�����1����������u�I��9���6�_j@�:���
�E�}�����jv>[�<F�3�����P��]���X��� �U)�����%��m=��H�}{���"�\o] f��>��?�c�<����a�'�1%t��Z��\eeT��Y�hUM�x"
��-f���+�w_�w����_i/Y635  :��!�)�8+�Y`�H�iDl!uN�(�"8^�gbW���_��f%�
��d)e���?��:��-1/���y4X�v`f4&��8
�Y8�N�'	���k	[�4��\,���������g8p���w����p\y���U9�Ur�/;U���
����tQ����.���.�d��"#s=I0�{�����$�L8�Q	��A�$��Ya1?������|���y���\P����t�������Qk
P�0����(�B@4�9FB��}�?BG��T�/j]��/��G�����g{��
�J8a�nD+d�S1<�������B�es���H����o1��r���9�u��,�U=���[����h����n��hZ#�~r�3���{�V����R�6����KJ��J���&`]���O��(�,�
�"G:s��8�>[XI[`�>�;�����x�������.B�hM�����|/yb�bM�P��a�,�[|�o�Da����A��lpC������)q�U����]0Y���1��(�e�
��JY�ILle�$��FI�g$�^Y������Q���-�T<@_�-:(A��x�$wLv[:��B#=�g;����F���F��)
�5&�����	pOYV��U�J?����C�����z�3$ck��^��X�0�	!9�d�-��k=qUeP];���;	���L;���� �j�����zFw>��k���B9�x����t�N���K�*�US��� _b-�I������&TU����X��i� ��q�Nf����:2.�UyI���:�g,��L��`gfU����dL�P�{�,�+�`	�m������C�X�W������<^}{F��J�b�3a�_Wa������=�g�|�����+b���@�
!v#��'�)���M�V�y�jd�3g^������"p��N�����s�������7�wJ��KsIc��ieD!��ZRMI�v�,�>�x���@��1x��p8�@i��p�N
��-�KS��F
dl���-}�&�j!�e.)P	S��p.�P*�x�S���`$�3�=��Q	1�����~����C`�xa���4�`<����lrJ-�2��@�~W{�����"Rg�?F�\���V��bQ���*K:����:_(�������+��\ ;�-O��xWq�~��Z���(�d4e�8<���),�0v�'��6f��K
$��QA<�LS���q�T�}5�S]+d�+aK�(g-��l�+^�����xP���{�3o\h���p�����h>�|H��N��|u}0��hL)B�	m���~��|�&��B���������T�P^���P)*��#��E���������^�AA8s���)��1-��X
&Z��Tq^�_;A�4[�II��9��p��g����39��C�]s�T�d������	o%R)\�����\B��c���/r�y�Xu{N-�������*R�B7�_S�t��u��aR55���|6�� V�oE�3C�)l��X����`=�05�����E��8M)��[�6�����������-]������,-���!T��<��s�_�OsU�{
�0�f��t���4(�O�Fm+��k��f��$���������K��J�8<�vt4��q���<XS������m��M��R+e)����w�.����4��W�&k_������Vh��i��4n�-�}�}(�?+P�:@:;kP��WV��f�y�pV�Xc��������&�{:x����H/��4(3~�*����P��D�$��p~ ��|��y�(Sb�
<<!Dv��p�rQ�px�>�����N��&����#i�86z�,���!{��H&�O��x���)�p;pf�wI(b��s����MG9�E�f���vK���;�����R����8�cR�a\�[��R�4�e���r�C������>�gi��4��q���w��e�N�T\Ll��&�p��=+Qe�������A;*�n����vA��(���EKL����E')@L�����31�3=�q���q�#�n�56LV�>E�)����_[��3oqY�������Y"{e����\\#4�'Z^p��(�%X��]Da�$
�s�H)Yl!��x�a��?�����/g5���7��yLk=��nx�1��[������1�u�Ip��(����ke��[�=W��#`��!?������I�a�������<����<5���
?�l��7�=oL�r:����#�d���',!Ghe��Rm�*�Qv�5�Y�4��!���H���9�r
�&��^�)��Wc��"���K�I#�������|\e>��Y*���gXq�o��.��B���)Z�&�	�X��G���������mW��X��_�I���1[�rKoF��t�b�� 8�����w�p3��a�9e�<Cf+�6WtC`
Pc�bz�H�����&N��.	�k�&5���u��"��W1�O�������?�*�E�s�����B �������� 1N�JrM�S����[���fEM��e�q�D�u���m�������)Zr��O��M��c)������w.L��GE~��4w�3����.aDF?�zcw� �7�D���FZ��|��fK��
�5y�Yi#�Czo��F�p�o��V��z^��%0���i�&�]t;A��5����"@�S�y��=��;9,������7�V���i���e��.�N/az��pp!������-��_�!`$��Ny��x��V3���$���������l�g���Nub
���k���������1<.�=q��B}$� �N�W��������hC������?�H1��n2k��\*Gk���
v��
gH�a3-�����\+����d���:��?3=�V�P�����/��<��+�2�\E1k�oBn(^
����
�<Ox�������-�vh�� k4�r�����l 7q��1!�]aQ�_Eo>|/��+7h�E$�.a4X��R�����������}4�$���+i#��yC��x��Y�0�JO��Eh��7[��}%�m|}�p�^�6Yt�����8v�����]�4��|0���c�iS�=��[Z�}�i?��h�Rv�;��:�j���g`�����D=�p��q��u��R�����h��=�A���7�v�R;���)��� 8�~��p����v(a;���&�����>����L�Ob\�m���Q������������Sc���D�N�����.���
OB���������^�e��IhEp��d<sK�{����2�%���_x��
�mXB
��k�\,���l��*+�%�������F)"�N�MNY�=On���+�?h`���&R�i����?�A�xS�F�������L Z�e�n&��0�>&�{a�N
.�U[S{�G�.�����&6�e��ZH��|	�(�Y�$ ����"�����������w�m,	�����)0�3c*�d �d�����g;����;���R���T���<�������@7��
���F������������p�;8������2�=�k�h��,u�E�_�QoY������H����W��<������-&����K�W�O*!�c4��:	����r��c�F%�2!C����@����H��R���=�zf���^tRXSB0C�%��#���V�;��y��	�>��wd�i-�>}��6�;�N���������qy��G�SwulHF�O������c�4�����X�N�A�d����WB���6w�h}��A~��`B��������=�q����wIJ���'h��H����Ev#B�P��{��g��r��I������14I��1/��n�lNu,|{j�c�C���_���b���)��=&Q����������>���oH
}V��@/��cV�d1leh%a�`����B�+�G��k
)��$��$E���7/��}�b�9-i�P������^�����I@Qq!�RRy������bA�Fv�L�ir����G���A�ybV�8J�bTj0��#�8�S������6}�0.8]���=_>����0_�����*��;�����������yo�5�k���3��|��_��dqKf
��}n���_/��:�n���j\�`�����z�@�����u��.��z��\����l�
����q�h@����7H�&W�Ivx2�o3����u�U�������h�W��!4R TX&�x�N��3�����G��<������������U/���wr�����(��&����@=6����A�,�U�����_-�o��4�"�R_JuH0�l��/���.��%��2�,�,����j��~�i��Qk����nVTM���������a��(���x��CG��o�C�"(c���oA[�e%xD���p0�.q�Dd����s��b�e��9F+/��
����tB��Ig��8���>K�*��qw}�6����:�
(~�>�y�:�lX'p[�|�g��x����GIl�e��t�T��%D��i|�`��%}��g���7l����gr���F���������IR��%�T����{H�|q{-��W�;��\���^�����[c�x�������:�%�32�����V���yd�:���pg���N�s�~�p�jW[l���OGm,���2�x�J��q�V��� �L��t���U���`�\��fG}0�;���Y�����#����
�2�1em2�2z��XXn��5����-@�F��wk������#�#CDC����|F��y!��-�G�b�;��O�Cjs(�<�:����Q3�����C��mn!�X�U@E��V<�M�k8�C�	a!��u������4%�d�%����o~8�|���W/������_�~8{��5Z��2@8�;��� A�A>��<!G�r/�D�c��T�UY*�&�� r��|�<f������������%v-s�����\1:�1�`@D���m��t���c0z����n��WJFV`[�5�Ei7R�9�J����S�~�W���KZ����xW�jb�
�U)����� BM,��2������7�0�� �'�e�nd�.�vA~b�E�5�5������(��/0<��z��(|�
����3%�w��
����������k(i���AZB�K:��l���T{���x�;���.�t��^�W��4�.��#Aa�u�#� N�^��YBXs�X)x�v��/X8��|_���B�\)�h����J�M�1���I:I����$��KO��,e.r������_�u��y:���	<t���u�3����-���4c��a�(_y��g�|�7�[+�������%H,6��8�1=��S��o���]���5~�����FL���-�W���As�������D�7���F�G�CN`r(�&F���<�pz��������������D��"��N�+_Q����P,���/���!:���0�����hWE�3�3��t.�%�']� �S��M��t�
��:�>�v�)�
z�+��a~x��L�Z���v��0��x���j
�������V��m`�0�[���������z���+\���gn*�����Qm[��me����wW�7�����bw���������tl����Erh�����'*���1�8,��Z8G�1��6%��@���lv��T6����"�@�5�j}��_d���j1���>�e�"����{
�F~����L�Y��H6F
"��i�O�L��<B;L
�����L|}����>���H���_.�`��~^����������)�����ep�	u����or%��]��g�����-���fJ�P?���5��C6�b�dqX�z
;V���]��3/���7\���>S#����������!X���>fK������'�/^�����ab�������}\h���q�(�����W���p|�&I|yr�����$%�����/��.T�3Yj��j���]���4�A����Atg�%`
*��&7����}��0@^���Ng)���e<K/����t��Q� j����B�:�l���"F����?��~�u
��n�����zB�P��)�D�a,�j(,���PZK}��/&K2�JHt��4x�[���'4����n�����0%�-|��P7�W�7��z�(��v�?��$'���8��:�A��gP�@�-1�2�v^, M0����j��B�����K��d#V_������sn��Al��*�v4�X]���M�����Y/�nyR�O�����xX��u���a������;����������;"����
J��i�?��b�]q��!8��&�c�I��Te�=�����f=�ao���y�9��C�m���}w��J����"|:�����ZV� �����D�G��9�;���wh�V)�c�=2�*�%�Xt�������\�:��d�X���4��
�S�5�A7.����s|��-{���[}X��f�W4��2�t��\����i�6��9q��������avp�.����n����Z�9_�����c��q����7d&a�
�������mFN�/��~�a>���R���c�n���7���a��q���#dy����A��\}M/�����}�|�.O=����f�8�}�-��V�R�_|�]����)�A3`�����!��:��s�w T�R���q��Z���
��nS20������l����N�7�x=��g��H���r�ewH��gt6���!��tE���"��n�8
��]������^�C i88�j��4m'��0j�����8[��2�B������ks��.H���G/{��a/�����4���
)��`=_u!���F4��,��R�f��	p<v)�CD��C���`W��O�[�����uiy��������H��vI�V��g�9��6^������Q"�Z��)��WW$�<��E��{�����t��]e���m��W�	la���X����(�#;#]Q��k�!t��������aO��O�uVAQ�uV���<:�&�����K�c�N�4N�"
����)Z���p6q��i���i�����^��)�{R�j���Y�+#���C��8�KRfq���n���D��Q�I�n]��`0�t2���>��t{�A4k�;���Y�D����
8[�g��p��B���=k��{��������������1�����
������5"����K/�"3���<�
�g�L�)#F0�%��j���+[0��-��+������i����k�F	(��y���[bf����""
����dIP���B�l�u�0Y�����s��"J&^�C�����b�h�������E�����c�l�G;K1~:_3�u�5�����A��t���l}�Y�n��a�����`����z�Z(�!���a�b��UT
�|	����5�L�V�:���_��W���}B&I���}��P �b���,�;{2�
@6��&K���{	���&�h!�-��td���9����05����(Y���%�R�[BS���J�"�l�d���*�=�%��}�9i���lhgo�����%��udka�B����7�Hf�����H���;�5F+��.�����=�"*�Ls��C	RAo)u���D�pB1:��2��y ��w�!r� ��md��xHh��?gG�C/� WU��+�j�C�������sDe�_�c�Q"|MP�s�G_�!8>b�" 4e���}���T7����3�)%�I���Cme�&����J��01�9�g���Pu!�:�xF�������%?~�*8�c5d�-`J�'O����!�Xf�8��il�'0����b�O��4�$��G�����$��j3���!�)�_c
��"�D����H/�c��'Gi��� 9�'G'^����
�e�ObL����FKZ��A�j���������l��q����?&/T���2H���x�t�]%d��_.Fo.~:;�|����F���Lph�_���(�R��������=�(��ZL��}�d\���b��w���z(P�N�w,�-I��t���2�7������x�B�8����b\�R����G�J����88H�	SqK���uD�������%�)n����/��
i�sL����:&�����P��Mo��T�v}�����8+��B�1�a�[F�>��E��pO#�wDZoz$T����E����N�����X�����K����0�<f�|��R5BR��=�d�E��b���J�z��X��2�'E8&�n	B	*���+#�Q��N������g���\�v�ARhW���32bRb�;�G~��']��h������������JD��+eC!��;fP�gwY-H���"T�soC�H��6�3#��r��O<�<����J��P�����F2m�b.�Q���El7����+Le�"���mh*1H+RR"������@U�L�z�C�\Tq7�T<Gk16�>a{\��9�,�*:�����T���J[�!B��2d����XA�'Inp��5 s�j:�M��&���]��\k�-h
�0���a�{��"Pv�w�MEfI���R`rDP*c�����o2d�#j���[q�N���!\.���4�����;���g���&S�<9V@�I��}(��}1�X����u�@���n|�������Rb��-�`��a�Vl�8MLx �h:G��R���W�B ��u�o���B���Da%�	pC�	Fv.!�A��	w���g$�Sp�e��l�2��9�D�zN5�j�9��q�Seex�xYD�����L+�N����:�Y��,��%�$��~�����K2U�w�X���3�h����i��I�9����
�6�|�YE�|�`��KV�{�m��0�?��2s5Op�]������!��MI�A���z@(��9
�P��
�p|U�3}bt�� �/���j\,�2�>������z��Cu.���O�#���q:�}���'8�
`q�Mp�^���!�C��Jsb���y���U��s��}�D��NR�H*28�*���UXV�	�;�P�������IpA"�y����)N�.�*�%l:����,�?�d��Q#G���	X����M*C/M��&�n1;{WN���DrF&O$�3sGNh���J>}�x����?,=�*�uNEG[�U�����O/�����������O��I]x�r].d�	&R�D��a�? ��H3p��R��v��������������9��E�}����{J�
�3������F��r� tT��2��l+��-T��x�9�k%{�?���t�vt���^�r�����)�v�OhE���|��WyO%0c���t9�]�M�A�*u��?��$���Tz�,�
�"��"������o"UEv�6�d�V�;�|�>�D�r*`���{Z�1�yg~-2��9��n�����+n7Y?K�'���o�0����+Hq�����?������g���#|u��7�#���dU�I���M�h�&���*���$ ��A�Pm���6L�/�wh^��T��5$��6]8�����a�d��0��`��~P�F���j��"���=h4W�,����tBM4y��Opi��������oA�b�Z(���� �7��������x89<�x���J�JY��&����d�&�`�=E�"�X�x
��4�{r������D�+�4�d��1������)F���%��5$�I������dh	e]^S�)eq���n4���Gc���
�W��>�c�LQ���_2��X�J2P.�KO�=a���Y��o�RQ�Vd���!$��0+��0�DPlD%��tc�lM&K2��yRh~6��VU�`t�@�5@�3��$�{�IHc`Q�����5M���,��z9�Pw9�5Etj�<DkI�N�f@bf6���A��Y�Y6�1itt'�&�EBp�'�qB��XZ#����B�dB���ie�O�R��A`�Y�b�����SY.f�u:;��X�����G�Y'�8����Y-l�V�`	��%#�	J	l '��$E�zk�
u	��]��8By�1��/��05����X����YvE��F-�����h��9���uE���j�YN2N�BmH�;����g���)k��a�f�`�A�r�I���
3�R��J�`�����m���2!�����v2`/~}��/T�u�����p��V*l����
K��:��f~;��(-����,�Hk
����������nV��H�I�d�O��yoD_p'���/@z$_��LzI��(�MG�����H�Vk{O����Y@z�X9���(���'�V/��n��K�r��@���et�d`a�����a�G�#v$!���0x>����Pn���a�+�_������XP>b�����9���^T�f�R��|�}}�XR�2<�����#$�;T���
D�������l*��;w�;��2N,�@7�-�����7�^��Dq�����,�S���$�.�$Y^���������c}��h�Yh��vN%@����mm��<����87��kK-���-�A%}`f�1
��[��T���|����-�����h����A��63�w������� ���4��Iz0�w(2W6r��D��,b*	���O�c0��:\!"���D��1�zW����	����a���P�(_5�Py;�T-��*,�V���b��T�$�e�����Mh��v�K���<��1z�@��$�8��8���ht>�\$�2�{�����L(�
��XQh�X���!j���M�b5bhz��/,���Y�H��3=���4(�����������!no��9Y�d-�
sY��\�����:OtXT���AvEu����6eE�[��s��z2T��(:�(L���0����9~t�Xe����������7@
�}�T5�:�nN��b�]E.`���>�K:�1�}h7�y����1���Z����PCu����a=m�}�R�#��DU�xo:�g��������M�T�"����J���N�%(S�@h�����BZ��z��\*w��dN�����k���h�]��=�������-���,��I!Z��:E4g�^!��:Ee*�&n�J�� @+�E��!�M,@��f.Pl!����Q�r�����a�j�AX���/"�+]��Fg���g�&��UJ����p1,x:2��1A%WS�[�c{*��������l����-�i�"��ts�]w�?���������
dr��4�i�usC��h#��s�>��� �2��V3�XDF�^S7�������D�������h���w��i�����>�g���=�.�^C��`��8��M�E)G�.�7�3��	�.�s�~�F��9������<3+�	��1�� �AW
�P�r�B��;�lH�������*��M R��f~�b����1&!]T|(@�	��RQaE���/�mk������Bc$u\B��h#y���H'�� s�u�` 9c���tv��Z;�D:�N�=M�}.�6�l�_x��]:�W�/�e4�8����n�"�����T�'���:���/����Y+�]v�_`_A�}�xK����C����]�U,^I���l�	z ��h�A��P	���\�����4�>�K4��z;��ig
^a�NzP�/�+�/#:��g��b�%�S�����z��G��t�X�0;�<!�<^�c(wi��=s����S�Cg��/��
5��GN]�,�����Xm��8�1=����W���Xk�B�']��7L���MR�A�{�/��<cX�)N�9x��Sp���3U�2���CU3�bKS[����29��n�����0��oMXwy��A�O�i�h�n�_��H��jE�\��1�'����+���	�q&�M!�����{�zlgT���2I�"��9���V��2�Nxp��0�e�X���J+��a��Du������p�A�5�|��px��`��2G�Y����F��?yr��X4hQ2'@��}������~�;��(M+���.��)5��0W�NR���p���QSQ�B�`CSF+i�iE�g�\P>KZ�]�,��/�G^*��r�B�U6{���j�)���c���&��������(�18F�O�22m�0���������y���Cv���c�0"�`�z���!���$�;{>�C�"��19?�-*?*��g�#���<'�7�S�������w��A%�F�7O����E	��s(�c[:~n-��)�|
�8?�R��Nc�r-:������s-�M���S��)^Z+kCs��W���!v���[���;P���,�@�O�3x��?��jz�[_�K*�S�{���������/��A�*!�����n����f.�8�BK�d�+�,v����������@��^d�N��Fv��X3�4n��1b8g8��o�l-����*��B�/kCt�`�����
��������[�ws���,'�Z158�EK�2J�33g���s�Q(����M�^,�����gL��!�����%��`�J�3��e>8�8Xb!�����l>��[���s��L��b���������d��c?#�������!�q���������)L�$�pSb���,��.�I�e�q�Q����3I��F�F���q������{�b�=�`�
��3���B@�t������^;��r,H�Yhp�����I	�$��\D	��>��\
l�>�Vo%����3��Qaf�7�\C����\�)�h�����L�Z6�|����Z���GO�Ff�:��!�������(Bt�"�k%���C@�N�:��f�g��^v��\�)������	�������|��,��N���pT����"���cpn��1=��Z�[	��;�o���7����J<RdV8�z/Y��)8���Z1�*t����\�O�$�l��)
�k�����A�K�	�k���`_a�L��������[b�$D�"E�aC@(�����m�Q���'s.��%t7n-�SRJ2M����!��]��'��8�A��R=h�3�#�������)"Nu&�
�HG)��(������X� -P5`h�XJ@�+]�9��f��K��N�Cy���>�7����E_qgo���]#�w��d��@*�����Q��s�����C=9hN�I�F��-A�L�X����W����D�Z��D,��(J��yd����e��9�
���R��>�����K4�S���,��
�4w���3�!K�g�������E��/}P� KI�Kp��C������`d.'�x< �wpu�]��C/�
��B�
�0�r~��>�>d.�(2.�SH�ij�nV�.��L^O�[��c�`M��-DLO;��2Z��9�~F�g%t`�[f�)����L<�����x��(
���w�|-33�&h'���2o����.Q<7��,�5�UMBo�O������4R��d}{7bd��	��+N����,�iV��.��#�F����)���Os�Q*T�����)N����g�
+�������������r��{�*��$�J��4�|X1�>@�*um�a��u�������I/���Qr�L{n|��Q_�c�.V�/W����'|r�8c&����Y�6m�_+���S*J/:�/[`�\
E�|�M)3)��@.���(����c�*��fa#�,	8 ��(��������G)�zB�L�\��_~8�8c.A�PV��>��0	���|5�8�
�G����#��`4�t������?N����P9�f	��jL��������{&�UL=t�l����-���W�]�<\cau<8�R���'�,%&u+9n>�Y�Zbg������57;�4?�a�s��$�"��2���"Tb��Z�L����h~H����>7�7����}�p��>��)M����+j�J�u�4|�����,AAi���T�<�Wc�7p��i�@~��9�}l����j���G������qY�Z;���=d�h�!C5�C����B�qZP	DX�������R���,)*z�t�!��xE{�!*����nA��������W=5���
J_�'��)����/o�H$,.2t����"4+kN��bf���hy�q������b���&KkG�<X���O��/L��#��B���'(����9��]�&�f����������x������d|p0><:&^��YS_m���	8��O0RA������K*
`�Z��i�k
H�9�x�.�w5�����& g�K���^{-=k�B�E�h��7���d<H'���UVZq��-A�=�j��@C~)�a"`@�>��@���D${X��,@�"���rb���&�i�P?����S��/
�z��}��mF��K#���K�!~�������\���LD��/~����aDp'�J����2<����@H��X������m�Y3��RO�\�������/���<1��j��tE��e�S��Jn���8���+>:�9_���Z4\�-PPG���n��h�o�ooG�o!ny�s�x!{&�gs��u�I��&�3��g���%��������*���j~���L5������P�hH�@������@��i���F�7�9�#��&��d"$$��E�"�����$��Y`~����p�=��Y���Bu��q��`P�iF1^����w<����9����iNf<����}������#��A	��.d]�~��-���3M��n����O/�7��F�NBD'j�j��oZ�_�������Z�������BJ����D1wut��Z������^*3$�H�>d�(�h�,;>�:<<88����I�M���J�9Y���(n,��
Iv/Bn�������lT=%�n����x���goHyL�����Xw#�;F$��jX'b�~zMs?�X����������nI����*�~�����q��u������-�N���
5�!�>��u��v�R�f�9��v#b�����9hq'�s��n7����e�5&"9{��l@B�@�
"5�p���e1@��-��a|3w�2��g�hTL��X�U��%����cV��,�8�o��\*R���j�T���F�8	e��e)�9#is�����[L��(H�OW(��&��4��dR�c�YU�06O-�
 	y�z�A�5��a�7"������x�?��T�^��0Fy�[�RCy�WC+f:>K48X���CED�Ux���VB���j���\@�\��1�J�"��JE�cdMt,<�~���xt��(�)�3
kN!T�L�cP��B����F��j$
4aZJGgh��Jk:F�	�r.���t�n1{��s��D I����\���m7�`a6M�2�����������U�&_[�n��-�fox���X�B�<3�tJf��E���D�*d���6J����U����4ohY	X�9?f��8����#�.���<��,B����_B�`'E��C2�$�������eJO�����7���J,��������ZJ����!F�	F�����Jf�����7)��|�3��A�<J��)_p�X�*�m0`y9��R=�o�"�[�y9]f��$K�P7��K��4�a`��	�Mm�r���pq��@�T&��,_�$��z6�\/�5�������������c/�� _����l��5�h���f��+�@�W���Mfk������md!���	z9���o��Z	H`[�~�b�B��������+B|�:I�BZ�w �����"�H�(�"���W\P�M�E����9A�s�v�26����zIc�X_ryS�����*������X�!S����N�Io�A�u���1�9�H�^�R���0W%�f_���R-�_+�@k\$�R��J�b�3�P;`���_6�O���[*n�9��m�T�C0�u�*�}��Z�f$�(��&k�����V�{rv
78����%OU�A�VBj�0���|�g��d�h�l��*El��M?�����Z��t������|fY�+3�������L(�5_�N��[	"$���a���7��Q����sE3I���0�R7��O��s�������^e�+�'���'*�D$�����&��3&��^)����y�P����`7���&��,�R
k$����G�c'�V�H�,#w��=�jL��5���@�!D�)xk+�TC�Q��p25���+G��Z�(JHB5�(���k���m������
fv�������wh<-�q�t��i����{�������������=��tNp&�~��,c�(cjI/�����	R��������0��Ts��r{�el�-L�U08���&���f�I��`�����)��f�q48��	Y����0�e�V�%�����|�<'�Yr�p!�+��bv�(y`)�S�{DQ�gH� �3T�S/+��@��QB����*Y����$��F1��n&��>�t�A���H����=�Xp�;�^@��)�:.X��.RX�	r9����&>��E���K�33#�?��Y�n$��������y�����
:!_�G��q=��������6�/�}�?�o����!:��
�?��<�!wM����.�
���t��)��Z�uj�C&z1��� �:�g1��������7B+��I/�'49C4P�m�i���G��[���JO�%+�����ZMy-��B�������8�����������W��M�v����9_��s�
���}�|R��O
��+�
 }�O���[*l��#���F��o�
�����9�F_l�b%0o%����!�U�/�_��/��mK_����hr�}�������;m�Z�����D�+~	T*�<���tL�0����X�!�CR�L+m��������P�(�Ur��Fa����������xxB_����d���	�fM�O��E����q�����[�������(]��l�@��L�-��)�R���
h����@��9�fz��1	�{�+�]��3�Bo���1&��.v�e��S]�&0�kv�Q������T2�k��4�������3'������ �$��E��(@]��VE���pK7� �+�6��=n]G,�o)��	�G:�
�q�#!B�:P��D�L�G%#��^f�#d�v�\TT����1�T���*�&����4I2=���l�P�x��#r�)���X�:q�oi�F$C&�C�h�`����.Y.	�V4Y��d��&p���f`��	�*V�U�����"��_e���i�(������6L`�N��y\�+��j�m�Y��#�4�q&��zgO�R"�2wD��M>��TW,t����4����Z(��
�
Oe^�K��e!
�\�~l
!j_��O?P�
RIYk�a���D)�B9��!heo���n/������)|������4	 l�
(�/7
�V��*�udGqN��.b2�\3$�������5F�gxi� 1������pZ@�3��S��f*e�r.1L��r��A"K����FqX-5��uy4Z��f��P�3@qB�8�b�����������&����x~R]������#��]��!����t��},�?�#��G�Q4g0�L�v��{�*����Hg��Z��"'N3�("x	�E�l!�1��c��CH�q:�E����t8R&��?���=d�.�&-��2��/���.�f6��J6��JJ�L�bW����$��
s�X~�R����`���;F�rwu���s�F��\���_�K���g�����U�������+R��>�f���"�����iL�UB����W�D�����Z�k:2�1��|�u�#�Ft�h%e2F�Qr���E��E7N5T�J�����_�����;��1��v�{�������1K�*:������s�n�\�F�S�Y�A��o^G�)�������1�
z�����6�����/����x��������2]�2H(�S0$�4������Y~o�?C���������� �Kc��2L"���G���+�������9�S/�jA
De,��>�]�B������������m������$0|��i�Y���9H���h����0e��g��0���L�|�&�����������R���?PHMoit�_���]�R�����!���"2{��>G^P
���3��Y$O���%p�l�hsRY���"n�P���0}U?��L�H���NW���T��x����]��U�?�X�s(D��oC8i����
f�.���a�����
�������"h���$����r	 �Zg{T%��������`\1G9](@�79A9f0z+)7_��K�7"�#oE��/8�C�TW"�������u",;}�aAA�7)�T�d�tFZ*�*����������p��!A���3��N�R��C���fo�_���6%L>���4�)$�dSC5�\����0J�9���H�R~\qr5��N��(�W�dJ�h�,��Q�n�i����G��
_T�����(C�>�����M��+��k����I��YsV[5Xv�����6l_U�P��Y+�� f�N��1;Bo��0|n��(�hi��;�br�Q��)y�z�T�/fz��&�74^"��{�X�H��x4�ex�����tq_��kW�8c���ru�{a��$p
~%=�����]fW�f�f���6�P��i�i��X`P
�P�`A������_"���D�k�GY�x&
Kf[s1���C���3|qs�/�9���IY��e�O�qH����H�V�t�9.dnN����F�#�N�&K�.W�o-�����?
Cj��
�_�<��Lr�4��<0���*K������I�B#WA>tF����|n��*���]{5�a>����f��&���{�����F���pq�H�V
]N���/�:G�+������g�o�.����������__�|���������@��u�G��|���|q6���{��.��'|��<�Ww�=�.�_���A�B�_\g�/
�����}�����j}Mx��g0�������g/����jx_%����q/�G��P���*�V)pc�X�����W�~r!�t�\���KYp����%��`a�9����X�9 �9�k����B0Q�3�;�Y��6b�k6���w|4�\&GW��ivV g�Y#��0���}��ph��o�_�dPc�@�����k4x���3q�������(zN=Q�9]q�3Cg�B���P������H��JxS|MSz�F��@L��q��<B{|���Oha����d�s�"�o��x`IUe�1Uf�f�eL��i��9��d��L��Nu�in��A|�����:&���0Oo�j��\\/��[MGMkX�u*d������?�0
��/6�aW}��J�Hi�BC�"�N�g���30Z���y�����\��;p�@=��k�y�)�a=?+�W�"w��+=���6KA`A�u%�O�$�v�C��4#���7��`!�
5������W]����)OhH���~��F<m��|��/�����g�?�?w�(=�Vb�C��B�
�%� ��na7��]0����R=Q:5-�g�,����Qm�)jX��s}���G���P�CY���VG9���qFJ?�jyT�-m�d��^��������k*�i���Snr��y'
���A~z�xh9J�/����Z��C���[���0<�U��0��e����^�����
�rS�	#��s�q�S�:��-�`UL-d�������f�i�3����@�K�~�a��96F�L��Zt�T���3��Sz
�T4t�OQ/r�N��|�� H����PsQa+BQ��6$�iKU<Tw�j
�S&��-^E��U<_�'�����-�)����cnC 8��������IoJ�3�H7�M�5�7��l�z3�^������U��7sE(�@qDOl�ee�9M��o�6]NS����|�����	����:����*Gm)`��eM`C;AC�t
1�� ��+s����d2K!��=��v��9h�v��Y2�9������W����L'''��S*)Z>�F�<�u���N�E�l��6�V���VB�����H�2"�#Oc���0Z���g�;�%8��yF�b2�J���|X;�|�C�3x�_��%t�/�?�~����'@�� �*����A�;���A�fY6���$�������������������l�w�`�6�04�A�C_)��a@Y�+��l4��oShjU����:�1�4�4�
(b;��vm�M��������d�%��^eF(��?��oB��p���?�C����k�D��=���`�}t�'�@L�v��y4��8C��=��@	jC�\�������KH�\lV���w��eF�m�TUB�������M���.7�����]��q�N��(��mxb���3�y������(voE����tp98�m?H��G'^�^��p�k%�$:���X#&p]���~F�c�J�����o�=n��];+~�p{`p�=�����,��A�FDVG*�a�����Oo^�@���y�������i3�z�����q�j|uiI�����h�; Tk�B�������d��!5]!��hR72?��j �HF�~�O��A���	�@>Z�]�����/�]��l���b��+���>��M����.�=rDzx<>:N1��:���aUI��(��Hz����
����Y����|��r7�|��d�7jy��
���t����1������w��K�� �a���Z,��	��	���\scGB����uk���}(:$*�aP�Z�>������w�#~�k~�)*�y�]{�/R�=%�P��u�|��p����^_���w(���C�El���w�x��j[���������-w�������m������M}���h����)��B�"�9V�\�=�R)����VB_6W	�x��:K�5J��i-�-������6U���
�����l%txPKt�	���������-�zV"���������*O�����U��M�����2�_,�A�wH�� !���5E6T��A�A��r�_�GX�5�eMD;������������E�uy'��M�z�#�����������j�?��Q��}�O�h���!�Fj<��4��w�s8���~�{��0�?���K����>zrm �l���6����}z����8����a�O���Y<>>9:L��Az���^6���$��/�����6���GQ����2�������Q�}J@!�����~�%'��*�L����@�WPk�#�V]��H���Wq�U�8��
��������p�*��_N/����������&�����sZ*���!����S�-�L�|��#K�������ox�w�*{x�pev�����J��P�m`.Y�;���fK���� ���e�����@���2�Kj�<tex��Vv�b��:��WF�,�D�	�Wc&l0g����'�;�a�5�;)�[��h�hG�$�����U�a1��	�$*w�%�T�I)&k�`���*8��+e��Ug7X0��R�1���D7���E�wT-�c�pOt�c�>�)��f�W�
w�D�{
NKsK)#��o�����y�P	�a��r�p����@}s<���XZ6,��(��!�AL���������O�����DRwb��*7F���!�}�^B�,�9Z�I���S����|A���0�+j*������6qv0�9'������Q�}��C��e������� �rq�s����R�\mf����P	�^m ��������f��ZL	EG'�^�E���-��X^v��c�$��(v�v	�%+9�PN�����W���+��^�1����$;�G�Q6��K+*��f�)eiFY*2�Mv"�Sds�;�kd�m�,%+����(��A�h^M�]w7�k�BJ�A.3#$�������6�FX�>�S>2������R��At����^JTA�<�?�7�������������������������o�!tic3�M�?����"���L���T��'��@�;�tuD�[�(�@��-�Y�J��mTE��r��~�d����PR#���}:��&S��K�A��HI���=��l����P���{2���N�:�T:G���Y��y��)���[��|���e���8��
�{��j01X��`�f�`eD5�����; ��cAx4F0��1�Q�=�?���R�dw`�@��:�,�_V��$����=E��������I��������������ONzZ�:���8 �?����S����+�-��[����K���l��_��aptb�L� Ii
@@t�������&Vd�33b���x����"��Vv�K;K��2l�T����4����f����5��Q�JD2afB�����"�-x�w#�Fy�2NK�2�e,M�"��,]Rnh5EC��>$�$��Xa'��w����SJ��,�P����L�q�,���prq�Tx�Rc�hz='�.
L���� �h��'j|`H������E�qL�2�����OhH�?(��it�Y���z|%xb?qz��|cl��fS7��(�'�K$5w��oS�hG�L��2���!5�4����[���D`e<��S8|�*����T�������UT����{W�f�i@	u8�t:<63�Xt��(���/�����������a���G�d1�����M�";&B}y����/����&Hl�n44�����J�i�
�~��� 
)r��e1��`�4�=E��^,�m^S\/��Y�89BG�x^tG|��{�>V�T����������<���R�����i��A�^�)��\�����w����a�M��
��#��a2���
#���k�����1|�z�������~@Fc���)]u&mB�|G<o�S�����)F��M'�5��(���(�g
�O���a%]X-dz�H	�R�L��L�g����GF��E���qA7��\���NFN{g/���a���*<�5�<#���1������f��/�d����59Oy8r�8�A%���sm������I�w��	JS��y�����������z����������2�����{hJ�\�"����Y��P�z���%��]����@Y��[�E(3O�������g}�C������_�l��`���$����*���� ���q2Hz���>.���Nz���0�gd5��zq������7�I|t5z��������{_
{_
b������F�?�����f<~5�����b	�}6��Ea�
�ZX=Q�����>8R�M'�>���@�s�S�X;JF2Pu�w����g��dN�h,�"����O�|v�������&J_���P-Jsa���}�d�k��a����M?��Y9>�d�(�x�b�= �u9��SB��JUa�����`#M���W�y��,{���X�&�/\x���S��)?R�i����dJ`t���WN�����#�~G.�|uJ���Wi%�Py�����~�Dt�R��T8-����I�
��Hn*���'�vC H �m`����Qww�`���?E�z�F��I��e:_�8�����2����*�,lM�a�����X����Z�"�<M���
�p�#�A��ft��;Xg�.U������p�JA�krl+�$~K�K�����D=4h-#�D�e�x��)��4�&���)�^����l��k����&�j��	��b���E�[t�#��08�K_.&�r]]�������.��j[��s*�I;�g�j��GSV����-�U���oZ7�g)�������F�tt�iP����_`��,ou���c(�������S�u�*��/\�q�k_g���xxp�N�'�U���+�z��"�4bra������%u�x?A��-�����k�H��$�7�v�c���N�E��o� ���
��������Q��o~�e��;�o��[�F�K�O�yF�~��=����F)9M�*���;�:��������Y@�U�{W�R�^9���C������OW<&?�SH����lX��d-,�
�����7�)�_��.~��;��+�"W��������s��VHc(�YQ��y�������a�/�������ZY�7
@��~C�G�0�;e�I1�/�H`z0�C.Q��u�Z<�@�QH]�#v7 ���(�8��W�tR>`�������=e��a�E|������fw��%���j�@�N��[����d8888�_��}�zyM��CFY*�������	�{�e�$��6�S����%�����+�Pu����JnI�Q|C�C�����GE2�K�!�P��m@H	&���BM����v�A�Auy]���w��q��B���h\��v�cc@J�AB����;�g�w�
(Z��KA,������u�"y�={�����r�� 7,7eA��y(�����(�P�u��A��'
���(�<1D�k�I�����i���^�O��Nyl�4�=?
Z�>p����o��,^�_�*�y�.1�)��^��Je��Y;H����2�8��f���dZnY�WIK�_���J��c��]Y�jP0����br���w"r��;9�]����8��0��L����TuV����������I'�)@g��y�a��s��#,�*��s�'b�+f���Z��,�e@������@���R8C]Dm�d��*;�-��GS�p	M��-[�����&�H���rk�V������)�����O�M�@\J��#Y7A�Mab�)F�z��kaUQ<sK���~�C)����@�a7�0CX�@?����	�*���5A��T~���F��u�g��,.S��lE+�~�����`�wd~H�$��0rb��	h�����z�XZ�It_c�0s��fL�a��I�X����J�pR��f�,*P��@��)�����L� 3V��E`0�B���������������zJ%BR����F�s`�x��L�����U�	&
!Q���h�mg����,e3n�[�:Z����)u�Z���T�L�q��;�z l[�g���Js���k9[w�Y� �"F�e0Wj�Y4XZ�82�6�A4)�t1�*~�V��T*�*�b:�C���������:NP'�y�<�~ri�E�a�|t�eX���
����2�a.��(�0���������� I�|)��"F�,�"!ju}��)C� �(����^�N�7_����
�}A����3���9�c%@�	���1�����}���Ax�.em��q���w��xG�z�--�i%3�i
-�oi0��
���N7>e���[����*�]����F2H�`�C�����uZ��M�E��������P�2N��U%�8�%���M�%3�k�(U���V�:��y�T
�Jq�x��9�=�7]�@C���Yb�fVS}��aP�H�91���%�BS�c�36�����T��C7{�.G���P�~S97���V8%9����x �-��`P�]��
;{�v�
��1\��T����W���o~���'����>�_���X�/�KAN��+�T�����N�>tDarv�;K!���0�'�t�9���8\�u]
Z�����7���o8C�S�z�L��lKkh���w���
��/���E�F�fQ\�2��Kd��9$C��M�2��!�l
P�A
�W�Ks�if>O�.��2��,��35��^������M�.F�`4%L�
F�����PRM+��A?YKS"�B19
���jK�+��dJ	)
v��,'�Qy�*#�F�	�TK+�o��7�49Z�3���`M�
$���a#�O<���t�C������'<�����7?e�]��W����"��o9h���4I7tvd�J/v"?/Q���ZL����I�|�N���ybo�4��6L��<��.���Dy��^v^O����4?��A>0�)	���M ��99,���<r������]A%��`��<h����Q�j�UZK�����\D>�0x���tS�U������Y:��� �����/�KN�����3Y��8�W|�����g�@S��Z��L`�/��] ��%�0A>s�Gx�1�-
d�*��U9)�������.&�\�K|?fN��9�i�E�Yal��@�\{L��9�&Qy}y���2������Ww���3W�x��@�+
@`���"zb{W���Wf-�;��81)�d��2fa��]��)@��.P�#`.!����@��p~D��I�Vf���`�|!3���Y|�b��M�0�����7B�tP�!OT��mI����	�n����1x�����=����
�c��P|l'�v������-M�W�]T�����-��A����a�mX]��j���
�n8�x���l��A9���i�������������;�&n S������a��sni�X'��B:Ae�,�<��o��UX_GS�}�_?�����������(M����
��+B���%��[���?;����N��jZL�D���s�?����c����n��+/�%<���U�]o���9o�e�{^��U���G<Z}��4�Cn�����JgeT�A�,����X��&J�
��JL���t��8gH�#u����d��7���K����,X��$t���Q���n�^�[m���l+u�8�����Z�z-p��K�� �T�� ?$�^a��H��V�KT�C	�:�������q���hF6����x�6���&|>�'��M��]^������x,��)��s�-G_^-�����+���F������;�j�#3S��o�s���8A�����AxN��x�c�$p�d��E�1&+vB���H?��1�1�����y��D��*��'s��q�+Y��0�I�<S�q�����
[m�^���Y3]����q���G��u���UJZ���Lc-E�%{%�OE��� ,����F�\�������7������"]�e$M��h�r�E'�4&��=�Z�@�Yv���)���L���cj3OW��5���=�Z�J��SD����%@�{i�1�Y��� ��<$*�l�F�G�s��O4����r�/S�d�������P���4�a�Td�	Q-�`&t�*R�$��m��VH�wS����.�T��Rv���1C�tF�����/��DoG,���Sw�85�������j��6����;���F��ZB�_�����7h��$��y��JP?gZ3B�����af�,�5����!n2�������M��D��aHaZ��Y���;�2���f�9~]&�,��z\B�4DU��;����M>Lf�BC�CV��,	�J��5�i���z��DF���]���3�������%f��*��!G��9����&��z���!�2�b�hU���	]��(l��`}�Z�
�K7���;X7���Cj�O�7eW���`��HO����+�E!Iah�3��sV
�e�F<�����f��K�
y�U�M�.��5{q�����v8����"lD5�Mz`�R[�[@y����@�W4��N i@�d�b�B��.�����>2�#
�����Uw#lQ��)�{�sO`���al�~��c��h��:&�>�f��r�[���vZ�$B��>pjN)Eq�3@��'��U�Z�q�Hq���2����������U&re����������#dyl��E��|������A[	\����'T���&zq�^�a���H�����CT����N���3.|��������W��� u���U�O��+�NF�!z����]�;_Zb����� 
���t�|�R����$��W�Z[Q.��]i��k������kB�|�\c@O�
��/�
��=2��-�Q��,��q��E��q�h�F&r�P�����:��/a'-1{b��i\j�oed�Q"s�r>��d~x��m�#W����!&A�,� W�(/[�#Aw���U��hE���00�*AQ��%\��W�e*q%�[�k\R�m��$���Y|�������REX
�� ��ZF���L�}�=�fp�=�J������uYK����N������}_:BE�x�Z��q����,=�����E�������dB��$��`zk%���e1�CL��c%{�+RzM%[�.vO�~�y\�*�uT�~d��f
W�C��r���
��y��<��v��nS��r�^�V����f���wL�����>�?�����������%�4��7�/���?���U����^rh���
����qa���e6����t|��I:d���qvx�%�8��g���V�?�D?�X�����G_%CW����� 	e�SNY�UD`4"0�����x�������J,���K����j����`����;�goz�������Hs�����������
}����p��d�v���1L�������[{���k*�\n����+H�����i�^�j��\�@��ev
���������mn	x���7�w�T���0���9�v�g�o+���q_����uG� �|&��'W���
0�i�K�#k|V2�,*���O�rC�D���S�-_�~�9zC���	#����3�*y�'D���q��L��3�WO�p,�Y;�����$��*}g�
�O�����8�#���2{�r+�\h�@���=��w)8���������S�6
B%�
!�TB�[h����%�z��i�h��P~32^c�;{�u������.�
�(�\?@�}+�u��7���o�����/������������'������/��A�s|t���V�,����|���z%��u�AQ��s��������("@1��:�X�=���
���a�%��A�X���62[����	�{ck�A��eKZ�p/�Ke�_�#auKSHCw$�=���9�M������k1du6P�q�0��ls;������^�E���:#$��u��A=6��	$/!xk�+W$��������:)p�:Fw}����br
F9}����������No��N<�9jI|j!t�����'�����i=/_�=;��^�&����g���_D�.��^�`R:���/~T�t�Vt����$tz�hxrBP�n����������g�O2m�'q�I�5�o~x���#�cxe��T����l��P�}����f��k1�P��^�,�H����
���-��/�H��	��@hl[��Bq(�����b.���<(Z�1A���.%����
���4�iL�4B��b:"��:����/�N_��:9;�����.!^�^�%�b0 ����]�wB�����DB&y���N����%��@TR::}���/Q<${��sB.\��I������_������v��U�H�o�5v��f}�!L�}�X�F��}�����w��V��7"�w�k��
�V�|1��*���~�_��
B�o��#AC�UA������-���SY
bP�����cm�/�����^Lg�+%s����z��k�&��l���5�uB_7(�+u��]����^a��WPf�U��s�u�h�^�Z�G5W�����~k7����pTi'&�J�ju�5�aV\�?�]��������7;� byW��8JQ01��6?�Y�N��G
��R
��������M��{�&�7Z���21a�����f�P;��j�,^�%y�Wg/.0)�6B�������I����9Z��dP�z�d������W
1���-I���GE5{\5LD���
���z��{�S'����^>�Eb�n7a����^~@����D������P����+�r��n�%}�K
�?��[�B�_������?����%�%F�'I�#�22or�J��We����h�[N�i�\��a*{s?$d���ra�dz�V���|s�'}�!+���]?�N_y���e5U]����@($���QF��l�^*4 g�4<(4k��4����Q
l�4�
E���8���m�K�?t��JDx���C��R�nGCl�hh��c|Cn���~����o��M�����D	��.E�� D��f6k�����"�"��
��+���W������w���2(���GG,3�����t�I�(��$��ii��X�B�R`��K=E5���?2A%g����O�!!��A7�w�|b<���Z�A����%�G_ ����Hb��2��F��\*%���D��'�8����yI��x�5v��r��q�O�DN{"�$�]K��$Q�$�$�������A )������Vn�2n��Aix�Fh���IU���=:����#X����L���>[�#��s��B���b��6h��&���e�=f_L�|��U�Mv���nr��y22ew7����.�"�� �+�����������A�ZF�11r�6������M�|���pW��~xA�'��������&���n$���z�L�c+���
�)�������:����u`b��V�a��!�=��Vgl�0��z�8�GLl������|�dK������V�N�%�c:7�|n*M�F����3����G��.��q��9������ �&8�:�C��;�����>�i[U�iz��^�Oj��8��]4:�^x��*l��X��3�����sP0�sR���t�I���7���L �p,�����������%K�}�k0�0�����J�k��>K%,�ad���!>c	�>D���Am,��.T��
����~�T�J>C�5�	}���h�r:�����y}
+�*��~�YJ���R�#���U���q��t3���h�������6�R����Yt:��t����H���u���Z��RR#TTJj�x�R�d��X�����ao����Im��}�����n��c�.nE3U�!,��95V���\(�	��C���^����� 6������B�AE,���P��O����_Y_;�A�������p@���[-Ss�U��qZ�t�0��D�y�����,��m>������>��+L�
I����u��J���k��Z,��W�������z��-��*�}x^v�Iv�=�	�m]�N+2��/�����U:[e���#���h�L���f�@�����x��Wa�u��Yk��{��Ky-0���8,���C�M��rlU3[���Ml;%�AS����������'�I����9�f��(X��--~1�}J�����U��>�Pn4Z�]�p"=@-���i��R�~T�hE�(����-��e$~�����.-uS�:�P_z��v��MJfV�����;[0����������9���'K���E� ��Q-�E��g7�����w���
N���.��S���Py�W};y�����,���
���&{|�����G����J�QE��1%���J��=)��L�MaU5���?o����j��|�I� ��T-`<<���'��v��KM��J6@@w!�#�r*�(f�d\�V�T��cb���������|{q�������
�U�T���C��I������iT�J����i<��#tm���cg`Zs�x���hP)��S>>]����v�Wy�~�5ht��"\��66�ZbG
+E��v����k������w.��wC�����6�������T0iK�SK|SK`���W����<��Cv��@G:���U��0��aa�����:�8;?}������H�`�����(a?�c���5�A](����j���8�f�Ac��h�b�I�t�: -�kq�U|�
V��k{��`r	u��������EGP����nWJ��<3B��U"~�����������	�$��	7�K�-��;��8����`9��V�*��%Tf�7���Ms�VK�F��������KQ�mz�x�Q[��d#`H����LbS�����[�jV�2J��9z[�7t���T8M������9��N��
IX�6���7������@�l>���ea(kRcZ����W_W��U���O��y0Q���[P���i
��1���{���K���$q�[���F� f_�A-���EJ���3�s��Oc��B/�����YyM�#�)~��]��-B{����[��F1��?��_
l����QY�E��3yh����F�B���k�����(fp�����������zm���X+fS9��������6�h�����\�5nZ�$�j\�u���2���%[�J��U(Q�����[�V��T��-nk@�$eh���d�|��e���1$�b�,Q�"6 �
�����|_kr�!��`1���tnm�>!����[���M�=���m�X��}�0+�J'Z����
�����hD�Q�����K@
o��]������������/	�<?}��Y�CJ�E��g����������y��y��<����_��2��nt�����5�~5�Q=�Ef�YW�WT�u��^�~{v~�|M��o��v���b��!>}�3���S\g�l����*[N�U����''�(��������������d-�b���f��x��������*Zc/��d����.����b�	���7��u�g��w)�=�&�"J!�����n�L|�'>p0����6&���-�Zh���5�Zl4k]%���$h�	h��b���b���4������������=���&�Hj�n��8�ICa
�0YZ���BIZY�C�Z��/�aG[�� �o���->
���
9�| ���S�qHEr���Yc����#�]cE�����^��h���9vu�G|�G��<.�:h��inP}��)�WX1z���EY����z����r��1�{���#s���u�Ii��BR$���8���$�,�S�/��(�����[Y��*pw��p�t7�������%�L c�L����&�@El$'VD�/��&C�~���G���(�$�B��nI��)��HC?�h	��)����9�&z*�����S�$�l
�3�
{�dc[�M���o��F�����CD����:�����8�ft�H�2K|��Q��f���������6�t�-4����|�������7���<]v����aNF�L��(�P����E��d�l��{M�n�^����}-�V��R����)�1]����.o�7�
����9n@g�4�Ov����GL$�H���1z1\��UKz����mK��r����?wV.\
��e)�����"P'�1d���x���*)�B#�������)�)�:R�u�W�)/�}(R(��m!��M��@
�*(7���.�����2�:HA�������~�}�W���3���?|��!I�3���b������lA�K�
��G���2�@����=�����'�
�L���Dv�`�]�{\�mHk:y}��F��]�[}%�������?�k��8�e���R�<N��'tn����h�Az��O?�O��n�>������"
h�?����J��������7PU�J��V$�mK�"�� D
/o~c6+=58\��-�E�o~��;V����T�`������z����xF���a"�K8��{�����F-��j��o�+����S��o�M��������*
�DP+
m������_�����V�JC?%i(6�JC[ih�JC[i�E�M���V����D�4��d9��r9�,��G�Y"�����:K�6�\c2Yg��F�{8��v��;�����,�<��OU�mI+��\����&S�����y���x��I�&gR�Yzis��s^�\�E	�U�UgN����k����+z)J����*�V��U��5 /��"���*J��z��a�`�A7�"�}�}5E����nm1��zG/m�#���#�h�[M�VA���-Hv�8<&g���n�e IWXd�M �@1`e`��/��[[<����KC�@r�*�?��1�{Z�6z�d���Ab���0�v�[�l���T��'��*�H�2��X���`�\�"��aZ%:��d��bZ�YzY�����>p3�V���L�4��vY%w�lD}�sA�fd��f!Z���J��%S}���L�OC2U���G�e������	�������h���*���L�Q���2��d��e}��h��TIYM*URV�4���$M%e5�QIYMzT\V�����P����JU�=���G���"U��3��m*c���__��;,�],����|��
����5���}ez/�%MIm�+P��_��\6Y/�3���?',��r^�O��
QN����`��
�����+���R�����*���f>�bHW���?��~�����6�]q���q�i��V������*����y|.	��!��{O�L�d���
���,b���K�3ZM��	'�l1�\��<�j��^�e��V��r�rc�r�c����buT~rt����b
��J`j!2��"�XAd2>��BA0"�GT���������&X�T(�����;B/��[CdU��l�D�_�U�����7D"K�6H��N�r'*"K�Q%���BF�65D�h��TM]���7�%
"K�"2�r��* �}���J�,`�eK�:���,�#�2Y�5DVe���J�
Y���+��Lx�&�������AHZ[��O�~�x}4����0��V.����Kh}IY}CZ�0__��4�(d�B6
��D"
�lZ4����8�5�F�����*[E����h����	���gY��q��V��b����h������%��W_����oy����M����E�^DPG����l��.CWw7�tF�<4PY,Y�6"3j|_�F������z�I"�/��<b��L��-���������>�zl_��"-Yg����XG��R�|u�J1WG��+����t`qN�m��i�Y��N6�q��Me�-�`��TJ|���OK��ulU�������iYt}4�Z���F��S;>
eZ�������
Dm]y�$����2��+��z�0vb@Gs[����V��A1W	��F�!XKW�������"���-���Ma{eXS�fS�1`��������Y���F��7U^C����O�]��]E�W�H�[��P�
�Ie��OE��@I��kB|�'
�A>����l�	�9<�wT�j�_#cu
�]��
�hQ��zUG}��
8��M�N���T���\E*FpJ��[�P@"�u���H���@Dt����[��������w.��i������M�+��X	���2��`��V��2z�X��z�����������#+]V#��w��U!��_v��!B�]=����J���+��w,�����YJ�5y����!��Lm�K����!�l%j�����
<�J4��
��v�2?������� )}��5xt�X�Z��4�����|��:t�jn\�A�&)k�^�$2B���#�I��1��I
C�v�2?��C�"�a>|7	?AF�b�4����cZ��:t�kn\|)C@����`hl����.=LJ65��4#1����/m�v��T�D�Kdz����������<A�e��	S�Q�x�?�
Q����q�u��{e�Z��J��U�orO���<�G��4>�:1�-���t��rG����z�������	$������K;]��TA,U���������1G|���1\�?�c�G����j�v�o��� ������W;f�zp�zt��c�r��s\w�x�(^�I��)�G�9�H�	�1���K��cS��$�u���c��B�k���k?Z��f��,-�.I����b��6���:��/�?���E����dtCv'|]��WW_k/����y�����������+�22~���?:���\������\�(��h������U.V���r�Z.q���(��������{�������b����D��������75 0~�~����'���(�*�s���JHGs,��,��hemB�U����<~
�D��������#�	m,0�4��U;%��<n������5�Czi����+��S��#)�Z�AX��]���z��kRs�b_���T�l��8zl�\�J�8�0�Q���eQ������;X�7���������qz��.*�������
e��cz[���,_�9����>(g��b�Zu}�3]WV�'A�$�1�jgA X�+hC�:�\�
�	<�s��g|T�t�����JgDs���3�;|�Q7}����J6d9#,Y����P�����A9�U���S�GI'�u%��sF8�*C�xF��%��6T������0��3��K;#�r	��;�9����mF�U��l�U��m3S�{H��{�OC{�s1����^���H�(�������������Ug�\�NB��R�D$���u�g�l��B��ga����u��i]/�[��-����
��`�9����t��_6bF���/�u���"�3]L]���� �T�P��Q��I�9�EX�������� ����5z��2�]�ty�����}�������EX���QD�]�	�.[i�c]*�y����(�������",��uP���Z[�b���Zy�F���{�<QDC������aX���"rV�GV9��`�Z���4_[5�}u�����8fUk]�Z���Z;���*v)P�������\���~=���um���9�������um��+`{��X���� ���Vl�0,��`�������E�CnVh���u�-��`]��[z�[�2�#k�k[���!�v�n�\G�0+�<��sa{��{v�A�Z�D��"�!8-4�l��v��o[����-���k
�������um�PT<#����J+&����#F���6i]��D��0�����g@n��Vn�*���j��'��D�U��
�=��
K������Ai�*��.��hL8���&0�K�FIr�-�����5n�
����CT������1�"�]�2��T���f��/R�U^���
b7*�@�D%�PM��B
�)���B
�5xh��T�O�oJ��@�h�������U�k����/�C_m�X��(T�v��������t?-�Qw��?���%�/��������{��l��h���� �a�������������y���|��4���>I����.�U�ln����V����7�/��y�����K��-�p;o{�mW�	����K=@?�fx�@_�������-[����P��J\U��v�
��T��;(r7}�Y��������5�������>���!�|L��?W��}��|D��uh�5={�������� �Y�51������G�y���W���SD�`#�So[�3Q
ATr^����1�Z���R��ZC+W������m�L>7/��Q��B�������YJ�t-��l��'�
������6#���������S�j���.{uzqv~�jg���'��Uq���S�)�GB~��t�����B]x��V����e����h�b��}�Z��4����a4-K%�{���dq^-wE��e@����u^�������^�d5o��q:�z�"���0�a�U�-��X����J��v���Zi�}����h����UK���Qo���_?�&���u����.n,]7L�y+}~�3��Bn��[&g��UZ��`�����+_�>_'���t����-$����N����f���� ��F�q�����S)�'��M�_��UQoydS���_r<���`t�`'L�R,�B��F��D8X	Ye\�,f����\��w��QN��6(��L��o�l��Y��Q���'�b6��49���h[`���v<�=��9�q���o��t��x�r���k9����P��~$�7p�>_�7��i������8�v�{`�{��L��y��n�N�����s���f>g�>�xB��]�\O����pm���f�������4�h���gd_E���:��J�Q'���u��-�&��t9�I���/��//^��@������g��:`M�6�_[�ZS~��o�����/��~={�Dz]�	@����.������FQ�J��K��5��U�����CZ���o��/�����5CU�.��^�`.;���/~�]��a��l�-�5��r��`.�'']�h���O_��������)�<�-����|]�LW�("L�Y
L�����2Z`�g}�EKt�YD����]F
M2�����e�d���n�L��'tp0�b��p���	������v8���al�Ck��p���Z�7���0��al�al�����G����t05�U�������j�Fc��!�������������E�m�)C���64�������tV�"6�D!a
tB4X������0$�1V�$�Q D��p$L$5 s%���/Hr[)6,�����S�,w����52z�N/E��7��P�
�������� �;W�m�[9�$$a$�=&h�
Mq�g��������*c"��H#cA��c^���
c�:MC�����+e1��X�2[1�i�G5�cb��P��;�����*�!(�'�H��12F!�����ziPf��Nr���0�#���e�����w���;N���%:[	& ��!�s44��1�!7�MB�I���7C���<-���f�v�>����R:66�-	��6wa�_���]�����-��,�.��N0��6���W
vG����q]6�H��0c���+�s���d��
�7��y<�_lf�bUc����F_(�!���BI��t%7�����H�D��b��W*��
g���y����:���O�5P�����T����hp��Bn��W��+V{l�����k�;!7� 7�@n\r����3�{�
�%>���k����:���u�`�
MAn��W��+Q{l�`d�����D���	����r���[f�T8�7��[�)r�5PT���hVM��2;t6�(��x�j���E��c�/dm,���w!d���X��Y&��|��I}��U?L����P�bR�
'ma�[m4�����+T�����f��������cF�1�[_���P��z���3���g��YWPC��z��)j���v�{���Q��Nw��g�
�=f�W��B�Q�!�����Y�OsJu���R�����V���Q���{�Q��0y]:��qM��m0����s�z�`.����� ���9����j���\������\l��XZ�����C��qGj_���8��t�^No;�*xz��b���YQ$i�$�Y����PD�YQ���0dI��@��]����a�Q�{`���wf{�9��O�����������G^����5R��]�g����O=�6'n'j�.D��~�"�u���N$y;�&'�@���q�aSP��I����_a���J�jod����mJ�m���X2����%��yc&�2�^_/�k��&7$�B���������N������J���uWaM����&�)���PP���� z]W	^WX�*S6Ey:V��f�tP�Hi��6F�U�Eo�7^l���K[����U_}���OU9*���&sw
9�7��jm����� 94�~�V������Vx0o�8>�?�ak���5�jO��vU�P7���
�z[���/��M��a�Z-�P�;����i%b�������}8$�8�6U�aY4�-���S��]�4�
z8D�8h��!����Sp�8��D�>���@�!=��J�O��&���g�����vS�K�+��n[u��=���������~;�
�#{%GC�������A�p�����C�p\u��u����v<�g���A�@���~��}N���uC����Axl�������A��`K��v<��u��
)�{��9�>�7����8���Y���A��|t`���qt���D���P���=��I������N�u�;8��I��/�}y������p�F ��������]N����[����E���+����,��rE�l�nxNg�9������Q��g�nyp"U�R!�q/��z��4o��z���=w�i�4{�u"��S[n���C__]_MAE�@
e����|]j�^��P�Oeu���p-�Q4�����
�#�=����6�z����
��� �h�������ru���6������V�E���e��Q�a���;���x��Q5�������E������6�'z�����(�i�L���'��:F�m%��Q<t����`B��Q�6L/�pZDY=�+Y+sT
��*k��`������F��
��^�a�mZ���Y���6�;o��^�c��Hp������81_8�Q��k����DF��*���<�Q�+ZEa��"9�^��0��x�G���Da��B�|�cQ�hCE
����DF��*����2�
9�P�S��t�%Bw�&��6��m�n�v�}f��H�%�����y�����-H�e`G�����J�[IqX+���!H����E������b������WR���.������{��N���3��DpE�,�Y�r
��D�%�*Q�I!���D>%�5'���K�D��+n7����Fz4��Iv����,��D���P��B��"_��L�&`l@�X[��-<c�
&f��U�f���G�v�ros-74[��
�V�Vg����JX��B�(,�g��`��v��#6 :�
�I�*��z��,��#�v��(�,��Yj��R��F���xf�a���[M���x��b�1�xL�v1M����4%qr�<���`�@Qie!iLS'L���4�G�4Kh���G"�j����e����Z��v��;���q[C��Y��/P����[9�����S��2�)�4L�U(�h�S.���f��C���q�f�e�$�^�!�����[9tCrh�(�G���j�����������2HV������w�^�Ab�8����Ir+�6����4V�����9#e!��g��#��U6#��7�����Cs��4N���a�����C?.F������WA"� AP��"�\Ia]:WRX���n%�uZIa]�V\����� ��*��D�[�m��z����V$��|�S�I�Oe����}���Wt����w���bM[d����5�F�}ez�uQ�K���n��O�-qNN����0%��I�g��)��,\��p��/������\o��\X]��<������������f6�amC���T�1v�S�4J4J��R)��D?�q�T��Sy�B��(��h��&�@E�%�b�:"�A����s����[��f��~�R�" ����zisF�)�� qK��)�+�#�W��������*��Xn�Xn�X.w���XU���S�'UP`l����}�J�P��\���XA���c�����W�Q�>�
(�����
�~�R���"
��Lo
V����UGX�?����x���
������uT�:*�Pp�Q&
L,(��.�
���P�((01P`����6
�����
,Q������_�T�P���;B/&[C�U6�l�DSV���(����aR[�J�����x��,���GC}��s�\��q�Kx}G�����~�,��6�o6
��a���r6
����t6-���N�u�[�u����J�v��wdm|uq*q#�
�e��oI�@Bj������R��p�K�]���T\5�����+���K�P��0��u�Zg�������{6�^��$�9����M�����;D�
q����r�!-�C.(��J�H�^�A����F�����h�
�*��c���	�2���
�v�^�b�A�@�{��@����zE��9�B��)��r&(5.�3�w�Z	��5��i��k=dX����F��H�����P�R{��P���05�������u}\�m�m����8�A�����f{&Y/�gInKpS��<�d�g$�#��H.Hp9�����h�"���H�Dp�<��4������L�N�v���P�e���
���D���_)B���j�,K���uO���k�G���-����[E���7��H5!�xy���h�C���<[��5lkUb������
����q#��m��V��-�Q�[��e����m��J�~�xy��a{���~����Q�*���?����$��mY�-��A+�i�rkK	������u���#sLN�I��V���V���l��B����7Y��f��y5]Ew���le�w��\���]"4u��X�>�������
&��s�������#���y�1��B�Jg���w���"3�0���bS���y@�
%����]���],�/On�4�%Z�-���a
�3�A2vb��}��f:t�-+��{���g]�=XW��)�W�U.�Ia}���0����r�#���Y�R����*DjQo
uDA��>�p�E�*��g�z��+'���u9OV��Y���
kH��iX��E-�x�H������Ytq���3���y�I��|�%��&��t9�I���/��//^��N������:�����4��o^��_�9��r������<��p?o�o�-<u�������=Z���o��/F�/��v���b���?����u
+����2]g�U��f�N�
NN���������������d",�b�c��������z���>��}�c}�����N�
���������uEb��8��������	��������k�l��Y��}�bs�b�r���Jj/Wb_��x�^�����Qw�X�c
����c�O>�R'�R'��N�R���$<�����6���lH�G�Y��9k����&����_]k]C^�!��X�A-$�X�����R,K9�K������D:�����>P��:gQzw7�f����D�h����Ut9����
�]C��d��-���P������Z���j'�Q[K�v�^(15��b���XH�bY�7��*��a]�9q�����j0�3ZX4V�� E��n�e����������7�Y%��E�
P�M�Y������~�>��C�U�JA��h�m%UnU�4���j�uN�a@��/������j:������YE..�~�n�o���������0\��Dm����048
����o"����t�0].�i
���M:�^}��K���!F�B�:��Q�,�k�>�i���=����3d�9u����0�qz�u�%��B�0)����f��v,��;��c��Ca�Y�/����l�����Md�+t7���e��-�v�|gN��l����!�AO�]j���b_�lw�'�c<��%i����t����%�����/��x���EsI����>�������y��UL�
������;'��LpiM��&ZJ���?���G7��_Q��Ja��Z��GM��"�2��Q�Hbl�H��@R���^������5�X4i����4��o�`�����z�?��e3�l*a��yk�Y���U�Gj�[I����xhU�aM6�KMP�P�Y��L:�	G��3�A�e�A����Q���7�����bZ�D�bZ�d�n�9%IAC��uL��a�qX����6��%��X�\f9��!�c[����^CBk ����o��.n�+J.���d���X�g�3�QS�~0�ZD������O(��^��e��c�6l�O�Fy�I����L ��_lGq8�����ZSE�;�G\y�N����y�v�a��r�aE��y������5:&�V#0�n��G���j�{`���#���<b;��-�#�#���"���<�D�j�s2������iP?f�GB�X�tVb��C�!�ewX�"{o-8�-h�Gm�2�r������f��gu����g:����^
)DaCE���
%lD��JC	k�4��!�+����5�� (5/�j���<����x��������<��o7��]-sJ��b&��������ze�)�lW�v�,%��Y��^�
��A;���a����=��������]�o���#Blk����#�^k��	~�|�5��|���F����=[�H��t�i����,���Z���hEv�Y�Nb�Zt��:h����Dv�TBn��!�S�o��eAto�a+a���2z�Xe(K�$i+=�k=���
R�^��A�=x*0
��[���P�w�(��������P�+�����'��@!��e��		��D������
���^��1��A4Z�T�����9?-goJ'�c��W��W8�}f,�}_����c6�Wc��M7�r�@^C�����X^��������^�^�����������Y���^�������zy�k������Q*��i��1-3�]�tm�6�d����.�3����X���\�o��D9RFr��y��@���� H.��*$�e����-�m\S��/�h�^Z�����U Ceh�~\�!##P���I���t9�{�zA�#���`��5;@/#�T��0�]?����C��gL?.f����c���@\���Y�����`�C��!w��:�A�A�	A��9����~D�4��% �^����������zi%)�(�!r�H"�5��1��0�(p�6�����8��t�GC�
KLn��8������qUA��:���vy*,a�����!����,!] ��;I<[~�-��&��p@��%��M����	���������5�?La�Y�F��:y��J�j�N�4�L��@�R���6rM���Z�VZB�F��L���<(��n�M�����q�j�2���?���Z���Fo��M(����`1�
��?D�'�.��A�s�=f".������RI��k�}.5y����5�k���&�j`�LD�zf�V������d&h�2+J%�kb ������������5�b���Y�Z��?	@8������4d�����*�4�3=�L�]W{-~�U��fN
N��#
���#
�UB�� ��#aV[�������=K�(n4X��F��M�-8���-)����$�b��K�!�������W;{O�<�dO8�O��{���@��HX���e�j�P����k4���uQ~B��)u�,�1h�v���uM��W��/u�#��$N�H��b����Ra�C1j;��N�^tBI���]jd�5��rs�FZ�W��;z�j���i�p�q������|����!���+��=���a�����rX�ZAj������\k]�
7�7f��S/��j��^�+b$X};+�]cn��,�R���[L%����J����8�i������=?t�O^����~��k�}I%�A9X{�$��.Q��'b�6����#��:E���V;�H���F|cn��,�R�x�[�%����X�_D�{L�#SN^��a��c
�V�d*�cW���U&��DlSX��)L�mL�P�����j�����r��0�!`��r�d�3J���~��b�Ar��D9.1\u�t�t G�M!��H�Mi��I%h9��mq�*���2+|�|���6�bO�0�K�%�PW
�[��s���'?�A2!��
�F�r�'����+NH�9vnBlBlB
M��xB��t��r��V����SC��'����c�7�G8$]cS�X�K�CQ#~��](o,*��=�����C�c��gPUB�PA;TSdQ|c���?�U�dQ7�C�OeT�7�v��c�:���,J��v�V���rc�&�eA��{��c�o������o��_�+��e���&<!�d�nV��b�V����<��g��������z������.�|D�te5p��_�[_sUIg���/2�$,���U�����*	��i�����
51h X`��A��AS:�J����&�&M�+��%�}5��p4�m�vjb�n�=��wd��T��[�k��c	������
�h=R��k�mM���iN][m��h����9�e[���M+t���N��������o������{�8���.�O�ev7�NR�p_���x~���m7�/��W��2�.�wf��� Y>%.@zB�W/�^tf����C������D�X[>�*mDI�
�������Dn	���o���63�:���:������n3����CV���~���R��:�T�d��?��]�
{���b��F�U>u�g��H�J�$�x������-�_���{x�>�>�'��=�����%�z�1�5��A��oy�j����|V)Z8U��X!f8�DWg�N�9��uz=���\9���J������������oVC�cL�&��D�a���C�%��x{L��C�@�pPR���)h�'�V��Jr��^��0�x'	��q��I��b�as�6�"��"m%�p��k���5ej���� ��0\-�}��������<�_��,�����
�7�	�=8�w����J�d��/�tW��e�/�9��e�P+����3���({Y��Z-3��p����QR�����H�L�����Z35,^����T��el��X��������K�8y�������VT������`���M�e��W��+Q{\�*��T��eb��D���\��=��w�.=L!���a�Gu��3D�����e������'C���y0���9���(�}"s"�ia%[�l�!E�-y����m�+��j0��1��!��x�`��>9}����[e�Ny��H�\�>y
��I�Uq"`R�����R�1���V(Q��";I:om_�����U%�gq?������?aq���P����i;����?davgPA@�p��a~��@kZ9�2:B�,���V�W�}_����V���-�79C��5���C�����[�<�C=l��s�8|
,�4	#XT^�YN%�h�����@���OLB�N'|�CJ�P�BJ�B����J�����W9����<\"6	#�>2������������oe�7��?����6��}��������L����^�����=R�a��!�JR<��y}�"x}l[��m��0��wQb�BD��*q�%<��U�v�Od���Lk���p��t;��^/�#�!�^�Pv��<�z�y����|�6��L5uz�=�NI��s'��!���$������G�8�9�<7z����x_�6J�X�j����~t!�#$&�KdF�K��l��VY��.{D�Y���>�-I9z}�����-R�������Z�r&���,R�^��"���>\�������:�pWn�7�?�%n�V�Am��K��`�
�v���&��G�BYh~.������0[1�Da��B�V����M��<�"8u;�s����uk��5�IB�C#���P*ps8��=���k���'5E��I��B<dY#�i����7�0_
B@�o������RB�BX������vi�\���A��;���!8�~����U5�?�6omj#�}�.���	Uq�z�g.���a�`8����v��z������**V�K��!,cq|�����7�o�3mc���1�7hw�P�J�i�%��k��VQ����X�
B��Gt�X<A��P
>��1��;n���q�
I�����D�P"�VA�#�$1@(Q@(�B��,c�����L��Xc��
Z�7���Q*��912����GI��`���
�t�[0�i\��<Yk4_����d����w�l��.��h:�V����l������o��2�
B��d�,�^D�EuD�
��&����s\mnw��H�]�Tx��-�����V��d���=0����?�����E&����a�B|+�9n�������bswz}����u�q��|�����?XH������3�������������s�-��|p�IzY�Z
t@�^j\B�u��^V~\��u~\��+g(+��}�$.��������s��~�Ru��������7�P�+���*��~]&����q��#@V�^�r����8`�eK�e
v�^�2s����,B��r	]���%t�E=0I� �P�@����J`0������;B�R�� o����d�
{�/2Xe�
8V��`b{=�U��Z.;��,�r���.����Uk�Gv��f�X�h��I�j�1���*��I��[YpW�9��U��!oGCG�9�=��Q8��G��;
�8�x|{a����(x`S	lL�iQ�	d-*hQ��QA5�{@v�nk��VG!q��$��H+�+�*K��U�����;��3�1�b�r<��&���?�L��3���+��=�vO�����0�E����y,�lo�R�n$K�_S�W���"���$�1oFX��L��?�`E�ac�����W�~v��&<`?�e!�i}]`�/���B���w��$_ '���}��_9�E��4���@���.���#]R�"]X�**�8��Rj�H���H���
�1���B�=��85*�)������1PXE"���
�G������$�����<���c���GyE���fE������<45H�[n��et�.��5Z"_-���I���:Zf�7j`|��s�~sA�Oa��v#��=f����<��A�H�}@�����('��M�z���SkG��X��X�]W,�a6_Wy�KU|��������K���`4�����o!�g�I�R�H�H=�PY�W�C$B��o��N	��y��`Gva;�?1�s���_}�&����y�EEAAo|eK��z@/9w�9w=���t'�s��{�Iv���8�o%�,���#���HsA������J��OUm@�ZK���H����}k�f��������X�&$.[�/5T�B�'D���mN��;�7�p��
�-�fRj��4���V"�h��
z��J�� ���e����xc��q6����,dYM��4$�����&+�UY�Hj�}4��/������t���*3y��$��?�7�H������I�S�����&���vS��=yt��������n���jBk��Igj�-�ex�E0iI��X�I�}M���mR����JK�c�U��||��@)>���GK�/Q�V>��}UK�}����	r&�-) ��]_�$�U@cgN*�������8��T��A�I\�9a��E|U&zJ�H0�����"s����sRb[��:������$0�z��(�sh�u���������U�����P�i_���6	XB�q�P���0�xr����� ��������q�I
�����������y����-S�j�+%�f��U�_i�+,LP����V������I����*(�z9w�0������e���=?;r�U���^dYq�_������0�a�[`n������as;��;��9���Y�L�� Z���)�����of�gmc�/5V8*=m���_g�uv�Z,�p�M���Nz�����}dP��`%	�+���S/%A�G����k�D���8T�6�������d�cv���3�������������i[�_/YHI�fR�/���U	{F�)a`���o�!�4�?���@4J��!���Vbk���F�4��#y�����G���p�kK:�N��=�-�v�g�w
����N������u7�,6��pq/�S����L���)������Oag�G�xT2���
@�����B�%/it16�PoP=�9,���f,(+��_�6A������w�7��/�Z�)�)o�0�yi�s�4�CmP����%�������>}{XH��}�P�P��=-�Hs=�ti@OM��Cfh��������'�c7�����TH%�/q������=5��iOM�w�SS��/������y�����(�ig�lF��<c~9�T~<��t�5)vU�C���A�Cu�D�X.N)_`D�b��������|�����<��V<H'j�<��B�������7_U����J/8� i6
]�u���2a�����	7N0_U��z\�>S!B��0�1��$������V��H�G�,�����N���K&lxlI�m'���%�i���J�1���=#������������ ���D��bJ���ae���LCt�|���������7gP"��sc��E��:���Vi'5ZNw�Q���e]h}Z����.���$�Gh1�=�gjFn�*r��L���=Ms�0�mY�3��:�Th��G�O
���#�Mx�9<�1���1H5��i�"��x,���su5[��O&~�$�����y,Ri�{n1�'����w�]�w���D|w�{7����|�e%ct��lh)k�'+{��2k�(+O~�X�������`��s��:<�����/�#hq��,S�vfjf�L&,5r�*�o<,no���S�|��L�J��(�!��4WGK�(��\3����C������zO�����q�j�2��)�?(��B�*(�j'K�J}
(����P(�T�6<A��oP,������1��h�������1PL��HTPL�N������h��8@1)��PtY����7(�������w\4�Z�Z��n�mGA��#�[��vU"�������Xr�����Zk.���+w_�-�z������B�J//��(]E��n6�`��h���C��e�.Z��k�#��:���+t��Z:#_]e�l>�<����z:��M�~�Nuzqv~�jg���'�t���~���$�GB~��t��{w���+��RI�����J����j�����i�X1�����;��������{�:�U�
fk��:��N�^t��d�)^s��j�N������YY�RXu
.�����^��.�L�*����9qLIa=Z����4�i��!0�P�TKQ;�L�[S^�mz�����_}���
zB�b;�^W�TU5�e����9�n/}�
�)`�5X�z�'aJ��K�_%�S[����b��2�e������CP�
��N�^W�zU���6�W���o�����p�B�^����9�`�a[��J"|�5s�y�_Tl�^��3��3�(�,���O�,��aG���7[@�MLd��MHPx��^l���C,Xo;��������������`���#9XoB:3P�`�o;�0��N>6��&d9CV_�o��Y]���M��o����M���3rN��2�]���&�PY�w�\�����lucm���i�V������p/�(���4��'mV�$�k\�Gz��|J^��.�!e�e�uS�Zz������F���l�N��$b{l2�������n
Hm���y��2N��K�t��J�
K���������yz���~�+D��Y�S��:�2'�V>OA��zY��b�� �:�	��669�������l��s��E���v9�.���l��s���=���v�8�*����l��s���5���v�8�&oU����v'}�;)z�e����&�/����l��gY��q��V��b����h������
���������Z��v����z{��6RE6Yg��Nsw7��S��L;�	�N���3F�[��K�;[�T�[�
��c���a�*�]K�4��������;Q�7��k���)#h!���b����b�.g���j�~�<����o���G��:��.���T��!yP����7M�������2B����[�v�^*G�[��Y�sz�$��z	��QG������e��zC���Y�U^�\�m���&���:�|Y������`����%�6���V��)�nL����es�������2|������B��T�&�?]���+���]��@ ��@��H���
u!b{*A�o��0��.���$u��u�ub�R)M����OG}n�N:�n���e������>�4��v�;�
YY���0#(�O!@S6����Ux�W��7���}l�tR�����c�W> �eU���T`E�����ERe���7P�3����"X��R�%d�'Q�����q��(���y�6���F�M}u��z��f��,�L�TL��/���fM�����`�����g�g���?��W4e��h�h���L`��r�Z.v�K�r��\������vcw���n�n���J�'p�p�w��]��]����;}}����M�����/�$�=I�'F�8WI��$�UB:�{`�Mf�OF+{{ves��n���y�&Z\]�/��g`&&^����2���6fj�3����#��8���%��"��~�x+�Wb����H�}�j
���
Vm2U�zv:������;:W^HE�Iw#��Yz��
F�����0�17$�9Ms�����R7I���Z+P��?N��\6�Oz�����;�F��eA��������\���ZY��v�^Q�}D~`�n`��5��t'7�J���
�J������9%��%��X�����c���*��b%:����*�������7�|�Sg5tF�����P?���Q���2�[H�_s���9����h��t��F��+o��s�t�:^�d����������26_x��!6�x�x'�SO�:q<|z�:\���f�ci^����"w�X��
���x�Q��;!�#j�;u{M���cI�!1��X��o�������j�N��;�+��o���2���9w��x��W<w�@�^�6_����)!Nz��&�������,�p����S���!�fU�o��ls;���6�jM�o1�������3��L���_eJ�	;^��+6�s�NDa#����J��~ya�6./,����8��ay�#Q������[Z����r�����W�o���T�q�v��[�3�r[�NZ��YY9����#�n(U1����j����Zb�������=��b��zR�X�P����������A=w������om�QC���U���'a����V��3��1��#��R�/����<�[��"����b��U�R���L]'��v�%��^5�'���F[�*9 ��Vqa�����^M^N���^��O�x�pXr���t]�d�u����Pi
���\���U��O6��r,������c9����8�c��l���<�}&!�X�+���\���*\�X�����h�>����U����|e�-��^�<4sc�~,��c�b5M��6>6z��*�	=��s�
R�����r,UY������������\����T�e#w�X�r,'����rb;�s)R<�}&!�X�+���\���*\�X�n���h�>����U����|e�-��^�\�sc�~,��c�b5M��7>6zy����z,"��d��?���Te]��JsP��J�r���7!t�5
�=��0*V�j���u�l��Yz��Y��j���uMn�^[���YW�j���u�o��\�����k���5��^zx���f9��XYG���wT\n�����r����M�x�q����(-�i�N�u���:��3F�e�c)�K��Y������*a�~�`�]�o\�i�)��(M��lJ��jt�Ia��3��R����4,�O��&7���?��z+�l��h�-��~���|����(a.;�~r7�����/u�Z�(m����(���L*	`�������Q�i�X��>�3lr�\��8d���a��+��	�

j���4[��`Et�p��UBL������@LG[BL��TV��
�T���m��HGL��#��
���I�a�s�!��z���!�����w�;������fU��J����1{#���������!������$�$l��������
FLAx���:�&�XCL�����N8��}$�������H��L����k'�Fh'�2h'�h'�
h'��g'��g'��g'��g'��g'�����g�?/[�M����K���/W�����wY�������=�
�om\�6�������4������J�R�"|��^b�j��6+����\/l�k��+��:��>�����/��c���Fbt�c���k�+���X��5K��|�3��z��7,��J��Z�B��
4<0��	���v h��Q��
3'�V���d9�K+(���'�+��j8��c�'���q��s�D;4����i���<��6��jYR��
��F�k��"�-���v��)�j#����vm�6����`�{D�mP����j���U�.�Z��*��P��k��iW���	~�?�u����j����:��y[
�W�p������k�������������:��Pz�B��Z1�:���z[��W���������d�\E���j�_=��Wm����X��:m������+��f;636�����9���y44^h69�o��A��e1����Us�-�
���+,����5��U)��������.�eA����Us�,;�����P�I��J!�lp��@R���v�*���o��;]Y\+=��?��"V
���#�D�0Gm��O{)���YJ�o:���U����l����.]��y5��~�oV����t���'������^0�W�g���v��<yI0����-�O�t~$�5�����
����Zy�:c
'�	�B�Tk�b�Q�1x�Z��l %�����Iw�v�<=f�*��_���7Y!���+A������E�|�8[��&���%��Q�O-����jM���bqW���\�e�oY��P9[�O_���Q-�)�u��;��c��R��Dn�-�]J�����ij�~ULC���p~���'�J#�d�1:Mg������C
�R�jEG0T�P9,�Y�� ���/K!����B�C
�2!��&�yv��0�r��f��a+z7�Mym��Y
m���/��>�
�vX��k5(�A,��q��������:��o�Q+Gl���d���<[o��z���������,��f���7�n�Y�y����[o��z���������o��h}3]E�y\~������h��.�ed�N�h����.�����6[�Xmgm���n�����-OK�J�}2���I�!O�mS� CK�#C���^%2������)[.�w<���~�|�2���j1�C �EY5��a�d�W�4����U[S���YL�e	U�GTi����o]��U�����G[��|D�`�>�+���t�
�0�l�;S�������v�3���y�jg9s�������v�3�}����ig7s�������vf3�y�y��ig5s|����o��B�v3���^6������&]����_��d'��l>g���B�yv���������l%^A����M���I��"�7#vF�X�Wd/�-���G����C����!�egX�n%�i~���!���m|d3�%�kq�;Zd��_Lo3A�\��UF���0uk��1!����#U�'��5����:��e��k�:����#��r.���p���������B6?5yi��Q08��F�];��B(LX`$�CG���)�d]�����J"��~+4@@���� �sQf���w:?E�!�*�M�� (���'�����n�5(���+�\qpq��T�������q����E��(^��4����f���Ur�?���C�r�?����C�T^<��C�6�?�>�+�))`�6����u���G:hep��������m�����m-|����r��S�|��� ���O�7�|1��of3�fE�w�l����y�����.q#�P����e��2��*����PG����l��.=O�/��&���y�;u���f<2�(��E&��#V� ?��h���� ��y\-�;���x�[r�C8��b&�
���9eb|��_w����rP�_��Uy_���j����{�,
�g�?����:���Z8V�&���f�g��|/��JEv�� |���W��k�y�l��d��y�p�J��<E�Y������]J���h�l�~�������-���1T�R���V��7�9�Q��*�S�pe$�1{��������A��5F#��GP��au$�k�k�k���A��$�k�k�k���AR�������:1U�fy���E`'��c�h3���e���)�f�#6����((qX���o��1�����c�O�R8�DcSb�Sb�SRu������8�c��1U[�$�N�A^,��W�
��I
�T;!�c�RH��	��;�H���Q�����e����S�G�
�?�����)�K1Ll4��T����.�	.�������f�����)�/�Z.n�
	`B�Kj"?S
3hlC���F,��\��S>�-�s�A������1[I��U������!HGl�������������4L��$r�s�M3+���
J����x�eEi�SM!�Z-cb��1�h�%-��&�G�i��~�:/v��-|�����2:�BP��5��.\>����Erg���z������\s��F���;����8�����YF6���Zz>����� ������Yh'�l� ��em�F���g<��������
k�IVs,jQw�[�B�������~�W��!Pl�q��B����[n(I����b:1,��G7�� �����U*�%�i�`+�]�V�:M�����c���>/�1Z���C����r���O �*�&e�b��~k4&AJ�9����$+
��~���+U3�+7+P!�u3$�D�Va�E����<����,x@�2eUX�2���cL����n�����
�gy��L+���bs���}��,��������}��,�����gq�}���,T��}�gB��}��
e���3oL��f��]�,��������'�)Tb�r���</A���zYfp���27�X/���c���5�~��^��9��;'�n6n6-�<\��7����1�Je��-^K}>rzI#jpxyi��@HH/r\n��f�hY��U'V�kG�Vk]����PM��2�
heNl��.V����U`a�3�=`^�����t�u�>�1�~f|[����|�D��\�b�w���+``wjl����Scs�����c��\��+�����p����[�2�
���R�)�3N�Z���4Ru��������}H.��V�l+1u�UE��)�G�U���}��*���R�\���7lz��U�����|jXW�Y	;�o�
�~W�s���P4~@1�,U!U��}�r�?�)����H�t;U����v���)��o��2���<(I�{Y�WT�I��}�X�����-��m��[��&��_�'(�
���$�*I�:p���<���aOIO������c�[���:a�J��MZ-�l_6��:X���pE�WC�-?� �{�0����'�� �hf��\t�q�x��[�w�?^��v#�b�}Rt��m��`���Xs	%���Aie�[��� ����������|i���fF
(�:��M-��Q��Q)���g7�"�S�?�,�!.����JX��\A�9����hH�*�Y�������#����_0h��@1w�!���ge��L�)��&.��s����5���
���M��V�����I�����@����k��<-��	K�#�������XcJL�a����zi�w��(��Ga�>
��QC()a��0���5QR�Q�q=���6�4�K���~AYTC�^��&M����� l6�� |��P��Jrk���
�Lw��K���&Q����n��w���6<�IO>�`�%�� ��N\��G���iy���jT���	�t�6���=��~����K
���������o�����4)I��L�B�����x�����>��~���&4��l�����=���D�)]�����s�+��g�g�j��ev���
C�5��JD�f�*���PF*M�������X{6�.~Z�b����.��}��%|�/�;ye���������N��+����:(���n����h����:�O<�4qZ���P-�O��Z�0�O%Z��bOK<Euz'[X��o�u����;a�d���kz�t%^-]���-]������|HG�'N^< �RMC�.���}��R\QX��2]��*[GW�etK��t���f�lJ��,(��������������Wg���T'���5�$�6�����M���~9=�xy���#��W/�^��kk%p��o^��_�9��r������*{
���=a���{O-
FE-:���������U��:[]}��!���~:+�sV��v�jt���F����Y9du��1���}Vf����3�>=}��n�9AIq�jtL��Fe�������_��o�D0�;����6cg�l�#���Ay�"��l�-�u6Ze�i������. �)��������7�h���3�������^Cl/����kHh
��|_Ep��1�7��"9���������iL�}@H+q0bM�E�.�A�]��a�r�62h!>�&T�M`���U��a���U��X5vbUw��X5�cU��4�Uc'V�+b���U�5c���U��0V��X5��Uc'Vu�X�U�F�j�����UcV��X5�c���Uc;V��X5�`���U���UcV��Z�`h&-^f�,^��f`�W�I���'�i��F(����&y-0�r����y���Br>
Kj9��N��2��/�
�k����������{������!y�W�q�����8�!Y�W�8�RW�|���>a���BQ��'mq7��zP�	%�+�8�w���G������;���<����Yv�t�r#��8���(V'�	wF�����Iy���D�����d�]O�;]�l�i���l���(���u��Mj����tg���~��e>�9O`������>����bn�c���*����}�6�(+�i�&|i�c�T;o�TB�����vn9��Cg��W-F���q'[���V�����sur����5����a�d�ZU#���Y2��]��_#7��� ����HU(k�{���F��pQ���sE����8V�*�q�6�5�3��� }��Q��mA��}���a��Ogo5�U)�%��D8�D������
}I9���T����z~��/Zx4r�@�������*�
8�
4���7pC����E*A���~��/#z4�@��N�L�����*�
��o��&�
��7,�>{������__�r���=xI������a�Sk�H��~������S�,��j���O�����?R�x�c��K�FD����CV�������nl �ia����e��������������i-k<l-kZ���+���C�	�u����;�$Pa�~ ���N?F��Jb>;�{��`2�������O�N��LWI�#�|�[�� �V�����~�T���~������"n��1�*��!L�����U���������������2���W��U/=�V7�Aze<�Zjl��iH����H�Y�h��vv�Z[��h��#�X�����K�]��}I,]F1 ��2-�K���F�c�5���[����5��>��J�b���m�6Wln����b��rY�X�-�\�X����7�k!�i
�\���6����6�g:�&�c�5���6�Hx��V�(2���=7Wl�\z�l���,��\Z�lus�rh�K+R���E+m.�����yZ���6����7���&�c�5��&7Wl�il�16��}r����N��<�b���N.��ss��\��U7���U��4�l�fO.; �km����V,��T���J�� 1Dpr%����N��<�����N����J�O.g�J�� Qz�B(������O.; ��o.���M��2kJ�Mn��X���cl����J��+��\�yr%��+q�\����=��E�n.���`!�i
�\��\v@0~��\a'W��XfM�T�o���>i�}k�������X$�^�|�"����o�����E�%����QX(-��b�����-��p_��YJ�}:#`~�-3�c�����m������������'O"�'�-��l���g��:	+��b�,]�)l"8���Mo8�M%���d��3�L���H�=��@���n�Z�I=�ju@F��2���?)�i�����:���B�_�6W�c�tH;W��������EG�����nW�;,�s)��:�JL�_c����bqg���^
v���X%M�Oo�0_k6�	yAw�����:u��D���<��?7���>��>��\�9]>f��F���RC��Me���0���������9��:C/�u8������Ii� �C5<�5����=���'	}.���z�^��MUM�E��q��Iv(�*�����Y<�L�F�Ix��X�0��f�02&�*�0�
B��q.����D��-�2��6��
�x��)z���	M}��������4��t�b�xD��ylD�I�s����	{]�������t�~���p�
�.��
U
�G3��hF��H
Y��t�����G�1����	gKg�O�0�eY�te��[��r�����
��`Ve��
|U
�j`:�>5V��b�<6":%#��T^O����������?��v�
�t��W8wA��s�j���G����Y�9K
���t�\��,s2}|x�0&�M,�>�[�t�em
�:.o�S��m��6���A�+�Z5h�������+}/c�<6��%���t�������jx����c:o��`LW8y�s��
*�p�h�n`��3J�Z�	��Q����	��@�1�lb�������,k#��,b�:E��v0��o�1=H_4��U���N:j���a;f�c#R]B�KL'_O����������?�3�����t��W8wA�n��p�b�!������&l:#�e�����L��%I	�&����-`:��6��
C�l�S��m��6�B�A�+x�'4uNs�q���G4�H
�?aV��	�1�x�S�[0���,��m���I��=YL��[��b�(��:����������5���W��S#+����a1��d�[�6ZK-Gq��a�%~X��,�]�������Z��h�y��{F:�lZ�����<x���z��<"i
@C���5m
@���5
�05��5U����hk2�e�����5-�Vk��Z�m�����5-�Vk��Z�m�����5-�Vk��Z�m�����5-�Vk��Z�m�����5�h-�ZK�O��5m�N�vZ�s�h'z���U�~�������|3���Ye�-��j�����<Zm�+-�m�5~|����WK��n�U]/aAQ��*�	���M9�"���y��I�hh�Q:�_F���-"��o'���E�#�;��N�T����!�~7�&�Aw�=�u�?J����+�2��l6��ao����������:G�]3�]`:�{n�0i�G�z}fY*%�
�M��WGH�U�4?x3	G���v3`Z�`�sTX5�v�����#�H{��\nc�w����m7a���<��kP7+���6��D�M�26~���N��5gl��A��9����Hr����bYnX���1U�-����k����Y���T-��2&W���1�uS�Lz���������u���L�+�Y�������B�(}L�<RU���58r�GZ&���C�?�4Q&Z�"D��&3TI�����������7L<#��ik:M������cS:6xlZ��g�@�����9��t���Q�x�\�F'6��,�������\B1�]$�k5�F��F��XW(<�C)�xk�'��7*���$��K�l5;��/���f[�Q�t���R���NLuX3����E�&����r�����������Q
ZB��@�S��s;�i�cc�}��o�V�weN ����p���]��+3��������O��:�+3��L+��i�O�5{Wh ��C�
s(��
m�+�bh���YOBQ=���?��w�A��������x=aw�Q�,�55����`�?��2����Tu��p*��$��$
���c��;H��d`����T5|�ph��-#�#������_)��3�m�3��,�e��i�?������S�D��+J���&�h~AJ������\�����6o�������I����2�c����]����^(s�"|�f&r��k
21MCB�@4"�i"RQR�1�B���eC3��Nf2:�^����zeR���UkY���o��$�+�a��XY2�a���9�����|mu��LOn��[4��4�H�$��5-F�����*?����0����.����u�����jRfIP��T��o���#'6���z���#G{a?r�"�G��h�#���������GN�"���,�7�����a����R�����j$���b��w���Q����<�b�Q��:������u����:?k�B�"}�|������F��%
:�j���0�����<���]g5lU�n�����S3v]����m����Y�����������V{��Y�������b��s����`�d�A��V�	3����.g�����c��m���u����g]b�u���K\g�����|�:g������sf�����;�
`DY�Z���]W�?���.�]�;�:��A*�OX�_I�b��sK��%����=}gOw<O-��;��\�'/R����w�lN��){���������@��@<��O��U���u������Itfcj�3�*1�-#�s�*�'��Q�g� o�@e�e0��k1����yUf�PJ�
6�����2=���F/�#,;szF_9}�}�R�h��ml����ocs���}�c��
U���`����5��?�R�X������b�),S^o�o�r:{��m��r�}[��*�#���O]m+W���6�4�U������m@E���vk}��3��u���+�[���@��ocs���[U=��8��7��-;o���:��M�i,�V�mu����i���7z9�v������]e��.d�&�y�����<o�ykS����}��:.�2��o�h�Qc1����t�SX���r���Lc����f}�+�[���@��ocs���[UE��Y�
@��-;o����z���|M�j,�V�mu����i���7z9�/��m^Y}oEY��H�:����8	��sd��~ko�1���?��Kv/��Q&R�{�^����Y[+�G�j~����#+|?Om�C��Z=F�����%W��R���;�x��Rkn+�p=5k����9h[n������n?���A�L��k�����
�]O�ZA���l}\[����lJ����n�m8�~����,��6���CR}�z��q[i��@+h?��[���,T�������l��S���j�r���2o�SqVPn6�-��F�Yh2��\�W{Y�+�2��mY]c���1�4�t�cSw�kZw���"���$���j+��K�^W��Vl��.d�~;��;�#���/����/{) ��Ut��T��$�r���[�����[;uL���-����F��u)�����v��Nd�%qf%3>Fj�`��H�P/���W��0����<*�4t��������f%�]��������Z�K7�g��K��C�����u=
��Qx��\������c�d�|{���O���n�xT9g�{X�.������A�<M���������:_���=���zi�s��V��z����n�4��e��1������V������Q�\4�q9�x��^OF����y��N�������|Y�����}S/���5�����C�_�|�����;�}��cpFiWl���)�����+_��J�W��~Vr�����+�hf��dG���|if*��M���H`�]���;:��t���+���7����E+�yv#�pwK�R�(��[x�yw������B�%>�R���#��y�1��F�Jg+rzt�G���6�)��Iy���s��OF��7��?���PN�6����[�"����bm���?l�S\��;�!�Y�����b��9,J��X��Y\
���-M��#�� �bv�����.� }���4Ax�.R������,�����L�4Ux�.&�m�����wq�9�i�pGK�S\e���zR+��T�4�x$4�iL�������U.�?�����P�D�����{�������j��PT�����I�@U�������O��T���y8k����-�\-��`K�G-�����_"�!���8
��hp��-	~���C��s��%�"�������{	�����g��t�A��2KB��e�&%	����w�:����Q��~4�������]�\OA��?D��jMJ�b�����������Wg�8�~9=�xy�������o��������-*������{�2����|����v���Kg2)��~��D����Iw�������Aq����>�q���&��s�?L|l��8p�c���jqL|�M�u�cm��S+S�$Iq�����}
����uv��et�X���.�F�=����t<�M���b�"�~E�Xm���Rxq�_e�l�&�j��%�.�x(��u������,������G��?��?� 8~gO�%�����[E��:��R$��[�u%-+Zte������b,!���04��0h:�����E �)�WyY��"#49$�^��������,�K� �fU~a|A@��^�!U���C�#�~���o��o.������
i�iqeH� -�7H�9�f�qX�+�����5:�J]����['Q�?z����o��!�7}JSn���3����G��.�NQ��4��Q�?���9��I�"p����=��p����}���P����:C���8�w��`�=�?����I���S��������Fh�O���2z��G����9(�C}�����B��@���Y5f%��7��g�N��}���[O���������?�����*�e'�F�n4��RjR����}��G���I��4�+~�c�W����E�M���)�N2����s�iW���5^�u�l�����)��&��������r�PEe�����>aN�����o����wIT�Z�9O�3�p.���S�Zv*��T�hXX��2.��v�1��C^Y/��f?i�y���L7�J�M�4�����U����5���t��(!�s����K���n�<��y���Zw������j�D����Bp���rTGF�U�H<��j!��:�QZ���D����B}�n���>
�O�iXm �q����O���74��*"��^��'u(�%%���~��Q����
#��GvxZ,/�%(�P���>tl]"��|�Bu��m�p�c	�B�eap{�R���!|���9�V.Pm�vYB��JEm�[���em��A�}Vi��:�������z5}k)��x���q_��o�!�:����}���z3����%����'�.��H���&�O��z�k��gxO=��
�S���ts��d����v-�q-� T7
F7+�Z��������v�������F�SY+]=��%�O8qNM��#}��G�}D��b��:����:{��L�!H��nfb��[ ����&"�4[x��BI�z��K/����h!��.�=��k�j4h�S/���E�@dA$��0�^9��q���&�+_�ww�L������\-�K:'�:e����Kd+uN�r�������������5�y��0�m3C������xUqV-t�U��
*u���V�����cc� ����N\$.?uPR��<PTR��'l�����>(���� ��AwE�V���)u5$�E�WC[$\5�E�SC�Z$�E���`����7j��a,�<]o�)���`r��Z�qi���`�4JW��?��-�KE�O/~=?}�N�A�'+�W�h��b�g,�}2C��u+=�$�����5Q�ZRoD+N�}����8^C�#e�a����
|���'7$�=����!Y��L#�������:��Z�4������E�*v(�?��9J������J�e��J�Q����W�d~��lZf������g���OQ����vV�I5(�k��j���<�M���������XT�������!$�
��\C����p������d���$�s����e�"�+Sx��������On.���� K�9��;E��P�
6�u�����X�)VA�9�b�\T�S�C��S��8�N1����S�%mf��Cxqoj����S�i��1V�)��z4�`�Xw�Q��NI�����)��P�N���NI�vJr?;�\�fvJq�$����)�{�$����%�k�~��J����80zO��.i��T	S��X�M&�n>�r��n�"\����"��r���D��E����v2�N^$����xhi(V�[L��o�!�������~�"���9��h������X-VK�����6�!��:Q
m#��U[��6��a�mm�M���=��=6��&��S�U:�m@M����y��pH)��)k�nf���,�.3��l��,�`^�ba7��,6�u��]��eWNB���b�y�a��j���u�����b�[ni[�T5��!�=���G�T�2�5�������m$9�W�W y'1eQ2	��<ZoV#�3Nl�G���g�>|@��C���%�$����+��	����L������{�\���v9F���v	L�8k1%����:0�Xiv� �(_3����N�O{�W(����W�E���������EX����'�W��|�
c�B��)�{����1xCBc�{�.a�R�GW�#�}%
f�<
����B<�0����0��+�������W��C\��Y-G'1+ya7�\���� �]�=g34*������E7��!)+�S�R������uH�����x�E^�243zxf�[P@����+c�U%����F�(�����BU2aEt���p�R�N�g����:�����
#6<������lM1����8<<�(�a��T,��y���6�<c�$�O��.8�8lc��KG
�L��^v�V��.���cf@�+)���oy@c�kw��J;�I# S���V�B�e�|J2���@��:���XSR����c�@�3����[�o�����Hs/��7������3���+r�t>k�u��������T
P�/,L������pe5��������h�
V�V�?��Q��2�����0&�����,��gp5�n)��D�fA#��u
�n[%�d-s�KeM�K�(4���+:�/N"M�K9��^l�^wT�9Lt'�����3�j��q��#����Q�(Z)�g�$?����U��U�x�����L������d�~d4�B�*�5EY���aES�>��������d�W��t���j9��]��j�nd��Y����4}�}��>����X��J���&���E)�����}��'�AU��
V1��&Lq�n�@����J'l-�.�`K�uq�<���~}��]V�(������	.�?�8LBQ)���!��&���b]�#�2K�OT��4��Y8����
ZU+�#-~[�3xRu	��}������/���U%�65,���KYd�Y$�C���Q..]��v�q$�y��u�J}a*-������R�l����-)d�0�n���3�zc��e���)��lrbP "B|��L���Y(���6�2�$��iUF$��	�)�������-�"������ ����*�Z��bp}��a5�|hU�Ku%m�����D�*q��.a@N��2�6Z��
-R*o��nd�0�!
�BA'��`��������(
�c
�&��@13�}������|1UV���%P��_��^���<Ut3��A�M�-�^aQ���>�/�E9�����-�c�R~�O�-�6d���	�dPoTqK�Q�&�\f/��9U�Rh�F��Z��K5Z�������M`7��.0��-�yF\U��o���b��0:h�g��Qdi>��}�����a���K$p�=M��yS�z�� �zZ���Pdi&��H�R��I%�G�{�Tq~!pfTV�
��0�R�H�)1�����,���A<��2�7f��$�,6���.|�zT��%K�����m����o_�/Y=ecio%���`����A�@&��W8�^���O�X�
�Ea�u�V%���?��%���,������_��%`���R`�=�S��*c���a�Z��c�Wt&_�l6;�&����Z���9d=���u���H������g��	��yc��f��M��Zlf[B�/af�}9?\.6�\8-��e}���R�����
������9�5Elt3;P��2���N�����������������f�S��E��g�����s������i~����<��mbjz�4?#\�W��������3<u����0s�akSi�X�/Q������������pLJ���4PC8���cS�,zP��}|,�gv�������~�h�69
GS�3���4��
�����|���#���������p�H����O���%1��!�������:����+��������_3�6�Rv��qQ&i���Y�<�a�X�]���([5�*e&���c�{����5���(+���i��)���T�v����~=B��.z����/���r��K�����W/U�8
��5�@����
���N&�|��8�(FU��f�b�@���/��3\�^��C�p~AG�j��.�\B�E��i�q��V���N��}����4�_�W��t(&j4��I����f0�\�)�u��9�=\]r�K�I�������#�������u��J?�P�/uW
?�M��E����<bq$i�qv���cQ	��q�y�2��)1eVq���������VY���
���:��@��*Ibk�xd�t������s������*�\�X����!�3	_�y�����?�g��3�����s�zGU�4E��aP�r���T�(<*v�GP�g ����M.#H��,x��M�U��W��N?S4�����ik�l�7���W]�Dd���6�����m�4�$���F:����sg�fopU�W�y���
�R>m}��yHf0{e�,��g��B�zz���p��"���d�@����S�{��%im����\�E
�����k�������c�����*6��:�C�m=w��W=�<rf�N�����j	�6<�"�_���
H���&{�%�����dU�RBL���@�b�l|\�ej�Ya��l���%+)�
�i��#O�����
������7�X|e�5����?�u����Og>	�o'FB^#%�� {��wR�'Yn���AW�$Y���K=
$i��,����;��\�	G��.�*|j����yDz���N,ubi�N,ub�t9�4('��C�(�t�'�&���z6k��TD�!�������k;g�G��M��iS��9�XU��jP\��$�'�|YJ��^]������l�n0��*����/[	�8?��R����QE�j���
C��2)�`�����K���t+b����X�T������WN�L3���e���._��E��qw$O�/=�K����@z�/�#y�����@lG��6Wk�%R>O�mI���]-�_*�@ �)P	E�I����r"Q��x�9���$a��c�������u~��f'm��R;��^P�������M�IcP���Yr�*T�Y����T�&���e�����W���y��`�hveZk_uVG?��:��H��}�7������q�2RR�z��f�]z4���g^E�z��o�|S������gg�m2�������on!/F_|�5�p,�<c�m�7{8+I=(16�~m:6&~����M
��5�AW#lz�^{yi������;I-��b�$��L'�Hs����Ml��\41'�lZ4�'�4C4�k/1�p��m<��\||.{�>*�=9���0�������v��u�����*(�9�����Q[7�D&X�	L�6X�/�(H��;z��!nvD���$����,&fv�^V�*�g(�5L����(l���`�V��V�z�������d��x�x��:��*�B_[����(����^���.�6q��[��q������j��I����r������A�uB.���K���0T����"OxA�]��$i�DCt
�kM��P�Q��Z��,�a�Q6�C%�D���O��#���4V�������G���m�)%:����rC�)/���������,�c�k�))H`�8-F�	�)>"F�}Z�K*��K�}��6/����� ���b���'l�g���]�4#�R� 0L�Jv�A���7��OUa��NYM����t�*�qZ�
R>�E)��ZW����q���?x�	��|/������;�N�	�W�3�����}te��� �[�m2�	_R)h���n4���U����<]�2��Y�Y��Q���S�����]����)"�pE�2}9K�,�lG����#�_�b�%��(cs��v�iU���my�6��Z��mM�=6W�������R�X�5���I����)������Y\�_�}��3����(�AWhG���$�^U�VU_�:���5��b$��G;R��+6����#UY�,OS�M�v��������{�Auj�1��V�*h�=���i��l�G�>��s�S2F�ea[���x�
"4#.'���1#������e��dWr~��M
�X`=��z���8���"��y]t�<Z��y]�G�78��4�6T�6l@�rV���,9E�������K��/)�]:���8�Iw9���B9��������4�JU�Jm@ k��`��L���;L^;WSH��9��\���?��,��H�Ni3P:�%���H�T	�]PE2�����daZ������z�����u[�&����hb��<S4��k�r{/�~�������q�h������L��H)��	4����F��\&Z�����
=�	��*NAf&l[��Y���/�s��>�����P��)<�D�y3�D_����7�:����tV����X��P�R����B���L�3�W�����0[(/YG�����/��D���3�z��W��9C�����s���S�+
�Z\�`�� m����f���\��SE�D��r���r�*��rj�1��S�G�)L�p7Jazd������U�)���e�pu��T���L
j��T�G��8���Y� :��������`��|V�&?B�NY�z����1����y��WA�r,��rJ�1����G��O�p7J}�!Vd��Z�I����V��I��]8�)8N�q
�Sp��SM��r
�Sp����-���P�a(���5xM�2]�y�����CH.n�����.W��`�
G������y�J���OyyQ�����!P��X��Wb�(0*����pc"�+���������h�Q6��5%O2'�X.����u���}IB��Bd������t�O���8�#��d��i2�>3�Z�>S45���F2[,�P-F���m�|��R2q�4��Ie�b��e4�9pY�d�!S��I}/�G2%�����6�{�T
��e���I�������%���I&��r�L��@XU+���^��_o�l���aU+a�3�_��;�1�����xM��3mD��P����"e4~��|�Y�)�{����]��|e�X,v��������n�_���:���~�|7g/��:������P" v�@��go�.L�#���	:P���zV����
�j�Yf��T
4��s&�,���&��F�y���E��D��2�D��&
��Y��/1�#�%�
1d��H�EZ���j���DO*g��k�7�5�o���n�~����F��4�N�k
��z��t��������+n�c&���_f��w�4��);�Y_��3���]�|�
�x������I���v��@Sx�
LV�D�T��!�(�v��@L6+Pv ���]��V�jd%�T���T^�pNj��f	=K�K\����,�+P�^�O�VM��T��
N�$;���]s������Z�
`�iG���!�AC+�$z����@��<}�R�.�ch�vAoy�zE���m�?��9��f�r	-4�d��
\����H]�d�\�2��Z-�Q�jQ���0S���ZS~=6�
�hF��-K�]���}k9�F��E�7��s��v�E	7��K�,}}XFqd�	��70�����|�Xx|6�5��2\-�V"24o��'	9j�����w�������^i/�,=m����f$F#�Z��?g���a6BG�e,���y3�]�P�$�Ui@MfO�JWN�\��~�q����1W��{���+Y�-z�������XJ���m(ZG�:�&�?|q���aA",��������\TJ"�����q(Z��:('�?|�cl0���<���l|�d�u6+��O����W�|���H`�p�k�����������w���=�=x�k��z���WL�W����W��Y���F��r^������:S��t>Y|9��e(��@h_���q�i�L:����*��3�����$T���sP�"OL��B�Q)��H	(��m�4�7K��*�*

�lC�RL�����,_�CaRx+	��1���b�T���hR�
r��/_�CRx+	��1�����bHT��#���Tl�`'���#1F%�'��  �O�/����@���,5�z���(mw\�#�5����e���/�(-�vg��������{g�m�]]���oXj��Q������*1#[h.�*.��H������������=gk6�SU��Y�Y��
;�������3JW�
Z��YQ������v��Xz�3����:��v����##�.��.�gO�����H{��|i;�C��]���9:��M���$��w�T��sv{��n�U��f�W�7�'��i��/�j�����;��]��{�fng�V�o��R�i{7�������]�����b�vF�����6`���H�1-�������\V�eYV��JQe1+
b�(M'G*�?,��@��xx�W��W���2�G{:��P=���I�KV����8�K,��,��������k�����+� gl���x�MXW7mW}Lq��2}n����"k7e�,c�d�F�u[�4�Z�w4��Wgo��??C=��;��b��wp���u��e�������������-�&�]E2^�Us��9�]��Z�Z~�jC��j:A�@�����w��n�s
��
Z��b��W�������2�P���`B�^M)NW��%��,���)dW"|)oW����!�}���c�
�g�~���h.e�'l\Y��Y)��U��Y��T�t�zC~uL%�e�����*�F�BWz�V���J`��_9"��������:���2K�j ����~������>���S �����)N���S �!`�5�N�x�
#@~�D��c(����en�t����S��8��77c����GRi���B1�
�Aw�/���������������SuZ���~u�3�?]�_�y	����{���������)n0�`��?$tJgxbH��xMH��K�)�	{zJ�������%�^���}b��v���)�O���P�~�t����0{������A�K�&_X%�%X�!@Y�h��&��zu����w��\C��z�A1�<;�y�M����L�:�9������F�L���C���P!�MH�C���~@�$�������{�������b�(�����H.��*�������������J�#) �^tEo+7���B#?^F!^��U���
�Sf�x���n������`�V�X�0�QG��~:����O���@P���3�����c=��z��z�GP]�1z��������=�-&=sL��w�)��oj���Q��#��W�!}9[���a7��%g9����AI��e�b���m���P��`Y�Z�3���zf�r���I��v�b�4-����5@hPO�x�V��u�oj�0V�j]������Jm����~[��H0��L?��4�"��n*�/�/��9����@|���{g�����~�_�w�GUW��b������H���Cc�KF|D-����C>~(�e��t+�n�HN��;�
�Y��x��~�ATP�R��vR��|I�@���g��`��[V�>�g�s�]�P��](O�20SA4e0s�A3@3�B	�|�P�^@LA�(��4�,�R��%X��	�&�hrI��%��hzI��-��:�P����I|\V��
@1 "Y����.g�k������?�]$,�����������X�$5	 �E|��������'�r�S.�N~z$���m����O���q�I���O���7"I-��wM�D��c#=��)'L9a�	S"�N�/'L9a�2ajv���S7���a���~h�
GFy\� ���	R�"G�����������vW���U���]:"���%	Ql��J��<�N��z��Pn������}�B��������#�|`��j�Z|p>������1_C���<r��'���z�*
Yg�TC�<���U*�����L�nj, ���8������WIV�(�3�w���E"����&���c�MH�����}Y�m�[�+{Pf�*�VW�s����j�DK��8$���%[����Nm�RZ���[+�u����K���/��lrd_K��q�>����:��q�����i��#��0}���x{m�B/M��h���U�/��F��3��F���y��T7(�)|�����,x`�3j^��mlOb�X�����f>��z���o�u��)�i����\�����G�PV�P�;����<����9p�z�l3��oH�.�p�X��R���r{\
o������@:������Z9k��\������~B�	�gKu 1Y��M4K>[�C���~BY��|�	R���w���'�f�UKx���M�O���@@�U���|���|���M���@V!/Z��RHvC���$��"A��0	'8�1QK7�6T���"B�c�/�)\'cTb�4QT�����`W�C��F|����A�G�^Y��?Xr��^,Hj3�1d@��D_��97��@)� nI(�%�v�z�����
b4���V\�����YZ�{���+��v�����(o��u�Q�!��+f�*�Q(IKC�����Q#-m�HK'�����!�����u2�z���k�������__��_���Wp������d�uU�=Zns�w���<�B`3��@�_YW��+YO�	'�E�B��1��a!�j���3�!���w�������n���2�h=#"���<O��|�1��!FI�Cf�xfwsv�
�rsz3����R�"�S�����$g�fI4���i1>�MC�E/a|�L0%�/j�J�=����1�����A��.�8T��$�A����s�m�l�`�o	���G_�a��%G����DB �A_�@|3��b<��]������1��d0/�G�yH
��r�u���EL��W����N��G��������������
��8����m��9K&����E�E:������&�a�@N{*{ee���Y`�����U���d`���'�9U,�5���7r����#�'�����}�{������2�)�x���D���R�J��S9 _4�U_�������+�>�AH�k��Z=R��������Dm
�N�V4�D�\p���Dm����#^����Qy�����p$R�[��W���*���n'v;�[��
9�;\'v;������-��!�mT���|�#�T��4�#w+�F:����N��@��nECN�����N���?�{;��n�z�����>F���bbr�r1�P���Nw��d�n*fn���1��p-��p3���(J�:M�@��89��L%�HqS(kedy�~f��)�0V��Y��MZJ��B���\��%zh�Zri��\�A-u�Q�C��MR:��������:BdL&`A��S���9�!J�6hvj��NmpjC,������1	����������B|Lf`1��S���9�!J�6hvj��NmpjC,����
��1I���
�������k�r9(�y
����S ����S L�t
DQ���)N����=4e�f�.�I�M�P�#9�F��������Dp�S"��@���D8%�)��lL�������h��9��r�X��&�w��������T�����`�S���^Nep*�<�2��)�24O\yL�fm�3lNQ�a����N"�<���cr%7����{��)<@�[��Y������<k�D@�?��,��C��"���W��������w����^f:���Lt��@���s��s����B���?E]�Ps�
I%���W&�&�'OW�w�lP9H��\���6v# �H?�d{M�/���@)�&�Q��^~��$0�0m�\!^#��,-�������O��S������
��Z��`)�y�jE���"	���E��w3I�L�I���*���'SJ���"*�)SPJ��d'��q"He�S���E�����^���%��>#x����!r�Z��XYl) E��og��_J�k�"�����!_�/+eYX���Q�����Za�gbQ�XV�l�Nr2�N�@N�����}i��b���&��D�~�a��d*�����xhj�2�DUg�����T0���a@��w��5;���w|i\��J�b�YP�D(�`1�H���C>����a��f.z|�:�� 6p��0;�e��\�f�8�%�M��
�s,�)��.},3�!��W��/({s���`����t�������b �b�������JC3�<gs���^�(n�o
g������_Y�����s`� K�.bH!+��
~?����+��t�>�J?�T�rP��@7�T�~p9���:�����NlO�M�D�-��&�l���mE6Qt����"�(�%L�+�
�Zo�����Y�b����J��Se5/��k{���������+�_���X�l�����u���ndw�Bo��&�o�����p�e/��An �K��J�
��dv3�
$w	rS�]y]R|�\�c%0�	��X��6�})]�Js���U1�
���}*)�s���:�}*)�s�����$����e�@��X���8u����c(���������x���j�����b�\�Te����c�����v��{��P�h����3��q"�Q�[!�[fN�Z����dC5���B�������W��=����,o"�fL�_��Y���n�3���	�~��v�H�I�|PN����a�����Q��a����qQ�Y�-�(�k������3��-�h��4�4�l���iPm��ry��	�g���l>^��8���1�����=�=x�k��z���W������{������
��������^�����-Cc��x.Y��'�/g��y6I�����U%��5b�5�
?e�nf��%�v-�����&���:e��cf�T`�c\`����Tp�	9 �����F�2���k��;j$��lh�\G��[��5,e���`h�`��>|�6�}�*)�Pn����qc\z�9>\.�o�}�F�4!����_.�.�H���.��>��:������,��{���)${a��nY�������r+�7	�;$���<�&�PC������DJO%K�3PO�6����Z�J;�(i����Py.�?|�H��,�2��&��I��;�{�,�$��LQ�_��5,������_fj�1�(l�D^�e�n����*��U��cXf����l�*�~���/����L��������U�C�^�DT]��D&��a�A�����v���IxF��].���]�P��Ei�����g�3)��$���>/����������)ll�N����/pI�R���v��+�
�7���<�^M-��/�j����;��]����X�w�^��vjg�v�ig�V��r\D3�w$Q�'x�
��-��Tn�����M�&,)���TjB����iW*� f����������!��+���G������3 �O����al�G/�:w��� J�%"+���)���y�Y��dI���_.�?]# �\�J�yBgQY��.Z�F����1������FQ0RHxl;�m���P{Lg�*�5�Z��gY���F��d��m�MJiV�G���E�+�.b�`r*��N�	�Ar!����<�<��@l$���-�\b� 9	$�N�=xuq�����3�)�w�,�^|w��,\��yzA�M�t'�lC���W��_�~�T���������v������{�@_|z�O�&���7	\\�m|e�i�|��]�t�g�2�/�h�!3������K5�a����K�	�J~��<��h�}m�;������6��`
��b[��S���n)7bL��WX�Gmo�l��R:��%i�_�IT^�[?L�U���O^A��yN�sr������C1'�99O�9�+�����T������������7����w���nn������!$�&1`��5��^_\�����}�
R���ch"S�����,-/._�r�EYZ��W�������������������]]\���X�t���CB�tiO�#W������{
n������K_y����}o��G��8z���S�&�?x@������������i����d�a����]B0*hQE�"�y��k2#�����5��W�?(���2��-�It3�����$������6B��t�� �.e@����"2l?��}����|�,���w���~z{q���&��3��R[V��\f���^��x^�yL�%�8�k�-`9��5�|E9���O�����zD�����zL����������,�k��J-� �5<�r��B�������2��g��,����i��>z��=,�?�������h��S�
?�
_��C�b��(���WdN�:��!f�*�#����~O� ������~���q���b]i����%bn���a�[�u�6V"��y���:�%1���"���Nv�d��4�H�02��m�O
����2�hX	L�q��L~i����|QU��*`�,���K5�������W��� �U���r���8z�Mu/[�F%P��v
(��e�s��e`eP��MHP�m���P���i����2U
��&D��HT�o,����hU6��������UZc �7�c0��}&�?�g��3���W��j�1e�TC�vca:�@]xd����
n���m@��@���6q��*�}�������J�Go�c�\��*���*��[�je�n�G\P	�s�&C�b�Q�T:h=8���*�U����d�����Y���)��1���@a�&.^\�? ��;B7L+|v ���-�$}��.�����6�/�T����P<�)�K�f�KMqi�*����@��=b50Q�X�� #��T�L%���$�*���6��S�2 �b��� ��-0�r�E@�'��[�Vw&[��mMU�B���n��� 0���QbUj�H�<%V�II�������+�@Y�IO���q��'%l��}��-=���������~��2��25B'���e�B�q$��$�C��#�.����,�Yy��jkJL:�(�
���Z���O1����xU�)^34|e;>MEZ��6��5|�8A�O���"",�2������Rk ��xv�(�,t���D��"b��U�XXK����6V�C��K��O���"�e5��C�2L95�O��|Y���=�I��U�I�����5��ez�4�UT���q#"f_Z���QE��Ri���Y�T�*��h�[Z�Z�-��
�Y�R��|���0��������=E������
����y�3I�#jk��L���<��\������sNG�o��N_�6��u.uE�.u6������&PS4�SU�elH����Sl��S��<Z�"���Jn��Q�M�b&�������D�.h'a�����+�Rd��?�����u9�O_��$��x�3}�wp��wr�rz{���"�T�f"0�����3�l�r�g�q���r�g�1D�o.g�1E�o���o���ov�x�����~��on��YCoP��2�#��v(�@�*��pWs@���i��9���Bk����H��p�B��A��6~ZI���(@O~��������&�[g^���eD��D�g�+�����hf]�|)1X�;Yk��aA�RjP F:��j��/��z*��`�Kdl��9w���L\�xX�-�"��U�eA�N-Iv�%����;�Y�V�-Q�<_+�L��>�m/y���__o*q����Y��5bW��hW�����uD����u����������x)*���_����a��b���%�j��W��EW�
{)J�X�Qh���]Z�j1�����}x��k9)j��:}�����)������v���@z�/�#e�A[���@lG��y�p�^��J���m�J�(F���em����m9���xkQ��Hk!J���adnx�?E*��Y�-
�5AK����n)���,������`cj���Y���QxY�j�V�}	���c$�T��Z1�
���(j/xV%f�*�EH�)�u��F����Z��j�T�������,�^�O��.�,����4^�^���cK8�V�0�Q��	������mf��t��x5{��z���A��Xw$�~���aT���<�K�R�;���G���a��j�FU��,�N�0e�R��x��J�s7/������S'�.7b&d�cC&�)�kL���L�Z�x�V� ��*��MTl���q5���0!��2�"p�8���]N�}uU4=l�������o�5�u�4��� ,ou���6#����@����1_����
�5pTc�@"�d�l?����U�BQ����WFs�By���Nj����[
i,6�Jyq���j#��E��f"g�'��D������Q��R�^[H��JP/$�W%��
�Z��nJ,f�!J��-H�|�h�T���aS>�������
�J>[4$j�$@��������z����P������7��x-�v�/�P�yw��[9�S.������;I0����6���b�\l^f�.6���5���b�6��+��.!�����/��l��I����t��I����Z���^3�O��%����/����/��r���+H�vJvDiI��9[{����C�����#p��H�m�&���h�@�)Z����/��:����������L����$`N&��R���	;T�	3��*NX�BJ�!�T!pp�2�a�
����*���
F�c�����	��$��$�^M��8uS,�R�.N��\��4,�SG�r%0
A��S��R55=Z.����eH����TJ�?22#x<ABm�eD��e��T}��� )�4<^����p5]�cxs�\�1��zTR��,p�������3�����V+t�<6
�����+��yz	vf}����?���M���VFT��������b�Zt���-1�'"���)O)�yu2O`���d�`��_v�-���%x�������������S��I���UI�]8K����n5!���x�R&�����(!&�	1��b����(�C���I@V����(�Q8���KJ1��IT��l��z^�6������i�����������(�}��>��Pq��:�+���n��f�9��j�{��R����*|�5W�;l�@Z{0m��+�v|m���|�mg �v�;���FXI	�-�J�X+cCU)g����*�\�=Uq��v����_�������VsK�;���
V��W�����
j!����~$_
�c���Z�M�QU=)PWO
��'�zRP_��<+�F��;UI���������8�eg4f��y�'k�c5Va����l�	�3gBY�l|���l� �D�B�<�����`�cL|���9�1����G[���#�=#��QjH��M����:�a���.gIu���z���S��`��S]|jE����O�\|��1gjLAp������:C�34��t��L(�����=+��.V�����X�������=5�^��,�.V�Y[������8�XU��bU]���Uqvdg@�v�r�����L�YP:r&���U�P�!7n�G����
��P[����n)�����~����X�3�T��&�IV����Y������������Y�������#1g/tFcg4.h4��f�Y��
����-;�w����5���3��;gvk�����hgE�/����V)��/F�V}������C��U�a'�6��S�aSwK�
��Wc�n����5;�c�+F��	��
���Y��F�L�mL�����[��j���i�
�B!sBU�KB����_P���q�;�k��F��FR;4l�M��LE
L���J���I��f K������������S��S��cc�)o�)d��1��7�2�d�l��l���9�Mec��6*�M-�������%vA��_�<.hQ@�-��E�����k���-6�2��]�b�fo����ZtA�(\��Z4�-������9xUC�����2��E.*��W7V���/R��_^m����/���/�u��3.����R;4\���y��/����]�b&������5�F�*%�2n��SK4���2���s���3�`F���]0�ft��.��3f�S|�`F���]0cb.��
\0�f��;
ftF4g�ld�<Z�=P�	w��N��.g�v�og�v�og�v�og�v�og��h�4 �r�og�v�og��C�Y�-���Y���;�����������Sg�������/����y}��=D���������j��:���,�����C����c`��6���s�_*g��g�V\5�W�m�f�����^pJ�����`��cd7�_�B�<�i���L{���=}��T�Eu�mp��p�o�M������ns�|	�/ji=�Hg��/���%`�KFd`�T��FjC �!��C`N�@�C �!�������2X/�����F�s1��+������p�4|�D�F��3��}A���1�*�F��.��	��	���L�l|���'c������>�����Z�t�d��jX���NA��J���T����Z������Eh����~1T��-Bb���+�����Q3�~�'�;���2���8o�����Mo����D1m�B���>^@��kUVw���H���,]�ox�Qx-o�T<7����x�8��O"�R���X)�$1Vb��$��r-|H��mC0p�� m���'�&-�/8�k�Z�����/����k�"�((����\�/�/��o�3H��LRS���&
��c�B�����.�x>LjN��7uJR]U�A������e�#���H�G������{�c���I��������{���OH�'������{���� �� ��(Q]�L��,�4W�){��@���)B"�������Rd���j$�a� )�0R�L�0S�$4lU&F��������@CT��l�BNc����32�XQ���|�B�����+�����0b�@_��s�
?���>#cT.T���(7_ES����T
e�T�_(2��Hz��y��
T3�F�"a�y���PPv���%@� G�mb�l9Z��rL�K����i�-SK��@N z��D�l�Z��I���6����|m������������2WUF������*���[8������F����zY2kG���le�,(
S���_�3^/8����EK1���h)�
-������h�.��g�b�Q�v�N�U��7��2���k��)f`	s3� �`	s��) f�a	s�� Fb	Sq
�y*��]��$xr	��K0'U�x�l�\xkC��dAs��*��r�kB��$-!\�0�@8��|a��m����L�
��p�!�����[B���[�p�"������������cZ�,����2�c���H��������Y��P����g�_#o�-��bi���%�(��He��3�������>��jQ�+�����g)�D�T��h��B����&�t�xv�N�B�����_��
��LmTx;�s3���|����F��c�853��Q�X��GM��a�;(���p6+$�/p��{����\��,���y��������(��������%�������_1Q^]|<��^_�g��;�Q�.��)>E3E�����w��g�&��'�/g���I��>!&Oh�����G�!-����_[��D�*�3�� ��H����7�t��pK����xm����5���E����7L��5��R1b�q�T�XC�'\�u��
��PV������RW�YR�Z������C �0�_Ve��JT�rce�e��R����e��������������_.�.�/g�m�]]���oX��aNw���\`Y�����XSN�DV������;���9���L#j��T�i���T�i�O�TKi��J��Gj
�QIH;����M��g$��v�X?@� �w�g@z�09�w��ocQ��E[�*���:��x)��|n$�Kq)u	��J()������g8"g����6\E*�����<�^M�g~I�qnhz�r|\���c+�B���[j@�c�9���r�-��cK�X��-E��c�������VKa�V3��ve��
rh���������B]��}4_��?\]|8�����J�=��K��������?�����!�P�j�]�����5B
�?�R;��6i���yo��&�C��q������xo�,"� ��[g-��BcIuq�����3��)/t�,�^|���,\��ee�-H_��Q���W��_��X�p^���	&s@�����Wx������R��N�?P������~r7��Oc(�*�2���N(�?�`��Jd�2��ZK#S���n�hrO�m�SN�
x��\h9���UE	f*p�fz�Oy(4{5���?m��	�>���9���E$<���$��>�`�gL��cma�]���H�(`U��������p��T�O^���p�r:���t����g���������������vM~vJ\#	�����1������o.O����-�{��{���>`�V��m)�KA9�����_���o�p�k#����S�x��vk���z9_�<�~���2��Y������OW�c�`Z�P�~H(����!y���5!��/���&l��)����..���=TjL���w
�S
������?��|�=�n��f�)��
���V[w	�� �� �1��<r���3����{s
'���
��j���D7�9X����"{��l�o�!���W�Hy
|�\F�����P���+%��7��}�>�����X�	&|�	��HezUF!g�j�$��e�^���%�:�2��EW����E3��w��(��\�
�8Z!���z�������`6�W��~�}�B�N���D!��0����`�~<������Uo qK�b�@4�L�"��0�l��"nw��=�����;{���8F�Y�0&8�2i������^�Qz���N?�$ifq�������2N�gT�Q�T����%����v�x�U/��X�+a�^�.��yP��{Zm�&?K�V+��
��F�� ZN�5���6?KuU��&��a�junh.�i����6D7��x����`ha�P���r���I���J��3b�.d��|��?Y��F��P�E������*�������������_�fnmP���_��nm�:hm�O�d�M���v B���N����5���4�_8�CS/����J�"��xx�? ��w�_�w]�KO����}�}T��
�p����vb0 ��Txx�9��V�8��K pg�����O>���������:���
y*��UYA�jB�I���x&�X4������v*��m|�g�W����}�����Pvd�d��@
"_�6���#�)���\�d:S��dB'x<��� o5a4q0j%]"�4���c$
>O��v�V`�����E�U\h�NI@��	F'��9�1P4$@u����L��a4	�S�@�Q�_1�oc�@tcj��E���[7n��/����i����p�fB~bn.�����5q �
�)�N	�WBBA��������#J�]���w������?���z���Xf���[�W��w���7��1]mWMz0��V�uU�1�&�3R����w���H���G
Z��YnV�,�A����u8��F33����L�
A�^5$�TiQSh�un�V�}35�M
�R`�U�S1^4#{c���C������m)^)w�g�#��9|�fa��5�T���2�Xs��fk���r]P�*V�� ��J���K�~��}����>�9�W��v�f��r��	X�]cl���on41F��nSP�*V � ���{����������r�q r���\���y�
���������q�H�k
_�*
�~�Tho���O���r�r������}=����_�Hi���W����M��HY�k
_6�����[��H�%r��r����������z��s���*��zW�{�7�"#m���(|e;;�{����Y����s�[=���;�+�u�}��<���������~��������{���{�����!����^�{{���������g���9xD���_R�
�`^.��7��F����nt��
R�]p�nT���]pc�U7:���:��g����P�s��a�����:]�h�.��:�@G�X0�r��.������@G��]��tt��.��:�\.��:&�s���A���@�@�G�����z�*�>����fZ��a���������L�[��m�����Y8���#s��h�/b`�5���7�Y���3,m�UHK����n)����b��]@*�b�:����B�"V�*��%����U��J������0��0�3}T����^�i�U( %����n�z�ee����s"t�%�����~I�X"��_�?V1��;GI]-��AJ/���E�y���D���N�U���+=��-���j���~�9k�\��?�r�S��3b�����8WQ������$�h H���@��R�F� �G��)��=�g5�z`�z�I59��hd�$T%-��e��t`;Ff6d=��3�.�}z�aA1�����\XF����U�0;��x��!S2�����f�� = EX4��LY��,���4|�r�����UtQ(�V�Z��(GT��A��a���R�S���rq;z@��h���(
sf�F���.��b�>�D*���e4I����#�`?�a��8R#ey-\�����S�S��,��,��X�/��6�C����e�G�����`-�a���R�v��r0z@��hj�2e�e������E"�j�2tA�Z�e�jQaq��hM !u��g�ra* ��#��sw�Nl��H������>�(��Y�>S��������j��wEy��r��|�j8��s9k��l������l� ���B�X|��d-�7N�*a������giZ6������%`����(�cg4_8������&����_�&K4���J�h�b�*by�pq�1����(c����y�b�y���J����<|�[�,�0NC)a�Q�Q����a��E���Yv���VPj��������j��������I>[4�&�l�����ESa���I>[4�%�l�D�����F-a�A�1kX�G�������$=/�������x.��N��N1Z���%�Wo�:ljM��eX�TU�2�$]���W�mY�v����j�f����{9����*�mX��Y�f�p
D�� �g��k,�(J����:%�eq@WDi3�O�2G�A5@���%��p@�+�@�-QT�z ����<���h[��2�\�F��[��-mi���D�m�\��w2r8M��t����<�=w�
�K;��~M(���������y�Qg�!�P1{��Qd����Z�L��#���l��S��	\���}�u?]���wfy'�*|����*�KD�F�������8
W�s����J"�E�*B58�8	� �u��������L��4�{�)�����-�i�S�SI�[���d&+�7��Y�pb���������
�����c��N��.�JXS_=��Q�|�����/��h���j%0u$r�u&jW9�����D]��
���9I���e\��P]#����-��EK.j��+`T���Z9��f8d�+F�tJ�xq?����t1����r������=E�M��0G������=^	Fjpm,�~75_���r=�9�4�����c�d�w��]�,��=X>H��
�$��,�������U���}H�yh��c���JAd#a��x����l�����9��k��jB���@��
����4���A�x��h�a� ����k����v#�L���HK�������������	J�h�i��
�|��-���RC�^B�E�����7��a�������^�������d�+��c_@�'�u*k1�[���>�Ho���>]�������^�[����W�T�@�H��^�E3���=o�Fe����Z
,z}2�Rf����E�@*�=���-
���,z�b����Z�Z����U��k�����������Y:4�1W7��H�;s(���z���Ci���S��J�5���G�PJ�E�����-�(�h�(.UO#�p�
s���7�A
�^�P��c�V������H= ��f��U�D�i����9Cr.�q�=!�3�9C�3$/�q���h8g�s�X`��!���!���!F�9gH>��3$J�q��(�r�����1zD�qn�[d9����_D�j�����9Fr.�q�=!�c�9F�c$/�q���h8��s�X`�#�1�#�1�#F�9�H>��c$J�q��(�r�����1zF�!q��k�f=�m�3"��[�R��#9�s�8�����1�#�1���s�8�HY4�c�9F,0s��q��q�#��c$
��1�	�s�8�H�u9Fv����"��8��8$��Xm��6x����L�z����+;�R���LW����(�P�re���D��rD�t= �5Z�F�I�B�i�:nI�c�1���!�����ax�I���^z��@���<f��Q�� �WQ��� ���U��
c�$;��.�e-*�>�9q�pZ"1�
3�FB�'h0'{���W����#�4��Bm��D�L+���4��Gm��W����$�1 h0����1������V'w��Z��#Ki��������]�����(0E��>7��4E��d���(���!.��,R��n�)�b���4)��n��V����qc[��n�B��g�N2�3�(@���oK��bL����I����Lm��������,V=���OtN�un���s�[�D��\�J�63�o�P�mWc�m&H���j��b��| gx����������.�������4��]��3��.gxw�w-��������2@�����;mk�����X��]�g�� 5��Rh���vN��}��3�;��3�;���3�;��3�k�p�w�����������h8��3�;��3��9���2����m���bw�r5��f&]���wE^���9|��3�;�3�;��3�;�3�k�p&x������	����h8�3�;�3��9|�&xgp�%l��xU5�\V�����o|q@X���P��������������D�����sfxg�wfxg�wfx
g�wfxg�wfxg����m]��=egx���mQ��w��gt��rFwgtwFwgt��wFwgtwFw
��.P��u�3�;��
gtwFwgtwFw gt�����lM���hog���:����e���>O�/1������V_�C�\MWS@�U8�E^<���FfxH�g�M����6��*T�_j��������:G[����l�F�N�+���&v�q��"�Z��#������/y��O�c�7��u�������&�������|�W0��f����E|*1�� 2.��C�����lM��W���L��1�`�J�9a@2A3��z_�����������C�}���@{���~���i�����v��+�}`N�@�} �>��}P�s���������
@��^���3��ff�L��"P�Dd����Y�m��||�<��#����
__���Q��_��d,�Z||_��~|���'c�T�����x��	*���T��/�����O �O�$Y���*Z~73B��%�/����K��"��i�H��m����G�=Nu��s�2s�"��ym���X,*���U�������*T�8��*�Z�����-�������,#�����G��f��`+�V(R�&:�|DZ�(4�����f�"�l���G��3�s:v����t�||�������w\y�~�kU��x���U�p�/�������'�~�� �>���{N�3��#�V�#�zD�����<��.W
�-d����7G��yl���<[|I�#��7f0�
����?	�J�ob�cr�|�:%\����m�z+�H����K�T$���(J�����e}p�����(N�l�<���P���l)�XnfK��[��
�\����u�����by/�g���������1E�&FB�XV��F�&CFB�X��1�lS�lrf$��e�(����@�8�FB�X.�1EF�������K;{��>�4]�I��F�R^�[�u�GEX�D��]���A�� vL;.��q}��I�Nb'���������(�����������o��u�B]*u�4�-,u���z�$�D�Kd����V���s�_i�suL���u��������1u�6��:�N��:W���Z��Z*q���y(��G�uO�����l
wO!�0��	@�y�M��)d��	�$���)�=�l������"�eS�{�=�]v�t������|�\wO��P%�A���*q+h����~?����{�
�]�g��*��m?5�q��*��;r���z*7^�/q��N<������y�LW�4	&��0�%4)N\M'��f.RFe3i�.������4U2['���%5�F��
��5���u�a��������R��/3�� a;
�;
�@^
����������V�/��F��D�D��AC+�A#C���PA�AE�2�E�Dd�b�V���]yl+e7�����X�]�MWv���te7h������,7E(&wh����f[���$'�K����T����v�\1/��~9@5���7�!�����T����|��2/���r�j5��o�s�%y�t �*�X��?�05T2�����x����f��x�{7����c������[<D�p�XZ����h����T�>��0��<��k�)���z���6����*j��}���r��H��&�����k(�b��}��5K��fe�M���,�������R^�0�%r�s������%e��/N�H����_�I�w�8E�PP������*2�@�����
�)�G����B�f
�� `pZT�`4�������]SCVe���VKI�Q1A?�?�O�C�g~@_�'�������>��>��y@>�l����?������~`;G_�'�a��������?�����p��p�?�������O��	��	��?l����?�������v���O����jrP��Z��k����7���ma��3)|�x{q~�����m�y��K^���+����b�}u��|�{}����J���#;t�t�{e((?ym����������g�<���|��rv{�e�H��-����I�A�w�&�R;S�\
u�/U�G&���]�C���e����cw�q�W-
w��;���������])"�4�)���	r�T\1[�\1�<�\1+�XL�=����w����m�w���%��������>�Op�1�xp9b#���^%���G��I��d�D�r~c�VlLF�t�3���\\]P����_�gWWg�qz���}����b����VJ^�6!�����CW�t���=��������b_\���}���CSC��]*z�1.U����J��t,KU@o4~E����(��Y��2@n��&b�}���A��k�����}!�^�.��D�.���4��(^#Q�,vl{��Qtc���������6-_.^J�G $c!������g8�g����6\E*�5���<�^Mc�_���c���	9!	E#�8���\��C��pq.A���C�E��!�#��T��� nx��(2%�'xS��f����,�,�FAf���I������Bm���}4_��?\]|8�����hO�P����)L�"�aB^��e�]?�D��;{^����F��_.�?]#t�Lv.�����?fD4�mo\eb,	y
7&l�<��A����S�.��`���S��D�7q���P�`�������T�30Q�M@��y*���(�& j��8��(�.|Z<�Xq6�GC#u�*��S��A!6�L��J����qGPv�����f�,���;�"k���Z���)������m��V��r���#(��@K�f�(��B,��2e���a��%e��#���#�[R���l��u�.���}~�z=���������b1���82�P�m��T}�|����W��_��T��&�l��|Z�]���Q'�{l5���cs��O����D�����/�Z��lje����JQJ���`����<����T��hyD-�-�lR�+A����W��l/���i	:�N��@g{q�T�K�����+{
t�J����N�b�@'�����T3iu7P��{�Eq���q�\P��"�/��J��^C���jj� ?v�
��f��AU���G�T�U ��33��� ��4U�6�4
�*'xM�h��*}�*C�t����k"�Dnv��v��W��p�=YW>}xuv��+U^����+����+����+����+EPr������+:��#RF�}�����^��P��.~~sy
]���x����w���#������	py}q}��w��-f�Ri����i�R����)Gcz�ek������ty~���%�exs�j�yW���.?6��C3~�!�tYO�#W�����{
n������L_y����}ogG�"����S��N=4���?���=������������i������a���f�����8�hx�"��z�`�������������|���N���,����"��v�o�!L���W�
CY|��H����������rN._)���y�����Oo/N��3�m���O�}���(����y�DnM�<w�r��%p�k@y�����������P��g*
W {��Q{��(���x��y
���z	�">��>^\����Wg�n�_�INU���R6�L�������{�I!�2�d*�[O?���\
n�;�S��?�vok��_E�z�q
��!�|c+��ytp=��!�&�A��sW;yB��#������A����
?e�|O&��>���C��RFq���Z��g�k~��WE��3����c��p�T����>	s��@�C�9B���X������d�����f,+�	�*��
�
�P�2����U����+�M�i�'���}_H3T��L&{
aFj�q�$�>����.����I�$"��*��;2`��Z��U�!j��f\1$rj�(g��l��j�!��������E�-jU���CW�a��$�7�s�rSPm9�������*h��,UnUcO� ���-���#.�O����o��S1���[-����|���V�?U�+���%�&/���_��3����=����|F����E7+b?@wD�A���(w��m1��xM�i��&��cD���H����&u)jf�$����'}��Is�+CX��_P�a*�!������@j,J>�z��y��� ������RrZ��
���(6��j?~���nX���e<,tv�E�R�3Z�a�	��P0��J�~���z*wj�k=��Q�B�pTs��'�S��t�~������������'������~KN�B��\�~������qm��l��������q��p��OzG�o�\+�\G�'\+>����!����o��q��\�=��	��J�����7���g��������p���S��c���qA���7���(����%8���8����l����h��'���>�C�����d���<_s��9�8z�<d���8��O��8>��q4�(�?� ;��t��.�-7�}�_Q�o7�,����/������p���f�w���s�Frm��psl��������Q�[��u�_����qd����[��<'s�}���/x�����_1�8�y�h��8�����~����0G���	n-�V}���x��k9����=���6����5���<F~M�~����[{z/�}��Sy�'O8Z��<�}��������'����|5?�8�=n��)�A��.���w�.�
7�z�����S�� ?���[��S_�<��8�������_�������������Cz����3�9�_5�q�(���xX����D���!�s��y�A����d}A������>�����A��An�4�������}�_1�=��Uu����������_��<�����8z�yY���1�}�?9�U�_Q8�
����V��x������_Q8�y�N�u��[y�xz�2;������/!�r7F�d���q�{?��q4��-O^b���x=�?~������<8�z\+7�AZ���f/O��/K�k/5�t�%N~��������c�7^K��]��:Ae�kCM��8�$q�xKP\]yIp4w�2����?�}}��n��*f�)D�(K������-2`�WC^r��z��k��JFCfmkd����)}L��T&�f��1��r�2Y�5��T)�mfR�t�!Bh_P��RP8�~W}������?��wtG�YM
�������,�3Y���d>�ME�L�g�3d��U�����e:�q��g�=	��F%|��r |&b������g�*��g�>��U?��55	�-j�>[�T$|&����$�g"l	�-j�>[TA>[T)>aU�lQ��������EM��g"�
�-j�>�����3AY�=9�"��Y��-TeQ���eY�z�����E��G�"#�L����9�m�K�z�a������}��	Wf���L�>3��|��6Y;	���=�9�gTk3��2+���G*��,k���*��b���)�b�G�r*��������*�����m�������b��J-�LL1#���Q��&��nOy��^����S�a�?AS����?��"��?��2>�E
�������v���<����38$18���JR��_O}�c3�{m6�m+l�F�Fm�Ix/�����H���|3�T�vpnQ��~����,���J���h(#)�9�2�W�"��"�OE�!g��{��(_3C��7��Fkd�1�d���d�����q�1���Qy��`�o�����~>���_�0�pJ���r<�!��2����MP�&���Pb��B)U(d�����@��������o�_����'�Rn&Q���j9:(����r������'�����,w�j�.V�����2�r}B|�e8"_O�A�WC��+��b�9N|I��7���O"����|��z3fj
U
	8�}�&	�����D)��
����]�3���b�z��J���6�X��&����
�+X�g��b������ ��g
������}�j��c������������:j(g�
�P��[������Y<ft[��R����"�����])�a'4i.S�XK<��
%��'<��t�)bL�GG���Z����19G����1���/k�@}�����J������9�U��k\���������8d��/^f�>���m��s�� ��J�G�of!e��>�R��Z.�LkS7��U%�K��e����]Y|�����*u�,���z���lM% zY��Et�B%?4:=r��\Mo��X���E'�������`go�T�Y9�WQ�dS`4Zu��L�f��`�,���#�b�2f(��)r$I�[�w�<����p2�)�j��N���"��F��S��
�<�F���Y��
W���F���i��g�����\DX��q��s���>���>��4��C��
q8�S[���d�y{���/�=���:{�������i��S���?��/�(�WIE����%�q�-s$�0����t~"q|zR�H��z4����VD������u`ar�V�R������������K~�����T����vNi��|��3��������� q�:�*7����L��D�NH�?y>�x�H�2V>b�(����"����V��D�-�4�x,�Y�*c���5��9�N�jA"���'%�fy���5(9%A[���z��wL��VZ���
���D/2��b{w`���9xgw��-o��e��>��/F"Y���Yc%��t%��7?�r��T��O$'(Sa��>���~�����^�qp��$��;-���0ENw-q��fOi����qP�]L��|L�z|4�/���J�{I�~8:=���8+[������IY���8&C�)�;M��X����c��6L���,��V�����Pd�q��2?5���zV�5��!��!�E�+��a�I���O����d{�l����*����~>o����_6G7�2���j,�\�S.�y.cW+��I��T4)�eA�T�nU\�2_^��2���P�+���vp�O�n�|�vs��\c�,�\I����uW[���0�zk
=�~��/��~�_�N �3���;�V��cS���K��m������[1���x��<[�Q�57Z	�v2pb�J�%6�����Zp�SWY�;���/C��H^�R���g�S��k6Ka"L�6�v���!�thJ��G4�\q����<	����;l�����3����N��0�@�����/��h��&�60��	Z4w(��N^U��/��&�uw�*vb�[T���2�Tw����%�x~9�o�����&�Y�-���#s1A}����������G<���}���,��<yrPem����M��I|g/�^���U���/�����'�m��Zt3��C$S�7f�6��4�190�)������)��'YX�r�9k�:�h���3�f�M�0t�u�e���u+;`N��I��|��J9y��K`1�>�$L�V21�F�R}f�U�]v�,���^Kq�r��G��o�-�t�������Wgo��??C���9����NW^�<^��������v��s
3�E�a<��&�Y��jMQ���������p<{�]����-0JF}�����������^7�{���_��z4��Cx���G�x9}@�{��c�'��l�X��}���h7��Xx�7�7���i�%����8Uo�hz;�}���5P��1�p����o��z������a��a���x
��8
��;�w2����I�
�/���/���x-9�;�
A��N>O����p����#������'(z>�.!��E�������x�1�v5�-�T������
��=[d�����seW��htog���G_��)����ID��l]|�&�A�h�=�D������n*ew�U������GpV�����(\E@"�����>�����W�,�6F��\�c��}�����E�aH��{t��p}�[��{��'��^�}����<�aC��2���W���;�!p�.�����x���}��� R��
<�}u=t9�|��X!d_}�"��#0�`��7������|S6�f�2��w����|�I:��Y$��73��b=��W��`�1[����j�X`���F�C �����S���.
�%no��q������^|9i��9�����!v��&��\����(2�Z�f�x1[���c�Wp��>���V�����T^�g�}�,.��Q�J�������'RP�L~�7kR����O��t���^���V�V]��QW��o�2�v��7�/���7��z�����b���|��\���`����`��h9��v��
^��@��4{vy����/�L�
�&���;�)��_�����,���*^+&@���r%�'��Z�C�*�_|��Cc�.��X!��@�5�(�����Hr���F�a�UL��2��|E+��+���Q�����e�(�LT��3<'��/��/����Tb"_�D>Y�KQE�N-��ee�56��u���|��(]h�^UN�dKA-l�ml�k��u�~��D��ZFpp�a5p\7�������1���
�4����a�y����<g�'�u��';&������bRfC,.�EaF|t�����8T`��}} �E��x=�m-��
��P��p�����p���l�c�D4^�� �����=�}��������'C!�m�����n�d���5����k�u��ai�K���#�/������S����)��
��c���E���a����Zb���c
P+[F3d��7���N���.T7,v�R)h_t�����u�0o�o
��0�d�&���=�v�c�&X�49v��wS��xhw`W0��
�*=�b`��;��P
�Yx�����}��5g�=mP�*+���u����6\E�p�G�Km�;^�%������QX�+Dh�a����gI��A��f��%J��������Fv��v�����=u4B����X�GD��UKy�m��"�B�V���S}��0�n(�%"E�D���	@	e�BFc`�j05����f5���|5�)[���P�����h���H�����+X�W���EV�5��V��lL���?��!f;kR��3-6�5��>H�9!$������<@�I��U�k�(�C.���<Z���@ak�)��6�w�YV����m�����`���N�=5AZ_�e��:Gx����q�	x��U���VxD
��c{�#���\��I��]g�F�Q
�U���Y��Q4�cA�x����0��Y.5s_n�k��e��
���cb����"Sn'Xw/�%H��a�C�=��cG���*�W4/�Zk�\�Z����m>~�.
�lF��j$��^5���8���o'7D-�%%W�	��
�B�
��:����l0Qj:d��L6����
7�6\L��C�6�u`c���MF�1���{�
��(���t����-����4��$���V$�,���X��I�� �����gj@�������19g#�1q���|u��<c���u�=��������@�s@1���G�����|�s�\�yGT����o�x�}1�Ex���M�
�����JeTR���Pg_�n0�u��.nm;l��a���.i�#�����V�cd6��~'��m�m�n}����A����*m0F������&g���hn���'l|���'�m�t��������T�OM��}��d$&���I��X��xoY����f(��3�ic/��x�L�{u������C���9����EE�I;4��������^��n(L��
�����5�����EKNd���TA%'��9�Ws��EsN����l������Jn��n(� 2'�[�D_��>��jR��v��cp��p�1���@g��q ?��}|���1'H��z/v7*D9�8T��
���D�����s6eO:IN39TI���Ei�]E_W��������X��f��:`������7`��'���}���!u��F�i�G���>��B����j�="���'��Si��-�7��P
((�������0��f[;��@�eg�>�)���Nq�l�4�)����k�u��?4�Zq��<����J�(1�e&���h���Nu$�,9�.�1�n�L]�G����rz�����
�x���3��](�U�m�.��b+���L��
��������|��=i��V����E�\����-)����	}���������g�����'0w���������lO!>�~�z��c��X����!�"�N�X'h�7�G]�������\_�a��u��!;���WO����/��eL����7���p7I���@���D�G���D�����q7k�,5�7k��`Z��9+�F�����A,9��s}����V�Z�����
|������	�/y//QY�kV��=>�IW�_b6W����!)-�V�\3�\WL�5G��fZ65~&�<����1���l"�v3���.kE������u��Bx�I�s�^���B���nb��
�57X�.qb�G��j�������	�Hr���%�K��n���"�����h�E��o>T���c���Ee �Qw��m���m�dq�������O��
`�8�v�
O2jE�:����_�{��*�iQ��l�BZt����Y<���*�K7A�s� S-*�o��G-��3�vNQ�=u>j�Rn*��;Is����cAP��
E��v����,���������$�CN���k5m�����`�`���`��%�
���(�Ws@_����@��W�������<��_e[9G��K-�������Wf_1��t0���/��4�`F�|��F�><�����b�.y�[���(�&���h��%�9H&�8�����"���B�B/���/���5V���b>���A{�h�%�oJ�_�oqy��R%6���G1JAC��x Q�#Z8C�y�U��o�p6�A�(:�����v��qn�s�����^�W����r-y����Mc)�*s�Z�P��]���<.ItH.���z�(����G�NC�T%����/E��K�@$�����U��\�F8(Ot�4G���c��!��k�(��x��I�(E�`}�f(y���MU�Y�*����Z�����8S�y���C��z:�v��v��O��:
���^�n�+���vU)�
�p���$��>���$AFG��q��P�8��!!�5����
u}��%<���zZ�XHH���o'�RY�_:��+,�MJ�r����_�|([*�|����/�|���C�������%I�����k�?w��Y-j��j4�y*i��C��;�n(�G�����_�|����W/�v�_�|�d�
M����,^*Gx�
��Lw����I��;��a]8�'�YU>j�&0�2�����C�V���<���w5��oy�)��e���zD�������j*PUZ��MK���Zat����m|P�����5����5���R\��v��7��a��hR�
�m�!%�B���o�9��R�*6
��eVTcH������"FU��B��@��6��+&����Y��'���9Rk�Bp�|w�s��g�y��4Q^6�Bk��_������0��Q��w}���������7_|?A{*�0�>�JqTY��WWv������+;h��-H����i��(I���I����������o�������������J��819��&;��Wp��Z	��6�jRo���S#V-�T�T.*Vr
6�"��.�;�LM*�M�%.���|e]����
�k�&��
�J��fGB����x�
�Z`E�L7�B�$MM���V:����\]���������H_�m
��e��p���)T,�7�%�+|o'��:q{����p����
2��������Q��|=�!{BZ_4��s/^�~[G�o�\������	Z��8�n�j��6�0�MD�U41��x}O�
b�2�e������Na�8��p�f��
�J�:l��-P1C���rF�vr��'�'��i�K#k�2gm�3��C����}��;`S��$�a��s���rq���Y;��P�c�������&��������K�k�}���h�����F��:�j1����iV5&]*���h�t�0���P%��u�
Q����1<�q��p���&�L�l������Eu���d��Y�����s�N����%�-Q������ (@�C����_R�D��f�.�fB�f@[o�*�[p&�VE�����l�*���R><�U�P�k�� ��0V�
[s�f� ���w5�HR���1��S�8_���y����6��I4��_6�a�((f3��Z(��K�;��T�Su2������Z��0)/���,�q
JRa����$�E��b���������:G�������x+������� �/(?!I`�,����,�8�
������2�s>��U+g0�\��f�W-�?�4,)�!�#<�"�7���[#z�������A/�~���0SP���$�(����d`�����P
�^��S#<����4fh�-j�������F������+ZX��~-oIIX�>������t>��z�8�'y�o._]���"�z�1*��p����|�9_�\�>h���~}}�>�����g�����S��Wm�����������{�	�|ZqOq��|�<_�@�M�3���e������U����PP�������������q�x�X<�?��6�I�����6�������"�;i��2��G��;p"##9��oP�w������������Npr����-8���B��C��#����QV���4s��<v�\��tp���|{�����W
���p��+��D?MN!�G��j��2	�C?�vc�:=����,I��Uc�`,�^t��2�J�/9V���2��c��e��Pke��G���3��z�VC*�����o�Hi�H=�p�C�/�!����c�q~a�bm�T[c�%��
fiT<�3� bX����(0�S��JH�DR�=I:E�}l� ������G/$��!��  r�G7d�s�99u�� M�)�M��GJIj���2Uk)���ptdB �1��$��}A������TC�#�����*6�mn�gqn�5�H��)W����AE��4{����AY��V��k��&�8�&�R�&��H7`�dlj�63o��h�cW�5 K��Q��	���ow&YQM9��D��U������w[�8Va�,[�is}�f�g�h���4��n��9R6��t�l����L�J�����b��
�[�d�����d�l��&9����M�-p=�|�6����<���6Y a�Y��la�>	W�G+t���z��������M����</������.�Q�������A#�S�a�v�X�zY=j:��3��jQ>0[�b����=es��o�Q��R���7�E
U�-f����O�P%��pUh����u���@A6Y-j�l1��sH�<����{�K���v��d2����23�=urML#�=una'��aQ����*��_�����U��u����~�u�������U_��j�R���k�U����k�U}�����^W5T�x]����_p]�������u��d]
�u��U_����u�W���b]�����^W}�����U����YW}��VS?��f��dL��\<����M������us���2�E��|�*
i4��`H�v~��1���M@p��kJ����yM��� ��Y�.'L05�Cu+!X�<�7�o!��mN`�D+U���~7I��r�������;���WLKn}N<I��Q��R�+�SIS�	���R�(l���2���a���
?b�JIn6?���(E����Z���4vn}3!y<������[�vw�v+K���8��R1I���z/�7���]����]�����]s�]�uk��m���$������^����.2f	���R2�l���G���
�VI�
%�'SF��6�S[��G�|�SV�oC����D��8������\�k����fE=�|�2��&�^���'o������a^�����[���km����yuX����b/�*�[���lmB��
4�&q���m}�0��������e���P���z��N&p�`4����*H.<��y7�Y�N���Ukg����$�e��k�����>���z[��7�Z
�`R�P8�5 eC�f��%���l�7h���6� -����y�s���t��e?�w|?���=����;�~���5�}��������2/���f�/��)[�������Q^Iz�!}�������#_!C_����[@zo�z�%t0�MW��1��V�t�^��i��/���h��B���{���y�H��N/=0p{��=����������>,(�<�!���	{����(n��GD�=�/�7M���q<������gh�]����h(�=}
���������d���
�3�5���3���>�����hUNR�{��[2�F�RN�Q������	�F����bo��.s�N�Xbd�Z7���
���-5F��$Z����n�pC���1�,��Z��p���5���cT��6
X�4 �)Kr��`�!��x}���]�%���e`|�-������m
d�E�*��������5���G'E�&��@�.���qo8Oq��������zoIk�ih����������m����
I��.��-J�J�,�9�z����a�~2_aeN�ig�
�t����H���:�����ZI�t��h�+��o��EH/�����YH�T/���(�Pl<�����pay�X�W�g{XfF��D���W��E�M�GBZ��V��S�e�%�@!m@Sc���[��k/�}�����8f��]��dQb��F�
:�uq$��u�)����f�5.�+���&�P��O��{
���g$����0 ��iTQ
y�}�g:L�*|+&�NG��e5��������h�������a��"�s��c�g</f����^F|-c��E=���uv�G��l�32���g(�%��&�e����f��6��t�z)-h��.��Z�zx��2�&z�������*a�#���)d&��"0�C�t���:-c�O	�g:��~�c
�l�/��w��0EV�)&�(�#��{����c�����2�Ezm��:�u1��JH~����rb������m��]�
(���_�Rl���k�DW���e����54M�?"^���P��^���� 1�x�
5/��m�����i���F��J����JO���}���P�J"w�K�����9�J{%����?*�.q��cfj^�N�#'pp�!�`�Z�N7�O����BX�
���t l7�$�8p��I��L"J�J�`���G�Z��m~,8A0D�����#B����
��@5Z8dX��}�D=i�'�����8��0%@��\��Pn4���w����H��*��$��Z���A���� ZC��=�{Xv�Lv"\����hl��*������Wf��P�qe��iC�l������6��c��M�1�� �
Gx9�_�����S58y��D��c���(c�L Mr��Y���"o��s� 8�'�Z+�{Tdu���e�C+p��!(8����4�u��C�QE���~4���=��Y����Ra��Vi{�LM��
���")Ke=P�h���KDU�VQh��p�U�0���a���[T��<�n]����,7���a����T���Z~=��0:4a���
3OA���������-3~�����P��Z&s�m�j�J9��M�dn/������6��r[
���(��6�k5C�mV+��0�4f��kq3v�6`�k�5��j���Z'p`
������
����l�1	�5���51�
�4>�*|5����Y��*��PHd}G]��_L�\�!���L1�����8�R��}!��pl�W�n��H78��=K�s�����*�/ ��V�}�"��u�a��q�
��I���r������=�3���{�Z���#�y|6��0�v��=��z-��}��~�����"
�������<�����^��_m�'�H�<�n{�{^��iA�����{|�k�g7��*�aag�(�`��e��Y6l�$�jk�s���.��>��:���i���^��y�U�4�
F�����F`���b������
*����Q%�p�`	������K�������K��E�B���D"��s�" �Eb�pvqk�����3����Q����R6������`%��wX�i6Kz�8�t�@�Q�H������o����{�(��"����~�b�Bu0?���_yw���W��qp�c�~��,�F
����mn���>�	����5��j0?]���M���p���{���>���k!A#������/X+0��W��\���zXR�����sTT�2��Y��������G@��\*�!�����<1$�\]�&h���{
n���������pq��W'Qh�!��v�w
�S
��������Og?��Un��f��H����&������{������
�������D7Sxl'�$�����C�>�� ����0:T�$Bo�3����|�JL�7��}B��%�	j|������t�����������	�3}�LN����gr=���x��v��rW<�f�h���	��|m$�}��k3)GA��8T�s;�Vc��]+�]W��R��C����=g������'������iGq�fB�$���d����F��}���`A+��q/:�)�G�;��6�	dPN
��Nc3����.R�"&h6*��d�T|���R^n+�G�R�P�����
F<�2����,Y�
���+L~	��]�����(	�Q_`3#\��rV���#]��A����~�����s��=y0�}�y�=�=n[�T����:AB?�%���p�yBk��)0�J��bV	�-�Ru1���u�HX�U(<�o�����a�nh"�d�a�&���F K�����?�
�HCym�n���3:��J��Se~i(�m�
3^�6���WENN�h*L>
��m@�a�+hV+�u���(�_*���'�1a3��'M����r�k(<Q`�n|
`�-@�m.-��W��5��&c]l�l�n�m����h5��(V��5���������Y�k
52�=S`�Xv�p7�W�Za�:�k(�u�3�;r�����h[n�.�$�,H0�3�S��a-:-;���i� �\�����v�]U�%18��q?� ��f�<���0,�o2��+b���iR��m���(�X�4o2����|������(�;�Pe��bI�l�E
�l����-��^T���&@�R)�Xa��-��N�������YM���b)JvLS5R��L��"����G^
�m�VGM�LN�R%k��9%�S��t\c��d������p�b���>�:v�
�k�4�\���Eu^�5	����K�tr�`�W�#�s�[
��%��@��<��z���������-+��t�_��bXN7�
_��C�"��I���D"��2'��._a#�Kl)#��*�}�K_�78��
��&�f��43:���@oe�j����d���@n��_�c�)���)��m�������'��q�����lNj�^I���7�1���Yj��u�0��dl�R7�,���z�U�Uy��6UL���m��QPWq�]=�U�D�����\� �L���Q�n�J[�K�J��T� ����UyT�U���n"M�����R5���]�h�X�/h����B����D`\�URi��Z� ��"�t\:H�w:��l��]��;rE�����A�>�3���P���l����;��g�c����S�b���h�V���t�a,��9QY����kH�t�n��A��"G����8Da;�]z�3�Q����`o�(5��2c���QK�#Fm3!���u�T_^��[�f-��-L�-�,��j+�mC5��<R����T�����-!�j���|�VyKb�����j�������s�aw��s�f|����~��=f������m�&����@%���~
U"���LM�fQ���{��l��f�����������:�������@�R:r���t"�w\�`I��7WUO��QeA���j7s���h�(���j��
���B��z�#��Kj�V��S���0�]�����i�����|Y���9�f��ChK%k7�a)�x�,p������WX�U���X����._Qw�<]-=�����(u�u���|r����������<2��lh4�|����w����(���k���u�&i�0VO������&��@���������dg���@s����9��3B^=�ms����%�g�x��=K�r=g��X�y�Uz����P��[z��Y�A���S4������{�'r�'wJV�&��G��rVb��A^��?��7������x��J�����������p����Z<A�D�.`Dh:�,H:�_�dv��#�6kMg��<�ZGGu�x#���oo������
u�"n�!��HW���U�;s��l���H�Nk�9�������35���Z�BC*�O���P����u�G"�2Ke���v2�	�I4*��<��*��w#�/_\3�E���#Be<�U�4(Q#<���Z��I\��\��N�C��������3��v�������f�Q?W��G����\n����K��G<ubeU(�i���Y�c��������b^$�������CeU��N�S�5��������(G����q��.�34�>������/���UI�-3&S�v�j�)�[�\�v�T9��(pc�e
�������zSk�n��3��r�[
3���e2���#�	)k]6iB��H\fj�;����e��Q--;L����l[#�nq�E��|��������%��YY�WWxa�����#�wx�u���F��8I�b��E����`JU��`8J�&gvli��lMZ����G�-�(�	n1��!�gp��R�j�L������~b�ng�9:}�8m��5�v�b��:=���"��6O��������_U
����
=t�t66��5��#�s{:�I����Q99�h
%�������8��D�����n�����Q�\��BvKg>l�]X�J���.[��
O
��������"$�lkJTM�z�����-6'�*�x������y�;�)y�XYis��|K��:�b��4e1��+v��*K�fK�8�bw�Aq�#@������ $h���x���!��c7��1F:�8���s@�9^6�� L�x�^4/��������p� {�!|������_�}��;8��~�&9x�	��a���������{p��B��!dL	������(���������"��n,��3��,=dFs��I9}Ox�Cn�3i�
gu�\{� �.�����*3��n�#��s@�G)����+��S��=���
8�7z�%�?����A�����!�z���n��G06<kl��<�0r��d�
SO?�f����f{+�=	;cF���+���_��~�@nn����w��O7����>D	45����@��^f�����U�u�S���g���M�f�:�J#%�Dnb#�����,���y'D(�x�0��(�H�y��}��i���Jg���MX�^���k�[0�������<y�ym�� ��#�W��|	��Y��D;x$��}v_���=/�Ig�'����!�I�����t����\�������p���8���$��������;�����7���6��6�F�7��q���cf��,p���_����7}2$p��������> ��t4��D���>�a�`(6��xp�/8����~"i���Q~�BOD�=���icC��A|p��a+G�B-g$�D�n��f��V��bo�wx|��p�j�����p;�<��Qc|�����p���8�$Q�1��>:D������������Wp|�����G����O]?8�������4{��U���{�z���������k����n?8�����p4��D���Ix<
�~���E���{����1z�z�^��#�����l�f~�����B�$�(�G���"h�{@r��p���c�U��~���Q�;�^G#�h�����c���������`������3h����{��{	�2q��C[���|��h��a�f~����a�,t@�(w�o���f���Z��/��"/�z���@��|�H�IK�0�C��(�@b��h�l�G��!;����k�+O���\F��b>A���D�����?���q�������B�)6I�#!��[-������|�}����w�����B�P�G�����V���;JW��� ����L=U��Vg���8m���a,m�fS�j�
����`s=hy�r�|���������~�����z5�=���<8�~��$;��s�����l��D�=����G �����$z����X��;�UBo<��������s����=(���Q�����|o�6��`�����wpp;]y�s#2��kM��������E`�:I����T7��P3��O������>��?����BL�_�#�lq���=������t���4�E��i_�	l���P8W��f�y?�X�����^�������x}]�� �G'�����F�[������6Z
)
�`�W��q� �����l�X�:-���z:��8W�x�+��3]����A;O����t�A��/@�0A/n3��3�{'mn�6��5E@�<3LH=�5�6����U����<�#l���r�`O6��x�h2����
�����I������(T)���-�E
%�(�����ZP9�����E}����N/�|$=��=��>�Io��3Z,f`�����s�YO�?��3�0�>C�=��7�>x����cv��
�-����>�s��g���mb�dP�����{������� (�����~>��!�<�12������������'h���)k�� K�����x��v�����b�+a�&��I���~��w�������r�e�ZNoo��j ��Z�����C��m�����e�Z/!��@�7�*����`����3W��&�|fy����(�@,HY�0iK0�4	�6��fpS��j����x����M���$��o^��m"m!�Ft~" �s��������~M��WQ<F�qb�lp6^.r=��s��	�S������@\��h�+X�~�]]���7�����������WC����;����
�YC����5��=4�0�a|(v�J���G�6;����P,j���=B9������	$����}(��<C�x.cJ���t���5\�o`�	:,������'�L�$
�>����d�]��B02t5���t�U��j�
�����t��JE#Z?��*S��1�d�A�<����K����E;����u������m�[�E`�C5m,�6�c���r��������	I�.m�!�c���K��&_O�-��A
^~z�������{�[�%��V��z:�����/X���WOS����8�g�
af�f��'�����h6Z��\����!lf��T�6~��$Q��t�LW�&�3+�x���A<�(���Q���������9���
��pNh�L��l�@{ne�!|��Ox:�n�K ���/�	��@`|�4��[�E���P�"�#b$(&���K���x`$>���(�h.`>�#L���V@j�������m������~:���;M~�,`�����6�t�����A��u/c��e7�����f�@�U�B1�CCj�	~�oa�)`����������z6����s�C�4A��C����4h���+����@W�#������F���ObgP���E�5����LY��h�g����������p��j��?�z�O�����&f����&���
W��^���3�Y���S����O~�N?�
z_��3�-���52;���m�+��n��+������N�]�x�������q6�� #E��rO����h�j�?r��0��]Y2W�M����7[��U�`���J�X�[�?GP��:���[T.g����@��7��W����H�+��q�?��OPh��:�F�D�#� ��o@S=�&4SC�S;9�?��<L( bY�S��}����q�L��/b(��c��8���o��+*������D�R�~��I�J����n�"�V�k
������6��wgp���7���	���6�������H9��iK���~"�]��^�P�o����fO�K�������Z�;�� ���`�3�t�6����C�~>y�8��@�������w��t�L�
q���@m�e��YP�q:A8e>s�0x.��)*1���fq����)� ��Y��Q�r���/�XOv#q��KB���Tt��4���H
v]!DU�n�x��M��T���<9G>!@�K����������j-��	����TAxB��tJ �Q�c�k���R1��3�D�c�S�0��%���I8����)�N�o���=OiC����Qdt�A�9���0x��� �uL���)d��
|�T��Z���52O������
.���\^�����>���<u��mr���,�c[W��07oc5d�(�UZ��g���od=�A��P���+����=�<|����������p�7Y|���.7��P����;DJ�a'a���w�}��w���2������/��q;�Dx�Kh�P�Y�S+^e$q�g�<��
�%�q��j+�2 l{�2�������m�����_�kY?X��2����/�F?�h���
u=�]�W�o��+��� ���$��IE�!5��9��Ck*�c��0���`��h<��
��u�j�^�|oC����#�-�&���z_���%DH��m1�i���C(�"��
'�������	L��'����?��q
�y
���7�Oa����k��j����uL"�
xr���L�rt!/XS��/�R��I@�^�NU�X����f��8��z�d==�����6A��p��URj,b�)���z,�Si�bs��!�H�u6G�����QOp��i3Sa��BF
>�IO�n/��F���0�R�x��C^��dqc�ai�Jm�W��=��<�w(�Q.���y�J�����Ld~��4���/�O����+��DM�����!>k�#6��@�p�CaX��X�S;�������]���<z�[�Ltayi������%��1��K���_��vt1��P��{�C��
�O8�
������U#k%?��-�! ���������Y�3/�%�U��(��
f	j<T
a�;_�?�WgNb9�zJ�=(�b����"hp�L,a����,��>�c���r�xO��	m�(�g�?��nopx��'����?hH��#H_ ua@#�p1�6?����/��iX8��y<�.u�gt��c�)�f����?b�`�[��)��m����GH����;/����U�'G@�%f��yT�p^����2�{�J
y�7u��"i#^,�����wSC6�P�&z9�W&y}�M=�52�D��``�Bs���B���4��Iq���t��cs��/oI�/�WE`���	`q��r��,b��|�6Z!����,����p�����>� �v�'���F��,�y���~���������1@JjE�W��M�5{{���,�S����#o�����l�sB�Aw8<t!YO�X�T�%��%��
j��M�~����wXBo|g�6�����.Ja�����)D�L[b@o�{z��$TI��A|�P�w�D�GL�pN�M@:r��iV�I�.�#vE�Bh%$����������8B��bb"�H����������[<��
��M�`o�&`���f_����d�C����c��Jb�!tj�� ��RD��t���8�����@�b
�-���g��a��B$�N`(������������JDF'^�>G��a����Vx�gb3�,b%���0��wM���#�j���E�����M@��F�}�s����t���<&*+Z�'��S�,���� ��jt(���?�����1@�)������I���x{���>��v��f&��'��YL��k����k��*;7t4LGk��|?�<I�������2=��C�M5/RO.�f�-Q�.��AV=q"�X���C���3
Dy}�@��6g����������������g �c�������Q�9���Y�{&�$����������`5*:�\��]aY��x�iUmH����I4FE(ac`.�
�/0u�ch:B=`�k&�j�m0A@7$�+R��kh;�C
�(��[��=�+VS8�����c�(bfN#����i7'&\"���qY*i��wi��=��=9�.����D|��L����o��W��`���`���8���WB���1<���$��"�;�b>���>�T��'0�B� ��u�����(1��,���*~�����Z�ND���0����'�|b��=�rD���\�*{��D��C(|��.]~%\�X�!w�(t�F�2��	8�E��|>J��b:3��{��H���x{����
��z���X�<�Tc�	Pg3��/����kW_�9�J�-=���k?�a����bm@�����FZ��!�r������ LO�B��Lm&p�������`T�X_��y�����3"X$Y����F�G0]!{��'�	����'�����$�%�Q�21G��<�~���v��a��%���':�#��	.l���|�*Z��$����T�?�^k0g]40��Kf���%�kE��Z�3W���c8�o�d��K
��+������gQ�&�U���}�4kW �a�|e�)�2Aj1�g�@����<��}N�M�kQ_I��������r�/����088��c�PE�D�hr�Z���������$&Me���� ���n����?l�B���h<	'���7�&� ���Q7�9���z������dS�����?�����Ci�j���c��+V�j]�l�M��d�@�&F�]�����	��B��|
�b����y�V�����i>�-:��|�2a~��z����o�_��m
����!yQ0�A���"j�������l�E���<7D����Gj"$�<�h-B4�����@A$�ZAZ*H�����nn@+��?�3BdW-Doc��	�@����(�K��	�"MWd8b��9�C�����	�Hc��1^||&ht�[����_�$�����/�������C�����:���e�6Z����U��F��`����(��lk83�a�����hh�KX�-N��l����?��z�TIv�t�[h4Ag�1����]�Brl��H
@������n�a@c�U���E�X�.��� ��C�<�<)�!is�s����C���upp��	=R�Na�b
�3�!=2g�S��;�3�h�A�AI��(D&�/P.��&�Io���+!����E�LH���q�vM�I7�4����;��Y�']���N��5�7H��y��<��� cC��~Ee�6|�-2��C	�&V��������8��r�L��P�-�'Xe7�Gh�UT�	WL#o�Ed�D���K��@��H<�����Gp������|��S=��������h�0v��&������~8����=��U6&�~�B
67#PU�7h{�5n��aAe��n
��0PH
ll�Y���g5}��	
�@x�Cv�5�q��-�,����K����4��~ �L�ci!��uNp��@���t�Dc'��?��Q�tF�?���i��`�!��q�l���:��8~�G"I�u����h$���f��y���-<�C4��$������2~��}T[|	oo�`� �R�ga/�)sp��5�bZ�I+`��H�""=:M�
LWq4�I�=�W�$l���(�z4e�7@��R�-�a���9���D�j+ ��
aT**!��mq2�|{�*��w�%���_�!B uEj2Z8\�"�������j�8��f��{?"R��T`Nf�����<!� B��e)�Qk�y����bJ���@�/*�7�e������XB�����@K�S ��c
ER~��J�yrD�jz{����`�b��$ *�����$s��s(#.�B7L��xs[�<wO�� �VB��F6Px���E���
������-���a����OC�I�o
$r�,�>�T���L�Uu���$����TS:n��qKpmd�������7����Bn��\R�����6~�9�	����)�jy�X��������'��K��#qv�Qt4>~qxxr�b�oL�t���u�g�-���E����������CaT����?�t�L�@����%d1�'`�3zd���@`�����
�``(�r
E �o`"��Rw��������T�|��n���"�2����V<�`Aa|�E�����\�ggk(f��%��(��.��}(~G���P@`��i��n���.�[R�;�z�n1A��T��.������H���	KZ��Cdx������'TUxN��\�S{�9�������?�H��I���F�����8��I����h
���F�=�h�A,��K"���K7h�c���l������&e�\�+d�A���|�F�4�\b7��4����>�V�{$p!����
�
L��~Y���9������/8�!4"��=_+I�1
��$�8�a��L�S��xA��c�������Y���c����P�Y5�	��(��-����pR��Y��^��7�8��� hf� �%CZ,���X2���4b�dh������Xxh�y8����y��<Og4�@��<�HK�Q8'�G�b�L�&�Q������a�/�RQ8w�E�����^�{��r�
���7�f�9����'H�6��1�����B���v��4�C�i������������������ Y���3�$2���`IV=/o����m��_�LW��<8,V�-<T~����������\5��w�n�q"�4�%�]+��O�y0~�o�������zF��V��,=��>������0U��r�=�0����R�euH
�$Y#��������*s�x��/�4\���D�g����k4�$�<��D�Z��9������t&�l�"�e�I�F/�q�/f1�3��,N���V���>�/fo&_��y�@wh+;����_���
:0�}�\H�
[�Z1-���{�G�8l�]JP[@�=�0���������7��p�6i\7�{��y�e�F:�4^V�Z<����c�0.��o�	�j�������^F9=�8�=�����Eb��R9����tk��+'��P�N�C���d����
A����R���=�k�[���c��6��X� ���h�di�D����lW+(E�n�t'\�������Z�� �xm~�1d	�+�����$�.IK�Y�)�D�:��dIR�2��4�))�W3-h�L�S���I
�7uJ�:(\
�d�dY<e��jvQ!�')���Za��S�dh�>��������J;,I�(���jXH3y����9,�����h�f������Ylpt�'
L�}����@w�
�$�7�H��J�������-�o�2��"=P%m�#
�
��K� ������X��������>FP������In;~������5$B���N�M\�
�����N�]2y��$��D�,2 	�K
;d02��p ,�A"	�/��a�.g�o�SZJ�
���Z{I��IfF�!�����C5�i��N������x���5�oX��O���1}����5�f�D����`�D�G��Q��1L��)��H�pI8
��g�WI�'��

����b������d���t�&+9�4s$:�ix�C�q�O���yxE��C��!=��G.`�
t�k�*���Y���.����!���y������q�Pe���1��4 �zW.u���������i�������A����G�c�����
���������������ht�b����q�7��f����������.���Pbm�`��y�}O}j����f8�)�]`=n�3ZN&��E�&>����[��
o&_�G�����9��s������-:�
���	 �_�($g�
��z���c1��>���;t���)�K��`��d=�<���x���q���J���7{J����/���H7����p�K�
`�����R�kZ���5B��%��,_ ��q��(z��� x1	��Mb���KP���<.
�����/x���
�����?�$��=��i����nO�d�v�[.� a�����?���':@�+��.��W����k/�'Yi�9`G�C^.`����<9�bt��n�����F��zYa]�=/9�^}uv��E;�S62|��}�����!_|������k������!r(����Z!0���A��=}�
�>0�%�Y7���6x���{�T��_�u�8��/��	���_]�������x��{���<�����O����=��
��.���������_���QB�p����NO;���������xOW�u�t/<�<t;HQ�����g�.}��������d������b��MGJ�y6�u��C"#���i�fS�*6
���W��_�����W��<c6U�����/~�Y��,VutO���v�s�[��������>���#@��\����z�ag��:��p�������%��?��w1)��e��%�;�������~�

�=���\\]�����������z�s���;@�+r�?�.����.��BG7s\�9���:�}����p�G�������I����*]Q3��^�������t�� ;^�hv�H�n"��
??��>���r5��C$����8e;������	�	�\p�7�_�'45�[>���[p���_�'�}��S��������}&Ar���x��%b��x}MF�<�,}�'
�?Y}OAh���4�������C-A�<O� *��������HI3K�qYS��.qS���i�fA:�|4�~��<I�y���$��3��~�[��p�C�����7a:Pr�(���d�A���M�YP�$PO�6D����?b��o��
��I���U�������6�<�i|�No�Zf�n�V�v��=�Jz�
6��n]{�W�|�>�1@Op7�`[��������nH[]���b,�������"G���(Wgo����	��t�b��
q���������&,��{��M��D��$NV�deLV�d�J~�]]d��N
�J���z����
&/&����M4>~a`V��a	V>��>��������
��
y���<Ex��'?���nq�;[1g"~�Fbg&vf��0;C�3����Q����#��5��e6����w���c�4�z�#�n��i��}p����a���&'�H�y>�>?����h��j��.Pi:(��O���,�������kY{��Xr����B��@�U�M��G�`-���YO�1|F�g
��_��8IWBe��?�?�	[U�Ja;W�V�����5��&�p� �7��J�h"_dE������xSB�a��}�GY�9�}��?���9*�_$Ui�^)�����V�A�3���L�hGd"a�S����������$��`�/e��|�9_�\�>������&�K�9���$�P�WmE��Ch����"�]��6�h�K�y���$@�~9_�'?��3hnOz6�1�
�b���+��j�����}8�~�z:"
�h)6�pH$���7���P���>�{���Z�g1�hD���m�����<y�$�C��	���:����`}��<����������K^8H��B�Q-G�E��eB]���3�{wuez�|��u����	%��<�{���r���x
M6\IyH2�R�U_��-rV�,zh���1Xk�I���E��\�&+��&m�W�@������@���C,,*�;���
��8�����[=�<���
��6���
p>~�O����� ��g�.m�~>H�����i�}�L?���(��A��??�m�{>��,���IiA��z��/F���;���L�(*
�h�������.��{%GX�b��D
����#s#(A�����&�Q�z�X<xo���A�{� i�o��u���Q�j�w�E2���*�-:M�����P?���G��?�/~'���w���	�E��g�Jv�����'����h������5�x��?�X��S{
���uB�@��
�6�!�s��qT��n����4��T��t
.F��3Qn���a�)�:"H�$0'H�I��:�������F,C�3I)t������Vz]��vO���>�<IW�����W����I�S�o[��
����~d����A���/�����U�@�7Xy;	aJC�"��I���T�}��n�;���.���N!����-�����NnD^5��D��s�	�GrJd�z���W����<Y��m���L���
�}<��ZpB@�������oA��X���4P�4��i�EMF�c[�Y�E�^X�%�]����F���m�Di��!4X
w(&�}�\����z]����,sW+d��/$�*���=��\��[x���@�4���.ww����A	P�E�Jm��������g�C�u�.�{�t����U��� ��B�o��j���b#���5��m���z���}�U��S�-�j�k�W�uOq�;~��5��x][�Z�}�U��S�-���k�7��oq��t��R#A�S�� ���x@�M��d<,�0�9�y&g<�C�n����t�	p`�Wx����
�A��@s/���%�-FF�z�(�5 �V�������&���F��Ppw�uk��+�P�,�Z�$-�i�Z���f�W1;e��C��4�=��*��g�����Y��h=gJ��|��G���(�U�W�$#�A�	��9K�����U���T�����Y���J��w��w��_0�M7��h�L�{��f���FdZw��D����0c9-�X�����\�����P��gG����%���=E��PC�����(�P0��R��X�����p~�["����9�����|$���~z`���S�X��a�4I_���^{���HH0�?+��:��gz^�����9�b�����Gag���������pN�lD���i����C��l�������k��6�����,��f����x�����	U���|��Ul�
oo��-�x�%6^���c����o�?��Qzw��"���k��/o`��z=%���i������:	��u����*W�j�;{*\������He���d���=�I���Y�&�J3���~=i,P�]��>*,W�|�G��
����J	�xj'#�����W��;����/��|�Ea�j�H��y���g�u��m��(��c�Y�u��!�)��VC
��K0H��a7%����&��4���|�M�W��2K5�s�2��E8��6�9mAH
��4n�
�d�Sn��"O�r&�c�U9@�E=��=��L�������"�R��W6�bN�YR�jK�?�AkFQ�,D���$]<�]�W�~����~"�
�"JV�<E?q�y/U�X#\����D�����y4��v��&�;�%?m����+��=�v���D��R?m����W/����v���'��A��@g���a�"�U����|j�-�07�tn���@
�����,��Q�
����YJ�&bY��@
��I��D{�������
�	�F���n�%!d:�����E;�D;X��i��[i�2�V���3k��i�2b�V��K��e��![2-�1�iY�5��6�J��)���G1������@\*�9������5���=���2Bh���FO�SL�����'��C��0yB����N����O��vr���)�<�������n.����F{��=u��HY@ta]B9R?V��WI��(��j+��.|�;LW�-Z99B������$��Hb�-_|�7y+�
�����HY���W��
u���y���_�+����.+yE�V^��"�B
-���H���p�Pm�O�YX�lF���> ��!��C|r�"q�ecq�������m%2����p����vB]t�0)���rdc�k#�h0�* sK�!���-*��ubC��\1������iM�
�4GD34!Sz)�V�����G����k�@�n�q&(�/�ND��s��9>\	_�_�o�X�e�n�{���dyu�M'�l��~i�6���Hz	�,�J��z�g��PLT���I��R���WO:m�@dX95���4����J�JQ9�)�9��B�b}��=K�	d
��Y�]�WA+E~q���B1r�����6��`Y+������>����+�, ���/��x�E���Lb\_U0�������T�
L�6d�������;>g��Ea�@Y;A!a�u
�`)A�Mj��T������DM4��_tR[��Y[6��3>{b���.S�B�=��C��J�W#����A��+,��WW�p9W�����1ZH����b,.j1��n��k{� �U���-W���2���j�)Q��MU����Rn�W�|!Kj�RU��E/il�/�C-:f��b9��������)/�5l������d���L�������b�"����O��:�?{AS����9N%R9��sb-��Jc3z��A�Z6�`22�AcuL��Y���V���5�Bc��c�8e�
F��eaM�, �5>��K��2�(�0:�2� mn��:���{��q������'C�M����.d�L{.$�U0����*?�b�B�l,�w:����ugv_'�:�"�Q�����7aF���Y���3��)$�O�=c���\B�qI �In�>���>�%x�e�����O6������Z�V�p���T�-oL/Z�?�L�A�-���P����#�r�H^w��\k��h'�Rwx�����T�`/<��"Z��=J�hd�l�l�s|�+��Tl�k�Z�]�lmx�D��mX�<�`�:A}lmE���:�:��,�z#�u�d�@����jgk��?��lO��	�.`��������������	�HW�����2�zz�
��������_I^����sU������x��
�|�
_��*u�4�J(�`��1�q�.�)\vjK
J��7�(�����He5 ��(b���������b�@��M��V�E�Y��_l��e��-.�#�*�
2b)�M:!]���k��!m���F��{�~u�����_�#��p�}j��'�#7N#|'V����������/Y��q� ��l�A)�{@6�\�<�ve�]�z���As��%��+5���-�U&G(����&	0�/�[�t��[1��I�|c���KK{��HY��W������^����?�h��Qa���UMh
��2v-r\�$\|��M+�@����)�a Q����F��@�-��c�O��o����t��]���!m�-Zp�^��vBy������s��j��'�������t�T*�)��l����V`zz���6B�bH�k/��X��`2t���{oqss��8o�`l[��c�������F��Hz�9����k�v����������=Jd	Q#"jj���n��k��K�.�h�Ca*��k+6����	�m�JO!*�L}���b9f�q�%MKf2�zK����Iga3c&�u���ME�i�0�B,�L��Cc!�����������<E��rC��l���m�����0���%�(e��00k�Z���T���C^�k���)�.�V[%���,�E�����*�c#��r�o`�1,�Qx���W1��[^-����u1�QS����4�����'R�K=��=��	@1k���k�	Ew����jS��*���G�	e��)u^>�_��P���L��~B����2��uy���'�N�����T�O
���3��^5}�E��Xw�����!��rN��W��q�8��BG�x��4���i(ZL���$t<����s5�&��:������'lp�W�+Ow�(��
�����@���W��E�����l�x��"�:���������V����Ij�*��I�`_���|~�ST�'y{��o��yW�tfrm+������d�K�����:��`\�5�M~���l4�-��9�6��>gh��dv�n!�����gc���.���d�2��}-�}�������s�����L��d2�-��������u�����mU]bZ�qg��Rg�����w�_����_���hOu�J�x��hOqs~n��@���GS��=�������v�H:�F<���b������%VP��U����$��3\y��������,�������`��l��)�{�v<�7 ��o��T��(�j������O����+J��v�H;���	���T�Ha�<�Z�w��=}����!���4	�����������0e�v��L�������Q�VSm^����a�V��0u�;[-��m�s<={���g�k'1;�`���8�P���U���m�n���"�y��^�|E�%���d���p5����p{NW��~�E[�^�
���������q3z����1��S.�p��[�<���c�����>������_��M[��.�QAJt@]n��C������:�O�2_d����2���!�E�G��JN�/�I�Y@��biUQ����S���_h�
��������{�2��i<F�@g�CG��:�1��!��f~
��2�q�7�QEL@}��@.H�H�pJ���nK���6>U��nn\����v�v����OT��?g$j�;V7r<P��i���`�p!����Jttm���6�=5���^�m;H����.����}5M������6LF����*��m�?�m��������m�`�0&P�h��X��q��
#�j�>9��}���m�8��n�m�������hiS�=P�I��k���9�a�jj>�i�e����m����(A��r-#���n`yho���'{���h�E��o�=�$�&9�1�X���g�pYg����{���~\O���>~���/�V�C�`���p�3	;zt�QT
�);���^��N[W���gNW�A*N$E�VFb����&%�,��+�d���oR+�f(����^�l�"�:�?����g�e�����"�f��hMR��62o&�?�KhhVG���f����"�+	����
�S���v)|���D{����p;����E%������mQ8����1����]h�h�d5����9�ae���u���9����{@P�� @���	yTN�~*����l��T��S�/8|�T���=|�TPR!*(W/_;�"S��N]s�S�g����SA]:�����������:�yc��-�;p1�}�=�=��Q�UD�XQ%�3C������3q��t�i�q9���e����b�,#3�.r�l�9N��J�X��pk1�u8�HAD��d$�{�Z�3�c&��~S�$���/��4�$^�Ht��JG: #������FZ]����,j�����V�U.�����l�VE����Il�O���]69��V�����w��j3��r�+��(��)(���|�g*��r�����z"O�;�`&��wg�r����GB�QTv��8_��,���;�9=�
2���'PeX�.�m� q���j)Dy�\H)$�
s��B���;o�����B�H� �Bn�.y���O���T�s{�.��V[I}�������y_��^"����mk��#!<?P�-zF�hV$k�/�t h.��:�%��B��������l���
{p���vn)SEoY�=�e�)D��M���(����K6��D�R��#l��I�,��D	���pr��[�5B�&�P*���RP
D�F���G$�ZH�$;��F�(����f�dS"cZ�����/$c�Yr��w�;�N�������U���M�e9�&��d����Qq�+�:Q���v�L�E�DI�J���*1�N����x!N`W��A;�7��<�y�\����N�q!���Ej�J��:����o��=�H\_$��$��#���)ip��54EUF�@��@����s���T%=4���\l����Le����wm�v����Y��O}�$_����AC&����4����gz�����������'O�x��p?���������������,
���|����s���A
}-=��L��:���1-�P����r>��m���V�J�O�&���=����Z�5����r�"���%Z)�mZ�e��3:����J(J��Pcg������^L��B<@=�>��
�N��2F����0��5_,W%�n�3Jyo�>�q�O9�[!
����2������p���_g�g�ur���������f�����A��6��z�r`D������S��1���p���_���_^MFM���G���$��g�C�����T�'�Eq,�VMf�Ip��qs��K�G��19
����j�e39M5�
����v�[��Ow�������>��D+z�@���y������/���iE@��1eBM����**������t	���0d�gy�>&����qs�B�@���U���%��a#M����h}BA(6�66Nj��C�|�������IY�sX�o�9BE�ai�s�vI�����:�s.'�=%�d0!�)<IE��:W'{*[D����\���.vj����%z��	z
�Z��E-���vEK���8��/U.��/}�������+�v���!PC�=}#�j�k��	�>��KK��sZ5m������!�aF���z�������C\��S�t�|i][�)�������Z�4��G����:P<���+oy���lM�).���]���(�&���hYM�&��"�L����l� *�(�;�E�+�~H:�N�c4�H�eR���d4��8cR�xr�+�T�#��W 3��'�����s����t6C]�g�P�Q3���*�VyFC�<
r��=��XH��{e�
�r�?;� f��p�#�{��1�qK*��g������o�������0@:-s�Y}��9���}�K/a�r|xj�g��X�����W��;��7�I����a��)���QNI0�&�8����<�y�m^�7�� `;���u Z�R.G�2��Q/����qo�b�7�����<mG7��M��}ZV�^������+�9h�^�[89�2��uo���s'Kl���T;��aM}�����`�b�������`qs�XN�%���b7��b���l������l����-������w���=���-�L�g�9`�<N�<f�6����,��r���KY���|l���+�!t|WI�*�O��6a��q����8[�+c��)����W�1�m6�y:9�/�1����c�;:�#�~
t����A������
�"k�������+��C��O�Q��

�-�E?L^L����`�$Ca �8�m����7FO3�b�r��{[]��/1�@W���"g��kd��P�*�����@p#�n��y����s����n������Y4���h��E�������fE��������X��-0�F}�����������^7�{����n�W~����t�N���(/��K��?�P���bw@1�E'.n<�
>#Xx�7G�h�~�F_b�a�\�����hz;"�~��W��IG6���y���7������CD���b<s4�`�;�������I�
��5�,�x_�g�5�f=C<���{���4^,�
g����?b�.���!�t1A���d���0�!����>2���o���B#~�������a���d��[���O���"�x
D��7o�G|�<�����a��!���h��D��a8M��G9�KlD3�����:~�	������j@���!�g�Pb�ZNoo�%���U�b�������s�4*��O����f
{��$+�`_������]D������9�X�R�	��t�:�� SX����p�D���sn��^r$������sxG�~������7�z��� M�1|arw�k/B�S��o������&�G1�4*����B"V���0��T��GL(�Q���T�W��e!`|A`|s���/��I6���,���>��������8E����U4����8��aS��(�ox�g���-�/
����(�=�{����N#�EG
�T�&NY��,��������*9�����-f������d�\�6k��U��dH��:`
N�t(����������������?M+^0�.��.t�-vJD�s��� ��@!	J��(��C!���:Z~���8l�>�5?-#������`^�&�<M����=�}�!��@|��h�T�73 $�����]z�w
5��D���wh|	
c
���~��hj�Gi��\��#�����-��������^����Hv���#J����R�z��D�'��W{.e����.[7�M�W*���	8ZU>F�8#wX�qs������t�&��~J�n�7D������oo�00�Y=��=Z��:^�������w�4�������_��V).=%"��p�����`��0�v��%s��=6�_����������P1YyqKZ��"��D�e����=TN�Z���7^��(Gm�+������l���)�3|�J�.-:��������O�{����������#{�w�N�V��gEm��V��c��p�� Z��!���OL��$��Hb�-_|�7y+�
���t�Hm"������[|5�������E�R��\�-�����E�Eu�lTJ'|�dD���U�}��a��
�%��:AH���/�7b����iFV�}�P�/���`%'_2Z�K�Fd�)�6�JI�6����O���������AU���Ba[���"�Q��{���f��m���@����M�����S_=���g<�HJ:D[dH�*rC\+_R�`^G�MQ��Y��e���V��pfm��c�B�+��m����nt�s�����K��j&�&`D�
C���hy��b.�.�������S��9xsS���x�<^N�j0�����9��i�aJ���!�!������6�$Q~E�/V��/Q�vh[��F���
���$��c��h�����QGVu5���~�������+++����38Z_E}^�S�uj�����FQ�P���Q\��h��ICUj�Fq-[��c�'
U�o�Y��i�F��
�Jh��t\u]���K���*
q��J�K���R%wY���kt�g���JG�+]�oITKD���O]U�y*���_?�O{L���}��G�����1���7g����s��}uD�
��n��[=�n�+��P�+�����~�~X�Q��<�<���E�s�U�nKq������E\�+�n�d�=,YHUs���.�V\i%��V�������h��.��i���0-rw��[P%�M�i�6T	{����`������g��rK�.��(����	�T��~��^,�g��X�.��g����"7���d������a�B�U�)���^����>�����}y����_�
����'��:��5��l�.�>���I�$�����`���v��nf�1���#|�=6Q��|�g����|���{�{��e�|s��<.#G y��PU%0y"�`�KN�vE"��a}��=[+!�d��,x��S���.�S�����kw�����8��!����|��Gz�W�%6����fH6�I|3D!�m3$��!��fH*7C5���!�o��,��Q��Tn��.�!��U��n�Dl��=��6CL���o�6(���3Cu2�o��)�+��r�<Gz�[��|)ks�c��1w�we���:���#�����.�|�N����4]��
�z/7?r/�m���eI�����le5�Q��������H����?8��v2B���H�mb��@b�mj�O$5�69�_ 9�6=�'��O� �?�[���}�/�2�/�Z���������?>�/u���������z���s����;�R������������<x�aQ�@��6���k8,�M�Pn����#��}z����_�����g�N�����$����v�N?S�?b�<�)�YZ\/��"�}~V}��>�b��f���L����4=��N��^�p�]����������������}�Z�s��J`
��\]�9R_��
�K3g���;��l6�.
z��d���
�����l�P���
��vr�i�w��~�����^���������o��^�����[�^Fh��9���st$>o;�U�=�z�i:8S��YA��e�dG�<A�9�|����lP_�y~3^&t6e��}�%^�#���{��xP4���!EXa�ll���e>����|9��h�!������p���j�i�z~
��I([�j���/��Z�e�����0��
G���{��O����YS�,U��T����]y:������zD�*� �h�g,��`q�S��P�������H=Qm�4�?�&����:���3��v������*�1�(:q��M�f�V{��Xg\C�^���G�2�36�M���8�j�b2���?�"���+S��f��������b��[���z�P��p<�d8��\fhw��g��*�n<�\�qB��0O�R�Q��z���z�T�������b�*��a����a�P�s�����u���z������JCU�o�*�NzgB��y�k�;{b<���l�^��2���w�s,�[g��L��{M���D}��y����$��nU��N4L�f`��`�����' ����m�	�Z�v��26��.�
�CCX3I�4�/�aX��l%���Y�d
����a���t������{8:m�����M���h��l����gHP���S�wPUc1��Pz<�5]�q�(Sa8���A����2<'
�tDi
`r����S��I�a	�lFs����P0�A�E/v���	�f����N�08J�v����z����WN-������%�8j7`��O�r�`��!]�$"O�n4��y�����h�oGW�g���������E�QpJPz��d���9d]�U��~�
�t�9��l�w���Mz@.z�I������f��=��������T��#IQ�����`�d��G�]����
�v�
E�e�j7h��4��5+%�R"*a��7�S&]s����������{?�ug��&&��Ss������}^����PJ������b�f��i��^����U�+r�Y��b�x��`W;�`�m�#��m�t�A:_�l/�K�YYQY	~8�
=�E�s�]���M�����zr����p�	��k%�$���S�$�2�~��:�QCA#6�����Fpk�
���=/F��TLd���}�����fQ��uK��Ne��r�	��P}-x�y�#"��4Z�����������(���y*��!�T�~��P|���X'����	����p����z�y����N}��$��3L6�t�Z�v�J�-X�_zb5��Y6_��*�@�~G�'�k����@&;,�����$���"Re�3�P��z��Ax��|g K]� u��=��I"�NY
�Q!�%9�������ipHQ��B�x1�s@2�Y�j���!���N
|]�#0&b>O��Z3���,�����a��_��pfa
��0b�����k�U�_��\a�����I�d5�U���G�rX��}��R�Q��WQ����^�7v��#i�����Jq�*m����
�#M���3$��gw�fo�)��@~R4S�	D,�%������I�P��;n�����b���.�Q�3���z�~|�������pX��#�W��H������7O���.�{P�i��XW�4�CZ?>�H�
�X-��V?dj]����@��S@�+� "��7	j�YPkL�Z{T0�H����W�^�j�/cN%3S���_4�O6��.
�w�Yz�^�Xd%�T(����n���<���_eso�����]��*bxx��P�Y{s����� �&��@�h2����d�~E=�k�]�7J*v~'F��L��u���c����4��zGp��]xSV\O��ub��p4��e�K�H
���`�����<��?P�]�2���AIZ>�.��u;��H��	}��b[�W���\�D�'�\����� ������e������!=6�&k4��M�~�r��	���.�����f+yK�G�����Q��w�Nf7�-ZG� l�S�-0s�\U�KP�Ly*/��M|�^��*����[���9t;��B���3����&�t����N4)$�SZh��3d��/>�W�I/
$z��3�'-N5��uJO��F�����ZC�A"���!�H�g-
=,��_��N$�2�� Ns���I�������"�����k>�#0�J�wq��
b�T�������J��2�����x�����>VpZ*��q���jf������S]���ed:c	����4�����K�p,+�H�/��}�B��jh�G�v9���X #`8PCR��&@���k��+0���n#'�_�h,-2�j��)Le�A����7��L�=�6�����j
��Jm	��iUj��ajpH���*B.4F��^t��w�~v��)yx�yb�G����������M@�\�7���?����'�z��������N���\�E����[���OkX��������>a�!Z��Gt'�D�v"(}>SC
��>}��7�]5y��:J����WI,�Eza�M�L�%��w0������d����� <�����w�|��m0��C��>	��^:EkI:1�U<�rGX��Q�4>T�'�]���KZ�/�M�����d<\ �,��GZ��0)*�&�2�_�e�Jv�ev�8�N���-V#w�5�q����ea5DSr�����H��k��	�A��uM#�0�IDD��2 �v�Q7���^il��b-bK���l�*����HF��KD�����9'-U4d<Z��j,����;n|�Pp��>�1�d)�?���E��s����o����[ Ty_r�%�_z 3����I\#@
E�D\���?>}t���t���,��J���]'�,�'+�T���4�=�{��Z�n����
�����xy����q�f|�]��|~�k��ON����x~�>�n}}�[�3���7]&���gtUhv����i���:5����j$8�#f���d�	i��������\���"#���`1���������T�K9����~����������y�`����1�6r��p������V��6m'��#YT�����M�\D_�tm,|��S���E�����*��M������i8�
b�]vBV�pV�R)c�c~�z�b_v)��z��k�����;���8��xJ!B�,�e���5f�-�P8�����Tr'��z

��2�XF�B����1
����`��$0�PP�
��F�L��c���(��`{
?y�vW�����5��'�B�
���E�
2�Y�(����|G��d����nPvB��+�E����2!�0��h<WlUx�v����-�eF�.{9!3PQ����'�_��dVG���o
���P��b��!"@ts��9�yq� ��%�`�|\J@�"�V	K���MFd��7N��-�> i��2���\T�4�t#-�P��0^ �|E`�q��7y���p�]r��N��!����mp��<%�)��<���8<��������pWj���Eh����73��W�^������+�A�������t���v��`����������_��e;�^���U�E���=5d����s������"�jE;*\����V��dAa����C��a�j8�43!��F�wG�*
j������%|�?��#�>x@�R�
��;�c�!���(%;�v�<�������7�%�"`J�g��FU _�](�v
�4���|1{r�e��iE���j��'���hX*#e��m��)��K�;~��oJ;7����P���n����E3�m���?�R��f�%�!7���������$�o��1on��@�w|hYG� B�%2�'�V~_T����K x���NKK.��LC$Ap��DNk���������X�[}{<�v'{�D���[����p@LX��=��kV7��9���VY�X-�QE�#k	�D6*����5�W�u����5FW\��v��"��f�t�����W��GD�_����A%�G��e
���b��[��~�n���O#�����FP��;�*���ZU����-����O$���I9��&D�>�4��a��*��^�<����zHFW�]�
��P��(q1k�`�f��`�r#4a�8H�f#�?�1�q��3�7i��� �������m�����-��&�@��&~�d"�s�����Lh'�����V�DB�
o��p��K5Do))�)����q�a2 ����qh��ln�=g�@���4J�^��W���Mx���o��,&#i1�j�@Q�X(��A���>^����3�*/.S�-�X������qH�b5��tl?�l���p���_����`Z�!Q��1`�
��'�����`V��Wr��;�Q���R��HC�"2v����P9���2a3R����(�Ji���^�G��`7�l|��T������;�c{���Z�;^7��}�W��}+����VY��1oX��y����H����C����:�G:�V�p�ko�q������dx�93�9�������EL#G���k�C����P�j�lV���H���g��c�#zz�~����XDYnt����&E�o���%�>d��������?��h�{nu}��.��A�Jhz�5�_eI	v]�`}�&�%���5���.�&���;rZ:��7����3[��O{H(�������:]s�I|���@����<+
�r	�Hl@���,7��v��]4���m�o���5��=�����e���#�$
����c���>Z]|$��`;�X��C_-�&`p/�Z:��q5�L��tL8�j����r���)���\�Z8��q|
���'�e���9?����p}S�^+����kD���������5v"j�%��`&��k�+���0+1�I6�#��\���[w��%U����Q�M�_���P���j=��k����l4��]�?]1��t�@"zm������0���nu����S����M��=��y\�CW�)�p�V9qSE��w���V���KKgspW�g�7�����/�����<Q�6����F|c��hfB$�X��C_�w1�e���.�8��W��)�_�������WY`<�I�E��
K��1������;gU�q��V0���������
D�Q�����|���6Di_����q }6h�1��0v�6p��a����@�����-�~*(�']���(�e���H,��������*�Y<c�0�����KS���;���Y���	���2R���^7�{������ZV��c��&�h,X������H,V�u��\+��D��l��:K����Q��,��=�6��������V�;g@	�b�)
�-| �w��& ���@�^�/�F��bM��L���H'�^��Tvf���h�YsF������N���lh����W��0~�n�cJ8���;Jo1Br��L| ����r���h��g�1���nP_XBp�W7V}��E|�2/��xm|
=�P+re����UL��IjUx�a�j� G��l��aj���W�'�k�I����K��!5�\��wp�E�C��WRo�+
�+��-�y���8*U\�&�j��������~�N�;����mP�
���y��9��G�s��#UX��)�����U����0�`���rQ$*K��{��^����Z�i.3���!1>�m��<:���X}�V�3�l�d��zp�G���Q,�r���b3��,�(6�t�U:��BG1=�Z�����h.�Z^�bqN�P�;��b���_z�d@���*������A��<���r8�"��
z))����RP{b8��1U 6�B��
sCjB
K9�����}���a���o5S�5�{�5��;���K+PMa�����);�_v�� �y�,�d`���:��f�\n�N�:X�h2��j�g��5���t����s��+W1Kj0>��J�Q:�o=�����P���7�7���=2���yQ^�]����m�R���[��,��U� de}=��,���6���s��=F?�����a��y�t������������-�O�c�_f�_��{��9���J���'��u���P��l�_\������-z�q��D&�>b�+O���>�����U:G����Z��YoP4��C��SLd�>��#���p�'����a���-ww�A��x:�M��!�9<C2�5hF�m�!i'��;IM�(�q���>�$�Q���	�|����s�,+��&:";
/�����`�8��/�"/3Z}�!H_:f��_�6�U��5cJ����'Z7�����t��t51��.{LX������.���)�3�0|1�$�'���
���<.�D�_����*	��
gy��0$� �K�(��ecf��,f����B���m�x������f��������F;�s���Kc��jw<�����R
PZ��d�5�d�&��u���V%t�e����lW�R@���Q��b����#V���r0���l����<!�����rT��;01S����M)�Ii(�������=���t^tQ�i�p�Zm����!���_��E����jMm�����z7!��%�w���"�����1��f�\�,I����h��R�J�W�4�k0����Z���������l������Wq�/�Kv�w��\g�5�W��}`<��`�|e�{X�Ah������_���W�s�l��?��gm�E��MC��_�Qc��q�nh\���S��SZ�������X����L��T&�c���P;��w���i���pD�]K�$���iO_���S���d0p\��H%����'���{}kL���V��`3���qq�W���<P'��XFJW�
v'/��KK�_�UD"dvE�s���=8�H*0����a���H�8�p��������h���Jr�k�$��� /��3��L�!L@���-k]3�b���jE��~��0M�^������6���4��wY}�: |�DVE�Kz�����2���?��Xpc0'���l���b�$e<���kol��������b:@a��n �'98��#����w	�L+U������TUmKn������m�r4�F@�(��$��Z1�����!���hR���p+�)�[��������R�3�iG`��t�B���R���_?d��3�@pG�wXW�x8�zh�X��i�%��VE�y�L���/b�x\�p�$B���XM���HK��f��U]F�����1EXm��t�w��z�g��@!C�)%�g������s��V�-����(O��e]zl!�^�Cv�__Vu+��������_~�j�c���#@�n�^ ]s�����_E�^y�l�KO	K)���H���w������������P��J�o:������D;5`t�C�RPlk6B�'7����PW��/���:�����@������9
��Q���"�{T����kN��d6(�����lUL�%HkMV�,�E����UV���1C��8�L��Wm�l��x��;�4_��@��M�~����X���@���w^e5C�}����d6�|qf'�� K��rgR�`���qeo0 )���r4/�1[���b�I����,?>���)6�XJg@*l0QH���g� ��B��������)-
�����]d�Rcb6�h6��j��o��,��px�}c.�����F��
�o�1����@�GG��o��u�����S���<2~�_ZoW9�
o��n{���5���;V�t,
%��X�o9�#�T_gb�G o�X��������<��RNzl�A����A����Kq+eg�rygM���m���
��I�{���B"|����@&���o2K[RJ�5X�F�u���D�92��j�����Fh�Mt��,��v%IU�R�b/����Nw�s�3 �"|�a9:����z�831���,~i�c������k�^�)v����b8���;���Q9��`��!�>a2�{�vh�A�ot�jn1�^���"3����Ff�h��I�C��[����7����mN��p7��d���G~��\��J2O�.�_J���0n-P?p������%�K��`,�N9��K���fG��+�����O5f�u=��Q<p�z�6�K����=�f>�,Pn]�^�;�w@0�����7{XLFO��M��M*7���)���/s���d+I���][q}|X�6�%���Zz����O��ft�u_9n��-`8,��F`��r��K�zuc*a6*m��oz����l[�����u"� x�sd.�j��8�y��aP���@���@�m)�
���0�e����`_g@#�I�/]9e�ST���R%��nL{r�����L����6��<4�F�w�Lq��7�<e��N^C�+)5v�T��q�V����ih�Y�T?��Aouj 7
a���QYrW�Y��Y�[��Ascy�w������R�#{����>2�9����22��W����v��cV���1��\�����s�T�r1>�.4�xP�"����UZ�>���dk���E&�]9�w�z���n��I�^Wlh9E�������K�wZ���{~I��J�����+t,<,f��^����p2�o��lv8M������^4���^�:�%}x��=<n6/{g���D�[���������-�����n��d���gj��mw���n���"A+��F��&�yCh�M��Ko�F]1��
���a�����]�c�[�s?m]>���Y�t��#F��������V�(w�9]�%e���pTq�����8�����s0���V8�^%���]~�t�,���W)���9+F��/s`���A��1L6���0�����-�����%�A6�M���{�it��5��>�sD$6�p�DL1]�����p�u\��*���AY(Z�L0��?��fi�ML�ea�!�&�C�z��l�2)x�����U��
�"��.,���>Ne����RX��Q?����2��4xZ���c\�w�q��`,.�Z��9���2`�	���Z�}s�EFZ��gt��]�������g��Q�@�u^�J��H��q_�=U���o�p~�_�w�.���-Ui�]a_�a�'N������1������^�B/ZO�x%�Q*S��-���~�`m�b��]4�LD��w�<��;�����8)��N��Q\�2+gN�JYQ��"8-�Ze<A�P�����_��]��� �b��;��_,H�?�����%B��_��z�������;�j��!���Sb?a�We��b�����u"-�+��1�W")�v�L������?u�M�;�3��!,=+c���C�x'�\���� ]����(����"�mf�l|��l�.R��k5$�aE���9���lv�q7$d����#�Dnt�"��c���v��:��Z��e�dPZ��W$�'���nJ�i1����D4�q��|� xs�e�5s��_������u�~
���1�Lk��3�$������.X[y$.�/����*�%��"K�J
Oz�:C�0�����x��?�P�����'������D�^���0�;������F��q��R��'��(N.l~�������&bIX��T��_.����F���w#�I7��}5�FK[:�o����^h�ec��!��ew\G�Q�P�S��bz�Wi�WJ�b��.$��G�1/���F��#I�e�)�Z�}�Sh\Pu�u�-z�,?�Xh��&"r���6O�p6�����fYg���b�</�cji�^�|��.U���<��&$�&7�ZJ��b>������-�f����@f�PdZLe�#XC�C�EWE��7�����v����(q�<P"C�48�K��Q�9Ko�C�uh����t2%��|��7@�N��L�	$�74�wN�����F��qp^�
��^��
,���^K���
U��Q�M�K�_��%I
�w��}����C~���A�DO��j���wr�I����Y������-�REi�O>������li38��	��c���Q�$���^ w��P�����9�:A/�r�1}�<�_���� ��`X`���d��1��������p'���4��{�S������_��^<�EsW���/_�������txN���d!D;w���d�bD~��Y�������*TxHM�	_Sk'"����j6����"R[��bP����D��c�c���w�n���h��gR�FZK�M)��]�Q�1���N������������j}'����(�U��Y6�0� ���S��9	.h��Y��?�T�u/P9'%��������w�n�;C_��o�
�"(��<B�*��~L�������{����N�76��G������ J�kP����?V�����Q	�R�X��jM�:�9�W�����OXFU���-6��1��8��sj��#IV�^w����7] �F��
�� 7@��m��-��(M��N�Z���Y��1��������&��gtM?��q�Qj)U��C���)��f��Je�
�(rQ�	-���;%��I{��]C��<�be)�=����,^���<�������'1�y��7��l�A�<�P&�[��V_�5�4��E��+(��x�
3�� � �>kwG��g����{B�-#6��o�O������g�����p���������|%���u
n����/Y��" ���Av�O�i>�:���������O��N����l��G����2��_V
��C\R�/����`M�����<�����/��s�G��)�{{��������e������J�����8 F7���o��7������)����a���8�d�b��J��Z�J��YJG���*�C�^���]�q�"wk�^�!,�? �
/����9�}O�����������z��?�3���Di�S�;y�j�:���`����|��bHG�)DG���A� ������#��}z�����c�����N�[��$���i��?S�?b�sNgJ}������t��U������A�>;J�V��$=n?<;�_��NO.%����i'�\^�]���A^gS�t�uN�W	�f��\��H}�*�/��5{����c6��H9�5A�{Rb���7��I[��j�>������kwZ����?���\��������V���~���U���"�U!��s2��_�&�mgB7���R��,R_�C�"�`)�\\,�����h2DYP�,����0x���/68f����z�5�98XV�R�^��+������!����@�?I%
+t@���B+�
u�������v�w�X
�<���������:��rFow�$��I��AGC��u������,�r����#���
IWkl����fZjO}U#o�7����;{���e�@F���z�}�q����f��"��3�����Q���g?~�	so.�zN^�TAC�9u^�|�����S��9#��� xm��F1��I=�n��eO
��c(i��+��3
�B�J���Es�6��?y�n�F��/X}����>lf�M�u�F�"cm%�]!��S���M�F�'^��$b��(�(8��x2>�$q>G9������n�����tu�Q0W���ZT�'����+�iV���d'�f3�g��`-��ZJ���H�N���,������g�q(�K��������42�����r��W���
sW��� /l+�����0���q���j>i��!����?N�AyAI��@*"���D�$?���+L-0��j��h<'>]�7�v���v����Yl{��/0����G������"�K����o�����/.JH����!
���v�}����NdRv������a�i�_6���-�r��+��}���;n��g���;^��+�cw%�qO�U%��"=�_��������1X�����B���j�{*4O��`�1Q%]�N��uQ5�W�'Q$�*Xi���a(��B'
TL-��[L`P)@+����_���,�:w%JN��
��
(���V���4�/���_DO�}����>�g�l���}����>�g�l���}����>�g�l���}����>�g�l���}����>�g�l���}>���
�tp
#184Antonin Houska
ah@cybertec.at
In reply to: Ashutosh Bapat (#183)
2 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

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

I originally thought to provide it along-with the changes to
expand_inherited_rtentry(), but that thread is taking longer. Jeevan
Chalke needs rebased patches for his work on aggregate pushdown and
Thomas might need them for further review. So, here they are.

Since I have related patch in the current commitfest
(https://commitfest.postgresql.org/14/1247/), I spent some time reviewing your
patch:

* generate_partition_wise_join_paths()

Right parenthesis is missing in the prologue.

* get_partitioned_child_rels_for_join()

I think the Assert() statement is easier to understand inside the loop, see
the assert.diff attachment.

* have_partkey_equi_join()

As the function handles generic join, this comment doesn't seem to me
relevant:

/*
* The equi-join between partition keys is strict if equi-join between
* at least one partition key is using a strict operator. See
* explanation about outer join reordering identity 3 in
* optimizer/README
*/
strict_op = op_strict(opexpr->opno);

And I think the function can return true even if strict_op is false for all
the operators evaluated in the loop.

* match_expr_to_partition_keys()

I'm not sure this comment is clear enough:

/*
* If it's a strict equi-join a NULL partition key on one side will
* not join a NULL partition key on the other side. So, rows with NULL
* partition key from a partition on one side can not join with those
* from a non-matching partition on the other side. So, search the
* nullable partition keys as well.
*/
if (!strict_op)
continue;

My understanding of the problem of NULL values generated by outer join is:
these NULL values --- if evaluated by non-strict expression --- can make row
of N-th partition on one side of the join match row(s) of *other than* N-th
partition(s) on the other side. Thus the nullable input expressions may only
be evaluated by strict operators. I think it'd be clearer if you stressed that
(undesired) *match* of partition keys can be a problem, rather than mismatch.

If you insist on your wording, then I think you should at least move the
comment below to the part that only deals with strict operators.

* There are several places where lfirst_node() macro should be used. For
example

rel = lfirst_node(RelOptInfo, lc);

instead of

rel = (RelOptInfo *) lfirst(lc);

* map_and_merge_partitions()

Besides a few changes proposed in map_and_merge_partitions.diff (a few of them
to suppress compiler warnings) I think that this part needs more thought:

{
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;
}

I could hit this path with the following test:

CREATE TABLE a(i int) PARTITION BY LIST(i);
CREATE TABLE a_0 PARTITION OF a FOR VALUES IN (0, 2);
CREATE TABLE b(j int) PARTITION BY LIST(j);
CREATE TABLE b_0 PARTITION OF b FOR VALUES IN (1, 2);

SET enable_partition_wise_join TO on;

SELECT *
FROM a
FULL JOIN
b ON i = j;

I don't think there's a reason not to join a_0 partition to b_0, is there?

--
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

Attachments:

assert.difftext/x-diffDownload
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
new file mode 100644
index a75b1a3..3094b56
*** a/src/backend/optimizer/plan/planner.c
--- b/src/backend/optimizer/plan/planner.c
*************** get_partitioned_child_rels_for_join(Plan
*** 6160,6170 ****
  		PartitionedChildRelInfo *pc = lfirst(l);
  
  		if (bms_is_member(pc->parent_relid, join_relids))
  			result = list_concat(result, list_copy(pc->child_rels));
  	}
  
- 	/* The root partitioned table is included as a child rel */
- 	Assert(list_length(result) >= bms_num_members(join_relids));
- 
  	return result;
  }
--- 6160,6172 ----
  		PartitionedChildRelInfo *pc = lfirst(l);
  
  		if (bms_is_member(pc->parent_relid, join_relids))
+ 		{
+ 			/* The root partitioned table is included as a child rel */
+ 			Assert(list_length(pc->child_rels) >= 1);
+ 
  			result = list_concat(result, list_copy(pc->child_rels));
+ 		}
  	}
  
  	return result;
  }
map_and_merge_partitions.difftext/x-diffDownload
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
new file mode 100644
index eb35fab..aa9c70d
*** a/src/backend/catalog/partition.c
--- b/src/backend/catalog/partition.c
*************** partition_list_bounds_merge(int partnatt
*** 3110,3116 ****
--- 3110,3119 ----
  			 */
  			if (jointype == JOIN_INNER || jointype == JOIN_RIGHT ||
  				jointype == JOIN_SEMI)
+ 			{
  				merged_index = -1;
+ 				merged_datum = NULL;
+ 			}
  			else if (jointype == JOIN_LEFT || jointype == JOIN_FULL ||
  					 jointype == JOIN_ANTI)
  			{
*************** partition_list_bounds_merge(int partnatt
*** 3140,3146 ****
--- 3143,3152 ----
  			 */
  			if (jointype == JOIN_INNER || jointype == JOIN_LEFT ||
  				jointype == JOIN_SEMI || jointype == JOIN_ANTI)
+ 			{
  				merged_index = -1;
+ 				merged_datum = NULL;
+ 			}
  			else if (jointype == JOIN_RIGHT || jointype == JOIN_FULL)
  			{
  				/*
*************** map_and_merge_partitions(int *partmap1,
*** 3334,3346 ****
  
  	/*
  	 * 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])
  		{
--- 3340,3352 ----
  
  	/*
  	 * 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, or
! 		 * neither maps at all yet, get the index of merged partition.
  		 */
  		if (mergemap1[index1] == mergemap2[index2])
  		{
*************** map_and_merge_partitions(int *partmap1,
*** 3352,3359 ****
  			 */
  			if (merged_index < 0)
  			{
! 				merged_index = *next_index;
! 				*next_index = *next_index + 1;
  				mergemap1[index1] = merged_index;
  				mergemap2[index2] = merged_index;
  			}
--- 3358,3364 ----
  			 */
  			if (merged_index < 0)
  			{
! 				merged_index = (*next_index)++;
  				mergemap1[index1] = merged_index;
  				mergemap2[index2] = merged_index;
  			}
*************** map_and_merge_partitions(int *partmap1,
*** 3366,3380 ****
  		 * 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] &&
--- 3371,3379 ----
  		 * partitions map to the same merged partition.
  		 */
  		else if (mergemap1[index1] >= 0 && mergemap2[index2] < 0)
! 			merged_index = mergemap2[index2] = mergemap1[index1];
  		else if (mergemap1[index1] < 0 && mergemap2[index2] >= 0)
! 			merged_index = mergemap1[index1] = mergemap2[index2];
  		else
  		{
  			Assert(mergemap1[index1] != mergemap2[index2] &&
#185Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Ashutosh Bapat (#183)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

Here's set of patches rebased on the latest head.

This rebase mainly changes patch 0001, which translates partition
hierarchy into inheritance hierarchy creating AppendRelInfos and
RelOptInfos for partitioned partitions. Because of that, it's not
necessary to record the partitioned partitions in a
PartitionedChildRelInfos::child_rels. The only RTI that goes in there
is the RTI of child RTE which is same as the parent RTE except inh
flag. I tried removing that with a series of changes but it seems that
following code in ExecInitModifyTable() requires it.
1897 /* The root table RT index is at the head of the
partitioned_rels list */
1898 if (node->partitioned_rels)
1899 {
1900 Index root_rti;
1901 Oid root_oid;
1902
1903 root_rti = linitial_int(node->partitioned_rels);
1904 root_oid = getrelid(root_rti, estate->es_range_table);
1905 rel = heap_open(root_oid, NoLock); /* locked by InitPlan */
1906 }
1907 else
1908 rel = mtstate->resultRelInfo->ri_RelationDesc;

I don't know whether we could change this code not to use
PartitionedChildRelInfos::child_rels. Removing
PartitionedChildRelInfos machinary seems like a separate patch.

The last two patches implement the advanced partition matching
algorithm and are here in this set for ready reference. Please use [1]/messages/by-id/CAFjFpRdjQvaUEV5DJX3TW6pU5eq54NCkadtxHX2JiJG_GvbrCA@mail.gmail.com -- Best Wishes, Ashutosh Bapat EnterpriseDB Corporation The Postgres Database Company
for discussing/reviewing those.

[1]: /messages/by-id/CAFjFpRdjQvaUEV5DJX3TW6pU5eq54NCkadtxHX2JiJG_GvbrCA@mail.gmail.com -- Best Wishes, Ashutosh Bapat EnterpriseDB Corporation The Postgres Database Company
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company

Attachments:

pg_dp_join_patches.v27.tar.gzapplication/x-gzip; name=pg_dp_join_patches.v27.tar.gzDownload
�_��Y�\{s�������@sgR����d;ir�$n��$�q|�3���@$d��H����������E�r����9�6�	`��],���F��oY�NW�B��z��>��c�9<������p8��1�����Wl��+���e�s���r�.2��������j[�(���uR��D|Iw��".�,Q���Dt��+�Jx��A����L�v�r8��M�����_���p��gKvz����xF�Q8���g���Ga�''��|<9��{����X��1��l�l"�g�L������[����N���Wy,E4����e�
/�3v�Xlx��g�5�==;=��`|8h~\�~a���������l0�G�2GE����T��*�l��RY���`q�y\�4l����b��9�5$���b����X���P�$��e\��0�����d�����V"��Dr��3��Y�x�xs��8��4�d<�Od��9�����F������[�#C�et����lk�_e�nKnk�A�C�������	�u��S3�l
{���"
�d!jg*SC������`|�Jb�Oa��iE9Q�d);���%��0�:�1�}�������\�������u&�<`2[
�!.v;Z%<M���	~�$p���f�Z
x
)i��HB�$����8�E�T1��!_+�l��bXv+N����D����$�F� �Mr���n�y�ab��J
���1�(<> e�'��~���	�Y�cMb�����L�ce�A�I"��I��r��Pk�e�|�������Y��l`
��2V���"W�,-D�����l�o@�@�$R�(���1�R)���e�Z\����H��a����wS��V[s��,���ZEJ�Q�7�x�0�'�M�)���K����+��)�L��f�^s)��f��U�9>���ae��X�|��
r�R�$�m��h�r��\4Q_@�Sa5[
�x*@�9,+�"x���d������e,Q�7����cN����o��2������**4�<�o���v�L�a��O`.�lU����D�;[�A���������s�:;�&�q
R�o0�F���������4f2�1�s��r���)�������%	����i��#�OA"�o�vo�������B��N��
�JD}-�^�.���N����v���,p���c,
:
�Q��@������:m��	�@���n������u�7 S��O������F�w6���0:����Fx&������g�����g��;�����w�5�iy�'!/8��>�ru�fyo�dGp}a�\��o��<���;e#�]��e�V)-A?_���m��wZ�������.���j��������L�qo'���6��(���)��TOZ�I����%��O�������h��(r�~���HP�x��4���5�4�p�$���epb���Dk:��w������I�=v�J����4|�U�h��7Aig�u�q�����������-Xa��K`���T��=vjK��V����p4���h<?:<���URwji�3�r8FQ��U5]e�@�'e����f%*
���	��i�i-�
c�D��V������4Ygr8F'��a.�(�j��b_�
�Z��"�i��b/��9�/������r�Rj��|C�M]:���p�z]�L��_`���q0���,9��Qp�WG���Q��m4�9k�)^������'���/�`�'�$���v;}u����q��Pz��P�Z������/;`.��o�Gx�?aW��
�<���O(�X"�0HMh:��@�%b��M# 0j�W��ty��t!�m�P\#p��o�b�����s�}�
��i#0����p��k*����ZR�x4��-�Q��vC;'CF�]Ll���C�h�����_��9��C�OQ�l�	=����vt}�.n�}\����}�T��U����Hxw����g��SR(�c
au��&;���|�J���%Gc�x�f���������^��z{�F;s����}	Bp�'�'@�lm��V���,�SHM�������w�_��8��dt82Xt2���KB����+9���	8�����w�1d��8j��Bu^��7=xV�1����o�AN�xW�vu�v��Q3J5�JWT:���V�=�����X�� ;��Cz�������:��p�h�l����v��5��:%��p�<��M�������*�_m����M�aQ%,���[��7�
3a�'
l�b�x<0�DGHx��S��p��7����%�������<8QV��D,��K������_�{3�x��5!+��p4����B���|_�������a�Y�&�=��=?��e��3)b���{���jv1��<�����xt{�\����[��E���*�����]�}��{3�w����2	k����3A�����lV@^�6��IeB�;P��b9�DU�a/^�y�����%�����(x������L������!k�eE
p�px0��T�C�y)�`��9�a�"�a{IO��D�!�{�\/Y��
��+���W	�������%&^1l"�FK�X�����$�Bd����Y
@�����Rb�ha2&�v}���H��P���h�6o�����y�j�Q9MDzS,Z���J�A���}jm�\p�_�t�W}��p���,�,�-��^�0�L��F�������U
�	H'!����8��������\P����t����_-��
�`a0�%��Q�9�h������=d��	qy�R�����9_�/�������<��5��9a�nD�x�����tc��L �2����S�U������G9���Rs��j�*�j���<����h����nYg�5��G��������{���,�1��E��S������a�����U&����g�&c����0�M}#�3��%��&��q}���w�G�{��]mn#Tk����MKwU�R��-V��v�2�����2QX1�2`�:��d<��A�ic�[��U�]�.�����~y��*�
��Ly�Nule�$��FjI�g$�^Y�P7/:Fq?&�(R	�}�[�P�)�	�I��2�t�V#�g�e��Z���A��
�%f��b��5�	��,���Q��>��-u��������o�O�Y{	����G�9O��m/�����~�����v��	w���v��� �����ql�����&P�|H����T�r�}@:��U�LH�v�7u��K�e�?��d�j���j��cpD��7,�{����'D:�����-�l���.����)��3�s��H_��V�dL��}�2�Y����0R����3�,?�G��CY�g���^a�������^�b�3a�����#�A��?������U��,!��1�h4VL�9���'&�o>�od������e/g��\Yx����e�l�xDI���I0>l�����>�����ei5H�w���g����c��Do�o���C�������U
-�s}MM�
��c��?��M.0�B���S�e�3Q�xM�B���8-�,�H2�#���������mj;J��Ib�!0v����A��1���W�MN��&&����o�9`ugG���H�~�h�kp�&�����n�����%�
�Si�����`�_7`�����nl�0�@��B���+��=���J��<>�|q��av�&��6��c�R
�*q�A&S�0S��vn��3���V�,��QJ"Z9+���6{	N�f�k�Lu��`g���Qg��p>�{��	��^��*�;����*�
������������@o9��5�<>�$g�6�\�y]E���L0��Y���C�Z�k���W
�X�a����F��T������zw���%Y�	`�pg��9�49`x�s�K��j�W���\k�J:U�fbT�i����u$��;D�u�oK���[ v��=�1�J�`oof+$��mGK��~j8\'dT�Umz����M����,��D0���,% ����h����{�����f�*h���4�!rtBw�GUa�e����m��2Ol��{WF�����J���s*cyc��X	�!��mju����L/��1�1��I
�?�����<afP����*�R��/�A��"��Qe���*0���[]@��N�3%w��������������	�4G�/�{��
���t\�����tD"�e�@�?yhh�;E}3�p�����k��]����o$F8�����R�.y�F�z|��t���M��
��G����S�{�����)��$����[I���%�&M����P��=*����A�+��K5^N�}�mk��Gm�[�X\��:Mi	��2����Gs�Ha.����w��%�1K��lt���w��F�,o���AM��r.�z3�#�NC�[�.�p���!N�������y�.T�4p�j:&�^H��{��e_�Nj���Z'�Q�@!��%tN�F�;O��O���i�9
=v������=l���v�*�C�=�wK��~\���i}�G�
�C�f������*�2���tm��(��)4�xnbT���l �3�U������^n�M;c��k�
�*����u���u���$����xpL��M�y3��>pr�|C���{�r0{j$�=\!�^�����}�����%�����|��\TM�c�2-�*o4���k���b�����1�����W��=f����@@�m���V�HY
�\�7�\���z�(�7�%��J�{nB�B���
��X��Bq�V0fV����J�2,�U���k�@�WIL��X�y+��(L�uM��)�4�����-��G�)�����G�HU��������[G�qC^��+�Qgt��
�F���eH3���'����*����:9�(�Z|H�����#���R(N����E�����n��A�aX���S�\������P5��T|��B����Ig���%��Ty`*S�6�����5�+,k������+x=\�n��h]�E>���n�+��b��T���UM��+y���q���1�t�.���hSr�� ����h��I���N�
�zw��0V9)�&�>����4)�����4��~��iR��m|~��
�lR�}�F��d���)j�qED�,�}�j��'E�����u�!�u�%������f/���?�KTtH����>Z[m�E��3���]����0`�?Q�U���Z��Q|9K�������U���Je��&09���U9q���=��;���q|g;M�i�����_�05��}uJ��uVs�!t����o��I�����]��o�{��V������J�l��������g�.YTP�&����_UuG������R��Q������������w	����vM�[1��&Y9����z {�����^����;�������Vx�?x���*#����b��>�e5�]`p����M��oC�X�t�1�;Gn��Q��S��S���O�����?K�����k���u�R�s�R����zp�R�'^�>�����Q��e��V"h��"`�K������
$Z�J�����f����X����B����u	 x|���t��u��K��x2�D��U�P�p�����S��D�y�N��l{��\�V:,p��\�:���}���J�����kB����	0��a�u��`���`lg_��v)t�����,������G��Y��������"�x5�0�u#^M�K�^'�U��<������zpO��QI�{�j����*�X��#��|��q����V�V�^�F�X�'����V%��v���F����:���!��A��}����!F�@�z98��	j�7��T�����F������uo�Rh)�.�"'�Tc����u/O���2�����t��B8��l��~A'���}K�s���+�^!��[�+� �d-�v���fs��a�j>��]��J=���{���4.��z?|�BG���������*�����K[~,3�I�E���w#�\�	"��+9������r�2��e�f��r4�`��&n�jU��������%a~��>&sfLE�L��l�S9�<��GV���y��)H�
E*\�x&���]�{��@Y���"������p��jAL%?0�t96�E���;+����C��la^��~R��'�����D�H�9�}3]f��d��2��k��C�D3�P���$wwE������([l�d�lT�����qnq����Qi�*J��g<��`������'�5��g�����aT8��tsUdm�A����`R�A<�� ��;���l!� ��'����O�I�n��y}�>C�*!��8U:�}2s�ZR���A��'4w;����[���%��
�������S�P������[�����tUtd��T_^��5�.��IX��o��_��������N����Q��1:�w�q��`�&�5]*�{&}"OY��'�1l9�3����-�/Y��=L�K	��ScG1
aq4�u���!�Q�GK���!\�|�84��b!%8k��nw���f��ir�
O..�����%���i��,�3~�!X���<�xYT���}{��z���3�i������\��`���2���u���z�!�#������cE_H7s��Q���"�CWCp��l��9����h�R�16DL[���W��K��r��l���g�9�|2�kS�����\r���G��4�(Ey�_�����u�Q�o��P�(�`<k�mrL�c"���u�N�{����(���p"8������6
����;�TE'�I��a����u&����x}C0�����q��EL]J&��b�qEy���_9�=�W����wpC�����&�F���(wx����c��a����T�!���j����|dj��-�q3���/�T>a�s�������
��.	��P�,J�c2���d�0��:�t�ho��qg����ogJ/:)�i7C��%9$"���5�;�����	�D}�;����`��Kg�Q�_^[�_[l�������������:6$����S�j�����Z��Zmh�h����]2��_c�+��HE��K��A~����������=�q�� 0���B=�O���
/�����F��"����}?:���<1W����I�G8�y��t3ds�c��S���~7�F��^D�M��c��;��3f0������IM�}Cj���lbx�p'�a+�@+	��t����)�G��k
)�����d��\�|~�UD���4��	%p���.������M��mn�X�*%%P�U|����-��(��*�ir����G���A�ybV�8J�bTi>��^�:'O(p+�A��>a�.����/�|iO�����JX����������pg�c����.{�����bJ�}(���t!s�d��~��0�S�pp������^���t����qzy9Nz'���d�N�N��I�2����w|��&�����?�XN�P��d!h���W��W��+���?� �r�� ����?�G�R|�Mn���74�
<B���E���Ng)�"Sm=�/��g�t:[Q�e�W���auB�8H�$BX�������`A��g��NquYpo�pBcS�f�n�=��H�b�����;@�Ai-�
��&K2�J�<����{���O�������[np2�*LIuEF��U���U���2�R%��N���6�ya�9*������p#���:e��W;/�iL!��}����$M���szyI��l��[�}�bn�V8����,���������d1��^������''��3������������3���3��w��]?9��e�3�@j��N*(i��0���QI��!�����q>Na���6��{<�����z:���[�1��=r�d��$k,����6	$��"|:��X�n?��zi_�]o&j=�N�!(��L�@# �Ja����nHi/I����������'���������A�nX������Jq�w��;�l����w�a�n��_�tL���]��K�����ai0,J���qc�+�Fi|H���� ������)�S���5�%����K�=F������!3	���27�B��������������|��3��wQ�����o��<]g�8�O�2�<D���@sK�������_��O������/�? �d:�f�~A�I��C*�����H��4l�w�O��X&�|J����,K���+j���j���B���d`^
}���,�m��o��zDc����i?�\������l.<�/�+��|��q��������.(Mh7���A�1�:Z���T;��Y]��2bABR���,S+T,�����A��D�_�I�Qm.����Q��`��1�n^�
)��`=_u!���Y:�hAO��a2Y������c��}0D��D�B]2�o��W� ������V@0/���`u��J�{F�!-k����]0�`�A�)��WW$�<��E��{�����t��]e�����]�����nQV�|[�Z�������� y�5���q<������d<��'>!�XE��X~�h�7+�S���o5]���@AxQZ<E#dL�m�ao�n��^��(P;4"(�Y������3M����G�+��4�G"�3^�T�YA�j�d���C�.eT0�N:�~�fvi6�� ��:glV;�x����F���+���`���gaMWv����)�����O��s9^.���$�(�d��z0��-pLZn����3�-���#���BUkR���eV<J�`�[;WV%���10[���
(��Bs�p@��z
�o��@���"�}�
�f\,	*�[�hygi��|5��z���������X��=1c4"�J��XR�"R�����|�����2�R�ue�=�-A=;{�@-]i���_Y�lV���7��>|��^�e��{>�P�C�`;b�b��UT
�|	������;�H+MK��/�^���_���d�jj^��1R1
$P�A���ogO�WdCd��Z
!�f�K@�hDVk����Z:2�����)������j�MP�%f��%4�m�x����x�K�&�r,��v'�+���
�YE�K�����}�.YK��uvY�AzYL�V7)��E,
���Y�l2x���H%F�������J�u"g8!�x{��� ����Sno��9A���62�`<��c�,�������<�UUD(�
c��C�����E1��0��w6�R����	�z���y��~��@h�b�?�2��pR�2Lf
���x{PdM)�L:4�T#N1IOCi�3��9�N2��/A�E���T�*�<�!)��V�!�!{�h3>=yb�p�-A��H�fJ����6"���',����a��OsJ�N�t�joy&�}�63�i9K�Zi�kp�m?�s��������0;�y��u�<��`�����^.�'�5�e��FcB�B%��`\
_��N��m�1�&|�;��J���^/_#��$]��y���/rA�x&�l}7����%���dz�b(�P�iN�
-} �&������q���`�*��3�v�]����+��'� ���`*n�~������o����9D��{�g�|a�E��}�BZ��VO��c�����.����i�3�����������L�0�fn��LFg>�s�����{!������GB*�B�W���	����7jJ4��t���$�4�����L;���j�|�'�/:G�:�(�UD�5q��C8&�n	B	*���+-�W,V�%���@<{�O���i��B�r��������8��^<�JmES������#� �r'����<])
	�����������D���\�v�p���n���?�\���z�S	�1
�4�P
�QL�2�P�;���f��LD#��
�W4.��@A�)��y�b5��wEVL0����������}s}�D�56��T�|N6K��N���	8A��
��U���s�3��"�#b��$�i�K����#�s��Y�����kMd�w���g�S��=�u(�EGH`S�Y��T
<�����.&�z�!����O����t������!���w<�,OXv�M�tx�(���������*�I;��,f�Z''G� �G���%��_��KJ�Qr��iEh��b3�ib�3����I_��F���W�B ���|��sgRh��aVr�7��ad'iS��j�N�I�dT/H|�`4� �^�%Q�F5�jX�N���eex��@X����r~�F��X��t��j�BYa�%�gi?�VLb��h��$Sx���;�<�6�n{�-�9V�s��VT}��l��*��^���������&��4�_f^�'l�.�������S�S:��J�2N+�,c�,�F��L��%����z����o�Oe���zE�g�P��������{����#�CG ����'@��@O/�.����=b����W1*\1��=XYu:g
�a�'O4���DR����P!�&�������M��	��L����LM�;�=M�����Y}�����OV��(�Z G�)
��_D�G��������T��M��&��aY�<��<#�����,gd�F��������i��+�����Z7���XD4\�O4�	8�b�h%�������5����"���O���]��&J��1�"UI$pX@6�bO�4�(�;,�
j'�I�\L�E&�.7��F�1�R��<kE�pM���f=�c~��D�B|9}��f<Z���m����*7O:�p��C#�7�������+�0����/o]S��!�Q�^�<UB~D�u���
�%�{TYv�HUo�'��������N������� �|1C{�s����:lETi�K���H��|L���@,�&�@[���_�M����������|�����]��r*07��������}iFn'cT�����?��W�n�~*�P��o�0����+p������_���������������a�hx�$�g���W�z��x�scD�B�\����azx�H��C�Z|�b�!�a�ei�Y�r��>$dKb�`��~P�F���j��"���=h4W�,����t�?��h��'����_������ Y1d-��rL��������`r<�O��JM��d�,K/K���&4�3�7��o��D�W�5hW�����~S��s�]Dr�JX���i��k��|�H1.�-Y��� 9N�,=8H/O.'C��dyM�k��e��q�������=�8{VB������e����1>�i��#�dd28��
l���#�6��:����g�����0"��Xb�KiG��t=��b���y�R�N����U�mU��@��_dI0�OH�'�!i,*��t���&NVhsJ��p]�5Etj�<DkI�N�f@bf6���A��Y�Y6���;�$���Ox9� ���������E!��NK�r�~���bL����G������SY.f�u:;��x�����f�p�)x�NE�x{�L�!���dD��B	������Oo�c�8�.A�0��G(��c�r*?_�w!���'��g��v��.�o���s�Y�M"������b��d�Z���(w�#~�>��O81���1���1&%����v)kJ%R�������EO��T��5��pr��B����^����N�X.U�J�-����[a	}_���o������j��	������%�VyM^g7+KT�`\������7�/���� =�/�E&�$so���#��Rqv�I�Z�{�����������>uFqu=q�z�_Pv�_����2%�/�sY����F�HB,a;a�|:���h�����-�AhKw��k�=���GL�u�<��2��
�LW
S�/���[cKjV�7��)��N�Q�CI*� ���R�5��M����nq�X������~}���&��r��(n��_R���c���U���dyuw/��{��2�����]d�u�Dc�8��&����3�8�^Zt��D��-m`���G���~����9�4�6n5�S��&�I�Z��B�����M�8��������fv�^a����7��.`(�Pd�0l��E�<�Y�T�9���?��p��GW�0�t%S�A��Z�6\�LeE������E��yx���9���Ta��
1��,���������?�	�s��bv��dS�I|w���,�b�l��s�
���qV+n3�3�t6�cEU�yb`m�7���b��5f
�@��u|a�/����@:��0��A���A1�
P���Ul���]�pq{�`}��&k)m��z�x���/��D�EU�*4aWT�?��JhSQV����0��'�PtbQ*�
�6a��9��r������%"����)�-o���X�jBu���h%����\�=�}�8�nc��n�@)6�c�4��0�_q-������C�z�
4\�P�*G�?4������tR�=b�+u]�M���,����J���N�%(S�@h�����BZ��z��\*w0�	��2f��"��=w�R��BW<�%�%�v�:�!��w~��u�)�9��
�������I��%Gb���"]���& N��F<���c�_����4������/"�+]��Fg���G\&��UJ����p1,x:2��1A%WS���c{*��<����t�_��4MP���n��.��7��n{��k4�����|W����
�~��8zv����?�:����VZ�bA{M�@��������kd�M/�����u��������C��t��z
a��,C#�f��!�G�.�7�3��	�.�s�~�F��9������<3+�	��1�� �AW
�P�r�B-��
�%������e�"AIn��,�m�lc�E����0(�!FQT�I��b�����Z�Zy 4vA�P�%cpp]i#y���H'�� s�u�` 9c���tv��Z;�D:�N)�1O�����lf�������tf/���� ,�x����9��X�p[�� �P����B>�F_h�������P�Za����<��
r���[����:������b�J�U�dcL��.E���h,��H�,o@�p�o��TK��%O��l��~	!Z�/�(�W����Oc�3~G�����&��,F�s])�e�N(O�1�~��]Z��j�?����G���!��K��BM����S�9�(�wq�!V�w9�~L� ,!�y������I�/�
�?�H�Do��1-�����!�y�.i(*{�8�#�-�mT�03,��4���j�*������X��
������q�G�����f�����UL��M��V��ELsqy`j��
�}�b�0�gr�2`�H���7Z�a�Q�;�$9�hs��"N�����g���p�84�$���V&(?|����v	�
#�4��kl�����:�1)BL�#�,��Zki�!M)
bMH��E=��X1����
kV�[����;�4�\@cDR���R��
sE�$��<!��SNi��ZzY�����J�{Z����/W���Vy�9������
;���h������sJ�)��Fe�L��$�}PwG���1b}��i��	7EU������l�3��C]�S�1��#�O
	��� ���s�y�@i_�E���thQ�Q���<�X��W�91���A�"���8��L>��m�Q,�<z6�%T��f>Q�M��!��<C�H�5P���K�g;]������&27B�q,��6	p���9�#�Kkemh�\��r"�>d��\�}�zsjP\���B��xO��G�B[M�~��?cIe�*u�c���5�1�e:HA%D�3������z��;G�PhI��t�+�,v����������@���2���jd��5CJ�Fh#�s�c`�� ����!.����)D��6D
������P8��,����p7����r"�S�Z��/�t:3�pvn����
�^x�������P�Kx�D��J�!`��iP��y��t&������@,d�w#�����w���BNJ�3��d���r��)�+g<�dLH�]0�C^������34oS�S�IJ���H��Y�#]^3���6�z5Kg����(��(P��1�~�8�}��pU���'l�\�w�%U�
��e�''5�vP1*�X����x���jR� 	���QFB��O�1������[I=z(�L�hTG��u�M$�&j:/��j�,��k@��&��� ���6��w�B�������N<dH�p�e�?�0(����)���L��L�k���B��Uy��^v��\�)������	�������|��,��N���pT��%��p��y���G��(jkh�n%�s�|��b���+�o+�H�Y�T��d������j����Wb��r}�>a���/�4@�1v��g%/�'�;�EF3>�}�12a��Vo�7v^oI��F��Y���aC@(�����m�Q���'s.��%t7n-�SRJ2M���������g3 �X�-uf�p@bDP��[8E�����P!�QZ)��(������X� -P5`h�XJ@�+]�9��f��K��N�Cy���>�7����E_qgo���]#�w��d��@*�����Q����B�u�'�"��;�"R�Dml�F�EGT�+�F�G"J-~J
" �!����G���Z�5�}�`�,�����O��9o'��
���809r�5�+���y����7��0��m0�K�T1��q��C������`d.������<8��'����������,���� �R���;d.�(2.�SH�ij�nV�.��L�N�[��c�`M��-DLO;��2Z��9�~F�g%t`�[f�)����L<�����x��(
���w�|-33�&h'���2o����.Q<7��,�5�UMBo�O������4R��d}{7bd��	��+N����,�iV��.��#�F����)���Os�Q*T�����)N����g�
+�������������r��{�*E3��$�J��4�|X1�>@�*um�a��u�������I/���Qr�L{n|��Q_�c�.V�/W����'|r�8c&����Y�6m�_+���S*J/:�/[`�\
E�|�M)3)��@.���(����c�*��fa#�,	8 ��(��������G)�+�|Y�_���
U`5(�|	��k^�W�?@�3��y$�n�>�
F#I�,X�q�'� v�C��2�%(��1�?��Wrw[����V1��m���7���s^�wQ�pE������K����������q�Lf9k��=��2[���� �Dx�,T���5����~���2�P�qbk�3k��!�R�K��0� ;vr���{��#�41�KC���+=~���������m�ukR���_�A3��U��������L#�1��jCR���M���7�e}k��������5���Y+[s�
U�iu@%a����&c�K��������m� ���)��D7���!�?p_7zK^����7(}��,����[���#�����Qn�����d�9Ic���n����j�����2��-_��,a�=��`E�[?aG�0�S�|
�wO���RG����:Tt��T��oHx��fjO�k��9��G�������h��0�fME|�Y�'�0J�=�H����.�4l��kal6�q�5 ����]�4����~�����5.�z�{������epr�����l2��c�*+�8��� ���5��V�!���00�A�� �VOD"�=,	H�
 E�pJ9�udS�4P�X�����j���R����C��#����Qb
����FG�|�ce�F�`&����?�FY�0"��{��A�u����p �xM�zf�C������A�'O.x�{\�_~���l�N���O5w}��U���)�aL%7
�xZ�����
����/�tK-�n�(���gw7�F4��������������M��=�b��9����$
�E���T��tz�����Cq��nq�R5?�Ej��K����N�P4��
�P�~ygS�	���C^w#��D�I��s2����"Ag��������0���Z@���A����gBu��:���YB0(c�4��dBj�;���N`���@z��4'3�]@�>��A���i���|u2��I�X�������&�B�_�U�����v�b'!�5p���7�������kZ�^-�uqUw�T!�fYqe���::HW-�V
C��D/�M�V�^���.��Q/988��'���M���J�9Y����H��r�$�!7I��P��B6���{7Ti�`�C�)��IQx��n�p��v:B
�A�O�i����1?��3X�b��-	0#�P�����1�Q:N[�.�}!=W>�����A�'�@?9��&{�:����s�5�F�R�-�s��N��E�n�[����kLDr�,����*��Dj�|	
:H?�b��[����	��a|�X���Q1�b�W��,����YE��,���]/r�H����eR!H
��$�E��������:"_No1���l �>]��[��C�|xD&59��UUc��B��@���G�TX=&�p#�!{��yz���S1YO���\	c����.5�g�~5�b�#��D��e��;TD�^��}��An�!�<{�V��Q�$�����(B:!�T�;F�D�B���W���G'��R*p��<���BE�4k0�[/i�mD;�F�@��tt��,8����sa�9� -�2
-M���w<=7��!�@����<���vsf��/�
�X�����A�W�WUo��U�V��Rl���p�!��@��/��3�M�dv�q���Pt:�I���AV.�k���-Z���hM�����E �s��c�����H`�>R-�R�~�C�"��y��%`�&qRD<$�Hr,j0~>�Z��d�.��}_�����{a�l/
)���_>�a����a����]��d� ��n�z�b^�Q��<���$M����5����#���)� ���d,B�����e6Y�@2���u�,Q���AHC�����������,��g��JeY
��K��{�g���_#�X8LL���.9,<��Z��N���V�^����l�n��
�{E�d���+`�(��A����sLl�fz�������+�*���.�/	�"�����-��y2)�k
-����B-bH/z�U��^�z�Z����1�mG-c������4��%�7�^Zm�Y��^Z��\AM�E2������0�D��VaDZ8\���0��,�l	sU�oaV����(�R�����Eb-Eh��.�9�
�F:�e�������������H%;3\G���nA����aF���nn��|I�k%���� g�p�S���Y�TEo%�
�y��y��K��F�6��R��;�����8�u�J�i�/���g���2���>�����Z����L�� 2@B��F��}C����I>W4S���J
��� u���<g�������U6�R}��m{�MD�9P/i�iA8c����b,��:p��
�m�Q�v���lb��/��F"}>9�x$>v�hu���2r�]�c����x�Qc.$rAD�����K5tU K
'S�)<�r������$T����VH}���x���`�`�����m~���rz�MW��F���tO�������5v"�����>��	������{�e�eLC
 ��P��c36A�R5W�Q?f\�j�Zn/������i�
�Xp�@��������/���p1S�2�4��l6���n���=>�c]6Za5Y�x��K�'�sr��%g2q�B�-6`G���b=��Au~�4
�>C9����%4�i�h��%��iH�m��a�f��Q��K���
�Z�/��w���;�����EM�"�E� ��k
�j�C}�Q�H���:33������F�.��O���W��;������w��H��?HQ��l��"������P?�����P���r�d[�]���Z�J�\Y����%^��:d��l2���x�8M^?[,~#��o��r}B�0D@����f>`�}��%�-:����^Q�2�0�;k��U�T��*d������)<,-L!�.�X��x%�9�dh��n�����p�97��Ya^���'e���A��j����W�D��o���F�=�}ml�^����� �������n��f.V�Vb^:R�p_E�r�;��!�v��Ez
��&��GI����[����E��@���@�����,L�4�8�
����9$u����f���H��	��"\%'�hT��7��������g ����;Lf���Ao���dmY�y�g�\H�E��Q�������dH�tAO�b���,u[��>�\��~��j�W]� �7����!8�*���q�c���b�X�M<���a��f���}�kI%���_K�O Q�O����;d����k�
b�L�M[T��e�kU�M�t3r�Bn������u������r�A?��H��A7�c~$DT����������d�p���r��������5�"f�j2_^��d���&I�'������*/�|D@�!%TW�\'.��/-��d�D[c�MlP?"=�%�%���&+A�����������9�6@�
`�
���O|�R�R����;���|����	,���0��tV-��;��td��?�D8_��)YJ�U������'�����������W�Q_T �@��U!PB����@si���,�!������!D�����*UaA*)�Ba�>�����(%P(��4��-����%���;2���8��<�&������fB���J��@����(�ia�b��EL��k����{>=������"-#$�������CH8`&V|*�[�L��[�%���_�0Hd	2��3�(��|�.�F����Jz(N���R�<��3����R���@SU}�O���7��r;�K�9�5���������E�q�#��(9��f�)�n�x�^%�������C��[��i�E/�(��#d2f�byl�2bi5N��c�r��G����'����l����%{Q����%W ���V�����U���tVI�����Z�
�Cs��d2���an�OYj����rqs�[���7vr��|�������b)|bT�LQ�������t����^{E�����,���\$}����8�I�JH�{�*�(���9�\�~MG&:y����p�c���n��L�(6J.S�#����#������JQI��y�����s��q�5&���co�`��>p!fI\E�8����c����+��s�:<(������ %s���x7�5F�A/��v0�F�=������/�����A\��T	�t
����������7�����`��y���Q�:{qa$pi��X�I�U�����Xw�^�|��1�r���@P-�����E|������RH����p�W�Y������T���3�1��1i�Z
�CV��f�L��l�&�v�)�������~=?�`����UJ29��
��-�n����~��R*�~7�1D��0 �APDfc������t�s��8��I�[�.=���m�C*�p�]�
��������'�����@�2����q���/�#S�+��*��!���v��h@�m'm[]�����Z>;l�3�t�U�S�9]�������B.�W�c���d�X5���]�R�+�(���&'(�F/`%���Rt	�F}����6��p(����A���w|�N�e��8,(��&�����L���CKe]�3y`w�1�\��2$(:R{f���@J\qh�V���������G���&4�$�lj���k��]F�1�v��`iQ���+N��3���%u���L��B��5���9���R�(�W]���*v����e��'�qc�x�	�}��#w�U;��>)�8kc� �j���4��y������]*X:kc%Z�L�)�:fG��1��o�m��-
�Sp�XL�0��3%oQ�����LOy�D���KB$q}��_��f��R�\�.�`�z��gL�9_�.z/���N��������C����*���l�\�T#�0-0�By
��� ,����t�K��R�w��(����a�l�`.fb�}��|�/nN��<G�t0)����	7�7Bc���J�����5g����)�Q���p����d�������%��0z�GaH
���+�g��I.�����^e�������7>��Ch�*������I�����2���W���y�o��kB���'���L	h�!0Z�����`������~��9�]����G��=;{v�����g�]�����G?�]����'��582����]���9�XX��/v�<�v��������u	���e�����~q���(���g��M�$���5������[��r���<����e|qW�G��^�4����
�U�C������}���.�[������x��^��{��^6(qu���>7&'�J�	8�	_�O����Z����3�z���^�i$��t{�7��X��Y��Ug�;�@!������C��x|s��'����/��_��;D������'�%�PD�s�������K���8��Z�5����LtGj�VB���k���6����b��4�#%����E�@�e�%��!}�M�K�*���2�5s/cBLM�����%3e}t��p�H�p���f�@��11���yz;�P����z���j�8jZ���S!<eF&���}�bCv�������)4�+2�~&��8#��>l�g��Jz��:�'�
�CZ�&������b�q5� r�����j@n�d^W"�tI@rMaw8�XL��1"��|s�B�P#��O	={��O�Y���������iD���h�'���/+��pv��s'���j%6<�((��Gq�_�����p�]��S����,��S��&����>�����e�8��O�d�|��>���h�p���Zg��S���@E��F�@���!*���/������6k�8��� 7+�wb��#<��{�xh9J�/����Z��C���[���0<�U��0��e����^�����
�rS�	#��s�q�S�:��-�`UL-d�������f�i�3����@�K�~�a��96F�L��Zt�T���3��Sz
�T4t�OQ/r�N��|�� H����PsQa+BQ��6$�iKU<Tw�j
�S&��-^E��U<_�'�����-�)����cnC 8��������IoJ�3�H7�M�5�7��l������I����\yz3W�"	�������s�
�Q��m���`!n{�La��_��q�%t�
��!U��R�B�����v��
�b��AF	$&�/V�r���d�B��{�����s���~��d2�sJe�tYr_Va29���L|�N��h��b���F{p;Q��q��Z1&Z	I�[�'g"-���<m�a/W�h����U�0����~�����*�w,
�U`����
��������1��^����g���z���W�o�z��8;8H��,��������CB���=V�R27�?z�X��-�NlcB��� ���`�+�0(�p�#�����mJM��|u5X0&����\El����������R�R�,�d���+��������E(�����G}���xs
q�������L���B�DC�i��;�F�#gh1���(Am��k:�5p>6t	����jtw��Y���(����Jh:����<���������� �k3#����}�7�
O�r�:b�5��8�`���������b�~�.�c�������k�k�n~�$�D�xk�����5���z�S���X����mQ��kg��n����G����~��BP1������H�`?l^�>?������>o0���5?m�Y/��<88�]��.-�}��-�Tk�@L`��.��T?� ����
�F�����y�TIE0r���?
��H0�������h���~q@��`�g���g\!�l�a�n��_�ta�Y�#�����qB����������J��Ga�D����hn�d��2��m������w��&s�Q��;��8��Tp��/��L���.&���&���]:E�c\��bL=H�WL�������;������[�� �C�!Q��B�z�)m�_�~���_��OQ�����:X�)��j$��0���3�X���Z�m�C��-b��w�+�;�W��\����L�l���}�m�w�nS�V}��n��N?F;����H���i��J�R���Ji|��`����J�������YB��QB_Nk	mQm%������B�DU��l���`+���Z��OTD�d��%en�#������4Ty�����"Dth���.���b����CO�I/��)���&j�]����>��	.#h"���wv���/2��;��Mn���9�_L���G��_fW������e��}2E��`
�6zP�����aO����A������0Nb�x�z	9���'�m}���@��(�?d�m������*{�H�p������w��\^�'i�����09�<O��a|�\
�����|��E�����
��b��;P�W�)������B�
���1���2JNX�Uv9>�,n���� ����.!F��$����������q�G���y����_E�����������~�����-n�(l,j;�����@�:��9���l���7+=��-��)KP��';p������Wf�;�����?u���%�cZ��i���:�9��PVA�<�
�<_�!�P����CW���oeG(F ��s~ye4����A��ze1f��sV0Xz2��f\���b����v�v�@2��/�_����@�A�r�\�Iu��b�f�K*�����R_uv#����+��)z>@t#k\�{Gu��Y 0���@78�����h�zE�pGND����4'��2���	����*�G��>,g
��A��7��p��y��5a���=���������h+��������*��L$u'�n�r3ad�����7�%������t���1e=����y	���B,�
:mi�a#/�Sp��:���Q�:4z�[��yM��K
b.8j</5���fFj��������o@xk�0`l�a�������0Qtt��]��>���8���e7::&M�)�bgo� ]�������;>y����r���:�MN����A|e������jFQ�R�f��"c�t`'�9E6����F�\�v�R�B��  �� �����uw���(�4�23�A�o�����n�k�e��=�#c?�Z�)��D�l���D4����~�<���Zz��/^�����_��N��O���A��B�66�����j@{/�~�|?�Y�AL�zR��(��HWG4���������rM�5�DI�FU��!����M��0���%5�����3��h25)��d9���n��������8�'Co�d8�3L�stMqI�E��-�R���5�|��K�P�_&O�_��PI���7`Y��E/xVj�	V�@tP�>
I,�B�;t�7Ac����%�c��c�:/uMv��@�S����`�>M"���PD�}�.�Y<���H�m�X^�l��1�|����%�sO�B�C�X�<5�|O�������pj��dK>�f��5G'F�����pdDG�^M!X0�hbE�<3#�+�/��P��h<+�ke�����HK!��Iu�H��)= jV��~^3Z��D$f&�?���+������x7h���(��d,�Q���d+r���%��VS4�7��s@��`I�;��v�{W�(�{8����2E��^�����0Q��'O��.5����sB�������Q���x2���D	���]]d��,3����������rJ��F��5�����Q�'��'�1�0����n6�qP��~��d@Rs�-�6E�v��4<)#��R3HC�k���	�KTV��[8��������N��O�m}i]E�}m���q�i����P�CH��c3S�Ew,�"k��b�=?������v�z��I�I_�@��DP-�c"���?j��b�9]k"������6@C3J!(�JM����6����|(��"w�X#�J3��S�
����5�����eP��#t���Ew��1���c�H����-=������}, ��/(H�!��������.;k{��8>&�d�� �92;&���0�H���miK��A�i/!L
��d4V������Ug�&��w��69uZ�Mn�b���t2]m�Bo.��}���4�/�V����@���� �*����>��y]�J��}d4]�mt�����z�d��w��ym��H���Ys^�3�=	�A�`Q���
Z�m�bIfh	��\����#��CT���<�����=������4����o�����^������_^�������1/��8]^������(B�H���
�����_R��e�]Y�%,�%_�2����+��{��>�������^\N�>���>��r
�t������b?�L���>�����=�%S�{4l���r����p����xxx��������x��������e�?��[��^�l��a���S�����c����G�/�P��)*(�����v�����7����B��v)X�hr{��Ub4���sT'�^]���-/��"U�J
���F]�vP�b	B���/��Gc��!�a\�K~��	��b�)��\���������i��S0a,#k����e�a>�8`�=�%����E�5I��lq�LL�w:�]9���k00�^I,�^�b���?��L�ur�&���� ���e������V
���!@��{^sbq��`���N@X���R��~���K��������������
��^�E;����A�2[�G��}!�[����\���T�T���w�N�[�H�%8 ��g���7l����gr�����r<��wv���~vQ[0%S"���^�^/�������!�c�`Y��d1�9~���EKfP�:�7��D�`�p`VF$gZ)_>���b�G�UESk9S��9�
L�dwNR����Lq�t��U�v�g��w��8c+���+�)E}:UC��.����2l����3�G{G1S�#s<�g�GFc�����DM���~Bgp',�K0�47�q�����o�����0��s�U��O��b,v~������Pl�}������#��0�����6C
�����v5�@$~�[+oP����wS�����5&���b�f�����-�`�����"���_N_��a���?{�M^�|��v �
X��� {� ���o��{�.�b��e�����
�*K��D7Q����������n���D��h[b�2��("���������y)�7���`�(��'�O���}�w0�Y���.J��?�G����8�@�e������n4����
�T�MA��
��{��[����D�����"��l���.�O��D�`�t��C�F����5�0O�(6�^o��E����}���I
�U��ld�XCI��o$���!S����M���M�7��?���-���W�������4����`��3��,f�`aEzh<p��V�fsi�	�6��f3���F�Qe�6I��@��f��=I2S%���(^�[����K��<Og��9���������3���b���f��p���#�=�P�������ZA���G6�����������f2���o���]�����������:1�����^=bF��;��������!�������a=$�Bu�it~��������<���_�(�N8��b��U���+��;X������y0����������]i|�t�X����C�p�t!�8Ou�6q���*�f�� ���M��7�VT&��D��L�oZ���v4������������R�|�60�l�V{{q~zq��_G�^���
���������zlcT�V=s[���g�����cy�e���]�~lwqj�Y������q���MU�����'*�����dU-�����N��V kBY6��U*���sF�q j��e�>��/��n|��zC���i�������N- r���,�o1!��Q�HB#�?92�����d�w.I�q$�f%�s�O�����������)j��d$�Bj��p�I�79��c�.�	���[����:�E��LA�=���5��C6����<,B=�������0�W�p1��~����e��?����2�?��>���G_b��G�Ul�1Lz��?N�^k�q�L�G����$=:����� 9d'���`x�N�'������j���q/�9��������.���h 
>�7���`g�?��
���/��Thh�s(�HSop��fO�u��(���4@���ll���,�h��CM4���zq�y��Bq�aI
0X���A�?A����G���v��2#�fd���`� ������������0�f)B���vh�����+;��<]��p������Y�Q�{G�?���45�^�����A��e�,X����r�y��)7��H4;��4�(���q�!�7mF
��#�~G.��|EX���W�k%�PP���h�]���}����Y�aK����f��X���_���`�'� �/�#�L�b�h��`���?E(���Cw����@��t��qc}eUD�q���6[eE�4��=�)|�a- ������eU����$Y�
:X���x�~Xg�.
� ��;�1*�����t�����./!S���q��'Z�Z �8ig����k&0��|�Q���
&��9�S���5?'�$G����HS{M���Sd��.*�b(f����b��H����3�?����Vc+E������H;��g�Zv
�"���
$������A������/B������Mn�NG��Q�&����s��,����A�e���]�J�
�y��"���U�f&i���������azi$�)������h�����!�lo	@].��_�5xK�tG��� R����0qK�c���A&�'�����~AZ��j���;���/��A��!�u�}[��n�bo����O�yF�{q���h���p�iB�P����s�l|�������-�$�V��]=J����M��Y�������+�tE��\2*�9����lX��d-,������7�)�_��.~��;�WieV��W�1����ei��
i
8����
+��qcG���s���e��V��M
a1;;A�&�)�L�aiE�C��#j��NC��?�x�����{X�$U�(�7��R���}�v���������������&�����2��1�w�p��1��^�\N������Uo��d!(��0p�Q��������`�g_F?- F)f��\O��4�������1*2@��y�jQx$)x���7�>tKr��H����c:���d+�R�����[��&b��z1��It=W��-�����g,_
<�yM����2�]�	��;������: ��]�X��y&�|��]W ����W/���*wIIrD�
���Cy����lF!�W�!7�^~yD�N����8�ybG�5d�Y{I�4I�au#	D����H�<�gA����b���.��g�/J�"�C��oE���
�Yr�0v��}���Y�h�~�.�x	�S�#�$�����d�=[��c��]Y�z�B�����19�]�;�9����.���y�)`��N�����v	s!�R]
��Y�j���)	7����[n����E������������\���G�m���J�J����#Q�5.����d
/�3���}_F�����dhQA(>��s��������u���1�!�l����4D���D9����c�b���Ki��tD3k��t��:A�_FN��)���1��[�'��O��R�b`�A<�&qfaK����zs7�4Q]��z�&(������������������#};EHFrX����������A�K��0gN��q�hn9��@�gL$$�k�*����G���\m���5��"dR�T�X��j��	>��@��)�����L� 3V��E`0�B�����TO���;SSR��)�Iq��I��U�%�2%Cj�KV�&�4�Dr:�i���9$z�!F�4zy�X�	L	5��� t������s��;�z m[�g<������Q�r��*q/dI�������s-,-�,t'�^^���[�bJU.�t�J!�T�UB�t����]9���"�<A�����ZJ.1Kx��I�^FtB2VAvy0R�<����x����arxppyy2H�+F�����2��H�F�;f��!K��E_@.���v:?��B}N�o�/������V�i+	1L���:^JA�<+X"�t);h���8+|�.�w�W����f�8M���-Mf>[A����OY�����~��fW|r#�3$}�B��������c�,�>��h���Y�B��, #�8ERV!�����<>(B�6A��H���T��{Z1��g��S5�/�E�
����<���Z��6��<�)v@��G����RD2�-�	=��Z�h��)���2$�1r���Y��r����E�?*��y�����$Xw6O1���*����ag���^��R�"��]���a$C��������_�D��	�)�H/�_���\�/�KAp���%�T��}��N�>tDarv���gq%���0�+��C�R����jI+���������
g(uJW0{�d[fJ������@L55<������"w���'Q\����Z�T.����=�nW8u�`��)@�}4t_�/�}�I��<�������7����H�i�E�14����}����:F���l��4�S��8��.J�=O	M�4���-���5��.R���9�����3���4�OD&����FYz3K���5?�9:L���@ri6���slzI��8Q/G$U#	OX�F<}�9�~���P���/�7E�E�r������e����K�����DQ�*k1M���'1���:�_:���!�����x=�f�K�r6QR�y���/_9'7�O.k�hJB/vfB���������;�9����t�����|/�12Zgtwt���;y���n�^8��E������8��^�*Z��{��,kN_�+��|�_r��u�������������78{��/�d x!�2�\�	�3�y
�A4&���,SE��� �2�tg�C���O�2By���,��n�E��^����*��!���*�sm����W�~)��	�lyu7�_�=s��G�IP���vH+r�'�wO({e���3J��
$#��1�����M��' u��s���p`s��=9y��I�Vf���`�|��S��Y|�b��M�0�����7B�tP�!OT��mI����	�n����1Dv�c�{+O������2N��)�'F�N)�45f\�p����m9��w���o�����W+�U(w����_g�u�A��N�/�^D��]�W^������5q����V��\��sK��:���	*�_��Sr����p�
��h���/�������p�3�x"<����"�UT|e@(�U�D�{�����g���"���PM���(�Wc�O�[y��zl������~���y��G���J��M�]?��z�� p�k~���:���G��T������q��)NT������c���(Q6d�*1�o���J��!��4�����d��7���K����,X��$t���Q���n�^�[m���l+u�8�����Z�z-�UN�Yn/�/�(R�;���,{�Y�"�[	/Q�%��|��&O���������3"����42���`�lH�0w�Vty�F>�.o���?� �-�|}y]����b��x��Y��f���{>������L!f��������5���r�9I>(��f�y��e�M�����	Q�����l?dNvL��b5Bi^�3YO'������d�R���4O��Q��W�;a�)�>y������#8��03�b�3�4]�q����b��>����D[���0 �J8���5AZ.7 B#A.�7�4-���z.#a`j*4D�M,�n{������f��P!�o�������0N�f��V�kXA<�z"%���F5��^u�uK�����cp���+�A
.�axJ\��n�%K!���+�h�M��)��_���Lu�g�-:�0�ai��L����Z,�L��U� ;k�v$h���a��������i]f��#��$�c���������_~y���,X�	����qj����% ��">���h��6�������]���ZB�g�����7h�&�<5��\��������<�o��=�du`r������z6}G��,9�l�R���n,`�G0F�`�p�w0e�;tb�/�`��^���%��<&xh	y�Um��Hk�R{7�0�AnB�
Yq�+�$+K�s�YC��	�l��=z�b#b��.
q���^^�L��b���yn�X�!���S�I�C]D��f��t���Z�M1[��_����iH6Ic��_-H���TG�&,�}@��	�������+Lky�Qi���pB������0����9���2V#��=(�&T���K�
y�U�M�.��5{q�����v8����"lD5�Mz`�R[�[H@z����@�W4��N��|�f�.
���]$q�%�}d|GdY�9_�>�F8���S��.;����_���2��?-��uL�}z���)��u;+h�U�A"���:^b�8'op(2��DZ��\<�����T+ye���������U����fw�� w��!�c��-����������J�������	�e���^�sX0&21���0������t�����Qv��g��A6H&��
�
����:
�����
�������/-1h���nT������;_�\)Eano��L�����.���������l�&���5���=�`���� ��#��b�E��_�"C���{a��A��R��=C�
�_��4z�����X0!����$����'B���)�= �����mk�����|q	�����BPG��b 	��-��R2)b�����%(����+�����L'��{�[�b�K
0�;�����I2H'Gn`�"�,�lN0����h�I����G�w�Yvu�� Yr/���^����{4o���3#�%��t����D�xc��[����pf��"<���prr298�N���a�^jk%���C,1�#�����B�L��tM�[�6vO���y|�*�uT��d�:�4"y��@x��,6������<�T�}�\��U(�!���n
�s����{��X�����Pp����������"��_f���W��9���~��>.��<�{W���$�&G��4M�dp��
N�W'�Y����p[��!�d|�n�Y��������������9�dS*CUu;
���#-������2���Dp��X}"�H��v6sU���R��\x$��B��OB=^��@X����@�������pQ�mFC��	���>.���]p��>�8�dxZ�p�]�l6�W-U'"�"��yp�}�;����+9>�q��fm��khq�"A�j��3�D������T`m2@��m(L���S�Jv���~G��{0���}�E�!�[2Ld����f�R��������5�P�S�����@�����d"<R�" S������G=���s3Z5��Yv��!�=r�a��`�,/��-�����S����c3�-��F��~�7�����e9F��L������ >�%��$��J�I*e1�Fc�Gv'�,��a3�@�+bu���b�<Z
���PH��$�[���{[�kEG����H�z���zKXH��s�I#�����������W� ������7����o.��v$R���!��HF�����7��2J��Wn
�'����D��`m�5�m*L���fJ&w9���,sH�R���Bi9���(=>��/����&���,
���	�}�?hG���CR?�$��W��3����G�p,�=k�s�� �3l�JT��N��Cm�C��h��*�;	�p]ta�oY� z=����ZT���S�1��+����X~8��g����0]����F�ZQ:Aa������b�}���I��G�JW�6z��@.4R~6�JmTnN``]c�	�T���+�'�YBtGkHcB�4�<������}�����T��9f�H�(���^af@f @i"&^��^a�;��/���5�P;�C'�8�u��tC%�vY7;���N;X�n�)�f�� ���<
�"H.���|�VJ�����i3������	������+��}7k>�	:�W���?JL':��l�+�VT��������JyCs��������CKC	���oQ�oBg�O��8�?qar�d�H����F�]�|���k�r{�����?�������y~�,�����o�������*��`0<��C��'�[��>.����^:�O����l����� ������wr���8MO�����N*��}5<�*��?� �� 	ez�c�UD`4"0�����x�������[ d���Vj�����4{���O/����������~��?#��~��������7���9��$�\6���-]}�^��"m{K��kJ-7��1�*�=���K,���f��\�Dd?/�k0d#��y���6�E�����]:_�L���=>������3V�3����$*W����B����$9��e��$�TT��a��
����`E��W���!o��q|�����%�z�*XP�@E�������F����S83�	�Q�v�`���sU�@�su4��yv��{6��f
�&�4�n��3]
N�����G���s��������Fg���I�s�SJ!H��[�4Xm��lcd��z	M�Zg����;�29�����G�&�V���m��)�����P/��g�.��^D�O����'8Q2]4d6�G��.*���s��JT��j�D��)��A!
��eZ-�3�|�S3kR��/`�����,�l!�)��>�^���|������������20+��`uP���Hx{ves����v1�ZY�
�y����og����b���N/�����_�Ew�u?���r���m��J��rE�������������ct�W
�yA+&�_���W����^���9��v�D���������a��s�s�����z��������_D/_�g����Wg�/"rv����LJ�������������Dm�~�^7����������W�/�3�'����8�$����7?����Q�X�!����uh5���<�)u+ �yv�� ������5��;0fq��?}���K:R6s��0�-��P
B�m�-��Ko�'��zLP4@(�B	����q�9 �s@S ��Pl����*P��w����:}�:������7o���f}{��L��]��X���v��	�7>��	��U��:��?~"0~B�p(�������(k�9'T2�%����� ���������kg�E~���.�k�����nCx������o���G��E�mZ<�����\]�hH�������WQ��[�r7WZ}����dXd�����1��K�n�1�� �(X�K<�����x�-���tF�Q2w����[a�F��[3r�{��AQ\�Q�������+���
�����1~�����K]�Q���Y�1[M���]8���_�[)_�����2��������Y��������$�W�1���8JQ01��6?�Y�N��G
��R
��������M��{�&�7Z���21a�����f�P;h,A]�%y�Wg/.0�6B�������I����9:��dP�z�d������W
1���-I���GE5{\5LD���
���z��{�S'����^>�Eb�n7a����^~@����D������P����+�r��n�%}�K
�?��[�B�_������?����%�%F�'Ii�ed��������S�5�B��R�T����T��~H����5���5����K9f���N��CV\�;�~���#��PS��o����B��{e�~����Br&M��B�v��Ab5�A�k(<���(cj�M/
L��I+����Y�sH)"�
�����&��
��o�M��fH��7��j���%�s����W���Y�����`�*L�.���__����.�������@�����>�+�s��&0��O���?��=��/`�EJ�=R/��@�6�q��	*(9�m�u|"	�7�����1��=��
Z�M?-<�I,�=�@���� E5H��R)��$zo=����Yz�
$%�1��=��
Z�M?mH9���Dv-��D��
�$^@���H���db<���Z�A�����}D�=0���6'Ue�����~sg��`1,��b�3m3Z��l���O���]Z���>x���\�n�"�>�/�F�j�jp"�^������;�A��L����ne�K���>���F*�6!c�9�%{����{L���
:�7��h1y|4��j�^P�I,,�u�a7���w�	|f�?����
�q%��g�?Cp{lg-��|��c@�X0��UxX�gn������`�1��^%N�[*gb2&*_"����$�����fI����2��J�Qft��Li8���Q'����-@���0x���-+x��� ��	N���!��N�w��v7��p�V�h�^>�W����3�n`���WFv�
�3�8�*�C�i��L����r&>]pR&�����3�>K%,�ad���!>c	�>D���Am1/�w���"���R	Ky��*l��X�����sP�����0��;)&�7�����gMcB��&�������a^��J�J�(�_q�R�-�����{q���b\�4�-�?��4��t���������G���.]��*e>�4?r����������!^�T5Ym+V�x-<s��n�v;~R�q�1���[n�����[���$���\(�	��C���^����� 6������B�AE,���P��O����_Y_;�A�������p@���[-Ss�U��qZ�t�0��D�y�����,��m>������>��+L�
I����u��J���k��Z,��W�������z��-��*�}x^v�Iv�=�	�m]�N+2��/�����U:[e���#���h�L���f�@�����x��Wa�u��Yk��{��Ky-0���8,���C�M��rlU3[���Ml;%�AS����������'�I����9�f��(X��--~1�}J�����U��>�Pn4Z�]�p"=@-���i��R�~T�hE�(����-��e$~�����.-uS�:�P_z��v��MJfV�����;[0����������9���'K���E� ��Q-�E��g7�����w���
N���.��S���Py�W};y�����,���
���&{|�����G����J�QE��1%���J��=)��L�MaU5���?o����j��|�I� ��T-`<<���'��v��KM��J6@@w!�#�r*�(f�d\�V�T��cb���������|{q�������
�U�T���C��I������iT�J����i<��#tm���cg`Zs�Fb��a;����5���\-����w��4�Z2\��66�Z1_�a�\�N�|�t�2?>�%Qj���{7�[�a�0�o�;���H�H�F��9��7�6-�~U�:��#9d�J	t��h�]e�3�z�H}�������W;{O�<����>8}K����#!?fY�Z���B]x��V���j��<�+�V+���OW��R��_�G�`E����&��Pg���:��N�^t�^��v%�����3#Tn�^%��*z��]���O�PL����p��t�"[����cJ
���j�l5�2�YBe�x�����4�l�Tj�kPZi<}�����W��p�U�J6��=O�]l*�X{]ykU�JQFi3�Ao���n2}�
�)`���xq���B�2#Qz���7�7���S�Tl�^���&5���+A�a�����l)�+o���@��'c�<�(���-��S�4�l��o���^���mA��ml�fN?��+���EJ���3�s��Oc��B/�����YyM�#�)~��]��-B{����[��F1��?��_
l����QY�E��3yh����F�B���k�����(fp�����������zm���X+fS9��������6�h�����\�5nZ�$�j\�u���2���%[�J��^Q������[�V��T��-nk@�$eh���d�|����e���D1P��J����_�/�����p��<����7�d,����.o��O�7m�@���`���e00���?(y�h���gd+D����1G�N��.5�M�w�rr�.w�_N�/^^�$L�����g�)��w����_[�*C~��o�����/��~={��{�����s����F�P�Qg]�_]P�uH�z�����E��5y��a�c�����LV���������2Lq���e��F�l9�V�^7��t�x7���O_��������i�<�-������o��.n��h�9���e�����>DW�e�&l,��[�d<��At��.�e�������(��^��s�a3A�e�����K���Z���hq8��v@��Th���u�Z\����&��@�-�D@��F�HhC���rG*�����"9�iL�=���'
�5��diM��
%ie!�k!(��@�m1�g�X���*�h�4��6+�����vN98�!�US�gU���[����w����{�Z���������Z�B��&i��f8�����~U�����<��L/?��7,Hi*7H��,��?2�>J[����:,�t49��4������3�a���MY��*pw��p�t7�������%�L c�L����&�@El$'VD�/��&C�~���G���(�$�B��nI��)��HC?�h	��)����9�&z*�����S�$�l
�3�
{�dc[�M���o��F�����CD����:�����8�ft�H�2K|��Q��f���������6�t�-4����|�������7���<]v����aNF�L��(�P����E��d�l��{M�n�^����}-�V��R����)�1]����.o�7�
����E�a�0�dw�
�y�D��4ZL�m^�\�����=������+7I/}�sg���0�]����ko�0j uC���D�PS��P "�V����b�b)7{���l�Kz�BD+�v* ��M��@
�*(w���.����l2�:HA��������>��X>4���?|��!I�3���b������lA�K�
��G���2�@����=�����'�
�L���Dv�`�]�{\�mHk:y}��F��]�[}%�������?�k��8�e���R�<N��'tn����h�Az��O?�O��n�>������"
h�?����J��������7PU�J��V$�mK�"�� D
/o~c6+=58\��-�E�o~��;V����T�`������z����xF���a"�K8��{�����F-��j��o�+����S��o�M��������*
�DP+
m������_�����V�JC?%i(6�JC[ih�JC[i�E�M���V����D�4��d9��r9�,��G�Y"�����:K�6�\c2Yg��F�{8��v��;�����,�<��OU�mI+��\����&S�����y���x��I�&gR�Yzis��s^�\�E	�U�UgN����k����+z)J����*�V��U��5 /��"���*J��z��a$��E���U��@�o{����[��T����U4��&_� Sk@�$;E��]�Q��2G,G^6��V�A�"q{����[��4��q����>yOk�Fo��u�=HL�f �n�v=�-�U��~��P��V���TU,��Y�z5lB�D'��L�^,B�6K/�<{2�n&��T�����>�.�����h���[�����h���*���L�Q���2Us<
�TM�Ri�UrW�v�z����T��r�~�e�voK��(LJ[�jc2���>�����4�F�T���&�*)�I�J�j���������&=*.�K�J�nM�jHLC���j����C[��v������?��������^���.���vS��U������`��>�2��������(����t.����aJ���f�9/�'��(�M�Bi��i
��Fx������z�n)�oz�H�_lf3V1��JO�cW?�K�UD�����_��x���x�s
GTT�o�
��I��<>��������'�t2l�eK���`�bG�����|���u��l�DX
\��C/��Rx���b�
b��c�����buT�:*?q�hSCd���J%0���xY� ���|�� ��#���L�HI�+!��	�-��+"�����Dd��Y�-&�+�W@d��C�<"+����,�� �;�����,QG����%��Y�!�R5u-D�3�@D�(�,���|��E#���J��f�+!��	�-�*"�����Dd��Y�-&�+Q+T@d��C�<"3��s���~���!im�W?y�U���Wo��Z�8�r�VP.��%e�
i}�|}��a�d�`�M
�(8dS�l4,�i�H�FZ4�(�0������l���o��b�?'�F:�eQ���2[����*����f��&[~��(�_}����e��2��7�*���zAQ��*��:�U\���S��e 4��@idi�dU�����}]�������'� ������3���|������n,���}����d�aoWWc
wKuc��=+�\u��t�^����9�)���g}:���U^6�����]�GR)�-��>-m���UM[���G�_�e���k��*��N�h�(�i�V�sZ�*�u�Y����2���������c����m��Z>�\%xo��0`-e\
x�Z�0`3���p�Z�8'R7���1`M%�MI����:��*4g�2|�J��Ty
a�ZZ��>�w
`�f�v1`^]W moTK@�R|(�'���?><�%Er�	���(�(�j�]&x����Q��)�@��)pwid*�k�iDmc�U�M8*���4y;��;R��s��)Un
B�<�}�:#�������o��
�3f������J���2�FD�69���c%���F�`s�Qx�:[�����bq��e��
�S/�Z�Jb/��tY�`2��W�$n~����v���;*9�HW�P'������g)q���w��T@2y�I/-������	��%cg�s+��fT��aU�f��f��@�Iy�������z�B��yM]������dPs����6IY����`��&�:��H�M�M2��I
C�v�2?��%�M2�/R��w��dd�!�I���,<�uY�C7���&����2�I/
��&������dS�lL33m�,>��&k�[J%�A�i�D�gP0;������|��]6I�0�����9���E��_'\�k�W�����t�^�&�d�=`�St�N�C����Rl|�@>.wD���x� ��^��=���@RM���:�����-K�R����������s�w�Q����x�#�;�|���|!�&l���N/
"��1�/�c����G��;f(w�1�1�uw����������{��3������:�4�86��M2[�L?v�+d�6�~����5���m���"�b1��t~�,6�h�������2�#[�_D��NF7dw�G���hqu���rN*��/����^��i���?�)#�G)�A����:��j��].Q�%�r@������]�b�\�.���G9J���q,�:�N��������*��*1�J��N_����Q�'�W�I�{��O�q��8WI���t4��������V�&4�\U=����P��I�<������>R����H��w]�S�(������X�=��6{[���8��9������+�����W��&%1w/�5�n�L%��VZ���W���u���8���eX�^��������U0x�9l|���������o�]��\��P�<��������#�����rv�*��U�G�1�ue�q���C�v�%��6T����A�0��s�;g�z6���Y��:Z��h.]�yFx��8#����@z�<@C��,g�%�PT��Y�3�:(g�*�u��(����Pu��^e���0��W���u���WzF�c@zigD\���H�V���q��L�h3��:e��:�l����C���C|�C��n��jT�EzF�E�����l�>v�:����uZ���%"��e�[<�f�^��<�\�u����M�z�jtm�m�vm����i�=]��
��3b�0�})���������`�2�@^�����j��"|L���((�b�\E����)6F�����`�9����E4d��k7���/�(�bD��"���M0u�J���Ra�C5�D>6���a����"�f���[_o���0��v����"2t�5w����?���Z�?����y����%��1�Z������>��)G5V�K����u�V������!���k������u�8�kC�^�#���&���7�b���a�G���L�<-Br�B��f�k�h���Z����������Yk]�Z�F�p���tC�:��Yi������o�����
Z�j&Z���i��g3��[�|��u�o��o][h�l����m�k#����Q��rm����`B���Y�7bd��h���I�� ���|�F�k��b[�n�v}[H@Ze��������4������f����B������)l����Pn�$g������)�\�����N�:D%�[�)A�Qs-���*��H%��mV���"�X���qJ�� v��@
D�LTB	��.���B`����.��Q��V!8L�����������J�)H^]%�����;����U��@�i7���h�K���A���Bug���oO^��"^���?��J��{��m���
�6.(�yp~�M�}������'\N����C�T��(��"_U��vL�i�}��ay���G���O�l�r�����vU���Y:(����n�W�U~p^�k?��R�u<����U�n���N����"w��M�����i�Z�\�jM��A��[P������ �C�s51������A�i[��]�������k:M��5[�Z:}�0}}��xh}�Z_=A�
6�=��e�?�D%��m�]��e|Q�(���5�r���}����s�2i��-�n�[h?���0Og�2����|��0��]�Lo3�j�?�Z��<���<_��W�g���v��<yIY5P�o�=e�~$��,KWk�h](����j��z�	[&��(��V+������iIl��F��T����jM��bqgP�.[D>l��Q��������J�JV�F9�S��`-2���AjX��m	��V��V�xU�s����J#i�YQ���?�$��X���F�z���"�����t]��1��K������}~�#,qBW�-r����J�$��:)����� ���t��������\��;����E#vA����{�	�3�R�O�%�����?�������?���.x
g��T�N8�
�X��P��"��p��6�88�Y��3��"#�� k����mP�����(R�� k����O�l��ir���������x{��s����m9���-�^��V�x��r���
\����Ho�:}�o09�R�-���q��h��6������&]�����_����&�|��}~
�����t�����9������l=��e�iX�r������.N�uF-�n�NM��.�[�M�w�rr�.w�_N�/^^�$��@�����:��u��m0v��V	��������7��_N_�z���������]�g��_����,W��Yk�W��k=�_��������_D/_��k���]�o��\v��O_���)�J��<[�kX��4[�\ON�`�8%��>}����=�= Sly[�%"��?��*�����5PD��,����gW�e�&����&�����:��������dX{��?�.����� �9O��`���C��� L�����pW�������0����Vo8���a��������������`j�/��ck�C������C
�?�����]C�/��BS��U5mhto�C%�����El^�B���h����?�3; �aHc�ZIR�@���H�Hj@�J�-�_�0��RlX*H-h��W�:Y>�
Y?nkd�h�^���o\��~��7A���o����h$!	#��1�Xmh�N�����*c"��}#cA��c^���
c�:MCYx��<���b�Tl�r�t���jC�����g��2�2}"�D�#cb���^���e&.�$��n�>r��X��K���i]���y�qZ���`�r;b;GC�M�r3�$��$���~3�o:y�s��m�k�i��-Zm.�cc�����`sV����^���������2���nS,�z�`w��~Y�ec��3&L��=H�?Lv���S���S��f6+V5V��zi��RRJ-��/NGPrC�nO���O��!�Jx���p&����/y��� ���d]���^����H�:}�����L ���z�j�b��6��
������rc
rc��!�L�]8�7��[�����?�C��P������&�z%j����FV�M�M4�M���h��X 7��e�L�3�{�
�%1�!�\�Gu�-�f�D,�#�Ag��x�GM�6��X4�;V�B������zq�A]�����e�(�w�����^�����
��-�!��p����@s�L<@��B�l�Zk�K]��<fs��599����o/:���zF��u5T������V��=a���{O���tG�}V���cv?q�k*���z�������4�T��*�g���h�Aj5~u:K�'E�S����!�k����sq}��m0g�7�b�Y�
����3-9H�����9��?��V����[�?D�w����@���@g����C����O-6oX�E�F@��M��E��U��C��
t��E���e���Ypg���'�+���0h~��>��y|x�%�O�]#5����=���is�v��B���w)�Y�]���D��jr2A
���6���D?����F��
����&@�.�����&���� ��K��Q�-��7fR_p!����2�&`j�irCB.���mz��\l��.�JYm����_w��;��mb��+YU��X��u��u����1eS��cuHnfNU����Zj�`�\uZ�}��6�8����q}\��Wx��P��r�^h2w���z�����N�l��C����a��_�]:���������F���[��jO��vU�P7���
�z[���/��M��a�Z-�P�;����i%R�k8��{�9�MUl'��l;���*���8��n���!:�A[���-��{�!u&Z�����ig�Wj~"�7����?���O���\�]a��p��st<8�4�x
�m���)Wp�+9Z��u��
����o� <Z���;�7�@�o���q�8�N�
��>��;��s2p�=�7�����c���u�
�[b���qt���P��nH�����8���Y���A���v`���'�����
����{���g�'�u�������N�=W�}�v
w�����N��}����������7�ov����E��pZ�n����|�-z��]�o'g�,�+jf�t�s:���E>�
�r�>v�����
9�+x�l�c��ys����d��[O���{���r��'�*���j
*jj(�d^���P���W�Z~*���\�ka��1mV%=V�)P�i���������'���}(&hP�e��t��
������<����
(
��/*7L/o��s�T��U��ke��A�Ve�V
,��Z������>��/�E0FiN�eb�����1Jl�(q-�������(�	��v*c��
���(�QVO��J��U����Z�:%,��l��m�}�Wa�~��s`���������+��CE�E�3\}!�Dq�*N��Ea�/��A0���+�Q`�
��5Da������CQEz�HN����(�2:^����#Q�n��,_�XF)�P��i?�:�Q,���r5��B�5T�T��b]z�����o�M�~����p���{,�bF���h7|�m%���x�b��-)������VR�J+)~�b5�p��8n%����{�������a��$�q+)n%���m%����x���#\�?�eV�\��*Qi��J�kRH%*-�O�r���d��R)Q��J���d�h���Fz`�]59F-5K�-�.�*���������!��	*�$z������&sU�Yz�������\�
��g�B��D������m�����;
���<7����j"���
�����x�g���Ajw�^.)��c���h��D+Ck`�Zp�����<*���p�*%�V56(^l@�$F�4��]LS��21MI�#O$�F#�&PTZYHZ��	���9���0��D>���ZsG��i�;~���a��q����v�V���aV��T��r�Vmt!/�Ta�L@h�=
�c�+���K6���u���bf\�YzY���pH�?u��V���4J�����9cd!��gz�#��U6#��7�����Cs��4N���a������4$�6��}~y�f�HY�:��$��-�d������ �~���$6��S:�y�*�r������(b� �U��)HE�8�H:WRX����n%�u�[Ia]�VRX��6�b%��'�6��u%����E�-��>�-�It'���m�S�&��k_�������b��F��X����na���e�F���@]T��������xK���{3�*L	�s����a���(�.�6�%���) ��5VW.8����|���#a���|X���*=U�]�T/����-�T�x)�q\*�{�T��Pu.
�+*��2PQu	�X��`�m���\r@rp�������_�T��m+v�^���j��,H�Rgs��J���U�?�2�-+���(0�� ��>���(0VG�����F(0��@_�R-�3�(0VP`l��XA��n����`���
4�@%����_�T�����;B/�[C�U6�l�D�QV���(��*��Dn�Dn�D.w���DU���3�hTA���
����}�"&

L�((0��}��E���K��f�+�����-*T*������D���P`��)�+��T@���C�<
4��s������,gn-�?'�g���P_{��,�C}q�^�Qq�!���3�u��M����p6Ag���M�8g�!�M�u���b��b���:���R����Y_]�J�t��@���[3�����g�0"�Th3��Rg�i�<W���lz'�������R!T@95<|����e�j���������|-	`���#`��Ew���B�i�x����G�@�����
x�7R��E{�����Qj�"�2Z���}��i�nB���2�����]�W��~�<����<��9g�^�q��=h
���	J�������V�ki
=vZE��ZDV��Bs�����0���wp=���p=���!L�s6���acp]W{s��v'6��A�u���i�����I�K�Y�������$�%�I�H�?�\�do/#9��H�E�%�!���A�!9
�i���{,�F���]��/�v�(G9��}!(Qe��W���e�Z6��'<s��a���Z��a%n�{��V��=��9.RM�)^�*�4�����-���l��G�7���2nE���7g�H5a[��UziK����|����1d[�����=^��h�����&lk��J/m��s��E<�C'f[�eK�e�Jn����R�����yv�������k����x��U��&[f������M-��bA^MW��r�.�G����C-=�u�M]�#���5���?b�	���>,&�G���Hz�f��_Lo3����*#��4���8����r*p�zC	,��y�8v�����'
z��x��2qjX��q���X�k�����q�
���/�Y�r�U!�@J�Uv�RX�4*����&�\�H��a�%�����)���Z��BQP�%�O>\eQ�
%�A�Y�^9��I���bE����kV��4��G����E��l�F�4<�x~~vzq]�~��w�hu�h:_w	���I�.]Nn��n��������d��-|~�����d�kk-������o��������-C:O�� �����pO�5��5b-��a�������������K���]�o�� ;���/~f��
��y�L��h�-�����F���.��)���������e�����2��6�4{�6��^����{l��X�%8#�������z�����~d]���"�����!���~������m�b�Z=�|�kh_��\���\�k�������+)^����$��q��7��X��������X���������e������2	���h2������R�Q|��z����m�	����u�W�Z���uH�2VsP�:�r`.�����R�R��Gw�r=�����*C�O�|���Y������e4��Q:Z���l]N����Bs���$Y�>v�"�8� &��i�VAtDF���	h�������i,������
!��~�����U�����/�#:���Ec��RtO�\�.��iI����(�}C�U�	Y�%�����,�+h�����>dZ%����v�VR��Q5L�H��fZ��dZ{�r~��A������h����U������f�&�|���X����H���C��@����&��[(Og�������o~�������j�t��zbt+�8��q�SQ�E����������3 �XJ��=C��PW���
*S��Y]r))�	�2����j�]h�����8v?����j�����	QO�{�D��Bw�]��Qv�-�"�m7�g�q���v���r��������ix.���v�|R;����]�F���MW�(�_Ri�^�b��gY�]4���.}���#K_��k����]]������@�*���s�����D�\h��$|��3_h4�qn����6�����oP(�-���t!�V���$����~��iep���D,�4��B�[�B�7D0����l=��e��A6�0���5�,�J���#���$fN�_<����&�u��&�B��,}�\&�����x����2���Z{��w���wX��k1�S�i1�S2v7������a�:&����8,T��pW��Js,V.��V�����tb{f�!�5��|���7_E7�%�j��RP,�3����?���}?�Z����v�L�'��`/P��F����j6��o���$NT�p&yX�/�#�8y�v�a��"����#��<b'�p���<b;��NI9��"���<�R�Ax���a��w7���#��bs5�=��\��GlG�y����G�q�G�@t�H5�9�[SrX���4(��B�#!L,q:+���!�����;,F�������6z�T�����E�z3F���f���3�t][o������J��\!3����-h�WC8"W��kHAPj^.!��=Py��q/N��w7��?�oy��n*2��Z�����H���lyNqKg����w6	�+C;[��@��W����F���uG��tv`H��A���tvD�.�������!�5wpr�Q�5��a>��C�����j�L����-B$�@:�4�hjQfCsv-j�J�"���n'�j-:�p�J]mZ";^*!7���)��m�� :�7�0����U��n�l�2��Q�������\��}�e� �<
U�-cfL(��}kP`�x�AU(h����rJ�����P����2N����CA�AAbBAR��C/F�t� -d�s��Yk�����7��1���B�+�
��3���/Kf�~�1��1f�&�f9x �!�t��pX,�~O�dOXT�X/�{�z�nV���J��,��@�����x_C�<�5��v�q��(�U�������.x�6�Q�B@BMW�����h[,q��q.��7�[�)#������e|�NvI@$��|��������6���~���j�K/-zQa��*��2�]?.�����JB�����L���s�� ���c0^r����i�yR����iBP�!H�3&�3�r��1�BPy .��x�,jv�^FT��!H���~L�	A	��D�����B���c�h?"O
��Y�����MHjj�Y����o����9T$���Pg�T8l��x�JN��\������%&7�
K	���aI���M����@Y�`IA�<��0�W���o�yK��.���$�-����M_S8�]�������}�
���Q���IWZ��������#Dn���b�^5{�^}��G�B��SjJ
���_Q�G+-�?]�u|&�DuXqy7���c�q��k5k��y����bc-b�b��q�&�V@�]����"��
�q���9�3��D�[fE��I`��5�
`��>��<�u��[�5vku�Q�?�Z&"X=������{�D\2
�o�����51Jb`���`IfML��X1k����f�i�� ����_Cub2��l�?�J���P&��+�������W~3''O���}����������a����0��qCs�
�~����_7,n�q�A���Up���`�tVN�H��������������'O�D2�'��o�=e��?�^�?`K$�XB���t�F(����5��j
��(?�p���	�k��4p��j��&�����������G'~$yb���X�n�0���5��Wg�o/:�����.�
2����x�9S#�����=5����{�������^U|�m���q��c�
������_��0����z9�R� 5|�FD�[��.�����������{B/���1�����17�b�Q�b��-��\ln�X�_�4�{��������'�p�Bv?���5���������]fo�(C���`��F�B��aC��W\s�j$ZSH#�17�cp�b��-��\b
Y,��?�	���?�����)
'�p��0E�1�F+T2�����L��*e�|"��),����6&m�S���kn�C��k`
i9�v��0U�T��`2��X�_?SS1� 9��s���:�a:S:�#��j�p$��4�����l���I[�D�>w>�kO�T��M�%��R����-��9t����� �s��F�^9���jV�'$�;7!6!6!��&dP<!mB:GG9EI+[Vd���g�F��N��1���#�����c,������?���.�7�r�	��E�S���������*!�b���)���
�1�H�	���h���
���'�2��C;e�1`P�W%VA;T�eQq���E�� p�������7��o�~�7�����u�2��C��h2K7+�c�Y+Iy^Z�[���Z��Ok]IU�u��G�D��v>�b��8P���������
�V��W������*	�B�m�l��j~��b�YS���4,��� ���)g%�fSj�J
��D����M������l8�6�G;
51����@�;��y�h�-�������V��V��Z��
)r�������&E
��4'��-��
tP4PC�l���������:}�B�OQ����v��{�G�����P~�����2��M')J�/��^<?}{�����+��y�?D��;3u��`�,�	 =����o/:��rz�!�����|"T�-�W�6��@��S�aJj"������e�eu�V�u�Y�Cv����]��u�Y�!�����Y��Os�\iZ*W2w���������I]���#�*�:���c$^�s�O���L����/�Tw�=<
w���?�����M����i=���D� ���<x5|Z�|>�-�*T
Z�3�J�����W���:�v�p��cJ%
b�GhO���AM\�7��!�1&�V��J"��^�����z�=�����c�Q8(������4��C�	MI%�pX/�UW���E��o��
b[1���F�v��m��[8S�5ea��25\Sf��S
MD�	�>��XA�h��/SC�e���J������;���t�B����z�+U�2������2J���q���_F����H��V�Nf^��(�����em$f�FD��X��/����Q*T�2��e��e����x��u����p�`�G��?��e@`��}���X���2Q�+Q���=.M��Q*T�2��e��eR.������
���pi����:\�����ev����T����!��`��<b����IZ��>�9�����-@�������dH�V�6��hJ5�e�_���I<R0{\�����S��2�	���
�|�I._���k����80)�y���[)�j	�[+�({K��$����n
���U��������Rv��������R�Xzl���V�U
����0�3��	�S�i�0?��Q �5��h!`�K�W+���������MR+n�z�������V�����_|����{�����9]�>c���
,*��,�D4MkzAF����'�!E�����!%o(C!%V!E�
��M%Hi�O���R�ls.���`�@hed���l^U~�����v����gf�>elg�Xc�D�w�lQ����I����0����|�)�r���@�>�-N�6�I��m��(�Q!"�\����b�*S��'2�~V���QP�Ml�QV��������]�w(;�C`���<���
�@>Q��y��:�U��M�$���s��h�S�����Q����^��b�=���n���/A%k�r�	Ze}?���%2���UB6�O���G�="�,T�u�����E����)��D�y��E���
-R9���G)G�wl�r�I.���mR�^_g��+���v��7F
������%]o0@��b�IUv�t��F�,4?��RWq�n���z��x
^�F��z��&�ZR	�:�����}��5K�����$��!��XkY(8�9���M��5BF�����"B��v��{����4�K���o�/! �7	���|�|)�W!,D~e��z�4M.M�� f�����	y?�is�*���O��6����DY������C=�3�A�0n0�CX��K;SE=d����d+��%f}��
��8>Q|jE��7��6��������;n(�%�����b��RA��G�Il�P��P�k�#��e,� �P(�i�k�7��A�����$P�YN{"A(�]+
� �`� �( �T��@��x�PCQ|��c��w��-����g��(�V������o����F0`LO����L�-�4��S��5�/��sr�b�����e�ZM�U4�G����M������p���oy!�o�U]/��"�:�tUd�uv��9�6���|G���Q*�k�d�n�u+ch2Y����a[F������"�_u��g!������H���r��;��^f��:�8QX>���j�,�d�L������Y�
�������od�9��^y>8�$��J�: k/�
.�x��T�W/+?���:?����3�����
{��f�Ke��N�M�l�:_�`G�����c�����e�	�|�.����8D� +P�@����J`0������;B/o��9��`\I�^V��.{���������W�W����}%0�~�Ru�F��W��c�7��\�l2N�=��2��r0�����*�k-���uT�_Lg��BC��5�#�
jl3f�s4o��]���kgA����t���,���W�*H����������(�c�#`��s~<���K�T
l<���6
&�������`����\�= �y7�5�x���8G	�QyJ����\�%v��C����}��k�m9�GbA�K�W��������Z��^�Z���D������<B��}�_7�����)����G
X��_��7#��C&X��|�"�����1oFRX�+I?;�_�������.��q���M�CD�;�D`�/�H����z�����NE�@�UT �pMv���.�Q�.,a�L�C�)5z�Z�T�	bXE���t!M��B�����z�{�VQ�(��QXE�#WE^��f�
�g����H�������������h��O����G�$�-7��2�K���-���������t~-�Y��50����9�^�������tO��������][���
$�>��~��V����l=g�����#VJ,UG���+��0���<��*>�A�DQj�i�%��D0[�~]����3��z�B$���p�,����!!��7�E����<�_��]���O�����W_�I���o��@QQP��@�����K�]l�]f,�]�I��u�^E{�G�]�f/!��[�+��G��H?9:�\��A��Dx|��n�F���C�@������@���4�<�#G�~���o�C�6���(���	����K
U��	�z G�S���
8\:�BkK���<B2�F�����4Z3��^9�u&H�:t����0�X�aA���n�/YV�&+
I������JgU����Z`�����m&��*������E^��5I����M/�F-������vg�I/j���jO��wv1{�A��{������Zyg���oe��h�LZ��>V�GRa_�*m��%�}������ )�!P���ha�����K���iy_�Ro�C�m's��	gK
��h��9Ii�����J�c!7�/4�#��is�fNXD�m_��������*k���x��>�������{��d�b*�:	���j%J�h�+�w��Ap?��l�<w�88��x��&+�M��y<��~lL3�����}l}?H�����qoxqG�����a�1�,-p=|�8�i��T�Z�J	��a��WZ��Ti7��Ui�b�>|q(�
��^�]3��g���l�do����lU/a�YV���m����(Ly@��[`.��}r��N�����o67bV +�V)��D
�G{��"������Y��f�K��JO[,x���j�]F��;�f�j+��^����d���4XI�����G��KI�����r�Z+Q8.!���j��0e�5Y����z�����9%��l�?y>Ai���KR������u�gU����pJ��h���AH�M���8���aH��f�������7���H�cF9����>�;�����G�S�pOfK�������-�t���+sl)g��M;��%yE�E���T�|�$S��r�$:����S��Q,��>m?@.������|�KD��+��TOdK�-~���B��J���M�<h�(+=��������{�d�[)�y^���)��P��%�l	���zj/��O�R�f��!��oO)�\O%]�S�*������`nzz��-����7�<=R	�K���=5E�iOM�~�S�������K��;e|y���h>7��n�-���(��_N;����%�fM
�]���{1E��P�:Q,��S����")#��m��#s�q�0�����+e�.����s��W� �)��<H��MD��Bu�d3�L���*1a���W�`���T�P�:L|r5	#�>�q�"+��%6<�����d����	[Ra�	�D|b��}������{L�tj��}j����t��m"6-H|� � ����7/jX�f8����'� ��1�!g����� ���ln����N���U��I���r�E�{Y�Z����j�K/�")I�QZLwG��������7'�crj@O��A y�D���j�N/�F%��*��� t�o�bOyL���rR
xj����"�{$�\]��������:I>����v�T��[�	��9��{��� �.���
�wn+>GY���v)Z�������,���4���_(������#�E(��6��������fu����Z\d=�������1�	K���J����f��+�}.����:4�Ck�Bv(�����6Jh+�8>3��e��D=�����1��h�������D������
����R��R��&��5T�5��
OP4g�K�~�b�t{�;.w�f-s�?ou!��6kj+(&(&PLJ@1i]��e3�
�%�9A�t�=���V��9��|�Q�����V%�]��#���s8�\��0�����fD�����o�������������2JW�2��M'��,�+���q���������l��&��
Fg�����WW�2�O2Oa��t���'k��_��_�^������{��I$]i`���%w@	����,]�������4�J���C�/����R��n�Z����ju@Z*V���q7����a�0������z�����Zt�������#s��\-�������?z�q�jVV�V�����67B���K����F`F:eNSRX��&i"Mu�-f�1����G��<����Wj�^9?w��W�D6qE�����N���7UUMw�c�1sANE��K���i
�hM���I�>�R�W	�����6*��>��`Y�����,�c�d������^U����
���n0����y*�������}�%�|������z��v�������������0e9�:��o�Qk����z%�z��[o;���N>���&�-1�oh����6Xo;�H�����8X��N>�����
a�	Y�����[o�@Vo~x�G���*�}�-?D���S�E��n�2��'T���/�����6[�Xmgm���n�����-�K�J�}2���I��I����:��W.�K�FHq�A����^%���������/[��������q0z�yB��R[�fn^>�����R%�c�i���4/�k�tF�|���.���F�hV����N�������C�=���^0���3���|�t��MN��d/&9�1�y��lg�s���C���v9�-����lg�s���;���v�8�)�y��lg�s|��3���[�9�;��I��N���}Y�~�������|3���Ye�-��j�����<Zm�t�-1h��k��6�-#��d��d�,�^��:���T�M��e�����;��1�N`���7 y��Q�%������&���GC��X1�vX�
�n�R#M&B��s}+4�N������&q�Z�xx��?0>����Y��f���_-O�����q��Q��������#U�hH;�Au�M�<�~���:���������������Q�V$|��^�3	�%�^��v�=�����i�g����@��n�y��'g����5�'_V*?�b,��<�or��M-|��n��[��1'q���t��8i�� ���>3�P'��I�OW�b�
��b��3uA>P'=i�jC]�X���JP��(�����"I]�*@]�@]���TJu�����Q�������[��.D`�B%�s�*�q��NzCVV7��?�
C�S���
��tz^��;��a�y�;�����A;���������DY|�'X�?�$�e�T��3�
�F"�c��z�T�}	��I��4&hl�6
�m*A�F�M�����o�B_]������:� S(���K�c�YG��e�G�"�xg������Y���g�M�6� ��?:��\������\�(����vc����n�����g��C����%��]��~���%��N_����Qll�$��=�sO��Q �U�*�s����Xz�Y������]D�d��B�!|^��WW��9������5B�!�L�=�����L��1� F��y*�3>�8BI3��%�_�&��4��X�4*.Rz_A�ZC�z��U�L�����2'��$t����R��`��H�Gi�^%r��'{�a%��l�
��bN���)p�.��MR7.����
�%���7����^��&f0�����|YG�&'e��76Ww��VV+�]�W�r�����X�bM�%�����b��%���R������fN�C	�4~��,��5�<���{w��i�X������Jdy�l�����8��Y
�>��>-���zb��5�����_��\�~i���3�>Zw�*�%��*��[�g�2���8Y��*rc�v��.���^���y�Mv�A�4�I���FE��N���W:w�Y�X����d���;�D|�B��1��g��NH��Z�N�^�+w�XRHLlF6����>w��uw�����N�J����y<�L���u�/-^n���0��W����p�sJ�G�A��;��4�s���|3�;�9�{�������bH�Y���d1����-����Z��[�m&���?�*6Sx�d�W�R�f����%�����Q�H��g����_^XF����h����"NmoX^�H>*/|,�����d����b�r������6v���r������V���p�ADA�C��m���JU`n}�������������.��fO��X'��T1V4��z#�9���r5jP��-��0y�[s�P��tU�1�I�3��Uj��keL������4#(��O��V����l���z�T�f1S�I��]qI���W���u������F�JH��U\�*�F/ma�W���S�c�W���2� V���<']�#c�*��#T����z4W)�a�c�������F����#�X��c96���v,���&�e�I:��
�~,��?�
W*V3:��$Z��c9`b�����:_�sKc�W%�������F/�X�XMn����^����xB��B��;C��s�c9��AU�u��4���t,{D��r,���?�c��:V���v,'������\���e�I:��
�~,��?�
W*V����$Z��c9`b�����:_�E}Kc�W%�������F/�X�XM~����^���������9w�Y�����r,UY����������D?�M�j���FEG��;����Z��|a]9�}���k�5�Z��|a]����� �k���Z0�|a]��1�#�j6t�Zx�|aMGl���6��Y�f7V�Q.������=��oo|S!�@`��*8'$�@�uZ��bg�������xYD��D�n���|q�c��/v�J��_)b���y��u��-J��'6�k��cR�w�L,���mv;
���+��
8:�����J8��<<�p��pr ��0;2J�������~!�j�K]��/J���%=
{x4�J��6C�z�z��g�s*V���O���C�C%�� ?w~��Jue��@C�Zh( ���?X=�,`���!�#o�40���� 1��p�%���=w��C=��9���)�/wnRg��i���b:j1���3�]���!&�|z��YU�����#�!CL���ih ��-!�ab*+��'I*	�{�6C�z�#&sF�SP^������9��q=�tl����m	+m&&�� �0)�.)�������������������������������������qz�����rSe%7�Rnq)E��U�;dm�]�������u�B�[���k�u�a���M����6����T�_c���i��
���b5���5�
����0�/4�����Xn�b��]��Fb����
�'.V�x|���4�����^��z��
�g�������l
��C����(�m��������`�Up"�Y���
�qb�������g�N����������{���\8�
D�;�o�8�-�h�
���C����i���-���v�)�6����E
��vm�6�]���`'�#���Qx����v�Z��k��������r7������w����l�����j��m<�Z��B���N�6D��B�U1\�p��q���y�j��}6q�B���N�6�^�Pz�V�N�6�����U1��p��!�>Y5�CQj�����W4�Ur��:-�i�Nrok!��
�������0����k�c/0@�
��M�����C�#hY�:�hu��l������
�>�xoM�dU���`c����*�|�`YP4�ph�-������!�ak���R���+4�T)�����ny����NW�J�h����U+w�H(�6�Q���^���h������ev�-��$[E��2�K��mF^M�'�_���'�y9]����:2v��������������'O�DL��:}K�&�	h
�t��p��4x��V�����kB�P �Z�Xc�k��V�%HI��w�n�t���0O���
 �����MV��*�J�e�:;}{�1_(�����j�e	=c��St�u�Z0~�X�8�7�s���Dh:T�����w�zT�{�d]>��N�����(�[j�D�0���zt�ZA��_���� (��} ��1���A8�p�N�9.C��g�PC�T)�Z�U,T��C��.������Rs#,�E�P��LH��	g�]�!����Yhc���
hS��A�f�_�B��l������V��Z
J��@fp��f?a���}����-�v��[o6>�z�0�������,l��f���7Cn�Y�t����[o~�z�0�������,��-'#�x���h?Z�LWa��yF���"Zf��w���,Zg������<;p���V7�F�Y�vk�[�!o�m��R���m�Lc�}�m����i���������<��W�-���$iAm��K�9��_/����$���@L����mQF�m}X4���.
��ea��T<~�GSmYB��UZ����[!mUx��d����F)Q5���
v�"����'L8��������:s,����q�v�3���Y��i�8s�����q�v~3�|����i�6s�����q�v^3�x�Y���`�8�[����L�f���t~�d��IW7'$����9�I3�%��Yf����k�]���;��&�*[�WP�*z�-a{����������/���wK�y�Qv{��P+��vHv��[Iw��}�Gl�t�r��G��Z����g���L+W�l��e�&L���kL��c��H��Iz�Dhf~�N7u�;���������9��K��+\�/"�g�a����O�C^��|�c��l�e�
�������B�1YW���'���������J
�)�,�\�Y�@&���O�y���zS()J�����m�$���w
Jo���9W\��j�(r����fE\h-�o�5�84������������7��P����?��P3��G��P����O5��i�A
X�����r�����Z�=g���=g�s��;g�D�8�~0��\���-_��9������M4_������YQ���2[��0j�6c��K��+�rE��oy��@z����,7��+�"���K���K}��8�n�u�N]������?��g�	2���:���8Z�1�(��oW���id+������v���t��{sE��������q�T�W�lU�������e#�:�$�B@�,���������p������I�m�A��b+��@��R��%�,����e��{$[)0��q \���.O�l��}�4������;�5���`�h����d�w`j����y#%����
{�z�����T#\YI�^�F�c��qx��fFPFz�E�����TgiX��$�$�$�k�4���$�$�$�k��A)���$)F�NLU�Y^y!q�����7���*j��jJ3��������h��
JV6&?�[�cL�1�����S�����������T]���u*@=�1���bcL��)��Sg��������bRC!�N��X���h���7���z�2(b�`�?l�����b���� �jJ�R
(6�b������{�d��d�n��c���zJ�����[�B�fG�����������/2�l,������r��oP��?��wh�V7�DU�w5#=y��t��"+$�12�~~d.
�4	�����_���k)%@�R��_��~�vY�@�TS�D�V��5�@�&�uIK~�	��E�.�������{���������gzM����cz�r��Y�z�^��!b9�:�\��Q.�(�N4G��(N��b��F���rc��������q��(Hx�#�9h�	(�:H�oY�Q&x��p�**�-��f���B���Z�������1j*���U�z�}\6���F �=��J�;����N�=��
�B6����6�k�JtIa$�JcE���DS��t���������g��d.��e����������H���I���1����I�Rd�f��2�J�����{a�J�L���
T�d��2��U�n��}/Oc��1P�LY�L����f*9�[:�����Y^�9)�Jx����g�m���>���,v���h���>���,6�Y\c�yh6>z��Y�!|���B��:��[S�?����x�>�s,m2�e��	p
���\&G*!�K�wD���^�\����M:��2��X/�d
��s��gn�����G��
��M8p�M��-}L�R��f���BD���^E��^^Za7����8�YbZF��zU����Q��ZWer5>T�����Z�����.dssX���Lp����zC) �s����{L�����F/�9;��"�2�X�����
����;5�������u����<�m���(�*�0��v�����b�;���c
��S��7z9�T];5�|;��@�`��"����*�JL���BUQ�oJ�j{F�q���������5��3�
�^}h� &)��9���~V���[m�������09��F0�P�'KUHU+f_�����O}
<s�&8��N��('��]�a~��a���Ln�>JR�^���}R�z�,#��p�`�by���V��I��W��	���,e*I��DR����B/m���G�SA�S���%�����,g�N��R�m�V�;��������2\Q����}�+H���,�6<q��9*8�l5�zz����V�����W�����H�Xg�]�y�?��67�\B��lv@Z��"?=�$��v��w�o|�_�`���J��*zSK�`T�sT��{�����6�T�O5b�K�����&2WPw����?��
g9=3�@�g�����#*���7P�s��(�Y�4�z�3���t�\ =�i����g�kq�����a�rv�+<sg�0��7(�Zx*O�0|��y���s;h0�.�����a�d�{�^���'
�6��QX����}�JJ�%�l�@M��p�t\%��M/
�������_P������I�()a�8��
c��1��)�t����������&��3��&��I��a�$��'��>1C�
Ol����&�y	m)o���r�����mZ=����iF�(���'0i1����:���~������t�[a�<1=MJ�,"��$D*b(:^����$�a�6��	��2��"�cAs��06��GJ%�Ei1�\����Y����xp�����}�$�����J��5��J:���bw�.��������V���.&��nn_�b	���'�N^Y���uq��8����'�J�e3������v��:��)�����O1M��x�?TK��Sq��4L���S�V����OQ�������v�����NX*�>���^-]�WKW�teKW�t�c�+������T������e�,�WV6�L�i�����b��1��e��Y4���"
)��������,�8�����&+�I��|M0	�M�w�rr�.w�_N�/^^�$�H������~��Z	�����o��������-����?�FpO�}@���SK�QQ��F�qa�E5�5*=w�h��VW��n��1����J��������b�Q�����wVY]G�~�f�o���z9��L�OO���wNPR���d�Q�����g����>�N���z1����>��w/~FP����:�g�t��V�r��:�����HyJ�>}}�����;�{@&����8�<�>���K'�g�Z��~�W��{kL�M�@�HN��|�q!?��by�J�X{�KzPf��2o�����Z�O�	qX5�c�x{X5vb��"V��X�]c1V��X�1+M`���U��X5vbUw��X5�cU��4�Uc'V�+b���U�5b������jlb���Uc;V��X5�`���Uc;V�-X5�c���b���U���7�I��Y=�����{�'�Iz��������I^L��>�D}�)�������Z������6�K�@�Z���/��j�0�^���A.�mH���`��t��A.�mH��� ����+�.�O�=5@�PT��I[��:�TiBI��J)�]������F��p?��Ns�"�|��o���:]���4�t�+��	e��Q�;��~R�i�+Q�xbu9�p��N�8�tg�8��=J;=d�v{�Z:=4:���l�&e��O~���{)�&?v�=��������~��J�6�q_�
7��l��	_���.��[,D�P�*|���[)��Y2�U�Q3le���=2���s�z�\���1p�&#��w4�����A���:�D@��2�A��o���
a#;�0�(R������-��y4\T ��\�&�(�>��F�
}��b
�L��<�A_9�s�}[��A�?G�h��@���[MrU
}	��D�>�"��/1�/qC_R}�"U��i������\#�t9�&�,�����
}}
�&�
��7(�>{�J��!�������*�t��&�.��!���
}C5�����	}C7�
���^�*�5#������}^�x8�&i|�����Z<h��#��z���)�y��Z>�E��S��,�����^�X{����}-z���������HmZ���a�A���=�p �����i-kZ��[������
����d�a�~ ���N?	TX���"�������������^�G-�L�F�nkc�����SE(�U���.����8����4���/h������������l��J:bS����yUa�2�w.�h�2�c�����l�F��`�KO@��Mv�^�A����dR�cj&R�GV&�������VG� .��<�o�}�Rr��s_��C�Q���L��/r���XfM��D���bcMc���1�R��n}������+�n����\V4�oK7W3V5�a���Ze�B6�G,���e�w����N���XfM����%���>����o���'��=[-56K�6��*[�\������o.g�J��'�t�Be������e�w����r���XfM����k}��1x�\�yr���+6O��zr���K{��\�'��h���{r,�2M!�������Z�+������)�6��a&H��\�yr%��+1O��zr%��+)?����Y���*H���4yo����������?�|����Ri��+6�46�c�>���Jl'Wb�\���J\'�����|O.g������*Xe�B6W�'����6W��U�?�YS*��[Ga��OZd�Z�5g����7���7����dx���sd�a�eI7�B���DJD�f���~�q�-����h�d���_e���y�g�t���:�8;?}������H�	E|K�)� �����N��%��,KWk
��kph��uS���f2��=��*�b�}��=��=��Vl@R��Z����L2�O�w-������������������X9R����{7��:;}{��&�b����f���\����N�S���2z�X�Ys��W���l"V�B����4����fB^��$9���N��<��?OA!��
?e��|���7WiN���y���1���2aSY��%L��~c�<>��b�����lN�&����pR�#H�P
)F�c��9f�c#�IB�������5qSU�l�{o�o����'�p�Bp�0�fa&�"�$f�5�����#����4��b�� ,�&<Q&z����Ma�B7�-t�^��`BS�cz��2hr#�M50���+1{Qv�\b:�z�^��t5�8D��1��G0�+�����t1�B�B���b-�QbF3RC�0]���+�QmLG! )�c�����S�LgY�F0]���:E��v0��o�1=�U4�_U���N�O�7��=���N�H�4����&����%z�������1]���]����9+���b-rVbF�R�c�0]9W�9��L���	gKg�O�0�eY��t���[��r�����
���ie��
�V
�j`:�s9�J���=���a	}.1�|=a��`��������c4�N^��a������b��������u��w�6s2}�$%`L8�X:#|*���,���+����N�+�Lh��`L�WM��~���������t������T��������&���f.z���L7sg��1]���]�r(��n�!v�a��	����jY'lzG4'�'jIR�����3��r����Ma��P-[��r�����
����e��
�	M����m\k.��%R�@�O�U�wB�����L6;&�{i���p?,FO�'������D9�c�c)��b?e����@
�,�U�����aw%~XL�,Y�����R�Q�u�p��.�m��j�e1��o0f��������:���h�E;����%�HZ���5m
@[��5m
@���@M8i
@�b�hk����A����5m
@�������A����5m
@�������A����5m
@�������A����5m
@�������A����5m�%ZK���S��j
@[������;�����l���o��b�?��f�x�E�w�l��.��h:�V�1�JK4C[}����e��� ��l�E�@XPG����lB��0CS���x�n�ukE�>�u�F�N���b���������8jQ��NDs�<U�1>`�����Iw�v�G����.�c�����E�.�Mig��5���f~�|����p��j�N��.L���&�^�Y�J�o�l����|��)��L�Q`9��� �V����4�c�w�<�^l7����6�|n�M�56~�����J46~���/3�vS�����#p��S�j�[z`�c�:C4�1�5��zt��X���6xLr��c���2&g���1UK������1xLEi��T!��>&kf�*cjh��%0S��Jh<��$f�c��7J�5�T�1�rG
�\�����1�P�O*MT�����Q�����B(�1��������
��h��N�blll4��T����>�>�������fN@�.]��k�3^-���I��e6K�����(;�P�pD��ZM���_��7(�
��P�-���	���J��!�{�:�D� ��Ks����x�F/� �i�7#�S��{�{Q��������F��-c������l�����$��T���e����p��z�U�]����16\>|zWk���53��a�����N���-5���aZ���A��Hi�Pc���wC[�J�~��G��PT����u�y�*���d��8^O��#�8FM���c��;X��$��$nn$U�&���$�e$IC#��1��#XF2hn$U
�'��$C�H��H:����'G�W����t����m9�F��|��o�k3�E���>������s:��8�_�Rs�-��!��4j�������l�x�"h�b�L�X0�9z��9��n��������������LL��P3�q��T�Tx�P��/x��=�������7����^��)js��FV�n�[�1	��j�,5V�LiX��~�E���:�?_[�/��[5���>M R"I*pM�e�j1������1&���r�7g,v���c5���Y�n:���[�2����#'�9����^���H���,Z���k�61~�9q�������-�M�%�g{���1��0�i���	������u�y����.6���z����N{a?��"�G��h����Z���H�&��.n��+�e�j�E���Z�1&�4mu�)��.`�Y
[���[c���D9����]�('�b[k���r��s/��.�:�����u~��fo�4������u@��u0�,X-~����c�Lsl����j7������b[��w]l���Y��g]b=��Y��p�:���Y����=�\���4�����Q���)t�U��1a��k������q�J���WR�~���0o���gsO�����S����*����T:6���=�8�i�6|6���i����)�_����3�C� ~|���E���h�����Jt��\���I�vT��9��+PY0�ZLz�d^�Y0��p���%=��LO������������WN_i���|���}��6�������u�����B��o=��b�lM����T4c+�����c
���[������}[f�\}���
��!��SW������-M}�h�b4�oP�6���Z���h`��z�J���i,P��������VU+*N����o��[?�o������G�l���}[]c�1�e�-��^���}����@WY�����y�&��61���z��T�y��}�������5��>�n�X���m-��������7z9�X�m��Y��J����,P��������VUQ+jV���o��[?�s����7�G�����}[]k�1�eZ�-��^��K�}�WV�[QV?F-R���@""N����o�����aL������A�����q����������4dV���
����_�����
��S��P��V�Qg�rzm�U��T�~���^;�����6\O�ZA�:.�u����`�,������Am�!S�����r���f���V��6�-[�����q-�R?�h��rN���.t�)K|��
�t��T_�^�s�V��z:�
������d?U������7t��!����\�����T����n�m��~���'���^��
�-l[V�n+yL=
f����]���]>~����1��`��J�������U�B�����N��������K# ����D
H�o��%Up*	�\�����6����N��jh��0�Qda]��Dks�e��]��Y�CI�Y������5�2R;���'d��8�y>�.�J:
��9%���l�?Y	�G��`����1A���������3��&��'�a@�q]���}��#W��ll%���'�(��#3���x�<U���s�K2"t�jz�1O������~����W�8l��{�^��7������u�$��'�nu��e�s�xl���������dT9�{\�)4���Q��3c�~������:,��7;_��+1zx��K'{zM�w����,_��f���a�&@��QZ����d�/,9b��J�W����l����\-0��C�J�=��'����$_��J��.��(�g��z��N�$]e+�
���M��� �_���d���,�����<�n��m��o���#6�v���x����}���f��_Lo3����������/�McJ`oA�i�������������;g�:���
a�E��c����F�-�X���<�����N�=GHl���!�����d����)ag����eKn�w%���1���K$H;?8M^��Tz��$�4Kx�.n1S8M^���h���i���]�b�p�4������W�������T�J9)�&�:��i����<������f��<�8Q�0f���@yhx8�����>4�����nR(P�x�p�7������;.mc�7p�DxK�+WK��;��QK�{w�%������'N�= \�~K�����P/�\zy	�H`/Bee4&�^�m����Yt5�ePt��R�P,}��II��1���]�N�3�'b���M����#��nt�.�SPtF��l�Z��Xx~~vzqF+�8������_N�/^^�$ ��������/��~={�J�~7�&�����C�c:_e��%�]�r����L
i��/.w���D��of��2��`P\b(j�O|u>������['>���:��Zko��X�y�����'IR\d �u�B����F��t~]-����K�Qe*|�,Og���h����C�_�6V��?{��6�����O��=m�XN$R���f[����M�<����{�O?J����G�������+�W A�r���#��98�;�(? etqVO�Y2Y��z�������[D��L/0l�������s�����5~� {�C������*�*%��H�������
Et%��D]��DTB��O�A������L�����d������Y��� �%��^v4pO�y�$(UH�&M��B�� �%�J�w
L8���������ep�����r�V�������qZH9
(��y-��k:u���0R�@@o���N�7ngy���y�0���w|��k���}B
���(������D!��w�/���o��$����
8�iyA�2�`�%���
$���]�Bg�%�=�M{�v������t%�[�'	���-k��D�x�{�9���L����l�.p�/rln�.\�[�e���l0+�pO�_\��=A
��������G����������e-��D�����L�s+54R��?}|s�3�����4��*d�v���xC�G������N�/��R�i�AIp�
?�u
�l�����1�b��Lx&	����=T�QY���%�OH�{��n[-7*�.

R����	I&���M��zP�6A���
%���.�B��%(*�&	yeP:�E8	q�ychGb^%x�"��/�,��7�W�o'|&�F��������w>�-��������F��;�O�!��3�p�>����&|��M�M��>2��>�W�>��`Q6���"���M�d���v��4�0����Xh����g���D�ok\�)
����9s��d�����%� 1w�;�tCG�-5?-�W�:���;(eM���Bl����&���[Ip�ke����B����F��C�>�7J�m>@�E�e��b%+�a�W.^��U( ���Y�%��������rX}kY���p6Zx��B�D��i��Nxd&?6�`��^,~����������_,����=�M�w���s��=��n�\y���m����j1����
��5,�������%�p|�*�\��~OG�(�m�x� �/9I���9����!����i2D%�uN���=�xz��#�&�]H���~��s���(���64J��k/�x�N�.\"�t����Wi:�dP�U#�iE�BdV"��1_�H��u�����iL��2�#7�
5�&W�a	��u���3o�~��r�l%��K������FZt6k��T�����
LVU�lQ��B=oPi�N�RmX#��U�hE�:vD�""|�������`�������yD�<2�E����5�Z)��[�W��U���^;l�qU0�YOKj�y�7����D���o���C<�W�eL�30���h���4�V�8�� �����������O'o��f������p4�!�g�W�$!4�{��Vz�K.#u*�������#�7E�����['�
t�����7����'
���7�Cb��>T���x|0Y'�g��_6��X����AE�����Qf�&-���|V"f0��+2�'k��A��OVp��Y��P>3�,�!3�#�=Y/V�a;%����5��j���s]�_�z�b�������R��f-t:�VZ��W�Y|�G�a�jd��`���a��+4So����h����+�����WpJ��c���1�-�,����"<]m���LQBEL����Y��"�\e��%q��"��54S�!u3S�9��3%���1�"��gJ�#�n�,�D��"<]m�D�L����(�z��(W�)Q�L�63S�!u3S��K2���L��3%��x�:��W�
�~lqB����`�
���4������C�$6 ���#7I�&}�Z���~v�'���c�S�E��V2�����$��>�V��p��(d�[��E��V2�:��}�����
:�s��h������!�H+}���p
I+8�j��h8t{Vm�[71��MM���c~�51�&����(��:����M�n�y����b�����}����w�$�J���D���5:����M�A'��|�y���BQD�}��	��{.6��<��@~���U�K��r��Ki�hj�7�{�v�t�e@[��\���v9F���f	LJ8k1%����:��Xiv� �(����'��=��
x����gQ���{>��[�������Ur����a�Qh_0%x��Jq�24�cHh�~��%.W
�(@��hl��D�,P�GA�~��]����������8���O|��<"��t���a7�\��� FM�=g34�������E7U��)+�W�R���i�/�~�K��8��)���~�8���/��/	P�f����h���rq�Q}J�����	 *�]{���j�;Y�_<�B\��8`n�"F<h*����o�������b��
p�qx"x	P��^�X<	D��'Dmy�\�:��8�.��8lc��KG
�L��^v����.�\�cf@G���vP�oe@c�����J;�I# s���V�B�u�|J2���@������XSR����c>�g��u�X���O��A
�9o���i`�W�W���|��~����>�L���T�P�/"�@�56�:\EM
�����nj�b�U�����n�r���9������3�E���&�� %����,hD5s�N��m������9���f�|i��f�WeE���I��t��?��
�u���`�;��R��F��V�"[�^���R�dh�0��14����zc��r��W�+g����6���_$K�#��*�Pa��)���]	+�j�����E�$������K\Tfn.Y���������d�����Y������%��]�AG�7���,Je�u�����8��B�P��)7a���z(�Z���PT:ak��p���=0�({�������?{8B���U�����?{���xvX���Zf�ue�X�,��<��'�iw�x��&���V@GZ���g��
�/��
+�
�!_&I5�J>�jX�3�����H���3�\\���?���H.�@�8���T���/L3��K�
���������aR�����
Z�R�U��*�P`G�M��A��b��
3A�of������P�L;�U��J&�:�X��+��*��,
hf�
��"����*�Z�(bp}��a��rhU�gu%Kf�eR���&��9SV�g��M��7�H�X�+�h���G4���b���:��$�*t2�90������L9`��i���b8\V�5*k��{O�����
�l*n��
�:����})-�)�($���n)����*n)�!��D}��%�z��[
���7Q33{������B�3�MTM^j�r�L�57�~�	��'p��Xi�o��3��b�|���X�c���A��X<�d�"Kc��o��>~.
S��X"�;�i������c������m��"K3Q3F�=E���T"y��Mu�1�wg�u��jc(���X�����\�]��/�xc&�Mr!�b�y-���G��]�d�M���P[������R�S1�&�:_3H���
�2qf���=��=-q�Z�Sl����k�m1c|sZ��#�Xb���2�*�Z[���]��<*���>5K��q1���f��l����3�
@�bf���496�&��*~��C����_��q����1T!��!&|H���^L,�qQ6]k��m	!���M���p��bs��J��1�5fvN-g3;��U���;�Kjk���fv���d`����!��������������fQk��E��g��y�z�����iyj���>�Zf��2-Ol�U`����� �:'�Ogg%:��r�Z�T�#V�K���a�I�?qb�>��Rz� ���F�2O��`5 
�v��%����}�B��r���>
�&G�h�zf�1����u�q[���|�����3��p��<y���c�BQ�$&8DX�v�K<[')���a����~>i����j`��e��3.�$�9�B>+�g<lk���0�e�Q����:v,u#"��<Qe��^9��2�J����t���/$�����8��:���_���	�tBYZp�z�R�QHv�i��u� �l��6���
��'q���Jb��,X��h��t�ecr��8H�zh�o�h\MW��K��$�2M�"n��j�R����P�t(?
������A���M�sR` ;�'G���HS����.9��d�qI
�yT�U�yTE~���u��R(���+�����"��g�k�8��a��0���X�����?N��X@a��:%��*.�Z�>� ��*��������Y-�(��$6���G��J'����M�<'QI��q������e������������b���������i�=������)���~��8��F�Q��>�(��8�Gx�mrAR6f�C�mb�\^�28�L!�X�2LK�R��E�=� C_uY(Q\x��������=�����j�46����e��Y�-� �R����|Ze6M�3�q����Eh�-ee�B���z���p��"���d�@���<P�{��%ym����\�E�����k�������cW����x�{�
�����^�l���q9�TgCQ]��
�l
a��/BT����q����jGDt��r)!��mI���K�Lm!�lh��m9��Q�5%[b6��r�iz{XF]q�M�F�����0��*�6��y����}:�I�m'zf$��R��(�w��xg(�{���p(��DI��)�O� �P�6Y������3��e�p�M�2���&����G�'�X��R/���R/�J�K��b�=��KwP���i&�^�g�VK@�">��k�.��svKEq����,�V0��S�U������I�z����4j��ar,���g{v����H���7�l%���`,�J��*�����7%JgRx���e/����J'�
�D�!)6�#fr�3Y�b��*��i�{��lSU������jw��SJ��R;�7?��H�H�`(=3�k����f���`[RG�*FdW����&P�
TBQeR<9/��ITcr�bN&��)K�vq��q�1S�����U�������]kGP��U@68z�z\��0i
{U k
5�LY\_��0Ss��TxZ����u�0���l���@�;�G�5������,��|_�
����b�3�D\������e��{G�������I����&�T>�v7�Y�Es��L���n
,��[�k��WvO0#�Xc�&7,'�5����L���o��&]}s���Z�F'�i��M�#��a�/M��2>8p'��[9K
�����4w��&'p�hb�s��|�h�i��bl�h������l�e��	��s�����2�Q�$��9�E>���"����uVA�@�)>����<����$2��M`
���
~�GA�l���u	q�#r�/$i}��dY0��#�������l�0q�Pj��e:F�}^�g[�{8�U1��"�����
u��U$�C_[��+�Z(�u��^���.�6q��[��q������j�������r��i��S@�uF.���K���0T�����LxA����$��DCt��kM��P�q��Z����a�Q7�C%�$Wp����G��|!h�8�/�S�3����VSKt�Q����1"RY<��������_����A����0 ����
�h\'�}Z�KqV�0.e������.h�f\L"`���	���m?a�1����\��?����Fm�`v��E���CU�j�AV���v<����v���������(E=U�*]}6;���2!�����<Ve�	1��Jy�Ub�%���l��$��~�4�M�5�K*����&U��
�#����/v(�X����	m:5	_�Z��F����La�����"�@�������"|)���,[�d�������U5Zl@����Z`�hml�52��\-�Z���j�"`E���z U^���h>V�N(gq
�vBm�U���'�3����=�!C�I�������u�QmjTs�H,:��v���Wmp�WGrY��LS*M����������{�A�j�1��V����L����T���W�t�y��
#n���-���X}<�w����Y�������y��1���"ys�B*��F��^��.������^]?V}�^����
^m0��
n��
H]�J��������<0b}�f�%E�K����;�.���^U(���1~�����X��U)����v�6��T,������s
�t��S_�����cX�BY����6�_B����$^��U�P@�(_H�5[�8����jB"?�m��X	����U��Ba�HL5�qc��L��V������WGK��u�2�j>F:OqM�ah���BX��2�=�$'�j��NP6Wq�23a���O'J�l��r}UE���j�Wx���f~�����o�u
�)!�(v����X��P�R����B���L�3%T�����0[(/DEG�����/�D�*�3�z��W��9�9Sp�p�Wp�_k���U��������B�L�����{����=U^qS_�S�=U^�2��{��({���V)L�S������8C�v����)w����}V����*@^�y ���Yy���W�������
&_�g�k"�T�T�����JP�pa�1��{����q����`y�W����,=�^}2��U��y�� +~��NN�����\CZ��+8^��
�Wp���7}��+8^��
�Pp�,&>@��������5��t�9�2M���Brq��.��x�����U<�%A:����V*���#��"���W�����L�SG�Q).D&�_��=F�t�F��Z������){
�9k�r��
���Fp��K�U"�}��|�K~��^�1�0'�O������;����H��e5���J��j1��uo����d������HF�H*�K�.��,��j$�
����H�{��8��(�E�$���+�R�6(�}N����}F�)lN0�g�3f�������P�/{�����-;�U�����|���L�l����������jY�,����E�h���|�Y�-�{����]��|�X,v��������n��n�y�N"������e�W�O�+��%"b����~�fo���;b/��9\"�	��ge���������Qd/K��AS�8g�/r���l�Yid��,��01�j��/�I��j�����]����;�\��Cv�u\���q����
J��r�Y����YckL���Xe�6��p�ek���Z��d���8�E�6�>����������������/�]9
s`�Nz�����
'JW�P�!������jP���T�?kw4$3nh�e�����9@���@!]���W &��8B�j�.�j+P��h����Z�|*�Z�+'�M�I���W�aa0K�
�W �S:�>R-*8�������
��+���h]+�A���`C���V����
�
L�Fpe����][�6�������JG���m�?7�9��f�r
-��d��
\����f��Y2r���L9�V�l�FT1#%�T�����_�
4�
��-�H��e�2����o-��H����FR}�<�<��F�y�c��e���e��@N�`�~��6�^�'P0N��g�_�`q�,��bi%"C�f�����&;�?�����������e����
]�����hd�Q���bQ�4�F����%�2o��kJ��D������|����d���V����s5����������7��X+Z������*���Uqd�nJ��7�#�9c*h�d��g�A�Xe��D�
��hUc��������qh0��8����o�l��f%��)P<�����8�	l�-�>��=;����.|���/�����?�
����������wV%���2�\��h��d�.��2�_-����0�w �/�����8��R&k��|������eoy{��~��9(@Q&��W���J!R
f`[8
����R�/gT����mHU�I5�2���r(L���S�kp���,��0`�	L����1��q&B�|U}��u!�c
~�3���H8�	_���~.	�N�D������2aK��g0��d�C$����P���}'�F=J����h�(*�rc��6JK��Y3�_~9�8������:''�/,���(A���N����[������`�q�:t�����m���HW)�.���@vn�qox�QsU���]g��47k"�yV`��Q�U���K��bf���3WP�N>|8td����b}�� �����@���-"�����\���A���y
B���i��v�+����.U?C����,��x�������M�	�j�>�K�qd�h�.�x�0vo���
���mr�6m��Q��Kz�����{W���h]��]���n�����1
���^�e�YV�eEX�9�YQ�D	h;9rQ�n����t���d���8�p���?:�9�����M��X"����}�yAf������_�N?]"�����^�9c�����n���i��C�����s�&��Y�)�dk$c6R�����je��1��:;y����	����]/�A��q�Y�N�C�l��r�p���E�����$���J���za7�����X�/P�a�"3YM��?P��}	��0�����)�A��Rl��*029�<���U�����!d����t@�X�NM��jn�Bv�!�W��v������>�M�qd�L�
�UD�r)3=ape�"g� �W��gKS��
l��g������rV��\�JOX�Y�V-��U"�>��������@���+��b�����x�2����W@��
�W �Qry�+��
�W ���F�+^�`��Q�h}%��d�La��?�����tV�������Ip���#�4l�q��m����g���'o�bR�^C����uZ�l����zg^:?�|���2��^u�����O�S\cJ�x�����0���<rq�������#p6��!r���g��{M���
:y��/�}J�������������M���E���>�a��U��]�5�e�F@l�����?}����K���?(F�g�=/���z:�S�Nd�v/��M2�1�����P(��/�G�p����Q0��+%��7��}�<����b�(����3�H.��*�������������J�#) ���Do+7���B#?Y&1^��U��
�Sf�t����;@O��;�8����qJ�����Bz:�?�OA@�J6�0�c��&����	��>Au���9����s(������1U_>���d����S{F-G�(r^����l
z�W���@���h�r��%m�_X���B�K�h5�V�����M�
�`A���������'Rc���1M��m1x
��)^��h]��Z)��D����Z�o���9�E;�������i,L&�~�M��^��K���Kt<�_��/���2��~����/}����������P�c���Q:�pSyh��a���������{~(�E����nE�-�)0u��p�HQ�	�']D�,�=j'E�{��T	$a{�9F+���}�=W��+��0����,3��0�T4�jy@�bb��(@Q&�Yd�-�b/N�4E3�k��,7d@3�kzDlA/�Y����M�N���"V��!(:�~7w��8cFXc�����'��"a)��?�u��p�\���z�"�II?(���Ff��'/?����J������Ll[f o\~r���'��'?����$���|n�(%���I/Lya�S^����xya�S���]��7"N]�g��ISH��4y`�q�����'H���+b����K��vW��]j�"�	���t,D^)(!K��Py� 8���I��?{|@=���!j.I��vn��C>T������[��?{4�kH`���?�G�q�I�x�^�3�!�l��`�jY`Q��]����.p=�T�����9H=>�z�e���;|g"P^%���Nh&P�*	�����k��E�F�U��b�e��rlu�:7-�j@a��H������l��,���u�+�u(z���q�3�]�_������l�#�Z���#��A��a���A��N�H�������O�Ut��<��a.V�8�^�����gT(1����`G�nPaS�*��
X������{������tW�=���*���g�0���S"��&u3��!�����2���P�;����<n��T8p�z�l3��oD�.����u�`-ie����[8��zm3�����6*w��V���6W�p?b���Pf@�@���:��,��M4�>�T�3����5��S$HM��A�f��������[���,���:��`+���|���|���m�O�@��-|{9$�!`�*	��HPm4DB�a�xL����-�#]VDp:F�R���m2F5fNEE�� &U���4��~��
�=z���>����K��bAV�)N!bo&��N�q��J�qKBa-��3�������&6�y�ho[q�v�Vjgi������V���J�/������uFe��?��XX"��F�${��=m���6jdO.��������!N�����:V���������&��ge�z���N>�y,Yl]b���w']�:���y����W������������+�B��	��a!�j���3�!���w�������n�����h=#"���<O��|�1��!FI�Ca�xfw�Kvwq�%���k��F)�cQj��i�Q�WoT�����Q��A����4�]����S�������g!W�(k]���q�k{���cu�N&�A:{rtN���
��{B9�!���X����v���LB �A_t ��Z1�v
���fcdDj����2�L���<&�i�p�:[���"&E�+j��S�Z��i��������E��e|��qj��e|	�6f�����������"N�xyCu��z �=U���2�k�,�����s�"w�,�8�%�E�IoQI�f��������`n����	��}���@�m�d!���y
+^v+7���`)E�u����/���W��G���}�J�tr�Z,��������Em/j{Q[��
yQ�\/j{Q;��D����!�mT�vW���D.tK�����U������n/vk��b��!/v����n/v����%�b7������7?���o�Jc3r��h������n
�^�V4���Rp���������#^����Q��b���c$p�MC!&&�,Ws
U.��ep8J!��b�vKIC�G�"�G1����P��iz�]��)�p*qF��BY�#���3#�L����|�J�o��RC��>�����B{r��=����\�A-u6Q�C��MR:��������&BdL&`E��W��x��J�6h�j��^m�jC,������!	����������B|Lf`5��W��x��J�6h�j��^m�jC,����
��!I���*�������k�r9(�y-����W �Q��W L��
DU���+^����=4u�v�.ImL�P�#y�F��������Dp�W"��@���Dx%�+��lL�������h��y��r�X��!�w��������U�����`�W���^^e�*�<�2����2�O\yH�fc�3l^Qh`����N"�2M��cr%����"����)<@7X��Y���;��<k�D@�?��8��#��"���W�������t�����^f:�\�
&��a��E�9��9�Db!������Q�9C��	������	0�������
����+{���nd��@�l�����{# ��(�D:����oE��!���+�k�_����~�A}^���0�\O�^uP���(;��.+W+�	`w�g�`��-P[~7�d�����J��I?~
E���J*�"� �
���Jv��g�T�8�}QIX��~.n��.*`1X�����3�W
��"����D�*0�bH)��};;�%0�R]��Hzk��r���DTdaU�V�%6V���j���:.����T��e�������=�?*��d����&��D�~�a��d*�����xhj�2�D�3BHIa.��E��("[V�;[���Xd�;�4.��S�7��}
3XL ��d���aqs<��#��E�/[G���zfc������r���_y��0����/�e�A7���aN�J��W5���nK����M�.��.v�
\���42��K6
�%GqKS8��
�gqKCS8���
��"��!��8Z6`�9"�l3��l��k�n��*�psP
�A�V�3P
	T�������p`f�o�:�=�6Sd3E��n�)���[]���L����f�l���0��7j�N���y��b����J�,Se5/U��k{���������+��H��p,w��_m��.�:��n7������m������PQ"�W���%t3�
�t	rSi]���n���.An*�+ oJ�/�KK��6�Z+}�F����[i��z�*��rQK�O5%t��TSB��O5%t��T���,�9��[&i
���e�
�S���z>��y*<>�����d�K�J(v!���AAU���8��?��s��g�	5�����v](�O2��
�
)�2sR�*��\'��!0�/����_�?����'��dx�6*��=*�|Tv�X�	��M`�`���F�H��r� �N[G��F����������X�Q���i��Y9gL?���M&���L����;���4��
g?�1"�2��Z&�|<{{vz��c W�w;��{�R�y�
����0���>���/���*$_-�k�lr
���J��A��
�d�e:�Z|=��)�I�v�f��*]���\�U�)3(w3�@��.��k!0&�M|�1���)����3���c����N���N�1/F��6:��Mm]���3��F���_���?�u$����\C�R�-����	���#L`C�G�I��z#lw����c����rc��62��	Y����rvq�G�epr����������}�B�A����BA�����n��L��C����%N?#�V/���5��:]��~�K��U2�$>�
���!�������C����=���s����D�7e��4e�0�B=��a�`�f	$	Ue
~Mc��0\7V���_|��M����yy��U��1Rj���
VaV�Q�������U���U_,�_��i':V�����]�C�^�LT]��D&��a��S�ZEv;���$<#��,�;�D�s�P��Ei��o���o�3���d���� ��OA������-ll������/pI�R���f���+�
�7���<�^�-��/�j���������v�@���j��];��O{���O+��C9.���;����D��a.7aQ�T���b��H��a.5!A�V���t���J(�Y"�>tr�"��2��v�
����|��pq��H��t�s�1��N&]d3��a��
�?�E
�;�Af��q%E�pp����O�H0���g��YU���K�7In�`d�	�j���B��s�3��?4����h���V��Y����Q������Mi �J����>�`��ElLN��#�	�#A�"H.1P����@r����0����@�$'�������N��}z��}����e�~���d���<���&b�s��������/`?k����b���VT��WW�+��{�@_Bz7��f���7	\\�m|�i�|+�]�u��2��z������^�����
�������~�r%?�@^X{���6)�
����yW�|0�YC��dBL���@]�G��1��
y�+������q�U@��F�������$*/�����U���O^A��y^��r�����WB1/�y9O�9�+�����T����O�������o�_@���[������.>�������6� x}vy�Kp��-4HZP���Lu�zZ:��<;���(-��wpzqM��?��^�y]_�:�������������@w��?f������<rq����O/�G�&l��2v�+�?����mx�(���C��"@��I������t�����&�m�"�CpF2�0��
��.!���Ax�<�"h3#�����%���?(���2��-���z:�!SH�9��-��M2�^��|�E�]���e=�Ed�~����!{�J9X��w�>]�����6F4S>��q�(�����T�2c�;�8������c�.a��_��l�9���+�I���.���#�.���c�.�<������g�_�=Tj�uq,��!���e��7^�A�����v=[Mg��dv�^�����7��p��H�3\�,Ni�-��~|!�����}�_��E6�����^g�3��9FEt`D4���YD��A���op6".��cU�+/��A���@��Z"L z����J����n�t�L%�G�z����0�a�,�#�)���g�����f�� 9�)4�0�)�
STR`�/�jS�����|�T�Z�l*|9�|�������C?����l�{�Z4�@Z�5,�tE���`		������MHPl��Pv���i����2U
��'D��HT�o,���hU6��������UZc �7�c1��&�??`�0���W�#���2k��X��0�\��<2v�qw7cq�6 �}��#�G���h�8�x�>I�rD��[�������B�~Y��^�+�V���B��#.���9A�!C1��h*�>jRF�*peU��`���V;,��
��������C}�0N�3/.�����>;�oG��a��	dlRKl�~���d*a�G�T�(�%�s��%�&���4K@C�F O������I,�H�VTjm��E��gDr#�QDV�jn��@�C1"rnYTrr������9�""��E��-L�;���z���*Z%�L{7��D����(���%�'�FI��4)	���>z8�@�%	���������|��P�����,��c�K���O���Gz� h�P#���_F/VG�*N�:$+9��(�x���U�X�����c.���������_)���h:J�JM	��������h*�bG�q0����	b}����a�9�L�g�Z1?���3G�d��%��������Z^�%������h�\2�|2��7ye$���|��Q�1���`!����F��BR��bN�'�-���F����A�Q�sR�olt����}i�J���234��������1R
��2��$�l�.Zt��Y�r��|���0��������=E����<�
6���y�I�'jk��L���<�\.ue�z
z�9�c���y�/Iz�&��*F�K����6����	����+��
)Z:s�-Z���G�8BDc`V)��1���T���2��=�����$��p(��D[�l]	���aO���.g�J6��d�a��/�.}��N[No>o�r�+U��Ls|&���7�����o��&����o�����oL���o���o�����6�|�j���d����l������L�Hd��#/g6�j�HU7
�?gQ�h-�����n_i_1����O+	�7�%��/_���?Pn2Q�u�%�ZG��H�xA���}���f�EG���J��j��Z���R�1�NU[�B���W��_� c����4�f���C��LG@��Z��|�[��,K"_��w�|�4[��y�Vl����~� ���_o*q����Y�����j�U�k�$ie�:�|�|�4������2^
���A��i���m|j������f����w�h������������5�V�QkQ���W}������!���>�M�'����B��PjG�|��gR;Rv��	��vK^0�W�e<sR��mKWG1*��,kCUs6�b���f�[�J�DZQff`cs��h��Ra���`lQ��	Zj�EtK���8d)��u�S#������G��eu��[i�54z��HX�n5��j*����(j/x�3k��"���&�z��j��{�j�T�������,�A��N��>.�,���~����C^��%�|�h�i��d����f�}��)��4^�>#�^$�zfP��"���!���n4��������nl�4��m�(�n�
��
Sf,���'n�,<w���;)^Z�\3u��r#VaBF86d��2!�����ll����5�whe2��zL�Fu��	���k��W�	e��	�����E��p�������a������
�dgi�����W����������jK�w@o����c�X���+^�5p�1` ��K��Me���m��G���+#��y��fZI'��F��-�4]���ZID�K�����O3����$�`��"\�7*�X\���TkW�z%���@^*�k}n�)��	�8(�~��@��s��Me�{4l*d�q�T��G����=5d qHT�����r�^?t;!��y�`f�u%�B�i����K8}����V�����|���6�NRL4�����&���<�W������y-t������������K%����K��1g2�1�'332�1��s}�����w��8r��t����2�=w�
���QZ��7p��>�u4���lx0�\t����D���\�5|�V{�� ���Nf�"�r?o9.�Pd�).	�S�I���x8a�
9q���B`��*������6��.NX�BB�
B(]�-f��$P�I$�H���F�q��X<�&F\��)V���iX
����j`����T-�jj~�(\�ig	���]_��ddF�x�����l���)'����wARTix��O��j�����d�HS$�3�pR��,p�������3�����V+��<6
�����+��y~	vf}����?�*�M���VFT��������b�Zt�(�-1�'"�E��)O)�y"u2Od���d�h��_Dv�-�#��5x���"����������S��I���UI�]8K���s���Ezd�C)b"uBLd����h�	1�]BLD��!c��$"��UJKD��(�Q���'��]�����&�?�����
*5:�A%�l�{n�1�o�en�"w�����'^�*Nq['�(����^o���#����g�u�k��wXs��V���vB�b�Pl'�V���w�v�Rn����j�NJ�0l�V��Z��J�9�g��T����������� }oUu|���2?7n[--A�-���*XX�R^�z�V�J��]�H��G���[�M�QU=)RWO���'E�zR�\��2+�F��;UI��b������h\��73F�2��5f���0�fco6.������i6>��0���1���P02���!�+��_�;k�cLw����V���m���+C�R�c�������o�����-�>>�[Ow�z��S3l}|��Ou;{������j���oj,�||�74zC�Joh,����;`�����rJ��cU����U<��9��S���.oa������P��>VU�����XU��cU}����7 {�#�����z�7!A�M��P�"Vu@�
���W!�Z6D�:CcI�(���r���U�*��c
�$S�J�|&Y%SpfC�fao6
���
sBo%�V��V�&������������	�1gU�6doC���m��q�?g�N�l)������3{ZZ����r�k_�Z��h�����@[�{�xm���N�mn�n�V�&���R6bwc�n����5;�cW�+F��	��
���Y��F�B�mL�nDj�V��Z.�j�raT�dNpeH�4!�,�.�7�sg�A�Z��5�4���acl�mf�j`rkZ�mT27'x�+��,
@����F�J�CO}O%���Y��A��)���S�|S�pSl�Zh�i���m6���mTF�F��71w�K���*�y|����Z�A�>h�-!�n��Zl�e�-�����>h��e���5P��E�h�Z�A�;eE�pWC�����2��E.*��W7V���/R��_^m����/V���/�u��.����R;4|���y��/����}�b!������5�A�*%�2n���H4���2���s����}0�f���>��3�`F�����
�|0�f���>��1�h	��>��3��3z#�7�;Yo���u4f��|����������������������������_����������e�y��%0x���~��Z��s���4|3c�m��U���u��|�&_Sxiq��.��x�����N�x<K�tz{��g������<������L����������j�-��RY�3��A����2���w������SH�G�1M��i��=�����j�����
�������86���m���#!�E-������C��e������J�5�H
a�:D��t���	t�:D"J5?q��Y�1X]u�H~�{��4��+=�
�P@#0�L�bdB���i��>����V�7�J���'�'�G3������O�2��'�	|��'���
�:���1�&�f�A���@���
��3zEm������jDh����~1T��-Bb���+���*�Q3�~�/�{���2���xo�����Mo����F1m�B���>Y@��������8��.1K�7�^`�%��$��Mp4$1C"������ih4V�6Y���*	��\��i��5�����Y���V���*�K��"��Z�Hd

�t�C6��x��LR�?������I�����Pt�o�>���S��M���TUa���~��CmY����#�����vu������c��1���vu����i��H��H��jW����v��t��t��Fua3��Yhi��S	<��U�L!7S�D���!	3��
�HB�VAR�a�&��a�,Ih��L
4�'4�'	
[%J���J�B����-'dd2,	��RW%���
��)�Wq��-�a�B���}�a."j�CF�p.T���U(7�"��R��<�J*���+E�������7>�@5�)2�.��k�i
e�b�Y��rD�K�� (��%@�	 ��4m
��2��3�3�^��Q�X��yNyNARv�����~7,_�7Bj=�+�A�y��}k����������#�[�^x������(�_V��H�c�H������ECanTy��9������X_X�����a}��R�/����r�}V+V�kW�_�
�9��bvB�
��P��%@�����%@�M�����%@�����%@LL�9���6vYwI�6&���`N�N��������f���fUZ�?��5���IZB��aZ�p&���a��%����G��C8jac3��p5c��Ex�!<��9�;�?nO���@Y����2IS���H�O����6�^�'P�O��g�_�`q�,��bi���%�(��������i���?��{�(�t����E�<�%��+o��s�����%�.^�������u.B��/��|3�C[����R�E�V�cA�53a�U��X$.G�LlnU@=~�Q�dm���`l)��
����^+���Wh%�x�k�v�������$Jv�����=x)���x���o�(��>���/��3`��(��ch�O�L�n��������_������������'4Fvk��#���s��u���jY"n����"H���PU����v[�f	_w�k+�����U-�����a�#�w�c���Vq�XK�'\�����PV������RW2YR�Z�g0���C$�0��PVe�}'*d��2�2�h)G��?��2����lt�\������/gg�����uN..N��_X��cN����]`E�����XsN
DV������{���y���L#j��T�i���T�i�O�TKi��J��DGj
�AIH;��*�M�g$��f�X�A� ���/���ar�t}�������iU.Cu��	r ���Hx/��R��s�QR�Y)�+��pDNnn��M�JT�z5{�y����������V�t������c��B���[j@�c�;�����-��cK�X��-E��C�������VKa�V3�����E9
���M�\��[&w1�.]	�6��@�.�>�uv���t%���x��i��;2�z��}4��s�)T��g=;�t����;�����MZ������)��{o���{o
�~ �&�H!���Y����X�_���}��������� ����,^��ee�-H_��Q�����_��X�p��j��9�������<�$PIJx�^M��?P�����av7��Oc��*�2���N��?�Z`��Jd�2��FK#S�F�n�hrO�m�S���x��\h9k����38U3���<T��Z{�E����T���G������"2#�s�D���G����X[Xk��&4e�*<8�����*^=8���W@g�:���u8��u8��Y��u8�� �u8��!�u�]����Jb,��'�e
+?����7�/����-�{�����.>`�V��m)�KA9�����_���o�p�#�����hU�����3��|�������s����z��.�.?]��y�i
C�~�1�t�B�'F�������?�����G/�0����pv������Pc�����>���?@k�����g����&�m�"�P�_Fa�0��j��.��>D4���G^M����w��\��u����?x�Z�`h�����fL�v���f���7��KN��.
R���P.#CH�c��(N�������w�>]������t	L��3&���2���B��!�*�.1�0�0+K�u$e����9�q�f��L�I�w�xi�Bv���l5�%_���l���3���	:�:�����F�>&�����FX�F�z�[�3��qg��hV��f��q�����������'�\�)���(�	�!�q��dp�8�OX�B��[<7w�&I3����^Q5�If<�����������������0�z_�b]	{�up���>��j+F0�EJ�Zq6�lT6����r:�	l���"�U��������������i��l�n�������6���z6��4�M�;��
�N��3b��d�.|��?[��F��P�U�������@Km�U������-5�W���C9��Z������������M���xh�"�������JlX�LA���?4�{�^���/�/����w�C��P|���{g�����~�_�w:G�V��c���������c��� ������6\A�;#��vF|��,��E|g��Y�uVn�(S��/gu��
q&�W����k`���R�~������%��$�s�\7lG������> e�D�d�m�l��S.��G�D�t�xG�����x�C�Q�j�h�`�J�D�i<!�X|��e�$n,��B���*����`����i# �NB�s�b�hH�0�
��	��!�h4�> ���$��b�����8���$�q� 
�1�n�0HC_�
����s;����4���\�qk!�k�P*�S��R�������Q9C�?�G�����-
�D��7
n�)����#�����U�X�Z��
@5�ob�-b�����`��
���r�1�f�3R�.�7�W��
5hQ��FY���YI�4����p>o�ff���|U���H�qiQSh�ui�V�}35�M
�)0��R��/����A����Q�g��6�����3���>R�0���G\p���6��n��#m�\7���������mo��������~���B����}����g��oli�[�=��M��6��4�������=���*rD�?b�?��q������Tl�kq���2P�s��=�i#�`@��Z������
�m�"�(�X�P�p�?����A9��������s����i#ks@���"�#S��y+t�I�D�R���?��?��(r�P���r�W?���������s���i#GF@����q��������=�,�������a^a����?�9LB���v��;�~,u�D�������g��%N�H������\�}�Z�~��w�h�o��G����%���	�����>��7���b�|p�nl���}p�D����@t�����x���Z�@5�ub�-b��.�ht���}��t���>��`��}��Q/>�1���>��:�@G��}��t,�|��t�h���@Go����"�1����2�9�>����fZ��a��h��P���L�[��m�����Y8���cs��h��b`�5���7�Y���3,m�UHK����n)����b��]@*�b�:����J�*V}F���j�X����j64&�f�j��l
�����B��t�DX�����+C9.6�����L&�o�%c���q�bq%w���Z���A��N��H��<�)��O�Ut�+}��-��n�^TW����f����v9��P�3r�B��u��$]�VE
�X�J4��|�s��p)w�y����b�����7��*
=�t�6$����]5������kI���E:*�#3��v	�h�>�����c`t.,#���4�c���^=���)�AL
�Y�^H��*,�kH�,ReWH���URT	��*�(�E���54�1�,�(mX475���T?r�^���*,����,J�����>������&���p,jM���`L�
�lX47N��HY�E����������E�E,�(�8�,�
���sp,j����hL�F
X�lX47g���]?�^��*,���LYtHYt�������c������]����q����FX�6���EsHL!�Y�^�
H��Hk��]��0V�9d&��s�G�BA������qo-e��6#��n�]Q��rn\�/V-g�f.o`-��m`=���;4��-d��W��/[����)\5�w2�X�,M���?��]f`�-�"8v�G������,&6Ie5����7YZ�a�T���@��V������L[v��
sPV�9�E�����,26�K5ly����Y�a��R���@��O���E���Yv���VPj��������j������=�m�}����s���d�{4&����/���i�>�h"K�	F��F-a�A�1kX�G1������$� N�������pt*��n��N1Z�N�Kv��"nM���nE+�����.7H�R������|�^_����-qk�r�'�9�mX��Y�f�p
D�� �g��k,�(J��ff���8�Q���S��t�h3��Fu��3�
t�EE8�����.��!	���Q�B������`T�{O[[bO[�}�������k^#�NF�)~���6qyV����^�q�������E��t>�T����@4�N�*f�1?����
:i$���J1�^�=�v�\q
rr�~�9A�x������Y���
�d���
��P�Q�Ds:"�@�;N�U�)�~���HjQ�E��g��o�N������y������x�96����:-w�q*i�#+wZ������aE��3kN������`���p�,:6
}�TA����50��sj��g�]=�����F����VSG"�nQ�a�v���i��N�E��P@������\�����502�9/~�'W����Z��0��5���V�����������:Y����x5]�Sxs�\�)7'��oOQv7,�~�l�(f�WB��ZE�C�����M�W��R�a�1M�1�8�X>����n��K�~�R��B(�f?���:�ejU���m�kZm�����R�H��*����i<��wbfo.;�Z��M��vW<�����f��cX��#��
?"��z�tx��%��.cD��
�i)�1����q1���8Q��q"�#�/m
V��/{��MUj(A��Kh��[V�0�F�:�Q��{���[���,zU�qb���D�Ne-fe����7 �M��7����]�t�p��@\���
ii~�k�h�|���-�pVX�
�a�E���7 �+g��.zY�*�"���^��0h`����W!�!"v��0OT���l����_XEX�Vn���i|K�lk�u�CcK%pS(���3��Lh���9�fzZ3���4[c�)zd��^t�
������e��j�y���an����F'j �+��r,�����aT;0�d�\��������=���4tygH���!��'�w�xg�w�����!�R
�����3�;C�3�;C�3�3�)�A��w�B��!�ReS���6�>D��8"�-�w�,�7�7�����ai������;F�cDOH����)��;F�c�.�1�#�y��w�x��w�x��f�1R�g�)��;F�c���#;n�}��iH�kD��^�f����=���4ty�H��#�1�'�w�x��w����#�1R
�����c�;F�c�;F�c�3�)�A��w�B�#�1ReS���6�>D��8"�-"	�2�X!�
����-������!��N�T%E����!��}�"TA�\�;#�����]HrM�7�x������A�[��X~���h�bD!�%n^-GR�{���2��$�=r\�1�E��g2�e�w����X+)��N��d]���"�O|N�9��H���a��Hh�
�dO[4B�h(=A�9+���%fZ�������>j���1j�������gj��P�F��.�Z��]�hU��,�MZ7f��K�w�#
v7��
������
��q��Q.W�C\�CY���^S12�2EiR	 �?�����oWH���tW+����;�,�d
gXU�4>/K��������#K��7�����Q�9��;�Y4��%������GOt��u��':��:;9`����C��]�i�� 7�7S�i��������}��7�{��7�{���7�{��7�k���w�����������hx��7�{��7��y���2����m�F��by�v��������H!�M��M8����_���
����
�2����
����A��*{�����wox���
����
���.�
�������m[\��}��dv�	�t����y)��M��^��M���M�2���M���A���*{���&xo����M���M��/�M������m�l�
�U�`||{YA0�7
���a�s���������������^��7�{3|�����������2����������-m�6�>(�{���n����1�;��/��������������.���������4��]��7��.ot�Fw-����������2@��������nkm����7��t�����t�9�2M������:X}]w�r5]M!W�x�����!���!�C6y��#�D��PE~i�S�F����;�9���]
�����0�;���fv�q��"�Z��c�����/y>���c�7��u�	����g�M~��A��|�`F��s=��Tf��Ad\<,J���]��C���
�����Qb���
���s��(�f�9��}(@�7,(�/1��7����C����C��i
���v��>2�}$�>h��>r@�����@�H�}dG����t��~�W���Bc���Pvr���L#�603��->��O(�Sf_U�cda53[�
�
�S�0��	��O�"��'�	|��'t6>���H�H���S�O�l|
R->��O(�c?>�<>���d�z`��hq���af����*�/����K�A#���qP{�b�H�H8�}��Q�������)k�C�\�,r�V-��2�����|qv�U��`Ud#XY[J�!7l�G�YF��G���l��f�V?j�R��MtZ���JQh6�g���*E��D���$�=f1�u��y�����x��c��������p�����.����Bz`����b?�h�P
����='�g��|��	J="B�QqFzA������`���c��<�EMn��-�$��M��0��u^��P��7
��3�{�@���K`�6S��y����R��%d*�Cf%|Cy���>��D�`k'U6K��i(�Rg�Tl�4���J�-y�Y.���r]�q2n�Z��:��/"Bs`d�l2aL�������@��������'�@�"[�1��	�j�3
�,�g,3�����K�@L�Q#�a��������f��0M�i�z�Q�T��v��Q2~t�BG��?�U;&����"v�����
b�b�b��"�����y������WE�ys�O�WI�Qa�G��^eq���&��&�e�^&�U����&����J[���dm�suB���u�N������	u����:�N����Z��}���@yD�<��{2�|GuS��
Y���tL:.��n
w_!c0=#=#=+�Y���B6`zNzNz^���)�}���.�=������+
�S/�����Y��� �T���t�MC]�����������z��c[�O-p��������\�n����W����q��1���t�8S��A)MA��|-��b	M�WDS�I*�Y��Q�L��lA�(�e)M���-�yqIM�������EiM5"e]-]��42Z
��e&ZWDC#l�a"~���������0�K��������h���%hh��*h��x*%��U������b\���2\�+�m��fY3��+�/���n������
_v��c�����
`��jv�b��A����yI�`��:���2��n�+�%y�4���9c��&9d^�7@���]4��o�[�%y�tPP���2�Mr��$o0�E@U%@����g���
C&�w�$M��y�h�~KW�mp��O�I8�MM��]��W���QMpH�����,���s��>�	��Q��{�8�pj�m���v��^QQ����
 P��	 FE
�6���he^C���X����Y�h�5+Km��d����V,5��z��.Q�(_fU/��,)��/|q�G�����M������JAa2��0;?�R����N��P�@ r����Q���A����J�h
�t=�a�$����2��{Z-%SOD��p��p�?����}���g���'�!0?�/����G���v���O����}���s�|��=�y����O�C���@?�G�G������D?<cx�?��9�>����D?���/�0h���A
�^+{�9�s��8��-,���e&��go�N/y32������=x)���x���o�m_�}<�^_�g������N���(�^�
�OA���%������Y(�e:�Z|=��)2`d}
���c�z��� �;G�B���L.�:����#���.�����j���
�W��?����V��?���vX��J����P���p���b*���-b��Jb���b-,���s]�;SMc�6�;�^T����@Vec�'����p���F����������G�z$3Q�\������`�����/gg��zr����������8}�r�A��{f1-e�A+5�B��Y�
cy��+|�e����������/>�����D�����]�.N��t��+�7���M���z��+BM��F�|�B��@u�}�����n"�;�2U�-=��BF��Y.�w����3�<H�t�Du�����2D�m�i���� ���mZ�|4���'�@��B�=�;��p4Onn��M�JTJ'j4{�y���0�d��JBE#�8���|��C���q>A���C(E��!�#��T�5� nx��(2%�'xS��f����,r�]� �����$tne�[&w1�6]m�6��@�.�>�\��t�s�e}��I��0!�=yYk��}����~����D�"�Qp����O�]0�����q�O�s�W��HB^��	�9Oeb�����T�13���T��0S�M@��y*�=�)�& n�<��t7{�
u�e��	�ZE�<!�*���V�C0V������H�r�s���+(�� �)��^N��te�hG�6Sf9v�Ys���VW�*��N���j��B�J	��IWP@������Q.�+�X��e��1����K��G.)wG����q����^���}������W+�� ���,^��y��m����c^��>��&������mW"�h�v1��p�F������
���@_Bz7��fz��'u�*�j
��=����W����"�����Ly�W+"3�:E���(Z22[����W�N��S��)��^4'U�2t"�rN����<�J��3��)W��l/��:C�r�P�N���+�f��n��m�<��p�W�N���'E�_h��R���A��A~��X��8���������@�gfP�7Cp�#h�+l�iTU���*�
eU����C�E��������I��%3o\Y�]����������Ko^qyy��7�x��7�x��7�x��7�TA��Wz��7� ��+H����BgL���z��C�|����0���-�������.>���'�AH������g���'o�b�(��nNO��,u��_��hL���;8�8�!D�?��^�y]_�:���������������L��$_��#����k2F?�����G/�h����pv����#J��
:9N�/4���?��=9�����g���������	��a��k�w	I�"p|��
\Eyl��N��{��N���l���:#x�\O�`	eW�n��x~��`J�t���T���eF�W��?����G�sr�JI����w�.O����[}&�M�=#�	�/����3���;���
����\�4�n�~�(/Q~�~=��R��~}�J��L��
�b/�}!jO�t��A2�,�s ��9p�^B���O�c�.`m�������������$�*���9�B&}s~Z����=���E;2����G��r.7������_d��5���/�t=+8��N��Q����<9���&�|��S���k�<1��1�o�������@�c��:c��'�ChwW���RFq���[��g�k~��WE��3����c��p��u��x��9Su!�1��O�f���D���E2Sh���H3��Nf��Q�F��ebI���'����Wd��~H�I���f��O�L����>�rI6}��L]b{�/��vID&��U�)vd���)�d�	��I�*E��bD�>�rU����2��C��I,Q-�ie�1��T��,X�
9��$��:�D�s�Tn
�-����2�t�P>�3����
`��D�3��8�Bu�E��bqR��=�T��|D���jFO>BCsk�?U�'���%�&�..�_�<0��������|L����%�+b?@wD�A���(w��m1��xM�i��&��cD;��~�H����&u)jf�$���)'}��Is�+CX��_Q�a��!��(���@j,J>�z��Q6�_��0�����j�
<hu�E��n���X����������QH����(��y�j
�0K
��[�I���:Q_�Nu���=5)T	G5�b9����z!����6d�E�h��I��a��6��������f8`�s=\��1�_?�q�����\+�8X��'�#���s��c��#�g\+!����!
���o��q�\�=��3�?�����1?*�o�[���?b��c��7�0����4����������`���Q�?d�E�9Jpt	�q���
����g
���Or�E�>�O	���1?�x��(r�q�y��Q�s�<����l ������������������B~Ea���r�\C��{/�{��v������3�M��5�k3�f���cC�fa��,n�z�����J��F�#G|������9���C~�~�sG��������,�F3�����?n>��F�_�9�D<Opk��~����_�9N��4��|����=<���x��1�k2��wD�����������=����3��o�<q������p+_�A��?���q`�s�q��}��������>�������(��F~p�6����<�s3��`���}Av�Wh~E�wm�?n��=���>�����x��W
n�9�8�#~v�+����n����*��@u��'5�@���q��n��0��z��y�[�<�8��e�#~�WL~O�gG�#~5�eh~G�p�W��=<�p���C��^���An����O~�W��C~��$2^��%2~��Wj�g�:���<~<=y����x�xI���x��#^2���q������8�����/���o����ks�a�KF}����U� ��pr���'�U��������y��'�r�
x�q��1��%��.�C�q���2�u���.N�?I��e�W�E^M�������7���h�A_����eq��Yh���c�6>����El�j�K.�QCOt���Y�h��m�L�b9<6���mZ����l5f�\T!+@�F9�*��L
�n0D���P

g'��������G9���(<��AX�>��)K��m�3���lS>3�Y��Y�s�*�����e:��q��g�=	��F%|�Q�@����3?��=�t�{��!|�Q�S����&�s��3��=j*>3�Z���3���=j�>���.|�Q�P���U�s����=j�>���R���Q�s�5�O(dD5��LFQ�E_����a�lUDYT(bY���r%{��"b��}�c	�J������gu=����#9�h�es����$,������O��&kgA��zA��jm&u@f%3��cy���J��aM����pVMyc<j�S1��+
��TV1�D������l*&�y{����#fDr�7����d���+#���W�B�60��^�����&j����S���?{e�����Oi/3����E�9���!����\(�=I������c3�6�m+l�F�Fm�Ix/�����MM�f"6�|����m��r5oYTS�/��d
4h�PFR�gr�y���+ �,���p;e�����u��0��0o�Fkd�1����l��G����lm����Q��7���>x?�}���S�u8%x�bG��cH���{}��	
�D2WJ�P(#�
�����<\��Q,&�J	�XH��2BQ�����Dq�7�rt�8+6_�����+f��s6C������Z��
^��T���	�u����W95����R�=:���Y'�$@��e����'��EEK��q�Zxr�H�)��4�Hh����%J�oX���R��,$��s(:�~g��c����=[]�W���.kY
��^%��
�&�Y���!��F������p7�!Y�s3;�\G
�L��^v���:�T������`+V���:�Qs]��'e7��&�eJkI�\��Z������E������R� �Vk���7f�h���?��5�emh�8�';���*�va|U���!0���)<�"�h�������6��wE���2��R����YI�����T�#���9���������Q_-�$�����"����M�dp���-����9(9�+�:f���W*�����3M�bz����-
_t���Kp��U0��7�
�����*t2�90��b�$`3��k�[���T�O�Q���%CJ��IR�V��]*"�2��~
��dg��KU��NS��D���K�K���Y��D��K��H^_��D���K�3y�E@K.",L�4I�B�%A�A��e|���f�Q�.^}
q</S[���d�y{���/�=�<�8y�w��?A��<�:��_,��/�2K�t�U���xYb�2Gr	�� MIg�'!��O@O�����������.+=�� ���*Uj�>�y{v�����q�������������U��zau���_v����)U�d�L���HX��2�s��O���/�*��N������4*�d���t��b���!S�+

�dV�� �b
ezN�������f�5kI�V�,��w���.�����hQs������*
>��ot����q����I�����C�@���d�v7a��L��-��.����e.R��!���L�
�����y��Kbz=���O�l'��N��`T9���	��=��P:��A!v1Qvp�1���e� �L
W+�o�%������Pv��l�C��A�/e�"��=�8�4��`U��|�+�0�K���2�K)��o+E��/�S�p�`�g%]cQ�\��>���L�t���MU�!(���J d��_�[����������������9����B�����X�3��M�rY�T��+.�	Y.��sY5)�
|��������-��b�n���kL����+������i�Z��� `M��C���P�%�/���K����w���!|������5C.���oGa���Ck3�VL�y:��d�h�K��V������n��
q����\��\�U��.3:����C���Tp����Tp���r�������ij�%�A��8��)W\'e�3O�0'>����|��,��g/��&-6P>`�p���!�����
LE�j����
(w�WU<�<��w�����NL|�
�s�Z�������U��3�/��-��^��9��e��=g�`d�"&�g�y��t���'����|�e��'����
��)�?�������8��v������%��v�-�Z�n���w�d���&��?�G4ff;e����X0�p�$�PI:��c�Qg��0t�5�<�I�������Qu�ne,I�0�RR���U�$���r�,&����	�J&&�HZj�,�������>���S��<��{�������=�t�>������Wg'o��?=A���9���LWA�4]N�������f���S
3�%�Q:��\�gI06xho�b_����h���� ��O�p�?��(��wpp`����[p��E����7�+@�[�g�I����8�,�w� ���	��	|;�/V����Y\��R*�p������4��w`�J����e2���~M����q�n��$����f=	���m|�������b2�{����g��j:Y�Y+W�*��.���3�Zv�w<�v� ��2M�o��b�k���?��sO@�BA����B�Z��*H~[O�Lfq�BlWS��K5?�+^����E��I�=Uv�����������z
f���*��	O���<�_��I�Y��*��t���tS��;8�z��T�0<��
.?Y&�*)����q�.���x���mc��5:�>����
���#	�po��>��/`v?�v����I��������a#��2���W�-w�C�^��gpy��28=�x��A�
��x&��z�r�g4:�:M�B����$�'`��rM)n��)]���l��eL>/R�����M�i�b �w��)��_��jp�
��\l1��f��M�|�3�"���U�����w������$�`G��� ����D�<���e��!v��&��\����(2�Z�f�d1[���c�gp��<���3�tw��U����$�Y,^����������:1�O�6��?��5n��������>��t�������D&�{��������d2��q;o�?�]\o��=�18L����������;�="��7	Pw����r���^7>��~�)h�������_���M�Jw�x>������3`Q���W�ZqX���{ W�},A�Y��0}\$��/	xh��%��+d�}"Pq����l�A�&�<�c��j�B��L�_��8��vdfT�#2��Da]&�05?��	,JlJlb6��(T2QH-�R�(�)�e�������������[����-�+��gL���F������]��ON�mid`�V�u{�]+�-����\M����� ��� g��Oo��9C�?t�S�?��2qw��7��bq��
3��S&�t�9���v�����)�*����u��F7l,�BE$]���s4���%`A��&������������n�1?��[�}6T�R���Hn��FM��\3X�+��f�\7�h�F�����8��������=
-o����`};�-X�Q�Z����9J�%v�;�����e2C��t������m�BM�b�)����E'~�mA^��yc|�h@u�!� ��d��a�c�7�:h�����������6�C������W�Ve���l��5�R �
���D����s,�%������"(;^��j�U��g@xD������uZb�:-�%�!Bk�uS(�<KZ�z����%-Q�E�6VM<�(�sfw:0������et��:>"QF6�Z*so�����J\�����D����H���"&%�1U��5���X�j���B{V���l�<�B�j����)��Z$�Z(�Z�`�P�j�Y-��ZHX�M��$3��RmC�v��F�3-��5��9H\������E�:�h<k@�� ����������x�5�D�:`
��
����Q��%*�Gt6svL������!H����,�S�o ��� �2�0/�a�
�8�*�����<~l_x�9T��z!������
<���[T��z|������{-G�������pi��J�_�F�x�m��v���Bt�LC��`���� sF	E��n�-<�*R�B$^��j�9r��W[���I�M�<������T@T��y��(cR�������\&��4[�7����&�c��D���1�2�\��6&6��p15�9�0t��I�=�	7I�<'��7k� �f��g��V�Z��������G^y�Z�l
��b��'���0��S�����������9���K���������/\����}.�?��E������<�\�>�����"�s8�b��Fx����,�[��Vh�V$��o9*������Po_�n0�u��.nm;l��a���.i�#�����V�cd6��~'��m�o�~}����A����*m0E������&g���ho���'l}���'�m����{g
{��>55���6S���J��e��S]z��e��nP��x��P�U��u�<s�h3o�����vhY7��,��s�f��l�n��^�m"{�V��0�*L��[�^��d/�j�-9��^o�S��V��P����91Ts��UcN�s�*��V������n�C
'�����m�����Z	�zc 6S	��:�2�@y�'�.�R�cN��z.v7*D9�8T��
�s�]&��D��9��'�d��<QI���Ei�]%�+a�}���eg"o^3�a�Qf��Dz��������������]q����'�
����}�kW�j5��v���g��#i��-�7��P
�(�������0��f[;��@�eg�>�)���Nq�l�6�)����k�u��?4�Zq��<����J�(1�e&���h���Nu$�,;�.���n�L]�G����rz����G
�x���3��T](�U�m�.��b+���L��
�]����VN>������~n��LS��"S��C���@YF��>�O�M�HH�3A�b����t���q�P��s�����?_�����1|G������#�#���Xgh�7�GS�������\_�a��u��!;���WO����/��eL����7���p7I���@���D����D��n��q7k��5�7k��`Z��9+�F�m���A�9��s}����V�����vC>fy��������W��,����������+�/1���I��DHN��1�5��������A���������<a���M��nfY�e�J�x��Y�B���Y;�����<G@�g��._�&�z��Zs����g��yr��_��y�����$�s��vi��51Y�C�q�n�������n�m��
�������D�#�N8�Mt�
�,����q_2��~�v9�sn1��$�V���s9���E���G�p�r��U���Q+��a�'�����C�����tCT:�2U����>|��:<sn���s���/U���/�������t���n(b/���H=`Q��
�(R����00�
<��m9�V�v$��mv$v����x/R2���H����5TK	Q��1��x+[�z��#a��U�Ur$���R���8��`�9��M�8�:MqM�f���`����l�\S,�eO�x~������:Y2"@
a�I!�Aa�f���@�����=��c/�� �m��`��_M����q����7������>FJ���^�����RU<���-�!���U��o�x6�F�(:�����v��qn�K�����^�W������*Y�t�iK��R<U�V/�������y\���\��?8�Q�LFK����b�j�7�9�_�(f5�"��H�93y3��%�(�pP6��xiN�+�!D���CH{��Q�n��u��+P���,���P�8A[��8�LUT+g3��7�??q�V�-q��)`
t�)�B/��D��t*nW	���AO�S���R*$�1%I�9}�-I�����i��jqX(CBt��I7��L�JxH}[��T���\};�NV��:�t��WX.��R���/�z�P�Tq���GXk�����]��#T/J��/��?�.a���Z�C�"h��T����*qw&�P���#T/�z��G�^>��.�f�����uEY�V��,Zg��.x����Y�w�5��	p�Or�r>j�&0+2�����C�V��n<��7�N1�r�)��e���zD�����^�j
�*-}�����C�0�LZ��6>�mz��c�!���d���v�W����fX1L\MJR��
~!��_h��a�-<G��C*W�F��X���j	��5����Q����d-�d�M���I";�d���h�|�����2������,E���M��������q�����r����\�}�<=�x���_�O���<\%_T�8\Vv��B�BueM��I��4m�%qY�AQ�D��**;����+;���!���F�MT�d���yu4�q�^�m@k%D��@jH�-:<�L-LX�0Q�R�p��>l4E�-]6wD��T�%.���Be]F�F���f�K6��b���P�u8�xK�[�tT�tC �I����n�������J�w76e�'b����m
��u�p8f�����F���
��I�w'n�8�r�]��YAf��S�5����g3dO�����t���o�d�����6�5?A���$M��E�Z�� Na�d�\�r���e1A���2U������ne�8��p�f��
�J����c[�b���i�W�vr��/�'��iJ#k�2gm�3<\����Q�������{�?v�\�����z��E�
���U~p����o�������"w{�/�R�Z`2n6�o+���Q��N}�C��:"*a��D�	C�
v4-����P�rBYG�5Z<��`O��HO
?�dj�vv�u<-�M#���f�&KZ���:�&+���FiB���AP�D��is��J�rM�.�vB�f@[o6��-�	�`�*�;wc���
��at����
�\�-q���+l��ZX��D\c��DV �[��C���tq��Y�����6�WW�t���
�8EA�0��Bq�AV�YN�2��WS�>Y���A����/���r�t���$���
<��[��,��+�n������{����wx���`p`�<���$��'d	W�y�DKE�C��9���L���d~5�L&�F9������C���"e2r�gR��xwkDO�0Z��<���~�	f
J����#�%������2�x^�������o��b������M�E���n>^�nTm�Z�������������E�3��K�L�W�}�N�y�7�����_�)���?����W��{.d���E�s�����7������~C��P�/>���~�fa���{��{�^$������������U�
��d9�Y�_��=\E.�d���8��x��}���~��8|<[,���I���McT�I��xH��N����Ya��#w�8������7(�{��N^p��SKw{OJ/8y��N����a���.^��	�i��(�q��4s��<t�|��t����|{�T����g
���>�W���6�";�p�����L�,���M����X@_���Z�$qV��[��������*�Z��X�b.�|�{���-�f�C����QwB��Z��[
�b��C��#�u�z�=��C�/�!����c�q~a�bm�8��5�����YZ���'�V��"1
���"�!��|��N��#�����������z�o8���
Y�w����) M�)YL��GJIj���2U)�4ptdF �1��$��A��c0��UC������*6�mn�gqn�
�H�+)W�����#z9�~p8oP�����m����)(N�I�4���>�-�6E�ZF����<��$B
��;�59�v���$+�)����V��Sv2#���n�1�*��u+;m�����X�Ka^�����5G�F�����m���^W�Z)9`��.���@���Oft��&%�v��5�����m�l�����������Y����	��Fg����xi�B���g���,������������A�o�8�/���:�z�:
4�:��mw���aP���Sx�������-����S6gy��UB-U��~�Z�PE�b����=��p
U
NWQ�f���[��k	�P���@��?�� �>������t�9hW��H�'�J}[*0��S� �$�4R�S���p;�	���2_�.��P�������v]
+���v]��X����uUC�j�]W���j�]W�-���z]�P���j�����j�U}���j�d]��u5��P����u5T���b]
��j�^WC�����p��j�YWC����M�d�zF2&mI.�HU����d@I���=�d@�L|U%�;�a ��)��o�NS,[�w����T��
�7T�?b���r�Sc�<T�	��@��l�����9]�@���V����m�$���eF��/�RJ��K�^1-��9K�$�"�yt�K��N%�IJ$���Ji��}�&��I�.(���r(%y���8�%Q����0t��i���f4B�x����������V�nDqV��.b�4��~&Vo&[��&Q�Y�,i�i�����)�6���b?D�I��������-��]d�\��f���m�����|[J�
N����m���E�L�����Rm3���Wp���,�\�k����fE=�|�2��&�^������7����`c����nor�V����Z[k�{bY��bc\�e[�~c����MH�����$������f_��T�>�l�\*��<�@/�zA���+8X0
	��|�T$>I���Y�N�N����n7o�H��
U�����������-��J�O0�K�������p�B�r���2�a@N��Hp����/�l
�������aX�D�����g�����a����a��F�MK�o����p�v��s���hr�WR>�(�$}��>���G]�s�����/tV��-���7H=��:����o�hL�e:Hq/�z�4k���n�i2K�I!����=K�W��z$�N���}��~p����w��CXP8{�C��	���*�(n��D�}�/w6M���q<������'h����;�h(�=z���p�. n�����y�����������0�@�r������������z��
���usH@4������Ap��3w�0�#���	��l��w�Ho�1Z,��%��H�F	74�y���nj�5_C��:E%kl��eM���$�[�����'i�^r���P����[<:����@�.jT��l%��<^�a��?�8)6Y-(u1�L��y>�����.?]���{K������p4�!�g�����(��ty��.C����k��Z�n��f���V�|�v���HO0-���G�t^7A�uS�^7Jj��N&�"��:��P��"+�����d�@�����d��Mf����",O����x��h���$] ���],J�<r�*����w���/�/�
i������w�^\�x���/W�'F��P+N%�kk���]WGrM�\��3�g��b���z�6��
����)��`��|5
��_^���$���@����gBF��$����b�1�t��[QS��!jj����X��?�	�,)�;~=&~��b!�k�ue��2�k[�sK	Xg'�m2_A�8#s�o�x��Zb�PnlBX&���j��!���KG����f���r�?�U����-Sn�g���@��@^��yS�0������cw���1�.���X�LL�?%�� N��QH�)@r�`9���.
Sd�b����1V��/v�z�8����]f�H���^�����7W����P]N��t�\����w%P�b�~?�&M��Jt5u/{��lh��i������:�:.��I9�#W�y�o�������L���N7�T��=���t�:����T�drp���n:��4�WB�>k���r���I0f����d=rg�
F�u�����7u4Z�B!���de��F���'n��"I��ID	Til@��h_�U���'����\sD������V�B���b��(#��'���{������
SRDO��uc��A�i���l�y�����RI�j���
4j�A
�549���e���d'�EY(�0��f�v1d<�����0c}�Z���k�
���m������[o�A{�����@7��~	G��GO���f:�83oQ���@���a����Y�����^ApO�1�V����6���A���V���CPq|�Wi�k��C�QE���v<���=��Y����X�0tX��}V��|U����H�RYT&�o���s���6�*
��N}��8Xf�b���x��t�f�G���`�i�
sO�����#j��0�������q�6�<M6�K���L���v�Q�l��L�����2�sZ��� V�^6�kM��mf��+V��6E�Q��m���r��V��a�i�����f�fm�
�X�=�v���N��<�A��[�y��
�c6�5��b61�m|�U�d1)w���U���������5�.��#��C:���b�Y+��p��,4��
BT	��P�,�`	F�np,T�z�$��)�,U�7Y@b���>^��������8rZ*$�s��m�(��f�'��L�������I��d�M��L�],�CO��AK�r��������J��&�vt��\��:Z�e<���~�����v�����������:�x���N1�6qv��qT��;+G�#^.�o��a�$9W[���t|r������������	q��^\%�Jk�r�0.v���mn������9�bP�����t�*a��Kx9�g� �04A����P��Xjn/
J��	$2������
(�x����>\�}89�GU�<�(e#���Y��V���s���(f����g��.H!�)�xuv�����tOB�~����$���]�T���_�K���w�L������S�����l� ~��
p��F1�������H0Xs�L���������{��6�������G�"�vs�$#(`��>�<�%kF�
W��W"�T�����t~��j�_F��+ ^�]~�8��}���K��?f8��O��#g�	�?�����G/A�W�8;g�k�(�O�P�����S
��c�����O'?��U�n��f/��$�e���M�}<}����K8��?(�+�����*���c;%Y��������ad����_� z���t�bfPb�y���o,�<HP�{���)-���mDE}=,�M����gr�L�@?��Q]n�+�������)7�(G���L�'�k#9^��
^�I9
�.���"�����qDv��v��-��s��[3 �{�8o=h�8�O�U�w+s�������jI�i�d_o�����c���V���^t�S���v�m�(��|�#��fD7��]��#&h7*���T|���R_n��G�R�P�����
�<�
����,Y�
���&����+.nP���
�.�/��n
��G���A���V� ��qB?�^HB�)G����
�>������-`���������{
,N ;Ol�t2&R	Ax`P,*��X\S�QX+pX�G���B���r+?H��
SuCA�: ��X�j�,�s_��|����6 �0��3:U�:'�`���2�����������2����)M����|�
H7�v�j���m��
��j�Ye{���y�D���)�A��������&������;�;6m`�6�j�d�t��o�G7�Ek��E���v��m�A�U���Z]��Qu���;��������:��6Q\C�����A��M�Gk�ru9 ��!Y��peA���9�S�k�i�v�^�	����P!�
�o����|(y��A������4����a�~���h]3>�L�#�u��W�dBb����Bs�"����F������2@�BA��&�v�0�|�[T���~��M���R(6�����-��N��qX�C������X�����F���)E�Ue2��(���m����R�I�@�f���"��|����k����V��j0N�B,��������N��r-�������@���wT����p�pJ�4N'_
�z5�	�='��`�]R,������g��,������R1�B�LW��(���t�Uo�>�/2��lzA$2z�!sV����6r���2�=������~��Z��Lnb�`NK�0������
�V�a�����$�`\�rK�e����M���Lqn#�/G���?���[�� `sR��J�m������W���(��c�!�`��z��1�`��������f��>��1}�K��Y�GA]��w�Wy]+7OO�z��21_�G-p������l-���~A �����`s���n"M����a�Y`/.I�E��4�pAE����v"�.-*��mN�V�	��Kw:.$�;��F�������7`�fPm��LpuX*���
Xt�rGb�#��{8�.9*�N�H�� oU���A������U �����TCG���9�*r���e�!
��v��w���GmS����}�4P���]T&�V-�-��R���������(��Z"0j[�[rXn	n+�mC5��<�v{�tjv�nBM��h��9.�F����{3�����)�-i���oD��'oG����r:����np��|o�����C���yj�fP	�����@���16S�]��d��&>[��]n���&��;ecO��Nt����9�7P�����,�����*	 ;��j��I�>�� �M�]-�fc��-���b�5Ym8�%��]��W�ul��Y�Q���%K
�%��%�dH���������n��a4�B[*Y�YK���f�[�~�F��a�V�6�CA�MY]���fy�-=���[�(M�u���|r����������>2��lh4p>��J��RWx[+��5e�w��J��Bk��q�GVQyW ~�����r�5�Y���)��6��z���9���PV�y��\���a	��4]i+@�r�\��(�s�U���*@�>�/f�V���eVgPf���������>���	�����Iu��p�����aa���t���zvv��Z	��	�[�M����n���$��1@D �F��������Nf`O�2rm��t����3ottT��b�<�j���*_[������ )��[2J�t�|[U�I��kz������'\
��^��Ps��m�R�~���������=)VX*[���P)&l'�pXUy,�Uq��V�_�
�f��h����x��2x+hP�Fxf����	��������\�-1�� �g,c�>Z������f��<W����z�D.��h
dj[O���c�:��*��4V�k���X+��n/9���W�n6�����A�PY�4�S�������cS�����H��0�g)��;C���-L�-�R\�,'����L���}�hoarm���rv�(pc�e
�������zSk�n��3�m9���mW�2�Z���������4�l[$�3
����t����������Y[���[\�69�+p?��ewI�pV�v��U^X�����H��s=Awi�Q�,^��Xpg6>=�R�;3��~�)�[Z�8@���'C�e�q�+
w��_L�w�1(�T��Z8S����2����v�^<L�6wx��l�G��N�hfG
�N��'W_ZC�U5�������B���`:[@�x����&$��m��pNN�A�B�;��p��r��V����
[��U�=����[�n�����+S��y�R�E�qxj�4o.�nN�����)��<�Q�A�[4lN (;T��`��a��dw�S������W�0
�&�u<��ii�b&'W�>.�U2���67p������(���w���� $h���p���!��c7��1A:�����s@�9_��� N�t��A2�,����Up�^����O�c�.������/�����>�*�3Nx�3�7����������!cJ��}�?�G�w�G/�o�.�+���=>�<��CfD0������g�0�=���pV��/�$���X��ReF6�
 v�=f�h %^#�q��q�<��/l���Fo����u�3�8�r9�7����`��c����g��w]�'F.�����q������T^��lo��'cg����m�s���#�rl��s?�_b�� J��	�
�v�2;�-��_���;j��x�[L:jfiA���<Rr_��f�0I�J+����wB���K0�S �L�"�d��x�&@���@�����t�z��E����K�&�������?��� ��#�W��|���Y��L��x$��Cv_���� MIg�'������r��������!*��mC1�W��k��s��`0?/���s�~�7�s��l������Ip���C��Q�Mq�0����o!\�����|����	��y!��i�H��
8�?�a��n�0�
�u�A���rw�CC$�}~6��S��(���T:ml��2��T �o�R���d�)�-q�l{�*������r�����������r�����Fp���1���O{�^xx���$�f���)��6y����4�;:����a����8D����a������������f�u���A��&�^-R-be����.��A4����������8|><~6#r�^
�����j�?z�����.���O��AFs6�SpBh�9L�����~��Z����;~2Y��i�U�J~
.�I78^'c�h���ip���yp�F�����?��S�_N�e�>��2e�	������a\$��w�7�����g�'
��#��D�Xp2��I��]���s�@p��C��U����:�S<���u=Z)q��Z>�����wd��I��k�+����\&��b~�����v#u!#�$���H���"L�m�
�6�����^�_������g�.zW1.9��9d�`�{�`��z�`�I<�L�
�����i��J!�S��l�����~<��@�1 ���l�[C���!r�	�e��t9y:�'�&�������v�����z5�=��<8	~�$;J�����l��Jp{����G���'��U�>�����O>S�z�x�}F��t���h��A���X�,�;�� ��������A�(8<�������F��M�W�}���� ~���(	�g`�
������L������Qp>�������!���g�������P�z���.�������I|:���t����tt=��������Q|��
�hM��t2J�W�1���3����n?H�0��$x�n|��F��F0��e�����
�hMy�\,V�=���z:��q��A��W�7�g�4=F�v~@���miF�Y�����,y������e���y�Of����_�o#4�Z��"��b�M�%^�L���� �M�q��ua�@)R�5������e1�zC���)hKE{�BYG*J{���{�9����C�E���h�N?�|$=��=��>��o��3^,f`���
���~x�88]�������������5X��7���������K<����C����v0�E�A��F����GP�j?����oO����7���1�����-������=�|�:9@p��FZ�28?�y4���.��1=_�]L~�#��C���Qp��?�����h~���r�u�ZNon��j ���������$�&������j��,�	���cC���G�\$2��0
���-f�k$��� g�'YX~�eI����5����P3Lz]7��o�@h��$���*nY�G0���������l�;����l�z��('&��g������`=g��Q<gjz��O�+����~�c�O/���������������p�j��>�����S�� kh�R��fS����f8���@��s��(�a'�')�:�=t��PNA;�'z+{I�h�`��f������+���9E�8�������70��?��_�%@�w�a�TU@6�.������Z���tsV��j�
�����4��JA#:?��*3��i�d�A�<��*�J����=�����$xs�a�D����G@j�X`](#�d���#�!EiCW`$��Dt=������/e���r-
�L�5x����'��e������/��z���	�X��%H@{�i�^����H�!�L�L^�?w��w��f@z<��`h�B���+<UE��_f I�+!%�������4�AD�-xO=
&�c/'���B��X������i8'4P&jng6K�=���>H�x:���K ���K��;���@iX%78�*���EzG��P�����,����H|2�Q������ND09[(X����6����u8�/������[W�4�	o���R�k.�;@��'a�QL��Lgx���������V����
��'x�m2��)������3NbH?/��+��0`pH�&���E#Dv.
Z�d��
�@$�)�U�H@�%D�EyG#uW�'�3��h�"�MP�t�,Jf4�3����
�W��b��5v��f=�'zGX�A��b��|��_��x?_����h������O~�N?�
z_��3�-���52;���m�+��n��+������N�]�x�������qru�/��DN�A<!�p�����#����Jve�RQ7t�.���lyV��-2;0+�b�o1��@�(�`��oP����v�w�@�����2@�V 
�Ps��U|{��?A�I���Y�|��'�
����H
�����d��1���e9l�������/�3��ACg�/�)d6|k2_Q),��gz�R��0�M*V���tw��~`��� ]>���o��|wW
8z��I|��:�M2� �w�(`.���c��qp��pg�E��g����j�<G�����B��\��Z�;�� ���`�3��*oQu�$T�|
�i�����/t�/��O����	x���:���z����+��
�T�����!�������?�#��IN
H38�X���xdq�A����_���(�F���CF���Tt��4���H)v]!DU�n����M��\����p�|B���p)�t���1n�\ D'���'�TA���%��@�,�������b�g���?����]V��#��g�Uo=O��8<�E�����R�S�(2������~������:���a����;b*��	-�@��N��@�N&��t�Oo�/G�N���>]��<u��mv���,NSYW��07oc5d�(�SZ��g���od=�A��H���+�����<|���������pip�����]��4YB����)	�����g���� �a�l��It�8���'����$��\FS����`b�������e�<�8��cI�g�{n��CY�x���I�Z��{�q�H�p�~�����7:�/�����2,P@��]���/p�������7��L�P�`���2 �����#$/�A�7sh�B�S�Q"; ���
0���2s�8�;�;���/3
���I1n�������v	�����-f#����`Zd����Bm�"!#d7A����M0�@�r.b

�*!���T0�=����d��j(S����M�
xr�����tt0Xu��0�t���@�^��u�T
���f��8����d�=�A���cO�����,������wJ���jP���TK�F�<	N� Tl ��&�6����~d�N���4W0C$������<��	m�/H2B��0	O��.��@��@[c�)�����kBmWi�c���C�eI��0ro����Yi��y?�e����t��� q��^	��3���-�������!��z�������<��t��2G)���7�!�=��!���aA#�W�����d����0b�tV��R�<������V�>\��p�7X$�P)���tq{�^%��5�� 
���6�E��	��/aM��(��.>4c�����Gxs�$um����g"�_
{�����'�h�,+b��
���$�� >�1`��tH�@?��H�<�*�L���4��.u�]tF�Y�)�f���?b��`
Y��)M���r�GH���������K| ��%����UN��\}���Y��ST�'���x�Y�b)�����h�!�
��Y0I����),BA��\�++*I�W:����M�X_����8�u��9����8P&�9��	�sFk��#�����n���8Y�bX�<�|�J�Q��^w��tX:Hr�����_�O�:�{�S��AM���Aq8��,`7@���>�?1�~�E$x��>"��X��`Wv�P\T�:����z��t�����7,beoP{*l
������u��Vm�mB?'Y#�Z���`�g/o���+����G��T}�V�=��M@A��n��1M�9A7�l�B�E�f����|p�������Dd��bd2�A��DV�%�,2���',�b�/�/�x,������l��W`��.\��J�+���������)�X���C��d�A���;$O�:��qs��`���[$�O�{@$S�\)v" �C	Xm<��@�E���P������N��}I����f;��]�9 �?��� +a�i���xh�M?8U�l���(>��0p��m:�5j�+� ����A�1Qy2�z~��?�nZ��"*\�A������#������\W�J��b�GW~�l�4���	��?���E���`J}�8���YOzZeA�&��x
������W?d�o� �oS�����oS���G��Y��rK�q��o�UO��D����S�xW��Gu���7x$0PO,ls�X�������@'�������)K|����}�6u���j��m��5���M�}�](>;_�qX�3�FEQ�_*�+,k4�,��"]���9�}0�&��#l��_�`��d�}MG��pQ\.�&����U%@��Cy
m����E/|���r�j
�|�V�{�E�y, �����x7M������@�>��P%-Q�)m������sE0y&S`Z&k���kK���+xP����$^jb\-\�/)�������c�i��t��b1�}C���J�i�g0�b� ��u������+��,���*~�����Z�N��!�0����'nrb��=�rD���\�*��yD�ub(|�3�._~%\�Y��@��'0P:�M@3d����Y���#�T+�33��8I�C]o@�qOA�[�aF�+�/����@\��l��u��5Yb��k<�R	�����b�GF,���[,�m�((=W��H�>5d�0:����V�	�����zW�6Wp���������
����h@��h@�a7���I������	`{��7���d���+�F��������I!�7CLlrjy����pQ9��b�$��2��tG*t�g��!n����~�h�^��G�9r5��NxaId8��u�D.��1G�����#\hi.\i�d��l~�$�\�PX:^��wPO��8)UF"Z%Ym��I�vb!�W�0���H-F�,��`�a�������	@�i�q]%�M��C�qln�fDi���faz��?O���NW�4�!����p�8�,���1�V�����r4�!��?�C��C��z���&.\���<�\�WG��?���%����Y��G���0y>$G�����?��?�ip���0D�0�2R#�����s(q31�������D�]-�t�hE�%����_K�pe�=��y,��x9]�STA=���������|�OK���(��Z^���ir;=���Wh��6�6UvIfLzDP���@J���OW~��`�F-�}n�!�e��'5�|B�=B4�����@A$XZAAZTH��R����A+O����]���M��*�9NV_�Qp<N0i�"�������������PE�%�����s4A�s�2f����d �`�$x�����H�K��NaB�o*�C�X�n����]�	h�	�Y\�{�B
����0S���t�E��P��f�l����Zv��Am���������*�����M�zL��ye���96���
$ ��N�pV��ba@�������U�X�.��� ��]�<�<)�iis�s����'�/	��
���(�HI;��g�)�DO��9#�@��e^4���������(ip���4������9��]���JDr���%�LH���Us��W�I7�<`/�wdA`	�uIDq2Z�f�"��B;i/{��bw��4\�`��'@$=x�
�B�h���-MxQ�^f�z�c������P�-�'Xe7�&h�UT��WL#o�Ed0D�N�K�:B��H<����7^���]�Q�>�M	��z�����;���;���A��
2����4�g���T����?���@�������t��:R��ltSh���Aj�`[`�.n�8��#N�h ��.6H� �b�� Z�8����jKCl��2��z"�!��.�	{��%3^�/�h��q���h j��h�g�:�P �$��0��
��R���y@�h$�8"6Y="��
B<���"��L������#�	���������-��77�0}�L��3�p��T88E�C���i��0}�H�""=:M�
LWi2���=���$l���8��>e�7@��R�
�b���9���&�j+ Z�
aT**!S��mi6�|{�=T��:K�gh����^�
�'P�&���J�@)�����k*?�����g�M��"��*���<��%+�GI*��O�L����5���O���1%x�x�	�h"�7�e�1�����XB���+�@��S"do�"���|9�%�����������?s+Q�`w�'Ug ��H.�CqI*�a�*0�3����[V����5����fV{�Nh� ���{�\��X��,�� ����:	���D�����:|�	���N���h�
K��t�����}��QTO�S0_����J�
�rI�C�����)�p�&��r��@��^��=�@�s�
q"Z���9%G���O�<;~>��&�:bKE�:�����|���d�C�����L:��0���bkQB4q�@e�.PJ�|&>�g@�m
f*����%�`
��l,�P��&�+u���AjK�,�B���g���FX0r���P��q(`��+&����p� ����>;[C1�^r�<�r9�bH���w�@m�+6~5M��M���)���N���[\!{I�Tg�FK�`TW��u��%-�M�!2<�
Z��^C�*	�*<���!�x�)�����@Jhe��?�D�I�
�F����O8��I����x
���F�=�h�A,��+
q���b2Y/1��y��rz^aWp���7�h�]!+�u�H��5���K���[�@�	��9���`P��#�15��g�W`������b����� ��_��}Bcb�Y��&���`��^e'?�+�)��5�%P�����qA`,��3��D�e��8&FV�QF������D(�V�l���
��d��E#��+�b���l�,�i� ��.����c�d,�u���H�<Kb������S�]���$��#G1T(R�`�x�jK����@JEa����n8����^�m+l��d�M���g`3�B0�����)��-��n�o7�L��y��!�9X���`�NV�C7+������
v��`O���w�����d�d�^P���U�6�?��v����a����*���uh�^`�E�b�;�
7�8P�����\�'��>�GG��'��Y���s#�Yj�Pd��F�)�^�88��1u��r<.0���AS�e�tI�,�$�������:s�d��/��4Z���D������K4�$m<��D�o��9������u&�b�2�u������q��,f)�3��,��_�������.fo����y��q�^FC��#������ppLW���P�%�f��2�������F���������N��]�}��r�f��u@�����O����H��
\�G7	��t��^�6�C
u0�����(���������C<�L
Z$&�-�#�Y�I��w�v��3
��	�����,��@��_!��^nUV��~��ck< #��/��d@�tQV@�4_%����lW+�E�n�t�� ��{���,iG%��?���T����,�/K����9�D�:��d�R�
��4��!�W3-h/���v�'Y4���e(�B��p5��
�e���?t��������aF���
���zA�fp��f��yXCi�%Ke����xX
i�lXx�
��{T����������>2��<#
L�}����@w�
�$�7�*����������-r`�2��"?P%o  
�
��k� �Jn���X���_����6%P������I�� x��kH� ����������h9;>�
�����3����dt<�5,�p��(
����$��,��q�L�A'��Nh)�*��h�g!�@B�>�C>�4D:�L�Gtz=�vO���Y��$��Z��`����k4"�N������e�6pc�NYS������p��/p���N��;�(t�T�`/��YF�|}K`�`��3J�G�3���"(�!���\���wH�=�G����L[�.8b�CS%���hr�E���$�r;O���9���!����]�m|����~�8[���W�
��p(��w��|��&.��������q<O����z|ttt5~~<����^|��`m*�����������~���?(1����j���+�����?I�����?v����������J���g?�:R@������+����}9?ezJ3���P�Eg�����b�%�Bv:�0
�!p���>�2�������g���x9E{Y����Z����  �P�f�,aA�{�����<x����y@�=�
B����,���x�[(�`���P#!�:X2���b��������!E��c!���Il��|	J3�P���������{_��Q�x��'�l�_�p�f�<�����Y�����H��x����ON��$�����vV�|L�����2H�Y�f����X���!��1�F'uwbh��c�g�����~��|O��89����+��v����U�P(>������e����}�9|���_-F������
�~��l�������@�F<����`�n=��:t��y����/���|NQ]Q<r��Mh

�����������~��MC����g� y�����:(�w4�z�#J�G]�R����H�j�G��:y�_^h:�8�����;�'�'o����!�,�[��yB;6+90��4�=����a�H���+6���4T�i��M�zl*�T�1�*Kd�3��,b�:�'��P��ER���������s����G���}�|�_����U�d�]EO&A<;��������=x��P�_�3|m_|��������I������3�Y��A8^�����{`��a7(U���������J��	��������PFv��H�����k�~8�S@��2��7_��q�4���f���N�n=�M'O�=l��xq��	 ���h�<a<a)<!��s�)h�j'c�H���I�v��!"])����V��Ohj�B�%��������Oh��k�e�	L��,N���*J�|{����y�q�DH(�����8���i
4�/U[),�F��y��9@T��������HI3K�q�ES��.sS���i�fA>�B4�~�<��y��=�~a	f���y1K���+B�C���B���T�rSH��b6
�fA�@=:���?��U�.�f7��&'A����3��M�um�y@������� ����\�����l\]�������
��B|c���n�'�������
=�����<���X�3�9�9HE����-P.N�f$q�Y����h��7���<�iM$X���>���I���I�������p�J���8+6��00+�[��Yb^
��_=yruu�L������X��������4���#h�r�B^`0 ��!��O?-�\���V����������x'���P�
�;o(~���A��l����F�M��0����c��X�-����`����<@��>��l������z<~����W���p�a�;�ZM��*M�V���ACL��v#������=�e,�C��B�;C�U�M����`-���Y_MS���/'�D�o���
�1�=~��\�Ja�T�V�����5��&�p� �7��J�h"_eE������SB�Q�}�GY���}.�?��E�9*�_$Um�^)�����V�A+3���L�hGd"a�s�������G���$��`�/e��B��P�\�>������&�K�9���$�P�WmE��Ch����*�]����h�K�~��I��r��_�����}�A����7d�eF��|��1�+Z"��B�����HL4�����(�#,![�d�*���7�)�Qh�i1u4����N�W[���q������|��S���@��>�q�O�Z��c�O��,�[]�
��#k�����2���R�����
=Z!���@����VL��\nR9��e���
��<$�S����/��S�%�m=4knm���
��x].��fu�M��ZC�N�B ]Yh��z�A��!��h�!��qd�����>|������N�DC����?�����p����3Q���nD�?�������~f���n���0~���6;���N��{���� \�������m_�A��'���0&pWv���x�b�l���.�h_A��;�=
7�)��lir���x����]�
~���g��7�Y/Q�����|7���Q$����n�i�&�hG�=��x��9���!����;����3��t�q[��z6s�C~&4vxX�?���G����o��)�3���R���o������;?n`�y
�_��g����u#������2���Sq1�D��!�r��l�$H���A"� �9A�MD��� �p`�g�br��$�h�#K:���x[�����=)�����$])����_~>��'N�m�^7�������!�6������?�6�ST���`��,`�)

��f'R��S)�]���)<��{��P�:u��z�o��Rw��t�p; ���)�g�<x����L�?�P"�����>J_{w�l��t�N�2
.6���p�k��B�������M�c9BC�HI�����E4%m�f�Qza��\v�og-�kw�����`)���p�%rSZ$��h�GY��V���_H�Ua��{0���U�.��W���4����tw��^�A	P7E:��c{��nbZ��!��HW��D�?iJ��*��ERN����j5wKR�������6y��AY��>����)��B5�������8�?���j��-^��>����)��B�����tO��8�@>��S������`P�Tn<�O�&��2��k���<�3�!C7y
�MD:�80�+��'�Y� �e���R����##C�c��]]�*��i�%u����%;�������0T%K��:I�w�������E�U��G�q���f�G�X���l�z�������K������yrx9�M��QyI
��`Y��T�.L^��.I�*m��.\<�PR������k������h���lF�rdZ����u��Qh����v�����bM�z��sC>�CA��%K��:���oP�'��@
�>�SR�`dC������/V�+�m����D��Gs������H������(@�1���Z�
y����e6:���	��`�Q��u�u3��� l��)1J"�+�Z7�/��.&
y��
2�e
��6���:��&�y��L����+"U�
��mfI���Y�5��"#�1�:_�������
���N��,�@�Kl��Ek�~7���������x_E������_��V)��J�1��&W�J�ups����t�������KW�P��L>[�L����r;������lVif�A����#"���IO���
���P��3p�lq��R�2���H!'tbf��s
N>�����%_fI��:+R,�E���	�d��p"?
��t�ki��Th~k����R�m�M�D�z8
�)l1O){?�}S���%��R
�����s��
`N[�f
�[��"��9���iJ�b��*�������g������`�B)�������Y�)�K5�uDX��A��cL���/�$��a�z�r�;~F��P��P��(��K������O�'����8-��O�;.7I���-�yK�&^��y�;��'����yK-L�z���hw\�S d<�-
���:�8�`��
3�;q5���
�I�������1!s)�A0Y��$�$�+���o))��u�B)P'��n���O?���@�~e���KB�t��������v"�v���B!�R~
vH��Y2�f�6-��e���LI��]�(X�C�dZcj���jkam ����P���9m�)�sq�6�T��G��V��x�k{�A����O1������P�N��	eR�":{��<����yJ��l�(�k��?���0Z� �?
���ee��u	�DY�X�RY%]��� ����j���0
\U�j��:�a�~���"���B�����H|+��bk,�eMb+^��6��J�����WB5�([*����e[e����
)��<F#Q'��AC��?Cfa�����D��c��V���M�D�e������U/h��jdt8�O�t�e�����a$R���rbc��h#�h0�* �K����-V`N�L�u�A�b$�2�}���4'vHJ�*���]���7	h`�D6w
�-�/�E������8c��>�� ����k��K����m"t ��,����f����o M��VI/�����tMo���[��D�����y/�e����+D��SC�+L������4����r�3��.t)������(g
��EE�]�WE�Lyq���J1r�����6�\0�����JXD�S�I��IB�$"@�WESVy����J1������8�\0�������T��L��d������+>g�_Ea�PY;A!a�u
�`)A�Mj��\������DM4����R�C��#Xi��=��?AF��y!���A�!���n�2�+HR��9�*�
�D�*����0Fi9�:\��E-"�5�-�}mo�$��9�C#���XV�r�T��`!%j���j��+,�F|5���V*UX[������\0��c&Z V�����/����2Z�6�m
q�LL�O?�������g-*R8�?��A9���3�4y����\"�SA�?��EWUil�O��?(W�f�F�2h��I��a>K"�U�J��f:4fnN0��S��`y\����
�a]�������!�+��������F���3�����`��3�U����2Z�Q[����2����`����U��l�2�X�0�Nd�_������^Fu$�����b2�&�����6+���c��?�d��)�o����K�2.	D2 �
�G���������}��`�K��)��wi��l;�e��������@*���r��!��K{�9O!���uG�(u��!\�v]%u������<N�}��S�)�U
������F�������!�g���\�O�����5�5����K������	���4��V�q������g����#%[GZ��t�8[����f{�@�M�u3VslmE�,�x��^/hoL�F�:>Le���Q��SU�/$��s������q�^�=W%�
��qL������k��e��R�����BQ�8��3'����e��4���}�B���)��TFQ�j�"��_lN�~^.�w:�T��a�]t��e|��v
]f�
��b;���nPK)�h�	�Re]Kei���Q�qw7h���_%#��p�}j��H�n���
Sz�������������oX�}6����=�A�P�������P{J��.�����Xz������V�*�#�J�O����mD�c������D>�1	������V�lB�kLx�W�f/��M���G4�����L�r��t��nD�+���o���b�h�3�1e��:d *��xpn��jd�bm=�4���.��OW>�����V����K��a�k�(�=3�5�����XmZX��q3����5$6��(�h��2�,�l������m��IP=��2H�0Vc�0:Y��}���~��8o�`l[��C�������F��Hz�9����k�v����������=Jd2	Q#"jj���n�(k��K�.�h�Ca*��k+6����id�JO!��L���b=f*q�%MKf2�zk����Iga3c����*�T���3)��z�Tl=4"-��\��!Z*���S�,7���������}�L��LY��R�+���+n5H�P�*<��x�FI�����j�U���R[���a]:�R?6b�-���"��M��p�S�������+�Q^�55�ZjO#�l@|mx"������c=��T�v�
�V�P��VCN-�W�W���HyAk��hjpa��I����'T�K=��~�������9}���e��<�|B\�)�
/�����Bwx�-���w�$�������5?!�z\�)��S4n'#�sP���@{����4
E�Y;���n�=W�V}�����z`B7���q[��
���[����_�V����@���R�h>P�z���TXe�T�[��8�������[6��6K�P�y0��k!�Z�or���$o����W>/�����L�me5�����-Y�3�brR�>���fz��o���F��T8�����;`��M���t�-�yJ��ll���uZ���@��#s�%s��9T�9�"s���-�-���M�H s��9��9��)�Y�Yg�l����P��%��wv{�uJ~�/�~��e�)�L	����Tg�D����g�0����
��4����C�^�xnH������_`�S\/��l��8]b����J����Rt�+/��}���3�������l��� S>�z��n�F�����i�eQM�}����
����vEiT��i���}F��?�$R�'�����x��>��������v�hU�@Xs�B����,T
��]�U�l�j�B&j��\���a�y����<L���V��m��N)t���I���'Xuz*�=��m;E� j��������v���;����D�
~p�&�a�0*���U�|�_n�������B'�'c�d��	w����1��y�����A"��1C�0Oag�$�����d��;b�cm�V�lJ�@�2P������Dl)b"���3��A����x�y�=(fh�	2��S�kx�fP��XZU�D6;��B��	u�Wz��q�>A������O��1r�M�)�������+��f~
�n��8����P_v'�=q��������	���O�.����>!Oz�;���T��?'$j68V7r<T��k���`�p!�����Jttm���6�}5���A�m;�����.��C5M����6LF���*��m�?�m�������m�`�0&R�i��X��q��
#j>y�m�Y��
��j�5�������`����M
�P�'����>����m��������~����KJ�����d����w
�C�d�p=�4�[.�$� ��[}���$7�	���:��/!���:[�V�c4�����rz�d���x�&|q5�Y�a��e�$���!�GQ)0��@j�z�Tx;m]����9\�8�0�[�	V'����<��Z�����I��Z`��������y�0t����l�+���?�>��;�%_�Yr��w��y3���%^BC�::O�6S��q^Y���%U ��%6�K�{T�%��\�w����p�-*���f�m���A�P4��eh�BK�H;p���9

��v#(�g��1�/�}P��ZWZNM��r*��S!TOeK��B��
a��j����������
�SA�z���V�
�v*��+�
!3�<�V�
���vN����u����,�en	��������)��BD�**��*y��j����"�������N��+���-W�Kf��v�cd;/q�Wr0���n#���D�,��S���;:B��7�HJ}Y��=8A�_X�6)��j�#:������H�&5���V�in�#��rm���U����V�|V�m��EQ����<F���/yd�M����9�;������\�J�=J��a

:<!_��*?��4��2�����N'�	��]���\�6���Pl��$��~5�n��hN~��h2�TU��~��H�}�-�(����B"�07�(T] ���������D
Ee�;�p�;t��~��������p�Z��� ����v�������A�|~�l[+=�	����m�3�D����!��lx������3(�L�4<*��Zz�;�����
*����K���La�e%��������N�D�%,�^�iW&r�"`�a3T�J��TE*���2��.5B�&�Q*F#���RPD�F���G$�ZH�$;��F�)����f�fS"c^����E�/$c�Yr�D�w��;�g��s
��k�9�d��,<+<r�M����3�q���;�:Q���v�L�E�L��J���*1�N����x!�`W��A;�7��2�y�\����N�q!��Ej�J��:���n��=6H�P$n�$n�#���)kp��54E9#n$rn���H�����Z��+��x�8C#� s����]���1��1����iN�����0h��~��I�x�Xo3}{ryvq�v���~��	�"~���	��/���\g���Y�+��]�z�'�:y���$ZzD���5Bur?HSZ�������N�Y�VZ�*�nH<���*��c�Jk7����p����kX�h�(�Qh1���f��4�>�+�(�b���d�w1�u��b�x�z~C��U��j�e��e�0��5_,W�$�m�3jyo�>�qFH9�[!*����2�����n8��B�/�3B�3�&9c[�emO���n3\X@T� nE��a�Q92�Pq�@��ac���1`c4�������)�W�W��������z�_u���=�9�_�/��pQ��U��b��i���}	���|aLN�d�p�������Z���G�Wi{�� �����}��xDbQO�=Y ���<M�X�1������s��2��}m�P���`I�tz��\z����<�T�T�����P�P)rvU��tI&k�Xv+�'Z��A�M��M@����,�P!����t�R�����~IP�I�&����]R�&�u������h�H"LH{OR�d������Q+��a�'��o����d�
t��se�����iQ�t��]�R�=/��K�K��K_����(n�}�`�����0D�a���a`D�HMvM�}���
������VM��0���1��r�CXP:`���j!!i�����z��-E=&_Z��<B����cyD�6�p�p��1��ni��*'A|w7�&W��k���.��M�W���d��XDT�\�vX	E�b�u�b���"��U?$U U?����H�eR���d4�.8cR�xv�+���Z�+��%��W�����K����t1C]�o�P�Q3�����VeFC�<�J��=��ZH��{e�
O�J�?;�"f��p�F��s`�����[V96�8C��.�7��E�������y�#���k�������[Z{	S���S��83��zUG���]������_%���u
��L���KJ��7������E��l����V���\���J�s92����z�\��W ��y��L����vts�����en%�O8�i�B`����e����,��Y�f[89w���<0O����lc����lZ���w�^.���d�/����M���`{+[{7��z�,�gh�|�co<�e���9U���as���I��9�U�����B�����?��sa].4r)�������4�|�7����)�B�\��9#�fl�1��:g�pE`,wp#%<�p��fc�������	t����1�k���r>��
*�������m�H����W �:�dQ�8���}�$��{I���o�*�R
��4��c9�m����~h`0��,;N���X�0x4�����@-��
j:��f�XL:���VL���>��;1P�;��R��T�{)����%�0�*O����`�DC����fz����]���+������b�>�������Y����]@ 8����E���^�����Ev7�i
J�"+���e6^��(�����������;=<<8������W���:�������������{�v���@��j��3��r5,G�|t�(AP���z�]j��3�����*�Q�eVd����<�*�|�X�E	���g�_L�P�/V�R7<�`e���Z]�Fj�/'�|6/��?�F���e8
��8QZble�.S-vM��+�������j:�q���n�T��M^��g�l�Z��������g���1���|=^��j��Q��O6 ���HM�����}��d�+�U�J�*���t����.s�f�e�E��Z�J]$Q\���5;�i��
���:>���A����������6R���b���O����?ng������_�7z&�����"[���2]Zf������W����<Z�3���y�'v2gf�h������t���8{so����&j�����;8<��d�U�e�&>z���G}�M`��SsX�����w�$��zs���b	Y��y	�=8�_����1@u�Z�45� ��}��9U`T�_';E
D�V8��a�'o�%-X��������z��5��|�}|���O���0eg�\����%�\R_n �
��D�!�T�a��� �f�<aA��~�"�D�o�I���o2��\'�7�^9��T_�wQ9��cF!B���,kM�%���\P��ji��m��rVd{Z����Fk/h��c7����t�����n���}KI���]`������^�k��"�E}�����^���u���M!���9Hn����������/	�8I_g��"@��.f 
A*-�	-�g��_��}5�|���d�U�N����>���D�K~�����G������w��H*�h=����}��n�X��|��z)���b6A&������z�������z�W����r�n��|��� Sl=������|�G�����6��&c�����8r��� k����'�V=���N�-k�^��|�����x��(���bW�5��\M�s�tU�����O�������������<�V�X���������m/<XK�.X��"��A�������#w�������'J�K-uM��[4nT
�L�=��DT�5�P�j4K��e��Vwbgg
���g"��g�������k�Y������<�u�]5�������Y�rt���;�_������m�RG�Zc�:�
D2�������{�XS5<�I�
�8�����6�a��_�����M��QPpZG-I�Z���%�SK���&�(�D�Zg6�	���fo%�3����H���H/S�������j����e~��]��l��G��s�]d��G�j����^�Z�CD���xn�~=����nk���K���M&�N����������8�/���������{�f��m��W0����{��wZna�D|���e��.�+�0d��p�*~���f~n@�g�}��;���}[V����E�3k�"�K��U�����;�>y��r������;����G�U���I�� �\�A�.��
�Z����?�^Nu���������Gi�Cg�����y<7����6�X>���8���&���6�I��6�XI�<h%]Uk[��mk�m;�d�U�V�I��i��c�']Ugo�����Y�&5��
�JX�t���(�T���*�/U���K���*�j��Jn�T1c���=��cB*�X�t]�%Q-��?ElU	�T�s�8�{L���}�������z1Z�����,b ������(Z�n�k���Z�n�k�u�6�F!-|�~�vXQ�|�o�d�|�C�o�d(}M��-YZ_��FK�����\���w��[q����{	���X.���fd}���LS���E�������j��t�F|C�mo�UC_��;�����g���(�,]��I�]M���t��8����6��qT�QQT6�4�?DR�QY�F������=�?������y�'���_�+���g��J��|1{�M�|�`j���;�V��S9�cWn�f�������������"{���Ni.n��e�����'E\G���!
����K$`�4D>�n8	�5�L6��-&�l�m1��Gf��BDVL��K�4N�Nj�?hj�f"��v�����������-B���k�n�kDt3$�o�$��-�l3$��!��fHj7C}���!�o��,��Q���n��&�!��u�5n�Dl��=��6C��STo�v1��if2�lZ\�/��\�N��9�Mn�IE�hs�cn�c���D��:��+"�����.���N����4]��
�z+7�q/�m�����$b�G�K�����������H�?f~���'#��Ir~L�;$����I
�19�� 9���pXz�?P���+���/���*e��,��5��%}��>�Ou\����N{���_>������l<�@�gs@;d�=�%�����t�7������l�	 ����\�-g{�h���{�l2�e[�~������|�/��_��_���a2H���������/j�}L��\����������t�~�����|��N���dt|:<<:�O���Q�t8�ONN��<������T}����l���j�>�_%z	;��}���H}�jRP��9�
���i��-�������i�Q�K-g�W//W]u���*tc��������$Q�����������hy_�����_|����W��S!}*A�6�x	���>����w�����wv��|eK������h2�������c��j�.WS`H}/�c����;��F���:��/!��x��YW]���/�7���jY�#�(�,������	�]���RH��n��)���M��9���0�,�U�g
zO�u:>q�:�"q���l:�7J�i1�p���9��R��H�F�9$T���k�F�\��:8�
�Vd����zA�gEF�v:�X�M��+�a�O����Q�����?��Ihj�E)o*�a���n�P��������#�
��w�.�h����L@#��US�.�s���qw���g��?Cy����GZ�����p������n�R���Y���U��s�?�+�����:{�����iW�~���������}[�}�R�:L���m���mi��..�gf����e������)�)��t��m~5�X<����]xX���!�h���x������i������kf���x9�k�fp����6����(�~�U�b�����)�u���N�s�V��o��}����L��i�����y�-���Y���8�~���)���J���$2<���nv�?��������l��g��g���,��6���2f�4���Z~��i)�?	��|�Y�c�O,�[�
u5[�P�C�S`p�r���sP���Q)Z���7>���=Zr�����E��M�=2��D:�
�`��������M��=�b��6l�
R����v����/��Z���2�O�?��.��v�X�f�����'gO����y����������/�|��;4q�0!�t��[��C$�A���'�MZ���D�����+(�U,Nm"���<?�"0y�[~N.�\cK\���2n<���b4�;��y�YB�fz��]\��?8�4NI�`@�_����(1��\�d^�v��shxT������@`���}I����;��fUPS>1��Hd�-W�i�T&����
�����p�_�h\�P�����.�*a_�o ��5:<�5:J����F��.�fPb�_�f+�gi�@�T� ���wo.��#�n*Z*��Z���O�O�1Ou�1�t���uC����W��A
:�v��*�;��=�lw���XXB�~�T�����Z�tz�A��d{���)�t4JoT^�9�U\���#���)�~k=It ���E��fo�-�����S��mc�"-��M�w�]�S�*`����v7�L����j��{�Z�&�ov�[F�	�8��)w���$�~7l2�z��
~�U���P����)Zm{�t�����?�"�C������	1�����bW�p59��E�3���}�����}���OH���E���T���O����s[B�����v#��Go��A�g�M�JyzN>ep����!�h��i�r���iO+��MU�A��&nS���w���p��������iG�Z!��#����y������{��_��^�|�����������B����7UtW��������U������Z��<��w��������2�m�c���t<I��S���Y\d�dK�h>}�5��SU#�c������V�jB��ns�b����J�����������fV�5���8^�N�c�;�.�j�[[Q]��D&?,R$���H-#�?=����G�g�������o����	�)D<�d�������12�������bl
��J�9��QG<��)�x0������]X1�g����b�v�e��u.h�mfR���r��&��F�N�tW�]G`+�o�1f�_>�-���D��d������]�"D��i�a�������;�po����?�,��p��/�����?�?���?����i���F�����|x~�f�G������������?~G���
t��.�������?�{�����|
%����������w|�c#��:�'������&[ p�r�W��'Z����[A��Zf��H�o�x�(#�M�~NG�K�L'�ap�zy���b��PF���I������w��k.��K0�
��*�
���c_\J�"�����|���	��V���s>���.���C�fN�F�^�=KU��,�Kwn;��������E|P�?�m��S�N��\�3��������r��>�����ow���.�H��]����\�.2�2f.��)���jO����s���uy 9��x����s�������M��0>�Ps��j��p�B�Z�tv�S��7%*����M�]���#�5����;^�2���l<�\�q$�����i>�nvb��t���J������>!`�]$�5���P�a,{w;@MS����O�c�.�����tu�w���L_g��g&�S��&����Lf���;�������'R{93C��eMh�N�MhO���`��!���,&�k5�L������9p������d���J8{Os_�%�j�t>�5]v*G�z�$�bD^��e���7�~������ �����l��'����nqyy��RD�V+��*��yq���1Di�
���OG�j�Ez|i{�������JG�N+�����1��K���t����]=v�gp"�hosy��.������3�	�S�}aTp���a�����'���.�|i�=Y�iT}�_Nq�-+%�R"*A����s�\K�K��}w�}������=�V�d��M�Mf�&s�yAb��L����~�'1N�]�-��N���
����_�
��RWy1�������9��#��_�j�.�=d�gm%8��aV��WEAU|�!�yg��{�:����
��ss�d���y��U����X�|d��7�G�����5�
P���6�����a���h>��3^QIm���MZN�g�C�������
�(��l��*����������<��D6^�3	4�u�b�>v`;�eV���{p��!�d��O��[��:�q1CL�ngW�kW��[�s2v�W�K	���ja���1G3�U���~����KqB�����h
�+���A�D��S�j�B�b��K!��P���9`��OY�k��+x�LhY����;d������Wu�\�yB�e�4��%m�g�������e�/l�B'17V�p���~K����J�U����hEU��j�CR/�8\�{�]�2�TU�8����>e�SZKk*��]iS�������on�	������eo�R�u��V��qt�	�;JT2���d�=\t7�9p �1�a���-���<*�P}����=���f��pW\��u�q��7-��*��s�9o*�[C!�
�E4q���Z�Q�g� (g�D�v�uGgA���zT0�zS���M�����m�~�Um[bE�Tw�(K��dG����'O17����t{���.�X���F��t�H��	�i��x��]{�5���`Fs�1L���������q��8�]�"Y���<���0t�g�',k{�M�Q���4Nf��Q�����qL	���3=��GS��=����a!�5wP��O�I�\P"���\��"�����������W6��^���<�6�]�R����38�(�q��?a����>�5�:��E���z���}Ddie,��dv���ux��<��1��IU���
��T�27���N��Nm3��4���Ih�-y+��&�b��nyA�tI�������n�O�z�HE���0��8���z�S|��� ,gc"h�_�>4r��I����0��'�)+Wd�{*Lp]nX�_KG�-H���3�<,hg�Z�3I~>�v�7Im�4��_��]7�o�1���t�G
@������E�4��!_+;�F0�������XC���fQ�`�b�S���D�7_i�����k�p��uug,���6d��I��9��j'F������v���;���9^��u49TH�I]3O���n�
]����6Vf�M�m
�����Z��sl%"=�J�K����w�'��@�����A�y��q���5P��g������0{	��1VDm�����G� CR�x�R����)O\.�������&���tM��[D)j��DG�z���g��ct��l:�L��ee�bfo6���?6�o�6����_���S��)>��C]��9v	��?oN�z�*(��u�o)����T�������)���B�V����- {�o����k�;r���[^o���5q}�o-�tv���uf�p
�
��IP�����jWV�����������S��q0�j�����2.c�,��c��.��6).�*�_c���*n�&7`�Sk�V�����CpO�������#b}���j��
���kz�Hg���am�l��#dz�(5��`WL�i)j�J�]��E��904mkj�U�����FW��KB�_���$'V/�	6�Ne�Ja��-���1js��P��Xx�@$�3��x�����O���[���~xo
���7|b�o��G5Ti�bS�O�������/�|�����N*�=��;N�i�'��������=-���j\�jO[���|/3^��:�U������0����<����v���o��@���I�'
����K����J�uVi!y���hp(G,�79D��4
�Zn����`��,j��9,�u��Qs�_��Kk��%����'��)�f�O�O����*z@������#}���9��g�������=�E���n��"�egt#�Ko�"E�5�7W �����.q�ma��������z����^��JyC�+����}m���[��R�i��|��b#Lk����X��-R����^f�QVE\8�A�V+��0 ����2Zp<:�����0s>=�N���SHjh���!���
h1�Y����<��w���5��DJ��.�K��F��8�g�l���D��Sd���[��C��+�E��T�#�(��h�.��~�v������eF�������(+�����ku9[�`�~�u�s��/�F�BA������/)�;-�H+1�`)	�_R��2c��%��@J&tb��)���v�fi�>Yd\���%/l�%+5���� ](�f�uD)�(+3bwt'���j���mp�M�ylL��6�������nF���`�=�%1�(�?^�X�d3S�����G��w��yDm�3y�C4
*=I�*O'�Sn&��+�5�.��_���v�=r�g��H�!��J`��v.�$���Y�X6�������1($�'�
�U}���b�F ��Y��	9����'�0��^w����K@�>��#�����R�
�HX9� +AR�c��-�Z�i{���#�����RJ�}K~p���;� ��]�J�s�pW����B7���x]M��D�6
�v8�l
���}&{G�(�vn~��8��y1���M|&����=�q3�%�Cn��:�M����>u�<��`��~�9
5�~K�A��	B]�1����k����{s}��<�0���+�$��3�Jd���M�����4VD[�wO������Q�i���[?[�����Z�������3�(�W?����du�r�)�?Yg�&�Su,�k���J��@�1�g�E�
+����������&b�Y���/`�8"f��������L��w��!q�^����	XL���h)�8�-5�Rw��6�6�u�Z�hM�P��m�R������Tu��������;{cm]���w���r���-�A���D7A�����|"����0�?
�r��brB_�Fu�*-�u�U8�EN����d~�p�\�k��pR��]�H�z�M3���U�4��
��(���>��E�8W9 ����� ���8�gd��K���=K�fs�9��+o�o�_��U��2�mM���(������R$.�[�JS�+���dz�U}�!����
�eJnE�=Tn����>9U�fP��;R�p�V��q��y
��K��_O5e]�{�C�&�fm�p���X���r���Kl#QP���d���pGa%rr' {�f�z��q7������v�V�����w��@e�������^S��^����v�Mlw�����wGc���t����t9"f�|��hn���x,�eY����8/?]U<5?ffoW)>��f�4R�<��!�<�;�jm�f��w�4y���<��g���#�}�W�fB,�4{)cC��I��;�J��P2�AM�;���J{�@������Y���jhM-�����]3X_�	��@De�~��G�B���;rv:-��������X��O����l����
�e���\Z���'�����L�
�y!��9��s�q�-�w�|q����i�J�!'z�0b�	�^|M�8��$�{����!��>Z�t���d,$�@�
����M1����O����Hw�i3e��Dsa�����^'�81��t����0W�����}��������><^S2�m����L)��!.�����-�
}X#���[��F������d������d1y����St&�jY���U;���F]������=d.G#��G[��D�����x�����Wu���>x#�}^�}o��{�-?�@��>cBr#fZF���tC�+�+��U��9N�X���pW`�a�;}����%�`UN�n��t*��~�r���(��w���\5~��J�M������-����"�8�.� Z-h�:k+�������W"�OW-�i\��$����3g����(7J3�	����@���3�K��g�,�0�n.�$�V����B.�V�yV�3����+�B(��rB�^��d&�(�Iq'��a�W�:�������I��R�e)��`�.U������,4�rqsS��|�f�=�:�x����q��e�_h��*�@X�z	���V����t|��#�����}���(�+�)�I_�X��l������)�c�"�b��@����%F����tm�-�5�Eb$�7F�sn(�#z����r�-�W\br$���_,R�Q�&di�FI��De@��H�����KYq$#��l���E�wm��9$`�RM6��x�xo6@m������4�>_d��G|��F�9��N���1���XA��6�W����l2[\�f���4�D�Qt����`����*�PF8�e&�5�R��F0�KTA:��z�$P�k�\p{���t�q�l-TB��,]��_B+&� s63n�0Q��j,|2�h��A�Y�A_�9-L��-��V�j��=�u������ �����j�&h�uaS���l�+���	����F���B[�m�Mp��y������H4�����^��X�(��P����kQqR�����,��u���-@C���q�sl�m����]7�L�G�3�
;+AX�|��\���U�<����W���A�a���o<�ko��*�R�[����M���m\�U���Y��3�t�d�ea��E�Y�!:�:DgGt.�g1D�����Et.BD'��~�cq�rt����Q����) �!V)q������]��+aL-|��Q<]����(�5����e�K�Q�<�sU���o���1��wTV
��
���R(����*D�D�����q~�8��)���������#W����L��f�XQxrY��<��V��^TNU��|���eI��[!U���}�)�f������������=0�/����~h$���$5���7\-�,��K �nx1�3��"��M��!�neg\4�QT��\����{���-[X�"o��z,�g�k�O���iB��%r�^-JN�
m�I]��$�CgnD�>��vi�	����@�6���x�a��DF�>��N��M��U�w&+����9�/��d7`s���
�-�c����H��&\�6a��3$s����LP������"5�fHF��QR]�@����t��g����dF�e(���/"�%4hCP�Y5�wL<S�h^M��54A�aB�_0E����2�G X�k���2eU3f�|1��.��fX����>"E"	/f�VrFH%����y�<�>���Ln'?�j4j6100��"������<��cAN�}Y��L-��=�������z��������O*��a=���f��FQ�����N����~+�3�m��h��D�mi��H+]���l����j{vm�*�F7[�E|Y�v%��'���C����;��Z��!�����g)�0|?R����'�
rX��I�p
�)%��r��q9E=��{�R��������	g�FkJB����Z�-&��W��&����7S��E�_bL��5L�����7;�:UM"6�^����r+t��lN[��D���8�}�v�b�6������I.�-J	���.�"l����g���+��y�5W��x�a�;�B��NU.��?8�b�|2�2��i)�����35>�4n���H)?u�8���
?(�F�E��)�L�� �\�������P��N��P��TK-j���R"9�\��$N��Q��)a�50��G�{}m\���d(i�f6�XT�����j��T��%]�@��������~����(vn�Y�I�g��A�5�d�6$�Ia�Bex[��E�JX*D����h��w�t_k�Z,����L�0H,�>�]k�:�b���j��
�;a>)zQ�G��Am"��q�����R������l�h�(y���h�
����3G,�q�364��eV��������+����@���W����|���-�nOsp�CA��=�!�s����J%��R.�JU~�����[����}�r4�FA�8��$��Z3�����!���h���p+/1Fc�J�Ef���
g�������x��Q=T��U��$~l��3������P�x��N��h������M�w=4���3�F:�`\1I��b2b�$��#�-��a�o�t�D��qEDm?PA�v�Oz�������T�p�PFI��/���sI�V����A@T'���-=�E�����]��gu����ct$+5��/U�z�~��H#�6`/n��m�<�����w�V��%]
3mL�tjp��b���J�ak)�rj�f�R�)o{��p&,�i��������j:�24�[0^�.�)]���W:���"ey���KPw����#�[���[���S67)H���-m2[5�dR����%���\y�����>f��T�������._!FO
������"��b6^�2�|��@tjEZ�C.�v������w����Oo.|�{��I�S��� r�.5�������I����2��p�{*�cc�,�g�n��Q]������
��@���)F7^��-

^�����2�3h������;�f���o�"��px�c�w�^Tj��������w�xt4L���/o��1�,�j�*J��W�i��:��@�u���������Z|�2�0k���`�dS}��]@h�X������}�Q4�`X�39��q^�?��Qf��+feg�$��YS���}�Y���M��
W6���t��]@@���D���������j2��pX
#�#������+'���w�QKt#�d(I��x�Z� ��%c�V:���3R��S��~��rx	��z�$����s������a2c@�����E
��3���2����l��#%<��s(_��d�2�o��t�����p���{5��K��O�K�,��������['�tMA��0���l�$�-�>:��,��_2�����$����&Z/
�3_�q�
������Rh�e��=�xUMyE�����4fz��l���)Ka&�������` CA���I�Bzl}�������uUt{A������W�=�����b2jY,�Y4�Y�q��18%�}�&�\���L~%�~�����[~[��6j��0�@-��!�����/-���	8��/	���p��6����	���.;�5Ta�mT����x����|[�g�l�8�c�QE8��:��6;5k^(�d�������=�d�B��9�TT�Vi#�|��L���$T�r�9C�t5�*��Q�:����##$wp�Nza}�d��
|����cv�:�P(���2]��������EJM��v4�B[����)�6�%��ED��sx�i��U����;��%�-K~���i������bdQ�TTwd�������������	*yqh~v�:%�x���U�Zt�K{d��������]9���%�mNjWY~�xt�E����E�n�OqP�MT����w���S��)��^��������%�W�k1
`�U{{,�W.F�������{��R�V��������vZN/{#5lW�C}9>=8:'��� �������Agoo��;��������ib��tO�.��(�l����gPi[�6��b6[v��Q?��~�I��("����U/�y��Xyl�}��������.�z�n�iF���@Q;�Kg�\�g��rG���XV�bt��*��S��5�"t���bF
����(����#]J�NB��U�S��cz�0|�y�E��
�{`87�������g�t1���CUm���^�ag�l�7��|Z��������{x�N�� �
�{����]�x�,0GY���xI���u�k����c�u�v	 l��qh��PKQD���RM�|q�[���LM���KDE��!���<5�>Nm����QX��Q?��Gm�1k9����3�cl����t�;�����U�*'��i�f�[��.�Pd�u��Wh	�ml�(n�=�/&�=��y�z0�m�"������j}���\�_����E���*]�����$�pt�����:�Y����9�E���S7^)gT�T��DK���~�T�6f�B�M"+�q{�"Y����g�qZ����	C��L2kgN�ZYS��"8-�ZU:�P�8��'����AW}��e�-� ��c��I���'/m�ff(N���5��/g/�`���U����z�
_%�tJ����R�����3�����GFlw��H�����lj.���������!6_h��Y��wm������
5D6�HriS#qQ����E���(������0���jH����F��o��m���!#��"I&r�����;A���d�|�DLM��H����}9����R�q�&���
���O��9��j����I��3��3���5H��D�dl=���a�#�����������+O��%��~����z����J������f�;&�l>�����[[������%�zaj:��Wg�4���i�H����n��(w��T�_�N��z�cY3��~��'�q���+�\l��Qt�-���dM���������	��Q��lY24�d�ec��C���n������\������W)�+%���T/G��xq}Z���n��i�U�)�x�}�S�\P���
/��)M|m"��pK��$gO}=��f��-���B#�A��Z:���~�-������E���u�������h��D�qqc���]h�~�07�3�[&M���-i�=�i5���h`]�6/�*z�u��s/��<3D�Q��y�F���,���?�,I����T��|����l�*�ywh�3<���2�&���e�?t����������yj]����n
Xf��]KU��
U��Q�M�I�_��M
�v������R�����>���;�_���'��(�{��q��4���)�����AQ\���Y�{
�N�6CCn�`z=����C������"�~�/�!��r ��U�lt9���B��8/!��X
����w�g�p���N::<�&G�<�0��O�9�������|s�����?���6� ����o��s!*��P��q���f�M7f��?5�d��1�75f<`|=��DP���j���ttZ!��7aQS$������m�c��w�����d�Zf���VK�%RjJ����M	j�:	8&s��	k�F��n���q�B�U�)��,C�g���)�`N��?��C��?����u�r�.<Lh�?���J��w0� �H'�$J�+��[�'��"���!��Sg�k��,�N�95X
�Q���a85�R���'��g���_A	�3�X��zK]�����J��
c�l����D��qm�Xx��93����&+���/������g�1�;�MA��"��P[��$q@1��g��������f�AjO��xM?b�q�RZ)U��v�&�����CST`EY!D�g���N�E^b�q�����C���bt{��N.����%����n�G�15�u���a����U�`]&�[���_�5�	��"�����d�7�� V@�>�t4q��On]�*m��A�}����Qmf���g��p��^���C~A2�������)(X�����
��_EZ-V���(]��������%��
%X�O��������?<J�O��|SN�o*�zz
Kzj�P��zM��X��_nI��r�\�������9�{w�c�R=y��9��/��+�������g���|	y	����-]�R�3�=���5g��L��]V	��Ka*�j�/]:�=���;4�*%]�J���[�x���?�������w��}������{����a���?����l<G�t6�K�����{O�{�xL������d�p4����
��=M�{=l��
�������?>�����C��?�&�D?>���`����/j�}L����RI���rVF�O?���A����x<$�(=?�������� %���a6N��������l���������hF+@<G��T�����Yo�
�w,��{��=�>�h���W/Wz�&}��^�D��f�����`�8���X
������O�?~���jp���RO���D�w���H���9�;�O���);��^�T�V���sB$Q�(��"�JrC���EP9hY
�\��c���n�<�^�F�W����!~����k�w)G
�'~���Z2'��4r��ZR��
��Y�V�qWpdd���NW�qVd���N�5Q����A��FG�� ==N��q����8DQe��4!�~IPF�#���]8�=�G��W[k~9,�L�tW�z����M�9���y�����pw|�:�o���>��?��KO�{���o���%='����_�}�������F:��K��5?��'�����eO]�De�C1���������Q���^�������'�^B�h��&�[O���������vjX��k�.�_1�I�6�$l4|���"������w��tO�����d�N!M'K�^�_^����9��n������pk��W��������~�������jdY~Q�XG��~W3,�VFx�S��.�����s��7GFaX��Pnk�s�q�)�\*�#7H ����a��m�Y��vsG���Hq��!��(��@@y�yRE(s��H.G��l��� "
�����^v�����Z�=s��y�`m{+�n$���q��A������s_@��D�|$nTH�^��G�C���r��#����{1���qH`���]�l~�M��������E�}�g���x�m����
C��KM��p]	c�HF���^/9�����6�XU�F����������=&�'?�z�!�%�'?�_�9�-�E���@������c�Lw�������^���=2��p���\S��:7eJ�����
(���V��VL
�<����A!��'{?>�~��t�����>�o_���{��F��>v�V�?8�������G��}���<8NNO���t4:�d������0;?���N���;����T=^]���G�pP��'�����3�
����~W�g�����EDB���u:O�E�k�O�eN�[c]�cd��-_���w�}QB�O^���V�,Qz>&pz��8Z�b�����cA���yh�X��iNS��0�Z60�����
�Oa�\���h+�Y�9�`J���Uw�(�C/����a@W�b���c��������U��
S��Y&�����L�c�����>����/]H���A@�H431�6��~K! �E�l=�����g��p���"�]:.�KW8��|u��bM��b�A-��tR�I��,��J��&(��������_�2X?� �������$=g)	����Y�^�6���'�q�L�4U�~x��9G�!T��K?��2}
8GL�Ay$��D/{'_�2�������0&.�����bt/[p���	q�=�}��Tb"���B��.6BY��~���O��.�'|e�&�P�#�� �|�L/�u��%�c!b��:�F���O����d��&3�,���n��4@b���VhrU
����=:m����ih���Ec�4\W�5��A����Z������z�B���E!V������w'�L�@��$t8��&����E;�V�eW������k�:���R#^E[���3jnA:��-��ws@�^QBE�C5�p�nmm�`��7w���w��J7p���T��LL�&���&3s����� ��,��\�Xx����~E�'��K�I� ��h��x���Jp�C��n)����_����	��u����Gl��w*��:�p��]l^���TLd���>Z�������Z-�*����J����d����Y��BGD01Di���p�)�-"���"������F�
�b�{(��jcq��]b����]0Mc��f�pKYI"A288�B28������>�O��t8����-t9�(��"]x�x�Xj�DE���n����Qe�H���"c)��m
�V�^������~�h��T�?��J�q�\��d����dL=\�5��Yy��z�xL�B')��f\]i��&g�:���W1��8����[g������^�'v��!�o�0�|�:���-1�\.&(�w���T2���K"u������b��9:��>k�yKD��}�����������]�]���*�&vH��O3�.��6����h�D&��Z�QF�[�}�~
sQl�M�j=��4���������������w���)��b������s�i���2�g��_��%V$N�h���3�E97�����I[`3��C�&CY��]{�Q��(�`S<�4L}�[���
f4��Xj���G�X�aN�N�.~��F���<�E=�K��{S&s�{���l��*8v���m�"�C|��3�s��;()�����e����;p���_�UQ@D'�����}������l�����"�t�
�e$�y����'�k�LX�|��v[�[��>"2�4�2���v9]���� ��g{�3�QU!/Q�B����u�&vBwj���o��5��f[�V�6M ��� ��]��)n&����G�?5���:3������g�0����A7�������OJ'z���M;g��qE� ����g�6+�k�H��z	_L���j��$����]�dg��mW�:"�U���t�sz�j�0E�)"���m5�������X�d�ik���( w���F0S1��}��`������_����P�j���3��~1��M�Pq���axc��������@S�w�������(P��f�h����u�_\��X��7��5L�k�AQ��BV"�sp��:H*�������r��48��0�v����Z���C����\f/A�t-UVDm�����G� CR�x�R���;6��$_��*�2� �!(}�5��R1�Z���te�1���Y����d:����`��.f�fs[��cC�f������/1Ar�2��@>�E�X�����b]x���X��UP���(�R�#_%�h��=@
1y�$��x'����aTy�d���{�`a
#|G���w~��m0�2�D
�G���Kg���[g��L��d���jWuj��ei���F�OY���|���x�e��<����]���#e4
]t�6�#T�Mn�8��"��Z[�-����Q���G��M��0830c�l����l.�����K���%v�S��k�J��I��K������������FW��KB�_���$'V/�	6�Ne�Ja��-���1�X>)�?��H��HnE�O���{S�7���|����F
EZ5������UI���uk��T�{l�wD^��y��M���MO��bh��u����*�T�x����Pg�J�0�^n.���3���4�������>��i��O�1mq&��Xi���*-$o��
��E�&��<%��'�e��,j��9,�u��Qs�_��{)�&�7)� �0Q��TO�x��KU���iVShT���[��n��_=��
�j�A���l��R�&�\����n�|a\8��B4p0����(b���hgX�v���Z����3���5�R�P��
wx��;V6�m��R�i��|��b�k����X��� R����^f�����p�;���VJ'@�<��]�$�������9h}�*������U4g�-4T����6q<�����6�����'����uP�f���>�\6X�j�(��l�����c����Yo+�td�!����3z\�Z��G�~�x�J&B���eF�������(+�����ku9[���z�=�;�2h�-��H���G ��2������J8XJB���+=c��%��@J&tb��)���v�U�"Jc�v,��X�������#l.`�t��dS�{c\d
ev%"4����Z�g�f�����m.x�y>
O�f��Z��#�Q��2����=R�d3S�����G��w��yDm�3y��p,=I�*O'�Sn&��+O+���G��5A�C�	��i��9�P��eC;\����Br}2��_���+i"w�>�t��|�xRM��l"{�!%��	=Z�O��A�B�h�Z��A	�� )�1�E���^����-����.����l�r�PAC��VtD����y�j���U;ui�E�*����Pnp]���n�����g�����C�7#�^">���s��4&��e�S�a���������P����4�� ����I��������1�hh�S�
k,��J��<���A6�[��f�#"m}�=�v�{G�eO6o�|l�^�;�jq��wG������_��w����k�1���d����O����y�+�+�08�����W��H�B�U��p�z��<�X7���+�:kG�+9S���,kH��Wm,kc3-%���G
9eKA��������y���4Z��4�[�f��5o�!�C��I�����7���{�f�2�]��!�]��24�	aIt�QtZ}����'��j���n9�i������8��25�Wi��c��.r�_7� ������\+����(��B����^q���gb8�����%/
�7��e���������b���� ��i/���,��
��<8������%��Ee��a�K���n5+M��'����lU��c������c��@|��[�b����v���*V3(���)F�w+���x��<��%j��!����1EZ�����.��;�Q��[#s�m$
J�������(�D�A��No�	+�^}`��
�����v����=xc6@��:>P��B�p�yl��l���u���w���w+�����m�=�F@���@+)]��/_�����vv:���;��&d>]U<5?ffoW)>��f�4�s�����^J�;�jm�f��w�4y���<��g���#�}�W�fB,�4{)cC��I���"����9��vPS��4n���7����-���wV����ZSK�u����`���WoB��?Q��������� ����N�}r?ft�,V��q\����j	���Yv�B�(=>��d{��V�"+)�&��P ���v��]7_�x��j��R`��=�w��_��N{�m��I2\�����h�L�.?L���r*{3�����O����Hw�i3e��Dsa�����^'�81��t����0c��"����W;"��}x��d��no���R�T�<�c
l��6�a�D��c��do#_{`n�ym���A�^�V|��<Xy��):
�[�,�P���fFy��xG�����2���z��-�r���	l�L�k��f�c���y�M��>�`��7������6�����������)�9�	$����^���s�*-��
:K���0��w���[
V����]@7@�2���q,'�i��+zw`��o�U����4��2O�_Kn�$<�bi�a�4B���^BC���<���UyW�+I���2h��Y(��3��D��LzBry{�2�(+�����������xw��h�Z�Ck_��[��Yi�d�b��l}�@CJ �	�z5�����'�������V�"�pa��#����'u3K!������"�.+'7�
5FG�e�\����+_9��D|��$*k{�+5-���L�+�@X�z	���V����t|��#�����}���(�+�)�I_�X��l������)�c��s��� ���~����(�x�2H�\��$b$�7F�sn(�#z����r�-�W\br$���_,R�Q�&di�FI��De@��H2���XIJ���T��	���U
��J���q!���{�j��<����4��TV���w�h1���)�97fXp+h���J�@5r�Mf�k������H6�!���3���1�yXE�g��d��]���[ey`;�$P�k�\p{�uv
��*��Z����Y��iZL�g�BP0g3�
�����'���O������]B��l�2�k51�Fz
 �d��DAt���|T�T�7A3���&\�V$xY�������F���B[�m�Mp��y������H4�����^��X�(��P����kQqR�����,��u���-@C���q�sl�m����]7�L�G�3�
;+AX�|��\���U�<����W���A�a���o<�ko��*�VS���h~�O�7�r��q=4V��G�f=\S���5�a����ZDgQ��,��E����Y���ZD�����<����i����V�G18�S(����X��q?�K/��i�v���1����F�t��>lt�����rGP�A/%Gu���U��1.B��X7�QeXM`(\��+�F�����SM��e_hzO��g��q���_1�xg?�W�X���f�����7���c3"��l�\r2-�{�]cE��e����[UzQ9U����o�+�%�'o�tTE�*���x�u_�F^�z�����".������7����Xl����l�[d�p�����,� ������7�(6�����]��qA��FQ9;r5B�����3�la�������t���q?�n�	-����{�(9A�7��'u�����������u&�R_1�|l*�Y���U���:��6a�W9����#�j���`+�����v�+�����n�#���p�����������2AM�b>_�����!��FIu�
�3�NR�1g�erwF���\�G������
AM�g� �1�L	��Ix5����y�	��y�Z����_�`E��H&��U�������� ��a]j�����$��[�!����c�������3����������H�|P#�������@J�E`8]�eyB2�d$?��nZ6��2R&j�j4?��V�m��v��=�E����F;���f����4��nx�������"�t��Z�]���������D�lY�e1���`��:j�#�{��7jeL��\�l���������HQN�"��S4�a��$��5(����%R���xf��9K5s�/!�yQX'�A�)mh��;C��"a2[|uKm��_��z3%�^$�%�t��\�t��y{�S�S�$b����l)�BG�������K�1�i��3��k�.Vk���8(�M��2����|�.�v��}�-.��`��_sU�1������a!D/��W�r������/��'s-�Y����+0
��?#P�#L��a�\���S7�SZ������h�@\��	�r�D\
R��j���9%;���	���A���v=�.%�c���L��,���\��w�����EinN���Vlf#��)��AeL�|��
O�\��Dy���x]�9~�W��	�b��K��`,�6}�^�KmC����+T��U�]d���B�k��y�F�|gK��V�e�"]h	�D	����������`(����|�������Uz4	�&��g�.-u�������v���8����.��9s���9cC#.�Xf�,I���"}��
��EZP�Z���l��B���4G�80���=w)X�L�Tb�,�b�T��})��A�u]zq���*Gsn4��KR���5�}h:�y�����[
��c4V�T\d���pF-��,������CU�_u`�K���o>L�(��E�����n��N;.���}�C3��8#�i$��
��]-&#�H2	8"�"-��MWo�A�-7QD��ti�����x�HkaKe7e���"��Z:��k���Du�?���cPt;�}��~}V����?FG�Rc��R�����>�4bl��v�����>�0:^y'l�K_��0��$K���*����������(��if-���������
g��f8<)z)0=9���+C�����"���}�x�3�K��� BP��Y�l��q'
<	<��E-���m9es���:O��&�U3M�!����Y2=�����j���c�
Ku�����X��b�� ���j�/�JA,f��(s�G�XD�FQ��;���a�>y��?[�)qG\�������G�����0��
"�pPc�ql��XN�4/  �)Cy����2a:1���xf��J��E�������Th��bt�!��������?���,#>�fL��M��h��	�6/q �[���;yw�E���kP����{���GG������>�������O~���_��
�nP��A�N�,����5�W+C	��V�[
�J6����d�&���_��/���E#��>��gP����#(e�)�`Vv6�H���5��;�w��
����pe��Lw����L�����;���&����0":�k
an��rb�Q�|�D7�K���j���
��Z2�h���(=#�I=%+��(��@h��N���.�,�7��L�,&3���L}_��y8���/��>���=R�c��;�2�UM!�>A�+�{P��q;A��8�<
��M�W�K�D��D�$���
���90
�p�K���?�,:�FL��b��(�2�%�j��+I2�z,q)a����Z0�E�������>(��X�.����W��Wt.	\�Mc�g��&9����f��;��12���t-���'h����1]�[WE��������x��ck��=(&�����E�U7��S��o����W��W�7��K����(l���S��}"
N��2*^��sNZ���	�iC�Q-�PI���c\CF �F��_��wl0�����}����9!U�#�����i�sP�����A�Qk���m����JF*��c�KE5m�6��7��4���HB)��3TMW�!�R%�c1���:2Br�����L����7�k:f��C��j�/�eJ�a�9y
��X����jG�(�U>���l�Y��\D�Ao9����0;ZE��-���Z������?���9�!@
�.FuKE�qG����]^�O�k��������g����S2���1L�^���E���G���h������Z���v�5����G'[D�M��X�F���D5i�^�lh9E���������;.]]�=\Rx���VZ��w��{�bto��^k*�7�/�l��lqo�./��l����7R�v�:�������q�����������t����~���������&���I�D���������kqx�Jp���n��.f�eWX����W�$��"�/�]�B�'yZ����]���Y��\�i������f��h	��Q�t��{6�(w�9[�ee/F�����?%�^s.B'i��/�`D�0*������>����$�Y��1��+9�W���Z�����s�LM�\���y�H��k<T������p6��z�i���=<�Na�����d��"��~��HL1^�����w��s���*���AY(Z�L�&
���0�M�QW1m��!�&��E�i�(���w�u-��������ATD�G���Sc�����A�:��m�#�y��C���.MK98���;�KG����oi1Z�r����&h������
EF[��gx�� ��f��f���b���z��s�F)����/�{���]^��������MZ��[���]a�a�NG����c���^��zQ�h=u��rF�L%�O�� \��Kkc+t>p�$2�����(�uH��1z���:���0d��$�v���%�5e�.����U� E���|��t���^���
��:����H����mf����OQhP�I�r�b	��mZ5��\���Ub_A�4��x/�<�)|9��	��{d�v� ]��(���������?q�����b��V�����}��!-�l�PCd3�$�65U<�X����"���q@,�S}���<���ot�6p��F{��2�+r��d"7�iKM�d]N�WkA�P�����$��h�����l-e��1j"�*��@m>-������o��d��?=�?=#��_��:L�N��C�nZ�?-�o�,�)
8X���HX���'^=���(O����8(<��m��c�+���x��_�
��U�~�=�X����S}�}�q�K#���v����?�����r�M%��$���7?v�53�,���x�������E�����;OA���K�_������e���%C�KfZ6v�>D������9J��u�;^Loy�r�R��ZM�r�m��`���=<�����^e�b��'��:��������|�����&"z
�D>O�p����+h�k���9*4"����C^:�G����p��]d�Z_��+.�M��M�76��0����gs�?#?�e��������3P�Vs������`����G^��<�����3@T%.�hd�8�b����d��H���w��������wg�8��m�(�mI�]���A�h����������5��K���e�=��T���Pe��������Z��`~a�)��Z*�~(�����8k�S���Ly���a��;�OO����2h��J�e?���������i34��	�����Q>D��/�;�Q)��������M��!a�^��F���_+����"��`�M�Q�~wxF�>�����nrt�s
{������8{��7g��:�����_l#����f:8'�2[	��iq1[hf1A�Dpc���SCMv�pSc���c��Lu��f��JG���z5E2<�I����<�_;p��
.�L��e&�kh���Q"���.�0��������c27��vhD����;A'*]������1�y�*<�����$\�cz8����X��\�1X �����v�ciI�a��tlx���t�I���9���B�.�����)8u�v/��9���S��m?�S�0)�!q�/�PYz����0�:S�5�����1���
����0�|�&��0�J��m����5���1��3c{j�BYx������|y��1��c��`(b�	��I�o���ON��w���m��4��z���#���q/��R��l�j2>9��?�1EV�BT}F�O��NQ�%�gk�	�;\�,F�G�)�d�"���[��,����q|�<SSZ7Z���{[�	�eb��+l���]����_,�L^A��P�QN��q�	bE����A�@�I����=��V	t�W�	����a���z��g8����9<�$���_�(*p����[8����!�U��U�b5����eZ�.����]�x�P���$9���{��������*�7����R������������d��5����TK-g��>��������ww>�*��K�����2��R	 ^��]
�q�.m�������/���)�_0����|_s6��$����e��I����f���#���*�C��R�����J��5��h�
�C+k�9��z8 ��((��N�w�;�v:����������?>�|������������T`
#186Etsuro Fujita
fujita.etsuro@lab.ntt.co.jp
In reply to: Ashutosh Bapat (#185)
Re: Partition-wise join for join between (declaratively) partitioned tables

On 2017/09/02 4:10, Ashutosh Bapat wrote:

This rebase mainly changes patch 0001, which translates partition
hierarchy into inheritance hierarchy creating AppendRelInfos and
RelOptInfos for partitioned partitions. Because of that, it's not
necessary to record the partitioned partitions in a
PartitionedChildRelInfos::child_rels. The only RTI that goes in there
is the RTI of child RTE which is same as the parent RTE except inh
flag. I tried removing that with a series of changes but it seems that
following code in ExecInitModifyTable() requires it.
1897 /* The root table RT index is at the head of the
partitioned_rels list */
1898 if (node->partitioned_rels)
1899 {
1900 Index root_rti;
1901 Oid root_oid;
1902
1903 root_rti = linitial_int(node->partitioned_rels);
1904 root_oid = getrelid(root_rti, estate->es_range_table);
1905 rel = heap_open(root_oid, NoLock); /* locked by InitPlan */
1906 }
1907 else
1908 rel = mtstate->resultRelInfo->ri_RelationDesc;

I don't know whether we could change this code not to use
PartitionedChildRelInfos::child_rels.

Though I haven't read the patch yet, I think the above code is useless.
And I proposed a patch to clean it up before [1]/messages/by-id/93cf9816-2f7d-0f67-8ed2-4a4e497a6ab8@lab.ntt.co.jp. I'll add that patch
to the next commitfest.

Best regards,
Etsuro Fujita

[1]: /messages/by-id/93cf9816-2f7d-0f67-8ed2-4a4e497a6ab8@lab.ntt.co.jp
/messages/by-id/93cf9816-2f7d-0f67-8ed2-4a4e497a6ab8@lab.ntt.co.jp

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

#187Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Etsuro Fujita (#186)
Re: Partition-wise join for join between (declaratively) partitioned tables

On 2017/09/04 12:38, Etsuro Fujita wrote:

On 2017/09/02 4:10, Ashutosh Bapat wrote:

This rebase mainly changes patch 0001, which translates partition
hierarchy into inheritance hierarchy creating AppendRelInfos and
RelOptInfos for partitioned partitions. Because of that, it's not
necessary to record the partitioned partitions in a
PartitionedChildRelInfos::child_rels. The only RTI that goes in there
is the RTI of child RTE which is same as the parent RTE except inh
flag. I tried removing that with a series of changes but it seems that
following code in ExecInitModifyTable() requires it.
1897     /* The root table RT index is at the head of the
partitioned_rels list */
1898     if (node->partitioned_rels)
1899     {
1900         Index       root_rti;
1901         Oid         root_oid;
1902
1903         root_rti = linitial_int(node->partitioned_rels);
1904         root_oid = getrelid(root_rti, estate->es_range_table);
1905         rel = heap_open(root_oid, NoLock);  /* locked by InitPlan */
1906     }
1907     else
1908         rel = mtstate->resultRelInfo->ri_RelationDesc;

I don't know whether we could change this code not to use
PartitionedChildRelInfos::child_rels.

For a root partitioned tables, ModifyTable.partitioned_rels comes from
PartitionedChildRelInfo.child_rels recorded for the table by
expand_inherited_rtnentry(). In fact, the latter is copied verbatim to
ModifyTablePath (or AppendPath/MergeAppendPath) when creating the same.
The only point of keeping those RT indexes around in the ModifyTable node
is for the executor to be able to locate partitioned table RT entries and
lock them. Without them, the executor wouldn't know about those tables at
all, because there won't be subplans corresponding to partitioned tables
in the tree and hence their RT indexes won't appear in the
ModifyTable.resultRelations list. If your patch adds partitioned child
rel AppendRelInfos back for whatever reason, you should also make sure in
inheritance_planner() that their RT indexes don't end up the
resultRelations list. See this piece of code in inheritance_planner():

1351 /* Build list of sub-paths */
1352 subpaths = lappend(subpaths, subpath);
1353
1354 /* Build list of modified subroots, too */
1355 subroots = lappend(subroots, subroot);
1356
1357 /* Build list of target-relation RT indexes */
1358 resultRelations = lappend_int(resultRelations,
appinfo->child_relid);

Maybe it won't happen, because if this appinfo corresponds to a
partitioned child table, recursion would occur and we'll get to this piece
of code for only the leaf children.

By the way, if you want to get rid of PartitionedChildRelInfo, you can do
that as long as you find some other way of putting together the
partitioned_rels list to add into the ModifyTable (Append/MergeAppend)
node created for the root partitioned table. Currently,
PartitionedChildRelInfo (and the root->pcinfo_list) is the way for
expand_inherited_rtentry() to pass that information to the planner's
path-generating code. We may be able to generate that list when actually
creating the path using set_append_rel_pathlist() or
inheritance_planner(), without having created a PartitionedChildRelInfo
node beforehand.

Though I haven't read the patch yet, I think the above code is useless.
And I proposed a patch to clean it up before [1].  I'll add that patch to
the next commitfest.

+1.

Thanks,
Amit

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

#188Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Antonin Houska (#184)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Sep 1, 2017 at 6:05 PM, Antonin Houska <ah@cybertec.at> wrote:

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

I originally thought to provide it along-with the changes to
expand_inherited_rtentry(), but that thread is taking longer. Jeevan
Chalke needs rebased patches for his work on aggregate pushdown and
Thomas might need them for further review. So, here they are.

Since I have related patch in the current commitfest
(https://commitfest.postgresql.org/14/1247/), I spent some time reviewing your
patch:

* generate_partition_wise_join_paths()

Right parenthesis is missing in the prologue.

Thanks for pointing that out. Fixed.

* get_partitioned_child_rels_for_join()

I think the Assert() statement is easier to understand inside the loop, see
the assert.diff attachment.

The assert at the end of function also checks that we have got
child_rels lists for all the parents passed in. That is not checked by
your version. Furthermore, we would checked that each child_rels has
at least one element while buildings paths for base relations.
Checking the same again for joins doesn't add any value.

* have_partkey_equi_join()

As the function handles generic join, this comment doesn't seem to me
relevant:

/*
* The equi-join between partition keys is strict if equi-join between
* at least one partition key is using a strict operator. See
* explanation about outer join reordering identity 3 in
* optimizer/README
*/
strict_op = op_strict(opexpr->opno);

What in that comment is not exactly relevant?

And I think the function can return true even if strict_op is false for all
the operators evaluated in the loop.

I think it does that. Do you have a case where it doesn't?

* match_expr_to_partition_keys()

I'm not sure this comment is clear enough:

/*
* If it's a strict equi-join a NULL partition key on one side will
* not join a NULL partition key on the other side. So, rows with NULL
* partition key from a partition on one side can not join with those
* from a non-matching partition on the other side. So, search the
* nullable partition keys as well.
*/
if (!strict_op)
continue;

My understanding of the problem of NULL values generated by outer join is:
these NULL values --- if evaluated by non-strict expression --- can make row
of N-th partition on one side of the join match row(s) of *other than* N-th
partition(s) on the other side. Thus the nullable input expressions may only
be evaluated by strict operators. I think it'd be clearer if you stressed that
(undesired) *match* of partition keys can be a problem, rather than mismatch

Sorry, I am not able to understand this. To me it looks like my
wording conveys what you are saying.

If you insist on your wording, then I think you should at least move the
comment below to the part that only deals with strict operators.

Done.

* There are several places where lfirst_node() macro should be used. For
example

rel = lfirst_node(RelOptInfo, lc);

instead of

rel = (RelOptInfo *) lfirst(lc);

Thanks for that.

* map_and_merge_partitions()

Besides a few changes proposed in map_and_merge_partitions.diff (a few of them
to suppress compiler warnings) I think that this part needs more thought:

{
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;
}

I could hit this path with the following test:

CREATE TABLE a(i int) PARTITION BY LIST(i);
CREATE TABLE a_0 PARTITION OF a FOR VALUES IN (0, 2);
CREATE TABLE b(j int) PARTITION BY LIST(j);
CREATE TABLE b_0 PARTITION OF b FOR VALUES IN (1, 2);

SET enable_partition_wise_join TO on;

SELECT *
FROM a
FULL JOIN
b ON i = j;

I don't think there's a reason not to join a_0 partition to b_0, is there?

With the latest patchset I am seeing that partition-wise join is used
in this case. I have started a new thread [1]/messages/by-id/CAFjFpRdjQvaUEV5DJX3TW6pU5eq54NCkadtxHX2JiJG_GvbrCA@mail.gmail.com for advanced partition
matching patches. Please post review comments about the last two
patches on that thread.

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

Attached patchset with above comments addressed.
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company

Attachments:

pg_dp_join_patches_v28.tar.gzapplication/x-gzip; name=pg_dp_join_patches_v28.tar.gzDownload
��:�Y�\�s�6����_�����E��lIv���I���I�s|����h ��P����������������fnN�F6,����.�^]O����,N�+^�!��<����d������~�����`t0<
F�'�|�����������{��b]dr���}�������Q<��������"����E\�Y*�n�g����W<���G���<n���`8����&�'���`������%;>��8���� 
g����?��0��h0����Q4c���}+6��~�9��� �&�y�N5��kz��3<����W"-D��c)�Y/��/�oy!����:`��?�����������u���~��z��������Wo~d���`�+sT�9*�HEY�����,�E����OC���y.nY6g��D��\�������D{�����d>��uV�<����j%��R$��<�l��l�'��7W����I3J����Df8a�3z�*ht���a���>24\�A��Yi�V�F�U�F�T���8Dq��,�h���Q\��<5C������(��Ir��q�25�HY
��	�W�$z����V�5K��S��Yr
s�s��o��#578��Xv�������LfK�;�%�N`G���i�^7�\�n�5��,Z�C!%��I��d7�p��h�*�8�k��[�sA���]���8��� �<�6 	��;�e�*���E�Ek��z�RC-�a�'
�H��	c����-aBi�E��l��%�m��X���|�m�H0�3p���\s+�n�
�eK�Y�e�,�ms6�����+���+~���\��Vy����o C)O�����|�q�n�2\-.�nR_�M���_�hW����fO���	���E���a�"%�����x�0�'�u�)���K����+��)�L��f�^s)��f��U�9>���ae��X���Y5����I�n-�2�0��h������j�6 $�T��sXV�E�4�?��Y�5e]�X��jvh�w�9�&���S��fh��Z����p����f��m2���3~s��VE���O��`g�}Pa�����?|Yg���~��T�'��C��t�O�c��q3���c}�X�?��I�
�������Q��4L���OA"r�x������3�/�,��5�r�*x+�k)��u��
wj��$�;���':
��`Q�!h��� v:,EN���i��Y���t��f���������d9���h$~g�h"�Q�79�����CT�]gl��v���+�c��'���&�/-/�4���\�Y�[<��\_Xi�'��t��U�����.��2N+��������P�����;��H���������{v9}{���?��������(8��1)�)��j
z8����Gext�=t��s����&k��k4)/�\����bY�	��^�����a�sM�4	<@�g�s�&���o�<r$�zgR.D���%}��S>���X��'MP���Dm��~:�������]V�n�,a'�z���R�����h����������J�N-�vFYF(J�����Lh��[���]��:�a��n�fZ���X8�h��a#na<<����*d�8J�+�����V�A�zZ���68a��w"�/e�\%����9�%��.����o�j=��@�~�/0�W�8���j����Qp�WG���Q��m4�9k�)^�����'���/�`�g�$���v;}}��~l������f)���@;��A��_���!\J������.���y���R��Da���t,��0�)0J,����F@`���!���[�Bp�h��F������Y�;����@h��$F`����\'<�T\������h4�[D��[�6�v<N4��2����#:4#�����	�s-
���s�n�������zt��S����]�L���5Q=0�����ZW?������k��g��3R(�c
au��&;�:���J���%Gc:��v���������^��~w�F;s����}	Bp�'�'@�lm��V���,�SHM�������w�_��8��xx�2Xt�����KB����+9���8��������/Y5kC�:��l��
<���d������:���]^��`���G�����Nu/��t�!��:.�|�&���zF��&d��2i���N/��>�d�xb������f�hL�$~���]��8�i
�J�����6Qy
�1�4�Q�R>��CyS��0�y��-���3H$q��gK9EXG[z
�<K[hoL��M��Ea%�H���l��x8�������p(]�"��@��z<������p�&^6;�R7��9l����(�<��H���G?�����A�.�sQ���8��n���U?x���0�\%��Y���Y�o��coF���}6V� !`����b&�s��9��
����-ftR�P�����X�D.QFUz����~$`q_`�DX�pu���=�3�����z�9dM����������J{�4/��0"6�� lT�1l/�)b�b�<����%K��a�s���*�\���}�����"�M��`)����1��$Y������;M���\qXJ,-Lf����O{�)�=�;����7��6lg��ZxTN�^������mF��~>gs�Z�3���
]�U-6#s=M1�{�����$L=8�)%�A�8���b�IH��:�����a0<|�j|����.�>��%b�Z!F��|U#
A�0��1�Hh[*�C����w(���A[��
���9�X���3Y���lG��G�<��)Ko��	Y0�x�����S�(G��RjNy[-:CeY��b~�G�z�2p���[�jM����."v�+`��>9|L,�h�?�8��)�d��nM���*���
�R��D��Ip�q�����L��c����}�������D��6��5@k��mKwU�R�6-V����2�����2QX1�2`�:��d<��A�ic�[��U�]�.�����>�� �U.,���l���� Ir�5���F�H�����n^t.��~LlQ�P���h�9R����2�el�L+4�Fd��� `���o����/4K��������&L����2G�2�B&��
���k�V/���N<1f	�%0���79�<!$W�Y�h[�v�!._��B`�'�JD{�JG�>���K���F���X@i���B����
����t�5^�D��x��o�&3����0(��	U�8&.%^���,��X���y����t���[v�./�_]P@�oSR�g�������#��������e|��'4^��a�t/�Yg�Y~�'������n�C��r{��iQw�|�pk��S��1fK~�����i=��4�YB>'(c��p��2�sL�
OL:��|0���=����^�V+���V�i�����dHI���Q0<l�����>�����ei5H�w���g����cW�Do�o���C�������U
-�s}MM�
��c��?��M.0�B���S�e�3Q�xM�B���8-�,�H2�#���������mj;J��Ib�!0v����A��1���S�MN��&&����o�9`ugG���H�~�h�kp�&�����n�����%�
�Si�����`�_�`�k����n���3�@��B���+����g��L�<<�|u��av�&��6��c�R
�*q�A&S�0S��rn��3���V�,��QJ"Z9+���6{	N�f�k�Lu��`k���Qg��p>�z��1
�N��*�;����*�
����}X�Xo�Q�n�7�?��b���3PN����"o�P&�U�����!x-��Cp�Kh,�0���{+dH���Lw���7?��x{��,�0N��3w��Q�0<���9��GH5����_w��f%�*u31*r�4Tfz�:Z�"�8�7%�h�,;b��}%N��3������%QU�=���2���6�]h���G��UUtG"���n��l��g�w���{�����f�*h���4�!rxDw�cUa�a����m��2Ol��{WF�w�9-���!�*cyc��X	�!��mj�VW�2��^�	�c�SFa�.�sE�K?x���N��U��_��EX����%FU`
v�E���a�"gJ��I�%�[�������3�,i��_0���\����<��M����
D���%��������hx�1�����a\�w�s_�����g��kJ���I
�I����U�!�6a�*D:�z�bbLe#�-���+�4
��7.��o%%J6�p�x|8�C~����{T&;��5��WX=�j�����0��l�O�47�5��Z-u����e�m
��<��\�q� 
� n9Klc�^���*!?�K�X�l�����w�\��f�G�����Y��BW�C���y��z]��i�P�tL*��"��l�������}��N,�l�B��K�����w�8��9����szl=C�]e�{����NU��l2zz�������9�������X�
��Nc���Ue����Q�sRhDqbbT���l �3�U������^n�M;c��k��Uv�6���\�I�j��Q����Q��M�y3��>pr�|C���{�r0{f$�=\!�^�����}�����%�����|��\TM�c�2-�*o4���k���b�����1�����W��=f����@@�m���V�HY
�\�7�\���z�(�7�%��J�{nB�B���-��X��Bq�V0fV����J�2,�U���k�@�WIL��X�y#��(L�uM��)�4�����-��G�)�����G�HU�'zEqQd��#��!/\���3�BV#�B��2�xz�~m�i�]bJJ�L�g->�N'$e��g����5gt@o�)���~hPg?��T�4�������2T�s!_����>�q�z+t�02U���M��6���D�����zM����.S�vy��/"��ey7����K1XM�H���&����/J�8�^��Q�H�Yh�)��� ����h��I���N�
�zw��0V9)�&�>����4)�����4��~��iR��l|~��
�lR�}����d���)j�rED�,�}�j��'E�����u�!�u�%������f/���?�KTtH����>Zm�E��3���]����0`?Q�U���Z��Q|=K�������U���Je��&09���U9q���=o�[���q|g3M�i���l�_�05��}uJ��uZs�!t�����J���+MUw��7���*��himu���`�|�)Le�S1P��]���VM�./U������!+���G�[3O;�d]����/`M!-����,b83M�r&5�e�@����^�g��i�	�+��s7������>����*#����
H�}��8jh����������R���0`,w��<��8=�(�A��D�Z��9�B+��7R��'��������9�Ws����<O��]����o���/[����R���/��C�7�h](��?����{�c��c�5�oW�%���}�b[����.e�Mt�����@���������K��@�r'���sw�ve��m��(��a������ap���[G�T������X� (�7eM���&�������c;�z����H�k�]�dQV&�?>���B���ofa�����a��jJ_�:����a�&������{���J��sW�c<�Q�����-a����;X���:x�R�6B��x0V��>�*�~�c��5&���Y��iRU���p�o1z��8�����NP�����Wd.��5��|M�{�[ ��R��+rBK5o��Z��T��,���Q9���I�Z1-��m���tr����$=g�|���r������N�o���-�a1w�>l}@������[�����c��y���6Q�����H��Q�����[��z����0��DY��*l�7�����'��(�����80�+)7/�;\�n6�/G�&Oj���g��b$��E1��������P�utNgueUgFd�c=z�Z����s��nu�}�>�[������++�VFf��s��Y<d>�|X����>����1 	d$~�!���m��
s���1g��GVWwG���2b�|�9��c�9���@��p����/Z�����L������m}��&e=la^��~�r��+��L��D�Ht����.�����u���������<�=F�/��mQ�t&w�,����O��Uk��)e�[�p�`\�JF��3'��8�cg��;�4B�Y&7d�$�<A3��\UY[z(��,��nP��-�h�����%K@�+H�o���-����=�u�q��aVB�������i���v����T,�3�D���������h�v�nqC>���!����x����@��nTuA�R�Qt`��T_^���5�.��HX����N�����������OO��A��1:�v7���b���5]*�s��/wYA�'�1l9�3����
-�E/Y��-���B�.������8����Cz������@(��/6

Z,�
2����w�����=����������wq1��rV ���E�G�����?�M��A�X���7	��gN$��5��-��U� VJ�`�xIH������g�����X�g����Y�!H����U�\�>I�Q"g<�����J
3���������U����Q��A6O�H���~>	A�����{hw����j��,�
4JI^�B��*��G�c����82
�y0�5(69��XWY@�����D��\@p�%#%bEmT�?&7"��JN��p:o��Z^.V���G{���B����	"h��Rn�����/�~���_�G����^Q�?���$).K���_4�[�H������
������gt�#����T�����R
�����[A'��������H�,J�c2�'p�va�
�v���������5 g_��^tb��n0���K�I0�m��p����N���&T������Z�u�&��qu�~qm�m�����6"�6"�������J�<���v����t�jCcE;�n=�j�����#�o�1h�Z��3�������v�	���|�Jb��_A�2$*��N_/������[����V�b��L���1�&Ia?�"��2�S_����}���0*���"
oL>c��;	�-3f0�v���.���nH
}V��@��cV��d��D�J�*�2�#=;[;
�Q��B#Jb&'1D����Wg���|�b�>
���B\�*�3Bz)��14��-f�k0V��I	<���Y����n�~�d79�M��z������`?1+ql%f1zh>��^ptN�P�V��8�5��0��]����_�����0_�����*Y�[����������N���]��O�]qF�)]v��.���!T����	���O����~���4Dm����0���A����y/<��G���������h���p�p�?N�6����r��*_�������{����O����(�������3~CnJ�U2��OZ��6p�oA:���b�E��z_��OV�t�R�e�W���auB�8H�$BX������� ���&�S\_]��������	�[hO�$r�X4�F���0uPZK}�G�%��1�+u�^a��Ux�zm+9�n�kV�)�.�h�u����Y�*��^&T��D^�J~��0/�0Geacu<��n��Y�,�z���t��4�R(�����L���[������Z���%���')s����AmtLf�dH��G����G���bz��3�;��2�/���/����c�N��.�)o�{�h�g����� ���:���Y3�a�����E8.�_���L8��7��(c���:�����{�n5��C���{��
���@���~P�$�hz����wd�e�PK���<�\O�z�!�CP���>~"�FP~/�a����nHi/��������f���ueyo��^ji����Lk&�R���A���[f%��M��o�����cRJWQ�3,��Gf���0/�������x��~8&{{!a..�����2UO8�ZX�K�w�6�/|I 	���
7_A�"���������(}7�|�H)�8�X��b������UB�~�
a�M�(|'���z;���Ya�4��M�{�,��I��d��g��4YA1��?{�Ux_B�����i���`�L����Y��k[����
=��������z��Y/�m���x=��G��H�����;�F�38����iJ���1�������]��	����t�`��
R8h"�N�iVn���A0��Tf�6I�
�r��!�#�(�0)_�-z�K�0�e<h[0��}7�e��aY���]�?g��E�Sjr�DV6�z��q��lU���:���x�L�[v��d��uia�u+ �g"!X��R�����H��x��j�=�[�l;� g�xqqA0pO��#
Y���c]�;|I�-�4�	�
m��?z,��u��b�����,�0}�mt���M��A����&�a8>�	��*������{�.�Y�7=z����%O�
���c4B�d���`F������
��E#�B�Y:�*8��������d���@Du&����)��I&YO:t�RA����	��d�n�fc������s�:�
���K1[�k��p�_�]���E4M��.[;�t93��>�rc��x� �B���b�5"���� 1i�YS�'4`�g[�-� F(�9������]L��<J���X+WV%���2[���
(��Bs�p@�z9���Sp���"�}�
�f\,	)�Y�hygi��l5��z���R�����,_������
��)�L"��l�g�<�YJ�!�*�^S����B���	��T�-%���j��n3�������{�����.�jA�@�D	�v����iP��|
������l;�H+�S��/`/���/zu@�c55�\���� T\K���#���!���s-��D3����hDfk�&�j-�!�0��jJ}��(6"��_z��fD�	Oq�|��"=�7���,���������
���b5i�9G���l-0��m���t�:Z�����0`�X�^�,�4��C[d����1�6c�s����R�*������b|��
o_� ��Sfm���E�k�62�`<��c8Y�'���{��*�Q�����4��>��j���ol�����������-}�K>����r�0�����YC�������m&�����������Y������N2���A]EH���oT�*���!)��=V�1�!k�h3>��o�p�-!��H�fJ����6"����,��U�a���9�S'^:f��<��>������RD�V�������Gq�`o���������|���|
�r!���A�.CF4�*Q=���j���w�Nn��94�-���Tjl��j�����j5_���l�L}�	���0�e����Z�(��?'�C����Ds�Tx�=�7�td�LWE��Os`V	���!���m��b�=V�OA�x�(��z�����/���g�>D����!��0�"���Q!-�nr�����h�p>���<��~�q��~G��k��t�j��M��dt�=A0�O������Ljo{$V��".�|��+����#���D����x�!��g�|�d�1�T�PS�E�������$2T	= �����7�����%(%���CWZJ�,V�}%�y{O�{��Yz�y�F����A�gd����c�G��@Wj�=��G#'�d����p{'`y�*
�����������L���\�r[s���n�U���>�����Cd 	hH=� ��*e���wD	[��}�*�F���i\����:H+Q�y������"+&���v�R�:�w__^)z�EA! U�
��z	UQ�.����T�����jC��s�q��k�z$����,7�u���b<��H��:��ou�Z�������F��!�v�H]����T��s#�O�,�������^%(�,����b?��}2x�h��B��
&���'�)��-
+��$��n_$4i����,�Q����������D��xyN91�������S9��X/�N��T4"{��b�{����5����mK���]���'����AL����+�p��^��N�h�a��,K:��k���<���L����Ba=�7*��Vb��������0
Kt�N,�~����C�}I@t���;�<�D7��
�������U�9�6��"�wr`N�"�����4�/3/f�c��y��;���������������
7�� ���q���'F�k)%�ee=^�K��\����
{�e�<Q��yx��������{���m�#NCG��O�s'H��@O/�&�����b�p&��W1*L���������bX��}
�����Ttp����UXV�	�;�R��������
zKSr��!kV_:�8u��U'�X-��c#���h@c/���F0���u���Xz�K�}&��QYv<���;#�����,gv#�I�eK���,~�T���~qK���y("-�;���m.R�C�������_={	�1HA�(���j�.����EaiL�FU�P�M~���S��0��������"Z�%K"���k����vx��"I��U��`���<7L��Q����	��|g�a�m���'�s�V�����;~=:~��:��?}��kH�o��/
�KP'C�JH���.���|�b	��S��(J�+��F����58)�"�3�s�>c��,
=Y������th��F
�@U�R��2�(+oS�J�*��	,�v�w�������A���]Y�����!��-[�F�#pcJ��L�9���`"M��q�d�j���������r��S����~k����?�^�'������x��'��oN���[x�<�/GDN��9���O��^��M���h�
 �8�<��x�lxx�Hqx����H�<ZC) l���������}L0�
��V�5��;�|�HG��"iKhqB�4����e�t2�P��M^4`�}���������[�����jte9�G�
���p��79N�'�zd��B=�R�e���eh�����\��70e"���W�����~����r�]r�JX���i�k��|�H����,��hFQ����G����c����9R��d�8G������5�{RB������e���	�Z|
i1�#Td$26��	l��T#�6��:����'�����0��,���4���p��U�Z[��x�Y'
�A3�*��z�v@�4_bI(�w��%�!g*r�t����&+�9��D���
��Z��~0 )3�k��
��p��Y��6��L;�$t��Wx9�� ���������/D!��N��j�~�j�B��Z
m����W���He��%��lo;�{_�y��D���v*����c�^���%c�EJ`'`�$EN�{�f����d���8�y�3�S��d He�
�� ���_�%��m��X�-js��fns4��+����z9I8�
�!S�(��"4^zLE;�ob6�����-�����kf���)UH��#����kH=A�S�[����+��?<�'��<U�ce`8U�f���m�����E��������7YM�0��9�������&������)i,.FY��?a��g�f��3P��!������=T|]h<R����������tz�)�:���(���#�V/sJ���M�p��# !r�U`��������T�����t�����H;0s{[*����
��h0�|LI,�5��jp���h/*T3M�.[vW�&��o4K�9;A
�=GR�I
X*������L*���7����,��h�[����OB�,?�M��!�Y�lL��j�]�,�����������c}��h�[h��-�|J�K�C���	�<���#pn!A��60Z���[�Y?�� �B�a�?R��y'�}-U�j����]��IlSdx�������W��b�v�L�,��;�3�mQ"�z��	v����HO${��#��%o��c������+�	����a���P�*_���v�A�&=�PZ���s�]�AL�_D�h��hA�������&��Ct 	�na���4�A�X6[y��@;s�4�&�J�aX+G�#�f8c��,��_`�0z���,��8"��<��14
d���iOL��O�*�*&�ct.p����A�>�c�ZJ��^#9�t��;�aA>��L�����Q%��Vd���0{�yLF����T1-�l�T�q�����W#M�#"����)��7@��}��SBu���j%����^��<�}�80ocP��n�@�6�c�
4����O�)�F����M���
2\{S�G����8OT�o�L��q�����Y��4Q;�T#��Q
��v�^�a
l��m��6(B+��{:��;���q
3���W��{T)kl�<�&�%p�fZ�e�;�>����)�9��
T�3�`����l��#18��"]u���N��F<���bv�^�����������O"�+N���3���.A�*�Q�Olj��G�����)���1�9.m�~��R@+N�s���	��S;����
������^��5��d�Yi�+�����J?�D;�J}�P
C@�eV+�f
������n G�mu������,�?�����>��u�����:O���{<]Z��0�A���������\W����
@A���\���M�f��U�h%@��o/7���J\�m0�^��#�*���[#�RA������\���Y��, r��fn�b����1&�]T\(@�	��oRUa9���+�mi�M���UBcduZ2��T3��U�s<��E�U�A��-���v�N"�p"�YAJ���*�p�]���,���{����w\�i74�a'4�V3��E�#'<�Z\v�@>��i��D�?����R�Za��2����
z���k����
k&^w�G�x$��c�0&��p�	[�{4@$h�7`W��7�w*��x��reo�f�!B+�������a��0��48�w-	O5��f0��GXx��J�.�cvBYF�9��=�8��U������;Ny@]+��*��IT?�h�2c�x/4�j{/#a����%��"�V=���,@;��K{������#d�7|������@��<��4
���e
u��6�03,F��c+	jyT&�����X���]��	+�c���7��i9�M=[�3i��E�����<�����B��� e/�0�Gr�3`�����'Z�aSP�'v 9�h0sTQ�a^��t�c��Tc8b�+��2�e�o�#0U�6����a$|f�w�M�64^&H)3�����k-1d)EE��	����
 ��^S<a��2��}tG���h�Hj�� 5��8��Lr��2w�<��F���e��MY����8�~�@1����N(d��Y#�q^�Q!P�������pJ�){��e�D�d'��SWG��c�:��i��	7EU��(��h6�H*���k�������S#�q%!h��vr��&U�g`bJ~�Z���nOh�i��@����.kP�(4=�
4��n~aA<O��s6S%\��z>Q�M��!���@�h�5T����H�;]��*���&27B�H,���h���������64_.}F9Ho2�`.���5DG-�
������NFq�7p<�w�=����
vkz�c_�J�<V�����X����hh��N��Rc������aG�m�-))�����4�8���2b�����������_�fHi\�%Z��ag���IR��Z@����E|=�x`���D��,�k��+�����`��3��,�K��8PJK@3��{3�k������X����x�Xrw*�	g� CHW�1�1X
*91�<g��l�>��ZC�������7��J��)g�C����\����m��	�B���F|��sng}
-���9BrW�(1�uu�������2 ���Q��8���{af���G?w�>�a��*�a_�7^��+���O�tX����Zc;��*��MslPp��zB�2a��u|���(#�T����+����������R)���Y�jr���s�{]�Sf�>]Ae5�d����*���w�L��E�q��N�dH�p�e@F0(
�����1��L/���D���B�gy���w��\13��	����~ow�B���1��6�8���N�4����$!�Q4�7�H�����[y������&�G���c*n)j,��KV��
����V��
���:.�K�U2x:�!{��3U@b0���B<�Zd|�=XW5(m���gg�2IS�I3+��7�
��S����sS��	��~����e:OJ���h��j E���C�w��B�,��FK�[0@������q~�3!\�K�������?�}��U *A���,� ��tB	�<����l&����8�+�Ql��_7`e�>���k��(������B6��h�%X�T���q�Q����b��~��Uq)s��g8(�H�������0���G������m��R3
H�HC/�;��.����Mi�K
�0��-u��U�|�ZW��D����SC�GB#��;O���v�����>�C�l�[�*�Y��!m���0��8������p?���{{��$9�yY�U��Y����?D]�mu��K5�B�Z}z*p����Xd[���
/L���K�&�e�F#���ra���dE������X���AE����_�p�o\��m��N�;�2#l�u���)p�<X���u��X�=2��){�$��t�~�H}NcW�wV�7#��@������,��I�
0�2�Xa��M��F9�d���`�Dj�@���l�4k��B=����L1z
��W��y~�H?����������;�*����u�U��D{����d/ A���6�9�&��BG{,�jp��� 
��LG�=3>f�O��1o��
�'��ab�����9��l�,����������J�9��S"�bY�B���hS
$e����r��~vl�s8�X�H�H>	�8J:�2�$���q��Jv_��J���BX
���\��������g8���b���D��t�J��w��5����P���3�MO6AI���+��-.�LU�X���X��[*�L����������x���p��WdY�)��s�^|&6������d�-Oo��i*<�*�b���oUN?&(GE����bk�=ok��"�R�Mz��� +vr����������.�����p�Qa���?�Du	�
��4�Z����f���Z����g+������F�
I�F�b{4)�T������;�!�Ek���V�������@J �2��%���*x�%�MQ�;��EX�+��=��o��~���� �{�]C�\��uw���bbnC�N�"%CG�,/B����$�) �\OG��t�����3��-o���,a����hE�[_a[���S�|	�g����R'��
��>T��5�6��-���d�.	�(�� ��&�������0�������j�,eN�������
�?%RRa$����lNCXk@�Y��xi<#�!�~��A9k�-���k	[�'B-�0���7��{{�d<�'���UV�Zq��6A�?��jZ��`Cv*��#`@�>�@L��N${X��?@�"��2j��vL C���B�v?P�t��h�(�1�;�'h0b�P%�0\��=�FAV`5�64���=<�0���1���#=22���\����kb^�Y������f�.J������|��oxek�pB�T����V%����1��(��i�o�W4|�^��P�1,���yZIE�H��z3�1�E���-��MD��=��n����T3��Qn�
*i�/�Z����|O�Y<}}h(.����N[���J�<F���t�w�#T
)�8T�^�����+=����@G5�����4��M���l��HP�����uAd�2�w��o�<6�>���H]��N2B>����f��^H�����PvL���w�K[q���������O��JXw!����5�.9n���=��^�	����|u%=qT�$Tt�f����aD���Z�o�	���B_�WuCvRj��W&�����t��O�24�N�:2C�������b<y��8<�E{{���$��y:YQ!?'�2�~y�|^.�e�b�&����Z���Sv����3���C*���RTN�c���1H���P#=A\ ��K�
B�?w���`m���wK��\������U��������2=W^������q�G�A?:��&{�:����s�O�� �6�[�����0��������cq�����IB���z� ��L8����,�� p�����r��>m|@Ge
i���
n-Y�U�����]D��^-2�I����eR.J
o�$�/������z�7���bM9l �>HQ��uc0��F��PUOal�[(@Z(rSu��k�'#G�E2�����?�z �^N��FYW\�eCe�jC+fgJ4^X�����������2N��4��'o���]j�����{c�XHgD��p����X(z<�\=�������\�,�=���P�2�#�q�V�i�����5�>i)}��1D�|(��RmN�L�xQCK�����
O���nH#A$-Gf��\�l�96��i��Y�W�E�Nv�������Go����-_����O�"��}��h�)�g�NI�b�>��!ow��DY��R�Fi�EZ�*eW��� -+s?0�������Q��8~�Z����G7������K���N"PD<J�HJ,�0�>�X�tg~����7oh�b��0^���M�RB.7U��0��M�CJ�t�.��V2��|t3u=�1��(�e�|���V�&L�����U�[l�A��!H���7	�n���t�LV#�,i�@�0K,.�O�P���|'�5#6,���<�U����	��R�V�b�|���f��q3���(3��ef�%��"f/���\����$���-��`f�7�P�|`���'�@x�(`��71@�������\+)m��OS�<��/��.	�"�����)d�y:)�c�-R���B-bh/z�U��^�:�Z��0��1gmG-c������4<��!�7eZm�Y��^Z��LAM�E2e����m1�D�WD��\��9��sE
�22����"��]�Q���k�/h��\[��VIf,�(��-�����S`�����lGF)n[#�����3o�����sJ���R�%���'������
N)��&�S� �+a�P��f���W���4J�yV������f�h�V�[W:vk}�\[>����9���q���&���n+c����\��0�Ok�����
PgH����E�fGj����������i�R�|,	��dv����zs�ME�9R/ij�8c
���b,�=�s��
�m�(S���G65���RY#�>����4Z��Y�P�;,��nWj��$C/��S��Vr�F�������)2��������&T����VX}����		��b�P�����n�����H��)Ft����G��`�p?���F�����G�8����C��	������z�e�eLC
`��R��c31A�j� �z�~��0��%�t_6G[os3{�N�����3����!�o�%�;�bv����f�qt��L��~w���
������]j�a���,�@����Fm�;r�<���1�{�A�(�i\��S/+\����QB����*Y��5
IB�b <�L�7*}p�x��=�a�B���[�����<��GStd\�@�]�����rz	AXM|�o1�)���X%f��_����n ����~�����{?rB��}L��q=��:�����2�/�]�?)o��gCt 
�?��<�!wM����.�
���t��)��Z*vj�C��%���	Q������i:��b�#��L��}�����6�4s�#��5�m����'wJ��$����-v�����pP!c\C*��@�x��aia
�x����+�����������c�g�=��`�g�ym_(�M�}��B��
�h_�����*=�����Pz����d���'���N��g���	Le���h�H��}���gl�:�}���g�%,3�/o5i�&+n�NP��qt�hG	B�+~\*F���xL��R��Y�e(����:V��{5H��9��Q�����*����������Dy�B_~�������		�zE�O��E����q�A�B.��o@u����]��d�@���z�[��g����4�Y��� ��W3������������Uqn����6&��.v�%��]]�&(����g��@�K�����B}�rx����>3'(k������`O$��E�����w�=���[��]����p���:�mi�X���a������1ez�Y;�%;���"Q/���W�jz*�J�Lp��Yz���"+&�p���I2c�;��,e�P<PR�n��P]u.�3qy�����HFQ�5�B����#�`����h�"t��y�L�>�K�0�L�S�(��}l�(�q���W.fKw���)�h-�
SX���aj�i
V-��;�td��?��8_l�(�K�U��H������0�������\�!]T$�H1��Q.RB��L�@�k���(�!����@�!L�����;�UaA*��Ba���+8��(%P)��G��-����%n���=��zh�k�K���\����zB���J�p��ZG�'X���w�1uAH�������5�F�gxPi� 1��;����p[@��H9(PZ���LZ�����_�Hd
��4�(��|�.�F�s�l��JzR(���g�R�<~�%���'?��	�M=��x~��L}B_��#��]�	����O���I�X4G�"����h�`���������U�����.�8���FI�&�QT���r�8�(c�/��(.s!�xU�x:0&+�m�x���D����d�.�&-��n{��L�L�[	3���W%���Y%�K��j�+d�]n������B*?e�Jx.�1���l#l��:���8t#�].c�����R���.��rT��a=��a������;���L�,/�I���/����v��%��'�B@���)p-�k:1�)��l�u�#n�Ft�h%e~F�P2���Eq�n�,j��%8��^�<9�%wFX��wv�z�K����@1qb#��6�c�>kW�A��udP������^L`���x;�5��A/���7�F�>y������g�����A���X	�|
������������6��
�c0t�<jq�(��<=��4f},c ���s�mQ����<�%dL���2x90T�h *cx�a�@n�r6v�,��
�����Y��=d�4�1�c��;������LL+�l&�v�GO����'{���MI!�{�@!5���M\~����QKE��j:�H|k�D8J�l�!���xA5�o���<9s����<�=��yHe.����t�x��#�������`�2��ij	�I���.�#S�+��*��!���v��h@�m'm��\D���S�=��(�����N�&��EbY
�n����U��EEtP0V�=^hA
�q�|�@n@a0x
3)_��KX7"�#oU��7��C�TW�����������r\P��U��W���7�����r!�glw�2�\��2$(g��d%�S������������g�~����G��/�q
y/h��5?�cv]����8�,����oW�]�&�S�%J��!��-��&8k�[s�e)��Q�����9Ul�cjC^��r�g�����&��F���W��{?p��A��V
�]i,y������]��:kcJ	a&��L�%��1��o�e�9�-
�Sp��L.0���C���+e����.@���K�$��^9`R?7��j�1���:���4��Ug��������)�]��R�Q��M�����<������������q��F�j�z��P
�`�(1XM��L~�|[�s���e���4,�n��C,�a6�M�����}���iYp�e���qH����@_V
r������� �"Q1lt<���X:�\1�����F/�
�A�7ye��r=���0��x���$>�����G=w�L���"p,~�G���lW����������<�����%�G��������<fA��������������g�dU~�����ON_��u������oGg?�|��������^}C��e�G��|���|v2�������6}�gs����3w�9�.�_���@�B���.����������z���"]]����@|/����9��i�/����]L{��8j��"�;�VQ�
�X,�4�))�+������������;��6(qq�'�.7&'�J�	:��\�K�g&��Z���#.z�]��30��a<������,`vV ��,��Bz�(�|���6a�";_��s`Pc�o@������C�����@dy"\�	�^<���x�9M�������	O�EEs�L����nH��JdS|LSz]�k<'��84��H�y���`����B�����3�"�o�y������c��d����S;�B�����2�>:�a8r�]�i�;��N��?tLM{�n_O'��e��\����q5�aa���
3#���(�?[�]���*�"�A

��B;���r:N�Hh�K�N��^3]%7���zH��d�^R��z^(��"w���V�p���� pME��������p�����C$u���Z,���F`_z��+�0�0�
	�����4��i[���������99�����Q�	��C�����/�xI�sc���n��)��������i���f�GE
������e�8����i6�:�`�zT��;�y�h��3P�)f�c��la�f �xe���f������fmAnlr��y'�9��x���
���D}����/�8?�l"�ka4x�{���p�)+V%����M�OU�)�p1e���`�T`b0n4c����u�h�zL-d�������b�i�3����@�K�~�
a��96F�L������<*�����cL�),S��y>�s��x�\d�wA.8bB�E�I�ID���!9L[2�������ZTS�@��
�dm�*�����A�<��.��d�K�>�Ws�I��G&���N~S���E��o����]$�����������of*����"�H�@`t�&RV���T���o��4q�S�g
k��� ���)�C� ���H���_�6�4T����g2J`1�|�2��5��Lf1��Y����9�S��/|�Lw�TaS�D��!&������g��J��O)�0�z�a�_G�$~6���Q+f"D+!	�����H�2"�#Oc���0Z���g�;�%:��yF�b:�J���|�X;�|�C�	x�_0�>�����^���d�RO�L�1�rk�����^�&I2�E��S�������{� !�$dn4�~�v��1Yf�.����M#
$P�=���G
�aP�
)F<������J�|u5\4&����	����u�]Yi��9��4��.Yp��O���N���gM8�oN���[}���x}	q�������L���B�H#Hi��;�F�#gh1���$Am��i:�6h>6t���u:��z��_�a�VMU%N:������x���|}}�]L�6���q������F'�9O0��7h��7�Z�j�b�RTK�e?����1Y���`|p����zr�Vw�C��5f��b���gD=����j],��\��(A������������G����~�BP)������H�`�l�?9����o���o0���1�m�I/�8��;�]�/�-�}���-�\k�@L`�A��R��n�Ug��t��r�I������@R�@�������F�95#���4��#�A������"���2�c��?~��K�L���h��9!�?F��?�_\X�0��$n{�H$�n�v�����,����|�y�<�|0���ZF�w�������x!<c:~Lu1!�<5�]&o�)p�0&u], ��>e�|��%����,��Ef����a/����G���hk������_��_�����kG�E�`���#aD0����g �����m�*�,�A�h��<�]1�q��V���-��������?h�>��6��3~��>��C�S���	�K�!}�v�dN��}���w�f+�O����<[	}
�%��%�����&�VB�Z[	�Q�/x�c�[%:&�J������#��-	�a��[���D�f1'���
U���1�;i��M�����2�],w�d)����^G��V����&j��,w�%�}D4ZZF�D���;�;�nm�k�"�8�b��*I��&��t��n6���2�Hw�s4����Nv	@v���m����|��=�����Q���?��"����^�c�^k)��rZ�����*z��^OaC������prA��a���x�������Ax�?����E�Nn��A��}����L�T�yp�`|T��0�����@r"����xo��~��
!/�C�]�z�tIj#u��������`�7�z[���?�����w��=�.��?������$��2��rZ���t������
g�D�I�8R��*[�,!�k�����L*���oek~�J���
�$Ki��|���2��#$�x�Ey�A�N �y����$5!:.�e^*[�B�:��+�a~o���G+��
-��q8`���3��s�����S�%�4�H�a^���o��D��a���O���l�XRu�e��H�2x����5.������hdq��-�!��|��:�����s�[�"p�*j�[�������R��`��&y�U"�+��@�_�5�?�_0,��L���(������D�1��/(��=8� ��<b�	,�*H�<L�P�b�8��5�a6�o�K�%^G�6���b�z�*�/����R� �]�4����8Kd��pA���k j?�&.�fK�9��-������ba����P�\�g����N	�^�!��HD��&1����H�&�a	���`��"����c�+PO�w��C�$X�*vv�	�%39KP+�����E���+��^������(9����A2���+���f�	di�X� �
��Ad}�+�����YLf���
tOrC^�,���n���s��~\&F�G�������6�F\�^�Q^2����e�R��AQ����^JA�<�?�7�a�����{�����}��dt|zz�[f��{�:�1�j��#=� ��H����b�!U��)�����"]�dV*I�Q��$�!��%Q=��y�KF?�����y[(��tu�6��i��A��J�A��HI���=��l��������{4�����
a���s�S���W<D�����W,��/�@A�*A,�C%�����e�L<��V��&��A��(&�`��oX���L��H�u�����dt�Y�����%�+��L"����QDP}�.��O�^�D�t�/�G�CL+�?:�i)���Q?�?��e�S�����M�N[A����K6��d��_� a�ub<LJ Ii��@,���B����Ud�3�a9��x���s �uVr�M;KM�6l�T����4t����f����5�]Q�JD�`f0����]}�{|��� ��=F��%?�x�
�'K�>;��TJ�h�oH ���b�R1q�#��=��"QPgp��sex>a��`B	2��%`��A�'O��.5�
��s���0�x�����D�9�d��%|;Jwu�iR#�0<�w�#��g*)a]�W�\�n�F	��O��4�8�Z�������F	�SUT��%���h��1��#]��H�m���/\S�-w��JPe���8��*�}LOP)�x�l�C�,*�j��-��Ls��4���9C:��r,�YX[�C_��o�OExe���^�-�Y��>�a�5@�3c �����l1���`!xhu����r
�R�2(!�F�;�
�($�]*v�(��=�,�u�j�@�yEi��-f�R���r����e�����X)Ri!�\����B���������M��wY�
��S�b�����Cd��n4d6��]����RA�}^}��%M�������I�C2��[t=���3m��;�w��:�����1V�l:���7D�7WE�:SD|����+�B����<%$��mh��:i�3W:`,;FWu�����bm�)%�����6
bl�O����.��������1d��oW-�6�~|�$ZB�"p����qHzJ�S�����U��S<��0���������������?��������3w�0�����{hJ�\i!��$�,m��'t>�XS�.�n�z�,�-�!�S:��c��7z�z�/��o���bq>�x��N+��Z��)�].V����n<�$7�]�3S����
����a{�{+�����p��.���G��LW��O�G������pBJ���������`?�����>�?C��o����C�Z�`�P|�B�b(�����K���E�nk��(-�P��������Dd-��x��yt�����2	��Tj�#w4�X���K���D~�@_<!O
�JS��dK�EdS���5����@i��4<���	�0���P���2Y3�y0��G�����r�5�W�lq�H��w�;��2�X�C���<��Bq���{��3.*�������$��;�I�s�e�#{�e+�z�p����w��������O���0�f���>~���!�T���i	7���hH�c��ZSE/�����e��p�2[�EW�]��[����L���T�
T���w�N�S�?�����5�~�6�qX�3��]��uq9��;;I�@?��������������Ka�"�6,H���
Dx0Y�xFt�h5���Q!X0	�2����HBZ)_���1�������)�Sqa����@�6�6���OGm���.��xx�\�6���h��N)��S50	���*<,�`@���?z�������<y;2��D�%j�}�8�>��\�(_B���!i3����|c��������S%��4�,F~`�7+��N9��Y�H����� ����
<���u�j��7y��	��\BX�x����5�x��z�����7������N�\N@�#���C����9~���������	�1y��%�
�f�p�sp0`�jb��!�|����
:���;,���qtU�
�DQ��|�Ex<~�Gs���$���5��2J>���bt<����#�c�R �i����~�s;a��5s���������~�s�i(H+�4;�������b�*xS�l�%E�7p��q_.m��.�W�����|�m[� _1����o�9��3�F��:�]��tz�^�S-f�q�K����'+�S�?���c-�Z|!y��<�
1����M�7��?��!���W�������4����`��=��,f�`AE2h�p��V��nI�	����f=����F�Qe���}���3����K�"�"����m�z���
�'�l>!���3s��n�{��[�+�)����<6[y�=g�|�'��Z+��W�e�3�~x;a� 5x	C��4��|�;��Fn�\t���i���A��DhrSj�:���)��m��M�j��C�N�TA�F��/�=}s|������~8�L���K���WE�����q�`�(�g��.��T?d!�n�N*�VU��9�9�v�
Q y��(%y�������Tn7W���n�2E��cE�`�%*�l��h���[�<�?",��Z%��`
�J��1[�xf+��������o;z�����85U�\T�U�c�������\W=s�0��/���Fb�������f�F����a�?�Uu�g�"cO��,���jD HXRT�H��4b;
$D�@�����U*��-s��!j��e�>��/��n|���G��f�E`�������N- 2���,�7�����a$�����BC
|�vD����;��d�8��sP�C���v��(��`�ld��i)���R����=q�M	�@�X�
3J r�������l�WSP�l�������
��n 7�2��\��Uu�hW"�E��+o��o?qc�O�����~v|	���|"���/���#����F����F�h����E�?���y<�����p�E��x��I��E��Q?�/.6����1���}>�}>p���DCQ�y����{[[)�%hEf|Ad^�BC`TC�5�z�c-4W��{������p�f:Iv�v/�S��B.+auY;�DS�k�7�����Q(�3,��I	t|��>~1�����5Pa6#[�OsI��4��g��?��y�0KZ�:~�EcD���d���� e����7P�':��|�X��?���46@�X�~K�� ���m��	��w��<����TB��-�n�@��@�8���6����s�%'
D>'����s�y����m�.�%�ds}����6�q li���,��^�K��l�d��Tc$�7[��&7�F��X��Z�'t�f��S�O-.�y:��������Z[2�,6'��00�%�=�|�a. \�����-eV����&Y�
;XB��x6Xg�N
� ������Bh���R�$}���s���_b��	���'N��%0y��)�/��B0�74,�BI.Dp���3m`�O���0����>����,#���[����x�a�u���)Rzy={D�G{����j$�����3�iy d�e�� `�
��@����h}<�"���^�����t�J�t��i
�8�5 ����Y�R[���>����0m��Z������8�13I�\���=�����s#�On�-�5F;�^������� �����)����Jw��_�#��;����1�;%�BJ�O�L�������T���;���g�i��/	}�%�a�����"����k�1y��G���*h�v�d�p���&��o��9�:����wO�X��H�w����c�Y�L�T������?�\��)��r���@�? ��Q=Z�����:4���f_���=]���cJWieV��W�1����ei���
i������
n+��qai����E��4�^+s�������� S���4�b�_Z����������@��w4�(��kt���)�=�*�L��X���c�������e��a�|�����Jf7��%��'j�A��!\�r�n��G���`o����-9�k�
c���i`~6�G�������D$�<��)5��0S`��\�5�@,T�0o�SI
��'������?R 3�d�e1�
$a�h��R���X8�����AuY]���w��q��L���7h��z�3�������Y|C�����?BdZ��/K�0�{�������my}�����r���F�� ���;��/E�/�`������ Br">��1�38[C�����NS"���l$��4�i����i�8C:��\,Lob�x���d&�8���JD�_]�L�)7cia�WYx���f��������&��RK�*iI�K������V*`����-�w�
���V�y�]���o��[����a���49 U'�&���Ju-Vf���\f& ��3�n���Z2��#,�*�����71W?9c���n������������DI��D{�.��m���Wr�=��3��I���p|DL��/����i�2�?`�C��he�9@3h�K;��J>���9F���9#P���V����D3��u$�2����ocM9c�K�ZO�#�8�%
��"�p�����,AN�������Bu������H�+_�zL��]3c=G��O���1M�a����b���"�-�*�0�5�M�����o�0�����U��E�,V
7���V���"dZ�Xl�X]�j��	���@��)�����N�`3�x�b0s�@��-@)-�S�e�����z
5BR����Rr`�x��L������U�)&
%Q���h�-gf��~(�Q@
�^!�HS���3�M��y<`xN^RV$�`�������r;�Z��m%��,IY������Rs&���%���D��T�r�QCM����T-$�
�J����P2�+��:C��'��3��I�C�%f	��7����NH�*�.A����D���p0�������Qt�#H�5�	RfYT	��{�,y6���Z��3���_O�{W�����
������OU������D����.��P*�>�9����K�A�+��Y��xi<#x�bKKZ��4����4u�,���d������5�a}5Mf��
�>�c!��������c�.�">]�h���Y�B���, #�8URV%�����,=�#�6E��H���T��[Z1��g��c5�/�E�5����<���J��6��<w)v@Q��	�Y��d�[zz�����1�6$�1r��
&U��r����E�?���Y�W���,2Xw6�1��
�*����ak�"��N��R��"c�]���a�@���������aG<��)�H/�����/�M6Ap���%�T�&y��Nn�uDa�w���gq!��0�+f�C�R����oj)*���������	(uNW;lgKfJ��u�k_��Xj|��c�gU��t�+�Q\��/0[�*����sh��K�+�2��08�{���������4
3����QO8>��qc]������[�C3��L����Y�c<��{*��H���C)����}q� �T�!��TmI�����t���`[,������8O���Mc�Dd2K+�w��7sr9Z�q���dM�
4���a#�W<���t�c�rDV5��p�n��������$�� ��(x8���,�����.�$-��X&+�'�p	�lU�b�9�Nb`��:��;���0�����^O���?�M����@tp����N��Y������$�bg6!�Hl'e���
�����b��]�wy_�cZgtut���;y���n�^8��I���#���8��^�����{�_-sN�+��|��r��u����g��?��?�f8�'�W
d
`x!�2��S�	��������������M�C��,����!���p^��12��s�{�yd����bly�r�@��\�r?xx���"������7��Z3�x��@��@`�����o{Vq��Wf-����8��@
bX�0�}\7I����.p�e  ���y��G����[������S�~�B� ������m��a�����7A�tP�� OT��mI����	�����]�"����=N���
�c�����2N��)�'F�N)�45fLy�pQ�o���5��m�5����duyR�����*���|���/�t��r8gg���?�1�P��W%;�:,$M�@�*���e+�������Nhu���������64;B3��"�:�r��+��	�6<����/D���4�����*�������H���Z��P���C�h�������q5�~��������.N���+/�s5<����U�\����)/�+e�{^��U���{�Z}���8��Z�����:Q����j��uk�D��M���k�1�w+�s����<�+J#R�_L��F/!;�3gr|���Pi�����]7=G��$��f����*��~su�5Y�V9�g���>(��*����+@-�Zl%�T%>����O�,�.o�yY��o�n����<<�+k�#�����L#��,o�1�����L��<����Ku��[�[��L�e3��
��j<#�����Y�h"+��\O �����1���M&,*�1Y	���c%��~����B=e5Bi^�3YM'p*�	�cY�.&�B�Yi��M��	�$;a�)��y������#8��^���g���8�r�Z1��h����BZ�-�\[*���e M����
�H����0M�d����H��

� F�����|! �E��`B���O�)�*�����q�N/a������*V�#y�
�-2������oq_Rp�
{�S���p�,qX1��]q�D#l��O1�2�Lf��(���	��KCfJE��j���-�HA����H�x�F�Z!U�L�+?���R�[J�N�3���Y�������Y���@�9�#h����%(��"=�t4x����m���B4�d���e�����l:��e��E!O���RW��9�����'�3�A��Lq����\���oh��'��SJ������H,�@�C'xsf��������cB���7I�z���F-E�w�w��&���p���"K"���:�4�0������W,!6����7l���9@�w����3�b9�����!&
uLf��
�UC6j�Z��n@W4$
�������t��������w�=4�V��S�p�i-O6*��N��bZ2��v/p^��j�*c5�����RoB]�����Zu����l��6Y�W����i��8P�\)�FT���6+e�e���7�:�d�E�A������lV��p�Z�y_��A�g�A�u�����n���M�}2�|o�m�>���l��s*hl|�@\�6	���\�������D��V!B����x��`��BPb����hU�x"L�73MU+ye�������}�*c��a��+V�JP�R����1�}�6�#y�c+F�������yS�\�5���v1�KB�s�DF"��G5�bU���#�3��8�1s;J.z�do�7H����Y��6rA'�!�2"_���/���k|��%�0�r���z0�s���T)�=s�x_$jmn�Tv�L�1�M��g'�a@.W�2����a���A��G&U���1���%�`D �����,���F��06�{���=�Wq�v	+	���`B,�+z�Q��'�O6����S�{@�����9�e��^>����ZEN!�#�l1����ZX)�1�H�A3����[��T����^���X���������hoo
�����X��K!@�#�f�oEi�����(���?��n�,K���!��R^��}�f-�<Vf��d_���P���ol�|�:���l\X�G��O��&{{�Qr����Sm�$;��bH%�}�C�Z��]OWT��lcgp�������"ZG%HF0!���#����5�W@���bM����m��M%�7��jK���9����?���������]]��w	��������*
��eR-�7y���~�������?
����wpt~1�8:��{�xp��������|�����|)%���x}��?��G����=���9'��2DQ�bK�X�x�e�����Y�_�v���'b�q����G���4g�#aDzJA�$��9������1]��I)D�:�"�Mp�`�qq���l������%��R����f�~���DDV�<�K���a6v=����Y0k ����5� 1�m@g����;29�A���d�hw�P�j[5��Tv���|K��[0���M��L�M&2Wkcm3})FB�z�������H��;�1�,�)7��5�Dx��D@�����-��G�z�{��f�j�Yr��!�=r�a��`�,/�������fS����C3�-Y�F��~�7�����e9������(������x<8��'��TOR)��5�S>�or�����z0��4����j��
@B��'��J.]\�J])!	�&"	�]"D��5!Qe�%
X�cT�C4���_���?�Bg��O_}�����J����HHu�
�H��e����]��� �_��;���Je�|�!�Q�H�	a�~WS������eB�2�.J�1�N����z�x20C+������E�W�?��o�M����bL�g������y�#vs�;�D���\�)�����j�b�pS[�P�,� �
����.�0���C������ZT�����m�!�VD3���	���_e�,�[��>+�l��x"�����i?�X�2|
�'�(������6���@.4R~2�
JmT.�``]c�	'
x7O.���{(�!
�!�	��<��d�N?��p�0���\�6��H�(���^`f f�@m"&^��J1����@�K��������b�I	:7h��2Z�,���Ee���d;��Y���kPX���,�0"�.���X����E6�xn���d>�I4�������o9��c�Y�nH�������8��t��9��������0���c\�����"����#&��CKC	���N���8���',���T���0�����!=RI��4�����_��-�],�����*#��r����7m��?p������)y��G�A+���5������`/����{'����ow���{�����z�Z��NN~	��G���D�G�<d�hX��^������>��?��W�����}w�����������������n�Z������������������c�)���x������?0��{��%��������=�g�����.{���k����=�������������,��?4��C��+�����=����
����=�������?�s�+K������������6{�{���?�?������M�����?�?g���un�1��W{�W{�W{�W{�W{�W{���}�D������B�������{�y�����W�����?z���|�K��k��7��{��[��{�9 �}��7�#�������@>���7�gB>;������������#����������?K>���������|����|�S���G���o���|������������F>��|�Y��g��y��B>���W���|�)��G���Od��|�y�yG> �����D���|������u�yA>_��C��O��?I>���{��?G>?�O�|���_'��_�
�q�������!�������.�.��q�n����1����$X�W����!����L���������^�*+�l��p&�{{��x���!�=ZZOQOz}������cq}�&^����q2����'�k��#~?�|	�j��';��k����u^������>�\Ovi]���������,'��>�������{;��GjU��]r�MW���E������!?�
#��	����2b���*���#<^\\|�������:��M	q�0b��S���LpS���2{'k�\%�M��X�Y�4��v�,��x;�pq.��wb<�.h��7�n�@0�����F$d�����L��F��k�
�����'8�*a����t�Bp�e��t�iR��6�t�S�����f������e�'s
sp+���o�_���ZW?H1���
�,�|��&���*�Oe��:����`��Z,���|����"
�R����Z�1���6T�Dw�������\-���p"`,]�L%��'c����;Q>~����eB&t� a���4��3���59J��'_>R��![�c��|1A��^^������]v>�49���a���������&���k�^�2:��#�Eg0�\t����P�KX����L&���Q�.}3M��{<�]�'���]�����@n�*�\��D&���2Y'K�u���|s�
�{����;|����=�i�1���
n ���E�(�kY��i:y���O�=���4�I�A�{d%�M=2�3?�q��A�(�!&<�r�Sgw"������_�E�l;NX<l��9@}�����l��
�!��hph�s��.�������/bjx��yf���t��#���l�2�i����7�L\��e���1`;i�����l@U�������C�k��Ky�l��$����!�T�t�\Yf�L}A.�?3
@�{����``��DrA�Xj~���)�}N���z�������<7�5��ac�,]�t�pb�Hy(��^&4�2�g��6T��q1�[�>��9���
`�O�������G�����2M��8�.���	u"���/[5�ZI�{�(S�0����Rb	%�+�+�C~^,�sDf����g|��3)k����~�RR�f���w�t>�
���{,���O�St{}A��z��?��w��j���q?��q�Q�h��Evqt1���M���wx�O&����q���1���k������"������%�� �]��G>�;"��W����&�����
��K��_$����D���&�����'������C>�<�,��5�|G>�������=�
��sC>/��K>[����w�I>�����gD>_�OH>=��7H�������/��c�����������������������!&��� ���b�K��|��3�_&,�p�,n��9NSG���d�4�uF��h�7�#���
atG�����z7����4@����{;�~@i��1uK>m�c��,�B��������������E�d�S�]	
#��������rFg-% ��RJ/���nXJ���9��S"*^�K���zWi���K>�p�I����
E�{��~��b������g&���	���Sk<�&U�O����H��>��gr��g�����	��
'i�>C����f�s��]qsd���~��k��Jf������>��G0}/FEZ�\9)��4�?�"Y�S
C��=�'�g$|��E ��A�P'�����BP��~c�������WA�B?���`���|�s<Y�_(���*j�`��0���8>8���&������yD��!��g�'��4��z>z|z������#8����Y�Q5�H�����������Z���1TBa�G[,����v��:�}����*���~/������p����(:��(��G�aorxp�G��pp1��GQO�pS�_@l��(x�c�����GCG�����E��,nvF-�y@p4 8��>;;{�������k8����g)�'��"��,N�t����[�g��{���o���OHs/_?������������|�<�HFHmyX�(N�A/���&�:���e��I��y{���e���<Z.��W@!���:��������i|y�~���i{�R]�5Z���^�I�iN������z���=,�a��G)��Y�4���\��P{;�i��
y��Y���<�$�z�LBq�	y�����s����_=���C,�����uH��S��C���{_LC�q��<y�h�����v_w������`��������ar8�9MjHs�eC�&��,�)5pv+X��2�z��#L�12c�{[;�u����?�.������bL������z<�B�����������7�����������p%�3������x���
e�<����>IT�C�F#"������&p���bf�>CXdN|��4{��d�28Q�y���-�:_��ko�)��l�)��Z��0�-���+�d�#�k��%v�`����I����1d��"%�-X�@��9-������$8;���	a�W��C
Y���k�����rF�������C��L���R��SZ1����?���u����������{�Z"�Z�=V�~�UO�St��z��|}rz<{I���O��<9��E��#J����� E�iE"�j�y(;�n0<:"$;��K�/�����i���l�;a�N�]��y�9xh�J���T��fV�sevV
R�zK�~� ��B4CM��?�@�	�k��k��t�l&��r(4�MZhC��,
�����bN���,*Z�1Q�@��.
E�6?�
�f�4� iH�4�B��B:"P�����������:�99�����.�Y_��&��
kX���.�?!��{c�"1�<�R���a����+,�����S��y��)��/Y���Z�+����������Kg��\�V�E�~�X������5�=���"P6���-��vM���n~6��6����zaEO������+|s;SZ}��@��lH8���W�*t[t��
>����s{������%Z������v�������k���9#�c�]4oPgnTi�Bl�p�r;�=��0+�b��{��G{�R�rT6C�l�f�!��x��:�:��f���f^��gSf����u��,n�^�l��lv�=e��i�����G�5?�^����w�F�
A���y���%�'��/��<�b�K� �)����S��v0YMt�,�<?yz��,��p�*<��}����|��A��N�A^��2C^��+�����,�O?(�����a ��woH�@}�������:������!-�u�������Zd���dB����P/?�E�Z����$/�������;���C��.�����y�k��*xq�A]f�y��u�����M+C+m�Fk��n1��m�r���������k���k��5Z�������a�����q����qt�����CMUg�i��$�2���(b�k�M/P2i����
�r|�A
�fX��C��=�"A�f�������h���n}��pC�����9�O���/�ID�����;Cx�sX�yVf��{n���T��b=�5��v�Vl��OE���~����t�����=uz�h|��,3�����t�I@��i2������v�)Lt�J�d��K�E5dj��%�LPA�^m����I��y�
�(��XC�X+7h7}�@���$�{(�$�]+�A�j�X��B��I��z"������I
�1��=��
Z�M_mI"	�H"I$�y!I� ITI"/$�{��$Q�HR�����Vn�2n���i�b����jU����?�����~9��@!bX���Hg�f�h������������]��Ad����F%e�����|&��2aNd�p{�:���{�4(���\��V��D�����Xk�j2&��!�h�I���A��m"�#���Z
�#�?������nO4��[���g���Z0���U�L�g���Y�����@Y0���UdX�g��H[���0#�z�$���R%S0Q�)��g��0��r�Tp�x�+t�@t��������Q��C�& ��c9|���
���O@W�B��Ch��_��S�v9�i[U�iz��^�wj��8��]4:�^d��*b��X�{P�����p�F�J&>]pr&��-���)��>K%,�ad���!>c)��r���Am,c^d�B%��E,��>K%,�ad���!>c)��r���Am���O*!`��(�����<�}�>k'&����D��9CA��~k�=Oa��C� {��,����R�-����.�,�UJ;����[��9��.���YJ=����[�3W]����yK;����-%O��J���R�t��*<���+H������.�v9~T�qw�1zb���s5�%�/�	�x����C0}��?��cD������
iN	bS|x��y���������I�zr�++�k77�q}�;����pc�ej.�J�;N��N���H1wu�t5��y`���2s;��'�x�SEoB���&���Y)�Uy������9���`�":�^�}*�*x����F�l�]{�3J{[���aE������}��gib�;="���F�����LhT�_�����Ub]���� u����Z^���m8,����&Lm�lU3[q��Ml;�AS���n�os�wKL��y[gq��pMA�3�M���sP����Q�ii�Y�������b?K+�#��SR����:7u8�^�E+�2z ����LXF�',�zm���n�X��������Qd�r
��9d�E'���u��f,��v{��@h~G�p��n9����9,\��\U�]��N��OCA��^�����B�b����?4�����]��p
>���V��W�3} ���V�l��A�e.hl*��9�Ar�i�=����_2�o1U
/,U�w1vw	�������P;}V5�
"����j9	
f�l\�V�U��cb����2�����4�P�S/]���?�/('�w�������O!����	���/#�xX�G��Q����i��9T
������k���-����w��4:[2\��663[!��a�\�N1<|�f����|A�Z2���
�fk�1�������2�<V��,aN-�M-�M��_��_�|����[�;�Y�L.�e��F�E�y��������[;���$�`�&���"�#"?fI��p�B�.<�V+�h�5gK�i���i�����Oq����U���]�;�+�!��&�t���>��=���JF��;gF��"�
�OU�e*v��a_��	�,��7�K�-��;
L ��G3X��������f�Y����~��9e���F���������KQ�mzed�Q[��dc`I���~�E�nk�+/��Y)�8mr���o�E��)L%�����!*D�RfDJ�7��F�F��w*�����"P:����H%�<�R�o�C�[���K�j"�"�����%�[z	jp�S�%�%X{����rKP�zK����;�J���e��9����G$��r�/�K�?���VY�r�&���-�����_��-Z|��!��\���������������2���\a�@�r,��F�,�F1CJ?�7l����k����H�Z1�������
XkK`���v
��5�����M��J��Wp��*����e��ux�����g�
�6o�Zm�B�����]C)-C���H���-�N����5�%u���Mn�Ki�K��5�`e$��2���L6�d,����/o���L6m�@�6�`�wwe00"���)y�h�'�'d)g�_?�1G	�N��.A5��o���*^n���=;{F� ����oO:Rz'o?
�����\�|��S���W��o���p��(����W
���)��	���.Qg]�_]P��O�z�����Y��%���a�c������!>x�V�F).�y��W�(M��$�����������T|����o���/s���r/���h6��W���<8����
sx�?&�$�u1��.�`E�X�
��Jx.���4�^�IH!0�!5�A��P��3`�f� �2�|�`�)�#�� Z�B���hayD��f���B;�Y�*@��>�EDMD-�Z-|��:������u
���T����7AD�Q��v;T�!w
k���5�J����8������v��a���������6�I���
���K��cR�\�����1Y~���Q������v@c�]<�h���%vu�G|�G��<W��iD��N��7@����b�R5�������r���M�)c���|������������:M�#:��Aj���~�}F1.T5�)j�^�
�NT.��-m�
',v'!v{	:����V@�na�$���$`Ed�<�6j��g=bw�L�G�z�t���#b
�0�e@��1���97��TQ�1H����x�0��4+�n��%k�\�����&��H��}�h@�U�Sg��;x��?y3��bDy�%�B�(^a������uA���c�B��������lAm@��^>~���Z��������P:���U���G^�b.���"�z�T�����j7E/�lF��X+Vr���|	�)�.�Wk{�7�z���_�y�a�0�d�������i��Z����jA�K.�Rr������r����?wV��
��E)�����F
B��1b��h�O�*k�_�(����U6��&
!'
�Nr���d�j6��/K��Vl�T 
U���A8WP�>-�]`�_����0���P�}�W�~h�5o��"�C��K���b�����C������n	�s���2�@��/��q
���*�e:'/� �+C�����mB[����Y6�o�%��+Y�����U�i]���9�-�����x�qju=ea���N@C
����x���h|J,�V�c���>I@����h�?pU���!�H������V�� ymZt��;�jpxy�/f��CU������qI����
��-������T����8�:���[V���P<.�rq����~O<�����e�T
Q��m}�^>�x
���nWP��>�tW���	j���6t��P#����V�jC[m���
�&Zmh�

Zmh�
�hC��V�jC�V��hC�7?]��X�#�5��u�oD�u7����,���F���������]!�
��+�=k3O�}�]����Ug~*�LC
�-�S+�G��;�����Mr��I��j���`��sV�\[D)������.�����-�T�o�W�R��q��U���G��;k@_VJGV��g�$�v����$t$��������(����[[
����K%P|ZU��j��
:��h�tge	�p6K8�v�^�%����� %����u��J��nm���zG/�p�_���I�����n�_�u�-hL�T�@�]B����FU���Sr��?�hu���NUE�"��E�W�&�Jt�
�To�"�j�����7�S��f��N�!��i<����j�j��VL�Q���Y�������M�T?��V�j��!��iT����j�j��V�_�Q���Y�������M�T?��V���N�N��wB#t���
M����*(�i�
�j����������=*(�i�������S���Z�v
�k��_C��v������?���
�������}/���rS���
\���5���}%z/�J���jWI�~mF��s�t�L� �NX��������� ����`��T
����s�+V�����l����U~���|D�2]Uz�����^��"�Dq�=�\��{�U��[�k8��|y�P�^B�����u)H���(!��{�gJ'�X����.-"V��4��j�aVJ9Qg���
��e��V�e`[R�o	Y(�A(v(�;T	Y��*TG��mj�,�Y��!�oIB*�,G�';�{DP���#�@�L�HA�+��-���+����I����*KL6W���@����^YBV�o	Y$�A$v$�;R	Y��*RG�2J���H#d������xK�H!dQ]B�s��7�
���<��}%BV����3����tG�e�hc�����+T d��C�,!3����V�~���;�im�W?z�U���g���!�ra���WN����7��
��u��M��BA��H�Z�!��Dd�Q�uKFZ2��G�����n�$N�4x;]]��|wN��x<K����e����<
�� ]�Z'�w������u�cB-�`u��Ip�V����H&�����������_A#7
�F�KV��D����H�����\�$B��#�7���_qpm�\�Zgc�{8��;���3�m��X����1x���\u��t�^�;3
nS�M���<t���y��l��l�:��_a�����x=i3�����n�w����B��F6v�v0� �rV�l�6N��`�����f���W��5��lg=Y
:)����R@��)�]8��Do�D�!
X�0���S�(`3�o)�fO���J�T�W��5�l�DY
9)����R@��)�]8��Do�(�!
X������(`3�v)`��.G��*����
��P����**|�AJ��^S��?Q	�Q���t�L����?��WS��������T8�1���c[������"��.!����? W��8W����2q+P
H���s��X���_~C�|�T���{���*�&{��Q
���b]��`*zp���(F���$]%����������6�^�����ZY���d�;D�
I���3E������w4Tr����^�N�c����l���Y�G�ihRI	���Mzi�w�<d�E�D-;���Ka$���b�
�R�k�4�,"$H�C�x.
�$��:����|���^�]$���'Pn��I/
	�.�c����Y$!_$�*�$7�H��Y���^�[$C�"�a>|	�AF�b����1��z]v�k.�|-C�6��!��X$��U\���ll��iFb�-��G_�dmqK�H5�2-�x9�Y��g-�����Y1I�R���;x�QU��W��L�+5�����MU:D�
r�f������x�kd��W��3�����j
"^"H�m�x���B �����:���p���HUJ�*�b��G�+f W�_1uW��T�%��]1����?�I5�V�o���0������W%V��������w���9�+�����Q����+�x���B"�����m�^F���M
[����Wp�?xx��+B���dy�p���K��9��X��u:�_��y�s����dn'�+�:����U����B{8'�����ONO��NX�9��`���Q�k������r�Z.t���r��0��1+?:cW�P-��Ej��Q�2��x�>��N�����~+4�
}����"������������I���f�D��@��$�Tf*!����&��'���	M,WU��[���U`|)O^ �}�\�F��X�<����vJez�*u�!?�5�Czi������)d����o��x�+�_��[MJb�^�k���3�8[i�� �J,�L�J�Sr��F���EE��j�j��`
�`���'����s����cW ;W��7��&K�m�g��|A��*�{�����J�k��^r�t]Y�j�E�k����%��^�T����A�0����w�zuo��<�x��h�=��tE�����Q7}�����J6d�#,Y����P�����A9�Uy��S�{I'�u%���G�p�U�^q�(���*��2m.�U����#�) ��="��Gx$l��h���f�RN3��:9e��:�l���]C�W��8XC��nP����s.�3*5(�A�[Z�f��a���|.�Z'��m)[���[���P5����5�,�s���37��et������������\��-�f#f��aX��[W�M�/!(0�5��e:�A���qw�:�$��1I6a?PH����p[k�o��Y�g��(0����I"�����v��a1��&�v�&��l�}H��t�2���c�$>6�&��
���_�!n�jmB���7k��?�]7�<IDC������aX��O"2V�V=�!��V��6j^���ie
w~����������v�Q�U��s�[��v#vs�Z����^~��9�m�HZ��:��0�k� �J,�L���VD9����=:��u�f��i�����|6c]�A��
X����^��������Z����6p�����*��&�J+����9�F��=;��u�f��i�P���|6c]�A��
X����^��������Z����6pT�#�T��M��VL�=bc���:��u�]R=�	��6�G��Q�Z������K�]B������=��
K�������Y{`ft�
Gsg��dlJ�y��(K�b�ec��;tr�����gJ�!*a��DH)�k���aR������9�^����t�7)����	D>$*��j�.���B� ��c�HCh'
�
��T�N/��J���j�����@*���_Cv��}��jk��tG	���|��h�K�(��Q7��\ug���oN_��"Z���?��
��k��m���*�1l\Q�u����P0x���1����O�����])���Q��E������^*���9oF�-�~�<�W~~��h��[���=���������K�A?�fd����w�Ky��;[��n������������[*�S�o� ���n3d�o����7��Z�������)�P��cbP�C�S51�q�o���h�����go�����41(�����:����G�y���W���S	D�`������!Q�@Tr^����1�Z���R��ZC�T����w����2���LZG�v	�K����
f1���Y�L.�e2�$)������uBM�&�_���S�j���*{~|vrz�|k������Uq��|�lQ�����%q�����B]x��V����e���
� MYcr]��iISl��F��T���I�"��|��18b�-�n�y~r���#�{����GF�����Y��;~�~'OX��m��g�
�9p���UY�%�Xsq�0��[�-��j���%Mm����|(������
���#���pn�K��|���7?�����Cn��]Wi�����'�Y�<�d�������e`}��Tu%��t�j��!�Qt�/>"v�<��z	�G�?��zPo�e;�?���[�����B�pJ������%dmTppH��EfP�GFH
�A�F%����
	��A��	aA�FE�� ���k��d#+`�-�u��5p'��.��[���x[�7�{��[U���_+�������{�xK���+��fgZn���?2��]���8=Q�w^�������qN�����,���Ox�|/W��t1'R��]p����7�$�6
)-�����������'�R�:���t�����I�&^N���v�������gd5,?=~��I$�XS�
���*�����z�[z��4����N^3�^�H��Cp�x��1��Q��������9k��j�`���k�����������K����H���V���C�|�����*]&�d�`&��$X���`�8%�<~����=��  ��-�"D��o^}�]M�`�?&�$!���b�����J�%���{�ir�x��B�kb�x������5���YFcNFX;�1A����a�����au<�xh��C;Zk����>F<M<-xZ�0|�x:��������A6��T��h���`��];y8 �5�1�0���l�P���

n-s�d<����+Qh�J:!"u���wd�.��Lc�ZIR�`���H�HjH�J�-���0��RlX�H�i��W�
,o����52z�N/E����P�
�T���� �\���o�x4����tw�B�66U'�u��`��<e���GP+��W/����N�XV>Ri�B�e��e*�b
��j���y���������2�2�y�J��1�V|V�^�K�2��u�}O����9l�s,���%c������r��o9-��L0��:b;C�I��e>��;Q��!}g�t����)[[�l�N�{���RJ�&��)�?��6����?��}7T�%?�)�f��)� �M�8�UC�Q��}\�M$�:@L��t�x� ���Y'�xw����,���b��K�/���{Pj!�$q>���{;JG$���1V +�V����0�^r?V������:���!qy�Ju�`����LI�
��
�~�j�m��sCsC
sC'�����
+bn�1v.$xo�1����$��s�����~�C�>X�#��)������~Ej�m����i�917�07�`nTs���r!�{���1�Jb�9��������>X�#�AJ�4����b9�������6���-�w�d9k���a��L���-�'�-�W�rz���`�����|U$i�P�"h��e&p����l��4�<KM�lV3������9�����^�u&�#��k���R�x���*w�����~�{�����z���!�����^�@�Q�>���}���d�����#8��h�Aj5~'�J�;��4@�=K�C0� ��s�
���8�p�Zo9�m8g����
��i�Aj5~fp.��\��p.��\(-�2�p�.H~�!�����$�������
?���amVIi�H&V2i��,�L���^w9bI��P��.������(���P��B;��G=	L���������]�������?~s���v��b�l�S�������	����gv��"��N���9P�"h�gX��b|{�����7�����:����Mj�m�������
�C��n���1���+�//��%AS�M�2!�mm����b}�u�V�j[���|���&�yoeS��X^�������������L��)��8�Cs3s:�����B����`�[���h����'��1h����6�@U��9{e����\�7��jm�*`e�9�\6�~�VY�y����j�`�:5qz�����K���m���A��]�B4�
z�������M���y��@p{T��e�iG�(��{Z���D��N���
	9
��Ul&��l�<
�*���4��n���!:�A[�
��-�-��:��}��!%���ig�=��Ho�a	�_�ng�O�/�p)s��m��V����^���q��`ph����{%C���������ak��;��������
(����q;��uG���>��;�08��/�����mL����������o��`�Y�����!erd�w���g������{<9t�}(��nH>:���q;8r�}$���#84��{h��~�Uw_�:�o�����O��u���/��oNQ���q_��n�{K��������%���[����2y�L��Y&W������t'���74��{o�.N���T�i\��f�3��7�|}���Kz��;����W��Om���'�*����T<�q���*�����>;��S�v��;�(;��t�UZ�T���l�����zC/�����e)A�g\]�NGY�`���ru���i@-�������A7F}F9F*��G��se�������Z5�H�,���FNl���b��Q��(��x������?t��(JX��x�
T:�(KQ�'����LQ�6L/��p^D�=qV0W��<��|�U���Kk-�k����>�+7l���S�����O��7Vh/���1Tg$��B���HU��-���_�5�bP��W$
��PU�ek�����
����0��T����}Qut��k��0*�T�Y���(�Z����~�u$
�ZLUkejfd��k��������%���uv�����u�����-�ES�81���������o@S,;�5����VS�j����j����X
-��)[Mq�)�n�)�t5�j�<Mq�j�[M��p[M���)�����(����:+Y�Au���@S%�5�����d��TS��|��(ww5��B2f�]H�B�c�]59F�c�hS�]zU�{!�[���C'�-�`l@�X[���<,'^�f��U�f���G���ros-74[_�*����.^U
n���6
��QD,��eh�M���RlP��������$���F)H����%Ep�������*������V�
��i4<�Jg6��jI��T�
�P)�R#��4@�,���z��� N��'�v�JSRUZYIZ���	���9���(��R�G9�Qv������e����Z��v��+��ZqUC��Y����aO��n��F�zO������0=V����O�f�[�YW}+f�U����\ 2��������C7��6��}~Y�f�Y�:����-K�*��C���q���qr�N���O�$��C��iHm+�(��Z�����u�3I��[��U6���5������� ��8�����r��nL�a	�wE-VJyUJ�TJ�:'O;WPX���n�u�[Aa]�VPX���6�b�7��6��u5����E�%�~}\Kn#��N6�)[$���L�a�>���s�����=X��{�vZd[���5�B�}%z�J�E5��j(���=���9�7��H���.�<
 ��.\��l�j��OSA�7�k���Xq^������������z6�m�tU���c���zi2�h�,�R&K�~��R��K���BUX�V�7TZ�e���A��8���l���R���Q���/��4�eK��e����3ZM1�J�[�,N�\�9BY���z���[E�e�e��U��
�Qy*�D�
	
ZH��Z�	��EY*$04H`���b������$Poh��
z_���l)���"	,�z�$0�	��8esGH`���+K��"	��2�����tG*	��QE��<C��F$0��@_��Z$�eI`���� ��B��$���?o�H`�����J$��eK�*I`����$���H`��)�+8��@���^Yh�[g?��
n5OwY��Z<J�����>���Y.���8�E����rCV_�g������7k���KQ��Br�
���Ft�-���NKu�����&���%���t�(������� 7H����%3	�tF#�mF[\��E�Y�W�Q���+���K�8�����I�J�����
N�]��mN��m�s����h�+�+xw�^�L����n>JZ<��\7�#S�	���$.��&d7/��BC�2
*��*������v�&�x.�*8�k@_�k���������>g,�+b4�l�����0�����������j%����c�Ut�E��a��5�.�5z�F����^�)�'^u�n�R#���x=l���jocn�����&y�	�.6�5M6�1$0)z	9KJ[B��2�����$$#)	�GJAB�����e�D#$)��D
$B��2��1���4M�D}�iZ�yZ�������(�f�/%��W}KQ��bV+fY��{�f����W����W���G���E�)���J/�w�wM}+����J,�[�7|�������o.��j�-��V��M�A�K���J,�![��d?|��������o.��j�-��V��M�a�=5O&���l+�lH�,5��v*76�0���`�\���c�{M�4I�#8�K��W�2!��|.��U,�ycAM��f�x�����f����[�K��.�j���~���!��eN���x[y$�}5Ov���� ��,M��F���a���B����J`����8#��Dt_��4�%Z�-?��a
���(:�N��"H���e2z��,����uU�>��z�\d����G/������!�8:x�eA 5��i
������&���T�%�O>Re^�r5����h�r������bE����mV�E4��{����%��l�FK4�<�xrzr|v���W�ht�`:_u���_��M��\��������gg��B'K�����'����Ze�WOy�O_��9~���kFt��A���W�7�8k}k�ZX�������'�gD#�����M��o�� ;��O_�Na���<Y��d�&�i�vz�`pt�:8%�<~������� ���-cm����5�?������P���d� �����������:#�}Fm�Y]C���#�}�����*t���a�g����
��
-���+�=]�}����xMOd�G�}cZ*8��|����?zS�SY�:�Sm��,@xN?%��
���lD�G�Y��9k����&������]k]C^�>����A-"����9��T,S9�S������T:�o�i�z�8��b�� ���M��`J�?�t����I�O/.���s�P��,j��AS\�!&��i�������c������t�^(sbi/���XX�2��Xo&�U�����s�����e�pD��h���a������&��,qZ��3_�Ok��L��`��&���&�\!c?&����i��RT�_�v[I�G�0�"��'���~70���g���g��:��/�)�GcV�K��o����`l����C�z�j RpH�����NWW����
`y<C�d��u��:�M/���k��[�kq�:U��P�(���������/xlK)^j���g�U,�h��
����}�A�^J*u�i�X��fX5���cq�����o
��jwq��X�'K}��'��/"c_����\~/+���oQ���X8s��E���W9pWWzRlS�4���g�b>��)��*�me���*���T��WF�X��YRs�%��k_��y��j�ZgudeWW1eu4��s��
s<�5�2��5	$W�h)	�9��'��Lp���?����&��^����
�E�eT��"������������v�oz3��n�^��E���_�pkc[���f�{�������M2#�F�f��F�������Z�V3��_8��aM6�KMP�8���5p�t@��B�f������k����A/k�a�7{�i�v�i����$QNC��y�����8�=���6~K*���X��2����m�#�={
��,��W���<8����]���2��@`1�Q���AEM����*�N���0��P��=���(���fP>~d�'PY������|��x���Gh'��*��I<�Z�#tw�>�#�+H������N�"s�����F���#�O<"���,s�bs�%��x�v�Z�Gh'��%��xP@�j�s[SsX���4(��\�#�L,p:+���a��0�9��;,F�������6z�\�8�����f��gu����gg�um��
0�.n(b#r��P�XC��^
��\���!=A�y��Pk�H��&�8����P/�h,�=��O|��D���j�SB���y��Lg�s�[:��M���Q��2��E)	��z����l�H����������3��-�����9!�q��x�["���N��D�k��;�w����=�w:�R�Z+�Z��"L���M���)��b�aB�r�\�WdW�4��I�Z��4\�ZW�)��.���j�����4�������J��U��n�l�2��Q����������}�i�`��<X��[$��X�W�(���\����X��(������g�Kb��.w�$M"�����QT��ic���,�h�P�i�.Z���0�X�)��K�^z�^������2���,��f�~5�l�,q�"�5��n0�����w��	����Y`�P��-��_�^�E��u;��k��g!��Z��0n�2��v�?�i�c�]���(YePBMWGJ����D���L�o���p���2ZS�
��:�%A�\����t���[���S}�/R�h�^Z�����U0Ch�~R��!#�#P��SH��t9�[�zK��Gp'��x�jv�^F���1H��~��A!� ���~�/�����\��qy�KgQ��2��5�A�����L�8EE&�*�>��=��e��X"���,����f��K[(Q�B�8 -�c@y!�
�u�QE������|��p��>�OX�`�e>a)���;,	���VU(�SZS�NO��!���4$�[b��)�Dx'�g�/�%?��������������}�
B��Q���IWZ���������#T�3y��J�j�N�4�L��`�R���6vM��������t������ayp���������q�j��z��?���\���Fo�(�XZ1t�bXC���/��Q��� �@��(�4���1G������B��nYC+��.d�n!��GV � �g�1d��$�H4��*J%�!kd���������52)kd�����F���eT�����U�4d�]�����R8�T�����2Yw]m\�5�
tV���9��y*����W}�QY5�^���/�w�@�����9�������_�.�K"�Z��u��-:���-1���� �b������������������2�'l����w���@���X���%q�B,������j
��*?�x�u#��,�1��mi��&��i�G�����a������I��o�4V�[,�~(���������gAPb�`{��zF�k�������U`������fe�����8���^U|�m��9�q@���:�C{f���t������rX�Z���#6":��S���p�~��z���<P�'�2P���2F����b�5�F^�2*V����TD��������c�&z���������|��������}���U2���������E�9 6@l`o�,�12l�S���kf�������o����\�/r������B����c�(z�O)�cc������])JrJ��
��C�`}�*�rv��2D�MP
������
u�^t��vYc��B�_��&3�#Un �t�A)�����j)=��s��:�a:S:P�F��M.����\�����J�r����FUlq#*v>�kOU����E��B��F�-��9t������2��
�F�2�'�����+��9v ���!��2��98����eE��,��(��	�1��#��C�bl�1�����qP#~����8���(_�-q�c9��{�8������GB����v�I���(����#bP�,�M9���'�aT�;������u@=��b��CO�,G\�wl������6���]����y�]����r�L0���'$���u�~,�+%i �K�sk~r����;��tT�����v]��y�����K���?��tfU������������������������R�O���<k���Q�����8h�����fS�&�L�R1VR+6�J,�>�
��g���iC�hGeML��-���G��Y��<UN�-��O�7c	Q����!�m�y���Ani�umE�m�Q�@K��4������
t�7PC�l�������x�U:}�J��Q����v��k�G�����P~�Wg���2��M'1j��N^�=9~}���o�#��y����3u��P�$� =���g��:��rz�!�����l"T�-�W�6��@���aJj"_����-3,������������}��6��:��gu��C����L*WZ�����.�������W9�+V��a�Y������G�U
�l�U>�f"�'�~I�����V��(d������(l:~�G�L�����@�����W34�����J����JN�b�SMtu�������=Z������v����1�#�#�b��&����������������\/�u�c��s�v��c�@�hPT���)l��C��MQ%�p�^�+7�x'*���y�6�m�����I��I�Hl�D
��������[|�B�h"�p�H ���&Z>F+�@~��,qFV*��!�v����S/}OW*$�.���R5-�~9�y)��[�g_�h�2��%�Fj�@X����=d/��N,���H�D�����Z5,^����T��eh��P����C���K�8y�������?��e����`������2R�+R��=.L��Q*T�2��e��eT/��s��������%������:^�����:bZua*�����ym0�<b���(IZ�}"s��i%[�l����k���k��W���`�%c|�c3$�H��q}t�e6��#3�q��Q���4����=��6y�*BMr������[�=1(yJP�Z�F�[��d��O����n���^U�x��sS�]���"����T,=6e�!���n���eevgP�$��p��0��8��VZFG(�B��J���a�o+�v���[��sJ����k#L�����/>no��=wuS/7��>��>�c����,*��,�F4�kzaFI����L���?�O��������P�I7,&7�0�9A�W1������LlT�{�B�#kNGf�������_����O~}�a��S�v�
5�Ldz������������wS����WZ��1g��+��c��Dl��������U"��UI��-�!��2�{{*C�ge^������Qe�z������z�r�#?����
������Z��jj�V9zN�����mH����kFq�A����^l��(%<7���R�}@�m)�([c��M��*����h 3q[*3�^Z5d~����wu����U~��F�!-G�o�r�z�E�aaQB^~`�r�zC�����D���E���Z�Y��k9z}����W�Y^����u;��7�
������$]o0@���v�G�M��|0
e���,����w�l����k�
5Z���6����J���y���e�o���X�~�'QK�b����R����I��4\#dD-<�(�,��q+�]�5"L3������R�|�w�'���0z�Bdg����S���4b&���O������6g������i���6F����"�����#�P����t�^�P�G�����2o�m�����g�5f}��
��8^Q|jE��7���XC�`�
Z�6y�����D�Pv�P�*�h(*(TA�����`�'
5J�lc
��&�7hw�P��9K�G�"������&��B��BQ%*�`�'
5��lc
�C�7hw�|<�;���j�������9N�n&�T�_����h���L��$��Z��b�;';+O~�Y&i:]��`:����u�|����p����	y!���4	.�j@A�B�d��{J����p�����%j��m������M�2�&�2�����2�|{������������+��X��q�����.�����er�M�����&��DJ��,�N](��e����/�LM}G��o����3M��z��@d����o^�*���e��u�[��uY�r��\��w����^2���Bz����/[�.�7�zy���*�q%=�zYu����t}A=4.s�QBW�^%�f�+�q	��������u�*�q%]�zY���A�K�:�zh\&f	}�z��]�����%�/[���h�#�*�{�fp�
�M�����"�U��#�Z.����Y���r�E]gAU��tv�.���Z32���&6c�:G�VI�UkF�vT�mLJ�h�*��j�H�����u0t4������3�|��pF�@��v��j���B���Z����H����l�T���d5og���Ouf8�r�D�i5vWe�<�6r;GyGu�=&ZlZ����X�����uF��~�{���������"�s����}!�������R������A�GX�o_��7����wC'X�?�`E�}ac�����W�~r���<`?�eU!�b}�c�T/���J���w���$[ ��s�}�K�2
�:i

VQ�Z�94]�Q�G�2�FE���U��2qMW�����h�S���a��c�C��45z�+qjT�)R��q�5[E9j�r����U��>rU��_nf���y��������<K��<�+j7�v�0+��6��]���A��r=O���x�����b���@
L���2��x�Q��������Z}S�����}�����6����c"�=������NTK����)[����)1U1W��X4���|���^��G"(�(�����{"�-p��Y����3��z�J$���r�(����e4B��/����<�[X�]X��_�������H�~�{eWIUQ��7>�2Jk�J��^v�	�@,��]�I:�:a��=�G]�g/`��SI+��G��H?:8�\��A��Dxx��n�F���}�A����������4�<�G�~���o���mn�����&$.[���P1�#��@�6�\��q�v@���S395��l��G�w+�i�f^�2Z%�L�fu����En��L��9���^��&OV�����'+�����$
��^�'�g��D��"��I�He�"����������M/�F-�{�����r{��/j���jO���wv1{�A��{������Zyf���og�i�L^��>T�GRa_S�*m��%�}������0)�!T���hq���;+�(A+�����������N�%.��(�U���p��
��'�T�Bo\_i\��T�������p�"Jo�������$�XUYsE����R%��0����U& ��Pi=������DA��Z�
�]���t?��,��t�aH(wq���Vn�,`�8z(����f<?:
����|��C�����0���wEdq���,���u�e�����S�j�W
��rV��J{��!��C��T<Ui�b�>rqYJU*�z9���?���OVH�������������,+�nWl���}/By�0�-2��\����d������	�ln��`&�V�RNu�����yy����]O��	�B�+���X��/�t���������I/�W�E���
�t� �qEY����$��HPV9y��)0�j�F5{��2��,u���z����z���x6�����4m���)H�Lj��:��*aa�h8l4����L�M��&?���a��
�`+�5��mn�������rjI�#}�����%�t������t;�3y�;[��H'_W��V�
�u����"N��*^>Z���j9E�T����v+�
�O�/qP^��I�.���4:��W�������+|�3&��������`y��VV��������f���������0��S��qlP����)���yz��}��0�5����,����4�#��T��%zjr�B0�&�/����z"AK0v#���O��VB���qOMx�S�����=5�:������\��{���Z���hD��<c~	v�?+
sK������!�b� ��:w�X,����1"-�b��������|���������x�3Qc��,U�eg�w����4�Wz��Ks����8��E]7�L/'8=�JL�p��Q%��'��*������]���W�}������n[�`�Y�P�b��
�-���\$^�d�>�Y�a[�=�u:�g�>6Rb��t��m#�-J|�(������7�/jX�f$������� ��1�!c��-� ���ln��������U�����r���{Y���>�%S�h�^
GR���
�������5#�7P��9nI&#��������8�mYS2��:�Tl�G��O����c�M|
9>e)�[
�H5��i��,>��X�#)��j�N/
�L����A��6���h���b�������0���g�x� �l��s[1��9�L6&��S��T��O,V�d�e��QR��B��W�%?i/���vc�
���8��mQG�l����E��D���������i�T���8`q{#���ze��%Z�PZ��qh
Y���:X�Fo����5#]R�K�S{�
n�q�y������-�?*��D�**�j'�J}
*����P*�<�mx��	oT,��Y��1�0o��������*F�DD**Fj'm��VT�4T���b�*�,�� ����9K�b!�=����V���
������s����f�D�����c��W}��r���B���r���������~ug/����<9�4X&7�����8%7����`�/��H�U0!�St��Z<#o]$�d>I<���t5�OV�_�<N~|vrz�|k�����t���~��|������%q��kW�
��+��RI�����J���A����t���������u�aw��2��������c������'���:��2f^3����j�N������YY����\7�����]��_0���g��H�����z�x7Qi*��l�@`��H��>�v����Iy�����s�B��"���+'���������z�]��n@��SQ����)L%ZS���xT�>�:�����%�mTP�}z����sQc�Y�!(��H�l�+/��FEQ���n0�[z�ip�S�������E�|���c%Q~�5��y�_Tl�^��3��3�(�"����,~E����>�~�"X���DFI�~	
����_[����l����������k+�������`����+���
a������~	]�����[�2���^}�*�
VW�4�i�,����S�E�L�o��'T�����d����]Xkme����.����6�����7�3�����U����5��#=�a>%�Xe�)��3�:@���Q-�
������F���l�N��;�b��t����w��vS`j�������8�{@/U�9�37������VOg���t�ue�A�,aD�f�NqV�����Z9�����L?�eA#J�:����G������_J��3�]F��v9#/�%���l�3��]>��v�8#+�����l�3��]6��v�8#'�%����Q�S���������	��o���`������Y<�%A���2I��b��y���t�-1hC��|���GK����4	.��0o#U$�Ur^�i����bq���i'a`���<`��|���biueK��b�����~��U;�R�@�m��&������|Gj�����p�2B/�7�)���2��(W�����w�>~[.�>�5�����u����|g7(���i������_��:]��������R%
��������p&a���K0�:��p�t��-�,��[�(�+���b�:p��oGM.��:�rY���K��`��T��)�6���V��)�n���&aL|�p��4�^�o��_R�LI��~���$����u��u9��B�K	�3P�J���:��H�U���E�]��u�q��a�tT\$��\����Q�jiJ`�9��XW������I�C���XW�0���JX�LT��@�������n�����2HS6���%���b��io
[������f���Fn?�V��Cr%����_�����%�/������k(��g0�E�����K��O�T�1Ec�x�Vo]	��
������|���b��N��I�B��.�����*�������P���'�'�g'������U0e��h�d���L`��r�Z.t���r��\����
�vCw���n�n���J�'Sp�p�{��^��^��e�;~y���M�����/s'����;F�0SI��$�TB:��a�Mb�OB+{}r$s������y�*X\\���g`&&����2v�k��a(���8F��9O�q�+�@��)q��[�I��pE�S�)���]��Z�U��M����NG��6�	���3��V�d0�n���4K������nX3!sC�R1�ic�8�,��MR7.����
�)����F�I/����c,t�_����I������w���`��+��R�������%�W�I��;��V��2hI��K)��|�Q3��C)Gi�QY��k�y����f��@�Kk�U�������E��?����#|F��[�/ot���kz�{�-���������w�{�}���U�Kh#
T�������8\�����Ud�^mo)���2_�
<l���dWT���;������������S����iV>����*(2��%���mL6����S&sD�}�n����w,�?$%6#K��M�;���;Vm�	�}�r�^�M�<�����:���)^f���r�K�J�/��|��2NUv��M���;�/5������a�34�����Cz������l}=�u`k������L���Tl��x���2�����}K���9^G���t���%�m����p��j��E�������(|P\�P�-,���p����2�������V��]r������FN'-��KY��r.m�5Gi��X�����P�8���\Kl���.��fO��X'�9�b�h���������QK����+��ykc�J_���?�>	�ta\�J��y�L(�|�f%�@�Jlz�L�&n���W�L�j3u�D�U���z���L�+;mhl�����[����l��&�z5Mx95>6z��|R�S��g?�I��H�X�����`��_��*%?��-�x��?�m���7V���Cm[�m9�m��:��m���e���o���O���
����<�nc[.X��1?��W������U�C33���rid���-W��	7���F�����x�n���9���:��n��e[.Ue]��J0��o�m�#f�m9�����l�F�������i�rdl��m[��Hmp[�B�mY�p��r���+w�B5��:O�����KV�j�O���]�746zUra�����\��en��i�O������w>3���r.q��������rd��KUY���r�[i[�x��&�nO�����3b=�a�8�:��gK�>kA�5�'�Z��la�$����1[�~��C���}K��k�35g�Zx�la���x.=�n�}��n�L��\��;�/7d��{f9�����B���q���2QZ��R���8�5Lu|-g���"��$PV;�����[�}��U�D}K�������<�S�mQ�4<���XS�����#cb!����i�,>�����������@�)��Gn���N�3fGFs������7�V���+���Ei;���Ga�fRI	A1���C�W�[��C�
���I�����x�d:TJB���_��x��2�]��A-2T"���?X���,`����o�40�����a**��K*)���m���N�L��&L�*�r�&u���F������ON;��o��a���g8�U%Lw+�_i�4d����0

�t�!�4,C��
{�I�J��=w�e�z�&��	S�
��}I���s���z���<N8���@Di31�-�P�I
�PwI��V6��V6e�V6=�V6�V6��V6��V6��V6u�V6M�V6%�����^4��*3�n�r�S)^��O���wY�������=�r�om\�6�������4������J�RI"|���a�j���)�I��j�6�5�
����0��l��/��c���Fbt����X����O\�@���Eyi�M���^��z%��C�����E�l-��	���v$h��Q��
3'�V���d9�+�����&����j4��c��&�Z>h����E�H�@���������-���Ts���W8�6�]����.9���`���"���vm�6�]���`'�#��={(I���v
�+{jV}��8w����U9�-KB��wm�;�j��}2���������xx����=����!�6"���B��������W7����W���������J��C�������zU�n*��m�������j�����Ww4�Ur��:-�i�Nroc!��*iA���a����9�����hh��a�$_�g�J9�����VW���(*�.����I��J��l���q�
1��X�;Z5G���cz�1������������*H����^EQ���mUs�+�k�G����L���U:�@8�6�Q�������`����er�,��$I�x~����:!������U�)i�O��t>Y�*~���c=?>;9=~��s���@�	l]�����i��G�I��(^�"
n����3�p��P-h���5F���I����l(%���
�Qw�v��yz���U��?��?��\�UQW�Y�����g���la��5Q-�,�g��|j�.�L�A����M�S~s=�-;�Ad��C�l8>}q��G���H��C���~�1o1�2���0Jt)����G�����V>��J����=1�P1�
��4����L���j(�*�P�8Jc���1���>+�]\���f!��	�_�"}���Lh�6�g�]������Ylc��~��<+�m���Y�m���/��>�
�vT��k5(�A,��aF����S��:�_~aG����&'[�,����"Q[�,b���"{[�,���"�[�,����"�[�,B���"�[�,��WFGp���W�n�������yB���"X&��7	Y��$X%��.���<�s���F�Z[Y�vi�K�./�M��b���M�Lc�|�M���+�6�I+�C�7�C���^:�Ru5�I+��l�P�q��:���.��>�$PP�	T}
�
�������&������^7��������~�j�
k�b>�J�_�|�*��*�x����<���(�#���i@]�NV��5P��g�����^RgF��	�.sfP����?�gF�����.of�O����=��fF�����.kfO����;����R���iS��ne1��(�����������?'+iF�d�>�� `���x5}C��$N�T<�z���U���I>d�&d��������%Y�� ��Y���W�$�l���;
����#4t:v���n���Dp�OvG������u"���x�&d��	S76x���7�T��������F�n��w��m�T����3
9���W�f�D��J�ln"��]���|�C��l��E�P',8���#}���Tc��
��w��Z5������
` bS���9/�v�N�;����P��&WSP*���'�����n��Tz�WNXqtq��U�����~�q����A��(]��4������Q�]~SxS+75�y~S�S+75SyqSxT+75�x~S�TC����s���4W��v���F+�c�������l��~z�l�d�#'�wf��s�����<�T�v��
����|=��6+H~�Y&iJ�Q� ]���.q!���
���h���*M��L7��)T�LV���n�P�n2���7]�R���h�#�����Y@F�Z�QG<&�r������C����%_;�G�R�D����:���F�7����A��R*��V�*����Zm��1Y�a�E!"���X��PCZ8T�ja��d�6����l��oe��R�����O�o����q�9���l�8.[iy��H6���U������S���M�O/����8Y��1���2o������a�Q��U��j�+� '������xh�=,������jQc�1�����4���"c"c��s50��D�D�D�� ���BQ��HR�jLU�Y^y.sQ���1o4��*j��j
3�������h��*�8�hL~�����BcLa�1��N�S���)2�)2�)�:OQc��Cz�c
�1�����S���� �i�+�|�~��B�����j)$����-n$����eP�b����|���'��?"�A����%'�R�+b�ZC�u���(�Q���r=�`O����],�h�8!���55���
�4���gn#&�X.��(����9_�����h�V7�����jF{r�#6�F�EV���(�����N���i���2��fV\I-�*��*+�+���������$�O����Bh�%-��"�����v��T����wo�]W������\�>����O_Nb)w�����TN��.5�k�1���)>���Xr\#��F��u������������bK�s�,�#8l� ��������'��
TTV[���5��d�9�w���\���������^e������e�
����rCIr�����bxtC�P��`|�O�
���� �R�8����L�C�Rg:�~l?�����g��d.��e����������I��G�"
p�b(}�2�(����Rc�J���e?����z2�+7+P1�u�,J����v�r]�{z��',x`��*"T����c�<��.�������:�qN�N%��Yh�����Bs���u��Y���Bs���u��,���<N6s|���Xg9J�����
���3�3���1f���Y�i�a-���@R�$�e29R
yV��%5��������en��^�yE�zY�h���#�<sshuw����li�Y��sw�������+�]o6x�/%D���!�����!�e�v!���H����v�Ely�WE�XN��[�uU!W�C5��*0+"�U8���D�\17S�E\��w�e=���3��=���2����6�7z9����fU�)�*�T�,�#��\���RC�J
��ZWjF��J���,�r���]o��?i�*,6�R�k<@�g������i��Z�����c��\�9>�V�Vd�9h�#
�I�y�Z���i\g����)	���~�������D9�=��P����J�9|�m������&��������d�R���.��A��S������[��2�H���'�Oy0��������������vUe����"��(0\�Y�XA�f���}o���a}�j�r�2��Ry*�B�Z}�����U��������������w��a��Y'l_�U��	%�V�;�W���
��e����!���V���=E�Mx��SsTPp43�j.��$�
�yc����[O� ��M��z��>*���6`����Xs	%�����*t���a$!����?�|��.xh���fF
*�<��M-��Q��Q���-g7&"�S�>�,�e\zUW
��ri"3u�����!��Hf��3�dH���vpD���A{���c�e>3��fROyF�3q����'<��6�T��q-n����?�_�Nz���Yb��[*�Z�T��a��%���}�r�p�]��CL�������zi���	NT�����}P��
����_����
�$I'I��H�O����/�<I�����Zf@���k�$I�:�
c��1`����)�t�G����������3���w�$I�8pT��G��}d�D��'�CL0��R
�P'����e��<z(i5z����Q��G�����Vc�||I�;��o�9]���=O��IM�Ee�V�������W��^������jjB���`�^du,hn���|O��d�(-��+�_Q�>�=sU���O?����$T�"�l&���|
e���e�R�o�.��b������.F����[���X 7��I�SVV�8`]�.�������|���K��r�m�S}lt�S{P����I��x�?TK��Sa��4L���S�V�q��%��gz�'[X�i_����y�u�R����������x�|e�W�|e�W~�|�]��>r����j�v�7=��e������*�d\,��5���Y�&��)y*����[;ONO��N������`:h2S�8��W����I�&^N���v�������gd	��?{}F��/���V K�z��~��4����N^�Q���n����~���� �EG�p;����5*=w�h��VW��n��C�}T*�*a~�y5:�b�Q�����*����}� ��Ce�ZN�;������������W�@��A���|}rzF����`�w����cg�h�#�=}���M	�e2O��*��r��Rbxt��<%u�<~������ ���s�-����5���������������yp��
x�?�*��o1��CZ��=z0����k%6F��=�%�(��=c��lZ�_/��jbE�U
�T5�U
�T5�HUC'Uu��OUC;Uu@�	�:�jX���N���1���v���J�T5tR��"U
�T�]c.U
��Q���&U
-T5�S��NUCU
�T5�S��BUC;U
�/U
T������LZ���Y��5���2 ������O�����Pw=`�M��c:�p��%��N����|��r�&?�e8��o�P���}��WC�Y�Jj`��*����^
�
jG�����t�^
�]�B�����	��
��U�jk�w�N�U�P���R�C��]��aX#{ay?��Ns�"�|��o��}��t��F�N#��]Q�N(����i/���Ns_�|��������w����D�����5�Q��!������������W���nr�M��g<	@��jo�cK��9N����-�W��Tk���p�������S�����y���p$T<�9i��C
�u��~��[w�u��m�����T'�}k\����0�]�fZ�O��j�HV"0KF���k�7�k�����Ts��eMK�^�VB��`�����KE����8U�*�q�6
5�3��r#�}��Q��m@���}��#��>]��4W��q��T��,"
�"�"7�E��g/R�����a��j���k��>]O��1�o��o�b��c�@����}7�
���^��m@���}�:�FAU�t����.��!����}C5������}C7�
���^�*�5���'����}w^��?��i������Z<h�d#��z�v��y��Z>�E��]��,����^�P��R���}-z���������kJ�[���q�A���=|w0�����i-kZ��[������*�Kw�.���u���Kw�.h��u��(.Jw�C����,��b�q+��L���[��1}���PP�
�
�����{��2��'v����^��4s�uq3�M�TYGa���t5�*LT�����V��,DW���-�(���z�	H���
�+�9H�Rc��LC�uL�D���*D[�������?��$"���s����u:�b�x�e�#��;/�B�����k�?�)���Z\�1�����C�U�^��lB��
��ZW�Z\.+����������rM��2��#U��eA�w����N���X��T���	���JEf[����
��K������2�KK��..[mcqiE���h����y�x"Je��a����#������O��D,PS*mrq����FCc�;Wh�\�m�
��+��\�k��8����,Zuq��\9�����jv��#�����*�s���jJ��-�� ��A"�@D%v����"���;Wd��"���\����,Ziq�$J/1
��W���eG�w����R���X��T���
�9
�>���w����"���;Wd��"���=p..���Y������r&BS�����eG�w��Un�����J����r�[�����k�r����X$�^�|�"����O"�����E�%����QX"ZN���>���x��}=�f1!�����E�L0�����C����������[;���d�PD���;f�5C���X���%q�������6��P7���`&�_��2�k �k ��@�Y_�A���s�4�#�QW�~g����4Z����g��w���Ns��0Vv�X�se��
:�O�_�uD�	>�low�����8�2a3�����%F��/7�/��`7+��U��d��6
��f����t�$�r����#�.p��H0����Sv���({K�&�|����,
�RC���2~�Dk���y�����bC�����:��M�Q����0�G)�C5:�5�������'�/���x���MUM�E��i��IvY*��\���Y<�L�F��x��P�0�f�02&�*�0�
B�
��x�z��6�MQ�\7�
t�^��`bS��cz��"lr#��M5(���+!�Qv"z_R:�x���t5�8D��)��GiJ��\���t!�B�C���B-�QdF3RCe(]���+�QmJG1 *���"��t�im���n�S��,��6��Ya�+�U5l�A����Xq�
������tISy<a�kR���_��������4��^.�JQ:.����"9+�"gEf�,5<V��K����L`���F`��B��r��2�MQ�\��
t�^��`bS��cz��"lrY��M5(���w��e���F������N>���u(]
�Q�{J��1Z���/v�(��c�@�t����
L�tFiS�:q�;J�	L��Qg�������Lk#��(b�:E��r0��o�1=H_6��U���N:j���a;d�C#R]D�KJ'O����������?�3����.M�r���R�n��p�R�!���F��&n:#�e�����O����3�R�pPn��Y��)J��e��Wf9��������\�#=��s����k�%���Dl(�P��'������P�`��1Y�K�&����a1z��>)_k(�M��8��0�?,�S+*{���b^e/N��vW����b�eo��hY,���^�	��a����v�[��^�/k��afmo���
��n�NKvZ�s��Nk�AZ���5-{���hk���h�a�POZP�Xk����e}�@������9�j
@]Wkz�
@������9�j
@]Wkz�
@������9�j
@]Wkz�
@������9�j
@]Wkz�
@�������%Vk��1Xb��-�i�NKvn��O��?{��6�����O��=m�X�H�d;���^�3�6q�8Nw���?�(�v�#KQJ�^�~��$^I�%�wG�(8��8�H�4�2]}
����|=���Y$��$M��yL�A��Ui�������������$M��\�`A��&�	���M������[����Pl���t<��x�=���k�+��0�'�g��u6An��YX���u�a��{�=��q��<+���]������/��������u�����v��In���C�D|}c�T��6�6���b��$���xGA�p�'`*�`��4X��f���108�������w;��sk��������cP�T��������=�e,|��@;Z������
o3J
i�A�8����b[��A��8U8[��Iy�h������T�HG'���8�h�S���x��'�U���8U;���Iw��5NE��Y�T��('�9RUp�����#^�W)�����5�5&*o�&������3K`�!Z�����<��ck:����Xs����k�=��?6�H������zX�>^��+dg�Y.��� o,�Y�n��N��#t����\�E���?"�n����bg���	v��&,ym���,jj�_\��*���5|�1�������&��k��JC�
�=U�4L�m����}�N��d�|�Xj�L�>��K�c�������%�p)c!���Z�+�����B��	tQ���-����Zi��&t�R\8�X�2�n�������O���z��et�(���)�j�Zv��x=!��)����Fb��&X�c*0	�aR5qjB��1��D�0��1�	����aR5�yB��1*0��tr�����+���=����k��5Ch��o��������y����N�b�p? ��r
��6�6�Z��v���K��:�'�A+����3�^�w�����^��
Ex�a&9��
���!�a ��
�h�0@����xj�W�L���O��/G�L��R�b���})t�s?��3C�t�s_�A�~�]-������2?��C����d�`�KRAj��0VK�7u~��G �s�[.��q6��7e���,��t���W/���-'��P����-��A��p�o9�G+m9u#�&�w�-',�r���:��0���
��T�G �
�d�S	��)b]���P��B�V�[]���B�V������G��:����:�h���"�L�[]�r�+�f�j�E���Z�C[u���Vg1�����`����f]��bLg�������g]������Y�urk�g�Y�qa�O&�Yif�Qg]�0V��o5�j�#L�V�:)T��������V���P�u�^�{]���"�^����u�{������t��f�d��u���a����v�U�G ��B��u�^�9:�rI=b�~%w���N#
�H��Uz���x��U$����r.�������;P%���0��a-�gm�?k�@�g��9/_��K���������]/"���I����|���	�Te���5�c	��T�_Z3L�_��/�2P)-7��/i@�2?i���K�K�@��hj��4ouN�����
�y��m(��P9o%�\v�j���r^����5��0�����b�9��
_�do��-�^�>o\[~D�y�����Tv�JCS��+0�0�����mc��K[�@9o-�������X�^����8o���u3.N��M3o��[3�o���f���<��`42o�{�
HX�Ao6|i�v��ms�m����Ag3o#q��T�m$���r�U��e��z�������5����������6 a��a���-L�����f�������Y������8o���uQ3nV�P3o��[3�s���f6�����`42o�{�
HX��o6|i�/�����>qV���m���"ND����p��_�!~���y���%��w;��G5��p�"����*����Z��n��w��o���?�2cT[�_
��~���G�(j��S���T��zn�
�qY����l"�����6_����0!������r�eS��yQ+�ONK���s\}�kI���n�eI���/��#KL��s:yN��W/�96ulA=h���i�P��7��l�S����O������]�M��Q��Y���vZ6�F�Mx2�?\��{Y?+T������1l���z�
��������]��[����$� j;���^��j��J������`y����#����BA�</{0,�����$�N%E��}���Uw~P�o1��c<_o!w,LiY8.�r��;����x�	�Q.�)��m��C����������+��0��9�<*��6�O�����A�c���h@����Q�d4���8���jh6�c;����>��]eb3�k��!��{$��4:iv����g����R��Dd��2����9��'�Ou:�'vsF{q�9q9���>���=���N����1=��`�����5;��Q��h�xiI<p3��Q�cg���s����:��';���`t{���{z.�;��M�z���Q�������
�(.��Eu8W�3]8��3�7W|3}7�z3�V���Z-Tf�N+�����\=qh'y*� TJ��n�I`�����gL�I�&i�4���O����V������_������������2��_B���8��@;L�'��`������]��H7�,�G'����Td��Mx�4��h��0�m,�����j�dj����j��X6v+5�a�s/Vs��f�������fI��
�n�_�����NH�������/�1�����+���G�wV%�m���	"��4uH8>%�>�
���
�	b4j��p|^x}<3����p�V������4@��J*.�:vd��2�����x�w��������gnr�(�ro��2��M�[�o
�j|��Q�4
T
�(���
�����<6�=c���B�����z�x!�D/�oVo����e�����{|s����.����x/@��`>��z
�����f��t��G'�$F����W�I�
���}���U<��BF:�`:��C����r5���`�5�M�x:�.�O��qW�~s���������W�����4�����?O�|<�����n��
[	
Zt����<M�t�]��>��5 
���/~"��a�Qw�����t��'�Y+j��A�+!�W-��>T>�$|�$�����!Gx%�C��J����(*~d����s����K�x~�,��E�"�=t�����t6]}
&�y
$�|��H�c��2�8����,���
~�\�U
��P�-"�N���}��~���yx�w��w�=�!���K��bP�I`���R]A��B�"���m�.V�"*!��g_!jtyx�A��tX�b��vJ�U���CQZ��J
f/	;��g<b�*$����P!_P��
�R%��&�L�����Cp��*���~UN9N+sZh�i��8-���u��V�5��o�k�J ����['b����?���������;�n�5\��>!��}�����S��
D�	��;����7�X���������� @�y0p��O����Y�.m�3������O;~����m���-M��������w"s<�=����^��Jqw6J8�96�].�-��Kvd6�l�g�./���"�����C���'�������kh����`�
&���)��?���z�QTG��
2y
�ciV�����#W�z}��z'����������$8g���:�s�b���S�s��jq&<��A]����,Ky���'$�=Kz7���@��\R��$g��&�i=�e� ���Y����Y�a���
B���2(��"���8��1�#1
��Q�g^��P�����+���H��vl%�{����Oi���/�#���������}m�L"���O��(��	��Cp�g����|���x�O�!X��M��>����y�G>���y��D0
!L�<�*Zkxh����2Q��sJ���b���#�.1+�Eh��H���N#�P�QzK�O��u���<���
EY9��[*�r�����V\�ZFYx�&���?|�Q�$������d�Pm�vY"�X�JE}�G���e}
��Ai|Vi��:���D����V�Z�j!"���2��P/��uZ������/�d��(n)�!h��|��%(ne�rE���n�\�y���8W@A��{h�vw��Z�c�r��r�cyKu#��:�e(@	*�
�.�����'�c[+=��KNGw�p�C��-~�-�G�sD��QIe��w�o�?�����D��~Rb��_`����"b"�e6��
��&���/^�S���H6]�!��U�N9Tz��t�CQ���HcbL��$��a�����eS����
��B���UpX�e��2���_0��"[	8����e#k���V����:�%�`�����UEU5[��P�T�x���T���`b�,ZQ����_Df���t�2X��r�<"c�1���A��,ur��V
wE�V��Uku,�E�W�[d\�E�S��Zd�M��t0�z.��!n����z�L��1k;-����#�4H��y>����������������1z+��AH���>I��^������H�J�F,�%����MQ��q�����m�a����
l��@��I��@p7��M��X���F�="L�	��g�����'��.*~P +x��k�Yw�IK�s$���2�J����Z1;F������%BV����� �r�L�dO���a�N�81�E&l����}��@�������X�"�f�2��:�T��Y�N���V� ��{�Qw����C�4�(��lX��
���v14&�6�x�Jgq�g���)�Xy��GrK!�6�r�OW�)!7S��P�b�<k�z��(W�)bIn��ay
�qH��s/���L	�3E���c�����[3�6�r�OW�)7S��3%���)2�UfJT4S���qH���������1S"�L��#^���z��~�[����#���i�`�-M�?��)����6�
����M�I�%���������1D�?���.���p`'������t3*:
Y�1c�?��������q�$�3{������0��1b�H+�J����?FC�
N��0��U[:��M��uSSl����cM����d�+J�����5t,��d�,a9���11k��g���,	���w0Q&�x���)6v�l��b=_u����PQ���r����M�A�<��&��"a���������F��)����������#]�B���/W�$�]�Q:��]���ZLI�4���,1V�+�/
�C��q����I�y@b~���/|�Y�(h��>/��0$�(�6�z~�<����{�L	���R������'t���=
��2[��+Q0��QP��pW��E�<��,���r2�*/�_97����7�r�q�M)W�j�6�QS�d����rw���j�M8|D�����T�r}Z�{�R�8��z
��a���y��Ep�K�Yj�26ZeA���\\jT��-,�'��f�
h���p�Z�N�����:����
#6<��������~m\a��E�����W+O�<�IQD^0W���'N��.>� ��QC9l�����.9�6� ���l�j���[���:�=���Nh��\�%����P@k;��kk:�����:%��������O��Dvq,��7�q�S@�y�a�B���j����h:�5y���4C��>2��0�"T���1��A���WQS��!jj�����`Uh�������.cl�qg�0�e���fQ<?���	v+H��%r6Q���S1w�*��r��e�u-��8_��@�Y�UY�a|qi>]���b�{�a�Ds��N���w��!f����c�W�����(Z)�g������X�t�\h��&���6>����������h8��3T�j���qW���}�=�d�3��)��"�����K7�z}n�nd�.Y����4}V�=��2�f�beW{��M<c%�R��G�g�h�+N����A9�b��M��t�
,�����,�N�Zt=@�'�gL(�����z8 B����k�p��z8�6�����!��&���b]�#�2K�=O8�	|���,�$w�|�*������<������.�����e��IR���d���L��,2�,������(�n��;�8��<P�:N��0�=������R�l����-�d�0�n���3�zc��e���9��lrbP �B|��L���Y)���6�2�$��iU+��	�)V���>��f4K�Y�Bs���,�D���G�V<�\}bXl�Z�Y]����D�T*q��.a@���2�vS��
-R*VG��>Z�0�
�BA'��`������.��
�c�&��@13,.S���������k���n�%�^��SGe��f�G�:��[*���'�}d_J�r
:
	lg?�[
-�������[
m��7Q_2z������B�2�M���^*/s���P��z�B���0�w������~?�	\a.V��[0�����1�"A=F3>�wat��:�2����|������K��<�H��{��s��x�g6A���r�����L���hOQ�j,�H3�ES]|�����q]%7���J�c���(�6/#��(�o�D��<���b�\��xB����Q9{�,@�CS{G�!�.z�}-��T��	�N@���Rcam�4�L��pf�xOK����[abl�c[����'�3�fb��J��B|�s��9.���-��O���x\���>�j9��!l��L����lv>M����5���`����&����b�#a?�CU�gv�	�yc��f\�M��Zmf[B�/af�}�<\.��\8���e}���S����`{�gv��������"6���gv�jf�s�~f�tfk-��bw�,��Y���l������@��`=5=dZ�Z-=������G�L�[E�C�%q�:,H���:�S��Y�3��1����50�j�`R�O���'��^=H#5����S3)XMHC���c�=��v��?�����OC���Q8���Y~����lh�v����.��7<t����3\@�?O�|<�P�P,��������I
��}X�����G�>{(0��<k�9���2I{N���������*�-E��AT��0a��K������1O�GY��WNc�L��d�z�]/6�	�*F��uN/�N�����ihB/�P��^��q��h=�x�0�/[z��+�����>�I�&)����6���(]�����*�d�Z���:W��t�z,	>O�/�F�_�Z���Bz:�?�O���x%�z�E�br�F����h������.�=g���KNy;Y\RwD{U�_��}]��J���J������`���G,�$u��o��.8a&���-+PX��N�)�������5����5odv���hV%
�$�
����}���c���#�ITR�o�C�0s�b��#&�l��������|�X�o���i��?`�g�����#h�������A5��QyT���.
���@���A�\F���Y��{��+��W��N?S4�����ik�l�7���W]�D���6�����m�4�����F:���+sg�foVp9���<��kEF)��E�M���`��@�ugKB�Pv��^$k��"���g�Y*��zy��#I^�p�{9�j��'����#~�������U6z�-���^qCg~����.���g�G�,��PTW��BC#��GCXd��U���5x�d��������\J��r[R����,S�E�*��`[��m�hM�V��Mk�y���QW}�C��o���*Lk�
�M�^��u,G��|j��N���VJ%@�N8��zO���;��(I�6%�)�J�&kY0}�w�=����i\&U��$A�����K�X���`�X��R��bi!P^,���Ub�
^2�$���l�
c���C��u-���v�n�(��?����
��Qs*������Z=ITO���F���"L�����l�n0�I�������]���V�����QE������D�L
��?��%�<R�$V���6$�x�L�t&�W�_%U2�|���m�J�|��=Z���#y�C��PjG��G�3����g�b;b
�y�Z�,��ylK�(W���jY�Z���B�J(�L�'��3�jL��[���p>e	�.�4.�>f
�[������Yp�u�k�j{A�
�G/V�k4&�Aa�*dM�F�)�����5fjn��
O+��6V�n�T_�����h{���&��9�a��E��k���qv_lu���+��������6s��������=�2I���~�����t���?�hns��������|sy�0�����	�ca�kl�����$����0����������o��pVK��D=���}d{1���	�V��$�t+�bI��^4������Ml�s.����M6-�X��M�!�4���m���6�t~.>>�=\F>*���0������Cq��U�p����*(�9�����Q[7�D&X�	L�16X�/�(H���z�.!nqD���$����,&v�^V��3��&��@m�L�H����l�x����#f "�Y$3=�[�t���p�k�1z�BT��N4�S�e�&��`��6�8W0��Y-�?�����Y.�?�Xz
������yv�^���6�]��	/���W�$��h�N�u��}�>���A���Ub>L�<�Fv�$��.~�S4�?��/��x��vj}����jj�0�����6FD*�v�[��9�Kr�1(���$�t��a��$���O�q)�����>Ss����M���I�� ��>a�p=��'�2�Y���0�A�a�U����
������rr�
S�w� �j:Y����V�N��P�b�q����j]���f��Z���Q&����VB~�����;!&T_)� �J��D������d�o�f���&|I���~��$�JQ]A~D]y����@�Y�4�M�&�KQK���[�)"��#��P�bh�#\w�6�X�/E1��eK�����s;���F�
h��X[l]����F�>��eWBk��V�Z����\���Y�����	�,���N�������v�R��'�#dh9	�W��U��?�M�j��E����5��
n���H.K��iJ�)�^_2�xs��~{�2�^m2�{�j������)�����V}���8�>�A c��X�Q������ B3�r�;+3b0����</S=����c$o�PH���(T����{]��\���������<r���
��y�������[�������c��F�/����(w�=0��u'��=0��
�=02��R����*�V���@�.�F���E�w��8v���.�s�����Y(����f�t�K(UV���"����
�����f�'�y�ZMH�g��-^+Avw4��|^(����5n���~���������h	����V��B��H�)�	4���V��\&Z�����W
=�	��*NAf&l[���D����9T�������T��
�#Qx^������|���N��:%$��!��P+�
�C��W�T�>��p����R_�f�������R����B��(^�s&R���
�8gB:g
�.Q�
N�k�4��jqU��v����@��	�Vs�zO]���+n��{�����]�PzO�e�0���*���y��U�b��NP���5��RxP�����R��>�D��>+��A����Yy��W�����vM$~�J�*���^	�].,c6F}{��`y5�A��,���J�1����G��O�p�J}�!Vd�oZ������V�kH��]x�+8^��
�Wp����O|y�+8^�y
����G��0����&~��>�����CH.n���Ep/W��`����$H�w��y�J����yyQ�����!P��X�)Wb�(0*����pc"�+�������.�h�Q6��5eO2g�X.����u���}IB��Bd������|�����8�#�d��i2�!3�Zq�P45���F�XP)P-F���m�|���r2q�4��Ie�b��e4�%pY�d��P��I}/�G2%�h���6�{�T
��e���I�������5���I&��z�L��@X��J�e/���7p�e�/�������/S���������"#^[��BQ-+�E?�2�H�_X{�/2K�ew/�z���X���b����87u�2?Y�����6��I�������l����ue�?�DD���r�/��-\�rG�%�3�K�2A��������0�7���e�:h
�L�EN]�M:+���B�e��&&"R���e":�WM:���_bzG�K�b�.�����3n�4WA��T�4k�r4 k��`��������F��4�l��]kX�,�����������Rq�3�V��2�E�+�aL�I�����Aa�D�
�W �S:�>R-*8����g�����d�
MV���>t��!�(�+���
��d�B�Pm��_m*S
�1SRk���B�Uw����);�Y_�
4,f)]�"�
�xJ3�G�E@'z��������yEw���`0(�#8�l�x���<��!sA�)�(�L����k����]�[�\�H�������F4g#��L[��������@����9P���5KF.U��)g�j��B��*f����_5������FPAp����,]&s;����|	���H�/��;��5�h;/cLP�l�p�L���)���`~�7��
����l�k,��e�Z,�Ddh�,�Or2��a��g����?���{��l������w���L4j!�\,J�������[C�-�v-Bi������������p������
t���b��_����W��[=��k�A+1��X"�Y�P�*��u�MI���wd0�#cL���q��5H�,��BV�8��cl�S���1
�8�X���-�M���R?�'VY}�'#�-���e����7�gW�V���v��w����������a��:�p��t���U	�Gn��(t8��+Y���L���/�����{A�r�"�;���I�Z�6_&�z&�c�[��$��3s
P����U)8*��A�����NC�}3�T��YUmh�eR�cR����f��
���$ F@��\f, ��9�q���hr�*r��/_�Crx�@���_��L�u�?�N`�Wq����K�����w$��$��L�d�d9���2�!��f�C���Q��v�%="Z3�J�\�_�����kw����_�/���wz���������K�}<��A����3����Vqy$Fbe��@�����{[��-�U
�K;�&��v�x�\���Fi���.����b��5`Tg������c�����x���T����v�X�C� �����p���t}���o�,)��3`)14A�������n���;�J�`+�K���:�����6^%*k6z5{�y�������F�+��K,�.��[5s{��}����M��a�������-7`����7Z��`�G�c�[#u$���CLCb.x�sYq�dY*E�bV�(Q�N�\��_&�1�.]��.��@��/����e���t��>bc�0'��r�}q^�Y�1~E�Cp�����W$��-��W)A��`�������n�����r7e�����1E�n�*Y������:i��Z�hL7��O��ywv�zz��p7�e�~�{�d�����,����'���2���bm�]�U%�^
g�������������0d������(w����n���vN����U|)6j|��u��Th��\pN��2�jKq�
�W,Y�&`]5�N!����Jy����P
t�s����82D�m�*�Q�������V��Z���Z�����P�6��3��g[i��~9��hT.t�'���U�d�*yO��g5.?6!P;:��,���������:^�L���Pw���@�\^��
���#�@��
��W �[� GC	�?Y/SXp���?��x�go�{��f�}���A*
r\)f[�7������_��7o0)A��!}~���:-E�wpvy�3?}�8�z���2��Yu�������S�`J�x�������0���<ry�!��/�'�&l��B���w��/���$4�#t�n�_���>������?����������;t}`������k0(���0���AS#~�����W��/��W�.�@{^���t��L��p�^���d#b��U��PN_(���&$����`��WJB����xu��7�/�<Q���=g,��$\�oUd?�%���%F��H!=�\GR@~=���Vn�5��F~�Lb����8H�2����*���w����w�qZ�cM��F=%; ����t�:������l�a��$�M�7�<��}��"M�s��'��P�9o1��c��|�{_�h�A[� ��Z�Q����������I�,9+����5J�t��k8���Z�2jT�B�[-_��j����}9�g��N��d;P5c�i�b� 4�gS�T�-���7�R��+��.�L�Us@%�v�cm}����X$�L���	�j�^/�
����x���_�!�e(�����3��_�R?�/�;�#����X+�b��t0r�����9���QQ�7��"�P���CE#.���[6�S`�Ns�<+��2�O��
jY*{�N����/�H6��ls�Vr+�3���{���W*�a��IYfV-@af�8hh������"1Q��LN��"�%8Z��^�i2�f �t{Yn��f������^������~���eE�HB$Pt�n��1�q�����A�/�O��E�R.�18����,�Y���ER��~P������O^~//?�����G"?����@�������O:�wO~j�~�Ij9���6QJ�I?6��^������0%��)������	S��coD��Y�f-����C#>h82�����)/HmO��9jW�(/Ee�$E����2��E(T�w�X��RPB�$D���*A*p�;
������zp+���B�\���=>��:����|��g������h���������p�I�x�^�3�!�l��`�jY`Q��]����.p=�T�����9H=>�z�e���;|g"P^%���Nh&P�*	�����k��E�F�U��b�e��rlu�:7-�j@a��H������l��,���u�+�u(z���q�3�]�_������l�#�Z���#��A��a����A��M�H������<L�Ut��<��a.V�8�^�����gT(1����`G�nPaS�*��
X������{������tW�=���:y��g_1���S"��&u3��!�����2���P�;����<n��T8p�z�l3��oD�.����u�`-ie����[8��zm3�����6*w��V���6W�p?b���Pf@�@���:��,��M4�>�T�3����5��S$HM��A�f��������[���,���:��`+���|���|���m�O�@��-|{9$�!`�*	��HPm4DB�a�xL����-�#]VDp:F�R���m2F5fNEE�� &U���4��~��
�=z���>����K7�bAV�)N!bo&����q��J�qKBa-��3�������&6�y�ho[q�v�Vjgi�o����V���J�/������uFe��?��XX"��F�${��=m���6jdO.��������!N�����:V���������&gW�e�z���N>�y,Yl]b���w']�:���y����W������������k�B��	��a!�j���3�!���w�������n�����h=#"���<O��|�1��!FI�Ca�xfw�Kvwq�%���k��F)�cQj��i�Q�WoT�����Q��A����4�]����S�������g!W�(k]���q�k{���cu�N&�A:{rtN���
��{B9�!���X����v���LB �A_t ��Z1�v
���fcdDj����2�Lw��<&�i�p�:[���"&E�+j��S�Z��i��������E��e|��qj��e|	�6f�����o�����"N�xyCu��z �=U���2�k�,�����s�"w�,�8�%�E�IoQI�f��������`n����	��}���@�m�d!���y
+^v+7���`)E�u����/���W��G���}�J��tr�Z,��������Em/j{Q[��
yQ�\/j{Q;��D����!�mT�vW���D.tK�����U������n/vk��b��!/v����n/v����%�b7������7?���o�Jc3r��h������n
�^�V4���Rp���������#^����Q��b��oc$p�MC!&&�,Ws
U.��ep8J!��b�vKIC�G�"�G1����P��iz�]��)�p*qF��BY�#���3#�L����|�J�o��RC��>�����B{r��=����\�A-u6Q�C��MR:��������&BdL&`E��W��x��J�6h�j��^m�jC,������1	����������B|Lf`5��W��x��J�6h�j��^m�jC,����
��1I���*�������k�r9(�y-����W �Q��W L��
DU���+^����=4u�v�.�ImL�P�#y�F��������Dp�W"��@���Dx%�+��lL�������h��y��r�X��!�w��������U�����`�W���^^e�*�<�2����2�O\yL�fc�3l^Qh`����N"�<M��cr%7���"����)<@7X��Y���{��<k�D@�?��4��#��"���W�������t�����^f:�\�
&��a��E�9��9�Db!������Q�9C��	������	0�������
����+{���nd��@�l�����{# ��(�D:����oE��!���+�k�_����~�A}^���0�\O�^uP���(;��.+W+�	`w�I0m�(�-��I2f�M�E%�P���	?�"P�E%Qh��H��R�E%;���3A�P����$,�|?��B��,	Xp@U��������P�r"cd���e����|)���E$��U�C�B_W"*��*m����u�a�gbQ�XV�l�^�2�N�@^�����}i��b�C}�K"�q?�0Z_2���Po<45G[���
��0L�"|o�-����x�N,2�_������C�����,&i^�p����9��i�������bw����^���uo��_��������B��\���2��rh��'%@�����c������_��&��@	�p.p��W�I�%��������)���������)�C���a��]��BV-0�~6�A�W6V�5D7�T�~�9���B��n���*��rP
EEu803��[���n�)���[]���L����f�l��V�m3E6Stk��W�
�^������^�mEv_�N�����*���=�KT���
��Z���w��n8�������Dt�ydh��]��P��6��D�uh�(��+�^���r�������Hf7��@r� 7���7%����%Vs�@-���o������4�B=]�R��%���:�}�)�s���:�}*�x�G��<�/�4rx��2�
����f=�@�<�MM��}��W��U%��������*��TW����������3���@[E@;�.��'��Go��n�9�j_F��
E���G���/��\������W���h�	�
�A>��},����&��E��Z�M#u
$��A9eK������s��C�����qQ�YQ,�(�k������3��{Z�&�iDi&������m��re��	�g�-�l>��9?���1�������x)���|���o�W�����.���*$_-�k�lr
���J��A��
�d�e:�^|9��-�I�v�f��*]���\�U�)3(w3�@��.��k!0&�M|�1���)����3���c����N���N�1/F��6:��Mm]����`P#q���`C���:�f�R��`)����XD��{���
&�!����@C��;v��q�1f��x���2�M�^���g�_~9�<�#�28��[�����o��w�>f�� ��}_� ��VvK�VK&K�!�Vo����D������B
�-��?�%R�*�X�����v������BW��FI����@�����e"�����K�2Ve�F�0R0�
����2���1kX�+�W�/���&cVQ����X�*t�)�i�UB�0��������l�*�~���/����B��������.��j/w&��.�L"��0���p�"����m��xo���=���)�(m���4J�wh��7��TQ\2��m_�w�� �G��[�6�Agx��j��$�)qz{�Ln�k���d�@���d��l��HZ��]b�vf�~$V�]�W�������}����������IT�	^���`��0����D�}�r�	KJ�r�0����D�p�g:��B%�,T;9s�~����tV��d����<
$��?:�9�
��Q'�.��D��Dd���"��� 3����"I88�����+$�KZ��L��*k���&��$�W02��Y��S�!
F
	�m���b���lSe�vUB�Z��h�C���NF��v��4�v�TK����b�"6&������� y$�(H��s �
�FArH��U f
��@�����W��o��;;E�������2H?��c2��ib�^Pm1��9�P������/`?k����b���VT��WW�k��{�@_Bz7��f���7	\\�m|�i�|+�]�u��2��z������^�����
�������~�r%?�@^X{���6)�
����yW�|0�YC��dBL���@]�G��1��
y�+������q�U@��F�������$*/�����U�����_A��y^��r�����WB1/�y9O�9�+�����T����O��������/^@�������g/?�������6� ���������<h�
�,�C����tfiy~��GQZ�e�����P�xqv����ets�����W//>���m���?_Z�#����OdD~|<7acO^���_y����}o��G��8���=L����,������A����o�9�3����V@w	!���E
��C��A����������|�^�4<��l\'7�9X�:@"���o�o��JO��.��R_(�."��}?8�G��W��
^�}�����o�_�1�������G�4�U�:���y���u>�pSv	#����eX��~�(_QN�~�w)�p��w)�p��w)�q�>'�f<�����R��cA
!��,����*�d�Lw��j:K>'�C�b^Gv�������@����fqJhA4�������P��C�b��(���WdN�:��1f�1*�#����~�� �����~���q����b]y����%bn���a�[�u�6V"w���t[u�Kf*1=�E������fi�L	`d�=���5+�e��	L�q��Lam����|QU�"0�N�����4��dS����/�v�����!0��`S���������a�+ZF%KHPF6PF��,<hB�r`�@��=�Hk|w���R�<!
�G��c�{�D��	T}�>~��D��������t0�������i�����-���YS
�����X�u������+�����^�u�>B�fD��)����I2�#�G�*}���V�Zp������r^����W��qA%<�	�
��GES����P�2��V�+��_�c�/��anH��������qrP��xq���@��0����|;����M c��Zb��;hv�$S	S<��bF���x(Q��-/y4	���Y��7y*��E��TD�HbaD���PkS�.2�<#����� ��Ts�L����s����;LT��TN���9�,R�oaZ�A�lH�K�5U�*�g���&����Fy���,	�<y06J"t�II�������b�,I��'�����t���6O�>g���\r�orGl?�{AC���g�2z��8Vq��!Y��U�@Y�c�����z�5%&s��^�h-�J����D��QWjJ�M_��FS�;�����y
_%N����G������ eb>����9��9J$+��'(�e���noU7��b/a����FD3����G���+#Y
�����r�A
�)_W�6r���vsR<qlq~�5z��>
z�J���c�#nD��K�T�5\����!������`D%��R������%�e�u������R��^�K������_�
�)���60��m������^H�>Q[+eJ��)�r�+s�k���9����;}I�`�4��U1"8\�lL�ag/vM��h4�\�elH����Sl�P�<Z�"���Ji��Q�M�bf������oD��h'a��C���'�Rd�J8�{���u9��P��%��x0}�wp��>wr�rz�i�EX���D`��3�'o�����7�x��7���7�xc��7�\�xc��7�x��7�x��7�����U���&����g����4�ebG"��P�x9��pW{@���i��9���Dk����H��p�J��A��6~ZI���(@O~�������r����3/�X�:"�F��������p4�.:�_UJV�N��geX�����p��z�v����'���[`m��`��5W4��e:
_�
PV����dgY�������k��5���b��/���	_���zS�+�|]�j����(|U;��"_3&I+��������k�	.��}��R0Tf2M��m�S�LG��g5K�������E;����~,�(�@��.�A���Z��@@�>�����T��
i�>�	mr=����J��R;r�[$=3�������Nxf(�#X��y�Z/���rwl[��8�Q1�fY���A[��6#�ZTj'�Z�23cx�D�O�
{e�c�B}M�R��/�[����!K����c,����v7u>�/�S��J����;=F�Ju���US�\�5DQ{����YC�,!U�`7���WSV�;�Wh�������f
��nj/��qYeI��$�t��J�-�,��F+`N��%�_
�7���,�M����j�	�2I�3��/��H������&w�y����`w�7���wc��$n�F�vCV`'V�2c)Ud<qse������I���������
2��!��	����fc�&T�q�C+{�q|�c�6�6L��F\�>�*L�(;�L��'�,z���_U]m�w>�����m�&;�H����������<��f�V["�z�V������l]�r`�������0��]��l*C�VmU=r6g^���5�J:��6Z_o�����*���J"��XZ_�$~�����H'��-���Q���J���Z��+	���R!\�s�M��L0�AI�s�Re�{4l*���aS!�������=$�}����!��C��O�'������	���c3�o*)�Z0Lc��_�����67�J��^���W�w�:`�	��m4��������}l���k�����m,6oW�$]B(�'_�?��8����?�������?�����f��8�K���_�+ E]�������W��������&��s��i��q�t`���D���H�m�&�������S��?_u2;'0���y�q��"SMqI��
L������	;T���'�P!%.V\��A���@�p!0p�b��PB��rl1�%�N"9-@2��4��S7��)50����O��EL�R8uT,GP�N=�j)US��E�bO;KX������d�� #3���$��XFd�\�L9A�G���	��J����x:�W��<�7'�E�"Y�������e�c7~&l���	�To�Z�����P���M^����K�3���n����U��nz���2����~�Vg��c��+DAl�9?�.�MyJ����y"�d��&�D[?�"�Ko��p1d��K�f��8DDH�����4�J��N����J"��Y������,�#�J��b"����&�D[O���b"�hF��&YM�RZ"�pFD��
�.>)���&]�~5��q
��mP���
*9d��s;�y+.s[�(�}��>��PVq��:��@i���z����V5�=�H�sEXC������B �=����b;���`����3�r�����W#tR��a���/�J��PUJ��8#7���57gOU�?��{�����U���q�ji	roa�V��������c0����V��>��G���<:f��Jl����I��zRdT=)������'�Y�7b?��*H���;fFo4�F������1����1k��X��7{�q!��l\eM��q���x@��0�
��y�1�]�������Ysc���w��j�Gh{F�]���*���%Uu|�l��.oI����z���S��a��S}|�#�����>>U��75� xSc���S����PzCc!���O����E���Sb��f��M�����YX��Zhwty��U����bm���"�>V����XU��cUE������^>V����	�JoB.����;`jW ������X��!��K��@��-��������VY�kp&��V��3�*��3�7{��Q���U�3z+��W�7a$����h�����M��9���!{��eoC�����9��p�sfK����n��������h��_�R�j$E���U��j���8�k[vbls�w3�61u������w������r\1:7O0�m�����-6RZm[`�v#R;�bo�r�U���B%s�+C��	��x`�/pa�i�;���Z���h���
cSm3SU�[�Rm���9���\�di27��7�T2��z��x*wl�:�
:�L96F����J��b��p�B�M��>o�q6��h�2�4�������Xb���U����$}��Z�A�>h��v�n|�b,�>h�-6h��A�-��E����-��Es�}��Z�)+�7��Zo����	�l.rQ�����_>|�Z�}���j��>|�B�>|��s�Xp����4�������K>|�4���A�������
�pT)�>�q���F�76������.���}0�f���>��3�`F�X�Nm@���}0�f���e��`FK`4���>���o4����!���zx>��{�18������.o���oo���oo���oo���oo�.h�6 ���oo���oo�.C�[�-���[�����o���������So�������/�����4���{�H��`�e����t5tZ��Y���{��<ke����l&P�� �8g�5g�V\
�W�m�f�����^p������`��cd7�_�B�<�i���L{���=��T�Eu�m���p�o��������nK�|	�/ji��Hg�u(�
�5`(KFd`T��FjC$�!�U�CdN�H�C$�!�P�����2X������F�s5��+���\��V8����f"#���L{d_�	|��J�QW��>�0>�0>�	^�O�l|
�	->��O(�c?>�<>�h��IwH���6�5�
�T*��?@mHu��+j�m
WW#�C��}��� =lSO�_���U�����)�����x����-���7�{s�7��mzs�$�6�i��UF��
������1��u�Y��	����6Y�&�x,n��!��q?�Ed�LC��r��b���UI���Z��6M��`�H�a�>�@��Z�_p���W]U�_�5���2E"SP����!��_p_��@g:���.��h_�M�6����H}K]��|���
�o�����
�>M�Sj�zG�����H�G���G������I�����G'�����OH�'���G�M;N:N:^����v�,��T��{��@���)B"�������Rd���j$�a� )�0R�L�0S�$4lU&F��������@CT��b�BNc��22�XQ���|�B�����������0b�@_��K��0���!#c8*�r�*��w��T)��U�J�WQ��"�]E�[F�U��A
�k���5�4���{��,rD9"��%h@�� ��c�^�6�N�b�Z��rB�K�� ��� �	 �	 H��=������k�FH��~0(4O6��o����\�bV�{�s���o�x�Z���c���Q�x�I_�B�����h(�
�*o�3g�^p4���bb}�R4�/Z���Ac��]���j�J�t�������7���]�N����
3����c���I�3����y�#����8�<��.�.	���\`���B�)�4[;���,"Y��,�J+�G���66IKW3L+�d��C8la���p5���"qG� ll���f�V <�8�"���>G|�������(�7y�_&i
������t��7�������l�k,��e�Z,��{���|�q68�>������z���n�]��H�'��:�~E��Mz�5|�������;���7��E�����U�ofrh�����\������ x,(��f&L�*T��������
���o9j���
��A�-��Y!�~�#��k%����
�d�-�.>��9?��D�.|���/�w����?�
���������wo
��<�����S4SD�A��%�{�n����z�����@7�B���'��	��������;���1e�k�Z����8#��E�7T�5%����Y��]��Jek�kFyU��7��=n���������U�#��	�|=����#��=�w$�����L���d���:�!�U�C��
Yo�L�L?Z��b��x���2�1]4W��0y�����9����o��������v��������{X������8�����h.|o�����}h��f�C3��Z�3�vZ��2�dZ��2�RZ��Ri ��ZyT��9�
�a����	Fs�].��.���3 �i��;]�u��g��mZ���A`|��?!7�������\e��|V�����3����er��b�^��d�@���3�d�87�=]%>.G�-��*��wly�������2��c�1��R!��cK���Xdg���gx���RX���L���,|QAN��q4�����}�KW@�K�+�������@����3�C��C viu������}�� �b
����_��>^!����!�C�~a�����{o
;�������B�����"R��u��n(d1��W��o��;;E=������2H?A�k2��ibYYn�WypT������/@n�i8wc5o�����b����l�$%�T�������}	��0��	a��1Ti_
�_Vg'T�_-0zo��D�w���)@#~�]49�����)g`E<X[.���5��rQ��
�����S��^�=��"�O�Gt*D�`�#hgNo�y����9I"Cp�#����Nt�-���s��2V�x
n}��
���+��z�\^��:���:���,��:���
��:�����:����^�k%1�����2����|�����ew��=��L����0H�d������������_��7o�p�#�����hU�����s���������w�������\�_}�������}�}F�|�O��#��?���2xn���� ��������^���B�Itr��_����������?����������C	~�a���-���n`�p�
�By4�3g���}}'����
��j����Nn�s��15�E����2��&#.9���(H���B��0!��}0D��8��WJB����xu��7�/��%H0�{��lG*�p��2
9��P�$��,���� T,!���y�_�{b��-��F�3Y&1���U��
������t�|Nf��i�����;@/$�4?0�HB��S������a���
$n)_������Wd�Y&���S������'P�"wBrg��r���?��&�P�!o�����>a]�1�o����$�,�,�{E�T&���j0��*�CP^�X��,�z���e|U�u%��/��e4J�t_���)�j���Q}���_D���&��.`�TW��j[T6�V���r��	l�!��N�S��g�P��-�,��x6��DjL60;�s��������q�lQ�A�CV���
_r-��W1��C�����_�fnm�Zj��bJ7�v�ZD��7h��-���v��6*�a
d0M�N���K��z����D�C���/C����a�����~a���>XA��r<�~@$W�*�] G6��#����p	��x�#���G��w���RgQ�Y��Le������6���_�g��E�.H��i�r����xV��h�m�s��}��#�&���j��	�Y�FD��O�H�e���e:�;��uxGex�	���Q(�Q��D�ci�y
��#��������,"��B�ujb��H�0:	
�����!��4 �& f:��I����� ��@���~3��S���-�4p��2�q� 
}17n�>H�W���7��ss�������C�T�N��JH��.�G���Q��2�(���(�u�����s���2KnV�b�j��+�<���������k7������,<�i�H����~Z�WD*��E=R�e�r�zd%
�@w4/����5���L?d�U	
�j 	��YDM��k��5Z����c4
4 ����J9�j�,hF����GdG
g�="�<R�r�f��G�=�s�H��,�kq���2�Xs��fk���r�P��V��"��J���K�~���B����>
9�W��v�f��r��	X�]kl���on41F��n�P��V �"���{����������r�q$r���R���y��o�@m����8c��L�
��j�+r?o*�����?��?`�@��q�@��������~�6�o�+b����^c����
�/�t�LE����e$��H��r��r�������C=���_���w��Sb���^c��
��bg�q����[�7���|.t�O��~�y���c��X�0	����!��������oso�~O�K�4$R����qo�~��:����,�"�6�U��K��E����}p�n�������F��"5�7��F�>��7��2��+$�=��f=��N�j���[�t�]>���r�=�@G��}�c#���:�@G�^|�c�}��t���>��:�@G�Xr�@G����:~����E�c0�W�e<s\}�mUW���W��C�����`W��������E�p,����fa�N_%���kl��o\�4���gX�0����jc}��RR�/(Y�,���TV�zu�����,�U��.��5����.lk5�i�lh.L"5� �L.4��k5���RC�(���U�_W�r\l���/�LD��K&��5���X��J�%u��)�4��V���;yS(y���4��W���[�j���~�9k�\��?�r�S��3f�����8�I������$�h H���@��R�F� �G��)��3=�o5Uz`�zmH59��jd�$T�%-��u��tT`;Ff6d=�2�.�}z�aA1&����\XF���iT�0;��z��!S2�����f�� = UX4��LY4�,���4|%q����UtQ(�:_-kh�c�Y6�Q��hnj����~�B��= UX4��MY��93q#�}z�eQ1ZM��X�2��Q�������hn�����>����R�Es��)�(�XPp,:YT�����X�2��Qk��Z����hn���Y�~@�= UX4�u�������e�!e�!��C�E�q���E-�2�
�����mP�
����:B��h�0��������V'6�a�s�L����>�(��Y��>��Z��}mFx�����<�o���:_�Z���\��Zt���z
��7h`�[>�2���5_�6Y��S�j��d$��Y��
�?�����[Ep�������?:YLl��j+�eo��@�8;��AK�F#f�*�W������6�@�$���s���7�9Ydl��j���eo��@�8
���G�F#V�*�I�����6�@��$�����q%�U��z3;
N%��{4�&���4��M��>�h*L��G�_��=���}��D�����F-a�A�1kX�G1������$� N�������pt&��n��N1Z�N�Kv��"nM���nE+�����.7H�R������|�^_����-qk�r�'�9�mX��Y�f�p
D�� �g��k,�(J��ff���8�Q���S��t�h3��Fu��3�
t�EE8�����.��1	���Q�B������`T�{O[[bO[�}�������k^#�FF�)~���6qu^����_�q�������E��t>�T����@4�N�*f�1?����
:i$���J1�^�=�v�\s
rr
�~�9A�x������Y���
�d���
��P�Q�Ds:"�@�;N�U�)�~���HjQ�E��g��o�N������y������x�96����:-w�q*i�#+wZ������aE��3kN������`���p�,:6
}�TA����50��sj��g�]=�����F����VSG"�nQ�a�v���i��N�E��P@������\�����502�9/~�'W����Z��0��5���V�����������:Y����x5]�Sxs�\�)7'��oOQv7,�~�l�(f�WB��ZE�C�����M�W��R�a�1M�1�8�X>����n��K�~�R��B(�f?���:�ejU���m�kZm�����R�H��*����i<��wbfo.;�Z��M��vW<�����f��cX��#��
?"��z�tx��%��.cD��
�i)�1����q1���8Q��q"�#�/m
V��/{��MUj(A��Kh��[V�0�F�:�Q��{���[���,zU�qb���D�Ne-fe����7 �M��7����]�t�p��@\���
ii~�k�h�|���-�pVX�
�a�E���7 �+g��.zY�*�"���^��0h`����W!�!"v��0OT���l����_XEX�Vn���i|K�lk�u�CcK%pS(���3��Lh���9�fzZ3���4[c�)zd��^tNE�a����������R��<���07pJM~�5��
�Z9k�`l�0���2KnV��]��A��vkP��3�������;C�3�;C��������w�xg�f���!���!�b��w������;C
����)��)g�n�{�GD���E���O��H]���4ty�H��#�1�'�w�x��w����#�1R
�����c�;F�c�;F�c�3�)�A��w�B�#�1ReS����>F��4$�5�w���g��xF��vkP��c�������;F�c�;F��������w�x��f�1�#�1�#�1b��w������;F
����)��)��n�{�_D��Wk�����r���Po����{e'\*���r[��U�>e��R������Z�C�����$�&��D<�Q@�?�Q� �-�~,?�Qq4d1��7��#�����K�qRS����9���2I�3�2���j�Va��c`�������Be�'>'�NK$FN�0�i$4B�s��-�c4�	�����j�F�
3�HBc@�`N�Ec�
�IBcH�`�3�Ec�C�|Uv�n��Q��gG��&�3_�%�����Q`��w}n��i�������(���!.��,R��n���b���4���n��V����qc[��n�J��g�N2�3�*@���oK��bL����I����Lm��������,V���Ot��u��':��:����s���mf���������L����)����`��@������������]������5hx��@eox�]���
�Z4��������e����q�w���m��c��o���nsAnzo��&W��&�x���/ox��wox��w|ox��wox���
����]wy��7�k���wox��wox�����ex�eS��-������j2��L�@sa������&�F/o��&xo��&x|o��&xo����M���	^wy�7�k��&xo��&xo���&��M����K6�
��j0>
��� ��	��������U���������������D����>�fxo��fxo��fx
o��fxo��fxo����m]��=gox���m���w��oto��Fwot�Fwot���Fwot�Fw
��.P��u�7�{��
ot�Fwot�Fw oto����lm��6hog���[:����e��|�&_RxQqq��,��x�������x<K�tzw��g����PO!�<eW�l"PU�"�4�)f���Uw�v�[����l�F�N�^`|3��8YiJ�����}�
X��<
�������
�:���_����&�������|�W0��f����E|*3�� 2.��C�����lM��W���L�(1�T`���9a@
A3���P�>���������C�C���@������������C;��h��>h	���h9�}dN�H�}$�>��}T@{�hU��+z�~�1��\(;9UKD��r��e��P�'�)��*�1�������	��	��)\u�����`���
��>��:��eS�O$�O$�O����'r6>��P�'���H���H�p=2�U�8�nf�0�Kd_
M��B�E�����}��8�=c1O${$��>c��e��Ei���5��X.T	9t���AGUfUp�8����Z�*���-�������,#�����G�U�f��`��V)R�&:�~DZ�(4�����f�"�l���G�U�3��:v����u�r|��]����w\y�v�kU��d������|!=�y��K�r4H��iF�����3�H��������8#���UCo�j�����mf��&7O�_�����M�C��:/�O�r���X����P��N	W�%�x�����D�RrD)��2	�!3���<�eY\n�v�5��*��{�4|�3[*6V��R�D����o�,��aW�.�87P-������902Z6�0�����H�U��Q ����d�H�U��Q f�-c��M���X��b�3�g�H�U��Q �����0�_T{i�`O3�g���4i���Q*�~����
�?��c������*���	b�U;nn�9���	A�� vR������U{N{N{^����?�^%Y�G����z���^s�P�����z��W]��J{�t���+mu�N������	u����:�N��:W'���^���:Y+:Wk%n�2���(���Pr��M��+d�c�1�����)�}���tB:!��tR7����
���������n
w_����n���=�������������b�l7���~�C%n��C�P�o����z����+@�B������SwA������#W���r��9�2g��c:�n+�7��qePJS�`2_��XB����q��h�"eT6���"[)�iYJS%�u�i^\R�o��1��hQZS�HY�FK�%6�L����|�������h���%hh�*h�E�B4L��4�b|E44�}!&�~	Z��
�@!�
B	*z��".:%�3�����c[)�Y�������n������/��@���4t�g�)B1�C�������f��&9a^�7���z��e����yI�`
��q�X��I���
���z�e����yI�`�T������3/��CPU	�o��������������2I��b�"��_�Ur���hN��g�_�`q�,��biuTR���q*K"���}�Oa�1y�����3��z�������WT� ;�fET�d�Q��M|��Z��P%��@4����k�:Z}��R���Y�=��K
��^a�K�&���Y�K�7>K�2�_�������x�(7��8E�RP�������2�@ l���*�)��#`vfT�p3�F08-�R0��a�AOs��.��1������VK��Q1A?�?�O�C?d~@_�'���������>��yD>�l����?�����~`;G_�'�a��������?�{���p��p�?�a�����O��	��	��?Dl����?�����v���O���c�9(�z�d���l����l��������>��9?����P��v��w����������a�}u��l?����[���=EGv*���F	�*4PP~
:G�+Q�����By�/�������m�#�ShF�x��&�y�9�J�Ler��!�T��6fvm\TC�V �����i�e^�4�i������_W���v����<���&&�S��Tl��T���T�ka1��������j�������
�P�F���*��?�%F@D������p�7�N�?�7�&F5?��#�����2�������3�g�_~9�<������uN//O��_X��c����3�iA(�Z�y����V3�c]��-sD&�P.����}��/>�e'b_M
��wq��c\\���'@o:����_j�4��cj]���3�D���t1������m����2���r���8@�>������9@�kD` ����m��!�ncL��?a>�n�"������<qB6b�����~��yz{�Ln�U�R:Q������<6��%��8���P��(��!�8���|����C�q:�|B)b>�1��B��8�p���F�)<��X4�vf=��`����
l7�&�s+��2����
h�w�|y����<���3�C-�C��0M���	y���Z������w��,,'2�������}�B����]�5/���h������D�ZnL��y*������B]��1��
��B���o�f�S���L�7q���Pg`�������T��/S�M@�*��q!WQ\]��z���l��F��+��S�']A!6�L�u��rJ��+(��@;���2�)������m���RT9�t��Ts��RWJ(�xN��j��l�r�^!�O/S6�	&>]R�8rI�;��%e���V._������wg����Zq�X�'�XLf�:M�+�m[�0U�8{������/@ir��m�9�v%_��o�_/�i�	�[M��?��9�%�w��n�g~R��K��������|�\)�)��L=Q��'|�"2��S�����%#���M�ze��<:�J���EsR5-C'��)W��l/���t:�reO���"@����3��)W���aK<��j&��
�v���7{����zR����\)��k��TM����X�U ���1�J�!��*�
�qfU3<�����v��AU��!����@PV�OTe(�.���vMD���N��.�y����:^}K����_�^y�����W�y��W�y��W�y��W�y�
J���@��W�y�Ao^yD����<:cb��O�������__���.go`<���$8�x���?Y�B������t~u�Kp��
f�Rm����i�R�������\�����sB��������.�/���Ug?�<��xy���
���������xbD�<�����/�'�&l��2��+���_�����R�0��N�����'&���oN/~�x��9���6�m�"�q�zF|�0��Z��]BRx�_4�W�G^�`��wo��������{�|�N�^'7�9XB��UE.��2��&#��3���(��$�B���!���� ���\�R=x������?�9�V��lS|��|��K�gF��������&r�h2���#�.�[�_#�K�_�_������_����4<Si����v_���4]'i��'���k`\��Pl ��3����KX[�����/�����!�.���8�i���I__\����x2)dQ@���L�#x������K�
p'|�4���m
����$]�
N��S4f�ol�~7O��wI&��3��as�'O�x�����#:����X�����������U�����Q\r(��s��Z��!�U���L=,���8�,<n�66�'a�T]�r�>����y�?Q7C�x��T�# 3��e�?�S�Y�`�#�Q(o�X�#b���*e�G��&����C�~(���S&�=�� ���\�M����S������]�	xv�Gl��tk
-�q��d5�
FQ3���\�3ls�LA���zRKTkZYr��:�*VfC��+�(��1Q���3���j�1���6�6��d�r�{j��n)N�PqQ~��X�Tk�?�'���������������O���Glhn������w�@,��r����zp(���3�1f������z�35J��"q�@��'^q�/�����F��!R$4�-�I]�Z�-�o m���@�|o�����V��WkX��z�y4J��%���O���`�
��� ��0���ZrZ��`*��@l%�=U�g7,a6,�sc�a�4?
7g�Z�"�����Vb����N�W�S���h�FM
U�Q����XN!o�GG��^�~�s�
�o:�<�}o��
����*�m�����\�fx����z\+,~�>��	�1�dp��6x��r��p��y����qp9������Q��A����3��p�q�D���Qa��B������[�!�>�>�?��DC�M���!�-z�Q��K��C��ox��7�8k������/��p8���xJpO������5G�����g�C��J�����{�����G3���#�c~N8�r�rc;���p�o��p�
9�"������!7�Bn'\�<�k$�f���!7�����Y�����_'�����G������i�s27���
�����������Y��f������|�W��
st�x���n�������s��Qi����o�_{xZs���c4��d�7n��������7�=���{����	���7N��8z��p��(��qP����=�f��d�r�B~w���p����+|�<u9
������
�=�9�����(q�}_����_Q�]���[�~�8���+<=#�#�U�w��������J$p+��{1�Jq;P�AE���A�A6d)n8�p89��y���!O3�{x��������S�Y�Q��_My��8����s�!�:���������yy��0�����_���������9l����x��_��������_y�����'/��{/	�/�pc�K��7������G3~��t�%�>�������wmN:�x������q�*�5NnV���J�����R3O^���A��/9�9���d^��p8�tQ��������'�����[�����K����3���t�F��3�+|�Op�,P1Mq��`,���"sTb����_
y�e?j�)��y�?+
����I^,����1�MKP�T�m������*deH�(gS����Ia�
��}AYJA���=���_Zb�(����g5;(:��5e	���-|f���m*�g&<�� +|�QZ������A9n����'�3����=����"|f����G�n�s�>��=�~
�{��$|�Qs���GME�g&^��@"|f����GM���U���=�
���*|�Q����G���5Q
��8*|�Q���	������	�(�����1=�b�m��(�
�O,��4S�dO_^Dz�/2b,�T�_��C�����Gv9v$���lN�2��eZ�Y�w�I��d�,����B/(�Q������d&#q,OY�T� {9��b3Y��)�b�G�r*��|�a���*��H�}�@���M�$>o/� 21u��H��F�r���?{}e������^���f����w{�D�?{z
�5�go������8�	 �e����; '�28d18��x �t��Spl�����p�m�������<	�%��8���	�L�&���]�[4�-��_��-�jj������-�H@J�LN�H�Ur
��=X����P�[���^��������hM��>������-8hy�W���UV?
6�z~�<�����u
���Q�(���q	���w7A�HF�J�5
e�T�P��S��KR7���C)�	XF(�_97�(N��[�.g���:Tsu��{�fhT��C5wW+�T��KR��s�>!��2��*�� ���P�|�B��9����(z��`�"��dR��h��>�WOU	8�}�&	�����D)��
����]�3���b�z�C���L�p���P�z`����*�=�e-��3��D��UA�$>k@4^4$C���T[����"�"�rnf������	6�B��n�X��
 ���l�j�Xg7�@c����������L	b-	��+�T���T�����1U�D�j
�3���m�;W����������`V2T�/�.���J�_�:&��2�GW��-|��0���&��n�����\!�T�=�~s0+)���1�
|��r1gZ�����W4��e�3��9CWD_"s�)��������qY=e GzY��Ut�J%?4:=r��\No?��E��nQ9 |	�^�
q�F�@�U�U�N�1F�UW��$l�q
v�R�*1b�)6*��0C��dHI��#I�����K�ADWf����O�����4{�j!�ij���V5z�r��=k����5{�bq��k���=6yIp&o�h�A��@�I�&)Ph?'�8�}�����LS ����O@!��ej�P��,;o����oN��/O��|��wA��!�N����K���8]eq�.^���C���\�h?HS���DH��3��2D���+����oyE���Jv����J�Z�Ou���~��d~\���~7��r�n��sjk����^X�y��F���]�?GudJ�)6�:s-V8�LD���4�S���������1a�� &��=�:,2����m�}����JC��2�U�2��XC���l���i��Y�@��F�U)K����x��+��u=��Z���e@����N�O`���9xgw��-o��e��>�/F"Y��MXc%��t%���?�r��T�fH$'(Sa�����<A�%1�����l'��N��`T9���	��=��P:��A!v1Qvp�1���e� �L
W+�o�%������Pv��l�C��p_�"E<�1zNq�i��tW%�#V�a���e�e�R��U�,4�_��V��R�J��4�<4��%|4�9	���W����CPtq�@��}���*��YA�/��+qokg5s.)��<�����g��*������*]W\��\^+��jR��*<?����[(���\c1���9�7W���y����)�A��B�������K�_�������;C�N�����5C.���oGa���Ck3�VL�y:��d�h�K��V������n��
q����\��\�U��.3:����C���Tp����Tp���r�������ij�%�A��8��)W\'e�3O�0'>����|��,��g/��&-6P>`�p���!�����
LE�j����
(w�WU<�<��w�����NL|�
�s�Z�������U��3�/��-��^��9��e��=g�`d�"&������y��t��������|�e��'����
��)�?�������8��v��������v�-�Z�n�����d���&��?�G4ff;e����X0�p�$�PI:��c�Qg��0t�5�<�I�������Qu�ne,I�0�RR���U�$���r�,&����	�J&&�HZj�,�������>���S��<��{�������=�t�>������W��o��;;E���9���NWA�C���#b~X&��$M��0�Y2��O��z�c����(���p88�������d7���`��Q_{f������^�=�p����x6����� ]���rz�b���������b�	���`��M@�A-����y���>O�/ipV�$��z�X&������k0�������O2�|
n��`<]����{��/&S���I��|���U��r��� ��K�p7�egy��h7
����t��:�-&��H��c�>G��k=��.!��E��������d�)�v5�-��T������
��=[������������]���<��L�,�[\'t:������g���0IB4�~�N>�0_�f��T��V�nL�n�Y���,�x��S���8H��S��d	��1N��C����t��S���J�w@g��f�g0���Q���H���l_z�~W���{��������;�!p���3�:�p��~8�� R��5<�}u=t9�3~��X!d_}I��0�`��7�������|U6�f�2	&�)h��'��&�4
f1�������/���5�����
.��
V����6Y>�xL�O�~�|J��d��[��a�n0�#��F��ANZ"p�AE�*H�;�_`��i^.�d�y�
-L�i���������s�{_����9V�;���T^�g|�,/��Q�J��������'RP��~�n�����7�?��tW������D&�{��������d2��q;�/>�_^�/�=�18L�����������p)zB�Uo���);J��4I;�n0|���JS���������2�+ �t'��D/�|x������'����!�������!�@���X��Ba�,�L�����K�4V��L���}�F��M$yh�Dc���*&
m�(T
��q�����hGdF����La&j~��X0��4��4�l*1Q�d��,Z���Q�S@�2EE�K����m�u���"=JZ�W����l)������u-w���/�������n=����v�V�[26��������;A�?�A���>�yS���d��N�d����qx3�PL�b��e�*����P�0��@C�
,����{�^�P�L������+���p�t
��O	�nP��mq
�H&��$����O�����x(�o5��P�FH�n@[�#��5�[Dr�`������s�4�UX���K��J�g7�2�4��y|*2�����`9G�Fh5�_��(���aR�������Y��
k;�g��5
���T����-@��y����
���p���f�u�����u�������&�.��n�����
f�^�Z��GR��}hg�J�0+ /�sz�����l�/Z������x]�gl��W�2���C���i����<n|�d��9��M���,iY:�q��N��DI��X5���������?�����FH������D��j�����RD[��*qq�w�GJ���"Z"Q�K$����P�L dT1��Pc��jV�Y-T�����H
%�)�*����j��j��j���B��[d�P�j!a57ec����I�
1�Y�����8� �� q9rBH8�kA�<@�M��U��(�C.��_Z���@a��)��6�w3�GEV����m����u0eWB7��� m�/:�DO�#��h��`�8�����*P�@�<"����}���Pi/�������^+�p�znQ�:��1��s,j����7���a�+�-��y�a��}L,���-2
1�v�u�[�D�a%<���9����H�
�xU#�
�����n�^m
���'�7a��p�vD;�FRQ
��Usl���I�vrC��_RrU�@~ �@l!4�`0j�s�����C�(�ds����pcl���h>�h��Y&1��&�d$��h�������M��m�Z1ke��k��Kb��y�A"h��)�����t
��jLqz��/nn^p?f�ld?f.�����*�g��p�.��B��P�\�>i�c(�_���s!�\�.b��4����Q<3�
�r F�o��[��[��V����T��BNCQ�}���d��K���������#��@���FhK��RXe����6��`��Q���m���>���n����YR�������nZ����N>����[>�p�Y���\�+��/����,�'�LAb"*
~�UkNu��w���Ai���:C�N�6�
���n��T�Sg**Z���9d�@��x��YT���C����.{�����;���d|�0�m1{�N��x��^��D&{��NTrbX�C5'*[4��P���V�91������;���"sb�EN5�Nt�"��3_�k%�����L$:�t���q����KA��9A��}��1����P1�*��=v����K�l��t��f�L%�#�]w�<��-���W���y�T�u�F���	�o��O#�o�C�v��V�n��7$C����]q���{D�9&O��'�v[Po�?@)���7��fR��L
[�m��Hy��)O���x��:�����\�8�C���)����k�u��@l�N��f+���8���2�q.[�:�� ��Hl����=3u���V��]l,ON�(t���j�L��Su�TW��m�X*��h��[0��+�v���;[9�0+:�{����2,3M��L����[$Re���?7#!MX�9��KO`��!dd�1C1�	��8B|�|���{"������D��:�b����,M�#g�o�r}Q�iv�5�����_=6���� ��1]����,����$�����=w��f��a\[���E��<�T��7�i����E�a����[��
s�7��Z��o�
�����38�_�^Y��D��236{|V��\��l��'��!9-�V�\3�\;&����z3��?b�'B���aBb6s��e-���*��mfm
�:g���GR����"�Ph|���m�k�
V��C����m��~�&�	,.�B?��O����7�dM������s2�����
*�2��_���2y��;��6u��6`�8�z��}���������]�����ZQ������'�C������!GZT��*[D������*�oV���J��
EP�;�TA�
�[��Q�����S�����T�����N�R�&n�},jW�����XT{�"��E�60�H=@������x�I�r|���H����H�H1�l�^�d��!�@%j��6��c���V��z�G����l��H�A��V3�q0��(s�&��q�e���F�h�/V�8	����4��X������6I����M�dD��$�B���h��y����	
q%z��^�A��+��d1����A{�d�%�oN�_��i}��R%6���O1�AC��x Q�#Z<C��0�����lz��Qt������2����N#�GI�X�7���u���N��6��x���^h5Cv-�g��d�!��/p8��;���);
�R��o�s$�Q�j.E���sf�fVAKrQ��l<�����W|C��5 �����A�~?����&3W���Y�9���q��6Uqf���V�fjuof~�L��)Z��S��tS��^�y�>9��4T��zM�����*�U�T*H�	bJ�(s�pZ�D
�1�C���P���6�:�n(���"������i�b!!��v���>Heu~�Lc��\>4)���#�_>B���l���j�����j�}�&�G�^>�$)_>���]>���#��!��E�P�����U��L��0��G�^>B��*��P�|��]>B����44����x��Y.��
�!\�v��'���k�u0�l��f�|�DM`V d�i9z[�d�e�,x6Oo?50�b��S���Z	��������'�8PUZ��MK���Zat����m|����G���Cdm���Ce���]����b��"���B{�BH���8/��[x�r��T���B��f���;k�O��e[!�Z��z�D��Dv��,^�����5}!�e���9���Y�<��Y�(��/[{�����W��`���M'(p1�:�puv���C7�/����=y�N>�Jq����+;�.*;�����m�(+;h��+J�����:�X�!TTv��-Vv5�BR���j��������h������&��J������z[txf�Z��ja�R�J	�X�}4�h��[(�l��25�26.K\�E�������-�k�&�l2��L�#���p:����V�����8@(���)�[�b�3z�������nl��O�:#��1�2��)���p���)8��F��������N��q��,\�����68 ��,�k�=_�f����M��<H������+5`+�]�k~���OI���`�`A��&��*�6��t}G�
b�2�e�{�������q<�7��:�W"T� |�����B�_��.0����{_"Ov���*�F��3d��2gx:��C����}��;`S��,�~��s���rq���Y;���t�1��&���+�E��b_(��4��>f�l4�V`Wk��m��
5�uDT�4+��.�h8Z$=-�i�*�����!j�x1�G�:����~"��*����xZ��F&����M����cu�MV4��o���v=;r���.
��~I��:��]���l����$l.�[pf�VU�w��l7�k��
�������F[�Z	�3V��K��y�������@�.���_I��r	�<��!~��m8����k��e��q���a6.���������Ne8U��0�}�U+��&�%"_�	��2��AI*l�!x�����YL�W���u�a��������/��
&0���y+�-H��O������,�8*�tYs�Y7L��9���j�3�L��r~��s����,-E�d�����9������"`�"`Gy�����#��`3;IG8J:��%!e,��$T�7�1���O+�;?4��z����|���������qE+�����-)	��gX�9����� ���,o������S��}4A�����\�>�����"�s!�o��7d�
���l���_|J;!�������B��"��H~�?!�O+�+����������r���,6R��{��\�

�Oq�	����`qs���q�x�X��?��6�I�����6�������"�?i��2��G��;p"##9��oP��������������^p�����-8���B��}�������QV���i���
x���@��6���t���Uc.�N��}���m�Ev
�<��W����YP�	��_�����@}'�`I�<�{�c1����iUr�x�1�"�\����>�%�[`���F++�=����Y��[�R�>7N�r[GH�"�`{�9
�0�_6CJgq1�j������:vq[k�%����*��)O1��iEb���EP%$B")��%���!6Fcg�[����]���p����=�����) M�)YL��GJIj���2U)�4ptdF �1��$��A��c0��UC������*6�mn�gqn�
�H�+)W�����#z9�~t8oP�����m����)(N�I�4���>�-�6E�ZF����<��$B
��;�59�v���$+�)����V��Sv2#���n�1�*��u+;m�����T�Ka^�����5G�F�����m���^W�Z)9`��.���@���Oft��&%�v��5�����m�l�����������Y����	��Fg����xi�B���g���,������������A�o�8�����:�z�:
4�:��mw���aP���Sx�������-����S6gy��UB-U��~�Z�PE�b����=��p
U
NWQ�f���[��k	�P���@��?�� �>������t�9hW��H�'�J}[*0��S� �$�4R�S���p;�	���2_�.��P�������v]
+���v]��X����uUC�j�]W���j�]W�-���z]�P���j�����j�U}���j�d]��u5��P����u5T���b]
��j�^WC�����p��j�YWC����M�d�zF2&mI.�HU����d@I���=�d@�L|U%�;�a ��)��o�NS,[�w����T��
�7T�?b���r�Sc�<T�	��H��l�����9]�@���V����m�$���eF��/�RJ��K�^1-��9K�$�"�yt�K��N%�IJ$���Ji��}�&��I�.(���r(%y���8�%Q����0t�oi���f4B�x����������V�nDqV��.b�4��~&Vo&[��&Q�Y�,i�i�����)�6���b?D�I��������-��]d�\��f���m������|[J�
N����m���E��L�����Rm3���Wp���,�\�k����fE=�|�2��&�^������7����`c����nor�V����Z[k�{bY��bc\�e[�~c����MH�����$������f_��T�6�l�\*��<�@/�zA���k8X0
	��|�T$>K�7�Y�N�N����n7��I��
U�y���������-��+J�O0�K�������p�B�r���2�a@N��Hp������l
�������aX�D�����g�����a����a��J�UK������p�v��s���hr�WR>�(�$}��>���G]�s�����/tV��-����H=��:������hL�e:Hq/�z�4k���n�i2K�I!����=KnV��z$�N���}��~���yx�wBXP8{�C?�	���*�(n��GD�}�/w6M���q<�����7�h�]��=�`(�=y�������. n�����y�����������0�@�r������������z��
���usH@4������Ap��3w�0�#���	��l��w�Ho�1Z,��%��H�F	74�y���nj�
_C��:E%kl��eM���$�[�����'i�^r���P����[<:����@�.jT��l%��<^�a��?�8)6Y-(u1�L��y>������>^���{K������p4�!�g�����(��ty��.C����k��Z�n��f���V�|�v���HO0-���G�t^7A�uS�^7Jj��J&�"��&��P��"+�����d�@�����d��Mf����",O����t��h�|�$] �<�],J�<r�*����w���/�/�
i������w�^\�x���/W�gF��P+N%�kk���]WGrM�\��3�'��b���z�6��
����)��`��|5
��_^���$y��@����gBF��$����b�1�t��[QS��!jj����X��?�	�,)�;~=&~��b!�k�ue��2�k[�sK	Xg'�]2_A�8#s�o�x��Zb�PnlBX&���j��!���KG����f���r�?�U����-Sn�g���@��@^��yS�0������cw���1�.���X�LL�?%�� N��QH�)@r�`9���.
Sd�b����1V��/v�z�8����]f�H���^�����7W����P]N��t�\����w%P�b�a?�&M�=Jt5u/{��lh��i������:�:.��I9�#W�y�o������>O�/�N7�T��=���t�:����T�drp���n:��4�WB�>k���r���I0f����d=rg�
F�u�����7u4Z�B!���de��F���'n��"I��ID	Til@��h_�U���'����\sD������V�B���b��(#��'���{������
SRDO��uc��A�i���l�y�����RI�j���
4j�A
�549���e���d'�EY(�0��f�v1d<�����0c}�Z���k�
���m������[o�A{7����@7��~	G��GO���f:�83oQ���@���a����y�����_ApO�1�V����6���A���V���CPq|�Wi�k��C�QE���n<���=��Y����X�0tX��}V��|U����H�RYT&�o���s���6�*
��N}��8Xf�b���x��t�f�G���`�i�
sO�����#j��0�������q�6�<M6�K���L���v�Q�l��L�����2�sZ��� V�^6�kM��mf��+V��6E�Q��m���r��V��a�i�����f�fm�
�X�=�v���N��<�A��[�y��
�c6�5��b61�m|�U�d1)w���U���������5�,��#��C:���b�Y+��p��,4��
BT	��P�,�`	F�np,T�z�$��)�,U�7Y@b=��>^��������8rZ*$�s��m�(��f�'��L�������I��d�M��=L�],�CO��AK�r���������J��&�vt��\��:Z�e<���~�����v��w��������:�p���N1�6qv��qT��;+G�#^.����a�$9W[���t|z������������	q��^\%�Jk�r�0.v���mn���=���)�lP�����t�*a��Kx9�g� �04A����P��Xjn/
J��	$2������
(�x�����_��?9�GU�<� e#���Y��V���s���(f�����g�H!�)�xu~�����StOB�~����$���]�T���_�K���w�L������S���5��l� ~|�
p��F1�������H0Xs�L�����__��{��677������"�vs�$#(`��t~u�K�
�6��&�D0��������*�	~����Hxy~���� �
�
.�����w�<1"�\��D���e����=yA�������^�D�}b��N���x�R����7�?<���z��6{��$�.��Lm�������o__���|�^A\q<�|_�����(������%���.� �=��B����v����3������Q}cI�A��s6FMi9<�4h#*�3�a)hm�L�@?��g����@��r+^a��}��O�G9Z��f=9_��B�l��L�Q��p)��N�n�#�k�k�n)��;d����s�y[�A�������[��v�m&�PK�O�H&�z��m�����
�"�����"x�����l��@���;��06#��\�"�1A�Q�W��@��K����r[�>�����v��/l0�)UX]@�-g�Jn�
v�0�},�_qq�ZP�/o�$pA}���pS4>�E%��t�"�n����B��8�w��U�H��`�����m�,���u��~�k`��p�ybk��)0�J��bQ	�-������Z���=Ve��[��A��o������0���rW�d�����#�0�����������2�9I+L����|�P7�xm\�9�ENN�h*L>-��m@�a��hV��un�GT�/U��*������7��&JD�M��
��0�7�0� �6�V����i���XW�&[���|�<�i/Z��.���sMm3�
2�jgo���@���C���)��$���U���F�����h����� or%8Z����A}
���+�����2fX�N���{��M�l�~�
�VP|;�vU�C�C"n��6���,���(�����F����ie�8�����%��M��/%u7��et��r
*�4)�[4���K�X����E���l�-�B�1�U��myv�X���g�
67�R$����5R��L)�*����GY
�m��DM�BN�R5k�L9%�S��t\c��d���T���p�b���>�&v�
�k�4�\�����:/�����S��q:�j0���L��9)����bY FW�n=[Mg��dv�^����rg��/�G�,�[�F�z��~����ec�"�����u�����%�����p��~�����z�grS�pZ���DV���[
���&���6�[*-����oJ��g�#p!~9�}����w�*}������U�l��
wL���@�F@y|3�+`�������(�^uU'F5C^��M���M"��R<
�*��k�����Z�yz��+����>j��������dk�$�
H�I,\X�G��.vi����K��{qI�-b]���f�*
����
piQI�msj��L�W\:���q� q��67��.vo������5�js�g���P���l����;��g�c��u��P�t�GB�y�RUu�-0�o����Q]m�5�:R��� ��V��}�,3Q�N�C��s�o=j[�e����r^f��2��j�x��ma&����N�����Ea�n���Q��D���rKp[�l��������S�Stj���@���q���0�������m'��O�mI��|#��&8y;������i�'�v�{��{�
���M�S�5+�J�����Dn����x���'��4��r��"pS%751��({
mu�s%E��9����t�(}e�D&���`TI��7WUO��QeAn��j7s���h�(���j��-���B��z�c�-�j��
v/�Xjp��.A�,q&C����u�7_Vv�~����R���zX�,�5�r�c7:�����qZn���u7��n��lE�2Fiz�����{6MT�-�����f�`C����.W������X���)C�#7U���X3E�[>�������gFG�����
,O������s�^�9&f��z���:��
K@���J[z�; �z��G������LU��~1��L5(�:�2[-�h�'�'�O��O�NH���M����������8�Co������x��J��N@�R�hZ�_8dw�l-� ���"b	0"4�we��/p2{r��k���Fu�y���:P�C�	W��7T��r��d��I���Q��������]8HR6^�c$g��d�<�j�v�����_m[��������>$����~���H��R�*��N�J1a;�����c���k���U�5�^PD��?"T6��E��[A�5�3�(�EMN�������6?�
l�.�=ck��r�OVL�5C����=�#'rQ�Ek S�z"U���a�UT�|��r]��'�ZI�u{�aU��Jv��?�g�
*������
��=j���������G*����8Ka���}�XoaRm���rf9��e�d*���P�E{�k�����E�3.S�$v�f�&�[�Z[v#��Yn��o5�h�"��H�����&�lt
��	e�"q�Ah���V����G���0-���m������>����]��Q--�K�����#�����D%��G�����	�KK�Jg�������8������������+L���������$?j-+��[\Q��b��C �O�@q�2��������������{��q�����d�=�=2t�8�@3C8j�t��<���*���~U5\'�*l������6�S����0!tnC��sr*
�J�1������8��D�����n�����Q�\��BvKg>l�]X�Jn��`��-���S�y�x�tsZ�D�mM	��i���2���asA����&�(�'�C���PL���6����Q�4���)�MKS39�b�q����$h�����.v�Ga<d�i�vt�!A�U��s��Q��Q���	��&��n�����qx����p���$
��d���M�����d�x�p~	�pg��~	o��u��q��|�� ���-��;�d8���fS���}�I>���>y�|�u_A7��)�y�2#�������/<��!7��<����P�}� ��//���*3��n��#��)s@f@)����+��s��}�`�8�7z�%�?����A�����!��^��'06<kl��
<�0r��e��sO?�f����f{��=;cF����lk�0���1��c�.�������wQMM�f�8P�����m��r5o��)Pk��S�b��P3K
�\�����73��HJVZ�������
�_�I��fa$�����.b�4��}m�����&,
o���k�0�/O��|��wA���O?��p����|�%q�����G�i<d�ey����t~"�N�>=)���}��[����6#x=��9'<���:
>��'x�;�����_�o����<<�����	�?N�����������
��������~�i�w����'2l���
C�����C8`~�qp@��b�ah�����F�y
=E�y" �J��
�[��!�
���A
����3��%��m/Z%�{�����=6���?��V���������9&����^�����|:�g�Yx�a
~�K���l�
����o�x�c��+:����G�aF��0�?�0:��O�k
i�Z��x�l���"�"V���^?���7���09~�<�9�><>������frt|=�=����Cr���^�G�� ����18%4
�&	�Hi�l��[�x�H�?�,����*^%?W��?%c�h���qp���yp�F����?��c�_�O�e����d����;��0� �Lf��W��7��BO
wG������t6��^K�����$H������!���K�=�u��x�>�z�R :�@��|�������=������W�@au�L�����Q|)�F�BF�Yp�i�&>&8��!D�b�4<�m~�0�i~��?���e4��]���$���e���Q�"�=��	�U$���+`SH�_����*��NA����K��w��0;	�����)n
5���l�M�g`�=�����q<�5�_���_M����,X�����7N��0��������e2�/��^0���8���l}���J���t�g��s	�w<�>��f:���'4�� �Ai,M������n�@x������~�NWA���F��M���C���� ~���(	�����Hs���j�}�����q�(8������^��b��������OP�z���.������OI|:���t�	���tt3��������Q|��
�hM��t2JV�1���	���y��Q�t��g�?�MV#�B�	��2r|�y�u��<].����x=�]��8�V� x�+�k�3]����A;�![�%�6�4�_�,^`�^�a�����n��2o��k��<��'�QF������F���t�a�;1�&�/���l��w��&�8^�:��k�)��u�``����^��!E������H��#������{�9����C�E���h�N?�|$=����>�����3^,f`���
������ip��!����a��	v���Oo�+��Oo�o��x���>y�k��p �5��~�A�����>��g_q#d��0T��w`{s;�3��������T4�
|��9����dv�N�����b�+a�&��I/�F�A������������h����&K�@�@�������I6Msi��-��z	Y.��?P��d+@����Ld�7a�3�;��7HB�BA������"��`k�k7./�f���n0_������-H�y��U*�&��`D�'�n�C{|S�}����_�e�U�N�@�� ����B�����fF������?-�!.�o2�,��?�.������W����W�a������~��O������K���M	��>�C��0>;�%�����@��L���P,����=B9������	$����}(��=C�x�|c�����z�kk�����tX�����@����}�SU9��Nc�<`>hj��O�	�Y�'��W�J"s�``{*��P�4��0��&�5����<j�P+��3�zb�^f�����M0�m�
/S�i`�u���eT[�<�=�
]����� �:&N>���_n��4�2����7o��E~�v+B�z��TN�@$�c�� ����z�*^��#��03A3y������Y2�-��.h���!lf��T�6~��$Q��t�LW�"�3+�x���A<�(��I��|�C��c=�sH��8��� �@�����,!�����B� ����0��.�`?#.�� 4o!��a���d�|O���#C1{��^��g~�[C ���D������:��l�`����nrw�����������]���'��&K�����Iw�Lh��G1Q�2�	�QvC��hf�ZE/��>4�����7������8�!��X��1�� �y�!I� ��3����4h���+���2�@W�#����!��]���������0�Oh�j�3eQ2���!��C��W@�������3��0�?�;�z�����^��#��*^����*&�`fE��^�����t��o������o�����!�Dn+_I��p�_!�u�nUt��^�L~������k~d�( ZpB�	y_�{@|m^�Gn>fP�++@�����Cuq~�f���
l���YI����
�@Y�t~�����ws�(���F����2�ip��?����4�	
MzP�_	��z���?�
h����Fjwn%'�'3��	D,�a�v��}<�x9������8�X})N!��[���Jy`�}�4����� ��mR����������u��IWO|����3�R��}J�k~�Am�Y�Gs��������L�;/��<���V����/�">��2�/�������{��9XX��y#�Z�;$���S0M3�
T�|���xqW�������'H������A\�
�K�U\����g��%?E%����,MrR@��1����#��
T���\��Da7��2:�ME�h�A38��2`�BT�&�������z��wg�'({��N�	�vP�Bt�>a�y�@���X�O	D0��p����~.�0{���c�y��e�^��1�y:��{c�y����q/2�<�
�z��G��up������EW���1�~�����7��Sq@LNh�b�t�<
w2Y/�+��~|}q5z{���a�%����o�����Lfq���������y��!F���
�=�.�|#���'F�'�_���@��=�����$��K���l�`�����j�_�}�HI�5�$��t�'�A����X!��
Hq���BO�c�I�����������{]�s�^y$�qN������
�*�]���8�3��(`I�6�3������:���	ot�_�3�YeX��r3���_��u5���W����K�
�.��)�e@h�C��GH^��No�����+���Dv@)�#�`�#b,�e�*q�w�w�_f�'Y�b�b��/����|3�a[�6F���
���������EBF�n�$(q ��`���\��U2B���`�{#�m�le�,P�<����$����-��E��"`����a��7�7�<;�d��
���;��d=�q�
u��{��f��-���u��Y<��+95)��`%G��i+��<����y�"A��@MRm =Bn������WH#h�`�H�AM�3�y4�3��_�d��a�:s]�8���2�fS�C7H�������fq�:7�4�
�Oa��|=�#�����~����;/�,c=A�����kg���Z��'3K�%CN#��-�Q�1y����e�RX0n�C�3<:��!���aA#�W�����d����0b�tV��R�<������V�>\��p�7X$�P)���lqw�^%��5�� 
���6�E��	��/aM��(��.>4c�����Gxs�$um����g"�_{77���g�h|�1a�r�\����H�0\�:�O�z�$h^O&@��!��.u�]tF�Y�)�f���?b��a
Y��)M���r�GH���������K| ��%���5�UN��\}���Y��3T�'���x�Y�b)�����h�!�
��Y0I����),BA��\�++*I�W:����M�X_���/8�u��9����8P&�97	�sFk��#�����n���8Y�bX�<�|�J�Q��^wpR:,$���rQ��������=�1����H�@��8�
b���m��\��"���wxk,~Z�+;c(.�y��T�l=QD:q����+��7�=6�Er^�a���'�����
����f���s0�����
vk��5�Y�q��~P�>h���I�&���`7H@��������t
6W���F��t]�D>8��Lp��Fb"��W12��gx&+�U��������g1����b��hc�bc6���k��b.�\^%��dDC����c�U��b�!tj�� K��=��E�t���8���E�@�]�-���= �)n�;��\���6��c��"xWS(�v�JDF']�>'���`����.���b��t���	�4C~[<4������i�RLb��J8JR�6���H���OI�������<�h=�^��`7�NF.��CjAL�t@�c��S�z��+A%�g1���+?}6w�b��L������H�f0�>TF���'=�� C�t��aF��w�����7s�����I������E�#��,�`�%�8e��7��'Nd�^`_�d�ks����Ow�x$0PO,ls�X�������@'�������)K|����}�6u���j��m��5���M�}�](>;_�qX�	V����/
��5O�J����T���>�DT�6���P��S��>��#�V�(.��tC�� �������}�P���f
�Q�b5�s>k��=f�"�<��A�oP��&psB`�u ��t�������O����w���"�<����1-�����kK���+xP����$^jb\-\�/)�������c�i��t��b1�}E���J�i�g0�b� ��u������+��,���*~�����Z�N��!�0����'nrb��=�rD���\�*��yD�ub(|�3�._~%\�Y��A��'0P:�M@3d����Y���#�T+�33���8I�C]o@�qOA�[�aF�+�/����@\��l��e��5Yb��K<�R	�����b�GF,���[,�m�((=W�^K�>5d�0:����V�	��;���z��6�p���������
����h@��h@�a7���I������	`{�%�7}��d���+�F��������=I!�7CLlrjy����pQ9��b�,��2���tG*t�g��!n����~�h�^��G�9r5��NxaId8��u�D.��1G�����#\hi.\i�e��l~�$�\�PX:^��wP���8)UF"Z%Ym��I�vb!�W�0���H-F�,��`�a�������	@�i�q]'�M��C�qln�fDi���faz��?O���NW�4�!����p�8�,���1�W�����r4�!��?�C��C�}��M\��C4�>I�������I8?����`&��d�������������?��~��?Q�����H��kd)�J�L�p�wN�m�7�k��$�?Yc��.�?���k\`q|�9^N�U�D���x�b���$�������W
� ��W���|��M�h��@����M�]��T�|.��=����70���A�@���F�l�v�IM�,��m�
G�!�g,P	V�VP�R1��`����������Bd-Dom����@����_��F���p��+�s�/�_�c3T�f	%x2��M�����oq��'H,>�}�:�$��h�S�F�����"���d���j�{�;F�iW����B��� ���u�'�wQ�.���2`��=�� pP�crw���,x�J&%����Ft�9h�@�S,@�
�`����w�9��{�X�l`�h�k~�0��K�k�.�dz#�(O
rZ�3���j�Y�Ka��p�,
=R�Na�b
�3�3zv�9� j�
/8f|���*7J�$12�~��#�5aN:x���(��\w�f	�2%�e����UGj�M7���YX�g]Q��`�Y������@�������zy
�0$����@������P�E�(��B�D�BS^T���g�X.>�i��=r�	VY�
w	Z~Uy����}L�n�����Ph?OEb�����c8a����O|SB���;�A����D��0vi�!~�����n4MG��}���;%�~��67#P��7h#�5��)�<���4%���ad�2�������5�k��.���D����
��=��� ��=�/�~���[(���s��H{q�Kp���gt����';a����6��3��n��-;��>��gCv��A����< x4�D��XI��
!��t�����l�M�	��t��w����{_�_��[��>H���D8�p*�"v��X�V�4E�
�>{$X���&���4��d���	c6�Xt�@}���k PC��V�@K
���]_�l�-@�0*��)u��4K���*��w�%��S|Yd/���3(R���a%p���pN��5����P��A�7��6���jN��L�p_����$D�gP�cY
k�h��'p����S<��f4���������H�Q,!h���f����)�7c
ER~����er������
����������X�����3��]$������
�0Q��n���
��|�Z�i��@�q3�=J'4m�����\.�S��]��H��It����@"���CL
��ZEQ���BK4|����T\:�K�c�>��(�'�0_����J�
�rI�C�������hM�%���@��^��=�@�s�
q"Z���9�O������gG�$ze��-����B�����n��
��G2�0c��
��E		��I��r�@)YL�	����Y��5��x�fg���5�{���B��k��� ��-���2����a}��
��C�r���D��/XP�|J&����������l
��x���0����!}��I��/���4]&����.�[R
 8������Kr�:8�4Z���"��K',i	nb��o�r��
�PIT�9��q�kL9��|BPz@+#\��GhOb�h06�04����L��-`�S@��W���D�
�`Q�_P����z���sL���[��;�/��	FC ��
Yy��(G"_�Q��}B<����2Mt.�on���	\�����?c��f��L�_3���_���_��W��c��4��$�����:�8�a\�L����,���
���cy����'
�(K��901�j�2��P.mG�� B��Je����n�/$S&/A�,\AK��Xd�g�dH�i�t��.��%�`���p��'F"�Y�<�e������"-�%���9��B�2��KT�[�X�5�LR*
;�=�u�Ap������Ro[a���&s�l2g�`=��5����0���L���B���v��4�g�������
v�d;t��:[��`��
��10�N����L�l��V�@�����g����t�=,���b�=��C��@\}�M�,��UC����J��OaY������$����t>{6�N��'��f��B�Yze�{����9a���,xZ`V����V�$��*$YRIV�!�[�o=Eu����_0�i��%��O�	�70�Wh�I�0x���D�`]
r"����L&�0e|�4��"�AQ��8Y�R2g��Y�M� ���?[�^_?�?�@����>��c
�F8L��Af���4����C�jKX� �e>�w����K	j��g�G���y3�z����>N�&���t��?��CY��..����n@��=���l���`���-��Q���q%�x���HL [,G����nM�q���'����g/Yt���E�B�7�����z�����x@F�_ �
:60�������li�N�����V8��Z�0�N� ��{���,iG%o�?���T����,�/K����9�D�:��d�R�
��4��!�W3-h/���v�'Y4���e(�B��p5��
�e���?t���������aF���
���zA�fp��f��yXCi�%Ke����xX
i�lXx�
��{T���������I���?N�Sg_G���3��� I�5���{�r�#k��r>v���h��T�H��B��� ���; 42��2��4�KG	��3�?BwkR4?��Fb��!�pu'�&.����9Z��'��<�b��c�UV������2Ca8� ��T����0���3��7��-%P��Q]��,�H���p���JC����B��h�Dq<P����W�K�����!��Y��F�!����W,����Y6j�7���5E��.G����*��������B��AE1[�����eD��wt�&+9�tx$:�ixA����%��x���C
�!=�TG.`�
t�k�*���E���/��� ���y�����q�P���0����o�K�}r��sD�8pxlqj�_q�w4����x��Q����7q����Q��x2��q�������09�<����7�����M����t��'(���"�s���V<O����O
������lJ�cX���)-�������M|�3�#�[���n���<������O?�|�.Tl�Yo!��@�&��	����6�r���eK��x�a�������=^N�^��s����$`/H9T��&KXPE�y��D�-^�b4|x��p�(|b�A?�7��4��J9��iEF��H��3�|�s'���(��jH�����7�Z6�m��/Ai�8
9`������p��4
C�����m��	�����Gp�x�w +u��r�	3�������������Q�/������o���^>�_�<����;B�r�]��#��9F7���Nm��`��L��������)���?�w�}e#�{�����A��
��^_|8��
^_\��"��O��������0�u�'?�EC��OGvId�M�o�����~0U��h:����}��O�.�_�|AQ]Q<r�����<�������������
�?�/�A����{_�tP��h��'�NO����������OV�u�d���<t;p�+5>�8}{�" N/N����%C.~X��B��vl:Vr`��i�{()t����MCWl��i�d������4T��<c6U���g(~�Y8�,VutO���vK���������A�����������F|�wW��qw=�`���}q���w�]L
�@�|	����}�!�b�����g�~��_�/�Qg�?�0x��]���9�����T���/��.*��'��n��BsB�u�#��#
H�	���O)�������u������%53�t:���l:y��`d��{�N����MD���	��	K�	1<�c(OA+W;;F��}Nr�c?�zH���p���C�BS��/!�5�=�B��� �X�-�M�;`$gq�N�W�P"�����������O��Bq�G ��+�c�����@S�R���rahd!(����De���������4���gX4�Z1�1��17����h�S(Ds������G��Q��I�K0sP���O�Yrn(_��&LJ�
��B:�V�i`7��)���?���G��u��0��79	��?G��mn��ks������F�e�hh���'��7��`��������n�'���C�w>��.�nl�	�����	��,�2���Y��A*r��o��py�&[� �����J�-v��@�70�1�3��Mk"�|��	�N�ON�dEMV�d�KV�����y�����aX���bU�s���hx��������dr����l���|}
��A0����y�4��O~�q����v�b�D����L���;a&��bo(�yC��0��Ghek'Tm7�l������k�*nil����Ag��:D���g{�����7��s���p�|�.:L��qgP��u{@������7�`����n�������%w��_Hs�����i��+���0��i
_�Q��C������M��P�?����q���C)l����V"�V����z�����Nbd@�#��U�M�������xu��`JH6�T�}�(�=�����"��?G�""����-�+�R�U��<heb������L$~.����V�H3��d�1,���,�\�>�����"�3�R���v	?'x�A�D��*����rH�������[���7�m��O��#2	P�_���s�4�/=�����L��H����v5�pEK�>\h?�i:"
�h)6�pD$���K�#�Jx���M`JGs�cAL�h���������'o��w�c=4!p�TG~��l��p����{���)�VW����!������k�~����B�V�x�.Q��0�������T�8|��y��+)I�T����K����E�j[D��[�"�o�4)^��h�Y�k��`�������FW�@�h���|��E�x1�a��Aqu�=t��'w�C�!��!��!n!�����!~>fw"�L��-��!��G���� os��d0��!��:�������N?���%��|?�-H����h�Ct�r��i����6m0�	���]u�(���+[8b��)�W5�NcG����E
7.[�\$)<��bq���_e*�f����F�K�>�Gi�)�Mp;d�b����[t��	 ��m�9�~���/a�~�_"�N$�3���;�n��Y�fNv�����'����h�����k��L���Ty��[h�!�0������m~��/U���Qa����_��VS�N���U"H�D�ej6�M���\� C��� �&	��LC�N8��3Z�9�g�R4��%I�}���{�]����C�y���m���?������~������C���o���in_�q�����o��v0����Et��q���.P��w���]@(l�:B|��7[@��O��U�Ex���3u�O�a`&��
(���}fj����;x��u:P��p�x������R!@C�PI��������i��idA�v�"�����H���(���K.�
�������;���x�Ch��RL8J���)-�{]4����pW�d��/$�����=��\�*[x���@]@�IW��ia/����"�S��=�J71-���uS�+�^"��4%vye�")�P�[z���%��H��o
eu�<�����~k�b�q�K�����Uu�S�����o
e5^���k�b�|�K�����M���[�a G���HP�Tn0(z*7��D��g��5�s�b��������&"a������q��2��k��j�f�����1�l
��.T��4��:d|����
��^C�wu���Tk���;mTc�qF���*f���8�nK���#L�bz�N=p����E�%�P�����<9���%Y�����$�M���Y�L�
�Ju��j�6�M.j(�����n���y^��Si4M�t6�Y92��e���k��(�����M�q�rZa��c����!�}�� }����K{JV�7(�{�b
��b��))Q0��`�Q���X�����p~�["����9�����|$���nzh���S�X��a�<I_��2�^{���HH0��(��:��gzY�����%�b�����Ga���������pN�lD���i����C��l�������k��6�����,��f���x�����	U���|��Ul��oo��-�x�%6]���c���o�?��q~w��"��������a��z}%���i���)���:	��u��
j
��U��zg�����Z�
Y�T&�-M&�A^]9���H�^�k6�4����W���v���'\�G���~(���a��]])a��d��:13��9��_|���/�$NW��?�"o���l�NV�
��iJ:���4}z*4�5P��}	f��6��f�X����������*����Bf��rnYA����T�0�-Y����VA�p��_��4%R1vE�DX����
���cL���F�J�m��FP��,�����:",�����Z�1�bM���I�x�������D�)&E$��D4�~���A�6�F������69�!NEh����M-v|Ka�R���W3{��3���=��A��@�^f�=���O|K����&�*X�w��E�N\����Bj�-707&tnL��@
@pL�,I'I�
����[J�&b]��@
��I��D{������a��D�_�e7��2��;}s�������H��@��P���_�R�`���Y�M�4f�t+S��x�2���P,�V���4���EX�}��.��*cNG[m
)��@\��9��:���5���=���3Bh����@�S������'��C��0{B���������O��vr���)�<J������n.��:��?��������2����r�,~�j���.jKQ�Y�VI5]�w���[�rr��0[?�
H����[��Vh�V$��o�5�G��&���i�x%T�Jh�+��W�-�W]V�������Ux�Zf����������!�������?�D��c��V���M�D�e������U/h��jdt8�?�����u�]�H��am����;�F�`�'T@3�#J�QZ8�����L���r�H�$d`��5=�iN���U
�5�!yo����l�:8�[�_�	�����q��}�@������+)�o���D�>@�1>Y^�s��(��@��
24��^B/�E������L������QU3��^*��I��W�����W����CSiR)*�9�8g�]�R��7�g�Q�t���v���������y�b�j��q�m|�`Y+*�����>����9���I
D�������j�y�bk3�q�q|�`Y	.w-�1	�`�V#l�����=�W|��������v�B����R��%������������hB�>�)�6��YG6��3>{b���.S�B�}��C*G%��ne*xW���+r�U�2\��U�e�9a��r�u���ZD�k�[,���lH|5s4�Fl+�
�������BJ��cS��WX���j�/dI�T�*���%�����`�E�L<�@��?{Q_��5�e��m�������~&�)�3�?a�Z,T�p���rV'�gh/h*�>�?��D*���N�EWUil�O��?(W�f�F�2h��I��a>K"�U�J��f:4fnN0��S��`y\����
�a]�������!�+��������F���3�����`��3�U����2Z�Q[����2����`����U��l�2�X�0�Nd�_������^Fu$�����b2�&�����6+���c��?�d��)�o����K�2.	D2 �
�G���������}��`�K��+��wi��l;�e��������@*���r��!��K{�9O!���uG�(u��!\�v]%u������<N�}��S�)�U
������F�������!�g���\�O�����5�5����K������	���4��V�q������g����#%[GZ��t�8[����f{�@�M�u3VslmE�,�x��^/hoL�F�:>Le���Q��SU�/$��s������q�^�=W%�
��qL������k��e��R�����BQ�8��3'����e��4���}�B���)��TFQ�j�"��_lN�~^.��:�T��a�]t��e|��v
]f�
��b;���nPK)�h�	�Re]Kei���Q�qw7h���_%#��p�}j��H�n���
Sz�������������oX�}6����=�A�P�������P{J��.�����Xz������V�*�#�J�O����mD�c������D>�1	������V�lB�kLx�W�f/��M���G4�����L�r��t��nD�+���o���b�h�3�1e��:d *��xpn��jd�bm=�8���.��OW>�����V����K��a�k'��N�x�A��0�&V�81n_ux�����rMyVf�e������3�K��	���Bb]��j�>&C'+\�77/�����m�~|�����W_�0�E���qI?�Z��/�y
���]S���up���GC�L&!jDDMm�����b��ti��
s(L�O�rm�FZ��#��Y��)DU�i�0�B\��L�"���i�L�Ro
Y�>3�,lf�3S�C��0��a&�<X������B�%3���5DK�sYy�b�����v9�����u�	ax�)KQ�|a`��b���jU����(��Sv][��JX��Yj��y5�KU��F������bcX���)9�bj���Z�ue1��bX��FS@�id�
��
O��!�z�S{�'�j�n]�j����j����j�ja��)/(b��M-.��6������*b�';��~BU����2����y���'�O��s����p�:^���z��~���R�w��"�P�+9E�N}����dx
=�hO��S���h1k���
��j����P���CL��6�*����_q�<��b���+����"�V�P=P��*T��U��
���x���[P��Zq��V�e��4&��}-d^�MNQ����������e\u������f�R����/�rF_LN����q�L4��v������
�p�;�{�y�)������2OBI���
����N��9�rd�d92�
2�Vd.u��%�����	d�82GZ2G�#�#+2���
����8������n/��B������9��=��)���P���l�H��@����������=������{a(���
�s���t��xJ���2�M��K����>S�3�;�P��p���7�?\���{��6n$ax�����u$���&%Yv�'q&�M��y�{rrt�dK��y6iE�������lR���1'c��@�V(�
U��2���������)?���������{�I�����exG����i����{�pN��!���I�#?���
/����0��|�-�o3IBC� ���%�Kn�`Il��p7[�$�`Q�-XrJ���K�pm>�|��Y���r1��������Dd��X�J����v{F�V��t�F� V��l��y����~��������]|5I��Kp��a2OW�������w�M��
}���^:�����M:���N=d9��'��Sd���|���|��
IF�{�X��T��+����de���!�}2��1�^�Eq�����X��^5��4@�`L���u���|��X6��D'�Y�-����������7]|�Y����"@�l��n	�"�<���`��~��#�O���E������[�n���/���y�m������%�SK��f\S
z��;���x��c�����5�N�@N��������`!��;��������/�
���w�c����s_N�$��$��I�v�}l����;�csrT	��}l�M���]�XW�>v_n�����U��I%����bD?��~�<����}l�����]�X�V�>u_�4m�~���{X�+���7��w�(�'��}���/��-������2[d�(��?���j�����Ij�����*�����[,���fx�O���?A��2[�T��	���4;x=�dV?~��E�W�E�,��
������/7`��*2b�c���F��LZ�����'�G� #	���Us1a���MBHm��n;���j5�J��FAA�J|���6�6��^��	�|����>�J=���m�g#������������q����[w��#~^���R�/I�vEl*fZ2-Tt��1;t#�1��
��5dTB�9FfA���VL�F]����i�U.\�W�b"�4F���c.M �8'�_�wkB�:n�*\�[!�|+$�����VH*�B��VH*�B5������Bt�o�(�J*�B��VH*�B��������`��V�y��dOu���_�4WC-�-�8�����9�)���,�6"�����1�2�k�������f�Jk�����������lI^s3s3�1>����#94��$�����`V��v_�^g��cmO+�u�Z[}6M�P�^��l����V�gV�w�+�����MV���������m��[-9��A�6Im�����f-�Ey\/s>���w?�����u(�sP����V�AT��W�>��` ���Y���4�M}O6t=�����.PT�w5�U=wmy��+6z�����|��fn�����|>����Y7���������rr��c�;Cg�����o���fh{�`���vg�;z�������'t�)���h��?�����[�SX!���a�����|-�DaWs�h�~v%�`�H��b�C��w�-���k���P����]^�l�xl��um�w���`LA�3��|q�o�S4�����)��A��3?�����{I�����[�r�h����,z!}�,&�,���%7��
&�]�b��b���^8��\����s5�kS$4^�#��I#���)��n/����=,�gUN���r�N~��
��$7K�M���������s��������e�\^�kS������,�W�
��V�N�0��e"�RY6�F���"�D�e��9:bW@RcWp3��@U,�:��6w�k����t*�p6���&Us�l9��;����pr�pr���&U�����|���Puk��1���^��`���*���j9Fn�mLu/����jR���N����Sf����xI?��S���p&9��z��~��������|�MkggG��@�����)#8�`��g�~���<K�%�f��)>��p��������f�����*
���P����%1���u7�V��R�{Z���Z�'����E|6��I��l����PK�^GQ9B���������3j���B���������3lc+0���������VW(9�E|���D�J�g%�8�#0�F�����0#1��Q��������u��o3k���aFR���.1��2.W���c�s�m��hn\�j�r�����n7�p��U�M`�&Js�x�+!|�)�o��
7�;��n��$�~��cY�~����C�o�P��8�HZ�J1Y����y���l�H?0�'a�/�QK���$��a]_�=�����n��m$����;,���,�;��y6m���8�a��\����N�	m�������hsR��� x�mZ��6�� y,�S�����(r�^E���\m
If6�p�
��J�P'�M\-6q(��H���0��������-���Bh����0�"���;��Rb���i����|��H�>�L*7T����Q��!��^R�x��&���U=��z�9� <-B�_�n��~����m��F��b�6�(����72�����_�^�2�M{/>����t?��K���wZ+`��%=�#��]�~�%�	���a?FH��m/���_�(����/]��BQX������oD�*����!�19��#m:[�A���<g#5�pM.��"�^d�������k��F.��
uL��l��u�l���f�K9�l�P)i���QY�U4��{��FS�pM��Xq�AOr.��#&�W��,*�������k�����z�bY��)B�V��z�[���j��0��zkr�n��x;����j�1����������rd!v��QJ���y�-�f#�&=O��)�����3��k5>��z�.��������8��d���IX<m�O�f�
��M��j�	��8��=x>e�P�U�F����kB�QMI�>U���O$����F
�d����d�QU��;�X����j4�l��2>��l;U{�f�l��
LJ�_����ew���^����r�6k�����78��	�b'nU��c������NK ������`v~0[���O�M��w����IY�����Z/+�nH��k�x����m gl��n.�x;i=rN5j���w���6�w��~F�XxS,l����xt���&�.}@����l-�%��
����;���^g$��f����B�-��M����������^����p����*X��aC3�8�0Z���!jlv����ji�$Q���M�s
���/���IE���k
I20��L��JT���L6����^���J��;^�+AU��-4k�a��l��m�jk�u�� Wc�<�����,����@cP��Y~V/��*���A��	���w�:�����q������^�Fm������������a�}��������#5_
��3��b5(���~��(�P��S��.�`x�	g��4C9��
���l�<{;��
5�-�i^�<����/�g����b��2X�l:�V�����t>����f����E~�{4RZb�2J��f������e��j�|5�8��L�-T:z;.f���|6|��Y���?��L7:�2h:/�������v�����#~g$&m���}�H���;��r�R���M��Muu9��i�[�5+�_�U��$�����f��a5��_A�]'���Q��t��`t2<^��B {+,�;�����;K���1���54fo�L�-���la�tiQ,X�5�����_�"��h��4�R�c=���(3sD3xH�Ng8:�g�!N��Q���T,�3o&��}����;:��d����NM|���s+������!'pX������%�zs���|	Y�F�*�:p��VV?sc��8��il^A2^�� s�@��['=E����$p���m��!��`��DN��lM���KY]
�M�/������������,��7�^�D�K���d����l�)%��[IZ�f�<aA�Om��E�D�o�x�����4d~i�JF*of�r�����o4�r���	���.���E�|g
�������)@����,�4��YT�
�
^��a7����t�U"�������o)���o������N�6�x!�]TG���n���%N���N[�)��!���2���B���U������r[��o2�j��C�f�.f�
�B�����Kyt���o��L��QX���������YO$����7^�?��6XL/G�6-x3����x��w����7
���m=M����K!���	YP����&���x_�oW�gxU�]�(�w�n�Mbo�L�����kPb{��������&��&c�����8R�W��\0�.�}oO��(z�=q�v[�R��-��]���t��^|����fk�g�� ��k��������SyM�
������}P�X�����������D,�f�z����������������kn���������	{pK���Ja�(b��[��R��j_
gi��l��N���!?�L��LT)��Yu-:+�=�WT���K�����d/�m��������Y�Rt�:%(�����Z��:�
DR�n���P�=H���$��S�J�ZI�Z��V���]���*���08���$�-������%
i��<�-QX����`K,�lv#&����0��@��>@z���
d����w�K��;�J�Yy	�6�3���H��G�h���*_�R��P3�x�������6����%xlB'�d�����j�����G-�o�h>�^��~������)�}�*�
���Z�H��^#�-,�������5�e|%�L�W�BZ������a��������������
42�Z��}�X�����E��h��A'����P�J�lw<���������&����
��2-.Q������c��T�����}�a��x�>p[��Z�/�E������65���f��s�CZgp�\E���V�6���&-	�����*uk��n-��1*�����M�Z�T�(O��J�6�Z��i�&��
�Jh�t���(�T��K���*
��R%���Bm�T�6KSvMn���;&����J��[�Q-��SDW�0O��1����~��P�������_�v@'FC��#��,��~�����(Z7��p���V�
�c\jsm��w�o�N�<�<����7Z���|���o�dW�
��-Y����FK�����T5���n��V�<n%�Z�|�Ov5!���'�&��	��"�����*ao�M#��J��m��V��N������3s9�$K�b�.������Hi?�;����8/���m�*rc����"/*l�i8�ai���/_��=�?~C�d��<�����/������3�NkJ��|1{�M�|w�`j�����|�y�����+�v3��t���p�{$d�a� ��f���\%4��r�2������'y\F�@�<
����J$`�4D��kN�nE"���9�����c2}d<_�����}I��)��^��u{u�*� N�aD�������w���5�
�����B����!�o�(��6CR��-7CR��!�o�$����~3D�WR��m6CR����n�Dl��=��6CL���o��1��jf<�l�_��t�x���y�x�[��������1������6�s��a�����k���.w�	q����k�tWaWo��������|�%�M/��j�G��B8�[V��'�������?LF�r��!�!1���Cj��$5���p��p���g���D	����b��C��1��"�d��|���P�q�v��9��v�Z��}���g~q6�c���9x;d��%����t}0������l�`.0&�sq����a6_L�� [�Pn����>|wO��7�y�K���u�zI�������wx�?��:|�*��B�����j9+"�������Wk���v��������@/Wr�;N�������P����V��W�\uO���#�O%z	[��z�s�>O5*�O��u����4g����l4�h������3�����:V���50
���=z�$j���w�z��3.�������U�A��"�T��J���9^��?�O���f�-���d��]�Z_��.t2�<N��{m�x!�ul�^���jb
,��D���3���%�Q����:��Ko=���b�VW�I�e�6�q8B�+�����%ct��1��K��Yj������7e�7�^\��8XYk��
>S`��{h����������{��d���a�L���������R'�j?�)��P]��E}r������d����zA{�eyF�Z�X�u��+�a�O����0�tF�$�"i�`�(�u�0�{r���}�>�@��}<T��~��rW���}��8�Y�2��R��	�w�}�Q{���3�h�zH���6H�
G�n��p'p[Z6��r����������W$��J-���������i|�?c���o�Vs����@����M^������������������_?%0��b�.��W������L����j���o�����?-w;�:St�zS�#gp�@�n2���������
Wl������Q����>����a����z��[�����O#�6f��E�l��3��
2^q��l\P�`:\�K]P^��t�$����I���P��$�6d[�d<�9�fWgAc��������t�{k��.��n�+��&A�(�(X����j��A�$������<��ALB�G�h5�������h�u��}��A|f�D<�
�~ LlfO}�PZ�:��	L���-���[�*~����n�]����X���_f����/^�������l���/^|����w���o�y�����������z��8�N�ON��n�0��Q������{�����=��gl3�

tK��D�H��.�O�LP����%�+�/bl����T��G��T��c�o�Q�#��S��L�<����t{!�S���hc���XrE�����L����?�F��h������K�v�(
L���|l���"[����L��u[��
�����P�����.�*a_�����5::�5:N�'��F��.�fPb9�X�VB��2����A�:h�
�o�����n�Z*����-�+O�1O�	Ou�1�t���vC�vv�W�� �mw��X�`����}8w�ba	E�]c�rn[K���a��p���.^��8R��(�Qq���qy�/�tO9�[�I���,}�$��[��d���o�F�E���v�������1�W,��I3a�Ue��{�Z�&�o��ZB��8��(�A��$�a;�m2�z��
~�VO���P���g\�����f����?� �C���c��11��w��s�����'"�Z���|���o��E~��g��	�]�,a^d�1���[��k�`�(�wr��n���~�F>���qRY���'���,�;�k(���b#�Sn��8�i�j���:�����m�x;����N`��i?���vt�R}<oX,A�wq
{��P�k��p�����������������.�
6�a������7�v��0�V��:�K5f�D���e�G����p��N�^���a����M�=�3�����d�|�@=jf$,'�F$�@	��7%Z�@��D��z�b�U��J��hX���O�T\v3��������iq��d���
|g'*K�Od��"!F�=���0R���Ph�>B;�U�����V���WS.S�x��<R�9��1M�bd �F�������Ps�����Sq��
%�5�������&��?Q��L���1�B����}��nHi@m��M=�������]�`-���1j�_?8[�����t���`wz�7������C���c&�V��~���9����o�������n>���0I����a6��z�������tt<��e���&����Z�S�tu�������^���#y�M~VC������m�Q����@r�����lD�N��,]�c�w���8B��K��-�Mt�����T����H�o�x�(#�����!�|�N�H��#���q^,pB��h_�3����E�|���%��5��%d��}U�d����~)���[�����O�������z�O4?�K�����:� �q'����*�b��=���y:�������U�A��Q��8�ssp��3r��;y4e]�D}�������~~7-]�B��.��k�M�rv��5�1s��M�NW{��Pf���+�h���	��^�Fg\}������V�l���D�B�?����1���NgW��~S�P�+�JQ5�v>�z$�E~�2�vG�a&�lK�&q���0:��"O���X7[�����p����G���������b<�o����-���z���g/�J����_�X��x�fw���`g�L���2��!l��e2�W�S.,T	��v>����bt/kD�u"0�A{����}n�X�j��L��D|3�zBz�@	4,��o����T��w4��-�v��24��.Z
�CC�f�z1�G^��e�����~��7M���*<����7�/���������|}�)�G�+�*������g#������0���(�����X���Q��
�����OKi�w���~h�N2����w����������P������=O��:%$?��g�D'�"��R�9�g��Jxb�<����V����F�%������R"+%�$�}}='�������>��~&Mg�4��:�nlj���m&f�yAd��������Ob���8+�Y�����
zY5�"��RV�|DK7
s
��1��6����^�{����Jp����������������S��� O.�vs�5�ss�d���y��U��/�X�|�HG#��G������jJ^���������`����>+�3^�����M '����!����`b�p���S[D�	3��"���l��@f�w�y
u"/���4^����������Y1dEn�a������n���.�)���4���u��u���;�c�r����KaI-���s4m��}��7�8��u-J3.V�6� z��HNY�$
Q�#�B>����18 � OY�k��+x�T�Y p���i$fQF���
�.9�|��rY1��$������^~���S�$��
����]�oIPt�\a���2��(�qWmuH���������A����B%�r����,iJc�cM�8��+m�����#M�����
��;;���+�;�k�m�� v���g�M�Z�.��8�1�a������<*�D}����=�pX��c��+���:�=���3��Y�f��*�M���`���h�n�C��5J����Y3���f"�M�I�����L�j<*�x���SI����a~����~S�Y9��{�b+>������������l�M�=|��g�$�|_��d�t�H��	�i@P<&���Mf�+��\;��:�QR����1�7&J�Y�H���k��1�#:���	K��zS���t�R*'3M��Lz
���q�	���S=����<���g�f!��;(Q�����X`"���Tn�|El�����c�#Q�lP7�����y���p)����3~,pQ��W7a�����	����5�E���c�]��>"4�4�2��r;��d�h�A���{5J`�>���� �Ly*s�M��>��.��^�~���&�����i.6����L�4���o�r��}��C"���	%��d�3&��q��_���E���|h�$����8�a��O�S<VD����T���H�'�C���f�m��hL���	�z�������$?�u#���o��N�{�$���]0\DL����ki�l��QV��a�*kh?��,�`�TL{Jk�.��+M��?{�2WuTWw���/0lC�P��=����L!vb4�;�k�m�����������9�B�@���x�v�t9���X��5��''�?���
��XKDr.��4�N)��o�M��Pm���Y��LW�uZ
�9|������%@���H��E\{;f�����w�!�xD�@y�r�g���=�h�����IYu�(E�$:*��_�x����Dpe��eZ�/+;3[��7����Y@h��������H?�;@���
�����%��o��z��U`���(�R�C_%�h��=@
2	R^7q'�����{[@����w��0�w�8���;^o����Q}�o-�tz���uf�p
�
������i�x\Y:�w>^,]��������X��D��`p������$\��0)*�2�_����Jf�:3`�Rk�V�����C�����zx���)�g�S�L��kz�Hf��am��}��!�#�����:���B����]��"�d��[u>��.������m�|����@3A����,c)����9Bm�L����w1���~��9>�q+�	��c�Mn�����@��01z���T��u`�����A�_=���kg�T�{l���XS3OV�����4�=����]�rO���|'3^��:�e������P�����x������S������i��O�Qm��J�5$��&������{�����Xdo�]B���k���W9���QC���`1������������[R�����`OV�}�5������^T����;��n�������V�
:�XeU�_����������/�A��x�A��f��J4�cm��k�M����%���F�T����_�]�k�.��
O��T����aZ�5N,6��}�,�yJ��e6fe�'�����C�$H�d ������A�sej������I 8�*�<�I
�9�����5�1
�lH��]wq�:(
�&���^p�`]�]4� A��i���zO���"l<�o1�
I����}*R�K���l��qt�U���������Yf���3�2eI��q�x�.gl����[���e�=Z(���5B������%z��j!,y�}���<Ab��.���	��~��6����I�OW�b���i�B
<B�p����iF�s0 JaDY��wGw�L
�V�n��l�(L��6������i]����������YC�`���Z���j��T���������g��hTz����N��&��+�3�.������t�\��:_$p�!����D��\����_bdcY�
W,z{�x!�>�P���s��+i��u�����CR������������~	��'O,�����!C�T�B��;X9�NV�d�����3�X��C����%U��R��[��K������BW��S���r� e��iE���j��'�6i4����J�-(�����=�PLxP���	^���_��I������6�b�z(�q3���p���Eo�`{v���n��'W�|�����Pc����T zh
�i��T\�~U������@�pC�jK���Df$�p+Et��V6Q�����Xbm}�=�v�{G�O��l����uz��Y��=��=�X��qG�����=�OV��"�h�"��5�n">��b[��+�+-�ap6�BWX������V�6�����m��Q���Vg��t%�j��e
�����%m�@@bB�L��f���B�B�RP#(u��ocus^w�6���1���4e;;Hm�JQ����L`���V���[����8^x�
��!�]��e6h0B������4j���H�v�Fp���4��F�/�����L���UZ��X�p�9�/�0H��/`n��������v]"
k7`D=��IF�f%GX�� }���/=�\���J
j�B8>�� lF^"�,��
��,�p\zv������1oh<�nF��,�f#q1�j�@aZ�X0����N�=��.����x,U_��N���v��>���R�jE���'��n������`Z�Di��Tc���g�1�l��`V��|�����(u�����6%xGD���uZ"G �Q�7lF
�W�%pC)������{���&�����}.w��vzE�F{=^7��}3Y��}������Y����7�G4ZIirD����w�������F"[���N�q6��OW%K������U
v�L3b)_���<�<'w�ju�f��w�4z���<��g��c�#�}��(���XD��R�$�Pz���w4!�%��d������r�����nu}i	5���g%����Z��2=��.f��x���������.0?���;rv:)��������X��O@��l����
�e���XZ��s�N������L�
��\��s����C�m�8�u������U;�)�gO"������d?��~�MR�]��(`"���9q| �0MFCB��p�C'04S@�1�Z��i����.?#n��r��c.l��q"�V����F8���Cw��s�A������x~�#�o��5%E�ns��vB��q~d��]�	����DZ�E$}��u[�j�MtJ�:����8P��b���N�����faK��r��#�u�:Ft.�����3���=�O�h��]3�5_��#h�����yC��Y4F���l.� �}���F����	8������i��`�RE\����pW`�a�;}�����J�l;���w*��~�r�@^��ch��?�^�aP|��E`�O
��P$�G���0��i�:�^BC�����"��W
�i\��D����A���e��+/0�&:�z�������,�3��%_��R
W���cC�|h�k��z��2+��l���+�B(��!%'�	�z5�����'����M�]�zh�#����'u=I!�������T�O�Y�Yh����fU��(�>��{.�����v������BS,��2���K�EY���>]�C(�[���A�����	�
Q��7 H_�X'�l,���R�)�c�z���_�S�oV?�oR��1��V)��D\Y���<Gq�{v=�+G������K������/)�Q�&di�FI�����0"��O�#a`�<�^,/e�=���w
cA�um��>$`�)��lR��p��l��1"���O�(�|�e�!����TZ����p�g����
���a��/P��d���$s����I6�!�;�ej1A=��������p��L�k���;�`����t��|IN�����8$����*�������Y�0q��TL.�����!����h�d�1���A�Y���N3��i�������5�@s��l���=�o�{eH�~o�uaS���l�+�d�	=k�Z>	m��h�o����]K����&�}�GY��V,W��P�ZZ��5�8)U\�&8U����E�Cd�\C�G����m�Ux����?���1g?z\�1W�i	��d���%D����s:��y[w�t��,���#���<�.U���zAl>�m���m�zh��������zf����j�[W�<���Wyt�U�y��sa�Y�<:���J��E������������h�{s�te��EJ�c������+��Y(Ss_�hO��y���<p��;�u�RRT��?�\�p�c"l��q�
U������]�o�Y
E?�"�W�����]�7{�N�s=6#� G�e���6������m���'�e{�B,nU	����*�/}��\Y�,�=������n�b�z��Y�w���W�����w��=6/�����PI,��a�k����g�`�����5XA��W3P>��,"�+����z�Mn�+gO�Fh^�#���.[h�"o�
�j_:O�W��n�o��[�)�S�*���o�����?��u��V�����.�3B�������Ne<�0Rq"�H1��'��&�����L���5�s�-���7`u�����[d�^7���z�M��l���������2A ��b>_���1�94C2�����"W��;IM�0�y�M����(���|��H7t
���	��-���|4
����� �0��
����W_&����V������LY��)-_������%PT��H���3�+9#O%+���y~y���[��������Q��9#�A�(���)a<>�I����N�mY�L����=�������j�������
�O
�%�a9���f��ZV�!���N+��l��������'�� Qy[�J+�H�n(%�5�?���]�J�f���/����l�I)��!s�u��F�����,A`���,%����$����rX��;S�5����7��2.��G3s�Y���|Y������������YNh-�	�����I����u;!��%�5FtH�\At��z{�S�S�$b����l)wBC�vs����5JX�^E���s������+�M��2�"��|�.���k�]�>0W_sU�2������a!D/����r���GC[��O����6M����4xv����4n�M���R~�FqJK6�W>
<.���L9g"���
�KO����
v���B���Xj�v=�]J$�.&3��b|�`i��L50��G�{}mL���d�i�f6�X��I�s����S�e.��zy���x]�9~�W�����M�"k�X")�l�>��f�`C����+T��U�]d���B�k �y������j�X��y�����E]������1�g�Tk7�;��iR����&��p���2�����K�w���HdKE�D�K<���.��9s���9��FL@��
�Y�Xx30E��(:@y����x���@�������I�q`����0z�R���V�������T��v%7b��u��qg[����<�f4I1������xt��m7���j5�J�K��X�Rq���Z�3�iG`��t�B����R���_?�~�` ���;�+J<�f+4S40t�q�������7������K���D�r11F�I@Ai�0�7��z"�d�DV�T������h!�b7e���"��J<��k�����DZ���"��fx�������[1�d��~���M���>�1�{q��lK�A>�:^y�l�K_��0c��S������y^�1�o
�EQN
f-�w��v���Qg�0�����P������u�+�E:����Rg����C?���, /���|	��^��$����d�������M
��<1�Mf�b�,Aj4Y��dz�+/�AY�S����X��!�����+����l7w�q�H+���V�������"K�w�e5C�}�bF�Q��8��������vo�3)z��Tr�+���A�9�����,i^2@����A:��������#���!6�����Q�7[��?`��w����������h�.����1!c4�N�*������o5^�����{/*5��]������{#���0���_�����g������O~��^�x+�u���
��doUE`��ZJ����r0V�����.�8@����uK���j��h�����r�c�*�L����2��]
0+;�p$����R�����r�lR��pe����1�.�C���D���������j2�nqX
��#������+����7�$�_�+I�x�Z�xxYM��t��g$7��dE�z�����d ��"����6�"�����[�M���S�<�����>���<R�c��;�2����H� ���=�x�8����W_��[�&���%]�Fy"]����
���9P
�p�K��)(�Yt:�������E#P�d<�K&��]W�x:�T���������/�q�2�����A����u��e�*���sI���4fz��l2&�3�S��L�����#j�����cF]��c��k����0]�[WE�������\�����7{PLF-��7�&8+7���)���������g��$�Jw�V\�-����{��Q��m�ah���/
��5	8��/	���p��6����	���.[�4T"�lT����x����l[�g�l�8�c�Q�9��:k���5i^(kd�������=-e�B���TT�Fi#���E�i8�
�� �6s���j2 Q*��e,�.����<���w�
�[&CSn`��5
�����Lq����2%��N^C�+)5u�T��q�(�D�O	��,QN.B������MC��$ET��S�-��X�[������ ��C�uKE�rG����}`^s�O�k��� ����g��|A��v+c���!Y�]��Y�b|���nk_<(jh��������3�l06�"bQ��SwU'}z�]���Ezn�F�./e�q�����%�W�+}@K�.���A�>��7K��K�[�;[<������i>����Y��%����(�t��w:<:R����~�upp������~�V��w�������j��=��-�����*�
�F�_�f���8�'Y�-=I�-E _p���>O�iZwA#g�?����K��[d����K������u`.���F�#��r,){5��GE�)���s:I�}1#:
����()���#]J�NB��U�Q�<���a���\�x�^��pn��)���r�-����U��2|s��F�\o2����G���),�q�}����>�`����gb������?�w��,VTI����B��f�5jh����l���I�� l�#����!HQ�<����n�
��L��U��
�"��.$��<5�>Ne����RX��Q?��G�F����iZ���c�������w0�-���ENP���|�b_]���H�x��������#������\�E��\�Q���E��ju���\�_����y���*m�����$�pt�����:�X���)�E���S5^�g�����DK���~�T�6f�B�M"+�q�E���=#g�sR����	C�X�2kgN�JYS��"8-�Ze<�P�8��g����^[}��e�-� ��#��I���6n3�%�|�L�L���WKP������Z�h���
:�qU�{)��O����N�E> %�;�J$F�g��lj.���3G�����!6_h��Y��wm������
%DV�HtiR#qQ����E�nf�l|�l�P��c5$�aE�C#s�����6��nH��,�G�����E,5�N�1t9Y<_�VC(V`�I��Q�/���Z��7�1M�l�@m>-����F�o��������������AR0&Z&c�!P7���D��;��{���<��'����+�%��"K�J
Ozd��o��J��0���W ����zO'������D|_�xi�����������v��n����*��x'a�^����,��gq;L��P��X�g�\l�(����y
�&�\Z��j�����(�E�,�5�`�ec��C����������\������WiLWJ4��T/GSH���^�����=�#��L:��K��u
��%���������&�6�k�<���Q_��l��e�qTHD�2��WK�y��QK{���s:t��jy]�c,��4) �7�����"�tZ��-�M�����H�n3$-�g 2����	��� x�U�#��m�{�P]�3@T%.�Hdg}���g������������(�wg�
p�'�.a�������^�p������
��P�����U`�}�z-U&�6T�;Fa7�.�~q��$)�_�yJ���J����t6�j��~1VS>����������I�DM��REq���=$u�}
�N�6�Cn�`z=����Ct<9�r�;
E��~�+�!��r���U�lx9�k��������#�XgSx��=���^�;����������^>��������~���������W�����|5���2[	��i~1[hb1A����,��'@uz�
PS����a��Lu�W�N3��N+$��*,E<<�I���V=�_{p��
.�L��e&�k������(��2��6%WS�I�c27�|�P�����w�<NX��Q�}�e#��Expr�~7'����������b��r����\�	��������w�e�;C]��O�����P�	���Ul��M�����{E��	���_
����G�pb&�6(n����(����o��8 X��������&�9�W������XFU���-6��1��8��sj��CIV�_�Y{��/�4a4�6�����"�M��H;��|�����a�{���f�@jO��xM?��q�Rj)U����Ru�'�Y�o�LQ�e�.�>��'tr���i��5H����\�,F�GS����<2�n�MY:�����Q�@LMi�hqc��v��'X��������v�c&~��2ye�!��OUa�����{��F�������	���� ���>�#�:�u:�^����z�?h
�I���W2�
\��`��MLo����*���E�|5��e��.���\2{]S���$y��t:�G�i�������p�|])X��SX�S���~�k2E�5����Ts-g��>������b�������T�^-a��g������J�������g���|
���U~����H���s#�}���'��CDC�U�&�R�J���K���g7Up��Y���]�q�"wkj�� �����|6�]���FF�@�����i�����_�3�8��$�������������A:�F�l9;cO2 8��q�g�����B(�����{rt(����~��u�zI��{����wx�O����h;](����\-gE���Y��O��
\-z'��^/�����h���G�y?&���A���I�������l���}x��S�^������<���>5s����C���|�e������Z��e������IW}�"����������f�z��W��?�����������Z�t������@��=��*DR��b��V�cR��)�G/�*U��t��9
!�(Pw��rr���E9hQ�\��c��j���y����o����C|��k����C���~�!H�$�!�KE��zZ(����������*��0�����:R�,����^���z�G�+�,��!�px������TOYS?Di��(�<_r��D$�#T!�z�.����CKv�W;���z�?����f��@�75s����L��]��n+h}u0��2����T_����/=������A�n������:/�}���W�������/��V�l4����C���.{�&*�1�.���L��*�M�j�vl-m>y��F��7�n<1�;7�q%����� �9W�\*�bF�@l�I�h��T;�E�u_Xo_g�����>%��$C��t�t��u��eG�RO��#H�v~��j���|x���?<<�t���A7�5�Y>�Z���u��w��`�7������q]z������92
����r;;��'�2��B:r�2z[���[�d���;��u@E�[�n
��E)�n��P��*B���E29����+L��A ��z��`eG>����E���}��4o�Y����	D��n�duF.�|c�{��/�o/�G�|(nTH�V�1H���v�C�����dRv�����"�!�i�����5�r��'.����E�5�D�3]Mx<���}�n������:�i����i����I��Iv��j�FFY�+$��a�H��O�J��/��^t�h������o�@�w�gQ$�*Ph/��xx���N��Z�Y1��\��&����5e!����(9�:8uP�5����-��y
w��s�B\�O~|���%h�E���]����_�)&�V��>v���������I��A���&Iz�?�F��^�x0|��������#�P�������<�#��GO�����]�{�
�?y�����pU �wu�����:|�	����j=K�X�|�u(��5��`�Kp��z���XQ����0�C�i��1��0K���	��B����!����HAF�E�|���_,��iJS�8w�0
���L8h:��*���4�����V����x�)}��W�h�h��#z��]����{���&�
_7
�x�-
S��Y&��c�<���S}���b
�M ��.�"���t��f�-��T���tZ��"�	S��g(��(b��\l��0�[�|�90d�5�R@���
��J�&q[���e��e6A�l�V	����C�KV��.r7[�n�ZI������Rw��A�@X��N2����i�^����K��C^��^���v���s�d�Gr�M����K^&�}u�=U���!T{=3C��e��Cu"0!�x�c��!���@��h�C��.��sS?V����O�M��	_E��1�jI��8N>y���:[���Qv��mC�b�&GGu�J��\�T�''j�R[xb�~b+��*�R�X��C�h��?
���nS�hl��J�$1�'Ywp�%�Q7==����Q�(�J�c+�rw��� L�@��$48�
'������A��bgg���Z�������U��8� ���Z���`�����1;@�\QBE�C540���lb`��77���7��J[X����M&&��S���l31��"�v1'�j�������+�<)�/�&a�\���A�0AU����(=�0R�c��:S�����G
���&�vJ���p��}/FTS*	&2�im-�r-5�!�V��R?��5������l91T��1/tDC��k�6���"r����w^D52U�S�C��W�����
���j��:{����$��C�/$����nB>������-
�,06|]�%
q^�.<r�H,57Q�D0c�^7VDA�(�2�H
���"c)��m
����
�N��R�������T�?��
v���ru�1^�v��}��:�����3��K�c`
��0R�IptM�U6���� ��������o�������UP^������?���"M�B�����#"����e����c2�gA�%�:����J��1L��Db�5���#����3~�k�}����r�.�.A�uH;���������
/.� Z$���B��5�^��`�4�_�D�f����j0
��<�`"�5��x�����m��jJ�!�X�f���}��?�@&�,����������
��2f6�#�&Wr�#il� dxh�d(K��kO0*9�#H����6����Fs��RkUw>�R
�D�4��gk�e���s^Q����Z��7e2���,��|���`��A8��MQ$s�����\GZs%��y��c�=��q}*7v�m���"
��2=u��!�4@���E�X9��H��^���'��k&,k�2���V��6��
!
�����N"��K����y�W?�U��($�)OQ�nb��q�vMj���X1	M�%o�?`�\l< ���%���f�[��<r����Ll��!��-T���1��;�=:�d#�w�_K?)��I��G4���FK������
��������h����X���I��1�]������X��
~��c:�9=t�@�"������������PYS���S�4U��~�;��X�����t��64�y�.Qo��?{�2WuTWw���/f��I*.���<o,?m�����Lr�)������|R�XH�Q]O��cT���_\�����^	
S��q��(F}!-����@��}����jc��48���f����j���C����\&/B�t-eRD�x������x���3��E�1�o��}9+�����t�A�+��E�j���/�����h^f�����n��=w�������������bGxPvp~�	�3�Y���-��������k^�b��W����tK)}���=��5��%�`�������Q�����6��Ax��5�9������`feN�
������N/[R�����f�{��>�������P{>^&=^�����n��a�:��� ������v=AG�h���mdG(����qJ�YZ-4*�Z[�{��r��C�z���j�O�3����(�9���^�DT�\]�,�����\�W��M
Z�����q[c�K#Mu���E��l[���8'/�	2�NeKa��.����X>)�?��H��HnE�������&�or~#S�Ds!y@
E�D��ra�Ra����9�'���=�W�z��dS9���in{��aU�n��6^����g�<��u���q3�����?8E89����o���U����7��I�3�-�`HMX�i��
�R#�!����g+OH��IoY|$��fN�I] tT���/����[����o�a�(��0��G^Sb��=�o�j����#}���9��q��@��t��,��h� �k��E��N�F���C?^���yZX%El���-�)��1�X����u*���F�T����_���b��f��=U�SM{�OdZl�p��8��8V`0��T�)	���7)y\8����Tr'��<��\�$�������9h}�J������Vt�|[�
(OxRl�x�_N������!�Ot����(@`���}b/�l�.�.V���`W'�^G��S�	��Vt��vC��+g2��H�.2����,h�L�.
�=��]v&rBf��,I�?�o��lQ� /��o9���P�h��E�t>�V��
��n2Z�KA���J��<Ab��.���	��~��6���w��������&/l�5��eH
L6e�7Fy��gW"B��H��6�7�'6<���ad��ixZ�{��u����5rb�P?J��#��73��W�^���C��#�����w���$�?.=��OL��W:�V���G��A�C������G�$�)��,`,+�A��E`�/$�'c
�U~��w� 
C�.���N����M�(`��Mdo,��_2���>���{�P(�P���V����@)�1�E���^�4�[RE�)&]�E�5����(�C7���x]M��D�&��U[Ui�E�J����Pnp]���n����E3�m���?�P��f�%�!7��
����1�P�-c��
c�P^�]��C����n�S���)<���Rq��U�{Ob�.P��
�-��J��4���AV�[��f�#"dm}�=�v�{G�O��l����uz��Y��=��=�X��qG�����=�OV��"�h�"��5�n">��b[��+�+-�ap6K�I�������6�o���6�y��n5�Nlu��HWr�v��Y�9+��X��$&t���h)�(�-5�R���6V7�u�j�hM����`�y/��A�'���I������V��{�f�2�]��!�]��e6h0B������4j���H�v5���w��4��F�/�����L���UZ��X�p�9�/�0H��/`n��������v"�|�U���L�'���a����
�c{�k`�c�����b���� ���������fs�9�8����B������0�m�0e��H\L��%P`������;��`����o���TAx|��;�bO����v���*V3(���=�F�w+���x��<��%J�o ����1Ej����.��;�Q��{-q�m$
J������%�(�D�@��No�	)�^�g��
����z�W������z�����8�i����x��n��d�v�����~w8f���C�0H�h%��}��7c�����vv:���=v�Sp��������d��13{;�J����iFL�9'm��K��NP�.�l]�N�F�{���>�L�vLD�/�����(�V���JoR�����aiQ�JF:�(O*�\i��V����P�+xVB[
����*�C`y��b��7�������;���Ch}��#g��B|���8��dU�g��Z��+0�]��=����w2�<��4�
J���#���a�f������N|�����\`H�>{Q�W/�&��������d������\��]~�&?#�T8����
&ZK<m3>#��g���Y}���r6N�
|{C���?�u���`�4�E�9�F�_������xMI��\c��P�T?�g
�����a
G��g���o#[{�n�Ym���AI_�Z|��<Xy��	:5�[�,l)yUN3{y�.YG�����2���z���r�g�	t�T�k��f�c�+�y�M��6�`��5������
�������Hn������t��J��q���9Jq-��t��]�A���0�=CV��m�]�n�@e���XNR��+zw`��o������4��}�'���A��g�M�#��e�V�4B���
/����f�oF��?�+��(PZCr2����Pp�f��D�R�z�s�9Cp��w�q��+z�sua�;\_��
-����_������g��b��ly����� 't��,Nf��r�wRk^d�Z�����>1r-�*{R��"Y��,+���e����!`t�Y���M����#P�}���\��Ca��r��E1��t�����/�e�J C�t]���o�SaG>��'�6Di_��H�p }�c�������K��t2����i�"�T�w��������M� Er9����D\Y���<Gq�{v=�+G������K������/)�Q�&di�FI�����0"��O�#���b))�������SS���RR.Y�B����fT�yx}�
q���,���I��_X�x���a�I��yH�+���I6�-�A2G���i��d�2��!X������<,{�g�����]Z��[e�@w�|IN����������U�1�5S	���t!���p��"A����!"�[��O)������.!H�6m���WC���c@� :���f�W��7�f\6uM���H��:�F[/�l�uh�j�$��?����_#l4v-�'�'���=e��Z�\Q8GBMjiI3����TqM��T�?��U��[pI��?��V���������	��qy�\a�%K�Ms:���J?����l���%F���������,
���23����#�&^n��6��F+����Y���g����0���u��#�y�Gg^����=:��E��sQ������\��<�Z��i����V�G1wL'P�N1�X��q?�K/��ia�����25���F�tPy�::Q��|�#^�A/%Eu��3�U��1&B��X7�PeHM�(\��%�F��?��sM��3e_h|O��g��=q��tXR�xg?�W�X���f�����oN�G�f����"�\r2���������l�Y���*�zQ:UE�����+�%�'7�tTy��U�[O�4����y��=}�Q��������*��v=,q�6������xZ�� ���j�g��E��b����]/;c����z�����K�_��e�U�mT�U�K���j������y�4Er�^���@�-���������
�}����uF�T_�b������gF*Nd�#&��D����������9"�z��[���n�]�x�l���&{0R/�	�Mq]����P�rA&y_����|11�fHF��QR]�@��p'���3��	�;#�%��\��|���A:5���B�e�����F��T�[�&�Ay�Z����_�����6�T�)��1����wAZ7�����)���bft%g��d��3�/�]�t�<S��:?�b4J61g$ >(E��?%���9)9q�i�-�c�	�A���gt��1�PM�3Q�|\!�I��d8l"�5����T��4�T6�ie����\��V��D�$*oK5@iE��
�d�&��W��k�U��lY�e1���-@=)e�1d��������1>���%������������d�x2��Bk�|�@b����������S���hf`�9K5q�/!�y�[#��6ZS��%nw�R��E�d6huGm����p�NH�fI�R$W�2�������%��|z�G7[�������6`��V�WQ�z���jm6{-�J`�&����%;����n;�����EW�L���\�����e��fxX�K;�e��ze����k������M�p�%7
������7�[w��b����Q���������F��*2!S����A��B����5g�d���3���{(�Z�]�g������L���+X%��S
h��Q�^_��9rFZ��
A�V�+c����*���|��v�^8f3^�n��UD~srE�s���Z0�H�?��b��%�� �'}��
�m�sY+a��Hr�G#g����Z*��`�.4�g��AbQ�8v��uf��Y=���
�;a��(��I� m"��q����R�����1�RQ/Q����f4�K6~����`���P,��{�$V�L�����P��"-0^��#6�m!4p{���p2r�����,x��*��,�b)U��]���A�u]zq���*Gsn4��MR���5�}b:�y�����Z
��c4��T\d�s�V��j�Xt3�P?�'�T�l������_�(�������
�
�v\b;�*����$qF $���/�+&�\LF��hPD�EZ:�
������!3Q����i����@<�FZ���C%����{��%�Z�<���<���u���0���6�r�>��VL��#Y��_~�j�c���"@�n�^�n8���DP��#��W�1[��t5���d�������x��r�[�pQ�S��Y��'�`�o������'E+�'#�tvep~l��
u�N��>w����&��`!(��,1_���u<	,��G5���m8es���<Oi����&K�MV�,�E���vPV���1C��8V�g��}�6�
6zj<���j�/�JB,f��0s�Gl,pD'����rY��k����-F��#��d~zsi���[�L��0��
"�pPc�q��e,'K��<�)Cy����2a:1���xfhH����"�v���V��Xh��bt�y�;����+����,#:�jL��M�S���1�6/q��[
���;�����J
�wWx}�������{t4L�������1�Yp���(��_�����9�
o��nw���Y�[UX+����a-����t��2��#��7q�w�R�|n�Z�,y0,�����8�
?�?����2f����&I����T}g���\�;�T�7\��?��{���`r2�-�*����j��L�[V���H�)���jD����Fh�Mt�n��J�����6^V�1B-wF��M�)YQ�^@9<����z�8����s������a2c�V{����:g��G�x�����&����r�����1$�'�zez"�.a/�������	�jxI��Q�H��4��F��'vTCn��bo
��i�`#"�n1w�%���I�d��$�N=�~)a�����`���e�:��o�)jP2��l]�c��)��\~u6����"��	�����0����l��Z0�� x��Q�������cl�#L���U��a��n~;W��������Q�b���	���
8�pJ���unsA#"��l%������z���0l#�^�)v��}z?1�K��xM�9n�Kb8, �
�lD�dB%m���1
��0��~�;���@p�:���:>N�oTb�l�������AM���G��*��5zOK��5v�A/��Q�(m|�iNjC�*H����j��H���(A�����##O������������&y�C��nyh!S\.�e�L�:������EJM�#U�h�6�'��S�m<K����:�-��r�fG+I�%��}�v;���G���1<�����z�RQ���=�����S��2"$H�d�����:_�)�]���G/kH�z���GV��h������Z���v�5����G'[D�M��X�F���D�I�^�lh9E�������K�w\���{~I��J�����p,|P,��������R�V�������v�O/:C5hV�E}�z��?�:�$����T�����o4m���������]#s��a�����nO�g���8�@$8�J�B�Q����-4��I�wKO�vK����z���q�����]���Y����i������&���0j��~m����Q��s�K�^
/�QE�
���\�N��C_������p(=J��g��H�R���zd�.F��1�b��<�,�h��=0�dj�����g�t1���CU�����!��Q6��Lc>���i�w
|�o��d��"��~q���b���������(�U�!1��P���`�Z�ye4�@��b�.9[�a84qGR�#OsE��F�����`+({�yi���2��	�&O���S6-����m�� ���`-fm��b&����4.-�����xK��*G�Td4A5_���W��(2�:�?�+���mt��n�=�/F�)�y�z0�m#��}���Z]vy=��W�����i/n�J�w�}���|8I8]gjt���!�z�}J�EQ���T�W��2�(>�Rz�p��/��Y����E��D��zG��g��C�s�������d'��(�����S�A��!�NK�VO-%�����?���V#yd�+����� �������%B�;#�����������j���/������Ni\��^�y:�S�r��j�H���A��Q���r4��KrF���6�w�g��Z"zV�6�]��0��n��B	�� ]��H\T�pb`��E:��[���D
�yX�����m`6�
x���=�r��d"7�iKM�d]N�Wk��P���pE�w�����l����k�@DS%'P�O,��0G�Q�[3'i������g���k�T����X{�M3��#�����������*���%��6����z����R�����Yg�&��>�����;;%������%�xaj:����3^����}�y�~���=���)�J�+�I�Wo~� Kf�Y��$��=V��/[k�F#���w���7��3��53��1�p�-vM/�h��a�9������(i+�)�x1��U��
k5���/n����=:jw��Hr/�N����c���jI��������#���MD��D6O�pv���+��fYg�����y^:�G����p��]$�Z^��.(M
H�M46&n��0����gs�?#;�%�D��I���L���~Dks;^tU���p��^>T����G����Y_2�����Yz�b�C�=> w��9����x����K��6�D�6�|��>�?9nw�w��<�*����nXf��^K���
U��Q�M�K�_��%I
�v������R�����>���=�_�����a2�v:G���i�5QS����AQ\��nI|��������[&�^�u��F�O�x����B��?��Js�:A/�p�1}�2^N��Z��4. ��H2����w�g�����N:>:n'��<�0����9�������|s����_����.�A@��=_M����VB�s�A�_��XL�o"�1Kw�	P��B�������uXs8A]f�U��L���
Im�
�@Og���U��������:��j�I�j-�7r�J����M���u�����:_;T"����� ���bTy�e��<k���_��I8p���p(w#��X��\�)h �h���v��� �a�]i���D���D�srp!�B�.r[?$BSp�,x�^��s��1�����h��0��I�
�[y��'���}��o ����b�bm����zN�Uh�Dg�2�=V�E�QU"�m����D�s<����C��P����Wg�����3M���
�� 5���mj�-��(&�����|?|����v�0���D��^���n�A���ZJ��`�T���iV�[*ST�EY���Oh�	��)��D��d
�!au�+�������d�"���[pS�����~|�<SSZ3Z���;�	�eb��+l���]����_,�L^A�x��(�SU����b���^����a����{B�-#6��?�O������g���/�p��^���C~AR�������)(X�uG�<$���8 hQ4_���t����vb:��^��`n>I���'����q�??-s�u0#_W
��������_��L�c
2~�%�\��r�O<�}����#�w>�"��WK�����e�q��x���) �Y��%_C#�a�_���+R�_0���_q6��$����e��I���(g���#��M��iV)i{Wj\�����k4�������9��z8G���Q:P�n��s�9j���������������������������������������������������������!�
#189Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Amit Langote (#187)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Mon, Sep 4, 2017 at 10:04 AM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

On 2017/09/04 12:38, Etsuro Fujita wrote:

On 2017/09/02 4:10, Ashutosh Bapat wrote:

This rebase mainly changes patch 0001, which translates partition
hierarchy into inheritance hierarchy creating AppendRelInfos and
RelOptInfos for partitioned partitions. Because of that, it's not
necessary to record the partitioned partitions in a
PartitionedChildRelInfos::child_rels. The only RTI that goes in there
is the RTI of child RTE which is same as the parent RTE except inh
flag. I tried removing that with a series of changes but it seems that
following code in ExecInitModifyTable() requires it.
1897 /* The root table RT index is at the head of the
partitioned_rels list */
1898 if (node->partitioned_rels)
1899 {
1900 Index root_rti;
1901 Oid root_oid;
1902
1903 root_rti = linitial_int(node->partitioned_rels);
1904 root_oid = getrelid(root_rti, estate->es_range_table);
1905 rel = heap_open(root_oid, NoLock); /* locked by InitPlan */
1906 }
1907 else
1908 rel = mtstate->resultRelInfo->ri_RelationDesc;

I don't know whether we could change this code not to use
PartitionedChildRelInfos::child_rels.

For a root partitioned tables, ModifyTable.partitioned_rels comes from
PartitionedChildRelInfo.child_rels recorded for the table by
expand_inherited_rtnentry(). In fact, the latter is copied verbatim to
ModifyTablePath (or AppendPath/MergeAppendPath) when creating the same.
The only point of keeping those RT indexes around in the ModifyTable node
is for the executor to be able to locate partitioned table RT entries and
lock them. Without them, the executor wouldn't know about those tables at
all, because there won't be subplans corresponding to partitioned tables
in the tree and hence their RT indexes won't appear in the
ModifyTable.resultRelations list. If your patch adds partitioned child
rel AppendRelInfos back for whatever reason, you should also make sure in
inheritance_planner() that their RT indexes don't end up the
resultRelations list. See this piece of code in inheritance_planner():

1351 /* Build list of sub-paths */
1352 subpaths = lappend(subpaths, subpath);
1353
1354 /* Build list of modified subroots, too */
1355 subroots = lappend(subroots, subroot);
1356
1357 /* Build list of target-relation RT indexes */
1358 resultRelations = lappend_int(resultRelations,
appinfo->child_relid);

Maybe it won't happen, because if this appinfo corresponds to a
partitioned child table, recursion would occur and we'll get to this piece
of code for only the leaf children.

You are right. We don't execute above lines for partitioned partitions.

By the way, if you want to get rid of PartitionedChildRelInfo, you can do
that as long as you find some other way of putting together the
partitioned_rels list to add into the ModifyTable (Append/MergeAppend)
node created for the root partitioned table. Currently,
PartitionedChildRelInfo (and the root->pcinfo_list) is the way for
expand_inherited_rtentry() to pass that information to the planner's
path-generating code. We may be able to generate that list when actually
creating the path using set_append_rel_pathlist() or
inheritance_planner(), without having created a PartitionedChildRelInfo
node beforehand.

AFAIU, the list contained RTIs of the relations, which didnt' have
corresponding AppendRelInfos to lock those relations. Now that we
create AppendRelInfos even for partitioned partitions, I don't think
we need the list to take care of the locks. Is there any other reason
why we maintain that list (apart from the trigger case I have raised
and Fujita-san says that the list is not required in that case as
well.)

Though I haven't read the patch yet, I think the above code is useless.
And I proposed a patch to clean it up before [1]. I'll add that patch to
the next commitfest.

+1.

+1. Will Fujita-san's patch also handle getting rid of partitioned_rels list?

--
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

#190Etsuro Fujita
fujita.etsuro@lab.ntt.co.jp
In reply to: Ashutosh Bapat (#189)
Re: Partition-wise join for join between (declaratively) partitioned tables

On 2017/09/04 21:32, Ashutosh Bapat wrote:

On Mon, Sep 4, 2017 at 10:04 AM, Amit Langote

By the way, if you want to get rid of PartitionedChildRelInfo, you can do
that as long as you find some other way of putting together the
partitioned_rels list to add into the ModifyTable (Append/MergeAppend)
node created for the root partitioned table. Currently,
PartitionedChildRelInfo (and the root->pcinfo_list) is the way for
expand_inherited_rtentry() to pass that information to the planner's
path-generating code. We may be able to generate that list when actually
creating the path using set_append_rel_pathlist() or
inheritance_planner(), without having created a PartitionedChildRelInfo
node beforehand.

AFAIU, the list contained RTIs of the relations, which didnt' have
corresponding AppendRelInfos to lock those relations. Now that we
create AppendRelInfos even for partitioned partitions, I don't think
we need the list to take care of the locks.

I don't think so either. (Since I haven't followed discussions on this
thread in detail yet, I don't understand the idea/need of creating
AppendRelInfos for partitioned partitions, though.)

Though I haven't read the patch yet, I think the above code is useless.
And I proposed a patch to clean it up before [1]. I'll add that patch to
the next commitfest.

+1.

+1. Will Fujita-san's patch also handle getting rid of partitioned_rels list?

No. The patch just removes the partitioned_rels list from
nodeModifyTable.c.

Best regards,
Etsuro Fujita

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

#191Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Ashutosh Bapat (#189)
Re: Partition-wise join for join between (declaratively) partitioned tables

On 2017/09/04 21:32, Ashutosh Bapat wrote:

On Mon, Sep 4, 2017 at 10:04 AM, Amit Langote wrote:

By the way, if you want to get rid of PartitionedChildRelInfo, you can do
that as long as you find some other way of putting together the
partitioned_rels list to add into the ModifyTable (Append/MergeAppend)
node created for the root partitioned table. Currently,
PartitionedChildRelInfo (and the root->pcinfo_list) is the way for
expand_inherited_rtentry() to pass that information to the planner's
path-generating code. We may be able to generate that list when actually
creating the path using set_append_rel_pathlist() or
inheritance_planner(), without having created a PartitionedChildRelInfo
node beforehand.

AFAIU, the list contained RTIs of the relations, which didnt' have
corresponding AppendRelInfos to lock those relations. Now that we
create AppendRelInfos even for partitioned partitions, I don't think
we need the list to take care of the locks. Is there any other reason
why we maintain that list (apart from the trigger case I have raised
and Fujita-san says that the list is not required in that case as
well.)

We do *need* the list in ModifyTable (Append/MergeAppend) node itself. We
can, however, get rid of the PartitionedChildRelInfo node that carries the
partitioned child RT indexes from an earlier planning phase
(expand_inherited_rtentry) to a later phase
(create_{modifytable|append|merge_append}_path). The later phase can
build that list from the AppendRelInfos that you mention we now [1]https://git.postgresql.org/gitweb/?p=postgresql.git;a=commit;h=30833ba154 build.

Though I haven't read the patch yet, I think the above code is useless.
And I proposed a patch to clean it up before [1]. I'll add that patch to
the next commitfest.

+1.

+1. Will Fujita-san's patch also handle getting rid of partitioned_rels list?

As Fujita-san mentioned, his patch won't. Actually, I suppose he didn't
say that partitioned_rels itself is useless, just that its particular
usage in ExecInitModifyTable is. We still need that list for planner to
tell the executor that there are some RT entries the latter would need to
lock before executing a given plan. Without that dedicated list, the
executor cannot know at all that certain tables in the partition tree
(viz. the partitioned ones) need to be locked. I mentioned the reason -
(Merge)Append.subplans, ModifyTable.resultRelations does not contain
respective entries corresponding to the partitioned tables, and
traditionally, the executor looks at those lists to figure out the tables
to lock.

Thanks,
Amit

[1]: https://git.postgresql.org/gitweb/?p=postgresql.git;a=commit;h=30833ba154

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

#192Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Amit Langote (#191)
Re: Partition-wise join for join between (declaratively) partitioned tables

On 2017/09/05 13:20, Amit Langote wrote:

The later phase can
build that list from the AppendRelInfos that you mention we now [1] build.

[1] https://git.postgresql.org/gitweb/?p=postgresql.git;a=commit;h=30833ba154

Looking at that commit again, AppendRelInfos are still not created for
partitioned child tables. Looking at the code in
expand_single_inheritance_child(), which exists as of 30833ba154:

/*
* Build an AppendRelInfo for this parent and child, unless the child is a
* partitioned table.
*/
if (childrte->relkind != RELKIND_PARTITIONED_TABLE && !childrte->inh)
{
...code that builds AppendRelInfo...
}
else
*partitioned_child_rels = lappend_int(*partitioned_child_rels,
childRTindex);

you can see that an AppendRelInfo won't get built for partitioned child
tables.

Also, even if the commit changed things so that the child RT entries (and
AppendRelInfos) now get built in an order determined by depth-first
traversal of the partition tree, the same original parent RT index is used
to mark all AppendRelInfos, so the expansion essentially flattens the
hierarchy. In the updated patch I will post on the "path toward faster
partition pruning" thread [1]/messages/by-id/044e2e09-9690-7aff-1749-2d318da38a11@lab.ntt.co.jp, I am planning to rejigger things so that
two things start to happen:

1. For partitioned child tables, build the child RT entry with inh = true
and also build an AppendRelInfos

2. When recursively expanding a partitioned child table in
expand_partitioned_rtentry(), pass its new RT index as the
parentRTindex to the recursive call of expand_partitioned_rtentry(), so
that the resulting AppendRelInfos reflect immediate parent-child
relationship

With 1 in place, build_simple_rel() will build RelOptInfos even for
partitioned child tables, so that for each one, we can recursively build
an Append path. So, instead of just one Append path for the root
partitioned table, there is one for each partitioned table in the tree.

I will be including the above described change in the partition-pruning
patch, because the other code in that patch relies on the same and I know
Ashuotsh has wanted that for a long time. :)

Thanks,
Amit

[1]: /messages/by-id/044e2e09-9690-7aff-1749-2d318da38a11@lab.ntt.co.jp
/messages/by-id/044e2e09-9690-7aff-1749-2d318da38a11@lab.ntt.co.jp

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

#193Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Amit Langote (#192)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Sep 5, 2017 at 11:54 AM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

On 2017/09/05 13:20, Amit Langote wrote:

The later phase can
build that list from the AppendRelInfos that you mention we now [1] build.

[1] https://git.postgresql.org/gitweb/?p=postgresql.git;a=commit;h=30833ba154

Looking at that commit again, AppendRelInfos are still not created for
partitioned child tables. Looking at the code in
expand_single_inheritance_child(), which exists as of 30833ba154:

/*
* Build an AppendRelInfo for this parent and child, unless the child is a
* partitioned table.
*/
if (childrte->relkind != RELKIND_PARTITIONED_TABLE && !childrte->inh)
{
...code that builds AppendRelInfo...
}
else
*partitioned_child_rels = lappend_int(*partitioned_child_rels,
childRTindex);

you can see that an AppendRelInfo won't get built for partitioned child
tables.

Also, even if the commit changed things so that the child RT entries (and
AppendRelInfos) now get built in an order determined by depth-first
traversal of the partition tree, the same original parent RT index is used
to mark all AppendRelInfos, so the expansion essentially flattens the
hierarchy. In the updated patch I will post on the "path toward faster
partition pruning" thread [1], I am planning to rejigger things so that
two things start to happen:

1. For partitioned child tables, build the child RT entry with inh = true
and also build an AppendRelInfos

2. When recursively expanding a partitioned child table in
expand_partitioned_rtentry(), pass its new RT index as the
parentRTindex to the recursive call of expand_partitioned_rtentry(), so
that the resulting AppendRelInfos reflect immediate parent-child
relationship

With 1 in place, build_simple_rel() will build RelOptInfos even for
partitioned child tables, so that for each one, we can recursively build
an Append path. So, instead of just one Append path for the root
partitioned table, there is one for each partitioned table in the tree.

I will be including the above described change in the partition-pruning
patch, because the other code in that patch relies on the same and I know
Ashuotsh has wanted that for a long time. :)

Those changes are already part of my updated 0001 patch. Aren't they?
May be you should just review those and see if those are suitable for
you?

--
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

#194Etsuro Fujita
fujita.etsuro@lab.ntt.co.jp
In reply to: Amit Langote (#191)
Re: Partition-wise join for join between (declaratively) partitioned tables

On 2017/09/05 13:20, Amit Langote wrote:

On 2017/09/04 21:32, Ashutosh Bapat wrote:

+1. Will Fujita-san's patch also handle getting rid of partitioned_rels list?

As Fujita-san mentioned, his patch won't. Actually, I suppose he didn't
say that partitioned_rels itself is useless, just that its particular
usage in ExecInitModifyTable is.

That's right. (I thought there would probably be no need to create that
list if we created AppendRelInfos even for partitioned partitions.)

We still need that list for planner to
tell the executor that there are some RT entries the latter would need to
lock before executing a given plan. Without that dedicated list, the
executor cannot know at all that certain tables in the partition tree
(viz. the partitioned ones) need to be locked. I mentioned the reason -
(Merge)Append.subplans, ModifyTable.resultRelations does not contain
respective entries corresponding to the partitioned tables, and
traditionally, the executor looks at those lists to figure out the tables
to lock.

I think so too.

Best regards,
Etsuro Fujita

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

#195Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Ashutosh Bapat (#193)
Re: Partition-wise join for join between (declaratively) partitioned tables

On 2017/09/05 15:30, Ashutosh Bapat wrote:

Those changes are already part of my updated 0001 patch. Aren't they?
May be you should just review those and see if those are suitable for
you?

Yeah, I think it's going to be the same patch, functionality-wise.

And sorry, I didn't realize you were talking about the case after applying
your patch on HEAD.

Thanks,
Amit

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

#196Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Amit Langote (#195)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Sep 5, 2017 at 12:06 PM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

On 2017/09/05 15:30, Ashutosh Bapat wrote:

Those changes are already part of my updated 0001 patch. Aren't they?
May be you should just review those and see if those are suitable for
you?

Yeah, I think it's going to be the same patch, functionality-wise.

And sorry, I didn't realize you were talking about the case after applying
your patch on HEAD.

Ok. Can you please answer my previous questions?

AFAIU, the list contained RTIs of the relations, which didnt' have
corresponding AppendRelInfos to lock those relations. Now that we
create AppendRelInfos even for partitioned partitions with my 0001
patch, I don't think
we need the list to take care of the locks. Is there any other reason
why we maintain that list (apart from the trigger case I have raised
and Fujita-san says that the list is not required in that case as
well.)?

Having asked that, I think my patch shouldn't deal with removing
partitioned_rels lists and related structures and code. It should be
done as a separate patch.
--
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

#197Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Ashutosh Bapat (#196)
Re: Partition-wise join for join between (declaratively) partitioned tables

On 2017/09/05 15:43, Ashutosh Bapat wrote:

Ok. Can you please answer my previous questions?

AFAIU, the list contained RTIs of the relations, which didnt' have
corresponding AppendRelInfos to lock those relations. Now that we
create AppendRelInfos even for partitioned partitions with my 0001
patch, I don't think
we need the list to take care of the locks. Is there any other reason
why we maintain that list (apart from the trigger case I have raised
and Fujita-san says that the list is not required in that case as
well.)?

AppendRelInfos exist within the planner (they come to be and go away
within the planner). Once we leave the planner, that information is gone.

Executor will receive a plan node that won't contain that information:

1. Append has an appendplans field, which contains one plan tree for every
leaf partition. None of its fields, other than partitined_rels,
contains the RT indexes of the partitioned tables. Similarly in the
case of MergeAppend.

2. ModifyTable has a resultRelations fields which contains a list of leaf
partition RT indexes and a plans field which contains one plan tree for
every RT index in the resultRelations list (that is a plan tree that
will scan the particular leaf partition). None of its fields, other
than partitined_rels, contains the RT indexes of the partitioned
tables.

I learned over the course of developing the patch that added this
partitioned_rels field [1]https://git.postgresql.org/gitweb/?p=postgresql.git;a=commitdiff;h=d3cc37f1d8 that the executor needs to identify all the
affected tables by a given plan tree so that it could lock them. Executor
needs to lock them separately even if the plan itself was built after
locking all the relevant tables [2]/messages/by-id/CA+TgmoYiwviCDRi3Zk+QuXj1r7uMu9T_kDNq+17PCWgzrbzw8A@mail.gmail.com. For example, see
ExecLockNonLeafAppendTables(), which will lock the tables in the
(Merge)Append.partitioned_rels list.

While I've been thinking all along that the same thing must be happening
for RT indexes in ModifyTable.partitioned_rels list (I said so a couple of
times on this thread), it's actually not. Instead,
ModifyTable.partitioned_rels of all ModifyTable nodes in a give query are
merged into PlannedStmt.nonleafResultRelations (which happens in
set_plan_refs) and that's where the executor finds them to lock them
(which happens in InitPlan).

So, it appears that ModifyTable.partitioned_rels is indeed unused in the
executor. But we still can't get rid of it from the ModifyTable node
itself without figuring out a way (a channel) to transfer that information
into PlannedStmt.nonleafResultRelations.

Having asked that, I think my patch shouldn't deal with removing
partitioned_rels lists and related structures and code. It should be> done as a separate patch.

Going back to your original email which started this discussion, it seems
that we agree on that the PartitionedChildRelInfo node can be removed, and
I agree that it shouldn't be done in the partitionwise-join patch series
but as a separate patch. As described above, we shouldn't try yet to get
rid of the partitioned_rels list that appears in some plan nodes.

Thanks,
Amit

[1]: https://git.postgresql.org/gitweb/?p=postgresql.git;a=commitdiff;h=d3cc37f1d8
https://git.postgresql.org/gitweb/?p=postgresql.git;a=commitdiff;h=d3cc37f1d8

[2]: /messages/by-id/CA+TgmoYiwviCDRi3Zk+QuXj1r7uMu9T_kDNq+17PCWgzrbzw8A@mail.gmail.com
/messages/by-id/CA+TgmoYiwviCDRi3Zk+QuXj1r7uMu9T_kDNq+17PCWgzrbzw8A@mail.gmail.com

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

#198Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Amit Langote (#197)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Sep 5, 2017 at 1:16 PM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

On 2017/09/05 15:43, Ashutosh Bapat wrote:

Ok. Can you please answer my previous questions?

AFAIU, the list contained RTIs of the relations, which didnt' have
corresponding AppendRelInfos to lock those relations. Now that we
create AppendRelInfos even for partitioned partitions with my 0001
patch, I don't think
we need the list to take care of the locks. Is there any other reason
why we maintain that list (apart from the trigger case I have raised
and Fujita-san says that the list is not required in that case as
well.)?

AppendRelInfos exist within the planner (they come to be and go away
within the planner). Once we leave the planner, that information is gone.

Executor will receive a plan node that won't contain that information:

1. Append has an appendplans field, which contains one plan tree for every
leaf partition. None of its fields, other than partitined_rels,
contains the RT indexes of the partitioned tables. Similarly in the
case of MergeAppend.

2. ModifyTable has a resultRelations fields which contains a list of leaf
partition RT indexes and a plans field which contains one plan tree for
every RT index in the resultRelations list (that is a plan tree that
will scan the particular leaf partition). None of its fields, other
than partitined_rels, contains the RT indexes of the partitioned
tables.

I learned over the course of developing the patch that added this
partitioned_rels field [1] that the executor needs to identify all the
affected tables by a given plan tree so that it could lock them. Executor
needs to lock them separately even if the plan itself was built after
locking all the relevant tables [2]. For example, see
ExecLockNonLeafAppendTables(), which will lock the tables in the
(Merge)Append.partitioned_rels list.

While I've been thinking all along that the same thing must be happening
for RT indexes in ModifyTable.partitioned_rels list (I said so a couple of
times on this thread), it's actually not. Instead,
ModifyTable.partitioned_rels of all ModifyTable nodes in a give query are
merged into PlannedStmt.nonleafResultRelations (which happens in
set_plan_refs) and that's where the executor finds them to lock them
(which happens in InitPlan).

So, it appears that ModifyTable.partitioned_rels is indeed unused in the
executor. But we still can't get rid of it from the ModifyTable node
itself without figuring out a way (a channel) to transfer that information
into PlannedStmt.nonleafResultRelations.

Thanks a lot for this detailed analysis. IIUC, in my 0001 patch, I am
not adding any partitioned partition other than the parent itself. But
since every partitioned partition in turn acts as parent, it appears
its own list. The list obtained by concatenating all such lists
together contains all the partitioned partition RTIs. In my patch, I
need to teach accumulate_append_subpath() to accumulate
partitioned_rels as well.

Having asked that, I think my patch shouldn't deal with removing
partitioned_rels lists and related structures and code. It should be> done as a separate patch.

Going back to your original email which started this discussion, it seems
that we agree on that the PartitionedChildRelInfo node can be removed, and
I agree that it shouldn't be done in the partitionwise-join patch series
but as a separate patch. As described above, we shouldn't try yet to get
rid of the partitioned_rels list that appears in some plan nodes.

Thanks.

--
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

#199Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Ashutosh Bapat (#198)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

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

On Tue, Sep 5, 2017 at 1:16 PM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

On 2017/09/05 15:43, Ashutosh Bapat wrote:

Ok. Can you please answer my previous questions?

AFAIU, the list contained RTIs of the relations, which didnt' have
corresponding AppendRelInfos to lock those relations. Now that we
create AppendRelInfos even for partitioned partitions with my 0001
patch, I don't think
we need the list to take care of the locks. Is there any other reason
why we maintain that list (apart from the trigger case I have raised
and Fujita-san says that the list is not required in that case as
well.)?

AppendRelInfos exist within the planner (they come to be and go away
within the planner). Once we leave the planner, that information is gone.

Executor will receive a plan node that won't contain that information:

1. Append has an appendplans field, which contains one plan tree for every
leaf partition. None of its fields, other than partitined_rels,
contains the RT indexes of the partitioned tables. Similarly in the
case of MergeAppend.

2. ModifyTable has a resultRelations fields which contains a list of leaf
partition RT indexes and a plans field which contains one plan tree for
every RT index in the resultRelations list (that is a plan tree that
will scan the particular leaf partition). None of its fields, other
than partitined_rels, contains the RT indexes of the partitioned
tables.

I learned over the course of developing the patch that added this
partitioned_rels field [1] that the executor needs to identify all the
affected tables by a given plan tree so that it could lock them. Executor
needs to lock them separately even if the plan itself was built after
locking all the relevant tables [2]. For example, see
ExecLockNonLeafAppendTables(), which will lock the tables in the
(Merge)Append.partitioned_rels list.

While I've been thinking all along that the same thing must be happening
for RT indexes in ModifyTable.partitioned_rels list (I said so a couple of
times on this thread), it's actually not. Instead,
ModifyTable.partitioned_rels of all ModifyTable nodes in a give query are
merged into PlannedStmt.nonleafResultRelations (which happens in
set_plan_refs) and that's where the executor finds them to lock them
(which happens in InitPlan).

So, it appears that ModifyTable.partitioned_rels is indeed unused in the
executor. But we still can't get rid of it from the ModifyTable node
itself without figuring out a way (a channel) to transfer that information
into PlannedStmt.nonleafResultRelations.

Thanks a lot for this detailed analysis. IIUC, in my 0001 patch, I am
not adding any partitioned partition other than the parent itself. But
since every partitioned partition in turn acts as parent, it appears
its own list. The list obtained by concatenating all such lists
together contains all the partitioned partition RTIs. In my patch, I
need to teach accumulate_append_subpath() to accumulate
partitioned_rels as well.

accumulate_append_subpath() is executed for every path instead of
every relation, so changing it would collect the same list multiple
times. Instead, I found the old way of associating all intermediate
partitions with the root partitioned relation work better. Here's the
updated patch set.
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company

Attachments:

pg_dp_join_patches_v29.tar.gzapplication/x-gzip; name=pg_dp_join_patches_v29.tar.gzDownload
#200Antonin Houska
ah@cybertec.at
In reply to: Ashutosh Bapat (#188)
Re: Partition-wise join for join between (declaratively) partitioned tables

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

On Fri, Sep 1, 2017 at 6:05 PM, Antonin Houska <ah@cybertec.at> wrote:

* get_partitioned_child_rels_for_join()

I think the Assert() statement is easier to understand inside the loop, see
the assert.diff attachment.

The assert at the end of function also checks that we have got
child_rels lists for all the parents passed in.

Really? I can imagine that some instances of PartitionedChildRelInfo have the
child_rels list empty, while other ones have these lists long enough to
compensate for the empty lists.

* have_partkey_equi_join()

As the function handles generic join, this comment doesn't seem to me
relevant:

/*
* The equi-join between partition keys is strict if equi-join between
* at least one partition key is using a strict operator. See
* explanation about outer join reordering identity 3 in
* optimizer/README
*/
strict_op = op_strict(opexpr->opno);

What in that comment is not exactly relevant?

Basically I don't understand why you mention join reordering here. The join
ordering questions must all have been resolved by the time
have_partkey_equi_join() is called.

And I think the function can return true even if strict_op is false for all
the operators evaluated in the loop.

I think it does that. Do you have a case where it doesn't?

Here I refer to this part of the comment above:

"... if equi-join between at least one partition key is using a strict
operator."

My understanding of the code (especially match_expr_to_partition_keys) is that
no operator actually needs to be strict as long as each operator involved in
the join matches at least one non-nullable expression on both sides of the
join.

* match_expr_to_partition_keys()

I'm not sure this comment is clear enough:

/*
* If it's a strict equi-join a NULL partition key on one side will
* not join a NULL partition key on the other side. So, rows with NULL
* partition key from a partition on one side can not join with those
* from a non-matching partition on the other side. So, search the
* nullable partition keys as well.
*/
if (!strict_op)
continue;

My understanding of the problem of NULL values generated by outer join is:
these NULL values --- if evaluated by non-strict expression --- can make row
of N-th partition on one side of the join match row(s) of *other than* N-th
partition(s) on the other side. Thus the nullable input expressions may only
be evaluated by strict operators. I think it'd be clearer if you stressed that
(undesired) *match* of partition keys can be a problem, rather than mismatch

Sorry, I am not able to understand this. To me it looks like my
wording conveys what you are saying.

I just tried to expreess the idea in a way that is clearer to me. I think we
both mean the same. Not sure I should spend more effort on another version of
the comment.

If you insist on your wording, then I think you should at least move the
comment below to the part that only deals with strict operators.

Done.

o.k.

* map_and_merge_partitions()

Besides a few changes proposed in map_and_merge_partitions.diff (a few of them
to suppress compiler warnings) I think that this part needs more thought:

{
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;
}

I could hit this path with the following test:

CREATE TABLE a(i int) PARTITION BY LIST(i);
CREATE TABLE a_0 PARTITION OF a FOR VALUES IN (0, 2);
CREATE TABLE b(j int) PARTITION BY LIST(j);
CREATE TABLE b_0 PARTITION OF b FOR VALUES IN (1, 2);

SET enable_partition_wise_join TO on;

SELECT *
FROM a
FULL JOIN
b ON i = j;

I don't think there's a reason not to join a_0 partition to b_0, is there?

With the latest patchset I am seeing that partition-wise join is used
in this case. I have started a new thread [1] for advanced partition
matching patches.

What plan do you get, with the patches from

/messages/by-id/CAFjFpRfdXpuSu0pxON3dKcr8WndJkaXLzHUVax_Laod0Tgc6UQ@mail.gmail.com

I still see the join above Append, not below:

QUERY PLAN
-------------------------------------------------------------------------
Merge Full Join (cost=359.57..860.00 rows=32512 width=8)
Merge Cond: (a_0.i = b_0.j)
-> Sort (cost=179.78..186.16 rows=2550 width=4)
Sort Key: a_0.i
-> Append (cost=0.00..35.50 rows=2550 width=4)
-> Seq Scan on a_0 (cost=0.00..35.50 rows=2550 width=4)
-> Sort (cost=179.78..186.16 rows=2550 width=4)
Sort Key: b_0.j
-> Append (cost=0.00..35.50 rows=2550 width=4)
-> Seq Scan on b_0 (cost=0.00..35.50 rows=2550 width=4)

Please post review comments about the last two patches on that thread.

ok, I'll do if I find any problem.

[1] /messages/by-id/CAFjFpRdjQvaUEV5DJX3TW6pU5eq54NCkadtxHX2JiJG_GvbrCA@mail.gmail.com

--
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

#201Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Antonin Houska (#200)
Re: Partition-wise join for join between (declaratively) partitioned tables

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

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

On Fri, Sep 1, 2017 at 6:05 PM, Antonin Houska <ah@cybertec.at> wrote:

* get_partitioned_child_rels_for_join()

I think the Assert() statement is easier to understand inside the loop, see
the assert.diff attachment.

The assert at the end of function also checks that we have got
child_rels lists for all the parents passed in.

Really? I can imagine that some instances of PartitionedChildRelInfo have the
child_rels list empty, while other ones have these lists long enough to
compensate for the empty lists.

That isn't true. Each child_rels list will at least have one entry.
Please see get_partitioned_child_rels().

* have_partkey_equi_join()

As the function handles generic join, this comment doesn't seem to me
relevant:

/*
* The equi-join between partition keys is strict if equi-join between
* at least one partition key is using a strict operator. See
* explanation about outer join reordering identity 3 in
* optimizer/README
*/
strict_op = op_strict(opexpr->opno);

What in that comment is not exactly relevant?

Basically I don't understand why you mention join reordering here. The join
ordering questions must all have been resolved by the time
have_partkey_equi_join() is called.

I am referring to a particular section in README which talks about the
relation between strict operator and legal join order.

And I think the function can return true even if strict_op is false for all
the operators evaluated in the loop.

I think it does that. Do you have a case where it doesn't?

Here I refer to this part of the comment above:

"... if equi-join between at least one partition key is using a strict
operator."

My understanding of the code (especially match_expr_to_partition_keys) is that
no operator actually needs to be strict as long as each operator involved in
the join matches at least one non-nullable expression on both sides of the
join.

I don't think so. A strict operator returns NULL when either of the
inputs is NULL. We can not say so for non-strict operators, which may
deem NULL and non-NULL arguments as equal, even though that looks
insane.

--
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

#202Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#199)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Sep 5, 2017 at 7:01 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

accumulate_append_subpath() is executed for every path instead of
every relation, so changing it would collect the same list multiple
times. Instead, I found the old way of associating all intermediate
partitions with the root partitioned relation work better. Here's the
updated patch set.

When I tried out patch 0001, it crashed repeatedly during 'make check'
because of an assertion failure in get_partitioned_child_rels. It
seemed to me that the way the patch was refactoring
expand_inherited_rtentry involved more code rearrangement than
necessary, so I reverted all the code rearrangement and just kept the
functional changes, and all the crashes went away. (That refactoring
also failed to initialize has_child properly.) In so doing, I
reintroduced the problem that the PartitionedChildRelInfo lists
weren't getting set up correctly, but after some thought I realized
that was just because expand_single_inheritance_child() was choosing
between adding an RTE and adding the OID to partitioned_child_rels,
whereas for an intermediate partitioned table it needs to do both. So
I inserted a trivial fix for that problem (replacing "else" with a new
"if"-test), basically:

-    else
+
+    if (childrte->relkind == RELKIND_PARTITIONED_TABLE)

Please check out the attached version of the patch. In addition to
the above simplifications, I did some adjustments to the comments in
various places - some just grammar and others a bit more substantive.
And I think I broke a long line in one place, too.

One thing I notice is that if I rip out the changes to initsplan.c,
the new regression test still passes. If it's possible to write a
test that fails without those changes, I think it would be a good idea
to include one in the patch. That's certainly one of the subtler
parts of this patch, IMHO.

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

Attachments:

expand-stepwise-rmh.patchapplication/octet-stream; name=expand-stepwise-rmh.patchDownload
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 2d7e1d84d0..7422cc6983 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -24,6 +24,7 @@
 #include "catalog/pg_operator.h"
 #include "catalog/pg_proc.h"
 #include "foreign/fdwapi.h"
+#include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
 #ifdef OPTIMIZER_DEBUG
@@ -867,6 +868,9 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 	int			nattrs;
 	ListCell   *l;
 
+	/* Guard against stack overflow due to overly deep inheritance tree. */
+	check_stack_depth();
+
 	Assert(IS_SIMPLE_REL(rel));
 
 	/*
diff --git a/src/backend/optimizer/plan/initsplan.c b/src/backend/optimizer/plan/initsplan.c
index 987c20ac9f..ad81f0f82f 100644
--- a/src/backend/optimizer/plan/initsplan.c
+++ b/src/backend/optimizer/plan/initsplan.c
@@ -15,6 +15,7 @@
 #include "postgres.h"
 
 #include "catalog/pg_type.h"
+#include "catalog/pg_class.h"
 #include "nodes/nodeFuncs.h"
 #include "optimizer/clauses.h"
 #include "optimizer/cost.h"
@@ -629,11 +630,28 @@ create_lateral_join_info(PlannerInfo *root)
 	for (rti = 1; rti < root->simple_rel_array_size; rti++)
 	{
 		RelOptInfo *brel = root->simple_rel_array[rti];
+		RangeTblEntry *brte = root->simple_rte_array[rti];
 
-		if (brel == NULL || brel->reloptkind != RELOPT_BASEREL)
+		if (brel == NULL)
+			continue;
+
+		/*
+		 * In the case of table inheritance, the parent RTE is directly linked
+		 * to every child table via an AppendRelInfo.  In the case of table
+		 * partitioning, the inheritance hierarchy is expanded one level at a
+		 * time rather than flattened.  Therefore, an other member rel that is
+		 * a partitioned table may have children of its own, and must
+		 * therefore be marked with the appropriate lateral info so that those
+		 * children eventually get marked also.
+		 */
+		Assert(IS_SIMPLE_REL(brel));
+		Assert(brte);
+		if (brel->reloptkind == RELOPT_OTHER_MEMBER_REL &&
+			(brte->rtekind != RTE_RELATION ||
+			 brte->relkind != RELKIND_PARTITIONED_TABLE))
 			continue;
 
-		if (root->simple_rte_array[rti]->inh)
+		if (brte->inh)
 		{
 			foreach(lc, root->append_rel_list)
 			{
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 6b79b3ad99..82b722b47b 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -1038,7 +1038,7 @@ static void
 inheritance_planner(PlannerInfo *root)
 {
 	Query	   *parse = root->parse;
-	int			parentRTindex = parse->resultRelation;
+	int			top_parentRTindex = parse->resultRelation;
 	Bitmapset  *subqueryRTindexes;
 	Bitmapset  *modifiableARIindexes;
 	int			nominalRelation = -1;
@@ -1056,6 +1056,10 @@ inheritance_planner(PlannerInfo *root)
 	Index		rti;
 	RangeTblEntry *parent_rte;
 	List	   *partitioned_rels = NIL;
+	PlannerInfo *parent_root;
+	Query	   *parent_parse;
+	Bitmapset  *parent_relids = bms_make_singleton(top_parentRTindex);
+	PlannerInfo **parent_roots = NULL;
 
 	Assert(parse->commandType != CMD_INSERT);
 
@@ -1121,9 +1125,22 @@ inheritance_planner(PlannerInfo *root)
 	 * opposite in the case of non-partitioned inheritance parent as described
 	 * below.
 	 */
-	parent_rte = rt_fetch(parentRTindex, root->parse->rtable);
+	parent_rte = rt_fetch(top_parentRTindex, root->parse->rtable);
 	if (parent_rte->relkind == RELKIND_PARTITIONED_TABLE)
-		nominalRelation = parentRTindex;
+		nominalRelation = top_parentRTindex;
+
+	/*
+	 * The PlannerInfo for each child is obtained by translating the relevant
+	 * members of the PlannerInfo for its immediate parent, which we find
+	 * using the parent_relid in its AppendRelInfo.  We save the PlannerInfo
+	 * for each parent in an array indexed by relid for fast retrieval. Since
+	 * the maximum number of parents is limited by the number of RTEs in the
+	 * query, we use that number to allocate the array. An extra entry is
+	 * needed since relids start from 1.
+	 */
+	parent_roots = (PlannerInfo **) palloc0((list_length(parse->rtable) + 1) *
+											sizeof(PlannerInfo *));
+	parent_roots[top_parentRTindex] = root;
 
 	/*
 	 * And now we can get on with generating a plan for each child table.
@@ -1137,15 +1154,24 @@ inheritance_planner(PlannerInfo *root)
 		Path	   *subpath;
 
 		/* append_rel_list contains all append rels; ignore others */
-		if (appinfo->parent_relid != parentRTindex)
+		if (!bms_is_member(appinfo->parent_relid, parent_relids))
 			continue;
 
 		/*
+		 * expand_inherited_rtentry() always processes a parent before any of
+		 * that parent's children, so the parent_root for this relation should
+		 * already be available.
+		 */
+		parent_root = parent_roots[appinfo->parent_relid];
+		Assert(parent_root != NULL);
+		parent_parse = parent_root->parse;
+
+		/*
 		 * We need a working copy of the PlannerInfo so that we can control
 		 * propagation of information back to the main copy.
 		 */
 		subroot = makeNode(PlannerInfo);
-		memcpy(subroot, root, sizeof(PlannerInfo));
+		memcpy(subroot, parent_root, sizeof(PlannerInfo));
 
 		/*
 		 * Generate modified query with this rel as target.  We first apply
@@ -1154,15 +1180,15 @@ inheritance_planner(PlannerInfo *root)
 		 * then fool around with subquery RTEs.
 		 */
 		subroot->parse = (Query *)
-			adjust_appendrel_attrs(root,
-								   (Node *) parse,
+			adjust_appendrel_attrs(parent_root,
+								   (Node *) parent_parse,
 								   1, &appinfo);
 
 		/*
 		 * If there are securityQuals attached to the parent, move them to the
 		 * child rel (they've already been transformed properly for that).
 		 */
-		parent_rte = rt_fetch(parentRTindex, subroot->parse->rtable);
+		parent_rte = rt_fetch(appinfo->parent_relid, subroot->parse->rtable);
 		child_rte = rt_fetch(appinfo->child_relid, subroot->parse->rtable);
 		child_rte->securityQuals = parent_rte->securityQuals;
 		parent_rte->securityQuals = NIL;
@@ -1173,7 +1199,7 @@ inheritance_planner(PlannerInfo *root)
 		 * executor doesn't need to see the modified copies --- we can just
 		 * pass it the original rowMarks list.)
 		 */
-		subroot->rowMarks = copyObject(root->rowMarks);
+		subroot->rowMarks = copyObject(parent_root->rowMarks);
 
 		/*
 		 * The append_rel_list likewise might contain references to subquery
@@ -1190,7 +1216,7 @@ inheritance_planner(PlannerInfo *root)
 			ListCell   *lc2;
 
 			subroot->append_rel_list = NIL;
-			foreach(lc2, root->append_rel_list)
+			foreach(lc2, parent_root->append_rel_list)
 			{
 				AppendRelInfo *appinfo2 = lfirst_node(AppendRelInfo, lc2);
 
@@ -1225,7 +1251,7 @@ inheritance_planner(PlannerInfo *root)
 			ListCell   *lr;
 
 			rti = 1;
-			foreach(lr, parse->rtable)
+			foreach(lr, parent_parse->rtable)
 			{
 				RangeTblEntry *rte = lfirst_node(RangeTblEntry, lr);
 
@@ -1272,6 +1298,22 @@ inheritance_planner(PlannerInfo *root)
 		/* hack to mark target relation as an inheritance partition */
 		subroot->hasInheritedTarget = true;
 
+		/*
+		 * If the child is further partitioned, remember it as a parent. Since
+		 * a partitioned table does not have any data, we don't need to create
+		 * a plan for it. We do, however, need to remember the PlannerInfo for
+		 * use when processing its children.
+		 */
+		if (child_rte->inh)
+		{
+			Assert(child_rte->relkind == RELKIND_PARTITIONED_TABLE);
+			parent_relids =
+				bms_add_member(parent_relids, appinfo->child_relid);
+			parent_roots[appinfo->child_relid] = subroot;
+
+			continue;
+		}
+
 		/* Generate Path(s) for accessing this result relation */
 		grouping_planner(subroot, true, 0.0 /* retrieve all tuples */ );
 
@@ -1370,7 +1412,7 @@ inheritance_planner(PlannerInfo *root)
 
 	if (parent_rte->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		partitioned_rels = get_partitioned_child_rels(root, parentRTindex);
+		partitioned_rels = get_partitioned_child_rels(root, top_parentRTindex);
 		/* The root partitioned table is included as a child rel */
 		Assert(list_length(partitioned_rels) >= 1);
 	}
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index ccf21453fd..131f597632 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -1348,9 +1348,9 @@ expand_inherited_tables(PlannerInfo *root)
 	ListCell   *rl;
 
 	/*
-	 * expand_inherited_rtentry may add RTEs to parse->rtable; there is no
-	 * need to scan them since they can't have inh=true.  So just scan as far
-	 * as the original end of the rtable list.
+	 * expand_inherited_rtentry may add RTEs to parse->rtable. The function is
+	 * expected to recursively handle any RTEs that it creates with inh=true.
+	 * So just scan as far as the original end of the rtable list.
 	 */
 	nrtes = list_length(root->parse->rtable);
 	rl = list_head(root->parse->rtable);
@@ -1649,17 +1649,25 @@ expand_single_inheritance_child(PlannerInfo *root, RangeTblEntry *parentrte,
 	childrte = copyObject(parentrte);
 	childrte->relid = childOID;
 	childrte->relkind = childrel->rd_rel->relkind;
-	childrte->inh = false;
+	/* A partitioned child will need to be expanded further. */
+	if (childOID != parentOID &&
+		childrte->relkind == RELKIND_PARTITIONED_TABLE)
+		childrte->inh = true;
+	else
+		childrte->inh = false;
 	childrte->requiredPerms = 0;
 	childrte->securityQuals = NIL;
 	parse->rtable = lappend(parse->rtable, childrte);
 	childRTindex = list_length(parse->rtable);
 
 	/*
-	 * Build an AppendRelInfo for this parent and child, unless the child is a
-	 * partitioned table.
+	 * We need an AppendRelInfo if paths will be built for the child RTE.
+	 * If childrte->inh is true, then we'll always need to generate append
+	 * paths for it.  If childrte->inh is false, we must scan it if it's
+	 * not a partitioned table; but if it is a partitioned table, then it
+	 * never has any data of its own and need not be scanned.
 	 */
-	if (childrte->relkind != RELKIND_PARTITIONED_TABLE)
+	if (childrte->relkind != RELKIND_PARTITIONED_TABLE || childrte->inh)
 	{
 		/* Remember if we saw a real child. */
 		if (childOID != parentOID)
@@ -1694,7 +1702,12 @@ expand_single_inheritance_child(PlannerInfo *root, RangeTblEntry *parentrte,
 														appinfo->translated_vars);
 		}
 	}
-	else
+
+	/*
+	 * If this is a partitioned table, it won't be scanned; add it to the
+	 * list of partitioned child relations so that it gets properly locked.
+	 */
+	if (childrte->relkind == RELKIND_PARTITIONED_TABLE)
 		*partitioned_child_rels = lappend_int(*partitioned_child_rels,
 											  childRTindex);
 
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index a39e59d8ac..d50ff55681 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -1935,10 +1935,10 @@ typedef struct SpecialJoinInfo
  *
  * When we expand an inheritable table or a UNION-ALL subselect into an
  * "append relation" (essentially, a list of child RTEs), we build an
- * AppendRelInfo for each non-partitioned child RTE.  The list of
- * AppendRelInfos indicates which child RTEs must be included when expanding
- * the parent, and each node carries information needed to translate Vars
- * referencing the parent into Vars referencing that child.
+ * AppendRelInfo for each child RTE.  The list of AppendRelInfos indicates
+ * which child RTEs must be included when expanding the parent, and each node
+ * carries information needed to translate Vars referencing the parent into
+ * Vars referencing that child.
  *
  * These structs are kept in the PlannerInfo node's append_rel_list.
  * Note that we just throw all the structs into one list, and scan the
diff --git a/src/test/regress/expected/inherit.out b/src/test/regress/expected/inherit.out
index 1fa9650ec9..2fb0b4d86e 100644
--- a/src/test/regress/expected/inherit.out
+++ b/src/test/regress/expected/inherit.out
@@ -625,6 +625,28 @@ select tableoid::regclass::text as relname, parted_tab.* from parted_tab order b
 (3 rows)
 
 drop table parted_tab;
+-- Check UPDATE with multi-level partitioned inherited target
+create table mlparted_tab (a int, b char, c text) partition by list (a);
+create table mlparted_tab_part1 partition of mlparted_tab for values in (1);
+create table mlparted_tab_part2 partition of mlparted_tab for values in (2) partition by list (b);
+create table mlparted_tab_part3 partition of mlparted_tab for values in (3);
+create table mlparted_tab_part2a partition of mlparted_tab_part2 for values in ('a');
+create table mlparted_tab_part2b partition of mlparted_tab_part2 for values in ('b');
+insert into mlparted_tab values (1, 'a'), (2, 'a'), (2, 'b'), (3, 'a');
+update mlparted_tab mlp set c = 'xxx'
+from
+  (select a from some_tab union all select a+1 from some_tab) ss (a)
+where (mlp.a = ss.a and mlp.b = 'b') or mlp.a = 3;
+select tableoid::regclass::text as relname, mlparted_tab.* from mlparted_tab order by 1,2;
+       relname       | a | b |  c  
+---------------------+---+---+-----
+ mlparted_tab_part1  | 1 | a | 
+ mlparted_tab_part2a | 2 | a | 
+ mlparted_tab_part2b | 2 | b | xxx
+ mlparted_tab_part3  | 3 | a | xxx
+(4 rows)
+
+drop table mlparted_tab;
 drop table some_tab cascade;
 NOTICE:  drop cascades to table some_tab_child
 /* Test multiple inheritance of column defaults */
diff --git a/src/test/regress/sql/inherit.sql b/src/test/regress/sql/inherit.sql
index c96580cd81..01780d4977 100644
--- a/src/test/regress/sql/inherit.sql
+++ b/src/test/regress/sql/inherit.sql
@@ -154,6 +154,23 @@ where parted_tab.a = ss.a;
 select tableoid::regclass::text as relname, parted_tab.* from parted_tab order by 1,2;
 
 drop table parted_tab;
+
+-- Check UPDATE with multi-level partitioned inherited target
+create table mlparted_tab (a int, b char, c text) partition by list (a);
+create table mlparted_tab_part1 partition of mlparted_tab for values in (1);
+create table mlparted_tab_part2 partition of mlparted_tab for values in (2) partition by list (b);
+create table mlparted_tab_part3 partition of mlparted_tab for values in (3);
+create table mlparted_tab_part2a partition of mlparted_tab_part2 for values in ('a');
+create table mlparted_tab_part2b partition of mlparted_tab_part2 for values in ('b');
+insert into mlparted_tab values (1, 'a'), (2, 'a'), (2, 'b'), (3, 'a');
+
+update mlparted_tab mlp set c = 'xxx'
+from
+  (select a from some_tab union all select a+1 from some_tab) ss (a)
+where (mlp.a = ss.a and mlp.b = 'b') or mlp.a = 3;
+select tableoid::regclass::text as relname, mlparted_tab.* from mlparted_tab order by 1,2;
+
+drop table mlparted_tab;
 drop table some_tab cascade;
 
 /* Test multiple inheritance of column defaults */
#203Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Robert Haas (#202)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On 2017/09/08 4:04, Robert Haas wrote:

On Tue, Sep 5, 2017 at 7:01 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

accumulate_append_subpath() is executed for every path instead of
every relation, so changing it would collect the same list multiple
times. Instead, I found the old way of associating all intermediate
partitions with the root partitioned relation work better. Here's the
updated patch set.

When I tried out patch 0001, it crashed repeatedly during 'make check'
because of an assertion failure in get_partitioned_child_rels. It
seemed to me that the way the patch was refactoring
expand_inherited_rtentry involved more code rearrangement than
necessary, so I reverted all the code rearrangement and just kept the
functional changes, and all the crashes went away. (That refactoring
also failed to initialize has_child properly.)

When I tried the attached patch, it doesn't seem to expand partitioning
inheritance in step-wise manner as the patch's title says. I think the
rewritten patch forgot to include Ashutosh's changes to
expand_single_inheritance_child() whereby the AppendRelInfo of the child
will be marked with the direct parent instead of always the root parent.

I updated the patch to include just those changes. I'm not sure about
one of the Ashutosh's changes whereby the child PlanRowMark is also passed
to expand_partitioned_rtentry() to use as the parent PlanRowMark. I think
the child RTE, child RT index and child Relation are fine, because they
are necessary for creating AppendRelInfos in a desired way for later
planning steps. But PlanRowMarks are not processed within the planner
afterwards and do not need to be marked with the immediate parent-child
association in the same way that AppendRelInfos need to be.

I also included the changes to add_paths_to_append_rel() from my patch on
the "path toward faster partition pruning" thread. We'd need that change,
because while add_paths_to_append_rel() is called for all partitioned
table RTEs in a given partition tree, expand_inherited_rtentry() would
have set up a PartitionedChildRelInfo only for the root parent, so
get_partitioned_child_rels() would not find the same for non-root
partitioned table rels and crash failing the Assert. The changes I made
are such that we call get_partitioned_child_rels() only for the parent
rels that are known to correspond root partitioned tables (or as you
pointed out on the thread, "the table named in the query" as opposed those
added to the query as result of inheritance expansion). In addition to
the relkind check on the input RTE, it would seem that checking that the
reloptkind is RELOPT_BASEREL would be enough. But actually not, because
if a partitioned table is accessed in a UNION ALL query, reloptkind even
for the root partitioned table (the table named in the query) would be
RELOPT_OTHER_MEMBER_REL. The only way to confirm that the input rel is
actually the root partitioned table is to check whether its parent rel is
not RTE_RELATION, because the parent in case of UNION ALL Append is a
RTE_SUBQUERY RT entry.

One thing I notice is that if I rip out the changes to initsplan.c,
the new regression test still passes. If it's possible to write a
test that fails without those changes, I think it would be a good idea
to include one in the patch. That's certainly one of the subtler
parts of this patch, IMHO.

Back when this (step-wise expansion of partition inheritance) used to be a
patch in the original declarative partitioning patch series, Ashutosh had
reported a test query [1]/messages/by-id/CAFjFpReZF34MDbY95xoATi0xVj2mAry4-LHBWVBayOc8gj=iqg@mail.gmail.com that would fail getting a plan, for which we
came up with the initsplan.c changes in this patch as the solution:

ERROR: could not devise a query plan for the given query

I tried that query again without the initsplan.c changes and somehow the
same error does not occur anymore. It's strange because without the
initsplan.c changes, there is no way for partitions lower in the tree than
the first level to get the direct_lateral_relids and lateral_relids from
the root parent rel. Maybe, Ashutosh has a way to devise the failing
query again.

I also confirmed that the partition-pruning patch set works fine with this
patch instead of the patch on that thread with the same functionality,
which I will now drop from that patch set. Sorry about the wasted time.

Thanks,
Amit

[1]: /messages/by-id/CAFjFpReZF34MDbY95xoATi0xVj2mAry4-LHBWVBayOc8gj=iqg@mail.gmail.com
/messages/by-id/CAFjFpReZF34MDbY95xoATi0xVj2mAry4-LHBWVBayOc8gj=iqg@mail.gmail.com

Attachments:

expand-stepwise-rmh-2.patchtext/plain; charset=UTF-8; name=expand-stepwise-rmh-2.patchDownload
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 2d7e1d84d0..71b5bdf95e 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -24,6 +24,7 @@
 #include "catalog/pg_operator.h"
 #include "catalog/pg_proc.h"
 #include "foreign/fdwapi.h"
+#include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
 #ifdef OPTIMIZER_DEBUG
@@ -867,6 +868,9 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 	int			nattrs;
 	ListCell   *l;
 
+	/* Guard against stack overflow due to overly deep inheritance tree. */
+	check_stack_depth();
+
 	Assert(IS_SIMPLE_REL(rel));
 
 	/*
@@ -1289,11 +1293,42 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	RangeTblEntry *rte;
 
 	rte = planner_rt_fetch(rel->relid, root);
+
+	/*
+	 * Get the partitioned_rels list from root->pcinfo_list after
+	 * confirming that rel is actually the table named in the query,
+	 * instead of a partitioned table that was added as a result of
+	 * inheritance expansion, because only the former gets a
+	 * PartitionedChildRelInfo.
+	 */
 	if (rte->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		partitioned_rels = get_partitioned_child_rels(root, rel->relid);
-		/* The root partitioned table is included as a child rel */
-		Assert(list_length(partitioned_rels) >= 1);
+		int		parent_relid;
+		bool	get_pcinfo = false;
+
+		/*
+		 * To distinguish the partitioned table rels added as result
+		 * of inheritance expansion, check using reloptkind if it's
+		 * otherrel.  But the original table could also be an otherrel,
+		 * if it's a child of a UNION ALL all query.
+		 */
+		if (!IS_OTHER_REL(rel))
+			get_pcinfo = true;
+		else if (bms_get_singleton_member(rel->top_parent_relids,
+										  &parent_relid))
+		{
+			RelOptInfo *parent_rel;
+
+			parent_rel = root->simple_rel_array[parent_relid];
+			get_pcinfo = (parent_rel->rtekind == RTE_SUBQUERY);
+		}
+
+		if (get_pcinfo)
+		{
+			partitioned_rels = get_partitioned_child_rels(root, rel->relid);
+			/* The root partitioned table is included as a child rel */
+			Assert(list_length(partitioned_rels) >= 1);
+		}
 	}
 
 	/*
diff --git a/src/backend/optimizer/plan/initsplan.c b/src/backend/optimizer/plan/initsplan.c
index 987c20ac9f..ad81f0f82f 100644
--- a/src/backend/optimizer/plan/initsplan.c
+++ b/src/backend/optimizer/plan/initsplan.c
@@ -15,6 +15,7 @@
 #include "postgres.h"
 
 #include "catalog/pg_type.h"
+#include "catalog/pg_class.h"
 #include "nodes/nodeFuncs.h"
 #include "optimizer/clauses.h"
 #include "optimizer/cost.h"
@@ -629,11 +630,28 @@ create_lateral_join_info(PlannerInfo *root)
 	for (rti = 1; rti < root->simple_rel_array_size; rti++)
 	{
 		RelOptInfo *brel = root->simple_rel_array[rti];
+		RangeTblEntry *brte = root->simple_rte_array[rti];
 
-		if (brel == NULL || brel->reloptkind != RELOPT_BASEREL)
+		if (brel == NULL)
+			continue;
+
+		/*
+		 * In the case of table inheritance, the parent RTE is directly linked
+		 * to every child table via an AppendRelInfo.  In the case of table
+		 * partitioning, the inheritance hierarchy is expanded one level at a
+		 * time rather than flattened.  Therefore, an other member rel that is
+		 * a partitioned table may have children of its own, and must
+		 * therefore be marked with the appropriate lateral info so that those
+		 * children eventually get marked also.
+		 */
+		Assert(IS_SIMPLE_REL(brel));
+		Assert(brte);
+		if (brel->reloptkind == RELOPT_OTHER_MEMBER_REL &&
+			(brte->rtekind != RTE_RELATION ||
+			 brte->relkind != RELKIND_PARTITIONED_TABLE))
 			continue;
 
-		if (root->simple_rte_array[rti]->inh)
+		if (brte->inh)
 		{
 			foreach(lc, root->append_rel_list)
 			{
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 6b79b3ad99..82b722b47b 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -1038,7 +1038,7 @@ static void
 inheritance_planner(PlannerInfo *root)
 {
 	Query	   *parse = root->parse;
-	int			parentRTindex = parse->resultRelation;
+	int			top_parentRTindex = parse->resultRelation;
 	Bitmapset  *subqueryRTindexes;
 	Bitmapset  *modifiableARIindexes;
 	int			nominalRelation = -1;
@@ -1056,6 +1056,10 @@ inheritance_planner(PlannerInfo *root)
 	Index		rti;
 	RangeTblEntry *parent_rte;
 	List	   *partitioned_rels = NIL;
+	PlannerInfo *parent_root;
+	Query	   *parent_parse;
+	Bitmapset  *parent_relids = bms_make_singleton(top_parentRTindex);
+	PlannerInfo **parent_roots = NULL;
 
 	Assert(parse->commandType != CMD_INSERT);
 
@@ -1121,9 +1125,22 @@ inheritance_planner(PlannerInfo *root)
 	 * opposite in the case of non-partitioned inheritance parent as described
 	 * below.
 	 */
-	parent_rte = rt_fetch(parentRTindex, root->parse->rtable);
+	parent_rte = rt_fetch(top_parentRTindex, root->parse->rtable);
 	if (parent_rte->relkind == RELKIND_PARTITIONED_TABLE)
-		nominalRelation = parentRTindex;
+		nominalRelation = top_parentRTindex;
+
+	/*
+	 * The PlannerInfo for each child is obtained by translating the relevant
+	 * members of the PlannerInfo for its immediate parent, which we find
+	 * using the parent_relid in its AppendRelInfo.  We save the PlannerInfo
+	 * for each parent in an array indexed by relid for fast retrieval. Since
+	 * the maximum number of parents is limited by the number of RTEs in the
+	 * query, we use that number to allocate the array. An extra entry is
+	 * needed since relids start from 1.
+	 */
+	parent_roots = (PlannerInfo **) palloc0((list_length(parse->rtable) + 1) *
+											sizeof(PlannerInfo *));
+	parent_roots[top_parentRTindex] = root;
 
 	/*
 	 * And now we can get on with generating a plan for each child table.
@@ -1137,15 +1154,24 @@ inheritance_planner(PlannerInfo *root)
 		Path	   *subpath;
 
 		/* append_rel_list contains all append rels; ignore others */
-		if (appinfo->parent_relid != parentRTindex)
+		if (!bms_is_member(appinfo->parent_relid, parent_relids))
 			continue;
 
 		/*
+		 * expand_inherited_rtentry() always processes a parent before any of
+		 * that parent's children, so the parent_root for this relation should
+		 * already be available.
+		 */
+		parent_root = parent_roots[appinfo->parent_relid];
+		Assert(parent_root != NULL);
+		parent_parse = parent_root->parse;
+
+		/*
 		 * We need a working copy of the PlannerInfo so that we can control
 		 * propagation of information back to the main copy.
 		 */
 		subroot = makeNode(PlannerInfo);
-		memcpy(subroot, root, sizeof(PlannerInfo));
+		memcpy(subroot, parent_root, sizeof(PlannerInfo));
 
 		/*
 		 * Generate modified query with this rel as target.  We first apply
@@ -1154,15 +1180,15 @@ inheritance_planner(PlannerInfo *root)
 		 * then fool around with subquery RTEs.
 		 */
 		subroot->parse = (Query *)
-			adjust_appendrel_attrs(root,
-								   (Node *) parse,
+			adjust_appendrel_attrs(parent_root,
+								   (Node *) parent_parse,
 								   1, &appinfo);
 
 		/*
 		 * If there are securityQuals attached to the parent, move them to the
 		 * child rel (they've already been transformed properly for that).
 		 */
-		parent_rte = rt_fetch(parentRTindex, subroot->parse->rtable);
+		parent_rte = rt_fetch(appinfo->parent_relid, subroot->parse->rtable);
 		child_rte = rt_fetch(appinfo->child_relid, subroot->parse->rtable);
 		child_rte->securityQuals = parent_rte->securityQuals;
 		parent_rte->securityQuals = NIL;
@@ -1173,7 +1199,7 @@ inheritance_planner(PlannerInfo *root)
 		 * executor doesn't need to see the modified copies --- we can just
 		 * pass it the original rowMarks list.)
 		 */
-		subroot->rowMarks = copyObject(root->rowMarks);
+		subroot->rowMarks = copyObject(parent_root->rowMarks);
 
 		/*
 		 * The append_rel_list likewise might contain references to subquery
@@ -1190,7 +1216,7 @@ inheritance_planner(PlannerInfo *root)
 			ListCell   *lc2;
 
 			subroot->append_rel_list = NIL;
-			foreach(lc2, root->append_rel_list)
+			foreach(lc2, parent_root->append_rel_list)
 			{
 				AppendRelInfo *appinfo2 = lfirst_node(AppendRelInfo, lc2);
 
@@ -1225,7 +1251,7 @@ inheritance_planner(PlannerInfo *root)
 			ListCell   *lr;
 
 			rti = 1;
-			foreach(lr, parse->rtable)
+			foreach(lr, parent_parse->rtable)
 			{
 				RangeTblEntry *rte = lfirst_node(RangeTblEntry, lr);
 
@@ -1272,6 +1298,22 @@ inheritance_planner(PlannerInfo *root)
 		/* hack to mark target relation as an inheritance partition */
 		subroot->hasInheritedTarget = true;
 
+		/*
+		 * If the child is further partitioned, remember it as a parent. Since
+		 * a partitioned table does not have any data, we don't need to create
+		 * a plan for it. We do, however, need to remember the PlannerInfo for
+		 * use when processing its children.
+		 */
+		if (child_rte->inh)
+		{
+			Assert(child_rte->relkind == RELKIND_PARTITIONED_TABLE);
+			parent_relids =
+				bms_add_member(parent_relids, appinfo->child_relid);
+			parent_roots[appinfo->child_relid] = subroot;
+
+			continue;
+		}
+
 		/* Generate Path(s) for accessing this result relation */
 		grouping_planner(subroot, true, 0.0 /* retrieve all tuples */ );
 
@@ -1370,7 +1412,7 @@ inheritance_planner(PlannerInfo *root)
 
 	if (parent_rte->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		partitioned_rels = get_partitioned_child_rels(root, parentRTindex);
+		partitioned_rels = get_partitioned_child_rels(root, top_parentRTindex);
 		/* The root partitioned table is included as a child rel */
 		Assert(list_length(partitioned_rels) >= 1);
 	}
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index ccf21453fd..95e12a5207 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -113,7 +113,9 @@ static void expand_single_inheritance_child(PlannerInfo *root,
 								Index parentRTindex, Relation parentrel,
 								PlanRowMark *parentrc, Relation childrel,
 								bool *has_child, List **appinfos,
-								List **partitioned_child_rels);
+								List **partitioned_child_rels,
+								RangeTblEntry **childrte_p,
+								Index *childRTindex_p);
 static void make_inh_translation_list(Relation oldrelation,
 						  Relation newrelation,
 						  Index newvarno,
@@ -1348,9 +1350,9 @@ expand_inherited_tables(PlannerInfo *root)
 	ListCell   *rl;
 
 	/*
-	 * expand_inherited_rtentry may add RTEs to parse->rtable; there is no
-	 * need to scan them since they can't have inh=true.  So just scan as far
-	 * as the original end of the rtable list.
+	 * expand_inherited_rtentry may add RTEs to parse->rtable. The function is
+	 * expected to recursively handle any RTEs that it creates with inh=true.
+	 * So just scan as far as the original end of the rtable list.
 	 */
 	nrtes = list_length(root->parse->rtable);
 	rl = list_head(root->parse->rtable);
@@ -1479,7 +1481,8 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 		expand_single_inheritance_child(root, rte, rti, oldrelation, oldrc,
 										oldrelation,
 										&has_child, &appinfos,
-										&partitioned_child_rels);
+										&partitioned_child_rels,
+										NULL, NULL);
 		expand_partitioned_rtentry(root, rte, rti, oldrelation, oldrc,
 									  RelationGetPartitionDesc(oldrelation),
 									  lockmode,
@@ -1519,7 +1522,8 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
 			expand_single_inheritance_child(root, rte, rti, oldrelation, oldrc,
 											newrelation,
 											&has_child, &appinfos,
-											&partitioned_child_rels);
+											&partitioned_child_rels,
+											NULL, NULL);
 
 			/* Close child relations, but keep locks */
 			if (childOID != parentOID)
@@ -1581,6 +1585,8 @@ expand_partitioned_rtentry(PlannerInfo *root, RangeTblEntry *parentrte,
 	{
 		Oid			childOID = partdesc->oids[i];
 		Relation	childrel;
+		RangeTblEntry *childrte;
+		Index		childRTindex;
 
 		/* Open rel; we already have required locks */
 		childrel = heap_open(childOID, NoLock);
@@ -1595,16 +1601,25 @@ expand_partitioned_rtentry(PlannerInfo *root, RangeTblEntry *parentrte,
 		expand_single_inheritance_child(root, parentrte, parentRTindex,
 										parentrel, parentrc, childrel,
 										has_child, appinfos,
-										partitioned_child_rels);
+										partitioned_child_rels,
+										&childrte, &childRTindex);
 
-		/* If this child is itself partitioned, recurse */
+		/*
+		 * If this child is itself partitioned, recurse.  Pass down the
+		 * childrte as the parent of the child RTEs that will be created in
+		 * the following call to ensure that the AppendRelInfos thus created
+		 * for the children will be marked with the immediate parent instead
+		 * of the root parent.
+		 */
 		if (childrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
-			expand_partitioned_rtentry(root, parentrte, parentRTindex,
-										  parentrel, parentrc,
+		{
+			expand_partitioned_rtentry(root, childrte, childRTindex,
+										  childrel, parentrc,
 										  RelationGetPartitionDesc(childrel),
 										  lockmode,
 										  has_child, appinfos,
 										  partitioned_child_rels);
+		}
 
 		/* Close child relation, but keep locks */
 		heap_close(childrel, NoLock);
@@ -1625,7 +1640,9 @@ expand_single_inheritance_child(PlannerInfo *root, RangeTblEntry *parentrte,
 								Index parentRTindex, Relation parentrel,
 								PlanRowMark *parentrc, Relation childrel,
 								bool *has_child, List **appinfos,
-								List **partitioned_child_rels)
+								List **partitioned_child_rels,
+								RangeTblEntry **childrte_p,
+								Index *childRTindex_p)
 {
 	Query	   *parse = root->parse;
 	Oid			parentOID = RelationGetRelid(parentrel);
@@ -1649,17 +1666,25 @@ expand_single_inheritance_child(PlannerInfo *root, RangeTblEntry *parentrte,
 	childrte = copyObject(parentrte);
 	childrte->relid = childOID;
 	childrte->relkind = childrel->rd_rel->relkind;
-	childrte->inh = false;
+	/* A partitioned child will need to be expanded further. */
+	if (childOID != parentOID &&
+		childrte->relkind == RELKIND_PARTITIONED_TABLE)
+		childrte->inh = true;
+	else
+		childrte->inh = false;
 	childrte->requiredPerms = 0;
 	childrte->securityQuals = NIL;
 	parse->rtable = lappend(parse->rtable, childrte);
 	childRTindex = list_length(parse->rtable);
 
 	/*
-	 * Build an AppendRelInfo for this parent and child, unless the child is a
-	 * partitioned table.
+	 * We need an AppendRelInfo if paths will be built for the child RTE.
+	 * If childrte->inh is true, then we'll always need to generate append
+	 * paths for it.  If childrte->inh is false, we must scan it if it's
+	 * not a partitioned table; but if it is a partitioned table, then it
+	 * never has any data of its own and need not be scanned.
 	 */
-	if (childrte->relkind != RELKIND_PARTITIONED_TABLE)
+	if (childrte->relkind != RELKIND_PARTITIONED_TABLE || childrte->inh)
 	{
 		/* Remember if we saw a real child. */
 		if (childOID != parentOID)
@@ -1694,7 +1719,12 @@ expand_single_inheritance_child(PlannerInfo *root, RangeTblEntry *parentrte,
 														appinfo->translated_vars);
 		}
 	}
-	else
+
+	/*
+	 * If this is a partitioned table, it won't be scanned; add it to the
+	 * list of partitioned child relations so that it gets properly locked.
+	 */
+	if (childrte->relkind == RELKIND_PARTITIONED_TABLE)
 		*partitioned_child_rels = lappend_int(*partitioned_child_rels,
 											  childRTindex);
 
@@ -1704,7 +1734,6 @@ expand_single_inheritance_child(PlannerInfo *root, RangeTblEntry *parentrte,
 	if (parentrc)
 	{
 		PlanRowMark *childrc = makeNode(PlanRowMark);
-
 		childrc->rti = childRTindex;
 		childrc->prti = parentRTindex;
 		childrc->rowmarkId = parentrc->rowmarkId;
@@ -1726,6 +1755,11 @@ expand_single_inheritance_child(PlannerInfo *root, RangeTblEntry *parentrte,
 
 		root->rowMarks = lappend(root->rowMarks, childrc);
 	}
+
+	if (childrte_p)
+		*childrte_p = childrte;
+	if (childRTindex_p)
+		*childRTindex_p = childRTindex;
 }
 
 /*
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index a39e59d8ac..d50ff55681 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -1935,10 +1935,10 @@ typedef struct SpecialJoinInfo
  *
  * When we expand an inheritable table or a UNION-ALL subselect into an
  * "append relation" (essentially, a list of child RTEs), we build an
- * AppendRelInfo for each non-partitioned child RTE.  The list of
- * AppendRelInfos indicates which child RTEs must be included when expanding
- * the parent, and each node carries information needed to translate Vars
- * referencing the parent into Vars referencing that child.
+ * AppendRelInfo for each child RTE.  The list of AppendRelInfos indicates
+ * which child RTEs must be included when expanding the parent, and each node
+ * carries information needed to translate Vars referencing the parent into
+ * Vars referencing that child.
  *
  * These structs are kept in the PlannerInfo node's append_rel_list.
  * Note that we just throw all the structs into one list, and scan the
diff --git a/src/test/regress/expected/inherit.out b/src/test/regress/expected/inherit.out
index 1fa9650ec9..2fb0b4d86e 100644
--- a/src/test/regress/expected/inherit.out
+++ b/src/test/regress/expected/inherit.out
@@ -625,6 +625,28 @@ select tableoid::regclass::text as relname, parted_tab.* from parted_tab order b
 (3 rows)
 
 drop table parted_tab;
+-- Check UPDATE with multi-level partitioned inherited target
+create table mlparted_tab (a int, b char, c text) partition by list (a);
+create table mlparted_tab_part1 partition of mlparted_tab for values in (1);
+create table mlparted_tab_part2 partition of mlparted_tab for values in (2) partition by list (b);
+create table mlparted_tab_part3 partition of mlparted_tab for values in (3);
+create table mlparted_tab_part2a partition of mlparted_tab_part2 for values in ('a');
+create table mlparted_tab_part2b partition of mlparted_tab_part2 for values in ('b');
+insert into mlparted_tab values (1, 'a'), (2, 'a'), (2, 'b'), (3, 'a');
+update mlparted_tab mlp set c = 'xxx'
+from
+  (select a from some_tab union all select a+1 from some_tab) ss (a)
+where (mlp.a = ss.a and mlp.b = 'b') or mlp.a = 3;
+select tableoid::regclass::text as relname, mlparted_tab.* from mlparted_tab order by 1,2;
+       relname       | a | b |  c  
+---------------------+---+---+-----
+ mlparted_tab_part1  | 1 | a | 
+ mlparted_tab_part2a | 2 | a | 
+ mlparted_tab_part2b | 2 | b | xxx
+ mlparted_tab_part3  | 3 | a | xxx
+(4 rows)
+
+drop table mlparted_tab;
 drop table some_tab cascade;
 NOTICE:  drop cascades to table some_tab_child
 /* Test multiple inheritance of column defaults */
diff --git a/src/test/regress/sql/inherit.sql b/src/test/regress/sql/inherit.sql
index c96580cd81..01780d4977 100644
--- a/src/test/regress/sql/inherit.sql
+++ b/src/test/regress/sql/inherit.sql
@@ -154,6 +154,23 @@ where parted_tab.a = ss.a;
 select tableoid::regclass::text as relname, parted_tab.* from parted_tab order by 1,2;
 
 drop table parted_tab;
+
+-- Check UPDATE with multi-level partitioned inherited target
+create table mlparted_tab (a int, b char, c text) partition by list (a);
+create table mlparted_tab_part1 partition of mlparted_tab for values in (1);
+create table mlparted_tab_part2 partition of mlparted_tab for values in (2) partition by list (b);
+create table mlparted_tab_part3 partition of mlparted_tab for values in (3);
+create table mlparted_tab_part2a partition of mlparted_tab_part2 for values in ('a');
+create table mlparted_tab_part2b partition of mlparted_tab_part2 for values in ('b');
+insert into mlparted_tab values (1, 'a'), (2, 'a'), (2, 'b'), (3, 'a');
+
+update mlparted_tab mlp set c = 'xxx'
+from
+  (select a from some_tab union all select a+1 from some_tab) ss (a)
+where (mlp.a = ss.a and mlp.b = 'b') or mlp.a = 3;
+select tableoid::regclass::text as relname, mlparted_tab.* from mlparted_tab order by 1,2;
+
+drop table mlparted_tab;
 drop table some_tab cascade;
 
 /* Test multiple inheritance of column defaults */
#204Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Amit Langote (#203)
Re: Partition-wise join for join between (declaratively) partitioned tables

On 2017/09/08 14:47, Amit Langote wrote:

When I tried the attached patch, it doesn't seem to expand partitioning
inheritance in step-wise manner as the patch's title says.

Oops. By "attached patch", I had meant to say the Robert's patch, that
is, expand-stepwise-rmh.patch. Not expand-stepwise-rmh-2.patch, which is
the updated version of the patch attached with the quoted message.

Thanks,
Amit

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

#205Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#202)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Sep 8, 2017 at 12:34 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Tue, Sep 5, 2017 at 7:01 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

accumulate_append_subpath() is executed for every path instead of
every relation, so changing it would collect the same list multiple
times. Instead, I found the old way of associating all intermediate
partitions with the root partitioned relation work better. Here's the
updated patch set.

When I tried out patch 0001, it crashed repeatedly during 'make check'
because of an assertion failure in get_partitioned_child_rels.

Running "make check" on the whole patchset doesn't give that failure.
So I didn't notice the crash since I was running regression on the
whole patchset. Sorry for that. Fortunately git rebase -i allows us to
execute shell commands while applying patches, so I have set it up to
compile each patch and run regression. Hopefully that will catch such
errors in future. That process showed me that patch
0003-In-add_paths_to_append_rel-get-partitioned_rels-for-.patch fixes
that crash by calling get_partitioned_child_rels() only on the root
partitioned table for which we have set up child_rels list. Amit
Langote has a similar fix reported in his reply. So, we will discuss
it there.

It
seemed to me that the way the patch was refactoring
expand_inherited_rtentry involved more code rearrangement than
necessary, so I reverted all the code rearrangement and just kept the
functional changes, and all the crashes went away. (That refactoring
also failed to initialize has_child properly.) In so doing, I
reintroduced the problem that the PartitionedChildRelInfo lists
weren't getting set up correctly, but after some thought I realized
that was just because expand_single_inheritance_child() was choosing
between adding an RTE and adding the OID to partitioned_child_rels,
whereas for an intermediate partitioned table it needs to do both. So
I inserted a trivial fix for that problem (replacing "else" with a new
"if"-test), basically:

-    else
+
+    if (childrte->relkind == RELKIND_PARTITIONED_TABLE)

Please check out the attached version of the patch. In addition to
the above simplifications, I did some adjustments to the comments in
various places - some just grammar and others a bit more substantive.
And I think I broke a long line in one place, too.

One thing I notice is that if I rip out the changes to initsplan.c,
the new regression test still passes. If it's possible to write a
test that fails without those changes, I think it would be a good idea
to include one in the patch. That's certainly one of the subtler
parts of this patch, IMHO.

Amit Langote has replied on these points as well. So, I will comment
in a reply to his reply.

--
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

#206Robert Haas
robertmhaas@gmail.com
In reply to: Amit Langote (#203)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Sep 8, 2017 at 1:47 AM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

When I tried the attached patch, it doesn't seem to expand partitioning
inheritance in step-wise manner as the patch's title says. I think the
rewritten patch forgot to include Ashutosh's changes to
expand_single_inheritance_child() whereby the AppendRelInfo of the child
will be marked with the direct parent instead of always the root parent.

Woops.

I updated the patch to include just those changes. I'm not sure about
one of the Ashutosh's changes whereby the child PlanRowMark is also passed
to expand_partitioned_rtentry() to use as the parent PlanRowMark. I think
the child RTE, child RT index and child Relation are fine, because they
are necessary for creating AppendRelInfos in a desired way for later
planning steps. But PlanRowMarks are not processed within the planner
afterwards and do not need to be marked with the immediate parent-child
association in the same way that AppendRelInfos need to be.

We probably need some better comments to explain which things need to
be marked using the immediate parent and which need to be marked using
the baserel, and why.

I also included the changes to add_paths_to_append_rel() from my patch on
the "path toward faster partition pruning" thread. We'd need that change,
because while add_paths_to_append_rel() is called for all partitioned
table RTEs in a given partition tree, expand_inherited_rtentry() would
have set up a PartitionedChildRelInfo only for the root parent, so
get_partitioned_child_rels() would not find the same for non-root
partitioned table rels and crash failing the Assert. The changes I made
are such that we call get_partitioned_child_rels() only for the parent
rels that are known to correspond root partitioned tables (or as you
pointed out on the thread, "the table named in the query" as opposed those
added to the query as result of inheritance expansion). In addition to
the relkind check on the input RTE, it would seem that checking that the
reloptkind is RELOPT_BASEREL would be enough. But actually not, because
if a partitioned table is accessed in a UNION ALL query, reloptkind even
for the root partitioned table (the table named in the query) would be
RELOPT_OTHER_MEMBER_REL. The only way to confirm that the input rel is
actually the root partitioned table is to check whether its parent rel is
not RTE_RELATION, because the parent in case of UNION ALL Append is a
RTE_SUBQUERY RT entry.

OK, so this needs some good comments, too...

--
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

#207Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Amit Langote (#203)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Sep 8, 2017 at 11:17 AM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

On 2017/09/08 4:04, Robert Haas wrote:

On Tue, Sep 5, 2017 at 7:01 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

accumulate_append_subpath() is executed for every path instead of
every relation, so changing it would collect the same list multiple
times. Instead, I found the old way of associating all intermediate
partitions with the root partitioned relation work better. Here's the
updated patch set.

When I tried out patch 0001, it crashed repeatedly during 'make check'
because of an assertion failure in get_partitioned_child_rels. It
seemed to me that the way the patch was refactoring
expand_inherited_rtentry involved more code rearrangement than
necessary, so I reverted all the code rearrangement and just kept the
functional changes, and all the crashes went away. (That refactoring
also failed to initialize has_child properly.)

When I tried the attached patch, it doesn't seem to expand partitioning
inheritance in step-wise manner as the patch's title says. I think the
rewritten patch forgot to include Ashutosh's changes to
expand_single_inheritance_child() whereby the AppendRelInfo of the child
will be marked with the direct parent instead of always the root parent.

Right. If we apply 0002 from partition-wise join patchset, which has
changed build_simple_rel() to collect direct children of a given
partitioned table, it introduces another crash because of Assertion
failure; for a partitioned table build_simple_rel() finds more
children than expected because indirect children are also counted as
direct children.

I updated the patch to include just those changes. I'm not sure about
one of the Ashutosh's changes whereby the child PlanRowMark is also passed
to expand_partitioned_rtentry() to use as the parent PlanRowMark. I think
the child RTE, child RT index and child Relation are fine, because they
are necessary for creating AppendRelInfos in a desired way for later
planning steps. But PlanRowMarks are not processed within the planner
afterwards and do not need to be marked with the immediate parent-child
association in the same way that AppendRelInfos need to be.

Passing top parent's row mark works today, since there is no
parent-child specific translation happening there. But if in future,
we introduce such a translation, row marks for indirect children in a
multi-level partitioned hierarchy won't be accurate. So, I think it's
better to pass row marks of the direct parent.

I also included the changes to add_paths_to_append_rel() from my patch on
the "path toward faster partition pruning" thread. We'd need that change,
because while add_paths_to_append_rel() is called for all partitioned
table RTEs in a given partition tree, expand_inherited_rtentry() would
have set up a PartitionedChildRelInfo only for the root parent, so
get_partitioned_child_rels() would not find the same for non-root
partitioned table rels and crash failing the Assert. The changes I made
are such that we call get_partitioned_child_rels() only for the parent
rels that are known to correspond root partitioned tables (or as you
pointed out on the thread, "the table named in the query" as opposed those
added to the query as result of inheritance expansion). In addition to
the relkind check on the input RTE, it would seem that checking that the
reloptkind is RELOPT_BASEREL would be enough. But actually not, because
if a partitioned table is accessed in a UNION ALL query, reloptkind even
for the root partitioned table (the table named in the query) would be
RELOPT_OTHER_MEMBER_REL. The only way to confirm that the input rel is
actually the root partitioned table is to check whether its parent rel is
not RTE_RELATION, because the parent in case of UNION ALL Append is a
RTE_SUBQUERY RT entry.

There was a change in my 0003 patch, which fixed the crash. It checked
for RELOPT_BASEREL and RELKIND_PARTITIONED_TABLE. I have pulled it in
my 0001 patch. It no more crashes. I tried various queries involving
set operations and bare multi-level partitioned table scan with my
patch, but none of them showed any anomaly. Do you have a testcase
which shows problem with my patch? May be your suggestion is correct,
but corresponding code implementation is slightly longer than I would
expect. So, we should go with it, if there is corresponding testcase
which shows why it's needed.

In your patch
+            parent_rel = root->simple_rel_array[parent_relid];
+            get_pcinfo = (parent_rel->rtekind == RTE_SUBQUERY);
Do you mean RTE_RELATION as you explained above?

One thing I notice is that if I rip out the changes to initsplan.c,
the new regression test still passes. If it's possible to write a
test that fails without those changes, I think it would be a good idea
to include one in the patch. That's certainly one of the subtler
parts of this patch, IMHO.

Back when this (step-wise expansion of partition inheritance) used to be a
patch in the original declarative partitioning patch series, Ashutosh had
reported a test query [1] that would fail getting a plan, for which we
came up with the initsplan.c changes in this patch as the solution:

ERROR: could not devise a query plan for the given query

I tried that query again without the initsplan.c changes and somehow the
same error does not occur anymore. It's strange because without the
initsplan.c changes, there is no way for partitions lower in the tree than
the first level to get the direct_lateral_relids and lateral_relids from
the root parent rel. Maybe, Ashutosh has a way to devise the failing
query again.

Thanks a lot for the reference. I devised a testcase slightly
modifying my original test. I have included the test in the latest
patch set.

I have included Robert's changes to parts other than
expand_inherited_rtentry() in the patch.

I also confirmed that the partition-pruning patch set works fine with this
patch instead of the patch on that thread with the same functionality,
which I will now drop from that patch set. Sorry about the wasted time.

Thanks a lot. Please review the patch in the updated patchset.

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

Attachments:

pg_dp_join_patches_v30.tar.gzapplication/x-gzip; name=pg_dp_join_patches_v30.tar.gzDownload
#208Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#207)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Sep 8, 2017 at 1:38 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

I also confirmed that the partition-pruning patch set works fine with this
patch instead of the patch on that thread with the same functionality,
which I will now drop from that patch set. Sorry about the wasted time.

Thanks a lot. Please review the patch in the updated patchset.

In set_append_rel_size(), I don't find the comment too clear (and this
part was taken from Amit's patch, right?). I suggest:

+    /*
+     * Associate the partitioned tables which are descendents of the table
+     * named in the query with the topmost append path (i.e. the one where
+     * rel->reloptkind is RELOPT_BASEREL).  This ensures that they get properly
+     * locked at execution time.
+     */

I'm a bit suspicious about the fact that there are now executor
changes related to the PlanRowMarks. If the rowmark's prti is now the
intermediate parent's RT index rather than the top-parent's RT index,
it'd seem like that'd matter somehow. Maybe it doesn't, because the
code that cares about prti seems to only care about whether it's
different from rti. But if that's true everywhere, then why even
change this? I think we might be well off not to tinker with things
that don't need to be changed.

Apart from that concern, now that I understand (from my own failed
attempt and some off-list discussion) why this patch works the way it
does, I think this is in fairly good shape.

--
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

#209Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Ashutosh Bapat (#207)
Re: Partition-wise join for join between (declaratively) partitioned tables

On 2017/09/09 2:38, Ashutosh Bapat wrote:

On Fri, Sep 8, 2017 at 11:17 AM, Amit Langote wrote:

I updated the patch to include just those changes. I'm not sure about
one of the Ashutosh's changes whereby the child PlanRowMark is also passed
to expand_partitioned_rtentry() to use as the parent PlanRowMark. I think
the child RTE, child RT index and child Relation are fine, because they
are necessary for creating AppendRelInfos in a desired way for later
planning steps. But PlanRowMarks are not processed within the planner
afterwards and do not need to be marked with the immediate parent-child
association in the same way that AppendRelInfos need to be.

Passing top parent's row mark works today, since there is no
parent-child specific translation happening there. But if in future,
we introduce such a translation, row marks for indirect children in a
multi-level partitioned hierarchy won't be accurate. So, I think it's
better to pass row marks of the direct parent.

IMHO, we should make it the responsibility of the future patch to set a
child PlanRowMark's prti to the direct parent's RT index, when we actually
know that it's needed for something. We clearly know today why we need to
pass the other objects like child RT entry, RT index, and Relation, so we
should limit this patch to pass only those objects to the recursive call.
That makes this patch a relatively easy to understand change.

I also included the changes to add_paths_to_append_rel() from my patch on
the "path toward faster partition pruning" thread. We'd need that change,
because while add_paths_to_append_rel() is called for all partitioned
table RTEs in a given partition tree, expand_inherited_rtentry() would
have set up a PartitionedChildRelInfo only for the root parent, so
get_partitioned_child_rels() would not find the same for non-root
partitioned table rels and crash failing the Assert. The changes I made
are such that we call get_partitioned_child_rels() only for the parent
rels that are known to correspond root partitioned tables (or as you
pointed out on the thread, "the table named in the query" as opposed those
added to the query as result of inheritance expansion). In addition to
the relkind check on the input RTE, it would seem that checking that the
reloptkind is RELOPT_BASEREL would be enough. But actually not, because
if a partitioned table is accessed in a UNION ALL query, reloptkind even
for the root partitioned table (the table named in the query) would be
RELOPT_OTHER_MEMBER_REL. The only way to confirm that the input rel is
actually the root partitioned table is to check whether its parent rel is
not RTE_RELATION, because the parent in case of UNION ALL Append is a
RTE_SUBQUERY RT entry.

There was a change in my 0003 patch, which fixed the crash. It checked
for RELOPT_BASEREL and RELKIND_PARTITIONED_TABLE. I have pulled it in
my 0001 patch. It no more crashes. I tried various queries involving
set operations and bare multi-level partitioned table scan with my
patch, but none of them showed any anomaly. Do you have a testcase
which shows problem with my patch? May be your suggestion is correct,
but corresponding code implementation is slightly longer than I would
expect. So, we should go with it, if there is corresponding testcase
which shows why it's needed.

If we go with your patch, partitioned tables won't get locked, for
example, in case of the following query (p is a partitioned table):

select 1 from p union all select 2 from p;

That's because the RelOptInfos for the two instances of p in the above
query are RELOPT_OTHER_MEMBER_REL, not RELOPT_BASEREL. They are children
of the Append corresponding to the UNION ALL subquery RTE. So,
partitioned_rels does not get set per your proposed code.

In your patch

+            parent_rel = root->simple_rel_array[parent_relid];
+            get_pcinfo = (parent_rel->rtekind == RTE_SUBQUERY);

Do you mean RTE_RELATION as you explained above?

No, I mean RTE_SUBQUERY.

If the partitioned table RTE in question corresponds to one named in the
query, we should be able to find its pcinfo in root->pcinfo_list. If the
partitioned table RTE is one added as result of inheritance expansion, it
won't have an associated pcinfo. So, we should find a way to distinguish
them from one another. The first idea that had occurred to me was the
same as yours -- RelOptInfo of the partitioned table RTE named in the
query would be of reloptkind RELOPT_BASEREL and those of the partitioned
table RTE added as result of inheritance expansion will be of reloptkind
RELOPT_OTHER_MEMBER_REL. Although the latter is always true, the former
is not. If the partitioned table named in the query appears under UNION
ALL query, then its reloptkind will be RELOPT_OTHER_MEMBER_REL. That
means we have to use some other means to distinguish partitioned table
RTEs that have an associated pcinfo from those that don't. So, I devised
this method of looking at the parent RTE (if any) for distinguishing the
two. Partitioned table named in the query either doesn't have the parent
or if it does, the parent could only ever be a UNION ALL subquery
(RTE_SUBQUERY). Partitioned tables added as part of inheritance expansion
will always have the parent and the parent will only ever be a table
(RTE_RELATION).

One thing I notice is that if I rip out the changes to initsplan.c,
the new regression test still passes. If it's possible to write a
test that fails without those changes, I think it would be a good idea
to include one in the patch. That's certainly one of the subtler
parts of this patch, IMHO.

Back when this (step-wise expansion of partition inheritance) used to be a
patch in the original declarative partitioning patch series, Ashutosh had
reported a test query [1] that would fail getting a plan, for which we
came up with the initsplan.c changes in this patch as the solution:

ERROR: could not devise a query plan for the given query

I tried that query again without the initsplan.c changes and somehow the
same error does not occur anymore. It's strange because without the
initsplan.c changes, there is no way for partitions lower in the tree than
the first level to get the direct_lateral_relids and lateral_relids from
the root parent rel. Maybe, Ashutosh has a way to devise the failing
query again.

Thanks a lot for the reference. I devised a testcase slightly
modifying my original test. I have included the test in the latest
patch set.

I have included Robert's changes to parts other than
expand_inherited_rtentry() in the patch.

I also confirmed that the partition-pruning patch set works fine with this
patch instead of the patch on that thread with the same functionality,
which I will now drop from that patch set. Sorry about the wasted time.

Thanks a lot. Please review the patch in the updated patchset.

Some comments:

In create_lateral_join_info():

+ Assert(IS_SIMPLE_REL(brel));
+ Assert(brte);

The second Assert is either unnecessary or should be placed first.

The following comment could be made a bit clearer.

+         * In the case of table inheritance, the parent RTE is directly
linked
+         * to every child table via an AppendRelInfo.  In the case of table
+         * partitioning, the inheritance hierarchy is expanded one level at a
+         * time rather than flattened.  Therefore, an other member rel
that is
+         * a partitioned table may have children of its own, and must
+         * therefore be marked with the appropriate lateral info so that
those
+         * children eventually get marked also.

How about: In the case of partitioned table inheritance, the original
parent RTE is linked, via AppendRelInfo, only to its immediate partitions.
Partitions below the first level are accessible only via their immediate
parent's RelOptInfo, which would be of kind RELOPT_OTHER_MEMBER_REL, so
consider those as well.

In expand_inherited_rtentry(), the following comment fragment is obsolete,
because we *do* now create AppendRelInfo's for partitioned children:

+        /*
+         * We keep a list of objects in root, each of which maps a
partitioned
+         * parent RT index to the list of RT indexes of its partitioned child
+         * tables which do not have AppendRelInfos associated with those.

By the way, when we call expand_single_inheritance_child() in the
non-partitioned inheritance case, we should pass NULL for childrte_p,
childRTindex_p, childrc_p, instead of declaring variables that won't be
used. Hence, expand_single_inheritance_child() should make those
arguments optional.

+
+    /*
+     * If the partitioned table has no partitions or all the partitions are
+     * temporary tables from other backends, treat this as non-inheritance
+     * case.
+     */
+    if (!has_child)
+        parentrte->inh = false;

I guess the above applies to all partitioned tables in the tree, so, I
think we should update the comment in set_rel_size():

else if (rte->relkind == RELKIND_PARTITIONED_TABLE)
{
/*
* A partitioned table without leaf partitions is marked
* as a dummy rel.
*/
set_dummy_rel_pathlist(rel);
}

to say: a partitioned table without partitions is marked as a dummy rel.

Thanks,
Amit

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

#210Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Robert Haas (#208)
Re: Partition-wise join for join between (declaratively) partitioned tables

On 2017/09/09 9:58, Robert Haas wrote:

I'm a bit suspicious about the fact that there are now executor
changes related to the PlanRowMarks. If the rowmark's prti is now the
intermediate parent's RT index rather than the top-parent's RT index,
it'd seem like that'd matter somehow. Maybe it doesn't, because the
code that cares about prti seems to only care about whether it's
different from rti.

Yes, it doesn't matter. The important point though is that nothing we
want to do in the short term requires us to set a child PlanRowMark's prti
to its immediate parent's RT index, as I also mentioned in reply to Ashutosh.

But if that's true everywhere, then why even
change this? I think we might be well off not to tinker with things
that don't need to be changed.

+1.

Apart from that concern, now that I understand (from my own failed
attempt and some off-list discussion) why this patch works the way it
does, I think this is in fairly good shape.

I too think so, except we still need to incorporate changes to
add_paths_to_append_rel() necessary to correctly set partitioned_rels, as
I explained in reply Ashutosh.

Thanks,
Amit

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

#211Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#208)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Sat, Sep 9, 2017 at 6:28 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Fri, Sep 8, 2017 at 1:38 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

I also confirmed that the partition-pruning patch set works fine with this
patch instead of the patch on that thread with the same functionality,
which I will now drop from that patch set. Sorry about the wasted time.

Thanks a lot. Please review the patch in the updated patchset.

In set_append_rel_size(), I don't find the comment too clear (and this
part was taken from Amit's patch, right?). I suggest:

No, I didn't take it from Amit's patch. Both of us have different
wordings. But yours is better than both of us. Included it in the
attached patches.

+    /*
+     * Associate the partitioned tables which are descendents of the table
+     * named in the query with the topmost append path (i.e. the one where
+     * rel->reloptkind is RELOPT_BASEREL).  This ensures that they get properly
+     * locked at execution time.
+     */

I'm a bit suspicious about the fact that there are now executor
changes related to the PlanRowMarks. If the rowmark's prti is now the
intermediate parent's RT index rather than the top-parent's RT index,
it'd seem like that'd matter somehow. Maybe it doesn't, because the
code that cares about prti seems to only care about whether it's
different from rti. But if that's true everywhere, then why even
change this? I think we might be well off not to tinker with things
that don't need to be changed.

In the definition of ExecRowMark, I see
Index prti; /* parent range table index, if child */
It just says parent, by which I take as direct parent. For
inheritance, which earlier flattened inheritance hierarchy, direct
parent was top parent. So, probably nobody thought whether a parent is
direct parent or top parent. But now that we have introduced that
concept we need to interpret this comment anew. And I think
interpreting it as direct parent is non-lossy. If we set top parent's
index, parent RTI in AppendRelInfo and PlanRowMark would not agree.
So, it looks quite natural that we set the direct parent's index in
PlanRowMark. From that POV, we aren't changing anything, we are
setting the same parent RTI in AppendRelInfo and PlanRowMark. Chaning
different parent RTIs in those two structure would be a real change.

--
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

#212Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Ashutosh Bapat (#211)
Re: Partition-wise join for join between (declaratively) partitioned tables

On 2017/09/11 16:23, Ashutosh Bapat wrote:

On Sat, Sep 9, 2017 at 6:28 AM, Robert Haas <robertmhaas@gmail.com> wrote:

I'm a bit suspicious about the fact that there are now executor
changes related to the PlanRowMarks. If the rowmark's prti is now the
intermediate parent's RT index rather than the top-parent's RT index,
it'd seem like that'd matter somehow. Maybe it doesn't, because the
code that cares about prti seems to only care about whether it's
different from rti. But if that's true everywhere, then why even
change this? I think we might be well off not to tinker with things
that don't need to be changed.

In the definition of ExecRowMark, I see
Index prti; /* parent range table index, if child */
It just says parent, by which I take as direct parent. For
inheritance, which earlier flattened inheritance hierarchy, direct
parent was top parent. So, probably nobody thought whether a parent is
direct parent or top parent. But now that we have introduced that
concept we need to interpret this comment anew. And I think
interpreting it as direct parent is non-lossy.

But then we also don't have anything to say about why we're making that
change. If you could describe what non-lossy is in this context, then
fine. But that we'd like to match with what we're going to do for
AppendRelInfos does not seem to be a sufficient explanation for this change.

If we set top parent's
index, parent RTI in AppendRelInfo and PlanRowMark would not agree.
So, it looks quite natural that we set the direct parent's index in
PlanRowMark.

They would not agree, yes, but aren't they unrelated? If we have a reason
for them to agree, (for example, row-locking breaks in the inherited table
case if we didn't), then we should definitely make them agree.

Updating the comment for prti definition might be something that this
patch could (should?) do, but I'm not quite sure about that too.

Thanks,
Amit

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

#213Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Amit Langote (#209)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Mon, Sep 11, 2017 at 12:16 PM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

On 2017/09/09 2:38, Ashutosh Bapat wrote:

On Fri, Sep 8, 2017 at 11:17 AM, Amit Langote wrote:

I updated the patch to include just those changes. I'm not sure about
one of the Ashutosh's changes whereby the child PlanRowMark is also passed
to expand_partitioned_rtentry() to use as the parent PlanRowMark. I think
the child RTE, child RT index and child Relation are fine, because they
are necessary for creating AppendRelInfos in a desired way for later
planning steps. But PlanRowMarks are not processed within the planner
afterwards and do not need to be marked with the immediate parent-child
association in the same way that AppendRelInfos need to be.

Passing top parent's row mark works today, since there is no
parent-child specific translation happening there. But if in future,
we introduce such a translation, row marks for indirect children in a
multi-level partitioned hierarchy won't be accurate. So, I think it's
better to pass row marks of the direct parent.

IMHO, we should make it the responsibility of the future patch to set a
child PlanRowMark's prti to the direct parent's RT index, when we actually
know that it's needed for something. We clearly know today why we need to
pass the other objects like child RT entry, RT index, and Relation, so we
should limit this patch to pass only those objects to the recursive call.
That makes this patch a relatively easy to understand change.

I think you are mixing two issues here 1. setting parent RTI in child
PlanRowMark and 2. passing immediate parent's PlanRowMark to
expand_single_inheritance_child().

I have discussed 1 in my reply to Robert.

About 2 you haven't given any particular comments to my reply. To me
it looks like it's this patch that introduces the notion of
multi-level expansion, so it's natural for this patch to pass
PlanRowMark in cascaded fashion similar to other structures.

I also included the changes to add_paths_to_append_rel() from my patch on
the "path toward faster partition pruning" thread. We'd need that change,
because while add_paths_to_append_rel() is called for all partitioned
table RTEs in a given partition tree, expand_inherited_rtentry() would
have set up a PartitionedChildRelInfo only for the root parent, so
get_partitioned_child_rels() would not find the same for non-root
partitioned table rels and crash failing the Assert. The changes I made
are such that we call get_partitioned_child_rels() only for the parent
rels that are known to correspond root partitioned tables (or as you
pointed out on the thread, "the table named in the query" as opposed those
added to the query as result of inheritance expansion). In addition to
the relkind check on the input RTE, it would seem that checking that the
reloptkind is RELOPT_BASEREL would be enough. But actually not, because
if a partitioned table is accessed in a UNION ALL query, reloptkind even
for the root partitioned table (the table named in the query) would be
RELOPT_OTHER_MEMBER_REL. The only way to confirm that the input rel is
actually the root partitioned table is to check whether its parent rel is
not RTE_RELATION, because the parent in case of UNION ALL Append is a
RTE_SUBQUERY RT entry.

There was a change in my 0003 patch, which fixed the crash. It checked
for RELOPT_BASEREL and RELKIND_PARTITIONED_TABLE. I have pulled it in
my 0001 patch. It no more crashes. I tried various queries involving
set operations and bare multi-level partitioned table scan with my
patch, but none of them showed any anomaly. Do you have a testcase
which shows problem with my patch? May be your suggestion is correct,
but corresponding code implementation is slightly longer than I would
expect. So, we should go with it, if there is corresponding testcase
which shows why it's needed.

If we go with your patch, partitioned tables won't get locked, for
example, in case of the following query (p is a partitioned table):

select 1 from p union all select 2 from p;

That's because the RelOptInfos for the two instances of p in the above
query are RELOPT_OTHER_MEMBER_REL, not RELOPT_BASEREL. They are children
of the Append corresponding to the UNION ALL subquery RTE. So,
partitioned_rels does not get set per your proposed code.

Session 1:
postgres=# begin;
BEGIN
postgres=# select 1 from t1 union all select 2 from t1;
?column?
----------
(0 rows)

postgres=# select pg_backend_pid();
pg_backend_pid
----------------
28843
(1 row)

Session 2
postgres=# select locktype, relation::regclass, virtualxid,
virtualtransaction, pid, mode, granted, fastpath from pg_locks;
locktype | relation | virtualxid | virtualtransaction | pid |
mode | granted | fastpath
------------+----------+------------+--------------------+-------+-----------------+---------+----------
relation | pg_locks | | 4/14 | 28854 |
AccessShareLock | t | t
virtualxid | | 4/14 | 4/14 | 28854 |
ExclusiveLock | t | t
relation | t1p1p1 | | 3/9 | 28843 |
AccessShareLock | t | t
relation | t1p1 | | 3/9 | 28843 |
AccessShareLock | t | t
relation | t1 | | 3/9 | 28843 |
AccessShareLock | t | t
virtualxid | | 3/9 | 3/9 | 28843 |
ExclusiveLock | t | t
(6 rows)

So, all partitioned partitions are getting locked correctly. Am I
missing something?

In your patch

+            parent_rel = root->simple_rel_array[parent_relid];
+            get_pcinfo = (parent_rel->rtekind == RTE_SUBQUERY);

Do you mean RTE_RELATION as you explained above?

No, I mean RTE_SUBQUERY.

If the partitioned table RTE in question corresponds to one named in the
query, we should be able to find its pcinfo in root->pcinfo_list. If the
partitioned table RTE is one added as result of inheritance expansion, it
won't have an associated pcinfo. So, we should find a way to distinguish
them from one another. The first idea that had occurred to me was the
same as yours -- RelOptInfo of the partitioned table RTE named in the
query would be of reloptkind RELOPT_BASEREL and those of the partitioned
table RTE added as result of inheritance expansion will be of reloptkind
RELOPT_OTHER_MEMBER_REL. Although the latter is always true, the former
is not. If the partitioned table named in the query appears under UNION
ALL query, then its reloptkind will be RELOPT_OTHER_MEMBER_REL. That
means we have to use some other means to distinguish partitioned table
RTEs that have an associated pcinfo from those that don't. So, I devised
this method of looking at the parent RTE (if any) for distinguishing the
two. Partitioned table named in the query either doesn't have the parent
or if it does, the parent could only ever be a UNION ALL subquery
(RTE_SUBQUERY). Partitioned tables added as part of inheritance expansion
will always have the parent and the parent will only ever be a table
(RTE_RELATION).

Actually, the original problem that caused this discussion started
with an assertion failure in get_partitioned_child_rels() as
Assert(list_length(result) >= 1);

This assertion fails if result is NIL when an intermediate partitioned
table is passed. May be we should assert (result == NIL ||
list_length(result) == 1) and allow that function to be called even
for intermediate partitioned partitions for which the function will
return NIL. That will leave the code in add_paths_to_append_rel()
simple. Thoughts?

In create_lateral_join_info():

+ Assert(IS_SIMPLE_REL(brel));
+ Assert(brte);

The second Assert is either unnecessary or should be placed first.

simple_rte_array[] may have some NULL entries. Second assert makes
sure that we aren't dealing with a NULL entry. Any particular reason
to reorder the asserts?

The following comment could be made a bit clearer.

+         * In the case of table inheritance, the parent RTE is directly
linked
+         * to every child table via an AppendRelInfo.  In the case of table
+         * partitioning, the inheritance hierarchy is expanded one level at a
+         * time rather than flattened.  Therefore, an other member rel
that is
+         * a partitioned table may have children of its own, and must
+         * therefore be marked with the appropriate lateral info so that
those
+         * children eventually get marked also.

How about: In the case of partitioned table inheritance, the original
parent RTE is linked, via AppendRelInfo, only to its immediate partitions.
Partitions below the first level are accessible only via their immediate
parent's RelOptInfo, which would be of kind RELOPT_OTHER_MEMBER_REL, so
consider those as well.

I don't see much difference between those two. We usually do not use
macros in comments, so usually comments mention "other member" rel.
Let's leave this for the committer to judge.

In expand_inherited_rtentry(), the following comment fragment is obsolete,
because we *do* now create AppendRelInfo's for partitioned children:

+        /*
+         * We keep a list of objects in root, each of which maps a
partitioned
+         * parent RT index to the list of RT indexes of its partitioned child
+         * tables which do not have AppendRelInfos associated with those.

Good catch. I have reworded it as
/*
* We keep a list of objects in root, each of which maps a root
* partitioned parent RT index to the list of RT indexes of descendant
* partitioned child tables.

Does that look good?

By the way, when we call expand_single_inheritance_child() in the
non-partitioned inheritance case, we should pass NULL for childrte_p,
childRTindex_p, childrc_p, instead of declaring variables that won't be
used. Hence, expand_single_inheritance_child() should make those
arguments optional.

That introduces an extra "if" condition, which is costlier than an
assignment. We have used both the styles in the code. Previously, I
have got comments otherwise. So, I am not sure.

+
+    /*
+     * If the partitioned table has no partitions or all the partitions are
+     * temporary tables from other backends, treat this as non-inheritance
+     * case.
+     */
+    if (!has_child)
+        parentrte->inh = false;

I guess the above applies to all partitioned tables in the tree, so, I
think we should update the comment in set_rel_size():

else if (rte->relkind == RELKIND_PARTITIONED_TABLE)
{
/*
* A partitioned table without leaf partitions is marked
* as a dummy rel.
*/
set_dummy_rel_pathlist(rel);
}

to say: a partitioned table without partitions is marked as a dummy rel.

Done. Thanks again for the catch.

I will update the patches once we have some resolution about 1. prti
in PlanRowMarks and 2. detection of root partitioned table in
add_paths_to_append_rel().

--
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

#214Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Amit Langote (#212)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Mon, Sep 11, 2017 at 2:16 PM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

On 2017/09/11 16:23, Ashutosh Bapat wrote:

On Sat, Sep 9, 2017 at 6:28 AM, Robert Haas <robertmhaas@gmail.com> wrote:

I'm a bit suspicious about the fact that there are now executor
changes related to the PlanRowMarks. If the rowmark's prti is now the
intermediate parent's RT index rather than the top-parent's RT index,
it'd seem like that'd matter somehow. Maybe it doesn't, because the
code that cares about prti seems to only care about whether it's
different from rti. But if that's true everywhere, then why even
change this? I think we might be well off not to tinker with things
that don't need to be changed.

In the definition of ExecRowMark, I see
Index prti; /* parent range table index, if child */
It just says parent, by which I take as direct parent. For
inheritance, which earlier flattened inheritance hierarchy, direct
parent was top parent. So, probably nobody thought whether a parent is
direct parent or top parent. But now that we have introduced that
concept we need to interpret this comment anew. And I think
interpreting it as direct parent is non-lossy.

But then we also don't have anything to say about why we're making that
change. If you could describe what non-lossy is in this context, then
fine.

By setting prti to the topmost parent RTI we are loosing information
that this child may be an intermediate child similar to what we did
earlier to AppendRelInfo. That's the lossy-ness in this context.

But that we'd like to match with what we're going to do for
AppendRelInfos does not seem to be a sufficient explanation for this change.

The purpose of this patch is to change the parent-child linkages for
partitioned table and prti is one of them. So, in fact, I am wondering
why not to change that along with AppendRelInfo.

If we set top parent's
index, parent RTI in AppendRelInfo and PlanRowMark would not agree.
So, it looks quite natural that we set the direct parent's index in
PlanRowMark.

They would not agree, yes, but aren't they unrelated? If we have a reason
for them to agree, (for example, row-locking breaks in the inherited table
case if we didn't), then we should definitely make them agree.

Updating the comment for prti definition might be something that this
patch could (should?) do, but I'm not quite sure about that too.

To me that looks backwards again for the reasons described above.

--
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

#215Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#213)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Mon, Sep 11, 2017 at 6:45 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

So, all partitioned partitions are getting locked correctly. Am I
missing something?

That's not a valid test. In that scenario, you're going to hold all
the locks acquired by the planner, all the locks acquired by the
rewriter, and all the locks acquired by the executor, but when using
prepared queries, it's possible to execute the plan after the planner
and rewriter locks are no longer held.

--
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

#216Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#215)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Mon, Sep 11, 2017 at 5:19 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Mon, Sep 11, 2017 at 6:45 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

So, all partitioned partitions are getting locked correctly. Am I
missing something?

That's not a valid test. In that scenario, you're going to hold all
the locks acquired by the planner, all the locks acquired by the
rewriter, and all the locks acquired by the executor, but when using
prepared queries, it's possible to execute the plan after the planner
and rewriter locks are no longer held.

I see the same thing when I use prepare and execute

Session 1
postgres=# prepare stmt as select 1 from t1 union all select 2 from t1;
PREPARE
postgres=# select pg_backend_pid();
pg_backend_pid
----------------
50912
(1 row)

postgres=# begin;
BEGIN
postgres=# execute stmt;
?column?
----------
(0 rows)

Session 2
postgres=# select locktype, relation::regclass, virtualxid,
virtualtransaction, pid, mode, granted, fastpath from pg_locks;
locktype | relation | virtualxid | virtualtransaction | pid |
mode | granted | fastpath
------------+----------+------------+--------------------+-------+-----------------+---------+----------
relation | pg_locks | | 4/4 | 50914 |
AccessShareLock | t | t
virtualxid | | 4/4 | 4/4 | 50914 |
ExclusiveLock | t | t
relation | t1p1p1 | | 3/12 | 50912 |
AccessShareLock | t | t
relation | t1p1 | | 3/12 | 50912 |
AccessShareLock | t | t
relation | t1 | | 3/12 | 50912 |
AccessShareLock | t | t
virtualxid | | 3/12 | 3/12 | 50912 |
ExclusiveLock | t | t
(6 rows)

--
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

#217Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#216)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Mon, Sep 11, 2017 at 8:07 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

I see the same thing when I use prepare and execute

Hmm. Well, that's good, but it doesn't prove there's no bug. We have
to understand where and why it's getting locked to know whether the
behavior will be correct in all cases. I haven't had time to look at
Amit's comments in detail yet so I don't know whether I agree with his
analysis or not, but we have to look at what's going on under the hood
to know whether the engine is working -- not just listen to the noise
it makes.

--
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

#218Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Ashutosh Bapat (#213)
Re: Partition-wise join for join between (declaratively) partitioned tables

On 2017/09/11 19:45, Ashutosh Bapat wrote:

On Mon, Sep 11, 2017 at 12:16 PM, Amit Langote wrote:

IMHO, we should make it the responsibility of the future patch to set a
child PlanRowMark's prti to the direct parent's RT index, when we actually
know that it's needed for something. We clearly know today why we need to
pass the other objects like child RT entry, RT index, and Relation, so we
should limit this patch to pass only those objects to the recursive call.
That makes this patch a relatively easy to understand change.

I think you are mixing two issues here 1. setting parent RTI in child
PlanRowMark and 2. passing immediate parent's PlanRowMark to
expand_single_inheritance_child().

I have discussed 1 in my reply to Robert.

About 2 you haven't given any particular comments to my reply. To me
it looks like it's this patch that introduces the notion of
multi-level expansion, so it's natural for this patch to pass
PlanRowMark in cascaded fashion similar to other structures.

You patch does 2 to be able to do 1, doesn't it? That is, to be able to
set the child PlanRowMark's prti to the direct parent's RT index, you pass
the immediate parent's PlanRowMark to the recursive call of
expand_single_inheritance_child().

All I am trying to say is that this patch's mission is to expand
inheritance step-wise to be able to do certain things in the *planner*
that weren't possible before. The patch accomplishes that by creating
child AppendRelInfos such that its parent_relid field is set to the
immediate parent's RT index. It's quite clear why we're doing so. It's
not clear why we should do so for PlanRowMarks too. Maybe it's fine as
long as nothing breaks.

If we go with your patch, partitioned tables won't get locked, for
example, in case of the following query (p is a partitioned table):

select 1 from p union all select 2 from p;

That's because the RelOptInfos for the two instances of p in the above
query are RELOPT_OTHER_MEMBER_REL, not RELOPT_BASEREL. They are children
of the Append corresponding to the UNION ALL subquery RTE. So,
partitioned_rels does not get set per your proposed code.

[...]

So, all partitioned partitions are getting locked correctly. Am I
missing something?

Will reply to this separately to your other email.

Actually, the original problem that caused this discussion started
with an assertion failure in get_partitioned_child_rels() as
Assert(list_length(result) >= 1);

This assertion fails if result is NIL when an intermediate partitioned
table is passed. May be we should assert (result == NIL ||
list_length(result) == 1) and allow that function to be called even
for intermediate partitioned partitions for which the function will
return NIL. That will leave the code in add_paths_to_append_rel()
simple. Thoughts?

Yeah, I guess that could work. We'll just have to write comments to
describe why the Assert is written that way.

In create_lateral_join_info():

+ Assert(IS_SIMPLE_REL(brel));
+ Assert(brte);

The second Assert is either unnecessary or should be placed first.

simple_rte_array[] may have some NULL entries. Second assert makes
sure that we aren't dealing with a NULL entry. Any particular reason
to reorder the asserts?

Sorry, I missed that the 2nd Assert has b"rte". I thought it's b"rel".

The following comment could be made a bit clearer.

+         * In the case of table inheritance, the parent RTE is directly
linked
+         * to every child table via an AppendRelInfo.  In the case of table
+         * partitioning, the inheritance hierarchy is expanded one level at a
+         * time rather than flattened.  Therefore, an other member rel
that is
+         * a partitioned table may have children of its own, and must
+         * therefore be marked with the appropriate lateral info so that
those
+         * children eventually get marked also.

How about: In the case of partitioned table inheritance, the original
parent RTE is linked, via AppendRelInfo, only to its immediate partitions.
Partitions below the first level are accessible only via their immediate
parent's RelOptInfo, which would be of kind RELOPT_OTHER_MEMBER_REL, so
consider those as well.

I don't see much difference between those two. We usually do not use
macros in comments, so usually comments mention "other member" rel.
Let's leave this for the committer to judge.

Sure.

In expand_inherited_rtentry(), the following comment fragment is obsolete,
because we *do* now create AppendRelInfo's for partitioned children:

+        /*
+         * We keep a list of objects in root, each of which maps a
partitioned
+         * parent RT index to the list of RT indexes of its partitioned child
+         * tables which do not have AppendRelInfos associated with those.

Good catch. I have reworded it as
/*
* We keep a list of objects in root, each of which maps a root
* partitioned parent RT index to the list of RT indexes of descendant
* partitioned child tables.

Does that look good?

Looks fine.

By the way, when we call expand_single_inheritance_child() in the
non-partitioned inheritance case, we should pass NULL for childrte_p,
childRTindex_p, childrc_p, instead of declaring variables that won't be
used. Hence, expand_single_inheritance_child() should make those
arguments optional.

That introduces an extra "if" condition, which is costlier than an
assignment. We have used both the styles in the code. Previously, I
have got comments otherwise. So, I am not sure.

OK. expand_single_inheritance_child's header comment does not mention the
new result fields. Maybe add a comment describing what their role is and
that they're not optional arguments.

I will update the patches once we have some resolution about 1. prti
in PlanRowMarks and 2. detection of root partitioned table in
add_paths_to_append_rel().

OK.

About 2, I somewhat agree with your proposed solution above, which might
be simpler to explain in comments than the code I proposed.

Thanks,
Amit

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

#219Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Amit Langote (#218)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Sep 12, 2017 at 7:31 AM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

On 2017/09/11 19:45, Ashutosh Bapat wrote:

On Mon, Sep 11, 2017 at 12:16 PM, Amit Langote wrote:

IMHO, we should make it the responsibility of the future patch to set a
child PlanRowMark's prti to the direct parent's RT index, when we actually
know that it's needed for something. We clearly know today why we need to
pass the other objects like child RT entry, RT index, and Relation, so we
should limit this patch to pass only those objects to the recursive call.
That makes this patch a relatively easy to understand change.

I think you are mixing two issues here 1. setting parent RTI in child
PlanRowMark and 2. passing immediate parent's PlanRowMark to
expand_single_inheritance_child().

I have discussed 1 in my reply to Robert.

About 2 you haven't given any particular comments to my reply. To me
it looks like it's this patch that introduces the notion of
multi-level expansion, so it's natural for this patch to pass
PlanRowMark in cascaded fashion similar to other structures.

You patch does 2 to be able to do 1, doesn't it? That is, to be able to
set the child PlanRowMark's prti to the direct parent's RT index, you pass
the immediate parent's PlanRowMark to the recursive call of
expand_single_inheritance_child().

No. child PlanRowMark's prti is set to parentRTIndex, which is a
separate argument and is used to also set parent_relid in
AppendRelInfo.

Actually, the original problem that caused this discussion started
with an assertion failure in get_partitioned_child_rels() as
Assert(list_length(result) >= 1);

This assertion fails if result is NIL when an intermediate partitioned
table is passed. May be we should assert (result == NIL ||
list_length(result) == 1) and allow that function to be called even
for intermediate partitioned partitions for which the function will
return NIL. That will leave the code in add_paths_to_append_rel()
simple. Thoughts?

Yeah, I guess that could work. We'll just have to write comments to
describe why the Assert is written that way.

By the way, when we call expand_single_inheritance_child() in the
non-partitioned inheritance case, we should pass NULL for childrte_p,
childRTindex_p, childrc_p, instead of declaring variables that won't be
used. Hence, expand_single_inheritance_child() should make those
arguments optional.

That introduces an extra "if" condition, which is costlier than an
assignment. We have used both the styles in the code. Previously, I
have got comments otherwise. So, I am not sure.

OK. expand_single_inheritance_child's header comment does not mention the
new result fields. Maybe add a comment describing what their role is and
that they're not optional arguments.

I will update the patches once we have some resolution about 1. prti
in PlanRowMarks and 2. detection of root partitioned table in
add_paths_to_append_rel().

OK.

About 2, I somewhat agree with your proposed solution above, which might
be simpler to explain in comments than the code I proposed.

After testing a few queries I am getting a feeling that
ExecLockNonLeafAppendTables isn't really locking anything. I will
write more about that in my reply to Robert's mail.

--
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

#220Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Ashutosh Bapat (#216)
Re: Partition-wise join for join between (declaratively) partitioned tables

On 2017/09/11 21:07, Ashutosh Bapat wrote:

On Mon, Sep 11, 2017 at 5:19 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Mon, Sep 11, 2017 at 6:45 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

So, all partitioned partitions are getting locked correctly. Am I
missing something?

That's not a valid test. In that scenario, you're going to hold all
the locks acquired by the planner, all the locks acquired by the
rewriter, and all the locks acquired by the executor, but when using
prepared queries, it's possible to execute the plan after the planner
and rewriter locks are no longer held.

I see the same thing when I use prepare and execute

So I looked at this a bit closely and came to the conclusion that we may
not need to keep partitioned table RT indexes in the
(Merge)Append.partitioned_rels after all, as far as execution-time locking
is concerned.

Consider two cases:

1. Plan is created and executed in the same transaction

In this case, locks taken on the partitioned tables by the planner will
suffice.

2. Plan is executed in a different transaction from the one in which it
was created (a cached plan)

In this case, AcquireExecutorLocks will lock all the relations in
PlannedStmt.rtable, which must include all partitioned tables of all
partition trees involved in the query. Of those, it will lock the tables
whose RT indexes appear in PlannedStmt.nonleafResultRelations with
RowExclusiveLock mode. PlannedStmt.nonleafResultRelations is a global
list of all partitioned table RT indexes obtained by concatenating
partitioned_rels lists of all ModifyTable nodes involved in the query
(set_plan_refs does that). We need to distinguish nonleafResultRelations,
because we need to take the stronger lock on a given table before any
weaker one if it happens to appear in the query as a non-result relation
too, to avoid lock strength upgrade deadlock hazard.

Moreover, because all the tables from plannedstmt->rtable, including the
partitioned tables, will be added to PlannedStmt.relationsOids, any
invalidation events affecting the partitioned tables (for example,
add/remove a partition) will cause the plan involving partitioned tables
to be recreated.

In none of this do we rely on the partitioned table RT indexes appearing
in the (Merge)Append node itself. Maybe, we should just remove
partitioned_rels from (Merge)AppendPath and (Merge)Append node in a
separate patch and move on.

Thoughts?

Thanks,
Amit

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

#221Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Amit Langote (#220)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Sep 12, 2017 at 1:16 PM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

On 2017/09/11 21:07, Ashutosh Bapat wrote:

On Mon, Sep 11, 2017 at 5:19 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Mon, Sep 11, 2017 at 6:45 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

So, all partitioned partitions are getting locked correctly. Am I
missing something?

That's not a valid test. In that scenario, you're going to hold all
the locks acquired by the planner, all the locks acquired by the
rewriter, and all the locks acquired by the executor, but when using
prepared queries, it's possible to execute the plan after the planner
and rewriter locks are no longer held.

I see the same thing when I use prepare and execute

So I looked at this a bit closely and came to the conclusion that we may
not need to keep partitioned table RT indexes in the
(Merge)Append.partitioned_rels after all, as far as execution-time locking
is concerned.

Consider two cases:

1. Plan is created and executed in the same transaction

In this case, locks taken on the partitioned tables by the planner will
suffice.

2. Plan is executed in a different transaction from the one in which it
was created (a cached plan)

In this case, AcquireExecutorLocks will lock all the relations in
PlannedStmt.rtable, which must include all partitioned tables of all
partition trees involved in the query. Of those, it will lock the tables
whose RT indexes appear in PlannedStmt.nonleafResultRelations with
RowExclusiveLock mode. PlannedStmt.nonleafResultRelations is a global
list of all partitioned table RT indexes obtained by concatenating
partitioned_rels lists of all ModifyTable nodes involved in the query
(set_plan_refs does that). We need to distinguish nonleafResultRelations,
because we need to take the stronger lock on a given table before any
weaker one if it happens to appear in the query as a non-result relation
too, to avoid lock strength upgrade deadlock hazard.

Moreover, because all the tables from plannedstmt->rtable, including the
partitioned tables, will be added to PlannedStmt.relationsOids, any
invalidation events affecting the partitioned tables (for example,
add/remove a partition) will cause the plan involving partitioned tables
to be recreated.

In none of this do we rely on the partitioned table RT indexes appearing
in the (Merge)Append node itself. Maybe, we should just remove
partitioned_rels from (Merge)AppendPath and (Merge)Append node in a
separate patch and move on.

Thoughts?

Yes, I did the same analysis (to which I refer in my earlier reply to
you). I too think we should just remove partitioned_rels from Append
paths. But then the question is those are then transferred to
ModifyTable node in create_modifytable_plan() and use it for something
else. What should we do about that code? I don't think we are really
using that list from ModifyTable node as well, so may be we could
remove it from there as well. What do you think? Does that mean
partitioned_rels isn't used at all in the code?

--
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

#222Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Ashutosh Bapat (#221)
Re: Partition-wise join for join between (declaratively) partitioned tables

On 2017/09/12 16:55, Ashutosh Bapat wrote:

On Tue, Sep 12, 2017 at 1:16 PM, Amit Langote wrote:

So I looked at this a bit closely and came to the conclusion that we may
not need to keep partitioned table RT indexes in the
(Merge)Append.partitioned_rels after all, as far as execution-time locking
is concerned.

Consider two cases:

1. Plan is created and executed in the same transaction

In this case, locks taken on the partitioned tables by the planner will
suffice.

2. Plan is executed in a different transaction from the one in which it
was created (a cached plan)

In this case, AcquireExecutorLocks will lock all the relations in
PlannedStmt.rtable, which must include all partitioned tables of all
partition trees involved in the query. Of those, it will lock the tables
whose RT indexes appear in PlannedStmt.nonleafResultRelations with
RowExclusiveLock mode. PlannedStmt.nonleafResultRelations is a global
list of all partitioned table RT indexes obtained by concatenating
partitioned_rels lists of all ModifyTable nodes involved in the query
(set_plan_refs does that). We need to distinguish nonleafResultRelations,
because we need to take the stronger lock on a given table before any
weaker one if it happens to appear in the query as a non-result relation
too, to avoid lock strength upgrade deadlock hazard.

Moreover, because all the tables from plannedstmt->rtable, including the
partitioned tables, will be added to PlannedStmt.relationsOids, any
invalidation events affecting the partitioned tables (for example,
add/remove a partition) will cause the plan involving partitioned tables
to be recreated.

In none of this do we rely on the partitioned table RT indexes appearing
in the (Merge)Append node itself. Maybe, we should just remove
partitioned_rels from (Merge)AppendPath and (Merge)Append node in a
separate patch and move on.

Thoughts?

Yes, I did the same analysis (to which I refer in my earlier reply to
you). I too think we should just remove partitioned_rels from Append
paths. But then the question is those are then transferred to
ModifyTable node in create_modifytable_plan() and use it for something
else. What should we do about that code? I don't think we are really
using that list from ModifyTable node as well, so may be we could
remove it from there as well. What do you think? Does that mean
partitioned_rels isn't used at all in the code?

No, we cannot simply get rid of partitioned_rels altogether. We'll need
to keep it in the ModifyTable node, because we *do* need the
nonleafResultRelations list in PlannedStmt to distinguish partitioned
table result relations, which set_plan_refs builds by concatenating
partitioned_rels lists of various ModifyTable nodes of the query. The
PlannedStmt.nonleafResultRelations list actually has some use (which
parallels PlannedStmt.resultRelations), but partitioned_rels list in the
individual (Merge)Append, as it turns out, doesn't.

So, we can remove partitioned_rels from (Merge)AppendPath and
(Merge)Append nodes and remove ExecLockNonLeafAppendTables().

Thanks,
Amit

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

#223Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Ashutosh Bapat (#219)
Re: Partition-wise join for join between (declaratively) partitioned tables

On 2017/09/12 16:39, Ashutosh Bapat wrote:

On Tue, Sep 12, 2017 at 7:31 AM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

On 2017/09/11 19:45, Ashutosh Bapat wrote:

On Mon, Sep 11, 2017 at 12:16 PM, Amit Langote wrote:

IMHO, we should make it the responsibility of the future patch to set a
child PlanRowMark's prti to the direct parent's RT index, when we actually
know that it's needed for something. We clearly know today why we need to
pass the other objects like child RT entry, RT index, and Relation, so we
should limit this patch to pass only those objects to the recursive call.
That makes this patch a relatively easy to understand change.

I think you are mixing two issues here 1. setting parent RTI in child
PlanRowMark and 2. passing immediate parent's PlanRowMark to
expand_single_inheritance_child().

I have discussed 1 in my reply to Robert.

About 2 you haven't given any particular comments to my reply. To me
it looks like it's this patch that introduces the notion of
multi-level expansion, so it's natural for this patch to pass
PlanRowMark in cascaded fashion similar to other structures.

You patch does 2 to be able to do 1, doesn't it? That is, to be able to
set the child PlanRowMark's prti to the direct parent's RT index, you pass
the immediate parent's PlanRowMark to the recursive call of
expand_single_inheritance_child().

No. child PlanRowMark's prti is set to parentRTIndex, which is a
separate argument and is used to also set parent_relid in
AppendRelInfo.

OK. So, to keep the old behavior (if at all), we'd actually need a new
argument rootParentRTindex. Old behavior being that all child
PlanRowMarks has the rootParentRTindex as their prti.

It seems though that the new behavior where prti will now be set to the
direct parent's RT index is more or less harmless, because whatever we set
prti to, as long as it's different from rti, we can consider it a child
PlanRowMark. So it might be fine to set prti to direct parent's RT index.

That said, I noticed that we might need to be careful about what the value
of the root parent's PlanRowMark's allMarkType field gets set to. We need
to make sure that it reflects markType of all partitions in the tree,
including those that are not root parent's direct children. Is that true
with the proposed implementation?

Thanks,
Amit

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

#224Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Amit Langote (#222)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Sep 12, 2017 at 1:42 PM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

On 2017/09/12 16:55, Ashutosh Bapat wrote:

On Tue, Sep 12, 2017 at 1:16 PM, Amit Langote wrote:

So I looked at this a bit closely and came to the conclusion that we may
not need to keep partitioned table RT indexes in the
(Merge)Append.partitioned_rels after all, as far as execution-time locking
is concerned.

Consider two cases:

1. Plan is created and executed in the same transaction

In this case, locks taken on the partitioned tables by the planner will
suffice.

2. Plan is executed in a different transaction from the one in which it
was created (a cached plan)

In this case, AcquireExecutorLocks will lock all the relations in
PlannedStmt.rtable, which must include all partitioned tables of all
partition trees involved in the query. Of those, it will lock the tables
whose RT indexes appear in PlannedStmt.nonleafResultRelations with
RowExclusiveLock mode. PlannedStmt.nonleafResultRelations is a global
list of all partitioned table RT indexes obtained by concatenating
partitioned_rels lists of all ModifyTable nodes involved in the query
(set_plan_refs does that). We need to distinguish nonleafResultRelations,
because we need to take the stronger lock on a given table before any
weaker one if it happens to appear in the query as a non-result relation
too, to avoid lock strength upgrade deadlock hazard.

Moreover, because all the tables from plannedstmt->rtable, including the
partitioned tables, will be added to PlannedStmt.relationsOids, any
invalidation events affecting the partitioned tables (for example,
add/remove a partition) will cause the plan involving partitioned tables
to be recreated.

In none of this do we rely on the partitioned table RT indexes appearing
in the (Merge)Append node itself. Maybe, we should just remove
partitioned_rels from (Merge)AppendPath and (Merge)Append node in a
separate patch and move on.

Thoughts?

Yes, I did the same analysis (to which I refer in my earlier reply to
you). I too think we should just remove partitioned_rels from Append
paths. But then the question is those are then transferred to
ModifyTable node in create_modifytable_plan() and use it for something
else. What should we do about that code? I don't think we are really
using that list from ModifyTable node as well, so may be we could
remove it from there as well. What do you think? Does that mean
partitioned_rels isn't used at all in the code?

No, we cannot simply get rid of partitioned_rels altogether. We'll need
to keep it in the ModifyTable node, because we *do* need the
nonleafResultRelations list in PlannedStmt to distinguish partitioned
table result relations, which set_plan_refs builds by concatenating
partitioned_rels lists of various ModifyTable nodes of the query. The
PlannedStmt.nonleafResultRelations list actually has some use (which
parallels PlannedStmt.resultRelations), but partitioned_rels list in the
individual (Merge)Append, as it turns out, doesn't.

So, we can remove partitioned_rels from (Merge)AppendPath and
(Merge)Append nodes and remove ExecLockNonLeafAppendTables().

Don't we need partitioned_rels from Append paths to be transferred to
ModifyTable node or we have a different way of calculating
nonleafResultRelations?

--
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

#225Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Ashutosh Bapat (#224)
Re: Partition-wise join for join between (declaratively) partitioned tables

On 2017/09/12 17:53, Ashutosh Bapat wrote:

On Tue, Sep 12, 2017 at 1:42 PM, Amit Langote wrote:

So, we can remove partitioned_rels from (Merge)AppendPath and
(Merge)Append nodes and remove ExecLockNonLeafAppendTables().

Don't we need partitioned_rels from Append paths to be transferred to
ModifyTable node or we have a different way of calculating
nonleafResultRelations?

No, we don't transfer partitioned_rels from Append path to ModifyTable
node. inheritance_planner(), that builds the ModifyTable path for
UPDATE/DELETE on a partitioned table, fetches partitioned_rels from
root->pcinfo_list itself and passes it to create_modifytable_path. No
Append path is involved in that case. PlannedStmt.nonleafResultRelations
is built by concatenating the partitioned_rels lists of all ModifyTable
nodes appearing in the query. It does not depend on Append's or
AppendPath's partitioned_rels.

Thanks,
Amit

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

#226Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Amit Langote (#223)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Sep 12, 2017 at 2:17 PM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

That said, I noticed that we might need to be careful about what the value
of the root parent's PlanRowMark's allMarkType field gets set to. We need
to make sure that it reflects markType of all partitions in the tree,
including those that are not root parent's direct children. Is that true
with the proposed implementation?

Yes. We include child's allMarkTypes into parent's allMarkTypes. So,
top parent's PlanRowMarks should have all descendant's allMarkTypes,
which is not happening in the patch right now. There are two ways to
fix that.

1. Pass top parent's PlanRowMark all the way down to the leaf
partitions, so that current expand_single_inheritance_child() collects
allMarkTypes of all children correctly. But this way, PlanRowMarks of
intermediate parent does not reflect allMarkTypes of its children,
only top root records that.
2. Pass immediate parent's PlanRowMark to
expand_single_inheritance_child(), so that it records allMarkTypes of
its children. In expand_partitioned_rtentry() have following sequence

expand_single_inheritance_child(root, parentrte, parentRTindex,
parentrel, parentrc, childrel,
appinfos, &childrte, &childRTindex,
&childrc);

/* If this child is itself partitioned, recurse */
if (childrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
{
expand_partitioned_rtentry(root, childrte, childRTindex,
childrel, childrc, lockmode, appinfos,
partitioned_child_rels);

/* Include child's rowmark type in parent's allMarkTypes */
parentrc->allMarkTypes |= childrc->allMarkTypes;
}
so that we push allMarkTypes up the hierarchy.

I like the second way, since every intermediate parent records
allMarkTypes of its descendants.

Thoughts?
--
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

#227Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Ashutosh Bapat (#226)
Re: Partition-wise join for join between (declaratively) partitioned tables

On 2017/09/12 18:49, Ashutosh Bapat wrote:

On Tue, Sep 12, 2017 at 2:17 PM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

That said, I noticed that we might need to be careful about what the value
of the root parent's PlanRowMark's allMarkType field gets set to. We need
to make sure that it reflects markType of all partitions in the tree,
including those that are not root parent's direct children. Is that true
with the proposed implementation?

Yes. We include child's allMarkTypes into parent's allMarkTypes. So,
top parent's PlanRowMarks should have all descendant's allMarkTypes,
which is not happening in the patch right now. There are two ways to
fix that.

1. Pass top parent's PlanRowMark all the way down to the leaf
partitions, so that current expand_single_inheritance_child() collects
allMarkTypes of all children correctly. But this way, PlanRowMarks of
intermediate parent does not reflect allMarkTypes of its children,
only top root records that.
2. Pass immediate parent's PlanRowMark to
expand_single_inheritance_child(), so that it records allMarkTypes of
its children. In expand_partitioned_rtentry() have following sequence

expand_single_inheritance_child(root, parentrte, parentRTindex,
parentrel, parentrc, childrel,
appinfos, &childrte, &childRTindex,
&childrc);

/* If this child is itself partitioned, recurse */
if (childrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
{
expand_partitioned_rtentry(root, childrte, childRTindex,
childrel, childrc, lockmode, appinfos,
partitioned_child_rels);

/* Include child's rowmark type in parent's allMarkTypes */
parentrc->allMarkTypes |= childrc->allMarkTypes;
}
so that we push allMarkTypes up the hierarchy.

I like the second way, since every intermediate parent records
allMarkTypes of its descendants.

I like the second way, too.

Thanks,
Amit

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

#228Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Amit Langote (#225)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Sep 12, 2017 at 2:35 PM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

On 2017/09/12 17:53, Ashutosh Bapat wrote:

On Tue, Sep 12, 2017 at 1:42 PM, Amit Langote wrote:

So, we can remove partitioned_rels from (Merge)AppendPath and
(Merge)Append nodes and remove ExecLockNonLeafAppendTables().

Don't we need partitioned_rels from Append paths to be transferred to
ModifyTable node or we have a different way of calculating
nonleafResultRelations?

No, we don't transfer partitioned_rels from Append path to ModifyTable
node. inheritance_planner(), that builds the ModifyTable path for
UPDATE/DELETE on a partitioned table, fetches partitioned_rels from
root->pcinfo_list itself and passes it to create_modifytable_path. No
Append path is involved in that case. PlannedStmt.nonleafResultRelations
is built by concatenating the partitioned_rels lists of all ModifyTable
nodes appearing in the query. It does not depend on Append's or
AppendPath's partitioned_rels.

Ok. Thanks for the explanation.

This make me examine inheritance_planner() closely and I think I have
spotted a thinko there. In inheritance_planner() parent_rte is set to
the RTE of parent to start with and then in the loop
1132 /*
1133 * And now we can get on with generating a plan for each child table.
1134 */
1135 foreach(lc, root->append_rel_list)
1136 {
... code clipped
1165 /*
1166 * If there are securityQuals attached to the parent,
move them to the
1167 * child rel (they've already been transformed properly for that).
1168 */
1169 parent_rte = rt_fetch(parentRTindex, subroot->parse->rtable);
1170 child_rte = rt_fetch(appinfo->child_relid, subroot->parse->rtable);
1171 child_rte->securityQuals = parent_rte->securityQuals;
1172 parent_rte->securityQuals = NIL;

we set parent_rte to the one obtained from subroot->parse, which
happens to be the same (at least in contents) as original parent_rte.
Later we use this parent_rte to pull partitioned_rels outside that
loop

1371 if (parent_rte->relkind == RELKIND_PARTITIONED_TABLE)
1372 {
1373 partitioned_rels = get_partitioned_child_rels(root, parentRTindex);
1374 /* The root partitioned table is included as a child rel */
1375 Assert(list_length(partitioned_rels) >= 1);
1376 }

I think the code here expects the original parent_rte and not the one
we set around line 1169.

This isn't a bug right now, since both the parent_rte s have same
content. But I am not sure if that will remain to be so. Here's patch
to fix the thinko.

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

Attachments:

inh_planner_prte.patchapplication/octet-stream; name=inh_planner_prte.patchDownload
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 6b79b3a..9da0bca 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -1056,6 +1056,7 @@ inheritance_planner(PlannerInfo *root)
 	Index		rti;
 	RangeTblEntry *parent_rte;
 	List	   *partitioned_rels = NIL;
+	bool		is_parent_partitioned;
 
 	Assert(parse->commandType != CMD_INSERT);
 
@@ -1123,7 +1124,10 @@ inheritance_planner(PlannerInfo *root)
 	 */
 	parent_rte = rt_fetch(parentRTindex, root->parse->rtable);
 	if (parent_rte->relkind == RELKIND_PARTITIONED_TABLE)
+	{
 		nominalRelation = parentRTindex;
+		is_parent_partitioned = true;
+	}
 
 	/*
 	 * And now we can get on with generating a plan for each child table.
@@ -1368,7 +1372,7 @@ inheritance_planner(PlannerInfo *root)
 		Assert(!parse->onConflict);
 	}
 
-	if (parent_rte->relkind == RELKIND_PARTITIONED_TABLE)
+	if (is_parent_partitioned)
 	{
 		partitioned_rels = get_partitioned_child_rels(root, parentRTindex);
 		/* The root partitioned table is included as a child rel */
#229Robert Haas
robertmhaas@gmail.com
In reply to: Amit Langote (#220)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Sep 12, 2017 at 3:46 AM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

In this case, AcquireExecutorLocks will lock all the relations in
PlannedStmt.rtable, which must include all partitioned tables of all
partition trees involved in the query. Of those, it will lock the tables
whose RT indexes appear in PlannedStmt.nonleafResultRelations with
RowExclusiveLock mode. PlannedStmt.nonleafResultRelations is a global
list of all partitioned table RT indexes obtained by concatenating
partitioned_rels lists of all ModifyTable nodes involved in the query
(set_plan_refs does that). We need to distinguish nonleafResultRelations,
because we need to take the stronger lock on a given table before any
weaker one if it happens to appear in the query as a non-result relation
too, to avoid lock strength upgrade deadlock hazard.

Hmm. The problem with this theory in my view is that it doesn't
explain why InitPlan() and ExecOpenScanRelation() lock the relations
instead of just assuming that they are already locked either by
AcquireExecutorLocks or by planning. If ExecLockNonLeafAppendTables()
doesn't really need to take locks, then ExecOpenScanRelation() must
not need to do it either. We invented ExecLockNonLeafAppendTables()
on the occasion of removing the scans of those tables which would
previously have caused ExecOpenScanRelation() to be invoked, so as to
keep the locking behavior unchanged.

AcquireExecutorLocks() looks like an odd bit of code to me. The
executor itself locks result tables in InitPlan() and then everything
else during InitPlan() and all of the others later on while walking
the plan tree -- comments in InitPlan() say that this is to avoid a
lock upgrade hazard if a result rel is also a source rel. But
AcquireExecutorLocks() has no such provision; it just locks everything
in RTE order. In theory, that's a deadlock hazard of another kind, as
we just talked about in the context of EIBO. In fact, expanding in
bound order has made the situation worse: before, expansion order and
locking order were the same, so maybe having AcquireExecutorLocks()
work in RTE order coincidentally happened to give the same result as
the executor code itself as long as there are no result relations.
But this is certainly not true any more. I'm not sure it's worth
expending a lot of time on this -- it's evidently not a problem in
practice, or somebody probably would've complained before now.

But that having been said, I don't think we should assume that all the
locks taken from the executor are worthless because plancache.c will
always do the job for us. I don't know of a case where we execute a
saved plan without going through the plan cache, but that doesn't mean
that there isn't one or that there couldn't be one in the future.
It's not the job of these partitioning patches to whack around the way
we do locking in general -- they should preserve the existing behavior
as much as possible. If we want to get rid of the locking in the
executor altogether, that's a separate discussion where, I have a
feeling, there will prove to be better reasons for the way things are
than we are right now supposing.

--
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

#230Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Ashutosh Bapat (#228)
Re: Partition-wise join for join between (declaratively) partitioned tables

On 2017/09/12 19:56, Ashutosh Bapat wrote:

I think the code here expects the original parent_rte and not the one
we set around line 1169.

This isn't a bug right now, since both the parent_rte s have same
content. But I am not sure if that will remain to be so. Here's patch
to fix the thinko.

Instead of the new bool is_parent_partitioned, why not move the code to
set partitioned_rels to the block where you're now setting
is_parent_partitioned.

Also, since we know this isn't a bug at the moment but will turn into one
once we have step-wise expansion, why not include this fix in that patch
itself?

Thanks,
Amit

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

#231Amit Khandekar
amitdkhan.pg@gmail.com
In reply to: Ashutosh Bapat (#228)
Re: Partition-wise join for join between (declaratively) partitioned tables

Hi,

Rafia had done some testing on TPCH queries using Partition-wise join
patch along with Parallel Append patch.

There, we had observed that for query 4, even though the partition
wise joins are under a Parallel Append, the join are all non-partial.

Specifically, the partition-wise join has non-partial nested loop
joins when actually it was expected to have partial nested loop joins.
(The difference can be seen by the observation that the outer relation
of that join is scanned by non-parallel Bitmap Heap scan when it
should have used Parallel Bitmap Heap Scan).

Here is the detailed analysis , including where I think is the issue :

/messages/by-id/CAJ3gD9cZms1ND3p=NN=hDYDFt_SeKq1htMBhbj85bOmvJwY5fg@mail.gmail.com

All the TPCH results are posted in the same above mail thread.

Thanks
-Amit

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

#232Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#229)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Sep 13, 2017 at 12:39 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Tue, Sep 12, 2017 at 3:46 AM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

In this case, AcquireExecutorLocks will lock all the relations in
PlannedStmt.rtable, which must include all partitioned tables of all
partition trees involved in the query. Of those, it will lock the tables
whose RT indexes appear in PlannedStmt.nonleafResultRelations with
RowExclusiveLock mode. PlannedStmt.nonleafResultRelations is a global
list of all partitioned table RT indexes obtained by concatenating
partitioned_rels lists of all ModifyTable nodes involved in the query
(set_plan_refs does that). We need to distinguish nonleafResultRelations,
because we need to take the stronger lock on a given table before any
weaker one if it happens to appear in the query as a non-result relation
too, to avoid lock strength upgrade deadlock hazard.

Hmm. The problem with this theory in my view is that it doesn't
explain why InitPlan() and ExecOpenScanRelation() lock the relations
instead of just assuming that they are already locked either by
AcquireExecutorLocks or by planning. If ExecLockNonLeafAppendTables()
doesn't really need to take locks, then ExecOpenScanRelation() must
not need to do it either. We invented ExecLockNonLeafAppendTables()
on the occasion of removing the scans of those tables which would
previously have caused ExecOpenScanRelation() to be invoked, so as to
keep the locking behavior unchanged.

AcquireExecutorLocks() looks like an odd bit of code to me. The
executor itself locks result tables in InitPlan() and then everything
else during InitPlan() and all of the others later on while walking
the plan tree -- comments in InitPlan() say that this is to avoid a
lock upgrade hazard if a result rel is also a source rel. But
AcquireExecutorLocks() has no such provision; it just locks everything
in RTE order. In theory, that's a deadlock hazard of another kind, as
we just talked about in the context of EIBO. In fact, expanding in
bound order has made the situation worse: before, expansion order and
locking order were the same, so maybe having AcquireExecutorLocks()
work in RTE order coincidentally happened to give the same result as
the executor code itself as long as there are no result relations.
But this is certainly not true any more. I'm not sure it's worth
expending a lot of time on this -- it's evidently not a problem in
practice, or somebody probably would've complained before now.

But that having been said, I don't think we should assume that all the
locks taken from the executor are worthless because plancache.c will
always do the job for us. I don't know of a case where we execute a
saved plan without going through the plan cache, but that doesn't mean
that there isn't one or that there couldn't be one in the future.
It's not the job of these partitioning patches to whack around the way
we do locking in general -- they should preserve the existing behavior
as much as possible. If we want to get rid of the locking in the
executor altogether, that's a separate discussion where, I have a
feeling, there will prove to be better reasons for the way things are
than we are right now supposing.

I agree that it's not the job of these patches to change the locking
or even get rid of partitioned_rels. In order to continue returning
partitioned_rels in Append paths esp. in the case of queries involving
set operations and partitioned table e.g "select 1 from t1 union all
select 2 from t1;" in which t1 is multi-level partitioned table, we
need a fix in add_paths_to_append_rels(). The fix provided in [1]/messages/by-id/d2f1cdcb-ebb4-76c5-e471-79348ca5d7a7@lab.ntt.co.jp is
correct but we will need a longer explanation of why we have to
involve RTE_SUBQUERY with RELKIND_PARTITIONED_TABLE. The explanation
is complicated. If we get rid of partitioned_rels, we don't need to
fix that code in add_paths_to_append_rel().

I suggested that [2]/messages/by-id/CAFjFpRfJ3GRRmmOugaMA-q4i=se5P6yjZ_C6A6HDRDQQTGXy1A@mail.gmail.com -- Best Wishes, Ashutosh Bapat EnterpriseDB Corporation The Postgres Database Company
-- (excerpt from [2]/messages/by-id/CAFjFpRfJ3GRRmmOugaMA-q4i=se5P6yjZ_C6A6HDRDQQTGXy1A@mail.gmail.com -- Best Wishes, Ashutosh Bapat EnterpriseDB Corporation The Postgres Database Company)

Actually, the original problem that caused this discussion started
with an assertion failure in get_partitioned_child_rels() as
Assert(list_length(result) >= 1);

This assertion fails if result is NIL when an intermediate partitioned
table is passed. May be we should assert (result == NIL ||
list_length(result) == 1) and allow that function to be called even
for intermediate partitioned partitions for which the function will
return NIL. That will leave the code in add_paths_to_append_rel()
simple. Thoughts?
--

Amit Langote agrees with this. It kind of makes the assertion lame but
keeps the code sane. What do you think?

[1]: /messages/by-id/d2f1cdcb-ebb4-76c5-e471-79348ca5d7a7@lab.ntt.co.jp
[2]: /messages/by-id/CAFjFpRfJ3GRRmmOugaMA-q4i=se5P6yjZ_C6A6HDRDQQTGXy1A@mail.gmail.com -- Best Wishes, Ashutosh Bapat EnterpriseDB Corporation The Postgres Database Company
--
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

#233Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Amit Langote (#230)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Sep 13, 2017 at 11:29 AM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

On 2017/09/12 19:56, Ashutosh Bapat wrote:

I think the code here expects the original parent_rte and not the one
we set around line 1169.

This isn't a bug right now, since both the parent_rte s have same
content. But I am not sure if that will remain to be so. Here's patch
to fix the thinko.

Instead of the new bool is_parent_partitioned, why not move the code to
set partitioned_rels to the block where you're now setting
is_parent_partitioned.

Also, since we know this isn't a bug at the moment but will turn into one
once we have step-wise expansion, why not include this fix in that patch
itself?

It won't turn into a bug with step-wise expansion since every
parent_rte will have RELKIND_PARTITIONED_TABLE for a partitioned top
parent, which is used to extract the partitioned_rels. But I guess,
it's better to fix the thinko in step-wise expansion since parent_rte
itself changes.
--
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

#234Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Amit Khandekar (#231)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Sep 13, 2017 at 12:32 PM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:

Hi,

Rafia had done some testing on TPCH queries using Partition-wise join
patch along with Parallel Append patch.

There, we had observed that for query 4, even though the partition
wise joins are under a Parallel Append, the join are all non-partial.

Specifically, the partition-wise join has non-partial nested loop
joins when actually it was expected to have partial nested loop joins.
(The difference can be seen by the observation that the outer relation
of that join is scanned by non-parallel Bitmap Heap scan when it
should have used Parallel Bitmap Heap Scan).

Here is the detailed analysis , including where I think is the issue :

/messages/by-id/CAJ3gD9cZms1ND3p=NN=hDYDFt_SeKq1htMBhbj85bOmvJwY5fg@mail.gmail.com

All the TPCH results are posted in the same above mail thread.

Can you please check if the attached patch fixes the issue.

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

Attachments:

reparameterize_partial_nestloop_inner.patchtext/x-patch; charset=US-ASCII; name=reparameterize_partial_nestloop_inner.patchDownload
commit 203b3083318e9da41ad614a2ccec532025877c3b
Author: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date:   Tue Sep 12 17:41:54 2017 +0530

    Reparamterize partial nestloop paths.
    
    We do not create partial nested looop paths if the inner path's
    parameterization is not fully covered by the outer relation. For partition-wise
    join, the test fails since the inner path is parameterized by the parent of the
    outer relation. Fix the test to check the parent relids instead of the child
    relids and also reparameterize the inner path to be parameterized by the outer
    child similar to try_nestloop_path().
    
    TODO: squash this patch with the reparameterization patch.
    
    Ashutosh Bapat, per report from Rafia and analysis by Amit Khandekar

diff --git a/src/backend/optimizer/path/joinpath.c b/src/backend/optimizer/path/joinpath.c
index 91f0b1c..c8da19c 100644
--- a/src/backend/optimizer/path/joinpath.c
+++ b/src/backend/optimizer/path/joinpath.c
@@ -496,8 +496,20 @@ try_partial_nestloop_path(PlannerInfo *root,
 	if (inner_path->param_info != NULL)
 	{
 		Relids		inner_paramrels = inner_path->param_info->ppi_req_outer;
+		RelOptInfo *outerrel = outer_path->parent;
+		Relids		outerrelids;
 
-		if (!bms_is_subset(inner_paramrels, outer_path->parent->relids))
+		/*
+		 * Paths are parameterized by top-level parent(s). Any paths parameterized
+		 * by the child relations, are not added to the pathlist. Hence run
+		 * parameterization tests on the parent relids.
+		 */
+		if (outerrel->top_parent_relids)
+			outerrelids = outerrel->top_parent_relids;
+		else
+			outerrelids = outerrel->relids;
+
+		if (!bms_is_subset(inner_paramrels, outerrelids))
 			return;
 	}
 
@@ -510,6 +522,32 @@ try_partial_nestloop_path(PlannerInfo *root,
 	if (!add_partial_path_precheck(joinrel, workspace.total_cost, pathkeys))
 		return;
 
+	/*
+	 * Since result produced by a child is part of the result produced by
+	 * its topmost parent and has same properties, the parameters
+	 * representing that parent may be substituted by values from a child.
+	 * Hence expressions and hence paths using those expressions,
+	 * parameterized by a parent can be said to be parameterized by any of
+	 * its child.  For a join between child relations, if the inner path
+	 * is parameterized by the parent of the outer relation,  translate
+	 * the inner path to be parameterized by the outer child relation and
+	 * create a nestloop join path.  The translated path should have the
+	 * same costs as the original path, so cost check above should still
+	 * hold.
+	 */
+	if (PATH_PARAM_BY_PARENT(inner_path, outer_path->parent))
+	{
+		inner_path = reparameterize_path_by_child(root, inner_path,
+												  outer_path->parent);
+
+		/*
+		 * If we could not translate the path, we can't create nest loop
+		 * path.
+		 */
+		if (!inner_path)
+			return;
+	}
+
 	/* Might be good enough to be worth trying, so let's try it. */
 	add_partial_path(joinrel, (Path *)
 					 create_nestloop_path(root,
#235Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Ashutosh Bapat (#232)
Re: Partition-wise join for join between (declaratively) partitioned tables

On 2017/09/13 16:21, Ashutosh Bapat wrote:

On Wed, Sep 13, 2017 at 12:39 AM, Robert Haas <robertmhaas@gmail.com> wrote:

locks taken from the executor are worthless because plancache.c will
always do the job for us. I don't know of a case where we execute a
saved plan without going through the plan cache, but that doesn't mean
that there isn't one or that there couldn't be one in the future.
It's not the job of these partitioning patches to whack around the way
we do locking in general -- they should preserve the existing behavior
as much as possible. If we want to get rid of the locking in the
executor altogether, that's a separate discussion where, I have a
feeling, there will prove to be better reasons for the way things are
than we are right now supposing.

I agree that it's not the job of these patches to change the locking
or even get rid of partitioned_rels. In order to continue returning
partitioned_rels in Append paths esp. in the case of queries involving
set operations and partitioned table e.g "select 1 from t1 union all
select 2 from t1;" in which t1 is multi-level partitioned table, we
need a fix in add_paths_to_append_rels(). The fix provided in [1] is
correct but we will need a longer explanation of why we have to
involve RTE_SUBQUERY with RELKIND_PARTITIONED_TABLE. The explanation
is complicated. If we get rid of partitioned_rels, we don't need to
fix that code in add_paths_to_append_rel().

Yeah, let's get on with setting partitioned_rels in AppendPath correctly
in this patch. Ashutosh's suggested approach seems fine, although it
needlessly requires to scan root->pcinfo_list. But it shouldn't be longer
than the number of partitioned tables in the query, so maybe that's fine
too. At least, it doesn't require us to add code to
add_paths_to_append_rel() that can be pretty hard to wrap one's head around.

That said, we might someday need to look carefully at some things that
Robert mentioned carefully, especially around the order of locks taken by
AcquireExecutorLocks() in light of the EIBO patch getting committed.

Thanks,
Amit

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

#236Amit Khandekar
amitdkhan.pg@gmail.com
In reply to: Ashutosh Bapat (#234)
Re: Partition-wise join for join between (declaratively) partitioned tables

On 13 September 2017 at 13:05, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

On Wed, Sep 13, 2017 at 12:32 PM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:

Hi,

Rafia had done some testing on TPCH queries using Partition-wise join
patch along with Parallel Append patch.

There, we had observed that for query 4, even though the partition
wise joins are under a Parallel Append, the join are all non-partial.

Specifically, the partition-wise join has non-partial nested loop
joins when actually it was expected to have partial nested loop joins.
(The difference can be seen by the observation that the outer relation
of that join is scanned by non-parallel Bitmap Heap scan when it
should have used Parallel Bitmap Heap Scan).

Here is the detailed analysis , including where I think is the issue :

/messages/by-id/CAJ3gD9cZms1ND3p=NN=hDYDFt_SeKq1htMBhbj85bOmvJwY5fg@mail.gmail.com

All the TPCH results are posted in the same above mail thread.

Can you please check if the attached patch fixes the issue.

Thanks Ashutosh. Yes, it does fix the issue. Partial Nested Loop joins
are generated now. If I see any unexpected differences in the
estimated or actual costs, I will report that in the Parallel Append
thread. As far as Partition-wise join is concerned, this issue is
solved, because Partial nested loop join does get created.

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

--
Thanks,
-Amit Khandekar
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

#237Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Ashutosh Bapat (#232)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Sep 13, 2017 at 12:51 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

On Wed, Sep 13, 2017 at 12:39 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Tue, Sep 12, 2017 at 3:46 AM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

In this case, AcquireExecutorLocks will lock all the relations in
PlannedStmt.rtable, which must include all partitioned tables of all
partition trees involved in the query. Of those, it will lock the tables
whose RT indexes appear in PlannedStmt.nonleafResultRelations with
RowExclusiveLock mode. PlannedStmt.nonleafResultRelations is a global
list of all partitioned table RT indexes obtained by concatenating
partitioned_rels lists of all ModifyTable nodes involved in the query
(set_plan_refs does that). We need to distinguish nonleafResultRelations,
because we need to take the stronger lock on a given table before any
weaker one if it happens to appear in the query as a non-result relation
too, to avoid lock strength upgrade deadlock hazard.

Hmm. The problem with this theory in my view is that it doesn't
explain why InitPlan() and ExecOpenScanRelation() lock the relations
instead of just assuming that they are already locked either by
AcquireExecutorLocks or by planning. If ExecLockNonLeafAppendTables()
doesn't really need to take locks, then ExecOpenScanRelation() must
not need to do it either. We invented ExecLockNonLeafAppendTables()
on the occasion of removing the scans of those tables which would
previously have caused ExecOpenScanRelation() to be invoked, so as to
keep the locking behavior unchanged.

AcquireExecutorLocks() looks like an odd bit of code to me. The
executor itself locks result tables in InitPlan() and then everything
else during InitPlan() and all of the others later on while walking
the plan tree -- comments in InitPlan() say that this is to avoid a
lock upgrade hazard if a result rel is also a source rel. But
AcquireExecutorLocks() has no such provision; it just locks everything
in RTE order. In theory, that's a deadlock hazard of another kind, as
we just talked about in the context of EIBO. In fact, expanding in
bound order has made the situation worse: before, expansion order and
locking order were the same, so maybe having AcquireExecutorLocks()
work in RTE order coincidentally happened to give the same result as
the executor code itself as long as there are no result relations.
But this is certainly not true any more. I'm not sure it's worth
expending a lot of time on this -- it's evidently not a problem in
practice, or somebody probably would've complained before now.

But that having been said, I don't think we should assume that all the
locks taken from the executor are worthless because plancache.c will
always do the job for us. I don't know of a case where we execute a
saved plan without going through the plan cache, but that doesn't mean
that there isn't one or that there couldn't be one in the future.
It's not the job of these partitioning patches to whack around the way
we do locking in general -- they should preserve the existing behavior
as much as possible. If we want to get rid of the locking in the
executor altogether, that's a separate discussion where, I have a
feeling, there will prove to be better reasons for the way things are
than we are right now supposing.

I agree that it's not the job of these patches to change the locking
or even get rid of partitioned_rels. In order to continue returning
partitioned_rels in Append paths esp. in the case of queries involving
set operations and partitioned table e.g "select 1 from t1 union all
select 2 from t1;" in which t1 is multi-level partitioned table, we
need a fix in add_paths_to_append_rels(). The fix provided in [1] is
correct but we will need a longer explanation of why we have to
involve RTE_SUBQUERY with RELKIND_PARTITIONED_TABLE. The explanation
is complicated. If we get rid of partitioned_rels, we don't need to
fix that code in add_paths_to_append_rel().

I suggested that [2]
-- (excerpt from [2])

Actually, the original problem that caused this discussion started
with an assertion failure in get_partitioned_child_rels() as
Assert(list_length(result) >= 1);

This assertion fails if result is NIL when an intermediate partitioned
table is passed. May be we should assert (result == NIL ||
list_length(result) == 1) and allow that function to be called even
for intermediate partitioned partitions for which the function will
return NIL. That will leave the code in add_paths_to_append_rel()
simple. Thoughts?
--

Amit Langote agrees with this. It kind of makes the assertion lame but
keeps the code sane. What do you think?

I debugged what happens in case of query "select 1 from t1 union all
select 2 from t1;" with the current HEAD (without multi-level
expansion patch attached). It doesn't set partitioned_rels in Append
path that gets converted into Append plan. Remember t1 is a
multi-level partitioned table here with t1p1 as its immediate
partition and t1p1p1 as partition of t1p1. So, the
set_append_rel_pathlist() recurses once as shown in the following
stack trace.

#0 add_paths_to_append_rel (root=0x23e4308, rel=0x23fb768,
live_childrels=0x23ff5f0) at allpaths.c:1281
#1 0x000000000076e170 in set_append_rel_pathlist (root=0x23e4308,
rel=0x23fb768, rti=4, rte=0x23f3268) at allpaths.c:1262
#2 0x000000000076cf23 in set_rel_pathlist (root=0x23e4308,
rel=0x23fb768, rti=4, rte=0x23f3268) at allpaths.c:431
#3 0x000000000076e0f6 in set_append_rel_pathlist (root=0x23e4308,
rel=0x23fb478, rti=1, rte=0x2382070) at allpaths.c:1247
#4 0x000000000076cf23 in set_rel_pathlist (root=0x23e4308,
rel=0x23fb478, rti=1, rte=0x2382070) at allpaths.c:431
#5 0x000000000076cc22 in set_base_rel_pathlists (root=0x23e4308) at
allpaths.c:309

When add_paths_to_append_rel() (frame 0) is called for t1, it gets
partitioned_rels and stuffs it in append path/s it creates. But those
paths are flattened into the append paths created for the set
operations when add_paths_to_append_rels() is called from frame 3.
While flattening the append paths in accumulate_append_subpath() we do
not pull any partitioned_rels that are stuffed in those paths and thus
the final append path/s created does not have partitioned_rels in
there.

The same behaviour is retained by my v30 patchset [1]/messages/by-id/CAFjFpRfHkJW3G=_PnSUc6PbXJE48AWYwyRzaGqtfKzzoU4wXXw@mail.gmail.com -- Best Wishes, Ashutosh Bapat EnterpriseDB Corporation The Postgres Database Company. I think we
should go ahead by fixing add_paths_to_append_rel() as done in that
patchset. partitioned_rels needs to be removed from append paths
anyway, so that code will be removed when we do that.

[1]: /messages/by-id/CAFjFpRfHkJW3G=_PnSUc6PbXJE48AWYwyRzaGqtfKzzoU4wXXw@mail.gmail.com -- Best Wishes, Ashutosh Bapat EnterpriseDB Corporation The Postgres Database Company
--
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

#238Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#237)
2 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Sep 13, 2017 at 12:56 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

I debugged what happens in case of query "select 1 from t1 union all
select 2 from t1;" with the current HEAD (without multi-level
expansion patch attached). It doesn't set partitioned_rels in Append
path that gets converted into Append plan. Remember t1 is a
multi-level partitioned table here with t1p1 as its immediate
partition and t1p1p1 as partition of t1p1. So, the
set_append_rel_pathlist() recurses once as shown in the following
stack trace.

Nice debugging. I spent some time today looking at this and I think
it's a bug in v10, and specifically in add_paths_to_append_rel(),
which only sets partitioned_rels correctly when the appendrel is a
partitioned rel, and not when it's a subquery RTE with one or more
partitioned queries beneath it.

Attached are two patches either one of which will fix it. First, I
wrote mechanical-partrels-fix.patch, which just mechanically
propagates partitioned_rels lists from accumulated subpaths into the
list used to construct the parent (Merge)AppendPath. I wasn't entire
happy with that, because it ends up building multiple partitioned_rels
lists for the same RelOptInfo. That seems silly, but there's no
principled way to avoid it; avoiding it amounts to hoping that all the
paths for the same relation carry the same partitioned_rels list,
which is uncomfortable.

So then I wrote pcinfo-for-subquery.patch. That patch notices when an
RTE_SUBQUERY appendrel is processed and accumulates the
partitioned_rels of its immediate children; in case there can be
multiple nested levels of subqueries before we get down to the actual
partitioned rel, it also adds a PartitionedChildRelInfo for the
subquery RTE, so that there's no need to walk the whole tree to build
the partitioned_rels list at higher levels, just the immediate
children. I find this fix a lot more satisfying. It adds less code
and does no extra work in the common case.

Notice that the choice of fix we adopt has consequences for your
0001-Multi-level-partitioned-table-expansion.patch -- with
mechanical-partrels-fix.patch, that patch could either associated all
partitioned_rels with the top-parent or it could work level by level
and everything would get properly assembled later. But with
pcinfo-for-subquery.patch, we need everything associated with the
top-parent. That doesn't seem like a problem to me, but it's
something to note.

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

Attachments:

mechanical-partrels-fix.patchapplication/octet-stream; name=mechanical-partrels-fix.patchDownload
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 2d7e1d84d0..e020c0fe6f 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -100,7 +100,8 @@ static void generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
 static Path *get_cheapest_parameterized_child_path(PlannerInfo *root,
 									  RelOptInfo *rel,
 									  Relids required_outer);
-static List *accumulate_append_subpath(List *subpaths, Path *path);
+static List *accumulate_append_subpath(List *subpaths, Path *path,
+						  List **partitioned_rels);
 static void set_subquery_pathlist(PlannerInfo *root, RelOptInfo *rel,
 					  Index rti, RangeTblEntry *rte);
 static void set_function_pathlist(PlannerInfo *root, RelOptInfo *rel,
@@ -1285,15 +1286,29 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	List	   *all_child_pathkeys = NIL;
 	List	   *all_child_outers = NIL;
 	ListCell   *l;
-	List	   *partitioned_rels = NIL;
+	List	   *original_partitioned_rels = NIL;
+	List	   *nonpartial_partitioned_rels = NIL;
+	List	   *partial_partitioned_rels = NIL;
 	RangeTblEntry *rte;
 
 	rte = planner_rt_fetch(rel->relid, root);
 	if (rte->relkind == RELKIND_PARTITIONED_TABLE)
 	{
-		partitioned_rels = get_partitioned_child_rels(root, rel->relid);
+		original_partitioned_rels =
+			get_partitioned_child_rels(root, rel->relid);
 		/* The root partitioned table is included as a child rel */
-		Assert(list_length(partitioned_rels) >= 1);
+		Assert(list_length(original_partitioned_rels) >= 1);
+
+		/*
+		 * It's sort of annoying that we end up accumulating separate lists of
+		 * partitioned rels for the partial and non-partial cases, but the
+		 * partial list is taken from the partial paths and the non-partial
+		 * case from the non-partial paths, which could in theory be
+		 * different.  It's unclear why that would ever actually happen in
+		 * practice, though; maybe there's a better way to handle this.
+		 */
+		nonpartial_partitioned_rels = list_copy(original_partitioned_rels);
+		partial_partitioned_rels = list_copy(original_partitioned_rels);
 	}
 
 	/*
@@ -1313,14 +1328,16 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		 */
 		if (childrel->cheapest_total_path->param_info == NULL)
 			subpaths = accumulate_append_subpath(subpaths,
-												 childrel->cheapest_total_path);
+												 childrel->cheapest_total_path,
+												 &nonpartial_partitioned_rels);
 		else
 			subpaths_valid = false;
 
 		/* Same idea, but for a partial plan. */
 		if (childrel->partial_pathlist != NIL)
 			partial_subpaths = accumulate_append_subpath(partial_subpaths,
-														 linitial(childrel->partial_pathlist));
+														 linitial(childrel->partial_pathlist),
+														 &partial_partitioned_rels);
 		else
 			partial_subpaths_valid = false;
 
@@ -1396,7 +1413,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	 */
 	if (subpaths_valid)
 		add_path(rel, (Path *) create_append_path(rel, subpaths, NULL, 0,
-												  partitioned_rels));
+												  nonpartial_partitioned_rels));
 
 	/*
 	 * Consider an append of partial unordered, unparameterized partial paths.
@@ -1423,7 +1440,8 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 
 		/* Generate a partial append path. */
 		appendpath = create_append_path(rel, partial_subpaths, NULL,
-										parallel_workers, partitioned_rels);
+										parallel_workers,
+										partial_partitioned_rels);
 		add_partial_path(rel, (Path *) appendpath);
 	}
 
@@ -1434,7 +1452,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	if (subpaths_valid)
 		generate_mergeappend_paths(root, rel, live_childrels,
 								   all_child_pathkeys,
-								   partitioned_rels);
+								   nonpartial_partitioned_rels);
 
 	/*
 	 * Build Append paths for each parameterization seen among the child rels.
@@ -1453,6 +1471,9 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	{
 		Relids		required_outer = (Relids) lfirst(l);
 		ListCell   *lcr;
+		List	   *outer_partitioned_rels;
+
+		outer_partitioned_rels = list_copy(original_partitioned_rels);
 
 		/* Select the child paths for an Append with this parameterization */
 		subpaths = NIL;
@@ -1471,13 +1492,14 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 				subpaths_valid = false;
 				break;
 			}
-			subpaths = accumulate_append_subpath(subpaths, subpath);
+			subpaths = accumulate_append_subpath(subpaths, subpath,
+												 &outer_partitioned_rels);
 		}
 
 		if (subpaths_valid)
 			add_path(rel, (Path *)
 					 create_append_path(rel, subpaths, required_outer, 0,
-										partitioned_rels));
+										outer_partitioned_rels));
 	}
 }
 
@@ -1511,6 +1533,8 @@ generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
 						   List *partitioned_rels)
 {
 	ListCell   *lcp;
+	List	   *startup_partitioned_rels = list_copy(partitioned_rels);
+	List	   *total_partitioned_rels = list_copy(partitioned_rels);
 
 	foreach(lcp, all_child_pathkeys)
 	{
@@ -1562,25 +1586,29 @@ generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
 				startup_neq_total = true;
 
 			startup_subpaths =
-				accumulate_append_subpath(startup_subpaths, cheapest_startup);
+				accumulate_append_subpath(startup_subpaths, cheapest_startup,
+										  &startup_partitioned_rels);
 			total_subpaths =
-				accumulate_append_subpath(total_subpaths, cheapest_total);
+				accumulate_append_subpath(total_subpaths, cheapest_total,
+										  &total_partitioned_rels);
 		}
 
 		/* ... and build the MergeAppend paths */
-		add_path(rel, (Path *) create_merge_append_path(root,
-														rel,
-														startup_subpaths,
-														pathkeys,
-														NULL,
-														partitioned_rels));
+		add_path(rel, (Path *)
+				 create_merge_append_path(root,
+										  rel,
+										  startup_subpaths,
+										  pathkeys,
+										  NULL,
+										  startup_partitioned_rels));
 		if (startup_neq_total)
-			add_path(rel, (Path *) create_merge_append_path(root,
-															rel,
-															total_subpaths,
-															pathkeys,
-															NULL,
-															partitioned_rels));
+			add_path(rel,
+					 (Path *) create_merge_append_path(root,
+													   rel,
+													   total_subpaths,
+													   pathkeys,
+													   NULL,
+													   total_partitioned_rels));
 	}
 }
 
@@ -1671,15 +1699,21 @@ get_cheapest_parameterized_child_path(PlannerInfo *root, RelOptInfo *rel,
  * omitting a sort step, which seems fine: if the parent is to be an Append,
  * its result would be unsorted anyway, while if the parent is to be a
  * MergeAppend, there's no point in a separate sort on a child.
+ *
+ * Also accumulate the subpath's partitioned_rels into the provided list.
  */
 static List *
-accumulate_append_subpath(List *subpaths, Path *path)
+accumulate_append_subpath(List *subpaths, Path *path, List **partitioned_rels)
 {
 	if (IsA(path, AppendPath))
 	{
 		AppendPath *apath = (AppendPath *) path;
 
 		/* list_copy is important here to avoid sharing list substructure */
+		*partitioned_rels =
+			list_concat(*partitioned_rels, apath->partitioned_rels);
+
+		/* list_copy is important here to avoid sharing list substructure */
 		return list_concat(subpaths, list_copy(apath->subpaths));
 	}
 	else if (IsA(path, MergeAppendPath))
@@ -1687,6 +1721,10 @@ accumulate_append_subpath(List *subpaths, Path *path)
 		MergeAppendPath *mpath = (MergeAppendPath *) path;
 
 		/* list_copy is important here to avoid sharing list substructure */
+		*partitioned_rels =
+			list_concat(*partitioned_rels, mpath->partitioned_rels);
+
+		/* list_copy is important here to avoid sharing list substructure */
 		return list_concat(subpaths, list_copy(mpath->subpaths));
 	}
 	else
pcinfo-for-subquery.patchapplication/octet-stream; name=pcinfo-for-subquery.patchDownload
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 2d7e1d84d0..9c85302321 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -1287,13 +1287,29 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	ListCell   *l;
 	List	   *partitioned_rels = NIL;
 	RangeTblEntry *rte;
+	bool		build_partitioned_rels = false;
 
+	/*
+	 * A plain relation will alread have a PartitionedChildRelInfo if it is
+	 * partitioned.  For a subquery RTE, no PartitionedChildRelInfo exists
+	 * yet; we must build one if it is required.
+	 */
 	rte = planner_rt_fetch(rel->relid, root);
-	if (rte->relkind == RELKIND_PARTITIONED_TABLE)
+	switch (rte->rtekind)
 	{
-		partitioned_rels = get_partitioned_child_rels(root, rel->relid);
-		/* The root partitioned table is included as a child rel */
-		Assert(list_length(partitioned_rels) >= 1);
+		case RTE_RELATION:
+			if (rte->relkind == RELKIND_PARTITIONED_TABLE)
+			{
+				partitioned_rels =
+					get_partitioned_child_rels(root, rel->relid);
+				Assert(list_length(partitioned_rels) >= 1);
+			}
+			break;
+		case RTE_SUBQUERY:
+			build_partitioned_rels = true;
+			break;
+		default:
+			elog(ERROR, "unexpcted rtekind: %d", (int) rte->rtekind);
 	}
 
 	/*
@@ -1307,6 +1323,19 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		ListCell   *lcp;
 
 		/*
+		 * If we need to build partitioned_rels, accumulate the partitioned
+		 * rels for this child.
+		 */
+		if (build_partitioned_rels)
+		{
+			List	   *cprels;
+
+			cprels = get_partitioned_child_rels(root, childrel->relid);
+			partitioned_rels = list_concat(partitioned_rels,
+										   list_copy(cprels));
+		}
+
+		/*
 		 * If child has an unparameterized cheapest-total path, add that to
 		 * the unparameterized Append path we are constructing for the parent.
 		 * If not, there's no workable unparameterized path.
@@ -1390,6 +1419,21 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	}
 
 	/*
+	 * If our parent is also an appendrel, it will need to build an
+	 * appropriate partitioned_rels list in turn, and that requires that we
+	 * have a PartitionedChildRelInfo.
+	 */
+	if (build_partitioned_rels && partitioned_rels != NIL)
+	{
+		PartitionedChildRelInfo *pcinfo;
+
+		pcinfo = makeNode(PartitionedChildRelInfo);
+		pcinfo->parent_relid = rel->relid;
+		pcinfo->child_rels = partitioned_rels;
+		root->pcinfo_list = lappend(root->pcinfo_list, pcinfo);
+	}
+
+	/*
 	 * If we found unparameterized paths for all children, build an unordered,
 	 * unparameterized Append path for the rel.  (Note: this is correct even
 	 * if we have zero or one live subpath due to constraint exclusion.)
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 6b79b3ad99..907622eadb 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -6076,7 +6076,8 @@ plan_cluster_use_sort(Oid tableOid, Oid indexOid)
  *		Returns a list of the RT indexes of the partitioned child relations
  *		with rti as the root parent RT index.
  *
- * Note: Only call this function on RTEs known to be partitioned tables.
+ * Note: This function might get called even for range table entries that
+ * are not partitioned tables; in such a case, it will simply return NIL.
  */
 List *
 get_partitioned_child_rels(PlannerInfo *root, Index rti)
@@ -6095,8 +6096,5 @@ get_partitioned_child_rels(PlannerInfo *root, Index rti)
 		}
 	}
 
-	/* The root partitioned table is included as a child rel */
-	Assert(list_length(result) >= 1);
-
 	return result;
 }
#239Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Robert Haas (#238)
Re: Partition-wise join for join between (declaratively) partitioned tables

On 2017/09/14 7:43, Robert Haas wrote:

On Wed, Sep 13, 2017 at 12:56 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

I debugged what happens in case of query "select 1 from t1 union all
select 2 from t1;" with the current HEAD (without multi-level
expansion patch attached). It doesn't set partitioned_rels in Append
path that gets converted into Append plan. Remember t1 is a
multi-level partitioned table here with t1p1 as its immediate
partition and t1p1p1 as partition of t1p1. So, the
set_append_rel_pathlist() recurses once as shown in the following
stack trace.

Nice debugging.

+1.

I spent some time today looking at this and I think
it's a bug in v10, and specifically in add_paths_to_append_rel(),
which only sets partitioned_rels correctly when the appendrel is a
partitioned rel, and not when it's a subquery RTE with one or more
partitioned queries beneath it.

Attached are two patches either one of which will fix it. First, I
wrote mechanical-partrels-fix.patch, which just mechanically
propagates partitioned_rels lists from accumulated subpaths into the
list used to construct the parent (Merge)AppendPath. I wasn't entire
happy with that, because it ends up building multiple partitioned_rels
lists for the same RelOptInfo. That seems silly, but there's no
principled way to avoid it; avoiding it amounts to hoping that all the
paths for the same relation carry the same partitioned_rels list,
which is uncomfortable.

So then I wrote pcinfo-for-subquery.patch. That patch notices when an
RTE_SUBQUERY appendrel is processed and accumulates the
partitioned_rels of its immediate children; in case there can be
multiple nested levels of subqueries before we get down to the actual
partitioned rel, it also adds a PartitionedChildRelInfo for the
subquery RTE, so that there's no need to walk the whole tree to build
the partitioned_rels list at higher levels, just the immediate
children. I find this fix a lot more satisfying. It adds less code
and does no extra work in the common case.

I very much like pcinfo-for-subquery.patch, although I'm not sure if we
need to create PartitionedChildRelInfo for the sub-query parent RTE as the
patch teaches add_paths_to_append_rel() to do. ISTM, nested UNION ALL
subqueries are flattened way before we get to add_paths_to_append_rel();
if it could not be flattened, there wouldn't be a call to
add_paths_to_append_rel() in the first place, because no AppendRelInfos
would be generated. See what happens when is_simple_union_all_recurse()
returns false to flatten_simple_union_all() -- no AppendRelInfos will be
generated and added to root->append_rel_list in that case.

IOW, there won't be nested AppendRelInfos for nested UNION ALL sub-queries
like we're setting out to build for multi-level partitioned tables.

So, as things stand today, there can at most be one recursive call of
add_path_to_append_rel() for a sub-query parent RTE, that is, if its child
sub-queries contain partitioned tables, but not more. The other patch
(multi-level expansion of partitioned tables) will change that, but even
then we won't need sub-query's own PartitioendChildRelInfo.

Notice that the choice of fix we adopt has consequences for your
0001-Multi-level-partitioned-table-expansion.patch -- with
mechanical-partrels-fix.patch, that patch could either associated all
partitioned_rels with the top-parent or it could work level by level
and everything would get properly assembled later. But with
pcinfo-for-subquery.patch, we need everything associated with the
top-parent. That doesn't seem like a problem to me, but it's
something to note.

I think it's fine.

With 0001-Multi-level-partitioned-table-expansion.patch,
get_partitioned_child_rels() will get called even for non-root partitioned
tables, for which it won't find a valid pcinfo. I think that patch must
also change its callers to stop Asserting that a valid pcinfo is returned.

Spotted a typo in pcinfo-for-subquery.patch:

+ * A plain relation will alread have

Thanks,
Amit

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

#240Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#238)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Sep 14, 2017 at 4:13 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Wed, Sep 13, 2017 at 12:56 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

I debugged what happens in case of query "select 1 from t1 union all
select 2 from t1;" with the current HEAD (without multi-level
expansion patch attached). It doesn't set partitioned_rels in Append
path that gets converted into Append plan. Remember t1 is a
multi-level partitioned table here with t1p1 as its immediate
partition and t1p1p1 as partition of t1p1. So, the
set_append_rel_pathlist() recurses once as shown in the following
stack trace.

Nice debugging. I spent some time today looking at this and I think
it's a bug in v10, and specifically in add_paths_to_append_rel(),
which only sets partitioned_rels correctly when the appendrel is a
partitioned rel, and not when it's a subquery RTE with one or more
partitioned queries beneath it.

Attached are two patches either one of which will fix it. First, I
wrote mechanical-partrels-fix.patch, which just mechanically
propagates partitioned_rels lists from accumulated subpaths into the
list used to construct the parent (Merge)AppendPath. I wasn't entire
happy with that, because it ends up building multiple partitioned_rels
lists for the same RelOptInfo. That seems silly, but there's no
principled way to avoid it; avoiding it amounts to hoping that all the
paths for the same relation carry the same partitioned_rels list,
which is uncomfortable.

So then I wrote pcinfo-for-subquery.patch. That patch notices when an
RTE_SUBQUERY appendrel is processed and accumulates the
partitioned_rels of its immediate children; in case there can be
multiple nested levels of subqueries before we get down to the actual
partitioned rel, it also adds a PartitionedChildRelInfo for the
subquery RTE, so that there's no need to walk the whole tree to build
the partitioned_rels list at higher levels, just the immediate
children. I find this fix a lot more satisfying. It adds less code
and does no extra work in the common case.

Thanks a lot for the patch. I have included pcinfo-for-subquery.patch
in my patchset as the first patch with typo corrections suggested by
Amit Langote.

Notice that the choice of fix we adopt has consequences for your
0001-Multi-level-partitioned-table-expansion.patch -- with
mechanical-partrels-fix.patch, that patch could either associated all
partitioned_rels with the top-parent or it could work level by level
and everything would get properly assembled later. But with
pcinfo-for-subquery.patch, we need everything associated with the
top-parent. That doesn't seem like a problem to me, but it's
something to note.

I have few changes to multi-level expansion patch as per discussion in
earlier mails
1. expand_single_inheritance_child() gets the top parent's PlanRowMark
from which it builds the child's PlanRowMark and also update
allMarkTypes of the top parent's PlanRowMark. The chlid's PlanRowMark
contains the RTI of the top parent, which is pulled from the top
parent's PlanRowMark. This is to keep the old behaviour intact.

2. Updated expand_single_inheritance_child's prologue to explain
various output arguments, per suggestion from Amit Langote. Also
included comments about the way we construct child PlanRowMark. Please
see if the comments look good.

3. As suggested by Amit Langote, with multi-level partitioned table
expansion, intermediate partitioned tables won't have pcinfo
associated them. So, that patch removes the assertion
Assert(list_length(partitioned_rels) >= 1) in
add_paths_to_append_rels(). I didn't remove that assertion from your
patch so that you could cherry-pick that commit to v10 where that
assertion holds true.

4. Fixed inheritance_planner() to use top parent's RTE to pull
partitioned_rels per discussion with Amit few mails back [1]/messages/by-id/CAFjFpRe62H0rTb4Rb7wOVSR25xfNW+mt1Ncp-OtzGaEtZBTLwA@mail.gmail.com.

Please let me know if I have missed anything; it's been some long discussion.

Apart from this I have included fix to reparameterize parallel nested
loop paths as per discussion in [2]/messages/by-id/CAJ3gD9ctVgv6r0-7B6js7Z5uPHXx+KA5jK-3=uFsGwKOXfTddg@mail.gmail.com.

Please note that I have removed the advanced partitioning patches from
the attached patchset since those need a rebase because of default
partition support.

[1]: /messages/by-id/CAFjFpRe62H0rTb4Rb7wOVSR25xfNW+mt1Ncp-OtzGaEtZBTLwA@mail.gmail.com
[2]: /messages/by-id/CAJ3gD9ctVgv6r0-7B6js7Z5uPHXx+KA5jK-3=uFsGwKOXfTddg@mail.gmail.com

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

Attachments:

pg_dp_join_patches_v31.tar.gzapplication/x-gzip; name=pg_dp_join_patches_v31.tar.gzDownload
��p�Y�\{s�F������U%�R���"�J�[�v$�R{�kE� ��i]��~�� (S��T]��v-�������{�Y�O����8��+�y�}��~��5����;������;pG������_����9'5�u��D�/d�Xgq��5�s����V�������j'Yg%�,��8�l�Tup`g'�\>O��IT�(l��
����?&e���������?��o�x)��=�F��d���:�(��y
����\�&����q$n�J���=���>h��d��s-#�R�)�����3����2���#������k����n��?��{��������;��h�k��g�(/{.~zw~��{��G����&*�&*�D�� ��6E�B������u�C����*TKxJ�E�(����!�
�6��E^�A�)�-��A�[C��ADq&f
�&`,��o�5�#M�)[�A
k�*�[����M���U��&������AW�"MZ�'���%�8�(:����dd�Z�JD���B2���]q��&96���C��|[�%�c\q
�0��}�N������5�KkQ45��%���6�b&����H4g]&��7K�nT�t7Jlx�U�������u�����*���/��,��x��W$e)VI<�����Wk �(��&7A� u������|X!V(t�'P�,�"�l��B5��L���N��i����AE�Q���%Qr��Jh��'~�����4�fB,��W'5��<J]F��w0rQ�x�}���W��/:�������2!��c}���f���P�'=w���ON�O����C���+6a�{���7�3�ON��h�����=��s���3�����J.�y,�8��Q��U��Q�AS4�!�BN�m�BVB�'O���u��-Z)�4����%:T�1������#)N:jv�\���\\M�?���_]M����M���}w�BN�msyK�.�.���x=�)��h4�����)�����n-_�{	6�	���T����B���!������-7�]�n_����Q���I���;<v�j ����6)��gM��&���j����j��>oI��7}g[#�g0����g8s��5`cv����������Q;�9��yMj���I���Es�T�R3���$9�@/����3���&��"D�A��$�D�W�9��8y��y�@O�4��$p�3����22�z�Lp�� ����!0�!�����`G��pv�����v���h�
`-`s�iAcf���&���f�w���`�<[�+0U�������mP��:3%{��������Y���	����w�)5a��G��S'>�P�����YZ���D�����N[D�`���X�'�"�N=������\a�k0���UH�^�B��J�O�!�������0q�=��.���!w
�o���1�7�SC��2
>�8�SQ|���v^�[���v)4��)mE���8?�?��>@^���O�I����O�Q~tp��y���,��Pc��C�O^h�Jp��h��I��;j6���_m���w��atB�Q��*�"u��PN���/������������������E���;�����
{������=5�L\o4R��� ���@�����2` �s�����|4�Q���T�[&Z&
�Z�������PI�It�"6�!�#Q���DR.��r	)�x�`2w}q� 	|$z�Z�XE���!x}9R�n���eH�%�.����vE��jM���#���=R��EO,���E���D� Q��f�WH�z���NNp&��Z�n1��*�^��i�Z �i����A �\#�Dy�$
@��e��-��x����
T�9t�7'���	q�4�!<f��f =O�iW�0��r$i�$Str�HMD�H���8��7[p"Bq$	�Qj�$��&����x][kh%5�)��$�L����/����&,��"��xe�:`(	tB)@�.,0n��5����
�"��v&����x�7P�@�_���@5��7�"�2(�
`�R4��[~j;u�;��
����PX��!K1��S��U�%f���h��RA
u(��&�u�0�����i�Vf��7Q�p�v��v���:�&�Z	�0Yg�����i^U��x��w�Cy��}���������e��0[tE4� �c3����7�U��.���
B��_�
���kt{P���I��3����*��"���vST$�T�I`��}*b����Rc^)�������qZ0E���;���c$7Z����'���g���&��(�@���t�����)z��M9�	�!�sj�V��:�}�j��`��gw�:9=@�X�E���(/�xQS�8�o���C�I� xCLT���b7����o��)����z�Q�t����0_�[�ae�Z����y��8�v7Cw��9��������s���;\7������6������m�o��$��#(4�Iw��# �z������h�o�*����!��'}��+�� 1������>�����}5o��]^_����������q�v����h���R����5��5P7*la��Lu^��@Y��L@����7�K=����W<�7���7|m5Y8���:D���F)A�T�~v:������������;�O^;�����@��4����sb*	�<U���`�HfY����+X���qb���g��%<�(E-�v��lS�>��*[����U�S�xlG�^^���(Z�y������qOD��w�[���{��,�d������9���@�)s�D�R�+�N�M�Gn����z{�����f���]8�gw�R����	=����3�l�9���j���k@��n(\��'�����'� ��w�3���4��sx�IN@�GFQ��|����L�G����Rq��1��[?'�xb��
u�����{�M��whH�-�`���-�r��l�J�@�8���-5k�pZ�����?����o�$<�f�h�N�J|U��3U��wr<��=��J�����{��*�Gs_u0y���[M~�8�MP���������5���t��e���\���<��!na�?q\W�����?���'9S]�p������6O
W�;(�=��kr����nSPd�I"(����jo���Yd�g��~��k�1��,��:�g���g��,����K��7�����=3.�!�|F^	h`�����R_�:���^AY�������'�q�
�2�Z�1e!�Z�LtUnhJX�}��gU2�c �D���������}B��.���W�p�-6(x��'XB=&�p�J
_��Lq���o��w�=�MA!��q�HWN��T�v��*�X�		vq��v�h��a��f� ���AB��=����h�yG
1M)_
vek��t fN��4��P�I��~����0�S�������}�����%�����_�u�<��aZI��F�Y)�<{,�0�,�6��G|��L��\��&?�$d��s�Z�
/���Y�(h���0�z�q����������� ���_���c���"���c����Zu!��"��Wp���g�	S1^�7w��3A�Q������N�hx��{N��A��+<��s�i*�X���������qy��
uX���j����2���C��+��%.���J�,��D�5���f����B	��0�a%|V���3?TC�$3	��-�)V����{�������������=H/\=�@����%��;
����_O/�@���Z�d����]�?p���q��H�z����~��8���Gg���3-:�0 �+7���'��R�H����T}����H��@��:���-�#�x�����x��R/	f���u�����@�����d���84�?#��)�2*�*�T�;�YL�A �!?�}���Rb�� ��[cO��a�}��W�x�8�/L��)LJf���s.J�V?y�.w�"O(_��@i�Ssb�=n�;J�5�������5pgYx�T���Mjy��z�`�"�.-P7���������&xX;�eP��-�,�Q�����*�X%��)X�[���,Xm�]ps�e��vHCFU��#�xuI&�o�8I�x����4A��p�\�{%*K�Xh�1�we��S�\/E�&�G�t��
!5gZPx���������(��q(�����xl���9"�]qh�H_(J/[1�)����u��k��[�,��Q�t����kU���>���{�/,��y��A��q�����g�(N���<7���^� ���T�2F���r�F'���qG�x���>)�4�A9A�@v��E��
��;_�t�\?D%��"�����#��I�!���V��J(���3L�dqr���N�I�P-�R�������,
o���@��x���1:�>��l<������d�)�p\�h|"LM.R�2���toO�?� d=��M�L���V ?�U�=����O-��Z�r�k�hAl��o%���z���.�����U���&����y��1�9�^b��<ji�Aj) �{)�6a����x����i�����8����k�1(��w���`���.}Y��� *�?s��O>6�;j�=&�}��rh���JI��t�mwj�AjMc8$�
����|3Aw����m}N���+]!Z�A��8wL�[D�
��"�Q��:�Km�[�����w&�x�5�����<Cx,�x����9[.�M�@-�0�v�s�T�~�V<��[�����ev����� �uP��;�;	a{��:9<��L�EY�E��zFw>5u[�d@e{�7���I���w�}^�wL�B��	���
��TM`�)��q�#���i���X:���9�x��n����X���R
������q7��������^��~��e���y\	�`�E~�c��Oy�l��k��U�����zS����vu\I0;zY.J�Xj;�!E�)V}��(G�E����X,��N,��������8��f�]'N)���V�-���>K`����I��>�������O�����b<�5��	���R��oU��B����1sB%�
|��Xh���V�w�ts;��L4��L	���M�����(����������1%��X�G���s�99W5���%��RCO�V�1l0������L���|�����vR�#M�����d\��G�%�
�2L��a�
A#M���AU:[[���0i���7��e2a,�1�!�=`!X�>�����9���	X@W��+�l������(�e0>v���;t�����E���v�8zG�0����������ol<�
Sm�t��L����4;�����x������Y��`JXP^}{����/>�%Ec�	z�'��k���&B�|����
���;��{|T���<�����7���Q��yl=���G���������u�6�V@v��uw\��@�b%%�$g�<���c&����Q��y?RO��7�|�u�9'�9����*����mlO�z����o__P�w�]a��+�CHS�B��/
�HK��Q����X�k�B~k�iWW����u
�����h�#Z���Jk4���Fc���9:��S����=�Qa���V���2*2:�9��rkK��i)MWUM��(h�w����n+g(&v�?�P8N�8R����8<�(�(vtn��?���1���|�R��l�����&�
7t��I]a��%�{��y��@���z+6��������#N�
V:����6����@���$�o�����na��\���kf]J�x�U�����4����3�z>�N&g.��L���o�i�lo����uQ������w5��\(����t��#��u���������Xx�r:_�q�a�f���~{�:E��g���j��L����j�=����tJ�������Q�������mI�8��Z��}vLE�L�.�8{G�x��slefg�7���,N(R����d���U�wt�e�!���F_���^�u�9�

�����N��RpE����1p���~w��>����>����'Q�L2Rj��M���������?��u`iE1�J. 'H�t����S,����q�w�.�J�6���=]�=ML^�Y��'KN'_��]��!0�<�/.3@LP�U y�����3���@N��������2�^q��B�����b�v�VC�8���X;�<0�,�����������S8��.B�Z��Yn�'9����)�L"����N��)J/����5�~��TS�Q�:Z�4���LlXixe[0�N�FY��E��y�4�65��a�����D��0���]���5����_*�zFz*%?�����#V��hU��e��6��b����IW
�J�2f��5��>�d=b}6��a[����?�1���? B��@%g��?s �������K�<1����O�����O	�;�w��~�r�
L��'���S%o�,r|�m�tB����&Z��5�����`�K=>t�]M���:����a�����F����=U�0f�~�#;�f���~CC�����v����b!�f	}-��h�gS���v�;C�]
�))_e+�������j����"�S�p����)���}�/����HT��Uvs+L��9����R��rl��m'n�b����A�	�?����4	8���\ ��8�2�v������v����]�!ehTD5i6�R�x�.n���]n�Z��:C��Y�������$b�kF�@D���`aM����"dt5M�	��p�	$��$L�X<b���=Vu��}��;PAS�;L��W=H�G(�U�u��2�U�1�?@����ST_��sS����r�t���<^����l=^����=^���7�I��2��w�?E1�=��l��t5�!�T�J��w��8�K6�-j�~���~Ezh�>9$TG����7z��h?yIB&�����?��6�{�F�2�0*)~��������� m_@=��g[j��f���^�[.'��>�����EO&�����]+KrD������$v�3@���=���s��S"����0�H����j
�j3a�Y�E<G6�l[��J����kgCD���	���[{x�G�O��@���gG��frG8�d��$��x����
d�,���>�P���:�R�YE���~�
`>�RT�B�������!�I<��3,��;8��wC����a��Mz��
�����������`D�������� gO����_��Sp)�u�����������Z3��C��]�Q�K�$�,�$�g���L&9�b���!(y�@���,�E�����.y({\����F��X:`�L�*��Ej�-���T\A�_��a�%b�H4�I�G���I/�<�JN"��gC��2�/0#�E�P�$�����d���������`��e����\K��<MO���r�?$S����M�V�^���B<�2SY���>���f� m�����d�d�!�o��=�s�K����GV��+X��i�:}
����}E���0���X�9N��M�#�02LS���e]�m:��c���li^�l�L��^�+hD�4����E?R����e���C���X`@��O�\�U����z�^��vh�����m(h�&3��}������B0�>F<\���`%��b�-o�!�lj����}�BI�# :���k�I������<I3o" ��PJ��*��$s����]
\-:(x�����uB
���R�t��C���E��q���RL���*]^#a�h5	���W)�U~u�z�����M����[a�_{I��O�Y.�]��P78|��W��W�|��~e�_�s����Q7�(�9�l^D_�'�WxF�N��:W�������5
��_)���d�2�����[����<|[�rF�s3!�WdAA+"��H����Q��w�A=8RvP�J[�!���j1!�),�K
��r�d����I����F}�&�Ia����o����y.�2G��DO}`	4sr
��L�Y4���@�YN��N��g!a|B=�|��zFb�	Em�1[�Pl����+zp��i�N����we�*h��I[-��b���J�R�����Q>���d��z6!�%��z2�p{=�;k73��(K���T�J������@�����#:�U�E^�Dv#�5&$)���]q�����1���b�Q�����G�Y��Q�wx�'�0F�u���7Z�)�{
�]��P��?zI�R�_A�2c�H�C��7���
i������'��G���y��J����2 �4�n��\���<�������1�#1F-N6���KvL?�S�=��B��zq���2?�!'��3p,a��B�f���~���w��g��~����QA��i)94�	�������	/8p�[�}��-��A/�����PPc<}�U�>��YJ��*������x�7�dT!��%7W�7��/m0��
����k���b�G����8�g��9���L}�9V�j}���i��G�S�kx�\$Q��F��Y�C:Y�8�N�����9R�fw���F��z��_)<`$�����A�<����s�LGq�{�ix�=�^�";9���G��S��Q�;��9����X��tP�L2{h��rp����	P"�RY��i7�gCr`���9wS�-^bo�E�9=o|�Q�k���&'������r���:��{+P.x�"�'�����?  ��d�������t����3	��_�rZD�wS����}?�b���%��r<����+�JE���}u ��l5��]�,I���]$���o���<�FKv|7E��	
�;_X?]�q���=��y8�L9��t&R�`m�5/�?�;���hi���(��_��U��fx-�E(d)�P����Dx'��:�I �c��(���p"���.3��e�~�����@C��I@+�!	����2+=�V�'HE���\�g4�4�,�+�3[���q���JO�����j�
.����&�F��Rhd
7���;������cVF�~�x��������g�������QO�
�h'�$p,�$�r1�U��l�����������d����<$��OA��	O�^tRXSF�A��C��bW3�	;��N
���&�ir{�yFK�O���5�F_����;{<*o#	o#	�q������a�0��Q�jGX-%^�]k�������f��k�%��U�&���U�=���_���3�u�.I	�!�~�.��TD���Yd7Z.�v�H�a��@�/R�t���I�G8�E��u�����"��1>�A?F��~���/s������G����_{�����&��a5�ym�1���
�x���p�I���W�e:�z�v�4����@Jr%�)��)����/��=��N���O���st�P/a�[K!4�|�������X:sU�$@*B�����}53��w����3EVSv���p�p%���>D/��/^�E���������y�Q�]�#�l.V�)�P^�����-������������1W���w�E1(��W��_�E����@��R�|��]��3Fa�l6�0Y�Kv��9}�>6�b���8������[JHF���h��i�mn�c`�[�����1��8�Sf<��\dU�ZG!���h��No�I��������c�Pu��p������Z�	���"p����F�?����g��hv�&�zF�tJ�A�?��m�����9H�!�h��v���?u���~O�����jE�w��$������� ������c����Y�\uX��4>����^�����S�^LMa�	�����b���g7%����@�K���]\�0
M����W�]�9�B/����b��x�^��W��<w^���=��
[;x�_���f�G��D�+��^}9g��KX`��qVo�.��m����"{=�N��y�ZI��-������tymw�����e�+���|v�8�Hd���E�I<P!��������r�IY=�1�v
�\��,P8
��Vq��������)��0��������}~���N^�kv>d��}E�}��mD+��c*IDGH�F���`����q���{O�k|i;5���[1�9�Hcao&\�s�-@�����,�!g)<�\"m�h�w��#�Wa�Mi��N���1#W�G���A	�jW�!V�bd?�^�k$�S��WA�Z�p��@ ��lE"�V$��<i"6�� �HO�As�=�������hxpp<���I�o�P�LeR�&�J�#����3�^���p���$������`��o��-[�.�����P�1G;����������l��}wqyK�]o��)[>��Sa��i���}T�g�����x~��a���Q��/���|=��p��������A�/��?<������Kz����D�������)�������|��>]e�?��9l��Q����O��Y�?G����U�?����8_�t�����o����z�������T�8:�s}�2P��$��`��O6c~��,����x~���w�G�`J|=�F�2Vu��q����Q�a��y�F�����<=�}�;|���#"���!������ ���vv� 58�����H�A��[n*��[9��A�W��!2����5���B�Ha�(��t� �i6�%��\}=�����e��*\f7���!�h�_�X)*r��lo+���d�������F$K�J�YG��(w��]`X�eg�F�"��I�����%����A2g��"�������xu�j9��+0��:PcK�
O?��nn�``N9�V _����N�6T8�<����D7VSn�fP]1TnU�Q,�\eB�B~��9�Ly���?1��F$��.�j|tA:�/OV,�PRD*�������z���8WB�� 
�'T�$�12����x
>��rr��5�Wn�������A�#�*�X[�\MP�A1�����s����#2H8�]�C$��a���`y�d��������2�SA5
?]0XsUs}����;Z��S�ZS�A�(�$�	��.�sH���G�5#/8�	���o)��k���^p�C�k�l�O&(�Nd����=2�h���
��� ��a�LWy��(xG�}�e�����������K;��A�^��)_���8���s�'�=���L���8�z9�s�*?�>���B-�3���\R�E8���aY���a��
T�*�q��Hcy��J}�z�V�*<�vF����".J�����FX�<�E_�j@$&�`Z[��^�n �"��~M�t�fc	&�W����0]������������A7�d������A]��.��L�C�:�e�6:��� �{����fd�s�r:\R�a���`�lG�r���?��1I�hH����B��v�����!%�o1�����C;�p@������������{�S�;?R0���0�w*b�!��L���vhA/p�����7���=����w���?����l� M��3��&�T�8E��s?cuH�@Vh���N6'�
x0�����09�Je`�,@���b��D{�2t���!���h�P8�
RI���LS|-���;�+y��Q�X�����\~�.)��� ��ja��P�����)�z��Z�`<�s���j��Lab{�}�V�ld�S$��t:HG���P����c)�R��9a��|�]&Ds�
����rd�Q�����@��W�����;o�'��.�)�8����LG�����3���BP6*X���[,��Z��E�W��������m���St��;w�iGB\���D�x�I�M &��aOc�~Q2�E�Q��Ec�H����P��t�@l��������-�#�_���`E�?��=���\|�10~D�8a��������(f#O*���n4�E$Xh��1��T���e����MTe����,�]d��t����XVM������9�`n��GdU/��c��u:��
��@K�����W1�s'W�Ti|��/���;�*��������k!��������5�;Je�1�!*@���t�	��X$hF}6�i�O
V��2;g�k �����TJ_L�D|�������U~9Y.�w��R��&�R<c����Lut&9Z=�g]����
+�[��I�p6K$����&�b�g��C�
�F8���n���e_}������a����z��
p*K.&����b4YA$�����2R>4a������x�^P�Y�P�����4�!����|����4�dRf>��u��5#����M�y�7�s�n��Ol�oc�`5l����Sv�����(����/��A��2�7�h��S�N.e��&��h%�?����A&�!�x�������i���,��0	�3���Z����:����A(#�,��U@9d�+	E^d��	F�=q�����1A�L~���w��&�;g�*�t��"��o~{�����aF�R��������xiv��v�k���3���a~�0���,l�?�4�(�����h�^[l�5��
�o������[�`��=
m�
Ovh�TPDb���E^�'�3���n?����c�����'�@`_��9x�
9����)��3��a�� b9G�{	W�$'����x������I�����d���1Q<����������*T�
V�,a��k�LV�������l��N_�S�b5�:���:�ih��Rq����^�!~�x5�D�6����L�G�w�H���cTT9j������
mDT�S���5��������������)�m�Z�{B�[��&���B���>�u�C"�����o�8T�������~�x������/_�=�Q5x����m���R6�DK/=\|���&c,|8��G��}�uh�vU�2�xrt��=v�i��
��d��\z��m����B���1v 
��Cl>�����o����NI�]��[��\	�p�!b�2R���5_f7�QdWx�/�����8@Pv?�q8*������G ��"��~}��f�K��Fj�l�:t�Yp�PS�7^
��
�L��ff�**�����!�F�.�
��������-/�k�(���3��av���4�CZ�a�������,�rY��J���)[�k��b�/�>������^���q�b��c��49���CO���|I�f�I�.�|���@��Zd/�c67s���+�x\�"~S�7���������k�����"BZ���qZ�X�X�}OBboa�a�����	��"� :��|�T&��%WYgA�<�5Z}9����\�
�����R�\EK���t�+P���X��b���~�1
R���q�%��:l�?��!�;`r���H�$;H!�������L�B������SFU`��S�i�jc�*�y�Uq����Is���W���D����$�w�j���g;��o��S���Hg�
�����D!����������=O��&�2��`J���V���-���B�F�
�[��7B����(F�����|v��p�������
E��On���aTL��w�*������w<��[������{p5�N�_����a_���pp�����7���/�7d��|1�B�?�� ���a�q�3����BDwv^�NC�U6�&�gP*�i05YNn&�t1���n������2[���R$�*vP�5�bDJ�������X����$�-w�}�!�K5kEeQ�/��,�MtO8n������+�fC���.>������k��
--A��c�!�;#�x����]&p�����6�(s����x�b��e���(���(�������1��j�+���?wN//Y��\}i�����-yV��G'����dnh�jG!��������&��owH0��3�l�gQA���B_{?|��2�.2F��m��0b�n�I����
C�b������
J�2���}W� |a����q������T�����~/�u�>^�&S�
f��F�GN��>��9�S���,q
?Oe�E�t�;q��1~�e��f�?z���C8���}wK����0@c ��}2�(��^��E����>�7hN����A�wKG�����;mY������z�w(�s�W[~\��d��M�P�Eq������Nl���������F�W\U4��������*��Z����n��Ex�\0
w����W����p�S8����1��$���YZ~����H+�M�q~'"���0��U����=#�C�
������hr��+�����������~]dW��?�����sP������b�����h��?�b���'��
d�R��p?���j���=v���5�MDk�^��H4�z$���=���:d�Ts���jT?���cg'��dI�{���-����|J�����	��V�o03��
h��j*[	H��L�P�<�mKF},A5���sn�q�5�"I.�`��BF�2W�n��E�hi�H�B��eec����x��C��C���`�(pb�-��-HF���Z�	�,�H��"���V��=BZ�&�wU�=mx1(A�������+�`����s���-]�u��/���]��o$�pnQ^�|[�Z�����d�O�N�9N>��A�1�����(H}�+(���"d�t4�B"a�S�h����E��G@Ax-���
c����*�3H��.�����y�@.�9�:���a#8���/��W�=@����/~���j��K��{���K�
{r�������]����g��g������N4Z/?r�=�/����~��v����|pg���_�2�b�q�r���U�%h<���\g���	�I\EN����W�d���/k�Ac]�t�ZMZ��r?'�0s���{���#���>fk"�p��e�B3�u�
;w�V�?�2�BEE��&P��Y�#`��j��f`�y�W��#���
R�����?,����@�"��k�Y>��Qg	��Wq�K+��������~�pn�p��I{��j>0z/��1w��}H���hD�wA���2*e�J���z��4���{�K!�/�^���_�IA��K�y6�p�@��cN)2�S�1��x��=���t ,z�����Z&2��[�o��H����� �<a���-�)n�� Q�5��>\q�h�� h9?�kw�'{��[b5��s��!F�Z���o�+����`G������g�����hy
������������cE�q�%H�%�*�������t 8He*����	"��	�|�]����8A���Qvt9������\a��;�ap�P��>��Q��u�z�_3�������}��0�*M3��>�:�O8�������4���{Bd3���R��A��%MT��#3ya(Of���3�|�1�.�����I�J���]��#Va$Hf{�j�E�����R8j�t��#�C�bt�t������p��b�zo)�Gn$,;�f����A6�k��m�����rp������~�=�t�

M��3�D�>xP@%|.�,��bx��a����%
{���������\�p�������b�����`y/S���;i���j5�s��[�>G2.~u1'����Z?����>�j&���,4Z�@�6e���i����,d��;��I��Kd����^��
T���AFc���_��'=��x���?��#��)��)d��}�B*)���1=���,;i9�w�:�!<zp���������h
��������L�D�Xi9�7*�}@d��GA�x*
���|_i�C����M$V��%]�9�����wg���/Y�i��>�������d�<%�<����\�1��1�u���agW��"�V|N�C�����9�/����o��U+����������#�����I�j+�z��	��<jy�Fmx����	��~?�:m��)o3�^B���~n��A����:��22�6H'R�s�p�k�c�b�%
hfH�$fZu��l��Z���6�>�PL���zU��. /����]����,���
���.3����d�QM�d^�i� �Fn4A�����>�9#�y�2A����`~`S�Y�Y�I
���2�]t��p�!�q���o�y:��2a����H���x��QA�Oh2�%��������
���d!H������q�r���e��$J��U�[��]��4�<�)�F�U����������6,�Y�"7���R
b H���*p����_���t���T/H|'B�C�������aP���6C+\U&z��.x�� x#����T�s
XgO���l��~�@��!���KLq��`SxG��!*�^~�4�����d�i�yE��\�6���������<���.���+�]`���u��nD9^�Kp8`_�����Lv�����~@�FS�fuX�f9�����d���D-�x�<�'���|��f�tf�9�W��	6��q��%v��C�B���g��}�3�Yc�l��;8:�#�r�,�"[����s�n������w��}����E�<P��I=��b�*�IEEk#���P��e�\���$�����������uXqJ�*
�L�>�]��<�����TO���>�Q��]�*7+���gl��6�x+���t?lH�O��]�C|H^�x	D+'�g+ll���_>���9X8"��f��=�xB��X�e����1��������+`��"[.Q����k��!��N���������cPU���Db��<Z[��=GOy$�!��D���!���`[�#�I
�d�j� �`���]+��-��a����<�������xQB��������oJ�?�!C���+ ��?G
�
�w�kdK�����V�����:���pq��z����A��$�H��� �l>�x��A��*66�	�����H��S�@V=&��p*�@L�[�3�_�CF���������|��O�'S$�NMl[<�82���`�8Gq�|)v�	�m��������'������j����N� ���_������\|w��O��.������f+Mz�?�W��&WT��D
y��5	�D#x0�4���ME�����&�wh]Kq^�c4$��&]08���}���?�	U�Cih����@���\���l1v��4����E�ur�P�IP������\���O?����y�Y�c-���r\��������`���`9�VS�Y+�"��s��������nn�@�=e}�������������#CtQb�(�@ �d7��6#i�*t�4����R��
��$�������r<t$+��t����F}���>��1�gh����������E����)>���TFA���b���������:���A��1v�x�w�)�
I�/}�L��O�u#48O�k�I���l������;��9���\� ��5���CE���K5Mc�����MA`g�4�!{��$�d*f��Z�P��)���:�e��F/���HR��D���eG9#CI��H��DB�OA&p�"1�b�>�J��W[��#w Dd#T�,���]:��E/�~�*�3�B�������k���R����f\(�@N`��e����R
�OhK�0Lo;��1����7�I|������a�{#~�+H��em�y�,l8mZW4��X���q&�U�
�rO�k��k�!�	�2�������I���5�����R������&XE�S��-k�o��������^�Uc�j��60\�j+Um�����
+�s��,w�)=E��0Y��8�u�W;�yMAg7/�g77��s�}��Uh�J��\�#�<d�K6G�F{n��/5_D�T��=������rz)(���O�(���#�Q/w�n��Kz9]~dS��2��H�������a�/F/���X�u���t����^���#-��2����P|�C���BbI�������s+�^��f����|�}s�XR�6<�l1�`��Q�����
"��dd=}&�Z����-Pb� (<L��������fY���m��k�YB7��I���]�I����W��{��r���Y�8Y�h�������� ��������������n��q�%�pu4��'2��P�M��$�s�T�h����]M	�O+��53�w��N�������4��Iz0�w(�W6v��D��,�	������a�;���"8�(2JJ2[�-���2Cr/%S���P�$�(����#
���qU�z�U�?;��P�A\�_��(��hA���3�^�&�����@��$����q�S���W��I���+��g�&&J�aYk��k��1dM��'��J���+%UD���?��1������ ����L�G�\���;�`}��&k�L��z�x�l%���;��A�c&����BU	mj������� ��d<��%��`Z6�����������2����#��l�pdOb<�k	�1ws��!�F$��>���DD q�
���1���ZZ�/��@M�����<m%n|�����-}��b�3��#K���DS���&*K���8����w�J��^�%(S�@h�����<&"+�4K�:��9�,��rka�H5�G�������--$[�Z3���;)u3U�����+2gW�b2>i�6�R�!1�����!),,@��V.P�\$u��������b�i�\a!m�Hu�Ks���LE��"e0�Ui�*x)f��G�����	&��1�9nmnf#t�r��e~��2A{�9����o�E���c"'�d�g��..�H����O4G���V�p���V��J�L �4�4M����n~?�������.F�^��_�Mu��������G��t�p:
���$���k�C<��������W���\��<
�4�,B�Vdv/i{u`�]n.�.]s��/X/ ������m w% 42]���5��,��U�%�@�2]	�d9l�c���.j. ��A�ITEQ�'�����Z22tu�@h�����KF���4F�pe����j���5<�����)���y�M����Y�Z�YO�e�L�Yx������]{'�AXV��@��pb{8`5#lY�7�pB���]�I7����0�Q��F�����gVy�������+H3�����u�~�W��:ec��)��E��`4F�.$W��7�w:�eN�����N{l��~
I"�/�(��W�������9���h���&���(�a�a<s�����	�	1��"�K�KU8��f��q"z������VQUM����S9�(�wqd V�w9�>�3KH{EQ���u�,@Z�p�
�?8En�������%gZ�g��������+�DCr��C! oE2Pi��1���RS�Tej����Ya�����"16��(~�����h�n��rbr�����<��R�p�����p�?�>�����CFZH����=v3�Bc��$oc�<��iK�[�N�"6�df��2%5�/EV�|im�����Dn�(,���L�� ���Zw#}�<�EA�
	��� �7>�<a��r�0}t/��Uh��������Eu�+z��e�Y8��i��TdU(����n����&�_�(�%���w����#/4v^�����e6}�c���JfPtQ��^�������(�1xFlN�62c�0�����
2�K4����P7%��0"��IiEnv-"���~g���HQ�}!6����C�G�=)y�P��W�91���A�&���8�h����\-�<�z6��t���X�7�������&�7@Q��s,U���1�:�b��|��=��P)c�@�L����H�R��
���\QA�C��%������%�:t��^��f��
�R��T>�'���X#�&�?����"��c��x,%J�M�8r�k�
�(������i���snA��h�DB�����r��C ��'���HE���v��$�Q�o����Y���	��d��>*T�t��v�����f���
�B��a�=�A�������r�p7���!�rR�%���t4#���A^;�Tb����@�N7�j��4<�p����tU� �����lY�t&�������g��5����~����f~;�r���#
z�q0e�#P��qTN�:�u���D1����Z���i|���pSb�ru��H�D�#��P�Tm�"�lk��Q��Q�"`��qx������{Or�x
�F�L����!�,��$kla�3�1[U�
�WXO�V&���?�p#�H��T�����Y����s
gjIqafV������������n�>YB�5�h�l���i]-����#�0��<Y��Uv����a�����)�O�&\l���?���+4�\v��\33��	�����������i�s�?�mq��Yqz�2�jG��(��g�
Z�~d\������N�8����.��Yz�����41N��K^�� ��(�Q�i�������O��<�P�^k�"��E�+�
�kQ��`_a�L����5����2YS�Q3/����
��j��vn�����������l�I%��	m���u�f���,(���/����[0@����v���=�3*����h7"��IX#�J@e@Z�}�"�JI'��4��t��N�C���`�p��"��-=�w���D������B�����x)y��@��0�
0D11a��80�H��[��
=#H@�ERLje��9��E�T�k'K�G2�-~Jf8$0�|c��j{k�\�}�)`�<Z��	�O��9o]��E��S�EB�P��'Y�e;���9�t�-�|�;�*Y�!n�����x�����b<���qzp�;���U?��������.����a��?�>�^�(�b/������Q�#M��S��Rh�H
�0�F/�������w��2-a!��D?��B-&����2B�j����Bn&7@e���7���13�ur��i��h0��\�K��?#��Oy��Te���1���bW�OV7�<��*�;�Q�%HvI�@`�e�����������3����Dj�A�����5�
\�������L3z��
�=��������9n0ir#��n��6QX�E��T�ZH���`��P��D��P7Vf�I�]����F$F��f�7��X����	�����I\�7�^<��%�O���s�L�/���|�T'�h���S*=z0�������6�aGMi3���@.��(���cs���f�#,	�$�RR\\i�u���`f��	XP���w��g��e(U��%Y��K���pT���
:��-&E�Kt�X�1@I����z����p��\�N�E�f��]��R�����E����i��5��q�" gzM�E������M>Qe�R4
����Tf9��=�f�1[��� 3Dx�,����5�����|�U�t"�q����D|l�����������`;v|��#t}��#�G��K�X���?�������H�`���nCBjT��4�
Bk�;{���h�Z��1$�����,kayk\���,?Y/�`�PM����
��Qu�VT1�)C��"S,����7EG�D�Y������=�nn����(��������!}�Q��?��N01�0�axK'������
V�<e�9Ec���n&�����o����+v|Q�n���vt�����;?�G�����|
�w���D��
I�>����T��oH��v�O�k���u|8<<�F�GG��0���k*����H�`vx��	/h(���0pI���_K���2p��4������z�P���n�h����F��F������r89�
���A6
��q8���Q��M������t#��_Jg�����bS�'����(��P�o����:r�	Tb(���l������G1�t�<��0��-�����*1��R|�'�{��������D�Gx�=����5'�;�Wfdd\��	�lR������������
�8(������+������
�	S����T�*{�Bd�Tq���g��-�^��Ez�C/@cXrM����:�H��~A1�e�����MF�=�n����3���o6
*)���3��������3������:���n�j��l5
K���#U(�vP�l��w���ky�a�����n����4��L��d��H����Y��D2]��w5��%�$�u<��]�������CYd�������.4�{^+����8�������R�g��-
���m�������0F�t@��z��0�\]+O;I��Y����i�~���d��{�����n���JM���d1u4H_-�V	C��� �M���^�4]��/!�n?=9�Fq0M�**��TQN�"M_L���A��8]�C�;��z"��I#�`�CT�����^�G�g1�1H����#=A\@��w�
B�?�|,�`����yv+�
�T�����U�����Ku_��k�����l��8Q�"�O�i�T�=C'���� �Q�n1u���<-��q�[����
�����D g�2�$U	�W��R�?&(� !-c"<oa��'0s��!gmB��eJn���
-x�U�����]��}������-nt��B���8�)(�<^L��?���U����`�5M��*}�D��1������Mjr8������\��bu'a3��z*r��Y���<������r�J��-�Q���l�g!W����,Q����q�����x��s'�FB��z��.��C�\��q�J,d�ZE�cTM4���u��X6��t�beE�a�O�D��G������4f;����5�>i�|��1A|h��\5'S���������|�^$��A7��D��Y����!�n���l�.g���~Q|�?:�u�]��^;�n��-a��"������}�m�)l:-����4����aO!e�
rrAO�����lU���5�AZU�~`�A���������8~�Z���YD7�FH�������L���D���=�@�����"�����g�M~yK�����x�]�69KI��T1*��oBRz�1� bZ�$B�����&�LW��#bP�ZI�0���+^ea�5,/�S�A����x�vG7/'�l�����r��Y�`y	|���2��;���)��-�Y�2���p������O��{a����_#�X8LL���.5<b��Y��^�y�-s��-7����d	(��1>��\��W������=��3Lu�zri������'K�<���.?,�b�����-d�y2)�k��D:F�������l�,J����u~�y�Q��L?c����q����K�-4/U�K��+h�C�>N��n�Hip5�Afz���3�9�H�^�R���0�%�f_���V-���`4.smiB[-��J�T��Z0�/����^M,����P��Gj�!�<r���j�3������b���Z�i���N��������<���H-�s���/����<;���W��mw���*�sT������\>���-�wUNy{}�=��	�������tkqe���;4����"S���i?���H��H��E~8����|�o��m���e ��2�^�>a��9Q�!"�����Z���zr�K�|��Fy�s�+����#��F��%�QH_L<��{q�>EJf�C���`X#:��0�sI�\�����R�&G
d���j6�'��w��Q���VQ@���o�8!A;	fv�Y��x��wh<��q�d�)���{�����J�����
��f8����L<N!N�~�Y��U�6��^
5���	J�j�������qaj8K��\�2��f�*�f�����KsC����0v��\���P��8J��q�����;�M�h���i�)��bX��B;K��U*q�P����%<�z
z��(��i��>E9yY��
���-�d�:k
IB	H1�sf��>�t�A���X��������w=��Gtd��@�]����q1yAxMb�0�+�Q�Xev��?��[�n$������t��jN��O
��/��a�P\ �.eE�c�
f�h_�O��R��"c��gC�O���m(\�]%v�K�C	���	��"�u�H�N�:l���l2��S<�15N�������V�9S.�(_G������;�1r�����D~���{Idealw�W�����T���
@!:�z8Z�@ ^p����J�s��0�a��C�1+�_sa�����'�'e���C������W�D���OI��G�����+���
b���y��M7�j=�+��,1U���0����.��G_�a��.�>O��6�|y�(I�s����j��C�;* Q��Ry0z~�#
B�5����W��:��Xe3�\�l�f�rF���N6�	������sG������m}�����"������-�9o��-��y��������(]��l1G��M�-���ga�����Y�� �LT3�����������!�U�7�^���dm���n������7�d���42��$���O��'�(�'h���rs�����6�����������~�����a�f�v���4�u�������-'���~�)��'�n���I�H���������H2�8�Evy�������D��	n3K��Wy6Y��veNR�>�%��%G������u+^���#aY������#A����j��	�
�#2`����(�����&���a`���lT����]Y�7>E.�_U���i�(�����d7\`�����]&K��	o����#�2����y���%.�V�;2m�&�{��k�gTJ�$B��@D@1��U!PB��J�@���IQ���b��5��}��!�HR��X
���8[�Nw��@�\X��w$����|H5�wl
_���oD.C�s
J������Q%�y i�'a�wZ���w�q��!97�^T���0"$8��H�����>��%�������
��0y)��s�b��Wk[�,,%�4��j����
h�<7��H�d&��Jqv,�����"W	��a%Mh
4]�����LC�����-	��n6Gj�v�=��������)�3�y�m����z��jjo��w�Z]� 'N�d4����6�(��������]�!^�(�L#��*d[&i��S��k��eWG���E�nw��\�\�W	;���W-���Y-�K��n�+y�]n������F,?��JD.q0���#|��&���9t#�],R"����B���.�����������Cy���������,���Z$s����K�K�JH�wB+�(���9M�C�f"�<���D8��i�@��(��3���K���G.��H-�u���Rt��t�x�������3�Z�?��@�7���}�	�'.�S��o���1��FF��n�9���}��Utz��9��`���'��P[;�������������l	Z� .�U�����!a���#��+�����{k�1:5�R\6��g����.�[��T{��-�u���Hq	s,�
^��
de<���>�m�B������������m������$p|��i�Y���9H��P
�����LL�j�\3�j�hS�^=���i}=;c���hJ19��
�/�n����~��R*�~��D�[s �A"s��x������#7�"R�N.AHD`{N����2\|q��d'���#H���r}#�!j��,�4I��x����]�������N���9�i��!��kalt3B�CZ1;|�3�t�U�����`w�&ql$��K��*���2:(�0^�;^h��(*�B	���	�1��sXI����.a�����U������.D|������}";})`AC���H^���t�Z�u���u���Hs�?�8v����H��Jd��)q%��Xi�7o^��{cJ�|��)�)���SC�k���v]����Q�Y�E�FW�\�'���%*��!���B������9��D�	���eA;\Mm��*T����1^<����������=�xL��1O`�U�e����j��UQ��,����-bf��J�#��h�7�6L��Ap
�X�)F�{��-��VQ���)����h��$��Uc"���hV���!
������i��>q������b��llI�|\	:Jz�����������t���gm@P��i�m��7�a��h5�����T��]�=�"�(2i8��j���8d�l<��K����Q1\���-��x �C��P,������	(��p&|���'���	G��M�9]��O����������
�_?z��Bi��y�����w��.�G���?�F��|��\P�����n��*���]{5yw��@����E�����o����)�<fA�����������_�u�lW���g?�={��������������W/^�������_Gx�]�G���|~���lF��{��.}!�9`G���|��K�Wn,{PE$���W��_�������}�
���r�����L`���
xay��]��0IS&��x�&Y�xj,�"�7VI�R��c��y_f�,�|
r!�t�XY�V�KUp�_��%��`a�5���X������5��}n!�����?���5��5��A2:NG�W�q���1��
��z�`���!
a�~�&Z����G1d,���>A�w�<;�/��K0�:��g������RH���(�Z��Ce�s��e�������Rz�D�k�@LJ�x��<B{|���+�����X�\���n+x�U�|L����{b�,d�//�q(���S�#G�E��~�_p�r�aS����g��dL/���Ezsc������Y'!<�F&�	�Q�>�f��{���E*�
��L;���b2��Hh�[�.��^s��n�	h
����������A����d��2o,�����,����<Y0�\���i8<F A7��o��Bh2�������]����iO($�g?|KO��h�������|����w�������������~��~I�#H"�[�Muu���O+?Xj�N'��e����>�����U�8���?6m!u�b�T�Z8�Y�`m�3��)W+`��li�v �������v�_S�L����r�`P��;1�������n<t%����z���a�k���d�]
��*�P���bUP�	/�����"������b�&���`�h�����:z8X����|��yUV�:�rF���6hy��/Q a{�i��A�m8��#K���*U�z�5��`JOi�����	�En��b���a�y�C�2�&�2$���!yL[r�������[tS�H���H��DyNtW� |��`�c��'�O�����@p���	�����T�gh�n���kNo&W�(�$����8����
��f�!	
F'6�����RAr��M�,�]o!�)�}�k<������|:�
�Q;
8(U����P!�@�_�(��D���\���1OS��3����pZ��'!K���{�T�K�%G��1c������8d��J��O+�0Oz�a��mw�����1�ZH� �@=9iYF�xic,cx�VKA��r�������^\&T+�cQ���oo�/_`���+>��������W��;��i H.K�q����w8�����?��l
��H\r0dT?���c
!%!w��������E���oL��t���$�78z�����c����du�R��e�������s��k�����������))�/u��KF�:���Qz/~x����������l�2��A� ���p��cJ�~?1bZ���������34��CkJ��7��5�Z8������zyq{��[���(����Jj:����<7�������
t�Al�:f<F�����>�op��4u�Mk��q��8h����[Q/%����rp8b�~���N���QO��7J�It���AL��\M~��\��>I�H���7�wE	��5?���bp������,��A�FdVO*�a�������_~��L�y�������i3�z�����q�jtu�H������R�qw1�yH`>�K�y�A^�R�2��&%q���~���$`�����4Z�h�`����E��9���+�y-2PS,	�������I�ay��Q`A�H�GG�	#�/�WGW>,�*��1I���=�a�e6�P?���:_�F��O>���L���@�q ����!^X���S]��1O&���}:A���\��|L=H��\�������0v�!D�5H�y�f)HA���C�"��Bd=��Z���Ov�#q�kq�&+�yv]{�/V�=�n�F�:c��`8s��UM���m��P��x�F��������N���:%W�x7c?��o]�	E��������	�������Nuwg/�.c��E�����T{����s/����l�j�\%�%��0;j�0��Y�XTW	si]%\���P���W%&$�J������������-��X���|�$
�����;��>C-7�E�?_��f)����A*B���+B6�l�����������X��eMD;�����������{�Q\�J6o|�-�z�d�Y:��&�t��H�y�����*���?��;�u���'������?<��^�
���Z��ZCH�(�������;��������hx��499
���a�(>�gG�q�x�����]���vD��n��Q��=������T�8:�s}�2 ��$��`��O!����.G���7;�1��8:_��(z��Xe}�z<8z<8��z����v=�';�G������Q/y���#��������f�ri�Fw�)�Iw3�W�N�;y|Wd�-p�g8��r�"��Ga�ii� �.`��9x�{8���f��lg)Sz�O0���~Ad�Uj��,�D~�`���?����S=����E�P^��p��+���'��\�F�X�G�67c���PN=4���o��c�
7:�>b������N�,�U���
+���
q#,szbB�1;��&;��J���2��/�U}�^�!u>F��0��L��	X���q��Q��3�D����n7���5�����N���|���R1�����.����a�����Br���(B%S|���������G�n��o/3`������H���\�;�'3FCNV�����{	����.��;4�\�&��X��t�8�����s��@#@�B�����K���RWe������7R���L���r9:(��t�I�!�EU{FU��<��F������\%��G�pt�r��M�u������yW�������T�<���bA4��)%6&u��dH��l�P��\AW'^O��5�/.�$�f���+]�=�O�{�6��F���2=9N�T�zgftF���<����4�i�m�Pk�����ilA��0�
U�wy
&��R$���L36��:��c<<tj���9$
�#}59�L+������bFv�?�2�V���?\��w���SUE��z����\��|I16ZI�"V_���W����in�U��!R�HHE#�%OKB��l�Y\E\�����4�Az������cNka�����tF��n�X`m}�-��Q����|�L�,Z�\f:Z�/�h��������d@k��|��;�������g"Kg�U���rK��vy�g�d��O�� �
/��le�@G�L�SC|��F��J�r���f�BF����yS~0�=%�qr�k�u1� ���9�Q-�����/Y]Od��ZtsZ��=�d�Z0R�sj��X��!q���r1h���l�1(d��4�k<�VU��~fTO�f��c#&�H�5
s$0�����(���AF�d�#��%���K-(*OA��1{�*co���4���
VH!6 E&��o�D��=�
�1.��CZ���{���V1r����*!��{�����g[w�=��6���<&������a$�r�#�mYAG �4�uW,m�U��k�
�4���/��S8m�*g�9�\���L��E��`���#���g"� _<���C���`�^��M�]R�!>��Nf���f.z
Z���n�7�c���W��"������e@��kzHB�{b$�V��N���x��V�����V���6<S
�B&�l�nuMa� ��A1�8
�{��Up�@?q�����Us�J�&���Z�Rn��
Qx6gSF�6�p8��8�>�
���D���Z���E�M.y8��������s�w,|�ss������
���F����K�_���~����#g����g|w����O��[84�O�c_+Z����h��?����7.*����7�sA��i�35�	������$t/n�o;v�`�� �%J���1��%���g��M>f)��1��������0���v�^�����#�'J���g�z�MRC�����?�f����:m+�C^�`��I�q1��b]MPfH��Sqe��y���Uh'�g�a����4����V�D3���Q�(L�">�u�^����:�S1�$0����Z-^!i����0���ig@�C%� ���J��o���bu��f�s��~Ci�/5���Dt�U$+G!VC���G�����KSa��y@!@�TJP*���
�3XS� �^2��|X�!5C^nd����<<Q�>�
#����:��+|R]sA�H�Qt=�A�uBlf��p��sAb=�����������W� �P|E�
qX��q������ ��W]V�����i|4�O�i:>���5O��R��*J ��n}-��"g>+��H{����E���I�2�,��z��F?�xu~���=�EV#�?.�����$��P����eutiVR�%W��B���e�0?����&�?������3t���$[W���LfIJ��q�����;LZ[+��2HD}-��X�O��1'�T�a�5�X�Y$���p��A~��Y>����*�O�!�"�3���*��C�)|O����X����d�1��Y�������u>���V������3�
D�V�b��)���sO������S{)��	H=�y�v�p"��t��%�N: ���
���R�9��wtq����k���*��c�������y���)��&#z%	���<���@!���.�~�����P}���-�md����+�f>�	7&z�zi����!���a@tL��
P�cR��WX��s�AQ�laMf6BM��.7#N��E=�������@go�>���L*Z��R��F��9�Nq��W�!�cN�^;"6�8D8�Q������Z|�&�4x���\[G-�����"����v���lb���Hl
c�����dW�>1�1tc��%�������q|�'
��������?���L����$2q�|���z	��_��)�
�c!�{�F���)��N��jH�),��S��Lq���=T�M9��s*F��S2Z��k�Flx���^�d�3l��J��)�<�������w�!x�V1	3
�k�Fhq �&��_@�eHWS����k�^`��a��j�(s����Vs���m�T��S�C_|On���*C�G����g�%v�e�5�����k��zI]�E�Q��eFR'n�"����K��T������'$\OX��gS�%���X�0�7��'E�*
	"��?�v�`W��V/eHt���d�_G��������t�FZ��eu�|�x����	-��b7q���Fq��3�#!���r�$Sv���B���1�\���$����tY:H�������#������=��?�t�TbW;h�#>A`a��n�gS
\��H�l�-voM� 8�:��F5?��bJ|��t2����\��!��P�p��1XX����K�PWT���-���"�fO�
��C�8h�jqDV?���-&{�G
:�+���k��4�YI�T�n��Ri��3�P����(���������_9�,c�<E%�0���8x��m� K�������h~�'8dh��$������U�`�������"��rT���Lj����������"=m�������a�;��-n��|q���+m�'@Mf)ct���	�����,������gPmgw��=)�M�S�	�4E���}�"�U�S�S���)����w��Bja>
����D��&.M�����Fd��HA$��Q.e�H�#P@�k��9�a� ���	=Jy�LR���!����"���T�P�����&K�$� ����`f�_��\>���T������	5��������_�Ap�&�8���2����;���b`Sxx�7y~�� ����B��g~}K���V�������k�JYP������]k���z�S�*[y��}���d�}LlT�t��D��au�1f���+u`V��rWe���^������C+)4�;l���f�[U����kx�RPW�^�(+�t�k�T�* ��P�3���7�"����"��w�[��i�E-�9b�w��=��^9��y�
>�j�������N��
Z�S�dRM���h��%��z5��SsMs)gD�?�%����"�p����fs�t�)U�pUy�Y�o��������f��j��&��_�+� f��\cVv#n���uR >g��S�77�]�C�����v�A]),��cL�QI����d�/@�c��od�lNn=�(��I�*L*���ew�!�UbAX*����X���m|��X6��(]1h/rx_2�X8������>8�����3���
f�����^�\���V��� '1.�0��+B�E�������
��<����:��O��7��o��:��������u�V9��A��sO���<�QK��o�!4*sw�AZ�g������?�B��h�p$1o5q4�Ho�a<b�Ga��0��l��d��$?t��Y:a �l��G�S*%�\%�6zy^������6 ��:H����x��W��Fs�m�>��I�(c����L��{��w��������\���:�kHw�>����@��R��Q��Y3�C�;J������G��rtt�]
����|�\���q2���GI�6��z�����@��{���7D@H�l%�n�(In���������"������!&P�bHe�jXe3�u�b�1���MA��`(Ki (��u�t�z-I������0X0���O|A��n~�p��bz.��I3\�����JAv���v����K����C�# vv������OS�����J���	: Q�X:D����&^��w��(����G�����<�A\D��>��G<��������v�a���p\=�=�Q�U�������y|p�����K4��"S4�,j��c����?b�`,H(A ��L`�Z��)2�?�K��e��v��< e��[���QN���:��E�^~���
�]>C�9�8������w����Y2��"�C��c�zb�vV�f	A���2-���l�`�K�}��<�����v���+K���}Z�O�:"�$}���#2A�'�5��N�?��:�?��=�G�:��#�M�BYTnr+`�@saH�%�t������-d�q!t����R�i����S���j�gyO�'�
j�{������������U���0���
SVBp����P�"n#�Pu��B��kX����vO��Q~r!��md�@7J����z*rjx58Q��O�������gW�r��l|���0w�
�1����z��*s��m�XoT�Y���a����|i�]�$
��w���M��7��}5_��ku���������f���r���sU;��a�=�s�l���C3��zB�/�!��[��8��������1��OB���|��=U��yH�/+M�-a2���Y���Q�N%��J�ru �����'���Sc�Zf�f���}���U�w�}
:�U���cn�f|���|G�G�-����4J�}�=R��bI�x����7����1�h��@��	�E�`��������\^�
��H�m�l�����]��Cc��
�PU�.8��� Ra
� ��*/��9E��H&9�
�QM��K,�D�����(INzG�q\���[Q!�j��U���Q��w��A�5�L��K

~�Da��~a�_�e6����1(��U\�T.�+X���"d�X!~�`
Qe��z���i�e4�;���:�9r<���������Y0M���Kuh��i.�>�h���M=�d����%�GV����� ����c�(��._��![^��'����)U�:�a%����u�od�$��}g<�;`�����}�q���5/d�������a^�w�����E��������?'���(��GG�G�t���^/>�]e��A2�d�WF����q�8�=$>���/f^�m1Z�ba@�q���������`{�4������z�����\�RZ0�*���Y�$+i�P7#45<��Z�Pk_���b[HfF5?�a�4��!AsA����@�?<�/��G����~|7���@ ����.�V,��3����p�r����\T��.��@�k9�[lE\���o_����3��R
�������}�`h��`h_Ig����r	�Y����7���|��������%�e�&�=J�*D<�@�7=P�%A�c�c�����Q��|B�)�^�!Glf�K9FL�xt���F4B�u��L��*E�����Q��������k�F�H�����W����z��a_N���c��
[�J�v���Lp�X�\��@�j���sA�P�7t0E�����T�=�Y^��7<?oG��)]�����?8�<<8����]H0�����{���-	�>���-�f�8�\,�F�x����= hc�@%�t ����!0��idL����"q��^��sy	GQ&���"z�%�h���X����.�d�!�Q-8��u(�e?Yr����M)h����������aW�CUj~v��Sd�B��)��PS����Q�� >��GW��	y%�y)��u��[����2�-���	9;/�5�J�i�H�YY���qVA��le�����������������������/�����?�u�H�!CP$��������W�����~����?���
��^�����[��]\(��'G����������\��������jxu2���e����O?@���$�!������'C����C��h$A���q�`4b0���8?���;;o������r����e�@� :��������_���;l��3����/�;{s�]t��`�g�A�������s�)N�	�41%���Ig�;�y=�����gK��a9���z1_����t<�%�����y}���M��z�>�-�'dZ��
����Y<��������(�{I��VD�D#5�t6�r=52/b�}���_����7��8>��8���Uo�LJ�$�%y����i��F����S8���b'
S���>�>���}��imz�����K��]f���~i:X�@����h��%p�?��o�YLg�C�%��6�����\�x�T�C��n��l���5�u_X��^�e��y���I]~�I5�%�SG��S��{�M'1t��}��s���%:?{{=;}{���DI��&�t`F6��Y6sdu?-�l5>:?'���o�������1��t�.��*c�����N�Ky���O�l�/��s�a6yw�jw���T��J�~�_�-�;���$,}:�=�C�<S3H0{�����,�4%��U?��`D��FtC�L���U+}������?�uR`ru\������S�����r�������7��:�]8�:����%	�e����|��!�@7��!������7���W�~���������]�?L+�.:����T�p�*z��������B����''��FV������o����M[�I�{�<��������B��$oW���(������ z����%vL���X��3������S�5�������\��@(�
B�k���K��'��zlP�@(n
B	���W_�8���9 �	Hs ;A(�a�-�c�����_��:9{����g]F��=�&��.`@�a�?H����������L��K��������O@1�JG�������C�x�~��d�K^3+����m\��@1��Hq������v�Xa^�W�k�{��E�l���G*�������6x�=���� )�y4�_X��9�J:8�O���\Ah�m�K�h$vZJ2�@��U��l�%7���j���,�%k�~HTh����)cF��u$(�F�=7�����M^�5c���������v1�R�z�
^A}�5W1���1�=����h��1o6������|^�}uT_�[)_�����6��������Y�]�j��jv�A��-��G)
&��F�?k��c����XA�1P�!~�7�!������$X�G7���P1��lb���ZN���43���fW+�>�2��<{~�,��Q��|j3�e�>�:�z�V�[2(w�k2(h���PP�1��[�(���I��q�2������:��=���X��|l���Hl��',*���Td`��%B�����,?�"C��N?�E�i+��.��]R�!����������g]24
�D���h�$���j#&7��zUv�Z������q��u���7wCB���!o@�7h�.���#G8�#���:��
�����k5�]�����0PH|����o�6]:4 g�6<h4k7���!���M�V��"�=xe�B���2��������i}��p8�D��h��
�7q�o��~��7I��!}3�o:���gm%J���R�X
F�^���vi�N�*�/��19����z��8�]����w�C�Aw}v�2������qW����?m���JHw��B��*�H��3PV�����B&����v����:$T�����OL�Xc�Xk7�7}Z"x�XM{��$V]+�A�j$�8��R�9H���-�s�,HJ&&`����t��>m	H5���Du-	�D���$A@b�6H���db���Z�A���S��>�D��3a+UmO��������j��:��pH����������������e6���Z&��'���}1�B�M���.������ �sZ�����^�����l���+�_��X%d2b�^�1>�2r�#�h��
<�m"�c����F
w��?G������&���n$������D�N�k`>S�Y7�v6���ge��P&NLj`V�Y7��6���me�1��^-N�3&�t��fLt�D�%�!HrICky��%�g:��|�kM�Z����3����>�]1$c�<V���~���G^a�������{��z�jw��^��M�b{�?�C:��v���~��aT���!!c�+�A�|�9�
���i��R�$�^�D���sf*`�RKy��:lH�X*`	bNs��1V1/rw���"V�����0��u����T�>����Ac,���.���
�������<�|�>z����r:�����y}
/�+��~�[J��P)��C�������J�*e>��4��L�������R�#�N�W����������RJTTJi��J)��(U����kp����������Em��}��Hc{�X�2-�(�	$��qw��O�P��� �8@]����� .�G��X�B�^E,���P��O��S�_[�8�A�������p@���-�p�����Z�t�0��D�������,��o>(�����9����(z��p94����[�W<1z9��F/!�O
�����[b]�{�������{������^d���On2�'L;�:;"���F�t�\l�4���A�� �w�x�������sIoe)�F��1Wgr�s��B[��V>�s�N�h0�T����������#�I��m�9�v����SZ�b
*���)*�!���U������j��.���t��ag�}tnJ9�I��5U�Hb"v���1�^����O��Cs}����9f6)�Y5�B���l�����7r���P���,)��{��@~G�p	���j4�5����p�pr�ewy���.�U0y�w�W^����Ulh6%4��'����B$F_��Y%��`���n�:8i�Q q����-���`p���[������G|���$T%�pT[������P��d�Z0��
��.��������@P����qe�[]R�E��}3�4�U7t��;��CoN�L�^l��E�xA�x�;DWH"��>����2'Pp@_.l�a�U�FH;���W�B���V�/l�|>B����������n�Z�VWK��y���V+�5���)����9�'�g�$J-Wy��a�5���]t�]i�`��0����������
�/9���G��R�4ZdW�"�L�A��"J�����������Dj�`��ew�E	����,]���B�.���+��h�	���-��
���7�������T��
W��UY������*�CH��E�F��g�o�;���/vw��P��N��j�HW����^�f�j*��'N(&��
'�T/�����hs����z�����Uv7K��*�l�����T��*B��cJk�����V�t�x�Q[���"`H����rS�����[�nV�2J���z[���&3��p�*l7�/.�0�UH���H�^o`�]�[�z��@j�O�����I�-��Jx8�����1�-�u�-X7H�d�^��[y�T8MU�`,�`�S/���mA��Ml�vN���+5���!%k��9�/���������o�b���������?���.�����W��R|�����B��v����H�����lZ������p���H5��X�,.]�����r�x���<a���b.��-���X[`���=p/��g.��-��$�j]�u���2���%[���~+�>��'0@
Y[�|��X��-X���������*��

v�g\J�)`H����D]���$������%���B�*�je��[��:2}A���.���_o���=��=������1fe�Q��D%��9c[!:?��%E�����p2[u��m�O��t��x������0�����Y��J�E��G���g}@e�_?�<�&�������r��nt�����5�a5�Q=��f�[WVT�uR]/^�={s�x���o8��t����&���|���Y9�x���E��.��b�-;�n4<9�F�n4a��:}�����=`��x;�%�����w�G���e��^���"K�m������"Z16b���3��� z����g�������(��^��Xs��3��e�����KW���h�������YS��n@s�Uhqs@Kr���;-vZ�	-�
�����u
��$�����";�)���8�IKa
�j���Ei������Z
��.�a�X�� ��BV�
�V���
9�| ���S���Dr���Yc���
���&�v@#~/_k4[����b�/�y����4�N3��koMI�����K�l��Z�+L�J�*RF�b�_�{��{\J[
e:�GvZ��4�sJ����P����e�
���Z��;:�[NX�OB���&� c�M�z;gm����HM��,_4�m�Z�������[��� 
�VzK�M�D�A��@,����x7�s�DO��t�a*���K��c�Y�O��-Y��z�]��o��F�����cDI�QJ�ki�P��?<y3�c$E�%��q���]ynt����f�cL�.������}�������V?]!~���:���������:,��U��PE^�b����"��T�����7EWp6#w_K�k���w���������O{�������o�"��k|�{�<b�xD��h��6W-�}��^��v�S}��n�.s�g�����]����ko�a4@�����������B���kYe�{0R�R�M�P`��s��)/�}U�PE��j�R��$]RTA����w��mg�*�h��	l
)��Z�]���a������I�o���������dX*m��!6Q��
un��a�N�=���ez/� ��C����������7���l��5K����|[k�*�3���������R���i+��:7a�]��� ][�O��_�������}���3�OM����~l�?p���TA[�9yw&�� ���h+	R��%A���������q���<�����E�oq��;N����T������������~;����\|.������;N��,_�!�r������Oa]������A_����P"�����4t��P+�����4t+
�JC�$i(6���n���V���:����V���F[i��D�6��9�r��r9�*��G�Y"���Z���:K�6�\k2Ug��F��?������{�C>�4���?_G��l���q�%�V��U��5������&��I��n�ts��s^���E�*��%��������-6�o�WtiJ����)�������� /�$#��B&g�(�q�2���Bt(��������� �v��X�������
>���n=�Z
�Zr�J����#`r6�8�v�.q�
q�e�m ��b���&�#\$�n��x~c���@�V zw�� ��Z�����n�������n��v}[h�"U���xBk$�2���LU�2��C���&�Nt�
�T��"�n�t9������s3��L�%��m<"�rJ����L�Q���Y�6�]Ni��d�����V�j��%��mT"�rJ�����_�Q���Y�6�]Ni��d���I�V���L�^���B"t���C�Q �*)kH�J����������!=*)kH��������Z��R������~mD���g�����6�.l��?�|I;,�]�����}��
����5��U}���^+J���WE�ymF�[p�d�\�S"��4���y�?Q�+EA8m�J��O[(l6"j.��\X\�������+D��|=����U���T�1��S�t^E�������K�H����:�pdE��V�p�;�Q����P�/B�����NV�`�R���2�X�#tsF���Y%�D�-��+�W�F�������j"�Xm�Xm�X-w�#�XU��*L&�4Yl �R	L#D2���,�Y��Ou�PEP��#���l�HI�k!�
�Z*���Dd�;B�����!�:[L5W"��������<"+����,Q� Q;Q����,�G���
%�4Yb �R5u#D2���,�Y������FT�������Bd&X�T�#���*w�.�%Cdu��j�D�P���]yDf�[�0i,�]�j���uk�����Z�����x��0��1��[A���K��R}�|}��a�d�a�u%��p��!YXd�E#[4�E#�r-�����Y�������:��g�3�m��ie��.��r2�-��,Z�G�����h�b����&�9c�Y����Y�n����K�"���������:�_Bc-���K�����Z�7��=j@+��q���D<�X<��>>���\�tc�'P��K�]�%�{���h�[����X)����~��tmN��������<������\*����]�GJ)�{GOK�ulT����3�������d���IU�lL�v4�,�i�V�sZ�:�q�Y����2���������c���=�m��Z��\-xg��0`#e\x�Z�0`;���p�Z�8'R����1`C%�KI����z��(�g�6��Z��Ty-a�FZ��.�w-`�v�v51`^]W m�
�>��
��P�7������@I��B|�'
�A>*���l��<�w$y�����:�.m�L
u�4��m\�j����8��M�L���TA��\G*FpJ��[�P@!�U_��H���ADt��;z�=�p>c�j���o�D�(3nE4����}9�����m�6'�����*��^����^n*��9�����$����.kL&�Ct�H���)���o�z�c@�����t
u����U�,%���N�iX%H��6�22�xi�:�@�Z2�F/������:��0�J��9���m1�!AR�$pk��&����3iAS�U��]����d�p����m��6�2�`��&Q:F�H�M�M2��I
C�T��c~B�Ka�d(�_�"�G�&'��u�8'-h���1��z]u�n\�)C�6�2�`hm����.�OJ6���#1�����l�v��T�D�Klz���������<A�e��	��M������(f���9��\���M[��N����7�'��g����4>:91����!]����!#��T=��G���j�����e��>���T�X�
;f�{p�{t�cj��s�t�p%A�H�S>�����USm���N�G����/Ua�y�y��Cw�P��c�c����%�9��c�G��$�j������2 ��Vg��l�1=���^U�k�����~�b9�����X��$�]���*Z/'�w�dv���-a���ln'�lw�G���h~u��x9cO����������i�����	'�/R������z��z��_.��%�r@_�x9�����z��_.��%�rD�^��X�t:�t�w�W��U�Ub��_��:}���&�'�W�I�{��O�q��8WI���u4��������*�&4q\u=����P��E�<������.R�Pc�H���]�S�h������X�=�����+_IT�B�I���
����z��7��N�����Z���L%���Z�*GW����b[�q*��(�Zt9D���q������-����yxb�p��8���v�sM�{Kh����z&*�d��&�w������Q]�$�L����IP&����YP,����rm/R��T<�s��g�E��! �W���:#�KWd��9�+�M�� �i��dC�3��e(*K
e������[���&u}�tB]_�&gD^m�5��j`IW�
��h{)�j��Q��2������������63����f�)(���Ce����{H�m��������7���jU��zF\���X�R9��O�����,j��V��l�H�~����l���5��8�c��1:���AF�][l��Y���hs��OW�x�~������UfS�J�tm0�� � ��Ri�}�N��"BL���h(�a��E����)6F���s�`�9���+E�d�j7���/�(�aD�"���m0��J����Ra�}5���"Bl���j(�a�E����)�����w�`�9���+E�d�j(���/�("g5~��3~L��j�s������<{^[�]�jk]���-�uv�S�S�R��mj]���V�k�zHW�um�a�=��um�A���U��
�#���&��Q
ol������h]k�hZ�z�f�&��X�n��m�����+���������vk]y������
��hf��S�\��+F��m���0�
��NM>��������k�{KW�um�!�=��u���6�(j�e*��6�Jk���1�V��}m���>���a�6���U�Z�����n��v}[H@Zf�,������airW=��(��
3��p4�&MNaS1��F$9����M�L��wpMuJ�!ja��DH��3�2�]Ue�6RI`o��s��H%Ny}�8%~T�QAE
D�L�B	��>���B� ��c�PC�F
Z��a*�L�]��8��Uw�[�TI^]'�����7�5�����@�m7��1���p�n���2��p���9y���������������=De���pU��.(��w~�m�}�����*�'\M����c%���Q>�E������^)��=�9o����~�<�Wa~��`��[������������K�C?�vx������Ky�;[
�~����8�������[)�S�o�����~d[���i������51�|M�>��[�!]!&<�~�&
4����8��m��u��m���L�5�&��5�&�t��a���o��������"o�Q�SoS�3QA�r^����5�F���R��ZC'W�����B���2��y�l��[h��l���h���<�F��*[d�q������"������d�ku���Z�|��^����9}������H����8}��)��#a?�Y�\�F�B�.���+���O�2��E�_�\����^.XK�`�n�0��������+�8/��[�"��2 �����:/�N��w$V�/P��7���8�R�{k�q�5��R���n+�(��j��������+�Xa�4������9%A�����6R�[g=4�PnM�w��"���[�X�6�-����[i�Su���!�L�-���N�A�����|��rK}J���{�<�9��u����L�_4��0�(9��_9c+��^��	���s*��\�{�C������p�N��������Y��(��+�j#���E�b�A��5xxUq
�A+�g��Hg<����X� ���k��d#;`ml��vl�����9�;�r�[�w��to����x���x��[q���'�x+������L�l��-���q��=����o�����������~b=�1>l��0�9����	����b5YM�3���>F7��jr;�"�iXR�go�����O�}yF�J7Q'�&�U��-����t1�N����o�_��`��A���W>�g�k
���}��hM���������D9}���[n ��2N�vltN�5�a5�z�V�\]X:o�IX����~R�/^�={s�x�^��9��t��������|����9Vz���E���\L�%������V������o���6��g��Y� �:�������d��"��x������<��/�6 V�Y�@���A�&����X�q�5�Q���������5�	�����^��*v:#�)!�������p���Yc8�n8t��qs8Lrp�p;�0v�a�	�0��!
�����Q>�<���(��R0�����"����
xYd�V5e(\u��Fw�9T�?���X���(%L�-�����{vfW�iX%�1V�%�� ���p$M$
 �%�V�/HrS+6,	RZ�a���OV��BA����Z�Ku���1��95��� H8W�]�[](H�H�{\ ��j�2�v����"a_��8E�(�XP/��aW�m(��4e�����2[q�\GD5�����P����Y�������"Qe��8��(>kP/�2���e��0�-�~�G[��~)x����x��w�6�;^Kt�\4�nGCl�hh��c|�n����I����!|�����l�\��v�>��5�R:.6�/	�������z��>�m��~hK����S@v�bq����_w���.K$iu�1�c�U�9@��a�Kv��b�<�����i���f����/���z�ja��~	:��
��=�#�>�j��+��Jh�����s���y��#��+��s
�>z!��
"����1;�f0!7��+���=v��N��-��
�����; 7�	�ej����	�����k����>���M���	
mAn��W��+�{��`��������������r���[f�T8�7��[��"��k����������cv$4�l�Q%���&P[fW���x!k�P��8��� ��}���]�29��;NO�;d������
����b�8iS��-����x��s�
��d�2,>P��<pYyL0�t�+vrX���/��w�y����M5T�����V��=�������zmu���>�o��1�������zy=G�~����S�C
n��#�����`�Z?-�:�R��'����u�4{
��0�`.ns����V���s�z+�\�����V�g������b'����-�G���[V�������,@����������6oX�E�FZ@��M����(3'�t�]
Y�:�es��[kX�"�=p`���;�O�=��8�'�C����������� ���R��]�3�����O[�c=P"�U�Kgw��E����d�(������+�������GN��
���������8��{�<�#�MvG.��aFIw�G��I!����-�wL6M~H��tw�M������W)�m����������KS`y�
��*<kA������2]6�m��t������2��VKm���N��bh��V��X����U��1@U���zU����\�7���l����� �jP�=����B�+��
��Mlp!���#���[�]/4�*�����n�e�
�z[P����M��a�F��P�;*F�o��Z8� :|�r���J8$8�1U������8��T�m��!�t�.����	l
Z>w�C�L��C1���JO;��R�a�Y��<�V�8�|l�t�R����
��;G����uA����������
�������q�[w_��n8������s8��cu�����~{G���D��n �cw���='o�u��!ql�� <v���[�P��nP�%�~{G����
��������N��u��
ym�<pr���X��nH>:p���8:��}�nP�	(M����������n�p������������A�}wp�f�}-��Zb���o��9�-z�����|�[����6z�L��i.W����F�t����|��r�?v�����9�kx�l�c��y���������n3��a��JTtx��v�O����
���4P�������2W�,?���J����G��6��������n��kK��+$���}UL�����2��E7��Q�	~�4����4�(���0]���s�q����,^+{T-j�j��`��Y�U{�hl[�]a1��(*c��4^6Fqy)���QbF�a��@-mEU�R=���Nm�R�a�l�"hm��
�d��Q������j�Q���V����m�Ot��wi�~�rP���ic���J��P�>F���W_�:Q����cY���f?�Jda��|�Y%������PF��.�3~�u(��N�|���HF��.4�>��Q�6��`��Nda��b�\�<���s
59U��\A�^��n[�����~[o7�v���6��")�����n�����x+)���Xv�K���m[I�VR\�����>H����E��x+)�J������~%�z�"Iq��o%����J�O���3������,�Y�r-��d�%�*Y�M!���D>���'�R�K�d��+�n$kE����H�L��'�h�fI6%���v@�����Z]�d��j��������j�U��l����,]�z�]G��67r�A�������Z"���U��6	k�R���
�A^7����z"���-�+�+���I�(i��,\R��H�h�T���6�,��l(�M��yt<���SJ��'jlQ���H���*�	���b�����4%qr�<���V0MEQim!iL�$L�������4�Kh�	���D��5w�o_���'�.F�;n��~_;n�bh�0kx�UT�l��[9�����S��2�-��L�u(�i�S.��f6�C���q�f�r�[/��L���[9tKrh�(9D���j��������q������9��7�]�A��x�������OKrh�X9D���j�������Ir������9��$7�]�A��x���S���59������X%�U%S%APY��"�\IaS:WR���6n%�MZIaS�V\����� ��*7�Do��n9��em��H�;�T�|����6��]����/i����w���bC[�����5�FW}���^U�E�/�����;���99�7�����?�]�y�H���p)��iS�P2�~�r�Qsau���J=����7]!����4����U��������e�P�Q��J9/%�!�K�� �*x��EauEC���U�0����
�k��+H
�@�b��b����Z*TT�mkv�.c����9�$ni�9Us%z�����?tY����[M�m�m���u����Q
�d�
�-;P`�X�
���(0�P`l��XC��n�����(�o
h��Jz_V�~�R���&
���lo�����UG
X�?t�Q`	��D�������Z�DG��>�DU`(8���&j���EU�h(0�P`����1
����
,Q�����+L�j�P�RV�]6
L6��lN�\��
����(����a�X��<�g9�����d�l�?�k�����xh.��\"�;*.7���{v�N��m|�����Yk(g�2�YHg��:���b���:w�u���v3��'�6��8����q�����"f !5G��aDb�������.�f*��QQ��N��4s}��B��rjx�*[�����|~���=��/�Z�k�/�Mt��5��;DW�8�a�`�[A�����P�
x�7Vq�E=h�wr�(5��"��Z���}��k�nC���2�����FW��a�<�����<0�9g�^�q��=h���	J���������ke
=�ZE��ZF���Bs��]��0���w�z(M��zh�uK��l
�����9�6��[�N\�����R�]3x�=�S�����%�)�SI�I1K�3R���$���H^Fq4�cQ|��KC"���HCq^�T��X��\��v�6�P�e�<��
���D���_iB�-��e�}�3�?v8���8��m�w�/�U4��~��US�/o�.�v;�-��g���U��5�j1n�����
;���qc�T�e��e,�Q�[����U�C�5�k�~�����
;����c�T�e��e,�q�=��'�����e�oYi%�����R�����Y�.]M��9f��8]fK�
T}���u��fa�_E��,ZdS�����,����}6������F.z|�.���GlHlk������9CXLV��m����,�?��d!^��e��;i�{�pX�Y�)�T�<���Xv��.�q�>=�'wO�������a
�3'@2�b��{��f���;V '�`}����{��J�RR/��\�����2�0�GOm
����C�,J��M�h�"��7�:�J��>�p�E�*��W
�b��;W^:V�(r���_�">��?�Pm[��\�-��"�{�4��9;=?��O�}y�;�bu�h2[u���q�>]����n������/�Fg[����?�u��O��4��������~����Ogo9�y��~�_���[x��1�k�5{T��Wo���3�=�_r|7�������|���)��]6��*�Xf�I��������xp�*>}u��o�-z�{�&��,v�1v�=e�6&������y�����g�:*�o�>�{_<?��#��������u
�=��~?�m�]k{�����\C�r��r����}��4^���\I�r
��'q/�����,��u'�u'���|��N��NK�����%�I.?���5���\H�G�,���5�W�YT�u
������������Vs��z�r`/�����R�R��G��b5�����2C�O�l��FY���N'�e4!��,-���l]N����Fs7��$9�>n�"��*A��7
�L��������0���C;�.��X��i�)U�X���A$�
{hX_qA\v�.��������;H1=����	p�����6��_������`(�f��>d�LCc?g��!�)q%P�~����*��j`�Z�5�&�~? ������W�n����E��_�xE>.���7�����������=0���Do�����48����#����t�0]��i
���u:�\}����k�!.n���t���|��x��������S�W�3l�9�����0�qz�w�'�RB�jR&^s�V���X|uw�����|���_�����<!"���Md�+�����O���h���v��6���}�.��"B�����d���b_����/j�r1�K���Y��r��K�����/V�h�5�E3E����>���L^���<��+����~�z_a���vD&��6�B#%�}@����G7��_�GR�l�2+B��P�APdZF�?)�B����H������������Xi����4v��
o���d�����HT�-����jVN���m%1�����Se�5��#5AR���k�3��&<���K�0=�a������A/o���7��tN���tN����wJ������x'�U��P�m]�*�=pX�Ls
X	C����������j`����w�G���%�`�?^d)(��)q�`�"j��S��A�&�����1��jXV�Q8�Vm�g��m�7�����TDN����#��<b7�p�Ty�^�7B�y��
A�y8��y8��Eq)�`<wA���p���y���G�����8l���#v#���<b����#���#� �(V�d����6�6b
J�����H
K��J,�C��&q���Qd�m�pm
��.�*��
N_D��c>mj^^?�I7��j(n�|������2Ck(�
��f5�#� �������
B���U@�9��owC������PD?	��!#��eN	:��s�-�)��lP6���&���Pg�R�
�{����>���������00�������!�q��x�;2����n;2������7���p��7�A��Zk���}�#� n�F��(S�!�=��s-Z�_��n&�j#:�rtJ]]Z"7^*!7�j��T��4}Y&�����J�����n�l�2��(I�J_����h��-R�A�@��{*�"hT]�����@�����{�k�BA[�X�S�������-�4m(H$$6$ePP�?b��A�B��Lu�Q7k����r��tBFX����C����U�~3��~(K��~�1��1f�&�[f9D ���t��px,�~�|�eOxT��,�{�f�~V���J��<��������x_C�<�54���q�(�U^o��sZfP����:/lE��z��&�pT��6������3�%��2��jM�+X��d�IrY�W!��z|Wll	m��������A�t��
��������q��\����Lz\����9�[d�;&H���tY���� ���q�6��<c3�q1��Fp�sU*�0� �E��eEk�4�����JJl*�|�'�I��{�D�0��p���}�/c�$�����@�I��4��&5��;����5�.�hh��	���~�S'�ox�!|�9�+(�V���`�<5��1�W�����y����~'���/�e?���o(0���������
��Q�������aM�[�`W ����N�t������/�>��cP��!(5�
�\������JK�O�h=��#QVE\����
w\4�F�:�@|^� ��Z�����m\c
c' �>X��c������m������Kf�����U�&������V�KM��M`�������I��:XQX���k���D\2-�������5�Jba���`I����5qb���Y�����?��p
�+|
��i���j�c���`P����e�����8h�[��~�7sRp��Q����5��b`�*K(tXI���0�[�������9K�(�
X\W����-8n�QGpoI6M�%����_�
�^����9}������H������-�������D��%��4K�+��.|�]����pl���k��M\��������K�5%V^.X��=�O��>�8���-�FvK��a:�,v^���=�H�����]�
�����x�9S+��Ub�����Z�fm����������:��.����xf�3A�u�J{n���t���re�V��V%5|�GD�-�Z�j����Y
�4�B���=��=k%�	����q�Hyq��T3jS	=��[+�����4M�>x����W����W8wUv����
������l��]R��.��(&b�5����*F�-u����V���hL���F���[�������K����X*^D��pL�2�(������)b�)Z��y�l�]eR��2��(&b��1�ma�`c��:EW\s�]���P��#a��
S�HU&&��K�����T����9Q�JW����)���#��j*�H�M��I-h5��mq�:���6+b�B8��6�cO�p�K8�%�PW�;�'p����>	A2U��[�~��O��Y��������	�	�	�4!��	�9:�)J��eM��Zz��`����~�����LE�K�x�T4E��!�ew��q�p���H�v
��o,���G�w@W	9C�����6*���#�$�����n*h��OeT�7�v��c�;���J��vH��Pq��qE�� p��������������l�~��������D�i�^���JK(������;�������T��O2Mt�h�*�k��+*��U��5W�tVU�U�E����U����p�oU�m�l��ja��b�Y[����,������A[:�Z����&�&m�k��%V�j��hM�P=�QU�p�����BGV; O-�c��v|3��U}��|���gK���B���j�@���V���'�n,�v
tP4PK����e�B'������:}���������8?���������i��n��q�������N����F���
2c^F���e��N]:e$���E�L�������t���t{x�p7�k��U�F�� z�9LYM������}WfX^��a�[w��9����O�u�Yg�u���������0����0R����~�Met}4��N����XyV��9�%�"�*�I>��L;��3W�,��Y��(�}s�|O��{�?��K����7@ z�������iM���V�pR��X#f8I��������:]�;~�P�q��1�#�'�b��&���%���X�CkCq-�e�^�u�c�=�����$0�E8(���#��4��CkMI-�p�^�UW��T��q���6�m������tg�������)��)��5e��8��F��z�@�"�����t0�V���2=dY���U ��LH����{�_���U�V]��Ow�j*�y��.��Z)g_���eE��Z����a���d�U���&}p��\�V�`fzD�L����a���(��B
.c'\�\��|L \:�)��-�~t����>\V�Z������\&�z%z�����r;Z�\&N�L�L��e��c�8E��2��\�3����a�hM�����&3����b��m0��<b����I:��!�9��t��[��dK����#�
��+��jp���2!���`��8}��� XefNy��(�\�>y
��M��qb`R�
���R�1��%(o�P�,E��t����)n�W=�U'�gq?7����7<aq�7�Pq���i;�VV5�=v�������&��p�����G�h��j�Ut�
�\j�ZK�^?�]��n�Z������]!jTo��~	q{�����z������D�9(p� $\�����z�rjAD��fdT�����TEH1�����yH����:�(��0��)��	�|�CJ5���K�&�H�OL led���\^Ua�����������m�����NW����L���"��������?����|�%)�
���@�>6-N�6�H��m��(�U!"�\����b�jS�w'2�~����PP�Il�QV������P�]�w�;�C`���:����@1Q���L�uz�=�NE��s'��%�^�$���g-���zy�Kr����9_I���w%h#��)W��u����>Cb��DfD^:%dc����rq�gD�U~��A�!)G���r�z�C�� QbQ~��r�zC���A�$���C���;�y�GH9z}������Y]��[��v�?�%n���A]��K��b��p�v���6���>����\VK]���e
�f����
x���u��M��<�
"u{�s����u�h
�IT���Fbed���$p�{�M7�N��
'�D�	Ep�e�8�9_�^�~�})i�I�7G��K��a!�+�&��K����b*����p������.g�h����i���.B����"����H8��?s!���9T��~g����������b�]e�Y? d�c,�O4�Z����C�-`�qp0����[��@����X�P��V*h��H0�-�5�!t�DWp�%�Z
�2mc���&�7�w�R�����'
������	&�B�BI-*
��K �� d���8o�1���x��R����#�?�ms�t������4����U�g�H�����;Y1�x���"[.'��2����z��:[|q�)\�M�s�^A���l�E���jAQ��*��*�����]�#���P*���lo��wkch6Y����c[��?�y���|��y�m��U�u����YG��?/����w���t��(,�d}v
�R�p6g�u�L��.�d��}��y4��z���+�������j���R�����K5xp�r��&�m��&�^;CY!o:�J�� ���}�=��j��Z��������}{����@��2��7e���W�wT��WE����Z`\a�UK�e-v��`��=��`\K�_N��){0����WI�YA^�_ev�k�q��W-��o���J�������f���Sq��j�W��:.�;�9���q�Y]oA���tv�.0��Zs<����6c�:O�NN�Wk�����mLJ�i����j�q������
=��x��wO�;C�)���#����}��Fh`���u-D��0�z�
l ���-*�<*�'���n��a
���($�Q�(�<%����\�%v���������{�����c���O%�+��j����DO�����Z���d������B��y�_7������)�k����&���$�5oG����L�	�>���~�2���Ia#�%-��Q������S����u�U�W��~o"2�(=�H�@N �����r�&
^Q�X�;4S���G�0�AE���WT 2���4��)hiR�!���c�C3�4
zd
qT�+R"���5WEb�j���UT >�U$_n���!yv��?��y{xx����(�h{xl��/����G��o�Yv����d���W��M�OV`2{-�i��[50�
��9�^�>�����t��������]����$�>�{��p#��ziS6�3e�����+%��#�j��Fi��7Ut���"@�>��j����w2�+p�� X�hR�t!���L8T����*!�����N��y��`Gva;�?1�s��?^�~�������JAoBeT�U�]j�b{�z0c	�3�N2���j��|v�h��\�U���zD�����#�)�O���':�Vl�?<9�	��{�,}dj,K#�3<��������V�J���������gk���*t	��]3���)Ww�!��uK���<B2�E�����4�YTAW�@�E�I��	]�2�*�7�kXg#����A�5��JC�z���d���j�IK-��f����&��s�N�Y�@e�"�A�_�I�W��.
�.������I?P���;�mz����u{��P������[7<4���;��4~;(�����&`�����N=�
���Uk��,��c������I��-�R>|��>X��|���� ���^��;H��eN�3lI�Y��2')Um��9�%:�r��B��t�@=������QzS�Wm��d<	��YU5�dN[��bNJCl�=Z��1�N�F^�j%J�\5�zP�����A��U����������1Y�i������c;`����$����� �������q$I�������g����a��nY�Z�V�RBmV����W����P����Ve��������b�J������k�9����~�Lr��������^��^TYy�[������0���o�y����������?�����G�d�o
�j�t�H�����W�~0��8k�,y���Q�i�%��*[�����|~��lSm�u���J&K������J��%��_Z���e���:��Q	q����a�
S�:����=�w��:��RVp�N'�����]a~�d!%��Y��\GxV%,
����_B�7���A�C�)D��oX�C;�J��zz���r�x���QN�z���p�v��Q��;����nGy&��`G:����[@�9'q������"�"��j^>F���j1A�U���!��(��Jf�����.������|�K=D��+4��LdK�?����!���R}��n$<�Jp�|�������2��V
s���9�J���h�>_�������p�3���T���`eJ4���i���.��S�*�0�Y5��'��tz������7�<=�R	�K���==E�iOO�~���������K��;�|y���h>�����3Z6��Q�1��v��U�sG������!Dwb����>u�Y,��/0"��b���������|�����<���x%����P��p����{��~U��+�� ����DTE������l���	��&�8�~U&�q�LUj���OA�&���O~�o���$y���
�i��$Y,?qd����T�n.��8�a��������:��3b�Z)�]:�0z[���$�T�h�a�L���=�l;C��i���o�P���3��Jd�anl.��R���?���:���F��5�������*Z�6���K�F�����-��c�lA������?'�cr@O��AE
��l��5l�.�.J�#��'�A���6<����������6gQ��x����su
[���'?u�|���M�|.Ri�{a1����w�]�w���D~w�{7���B�m%[ct�K��R6�OV�l�U����<��fI��~x�^T����~5�q�u���y�I���g���3�33f*a���T�@|�`�{#5��~���eF�P���8��d��\	,�QB[�f�����.+�%(�
���q�E�n��c��%R�pP����uP��N�
���4P��_��Pl��6A���`P,��YK�;`�q��5���yk���������tYS;A11@1��bR�I���/��`P,��YK�;`�q��5������U"�?s*p���J�����9G��
��1��Z��aD�����-������{{!|����e�.�Ev;��1�Y4J��!��2{-��;�#��*��Kt��Z:e_]e�l6�����j2�l�~�Nyz~�������"�J���-�H�����f�rE{w�
��k��J����K�R��n�\����ry�Z*V���q7����aUaD]�I
��#

���F��g�o�;�N2���Z��	5��t�q�VV��������7B���KU����F��t��x���#�M�F�
r�-f�1����G4�<����j�����s��o"����AO�.���u�MUW�]��n�\%��������i����
,E�I5%|������-lTP�}��`Y��h���4c����\�ko��FeQ��+��`����3��p��l�X,w�s.���m�+��[���.���f�t�[�Z��0o���Pv�����xK��u�>p�\�7)0QQ�7)Aq�z����������.8oR������N>h��������I��@�������@8o;������y}����du�����~���,�_���c4��9��G��f�>c�|L�H�~9�e�}����Z;k��Z��u�����{)m�t�'�Hl��]�^%�^�b<���f���Evi��)#!�����U"�TW87�^�U�M���#��&��<���U��R�����|Zgx��%�#Z��eI]���m�tV�b�����J?��F�hW����N[��U���� {���`D��3���|�t��MN��� &9�1�y���f�s���C���n9�-�����f�s���;���n�8�)�y���f�s|��3����9mw�v'�~wR������u4���g��4M�(��v�-���lMf�r=�����
�'��M�s�^-�^���Y�n{��6VE6^e�U��n��k����v����n�*�(-��w�2�.�H=����Y��*U2t��)�Y���Vh����[����M��1���x�x`}H���,cs�\��������������k�3��r?r��
��bg7(���m�����\\��z]��_����t��hE�ga����I�/!������)<e]��e��f�%vs���9�8��oOM��ux��R�Ic���(~SKlj��XwS�nLc���es�����tY�A~Isf*B����&������HJ�.��@E��d�S��G"u�1�U���P�B�U�:�"��H��\
�K4�+��Ji*@�=��PW��h�m�S�����PWE��B-���*�q`�^zCU�4�A8�H
C�S�)��
�Uz���t0���}��t����������G|%�����H`EbI��!�ryg�[(��db�H,�T�H�r��,�iM��:��5�[����z��aom�z}M��<@o���,�L�$�Kg���|��&����h������7g��g��W���W4�����4���a��r�^.��K�r��\������vc���n�o��^�����K�����.�.1�K����:}���&ll�d��=�sO��U �U�*�s����8z�9��Qeo���l���kF�!|����WW����)����
B�!�L�=����p�P�Xa#w�<�Y�`��rI����[�����F�EZ�kHW�V�T����������d�NB��\y)�L��h��U"7�qr�V�M�F���R�ij��G�K�$M��z�-�@[��8�as�j?���5��yw��.���8
59)������\��:Xy�
�����&�&�����k��A���xk-f����V�u��|4����P�a��DT��q�C����Y�4��DG�ZtU'��}6��dQ��ikgD�(�O�J����{M�}v�B�K�����y7|��G��X���i����a�U����'K�_En������KW����p��y����x�'��O���Z�����Z��2+)��U��s����*O�;wF~�T����i�k�r��#����vdce���s�9Z���;�y���4(�I��C���u����������S
t����r.vN���QcPU���D����Q�����C�~i��T3�7���}<��of����:�����.�z~��f�)<^*��J��2a����c�ez���,l%]�-�C�s�//�"����U��Aya��7,/|$�>VAsK�{*\nya9u9���m��p�;n��r�-�;�r�N:��W� �d9���.k��vC����/�P��~~n�%.-h��no��i���'��-u���tN(��\�Z����|y�����9jh}�����$�2�q�*9�Z9S.;�*��2�J�'!�)qU(2)[�
�������,f�:�6+.�<���`}r���`����U�)7��S��2�~5mx9�>6�*z>i������s�����$clReSw�ZsP����j%?�s,x��?�c���7�T9�c�X��c9v��8��c9d*�f�?�����������:���X�0���`}r��������U�C37���re`��>�kV��h�c���k�6���r!r�����s�c9v���l�pWk
�[�X��Y�X.p�j���;t���r,'���X�r�:�s)R[<�C&���lV��c�q��*\�X�n���l�.��
�_
�'���.�]�\�sc�,W6��c�f5m���>6��}�s��z,"��Z��?,'�c�R�M]�k�Aak�9O�����[�Q5��}����b���0_�T�V�>A�5��\#�a�������6� �k6��F0�|aS�[IcnF��l����������z�<�%l�s�e7��S.�����=�\x�`|S#�@�8upN�([���:[��-�2�	���/���H���[�/�r|W����	�+
C�
|�3O�N	�Ei���eS�LUczLJ������:��n&����
r����Xo%�M�<<�p���qr����;r�p���0�}��j��)Pk��vy������
�ba�U�z�{��g4pjV�����69���J�C�8d�����OP�+�%4BC�lY��u����V1
8b:
FL1m1
� ���n���j{��f����������RA����j�sd ��f���%����w�;l�����Yfu��J�W1
9b:FLC1o1
� ���~���j{��f����������RA����j�sl ��f���V'9�~&�������H
��L����k'�Fh'�2h'�h'�
h'��g'��g'��g'��g'��g'�����g�?/[�u��\o�r�K)^.�_  �6�e��Z�^��*��m�Zn�Z��q-�A.�
���*U("�����FZk��0��X���F�f-���!!�����|����Yn�bo#1��q���k�/��
�X���L��|�3��z��UnX<���_���>��h����wH@6t�6J�6J`�Q}a�T0�:8Q��<g}i�81�������g�N��������VZ�y����D7�����i��)n�/F��pZ�9dI�/+��6��v��!r����6��g��T��F��F��F��F��F�SW����*��mP���U���_�m��hcq��(w���m��m�;��������?kt�o��5��WU��D����P��:�5N�m��m�<um���n��U��6:�������j���D�F��Pt�:f75N�m��/V�u_����W��W�4��6���l���lC�m,�^-h6c3�2��|�����W1@�=
�W5�������J��e1�����s�-�
g��+,��{k�'�V�{{�j��R����E�V���,��p,��w@�6=oj��������*�|��WY���x[�����Z���!?�j���?Jt�h���^���h����N�Ev�-��8[F��2�M�M�^M�'�_��e �y9Y�&��*�v�������������;{<����u����\:?�d�rEp��4x��W�����kLR(�j-��1��7��k�R
|����t�a������]��l��:+dZ5q%��^���=��/4gK{�F�t9B�Xu��]|�-W�_���N���\���Q�����������"Y������-�,"rKma��R�= ^X�ISk����b�pT
fH=��P1'��I:e�e�n�94.�
�V�pT�*���c2!KW	���'��R�#��E�P�C��Vm���@�S;lU�����-h��U�6����,�6����j
h������� ��8��~�\������[�7���#v�\|���`��7G��9�l���{;o��ysp����uw�����`��7���9��[NFp�����~���,#�<.>F��qS�y��n��3�E�Y��`��.��������Xkcg��[k�������<-�
�n�d���nB�V]��IIZZ�Z`�t���*���$�R���Ry�}N����(#�|�@m@ &A�S�6(#s�M�{��KC�nXX�A1������T�P�bD�V�<���EH�x��Xx�a�Q*FT�������H'o���.�3
g;����f:s����1�n�3��9���f8s�����1�nv3�{�����f6s�����1�nV3�w�!N��TW���L��t'�������:]^���O��3����K�����O��k��KW������2[�WP�2�p�-`{����������/������ygQvs���(��qHv��[�t��}�Gl�t�r���G��Z���������M&���t����9�nl��5����q����?�V"4s�Q����vm�D�bpf��@�'v����Y),�������K[����q�5���a����Fr>t�/y�PbL�U)9���_�$��>��[%�"�����2k�d���i��WQo
%��k�}
q[*�>��]+��1z��+.^q5��s�x��_�">��7��Z��F�����7��N�����Z{hX�����Z{h������Z{h������b��� UX�u���r�lW��*8���u���9�=g�l�h�G����Uk�=e�-zr���du�����z:iV��z���KF���zDk����D)Wt����W��W�,z7���:�t	Ud�Uvx�m.7��������U�b*"�����Yl��:b���,�xLj
r�[����1E��o�m�hW��X7���o�(�����{�=���*�j��*����V�\DVg��dQ�"���1��=���
�z�X/����F����+����.���h-A�d����7�� �Z��.�y�p�Z��"E�]������]Joewk6E?�|Q����l�w`j�5��E#%�U��
{�z������#\yI�^�V�c��q����������Z#����GP���u$�k�Xk�Xk�T_���5H�� �� �� ��I���JAH�0�sb�Z�������<*�yc������*��4s�=��Jl�&�;�J����fx�`�)���Sx���1 ��k�k�������N��;��Sl���:%�u��b�-�����od(��	9��B�C�?��@��^�,�"�����7�%6P�0^B���.�0�6�b]*�X��pqGp�Ivn�{Z�&d_t������4;*��Da�63hlC_dn#�
X���i�����9B������1[I��UM�����>HG\����*A����#�i��2$j��E�WJJ�������<��"������~��51zz��O��R��r4#@��]v?u��������AY�!�����q._����"Vrg	��~Y����$��\s��F���;����8���u��Enl������t�xr`$������M��7��[k���p�*:�-��a���B���:�������5j��u7���u(���\T�G���{$,7�$wz�Q1�X-�@@7�� ��i��*����i�`+�|�F�:cC�SI�c��ne�{ A>#�$�Q�>���L�]�Sy4.�3 ������)M�l�i��5lj,����t�����@�L��� Y&rw
��u��e�iMv3f!*�)��B�)77�k�l%�sK��.�U�}�Wq�����,��Y��g���b�>�}�,.�g�����>��}7�g��B����
��@�P}���B��&�,X��?����x�>�s,m2ld��p
���\&G���%�;2Qc�,�
.�fY�&�e�Wtl������91�s7��Qw�����V���p�/���B�[��x�����O%��~?r���
8����m $�9.�uv����,����+���r�����|�!]�f�t2'.�]+\���*p������1��d�
�R@��:�����03����.�9?��"�2�X�����
��;5�wj�����Sc�N�1�y.����Q�U�`����O�-:���wj})@���n�oty�T};5�|{��@����"����)�Jl���BWQ�oJ�z{V�q���s����45�3��6���jALR��s>	
�+�����w��k��ar
�`(�P�'KUHu+�_�����O}	<s�&8��N��('��]��~��a����n�>�JR�_���}Q�z�,#��p�`�by���V��I��V�����Y���J��J����m��'D�SC�S��$������,g�N��R�m*��wv(��\l*�e����%�������Y�Mx�6s�p�3�z.��8��yk����������l��f���(���m������X{	%������to!���H��o&��D�&T\�����i�����7��F%>G��z0����bL���� Vq��]5`U����4�3�n�_�V�3�����@�g����#*�:�7P�s�r���LP�I3��g�z�Lx��m�t�,p-n����?lX�N��g��0t���s�z*O�0B�����w;0�/��S��a����L���;	NT��G��}T��G-����_����
�FI�@I��PRH�t��l%%�*eQ�2 �2�5i%%��1���a�c�J:CI�b��U`S�&��3�2&��M�$`��Z�O�w���6<qIO>�`�%t� ���N���G���y���jU���	�t��_�����~w���K
�������3�o������)Is�L�B�*R7@�x������������'4��l��������D��(]��e����+����g�k�!dv���
C�����DT�DU����T���jZ*~�b�YL�,��i��	�b������.����|R����.x����.zy��$_.��Jy��v��T_G=����z�)��3O��zI�x*����	��x*1��{F�)��}�la�}�]�
����Z�O��U�-]�������[�rKW~�t�}:��p����z���oz���,����.�U-�Ut5_D7oL����lM'��������={svz~��~���A�����d�b��n��}�_������7�/�_�uds����sF?<qVG�*�������7�_N_�t�Uea�v#�'�>�{����`T���Qx\�nQ�q����jtV����{7����O��J�������=���Q����$tVy]G�~�g�����j1����O���NPR���=��Q�����go�����b"8��t����6cg�h�c�=�Ay���l�-�Uv���l�a%�'']@�V������o���6�g����������]:q=s��P
��|��s8�V�����X��~���#�Bq��by0�J�X�:(��k��|YN�F-$�����
���j�9�{�j\��^������n����6�j���qM�{����b����gVZ�����5�j�����j�
VMrX5��j������n�;�j���������j�i�j���4��
����e�����6����Ob�$<IO[�&BY���.�k��������/8E���,��N:}(�`���� �������
��U���7(D�-�S��[���7(��-�:����z��������SdE���t��h�A�&�$:���0���BWr6�^X�O�����(0���5��
�.QnT�4�t�+��	e,�Q�;�~R�i�+Q�x�t9���N�8�T�4�tq6x�{�:=���{�::=�:����R�O��
���'0�J��~�Hz��)17��#�
�QUbm���b���Yo������t�v�r!j���W�'��pH������F��e+�O���������suj�w����������"�
���.8��J$d�(t]�-���C���t��ems�A�V���l����grE����V��u���"6���n����c8O�Z��QN��s��
;\�L���\�B_"�/��O��H�Kl�K����C��H]�k[�}����F�Q�L9�!�,������}}�6�
��7(�>w�Z��!o����>UE�3N�L�����:�
��o
��6�
��7,�>w�������\�r���{I������~�SZk�H��~������S�,�z�X�O���D�?��D�c��O��D?�F��!/���yGqi�.�Zoa����e����������e���fkY����fkYs����zd��:}OD��;}$P�:}O�;�9Z��S�s�����&�]�;���}�iah���6]%��h�����A>�L�����t�@���&�]��E���b<u�C����]��
���siE��iK��oe�J7JW���/3�Q7�Afe"�^jd�RiH����H�9�h��nv�Y[��l��#
X�����K�]g�}I,�]F1 X��2-�K���V���5���[��[k[}��1�Z��u��&vm���\�ss������q~[������
�o.�Bh�Tes������`�n���i����i����d�c���G��V���b��2�g��Fv���2Re����C��\F����-Zks�d�._�J����oos���]s��\n�?�Y�*mss����Vck�'Wl�\������+v�\���2^x7W���-Zws��\�MS�������w��U��j���i����a&H,�T8���J\'Wb�\���J|'WR~r%�'��h��U�(��Bh�����O.7 X��o����m��1kZ�mn��Z���cl�!��J��+q�\�}r%��+��\���
=��E�n����`!�i����=���`�n����\����5�RS�uTM��E����{�Y��?��G����7<�,��&��oxdY�
�<�$�����+���_���-�7��h�2d�N�_e����g�����<=?{s�rg���������S�A��3��N��%��4K�+�M�8��
���DB{��L&bp��D����@b��8��v���H����6}��O��I�N�"�������N�)~]�\y.���!��\9�w�������Yh�/���]ef��<��L�L�t�������������E�Z�fm�Z�����a��l.���x&�3G�u����$��y"	
in�){}�K}���J{�B����,�	%C��Oe���0�����4��9��:C��&��m������4�G%�C=<�5�4���?���'	=�XI{=�����&����8*�$�*�*�������D��X�0��3�a&�#��adl�Ua���
�\T�b|�m�7��\��F+t��@���m��.3�A�/Q=hj���'�H����������s����1��5�����t�~�1]���]%L(�)4�(6�%v4#=dQ���
�5�tI	X�'�fDL�0�cY[�te���]��`CS�cf0�2h���M
0�r�inT1[�����^����������}8�v����
'�p�*a:�����9+6"g%v�,=<V��s��������MJ ��p>�4#b*7�����+t\�@���m��.3��A�/�Z=hj�������|/c�<���%�\a:�z�_7�t
<Fe��1]��heLW8y�sW	�
tL7;00���Mo�6��6�����< )�k���������s,k+��,b�:EWn;���w�����|��ASL��G]������.��
���c��!���f.{��l7so�+c���+��J�n(�p�c��������6lz#�e�������%I	X�'�fDL�0�cY��t��Z6�)�r���������e��M����m��\��K����o0����~�����L.;&�{e���p�?FO�'����r�Dy�c�c)��a?���r�@�U�*wq2���]�S,�A����rXjy�c�.��a����r�@�^�/g��a�m���-�����-����{�v���%�H��U����tkm
@����y@m8��j���[��h�A�+�<4m
@����������6�C��tkZ��������k�<4m
@����������6�C��tkZ��������k�<4m
@��[K��%��`��5���-����;G;��t�-���u4���g��4M�(��v�-���lMf�r="��@3����&�9c�1]g�,z7�uD�������fh*P����n�H�GC������e�����2�.���+�Z4<��Q�w�G�
� ����q7����a��{�I�E~�\�_��h���)�{�V>��������9����{���q,Qp�������$)�I�S���U�����Inw�l?z(���%K.�$�l����+	���D�`w�E����v��In�c��
;"��)���
��a�����IZF^<�C9��	�&�
�����1�c��5�^4{��X�n���yn��5�[cP{��J4��a`v2Q�G�����(�@i^��[�a�mFJ!�2h@���|����tk�����<N��F]pR��f�����N�#�q��h���Iz<N���\p�4Nn�18�4��Iw��5N�F�8I��r��tv��D�	J�L��������1Qz�5!�����[��H>�����?����[��m�V���c��#wl8����2 
�o��9`��t������r����	����f)�mh>����;F�B�k�>F���FBf�����b����	v��&,ym���,jr�_\��,���5|�1�������&��+��*C�
�=�v&��J�+��>U�Q��D1K,5N&�N�P������1�PV�3tUI \��XH�0������0���m]R��`K.�r,�V�@7�	�6@���P&�
e�UZ1��I�Y�B�^VA��N��2���eg'��������/Lj$VNh��9&���&��S�@e�I"�$��I���	M�0�d �d����	
�6�d(�d(b��/����t�����NR��S�7���?<J�V\u���������N�b��? ��r
��6�.�Z��v���K��k���Ic���B_�w���<��^��E��a&��
���!�a ��q�T��u��^����U*����������+�*Gm�Y1�����:+��q
�O��!c:���_�����w��_n��W���r�G�'��TX�4�i<�X-���Y�`b��j�Pk���3�t�W��H��I�F^���X�rb��������=��r��:m9u#�&�w�-'�l9���:�Ro����a\�H*��x���A�<|�X��.��X����VK��X��q?��:��V�|�i�3����d2��b�[��G��e���j�#LmU-S[�����2�-��2�u	��1�	�.avD&�V�u�d���~����h�+�V{��Ek��x2���D�0��b8�4<�X-�����@01[:�J��f]9�lI������YW��q�K�{]�������t�S>�:�L�:U`&O&�Y�o���3`��M����`b
�j������3+��=��;�S�<viX�D<8�������]I���*���u.Yv���,	��k�>{��j?m��x����[���m� ����p���j�'Qy�`�U�[%�����'i�V4����p�_Z3L�_��_vejP�,7��/i@�*?i���K�K������N�4oUN>�g�r�����e�6�m,��%���PU�[�\����3��0��[wg�	���
��/e��j�VE/��[�kK�G�����m[�����44�]��	���zp�zp�6��
����/�4oU�F�{�r����-���{�qqJ�o�y[���9~�>��;�l
����u�X�����0l�R�P����[��R��������&��6��D���\�e��|�����^���M;��
�����y����0l�R���[���'���Uy;5.>�y������.j��*u*�m�~k�|���kf����*F#���kn@�*/~���KY|I5o�����g�>z��&���+�$����W���e�w�=���*_�zxw�L�W��
W/"oe�,��E
��e���|w��}�����y���z��j(U�A�[�y��E�N-�fS����Y���Xg�i�D����6_����1!������j�eS��yQ���e�q
9�!����f�Q����$��m��S����N������}�M[P�����8-��}�����1uozL��9����������\��M����4���l�p)S�e��P	jv���c���1�<���������]��[�agL��?��r�f��u�bL�6p���L���L��#�������E^�$bX
}g�;�I��*�,W�l������2�b>S�x�
�B�X��*�p\�r��;����y�	�q!�I��]��B�����������q���AU
��%�O�����
A��s�((f�rt\9
!c/�8�&�Z������{�OT`�Ll�p�1=d�zz����F��������5ZJ
W��l��� czL�9��������NG����QC�^q����t\�������^�zqf����b0��T��������,5^J���d�|���?c���\�C��d���W�n{�bO��|�c���A�;@.��9��H���R�V,���R����F�������������}��j�2��pZ���O����3�v����*��	�
2	���t=��	=IW�*�	
V���l	��w	++������.�H����v�uo��}��/�`��[L������I��������,_�.��
��d�/��1�7��9��/�������
N�uX��	c����6c������.�X�1Z�	�n�cR��%��+$�M~ic.��h�;$��87+Oq���/�1�����O����;�
������[o�:$�^�O
�G��15y\8>/�>�
��
W�Xyp8^��������4@�RJ*>�:�d��2�����x���������gnr�(�ro��2��M�[�o
�j|��Q�4
�Oh��K��u)���i�{���p�
B�� �GA71�����?�s�E2x1�A����l��K/. +����2���^Co������r:����e�"EO_�k�$X�m��7��:��?!#F��~4���.���t��BGg4����5x:^���~<�
|<���3���~==�������$����<�?�o?�}@N�~7�&�Vb�V�!lc:_e@��.�s���t������q�"��7����Aw0�?1�[�>�:_	��*	�UC�XJ��������V��9�K)s���>fH�$���AN���1����Y�����r��.���
����������k4Y�W@���p����]�U�W�,���
~�\�U
��P�-"�N���C��~tw����w���p�?�!����[E��:�J%��H�������
Ete��L^��DTB��O�B������L�����d�T���Y�]Z��L
f�v4pO�x�JP��T#L���!C^S��
���]N.�Cn�!z��c�;���s�;sZl�i��8-���u��;��J���5����z��I����,�w0=�f�}��b�����x
>�OH�t�����?|�(F�4�������
4�$���/�G4-/�Pf��D����$�a�B��K[��d���i��N��w�q���xKS�Tpov���{�9���m�dx/ScUq6J8�9��]k��l�;2�
6�W������"���/g��E�������.k��&�N����hrXX���������w?A� ��Hs���c&Abw���7���q��\��Q�$1���15�v������\'q�:��/>��1�p��3��$�j�PGeU�;��>!I�y��l����$��sI��L�'�@���Vm���;e=R��.c���b���*(��xN�C�k������o�����%�e���������J��vl%�{����Oi���.�#���������}m�L"���O��$��	��Cp�g����|��x�O�!X��M��>����y�G>���E��D0
!L�<�2�(xh���2Q��sZE��w�9s��d�����%� 1w�;�tCG�-9?-��:���;���&rl]!�R|�Ju��mIp�k�����B����F��C�>�7�$�b�j���
1�JV��a�W-^T���E������Wu��e��a	
{y�����BD8-�e�V/)����Q'>6��_��^/?����CPn���/�K������&�w���s��{x����
�]���i�+��8v(7a,�8���T72�nV`�(A���U���6�x:�Dqlk����I���q�5���E��}�H�y 2������������"���)19<���p1����
���aC��I������d��e!)�.����JM��Zz��tZ@Q���HcbL�WI���:�����eS�j���m�S����`�:iU�Y4Z�`Z�E�	8�*Yg�F�p#�:�5Eu*K���
v�
LVU�lQP�B=oPe�N��m
X��U�h%�:vB�"!|��������`�J�����yB�<1�%�e�3(��k�p���Jl�J��`���^;���*Zu�S���3���R:�h=����c���z�L�v�&W�1k;-����#��U�����?���w�?���U��+��������qQ������[��.a�L���	kE�n8W�y�8�Wp�������@�z;�1�W=ip �&w��-���H_�G����:�!������������Y��/^���LZ��'��B�`(�#W)d�O���1��8�Ls��Y��P>3�,) 3�#�?Y/V�a;)����5�_7�F�9�>��_={��E���e@
u�@y�:�
e+��A��+�,>���0G9��C��H��jX���f
�m[����/������3�4�d�8:����An)d�&���i��s3Ej1���y��3����LK�p3E�kh��C�g��s��3%V��1�"��3%)a�f�m�L�v�)	7S��3%)��)e�]fJ��)�vf�8�~f���$shj��D=S����Xe�J����-N��?b���A0��&�����A����|p�&	���P�����d��{�}����Jn�By@8��Dv�Gb�
�%��u�����Jn�B�z��O��?B[Ag|d
X�1��?��i�Ob^��!i'Q
e
�~����`�&f���)�	s,��&���:�����/��l���*�gKX)k{J��7��zz;���,�L��,��sy��n�
:Yl���w�8)E�����@����b�y�!���	��(^X��4��pl���fpJ��&|3`�'-`�H�P����5<�n�c�N��������S�K���s+��f�
���p�=>N8�O�/z���}����FA�w|��[�������Ev����b�Qh_4%x�R�8�24�cHh�~��%�V
�(@��hl�/E�,P�GA�~P����E�:��*���r2�*������Gd���~9�8����c9W���`��l�Fu�;�s�[t���HY��w.W�����.%����S����p��_�������J����*:����R���n����	 2�}{���j�;Y��M!�Mv07b	#4Flx���i�wlM1��8m��E��H����W�<�IQD��+y����E���6��T���p��e�K����:3:��X-��J~�s]��W����4�P{	)`�(�Z��'%���4���q����bS��s
z�Ad��bx�w<D�G+ ��xCX:_O#��z�"M���fw��z�#�g����\�J	��E�����FX�K�����55�M
ASl�*�Z���F����c�YaLj[��,���p�n��h^"g�����:���V	�����9���f�|i��f�WUE���I��t��?��
�u���`�:����Z�C����c�W�URj`���3�F�X�_o�b:V>�q������]����"Y�
�Pq�
#��(�w�h��W�S+=�Xm���K\Ren�X���������b���R��J{��U�������'��x�*�*{�����@U�xU!�r(X�,7a���z(�Z����T:ak��p���=0!){�������?{8�\��4��8���
���1��M�!6��)��&��7���Y���Iv�����Xi��&���*`�K�Lo2(��`\�|�enV�b ;��Dgz�`�1f��}gD��t��Q��\��*�q*������0���.E�V(���'��u���q���%)�X��X@��6!'"��Kf���L������P�L;����J&�:�X��[��9��,
hf�
��Y|��M�X�x�\}bXl�Z�Y]���]�L�J�+�K�seEz���n*V��EJ��(���V8�~DC�P�	)6���(���+"\�+�X���0P���Tf���).�!��g�Z����q	���x���J���hPgSrK�WX������KeQNAG!����pK����T]�SrK�
�&�KF/�����]F�����K�eN%�$��Qo�Zh�R��f�n���	8L��>��������g�P���	�)���!����&���x��E�������}�(\����Dw����7���8�	r���L
E�f�f�D�rW�R��1�^4�����A��UrS�-��T8�xJ�bAl�2R(�r�vOh_���L���B����Z��/V�*��f�b��;�
��p���+�E��Kx���/�$���i�83^��x��V8U��)�����5���1�9�N�f,1��fLN��B|�s��9��*��{��fiU<.����a�Z��q�s��.����f���X�\S��	vY�jb~�*,��3;�P�xf���1�7�{1�l��l�V��m	!���M���p�X��p���e}��]P���N`{�3� |�3������jf'���@�������1��JK{�����?5�Z�V-�>S�Pi�q����! ���j�Q}���45=
dZ��*�����KUaA
uN��*��Kt�������4GL�KT��a��I�?qb�>��Rz� M��&�*O�D����H;''%����}�B��r���>
�&G�h�zf�1�����u�q����z�6x����Og����9}����{�(X",m��9�m���a����a1���;����Y��y`\������|V&�y�>�@mWal)�V
�
����:v,y#"��<Qe��^9���r�N�l�^l��9F��uN/p�����z��K'.K�W/Y�8
�n4�@�i��-����^\@�A��$]e+T%q��,Z��	h�B���,��L�i�����e:����q=]O`.�����4���a���u��^����O��������W=��C1�P��|N
�4�����_`\�z�p�����v�����<*��*�<*�_��}]��J��J���Q[D��~�#G�zL�7fT��?p���V�;�Sb����q�E�#A
2)�2@�����;���e���P��>V��1\�����$*)�7���U��\���	�C6`������}>],�7~~��?`�0������}�E��a��
� �B�p;���(��3�y�gh�&�T��t<��&��w��^���g��Z�aZ��<m-)��
�7�U��"���;�f��_6�������:�a����*w�i���[�C^��>_+1J��.�l�g��_J���8[��������"Y6��eI`?+?����^_��#����_R�&��^��Z���.>����?� >{:��Fo�����7t��Z�z��]^������:
we[-44�mx4�E6���
H���&{�%�nGDtuU������x}��%X����9��`[��m�hM�V��Mk�y�����G��P�����X|i��]!�)��k�������OBm;�3#!����������a��SYn����aO�$Y��,���M��4`�b�{�2O8��q�T�S+����GJO�4��A,��� ��� �j�
b�=��K�P��i.�^nf�VK@�">��k��^�9��Q�6-��L���dbU3��Aq�z���,��R5���0y��*��=��Lf�$U;��/[	��������QE������D�M
w�_�y
q:�U(&�I�1�;��������J���}������W9;����;%O}\z&.�S��'�g�vJ�hXzf(�#�`���
���'��$�r��������M ��
TBQeR<�(��KTcr�bN&��)O��q��q�1S�����U�������]kG���* ���h"L��^.dM��$S�����5fjn�N��%��6V�n����F�����#u�YA���0��"�����@�8�/�:�J���Hy�a�]v��wt�QXn���y��6��z���*��fwS����d������r����v��8�{��Xy�����ayI=�16��g:6&~��6i���N�t�N�S��Gv�^_��ie|p�^R@��,�h���Ii�zMN�0��>����8�d��������IS{�����m������������%�'��.�Sq�����*�p����*(�9�c�4�a�����k7�)8�+���}W����G��^HR�<X!��`��#������!m�0q^+�Q�2#�>/����=<z��G�@Dj�Hf<z��B�`����Vc����
m�d������M�����m\r�`�'�Z0r��'�\0��h������y��^���6��xV%����^C�t
�!:���&i���K^-cb��|��y����I4�\���h�~�_
+�~��vju����m5�Dy�`eh#"U�{�������?�T�hj
%�T��a��%���O�q�g��Re������RM���I�� VM��}����3vS,��*D�� �0�*m�i�	fG�^dK59d��6{Ad5�,n��y
�h���P�b���%��j]���f��Z���^&����VB��cUu�����g�_%�]!����	��A2
��J3�fZ�J���~��$���� ?��<]�b��:�,G�(Mh��I�����5�d�j2E���/S�@��f;�ugo�E��#�Y�D���=�3O�j���fm��������hkd���Zv%��-o��E���y���Ty=����XE;q9�k(�+���v�=��a)�k��2�����J����ZG��F�P���s�hG�xn����:���eU�Re�t��� ������]5�M�po[m�<�A{2�{��S���A}R��6d�����mA�f���!�+����������L��?��T��Wr���-�R��f4
Y���u	^��E0W�������C��D�\oj�9hAm��6lA�
V���,E���6���K6�/I�]L���Iu�CU������������J�U�� ���E�Um��h����5�erN��K���cX�BZ�DwJ����/�xTUE�����UD+�:���i�3Nf�t�����u[�&V���hb�|�V��T�7���d?+���W��	�h����:s�V�1�y�5������
aq��D+���|�����l��4df����O/J�l���}����R�'(<�D�y3�������W�f�)!�(�����X��P�=R���$C���L���gJ}UJ�-��Q�{�~�K��=Q��s&����
�8gb:g4GW(q�S�Z+
?T-�5o/H[�q
�������5x���<UAq�_�S<UA�2�2x��(���V)L��S����Pu����,��k�]%����75��
PP}���|V�g%�����
>��������7��*u�����T�\X�l�����
j����<X���0c(�K�rP���n���G�6��Z�)�z��V�kH��]'(8A�	
NPp����O|'(8A��
����{��0}hj
^�L����i�e�!$����"�M���z
0]��Y��7��y�J���/xy������!���X��Vb�(02����pc"�+��������k4����������v,O�@��iW��TB�U!2���z�W�����e���2q�4��I���M�F�.���*Z�b$�������� �O��L����+V�]F#Y��H�
Z��b$���K�D����R����R)K[��������}F�)lO0�g�3f�������P�/{�����-;���V�fg��L�w&c6����uF����ZQ-+�E?�2�H�_\{�������k���*��Koc������.S�'�y����:���~�����M^]�?��j��H��:��������Y���Cx�pId&�����c38�B�����^����F�8g��9uEh���42�mVmz���H5�����$B��082tz��_bz��%����h��"%��[D6�eP�'�3����
�30Xc
����*#��,�+�(_k~�&�����.B�m����S)n�c&���_f���s�����,�/q�h'*W�X�!�R����l������?kw4$3nh�������9@���@1]���W &�H!B�m���
T�0�c����_Z�U	�sR�����,�/qj�Y*W�D�!�R����l����d�_��gn�o^����@�X�v�^�-ZZ�'���d.������mKM���m�]��U���tiK-�\nDs6�����Z��d��
����Ai3R�,�RE���Y�e6
Y#���f�~�Pk�����.�A0��mY�J��&�[��6~#���T_!�w�k���^:���`���v��V@^!�x�����r3�@�x%<>���E��l��K+�7+����5q�q��w���o�G�^m/�,=m�j��f$F#�\��?�Ei�0�#�2�|k��Zi�"��!I��P��	_��	�������Aw��7�j��0:}��,������c�0h%�+D?�@����Z�T��/n|G#<�0�T���W�X�d1g��B�
��h9��uPNE���86�Xcu���H6]g�H�(�XU����\��L��U	^�����X������a��u�����wL��g^F?�������=7Z���:���,�Z���������CuB��^�C(]���N)�0`�������j3+�c�[�^I�?f���S��SpT��)3�-��B�fb���������6�����������
S��% F@��\f, ��y�����h
�9�D�/_���^ tL���q������	/0�K�?��`�;A$��12��(�}F����H��2�!.K����]����K�G�f�t�L�2�e�������?���Q}����;�����O,���(FGQ��^����_������`�"q�:t��ql���H_)�>���@vn�qox�Qs�C����A}���<+0k���*l���������x���*�������xx�\ln!\���3 �i<�{��AD:4p�TK�90�������(����6ml���*Y������'H����ev��3�5����<�^-�g~�W#���v�
��c�N����-m�&7�a��~��1���|�-���x��=���m�X��H��cZ�!�!1<����8�
���"o1+bT(m'G!��.��@�Z��&��A�����z
���?;�9�����M��X"����}�yQn��������og�>}D �mi]=�9c�����n���m��}�����s�&��Y�-�dk$c6������je��1��>;}����S��~��\,��5��&�t��l�����}������?�-�&�]./�j8����.&��/`-�H��!��d=�@�@���%�w��n�s
�$��K�Q�Kcd��y����\pN��2�jKq:�K��	XW��S��"|���s,>�2h�#��G4��!2m+X���s)35a=pe�"g� ��[�3��T(����Yx���uZ#w_��
�BWj�z��Z%�jA��
���������&jO]�%x�{^}��^�����~}
���@"(WP ���@B�((rD�q�F�|�D��c(��'��
����Oo�����Wo�{��r��t��T6��)f[�7������~�N�����6�����/8�i)���W�g�;���w�>�y��2��\w�������}Lq�)�
��6�S1��#������t�������G/��W��z��}�IB�>1BQ�����O����P�~�t��Yt;��Z�>{Qp�.�l�|a�0p�`
�e���<�"jj�_����7!S���U2�<;�y]d��9X�2u"s��{���������]�B9|�<B�����~t}��I���:z��/�>�����1Omx|�9�l%	��[��i�Bbh~�����RH%�q)����Do+7���B#?Yf)^��u��52����uQ���=�����&���+u���O/JO������@����3�����c=��z��zrHP]�V�9����s,�\����1U�|�{_�h�A[� ��Z�����R��5�_��F��Uh�����������%W���K+q��k�b%X�_�l_N�����Rce;��1M��m1x
��)^����.�M��`"�J}@��7Su��E;�������i,L"&�~�E��^�WH���Kr2�_N�/���2��~����/�R?�/�;�c��KC����&F�`�����9��������{	~(�%�!���n%�-�)2u��p�i�������.��\������=��T	$a{�9F)�����>���wu���P��e�k
s0+�A3@]�#X�ub�E��f�E\Kp���8!�d�@�������@���]������z���e%�DB"�;�~?w��8cFXc�����W��E�R!�18������r-6�IMH�At�q����$^A~�3�O�D~2�m���u���[��T�������z+��rzu�6QJ�I=6�'�0�� LaJ4S��� Ly��a���8u���Z&M	 ��F|�pd�S� HAjw����Q�"F)*�JR��
Q�+C�Z��!A�}�N��+�%`��(6T^&HE�x�aR}��Pn%��@���c����s����[�������������s�����=���g\z�<]o���k�:��<�M �,
��T����s�L�nj, ���8�������yV���3�w&��]"��������`�MH�����}Y�m�[�+{�f�J�VU�s����j�DK��8/�^g����s�X)�C�����c������o�d�+������/MA�"��(#�d�V���E@"���=L��n�Z��y�_$�q4,�J9����Q3�����Yp�;��i��W���Xh��]S������$����\��7���.z?�}��nVp]��f4������^?��2�CU��Z���A��v�b���f�Y
���]a�X���Z����=��W;��|m3�����6*w��R���6�x�1_�O(3 a �<�HL���&����C���~BY��H��	�	��_�O���}Kx��mL��,��t@@��g�r�XoW��mi�����p }������0}��W$p
�Pr�7�tc�oK�H���Q��'9e��Q���FQ�g1���`��C���|����E�G�^]��?Xr��^,�k3�+��������W�8v�%���%�������IB��\P��<h�����R;k+����C���/������������������5�%�
oJr��!9P�(�F��"�8�e����no��n�aUa\��kTj���Y����;{
G�SnK�ZW�8��6��I��:��.�1C�u�x����@��^�z�]p�XvWH�9A�8,�X
;�wf1��9�.��������Q�_V�gD$V��������cjb� )z��gv��bw��[���(�v
e��8�f��&9x�F�9�}d���
����!����0>U&���9x5��8�}|���U}v�A����/N��:�xP���s�m�l�`�?�1������r�A9j����KD6H���7���H�c�����l��H-�����&���x:OI�{Z.����������%�h�����h���?yu�t������0N����/����Y0�h!|X	��p2�+�����r��@���(�,`��..��}K�U���db����'�%U,�5����r����+�'������{�����(3�I�x���Dv@��$�������|�����=?����+���A(�k��Z3R���[���Dm�A��4D�Jp��D������/�C������(�=�B�.U�kF����bw�����2������]	n���=<�{G��~H{[��=�o��CQ�b��f�nI�� v�;��
(��-i(�����;�����w#^����U������	z�P���)�nN!���A��ha73w[J��p8��p8���p���J�3��4'�X��U��7��VG�W�gF:�DcE���(�V���<)�}8(�{8(z8(Wx8(�v8(�t�K�MT�(��&)���/a��JC!2&������6DAm��2�
J���`sP������
;�Z����[��5|'�_Yqh ��d�9�����(�
�P�A�pP�`jCPt�<@�aB�}�:w�58�������k�r9H�y-���WP ��y<(&p�,�

DP T�B
M]����}�CS#d�H��e�7���D}%"(����D0%"(A�J���)��Y��������6X��x5�7�.�4��WP���y<�&p��,�
*CPT��A
M]��}��}�7��a�B��6t��i���(����_�m�\O���:��h5����FJ$�wp��+f�6 �'$�%cl6~�hu�a��B7����@/����@v(z���N��X�q����k85g��x_�2�7�8���]�AU U�s�������� �H��5��~odR�@�HG��j�M�I0`2a�n�B�F��-XJ�W=���;.�������U�z-�N
����r�"��v'~VL[��b�f���p���;T ����V����"*
rI+(�_d�m�$���T�E&a�������������Tv��j�]��E-/2�� �
 R���vv\K`�%5�z���V�
}]�Hga��V�6V���j���:�����T�d����������}i��b�]}�K"�q?�0Z�d*,c����el��g��HIq!��%��(![V�;[���Xd�;�.��W�7������,&)^�p���9��i�������bw���������r~	rS7��
as�K_���n������N������c���������M�.��%\��$��U;^id&�Wl.
�K����p�8�����p	�C8�:���BV%0��~6�A�;+��t��J=�T�zP��@5�TC�zp9����:����N�N���\�u�msE6Wt�u�\��]w�6WdsE��i
�A��z��;����1�}�:Y��*^r��+{������%���[�����Xn��_m����:���mdw�B��o�����B*w���%t3�
������r#��r�����.��))�R.����jY��}���.o��k�tYxK���x�jJ�������y�jJ����*���y�svw��V+ ���X��q�&���'P8_	����e��6[����*��]HrA�cPP�aF�+Np��z��{D�{B�b��"��@k��I.�Q�["�[fN�Z����dK5���B�������������������h�	�
�� ���>gBea�c��v�H�$Q?(��b���utxpn884���
��#��i�N��r��~(E�\���|�7wH�	h�U��'�cD�e:��J��p����G����9�����������_���}}���a����_�
��E|�[�MA���������0Hj��K�_��������*�$jGhF��r���1��
Q���2�t7����_�cb��Wa�l�e��cf��`cB`�C��$I!8����b�h������5�=j$��lh�\���[��5,e	���`Kh�`�>b�6�}��h�7�v���1�=���.��W��i#����l���g�gx$_F����9=??�����Y(:�z�8�(H��������������t+�7NW�H�U�UvM|��^-��%R�2��$>�����!=������C����=���s���e"�����K�*V��F�0�0�
���\���_��5,������_fj�1�Hl�D^�d�n���t�Ub	�0��������l�*�~���/V��/�~�IN�r|��~��P��;�D�u.���gy�U����N|~��g$���bsq��]���M]��F��
�|
|&.�K�����N��b��m�`ct��h��K�O��WW��*]�V0���M�	�jaIf~�W;���f�
�7��=�Z���z���>���>-�O�q����D��%*���	�J��G����D*w�		J�
|�s�-TBB�
A����Ao��m
�[��*z����_��~=��������}����I���tX"���}����a���q\�N���v���G$�KJ��J�t�5A�d����W02��Y��S�!
F	�m���b���lSe�vUBs�|��X5���Q7�]�)
�]�ni��b��]�f��T0�?b� 9$/��B�y y$���(H�[ �
�LArH�]9��������_�"h_�{��b����1���Uf�^�������m(���y��g����B�v1��|+�E�]�������^�/1��w��E��.vo_�=��oZ�E_�����������f��uf/u�P�Gp�&���
r���X���h�}mR9���������`
��b;���P���Gw�1��
��%V�QA��8�*��N�`I�D��~���6����b��__A5�yA�r^����WA� �9��s�W&�;Ao/�����l�+X#�/g?�y���^��n���I������^�0B��Rc?�}|�st��-4�-5�chR�:G=%�YZ��{���(-�rp����P�����7���_F����at~�������/�@w�����/����<r~�#��_F��M���d��W��z��}o��G��8F���=L�o���,������@��W��g/
>$�$�
�/���B�QA�*���#/�63�������#�����0
�e@%[F��tC���sp�[���l�����������z����}����o�C��k�`Eo~�������={������3���XV��\f����^��x^�yL�%�8�k�-`9��5�|E9��u@�����zL�����zB����������,�k��J-� �<�r��R����4Zekd:�����Y�9�=F/ud���|��v	$��n��h�-��~|Qz#�����}�_��E6�����^g�3��9FEt`D4����yD��QT���op6".V+��U���pJq�17`h�0��-�:\[+�{�=������JL�n�F{';a2��Y�GJ�02��mJL
����2�h���8����6LIE�U���	L����S�r�&0
�0�T��j���]u/;+�zL���!�T���hx�2��kX@���I��%((Pj�(A9��r B�I�+
�5����T)�K����#Q�����=`�U��>s?�g"Ti����l�U�D���V���i���f^���\c����b��t,
��yd����n���mA��@'�����6q��:�}�����Q�J;Bo�c��~Y�U^�+�:\�����*�qN�|�PL?*�J����Q��
\Y��,�|��� p@r>e��C&�P(��������'zpG��i����Q�e��o�E�%6o��f�K2�0��|*�Ox��%�s��%�"��]��
��|#���G,�&�"jB���hB�M��T�LHn`B2�Md�����
�	9#!��%'w��h�����s-r<Y"=�����0�4H�K�5U���3��LA`�S�<D_jV	�"y05J"��I�)����{�@Y*R���������t8��y��9������z�+w��SzO4���<{���K�����SY*+9e�%����e��h�Y�L�)�\���;��_�����h:J�KM�����K��`4��bG�q0����	b}����a�9�J�g�Z1�����#E���?A��:E�v{����V{�m�6�0"�1��(��8��D^��p\�������mPK��2��{NK��U�I�����5��i=}
�<�8'��FG������q�5|��q��j��^�z0��H*�:�e�o�����Th��n�g�+t/������KG��/��Y�
j��6��RW�a���O�V��L���<�|.uU�zz�9�c�������
�A�K����Rgc��
;{�k5E�9�-cCJ	-�9��Ek��e!"��10�TF����*fnH��{ :������;��hK)[W��xg��-���%���d�a��/�.}��N[N���`����J�L�9>+��7����&o��&���&o���+oL�
��`�	��`�	���6�<P�:�or���f6kI��Js\&v$2����7�w�����MC��yT�9Z����TF����;�+����i)A�f�D5���e�����Ln�y������-�	���oc�3�����	|��t�N��geX�����p��z�v����'���[`m��b��5;��2=�/���|]X��,K"_��w�|-5[��y��l���#�a� ���_o+q������~��������v�|��$��#"_'�����&�X4^�e�K�P�?�4
�z��O-3��O7K������E{���$�~,�(�@��.�@��F�Du �j^�p-'���!���>�M�W�S�j�K���v��oI��A��Rv��	��vK^4O��e:�R��mKUG2*��,kCUs6(��Hi3��E�v"��(730�����A���T���.[�k���_}�R,�/Y
?�Ec�����������|xY�j�V�}
���1V�[
u�ME�!��E�O_bf
�R/B�N�nb�7*�&�pp���Le9OUm��"��������e�Kz�$������C^��%�|�h�i��d����f�}���n|
�f��P���ffP��"��� �j�j�n�q)������nl�4��o�(�nHvb�)3��E�7W���x��/-x��:aM�]���
�0�L��m|05�/0�l��Z������Q]�aB�5����0!��2�$p�8����N��\]m��>�^\�.\��a�P��OS�]
��V��3r�-����w��|�n%[;^�5p�1` �����&3d�������9����`n^���:��6�h}��Ic��U���N"��XZ_u?�D�jO��D����e�p2���k�Tk_���p�K ���>���X�C�D?h U�y@����63i�8H*�<�AR��
�2	�8$*��|�y�@�����<v03��I9����ac'0��D�w���U1<�2>��:���+�&��>�n�	!6/��i;�y!6���������}^�t	�� |�|I�0f��On8f���c�On<f�4b���{��9q~�(�#'�L�@��xV%����^C�v
�#J�4�-��uHk���{�&}D*�:4Q&�gp
���^�0�B(���9��H���;���L5�%s*0�6'�P!'�;>NX�BJ�V|��A��T#l�8a�
	�TX��P0�[�FtE���HN�L{5����M�xJM�"�8�S�rQ��N���(�SO�ZJ���hQ����2�!��
)� ����q?	�1����)SMP���ARTix��O��z������r�Z!Y�������e�c�~&l���	�To�Za����P���M^!���K�3��������U��nz���2����~�Vg��/��EWMl�9?�.�MyJ�����y�d��&�$;?�"�KoI�p�0d��K�f���8$DHN���T�4�Jt�N����J"��Y������,�#�J���b����&�$;O�I�b�h&��&	YM�RZ�p&D�L4N���bvq�.������z^�2������m�������������>r[�x}(]���Np/PZ{�m����#��U
��2R�\�P����*|��Hk�����C��XYi�h�a��r�����W#�R��a���/6R��RU��N�Q�R������J���
���U��U������m��y���`aUKy.lu�>,�N��{]��|I,������2��zR"���UOJh�����IUV������
R��X�=3c0����`4f��U�'k�Zc5�a���l��2���P�4��}���������P02���!�+��_�;k�cLw����N�������/CTRK��7gI���2[��+XRC|j�����4���������	v�
��Q�OU>L���X}���`h�F5��������{`	����r��#����?
�x�}sV����=]��bU����X[C���s�U
��!V5���XU�`@dO��W�U
&�`B�AL�Z([���vBn��+�P�U�"\�����-���Qy�{m�r����X�3�d��6�I�d
�m��,��Fas�Va�@���J�h%n�H����8���M��9�Z�!���`C~�A���YC8�9��@�sfwv��Ili�V�#���~)j5)E���U��*���8�kW�bl�w3��1uw�����w��������W����[�e��p�Fj����j?"�w+�N-�;5M�0*8�|,M�K��M��v����m�F;��vh��j��\
L~MK��J��$����di27��7�8�{l=�M<N��N}���)���S�|�d���l���l�������
F������m��������/G3OZ�A�!h1-��E#��m�	A�-��������;-jZA�!hQEZA��@�����WV�`�5���]�-��\��$ �������H-�!|Qx�F����y_4�<�/j�����R;4B���y)�/V��C������A�k����	G�wm�i$�qkSA��9��
��!�13�`���C0cf���vj����C0cf�B,3Z� @f���h0c0�C���
�bhY�@c&p�������`���`���`���`���`���S|�w�~�w�~W!����(�������Z����^��1
6���*Ni�2]_G�����������_�m�\O�S@�u:�e�jzs��g������<��D����L ���������j�-���Y�s��A����2���w������SH�G�1M��i��=����j�����
������986��
�m���"!�E.�W�����c��U������L�5�H
aH:$W:$�tH:$����8a�,��uVW4����c�B�f���'Z�h�&���G&������� ���k�|���K+|a|a|\�O�m|4���X�'����<>�d��I�H�M�6�5�-�T*`I����>�W�V+�8��F������C%���EHL=��������$jF/�����>��-���7'xs�7��]zs�$�6�i��eF��
�w�����1(=��s�
&x���/��*+�cq
I�������<"�`�U�Mc%����^^��	a��m�����G����q
Q+x����%]c[x-W$r_*y�!����<�
t��R�?�E�)���I�����Pt�R��.�x>LjN��7UJRSU�A������e�c���I������������OH�'��������g��?#�?#�?�]]�?zn��s��s�������o���j�x�e��Bn���[�Ce$��"4T��
�
#5�
3e�����$A�Hq�@�@}*�a�DI�U�H/T���b����L�%VT��$�PQ����K����*�X(�W��
�".DDe�b����
�7�r�>����|��8e����w���Io=o|T�l5(R��]!/��
��z���1�����-A���K��@N ji�:
�2u	�g�g�dm��^�.���������s_�~X�vo�Tz���y��}k����������#_X�^x�{�%|��Q0&���N�c�H������ECaa�y��9������X_X�����a}��R�/����r�}V)V�k�v�/���ov1;!�{S(��%@�����K����M13�17[b`$.bb*.@0O�����K��1!���s�Pu�7�����64�H:������@����!��M�%����s.��@��x]B���-A8�'�I3��K��%(����pg���+�qwz=�����n��j��R�W_W��&���'P�_	����e��6[����J��K�Q�o��3����G��=��Q�=���U�u�<�%��ko��s�����%�.��0�� t��������0����V��c��1C�UA�XP�F�L�nU�<��Q3�[P���j���2�~e0��v����xro����K��e:��J��p������(���v��������5����c��>���0����/��gEym�)�)"��������]���u:�X|9��2�M���� �	1yBcd�� `/>�qh=GLU�����%��-��*��k��-UoM	h��pk���W���lmp�H/�HS\�����XK�-�k�[�?b-q�p���^!\�CYY�S�@R�L]9�eIAkA���������������E��7V&Zf-�h1�g�\�_���.Z��;�<������r����������'�v��������z�.0>����Hd%��;�G��Z��Y���4�V��L��Vy�L5�V��L��Vy�dHr,�@����g*�8l[u�?#�hW����A�N?����A�W����\�M�r9:��OT�Cs#�X�K�K`�UN���J�;\��#rzu����u&S,��������}��|�����U�����
�--��c+8����VplU�[����[����";#�<�k/����'�f�mo��rh������w��nS�j
$��l���z~��)Pg���L�Pr���]�F��##�7�G�'�=��B:�4:����OR`����!T��I��?�{�7���&xo��F�=��0YDA@���Z�
�,����������NQO/x��r��V�P�����*��,���:8*���y��g 7�4������d��v1��|O6�d�^����Trp����n����0�i.��K"�����������N� �(c`�n�42h����&�`���u9���k���s�<X>J0S�S6�[�������(b���G�B��<�v����#�����$r<����$'��Z� >7���(`U������6�����p�~}
t����+�pA�S]A�:�XA�:����N�`���M~J\+	����l�+X��/g?�y���^��qO��������U�iF;���P����W?G�o���p7F��w�_p��U��^��A/���������;����r�9���>~:�0�%�5%������
��G��~$���e����=zA��x���g����=TjL��S~������h�?}:��,���^�~��(8���S6L���2�K�����.��QS<���/���'���_%��3�f��"����j��l�nf�t~��`��t��� E�k��2�0�����}�����:z��/�>������t	L��s&���2���B��!�*�/1���0+K�u\�<,��=1r���p#��,��r�:�V���n6��t�}�f�g���8����������#Q�;]����`�~|Qz#����@�������h��~E&��a��z9E�:�>:}�+r'&w	-����(�	�!.�P����qn��.
���xn��&I3��������$7�Q
FRS������
,|o�[�aP�2��b]	{��:x��%}���V�`�uJ�\q6�lT6��W�r:�	l��b��*WWM`Kj�F����\N�4�m�6D7��x����`*a�P�6��4�M�++5V60{�s�������}��?_�]#H(cW+�1��%/�R���q�Z_���|��9��'h�����n������7h��-���v���*�a
d1M�N�P�K��z����$'C���/C����a��/������9�}�Bi86����1�Bmr�@�|�G�S5������'�3��Ohg1�,�;�K�%yg��*����VP���gR}i�<^�f]4�J��N�T��/��x/������`;�eG�M��R�A�K.��1	�v�>�Ez�(�$���$��9�	�w��;��[NE�\@)��*�'!����pU;�+��k� ��@eA`�S3m��������(%@u����L�(�h4�> �A	FI �`��q���Ip�A8cn��a�����n��+�v���	i����p��B���a�T�J�JH��
�.�G���Q��2�(����(�u�����s���2�.��b�n��+�<���������k?�b��Gc��g���O��������G
ZcY�����A����u8��B33�����|9A�^
$��4��)�s���F����b����|��cr�eA3�76\="8j8������Up7{&=�����Grfy\��.������-4[c�������[]/G���n{���tU�,���}s�/��#���>��~c�H���
����hb��������V ���y{��YU���r�rB�?��?�?Qs���W<����e���~s{�1�F&����[�AG��M���|����,�(�8���?Ps���������-zE�����k�������ec�.�q�~�
]�@*_"�)�Y�R�r�?����a5������]qr@�����`���#����wv������]�G��s�[}����+���c��*�I��=����~���������{{�������IC"���������Q���Y��-�ms��P5?�J=�h�y�Bpcn��!�Q�Xn��-R�Cpcn���Cp�D���A!��������uz�T�\'v�"�k�
��F��9:�@��_!�1:���C�ct��!�1:�@��+:�@��v!��:tt����<]o���s�y�UU5��^
kEs����2=o������E�pZ2�������%���kl��o\�4���gX�0����jc}��RR�/(Y�,���dV�zu����N~��#m
���1��m��=�����$R��f������^�4V
H
��"���R�����8'B_r9�����\�%�k������J�%U���)�Uv3u���;yS(������C^���n	t������U��5s5����OZ}g��-M��q���f��)8��T�� ����e�����c�����a�Te�����!��Do��3H�:���\K"��.R��v��l�z4�%f�]���.��bL���Q���h���av
�{��!S2�����f�� 5 .,ZhH�,SeWH���URT���*�(�E���54�1�,�(mX�05���T?r�^��-�aS�a�L�~��eYT�VG��BE8��&i�~0�v��6,Z'Sj������k����ha�0e�e������E"�*C>T��Z�|4j-S�Q�"-��)5k������E[�)�)�YRr,:YT��
��X�2.�Q�����m�J�mX�p���R�E���0�t�������:�Qci�Cn�?�p�)D$�<�:����R��3��������w��|�n9�7s��j�����h`�[>�2����/[����)\5�we$��Y��
��x�����[Ep�������?zYLl��j+�eo��@�8;��AK�F#f-������L[v��
s%���s���7�yYdl��j���eo��@�8
���G�F#V���E���Yv���V%����������j�������m����������4&�<��/���i�?h"K�	F��V-a�A�3kX�G1������$�(]�U'��/���0'��O:�h�;�8,��f��5aSk�Mc)�UU�.7H��������B��P�7�������{9��In^j��i:��N���da��Qx�E�z���S�^�'J��}j�9��N�m����N�=����[���P�3PEy�'A��� *�O�h�\�F����%��!�G������5�2r8M��t
���gU:1z��5�N1|E��T4OM��mU�9�D���K�b��������F�L��#���o�� gp������b}���������I�.��`/UJ4�#�
4���\������~H�$5}�jp�q�A��uZ�������L�:wZ�=���WL��uZ��T��F�:wZ������aE��%fV8�5�0z�=�n%_�0PYtl����4�%W	k`����:�����z��9�z�sU���DN��n�\������S���0W���0')�W������k`��sQ���\���\���\��^c8�hl�p�i/!~8�)����x:O���|oN���
�[����')���?q�X��+��H������o�����?���cXqLS�x-�:.�L�q���e/������)�Y!��d����Xu�*��!�y�����V�:&,�o��	sW�/)xi���������N�;�������60�C�|0`|{$.��	���,I�'K���_���2F��D����2���1���9	�_:'�u8N�x�����*���'h�FS�*�0��/r���0�����F��o;{qKC����E�e'���0OR�TVbV��[/z���Q^�t�����.zn���^�Sa�#�/z
�(_��y4�V�CCt�+pX�dz���E/!��C*�=���-
���,z�	��'�y���%fC%fU��*���r�u�M�[�d[���X)��B�R��J3a��rg�P��i�T�2��l�i���9�����l����^|4��
/U/"�p�
s��7:I�^�Pl�c��������(< ��r����D�i���+8C*��	�5!�3$8C�3���	���hgHp�X`�!��!��!F�gH5
��3Dep�g����!�m���qD�[D�YN����)u�_���#Wp������1#�1R�Wp��H]4�c$8F,0���	���	�#��c���1��28F�cDeS��=���G�HiH�kD����f����=���4t�H�#�1�&dp��Hp�T�#�1R��	���c$8F�c$8F�c�����A�sp�h�����@��cd�����/"�Hp��C���5VHk����8a�d���ki���.@IQ���:D��OY�*��+gD�@D����@D��I���*Oz�O{T9pK����y�
�GbZ����r$P�{�1���T�cFO�;4>�V�Y�*��]5h�0�J��)�t��k��PYD����;����W4�t�1A�9����3JO	�����j�F�
3��������>j���3jS	�!A�9����
��U]������nU��,�MZ7f����w"
�7����������������.W�!.��,���^��b�"�4qH���{K���)����j�[�	 ����I&q����yY��$>,�$}�X�����������������a5b-��D��\�z�sq��=��8����fF�m���jL������BM�\v������=���=����{0����`x���+���]�F0��{0��{�`x�_��������z_,������\P��)���`�	'����+���=����~0��{0�+��w��������=��h�{0��{0��
���ex�gS��-������j2��L�@sa������&�F�`�&�`�&�2��L���@#��*��
&�`�W�L��L��_(���7���>�`4�����(����`�o$~���b�VW0�3|0�3|0�3��`�f�(���>���>���h3|0�3|0�3�}�����z������n��,�����{�W0��{0��{�`tF�`tW������]u�{0�+�F�`tF�`t/��������������S{K���L����i�e�!*..���Et�.���r��gY�����?�Y#3<$�w�M�cW�l"�U�"�4�)f���Uw�v�=[����l�F�^�=^`|s��8YiJ�����}��X��<����[�
�:��������&�������|�W0��f����E|*7�� 2.��C����!mM����Fb�zT�rX�|��0 E�Lc1�>�����)��b3���X�},�>��}����9�c���@������'��O�'�;�'h���>h��O�h�hhO7������k��ff`���	T%�F�m`f�a[|b�X����*����jf"��'�'�G�0������f�T���>��{��)�'�'�G�x��I���f!U���>������38.Y����*Z�03B��%�/ZSE�Ek���h
�����3��d�����g�����!M^��f����"Q��es��QUY���/�n��X�:���2P�e{�:��2��<��~d�S4�M[��5�H5����iNQh6�g����"�l���G�9E��G�[C��c���cW�t�=W��z-�,�}�#^�}�/Jl�=����r4H��iF�~��w��w$�jtL�R���x���C/�r���B�l9sl���������WI{d��&�1Cv��'T��MC��L�^,�C����X��Toi^�@�RrD%��2	�13�%|���We}p����V(N�l�*���P�%�lql�2��
	����o�,��a_�.z��p�{�@�8�ED�1���M&�)b6�0%���b$�50b62%���d$�Yd��"f�3SB�-sF��E��b�Y4%��ri$�I2jJh��/���k���3L�u���m�(Uu��]gt��B&����X���������	A�������g.�=#�=#�=sE�Ys��s�������"������s��zT�Qi��,�����n�^.�����������\%�W��\�P'k{���dm�suB���u�N������	u�::Wk%n�%2�1��8������;������*@'��I@'uS������Y@���p�%��s�s��*���M����tv���u�G�^P��zy�}�^�B���v �x�����|h��p\/�{��]�k�z��c[�O-p�E�������\�n����W����q��1���V*o������� ����a��&��+�)�T*�Y��Q�L��lA�(�e)M��������&�hucJa�������5FK�%6�L���t�2��P�Z4L��
4��j]����^��R�wDC!�k�0�+�P
�.hh�-�
B*j������L���E_yl'e7�����X�C��Pv3��e7%h������,7I(&wh����q��7�<���%y�Q���v�Xf��\1/��q=@�����C�%y�4���Ec��Vr��$o0���p�X����3/��CPY	�`�{xva<j�0dvw��V��b�B4[}]����r3�@��Jx|6�-���2]/�VG5�!E�>��D �?����&�G��=�8����	+o�������7k@ ; '@�����_z��y
.Q/:���W__����kV��T}�R����Xj(��
c]�2Q�2��^*��YR�a_��4�|�����D�q��)RNAae�G
 `v~�S����N�r
(� ��G���(�p3	�F08-�)M��P�@g�Sv�Obj��,c�y��Rr�DTL������������~x�������>�	�yB>�l����?�����~`;G_�'�a��������?�{���p��p�?�q�����O��3��g������/��0`;�O��9�>��O�� �����7��go����2��VeR�p����G����n�0z^�:��_�?�;f��g^F?����`%oO���
:k�Q���
�OQ���'������Y(�u:�X|9���0�>�fD���w=o�� �;G�B���L.N�KV���`cf�����E5T�l�5�vN;���U�N;�G���
4��+E��9)\61A��J(�b�X(�R�X(�b�X��t��B�}`�i��f�@��,PBA?#��l����N�#6�a@��{9���8���HZ�$c&J���+�bc2��;�y������WO���sz~~����"N�\t���[L5�L!h����	��0f�����[<�L��\|���b_B��^�������m���z�q-^��v,�/���"�t(���,��������a��&bP�w-S6���#�2���r���8@����`�L� �VD` �������#�ncL����O�i�
���j�8!1�@��\?��<��ZfW�:�)����M�	�j����G[8&T�$x�Fq!!�!�8��PB#�!�8B!���`�X�C�!�`B�p���F�)<��X4�vf=��`���	�
l7�����v�����h�7�|
��������h���t��Q��4�"R&��'/k���/��9������3Eg;{��#B�����y)\�FD����U&&%!�������21����y*�����y*�a�+�& n�<���x�{�
u�
�	��=O���r��D��]�s���O���gcx40R�|)��"<�

�1�f��/��Sr']A�5����YN��tE�h#�����)������m���RB9�s�Ps�K������%B,��2e���a��%e��#���#�[R���l���^���}���)���V\.���*�Y�Ye��v�_��o��=���y��g�4���6��m��/@����o����v����������q~7�3?��V�%Qk����z�_-W�
�l7SM�*�	_�����Ec�#b�h���a�&U�rtb�j�L����9�������T+ptv�IU���N��'Agw�T�����T+�t*����jVZ�
���g�7{����zR����\)-����/���#����+A�'cP�6G0�4Uv%������xMc	��8M���9�CAS%Z�`Y�~&+Cyt����k"�Dnw��n��W6���!YW>����c0����y%�W�y%�W�y%�W�y%�W\P
�	����+�y�)���r_����?�,W�P����������-�����D�>��!�O^9�����#.?�}|�st��-f�Rm���i�R��������G���`���������;����r�9���>~:���%��������xbD9?�����/�G�&l��2��+�={����#J���:N�/"4����~�=}��������W��g/
'����
�/�5�%$����E�+py�E�
�z���_�|������6��n�#x�]N�`	eW�n��t~��`J�t���T���eF�W�������[�s�������_~����/o�^��gr��32�`�R��Q(;#.,���� �Ly���H�K��������c�+�-���=�$
�T
�@,�����t��d�(�O�9�����,��@��'��g�������?G�_N��������+�2��w�@K�����( c�aA���������K�
p'~$5��l��x��y���4���)�2�7�B��g�?No�\�Lg�)���5N����7��Ft������O�1_���!��u��������C��������)����g�a���d�������>	s��B�S�9F���X����r�K�L����f\V�s8e0��F9�����%=!��D�rP�~_)�d��c�~L���4C��d�jR���%���^/0u��-�`N�%��gWy����K����'�J&Q#�`5����P���a��e
�%�X��2X�ZX���c$���UY�2r]IF�;�$�s�Tn
�-����2�t�P>�3(K�;��S+��gpKr������S/����[�����[�����[�����[�i�����9�#3��=p8B���h����rM���h=H�%��N�6����D��k��8F��Q��{�	�akR��R`FK�[H��p"�'�[�anye+]�� W9�F�?�RcQ��ROV0����tFt�RQ-����`�Ux�
 ������g7,q>,�sc�aq��������D��7��JL������e�T�k=��Q�B�pTsj0 �S�[��1������oC�[��6/�d�&�o��-?�
}pm���>�C��������
�_��������}2:f<�Z9�z8��|����qp9������Q��A����3�=���ZI8Z���������o�c�?�~C#|~}A3~l9�9.H8���\���C�[���G���>�����o�q����$�_2�a�p�s�����������k�1G�����>G��3�=��~�8
�9�N�9=���a��� �W�[���!K��7��K��"�nl�����=�����_#�6n��96�h�x��������:��D�l�8rp����-Ok����>�W��<�p�|���\�"n4cn�������j��U��K���p�~4�w<~�����d�J�>��|������{��H�!�&s�q{G��=�������<���8Z?�y����	G�!?��+
��%����p���8�������z|+�����
�;O]���l�g7oc~O}��<7�8
&�n�d7~��W~��������8���
O���H~��������:�a�g�	�����^��R�qP'	�~rPs�
Y���C���}��up����^&:��1~���T~Vq�=�WS^��w~u����s�Np�;��9�e}^��(�y��p�Wa~E�x�?��w[A"��:^"�Wh~E����{�������������=����x	��{�1�%c���D��y���?oy��K����q����6'&���a��ZI�Y��'7�xy�_�xY�_�x���/q�� ���������Z2��r8D�	�(�_j���T���qL\�-Aqu]�%��t�l�x:z#�����>��,�*f�)�R����Pd�J�����!/��G=��+��JFCfmkd����)}Ll�%�L*�6Qcv��@iY�9��Tq�63)L��!�/(KB)(���'?�s�KK����#���`AG�<��,�3_���|>�ME���g�3d���@���=� !�Mr���$|���y@��3S��\�>��-|P���y@�O�������j�>��H���k�3H��\�>�	R�<�
��y@�B�3V��j�>��C�<�&J�3G��j�>������3A���~9�"��Y��-Te�P����I3�J��E��G�"#��K�������u=����'9G���9��lp�i��v�������y��u�Z/(�Q������d&#�,OY�Tq��<�T1���gU�W1��v9�X��0KEesLJ�}�@���M�$>��� 21u��H��F�r���?{}i������^���f�����w�D�?{	z
�5�go ��;��8�	 ������; '�28�18���#�t�;�Swpl�����p�m�������"	�%��#8���	�L�&���]�[4�-��_��-�jj������-�H@J�\Nx����D�`E���2C�n]�:{Q��!�����hM��>�]M_���;�<@����`k�GU���M��_dw����+�~��Y�S��(���a�8����S����M$��Sb��B)e(���y�"uC/&�J	wXH��2��O|�L�8��~9Z+���k�:�s�c��=g34��������?9pxE*�;��2��,�Q��95��w�R�:�����_%@��U��h���K���U��q�Jx
�H����2�Hh����%J�oX������,$��s(z�~g��c����[]��X�P��,�|��o���H|V�h�h�@�U�oRmQ}L��Z��������[���t&��U/�]r`�Z*���13����b;��n��\�9����B��2%��$�b�PR-{�w���E���H��R� �Vk���7��h���?��5�emh�8pP
v`%CY�R���r���!0��V)<�"Z���K�l��t������_yr���)�����tR�-�cH����b��6u#��k�(�W�2	f&}s�vD_"s�)��������qY5� 'jY��.��S��N���k2���k�[�����Kp��U0��7�
���
���2�0
���~��`�5�-K}l����������e��!eB�KR�N��}*"�2#��~
���N��\��NS��D���K�%OD��Qo���%��*%��Qo����%����"��&�*[��s�������&��tD��t}
�t^��
5/�������8~��������G�|�Muh��S���?���/�,]����i/KL�1[��\��0Z�Hg�'!�Z==IC$�����_�^TD����`�:0�0?{�R�E�T���������K~8<��T����vNm��|��3�P���������L�2U����\��.Q�` �T�8���"i�dLX�8�I��'[�E�[���V��\ypW<���Z�1@�����f����
V�x����J��R���;���	Z%�����hQs����zq����T�7�s�����������������b$�����5V2�J7PR;����
����e*l�<���4.�^����g;q�p���t�'�n��V�t���D�b����cj����A|���Z��������S���Y�:�DO����E�x�c2�����~��t�|��m��%iYb���h����B���e~j���zV�5�����E�+��Q�I��|D������A���U!���j�r����_6G7:qokg5s.�)��<��������#*���2�T��/.�	Y-�Us���h���gv�cu:w�����+,�
�8g��Jb��5�{���=�a5�XS���s2�c��#���w��;C����y�
�sl��!qm��#1[����x+f�y:��d�h�+��V�����+n��
q����\��B�����2:����C���8�W��z2�*�fL�	����.�49�%�A��8��)W\'U�3O�� >����|���{���y����j����MQ���&���0�D��C�N���Gc�G����0����oQ�s�]��SY�����p���|���]������[�svF�(b�����Wx^���t<���H}�sT>�2*���U��������w���w�	���g;{��#��r���Z]7S��/�d���&��?UD4��;e����X0�p�$�PE:��c�Qg��0t�5�<�I��3Wg�n�c��[�+�/L��T�f`U���0�\b����"a����	4%-5f�[��ew�r�H�\�o����'���A�t�>�������g�o��u����sr�&z��j�������)��y�����j����f�l��\g�Y�
:��������8>yr2����!����0JF}����?G����{�?�|�w��l:I����j3^M��[t{�?#f{�N���5���� ���v�ZZ	/<����~�y�}YE�`�J����e6���~��F���q�n��d����j3����Mz�������b2��*K��k��b:Y��V.�u}D]|��nf���,�t6������t�X~�������c�>G��=��.!��E�ZG������,]� ��)lq�������/m`���B?V���J�z�F������y�%���Yx����t�����'O��a��h�=��>?�of3�T�w�U������pV���'�,]g@"�����&�V�h}��1Y"�m��h�A���4�2]����P;��@	�����,�f��3j��������K/`��:�6�A{/���xx�rg9�����>�}��:�p��A�
��7�L����-�P�i���t�@���,cX<3�kZ�����V0"_�
@�������bZ�r
>�$���Y
$�.�73��b3�@
����pc���-f����l1���O�|^S�Z�/`��,�����\&�C8boD��IK��H��9����$b���9t��E�_�BC�l�,f��9~�����?����V�;���T^�g}�,�.�Q�J������:)�OJm@)�x����aX������t��h�@���>�@]]�Jb���G�9����M�A&������������w�z��]���b���|��/p)zD�U�2���);Ze�i��������.�+MA���N������	_�Jw������_��>�x
Xq;�U�V\�:���p�K`Vh!\=������<4�����52�>��A�E�l�A�&�<�c��l�b��L�_��8��v��(iGdF����L�`&j~��X0.�i\b��i��b)�d�b,E�"�"Z�9�e�4���M���~��(]h�Y{�?c���:e[������v�xr"oK##8���8����F�o��t���h~�F���dX9C~���m�3��A�:E���������!��z��e�f�G�P�0��@C�
,����[�^��V����d��t��r�
TDV�w_g@s�Z@]��+�D6YW� �����^�:���P|�j���������Gr#F7j2�CDr�`�qBs���iQ�F�����8������2�4��|W��v��Y�Q�Z�o��9J�%v�v�Caje�l�,��-k;�gv�5
���T���� �����
�������C�E�i3�*�Q�l�:@o�u�������gy7e�m���v	3Y/a-g���l?�3k�@��������XX+6�-�WfE�v���36���l�����r��
����vuZ�7>K�G�6F��P�{��,t��i'KZ��
�(l��x�Qf���t`���;�G#d����u|D&�l��T��FmI�-dmU���;�#%FY��$Z"��Kd������L dd1��PS��rV��Y-������H
)�I������jI��b��b	��V�w�j���b�j~��d�Q��jb��"5��i�
pl@6�A�s���pn��A�=@�M�ldm�(�C�=��~�h��ZM��#�����������/Q=������`�]	��{�����%z��
D��S�&�%?�V�����<����G�C����*+��z�����Ee����(����m��rDl�O��l����2�U7j�y�e��L,������p7����� F�~z7��R��r��s���Pk��k������O�o������v��������8�G����������r�@a ��@� 4�`0j�s����C�(�ds����pSl���h>�h��Y&1��&�f$��h�������m��m�Z1kU�nj��Kb��E�A"h��)�����t
��jLqz��/./_p?��l�?�.�7�^��M<��j]p���s����}.Q<�P��p���b��X�\�>�(���t�0�0��H�f����b������S�/��������d��K���������#�A �RG#����W��#��m��`�����m���>���n������g��9���E{��B>a���B>�~�d�'<8:?���_���Y�O�Mb"*
~�Wk^��o,6w���y���$m��c�H��x#�T����Cs���2g�F��(i5o�f�u#U��M��7�����d|����E���"{����E��<UP���;'�rN��h������sb\�e��7�/H��rb����p������z�H!�l
�f� ��@�F���d�_4�>��[���b�p�D�3�C�<+a?����>@.��){�I~����X,����n-l�o�|����7�����(�qA"=B�
�y�Ib�����}H����*������oL�>W�+n��v�I;'��3o�Qi��-�7��P
p��f|����0+�-�vv�$Ay��IO���x��:�����\'9�C���I��P�k�u��@l�N��fk���8�Y)�eF�\v�u�#Af�� �v).��u{f��=*������X�>���3y���X�u��WQ�m�XJ��(��[0��+�v�x[9�0+:�{��sWeXf��?�z���WH��2�'�!�2nFB���	rs�����C�.��c�b��<-p��$�����_��w�
8�//��M��f�x4�����x3���:L����7��0l�U�a����M��=�����?�M�l(!�lY��=��v"�w7�k����H�����5y0���o���Udfp~k���\�2�?T7\}�x��n������Y�������Kt���������t��%f{@)��)h��"�����397=7�!h�����<b�'�����,kq������	��)4���Gx�I�s�~���B���nc�w
�57X��qn�gW�z���'���	�Hr���%�K��.���"������le7���[>T�e�g���Ee �Qw��m���]�dq�����*_��`/p��"%�d��<\u^�{D|Q0��Q/�r�EY���E�
iq���]e�M�P��*�t^�!	*�c�,hQB}>jA�9�s�����Q��,pS���IZ���m��A��nHb/����D>`�~�F��H�v_�g9i�_�h;��.;;�vB[��H``�$�_F�������('����|��-%���H��||�mU	3���*s �@2��`��d<��pM�f���h�E���l�]DS,��OG�t~������e�dD��$�D���h��y����	
�=��c/��(�}��h��_L����q�������o��U}��R%6�	�tH1*@C��x Q�#Z:C�E0���7�lz��QT������2���WNSF��z�Y?^\>^,/�%Ow���m,�Sen�B����k>��%���b8� �i����`z0ZzJ�i(��&~��#���bVs)B��3�7�
Z���e�����l��Bt��a<��w���oU7���(>�e��J'h+Sg���r�l&W�f��'��j��%�P;	��J7�](���W����JC��J�W�;���Ti���J	I8ALJiN�BI�D��P3�����Z��@��tC���$9�����eO�*��o�����T�����
��C�R*]>b��#�/����X�|����X�|��5Y>b��!%I��!]�c��W.�P����AC�SIk/���Y���xT^>b�����X�|���#���+���,���PW��k��
�u�u�p���On�6�&`��>���������:@�4B�r�v�F��X�l�^]70�b��S���F
��������'�xPYZ��MK���Zat����m|����{���Cdc���Ce������fX1L\MJR��
~!��_h��a�-<G���T���B��f��j	��5�E��Q����d#�d�K���I";�d�n��h�|�����2n~���l�"�c~�&���/[{����o�O��q;�NP�b�����W��>t����	�P������������C���C,���h�� �����m��$>+;H����bIe�XR�!�ae�XQ�!&�����L��81���Vv��Wp��F
��1�Rou�gV���f2U�����{����������HS����Y��.
(��e�m�n!\�rI�� �Z�T	e_�������@O�L��B�$EM����z��;��\]�c������D�2�{�Ca\�������g��`�(��[�{7)����=����k��oV�����J��Ez�7��'�EW�t�6��7��+5`+�M�[~����l�EW�h��`Q��Md�uva����
-+�	�,Su��n�?w���d(�o�%�t|(+D(+A�"`k��C�_��>0v�vr�}X"O~���*.��g�9k�����sG2��7w���y���r3�+�����7@�v^���c��M�w�?��/
�������{�q��|[�]��B�u�+�b,��	��$jL�T���i��0����b�u�-Q����1��	��8����L-�mg�X'���4�0�mmm��e���m��a�~k�&����Hti��2��T)�\G������m�6����xn�<��~�n�vC�����"�^�@�<�@��k�� n�0zc���T�5�+�5�]E�I��\�3~���%���G���?��qzq�]@�/��0NW((f3��Z(�<��;��������O��je�����$�K2a�\&]4 (I�m<fO��u>�i�J�w���;�����=~����&0���y+�-H��O�.����,�8*�t�p��4L��9���j�3�L��r~�������,-E�e�����9������"`�"`Gz���}�=�,�fv��pT����d`�����P
�^���#<-����fh�-j�Gt��bv�js��N�-�HV���W�$,Z�am\Z�`:�����$��y�o��>�[4E���GT��*�q���s����}.Q<����~c��X�o�������3���,l���/6}/��K���'��i�}����C���x�bnr9����,6R��{��|��9F���������������l��-~'m��6�Q
'm��!�I�{E�p���2��G��{p"##9����t� 8�)N-��)��� 8�Zp������t��d��������i���
����@������tq�|�X��3����p��+��D��B8�����3&eT�~����/��X@_���Z�$qV��]���G�����*�\��X�b.�|�{���-�f�C����QwB��\��[
�����������H=�p��!����RY\���:�0g���]<��n)p��,�����'��rZ�fjT	��HJ��I��{�������V����|��=�7Dn��FY��t���������S��H�R����R;�j#������	�8����>v$��`��j�?�1��� ��b���f{�N���I���rE��A`jx�����{��e=kZm�F�-NMq�m�Mf��n���lj��3o��h�c��5P�X�Q��	����v&YQM:�vD��U������w[�9Va��[�i{}�f��D���Mm�+�(;s�lU�st�����:�k���y7|}�Q;�zc!����mJ��Fog�C�[k�t��3��]c��k����]��5������:�V��:|�����g��l�������/��.g�9]���e'ZOd�A�Fd�X���N�.w�zTt
ok���X>0[����m���9�����+���������E��������
�hN�Q�f���[��+	��A��� ��E����dy���;�{��=��nH������.��)��[(�$�4������vp;�	���e��}���|]��[Wc��;���r]U��_Wc������u5V�����+�Uu��u5���
�x^Wc��;���r]U��]Wc/�jRZWcq]�%�j,_Wc��K��X����u5����|]�w����u5&������H6�g$c����c�*�6��()���a�(��wU"����j�������m@�+c�e�����#��,�o�
�����gfA��0�����B0�����oQrc��.�
 �N�V�L�w�m�$���eF��/�RJ�L+�^1-��9O�$�"�Et��R]�JZ�0.�0/�ZJ���4Qf��;L�n��#��a)�#���G�K.�T���m���<�������#�of����c�_Y�.:���K]�$iR9��\��N���&Q�Y�,i7���p���u��m���$������^����.2f	���R3�l���{���
�VI�-%�'�&�u�i�|�=A��I���R�mg>� uG��|�j�59���
��aV������nrp�K����99x{}�K6�yS��&�D{t���
X����6�c/�)�;���lmC��4w&qT��m}v0���������EsA��
�@�l��K/.�`�4h$Fc�	SU�\�d��,���2y��*[��p�������PE��>�������������j������-��R6nV�[B�m+�A+����l	�������6�(��~_�D��c�Iz��������a����q��J�UI������p�v�/��I[Q>�/�|�Q^J��!}�$�G9]�s�����/tV��-���WH=��:������hL�e
:X�^V�1��Y_"���Y�N
A��Yv����c A?p:����
�������]�{w��������N�U4_�#��N:�Gt=$�rg���u��a�����=E����/g��G�`�������+�7�����<9��(n����O0�@+s����c�n�XxZK=�F��F���9$ ��l�[EGQ�n1g��a�%F&�u��?�A���%�h����p"���x�M?���!��0|
-&�*Yc\�4 �Kr��a�!��x}������%���e`|�-���#��md��F%z�nP�H���f��3���a�n!�@�Ka ��?��x�o��������jo��������(&���>���.�=�eZ,uZ�5aa�h7�
'����@�
+s1M�Ce@d ��`^�#�t�4A�MS��4Jj��_g���L~�Ng(Bz��}��R�D�zi�@������&�t��L������f��|w�ef�Q���.�g��.�y	9h�+^X��tO-���t��MM�f`�;n/�]�h������	#�����E���E*�t7�Hn(�
��^��f�5.6k���&�P���L��{M��id$����0 ��tTQ
y�}&f:L�*|k&�NG����������!h����������"�w��c��x^���)��qF|S�|c�za)��$���k��gd.�}��PVK*��M��2�Y��:s���Z��C���Oh�r���F�����/?1o8�7>a�#���)e&���"0�C�t���:-�OI�g1��>~�c
�<8ZN����a��bSL�q>=�r���~�^���V������9��j�7nxs������rb������mw�]�(��;w�6���;�DW�Q��W�����6�/���P��^����� +���j^�����m�����������������t�~:.?��^���.r)s;�t>'Rin��v}��G%��b��L�1�FT����	�u�7��������Qh!,|��
@b1����j�<N�$<CE�'����4�:� ���V�z�N,Q�/>����?� k�u�B���d�.QFEM����i�w���6LI5a�7�-��
�0��z��)�n�@*%	�V��8h���Dkhr�'t�����N���Pna0��v�c�x��-}�0c}�Z���������6�}���7��c����C]��c���/�H����Z\y��F���Qg�-��	�y.6+��xV5z�������`L���C*����e�r�C+p��!p�j�e���+�xT�g��O��`5�@�a�� VJ�*m���i��0�]�<IY2���D#��WTx.U�fXE�y�����0���e���-| �3OI�n�{d�~�����2�TXJ�/=����H�mXzk���h��\z�����eb�w�1|.D-��A�2�]�9�e���&69�At�^��kM��kfs����M�d���m�Z�P�����=�8�Y;�Z����-X�Zk��'�.w��	���g<�E�`�9�1 [�rL�f��2tC�&F����v
_�,V
��.wq�a($����`
�/��������L1�����8��,4��B��pl�W�n��H78��=k�s��l��2��, ���p���cS|�t�F�-i���)�A�)������ 1W_W�����I��d�M��-L�],�CO��AK�r��������%Ba�}{:�A�R^=-�2��f[?qL���w;��{�R�)
�@�^�}xu��N1�6qv���T��;+����.��Wf��a����I�U:>}������������	q��Q\Ub����a\���/6�\-�[�;|�:�lP������U�L
��j��A(ah����X��Xjn/*��	$r������
(�t����~=?��0���T=��C)��l������]��;����%���������z����N��}���+��r
�!��,I��w;����g@���x��5.�e���,����fa�j?��p��F1�����Z�H0Xs�\A
�/g?�y�&���p����D�>�/"`7W'A#������~��Z�������kL����m���*�	~]^��Hx~��������+�T~�m���QD.���<r~�#A����#p6��!H���_����5I�'F(�����)���6z{���O�?��Uo�V��^#I���S�6��h���_~y�����_%����et�]N�����,�����Pb����
2���P�/t��	�`W:���13H1�����'T���� A��9����n
�����2�~�N����g��Y9������X��|�+�r3�r�x{�zr�6�������Y)GA�%8T�s7��1����H��xwG���!�����3��BZ�G��n��n����8�2�_C��ZE�����n�f����l� ���q/*�I�GZ�{��.�	h(W
�S��fD7��]��'&h7*��hd*���LL�/����i)M(j`\��c�R��"lK:�!�������0~�����_�@J`M}���pS4>���G���A����~�����W��j�J`$�D4��{�{�������a�#��Xx,�@v��Z�d
LJ%��AQWBa��.����������UY���j+?Hn������ n���M,w��@�����P=��HKym�n�������K"X�`��/-�����e��h�r�<99��)1����v������V�����p�/�mf��D3&l'�
�����)�A���C�����Iw����u���
��&c��M��I���yt�^�K]���sMm3�
2�lgo���@���gj��n�V�*[�`+L�Dq
i�N{zAn�7��a�-����~���K����e�����a��E� �^�����n�������D��m��fsy~��0,�o3�[�+b���4�>r;�mw5J&$�����$�/u7���4�s�yK���
`��'�(���~��m���R(6�1g���Ga����x,�Q������XJ	%;���T�5St��2l�~T�P�5lM�T�r�6��Yce"�)���,��+,&�Uu����iR�eW��X�����P�e���Tp�{8�������O�������W�� �sRY
��%��@��<��f�������1zUY*V�������0�����X��C�"��)���D"��2��N���F���RF���U��QT���opT��)LL9��iif>t�Y������M����@���_�k�Y�e=������	���O��V�cG�lNjx�$�e���L{��,����:f�W4�C����x�t$[������!���&��}�H���������!��$�Vn��8��Idb���\�4p#{m�)�Z(	dR��@%�V�������n"E����c�����/I�C��4R����~ph;���J*��S�
2�^q� �N����}����V��%�� {�F
�����W�%������S�H,v��u��%�b��)	���JU�=�Vc,��9QZ�]m�7�:RW�j�@����v��Hl���K�;���L�*����@9/3v��|[�D�c�v0*��;'���bf�"1@�k�����`"��
`�%��x��pG����%���)�
5}WB�����|�NyGb�����j�����]�9����
N����3��t��	������Da����}�j�
 ����@���16S�]�dzH�-W�.7UrS��@�8Ph��+)�0�����A(G�KK'2y���%do��^mT5i�G�=����n�0iQz����Xl�&�
���>��������(�9�7���b��a������i�����~Y���9�f��ChG%k��aqY�k����nu�=k-���/h�)��W��.O��G�y����nSP�O���4Q�����G��u�
���z��������w|E�=���$���)r�����D���?3z����Lv�`y
4������{=�����9�z���6,=����
���Y���~�9�t�g�
��������g�A����z9E�?�>:}t~"wbr�f%hR�},�>P�J��0����:�fw={;��x#�x��������Cv����	|�� "�p#B�EgI�Y�Wf�@�*r����v`d��7::��[1D�p5x{K��-������ I��[2J�t�|WU���T��kz���i-�@���^�[��������
�^?��CB�o[����������w28���%�*�K|����
��WW���v+�G�l��u��[A�5�s�(�EMN����_	�m~�%�#\4��{�2����-�LO�
C����=<�#'rQ�Ck S�zR�n=���KW�|���]��'�FJ�M{�aU��%�������CiU����)�a����~��y3���at3������f�L��R|�,/����L���=P�w0�v���9�|�1�2�Kboh�0����������[~�aF��LF��t
�4!e�k�6M(���Bu�w:v,p���e�iy��]k��F��m���
��ji�_R����qu�K��w���{n �>-52�%H2N#����S�zo#P2�0C�gKKg(���d���"�pE�N���!��~�+�!����hn���'�v������^ '��Q��������QC�S�����_ZC��j�^U
���B���`:[@�x����&$��m���NN�A�B�{��p��j��N����
;������u�/dwt��n���T�{�T��g<�P�7�K?�E����)��<�Q�A�[4lN �:T��`��a��d�S��������a,M��x�]����LN��\j�d,	�-mn������Q������ $h��}���#��c7���� �oRy�F�9�����l�A|<]�6�*����f�4��.6K�W�=�	|���9���:?��s�����.���sN��f8�o��-��{�d8���fS������b��
p'~�����"��n,��w0������%&��C��>�A�!7���r��6@�wv~�}�T���u�D�aG�1dF4�����B��P���
6��Cx�7^������D��M��i��h0�1Y?���Yc�];���:����7-<���%�������{rv���q���:w`p� ��;b8!��]:��
�%��������pq��k/�����z�z�W����T�I�B�,-(tr]DJ���F")Yi%_bw��^�6P~�&�
H7�#��6]�7s�+ ����V:KM�n�������xMx&������o��&�:tA�����	���/�,]�sA8���`��}���F���DV���	�I����b���U\��!����%��9�q0����Ut�������3[,n��~[^e�����������8a��	�?�@8������7c2$p�:������1 ��t4��D���>�a�`H6����10��8x wG2�04D�8�gc�y
=Ey" �*��
�[���h*��rI�rF2�����ov�hU\@�=�?9y������_��P���������9&���i��{��W��H���-�'�1�>@������?����^2�'�������xp���K/N���_�^C8s�f�N�Q�/`��+%fU����#\N/�I��2}�l2�
��a�,���gY��$��I�������2�������z���G1�����)�Q��pI�'J�'c������"w�d�����u����>n�nt���Ac��~�����I/:�
�������`�>��_O�e���i�����P�	`=88����0�:c�Fl�Z����:���E3���j���B/X����.V��e�]^|y���@$�����S�����-�"���>Yl��Q���G�7��Kt�7�a�W��S��e[1�������gC��������,��V`�������  �����S�{����wy�������{]NgP��6��n'��@>�#�C��������y|��O�Y\dQ,Q����#qp���_M�Q���a��/L��]4�?'I���A�<�/	��g�a8T������$��.`��'�	�����
U��p��>���^3ZM�+{y7�G�xqu��r���At���������H��|�A�~�(6���^>�}��9L��;B�r1�{���*�D�z'���.����I�9]�_F@n��9~�{~����Nz(mdt�y������b��7�>��������h�h�.��z1��t�O��n���_����U6�y!�.�������a4���h=����S�����7?����)���Hh

�������Dsz6
��o�A*��t�[��W���|�Q��7Y����Q�BG0��7��p<Z/7���*�b#��.����G�N9����N����&C.~X�o�2����t,���g�X�P,>$2R��M9����4�b�X���76���i,eS	x�l*0����X�^f��0X�K��z�e8���U�:�e�������Rpcv��9��,��?r���Og��~��^G^��_�����������������(:����i���lz��	���g3|:��Q�s����'��Av������	�Dr���
�����9��������)����sR�������	�	�\p�?�_�'45�[1��[Cpk��_�'��PdPP�
b���h���p��,]m2�U".Sii�dL��4��-�\����i
_r[),�F�<��4n_|�}��gF�4���g��J��p���J�?Ls4�)�9�G�I��c���x�Y�K0sPP����,{n��K� �TN�.�8Ji�-d+��4��
L��@��;��{��u��8�69	��?O��mn��k{�����F�d��
�
�.���uHZ)n�_������������O@�!�7��	�|lk]�������V��r�5<��x��3�y�wiz@2.���$��0��y��D���O�"���O��pR.xr'+j�2&+\�R%?�������a�?@a��=K���������'O...���ss��5�%X�<4�O������
��z���0�i�	2�|��2�������V������8����x/���P�{o(����At�l����F�m�u?����c��X�-��?�`0*�
�!�O_O�\>��3��<��>?��Qi�:�ZM��*M�V���������21����=`^g�/Y6/~���B�;C�����z:��i��e�����*����O�4
X�n�/��X(`}�h5�_�����?�	[>���])k�[ILZ�j���vb�v8�E�����V%F4�/�����LB�Iz"�y���o���o��Xx��#3�\�>��K������" ���|/�f+�Yy+���9���9J���Q'�a�i]NHT��7���_82s���s����}.��1�P
�0�H5�B�����T9&�6`�����7������I���j1����a���������H��4�_�(lWa(��D,�Z������hD�)�#"k�GXB���s�3F}oS:����b�iD�����/��7<y�<�C����<&����`�}�#@����������KU�H�]����S�H�sg�F1w'���3���l�UaB�VL���st��t���x
-6\IyH�S�qU��z��G���VG��[�"�o�4���:�(V��41X�4���,8����D�x!.G�5���� ����q�Q��C���yr?��;'��b�|\<���a~'��$=����|R<?���6��A�1R,�����!~~H���cn;b�-H�9����,�|��Wx
�h���}	��K�~�b�|�H�.�h_A��;�=�A
�h7.[����$���o�������aH��d�Dl��Nn���Mp;d�b�vT�-:m������Q?'C��8F��?�/	~')�3���;�g�!@��C������������U�A�?�X��V�-���uBsK#�
�6?B�W*�y��y�Fj?|M1[Me`;M�q1r"H�D�e*6�m�uuv"H�$1'H�M��:S����V,C����$
{dIG�o+�W�'�����$2�|���+��'��$�i��������!��?2������������
��@���b�8%aO4����f�K�/�v���������^g�,�Z@�(l#JW���l�n�?w]�Ex���su�O�a`&��
(���}fj����;x��u:P�Yp�dx����J!@A�XJ����X�P�4��4��i�EE��}[�Y�E�^X���]����F����Di��!4X
w)&����)]$��h�G�vWs�h�|9l{E�;���v�O�=�*w7%��
J��)�y����rS������"��{�bR��U��� ��B�o��j�VI�F�kXV��S���o�S�:���oITsU[���z�S��S��aY�W������X5?�������x��)�g(���Ta$�=UtO���hB�����<cj��pjhq���%���F����8Hxh���v�d����X�e�\����B�f[R'/�G��Ppw������PN�R�u���Q-vZ�~f�G�X���l�z�������+�aI?*_�������,��2��Sg�$�D?4��>g�2]���*�]��*m��>\<�P�����n���y�`�qM7��h:G�5���Yw��Z�.}��|���j�S���������|����=;JV.�u(��o��'��@
�>WP�D���������/����p>�-����Q��c��h>��Cv3}l�'��hm+���v������~$DB�����k�P�f��UA��_SbTD\;�Z7��������V�����pN�lDm��	u~�eE��l�_��\b��v�YRR�n.���"#�#1��X����������N�^]-�+@�
K�js����n�~���$w������t�������J�����Y�����R*�����[g}�����\��w�(x\��Bm�j$3��h2������@�����R��#����c"���IO���
�����#g����v��<�y\��g��x����)\V���g6�Zm�nj&��+��F�b�R�~>�*�+�����s�4��"��6�9mA�K�54n�
�h��7r|�'��H���*���G�/7F��1��A�B)�������Y�)�K6��:",��A��cL���/�J�����x���L�'L�0I"��JTDC��'.Jjk�+>{�h��T���?����$�b��-�J�x0����8��h��[-
�0��en���q�N�����4,Z�`����~7�\����<@��:�&i�rscB����
�DG�d����$��_��CKI�D@�+H��8	Vu�hO��~�����h�+��^B�W�O��}xu���B$���9-r+��`�T*���7u�t��,#VneR��w-�8`ez��cj�Ht�5tX�}����t������B�17�jcNe�N|�p
�tmOw������8�1��������	��P�4���&EK���'��g�(��\�$+
*��r!]�GZDFkE�������q&-���.��I�"�Z�*�����j��UQZ����K��*�����e�3d����o@"�$����b������-�<����xN�T�+��Wb{^���"m����W�mU�av�r�
[��D��
�������fd<�2Z�;�'7q���/��o��Z������pT�������9u�]�H��cm����;QF�`�'T@3�#J��-<V`��L�u�A�b$�2��}���4gvHJ�Bm�.AH��D4�lR6w=�[�_�	�����q��}�@����_���X�e�n������yy�M7�l�K��4-dh��^B/�E������L������QU1��^�U����"�������e��PT����UN9�@u���M�YbT0���nW��h��.�]t�#W�=��m������j�_���z�$.�$�L���MY����.�bk3�q�q|�`��\�Z�bR�61�F��=^��G���������		K�SXK	b�hR�r�"��>x,
h�	�>��'��<�g����������dt���XdrPy*��Gp�R���$u�,��W��p�F��r�-����p1����t�X����
��j�h��V)bi�������BJT�cS��WX���j�/��Z�T����%�����`�D�L<�@��?{Q_�ck��h-��w)�I21�?�\6�$g����X(I�d���Y������)��d��i9����Zt���f��{���j�����X�51I�<�gI��
[��6�L����	��q�6�$���2Z�� �5>��K��2�(�0:��u��7�n���g'y��Hg���X��U2�0����������I��0�����?�b�B�l,�w��w'1�����d� �z�Q[u�
_1fdQ@y����1��_@2��?4�ZI�%D�"�����xI�S_�W\V������%�
n%�����X�K��m}c�k1HZ��\[57���Ei/<G�)"�<���h�.�*��h��9�;<n���qr��O������CiC�W��c��c��*��r�?[�J�Vt�8[�/Q�f[V �6��A'h�����������*�7"['R�N�l���q�6<������<�`k3VslmE�<�x���� hoM�F�:>Le\��Q��SU�/$���������q�^�=W��
��qB��(���k��e��R�/���BQ�8��3'����e��4���}�B!����DJ��q�G���/6W�~Z.6�:�T��a�]t��e|�n��*�G�������hb)K5�TBz�2����2Fi���Q�q7h���_%#��p�}*��H�n���
Sj����$���������o��>�zP����� ��'���"�jO���54W��XK����]��]e������&f��x���p+�91)�Ob,^�2{mio�)��������Kp`���1��=vv`�IX~BS�����qzI�}�lLsb�A�@'��N�0����'����!����z,�i>�}���OW>����i�h����%�����g��������0�'V�xf����`
�M% 
@����6� ;9=a
�gt�-�2Ta��1��0Vct
�����}���|��8o�`l[��}�������4�����s���"��P���k��?����j4����BDT�fQ����k��K�.�h�Ca*��k+6R���id�J���0��a&��X���"���i�L�Ro
Y�>3�,lf�3���M.�4d�I"�c&���X��d&s���h)q.KOQl���:#�>�W_;��n1!/3U�#R�Of�QKV\7H�P�*<T�x�FI�����j�U���R[J��a]9��~l�~[�
,6�E0��M��pn��e_7x����bT���FM������2_[�H�C\��N���Lp�����nJZ�rr��
^%��z"�E�Q�������&�_��PUC\��N���Pn�S~Z�0y�!O���D�	q~&�6��SG���7@�������Ju�����)��*N�����q#9��BG�F��4n��iHZ����$t#��7�s5$����������6��+n��;Xs~xZ~X�t�b����%m�|�b�@I[5��e�d�[��8�������6��&O���y0��k1�Z�or���$o���:������3�kWY��%7w��_��������N�"��h��g��o)�� w���8�@S�%�pU����?����N]�d�92�J2��c	�c+2W:l����L��dN2'�%��������U�����7�J�.1�����K��P��}���p�/sOrfJ,y.��';[%�<7��'9��9?�xn(i��+���=�^��%�����N��t��xZE��e4����.������L��)	��W^*}������,�������`��l���{u#�7!�o��H��(�*������oL�>W�+J��v�I;'��3o�QI"�-p��z1�\����)������N��J�kX,��������k7`�l���Z
X�a�D��Zh��iXy�l-=S5���rz�;���G]~F�v%f��t���s�m�NQ�9������x."�]����+:/���������5F���s�t�g��
m��]�O(t�2�NF~�p�y��q3z����N�x$r��3l�vVO������l�|G� ��b�m���M����P��rC�|���]��(~j��b�-�Efq*C�Jdb�����������<.�V)����p��;�����_��������~���	#O,������8�k'&�_1�6�k�(���y�<F�����;I� �=����]�?������������OA�����p�����3�}��)���N�����������6\H�r���U�q��
gG_��v�(�N�?�m��r��XN����A��
�b9����&Ce���l��'r���ce����6��I�p'
>9Q�}R��m�������O�)�~V��m�8��n����������shi��=��I��j���9�a�r�r>�+�e�_��m��%�Q�b��:[f`+�]����2��k\Oz�.��y������{,I�M~���X�>�%f��������{���~�=�8��r��e:[e|q5�Y�a����$���!�GQI0��@j�z�5*���.��I��.����H���-Mb����&%�,��skR+���T��������������+��	�~�\F�LL�("��,����������������%44�����n3y@�L��y)[��IZbC�$��H�Z�=(��}k�}�+�hQ
���8wd([�J�����F,�HC�pJ2$���'��4�4�!� 
r��.8��������.
��
��G�T���B,�
����B��
��T��SA��~*��� �B�T��^�r*�.S!VNUs��3S�����T�E���3p:� 5���Ig�(sK�\�g_A�xO�v�::@�jy��=Oy����g�����S��*J������K���&5�.t���^5})S,+N�6�8�HAdI}z���X.��Q���B$�tf����Eq����?��N�H'^G:!#�sN*�C#-����H'E������V�>�ZE��&�� ��	�eE�$��_�����c�s�w�;Bm��U�>���`��$�xB��7U}P�i��e�I3�'��N0�7<���\�6���`l�]�$��~5�n��hN~��i2�T9�E�O:�l#�K!�3(G����7�
;
�K�[�_
q{oA�D�PRE��EWy�N��Ow��x��#v�P�y&}��Un����X��Z$H���m+��%!<?��-�F�h�6�
X��}e���������G���([��,��&��`��-�
�.+��|��B�h�x��&�/�^|���J��E�*K�v��U�)��T���UrI]**��mR1�TL8*&"�bPD�"��m�)	�Ce��#��|�F��HO���f�fc"cQ�����%�/$iGZr�$�w��;�g��s	S���9����0<�;�&�������Q��V��Yqb7�&�2U.���%fMT��q���H����q������Rl�	�%�^����26����H�XJ�XE������)���A��"qc)qcqm
Ny�['��1�q�s)�&*�M8��V���h��)�]n����,�,�����Ss�?s����0��f)`��	(y�e������f�����O�}��7Qq@\�O?��)�p��0=���m�/	�2���1g���E���c^� �>���Pv&��P�<�V+Z���VO\,��;z�����*�nI>U���{��w��p���;�t�&.��%�)��J�m���:�����)J��Tai���}]G���P�o�9'��]��YU��C��S��r����]pF-���g��3���"�D�1�V�/���6b|q��8#n�3v�_V���Ft��"'��m���f���B�����Q���)ac8��l�/����'���"�!x;����$��g@�����T�'�Er6�ROf+Jp��q*s�%�kH��19����:`kgr���
��>����[Q��x������H�>�^�V�x���W�4�c`��@�cP�=yv��
5�ch����.9�
��Y�
:��A����H2�RU���k�un�$sWnK�d���������
���D[�������%�&cW���/���a�����b���L��;�KMTd������rJ�#�N����Ty�sy���E�
iq���K_�7�%OMr��t��4MOBS�"��y����t)�^�U�%�(���.eZY~���N0jW�XR{�0$�a�=��]�v_ �3MuZ�v9�U�v"��]a"���j
�c8�-$$�6��2�������dM��*���-.��<"�Z�b8h8�C�
J���b��(���M��h��6%���*[E���l��dDT�a&�X��"[1��:R1��Nr)W��?T�C*��T��*����9��j�=�9iR�x~�+���Z�+)3K��/���#�+�jwiZ�PD���2�l����j�����j(�GI������E��W�8����b�d1S�u��Z�*O��?����~l�p��]l	n����(�}3}�H���;�/Y6g�����5�V(�����'�Rc�����^�>�v�xR��/�;��fK�L���+
��7���O����c�y��X����@Q4L��h&+���x~G�@.Og���������\��_��;�9V3i���-pS����v+v8h�1^�[89����uc����'kl��c�d;��a�6�z�Nj���0)]{��������"[�1=1v��*&�����[cS�]�M5�\��co<�U���9e�G�������j�������s����u.��e	\X�
�\���X<��c3M6_�
�C��c�Kxr���9l�+��N�	;\!��H	�%���<�������<�c��]1��wtG����8����A����5��"k�%��C��������g�H��E6���?&/�sIHD0\��0�F���O��JdH��g����\��N�e>�K%P��r�@T����n���Y9/���(��������y��������t�]�s7�Jg�l6ZM����,��L�!!��H.���'����q2�V���A�������������^�=����g�_�>���g�	:$Zm���rz�D�3b
|;S�(�W�\��eD��'/<����Y6Y�>O�/��v�\���3����j>B�l�Z��/28��|�5��L��t}��.nW@��/&S0GW��/�|81j�"]�@��_,�Dw73�Zt��O �l�]E����j��:�-&�E�\t��}�@��d�_L����A5��d����{�P������4����=�rs�d���di�[�������"�������f�#���v+ft����%<Bf����w�%O��������bv��(����Y'��	�Y����YP
8 �cH��}������lIo���:g1a`*����)�
��'����r
{���(�0{�z�$����%"������9�X���	�������9 SX���Gp�D�>r��2[}�M��F��Co�����]����`r^�����t1]�.����c���G�u�H�^n��#ys�����A��N��r��,<������v$�e��%!e���/�����Z�Q��o��>�2�C��RF����=�������?���!�)�y9�@�f��,h�z%�H�����u���x!Q�B�}���{��C����I�96��,!L)�m����;��������:?�Y��\/f�c z!��&�*l���+9���Mu���,�P����P���XT��A��_����0�Z�J���%"�+��� ��@��P��-�����HA�}�-���	��W8��&�-?-3�����`Q��MI?��?��^t{���B�����)����'h@H�#������j���g1��8
�%4�)P&��y�X���y��r3����.����B3A~6�v���Je#�������m��f��D�'��W{.e	w�f�m��&��
��v�V��
�����9�NEbIt1e�UO=%_l�7D������K���090�X=W��rv����'�;.���"�������Jq�Ka�����+f��	�hW4 !_���4@������H��G������.���	i4Qq��(�B�	Q*���@tM�,[M���;<�X~vM�t��P��%R�0��m������OE����������3{�x&O���TeEm����Ua��p���:j��(�Yg�
>�s�5��|$�Xx+��M�J���-.<�����rp���X�-�=��rn��Tm?�r���*�����j	��o������*{�6 0L�4�zGM��I�����/�Wb����iFV�#�P�/
���cUN�d���(v�6�����B�������'n�MF���@���r�}�P�����s������l���y�*�&��0���f~hd��C��^����2��Q�R�������>�(�-�7b9����~��
����#LX��W��6`j�|�F�<'
��@Y����arnF��0�/���W8tn��]��kdz�������9�I���a�� m�����3��y!���FcJ���6�.UV �17��+mj7r����J���P������mM�b�5�t#���Fne����C�'�Heo�q��q���5�b�kxZn:�U�>T�|��-�U,*i��C����u���c���\*�:��y-f^���Ib���L��1~����S���L�����yOdk�L�oQ�r&������F���Q������]q>*�-�pV�'���]?�-g�H��,��j�$Q�3>n�m�$q��v��Lg�h�j�bC&���<���S��K8�{	U�;[/�7��=:}$�ifr?�L���NQ�/H���4����m�MU�Wh����9;�iSX�,Jb����5BD�H�����(����bavfC�p��uT���bVrT��P���ls�M�������?|G������� �	����"�P��[�������s6�{T��\�{��������	U*�A����p��	��H��*��g��l�E>��������\W������:��%.���F�#�	�N`�����8b���`�b����e:��
\,�d�"y�dq���D9p�D�����@;�����.'|�(JZW�%,'��!���d���A���d���!v��r2�[�O�X>�T����+VN��e2����jN;bf2H���&���3+g�vQ}�h�%�Q��W��-�W�]�Vt�,�l��FM�����H��PfqV��tG;�B��T���`�}}���,,V/��
CQ${OfO�_Lm���u����g�kK�'���)�������������k��q���{px\8>�e����r�:{���Nt����gG���a��x�����GU�A��''O�?�K����jtq�
��na�B�z
e������&[�-������������l�=e�����o�d�c��_�X�K?��'�������d0��K�ke�����2��%]]o���������V�:�<��/{Y�Y�M�$���ivr�\��e:/����/`9���F������?��h�f��N	�����M�?Q�=��Kk��]���b�d�����5�������"��{�{����7Lz6�f�����z����Q���~���d�����_3�=��j��@$�,U0P�V��1G2%v��V���x����+z�p����${�9[��',���t���o�$*�B�����k�Y��!@�<��I�j�Dwp�g���������������C�D�y1��e�V��5A�2��`5�Y�A��oE��"?����gST?v�~=gyJW�;���hE�}�����%'�	�m���)�:��� ��iQy��Wa3
!0������s7�����w��9�0��
7>q}�{����?��q7�F�~��?(�G�K���� @��)
BOC��,��+�������>�$zw�Uv����#
;��J�V�������e ����1�d+�H���F� .B�<����\�VS�:�����I�����|
s�,dL�!d�+�{0B.��hm����8`F��%AY<+)wE�������C�6��U>3X1K��m�3\��V>�T�~|�W@��l�N~�����Qv���C?_���//������[I.-�P��.�D��XP�������
��^\�f�}4A���^N3>�yw���d'U�
�~���$}��{6����+�8����/u`w5]i�����'%V�f��������e�:���^��q�~zA#s���A>H��T7�+���g�jz3��x�/��	TP:���!��]�6YD�nxn�)�Y,�����r�F������g��VxC���L����<���.!�r{�:>�b1Ag#��nfO�b�]>����h������� �C���O�k��e����pd��������X��43�.�U�� ~N_S_�=�3��E��G�Oo�O�y���Hxv�z����9Z�7����<�&��y>����)�v$� 
�	d�l�6���a7DX��������k&�`���Y/I�<���,~~��$�m���}� ?�����o|�X�b�e�#xTQ����5\q��4������??����/���?�o���������u����HuY�����?��e����s�������.4���z���Z1jCt�����
��] Y�;o������9�.���
N���@����G�-�u7��2kW{�������5n-#�"A��M>�n{���@@���I�@��HI)^�dknI`��x5�!����i6E��9�z�^
qc�%�o�IW=��*��t�A(P�Em!���EF���^�~�F�y?f
53&���f�!�$,�^n&�9��f��c^�P���B]��������a��
�����(Eg�����z��0A����#<�b��[)� �@A�'����mG���������'�u6��
�e��}�W���st��x~1�<�<�H��'���[*��~�d�ql3�������~��X��
���L�xd90����2����F�2����F�=�E��M.�p���A{����L�[�2z���[�*<���%K�a�Bp������)X����_�T�	GPsg��g�L������#���,���w���}�A7��zA97�_�+��FPD���X�=?��F�?-?�E/��a����5�� *#,���d}8{��@��$
C�q���D�j�������D?t��t!���`8B����$[@�R��O��?#=�;$����hr$��nH5�I�����Jb��WX���C��X���M�D�����7,0H���Y���ZE#�\|�.`-~#��7�U(����7:����I�$�V<y$k���4@�-�!�?��+�=d��x)�+���7�R�M��
l0 Ugu�|+�-b�1�V
tt�y\�/%T������gd�Xn������mJ�yY
{��k�7@��#V��p�P�o�i8R������I�y���U,������C`$#H�WXS]�
#��8��n+j��f3~���+�Ak*Q�qc�1����.����gE4x�b���p�l��2��stl�M���C����h?D�DP���~}!�2������a��-�����3ZZ�T ����+Y����	�s��M�r�e��k�D���k�]�p�����\����0��1�����������'6J��B�I��QE�4Vj�dhp�l��`>^+��%�
9��(u�1��?�7D����6���6�Z������ Aw�J�ZR������}W3V�T%`���
�-BZ��f,�;%�m��4$X��Q�9�f,�9��.�!�H-#���]���������}�'�
��j���r
�Q�����L�Z�2�)�:�v���?Ek���y 2G�u%{�!Z����x,d:E���_�eF��R�^
�L�8��D.��{x�5�#x�f-��C@�&[��)���0�0n��7K���7�����
�*7
"q�/,�Ka��,�k.v��o0�>����n������|�,�G%����	��������)��~�k#"��v��\4:��W��'i�A\a�T��`'4�RC��tF��P���Fp7��a�es��k�,�(�!f�D`�D������>�cfc�d����~-)`'B]1�;}��c�r��lu�m_���{���n���tO[������i?���3�y�(����o����uy������]�YzK{x�����w�/(�q�>/.���o�j�����g�W��iO�%�m��2I��:O�$u��v:
EQ2���T\O����$A��������N`,v�b� N�3-`HP@�[P�(����H��|zU{��X�h�?��5y2+��:><::����p8m�?�RZ�?�����c::���tS�N���y���d���'�w>u��/���a�v5�>�+6��#o�Ys:�Sc�9�{�n`����I����"9��~��J�KF5��<
�J?�g�������(��E���y0�8�fA4��U23���������=<]���wa�����T/:���g�%���P���o�,i�I����0�B�n?V�y�����Z����a��2���+�T�v��9�j_=��0/"o�1Bcg���(�FP�/���>�J���	'9EP��(����2EM<RR�B�I���cpN�_��p>�V�e�k3R�xQBK��Gk����3o��>f��fA��^9�P=35��nn�k�~���O�F/������UF�BO��8�|���Q:BQ��o���"W�r)������a���t'��[j��I�:��P����>0��@I�������^eW�_����P	�i
�sk H SBy��:������V���U��G�/����p �'}%��:{�/?x(u����^������5�d��p�Z�M���C�h`����Xha�����}M�Y�:Y���w����c0Fe)%d�q�7��+�Q|!

���vME)�n#�A��L�x1������gY��=Z#;�.H�QKSk�y�[d����wx��������Q%�EZ�����sZ��O�,UZ6�����o�|�-^��&T�$�_��,N��U.��y����J`5m�u�(U'��V
�i�S����:�'
|�`�g�m�x�"���F�����i<�����������
u�pM	��N��j��2GQ��f��vD�0�r(_xa��F��k������!�hH5��;{'�T�'4����A�Z���a�9���C�����s�x��z?��x�b������7���%(��<��0R����p�+���x�t��eq���?H;
�S�PJ��u��z�1Xy��t���������S�r[���e���V�\���w��Z��.�rT�������5�����E�X�Bt����:6�#�(G�g�Pnb���9��'�mo���\2�?_�7KH�+\��*e�d��r�a��B<�?E������P�w����%l7O[����P�0��C�om��f_c�}[�L]�v���g~�H��g��
{\n�����������cG?��V����lIn����?�Aw���*���"�[�b�[0��=����4��"�-��r����7A�����d6�~X� �`��&e���E�I��~��s�r��Y�z�?X#J���v�gy�l�����Z[�����������C9��X�V������}�J@����*�m-j�����E�7'�s�vR�C��6,_NSe�E�w��v�Ba_y�uk}����JO���K�Q��Gu�-����A�����Zt���_qjm�U<�n�t�|V��;��4�-�N-mt�j\r%��\^%q�������\oH�'���3w#f�Z�t�+�DW����i���5	s�4:�w&7�1���R�k���CUmf����,D����Kv��V�V����l�?�kJ���r�r��Y	�5X������.Wqdis����Z>?����YB�F������*����-�[����6�Uw�j�g�`�R�Y������5?���Jpf�e����)mY�Ac�l��h��6&��rZ��"R�����;���8@slA�b�X��
�Pzh��"��4$�$}<��"u��kM�%�D2�%F���e��%k�{pZ�r#g4a
�eP�r�
��F�]��N���z+&�������[:�������(jF�����5	j?|%��������z��u���b��a&�����^��T9��i��� X(?�/�����e��c�8�e�eY8�,�GZP���PZTsJ�����H��^%�46����	����|%���#�d�3�!��[���P,9�`��f�C��0���by$�
TH3��XBN�#� ��5�:�_7�H-��6�\k��u3	��^{���V$^�{wN ��U�|��DD�X#�g��)�4_a{�����E1��K�OK"����h�$01���P��0������((�9h1�7aJ����_�Jt>R8�
c%���&�y?
d�4NI�`����@}����}`���c)��!��ut�K�����K7$<5;�gZ@������:�������=�Q��\
V�\�.s���dc9��z�W���ayz�r(��&��n,5IB�k�"���������(?Z�(�X����Yp�I�u�O�4�h�,���@�"0�ytK��}�������Eh��`�Y1��W�(�,�.h\_�2�H�����lq{i�@z��{
��J:��$\/�����{$b4��-El��-�*Fc���;u��Jn�^X����z��u�X+�R��<.m��m���e0�Z�sOi��Aq��-��6��S����V�2-���Bp��I�O�RD�1P��7�@�7����DHFL�o0�����6���]�����(!{����A�v���>��BL��E�.il�*����?�/O���bxG�K����C�]9��uU&�O�	{���&��!�������/�d
���������+���Q0������`g?����F��Qc�dt�����Z��/����u@�C[j
��3N���gV�wrM��7
lj���S�ej�xo�j��x������m��N�u@��EU�����dq_� ��%���&7���p#M������`����IQ�`X0�����?����x
=S����!^�>����k�Us��l��(H�$VdB�D�}X���s�Uq�(��7�v]��l�h���_��b'�f��;�0�k2�	�9<��N�?<���M��D�82������:����\p���w,�4e�j��%K-A�}@���JMn�G(��LU�����X��n��x�~k�mf'��xmj�?�ON�� 8	&��^�P;��������.Y	�s�����y��wKbc#��xw�n������b0���F���d��Jt�O�|z]�u��6��4�T6<�,�w����q��q�����a/��3X4�a����^��MF&��Mo��~�c�`���������?~�z�c��z���7>�9�L���`||p����������x�s�����N���������9nz�����@�����.�@�//D��:��*| �T��_� I�����o�5����mW�j�z�l�[`���� ���@{��L��<�Q�7���� ���x�C����v��M�z�&�F���O!�A��0fT��/���O`������a0Nn��0��t��M���P;�!Az��z�q��!�2=Q��Az�7�@w�7v�-������<H�/0�1�~�2�I��kz��R|�>�����/m_�Q���O�cp�7m��0��D�����"|�
���7�4�&l�����f���P ~$`�7�.�@A�T���?� F$pI���"�9�c��<�����)Xz�i8���B#�[`>��!�Fx3}�6�10R# ��Q&�$���a�����ml��o�.�Ac��1@���)N+�w\R�q����c|�y��5z�k��Kt^�)Z&�1I��
�-�.#�+�pN��^�?���H
���d��c	G-^=z����ye������n��@���V�i��.e��3;�Ip�?���A+��2������4F����
"�X��
9^�I���9��z0B��[4��tAg������������P���i�7����UM�)H�?Vk��������U������y��+�-B�Gy����U�C������'�fc���������y}6:��<���������R��Ls���;E��!"�T�gv��	���G0A��7�h�Y���r�`��U��ps0�9~5a���n+��|Da#7�����}Rq���!�T�B�l@���w[ZAS���=�n���'�P����a:T�A��t,�U�����	��"�<�T����H�.�Wd�"��\/��������S\t���6z M����~��]������'GC���?��"N=���\�����������M`|>W��[
JqxP���&k���IGj���Gg�~�j��d�����a.���E�������p�C�O<xe����D�
�8��xFB	M���	��������T����B1�'j�t�	�-d�9m�'[�X����a} �����5�ZR�h��R�e����W��������y�~��8���IOfT��$SY�=]�7A�e*���6�]�s`�W @�v���vk]���;�=Y�k\V�U
�R��[��\��.t#b.
�-�k��Y]F}"�����Ft��N�����u;��~�1��;b&���%��wJh�F�%OR�>+9tE1{�i�J����s~�
�4�|�=��Z�.	Hd#�L�����Ftq���|q������N��a�i�6h�x{�����sO_bv&:G�B����=�����~4MR-!�)T����P�����Vk���e ���V� ���6'��vWc�o60�N��a�^�v���Od����z����|l�a�T���m���� �QP��������U�����-wC�$�����dE�3�#:.��[�{��x��������Y�|O=v���Td��F�jBZ�EF�:����a-<��`�d�=�9F�����aox x�wd�m�A5�0��iM��������P	2	�M�	f5����6�Sv4�����:Sq��a���2����a.����3k��$LA��"�K2��tr�Z���j��f:��W������M�����vt3���n��w�����k�����.k�dw4�K��_<pd)���G����z���T�qW��o����2�p]9���R�h={Ok����������v��=S/��]�8�_�{}���j�����L����)�����Yms�It�2�s���g ��Q����d��[r�CkT/��{�cs����v���{<�~�y�y�y�y�y�y�����XL���
#241Robert Haas
robertmhaas@gmail.com
In reply to: Amit Langote (#239)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Sep 13, 2017 at 10:57 PM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

I very much like pcinfo-for-subquery.patch, although I'm not sure if we
need to create PartitionedChildRelInfo for the sub-query parent RTE as the
patch teaches add_paths_to_append_rel() to do. ISTM, nested UNION ALL
subqueries are flattened way before we get to add_paths_to_append_rel();
if it could not be flattened, there wouldn't be a call to
add_paths_to_append_rel() in the first place, because no AppendRelInfos
would be generated. See what happens when is_simple_union_all_recurse()
returns false to flatten_simple_union_all() -- no AppendRelInfos will be
generated and added to root->append_rel_list in that case.

IOW, there won't be nested AppendRelInfos for nested UNION ALL sub-queries
like we're setting out to build for multi-level partitioned tables.

So, as things stand today, there can at most be one recursive call of
add_path_to_append_rel() for a sub-query parent RTE, that is, if its child
sub-queries contain partitioned tables, but not more. The other patch
(multi-level expansion of partitioned tables) will change that, but even
then we won't need sub-query's own PartitioendChildRelInfo.

OK, let's assume you're correct unless some contrary evidence emerges.
Committed without that part; thanks for the review.

--
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

#242Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#240)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Sep 14, 2017 at 8:06 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

I have few changes to multi-level expansion patch as per discussion in
earlier mails

OK, I have committed
0002-Multi-level-partitioned-table-expansion.patch with a few cosmetic
changes.

Phew, getting that sorted out has been an astonishing amount of work.

--
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

#243Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Robert Haas (#242)
Re: Partition-wise join for join between (declaratively) partitioned tables

On 2017/09/15 4:43, Robert Haas wrote:

On Thu, Sep 14, 2017 at 8:06 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

I have few changes to multi-level expansion patch as per discussion in
earlier mails

OK, I have committed
0002-Multi-level-partitioned-table-expansion.patch with a few cosmetic
changes.

Phew, getting that sorted out has been an astonishing amount of work.

Yeah, thanks to both of you. Now on to other complicated stuff. :)

Regards,
Amit

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

#244Rafia Sabih
rafia.sabih@enterprisedb.com
In reply to: Amit Langote (#239)
3 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Sep 14, 2017 at 8:27 AM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

On 2017/09/14 7:43, Robert Haas wrote:

On Wed, Sep 13, 2017 at 12:56 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

I debugged what happens in case of query "select 1 from t1 union all
select 2 from t1;" with the current HEAD (without multi-level
expansion patch attached). It doesn't set partitioned_rels in Append
path that gets converted into Append plan. Remember t1 is a
multi-level partitioned table here with t1p1 as its immediate
partition and t1p1p1 as partition of t1p1. So, the
set_append_rel_pathlist() recurses once as shown in the following
stack trace.

Nice debugging.

+1.

I spent some time today looking at this and I think
it's a bug in v10, and specifically in add_paths_to_append_rel(),
which only sets partitioned_rels correctly when the appendrel is a
partitioned rel, and not when it's a subquery RTE with one or more
partitioned queries beneath it.

Attached are two patches either one of which will fix it. First, I
wrote mechanical-partrels-fix.patch, which just mechanically
propagates partitioned_rels lists from accumulated subpaths into the
list used to construct the parent (Merge)AppendPath. I wasn't entire
happy with that, because it ends up building multiple partitioned_rels
lists for the same RelOptInfo. That seems silly, but there's no
principled way to avoid it; avoiding it amounts to hoping that all the
paths for the same relation carry the same partitioned_rels list,
which is uncomfortable.

So then I wrote pcinfo-for-subquery.patch. That patch notices when an
RTE_SUBQUERY appendrel is processed and accumulates the
partitioned_rels of its immediate children; in case there can be
multiple nested levels of subqueries before we get down to the actual
partitioned rel, it also adds a PartitionedChildRelInfo for the
subquery RTE, so that there's no need to walk the whole tree to build
the partitioned_rels list at higher levels, just the immediate
children. I find this fix a lot more satisfying. It adds less code
and does no extra work in the common case.

I very much like pcinfo-for-subquery.patch, although I'm not sure if we
need to create PartitionedChildRelInfo for the sub-query parent RTE as the
patch teaches add_paths_to_append_rel() to do. ISTM, nested UNION ALL
subqueries are flattened way before we get to add_paths_to_append_rel();
if it could not be flattened, there wouldn't be a call to
add_paths_to_append_rel() in the first place, because no AppendRelInfos
would be generated. See what happens when is_simple_union_all_recurse()
returns false to flatten_simple_union_all() -- no AppendRelInfos will be
generated and added to root->append_rel_list in that case.

IOW, there won't be nested AppendRelInfos for nested UNION ALL sub-queries
like we're setting out to build for multi-level partitioned tables.

So, as things stand today, there can at most be one recursive call of
add_path_to_append_rel() for a sub-query parent RTE, that is, if its child
sub-queries contain partitioned tables, but not more. The other patch
(multi-level expansion of partitioned tables) will change that, but even
then we won't need sub-query's own PartitioendChildRelInfo.

Notice that the choice of fix we adopt has consequences for your
0001-Multi-level-partitioned-table-expansion.patch -- with
mechanical-partrels-fix.patch, that patch could either associated all
partitioned_rels with the top-parent or it could work level by level
and everything would get properly assembled later. But with
pcinfo-for-subquery.patch, we need everything associated with the
top-parent. That doesn't seem like a problem to me, but it's
something to note.

I think it's fine.

With 0001-Multi-level-partitioned-table-expansion.patch,
get_partitioned_child_rels() will get called even for non-root partitioned
tables, for which it won't find a valid pcinfo. I think that patch must
also change its callers to stop Asserting that a valid pcinfo is returned.

Spotted a typo in pcinfo-for-subquery.patch:

+ * A plain relation will alread have

Thanks,
Amit

On TPC-H benchmarking of this patch, I found a regression in Q7. It
was taking some 1500s with the patch and some 900s without the patch.
Please find the attached pwd_reg.zip for the output of explain analyse
on head and with patch.

The experimental settings used were,
commit-id = 0c504a80cf2e6f66df2cdea563e879bf4abd1629
patch-version = v26

Server settings:
work_mem = 1GB
shared_buffers = 10GB
effective_cache_size = 10GB
max_parallel_workers_per_gather = 4

Partitioning information:
Partitioning scheme = by range
Number of partitions in lineitem and orders table = 106
partition key for lineitem = l_orderkey
partition key for orders = o_orderkey

Apart from these there is a regression case on a custom table, on head
query completes in 20s and with this patch it takes 27s. Please find
the attached .out and .sql file for the output and schema for the test
case respectively. I have reported this case before (sometime around
March this year) as well, but I am not sure if it was overlooked or is
an unimportant and expected behaviour for some reason.

--
Regards,
Rafia Sabih
EnterpriseDB: http://www.enterprisedb.com/

Attachments:

pwj_reg.outapplication/octet-stream; name=pwj_reg.outDownload
test_case_pwj.sqlapplication/octet-stream; name=test_case_pwj.sqlDownload
pwj_reg.zipapplication/zip; name=pwj_reg.zipDownload
PK
�j/Kpwj_reg/UX���Y���Y�PK�p/Kpwj_reg/.DS_StoreUX���Y���Y��ZMlE~�I����I��!�j	��
�Q�7�j���I0k�6^����k;ETT���$�O��p����9p�Fp�'$@�#��qHR�)�*�o���o���o����p8���@���B�����"������k�p���.��I���a�p��:��>��/qF��\�&�
�	^����#�Z�$�
�	^�,-���b��T�_p�?}d.n(8�K=d`���3���2(���I�f�������e�+����Q�S"�u�Ak���B����5,gO�i���1�N�������������y�4lY3������f(fa���5+�1M�Sq�2�U�,tL6�I������I�9��ua!��\����Y{`���"=+I�����,����[o�������,�V��!=�*�G���&�tf����V~����@)��WSI������8�9K}P�e[���V�8�e��K����������iL�z2�����G��Uk�V�m�5c�Y{��8��X��MN������5�f-��\�o���������
7����4��NVuu��$|Z����h������hj�4V�'���VyT�w�J�
�ISq|�b����!I��HVe��L��}�������-]��P��?�f�����D2��	U���H�D��X�����A�d�MG�����A�!�~)=���	��OeT��Q�'I�NOXQTE���)������=��0
3�
�<�/�kp.��|_�W�5|�a~���W���?Y�#X����M��7�^�o��x'�0�a���	��i��$��F3�>���9|_��q���Sid�F��-�'�aO���|�
�G�Ex+��D46�nn���_QXk5S�	�AQ���0�����D�k'��y���:_T�Y�&~���l
<���-1YS�����*�[�B���d��V��Ar��*��O�B�����|"Q,#�Me����1|�D�
|��o�7Z
Rl�6�1�u�m���x��A&�A�����Q<��q��-�SE�)x�P�,Z��<��������|	_��\��wt���+�/mF"J�N����P�52m*W��I]����f
�4��om���i�,�)��]���2%���[���w
�^e��e�-{���U����j�\W���.S�*xo(.\pTq�;������.����H42����v?;)@����a+����f��E�������PK"��O�(PK
�p/K	__MACOSX/UX���Y���Y�PK
�p/K__MACOSX/pwj_reg/UX���Y���Y�PK�p/K__MACOSX/pwj_reg/._.DS_StoreUX���Y���Y�c`cg`b`�MLV�V�P��'���@�W!��8��A�0]�PK���4xPK
�j/Kpwj_reg/pg_part_head/UX���Y���Y�PK�j/Kpwj_reg/pg_part_head/7_1.outUX���Yg��Y��]Q���~���7KUgA �:�J���l�''��'�Z���$�nvW��~��3���f{V���8�X����k���nW�������~����G��������l��������������"�G��Ak��/��n��~���~����W�=�=�xu���������+���&a4&����w��7�����/~��}{{����/����������}0���Z�__�����n	����rw��������������kY������'��_�~�������?����_��������wo.o��O�������7���7��������j�4�n�����2���5D������������.=u�����=��lZ��������l?��s���������w0?��_�����	�����)���{������G����@������w���|�����\m����q���-���Wr+S|�o_�o��?���uo���_?/����������V~��'�S������~����=&O���Wo.n���=��������{�������?��M~��'~����d�z!h�^��������Oo@Qt�/_�����7���q���`�<FGh�=�5P��Oy�4=%2��K���xep��=����ky8�w��]\]����9�������7���e?�4X(7���CLd�x�>$��p���j@�_E�=��=��=��Y��ps����{w��5w�����������^��+��E�j|wT|��9>������`�u*Dc@%}�p���~���x�n�n���o��^]\���v9���DK�U�M��k��|P==(y�\�y@�k��&��Nv.������O�n�����}�_������������?.���������O�&���l���/�%^
�Y1��?Yz�����O�7j��������}?�Wc,��PD��<@����<�XW�q�h���?�|��<��#p)S�-�<��	<���b`���(�<[q?U������|�5����I��
���P����eA��/����*�a|~��>����qA�W�F��������FM���q�8H�[3��DJ�'��nT5)5��	|�9�B�
���$(*��&�!��i`�X� �,^vE���O���Q�#��V��~�� ��#��U�	����G�1���CA�w���p�g�%��zX�{M���V�t�����	��z@�g�1>������~�P��u$�I��`Q����.���Y8�\��*z��{f�VtO�-/j����[Q9 ����U>���oR��
��������
WXV�h�����d)S?M���TU��8�cu�q�%�PT?>��i5UVH��1�[}�����&�3�������s1���x6&Ef���B��?`���`Fx�jzc���}9�F���7<��#��TZ����1e��^��C�/a(J��IU��Bj���&a��?$��|���
a5~T���(������+[��C���&�"����������Q� ��> ���&�+�Pgt=��$�4V��9��e��XqZ�Q��v�a�a�$
��[�-z>�]�	���"X�1�#_Z�����(�KdB�����cl� �Z�9�u��h/�n��6�����'f���Q���db�*�s���~8&���A�&����#�
Y���\!���g��Z7����<�H*�t�.�<�D�QU�;g�h)c���n�q�(��/S����,�"��A���fn��/{-Q��B����aM�������'T;�����N��!������T�=��\R��������=���C�]W��/0^�2����LE�z�5������R���.��x�9��[_����Zb"�Gt�pp���?$_�9��Dy���Q-�^`��	���c�o���f����N�u�%�X.�M%A�_�\�3����m���|���t�&��������Fw�<//>��=��"��U�|����`p��w���'
Cs]A��w6;����A���&��s��������?�g��Yx��W@��&c�FM�����A�kp�Xa����`-�G�s6�Z�����7�:�>$��(�.��/��-�qZ�#�A�e���f'�5��:����X=���J�Q�2����]v�B��
�/\�����L��d8����}	~�K�����M<�W-q�y��f�cJ�7!W��YtOJ��
h��LU���k7��:���N�x�S�'u����$�N�'�"�5�N��^Q��s�!Pp+�~Q��u���r�d~g�-/�P���-bNa���LE�ew����D���p��� ��c��Z\`��Y�=q�2���2�>Iu�;	8����m��+p������"
]K)��x[t����?rU����u����y�_���VFD1!���&{��������><h�O�u���'�:��
��Wxp���85�_�0ty�]�E�E���kw!1���}3��Z�@U��_mP���3�h����&�C$��y��I����:�_=U=������\�Z������ ��s��l�C��p������_�}��_S���A��f}��$��\U�3���(o�
[���rQ>���������N�N��?@��|��>���h�/Zy-KAm�����E�bU�s�{����������1�:=���[�<����`���<c��c���MVH���
�����s��un<��S48��F���V��|��6�$*2���H{�V=��s�+�~�������|�@;���LLc���6=1�5���^AT����]��$2j�M�a,�-u��Rjz>��z��s�$Z���k9z���J{�h��Q�7�|����F��^e2��j��At1�NQ�����mw�b��T��z��8��1O����E��Y�Q}����t��&C>�����u�;8�J������,R�����dlQ����zyNz�==�!����^�������M�jl=�6YW�y��'��Em�7���J�f���z�Z�#!Y���n���|xBz�b"�NU��+�.�^��&`�>��o��$A�:�`������b�T�zx�)��5z�
b����0�^��Xc �{�9�tZ�<5)fd�5ff_�LWp:����w�WL�vC�����"5z���4��/��7C�L�=a����i.-~Rv����Z����s�g�Z����"�Q������f��6��r����@�;��������s�x�<��f�Z�U��)�Z3��}�TfXr�8������f�Tm��m���0�^8SnkS*�c-���
zv}�hi=t��'[��-���aq] �>p1��:G'�!�p���$�Q������x��RY����+�{������
.�)��g�����%m4��s�	�(��h�l��gX^?����E��j��8��)��`2�}L4��V`���,�b���Gk�s��u��EB[��Rw������!5)������6��w�r�|^����C�6
�KMD�DU����FQ:8���_�������+ZT:\��h\ �D�f�p(�O����`�8��s�-:��n?.[��0^�m\
�i���zr:jr�eHU;���xrN��[��5]GH�^��2�6'� ���~Nl�{��������h1����|~�����B�6yc)s2�����s��v�f���l{�X3���	[���Q����V���?������d��'�TH	[��<��&H�
M=C������9�����aM[���6R!�i+�4��|��Hp,=����n�6�x��l���A�kAd
�!�h0���m���������������-D��k3�C�I��;a7�x-���6��+���Q���d4�&��&���u�������<��U��)O������Y�h2����t���������I�������}m��W�����U���9�M^'t
_Tz�h�u�r�xr��l5���kb?'�cF��X�\nj5l
� "�6�'@5��`J��0�
u�WY�;�k "f%:��T:Xu������k�)��Zd
���z%��m��qU_�K���5q�#3��t�)��h ?u�M����.�I[-������W&��[�z�9e5k[�d���]������)�>��R1	��4)/�m�8��S��.�m�y� �4yd�/��&0��v�� ���.�s~�MCp�XHa-�y;(W�`���O����jR,�����Z���4��IW�Q\����(�v�*�s��e�B���
n��W.JN�Y�G4[��P�Jp�S��A�k���P�JJ��zM��*�^��Chor��Q����T��pq���x��(����m���5�������tV����*a��`P�������Nd��$��/�_������ �{�q����_���n��������x}y5���r��0�!Q�NCUpt >�v�W���__����ps��rk�^^]�_^_���}��s5����_o=�W��8�R�	�b���8/�eM��$��H�K>����h0��JDO�j����O[B�D���eL���F���?Uy�
w��k���9����5�6���{������������3QW?��zsq�t������x�|�����E�R������P[/n�v��r�QJ)E*B�t���E���kY6
S�a�&G)���$	����-�uq��������mA�Ku���Q�&��J���cV��@=�>��_�o��o�v?������;���������"���[��b� �Y�'8�����=C��CO5�` �LbK}<��g�t���o_�=���d�_��pw��s}w�^���������������S}�/�G�c?�t���c��p-����N{���
�{�����=.o��������]�~=��5s��{Ty�?�r�l��1.Q�����RP�G{�Z�$*��0��n�@�c3�6h_z� _��-gKu�-
h�,�a~��Ri�T�R�6t����`����.u���������.5|�R��C��m��6�?]t�����CF�q�������\�~
+�TZE!����dg�u8z� ���^����[���b�i!5�%����u"��=�r��K��D�rS��	������6��K��rtS	]2Z�o������rq�\���C*o5�t����n��hi�1�j���t������~���|���X/+����E �H3��Vc�l����`��%AG�h��%E6����
�v�����f�s����8�����)h�=�(��m�\�`����A�0uC��$H��w��	o8���X�d+�v�����v��������H:cN�Q�mt�[�k!�
+>C7�$K�=�r�v
��Z�]�����=x�Y����n��kQ�n�W����,w��%r�J�W�
�%c����;�k���}2,�d�|"��4��]�|�:o��'d��Na���Z���C����������h����F&�v-���t���I����`��p�a��"s]?�-�,wn������%R�O7�����i�sEv��
4���d�&4����)�]�X{�Gz��g��,����"[�?�<�b���Hx5����wbuy��c�f�8We^4S���������������������V���������wA��9�`K�B�V'ZV�q7�.:
�����0���x��6�
cO�s�qH����6�bO����M+N����-w�G���#�qS�h��M��,$.�V���9��<n����iKy�T[;��e]
�t�m���e�x&�!Z��>i��M�Ks�Y�n��I�xl���h9�������h,GG�%���Kv-���W��zC��q�0�I�����k;7�L�����w]�=3y0I���m������rO�>����r���.�Y��]������5�b�HT���6Bw���9I��e��It���(�>�B���sN�+�7ZQ���F�m:�Y�+.a���4P`����Qq�t�q�DN�*��&�tXN�F�c��"��5D�����F�ct���G�b)z]�P�����Y��>�s�-yeA��FN�63�1���������[�=m$v���tY1:"�B��1%(��r��I
.E��9���n��M�fb9��]u�4Ia�f|m:g��o�bh/��8D�k7���L�C]EN��9�6�t���p$�"*�T��m������z��~{��+���o��;�/stW��Lh}�6.�t�\����`;T�'���oc&�c�py��V�dz*���sUoN'�Y��_��k}|J���G^�b\�e7��������i:/�@.�}��\�T3�8��V[�j��x�[
�p����`����:k���8j�q�;d��I�n-���8v������C�	Vk���`����F�M���1��M����t����i�N��������B�-�����P�>$}v�^�������N�9&�j8������!5����.��Xs��aM5h����6�tZ����d-��dl��M���y��.vS�AR	]�p�\�U�n��Lg2����yRa��^�7*���1����aA���������i��>y�)qa4�\/�[��M����{�_�A����6���6��k��
���!�^��F{m�����bk�L\b���]����\��0v��X��F�r����i�o��oH�����7�HO�e%�j���O���T��a����r������F^��'��&'I��IQ�N���N�����Nk������e��xe����[v^�B,v�o�:|��H�itn�VBe:K\��F��Y���{�<#��p9j;���w���	�/')�;��������u1�#�o��j���M����ic����q��:ZU���m��.�72'��/;�k�F��������f��-���^���d��N��'�:������	�f��v��������������+����1.v�-���������O�r�������JK9�����}��Z>����Qzv������7���_��8t�����D.��o���M��J����w#�@���������l\�'5=<$ �?��-�_���p�����<t�x�������� �69�t@\�����������wy�K��('�����v��c��Cj�ob��Sg@��	v���>�_PK7u��7u�PK�j/Kpwj_reg/pg_part_head/7_2.outUX���Yj��Y��]mo7���_1�b�E��������f/���],�����X�<���v}����g�9���M��q�cyZS�t���7������wo�k����^�����������m~��xq��<?���{�<Zo
Zc��&������/a���ww�_&�b�������rsw�q�2`cc2��>��E�WW����g�����l�����~��o7�����s_M�<r��b-��_O��G����������o6;0��������\���;��������>m�n������?��./v���������|{���Z?�B> ����}�)��_��5�m��j�}�}�����<�18kB�K��,�������`����"S{����i���W#��
�?m�s���������m�O��W7"�m����u
��n�o�xu�;�d�
	LBE�E��Pt��&	�&��T��A�S
4c�j����N�d�����������M�����U��������_\�mo�w>�=$�=_�z��w�����W�P~�>v�b����m�w�O?���W�������f�{�?1�+&�h� ��Ak�F��o�m~�t|���~���$�����tO
�1<F��N�!Dk��a"
&�E�I.
(8�<�8>�,��_�����n~�<����\��O�xv�;����U|w�`��1����Z�)X��i���c���A���7 �}��=��{n_�|{}�����������O��z������w�{����#1�����<s�L��u8��`�^e&F����6�S�����������r{��e���/�g���n�cp,7�#e�R�M@�2�1>E����.��<���[��o~'+�j}mA������vyg���c�5��}��G
0\��\W�c�w#��hz�������(�����<�^a�
�W�~�����!��7L�V��g��OVFo��z��Z��bY��������^h����P�$�PJ�ld��hK���G���+�`�`,��%*��j��cI�ilh�a��.law�U�!���qb���5����;�pg	�a����9�#�>C4����J[��EI���c�����w��	�����$���LJ���J��F|�A2�zF�`���` 	��)I���-P�������#:oI����c�58��|6�u,��!u�#��I�X��XV�����hg��|V,~�����c����F�C�j�8�%�����y���$���C#G��hm��s�����Y��������Q�=8�%�3�;^y�=5>��V
��5�0�M���W��5��������p(
�(�����.�q>��C��7&zy�2�S%�D
Em�(�u��������-��P��=��S_!��m*	�(���N��O�	������dd������d��pW|�7��w�	���~	���gX\��z��k�����dG�Zh�����<���0��
��y41f���5�~NP���q�+��������O,l���~Y9��\��%��MD���������'Y)B���K���*k���E��J�%V��uj���Y�r�hF]�52�����"���|�K�3�,n:h�G#�(�~�����%�G������@W�l�S��#	���f�-����"bQ��Q�f0�/��'�GGU�cQ(h��Vv1���g"���o-��)g6��P���:��e�{��5��BQ�;�z��,�
^����;�D����bF<��Xn	�J��8�%
g�u	���^��������f)���^�~�IEI�'j����E�1��r��~N�4���oC��' �4>~"�W���6�Y������rKF<����d��OD������g?�n6������O�|$��~N�(���S���l��<���
�f�N�}��GJ&���x
����h��q>���
G�?��|��m�mI�������3�_����C�F�{e��k��h��Oy��}���Q�#1V��=�L5F]��d����3��g9�U~` ��N�j����n�L�%y'�y'!�L���g{u�aKF����2��F��Jf�p�;=� ����.����/'*��2�t�_����A��$�C�	BXD��E+��P�V���A%�Y���F�)�dI���Z��NY��f�����E�:�:�i_b��	Pe�}�7�Z,`saLi��em2��� O�(���d��h�'j-$a���xa�a6�L��������z�O�jD�$��a�+�?B�e�E��,��bK�)�����.������-&8Qh���3R���tB����q���;���-�X��'��@Y�=Q���tV�Q����]�Y�	s�5v	l��?��hb�_E�3"u)j�?B�����W����~��R'$��}�q����4?6�v|������$���
�*�Z��IP�&��A-�?`3�A��y�7b�v_�DI�'Z]^`�����uo��5������lo�P�q��9e����y���X�5��u_c�k��r��N���n����J5��F��S�Q�2�CT���-��=L����@���~a.�������Q���*�]e+	��������j1���G���&�@���WX\�U��!5=�]�����g��YIk�:�����n�F�	bQ�9v�����3�y�g
��u�(QBV`�6�H9��Eb+�����E��y6��t���(��f�/��F��hS��]�)�S
Qf�����&,�� Q������1�E[t'��]!����>��s�	J9�!����:9�������]�p?������{��N�z�X�O���*�J��<4�R�v1���K4��v��d!k3�_����]��H�h�������YNT���it�O��c��?�v�f5�=�8&���9>1R�f�U`��F�Pe�'��xib����R�4_�eR�MeW[��>~�"S����]��^w0��A��'Jhz@	B�X�3���h+���E�X7����~>m���s&j?�������yu�W�����h%z�l��X��hK�q�7�������N��%l����m�"�4Kx������f�M���H���#�cLY;	���S������gS"��2�L���-v6'��gx�9v����R�RR���P��[�����{o�qkb�%|]��
��c���}[g1����z/p�3�����B�^m����52��Y�"�,�r��^�c�\p���c��Ah~�4Nu��%[{b�r���DN���`s�I�����(�f�|>�{�������(����F�@Z�|��d���q��L�5t�����}$��u:�>�1=l�Gq�R�lvt:_|{	�u���6G#��=!��>���`���~wnX��������Q���zR*��q\��M�2���1B[y�|~�b��������8��|ob@;?���38�|J�}~�M�R��(����6y�'Q9��-�����jwmv��KU�{@)�9��6�t�>�&a���s�C��SV=�\����[B�D\�9��������f[e�O�j��\6'�Z������\Z��VO���N\��LKNE��Q\��h[��{��^y���6��<������6B��4t�����$��7dU9�1`�q�q�^�t|{�1�0h>Gs
:f_���si��M�K�s��^��So����~������R6���lO�!��)��z��Y��@H�����!0.�,�Ve�P����M��6�y������)���a��>����*����k�(��c��3� ���U�uFO>�+������>M�k�3���C�?�����-��m�w�W��e��M��6�6:e�����N]�\����V�N��=�����V���<�6!>�k�c��'PB�������i���L�d�$���G��U���y�H�u�Xt������~�3���B�}dc�A#��i�:��$���V������J-���{��>�}����@(�C;�����(����,'�N�b��P�����H������?D^b:��
1yB_��P�h�@E>��������),�t���6q�NU��Ce����0��EN��z��z9�)6��p�5���d|�X}��<��[�����s�i(zX4����K��)H��4�V2���V}�������(�cba���U�v��g)5kL-�+b���������BX����wB�M��8�Xc
E�M�6,�����E
�4��6oX�5Q
p�c��@�F��\\LXl�����[0tH@�g��|n�F!�rJ�������XAB)B�����-PN8�=XtoG���a�{����W��>��~{�a{w+:D���o����o���$�{�i�������Yn���������xu���ZV�#oM���,�}��g�m�j���������������������]\�>l?m^nv`v������eu����(@�
���4��7��h����^���d�����:����J��q������:
��h�y�K����F���>Uy�w��#�����ew��z���q+����7����?�~%������g7/6z��>���W�������)35���'6���#���^GCJ)�YH���D����Z��s�a�&{-"fv@�&�P����_V'���������\�73:���7�Bu�����"�����l_��������|y��m�������xv�\���p�F\y��$s�g�D~����'�8	uL�ah��R0��:�����wr�o�7������w�9�wW���~x�oW^]_ow���ELw	��(��&�U�'L�Fb�v��n{y�������7��z$Fr��^)�z�2K�N�����?��n$.d8m�WG���MC|�L�AR0���9|w����9ae�#Y�LT��N�����mQ�HT��:RRE���O+*}���2)���UD�/A�T�k�{ZQ����T��A5��k�wr5���)�e�,�S��)�����F��q*O,����~�����*�ZX�hA��;�^���6R7-�x�-����u��S��rt����$���J�reH����+dqq���1_jqa���zL��+-5�+�����@n��(�8 ��<^jd2+����SKM.!a5���Y����!�S��!���W���W����_���U���`�u^-�5 �UV���^M��#j���Bcw�3�u8���j�?ZXr��g�������j�[�%F[���c�zS���m���VBw���2p�&x�Mh )�k-�	���v���22
N"�#���i�?gd���q���]Iw'�Z������"!H��V"�n"X�F?4�.P��4�7~%x��[&�n--�5�E^�]�O�	��(�AK�W�M���%�c��t�qnb�Ib��Nf��-�co�a�	�u���d���.�����!�~��q���X�%t5�$2n-e�4d>'
���&��W�-ql��V��pbi�5����<���,��-_n�M�2X�-�:�������/�L��%Q��$��9�F�\�����#�$�d���4�Hj�����8a��:i�.�����N�s8=/����	��q8�j�0�N��0��}���;��R�1����	^JVO�^G��W�xz(�� %r_�	��W�lxe����A�5���
����@���]��bwWw��������.����:�n��6<{��S�]�H�J������HA���=_i�Md��9
e`�	��@��mu~by'��5>��@!@'%^G��h

���]�=�T[9��{���Z���x���t�N�k^w�r���0�H�U���5n9��9q��G���O�����q�y�,.���&�,�u8����}j�������5�l���;��|��"�����V��[k��Y]��yZ/.�5er�g['�0<��-����HtZ�d0i-e�tl�(h�e��B�2��?4<,s���
�"+��Rd��X��t�V|�h��z�4����tw�K�>"��D����wx���e$�Z*����IO��u�������=��^
�I����uJ��s�����%q�FH���&�G�������:��9lXIw�{��<YX�������`�7d9��s^��k�Z�,�:�������r���@gaL����w��Ys�Q��R�^/Ibw�I�O�j����7lv>:%;��;����]m0Q78�"���V\n�*�(Bs�����~;r�K�jn$0I����[�b�7�Ky���HK������2m"��pm��i���R+5��ci��'w�i�BB5VJ5O0im��m
��������S?����0�����h�:�wxP�>x�����5y� <���d[w�Ydx �$b�:4rx ����1=O��$&^���0Pq����6�n�aX��
������d�B@���&������#�Y��#��Z1����6�����ZT�x^��
G�����A[�t|Z�(8c�'��ir�L�mSeS�����-�.��K����E��N\1�����pj�A�V�����N���8��^�9�T=/W*��2��s��4���5�������v_��������(dy%o<�����j��l�/�
��
�m�:�/wx k�2��R����3G{��v56����h%�'\\�A�K�������������2�`�J�/���mwz�������4F33�{�����=S�Aw����y8kbr��.���H�$$����&3�m�E}�W�I�O��Y��Fc��%����~t����W=q�	E� :Z��v��
�3��F�����.��sz$>v������a`u�����3������o_�t�l���C��{$rm=|�`����(A������[�����"��C��*�g/�^���~h�����~� ���g �7=5	q@�NC����Ph_S���F!NK�b!�\�OEl��]�~k���r����g����=�o@m�-2%�����>������PK&r�hT�PK�j/Kpwj_reg/pg_part_head/7_3.outUX���Yh��Y��]_������7KUgI�:�J���l��$��'�F���$�nvW��>��3���fkV���8�X���
���nW�������~��������}������/�_��vO^]��A�`M��������w����/_���&���'��?\���_���e6o<;����?^����F�|��p����v��^�y��/��_.����&$`C�O������x�z����_�����������W��������O^a/o.n��|������������^�������~������{y���F��T� ���_��*��\��h������i����?^��9�[�x��Y������!���z-$�����K��i���WG��
2~���^�����Wo�:���}+d������N�S���������WG.:��brF�!\�:3��:=&4�Q������xiH	���������+��){�o_�o���}��~��~=�������������|��'�C��������{���?�����Wo.n���=��������~x�������?��M>����E���^�v/������������r�,_�����7���i��H�Z�3!��1�d�b�0Z���h�0����N�|%���@8�{���/��pn�v?������s
�����W��|���"�`�ig�.���=j?�	�+��0��(���^�� ��C��=��=��Y��ps������rk���<���������^��+��D�U������Sd:����e���g����;2*�S�����������������������j����n^��qGKd��u����
q��Gx��_����7rr�h���������s����k�6����`x�	��#A�^��S�P���Q�z��r�J���'���:���`��+|�����_�ZU�����G��+��E�c2��)yqVh`=�Cm��!��rf�Ol��^
\�� B��0���=K��lQ����<�9E����(O��yg�\��wxT����x�5Os]#�����kb4V4��yA�8�ya}�����3��`����(/2�����d�s3������xaFM��%�Jq�x�b]��bI���B3�Q��� �����O3�R�(H�_�\Q���p��	p�w4�L�>��]�����SoIP����U:k���F����[�:$X�&�-����!�����<*���A0U���t-�WU������>*�TT>�-1�/
����m����j�i=
���D�5��hz���`'��f>>��U��|f�V���7h����<*���v��
���a������E����0(�	�|4����L�4�w����q=!�������'X
�9��_����G�A��T��3����D�j@�X~yT���dE�{���.4�~�&�r3�+6�/	�]q<� �.Sf!	�~p�"�b��1e7G��n� .J�S���qD1��6�|B������|��
^50�~ W�~-�pQ3�jzE��7������~��zA9*�����s!��/o�?S>��L�HcU���^�Y��y�UE�'��Q>��AtH�E��sM��Qop�c>���*+�E�!�d��b��g������^h�z��)p��U=�Q��t��d^1����c��q�1�NT9�o��]Q�P�)nOhs1CU��y��������}��Q��j�M���CMr�cU�;G��b�x�n��}���r?9�4�@��L"����9�%�����`���[���}�F@n��B��??�I����_��j��w�E,k��N5��S�Q��j���jjX���2����kV�N�cD>O2�
�jf�`�Fo�`u�8�+�Gx.:Gd�����zn5�}�Z��s��Y�,�`K{�������&Y��h���c�03�=N�r�^�sr(b]!����a�(R�a�M��1��>�=NOd���XW��$J
:�'���dV7q��T�����_`� �>bY�hG"�r_P����djm!QM�s�S��Y�!e_�n���4��j�y�MF��5=�0�8�>�
z_C���B�y��E�g�w-���X������	�P|����$v��G�&��E�2�{j1��"�5�?�8�x+Z ���)��vEi]`
���
L7�|��5]�0O�h�eU=�_q�!Q�X��\A�d�S�hjr�\�.d`+������joU�;��5��>j���j�3,$W���uq��`�o�]�])�.�h����t8�����n�eF��rBQ:�t����
�r-�\�\��$,�V7�>���tn�*-����"��� �b�R���tE��9�D��H����(�s��<�-��E��r���Y5��2 h����n�V���]�]{��TU�q����X��h��@G����NP������*49�$V��8O��������h �:��4�qyZ7�&��5��PI�����6�a����%����6K�r�uA��g�PV=�D�{�����<����Y�{�X�j�#�!A��-D2�Q�wF[-F�S����j�:.4�Y4n�a�����}�S���u��&�.y_U�������XDb���t<hR�3b0�%=�&��.U�<i]�����r1��oU�sT��u�Yn���bU�sv�!���3aY�;��crA;�B����=N���_h��K�f���H&��j�5Y�FUk�h�v]L��1�5bV���Vu����k���V���9�/?��CZ�U��^�BWR��������#��&��h�3<�!���
�0u��u�8�s����5�Z@��Y�"�P�[Q��XC�k����oR����E��]U�a(��"�b��	����D�(���� 3zW��S��5�)n|����z�� �;;�#.g�YI	6��X�_���t�ub�rfW�_?�'8�V)���SlQ�����4�*�8z����}P]n�9C9�u�M��j�m���G�����Wf0{
�A_R���gU������:��&`7X�$.��MWF[��@�v��l�~HM��DW������bH��-r.�&j�S�#��:��7	����g7�}�)�'a���x�{���_����M�	�vl�_�9hp����S�����U��tZ9��������aA��pIpC���k�n-��c�������9���\6��,V���d���mv�R5�����&��O�����9IGg����)�d�������L�������r�N�p�6S��5k���g�w���6�r���{�b�lR����Z��q�s�����L�$�b������|$�LXu��p��R����u����US0������E�Sp5yg��KC(Y���5�^��d��*�����RM\��6��f1�U���`d�����w�C�xN\��>oM�`���U�+�����\0����sU3�qm��Oy.�D��fZ��'M�9��Uu��F�9a��	�]���@������'�-���p����+����5���������D-��\7~p��U���k��n�58�s��B)lr�%��c�������-��Nm�SiWn>}����lU}?���a�SKTPy_�1g��W��9XC�	�q!_�@����,`�c;?�u�B>�����P3����ZFo�����*?�{�������Twn�<
k��B�Vg+�=��+�V�{���`
E�biM�uUgM��qm�h|�Cqe��hyQP&�|k���;m�m��\7_�f�V��+��\��ih��E�b
�m6�B�����|��	c����������z]g�"��NI3\�N�{�������!_���1����#�UU��p��=�qL-B�H,�']��Y]�k2~O�k��i�kC6�4��_�!{�}d����XZ�:G\��:g�k��������B�r�,�6�{hr���6���9���[i������M^����1S���5c��."?���Y!�::]�V�-����M��.�pv)c��
|u�d@��5����T��wv	�Z�02�zHG�aE;�9�M���.�r���k���j�|�d������@rU���3>c���@�t��J 5i��"6$C��L��nhg��>e[S�LtT���6)���4�����.���6
���W���W�s5�P����bhA|���������������BZ�	��]np[���������.�����jp������ZL� W���S�;r	�.s11X.���v"�h]��a	���>nA�.���R�����7G�:�R���?������F;����	]^����*��O0��~x�v'2D����o/�_���/��$�{����w���r���o"e�
����j�k!��7��Nc����l�'�m�������k���77�.���������������ovW�\��x���������'*���Cj,.T�;����9����O�$3y�&�0�6_~�b��8���H�_��n}Y����7���i�OU��D��u����1h�yw��F���~/�������O?<&���7��\�>����.{�/�=��3��hS
9����OoA������ ��R�Xf)z�Q�
_�����I�������;�$)L%����uq�������xl�8��t�'��x������E�>��|�}���\������~~wqu�=v��?^����+��\tFnY��!��tl�9����DP����*��h�����R�G6��e:W�����_�'����>����\�~/Z����g�r����������Ip���pD���6�!����5���vi����w����w=R���������8�L�>���\�M��S��\?#�ex�y�_1��a]b��!wLl��p���8�5K��'^�:Z��0#��M��%*�
�8#�'{R���K*}�������9�B�U-z������n����	�i�_ql���+�K:�
�6���%s�i�����%r��������`�hu��"�c&����!wn��+N���V �0V���;�!��!��Mk�Uc�R�%���
sr�|�\��Q�-��!����Q�+G���T�G&�����^v����j���7�K{dry~��$���b$�������H��"!#w������������[5qV|nQ�Z�J�"=�X5�`����Aqlp��!w��	l����5��a�����.Z5��f&��Y�#��l�8�+YT����F�0�jZ����k�j�{*7"wn�D3pn&x-B��i#aX�j��\�����:_Y<���]?�j���=��v�B����&������>�2�(G�����Hv�Z6�����Q�L��h�V
M��ZZ��i��Y���
�VM'�g_Nk�L��sZ!�
�s�������5R��*��.��C�ng&,���m�]Td��{]���0
�K�����x,j��w�G�n����"��g����������0Wd ''���]AK���,��-_��pW���o$�Sg{�L`�
6�ja���8��\�B.;�����������Q���3(n%�A'6)�w���M$a��HXp���at$���0�j�3��F�����`�b��dw��%s�mZ	��n��v��[5�G���]��
wqn�\6��;j%a���g�
+�!w9��;�Xrq��.Y
:FcE�V-��s7��+���-���m��t��C�������8��U�J�=?���IrA����w�#�I��D�X��&�n���I�����L���m������s�?_c�*^�=>���.������SO�v�&+�d#p9����j)E����!i����1�m�a����!�D�������w�NwLE'�k�;m�6�G&w���q$^��%qp��H/1�QL�������%��������rD1]����&�a�X�H]�t��6�.X6oN�tk��>�i'r���7l��'���]HZ����W6]�y����w�+��W��	a#a��5���2l%��V�9�T���](�� M�0`����N�*F����t��0-*5�o��LW���I��
x���.8���2��5y��e��z�P�fZ����r�Sv~rMe�Ax�;]�v�}�e�(�`[�lr������Y�Z�hM6]��{���.�������tK�^�F
�b����o�v�.����� ������h#E�`���*�:��7"w������|
w1i������\�v�EW��iwA����G&w�)�Q��(�c�a���h�.�!)k+5�$�S�6���b�c�g��uz����g�n/�5��E
�����������C�4WK�#����8������r��Z\�k�W���m�8����J�A��I]��N���k��b8T�U��tP�\���xr���\<;��i�D������<
���p�a�6�m:$������	L����7"w���3c����uV$6�
Og�t�YpM���t n��M��|������7=k.�`�m�]�����]+!R��4��	�M�d�5D���7B������'E�+^�n�KVS�)���A_yH�K!i��VU�H�mT�l|�J����|�H��X�d�F������_vn�"j`/i��F%e�!�}0������W�b;�6�C������r�����e��������'~%K����Ep��N�I��s[�}\��l��g?{�vn�!jh�5�Ku�NK��rq\	��:g�[��`��99>+%�]��a���l#������e�#���mu�?i��l���C��u���
�h5z��3��V���<�x�P�h��DV3�t	5��ar^w��N�[�i�0X��u�i���S���_�8�l���tC��
GN��z���P=Q��������_qo�������O>`#��4���M����~!�����7�4�p�!���:.`�����4�t��d���t��&��uyb7e���������vV���}����W:������"��~r\����/�PK�����;�PK�j/Kpwj_reg/pg_part_head/7_4.outUX���Yi��Y��]mo����_��,�FWuuu�q �|�s����I���EHZ2$u���_���L��4�Rs.����]n�35UO�t�fS������m~�����{�����o�/6?^����l������&;������O�����~��_��{�M���'��>\���]��~!y��C�O�??����Z���A���m6��\}�~��o7��.��K_��]0���b-��_O������_����~�z��{��x��������^^_��=�������������N�������n����`���}sy��}*oIn�.�_��wo�>�u�m��j�}�}�����2���������������C�dY/<�D��%������:�o ��i+�{���o._���`�i���F�r6&����?��������W��#��6Z��$U"
��<E�Q��E��;z����E��N��_��J��\������~�|��~��~��~�/v������7�7�<�o���/���������3�)�^�zsq�t����m���O����g��7<����f�wN�1�'f�hz#����������������Etn���I��{����&������"z2��|��n&��f:�@Dc���������������_�����n~~w��m���9��������{�>�����������_��[X�h�$����E�+��@r�f_������������?�������/���z��������k����c1���VO�9����E]/�d����b���p*�]v�������������/��m~yu��\�69�r��n�����e�$76���{�
�_����w���X��,����������������W�_��m`��B/�:���������}/��"�)u��1c�z�����~�=����1�������������&�����MAnik�����z������+���%���S�#z�2�������)�bA���,����I����<B���Xw������]�BA���cC~�~�]�owWU��� ��D���pFa� �����*��=�����9���Wx,8V
�� a���I�A��&�iB'����0����{�G���ojR���7>���� ;!�Vh��<�e�#�H%������6/>�E��#��&�C��)�E~#a��	
�3j	-�vPU���,��~^�=b��WF%:�a
��/!V���5������>����Z�~�L�c���|xH=��#��������b|B�yI�
�G
f������ZrM�����?	l���C-�r����i.���|t��i�t����"{g"
)d���s0�$�x`v��� ��=���T�I�������qp�����&Z	�\����,�3�R��&���Kg���1�J/��cM��S��| �q�j&�a�j��X'��),��l(����8�h�C������f��`�����&�;����MJ�"�����L���d�<��D�
:���b��w�d�Pt���O�]���j)�]K�[�x"��^>,o�O�E�����O�]k���u$h�les�u��}�� �A��&��X�����|,Y�%�U��x�}y*��%UE�����z���iP��Y����L��&��\U�3�Z��}~�����Qu?��iHr�HM��������$�d�u��6�������	�����	j}��f9��.�hBVO���T]t�������f|�Z���g!5����ol��~��08	���Mi�����E���jZ7���1NO6���J�hP��b2�
��EM*�������Q+%��q����w$|�O��)��6e	��?$W�i���1�5Xr�=�	�D�l�AnD�~�"��2j&���t
��O�h�3�b��@�����~��Y���f��lM�����Mx��e��AI?���vf�
)�?q��7��r
m��\~I�#��(o��B���sy�����NM'��hQ��#g@��Yq�6��0jO�O�Bo�5�]j*D�]���)��h��=��]�����xJ������8��
#F^�=	�I0TY�.�X]���5��N�"�����C-�w��?��,z�6o�dh�uJ$Y������>��z4a��>��z��;��`�=4��$W����E��B.�)-r�(�|��o�*��f�&9g�����"��#����r��j%�+��&������>@M�I�xW�~���h>�2����"��Q��-h���}��WB35����@�������

r^cq-v�yk���L��9��'N��v��z*��#b����f-��9T=�B3���&f=U�<�A�����<�$?���]{�G���~�������5�Z��k?!���L�u?Q��HbU���%�<��}	��b�A�~���<�B�Fj������L?!��u�l����[>
�OYy���l�*��P����n�H��G��A�����DC�����9?
u5�����;�����[�����3����V=��g��Yo`�d&W�<zT=v]=�']=�u�����v1��C/���������^Z��nN���C�jyf*�.��;Qh��S�#���k���]Y]���g�U�4��F���/�>�@�.�/p�\�~l�q������9�L��.K U���F,��yR0����dCWmh�I��:�!�5_�h�5�������g�b�K���-������I=�=\%�Vu����x��
mr��5����.jS�`��-����8�3�y+-7�P��&���X�}vr�G*D[�e]����{"gv�I��.�,,�4�%���y#�r�t�l��Y�Pd�2�[�0{����c]���q��5Q�f�E��v��(�c6�7������]�9:����3/�u��������8��)�h�c�C�<
u�\�<��r7O���:�K&e]�l��V�yx�G���x���Cz?f/#����E�F{���
�Y\��Czx����!;2-�n9��l��c�>6JM���P��$�)�.f'� -�����cm����@�U��&����f�ay,�5�rVm�h*l2�������36�?$x��,����c���H��#+���
nF�����������3Q���x�f)����M��!
��f.��g��,'C�]��D9R�Rn�3���Cr9����V��O����r�#h%���=�����L����Jm���l$Y:��,$J�Y3��{P�2����O�(���^��s6��\z�l���'6��K�:`x���z���\�s��N�-&��W�+&q�R�l�uD�����o"%����S-b�5�����[H}*Y���������a�9l��s�R��6N���|m�S(r��� �n;J(��x�f>'�����K�d���+��-�N�a��&5���*����V75eC�0,��V_�mj�5O.k�LM.9�����i\����������+��Rs�qZ�9\�<h��k5��-W��X=�\���z��;��"�u��q&��Si����K�9g <�����!U�'q-�4��P�[w*��� m���Q��=�mU{?�k#�l�����-�G_�3H�zm���=U�g��b���8����{��I���"Y.�5y�Q�|N��k�G�{��5~_�����|�Z��{'\�&�3�Z�<���[35��r
��p�kc�5C����6M���u��K+��S��;J���t�#���v��>����I\���r�l)����f3!fu+���Ok��4S�E40��}\^`��[�������wM�}���i�z~o��}���I�w��.�|�P�<'p]{?�kIg4fyLmB,a��?'%�1;M��9`M~��qmUf��Js����D48���hsD����i\��<C,��;�!	��Q��A�:��&���VC���J}m�z������Y;_3�v����%���J��;|�mOq�M�CU�G����b���c(FW���3@���or�����\hkM���K)@M��1����M����u����z08�"K�T���
Dbv��@�b_ C���J�'�K�54H�3-�|�jl���Y��I���vh�3U[�$����8G_�:��x�3MnR����F��bG>�W�Vz��6���S��j�X`g��1�������'�>��
��n�57Q�i���B�v�7��I���{��l���C�W�:�����_n��!���:X���Y|�F!���RA�����#� 1
����=P�P�u����&5��`��H����	����n�nE�AWU|{q���V~!��Or�77���m+���-���!Z�]��.w�U�S����y���i�O�������W���O����]no����������������n����^y�X����J�ah�p
A_�Sc�X�����|�'��������6\�Z��.<�����&���k t����/��{��������������}�q���`�T��oE��|�����������Z�z��������^l�w������?~��/����Do<+\��@��_:�SJ�/B��t�,��~>|�����Fi(^��EB����&�X�8����:�u~q�fxlA�%C0���PBG��G�_��0�N�}�u�_��y�������b���;��?������":�
7o����@��M2��hQpP����\�a�h��X��;��H�J]<�@g�tn��n^�5���d�^��p{�7����{�����<��+��������"&�����c����
��	���H�,�A?���ef������r�u�]���bAf�}���M'?��8��P���
�W���}��#�;J
����cl��y��>t�����&*MD��D���W6���\Q�DTw6�.uO�q~X�3ee[V5|��������UD��**�����D������G�:
F@�ul\�b����3��<�bkA�x�w�}���P}��� |]�:�6�J�N��������
QuA�\��nNR�oZ�W�$�����>��4W�1�������l:�#��������6��]���~%qy*n0	se(��.E!�����#���dsy�]ZD}� ��v\G�8g��2y�A�3$�����-&zdq��M�)���%[F".yW���l���AGh���1e8��$v8������u�bLhG���
r6K�B���Z@�u�0������+��V�#���uEr�VW��dtn�*�N��X��;�P�]y�X�4��3~M���nX��*��]qkv%��S�����Q�QS���V�Z�;��|.��T/(��k)�l��#�;�e�:x��+���(��_�&�~-���F�1�\I���s/�S��A���!@�,����5!����Qq���}N����A����B$�J����2{��[2��t���TcXIfLY��;SV��Q�{Dq������-s9op%�@���4�:����L ���c�a���^<Ep��;c�������%O�9bT5^����-cs�X,y�$[�V����w9��X�;x�kXI{g�OggsX=j�oWA3��`R�zS��*-D'�m�K3,���X�[���u�WR��k��-sT����l�,����,��oK���r�]�_U�~%k6�m����X�@L�����~�� �e�=o��Qw�������J��T�� _c����x�8l���o_���s��:�;^E����f�$����w/oWy���H=f��-��u"��������<����R_+�s��x?�>�<q���J��_'�o����g���9o�Z�w������u^np��@$��n-xg7g(���:����WJ������8cW�J&��!;���g�'�b)�H�C��,��oK������i��yw��b�zVm��{^D{K����J�w&pC�_,���k�����3�
����eI�uW�8h�0~�t��8/Wp�\��j9f-K6������x����<V2dS�&�p�}U��i1H���:~m��q�����2���2|����������J�Q��w�����4sBx��M�����W�
�9�%q5�ebX�L<�6��d���2O������C/\�/����z5y6��c�;��,p\���!��p�;^�'�!��P�]W�dW
����e
�R�B��H,�d�f�����}I�n�����Ws��tEb�0�J���J��2`n�J�x�������-@�N��]B��NL�[G��s�y���Mc)o�n"h��J���!9�����I$�J��x5>�qO+���jI��F��e8����w��H����i���l�N�����R��&�{��{:��^I���n�H_"���]o�R$<��p�%:��I�!�)Y���x�������hF���u���@�fq�)t��:��������{��_�mo<i~�Itv�T=��R�6�~�g�j��L�u��x���@�VKmpQ>��x�����H��n����u�{��J��x�x~�A��]���1���������{��tJ�J����$�tt(��y��	k���3V�����
���k���k�N���k�;���9	m���5������c�����{��R8��O<y	A`�C���{�p��Z�aO��J���4�}���C���1��R�d�>d;;��[]X@^�$�df��4P���\����`V{�f��*p�m��V��L&3:����D{K���
<f��������IgB������W����/������hL����:T��'B�����:e�0[u%=t�\i���0����L�l��w��W>+�u2�?c���D���o��K���J��a���3���2�(�'?`���4����������d�z���E�MB��P���`�����C<�v8�83,���xq�<��w����BEZ~����b��?��>t���[���5�����!����/�PKuz�/d=�PK
k/Kpwj_reg/pg_part_patch/UX���Y��Y�PKti/Kpwj_reg/pg_part_patch/7_1.outUX��Y3��Y��}]�%����~E�Y�� ���@���z��x�,�I�i��[�=��3�_�q�������Y*2���aQ���[����8q��n��_�����������������/~}����^�������/�~���Hd9j�bP��/���{���?������>���d��������^��������ol4Fy�JU�2����o��?��B��������Wo^�~���w�{����o~�����/>��>��d�J���i������OBk��������_��{c�����������������}�����_����/�����?���G���Wo�_}���{��z����^���~%� ����?��������>�Ow��o��U�����^|���������4%&R��o����
`�w��_@��(x�e�����r�����^�x������\�;������?�~~�?��^��i?Z�3S~3�U0���I~�h�of�I�/F��f��7��?��&����[��lP4=j����uX���N9J�"�pX��e'����1KY���mS}�?����+�o�^����l�?�����|Dv��_���#��~������L�����D�|�l$�Z����>%�xI��:Q���^�Yr�E���e������������#�������>|���_�~s���������������^��x�^��/���������������������w/z����o���w_�O������v��������G���X�#V���������w�����?���_�2b����������H�<qy�������������5��k����!���h�����_V����O��������0���>����w�o~�,`B�d�o!�y4A^�t�-��1�d�� E����,����m*'!X��l����������A�E��h���(�}L�j7L9�OP�����.O��i���/��.�����_�������������o���|2�VZ�)8K:*6YK�������e������Z�h���N������O "_F�o����������o{��y/���^�x��lY��;���lH���8m/w����&�&>zL6;hJM��F���$�����������ZQ�������1��z�/^�x/q�L�}�~�.�����������J�$����=,�DC�EH�'��������D�`��ax�l�(�x>P^�_
�e5��6����>�
}�A���Z��<^o���������g�
}X�����rJ�o�����9�O�-�Q�y��^��f#	���)���~�������K������"���$N�S�KG*T�)��=x�<���c�1���;{�RDQ��H4�����c`��R��4&����8J���o/ps�<���`����6�~��5�l���R9���[��Mfl����;_���6�8#x)��Dqr]�U8��q
�q{�����$��&K!O��@T���'�\C�u���9�-�ZB�m')��������8�V��8#!�������3�\��u��$5q��������PLKf�>%�z�8�8����3Q�b)g���,����s����:���g�|��F�!�,������8��qf�u�$�R$@��g(��y>=r0�8c	�����[��5-h�pgx����I<?��R+��Z�N����:�P7��8����.�3�?g���C��a~f�?#�����	m��N�)M~�y9>^���	
�	{�L��� �o����`��?��1w3TYG����g��1"ypesF�	��	��.���ge����7��F�"@e;e�\�\��,#6p&6p&��3q-
�q���^���EN�z��Nef����`&hydxnesF����/!�\��C����8�i�SP��7%s#�����61j�n�o&�n����=�&]��[�#���� ��LL&�hl?c�u�ab�-#,o`�$=��X$�/�dM4 ����v'-v����V#��$����NN�)19������h���F�jDT�$�n���#b*y[V�G��x�h����o�~��(�����f��Xe~l�n�#W9_N�#Ms�&������T��~��O2���Fi��'9���cUH����2K����Q�B:[�ZV�tu�3"�2
����hlV�������K,"��D\*Yq����3�U�a��L?�G^W��,?��8Yqf]^��B���;)����	^a���RLC�����h�F�9��snF)Z�"���]B�XX�s����@����A46�Dc	7����EE>�[cp^.�g���>!�n����mu�G�,��������N�>+D��F�A�E�/���WTTb/�����Dk��k��Q+_2?�������#h���66{�������v��,��
������S��5VYGA�H�3�
4G���S�������O��}V����4�Or�u��X}��X9����<y�t���#h&�i�������^
����l�X�z�:�|�\��m�bfmPz�~�*�(�����������*����GypZ��I�g����H��x����h�A�T��Q�%`��n05��ln���{��_�C���AN6{���9�Z�
��E��D���$��S-����:
�8��2?N99Y�V��K3���a�9��w'����f#����Q�N<�!r%�,^�{����n1w�>6�A�X�L����,�=(����l�#���Z�Y�d{���3VYGA�[����y9X�3,n:?��J�����I�g����Hh1�1�0'����s�}tKp n��$q����k>J�32\#��Ad6{������%����0.�9[O%��k����>ri���D���*���j������RM�����y������z�����������@F/��ejV�1)�*��E���e���i����fsEo��Y7Z��,������~�*�0�C�O����D^�
*BhF^hd����iwR�Y!;o6?�G"��&~�&�s�7��^����} �����\D���I�nH���w6{���5��yy�i��lH���9/�6�3VY�A�d�E^����:=j���P!ad��I�g����H�;���$��yA��D^�?JK���}#qD�~��>&�,��dG��m���tg{Mwv���&kY��r��3W�*z=�Xe}������i�������m3��
HM�s�����IF��^`
���5Q|���yr�����B��ECf��"6�
�����lW�����&kY�%|q�B�P�g����>����N��9��X��^/'��CNu��}���[���3�y��j�����H*�k��`C�!�.�R@�l�^O��XT�#j^��v�{���5������Zd�CF(r
`8�~�*�(���t��<�A��q�7����[��ewR�Yg;o3�	��K���>2t����@��:m�!pEs�b�TYg���i���lg{�vvF+6Y#��{D��h����>A�EG�s�D��uAhF^Y�*��������v�f$��r�]������\���3$g�<*c~W��%g/f]w9�,Gk�:����S���>
�����l��������Z�x:WZ�
d������s����K�����6#/o,&�}�����Bv�l$����{V�����D~��T�mu����������<X��n6Ve7�mh\g���^s�D�&kYGy���-+A�g����O�/����O�V���c9��E6.�����*�V#i�o��K�ZCc��mP���	:.Yg63#`���uy9o���`"�!���:�=��v����<��3�"X�d�$�������:�X�/��,�T���S��#�@.u��ewR�Y�:o5�F@#�$����V��(�y����AP��s[��
�!-��h��>
�����l���N�z��-kYN��A��NC?c�u��������t��
����ab�e��}V�����x��B��x0�'^�'�)*����G�:D���"�cTk��/��\V}\g���^s����d��u���l��\?c�u�a����^����"�
���=W������u�j�����^��&w��
+mt�"M����w���3�&Y��Cj^
�����lW��������e�9��7������U�Q�����
}�{��}���c}l�.�����������H�y���X������^"/6��L����Z�� �0��8��R�(,a��\g���L+\gJU����H�U���Kp�\?c�u�%�#�$3�T7P�f�������;���>���'	|��%9Zzt�EP:�}B|��i�W*��2��������Fh�R��L{p�i������Z6��~���F��Xe}B�~�m�Y9�+�M��M�%���I�g����H�:�OB������Dy�`�!kG��Tz�bj^����Hj���O��L{p����T���eA���b�'������O�����<a�V�����H	���������
�y���e����Jv0�Y��>A"����1i��e
}.|'��j]�C�R��N{T�������7Y��99���B���3VYGA�@��Td�]*K�V�M�����5��;)��MG�j$�
S�q'xA���r�9ppU�'V3��k^���hw���R�>��;�Qq����e�x��2s"�G�	XrE�z���:
��k�O�9��w	*���O�q-�aHlg�������Vq�j�i� �f1��:��;�mK��T�yA7A��8�=x���
<�mH��;�Qq���������kY�rU�7�G��(�\c�u��r�;�
y���S����w9�s��}�*�[�tAv~�6��T��F����pm����qy�����+����C��;�Qq���;�h��H'#^/�'h�
�~�*�(��lK���s9�U
����2ee����N�>��mFB�EmC�G�������}�������x���_�����Tc���}\���������;jXf��l|����U����*�(����e��u�K�6��v�Y.tw_M�s�����IFB���|�y
VV��>F������M:���;.zY��x�\�mhy���kT��w���N�"��Z��$s��bs5T?�Xe}��?������}h�Z�)��=O�$D��=��+���	��T�7	\��������X��&�y�X�xS���i|r���)
lF�]����(���wCV07�-��m�w)�S?[�u�	�����|i����PSW��G)9��;)����7I���b�
z�Gb�\���9�����8s!���s�p
n�����[�8��pw
m1���������M����D�^pH�3VYGAG�F�&D�`�m�N5M�'�7z�������-��H}
��w���0�T�	����bJm����V��3���u���ih��=�����&8/\����Z���JJ[��Xe}R��_2��%*.��;)��
R�>����MZCCgp�iP���P�os����V���s��}���Y���ih��=�����X�l��\;�
P�5���������1���a���J�i^�IAY[M�������F�r���~�������g	j&y*S9��1�H��Rn�N��#���s���� :��DgC6.
���EV�'��5�b��UYG����g��_�M��T�oewR�Y�9o6�U��]����X���������2���os}B�)������i�4b��k���<g��,1l��Z��
r=-��m?c�u��}7��~�+��d_�wiE�h���I�g����H����vl������,I��Ip�6�9��}��,\\d
�`D�(�<g������=W��7�E�x�1�.�k�F�g�����
}"��]
��V���db�#�Vj��}Vx���mLQ������%�P�}������^���F���!��IV��2iL��Atv{��
�9el2�����Ru�*���Xe~"wt~BVf����n�-�>!�7�N
?+D��F��k����z�K�e}=���6�GPK�A���P���>I�����Dg���_�=��n��M�"�����������~Ig9M����u>����|��,-Vv�D�������6�\��T��-�
E�0$�]������8`��6������r���nD��7��~�����tK��\��G(�D�>#���U�Q����S�p���H��{�=G�c
]�>^v'��5m��F��z�R���SJT)�%�������<�-wy��q�����Au�{P���e[k����@5:�����Eh�k���>��C���2O����mW��]����v'E�5��V#!M����|�>|�[S�m�d��v��n�^r��C�'9�+*��!%w��:�=�����LP+�f-y��y���NE������w�:C�vP�+/�h��}�>���~��}V����4�L=�c�����z��rs"U<��l����t�;�S��
9�#�>
���������1��d-�b��uU}`e�3VYGA���_�K��W����y���<[��N�>+\��F���6d�����U���v�V_p�.�$f���������p�}�����:�k�3$�f��mkT����s����:
����"/9MS�9o�S�N;S�*J��+������mF����bQ<�8���������8�^��E�	K���l_u�|���dg���_�:���7Y�4����(,R?c�u�	�qt��ES��*����C�%�y]v'E��9�[��a�q'c,�����'W}�QkjOq'\��1�4�%<D0���i���lg���L���mk���{��y=�Xe}��~}^N����m}�76�
�#��N�>k��[�d�G�6�2M������j�KA>B����(�y�c�{-��Rqo���lg�v�_�"(��A/����M�:��\c�u�����������5������u��N�>�s���*�r���<�����e�����L�"r��I��1�<�bl��0�����L}����`;�k���o�g�m.��)
F���}0����*�0��G��<�w�V=��u���ZL��v'��5Y��F2
aFZ8%��X�G`����\`o>O=�M���c�$~��*�C�G����v�=��|�v61�%���"L��X-�q�*x��Xe}$N�'*/o`���T1�d����SG���;'�<~_�d$�Uu��Aq��!�BF!��3m;����0A�iM5H9)zq���{�����l��E��X��KdF����~�*�(�#X��n�����^�|�|s��#r���;)�����I�d����w��wx3��Hh��%�A�����U!e�t�t%*/?9b�7���������,��k��g	��w��3VY�Az�����X�&����������Fv�j$0���	��G����T��z�`t7��D�tN��Iu�gL�� ;�dg^!;��G|�Z����,��$�3VY�AJ�)��XO���a�����jQv'E�5��V#!��x�<�B���a#�lXJ�B�y�h#�\�6wzA���ex�#D6��v�=��|�vvh]O��E,V��/$;�����lc�u������3n����mU�v�)���"(�������V#Y�V]N��"3���c��(�����%5��LY�2�����LN~H�7�������-������m,B�S���CN��~�*�0�m��{�*���)�Wtl�}2SD_*��;)��q��	�D��>��L�I�s�u-�e������F�2�K?*�;-0�b�4��������l��f��l�kN6�F�������O�(������f-���x�o�}������I�g]�y��tQ��Y?���nuZ�R�BG�\�� ��9�J�J��|��%`���F��r���{�����Lvz�d.�>�Q
.�R?c�u�aq��^b�)��m���M���(����;)������`x;/�$*,��f�_�T��>�v
98�.1?|L��),�
����� ;�#��<���7Y� �
�v!���~�*�(��M�9H�MM�4g�r�k7��x��_v'E�5i��F"��Y�%����)���Ey;j��h��9&G	��1�%>��:wHg
@�����:�#�����������Pf!v ��~�*�8��o�N�����*�w��������Bt�l$�"���bQ���!V��)���\�C"��f	�f3�B�-	�VM������9��t�{(*�6YK����i�:�~�*�(�#'�[�'�!t��f�6����C��-�sb����IF%?g�=K�6��	���h�
C�����8�	=����ej8�k�t;D]#4h�a�s0W�c�������rGSQ[��Rez|���er��������p���ewR�Y�8o6�D>	�]F����oIkd����_3�0�q����(�H�%���=�����q{p��]���q����E��^�ht?K�u����uL5O���U�
=���)s|�����Fp�j$+��A# 4������a@/������>1Cz��d��%���9���Cr7l������ 	K�U�~�*�0��:�j�J���MA%�hR|��gW}��zV����d��&F-�8��GO�c���^��X��������yQ=C�!1��An{�����z�7YKcr ~c����TYG������%��Se�4F4�\O� "�^ewR�Yc6o5��@`.�?6��*��5�Y;3�
E��
=h�d��r���F�(q��9�Am~z�Zi���"T,���]?K�u�a�OB���������R������I�g����H^it�Xd|:�/oi��R]G���������i�\�3��� �������9�Al�z0��znZ�)�p�`v���Y���@O0�|��7�1�JLnU�f�����v'��5��F����Rp	��r����.�h���a�F%J,6A+
Rs�����`E�f�e--���=�����*�(��t���&G�mM��vq=(q�iwR�Yc4o5�U =I�%!)
n%���+ifv3��)pS]�]T	U8�A o�4��q��Fh0�����.f�L�d-�AP�*P�g���=��n.��,��P��Bhk
4'6/��B��y��H�������H�uC����r=��s=�Y��9�IkVb��a�4��a:s��3�V,�&k9E9���kc�Y���@�����d��y[K���G~$e���;)������P������=V�j+����fN���z����A.\��n���l���9^��=�q�7YS
��[C?K�u��)uc3K1W������F�K����.�sB����IFr��z��QZ.��,e��y�x�p��������qT:��i���l�x�fN`i�M���gJ�U�;Z���@�u����x�)��fVy�mfP��1�_r=������e}����A����FK���������t�;Z�]R�|�1H��XQ
M��4��q6s\a3f/coZK�'W
�s�a���TY�������Q�x����*�����H�����I�g����HV%T�H'�bi�[���z�Vfa3��������Q2����=
6s������	�oa��,�s�&��=�,U�Q���e�����*W:����l���sW��z����	�qB�����p^O(�����!2�i{=,~�z0��l%�EaH��`3�=������fk9�)/��I�~�*�0�cL���q�k	����-�%0�N
=kl��F2P3��z|�1�t�K��b0��./�pS��w���K~�����<��l���9�����&k�2��0��Y��RezBO�R&�D]S�OL�}�Ly0�ewR�Yg3o3�Q�3��C#�X�����f��N3���J���C�0���*>����
6s���W��:�����8�`h�W^��-U�a�G^���Z(+u:44)����)������f�n$�P\Fs�>5�[�q�b6���J��k�v�E��S*\Q�2Tl�@O���`3�+6�����m�������g[���@�c���n�[f��j�8R8-�B����C���N
=kl��F2*�U
e��$�4R�R2���l!��
�������!�SM�q~H���f�{���5��&�M�g���z���L?K�u��������&�l�*5E3L��<\��I�g����H^�u��C��{�R.Sf�������F�YbW(�^�	�@�FN��i���l��"�,o�*B�ZZa�N��1��Y���@����z�@,�PoB;�0��s��z�����������8�}����.�/KW��	=�s����0.(�[%F�l���9]k3c���nt�Z�6�.G(���TYG���_#������)+M�'�������;'�<~Y�d$W�d)����f���Y��u������%��!.��A���!���fN{����63������uux=����~�*�(�#��-���S�:������A�|�^v'��56�V#Azi���p6�
yD3�"��������7��pi�ZK��;�}45��i6s�f3G���3�����z\�;��Y���@O����rY����'�L3��^�ewR�Y�f�j$	fb�"�[�����1&�^�����D��Jq��]�`;��fN
6s�������^B�J����z������L�,U�a��b?^�Sv�`�Se���CP��������f�j$4�(�'_���.L�����_��k*�^9gJ�U�#<
�zl���9�h3���q�7�e��"^����TY���O��/*���_l���T��T��I�g����HQ��M=��.�tQ)D@�{<��1�B��D�����V^OR�J
6s��������'��&kA>���V�\?K�u������t����Hj{=r_��l��;)����7��!$�,7��>�
�Y�0����bFv[4�J�F!���,(3D�'5��i6s�b3�����[���?�G}���Y���@��������].�`���U_������������H���&�h�0�M��H�j
H���`���9\�x�j�&
Q)L
6s�������	�E�&kiHe���Z���Y���@O�~WT����T��E#{%�,��+\��I�g����H�����rf�63��Q����R�2�F#�!S.o'�s}���4s����`3�k6s�zL�Mk9���Ra��~�*�0���O*,���=�|��Ri��|��z���[�$���ifv(���@�	��s�w�43�V���������`3�=�����Q)O����:���h���,U�Q���M4S	���U�����r����N
=+l��F"�e�.��0������Y-lf�\zpm��.��!WA�p��3���G��k��Y�&���@�A���lc�u�aG�z)b.����`��F�M
�5���>Wo�����K����U]Lts��/�f#�o&�yC������|��.7{���	�#��Ai����5�f��6��L/�`�������~��<Dt��~0�&6������/�����y���������2��ts����\�d{�7z����-��!���B�!�F7h�����Fl��}��iq|P��&�;��Xe~��?h!�gl����6���B}DM�����y���0-4���VR�2��\���IT�(��D��C���c���7[��lt���/�?+R���X��`x��C�W%��lc�u�ac���@y&�q�� n���S)_��Y�g����J����X!w��n��)B�P�$����i�(���UcL�Rx8QD#S�n����=�g��,��2����e�H���~�*�(�:T�z�
s)��ua��L�����r��~��[�d�4iTh�`��YHC[��R���c~0\��?N��\��4(�����"�,h�M#��'��
�U�Q�����?rI/���J��m����'��������r�j%+z�KVi=8��TN���t�R����P�Y?N�&����e���4g|a��&:G����F`P����R5'���*�(��C��6�`�I1^/�zV�iz?����Z��;+���6o��Q�f���Ep�<��d�������!o��?�*_�w9��3�Au�������>��sb����U����*�0�����'�!��L��C+����wg��5��V+��?��F���|�`�rK&����s�Z�L�2�!QO��Rb a6�g������,d6L+��wT]��h��?�#��
sR����k�O�'F���;+���7o���s�'�)�
?8Zb/�y�jq��Q]��{g=J��t��#��sM!�1y��_�zsH�=n�2�mk�"$����2����*��c�]�%���3�t�iw���lFK��f�;H7O��OE���;����`'2�0u3��`��|�Oi�Z����/$H���/�m����V>�������_�]~��7���gyVl����Ge��x�����1+O��!��������7/�����{��������_�}S��Qo������~I�������E����9�����+�r��G��9\��y|��C[�zBZ����?�o��%F��og�	"�i��t�m�B{
�g����
�����n����~��o����~#W���{����_���ww�w���o�]����t�7��\�[�&Lb�2����.3*�M;;f���u�^~����[}~��I����-Kc��o�-|��m�E������[?�w.�����>||��`���h����O��w<:\���>�����E��k ��%�(6<���������12����
{-�{�Q���#c�-�����=B��5����o��z�c�����!��h�~����o�����l��Ey�s>&������6_}���PKc���2U�PKxi/Kpwj_reg/pg_part_patch/7_2.outUX���Y<��Y��}]�%�q�;�}�BH (,G���V^��Rv8��1v���gz�������p���*m�AN�����A~�<y�l��������s�����f��p�|��#�/.����������>>�:Y�������5��������?�|������������^��<������k�\P�$������w���_}!����_.������7�uw���w�����~x�p��N��*�������?	����>IY�#\���_u�7����W?�������_~/����><|��������_����?�����{�p��wZ����?�y���J�@>���W?�������O��]�����W���-~������?�}��iJL�����i����eE\=
�m�V^0@Y����_�>|���������uq�����O��\��>C��O�~�>�f��f)�Z�h]J�����o�C��~3�����S������A,'���%m�g�,B>)��XD��QR�u�^2���Y���oo�����N�G^�����_>f{����w�#2E���[��?_}���O��������\>S6{�l\ch�t0��b���V�������2sY����w���}����r����}�����������v�~���^��q����|�������_~9=J�������7���o��G����|�����������S�W����������?{y��<����b��~�e}+��|+O���}��_�2�m���d��'9R,O\��q:�&e��7!&����;~z�l�FG��NI[+�:�j18G><���U?�����������y>��q��#|�������LH���b����U����&���	S���\bm���<]���@����Y����rs����A~~��h���������1�������U�-����������7\�����{{����_���������������gIGe�*kY�����)7����Xe�H4_���0)�_j#�'�/�#���]�2�������^�a�������#^�1k����/�%�������n_E���Q�'O"�`�������F���$������=|��E\-��(����xp�_�x�^������ ����o����~c�C��}}����O+nQ`b5��Pv����3�����S�������h(&����K����BdY
��
��{���Bo����e����8V��~�*�(���wCf�������:XLrY-�����������F��F��DY	�}��Y��>.�D�i�Y��$�)��1I�4�G���*�ko<��x�)�+w����g�	��%��u�SY�����9<i:L���q��"�5l}KG��r���Xc8�M����f�u��|���w�`�&��3N�Gv����M��G�� �f��PG���7�3n�qW0#�WY��J��X<""��Nef�vNw����a�A
I�JA{5l�3n!�Z
3.I4�X���aF"�1/m(D9H�@�)��2$�A�%m�s�����2��2~���(U�5���T0��|3��SY�A�{�L��Q\w�X�0�F6gD�2+���(#���V;��������t�����"��9�������pfx����I�>��R��G�u�H_n��3���y6���H��a�_3p�(iEQ������#��0#���������X�{�����=`&\��`�K�R�D�3Ee��f���3b����$NL��V6g����qr:�6��>�LTnl;���6���Q.�9xa��2��2q��K	`�ke=��\�����f����2r��v1����_��e���B��N=���pw$slt
�T��d��@22�A���f�x�Jn�?�����=�&]U�j����G�b+�f������n�`��������������^�|�����U���^k$���ldp�b���<D�9��^�8Y�!I4 _��6Z�Av�F8����^F����� ��F���k� �*QF��X�~�*�0�c#uC��y8X�VL4�M�!	��>ewN�y��>�HZM��D/��}�
�0���1s���n�[�6�%2��Q��������4�f��Y �ZS>c.q~��5^z�*z��Xe~��s~�u��d��k�����*��Bjtx������D0^k$��b�3]X���������V]��&"K�3�ea��|��{�cl�O�`l� �k�1#��WY��E+d��pz���bc�u���[���.�i[�����9�Hf�I��I�g�`��H@yW�e�3y[�G�zY��T��f��]�rY&��������oh�i���,cs�2�^���*k��W���!hJs����:
�D���W�)����B�Z��O�Z���I�g�s��HZY$�%J6b!�-�%?��Grt:O��7����5J�*V 9�$�j����csE>	u�����`�|���*�~�*�(����C�=���
*�����f�d;�N�>KT��F�����qM������=:���/qtm\2��9�g���V\gR�&v� %�=H����l���7��������bcWU_l����>�}���S`�L��Z�iU�#�$���;)�,P�W	����^c�E9D�����"/��y��O����\rg��6V��&���i��4��f����*[��ad�~�Z��S�#�;��x���3VYGA&���X����	��`q���'���������sy��������*�.����3+PQ�>��$87ebb�I%�
�x�f��������0�=(����Rn���Z6N�,��>.gb?c�u�	.����sH�b[�f�Y#I	5�qwR�Y"4�5?(��(wfR���O*�OLF����L�r
��hkJ��l�������4��fj���6[��+�%O>����������:����K��W��������>~�����@t^m$D^���A��.����C#��i����{�������?�#�ab]�r��}dg���\������VY��8�:F�i�4u4VY�A�d�E^���������JFn�qwR�Y ;�6|������#0�J��F�4�Q�M��IP�qY�[�.�$��HMv��
�����l���?)r��E�!@�E���������x��|��r�q�V���-�	L*A�~��}����2��>h tt���.�.�O��M���v���^|���luR>U�����m���dg�@v���*s'6��������:
�J�K;;��U�G)�VS�v#e0�N
?Kd��F�de���
�L*�k�N9(�L�@�*����$V%�IIoQ��
�����l����G���\���H5�(�E����:
�x9c/OSX����&��DQev'��e��:#��k���q�b;3��c�����6�'�y���T��{D�sq�i���lg{�vvr���Z6���U�0� �g����>A�EG�3��i[����{�1�f��}�����$�Rb/NO����\����4;eR�A�ygt��,�#�������FM]��4��v���f;[���*k��ZW,��hm�~�*�(���m��>�m�$����D^���0M|�tV�����YF�QH�j��V���Wz����6g�v�@������L���*Yn��m���lg{�vv.(gWY�r��8k�[����3VYGA��_��'�����
�O+�Z��I�gI�y��}$�Q�6F� W$~�^�s����	�FLf��vv���(��n�i���lg��v�vK�k����������3VY�A���������T�Ybj2~���nw'E�%��Z#���>�����-	
�����A�����y�MUDQ�d*�1�[������v��lg����c��e��*p�6H���g����>��u�.����;��r�o�+���I�g����H���
������FA��Wi�jG^	5e��|���f���)�e5���v�{���5��	��U���(1-��7�e���U�Q����&���#��|�@�k�z�]�3�N�>Kl��FB�9kl��o������a��n�oG^�}r��q���[m>P�oR�j���lg� ��D����1���U3W_l����>^n�n��s������cRnK�=wU����gI�y���k��>�ik�
�%�bSM��NOy�k���(���]c��}(lB7����`;���R���i-MP�g9�D�?�U�Q�������t�R�B�Z-�����qwN�y��>�H`;g���
���	Jg�O��z�"���B�{��-�b�>�j^����*5���������q��eC����+o�J������OH�}�G���)�����@]�������y����YX�CTNo�64��X"/�5d��=��KL���>�I��m�>
�3��u�k��C=��(�e�
����'��Xe}���468����*��v�=:����S:)�����,#��<v���1�L��>!r]����������56 ]c]Uq��)�F������u��C�\���D�A��lCV%��h_j����>�t���r���������3|�qwR�Y����Hc�9Y(qn<TG.��u\�������iq^3�X����g�:L��}w���N�wLJ��Z6Z��<�����(�3VYGA�Gz%������Bd�!����I�g����H��p�r'e��w������6wZx};��`A��5/�J��@��mH��;�Qq���������k�U�lC�l���U�Q���:�
y���S����I��d��I�g����HP��P�I�����v�j���Ix��y��46p�xL� �tz�j��i��;]��a�D��EN=$~���t4VYG�1R���S=�U��5����9�U��I�g����H�4e���������I��������8��){�K��qD����2[~\����(����;��o�Y�lN����hQ�3VYGA�`M�6����s������H;k����^ewN�y��>�H��!��#�8�?�#����m��F-$�}��P��,o!,�%w�G��=-�S��iZg-��)�[-@k�1�3VYGA/�O7�!�G����pJ�n�
}_?������B�}���>�=�H�x���eAL�������3jrD:�R�g)�<�q�lQqw��������V��
B�5��AL��_��Xy��Ve{�x�-�N=Tw�)��t%���;)�,�W	Tg��`n�=��wS�9��>:Nd��
�����9�����&-��!.��s��bV�
�U��x�p��/���������~�L�Uf�|����������;)�,���6-0�=E��>�(+��bt��J�.����xy�:������}��-
��!.��s��b��<�
oZ��Zt��������*�0��:����a�*�p���I�gi��3���b�����C-����}(��yr�j^.*��������b9S[�OC\��!.������f��l������b"��0VY�Ac���<�+<�U�n��8�����;)�,�R^k�	}�U����[��y���H��
r>�6��d���3�s�&�$�I���tv{0��S��!������,1��x���b?[�u�a�{�Y���`�0A�1 �9�N�=D��FB���PS��1�`�f!�$��$�����>����D�@;����d�@�����tv�b��6��E�na�K����Xe~<E�
~P��c-�������I�g����H�������f:�P����xq�6�9!�Ne�`�W]^!l�)�Lg���-0�%��5�oZ�|t�9��Y�Z\�h����>L7����a�V����RHe���;)�,0�WI��b����������*�a���a����3��^�yB�I��Lg���-0�%��q��9��@A�\C�g���?�;:?!k3L��^��&S��)��v'����j#Y(:��;?{�!���i�\�k���x�F������x���G�`:�=������}�.��X�r�XrGn���bc�u���/�,���t��N�r[T�yh2v�D������48?rI�.7nq�*i���2�����0�]��(�<�6V=�M[�O�����:�ku1�^�p����I���AY��~�*�(�#�S�p�<���{�9%���#�L������.��H���L�<�f����e
��Y�=97�w���lY{h�V�5n���Av�{����(e[���4�4��R?P4���*�0�|?����#ul}�����l'8�N
?Kl��F�`���|g�����C��H{���M�Y������s)G<�-���Aw�{���5��K��\�����'D�U�����:
�0q7�3�<�\WR�u3�#�S�������@w^m$
�@4[���m3�!�? N(B���sJ�a��y�$���S�(���i���|g�w�����2��y
G��J����*�(�4�+|�)�������)�m���&u6o��?����j#~|��$���c�+c^��Y�/�sz~P��e�N�j?�\q�����g���_���S��g�E�"����`}d��Xe~��n���1����1i����<+x��~��)�3�)g����o��U��\�8��8�^���Y�RV�wD�����q��`<�=��Z�9_&��Zd�+O��zk����������U���Va��<G�9��s��}��)�5�Q�
x��n5��!f�O��'����(8*����Q�>��&��`<�=�~A��*����"uU��f�����U�Q��M�����2���zSf�'�,S~ywR�Y�v^k$��W����}��WR�d~���0�����-�b��@�I*U-��&U�����x���g���$���"$�]~��%����Xe}�#�p~q�mE�u��3Y����\v'E��i���D���Fq+�����&�}��e0�b�%�\���\"/hOZ ���[�3�
�������	��h�����H'���o�2��U�a��;�35y����(�M��fDk��>K��k���<)�O����r�*��^r�+?K�E�A6�Gbg�k�{�$�X,�� <s���{����lb�k�7�e1=�
��1"����:
�8����!X2]��"��jg��1�9��w�D������D��B���[�O�P d���T���1}��u��7L���~��Y6lR������;�5����j��V��$�w�Cl=����*�(�y���C�&.�������3�$�+D�qwR�Y�;�5�V�a����y�P�Z@������m�!��:g�]$��T����Pn��y�3_��I@7�U�"��'�Ygy�����U�a������8m�SbM���q�$��I�g����HV�.�}bT+o�>C�)��b����M�K�L�d���$���n����v�=����vf?�	�������v0��'o������~������F��U�:�"��:ewR�Yb;�5I��#/t�m=�B`$�}��#��;�p���E��w�g+�O�\�-o���
�3��v�k���<[Ze.�*#S��-+�]?c�u�������UfL�`[E����$�E`��~����ih��'�+���Jq���X������i��w�|5���&�^��:�\g~�u��kW���mc�x��/F�����:�D�����v�a�u�86�>V���x
��b��y�����������c�TJ��k�,�7����E7/��u��y�QS�$���:�Tg��:����������~�2oc�u����4SN��I�x������	�-����I�gY�y���.�"���
�;K�dK�+ZHzLiT��=��f�/	1P����6@�����:�5������Zd%�f���F��W?c�u�aJ�[�����V
�D�X%T���I�g����HVP9K�^�����@:��3�����]BN~��@7��\u�#�}Tg����O�����Qx����iz%��*�����*�(��M�9K��=�4%�r�k�y�����>�N�>K��k�dT@���
��3m�����>^��Y���slP�����az4Vg���Au�=����:�����"�����Fc��r�p~���:�����o�-�jr���������s^m$������aC���U�=�O�8�1Ljp�p�z��	
c}�n�4
�s����5�`��[g-��4�'-��*�(���/���i[�w�4C�?$���9�����,#�wcn�HJ���}����F�0�/sC���a�-��([Ke�M�5B��� 9s=�C�unZ�*6A����Rez|���ej���b��d�8
/��ewR�Y`8�6�$�(��e��������40A����`�r�^2�d*=y��x=
�s����B�%.L\e-�"$*�L���Rez��c3�#��m�DmCO��Lf��L'm,}��>�HF�%��%���\�J��������=,���������*��7��
zs���h��!Ef����R��B$����:����$�p�hj	*�D��bD���;)�,p�W�+t���1ry��v=B��i��`k�����1}3����� �������9�k��w�*ki���z�����RezX|���c�t�L�������!)����;)�,����
$U�!]��[��1���v�N�fC�rhCDi\��������������9�Al~z�Zi�����:�I�}?K�u�a�O@��~�����~����,��;)�,���I�%d��4���)��R\gP{*bajS���w�����&Ta�&��Ak{��?��������G	G��Rez��(���m����\��(K�qwR�Y�o^k$Sj��=���Ns�z}���!>�M�G�NbC��Wl������4�=8�!\A�1�N3���	���H$]M�z���:
�$�qf����i[�����PT>����I�g����H�z��vm���-����@���D��a
��B��
�NR�p�l=&n������9�����+WYK��)��9>/|���:
�8�U� 9S
C���I)��\ewR�Yb3�5�<t�(T�\)��b�0H|�^�����@84�I���o|U2va�i���l�p�fF�XM�mky���Cs.B?K�u������=#��I@���G%��4��Sv'��%6�Z#D0��K4�1�X� 	��������1m��uK-�uv�|���B3>6��q6s�b3{-��U���2���8��TYG��R76�S���Z���p!=����sB����YF�.����s=N���G[R�n��Q\'/QD�c�������qJal���l�x�fN`i�U�rh��]�Y/�TY�����p��n�4�S�Z"l;���<%p��z����I��2)����m��
�%�����y�� ��A,��RPS��r&m��4��q6s\`3�q�����|)��J*���*�0����QAl����M��,n=^�awR�Y`3�6�Q��/f��7��eH��ief6�
���.gOB�p������zl���9^�����*k��������g���=,/[���4eP�RED��R��}9�N
=KJ�k��`^O��+��"t��
�@���
��M���[^E�>�Z�6	�l���9^��cXm-��d631��~�*�0�cL���q�k���eI��<��N
=Kl��F�o�fQ�yc�����sy����K�g�A��������'6��q6s\`3#<�U���"$��c����*�(�z��t���k����;��*�j��z������4s�
��=�1�y%�,pR�����Q�-�K��U����B|l���l���f�i�����tQ)$J�0����:������C3e��_B�R��
Ig���;)�,��W)bhw��a�m�G~��+`p��@�5���+`�]�FqaC��87���9��f�Wlf��5oY*��p���TYG���_q��3�&��j�xr�i���J��yr��z���k�m��8���������$efH"
��
C�z��F{	�R�>��&��9��f��l�h��&����+u"��c�g���=��~=\^�(��m�A��h�#���?�N
=l��FB�){=�������8�8�f��\.�m��`4W����	�N��zl���9.h3�������H��),D{�Y���@���z�@��PoB3���-�:Q���������H����8qI�N3;e�.(��]IV�&�$q6$��Wd�c%�6E�l���9]k3o+u����������*�(�#�r�F
�}�i[SV�^�z���������e}����R�=�7��B:ky�n�&5������p�\����p�MRS����`3�mf	����[�"��lfgt?K�u�x��f�6����VG$�;�aVn�(��B��y�� ������'��y=!���h�Pf�f������ 
" �U�s���E�hj���l�t�f�>N�g�%1�0�G���TYG���S�����G=��L3CZ3QH-��B��6�Z#8!��@�g���N�=;�^�I��>�0G<��6���:y*W���`3�=������%��4oZ���'V�~�*�0�C��G��y����-�	=�������B��y��H��*�����[���.N(UM�����K�-�Y���N��f��6�zl���9-h3���a�7�����+��J?K�u�1�����R�	���J!�P�iv'��%6�Z#��\r=FE�q��U��" �=�?�����m����a�(mR�J
6s�������^~q�i���1��������Rez�c�GK�?�*l�#��^a~���ywR�Y`3�6�U���Q�+l>�
Y�0J<0��1"��f&#R�����z���D�'5��i6s�b3�����[�����BD�:Z���@���������/�`����6�����������H�BH�#�P��m��d��r��I��qI��y= i��dL��Jaj���l�t�fN�/�WYKYN����;Z���@���+\�L��T��Y#�:�,�C�h��z���k�$_�.72�
4��p�!������e��F
"���R\O����[@O����`3�k6s�zL�MkIH�a�����/�TY���~RaI7]�u��	���#g2��;)�,���	�u�
s��m��&�T0�+L!f�5�A�'+������p9���TXj���l�t�f�.V�����,<���#)�[���@�7��h��������2������0bW���B��y��$��"���m��W��u��8���s�
�w��wE4�bU�op��3���)����S��`
������7�U�Q�������4;m���E�,O�)�����s��>�JydS��I�n�j���F���7����]����r�@�q����8L �~�f����$����*�ye����|E�~���:
�x�G������N/n��6�)����Z��;+�,���ZI+��0��!�J7g=���UO�<��\5K����	���n"���{~��5�OY 6��>���@lF��{/q��g��?���� �1���uQ���=
�1������l^k%�n�`�L��/������D��'
�I��}����1A~�j����&���6�����f4$�UC�E�`�x���Xe~��~�=bI3����t�f�+�y�lvg��v�j+9EH��f��tI���~�T�0�a�H'Lll7V�)��d$�]���"�B0�����,�1�,�2����2�ZG���*�(�:T�z�
S)��1�mj�b�\����wg��%��Z+�\�����}deo���4���z�zq��	?���DY�=�	V����O���?�~$�-�`V�QLpI+gb?c�u��k��8��x.����������n��Y�g����J�,�jU�{?I��Rt��Je8n����i�|i]���[�O���?�~�����*�9qi�/�\=�U�Q�����m����m�R��~*���b��~�u��YI.U40�%�L�~� E�����oK�G#_��$a��r�BF�����,H7�X�Y�4�W��Y9A<���Xe~��55��8�I` ��b�q����������v^m%���q�n�^Zx��B�U!�F��#O����)=������3�Aw��?���,dVL���u.��H����:����g������:X��c�����z��~����YI��1h�����v� ����jg���qV������Y��a���d4&���Bp�$~|g�����D��f�������J7S�u�1�����q���s~��n���tFK�X�XWB�q7|W����N�cq�{{w"s
���Sj�3l�����V�g<�?�@W����rY�p���`]F���H?e`c*`�2�7���"�M~����[#(|�Q���}C,�ze���~y�� 
$�,w����A����������>~w��������F����gk���?�����Y	����IITC���:����oy��p�����f�mm�	�"'u������h{��L=A�<my�O��UV�>[��~V(k���>�t'��/�������_~���re����_}�����^�����_3���?D��^!�r�n����������(	6�,��C�.�q;|��(�\�Oo�3�����em,���������(��]���������_����'����1������g<9\��(�8��#W��#��)a(���������O�b���q�����=%)���1�1�����1�=A��"m  ����������}q`�ac������o�����l�����_��9N ���/�PK�W4��2Q�PKvi/Kpwj_reg/pg_part_patch/7_3.outUX��Y8��Y��}[�&�q�;E���X!d���A��Zy%ZK���'�h�ANh83������<@]��j����V!8�=����A"q����N������?w��7���/��}��3�/�~���W���|����_E"��f�R�xe�������W��?^����_%����/^~������W?���8��$e�+U���c���}'�����/����_�y���������O�������?��x����X����i��������,}�2�G��_�����Qo������v��N�/������?~�����x�����x����_��������6���~z����|�|�_��N>����~��?���}s�U�o��������������	%O��+�k��
!�c�!�h=`�(��������wx���������w����7/Z��>�^��i?Z�3S~3�U0���H�����SH(�o�n~���c~�������d���Qs�f|�a	c�
�*�G"���%y�]�,e������0�����W��?�z���������|D�(�?���_����?����WoF���A0Q������V6n������0;��?������&������}#���o�� K*������>|���_�~u���������n�{�����^�/��_N�R��_|��o~������o�������^�����o�����S�w���o�������������;���h�q���@y��<��{���������#���%��=���������k�=�M���q_c�x�Q�`O'��'T�ay��L.<���Q?������w����+y>������{w����#�HhSv�hBt*
c�����X~��7^��J���f��K��Oa�yU�6q @>��u�i��:�{�	%F7m+I���U=V[^���=X����U�����x����������/_��{�����G����J���2a\�N�O�\d����Xe�\4ow/aRr��F�OH"o�H�o���m������o{��yL0���NB�x�l��;����N�8N�����Jcd�.>��0y����'atx������?~��N�-!�(����p�'f�X������6d����y\�7v^��'��%����X�=j���$$v������WM���Y�x�T�~ZQN������#�����38�l��d#U�cO�rT���:�U�Y��y��~O7�O���$�&�X	�[��a6�G��{K?�A��%-�yk��=j����(���u\M��b�{L���#L�#>������;5��� zL<��*�]C��QA��$����p*��#��~O�f��h,�����p��?��l�e������L���X��������f�����M�fl������6�`u;�7�|��uD�W��]�n�A3|C3I��MH���d��;.�/x��S����u���|d���<K,�d)
�+�/�6���I�3���4#G*?��(DYH�hFN��h����]���dZoG�f\�f�4�niF8�� ����$rO��w�����L��h&N)n[H��������4�nif#:��$��l�f����f$&�jJ�P���f�P2�fB�*T�bY@{��o��?�f���I�>�)���`+���K�
�2NC3�-�y:�8/g%#��L�H3�4��1E����D���'�	�CS�	���e��xNt:�����#h&���l��oA����DF8Y9�n8�q���f4�SRN�o5L�H3�9���xVl�1�LT<l;���a���B�5��y��`��`�x�����[�"�T�/��
�2��2���v,c��/?3L��2�9,-��>Q��e��K��A��Lu���J2������?s
AD�����#�&��s[#���y��<�RC���2��8����
�C~��V
��VB8D�!#\f��NY�� ��@�$��=j�Oy���,��0G�[9NY/�NV�{�X��. 1Tl����e��?��~�Znd6�O����P8`�V�%��2NC?6R7����+�J��}�~�?�j�]�~��O	�@?&�����������D1����n����,*rqQ.��U��'-���#4�fAc,[(�Mh�$��9�s�����Xe��}��~I���v,�8�-o�'��i
D~������1�
�h���i����CV��1��Y�**����a"�6�d���5$����������>
��9Bbln%���b�	-B�b�pH�XM?��8
��h�e{l.U����M�}��������� 1���X��c��i;������
��)���L�Kr��2c��]%����4
��9Bhln�������&��~�����V(�Xe��~��B;�M��P��)4���7������ ;���5�G���7�-��*�������
�O��+.�])�T*�.G�����?67�c�Y�w^C��X+���{.�{u��2��>�a�c�����:UP3�#�`Er}U�]�}���[AB}.�t`����J�2� ���T�����$���V���_���8���i����ds�K��e�&�HbCu"�����U�Y������a�nXY�V���G+���B�Y������Jy3H�\{�����~dq�;GW������ZO1�
/��S��#U���H���4�������+[�oB�RL�;��{�������'��������5�ya�f�#J1!�<�.�>��� �}!+�����$|�-��A���W�ul���6[�0+��s��������!c672fs�j�U�H�?o�O��`�q�	�������3���ZI<'yj>'������$j�
�h�c%�a����d$��(ga=?�S� T[�JgsQ��:�,��=$���o6G���������.
����
���V���~�^��|�
��"���I��o���E�gA��$$�9�~��>��v�C���5������v�����P��������.�����!x6��gv�4�&���(���*V�g�U�i�'�ng�h�������,���Bq8�.�>��� �}`��`��������J�#�f�3��q^1��G�Yr���X�R���~lC�l�<�[�3K�E�	.J%PB~���b)����������������/U���-�qN�*_��5�����$�4�|�:�v�E����%�A�������{���B�<;Ih��v����6����]P<�W7�E��|g�I �Vg�a�~�g9���X��H��^I�M�;�avQ�YR<oi�~�O8�o��H��O��:��Sr�~X#?U��Q�Ug������6$������<[���MpQ���G�i
��Xe��~���_��s����2���y�]�~�%��@�����W��C�k��`�k�X��������z�>�~�;em�6t�{�OC�l��<�[�3?7�E!���SBEI�����O���Q��>`��.&M��c���g��e��6�Sw9�Fyk�5u�=��"y��p��z
me��<;r9�q`Z>g3���N���4$������<Cqi�&�,E#\
kg���Xe��}����(o��-!N/nV����Uz��Q����� y��Ma�lcBp�o��������vs�}���h<B(�����r��]D��!y�GH������u�n�Kx�T�5a�*S���
Vg�����KON?yZ����ONuBR7�.J?K��[A�~I8�s�c�����G;g���dK��0?���&���*�����OC�l��<����S���E�9����R(�;�U�i���~�g$*�SE��,1�.�����0�(�,I����I�?���My�s�^����Z��$��$��Y�:Nx����mH���g{+yf��dkpZ�\e�n�~`�q���}��7a.v��C����l�(�g����f������AJ7��x.�
��5��K�}��Qy���u�U�i��[.�������!y���gO���	.��)�k��v��2�B?���}|�3�xU��������D��E�gI��$�hG�{�U���^4|AtXy����g�$��)���\����n�k����!y���M"���E�Z%HMB���c?��8�8�v������/��zl��K�2��g�]�}�<���$?mN<�^��G>TI<{9���g�L-\{41�N?��i�Ha�!5�t������4�*Z��$��/R �����������'�ia���25�M�+�@��k�����I A����
	-�|�
Jg�O�
�"�G��c���r@$]5 ��e�i(���3-(�Q?�����I�o��Y��=���*�,���R~��`:/���(E,6�qvQ�YR<o	G���^N^�}=�����^p<���5S�?���9�y�	�6��rY%��~�g:B�L��gN�.�X�����`BZuC���U�Y�'Z��kC��0�~��q�+���uV+���8�(�,(�7�4v*E	������J���q���6h���YV�����^v�����u�#n�����I�5�v]E�&y�#A�cRxX��\��8����#�q���ja��b�sS�avQ�Y���$?���l{������s�r,�3?��>v�>2��q�	#S�o�]��q�NG\������^%�	-2��F�x�%��\<�2��>�;6�J
�d�V��p�����[p�]�}�.����	`��}�{�<��������=�2�����*�=���s�#�����]�g���Z�M���:����Vga�}7�s�s�'>�i�v���)s�i�]�}�����4$~u�a����[$	b��w���nn��%�s������G������t��;�^����z\S,J�>�T���
Vg����;��F��2J�f�������E�g��}HF�J����8~��^B9�A!����|jv
N�i�x!?���������|��;�^�{y��6���>���e���5�����O��[�i�)�����J�ig��S�z�2�&�<~_��e�U�m�m�Nc��xH����&m�c5jV\�]y|�L���`�w�|��;?�s��.��
���;n����~`�q�q�tcRz����q	�����P{���g�]�}��7��b/x�F�T�;�{�>:���c��5�g��^g�A�j-��f�wn���7������uF����`�e/�b���dv��*�,�����y�������*�J��U��.�=��A2�+�w��0)�S�9�?:Nr�[�A	,�L>�n=�N:�K�;7���1�u��#�7�eG�SFq��~`�q�a���4Ye��'O�|�iF>r@.)�qvQ�Yp��#t���_unkj�)���9����8��m��mT�V��=.���0�G8�����9�b�U��}�u.���n?��8
���U�MWS��5�������RS�� Y�f��p=��H�J-�X��Z}��K\�>x�'�El(���q�}���=��a0�G��������Mh�����R���Xe��}��w���^�A�������Q�����.�>K=���dK���N�o�Yh.���M7�!�`�������DLT��&�K���t�#���X�l���5��`������h���aU�Y�-�z�Y���`[OA��u���0�(�,�7��Q��*�,��W�,��O��:��`��W"lpI9K@�+���w �����:���,��&���W1�����1����������OTqdL�
�&��5����E�gA��$���$�����������X��q:Ix�^��=wC��C���d����<7��|������W���pK ��<W8�T=�
Vg�L7��pa��iU�MM���Iy���evQ�YP:o�*B(�i����45.� ��\��go��
#zt����hYV����}.�Jg>B��Jg�����E�����;p��Vg���;F?!�3L�jc��"Sgeo�X>�.J?J�� q)���#v?{���e]����v��Y%��%
���j��$I�q
��;B��n�����mh� ���e��&�Pk��Vga�e�e5y7-,_'T}����R�V-�k�����I ����������f��U��T�]�B��"����������Uz�=2?��tvG(�����^Si�:ZYe
72�Ak���2��>�S�47��)�.�y[_�������}������.�$��gsCY����>a,2�D��!Jr��Ch\�,t�jQ�}v�j�Rgw���-�S����*Z6�6���w�*o�g�U�i�'�~������z[����������E�gI��$��q�c�g���z����0SYF���K>�����ecg��x`�+w��:�#���V�L���
-B5$�E�i
��*�,��A�������}����[�$+
��8�(�,h�7��
�}{�������2�@:�[ ��V���cd��^oh�V��%�i���bgw+vF�"�Mp�k��Uo����)=�*�,���w����W�����������evQ�Y;oi�!/4)�s�)�xw1��r�L�k�de+7�%?���s6�"vv
��;B��n���0�	-�d���Z�@��*�,����v�B�7-,_�p�ig�k{��L������MyH���n�wN��p����������$	\s���R����=�}\C���P;�[_����MhRdTJ-P�[7�.Xe��}�I��U��.CN��TA��G6t�?�O�]�}��)o����D;{������
'��4c����B/�(�D1&�D��4�����[�u���g-�h�%�AE��U�Y��������-60������<�DY�\fe�%_�� eRn��+���<������Mz� ai;���+v�\z�ctUOw�ro���jgw�v�_�(��eC��Tg[��N��*�4�C���w�VB^n��m.����0�(�,�R�����3�,� ��&�]i��N�sC/"^)rwI]��PT�b�G7S�;�#���V���M-���"����M�����U�i��w�fjr�iZ�P�&�p�W��U��)����'��d��$a��m�Z�`���jE�#��f{���fO�d	�b�l�}�m��;����!v�7bg����p-a&y���(��u?��8����L���M���*%��f�EP =���>���'�d��	�
mv���0	��9[��<:O������Yg�2���r����7������;r��g��):\�`��A1�~`�q���)7t<)��7/,���%�y��E�]�}���[AB/��uFs���>�������3��'�Q�^�yE����s�{4��
��?B��o�����	-������B^�~`�q���g7AT(N�j�X�d4v�v'�����$v�
�����p����m*��2S�W�#��s��&]�E���������!v�G��������E�g�"C��O�H��2N�>��uR������NU��M��.�s�i�]�}���[A��l�j*3����(��>�H �w
a��� ����������Y����o���bg+vfX��MpZ����
W)�������n����2c���Ta�b����2�(�,9;oI�kB	�O;;�Q��K&��|,�H5�4L(��z�G��b�K��oh��Zg�X�lQ���+�u�$�	���%�$7T���U���h�]�G���J�b�"�)�am�N�O�]�{���[A�������������HY�~Y��;3�R�mPo)��L>V�i�iH��Rg+u�F�PfZ�B��p���&�~`�q���v�d*Gg&��������	fTH���E�g��yHH:����wv��8���Mr���������B^t��J��R�����7�������:�F��Mh�E�S�������Xe��}<������{j����t�}PTr��0�(�,I����!��=G#�Z�^��3��b���w1�������l,'9��}�wq�
��?B��IG�uQ��pZu�����JU���`�q�q&����c�;�:�1�����j�]�}�����$q���+��Nq�.�:_y	P�v�~Y���s�]V��
*����Z ���4�����$e�����U����T����T���~
u����U%]����E�gA��$yC��(��=�e'�-6�z������0�y�#Gf8q���8��|�
��Ci:�p��9�[���4oB������|��H�q���/�K�iZ�w�2Cy
9�:��I=�_�'�dU,]L��&����b4A����srkS�7���v���vU���b�"�p��9����rn������A)����T������Y�?M��Dj�|Pcg��g����f��&��f�������������{�I=N��0)F�6����.QOC��P8�p�r��MhY��E���JW��l��8�8y�:f�G���*���X*e��qvQ�Y�7o��~��������� ��`����p�T�x�N�d@���+���8����9!o����dZC5�<I�Le��l��8
�pGG
Ri����$��%�����avQ�Y�6o	�\����kg71=R<��\Y��i�F�*E�J�����*�C���94���is����#���� �^g�_��6Re��z�W�f��j�V����4s=�{�-f>yvQ�Y�5o���m�q���:����N�fC����/��m	��p��Z���b���6�#���-P���6�%�}�'�A+�gtC���P�����i�'��*���'Ea:W�qvQ�Y�5o����B�
�����{���C�S)S[+�o���"RU(�������9�k�1��p���J�{H|R?��8�H(���'(�i]�����ey�]���r���V��s��=	���}|�K�C(,�=�������=[�|�L�����9!j��z��Q������H�v���)�C���PO�;��|;2Mk�5�'Ze}��2�(�,)�����'���rz�dO�	��eS�s}R
�i��)|N������mN��F#4��Es�7'./���MhA����r��u���"U�Y��]�v����)�\.�v��M��=yvQ�Y�3o�)��P�Ev�W���$��*�c�J�'��?��,����9�B=
9s8B�n���,V��:ZA`��Lv�=�*�,��������i�L����/��avQ�Y�3o�bs��=�������u-F[���,gN���f�R.�3&y�:���0��
9s<B�o�����@�3hI�
u:�����Tg��R79�
�+.3���,���b�Ur�?��I=�_�'�d�-�����Y��
���z�%�u%������*�&�r������t�CSr�x��9���dZfZ�p(��5E�m��T������$@�c�����m���{�]�,g���R��a�V�tiCC�-��|�9� �\����u�B�)�A��"���r��^n�.QOC���3�9���sS�U�rrQ�q*Vk�F���PO�w��Si0�.ol\��l6�.J=r�� ����pE��������Ay�b�!�����r�G����������3�#���V��P�6�e���<��<�W{&Re��z��l=K���A��
h�m��7�X5��E�g��y+H����K��u=�P�cYLsEd�����0�����+���HC�@=
9s<B�o��1lF���P�ff1�~H�q�1�go�8����qlR:5�"�avQ�Y�3oI���
-�6�o��h8p�L����M,o:;��f�AE�mh$��C�r�x��9.��q<������I�8F�*����*�,�z��2aR��:j���hQ�B��<�(�,����D����&M��'��9��f���N3���QGI+G=�c$WI
]��">6���5s\P3�4��-�����`q���2NC=������,Y�I$4%���m�N=�.J=j�� 	{p'bcJ*���)���o^^
*���'.������+�+3���=
9s<B�o��
���khAS��(�bu��l��8����n�����/\�O������<���E�gI��$VF\d�eU�L=C%v%3���#�]3Cs
���\�9o��n�+���9!g��r�h�D����C��DN?���2�B=�]�".�h����J�R�6�d9�K���R���y3HF1�6:�L���
I�`�1_���b�����X�"sgl��Cf�i���r����������D=Y�E�K��!U�Y�G|�".� �+�P���v;@J�nOTf��ew�m �B�=e���|�N��P����,�J�����QAI�.�l���1|��ij���r�t���b���h��}�T�v��Re��z��~�.������4��`���}�]�z��O���,g��������������fcq/����f����~t����3�#��i���b]w��VP�rT��\G��8�=w�3���X��i�Db�t=� �)��R���y+H�#1G=�Kp�s%E�����!�3�3G�+������'�"gM�	v�����3�#���V�]�6���������%�R?��8������0f����N��fF�.�4�0�(�,�3o��L=B
����BB�#��*�1I��f�(�K�z��=p��s~w���3�#���F����U�:������fl�Re��z(����s��Y��B%K��p�����$g�

2�).����+]:q��T���nN�/D=���|�9\�8�%�i���r����,����W�"�S����BG��8��~��n�)t���M!�s&7,��R���y+HA�I�~T�I��k�.6�8�y�GM��l��>�k��]n�RC����3�9�3���MhA������~H�q��s=0mV��I�����!gf��9�f�Li�-��S���K=pBl
��s�(�d�o�"���~uO
��aOj���r�t#g��'���khA{�s����5��*�,������I�R|��z0mcxF�D6�/��R��9�f�����M�U�;G=��L=�\U�4��f���v��+��W�D�����9�fN�jfT����hi]������!U�i�'�~.��M�7�ZU�I��H��R��<�(�,�����6�����{{q�!���W�
��+�6�����p���%��P3�#���V���n��|�P?�~���Re��zd�w�zOz����v'.9l��5�.J=Kj�� �
&�3;�����Ru�Mv��XgV��l��Qi�Z9PP����+,5���5s�U3���,)\EK���r�:�d|?��8�89������yZePW����(��yvQ�YP3oI��.d�T"��[������pP����2����<<�U]�����s�����f�|����G������)v`��#y	|G��8�x�~�Q���+O���.����3��I?7o��P�2r%�����k���I���7)���aEW��2�:C�W�fF�=���i����%�f�x`�8�|P��)���������=~�H�z�B!���w�����U�gI��%���=���1[m�4LNRsQ��n���)W�������G6�=�3�n����#�gI�����3�YE)5[��W�V�����OPnnE�j�kk|����gW��%i�V�,*�@?��.
���K+c������Eu	'�f���kI��9t�d��]���m�����fg������g�e;�'�>�������7��d��z�>F�o�x����J�'��J?���(�7��&����3���r��vq����W��z�+��qL���D{X���7�G����h����2�V���*�,�=T�w��d[�1������~6_�����z��$q���S���Bz����IC[��|���|��3��X#�D�*��(v����_8�~L�-�m���d4jK��U�Y��{�G?�����WIU���f�hx��U�gI��%��m�K����b�����U*Cho���y7�|�m��/sl�������_8�~n��Q�c�	0d�r�G^}��2�B?.�nrC9���O��65��G����q�]�~�����$�c2����������}p���^u���[q���\>{o�����J�=��!v�������J�U�xh��MT�L?��8
�D�]5��9��V���:&@[������w��^���JR���O���di��J�n%���s�B?!�VY��03�!x����[�s�2���O��>�����*�4�:J�e��)����"nwf'
�y�y����z�����
%dL�z$A�~�#����}5[-Q���FbS�n}g������|�d2���d~�g|��y,#�FN�E�:Z��9�)?��>�2N�<&o�]�>�H�9:������-cb��.���8~*bI�8O�c��;��;��{��<�?#cV�PU@��3��1�6�g��aZ������"�x�8�`�h���S$��_�����-���ge$�x����V+O���b`���,B�����e?|z���������o^||��Ma�7F�����f��-����h�]dFn�{�k�%S�{�R����</v�y[sK3nm�	i� ?��3~kG21�����'���-O��f�	�P���8�
e���������7�������o��F��?�{����_���ww�w���o�������78���[�&oBNz���]��l���!�a�5N�tB����Oo�$������x����O���2Ix�)zTp��~�2j|�����?�|����xu������,��
u���&W����x�i������O'?�n��z,��i�Ex�e��~s�	OSG�[���D���xp�����^�x��������]n��,������_����O��E�8X9�xJ���c�������PKv���2b�PKwi/Kpwj_reg/pg_part_patch/7_4.outUX��Y9��Y��}k����w���f)b����c=�]�����+���|R�T��0Er�����<@=�}�j5��U�p����7o� 'O������?���������f�����_�y}q��W?��xw����>�:Y5����+k������6w�����?�:����|������>������Y�U::���_����w��W_��~�www��������������~z��?���������`M����Nk��'�~'f�����������������7/~��/wo����{yk��{��������������>���G��������x��wZ���?�z���J~@����/~z'����o?}����?�������a���������?����iJ�H������k`�THz4�Q)��R�`����������p���/�����Ww�����������?���G1�+y7����Of�'�HQ���������'3��"F>���d�����������A�O6(�jN����[��|R:,bR�����Z2l���Y��o�oo�����^^#���}z��/���p������c�����F�/>�x�?��z3Z�j�&���������-�v1:�l9R&��G�,�c�f����oo���������He�������r���_�a������������Y��������o��������_���|�O���k�������/�u��7�����[�O����������������'���h��.�[1����
���������FL9|�/Y���)/��|����kRF�x����������\��8������kc`&����U��O��w�����W�|s��#|������G���@%y�������x7G��L`��	���&E���&Mh�(X��l����z���x���4���7���Z@���K�?��Q�Z���.����<���������������t�owz�����7w�?>�L+-������M���U!��<�FE�~�*kF��Z��;A9E�����"�eQ`����Z~����_��c�x3��e|s���o��-kzz���
,���'_V������A�=����t#au�&���������;q�������������Z���/~���!�o����~b�C��y}��4PG)oVYO=,�D�:��]2A�}�x�M��{A�_��`Y;(�������c�c�@{�>� ^�d-�
����J��q?c�u���>�+7�O���$�&�$q��j���Q�u�����f#�K���3-A�6��������%�����i��[�)Fq������� �z�����C�q��Xy�k���LZ�������`���9J��w�t����En,+�����t�g��u�1>�]��n�f�u�N^AZ�L:���2��kyk`�V����QY�����E.VA�.�
7`�����|�M�����[��v�SY����u��j���T������a����0�f�V!�
34��T~L�P��G�AFa����e�}J��v5P�5P��2�e�
��P�E�>���6��T�iP&�^(�s7:3Q<O	����"��[��hy��@�\�P��n,t����vf�D�P��FP3�0#h��
��G��������[,e��Hb,�MR�����:
�P7g��0���asE������%��*>z�^���jK��v�$��s1��'Kr��3�3��	70���b)��9/@Q�s��Ne�f�H;�1�PZ�����0�3��(��d��f��9������iC���a���}f�3�3���K`�X
)�s�)(�����:����7���������+�L|��#�XJ�%�lg�(\���2U�)�f4�<x�>sl��Jf���&5�&�5���m%r~��H;V&�V�xB�����:�x��[2X��4�;���#��L�cR�����h�;-$��I^R�>���zX��>�KL Uz�f7G�k	1y��@�����������t����e����}Yn86�Om���
�$��H��VT���m��N�>6R7��������}��-7�c����&�<~^�d$�q���`����U!��s���G�R�}$�1�V��l�G�e�bs��,��M�)����N�2�G<EA��Xe�}���}�i��`��i��������N�����.�>K���F�d�G�
���}H���/�F~�|���y�!����b���s0������>
~�9�_ln��^p��&kYt�q���|�������O��[���&�i[]����'���2���E�g�_��HZ<���%�]������t�8�>����v!�JAh�#/�����8���4X�����a[b��d-�"=��Ev�r?c�u���{F^���"�=@����x��g/��.�>���F�lB��yI(`r���4�'��M�#�����U�]E^L�D^
��9�|ln��y�:��f-29��s���J��U�Y���}(�$O�*��f�j�e6�.�>KT��F�G��J��b|n���+��M7e2|�6.����/B�f�`b���i����dsCJ��e�&kY���Gn\g}?c�u�a���}X����.������y��E�g����H�}l�y�;wvZ9D\��`�M����~�yPV��3�V���8i�ip��\es�U�^��Ql�3�� �1��*{�)�3VYgA/�t7��s8X3�3,��}��������E�g����H���	E�?�o�%���GF�m>�o����#�����e���=Z%L��l��0�3X�U#���(z[\�����R?c�u�	��Q��c���d�����s����(�,1��I#�Y��TzH��#���������c=�K��������K�vg�.���@���Am67�f��u����\��g���J����*�4�C�O��U�C�PE��7�����k�]}������P����v.��HvfM��1��i
��3�/��
du]��]?
��9��ln��� ���Z6��o7n���H���:
�$�-����&�-����F������(�,��7	��a������a�%����2�/�4os	�$6�\�r!A�z��p�{��m���lg{�vc�x����Q�����K�3VYg���w����q�V�������5��	?���'ip~|bM;3~X\����Sf>O���KFyD����G��	�E��6�����]�;KtLq��$���K ^��l?c�u����xf��V<����^!A�m/�����Dw�j$�C��3$�\���Z�eR��X+u�J�92)���}LA������u/��;�#�������~M���`p�<����v4VYg�'��c��h�+�dby���v��e��6#	��(��U�]XQk|�`��{�8t�����o
P������9�?
��=��lo	��L�E��"��1~u/��b?c�u�	r0:��";m�N���->@��v��e��6#�Kb&��B����|��3�y�\�7��v��`h[C���*��4m�v5���x�G0��-���7��H�:��#��?��~���:�0s7����|�mE��������S������x�l�q���4k����$��Pv��u���nv�Y�44�R�g��\OX��th�g{���R��o��\+gQ�(��<��Xe�~B�~���d~��:Ym�<�h,R���������H`Ca��|E��6�Fy�\d~�f�y2/`~4�k4��{
�JgD�~�g{��.p���f}�\�����|����g��N?��K=#U9��b�����x���}>�.
?K���F��=Xy����[�|�1'���P�ahi;�,������e�LUK�v�1��<�#8���������5s���\�,Q������U�Y�G"�~o����q�v�;4��)B?ywQ�Y�<o6:@r?:�:�����Dp�����|9�8�7����T���\V~�g{�����!�&s�1	�;����{�������g�&/O�?~>Y`��hm�Wd��[�]~�H�[�d:�~@T������/�+�t]3�H���_�@'q���W�K��Az�G�����sp��3�"H�[���u�|�g������k������b}��&��g��awQ�YRx�j$�1$�^�K��GyS%����X+�L��[���!�6�X 2T=La�!58�t��8�h
���������E�~�*�,�jxG�If:X���L�vS�������k�����IF��"��J����^cf�����I���z��)b��T��\@@)��"���<��gZ�<�1�M�"�0D��!zq�t?c�u�'����#����X�d��y����������y�j$�6��x���}�^F��XB/(�ud
n\;�c�����B~	���%�~�g:��L��gN�n�X3�����B?%����g���?��nb>����*���t��+�m�����y�����*X �}���	!
��b�
9.�
��tRP��
�������e^)5��tD��n���f�������)��y����*�,�H��,al�	-�:Y16S?��x$��E�giH�V#���"��7fg���('���KaN��j��R�%��r����+�U4��?��;Qv���;��&��\��,��(G��Xe�~�w����SC\��V�|L����G]n8v�����V#
-6i�xg�1���*��L�r�0�j�����
�V�����c�i�����;����&����K�	�leZ8���g���?��n�g���:�a����:�U�/J:|��>�HZ������
F\$o���	�=�x?N\�|��\0�T�~�.j?����uw����r�6���AV �)��g���?b��uwSQ��^��m��k3���awQ�Y��o3�.�����v�
_������3��O�Y�|�-����QU&]3Y��n����;���<�f��HN�
`�G�s������O��[�i�)���m>&��3�_
�ywM�y��>�H�pS���%9���1�c<d3��\�hW�~t���2�E���~�yn����;?.�S�$j�f.b0��<����g���?N�n�C�t'�&�%��-I�?Bnc�]~
���d����w43��:�
9�$�Ce$XW��%������-�L��o���s���G���q���uF~�b-�D
��������Ve�|�*������9m�CB-
Q�z�1^p�]|������`�XwV�gR���s��~t�H��B����2b���SM�qf�nwnH��Rc|+5f]R�7���G1~�[�~�*�,���o�i�;z�}������~/V��y����� 5��H�{���y��b�tS#�Oud����q���Y|:=�^�f��;7����1���T��t�j.r��gxIr�~�*�4��:v�����^<u�����(�,
W�>�43���z����E��R}�'�h~8��C�����G9Q^�����#��xAj�k��&sf0zt��A:R?c�u������?��������������E�gi��V#���b:���,�"��$:��!�`���
ff�1&�~�J�&�]��
�3�x���g����uc� �b@����:�g����=^.���nn%�8^P�t�D���(�,�7i�6�}Rcm��g��@:lq���{�Q�L�p�Q[���yy�c�)7��|��d���m6Y�FP���R��&>�Xe�}E�
}��#�`[w�=q����@�]}�����������$V�%�19#a�|�+|�BF�2���*����������;��Y�Yb~�\<)~T�u��&���U�Y���
~����p��U75	�������t��P��G�g����H��J=y/������������Im0+����N�����}j�
�3�w����S��g�EP��u�%������O����U�mu��v����C�+�����y���v�:c�`�(���^v����'\�C����D�{�hw_�q$&�}\�����;�[����m��G5wx�r�1�����:���/�,����`�:�������������>���'	�	��*����eU(*�!Ee�yUa�������^�nU���{�~\�����;�[�1�^���n.��*�a�(�x��������S�p��<���{�=[�����s�]~�t��	�9�5o��0��b\�,s����yJ�g�C/4=���w�o�tgw��-�V�������0B�xN�kj������������v�c�{����Z90~��E�g����H�pA�Ne�.H��D	�8A�R����+I� bJ��*������tgw�����!c�Y��#K�GGL�v��U�Y�G��ntgh��3�k-P�u��	������E�g����H����v�� �l�(�<%R�Zm*o_98hz�����%�����;�#�����,gI��d.B�j��r�i
��U�Y�'���
~�z�m}L���g�y����(�,��7	y����/�.�i~���2�4W�}��]�I��<V�%�@��t�l����lgw������vS��g�E�k12�b�g����>��n����1����)i���V�������<Xy������b��q��^TZM����l
����Cq�����hr��=D~\�����;�[��|��M������L|(I�lc�u�	&�SX�|�!��mUP��	Z%��E�gi��V#
|��v�~�Fw�����>}�:���dA��M��c����T��.���;�#��nA��*��uk�$�cM��������:�x�uz���������� q�u�3�.�>K��[�$��6�&����\?r1�i�&�V|P!		##g�=C:�Rro���tgwKw��Y1V�e�p+mr��F��3VY�A��>>?���b�r3�����qwQ�Y����H�3E��j��i�^/9���2���b	"^is�8ntY��
Fr-��~����AwvG���-������������ �,.$i�������8����������M�IV!j:�����$���H$����U���&p��h/����5�"
������S�`���.z�=���Aw�G���
���T�W�e#���Op��������k�O\^�4������j���
��j�]}?�O2��������c�1�]��9[�^C(�|CAG4�����PgR�.E/�`;�#�����l 0�6Y���e���}��Xe�}k{�
OT�I	�5{a���`;�����y��P�G�=��d���&~Bm�;�20�����i��`hV�P]���1X�7������e;�<��m����F���bc��Xe�}��wC��{�muJ�Y#��<�t�]}���[������c����gh4���^��m]
�)**���:��}"����v�lg��A�����������R?c�u���o�2}������T� ���W�]}���[����f�1L����B�����7�%w
a��t�\�b����:w��!��dg������li��c_I
d�)�3VYgAy��i;[e���UPa�yem�awQ�Y�v�j$���>S����s�N/���sy52 GM�6Sr�������u������7������u��LR�b�"�I��:�~�*�4�m��{�������vW�>ZAbR���b��y�����"S`��dCStR�<e]f�����E�1	�,+oS�%���:�#������%&�f��x��O���~�*�,���6��'���Z<��!��Y+Wm��b�����,���H������;�]�l�xE��������������]/!X�����=���Au�GP��-��%�6Y�(es���S�����:�xJ�[���i^��V�M�!�Z�B�����Du�j$P����W��0wL$�����]��
r@�<�?0"uU�a�E]�7�����Du�EA�&katN�����[�3}���:�8�zS�����\�`�f���e����(�,);o5��f��0�c��
`?�@��:O����N�3�\	*��T#���4������e��V��UkiH��:��J���-U�y���D�F���*���[���s��r��@s�l$'gyT��~g���z���qHc��,��A�.C��{���`� ��9As�zH�^�d��^D����Y���@��~�Xf�M����dz�~�y�i�]z?�O2�nqMbB��;���b��aD�����a��K�������j�|��.���qGp�����S5Mg�Z�=�R��a���=�_����?m�X"5I>I<�����]z���dT�[
���>�@���$���Gn�O�g�J� y���x����4���s��S.n���aid=v����:�8y�:f�G���*�����9�?�.
=K���FW>3|�=���]��b(t�L%r�q����!
���B��t"X��$��p�9���Cj��ukI�������~�*�4��5P"NMA%�h|�����w��j�f#�b34VE�����=z��;E����i�=t��	��i��V%�=����mGp���=�(�M��
�S��V���m����=>r7g���t�L������� .~n)v��%b�f#)1G�Q�b�}�w������������
=�7rT('H�9��*��%��`6�#���-@�X+m�H����h@����:
�x�O@��~����+��r���%���(�,���)����v��������ZW�����y@�.yf�$P��f���jh������CFW^���,�������g���=�t��	
����T�M�� qV�c���������HN9�SK)������:{=���Y�o�M�	����1^�"7��-���=
Rs8���
�H�Y����%!-LE>����TYg���;N,5�:2mk����!��U4��E�g����H�(�)L����]U��g�S��I)��z���,����C=	&�����p�9����7YKC9,������Re�zX�����0s
CE��=�ZwP�w��%:�V#Ap��f
�,�;0�[�S+�r=����3;�T5&k�%.�@O����3�:3����M�besu��R�A��3-U�Y����=&K/L�Z��MgNVAKI���B��y��P�������X�B���*�t�dL���/�5H��T
�������
:s<��o���J���%��!��3��a���=.�ntfHz�K�*'a������������a}��p%@�DC�s�e�����?XwR��Q��
�D	��0�!���4���:s��3����&k���\�$o�Y���@��nM\���1������6�� ����1�.
=Kz�[�$v�$�M
�>��k�=K�mIs�:� ��,3=6O<�*\&���4���:s\�3�3<U\�����#!��g��N=�_q]BW����U���f��
eZc�]z����$/	������,�S�tz0f}�3�VZ�)A3#�%��U��b����3�#������AA���j-�����l�g���=^����i���J4��j����v��%���F���&	L�OH���	z�a�;"�7m���2�\�V:���K��`3�#������fk�u+{=t�g��N=���M��\�V ������9�<�"�.
=Kl��F��pV�@�i�43�p�L5���M,��:���Pq�Ha����l�x�9.����M�� �u]�C�����:����=�L�����U�f�(���3���.
=�l�mF���P$�~�{.2h9\���l���q����SH�� [���K{h���9Ag�tf�&���Xq�=]�P��s-U�i�G����9+u�&49��%��j�]{�����0CP�����{&���o^�*
��V��'��
c���aNa���@O����3�:3G������0Z���B?K�u�a��U��|���&�'n�c���%���(�,���	���?J�P��C�N
\Jf3�&�m�_yN\�a�q��S5q�]J\
:s<��o���8�97Y���y1���Re�z�~M\N�(��m�B��l�8J�D\ewQ�Y�3o6tu&�i	kv��H�N�U3�\����C!W�S��fIW��zt�x�9.�3'z�Eh[�n�~g����Re�z<q�&.s�+����2P���ewQ�YVg�f$���S�%�;���M\������8����a`F�3��U�.�HS�����3�[u����i����Y��U���g[���@9����e_}���������|_��kB����IF�_�C�z'�5�'��I��}�?�Lj����*�GA�\��U�9�"��t�t�9-�3S��NV��V7@�AD��Y���@��s�4s�*����VG$���r��<a�]z���[�D%O����w��<��Z�����AY���?��8�Q3��*&�=�GS�����3�[:sGf�]��8��]���~�*�,�z�3{�,'��%O�L3C!�da�awQ�YRg�j$�R���6�}�	��@�1��zL�������4:��N��P1���C�95���:s��3;P�"o��� F����TY�����=�q�>U�	=��|��v��%:�V#	� �KYs�O��Z��0��'h�M�����^����1et�V�]���9AgN�����M�����C0����:��~��n�)t���L�a.7q��E�g����Hh�B5Wz2;�z$(2��|����\c�]�z,�W��i�
Wj���t�tCgvi�����<�K,U5�?�Re�z|�\�Vi<U�VG$5��k6eJa�]z����DJ3��RH���z��}�)�*������p�xn�b���*�0����9�fN7lf��I���f-y�������}���:��`��zR�����LSc�c8�.
=K��[�$V
���A�{7�p2�DrUo@��"��}�&T����]"��9AgN�t��	#z���2�����g[���`O��"�����7�:V�J���#.r@:�.�=Kt��F��f�N���_�D�� 5��L\�� 9ELe
���W��{@O�����3�[:s�zR���X�X���F�=�Re�z$N����n�PK���$.6e
���(�,�����q2�����/���"��C���{�i��	-L�P
���C���E,,5���:s��3G��p�Z��fu��b��,U�Y�����z|���m�Bm ����,�:�.
=t��F
*"�oA�uf_b4����Aq�33�0��xH����!�YU3�%�2��g��>���:>�x�M�eH}�����\L��)b��N��`�f�+xR��>����s��>�J*�F.c�����
�Y\�81���a���������{V�`����4����%�f�x����Ev�������������=P��H����n��B�0y�]~�h�[����8��'��@|�R�
3\5e;�����3���V�hG���wQ�0��k�?K�f=�'�1��%�Cp������s��G��y�-s���D�ms���*�,Q��Z�(.�) �w�
�CCWR��0�D�=��ST.e�A��P�����lt���?8~�����Z�`^P��g��g���?^n��Co&�\=N3@Z/��}�l���*�,��7[�����������//���I�^�G��g��xMi�����3��4����#�g��l���n�2B�,�����U�Y�|���v�R���������j�]~�(�����cI��7�LJZ�=W�|��
?))�DF�G0������O���?8~D�m��x����`!�f��y���:����~X�y@�����W4%���m��X�]~�h�[�d#>�1��;����)Ew�T��"_H\���d���%�_�~<g���s�t�b;����Z�t}��,�k���?.�nl��'�'ok�������Z:��
?�����d��?�D�������:�\�M���-��6�w�PgS��3�Au�?����������>��@�,�3VY�������F�~R�d��"�
�N+�3��
?Kl��VB<���r�vnm/�Q�S�/��L�w+�g�Yb��x�����=���n���G��-�9+�M�b����^�g��N?�#��I�?%U����`���s�����,���JF^�Sp2xGA1A-�t��j��F�s����;�a#�J.�N\�Qq�)�'���;�����$Ro$n�����rJ�sB��������S ���m��'����q7`y@:��K,'�
�`���^q�z���X\����N�>�}`�15�	6��
�0	��f��k���h�Y���-����j�=�~\���:����U$���������l+_��{��7���W�<��K`JYk�������A���w�^���>|����Wo��}c���7~6����/=|B�wf���|��fV�P~��M�~��\���5O�4Slk[��V�W���G;b��v��L=A�|���?��6Y���b?+�5�ox��g�_���|�O���k�������/�u����U?��o��f��g���G�A����
6yr�{���GI�ig��$4E��/t����V��46�I�>lY��?qk(���
V�����{�g.�����>||��`���h����O��o<:\�r������
mFnB��$�bT�����#��������/=�������$�M�����G�_�l�����6����o��z�C����%DMr����o�����l������I�����8��}���PK���?�2c�PK
�j/K@�Apwj_reg/UX���Y���YPK�p/K"��O�(@��6pwj_reg/.DS_StoreUX���Y���YPK
�p/K	@�AW__MACOSX/UX���Y���YPK
�p/K@�A�__MACOSX/pwj_reg/UX���Y���YPK�p/K���4x@���__MACOSX/pwj_reg/._.DS_StoreUX���Y���YPK
�j/K@�A[pwj_reg/pg_part_head/UX���Y���YPK�j/K7u��7u�@���pwj_reg/pg_part_head/7_1.outUX���Yg��YPK�j/K&r�hT�@��/#pwj_reg/pg_part_head/7_2.outUX���Yj��YPK�j/K�����;�@���>pwj_reg/pg_part_head/7_3.outUX���Yh��YPK�j/Kuz�/d=�@���Zpwj_reg/pg_part_head/7_4.outUX���Yi��YPK
k/K@�A�vpwj_reg/pg_part_patch/UX���Y��YPKti/Kc���2U�@���vpwj_reg/pg_part_patch/7_1.outUX��Y3��YPKxi/K�W4��2Q�@����pwj_reg/pg_part_patch/7_2.outUX���Y<��YPKvi/Kv���2b�@����pwj_reg/pg_part_patch/7_3.outUX��Y8��YPKwi/K���?�2c�@���pwj_reg/pg_part_patch/7_4.outUX��Y9��YPK��B
#245Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#242)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Sep 15, 2017 at 1:13 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Thu, Sep 14, 2017 at 8:06 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

I have few changes to multi-level expansion patch as per discussion in
earlier mails

OK, I have committed
0002-Multi-level-partitioned-table-expansion.patch with a few cosmetic
changes.

Phew, getting that sorted out has been an astonishing amount of work.

Thanks a lot Robert.

Here are rebased patches.

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

Attachments:

pg_dp_join_patches_v32.tar.gzapplication/x-gzip; name=pg_dp_join_patches_v32.tar.gzDownload
#246Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Rafia Sabih (#244)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Sep 15, 2017 at 2:09 PM, Rafia Sabih
<rafia.sabih@enterprisedb.com> wrote:

On TPC-H benchmarking of this patch, I found a regression in Q7. It
was taking some 1500s with the patch and some 900s without the patch.
Please find the attached pwd_reg.zip for the output of explain analyse
on head and with patch.

The experimental settings used were,
commit-id = 0c504a80cf2e6f66df2cdea563e879bf4abd1629
patch-version = v26

Server settings:
work_mem = 1GB
shared_buffers = 10GB
effective_cache_size = 10GB
max_parallel_workers_per_gather = 4

Partitioning information:
Partitioning scheme = by range
Number of partitions in lineitem and orders table = 106
partition key for lineitem = l_orderkey
partition key for orders = o_orderkey

I observe that with partition-wise join patch the planner is using
GatherMerge along-with partition-wise join and on head its not using
GatherMerge. Just to make sure that its partition-wise join which is
causing regression and not GatherMerge, can you please run the query
with enable_gathermerge = false?

I see following lines explain analyze output 7_1.out without the patch
-> Sort (cost=84634030.40..84638520.55 rows=1796063
width=72) (actual time=1061001.435..1061106.608 rows=437209 loops=1)
Sort Key: n1.n_name, n2.n_name,
(date_part('year'::text, (lineitem_001.l_shipdate)::timestamp without
time zone))
Sort Method: quicksort Memory: 308912kB
-> Hash Join (cost=16080591.94..84447451.72
rows=1796063 width=72) (actual time=252745.701..1057447.219
rows=1749956 loops=1)
Since Sort doesn't filter any rows, we would expect it to output the
same number of rows as hash join underneath it. But the number of rows
differ in this case. I am wondering whether there's some problem with
the explain analyze output itself.

Apart from these there is a regression case on a custom table, on head
query completes in 20s and with this patch it takes 27s. Please find
the attached .out and .sql file for the output and schema for the test
case respectively. I have reported this case before (sometime around
March this year) as well, but I am not sure if it was overlooked or is
an unimportant and expected behaviour for some reason.

Are you talking about [1]/messages/by-id/CAOGQiiMwcjNrunJ_fCDBscrTLeJ-CLp7exfzzipe2ut71n4LUA@mail.gmail.com? I have explained about the regression in
[2]: /messages/by-id/CAFjFpRedUZPa7tKbCLEGK3u5UWdDNQoN=eYfb7ieG5d0D1PbsQ@mail.gmail.com

[1]: /messages/by-id/CAOGQiiMwcjNrunJ_fCDBscrTLeJ-CLp7exfzzipe2ut71n4LUA@mail.gmail.com
[2]: /messages/by-id/CAFjFpRedUZPa7tKbCLEGK3u5UWdDNQoN=eYfb7ieG5d0D1PbsQ@mail.gmail.com
[3]: /messages/by-id/CAFjFpReJKSdCfaeuZjGD79hOETzpz5BKDxLJgxr7qznrXX+TRw@mail.gmail.com -- Best Wishes, Ashutosh Bapat EnterpriseDB Corporation The Postgres Database Company
--
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

#247Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#245)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Sep 15, 2017 at 6:11 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Thanks a lot Robert.

Here are rebased patches.

I didn't get quite as much time to look at these today as I would have
liked, but here's what I've got so far.

Comments on 0001:

- In the RelOptInfo, part_oids is defined in a completely different
part of the structure than nparts, but you can't use it without nparts
because you don't know how long it is. I suggest moving the
definition to just after nparts.

- On the other hand, maybe we should just remove it completely. I
don't see it in any of the subsequent patches. If it's used by the
advanced matching code, let's leave it out of 0001 for now and add it
back after the basic feature is committed.

- Similarly, partsupfunc isn't used by the later patches either. It
seems it could also be removed, at least for now.

- The comment for partexprs isn't very clear about how the lists
inside the array work. My understanding is that the lists have as
many members as the partition key has columns/expressions.

- I'm not entirely sure whether maintaining partexprs and
nullable_partexprs is the right design. If I understand correctly,
whether or not a partexpr is nullable is really a per-RTI property,
not a per-expression property. You could consider something like
"Relids nullable_rels".

Comments on 0002:

- The relationship between deciding to set partition scheme and
related details and the configured value of enable_partition_wise_join
needs some consideration. If the only use of the partition scheme is
partition-wise join, there's no point in setting it even for a baserel
unless enable_partition_wise_join is set -- but if there are other
important uses for that data, such as Amit's partition pruning work,
then we might want to always set it. And similarly for a join: if the
details are only needed in the partition-wise join case, then we only
need to set them in that case, but if there are other uses, then it's
different. If it turns out that setting these details for a baserel
is useful in other cases but that it's only for a joinrel in the
partition-wise join case, then the patch gets it exactly right. But
is that correct? I'm not sure.

- The naming of enable_partition_wise_join might also need some
thought. What happens when we also have partition-wise aggregate?
What about the proposal to strength-reduce MergeAppend to Append --
would that use this infrastructure? I wonder if we out to call this
enable_partition_wise or enable_partition_wise_planning to make it a
bit more general. Then, too, I've never really liked having
partition_wise in the GUC name because it might make someone think
that it makes you partitions have a lot of wisdom. Removing the
underscore might help: partitionwise. Or maybe there is some whole
different name that would be better. If anyone wants to bikeshed,
now's the time.

- It seems to me that build_joinrel_partition_info() could be
simplified a bit. One thing is that list_copy() is perfectly capable
of handling a NIL input, so there's no need to test for that before
calling it.

Comments on 0003:

- Instead of reorganizing add_paths_to_append_rel as you did, could
you just add an RTE_JOIN case to the switch? Not sure if there's some
problem with that idea, but it seems like it might come out nicer.

On the overall patch set:

- I am curious to know how this has been tested. How much of the new
code is covered by the tests in 0007-Partition-wise-join-tests.patch?
How much does coverage improve with
0008-Extra-testcases-for-partition-wise-join-NOT-FOR-COMM.patch? What
code, if any, is not covered by either of those test suites? Could we
do meaningful testing of this with something like Andreas
Seltenreich's sqlsmith?

--
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

#248Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Robert Haas (#247)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Sat, Sep 16, 2017 at 9:23 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On the overall patch set:

- I am curious to know how this has been tested. How much of the new
code is covered by the tests in 0007-Partition-wise-join-tests.patch?
How much does coverage improve with
0008-Extra-testcases-for-partition-wise-join-NOT-FOR-COMM.patch? What
code, if any, is not covered by either of those test suites? Could we
do meaningful testing of this with something like Andreas
Seltenreich's sqlsmith?

FWIW I'm working on an answer to both of those question, but keep
getting distracted by other things catching on fire...

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

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

#249Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Thomas Munro (#248)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Sat, Sep 16, 2017 at 9:38 AM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

On Sat, Sep 16, 2017 at 9:23 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On the overall patch set:

- I am curious to know how this has been tested. How much of the new
code is covered by the tests in 0007-Partition-wise-join-tests.patch?
How much does coverage improve with
0008-Extra-testcases-for-partition-wise-join-NOT-FOR-COMM.patch? What
code, if any, is not covered by either of those test suites? Could we
do meaningful testing of this with something like Andreas
Seltenreich's sqlsmith?

FWIW I'm working on an answer to both of those question, but keep
getting distracted by other things catching on fire...

I cobbled together some scripts to figure out the test coverage of
lines actually modified by this patch set. Please see attached.

I'm not sure if there is an established or better way to do this, but
I used git-blame to figure out which lines of gcov output can be
blamed on Ashutosh and prepended that to the lines of gcov's output.
That allowed me to find new/changed code not covered by "make check".
I found 94 untested new lines with 0007 applied and 88 untested new
lines with 0008 applied. The 6 lines that 0008 reaches and 0007
doesn't are:

======== src/backend/optimizer/path/allpaths.c ========
-[TOUCHED BY PATCH SET] #####: 3303: mark_dummy_rel(rel);
-[TOUCHED BY PATCH SET] #####: 3304: return;
-[TOUCHED BY PATCH SET] #####: 1515: continue;
-[TOUCHED BY PATCH SET] #####: 1526: continue;
======== src/backend/optimizer/util/pathnode.c ========
-[TOUCHED BY PATCH SET] #####: 3433: break;
-[TOUCHED BY PATCH SET] #####: 3435: return NULL;

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

Attachments:

patchset-coverage-0007.txt.gzapplication/x-gzip; name=patchset-coverage-0007.txt.gzDownload
����Ypatchset-coverage-0007.txt��is�H�6�Y�+P�x�������5We��4��e��������6H�P*����{���BJ8���p�zb�m)����g}��W������H/^����*�rz9�}���JfU2o��y��,���������EY��_-���ez����z����,)�k�M/���d9���t��3)^���yw"n~������M�=��Z�9<����>��b�5��~�s��^�����7q�>���O����(���X�WE�(�l����g���G6�sd=��t��5��<�j������eRZUn����J�_�IqgUE����|�d������|YZ���]�^]W�����'�����Z�
������Y~g���&���"YV��E�^���Yg��YZ��U]'��ez�eZ��O^�Yz��TH��x#&81��uh��C�6�|,b�Q����B��=����Jj>��m]�	������=�������,�]d��>WI9���?���eY����x]���Jg@���<��e����=�c�����t�.�,xy��dv�������?R�_����x]=����E�^$���y���I�sd���xyG�O��=g,&H������g��ek��'Z�x~�D��������YW���}�7����T�����)�����P����um>(����.��#�_��q�[�Y����<wc����$�&=�#��/���"�BHU�'l�#�=�����'�X��H���!n4��"�b~jo����U�X$S%���
���U2O.��*��J�Ux�?�����Y�}XU�@�3�8����,M���'���2�p����k����
8`	}c���$��.W�T���>��J���G�k��/^��Y����?o����-��'S����,^.��7���j������z�Ln���lH�v�"�i�g��pE��)}:O���/�d�'�e��N�#~����qRL��F��8�X:�@xB��Epe-'-�@��/&}&ag�u�<�g��BC������yz��� �+I{���|����D>�D���2���
�D���b�o��H�����Q��.�|�*rx��'(�2��Q���������������E?8z
;�p[�!9�kz���9Y+���T����%��DB6�|�,��EY�lkq7���.L0�v�g���k{;��}���~��]�����3�0�t�dyU]�������~9?��z�\���s_�3��hO�4��������j��Ho�o:K�$�����>��k82�J�(�����P
����c�,S�yj�a�@v��	�$j�i�O `%5o�?��{��|`.��X �%����O�O++���
�0�
'.��!i��6^���r��0�����������������>������9�l<������[��4����i	�t�L�Ye�M�xe�4 l�����&���/���n��J�d�?9;�p&<����{X�b!�����|�d�)�a���.
���1�08��,���'��D�e�������?��O���t��������=��'L�E��an+l-������|���P>	'2��<$b�oht��9�Zm��������Q��f S�U�Ng�@�w�f����c�_���J��cx��|��A�>�������d�z:���Fcj��{7���DW�uo���{�d�-�)��k��e�.�b���-�p�����������D��uf0"�e��J�W��?���x�������B��X3�p�45j^�W�8?w��>��� qY�x��J���!~
Z��Z��
8#�����w�C���S�gpZtT���	�
K*���&A��e����<�]!<Uz��@�e�����,����Z�@���6�b�'�����2f:��o��3�k����;lk���fy�T��moP���/	��"Y2�����7�6�4�jW�|^���"��Eo����L�8����}�
���w������ux�`�0�����e��
�%4����j7������������F�1	�[��W]+�,u?pwx6&��g�zE�'|c&Q��G�;�e�&��E��=��=��B	�x�.��Zh�3�����Y��n�*�0�/��j������+eF!.Tf����������a�+tL(%h�d��|
�5�@^N�t�d���j�Y���_�B�90��$�;c�
y�z�b��l�|$xX+���{��:��q�^r|l`U#����b���0���0�p���,Sdc�-���a����HE�O`wd�A��EF~���_�������69���=o�m �]_w�6����������y����?���x����*���2���L�zh�S�
�I��#�eJ8"���Z����C����G�	*S�pJ2��v������|�R��zd���#z��l��j#���b�y8?�@�p�D����r:w�������U%�&�V�g;~��5�m��l;�}�� {�-t������8I\&l��66�x��:Eb��F��k�3�'+��mq��>V����������9���b��]�H���4��i��6S�����C|~�k�M=��tw�n7��$����Lp��Z%��2M���"������N�&}�S�OQFz��R�`_��G�<+i���������q�N��t�1�(r{
��������4�px����A�`�PT�<b=JQ3J��[����,����F.
c�\�����T�%�Z�p��4:�����:c�
�b���H����O���x[v%[�B���#��uB��A�D�����:D��������%�����o��H�#C�2z�@�R����,���LY���;�����hy|����~j���R�Z��VX< ���\����'�02���1���v�t�^��x/J�SI.D����������Kc�*�Q�����o���5����W�����4x�L�||8*-[��b�����sbNI���
F��l�������@H��&S]����F��8q���
V�������$YX�e�NY�g3�[2#�d���W��N��L��*��2���/��[���6�x{�v��g��\<0�e����D\�m�2�!����������Q�i+R� ���
���k8{���[m?d���2-D�b@Y�KD;]�)���l4��
�����F$��G�������Ho8�������:�rH-(���G��*�;%l��E[0�ffpq����Lb���%�5_'��o��O������L�
�'�~��'�2�����zyH��1�';,��d����)3��-u���s�������=jXq��n^��]�����_�����0z�������7~w��	�I\������(A���R�9H�a���������A��P����`��K�|�q-����t_���	��w��tQ���|�(y�U�����I����8e�U��]wW>�����\�f��\��jw�Fl��r��14���\9��O���l<����6��z��� �N��)����U��-��bU���]p�%<���%�z��rtD����a��nA��3���=!�F�%m6%�!�C����?#�<K8+��e~����:��e#*����iO���e�������!���V[��%��$&�����&mz{+��)H/�PtD�*�T�81�[�����������*s�"I�>r����m�Dd;"��m5�+�������B Pl7�}��`��K����������~'{(�-��������lM��t9O~.�_�WR�!�c'(L�_
�v&b�#����h��9a&��@�v�m/M��BW@?�m��v��n���9��g	�?�������N��rY]������V����#V�xt�6�L�s?�l�)m���������MR�@����C\�,�Y�������RA� �b �>�S��������^�f�,����Ww����@�pG��;Q���x�����5n1��&�Yi��&��������6+F���B��3�(���l;�v;��r����P�l0)�t�_��F���N���%Mk�hu�i�Yk!�r���7�>}��3	_g3����vW9crW���;$+>�q?�i:���4>3�M����lp��dK���b��y^=2e,\>pu.���{Sk������7�<��D=b�c���c���M:G������2�������P:s������W=��l�G��r�n�&�9:�_i�A����=0�pt���qd^'��z��o��&��<HZ+m+@�X��""I�qB���q�J��RHXY0$O��=[�b�~��k����I�F}F�u�%�^s�KG����b���D#�dkm��-��M+���	�^��e�H����d��\���X�'aC,��������9Wjw\��C�^��8��������'	��'ud��I�^�2L�%�x���HH��N��URPp�\����8#��A�J6��9�	b���J�
��y~d��>���.��j����9��9���%����x�g�$�x2����Dx�����&W�kr\��!��&��3��������2����?�Y���>���	�����v�g�-j�P���1�:�?������E���DS��=��$2U����vT�����l}�O.��:8�]S�_���('
����y��UBE���dVx�W�_v��9����6g��l�@
f,��i��b�.#��UqZO/�%(u�P�%��.��3�"E(�����:.)R#Q�)�:1����:�X��R!�`r�^�e2c����V�vD�8��	��-��?�v����_N�X?���x|�������;}8u[��KY'@��v���?$��b/�[�$����>���R���X�����G�6.�����5QN#5�	��<QnB�����(��ot:	���%��f8���1�>�@:AP�[w�!������t���S>h���OG��s�|~CBR����j&2�MKu�hM�a�I]4�R��
��6��D����b��>q�����^�8_�B�� �tn��.���c����+U���v�6�eRMu�����F����9�(����Xd?U6�N:br�d�u�|��uGn�#R�~��)���;b�{+���7=H�Q���D����d��:��I~cz������"F��9�Og�(�kEh���k��8�D3rI��a�g�l]��
��m�4��y��������1
���� W2�2_�&O)�����4�m�+p�e{����Lej��sa�l4Z�!w��<J>����xLM�<��'�W�$��,C��3D�1�������@�L�Z60/	%�.+mw�����U0�Q�H�kG)��3df ��3�f�bAQ;����q�
���n�a��0mJ���B��!�Wy#�XD��X���8}A(�j�E.���������jkw��������6����5E������W���~����<5�ifo�^Aoj	�%���&��������g{1?����B�~�h&�����S���R'Y�(m�P�*�g;���i��N=�������������]'��+2�����~�-R&$��k�.����O���'E����_^*	�I���4�=i(��0���31[L~*������|�����	�T�l�EC�B?n��3���J��e�/�����N(t~��g���k���T�0}���e/3�/Q2�K�&�}��l �I���k=
@��:����ve^����.�j�7���� =�g��D�Q���F�f��Y(��\r���lX-��PN(�r��n�/J���Q���kP����Q:w^�����V�U���^x�
@��ic��q�^�V�q�1\����u���|7��������
r��c�\�%���D��A�����a(L)��G�W�
C��$�z��������N_��~x/�����Y�b���?r�'G��@��W��4~W�q{�+���z���������+��*�p��fU��W������Y~�bu�w\�a�����	%�r�r��� ���$+�y��%���'a��EZ���"]��n��#Z{�]����L����>��?
��0�/������w����l�����?���7f�*�e56x�,�
�����(_����-�.��.����>v����C<��{����������S��HV�.��30����������>��M,��&��������2�������"_��4��Hn��J^��}/S�1�#2x�f��Z>��Q�E���A!I����D�iZNW��z��.�e|)�����V��
zz�u�H���#�Q@�q���8��l�X�/�����`���A�U��.�S��������%/-j�cV2���%[7��q �O�a�!�&��;ZJ��3�������%�J�>V@@��2�A�x��l��������9O�x?�B���+��}!d���K�1��V�Xbk;��s�%a�W�p19]j�/(&�@��lJxS���
���+g[W�&8�+a���
v��cjW��*ehLQ�q�+J�J*
��������G����0��F���!���/k��Txt�SSV�r�i�a�:��&pC���O��N��C����R}��s���=S�����)�(����t)n�s� H���=����I�+z�6z�S�����S��h��=����J��xy��_d'K�4yVT�z;a�u���>��9��q�m�.��~��VQ��������R�e����T�^���Cn��������mS����f�6oe}��N�*��N���1=�e��`�O���P�S{���[������dc����[���v��d����>����������S��N�*�o�1��d��5����������}M�6E;�UH���#��UJ`��$S�$�!��v/��p'4�/���d<���H D)@��kMo=�j��O[h��y�!A�����A ���t��^�yJu���v�h<��Xr/��\_�j���_�H��C:��]3����iTJ�w��V����%E1|��Gz���w�����I������y���;G�n�7������2��xFYA�=W��D,x��(��+��%P�o0���e4,����m^|a+���8z�0�9��DAU��s������A�V���V��?R��g���R?���@6�m^�m�[�H���eI��PP�K����|�@�I�a�����G�w�0CX�S6�O��0Q�j��}�u��G<�l�v,����8�)�_�������aE�C�������k�Q�:*Lr�N���q�}�r�[��������0�Dm=��t�Ik<�g�RC�`J��d���.��Sj�+�~����"]�~��= �p�%m2���_~W=��?w�3wLN���X��l�+l����qX��9h<W*������=������=���������7�VD�b�
!n��3G�DI�s��5a���M~��Q�#Uh�~$�U���}�Y��5o7���}%�iha�q� �������G�
�:���2/<�������L]������7����o���}�p|��5��<s:r��O���/�eN�7��ZC
<���E!u�k�*%�Sd���K��p\��������+r"�q�e�"�����3����WOA�����3�*���/+Y��;���J��(W�R��.����u����/��,��_0����*n>8�p2��$��k=]����'#:cDn.}��k���(�xq����R�K�8g<�|i�a��1�h���]-1����T@y���H�O�6��,hl8-+_U_0���W������?:��>|NH��T�`�1<L�v�T��a�^��E9Eb�,.�b���j�������X�@��9b��K���)Z���
�p1�"Q8���}Hq����"���Ar��-v�z�j�z��*%����0e�5k]f���V����@�����L��0�� !�~$��S�%F��M��O,�������������?o�H��� k���%FG��~ �z|��58a���g�9*�E>!H�x����9�jcx�\�� 
#������5�n]	�u���/0�����E��d�� �j��	��C
",o'���!���SN(��A�*�G�����6�N�F���Ug���N���%U�&��6��������G�U�~�q�s^��[(��4W.�����0h�_�3ZlwS	���B���������e�Hx�n�2q!�DA(�p#-�\{��:��Z��R��l��	\x6���ZQ6��S�3|�q#�~u���JkW	����6N���:��	��M_{��wz��8+���p�7��%-�um������/)��P�v�E�����n���#\��������LSZ>�%H:Y������9?(����u�������"�(�m�
De� e�1���!_���EG��UI�H�*�~��K�t��T6�����??�ebe����Q�*��0���B��+��J6���h����b�M��E�@�;�qX*e D����!��������*l�;��.�K����;���p�B�c��MU5B���A����x!b�2��f�1�����j�9���P���V�tD��P�)n�du��Q�)�������5�:�,��#h��������Ki�b��@u��cU����c$O��^/��_��OR�Ja���MK�� �y��a�TT��2���������X���;$�l�~���q��&����?�������h�N`�����R����������4G4���y�(o��+��?���"�D��{ C<������!��o@�&�1���u�/�EDR�-��p
"�����.Dw{+�0�&W^�����1�(]��C��3��l��>�0=>y�I��������Q{��M����z�@X���������;�Pd�4!�X�7�K�t�z���t����/��`0���Q�Z`=tm1��\�[c,�M��M��x�B\�������LEL�'R��X�V~�JW�6?
)
�EE�dcQ�������8��I�aw$Rm��w��J�&�������D�a�O�{,sT�����N���nQ�����A�f1��9����M/��x��-��u,�M��6���u$..,��-]\T�A����
����z��@�f���c!����{�k)�T1�;��*r�`
���$([����W:�a�-�PxD�WNP���1URa�!<L��7��
����5?���c�l�P���\K?i�Pu��[���J�o|I��FT]Z�����#��t�x�����w�u�|M(��~�eC�<�z�Z��#c���nQ�� �aEi/g�J��.!A��m�I\N����
��5�AU��q�x>����g�b���<*D���8�Qg'd�<�U��
b �����%D%�� dB���l��"��Zo�Kt���?�5�����?<��C���p�)�>������}%�*�dCk�[����t�M$��	�D�����k��Je��rDmm������u�Li
����`�,�/���l��������&<�	���S�9OU!P��P����G�,���{
�z�	����w;TDY��7����D��K�������	���I��R��*2���!���=�R��E��B;I�.g;��4'4�x�7������]��4�RxL	�������0�G��'�G�$gm<���R��>�z���V���iHX�@'n��A�"����6v�\[�e��u)6�`U��2����QKNz���j��"��~��r�02��]�W����R0����YM��lo>���rm�8]VeW�.��dZ�b�u�����y�\&,����T �&��z)�]��#f�M�h�\�jw�|�X�����7����-�����I�]�(Q������z��g��:I,�����u�U���(5 U��
6Qk��^
Q������6,yO�aJ�o,Dd�@������J�����m,
���T����w�
��T|<�*K�w��p��d�/�Z/NY5���h���$_����A���c2�LV��'K���W��k�#������f�0xU-z<i��*�
��?��	�X��	�"�W%�?��ka�����xJ,=U�K��4��d�����]Y�P��^=��w/��������uK��O���[���%�Ly�V�r���J��
���
��8b��5��X�0.?�LyC;0��S����o�?}8;9��������'��3�l����}A����d,��l����7p�v1,�\�������9
�}��l�����S<��7C���}K����(k��T�Y��2����I1C�������7C�XdMr����/������AJ�������HH��v��i�����fhg����9�h��!^+*x��m����q�m!f_��s^�t�"�E0P�Ap���
���H��5��R������{��|���?���U��B�9d�rU�c�9U>Mtp`�^���B��g��<�Q��m��tU��=A������-��T����L-'��`��xZ���qUi���N��W���&�����6%1��mH����_���@������45��z-=+0]��W��V@/2.����p�f	5���k0}������t������x�O���,@A��:{��d��J����������S���!�T��^��W����F6��� B���6����'���&��7Q4�ZD�d��N^&E���Z�<�c5�=�`;=�
��_����i��
��a���x����7n�������9������sP��H�����$D����co�r���e���g�exo����j�������z�^&���f~)���?�'#k?]VVK�N,��H��ab���-�l1�������iVh��V�$�'4
X�
o��&En�o��B�'<[�D�f�<{��5����O�7����+Z/HQ�I7}
+{�,������=;j�����}��~$���1��o�s�Q�#�:p�c��O��
�����#8����^~��D�d'���n������?(L�v�~z��yE�w����w�c��-���
-'�Z�=te�VN��o`1��-"��}��;[�=tO��	:��}VcoB-���{�&<�hw���N������r����_e5�<g�j�U����r���y�h5�<��_k5���~��Wnb$�z�T�W��!���������X�N���1���>������(�0v�;�����\cb>&Y"�������4�l�����y��Re>����8���V��VY�<�C��+?��M������N��\�kWo�91V�(���\���$��?x��f2�i���'��Y���x6�y���m��������p7��|/�f#�#�eea7����_cQ}h��UE}��Z&Uvd��(��7��(G���L���H
-�	��<9x����NU2���h�Q�I}c,��7<�Y��������P��y�s���"��<��e)�fh�!��lDFYA�k�b���"��!���a0�K�I��G����
3�*��x����XIv����9�t��
�l`�i��:Z�IN9Mf=�c�6y�e�N�}Z��s�j@��21;��<4&���bPN�R�9LL=(^�*%�5�*�>o"�GWA�d�-PW>E�I�)0Y����0x�c�D�R�V�M�ii	�������2�U��O�3qi�^�_�xX xg�"���������3�'YR%��E>O/�t��|���Na�Q��\hI�r�Bw�������y9����l{�W�P��.>�Z�!~�P(�e�>�_��`,�l�|t����_������a9�K�G������=9��9������������j�>U5@`��?�O�_��lK�(+�X�����(+�����#hD���D�U[;�_�V]���xCEl��m�-��q��2�Xjj������}��,^7��
M�okS*�Z�p4p����	���"���������Y�+(5��������W���){�������oP��I����������n60���ud�@��k�l�_����&AgZ�:r\5�.�ftv����a!uw�,h��eA��4�w(��`��b���A������x�/h7	����cd������,�{�C����������Q�����#��|������,�u�,=6B�p=�<M(M�1������$bH�
l[��.��oP9������J������2�YT��;��[�6�/���������s����bO_����|��z������<-	�D)�
�t��#]�W�x?X![�'��|��P"�B1����d�$S��@
��i�y�
]���O�O��>�=�K�3rZ�$��|X��c%�p|�I>w�U��$���������-Q�����j.�E2�"ad�F���|�&�X�����S�G�X�E\�2�|�8!N��X���y��f.#v[�U�����d;��H��X��U ���W��8v1x�<A���w�yzM��_"r��"��H��Q��k�\�>��?`L�
k��|�}kL���QG��oh�?�B�Q�#�l�@���l /1~FQ�7	J!9��W�J��
�H���\�	�[L�R�E����h��%�r��/`��7�s�#���c���|^7�M�*{���[�!�C���
�$W�����Z�M�B:?P��'g�N?���}2=?y���1����K�_y�)�T�sp�@_���7�Sa�c�E�f�=��(�V��}��q�
�.M�a��\�T�2P���j�NuL�>���:^�g�\���2%��
�J���*r��_Y���q�:S���$	K��a��sR>:����`>�}�x|v�������������@_��������9|�5U�$��*k������5@/e��CyC&�6�U�}���P��m�jNx\~�~z������N��b�`L�.���X�O�@6���]�x���x}�p����-V�BD�|���O2��������H3�eLO�zR`�~��w�o�5���y���(JKL���E\Fy�� +��J
*�@��0�
���-LKL�k�����,�?-��z(e*a�E&�|P����}R�#�ptz�f�jd%��j�	�*�9��qc�|Nhz�����L����9H�X����P�Q���������q�>*��|J�Uz���o8�i�|A��c��$'H��D��s~��AN�(qNHg�=�S�y\{rUqeFV�+F�KS�����
���nS�����U�����D��?i����k4���"E��l��`�2�&�d����$jf�F��@����P��n1�A�\�ni�i>-��T�Q3��SY���36>��ty����w��4a��d
�0��c%�d��|�L+����w=�qcM��J��Q��<)Y�D�G
�����C�T�Re�i�$:�[��P��>~�vB����)	B8���.����f�b���o�%O�������{\$����J�ck�GWEH/�e@K���O�aeCi��Z��*R@��/\�@��9/�J��"��i����WQ�Yj-	�����{ Ck��������0�3��Z�]��n`k��������Iv92�h.V��e�F�-Q�-+"����g[f���Y�g,�#��QfR�Y�k��Y��Y�����0�#p��(��62e;p��G�� ���a���Q�-��R&t5t�3t,(L��O@ ��$�u�����c�?��ma���7�f��> ����v_�	4�������;��\!e�-����NWT�:HW*�@��-@>
1���N�P�r����HKU]���	UO?��_���
�������/r��my��[[6�~]�lf�+�!Q�-Y^$�1]6M\�jd����/d���7��-:E�����o�l]��Jb*��j5j�x�X�y����.�mW��A$A^	%HOK�<�D����Lf%�H$
�]N��~|Y�Q@�����z*2���?��IX�u�f��!�5�sH���&Y�������at�iPv��L�����f_"x�R����4�&�:���g-��c	�o�T7w�5���/�RPY�+T�L��?�M(�z��k:��*�c�5`�*/�U�6t$MJ�/+��e�f,�F����+UsM��a��|�HV?R��X#�p���I���!�NY#�PW�_�jv�k�rG���/�r )\��������Kb����q�<�,��fsz�-|@�H�u���CIa���F����P�'\� P��"�����G,�*
P���
��P���|�����e�j�RsPj��mM�/��B�����79��P�$#��bk��Q�����i#����?g'����9{�s���A�����U!t��xt�B�>� =]����%���c�+@���h�!Ee����	�Pv�\a�py��	C�|�����y����
<���(������W�
3�h"�
��v�q;%���Sz;d�&�}��o��}�s��
DS#J�7�
Y�������1.�����T�KY�K�����&:!�|M�����������\
K *�|���@��f��f����K�Y��`��)k��C��	����B}��l�!:�L~ gSv�T��E���Z���wC�G-s\2��8�FBsm��q�n#�u�����=LAp�=�P��Nm!�!���[�k(,��/�I���l��:V3W)A��XF���C���o��][j�\/M��S}B[�����	m�����O/�NZ��N-�x�������w�}�
&~�C[����cGG\��W\�b�����
��������Rv��y�Y6zs�#�#2��|$����#�0C�
��C[x9�N��@�	����x��F��C�����Aa\��1��	�W7{�`
7������yp�t-`�0]#td��!�z0�d����z�T���P���{��^��mS��Q�Y��_�<�KU'i�=����X�	]�c�:RN��Z��V�A����l]v�d��b��R�g�@�^�{�W|C�%t�^�&�H�����t�YU�P���F���P����#
����C�h��,�}y�+�������k�������	��Iu���������{��C/���,MU�z���P���b?@����6Y��PJ����Ro
�|��$��v����He� �d�F��"���_�G�y"�F+|�t��p7\=�����Y
�Xb���b��������1w���IU3�}�-��OTo�v�3*��n�?:oh�����z��W"��|B"��J�?)��T�.��"���.h�'K��
T��L�;uaK�^6����F������0!v~��E���	�h��������q{�gI����c	Y/n������D��J���}�x�^"Fj0�]0 ��8�d<1p�}U�p+D���6P���~e
y��D���^��1<Lw�T�:��P,�����C��a����w�gC�<�g��������2�j8�*C<j���_8�.M������a���{:#2��d�=�����/P����	s
(,��1�H���2����-M��c���H(D]����3��9:��|eL��{��B�F��
���p��4����DNxM�E���q�8Z�)���X����L��R6CGZ��dAny?0]� "@�&*B�5$#�niE��z�������\�����n�K�w�i������lHV�v^��x��2�,J
N��WH1�o�h�=�9J�`FV3�g�K�k��2��)_�s�C�T�I�lB�n1n��_�Lk(�J�i�</�����!��fIris%��XMIN��=(�P��U[���	a�!KaA�h��>�9�4�HP#N�"b����B�_�U�]�"������/Fi���Rb�"��@���������_4v�>�x2I\���wo��c~����l]�2f�*0��}Dcf6��(%8_�(�Y|k4&"k�����Z&����t
];
�V75�|`�d"�X��e������xV�B��t��^P��
�E�7�K�����Y��1�����@�8�)�9d��m��_}/��z��a�bQN���zl^�D����O�������������	��%�}���R<����R[�YV�)�T:Q�yD�p������g��_(&�p��d�{dK�	Ev/0�$����
�LG�ta.��M���������b�
���%��,��n��J9c��9�V2�u�e�:�X���'�l�$:;?���J�������s�?���[*������	��Q�^vt��]�NF5Y!�p�(aQ�9��$��f��!��"�P����P%������l��,N#b�M����$X����	�m����9Vz�
���/�S��^^�*�XM�S�������D��`d�
y�+-�Q��1eE.���������u##O\7����m[�7�:�+�:�o�]TpKAV���*P��M����� 	I�����!W��Z �v�<��:�����g��;P*���>pw�$��#e���.k��9A�0-d~�1]�|9�2H�")�fH��3'P��e.��'�v#G�������%�$�cp
�+ ��D�2�
Iar)�u���q�4_3	4��b�`�tc�PC��
�Sl^\&�����dh�U����V�bc1�A��& 48�3:U�<C�F�A?��<���O�k:j���0�$�lL�����<GO3�(/"Um�H�1�D�����<�����8�\�(`d��%~,L�H!e�}}��(���Kr��p�g���A7�LB��E�g{{@2�,�I��Y��_��,Pt��#tIv>�C~��-����qU��t���%})�jr�����	<�0v&���R��"?��$��S1rx�����|%�o�*�$Ym���LG���64�t����pQ}�Oz�����P������l�>5���������(���(*����������T���M�������v5���,/���KB{<��ew����-�^� -��_�����@�G�u���"�>OA����*�9���M�O.��S=�'
��<���@/p�k.Xw��:r���XQW��I�U�><�O�o=��2��1�sv�c��I0�*�%�k�CFy�������Z�����fKe�����M@��?S�?��=�XeM��R���]���p�=����W�#�nQ!Q�n����`����<H�'���y�h����`;f�ZUG��U����E�A\�Qt�n������P=lTwa5�j�x,��9j��>�����/��������%��w�-��3�1�>��n��{�������]��R��>:��G�G[��FC��}�q��P��h�	���r��h��)�4H�,���=[��������/5�&�GDiJ�}<!<���z9��2uSW�.�&���b���-E��F|��a��N�>�d��}'����l�w�����t���_-AK�L�7�PR��f����������CY��<s �v.
P�%L�Z�*_Z��-�+���/���s�GT��
�$\�0�����B=|O}�.�����3M�B,�I_~��D�C�k^:��f��s���������d�����4�tE|���x�G&�O�Q&�O�Q������l*��hg�~gj
s1��������9yc��W�����_�O'�����iD�/�e9��)�d��A>e���� �H�����=6#+�hs��A��E
TZ����j��99��;y�#���}x�J�U4W]0�\��:�"9�
��?���UuO:���������	^�i�K!�k(���aO���
���5'��?g�
(X6�F6���8Ddz�:#E���3�������Ku�sEzI\�\�����b�����qm��@k��I��0�$N}�����J\|i���+���;��.��?4���i~���u�XdX3��#�>H�uD�&��yn�(de����Z5~�7f��[�
�^Q"��]Y%�Z�+������e��=Gi�YW00g6(x�?�>�M<�L�*�`���5��?��D��2��l������ps�e��*
<W2~���=[� n6fR�����?AVD����u-<��O�S�%G��&Ej�����R��r��T)$��#�%�T9!��������>�����^{���.����]�l���!�������T
0.��>���>�<���������jV
�!�n��c�����������s�K���d����	�"���cA�oJH��LG�����x�����f�
��?SUw�I��	Hzs+Y�*
x�WB6���Q����aM+4la&��Z�����<J!�4�CHd�yG�@�.)�V�/�%�WT*�nj�����H�fb��k
�-�<k�i-������2�������$e���.)CZ3��S�E7Q�����uVD�E�f���j3�<�J1��w��r�7	E���b��R8��Xp�����)�5]�}� ���d0�D������<�'~��������p�J��x�[��\'w�r�C"s�l<���L�lN���x��=����2��\g�c��a�Ell�������f"����V���N
Y�!��aY7~�um��Q���0h�h�����#|��P���~��'����\��~V]�+�P�Q�I
�!`I�
)j7����q�dY���4�$%�W`�l;�
�H������fIB��J�����tP#[6�z�����{J��1L6�Q��G�o��S���]�X�/�]{A��6�aE��[��V6�����VRd:��U�������a����#����������~�����nD���J]Rm��Z�@��:RO&���A��������	�)�u�k����r��i\S��VA*�g�f���0e�������LGSMG����y���dC�R}h�4v�iTZw���
���^h��}����e���\�S'�X�	z�6����S����� Jp������T�vR6���������P5'����o*�I�
�����
�.�*�U$�e_$��T�s���[�Nl�_
�&uy���@�P�+�DE\����,�+�n�9��S>������lw#4D�S�������o������X�����	����[���+��S���D��a8-,�3<$Nz��p�~�������U���95����!�w�1{Y���A��q���3����SI�=�'�Hpc��q^2�A*[�@%@c^�fr-��d!Kq��|�`4,��T����Z�y?�5��@����?�������2�������x��S��5|��������_���-��������?�^��������Z��#���1������Le-r|V���������(":'k�����d[k����=��f����w?����E���H2kD3�C
���&���)�6�Hb�����Y[]�g�]bb�"��;�t�bC!����s{���!�1,�����uQZ����������A*Q1�������.��5���}����~0Nv�����e#D0t��������z�sR��������ty�r}���o�
���G���O�o?��C�@��p��E��������G����|2Q�jIcv�J�����	�X��(��`������f�#q9oh�*k)�������@�����vD�0�����%8�OW�]������ 3����6K����m*�.{��^���W'�_�|�E�j^8G����#�����Q���hR��cGB�j"d��_���p�j���D�I�%�����[�y�Ni	�����B�O��%z�"T*#��t�S���uq��/��_�����������IV(�p��=�o0�xE���#�"��������H�y���G%
K��@����%qiE�OY��������)W-4�JI���%�g,O��x����+}i�3I?����u��FE��~��P�3,�.���KI{s����x�4�'Y|�� ��Cw��6������V��v{�7zc�����Xs�*�D��'hq�b����\��m�+V��M��]����t�}O�+���ryK��:hgKM��V�6N����D>��U�uX=��
!��)C6��M�������g)���!����Y\��&�0�(����z]�L���r�!����F��W6�J���q����m��������o_�2y��(���i����0H�e�FdY-
���9$�g�l����{QAu!�n;���6�m��YTl��X
�)�y�.R<hCY��E�^�E�&���q����r��9�0�D���U�<�M�<��S�Ys��9P��w��Nz4�<g<y��R��A�54�)a!2�V~r�1���Fm>;?�9��LN"�w������RG��S�1�������]�VEL���%&�N��L�
�FR5B�����r��,�h�"qZ����l�"G&2�&EJbAY��n���gQ/��jW�Y�5e��GD�$1��BDmx���~u����U�5�5��[DZ�q�V��2�{�Fl���#�k���.��d����"��D#�D��d��LM�	�=��~'�I�<P���a����MH(������l�EraT;-?�����te
co�|��1�I�m����������������Xhzt���K�;d_B�=�f���fE��M�=���&���l��!)|����a�e��2���M������s��kz�BVC����L�����D�r�|}�hHaF����Q_�S4�G�l����	��T�Q+�/`zq7m�4�LuHs2��m���B��&����(Fy�X����,G�aC#���df;>�v��t_;P�#�0Y5Fh��c�>�&��Q\������&��_,(��}����7���'�&wZ����U����n�F���l�i�[�F�����2���=�]��{^*���X
�]�szi�4������,�� K��3h=�����,��e�� Y���$.���Q�9���8�����e����s���,�s�f#; �z����O�vd.����L`���{�V9F�an�
!������ eaJ�l0e���*Y�@�V�A:�n�PU�����:�#�h�DvS7��y���d|����M�~�������Z��iK��������N��d7��0o�
��
{���i=-�����3� �S�:�{������'�������2��V��
WiK�(��*���B���s�Ye�)�+Z�����JD��&��s�'�����2�����5e�i�k>�2K�%]
����"�5����������0����'�Q���� s�� jh�A�&0Z���gTt�yc���Ii��m`���k�"�h'����ya�|�^����]U#�c�����F��Mel��5&�����,i�2c"�x�����.(��l>F�_�����q]HZg[��F������n���w.��TwE	��R%�t�P��!UU���8D�@���z�a@�l,�p�H�����q7iIn@����kAI'�}�\����#,��[��F9����'����C�]-�|�-���H�����N�@fJ��)������l�Xg��D����yQ9�X�k���gj���8=���5>�at�������&�E?:���;j��v���+���A����=���p��WIq8���b�@���knKH��K��R(����w��;��=�����4���� ��e�)��B2`n���0j	��;8��^����0_O4����T��0'����v^C���c�-k����N�}�##�7�J�>��C�c't��pR��n@�J���|���ot_��h~����f-{�%� ��Z�M��b�^������P���^m�2]�R�N�P=wbS����0�����F�vy\/@a���b7[�^�K���L!�/��YU�i2��t���]G#��M����1���6�o�tx���������(���&#���"yu^�t.<�P�~w+����C�gN���������37<��;05���7������ZEN��1g��Cr=8��w�"�����*�����#�:������':�v��e���������297�/5�~������������h�����>�S�!o���DV���/~�p���}���^t}���T�-N������I�*�Dna��d��$k��H

Pv�v��Cue�]Tk
My^�2�P\l���P=��l���
���h�_4��c��`)����2+�~o�jV����X�zq�*�����;���.�j�t-uxB���r���{�U����U���K�S��Op�O����9^��`����H�4�B���5niy(\����L�v�;�"��t�Pr�4�E����C�U(�w��#EQ��b�uKG~��v���3� ��jK�l �k�9�Y���y��"�m��1�F]��d�i��pI86��E����.6�1L����6u��`��G���{}���5d$��>��zO�k��,�d�E�2y��-*�
e.�se������8���������+����4���A�p@11�T^�-&6���lV:���+���?��(��B����t�Q��e8f��%��3����-!���*��������S���W�D�X���\��h�K�WA�������������1����z,$�M�A��oU��w�v-W�[��J�|dw-W��,�N���oV1C��y�������}3�q���v!G�=��Y����R��?������D>]��S��8u8L�M|���1�n�
�������'�q%��*��������]oc���@���^r$Y��
� iv��2b.�&�����)�o_i*���;��q����)Y[6�rG��A���yj:��	${D"Yb��t���������z�=���DB=��k�m�bM�@���>��Zqi�,�;�D2�'��wl26w��	W��� I)r"K�����t���&dMbo��w���b�
������@��U��9�+���I�;����$�o�-�����\+���k���'*������e�������)�\�:�^>i�9���\�� �*u����g�Y���2��;ve:�;�Q�;�I��8����K�o�YX�{o��MC�U���G�~���T�hJ�'4�y�e���Av���m)�Q
�b9<`���r|�	��?�(=I���*��v?����}��H�S��6F�=6�93��H�r1��Y
�O����������9Z��T6"�Z�s��( �Uo}G"[��m�X�Z��2=76AJTb�k0Pa�$:��5=��������:lN���k�R�A�1*��>H�9Iw��:���){5Dq���1u���f���t{�eV~cV�d����2�`kW�d:�����b���G�
�:z���P4>������[5	���,�t�<�."��k�����:GZj�t����[�A��8���~"�TZ����z���P���g���yX�Jo������PI	
/���������"��	�K��
?G&��O���uo/-�
�
�IA�JD����m|�Ia�\��"L�@%a&�={e%�a��������
�!&���&�v��������)�����
�)�������~���RtX�hgF�T�FonZ&/����g`L��!u��KPK8��;�M�����f�(���[��w���m�@������\��|�F��E�I���\��GY$%�������B0����9aa���".������Ccl��J���|�:�E5�L���~������:�K2������C��G�`�[���#Y�Yh����bEv||~��};�Cs4#�]�$��Q�����L?���b�'o���?�=�Y
\��~�|�d�kHF�k�al��#:.��h��9$P!K��q����?����g��n�s\���4KP�����`�yr�{p���e@��$���O��>���'�e����%Y�g�dd���s`��[8/A�@u��k��!H����2��TSl+0��N��_M.���0�#aW.�h$�c�
��N�Z���R��/
�P:n��P����,f�_6���� �;QD�A]��m�[�3��e^Pm���R_�Y
�F���Xj3~%k�H�\����lq��
F�`�X�W	�,�������9<����
0=���%��j_\�e����uZ$J�C;�PYG��w�Y"i��!���A�H�.�
[Q��S�����\z�������v�A�	.9����+��BZ��q���Rwz�k��U���}���6��S�X�Hen$*����<k�Pm�d$B7Q���YN�h1[Q��������&\Nc�����`-��F!�r�����|	O��q�:i���U���}^��%�����q�g�;���~�E�%(Q
6WI���*���yD�:�/LY'8z�=�q������I�\m�-d�LV��-sR,
jb�����.Y�"����XCd4��NiK���6�)��b����w�.���R
�m����\�r_7���������/�y���a��C�����1Y;�	���XE�}z@�%���7��!>�p���u\=`�����c�u�ahm3����m�S�@���X�/�X���y��{�T|[������Xqo@�����^+�&�Y5�i-[o���X�[4�_G|��^"L5L� ��4�OKF�
�^��d]�R*~��8���,�{�4..�l�I�U�YPS�v�(�[;�E���#g�#���]�o��5a��u�����j`S����}�GN�������@�W=��t���6�Ty���&���O������X�z���_�g'�~�|~r�o�2G&�x\����e�aL
l�'@��b��;jml����x�d(3<�6L����d����F�7:�����f���{�� ?�����"O�2�
�]`(�)�X�$*�J�������&����k��2�6���
�Rh};�)������� gL��K���_}��'Y�i(�"�|�Bl]�zcJ"
�s[���xPp������~��������]+ �X- hR8����������t�2���y�wM��bc��bd�5����(2�>��:n	o��db6���jk��Z������VU�&�	O,��iL�$k�;����T��{?��!��
yA��d^ze4��%	��^m{[�s^������~ ������=Fm��a�j�Q�������y��=��Y��B�@�)-�6mB���W
QM���}P=�~7.�����;3Q+��l#W��5o8�����y}���� ����5��"�Q�����+"�w�|���rzC��|^Q�l�-���AJ��
�|>��D����2���E�R����CEh@���;�
�fT��u�����K�V��%#=�����P��Q-K�J�B	�^Q��"�){�����r��I�S��F�U�k�6�]$e���^����!�z}�6��B=�L��ENQ�Oy
F��,mK�+�*o4X���������"�K��!�F�`ME�1?�������A�[���.3�D�	���
o5�(j"E:������I��G��s�BZ���r�=*ep
(Cp���px�Dh����G=��yC���I�5���D�$���>QiZ�k)2��.�9X���U5�E�Z��Q�X8}`4����c�������SQ���+S|j�rM�D8`l]�/`�j�)G�(>��K	�F2�?��#@�V��hR�\������������0�N	RU���X�\hOH�����,���i�;����I�l�M���S>�U���:f��h��z��������p��K���,|��q�=91��?�!	��B�!?�����	���S���hb�����?��)�tS:�vT�GT�i}r���_��:��4�9�H$��$d������WJ��>���;V�\����_|�%B����$�P�\����2�O�
$0]�.4(��?�UF�NM���Mm��6{J���?�w�(A��(A?6]s����E2��/���T�aJ��E|���DV��& �b�8��Kk�#HP2�\WE<��Y,����b��J��3_���!sp*4zlx=����F�Ctv����Wh��5����h����"��|��CYS���;BMZ�Iv��cp�	�����l���!�F����9��'��;���:*@�������"���/��X-�&�cX���Q��5���a����!u�h�Z/�%#�������K�H�.R,���]^]�7����WT�;�<�������y%��)������'��j�x�6��/-�gI��-V��w����j)a��.��%�G�U�F��7��U��	4�brk�g��D"�/<�=U��SBq)�[����P���<�<]���'w���a���[s@8;>9�j���)�����e�����'�z����c@hM��fT#�/�o{cj~3VP��`��nH3���o���(H������k���z��v���J"*�,|�o�W�Vp��N)�Wb��Q�Vu�W;T��n�4+_��v}��`��������jv�
�>��T�� �E���qH�y�	6X��um��:P&a���M�R����hZ{�����{�g���D2x_~���BE%$����<Y'N�gkb�:�3�d��B�?$D���F1R�t���`��/����i�C��TYe[��C�JN�2ca��:����2�+���d�F4�R��I�!]+v�c���5=����[�FE ���:Q�d�������lq������M~d��W�.��l�Q��1aP1Q��H�Mi�0H���p�pM*�:&�6�wr�S�R�fHI���,Rf�/�i�7��z�*1�����gw��
�E� =��������,�*�"����@1�SA����D�n-a�,�W�2&��j�Y�%�D��~DW�������$�4�ej���H��R�����eoH����������h������i�+�aW�(lx��L8��\)�	S���.-�W7"�A^�r�/����V�:W�]�`;P���q���a�3����C/ePS]T03��m8*��#I+�QB\w��n�����I�����O	��baA��2!~���tc/�N���;!���S�.Z��p��A�"�B#���B��w���'��F�GB�':*~�
pL�K��eW;�$���S��?�� �t���N	�A@&U�v�Q;�c���`�_���c`|��E��&v�.�f�JP��{��=q��N� �)�95�e�+g
-
�c��!,��}�����Jjx�\��J:\@N���=�K�!��#zf����=�m&%)��:a	l+����B8	�w4�t5<��5.~A���9V��2�M^����&V�����c6�t�j�5������43��R3-���g��\�������������?|:v��7%V�G�j_�O��*��8��<���#	v?���������C$��@��2KN���������Y�z�eJi�w�P���'���������� \�Sx�t}i�n@G��,
5�D5)�q��z���������.��h������Irs*3�
���A���4��~���P�M���f�z���"��	oEj�-���D�������&�`��K�}J��u����h;���eB&���$�[��<���4�'�}O���������
�����..P��R&��|5��j k��Z��7���^'���/���~��5�W��	T�6[����c�j����[�Q��x}:����&	E�-v�$��I�{u�L���u�]n_(��
����������7s��7�(>���yy���o	{TU�ewd�rD��N7����|.����{��MQ�����x�A�g��������?��'�%��2���D:��7�w���G�I��$j�"��PZ5`��(ND���\�U4��Lu2#(���L�g"�W�����&�@�KYKr"�%��X�,���]/)����A��X��_Qd����t�E�o�}$�z$�w5X�eq]'�A��D���`���]Kv?#�k�~'K���z�J�{�� U���:%\��z4���'l@=���6����n^H�z�=��>��G�i��`�����E���g�j#��k��4i��YhI~��?�����������&����w���%�k��6�u?�_ :���Iy�l"�h��y�,)$e;'����$�����b*��������$PJ�*�H�`^{z���

���}����7�
��������P}`#|E���b����1X�-�:lXS�k��mT��U����-o�j*�Xn�qp��x������-W:��V��ou�������<��L����x$��?�`t����v
�C�iIO*�����CyC��}	�7����M�q-�8��^3%�IrC!-��0���T!q�l� ��� �D����`�b����K�b�INL�>;�9����)��.��C6���q�x��v�0��i�������&!���o��8���)����d��f��6�&��.V�5v�"�����R����B��T���J���K���b�EE�T������m�2Q�Fy~#U�]l#f�?]��}�u���K��ne�>3-)��*��@!���s�
��#��������)��{H���R4�jl��n>�����3Z$�J�
�����k�/�R� ����sg�G*���6�T<�{��9~��!"���,Z����������}�vJ"�@2����o60kk=���e��g��>�O�a�+'�`v��.�%��F�Cw�:�b���8��X9�40��{�����[�m�V�z���uk���G�-*0M�8���q6F$���%��x
�=��*`�`L9u�����<e��sz��}?2F��,������|@�'����� �,�q�%� -�q�F�;}*XR���XgF��j]y���F���G�����q����$�_�UU��_���:W��9ib��g��c�E�bF9�R����Y��<_dK��r��@�U��6�*Cr�<U�#}�6�m��������~3/(�C���j������������Q�WJ�T����_!Ii��X*-��(�N3�;\.�t��=���k����v�T�h�����NC�R�ID����&^���"u-�!]3�0z��2����S�y�Z�MJI�`�Rt_+�w
�����}�T�`o>��Z9Kd�����sLV�F�=<�\c���{?
C��#o|��X�i|��F�$����dmg����������8�C������D�.��V��?|:�-�@�/��)��,uA�
�"�$�Q��S]I�?
��ru�����5�lY�%���l����� X�\�����$�l� l��u�u�������:7q�Z�����S�1�8�#�nV���x���g�������IH���q�^5�k3k<{:NF��Vf��%5���	��#��-�u@��#G�e��������+V��9A1����O��H1�O��w�q���(�pi^��|Jk�y�����Z��JG���*�2N�AY�h�Q���CgOINF``���W����R��������l�*+��f�	rXK�4�,�'��a��/�N)i9U�������;��H:P�f��r@���S�\.��x7��m8�1���>�����Rk
��rB��J �+�����f��n]���M1S|���o�I�5��iR1��A5�E��|^gd�<x���J��R�_v^�4[���U~���T&��X���Y�a���^�9d��?N�swI/�Yh2�c�f0���*&��)��6�k��0G2
P�g��tJ���
[�b����gT�x�t����j���Y�m�A�.�.:v���iy�3d�<���=+Hy�%�dV}��c�4[/g��h/�
��9��C��&	��X
1��{O������JAH��!YI�f�M8�7����!si%b��Af�DI~��KAt�Gc�S0�-��%.E�9�������bU�:�@1�n����S�	[�^��
�6I����{	���6Q�d]\��	�nrU��X*�����51+j0��z������N�'w�����Z�7=����%o�����S=��v�/�Z�J�V����;�V����7��kl��~�p���w�)�%�r�L����+�^5��I��y<����`7[�a��}a�:�Kry��i(F�|=��	�|��2=���������@�	j�������GT����u����_wI����!0������WpJE���\�pU-=k��S��Y�������Z�GFYSj���|u;HWK(t���t�{9%#v6��c�����O�
�*"���|c�34��6�G3A=�"���$`�
�p�*F���I��$�J�d�k/�L/�X|��l����t#lLw�+*��4���T���KMFW6W�]fd������}{������)������i31IZ�v�������#}�HB�hpZ�6�A&��{j�tmw��R���c���������j�IS�j�[���������qVs1�����
lT���7������u�k%B�c��XW�)�����]�:s�8j
)��I2��:�T�x�\�E�	TO��2?;�-�U����1G}��^e�����V������+{�����X:U=�9[���A�l�1i����AL�x���.��VRY��t,`GJ����y��Q�&i��A��%�}4��_j&�R)��(�.�E�'!��PJ�|F��#�T6k�M�3����lV�T��Y������+�|+-�I$��J���]��!{Q|j����t�j�*�����`�
��5��D�0W�zv�����v��	�z�^*^?kx�"J�;�B����J���D>����E������{N�g{����#�"��.��D������}h��Z8_���8��wa�xc�
(����0)c"���f������[����46����i��X�Pr��$���F���!��@��-�nV=�A��b�M�uI��4K���p5u���^�WY[���h�������eTq#�XLp����
��!�%��W�1>![4��y����&���)����b���|��kF�So�"����rP�V�"d���Q���/���(~��������N���,R5y��!����\/�;
����.�GS5��8��N�
�7bv��M��Js���?56[yI`,/�����|M���4_=������ml�"���z���ho����|y)����/����k�V;�*2w	�T'��:�-�Ih�u�,�X������B�K��J���c�Q��u���3lD�(����S�q�|,r�Bq�
S�c�+f�������
(W�;%��(�i
H�N�y�}�u.�K���x&r�-�����#��R��u,]����
(�Q��_gUeE�Z3P���"����F��J�x��]W�5o�Q6h�)�:�����K���7������J���k����8�������t6��9{���f��\I��{L�,�LIL�����#�������7�cCFZD�����1�����~��Z�6F���vJp&�x�������<�IR!y�p�������fhn\���3`��/�����7�=.
�	�M���4Ek
����5�	���?����m'7U���	��s�P�w��.[��X�?��5�@s0����n.���+�N�l��U����X��\��2����}K��&�p���F��@**io�E��u��g%��P7z�6�i�1q�XP�lTv^~����Q���{��&��M�<h�d�XP�-}�D�����q��uL�������"�!��s��7.�9z]�����[g�O*�,��Phx���i��H���@���b�OfF�a)X�
�X��
}�n��>�>'.U}���������s�Z���u���������Om�[\$�i�����w�n���[E��?K4��
�. ������
�ZI9c�ct��C#�9e?�:��:Ax�����t]o��it]B�����u�?�~��9����B�\���Q<������%Wv34��?�Q6|�]1k�L��oPs?�*/�x�eHS��x������{!�1rs�iA�4l/&��SLSyL%��7(�6���R�����kNA�<6���@����r_�T��o������Y�����?�Q�=�u��V<w��N5���"o�����\�L�m�(�=����K)�����DT��R$�/Vy�w��<*0�2
`�3
[��M�M��@���<��=%��p"~.����8���E� �U���A�w��YN����J����#��;��m=Kz��g���
���H����8 �4�z�Nvy��/�M@����_f��M����'3����2�NF
W�>����L��3��8�_i(���K\��s%>�r��h`H	*U5�����fzJ���Ju��m8���J*?x������r�����}�������^����K�^�?���x�sur�y�/�'��J�T_��d���e�i������58�Q��g0Q�����LKtw(�}�6��S/��o���������
����4/Nr�����2������
:YSq��M\����lY�s��h���`i9=Q�5f:��5�sV�@��9b���������wm���%���MZ���yc�2)7���yy�Rl���-h\,6�[��	������c��4�d������ �8���\���uQLw��=��a��jq�(����-�A|�Jc�5i�����$w! 4u�h��%��}�g]�����:�d�d��K4��,����OU���S
�����R
�J4n���t5��IQ����^Q&�)b��%7�yz-����	b��T�����@Q1���^MX$���������������������|�2%}�3)-<t�p:�5{rt���������ix�x,�$6}�X�J1d�4��!�T����R$������.��P����(�������h�����\�>g4�v\4��B�:��3�	��j0��FM���H�7U�5-q�� ����B;�i�:W����4�������L�W�����|�
���c~�����F���M?�g�>�9��E����Ev���a��3�?����e��xlJ�?�Qu�)�g���o�$�N'��,�����������gO��&�c��f[Ds�%�2R7B[ �AQN��-?�n{#�[wL/VH�W�tK������nB13��,�� ?���� �qK����/g������L2�&G���#�����{��3i�m6�Fl����
��q� >����Zb��UC��7������k�L����(������l�#���O��P:��&C��}0�C5
�l����I8��n$���Ij��W�0~X�]]2�Y���T ��'$6�5���Q��t2�I��05���c�_�8����*L�;������5ke:�����Ee�K��'z�*�.���~����/�z�w����&��ab(*���S��)/�^rB���W�������8mF�$�
�#$3�L]�
������Ir�	��c4��"�����N%�
9�F�HGh�C�OD�����9�F��?e+Z	��l.��j�
�P#!��&*���$|I��]���
�%T'��%���u�z�&��Gj�������E�iB�.}4�{�7^�������0�u3�h3zJ�RB��jz�����2ML�D�����,Y�����\���v��H;���(��B��Mh�z��wr�
���I�<�t�46-o	K@��jnb���C^aC����7���G�#�R��o���v�\�F��+�hZU5e��'G*M���Xr[v2�0n9�����_�]_��$Zz��e����dE���!�>Z�>�����_i���L����������c�@o�!�y#]��v$��c-f<OH�����z���~GP�N�P�M���LN>���sE�0b��� 1MA�
`{�f{g�k���V�u����!P����
��A.ch�X�<c�ZF���z���{�#+�x�����s����@�xT&�1Y�G/;�33�+,�]�&,�E��B�5J2�30ceTL �_����(���-B�x����u1q�������8���A��"#�=�)9A�dR9�LM������d_����&�H���[Z�an��(�������OAy��%�����^BlBFP����IH[wl^��6�v�0��V;���$ SM�V#��d�����f�a8I�R�����G����<�"�Iu#�Y�K���5�[�4iZi�s�p�
�=(�����S�nd���	6����kba�	2������6o
�H���G��;�������'U������}xw���i9h�6#���a�����Y���I��� ]w��
(4�$]f��;�I:d�W���^���O�.�']�����b|�ua>����'�^c���ak�����5��^�I�����h���\^if����PQ��:
�P���^2X��T�R���#�`�u>��R�2�Z������[�)/��`6K!��z��I�np�v���r����x
j,��3,�X�q������A)��g���g�e��������[?r��D$���)�c(���,A}M��v-l�1*�z�x�	�����.0Cq�m1?��7+����ot o�wn�7�����SM��a�X����=S
^rA�a��� OKcW|EJ U�N9i����=��T-D}��I��������M������$S`3C�7������8�#[�W��T���@�'Y�l��
���$����;:9y{��<>�f�=�E��}���?���T���!��bX������k��_�3`��'����eYT����*p`?�r%a���fW�-R��Aq�kE���W0�TPd1�[/�:-%GT��E<W�-������V������
�R���/�#�A�0�� �F}��A%d���t��(q3,�_��E�<#�S,�j0�7n�|������1�H�����tq�7m]����x��)���~)��> ��:�����G���j=
�1�Z��G�������NS���X}�Mf�:i����s�
Bg���5�Oah����&�~�q��9z���j����w�R����f����8�7�J��5�����[f���X���q���@)��X��@y!e����]���h�:y��B	��0��j~���(��a�!L����X�\5^���	��@A!�=��F,[H�s?�H�g�n��FE�_��m��
U��P�
�,T�Z�B�Rg�LZ��j��%�^h �	6��`�
�2M.���e~AF�Fl��2�~�K32f��F�q�
1�����q�.J�B��8�1P>��C@J��E�em�@�7J�-��9�T^�]����<�s1~��h��j�}�������;<7��;�=����`�Sw�/�xR�f�]��=����s����C��������F�<����Wy]�:yV�����O�2��=oxA?�)(x��u�*x������Ab��
�{�<T�<Th�yO{������������y�������|���Q���K&-�����>��Q��C�0�"}���ciq^��y�M�k
�h�9Cu�
�=�W�=�/�P\W~i��p"	�����^���������7�������0Zp���+���U��b�1v� #���e=�9c:��U��=�YvA��_�6���v�����&��r�$K1����<�^?T y�����_����������w�'{�9�WLs��.)62�wz��h�]<�{E_}�U��D�{��F��-�t4�7����P�������U���"�T5
>SC�2
6\��V���J�<YH�8������z��`,8�e!���%�z�n���um��-Po1+8��j�qU994S
���
u~I��:^��6`=��.�%�/����P�/��V�bV�L��eF>v^,��%��&�����(Wl���k�f��<�S���O>�1_]r!���.����9��	~q����H�Juwm�'*�D&l�X����&�bF���J������M\S���o1���$�b�{�!Y�Q�?��lJ��+_7�Z�e(�@:j.�/�}�g���d(	���%��O�U�,!����x'IZ-8oh�#_[��j�P��0�G������;+;I���8O;�:u��e������+�B-�8=�`�7�Z%L��)���9{oO�2{��?�P}�`��y�.�u&�r$*V�9�	�����Z��5�,e������M����o��@*��P�1��^���/������$GX)�PEU5�DZ�������E�n���&���7O���P�=��v^I�Y���$����H@���-`=�VNE�hu���w�b����#l�Y9��D5S�J��^>���#�Z�C	��q9R��3����������<`���@�{��'T��$D���^�rU[B��'�"ThX��PF	���,R�� ����W����}��L�o!'"��l:�Sq�b����oz=p8��n�l+������}�A��m�l�I��z�\s�{���t
R��B����L>���{�2�T�+��$deF�$��/����.�8�:���^f�c��c�q��:��<��D���l5�T�tXKR��lZ.kDt�Y.���i�K���S��~*����a{���YVg���C����:��kcma��O���\��9�}��T	����L���>�I������D����E�4���EZf}����V~�
������G��US�t}Y��"�.��ozE��l?C%U�
m!xf��������{��e�(�J(/W�b!
�e�*�����+Y����$����B����N���y�F�.�������w��'��m�;�����6l;������7�~�1�Dm�>�tT��������JC��k*��4�(�n����v].gz��Ao�/4\h��i*����9��
������������a��K�^
����f��4&y���
��9���
��7%���a_�Q7�R������mn��LQJ-������~��U�~}�M3k�oCp|2�&��$���Qx>����S��������q�������Y2��J�U��,��X�OU3Vk�~RY5���� �|����bt���F-�V�?ln:gu���#H�����=A��^��\X�J���E,��|���;�!��=��Y�[��P�>'u��>U�����=�
K��zi��l0e������g�1]+����n�S��,��D�8����:���S�@�I�lf1��E^����;��e.�����M}���%��Y�)�
����y���������b72�KtK��ARtN9��Kwl�m\K��K�O����*�R�������� �>E��(�	;<r����|"�	��	�����Yo(.��U�A��h�l3��V�%�m��-����4Z��|}Y,�������������<,I���g����������s�\k��MfB����ik�E���l��F�$��R4.��:���{���\�e^���P����r~�����o��/��T�r��1��������}%�����h�9���Z0��l�3w���N�&w���3�����Q[~��'|��N����)U�~1���vsU�g������Z�����#���������G���<���,[�&���r�
@��B���u��Q:�w��y*fN�W�1N�K;(D��o�l����#�L&��x"�N��^�!�!�dJ Ro����:��k�������;��C O����T5���IY"������|�!��D,Z���� �-�c����kg����mx"p����(���$�����������V^]^I��S
�[r.��!��9�U�
���L������ux8���l,?a�$$���t����.q+�z�?�7B��^��z�^�q�S��U�C�p&
/.y�=��R#��T����/l�6	dZs��I-f:��C)����F�K�qUrJ���L��Ga-���[*�&�1��|�}Q*)��1��s�����^����cC�]l���5{��F��I�?��V��S�q#�b#r ����9�W��S�c2����.��_��|���� �l.2��8���;���t��,5Dl0��q0�k�Q����Z�j���j�����A:��/�L�9`c����FV���yx�y�T,��[S���`��a-��
,5U8���*W����I��������b�a�h��H��	���Ia{v��J��z�%nn�����/[����72��"��c�O��}��	s��sucnm^U���6v�8@o��7ZNJ�JS��T����V�z��G���#6#����<}�1��1�8��4a;�:�nS���!R�!���b3r#Zo*&��+p#X���$s�����Ny;k3V�]�!���\E��{�K0q���`����
m;f;Ts����+��D� k�b	F��o(�|~i�)�d0������������8�3�<�~d��G~���cz��g$�JV�&��?�3�J4R{���M���H�l|��o�!]�I��Q6��O
x@��S �(	������\��6�g��<�����7��[U���c�@K��h��H�
	/Y���9���5�D�]����~��OF��M���"�u�%�����xc`v��J{fk&�a�-
8;��4H��d#�������WO��4��������k�}J�Q�3//��QOd��y@������x�����$���@�����8�n�����'�<���*��
�4eE1l��������)��KB�1K]g�������l�h1�P�8�����@�=�c�q����J=�b�y��8{���G��o���;<{��szt�kw,S���O���7��@h�u���\m1����)7&�j���V!$��H�����X�@Nj+��d�l��88��1S��H����mP��t�;��ic��R��@;�����s�0�������Ri����
�9�A>�K�q�s+��#�/s���K��f�������@���l�y]�DS� �%6��;i{���
�b;��`������:�a�k����%��^�)v��zI����=����]����O��C���i�r�A�D� �wu�?�
��F��[�$��5f�9o9~�u�����1���J_�����Rt��'�����JR���^����X�����D�!�6{�Y���X�������S����`���0{.`�-E���l��;nM�N���/�>��F��v������d�)Zc�����I�k�#1q�+�+0���F{�v�����Y~��}wv�����L^}��{�k��Y~���\�
X�b������RW����x=��C�-195�5)���%((�Bmh��n�~ "�eZ_	]u�����"�S&j!�L�n����vE�h
��qv@y��o����d�5��s�z�JF�nOv��N0����y����9�dN�9Ew��]�
��"�%cX��M)��� d���o����Q=v ?z�����7�;{���y���7��������o����������_��A����5����c&E5!P�Yy����"wt-����,��C�� �M���gG��-U�	/��+9MH���J"�0�k�6{�� �d����&�1?(�m��q������L�B���h����S�>�����b��������{I9.2q�=������Cq���4�DsvM�Z	)g^j��k�VM�$�D/S���Lc���/a���iN{�	wK��Ai=$E\7����u���9�����3�����~-S��t���lD�����B�m��{#$z3��>��L����C��������g��t���4��k�&���FY�Z)y��&�j�������u^im��]��r�1��oQIT�qfJ�Q$��q�L��WM$���������!?a��o�E���?����]n�]'�8{zs�5���@ �s�!��9gPJ�'����vy��/%��CG�������w���{|oo�i���b��Aq1D#L?�D�5M���N���zo�����k�f�	�I��Z+�@��~����=��F���M��s[�Q�=������ ����~^��X6�;�*��Q�ACf����:b��������Z4�������@��:8n�oDh�z�}���!w���on�l@O"���������R������!���|7�{���L%�y�����Q)H	�U�'��
>�#n���p;�g|z�F���QFnN4��s���-�g��)�m^��X,Y�.�Z�Sn��ZA'�3��yA���[-�\N����i�)m�I����%\�9�g<��{�$����$A}$�H���M��%;3-������{"���b�H�\���b�%��V�U>;h�
	/���|�����Q��3�`\/���
 �~S�������{)+���_�%��t�N�I����W��Fu:����k��,N�UE(���6�;U)�_��7���Ez���U��Uvk���h�/������Ei<fK��{J�GD~������B��d�TE6HPu�����D��b�vv��$��E#2�$���h�F1�����UM8!Q��$R�)�yES���9�5�������7�`n�L���*�-��Y1aS=W|�H���9���t�����|��H����r��X���%>�1E�\�:)v��K�
���4�!�K���>G����4���wn������Tc_���a�$B��	�.^�������!��h<>���ub�j�q���o���
�SP�m����Z��:[)�)�%����c���r ����|��49��Cs�0��\)0+L��\�HQL�w���L����wQj�sb�$Z|+`@��w�JGW4_J���>����faL�5��V5C��JS���B-<��'<�u�y�n�8�Y@�"Cg[�:�~F�Wz���:�%��b�Y��i`�5k�]��e��^����-��:G|5����L�#��-�{W+��|+����K�%������h'�
i�+,q^|��w'sl.��M=�U/lHf��i^���B<J�P�B�~���1�o���q�h��#)�U�m>��
�Q�����
o��X|4,!:J��@@@n��F>���N0j L"��,��@�]f����>W���5��u,8�F�zz�1E�h�����h�?>ZX[�L�J�a��Juz�]WJ6S���)N�����P�����s��L9:lP���tn1:,���`/,�����LGZt��mt��j-��Z}I6�Q�Wd��h���2� �6�#�+HYZ�}^�F�[H@K��O���"*�\?@�3/����M��zd�j
a��D�5G�2sv�Ka��0
Z�yY�\2Q�O���~�=q�|e�k��W�G���)�����k5"�r]
���1��R��i^�5��i��y�%V�*L�j%�H���d������'�O �~
���z��������	����9
�O��o�(�X
�5 %�=�`X����]��	�����%T7�����K�B������~���N���g~��S/_���K4�I���[��<�����Q=P���9�'c�z��+7����E�"W�Q�/�"�1�^�]8���Lo��>6���R������U��bA%~8*M�/�"�l�7�A��\��$���L������D�g�A������T���L���)�h�����Ev0�1AUN]!(7��U��U6���3�;#���MD����w���-���{?�^�.e�q����$a�S�������3�kq��}��0�z��Qw�h������BVE�@V9g�2*cK"*��?�{����g
��u��S�0���k�I�c����1��3�����A�V?k��y"<���E��G���%$�p�%�&~��[Zp{d�QLh��uY���/@v�}o����}Y���7W�c�yc�	�`	o��^5�^/�.q��U5���m�y
�nlG/�����8H��V��|�����y����5������#����	�Q�����~0+���>�Na���m�(`�&p�`�(�j�E��EB�C>�(��"N�nc����Zq���}"�
��W�.��x7�]���T�(����4�(H{56YS�_j�z���!�K��r�����^q�qa�����c�]����N��Z��X�m6���x ����|����m�*��7v���1z��a�v�?������A��<4��%��"h� ��4DI��aM{��
H���+$�����m���m�I�)Q�d��~�u����.�r���)�����T�b-���Ur�������&22&�pj�����U����wt���^Y��C��;�\X2xa2��r���'K?`�c�,MDi��N��<u���w�S?�:��MLj�� �A#��|���<X��P��|���*PI����	��V��������0�bcz�;0���u�� �LJ'k`b��j���q���el��q:s�TSZ��"������K:�6?�oN\���v�`8�2���)��`)�Qbbz�H��@�*��mi���;`����tf�)H�(pd$�1���}�&��B��W�������H\cFB�����M���������-��&���Z�
��`n;]G�=�k���%������������:��+����V�6�E��tQO�b��[q(�������(
u��E{hr�no��f4�%Jc$��y��7�g�F'B��������������T�7!��f��I��,���l�.������n��	��@�6ez�]�������_��YP\l��5�N���q�<��neW%\
J�3����2�.l�Y��U��0G<B���QpG���-ow�^��a8�"��	kU�o#N����kvb�4e��,'�F��S����������7��!	w:���h6%�pC0*1G���U�.��������)j� �i&z����]��(x�X�FQ�wg`]�<@�i:_s�EIac���U�H���J�I6O�nNg���H1����#�sH�H�:�eN8��L�����L�@=w_
	A�7���!�d���.�WB��c7��P��(�"���
�����T�=��D�p�~L��m��]pI����21����JT��7�V���������j���B
K�t�����Kq���?�;E��E����,�	����1�
����_�:�WR@���"������$6����i�!_
�c������}�������xpI������'u��ll���j��z����7�#>vv�v=����O��4By��|����������Z���������c'	�*;=w��8R0�G�?Y.I��N�J�����������'#,uk��h�lb�����3N/N�o/F_��t����#k��,/����{��i+f��?��O0w]�W:L�����l��h�A����xy��z#�-�w��Jk.fo�Y�%�F�d�h�80(c�N�!V�8b����Yk�E�����SyHAj1/R�wP����&����8*4�]$����{4���������O�W	��q@�2�q[�-~�V"w)������(L�4���������bD�����y��#�@���#�����(�I���7����h�7�o��NN�^�5���l8������V�
L%I���]J���l`�(���Z'��^��>_MM@p��:$#0����$z3�hca�P���,k��s=�R6���bC�b����$x��Jq!��,�\��o@C�� T]^k�	���l��-7L!��t-iRC2�cI�%�Y3[>+]���;�����5��I����t�:���zQ�*A���R�N�����7z� ��:g���L������B��xL���B�u>-.A����B�w��V=��|Ew�f�i����p�Q#,)����@������X������y{��O�A��r1������&�j�t�46���,T�N���aKB�b����)��E���'����2�$
ba�1�AQ\q�aG��utI��<���
�MI/p��G�6������n{���Mj���rFz��5�
�_���gXu]��x�X��;87�R�H-h f4�)�G�q�^�Y��hZ�L�(g4Q��,�y������2��#���Y��j;5���#���&#Pk�����0�?����@�=�*�'��w����"5T��\FoU�1U�3H����a�HU���B���,���*��X�h�K"�����0�P���\4���^��n]	`Q���_���B�O��<���2���"v�+�,�5�uM���^"�@&��`��
@ux	�8���M�_2������d��S����j�"�������;`NO���b:I�p|��d����x�]�F���P��<yI^"�D�vi���mNrx��
���t�Z���5���������{�����'`R!���������X�%�|88�8���L��n7�x1�.�N��K�wI��w"�4�V�(GPn��BP�	�B��6H&@�YW�.�)�������!��e�1o�:����=9;�����Sc�^����e�[�v�����������Nv�}�!���6n�D(24�	w��=
K�7�(�O��Zcuy��.j���)W��hoL#���/c|�Mud
��{�%��JO�T��Xj�H*��F�;���bR�����A���
�1��7A;�`H�\"h���.
�K$S�o-�u�}�W2��vr��MD�'Wc�)[	��~���e��c�&����)f�M�NQP�2MX�O�9��������2Z�U��<�]{�F=6���\���b��@�(Z�JH�����U��\�w$���@1��n�fG�tG8�M2"�a��pe��;����p�����>	���I�����C���u�Y 6�����&a/j�Y]2~D[!��!�4	�IX�����
����g�-����$V<��<�s�4�*�<�p�����lG��n���W������ ��Qo��n��	�}����^��O���x �G�j�x|GL�}i{����"���m����C�����PFc����6����9��I�Ivc'���{'����F��'�i�����&t
���,~y� y�
M��R��m�����`�P�������
XO�� 79�r��ZHv��~�?L"	V�`�]�7�r�I�
L����=�����V��h������9��D�_����|'�A�����������&�e{%5Ea��`1D	9�@�Hb��Sf+R3�[ ���Y�c�1q��O_bJ�y����XP�3=�HH���DG�
���a���=�����rr�������&�^�<����WG'�}�������}T��X �������=���.>2�Y�H�h�[A�Z��T��<��y�JUa��_~q���~7m~7�������r�����=l����{:�dD��9�~��T�P�O*"IJ-�@���"��44U��jc(��Z�NV�Q�E�����.���$�U���lI��^���cxl�ha��\�.'%�p��$�U�iW��O���}-�Z�	Qt�O��L�����x3�w�#�|��]���~�.O:����t��X!{��Z�Yj�^��(U�9�}w��m�������sg�1�t�UN�s��?,���PDw��i���.N���J.N.�n����&�#��Mv��g:���?#sBb��r|�&����.��$�z(�J����"%���\��|�~p���A�[���<��2�^x/�2���m�l�����N���:��-��\=r��K�$h<�_K��kn���T���l�������ca�����Nfz����(����8�����N���-�Z){�^� �]���i�<���5�L��w������`��O���O�a��nI)S(JS���i���i�es�����k��(�r'�D���j?�`�@�_��O���E���jM��6�$�����-B��u�Ws_���%wk�:y��6�������q&�{�hQ�����I�e"�����"�K�X�z#�������@�GotR�.�b��Xto��p6�AI������C$rC����*P����S������\�!>:��V�����k����h�/:6�A`~�3!��o�sr����ON'��oO�&:Fpv���S'��t}��\� ��G���"������7���D%��i����2\>�����������JCM�����U����>�j��Ps���)��8��4���<<qf��
���t>��`��4B��6t�:�����[��'c�?l�Bm�)���o����1������������:V�l9�|��F��[F��F=%�R��;���P�������l5��������lt:l9X:y������+��q�xk
��+�b��<��r/�H_�B5��yd, Vc�o\������`��<�dd5��t�T�T���$��g��S���4���,E4�}P�As@�*�
�wAt��AnW2 c��M�8o��
��&�����/��%n�-�+�>�;M�@\�[Q����`�u��x�[��/����"�c'R�8�;�O���&���u�.�3��4��������'�7����N��Kr20=��`�4�g�2�p��`���c���;���~�1�%X�t@	�4�s3j��?b���V]����z�u�������B��jG������*�R�N�|�������r�����?:��svx��������S�3z�����W�B�����\
�K�`'�%�-L,��&*�#���D��	��	����2I��zs�UKl���a�����X�7�S@[�YZ����`hbr8�F0������w%���b7`���M��
�<�!��^����>�/����P����?#�X�Q
��o�1��������G�5n�Y�������X���d����v��	�>'P����y�ma���<��L����+�s��!��q�2��F����X��*��>Ti��1�pU�]1��~z`,�j���g���~��Ai�Y�b�L{��[*��m%(���^��aC��^9%�Px�1�K��6f?[�I��F�5"7��e�Cp\��b@��-�Yq�F���	�������Gy��I4����p������1Gq�rH7S���o�	)Y���^d���|�����������)^���J�Cq�������R2��M1��@�(_L���%)(3���b9e����R.T������-����P?<=z����:��������Q��
~�|].T��J5"�d0>R:T����>J������H����0I#S��C���$-j��F����.s{�q%x�� ��j@)���-9�{h�"7������.X!v����(���~O��?����(�H�h5B0����������D��,�b�9���O\_�.����$��'6�D�x������B��3+���iU(&B����I���/��Ed��t���cC
�R-]����>�,3����5�Z����]T\�@D���cW�D������j>�
yA6�IvM���I��t,�K�'\{�"v�d*l������u�"�[���~�.F#n�8�	��8���Z���e��F-����������)!�/�],5�w���#��A�L�v0����v��8��0������m�`���
mJxP�<���t��rd(��x(����7�\����w�@���&�����|��u(�C�!�q�����Z��M*��%7��u����
&�_i�,�a� J�{K�?zC��t.rz;�F���Q���F�^��g������'f�i�\E�<{�<�1����y���.O�K�@c��5?�Q�S��Z�Z���v���-c���+�\k 1L���Vi�z���<��"S�(����D�S�c ��A���L��S���\.��Z�_���������SU�#E�n�at��t�������;�bz\��llfS�GK~��������o����t�����2N�.[��|kx�I���yU��k����t��<�F���E_�z�I�:�I���uX��v���5����Fe3��:�V���o����Uj�x��$sR
s j�����7�m%��dTj)6���"0����,X�r.���^,�W�C�Z�%a��Ba�Q�N�����������(��������7P�j������]��[[�rv?�te�Bd�CQ�-"d�o`��Xl#�����R!���Uj�]�xb����6+�y����6E%[�|�����<F���_�(�)���c����H�G�
U��u����y������EScL<�(L.����%�#�I��g���I \C��&4�;jQ���*0U�M/����	t�1��y_���
��(�L56���R9V�~�Xr�j���j	���w{�cNO����vX�
��ZxGV��G�s��&���z3�>'�g,t-rq���@v������I{��|#���*����Pc����2	�G��J����7���%ma�\,t�<.�D��y���(�jX7�v7d^eG�!l�J=��z��z�	��"0����Z�u�|�P��|"VdV1�X���R�������c��R��C}�l�U.�����nJ��{��A�R����{V��n�-��J�����rOb��[zzG;�������{�MC�\�������<�O(!)������W���F=\���z�3�s����br}��4m\�Rj�M1E���=L��:�V�E��cU������s���w�gPv��r5�����q��y��6�!���1�����=^Og�������G�s���y��7�?�����@����2sC~Y�e��f�*��]��k���8M
Da���lFq����$6^+���M����r6�
}F�|P
G�� (���v�����x��,"U%9�|�!����yt�Zk��!�s2sJ~T��J3�@0#�����>%�?3���xW�u;�U��UIX���(��b�T=��j���K����!0���X���
�M����]��H�Z��t��(���$L������Q�D���E�5p� $�j�Ak����������F�/�^��
�9�7V/�}M=rUM��]���8����4dc��ZLj�z��4���Z�s7�$�f�����/�����Mve.:�bDv�j�(h�T4)��n�Q�!�N��eT!�����`9���v�(9�^v�x�$��P�D����1{z�����C&��	�:��Z���5���-�~0��S�j�����t��"?���>`=aM��b���J���gT�� ��o��$�����:y�/�12#o�
���.�%E���/�b�L�T#i���8;�uT
��Ld����x��1��
����h��#(&x��3b�����(T�uC���i#�Ev��>�B�*7�V�������o:���56(�5R����=���~��l������k�]�M)�z�������3����~�k�+�b��(����:�����}��P��&;�>$v���6��_5��oR�F�����g���G�/y��2������o��� E��)������2�p����=����#�xq����a�.�I����bk [eg��Sy���u��?//��R�tl�P���A�{[f�l���3��,g�-&�dr�o�����������3�����}k�e��ln�`��*n�H<��bQX4��.���������I�8S�W��E%
��P~�W����q>F��w_
�z����
(��1��w�:����i�>s)Or�����{������>�����W�e����.��'��������{���`G��?w|�#��h����^8�o�&?��|����^MF�7������c��c������`��%/Z
dz��6w��J=�3TM{�����Y���+�Q$;�z2��.�
�'���7�c�����������k��3&�4�S�(���
����9ym�h�lK2�g�X��������8��gjV�q��������wXn���p����[5�x�:����|����������B�_y}���c�i���
��	������b�������1��15���dP���]s�=l�x��@HK�R�j��)O��HK_�
@&'�NIn\*�u{&J2�\J��3�������}����WT����������_���O"�L/=�T�s��vz����Z|�m�@R�����������	�d����Lle�	�'���Pg\�j�i���d��y�
��0Is�39��{���S����3l�H��i�Z9�>��E%6����:g`����Z���.(�Y�i�J*�1��zih�T�=lV;���g�[c�w�aY_i���m�sduN��X2�C#r�3m����&���P�PSO�-�kR9]j���)�x�������V]����	��2�	?~�1��v��yy W��3�p�^�P�������%w3�}�q)JS���r�MF����J��>�1�iG��O'��������k�h>��X���Id&|��.S^_����o�����)bX�Y�3�e��������.r�"l@I)�?Q�t���E���d�������a#�� �6`lK=�0�cX��������u����i]W��N�Q����-����y.8���r�� ^5�:�e����r��������	����;y����P�PR<F�B���j�������j�d�x\S����F���9����m����9:W������m��%U��Tw��u��Lf����@����A�("�,�8�C4��)<�t0�)�=��m��6I�_*H._�������}|��TD�{��`�����/N�84?��k�)Lk���>!e���	7�
/���*j���������& f�N��%Y6�W���r�PR�iD� �'�^�u^5�Cb��b�
4����
�����g�zo_�������������D�n�>��t����UyvNQ^���n�����)�:S�_��,�p��;;����N&���}�=�5Q�,?__N8��B:�N5�����N��l!���:�a�����L~L�M1���Dy,%mA��-����=�{%Kq���j�##iv6b����a�F ��1��V�U��	Q�mn�u0LJ�����?�L����0j�f�:����R��&��ws���U�L���d���T_94�u���4�������0��"�]���������3�sN�������|Q���n|w�����|7�t��x��}0��]&�G@����g,��A�)r�	���o��<�|�
)�U��vu��w����?�&����z��w��%7��Q�=��1����c�1��|��I	�D/*�~d��l��n%z�7�V����nGJ�����	*�:��k�c�x������*�8x����Zf
^��j��AgF��t��@$���y��"����1�����?��
��'
U���T�{���e@������6;��f>eY��}g��Rx�e���5%![����`��>W����t�'k��������*��.H��0��X����n��p�0?�b�������0|k�P�N
�qU$��~K.���K�E#���[���		b���Dmp��|��4�����T�`/�h\��:�����_�����K�����)�"-�Ya�������p�	���j�Wtl�#$P��D=�����i�'��a2A��}��~W�z�>�G��f$.��������)���)��)k\��6'���v8|�Q{��i,���7����v2|�i{�g������u��f��>��'���'^{�?e�u^
���1��t���)��)����$�����l<|�I{�?��|���nL��8�t4x��o�H��asM����������?|�w�K��������k�[*
v�g���V2t�i3Q�
@�g�l�M��m��'H�>��S:�>\����1g�:�`����)�]
�q�c��:�x���f�g���M;sU� N
2MF��!���u��|����?t�-A�;���C�v���2�h�t�f����0��3[��h:p���'�c�����V���l[}�S���!S��)�0���������D�j��z�~$��"��j�a3��e"j������8��0������%/�E^���b��]������������x(��[<�{�R��.�����G��oF���>�~�A|�*�Y7��w����`��	�Y<t#���W�mm�i>�e���P]���z����0h���r5d���>F����^�P�X���V��;�r@�\x����]y8���-���r���a�	��@wa�n���{1��-vxxy9`�|P}�=�b�z���9��"o�����b:������t����L�C�_������A�m���t#�:��e���X���A�����8]�����3��Z<T��[�W+�]]��FWn=[f��|�
�T��m^|��"��]U��
�C�����.�N��,'����p�������C�.��&���G.�v���}y[}�1�����&H���R�N3�:/3�C+j�K[6�7y5`�i�G�.K���@����� j�M[�\6
X`���_�5at���V8��l{~��+x���?�>���'`�����G0d��T8��}��S�h�d��H�����eUHvU��E����S*�	��������
���	��mD?�"dB2���������E�cB��������W&Z"��3��A�����f"O���(U������b�]�0��dc��^����U>����"Ut]l��o��������w��{�^��/B��7[���|6�p�eU��y����w��g����!����FTu��������.�l�?A���#�d�oSP}�!���RC����mI��������������1�����oIkb>�����s
}�},m!$>*=���F�Q�=�DU����I|�?��+�t,�<������K?:t'��k��n��0���^�Yuj+�}���~	O	�$��s^nu�'��V��bZC��c�%�v;�"�W��A���%���&�P�}
�/mm�F�%��{������n��B�R���g��#�yzr����x����W�u��+>�I���g����ng�UN���CtG�l1�
V`���
�on.o$]_l�eI��EH�='w������S�e�z���v:[Y&��y���x��"�����a�W��m��fv=uci!����*�Vv'�[z�@����-.��	}��o�q����V�����bV_i���e�!u�^p;t0,Y'$����'�����u��s�q���7��e�8��Av/�<y���)�S��B�?��C�$��\�?�$@��N�4|�$�����m��[���mB>�S�I��I�u�f:����,�~�b�Q�����.rR��k�p^0��]��k3��\ /�-�77�,,�"���8L�i��l��OR����w���?�h��N�^����f�F��m����5�F���M�1oa����bl ��s���V�o����|9��
h-�
Y}C]~CV9:x6uLs`�g4>-��i>nz@?�~�]_5�]N�H	A��UVg�/gH�'��d��������T����������lH+���0���c
����R�e��ZL��=*�X����z���#��<6XnQ�a��w�{F�Z�Y�^Q!����yQ�����^R�)uy�s��R���[���r*pS�;�u]0~T*D�p�
\�?�������=zY2������}M��T.s�+�B�N�h��\��n�&(������M�������o���������b4���)�T8-s�l��"`��X���4�'�!}3$!�fu��n\{��S>����1��������W�)���p���Y���&����EV�/K�xl��3����icW?b�q%TW��E�(������aQ!�X��z-\�v?7�B	���o�Z�T������j)����\/y�J:�d��7���������R
��rv����JE�*�?h��8��T�m�alPF,g�����A@�LQ���2�\e`�?s�����c���x�������IZ2�|��a3.�M���?�SE>.�U�RJ���������3D�R���y{�>o�\s��U���"�]�5�y�_\P
���|���)�e��a��c8+����Z�;���J�u�v��������F[N�3*;�HL�4Ml4.@����Z��j�,^�W�T��<�f�����T���\
����
^�h�����
���{��=}{��(���v����.Q%[�f���~��{��\2b2ocqA%���$6 _T���8k��%�W����%�"mK<J�2	�C��r����-��uV��5�LaKY\#!�DBUl�w���g.���������z��s�q�;�����H�^*��X���H���E����q,�m�&��0�%>���W�l�K�
y����L�lT�����u6mvR�=q2�'�&�\��R�yq=�n�j^�`��;��~����Y�:�%�_'�&N�(���%���";������+'�d`�/�����"���h�����Ia����$�>������g�V���������H'K��z�nF�l"B�Zq6�u�}���D�W�ep+Y�~��B�i�A�X�C��dIr�N�*zi����K!�j�/�UQV����&n��'��9��<>�s�eO�o!u^�]M)G�����={��)�����y�	�z��LT�c/5����L��'$P����R�e�����\�@��<&��K�_���8��Ku�K�3U_�y}��Mm0�������Rsm�(�k���Z��<���5�O(��{�d��NGO���	B���7�(*����aM��q�V���1k��DL��>m$1%cRmg��0�6�
`C�2��c.De����5��)5��$,�k��}I%���-	f2���<�$~�$�X^��d����m����f3�&0��
�����_�27+�t{l
s����@*U~$��@�u�hP)&���
m��;�T�Q���x��>:�[M�D�=��a������hn�$�w#��M!_��������^H��[�XQ����
���S���^���<�e�00R��jS4�`K��!%��Bu�����=q�n�lP����F�S%�E��j.�R��"��J�c:9=����)6^�5F��XK�L��zo��:]�9�mt�W�����X@s������R����v�,7R=���/���k��S��k�f��w*�
��C�����3�L�MS���J�h%g�Yu��@��"�9RF^�������
��:�-��^�������] �Z�:gh����H��)D����3������������v�u���O�j}t���x����{�|?/���}E,��&q��lPl����
7����=����5���M`j��f/������o����7g���<<;~��0�8�T�z�m�������X�U���P�����|����km|yv���tQ�U?������������_������^P��7"L�j��NtG"S�j��^��z�}���vik��))�d���@7?mwpE�����!&�v?u5h&mk|���������E�S��zY�4�-�%�u���p� hM|?�V
3��9p/��^H�B�HvvA\��M����n�����}��iN��s�JR�����������E��K�����w��[N��-V�W�r���6�*p^}3�n+1:�}D[}T��b�
H��.���f�lu!��^�>�(U�A1���"=��)��y��{o��S����K��7��Y��n��(�,BF~w�Q�����Y�>�����l����������&�G��;��h����d�pG��u���t|r����������V�/������zP�q��-�u���7��~>F�������������'CzOm��<�M��O_�>z=x��h{���y��Gj�!#`r��zc����_lM�����^���Vy���������3o��_r�����`��}6��8V��U�U��x��Pp�j���hB��eu��S��Y�����_���x����hxOn��sF������.f���7���H�:�\)8���}���S}��>�J.������D�N�lX'�F'��7�A��l���N6�V.OhP?���(�:������7&4������B/���� �0?��CW��m���}�=�.�_��QT�$����e�B�����8R�I�e'K*�M��u��X�I��V�L�(nJ����������2�t>I����D�L��T;S��!$#Y��[�iE*���{ ���R��C~.2�����8�"���ST���D9�+�u�E7��j �����M�;�I�7�\���&���wP�:M�:l�������!F��q��%J���J]���*[�QH7
p�Sl{2�^��k;mt���+�.2��\�������r�2y�h�����K�U������mpp*�����
�@�D�nLs���I���<1���2	����DC>�w<�	H�lou�I��	�n���D��unu�'�w����:�*��N^VT��_������U�l3��U���kw���.�����c�n�&��z�Yn����-������-��$�������S�/H���Z�LLm����u}��T_�z_�b�����V�����A����uEV{P]T�I2���y'���Q���#��}�!	[|�&���$K���T���7�
��+V|��_���������D�!Af����tz�!�,�L��G��J�Jr����'��+�;�'� Yn``���(��O������N8��b��X�t������������6��h���4��ut���93�L��2���������'��el.�e���i����!*w��g�����R����)����ZF�.�r58f�+��/�u]aA(D�4Q??��#��p;s"�3]�8wU^#s`5���*��3�v�����z�I������Y�����}%��He��d�Q>��q� ��K��Ed;mk�%P�oY����ma��B����eB/D>�r����2�j'�,�X�?#h����(t���n��������B��6����<�R�9�<�j�3D�P4���l�F%�s�	{�4�i[������!�$�{8�TVF6s�<���Q����mv������4��d?�@�U�|X-=�h��y0�
�Z���V���d��=�/����~�o�a0�c����m1�hy��]4me
�Y�u��9�$�	?��Xd_��>�P#Eo^�6�w��\�|�������Q[���|����7�����>V��J1����_��]������^�@��
�:�
����NTW����.��T��b��TZ��^7�d�;�Ap����
T������h���.a�r���[
P����j��#�i�)�N"x�5�{���@���Q���LC��D��v]6��EE.�*=1�����r\�n]�F��l����<~��"���
�(����ub�"X��L)Bd}�vc'�k�#HZ�.�'��*[�Px�B>u�S��0��C}��[���aE����{:����}�nL�$u�k��K�7�c�T��T���4�����=u�'�-Dw��5���-��JiC�p������XPU�j��/i�H5�oLU���c�m.��&68x{�~�Y���m���<� ���o���?�H�Ud�v;0
������K��E]0�V$I6�����0��%��F��6��#��'�����;I�(r���:���	� �H�T� }��fI5U����k�;-��d
{@ ����>H]Xy����|��-R-����uJ��W=vY"}~��_"[MF�gj��_��=�[�N��4����^���]L��A��5��Q�HF/`z�eiH_z�O6�A�,��8\zb�Vy&	��J
OF,dbf�&'��l
9���w�v�8	�>	3V���9���w����tB(2��iY�E�����������&�X����>��.b.XDs3������f���^m�X!J��s���0Z��Q�o�8B'�X���._���F���~�<�����e� P�v���P2��4���J���������&���$9pd-����|ssc��Rsc�Fdj`R���ig�`q�|���aqL������}-����|����J�o~-f������l������dw���D[�����z,�^�����������N��gl����a��`��.���������U�
Q<����F2��Q0ED����|��Tf}u}����.,��
.�+�����l��vd������7up�����5VL�}dG�n������x����_���w�I
���Q�`*�G�z��yz`���O6�4
�S�]~0�t��>���XU���x�t(L�
�J���^O�B��W1����c���Q������P��
�wJ������?N����`�96�)O���f-b5 W{W�����r�	B=!u�O~��7����h[����:z��6d�
�Y!;�dM��
8�c6�(E�m�F��;\FTI8bB��>-:�n������S���|��@�!����^��g�!$
�e{��2�������{��E���}k�}��{�:{��}ps8R�y������2�>�dPhhFw��7?~x�����|x�T*D���K�L�3��Ef�
�YA�3'�l>S�F�D�tS���������L��%Ir2j|���l�v����Z�u2�Am|�9���1�k����Bl�����6��pa&s�x���~�`����y������1mq��<�����y����e�?��j?��u;�)u;�~��������C�kst]%��4�I�)�xe56U;i�u���dk�4L�W������R��0��w����l��]1�*�2�0��7��a���8���4A���@!�M��{��(:!��WF��.�J9�%�6'�^�s=E�"m\Y�����a�{1^��g!g�"\�-�X�
�fg���Lj�
�\`�E�1��N���/�o�
�@h']e��U�+%��<��<�0��I6��i�5��{�mCYYO�;����^[_$���;��>my!x�P�/���x�>����c+��m?��������[�4��V�������C��z�J�~���������6t�n���c>�	��1^oB�!�>����g�����Q f��#�)$n�������%�Mh�^��)���!X/4�_,4�E��Z����a��d�z��T�'S��9L��t��!�o��g-#2�\R����0#w3�����:�d&}Z�'�!]�UJ.���X����4��(B$������/��F6`{<��
��x~����P�P�DP�aF���]�0��%���^L�����sM��eU��a�&@���h��yFL��8�dV7���H+�D�X����=
���	��F��/x�%�t���]_me)��������?��Q>�6;��VG�zI3�5�����`,���mP ��	�������l�	�z��pOa�|�/��Vvg����&�L�I�>Q�N��P���������R���p��v�W"tS!�C�������{�����z|�;��fs�v2P�K�X�!���,�7��O&�D>�:�L&���~����D��t1� RS`Z������H:�l8�������N��~��cv�L�\��+QY<.��%#�����U#.[sD�]	����q�����Df��C��'P�x��U������N�3����{��}�������T���0{1�e�2��_�!��.��L��M�.!|���P�/G������2�up"�_!z��D�������Jew./�3����V��t���dT5��'	��+��f�r�L��|��_l���9��Q�5jh��=�YqM����5�J���b�`�����6��~�{$����;&k�|Uh
�>�e)����7�r��1S��Q��_�
u4��V\Rd�z-��-\�����R���*�Qc?238��.��Np��-o?&��{�R�_������0s���{|O�^�9�B��T��8vnq.����G�s"2���xC��,���~�_�b����O�H�)�G��lS��M�wC�T����Pdv�(��BGQ$�[E�N#G8����$L�B���O�0Y�L�%W7��IY������q����D[;���r���i)��E\7�Q�	�t*��Q���<*$sp��k.�� ���h�N�^����|m$7I�o,��0�c2�����N��%o�/���_$.�q���3��0�-��m�����v��������E\�M���_K�W��i��I�J�*����������N�l	#v���?8�����K��F\������S��D�w�y���ad&bM�&�E9���l��),�����i�L1]��p�QM�}���Y��Q��l���������(:�u�"(�����@�nF���Z���\����K��)�T2r���%A��5G	����q���|��j,��5�,�QB��C������-����MHII4�����u;��`f��<��� �%LuGQ��'%E1�-��!iP����D&��OB�'��pj��P��/P�8k�wY�g���w�)v��~�Y��`���}n������i�>���-i�������n�����^�l��-l�,��.,R��Z��b[B��c���l�`�l�=zJ<�x�qmF�E����d����(������thV\W������7\%�0���9�v]��r!O3���sP},�k��C"�YF/�6�����p�`<��>��0����2{�W�����(�;k���.�e8Q.����T���jZ�� ���^W�MF��xI���V�J�9�qKx�<j��su���)��E[DG�(r�"�p��J��V��A���8�b��<�aS�fm?�CB2q�o{���uO��'�z�����)�[ma@�G<b'��O$����f�q<
�HG$`
J���Y��+s������[i�o�#
\dt�1@z�����z�F����,���SJI���|bh��=G�u��dK����]uk:�8���������0D�����=�q��!�������h���?r��c��������9������?<�9�<��������p;��	�����Nw��pmr�6��Z���k?���<���!��������F���������x�~��6��9��Gb��3f���i���p)�a���O8M��F���j�����E�r�F�2P�2�,���vv+ �\���~������;�d��1������N�W�	���Z��$$���T���R>g�Kj,����jBL�����I��U Jc�QX����GF!z/o<~F����������Va������J�2A&�����m�
�F�y���s��v)��V64��ad�L=�R��O�6"�1������q��#�v��
Y���_�����O�'\��f�����X���,�{\E����w#E$�#>�7��C��!�1���y�U���Ew,d$Y�o��d����ptY��q4@7�l^C����2���.ju�d������BT�8"��3�,Z����m�<^��rz�Y�b���+�%�`�U����"h/�w2�[�2P�	��P���|��L�FHVl��6@&��`3�]�pb�O�`����q��Z`?q|��8>�O�|L|���'��9���G��e��:6�Y���!���dB�^�����MB3��������:�`b������mS�Ahw������_bE u����;6��\�?�K�������Gu)��k_zsg�������(���}x��>b�JF���Z�FY?:l+���p�_�g�]��������U���VF�U�#���+�d�������w�s��K��x�R[v\��^F��M�s�[�w��9��P���d���Fbvp��wU-��X�;��e��_3��/�K����|<^/)�\h8MYbk]?��#<sn�L��X�9�G0��s��H;�����8�K�j�w��6.���H���h���]�����Icu�R/'�u�B��wZ_\���#N#�+���s:���-]�C0�oT��
}VKr@)��s`����
g��x�
h�������p��1�N�J}���2��&b���"�WR&����;!W�P�3�\c�a�e��zk�����9X �]VS\s�,n`�$�1���\P����e�+"���)�����*w�(��\m��,abR��������,�5��is!���,�F�mE�<�S����l^<F����:�E��;w�����K�5���7-�}!~5^�#�����{b�t��q6U�S������$q��rX���c�^��a9u6���dt�>��m�����Lc��"4�q=�����p���]����/1�9�:,������W2��\��-B���:��+���	S�a�����-��C���iJ�5����V���,
��!�]�G�j4 *���Xh�2b�Z��I��h)+���A��%g�uK#�&sw���m�y�t��q.w�`��6{��q��*�����J�{Lz�	���t�/���$A@��������v0%��/GJ�����zQ��}�I��f'b�<�CC��Q�;�R��l�vq�e{���������I2���>e��-JO�*�j8�v%�l���d�4�g�
�g��A�5�F�I�a�����oa��`i�J)�-���4�J��'�f�0n'7Y�x9�kf�,�+:�Dk��&/�����
����*����xxH!U�	�Au4��,�"�5���+:�4��n�����p�0��!zi�X�r���4)��j�DA���x����OI��
B���;���.��$n�f�q�h���V�P_�ZN�pD^������\B�[���}Z������eN�!��������I0z����a^�$�{y��'���4���
�o��=o��m#4�k�����$p�K#���ETG���
5�>SO��z8� ��
�Yvet4n���5�X���F��[�\�X$�'�@fF(B�������+���HLO�=���'I��4(���4��������d&�G���$�9�)�W6��!�n� �L����s�����m�@��s��
��i�
��u�����x���u���'���~��2����L�[Xp�1�B{���	��w��f���Ol���>��7b��F�c�T���:`���w��C^�&���$
*��+R��l������|�$��_'I�����c��6,O6�X&%���I�:��v�Nm��D����	��;��$k�y�w,�����N���}��\C�R��$1�R�����nnT>n��f��F*�i��D����#�r��)��t S��<���)n��'}�^f�2���O�J����!���2gH�M�K���x�	�4�Vx�qn���XFZ�;F	�a<$�� 
�Q�����blYl��&xB`�-���G>�>��L6'/s%�m�K�L�k�~�?�H���n
�������
"|�3Y�\'��g���8�B� M�p�	�k@Nll������L_mR#a�|A:���L��?�*'��H��������Z����2��0.o1���:I����JI���|�'4%���0���!;%y�b-���2w�����r-��Nr���y0�r��s�����mx0��gh�u0������H&��	R����FI6��p���4��eh�N�Q��)��n�?	wq���OG��WM%�;��L�&M����N�����2��(�����j�YN����HD��b��u�H�2,�S�l1]�[���<!�V3��+��D
�M!4E�L��A'j�w�:����=B�!��<jV�;6��e������R�bh!9����
�����O�Gx��Rtn�����R?����l�]��^�U��u�n.���7+�
C8M�������������HeS2��j������-�Dj�I���+EbFJ>,��	N�/U�f�/�E����n�B��������ft��Z��uG(�,�)
tv���{��:�2���9���O�wKP�G��k���z�1#���~�4��m%4���R�[Jp����q�����YRO(�A����x��:5
	Iqr���L����n�*�SX
(�����~�Q����J9M�e/[��
��\#���$�OC��dD�Q:�TdHI�f&��}���:���q�#������j�����.�@���s���R!M�	�|9>v���!�����]��(gu��\�zj�eD	��z��J����JM�$c0]�h��i@�� ,md���\����&��we#������i�z�'��}P��%���(V��Cc-!� �~6����2�,K�N����k])�	q�o�E-�R\�SM9��-6�U�g�5���Z��nzb��e�t���0��5��>��<h[�[ �L��*�I��T(s��\���\��-�B�u?�"�����)=��~4�0��<��x8 ��/�����ub��*>��+�`�2������ek%�XP,��Z$���7���F���#���]��L�#�c�H��5�jc�������OE��Pj�'V�4����1#]��Q�@������� {��j��o�Q�z������
��G�3�NO�"z�h�8zsmd+���J���%��jP ��K�i��)e�=��G���t,���@
;�P�S���.��9�S����V�0z� ������<��<@c�,���i-BHrB]J������jj�)s�X-��=�h��w�����Jr`vG[�)�'Y)�������>��L��I�����K:�J�^�����{�^�	u�m�	�"�����r_�_�|����������3�Ql�*o�m�Y�S|��R� �CB>L{��\��0(;�pp��-��K�m�:�MH *�����R!M�����}�r�����@4��&�B�V�m��0`<���\;(\�i.Ua���m������P=`a��s.f���K�S5��2~��7�������{��w1�&���s�g���<�����;_*2�p)�FZ�����UA`���ku�K��������;����c�S�X���<�Qf����(�d��%L����������	a�r�R��'����_~�^��SI���g�+��wV??���+Vy���s�/e���y���\����KZ�3�.��htr)
2�(O.���:8i|�������-�~L��mJ��GaG5�s����$�k��{.�qF��\�a�L�m��V�2c�pa�p�:��ct�t}
�e���Jp��1���pm���EG�����fe:1�n�
� 6J�Q�h����9	EVk],zw3W��DjBo$��1����i�iN�-#�h\
��mb��B�0�T�AZ�nERk�����x��f
e�"�q��):�WO6�)K�Ms�cl��<���t,M�D#�[@�X'����$-���#H���Tg������:�{m�z�!v��e�M;�����K8)!���F����b�8�l!�n)����,�*I�e#����;Aq��z��Q;��{l�2r5�3���T�\���P��h�{����#]�?nF|q��������:���w�Pw�����wC�����E��!r���P�����80Ob���\&��NM�t Lp+�F�v&O�FRKx6�k�U�����d�Q�"#u9c�{�����5�?z��].E����w9��9��\��>�����V����@�]U������k�1\�=�e!CM�|�u*��S�A�j�le���t[������~Di�3�C�|���0p���-��d��O�J����c�cd~������-2�QJ�\�T:%�RF@X�6�l�q{��%���O0�"�l�%<:����=H@�>�y��p���Y������pj2}>��m���

[��	���6��ue�S��LN��e���=���tz��w�c�s���������G�_��|�-3����!����������m��a9/q���
�7�7�>l�������f��������|;CY�qjN�fq7��Z��a�����K�	vs�W��|F��X��A�f_��T�����K��eA>Y�A>���Y��H�H�����a�������y\��f���C���qi���05$a�?~����vG$����w[�3$���i����������G(����,�Ox�"i\N
�G�bi>zF���Y�V��a������_�
��N�e���x=w��f�IG��OT������dFG.����/97�k\7�L�x�Q����P��l��m��/!����<��D:�
�V��L.��K��G��s��J�I��:���h�]q;��'��,����,%�A�$	"��BnuH��kkt7~z�%@�fb�����df*6��d�9���SSK�O�T���`j���5��Z��������NH�P�����xU�,=���XX���rN@I���5��2�\����8����m����	���9���
��+�1"S��*�I�(&���p�R�Y �L�J�uP�#�(?�r~���0�&���w$����*r
mAJ����@(mE�?����C��;����?�U���&X���K�?���h�g�E�{QP`�:�?����s�A���g�����9�pD�#'��(�o��}�(/#�Ps�{����ys���T�QV��q����rX�UK]��{��|\8$y_7�>`g���b�>#���G$��K������|�#B8��i�_����):"��/��)|[�]v>r$���K�,�7W�{Q���(?�h����|K>i�9����|rN�=��d���9R��������_�������lY���3%x�",���/���&<����m�/�'Y�:�n�G�T�2�7�e���	e���Y`�W�,�g��������������[6H�3�y�k3��eu������&f'�A��}r�=�@������_��k<s�Lm���'\n����j�_P^�-��5�#�t���2�>O���g�3$|���,uGD�Y�n�K"�<(J��1|���t����[j���Z�W3Y{p��+��:5x���M������4<%b�c�a�]�g�R����q������@�!��R=��X}�����be$�S�^�i>��8����V>�u>�^�fw���4�C���u��w�������t�������0��C�U���:�]��(�}d���R,����P��}���Z�Oy���SC��+.�8����kH#���&�Ai�9��<<�=���8�sx:���n���>��U��c��r�'���\X���`�9�xl�e^��������=�$�k�`<=r����j���*F��o�cU~VW�{���������>�9��L(�� ?�v>-�������"x#�������;x'\a`Cj=�xws���.���:��[�]e������Q>�'�^?=�w+�nu�v�o~^��)nR}%������'��N�
��� �o;k��|���x5E�I��dMH����I��{���)���tK=P�U���~b��F|����Y��'��9r�o�i���9Mv��W*b�Sn���/b���N7{1E.���&*oT�c�MV�jCuW�bG�K-��W,�R�N�����"���
���'bY�c�.���d�I��Y����m�C���m$L�����7S<V��/ia���3�Z�
h^5)��NQ��P���z}����U~T��q!>�%�����'��������-hI��{��E�p�q���=l���A�N�����-W������W5�L1.���n���'KmP�6��jP�7����*�d�����y�u?^��%v��ceM�K��4PH"B�%F��ng�@sD�����z/�BA����}���]"D�Pl���������e�9s~-7�bNf���W�'��t�~����M�TyoLe���7���0=������jY(4���O�F�$����D�
�������Jy������0yMd@!7�w���1������__
���wp�������v�A�����6��~�s����_i�h��<'F�S��x�j�}l73��@�n����fH�~W������<EC�(�Z��t����	{P���<:��H
T��o��i������i������>7�3��%�>�v(�������]Y��G���>e���(��%2��j�t=�pF~��{�x��X)�MZ9����
g����y���R�_F.:�T���i$>`,�Q8:��-�J��'n������}hSfD���c��B]T}�.AYM��6���k�����N����j�-���8��m����|4��Umu���,R��
'��.K%���6�R���������W2j���m1�&
��	4�>d}�-���U����[�D���(��fI�PT��|�Y�d����nj�Ly�^!�g
��y��X�F����#7����^��e�Y�/�e>_2;���������E�����ux���*�`����i��v���.������Tk�Co(�k=����C"��8 �S���<mh����"��~.d�q���?�2'm��}'L�����^1}4k� O�4�	v%G��J��,���z��uU���~�w�����bW"r#�v�T�J����"@b�+�X�j�����1O�a,�R-je�g�U��=jh>�:����a
x����W�k:\��J2ErI���z�u9�E�5I�OF�`yV�5<,�t��a��0jK"q
rB�QR.WUY{G<i�����U�rmB!����+�K�U�36K�e-�<;S/�����" ���*�����S�n�wj5E�%�_,��A���h�n��g���5���C5�Y�!�X�C�_��15��]xX�g�4<�.��Q6��4��?ys���B�������\��W�k�T����<`M�N7D_u�.�Ku
��V�5����A������(���Jj����g��������ptu��3���s�R�zE�>�����~�b�����,{V.aEW���^L-�/fiWs]<R�12]���"xI��r���$���R��p|�k�7��|���*z5�(����j$�����z��D}
FKH��m�*]�/M��F
���GeY��r��S�u�/p��'��N71~�
K�����T���V�����3��H.:/^�i����S*�;��������J�S��	�������s"�2�#r�����H��C�>�A=������6d�3E7���g�@��`�).�_J�)�Z�b���(�"�Lg���_�v�������m������Pl����T����_=��k'�
�����O%����s�dK���Sf�H���D �zm���nY���1���g2i�����]�k�I�>�1��]?�y�� �J�~mH��:���&������ �2�+z}VkY�nLA�s�?���?�"��HV5��j��"��*����YH�J<����.�A*�� ���<R��\"
�g���J�~8���I]���T5�;��	YK���wK�7�c&���)�KI����me�q���U&��#'���G$���hq��
�^��L*���T.����[����K^��%��Q/����8��@q��z��F��8T6�<������=b�Q����]�����-��6�:��]B�����~@p�.��������B��lGi��]��9����=�$B�'N�x��&B��Yv�����K������G�EP=r��Z���KS�R����s	M����
���P��$�D��H��O���'�\F�0����
ofO�N8��I��"�if{e����}���[�A]��S���o�"���_{WjD9��q�u�w���8l�Vw1�8��g����6?���/��i&v��M�z����iv�<k��m����h�|@����QsN�.8pv��S8(;��B������h��h
=�������a��"�-E�����E��/sW���H0�����/����V�w�0����z��E�-;e��'6���v�(�6-�o����������,�'����F,�;&��k�1� ����'��|o��~�XI�vO6f���b��.��OC[&B����IS��RP���R�u���G�{�C���!�����,��s���cX���bZ��W7GvpR���������8D������>��Kw��_}wv�}�i����z"0��x�Z�{�x�u������qW�,�-I��]�T���^����RR�[%��'5G�l��2���
&�s����R�D��<���Y�_l1G�]m1�]�mu\B��b�Z���w��k�c[���
d���XF�
�TE
ux���1����{ND_�x:�U���1@�
|6��h����bH�Z�5��A}��
�#u��\����qf:�4h{-���/@�)�Y��b7�Gl+���_�/9gc~uE���7.��j~_�=�83���C�e�Ql���|9�.Oi w�D� I�$fp)q5c�~2����>���n�
���g�6s��������G���(3'�QfT7}x��/��I#����"�x�
�������8�-�>8����O	�j�j��tJ�����c*S����2�����wBr��9";�2�~xe����iK�v"�1�`��BA(�7B�*u�nMl�x���A�Q=�@�Q�d����,�"��.�g`}rrbPH
s��~	N�� \b���Y�[�B^xC�1A�'H�3�g?v��	D�
f@��Y�
�����Q?��V�5������H�!�����P���@��"��t�Sc�z�!����&k73���Xf�
���m�)���v�F_;o;��h��$���v]���v}
���?�.����S������q�A
v�2���$�D�P>�!�w%4}s?[@�Q"?[���~6
D��tc��>� �����@��y�]o�s�i_1����j�q�����TC�cA��R������ein-��z��9ki������
���?��������_�?���
9h6:����)�����G�����i����D
$Hy�����V��=IAH�k���s�b`
��T�D���oHPk��8)����>]ok$#��/��
����;��� c (V��0�����[��\�L�f�my���ll��������d��!g���K�#JM�X3���*�;#����Qmm�M�=�1V���i�f�=
;���)�=�������p�:������&����e��-,��U;�[haz�!�pka/�S�l��C��f����e��������������v�����=4�����j(3>�#a9S?�GVA�Z���H����+h��XA}aS�2<��r���V�By>W(���C��
M>Ws�
���S�B?�Q������'��~��[G}<B9�p�,�/��bv]�_N��V�{�s+��qc��9
��,�=uB{H{�8��\q}@"XA�X�U�X������Aez
]���|C������� �0��y���A�v>����O�.H1%A�$J����P��Rt����" ����������tn�����������)����#f��x,^�5��`Y��',s�(:���pcf�
��F`�'� �)��|�y�Q,���D(a���,������i�YX#�����fA��5 �H��H�J<R;<nY,�2���xN�j6��w��U�,���W/`1��1��B5q���MA@��_�������3�~Z����^�1Yd����3���8f�L��3|���A�����4�
U�9ZF1�`�WJ��(�%�����a,2Id�H�g|j��c�J/������0��h��l����4�����J,�����u;����D�����iv�p���,�JF������*"�&�o���S=�o����}�a��ed��x������i���7&2�J�J}�a�|��+�}�{9�9���r���J��azh�t������T��S�{��P_Z�����n�����W�d�Ni�_��)-�a��7�r�G��t�=�9"[m_�G��m������������J��>��;������������;A�bW.����Y�zB�P��n�����<�7����^�G@z�U�v���h4@����	$�R}:�N��F�����Q�E�v�WP��Q�8�:�Wm�����P7H��s�PG�8=�-��PG���L;�Q��<���F
u�_K�iKM��
��,�:
��?9�B�2�#��o������?.���
F_[���^�Jp�`o�Ux�`o���)
d�rQ �k���h7�H��(�����|���'qAH�����@MQ��oh�ad�t�M:��D��N;�|5:�J\O�|:
�>��,�S�6�g{����{��Q4�_��"�y6��=�Y>)�K�/x����no}����x3
(�T�H��R�43J������aT���~���-~��cU�H��qn��)k_��=��c7��J�H����P��3h�,JANBk=V��l����V�����.�S'�n�T���$4�[�����dl.��>G��m�z��qF}n0�.{�:b��;u$���������,�n�w�G��+�#�����-�3������7���H�gj0C����������/�GR�Q7JC�>��0T��R����k�2����\!��ej���Z��P�a�C}5��\�,K�P�z�����1�����)U�[�������h��%���r
D#Xdo��C
���(	�(�PF�*��'�����>}1��+��,��4���D�2R��Mh�8K��&(c_�V:&���i�P�(�jE�� Sd2R���4�g��e=���������L���
m�����!��,~���[����i�������e�R���(o�P6��U��Gz��^��1J���_iSC�p}E�l��klf:�46����jl�Hc���hl�Hv*��J]�:T���
4�Zv�X9���I�T4�N�p�����&v��c���]@c<������h;�h<���	�_!�1�����6h;f��F�1�~�����c�f(@�1]m��)�h�}YDcH#��E4���r���T������Z3�BFK\�!D�(�4t��-KpA7���5d�8�/��g�w���^�x�Hg�[��CY7eV�8��u���V���m��8�N����go~~����~������_��o��4�0=|���Z���Su����1�cpn0�{��+��5���������B'X��`���A��N��z��������~���?�u�_C��
�o_j�% \Fw2\F���X�G�T*�Q�2��o]/��K�%7e��bIMm��������-��C��1P(%OEX-��V_���B��p!e-���Q�a���t��l>sS�0���������c���U�M���k9t�;���fp����?d2c�k5
��Ge��f�Ae�U���W4���)��cb��7�:�r���N�3����A�G��iI���;7C�vf��v��/��(:m,n��k-|�S1r^���G���V�bY���p�,r_�H�?j��j��&���g�s�d��gp�����k���0����|>����M�l��^&#�����.�3ds
��[.�Q���ro����*�5�}����2��K(a��R�e���e�RK'�w�G��-�����|o���fa��vz��Lh��b�q�L�j��$��f��b�A)���	�&��i�����3���������e1Z!]E	*'�����){\RvP��j������>��-M: W���	��Y�:�,�;���u���|t���in��&T�����#��7�.�rc�R�=�S��G+D����dq.E�KF���/E<
@�S}�{�l<]O����H�V��.�bb�~5�=)V���'#ibx2�@��\����e>�������	R�YY�9lF�\��J$�+�)���}�Q��jJ'���b]���Z��ZW������WY� ��z1�_G���e��S�$���/*qd�J�K���2{h)���J����V�U�5���[���^�+���TWKF��d���|Y�nnM�0�%�j��m��Z�J�T&I���T����A��\��X��UL�]�����+h-������nKx���e5=R���h��y�)�}�:,u�� ��.��=��A�_�����Y�lr3��|9Q�)���U��r,;\J����P��/J��.��M�����!�&#��{��	��J�������(�iJR����.�W����cB��d����/��������;i\���kg8�Z�"������c6�s�R
��
3�a!��r \f��>�.�r`��P���j`�����CM���=>������H�e��*�L����GWQ����������g0�n�Q��u���D�N ��W4��R�q���>~��F���V%f�t���,QA�p���y����$��O�������7)�
u�h�M�����	P7<�n����[F�����l>�L_�����-����IC�f��q��]Y|��r);r���b���=$��|��r9����QX��{S������]E��J����wS���L�;sU�n��j
h��]~�)�����L������~��x���@����y�4	ZF�����M,�v9PF�3�g���#�f&Fx�uM7�)�g�|�ag5Du~y�*��� ��f����\_��T]����0�a�[��s5���)��Q��������6���&�%M��ju��z����
B����;�1G%SsQ�r(���w�T:X3��Q�
��w�����[W
]���� ^=�����������~��E����Z-��/(Q�VF�n���d�cB����mg�����x4	�� ��K#��J���
�Z������/-��"�z�����Z��{�b���]y?@���v������:���V>��k0�>���$�J��8{T/��9M��Mfg2��S����_�=�Y��H�fI�~7K�QFm��>@�O����?���0���+�8f~��[C__�V��������P�n�}YS�]F��15��pozp��j������i-r
\w����t����O�p���l�������!�l�)��6�x�����'���Z-�>vz�uE�Q}�g��������2���(�/�C�WNzj��&6W�V����4I���IW�]1[���c��������BWR0��c<(2�,�|���-"��xZ�c�����cYW��[���S/���F���^��X�v&�#��������G(`�Q_�!��^�3�n�l�IB�d�{����h�����

C8]$MvM!���!���P�����rl����L�������}x����?�'�������O[/���!O
��}�	qL-�>�b��4)�-��&9ms���A���g.m��#�CZs��=+�Y�%\�+%z[L��5^P�2�',[�l7��=���Tl�������\����_)��H\���`�m�������M�����F��+�@\oj{����X#b�P�I���<�������}��{�R�W�p]�do����Xt\L�~�[���Q��3�_����Qg�eg>`KD�o��������(�^�������[�8L���+>���|��7���32�����$�z���7�������������_-Q�NC*�?L�H����j����P����[5�5�e���R��_����\�z8F`�|C������!�IG��7w��
�z4#jt+]u��W�'�/e�a��2�Y��E����y!���, E� E��e�N0rC�
[����h�_i��n 97~{"=��t�s��H
�_�3z����&?K�/A,�Ed�|��	�h���5$����Q��Z��^�z�K��������z��=7�E��N��\��F��5��l5>��TFN3�B�:����^������'�#��3u~d����L��*��U�/��Zy7��
<V�����_�j�����wN^$X3�CD(������D	du_f������r^������"4Eu>I$1�t~^�}�����}�����JA��{���q��BN��p=��Lf�=
"P�@ ���5�k(K�M���z���F�g�5D���5���\�+�D>4��D#i�K2�8���+�>�10��=�xc
��(�2zl�2^����N�7��3<>�*�1�~��Y
3U��c���y�������6�P�Q���1m]B9F�����:��<�d	��	5���+t����9�R��F��a�K��g6|�I��~W��\�4L������T��}��A��e��4=6��u�,�����N�H(�G�0�4�bv�
l���w�F�8���~F�o�s�J��I#����M*4������S�(D�EJ�Z-���>V��#c)�IZ �8�qE�#��U�d�WL������r~���LX�[�.lL�[��gn�wft_�:�A�������l~1�nM&���IN)Y����F �����X���U�]�������|����#;7�4N*�)���H����/���W^*�J�� �>F�A�q��7�\!��3h��zY��7�=0����M,���Q�d~�4�FY�qj��h����208D�B��R��::������p����"]\*h����k��
�U��o����@�)��*���%�G��l\�����F��&�4��uK,Q� �:��a�������{��U�VH-nPs�����SRJ��hF�j�4|$M2	"b�T��t�?@I
���m�$���I����j�4�M+:�m���@w�
h���4�Eg�UN�}������5)��s��*�<u�g�O���/B��P����m�U[��a^�4�zw��y�6���kx��{"���w�����{���-��S]�M�LhN�d��i�1Jt��;��P�uH7�
�f���4����B��%v�������o�9c���D�IS
N�K�<���d�������t ��]js:�"s���8�8�73����)Okwj��2�:3��i~����|�|��'I�HP�]��l�\��l���c�Jpu���H
"�����XB�iCF����d�;�H�����B��U1�}'�1�V�~T�p���:��RE*�����a7M����v�0�,2�!'��<�j�P��n
���]Y�4%�F	W!�BlP�u���2���4[�&w�S~��+�����a��v��O;tZ���4�7liu�E���e��f1?gQ��0��+R.T����D���Ts���Ws�CT�Vm��i�7�61�*~�*#'dW�jd	���,����Z<�_
|\����Rx'���8iv��}�wj�
#�����A�Y�[w������Pw���T������7%,�l������Z�~�"�8=h%~���qY^M�����?��	��ksN�^��! f��9�+6y'td����Dz.�}o��r�����psbo_��S���g��V�U9c���t"��B���])���oQ���|��,�7p�tcj�Gu�f��D�v�u{EH���
w���y���
5���TEa��C�����[���|��Y��M�����r�5p��
k!����w����k�8z�NuQ�F�}��j5���^�,��z�e���w,[�:
u���1P��d�t���,����U#uriv+@%����!�
�_�����7�x��K�x�����X��.QI�A��c�CX(M�������9�8g��*J�C�x��:S{&)�Y��E�V�g�Y�-�����r3|�r����=t��(W4�~�e�{�>r�U�� p;LoH�pC�����
����&�8���l���������`���d$4!���$���5H"Hq-S�����;��\_;/�	��XSC��jU��.��������S���r�%W�d��8jZq,��������Z������>2�d����
F)�1��ly�Lx��VB�s�����,�j}4Y5�z�|}�7������������m!	�*y��#��f�o�Z�e���]PK��#�k*.8����s!�y�V�S�q�jF����Ofq}�q:���)�����a���e���v}����imS�����(��kR
'���]���Y���o��8<��}��
=��
����������������a7BA��f���BL�'����b���'*�����9u����k���j�]OAO���k�g�Q�-q�t9�-,��o>������ �2~.o?������?�F���<�:joSo������jxO��^$'�$�\V���R�%���PPmVc��2Z�W�g�0G����d}fB�u2zl�X���rH����5��r;����ZF��2����6���A������_}2]����!��B#�������
iCj}�kKx%��V�[c)E�dY)�g-i4WB���}���9{1�.L������C�������8�m621�Cj�����{�HDi�U����������\W�T���e��Uc}�V�Q��[�F
���{���
W"���>JIg7�
27�R��c����0s����c�(9M3G�6��{~�t����n9�y|��h��`ja�aw7����-#��bk<�@� ^����%����e(9��������0Q���:A�1"%O���t�����f�8TnA�+���u��R�� �l2�Rlk�T��NW2����-������[�Ff�����������8�������n�	v�c�-�!�.������e�����A���#EUKh]���o����s������x���F^���n�d{���d$���p"*ne{���6R5��`��#c_|�@��S���(�jj���c`���.���k��p��/�R�'���`x}mF���$to��������z��m��v��uQ�G}�����
��g'��\)��g�@�t�q�;�]_yw�E�~����,�Ah��K��-:����9)z�1�n���NY���p9��2%-c��R�o��������WRR�r/tL�tfp;�����i;S��c�>SQ���KK$� r�����}=��R�c:Ns������Wlfh���*�C�H�����*E�#��n�O���h��}�v.��<V��=���
�U>�����L���(�"����^����/�!�
.��W���)��|�M������r����.v��=�V�@8g��`l�}����j�r0�{����2m��,�y
}�9gZ���N���;��6�����U�ND��ksr��]�'����}���|��\1N����R��s�1OG�4M(�������T��M����ll�4����K��?��G���P��
��Y����.�\	?2��%Q���i��/��{���N�o/�c�w}���Te���=�?���(�O�<��L�e�36N3ojY�*���8X�\3j�+2%���M$<V<m-
K���a�c�v|ls����FYY��\W��U:^5#�3C�l�#���tUA&�Gv�^bPd����B���xj�����M�������&k������Ga4��3�7�;==;��q����la�*��{����V�+�)v/#��n�c-�:�!�ffv���/��t �����2gs72�.)fg��-�(d�N������O�	ROU�������k�2
wo_�!kTKp�����{��5e
]@Z8r?��cY�aN���a�O9�!���Z�d��Xp��y�Tk�
]��K���\]�����/�4:��)i�����y.J�F#��Iu
����F��G�4G�0��	������x��)�p$��s��������wt���=��/�WH1}�o�l<4�5[�U���f?V��l��I7���Q�U[���F���b8��Y�k��C�����>��2z�{6��M-����y,^]N	�Z���
����|��t}�ct���&������L�}����i�p�YG�^�m)��P������&���'�Li��a�/d��n0�
�`	p[Y�F�G9`���6;L�Q0F?������'!bV^�r�.�^��S�n��f*pI���z�%�����l��@^�92���L�V�a���w�`�q�9�
�����6�V��k><j(��w--w�5���{����A
�!�v�h{���Y���Yb�Pw�TCQ��j'LtP-]�q�G{�:xdf<S������:�����������$k���HMq�(������jS����T��;P���/<�J&7�wt����@fqS
����o����/����!�`9���M{��Lu#d|����j��boY�B���RW)�����62��(���jm�W���`�_q��l��'�q7�%�Dlh��R�e�=��_L��3�?�'f�z�O�v#����"{�j'����?#
���/v������~Y���6+������y|K��y5�]�i����t� �Y*{0��a�d|�5��8k3P�YmC�����1J�����e��83jz�4�<��9��(T��2b\�X)`�NtD�CM�S&y���j�(HH���s�3w�*t�-u�VP���lJ�v2E0+-�KE��S��R	NDC���b��Y�h�d�L�e��v��o���
{O9����I�Ph3�z�hk�K�IF����1��X�����W7�����5
�
j� R��EB���QMV�r�����U��U����/� 8o����V�Z����5���D���F�M�Vo��_~;!^�:��	`����?-�V�j{���02��|B�_>�����7g��i)���YE����d4�(>�/s}%�Q��K�WL�	�e�����3��E1.l���j�R��RoC��5Q��M��62�T,Jm���K$\#��~��1V76du�(-h4���C����(�`w
�����~22'2���W��=�����g���M ;��d����T���*��D��j���Q��h$b]0K{u�8��f�[%��MU���j�PJ�nN���`��E������qF���u��L����q�q[+��o	}��6�W���L-���j�]q�7��^4l�}4I�}����B]�`5���2 �j./=�y�9@�����U"��lu#A#q_�[1�$��S��	le���������af�}��6sOG|}��\�O�<}`[��D[:q�*���^dN�x�F{����X�d���i��]&5B�:������So<-�%����(��������u'��z�Z"r:���DT%��D�h�-)��v/]C����D���2w��%y�,��[�*>�Mj2R����?�����w���_a�q�d����qd)�M��&�N�C#�Il}�����w>oa�|� ��\mtu��"�z�G����+���du�m_�����[_Kr��Y�{.C���xT��j��
if�����hJH���iO��?;'�O��$��x:�Y��u)���@���Z�H�@����&H#�k�*���W�P���		�z3��,���@��p%�	����+v�h�J��� ��Vmu��h������������2��CSE��dOu��28�]�hatk����e�e5`uc����E���������6k��~w����Uo�
�����uo�t7_&}����2������:S��+8V��J�����Pz��n������Yw~�WO3���u�y�l����inC�� �z�$��o��m��E�UKY�������v8t^�HT�!@��Q�g56f���D�H�`����������[�=���5�O�Iv��'l��0D�s��*�����������jI��=��~�k���re��V�3R��QN,?fb����Y��	b����*jk,����K��� ��o3�G}�Piuo���E!��-��R�2����� E�*���[�6K�����b�w�d<y��Ob���G_�;d�,����^S�5X%���w��n9�v�5�t����~�e~0���3m�^�Q^����>}n����)��L�^�-F�-�^��������a1�>rmB	R���X��A��<#��Bq"���r�B�
*LLa:�&�(��R�nm���S�������ED0�e��'���A���Y1�����������O����@jd#R)���u�w{d������U�o��7.��H}-T�8���X~�!���G]��c�E����p�V!qk�w���E�,n/�~�,A�0�xu�{�[{T?,����G�c�$!���W�H9[
p���������	0g���NC��<L�7/�����n����|�:�H�������m(�-k�)XEM�s��?=u�z,�>��'����OI�O�������nO���\���z�M;��D��D��Yp`x���1���-Ql�U)1$�6N�-]��Jt������a�@f��t��ru3��%��Z�3��m��G]��������<��R��k������D�1�r�}�����d�
S���zA��GO����6�|G����q���-���B�#�q�A�B�]�dcZ�����l�
(wp�=^�����E���������=>���C��?��K�et������K����n�v�>�����/��k�r����h���l�np�?��[�������.$PwC�G�G;�GR*�b��9$�p!���(�e�4+���;"0�+��B��W9G������� ���WGD��������E9>�d�:6W���S�t�+���x	�����/���{(j1S�����
�I���X,�V�����&$�jz��.�d��i/!��Y4�.�o�y,�)�B�M2 �l���RR�EHbX��bM���r�0"���>2�$�:�G�?ju�2��|Muz���!���a�q?rnr������B�'?��m�K\�l|N��S���O����u��[r5G%�����t�^yG[f������2�:e�����B��K����dsY���)>���K*Yr�0t�w5���{��Ox<EV� ��L'@����"\��������'��	�t�����Y�-g����`����	e�$A4�oY�>q�L�)�3��R��1tP�.��r���g�x���G�=s��Y���e�Yr�]V�F��3��xVe��{6Ys&MA@t�5_�:/B�
x��9iW�- ����������'j����8�P�9E��8B�����v*R��YF��]����[�p�w�4�GWF*ieP�?�z��b�����YA3�H�x�d��k�����D��Yc�j2�SZa[`82Z�{�	j���.hQ���^�a�&�����t	�U�	~��*��E�����	��P�v�co��N��t��V������Q�Lq��\�`2Z��e��.!=K���H�3�)+	��e�bM���V������6����7��=��P�K�l��5��|������ue�RMi��M9�!���-`�)�(��i*F�����G�v�5����l<A1o*��_�X<S�{N�_����w�jE�xq��]����	�H���LD+������[�U�R���(�
�u������5L���]q��<���<�Ab�_B������a���#\�l����Ut����Jd�f=UT��l���~8.k��e��k��p-)�/Z_dEK�Jn�j��+bk_2��[�$�����L�����Vp��-������D���J�����
����`����2W�#x�i�]�X����\l��uuugy��2Vj�
���B�9�TYN����%�6)��h�qm�j��O��+T�!;���F$;����PQS�������\�5�-�,�K���/f������j2sUH2�����=�#�|����8�"WZ�Q��0$7�w[(��[�%���JY���L�1o�6��p}��I�?_I���	��uf�fN 3��v��t:�<�=������.���~4)%}�k�"P���y�s�qPF�j��I�j
�4]��6����:��/*i�U��3]]]Uc6�J��o��_�Q����2J��C�����XW�U�����9IhBa6���b����:u_d������*/hc)-��]�<t����T���Q��;>�v�2JF���������`+r�B���kX6|�d��/#7M�rJ��V�D�����Yu�ax��� ���6:zlF�Ew�����W ����1{�~����}4V�_����P����1�l}�X�fc@G�uv;L����E�OkN��l��h#)a�K���E������G��GQ�q���^N�e!A
�?�>?�<i����0�#s�!��F��
��X�>�7�i�x��n�S�	�
S
����*��~~%(�e(o!E����@3�Z�=g�o5@������|�����)�@$L��4�+�$���4,�[�`'� �P�e�����]�"�-�K���������I�{�R�):����j�cH���@��40Y�X;�YX ���������'�W��J���t3!�������q+8���8�nY0�#�����$_xCb��Px�CYM����4lEE�Q��*I�8�������"\���i���nV�W'?�&��6�oW��u���
�w��q��O�!���!���V��Ct��<�0�Y&�0����h���T�%�#J��[��F�0�/��o�PI1��
�2���*������0��!��� �r��1*�x��1*����Q��z�}6��#A��
������^�e����J"=�:Et��L;���B���
���,-{Abf%��6A 3�%�H��j0����s������ M17 �{d�*��W����P����B"���P��7��)�Z�M9�������S�:�F3�"&������;�k��������D�tR�a�0CJ�>4f�n�8�������3��X�7OD�&��*�b�B������a"����V������_t�u�����~6G@��W����Q�
��a�&G�~��������>��������I1�$��P�'W::�.�~��0�h�{"����n�m2a���@(�auF�:����v�lo8�0��C�'"��}�
�_u��������bf�#PTCx������Y�\���/�����o�I��p�Juof/����R���|��^*
K�]��=�b��d�����L�Op��P}O��������X�AA���B��y�-������<<8����`�w����3�p��!$�y|�E~��P����7ZxvFN�S��W
gw���z-����,��+%���!�N?�z.��SuZ�q����	�0�Pb�J���pcvmF�p�)s����&P[��,	��eGW�v�!�h��h�}i��/��.9����e��S+��y\t�����.H�$�b� ��`Z)]��aR]�P�QL�u�$K�DL��X0��6";��C�X�(����g�U�Vj����ie������V)#���1�����s���*����crs�����b�N4������mN�W�#�-���Q��R����n����=}��m��������p�!��U�����%�������z������+��u��xe�9�!G3N�.�h��a��j��x���������������a������I�e%L��#��N���{Q��+B��\�p�XT:���d�_NH���j���G�0�l��;���:�>m�������O��j�j����iK%H
�M]�����\��~�o��������3��y�(j��C�
G����i,ku\.���T~���6�8H��D������|1_\��b���U[��A�uO�F�o0��>��+o�U�t�6���6h*��d�����?� ��-�7�����������m#���g�dv�6��l�h�I9�g����ym'�sN�RA$$!�@ZVj�������@�9�T����F_V����=�o����-�r�G�����~������]W���cAQ����/���;GH�a��#��
��u�C(�@����,��RA^1���zz��_5R7��y�_���U{�n/�r3q�>���������m����������M��F#,J�=E�N������4(-���4"@���h��n�N4#��	�������nco�m
]���mO���g]�D����k�E���K��"���m��>�
(1�
�O5��#&�������m��l���a}��} ������n#+�z{/��/��8p!��{��K&f�������x���=;����'�;�z�3�X?����|y{g�������pn���AU�2��%�����nA��$�z�(�fsFP�+RP�+�QXR��zjIH{@��n/������3��%H�Kl�s�b�����>=ver�t��L�(���(%���z�Y�@��VQ~`�����q�2���i�X��gN	8z���g��|
�I�6z������e��4��s ������BCV�� 9�Grt���� =s�Zp7uX��
���.�3'���=�!�����22C���6�{�����~����H|�lY�W�#B��=J���e�@|�^�V����?�kG�*�;QbO���x�%�z�'���+//U�tbXE�["�L�]���H:�����)0���j�c�5}S|*[��@�<����B��l���	�,��h�����{���n<M���:M�)�0MCT�{=9����B�m��s��q��Q~�������73��(Y!N�����[�/8���4a��s�����?u�L7���c"
U�}���G&�>yl�����jSyX�#|%8s\+��~bf���e�RY#�N#�I~��4a��!�%%��B<{	��tZ�)��e�7�����_;��
VS�>�l�g�j���j�������������a=r�=����1���@�|���4��+�O�h��"p"�$�`���!�vP~���1BBKIo�rB���}2����Z������g�g�Vc��/L����0����� r����1|Lz����C"��������t�=h�:>y5���~��{��L�F�X�������X������{,�������a�1��0:�-�13���}G��1;��3�M�t���c<$�WbL2�!$�e���^��c��p��E�e��M��;��?��;�������3��a��@���4�x��`�c"a�=���������	�1����|��Hn����C>�>k���UM���^��T*=5��f+Ja�c��U�q��k���Y���R#�BfCox���m�� Z�>r8��5��~��Vq3I��^r�A��\�2e�"U'�>*��,�,���`W�c�fOf��+
0��]���fWV=��$NB��4�e?!�*�e�N|�`T�����f'$a�!l+�7>��hD[��q����O���{F��:�z����d�q*K���@���a�{Tes�F�a�����8���������Q�A�L��d_��
9� ���s�(���
��T-�e��@�����)�O���F\S5?�5e���+���4�^i1 ul�N��3��9Fw�����  ����a�y[����l�|����/�PV���Z�C�&h�u���^3�8�a��T�;�������bQp�R�
�0'�!���������f\:h�_���[��S�1�`�EK�����kN(�&}�Zb-��w[�6��3 �@���FBdn�����E�bm=��_��=������
�k�KD������(a������x���eA&2�D�P"��<��\6j�c��m������6/g �Z�
Qo�1P�ez���:����+��M�ai�+��}Y�$�8� y���D�^�6���	�[6���QD�L<�8�0�/�b�Q�Bq���&������ ��Z��":M�%�V�E�"Y�'�BubX�N��d:���7��KP&C�0$J�r�n��v��tqaO���"<��&����ZM}��]q�~s��8�D��0H��f0� ����ib� "�P�0�n�d�Q!�`"�NSG�%$��0Z����k�=���0����KS�5���c�a���5���p�px�Me �����w!	��������Z���O����@\�c��Z������X��=G������Ot�Z������v�Z����?N�������G|O���^$Y��� ��d�OQ��#��%��%>�����*�:�G��8n<��U���M��G���%1���>> ����r1<6	�L
��V��L8a0�o���+�?���o���g�v�I TM ���$c�Q">��M���R<l}�!�����m���7���
M8�TC������q�uu'��"�����4B|89(���F��C����:A�It�L!qxYvV��x��K����9�o����q��%!��']^w�!"���|����O��*��%���<(,[�g�e
�������s��u�o��99I$�~N �,s'��������-��HI��.�=���k(�.�`�U�`.< �*1S�i��
Fx�[�hiK���z!L�O"� �/1X.��D���y�pv��.JY�/���H�����%��Z;QTL����&���m�X�H�@�y8V��������V��5�}R��8!�+k^���z�p�vj/e�z	��e���'�)}���NBI`[Ow[w�i�:���v�����N������m�y�a���i����JdtI"��K���
�������7�.Bm-cr���I����E�1�UW�o�[�WubI�5f�@@�i���rS���B��u)�l%	����u%w �9�vS���s&�b�v���;�����v2���m�i��4����_vj���
?M�MI�N���S4�k�$��}��I�|t�4��g57��S*AnV=m�t��u�=~|p:B�Q- a� �����js�����8Q��5�����c�d�
�L��3�v��/�M��(HT��������Ku�H���
%i"�@)�"���l_��2�A��������t'u���^F)L�Q?����~���}�QYbd7��~������z����\�����
u�=�Z���d�>���B�:j=�\��H���-��ucq�nXV����i��v�����+K�@�d���M�;�f�����4_�Y��}��%�O-��6�5�K�d���X�a�<4��)�<TU*{�N�O{�]
�
����N�h��1�F]�[�A�\�&�@�lk�cdQN7PF���^�[n����{�p:`�����9��{@v���?��������4R���U�(W��&���V�d���ng$%�e>��reoh�?���So�>eE�`��J�8Y'�x7�t��"��)����j��^
������Z���s���G5���5f��M����yX<`Y�dz�~�jt�?�?8�DKwZN�?�Q�$��Kq2p�g�	����
D/�FNh�������f�j�v(C%���w�	 HNl�0��JH���rE&7�"PY�t\�j�T��3��v{�A _T�v��}���L��2��-�9�$W����bExH��@e�(��u��d����mpf�?�^��v?Zm%O1e�F���S;M3�����������������J�@����+���`�H���ad�n�3�2��t�����!�~��{��!ur`T`��"K���c 6,��I!�;P�y�d�wP}���c�GR,����d�6:�:}�tb�a���<b<����8���a�CXt��H�_S�>����t�@A�M	;T�@�����~����2p��#�7�������������4��	���s@�s��!?�����;���^C�d�J��@�
������x��>�$��-�Z+��(��6�\��;��!��^����.����TW������� �+
���W_�%}�~�������|�i�j��V/��}]u�	y���~7k�~%L�F��Y��VQ���T7w�&P�]"�7�e9����@�!}����h����Bv�]�.�M�� )�{�;,�1�N���6
���4��;Fg��5�%�b���1�M����!
���BVL��B��'im��+u�(u���w:�����N��B�9X�'$���h���	�C���g,���z��C3T��y����F�m���T�sd"N3��e�����v
�w(���OSe�@MA��$n&�h���f�:��M�u���,�����M�\:^����^���bTj���D%zm���CrJEK�S�9 m���T'��r�U��R�Ys�3V�)���� �fA��x{����i4��]�D`���W�J����j�@�M�~f�������E�����D��r�@���~g(���f�����e-��{�w����Q"�Q*�Ng�b�N�c$Mw�TE��N�NB�h+	�"il��vKh��'�X�]��b�����S�9,x|��]�[*��TaL=�7���E�
��O�b�E�P��@KG-�����C)��
��������c�w����t���?D�
1��_eq�+�o���IH����Dyu�[e��l�Y��v@$�j�ArYo�*��u�l��&����j('�I��C������!�m�D�S����Q�oD�v���OdS
����A��Ra�(}�0��R2���|�Q�4�WHf*=+���J��b��z+����$�j�O��<�JG�K,�����,��,W�mj�1}��S�H�X�&�����i�"�y��$�Q�l����@��5}�<��&9'�����-�O����X�rxfE���=�8(c��.���/�����7X(z]R4�9=�GL������"A&u��D��WH��"m
�x:��J��F4����(�d~��6�bc��e����7�����r�"H��g��U��@���\��4�4�J�������Z��~O��I=�]D�_�P7���y.��<�</�m=ex��_o�/�_LYX������f�k�9��\��".w�����$��;�Q6�.@���zu��;�T��t(}������g���`�*#O��{�n�;8;2�[�Y
�����8���S���m�G��qY��'��u��3�h��S��Z�U9WB
2�{r��`#��/;eI�)�����ME�K�,�|�?��v�D*��m�2����;eSC�_sx��,@�8gf�Gni����G��P)������/��Lh
@��A�e�HR��N��eUBiNUB�����JT��i��J�^��{r�`���<�f�x���'�DQ��=,�#������<�p���g�	����d����%+F�f2WK6�Z2bQ���uKR�6����j��|+%bmt���;��x6#!����R�l�X���NF�d�I��G4.���j�d�/7������My�[C�PG��)��;�2N��9�I�����2�G5hs�o3H��;�g����?�d5�U��s���m�+'�f���v"��fXb�C���'���f�CD&�3?��`f�!qW����_��`�6���6�:���CY_���GYg�B<���7����J}�XT�:�+�)��uF��c�prSj>��z	y9��������JySJ��}���u2�
�Q�����n�;�P#%�k;��D�KY v1et�x��.�c��j�cxn
XZB<�I��i���+�E�R���l8���p6����:O�h��JD���l_
��P)��sf(��;�{o%����vY����|#���f!��{(��5d�N����
�ZJnMq
M4����pe�=w��:c���_.M��9��;�{�r�3>P�a���N7����V�=�JPO��g2��l�����fB��,B�/�F1A3�t8�e�R�,
�Z�)�{{����i=�(:�q��tz����*4!���sPv;D���yP���"��"�$zU�����.����l���F��"�KF���,��5"v}�a����7G��fP�kyS7w������t��bN� +��b,���R��9%h5����0���J?�d��O���4x��&���x�&�Jix2ux�'{������V5���r�����0�
>��5t�/^h$�.��a��m�7j	�{��%��@����a��u`~~�Q3��*���<��v�4����2�! ��� b��,�G���U����K�2`PI	|4!P�s�S��������7yygj��b{��d����*l"����5�OIg{�p6�l ��|<^�����y����#D~u^��	+3���H�9H�����6h0-���)�%3mh���Xs�(����>(�i~������������!`3�W��O�Z���|q��r������ ��#���a{��)?(���g�~�I�������\a��i�=��Y�����N[�Z��C�Nb��l�Yv�-l-��f�5Z����by�4*�
,y^L��zJW��p���z0��/���v9�Qh���P�Hh
9Z�<����,��&�.�ki�}����<H������c��nZ����R��|� �T�^���������^l��p[6�@&���y���n���yY��w���BoL���=����	����9�uK��;�	�� c��y�y@;m1���c*����������/r���03(Z�*����q���R�7"�1��������-KNm�������e}"~7VK�
8�L�*��� �B]\���(�5D��$Y�x��a����
�
��5%���u
U6jx������������K������0�����D��l�������|F�����B��[&�K��~WX(Q>�xH{�����G8r�0��ll�\Q��������K�����Pc���N���rlq��u����~���r���$_�}|B��0N�1�4�u�IGBA���~��Md�F�<�m�Y�k����d��-�6����������o�sL����q�f��?�|�_8f���~nx�?��9�s5<hk��;R�Sa�h����=u6��m8{�R���O�����t4�@$�}�v��{�������=@��w�>(����@��L�^| F�����R����sg�����9���9�#M����r#���[�o2�������Sy{~3�,}Yfp�!�{����[Ha��i��Lz�<=�$���uc����������	+��HV��G�j��K5_����T<�X�}Ff�1��K���fq(5M���PN%dz��Y^���y���@�ig0������(�{����t�f������"�b9h������n��1Z2 ��#F?|�7�Jf$N������'!�=M�{
	,�/����s���p�{�ijJ �uI��t4�I�T���T��T�����J��)mTpJ���C�lT��Y��#As�/t@$8��������p*9�D���{�'���������x8���
�W��"�D���sn�F���]�2i��Wy����"(�<L��in77%l~�0fpe]���g���Lp�ij/o����^=�f���|��E'�XX [��5����;�������������~�����p�����#a��D&�&�i����2t����[�����r���Y-��d
:=\AT\y��"���6�<TL��?��v�dz�@��M����������A��Q���=�Qe$��D���"�]�:����{LA�a�[hN�P��G���+�<��J:��U��W��+T����y*F���*��eON����.*T%M����~�w��N@����5�T�]D�f����,�\D���F�!��U7p_s5M[���������4�t���;�\���zZ��=M�D��-����a5��\Hm��V��l�uI
�jH�G�B:e�o�Lv�{?z��A��9?��I�a���V�~�����l<�p`�������vK���n3t���;�Q%w���;��uw2��
Y���deI�����+���W��op�L��^��>���3afS���e�|���B#._mk�WM.��9%��M��jnZ�CVxH!�{����f�^�i��3���R������-������J' �����g�I;�XA�$t������X�R[��m+E���L��Q�����zcI�p��!���i;�x��q�k��#"��+aHW=�����QE^P��m���g�|VZB�V#����D<��?�<�z�d	�����^���K��2E��(��/@�G�ar�l�lx����?�eE!ipF�������TK�m�� �je �c�%}`bi�p�������
�6��
�6}�S�r(��_�
�T��>��i-��]u����L��nLk�Ey�K���:Y�r�0��� V
d_Aa�^w��e���p��B?Wj��!F���-^��-(=A�H���������]�B���R'I�������Xo����B^��0(��X��I�@�#���8����9�y���A��Ay�U���7&Lf�d��V�
9(���cy�u�7�V]}�I�/"��R�6����V��)^`�W��x�U���s,!l���{QB����cK��I�{'C�t�{�X���s[�:�I�u��c9��f|2�X�	X�r�������yZ�mNBG��6v���"Yx����w��t�w�
��sRNp�h��%:Lj�$e�6#��W��@������9��c���D�
���^@0u:Wt��[:/V���Y�i. �@�hm�(�^�o��%Qm/��,�l>@�%� )��z�����3D����Z�R�]N������<F���Z��a\Z�2��
Q{5|Nu��@O�H�{�r���!�s�`l]&�� ��������[�j&�Z�E ��a���Y�v��P���a*@��Gb��h���u��d���� 2���g�0+�so���
��A�i ����6J�,�V (��
g�Q����I����I���_?����340����(�zn���KAOp��Je�G���Cq��!�kE���b�>�6��C�n0��5xA6�5�1l���(�y�@�<�I�[.�4
�O�z[������6XM��!b�s���hqe]��U��Z�	"c~W�����.(����R�����������������U?���T����%��;��t�<���mZ�@Q�aG�VB!.��;��T�u�����1��1q��Z)���+���>E�tE�q��W�y��h���=�\�|f�b0W�kR���{�<`[�;%�s!�
�e)��i��A7�};��/�j��#�"b�{fF*SL�����k�%�|'��'��%�	!����;��&#N�����h��g����� 4f��8�\(8��������JU�i���!l$�5�x����6�K���`�z���s���P=f�v�a�Hog?0G���M8��"���h�T+�)�w�����
��n}�BQ�A)	�1"!i��l��RR8a�}0����|q.�|���m��f_-����e��_$�@��/����=�������S�*�����=�=J�}q��E���s������iF������lP��s&A��e^8{����Z��[�?���0�<)h�v���:��2��;��3��kdV
%��Rb�ed���ctO�R��/�&
�H���S��:�yr]/�k�7RTr��'l���;p0�3��X��$
%(]�-�>(���Y��y��Z����N����"��F+HyZ�ss
�����l(�z��B��-D��KJ�������"�]�@l����p�"Y�}����QA:��DA?��4�![����v5
��Y4D�Z�h(+��\��f)e&\t��Y��o�2�o��0��������~�|�p���tO�������ko(G|���]�>���df�Q-[�:���H2�sR�L�h�u�4����'D�3�3����3��~��q��#����N���%��f�C
�1�
^'���AG��]m@�0mg�m�yC�NZ����\Wj�C��N_x�����v�rp?<����%d��(�_=���;Ts�;��h77D#����~���9�
x��?�LH�O,��=�!q�������5�J0���G�N�9|��o�N��)<]Y�f��\���cQ�N�"��C�T�J*�d��*^�
V���z*�PHL�_.
������������K�Ro����v�����a��Y��$�"����>D�47J���S�&��B�)�7���vU����*���8������?���
���'��9��}���="�)u{z�b��T��
�O@_���Yuy*��40,�6��2Ei%���y@����7�z��C=>x��3U��B��S��aMA}6�h�?H�Ymh�n��'������#�D��`1���:R��"�j�B����##�Pb����^�q����R�N�Yft��
Z b�rW�!j�j����#���-o*�����:N���Aw���3�/p��{�=��3�=L���7T�h�/��(u!����S�x��`�(:kpX���u�,�M<��p�O�;H�f�b��
�o�
o���z����
�vJ�fJ%���LbG���3//�F���������}�K�r��:��ok�.��z���(Yg��%��BK�ZY��e��Z�����v��J���;��X���B:�X� *+�xS�&��|FN��-����P�.��/V%a����u������(� ��u9��m9�W�Z���r Isyg�����_��p%�;Z�u�W+��9��uJ������b
p~ OB���jx[�L�M�C�i�<��%��o����|�9_���j����>t-��-0�E��Io����RQ���\��%���Ku���u�Q���n��
9��I^����y|pID"!�InN��_���]"�
)����{n�i�����?�t�dA �#U��x ��~Qf��S�t���k��(����`��aM�!*K�=t�u��L]�h#0���`$5H���t~x[������`�[�Y���eJA������vs�����'��zcV7���J���G)O��,��n@���N��I�%�/A�i�e[[�ZYw,����D6}��Y�~}�"�A�b��t��r��7��
�^�/�G�)��%}����"�|�]mT�V��
tE��
��B���N��O�����&pQnfyQ�]rx���f[�+���r�7o!�h� \_q��������c ������c��ac�8��X
Z:c�� ��������\}���n@!p��cbb��	SI�P��i�621N�=�zpe�=�7�����m���������TEq&e�#6(F�w�����t�-�&[�����k1�<���]=����-v������?����?������-o��!���hI�x�b�Z����M�a��#
��Hx\��?�^l�t���j�c��M��/��a���!��������������CVU ��U��@��sB�)�dJ�E��Z�B��%�D�&�+����&�[m����I�� �Y�*C3� �N���I��@���+����D�p7��"�	�J��F���%w�F���N�TM�\���R��TB�8��E#w� (�����Dg�J�Cc�K�����>�%�
HF�LP�Vrl����� ��o���(��Vg@�Y)���`����|7?�?��I�`\��"�j����;�6��k�����?������>U�$�������CJ�0��:���=��0���c;4o-�v9h��A�F��^|MS�F��t${�J:\I��|��
IH���d}�f,�������b�Zk�������r-�V��W`t���#���:��A�����n*D�9����O
���p8f2@����m���/��x�=�-�32�=���m���Y��?@��|jc�L���9hk��%�,�1 4n���;gT b�&��2>��GHJ�{u��l&�+�L�!�5D(6DE��$n]_��W�D�b�Dt
���5��D����,0��h�e��he��e���2�"���1��\,���_[�A4��"���V;p�U��n���z%�M� �>������Qw"�������A���GXu'�1����>�>�l�e}~Bk/���4�sYWZ��mnR�l��&/D]Ek�'��PT���X<I����M�$����>�������;�)�Hn6�!
�f4������K�����}����������_����Hw
1P*:A!��*
��
g]�PM������[���eS#��b_���3�	���Z�����:<�z�c�[Z���E����W=�sD�;�/�E��Pr���sr�wo�o��q��T��R�- AQ~�v�����K���+��]���x���|��j4��������5��5w�1fFKF�l�}J��[ieX��Y�}btU����UG��r�I���*q���ln0�)��@V�bhQv��JDV�.��q��"�t���d	�a�j|'I
�����O�����7��o���d��>�Q����XV�� ;i�Nv�m�B��_"=>X�����B����u*�����s�<.�+��HJ#{/�$
G3����{���
�XA0�}������^��B?
�@����;��	����
���A0,`1.y��;^�0�����/C�����5i,2�E�)D�Qs��\���;<D�`�4
��~����iSHC
�s�HVR�}$��0Pv��${+B��+#r�.�U�Br��0VmSDzN��R&)������/e������^�pf�*5#�@����S����
i�����
�TW���mM��X�y{��@S_�����[K�Y��i�T������I�rj35t,����,�2����r�m�F������*��h2�0q��h�I��&��,���Mj��.(����c�s���a<��#1�7]��|�@�:Tk�?��C������
W�/����c$��(���;i�-
/����B�������9���J��7R�G������j
���e��|nFX�aaC�����Z��f/��-�����o�V;<�Ma��!���
d�Q���a��9rm_k$�Hce�+tS��������q�������Bv.�������n�S�{�����#�(��B`�|������A}��Kd!�+r��@��w�+-��z����l�% N�52D��'^�@���^���������!���Z~ )�`H\��dC�.D�����UdLZ������#��
1�1��p��
H���|��\��&O���zQ&�R
a����~dr&et���!�]�����p@�Pxu��F'Yd��H�z�j�MY|���(���X 4���T��b�SzS�c�W*}5��)!�P��I��&kS^l���D�d}�=K�����/-�le3�>B�ZxwX�����:��>�����q��Fqs�doF�d@ ��U��;�>'����#�R�{:���F�N#~f������e�+��Q��pmr�6&E���r�(u�7X�vx
4�LH`����������E��?!���yK����u�,�^mJ���;�x���[�z"_y����HF���5<G�w<~p�+f���P�#�"�]X�l�W
���A,��lUki�DIO#�����)*S��EH����
r�e=����L�����H�G�;��g3|�$�[[�3u�$��&��a[V���X��Y��!c���`B>��6����9�[���G�44��o+Kt���`
�M�y�����D�k�0��sip]%��p�'n4�s"My
�2t\d�<bp]���2{^VK����?�����oY<4��Ul�hz��z%a�����ri)��+T	�Hr��5*j��p����*�P_::t �o"��������o��aqe�T���n)p�a�i��b����"�:b�\.��d��$,���OW �kj��(���@�H��0�bP+��C�����r��+p4_(����)�U�p��v��y��rM�Yu��w����-b�]�0��\�8�_m'�D�s�u�0�RGD�[��=w��]���.��A.���Y���8#����wR@eW�m��+�6<��l��J`��}�s%3f�Lh���_�X��g%��r���a��H��o�2
���m#�C]���)z<�JF�L��\ �
�����p��m���Ca�������s�c	Q������6�e��Q	�����n)��E
�v���WU��������6Xff����6��BY��c�<]���;}��/��S�������,\q!^'�k�tc;�C��VwBA���QL�_��/�YT�hu�����(T���f��8�2�%&?��f�g�#/�����+��O�k���fq��d�+��������x�u}�-*��q�S�)�m�)V�Se^'�)�ka�^���x���������2��2�7\�%.�Y�Z<�3��|�Uo�:�/CJ����$e
�N~�	��]�@
���������Z��QzX�@&"�0@�_{Ke�,�g���M�B?�2�
?i
A����mG��;���J�"@�U"
�#-�b/D8�F�a��)Z��Sp[,?��~0������n_����0�� n�T[A�>�m�LX�e��}���� ���7�>����%��S.>�0c���0sC�D<$L
�H��`'1 ��g����������i�)�s_(�!l�b>?���)�F�k���Hy�4�<�����mg;D_%=��V��6�p�h���.T��Q�\�s�@�=�J��q:�W����8:pl�}����S=���V��!%T�����I��_~9��T�����yA���e<�L��CtC&sc�`�v����0-�'���\5)0��_���o��[eE�O��pJ�����wUoO������OF���6����~S#4	d�*K�{�2�
�S�&9!K���Q�#NdS�~x���8SN�	���J���/�{b?Bm
�,���*�������2�����2 ��'a#�����y���	n+�>��1�{��
q��}jn)�`�q�Sp�Q����F���@T�#<�^5���*�_�=������������mg�����8�r:��y3����>U�k�m�t���j��T.N��o�ZZ��[��:��-)������M�]��D�b@�8�$��e	M2���a�����sdb�]k��^�Jq��@����J�
f)d�������Q� ��0�C��WNG2V-o?o���E���:������Ct,F�}��X�����43����Y�&7c>f3R8����o���H��#�&D��n��q.����4a��Pm���=����Rp(�����wq�0�H����Y�������a��|��������v�����8l��n���������1��x�p7�����Z.�r�Y����%FYb 
4��3��:��S}@E��;q���p�:��R@��hMm��9��_���;o}u�n��3�?���)oJ�>����rS	YG�8eM^�g��N:��0�W�(����h����H*���"$��&��_ph���M^����VI��{�~
{������c�y:DM��Ch8�t�\�a�$�x
2�>r��'���>+�m[]TK���/������f��De-qBv��Q���{����<�o����G�N
5��86
�(���6u���\�+��r�Oup��D��U����Bo���J�]'L([[�7]�I�P�;��0���&�#4�:�W5b*�^�'(9N���E����X[G��B�ZY��AJ�����R�A5w<��<��4��f���������&tl_��!�j�-���Xm�x}��V�jr&z���pj*�D=�tp�*W����$o��������s��Q]�sQ:p�,]�����J�X��g+���Sv��n
`M`��rQ�����q�/��i����R����!�LS��"��~�����hH�X��K��S���N��V���/)���u���vjI�`J�i�����4���, c/M��"(��Q��("%UJN�.�t����d�F���*7�0�xO&=f�d:9�~]-N�����r�LM�oP�e��D�[�4��
�Y��&N�^�}�/p�=&h��f!��Zy�].�\�X�L��g����|��M�UW
�~)�P"5�6;�r�@�����X�f>v���j��5-�dB�jV��������T�HS�W���s�����������N�<@)����A���y�4�oC��C
�l���Kh,hA'���c6h7��U�1����F����o�JJou>2����i��)��d���|)�q�X'��������:��+��\7��}���(��AL����Ge@���|(������l{@���~|�/xa�5b����Y��p��d�	�������9����XW�����1�;�l��@���p��}yu���Cr^�zL�X����U�E}���'P&:�	�{��z2c�0�4e��8���mX2�.�T�-�)��rXd�&3ME�fQ*{~V������k��w�R��d���e�T3`�c��3�e�6��:�H��|����3�0�b'�r@�!�Wmb!���;&b2��((a
���<��%tT^B?��)��%���2���g:&%�� 1��0�`��bu�e5�19���Y�d�]��!�OS�.rK�~�KH����	��u�x��E��c<�v��>&1I���.��������7���*��C:C)�1X��iU���pu~1"�`<�]���A
;O��o^���y�������~.u'��e>�$ �x(3�rK����I�G������Fb�m	X�o�p�L�:��H}@2���jj�?���z��t�zPJw�z�������v��&�g���FOmn���qnA��l������2^�������;�z�	�C������g�=�hY$��I
lS��I�&���R��\���L6q����]8�A8�����fVR�M�eO/&�70xX?��[�)�E�u��4�(���+��Q�e3a��]��8�$��+�����b�����L��X�C`�l>�%�TH��K�a(������f%��F%c���������BFV�7c.dw�6�Wr4���hK=��(IH��oU�����%�eU�^Wzc��y�N�r.�t� N+w�;hW���C28��Dp�h<��S&��h(�o��4�ODo���_������4�}d7O��g�~�?���*5��`K�����Z���R2�$�3Z���]&���7E540$P'���V���2(������v�Gt���A(o�awQ$����
u}<�����2=,�#?�	���<d����N���W�dJ���\j�uuuM6l@�hM8�\��6�����7!���zt�>�p���j�j��9��R���3q[�5o�ToOH��Dq�'����"��N�K�Nu�#2��$%���j���{���@�kbnT�w������SZ����B:�}������(�`��MS�3O���4�:�J����1^�W^��m4]��i�h�9���0 �Bs29����������,B�0A�7�`�G���u�J,�YUX
�p1!�:c�ZS��X2��7��l*�d��wT �M��@����hO�=hR�1��P���8>(��"��i9�k���?O�!�d��i�8S�{[�6����?��w�9
���+��K4g�&�c��^mXj�ri	'��|�����P_�^-��NkL XxXr�]$���Z`�W��h.���#""�"�esQm���������v������z6�o�����A��04����%!����d����+7�8�Y��������`�l�6�n�+H]�]�z�xe	�'H�(�/��q�!Z�TD���#�"���������~;B��C<qf(�=:�K������F��c��V��L]hH]K$�q��8tR���gTBM����^.���PGOg��	�d.>������
��(9��|5�^�?Wm���W�J�9"��T�-��>B"�#i	�
�C/������H�Y��Y����>
+��x
���:���9�zE������������d4����y&%�M}���X��a��q���M}}�O�����PB
���� �	,��?��"\�S�g�������	��KB���u�t_���q���������K,�a�Y��(C*R��R!�����5�����H�b�s����j@Ha1�@F��Bx��Fb�4J1,�"L�{��!�V��e��;�@f�K

M����H���r�TXw���#
&]�
gx��TH9'���<��������h�^>A����PHHq]�Dzs��?e�!&>�-l�N.�}X��"����EG�E��	}��=���Dg?<�}mfJ�f �GJ���#!
��Q�>�0MS����%�'�1i�����DJp��lY��D�_�3���5f��!'��Fi��n
	�T����f
N����jQ~����"�������
  a�	�`md�:i��5�G�W%~��Sl��*�R�(��?aUo[��Rz��p��gTh|��G�Mr�DV���b<�4���L��/�fg���}�L����5�w��%�[��DBG`��f0U��aK�i���8R��F�a�;Ce��a
;�Z�iDV���M?#;I�����1[!��3�(���I������s��z]�s�S��^_�@���N���� �E�?����+R��Q�N�����+(M�����$��=�yH�e�����1��7���E,��Oc������y[�T�����&8��C�
�&b2���[��Y�W �8���>���
p�~|���������}KJ��������>�w�u
���=��o��i���WjO,�z���>���:N�\��3M�)���7��V7a[+��.c�5Y�zr��P�c������NBD*��^k0��	q[�Rs�x���zNY\�j�~P�0n:h��z�b��-^:��kx�� (1���d����{K��b5*����&�
{K���m� S�4d�uFJ�V�+T�� �1>�,�$�Z����#�^#�-�������PV*�R�d]�Y�wn����U�p���a�M"�l�R)�i��t���2�<qO�T��9����
h�����
ijx��y}����vW��f�Vf�3�<d��x������ v@�?H��Ua�\�*�����br�} �-��C]Q}��d(4�gQ,�SD���(����Q���w}.Q��#��S���^#�rI�t���zv�$���(/�u�VN�xB���`�f��^�b�;�����sA6���y�Ba3�=��#!l&r���������6b���~J�[�R��D��6���Qk�R-��'�������,�a��Y�@�5����_����&u�]�y���[|�BME�����u[���9�������1�6����1��i.��!.*T�spx��$����
�L��`�StK�S�;�����:���n5��j\���������\@��KK
���'�X`����j���|^������T�*�:8|{]/E���f^��-����/�r��:�w�5�W����o1�Q6�cu�z��p��l?���%�6��~�����?�~�;%;���l��K%�"K�5��M���E��`�����[=	��������� 
����y�z��2�x�C�fs^��ur�3�O�OL=!�z6 ����
"���j�i��&:��~���
���I_�}����M	�K8~���<�?���<
;k��o#w7?B�K� �'����G���r����]?3"�5�H�������8��T�����D�,f�o�nv��:L^3�+�}�H%�6�y�����3;��l���!d7}�o�,���%��\�&��X�w���A�O���?j�9��3Z Y�*h��M��|nk7�~Y����%���J�e���L��n�Muu�[���V�n���C5�m��u[�G����o_��ml(|��f�]�Vd��4����3�Pn����m�������9	q5hA��i'���P6���v
a��vdsYo}�����;�V~O[�?�T4���((R�%OK��|�����N�(L�������/�(}E�}��$���}��3��f(�g�C
�%f�,���"���;���q��-��#R+.��:5V�~��/�C���+B2R�T6�G�}�i�@|T����1).�	Y)�fVv�d�I%�f[��)�PG��*f�@2�K�>62����_���}E�)����z��@�<2Lr�������~"cKn7V��?�d�ufPO "�����4y��H:6����qy����4 ��=����5\!Z��Vw����a��OZ�;�w��M��P�&�����/h�9���(;���u�W�.u�"Hb���d/�[�3b:-�S]b������}4"��-�����x�l��A�+�&�����7;��d�:��4W���P�Y����E������@�A�M�������X&*��x�,n��.��j�Izp~x`yw`�_&�����l�R;�f�f�/0�����SC�g��@�(At��?��:���A��N%Kb��{�QK$
^=���D�D�����,���!��@^Sn`��h�8`�e���
�n�K"pud�o0K��R�L6����,��SFu�	�*Rv��uivh�aV�7��-~C7
�GW4,b������jsS�e=F�����5J@�;S	�!�$�� �_�tD�b2)�a��vEW�������5e���H�8STu���m�rz���N1[�
9�����e�:��dj�$��l�	��U�()��@���g�Cw�k��G'Y:�a������PJV������*cc��q�m�O�?e�q�)c�@���+v�d��\i����cW�\���	��\p��E�s�-�����]��{xc���XE1�O��������-����;��$���sPt�2�	�
����&c��J����f����I�2V@/E��|vT�l>XV��C~��j���V&~p��$���,;�6hJ;W�Lw&6�h`�(�1�1�6����
|�5V�,d�W���w��c����/��{�M&����L�	����=P��B];�d���0��?�
�����3��/������|^�7T],�PY�
��)���`$0G��/�����<���o��wxX8q��v@�.���4��g�����6������Z�s��`�w���i@b�c��m�&��H��1��(/��r#��B���c2K!'~C����g��Q]��M+���.�Ji��Q�?���� 79�-v4���J^��p}�^c[��T���/MG���Ezi�Y�T�8��q����B�������<�N�<$W��/�M�*{�����D&��yg������
��TE���sQ�-Q��Y~Q�8`<4�#��|������_����Ru��� #
��69����S�%�p�1
$0���Ph[s���p��k
�N���a��i!*ftrY�*�R�� �b���:%P2"b�ob2
�� P�����#5��b`f!c��Iz^��\I{U�D�x8�d�s�`��S����8U���CS���c���B$`����Ih�`p�����f��\��t�g4t�{��7�Bm�b�9�ge�S�<�	q2��>B����r�L�.aN�������lgaM&��	�)��e��L�\Z����ra1[..f����X+
�U��U�2�=�?�������jlT����h�^�=�tFX��S��#�3�V���^-*y�L.�Z��j��.�6�rt+�{����O�	��-L3&:��2yB�k@%m���Y"��X0<��e�������dO��3��`��G����"�W����4z�S���@���pt47�a���n*(�t�mJ�������sY]m�����xh����8H�M	��j� E�0v��k�Z�����/33��N�3;w�1�SI�RP��j��H�3��F��Y���u��Pa��3�J��oe��A�?f�=6��T���1i�ee���ZS$�u��@��m�!hu�P�,Z"�tS���p|���|��������%O	���/�BM���J�6F��H���K��s5"bj~�(O�-�>��@�X���*��5��x"? )"s�����N���k���&��
1��:��|
|��Gk����a��]�E����

�FU��S8�-���{�W����L�����oK��b�P��3_�nS��<���g���:�UrX��j��+�(eI6�4��R��=�;�1���(>����#�h���[Nl��N�k��l�5���p��e�2�C(����m|W��d�)���iNY6�B��v
����\���-o��s.��p��7�
Q��I��
���^��9&Li�H:]���{��Q}���^���}F����|Fn�8:�;�q�[e�h#`}t\��m"0M��m[
���O���D���3�m+q��1I��������8Wd�e�gt�������mE�������f��G������KK�cf)�}�"�jL����)S�HC����$zW�����6W�V��
�/�����{:��{��s��#'���i���v�I��G=G�/�`�U�mC�AO�����<@��h5:���F�Tb�Nfz����1@
Ri���8t�L8��/����R
T�E2�z2>�'k�B��G�Z�1�[xD���v��Oe��z2��3���,.�����SEN!��Iv�`�d����8�S�cy�#�P�R�+��P�����5.O�e�\������O����l� ��#���Wq�8Q�����D#8�2���qw�$w�M��_8S�������r=Dg��/Wj��md%����#��9]!x����
i�����FTy����V���@4~*��fgP50+iZ6�\dQ#;��_WR�T�&
���
�@�/�����A�7��D�l�n&��k��?L2	>��g�������`��D"��+�S���7;��R0�N��#*�S��.���
�h���1;Q�8Twfz!pg?^��\6i���r���{����B\N�Y(9<���p(�
����S����B*��bQQ�z.��<�C�N�QG<���G�"*�jj=�t&L:5�#��DX����0������	�;N*�T6T�$#�pG$���'����e5kV�����$=�I�QOgg�7���I��}������N������_,�#2���Q��P��y.zj����e�l�1����N�w���Y}�l��v�
�Y�U`S�S~�4E�|�wnm�[&�����U�i�j����[�<s�����-��>Ku���W,�X���T,�R	>���[��@�Q����L����I���N[f�q���wf~$h&8B_��>����`��u�e3�����j;%����dz��x�@0���:��
�|Ef��x�,#h��k��B��?o���C����F����X�F7��ps���@���3��$�^��������:0��;�Z[>��h���Z[�h���T���SE�b�t�<�jDy~���~�T��TH:�g���^��r!%�O7�0:��d��WJ�5����|�"�����?m�B���>���y[���]����u����Ou��1]w���r["�e���CZWJ�6s)�1��Z*�B6A3�������;�R��!Ly�jo��,�,G_���
�����3[#4	s�z����e�a�
����dbbP����{wy�*���j(��f���
�����g�F��%��"��H��6�F�3ZmX�#72��K1������X:<y��
N�RJMd�0j�(WDm��:��v-�T�|�v|:��O��7]���@�j;�z������������;hRN|*�5u]�pB�*��?�(^.1�����@'2�IQd�wB�X�#���C����a�1i����5��a��s�.��_l���\hP�����T�(�;�������b�����^���,9����e���o/�����`��F$7ci���<�����\����,6�RP�q\*����f��u(��c$��>v!�{z��6��3�Q���F���p�}�����hmCa��eqf?�Z�O<x��1wg���������P���D]�	(��E�����������Xw9�����aB��T��.����ox��� ��JeHN+Z9�(��_%��#T7��.B�9w2�a>�:�}N���� b�����Z[d���?����oA�W���f����&���]��K�9�Wu���aI�E��(���h�JO�I��������p&���O��!��t����n���	�	+D�M�<2_����N��ze
M��~�eE�0<�>#\ ��V|�		���[q,��P��;g�xG��Q#�5��j���+�P`�N.Ae�����-���/R��X��;F�c����D��N��"S
����N�']W��CG��9PR]�*���
3B�H��@��}��zH� q�sOk�n�������q#����\�Og��eD#~��1R6
:��HX�L��D�C��4:&��
������:9"�����H�='�L�������5BY)������iK&;�W#�z�q����
�L.eb�Ai���1��H�����[���v�_g�r�����|��0�NznQ�`�-��[vS+#�^d]&T[@����]����t�te��%���$t��7������K =^��� GqF~~��n��)��	�;�;�gw�w���J�D�@��<>��<���L3�PygxNDP& ����S����h*�}���D��f]-O}{!�|f��?�����r>���~Z��J}��K�4��	Mui�^�R�_��$�r�Y�����O\�����
�����c�3Yf@0��X0�l�Ws�'p�O�2u��H��F�X;����H���������r���^���z$�&���&�1 x��w�6�M���������=&f7+@t��Lj�>�(��|S@-����q��YF��x��������_e����`i@���4�ES��^������T{
|2.���,��h�tmw���:��F�N�I����#��H3����9m�ja���6�Z��.�a��2e`�P3����S\�>�d������Jt{��L�C;��jq0ztv�c���q�28X��7W
�iaZ�#�z��Q���2��O��Omp��eUn:+�����*]P���l�`��#+��HH�� Z�?�Fx����Z�*�U��\���#����A�T;A�7t7������)p����#�w��zN�L����z�:�D3�l���=�BxG�cC!$4�2������:hi'$HA��6X���c*b�R�LH�'�>)��)�WeA���";5�L~jB���yU#� ;o�N<	{w�1[�`���&h������(�����)"4�R�0R/��-�$����9^8�BtN��",V-��������f���X�>P*iFh|�#�����=��n�X��tg�j%� ���F4��aI�U���S=*��"�~��f�va���>�X(�P<�Q��(�x����.�����HH����-���s����7gn���eo'G�Zz��ecg��9����(�C0[��J�#�4�����~w"$CG?=2��!}A��G�
�=��������������n�M�_��a��������p_��)e+���������n6O�jy�c���.ZB�e�zFB��9u�v�C/�:�0Y_��w���o.�kkQSS����L��V�Q9?�P5��#S��W��������~'������;g�K+[*Y��2���:\B"����J��v�,����w�X��u�Q���S��0O\���]�l�u��M����L!��z��#�[$�[$�E�%��C�|�#�j�Ey	��>���Kr������#��z�t����w������([��h��d[h;J$5���e��2")��a��5�������D����6����}j\^OG(�d����k����QO���^:����>�G��/��bL��6���?:=���v��v(@b?��Q�����]��f�����2t9B��z��El=���^��U�Qk`�`�jP ��B���� @4z�W�q� �%�MW��&�_L��G������Af@��OG�;��p�
|:��������w�3C��n8����p��G�X��1/���ZI9i@6��������&�W�����{?�����r^_�l
.\�g��Z|7�����gnf�Wvv�6:�75B"�3���6	��lVTB���E����,�.\�b���.��W�����yS,�����S��)����Q���2$���%T��Tv��?��
vS��h/@����u���������
$_��6���mea��&��zDXLJ�s��.��prC!�z����E��je��A��g�����U�Qz	L?$�)��K,����6�p��,x����d����N��}=Y���A/���^no z��}��v�f��Aj�S\�5���:�5A��l�,�-���E��#;$�����=<�|w���k�J3,M�[6�l0o}u�I�-�=�������3
b����Eu����(,��
�h&M��\'g~�2-����`D��<�0�����m�_��`���7b�_�:�}�9�R;����H�6��d@	��i"}��D���<y[7�8q�3	2E ��{��X����C~����F��c�c��b�^�
��������c!|��d�	9����|S@ k����$��������3y���41��_�e�T���
<r��K<���7��A�d!��rw�\�W�j�{u�������WW������0�P_���&US;�qL1�v���x���hNj"j�@���.Y��%�U���-���r+a�_����3���a�s���i�a9F'��OT�Ib��HN���N���y4���
{��v��A������{NdD�@W���k�&���wF8�����v��{>z�(@��~)_�z�?$x(��>�}SV
���9	8���4�z���X�Z�j*���T�!o��2`(����O���T�j5>�#�O��PgT����#�LVN�������l���r-�-�����Jy���W����&��eg����?no\�DY�\�aSN|u��F�]N��)�V�Br ma��)��x�W.�
bx�lf�4<&�~��v���`�����0���/���������(7�3"�b�r��d��2��������ErO���!����Z:JB��
k����'�����kt��Q> ��0��/~���??�..�gX6�n����[��N�5|3�l\������Xr7�$bJn��U�_8���o��[��x����#�U�p]�]�^W�I�r��������?�y������������`19R�g���0��%x�G��zQ]���t��E��N������DX��|b���-��GC��! �l��s���l�Qp����BT��U����L)��+��*�9�F!���Hd<@T��O=����8�c�C���5C_��M�Qs���c�Z$i�C���������.�$����0H	
���_��
�a!��$��	�Q���&s��C��'����-�,W@�T���g:O�?6�*�ght����&+��y%�%���&u$����%���QHH�+,�W�m�����U�B����n����Z�o�H�#�H&��
�V^������Sy��}��l_`�spc��Qd��>�B��r.�����l��go����|�����S(�l#9��`�(]	��x��$7����X�Ml��5�N(�����C�y�J���i�����8#�:(8�o�����}6��X��r"��u��n
=t���X	T���1u�Q4��|J��u��.P-�����0�K�P+�����Ee��f]�T�j���:�S�7��F@oQ^����4
�,�j9�h(��g�y����T�� �#0}����]�J��k�\��z	M/m��-�N����n�Y���k�E`)�a�CCS��0�n�	��+��q�9�2.�9�CQ�]T��9��j������_�_�~�tH�� t��{EJ\5������"D�T��B����i�J��0qQ����"������-�|]-8�9
�[Uo[}g���P���#b�[�J<}�'G�m�T�m1�)
3�mQs$s�|�q�������J�u�\�u�~���[�_��o]~��F�3
��5�,+�T�����t�c���z���/����\����Uc.L�N��^���u�E]�G�9?U�E�l���M"*p�\����8a5�pD2�t���Q�z���S-I�4���T}@x���	U�8�?8dUs�O%�ytb����"#����(f�4ajih�U	E�yc7��]���+dD~��H��]�O_���@&mc�6��M�]lhMv�	4"
X���^:p�)
mY�e)����_��6/�MQ�%���Qc�E^m*�	txc(E�;�h=@#�m�>0t�^��Oe��3v}]�M]�������`����e�S�}������>����������_Pu�S�0d/z�v���lMv��?>�`��h��#�J:/���<@������Q��P�8y����Y��'�r���6TS��d�M�O���+�$���%1���^�f��q����5L��7>��)�)p�T���_��FQ����!����R��Dj�b��t�1�=�M}�
����G����������/8�
c|�'2
)��BD�����jER�"���cIv���x������V	
Tu)5j�+��M
�����l�4�����Gqj5�%�����0D�W�?2
���4X��{;?m*��A�!���d���FD���7��p���pL�1[����"7Q&��D:��54���uqW_^Z~�=�Z��F�����QL'�Q]h��oI��7��R�A����_�b�Q��l�1c������Y�F����H)(`Q�K\/���F��/�o��e�pNO����RLI�j
��P�!�FW1GT-5r3�I�����']�,�lX�J�(3����;=08(��������(�b�g�W	����&�J&�O��$8��;��g�l�/�r'�-����sKtzim�8���4���2�[GdoK&E��/�h��M�Y���lSpV�lhG�b�5�=�6H��&/�s����3�����b[��M����>�g���	�q\W�>���B����!>`�C�3��a��QA������~�m$�~��d����do@��=��$�u���lv��=<	IS� �h�����HH�?$�t������+�5�(�3�
p�CH�{w8{k�6��*w1u�+T:�U���6��'D�������u�"K�-\Eu��L���A��������R�d�P�A��������'1��f��@�����A��B������$���:��MPX�j����4�afw��P+������{��4��Q%�U�`+��A��dTD
#{;��V���iZ
	�=����Q�0��}�
�
���)5u.(�J���6�����:<PuM������ S����AF)?u"��G���A�
a��GW��A0#�����������p ���<���sH���T?���N�gC-5$��
��{:D�k5��]$I����8V��mw���S����[�u��H�� 8d�"E��*0	K
��|"4���4,8$�nE)���R-�SL��3�����o�1:�����~)�8R��n��X\$���~o���Y���i��vV���%#L�s����r�� �I?����Vd�%�����[�pBg���^�R�8P\��������p��.n�|�(�k��f��c�.��
�Q��\d��X����QH���E%1�?��Rt�Ou����"���;G����P��C)^�������^��%*L�C��gH��8v��CY��8�v���u&v�i��������-~�kQ�[j���V�zS�H��DWv�-�����/0���d��rG��x.�����m[�D��Q���i�l~u[����������Ku�Au5c�\�Ri���?6��b�w!$ mHS�K���D��9��k���g	�D\~Gy�pBYq<9=�����"�z�=E��{@�����!]�dL�S&O����\xyz�.j��C����X^��������p�������g?L|�n�����w���o����p�xT�u,��s�L�=+dd:�w���A��B/�b	���/��_4���#��}�I/�8?�Su	F����r���������O�/�(=n�_���t��^�`Q9_�e�}��zD��iS����AW{w~6}���������3L��?��y%����,%N���17��^b�*���T`���S&���<������=6Y5�Q�. ��@�BKL)�M�QI-�1?v��dSo��L*&i?���V`��@��d�]����H��c��_={������Wo^�>��)����5�\�'���Ni�N�JC����8��,��FM�'�]����o�U��
��ji�����d�Q���N�Q��jFp�����V�����d�Q��������K�Gw&X�j��[���\m�YK����/}�d������C�uy���l��f�%;���I1-��������r��t���01������'�4;���5J"����������S��
�O�LO��%�	�-����	C��Zn���)o��a��h=�@v4)�7�?��:(�3�?V&FArI�~bf)9]\e����\x3�vgB�$K������f�L��*��Y,�c.`��[/�.�z9e�T,���Y1����l��v����,��d�����������" 5�<[az�+r);�YmV��>������X�f��53Q�;L��P��P�a���N@�d��������k�p�L���*��/U/�������9f��a����;�9��|b��"���@���L���V��/�:�CFI���T�$V����T]���5��?^�W[�BvTt�s�f.��xi�P����R������MB,��O]
w�{������-B���E�����Y�m�b��7�	$��G�7����b�e�2�LB��r�f������a���?f�mC�$�R��TW	���Z	7)'���0��)R��%�Zx�"l��Q{�S���Xn���j��;��a�����9&���q�$�k`5���D���'�L C�zL(�I���G��������'�U�Rv�5������qJ6���)����}z������E����W�o��zU+3N���|���	�N���,��'����6���;�~��r����������R:1��K���_���J]�	�w��^�'�[��U��B�5�0\���<��������.�c�m���`��~�����Qd�'R��Dt����:��h����Da��,h�v���71�q�`��-��(�c/�&C0��'�O��O�X��O�P��G�R|6:=�_���O�>�T�`��i.PX;�7�C�	��������w��]���1�q���h�m=bp����oo^��z�Fx�FBQ�]�I�.���{D�������0����r-��&EvU��$��(j"�X�mU�(;���1��%�:�{�MA����b����M5$;�X���	"0>�J�>�?�E���	3h�wr�[C�M������-��������,q��y����w�J}��hw}�rL�6^^��q�5���c	?3�jk���i���G��cU�chkj�d"����v���@��%�chk���5�q��K>�������$O&���({�t���=���Z���k���g��C��h
[
��d9$����`��jdf��%+$i66Y�|A�X:�1����_|��;�3@�B��Rv5"�%Y�S�<?��f><y���'LpL2�5�I��$8r��1�Z���F�(L������$���I
��r�
�'����WZ�	�4��g��{y����t2��g)yF: ��,��b�f���������7�����K�?���GL�;����og�e�N:�/�$�z�I��Ip���K��1}"CD�W��m�'��(8{"�P��-u]��
��0Ua�~�s{h��TK&�����!���Z�7���U ��>�����Ya��Hy��#sOE�&��~/�f(��j�-��1%]��Y��)���<�����>�����d�C*����D�>�����&G�)*@cu���g6]A�y�-�Y�Y����A�������m�rK�g�L�5���(iG�
]!P���S����k�n(g�\���A�]���{q�(���'�vR����4���`������ Pu�N��o��I��B����>}%b��x���:����^�d����OM��#f�<Z����� �ptC��d��RD�I2"�U���&e�++@��/���XW��\�},���ipV�gHF��e!�B�--��2����5�D�.<,�����W>�s�e��A"���!"i�t�a!�V���e�9���C�R����46�f���.	�-��5l�0 $[#!|kG��������P��p�P�~�8��zq�U=�V���m��(�/��W�,���_"m'�&����8�4�w�R0I�d�i�)����$��C(��&�i?�=�)����v�U�RY`2���Q����!����c���8�z?z�(�1>�O�y������������wR�*�U��i*v+a�����j]���.��\\�x8�'��DF��c/��l��X2�2�F7��^�6���.1F�E6�XTT��^�;�^��w��"����.�)��6w+�w���i���;��*�I���2q���b���Y�\�p9�(�����������B%���(�,�
+��,}�0��w�Ha05\.TnrY�k��R���J���m��(5p���d����X�0j3�I�? ��"d��2�+i^����T�u����Cd.b�V!��q�.����<_nv}@��n>��������/d)��i���������j�~���bc�l�~�?����DF�|T�S�Mw�$\m���7U�s�}_�����9u�2�yg5,�����)�%77�\���$�������,�Px���A����W�)x��A[X���W2����;Y�j�O��G�A����y2iPz�1
�����T�e6�������#g��q�lK5�l��j��j�&���"��4�l��4�l20G����Y�#��g���zK'P��Iu�8��`�$4�s��	��?���_�
J�)b@NKG_������#�ux�Q�Cm��TN�)���vaM��������2q�,��23M�{�0�U����kT��:2�!��e6*v�'����F��5����� ������1���Y��8���H�L�-�W����h0����NwBE�_l�{�-t�`/�H6�)������o����H�s[>�s�Hbh�In�����������p2�����u�.K3h�@�0s�j%���W��wF��8��Z�T
Y��
-�0q!V��Xo�?��d���n���]���"^!�q+$#GV������Vy��!d@�P�5.�9�B�6f�
�4TH�	�7�#���(�o��5>���T������u^Pom����Qp�� &��T#N��/�q�y�v]�mG��L�o5�E��B�����d�Y��Q_�`�6}��)�Zg��y<(��Q7+���D~�](>$#H��b[-���S�%��<�����zN��r~>�2�n����?m`�����^�yH��������Xe���n�����Kq��E�K���:���j� r������q�t� S2�A�r���iz�+��"���rN�"���"�s���
Bd�d4{/7��H�������z����0��br�H��`�R��,>��A��sa�6�C�VD���<��Q����'c��,,i-��+�9f��)����A����I"���}���5����Z����7���jBX���Am�c���B�����w/^��N{�v��t�H�@���1j(�P����Z�U
?]s��ZZ���n�YQL�Qe��`��T��wh���#�o���
"i"g����__#KB��P�vX	��sk7�1�<U@ ��������(���m���INPs���N�;dlGm�2��.�_��Km�9����W3A�D�} m|����9,#����#�E��q�dsIXK�e`}a��P-����R��t@#�,����.��D�	�,!@K�A��� Wn����+.U�H�A��f0�Q�'a]��'�5���XvJ�	�4���! ~���n�;�`{;e�=�����M�v�zn->@.#GYzR&,d��!�i;7�X7����V<�X��lV���&���I%�]������T8������G��2Oa�-|���B��&?��l��Z~���,�����Y���VE��c�,�f���r�f���"K����e@h������9��<��0n�lZZ_��{��X�}
�a-����r����rw���"�m6��+�t��;n��;�������s���o�7�4�Sz�Yp��^�f�c����aBf'�?(d��{�	���*9wL�I���<{C������.������QAg�Cu����n{��f�������*)�e�����r�s_��s4�9�������M����R	%�s`�#���p�h��������{��� U����_����^��5H2�A ��8j�g�����{�m����V7����xb�j��6�"wg��n��K��1�).���&c�Z��9������72b��UU���������Y]DK��c�Q�;��suM�
s����_O��*2
���(��r�o��m�x	�D!`�����j6���s9�nu-;Gj(����R3 ��u2�+X��SE�iE!3��>W"|��o�����Z����D����^i�����zy8��^FV|pd��Q��n��b��dXyR3�a�,�v21J�{����� �;����Z;�����!T���Vy\��6��L�zw6��^����y(����~�(��� �K�d��<��#p��a����Q���I�a����v��n�D�L�����F�*r���b��S��@Cw�������J}q�#"Y���D=b��"Q��~:��5pB��1*���,�8���t��a^]j�]K9��&����7�FF�;�8I��G2�iH����1�c����dT����([��'�ow�p2��E����MJ:Z����?_�~���#�"�/�8���L��Oq_,T�v����M��F����>(s������ H+��Sm8�H��!�FN��j�����4���i�{"[�
�o���u���"E�sNQ ��D� ��(���U����|���`u]S+�������X�7���E%\��\F?!�����*LR���vB6��?�����	����A;F{��c/�X*n���}����b!��Wa�h�=1���g�T�_&OS]q��1������}���n��2���s��,���*\��q��}����:�A�~�}7ndQ�<�%5�js(�Q�����dl�ghn���O����_{�Yf�h7#E�`����bY:t.,"��T���g�Ns�)gSRT�)�y�����j.�a��n4���q���&��I�9�z/0j��7t�-?���U8�GmK%�oZ���`>g�`~c%4���9��B���RH0����
�vgl�pv����[�f��	0?��b����7�!���oU	[������2�I9{pM[��N%���)�*�F����-�QKZ�����U���[���V�zi��7*�PN��R�?�)U����j�K���������+��s����WC����Z����t���K�� �]Q<�pO��a_��K�_���@������`Tn�%�I���D�Xx.��)�v!@��JiGH;#N��1[WYcn<<�io08u2��}�
���[��/uF;�IX��S�����������[^=0�
�IF;!�eR
<��Vd�%���M������:X���$����!���Fc5x
z8����o�i��I��*k�{;I�t/J��Q*7��{jR{���A�7t�^\�"�(R�X�cL��nkj���+��~������8�`���$c�|���z�M�d�����s�p6d�hA6a�����h;������Y"�p�)%6��1�"a��%���J*ypz��E�������o��)Zv�X
9$�^c6���t*N���3i"<+*e�'�D���}��xba���|HDq�����f�X*����!~p�	��%iM�Ox::<��|��:��g0�AGd$��[��C���$fu����-q8*x
/V�������~��<����llGH��{�+�9�j���������O��cDO���|�������-�1C���0����*���d�������C��s��%�]y���lL����M<(B���m�����U`��N!Q�^��d�m�@��MM������ �pL
fuE�FG��d�b&F[����`�4!Q�&E��GS]��x�]!Z��Rv���G
��g�]��Z���=���x6�l�q�xG&���?�E��l������z.�2oF^�p��g3�F���� T>�I�������<��)u���88����G39�4��x)?5+�@��{��(��owVV��=GH�������_����#d�\��a@�h����-Qy�.T ��I�2]��O��`��V#+�r����5��v��� ��bs-�8q�'xt���O>�<?�"w���y��j�����o�;E���vwR}���-b�T�(�|g~�VP�h>F
����S����p�ic�s,����P�GT��L����z�A�5�+�[���o���I��d�/r�,2��B%�]n`���CI������rSq��,h�2�)c��Q��)��lU���Q8��`q+� �sR�E���0�o1�#��*���5
��/���<�
������
�(#�`��K���������]X�)7�������O!dY��>���e���R���)1XJ�/�_���%�&-�BX��MY`��r+��z���n?�em���e}�d=@IE�Y��y����%�}�}��
�'i~`p|�t����I��MQ-�;s�LFJ�Y���_��O�4�
�4{��g=��������b����K
���4���u�����A�
�#{�T-4�����[[�K_F(Zm~ ���@j������YmA/��67�jJ���u!0<;�vc�;BU�e�K_����,sb{���~�"�`���N��Sr�����'��^I]	P����)/�z5%�����?�v)���d�M(k\	�%/�(��8�7�u��6Fa����������1�G	�t.|%r52����1���}�o�������9)�?<�������lt&��S"�VQ���Q�������%V�M��Mz�s�0~CPu�6��d�sB���<��0��\�#����B�~D�}�<�N?�>R-�&\�x�*b�L�=�����Ya?#�<�.��H����%<�,a�	��s9^��<����=�x�u~}�
��K����C���"��9<)nN���d���.W%%��{�8����7�U����{����@Ou'��~)f��q������X�|�]�E�������2j���������FHc%S{��1�Y%LL��hd��e9@K�SA�wN2z8;�\"X���j���Q�(~�^��������	���L��B����W�|}*�w����u� R{����Y�i���L=���4��c����'�Z��OR{�����q�0�V�'������(�	��S�����*��w��#��sL���j
���*1F����~�n���O'�g(�&���{�d/R�(x�M�M�"=
:|�/�����3S1g�@��+U��	�W�����cY����TB��x�N��a�>�O$����C�B
/�L]��8�Z�K�@�!+I�l������5��s��/����
.����s��^L����L	�LN�����<�#H_�i]���{W���aI��Pl
���J��|�]6����_�����iN}��
�����H
�V��@�n��X���kW�;�.�Rc`�q�Fw�n�����$%~��'C��_�9�x/������������`����D��Z������iQ_�1���+�e�����+��0��!W0f�=��-����AS~��XT�>�J���F�'�>�S��W?��9����/�_�}#{�g���b��4��|��H�YC�{�'
����Y�yv���e���2-����"_'{Q�����3v�����/��g�������XU�!|g�a��%"�;�&z]����C��_�o{������~}q��g����w�����(��9���K���~��$���j�X���f]/�{>�
�f���xp��:z�,/"�
��=�?��@uJy���qe��2
�p�9z�%OR�rLS�6�r
�a����:��IlB��������J�Q_w��/�1IhH��
�"(N��p%��%?O}���/����I������������^{���DC����f��=�buD����m�����w����Dl-m�'f��M����)��7O��FK�Y�wg����^������_����A��LBDP(�������Z2z����_���-�$#Q\��F'�f}���	��r}U�'(%#�qQ<��R�P8Hd������>L8B����(FH�X'�Y?�;�
�O�����}!#��A��
�|H���b� ���o����O�������C�����Yhqz���W��C���2��-��)�Q_Pu?���[#���?����}��>x����v;����kU~�5J>���n���i�����Y�Ao�Xi�� 
���A���F���2l3���r 	M$Q���;m�&F��}��!n^�o��[�/�/�n\��o�d��&����3�g�q�R����V�6�)�I/�4�DB.
?��K�!����2uO�$c�HjT�����b�I�}c��o�R���hmG0�!�����yJi�m�V�
"0N�dp����h��7=��Y��
�����	r?��[�;�x����]���=�T���J������! ��3~�6W��2�����\���	���In(���'{]�t�D2E��)'m����Z/�/�����z{]��U�r�a_\�b��.��� Q���S���'�GgGF0e�8�3�]S�
L|�=u_�)A�1_� _zA�4��c�hfL���f�?V#u�U5�B'e)&y��.o����S���L�W2�9t�a���M����;w~����d�," �yuY��)������\G�{R�7���c����ly�FH��c�q��_j��u��8H:�HGJ�!�e��M���JBi������z������q����rQ�V�`���}����q&��X����P�X��4U"�;�����Z�\O�w��6�rN�����	X���� �w\4��j@��e������Ua��|�����r�U+����E�e,�������oo^���3v�I�������R^�� �D�8��]V���'�!��`1)jm��#p�X�X�w���j�(�s2z��X*����cT'l�^��Q��u�*��~w�A�rZ$�	^��F�X��)�%����H���:6��I_�m��KwiI��KF�2�U#m�m�Iz��]'''��U�\��S�jK�{�ige#��DB���Pe�.��~yex�e�����4���7������g�����q���g{��!������P��d���o?���Ye�kc�=K���9�X��m9��l� V �?��`-������>
L;�����}��n������Q���/��������1V}!�S]��S�;=�:L�;o���w��s���U^��[�!J
t���W����%�;|IP��M�*���0)���%:�d�g��\����@�.f[�,��[2z�*�����2���:+3�������`���b��G7-�LH�I�
}b0X����[)�[�60Mn�wg���z>}{����)^���xi�Fl�?Cb'X.2Y���7�5�&D�����B�'�k���HN=s�#�Z:�<��n�
�PN8s��{���p�H������_����@�P���tv�sx�������pI���Q���e��IW%�Z����Q�bU2}�Lp|Tv�"��=s����yOS�Xe5������_Fw�����,#���U�PS�t��b��FN,����s[�NM������~w[�
<=_5(�=�P/GH	����ov7#��%�}�0�?�)����X��U�����h[���{S������d(���2�,�������o�������x�hJ!>�������SU����=%�Q���~Av���U���b��&�����?�g���R"��*����������,����-:n��B�k]�-Y�����!��*��q1��7�=�r���_l�:�|�E]�=*�I�z�`����ZKZ�
{
������2��ZU�
��|C
��������cj��?7v{�g2�	��E�QJ&���Y�j��2a��r���
��`�i�D�GE�b��.�J�� #�M�eB!e�"���D$���L�{.�c�U��������>�D�����CS�����L�R0�N���=�	a�q����` �)7G�K0����U�?��I �����8%�Y:V���h�����Gr�C�+�����G�50�^��J��i���1Z�Dy�Qb�3�@^�x���,���ae=J�P����|�}��g�C6?�0�0�Q?t��d���}*#1$+)��'E����O�vH���h)��M� f�����2(�sK��C��[�G@M�S�U�
����VX���h?���/_K���I+�W`S��$�r��&YRH��^TC��|�K��Y0Y�H�r���$hJ����8���+$������G��`��bM��{O���n�7%H.]��P��v�K;�w���&����`��0X��`�2�e\0	w�zdP k���R�(�W�|P�=��E��o�������f��DD���^��S�BP��N�D��!f���)TR&#��Oh����'���wLq}��w�a�$J#����_h�0����I��`j1���#3��}��=�e�W�x�I����_+w|��d!�"�QxN
<
��[��r�W��uI9#��M���*����N������RF�O���8s�a���~L���LB��Z�{l]2���@� ��1�
����3����[:*�>�KB
��	f���y��b0����g�4&�Awl��{a�WF�}�i����3)�����T-A+)��d�i:H)"z�D)9�+�g��V��B�<���fv]�t8��>{��9%a�u��b��]|�
�.;j��
��`��b��]WW�Jg��{�1
2������QB�U�L�;��� P��7J������Ua
P�b`��#"�*B���@���3���^��j���0�0l�f��Y��I���#!�2v�k���I�>>�Z�=���v~7�- ��/u�Jh
S��<:=��p����Xx(3��3������^���z�U���N������F��D�,F��r����S�m`r��8E���f��J��R�l//�YE%�N|A���Bo�����@�%��J����pQ^t����^t���h�S�b�
}0���]�����3	�����=��0�G��c��?>�����6��5���<���=���,��B���/E�k�eOM�O��0��"��D�t������M%�zIq�yuy	W�r�����HL��t��9�0'��Z(hd��\H��Z�+z5f��9�����kHA�C�+)~2s�k���v�����5�{z����N���lF������M�l8o���:��iO��51�.r����!�x���Tqh�[�������������P@���c��Xru�lg�}���eU*M��c
�\t�.���,�i
 ��T��Hw�cm��M����N3�d{
�_���&��{j�I��,�<(P��%��V��@���A�|q55�������V�v���pv`���)����m�f�a7���Ed2���5t��D&�s���Y������)t�=��0:Ut�X�^'����]n#sr�J���}d���~V����u��rWt�(W�����M�MH����^h�Uo�X��-hvV_(�
�����.A��{�0���X�$��|��,�-!W���j�5��f1;��92r�_\Y[�\=��/�

�<��*��\%�|�8����� q�-OQ�������<GPF-6:���8�6�O�#SK,u��t]]T���9���F}&3�Sy
l��:�����H��T<�>����<<�v����{��|����d�	��L
��2o�y'���<���@��;G��������o1��Zld��@2���;l#�s�T��d�:���KTs�+t|.���	����`2�xX�t��np@x&Q]c��p-U_��jY]���9�J�������F�nR�D�����O��u��)�332;��8;'1V�q��*��N���}����������	��q������IC�6�����,tY�x����r������}=����?�%%���R2OO���,_!�e�w��eB/�0#G�=R7��Zu�%!ZdVX8�Ub.z���D�U
��/������Y�W�z��;���h�J5�pX
���^�:G�(3q��i.)�W��������N�V�����8�>�0*r�^|����p�P'�����29�����Q�BG�[��\'���Q���;��eOa����0�����MH�~�r�����{`�5f���*�nD8T�	W�����i*��(vi��+yA%�Qhp�m��y}����x��x;
���.�������O����li����w?��y�tfs������!;�A�D����g�o���=HBt����N�aF!�[����0c�8�a��bY����������y���kr��EG�������c`_=o������&�����-R!*N����M�4�-��u�]S��|�+a�s�d&H�g���& 27UC�(��x����;��� ��b�u[�1%w�������}���q��2�*����_W5u1Ws�f�����6oQ�n��i$*1HF0h�Pa�`��
��P'-���}���W6���������%��&+'
Q�o�����x�_qJ���>M��P3���wP_��x��R����M��{�YF�f,���f�`��V� m����*���Xg�����"��u�4�a��k=�)X'dj��(�	L���������!5�������>	����6��$T�,pl��m:��$�}X��m��w�������-"���yJ.�K�ulmU�%�]��f���D��$����r���L|3"�)x3<�g��������t���3�5&���Ch�n�6�.�QF��A-<(.�Uy�+�p�>���/{Ya�
'��U2��r�q)>�
q�8*�o��E1���g}����;���cw{�S�8���������js������0	������8�*K1	h�������w��q�V2q�o���
��Xc�oy�����
���:V�zN0����>V����a���V2q!J��h��+�����T
�z<0w���G�(���������� T��cA�CH}�[�&�{)�ra
]��^��������b������fU �����A�alw\�������&eOkR;dp����$�N5�K��P
�b�U��H��/����V��|:]��=��t>���v��n'�����4��[�)_��	]��
���	��+�����3����j=/"���F�^���2�����m��{�B��.e���)���lL/�(�\�L6��7Jx"S�F���Z$�RH�0�9-N��^D(�Db�uh&Q���pr�Q=��v�2^:[�g�	�t���
-��b��L�,�<���ATe�a.�U��K��B
	��D��4��*������l��x��4y,XV����Sy���2��/�w������F8a���h1o��vN������U����tu�QF��%5Z��R@Rl�G���nF:eF\���!��V-�u�9~{�I��y����&b�u�l�\,fV�m���5����(R��N��-��!������1z�`L������F�C:�UB����#sE�n#��sz
��+��x_��iW��]
��v���tHD�=/""�xD��<"������fC�����^������G�`EA�>��$�M��3���uYv6��v7C�������|i�}����#_����C���.gZ���g@���2d�W_|�`�w'���G�2(�/.�����-)�V+s]��!���\6op����TK�/m�O����:��	���wQk��U�J�.�|�D���+�`�6����0�pn���R�:@�+JD���{�r�,x�"�U�nn6��8�����M��f�#t�D�N�/+G�d)8�+6�������^]s0���K�gOV��t�����������
��5���>7�w�!����p�T��zI"���i,��6���0gC��c������E 3�� <�!�	�D��C`�D"r��Y�gx�l�+Z�c�����*�v�4 �#tE�����EOf\�+z6W���!d�BY�\��i��O��Q�1��4]Pc.�m8���r�����
 *��tyrh�0�T�B�m��V�Q}(�Z���"z�L�
tU-�I8+�)���B�?��d����w�E��sST�5pVm�e�"�[�FY��w�U7�/�����T������)5���l���KT�w�;�[3K�=W�^���h���'N�i���C�:�.��15:J(��Mb]]Qg��M���Jl�|MM�)5%B�k�q�&�@���A�$dfP�2���}��;f�!+���'[�����J�.��Mtq7%NQW�:}P���77�e�"�i����[���g
��L���^�{�#�����	X�He���#����(��Q����������E�c�K��2S(>�d�(6z���;������1T�J�(��w��];y�����8��*�?��}��ceo�������[��|eo� ��$��h�����fP�2J"���N+;#����y���}<�����,Agt�@y��E\*v���,�"Je.�(�6d��HgS����*�4���P�}���(M�gW�,�dY�_H�	i*����L�m�7��^�O�]��Y��o;:�"�l+�Q���^;.�"��k�d_D�m�����v�=��!H�$aE��
;��2!���	���|��	w������b�0�<2���vq�5 uP�M9et��I�f���I2�\��p�]�l%0u�Z`y*[U������@�����eBKA+M�
vO�k�n*�1�����8�|�4�zs��Pn�-'�@���sf����FNqC�{���2Hf���n�,�I���O���1��U��=��$��3%i�`�1�)��o�"�0&�d�QO�w�3������	5�D�=rFM<��yf�'0otb��D	�Df��f���W���<M��������1Y��6&�s���8��I��
�1\(��c?]�1�ds��k�s��cc�
�$�F/o��-����!.8�Ut
}�p=~_T���`���j�y"����I0"U;�s�=g���1l6<&9��U}
3�24�� ����A*c���B�����J-5�&�n� ��e��?OH*�'�H�Av�Rq���O���I7��S��b����D�b���s�b����E���g	E���g�D���V��������9|V��~�0T����(k@��A�8�f��A�,z.�[���?y*���/@����������b�
�712&�7c�[�!���x������mj�E�)M����������
�����J��a�l�	;(�e��
���c7H;,�����b�b�F����b7�6(��q����
�
y�-���1��
�

z�n�
^&��,�'���8�U�����5NL���_�g�d���Fy��L�<z�k���]�T\��TG�p�k{UG�^�T�������xx������3����1a�8M�o;���#��h��8�#��������G�
�y�����z��cF����S���S����Eg��cWqv�Ef�l��
CW�?��f|Y���v��\$d������}u� �}���b ���	�<t�.�hg��No�L}�Iw��'?O�wE�r��(�5����@2����:�����.�I��V�_w)�-�I�[}�$�g�{k����A��K&m����}��@<NR"SH�8|�p\I�H
'�\������.�#8^���������K�������w�M����`�h�!a������U��{���qZF0U
l;�������?%�2�W?w�����av?�9T�a�/��6���M�8|�j���4��sQ�10�������y�� �D>:C���I���/� !��;����I���3`��Av0?�C{���I ��&* ��D���
��C��\#����J|b���4�����p-)	m��}�J�8~g�O�������t�^O��^���bPDnl��hpQ��h�^�%%,4x2]B`��[%cM�$je��y���6���;
���U���J?���D�D���Z������(���n�$jp3���U��aw��2�"1��n�0���;�Gz�t��<�cH+"V��f
M<����>��L(n����c���d��$��K�����$~�"b��=I�{K">��=A��}pD�`.2/|���:G������m�7>IR�����'��7�#�$yw��>uT�^�$�����i�c�xM�� #T���v���'i�btqG�
�;���;�TV������D�����i��q������9�X,��E	z�PV� �N���s:�zpQ���7��_d�SZ����u�(�^8����${�s��:�{�U]���{���q~�����iN�A��~/v������{��ND�~//�}�_6����v<��S��<�f����7���Nd8i�D������!Y>���
�$Z��%���tb�,����3�C�_��Q���I�
��C"�VNp��X��) �S_V����'�;��W�=<�P��f<�C�����A.��Ow_������$K�v
�/Y����������7����O�(K��=� k,
�Y���
�l����A�g�I� ��}di ��������R�N�N��������A����RO����4{p�(���4dN�V��IC�8�k��:p��&���q��4�����5tJ#��7���F���h�F�2�FEJL1��P�H�Z2��VT�jE�S�R����R�RAG*s��#�'NqJc��bH���kJC�cR���$���O���bFvt(���)N>&����X�b`BbK]=i"s������&�8���%`�Y��`�����cc{Yb�i���B��f��8q��y:��L��=);�I���!C�I�
����/����}�����'���#	$u3���:1�7�^���H1p�����|2x@��#l��U9�����!�Hw_�+=i+b�Y��r�i+�a�U��R����d�����Y+�aD�F��o1"�3inl��i�D��������8!t�4�������ja�%�}�]������u��2an����1��iQ$�<��y"�� �|%=�?^;���#{���v��M&�J�lb3������=����I�a�=���"a�M���&2{:����l�}f �l�f ���|. ���I���|J�d�y��
3l�h9��w�����1�#g�)��({��
?�E��ld��:dal����� ���vY}2X�,���
�3��n���q�v���C��`"y9��.e�n~r�dBtG
���i�g�x�,|N>'���d��pp��A88Y�888Y����Y���7��H��E�Z�,�B�dA���N�Ll�7�fv-n��E
���n�e�f>�����kD#S�T���e�n�
�y�=�%�"�����O���oY5$�d�u�wQ-�?�J�p]^������I=y���,�������Hf�&�D�!k�+<�����1�������.'I�xZ��4V�!,=l���������U�G�L���d���b]����UQ�A�E�F��0��>�)��EF���b}�I/���CjJ�?��pi/�"�*������@i��.�T�e���M���Z,�g�	�W���9�f�QHo&����wJ�)�j��-7��9�
j9���Af��a�9��8T��ml�5+��C�Vh���#���7����I������A��
���
�������b5>=�� %<��:�	�y�������;Jd�������I@1�Lw�d����}�N���pc5�����p{3�����Dv�����Q$����,`Io����i1e@��F����U(0��'�Uy����n�|a��L#����
�4��<������k�f��A���{���&m���t�������K��F���#@��`��&%*v��N-�^+>��]���!������l"��2R3�h�>i.��o�J6�O��5#�����'Z������l��LM03�Y����'��4Y'O<UC�`��O*� 1��l�D��d�81F_5����;"## �Fn��4��T����S}�C(col�B��=0z��D/�6�*z���ES|,��{�����X��_�y�N�I�r��lM�����D���B����|tjZ����}�5�e��.L�,����h�b�>(LW����/�jFX_����P�)��0	-e�ug:l���x�N��xB��,���;e�(�]&��F�/�f{��#��d�ta���Q0��n1/T���t��W�H;��Qx��Li��cz��D�e9�
EK)���=���st�\n�3���F�o�~o�P��l^�_�YD���,hw�?f�j�����qd��a^��\�$c�p���S��T?t�l�5�Q���^�V�����U���9���%wL��R����NEm^�*n��������8�����������JY������W���S�-�H;��2B|�����E1�P.����>����wg/~��L��~�EK�c~�Jhu�2��Cb��ciV���5#��i�$���}q�������9{c��s	���|L��:)�K��G���OP
�o1���
�b��a���I�$`9��_'�~�=&eso��X'2����L��A:o��qJ���:��;{}��C������+�0,�_v��i��c�UH���������9w������������f�@�.h�t�1���>����%�C�h�n����.����l2I��Z�q@��y����W����z3������v6}����;��� RaB�Aq��|��Im��ub�� �&�H��H�
�5�@�S_HJ1��JI�cI_���cg���z���������?BV�'1�g��m)Q�mi���Us��_�z���A�f�"����$�R��S[*��/��R��9���pR�}.1U�R���A���hh�H�j<G�4�����r���L7���-V�8������>��3
@
�y�r������������d^s�-&`�
8=Wn�'��c�
�����7��zX�Ms}���������`]K��y������O?�����peT�CvV{�-"�����]���*R��82����"*P���3��U�����%������S��*V�j��+���GBQ�v�|����.�j�����G��3L���p���ah�{?���������I4��PdP><�o��=)����`�}�)�����e���Jh;�����������X%Ov7��i��}2�qHC��m:���&dis��m���F���B�����*7Sw#�B)s��<�����K���y*K'�Sq:i��0>����n�t��t�O'�d5|1]�E��zNR��7�.)g��;i(�4���Ql<�}�F�0��g�=��V���U��wQ4�n������3�`�)���;ea�,�rW�!Jk���i��������m�6r�W6W�p�
���������F^F+��	9�(����K���
��o�������K�n���������?/�n+,)^o��
��;��2r���Bh��rs
i5�����yWd�|�@��r]���9�LXn����r���c��~�/E���+�Z�L{�X3��mt��������q����tE�'�7�(hS�L|�rZW
Wh�������_��T�y*������w��
��7XL~���/��A�HG��)�E>��(�nA�#�f�Lvu�-������T7*���s�8���J@���m�e��(%�)�;m����e�nd"*��\+�4����
o�v�����E�HAX���;��-��QN���N���tB al.%���Xnzwo�����-����O>�$���x���4V��kJ��y���P9�>7������@��-3�:E��s�O��	���>��kM�w^`_�9�
��^���S�lSaU��u�#�[��HFPw��;ks[{�5�.�6��?�+�;���-X��M����l@�����Y1��h���PE�h����6B�r�%N����"��o�����r��Zj�Qe������+#�Z������l����
sn�v����>�uF�����}.����Tq0,e��T����LTP��c���j��<�Z����nll��������������p�TZH����
)������-i��x���G��q�G��0@_I���O��f�.��r�i�q���`��ur$�h��:���SbA8F��A� ����s�e2��/S?5�2}���Ad��s*���������Q��������{S���bD����8����Q��z8	�Yp(�y2�<�@����)����(��i�|K�Y��g�������[�k}��R�����pp��8�e�����R	�X��jo\��������!:��@5�{_���P+
u
�������6B�"H��1��f��Z?#)��a��:Xg��L��	#�;#�})|9�y���B�M�nm���Sym�=t�z�<��1v����'07�6�G�}{�Q4����*��(6
��<����b��s���s��{��C�.�"h,���0�!���u��_�����}.r�����\��G�oaj�uD�Wew�&���@RSSJk�����Cd��|�R���\FK%{��X�}cy�jw�\f���P9����g}������l���8*W�J�#������T����@�3�����L`W�l�1+0K	f�+�2�$69��^����l��2�����n�y�?�bs���,�j
}�Nt���t�9��bN_\
�:���j��]��R�<���-��H���� t&>����Qr�;}�������;�b�B�����T.�(��i�+1�=����1�����w#r})%.[-�L6��'�g���h��a��koO��/���_`l�����"i7�$+�v�������p�v/�1�$I��2J�$�v��I���*R������J���
�
g��2�	xP�>�	�'�IB�������xBa���b�f��_�=h:p���\��B�	xRM�h���_]���S�,+��'T�IN�����#��~[���F���� U~�q���O	���=�~w�b6�jC�w�
Lq2�T$J�njJ�bj0��/G�6*"r�,�k�5��*���l��c�"��(�La���e�����g"��o��Q�7�(7:��b8"���K��������D/��,�X�e���h|�J;@��*���?��9������p���H%f+�/w����P����9��LLQ�r�7a��s�>��
�z<L�5-
�LX����b� V��|rz��W�C{dE.�bH�3�a4n�Tn��F�U����
r��!_P.�RR���l��+��#���O;P��h�4���z��8��uK`5�A��D��1�����I��C��
�|?�[I�`������r������G)�����n,��b��A���w�{��F�������n�w������������aB����>���Zmx�������Y���V4�&],���\6v��A?Q8��]�B���	����w���A�����!_V 
�IK�`�9��L���Z�V�t��o�Vv�u���sQ��Q��eU���2��������h��Gi�������
��W�J��u%����* �Y@���P������Z�+��9I��������'�
|@����Z��V^����g�@F�n��e�8�P'�a&�k��|�r�AtA�Rs'*#�tr������-!���i7Fzh,����
�����)&�����(����Q�Q
c�N���?�O9N��v��y��$��zC��������b]x#�s�d	hG��*Xn)�}- CzE��my�yO��y�,�Bi���^�7����
"{����E�di�-(�����t�~���u�����~�76��f��[$���nQ+��G�i������X���
K�"?�����QLT=�,��T�s�Cg�=8C0P��M�����w���%�G�dyo��v���N��!���
nA�c�\g��3�r�\������J���{Hh��q:oX���A:�ag���ys����p�7**����7�<��y����I���e��Z�$���qn^��B�db?��
d*��D�I�7B����v��U��5���Q��\�����$
.�cK�%�����Wd"�C5D*��k����.����(��z����U1;oONNdY��e��u �MQ_+'7�C�jS�jNj](�F����\_�n��j�E�@���&�N)�r���������<��~d�y*��R��^2�8�Px����T���.�H�,]�##u
!����%���3��!��k<n���X�CXfI�q��`(��Sm�N�`���t�<�.��1��)C,�.�����[N��`D�,���P���F��x�o��ds��������.�[Y���\�
��in�wU},�iL����6���)^+10��Z-dG��zz���A��.M{��`�d��B�N���� �^������]Ux���j��p��g�e�B�/y����b������[Bz�-�d�D\�	����5����������(&�<l�B	\#��u���n��8�qQ���s$Q�����3Q�)s@�������3X��4���TnBu�U5����IiW �Bn����h�
�c�ISx�����"�&�!t�d���f}rY��l�7�E4_Vo���^�"�Fs?k�q���TK	�&i��X
�Q�����d,l�{ck$n_b��D���w�����dD��j<�^)Z�P5���K1iS�YB�db2R���
<7��Vl��w������X�g�+��Tv9�V������~��t\$2�����e���������G�Y��!��9w�`Z�D����cK��u�\7*�C�!\QY��Za�ax8n�,�����h����@oe�(��Nk�bI'��mK?b�'����0J������uc�^���:u=f�R5vi
���c"��-?u�Y�U~@������H*��|"�!(�M
��n+�f�����Q���`���S
�{Nm�K&[K�A���~��9>��zG6����{����&��OND"0�m�i�$���{?�S$1��*�]��6D��������������������]M�H!�Q�;O��<is�K��zm�#�K�(��������c#���	�O0��w�?1g�����g��?��|��T>�����r������}bI��u�����`q����u���iA��Who�p�n���� ��u�={�����������b_�^m��+�Z
���q�0������P
u~�E�=v���co!�r��2���S�/�D�WTx)������o�����\~,�'���
`I�������|����Rk�~��t@��F��y�16y��]���5%\3���E"z����FD4t�����
v"�cd�N�����/���[1�����3���w��5e���j'��WI��eaFY���l�
�R��\%��h`�%�AS&9<�I�np���a����I�v�/G���h���J:{#�}�O���gg��H�T��q=d�K�]��aeg��#�V@���YgS(���\��($+���e���!z��;�\-K��������u�b�����3*�|�b��G�9;��f\@)U�Z�dK��X��=D=�JK�z�����{��P�N���'���$*�z��Wh&����DO��.M��-�R�}�A��Y/�o=L���4��n��A���Eu��w�e�H��4S#����mI�b'?��e��_������	����F�9S����������GS���+��dw�*t���.W���j]��j���)��u��>U�|���L��v�_I�KU�=H���]�
����}u��m�N#��<u_�~��;�P�;�����t8��{��n�X$Mo�N���LB)������)(����!��D�<��`����*�(\�(��-�.��������1�-���Q��-Z�
}��2$����]��P=>��+�d������������/�'���R��%+�����z����A�X�k,YbHf��bx�o���������f���([�[&�,1v�^4�j�eTv���	�0�wi`�s�_��3K[w�xGN�%�)Q6�;���3��}Q"�F��TvR�@>�)�����
�,0�m�a'"�*��x��j����� �A/$���=�~���R^�x{�J9R�1u�-z(�I�7��dd���v�&�5�&G��p*i���_��O@�.�h���&����3�02����O��H���7�����i�6�})�xH@5�r��s7;�^�!��C�i�4Z�W5���0��Q{��=��������`3�����K�DQ�3�Q������p�Z6�'��=�Tnp�|^q���p���"�A��Y�P��z�a49�w�e	u�h4&K�G����!�=CBu	#]^5�%�b��}�,"7H%�QH�6���b@	!S��n6������7[���Y�b(�����H|H�lK�w��+�8k-�`�K��`����(+]
cq�j���7Z?"������[���c�)o*�����"�0=%��QT���d[�f���pV����#��DO�7������Hr�9hcmT���U��:O9���jN����b��W���� ��#2�����{|f��*lDE_��,W�`��X��a���u���z�vn���� ��U)��#���=<g�}thV ���
R?I� 1��������W���g?�_�[/�?���������7��_����Er�Y1	��FK"�2����e�p�|�s��Lv���T�B\�o"���im��`g��gh��-Q�����+�f!�I�P��$��t69V4�I�
�I�r>�'�/��3����v�9�B��rh�\�WGg���}w�}��rV_-������z�k��L]S$Zn�}I�l��G�#0p��B��Ez�x�f�������o���-��e*T�QJ���w�!IM��%��dHm����*�1�+��M�������"�� ����el���m9m���W-�@,�����`�k�b���rtjor'��r�CQ��������__�{�����O���>�_��m����C��Mf��^/��sl|�!*)�e��t<��e��r���n�_��T���;5���L���D��E�z�M[��E�z��= -��T��]��s������g"����a0���s��K��0��&�������l ��g��2:�dc��S`����T������$Q[P��}�����n/?�""��3�����2��P���T�9���K�K��5^��XOJl�R����v-^�D�=�����S�7

� ��#�!��l#�3-��X�I�#�����K3R��u�^wR_���e������~y�G9��Y��4���������'t�FD��]���V�X����g���Q�@Z���Y�$��>�j'�ps��;���
���p�"�?<��hn&K�Yl�ts�K���sD��(��D�<� 
T����9R����jz���dY��n�k(HB0��$�]��f��G��Q��A�� G���0�W����j�&{VI��*���c�h�!h������I������!�z��?��Q"Z�(���QK�</�
"��U��1�,�yV+,J5Z�-���@��@��h����b��z%yMd�����b�����:����X� ���@�T�I��J�P�^�v�7(�E�l�/��:F�������XB�����H(a�6R�����I�2.���G��N8z$q����Z�`��L��U�M@����Yv�rp���w!�)�����+Bh����!/^;�W��q :��
,���O�3�[L=6�'��y%�5���*�-��TB��)�1�OU��<b�}��1�e+1I��T��x`�i�^NOe`�t�8�zC�	�����!%N��h��(q��F$JGI�����%N���kE��� ��izD������`)�K����'�Q�W�(�9���n�-��`�m�iK������=�d���+�v��T'��/�T��*��(#���]0[l
��v�6_P���,��?"�d�s-�����i��$���I��1�����8D6�cl�������E��T��9;lnY���e+�3�e����b��b:skQ���D�[�2F:gyg�r����(C�3�:���\��'�J���P:���j�n���Fk�{��}
f'������n_
����M�������i��i�����i��IM]�f�0.�[uO��4n�>
���
�DLOd�X��������#���S�Q�ls� �'���Q�*�����`�>�#�Y����K��v����`�����j���2h�XlKXv�������Sd�����������wj@[,�<<�#�1�Nk�{��cD���lH����&��B<�X��,V��^:.X��n�T���E�4���5��,E>�8���L�X�x�9�����#�}����O����(���t�oF8m���8���X���>h���f�� �6����W�%{W[��rSj	�M���"+���U�[�0Mb�@��XLa5d�:��K�p�1�`.D�O�z���#����jMX�jcZ�M}�p{m>���.h���yT����[���g�[bB5��\�AX��[Y�O,h��SDb� M1'X=��L5��!��9��b����/����.%����[���Q��e��;�W���U������K���:l��z�i������.n�i�L����������\���}�XKY+7bw�~Q+1m�0jx�:4Q{1���W���]��KG��-���q41����)�d��^��<�%#�=2q�B�v`�h��T�J���$c��},���#�������w^���\ru�5�d�(����!�	��^�����H���<�����#F��>w%���G>,����b��v)�����2��=�k�3g�\ �#��mX������"�`,71{K����1`�����Q�v���/KN�.>�^�A6��4>Y)� �p3r�D�Ld�;���qPE���#%d�s���M���'�@�
�7��r9�A������I�U�j���p���rOBF-���I�8%�q�>j�89�D<N��,���c����og�9�Ovu�!z}diz���"t#4�����W���2�=N�}CJ���t���}j������4w�W�.'3�dI��(�H���v������'�e���k�����(���+�O��Y�������mKR�zI���vL��M��)p���8��N�9
�	o�\K?VWX?���o1�����������g��<2�-�r����%�*�)�*������z��N�	�)�8b������\�!N	w�%���W]2����d"v$�@��&�p���=8�A��R~�?o���������S���X�R6q��l9��x����2���)DBF$�%H�����?}.}I��z1K�>}=}����T��q�����[$\��q��d�=����G~���x�o���-�&�X4M�>�)����K456��$��8��/�M������t����^z�����^�P?owq0hR�$K�[p�=hR(�&��4��������L�:aB�#�Ch�>������K���t��Bo�)#��w�B�����$����	��"-�/fg}Ca6	u�;\�MB`@�n'� ��R$��p��\�4<@
hUX���=�2z��7���%>��`})��F6�������K�EFM�� �N��V�vf�sL����.����d[$����}�E}�X�$�.��g����k�q#}�Hx��9e"����Fd�y�A��av<�3�{j�����G�����X�9H�%%��TsJ��>7x;����7_���2<���X�0�Hf�4���Rn��V+H<���f��'#���P��U��1,���}l^x��`OYC�L�8W4�3&�T��E],��tgH+�^���m���G�)K(��C��b��-�nm����U����%��������������D�>��b��7Ri��J���$���L�g�&�&�%!�~�aIb*�|�w:��|����B�5������nI����jR�k��9������"2�u�\ST�H�iyU���lU��f'j
|]L�q8�������@�d$��L���?F9����&��*��LjAK�G��e����e��Z���Q-hY���,[��X�0���+f���Kq��L'|n8�i_�������t���2]�����<�B�m/���e��H��W
�;^��wg"<��"���3�8p}
%��j�g����tp3	�������*���'k�	=F������bQT�V)~yRBD���4i~��H��t�Q��u��`j�"z%�!fI��R���S�}x���jY�W ��*����Z�p������HR��lj%�6he����Y"��#AD>�����=�����1&���B��1p|�VbCk
@���B�i��K?3f���(%��Wk����t(�H}y�@��&��-5wd������R};o��������j���~����b8
YE�*^S{�j|#C�K}A|��,���:l���Z���OPln�\���D��Ke��2�g�*7���~�U��hJJ
����8G�'�# *x��>��>4����bUca�ws� �����;^7�7o���kT���D2�*
d��4���=�#����@fW���6`������~����A�3:�PX��Lv$��B$
��|J����QI
�v6lW�tg�L�g?�w��wU�3weg������,C	`�'��`��b�(��#�������@��B�
�WU��JE����������'�W.dhb���J1�Sv���2@���b�E�PZ�YgMF�b�jc(yU��s�x�����QH��C��]�k�<�]m�"���~�s)z���c��(M}'r�n#��QZ�NZd��CIZ=H�A��}tZS�c:����8�T,��\�r���0��r^Y2YE[�d�I7�s���z]������8����D��}�Ll�!8L]���\2��P����H#���F~�y�Hp���R�qvrn��S�<�]b���D������������)%��H[Kc�����w��uIl���^,����n[��d�eZ!o���N���M�y)>�q�0s��	���'
{D�4vap6]�|�yVx�(��ceW��~�D�D�o
*�_��J���b(M���������cY/x�+��>��+ 6l�.dbR������k~g*�x��,�;R�}k�+`�
4�cg��NS*2�X�R�dZ��*�`?��e�N�p��'�VA���Yv+�'~ ������4=�U��c�
��Va��`�(������v��}�5�G-����� Nyg;_������Y��COXv(�]�W�=;aG����3
��9���K�q��\R�b����9����(�4��S`�&�O/UC�m����o����s�����J�c�72���`�y�Z
)%x!�S��[���kAn�*C=/�^�\cw�jK�k�A0����'�5y�.�F8�
5t��W����s��3z�}yY��r.����2�o���!IV;(>>F�-��	�b��_8�@o�C�r6L���p'3����a�\9��������3?8�+g���� ��UF�����<8d<#��~Gk
E��-����i�|Y�B��]�7$�T��B��\.^���>�����>L���TOb��,�������������*�!�<�?�y�A�Y���qS�@������7��Y;������+��El�6/;�a�{���>�|���y��,4�����Bm)�w���@~5��]b�����y��Z[f�������5a�0����0�\�Fi��j	�Y*i��,����a���i}�,9�l=��(p?��|�E��q�h��-�u����s�Q�~�G���+�F�Q��mg��T�������L�sq�1j�����6�.n%vcj(	Qg�������2Z������YF���R ����e���y/���XF�+���=�61�E�jn���u��7 �e���s/�5����i���2�=9�0�-#/���	��~���� 'y�aG����P�H�C�x0�:�DF���L��zJ8�������
aR�F\�>K��ZM6o�|Z�~u���`%�j^A'G{�d�Hen��Oe��T��P/g�f*�G1�(��w\����N�u�����ufO��8p������T��9�MY,����Yw����d��uW�}?���������Ldw��8���9�{��s����dw�����J��@Q�]�E������������:`1\��|(bs���]��v�j��$�Z���P�9�%�������������]e��������i���������,����ravn���s�\���������r�r�
y�d]p��v�T���2�!����b�V�:�)2Z���C��
�W���J����RW�����4W7�1W�{�l�	/���-Vk�IX��?�w�c���_�?;UlOB/bJ�v[������=W{^���p��M-E>��_@�)�����^>o�
���J]�;m
d��Y�GK����A9d1+�5 ��u�� V�C�*���V�rZ\��[m����2r-�G@���z��j��/}{9�JB�:���bD�K1^UR�	�f����'��X0���p�(�{�7�
o�yr
�	�e\�Z�W����5f]�k�������i\.���������G�#v��%�C�Q��9+�7��OlL-\���x]�2�l4L|�V;0_MEv���#/JX9}Z(�_5JRh����D5P}����n8 $�������M�K�t������BI##~��^���N��h���GLIX��k1S7������IT7sH
�C���M)�>^KS��}�(!�K=�����b���$������*�FWRX_%�0-�s��|���
���e8�?JO�{���-2��x�E�<0 ����L��J�K���
{���Ar��|�A���@���t@��M�'#����h�G�xh��Tw
/�H� po�51w��L�"�;���yG:_8��ZnM'9�wVx>���p��@K8g�2+<e
BFbF������2������N�;��t�cH���F]�$�P�l���:���\����h��6g2r���1��
�WI���$�_B���0;R��$�w��S��:�����;�d`�y�`����Bo
�Cf�<�����*���
��Bk����������^\*�i
9V?"��m���7M���<�A������-+jO��cA^�a���(P8��j�Bj%������#�)��P����o��E�\�Q�^k^2Qj8�����>~�Q�u<0(��+��2;�KZ�h���H�������k^������:�C�(d���������nP��� /��!+����,n/�3��3���
LD�l<��q�	��0<��������3��g�t�<K�P��gTv��p�������w������%���*�����z>Q��+�6��$#����@Y��
&IF)�U�n����� ����)"�j�l3�B���A.MS�+$��/F���2b�l���s�t����n��zz�}�"��2R�&�^�r�����`�@�a���jqM������R�N���w�r����d�C`Eg��P����#k�R?��@��. cl-��^��R��i9a�5�eo+Y�N��2�y����4
]P����+�C���y(�jI|�a��%(������l�}S.�p>|�D�u��z!A�\M�BX��'�jT����`�i}KN6V�y��S�mp������)��O�WM�`��WW<����dZ^�^���,��p�c��_O��c}ZK%����^Pm�O��nKSx`Q\��2z���t�^1^�cb-���"��E������T���h�~�S�D\�M���H.*��LJt�V���x���\�L�������V�Q�M~j�@��	����S9��}��/�*���1zfT;�v�f���g���NG�O��������6�`5�NL��T_?�LKA�x+�g_�4�.|t���1���3�]�C:k���x���r�d���|~�"�{�Y�m{��s~{����M�`e�9�M���`�P��KTs���.�k��)�;�f�:����z�����V�n�N�	����5J�SO�5'���2L3��
>36zS������%�����>��������W�
����B1���S��1)/u��_�(��D~����?^���(�7����J��Q._����=F�������~�'g�z9.Ol�Y�k���x.��O��r"y�����ys�H[Z�W�b����/�T��c��H2���Agl�m�z��� W��0��%����R��3�E���d$�%���$|R����w$w��^��`����Y]-���z��2�/���V���z����e��U?�~z�@���rn5����R��ySL+%!�+��,��\�����=�p���wo^��F!��6^�3��_�?�������O��}z}��W����o=�I����x�HDRZ#0>O���J����Z�����c��	�s"��u���h6��5��2�b�]�Nv�u��p��pg/pr@&�&�wv�f'Mpo7����X�q���UGI��{g�6�f��C�E�����Sm|-_P��<s�v�[~aw!C�]�q�
`F�@����h��L'#a�
���>L4��h'�����K�5S�|�l�������=d���
F�\4��c��1�����!$�l��W�����X�"V5��z6���9��Q�������D�H��T#�9�+%"��!0Y��d�������k�m��#pPX�`�}��m>����`w��S��}�(�U1�2����;�����*�mK����y��|���I�;.\Q�:�|ky`�^�x�d��sZR�aceE��Wa�.8L���%���%������P;h��ZV��-"!���^H�*�a;_��BZ�Q������'*�H�}���L�}q�p7fL���5�l�|-����=��c��L����YC�#d��BA��\h�kr7/f��xA��(���u-SA�����zR�S���Xr�gZ��/�x;L�INA��eYH=K� �����E57��`�Y�J��bt�t�����^'i!a@2!z�f�i	Z0�$p6����~h��x2Bb6�P|��	���|9`i^�Ty�J�V3��&���<��S:������������'�
9��GH�9���'�+�W-����Mkgw���o�]�p �6�D']���@�m|����z���%&b�u�?d��!5!������Z3[N6����]�3%@�I��@d�������B4C������`C!=Q4�jb�������zI�G��������nH�T�	�b	���O��3�S���Y���^�MK�{�l�D�|QMSt�����K�At`3 ��8��
�@���0C�y������]�%�p����WP
�$��6�1��T�[,�K�����^c ���7��P�����E����ES��S���������K������*��$`Hb�=tfq|�^,��;F>t
Z����n%|K���g@%Qh���������{��d�"���`���������>%%�_���n,qB��2�O�H3��{�y���z�o���C�l.xE�c`����aF��D����at���\��0Gf��:���d��*j����g���:$$�H����)��6F�{Q�nKD�6���@�A)�2R��u�(�oZ�o8������<����Y�%��b�5�^����w3��Q0n��1�<]-�25}���Z�J�E���P�>�X��y����q���`�P�=0O<[�D�i�h��h{�"G�f�rY/��Y�1��Q�z��z�6,�-F���D����N!x7�I)"QrD[�'/x���V+m$������b��7H��H�f��AQ��~�M^@���a�h�����c�����CQ��i��/m��"��w�Lk>�[�i��~T[E�B��,�+O�k�1�$��
�?��������)��Q�<i�lK�J��R;�c�cLj�G���X]���v���b���N�'��[�}������i�����$�
��O��\�b��--{�}_�����DSfZ�n��dA��;Z�nu���T(��K`�@�x[:�q_�Y#O��n��Y'�P���f�n�I%l��ld�j���3��� ��B&h�"]��xpe.E�Ej�hi���E�ky�
T�3x�t��k
*R�
�!r9�k�q�[�@��*�th���,�O���Ey�����:�E���f%���O�`F�J��2bZd"8��XL�F��R�L2��R���]��=,�����h�����QI8l}���2UtA��*��[�R��ZjQ�!�SOn{���us}G��g-�	[<��w�~�e��1sY�)�:%������w&��j��b�#Y�L�^03�����8��`Y��\kGNctQ(����YW�M�(����t�@[�ZC7��Ed�Y�)WI^-�[���~X��g���;.���o8na��n�\���w�o�������_v�%�Q��^�Z&K������Hf���?*�e\p�;����>v�u�)���������;���_5����������=��Xz�G{]L/ou��y'�m����XC�(=����[��g��������+:�J�v %�2���L|��r���6����|���M�����)�RG�KY��O�O��#:)�����aA�^�S�phQ����~	�Z�)���'�{���T�@�1�X��Hc_
O�zW-����Pe*V	�#��ow����������+|��\]�������;�g�>��nTr�J�W�VA��<�!��{l`�h���"1b��v�z'�Tq�����Uj��rK����O�:�#`����&GK�|9&=<�]���8U�8�����Ye�|��n����@�������r9-��d�����_�r��$v/���}*A�5j�������=�X��N�'A��[���ZJV����d/QH�1/�;EF�jI��
�2^�q�;�,@�A{������hznk��C���J�(��Q}��{�����Z����v��������@�x�!��3��`�2u�!�e��'(l+W�s��.��&P�����9����U:�`�'lNt�AF�(���;�M����J?��"�H�|�(nP�t3�{^a�60��Z��a��i��J0�vSF���/j�8%|�$���[$4��dh^�#��`\{w�J��(�����~R�HN�-A
3@V��������	��\����W
�Y�l������4Z+��`���W����E�5�Lf��1m��!zQ�a ���g���������s�������?������������d�d�G����~>����P���W����w�_�����`��DW�[=kO�Yk�������J�Z�:�������z�ac�	�TD��0 �q���:���R�WD�t��z"�r=��\J�";��A�Hu�j[�����pN`;�Q��w�(-��0��o���!f"�����R|[5j���&Q����������(m�	�1����q�����2-c�=-��fZ\���wB��J(:1.+
���x����a��~��t�5����>��V�3,-@�`�$���F6� ��s����
�<c�:�$n}�_PHQUSY�f�b��0~� �hQ�i���g\%Q�%L����P��b�}8{3>�@�/�t]���k�@�^5����e��M�`�{���:!@�'�W�{G[}�UH��/_x�?���q�w������W�g�cq����/��B��j� �Wl�����~��b�7�h��tE�C�@Fe�����4'�[���h�16�`�0�1��BG`�>�W�G@����]GM��^���� �q��jI���"�����f�����n\s5E��T��� �r>�R����E|s�Xd���5�h�
�7 76��Z�I��oc��m@�\�J����:�d�Q,{�#l��<�#!	�@��
T��0�3h������'O����yO����t�!�v����::��������Z�H�����l�9D��\���B-}���R�j(K�	�P��`r����>8_�'\������F����,�<�KC!Q��UA�WD���������>N�F���r���C��ihH�VB��`�'��@�`G&^;oI���R����	�Us����`;W�%��d	�]�4�-*|��@<B<����t������\r�;�{!;N�yt��q���Un����}6�aI�	x&�%>++m���B�!�$�V��[�b�L�P����7+���uif��*#���4_�%4���_G���;�b�������� �"j	"j��j����Zg%���O6����$)�1���'�"
�=����#8_'
�d���A�e��>�@������P��_���]}j`pVuI�	s�V����g���2rxm�b��NL'��
�2�y�����g���e�����|�T�����ib���n#�$�H^6
)��>��0����ON6����� �,�4V�6S
5�{m:RI.���
|�������Unf��R�?�k���_
�N,�U���ji�<��T��$�h�(�K|�v���v��S��\%����P���b|~�o���~X��E��>��BT"�����<`��
4��"��-��9�]�����gvd���m�����X����@H5~��dRi��������ddPb7g���U�%{e-��������(��}�*�R�"�*�b��K1�?f�D���C!�P�*��������eV���q�m�"C���V���8��q�2�VV�w���!d;�Myj�m�\2�$��)�:�B�C��I�#���;"�oe�D�������A"a� 5�p��.��]��NY�"����9:���"	:{����0���v���K�|����4��8���X��Bf4	!MO�n!>�o���3@`&TOD����kX��~�����>���G�?tA���CF&����;H��Li��/�����ht����D�aH������2���^��T���
w�A�a$^x|�?B����44�c4� ���PoJ����R��N��A+{�B��A�<��a"����s\
RY7�1���^O2f���~	�.u��� ��p���$c�y6�#(�2	�������w�_�S.r,f,�h����_��?i�%X���	d[@H�(����I�9��l-��������d�M�:����J��"O��4�����z���L�p���U����L�ut��+����A�s5��A��c4�w��~n������������� ��qL`����!��.XRP�;)C�?�����v[,A�i(��9H��g�xT�=�Gq����.d+Z>�#����b�T����=��{����j��w4���d~��0L�+�B��y�-"�	Ev�kD"WSn�a�Z�V�Q�"��+/#G1�h#�9FI�J���V����$�PP�����2�^gR����:�.���:�����H/���G�u���nv�r���3�L}�&������z���/��X�Ux=u�Ud����eR�3{��|���c�V�0r��=�5]�^$\������r2��I�������t�Y
��K3��?C�\5���	�TP���t��OG��O3����A����:�~�<��;�z���^0���GF�7E��}F���tY(������Q�E�!5q;�J�zxB�
<����z�l[u(c	����=�L����TI���B	!G������.cN��}�S�PEM;q5n�6���g�2��{���FF������Wda��|�s�����n�/Rg%��xg	m���������<�������	��9��K$$�P���}���a~����G:�q�s]`����9�(jJ	gV��i!��9JF���w\�����eQ���I���J������OV,m�e�_�:T.5�ji�M���z~E��-�y����5�Xp��Yt��a ��a�Rb4��_I B:�b���������h�
��P�O550��/Fsw?3��x��4(r�(�=��9��N{�u�C���������m����'&����P�v���^�&�������@D�)�K��DX�`�`���\��|����62*Zn�s��@�]�K�#fd��31���k�Y���^���{9�8���fHN"����0�L�iY� ���1�w��!�	8�v��UD�K���p��%NRT��o&� &�^�K����E���k���i�9�k�e9D�H�<6�O5����M'��LLi�-����k:@F��F��EK;�K@�rf
�����{QDd�{�-�����V�������$������A�qL�x��O������O�(5`�9%{����R��=�7�;*F�j���`;�a�W���/e���U1�V"�Q:��|��Z���e��Q$����xl���F����dZ��'�?��z�p�/DIyQ����Jq�����#��#�T[���L�H�,�dT����M�Q����tk����2]E�� T�2���L8\�N�5����6�e��A���a/JH*��M��t��|
ep�i�G������w/����?�r,<��Vn�)���t2���R�u�`�Y�Q���0��6py}�������{�%?{Ve�4L��`[j�E9V�]�Wp�|}�d�,D:� �f��W@��^k��L
�������Aif�"��'���I#��)�z}������J���\�I_^�����������w����R^F(�����D4%�=���.p����C ��x�F�^)
��k����i�
��Q�� bB;��{��A�`������<�uCcR������_ ��)�f��Q
�K���C��R[��*��K��W�\�Y�h�]��
��7�J�������K�`
O��W=<�����R=��6tx��a:8H��o�"l8h�2)��v��1���dR�mU&��0�y�h��u�L����IS���A2)�ce2)����
��""���=�0�!���o�G�=^Gr=��p��?���UfUD��jO�uaL�������
>�����q\?��9��f2bZ2����k�9=�3��TXlQ��1���{~W�N4u-]8�KU��l���z<����������~P�Y1YE�(���	,uj���#���4�I9F�:��'����(�����2��W�+�����p]NO�,)�*#���5Vb��!�������
^a�F���(N{Z(��}�Qzsn�{�e��#�'l����"4`!C+_������q��7��j(��j�2����ZO-�F8��0���B����I�Y��^FL��|�wu=_UJ���r�J�H�Y2�������E��K����H��������?�1!Y�Md�S�sEJ\%����i��v����k�I�!�WX��������Qf���#
s
�;�H%?�������>�PT�������F���N�4�+,��b��v�V�A�"�[�=�R�}12�.��(����}���H��n��!�LU4�� �*�R8�:�p�d���P5#���
�|�9����J������"�`���xh���9;�N��,"����@�=G
xV2�e<���{0������s��0MT+�}y�����J������C};���A87���0d�(9�Aj����k��!��5�R���:����R�������/hM�Y�.L�P�����u�J%5��\�\4���vF�o��q�XF*qd�Ec{A�v��~���,���'��Rns�M���z�gv3��'�h�k8�0�cSo���*�u&���%�P.����(�c�.��.�/�$8Q�T�u;�\������X���[
�p �TB��B�p�+��S��>K��&�6��j}���|"
��B4�xEve%��,�Z\�/x�bP/y�[gXW��`�&J�/?�x�JKQB�:�Ze���
 ��(�a�����1C'�������8��>)�+c��g�_'��k��0)-��8pc��Ix�|���4lH��g%�������2q=�+i
0�-��o��NS^�<��Gm#��:�F/�`p������FB=m�8�nFt2;%�������>y�������Jr%.[�GI8�:��N.�~'��=xm/�����O?�~� ������e�"�V�v8X��V-�����%BJ�d�����,_3FGw�[�a�e|%�=y�:����`����(G5�\������I��6r��cO^#�9��-ms������TS6�Yf��>;�D�5�2u��5��R��b4	2z��Rxs��������[���

$f�b�Qs�M\�=.�Y��jK��#"�������;$�^���N�K[h��`i��D�cg�)SF��[��+aD-!bo�J���dMv|
��FPS� MDF.���H-�y'�s�*&|�^�9�.(�h��)�����hc.��l��8I0N��mx	d?oR6�euA!��tU�%lF��/��q"II�����pG6����M]d�r�2�);������m:A��(�����9}��������Y�Y���b�s��e�]����*�1f+�I�=X�k��l�s�p�\�k�<�h6�1,>�^�];'��;N���%�v�wE�;���o�S4Em�.i����~K������-h�[`���[�;c/=8�%����~���LQK>u�Ev��K���ML�F�7�i ���*�������:��*�Yh���J�NYk� �����~�}��/�u��z]v�da��&;��q����1��9!����D4#^�L`'��dh��0}�(Q}�Y��Ob>�apx����@.��K�����������N6�����aT�����|fT�ts�>F�%C���:��8�Q��#2*�.��Qu�Y��

:�����*�3�s������u�����U4Q���jH6�DVV�%!���g�r�u;6��y5[�6
�~�������]��������p����lk��<'C1��6G��'�+�!,'Z�(Ke��j6P+�Zs���0�����[��u*&��x�P������  ����42z���M�^���d�6UD�D`�`#��X�������������L"����55��*���HPm�d
E����b3���d����)����^��"�a��,�:K�lh�1sp��S������gk�bg3�-�9f&�\W{�B��4�rp�K�v��q�LNdH���5�Tc`[�V������m�)�	���N1J46.9�T�y�?ak�"i4��	^N��}kY�S��.�6]����w C�J������R��M���M;'�I�������K�9�#��C
4��t��@,K:A]#lL
�7�-K:���@)���4��d�3�[�M���6�$��k�����D�@zL��Rl���v��b2Q�Es��{3���63y�;L�3<;5*=oq�EB9�B5�,o�1����W�
0m�f��.�FXg	�0��l��(,�������M|h-�����w1dg6Lg��'}�3k����Y�������C�l4�sd);��1?��������:o ����s���k�m?�]'�����,���uU��9���LYM�n��4F_%B�v!n�����`N|EW�A�%�U�F��#E����F��!F����]���I,hKbY)�$��%���N@�}�E���A#�H�V�&��
k�X�}})���9��[�!�nb�L����GF�K,�*,������eb�+]*��dp��-������{,�
����0Mc���
���:a4li�C�S���P�#��F
��Z/t*�G�:��lF�N��a�TYK�����bp|�t��>[�S����`�x
ed�*jz��[��)k�����_���y�������U�Q��(�D�������[�'�LI���U�~��h�M��d
�T�@x,rY
�<��6KJ�����5J�ApP�������a��B�It�����~S�����X
�X����?S
^ok�$�(��3B���,-��\f�����&�G_7M�\u�x�;@8SsN5�7���
���B*�����U���6�����!�K����l��21�'�=;�Z����������,�����')�A�\�:���wt��w��
�,p��C7��7�Q�G�(����,�8�yn#���
�Qw!5-eP�	x�2��.Qx�Vu
y�%xL���������9�l�l<Q�4\��?����X���p5����DQ�S�zD�u	z���z�����O��]'k'.�]J�����T��qS��AV����!�i`�����Y@X-����+�S5^U_7�Q�8eUia=�9�Mo��-��9_��VY)�+M�� ~��D
n,k�I����Sj�����CK8y����XT1��'�*DV;�|���������U?}����{L�|3{v8���n����ENw�|J��w>�~1�=H`fb�/�wI�cO7h	;=��J��*����N(^p��f���4����dRN���Z��K*������rV��J9�P�X�<2��:�.��~���~��X�K���wB�����f�V����u����^����T�z���K6$���
K��cI}H��#v���~��6ddQ,W�x=-L)e���K��K���D7k��Y���R�lH�.�w��t�4
�\�[��Q��k���������bu7�:��LT��+d�H�@G���(��HCA�8���,�G\�>��lv�;�{(�S�����'���a?��������(������o~=}���?����o~��N���n��#j�G�{=����9REQ4C�b_oz����kp��xf#���%������VQh�e��Hf�O�eK�Y���`-����z9���]L��9w�~���L�#��M�P�!3����Aa��J���a��T�t�%�}�/'�e&�4�>�4Fs��.�%&����#�~��S�o�y�<~-��K�;��L����|�
&��x�����Jx���d����a�����*����Uct��@F�]T�
�m/��q}�41�����~[�=�i�������lh&�n�}��QsY����c�x`o��e�o",Y�&�X�IH�|���n����p����k6��/�j��$$�Zf?I�7��*u�E��;���HB�q�a���+�9M<��6�U]���$��o����<kQY�Az�	���Z�r�����23	�=���(Xa��c��.����<W_(�MH�x�z�����Uu\R~�[K���e�=d^��\�[���t\���=���
&MC
�96H����Q�����V����l�� ������^�������{�
m���a�d��4f�rC_��/���v�:���#Z��BF$���]��z1H�b�\��T$�8�~��8�TxnRc5�Nq��t*�q��c���D��k��[���+�VR���V��+��p��[�)�<��`�`���c'N�O	p@��X��Q$Z���q\2�tC��M6�LRb��gp�5�=VLs+PRv��no�n	f@d�vF�+=�	�X����m)� ����Oo�������8R�:���'��4��t�����	�lJY�\��������S[�>(�����72��2��j���P �~��0Z�Z�k[�Z��lV�uG?=9�l=�E3������l`�#L.}����9y��L�~�2�Q��,N�����1a�c�H�}.%G��dFP��w2�����$������v�h���f64��?��g\u�������r:���3j�\dA2�Y�����n\]��nn��}o�������aK���s?������;�������R�>�� �����c���U����K�}�C��dW��E�G�f��]	r��M��i�+����]Q2Z�����r���U
������i��i=O��[���j�>|n��s��eA�Y�����TC��<(
ds���\^��B������q��PWj�p�N�y~�Fd���:�����#��1S���T0�B!�zVB����)/�S�\.�Tfe�W�wt��"]cq.�]�B]��rZ~�.*����@
,Ki����X���,%"5�v�������F��|*z-���B���2F�u
K�{u�'�w����d�'��5Y�������?���I=;R[9�'%|�����������|:���|t���O?���}!�U��=�2k���a�"�{�c��0X�+�wq�bA��z>A�[��Ew,3�1%��x0��>���/c�,�#�z��g����[�T�Y1uj���R>x�M���=��H�r�@���R���-d�F�M�8��� D��"[C��z���`�a�5o��'c�(��:�b5f��)���F�$��|����\��,�a���^��[�G���&9����A��,���)BMgT3�Q-^j�x�c�����+�����u���4r��M��t��MQ��N���6����Z�_>����d��N�v�H|��B.����Il�@�Z0�j1��0It*�B�����2c�g�<X��
I�Zl3��V�m-����-���-�+����2�+V�����a�!��=i�@�l?PWBx�t�������F�+]P��;a��B�rB5�����N�k�)�\�����Rd���!��w��$i��QI����*���C+�f���z�!5I3�U�z�4�pE�G~�B�:�����
�)i~��d:a�>�S&�����o�:e������w�����,�|��4�>��e]��{�������L����k��������B��U���Z�s���C3�����X��ob%%��Z���A��A?�,~"�Y�V�����:��fN�|�7[���5����
�Z>t<2�"�����"����zna����qR^��J$��C
�N69�r*	����(�`:Biq����3'7c9���@�=[���W��B�FN��<(F1W��VD�p�i������8�%��p���e���,�9�a�u�#j	���^/�2r���~��m�H����2�,��k�����2���5�������'��s�cm���
�OJ��(ZO,�=�Jr?��>�c��k�T�^5.)j���J���Xm:�uV�Q~�
�������%�4������^/J2� �t�t_BJ� �����X���)��R���d��/�5��.!@����|�������eRj�4�5d���b��o��6�5m�i{��9�MS����t~@�/v������9a��( ����N�C�O���-d�:�^��y%�;P�&-������B.Q$�b}�W����O(4h�^c�����*������Z^LK��"���9R�1��\:��+��#*}���,��H<J������}z6-���@�32pK'���4A����+Q2����&��|��t�>�n
��������p�4��\��k��n<�4�5!�:�\|g��Ly�7�����T��}�K�J�08L��m�y�ax��i��O"��N��O�����#��L(��P��"BPaM&%T�7�:����,�,����5#s����++�5����]����/ ����W�r2�	�����=R��HAE)J�c�V(�K{���D'm�����<���������/�d�P��(��#��_����yI�8:T�K��>O�q�k=U{��b)��%�8��i�5}	<PD2�{��1L��ZR��
,
B���(1STF��W����S)I�'���c���5������P����2��1%��@��.~�k�Y�*h c�q�}3���9x:���pJ������spjv��cl	��s��X\k���?�EZ��.\��s�qn;�������z������4S2?����\T@.'�hon�P�v�����$���r�����Vm���%�]�7~Q�'����]�f�y�f��2�����vJ���5�taZ��
��!����"#�p���x��N1����zi�$u��N�^5�z<^/e�p"�*�'nU�t��K�0���AW�
����sGux&6�0n���������zA��S*�~F�!�D�x>FY�����\�c8N�n�oYP"Oe�y��R��pW�r���l����X5J��\��wIxz�������,��t�j.ek�PXGw%�m���<�g�f�-�G'�V��3���ow�1�*'w�7
���?���i�9�;�&yx�����:���"��Y�o���(`���|��������>o�M���OR�c���e?uS���H���s�������6[�ol.��q����J��V��Q�2i��?c���(`��"3p=�1n�_�X���I�P�
��e��5�)�D��T��c��@5vE��j-�BES���s#�v���a�Ct|J�US�����^F}
/�j*z�U;~�����
,;u�J�qLI:v���)��0��o��a��}�2F��0���~�Q} e����Q�E��vR�F�4�h����(`=b�fF!���`)�^bL���
�0�c�*W�� ��P����(����h+d�'�a0���0/�5��6.�9j�E5Q%��`vbk�^#>�)Y������V�T�j�,���))QGFQ�\]�U�[m�S/�J]�K���dx���pOk*P^��c�a�4���������j^h�f�wE��U������S�
,���Q�����1�@��#��H���1����B�	�e�I�&������f�W�b�b����3d���H}��Z�?�l ��O5	�������,"E��)"����}BQ���}�(�%�u@
J��HL��B�=���z3��#'i�m���)��������sQ�����{O����VB�Bw���"�����]��|��/"G�"��\��0BG��}n����E��:}A�25���4PG���Q��LC����Y�������f��]�0�)��g �<�,�����j��%f��M%-:__��$�I��X|*H����piEn�_$t�����X������$'�.�\Q8�T�U�fx��g]��<Op��C7=�S���� �`��|��0!"�Q��l��.�U��"h>�oJu%x������o��R:��[���������h����f�tu,1��n��R�0V�Q-��IK
��b�W�10����5�+
�Ln�<>���>�/�G.����Nv���'�p�?����l�>�4���a��]5��?S�k��#���rMQ��l��g�)
���@���,���xQ(�����O�����KU�tm:W����I]6\X1V�u�B�:S���e��[%�e�k�:pB<��p��g��T�0�
�G��V���&f�R���P����+��U-��mJ�����Y�r��0R��r�\�<�t�Q�e�Q&V�rT���N�lB�`����
=l����1���rT��y*������k�:�(_���Z
[1 ��n�PYHJ"S1����]�g�����`�,5��p��e�n�gx��Q#�60����l�� *G_9���7�!����������\C��U�F@�D�rd���?��p��mz��7G�%48��������k�Vl�^k�lTm��@���z=���z��R1�BI.�K}D�J����i]�@�{�d�0f" ����8Lwv�r�=�8��\��D"���K�N���$���T��������m*�l��R�L_�V��o��U�U�Z�z��R���r��,��08�i-}$���2���,��z?=���r�~��3��5c���@8��2���h	��f�?�3���&3��2���y�o�����3��:���_�u�0�����[
p/ge�g�`H{	�~����i���0���g�R���%Vj�>��UW_�������h�]��5��q�"O�nF4f}X$���A���^�]��B��3�:��y� B�0������[Zul�.	�:�`���F���@��J" �����#C�a�{W��4�m?����"������#�	Mx�!����4���Na�N5��]|�����L����\�NB���B��'�9C���Z]��5d�$+�h�A�w����J������+))��e4Q�a�������N13�L��b3#����_?��QEE����Af�Y+�C��z
�qd?��0u{����"��'������(�a���~z�����b�l,��N�"����Z��^��
��X/t2&��8�5.#x����h��4�^QE������>�I�
e}�3���R+����

K��%��J� ���: �� ��S����|P�x���	F�<���@��d�a�_e�U�j�Dv�;�b&��#���z9g�M����8e��e��^T����9��� Q% �y�`����s$���fV��Q�����~��7��������-�#*���YQ��6���/���@&'K�CY������SX;4�kS��8��y��;B�=m�D�[�PceF����|]��X�*�+�������H�������E5���UssG��8�b�Z��EF5f�w:T�	����x��o|p=_7��{k\��"#��@�Up`k����_�j�<0Y�2�UG�xC3z4!]�nN�����������,�-n��%�D2j|!�E&K���hWWl�u����*�R���QM��H����L�tU_����F$��z7ws��O�BQ@^/ ������Nq.���]�2lu��%H��NF5`��>����i��O��R���0OV-�h"S��$|�&�\m&�xi��V��H�s����+*��?����X�`����xG��/��6��H�Q����+�-#�n�/qgF��B�Q�	�a�x�=,��||(	��]����D���>/LQeA��aQ"R�X��UE��>m���#8�}�
�I#�>i��8�����T��\*�Q�(���1n�����iVhu�=���kS����}���*fn;�<��PQ7���%
� �^��s�TD�t�0�P��Qs��/��C��s���[/V'�Hp@����

B��Q����	���~J&��~�Lr��"K���Lh���&�|(�V�c�/�`!�o��U����!�:��y(��A��~�� Wr��XD����wg��������>��3��������B.�H���~�������i��H�	�h��(J�o�������>�� ������A��"a��wH4l�@(�G qJ
�3L@*�����2c�3D,5����[	�Ye�}0D�P\VW}��/��~�h��(vV�����+�
�K���i�E:4��J�W~]Pz���E��&��X����d���n��_�H��3���S����%K,,����Z]��������o~=}���?����o~��N���Au>�9���N<�q�>}]����w�E�8"|o��oQ�h/�a�5Y�@*�=uv����R�)&������L��Vdt�I
��i5�����������4��/~
�����}D ��P�v~t����N9z�?>��]>���D�A�����N���Q�����|�k��G�����:�,���?�	b.�� �U����!����FN>���x�).����`����'v_�&���3,~^�R�����
��VI8kY7�t�����H�i�(��YU3�qHg���^��|�C� �.�w����9���/U�n v�Z����U��R�`�TT�����}��Cb�uB3�ah��1��S'��:r����G�����M��l\�����t��%=v��������u�� �v���1^���N�������G�W��t;j�D��r���W\H2�D� A��v��R����{k!�>�^_���;�u?�~v*���H,�������V�!��K0}�Lz�4�����#�99�r"������AH0���)��3��=��?��9������)EF�@+��Zsh��J��@~�0'
s�9�MU�2=Q�	�����'��s�u�R�(�����N�Us�a����j�����Lx����Ax&
��M2Q.�������h�14��S�8���)�`������aq\���fI����'���%`��-S�o�]&��R�4��E�Y��X(�R\d?Hv�m�,*Fq*.�/�D�`��qD5���U��$#F����A���. 	V)I�[A�d8����2r�J���`��g��l�3��l3�!���i���\HJ����E����uk�9�A�x���Vz�QK��.9�
��)F"V���B���
�-���$���B��x�w�yO�d�W�\|s�
�� ���y#��[�x�<>N�W#~��n�r����.�����pbl�q��98$�W3_����(��
���z���40��8���������h�^�We	Np�d��4��T�j:�k��L�N����\�}��X�gJI�I=#�(�����#��%�Z���2�5Ep�=-j����>Q���p ���% ��Z���i UGQi�ruN2�a((���9I��v�],��r.��b(	Is�Il=
9���5?(Q\Iq�Xo���2J(~>3=c��{&��
e-�~�i�
�A-��K��\e��"��h�Q8��aD�zygj^R�W~�+Fm����Y��wS�52����G�Ux����H������d���t
��#%���9�����U�(��>��~����N&B�����/�7��@�� K!;���^
7�*Q]W��1C�AF�d���X�������-x��Vt��Q��&X�����������M9\��5E����"A��xgOc������,P�)����h�!��d-C�w�u�m&�A�1N`0��[('s�����/�AM|�,����Y��B_�KNj<�!^V�
1j����� �U�����>IOV��]{%Ho�Q�g hv�]�����J�����"
\:�	i��C[������+j�d�4*�sLc�!���Wc5�>�"�"$HO��5��b&��V���//��
���&J�p���J������e�)�G
��P�!���\���^�������Mm>h~3T���S��a������������}�p�c2t����(����S���$�����Lp	�R/��f�G/0#���[���Lm����p
����"K�+~�0��Rst������v���`@��8�Q��n���6��������N��	���a@���"�v�D5v����M��FG\S��\b-/,�i"dQ�y��d��c��WCF��o��}q~Z�ap+�Lwh�{���Q$�����h(�`^^V6��[��r�[�����h�[�Sw!{�C.����K!�
��l�t!d�V�W��DG�k�����'#��%�[p�fK���)g�xAt��@e��c.���jj�:���������~�2��=RSl�L(������[EP�����
0���[���}e*P%���OG��>�~��}������t�������c)j��\D/�)����������{.���TV�����+����I���\�w��y��4����\��j
X�H�^wu����
:'���d���$�
���i#@]�@+R������-P�L`��J	%��u���eG�L�G�����^66���$N>p�1�L�K1���}[GZ���>�dw��w��E��xn?��5�_~��lX����t�~)��X���|{3����B)E;������Y���sD�������������b%���G��(���SY7��{�Y#g��������ea4���uR��b�jd\$	.r��-M����H�����b*�4��)�A���3��>�����X����9���w/s�g����K9���B2y?Z�}Di ��~9��������/��{���Z��d��WDC�gQ����^>��4�_x��>[A�z��������+�B��ha�p������,
�s
�-���@�a�Q`�Q���
��h%j[�K���C���o������C0������(i�{t��+*��2������~����3����FSQ��g	�d�)�(���1%Q������O��B.8�+����f ����+��#�[a�l�QX�wdl�w���;��w��o2�w�3{[��	-�
�>\W8�����,���o�S���hz�����w=�|�:���q�~��!�(���h�a/���2G��X����'�����;T1�)Xi%Hn��_
�{����~x�jyg8��|����`E��tsu>�X��e�V��=�Z�N�����9��������|�3P���OF����;�"g���q
�����^�h(�cDC���
���S�~�*_.�Y��j��s����"h���E�<F\+S�#�at9l>�|���_xW��C�abaB��q�d��u$>���Lx0r��@<���p�Wy��N�1����v�xdP������v����?�q\���V,������+&�u��=`4T�����l@�����G�I���e�voujx���r�
p��G���i��������,�D���}��\$�E���O�rUSD����c�L���=�^j��Ako�:���� lw��Ga��=X�t�q�����������!��(�4
��cYn��Zc��#HW�-
�tG�I�,�����r�8��g��i��5���&6;k� d�#t�F�h��l!��8��Z�n��S7_g���$c	�����,��e���doZ��x��M���0��'�}��{l�����Y	7<v����,l<���~�](�(��34to8wEC8(�L�$�8d��V�3�KQ�"s�=�T"��Je�S?E!�Q�8��������/� ����.Y/�g�iT��N��Iqg�"O�&B��|q-����Ov�g���'�.l�?��l8.������L�PD��!.nFP�C'��O��'�C�z!������2��5b'"B�^��c��|�.��r���+{!b������
��:��m��!�c�o�����;GI�����:c|(�u��cR��tk���mfP1�w�_�F?�P:�.M=��'x��O��P�,�.�&���h��h ~\�~],�Y��
~������F��cct�7Rj��������C�J��e=��t�(��+��BfY�]���A����F??�<���GF��H�{�f��������)�}&�z���Pn��)�Ep
:8ZCr����:f��<Vdh���������*o0q�
?��{'^��Zk���b��H���>����\^�����U���r���X�N�������&N�#MY��o�}u���l���
}.�7�G$�Qr2�c��vF�f$e���d������
`Q >}��4	T��e���eZ�z���22�'�]1�����\��ei
W��UG9F�G9)�[����3�������f�Y�#�'���?��y-;D��������C�J��9�3V&�wN$Nr�Rcl4���y
.y+0�3�06��PA���������D�u[u�0��BH:\���U����tO��	��SJ����}*0aBV�j
G�6�;�w���"B
�'�+��^�<=Dv�Ya�(hm�!��GJ��1��`�����19wz=��0�/��~�''���y��.K���\���N��xy@�P�.�X<KF&��T*���`(g�1�r@�W����\6Z���fr�g����3\����T4EJ��
�.D��3{�|�E�b����!�P����Tb;��sB�(I�������2�Q����t{<��L��]y���7�)N*�7����5���^���M���3��3���N��=�4���}N�[�9G�����'�v�
����Q�v��AA�2"	��6wT�������,�����'�#��6{F6����n��6��FC�����q_w������������=1qj�vy����5��6�8^<(�b�^��-k�_1R1�R)f2! >�e��i�H�2O�U�Z�;�L�K!�Y�b6����}���Wv	�_	_cp�t
�(�B���N0�����3�S���q� ���Hv�}�=r�� W1�Y�'%����KU���u�����Io|�8�e��8*��/���T�������o
����SSd�&��h9��%������T��_�S����b�=�e
�45�5�M5�H�{_$'{%I2���|���j�K�S0L�M��);��,3#N13��	�bFK�s��Za��X����qg.�nG����Y��4��(��t���0����3��n��i��9���;�~���������Z{4.���3��]Q1.LKL��C��~������=�|�D���Fv���b��LZ����V��WB���o/>��3� ���!��^M3(
����_���/����wB��o'�<�bq��+�vAP�~��$���b������I]�q���j��>)�5I}=:d��V�ly����'�Wd/y�����:�k�����0*G~�v�����v�����RU�be���k��G=G7
��"��Xjm	{��u��A>a!���od��
{����\��������q��&0W��6b�}�kJ�S��A$Cm�a6�e����T$�%C�I��4�]*��Y3���UJ�����C�Duo�Lt�O7M�n�A�)����i2�42�qS��67���f9~uQ�o����u;�������{�2��������<q��~���X\�_^������bU���HJ�������$����������/�T�������W?���,���U�\C���c��;)�
 �GRD�� ���RT��C�p��I�������\�X����2"���������r�)T?	���^PmI
)�F�~)�S����7����J�E��`����Y]-���z��2�/���
8���~Y����^��^�����@��>��6��~���Vj����&aQBYKa�bdJ�y{��������4��!������������{��DXx�������Z�E�t�����d�*��%����Z��"�'�����T	Z��E����]�b�������|=��C������Kq���X=`�����
��x?������j��rr[,*q����j��dV��_��	�p�W��U9G8}q'�V'P|�\���������O�s��Y~��T����{QWM�E��L����cZ�K@�)�'5���i�������^�<�����n�=����E	.���>e;�1i��! �*��4�4#�'�}�1,_������o�`��4xz���_G�OxO?�e)��=�Jf�6���Y�����Qy �.�Ke�UK��W��������_,uM�K�����('�����1���j1�h�\a�)u�����Q��Bw���X,b=?������~s>:�����54���.7��pxBf5�e4Q��R)���#�bu����[��-���XV
���n��ZF+�����4?@fS�z���S����qj��������K��2z��i�+�O�[��x�P���E�(�'x~����P��1J�Cm(�����D|b�+�qb
o^,�j�K�b�[y������VI��*ze�6
���S��(n0({��)�W�����3������1^�f�d:v������Ph^�I"������i�uw_�q=]������2�x���s�V�
[�7�s!pj�^��n�[��%u�(�\��%�����@��2��nB*�>d�@��5��z��k�����K����X�����D	P������������p��7 ���B7IJ,u�9W��H���]��#�K�9�L"4h�8g
���:tRb��y�PQ;_�>������Q���z_@	m������]T�P�� �f��w���3��
�@\��4�#�SO�7@w��~����H����(��������TBe�Vi{V������+���G-��R*���eB����E�AK�.s��F@p�$t6�9h�9v���a�TJ�����u/=BV<�b��"F�v��
=������f=]�K�������4M1�O���J�=+-��j���=���`X�o���,�v����878��e���=d��3������w��a#����=�\�����Q��w����
�w�>K(�q�T�)o��j��we��f}���y�[�]���75�e)�������mI����mm�)M��kG����	Y�J�����`4�&�+
�Q�����|TUW7@tHz"��zgef����q�#��j�����/��|��a��v'�������G���L��������B��
�7��/./;Yry����������$.��>�j{���-bdQ/��C+��^�V~=~����Q/�1go�5�rhp��������w�E��nn���1��������aw>
��w �����p��}������L���[����E�Vw���������}���������I��}�] }��[������3������g_����)�:��e=d]�L�'���]G��P��w*M:�b��f=sZN�����e5A0\
t`#$��\�7���\��e?`$���D�8���qJb)�[3�/n>���pb�m��s�����qw;����p�v�_����H�d������tT���#���)~�L9���72�c�R���O������u�Q�{�������_��P�ad	J+��`��tIF��To����\�wN?�������
�����4����U���E�wh�>�'�����������K�j����_���\>k��u5�tV1i���g=
������u���(������<�<H�xg:7���q��
;~O�+2��*=��:g��q���B���?���-��V�����Id.+�/x&�{��R������{�g-�yV�����t��=��5�Fo~[.��'r`�c���%�mc�vgFwEv�[��.s��h����z+��_������&0s���Q�}���\�jw�r�{U:bs�_n���C?���:$o���'��B'F����i������+�8����N�'����V]s�����z��6�q�����b�#G5����+���v����{L
(�K8V"�l���-CU���v��������\��ct8�:e�Xue�+M����6�,��E���h�g��{����8QAq���J��j���E�
^�G
4����o��C�9m�������V�^��)-+V�JS��3�)d�TS4L4X��6�"�P�A����o��6��3�5��HB��Q7�Z#���T:B�;�:� `{�w��{Oz�Sh����No�=�?��� 56�Y�Es"[�Xs�Nw���	�����UO��v&e�#b��������>�4�p�XO`�����M+=��\w��5uc{����6��@�Ru�cI1?4��R�xjk>6�<���C���7+���c��_��?�kv�{f�g�T�w�i�������=q�e�i�����NQ�W������d]�{�tp�u�cw>�.��
����l{�����A�8�m��yxKs��U�j��}���G��}���Q���Cc���q��3��F�-P�I�<�����*��ph��/�z�CtR}{-KD�Z{:*���P5=jGG����tKn���������i@��M���M����
���H}O�^3.��3���Z������B�����p���6�����[%�.�T�������������9�gw����A��_[����t������=��{X�q���jzU"��r�K�q�u�V�������[���z�t]� �q�)s���+��[���m�?R�Z���a6�
�)��0U��5r*�QE���PG���f"o�C���z���o
���x���������(�Rz&�@g{a(B�k��
���B�y������w�>��d���C�����{�Z��]cdR"���$�6��d��
aH�H�����z�:�di����Z���i_���}_9�G���.�K��[��!�����>�&�p6
�aCK���&a%lI@�vl$v���g�=z�BE����p���Uv1���6��}� ��g��z��^�4;�ls����:w:�6s�,�B[�(��� ���R>m������(��e(�#Q}����S]Oa�����s����5��B��H�F�Oc����A�����D��b��B��x��Jh(�hK�5�?���R�V�
�����p��!����h-��,��	����H���5���SX������O���0��h���ep��{��Bj������Xp�������9&���"�"���6f
��&���L�G1���ho�41��Wn�/�}��U�A�rvVMu��%������K\���,s$L���D�����
�7!9$����r�j�[����D���&��(���hw1�[�@s��p�^���0d�v��If�G�sr\�����h���z*�29����l��o�v�Qo7��!���@aX#Y�?���d���Z�\�[�z�R��� `Ga�m��,g���^�Gx���_%����O��o�)��
j!�-R~L�r1x����;��#Wo�J:[����t� c��:vY�mw�����{���?��5��V	�����:M�i��qw%��iZh���j�:���aJa�����cTN����Y���Y�*�c��&������.f*��#�an<��1�����=o������/�T5_O8J��e���jL
!��N ����OXiNY������A��Tle������1lI�.�fDG�@�~naz���G������������p<2H1�!@��6�"� Z�]���$�����Jf/���>��L���]+���:]��1����v�]b(�c��?�~&��*�=�M������fy1��.��,���G�.�|�UC���C@x��W�5�Q��a.
�:���\6�EW�vC�
�}#��Kj��M�����X�x��7vq!�eX<�.4�|�Nq�	���
���ZH��O�1<y�a��l��G��-y]@����Jo�������%V��k�D8'�E�]o�z�GT��(WYs}QOK���ji�q�x�a�
1|�P�������\:	.� �%�i�w����.R�
2� ���{��!U���Z?iQ
�`�
�`���y.$������V9O7$R��h�O����w���5"�$.0����F��3����@��UU.��DK��P�1��)��C_�q���O-�#��z*�Dr���>��
��}�?D\�����������6���pO0 ��n��9��r�u?F�����+�{��G{�~�K2�2�������KPU��7��)����W����� ���F�*������V��s^l�;L"�Sa���K����]�W���D������z�;B� "{����E���J�#�nT�Zi��-
&��^�o�DV�c�UJBz��e��zXD�3���B:�G���-0h���L�u�����
�h0tr�rC�Q#�X�q�6]l�KE��������p�Q	�EjQ�.p�0����Y>�l8�C���
������v���Vt�Qu�d��<^�z��C��W���+��
�zQ�m�Q����xR�/B�:�Cxd^���`�v�bbh�t')�:�k1~
_���GU��+e��K{zPl�~��]"�:F��(g%��B��g?v����i�z�c�~^�Px�B�u
�:��lJD�^�)�d���3���M�"F�S�������&������z5�TM�z���I�]68<����K4�F��T�c��k�@4���5���L�_�
�X�������V���l@�O8�9S_���N�����q�Q�����?��n)��XPE.�����
��of���1��i�z���L����s�R9\'t��$J"�3s�J��M4Q${�F���I6d��������h	��K�/A��eJ���j������3���|����T�W�MD������������EY����zt���;u�83����C/������$�����-��3"N��������lD&�u����'������}�",��F*�t�*�|��!�O�?��2��(��&��K��Qe�C��T��w��
��NVf&��(�e�r?���S|�b��t�:,����d�K�oMt9��L���V������Y
V-�\a@�l1����������T&��\�_��|\a��kk���M�AS��)l�t���s��i�^��	f�vY�Z��y���l�.�I��8����5����m-�Ry4��y��?��-
��#���5SG������`��@�MlK�1}s@w�c`dv]S��9��Y��2��t��p,�3�����ErZ�:�2��(-�v������"�����t�'�zRO��P����Eu�=4�gYJ���W����&��vk�G�]�B�~3-W�����6�Jb�bwv�n������B�)���j��X����	����iNVN����R,v�/�p������Gt�4�#W��s��z����A�;��������~�FQ�������C���e!�K���\���R��<5��G��i�����0��75���M������>���/���2�)O����7OG��.=tvE���N���M��;Gl}.�g�1�`�PX�������mmq<����@��>�N����PO�QP��%�u����`�:�L�J�Z����}����Y"�o@�"$v���u/-]�=��}A������A}g�9�q�3'<tx�1����d���r����qtq���9o'����3�����>}����i��JuW�QT�;��V����������AE"!��'r�x���8Dn���BG���]B�a�J�7�k��HP5E�gJY���9�aR�0�����|��!�=P�IW�����N(�:�,��s�����P��!�c:������T��I��&VE3*R�K]��2��=BDKW���#��5�F�����}lp���|S�Q>���"sWq9iO,U8E��T	#>�$: U��n�&���CZ�/�$���h�qwX6hK\�ab�C��6��N1�A�7���}"��=(���j����&�9I0Of���Q�������9lF<`sV�:gA�x�L?QKs������h4�����{��B���=
S���bz.7�'M.�]��,:/id0�[���_M�3Y��M����).�?^d����������F���������\����}Z���uspMD��g�� �-C����{N�a~�dU���F��Um�����v���n�����F�b�������M�r��l��&;PN����D��YuQ��8t�V���^��v	L��\��]�8��&N��5{,��|:��k�V�l5��X)����}��F	��y������X�x���{�62�M��D�`2(~���J�E&����12�w�0T*�����)g����a��K�S�K�_s�&�P�j�Q�)9IWcw<�J���.3����c�T2�����C57u,`��z�nn�����`�)�b��v��s�9�Ox�:vPXP���zT�C�xa�����I�����n���3-�D�/%�
l�5���m�pa�?��B���R�"Q��S�-��V���7��u��	��V�
^C��	��j����Y��w}UO����	���+S���;|��Zv{��"K�K���x}���5��������A�x���b�+oYU�i���g�P�}0��{�������y
����h����f�V'^�`��l@�]�N��x�)E�������T<��-1�d����U�@�k�m#f��i�9�\
3XV�^�k�3��MdCjm_���j�w�Q�q�U�S_��:�qo���%�5Q�@���i�����gH��W�e���.o�/u���52�$���0o�!�565AW�(65	���[�hf���0Qx�d��TW[��a�l7��11u��(���~�J@���6��<�������|N}9�1�[{�<����z88���h�O��$��H���D�X�E��Y:���`Xn9�G|�r(�#M+-��R6 �|sU�������TQ�H�I+�O�\,w"�����k]6u	,��w�r��*��dg;��'�%�/g'�pB/F_�+T��03	�`!v�8�gJ�Q	�p��{u3�Ub<�4�w������{U�xLb!e�r����B-������J�sb�^O�Q�@����S+/T@���lH�m�������������TR��2�C��h����p�\x:�+���[ozU�g�j�@���\���%�a�<�#���������W��?��L�Pl����k0%_F�I$#C�h	�0I�!#����5��Q��V#�t�f+5|�Zy���c�'mys���'�x��Q�oH��vK9O���(���*N�J��L�<��D�]�*z���\���g���c�oV�z��%�z](-j����W�r�{��mk�Mv�[�n5$S ����M
����oKL����[;�FE�3@b����"N�@���^���/�oL�;�5zy�������1{�{�8���~�r�����'�N��e�:���&�K�,�7!���%��
H��-^:�F�B�Q����:�]��E������Z��!GRI�6��?��T���(����f����IQa6"��c	� a��3 "'^i
�6��|j��w�>y�d�99���	��F.w�����'M��[����H��e�.3}�����n��$8Q}��`d�<����l��V�E�\6��.��r������	���LT���-�
�{�����G';|���O��k�=T�a_��Us��\c�c��c�f�vUh�W3
)�7�YR����3V�os�=F���Kt�����A�%q�K��{��Bld(?����Mw6|��Lv
;YX��]`@%6<��J3;�������\O�~���L:HQ����n���u*�eO�������������b��:��������1
"��Ir9�|������8�?S!�]X�QB;r�ip(:�x\+�Gy�(����Q.$����	�R�E	bb,I��c�i�BR;��'"MQ�J9����=������s��~�����k���u��KC��4��>.��IlH��8�N~*:�&w��G}��"Y�A��������I#�h�S��+����1R�]O��d�h:���c��(?V
2;�*W�eilc�1m?M��!�j�/w=�x�>�M��y�� }����s��������7��2p�4>
Z(�b����
����!R���W28p8�$��y��Ic5�D�L+��	�)����^av��;Y>����vr1�8%oN��v�������ChaR�=���:;������z\����r\��s������>R���L���)Pv��0�����������X�c���r>z7�#��pz��v�s���:�->��^�:�;s?�b��������?�-���<�`n�;���� ?���
H�sh�PUs��yz!����&���8�� ,��������!�L��8�v��s`�H��H5�����G�m��Y��El������!���:;����0��wr�?L�v�x�?4�9\<�Q���!�2r7���e�d����Ly�����������w�x�/�zfBC���� �����F����4A���_��Yu��*�V!�Z�T��nx&�[T��0e��y2%�WV����l0N��hHQDM�	:����
GFIf�l��p7�qu��o0���F��r
�|�,�e��Y({�f��/�%}���~z,���TY�7va��&aY����W[Y�UJk�e!����K���Sy���R����c
��'OM����piYhf����u��*/����p��44M�T��y�Q�������Nx�����mY�N�4�����+)(����I�2N���0�yS�����!�I�hI��e����.\��j>��}6;��
� �E�d��/'{�Z}]�ei��!W�YD_�L�l'�p���qq����*.O��!y ,��P��
}�"�L�<,�����, ��,������'p�M�"���
�p]|�D�����s����<�/�*��>�
��06Y
l��.����[R+0�	kNq��FV`)#��V���j~#�v���w�	$
��c�P�%=�9�����F�i���(��	iz����<92��al<�&.���U:���[���M	��]3T������ ��[r�����V��QAmU�{��GX_/���^h�/f������K?���gVH��q�u'LL�	Hu4����o�UbD�)��(G��	��Tf�C�!�MQ��vSFd��x����i=�Lr��c�wj���v��Na��7+�����T�[���bu2n��u���vX�^��H��=�8u;W|�,Q��d�[�����������Z� �ka�������e�l�,Ku���
&dU5�U��4b�Q�.(�X'	q9Y)Ue�����Jy�#��7������~����7���A���uSrQ(NDB���P���V���*�#I�\�q]������P!��:TA�[6$[U��\��>Y�/�O���+��7��&�Q��N�������i�Y>^=��������J���M����|��������������[��okP~�S_)�nG��?����uD~H�(�������dF���t���Oo^ao�=�����-=��o~�q�����sd_7p�c,�5�����B(���!�K[�^jPJ��*�K`a��$������Me��R'<q��X�2�qTTj�����"��{���p��5Y.e#07����6T��%����A@����������lu��En_�j��6�v^�y����O/>��������8��������>�c.����LHXP��=�����5p@X�bD*��d����X\�T!��SxZ3�i������u�a:W�|�,�����8���L��zQ0@�������8�}c\.���@��UZ�+SJ��(�5���1���*odu�s?�%�������g}�;����vA�~~-���9B�	i1v�����o�M�0T�s
�P=[a��*S������\�w&d����*���E�3C��L#���Y &LyYy�m�]�(nR6`������a�|z��F��,$�`\�H�"����������A�z����j��A�	���{��c)qqQ�}�m@��/���=�`�n�q��g�����{����G�����r.Y�N������B6�������@F�`R�hV����=z������&�zs�S�_v%B���T�*���=B6����H��O�B�w�6����������8���'c�Oe�BP��*f%>}W(BG:_6K���Y}�L������O���}:����s���L��aw�>a�?�������9����$���n�0�$���5y�X
����i���O����U����"'���$��l$)d#�^j_L�o���F&�F|4*}E�sL��~�����5V/'�A{E�u�!MX71Z�>K�F�G
�i4-��7z����F����B7���4.�!����?��\����g��e�m��=[-����@/�\�F�S~��W��(h��6*�/u��q�����������7���eN�<�9qT�$q�m�|��������U���a5��	r!'og��m9�A��:�H6�X�����V���Q���]��9����M�\�f��D[%�v1���d,tDK�%�9w-����l��.$$�T������` I|������Xa@��j���I�$�DE��u�b&7�~���/��?	w{,YK��b�y�g?x��S}�\�^���g�O��������Z�L@w��1��:���������qH/R1L]@�H�����s+g3=���O<����Sv����������X`w%(�Z�FS�I4����S��p��TKG��
�-�,��z+��L���3�4���<#���Q���������kLEy���m���'��E���T�c�r�;�a:��Lz�,eOC�U�N�N)�����P�<+���<��QpaP��<Nw;�xJ�G��Lh��a�h�!�������i>#�\W�g6lkz?�=�7�2c������U����^���stea���U�������U��r��n`�J�H#m[�	���2	��41d
E7�f�;�A������%�g�!��1���=��^�'/\��bY���{��4�������F	��^#����);��.�e��Av���B^\�E,��Ebq�"�e�S��?�@�=����e=fR�[C|1����J�n�rk��<��
���e!�������p�Ml�����,��)v^63V����um�,b����o�nSw�Rp_�cy�z�@!SUD{��"Cl(��"�9�����T�BW)�DH�#@w����j��9���ZE�w�'���V_���^7p�WM=�N<�q������hV���V��e#��}���*�ID�/�Z.��p�]����`H����`�yu��61	���X��I0��mtIIP��y���
�(t��ryYy�FEC!�5�Z��d�kt������e��t��h�#�&m{m���{g���n�q����R6�*B��*��*E��=o����6t{�_�?)761Mm�������tZ��_�������/���n'�j���|�ZQ~]��+�R^�]l�6���F��V������f�����&����
��W�����K#������WbE�J�GW�M��l����Q��O�>����M�^5s��y���*�4!sz��nNL�K-z�a?�e]]��xC�,������P�5�R��x����Re�d��QL�Q<}]#�b]K�����p#`�	{����������d�e;�x^��SHy��9��A�II9��48��,>&���ED~Z��n}��O�Jmy����n���F WIp�g�O4�KW�2Z�@������l�_�������yR8(NQ"{4�P�D���PD��h� K����x<|2�����+�h�x��81U��7O�3<�7��D��
8\�Z5��U�p!��	�C �OU��8:���g���?*I�4�8j��r��X`��]*=chjf��[���;�E����Ib�rP�^<�1�VQ�C�TI�;o1�C��I'����t�R+C������ !�Y�������������f���WEb�����Y��oF!�<������mF:�����\7C�0u�'*�����Q���^�4
������	��l��8�"u��,��$�B���r>��j8�"�:8�"�4�l����W�&�if*�A�@���-9��}j�
�;&��A���u�[�Q�E&���8����������?��Y�L�7U���
�7�%����Jl������UK�H�����g2�f[zT�P3
�4�W������S���l"m��sEF��s�/����o��������oA��"�5�����P4��L���)O1�CZo���:��OH�>u�aFC=�.�y��G�I27H]��4�%���'�<���r&�?Y#J�� @�+�N��L�����K��V�W��V�5��/��J�rz��F�y$��%����J�+�������:B��xox����>uyS�;62Oc���=��$�x���1|����>��*�=�<�x3�h�c�[�YC(K{+����3�5�	[����/N,
|�y��<8OUV@
����YV)���p�n��Ex�7��B������t���
2V[F����7�.7����%�Fdi��v�k���6��65|�h��R�
��kB|����W��l%��rI��S��l���?�D���}���
E�{9��0�#�-�&��>i+L��������}g�B����y��K5�*/�����������_:2T�$�C:X�$zF^���@�����5��F��!}���?lw�aT�ew0Z��a����OM3��	�J�3�x8l���<���F�H_�f���tBSG�]�z�"��Dv��>��Sw��"+�^N'��$�R��QB��O4��{����s`�ee����{��e�;M�0���d9��Hr<��ST�f|�n��mV
���P�c���o[
zm�VeQ#
����B`(�GTUTo����w�
rI��]�
��;'���f�#�f:Q7�����?�������J�",�.�[1�������&���~�r�1]�Ib��'3J�;o4bT�����W�#�L|��H6�����qA{������������qa���s�v��r
J�2�0�@T%S) �`���I�+L�z�)�\�~�d��z��R�ezIH
(����/�G�w@��>�����}�u��u�j�������3JNo�h�*���V]7��X7��"��I.2s��B��23|Xf�4:;��p��'/��<`��x���s��;<�Nv����AeZ�Q~z���3{X�r��#-Fm</t��Ak�e��=��(�H]�h��[��������L+z���}�n([n�k�$�5����=�l2QH|'�E�Os�������d�fvD)',cmr"jjW�z��9��
pG�G"�e���_����~�\�����V��2v�+a��q�3L����� �*:����A����S/g��z� M%����p�H�u���(����r�����8��f+jE��}?2U�1��~��fu�E����D�@�
�R-S�r^]l6��f����n�Fk���D���[�]��Y��u��@������f�R������7�2?����L5�����^����1����J�x{8(��
Ef^�K����]d���Ya��n���r�x�����dO������Wa'�-��[��YK'�����m��N��{2���A/����bB��~�	�z�_�k�W���o'x���E����F��!��"�	a� ��)������g)�Z
����P�52�+�*`��M_����F�f�i��&��j?|�����b?���g�O�S�g|�D�%r[��=B�	��`L������g�Li� ��vg�F.�X�\���ak��?(�g��n����������4	e�����Hu����W,�+��-W�B-�[����#�j��U��{�q��J���c�%���W���{���W�j�6�$������wS,���]8�8�7u"��D�Kk���dy�!��K�����x���R=AA��{�	���B]�Fe4b�
En��Z�;f�YE�����.�X.h"UF
���T��->����iT�����KO2)R%t������/���,��)z�F��+��%��o�UG��N+����{��h:��N#�.��t��X7�H6��C{z�Qhs1�����K���b'���I�!����:�������pk�?�v�����x��b�X��[����ps�S
w�j�/��WRG�<�xg��<���0�$���DUN8K���� a�\w	�]C�A�SZ{����������kF��� A�h.����T�����h��?����r��0���q�FG��8��NrV[%�lVt����y���*�����3�'+���%+e�^���r^/����������N��;�)����cP$�J)������X��t4���y��)���%<����*���O���/Lo�^�l��(����]_/.E��

�a*L��/�Tl�{y�t��m�2�+S!���B*����
��2A"'�$�D��X.D����������6�6x�<C���*���g���
�%�g����fJ��,=�%�}�b��Wq�e�0���<�[�h���/W�}s���	�Nr1�����v-��"��#�A����:G��N1)ZfV����#J�u��us���G����I����g��Z7��aC����$5�@�F�2�@�N�_���������E��4��=�������������0-|�n�#�)�4�P���*�t�}�*|oY{)b�
{(�.~��O��/S�B_��j8?�&��it���[M���l!�q����"�����5;�>����,5t4=���1�{=��/,r�����U�d�+�`,X[O$u�a��d@(t�A!�!BtQ�X��B*�0F��.�=�4�;���a$�)a�M��X�U(S�f�L1�62I�X�0;�wf{�K�a�X����r���r\X��H��1(�](�;	��
I����R}�*�&C�1�s �6\����
�p%W=�����=��g\�X7�=FfeAU��l�D�^R4��@�	�\7��+�G����j��r������7zL��e���4&r���b<M71���.|`1��[=��1
l����R[�����;)r����;up�".��"��R�IxG����".��2(�%��g�O��I�U�%�P�a2�{:u�N���Zb����Y����U
���Jk	V�eu��\C������
 �De��!]
5�����ZH��v�y�`�#���"$��w��K��B����u���t���26�%N��]�~/����r���Zt�3�/�l��P��H��~�dW����R�TV3�g��
F�J��`Fl�����k�������Wg�_|�����w�E!&i���'2Y.q��s���s�_����[(���|�n+JR.x�GU���h����#��`���Q����r�P1�
�-|��t���8��;y�pC@����R�[�+����
5�E3K�[�y�B�|D�2{BY����21�)F�O�����#�58�1ace��+o���{�%�c0$?��Z�%U)�4:P=����9����S
9��������ZneG����(c��\�rS����������Z�[�;��d��[f ye���{��"�����z��l�TA��h�.Q�k��i��c�^��V�^��U
���yW��B�q]�x	��|APw�?���Hok������?�0�r�Q��v�O	��L���H��#�6�n�����s�~�i��[6�S�����,�����<6��b����f�s��_�@XP��u�c?��E�T����t�~o�:��KH��`�������K�11�F�<���#s�v��;fH��,�������1����rs5��>���aw�����7q� ���+�L-S����X���0{k�m�@�f���8�B�.�+S#tk=�D1�����
��d{�������f���q��V��O�
�0�����pc3Ap����d��������)l�.��,m��em�j+N����6�z�����)����:O��FSqg����T��f�������AD�Q"�Y��'�������D��tif�?���?x>���IF����C���(T)��+\i�z����4���>Z���@������r<Xq��.�u�t�%����`]��F������+TL�����4�%)������Hu{�����\�C�
r���}���yqJ�7�31��%�uU�6���`�����[/��lM��r��2���~&�������PN���^6�e8=��=�T'�\��O2&�jF�w���)'y��`���,+|L]����m�8U�4�r�p�}QN.z���
��o����p(|�C�w��f�,7���*3

[���l<~�Q,W������"�+���6%��F����/����Ud#f�SF������/���m���/	f�
�}d	�=�)Ra#t��F��Rl)�����-.f�����
�j3������%�Oc@�f^��m�y�c����;h��
$��h�>4pJ:N��Sy��=k���c4pd����A���������K��(�9����%[�;�
�{Wx4#OO����e������6���i�&B0A��T��$RS��/>�*#�dMB����=n�	Q"s�DI!�Q�����l�@$�u�Hi*$i6��o^\^je�1d<=U��[�����&m��B	�%���~����O��R6��vn�t����������,UMK6����e����5<�>�����8����e��'����l�
:*o(ct4��GIO�J�v-�X"����n[��
Xc>%��7mv4"���gb1��]G���������g��a��>Vh�Z`����/c�E��^8��U��L2�����l����U�����|���'�A���������8q;
qFe�����w������+��c�u����#��7X��.��S���\�txf���B�Z����y����.�S^8��+�l8��{j�����wC?�~��������;�q��Vq�������*����_H�x���m�k�������>�����W���h6���"-%N��4v��
������uy�;���"���qK#�
)({��&�Q�?���!}�T�A�6��~6K�b#����� tb_!�w���}��������M��WH�l�QB�F`Ti��]��bj�Q���-$:����G���O�:b�a���|�������E����5h"��,&-����e�o�*�I��XK�Y��!��w�����.#0������@w�	������1;�*sz5}-n��#V��@��H�9�L�[���$��b����D*���)�/!������7x}����P�O���)&W��V���l�B
u����|c�]�C����u���Y�Q�i�6���|8���^8����d��������Q�n��������Si������:;�bg����1����{���z���-=L�
�
g��9�������P�����$�4A���`�����Cu��m[_.�qi2��z���*��Y
���T7R$��yl77U�ip���@���#&�nR�c��}��_6��M�ZN�[���(�����r��Jj*P����X<��CF��7���12o�p�<W���V }�p�H�"x�g�sA�9w���W�8;���m��oD��3���X������_!p6Bb�/E�R���{�0y0bt���p\b}��pb�s����B�o'����] ���)���BU���vq"����x���
u��yq��!3/N����RK�Ll�f���������c�E�d����-�=n��dt�G����N%!a;�N���H�8���s��^r0�:R���$Y��.����W���^r������'�R(c��P�?��J�%��2J3?4X�2bN3I��ezt�cp��tT���e��;���T�����IV7W
h���� gq��Z�r�8Cz��&�8K�~>~<c���]US��(����s���6%�;-�y�X8�����Rg��~5��c�����#�)
ft������	�+��*�"���#��{����9'�O��7��-'�8�1��0yj�Y�Yp�B��x���N
0��������jO^H��������P��
��}��8�����O��^�����6>�m�o��h����"*C�#���r�@�Ev�*����o�C����}������g����`[��,���"{����f�������tA���L�����g`Q��+L���Z��'�JW�]Q�m���_w���X>w��z���,M�O�����$���x"���w��(�(�Mi��Q�:������`��h�����$����h�[o���;��v�tENHQ�c�mU����K�����}C�u��d
I��9_p�R��u������i-{8�c�%�ka�jH��hJ�v7��KI������3��9I�|����*���q��Iz%�����$��R��q���25:	e�}	:��|�"RL�U�+$�D��2��u���~}���5�K�M��k���F���\�4����L����G�&>M{}��.[�K�(*@V�b�f�����Ok��X�g
HF��ag
px7%q$�$BR'BK@����:�T���<������������ ��A��p�]xbqY�^^�u�&"��cz�fQ!�L�#�I��W�+:k��Nd�H0Qd�H���0%SV�D�����;|r�="Ib�="�\,w������$Iv�O&&]%I�WA��cM��u'�|O�1����PeM�K,�����C�=`�����q r��W��<��&g�w�����m��{#[Q����fF;�\��Z��8��H0���������3I��jS!��*W����'��1�f���f���������m���O)���~^a�\9����P��������H��z�f��1!ozU./{�@9��f��z�d��5yA�KB��U%v
�Nv^�b{SBl����v��^������)��"�*������a
�W���"����V�/,Tu���h�������
_�@�5�������*�d>�
�x|4�E	7���F!�����0�5�X+d:
,b��;�a
d��j/�n	1�P7UI�b���zM7�1���VN���r�y��4h�yU"��V_	���1 �!G���.|8o.A���`�����	�nC��������eK����f�m��P
�����1��LA��\��Uo�[�_`��]������9<�� ����v0`��1�Hc���S]~��d��5���]�(dAe�5)0J���
������9��������A���^����xCW�w�����-����}��r�q���=�(�������j6X�l�����_V���}�3�Y�!��Y^�3���9>�g?�7����{�h���e�T�;���sbq�a(���M��5fG ����UHc@�����%��D}�.��>ut��ES'�m����x�I����S6��������H1KPtH�/{��~����B�N:������wR��;���ddl�x�
�}��f������p��h��tyW��5�_T�x
S��~�Yx.�0��H�g�����y�h���L���al�BF�+][�k�����U\�"
����X3c&$�+�Sx��x���.�/�t{�:Y����P�X8��m`\��n�~��E�|n/aK��~T������i��/��g��lP����jMZ��:����L}W_�%'���PO9\<�����V�r��v��Avz�wb�)�&����XX���"�2
���Rt|�����p��Y5�5F������<[h�IC~��F��\
Ow���[	��V_�9?W�-<�V��U������l@U���6Fg������-�m�"�w�H�%f��W����Y3����?���f�^n6��5����VN���m�]��_WXAjOC��<���i�����3�]T�U�����7/^��x$����a�l�\���-�Z&pM�'[����~�^K����f��*�a1�@57����"m�����(��J]�]��]KNG3�A�#�P�dJ����F�T��2��E�'bu'+%���y��;�'��Rd+��>�46>#Z��M�������l�>o�x��Wi,t��Kg�G���2J����w�X��S5G���u��n��=��q$������>������~F��/>����������P�1M�q��&PD�|��ib!@���c���A���_��Px�m��.���d����D��a�G�����T=�x:��4��T�{��T\�������ry�&Ry���,��V��E`��X`��I����m>��4�Owvzo������4��
j4���B��q�
}\G7���2��D�D�f����D���������RA�Y�
h���k��Z�����9G<C+�B��U�5�"���G0p���k"})�9��D]s����j�[��!�����^���|����E�����P5�N"J�������4�����_�S%`���e}�L����jdE����5R/t#��Z���E#��Y�<�SN7g����:oo�\�����8�t���slV�MQ��/s�H�W�#C���*I�(���dG(IE.T��0���������X_��0v�KM���5%�3?�����b�u���Z��F V	4�No�������{)r��B+v���xH75*������,�����P��dG;_���d�l��%W�}dAl�����P<���c$m��
���Y���g����q�jEE_���v�������_�����]-���m������.)��pT���$�2.S����ibY�Ke\XX2����������(���%a���`����(�(��
�1b��H����t�����w��?�R���T+����B���V�� k��TG<�3A�'��1�I<�;>��E/2V�������q�	6>^\'��\fb�=��x���f����y�g���)=�3�G���L(=��h��i�:���Mu)���7b��j^��!�n�\���'d����wkB�d�5���-�(�e�w5&d���Hq���wm����!$�TS���X���'L�zPU0�����G��i&TSY�_�RU0��_jo+Y�(�uF���t��\X�A�M{�Rq���+�e�V0&�]5��dq�t�Ll]�(�P�g���(W�=W����c`s�!���KVM[o��lX-4A�[�\%H]���5�f`&����+�e�Otc��h��&�b�	>J���#�f�Ab���n �A��t]���Yg�O!��D�
�f6
�����{�=��a��n1zN�n��5�T�S��j}�PU����v ����k-l0���[�� {C�Ls���G��fn�[f����nZUS\�
6�H�5�1���	_�j<X�>G��)�3=��)���
#*<�S�F���2�T�!�XV����I����w6u����Sb����$�d�l�����Aq)���L����������[��Wl���p�=�`r��8���gB���C�YT�z���i�R�(;�&�]� ����>��$\��c�\.��r��ck���B?XX{�1�M����V�d��xB���9����UO����S�!�o0��4XdO3���PR�]qM�u���%!,�.�7������D8��b}<�*��cOE�}���t(*3_�i�BbU#1�I���s�5,������6�E��<���Du�2�X+#��L�EhD�y�L�������x��/���u�>'���qPN9/"-/`a~�Qzz��5���kWhd2h�x~�����"4��N��!s
O(x&�N�������
���P+���+�=��"ktq������\P ��cL���b��81OU����q�W��e����P���uT�R]�VW|�.����@}�R�v�E;����h�Q$q�A Z�_i���Y&9���>��&���b.>m#�v�zuV}�b��P(�#�~��,x������70�W�7$aV�x�^�7��8��z�!j�B���=�[.�2j��r���a��7i�6Ap9:�vv�SLn�����C/���SO���ur9������9�� ��$����=
������KG��t���-f�#�DM�{/x��:�I���c�
�&�����o���l�Z�����@�G����79~��u�H��Z���$&�X�U,+	��[Gx�c�zV�.��-${n��	��"l�e#�@!���|@�b���t��3wA���>�*�-�Qhs��b �.��A���I����@������r�p�<�#��FH�����eR%%��SN�&�F�����dM�WDX�bZoJLkJ#0�I[��wC&�h�Vu�hf �7;��4�@J�%�������D�X��+��e��p;�������_|z���_�y���E����^����Ksl��Sj�7�����5�p5�{/��A����%�����+0(�}q��P�dY_y&F���/
A�����`K�?�������-I�h��E�F�;������a�Q����5M��t��pu���Z�F���+�9S1u���V�r�S�r�8�~�/"��P5V@s
��*�	��z���hP�K���ae-|�{���&z�P���%�=���@&�%r1i!�����XV_7<�����)�^/�+F� J	������;�|��~F{�����������K�V3�
w��$��5�*��PgSX0)R�b����2��yH��
m�������9��h7����t�1���1����3�L':BR���Q��~TqK0	��X^&����.l� ����g�|F���|SA^fh���[�����Q!����a��	e����WP�tr��1Lk�����]Z�Q��'����^NQJ�O�(R�_F|��~�c�s�7d����<�Q[��k���>U�:P��w����_�~}���W?�
������-�?}��?69
�*�bgr$�e{�zn#�G4X�����O�;�F��A�;�.�sw��0�TX��;�f�r;�`a��n;�������>�����Wc]0���N�^&|��I���WA�~�/�7dt���u�/!�[�����#��������QF	Y�xJ����S��*B�Hx|�O�y{1�G�����y�����g���R����E��8�/"�0�\��$�0�X'("����/�O0��*$V*��$���h)�G.���^s�����O<�B��!Y	��
m{E$K+"1.T�<2h�6xyX��<�E���4�[�#���fny
_�CA�L�I�iV�Ab�m�����)6��x��gz�{��H1�3����^-��,NG��)
$n��H��d�'�ro��#l��vM�)t��#G�G����}l��Z_����V��Y�1�9G�0:�9��M1�Qw�y7nY0�h�#��B����d	�86�+,�1�iP7
�-����r�
t��D�X���q�~c^4b�R!��)�y8�8��<�������KbvA�5�/��

������.q�E&V,2���2����	W����X����o��#3��L�X�C����Xd�e\�T�^�9R����D3:_�*�^������K8�'D#n���x����|�`1����,��)��0/'��
Q0�,���2��K����e�ac�Q��a(L
v���?qgM0+�zp�+�Cx����#��\�X�k@�8�0to^�P�M�QPr�)PzO��{��BN�$���5V���y"m���l���7�{��#����"d���U�oP��ToaE[1��d6�ayu�B-��z.�""MA.. `�=���E$�"�Q)9#vC5�rN�;���c���Oo�P3��f"��),g����UE����U�Q�[>���^"x�h�#����������@���qi&�)�F��4dU�h��]�k���}��?J��l��]��:����q[��{Z���Z�Jw���zvT7�n7
�8MG��[��C?��k�Y�a�Tj�VTG���B�@-��)?��MF3	�e!ka�b?g�-X��C6���������l_��Z�M��=�����R���g4���A�HVz��T3��k�L�wy:Vq�1F_kj.���H��HE��;i&|*	��47�_���2`%+�u�
��7���fi��&J�-T��HDI�K,�����Y6��}�fzbbW�V�T+�q�����a�n��9����S�9��%ES.����_n����5U��hG0\*�7�O��.�F���xS����Z�}�������/u��S�U8�8e���[
8d���"o�������9E��Q5�;�!4����"_|';�/m���z9k�,k�!���������0�p���>5���j��^bR�b�~���}�5���5I����!2]�����NN���o,�Kd7">"��[&�1�>h�vJt�#$��i�O@��r�(�����=2���(D�YK����I�������(z/��yK������4&��K���4�ov��i��hl��i��z��9�<$A{��c��8�u����
|'M��OY���������.y��
z��@�Q�y��hp]IM�x��g&|ge�wV�Bi�A0q]��;r6>p*ak��p�l$~b��e���vf2�
�:O0�W��L�� arOGG�O�|z�s����8�����j��v���z�d{�5�{���Z�X��|�m'�������A{�E`~��ca�x�~-���;!,�w�I��U��,�"��HPC����8��"c^��P���b�o
���������X�eN&���D��z*lTX���2�z)|�g��
�������ZZ��y4l�4x<l�x����S����Ak��g��f���T��X���.��$��$[|���N������������(�h�h��;Crc]]����+�~�k�pp����Q\�'��#�~j7u�rf�t�q�;��u�����#��^�tp���k_-gN�;s	�����8|
���c�����a��i/d�� �����L{��^Q'!A>�E3�T���`����B���zs����6��##�#F���M���B�@�����cPa�B�5�I=���/5L����#���� .��nk#���'���/	��i�$�q��'���������p��6��l<���N�m�i]b%krdqr�I�S��y�yZU����C\+�j������>�A����=�1�
�l{t�}�
(OH[9�	;��|o]����qQ)z�\7��V�ks��~��������� �n u���Y���$�-���L����(ek�
Dp\3X+�����g].[ ��F<���YN���rE<�[F<T��C\��+�"Z8�K�\�3\ys�i�l���St���/���
��^;����n�V6(�r�o�����������O�����_�w~~����1l.��+�.=6H�$f���__}x�a^	�FO%r�)���gdP��5��.��;b�;��O�U����mK��t���5�����
5��(�x���Tk ������WR�v�5L�v)�Q*�rQ������M@3�Q#x���;_o7�3 �iu�x��-{\��Y6&���Yb8��^0E�/,]1�n�m��y��q�y?�PaH��@9�6m^�J�.��b$����[ze�a<�����mU-��7W��#���p��P��)Hbf�����t����u u�q�c��)��������V�X�q���Gq����� �>��=���t�'���s~B��s�7��\Tq�:��C���#�Uq>X���*.nY��[+������e}���W$���S�}Iv���$�e��#��w��w�&��!91����	CY��Az��{zV�
M�S���5�q��""���3��q�A�;0��bG������z���we�A�Am]/^'���Pu�g�|��M���
;�`wz�����@����������F��u���vs�P�&)���B�1����	����L��_0�1a����������F��jI:���I�NB/Q���X���� �M�����v�y�q��<V���OH��>�Y<@�s����
/����R�.���a��0�e�FXwP��m�oh��.
����zNm����u�������p�U�DIW��n�	� ��p3��q�h���JH#��F�~=�`mB�a��=e�6������� ��=R�:z��.
��J�V�.�!z��Z���f~��!�e"S�����Z�U0�����{t��8�g~�M,���_�x�wO����������'V���aw����l�1i����Vu�������mf����c�@�E�w����R~�����d�����t�����������������$��������[�gw�o����xu�����}�"�*��������Q��bF_��i}�p������fn��E���4�:���mrg�2������Z(�4�'��Y��\V3]M��U	��k����\_^m,���?K_9!��i���`�����P�d6��W>��p�=�(�9GX��[49�sDr������B��yw#���%������d�0��N��}�*����I���R�Ef�\��h������XBr?���a�ks=�dl�y������g��>�+'dF����2�m��4���hD'�
h�s��_�c���h����e�P�P���c�.$;����U-G.��*[ta����wo:_�y�"�Q�&��?/������p����}�\�p`���������X��b�k�i���p'����Pr�v�=!l<��ys��x��^��A����b"������������z��y�����^�S�hY�{�����s�.f�Sz��;rA���]�����1�yw�^N7oI>
���)��Ij,�~'�Z��$�#�}B���J/5y���0%?5��=�/Ld���C�G�
_��v�k�Zl������T)��&��w���Da?^�I����6^oIc�zn���~���(�d��i.T���4\�����d�!�vhXM����6<*{7DG�QX&B<��N�A%���s���u��t�f���8F����-!��(529�Ho�T���u6���x�I���!���#���
�T��hf���C�@��,2���T�x��K�K��|TU���C���ss�>	����&la�.�����eg���%7�G�E��UN����\U�b�@���1��*�x5(5�����*T!Kj�T�B��B���gD��+OE.U��b�<���> d�}�M���Am�|qpg���N�������F���|���{��dTa��6��Od�U���v+S�����L�/KH�|NH4�;�4J�<��r,z�l��X���O�]��L�}O�i�+���������r\�h�PO,~X>cRO$u��Q C~��$���(�!���d=q�~�^>a/{y��]���P=�?���Y���R�w������<<�8�M����`��b�w����B�;|�Xo1�tkVd��+ �
������2)�U�T;�5��4�d?QD?�O�4�{r<�
���X�;�	.����=�[�~�PkvD?�h%
����W��?�c�N���������j����{�����W(<�����zA,{�F�+z��X���Q3�Yf���e�����,��+ T+$w����E�,=I����*-�	5�������i��LM��{�}�o/+�$����5�x`���i(������U�qn�Z#?1C�{[x;p�������\3��?~��}���q*��D&�zT���`)�7�1�@�$�C���o�����7�z�G����2����0u��^R�ZK7�v+��c�/P��P�Z6V�c�W��z ��p��X!;����-�
W(����z�=�����cZ
j�3�X7��m	]x�
��V&��mD��^�k���HY�����7����Q���K��/B%"�J���w�?:��Pp��%2�hD��@�}�,f��5<j�E�7���k���.f8������U����n��0��F�~������3��d��U%�����	��xm#n*3��;�^H����
�#��*�^?���s�E���R$���Qj�,�������C�~i�����'�4J�=},L���������==��, �bB�*M�t��zN�������]�g�R����~f�T����{�o%�p.�:��|Q����� G.��TH��>�u�+d#O�8*b�N�s��$���i�eBh�(�A�D�'�l�k*#�'�R��(Z�&���}���]G��:�
�5�d���V�����_����\v���Ft���9t�;MV�+�0U����a�U�R;�d�u!��D�-U�r���4`a/����T�Qw{q(+c�Gz����G��D
w&������*[�/3~�~��>�'U
������_=����$����[��W�~}��GY'����G�}�O$b�O*V��i\�@�?[P��B<Xdv&y�C�6�9�S��V�����scKn�p� �)��7p��K���f����@������'�K���YI���G\���/b�����=��O��������|���M"} ��r1&��m�<5�8�{�	:����+������~	D_n�m�q�P�4C.$��"�_'�|C�����3��Lk?z���1�c�~��7��h����^�i�(�cq(�CF������<(��2��k�@�1X'��>8Zj�8�g��K�G���	�h����~P����.�uN�P���h��2 ^}�l��U[mg
����3
W�>���5�B���_�0��p�����P-�����.{��0�/��FsS��v	;4o��*�&^s��H}.���O$����er�h���w���������%���M�Z�G':�dJ^4B�fES��&��q����O�8�7&^'��xDK���g�h
�Q������Dg�J53��D�2MKM4ql�h�r��;���%�P�3�<�������b-/���G P�<����$>����%��������?1�R��_�3�C�{Q��\^V�������
��FV1-F�yDyU%c�����T(�������%��"��77=R	5�A���a&�3��Sm�WV'����t�)7�����
�5���:��J�>3��n�����o<ZJW���g������E!��,���h�����ha.S�r�L�6kN��Ec~�(C'%�����Es����i�P��~��Q�&����������K����ID|��D���-�E������|+������]&��N��W�?��[������S�x&�:�uPa^�B�R����a^jT��E:��� G���
��W<��I1�'a��J^s�e�24i��#s]������~���k�'������"�]�Wko�z z�=�-��oT�7-�s���s�<��&X�*n���B�1q�j�����g��J<����YUf��!C�>����\�i_+����TO����5FQR�����i�����_��O|��<S�`#�����=/A�$R��#�:���h�:��?��~Sg���%�'3�h�^��x��wj�`txLn ����;��&����?S����F�NI�?���������<����}S��?�M�������5��~E���-|�'�Nev�����af���,X�����0L/���E���.��]�]��B��%�&A������,h 	�;���z-?�!�q�i��w���}_Y����P,��,��F�5�>�,����,1Y6�wu��Na���0Q2����.�n$��%�$@��3�l�=�������"��J"q��$�����av�u��+���m4[�c��J�s"(ML^+��t;�P��y{��6�k]�����U�)��1�y��Y�u�#��r�S��X 0$�1.������]����<�����W�W������
�>^J��j]�7�$$��H6
i���5��N���fv�M�S���KMZm���;�����wL��H��XI��8�,�2���e�P��]���/��Q���]R������U5��aA$�T���OSw�V;���$ VdHk��.M�����x�����=����D,+���k�&�t��N�S�����U�:�m���R�.4Gn����%�V-��j�={����u���u���t���Zf�������9-���#yc������D�(����s�%)5��>������`�~jf�*I}�kv�l���R���|2�L�
-���	�Y���vim�\�G
�Fna�`\�s�FisgS3�d��-��8'ja�~�x~��9�"�ZD�w-N�ES�9���m��^��7�^�_-��]�����>v�d����7���6�:�g����\��_q�����9�8���b�����|�Z8R��b�x��M�i<cn�j���r9P��������8+{\�G�;{K�=^�J��iN�����+-sH��!�������}���K:������e����~�pj��w:\�|�jz`��u�q�L����������%+��&��g]�v�cJx��(�����[��3���+�����S��SL%���k�������*��������_���*��������&n�1�L/k��U�4������:n���g7GO�]{��k������l���4��T;��^�o������o��m[��H�pW"Y�8�}�������-�ru�y�Xo�Y��=�~��U�/�M���q��Zp�r�-��3��l�1sB��#����r��t4N�CMO���Y�������Sv�������e�K��E[�d���
�7z/g/2�������-I��+�����#���`��y����
K5bu���)�����	�	�N���H����`�d����[��m�9�����M}Uq-���z�)������G��2c��n���2���3��N������{}O��f�.��'����8���G{Fj_M�,��
=c��L.���a���ac���9��9:���W_��E���<�U����v�?V�Q���P6����E:�t�z-��'��������}���3�q�>�?���]�
�M����������S	�|o���/���X�MZ~�7����-�5v����%W��)l�(U�G+[,���2�Jx��j��h"�����1;�K.��
c+�77Rq�qS�6����b��������4��~��xy�u8������������kB? f#k8����bA�V���Z�"�~�Y�"��)��O�k9������PK�v���`�\����F�lx���YC����#���E�����V��R�i�����S�JL��F1,�e��*���!�bW^�q��i,_�[���?$l,S'46��_��-��k���ao��TU�6[8$�������l�z�<�w�Y:U�}�[3����U-�,4��[��"�s
�aQ@�`:^�9x�&fa��Xv=���������E���M�8�O���E���my��!vp@�r�.e�<Dy��]���i
���������������f�R��R�?Tg7b��:3�s�X�T��3���-z��t������=<Xoyn��~������;�Q��1��|���8������������w�KvO5������g�'uoP�{�Cyp����{�tE�����J�n�+���N�t j.�jw��������.JT�4r���q�~��@�*��� NZR|*�����:�4	�4��V[UU�u7<����M��� X�~b7���G�7�+�~����,��<��N*��`��m�.�`$� ���oV'Q^k�6�����ffJ�ujj^��z������O��?m�����O�/��Jl
�}S=n���}��.����+c�8mh���z�5�.���b����/������t����{�	�	Mo�`^�<��	1�rn�����0�X&��~v��F�>�4Gm`�C������>�4�4uAi`�����@���N����Hz�$����\�d�u�=�s���w�m���C��j����y.W���}'m�)�|&<��e��9��e�:��e��y�	�:�t�&�Io
xN-�lw5��o#�l#���~�S��<Y��>E�0���/
��%:0�)�c,�)S2�<3��<��Y2$�)���zJ&x���,/>[�S��e�������,�O�x$�����
W�z{#L��
�$�����a������r��<+�j(���:��
����[�����W�`�H���z�5.w�)^�B�����detH�E�.���3�����L�k������}o�n�����nd�x��;���3����bF��7$?0_u����U3��=^�k��v�w!c�v��G��I?~��~$���Q�E�^)l?�Q�c���>�������@h/UZ,ev���j��x��$���n����������x����x����x������u����[�m���{���x�o��S�'&�z���a����8��y���)'a��QK�}=/y��;_+�����G78��)�0w�"sK�����������/!:MAU�C{y���j��r�5�_+�����c���YB.�<,��:o��F��E�]�xm��o��h���Z:����G��?�*����^�%��(6^b�h`��lO]���j���b���^�7�(�*�{.7�No�����W�3�MU#�\����.����^�����cu�E�r�n��������K��6�|3��N���!K?ia��y!�����������At�$����K���l�*T4+7��jO�Q���r�TkpX�aa�Z�u����#�f�wv����ee�R��`��v)N:�5����b��������]�;�q��a�8�\��[%��������
+�e����8@����?����G����_���fno��y�/�����v��
h���;��&J����I\C���`k����X��v�3�V����������_���Q����A	��?���
�B2C���e�	�<�>"�A'T�Q�9�g!;{�b5Ps��>X����i��Q�9X1O��e�f�>J�}�W�}�['Ln�?X��V�����6��l��3�9^@�H?��hy�����d��tY����D�����v��W��P&-���J����lU����J���|h4�@l�;����������`�F�l� ������j��l�r���0C3�U�,��iy��q�z���64�����x�Q3�f.�6�ca�@M�*!�Q�jgs��-S1��$�\3[�C���H�9����_|X~���o�8K�wy�����u��^���`�G�y�`0w���0��������/�&f��d�2����gbb��u���]��Y����9��x����,����A����Ar�d^�u��R�M�J
��Er8��3���h��bd
N��XR_^F �l���g����U�t�VK@�"�s���`��\�6������g�����^��|�#?qW����[�m��k����0�����j<X��B�Y���{Z��vV�g�$W+u�8�((t�U��,�^Ik����$"5��F���4X�c.���O����5�yp}Y�tJ$��i��@��O�i���������O����X�Nm�e �,�����0d���'���b���Fz��Hg�"!{�26I?�co���B���/�����p2���n�u���]��������:=��'��0X��p-h������9'8}�����0]!�:��F�l�W4����}����lyDV��e��)���A���M��j!z��ZW�����L�J�q����c�����������}%>T������_��r���x��F�����,��6��j���v���\�y�n�����}��$Wk���
�����	?�� vwQn������]��yu������.&��B�@X�.�P�������F;a��5r{U�aA4Eu����9���>���1��.����`i�{R��:��"oV/=����.��Oy�z���O��H�n�'�.�gc�FY����W��*!;�=`h=(��Pp9���Mk�)�[:��8�h:e�I��XD����,YL�.l��q���N^�3���U_5F��;h���W�dW����O���V�����4�
�/sv*�)�o���L�*$O������uR�#�o��cR��3������XqY�9�PH��w���^��t����S�lV�=�	�'��$��Fhe�2"q�����`��B�e�G���#���.��v�#�W%������x�9e���z�;l�m�u�$��%����?66��$I�V��;X���j��<�B��N2����:_���
�I�U��p�t�C�������7�`lZ��{���_�6f�W����yV�w��7���,?��g�N�c�Gl�?D�M��%�%���x���������������K;dcu9�p+���]m��J�O;6��w��e;`�3�����������gq�����Z��n��8�����:�.�d��{��8��WnvJ�Q�����n�'��}��
�����Rp���zC���">����zU����VH�!��;����>Xp\�.���������@�|K���e`S��u��#@���^Q�Zb71��E9���d�,'���&��T�qP��e�T�!x�t5��������������
�=C�#Wm�b����:5Bo�!��B�?��
��.Sw��o�w�4E��\e��'��4(0����S;9gW�=
����Z��"�O-��Iw��/9����������lFG7�b)�0X��,����d�_Z�G�w~��%��G�3�C�3�d
�G����g�aV"�n��,�#�/�h���Gg�����A�*��s�/�4�%�i�A�Ir2H��~V,��*�ViL~��f�����Z��v�X~�V�!��A_��:f]��I�������g�cgZ����I���;L��H���"/�^s��W �M���p
O���ix������	�kxE/�R,Jwa��G������4���&�*f��B{?
O	�#���>yB����T�u����	�A������%�w�%�(5\�WW���2E�xZ��Mt'f����S��@8��zk�IC:����p1`���V�U��%/��w��v��4]3<Wl���c��������r��Z��M�IP��Ho�6�fU�����x6^�\.�7��,�X9�+�^vkA~92�/��HC�n�	��~��u��)>f���Z@�f9J�gS��R0v]��Vt&�!�n��{��i/L��]�� �z^s0��v\�H�p{����! 	�*�3�.�N���eh����e�9��c'-��i�9��c��-Q��c��1�4�w���;��|k������G� ���S��c�;���9��h#��Fmd��\�P��p���o������U4��I}�/���e����jHI��E���*������:������������"7���#�:�qZ������`l��6���j3�5�������������_�������{S� �E8�&)�%�A�������eq	o������h�x�6`�\���;�UxC��x��x�Cv%��0s��O>h��>��$��N���
(���gu��2���,0[���db_�����#�xU����8,F�X��)Ed9^l��J��Kq�
7�� �m)~:~3)�����z�DD���������7���2["��B�{6!�9u��-&d�(S���A
+5�HWv�m�W���?&o[�T���!���ZJx�C1l�\/���M�nQVeUMJ�ZvQr���RTZ�Fz;
�Uv!rw��y=��������j�hDR
�[�y��j����e�X�Z�����QPF�G}�D����"��r*��D,��u\���k�:���t���P5�u=������(!��W��A��tG9�|�������/��)g�.��*�OJ!�w}>W����-�^�B(��>����R�Ew���/���R���b'>A�b�q�\kDy�"������j�#�q���hR_��������)7&���Um-.G������G��:KLg
���@d~Dr�Y]J��k%(�9Y���IM?�j�:1��$�Z��z�D9_��Sx�><9���o����y���0-�����
X��N��c����9@!p�����������y��3���S+��70v;��X���s6�iZ���Ga�L�TN�~�������[_!O7�������[#�������-��~Ll"���v_t�	�*�5�Jv����������)��!�}�\��x�$�s�����_�_x�3���{��o��g������X�*6��$&c�E%�����	����x��0���FP�Jn���i�]�%7���v���C���u*��-����(����������5���j�)�m��T�]�b���KQ#�`�����|���0�y"B9i�!�4�A0�\��g�^Z>j�!�x����������b��f� W��c���{����W�[�~\v��|T\�xA
}��cvv�
��,��4��=�� �1��������;3�u��j���(7�?�	�s��FD�������x�<�3����k;�{k�������7z����4L>�JI�w@F�*�{�5���i����.�S�jS�'��������zT40wT��3��^�x����x=��
D���W�=b�[�p��f��/v"�ZrO���I�����@n�z�����j[g~o��y]`�6���6#I��b��":����c`���v�PSgO��7�BW(�*����:�
�1:��������Pv=}�5�/��f	Q-o��{i����dI�K�����(*I<�*��f	��f�,���dg�i~���.d� �`������nw��-n��5��f���X�nUA��~���@�3���?�9B-��S�]�O��fPU.3w�L;D�s�C�^����������o�������)�}F ��#�M6�U��]�z�:HR��1��+�eR��s#oa ����}i�=z��6�a�2�����kF� Cs�
�&I�)����P��[���v��Q������z3�N2��n?�N��!�pp�fv�zDU��v>hq;���U���2�]�����s�0�>'��=|,B���=5j�P=�F��B�:!d?�x;-����������>\���>�����Xe��d��v�jVY��Q�m�����
G�I�
�2�������g�#L�
G���|=�)�g��m=�'�8zV8�lzV�?�a������'r|Z��m�r�n��[� �#}HywIP��.N
����&��!��������9�;<#5l�{�4�c`[L�
��Z��l�Ve�1:T����)�%��7���r}z^�'�=�X�)F����v��������s��Iv���2��;T$}�>��o'�=��FPR}W|>a���|ta?�1�m��,��9�3kG�$�������C�2����C��p8%��[����_fF�%m��5?�C�.�d�1
�8=���E_�������kGW�E�+tEX�w���$�������I��7�R$=�s<T�w�I�ZfG�*�'(�c�b�]�J�I���1���&�����9��#��|C?�l�F�l��.n����Y(z.��u����x0����#��(�s��������O��='|*�sf����}�C��]�P"�]J��:,n��OJ�w�;E�/���k��Qr$&���2����mV<h��������2>�xE���1vOg��F->���c��a�<v���szo�}i}��t��S�������%���e|]����Q��qA�'��:L��w��y��B�|1�~��O�n��{�hf6�cK�E/o7#)�����K��!*�K�=+��6to��R���["�$�K-��#�K>L�/3+��]��\�e�b�<�$o�g�I�e�I�;���@/��L4
A����v�1��3��v�JY@�9f�F�>1^�$�����m����xK��S�����{�+J��/�?|B��F�����i������ ��1����sp"0�,
;��y_� 9>
��o����?�r|
I�9�p4F��
������YN��b�� �~G��������3��d;�~+d<��Eml�!���4S���������fGrh\N�q9Q�	1��~\�d]
��x;$�Px	?�?[4N��8Q��q^�+D�gc,�s�8��?�l%����G	�1��]�`@��]������C�o��rI��Xy������
������u���.m�	�������Q"|�����%u@W��0�C�O�����^b���7��`��������8�b~�����k�P�,}!�V���d�5�.���l�&�D:H��_xV
6j�4
hvZ[h����^0R�����D������fW/�6���+{��RH�<���:s��F����D��y���gUpd;<9����0��N��l�oGh��7!��*����S�R�tP`�������v����t �����1	���Z��/�<v4_��:�E0v_b.�(�����Yt��6��E0�yI������x�hN,��?��LKl���N�)��u�i^��^�)7��q�����U=�N��4E��
#���O��f��~m�t�f��s�%�h����E�AR~u��DO� ��4(
i@����M�;�`���W�|y�fm���������;H�-'��������~���;�v���$.O�G�*m],~������uF�����.J�8}h����^��k��d��8"����}�S�oC����7��E;�7�!������p���wCJ��.{�����N�3���,D�\����	4U�v�Pk�@d5�4n�r����V���_��j�L���Z��f�\jK+��=�\�lh������P3��qe��7�|�����G�!?��c�D�d��yUM���������K��?�����3zl�s_����I��l����HR�k���l��&_��k����57����������Y��I�G�"����I13w�����X�w���I�sN��I�C������I��~��yR�����cl�7��`l��B�#�`����J�����i'���qo������\c��L�"��%�3���M�j��5��A~����B�{�I�<G�����H:���P?D'������[2(@2P�8}������6�!\q�c.��S���|��Z�����9��� �8��d������}�>~��}C1��"����^����_~~���w�{��v90��8�4���(TQ3�R�������:�t���f?&��.z��{3����&��DM��������O�L�M	���cl���#j�y�F��F�ib�fH�|9�������b���
�Yop�Ir�����������gVx';K$�`����[�f�������s���
�yd���wG�:~q�N<�������	X���E��*�����?�����C���<,�#.�[�z����K���!�T�x��
�mXB[����m��O�6eE������y������\�f)G	��TH�����U`����zi�J����y�����W�	s�Ys���Du��+S��m�����-;�1fZ�`��1���1�D�\�b���-�8�+��;m�~r������Y��oux8M\���vT�;�F%�����4�(|�p���\}b��c22G������"����Lu��n;

sz�Ic�d��]�G5 $#����>�Af>�!	1B>�A�|oB��$��!	�����!��84�����QT�t�D���~���%�0)9�������2#9���D�	!�e%jBH"����C���h���Q���r����r���� ����/:�~��cF��l/�>����� ����!LI�vtu����T���x
X���CL�]=��`v��7�Oc"������!��Z
��$=~����[��|z��.kR(�0�l`�X�t�h�4�.���+��N�
�t���g�~�]����,��s�
�����S;�h��E�'������5|�����|�����o~��.4(����4U�����C�'&\~����<����&�����������w�`_�c�i����M=w}<�XM������z��LfX?}d�%K�����W��
��;j�`uT[r��� ��d��p�Ep�����R������J2,R/��2CI��tD����*�6��"A�m9��/?��R3P���K�#|��+�:Y����k��e��zu���x����V�EBw����vWske%�s��?Wk���Z�����3�6s#������Xg�E_�	��������YSZF��Ewx��F����4<�S��^�����
=��t��f7\�2�>Us:.S=SG�Rk(�+1�_\�tE����`�\U���0X'��$g�l���U\���&~�u���^��j���������J4�;�y����$���Z��t.%��|�W�?��0�!���'O��y�i�O����:��^��3H
zE|0i�5M���W���'��wt9�O�mr/,w�l*8��MR��q���7�����j��w���X���d�~�#:��S��9��_�[:�������n��W�p�������b[+���>�v�x���o4����=�BkZ��*��u��X���^���v"����x�H]]����*��!�t9��_xto���J��^O�G�+���:w>JS#����������,����%�%���:_f���u��?|�����>���� �}��F�D��?�eaU�69��c~-'��y:~bZ�������x�c�
�Q�x��\yB��R����%j��~����?Th���M �gH��H��9�R`�����(�}^�I2�L��6�$�x��?4���#���#s��r��^�.�s�������Ysv���`�n��3����
�� b� ����)~z�����x	�H��l.a�����_FF;���������������p!
�*z\�/���f����	����8�j��z,���R_����*��b�p�i�15�`��^���9�%������������,��r���MI�������X�Xm&��a�nb��m�����8S���[�����n�.���`���^1CI�[o�;�CG=�|�Ct�����wg����Z�������&�(��/b����4J`��V�`�V��-���Z�f��c�z����f������6W���"��(38=�+����]���(:��c��B���g`����T*�	TW����K�d�#�����9�+��X�I�X}�����X/�7��eB+����"�z�7��F=��5��5O����#��S���
j�������O>8����Pz2��������]�SM0�C9�il��DN�7v� �sc�M0����6k����7A�����_�x�K�[���r3^5�&H�
����`OD�y�����^_������z�\�����!�R���4p,����L@��Q��I�4A\[����c|�nM������-HRb;
�)��|sX�c5�^|:j�
�%�
����2P�A��^s,zv �fl���jN8T!p�\�C�q��M��W='�X���PPo+l2H�e_��>YlH�'��5oVG���]����:E�W�a����S��~�,x9��;r=�n�q�>��v�|�}���Z8UXje�-x�S�|:������!���`T��'��������<��tA�����E�^DL�]���ms}������%����=�^�����=��ssmC~����� ���7A�]Psp0��7�`GF����)_����i�:U��	0����3pl�=�g�
o}q{�'-%��q��4�c��?�����0�C���eU
���t7�E;�CK��X�'���9Z�'��R=z��+8z1�_����\(�8x%}��J<�cqUa�lW�IsWAG;�c��U��u�3�������4�::i��I9�_�	V����A�
S�/�8�Ad���7�^����W�&�����������y��������]��m<��g���m�do$���k_"��v�q�������=����a�d���$Chz��|�����mQ%h�*q[T��%��=�Q$n������2��0���R[�b�|�9��Y����#)	��d�e�[�G��KKL_�F0>y�>2Vn>Vf�����H����/6@i����2��>2�6d��mC��q�	��5�#G����ld�;�|���9]����������R�|.��`�|����9[��i���[m�e�h�����b ���F(�B�RwdxR��@��,�X5������iYxG���E6e!���Z���X��$��}i���:2�#�0�%�`��,��5�u�G�m�f��e��M,�;�X�I�=����u���0O��{�-s�
�'��f���P]_8�3�aa���O�������Y,��8��8����#����O��s��g���y{�Yw�V��b,?%c������-��e��(K@���E�!��z�@S�H��������o�?����_�f	�����~��.�,�~k�&iz���W���a�@[���o���a�^��9c{�3_e��&������{a�����c�q-�`����s��3u�SF����MU����(j�.R�^�Uk�l����O�d[}�`��k-���5/�I2���"�����=�������r�=�HFz���m��d�t?bb��o�����-�nK.�������?0���C�~��RwG'���^M�"Az�!	��z��&�q���R�ZP��S�}��8<3YZ���g��r��ri���� ���
�$����X��N�	F �"5��e���{��_�N7sRT�v�|7o���<�m���D�JX}��� �F�|�6K����c&�,���K�����q6��f����9�n��w88y���E`%Z�(���rS�����L��sp�����d������N������B�7��)���
�z9����jsf9_Y���R/]�"v�J�~����)��G[���fQaY�����k����[p\"�����s��Yx6��<a�$(_�@�
!��'�/����������=6���f'�Xn�9�Q����r{�@�Gr�U�-��(i�vKs]�I��������(A��L��^�2{+P�����D�,v��e3������;%�s��:C��+R%8���>GP*���X��`Oo>2YS�>��s�=y���g����R
v�X�j�����u��W�Jj���#��Wc�:R�*H��<�*^@.���������iu]m�Ni�r��y��m�.9����pqip����4�^gk^^V����U~�] �_`�f��9������Y�/��j3-�l>c����_]���Ia/v�`q���k���t���g������`^~��9��i��W=���h��nlg����"����;��d�K�X��Df�}��:s`����`4Z�����h������vz�{n�I_��c�l��ZB���/��x(C��� ������������)� ���Up?����&ttt�%�W���S���e����H]9:� ��L=k�Hu�9[U�#�->��R��.k�\k� �M$�7aF^������1sr� 
�e��cZ���C���s���v��s����
|������:�#�E��k	�4�U�����>��1!t������Q{}%k������`��m��3�6�zv&R�m��m�n`���m���Y�$�KD���������y�c�r=��c���W@P��8i��8k����9��4���y����il�������C7(E^K�h���n"z����H�5�i��s�V��k��[u��1go�`_�]u�V�k��+��l]Xy�,��70��,i.�kXs�!�pj?V7h{��*;4�=��$	x�j��s���}>��g5���n?if&G�P��;
�5�2^������^�������'����W?�����.v*�5���0��f+�9A
��A)�y�%����%
V�*����u_Cs{�)��������{���H=���r����4�>��}�}��-#��(f�$9����U�1���?�bl@]�a�Ys��O�u���,j�X����k����_s"�j3����X�,��(&�6�+���3e���k��{�Q	�f�Z�3���W�:^�`���?��1v

w��9�=2��.m����]h�J1S��7??
����~�����|����{!���C��{.��- �t���=�w��WG��?������>�9K�/
�G��U]/��~�������5s�1���q�=����k��uvK���s�=V�%��>�� ���2|��s.�?�j;�P�s1��	�/���9a
�d��yXa.�Ez������m�,��}e�p]��p�|�|����)�	ui�-EE�����0����/����P�H�Ke��:S�>��vN��}��o�{6LG�6_W�n}n*�v�������"�!H����$�~@������tU��K�,�/	x��f���M�i�C�&�M�)��P�!(E4$e�h����M��d��nEhjL`+"�0�����r�e:gT�4�:���i���9�7��@Za�
���:_�B2{�� hn��Y��A�RP��zu�=l{���/����\xE��G`
�aHC)������I\����Q�l8V6�JS�-��j�A�re��h��A$���s��lhp&Z��o��;����b3�N3�@��$_�����jXd������E1J���)�A���z4��U$]�k��n��o��W<�I��`W�e������R0C9���U�?���8N���Z�%�S�M�1��K�@"�%bwfO��]YpQ
~T���	g�2��:�B����-L�����0 ri�h��vl4���<��K��&q����j�r�������v:��t"iv���,H/����xi��2Zve���JT@�:F�A��	-��W�����dhm��:���ri�;w�H�K�K���^d,2d��_-�I}^��%��t3����Eb��e]q��ee)�;�UQd���kJ�1�p����KML������F��mq1ov�2����7R+�/#������y�0<~�m0��`�z������Y����pg�{��c�U��h���
�X_��5�d��@jN��f�%����.��7����WUv����`a��\c���|�:��AKm�V��$�~K�9�Z%LG|)���q�x����'����fm�|���|�O���\����Ed2���������)@����5��hL�����B�n�uA��`��o[�����C�A��B9�:���������Jme���q/������q�Ja�Ce�����<!��|33���W�Z�g��^�����f���Ji_����?��G�[�����r5�V�z�t,������f�U�)uV�/��G�f���vK�K�N0�Ml�6L�_�w����x6k&m�����4��w�~b�(.���{HMm�
�P��w�@�{�L���8D��.���(��'v,t�f���1����9����]:��S���Q�G=��e(������ad�J����|��
�^�yNY��@�(�})��������3v��iX�Yhx6+�,�W�K�D�Ea�/b��H���[��a�����4�r�2""@���l��C���c��V��;���):L�T|�w�;"�T�����=����K����F�L>�� sL�E;��N��(@KC��6�%W��'b���9������2���A�0�V\j&��b�1D������3���E�aFR�#�BS1�y�RHZ
�0
v������-L\.#�F1P��:�\\�T��,IxdG��q��>,:��$����B��k�?���L����o��C
�Z����:�?��:Y7'���a����/yk����jN&��+�����*V�j��O9�H
�B=I��@X���`�vZfK-�R�����uu��xI2��l��L�$���m����bN+XV�����L���!��������K���i"����l_�Z	�V�,z���E�v����hj���j}��KA ;�jV]�@��+6��
����
6G��8h����j�F�o|f������e���e�/;74�?qP5	v���Q�x��J���p#��E��5����Ky�4[��Db!3������	Zj�Rs��&i��)$�XJ���Y������c���7(��4l,�0&������q;`�t�`l�IYV�D$�����K9-t=J�-��\������W���/�7�{��`pK-��*@�I�w:4�T�&�P�25�]�P�Xz�U���r� �mLjd�
��H��&�������v%�^��H�	�����w��=�� �:HS��Jsq����w���/5
����X��L�Z������X���i�~j���=�~*G	�����OoN:�x��(Y���@U3�i���h*����%�/��:4�����-���=ozK*��CC^S�a���{.���c��S��+S]�����G��9����^���5%��A���*�2�OU��� ��T��Tkb}�T��b��u{�AR]��R�I���T�C�cfL�*����H��
��s�cP�m9��TW�R]�Hu]K:6�h�TW�;R]��j�Z��R]i����Ju����b-��]����%R�k���~��T��F����b� �F�G�k#j��S'pS�-`J�&�TG_��TG�f�C���|���D3mZl�(���R�`����hV��J����]�������ZR����*�d�TGMR�F(��0��Z��R�C�m��`�5�2��Qd0y� M���������	�4�����oc60Q'�;
�
�QF�����a�#_[��������-����#��x�O�����Cm���v�C[�N�B�M��vT��sR%����(��������GK���r=����X`�%�Id_��Y��_�N@����VF�C�kD��$��=D��'���l�����$�hh��Q/{�z��+|�|����/��=�
��O5`����j�V�q��j��J��Y2��om��[rT?��#�������I���RK�;�������[l�&�J=yg\�v�y��^M��4\��Q�w���M6KN{h/-��!�H�����E��U������8�i^���zn%K�}��$�K\Wg�����U$JJ�PTEj���U����|@6�m^l7��l_�t��&F!mSh�"H!���2A�B�����<j��SD���:�~8�~�I��#Z�]�|�I�^��;�O9-���e��A
�N�wZ���s������t�`���Eq����$��h�X,0uX�/s �`2���;mA�����IB�"�]bQ3�.��0���f��������$r�H�0X���Ir1�>�S�&;1����\r|x�����@�Mj��i�+�>�&u��I��{�mRs���j��(3��OV�����}�f�i<����
��/����
��W�ls5oi�I�R�������@�hJ(F��g�Uzl�C�1�
��c��F&t8Ers����6��kQ:<xCz!w�
=8��+�^�c]&6qg<'��&R(Xq������H���B
OS��<��~�N#�W:+����e,�y
����~���svX}LF��m������L�Z?�=�����g�^���h�U�����'VB���]�(0�n����H]*��W}���`��<_y�����T�K��V����H]�����|x����;���������b.E�����D
�����C}z!�� )9i���c.r,�g��#a.�.����H�KtB�|�@��h��6��'�>�"�?�"QN` d/�?Sq�k�W(�v�
\�;ps�"a��p���g�[L�ht7���"�!��7�;�.���p�D��(�I�vQ`�*���X��J�!��r�a�����'+*P�a�(�VC��B
zp��}�E�1�F[��B�2�U�|�]~��}����G��cb�?�!����a�bl���E2�@R�3#��_�:7Tm����?������t���&j�qyh�Z����ze�[����x3���`l�01��cu4�u}U���{�1.Q6�7�y0<Ma��f�
���$yC�6����V��GV_O�e�_��yhM��y��Wx���IsuE7������
G�F�6�rzYO.[s������p�.U������R�Ko��$v!j�u�����f�5T�H8���-^��'�Ps
�C~��z^s:��p�F\��Pqv����L!f��8�����f��;�����pTX����g��Zi��0^^�J��`��+�*���W����P�X�h���ay���%����[�36�2���?��Xm*����Hk]%b�<
[�>	�����g�����&x�\��{��`�p���Z�b*�>��N���i����z����.��X�����X��B�������_�����wo�u�����OD�����;&t��
���	�������$�82�'B�R��LI�a���m��K�X����K�����,��;�g�"�?��,~�
z	�J���3���QM���zw���I��2�i�ij���d���e�����Z�[�i_�XRX�7���@0�$&aTbz�
��GZ�M��f�����`�������y5!������/��e�E��F��i5������f�����~����
�1t��C�T��%U�����?�X��Y(z��P���H�1ba����^ �����"�`��5���F����M��a4J/:���(��ex���x�^�g�������	�u�J�.b$	���������lY����C���q-p����|��o������7o������_�������/aA��(}�����N"g���5br^�������r�D�����S3�VK��X�/�O�����,�������.����/g��gB�`3�0�;yNv�G����U-�c(["z�d�|��IZ��p�N	����.c�N��q���SA��z�v{S�b�m�PN�!��c��/v�C��Z�]���k�q���m���FC���E!V��>T���&XU,o�a�	XVUp=�}S�����k:�M���3,��p�LO�6k�>���b/T���E��4�B|�(�Ki���nfJM�$;���e����p����F�P"q(���MU�I���V�n	�����a�V<�������"��� dd����l����g�^�@�9��^Sc�����X�S����-�h��O�����z�xU��/�P�Q��iQ�5��{`���#���]�)��"��|+>f�j�������R�9���Fo�W�T_ t��1R~qFw���Xq�Qk�;��>�}t���3q���1�'P`��u�����
��d���h�������$j9�t'�$}\���)M�8\����JL����9.��}EYa�d�}o���=c�:���#C��������W$��3��������Mv���v�k�%���_������z
N#������:o�����p�@>�A,r��[��_�,���}7��V����Su��y!�_��{q����z��6*�L����;����|���5�g�=L}������wt�NH'����W�;�Z�}����lL��|z���r�������3��%���<Wf���{���/�$�Gz��r[��jO�������0F��t� y�������y�$~��$ q�*��	�GE�e����{_a(p�����	]��Cd!�+	:h�
��+����������
a�?��y����g�Q���K�	��~�\V�������m�bV�8����6_������\X���^�Z��P��b9>��o�E�bC8VdW�����}g!�+���/���B��G��E� �L��4`�������#)r#�x)SU��w��
Z���b��{}�7��_�
����(�S$U���-"@`#��^w��5-��3-d��j-��*��$�OZk�*Rt������������5�0{$=�C�
6�����!������$i��Wo�fJ�I�a=E�'�v�C�w�ocdM.k	���������%tb6�	�,vK(���g�l8WMQ�����6�����h����	���{/�g��zkI��i��&v������R�����1�R��=�Q�h�U��&����9o��Q�����1��b��W����l�k71��_)s�3�'{�����'��-�=�E�V����Gc"e��<�
���@]��B+ m����~���"�c]V�����|�6���sQ�1v�J{�7��U ���k�'9�[����\��W*�U�#��yu�"��_��ZL��s�A�
���&��#��<X�r��L���!+WH����W��dy�����z��U2�� �sTz��OT�B�Wz�&7�.�?�V��Q����ORo��\�G��*O�i�V4S�(����V$&%�I�PF������!�V�Z��F�x�]�CP���	b��C��PF+;N��y�y!qT�8�QU�09�h��ki���h��d�3Z9:���u��P)�x������D���S��6^e�x�bNp3���)�Y�a]���T��x��	�L|�G���;��(��1�,G9yG�a5���D�!!(qTj8�OZ�\[�t8��L����EU���9�#�I2{�Qj��3$��q�����f��i3����������V��v��V�����q���sA���m���5T�"|��i�x�����_~x���������,��������&�
i�mo6���*fb���1t)$8�����_����?_�rA7�X��Vv��^lFg��P��C��Xs4�_9Wn���%r�f�!���,%�P�2���I����'�6y�tO?��#-�I�7�D#��dd�H���Y�Ki��c!��4�������
���e
�����I�V�Y��]��&���A���U�,�/su��R��{����^���\����it�{��Z��`ly�+�N~��C�@��������T^��$��@�P��i�����S[�����c��e�K~�|���;�t!����X��8��
?o�'�mr���s�K
��|����.����e$�t�KY��z�����=��l����v�e�7��P_6�qH������g&�dKe�rib�}������/�k�3P���f}����f��<��m/9V'C�
��a����45����J���Q�;���lQR%�M���]�	3��p=�*���C�[��Y*������'����8G��k��x�c>���l<��V�!�u_���.y���,m��^?�+���C^,	�E(Y����x@I��8v(,�|(I��{XI�"	��$��m��J�pI�"	��$�w�<THF�$���d���$C�$Ci�$`5DO"�<rJ�sH���Z��>�r�lu��f^���c3C�pk�t���	��ze�R�1{xb	&I����������zJ�����71�Z�_N����8(Pc#ehYK����^�c=e�����yS�b�Y�i�k"����A��f����+��� ������1���+p����%&��R���y�����%������G���?���;�f��-���>�g�cf�v/����5��b����xRo�HR��Z�=:L<7oR��c��Y�:�u^r�,�e}F��|�XS�.����H�e}Q����\��L�����k ����h0���6k��4]����~��hn'�@�0LvQ���Fh{6��"���/?Z�����R,���|&r-���:_�K���>4��({�2������8��E����z��&���S���Aa�h���'�q�?F�O�#���zw_������Z&�nzfh�OXBBE\i(���.o�b�-nA�O��������FY��<�/n{\�yH��k}i�xI���E��D"���n�Y�����KF��g�{�9y���W�v�Xu��S�
�r�'���J���k��gn�fL6=�w���$�%�
H���2G<��������-�43L�<��!��b"������R"��7��)�CG�<�>���ga���Y��i�Z��;-E
��[�@��[b`-IR�i-�8��G
u�j���I}�8ey%X�cN�?S�p�J����\�]�h�!�v����z~��g��kpNH6�=@�C��;A�f9���)\&vH���.7 ����X���*�Y`��?% �T��d�,� �C�A��m�-�����,V(W�� kab|^[������r����u����/�����M����n��L`�@��j�\��ds8���*��n�$A�jA"b�
L"J2��1��S�u��eW���r$�����c�V����,��*~�y���aj���Db���=y�K����*xxr�%Qxxr����_��0��^���"C�|Fj�
y<d���������Y�YT��|z��)&�����4q�u��5��~�4F��Ow\k���o��8q����y����W��	]f���c��Oso���^b�����������5�N+���s��O�~���]G�;2{�4U���M��^{����:���	'��v���������.�x�\Ot���o�zUZ9m�����).b�<m�<%�D������iV�[F�-iU��q��W�	$]�9�K���'�z�MiS(?,���8�k���pV��)��\vy���rV�� ������^}���T����9�bV8���R�`Ud!
�#G.t���h�yO�	&�P9�_�yw�r�����m��\��������
��HHG	~���-[�MGV���X�!$Mvk[i�����v���U ��X�,
��i8@^4�'-bV�4�������sY���.'$���T3��q��_�1����Y������������D3�]^����w��;�����I�nr��G2
�n���l\���$saN������P>�n���i�^rQ�!��T��zChS�T�NG���X�"{�4H�N#��O#��o����*1L/u�F�i2�lD���
%����[v�D^SXO�>�Ny<���'�4�6����+��x�M�r�����>T�5x������>�5Z��rk	��<�`��4������9��itX<L�[X�.�-����B���d��-~����O-/�$������:k�R�!�Y��z^�~	NX�"���V����������Sq��q�����?CL�i���I����s0e/�IT>����#,�)M�v#��E`�S����')
~v30�)M���'C
�~�:J�|JS4�#MM@�7���A/�p����h1g��(�i���f�6i7T��h�V�+�!:��!2&V_�:��#�L=�le�k:a9�N�N��kC���L�9��|������%�1#�A5�k[.RE���v���h	�4�q2r�����d2�t��O#��Z�I���W�aD�,�DR�=t�H���D��A��>�CbDRubD�>��].������0�4��Sf������0��m�"��p�H�����y������).i�d wA	f���&��Ai�Y�<��H�����y�Y����1H",z�y3��u�2j:{������$�������ut����u�}%��w�:���u<�k>-S��j�%�<�������!��t,p�k>��
e��A��L>w���dS�3���j���v� d
��W����2�v��K������2���*��$�lT`:r6��u�]8u�LK���iX�H��W�����cb��x���rR��-l���L
�a_&&��%��j_'���R����o���_����/��bj�O�^M��k'�'�$8��,�U�2������I^T�7S�V�&H`��b�@���|&�f�:���T���ZHE=�CQ���wh��n�V;{�b]��J\	��PY4����_Q����W�/�G�d�d���A���F1H=���:d�k,��������Li�<�12�/����Y���M��$*c�H���� �OHZ�H���uQ�WK���]9�J$������J�R1*����k���#��-*�T�l�������2�n����}0
�����?l���r�on����2�^�v�����_,�����c����"-0�[]�5�7K�,J��C�u�h��b�9�K�=5�U�"�D��
r�� ����CW3*[�{V[lq��e����DiR	������rk��%�V�}�A"��,���&AR��������dAA��������Jb*��kRb�\~�	@�5c�6��%�8���O����>VM��1"�y��vM��(�\��^<�_���\V�i�7-����T�fpD�.�}���.?�����s)`�4���-�\s��_Z	�2CU�O�e�]J��YM����uC{]];��������Sj���HdK�(�a�w�[�9��]~�:�g���|�3������W����md�$��f�{���6�������c?��3���[3n�1v��MV�Q�.��|l�C�ec�4=nG�5:�<(}�3/�CX����
}z��D���W��T�OZ��I����Dv�2
���`=����<�����a){�z����������~��u/�u�,V�Q�n�1���}8~��'���xW��e��M]!SJ1M}���]���
1]T���u�F�������j��+��fR����jy������smq4^�	��Ia���L/���I�\�X�YV����AT��I�W��J���H����"\���<;���28t$@M����Q����f�)&����w�t�&��b����1���
o�w>��lc�2�3d��r\6��{��;4 ��������@���1��,�9��/�����>M����^�	�����e[�t@(K����e/�]�#%�N�{�����\^lLy������L����x$F���N�	�,�p�X�����E��Y��9;�z��`�����-��_�����68�����e�I�F{��j�K��fw�`����V���:�����]��������t����W )��i�;���V�q����J��)/i�gd_��@�i�����f�C�l�uY9���9���`���.���yi���b�V4��
�N#&���a�c9�������N�"1Hj��_��L�}2��UD]����f!�6]o����,��'r��O�)[����tf��V��d
��*�����$Zv���	���0b��d��;�����'�X�� �z�� ��%���`����R�@
�^���5�2k�g��dd�M�#�1 �\����2��z$�����|�'v�a�����`r���N����<���J�1h{��+�}���
��a.	Ths�WI	C)��k��y�z�:�*�����Ew�k�h���@o�
�=�mC��b���d�p�LK�)�y�[t�|W�Jl�� ����!?@�����lk�] [)�� �$C0�P[��DApQ`�;Q��x��W� 
�h]s[Y���)����������z}�	�V�3����+,	=O�Biy"�-��"�+.���"��/k1�p*�6���*k���f��,Ya}J�4W���XkE���*��o!N�*%�����1���I���+��*c�Ld/IF��{l�;��h����!M9OL���w�S���,L�Z�o�������i�����
&Xj)�j��4=:�&�:�h����������& �V_��Jwy:��qB?��F�e5^p"��d�1M���y*�
�L��}{�Z ��Rx�F�nY����PQ�N��f�jF�|��vD�z�jbkd}uUMkZ��W�%�WDU�L���I�y,��41�IP�L�D`z��V\����)�:����|`7�1������6����f�!8��:��J�B��t���E��r�PY.)m�i�z�z��z��*V�&t-x����`������g�%�\�k�����tk-�&�S��=�j�d�a_��|9S��Q�=�n��NX��g�C���[;��I�O���6�K�
Wk��$��8K��}��������-�V�yC��4sm���z�4�oE��������)�$7L��|�s��F�[(LM+vj=�@L�n���
���3��V�)V��'L������q_5WR����(�ww.
,m9r3��`�&p#�����xb���VQ�����j�{���Mp�<��w������C�~A��/�l�8���J���=��bf��/u��l�S:����K�%����r����)�m|Rfbg[�Aqb8��i�:,��]��7Y�I��kB������4���)c�)�c>�z�Zp{�3��aV��i�R����������8�q��$�(~�+��^�u��9�Cw�r���iunL�X341��!(���q_+~��2��uv�`~��+�b�9u����M_H��H��j�����bdr���� ��vp|
S�����`;G����E�9l7� Pc^H�4qM��#���x���#�Bk������Th@�����>��Lq��������+����[bAy���9C.2\3c��NW��D6���N���o<��r�hN�`������OZt;��b�)������u���G�H����g/���'\��}$���<��S5���>���O�����;?���#��������y7M�<�/K0Oc2�d���{�7�^�Q��p!�v�3�u�����MN.�p=�v4�)�������E@�G"��h�**#zN����c %��A��>�d,F�A��K1��/�����#��Y�r������.���|X�sJ�X��l�Qp�$IX*'a}Y7g�f���x3�z:��7�j��I�PR�n�x��c}�G����$�$;�3�,6�������c�SH���}�cH�.��s�(��M(���6�F��hX>'����X'6q����	�����3v���8I������%�vVP��,p�9���n���-m��7����E�m-^`,��Y�W4�:F}4NV�>�E����*n��(�7V��_]����b��f���C�a���2��E~;~s�$�������)?�;���q�l�O
����s�x�>�v~�����y�#��D�G=��y ����\?�8^�8�B��/��\q���|�EK+"���d������k�\��;������n����Z�l��1;X�'���W���B�M��9�����X����o�Z�
VWl�2a6���zS�������m}��j��M35�����r�=
�/��N����n�����`�M����e��d�}��egF�7A��Hc'RrU��l�LI.�������� 
���JA#V�3���V;�44"/9�������~~�<��ON��R�0O�Sap�����f�����=8	Bp��|�,����>f������6o
��������YM|r�������L�H+�������G���<n��ivR\���E�����E�e�
��~�{�f����/��b�B�!���'H
�q6V0z����������P_�n�����h ��T��\��>��pl)��B�����i.Be���HK,B��0�S��`�J�>�W�i�b[��)��������
	&K�5���{�P��z�$	��vr��6�~a�����$����/?�|��/Q'���0U-��d�`��p<�����U��aQ�g��<�2	��� �B�e�������p�N&M�<���j^-��@	��:�o!x6��������8��
m����b���r�Y�3������y�=N�Sz�f�~�Z�H+��8��+|�,)�w�H�!r]Oi!
^j��EE�?�$cG����5{�Y�W�f�CS@�I�� �dB9f�/�eBy0!5Y���@��*!�J@�a�3����Yo�x&n�K�������k����[T���_����Rj9E�_���h���^���E���c{N�����{_��Vv��w?���;n��{��EB�K�]u��T�GHn~o��iy+C�C������I4�\��k�<�2| n�R�@nUb���2A�U)Q%�r������NZ��zV���+���&����|L�e5Y�0�w�)@��$�E3�2Gr�1�L<�41[�+��n���YU��h���5�c2�L��X��V ����s��B�j�l��U�r}���k��(�8XV'���$��t(�t�~R����`>�Gy8N��dVn�H��ph	�"C�r4��������X�RD��>����/!���xy�QE'���6�����M/G1�4qI���,�������V��,�3f"��9�:+��<�
�^�b�W����b���+gn�g��:�O��tY*�N�U����������%��}62;-z�;���:]�Y���{i\y�]�
)���������n�`�|6P������
l����������b��w��%)�}�qq�Q�;��2���
V�j���KW{��Y���r���������z�LdI��Ii�#.�8���RF��RF��RF"�������S�d���t�W��FP�P�j��'����N�&, (���V�������d
E5��-�\:3���l�X��2*��.*���������/�Q��g�
���8N�6jA;�������|��X���w��1"mM�����6?:r��X-�����Pd�L3}�-����]�;J��1�[zCr��e�A�wJbQ���Wb�76��C�sT����e�x�v�G��9�kF�k�������$Y]�9%�$&V���`������>!M�m������?�������k���o��}�+FM;(5���<��y{����$a�g���.���C|�s�YNt��^Y���^]� ��C�Vtu������`@R�$�kk#����AOO7Y�����@�XX�����cC���R��ohd��}J�!%%mV�N��=�r�&4�t������������Zy�I[�����K��a�
�����&p�
N�}� ?�,&��?�������a�&��*�ju�nYT�Q�L����e�!%5�������*�_�h,���i��c���9�N��d3���Zi;6g
�CJ�Zi���Q�Q�"9���Y/:���f1�,bj�a�E�	�����5����X|t)K�1��d�XfE��T�(�?���w0-b�qv�eQ�M�G�T��?�k�
 C���H���
���\i�C�T�@&%.�H/0�Ho��qu{����qch�e$�����A�& ��G�p�,�Z{�T��r�6�$EI�h�C8�h��=�j���������5�����[���fpz�Oq
�Ku�T����9���	N2�+��X�d8�0s��'�3W�|�u��c���9F����UN���9��4w�H�M��e��?0����2@�N����B�2wYd��F4�/��� �`�'�����|�� �� eE���A���%���>m��h��K.U��]�����-����qJ!Wy1s�z�Cy����
��|�_�K|z_�����Yg��e�pt�4jX�h4����p	�w�1��n K�l0���C�{M�M��4AM���� WR3�,<�n�,��p��&�������a��'h0f����`�z�?���p�7���2z)������P�����o!�xB~K:�t3�����BP�>P����� �� ��6���'d���ghp����_��	4���N�d:<(�	���sk�_�n%j�z���us�0v��
�T�}����z����gzx����Y����Q8�3���U� ���0��zq��8>M2���"��w����(i��c����(�1�P�4�l�����%5<
T��x������?
����=��3��W�&�����Kx���Z�k���[)����#Y3x�D&�8)��cGA���|��b9����_��7D���n�8�c8�J2�'�k�g��vv{�5o;���t�X�9v�D4�d��9cE��W������5<�Q3�A��Al���m3>z(�6���l�H��
���O�W��{4H�\2	�d���c��\���X��$���a�IbM���{��\�]�S�#�L'���~F0�Z�)�����v�s�]�������������qxk� ;�1.������pPY����j|�p.� ���^���B��L���r)��,7������X�)����CYV
,��������B
��D2N�lue��-���������T��Gb/��������8 �`�R
: ��h�R����T������G������F�j�����j����F�7j����Q��C(��`�36j�@��s���L��A�3d���U
.X����;�D�Kp]u7w'����%�h�����K?T����L��0�uB������Z��p-PCj��J�x�S0d�,p�[
�t�	2!�	���\8�,���M���0�%�>��:���}�a1a�Al�!��E]�(��^M1�Z���T�dSz��X�/h�M������hl*t�)�1z��0�)=x �����)=��lJC2l��)����$6��x���l���Jj���6y���S(%���7�V����f
�}q8�S���C<�~��W�����8��o68��F�hr��N}N�}4�2�k�w������1�����x�>�����aR�	(E�C�N�-{m�3�C!"n%Rq+�[���Q�6e#�8vU��J��B���4�N������B���^�2S�t��Ep�����`��\�EL����
c���`d�!�E��q=<�]�����EC2|��8�c������Z�L��L�J���L��Z��7
��G�`&��	1�f���`���9X�Z�=�:�S���3�XsG��I�)P�~�������4�GcG���4���#���#=x ;����#=���HC2��)1��6�W���`���4��R0�K�H&/�����rzN����J��G�f�+�i���@p]�Al.P����.�u�lr�����.J������~������k�CN�M�x,F��K1��/����LqFU�\����
bp70�i�j�P-P
�����*H��)r�/	����&5�B�{���W�9�W��F4Z�2���Y��X)�����0.f��$x���rW
x�������^���|��>�rw�P"�)� L(���C0=����d8�"9�������jD�� j�A$�����
\�������X�A�T�K9���*��?k�[�X�)]9��7����J���-��\U_��,r3���v8N~�g��m�rr]N>S?H�*T��L��u���&�����	p�u�x#��g�d�Xq�A"����EB]Q�mqyf����}h/"b����'Z*O���U�hiBdX��v�C�O��L�,�����������O�������f���������j���������.85i���gV�-�@OS`�'!���r7��B.��uX����N�r1��_.�U�u^nb�\�G&��I�j�o�u9?�g_g�Y;�S�n�~�����l�A��-���`<g
����T.�*
|�����V|����z����*W�~<r�r�$ sd�S}�s�E�u�}��O.�E��[g���N��F�_]�(+
B���8;���5=|������Mf�+!H�#-zh�'���]��F8Du��������3�3lr�]�?-���H��RbVz�x���c,�G�����r���3���p�N��L$�����)������W��f��
�����7K�G�p ���+m��m6��]f�	GwI��l������������Y��Z_��*jw���F��O��{B;Y��A��}�"�>
��l��H�0SM��
��s��[3<�QG��j���V�`�������.u��.�,��v��1$�|�ZKz����� � �b�grW�N�"X���6Yo~`�����_�����z���H�<�'��^DOF�K���vQ{�V�0��+x1�����)��zv:�S8�4�������G8
4�6�K�D3�6�������������+~
��lVO��P}i���E�[ss5�D%�]@dXUb|�<-8t�r|"9�f��TL�xG�����-��_B��5���ak�	������z��j�����;�x�6>���d�^���,>S��+���wA�.x��>LL�	U����z�/��W��C��i������
��l�p_r�P^�������R��7<y��7)���Zj���k`�$<����E�����)J��C�����9�33��P�3���aB�k�|?�YF��+���s�B6 � ����\�p���+�u��~��n���X(�N0�98+�i��.�������+�B������
J�3��U�"��]�L>r`�25<Z�L�7b�E����V&ns#��$����lU�%�����\"CJ����WC����w���L�I�y��������w;����]�G!�_#Ep8����(�����H��>�0��<��J��B8��J�V&���V���f�����c����j�f0���-S��y��5��E�V���^c7�~�c{<.��������a��c{3U{�?a��o�������U�u�d���,M���bK�|����<�Fo(j��9x���q��������`�����x��4*�����t�q1�nk�@�O!����?�e�E���hg^j�-v"�Z�+@m�|�W4rU��*	��h$����~)�kd)��162��J���a����4����Q� ����<��o�?��G�_���V����7c��3�@�������o�����~�?r�����B�����b��[X/P������x���`�OB@���'~�{��K|�x��Wm�r��B�?
e��� d�a�2��!���\���\�Pp�pdD#-/8�g��B!>2��.�TV2�A�.���(e��7�2�{*��y��h�BvG���o{��l�;|��3r_�S?�7����v-r��4x�:b���5��N6��0=+(�A&��(�Vp�'����d4�k��F�,�;�M5�_K�P��a29�T�X�8n� G�6`�%������('��~�Fp<���W�J �0�g=����D�^��J<�Dw�#��0���!o��n7�ZK�����}�[�
���^�-	�����s��������~ed�S����{G�����/����������-E����Q�!��Ib�=)
��@�,�m���g�;!Ps��<
���(�t@�Q�M���N,�;6��L�|#�u~4�OlY�`������`�e�]C[/����Gq3��GS�2W�9�2 �d�P��d�Q����,;��y\!�^�����{E������E��k�bS��^A��Dk�����!�d>��s�������R�����y8����~��DZ�FX���G�f�_El�UYLo_��G��J���K�mt��r��q��J��n+�$�@:�%�����3{SI>���lb�g��`I��a����`��"n�^-py�4��{�����@!��h�_�6{�y�����
��M��x`���j���U��$��Im0v\�����'��/��
�����n?���p��������ok�����~$��b?F��\��b��q|����e���\�62 	Ag����n�b�U������K����M��^���{~]5����$���Yz���.�C�\p��p��j�=�������e8�N�BG�
P$��x��D��6$�r������\��\;0�9�&�m��n�b���/"<���.�EYNy������Y|f��s5�[9�,6��cQ�g��b�Ycr^1/�����j2�|�y���
���F���&��mU����	
��a�M|�����0.�vnB>[c>1�`)|O�B���p
�P]V�L&�(
��L%����X��Z�@��������W�T�G����T+�C�LxS�#��T�c��qE��~s�g�Np	�6$!�o^Oq0!�9=.�������o
�N-"B��k�����M!G,����0m�����m�Z}F�CW�s���Yq%��m�NE�[��?���������+��D���e(�X#XPb�gD�1��L�\OdL%��3��}^��1���xB�����\�t&��O����r1)�gE��������\
[��0�e���q]�Czr�!"U�f�����M�7�\��rl��*�E#�D�����i*�iT�'�l���y`��C1h}]��pH��J��C����zJ�CI�U��~��e���2�h��*4U>:���TAA����R�A��� 1��p)Fa_�2�+�\����B�\��d�������&������g�M���p��fZ��r8�8�����Go�P�>�IPqf�$'lK	�LED���w6��1��#��8�J�����c�BKm�W���{�Z7��G
����<E�p�5xyZ>C�(�]n��#Sv�v:Z��T�H>����Ru�r!�K�Z$��;cY�V�7'n��w�+�.<��/�M	�!S'�{p�K`�������3�	�B��>�4��NB�*VS��Z��J6N����-C��������c��0����OR/����MEL@�}�G�S�U|F�i�����ioZ:S������hGzy�@�A���%5�b�6����T��Pjn����ml6�wP��qP|t�F�c�5`�g�~�\K�����m,�����_w�y���Z`d��{����9i;R��h��,�e�)e���%�~��8	v���]�I����hozE�$rE5���������or�IS���R�p�&�P ��j�x D����z���������` ;|�l��i�Ke�bt�
�d�xe���	��@J�c�C����aO;���j����kQ�1v����JnWK{�N���$���/v`�{���>�b��=�b��R@��#�'��mY>`�t?|F��N����nfl����G�r��X���c��BT�F]��^����F�4�'�n��������V����h����m-����'F����<���F��(M���k���U���	o�=�(��
��1�}s� �q���:�Y
��a@H@�R<,�*���p������4�%H���?�����0�d�R19k����(	��;!�x�m��"�@���N�H)C�)D��m��P�F�����'g�O���6/��DO�����J����BEJp��,��D)����:FFO��i�e��Uy�����
�5��*�	+v&�d\9��N����z�����������0�f�<sJ���T?���COX��GV&[�lI@�E3����R�t2���`��\-�uU���=�Os3^�%�';��D�$�G�����D��Ak������W$��&�	���xE���W��1����-(*��9��d�6�h�����kW@��oOPZ�3
�(�$/'�*cx�_�0��0�P��c��6���4�2)�]\�P�D\]s�*(���OYe�"�,bG���������=�
vbF���7�`�S�0�mk�����i{`����5��i]6[�O�����1s�U��akRZ���U��fU�������p�7���Rf%n]
�z��sX7�ZqQo�w6����0e"���=�w�'�5��S��U!����d��v,3�`�o��P^H:�.&�[��/��tl���V2��P����Vfc]��f.I��Je�+PUI�pv�Y�s	��yl���C��W�V6��5Q�0���s���$��E��{��`(�T��x�:�pN�/)>��G���O�����Q�+*�|�=r��
�Iv��uqK.�E�xN�b��|9dq ~+C�X����3p�7�
p��\��g6���'C����Y�Qe�W7������'	P]��%�u%C�us����m��`���(��[���l����B��+���`�;8'��zY8����p
���0�#�F���3���7c��a��������_b�]iI��i��5�7���G���)_����+
s�<
�����*�@@�/dL6R2��a-�����T^e�z�M��j�����)�h�p��eu�Y!��L���>�s��bT��=uR�������q����\��$�H��9�W	����TM�l	$7:@�����jn(�2q6J�TN*i��	��5/^�x�y�j�A�Q�
v��
Id�:zvAB%�5�7��X�
8G��������Z�(���))���I�������)���$gJu��k��c�2�����>�ddG�%#�LJvij���d��b�(�����"����0��2�E9�C��������C(�tQEV~TfLm�F,�'b�`��0&��U��A���;�JAl*P�cb(����_ ��If(fuk�n����Z��
��:T��
*jo�W���~�e�E9�)�[��BN7_�U\7��V��6N�d��MI�e���@��
8G}�f��x+D�5@L�{��rl-d�~)���5�Q������p/�4�7823��!�R�����|��~��\&��$dwI�L��O�<���K���O]u�����4����,�2d�@��E�u�����^)���|�����=��e����&B^�r�����L�x$�6�]�)��V���&�oW�Gw&Y�1������$�R�{��D��"��� �N�X#���P�~������O����FO��W���xO��{k����\1�quI%��m^5cZ��dml���q3��3WO2?-&�!���;vz��\r��A����m�JS���8sA�&�h8�A���P�^~/y����z����fG#��Lc�\��l.�r��� 3-��xXJ|�y����m����,V����\�fj���G|3����1{0��F#
�JG:����}��������O�O��~�����W�'�����������{����Zp7�o_���������NK3���j1�o�����M�=��d�[������|������N0�ig�@]n��j��5������+�Hf�%�K�.�����ePo��`S.L���,�`v�97�7�����@��f)��fK�����x�)R){���$x����|���K�����O({M"$+��+��*qB�^������s�\OK}����Q��2H$�.�����@�@ATi`�������<L�8LW���&�'�����.;!�ti���6Q�����8mjv`�;3n�]�a28R��������.�@E�Ba�r���w��[�U�{���S��{���\�������R������)i�*���U��;��5'd(�����ilB�2%�����N�+���������`�#jK�++��}3t�-���{��){�����5,���T�R�Zj�:A�| ��x�L.3j��a�5����$��=H&�^�V~Id�C4�/
��C:��*t a�����uP,�^���Q���CJ� !��#H�-Z7K�|�9��c2���*(H����E��I������|5AsX7�X�����qm�����d�S�Q/�.���`�X\���:��:�x��&������}{��UT�V������o��S��T��(^�
�L�#����UC�iCQ�6=�����m���������[�*���v�t���wa�
/WA�+~.��J�E����\����>�D�'��G�>.W��fz[%��}��mP�fk���fEl�_"=5T�3���N>��`�	d2��8���b� Ry��Z��(T�F����
T�I��e��eP�&#30�l2�r�4�^�����Nf_�bM�����bM���
A(+���BR��)������P	�'o�	�������I��Z�+s���<M�J���)�{oW��Pi�iM;���
"�H����
�������#�U������}9H�;0�m�=o;iH 2:I�2�@��Kb�f\�09�Z
L�i���p���3?���M��wQ-���st���Fs����X���%�U�\]�v�������\�e�X�4o�1;�@���;��y�>�;��a�j�I'dc�\�|,h��_�>��I����-?�	=g�
�&�\�i�����w5�An��D(<R��!�r.'�\H����]�U���6#���|����9e\gM���\.C=�;\Z��hz��_�O��������>����.���G�.�w�W�I��)�g��#�Q���	��}e��2{bFN�~���Kj�:g��5R�zU-1I�O�W*l�	�G�O���QV��������z��x�o��v�������<6f32M[Xn�,A]H��>�}�/��(���r�'�����.%b�%�\�D�c��@�@d�L��R $L�JF�����{�t�c
����Ki=N���k>���U<��uC3���/��Q��N�PF��l���V�M5O�|O�%�������Y����� ��z�bP��,�):f��������::`�*�#��0^^'Cw���:Y$D��������g�9cV�U�����&��+�A���0!�r��sZ�Ui,��N�P�*cYw���lY�t^�F��gQ��]��aY�u����N=�_��,��}��RX��{`���H<��1U����7���)�G�������I������oPyJ����CGa�+H��L��N,S�3���U2�#��_�c���_���#ZI�D�Jg,FA��3$��`������P�H�#�H�HuI�m���u�_Pm��V�9��dn)�g�[�A�?�\`�>��T8�,��'�!�f�<�\`�y)��,�7X�Ie^�QH�����2�W��N
vST��|nj��OQu*�����j7�eX��������-_���A��z~~����e�hz&Q�z~
�-����	F6��et7�CQs���<�m����.}(��)�����,�<k�������#)i>�V��8���qLf�MIP�����s��\�^�mZ�qU	_
�����'.�0p�A���drf���<��q�r���`�\��<L,�G����H����@(���p�X�G�����|9����E�u���u-^�X(��#��~���}p�@��Rt�;��Z���h��F���2��P�|��^Z\����@7���C,��e���M��=��ED����a�%L�0s�{�%��������v���[�5���8��.�,�IE�/�I���
EL�B�����z����<�5K��j����^�*����"E���T�J��xC�0K*
#GF�L�M1����2�W���I�5�d�\��=���q#�l�t�-(�N4�6U�m��P����?@�����E����ck�9f��"s�g
�"s� z������3��j�9&

�"�@����\���(*�O��'�)/0�p��Q#6�+��
E���H?��jd:=�j&qo5�|�9��,5�P�lv�[��vk�d�O�.�p5R!����N�Q�v-Q�O�Q����+VO���D��t�R������u.s,�
��B:8��x�>7�0�6w�6�5D��+p�K'&����
h�uUb�%�W%V���+�� ���4�<v�+3��T�8W����q�q	5�X����Y��K|���rm����!�g�t_hl@'f�{:���\���a:�C������������v�7#��]�
�xXxO��������[��I&<�,-cx���g��!J�> 4�S�S?�8d����&c�aj����f%��������!�J����J�����0�}o�b���&�X2��$�����QCMDU�R��'d	�BO�P��c1[��,a�}�)�i�K��R/E���R��vYL�5����S��P	����'�Y�h�]]�P����-17p?p��D���-���b�E�	���������N��A��O2���dl9����)�9,z�p��<�a�W��n��[U#�V����&�@����}\j����6�������	�&����%�@��Ru����
����Uy��l����r'��u$�c��d��E�D�|!����erN�N�kX
"��W^^b!�/������L����-���M���u�mr
���eQ�6��'���;?���sb�-�)Yg�6/�u�o����Y����zs9��t#�{`!H��>A�"|��d�=k�'�F�����?j�@�����I����P�x��.<9
�c����$}��S4�X���;�\dD�<d^�I ?��L(v��ia��0�����!$#U�����?^�7"u�{�T�����%"�����iK_�_��4�n���R1� �m���|)o�m��H�
A&6H]���A���Ck���}���Sgz���	����<�����w}2�e�^xP�����#,�����Q���C�f��+��Q&6��z��\Q�k��J�x�-/�?,gXC��?�P���Z^l�lg�1dp������^&
�B�v,\b�c��������,�N��I
r`.+�y�@{���Y�6�����`��
If1^�p�(��U]��fR��yq�z���7@,[]���o�1�z����EA!>�~��+<����{�����__�S$�x��n����i�����>	�cj�c��<!�L����3��O�B�5sH��*q;j ��2@l���Q��Ve^��N
����p��m"-mW��2��u����
��]x��h��G��#
�x~����7�/z�:�9��p~�^S��6h d�-|D�_2r9�4���m��A�PL�U���"m�n���a�82f�����H��i�'��[����-�<��AD[�c�������E�_���U�"�T��{��<��/��Bs	�|�T��_�z����)���r1U_�P�
��"v����F�RK�����Y�[�A��]��������\c=�"I*�f>	%6 s
��`R�}v>����=�� �[`����2 ��b���(e.x���&i}�2��uXQ�l?	�k�qtk��e4�,z���C/R!�*� ��]�	�a���s��c�j�G"�D�����i����jB���0�Cn�Tz����=57;��N7	��&X��nu.;7�)��^o�u��]_6:���#HD� ��g�7��|�'��
��hg�*���SU��G��3gl�B?D�4P�(�H�`%���iT(���[�C�3��k�p.�A���bt-�h��c!�$�s�,���T
�p����[��pw���WM�������:���}7$y'���>�����cYU��c���E������)�U�a
.�0eC���-=3�5M��6�~v+�� _���G}q�UH��a��aoh�n���"6��k�r��A�{���������a���V��b�v�yg/� 3
�sy��E����y7��<!��n��"������o�]43��)�6Q����cE�i����G����A�/��e��x�C"&*Qc;
�Y�����\����i�'���zT��ZKJ%'~����n���/�T���t�W
�>:�G�Go$<��)i���i!�� �T��I1E����+I�x�1w|DD�Y9|���|>$G�e�M��$�����{O��}���,H���3���EM�Z��/�����@j�
,��^�!K�hp�����c|Y�	��m"�����wxR����{�)���_�'��Ln���������H��L|����L�(�~D���/���R�s��H����V���i���r�K5)~����(�)d��Z�lW���T����I�syZX�������?H�V@��rq>L.��������_��9�����.�!2��������21�HF���94��@,���k��
�y|a�i��J�.�<Rs��z>��!�s����C�/^����(�v���j.j
��0�^z�+����J����wk=I���0d__bW���*�<&4
����(����nI����(�N|����(�A}����u[[��1[m�J�� ����v�9%��	sR�'Cw�V�������'�R2H$�� �v���w���-K�s>�P-�"��m��*����T��3.���6,�9y�a�[���K���)kV��8}2����ui���y����\G�{�	�K����6��p����R��~1�7�-��R�s>�g1Fn��t�ir%��Tr����>9����Ms�������)(B�����n�M+,��z����T�Nb5��i����2^��5������G%@�L�Ny�v���.���
x�Q���c�d��.h����Pfu�Kz��a�5����:(�F�R�tH������H�\��;[��R�=^�0(�G�S�N9�2KJ�����qd�!��Cx��c��M���&P2�������u$�;�)`��4�5	�{	�wr!o��+���y�A�$�E"��]5�����"�]�����wr,��H��mM�4���~�s�dH�n]q�
%'���>�'F���C�`qu���_9�f5�$��r?$�QI���HH�������1���V|�t'89|�4UU��6T�")#P�T���\���=��N�w/��.'bP�/�eXdA"'���U����T�AJ���O���)+��e�1AJ����������E��:�6�;�A�L"�Bg^#R����3�m+��g�l�?hx8���t~�oK��)��S��
M�+�Y�;����w���v4��/�a-���v��Z���R�����4(���&V�������=�bA$l�O:��d�
�`�WL�S`������n���W,�d-_�AQ�x.7���p��
qX<R��4H�����IH�\��|R7��s)�
����#	j�������M��$��Y9��K�M�-�zW������-������O���+{x�q;�4F���Ud$�p�o��.%��jp����^�B8����G+�F.��B1��6m������(w��$d"��������DE���)D�>�
�6�������d�x�2�T��b����������p��� �F����E�^GH
n�B#�r]/h��|�M��C�cJb��0�"&d�����w�W/g���_��S����/eXtB��6���G���[������
����������D��@���@t��N��F�����V��)z�f`tt�n��[sGv��Hp�D�����oi$�1���]�E-���X	���6::��G���L���>M�C�S���uB-"�}�H�p�����G����!:+��y!=��h���h�����jZh4�5����������\���{�������D$7����=/��mR�K��d����SlIVi��H������H|�+ ��f3}��Dw�b$*]��L`J:���u�o!��Tk���&
Y�_YUzxPT��K�:`��O� ����l\���~������aYHS^V&�*Ua8�J���X"+�u�wG�9��k�������T�Kv���Ka�s�0������\|S��n$���,y,y[L{��z��*]���t����'2�GZ�������	M���Q
g��5�I�a$L�#Y}I���(k�W5�D&�0��x�������q��cp5IH���v=;���|�o��_��q�Y%p�,K�'BU�����/��!
R����'��=Z���?OD�|@����<���kE�1��"1�b�O�	�@D�}��e����|+I�|+�e��I&�[In��`�JG]�u���q�T\H#LM!
�����~���HU!
^Y�P!eC<a��~#�)����@Og�:<������������l�����0��hj�eCK��ZI��C��-�aF�v|d�'-�RJ���;�����X��0�"t��R��R���Z������R��~�I}���l���1��MV������x��h{��9�����XN����c�n����9�1�1�N�r�0�h4�����;�G�����FN1F� �D#�����9��Z1f^O���uq1�����e~���8���|�@�F������(
F4VI5�BI;[����e��Z�."*����q�/n���]�b��\�j�T������X6���)����j]��3aa�H�k�5��k���dj�����r������.��jq����g�	����q���U��	�K<��y��/r6|�D����4�l���"�������n����������b����`���lj6 ��Gh�P�!
L�D%��Q��N����m.����k
v���6�#x�����_��T��>3��V<����=b=������P�09�zemO
|�/���}WaF���:@z��B
���t���������R�Z)��.�rZN]���]��]G"���!Mdm�x}�aMv��p?�?��������D���uG�;������o��W����,
�>I�����;YNx�=���TF��������f�I�����������>���c�%�to�P�R+
����Y�/
(�"���L�Fy5QT�����wo������w�������^
�]����������0���Z�(��_}� ��<�g���@G3<�~�����C�B�C��a�W^sBz:+����:a�'n�d���=&d�7!�}$��g?l�U�:Y�(��)�:6TQ��8O}�q
:W����	.&�W�J�M���	��R��z j+�B�������/^���C�}�bQ�a&���}�����2<*��X�A�N����P�[���$I�Z�G�F�N�2,:��KQ�HI�G�(�VQS���EhC����Z�^/B�����=7�$���ljV?��\���������r
�]���"�l��F����x��vp^
�-�j���B.)a@����RY�B� !�I��7V���Y�u}��OJ�������<j�F�������o�ZAQjjZ7��f���}��j�������d3�����J��r��&;���c�_
����")A�:�c���+l��,�|D|	�EvQ�m�V�G�ST����D�I���P�[N����D��>d�o�7g�f]�G����T��m�p8�Q�b>��.��\x��=������uC�Ti�z��|��M��f����H��KY�A�+��t@���kSO�A���^��������6���*^�u����n���B	!�����:UC�0-TT�u5W�@S��@�]
5�����$b��b����(W��7�p����m����n��*o`�vw1��J3��cI��\��������m����=�>0+u��a��lL�/c������^�>/�He;��/����7����,�Kje�G)��"���O���3���F��AI�-o������b�}%��b�'��p��]�0��>Q�u����Uy~1SI$�5�l�n�1��-�p��~zr!6v5�7s���r�D�u�K�|������N��:?}���W��bVL�$�����;}�z�����������~z{"�o�rW�w�s�n��O�(dO
�j*�&_��q�\��pw"b�n��i������m��j1w����aEI�d�p�-Dz|	��
����H�J�E���UC����b����,&l�T��K�5���w<�.�E�"��Y����P���\UWWX��^���%��p�o���x����M��jsa!�x$���<4�t������u9�`�bt�u#Y-3>TW�E���p�=��Q�.�����n��4'vG��oC����2�y�F�A� ��6����A��|�P
��7XE��X?:�����xL���,���!�{^���
��Y�<��*M��:�<��y��b����TWY���{��:��#��%����{'|���Y�KR���kz���K����D���=T���<���D�2��N�mP������0��r����=�!UGD����8rf�*3D���`N`J�j�/����,p�y��N�����U@��`@tk,- ���4�5��Pu�'0���X�dA� d@(��	1~�L�fY�]U_����� ��a �Mk�E(�-bql�Ghv��� ���ltt����w6��q�Z��	�5�Z7*�(6TzY��M����3
~��T��������.�^7��������__#B��2A�6~�8���/Es-�V����~�����p������QA;��	��6,�#��tR�pdl��t!�����6�N88������"��Y;���lV
���z�
���{&lb�)/K������X��`&d��.t�3Qg��jl\2
/J,�i@k�hJ�z��.s>����T��,Ts�GY��D�_���a5O�J�C�pp��IS�	��\<��b�pq�n��I�*Q�Z�L��`��:eWu]q��~J�1�V��w�q���������W���
~���<�	���Y�"�q������������2�wYT����m�"����vQ�9�:272a���-��S�.���(�XL�9`R����*��I�W���z�"\#a���W��w��1�E������������e�4�k��pT�Q��Q`�H(I��N��?�SM��b3����d��<�
[�����Ug��������F��\�X2�e�%�\�X2��{$��,�%���$`���@	8O�p�9%`�����.	�����I\�h$����|e	X/1��'-��8/�hI�5�B8E�d��Q2@NF� 8eC�d��4DN|���� Y2����s�G�gm�J8�������`i%6�O8I��?��H+I0�WH+I�����$$�$A(�V����Ks/�?�YM~@����������psP;�b���������f5)����W�����,$�c4�u�.��S�
C�6��(�>
4~�*���/��p�����P��M�$�����"�����p��E�����wQPL^���s(!f��>
g�����Or���[Ju�&O=?�����O��f}�*?��������)��sZ/1F��z2|����C���"����p���P��K���U��jQ	�1oD.}�R����'��O��S �l�*� �2�b���0����Z"�F`����[-&�
��.�"������lx.k��;*?�����9�j���M���I�4?\�7����`&��,w�`O���^)�pFs�`�g1�h���d-[�'p]����Wczii=�\��%����5�H��@�N���ou��Lt)�|��^]]I��#�����rzS,+1�b�U3)��J�NB�Ng���S�c��+�n���(�
�`1�Rq�0w����.�w`������rZ^R�����������O�~��Nwf�\�)c��K���=m�`{�q�l����~(��� �� ���� �	F��p?���]��DwB����w�AD
%�G1�t/�fs���R~D�^8���a���/�xCb�b�]50�!������ �?X�@$����_�M���f����_�E%~��M�
\F�U�����1�\����$�Vr�%NvF}���q�bB���\��K�;�� ��Q�9Y
-<�J����c�:s�k��
�Ip4�7�����a��
�
\�
�������Oo�����>�?����d����1�Nc`	��R�!�r��U�f���7���O�^��z�������7B��v)�\��6����lsU�����@�\�j*�����S��%��2��$*z�7=���P������`.2���W�vi��Y.��Ge.�k��&���h,�5�������B���?�s�G�;i��S���'�:�\�Qu[-���wa�CO����R)����[�G� D��_��|r�0d&�4��qv����c��`$w��}u~r�k
e��]H�x����G�M$�����M,���Bz�7m�����vA��������#���wa|��8�L �G'?�=���W��O��20{���O������'����6�K�0Ev��t�=�k��w��h8�|�sX���P4
�U@?
�'7����b5���,!3��#����q��l���~�L>/�$�������@[������rtx�)FY�4T����n�+�UU,n���U���@�_��
S�<n��xe��"��o����W����u\��bYT�F���/�<���"��%�|-$
��GE���\>�H}a`<h�� 3agi��u�\c	�f\�:������c��6s�`�w�����w����.w1n����P��H�
DO����Ek��B�����)�x]�'�
���`r8�F��W�LV6�����Oj��8���(;1~[D�K�����U��3\sJQ�%�4������=�� ���Tk����k�yI�E|�����XLJ�
� ���;�w��=�����0���-�`���|����*��i��K���<|�2��J�������BN*��=]:�
����Op��&��\����]D��o�X�g����^��m���3��!N#��K"�YM]R�e�����	6<�o��5��(�8�D�������D)���j�i�A�&��[|���>"��t�v�T0��;��<�w���..��&�1|����#|%�s����\m��]��65���J����O���m�0/]��m�����.�������Ad����9Jh���/dC���q�+�=�{�#Vc�#W���D�����,�b%�	�i6�qm-���3���7�!|�NI�����������
z��.�����hP�ZS�^:(��r��1B ����mv����
!���l>���[�wN.N��Y�|��G�XtK�|�+,j0}�>�t������!�
F���3zR,�(
'���PM�j�_�J��<�i����c��9�ljy����?����p�W� ���8��Ro���("���*�e�Flp�������)@Il�#����]F���^t^���m�s��d�7,���d�Qg�pzI����LH��&	���NmMb�$�=�b
`����Ir�� �hz�DI�+�a������h2z)������i��W�����'��nl�����j���q�]`�d�;'K1}������U:���K�o��p�k���z9&���Ey3V�8��zU]�7R|[�3G@���+6���w���M�|=�xb�2�w5	�k�p����;�Hq}���m���J-����N���l����u���;�J���v�p�����d��E�3L����F(��R���V�b�I�*8�������M�L=3�zA�fc�Q����x��!��e=<�9�r���DX���I����};��*�.�j!s��?�����k6�%	oVS�HP�7a���^T�z���TI7��xQ�V�JF+X��qV%4���`�/���Dw�6z�_x����#�������k���.�y�g�nau+��T2��������^�����*w����� M��(��'���IL����k�U�.6��z����5�Uu��;��I�L`���/��U6�w[o���{X��N��-�2�
�s{/���M�pJ���T������`$=;���c�94�Qp�|t�q=w��
4���P�/v\���xP.�	L��/ '%�|QoS�JTBPE� �����.)-b���� �v�)7��C5%�a$�@���K��YV)����;{����=��j�������{��bB��G��N|Y�xPT!�*n�'u�Xw7�k���l�L@�~��Tr��+��pp��t����X�`G~r��<T��l8h�(�R��E��J������Z��W6����.E�V���i�J�I��&6�(7�����w���z�jc�5%OU�/57����j�t�pB�cSLVu�P*A�����9�@��(���eu���M���l�C�f���v��r���P(��������7������-w�Y�+���O�3 (�U�*��yv3����o�x���������-�����O��P�����������|���L�^�h
�
;}-�5���5bNN�g�����^ �v��D�@
dQ/feqy�X���� t��]_M�A0�
������y���>D�'�gk�v���Y��?�r:�16���q��^�r nd} ��6�+8���T2M���b�sZb�������3"��@\�f�s���Cg!�bjP�M];*]zY�?
y>u6�G�h��X��o��L�/��mT�mZ3�wH��a�FU6���&�x��������
�_B��
���)�����t�m�w@?�����:n4�U�0%��gp�6�����s�f�FV�\��_=�,��R��,��!�8M]_VI*��ie0I=��L��E}s�6�����d`�7��0���m9E����> ��z��\���[�e"�z�!�Ei�0R�,z�24$o���^UI}/y�(&5%�O��'��
_[3��\VT?@�_^�x��+{�I$�|$a�T�/H��Uc���hO�]�ZlTPmUN����l>��z��<QU3q'�)(n�������kXe^H���&HGXk
8��:�����O�>�U�m%S��5��5��!��u�||.���W������V��35�>�y�����u�Y��ZT!)Hr�s�����Q�����lWF���X��������9��"�"�)�}��Gk��+T�����S�gjd����TB�|�w^��4�=�,��&�������Sy��?�[)�{�
JF��'�(�i��������\a��9P5�����|,���Q7��.�y57����y����b������O���Z�;��i���!����E{��^���}=�x�����,e>�0�o������p����
GXO`��p^��f���4�������������#������A;x��S'V,V4��5����G�D5f�/�1�m�(g�$�\���[���^����,��x��������Q�d�)����������o!�-����`���CC�]h�v?��7��������u��-3���S�N�/,��|������!�#�;�t���+
+Nb:�T���7��s�o����w��zI�l�m�j0�\��Y��]�;n�G��rb� ���4���a��k�c9��y���
H�pM�S�4�����%\w?3��Q�E����`�0I�2�,2�h8�&�/tT$�B��n}7G�������Xf�Y����ibE���K��v���mUbeCYt`�+�u��Q�������<p>|��|q��>$�@i����e�G�v�u���m9���6�o�o���o����~�Wo��jq�([����j���7(p����B�����G+8�8��+������BO�+����
p���Y7m���2 �:��s���Sn��2/����(�_Sl ,[�u���]dD�����
����h�A�ue�w]2��V�$�p��@�!m!P':%���tg_��z�MRe�Xy�%,o
W��;���\T�B(�I�[:�2�Yw?���N�����Yc}+����s��Rc�T	�8��^����������!Y���
Ac���>����zO2�(�e�l��5���RXs,/r�djjZ�����BuF���R�RjU!��G��������Y�}�N$N�X_$���N����4�[�Fl�[bi�i��O7 �^����q#���q�l�����_d�T�aj�������7�^�g�������E��})���T�a�2:P6�\����������X�v����/2�u"�����
�b.��I�]S�2�}j�:���F7h9�=�a���[�0t<�2�@��f>�]���)��<����"+!���M������dC���W���`��[���0{y[[M$��:@��X�	)�WL�
(�x���#���j�	�(��YW�Ow���e����o������SB="#��r���N`�(���}PSbdz[��K�1�Q��h�Cd��;��9 A���g"�N��	�N�/��������b���_]����;����K���bu|�*[�Jlx]�q�o�QD��}������r��D��/a����+����D�I��d�`^`�d��k6([Ue���h.v����x���C���SacM R��6����AeL�H�L�7���`
{u(����Q`�
��u���Y�X K����r=�VE��w���'gg ��9}��-o�3�P!j6�\(���1
�;�W���%�L�n�������u���l��6c�����q,$������5_P��bCWM�@����lF-�a4����7W� �����b5�p�i�v��^M��x���S~l"C�]L��w�&�����%�<>{�����0���$����<�|���'�9ac4��m��2�,�����oD_5�%������'�����a"'Q���WCBQA��L���5?�<�Y���9�iS� C��C�=/��Z����X�0�A��"B�����a�m�q�2:M$�BHO"��^s1F�v+���R��vb�d�'��@X��B�Kk����	`J���Bj����
aV^����NL*�E��,Q	d�Zp�v�v��P*�Y�u�)A����)�- ��K�1S/�S���>;�����G!)��X�0M�b�4���bR2�P|Mj�z+,��fR.�v���v�Z���L�K�
�����&S�X)�P�m��DS�)�	�q�����z��F(���{U���+EidZ��)����k�hO7j�lj�51���j�M!m2l1c���Os��YcK�0o��\{�h�d��W�yQ� ��q����J����6K*7����"�
	
��Q���WOZN����5���f���������ob�_��S����}	��S�1��w���YC_�;��TS�K����B��`�!���Nj��	���>�0��rs6=�?�	�j^HO
�iI�~��=^	ikn�'�O�k�����)B�\M��'e�"�c�xm���\�[ek�`�S�R��.xn�$�P�@=_nd���rCr��n�}��.	����*u�nB� ����E&D��9��a��[�����^�e������U�;*�G����mg^��1���g�YT8���{�-�h[DWI�~H�jQq�Kjde,���o�l��.U!�<��b��j}��xV.����l��(�����u�:��,%�8��E
 f�C���������h�����y�&��I
T�Q�e��SovY��
W%]1�Mr��=@e`�������Re"s�j�i�AU1>~���d6PS9��,��<�~NgHE������"��F�H��3
���Q0$��i���gk���I#MT�]��m$�1�V�k'u��x�]9d��&�S��L�b�����-�z��T^�x��s?���Y|mf�����q��b�RZ���6�
@+���}x_����``��������<y_��:rg�Y�*�N�B�}�'(����bQ2q_l���]�@���/��L��<e�'\��h�E�����[�X_�8��D��Q:����B
[H ��Q�46�T%B�%���:�7}8&�Z�;�f�[?!����������vT%�����sL�d�
��-����=��B�YT�?	��xX�y������D�:������V�Wf�������IB4�����~�P!D`1�vA��������"�S�@4�7����-rP��^jO������/i��Nz�v���u�:��������4_4`�+�������^lf��p��p4��f\i�_��u�l\��Z_��!�NX��3U�����Z5��Z�,��0G[�g<��T�$�zR���j$�c�sX��������
�m����r���/�g����k�1��r���|�n��9�u2\���<*A�������PX8/J�$#���x�{
Q��
�=��6BS[J5��5ZY��z���Uq��K���<�-[gK���f8&�Ca���rZ�:�G.|�=AAS9@��#C�����(�r�I���U��quJA0Y`#��N�Q�-d���V�ZX'��C�g�.1�L���)_U�$Y�O����UxK�	�dVT������c�f��c&-�a2�3�������!o0�^c��U������Q�
H;T�a��@���]@��J�^�K�Y�3��hv�QXk1Q��U`����+�>���`=*7����	�8yF'^��r|�1����T����
���\��*�`}#�#d���jT��m�����B3>��O�!�1	��nsX��nm3��XLg�U4
m�0�(���\���������F�;d��
s�H�.<�.�
{�7����5�.����7�}���L� l�3>-J:�7���lc��	O_��T�d]M��w������)��5\��������!��v#g�����7x�0�M�� Pi�e�u�u�����\�YIi�e�����a���r�th�������F;9>��`��V���z�b�����j'�����n��G���Bl<�	eG;��{�qz�b���p���!yf�"(��;Up)w[J8�����/+m�{X����������
x��u9��B{�P�.�_�/��cc�6������+1��!�Mc����)31��A�=��LE���jh��h��]��:1:��~�n%�qX;
�HE�����R����
)Q���P���,�eG���S���x�w��lqE�
�.!����P@QID �j�w��j���s����?q1�,��J�}��b/���*���6W5�s=~��6��J�B0����2���
)��'���o��X����X�Q���^NP����ey��
����\0�;�z�C{��;���|������Y���p�.��u�m4Dmt�y��u�C��j������^/#�<��(M�
�u���D
C�pO�r��r�:����H��5h��H.BP���=�%�D#�@��5s�0���pf���X�������$����;&��n����F�R)un�(+��-+�\�G#��&3%��4�-�t�=�&+R8�_M9�B����|�\��+�T�r/q$����?R$����%:b�{q��nW��>��Byct�J�My��rp�^��?�6N�y���#���j���������	����O��������.U��UW��LW�$O1 �#��9s�w�;�}XL��-qhSRi2t�*��&g��\��[n����l�_��x�*��=�M�2�S`�����J�2X���@c_N����X�^��D�c�����;��6�C�Pi,�f��XL��4�G]G��gq.x�~nb/�*������Lb�2t|(�u�Rl��2H-�X-�N�-�w
��!Xf��Uad���`%f.��*����j�*��N��V�_�����!#t���a���l��lo�Hd!�XiHJ*���%T�sd�$�}C�BN!����C������c���e��^�p����c��k�e��8������v:Q��������Wo���z��b��L���*
������*����c�����0z��M�B;�r9��t����^9��P���p��6�Se(��j��2��Nqi����}
4@h���T�q!p*�"��L�(���4��p�����P����c�0Ut0f:o0~\�f}HY�<��F9q����S�@va�&Xr�Y����%�s��������qm�8U�r�Q&���CC���5�Uj����UE3����}��F/��$3��O�yX���?�S_�z�\�uU��~�&!z��R�Q@*C�g�����O�^��mk0 ��=��,�����1�U�
����=���!��������z�,�tW4����/��-o��C1n���Z44��&f���	z���P�qx��;�Y�����C�I�q���:���P
�g�u���y���;�sG$��%�"3���1���S-[u��6
0�
�l�}��No40(��}�q��(�+/�.��e��7���u'�!q�eL	���e�����=c"w��9v�u��{@zLV�N��d�D�H\%Qj?��
��������om��p�j/�[hx!(��!z���?����8��i����*�)�� Xsn0�M� �X(�1�r��L�e2�Jp�c3�1�&��?*�����82�'��O�O�5�4�o�#m{�,��M����4����sB�s���o��S��	�tY���p>_u�Q����;���cp�[z8�����{Bu�c��w�;7YzH.�T'����"�buUq�����}>6��e5��4�1�t�/I=$��,<�R]{pM��2��9��F+�r�`����3c��X�Y���W)�����=N	��8�}N�/<)���#�n�����I���EI��~y���w?+�%r,���(���~'��bm����S�(�	�4��s���O�J�~�aT���/��UE"�&1���V-�������M����j��2sc�%�$�%dk�_ZH��7/e�i���I�p�h�*�����Z�9l��yZa�o��rL��uM~H�RSk��r3���[���D�v��C�������-�D&��V�B^�;��R���Q3�!T������C; �"�m�>[1�>�8$C�G@GM����e���b��k��m3tA��d������6������Q((2eT0!A'�:0��-�������
��sq/7Ry�����;�D�g	����w��g;}�z|�����s��z�P3i�yY��H�|��z|���������>:.�N�t<{�2}>C���T��������
���~��������HyO��,�@-/�g@���(R���]?�.b	:2�lM�{v3���	[���	I�2�����
x9����9d�9���r}������^9oJ5�X��n���Z���&0�Jw�'���?o���������o�d��P�o@g�c�������L2w��\	IS� -k���~��h�Q�=w�Y��6�����z�j����T�NCXr�3As+ �/5��&�F=a��|-����BYl�*��d�~�8{���j��}���Qb�V�C�������@��n��N�|/�`�G>�r�1��e�A��= b�nu�
��x����x�%������Z��\�
C�Q����XX��X������-���������7�1�^]U���_�7��qK<����a��'�Nz/$��k�������o��|��E���R�k,��q-y�m!;��� j���M��7����.���N8�����"i���^��+eX�%�1A�Ph�_z��\S9�)[_B�����$//����c$f��F	���n�lMUv�Ix�q�/�~>������w}��w/��{�c��r���i�����	�*��+���{���S���<�*wf�^���J=!�5�*�����7�n�w.���!F��/��h?���&����<�`5&�<[���t��:W,���+2����[ ���1:��@�����}�e��O<���INtV���0y^~�p��������,A5�P����q�zy
E�k�1��� ���vW1D'�(%#��5}�����I�s)1�3�(CO	fJ�~�`��Z�Y�O�O����E#R�C) g�o_^����@bD������Xh�&��Rp,�L8Q�J��n��1����9�Q�k�1Yh>Z��qbK�^��de7�M�Yo��o��_���V�T=�f���bc���=�&s]�q���������d��
Ee�16l�rbl�h�!�����w�
��7��sIdMlc/`B�8�z:j��1��
�jj�{���B��g��y��[F,dbv�h��c�#��zV�A���=�s!��^S���u	�&\�A#8�cj��p�1�/
���A]�	���l�h
_T:IR8zB��W|�����q'	Z�����v9��?�%�Y�C�z ��K�P�k[1��	o?��������~�����W�w�
e�AdA����lI7��$ru���[z���Z�;-�i���k�zvk��4����� �jR��H��	�'3	~����4�\l)_��
�����
�(�6�#�?����^����1���x��c��3���������?��
(�#�� 	zs\�wr���}<���o;>2H"g���A>e(���E���1����V���/=�I��!h�S���������MlG`����:���vr�v�t��OZ_��~J:�D���0�#�����ZwQ��2+sh�*���������\l��sq����z�t9���Y�,���W�+���/�����������d��y��q���v�*5�S���CE���Wa�O	���n�"G��,~e��Y���S&=����Kc���^���O��}�h���g��Y�Q����8�����X�>B���8?/����!�F�G���6��(*������X�/-����Q�/��/1��dHs����^��JT����T��jq��r�c�m�TB
�"���FdP��	�V�v�-����6�c*n$��7T��,�2,�����V�M�	��{����ZEJ���.�f���-?Qm�.��y)��6WJ~-7f��n��&#Ii��n����=�:g�xtV.��P>k�0�hVS�x=c�H7�������e�������1a����� ��J�\#B��6/��q�U&"��S�)�i��wO\���.n��������:��{Y)B�e�����+]
Zl�xyx�������I"���9d�I�1��C���l���2C+j��TW���*~���Tr�o�h�w�,���t�������STgGAD )�R	���g
��W���}j����wEEu��*�����\*#G�~�b#����a�m�CDI]�6�1?�TU�PF/���j�gs�Nd:I�?�-������$���o������/������K�7�9�2���{�0�}��	/"KQM�$������8�����re���PTC[TJ_��n�u�o~��,��
]��-�F��u������H��n������]���3=��$XI�yR������p j1��s������YV���BRF�gx��[(�r.��������z���iDk,2'�Js�./`Q	�&�kwJ�8�J	��_j��g'
��/�W�.���cVF#��62))���r?����Y������nD�%
��"c'����n+J��\q�Z���t��$!�@&�R:
��@�5���<� 	C�����j'�n~���y�yq��;�Ak��������I�#��^��9��P�q����o���7����{���������)-kd�Q0ZH��iK�8�����=�V�c	<���q���$��������N�7|n%������"dl��GL�����I7�|��>����'���'�wh+�����@>}@��W�oWP�p \�F[��F�%R�F���>J�5F�e{��mL�����$-������	���$��wBqr�I����[��Sl~��GHs�3�������e^T6wU��i�z���46�����k���1�.���������c����/A�Zp�����(�.��<��j���_eW�K�	������`�l��4�v�����(���"�)�G����X�R��<�
e�5���g`&+ #1����"���c�9��@��6=qg7������������N��,m��iz��/��,���@ f��1F��L+��(f �c�>1����-�B����HFJP��z^������f�J�e���Ha�p�z3��7J����i	��7��,�a�Y1��JX�A�A`���s����F�I�B.�\��V�������M;$�q��v�O�a�y	�1���kHF��\>o�M���b������0�-��PV���t����<�W�S���ve8�I(�9LR��B��}��.>'���q�]6G�b�Y[*>	��	��b���{�XC����[������GX�R���R�����hE�C�u��HC;`\�j�d+bCG���~��@_���!
{U@��y�H������T�r�x\�rM��75�X,t����e#������:���w�e�&/��H��Db��$"o���#2i��
_�"'�M�;�Y�6�8}�h�����_%��$/x����!Twh�A�]nxB�(_�Xc�]��\��?���C��rQ|�r&;���p��
A��L(-c>�$����PV���*��_7��v�J�@�>6��GGB~�N�����C#�'O^�;���Z9�w��0�3;'Pf�b��'O(B����g3�>
�Ma�����2���6�DO#��A�1D�2%��h�A�2��ST(�$�t�-f����#7qS�e(t����;���
�7�d&�Z ���$b����~��C��n��G%%�Nm�t)���{.g<��4������1%<T�D�
��@z����?����eg�z
�����BE&Al�k�r��&����
6�L�r14�#�z�����LP��2Qg��,I*t8�r�Cj�H�)� ��54x��VE����T�&#F��s<�j=NP�T�s�	���#�G�U7��K?�2\���f�	���3�����C�<R��B�&?(���TZ��v+
J�u8!5�r&I�J���+7�����u�N~��N�����SN�Q$�C�������~�"���B0��w@��������I�f���[
}����}|���3
Yo��t�b��t�p�PX����1wI����-��d�\:�A��q��tB<w�����`e�5i2�#�\���	��JUU���f]��Rv$�;��6�*�i�9����Hz��	����[V�%e��Br�Z�T@
�K��d���Y�W��� ��U��8��1�	��f�U�Uh�<i-����}���v�4O���u�����_�*�����J��,� e�[d���8���^RJ����P��VU�q��r�9��SF1�[����0�����3�����!���q�,���7P��gN�Z�����>�QLu3�[l?a��1	m�B�Bu�����!{g�:��"P/8�����X���X�f�XC��)������;�n��]Ti�[�*@���c5�6��O�g���=:����%K����U��x�p�d%)�'6,@f���j�,,5��#�V��^�'�x��}v�D,��K{ru\��w+_�dL��^"���F�������a��w�_���������w'����������VS�s]��m�}����w��/����`p�����G��/���k)�En��� >� >���S���E�E�
�A���T����wK�X|e��l4���>~���;�,�[K9n��h	�j�v;�gt��x�s_n)�s�.?~�n���t�/4��!l�S��E��+�������V+��A��d4C$�\8/|b(��t�R�����d��@��Abs����'��1	&��bH�f���u��#�)���b�1�����r�k���)�������N:��6lE�Cm���g"�VJG�E\������P�BJ�)W=�5��lq���UdtB}��"�Y d�S����=�a����|�+n�f,#m�����|�d|5����9���d�1H������#W�H#��/�|a�"(�F���+$D���n8�T�J�ce���w!�������T��:��u(	�&W�v9�"�b){�XIH��s$�����"�Iz��v?L�����'��f.1{�hp���/���3��\^���'�K&8v4��Y~%��N��R!Ic��}u����O7�S;=�%��U�&�^��uy�F���VzQ�y�)T&>���xzR����n4�Jv0�����L>���w�]�>v�����?�O�1�n��7EB������FX��:y�����~D�z���: �������\o���"��]�������s=�
U�����?�d}���fLO8��L���`�5m���!��|/A�M�X@��/��H���n����������v�T�gy�,���j��t������+SC��#�lp�����Yz��T�D�Y�G��+S���^K�Z�|�����A�q �w�-S�|S��r1������uC�r�y����K�Et?����a����k`����uI-������?���4Y�EO�����z�S�ikOt/�'w������1SFsv@W��������(4��9����n�����B�����h��f�5�|���E�2�ryZ��.��DqBE�~v�p�������?�� p��z@���N���1��O=#��:`uS(i���4H}����u�_7��W�K������F�D	��iLZ��������d�
�d2��a�w�T����)��_{�����&$�Vt�������C&?�k2!�@��nU���n'��
�1�������`1��
�D�0��������zV7T�EO���������/�DFK�,75q��"�O��=����X�������,U���A�i� 3	I|x0\�(� *b��(��o^��pL>�V�
W�H�/=|r���=o�����_*����-w����v��yq/Q��D��&�������z���YB�!�����^����w�.D��g�n���n|	�|`���^vQ�����p�3j�����R ��
Fad	��P��,�QQ����:�������j��`Ta�}.�/�i��&J	��Z�}�s�44�I�N'���e!�PT����]�(�n
�RZdu6j�H�f+[J�ys�HK�$9�*��y�e����>�DB{��� l�:\����0��C(}%���H����0~#���M��C���bs���,�y�uB<5�S���a�;�u�<;��+
�J�%�&-L������t#�g��sE�F�G���D*����"[���i�/��pe/�%��6�cO��9/D	�<Vb��*P\
YS��p��kh���q\k�V7��+�kL���(�x�|�0���l�Q�����T���Z���N��	\.(n-��~lv� q���P�6�}���f�q��
�g"��q'Q[����U(�����@�(�����j�AD��,��2�vp������b��VB9���q6��-_V���k�!�B)���:��z�zz����L�'N�(*�r'��=��I��2���@M�"��1��uq��
ue�R�e)��.�g-�D�:�FA�������:Q:�"��D���D�%�����w��{�AG�''Xm����n2�NEM��ns����%3	��WfOk�9mcpm� ��b~>�;b��������q����BL�-J�x(��R����%�(���O�!�(G�i���4��%SH}�����-)r
*��!x��
�[-��������e)��Le�jm���5+W7CA�*�@�D���(���}��3#�6���n��GE�/�-y���J���Zmq����Q��Q�����Y!������v�R&/$��YQ�m%3�I��j����d�C����zs�����?�}@�����UV��2���"u�`8���+���yn8�&v
gP��-����%���*��d)|�;I���S���mQ��glK�K����������C8��TjG��9�1i�����n��M���O� ��g�h�������*�����m&|��>�s=�tOG5i�,�������@����m~{���O5���`I���M����H����_��f�] V��l�}�y�U7:Q��p�%B�)+��b������(�x�.���f��UV�[w���	�7���2r1kH+��@��`�G�uW��99��J��q�$���p�h��$�)��K�9Pe�e�U���'�m�K��-�J�V�J����.�A�-����J��A��S��R6+zf�A�V������6��m��>~�)D{�N\�]�����'�Vj���?/��\e��J��X��Bk���n�t�B1�����>w��*;�]C�w��V�"�AB^=��'�Qf#|7������
�J�����5���"Z�4���G��J�����B���;���RT�I���c�x�����e�	Qw��4�D����H�E�A��/�jd���.v�T�9�����(2����T{�9�;�>pcZ�{�W�%2u'h� ���R�������D�#��Sq�D�#�\��8��h�8�|t;�r-0����W��u���a��J�,Fr�#UD�������t=2�-�2j�cM�<����:�;��h���(�X�� 3����9e\.�k�#�U���l��s�����$��\a�w�HKP�{��E�9C/��!2�����6B�d�����P>��� �����HN���dcv1 b��!����llzP���E9�HI6�/D �7�P��~�����c�<������F} ��2��w
�%��@��XX�o)�����	4(=f8��p�]|4�tt7$����U�hX8�:P���k#�uV�p�;/��.,��G��ms����
��{Y�WO;���L��4j/������Jg.�L��u&S�I;��G����hb��N[�Q�`h�s[+FX���S&K&pz�Z��M����N�$���@���=d�J��V�@6��>J�ME�����K�6�E8��8&mw�����C\��u����s0|dx�}.n���r�{K���
]�6|������w\���RI��y����Q�p�
v*/�����y	Us�w��v��Q�l��<���b��R��se�?�Gp�`�Z�s��M�h�����-
�W��AI���������P^5i������Is���#W��i*2!����w��r�)���K���'�������W7� ^K.��N^�_~���~�����G'���|�
�7q������+��������yj�a�.vS�<>;IF�SQ����']�u�V�����DQ��)��P$��>l	s��LT�oDn�F�B�I���������VzH"�],�n������;5p��:�I#e�-���� ��Y�c<�����ZF�����=�������uP	+X=�&���g����5(�1�&����X�������Kn',��'��i��N�{�����Z��Ihg�p�������F�F���RR\�9K�1����u�q[xh��M�R�-V~�8�CF���^g���5�-��hcu��i9'�9X	�R_	d>����1�'q�'J`�V�#pl1�������O�-���/?�$�C05"&E�&T8�@W�K���$�B�1�bA1�mM������fmj�8����� fC��2�8���V���l;���i�6� iP��{8��H�A
+��|7nx��M�!|��|���q����Q<E����u����v@��W�����c�;���"H�X�,�,)�o�t�]�n�6�����Z$�(Sf��!3��$���Tcb��J?Q����=����@�wi<���r��	�p0��@M�������A�G� �\;��y��{��o�������/�"�&������o���E;�����G��X�j���05���m��Vw��u	|W�W(�����#l�Q#�
Lz��I��mv�������
(�CMC?���&�	���������9���4���M����.r/��_C�������q���r���1�^���TYl-�B5����oTPs +��[����?9}��|��:�jm(:�D��5�V���j���~�yI�c�X�.�����r)��<�o����F��yBO�Ho$�-�S��"X���~�����
��0�U��n]�pE����}�}��gZ��G������l�vl_nGGq��	��Vu�i�K-HdK�&n8�"'IgZ����J��s��f��'�H��������]������0!�vXz�����%4�{��[��o�������������(�A|���x-%���z�kPG�Hw��j=<�o���m7��G�@{8Gw�}��a��&!H'C�.
mz�����T�>���M�Y����F�A���
��a�C���J�J�����`��dG2m���(_�����������2i�F���T,����ga�����h����*Zr���F����L���EY~��U?�%�B�6�c`�;�
����=`9r�������xH�)��w]��}���k4��j�\#�"��y����H�ZNl��D�`����U b
0����0�^�7�ep���������\adZ|
C���A��������\��s]�(�N��&���.�"ms��s��tN=o�?��EF�U��7��J��-�v����K/�3R�����5����7���A-�������a�!_e�!FQ���X��7-�E��QRaM�`���P���#rPq�y��1��t��J2`f5~�2��>���]�����d_`8T��������d��e�qz�P`��k_v��5��.>��Qp
���?���F'V5\~�g���U"%/�
h9��A�)Kp0����1�U/�{�oh�O��f�=�iW�N|���	�8���%������Yn�@���WT��Y�h�Z]F�$�Ewc��2v&�����B"�7�om�o�s���B��CiX3��7�~�L%�����:n����L�$:��(gDo$���3"���"<d?
������$F[:	����#$}�-zB����� #�4�������K�t��.���#�W��*4�uP��k�V:�I���8Q!�3��_���)�Rf���	������C��S��=��RJ�-����Y��������	�W@�Y��&�c�s�0�_~]�6����_����cGQ�{2g�=3ah����!;��!G�`�P�B����5L�%�nH�t��r]=f�\Bq��H��a���v BM��x�������xA�^�cf+�����;����������������rh�Lo?�:kq@������������Y��g���A�-�]G�%��z:�Z����4/���zd���B
=v|l+.G��4094}W�)d^su������W���^�U�\��/�G��������J������r�*p#�YDK�����)��hPq0$�������1�aY�>e��'���=��j�P�*��f��9���B���>���^�[������_��T:�����S]L��4���tG�x�,p��g�����sr�B��m��yc8��]6
�P�2*���j��8.�r��wG�-��T$�+�E�|��e���XG�k��b��n��Yv�c4N�<�f1������@/m(-���G�)��W�������
���o|�HB�����z��?X<����8?���SM
�,�����~�M�E����|[�R���}�\!*����\��@���#q���s�����9�A
J�Y�����R���["��s��Q����;D6�����(�T��f������U�������,��)8���������0����EW��VM@ ���Z���T��2Z|���Pi
Yh�R��"|o��Ug�����������2Z|�����n!�H�	a
�}>$�|��r����^��*2���o����@�T���M��
���Q�����$���[@2j:��ZAM�!����|`�t�6J���I��*�PQ���ej�����N����8�y�N��BC�gD��	�%"�B��ap%�z+j���0-9g�':��n�za+�_��5%Ijv��	�kN�9����H��,!�uy�:g�e�b���Zq�����gN���mT��$4]<c4S�!�-i�)n�T\q���|�;�
�3r�yu��]V�c��`��uV,7���C���2r|���j���8��;�)�=9�������){������D��/,{����Q�<*~�(_�sZx�cT,W����x@~���^��U�raY�T���� �z������5������xEfmj	�y&�W��1��6	�c�+3��=�����6�~|+a������@�#|v���/H�l�XX�� e0��K���[�O7Y�����Gv����'�N�i�e�>Fa�/a����4��4�{
CP�l� YZ�����@6����T#X�J)���
�z��6|2j��^u���B������L�(��"�|Y�����!#���nF���s�=���|g3����vo[,d����}t���������dnE{/��O/��61},� ��0$'(SVD!���1$��?�<�����OQ���7.I)k���>�&���WSNk8NP�z�"B��~�Pp��*)�
�1T���e��`owS��\��k������F.O�����8� ���2��	$�X��:xj��U�N�8�W
�I�
S"a���[�*e�t0!}��y�\�*���RI6���V&+���)2����+Y;nN�V�0��et��u���MX�L��t-2X�`u�������l�BH�������5U\;j��cc\���L��rQ�Vo���_�bZ����og�O��b�����	Z\��V�[��((�h��Y���{�hx���w���I�$h�!`k���`�wqL��i&����QG�Z�;��Cc�2�s�d����Vc�az��U��Z311)�q �7�X���/�
6$�.�M����\T�������/���>���KzB�
�2�$`���u#�,0�00�jg�b�pcpZ���n�kH������d��W������Gx�,!0������y��Xkc�N�c7�)��2���v�%�$��#��O����
��'�D
��������Z��q�1���
R��Mw������h>>����P5D�V�j�u�l@y�������U���	�����`\��?��9x�FbG�+:�b[v�����^7�����A�0��2�x����������@,�8dXS';}F��N�����%;����	�<w��`c��>��X����D�H�mx e�@��=�y�u.:|h��Qe`oD��}�	��m�d-o���8�L��������"V��K�I0GX�
�C�jd�-4-K�
�B��Pj��6�$:�����JPa�����V�e��Y�nn���w�{�����d0�9��	%`�X�+��
�`���9������]7�?�����3[@�3y�����YyWQ�xMe��#�����bCu��~b
(��� ��1��jw1 "����9���$v��e)��l�G��������>2��H��o���f�)�M*�2/!A!F���+j�	������
](��e�{}�����:6J@q3����R�.cg��{�}��T��5l����0�a�d�m���D�s�p� ����$[�����Z8��wZ����$��CX�%N�	$�e���O��h� ��\s�t�a����82x��@Wh�~��|f��g6���O��*���j��
L�Xg�x1�Q� ��do^"�����������~�G���-�i�~��q>�k��4��M����,=�������5�lRO��)J����Ir�U�X��Qe�|N������b�SZ.�E��>4L�,?B}��^
�t>�m��Q����~���N��L�#�]����v5��r90��.�~�sIH��,?���r�4��1�L����{h>i�:���L��I��u���,E��C��������0X29R�9&a��F	M�4^�4��az��f�����`aSi����m��472X� ���)�>_+�dCm��Jp�U2��&t�����b��ef	y�k(���p3S��S{�w�����e���{	�pw���LF�����ri��iC���=����vc���5�z8s������2���'�N.|�t2��,B(�1v8��s�;f�a�\'P_m:���m}P$ �l�l�p�/����^�����>SE�z

��N��r���?���f���sf>��f�N��[8���}��
R/Z� ����i$�@��z���@)�j�����5V8�WN��"��&����"{�B��c5����%����wTm���r��'�zg\���S��hIC��%��>�2~�N����h�5���2���^Q����{��h`zi��z������l��-0�r�W�XJ�WoN<yR^]��8�D����|Vn�
1?��'O���������L:����<�K7�������
F����!!�y������dV��
G����.sf�������.K8��K�.��'����|)��Ok ��d����t��]�}d�wX�Gf���[���d|q<}�����Gr���HCN}���������x�dB���2�1|	^��r^ �6�mS��;b6���@�o�]|W������2S:�{���P��r�"Y	���t���H�qF>p�K���Z�Y��q�Ztq���WWJ����p����]b��r-�)|\�>�T����R�����$�Ra����'�K�bvD����i�P�eMGv�O�Q�@�(���?l]��gg�����'��R�i��f�w.��e�����\��l�+j�OW��P����bAP�����
�V�[�,������3���fN�������
���C��J���K

.9
!V+����I�=���ZFM'��P8�s�LR����I�c]h�;%x�n��M�ZT���2����QF��7�����85�����H����������0p��@�/O�����j����\Y�9������'�S(k�
���ahp\�Q�
�~�@�)�]�\�I��YOY������0b�e��dt#�7455��M��aWqAH�H>��d�\]����Q7��l������?�i�����v��s=;��*�l
�j�o����kQ�[A��e�(6���!W
vq���`/b����A�%<��D���e8��0�5����?�T5�����%�+��(H�T��T�d�����W�!S��065I6�c^DLWvI�M�� ���:
��
��������J��j�������Mj,���;\���}�C��X���u1x�T��Z�X8�P�v��������)���{�mj� �����y��<������b�[�:D�6SZ���r�����;tAB�C���{j-�|��"�QY����]���P�A�0P�9�L�+l�%���\S�PV�Ck���n��I�Y�5LF���*�a<� ,?B9;�\W�[<�K`H������z��!�^�Y�Wf]t��������`�W��V	�d�(�[|����]�LFg�ck@�Q�[Q����y�)��W��se^>��j$H
.�4,L}�IczW����P:<����;Yp6L��3XV�����d��W�������\M&�d���3H�T�}u��x�����og�������>|8=�8~} SXt���'�Ue`�������*=��B���G����er���U�&Z_-���}K����9���fz?���8>;WO����:�p��h��I�y�
Rb�7g-��SQ�����ke��9m�Q��f�����h"���D��6��3_e���\�0U����t?-+����8	�@/d�"�]�&���5�r���yWW����95�����S��M�b`�M���6���\6�;iD���I\�?1��AA�)��f������e��!;t�_+6[�9VBy��v�	��X���!��adM�hgY�(�z�=�<E!k/�Dq-/�'�B8����X��6�n�-�{��O���������x�����H��-����_^_�txUB�lv}��Q����+ak������{�]���������m!�D8R�����)��%f=��a�
���#a}��`V�a���R*��|����G��l_��F>w/-��fXUe�9t�7b�a���������r)���P�Y0�+��
���3����o���fjC���tE-�X8W��"2F*��HU��*ls���k0��b�F����E����C���`����o�cO(v=����b�,F�`���;����uqk��Deg�+���v������7m^l�����:�`����������� ��`i�{>��������q�FF����r�����GN��M��O�Z�P~�PT�(;�P�����(�L�����Q�#�hx��7
�`�b:���Q^d��(��w��t��#M:#'��u�(#�h"l3�`������jyN�X���E%���B���Rl�����X�TW�/3m�7��}�\��2��H�7a���EuC�fd�iQ$�^4 �:��(j��A�]������rPi����)F���*�@��Q9�.:��Vp��<q����fGp��.������	+%�0	R��?�}8����S�F,��w�%�2r|s������h��e^alP�x��pY�����wVF�/��k,?��K�{�*���� ���j$��C���fQcz[G��&��"#�����^{��%�+`�B)��I�#�����>T,t�)��B�(����zUR;ET�C��*B��Op��C�n��&#�y$���!���p�������fb�f
X��nCJ��*��84�u���}�ge�l ��/���UN?#��<�]~����w�;��FDh�=��z���G)�K
$4O0�_���V}J�����8y�(���D�6�^������'���w�������w�G�1S�z�h���:th���Y�������H��@�_������������bj1M��V�F����-�`NQ��_�&�gC'���A3��$v�Ptu6���h.���(�ZSY>;�S���E	�N��<_��Pf��0�3Y�M���tU.�X7��J�1�wz�))��������L:D�J�aP���2#�=������Z�9�����MS�M��`���hfic��?���~o�������|��r{km�S��0�P�l{+l�0�������i
��0��,��(~����������x���S�!C�x�'�vX��w'�|��Ks�^"�F<9m-�z�s)V:��P���H������*����2����)�/��X=)�V�X[	R�D���\)���G��9�@�K-K��9Yk�H�������?�:�h�O����Sd�r�(i|�}���=��2�x��3Ab���Mq
��`���:+� ���Bu������/#����z����*���E����F����^M=?P��d2r|�(sB�9��0��j������P���y���w@{���&�T��crRC����A

p���������{�I
�j��rC�����	�Z��z�P6^@�v�����mi��[���w�o/$�^s�:'�g��`�K���'&d����{dHL2���-��!�`����U3�h��I���e��g��_{��V��Q��H�
!#�"#�:��L��3`K3@[-j�Y8�k���L��/��7�����~	K��vU9v�YQ����S���M9q��y�����j�����W��EXP(��uA�i>R����K��\�9���O+S�2�'����+h�*�+��sC�-��A'Fzc�M��������@��t��!�y�O_7�o�R-�n����*�����:����FkED��X7�,���A��@�R�kb�B����c��J���l)p"�JXQ���;�i� �4���.����
p:���94������?<��cQ�mb��Cd�W�Z���i�]���Q�.,k��(�e����#!t�@��fk�wR�e�n�BMs����86P,�k��m(�r|E0h�N�'�:.�[��U'�^�\T	��?M��X.;6@����3�apN5�Q"���@I��{��_��v����Vo�9;��5]!*��u��*��,���O��� �0���B�>�I����;� ���#���mv����]L��w���WgvS�0���7�qU�H��#�g<�qQUb�3n�JX����F����	�`������J1M�wfp�-�{�"����L��a��I��He�	#�p�&���l+GW�$�2?�.��W�K6P,�\���b��D��@���<.e<H��/"���{dZ�X>����t���Db��$�C���2w�y��U���BF���|��F� d7��0��AibK�c���k��;V�p7T��H �����#9]��^+��h�yJL��1�������?�q��Xx1"����v��u�=���4���R��;<���������Q�*�u��R2���t!��j��N���@���@�bs����?��|ZF���<G��g�F�������l����P��y���(M�9
NIP����k5P��uMNe�Gq*�B17+mA�4kqe���@���+*��D�M	��6��*z�8Wn���#^���2���U��{>��%z����7�]�
Y8������>ID+O���ed6}��
k���j>�J��

�L_&I���,��9��������o�02�H0������&'<^����R8�(��Q�)�C�����KYJ,}�g�V#d*m3N�E�m���m�Se��y�MLv�v�J��v��p��a����f�6U)��Af��)�D�*mN0HX���/���Nsd�B�3�D��XD��"q��\hl
�H���F�`��~�G���J	�K���2
��E�L�'�c�_R�)��20+���q��CAS��+�8�X�/��j^���P�����c��L��j�*I�T�v��/��v�3�b�
��I|q�\���_�0��l�)�3���c��i�Q�2�vWF���z�u��7��#Y��]*�],g�j�[Q�u~��.�0�h���s��u%': ����,t��4�o+g��dq����2�+(�#���������x�@(�~����@�@��[�O|��+��k_���C��@�:3�����
�40=E�tP��A�x�q���N�|^��e�������(P�6�G��@;�Jmz���m����4W��t@�?	^�pcFS�l��dL���}o�0e�����&�=Cn1��(G���	��a8��{��)'������ ���v=����QNk?�tS�]��3�1\�@����(@��
uU�����#�z>���E������Jq&}�=z����*�W����@�Tc����5|Kn����E/��_S���TN�T�E�T�l��Fgk��a�������SN�M�M6a}�xEdK$Wn"�r
Pn4��o�)����jm�����v��(�$�t	"4�dkt��]�������-B�:����rV!8��Uh,����={�����x�^0k7��X�9g�I�5�������j�FZ��� :&x���U��R�P ������(�\!Y�C�������oQ�
�
�����x��ol��|���iK�|�]�����:��T����]N�z]��RV�huC���� 4 y�����{�{�)����a�N��D��B����tx$���v�=W����N.�c�|�
U!��F���%���E��]W�T3�M���J���U�5ZA����~G����N�U�&G���
�[T���uO��N.[����L��@��Cp�0�-l�r��@���7�,����:����ZY����h�Kr�����&���~��&���9�S������7��������C��T�=){�e#��{G��f����O`��f��#���B���n�T�m4�M��N��j����b�����������C�16$dR�7S�X��lQ@y����j�H���w�	~���_��{@���y��,?�P��r����VP�������� ���t��H)�K���K"��:<4T���Z��.YL(��zw�������r�T���S�����8��N&2��V�!�HF�MHJb���^������^n�7)-�4���Hir]~*�(�z���N`Q�7�����([H�������!��������X�z�P����Z�m>/����7A��F���7b������[�kj�3'D7j����������T�5d2���mvk��7[�}H�y=x�7��'�_���������]��@����_~��#�
SO�����C�z��6�I���QV��]{`��7����q��m�m��r[��fuCiSQ��4���RC)�]+�0�4��B��������P���qmJ�fy+�u/ex��z~�7u�����,eyLnV�7+Y,H-�����W�$�:��M[9i���M��\>�=���;�� ���j'���������1�	9e�g Te����
��{�W9
39���|C�Y h����D�!u>��#�J�R���f\4T�
��pE��%���E����i�6���\�MLbG�U	�7��2�4|P��f����w�/��>��@��
���$�4L������j$�����r�
Y���������.��^�
Z�P�F�?�UI�Q#W�V�&@�L�F������US,a��S�\����
!>��2!%J�%vj��|����3O-&�	��l���Nf�h��a{�xS���K2)"�#t�b]���gVW,aW��r5@c,��&�
"���8���2:��fY�K�]��G�`3�h��?vi�R�G�P��p�<�=��������M����HA,��.L�,�j��������\��V!�G�R��@^�m4G��6p���`��b'as@������yw��F��03Q/��_7H����M!����$��yxQ]jE��=����O�=���������>%�Q�;�6�t��,��aZ��N���L���\�@��6�.�U���L�������1�&���D���Q�r�f_T�=�~3MM����7��eQ*�7�0<��������oO�������e}���������t��'���} �l
f����;o�����zf��S����F�Fa
^|�k�R�["#I��8�~���J��o�������O���	@���08���������+�Y��za���l�W�v����c�8&Hp�{]k~�l�[��M�H�M�bSt����}�m��2_��B�3`�_Cs���c2"D���J�3�'���[����*_+�.��F���{"9z��1��C��
;,*�{d��BQ����n]dC���;��A}'}��� ��&e�-�.DB���S�	{Cq�F���lx������Yy�~�.�X��a��B�QZr6G\�+}���s{�6T������Ji�����za�_�f�(y|��M����w�o�.w�����
�����"����8�xDp�wj_8��TI�h���-Ey�_�hqwj@��E�,���?�4�jL]/L�����%\��^��6�a�
*(S�����`��u�\�UT��t
B�cvE����J�05�6��I	��}%>�S4#���spx�4��1��N�J����D8F��z�W����V��2���-� �vqR��P���H��6 
T�<}����bW��/���@�u��:r&�3A,(��W#8������<����]�`;�C?�?<*��aZ��0��j��
97����I(�������Q\��tN���]����ke�]��D�{thn��%C�������H�T���d�Y>5-���h?�J����������U���_�P�����)����r��g��"_��"��������8(��n����<���\�P�3Q�uTC�H�C�x��X�S���\p���i��������_�W����s�����=2����og�O��B(���!�a��)�I
Q�	x����!��Q;N�:�^��	j�I�.VA4�j3��E�A*�E��-���=0+a������{��r

�f��$�	�NT%��yU:��9�l%2��./����x�bQ��.p���Vq ��k����2b����N��^D�2�=�"���O�F��ZV%�H%�m/R� �*�q:D$;�+��#��9�����H"�I=Q�M�����0����P�uT�z��N�uO�w�>���l�����-��,)�$[���"���1���7iT�)�������"biuNF�����^��#!���G�vP���c�r�\8�m����6-�������	������a��b�����FwB���R�z�M��u��O����p�FM�$Z��l�.D���=�.D�d�2��;^�%�78�P�LU�;�k�S_fU�E4v_�JPY��Ze���M���|�#���Y+�[�Z��!����E���%��

�!z�],J0jw\�cn�$�QQ_!����0��8�y����+[�UEhp�j�/j;L����:�J6�	P��MY~n?�����p�d�-��e[-bP��GT�
�~�����E���S��.���H�p����Lp�j��w��PnyJ���n�
����dE0���B
�T*YG�J��z��rC��=��� �?������9`%c�������_o��A���lxD���o�����r�%�ec�!4{`�kL�(�����E��:`��&Y2��!�\U�&G�{V$�~a�S/��'a������f����'�������Q���hC`�r�q�����?�n6D���&��3����GJ��f����"-�"�~�W�bEs1�.l��Y8[%b��z��`e�OF�n��h�;������|��R�����k�b��O���+�To��&�����O��~z�U7�;�E�\a���W�������1<=��>rg�*��r�{!2��������q!�{����c�M��r{1>���s&�&,���U��=�S4���)�bu�yz=�%�K�v�����E�)��y������1u=;�i����g��l'{�j�������}.qb(�w42u-�\$�\�'�
l����G�����������Q�3��h����Ol�z"�������T��,
.��G8�k��-z:W��h��")���Tl�GB�<B��y	/d������
P7��:�G@�W�>�*R����MQ�Jm ����r{;U��T�	Q�wv4b���qX�S�n��:e�H�0�F��972��<���CfEZ�����T�s�4\D5���Q����xn<��Lu���3��&I�����Hd�S6�:�2��6�q�quQ�/�rM�-�^F�R�1�J���s�h���,��p�j���6�]�P��B����E��A�Z9~�6��AP���]��	f�O��T�wQ�R�,P���_<"m��G6d�pN�����j
Fi�|�y	�7��1�	s5T���6p=���Y���4r�_W�F�}�v
�4�z��?��������$���s7z~
�)���8��d*���w�Z`�eK;�3!�B6y�����`E���=s�h9g�lq_)�+:��I%�St<M�j?����T����`��]���Q2�q�}���eB3�����
Lg]���SE��!�V�y]o�u���9��H��������C�p�@huS�UJ)[�k�;'w�}Hk���JK!��|�tpI�k��mt|vvz&s��&�9��k`B�������7�//~;;��?�����������kY���{�9j���z�-!�Bz[]��X���V:�k;�7Y�}{r��Sds����T��r��.�{\���>�W�<��@��y1D���e����9��������-P�R����mdzy]�����l���v1��:%��<L�1�k���j���{�@��s����K�%�sOp?�p�vt�]�	��a�����JF��cGt�D��k�D'�X����xS������p�Hc�
����_��,����~����'�.�<�.>#S���3�n���h(e�]�e�8��P�

�r�7�������9��h��AG�4B ��6��npal��V��@C�JuX�D��e9j��	�{3Q*�81�c�@��\�T�=���=L%�xX�)�������;�x�v��+�P}���d`c\����-K��t�������A6�6�we[��L}Qt�}�I��ad��:=v`k��P�|y���t�}W������-]����	����%U]|Y	���K�<����\��b�Dg���g~<hH'!;$C����i~����s��!"T�
�����s+�wu]�����7E�I{'��"D����`%O��*����Iu�&����Q����>�wH�CA�>k.���_!q�B�-#"C{���d \xsy7�$���`�y\
i�a���!�m��?l�on����FS�5�CJ�RP�
{$���j�Mx��uv*���	�

?R�A�V���+4S��k����4N�����e�/��$�
8+Hj�~�?�CKm�����������B*d�V��k��
�E>���`����I5t.Y�&Z�h��8���QU,g�n"�}!��01Vi��=R9���H��x	�F�����}�/Um}#u%�%�]g���Et�����Vn��D�����1����Z2c\����Jx���D!����R7�X����[����4
ta��/"O,�m��u'U�s+EbJ�YX��H����]Zk�D=C8���q���_Y�`g49�4���A��a��kc�Q���F;K�P��-�d�Y+2�H��=.�	(����8�<Xf�c�������~M���Q��"������G���x(�.t��I{�����15�}�{0�&�a��n�L�(��PEH�����.EQAA|v�a2�
�BQ���P"V"~��x�����k�����V�]�����(�
H�"!B���\F���h��W�Re�9��e����(�%�<�#C)���9D�\jQY�k ���-�;�PHD:1{�S��������V7j���nG`e#y#=F@�=���P
����f�������Z3�:�$bY������^�
S�}(�~��P�����>�2�8@x��?���ld`Mh��c
��j����Z�������.����N�O�@�O|�@O���?�1a��G^8f�}�'�l��q-�`�D?�&�u^-�@q_��^��eY��g������`�o�����t����Z�kC��}�fMI�(�/A������	.=��.������S2�?����[��"�����	�U��=���1�fW��Q�5����/�d���a�%vd��%���	��]����j����n'�i�1�k}k�1������������K#�Ap�qa���0m-�_`\85_j%�~���R�F�P�t�	d��/��~V}'���~"�o�K�
���nr�RR�j����������T�:��A'���"r�Sj�y���s��D�rb���C3�C2x������xe:
tW�-?��yU�V��������#��x���@6�-S3{�������^���s�-���cdz��<�n6�%��u�����>@+7�=���n��./3(�O��2C������Q8��"����8�"���G�e�����F~��!l�b:Oq�d������q�*�T/G0��YC��!�"$���z�b���F��Pg�D8����L���8��	�4@c��iQ�^����k����/������6�_c*G��\���/F�9$:i����w����1�K������T�D����o�j�]�A��#����mo�7?��5j����m�f=";*���|B����~����k������z �H���#�"�0
���j0c������Lc]�m���X$��6�?������$�'�Q��Y*���~1]�
�F�������R�a��_���>����^�u��Lw��R�g(d?H�������c�}(�y�}��^�$�^t�	^��L=pm��>��J�Z�t2X??	��&�S������	��_)-��\s��"��qd����k��'(l��!��C���(U9cugS��B&o���������U1�^���=+��������zf6[�QM�UV[����|3�)|�(�<+�1��dv�Cd�-����[	g,���
@r5$��ge�o����i%��,[�?����/��ICx"t���'�%�(�������_��$#�l_;�����A�B#h�qr���L3�kl@�,�:�������wp}�/P��".
&R.
&�f\�;i3.�)5��=o��6�Bc�C�s<_Q>��[(�(�RDL_��K���W
:��W~�[����M~��(�IF/:2!PL�SC=�@��x6 l������<�6�s�I�Y`��jZ,��(������*(�h�P������d�4;%�.�/+��&9p�W��8�|�q�oM�rkl��F�w����S���l�O����,`"���-kq��sv�n���2��I\�n�2-�fC��<n�H�Z��BRn�_R-��S�bh9����kZ��A��e����rq��r���
���)"!+S]-mS���(��� 6��>����n-�nBd�#B��<������^��L�K��Y�2�`n�,�!~�r���c���LPdC�4b���.k��2���OW�������-i�������<8��3_�+K�S��>p�������a	yYA+�J�R�\�dF{��8B��]�N��P��L��*
��"��S[�L1��pn`8������5�(�d\]}n�{ )V87� @NWj�6�3�����R�f�d��4Ur
\2t���d��x��'��bs�&�;2r|��N��{y�����c}�0����Zk����/�/d��u�
%���r��s`�����������A�8�!4m-�B�Y����Lz�n�.�[��$�d��Ja������zp��/�����/������~�aD~��Eyy��"�6<���t�~=4�����B�U;��P/�B�.5-5��&���wL�x]�OrZ������zF�������z����|(������B8X\VT2n�(����w��o[A��h�wc�Y�
�p��v��G(�|C_7R�FDM�mI
��S��z�i���e8@	�O����CO��[���P�hr����x�fc<�S9mh���WO#����	�^U��e��=~�4���������xB��d����|~�������}��&����:�j�}u>}��X:|�u_6�t�zr�A�^������gM��P���V����`��� q/�'S
����-�/{R�����S��
Vs	Aq"�v�E#�]2F;�s�)��&�*�8���:�!U��$��Ev�OBdu]���f�O6���MF5�M/��;������d�I�;���T�>�bC�����-���c<���{0��O�L�AcO���Z8A>�u�%W�}��������9y�z�����������?���g ��)�B�����!�8��\��A�@p8�e�
�;����C1v�f~w�
I�6�C�o?����������������X�%b5���-�!`p��M�p:��olzv����9�T8t1���3��2L����z8��%���G&�z�8�0����E�&0��g-�
tP!/�J��!��Uw [���p���VJ�����#z���,I7$�6��NS��uT��Q��rp���
�,���j���."��N�4"1t5�f�����>�����7�g��_����(��\*����A�L�����0������U������'
�v���d����t�y�����(Mc��O�&Q�C�(���9m���Wo;?��p��7x�~{�����y(���s�Cg�u/dC���Q���fXS���r����e�/%T:����;Fh���od�%�Y��N ���&�y�SY��J�:_�@*�e����6�
}������C������������������p��������Bw�/M�}L��=�����JL6}�pR?6g���&P9
}l�Da������_,!������O����W0�.PPlM��2��
��:P����� ���b^F���Z��K�����{��q�>���[���x������|U%��X�D������a�U�)@
�7�\�t��j�:r��+@���+e��n�%"G<��'�4�� P"B���e��e ��t0-2���e��e,�X��4kH�\�1����K�j����S�GU�W��rP�r�>!���KB��H;�����J�17@�:�+��������!V������d_bS8���L�\n�MFM����J���n�17���<,�l��:1�� �4_!NZsR�1
b�W`�
O;�D���Wk�If��E�g�z���f�1������h���.��\��gY�B$R�R�:�~���=*'�������n����y��i��b��uw��G�-z����p�Y�����k��F�~�)��(j��_)��v
�0����&���f3���Rq�oT�eN
1��v��df-�{a�x��b��06�u���r��o���S<U��������
%�Q��R3�K����L�Y�������4�?����S�FF��v;j�u���n/�*!u��v���S_\m�H}�����hI����EA#G�>1������@7��P�^<�����;����� �@��Pn(�10�����(!����.�������������A�#��A�R��/��U��{��/��0��_�;��4
��,�T������Yr ���_��F�����'��<p�T����D���a�����>&���s���$]<��,!p��3�/��R� ���S�HvT����K���hlS��1H��J���B��L3�iG�.]�Gp~��$���V[�)���;�CZ^�(!,|��F:�
R1���DN���zR8����o-�/P:�,F�������>����\�m�)�!���Y
0q��cji$�:��cjQ��{��rO�����<&"��.Y�	��G�#�t+��|���B����B����_��}�H� B%�h,�.�\E��9����h�gJ�H�\F����o�8q�p5�/����I�/Y�cKt�A��\l��-��W��R0�+���s���\��ky���;R���y`Y��[��ke8U��.������^��NG�lU��l���w�������+v���r
�K�v�=�����9g��qR��S�?>������t�7�y����������xX���������h���{R�}�Z�K�?
[4Z-��rl��?����>�^��0B����������`�m�#�]c�G�,�yr��Hw���k��R���<FkL��S��0��j$j~�&w�sE�c�mG(6rO�*����l��S��&(��E������R/r\�P�i��D�(����"A���@%�97�v]T�x��P�F��������?�E�d�PVq���T���?)���P��tA2#�t�o��A�D�(���������8�������p��������d�k���K��7��<Q`L}���!GS}��#��g�����PI�� �j2��3Q�K�tq�z�����q|8��+0r(w
F��:��D��3��8�rB�����B�[0
��P���F��o=23=
����'�7�F�b
���[I�1��w�=fN!ATf�B|v����c"�|����E�(n�
#n�b�`4����&���G��D���n��l
+]C�*u&�>S��M���Bt��PC��lLU>��Y�a��FK�b	i
P��v�����RF�>���}u;���q��VA}����+|lYF)f�b{���9XW��V���X�*B��;�
��c�/�`��sC6�L��6�o�NYAZ�'�,�\M�6N�?�h�d���]��s���cc�?��G��/0�cIj3����~������d��)�6��}��f7�BO81�/�q��C�������L�;������t=�B_D8����r�=Y�G�=����.H"�&���il�/0�0d$B�����]�^#{��?M�l;UiX���T��P[���q��d�|�*r�4���
D��:�Y*V�q�:�H����'�v=���r��1p#;�����-�G���D&�'����D���'M����/������v���\e.g�Q�z�)K�r�~�f�����t��v������m���g�$U'��C�e�]��I\ob{����gj�
"!	c���eM���^�n\H$eO�=cI���^������Q2CS������xw6���OfX8a��Y�����B;�Y�\
#������PW�����}0;�9��%�Yg���M�4*�`����j��W(�t�����A�vq��R�u�!a��t)���������g���� �S�rG�s$����\�
E����Q�GSW���f���E���7�R5v���
��yA�D�~��Yy(*�!���o�A��fG>����)�����SZ##�U�`��+H��fQ��on��nuL���x�.BQ��Fo���'~��g0�t���T�cYUX��XX�f���b����-�4���	y~(�xa�`�lE���p�o�[��!��{7�5xj'k�.�r���~QU�Xq��V�)M�����c�h��6���	%�e�It�fl���x�:�x(��6LZ���E0�������\C��~���#�B�
�^����� �4�����h�QsI;���:�$z�0�n#+	������g{�|��G�����V �(���@����������3c1���{!3�La<�d��OH��s�
��5pl��;�����D���@y��ir2*v����MZ	��M��{];��2=nd[��B(=���Z��[)s���:�iG:(��y��}�������c�v�q�L��p`��L�q2�1�,;~����v-��;s5��t
�>����i5�l���%X��d7��Q�.�$?�P�Ow�zdD9��<�����+,�3IHyx���� @L��j"?�}���o��vrl�7�b�Z-H%Bl���;+X����l�AM�G>�H�J�-S��N��7�s��q�-����?���w������eu�T�aO�n�[���<�#��ASV5���JR�M����E����
��F�	��I����4�-�8I��9&��tmu�����M �t�=R!�����C�y9���[�B���[R5�T�z����J�����E�����cC����TN��R5��q�d\��	�jU�;���V���j���{'����m����hOc�*�4B�:ek�K��a�k��V��-���1����NI�J{����Jv&i"Z���$H��Pd���u������(;I�"���7�uHV����z}���o��sZ�������dGU=vhRLTd�Y��PI6�u�l���)���r�q�sI��s3"u=!TSb��]F���*�3��n�nC���$��;��O�Ve�e�?�l���S��WjH&��j]�H��8�
[#���~l���]���S�=�t�e��NUek��������
c�]���Q	*�b�TJ�+�5?�<x^.��J_�Q�)�pix�+^G�����8�+%X+a��)�>X�&	��x�f`��)��0�2%�IEdp?(�����'�%�������Srz��J�������T�����!�����l<�Xp�.��@x$�����.
�i�"*ge���tR��)�����:��@��o]PE����f���i���S��9�����m_ul��a<�LB��;#����=�/���zM��� ���l ��_1�	Jj��|��b?5������P=�`d"1w���ym����t�����(e
�����k7��R_���i�tqYh�pX��E��F
Y��D�>am5��j����d�&��f��\�K���\���S[�I���dfQ"���.��~�
��W���0���F<�$��(����k
s����e���}#��T~�
��0�������tq��Hj����������K��y��|v���/!���bG!*|���FU_H�z�
����PA)�d����/�`��,4<QI�p��U5�!���^��_�y�����nB�;"������S$���-�q����d���(����+��<����R]����?���v�)�v���Ib�4@M��`�&m��Mi8�u�sjNWE���b�I��v�;��)v(jZ�P0�fm��b~��i$��s9���q;k^�� 2:V\/
���	�bMg(�f��m�<���9����It��R��V��m������E�#\�#w`k}����6E�hG;�o6���2���� ���AH�5���}N�L�Kx@vQ3(%�����EN]3���r�X����?������>��u��F
F�A]>�����-���0T����a���U��U�%a�5U�Ev���k�V�lX�=�i��t�����C�v"{�*����D�%?�fs�p�[nK��vl���l����tz���s�:�t����,�q}Y�'�����������������r3�����q�N�^��i��H������sG�]����k1���s�D�X�ZoR>-�r��9w6����w{94{m��s�B��g��������O��r"Z����������d���O3$P$�S��T0���	��L9�$���RAWZ����T/����j,jn_K�6�:UT`��,eg�U%1RI�|~���u�ei���r�y��8�3��~����g�[�&����3���1�ie�+%J�T���?������v�|�hT!�T����h�.�{n$�:9l[.�s6f:pS��8
9>�z�P�������i*�]Q���:J�����kE��Z�9/L
�(�����|���d{�0���"E�:J�����jdf�]�B3P�d2�������sWx�g��������%M�����b�*�wZ����6�g4��4��qO�vgy����f���%��O=������2,,R0xI&I*s��B!��`�+��d�����8:����[���_^�g�J�()/���M��,��C����bm�9?�2�\���E�g��Kg�	�3I��Y3X@D��(����]\[�8)�/��5�J���a�W}'J��lC�jF����9n;��?����If��8R�����|\��f*x����gS"���|�[4��#�\|����������2q���0�����dR����u�"�����b�=d��B�^�x{�����2-���U[\h���������Q���R���y�n`�d�Cf����!�����T��sg��r�GX�m�cx�'`�l&�d���r����]��Yq����c#�	h�M@+_���
V��1��Y���p�,���1�$�l"%y�B[n�by����U�����./V��������\������/��A!�0��3����XZZ@N5o.��z�����'Ts	��a 1��h��s�v�z��H]��V��wf�m�
�
W7�l0��X&%��a���d���R
�iPo+�N���������{4b�1��W���d>�����H��Y��������P�

=��&L	<'�K��>B���r���Ft��#
�&T^�����s���&����:���R�K�t����<�����,�u�m��2]�A������8ds��������eXX �|+]��.aK���Ks���:w�0.�j��vTv���]l��9OlP���l��M��d4��P�]�	�}(�k�x��X/6$�cy�]<{g��Q��kh��y%��i�/{�J�NXe�p�/x�	m����6&!}�b�>�N���JqI�|���Mk��4�[x��c��r@G�[���{Hi�,cm�|y�t~v���%�]�������\��������0m�e�v��}*��1�1������8�*����1
8I���)��/��#'�y�y����;$��n����|�W���z��?�0���vf�H���=
g�������`��,�-)�uq��nob���
�Y�Y�3r�����FFpb5�W�2k��f�`�?�sI�N:����m
9�}6�1��n�>)a{�Q�m�����r��t��*�9��n.^k�2��m�16Z"�N$H���}��;�7��$�4W��7��e,?a�r��d��������
�T7����� Yv�/vF�0<*K���v���"/_�����^�7'>c����
����a�k�2D�3�����K�X����03�EL����0l���W;xI3�$n��|9!o=���B�x��e6��Lj�����Y�c�e�WKi:�8K�=<c�F0e��Ru��l�m��H	_!2��x�>x���v��d����
Q��*p���>��2��"b�kx�T{-���#l���^:wr���O`���w\W�q�V���U�!wN
����z,�I������M�������
�2�b�=7M��}�'������~~��+����z�����c��6��k}��D�,�J��k�<�����G3wl�������h���DR���R�<�2|��Yb�H���5�SM�#6����m�4S$��%��se+��c���\b.����[cJ�{'����^���Pl[������a�I��e�.F���wH��e�����<���1E����!����t3���d'b��8�G�m�����;���f7�W^��%��,b�<���`����f����og-�������H����4�8(��eA��x].^��CJ#*�|\�o�E�b5�#f���8��LT$�@�L���J[B�>�cZB�(�NK���SB|���6��q�E��YL�M�����g�l��&
������=,�4�B�U�ps����x�iK��u�"^w�f������n����������x{C������I����{��Y������#��bZ*�z�� �5dG�TF#����<M�eJ@����7k�!bL���!��@%��AQ~o��7{�:�
�jw�������P~k0G�Yt"Zt��,�����������`��E�1;r�2	�����)
a�����������/=~����B�>I��c��A3t����2��~�Y_{�tw��~D� �,,aw�Z�rd���2lT��g7:��rGy������G`aEr���m�?�yc��5���u�r���1^�Y)�K�|}i]��P-%�r������*�4F3({����$Q��r�5-����� ��d������x�����P�E33� �T����{��i�Ce�Y���;/��87c���5|K�4e)�am����O���jAY��������(������3�@���>Bv�_��sN5�_�!^��2�+1���D<�3���@HW���/.e�a�2�VP\j0�P�%�t6��e�lG�I!���<N�t'��\���R3nh��w�1O7���K:�������oq��������?)W��?`0;�����0��I[n]��ao��<s�?���,�`JnG��y�.x+�og=���hfy�����5������4O*O:_�+���q�|�5������KY���"��r�w���0;�+�u���|���ro02��>���^SB�wIe"��.e�;P
H|���D�9.�-����kK(�A���<6���*jI��=A����������=������e�lz���c����$)���V��"�0z�lEz���|�a��Qb�f��Lm
���C����R�������p&E"c�e��o��0Dq�'*���C�3�����v�B�.�]�����\2u��kE�����&�]���C���E��l���Mgi���� �(fwa)hNI��r���}$Vu���`no]��qD���2�tg���c�=�������j!WyN��z�,9�q#�I�����+���e��H!�������,��S�K�	j�u������|CF��M�������=;�|0��4N^l� c~���${�\
��<�4���,��"��	M�(�������?���+���%���S�q�6��
�vz0������#=K��?VG������{����q�O���R��������reD�	qd�Y�vE1c�C�D�Xy��[�L���3�6>�y��B��f���SW�I���4j$�Vc�����M���RXx����G��w�����N�����>g���$�k��\w�b��}O:*�geK2�(��ZF T#{%�����0��44�
����!���x�a�G�����0b2r����(��������<�|��|6����i'��SzN�F��4VD��a�}�� dZ�����y��E�(R���v�T�Lz�O��S�@��]�:R�]�`��$�uD���V��������!EL�v�N��yY�B�S�������);"���
�����lf��-�0M����/�D6r��:�(����q_o6$�"�l���!}S|,����(���W�k�k	<�Di1M%4���kf���R&6A��gs�������>x��o�o����#���?�D6mZ��H�a#�}�g������2yh3g2`i�����w
��O$c�� E#���D���	���y-����(72'��L��<<�l�v��CO(f���8�	�8���G�����������k���P�lEl����B`�,�������� C����������5T[�8cdo�I�|�2�$��hCTb���'�T\������1�i��#���v�bY�@#��*�%j���b;�����c j�1�����Ly�F26O����z���SCO�)������pm�z�2�"�"�w��|�
��?�I��/F<d2TZ��q����p1Ts=�A����)���A�N�i��N%���I��%K�������=������w��d��;I���)���<<�M����L�!}�
�8'h�T��7-A���-v��!�l�v��R��>_������\�L�Q��AjBI��
+a�"�c��t����r�5�r$9����(~[0S_���Z�	x��^n��*���b��)��j�4�5�:���f�����J>���>>6�����srK��iJ���������UP�L�����?>��;� ����nv�x!=�k����J|]�/�����<Jb��K���J:Bj��p�{��+5��H�A�����L��x����g��I�-�5��pL���vf���������K�z�����`/v�C|RF��6(� __}e�����[cc���6�������V����\O�i��>N��7�����Gl�|��B�Z��w��0�3����
��v�8��R]C�V\��l#G�s|T������B�7R������\Y$&2�\�yC;�%�g�Y�:J����m�L���l�����SE��j�S�����S�*��3�DK��Rq��/9����.r�������3�]]�j\���1���|�}���3�H
���z������9�T7���!'7�n/�h>}�s�2%��(w�c���-��b��������G���"��H�`�2�X6���n?1i��
���������j,�j(��$�`I8��+W���7��������T��n��e����j�`�����`q�~��P�@���B@{<a�s��du_��Q)�.��	��
�� ���e����l�����6z�{�kl�d��}�~9���������=�����!�����4�6{�Kp��>�R8������r�O�t>?�h
����;������s�'qC�Hs#����l�9��G���R��+TtY���I����i��4�S�C��������������:��NF����\����Tm�
*x��5?�dA�l�t��27�"�;���i�a$�Q����rx3����&f��b�6*J�)�o�`NH�	C�}�+zi�da����S�l���C$\����36 ����!k��x���������ZM(4s�Z��H�i$-0����������I�^��w���_���{���;o2�������&���[pX��96,_�"�v6n��
��&U�q0&���k����P9��PZ���w
e�K���a|�\?��"?Bd�N[r�(H��� �O��?j���Of����S�� V�}N�����P�b��^ez�j��6Ub8��
y�b��qH��]O.h R�^�G���4�I#v60��X�~���I
8p��l����uO������dRR���8�>"GsXu�v���
V���eL,CZ����.7������f3t��6���;i��������)F�m5�����h��2����K-"�/o:�0pCS<���;84��;,8�W�;I��Xl���-��"�X�2m������B�=j:���:��g��������S8�g�=(�����a#!1
d���B�$�h��=+��S�N�i:�('/F������;��U���f2������zQ[k �	�
O�u��L��D=&���D���<&�|�-����d�!oBf3������`�&�`�J�����@`�M����|{����9�MP������wQ��06T���,��A������)��'�E�Y��{}:W0�C��K��  �z+i&����uC���bx���/B����c��1k�<�l_$�PL �S�?�o���{�ZT�u���_�kzS��`M%S��|9�-�J	�Uw��P�dP�z�_(h�d���c#j
c�Lyq�����b8��rAN��FW������|��{-C� 3#Jj����\�j2�s]b�$��{]�.6{� CH���^?�)ud�Q�B���W_
�%�	�p9�Em*j��n��e�+=�r7����=��ga��A��)�a5�[=a�mj�.��y�vz���s�����0�����?�n��0$����U{i��u�6��mu���-���j�{Y]��d�d-C�d����! 7�P4A���D�2�Uo�E�]�������G^g^���C���O��T��}���.mFPy��M���9�e��ra��<I^��+�,�qZ�����!C��PrF�pD�2�/�����x��n-d�7��n#��0;�]�N�.<�6������rF�i)	Yc����n!NM	!��QB�G;������<�a�����j$&������rf������?��"����B���3�0��i�I�/QY0a,���� NZ����+���[c*C7���K�46��q�7T?r)14�����>?Rx��>��;C��$���X�����r��k���o�!��?Tdeo�r���I���N\W������l��jC'��8P��D"���Y(WCa��
x�;?/���S$W����6��'�U�,o�/���n?6��L����2$0fM��`R�E�f�'5�����S�y�3�]�E��[]��9l���I�
�\���F[��%�l�)b&����R�-�oGE����W|Z/�r�H��������T������K�}`D7�)��U<��eU�%F�]��fwu{{�����W�uYll�s�+�_hNq�/��%z�S��z���}��a������	;�a
e��a�g��6�n���h&��'M�A�w("l��� ��6�^��N/LS��i��^�!���������X����~��iP\�l�����s�p�\��l�p���uz~��(��[������s6(�!�����$aw��4�&�vjV�8B��9C�rn�������^kw�y*8{�������dRb�������OZTI�~-2)������r����94�cJN��v�6��Q��jW����� Ed��NZo8�ip���XX�)nQc%n�g��m�
=Opz'yn���� �B��X.v�R�#(D�"�,�k�A�H=l0��_���h�
���WWg��Bx8����R�[k>C/YH��3'�F����vg�#�3��G�	UO�<��%��>������	:�����S���v����t=�i6�f�N^��H��(�J��������cX0f9�T��Ijl���������3����z�i�A�3�0�J8�QW����g0T8��T]�Q[,�a
�9��t��P�.�����z�9�>|2��3�GN}(���=%�P���|����;����l�k.;�������y�������#-��
������)�{�-K�4 `@�.u*,]�����:��f��]�_&ZFh�C,��������R����P����*1�	����$�`8�*�v���#�S��(o8���;~�0b����X�"��b��t�p_�0
�L$n��{-x�� j}�oe��,;&A�� ���]��P:�E���H�e�����W0"X8�����������(x�nTP��;��}����
���nGB��7��g��l�	�/�_>���FU���C������v��s�I0���
�D�N�C{��Y����H��2�1�x��O�-�A���p�n����G�F��w+��8�SBB�U��e�v�G�v����$���8��#��}$�����`�������������z�)�%z
�����Pfx%���0��	�-�� 4K"�� �U�$���YP\M�g�x(y?*N������`��]R��w~�ix��?�N�v������$���<!���	{�Z�-o��-*�������^0�L7��
y��O�6d`-,��aKC�Cqv�sE�GU�	��1����(�����5~a��?�P3�����G3����Kl�3��y@�z���e�:������%��M���l�]�t���4F).�*��j9kVm����)a������Z�'5�F���y+#-��4��l�C���	�d����r��DZC��,�j�	���j�a+l�;�![��cGI��5�(�?��F29�f��������W4�yUo���^�
Z-Z�yg"B;��p�N�����RS�*�j�|�;b3������m���b]��=U��P,��-��K�t��h"����O���
��|w������YuDr=j%!����n�@N���N\W�W���QD��ILq��{Y�V��2�a����-�T(r���!�]�Yz��{4yC���h������ymD�����{��.6C�n�_�G�!q����)�C����#>��vs���������"����6���;���z��*$�5�k!����������[����?���O�&t,��^��Jq3����7���1w��c��_�d�{���E�(6�
zIl�D��T����n9a���S��v��j)�N!�E=f���v��Q��]��hlpiY3e�&��0qL*���|+{sd�M���5&�y�W��<�,]0��j	�\��0�EY���Y�����|+������|��z��3��0�u�,�7������\�X����J,������#N�7Lq���I_����l��f�E�V3�eyqi���k
��0��U�S����
������(�x�/��+.��eg��B�pb�~���S'�u	��h0�@0�I�*�,��7�\.:��!��(�X��7F��	����
0Bo"�s�pAOe�	u'����FO��thpjm�0`�f>'vB��R�6����U��C�d�<)'?k�M���,Z���������M��
�{~F������D5�&�_�����_^�?}���7�� p3,�$
96M6��|r'O��^�7J-����s��;0����.�2 �y�y�(�1D�F���	9D��H���Ok�0�r��x�UHJIJ-����FR%�Y����L�~/Zr4	U+�yS%�u��z�UlJx�|�l5��f^x���N�=C��G���9~M)7���P�������],���VE)	KF%����?L��\E��g@�3�_���� ����m�	��H���@j�u���3�:�S�8���;��a�T<����b�z�����
��G�r�U�cc�>6�1���n�Ih;�;�^n��3`���AA�V��Y��`d�Jz��J���h����k�<��R�{$���?m�m���@E�e�c��)�+6Cr����a���.+�Q4("�[����m�|�����out)��U#7�����j�������m0�:���h|$�;�������1����j�u]�D~���#�	~�����}!�~�z��~RP�Ogw��R�@����G#�����bQ�6�JO�QyjD�#��5��C���|��jN����/O���z�~{�����/~9���+��OT�R�����{e��	`X���������DAC�_3Hs�s�=z9�����i*6�$
*�c�������A�R�������@!�n'�����L����
@��
A9	���8I����ISV�D)E�X� ;�u=K�)w];��s4��\��$��CA!���<�X�}������!;|�K�9��^y��~g��EU���0���1����]]l(d~�=�o�G*&7f�S��u)�;�����;�PW�C=�?�C��!���G�����C�9M];"�*�r#�~Q(x^���H%�������&�k&b(T�P�� ��������E/�������*��"'�S�������%O�(�
{�b��>�3���j�L;l4_F3_�������{i���U�zH*a,��Ap=+��S��#�� n�62D����H3���.U-6���������h�)6��86�v��-��~�V.wQ�	-����|N�Fv�q��^��H�*���I����oJ�h�cV��XJEj��V�8k.J�������b�MV�G���o_�������x9[�x��Td05�������|u\W����\��B1;f��v)j�1,�t�Ev�]Mh^m� B@L�$�A�q�_2i��b��w�����E(T���k.���)�6������X�s+��������cR`����Hn�T�[m*�A��br7�	����2�L<�I��w�c���qv�Y�$�N���8��(�����zSn%�cQX�-�����8��!sJ�JA�K(M�=��!����7%9�i�z/%�l������p���8����Z�H����+^��e�,%���/�DW�����T;�!E���^F���^���k!�wf�������K.���b-�����I,��`�],yxMY��8�Z<�����!�Z7��X��3�BC'|6C)���52�&��YZ|wl�l�4
Yo�����vd������$����U�_������1N2PI�������u�����b��Q�?�&���
?L���	�.k7�>����^���K�\?�����L��P�9}
�9W{�$U�C�����F��#	�
�EE�(�+1j������CY�{I����r7���~8�Nf��t���m�e@���vB4{�g�`�?_n��������"H��h�T���������\1��a_�-G�X��8���E\�,�&	����o�Vr�u������8	�/m�[I�`jQ�D�n�����?q��XG����@c���W�+��P{WBO�
�3���/�4Q������g��
*x��1��%�	�|�+$C!c;�^�
+���a[�������\����L�z�tL�W,V���w���G�`+:���E����m���%J
�~�<������'!g�96b������b3�����2��W��=��y9��SP W����h*���zt�������8bD}J������U<#H���m X�0���1������~A
�0�a���r����i��IpP�A��RRt8v��tgB�?���q�Q>�&]���
�;p����	�<�`
{�� �a�5���B���7������ag�#�.�C+6�p���:c�J������4�P^!�6!���G(P
��
���#k���e�I'SyK^|*�;�C_X�Y�7�C�<�k�������4Nz��S���V����(��4{���TU1�L}�t%o+�s�����8��r�Z���K��;�>l�@I��.��<��=���b���G~���*���m��-���� '�������s�?������\������D��F�|-�e_��A�La�d��P����e"�0�3GS���@�v��S��
(�T���\M������Qc��������g�oi4���	9,{zv2Q�!��1�O2�[7A�}���y�A.����G/��o��;�3gu��k%&3LNf��=�EV�-a�zgC��dh��[;?6��s�
azu2��r��7���������^cl<�=������L��j����q��-#���IY��p�����S��B+���=7�_�������2�
'������������.aO�&�s�o7&l��B�d�
�?=I���cc!)	y�n7���$��Q�c�t�����������m��V ��k}�%�����q�;:K}����v46*�L��t;��/I�e��/I���G��eu������
���E];��H
,aY�%��������� ��['A����.QD��!�m�
xJB����vC�Z����r��������(�m�p6-Y���W�J_1�V����bS��6I���$����\5�.N�o��+��A���(4D1o�p�M��Y�%��	�_V�#�T�p?��G���&
��D���Bg$B��D����Br7����mdQ�e�Uk������+t�}�l~s$E$��$C��c�Bg��Gv��E�$��Z7o%�F)�x��m�
Pt'�
��2Wn��|#��ue�$��}l���bmX��*�$����YI��7F����)�m>@:({I����I����5W%}h�x������C�T�@#��x�����c-(��K�qR�C�r��N(Z���H��J��	��Ncy	'��;�PO��og�e����U�?i�)6�Fzr<����a)��$��n�o���la��	Q�=�a�G��a�o;�����PM�0%��H��G��N
��h�|c�,B@�fm�1����s;cS0j'L4���Vc_bQ�B����J-�y��Z!?��
��J;a8�����^���g/��7�$]�[� ���N�..��N,t I���$U*��2�vN(`���3������8(�\cAt %�&����.�S���w��M-���3�j/���1 @u>�f�
����+#�z���]�{
S8�9I5���.���c�{����5V�x��R�qM���P�
�s8+/_�|���/��^�5�v���z��^�K�M8c�vX��1��	����e;�����Wf3x1j;��v�n�0��B������SKj��=��]�S8�/���1�H��r	���|�H��R/��\)�}]T P4K#�HR��4f�]�S�F����|�(�DwM����Rv��l���'�J��Jr��+k��l��j�vt:�9$T��de��=��l
d+��P9F���e�t�m*60����1��N�P+�xLY[�DB#}���a���&�s��z7�=�v��-���YOD��Xt.�
��L�H�����t�j�3���_�v�i�qd���K|��dOc�4D��Vd��>��1����R�&���O��]�Y\�H�~��!�����$��}���;9�q�����)x%��V~���8����������7�����2B�H�v�K"T���`voZ�e�F�B������C��N�r��_$���{��������/�_�g�G��Q
�`���X�#
��s��
~�/�s@?{�����$`HLj���M����i���k�0�m�r����j{��<o�e�H���u�����3��{�����w��t&I���`Ul���<M��gU1
D��$<Q/u�z����US��/�����U4�t��L}�;�6��-�
��v*��_��������v4�Ki��Xc�O��K��7�bG��N��X|C��y��>a�+������
��
������]P���Uq����������{�Gzu����~�F�1R�/�� �������w6���9��Kz���q���{|��0���,S�����/~������H��{+�����F��q��tW��t�������G:����I��vi��u���86q�c�
)���[hS4�5M���������rT�e�f6�r�hy���#L�����tn��H���1�J���pNX�G���)��s'+����1��?�H�u8�H��fj��&�;�&��]�l*�!�d��Z%u[9E�@3g��m�~����?�i2�����"B6����~�M{���������0Kf�A�F���D^U�Fj�S6�5�V�U|�r���������w�����3H�Y��U t9,v<F��w]�f]QQ����du�#�E:������]��U�
;e�
9
�7��L�S�`b�1�Y7��g�������35��F�k��:�O�_�Px^f��q�-��A��zz�B0o�>���6��\
u�(���S�����]������@7��a��aM���}�{E��>��9�K��*� 5a��xw���uG�n�&�7�GgH�	2n�Mm�6h8�?Y/�������1��v�!#A���1c=0fbi^����L*H1&��>�7��]��>����#��7���u<.��Y����Pk�q�7�X���d�������e��;\hu�@��L�)h�T�����&��kYp�GmqB�f��*a�.Ao�M+�"�>�7p�C���q��2v��M���,�6�H6HEhr�����0�d:��w���h?Nm=laa$��ZT ��wD��*��_s[1Fe�YFma{Z�e����)�,co@�12�-���s���x�L�2�������I	M��#!�����5������!X���a�}W�>�������8���+����5�%���_���t	V9_�fk��\���_����x���Vba:R$=�I�7��5?�����;�%$�[��[5&\�/_��=�EF�u� ��20����a��g+��q`�,��r��V<d��<y�:g�f�w�������~3��C�����Ya�v���qj�]��4j�t�:����������N��Q��w��@�^�/�vG|`SD����O�G����\o[��
�����L���^tF�Fh4`�>�{��)���;i�Y�tG0�t�Bo���"�K+�[�?��`���	����"Ma����dz��-�.�� �8Q��=!=T������Vs�����}��������l6iX�M�%�Xsc�E��j���[�	��������������cMk���*��l�P��Ui[�	/��Y�Da�D$/3���Lk��v�)<(@��nV�"�J��ppo
��A�*�i�(���zwv��g������?V^}H{�2��i�:�5�9��K�+�n�w�N��� ��;�����
6��������m��cl/���g�����17�O=YG�XV���>Z�aeF��ue���$~f]��6���1�+_������e��iNppvG���8�������Re���x�N��B�0}7���4U�$��wCS�(l��s���^m6�|��������l�������n��,e�UY�s~}��QE���gBg��,d�)wde!�X�����g��7��M;�����V-����7�Q3�u6J����Z<�������+���$p�����4��)t��{��{�Mg=k������#�������H���.d`]`w�}a��%�K���m�	����r_b�z�(���G� ��VF�?�)����{SLg��l�U�����F����TL/���J�-M�U��C�$�+�O*��i���#�;Wb���)�_�a�q����F�8��t_:��~��i,��L�u����Z��9[����l��'�/�W�p�\���O��kz�_�6c�J��ks�P�\���`�^��T�*e�~b����$�,4��3�$��T'�Hp�mX��;�q\�/��t���4MG��4��4a6���z�m=,{���~����H��~���
O1����k�Y���M��$�Ql�p��~��K*i�T�5�����'.og�Q�G�%�����(yq)����la�k���V�Rs��UG��kviM������l�-^b�{Y��V���\d�A�j��������z�1�cH�B�=s��R���'k[�O��AP"j�WL��/L�@�R���2S��+8�D�gC��\����@';SG���N�������d����Gi�u�g��N����k�Im��l�Nz_�
�7]��F���H:��{���Y�9;�I��:����L�S�0c��W6�_kK�C����[P5�����X��_����n,��xT��tx��u1#��4�UXR�a�x�	iU�`�?XH��	ie3���_	i�i
FU�����'�&����f�����;g�xu�%��$(uY�W'���l��c���+������=���<8`�A�2������0��<�`��@����
��evo�L{r�����g��x
{�~��w\6�D���f���Il��Y~�Ii���S��#K�W��`����w�:w��p49��1MA���1���'�D�{��rMa�fSm�<����RB��..��[K�1C)������DTWl
�]6y���������eb�Ge����_��������������F����vx=�j�-��/&_Je�U�mv��3|����>������wT�P��En6�fMuInW�6���,[}��Do�~	��
g1T���C������g����+��~g{��T�^���lLl�6�>U�7S�=�����`�K��%���6�L���(��MQ^������M!T�*���h�|qm~4Q��h>����3}�S��U��^^����7���9�%1���/�L���
�����5�J�	J�X����F���R�N�����
0C
R���\Q|.>`B�c��;�(qS�^u;���<�\�=7���w���
5�v��E���h��3�6���k�.����0���!�.�9X"��~/��Y���
����t�M
m��^V�4b:m�F��tU���6Ep����Q��[�F��;�.vH�B3W�s���[Q�h^�b\/���q���y����`\�6����16�<	Tm�O�a�����ph�V�{���oQ.V_C��Y*������"�fU�����������������F�w���m�.��<H�fiX��,����|l�Zp<���k�\#7��F�b����#;����J2(E���O�����		
T��"����N�����|Y#�Hr��q��e����Gt<����n�3������V��J�%�h�8�9�e��n�-��)4�h�myU��m�\���f����t���<��V�\+��@,1���'��B�Ci���H��^����R�X�J�"1��d��p �6MO�����X�l���$;�-p�
��b�ru^yO`���$��-���[�~���o����*�&w��$�D�����m�>��~/%(�����������HH�2��.�:8������������L��m�Az2`����=���,��S�M]���'�5Sp����������&NrVH��9�g�C�4����&��\(���F�7_���r��z^n���U��#�g��cM#yd�HM�����(�h��s�I��4����NeK&�M]%���w=_�Im��"#����G`x�?�J��MqN�_�%bY`cu=������E'�uU,����~g��V�~x������W�v�*JZF�����:4�/���������^�Ya^1.`.�l��6���L� #�>P��}_C�ZU:P���|v��������>�T��8��R;!�������nE���6�%���/�������qzp&I���j��i=c���h���������0db�Y����o����2�-W%���F�^L.����T��[����nK�}a8	��t�����#������K�Lp$IXwd���z7���)8��C�#�T21�T�C$'M����G���4'`
��E������P�OWd:o�����5�5��8��N�7��w++���t���U�i;���{���~8�_�fO���
f����#��>a#{z�^�`���ll>��@�0T���"9�y���4�5��Q��j��"���g+�j�O�hw�� �anca�c2���	��H,A�I���"��.���iGU5����]��������D )���S�H���v����12��;f�M��o������ck�3�}����a>��h�jP���-t����7���D����h	��e��4*W��c���*z`�id�H�+�v����v�YX	�Z��
E�y	������iX$�W�	����
x��!;���Q�Y�)-�tn�F��\�>=��pL�m�r�-�-���xDF������z�����tj����.��r�H��v���j�����a����~M����}��7FZccY��tF���E���J�� >f��T���������xg����`�[�aT4�c�,�H����E��1!>�
4s���F����Xc�
A0DW�-g� �#��-�*��B"��\-�����1:r����Ts���vLh���&�_� ���h�����7��XV���a@����/�\i��������6C&���,
�7�5-�K^����������Z.�_�x������T��%j��5���)��OMf�p�����{-vmG`�s9m����>�������k~=$"����_�#�#��pY���\Y�����3	�/O����
N���o8Y�k�9������ �j�~�����na'�����m��i�<N����hcN2!���+��"1;b�����l���Mo2{<4;H�d���:g�����u�u�#GFq��O�U{�������� �8�vA������z�$ao��?�)Mu��2�F� �����('C-&�����{��q4�G���Q�1�u"7�k=�w���#=	�:�c��w��h`S,�,�L2L�fj�_�X\�������������9-H�e|~#�bC�jV�9)>#CBY��h�:���s��;rQ����o����j��m��L1�������W���E���y,	�%��G�^z�QR����w�7�����E���W��E�#[#ToTH�	|����2���?���J����~�__�t��|�AT��4%�s��Z��
��2k����;�)�p�����a1��i'A����F�����:]��D'�y�Y6Z����d�}�n���/:���}$'�y��t����^�kv���~'��	T���������\���jG���lA73lt.[��
����n&�������7�+��������&T�����>�-D]t�1Ig����#����D�Dq�%bQ;�����Jd�kW}�
���Q�p[�6���6n;�HL�B9;e��i��V�fZ��q��`:���N�(�o�+q1�0d�6�?������N�c�/�p<�&-
yf����pq��E�N���WA��Al������&������3����I��u^���B��%Ih<�����s���v�����LKl4�����9a�zW�
��C|5!�b�	0�1q����������������G%���P����~g�p���h���x/g����R�?FU��9Ia��c�Ld��avx(�~�y)����z�����n�U�����N��
W��=��'pa#H1I!�U7�Xnk�����9<~����W��o|�g\��Bo=���A	�A�Z�%��ukk���tS?��}���[� ��,Z��Q����2�#q���-����J�����4�~8��fg��
8��}k~����uA���,��|�������'��aiNG`a�o��*_�'HU�-��1���zd�%�	�7dxiA����'���	t����+��|���d��:aS���=l��&O9���4��l�4{���5���f��tl,������>�cm�!98W_P�����Y&3�l y�	�jk��l�0#�f�I��t��%X� �l�euM�
G�|��0������./VM�G�<��$6b�������|�}_����D2��y��4�M��BOn(=e6��&�M�[�!��C���u�xa�A�XP�'?�^�p:�C<���"y�������ko���&�Nn�k�PG��K8���S	�����+�W0O9�&_]Z�WS�+�m���E�� �`c��~�o.���N�[UM1)��tQvb��J���>�
f�<�,����F�Y�D����AI����(��������������j���"*l����'1TEY��.��?�����pj��b��n9��KY%��Y����-�"�o6������������M��Z��G[�
��e��X�%�������U"k^xR�V���%�H(����-��u�v��o���>�������U�\R���%A�U���?M��|���IE�ys�DGL$�I���-e�K}]s�3��f[�|l@�V"��R�L�}���j��&��������yHuV��o]��tO��g���t��I`R`�C)g��:�,md�ea�l�GQ�t���.�
�K���s0���mU�1R����[�Mx��vy�)���d~Y~	
���d�����������������oyua���4GL�	����jyW�&g��:gk�jQ��a���c�Yj���V4�0��������r0�
K�	}$��Kdb��Sb��a�J��X������KH���m�1�Y����l����?z4N��~������Z<J�SYJ1�����U���v
*��Vz�.9�X,�Z\JzI�<�g�����5�9���CW	���S�)��(��^�_��D^�~�z��)��2���6�����LA�r��!+��i�p�kZ���zm^
�����jc���o=�#o�����(��bE���Syenz�Hq^m� �.MX�``}J�5<#<�G���
�� .+����X[(��t�I�^o�O���7�Y�
C�aSX�
�a6���%������E��#���E�UZ�����G^y���w��+��E�VN�Q`����U}Z��=1i�����6#����ndk��~��E��:�p�b��U�)rJ4���5��Ie0!��B����*�d�j�A*����p���!#h�=�aS'����G���P��W&������
�@��e.�c?zhl����}��6Q����D%��%���0����W�3FH�������0d[��
�z��]�W_y�R� �P�z��ynG9��d(��yM;���3��9�8)R�!I�)����eG>�3L��5dpf\k�YXT#��	�a�
��8*��g��f����.���P���'��Y��}�\�0���wNu�
�:��w�t��ub��%��.:a�#�\lQ�e��L��r{�e�;���ia����B O��`rb�����/�)�=@�l�M�1M�n�y)����cfuz0����=L\
~�t?6u�������]��j
�
����T\�]��sF�9���I��+��flE3GXc�"�di�PL�k��"��\�s�h���������������o��
��.�l��)4s�;�w���j����R
����j0����|����O0������F~�HG������������`�k�Z���4��v��0�W�����]a��^�u�c'S9>�����,�v��.�(��	��-Fpb�#&���m��1��o�}��>�V�X���u���6X��M]�����mU�m�����������4�SR�QF�&}�D
��X ��j�D�^�9M��b#�4��9�V�w�r�����w=�.s���B6��1�'����w��,�1�t!���-N80L�\5��A~�����l�"��	#��G���0���m���w����4<9�������4�: X�m� ���{I�{�\*������~�o���������{�7��fy����na4���
�{�>4%��BD>�;d
D`�W$�^\
aE1d����������W�O���7�E�$�f���Zl��Z"���_�3�X�2���0Q��*\i�+�R
l���yLeU���y��94^<��H�����(	��p-���.����_�;E�T�i?L A�/���Zw��!B��n���,#�`��K�i���X��w����@�-**-}1FE��;��+�M�4Jbw�ug��M�xp�!N#D�]�2�#M}��Xusgmf���G�_�����H/��q��
�����������FA��KnC^������c�)���"*��!����w����=�����C��������!���*W����=FI`.ZD�h %%����v�(N2q�����tcA^��Ra�Z��X<9gb����G�7��r��D�c�\��Yi�1�D+�o8��6/C}~�E<Z")n:�����F)rG����;�v��bT���<\Ci!3c�T}�X9{�&hh@4���Z��.:i��)����(�J�dn���3�_$�l����t,�;�`�qX�V-�Zu4�f�����_5l�:k���Zn)���+-o���T��ow0B����~��+���i���H�L�M+���#Jyq����u���A���S\P��Z��3�$V�5f���.�Q�5f���n?�Da���a�mm���o<�$�x?t9Ww��������Q�u�~����j��W,���S��n-�,uJ����Pl�)t%(GZL�Y�9b0k(�%�S ��m�������^�u�Mj����<5�C��+PMM����Pj���n�0���V��g+m��Ic2���3�:d�Q��*�,j����y!�4��?������3!��?����C;b?�s�	���f���F��dY;�'���f��m��������w��,�A��;��c����h�L�L%�k��]�������#;����8��i����iz��i��F�E�Sj������3\uAm4�HY������Pq�[�T�)%^�p�����S�T���%K^F�����^b��A&�0r5�	4������]ib�A�Ergl_T�B�d�)�q�������!4xE�����0A!��P��6�o�}�I���|#E��~�m@�F����P�����"����E:�^���Z�\��& �d-?��8��nq4����1�e;�Y��8���]Q�y�6\7�T;#���$
��o��D��/[i��z�YmBJ�KB~y��3_Jnt��+lDAl���D�O�u��8������u�.�U��@��8
l-�*��y~���vV��j����!�/�7Q��|c�5�!l�8�[O���"/����\���H�,������Vs#�X�'��G���j�@�\�Hf7��#*Z@��r�%�u^g��XDC�NTF��e-��w�'0��oL�Fu�k����y/	���#U'�
`��
v1���"��$���g�~>��<i��g���6�9�e����)t�0�`K�G��������n�on�)[�.��B�m�����49�k� g17�Y~���?1���}��<N������t��������G?F���K8G,��sw9b���$T�V�fx
{�Z���j2���6�W�A��l���!%�iSRn[l��EIM� .��S���e�[LH-��1�/�����S���v5S�3`oV��AXc���{���7�v��V���Q\���}K�`��
�TNl�-��>�F�#�����Nr].��cc`��lkV%��Y>v�������1��"f
��I���7~��x�G�d-A���W��s�yS���������<�����(��?���;�G� O���0d�5)�A��)��s�{�M^Y�^����XF�No���8�o�Qa����:�x���p�����F(:�������8C*�hS[�k�$�q�D�����re�x��.*� ��5q��
���$���P�~�l2���'SL�L���E��Vt����/K."�t������� �����i���������{���
;�A����\NT�+�i>��J}g�]���Y�JbYp��|���$$M�#%��X��������^�{������4��=qX����X������q���[
r��������(�r�������-l@���*���V��w��r_o���-V�m3�� Q)����^<�O���4zG�L�;O��t
H�I�d���7����y��o�-�81�i�2�t����������=�
����������=!����[��~��V��E&!��YU-�@F����~�t����;�X$���p���t��,�<���>��<������D���^�#����Bp��se3-�<�]%���P�=��_(��nBC�b�"0������n�o��'�p�5�k0]���~�M�>}��9��r���Scm�
����/��m�g@c�/-�����JA��2��:�ZCp�^w����\v!u���)�pz�m)$���}4	���$�O�D{���FH<��*9���z��	����������p��>a��c<PE�OF$O���mA\�y	���ek���l�N"��T��6��x��
*U���y��!<*�P����c����`29�0B�WjN��R�n_^7�~)���bS\��q�����k�-%�Vk�JM���G�*��NJzl��e��9�M52�,ok'Y�%I����!7&3DO�#���J��H�-L+&hB�(��+$�'�D*4�s3��~{���{���K��P{�����j���V������oo���_�p�v8���������9$�6���w����A��i|�B��+-���G��%I�W�=���������!���>S���4p�=<��������=��^)R��p�$���}D��3���D�mNH"�+�]�4��r[�h�>a�ZL�����.rQa&���%�����"�5�i
���!2�1��#��#�_�c4� �nl(����q�����FzO9�.QgQI�i�r^�8�s4=a�V+)�eceE����5s���d����*�N�����8\�=���3���2=�E�!'k�i��
�wZ���?�xk���'���j�sE����� l�jn����ue	fQi�[ z�&��8���_�L��C;C����7un���v.
����,��X�Kc���nD��I`�BZPA
Ne��\���|bq���0.�%#�hv��7Z��0%�����!eG[K�72�}_�Q����3�]�Q���m��2�_����@,(��8x�����������s�|��:�m���0nCF}����!���)����*�;�i�a�P�M��)���B�R��5����K����������
��:�������1��c
��4�U?�a���^�=��L[uS�����T�2,����z����4����M�,
��R_�asr������3fh�T�$���e+������[y�X����969������������+2w��O�>k�,S������w
�M��Z�����8is�-�y���~��A�]!ONg����.=��K'�����}7��G�F��g�A��0N��I��X����0N�A)U��+i����$��_���������p/h�������#i����!>�j^�3����0�����j*�K����W@�Sk������n�Q�&�Mc��cN��'R�KL5�lQ�7F����.��/�r^H_���P������&�k.�6�X�4��B�9c�	���H���X_9'�{�����`���zrV�E-������Z>����||������������$���s��fA�5�u��i��47�Or�l�E�(%Sk��4�m���1�����n~)2&���Tb�aiH
	�F�4��)�G~��ES��>���T3[mK��y�Y�RA����)�����:g�rK_T}fh�wE�C`dYpl�9��y��);n�F�<y����9S�x_t�qp��s���yn�Y�+�q�(�\��"2�am+�A�E*%��RKc���r����?��:���7T^{��l��f�9��y���-�=�/M�C��.�!�	�l��>�~������������	��`���7�R�~�����:��\�u-���.r�R��{���z�Otoc��+�
Dw�}���������8t��)W�E�ag��e%��\pV�&�F.�[����6������%��b��B����t���/oy}��P]C�W�&��;M/��Lr#j�������|�	������T��f�<��_�a���f�����v����!���^����%8^�H[p�X�Itk'm�v����#??�3Xp$M�����������������A+
��qi����GmHKr��_��,�����c
@���9��'
s��X��wF��l��}��K�6	��9=3Pf#���d6����?��;k�&/��}�<���Y���	���������/�������W�
���
W�5�L��>������}C���&�}����>�����w������J&�u�nMa�#�a�bz����<��1Hr��ygS8$�������4xPt�l��J6=H���E3���\'�@��Y8���&`����@��Yn���As����p�4��<��i4c���q�>k���3��H6���6j�7����e������P�&Gg<e����@Gb����L��,r!E��A��|��W&��x��k���1�%���+���VS�kRb�f�~�fd����p��{���V�������y�z�p��-cx�~q�N�p��R��A3_3.l�YD�6�
{��R�S*�J�R�I1�ei�����>i���U��B��\o'��1��E�=2d|<c�������/���xnQ���X�!�(���DY�$\�4>9�
��'�9���W�����:B����<�)��>d����@���>���)�Y�+	f�
~o����B�	��`�M��?2]�m�y�[��V���3gx_��>�,��wT]�,_�d�)��������a����_�O�i�u����R��[��v>2#�-��i-��gY�,�|�Y���(�G~iki�	�;{��s����d��Ng�N��\�:�v���n��7���f�w�S.�����!��3�����Y:��$H<T�p�>����b0wX�0������m�z[�_'B�i�n��^���kqs����c������*�P�*�h�ihc��;H�UK[�6����]�qN��������`s���C�)��9���&�5v�Y+�A��v�d������4�u;���}�fI1�E&>�������*�N��
-
�s�^��/�����U	���MD�z��9m�gUMi7B^"�@�G��������6�}��}?&g�D�H6���������ql�,Of�}���HH�6Ip��:���zu�ARB���k&��FW?��������"�y�{��T�j&��W��yt�)[��
K�P�1�v-����iy7?w��LB�iI�l���~�ot�b�q�^�6*$o-pd1�c�l3���W\�_�vH:��A�'M�l��<>����)�zp	���y��i�d�RQv*�wH�������b�x\��9<s@�R��R�K?���N����o� ���5�})�-Wv�XG������O�c�� 8�����-fJ��
�$��AA,qI;qWT�juw���Yq���M9o(��nDB56��X$Nu����x����+=����u�S>�����S<�T.���a�����w�v(�����{>�k����������a�']�!OR-WE���K]#x��^��;�U��tz��W���Bu��.�jh�"���vz���"��7�X�n��z>]QGK�����-;��]pI�e�U9������y���T�=;��	r`)���)	}&�|/n�ne��8���� []�C�}/'��#}���p����*�}����S��-1�7{��/��|L5�v�N����}uy�P��$��-���2����]�c�H��~�}��<a-��fx�>���B�	�~:�w������$��7���+����J�	U/Ch�$/C�7�c_��=}��!����!����)+�/�xi_��T�2����!����i��Wz�S��A��Sl����{h�t�mzH�"X
����{��P�rq������C����#��:P�l�q�="P
:���
[r��q��r�A;�S
�f�og+�\r*��t"��"�S�[���v��k����jLk���:��O���+�{&H������-|�#�-��F�
|�o#U�4T��B;%�$�������*������p����1�1,����H9Q4x��3]<��*����E��V-)�����M%C�������l}�g�{!����e/R�t�����L�����@K�!�����H��AH��:�T3��D����p�7���������z��
�:��Jc�Z/��)#&�ebb9��
�X
��^-�����V���@L{�JM4 �K��P7��h94��]|��5)?�\�P����P�oGj�������O��j
��$�����d��?bT:�%$�R����b�RB����i���������bF!����X�KX����2�jI������~�������i[UpI �=o�v`J/�6�x9���{�����Z�������c8��L����O�;�y������o���5��;(�#�����a�b+�6XAw�`aV1u|��~�'t�oI#=�����2�~�
*��+�l��m�:bt����4]*W�R8��!di����(SV���t�~ r���w�}����[t�3q��k2����Xg�C�����1���,��SCs6:��
�>�V�����������}����M�t���r���p_3����y�^�Q�f�Ck6C��Z�e���Tg���I����0�i������`�d�Ng��G��y��bS?��	.%EE�������W��#|!���Qd���ay=����A����\(dE���u�
�"��%����/�e;4��3��-V�B�=�9�#�d�=z�v���P�:�x�p�1V���A7r��
&�W7��7r1�#j�F}���$��o�j#moh�8�O�?�=k��q% ����x���w�������������elg�3�P���S�C
������| ��`5�b^��%�9�A�9Y0�_7?me�������)�|J>�MYA���@��z'�n>sEsT�t?���=�n��	�uh��;������{).��]���2����D��j���n9�>e�O�����M"��e9+)L����CJ*���h�BkLuiA�_O�x�
k��;i�;@psl�(���wJ�L���/x��[j��B�����4�����PBK�e��v\,��:t��w��<�-d
���;�
a�k�vk�"�
�F�]Dvm����� .���MD�mJ�������]��	��<S�y[���S���Ha�}{NX=���S�vJp��\Y��+��vs���
Jb��8�=]7�js�}8C��s����������|<]������v����_�Bd��+r�h_������b��u�y\��k�������	���]%0��hX�j�������{kq@A�!��up<5U����l���
�A�w���*�8o��8������8;y�������_=;�����'������������~���������6�:}�����'��~s���`�?)���C�e^���<��t�[)��n?��~ ��b��HY��Vjf�'���eY��"����r��
D�2������������@� h�~��M����B4��l9q�5�W�zAX��v�X;�P�����l�X�����:�L������;~��=y&����Z'��V���/������N0������<�~����}�)z��&���s��-��9��������������
���n����8D�e�:��9�1)�[���7q��2\U�b������U.|��i����8�S��bA��a��
�q����w��i1.o���\��h2�.;�'��8��+XB6��X����4��r�����r�����	=7������3��]�^�
�oE�	���4S��rC\�u��q+���^`���j��F� ��]�����
iI�1d_�uH�F�_����~������I�>��Fs�Vfa��.z
���)����\x|L.|���wtPj+�B�@A���"�{��L����fo�g��w?��P�Tfo�O�|#a��Y����}���-���|���a���9���������Rx�1~{>�������SU���SyJ{d��8{z�w�U��ES0u���`y
�J�rQ�v�	�+��;��:E���d����;�=��e���5U���J�����S�������U>����n���Z����PJ�n��Q*I��e��_Y�9t���N�M�}��/��a�����NpD��M$|@n�omt�I+�� ��7��K���%��afC\�������Y��L��
�U$C���e���R���}�yE�����k��:�|���k��:Z�:7Ol*B'I�����������8cy�f�y�Z�mg��x��y�U}SY���x"������9���T#`��_�$��a,e3���^�IZ�����|���H�x`D�\�oZ~-�&Q�	5~2�"�!6c.�E��d��

�f�l���.\�l�J�e�4����K��}�����C��U���y���X\���@L��a�����������y}���[t`:���4{����yd��I��x� ����y_�^��ba`�za���r=^�����c����X�W����
G���W���^Qn����k����0�GC��q:D��)H�������:����u3^���=����r�>���t����Z(��i�3D�Y���iqg-D�ry�[tLX�+����-�s��������O'g�/�${Y[�B��J����-�/�DY^Z����]��G����=�`�
�} X��\�A��P��[���dB�K�{�ig�� 
R]i�&J�r��p;���k����T�X[��B+�}���9����ZGo�i�������xn�K�h�/����2���S|~G��F~��GIi*�T��x���J#�DYx�K"��J�����"?]���S)����$
{z_-�y���Or7a�������/�	���i�w�B&�"�����b�������,�n��H��qJ���xj���1��2�W�6�i��$r��p��Q�?���9��:j3z�����������MI��	������b���<6�F���M�Vm>l%��<��\�D`L7^�������8C��@���??9{��:�ta��b-�CdVf��_����5�/��B)�rE���o)����E������/�0�+Wt�%W�����8����g���������������D96c'
�/f�%^\������]
'�L1�Apk�J�Q�����b�N�t�p�E�Gl��h���e�\��T/.VZKE8��_�?P{u��l*�5�T�]_��dC�2�4��i���K�y���-M��M����U������r�;�����Q���'�|����pm]���+�d����RM�Jx����K%�B?4����nXU�]U�z����e�B��@����S�����`�
;�b\�)�D��!�s��$�/�6y�:�f��n@�^�c�-J��^��&	�N�x��������s��X�:�n��t��:�/F�2��!4������� _�+�Z^`e�g�:!�)a�P��12��P�D���Y�@&�/}�:����?(���p�������t������\�Kyc:�AH���z�H^;N	��Q����,M.���������>9E
������
|�g6��(l���E����	~�cv�)$ 8����|�������=@�����k��e[���}A��c�T���mu+v`���Z
!{�F�Pr�O���%.'�i������F��3�y���������6zu������&��>�Tm��}�]���iV��#R�P���p�N�BF���S�Z.����Z^�'Z�+E�;��
�s��C�8)����3�ms�**��
#��/��.a/����4]�W���Q�s����`v��������G9F��N�AK�R��G�t��NX��\�]����@��x{�+�s�q��m��	?�{M������tt��!:��?��d�d����!�`�\����)m6���Sz�m�,�zy�(���yn���H��,�cK��;���7��kP����Q��Y���U�y�����3����Q1-6Nw���d��>������o����g���Y��ti���m;��h�������c���|db��'��ToB�t��Io�����F�0	�*<I�y�&��be��b�	������rE��j~��Ty6������L���/�jc?���p5�j{����O����k��(r0LS�I�10��!�B��c�����,�
b�I��6�6%Cm�s������Wty��y1��W�b�������
Y�����W��Lf>�	73�[�x9�����&pU�	p����B8�)[�q�y-hZc�?��]U�8/��V����y�L�!��`j�����c�����EW.���C�xN��$M���a���\���Z�\��kQMYN1�s����p�]%v ���FMUG1��%�sd�tV��u��)�����������3�=y�:TX
#���	�^)�@�et�)���nI�RJS;W�����4����O��L�Q��1�o��S����aV
���7��/8�D�t����c.J���{2��P��a�kw/�+7��s�9��Z!t�=C��)����
=~�Cz�@v��v=Au��S��J����c�RjJJ��9	�F���y�!qc���GO#�p-s]�6��H|���A�,Pu���O��$T#r���X"� ��V�1�������il^D��K�=��������9�����&����S��R$+px��p��g��1t{q��J#�Fb��O�J�������|�F���h�M�����|�����>D9]xl�GJ�T�.*J:ga;�?"����,�^_A��|-�@��C�w,�h�H�,F��2�g�H9���#�8T�oEBX=kL!a�R�:����
�]
�:l��"2�P(�@�RV�7���z��xn-��*��eR�u'eP�ux�}����������@
-��LU���+%�d� ��
P�1	�b@|��	���]��d�����fTyXq,m�I�z���-[F��L}��p���`��z�A���2��-�+�3���+�A�e@3>l����U����9�������X
<,��:�h�B�G�������`R���T{Cy�$���������{�/���&������c�����Z�7��"r%>��d��u&�����x��L�&�	V%3F��KIU�z�����,ISj����5�=O��!JsI���u"H������9�����0��PP�z.-I��H�T�9���oM��oZ���C��3!i�:r/���ZM��3�&�U3�"��yH��5�n(x�����u�����S1�A��\�D�J�E;�M�Syh�;j/Ks7��e\��{��5$���B����d5�M���0�(�+k1D���9�wOc�'�����^7X�L���<(Q@�g;k����^Q�Y?�b����S��H-�W=�N��TGlEQ:8JQv|�����M~l/��?���V�m�Iu��}~UYL?|<DU���G����O��j5+���H�(:Omd<�4�������8��u	���C��"����`�����[�����BH4nH��\�3LFI�u�E	�����)�;�ag�
����42���R���F�q�i����3.84��B���y,�t�*Q�R���1g�,������]���{�#j�+�|���
��_�F
bq)h�
Z�*@���ASG2T�#�J�A,�����#��x��t�p������B�
-�]��Dz`�%#�����ii��;0cI����G�M�]{�Z�uU;�@
4�F����z�R�|��@g��6��'�����]�R6������O�f���qF$�-�M<�P�����!�m���r�a!���CU�X0����hh���uL�R)���"��{P�������G���f1��$�o�s�![aS�&>�.:*:E@����@�d�l�������G:�M<����pj�D%��Q�z���������+Jn�����\|�S�cx�-���]vR��1�����E�3!����G7�1���rec��v#5X�/�w0@-���Tf��a�G��0���=��k����	���h���d|�F���}��)M`��������KC��J��(D���������	�%��q���YLe�TwbLe��P�q�������r��������f}������%t�G��A��uD.���^;�:����!��q�x���}��"*8�N	�c��~o��i�6����xOQ���<=W�eQ���0zL]�O��~�K��-N��J��O�~���6�'_[/���K��&� �8L%\}U���W����#�����Mlq1W�c)hg�j-�0nk��+���)��[A���AG��|a�7�i����&l��y�}{7@�|��������S��V�/�s$Y��f&s|l�m"�u�+��j��x����@N�D�nDn�����Q����@�q�;�!H�f1�au�6^y#D� �8�������6t��I<��=��=������LG�6�:,�.��(D���Slu�#�w]�Ce�8�f��~�w
o����496����Il�"x�g&�����������%N��w-���XU��R:qB8�C=�_���[ ��o�&4�[�U ��6�Z4���xo���������8���q��mR0�E��#L2�v�EeXJ���b������+O.�7��1*]w���]|�0��`����0R���>���Bl������v��� �B�����N������Kw�&���#�c���H�������3)��C��f�u ���Dg��S2��A^���%�2_�1��Y{�{����v�?��>UC���mW�[,&'� 	��n^dS��K���P`~9y}���������O�?~�pv~����O2d<�e�5���-�����DG~��>��O�(��1�w^-����	[J���p������y/���\6��9u6�����o��������]3A����v��)�h�I��#��_�;A-���n��("���~�mQdRw��|\���w;�{��B����v�aC%o�v�v��v�t����dq>j���H����G��%3D�`������V������jz���\�����������#�|>
J��sm�m2��JN~�Q[y=7u�HZ��$Q��H����y�������o�)0��������
F���&��|����1�����bc�!V�N���()��������d����Ql^H3�ZZ^���A6�R��*Kz$�����@r�f!�HR(����
x�4Fv�e���$�B �%��Ra�'�T�z�k�$�|��%�������-���Cc�%��*R�@���k1_��4M������"aSW+�����d+(#�����Sa���X���!E���y!�L���*���Jf����6Ew�������CS��A�VF �p���+m	����c����%!�0�`����j���\���v�J�B�K}�l�[����kc�I���EY/	�dQO���x����v��y������r�lL'�"���
��{��/V�Z2�J��e�;���(>�H�	���xt�	0��K:��p����|	'�;����.aS��X�����
��z>��yrC�����h� ���E�L
uSJ���U�4��z��6^q)skh�%I�'����C���V�9��m_ z������t��N�p.�+A�:&�ub�]��A�!x���&�\+TYp	'��Q���|�����#�O%:�>�r��\���X��O8�4X|"�Ua��m�[x*x$�i5�tY���d^�Pt�K,GO��A��V�Gm����w���d��U]��O;C?�N��q�����������L�����Y!�~+7�^:r�{"�K���@���I$�5�&�!
�
FMC�p��)������V�����@E^D���5
��l����Q��n����F0R_qh���-����[%DG����G���$�w�Jq���-��"'�:�#�;�1.hA�~�bp[s��{��Y����������J����Z��i�&����L��m+�z�����@��i�P�~��������,�X\�&�/s�
��VZ	���h�/��,��_��SW7��M�F���"��;�����B�Q��EQ[u��[�xX���3�G�}Y�$����0�SnA��1��iDl��������E��������5�ecJ7���e�|���/���U�,�M�P���$�Bg$�Q��'W��l��N,5������Q�0>,=:�,���A�3���*�b-�_�2��X��%)�b1�b9��7��r�p�.]s�����S[�F
J)]��r�#B�d���st/����uD�AB��`��������k& @�������Y��@�cMx�Sw����|U�����	��MG���7�=����6���7�B
�0e�u����(���@Bn�::��	�vV����$x�K����K���+��p,����|���ZS90�UK���U�_'sR�%f�eGS�E�����]e�L�I�k�O��95"���������+b����D�.�(g��(>��N0�.��B�H�#�f�������3c_��q��Hb:�?�uxa���fy�0 ]�z3�7v��ASG@}&ck��We�'q�����h6�����W~��������OG,e�-(���`s��cL�)\}�skJ���o���j=U�]�^�\�C�	[Z�w0�I�q~��D��sWyV�LF�-����;��;�G��u��$	C+�9�4<��4�X!I$��(d�$�mG�u=���:��$=Qv1��XH0}�Q	��0<�\
��`�6�
>#�RWI#����Yd�R~���h��d��wF�R~u>��������c�_��t`�yR�q������`0�4��5F���`0�l�l�#���6�8P���c��������}�}B�n��"l�vK$�r=�S,)`���+�����9N��%���.�0�r��s��^9	c��X������Q��g�v1���&����zus�+c����=+�B�@Y�?�z:������)�����1�c��+���{�F�@�Qf���r����������vX�&���� 6��&�7��/\���nl�<I�����O�L�����(+!rbP���C��y�h;�����JG�(+OM):������,��gA�p����q-}U�����4�����4�Z�e�h���Q��MY���N.����4l���3�!�m�iy�j�:)!KS�6=KB����3 �?�B�VQ}��gn�>��������6���klY���
�_G��r�E������#��1J	z�7���cQ���@����
|N)�]y�R�B�X\2}:�M7��6�9nE�(hEV>�*Z����������/���D���v	��:������+�c}ZQl����
�L�ErG�������BJ[c��6=��ju�����
�Tu�5~'c����a��!�4GOU`�D�r���?���k��}��t�Q��
��V���#��Y���x4)���.,/��)������7��y#��Ku�R�i�(O�����w�"�p��*�4��tmv����EMeQu�%M�G�ble�jA�a.e
�yM�x'��Q�u�rC�=/�g�=�0�I�H0�3x��W�6�'����<�������	��������^T��4d��������<����&��}I3�.�0���0.Pic������^��r�=�F���8����jY+L�F`����Z-a'�p�������}k|����qP:m��?t,��k)_/������1�0�������^�:J-���<����:�F*~2�1��|0�
{�~���5a�t��O�$������G��>�������U���	�(��H���b"�^\G
XAi���X�Cs\�;�!Y�RY���^}�b()P�����G}������+v/}�����S	=F���T��~�|�S��>u�W���"j��~N��C�^��N�����m/�Wv9�uO�z|�����PvV'���B�C)��v�hz�:���{����]}�=<��d@�r��;����S8�x���xh`�(��}�*�Qh�1�����b����a�?=��}����$;���A�����h/��<(���Z����hg����������D�v,��/R]���BJ������b��������9�����1�k�0b��Q4�����
7�q�lAf�eIUb;��K���'#l3�����(����{N��������}*}@�At:���HK�^TVi���*6N7�L�S��`�"��k�p/t$Ct���-��~D�����I�����FhX���J��Yd�����W�4�H]6���?�����k��4��^-tr~*����fz
J��^�l2�Q�������p��j$��\2� (�]���+O�Dw���������N���u�<
]���Mrb��N���mfFn'�J���H�Rg#�u����}6���<|@4��::|������&��� (�:b�n���EA�,��K�-���7�"k�������x��|a5�+�~����^�"���l�����u/�>��-I����5x����������}�?���>���0���������J���n��K��d�t��������:kC���}�R�C��
���H?�W�C�,P��bUhkg�I�)�EI[�o_�Uq��n��/�<�Lo�*��kX�eU~�7��qW�cc_��V5Ua8�s�Q���&,s��U?�`�D��tR5�FVg���{��g�	W������kq�G��:�0��Ct�p�M�l��s���,qE�s�t�K�K�
����~z��	[G����Y9��XI�u9=����W�$�0~8��T�A�����a_���:���uU�d�������zh�.�0�+V��9������{D��#��^���)m���
^�/��	B,;�]8f4������JuDJ5�0��KP�/��A������D���~��C����AE�-�?F/^�
t��HF;��W����-���Lv��{��������|���r�h)���5�� -5X$��{��P����Z����;`��>��Wm�g��HSF�+�=Bz��^�����X���rS����-;���2��Q���TP"c�s���!����	
Y��q�
A����:U'�W�s�N���E�+��DgD��h(�.��-&(xi����R��3��}����T�M���-W�z�(���:3��I�5Dn+�S]�m�F����W�Q��<��=
4���FX�0�b)0-:�	�[,|������Z��b����6E�F4j�f�D0�p�K+��]�����YL���$'���w���%�yK��J��C1,+�@�
(��@�[h���%��J>|�tH���a�d�yn+Y�g��-m:r1����������I�]>K�9�Y^��$�m��}aB����7����,��\��EY���6�<�P�����rV�<[�ZG��4��NK�z�Z�O�s��z=�<�k>M�w
�]���o��2�2)��<���_;��Se�\�{���������kN����#�����yw��`�B����T 4.@������.4"�b��������1���D�{���^m]����~�2�}�A�k3���v�\��.��^����|Z�/�XG�-$�	xu�l�����4���,P������'sk��Ab�XQ2jC�^�������S#�Y���|Z}������l�5-T���_H�����_H����B�+B��;��sWe1��4�6J�w{O��}�� ^�C���_�h7E�UA��45:`8W�E�cTX�U���)��[��X3]���N�C����;�Y/&�re�b�|y��Ye�&.#
��AEI���'o�����"@�?h��O�f}y����um:����t(]��SN�c���W�i>�u��Q���L
��r�t���Q��X��o�0{C�
������Y5��}��%������X2�R������
q6��p�~�,��{qz����Q���k����.#���D���Z��8
6?^&u��V�jG�.�kB�r���ckf���<{���3�����=.��I���Z�.�O��@r��!f<�Zm6�����2Pe�v-�i7<�%�O����>����6'�^����t�R����2P��G	_��(��]�F�~���i������� �8�I�x/������5�*G�m��1�j�gv+W�1�Pg;�C
���������]P�
X,�4���C�<y��W���azK�W�����)[�	�����	���Ko1�dC���\;��	��%z}�%�-��"�6zKK �6zK�3<���
��v��p���8��zK�=Bo��G�-��5�����i�����w*#_.�f�$RY���0Y	�������<����oq�����xS3�����>��&�m�>���8����z	�D�'���<�E���6",g/��,)!J��)���r��Y�|�5�����@Z
Xb>!�V�����'Z,���f/�U�n����V�;��-i�8�,2~�����C�IG��f�y���^�Lb��2��CS��������kCE�=f�����%�����������]��1]�p���K��1���E�*z�<��[Z_^M��l��!��������$��._�5"��Z5����V�/����z���$s�C�.5�S�;�n��w@�k=�*p�l,-VN�.)~���K����3[����~��U�����9���}/:�\K='�ixH��)W���d'��C���,�F0Tpn�Z@k���6�aX���TbS�\N��x��a�)��+��u��5X���L�XG���b8*)=Q����ZOEd�QDRlK�XP	U�1r��iS�������B���ou.�<�F���o ��Y�%���TfR��k6���1��^1��j�,��&9�W��N��J|$��kS��RP���p5.�.C5!���eec�h��/����q_�h.f �`��.A=�x����!��S������HFA
��cM���L�v������l ��N���y���t1�L����g�Y%��f\O���+m(�:�5�m+���0p[~�V�Dywh�O���6�5O�H��-�����_�I���E��`����N8@��~�R��u8�y�8��TTg�3G�C9Y���jH�N���8�Ij���Y�d�z�$��f��9��U�;4T,C;����j>��M8��x
��\"����n|���g)&"���,��4�7�>:?93>�����w�()U}h��)P��PMLq��oY_�;,0�s�
w��9)��~��X2���>��M�"b&�
p�w���;�����:�A�T���%����L���m(�rIEL��r���1�*~�n�>��h��I:�*���J�l�����w��O��(	({h��	J���`��
k���R3R��nA��i=�b2\�/ )�\h����-�?9�y��]
�z�����K:NU�04D����:�PC�]��O�Y������N�Kx%�x
�V~��������9V��^�o���T~��)m�����S�At�����x��Q����p}���1P5�|��n���Op ?*�T�u*_?�O_�'��2��L�����x��*��t�6h��6����i�:W��ZBj9Y��=�*V���
�X��+��>:�
���X�]�	�����"�1�1i��V����8�C��x�b����z6��W=�*�R��������A�J�I�~��;H�����vw�"�������N���9WrH�%D1��#�T:g2��9����
��z(w���.8�$t���qh>����4�O��;�^y������|1�WB,y���zy��U�G������^����;�o�
.�etP����r��y5�����
��������@��Sr�������_0%���������������rJKg��A������}��m;\������r��j��N��/$B!A�����P����f������e={3��>����dK~�HX>n�+%Ko��i�������%��+����C���6���8P#�&#]F�����jr��"�����%���Y��CUl����E�0�����#�H�8����&>~f�P��Cz1�I�@|��X �J}������^���q�j`�G��K����~e�a�J�Z��L9���n�S��Z*
:jGh�i��n���[���
!|9����&
%�uPGP��W'eS-Ky��d�
��sChc���a���JDF�T~��I�?!�w(���%{jA5�����\��Q����39@�k�-P�r�TP]�kBhw�*��Tj�J�������1i�j%�d�'�!�������"���x1��t�v�3U�*�'f�L��c^&�*�w�IP&�O]
��?��$S�Ll���pQr�DJ2Y�������I,\��MZ:B|\0�
�Ex#b����Y�*�*
�6�@
�t�Z���� ���,���1�!	�e����e��rj�������������\�Y?�
�����w�(���|e�\��#��*a�j�_�[�H�a����=A����w�8���o&������ou[���F���c���(��A�Z�o��z���K�����@x�S(���7�:���
����Q�k�u��T������k$\��WW��B��*�*PQ�Y{�����J�w�{���~�@�E�f�$���k��^`����&���^���f����#�.��X?��Z�������	�g��������c�l�o�&t�)��c���_iL��P��L��o[3yg�74�L�����!.���W������@��	uY���DMP
8rXU�bV��#�E��H�*�-�K��m�_�����������cw��	R��O��N����+�v�K��-��[�:����:����Q��r��~��V����z��_�<�����	Nbuh�\����%J�������ga����
>����>����3vn����� ��������C�#JLPI.'rT�zB@2
�X�>qCT���2�Z:�]	GDC�T'�Iz)g��u
��d@�B���F7r�f��Q��O��/���X�3JJh�\�Z�Zo�D���pk������l�������*���j�����m\��>�����ztr`����.��h��P[����=�G��������=9+����V����������K�����c�������+=SP�[N@LN(y��H����TK�"ula�q���������_��������?7��Q���)�IjZ����vr����c���t
U��g��Y�Z�_V�s�_u5���QW�m�O����Kr�LJ���)�_ ��w���"��g����0'�Q��P���p�����(��^��d$�UMs#�n�0��GR����Kax1���o�\5��i�Zp�4W�	���������8�vq�4WQKP�?�2����|/�4'�O��g����*\������r�%�;����z%�f9~yY������z/�����%z�_6�
S�^����(d?����qyl;�~���X���������m�*�'�T	��l=o�m����zY��c_��i=�Ie8"�a��a{1j�G���X���C�)6��SG�cF�)����cN��D��*���L�+��0����i�lT�M~IC��u}'�����GE��zq��g������yr����V�zY~��;O��bI��x�_/]G/{�^�����.���<���l��?ySL+X�y�<�v!|���Qv�N���??���
�J�>x�5rI�a�32���������QQ���i����t��,�4hK���~Ii��LM*w��X���/����#9}��K,�G�z��lo����eP���������w�����fFc�����n�����xg/��������^���du��~���6w����>��>w���7�����cd�I	�
����*P�CC���\��Z��������R+P���z�Z��Z��c�ZQ����p�L�6�tB��������kr��1����J�����}�6��_E�����7�F�[�&aB`���;z�r4=�UoI��G��
�3�3�p�{�n�x��2��A�����TsS4����
�\��[��P���hI�.�y=�E�`E�w�]c����M��sXJ�KA�(��[��C�H�#����]�X4T��J�#���5��
���4<D�cB�������ywr������_�v��_N������f��$�&�����R7�#M���j��F����-W��U����E�b��I�a����& ��W���]��������	X�i.^T�2�������6�Tp?��^���.8G�^���@
�=f8�J�C�xHz�9�a�7$����I`���v���-�C���~;���sT2b�����"����-	��
F�R�W�ft�r/����h6�H1�v�
0�xU���\��",���D-��*�@oLq�A�s�<��(�Q��������
�W��|��>�D"�h���a6��e�����YzLj�\��~h�g��&%)����0E;�m�l�t��c#7���h�f��Sx~cEH������ 0�������o����������������z1zp@_`Vpm�+���.���j��&�+��s,����r����z:�-N�J~�v�]{Y#Ht����Dv��)�ub���� �Z� ������**
��nY�p�\�T��
x���yv��P�����%�|U~g �u�!�U9�X��l��v�Vz�)�Z���IC64i��S.e�QPz;�+	��]ky�R>��r���~17zE�7h��m��``���+%�`�����w���yy
[�X2=��Oc;l]��c�Y�����r��j��Y�I:<E�*��������W�Qv'y�=0m�y� �������?���#��QT#����B�>��������s��U={�)�vxo�_��x�#��>��G��c���!X\���z��#�w��a��?�d�����s���4D�-��\��/h���
�[��C�O���>������
�"�����z.��k���U�\T����;��rs����=��VN�er�+Q��'��C�9����X�o.:�(�^���C'|�F�ZK���^�����������2�I�G�tF,����������g0���@��:0����`~�������������l�Y6>w����p y+��xZ�K�Q~��Z����(���k�Tv�;��5t�%�|��nM�;nmU������iz3��� [����A!��<�]xl������$}k/a�v �7>���9���w>5;�mu��g]��P���k�����z��|���=T8n��M�b�dU/��@��?��A��3�L*;i�������cL)E�$�2�C��������0f���������]�����~a4T�!Z&��j��lo���������)#�0:��?}���w��{g'�����s�I�u���(s�I�Ae���u];���6���D������p)��1��W:	�\D0��~|b��I����b<���{�����?!��@��x�vmr(0���3x>nR��J���a���B'r^�J�)�������	a��/�3&�b��U�m��r�^����>e��C%�6��X���Jl����r:���!��To�A�U����~B�������1&ih��2�<\��35S2(�#���F��E����1�5<R8����Y2.�aG�
���
��UWU*�o�(����=z��f[�����)�ri&��It����s�@/�$!=�z|d��fW&�Y5�����������j���W�����������OO��������LJ^�1L���V ��'�8y#-u�U^�E���������d���i3l�Sm���>��8y3������t��*������ ��5}��s���`����^<W�N�*��=��:�G:|��WW����X��i�|����e������H�pP�
s����c�_V_��k1�&��>3���M�K|G�aZ�y��t/p���
�6%�RE�*���{�c?]a��/���V������Z]����a=�y���Q�4�Pq�Y��,���]�mi��Rz����o=:�����8���.���L�/(
i��;�<���+�d|��*u=7��F��PR�
o=�@#�N�*G�V�2�rp���R��e������6��j����D����s�O��4���p��"$�P���e,�=�"6*�oG�B���X.*�DvB���De��X�U�7I:��B�,Y��b<\�������C��M���a9*z9�T�@��g9&2PY|�+�+���#���!���X��`51�
Y�t<Ydw�u4~�,���������X �w��B���\���F3q��q���u$C��ZV��1��/^��p��
{�-�jw����2�T�'H	�Nk�+Q����,>�����j$�����Y��Y�,@g���\-���7����Z�����_�d�p�]GH���r��&w�"�����<"��"�%aw"�Q����77���N��t�'�M}�V0k�v�y��5����J**o�����FZ�6�f=ZNt������Fhu����L"~�;#:�\imY�����f%���XJM:�]d��j���s��_�:����R�u��hA��}s4#�� �d����.-D�N����,aW���X�H�MT6a�,&"���o�
�#z-Z�{����fs+�%����[�A4��b<lp���U�?*�MW!���b>=����2
���@��XF���]�`>�%P��bD�4���b��3
���L"��4N�VS��8��1;jDYR��x���p8�������v���8�`j�������d3"���O�����`KD������S�������r%��`!�����5������C����m��X�8T�L��~�[S4}���������-������JITE�G��9�#��Js�?����m+&(�Y�[�����$�{QO-�a-��"����
�7]V��Z�H�=H�uoX\����7��I�M���W����/� ��.��m�
�}:�K�n�����K�-���Ld�Z�[��!��x^�.���:��C����>�w���s�1-I3��1�����)>��n�:���q��tf�4��� jw}���y���1veV����"�=�����r���c�l\�}�����#��h�8���\��Rn��T��_y��/�����bX2����PB���#���{�S{��)7s�5x��R�dD\a�}}K8?��T��_��T���9,>!�R�+B9�\��	��pDD���o�*o�Y������#���F?���/�8�Fs����P�C��:�$���������oX��B�<��kM�i�F���Mr��g��.����%}�"���lN+H�7{U�]�"h��H
M����-V���quH��5���Wd^�RnmE�B�pUS�+(��V���$Rkf��g�<$�euM�[[����R�RN���ee��"�d�����B���W��*���@�Dh����.������`��������O@�B� �����P%���s����!��o�v��.��n��v��u��=�m�)zO!'�3,���S��&�?h�B��� �w���IYXO�N����E�y2�CmC���u	g�=�t�X�v=�!��W.�,��2��)>9�������IrR����#���+�>>�ldJ��c�X��,l���S8 J��g�rS�.��y��jY���gxvr�p0��r���a�a:P9i�V�e��R&�W��)�;����-���$��)����b����K�:1iX	�}�G��IO"��V�Ng�����R7�}z:����f���B��i�����VX����18a�-����Y�.K�EM)�p������c���
N����@G���!���r���F
�R=�������iX�N�/�m��C�oLK�������ac�&4N�v�u�~����}>9�w����-}���6�^�9e����79"�%@�����`���dxoy!g/o�q����1��?U%���(����I���H�J��<},��m9l�u�������� ����#N-]Z����/�p��T���dC8�$���w0�����k��x����7�!c���d��-�����Y��,���������e1���1�Q�-�D��z��U�$���gW���EC>��J��R\��7`<T� ��B:�"�C���br`�������v
��w;�tdBC�#%W��G�]��*\d����aX�H�}d�w�%�dE?J!�����+]�F�t%WW�������6�26r=^n4n�V�=^G�/�^��������IT����)�CG�=ae�����m�����d`�_�[$����T�����&NQG����)�$�#��c@���^�K�T���(�A�9r�Q��a�`��$M0�0�OG��:��G�-�R����Cb?4��c��������s�q���(�j^_Ry�W�3�N$���#�3>��K��O��Z�������~�x������W%��6Rq���r���u�������3��������I��~+t�&�^?�;>�F�^!yu����65���zA�������3���_�:�E���0U�/�0�u_���7B������t��xua4�:	�-d�?pL72%�wQ�g�s#�q��Y��;����8�N��Rk�
�v���|�?����ys�KM�G�r�`/�=Zu��������'�x�����X�O�B��$S�)%:q����?�<Q��e�!Xt�����D���)$Wy�e��q��I�p����,7�,f�����d����
C����2����0���I�X�)����
:c�r�������gd�)�8�4-=��*��2����������
r�S�F��@��?C���t���3������OqX19������3������nXO'��ho��8E{�����Cs�b��?��&+��,�(�l�.F�By����EW9*�U��r0���p��eV_='E������#��������_�r:��|W�7�=�����@��� ^�X�{��rW�|HN�0!�������d=�$�~\\��1+�^��\�p��:���%�d�R����\H�#B`I��1�U�a�����^�_9�O�����}�:v�'������{����h*'o�i�hNhv��i�BJ�����0_^V�Y�h�������G��,W�����18�����Z�������|���Z����U��,�>"����$=~$��nd�����
"�R^�J�Z�S������5��bR��a�V����1+��8�N�����,>8��x'�uU�Fc�Y���.�.(2�9(����[����<j%�v��LG?�_�@G��:�r!���l�A�v��l$��ZL���������w������yD�kz`|�&2���c\O9�}������z2������d�
�$vx��ju��5���r�������0�V��=Q��9��t�2��e�H���i�e5�nj�}-�W���'�#���]����(rzP(oh��+?�9YgG�&�����A� �?Q�8���0"k�D��Q��>	�[6���m�����������7���]�����gu=�����zG{hW�z�:89;�p�\f�A���sI��7��\|<��+B��;����6oNN���}��T�[����>�nU��3s��:��=W^6�2-�g�E��vjJ�H�q��7f����f���$P�E�2�{i"�H��,0�����'��Gx��v|�u�����L�N��^�sQ�)b���(b���Uy���[�%)0������(r���9}M����V[?��>M��_x�@��*�FG"w�n$� �8�D6+�
�6F[�X;�q����N�;��A��''�%���[z��
z�Z�[�P��q�t���X*��@Dt�����7�`���X6f��C3�r
r�TD�^�������0��|�1��v�W�,��vs�Jh�N���@g����rQ)
Qo�_4�m�����
'#���<�U{���#�h�Q�v�<���q�G�q�w���r�k22�Im��N�>��������ILk�
i�bf>|�v�^5����I��k|���".�SDy�A���~�
�~.0���F����9�(}�Q�����L�D��c9����e����%�G��W����AN5���d�=/��t���t�8[�(����S����[<�k����-�B����
��w`~�&��n���%�jf����1Y��H�enGt�;�t}�3aWu���S8
�d�5L�
0�u�.�f�F���qe���������G��G��I���TJ��[�%�2���!���J+��m���=���o�6��MG������}����m�u ���d��x@hO�%oP�D�k�1�]��i>�i�	��1%�(���#J����f�'4������Rp�����h�c�7��m������GP�Q�m���}@m��:��B����C�����v9|h���a��vx�
F���C�G[C���ryy�a��a��`p�,c�`6{,j�������'(j)o���2�N�r���(A�v|�����?������H��/��GK�K��:����x���9oL�a��y���	����H��S��,d7�8������x��Ql����f�j&�k�Dkj�����3��M�{�&`#�n%��n�6�����f�����_i��p�+�����>���	��
��f�������kY,���W(�~���6w��!�u��_9�A�l�<�l{{�l;������l��f�����9lM}(��~5����S��W}�RF�<��2%������#�D�Y���e������Q4?t�]DBx�	��X����f"�(��g����+�f"�"�vf"��1����s ������fz|�"��+���F���I��>.n�����56x��i�&_@���,��m+V������'��~�|A��k3������.G_�W����]����E	,|�PB0�
����.G�u<D��=�������bz��������) ��=�Z���.���a
�����N`���~�l�,��C��v�@��6@��w��h����?��?�S>���2�v���G��Cb�) #������-y�"�k���B����N��H�������v��n�x�!�.*�k�����BJ��g��!�
�!���C�;E��g��!��23�$����p��L�|������Mo��!KG���!:��!�.���P�h��]������S�i��f,CMf�c2{���q���V��}�-3����h��������&��'1���k �}c������v��d���D��f4�@������bVN���i�9�X=W��0��&��A�������I������lo�}����������}f��f�3��e�Q�d4�0�O��&��{���1QZ�O�y���#j���C������v?��v�6�����=�s�sGCD�~��t�y������^���}����f��!	O�/ X��~����K�����+�tw8����5+*W��5����-��n����Efbc���y���?���3I���KGK}����^(�t��J8[�����/pd�_�����h����Aq�1}�ZcWCW�����a���v��Z`����h�;Z��0��>��Zm�������:��(��%a�e
�W+�V��eULu_F�H�T/W����gJR~8����!�t�t��)�c0�ucRh	���2��jME�������I��:*�Pi/��-�P�q�WqV1��zN�m+J�]�J�&�UBJ�������4bG�`|Y�&n,���J&TL�>aahV�B_D
X�MSa�4N�E���',D���)��aqN�h�����k��=::�n�5��� H�0V��R��&����.FiRk{�����+W�	HLb�e'����Q����n���B�>��Bf?t���TIl��w��6k${�j��Q��5SnN2hs�H4k��<���gP���w���Y}���3��������\3�����O�d:q@)�.�8���/�P@<'��v������V�V:x�$18���z�<��/k*�%9rT��8Ee+�z�X�xDb7E'u�����4j��b�*�b���:0:��(���&����������AUsI��;��3�����s7���R<�I�yI��%>���� i;���j�2����T�X&J�>�3G�|W�*�j�i��������1�����a�0S*���Q��,��B�.A��MG��)G�r�,	��W{s�4`Y�dS�/%�-dq�u~(����f��I�9������V�=`e:%[�����ui*����,��ba-��T��t�?�(��.oC�c����@c�3���HW��t���[�{S����m���9X+�5�s3��Zc�h����U�')�Y��p��5��X\_�p����H�&��!jb:�j"��4�#X���"�oQ�-XL�`��������B�������`�}�2kE��5v{E5y��jO�<St4���1A��B���{
������jQ���B�-�M\����������ND�y������~����������r���fz�3�N[�����5�
K�����������2�)5�Q��.,�]�6�i���jL�R���~T�[�I�r����Xt3K���|;@'M�0�SJ~��v�<�zx�f�@Cy8��_�����I�����n��k4t�����Gt}���8I�/���@�d#Y
po�Z'�����+���4��
O�:���/��3����e�o�A]�����v�n
��%
���)fh����!�wk�:"V�z'<^�O�Xu�&���[qG#��N]6�o���t��1W�}���O�
n3
y{p���|)d�H4Tv���3�u�*������=����w�5(�����HW�"���iM#���f;����~a��O�Lu	��>uT���������
��b�z�3R���fH�j��;yZ^����~/F�bg�B>�h�%,K�k����?�H�3iF��������a�Z��@�����S���J%�H�}
�����Ip��|ES�a���nR�|�����.��&��u3}<��������@�k�30C#uQ�>.��*��Oe�X���}�������4q�~C��� ��~��A�������\N������9�zz��r(l��1x2Ch��X�-�N��m����hVz�����G\��}��K{���x�����I��O�u6I&�s��2j,�$�O����={7�vFj��:�����!�?���	�����&��|�PV�JM��R�3yE\4���R�%ud���>�o]�������bLX����E��-����,"�s�����O��V���	z{�[<�����<���(B���d��K���< ��v�1l�2:q��{U��Ng��r�&��OJdc2��VL,7����Is�F�%H3�y��������jN=.�=�&�	��O���t�PJ��/��P7�-����W5jE�o�&7DT_g�����.���m�f��=�=�5:�z0ys?��K�w��q9t����>w ��L�N�v�}
@IX���Hy�d����re"��V��a
����O�j-�-PgB��}dj;�1��N�?��?}�+���9��*k�����J������4�i��>��=��Z*9������:��\�eQ7�E=/����J�x����
�ijr������cB�&u�5�#�3���%���&�"c��i��UJAf�VJ� ���b�
AU?\�[��y��T>���Q�x�T������g����6�.�B4��'�
���7������9�9��?,�r=Q�d��BB��������+,��5{��''�x������ ��^A�(y�5��G_>�{BV�1�����b��v���;z;}��������j\��w&`�������9�l��(�
�X���b
=N�l4|+�0��Z!�����2�=�W�����0,&���t���B���@:��z�Fh~R����������r�pH�����+����}���0�B���T�K�����4���|����;�A��Y^�&�B�9B]�A�_��F����y��+t=3����H^R��y�)�����d�ISf��=��;��W�F�[���w�nZ�E�F�����6�����$-3x�r��c���D�C����#\��<���%�v;���E�Y�##�w�
�K+N
(Ya�����t��OG�g}�MV��v,��(�g�(:cE&����pg3�(sM����F���QB��{��y~u�5�!��q�$�?VO�����O��%���<�Y���B�^�<3���BL�M
�M��IIB�q�V�L�U:N4����	�qa�~����cr���w��0
�����������E�]<�
�����d�b�������JOKP�[YG���
^6����a�G��.]c�a�,��0dI����6��E2�re��i9^I�4VQ�x��u!�ud��������w��������T�	�V�����0H�Y���t^������[v\�^��5����wq����l�.�*���]em��)W��a���cSQI�VY����jt2�����E^�$�'�fk}�!�����|(;x��O�c���@
�D#	�q[x\E)�F*Jw�Z�i1mj�KY.6fd���CnmTM	kQ������R�+R(���d�44b���t����I��y8H@����}�P�)���/��P�Uzp������y=����� ����C���Op����������� [��1n��c9+g���ZV��X�f��A�\�U�i����Y%�^�m��~|w�w]n���C�!7����5�/S�_=]�� E.l�C���\�x*E.)�&�����H���b�Nu��#
�dJ�����������D�.��^Ii��s���yL����E\��h2�-��Pl����{�9J������L>0; {O�vl�Mm�������}���&�5OE/�������F�\�Z=Y^X����e��U���ox�S(j.��G��m_�q���<�(�2�U��sM�(��CKF�n�(��"Que���r.#�����t�`��%��@U�6O�s�M�k�<�t
��:f7�_�t�$�:�%������)�����q�$�5R���7�m�w�<8@�F�7�?�UN/\m_���({�p��*�p�5�z����L�<�d������Z~�����A����-T�{E�`��r�����}Y ���?g;����[�I_ko�XSgx���5I���U�����,��F�����ZB�L����.�2�H"
��<���t��	.���)./����\�s��Lt� �N~�R���DR2y��L�H�T%%y���R2
6��r��g4��/��dx�'~8���:�z�&��a�4�U�u��v�y���8�~��'�	�k���	IL�����W�+x�����:����WU!#�����r��L���y������)���������C���|Y���y��r"��w�����'�)�S�6��?���k�3��|`���+����3��'�oRt��3�}}�-I�D�;�uY����������5M�����c]��0_Q�Z�*�������3BQ.U��9zI��e�,y�:��A�$�y��|�j�u���5!��9*&�B.�v��������c���}UTSt��������#�1���1#��F�&��M���=�&�����j���*=�H�A3j�����J���^���Y��	�eu}�O��X��
�#��%����>��5��2��]z���2;�TVr���f5g]7p�����������-��a�������p��-Q@S�nF���M|Z�� ����p��S4��P������m��H�_�>���(
)7��_��J��]��R��{�$1�n�h������U���H4��["��BUVV>�R�h��e�y8�?�[V����x�M��X�m�5���bF%���U��a�*Y�jT,����2`��;RqDB8�d��]W"��iU\v&�>_/�"�"���#?�M�S�U��/V��be��`mN�W��BZ�	d�u��\���jh��Q��V��^�&���xo�j�y!�"C
�Zm����V������s���P���C�o���iJQ��f	�4RSC��{4%f�~[��:NZrk�1������i��u�������X&|�5oJ�\����!��;5�![�+��[!���X�6!�{�����L������c*�*�k*P����{�9���^�Tow;�+g[�H������7��,��~�n������
~�zp��y����"xf��0������|��_6�M��.���+Ue0� C��$6Z������ ��CJ��p"U�CW��`�c
j�>�~������j$���R]?�u58���MU��r���=������6g��C���F���z���K8G ���[�r��X$V��x�*�N��^J�wl���ph�������#S�9^
�\��q��|�q��������%�V����E�&�9(��r�����B�V=��v��NM=��O�������+;�>�0��0>�B��2#|�yG0w�g���D(��&h=��gF7����c��Mh�:�/�G:��o�o�O��T�S��&��gN9�e�����bUB�O���mP"���t$�9:?�k�j=�[\I:�l��X�6�Fu�S���]EHs���*G;|��*�z�e�@�(������+����[��9�h,��4�������f��@��h>�9�K�F�~L�v
~�Kx�������)��N�0zyY���
�V�����,��P������<
�� t�Q�1�A�P����8,@�I:�Y`���g�����E��4�jQ������ f_:��rEMH��u���2=��a����?��C�!S�N�LP@l��;!~�Z#{�����p��C�o��e"���2�}�<�\[��8Y���b��	����6v;�`g(���)O��)�"03 OJ����(��?%�*������������
r��}r�#�r #�L-��H�^�����+����tw�������w9Z(gT&�*B�4�o����/���mQ��f���v����M1U7�l8%��(Ra�mQ?'���s��$�*K	�l~���^~�y�T8X3��U����VJ
���l4%��8]���/��bI��D�;I%��W3<LEG��kRn���k\F�R_��
m�b\J5��e�������t��x]w(�KO#����.&+�d�l�1�2r\�_)�����.��'�j��<����(Z��(����+o�������69vf����0��H	�D���B�����#c����Es��U)���8�I��D����F�l����Y���u��PdU�eX'�����H=_��j��eY����O���V;|�_%�]_�sr1\v���i�e);���v9,��M`��
�F���*EE���"!�����+0�TJ3�!��5l�\�������I���w���@�*������cq-����2p�.����C��<��dC1zDv������"qw���F��5���d������x->�E��}��c���3�	�jG�N:_/��q��?\P����r��L���0(c���%�M�����%�]M�I�i3���FWe�4��3EJy�*���� �X��H6����Yqs����>N����r���H/���pe>]��!T.@W,W� ��h-�F�&�<���
�"LL-_Q���?:���'����9���i��z�7�X��{
3T�@�v�*kF����&=(4�r��5����}"����*����
iO��������p���Ag)=t�1��pRJ���D�%���g�]�*����T����f):E���KULC�/"�m\����@'�3���}x`:�ni��B5L2��J�#�v�����@�:(�t�S4�Y�/����Pc%�p�6}����/fM��,k�M�Ynb�&����^��4�&�4��M��CI�&\��<����V;q�T����������n���b}���
��[�3
�$Zw��
:���F�7�_��v��YL�����������DF��&[i�DZ���vL�3���-{�X�d����������R6N u������*���F���+?����Z��>��x;WN6�r.��}^�C���dX�k����m��fdo�:�=�Mv�azJ������S����xLo���
	���cz"��t4���<^O�0���M,�)�=�ko��	�-��a��J6C�S��FEc�A7�ctK�=��"�$�3v�0�^;����@+e+����������iw��B~�&0j������7�37��x�~���{o����6�]��\�b=��h�|~�	l�*Y����;v5��j����0�x;��-#+���T�F=���E��H�<���_�/��y������`���,�$H�0�r��f��*��.�m�R�����L<�	���$���2�x�c���>�\6�D�����)�;-o�m!�����cE�kd"`���~.���������]������0������A���B2 ��1�9Gp^4�����Z@���B&��VG�2=��PHK9�������p��/�����`�L/�������k���1�[��H?��4y����e�)�PI~0��)\�!b;\$s�y�������C'7������b/�M�y���HR'��R'�yc�|�b������yQ� ������=|^�Z+�|^�|� :���
/�,��=����q� �G���4�:�X�@�b�3u���>��3��X�C�b�!nUL�Y'����o,��x��Q���J���e;���
�������d��>�1=�0I��L����DN��Md�i���}|v���fD�����:Q�1����
���@�,�S���.u�	�w�&����G���i�����[��,�8H��{�����d�SF���{����5}q��?���W����e���y~��#��Ux�����p`;�G�S`�I/#p�+i��=�M����'B����?�N���e�s�9�V���#FP��Z���nc@�v�_1����[d^�B>m��=(�p�����"���������J{��|7��%�%������]R>sF��|ow"XK>pIu�r/���r��[���
y��k���J��!�h�'�@�}i���3���[&i�o0F�s5�������>��N��XptU�rVs�)B�.�����*!�vz��Us�����Z6A�d�%f�v���X"|U����_�������bJ�������R����8���&e�6�7(=�r�s�L�]�����W�c����-�d"�Ow{��a1�������������.���W�.;;����dY;,~�N��{�I����
�U��HzP�X<�r�(E�]��A�4���tq�A�����R��'6��������?���#����E����vJG�0U�Ycl��s���H��5e�����&>�N���V��Zd��a���x�����*�����V ���G�3���H�!/�����u����O����S�u^���{
k�e���&\Q
�F@(W;��B9a�?.B�I�r���O�g���l7P:��~�u�*��z��5P~��m��a+"{����)|Vw��`V%L�R���t*	�[��KS�'��4����B��� q����M���A\����0<��6��W��qO�l	�8�B���!RP5?b����)���[���_���B5�8x���<�J������D��������$��B��������>r��������������<i�Vi�^g��W�R���G����I2�'����]���d��?�7IW.t�pe�Oe/��r��Y�v������zX?3�z1��������ODD<���W�v+~\���\��m�������I���m�:����6�q��.Q�r��sp�.~��;����>��w��lL�����p�1D�s����3��-����>h�$�e��&�����=���UQ��
���+)��@��nR��${��U	��U���~:��Ug����j�F��\LlzOz�}��@��n�	)�F�+��6�	G�M�*���n�g�g��MUew�D�v+NRz2Q�Ug��m�+t-��1�&�L:[s����P0h�F7YM���u�ho������	0�4x�<z2P��`������P���Z���j���\��h{��#.��p�������'��#�����vR<	���"M��iI�|W_+�>aX(p9��yK�Jh6�A��[�����~>?����7�?��x���|���Bw��H����;1�y���j�yF�#/@�>���!#h��]����
{_Og�iG���?�w8��!83/
�]>���~����X^��������#\#�����k����d��(����i��5J��"u��V@Z���C�:v������)!�T`�Ww���5Y�H�M����B�G0��G�2���,����\#+a&���/<�BM
��S�Qe��
c��G),>.��'9�y�M%���
�A����#u"����
V_�H���_�T�f�,k��K�8*������7��9@M��O���p�}Q+S��0��FP��b�4O{2>��;�X*��0����>���G��Vg��k����F����/F�����l��.0��P�	�HlVY�-.�A��"=�&(����n"���AM>j����~{�����O���M�s��$,[j	�B���@G;M�b��8�w�b=o
�D����p��=�-�sqv��}p����T+��,�>����t�,�/��.���Kh�0l�-��z�0D�����;_
\�9ap���m�z����%��fl��x����Z,)u��#�Q�mZ�%N�`����71e��������
�E-5�@@��E�Gg����G&�����.����*	Z) �0\s����+gw��k�z��c�Pn?w��Y��pN�i6��4�S8���w����������ff%G�w,���C���#���g�Pk�����7n�����J�
����bK/��P}���lOb@G723��T��3���+������|��of ���\��gF�Vp\6N$�����Z��L�y�����"�I��X���3�`
kl���dN�8aHg��b��?)����^�����Y^���a��0**�9�BR�P�2�j�?F����j:�1�O0qG�n�#������V�!)$2U%��*�U%M��\�	�*��N,8_�$C��y�zg.�����l��HJv'�_BX~I]����s����zY'������Zn���R������r�����Qhh���V\��D�EH�?�����O����q�PS	���E��Ev}]�����
0qscz2j/�.k-�
�%�Q����K�M�4�W����HO�����!!����r*l���o�y�>��B�e�y<�r�����
n$��[��Q��}W�=m=�~C�Y�_x+�&�'����v��������c2��+���tt06�q��h�R����Uqy'�M6�BF�L82��}��X�z!�v���A,�w�])�P��NL�4b���Q�R>�����T�9�[��c�|�����r�x����n:�U���/�y�W��F8�?E�r(i���9Ee;�h���=����l�5'�Z�Vq�+t|s��B�&����
J�;��������J��
/�z�f��S�%|�Yy��=��C�������z�8
�=z�~z�0j��(o(�����%>������kt��$A���68~��O��*9��JV��k��zv��
�,��+������s,��F�����o Se/��h�������������4��4	�����
��6��AI3_�{_ZJ�}��2I)i/���/>:y{��=�~��"��XxAH-dB�NY���+md��&Q��i�d��(�����aw*��fc)���#-�	�!�N���e�^R�O)#6���Zn9���Vu��%��0-�[Al�1��a H�#8��p��6I��}��^.�l��$�\�1��y�4N�d��Y�t�a���R���(�����5��e�Di!�W���uV]��Q��������Rh�����o����'��Sn��|~3l���
�~0�����$�
/%���!����O`5�
-�U����bt��0R ��>�	K��<z[c�Ld�N�L���J[�������J��F�`D�T87;|�z �~��J�ya(n�F���f+6�:dS�
��M�U����cdy�����!X�v���O�X��Oz}�v]U���8"}:��>u�'� YgT��I���w�p�B�VP��D��Ds8z=�}�����
XK,�W����Bn����h��PSJ��&n3�b��7��'rg,HGVO���
�r�w����^�1f��Wb�wK#{�b��s�>�l�����o�!m��c���&Az��x�ik6f��F5���
��ih�'t�`�R�����J�SE-i��}w�.��Oz����Ta���������k��=&�<���6���O����$<��/{l�O�,��P<������b�����0
���@�N1�_��ha]�C���������
��q�Yl�'�R'f*tb���0�4����b,�p���'���WwV#gJo�p���W`���yo��p��S�.��g�hD�t����������Q/�FLyD��H5DS��g��]K��Gx	����*0�(|�r���0����p��Z4Yy��	��kk��0�`��]���2�h�*���6_�R�&M�i����C;�����$8����R���>�G�@4Lr�%G'��(%����C�J�hw���Aq�[P��E�xc�����4��^�.�/O��,�D�
"<�1D�G�^�)�S�
K�.x���l�]����=B��#F0�*�G"�����+�@���n
c�B�`1s������a�(bn����c��s�����Ra�yy]L���������IZ�9h(R�<(Oe#�<����SD���p�s�r���
�B�27 �dm�"���(gy��/�������&����e�s���$gp��zN������c�l"�
�P��Vc����2t�v`[�q�d�|��}~�������v��S<�������~y�d���$#��$����J^��n���|Z\!�V���o������-3�#Q����DSNSw����O@!��V�Bh{��#@��h���e���^��^/a�|���:���>J���*�B+{�Li�W:�x�L�|�0�r�~c�v����M���K�=:c�NY0�S/&�O�B^�����M�L�����|�;E�Ou�@��vc���8�"��;:���YmJ��,�=X0:��Z��>X��4�PZ�D����YMi
U>[O9u,���t�ci� 
�<i�C����j�Yx~@}	-�����R�nDq3C����2?%>��=*r���G�IT����FIq���4B�����{�:���TB�g��kb(+����=����?�;���3�CM~������dsFW���x����fd�U�����Y�3�k<_F��ql�7�H�����Q,C(�bOl�b��am�P�a,��3���u4��s��M1������1/U�4����]���fK�����#2f�w�W}�l�S�^��rT�CU_	mU��:�P��5��F�}B��(fs�W0E+����h���d��M�U]�i
��Ta�p�R��l�O�yV������\^|���]��~	����u��lr���.8C����C���YU��}��U6m����!+%d�;3�t�^��s�7�y�T`���K�����7�,���"*��������"W�%�C���{����@K��(�B��F�A��[S�:��mv'����;1Q���A�u@�Xv�Ip�����y����g�O~�"�i���������������/C/2 E����{����l��������K�����j������_��
���d[���^����:�������dpk��[S�e�4�x�* ����l=ov�S�e�=��C��N�d@qPr ��-+������ZO|[cw��R�?�-�x��|��.q]o�{3�t��b�&����<���RX{Te\��4~'\�4~\�y4r\�����Pm��(�q�:{]�������Z���,�!ve
k�W#�#bo�3��9.�!�T��7F��fx���CC((��J�b��qtj��3��?�����P���qc�
!��b��A$�)��2����'m�{��L4��0c���F��z$�e�����sN��Y��Y����Y��(,C�j�7�7��!��[0��o����,K�p�l4����h��n����5F���g���mh0IEC�t��/�U�^��6�Re�3a7K4g�\\X�
Z0�ec�'��c]T0,5ykYG�+B�bk���l���4_!�:7�-��J&m�]����
u�J�q��~.��T��_V5?&��h�exw����#�%���u}C>%�n��Z/�Z�3��x�U�N���1	�F5�S���Nt9�-�b'{ksF��dCR�,`��8�^������<�{b�mO��$#�"���x�M�:������e��23���I	����A��e�11t�K�Q����@3Pf�?��)�����_P����"����>����=q0�Fn&S*Sw�$H�I����L,�����l���!6��'1���Wk���J5�.��0p�;��k"��x�~�
V��`n�����]����A�_bne�4���\r�p1+�Kna�P���}���?:�.MOpy��4#�7���m�V��8D/{�M�:����EG<\�]��e}����Y��KH+���p�h��",@����(�&0�V|h�	o=��rlwitN���qk�|��{u\{�e��1.#��FFS�H�Shm�(��61��G�Pl���[d7��k�b�[1v�b�����e#�����+v����"RB!�UC�fZIH�C�m�5�`�6~w�O��� ��0����a{�P�@�_=*F�xT��������	G����@�Jk�"���74�9I`^y���9R@%B�CK�s�~��0�uaKhl�#z.����S����F�\�m	K,t�M�%���!S�.b��"��8%�::.����s�>`P^�����k�e�
���-4PT�e���^.s�'��5E�����%S��w�;-Y�Uq],���G�$}��Vo������|^Qr��{��g���3��8�W %���\H�Y�J�fc�)s�\5�@L�&_nu��2�yh����P�$��r�c�n������[^����E$���E�Jy���A�=G,Z����Ny;�� &,%(����L���U_X��'TX&{�.�Q(��N+�B��nV����8�eS�qop�_N�����@��B����]+�S�3c����=eG��~UL�����v��'D�o+��(�����")���iY��/���/s�&���HE�[�;�76���V�Ve]����Bddj~�����P�&�9KU'�s!2�R��Vr�3R�(����������0!w�}n�$��,��^��tC\�#q�@*u����o�TU'�!G���;����
����8��� ����0 *q�
t�P�. 
9WwO���:vY��7�W������ �������&�j/B���uL��F��pc�uN���mrT�3�@�p�9(~#h�K��'zy	��f g�-���7���Lu�q*�d������d�*����j�.j�]c����Ns�C(#��\P�
cn�����V�j�iH1��*o���<��,�fi06�.;�x
���|�]4m?xF����<��g{���SgT�=���|1]�j��f$Pg���+MT�D2�!������ �V9�,����(�V���:��l�Rp��fo��E�5�������l(u0(���$z"X1G��UI� -\(s���X6�:J���&	�.'��s+����)��w��ey��H��X^��c�w�Z.���h��N/^�xL���q��r��,I|	
6�7��ck]4�-]��i�L?
�%;�~<�HRM��8�����������
^���BEs�%�q���.T=��I�=?��������?�	!�d��'e�*�=�#�H����O8J��sZ�`)H�#�FR<k[�	8*2���/�B��O���� c�`%�_��Pj�}7��������~�8=;��pj�O����!^
���!TW��s�
�����9X3�2����5��b;�94"�[�=���n��n�Y6/;(C�G��D�!g��j��=��~���?�����;��� ��������G���`u��P*s�aB����l��@��/.T	W%�,�D��UI����z��S�q��Ir2�����";R&[*V"��x0M�����MI�:v2QI���(��u�O4��#i~o(���,�1�����R��)*kQk�jn
��(����B&�	kHM�����#Q4b<�%;����7�gd��*w���_Le+����QxV6�1��
X��)��d�%5���=/�",����1��Q�v
ws��R�f�������	��u�b��� ^e��i��������*�Nq2�[���"��L'q||�8xP�['��[	�=0�,� ����$���u�8�>y\�n$�?<�+�P���S!�����c�q�dQ�5�9���[th��.~lS[���I"�"H��Nx#R�
J��7Jq<'kwer����������T�����W�����_������vz��N��,K7�����:W��&���*lm$�J����]���F I}m!t"��PQ_��u�&���%�i0��6Mu���3��B
��T�'^\���.��rF����`���Vy��85���E|�lM��K����y�S��&�T��z?�8���$y���?�
l���d�
��H}OMAY	_@{k���a�	�=B�r�/��&���s���t��P��v5�-�F���R]����M)�2�
>�}���+������eK���-��C�~$c{*���=�I{,��H����+<��T������j6,��P��+@�2���Vl"\"q&f:P�g��biH4�(�v_�����Rq�c�`�
6�Sh�1�G�L=�4�~E6�]�d4�zzce���5�m�"��#*��L
L=i1i��n�"��������G�������hD�b0��l<�m�r�Z���S���K@�{��i����{e�h�\���1�P�\5�#��� ���;���*�����d���N���'.8V�8QC����~P`%(�82�,OPr������+������d��qwn
���jn���@>���L^QCt�/D��R��H���rUf�E�@����!��I}��*���� �S����TG4�py8'Q�E1��O��x��DfBS�U�t�����Mv�`�0/?�o�Y��h+���]���n�^q������1���fa'h�=i�|-)��@�#�
�p�D�Q���?���>.�4
y,rCPNJ/�Kg.����N=/���5F���t�b0��l�L]�Ao=w�XK����|^NX�-������~������/{���;���������_��'_��G�2����w�AxF9���:a�G�����zI@CQ+3�X��a�:b��n�P9-��Kd�NK%�R�H���M9�/��V����O��!s���J��o������dU�
�;3�q�o�G��(\������GV:�Y^�	��vJ� ����Qr�F�5�vx�+j-�t��m`|&Dn�s�>Jz���~��4��f������=�������.nk�h��gb��Z-)�"E���#$�"��l���m�Af|4P��3�%�Z��19-��������	"^���$X �0�s��.t��t<^�]=.�j�6 #s�{!*�t���+��mIy����,I�!"=��z��t����#}�2l��Y;t`��(sI(������s1��;�l�d������B^:��6�����&���6���tb��)��1A6�(&{�������l8*��`=���2�
�J0�h��U�%���r���-��`~����Q�A\Eg��@)���!"�\�=���}�d��/��.�u2���2zNx�#�����)���F%�����O���L�8T�[85v�P�v2h+���\k���CL[�b�U|����8�Er
�OI��Q��pU���(���[�����}�����r��G��q,;�JfS��>N�=�c��!u�n�����M�C����z	:+\|s���M&��n���-���k<�����k"?/<~w�Y�(��k<�$�G1�[z��0���Pr����c<�����-��sK�4��	��
���,�	1��Ft�Q�F���[���3���d��aE���;�������� ���h��1�
�y���Q���h�y��5������{���
b<y�,�����[*I��=���Vj��f�K������x�w�i 1]�YA���	<�C_uM���}�=��n�'��6%�����V 9���C�{���$���VxPT�
�y��0<�E��n����q����|��a)�E��I�9N1g������v��!����F��.���%�����NRrcE�=?9r�
�<T	�]��AM3�Pg���`PFAL�|Y�+�6�0h�/���2<���O�.�Y8,��H���7	'��kj�;~�����xg��(Q��V�D��A\�2��6���b�n���4��4������7MS�Q��'{�kz�{�8�]������������t
���A��zS��]oc��KR�@�7������D�C! 7��KO�J}��7����������������GC���3��Ru���b�4��QG!5)�(�1�38e����jGL6Ra<}vw�'i��9@�z'#��!BR�g-��H�V����J��N�D����
xq(@��zM�Qd�Q
l<�������;�2O�3�E%�rbY{xP����u�$���������4n���S"���'�a�	gB�_�
cL�.8�
H4�B�)$����D�&��}�rrL&;v��j��*�2�V5.��U~��"���IC��H�R�u���E��,XxF
d:�|`=�/�D���W�I
"aC`[��|A���T�T���iW�m�2M��W���8�aWU>m��|U�����8���^��k2��i"�,S�f	���n�t�f���8o���e����o�G{
��OT��A��}g�L2��L�a��P��$��%W��G:�`)�������b>�;���%�$��dCR�D)�#y(Kgp.�e&�05m�s��'�RA=Kd�q�E���9����dv�#��X.�����B����	7_��s�;�TA�e�����5��Z�M�XR��mgU	+9��>D���x�]�!t1�����U���
o�����X��=��C��H�f�f'h1���������;v��'�����#��DyU�ha��D�K%K_�
o9���8>��"���\��/�E.�1������ 2����
�����>���!������
�/W�1�L�B���s=!�2<��T�;�',=��{1�o1H����6Q�3r�Lk4p�P�+]�3L��dO��J���V�]O�O�v��j��t����S��1���7��b�@-8F��2�����b��b=o
��f���G
��r�}+13���*[d
�?nP�N������2XE�:e�P�����\;��(��w�v���2REv}������6E*�Me���������~$S�]�_2�I���R��\j���}E4B�
|L�������Oe�?7>�s���{i(��M>�j�JV��g�l8�o��q�$��lI�UCW}@i�����W�u~�q�R&J��Sg�R����@�����<�����~
�0��2����$
QS��?����{{�+�~N\���dZ[8W��K����d�P��ueD���%�F�.��o���Z��4�7����V��|@0�qnR:�gkD����H��������UL���
�j8�X�knO��b�95����xP��������T�\��k��n�v�������	�8�����u����Zm
"�a���j��i���+k�3��������7['zk|��Y�u�	������0Z����)��E�>=;�� �&�Kv��~.����������1������W�iD�H��rG���:�J���s����:F���.<�*&P�����9?�������/��ipk��F�!�1.K��E��(��L(RU<�W�<�����6�50����a�����q�m�������	-����j%`Hun�y���$`];v;v�����-�hQO����=*A������8�d��@������x���x�����_��>�{��s�������"
���y������;y���l���U'UW�Tr�o;�;6[���z����hM���J��z��bH�^��S�����,�_^4`��Y������a�h�l@#��&P��'�8�L��3�3�����%�e�����3{i�&�l]�J ��N����Y�d}����/{����l��L�U7�N1?E�l�u�&�$r�VR�����R��	Cu�l�@��'�T�D��aq2�u���F�lP~%�us�F�J�g:���1v�g����m��PiM��.�������t�u���e�V��os
�q�<����M�q�:�5�����~�$�B�^��;F��.���c�����
"{U��\g��J������GX���=|B
�����Mo��|x�+;[w���5��>���4�xX���D��{������/�.K��BbqK�����KZb�\�B���Nj�a��!��N���'�y�W`��
~�����a~����2w�����s��#��o�>���vn��
���������h�''c{�%A�����5���oy�~~��L}���\6�R����S
&moDD-h�9nA��G��+�}kU���0�UW�+V4��Wi��kG�m�%\�A�G!u|4�x�wV`��l�����q�~��V��5��3l������<���2�6�y�[��[����2������������O��a�20\h/�4�4��9P��lj_�26��\�Ed�E	��|6�����[d��>�o��}�aljV��tI��I8���i ��x�,���X�Om2%�tUd'�qn
ak���A3&�L:�����W�����%cj�Uq���kl��z�������j�i���hW��`)�Q���|��aF1'���^(��Pq��	5��D<9�e�������������C���e���4�!�����EO�z�`��.
��������5SF�	!xX\��;��bu�
U����hq��Y��J����*�����E�r}	2�Ft�T����� ����`�2��n[���nA8,v0��dw�.08��O��,�;����(�3��y����A
�����]�T�6�f�X����K��p���=8�J��8eBr�J��+�����2^��>1��
r���x�P��';9eH��)�G���=��	oluN��/������>4]�����[x��0@����}�B��?�;�8z��a`��}$� �x��g�O|�	�1Q��M'�C�7��'b�d�@���*S!��qI���_�:��G�u��{�����&�o1w�m!��
�/^j{����VI��
jw5$+�}��V�0�9w�p����	S	�	|w&�'�3a�"�`B�E�)��=�����v3�h�����'C�;������9�U�bZ{���I������i����`j��4�H��!��Twm1UPNK,8��.�p�QT�tO���[N�\����P�a,W9��1ti�H��!w;�Ihs�kc��h.��=Gp����A���h���������ms\�9WX�����?>��m0���T�#��������w��t���{^��\J���G�a|�Hz�xC��un�'#��(�bQ.��pX��bY�u�G�f=�h�� $����j����c��es�/�����.�������E�����������6�x>''%nOm%�s��(���U�&z�=�)�3��;�����x�.L�lM�h���������u���������C�=/�4��{��y���@��<�K�Ry������V3�	%���o�\4(w�2.��3�o�J/�9���P���~B���4+�<.�T>��*+�r�-�E-o�o���������Hg
���d���U��3��3�;9K�R�2�)��g��(/�Mr��}�:_b/:v�`��&_�]���m�(DY�>C��y8k�~Ik<|f�`��h�f����mi5���q��/�
Ji��[k�T"�N������w0:25���:n�^�'l��6�4��������y�y�T:����FQ2�N����r��1rN]��KZ����U��.��
�����U���I�:�����/e�b(M
�CN
GN}v���xgqj??� ��k�1�E�p"��>R']oM���K���"|F8L�_��0a���g`��
v����9���a�������N��d��$}nXQ�gM3�)u���eC�I���8<4X�����f�v�E�<��Y&���H��G'T����$<��3]�����U����)L����){1T/�����NH+R�z`���c3��Y�~���4}#v�{�4h?���o��������^�rw7�0��@H�o	�g!�hK$x@��!�n>k;:V���d?����E����D~��������{>�����
G�'�������}���d��|t���Ov�tB�dB^'�V�w~��q���y�@�S�����~2�`FKE��1	�~72g��������El��O�{��$�z��w���y���|����5��effa���L�.X������=���T��GM7|��=~�W0�\��s�w���s��FVO���O�	&���w�����L�O��CD���
!1x��O�H��c����L��<aL%�<0?��R�T�GH�	0B���A�P$p����`�ogR����Qh��l�����PjC;a����
���@��C��<����I����r��x�.Q�Q�.e�����8],��������j����8R#������s����
�1mR{3���BC��uO����k]
�~B��9jw_��Q!�w���#C�b�����o��!����a<�#�����'y��^��X?G#��m�/����t�
�����
�#oE6�NM�q0��c0��L	N�����$�I�R��P,?d���m�P={z���
-lU7�����6]__�#0��:�3!��b�FM��Xi�5R�F���{��]�/��O3I?�JW��y �Z���$�$W�8VQ�\j@QN���B)�w��e��O&�f��E3�{�WoSz<��J 
�������I�;'a?��A�Pxo���`��+�D�D� a��T� t`W1z2~�T��Z�x�DA���n��(E��~�DA$���EkA�s����N;����uX]wd�8������@�~9}V;�e��d`o�Q����9@�����o��[����8����&��=��9e��:�7i
��8pK�Z=��>�0�h@����Wy*Jq���NU��\i�!,t��/�-Ad����t�'n)����a��C(��A��
aL�>��l8�0X�z��!���<[9�z���]!,�EI.��sL#����q5E����)y�@�o���Bx����i!���7n��2�^�h~�>X��uqd�m#����_4���XrK[��)���3?"��������B�J�bi�A36]�R8����;wDF��38vt����������X�,j��=L��w_��D#2���W)B6@�Xa��i�K��������Z�KE,�����K��G���i��,Ih�A����0E
$��!c>��[]��!C>�"����������Cnk�3�.cW��G�]�2W	!�)U�A��`�d�w\<�I�5���U����a.�lD:d�:�������b$���u�3%|D��
	�~�M��t�r����������<71il@X�j7���3o�p�>��s�����+�6�������T��L�	�����Tr�$mk`pt���w��ml�3r�#$�FtFg�i,KP0����LaOe^�ke�x�:���y���e9PUNp�xQ�=����VS
�r�98��@9G�G�+8�� ������u��n[������[!� QL�E�����*9�w������a����w/�.�#����1�������P��X11��j�}i�J"h#
��G�L�	1�)�����HvC�J�$��k;��$J[f�nX���p��#��:��IRZ�K��33��E��*�1��	�I6`����!����5�Cu�Z��a��.w�\�a�ZW�(Xe43���(���Tr'M����)��]>����zP���~u2�g�-�O���
��M�W6���q��L��3Y�j���m+�-bl��^���v��L� c������c�
������CMzn��EM��2,�kCW��8���
Mx�qFxCFu(qM��7ig8�6M�������C�*�
.]]Uy~PHz��C���|��Y����fv��,��e�a����#�g���o'����;�tr�����9���������%�%��H}�_4��~��J%����A}�^N��\�v���E?d�WN�a��S��Z�y��=�(���z���Q&�}��4�;�w@�q{]
E*���XbP&G~ TyM��7U4Z�,�����g��\��`���B"��l q�0���~����awJx�=��-f�����73��BX��BX
��,eLd�q������Tj��@��k��}7�T�df�Z��n�Jaq+���F������`*�F�Y��Z�e?�{������tct�g���0:U�@^�G�gr�������n?������R4`�zci���\r�	���W�����Q���8`}*��V�[��W���]�k�����X�����JQ�F��^���Z6{�kP������08|[�B����/'�z�9f��P&��+����{VPv�0l��T	EC
��1>Ef:����WTC��1Vz�yH���2��0d�e���$�(���Q�Mb���}�EiKgu}�	l�Q�&��N[D@�������a�K[uT��g@�AO�2ec%<�<h���S�����jh�B5!�Dr���Y��������O���t��:n��MV��
��_���]�)���(�2�<��$ Z�:@�]���,=ri4jo>��g� N��C
_6'3xv���C�]����K���*{$T��D�+����x68h�B>����_(*���n�k��sFE{H�����7��x���/��0�
,Y�������W9���[�+]����6]�R#s�������������+,\1�~��nBvu�O�����g���M�8��h������d����
�
X~�O���R��lH���=~O,���dP3v�C#�lz|?���"��L�%��B�Q^��a!w��%}d$���eY�� �����9�mEs'�=_��c5�2 ~i���2��5���1X���9�1����B���l���;%�HIlr�9�*5^��j/��z�?����P���)D��/>�������#��yyu���������X�������`)#�����m!W�\���;j��l�W�����C�������B���g�S����.=F
�P����,�>L4]3V�n3��6������B|~c��8I��~;��Q~���������P.n����:`�����T�h��Q�#���/o��h�"��pu���d��z�U��p��Z S����C4�t\/�]�D.��W����c�,��<S�M@(�9�#K��IL�~D����hP��c������u^5?�_&,cn�����B���^W���.5�.[&9�m��4�����o��$O���Q�E
��l*r������Iti��b{��s��������X{T�D���sMU�:��jX^��/$�_9o?��������/�%[%
j���X2C.bxO��\/1Sx�sJ���v�����G����vN�������sr\����X��9���|v���{G�.`SO��G�:�����+�
�'�s~��}�����������,o%M;������� F����.��k ��na�`�����C���L�J�,	d�6��m���P�8"}�)T+>���b��uP������qO�����a��>�o�j��Sv0e��U@�������q;�����=�<LE&-������
��Qb�sR|�������u;��tt�^�HK3	�|F�O��K��;�rrq~������g�����"�d��]-p��&�g�mNy�.@�=�A�����s�{��#(��s8��}�~�����/�}z������������2"1.wZ�
�0��[��J\I3�p�J�x+h�&�����B3��1D���'Dj%0�u���T�Z�P���)������M5`]�iP	� Da@�Q�F��"
�@.��
�I��7�}�Q>�����[K�D4�cvIF�&DQ��]�$��Ad�����
�B�����@�p9#]�&���V������DcE�����,��0S^�"1-(������yG6�����<f����K�#��o�:w,@��XFE�����@�N-�������R
�Z����?-sJ����T�=����U9�j��a�wn���De��E��cB?j�AO�z'�j+E�������c��ESH��s��'���
{A������������*)��\��_��M���n��0�R(c�����c_&���j����������/����C�J����$�"l5~��d���������:�D���2�?���=�8�������/o�|�2�JM��w�"���o�����q��[VA�a�p�-N����*�:������DDP�Pm|�~������`��|��8u5��2�~���r����y�����K�'�:�y�6&���u5��-�7�������TTh@�������L��E�����q$}�SU^c��cW��:��c��
)I�?Tk<z��5���sw����:o��&��}�#�!^_��Xf����%F�x!����Q	�~�U�����������M��P�2�R�DX�@d����\WYscJ�u�� ����l�D�P3��Q":���
����/�5p����1�F�]�?��R������`KaC���i�4Oa3�"x�juT�h,�pT[N@��2j�G���=[^`&H�E��"H{tVMT��93bEH����XV
�O��u���G#j��o�R�9
gQr�����ow���"��G�h�f���j� 1��GZ����#0��P
��K��sb��S���L���<���C��.#0�^[:��~����D��,�b	�����>��?y������X���9��2�tH�-��y��UY!����}��S��3�����Cd������T��G�������ruWX�w0}d������!<Q7�U��?�;*���E�����������/8t��k���o��^��O�fp4�jY��lO|�y��x�j8N���}9���-�Me/��I�L�H�/%���/+�q.JC7���],��5���+�����S���s��u� �g����e��*��9��]�	�f���c�-���b�7Y1�/J�I�������$"����yy��q������\_(9]_\�I%�I��&�as�����lQ�gL��)��'������
����hx�V�L���I�[I�*@=��d�{���&�N�E6bs�{���(�o��YL"�J���4:C������i���,�N��C���������Z1
o��L������|>���*f�9Q":��Q������S�	��D������B���bn��y����SD2���;9~�����|���I�=��>�k���x���;������|�0�"V���={Py���W��8iY>�[�N��sL6����� �����}�C�nd�ELL��3��+���b  �>����u��I$���\SB��l� �@��4�������Y]��r��R�/f��
���=Y!�!���;�:$���T���RdA��J�� V���������]q|��-��w[��w��Z����S�88�I�L�[^��M1��|q�kf������+SH��-H�������E�D�e?4��������<zYSK�1��C�ln�t�����Lo��kO�^��j\����-��	��D�f�@C�R�v��3�������������I7 C�z�x�f�O��}@?#zh�M����Z�@��>(�'��o��A<����B�7�(h]�?�w~�k���G\�I�=����{���@�6b��u��X�����Ka��r'�y�?��v��D�����j��RvY��*��Q%IU��y[1T6�p����/����B:�`�d�m�teh|L�A��tV��QK�?�	�!`5t��lE8�	��ft	��%�"���)���������g^N�.���@�>��6Z��d�7i)�-��t�""�<���MLy�}t[���m����>�.�-:��M4��2(%�������/�2H����AoA��Z����
�������_4U����I<0KP����`��"��v$����oa
���I�����P��r�S�oE��V�b�%���bU��V��������9k<Bu��5_���I�����
���9=�����fS8s�8��Z�7�~v�~��:$��x�_���}B�l����&�^���%W�\�u��f8r5EH.���j�Ke�/��p�����v��`��8����t��[~.eEP�F+����~:=�rr������N�x{����\
����]�H$�����[�<��*���4��L����7�����fy�H����|<wr���U�jNQ��?9z]�U���i�0"��%dE�N�5����J3b����_�y�5�|z����N��K����,��������,������S�.W
�)U^J�	r���h������W�����(LE=X���`f���L���D�o�6��l��*��L�S�%9	4WkP�Y6�-����fA��wl-������w��-cS��� �N�a��h���@8���1 k5��[�{��Vs��CGA[�;�!U����^����bj*����!��^5�:�DF��la3��u����Mkn������1�f|U�w��}'`���w��n�0X�"��O/,<��R�"Dm���<�z����bB�Oy�����y��ASL������]H/�������F��j?���Ck��"�O����v�iy}Z3X�b9mFQS�1Rh��0��h��n0H-��R�u����5����{�J�q��S�}1p-f�bM�����A�+Y\����k��n��-hd����B��De��%����� ����`� s�F=��0�X�$�bdW��1w<���|_G�sj�
�XXz��
 {rw��z�=�\q�dCr�l��]�3n)�9���gP�?:���L ��d�\�;q ��2�Cb����j�v��!b$�����t�R���?XO���@}jB[���Y�h��[��9M��cs!=`hKs0j�fo6 {����E��z��)��p��h��O�f�!�@4�V�w�� �
�%��7�B����nu���r]�����������y���l�w�,J4Dd���|��:���j���y@��Y�d�����_��Zdw�����#�R^��������Bs$���#�F�D����������h����������{m�\ B�q�T��
5
�����r��W`����g��Ym����
���v$�K�����>��@y�#����Z�����r��20�N(�.SIo��,���`�tI��N�v0�B��S�Y�r�32�9�v�S��i��2�a=�<�I��"��E�#�'&3y1�:���n���)n�	���F� {(DB����4�:2U�-��x�C�Gc����aN	f<	�����s��H
W�*�sS.�9��|�T��S���X�&LA�_aw^�"fPM�	���V�&����RP�cpy=��^��'Y�����������>8j��(�Hpq8_P"ou;�9uU!ao�Ir=���	�J����R{u��^{����5m�G9+��D��m2G�����gD��������>P��M����c�m�u�a�P&=\���y��^��`�P��{T��m���+{z=�J����.o�?�~��Z��l��
n�����^aZ�Fe����>�f3����sH��[���j��~�� N�?�|��[F$a[o�����J�J?"�{}Z���LZ'Q��)��;/w�DPj�����]��}y��j��*��5�-"�����p�XO�}V�Q����(?����*s�z����8����"�exP3������������@�`.�&l��RP{G8���=�*�BW�'��Z_h��>J����uF���JO�eMF��lO
�^��Q��q�0^�1�.�7*e5eG+.hF_�N�<�l(�9�G�~���(�23������Y1ed�o��_��2��	��v(�?vG��A����{!��y�	*D�F�����������B)�u�j�L���Zv :fv��q��FO�����<�]k�f6��Yu{�����&
������u�a���/�����#
��ql��{�c<����h���4��=���Xk��KA��
Y
��r�-�6���9P6$�M�`�[�
ir����2���2�R�f�6q��tD��u15D
R��5Z+x�D���t.��kn�F;�-�lL:�(�[�55�[�ohf����������YE����tV�_Kw��b��}N �%,r�1g0�jI�(X�Yq��e��i����<�xM��C+��c����#D~�+�(��l<:�i���BS����9w�������Rha+S�Y ���"�[���%�;!�����>��~	���<��ul��gU�>B��g��Y��Y�[��d$�_�{��x}j_;� �hX�:0���Q�i'T5�
�"D4��������$�'#�cOz��M���boa�I�2���"6-���s��KXN�Q��=0�r��b�&��g�xT::>c��j�6eG��H�lR��5���f[G�=a���M�%G%�|Mv�.2;y��]��w�Fw9����B
��%����H	"��J���b:2�d���\�j�
��f}Ax\�/�c�+^K\�z�����-�����'�`����?/U�2(����Z{�Q�$H@��X�>�����~ij��rZ�I)�R!o����/:B�.A����W(����~M��s�����N3����_�.���]��+�j���*�e�o��u�F��B({
7L�����X�[9mz����-���B42���7e��G��sU ��GV;T�������D0;!A$>|
2
�����6>k�(�������>89?�x~�<]/�?W��L���@��e��f���|Y7xN��u����P���������~7C:o����26s��d��n�%��l�^"��Q%����y�i8���I���+\c��`�&[�d��
@���b�h[���/�R��B���}W&�}W�����Az)E��T��(���R�1�R^�|J!�EI�T���X�����,�r�Id����Yp�J(�o���S��Q��$���
��|z���L����|+�D�6*b.
����\D��M�;��.$�se�Ie�B���p����b�������pt�3��i����.{64�9;�.�Vj�Aw��:/�������#C�Z��y�{�u�F%�^���j��F{��OK�(���)o����)�c�]�R�|O�w�����d�������A������+L	w:�MY�y����|/q}+���a�K��>#{I}#����U�@�cE�BlV�!�}-��k0���2�wU�E��NQ������LEdC���cp4����rX��\z6�H��!�d�a�`�����u6�w3�~����D��a���QKs�P�����.���!);o|?
����6r��b�H�����9]8M�q@��3�m;u��X����$�&Y���r?����B����y��[�T�a5�����>b7��<~8���PYp':��"9��K�T�Q�0��i��9��(�xl��zc��������&�J���C]RCd�2�HJ��+�6��+��T��	�Y�����M$����$T����r��`�9��c,8������4���Qi�q.V;7�p��
�rb[���kJ��b�V��C2��I����d�9����|�f�u�q��`J��u��%�=�����Fm���F()s4Tr�������"���)!�8���'����.e~�H9i�}��#6e�eM�\^��Q��pH/�~�9�F1W�a���2��%�����xj�T����<�U.�{qlU��@�m�����mV�E�0���Jp��(��n�/�9M�
FX�}�o%<�bT}����:,�)��42�G��}�%G$�@���_�rz��%���x������`�%�+w&��T�1������Y���\_/�Uh�V�����'��V����}F���"Zv���v��4n�$��p���C�@�(�kk���l�[?������F��>�b���(�����p�1�9JD�����x�����	!`!����������zQ��C�\
���
���m���X����W��Od�^>*A�V
�0e�B�:%H����}�����DJ�b�.�+�\����6{�
����*�yT����&������'i���t?�t�	��S8R �����_�z��� �o	��Cb����G/���Lc���*
��e�Zi���9W��]�T�����02����l�n��liI��>��u������"��l8��^*�>���u��������<�`%�*+������t�
a���9w�D,�3����#���J	&�	�q%`G�e[O!q��l���K���6�����;����Y����3�0���+e��$�YH�P!:l$�p��Q��1�P�cG�`&qf����5��
�@G��y���.�I���2�?M�]I��Xh0��'!�I����� �/L��A'Y�i�����|p'�!��m����X$;!M�N�Q�Az��G�PnW��pn7e"|���"�y	�����$��M;�z�iS8�)r7���u"@l	1���}�������w�s[F�k������"SX�����B���[�*E�_��FiFH�
�(�|��b��t�*�������L�Np����g�T��@t�e�9_�k6�����}!sv�^���n�}B���<_T���D6U}�m��h2M�����&��|l��]nqKd$Si�FL�h0�G��c�I�Y���N{OA�wHt���� ��^����X���w�	�s��#�p�ym>���z��^j�^�x#p�����o���d��Nr~�����e3O��]s����PAH�Cpd�
e8����2:)+�b#(��P9�������\LF�\L���"Lg�����p����f������X�����c����|���[�Ov��6�S{Q����*��1���%f%�]p��g�u�h��L$����S
�(�D�X#v%ttT�
ao-��=6UQ�,��/9ZC.1��&^f�EPD�?s��F�Q��(�#|rA���#E��|��5���&���&���}��(�)�h�[����~�e�p�T;���,�0(�e�rP��Y���a4�
��c�m���� ��LQ��D|U��++"�<*_Ya*g4_��C�*�����h�JF����t���� ��`A�~N.���Fuo�~�oK����h�9�B�b�'���!E��tX�����q��sD)Q��lJ�/���lA�X�#�����$�`3�h��{���B�@��(��+���b��C]�9g$��J�F�)c#_YZa6���&�Y
�M��Pt�h�2��z�bf�-Ld���7�����'��t�n��.�b*P������%����2�G��p"K'��w��8�������(����KQS�TT������k�$��~0>NH&�oR�� ��)���w�~|wr����/���\�}�r�����_N���Y���9@tQ_<���F�����y*����gy����o@�T9�P�^��O��|�D&�BW��*X��j/�o��(�w)�U�P6����
���+X�5?��nA�6t5����%�+�����y������H���+�j!F_DR-ten�,�
�^��%w�3`H�?!�T��R���{Jj���R6�*|�
�����ks�V�.����[���dw@i��m+�qb��L�93�T	Ih�� �����mol�� +qUwlic�����,��Z+��@�A`����]O
v>��h���YJ��Q�y��'��4�(��t&cX��2sr������8��q�KG��@�?�� �N���jc4z�����S�8�!�A.
S�A~��vO����P���]�I�z �Rv���Q��D������l�Gx���5q7��6'��uSuuw�4��n����F��\��������{���'�~���O���;=�R�\��ZG�c��.���!�*����T��"������cRVuM���p��2>B
Xb,o�����[�g�����H�#K������m�/<����P�'�� ��X4�G�����M*���=;j$�Uz�8��;�������YZ��q�E�q"�[N+��;S��g�d(��Y�nWKbT�y�
U\Ys9,Z�[�7��`���p3���Y.k�J�z�d_��W���1���;���V!H"U8bL�u���r�F&���y@K��Tb��1I��	\�����Y�3
�2F*�&]�?�����D
u���PFsL�O�x:�Z^(�ea�X_.��p�r�Kk9�kY
�����![��
2��L+p�X����.�f�XK���S:��qg���5�"�.�l�T���g�����o�RZ/��S��B���_��TVS�Y.Uk���w1������0�f�����.�w
C�(�J��C��V'fkzI��fy`5,-�����q���@�!�al��#|(u,�)�y��Y�E�[�Q��f�Tg�A��H�6�������`��t\�7�����\&�����e�0q��#�E��e*|=�^�y�c~
�5��Z4��,_���T.�X�Y�%bPs�,�UC;a��$�2�KX��j5D�`_�7/�7O@��Z�	���[j�+�[�f�n�'X��V�'~��)�u��J��5��X������X<Wr)��T�R��F
�n�u.+^
tY�c�Mg�`k����s(x��C=Je��s�8����Y1g��W���	�"�1���!��v�c�m�X|�AP9W%���\6g.��k�"��2R��g8z)4)z�A�-&*A.���s�P������!���Ou&���>�	�j
�r��QF,r������g]]�7��l2R�7LF, �]����8�����g�$��&e��W���HSR^4����P�u���
5M���XB��RX9��HY����4*��^8N��6�!�����������1�����I��WW�xE�y��,=<��$\��z�����-R9�^�R�5�H�T�H|jn��J�@���
��������r9[�]��>�g���g�� ��f'Ax|OB�c�
��-��`�A�D;�����@�A���Y����Ho���?H0#����[gI{2���������Le�Qg��>����6��2�|��(5
s'�n�����9O��������S�7����=�C�"��h�g���	-g�W=dX_���uSR0��;8:�"��&O����+85��<A(a�r~���u�%I�.I"�����D*��y�������:������:�f��^9o�K�UK�w��Ksi'�&U<�������>s�����w]fxm�{wC�����������$��z�`N���oT���(]nB�_�QI�Y���cu7�y��t�+v�$�N����">+�+z<�(Q�#��Q2�tdm<F�0�����	�HLzp�6��+e��B]l|$�s�!x������N@1C2�
F�3���d@������|���{��U�j��w����)���TFFS����sXMA������7+�y#��������1F�R�*#;�At�/U�_��������i��$����Z�b8s}MN.�d���b���|�w������1bV�R�0$�K�Y���MaH����@f�,�w�Y����~���=K��to:`��v������9�%�Y��LmO���=&�������1��h�c���{���9�d�"����
?Q�:�jK��`��+U'ydbG��5���#�@����*��
�(���S�N��,)�,�r��<�s{���J����@�g��QM�^_'M1E?��=��H����@��Q���t��S����t�3!���w���d��G5UfS�>���>c�'�N!U�`=,[0U��Rq-������@s��WGN�:e~��]���3C.;��k�-���PoO���J�9U��T���n��I��w�D�;UV+#S���������j,�����$������kz�)
F����{���6�u�nd��=_a��S�O��	WG�i~�..J�'��vF�|�z����L���P��HolG��I�����\���Y���*�m{��Y���}��-h;�B�-��u�����Y*�n���UU_(4� ���fJ��n��`!�o���lx��h{�����]Z�#��������S�`s�~�����d����E�>��;ts[�=`��~i�3��a�f�!G��+����L�=���j@b�#Y��9�8�yJY<
3�EC����PrS���r����%�?N_�*�:c���uw���5����x������<��z
�)�
%TX��!2 �����`/c��3�R�VM?���i�i�>����4r�l����V�f���<�4�1�Fi����k6{�78���Td/��[H�;�_(�R��.���������������@���A=1o@]�4�F�������*wQ�.��wYZ����������m<�t"���w-���w�B��Q��'�p?�N���i1s����/�wNLjsB�S� �N�!��G��*��������]��b.���),�K9��]���u��)��
Y `U�I��^S	����S>e���v���7E��`����:=+���������N��S}��@s��C���)a.��B	.���_S)��H���^��N����<[/���/u���t,������H��ki��W�1���RtUm ������������,�����:Qi+�t�,�8���):������w�X���NZ����N��f�sA*�x��B)G���|8?��h�+�"�����L��I���+� -�*]�\�t��q����jH`}������f����%�|���X�����%@��-�N���Sl���086�f0�ncC���{.���z
%c�!�*t��#]��4'�gl*7��k�K Z�
ki�R*����K�:S�fy����Qk����;9�B���z��)�T�%��+]1�g�"m�w�<���5�m
��D�3�6*�?��.��>����W��
k�R��B0T�Ky�SmL}�g&��#���c��(?�'���;_����F6�};De!�SI�����ll����D�������zi�b���S�F��0��}^��=g�R�1C�X����(��>�3��7�1P2T�*m"���out�$��K����Y?M��� �p������m�U������r�&P�>��`^��l6�	=�\UX�T\%�Z��2���xM�}�
��L�q�P;�����n�Y�����@���,|�Y�JI$� �0��A0|D0�l�0���c��}+���3�X�����������"4��A'�������L	z��jkjZ�/�b�OQ ��YoKZ�d�,��rY[�1�e��<�����rc?Se�z;a:����*�P5����V�j�V�f�=����I�����N��;�(������3u�zFa��$����ZT3�Nh�S"I��Y6���.T,+���"��D���,�@	Mv�P������a��R`#cS��Y�A��������
:e��}e*���h�v�d���	U��40�$�6�k�����M���0�h��e�J�@�������#��� h�}��Z(�!o@����^�,���f1�*�L&�A&�z��!e4V���d�NBi���v3����0�Y<���;�89����'[��r�<��N�$�y���NK���w<��,q�%��a�����:lY��a���qB2ym���{�df���L��;$Sf�~EHfF�����DZ��jT�L�����R� ��2U�������Nx�	��u��5�hM�.��>��%I�N�i��\.�soV�����$����V��$HL�b���$-��7�`3%��|$\v%��SN�%x��[L*��p4���jE����K5�'�vY�\\H
��)xa]���O����5f|^R��G,MUe�b&��+���Ty�2�V�)�u&�����eV��j���2+?�J-�77SH��1����������`}P�E��y-��a���ALt��c�����
Y��������n��#��r��������~��O���_|:����%W?0���F/��+W��T�g��5�G�k3+3�3��Bg��!-s�R������W����9��c���?��{p���#����_>r�cE��MS��D��M�������������>���V&�(�s��yeB;�����������w��O�����9�<#n�
�j�����a��P>���d��h�D��!@��h0W
�j����wf����oK�x2h\���C�
�g]Wj|���D���O��>	�w����b�!~�������E��.���x��#��?��cU���u���s��r=���r���pf��2����1FN����*/!��m\����}���;5�Z�W�@�|��N�|��CET�q (�O����rtx�w8��E�����2�*�$*��M	��U�0��T���@q���<��Z�	B���c4�9�A�GmiP����=d�G��pYK��o=3z��B��J��)�����F@0��5"���X����!��`�����A�x�|j����sW
����
<{���st�t:�]������N�K)���������%��1+����^-�y3�ky���%��A�jQ��G$f�noU�k�z��u`	������L!���������[#G�h"�H_�6�u����JyT�p<2��sd(�Y��q���+����T����Y�����Xn��jp�\j`�3�	9��VI�B�3������9��b�K���B#���Y9S�T�D�����-��� ���o�������r�Jy��w�����n0���D��������9��?���\*f�-� ctG���'��v���p�|4y?�x%:��<Q�f�R��SY�l�>��V
���5��~��Oj��F�.�����[��

��[d���AA�����
_�c����H��^���Hty�q����!Q�8'�3�iT�/��DNN�U@��������C��o�I����N(�kmVN��5��h����.�M%j&�YLk���(]H#�����@!���mi�������dm1x^�m0Y�fU)-T����iq�WU9�4?�l]��F�Z����4���a�R	���|G�:�w��F����F}��#���{j3rN��O<W!���)�����Z*��^\�x;�!?*�\G��"�T����.� �B5s��Wp�o��UB���s��{��/�����+����+t7p�&%�e�/d���;%���/���@H�<S��iDo�Q�_��
��}'�Woc��C�=.t�_(O��R�]"��F����)�=�s��I���6�z�/���#	�:�lG��	����!o�,�/Q.56oE��m	<��Km�����Eu|^(s��o{�_�nrS�a��{��
aS����&WJR�^��3�lf&�eR����%7&d����M����<������t":������X��J{T���^Lo���D|4R����m��ok?�R���Z`$_����+dm �����_���W�R�I�2jQ=q$��v����9o\���v�qt���s{�����2-�Tg3��Y'����bQ.�����RD��v����+;_�B��/h�V���@���*���t~!hw
P�3��	�����Z,c�D��
-�c�d��������W2�.l�y�5I�A&����^�tl]7��p�+����OC���Z>�N��@a���G���.3���?u�N��L�x�Q,u3 �0����p+EQ|��w�`~�4%��g�`
�(���P:r���#�����Ntk��(@��[�!���P�}�f�B�#��G����t� #6$�Q����$�|S��ai������LgN��Dy��I��7��F�������$u��sn�����7 �u$�>��_�����Q��90_G��.�>����*�	���X�8���x���������aK�F9+8:��}��P\P���~9�G���K�g�|h�u�(��b��������g(�`��
�������>"��t�X����`�:�b#%8W0����,�lx-+�}�6��ic]��'����(��BK<w�	�6�Pp���tQWe����%��� 5���S�9r�p�0F���g��A5`��&y�����
���q������\a��A����L'���iP����\
�e�_��F���/�C	�-�o:���Q-��!�GC����]�X��������&�������W�(e~��|�)�Z��
Un�?"lE�I�(��g��"���U��:��\��	������lv��v@��J���������cG��rO�h��� ���z�H\4G�Lq�)�����7��&f���K�RD���Q�l]o1�/�&����F>C�}��l��I�x/����SK���}���������v��/�������.6��/k���:Kef�(�i.�^���v.T�d�L�C���P������[���;���|�%_,w�	@Q�lw��$��rk�Q}��U��(�y�[�+�c�S�Q\�� �P����[��/}���3�5x����F?:�I���9���`�%��/GA>�h.A�����7LI�1��xP�9�g%�"m�t����|-��K�
��k����������rf��]X���+#��$VJ1��#9�L� g��`�c�kIe���E���x62��*���/V��U��2R]g�t����q@_���~���������T��8�&�zJ�'xA�X�1��"[�1/�
�f�R�����T��f2s�	�����)�
E������������%G�Y�X�l�m���el4���E�l�J�p]g�t�>�,�.��A��5�,X�S���M�S�).��~��P�j� t����Z[X'k���D�]'J.��S7������6z���Z@,V�.AS����z�>	���q������>��b5�)�:��$D��9����\?��6an�y>�8X����hW�*�;0&���M��P��MM��k����7�c9k��X!�x"��/��-c��^�RcC��*d�-����vsV(�/8q��X��&,}�.k���\��F4��c�M%�JWp(Dj}��^a���L����-�Nb�,B�'$5@��*��F�]�VPT��J>Q��o�x�����i��Z/I����,�
��H��O.I#v�r�+{�OU��O>�^����9}��Zg�p��j�w������B��
�Cht��d��v�Y�~89{{r�	��oO�/���t�������~o%9�h�vx����[���m��=����w�f�Z�B0&�����J*yKe������cV+f�r�ZPLS.t�����w�{!�@X�_�8&e.�xXn�����#7�Y�W��,�~�	8<N�������#\�������W\�W�]l�_�=��\6�2�Hh6&�����l�=c��U�}�[2�r���t���EYy"��^M���[�v���Ru�T�?-.&��;���A47z�����2�Gg�J���1����IdN�_���P�J��/�t]���flB��Rz��}��Y"K�����S8�jm�V��C������?T��Y�8w>n(�+��A/��w*����-W���zFGA�G;�UQ�]�|�����������*�5���U2s��_����O���Z3��18
�
u%�Cg��y�&ZM�-_��p2���g���p2����d0��z�(����
(����(���
(�AX&b�e��#��� �;::t���X0��^�=� �&���O�E2�=����������ol�e����t=|��$�H.�3d)�Nw�C�-%�����L�
��-��{_�H�j^��F'�%��$������gQg��������%t�0���f[�'I}��\��pFm�����f#P��e$���;Y���pU]��Y����8���M��m�@�J<c�*z���7k<��xE�1JY
EJ]j&��;�M*�S�I5D5��k��t4��LD�j��q��
>r�����.�=�Ma9o;�O�u����@?��"�J&�4_�������\�y
7M�}��{��:"��K���Z�3%�LNX������F
���)�gQw���S��1M28��v	���k����9D�4���=Q����O� 	*�q�L�i��9gZ�^�A6T1�+�� �Fr���"	C�/�%p���X��p��������wI$�
h�>�u3�M�����fvd�uT�X�b|��*�E.�wUQ���e	EAB�b�E[:+:�N�g�H����p�q�aY��������PG�v��P5�V���_����jyi'��Xop7�B5��5 4����
����f��,=�W�~�"���xQSz-;�����j����`��Ye�BJ=Z�xm��F�FlP2hJ}�!�7�$����o^���
���|;�����F�����CH��cHeE�����.����x�	>$��QR	�m�<kq���������+V������g��p���e�7���z�'��@�2��2w�����\�x��5�VYP�v�l�R������L��H'K�hRY�v')�� p\{\9��8�p 5�(�
d�#������@��}�q��5������W�����c+�1i������&��My������^����Ok
X�������	�R�r���Z���������P/�!�}�w>��������L��N������
:����}8 s�����#�G}	A���7��bo'����������u�^9R�
��i�Ey�zs��X��M�1�����0�Dt��#�G������d�����!���S5�3�����:�%�Z�do�tD���G~�cO6)�n�9�����U�&A����q/|�����&A��!�&A����&A����&����%��9JF�n$J�IYd_�0�L'
%}'U�$n�Mt�u��Z����D�I�+��g�T�s����x�����t��$m�5��i��m	�6(r``K�F��C�%���Q!�Ai�U���F��x�
	R'dR"��8� ���YC���n��)V����rb��xd�.�$Pz/�,U�yb�+��#���'O�Bzb��u������������$��[�*]I�f]�MB)h��5��[��?���AN�1s�+�����}�,��L
��Y5/'m�=�y_P{FOz��(�L�E?��)��U�wi�/&5e
P���7TzD�Mt�Q����%��,1����#��X������N�U1V�9�f�������n)���iQf��M~�i����WNU��AN��+D�z�
��"�!�.=�$��P�g���j��<_������M\��Z��r�Rb���fp�h�_�$���f�Za���.���A�8z=��	��������p�;�
��<ex�����x��N	
!:d�/jJUnO?��tA�-$��v����.���}��?o�u�D����,��Mz���t��H}�H�6����.���Cv���g�����*�*��r-�8vu=G�����Q�+����r%�Ei�����n��'��~�0YiB����j��*�u�w.M;_7A+�HyA�����F�45�q����|���W�ZK�/3<���?t��`8G�a;�x���^�O�5}�<��y�N��w�'�L�i�M���o�6X�>(������0����'5�8�aHh� �]Ts��*��%YX%��aB�	� I�[sx���R�YJ��*J���U`�,�L��R���C ��UW���?������R0�D�MI�Ui�a�QOH�)��)�y*�����\�5���tzv~b`(��@���b����|\$�d�Z�|P"��K]��A�am��{��vEB������4�F0�O����R7��D��N�M�����
#�siAS�ZsX�@���+&9d5k��5iO^PH��z�8�:�0�.A`�nY������� 70�iI�������4�{���B��������$��h��b��!��eu]!�:^k1!�I�Wv������.t�6�@2�zg���-E�0M���C��QDV�0�hF��4K{��?���f����^w��>y�G��W�Y����8vW��l����*C��x*l;����
R����W�l�!I.�����A��>�v�%�y�_-K����d�;.+�#�y��f����}�uW4����
T�Cbl�lF�t������.f$�}�x�C�~��	Z��D�c��)=d��to�����pBr�/�$��A��PW�x������C��R��:�	%����e'%����������i+e�Tk�
S2�vY���
���yr�01:�M�3��i��}����KH���|��a��Uthq(�xQ1Sv�vX>H�;6O ��k���r�������Hw� �$AoV���L�AW/�/SJ;{�7nAc�!�j;�Xo��q�t�e�IyM��C���e�m#�i���R��Y`��m�p��^u�wJ��;qLsy�,j./%����;�E�k6�qvE�����w�@*����Xt[�Q��OS$9&((�gCk���Rc6�������Y������j�&����W�1b@ok6F��Ql}[:�"(y����|�����7|��jO����C���+����~|^?��S~B.�Y}�s���=���)-���t&����Yb���o����N�L���<^�O��[
�U����CB��.�*�L��Xc��s��01������T����$��G��5P�{�����3��z��)��)��Q�G�_�6��y�^���3�c4�"e�GN������_���sTU7*��%�
l�;a�n����O���o�����^����f4�FGE#���c�p�=�O������|��� i8�� G��(���t��{s�n���Y_���X��T�H]+2����hL��ey��\z|{�E���[�&7��i�gm>���!{����mK��7��U9�id���"����la8�_��UA�h{iq�=d������|��i���:1����4�S}����
B��~u[�������z���E���5�C��IV`�Z�T����JJ�Q)��}G{o[dT&MO�KY�����p��:�2ze~q��n~���vU�Jte��b~}�����.`[!'��]�K��v�Z	n+T~��:tW������g����9?y���7g'�J��(`C!��@��n`~��\���P/�W0��S>�wwz�,.u�q��
z��)�����3<�"�����'
B{��|�3"��n�
9�~��E|��s��!7��o�0�=O8�<T���9��bEvP��d,��"�	��.���b]�x�(�w-���>���J����Y�Q�����0��3����J9P��g�y��~���|V������G���u����h�8Xw��mPm&���/Fh���A���K��Qg��B�x>��X�X�&$�
���
-������w���		����@����
X/q��3p8 +0
u��Q���B�`����QI.k�@�D�1�����L��E���y}D���7~R��������&���B|����x8��3�\��n�:�2��Q�q�o
Me���
"�.�����F���z6	��������^c���Q}�H��B���"m�i2P���0�H�a��v����Ce�|��Sv[�2l�*r[����c����h0�����{v!��tu��dH�m��J>��C��CNX(�"���w�}��<U3���X�c=��
�Lr����y��o�y��}�^ [d��NZ8����d^X������L���ne��+��%�z��q�qtlh�X+��w��e���(u�$l_�N:�l	���6��hD������������3V�oMFLo�������/���s�Q������@L���
��Z���?�����7g��h;ywq��/�O���C�(Ly�R)"q*�Slk��u������y ��)�/t�I	�C�/;���j����oi�J��-J�6��=�=���?teb#�����3���%V�L\���>�����"��T_�1J��
0��e�p�����@�^`��/������W��2)��:����c�
������q����?�?7j4P������3�����,0�9H�<Az,����&��\^�(0$A��T�>���nC��K�`�k�E*��f�@�[�������j���Wwf��Z�G<�!P!0��@A��{��o����VT�K�2p���m�o�K��|1����k��������S�DY�>>�g�N���T��	[zA��E�k*�K��b��%Y�1+��H�������P�T���S��e�l�h�:�q��s��e��H�8�AB���p�s��J�A��I(������*5����#}����:�JC�����K<�LS�F8�����&��#K���~u[Z�0�KUp_l�Z�0}��g<�TO�r{�8uP	��M+���~��E�8�l���N�H�3���`�~PC�+�c���5Y�mfl�d]�u�-����u��p�>^vI�y�����-�C,8���B
�aJh�ir����Q,8��A
.���q1�*y�o�\�R�Q���a��X1d�V��U9[�:�����E���s/��gx����8��
�2v�y�U*uk��������Ao���+.`X5��l����Do��5���,g��3-���o�),^R'Q���',�^�,����/G"R.N?]|8����������Y��=�j�����8�O�l<�����uL��m-�A�f�zt���P�;4��P�������yYh{,%;;5���}�)����@I7r��N�����(����.�,f�C�.``B7���XeR�=#a������@z<Qw0C�Y,��f�8j�b��%�R;N{�����l����u �=&/w���j��][��L�N�!���t�Y��bV�b�A9�A���F����T�.�#�D���_��,��,+��Bs��Z�j���<��9]g�� ,�r5T�E.!��M��a��C��<q�/OLRZI�����2�T5L�-V�D�q��>�����h:p�\���1:��ywq�8����R$�3e�W����7a��$�Ut�m�wrfvd�����U�l��tU����FQ}2�V'�Y�%�a.�����*qW��E���8��y�����Ir��A����<\-�$���$��	�sh�D~7FI8k	w�
5Ih�=�
�:It���:C,=o8��=���I�R?�d��`_e7C��p���.t�a�*���WE��C;^:���Q��������(l�]`F��B�.����O����Nt�,qJ�,��U�{���6#G��Z����^�!3B������*����hz��Wp���]�_�I���c2��]yi+���KGP�c��p��
dfY�{}��_C	��Ovi��C��Ly�3uLV,u��w�����\X���F�B�>�������.9X����~]LPQ�������$��s��O
����x��k�A��k"����o��H�@�1^��Jw����=�G)�a�	��R���4�W�2d����9����*A+b�G���yB2|�T8hm� f5�U��K�nrAY_��']z`�t@%���~�����B$��2C~�}���(���������=�����l0-Dyo�&�)��=6Z3{��6������p�,GW4]
��&��d�M�G�R0�;*����4R
��m�_��+{BN'WvT��&~�3]��Z#���n@����}���o�^G�y���O|�����e������mv��_M���S�Lr�zI��2��gyF������v5����P���=����m��&A�!�n����%��%���~��n��x9vr�A�J��,H��Rl�6�'7����r~
�B���f���^������t=����C����q���L�z�5	#'�6_e�N�I�U,Y�Pi\MB�q5	���$t���==*J���D�$�	�'b�mX�n#�q5���j^*}��`T�x�9�[��}�@E�
��Oj��C2��;�������:|)�	�01<w dpat�Z��
<w�nI��j���G�
��&d{L<�owZ8���>0������/�/�N���+�h�B��a3��F������)�C����MXJ"Bir9f4��D
�"���|zp�f�rB	�N���;�x�M'uRa J����'�'��h��R.��W���Y]/�O{�	:4�[e,r3:S#��	�XJ ���0p^��E�w=�zIt�y@o��� ���i�����:U8]]
w�Eu�7������-5t����wms�������t:�Y��a�U	���,���l=%�Q*�$o&��r��P�I���1]����zJ|�Q%�RW�h�9�����������b�S�p�-���*����Kv�&l1a��hG+��S�O��}�x��39�SYK��nf�GT���$P��?����O�|lg[�������$��o���R��&����i�X���9Q�N7�y����^��8����_�f�`���Z��u�Ij��%�o[6��Z��N��Zwc�0w��|�����N���8���2��T��B��HN��j�Ij��E�i�q���:--��f����X�	���{��	���r1-�-�f��n��&���\���a��?E<^��1>,�3��g���\����]�� v���jXH#�,i�,a�&���}���>�� ��d-Go7�;D
������&Iy����������[a���b1U��$Yp�����z�	qQ'b!X����!�(��Y����jI��A�q��j��0<Zn-NP%��+��Jm�j\2�g��E��X�i��5�S������/��`���L�Xv��%A��u*�����.��{FU��B%Y"��Y-+���0���Wr<{���g:�$K�-)��c������\V|P���u�X$����{���/����4�
l����
�8=�c��,XN�u|�xX,>��z����w&L���v_.�~�'��!�����9�*�79o[�����U�`0
��Fh��})lt�d�C��!���]"]���D2):q;5�����[�����#s�E���1��CK207��������.'e�Q]Sf�Ge� ��JLo�6����N����,w�(iAq�1D}��w�-�^eX@��q�4*]�,�q�
c�o�.�p�����`@1�Ee�KG�V�	��H�#zl��j!��V��~S(���&�RP����T5�x�;�W=v����1�Q['��!���!����*�/E��~��O	
0%���n�{����aK+jim��w4z%���g����<�Z�{t��Ei���N�/������;���1���1+��V�(9�U���pp������[;�]9���]y�+�c�k�4��V�z��t|��9&��N�O]�EJ�k�CH���Y0����;��e�"lA1����5
���4�$
\A�Tw2Cw���o���dQ�G�������4H����m�F�����[�]�YtJDJy�:�mJ���,q�Q����%a�x}}�	c���r���J�MA�������<��Vs�'.'o�i�rM���wUw�������.��W�&	cY]_�����\#��rQ7JD�4�5�r)-��������@��4$��+3�!���r�/�~],������v[��_���|(4H�p�B����	��=��;�����i�~�1��������i���O#���q��(r������V
!y�@��]�H`m�H��>���0�Z���xd���;�K���{�����S2���zj5P�*�Ke�d�U#�JZAgD~:�&h~�[�����d��������h"\�5!X����k�9D��7L��6 ���U�t�!�2���4 �������0w���+�����'okZ��K��kk�/�D3��mQ�>��j|�E����p�0�D��&������m�*���y��I8�6�B��&`Qg�t�*
cr���������a
������Q�����L�f�x��b���c�I����s����p��w��h5Mt��)y�1��OOZ���<N�wtn�v�x/�7�C�_����\9]�SB����[�ym��r��^�������j�*�����j�5��M�hP�UO� ��!'�O�(��Z��]�
-�T�z�(��� '�H�:Z�#3�b^W:da����eL�T�!Z����q���r
L����������2J`�o��q4	���@DifrL>P
`^+?������I���=A4vm��]��1����bZbZ8b�_�{7�����+�g.yX�}�]�/%��$����pl��3������3���@1g�*b��^�;#�@�.�&�{T�$�y��$|Z������O�Y������*������if����s���f��X2�)x�������w��z
�����K�����n��}�>�(�����"��n�0��`
t�0�{�6�����q��g�j(68�����j& h.��9(���^��mj0��z���$������E=��?��$Y��3	���3d���G�Q?rZrkFy�r����>�����m�P�lUs��!�����\�.���b��QlE����{�������*c����R�#���M�� 
>uD
:wOw"���/��l��~mM�)�	jJ�z��
-�nN&7�S���+,��69�z�6��BSFS�yQ�a�@)i�.��w�|��5�1{�>��{�3������O<se������UQ~m|l�������YK3��gi�#S)}�2�4r��5�)���W�8"���w�n�eq�^����L|T�M��op����K�J��0�:"�2�o��J�|����-��s��|��������7��yo?����_(vd����E7J911&����k�������(����nEl*����V�(\W�5,����EZ��.�Mj�=�7%�c��6�iKkDQ&9���/^�:H����J
FT�Tn������]H@���"��X�pE�T���[������8<V���������y����`U���KC��\OvI����J�������J�q�o�DN��?����5����=DC'&6������@d����3|G�_���3�,�V�Mt��OH���A��=���M���)w��b���8?Uj���j�8��'����g|\��K)��wf��,���>�������n�\����kFN��7���5	��~����
B �Y<)���������C�)�	"����6c�5���k��)�iV��=�I5������&_�_q����EL�r�46��8��#�.��<� v=���jBl��������m��o6�:�l��o6�5�.'�n�b/[�9�wO%�Z��	��G�g�xI�.^����PE��>�*���K=�RY�=�}vI��<�g�9DZ\���Z����%���j.u�����.����W�2�(����<��M��3�GeF��X��z1eM�T���?�+BL�u���6��0!����X�
��a�c�<2������AHP�^g�/�����w��)������LJ�e�� �\�f��	�VM��(��y+gid��7��<Z�����bY,RxL�qK�m@��8�60lB�Z�8YF8n�,�:���i4�C�|	������x����M%�4%(#-��Z����lf;y��oi��Y0�fU���,#����f�:G��M�Ak�#�T������u�bo��Ca�<�2?��9��Z��$��~����0ev�>���~���3&��Jh!�c�W�������t!0~M�ad|��Nt����E�?�z:V~���y�e���L�Z�lX����)Y�4��a�c���/�h��q���=�)�����X�1=�l����Q}G�W���N���"������H�������Y���@�U��|d������b�II��i��{^q��db��R9�M{�r�:�'G��J��G:��|D��[R��h�$��0)�,)Q��o��,k{dZ��1���7��r�uQ�9z1O��!�:^m�W�n��������M�<!���o������;7�2��Dp��n�-�x@��{y�OB=�D�
��'&B����u��W��T�����s�k"e�j���~a�+���zF
�
h�i"A�WmB�/����LA�����y���_��?N��������A�6�F4������a���G�����j���]s��)�����Iu�f��D�A�T5�����	:�.��5������<�Y�{k��7�r\]aUw�jG����L�����N�-2*��'�����i5�gv --t1f�R�O��.N>�b��!�eayI���{X��_�WM5�9r���$o�N%�RF��?@�Z-KJ1�&:]'�8�y�;^D�I%-1�p��,)�<�D�`���V��>_La7u�&$��]09�zQ2��cW�SCGx�*e�Hjc�s��N������p�A
]9@���t�^\�=���&�����+������Q�B�}|g�S5�����$0]jh�<
�1Ml�b����ay�f���.����]�Y���I	�{�����D��/��6���&����rCA���.XqU�����Jz�|�q��_����~��|�>@����W��[2;ae��<��PA\�@[4�w�Px���/re=k�P�S�l	�va^��|�����_)j�J�-VC
�1����&T,�!�����9i���<��w�a��;������Yo�k�~9��REYX�����J����<��%�[���`�M�/�P#3�x���I_���r0
\	�.p�/f�U����7��Q�Y_t�X��#��J�����$��r��4\�����~q��_>��\������_����������dY6T��6�L����Og�(�[y�������Q��i���J@�i_�s�6�Y��������������i�i1c�������p�����g�������P��5J8���i�y����c�e����������E��{���&���B&)'���
���_�(��\���-����K�v�<O�#���7A��g�"T��
�)yo�G3`<�����;`���7��#N�rT
�C�g<���~�,o�a�J��
�}|1�L,CD�|�4>���`yn1����x�Rw���_��s�
�0l,:��GFnvDG�G�h'�l�T� \�)K��[�����EL'������JA�~��4���J�����,Bs|�������p���~p����RU�������(��;��h�F��@��{K���m�W��:\��~W&>#j�>����>1K���47������cC�.�#��uc�t�|6z?D�ki�^P�"����6R������b��oJ^���rA�zA!��z�\�����Re�-e�{�����j*�pz�dq�EB�{��e!����V�-kh ����E%��Qd��)�����i�G	����{�G�m�7v�dT�����Q���Cf��=�� �6�}��Y��=](uZ���SN#�J���-_=�6=��W����G������s!_)]�P����#��8X���������0]�zf���:� ��Li�&�d�c��O�����6�K?AL!�n��AM\ t��n��
'���.j7��S��.U���������U�`:�88p��6����B������2+�u���gg&f	��Z'�-T~ +������"���K����6������g
��h9�����!5]Il��|��rr�s��*������{������S���#�7��dL�o�H�������/]��*���u��P�!�X^�*��x�L��/�aR��+��}�b+��-U�zp�g*o?���=�8��������o�|:WR
Z�X�]��G�nYc����3l�ys��z1��B���V+�����2����{
�9��y���T�����2q9�:� ��l�IR�*_��*���9��gL�\@Wa�d�(��	�k�0�(���Xs�>t�C;�S��(�+�}��
J3�b��wv����;DxFW�NX2�l|D1+=P���n����w�$��[��{l{�{I���rv~��\C�&�4�c��� M|�B���5���-W��J���u��n(��.�m�eH���;���I���;�{3Y�fw$�+�d[��6���s-<X�o����$i�f��U���e�(���������Z!��'\�j���*W��]��.#(��N��5�:%�w��K�����`�m!-GW��Hd����!1p��-g�+ji�_��0�\6������U���
6�##&���ur�6P.�AS�<@~���?3�SZr��d�@n,m[:� �5[�0NTV��z.�����/�g4!f:@SJ��\(����}�A���-��o�K��pEWR���}~���_u�����l�|���Mol��%yT�A�k=n{�70(�&�P�������P6�pT�#��XF�,m��a�p���'g��-����k*Iv�����������X&Zf����.g�������,9�A5�'7���Ft_;?{�m��>|��.x�����Eg���G��~���91�V�
Gpa������BT�-m��rW
�|�]e_������y�W��V[��JyS1�g�MM�[F�7���\�����RY%|�)a���Y1=�	�Tm��<
?i3��b_���ci�����e�(	�'=� g�?�A�Q���a���Um�
�u-%Xh]�6^uH4���.��yU�f�rm��T�;[&��<Sl;]]�=�
if?�~�
�A����0�<�e����7�/~�|�m��C�m��.3mCM8�j�`�J��8"���e��j?b>�O�tR.�1S��^���6n��Gf��T�������n�Q���,�AT�{nv�<4���G��q�I��|4��|��T�+�6��������
�����s$
	b��=��icr��8-w�N*��S1�����[J3�l�����\��	C|��C7����O�OO�|^����V����kG6��{$aF�=8�E��e/�0'���[K���/LK,�E�[ZYK*�E�9���E���\�.~{s}� �xOT�a�N�MSSo�D1�<3f�e��$J���wfz�������b��^u�q1r����(�+�S�y�n0-����q��
]W\�����b$�PG���n�����xS�0��
fa*5��{������<��������5M{k�qNBya:{����|�_p~���h�F[t���fe���t�<U����0��jZ������������if��yV������8�_���e}���v6*h�p�p(��&��KI�Q�
��)�W��+F�B.�h#Ft4H��I����,���p�����{���dqC%�ABP(=��{���iW,rW�^>p�����n������T
�����"���-�>S"$�#:m�	��Nb�6�?X��G�R�^f���i�����D���C���=����Oj�:to�zU.����U{�T�����Cg��Sw���t���r���#��l^���`�y_���J�NS�����}���F�����>�p����r�����`t�M�j�����	F����r-�����x�A���+�D�s�i�a��V-����
(��j!{��u����3-M����*������/u{8�aIs��_�_�9(��"�I?&:���:M5�VSM��KJ��+�:]5���7QD\���C������"=y�r��*-��v�Jn��m)ax��Y� �4����XJ��g�������5�F�����*��RH��Pi�s���M1TK&����z�DC�z�I0�����[���\��4�H�==ONyO]��.K����I	8-��7H�F� `��/�O�n���ke���b���-��,�N��t��i��.�br�v���y/f6���9����:��8T����XQ��6D�/��T�}��N�`�NA����]E�a���1�A��
���y�0QG@CU�h��P+�%A��f�����E����v����h�Xa�d����eq��GF��x��sM3���~I���Z�J�}-p4��4���S,-=��,�FG�e
�L�`}_��]F�a2���]�y����>��9�>�*#?E�ehg@���D}�dpR����1�*�U�-�.0v�Y����f1��5���f������ ~9�:[yk�0Al�0S�XA�wNV����_��xC����E���o����?
c��}�������d�������$�@��	�����{B6z��N�����?o��0�TE��i�,"-Uu��]dP'o�����v:3��$�u�d���{���|,�Yp�'vV�����������������}|s��'������h���m:tg�5���EN����sjs_7i����hUx�P�.E�*UC_7�4@�C��j��1�/6�K=%�i���5��*;��j������/����+�`i]�����-��Q��)�1�����M�svE%����\bE��:b�q�)�M�cb���)�#�we�sAS�i�#��8�QY�*�D�w{�D�����~6�H���@Lw_0-j,��CYk���,�<�i#����Z*�Y�z���V3D�������"��%���E���f����u��4��X�	U��SN���p��Z�����#����r�t�s~�&��^��+,[e���������sO�/��b2�l=�y���4������a������;�1�^�o��q]5�Dw����[��\�W�91�*�4�
I�h����������2�\���FO����'��p4D�Qt���z�1*6��lCb���~�X5,J}P��;��]|����y�]�{$�Z���������������������a!��:<����K:�G��
�iw�*�F�������H�r�i���h���c��������1
��!�t��q!���	��zyG��5"W]��S	�{�47���>�������y�I������
�~��lB�����!�N�}=��}��A�a�,�����7	^��JspP�7E����)0����CY������{)Q�� ������G�Z��V����������z���;6s�����C2���7��\�{���<<�C�W�F�C��A<��Q/���
�	M��#�ytU�$������3���_��<����B�.�}�`�*/H.c��ao�Q���,"��S�3���/%��G-����R�)����9�En��g\,����3.
VZ��������!�M�_h��nQ���.V9�jQ���9�,1�b!fV�T�n�~�'J���!���'wNg�ac8����L�Y�C��j��,����KK���%��z��fe�(hi�E	v�S1
��p�Q�>������u��?+R��P7:r:��Ba������'����{*H�a���hV:t��`2c�L�`�T/*A��i�
��
�������EC�����oS3j�O������K]}c�JU�4cN�9��)����'�r����!��_9��p���U7x������V�(�!���tA��T�����17�Liy��u=k��T
�7-m����*���!w�;�qJ:Z��%Z�]����;?{&�Va��D�E���k�q�/S;vS��L��X�?kj�!��lI�'n�V�A9�&~����&��|_��[���S�3H�9Fc�Ljm�4�[06
��pca�H7���/��C�8�:w1��={�w�`��;t(��^;�
���Hx���vj1X������!p+��dS�>�,
��Q������D����7���a��V��E$�����+��\���������N��wjW��pJL�.PG\Y-{
C�CN�~�0�C;j[�R�Btp)u�ld�-��nT|�nGMmP������\�{l5Y]g�w��	��;��������M�z*��+������
$f��I�f��|�]�66�����%���5�tR��V0�����^3C���)��QC'��,W+�YQ����t��i��L��i�h������c0@�z��?D���5���C8�(h+oZ�!z�����6|�������O��[!z���LD��r�d�
�D����c"j�����GG|�w6��8f@jb�H�fi9u��Fb��Q��Z0@��x%�o��\D����)G����L�H���E)J���J�g�W�v�H]�(iC���L�$�v�C��L��8������E~���O�!b�C�	�6�P��Y��-�z~4��D~o�����z:b���!_KF�W��������-��\��+s� ���aF&D��
���c�J�J�Y�������B+����ji���0�G�f"����k{R������9�:�����H����o���CgYR��#�'\G�R����N&������ksS�E|3�oI!�^�J'AF���]� ?������
����5<�E�
��D����<�`���x>�^#�i������]��\���y��$;"U�Cs-���G������>��z&q9�����P��l�#2E7���56�C����S�Ig���a1c��b�:
R���(z�D�(�5���FT��a����������T���E������b{WH�D<$^l����Q��Tc�T��q~�F0�j��p;���X�s����[���k�����f�w�:k�;hK5d%G_�Vt�"�+JF���L�d�hB6StY#Q�&g�E���(����A�M��Cd�,�����
����k]?$����=2�ru[���v��s!^�X���:%������n)z�a"�i$ ?�@W����H��L�S%���2�lU��8�%V��G�E�<�����>�c��(��Ts�\wC��gST��Q����	�����E��D����t��K�������M�X��b�N��:���y�����z����o3d�#�{�n�@U
}t�t��W��@;i��u����n�v������qv1�T���y�nl���g}��fJp�y�^����� ��=Q���*��E��vw@vt������Z)�o�*���F��%�|��SF���Fs��0��n���;l:��!i7��V9���7,��b�
]ZA�'���(g$e�����%1Gt(��e�a_?�8a�T�sr�������|L��}�&����Po�g�*g�zYpZ�l��F���3]k&eBO�
tU��n9a� �)���)�������GB NQ�Q�ju��&-1�,�x���rV],o@E���["�����L���6����hQ��W��x!�>�f���������!��H�����k���������+�`�����G:��������z����p���@z��'?���n�L��X��a��1HG�!,]������IJ���sx���cA����
��N�w4@d~�.)=�8�����^*����T�o�6��&I�eh(����.�8o*,{�������I������.������c	07.:�]�����&��� �����Kx�st�]*����u��S����Q7~�����n���N�������6�b:m��&;��q�D�v���K��t��j_�k������V��:^����������������X�CF�@�����ta�:a�d.^��5���M~��V0c����xo_�y��KR�K��A���*{����2�_��Y�_^��0���bU���._�W��%�����b��������S�^��������.��M���x�I��O����rRh�<[���D�����^����}���t�Xt]:)���4l�$&���c2/%ci^-�Sv�q6����Y�u����?�B�X/Y��*,l�??��<O����B�fu�,?�����i}YL�w���3������]���"�S��Ua���+*	�"������Wi^)O�2�����>eA��;�������o�O?������W�E�I��?������i|��U��t
J������S��
�������O3X�7�u�^I����bV5�b2��8
����_�U����z>n�4�-�o�r�T��\O�["����/����� `�v �N��hG�KF?�d7
��kK=�t7�e�P�$�I����M���(��v��R,��3�(�
�
���D�3�ZKu=y5�-���{`�7/��]C@�j�
�%�/j�	�6��J�����v���nQNJ4��g�or��������������~}���	+$� ����?�YM�<�7��*�0���������8��������$e5����F���n��A����{�������������P��>&5�4 (26yM�����������}bo�����`��s����(�Ge���$(~:w������\������3\K�����<���K�b�i�%����$�����$���R��B������
?�py���e�%���y��S��cx&e}���	`���H-��z|������x�,�w�~�{!��-Rg9�Iyb��p�A;����H���0�]�E1��q��r4Ie�1UV �@�=��g8�3��t��}x����Oo_�������������O���l?��IRA��:�2��0������@to������������^N(���{�;���-���ibL���{��iW��@Y�r��'l�^�t#*����qr�r7��n�^�h�%��ir����>`�,����#8���������h������<l�q������O�o���x�������^#�7&Kyr���k)uE����w����#�1���������(���`�`^���
H7t����|�������������	6��x����C������i����=;d�7]g)w�����X����%�����t��������6NQ���������l4�yX?������<��d�.�ZF����k��b���P��#mI�	���t�� �"|�=��x�E4�lI�J1�v��J���+�^:���:��]w!)(�&#���G����]`��	0�I����Ir
����`���nVy�=��j����fh��j�R*[b�(U������ �Q�J��(VJ�W�j|ckYQ�LQ�Y"��bx7��MK�����P���4m`
f��P�sy�����������8�(��*�� 
����7n:�Y�&�Yus��Z�2k���>���V�vN�F����F,z��^#Z��F%1�~O]��!�?T��Y���%����#��L�f]Ea�����kO��Wh��`��I�����Jt����{VK����F�0��l�+o�S�}��$4F*�jg�|�\q���v�2�Ad�c���An>|��OL>E�V�O�q�^�_|����Ny�nw�u�������6�:n���7��o�����>���}�iVGD,)l�f���X��"\b������8.��MQMt��|���d�zkQ�=�6��g���#�9{8�\�����s���3�NA�[?����K����/!K���%�6B�M��o������W��������blC���
\�5�X^�)��`����u�M����``�C�������;�r�������X��-!�P0����72U���/<�G�\�[1���3cz�|J�d]�y�4���U��'x�����n�P���]��k� �N�n|������\W��td�zG>���c�%�nA#Y�4b��e�A]d������nh�h��3���F�4���5:���r����'�m��gKqw���f�]^yQ����q,���AX�wSO�0��#V������9��:��^mO
��8/�]`��R�!�W��p���@��J�_VK�T=��)&�b��8[u=�����-������B����N�����,:t������x+�����R���:-���y��T*�]F���.$����e}1�<6g���a�����%����&TTCi��c�����z�5���:-��|����z���u=��8�y��� ��/��e�l�g�1��iA�E����yD�|Z7Z)�����_�+���x���/�Wg�azw[���91��^p*d����*,�dV,0���X�)����8��#����'� �w���(���e������-�e�@�����~�������t�^�����3��o=���=��}�
A-��g~��^�����T.����'���r����r��}9��6�!���D������	��:�����om���8�s��G�OE��3UKK������#�n���EA�F���H�w���g�����`���)sv�5��NC�C��?��6M$[�����-f����!�K��/Eo�	��N\���k��	�Sg �%L���y6�|���>����ct�1^�����{%yF�!����:��^t]�N�.���q�W'H"��{Ssu����a�V�����r<=h�R��a$������<���9���k��(��8K�����m������w�JN%�x�I���i}O���s��I"!�1E�iEg����3�9 �|h-	����~���W�_� ����i^YD����v����gVw�*|��gY�[��W��S��Oe���N]�������!�0F�at��'a����{�i#��
fAsR��_��>%��������*�wy�*������gy>Cb�U{���.��,K_]���������� #$�nH{~N+��
��)<[��
���q.�K)������U������[C]�Q�^����x-jNc����������K��o��J���K��l���I���������c����+!���l�T�����M>�M���u���{
n���9x3�������)!�O��[��lc(H������Z=�3�� �D���Bp��h�\}���7��9-�	
�oR�y4Z��-��,hE ;YA���fQ�Q�i?�=��ZAF�	�s�
�p�-�������V�u2�����}�l~��57X�����>�O�3���QL�k��k�C�WQ
�?l�G���Yx�3����p��?��4&H,Z��j�������;��OC=d#��u��FY�]��J�j�2���F82���&�m��	g�����S��O��6S��A
���~��nU�
��P�����������I}�^�b�X<N���_6R���wua��lj�_�@P����#`"��!���[����r�D��[��%j�ms����HSt2������J���[����?����:�N�
(�O��~`9��@*��av���M�m� ���Z���r�����q��);��L���>����v@��4d�����z��J��rc��
Q����R���~�v���|i�^��6�o����]�FPW����R��Ap�pN0(��GQFs���S�(9 ��(?/kH�����<a5��d����[����)�fm�a��b�������9\�.v4uJ�����"/�`���2���b����`���!�.�L��@��'�?c.!oG�L�����
����"G���h��k��d;

�^�[
���Hd��)U����c��@5HHN��s���
��2]��#����Z�0�>��W-*���w���"�����V��sE�j�`)V�T;�p��z�����������v�vV������>�qU�a�I���M�@�,�-��s��oIe�j{�Q��������P�l����zt,����Od��.%�����8�����)�n��Mc��\!����1�����Mr�'�usR��T����	���a�V�K}l�i�w7��R6�9�:�����O?x���K�F��M���C��#/E���*�\��|���`{���ew,�h�JF@Vp��,S��@���P"�`m�EWw����,����������F��]&RZ��p�4�O����:����z����	�my4�0�����L��_���O���`"�����S��G�7Nj�Z��:�KW\���<�����.Gqv�l����O�i
p_k��o��@�0\v_U�0o�1|;F�K��L���L����5�>�l��9��/������RE��U��PA�����;�Kv�B4�t��/���3<~�gSi�@��byb�j��	�?:�����p�Q�^U��g��sb3�}��L?1S��B������2�*wrNK>�2���Z������������Z��-�JM���r��V��DbP�1mR+�Qr�.�e]m��K����*U,
1_�\�����gE�3!+A8{� �4%�J��nE4�GWg����E������r���`� �D\lL�]nKY�1�� ���wMF"�X)���x~��p���p@�YCl���7���	�!����V���	u|��
w����H��������7�.Rf��6���6�Q����&��a/��v��y��
]���PGm�)�LR�q]�*w�Z���`��\'��j���S~;������&;;�v��>��2��<�R���z����Y(�O����Bi���Hr�����
h�-�� ��������HZB����	��I��:��4�e4���{O�9�FD���c^�Z�:K%�)�����	Nv ,X��jBVq�m��oxn��#G��j�j�J�8�j�������XK>�C~�^�{W�bf�n����:`�N��,Ng�.�3�0_���$���0�%|�q&L����$=�^Ifp��@(������%��C6�(��c��!�,����6�es+M�����C|^�������j(����������z��0>�9����JTj0P��%C�}��Y,���Z#��wW�a9�RU>B]i�&TI����U�_������hu#�yC64A�9����������YR��q!����4�7�����m���i{N��q�<p����${� �Ll��^��\�y��^bW���������	Vp~+H�GH���V.�w'�1��YW��tn6��w�
���m�NyU���?�Vzb�^��T���z�7���e5����)��7k���l����U8����7J6 _]{�s]f�W	w��s��\��B���^
'nEe�a���vy�]��Wk�R8Y�T=<T�r5������)���/�=����F�[�n������1�����o����z7����!�(yt�6m�q_�g�%��0
�����^����.]-).]����`r��EF�%��i.��[,�A����pA~l<����.���{dA}��>,��66�]���P�d]�����a��o~4�y`�'
���\��( ���MQ/�[i�>A���s^�������,�����?���j���;l�	���������{-;^�0'��R����9Qi5��A��
'�$1.�	�X�CC��V�7�(g��5�,90`���9�b�g�����b�����M�k��Aq�,��Bu�y���
�kn�FCM���y�����BbJ��������%����[�Q�p*���r:{�u�	D��2�����������14����R�s�5���K$6�/��Y��E��l�������n��(m�@��F�q�#�Y��B���%3:�[j�����W��?�[�0:L+���*,��������+}������������}�X�0]�-��$�!���+��:����S��-�vi�I0Y�����L��YB��u�
�]m������2���P��N.2O)7ZK��-W�2�K��[�N���ug�������C)�����8��
�����������n�
�pgA���b�q��U���D�,�I��h��f�H����+QV)��)?/�]�z�,�s��N-E/��UE�����~
l���z$��w��uaZiy*;pS2��H�4�3U��E��S�A���3�qy�~{��.��y�e��:'������h&��x���J@���MI
Jd�(<a�/���F]{Wo9�^}��WJ�+!{��]�9�MFe���d�D�v}������8,�9�I!���Y/�_��K�z�M����U��C��r�P?����r{_<�U�a:F��.O;M������o�awS8%��Ax�Z�m���5�$HlrV�
.��ed6 ���~v��fu9��Gfmq�P�+������$lD��g��qF,������FHX�mGX��@��� ���������`U�p��+�W%��z
6���UX�Lo�(&���UY�	�W/n�j����k�Y��ut����N����?�����bG�4�f��r
��kR�~�zUg��$�����n?�q��,�k&&,R"H�hUe�'`���@��,|9���.�
_��E�x
`G^4
&�f��R���H{��M���-W�]c�w�)��!|�[��U���^e�,P��M���,�ND6{ZChT�������?��~����w�~������^���#H��r������$���u�p�P@�+���JW��g�J������#:��Zs�BY	P��#R,���z���aS)
Jm�I�.�E%����$�[%�:��lH���'�/����u���;c�5j���(��t	�B���=
�A����J��j�<��z��g�|S-�Pc=�
H�������!���lI��!=�P��lVPZ��	��{�;z��&x��,�E�/���S���1w������*���<�4k"���^]��Y��#D"��a�pr�!���=+���l��$G�����u�]��j�2��+� GLK�!��AC`r
�9�������	��w~;�,7(�#q��r����7wh��%GI�-��b�����.g��o�M����`0����jfy��C/�E�M���N[�����EX,�X�kr�������������G	7^�I���zk���b��(]Z>�S�6/*��#J���	&_� �����'�N�yp�e^jw�T'hc���
����3X:\�R�WE
&��?!E%{e���/���	n	����,�y�a0Ae�qL�Z8����4����4F|�9�p���Ui���/H{�h`����_����-��l�L�jw��@�)���=�������PA�g@���X�l��WWlVK)y��Rv��~���)��)*5�V�DT(���{��W�`�����"�r����ic8k&<�pN���,������{�(�7h�4�(��kX���!��E2��6���]o=�L���)���Y>�h	
��<��Q���O�p�@^�q}4U_6@1��,@D�HL�����:�������4��ME�g����.��l��pQ�����^��/�n]U����7'F?����8wW�0�����%���nj���h��j���+`��5�>��I��g*��$uJ�5�����'g�6���l(�&���~W��n���.�p�p6�]/�%�X'�>X��t<�Xv�0�J=���`iZ��`	����>Q��`};�`������_S
��uh���qg��,���d���[����m8Y�.�?X�u�E+M �J=5����xf�H6�3s���]<k�[������9��VV2�Y�v��0h�h�aR�1V����R�,o���7Ra_j3�w��8�"e?�gM�:]Y����q�I�l@oD��� �oWS?U8��I����o�����:�����ql[�"��z]=���*��
���'��_�Y
��Y�9�Q��b�lrU��j�j\
��y���Q>a���s��,�S
2��K��(67��F�Z%p0e��&eL�Lt
����.�e+�LF�u/��������!�[�K��;�
r��qd����h:~E"]4��a��Q��+�����H9X9��(b
���z�G����D��Z0d�(���]j��w�:B�Mb������9�^��9}����I�(L�I���S,�3���9�o��Y�a���Y6'��IK��+6a�:WMH!�S�_��>����n8�A2�eG'?f�qkG��P3�8�]%��Xs��0��]��%=4w�������`���\�G��i�[�� ���v�����ZQ��8MI�M���<�>���=�b�.O�P� ����B��-��l��nAZ}acYb��r�|+9}��X�q��
l1���� I2�N��(���c\9q�o�A���sl*���?�_��uy�D���#���h��d�g�j#}��� 6k[�p$#U�#ay�N�F�8�������(�d&��.$����{��a��(�!���Pt�d>�8�$�/���I�g�v��D�/)\'���/�}:�I��Np��"�5���$b�<��L�C���� OS<���l6
$�\�t<Fl�F�8X5v���s����>��#�l���[Ls$w�A��]'���[�d5q����1��[��D�8��["��8���0��9�)�+|N��QYp��W�g�8e�a{p2�RKje\�(��]�����������c��
���������'X���7��B����{
Y��I�2�@#���=����r���!{��Zdk�?�<�E�-([�\\B69zi�n��sc������;�WIt��o���q��J��x��RzP�r$���<n������UI�T��3�cH��;E7;�QgVq27@If�
����>j.
�t<�$np��.J�37�`� �	'�~�f���P��?�1��G'M���N����(�����_Q�4��mr����$��=��b�������!�����A-5��'�*�+�����5�
7{����`�Z4����w�^���i���K�-��)8fV�$��?���������y�d�)E�

���b}�#���3�������[����uP�����K���\��D�z�i��#���b�����?DC�dI��6Y����i����'�[���5I[02�hI0��b�Y��ttHGA�U{���ld����
�hb)�1�	������'�>��C;|�����Q"�	��0���H���!VuhK���e�*\�0[&�����"{���4���%2�b�x�Z����F�F������B����_A|��n����
G@��������r�U�u7 Dk��F�ra�}aWk�����
�gw���k.�E�,���M~
LU6�o�,#���<�����nJ�l��!Q�j~*2���(G6(� ��o��!���p�}T*�,�<��cv���!��9%�)X���l@j�
-�+S-D�R�<�
/Z7���?K"���o([i����"X�>�����?�����-��^�{{K��������,Z���U��$�(")� ��>A��:�%�2�z�����!x���v���ov����1J����^�����Z��=�S�����5V.?,��RAY�n���I�_A��A"O3�!������!c�'�&`C?���t��+��Z\j�p<�#��lU���rQw��dGI�O2��UG�[$��z�.>���l8F",�L��DH���a�#q�!P�Nu��A�����@ ����DB��h�p��(��<��������X���� ��pk:��Y4������L���8I\~��t���������OP����n/�9_�wI�G�OB�]U]2D�Y����t���s.�NR�D�Z#Z�ZI|�����J�}��\:l���W����Lj���+�5�Q�.�gI6�
3fs�2y�R\C�,��~�NS�I���RJ�C��	��ien���5�6d���AG*�Mt*�E��H�LTR����Y@6V���Mu'�ot8\}0�ix�T/Z+o���vA�,Z�2��NT�w��7�R����j�,6�(�d�.��W��
!WFYl�d�[g$>���s�}�jc;�wO^n�4�'���	r�o_���8���9��!&������lP��;.?�Qa�p�k!���}}�Y��m�b��p��� \�NcPK���b��:J���b�
"'r��lj�� ��\�(|�'$�2>*�g��#��Iu�N�0w|������������VZG�
��`�����^�VW4"bF�����t�Q�WD�E:���0��E�3;}J1�';�>y��!FCR�����TJ�X�]O�Mc!cOx*��z�7�*n�������Si�����c��t��m�MP�_42�0��e�^��/��Qk��6Q���Sx7������j��R�i)	��4�]�C�����,����Z��r#�M��OY���</1������G���3��;
|��\��\�*��H�
M��`����B��Go����Bu�6�����J�nk�MA�.J��bK
`��j���UI�]
G-�#@X��5mm`<�\��\�o�a��U��f����6�H��D��������p
�%	���J��!�������T���~:�~�E�}��\4��44�G�� �o*�����U���,An��[��&�$���i�/*Tqr��2K�N���j�y%V�������d=������nM�B�J����DGFv�����]��H��1��
��5X�$Lp����F>�8&CF7��R�q"
cZ���a���o������u���[�U���:Lr�0�U�CC�	��vT�s������F�J6f�Q�A.���Y�y�^����%����D�n�%�	=��L����fvH�Ku����5�TL����YC�\������ycs/QL?�gj;��OGU�0���H���^��=��N4Q�/���@o�������x3���o7��N�����z(���!�����F1����oA���D�����G�m�/�FcV�R[*�5�_M�.���>-`�.�OW��FEY������2m���Ye���`��`m�I��SH#f���2�~Y����z"|lD�2m���V\[#B?a;��F9T\�:3!>$�_��k�J�=W��a��T�E�^$����ObA$�+kQnt7bF{S���\�a5��(��.w3_4\�����|��p�K��a���%7ra�J���? ���<���)������
�.h������4r^3GP�;�����P���F��7?��o9c��1���\��@��� g�X�ou2
�,YoD���f�y2d;}Yd�6����IGOl����'�5EN��#���hL��{s���/��#M�I�i���!���&����i"+aJlb����v�����Q]'�D�L<����-�`{�c4�V����� ��#{F�)u(�1����i����r����:��`�i��:
���ry��� ����Nm�G����a� ��a�{to�Bu8����S�&���7A��T�#/���������M�L�4&��3p3�.���iBD������h�tUB�U���������x�sTv�E���^_����0�S,�]"
F�/<;e�DJ�hY��������Cmo��]_�d����)�[�����B������/
�8�<P;/[v��4����Y�:[L��/��7�]O%����<G��~m
,�v���OZW<����	����+I ��p���T"YuF1hG��OR|�o}�c�3�l}���s�@)^�����������>b�J�$bA��2.U������L��)K�(�-������d����J���D^����	�	(����{i����6��������6�^��&U���:����:�����g���=>�`��`������Wk"��������%�!9 S�Y����=����_�xp�5	"�<IS@2a��Y��,H%h%���D��,$�jE43U�#QK�P��
n�Q�_�Y�72Btb@'m�\"/�����vGF����pd�r*;�n*�L��J1�jL�����|*�B���d0��'���1��4�,�;�S�a�F���a8�H���E2�:���JY�|�,Ja8�w�
�!�;�q��A8�U�U?���L�B�@opM�z�������������T>N���b���<�L��p9pg�t�!>�8�
�\�f����9v#CHN'[����`���!DZ�a����Z��D|!X#hP�(;XI|
�sSs
�F%��wB��$��g���6Kg�T�,��R�Gndi �h������0�!K��=7�3��!Z�bCx���,E��Z��?q����@�`NJ+�����+���N&aSD: @��&E��#��E'���$_+%r2������4����4 �}Yl��>���,��u(�PKR���25��
=�:y6sBxa�8�Y����P���L��v�drY�6����	a�,Z���&���/�S�����U:��?@����
z!����0�3:dJ\`�U�-X;ah{������s�
H!w.%V�����\,1�W}_*�8�
�Yw�&�zp���%�b����j�[�V�c���yj�2v[��hm��TO-a�f�&�M�� F$�������x�X<��L&(��QHpc�:eCFW��>wOw��9�y���l��1�Iz������W�k�~���KD��S��)]E�y�������mO��QH�9����<t�N���K9�I����T`9����9�g�l�~�����#B�f��d��\PUi���whB���E������G���!��j\��������9�*s�*]a�1drn�8��1��b��R��Q�i����E�{���'��5g��L��k�]�#�4���j���jw���������y���d�N�YJ���'�i]�/M!&���U���#2z����6@���~����d����Q��}A5R���F�	����2=H�ia������+��b��7��l@Ro���BTQ�dm��{�mP�[��
p�d;5���Mn7

�����p��}�
H*��2�<��������{I{XpM��Y����x��%���tK'�C@s~?4!\��	v��!<7���3��(e��y(�/����_��?��XS�$����?������o~z�����o��~/#��S�'�)����?�%�9��q�u]�B+�*�\��}����2^m'yP&'��B�At��!.<:��
���`
2�%v����+��=��!�����j`���������fSU[Dy4)e���
BDB�`kX�5c����i���1q��+k��o�?��Z7��*;��z(29�O0��������]>������t0�pW��!�����0��o���/����t4n�7`�_�u�(`"<;��#>0q��kUT8��;S<a�#h�����25f�Q�xW�f�����~���$�UH9��'�_��l�=���
:1�������kTR�	�Ao���\�L�sj��'���U3$���G�
��B�C���'�T�%QS�����zI<(��=R%'Q]x[(?@�'VT�$+#�tv5�`��:�8Org��fe\��6�SU�1�����)p,�.��	�~t
�c�����N�!p�L��Gd&1���"�R�j]�2���zZEW���y�q��|-F��'�$����s�&J�L��gB�)OY�x�2w���M'+D�\8/�H)�x��g8��+oW���_	�,��
�V5�� Tu��F
8�$J`e C�Y.�A�o�FC��/(���E�[�#�[,	��1�F�\�]��Z�j����zY4�zbw�fp��;��r����,':�c^�G�u�

	�K����7O��qS`3�5����j}#����?
�r��m���5��R������X�����f���Ls�3O]��fy���v��'$����������j����?+Q<'�����b����H��wg��#_������X��U��G
�[��N���������h�.o
���D���U��b�<�`��u5���%�N��_��,��UU��n�5c5`���r���@��I���bNHU"���.�n��D��	�Cxo�(?JQ�����{�{�n��&�dE���v��g>��w�^t�H=������
���)�����N����q&^���I[����B|�Lk�'!�d`�Bi��mB
�^
#�������3]���c�q	q��I���v��~y��,���"-�h��C	���h%���w���M����}���'���ai���q�(r$Z�������	_�Uz����&�a����.0f����P_5�"��y�>\sQ?�����o�>;X���J�8�D��O��X�=g���p��������N;��s��O�%��'7��b���<�F�T,�|���cE�SvK"Q�$��O�]��3�i�
A���&�[
J&=�B�uA-����;�����q�YoIfo���Bs�`J��Z|F��Q�D���9?�rI�2�i��=N��'�[v|��GD�oZ�+$����i��q4���AR�-���2�l�4[����������`s��_	�=��)i����D:J��;Cq��������,�5�Y$�����T�U����s��
�����a9��A�U|WD����H�d 5��{F#��b���i�%lt���n��!��~�����S8��������0�y�B"l���D�]f��,�0���m�
j.���?Tk�������V��T�W=�_�qn�I~8C6�W{��T�B���	�L�m�fzAk�wM�zl�j��p�D�5�]�Z�W��N�	��:P|��S��������3BI�Vr�$�����k;�;-��<���DNg�Y�_��������_�����2c=�9h�����b���:��5z6� d���&���dy����Avy���Pv����&����������f����FH)6������-?��{�NT�s��zP���3!��z������j��,��Y�b��UnEZ��q;�EQ��UKsv�r!�_
�|��s�8Q�h1>������V:W\@�f�F0{�_�Gd�E�v�R��/R���a����9���4���2�'4�km�����������
�sRl�(U�}O �����4����
��Y��K6\��P_���0�������-Q�W#���!	�b���*ta�.��$��K��V������]p.��]������a�-���L��\y��O�5�Z���1�@�<��s^�	��Ln���"G��������M��B�i��������f�
�wuK��� �@�C���b(�r[�Hu�����;F>�
w���w�U��Z��e�Z�	�K��`S�I���I�HP"H���46?�d�
Yr�H56l������\a�}��
�O2�J���@�]�D�k��DM��_
.{��Z�������d]-���/<�������=!�'�3c�'0d��4����6h��5&2lyOT��f�(���XT�����Z~QE-�����_PQ��gV��T����<����1���s
E��'��h���8o
:���LQK�5&<����Z"T���(j�����)<]OK��b4����2�	���#��TOC�O��m��1%���H|��O��7���2PCJ[W�G��K��
B����j`'D���\�����~��6�)g�����5wz3�=���w��pW����<�hejTI})���J��'+eL3�2�3?K����	�f���L���?JZ3���iM��!�2)��E���s@/����N�����.fm�"`�8>����0!GqD��R&F�|i�F�q+hE�_*f�=����]{�!i�-.�F
����"aZR@ed-<9l�I���F�kx���,v��Om�].����I�G�+�A���4�c@%��|���W�MBt���.��?�0:�bS��d����94
�l��5�jH�����������.8K3�=�_�"4_���5(����!2������������IPF�jH�%e��Fi!f���-D��Gh!��P(�8`%K[�`H���������u���3b���-t��y��t�c%�5��[����n����zN0
�Swr��di��~65&�,�d}������]�^./b��G)��=���J+{��_I�h�p�)���J�w�x�����a��g�n��������0���N,�u����>���c������������|�2�2���K�f���o�v�b��?l�_����G�G�6[Ik
5��bWFy�s�n��>��r�Q��!��%iK��rB}y���u�Ib@3Qd��Ju��U�A�N��(�����y����P{p��q�C����aMJ��_�u�q��~�{���C�87��X<.���/Uw	G����2�j��������v5NUs�����"@�Q��>���>�g�{��`LY3�l�~dY�P�M���R`1��=����Lg{�����[`��jh�v~s__�`��������������������#�h�A�8��/�S�X��8*��s�� n�0��J��6���,��������{�O1�qN��W�yJ� I�ES�����iA�q&�E�����������ZS��4�>����)L��g"�����������yH�q�yAY�R{��B�P������?�������}`�����m��5�c���	����p?����\O�~������	�h�"��+���7�
i=�'��pO����zN�I��z���������������o����}ZW�kn~I]q����U��@&�G&�����_�aN����lU=z�C�V��2�?�vwO�d����9t�"D[�*��Y(�F���o���Q���$FM=o�f���R1�����A��_�><���q!����p���e�$�c�i��p�w���{�����M�)XWB�L���%�^�����2�!�Z_����J����
���m�w#��q�	���'��z7���8�q��	�
4��3��^��8����L� H=��FK9�K�I�u��_c2��OvF������E<�?|�����{[{q��,EO�n/	�~p�:�B��>f����Or
��%{�K��H$�w��%���������U�Y�)s@�9��W��=l>�����j��o�9/� �j(�d�M��o�F1�z(�
�8���S�K��2�vN=G������\D
y�����$5���T6������42	��P$�	��)�B2�m?]~�A���������S����}y/|/kn{_�qw,���8�><���D7��e)��X�J{h�GE�V�s���y�����s�yC;	wE�4@=�b��[e�P����n�m����i�a�;�4��{R�d�{<���U���~���h���/���=�B�~���N��]!*���D?}	��a$3��H������D��C���<����wH�RYY�'��=�"�
{u��#�����eBX�����+�P�w�.���#$�{�h�{� ���1f&�R-�P&[MHca^������;�U�s������A�=���u~E2[���q��Xo�����8q[��!����!?�����;
��{J�'�>,�i
Q	���=�	-�2��������Sz��]W%������W��m�:$ �[r��@�2���F��-1'���L+R���;WOv��%Z�\���_��B�U�pu����M�"�s�����)		NE#�Kx@���`/�/���1:�2mh���o��e}�0����J���^6m���;2'X{o|P��*���|���2�dV?B#�C�8"#MKM������9������g�����u�=�c
��sO:4@��q�������:�8�b[O�2��2��i�N����IZm{�D���A<L��:Z��S;���zQ��M=(��������VnV������������&����6��-q��P}D=I�����L������A����e3r q�I(����`�F�	!�qa�	(�����
2����9tZ3�C��K=,n�Z4;F�5DZ�xVC�����'�[!����Av@n�.���Y@�nB��.�n�U��#�ze�A��hXM��DW"r����@��V6�#��k\,��z.9�p�?�`DJH#���o�L��V����v#����x���E���F��b���F�?H�a��Y�[/��+�:F��<v%�3��p�n�T�([�Jen7��T��.��|�E�-^y�����
Q�v�&$���+UG+9�_�\h�l<F����%������'�������[.���������A�l�������c���n����z�e.�i#�1v���D���t��Q���x�%4AN��D������;4�1c!�0�"�@��B�����,�%p}�Mi��cg�#*kv�{�c�=t]�@�-EaE�
�������(��~�o�Vd�!a�;&e�qAEP(-�V�rC�����\�����>��/R���
U�*6���q���z�SE�#����NB\=&��A�}�y��R�WEd���?�k�����X������{���X�[��5�(v�,����B`L�[sI�X-���n��9���6P�0QJi�����!������Km,����.w�y�T���v�
����Z]~�q4�����Oo�^���7��&W�u+T�R?��7���[�yF����;K�e�]on��(V�q-���.�sA�,e�kc�<4��av�g�#���V�?o����^���"3F��H��Y_U�q/��`6	e���@����M�1�	��k�����Q����������N�C3�_4�%�Y�9��9�Ql��1��\��
��y��U	u��5�,6O�>����X5G��R����C2�_�-(��CS��qY��_^��'�Y���.����:���q��}d�D,��1;;�����lI���������R�������Bv��Q��h8B�.-{����w�����1��uT��&r��M�Q�3��d	�4��='S�A���!��E�������������Me�j����.%�\%�w�d�T�-G����\0j����X���Tl����7���
Ix�JKW	+e�c
�=h
P3�H���BY��([U����(M��m�tM��C���z��\tL$��3�� A������B��3��u��C���cM�mN&<����ftj�4+_��I�V�x�L�"��Z:R���F�Z���4�ss�(���d
���	�����4�pT��3�+5J����kl���V���s���7JMYb.�lD�wt����B���T�Vs\>�2��M�[1�Oj]`�����������������5J�5]����`'@�<��
@N�-4�m=���~){)�,+C�����?����d���+���X���~~��?������?����w���W���W���4P|t4����L1_�b)gCES��~��z�x��|��l���r|��������e`����k�v���8rU�,�x&���g�Tv��l��RG�3z?V��������F{:���
Ic�d��G�\�Dca��zXK�p>�����-��!���Z��� JPn"��B7_������U�����_�����������l@������W�n�Tlu�v�W�s�7��
�r^g��$���?�\dC,�!c����c�^��uT�?,��&�J����O�)��Om46�m��9��y��8p�'_���(��@��l��?!(�5��^r��3� �N|�u��}*�P�h�2�=�c��C]�>����R������
R3|���b}W~�Y�^o�|���RXvzl�����y
���\�Xu��`�,��u�{xPfZ}��5��s5540���cE�(�S!d��R��-C>���L�����a�T��RV�
�W�2<����@�����O)���[`$����W������d�����~~�h��_���A�RC;���g�Rec,��)�qH��������Ph�0�u2�6�!�'Qj�0�}�Z��=v|:�����B����N��FBu�;�MRn���L�����q$������9�}��$�y���M���U_�oB�`�w_���_+��w�j�P���E�(o����8��iF"|�o�2������ G�56��v\��8��XJs5��#G&biQGa>�1q>Pt�!4�����v���B�P�!���g�jt�Fb�R�T�������j���o��29{y��	�� ),B����Q�q�37�Ix�!oC�F�L��x�#�7fw@-�V�H������,kee���:���W�?e���X�{
�2`�".4���W�>�>��x����_����4ho���Q��W�1�������z�q��w�]��
Q����(j�"�u�qb�k��p)�>������ �GdP���I�����������!Jb0����
�y�
c R�@���3���R���hy7O�|����(D�e�p���j�RvN2_tN��_=����`�8k@n�*�-��^�Jv:5iv{���jZ����P��?�e��c�cA����"C+�q�C�����3����Z�?���&�m+� �Q2UP`V�����J���R�>zTG~W,e@q��������z1xC�m�q�\��c����'
�_�D������ ,q������_��������3e�_�(L�P/�N����a�^��i��X�+�MFZ��}�d�J�%!�@�~e}0+�K+���n���WS��� 2�c
����:��}Ep&E_ �	!�0�� ��s��?�� =|����;�#B� t�\�fx	D����y�,_0-�g�E���aW��>Q�>ce����D�&���M�^\2
8���Q��P�$X�v����J&��&��L�������L������2NQ�'c��t���G����GF���}M�}���z p���(A*�$�@&������@lE�u���`����������"$��a'������0�P��!
�P
���M0��r�wPZ���R��$�I�3h&���7�3dC�4����I$�}K���/8QDyB�1�h��z[�L��Id������p:�%��h��-�@�(O��U��7P����UKtr�xM>���E���T���v-�>��#�*(=V�����/\yl���ir����G*
i��4��p��8�ZMD���������elo{=��u�i%.[��m����DZ/�<�`D�By��E��S�m����q5�"I�����4&�p�5�z��I�l�i�d@��j�����zh���i[$��-���������{@��i� !H���
�8�
��o�
�4HR��^�
N2��oe:N=���d����kl�)�aM�v'��]K��I������	�]0�-t�`��dh��l��IL6�3�<
�\�[5-�~�ih}���d�%7-1D���<6����N��s���@���-�-��`���&M��8����+�+��v}:c8���[~.����v�N��z��P�����w1�Xd��3��zMu@b��>����BR�������	&S'�0�F
W	;�jW��
���9��M�1
����������5,V�(��hw,�!��R����%�h��1��������98i���|�3{�L���c��u��Q'x��,B6  h`�7?
w1��h ���,�9�8\/��R~5�iO�����tXE
q����M86DuL��?�������5���		>����^7@��H;�[g�G�J�h������K}7�-+���o`�����������p���/�od3GeP�v����u������BJ
��e��F�B	�{�F���b���7�Vg��~y��U ;n�(C�[	�q7c��o*�J����7
<�Ry�A�����yK���p
dt�R���;��L��',��(�o��P��Z�X����ttF��F��R��3
C1�#v�K������W���L~�4�V���T��4tz����'��D������!�&Gy�y<G��o�!t���S������5��QHk����]��	�Lua�	��G��5�L�4�����9����[�; ���a�n��O��?g�HO(8������w�k��^�Jb����QQ������K��6����,���+W��z�)�3,�v�Z����ch$�&(���n�p�(��j�-�?�v����~�4q�2���i~�4I�zl"M�L��&yC��V>��rm6�q]p^X���I9}���TjSq"G��W�y~�����O=���dr����4��drt�6��G�J}����������}Jx�\�rf"�~(����)fG�D�DN���t�D9�f�A��������(#4;�����~��a&���b�h
�Y�U�h��2����m0�N�:%u���`&����#���{�e�Lib�46NsFs�K��y�	�n�8Nr�����$w	>�-�/�B\*�&y~��f�����n��O����<U�l�^Rg3W����,��
h�P�\�pf�4���f�<��:�N,���=
a�fr~G=x��~[��z�Aw��%��R����R�)�]+�NO7���7^x���EY�k�E�8)�����l�iR,f2)�	�I�\ �bv��R�a+G��Lw��(���~�0�t�c����	� �3��Na���j���L��4������&�C;��
���<f��o���X�an�e�uY�48�����i�4����W�#��L��BZ;S�O-�4���
�
h:=A'*�H��vm66 �%���G���b"'n�����3���EB�j&���ER�j5�������I�x�k��������LvK����Y,��fq(�e:{��xT;�w2Ak,��Y������d��|\��7}� ��U�cRI��I��~����I1M�g��^�%�QRL�t��#�4���RLS��#������O.����z"�b�}�\����z��Mai a�;o����",�"l�d�(��&����Z_����Rg����^��j�/��L,��\$�|l�e���}��L��,����LNp6=f/2d���R �4�.K�������iRI�K�_*e�QRI�t�"#��R>l#�4e�H����������K�p�8��r���"Q�N.���'=�d�|�S�|�]R[?��SN�F�W��}� ��m�'���?AV���q$.I��CSpI��8	:�E2�!:���I����������.���3�Y�,;���g�^%4f7J�poa{?���8�f����~�Nt$2�^P���.����<�a�i�#�)
JG9�rOu�Hd:���S�`��_M��P�
0�S�����I�d]�2Y�N�&?Ke�������qR.H'$kN��� ;���TJv��P���i�j�N�L���	�dy�%�):���	K<f�t���jc1P&�mD��������������������hv�l��vG�dQN9N�����1�����'�m�,]/��'5 ���pX*�`���XM�n�*6�4��:'�kL��(�#�)���T������Y����&��q3����w�2#�X�Rb�!i �'�0?�R-eI%y�I%������������������(�
���[���:�:C�&����.��
y��������S;���)�*��[n�jh�Y�s,���cB6����������h_@���_@�����icl�R�vI���>�����j�7�<��b'����)3Ag4�\=a���e�Z�������Z�#����s����a+�b5��lH�!<����Zt����! ������rYk5ty[�}�� �
�e��m���@��V%���6��-����f���R��S�D�I����9|�u�[�WH���LAL���)���)6\�^$��HJ���ay���a�Dg�
���h����x_�.�m�*2
~k	@�a�����L�N����9-��:cC
x���N��vx�*��T�-�#�T-��J�S�f�Q�k�O����QQf���ecUvKn���h���.8��[b}��WV�R6((���������Z�#[������Y>v��2o����%����E�]��;��$[��>��OL��sa�D������iyNiJ5�^��vUU�m� #��%�����k�AL2-�@haxb���dO�&����W�����[�J������[�Q�B�\"`B6��K}{�l6�"-���q�9>�_������%�h]��3 �H�d�"$%v�,A�5�h����D�Qdb7=��
8a���\M��(�V=��?E/L�b��1
�w�����Ke7)��.������������U�p������R��3�����E�Dm���D�<	���c6����)3m�;��f�7��Pi�v`{�����b�CZ�RT��q�k�����W����W������785���k��l���y��}��5t6fs]=*x���e��S��N��d�RJW�����xCHn�R4�t�t���%��-�\l�������Y����oWOd���V�?l������`������Z��]���&x�<�vS���n�%A7����}���.�C�A-����T�{����������*:��eG��G(;����F��4�`�(��'���x}i��z4h<a
xI�tt��&�aKsS���+lew2t(
vR
�������~u�^�����?�~wn�u��Z�����}��5���Qx����\Dj�����[B�C�O�� 	���rE�[��C�Cu���g��������1Z��s�"�FD��u�c��a�_V��R	���d�x��N�U5�2�,�>�������DJ$K�T�R8�s��`�R85�s�x0o�����|SnK�����Vc:��r"�Z���]=�0�g095���)�d���1cC	|��:����$���(b��x���bnI��D.�\/�����x�e�1�xV����zj���<�A�u�������rf����5%e�0nxfy�!d�;!
����n����|U���>�@�z��/R��\������S�L�� (p�)�A�u�8R�J��u���
�H��6l*q��I���C�<�p�q����"���$�z���;��`�����)�&�;�7���?�8��A���4d���a�j�}e����� {��<�����1���2=�W�z�z�^u����c����[D�D^���=��
GZ�9�|3W`O����a�������Z����U�3W��~�N��:p���-�	�]|���H�J=D�r�QI�~�
�`�sE����]��3��Z@o��31J`������,�:/�F
�|``%.��W�I���v�Z�Z��x���^� ���)��2������7`)���6�d��bt���!)��M	0�?��%�h�dC�N{��F�9���$��rWs�8Tf�z��:WM���Y��F�}��
e�Y�����_7�p~�GA�GQ��i�3�x8
���?p|����_���B)��&>�=R��g��,~�������m����W�
��$P���(�Xp�*���L�:����1E�J�>���?��3G�u���>���[���_��-��U��46������1�o<=�p�X���<�9���C<���@���t\�������#�i����A��#&&�42t��|�L#
8�����JI}#���?���~�xs�z�7��<o�r��q�eT�/�����T�'��'��F ��~����cp���<+
b�P�t��-s`��ZGq��\
���:�Ec�����c���zg��)WS�o�Jiiu��V��n��������5_!��
.Y��m�9�����J]��6`���L+[�>5fT��Z5+���Y���iq���H!��7�5{����������N�*��8�>T��v��z69�TO����BM <\c�1�lU�(+[U��V�s��U�d�l���"J��~8X����ohy�eq�����~��������~!���}?
����^���������4~?�O������9��(}6�e����	#���p��'T������"8�����?V���d�������$������������;~][e�K~���Q�(tv��"-�R�FW����x)��+���-&�A�X��'B�8�$�EB�8k8*��y���X������`������|����5�7%�'Q�lL+ESD���KH!iShq	!��M��d,*�9���B\��������P�uAe���VE�/��j;�pt��7D��DG��,���'z�,�h�dE�&K)>��)M�����S��ojT�M�7e�#�M�?��d>av����:�"ND�������������{��/�{�_$qm�	<Pv���URh(f}�c�Q�X�'v�O��n�����)Ks@�L�OY�W�h/�!��@^7�����4���O+zK��B�}�d�`�g���	\���?-���es�!9�rn��y�:��0Gm��b�,V����%�O��X5�����f�9��L!f��r����\���(W=',�UO6���nd��1�
����;���FR��D�`&��UOf��#���~�����#�o##�?�1��}n����x���q�����
�q�E��}9�b�����S6&��?����m��/��gv�)-H�.S\���joY�%c���WC|l��,�Sg��,�[g��,���Gy�� j���(����'����H;5�c�/Y4r�'�����'����k�����4t�3�`�{I�]������zn�w	w;�����%C.~���~�z9���4�� �&:��0��\
��t�R�G9��(8��D��%�	�C����%��D����9��k0�<�s)��/�\2_d�9sz��Y��T9�g�z� �1��;Ok+�~4~*�t��"����)��j)S�b*�����Q�r��CU�K�����3�{�a��V��V�H��=�w��jw��8��O*3d��q-B�k9�m1��T�eH'$��1�]@I����Cu��/�6�a���j��@`�L�B*��_B���r����R��H)�\o�%j�D�5�o!�J0�#�`#�nc/U5����C
wS,�QS����~!��S*f��1��BU��3���?�o��u ��w�[x��Xo����%a�J�MF��Du�t�iO�����_a^���O)AMN.x�D�e����	�uU��;�d���L+9�kt������rOb����iC{@xNuO��s�{�����vO��"������6~�������O#I�	{��\7�����5r���dq>������gr*l��?7����������5k���~i3��aS�R"���	��O�d�*%�k���bQn���C���d�W��({2g������g�����|Y��@� �����������x��z5�{����$�����FC�lF�f�u�e�9}31�Q��W7_G��7]�gZAw�T/X���<�[�o�y���_L�v�z�
�-��+��l���{>�oi?��_L"��T��qP��4[��xV�j
��i8q���.d��C��F�c�\6�6��~�=)Y�W�(����0iT#���j��7r]K=�A�o�-�����j�7�Q��(�:�n�p x~��0�6Nj�{px�)� �j�e}�z��s�ZP|K��������G��2����N����xB�kU���~m'l��2�y�h
�T8;X8��
���p�RR��--���
�(��T�	gdB�4�-8����m1��RD����L|�R �G�8��d
XKNrZ����j��(����[+^wUs�	z����D#�3=�2�5���.�c���'d�&���7�"�C!��C��M��B[���}��#�|,���kD��[~.q�vZ������9�����.���|����m��Z���*nO������)��O�'��T�LY��[l�����^HDP����Vd���I���j��ku:Q'�Ge�=K�r�����g>^[(�`�v)��QP��������D4����bunt���,��B(��f`�`�o����_��,�Q�`	��i������(��BO����p��g!
��{u��[�2F���������64�X4�w��
���K8W���_����=C!�
'�rv���c
!Fy;���'4���/�,���L����w�5o�/M@;�O����L�����6��O��pO�Y��2�����y����kK��=>]��KB(�.{8���$��'8
�'B����i1�{�����x�u��I	!���|��\kY�+0����oI����`=(�D)T�nd��4\��X�
�@A&�4�:��l(� 5��y�42k�p���`��h��E�^b0�X��,���.�cz	dcsj�cV���tm��9����{'�+E��'�.[8[s��5�zP�-H[s�'1S������C����������$�$��������P�=�4P���-D4]���i���l�j%A��_�qd�����|����lH������T\�`����&�+^8G�!�@qH��
���tk��Z��V�N`�����M�����F�
�MB�����"M��N�Wi�/��zD���B�aKh�!�pz��K�,�qA�]���o���i-����3|����<�������g�zT�)�RMTO��r���pq�'EC��'�#��l^�F)f��!#}XtB���>�E%�h��L����Co�`-FS�da��j���56/�C��
�E����bAi��7��M�����m�I
��������������8}������D�]�w
���
�n��$Lg�SZ}� X�'���1!L�+w�+H� ����}"sb�+4���b_����l4I�f�|sB�+LsY�2hx��J�����Ntvo�o
�,�}�W����jyH	����nXE���������F'h��d� s������l���v.��]
e��;���'4lW1|���Z��"���S�3��7>�_����]_���U����/���;(��	��<���|l�h�X`c�?6�����I������c�
����7�k��1_2�gJ@m�?.n��a453T��j��x�]Yb�7���Sm[��1��v)�U:��F�X6�%
G3q�p4�����?"�n��W+�Zy,��Z~�T�b�A�mDfd4���bd�L��������Pm�+��\���D3b����V�����tYh#�����0�%��)�#���i�6���@"G�������P6W��7�w�������F���"����T��f'�����%�S������y��/=q9�e�"����%S���#���g��+A���g�����|�X��,�Q+�c��Z.�N|2z�g���������f1�{��5\y*^e��86:%��ra�E�l���1\Fc���)1��7��kp
W���gS�N�D�����j�YW��U�>��c�;���
��C]]o�]��
��������'��Fc���L���8���>�f���
D�v��X�z�����h��A����^����4�����4������h?%!�& �(�9�`�A?�R� /T=�R�����4+?L�T�A����iE���4C?�I!��h"1�3t�A�D�P�2^��w��
1*@�H��6p~��E��L �~,M�	<|�����R|F�=C�	@TvYCI�>�b�L��{R�D-��uB����k=;��e#�"�F.��D�PUP�Ax���9V]n�$�G��{T>>'��;KH<�5�K�q z�[���*Q$�G�����C�>WZ�0���hF��+�hF�L���/��<�������s�w3�ue�7-�?�CK�3J��Q��	n�(v�Y�;���~�����������j�^�@����aA���fKQ�r4����n,���`����`���@�S�0��%��(��\W�)�')���������<�������Hd�.�{y�7��
�t6��_�����*�UGVD���_	�"�4;>S�z�����������
����"����������������(5%�H���>�2�a#���Q&�p���uy����Yi�N�6_Dh+H���]\�o��y��z��$�&`�:D*���$�pm�� Rs�Gv�� �g��r[��Eh�:zz�`/i�X��]���<��^�_�a9��k�����������8v
�>��=��V\&�P�����!�cI�L��=�'����)Z9��)D�����)�����X����DE��	����]��b��^�q���(�a�N�tS�/t�U��l�W��X.���u����e-��rR���ch��	�I4��jJ���<�%���r<I�-���)��9�Ly_k8:��c���\[�>8�&���v/�F��C���M���^ bK��VO&���+�)b{��J1��B�QL���.Wz��Y#��xg�D�j9_nQ�q��feO�����9���;1��j5������������jy���5���5sVjh�[C�.��]�E
�&��`3��^�P"���p�2X|��7 oy���(&RU��W����SA	�����{&��b�p
'����p
'��b{��9'f�W��W�yJD*w������h�d�o�T�]�v���4 ;����F�a5v�� ������M�G0*����� ��y����
���:�#O������X�U���#<V���V�#<V����� ��) +r��b��
{��>��;��d�X��x�`��1��>q��#��T;(�CL0��k����ar�K�����.$t��{�U�wT��
���o����Z?Eg�jv�C1����S�Y���6��{�)�������D���}y�K�/9Tf�%�D�=����~5�gR���P����r�[K���q�T�����P.��A1V����[�������@��a��*��3|�8�gT���$jv�L�������
eR���������y����A�Wb^F���U�32?����e-�X�'c)��cK��^�">	�z7l�e��-���c�n9%NsS�� ��@���h�@��i�}8<�3~�Ms|dvI��!���1{6���w��q!~���v8�kH&u��&r:l�8�"�vz��c�iJ���t$�\�vJ�Awk��N��l(�����s�0�y66/G��?$���g�7���s�9;����	L����0���v���?	?v�t#
�7(��sv���09�M8�=v�h�|F�*:��r}��pc�/�d���RD��Z�����F\v	c�B�rY�B����B�#B$�k�m0Qz�"n��:����A7AQ�#h�'���
�f_HL~
�}��'X�bi&3_fi&3W��~.Z��,jX����/68l�xT��g�������ZwmUQ�)7iL(���uN����|
:��A�T���0�[,o�2�Z&�F���~
��DW(�
�i��],��(%1<�8q.y�\����#pJF|_����1Na�y����Efj��S���/M}O|�,j�h��������4�l8
���4(q���Tj[z�]��6��i9m���?��X�S�K ];�M{�p�g ���E
������
$�J
J����j��'Y�I�L}�[�����2�;�4�$p�4�����������5��;.��p�E����z����D�0�"�^�p�n�s�)
g��v��K������ �������9����Yn������x��!��\�������\L���P,����Z8cY��bzB� �r�S,��C�]�p��6�����ho�B`���)����R��!5HJgW�.�)�.o�T\���A�H�����rg>O���V�dm��H�.�?r��J�L]�Y����;6M��
 _�?{7��+�ae�}�g�Cv�c���'�nB���(�e��I,��L�f�fqw'���i��ww��E���Dl;&�n;&�l;��I��	�j�������v�dww���o�z9������>���d��f���r;4�>�%>�;u��$��|��uB��'�^'�	�NY�J�J-es�!@G1�d�'}��!s���m�f�d�*�����RgB`1����]��6�����H�<��H6q4��Qd9��cy$e�=����HH��$8hY������K7.��%Gl���R]]�rQ
�]��]����������T��7o��<��Z�K>�V�0a��<K�R���E��y�q�m��>�(������"�J �I��#�Y"�/6����4a[����B�B_Z�����
k_5��Z�.uo7�EF��xw��@a�oq�
��FA�����T|�����������2?=PX#��x`��Yo�X�?�<�<��:}��i��d;z�l���Q�X}V�����/�9{��`�"<n�����.�P������3�=}�JF����i�L��a������*�6��M W

 �*0�)��/�E�	8�B���)B����B�Ar8�
m��c�����s�W���	V]������<gt��i{�?=���0�=�7��O��0��FtEh���k�;����0�?�eq���Vf�	��8~������:;���|q��G��/O�.��Ni0���g�����������g��E�X��	�*&�XQ�B����n4)`����K$��;��	�x}X2�Z�ic��f���X'��C����N��-d�UY����B�d7�K>�������d(�--Z������2�D�dPdA�y&���;A��S�I��R�Q�G����8�7������f����o�����7z�1���!�L���L��SlR��6���E���y�@4 �z��BQB/�Eaj��&!w���O	Ooh5E=0E��y>��t[%�������:f��:
��>��p7��]�����}4��L��� ���Hw7z�-����F�h �X���4�����u�>~�l'����v�P��w�v�	wV}����i�tTY0wH%De�R]�K�f7������_�v�LA���r w�_��}�e���g���a�H����/�o,��r��Je��� �T�"|6��7����������^8��l=���~Mz�"��"+�df@�hY��n�{��0!�q�i�3�&#���E'"L��+�d!8��$��{ta�����ax��&2N�7�����kq���G_e��DZ�Z��*����
�0l�
`5OJ�7"U�PGXIA��E�`OO� ��uq������h���C�B�������F#��e�	���k�F8��2a��J�f�u����9�#��c����::f��2�}4�W6x����xp��7!���z^Sj���3�16p���C���
�X�h������������-�)D�lCCL�14?�c�E�����d��zU^�J��������B�����G�_o��x5$pq���}�b`�Rj�[+~/J�Zj��U�4������t�e��h$�D������,�U�:���V���"�Jq-�������Of�,��t�� k+�a7
p~�c
��~��]�����+������I�&L�����s6.���Y8,�������vo%@�%��p7���po1��	��l�������b�nl,i�E�0BWH�<!�=v�Yv"��~=���llR�l�zs��+���e�y��U�m��0�mds��G��2Q�@Oo�U7�zy���f�l&���w��-��*@t��H4"d�m�����"L��/#L���M��~����"��so��P�p��z>@N��7(m���	��V�u�v���d����p����^ol5�b���'�1'\a���B9����/���$_/(A��9{)/�t��\qE��u5���&2�i�r5�^��!_��s,�f��Dl�������ln�������$c?n�Ad��'j8���P;���%��{�5���^��iG�I ��}�$���i�sI�5�@~���u�1�i=v>k'������E�)�R��S�*�J��!>9������-����oo�]�R�T.�j}����Z������&q����L7�|�I2��mT������M�������52CEC9n0�����7p��qCD���..�&��s�o���GFTR�>�>�b��j=n�L�4V���'�����j�U���E6}h�&|R���B�Y+z�&�Q��*z������������s�}d��G���F��_p��G��_p�����<�2$�8����)\��O7��g^�z�G�&35A��pj�W����/�E��:�W��
:�5[��|9���`��\�'�p��*��S#�H�*8�GGf��x?���b�)�I�;�E1_���]�bm��5^/��Q�����,��7!���,qo`�^v�����+����PX
���#%+�"����/�;u�G2���a�h�����_�#@�=��	Zo]��������;C��?��
��;3������@9��V��`|��!�ty��*j�e�(N�c�F|��q����-�#�D�'h�(n$[����Uy��
�<�G�������#AV�vo�]F�q�L�g�O���vTD�I�2�5�{y~���$�M����2(�G���i��W����H���$d��+�N�p}�D�^��w�b���)Z���9�~���-�c���;j]���@�t�fN�^;��D18_��&m���MG��V��*���:>�YO�T	�^[���F;�JE���T�]v{c�iA���v��L;�]Y�����i�5��>[L��!�iA��i���J������i��_Y7���WM��=��u���\;Z�wj�������!zo����^n��qx5�F:�S�o}�UH��<�M��G��%���������?��:�x{z��pI�>��)��LJ����l�2��M��X���M��Osh.C�f������3�u�Q"�B��H���r�����Khi�a�N��J�M&���!���,
���-
��
c��f��o�)j�Q�X)G��9�B0��849��??�����W�F�n�L�G�����LC��D���>���u��1�GL�X(/�k[���3�hH[j2���F��d�+e��#o�sFOxK��s�x��	����t�(Su�.(����g�I%���L(��yJ����T~���������B��.�,!�gE����C@]�WK%�7�HF��p�?X���b�,����D�B��sN��g
��>Y�}��2�A
X�W�=���i�$�����Q�|����������.s`yk��Im`�|��dJ�GA�R	���eSQ��_d�$W�G�a�TIRMO���#���KL����H�0�y���'����E��f,�t��'�D�Q�d`�GE �qo>?��7����b�3#(M�o
8���w�F0B���M��w4�������q$5���nVEZ��F��h4���s��������
.��?NN?^�0�2�[=�|]k���~Mk��O6��|j`�t�.��U�su#g����X���k�*��Oa'����]�������I��d���B8�� ���Z�\a���Yr{����_��H���Q���/�������H
���>�c�N�2�b����(��pC�j(\U5G	Q(����zh���=�%�G�������R�xI�����[1T���#�W�G����<�5"k�z�;������,-��`�Rok�f��J������|j)�"|5��j�V
��S@���i�~�e(UL8.�����La����)|��z6�\	_w��WGA�7�7������w����"�T��uW2�G��k���C��:�"���) d�@�_�I��}����&m
������,�L�c|�����b|���oO��q������G����L<�$��r��N�*\�S��$��d��X2Xg�^6�������1o���M�:������o4�'6c ��.K@�+|�qju^�E^4����L���\��^]�p �BP
�7W)@��5����[�8�R��s��f������gOa�w��'�Hz���tTr|��xV�!��E�w����N.��W78��Q�8�x6>�GVR��h�S��n�.�EY��;
DA��9a��z����J'�U�}N�%���L�<�%������5HSX�9.������0��d���
�R��d�Pq�����$}�2�� �D�f�s,)���m=
�4r_~i�L��- ���@���D�x2���hN��:R�����x4�����
)�F��lQuW�����$�3���jB��B�7`"�
���w���9����������F��>��������D�&���J
FU
`gcNT�@[/��U=��j�+��"#�Qv�`�Qe�9�u%����`z�+���f�O3�0��N�;�-�#vj~m�
���p����|2���-6�'����O_3���
��2�"�����ru-�Q(�}���j������������6Zp)��E����
���!�R�O��s���>y�.r��~��En��^���f9�&�����l g(hP�.�<�9��5c��
�p+�M�{����J� ����`2F�d3����:�nM��eG��	�J�����-U+m��V���'P��
�mP('�Xg��3�J${��T�����M�v�U�l������d���>r�}��p�HiL��x\��+��s2����Uz�O2�*
������Mk�D�p<S�)�� ��c'�vt�B�	l:�F�X��Q$���'�)*��\�.y����o9}<T�K�I�u_�����>��h;���1?��Q�G����h�����p�.xkH�$xkp���h[|���~
l33���
��p�E	��MnG���}e.�Z�"�)��8�&���������H�!e()�[���]BFEu�����3�=E]�\%�'i����uQ�`����
�it�S\�H�,���U��W!�A��F���
�.$EON���������l��w�bVTg�_�Ue��"� `hAI"����fJ��n�rr�r
'�:(����H�;�c�����*f�8um�H�2����F.#n_���;�o2�g��|t8������X�����-�;�Q~(��|�w��c����� ��rU*�w�}|�������o�_��~{�����Td�A �k����u��b�M�'��j��i����L�+2�Zc�6%�i��j>�K���N�f��'iQ��g6����dv���^��+�;L�/�������D��TB��N�j�Y5\�^��*�eZ�2�0�3girC.�B�Z��0��gAx�����X~���dE��@	���q}�r��Z&u���d�"Q���G���4���}���2����Ff��:h���V��	vc!=������L:bs46�t��Z�����E&����t�I��jA
��bqr��������������o��c�3_hGQ������������I���e��%��:9��������� �t0��+�lBi=
�h|�+�eWp�f��+<2)�w��Yn���d>�S��'����3�p!<�D=Cv�b��"���B����f�&���57��05s��7^�:���8�B�As���v�8��r�e��^V�GX���EQ�=����@6���$�zHh��,���
��x��S(���ux
Q�^�>���Rn��h���W��W�^n_�8���
���o�t8�����/�b��fzu�����GV�Gp�����n"��t�>G*�������+�R9�'U���;#j�^3�fTUD�U�E��pp��"�'�����^���#����c�?��8^����L����{���%�`�����RWno6�D�dz��X��a�e�z�*_+{���|��������$��HIW�����{�#2;�ele�����'�����r�u���.�#�w��Q����T�d���jo��X���]�b��rz�W��#�����(*2LA��4'A���3���U��}����S|���ex�4M���Bl�
~8K?�3*���PT�����[d����@m������n`F�GX��.�&i�&��1���>Y��D�
*T�����Z����I:����#�L�����3gk�9��K��WR��U�g�{�����fUc#���<��h��.�Q�j�|\�f�I Z�X�1��1wh�M�\t�&���W-����)
Q��gw��cc��I��N'�tG�y�uh�'����2����i�Vh;9p���+*����S��T�M���z������?��3M������?��VV��yb����*_�����u�t	���Uw�b���:�� XJyl2���y-��Nsxt����.Y.S0����J��j�"��!2^V�|��?�M�Q�����Oxg%JZW7�5Md��8����V`
]��Gk���yD�) -��fx%%3�m�/�����E�_��J��L�A4���S����	g���������B@
��e��8��'B�-��i����������2m��:qK?G����hW�v-�E
�]�0�i����X]�7�0Z&��������yH��N��C*|� ���m���y��`$��;Ao�~�z��T���l��b��>��>��g��]�>���TTz�	�e����f:[�*� I�8�a���z�"q<uESv�����`�XQ)uBVeP|�g+3}Z�EPe�/��^������0y�����`�<����+K�Qm?d�b3C�KK�t8p�s�xh[������_q����D��3
8�����M�@��F�~�}N�uk�9��>��)jY�M�{�c�a�0"8�U~���a�8-���c*V�b��tdh�=��7	���dH�.b'�����Jb��>�^���{~.����d��~���(���S#���GH<B��N��������?1�(3�����5��S�������P)�����{2ko_�[�u�����!���$���_�n��)u��,�^�LCz��e@�+�-S?�c�����	��e����!kA��Q�K<����l�)���x�w����>����8<��
�/!0w�;�!� �H��L=�K
����Y�S��c�nUe�b�����/�L&�����=�'"���Q��0{�~����g�>*Lp/�E���������;:��^���;J��:]=���ds0�B;�2�6V5���V,��4[B������$YH�2�H�Vum���;����7�1Tm�r�����Na���������� ��j����U>���S{�]�$�E�����L���iU�_k�
����2��PVp�E:���y�(!p���������tA%����a�����.�9(W��L�-�@�x^;{��>��0�0��+����M��� ��[���X�XG�1.���uz
P����s�+X�M��_�������~L���R 1~����i��7QOf�|	�����(i���j�(AsQ��!��n������v���E<����NJ��=�;D!���~��cv�X�dE�I�}i�W��0o)Cz��{��p������-[F�������tR\aa��L�u��33j��^ZG�����*9C�-�������T���h@�$����jduvf�	��
���O{��������y/C��C�����Y�
B��Q�Yj�c[��l��ss9������r���:�c4I��<���Z�.�:}���a.no�-��]�o]i��+��u{{��s�H�t1�T��<&��u1[vo�Xi.�%*}����C����c�-�WF8W��q)a����BjI5�k,������k��r ��LW�l��JY�����}�5�tQ�W����8�Ti�`]EM�W�%:U����i�c��
k�bV��F������IB��Ri@��H�#T��b���6������-����"��$q���;`��ip�u�X/�TBr@�[��)2[���4�R��E�N�~O9�A��r�J&�R���3A�}��)��j��N��x?�e�Qj�`���j	�9���^��D�tIa'w�h{IA�2����J�}���35cT��|a�/�xN�3�a��Q���r����$�C��<��HI�j6F�{�#)Z~E������Ce=�p^��.OV��+�:N�*��{8��z�P�����G����C�����`�i2�!�H��nI<����ou0�._#�
]];�"�eS��
����
x2�z0���9HEJ����[u���-W*|��H�f���]��"�(J@��-�C4P�4�W��j�<-�T�o���8+�J T������_����vuz����������g��Zc�}���R�zq������y��m �W��~�=�����q�po�TMb�<������B�Bg|�m���MC48p[��%���w|f�c�6>����c�������u�W�����������{]-	eT#v�g����>T�����@� �=5A�k�r
����Lp�F���]�X@�e��RH
�4��cx�����na~�7�5&�}Fe���u82�{E)� �*)aAz"��9��]�8����D��F��%���	Jw�����n�x���%�F�Fr3K�CO�<�������V��i��9I���3+�����Gt�{#n����k��<�d���a�I����?r�ojd��n+�t\G��
�$�M���b�-a$���]�U���
�<F��V�@Wr�X�t��\����g��Y<pj��SV���/i�sO�v���/`n��I)k!`��8��U�1���,�Gyf���&������r���d�
��N�1������Z@��,�:<g:5�d�����~�S?P0Z=K�WGB�_h��j	�����J�:��%Z�F��AKL�N���-�B7=�M�J#@H�mS�N���2-����o'����~�oKi''�@��m=�8Mo��L]�[�0��l{�j�>�E�-�"��QS����S
�N�z.���������a��O����~F��~��SJF�E�s���t����M��U�(f	��K�"_��x	Ys5�W2|X�Q���s2n��	�RM6�B7u�*�r1�7)Z6C4,�0zd'�N�:���U �oZ.eX}m����~���$~����
���w���:����g�Jt����2S�0M���^��H�T�{X��Xh\C��0��{
"���%]��>/�k�R4^/30Z�%T}$[X��A�������� n}���	�i���hK�)Q4 ]�m������X�P�?����:�G��n�l�Z��_�O�?���k�p��X���-���d����oi0/���@
�&-�
�k��gu��}a��x��S�<l8�P��$��^u�UEY�>�9Xf�����sE1���#6_�v���p��n��Q��=���C.��q=l�����0_���}2��
b�����
������m��+k3�������%Z����j�,�{&��Y����j�_��������r�]q���8B�����@������t�/�m��vb�#���t����&��0����h;��g��*���1�ce%O���u^`}��#	PN��-0&A��e��3Pw����
��9��l�=��1^8�>�%&�b�Z����u��V�A@ IK������2�1����������1DQ���M3��`��dq[�fIg��)_Xc��M��
�����	�9��69�l�cwc�)��k���5�}���P�k�?x�5��
^t���?�F6m�pV�xj�
Gc��q5$�1�Q����T�w���g�����N�f>��	�L>�}Z��0i�����Y��5�D�(s�#����w�n"k[@X��	'?J�C6��@";�O���dW�sn���S9���GF��_�w���=�bDm�������?V�cj��F��i�.�<p�B�c�M�($��B$������yb���~=��8��v	�B�wks�fto��>�z�+p��z.��t���;^!�M���_H��������z�"����%�q:Y��.��T(j6�+�l|�k�����#�r��w�G$$�`��+��!���0D7�fo���M�H� �������!eZ	���>?��B��?+?�����n^6�z�B�N����d��S�:�g~������s�������tW=�W��* ����D7>~YX��bY�����<�HH=�I�(���fS:j��G�=���0����:x�dVY����Q��X�����rdXLdU��n�����t��VS�������e����
�}�?z�K�U<T���Z@�;u{e���
G����pX/o���\4n���N�-����r�	������n��T���B�'p������_]2 |�vAx>+H�q�v����b���A81��x+��R����}$#�Hh�����_Z���
<w�����R�L��+)+�r�@)f�2����j{�\e������*o>$���i�������Y�n
`��P�	Yk=W�d]��\�Y����"
��D�T��Euc'�Q�&8@=Hu��zG�i<��Yo�B�<�S�7��	��4�P�#�P������(������Q����N���>�w���e{.�.������`t8����]�^
�	�S�A=P��Z\!����
kd�
��v�.�P+S����v�:������)�<�����i �ddw{��h�_yO,�T�dRi��.]���*�1"#cr��o������������(�wf@������?�l�z�������;���������KOQH=>�
3����Q(�r��NGl@F!���������3��
���Y�0
���X�L�A�r�^�����Li�T��ycNx��Gxgs���d�>�Ew�<	���
op��^���������
�P�+��	���5]��r�`���0�[�����o��$������u�mF ��=>�
�_�h�&�N��d3%0��p
t�Zv���.�&�v�'\~G�������L������4�5@��?�W���;#w������^�\�@[a��A|��[�]6Y�R��
I�f]V��F���M�z6�Z�����D K�%�C@�@��_��N~����Q�������������A��
��Ag [#pT�@x��&|������X4Q��	Wq���w����+��x���p����Wq��K>m����W1/�cl�y���;�c��8zL�|�%@������w���W��i�^Y��	��e����B���������s}�0c������A�\\��;F2�����������PX/E1�X�%h{�j�������8�W{S�y�A�E�H��������;���zz�^Q~g��^m���K�w�P�[���B�����VN@��G[��~='@ D��5n4��qe�X���x},�1��XD8�UN19�n5#��yD�~�Z�y��i�M����p��g;p��'%�f����*Z� 2	�]V��!��?�aT��gd�Iv(��(7�aT!�m/��c8��~�p���3P��]t�\M���d�L,W��\���VG2�^:� ����^o��kc�;��
�y6�1����e��p6.���o!���f�
��%�nr\
�nWV&D�S�o��k�)�l"ms�P��z��T����;kh]��j��f��+����>���t�m�����Z8����J��k�s�����5|c(��T/���]��'VC�!z��8!�J�1��
`&��m�_+��s�#rf��rf���
���V�T�v�X-S�|[g&����.����L6@�u�3���g&[�;6K���0�������-q�����i�Q>@���`�)	��7i\@QA&
��Z�`�!Z���a�@8����%�!�	��"a(	��H�.~-� �a
��"���ib��xLPo��MPM�{<@����)�XE^o��a"�	�kB��K=%��0�:�����r����.}{-*��k
���;J�2+Cr����Q�h�EZ(QM���,�7$�p/��]�?��pn^<l��,6�����2T������:C�+
Zil+(n����/�sL���q�32��q;Y"���4�f����Hg�����������z�J'��kv��CqC*���X���|a5+��L,+%�#D���i���w�R V�?��:j�u�.��")��N�a��� ��Z�a���b������!<1�<=FK�3ov�/�j���r�W�
ds/{��@�z>����Z_YW����+7�C�n	��@�+��>y������a��o������"p��T���,��s���
|F�=���I��y�����^���r4��U�W�cm��J-;F3j��y�j��L��n�!��R�wS��Q�(�B��Lp������ �)h��������u�e�����G�����D�N
W�V�z��4�"��5(+�o���L��e,G��������t����c��;(�	��Y����Ld�u	��,^Y��A��x/�r#pA�f*Y�������ZBAv������)u������!S������/��K�
Te=�8�����J�[���-��f
���B���e2�������B�0R�c���y�hp����,=��m:~�@�Xv�J
@�y�.a����;F�jK�sj�J�9����6P��+�L���z�8=
&N/lo�AG?T�;��K�u0�yR�e��2�t���T�P�b���N���B��^t���)���a�|���k&��l��L�m�hu������j�G�H~��k�3k�%G���"l��+�����7x�=���~�z��Z�����}j�p�T�T���R�W���i�_�����}y6p��-�
@��M?�F}�>�V\�j]�P5j^��v�����a���	���7E�Y�<=<�
�k��	t%��|�y���:S�vuDnWt��"��
�`������X�a�H��SU4���/�q�N�g��H1�t4 ���$}����L�	\y]�����9"Ab=��e4��T���T��MK�A������n���("\Pi�k
R*��
�4��lwH}$Z`���0dIr����V�W�=���C+��
"����[��Q������}b���
D��9��>OQ�:���������)(����`��q��~�c����x��w�9�Q29
��A{��,Z��/��]=r�s��]����F���{����XM^_'�Oja^��2�g�LW��={��Iz�z�z5������gtt��W���D����dy�x�v��t2���&e�zy�H_�X/��@���U~�.�����a�~y�\?2��5P�$v
�a����"����/��%+��k�R0O�r��][�dH|�4�����c�������i������8���'��v�^�����,�Nf�[��
�?�y,e������J����o��J�\Y�m�N�Yv�����c�B�#�����������wg'�Wg�d/:D$Rv#���
4��<�?���^=�[������vI�V���V������jH�]S����9����>h�)��:A�u������hH
�y��Tb0n7�U���:A|N�s�k�P�g4����W6�4�V���.����u��BJ�NtT���X��l�v�j�
%�L��"���I���K�)i����=��F
>��"sG�tm�s4Q=�78���������|�����sr�h�g�|{%-���c�C�V�?��`^<,�d��6\��hf\������x�l%�g�J���C!���g�y���g9l��0��sC�H�T}N�
�H���/DC����+t=mMfy��a�l|-<9��s"4~��)|JI���*��_��o�E
{�K$|;0o+>������U�!���/g���jh��F���R�	+.Vl:�+\2�������0_��|�am��7���������t[i�������������m��R�~��C�-��jv�������0[�+w�*�W
���V�J���H<����g���T�(�NVe]�0�WvJ�)Z�Kd�����e�d-nh���!8��>�S���Z x�D(Q��-\/�l��}}h+�����
,'
uyX�I)��r-g�<)�
Z��{����dY��*�����P��������V���Z$���(�����t�������R�JIXezD�\���y^�K�+�P��yB����Y����m�4`j�����QNS���zR��+�t���lQh����df���CA��z��^�bT>-�clPm]Xn�\4	�		�_tB^GG���K6��d����-�:�g�}��5��'����\C�W7=�v�4����1p��J.i'_���k!F��H4�q��.���>�~J���������)�;�,�}9�����4}DG{������
1G�9B�0�{f�*:T�A4v=��:m��e$��=4�D>	^��o���FYwDp�����jYU<{�`G!�����C[EO
�/"�S�^�^k�A*������"�d#v�}#�������F`s���)2C�P��~+$��W��Y@����.}��DD8!NW�4��!�>L��CNV�<��{:��-�/��+#�b
B;^4<���I��E6�P;%=}_��
������Q��������F�����k�[D[z�����XIw$leQ}}7N?'��T��hn�&z�)�h�Mp������k�I'��dp�(�MS�H�u���G��l��6�'��R��y�o\�5@/�1��)0�A�T����r��>�j�)����q:_���X_��C �)h��]Zc�L`��e�R�mQb3|�<b�x�X�3^%�����Z����
�@����rK;��jg���"���tB��m��n<���6�Qs��+�A{C�������7�3���z�.y�����Q$�`M���mJ�9�-�^N�Vnh����7�n^4 �"0���2�h��T\|�@&!Sul��~�������S�A�A������. ,�X�i%�����t�E��X�i9V������>��+����qlFU�P2�y>�*%�>�����NQ��������92����R��v(���W��r���'�(~t��8	�1�� BII������b��k��4�H��	Z6%�r��%g=�i$�zIi���D�l>O��"����w|�=!�����.P#��t)��yT��X!F{�Fd;��n}�k����t��q�EJ�E��e���0nZhPlU0|��
)C'����'�����v�9��ScJ�����1mc�������a��i#%l�(JT�f���@f1ZZu�+��p���Gus:[<�S�do@kc�!�{&�C2lh��w��F���R�I����;o�"TZ���z#�l�����{��'��g���$���K�
O�R�f���Ur?S��9�4k(Wj��=���=�6���Y��`�6Y�a�����H����3j\}��sm��Z�mO��e����}�������zzC���.��`Qh�V��% ^�����k=��)�F@�����DC�:�uVwC<������H�A�!��������Tx}C�������ob_f���Pz����P%��� �i��H�Td����K �����IqT:�F������$R&s%���Y\��#����)��x1���>���'�E|CgVd������=�D#c�'�X��_]�.��)#s����p'3�����-�XE�����9����I��0��=��w�0���'�'�K����_���|-p2F��1n<�{Bn��,��
�&���O�n�0C�d0E����lce��M�4Zaq$��;�n�����G}���MEU���*���Z@�'�m����9_�X��#����O����[����R�uX���oKgK�)�j?�t1�^��\x
u�h�3��.c���-��������(F��'�l���g9D�
$�������r3�n����S����_lk�
H��y��Uq��B�1���c���fy���`U���z��^GX�K�� [��X��N6�	�������EBa���������M)�AA�I[�e�Q;`�lkr �c#��)�n�Rx��s��b�������G����e��!�J�����J���"��R�>��7��Y���R��C�a���Y`�5��j��x@?��������T��{���ql*�/�f���E���Qz��I��!5�>�
�a2��������u2�8=����t|����?_�^Z�����~������~�p"�4!�}8y��������'����{zq���Q#���d��DgX.���F���j�zd��$����&���L���~k�@H��HKy���&���`=iK[��O`�]\v���l��	{�+���DG�uq�6������k���e��rP����a����`��	B?�;�
&|���hyT�i��~Q�es<!uyR/�C���� r����p��Z`{&�����2�R����G����	K�o���s��4b�'�x��u2|�U�04��`��Zu����1��KL
)�Y:G�,��au�F�f����A`:JVG���s4*Kkp�����rf���5���ok�����x��~p��
�.9e���p�t�<��R���)���~�/�;����U�L��/_�+�PV���)�L���B��Z�a�09���C:S�!�"}�|��c8�b�?�c	����s<�@2|���Tr>#�������Y����o�s���/��bn�-�%+*&	�Z��V�uV�U;a'�J)n�9"��bL�MYI)3h���n�T\�����l�N2��.u���M��D.P;
B�������-taL8��E�&s%T�X��E1�e>�&����J�R6)�r-���R����f^H���|f�u������HY�gJ^�X>�t�R�j76�o!��.���^�B�
�/�����\�u	q�	T#<D�]���
\��t�kp�^�����1DB<��
Pm�HC��N0|:M������B=�\��5�'�uY��C9n���.��m�pk����I4L��v�4��K\���D�a
�L��^'?x�F�lp�]F���/��0'30g���7Q��d#�J7x��9L�L�z,w6�����F�����Q���lF�m�_^��~��\p��R�T�Sm(&j�����z�^
(�'�7A_��;������������8�7�f>�l	%{�Ym�������W��r��_�������"<�������Cu),5)(����|�.R(�e�eR8���.�����Q���}�$j�)�:������,�.r�Iz�t������J���G�,#�lBRb�ur�N>����RaM��*y�}����	�����������6AK��V�rc�����vW@V�����o&��4��H�2d|LtG���c��~��cL���s3Kn����(Tc>)�X_��0$S]���������j����$��Rl��������k��*�b_�a,ei2>6�3d�mJ���Yt����7j�^q}�c�������z;���%�t*u�+���������7�bBpMo|U�����
���Q���<�b����GDv�Z��"�����Kk�)+T����?[wu'�-��3u��tuo�Al���d�Qh��@�DI���(���h��9%��?�����Gf���������5R�b�F���hk���Yj����7o��<?�0~�j����Y�#���B`�r�x�n��\�]m�C���h�k)�X��,hOX�TC�	���]~���O�@#l�"�s��v e����p1���5$Q������t��'MH��5�$����D��T��&���N����3�2������cW�����
m��Jmn��4.]p����/����;>�t��,
.z@�gI�Ws���L�s��]i��qNp}�6���D����/eB���R�o���Vw�l���Q�����l�!������u]���=]���\�������QJ^�,�~8��~�o9�8)�}�g!2!|i�4�m�KkT�R�C���������/��U�:���R��F�(���h�f��
�����I������dN�=T_`����+SU|e9��2�hF��|O��F��J���)��k�i:�YR����DCr��m#�
R��c��t�����l�
�!�(Ou��qO*�<:t(��VJ�H)�o;(Wn��>�z�u}�5Z}tj�����	=�2��o����|�-E����-������
��3eF��]J���Ji�����S�`R�[��_j�)����.9�1�+�,r�`n\Q�iB��}�Z=P�7�i�E������#q"���G&�c:��k���|�C=4��~1���M=�?[�0Zy��
��]����'AV����z_�G"�z��E�l��������`�8���xn`oB�! �����r�AG�e����\��+�u�)���M�����b��L�j�>���@(���l7@�	��G��S'���p�B��>�e�F�6�r	��|_�6 ^p���lb��!D�h�Yn���i*Vk�D���������.�����@
�L]v�F^$�������_9�AH����|74��D��a�&���Pa�
QvA_�*UJa2��dhL�N��4����B�/bqBK� OP�����4(�@u���l^�A��e�f��D��q��"h/����V�g�q+����%�� ����a,%����C�����:�	��."����" 5�W�p���k
�"������+S�I�2�2~������v���CQ��������e�`[ae�������@Nt b���R#��Z�N�F���F_t.�����t-v����]}�*c!�����m2��qOw�k4�k�m�wO����E����FN���L���^EI�cX��&Kg���L�U�u��Ctw����/V�Mo$l��	���H�~����^�+��'�}g7%�.{#��|�q�U�6])�^��|$��?��/2t���P��f�;n�]��H����|pl�I��M�Y����`�B[��b�!��@�8����]A�Y�E��!#SE1�Rt�<����}L��l�S�������3X-�I��y��]��-c���g�>���U����-a�|�F�y���F�����G���$8y�L��U�����yX�<�����
��>�J>�����>��������������}^���7f�M����&�j�e����0r?�|�y6��s5R0�C���8y0�x=�	8a��\����F/6lK�~���`X� ���i�,v$�����%�"�W���V*��=�i�����2=
�
�$�sqM���
+Vd�\����#E����B}�G�z���K�{�������t�$|�����3�Iqe!P��C�dOS�F��8������y�_��Fn&
���X�yL�@7IU!�J3
����D�����A��kEx������w�
J������5���5�x���V�f&A^]_�9�)N��p�G�a��.X�(��*	7�^N(��G�AZ]&��E�!��3��M��R��mA�c�����<O#��i������aP��`�{�>]���,J���1q,sR�E����c�|�"�*���2���T����1g����/��BA29�K���	yU��7���~���/A�{�wF�v!1��A"�u��<�������u2;Q��K/��~�����9W���
������w�aoek������ZH{��i|O�ua�fL]/���-��@�M1
�|"��P�����+;,�8�qA=�|-��7���}�b�u��F�����x���a;Yu4'�)���T[$z�;?�.J���.JHY�t�\��'uQb%6c-��b7��*Y���n���������k�}Y�F��2�TP������
����3+�E��D��d���,D���^�����}|Xl�	}s$����&z�5��p��"z�����FIC|�4��d$o�B�"�
$9RR������Mv�sd�D��1����1�<�]E�(�n�_T{$�]��=����t���TeQ���X�Rt�b]
��gI6����U���������Hf��9��U���]�������H�������;C4=���8B|Q�he6kSC��koE�bD]��U��*	���t
J�q�x1*:-F�G=�����XBc�p�h=��#����k�!V�$8����PkJ_odB����&�2_LAoB�{%��B�#� %Rrb;m����M�Z�>��bJV�k�|Y~���o�X�D����
�K} "�2�h��Hh|��]�c��A���,W|�e��I��S�2%�n1LE��� �J�/�^�Qg�����������?$\�V�5�
��~�S��+&������x��#��}nDw��Rk}�����K��K���o�Y�������|������>���������[V*�V�5��
�X�j��:�|Rt�� e�LW��e6{
�&I�jb��%��e��.��j���~^%�;���d!
��7�&eR{q*��/^��Q }��*�]%�����E��G���"���"C����'�*W$����&P{[[I���T�����[��_Q�x���IJ�Wd�?���j�O^@����������]��{o�<�������%���~V3]��E��y��B��BKa�-2%�Y�r�I2��j.2!������%-�Nm��=�pu����������E��B�Md]���$��3^YY����������o�Di���oe/�"0,;����<)�^��({QVe�rF�Y�i�(9�Kg�����d9��A]f�iZ&�L����	gq�=���M����
 ����� P���&�/����k�3�~���aD0�)���b� w���pG���lq�����,���.PJR��b(N���������?���_J1�:���&�]����'�L�N����$�y&_�:��_�x
�z�\�N����}>�P�n( KA�� ��&�������i��������N���o��k�����!����*�W8}����,2���$k�����W1��)_%�������v%Q�^���������rR����w&��N����� ���`��mo`�,�d�W����n�|O�h��������Sy����G�z1tK����0���ur���������������]*qz�����k������������Q)��-/�N����g����}�����|�����w
�+ZugQ#E�l�i �`v��Ms�|�?���v[	:��8���H�%������;����i���&�~����|�����
,����'w�gs=�)�e��=�F�M�\�gb����M��z�;�q�z��\��^��TI3$�__�~�%���S�1�@�\���]���.�e�g���r����"���z[�)�y	%�d�o+��3�V�����P�.�>��k	�/��C��
�t��p���e���y�����I��i"|Y��r4�=0lnU�o]�n-�WyH���y���.��<O���h���S�w���E���������
�Z�7�,^1��^C}���AA_f��0����F���g�lI�M���:�}Nm8m�G++w�3R����=�g��I�ul��n0jr���+(���70]�`sN�������FD��M��f����@ \/{�y�s2S�<V�|XC@��#�'_6��������: �&uM6v��C$��2�26)�c������Z�
���o9���[����)����Tb��ea��w#������2�xLn���x������T4l!�f�h�^*���[�Fb| �M�����@��@����&H�����=Dq���N��������9����2�r�O���j0���% Zc|������\ViV�G���AV�O��Aa��~��)��/���*y�D���5�8I>��,_P)0��
Q��LWy��P �����c���#�������/�\5M�������C��y'���
�TP��j"�%<���G��Qh�8�j
�S���x�c���r3�:����2�����dR�1do�B��������)�6	��98���1����zy+�UQ�����.�1�A��~D�BX
��x�[�W%$��f��bcB�H��Ga���xa�M��K�����
�H���d���e�����z2fb���,�1��XU���I5���r�����UWG��%�����Q{�^�����p&����,=�� aKsO��,x�!�o�O>5�������Pw�����
e�2�lH9gv��a��	�+�9�:����N���q�C��4Y���O�T��t��RS(i�N%u��z,��B)j�ZoP�����o���{�p�^���bj;�u��I���bg���En�i-�1�Fs�I�SJ��&��p�/���0�i}���~���{3c�`�g�QX�e�	l�x��K���-]�����V�S����=�;=����t���j|�����W�o_ar���9/n���@���cg��,z��Z���BX��v����^�GTH�H��AT����l���/� A���d���c�`mof�C;�g�I����sZ�����)���V�H�e�L&i�;u'Wp[�����~����&�9�����,�����I�|Aih/��

������h�5����`k_`[���
����
OP-K�j�l�6�u��FzB\�E�v���m�����W�l��JJ<<<��H�m��,_�&��'w2���]g "]
���S��:����g�w��h�h{�(�*=����A�A�u���'�g���,�2��������
��8��d=�fCf��B��r��*�����C����Hy�qEa�
�9�6�B�~eW�g��w�:���A��:6���f���@V��9<�^�U�����%p��m(e�%��R,����oZB|�#�;}Y[�����<-_V~��1��b������;08�.r��(S�
��2h9D��2J�����X���jI`��������o!��D�w�l@'��l��c�&���"]����, ]���4	Twz_	lY
:8���%m���W����W�5���=NE�SS9A���vB5�<����L}�I�'�zmrD�X����
����Y�{�������*�|o6W�������J��i���i5�&�A2|�U�y������9��
YL�]��J$����}������k���C����W���vH-Q�'aa�#cWk�������E���NCb
��W�rH�F�� ��%Pb?O��[qg"@=Y�T���S&3�`Q;X��V��|A}�'���"����p���*�M������>W��!;�)���lM��F�<Z��Gh��^����^+(��`����|�-�d�?�����p}���/p�����\�B�S�V�(�����m�x��t���v�9���<b��^NnCZ*<m�1��~	t9�v���A�����M���U�E2.�]v��q�����(��g���������c_`c)����.E�R�(�c��&C��f���d�=�_���z���e( ��Y�k;�J}T�a��������M����L����G���7
%�����T�CI���U>oZs�i1Q�n.}F[%c":s}+14J)�
�R$JQNh ��{`��N������.�������S����\q��iU�bX�?`�:�J�zEr�b?rpO�]�J4W�� ��v�j�-U�j�E�������j2��C�Pmg��	5O�J�8��]��Lh0��+2T.��
����-�(G�*���~[�)���~�[e���S��������+�h�QpF��z�
��/��Y�5�2�{�|���sx�&J�J�E�� p\WaP�w�z+*��25�����hkR:��P,y��������)Z�sK
e�4!��CB"���t���U�������Le���C����|����o=}�����iVL�XTF�v "fh�	O���qET�)q{���<[�Wm��gw��Kn%�s6��J'�����gq
��e����������G���%�(��:�]���>�
0	��3�I�0w�%�v�_�kJ����,s��|38w��MMle�������(C;�X�r����~]��I�NO'�
a�"k9�"��KdA-����Y���b��E�����A��
������:��Z8V��~�Vk���v����V������l���_���Q�������k�0��	�_60�w��W���M2��P��(�:���G��8�
��9lX����
$������q�E>t����8�����A��be��d��~��i��O��
��&�"�����A���WP�;�o�����W�X����CrDn
�&��0��B���_= ���������i�����,#B*-��<�m�E
.��;�D���p�2�C�_c@to�L�&�f)e0F�(�*���x���_��iRq����I��7��%���c{����1X�����:lrp&3u���k_	!{u�E�����*��\�J���	fE7d�J9h���B�\h��rX�|��i��[�7@�/��,S���G���r�,�0�7Q�����R�l,O�;l�L��E�(Dc�<�=.���!Js���wi9��>�Sv+��(�hIB�I��L'�j~�m����������������21}�2?�����w��Ip��45�t�X��������.��q���j��0�=������Y�:?�gC����q���;y�������
c]����$����� xuf�B1�t���L�n��.� �S�������|�L�sD�� S�D�[O�a
�0[[�,h{{��}L��;�OZ4���W0�l=����d�C���.���������`����8u��~>��������+�j�tdbk$���~��9��J�;���.
����������T�<S��|��Q��80��,-�y
NTuHZ=`�cn�(A>�Z�����o���
�KP=2�����JNl:$��"��<A�*H��:�K>g9&&5f�]<2D�G������&`w��f��n~${�40���?��yZ���D]-�3��S���7(-�ANlZ����
�%E��#��er�^���_9��x�Kg:AA�2��xg0DG]�z��J��t��[��}���u���.6������L�R��0��������X71��#�>�u�tw5���%����d�I���X(��:��
�$�4�-3���i�S�4U"sd��	���Ki����Ion��, �Y��}$��t��;��Q��BW�q���Rar���mo�]
�&��%w��;�B~�����)�g�(f%|1��x]��_�6�e�o��N���X������v��������[���36�'m���B2P�!{W�+������nuuV�M������Y!��)`��`~�z!yLC�O&dP2����uL�`_��
��.8k�j���Hw���	�lvN�7�����|
A�� x�e{�x0�A����5T�����b` ���)��d���-R7\G�������'n�3��� *�zGS��eO�c(Gf��L�s]�����g��n�F��lf�z��R�y
�h	A�,�)�wMr[os>�t��o���[p���7����p����3����j}da�.���nz,��]Oy���SLK���NIlRp�-f�����c�(���e.�.C��eX=���.uym��,V]�T��q6Q��X�-�
B����cX�����,3���_C��Ft1W�a�W�����`E�vRR�=�[��)����\H���r�%����N��o�$@�XK���s80\���l�Ab|�i�����w��{��@X>j��
�	cmM���x��`l����J!p"O0C���Dz���U��e�k$l�1X\�!c�^*���*� A	�����V]�?��=��uQ���/g)������o!�N�ZK����QJ�������`{��j4���H�� ��v�T��y��W�}���W��kG����&��Y��<�d�m�e}(����d��su��7]��?�9���`�+�4���ZW�%����]��S�����8�e���ba��dVs�B��� ����C������xa��'������
e���=+��K(�I�x��N�@#3���g�c2�����:4=���/�tYj�����^������c=(`1BD�Y������Kn(�@g�
�n��y�Ls$�eO����g����������)���VF�j(�����%;�����������z�)��y���kE{{UU��X��N���������=�zn��r��<���J��b��nfY��6�@�ejw��v��r��!a���[.���F���4��h����#sP�k��;n<���7��2��I�v	S��_v�b��*��u�*�z!�q��^	����5�.�
%HI���PMe��In=����9`�l\_���P-.����Du��]/>-������P�qgE��]���5��X���z��$&do�1�O��#��=^V�`��"gRor�0�
wEx���#+��������,���I��A�G�1���$]C��uM9S�[��X�a�8�*d.#����ycJ���#`e�a��go���9<��J�u!�C�(�R���������x�l��Y��l��g����_����a|u���S���a���|��V�������)X�K��l���:�g�������j�z7���_U��1N��v���Q��s��;~-������{������������r5���\�
���Tdm������{������B<"'�l��	T��"�h=�9�����6���M>����&
Fp��-$�H,�08���-z�*n_�x	�3�����(��s��f=��_��t*R=��T��q�vm�������fe���[;G �[|[����
w{�o5gE�X��d��nb����'/��q���r�3��L�2_qK���Di�P)�=2T��&��{eY�J������%��B��&=��J���I�>z\hs����T�J��������arc�:��q��p<>���Dj��������*�n�������,��nD����"a� ��7����_������j�����|i��,$�5}h������+���_����e.��m ��D�%9�)�T�M��x�<j����&�	����U��'�������v��:`jM��#6�X/�l����
BA��N��b�P*=B��/�Js�K'eg>vwi,�gq��U~�d�����=@e?_����^��@��Z�'&:�~|����LM����jD=e��&4?p���+������k �����vO�	��{O����{��^ n8��p�����~���b�	0>!Y��_���>A7U��oan������u;��HcH�)�f73u���\�k��0d��3��9��S�(�^�3a$��7�p����g��F��|Y��
J����?_��_�
����#@G�wf�����'�}a���l?5����($m��e�
-�/&<p/�������fme�d�@i��w�:T��K����(}j�{s��']27�Z������`���1d��
���p��)�&���V��*]C��L���@��(���j:n�$��KY�}]^J����F6RUd����^sc$�G���KH��T������"����
2|D�h�����c�D:R��Y5rl8�p{�������+B��{
8C6��<�An�+t�f��F������n($�H@2.1"p��n�V�%��P��e�;$�	�#&��]v~y�l(�$7#�V:*���P9�lx#R�a�e����#����9q�{����A ���k6�tO����S�o��p���KZbt������n��g�B&����6E!���SJ��|���Bj���TK�;kx1����"��dsi��M�t�,F*�/��.��i
��������y�|B�	�|q��_p_�P��P��Bh����H�b�&��:U-�6t�U2	J4z�z)0�Em��2CO���:���F�����$5��Q���#G��<�������X5�%RR|�i��<��3�$Xp��;L�ci������4��(��>��q����J�1����8�7e�JS�^cy*s��7��' ��)aFE��}�=�����F�����w\7C/�������4�A��R���*d��������������

O:I�KJ���F���x�*�w,�1d3*�!���G��9�^]�N����s�O���F��t�u����!��>���h��?� ���`�&5��1	��b�����A�OYs�h�h�H��yu:V��1�IztL�.%((�5�q@����xHN�3�)a��@�7���gS�qVx�r*B����X;^dp�9wn�����P�b)�N�<����pW3���%�RTM)��y{����c_�o�O��r(/�(��%���Nl����9O*��HHBLZ��������������\W%kK@�`���_M�"L���C�'L��L�}�5���3�q[Y9[(�r�W�6@�68�����T�,T:������l5+E6���y�jV��BF��[��1�<�O��a���}��'��y�����������f�H7}�Jv
����U.7�?��.��}�i�;��AT��/�@����Ql�����'z�t�O�&�� ����R(����!�4_mQ�kT=�K��~m��\��\x�"����m��s���jt�6�f��- ��R�bi&`���'SY�6��;t�Q��#���j������X��EhC�Q>s�79�;�d%��T��&Em.�?&���N����%f�Q�
��*���I�-~�x��j�\�
*��N.`|��1�PM�dCF�l>_�#��g�*�sGp���I&M���+����1G�{��_�g�xQ�a��
��G�;�f���3=U�`�FM�s�����r�_�{,5r��,"��U��
��N��.N������a���j}���
�B0K�myRi�����O������� �,*C���~Kym�����#��q��'=�\a��;bPMu8���)]yO_^1VC�9t�����|�f���KV�hSZ���F��pS����k��*����(R��5U����2�	���P�XQQ�)Amdqr��F���8T�SJ��]N�9^=�nO�x�zVo.��<-����)[��%�8�o��eO�_.��G�_w��3kJ~+z��iG��r�`""���A^bYd.�
����aD���
������ ��t(kDc`�K��������������W�v���g��)��A;_���>��%���r1Q����(��2%(�e8�1e88g����`�K����
��i���F�A�b���<�.V-_zd�����;�����i�fJ�m��}'��>Ohv4`l��i�>`*�����9s�5�������c��	]�kr���c����h2ak)��)\d�*l�}������Xu����QM0�����}f'+
��46������|&�����H^�pN�?�}wvr��9w>�_�z��
/�)M�<����8��p�ZXd���R�-�1���o��NYPP��l<�R�\E ���h1�\Y������[9���}e��?e*�E�'�������8w�����l$����&QOL��:���9��&�@�%Z�eD\�&&%D���������#\�7���{������(����I�c��?����8���"�y:�H�-�h�-AL���=25�J*���<M*�,F���Wk��D�@�kT�aPs��$o��1�{z��
������VB���w�iue���J�����J����^���#	���s��\�5s����%����R=9�
U��sI�!*�(�S�F�Q��p����vF�2����iA��F�h7���;e#����,&���)��*�����07"������a�1y*R�#��vh��b��)v�nN�y�����6~��M�,H��e+�|�%�A�F����5Y|5������)���Z��~����GCf*�
QPST\y���h :�����w%W�J��{SX[��4�5���%7?�k�|o�*�A���<q)z�������s�J�A���t����t����1R�?Ai"f�������������� ��-�U=8��8��][�s��V�������9�==��tq:ey|��������7�&����t����<��+1�X��{�w������������Q����������k�z^�A��P����m�)��:oQ��+�����k��e&�4n�K�-�/�����*$�q�����B����B�2pc��y7#U����C�"��Sep�>8�X�E�d�����W�����^F�
 Bz�����I53a��#�r��B���YM�w�M�(Fx1|��W8zDx�����/����uzr5�����B�����m�bmc^��2r�gUUN���+��"��Nv(+9�Bq�Ar.��J�yl���m#_1��J���
�]����
����k�W�W������uo�X~���8'w��>!�?
����ZJ���Lp �g�V�Z!uZ�]�\����D |��Q�?��.��j$|3���xy;VW��+�@�>*�mO*Y�tID*E���S������b����(q�|�?����7���f�r�����p3������j$Jo��TT5�&�bYs�����C�K6���8U�*�D1A�(.�������������3���A�D~IJF�E����g�5|�L� pKYh:",K���S�(��F�W�`�:�s*T_��V�d��<X��5���2���D�q���}���=��/8Y1YD��X�3��>�.F����[�5B�d)dZ>��KK��sQ��~���U�%���4e�)��B�4�n�����h��7>$������;���8��%���_���1S?��D���)?�jm���
�+�X�u�����_����Y���[�$����ETHD1�H���4��s���$���Z]`�M�:����J�Y�sl�s
�[���#�M~��P���]Wr�����iD��BH�$E���N����9h����f�s��b�p������EihoK;�F�����f�Y�'�U��Q�D!����:�
tP���I$�B������=�FG ��Ox���������#��SC?h��8�[l�H�U�^&[<������;��r'Q�AP����AS�.���Q�=�s$��8����.�~:��&��`��e���|'�����-S��1#&C��oc:	��]���#�v{.�b1U���������������������8�R9t�}���2��O��uc��[��y��y������D�����]��\��]�Y��ukU���`�y�L6x-x�i�d�)���dv��c'���[>
#�rs�FR���,�[da�����F�[����M�j�� �8c����B��cS��/��}]X��+��;}I/G�=��� ���rD|;:R�F���v���?�d8�������T��!<��)>��s�8�����9x"l�W���:��^�K���
��x�]�*�B��}���[���2e�o�Yt����)\&���(�:;��VR���8`(�N��*^�(}\[jQ�^V�	su�C���	�@������-���
���T�f�NRG`�$�ZU���f_�� =�#��+�z�_t����}����RT;�t�4�1Y�?�d���u;������b5T�U �.`(�r�r��aV�Kj��*�`���1����[�a��p�� *�e������
�"�,�F�}�	{X����U�U���"[=�mi��vn4�M2meo	/��W��~��5qwhZ���5�M��M�����3t����xw6ND�p�COi
�F��l��l|�I>��=���ld�r���_��@�D��b�H��Yi-��.'��TCE�����TX9pwVr*��i��7o�.�4��T�N.����*OH�rSbn��/�)�dN�8�{�J��w�'�$�����b�	"s��U���7�J�{����������V=&���$2de�i�?G�/�|��O������DB	�Q�q)*6��C����&"�sv$��o������LC'*�C�_�X9�j���I�7�c�����m��u�S�<
�.�����D���'D=b���4�XV5SlMxRu\�Cy�@yn��p~��q`���{���ga�����������U�H�2F�M����hW<����!U�NDQ�n���&���6J*[��5!��u����R0��l,���P��s/cp���,��h�L�$��y�R#��B}(>��N�5�uJ�$���V�t���	�IXZ'=���i������T���w��E�?� L�#�������i�-LE�77 U��t&�d�e���'��Z��M����X����a`f6t\�{�����4R�W�v�>E`��	�|R���U�Y�G8���>�(T�-R��,u����&�h�C�m�;4�I:�dz�@#��	2��EA�?�����X������������������-��'��d���& i�%��,b���>6�hM �� /�Q&�x��?�������w$��I����	�}�?��K�C�I�c����W�����p���A�<R��'����6���#����Z�"Z)����/�����Q��y_�K8��,�0+x�oBgX�{����J�;LhS����e��L�%wd(*m�b�
E�@�������
�j�$5m�/6<��#���gS��+�@J/�3��VS|��g�&��5�q���kky���I�%�g������[���eU�b���S��<9
���D2%<��P�#��m-A��.5�
���o��zQ?P���Ed�|�����L���"��(h��&�q(^c:�o���0R|m��l'f����\!��@6(���~�*�M`�m�*ljo���l��^d�Lh����N��oC6����\����Y6��e	�(�0Q%6�&�vk>}����T�5����E�<Q��?&x�E~�F�Q�&��lT��bc8��Gg*(�CK�qc�nWQ���]�<_��$
����F��+)a��oY�������Nvz�Yvz|:�M�iq�������{M��*M�.ks��t�c�h��+�n�.����-hxD�KO��:��C���q�}V��tFm<��c��G����O��S	�r�Kv��D��`t���#$:lz})�L4�@�+>s6�M|I���h��_��^���r"W�M6�f��A���7(
��������������j����I.�����0�����L�����P�@��S^S'�nz���6�	�t�fH��R�e�3��buK�S�s!��������`t��[�4���C���9����h�{b����D�o��Ms0/�Y���l�;�%�`Q�>�S�`a���M��(�/�^�PMB���]PK,�F1�G�E+�h��@��-h���f��Sj�h�$�����O��9���&Qg�Y${���\���J�����?	0��>�I
��H�6����)e�9�������������o����'�z�ymf/�������"�����]l�3��P��H
%x�B�|�	�x$�T5)L���0���>�EP2��wg�����e�9	�O���X��0Y�tc���*���K��)���r�W�z���XP��M�9�dh�no������dz�!��XD+�_�SI��Sf�~{E!�$��A�$b�$*��d$�m)*���s��������xx�X*��;y��'':��Rt�s�H'IM���6��A���J �E�NR���������Sp��	������@e��0�'��
!�D@�E}g���::���K@���BV�`�T�Mu�i�bY/�dwU��H��C���#��1�~/��JO��LG:��3�y�4B��:�]�}��N�e���t�n�$.l�Fv�r�N���R6�'�#�HG����|��jV�>��
�th��8E*�����nv����D)�@8eaK���ac����H��Sfp�����c��g���n��p]w�ms�_�.���'u���D�
X���
r��J���<?NH���3��g��/��eN����H�^,0��l��BO���<� K5����)E7eO�����������p�,��d|J�7�����!_���K�c;�X��������^P�I
{4�p%w�&Xf#���R�I9�S������j�[wO�-�'��z�f�{��P��g����20���/j�s^Z�'��������Ftv�:/���p���y!��������x&D�}�27n����D&��8�P��B�mX�>o���Hm����&[��$�H�����1 {���VV�����uO<l2)�I,�m%U�Q*������}xc�W�.��l��;�o��������~;;����h����������}:R�4�J��c�Xq���Y/#YJ��v�!1`���i�>�4Hf����
7X���B��4���4�z���'9Ba����L�p��Y�d��z��4�X-E���t��zN���k� w�5��s���9:�;Cu	�f(_�tU�zN�t�����!7�Wm�������3�4������u)�u;E�
��cZ�0��jK�{�;����,����.M��A�cyTA=@�	����v�0�1d#m�����)>�$e��U�Z;���
X1>��gg���B��)Z��0���?�=�
�Ac�i�5R�0|L�k�tA���ZJk�RSGHm����nje�1�������R�I1!�z�K����YW���k<��H�S6�8���,�$
���]!�6f�3j��G�z0�5��9p&Z��w?{��4�9���d&�����������q����J�8���Q��j��'����>�`O"�'"���u,*l
�s�u�y��0m��i��d���T�*�T�&IU�6���:�/������z%St���a]����r�*�������c��s�y��a\��������<��O.��|7�
�)���X`�||�`n��xQ�e���:��z�����KCx��f�r~XQ=g�_��$�4����\�������0�oWp�T�*��;vF}�E��42.�"^�j����4B[������ML��#/���/1<_re��TD?���fv��W(����;_�=T�l�]�����M��FE��I���F�G����9������u^.ei���52�un���k6S%�8M>�N�p{s������J6t�����#�$��^�S��46��F=�3j�#7n�=��uW9��f�1 c��X������&�;��9N�M��������jQjE�E����M��p�zA��6�K)��V��]|j<zBS���/�<��b����3�����FX<<���f+#���I��K�H����j�i��D�qt���1�Y���2�]���Y��t�9���R,_�����/��c>�Dc`�W���J�4�B��4�~��J0�|1��S��;���l�c�>�m�wQCVA�YN��Y��'	{�i�>���)4]�
�iQ1����u���������,K6�V5�QCu�V�J����5����f�<W�%\o2|�
�`Z�q������z�B[����q��"m���Z}��Wy>D%����_�>�m��5����Kqy���LC��i����\9����i���N�P�������g����jW["��i�F;<�*������_�s�|�f��R���PK�9�����6�k����K��9Vb��(�����Hm�%n�>il�g���-q(���f��2"4ix<�\��Yb�@�������bR*�����,�%�%@,�|*�H�#w��i������nQ84]���e�lW��P��.HESW9���d�b����.z2���(Se`\7/`��b=�_iG%,�o�'�>�M�!�P��F��H�/V^Qt@}��ki�^i��6F�l�@���`JAK�/�E�=,?ec�z��{Q��S"��D�H�������<bk�����_a)����J�*n�h/ ��`]-g� ;P���^�#Y�f�
����!���������a��`/gp}��9;�%�/w��3� ��i���bR��FV�Q���R�^��q����l��Bbb�

=S�7-D��@SHI��SE!�g�Hq�X&���*�x�2>�Lo�W���F!�?P��
\wD��h��r������_>.�d�7�Tz�O�_��(��t��tEgu>?�����Bl@M_I(
�,�	��+~Z�F�u���d����f����<x���/���s����|���N3�;��~3J�^��h�K���pX���V}A�l�%����/����>�,
��A�U�(M@�q���$#H�����D�y0�v&NK���IqR�E�S5T����JA
���`���H���tW�aY%�����]7�;�M�iL���3��y�����H����	�J�%��q;�s�V&�J&�|	`\���W�aa������C���A�'����]�B���]W���'kk���~n�\��r����.��+;M��T�=D����C=�?�|uf�8-v=�[������22	���K$��)��g�C�5���s�uJ�$�����{���-)��qT���O���\k��������)\�������j����{M�"���4x��
W���f��{�J��]��������-q�mA~Sz�pR�e���Lu����NM�Sc_5�C����������q�����%h�����y}�y����{�R�R�Q(J��EH����0�Kd)��$��n������,��$��S;�d����T�4 �
�-�?9hy[^T��+��������$gJ}}p��f:��~�y��\!��p-�:�$�c4��{������K�$Z�)��l��m�]Y��h%Sc������8AB9�H����:*�%��4=���R�x�c�8��9a�^�_�[�@�/P�9.:�9�*���
G�k��rQ1 >~���$�
)�K���4�u5���-H��w��6yWr�������o3��h�;Bt��@���(�/���^���fc����T/�j��wk�/U��S�
�a��#�����[\wn��\EON�7y5���P�v���d��2p�DQ��>~�t?D������������J��.y����u}JR��<����(�1jWI��x@����{��D�6j��\4?p�$���o���8�t;r
�pHm����+p���W��`�H!rG"�Kx��M�@������R����A�7������H��~
�+�%�W���-A\ &.�y`����@�X�t:#��������c���R?,���3?���?�e�o3Y0�������,d���I��K?O{( "��r4n��������6UO���h��p;*�M�b4K��t����rq]������"q�
���g��T�'z�!AO�<��9�xF>�m-�����s�IF��FI��]�pda�mm��NOF�Ht	GT������9��pp������e��?�EU�`d�?6d��Q-���o�*j���G��A����C�o���!fo	����cP�P��!�p����F8��8����Z�G#q
(D�3s��)Ws��t�����
\n!�X(	e}�\WVf�{$����%1���2Ro����"�C�g�����_OO�b�����T��-�x����^ D
w�MM��	jr�lU��aG��$6�������C�����j�-��:��i�/*n@F97w���#_|o���D��0���p�	j����TgCf�Yy��b��g��<���n�P���Th�-r�g,5�l%��!�F����4$9����^�g�����C����9g����+R���{��b��LE�9�����NfR��b(������.��`�xC���C(>m�dw��Q�|M�a��R�8�KElKUv���LP���
����Bq/��/T�3SVF^>34��2n8,{���J���R���M�nT�tZy��gp��/���M)\�H�!��v�1"���\DM�����D��#�4N�y�*�Z��rP�J�"�����'��th-P���b>�Dx� ���������`���`��^�����y�lj����C�C�&��1*�D���i��������#���,��(����@e]�)�]k�jYe7������p��n�U��XJ��^������#��L�;��Dc6���M6�TGEN�ch��
C������)�������'���l~&\��P�y���K��c������G�
w����n"��G).l��&x���*�%��I��17�	cg7:����p���D
	[X
�C^SA%���2]C9Wp^vZ�H�-|br.q��f��hC�^���dB����1�,�~���*%��Z\�D���}��D�������
=�\�$dSNt�^��h����V�6@�>����A:�3]SId�lX5�Q���OL���
u1W�p�U�������M~��BR�����R#�n��
��fX�y�\�8
��P����F�����Qs����(bN�X:YQ�|�F.LE�����3y2�Q�gE�Y�����sZ����a����3�[Ne�c���X���77�����K����:;����D��
}75�h.J�J=�-��:]O��+�����|�=F���=��f��Z�j@��C�N���JO��P� .�m��	kr@8���>�S��HS�-��3�v P�,�����j�=�*��	���,�O����5���t�<��4���d�0��W����:�m�����c@A;)���o������G��%��R���<��fZN��|P��%���klt�`���s�>������E����p��^�"��S%Jd;�������e�du�>�R$hxTfy�B�>�N��H�������C.��X�=+�������?��h������b�����?�	p��2
��N��{$�����<��������^T7���y��l3�tvO�
��[Z��+������zJ\d'�X�C��=:��U	�Uu��0�0����XK�g�`�K��w�E��}�����&��xj)���5���L���0$d�B����^;eAF#x����	!����Dt:oM�%qkK��o���������d�+�W\(��U������Z��?25���+��b\D�0e�8G�f�-��,FpFl�X��'E�����s��w�����a<3]4t���BG#���, �������������==/��_�q3)>P9���U� �k"a����h��aW)�����Q=X�j�C��b��r��W���6q=��>)�xXA��
k����4��|%L'�NI�i������i�
}Cv,�!�<�LXv���FK���H,<����:��xf��D�!��p��+#�	�0�X��)Pc6r��������D�>u�,=��s�=������H�i�����X���t����/����e7�)VmgA��|���s���:�n-0���9:hL�F�������,G��e��^
so<.�i�)����"��G0������k�PS���MV����"�D�j�q�� s�R���+��K���,�	�j2�a��jx6_���6����%hj�������\|����AJZ�(����Ald*CBz���@�i%�\��Y�pst�_�$��M/���n�G#zi#s�Y��f�@���h�y	�7����D�b�Bm�����6yU�43|�NrP���,�u����0�d}T[��S���\z������CO`(�24���f*�m�hT�>�<���YG����n��O0�@zQ�H{rc�>S���B:�7�gw�z�����__�<[Vy
,�9 �����F��N�6���`
~
��2�V�������������_���::M��MV���%��W5�nn���UtQ�?R
���������Q�����b�
�#4�-A��������8I�Q%T,���N�9p~9�����tr5�O��Pe��b������H�vP��j�=p~)�����]��c140���#e�g����
�o'�]��Y���\a.-LA��k(�Y�G�**,=e�!�x}��+����?K�%W2e���"'B��O��,�"�?W�W<�����9�1����H	��y5����<����t�=��m[�C�n��
�]e�����ZerN	}�����t�\m!�������
���2�rRf�A��DM-���c#a���U��m/���r]o>!]��P��F���7V�h�o�"�I�
	����U��WJu��i����'���8X�p] 39$-el��/v).�����cR���uF�������9�x��������K!�{��Z��
�&�V^��#�4
�
��H�O�u�������������:~?v��o��c��FGW�Ve��ze���*���_ ���A�1C�����w�Tl[F8��<�8M��?��X?�������Z�0A�e@���C��._\r�s��N����MS`��M�5U�1l7�)q�����l<��T����X�S�B�Kt���b� d	
����z�_i��OE76���ek2/�
����AS�C^��k6����(��u������P�����^��`V
8�O\�s��1�S53l����2�f�0�=Kb��"�MM}_�z?���1?L���e����Z�����=�\�7�5`�l��0�pu=;%`���:����6�A��K8����6�����1)�l��ZX!�p�[7�'��[���m��a��X�b�`�~V���	E�W������|qv9�<{���)v:%H������aXFx�`9��{�W��VyM9��b��Y��J��
�i\n�Fw������a�:�T	�Au4������7%#�c"�lH���]Or�y�x`@-�����c\�|����C��S�AY2^�l�@
F�:�+�&�����Ux����(�Q�K��y}�j�����Sy��*��`<N�*��^v�LKP��9�]6^D����*/������f�f@u:O_����
�
��5��K�QC8��1���?�u�s���QN���>6x��O,[BS���Y��e������;��yX��l�!=������CY������l����:����h�a ��<�;��Fc��-�JI�L����������X�k�Py�Aw���w�|��)�U�WT������<��5&?����~
z�����w��!P�Uk(F�Y	��D��e
�\?�K��������x��X����.�F������'*����c��,�}}�:9�pyuq|��
���w�.��?������4���D9����Wg�Z���@rx��-����������_�zz1~��5�~�1{7A�|���~@);� ~�}"��U�B����d�s���|}|y���"��<�lrQ�).�U��O�������s��a�]�e�c �>�\+�=��T��#�RvN(y�bTheb=�*��E0��:��w`�+��"�2��5EMD5EErQ.��|�}Bo�m��m��H�K]V��[���oe����m\����*L'���(h�Sja��DFZ0"eK�(�Jl��/A��qZW����G�an�5Yh��'�������&i��$^����10y�]m��u��{B�����[��$sD�O+�?�6�EI�A����GAP�M��P��&3d�+�K�A#�b�1����L���i��Z�K��L�480vl��Q���pc �w��7Ov74Wa��rqw�����#���z��p���1�+����i�[��a{cas��.��<vV'W��5�Zun:�^��}Lz���;��bq��aB�N���X�QS���{�cQz�k7��G~��L��-l$�hF�%*�hj)TN���z��ofK�	WO \URh��
U�Y����	����:$��d!$`��5��Y�%)����YMT���'�Qb�T��X��yT%�*5���N�Z��V���^6���(�\7��e�����������v�E��K,M�
�+oE��T��onMq#�|\�F�jW�r��J���:�g
n�
�4�<��2��C9�c�p��*Q)B�����Rl!�2�wA�;~Hh��
>�:�����>��z���V�k;Z)ir�Q�����_
I��~�/�Y�Z��[B	�����}���onP{��`Pt�@*�R��$�}�j�c�2Uw���@=��lxeYgp7�����o�7�`g$J,�V���q���g����_�)����m����pd�?�����6?�{`[1�*�����!]%����[�[X�K����_������E�3d�Oz��/��+�n�]��[�-�I��X�@�N�?�
��dA� �"��+�-Q��6�tpDk�����aG���L=9z�b��D� ��'���J&��&L�dR�xy�Pa1����]��^6�(����}3�&7���O�����EpVMw�J�yD���S����D%	��u
�9�-�I��9���-����KY�TA�����lP����|�2z�^��o���0p<nr�f�:&��k��t�SI��nXuJ���5G(�x4S��F�"��k��&����(��b�U�������Ie���L.���
!�6���_{��k�A)5F�,�fl3������_=,���a_��M����F����\{�t���,�����������6%���T$�pV.n��a�U����g��T�5��[Z�iq�4���TJ�(�&���� +vc���
����K�X�' �b����d8����]�:$+���R�A��\*��E�!��}kc��al��$),�>�@�=I�i�/:QP���rU3�Q��y�?���!Yc�����)�d�ak.��M�����i5�:�R�����L
����lt���<��Kx��U����f�?�s��N��}#%~��N��	�0*J�]+W���eO�*/<�H���G� K��]W��t���(���Ynh&b���&�������:lH�B������xU��o��q(�&�������Y����
�c��lE���0k�[�S*/�rm�u!|P������o����4l����:���o�����'p�<�2�O;=-�I��p/^��-R���B�s� �>7��\:6��ce
�%=|�*�BFt�
�l�a� ��o���;�AG>Ae���`�����AM���4��	�B�5������l�8 ���G;@�dS�[XT
�
�S���{��OJm@5��'��	);5��j�*������6d$t���w�B��:�E��$o82���uv��O�,C.���O�&��p]�.�O��%�����!�B7}���-�F�i�p=�!#��	;�f����p�vA��A��B�u�;�YG��4?���Omc���s�
sG]�e/�e���	�Y�v�����K����c����,�R���
.?��������j�E��!##jm�a����c����F�.LR�^?���e��h!c#vw�I�Y����@E|��Y�2�1T��2we�p��;���?�����������juV�
�L�����*PD�Y�T�S�ll��	5@S�����b�m��G��.r�XIYH�m�������'�Q�a3J�}9����i���q)��A�W��q���"oHEj8�T%M��".���%��~�p"70���{u���)��/�v��������~���T�Zl��������O��^^�_HI$���P���/�t���������j��b�n�l��C�c�]f�)L��9o�������R����/td�����l:P�|�%�c�����_�
�u�I���[d���'\?�, dC��E����887�J�}{Qi��y�\;�"�{�G�v��%JDw�Y�����MP#���!�F��E
c���������b����l�+���PQ�w�!��������(�P&����~�}�2-�=�db����
�_�4����Z�F�.��^9�jU�:���`.�j!��*����'5��x.��DI�A����#U��q��Vb�]�X�����z'������D���=�(�4h(
����'BL7�=�=a��u�O����!�p�)( ���#)�0�Z][�����fxS��7����������s�
|���g.a�#tE�j@����I���HC�p������O���#Q0��dH,(aG�������(�^`��l�����e_R2�5K)`\4�0�Q�T��N���dZ<�+�IaB���E�,b??(�y�G}�W�Q�?�|M��ph�v����M�V�z��` �Y���IF=�}07��r#�������A�aW�����Eb�+����`
��:Z�N9����v�Q�hxYK-�9h�L�q�+�����4�u��5SN�t%5�`���Jp��QO9����BA�.�G�m�����JP�l�>�����kE�2����o������v�A�pja��Z�p����YS��.��b*��8
�����q��#^�x���A��Iq�'����e�Q��l�����m�"d-�lw������"hQ��ChQ���chQ�#�0�Q�A�-GH�b��i�#?��k��'�5��E60��\J����0�T0�e���JP`���:4Q�S��w-��`�>����e�[J�34�[x��GMm�d��00���H����d�A��QH�v��Wv�Z�C$j���C��fIF�
`Hk�%��
�c����u��Qn���`�#B��KC�YJ�$�P�I�"�q�S�����F����Fv�3c��d�1ocp��|�QE�����e����MuYD�f�-��c
9�Z���Y9�Y;U�P��o�f��?�M�+tZ3���6_	�/���B���u�������@�*A!0�����l�R�<��%GT�"S����R�O��W��G�����Q�������C�}�:�-KO���5j�X�I����	h����Ml�5{>��E��o?�Ck�>�ZU��
�
K����Y���U��d-�#3����: �}�������\�9���j��q����2�H\_�!�F�B��i7V���D![��>��#��,���u&p�����>��F1=�5}�4��[b�]%q228�31P";�F���G+�h����3��L����������:;B�������O���v��9[��g�"��:�f�T�0��kD@+�����~'��������fn)��;r����P�$���xI�Y�����G���P�K��0���G<�	��[�K��Q����p.OR�b��w�v1-�M�],�\�#_�?9��)�m���XG(��7��2���6����y���Tz��`I&�V�
_g]����i��KV�S�����
5�QXu��~��0�����c�	�W�v ���������E��oP�b7��+v�n��mI�~��trE@o���~�^�L0��\��j�?DvF<���������{�wV�b�C7�V�R����F��LRbj�PB].�B13��~'].���N]�D��)q��������]��}aG�X�X���v�}J�����zQ�k�sn��Ui!�'������Z�RU��F�3AQ�X�0ne�4
W}���	/�-���2�EC���[��@�h�X����f�D���t}�h�F�F��G�{����EE�T�y>� �:A��
�C6ip�.�o��= 8H���8H�X�]R�P��_��3��p%��1���%������g�qCM�l��MG�/U��u~/���Pq��b��P���������&��H��qg������������n��Al.Pf�M����.11�dZ98�HE�����`0�v�'{Qb}��+1BS���&�Kp\1Uo�H���d��MxR�Q,e�xL�Ba����@)�L+�N��<h�����T�V-5I�i	h���r��n0������3Y����/�����F!"����fo���Nm��������Ne0U�1N)�T����]���dP��f�Uk2�`�*���U����&3�_�o]����#��28���J�������qT�����^�������=;T���������������t7j��A!�����,
M���H�$c%,�#y�'V��R
$*����=
� `��l��c@i����)!����	�|q,��cY�x�q Uc�
��Uy?���6_�jWc���=?��p[����((`:���K:��wZ�"�
�Lm��N���O��hs��Ev�:'�)y�h���6�h3����B�T��Fq�?�(����
&:/q��o�*��B���I�C��3�D���f��f����u��H�#=z� Ke\�����0��F���+%�wSS|�������������g�s������$�/
(��p���xJ@-��/�!�����q�N����j��;�pz������+��D*\�O�w�8����z��':����%�t���������\��O��	��0��|��Qh/���w��������������I���\|E����O]|F�����*%)�l��v�Q��Iy�(�{��~�����PA�����>��W��N~=}�������������?6^��5�'�@��\���W�1rDTb�k��h����k��_c��F����T���$d����w/���C������
���1�qg85s����6j�t�W�����m���[��i�1�jr�#����f�l0����`�d`C��I_#\E���go����Ac���/���ZD������mD4�����=3�GG{Q�:��)F1�bQFE�\��?��*�$�@E����SK
t0qS9�����j����z;"�z���������o�6��o�f��lfw|����z�����5��2Ed�#n,���|������8��5	GP��aD���� z��Y�U�!Zn��e[�����������LV�i�d�zE���aUb�
y��G�_�?�1X+NXPo�����8*H���;aC��TH��O*�j\�};f��4�g,
�)���pd���l� $�
���=�K����};���P�8��+h��+�h�!b�
.fPQ�0�t�
�z����(�.�����o0`1a�wj�j%��d����i���mA�8����_9g�DMOO6d�Z��Q����n�v��nBa-�h<���(���(�uiN&��G�1E����}v)`Xu�"�h���[mz��]z��Kh�=�Y�Z�]�6<!���T�
�v�F����)-�Z*�1���-N[a�I8	�so�>�<,�X��R��U�=���Kj����f��U���"�=�P�Y���HX6$IN���}���B�q�a����s��vd��\�F%;]X�q�<La�'���f>�W�h0!j��{\�
	\�@��>p5c�b$T�vqF�[!�0T��>_�/~�}?iN�H�Y5l�I���kwb�)���ac�@��h���J�?f�	��>��UQ���1q�������d��N[������a���W]���z �AU�o^�`���=yY	'E�[L����g<Q�����'$��sNG������;f�#�����S��l����2�p�P1�|��|q[����~�B9/��O]o$�9=�����"Hi�2u-�U���i�
����n.�_�f��|
����?�DH��I���?i�����_I&��`W3q�b?a����DL)��#zr�"����S^��)��VD8�P�HG]>
�\,��#��}S0P:Z�����&�'���<��-j�p)���>|�M@?�Q�U�gS��(�w%vb]���jH?�Z����k�\u@v�5��b�S���%�r���ANf9��?"--u�o-uT�$�����i�<vuek���(������	�e.���g�B���
��6x6+'��:|����A��P���,U�R����$�
��dh���Z����.$�$���m��B���JQ�$�~,$�� \���(�$�~,#�`u�\j;&?�uw�e�`�0�\&�=���E`hrO�m=����0i9��gk��MC�i���+XxMz����t��'�j^,s��X�%z.�uS�����~�h�Bo�Se�	���0�� n+f��k��-��lL�fv�%�Xf��}�@C���W�q�`���7�.�
h��l��x�>�����	L��U6Z���n��V���PD&&-�������/�Y�
J�z�7�[�V Y����'#K}�����;��i��u~���:��vG��#��7�������^_��~����,dy8)�,���&�&|7|�G:�N����38����8������
G���t���#�u����_�q����7����[������	�<)���
�x����/n�N?@`f��a�KY?�BR�_��N1�}&Mi�3.���Y�8�w��?���:���#��F�=���fB���b-�bq?�0<~!S�_/xs��Tp 1
!�:��q?��%Q�q�yJ�T�=����&���P�*{1%R��P���H�{H�E��������TC�J��%lHu^)C�!������{��i�������Q���p�b��8�u������NR�,x����k+��J�Z��1�F'�r��Y�w)���'�������������
�i#&�LL�WAK�yi,gG�����8�j��u6������.�pg�~\���GP�H^!k�A*7�]���$?2������w���������,�����^�X/��H���Uy�����+{����{��
�������^�=B���r��:����^��jU��*�Z���I�#���v�X�(��������A]:�N�Co��Cx��oW��_�9����l�����rI��/0��l�x�x���M���irjU)��PD@�[B����!�1v-�O8{s�������1���^ty�%"BFP��.��������X�-��	���9�zu�\����O� ����0���WE������yQM���XH?�Yo7|�h����PqwS�UO��Ib��w��R2��d����i��I��5�JL)|��b���&�Me�/�$��$�)z�������^L��Y�A��w�yI)�`u���.��w���i~���G��f���h�"�tLPz1��=#,M�������I�:�<����+�������"lXEb��$5|�F�����]���>L�s�]O�B8E��>�m�{�HH4�J��	� ������Q�����>C�L]6�.er��j��&:��������� _�`�F�i���������h'����9�Z_�a���A�v�@�:
�qw]C$�$���[~t8�����,�+��\��^
�h�s5�]4�/cu#�
�$X�
5�����mt��o��0l}A6�mB
By+(�s��8U�>}����Z	�7���>�m�g��t����EB�&repH��I����cnKH\��Z2
>�A��h.�Y���1�(������|�=8�d�3�r+�*F���&\QQ�������lt��
l�� (�Bu/j�����%J�����b!��.�!yqOy����w��2�u����b������c��8�����l��zmc���	�V����9!�P��lF�uI�������>����������o�`DU!���o!_��\{B3�"T��#m�"���3i9�/�}r�}��u��uL7������F.%U��[��8�����56�����������H��uD��t!�z	b����`�DU���V�h��K�9 o����������@��Z�����ff!<(������M���
��^���=�.y's@y�u�Tk]����E�h�/��'�x���M�#[����B6�;���2�R=�!�B���@�C�#ut-o�����KF����]��#.���� L��2j����K-�;rmP�vF
���gg�l�/�<������i��i���T��m�?�������g��%@U��q��[��e����9c�u��\���	=Z���&�6v��s�G�A�f��#
��!/�wb�����e>���A����W���_O/��O�������#�������W�`n3#\\��1N~���u��!�i��}E�[�v9�|��c;����a%�������2j�5��T�J�iM���j��-�`��q����__e�h���������K���K��u���S�����j6#.�Q��0J��IM}��:#L����~c��3�.RbV�;������q
�����g#���2~�<}r�g�;��5�� "�m���ZM�!���d��R�Ii�����G�^��O����c��l66r���4����{�[L��j��{DIg������c�cR�.���o���E�O����}1��F?C�c�
���\��Z_�U�������q����~k�k67m&zMC�6�G{���w_K�{�k�U#Q�(H�/E�WH��5sV�k����r����r>��v<m��M����3o��KW
�"�e���i��|�R�4s�^�D�%��|.=����u�(�i�V=��'������ST���t.	�xCE����^���� h�/W����I@����x���b�@s���[�hF�J�&[B�=j�$;�PnL�M�����H-Ao�3.'f��lb�9��������Og�����[HR����Y�P�/�"�'���(rT5,"�[��&��d�C��?P(�B��6``�q#k������p=��q�D:��~��t��H/���h$l�	�r��G`�����|Q9Jo������$_�!:�]w�jr�@�u�
^���������ZM�kc��}�ae�!0Nj�F�������������-�Z���6���y���:_Pj=u$�AQD������|l�Y��"��Q��b^�V�(�(�S����~���V��~f��gs�:F������g���qh������#]O�w*�QBO�{��>,�0���S8����<	|�5g����j���%5G#�0�HN��!����U(*g'��=��Cw�
#�b�L�~vc�����}`��-�X�m��^��a��v 5]���z��|��H�f�mgx3�E��q�-�X([-`�� ���7��������=��^��O�������+��������j�������/G�����tz)|3��<�h������}8~�����w��W�R�[<Ul({��	����55�,�!�@�?��g ���W�~=�pA\��l0O
�5u�[����3_\���LV��r;�rq��_Js&t�A]:�z�,W5C&b���2Q�2���1����  I�}}HkGM�K��7/+��f��y0n�y;[d�)�����H�L��?����lF��g{��?���{����;R6c��53V.���O��	'�)8�'M����l�(�������'�a3�7���l������._�WD}?�+��yRV����3z�8h
�pu����
��$ B�m�
uk)!Ca���Vr
�V���Ix�I{�e��QSco�Y`'h�Zb�V�� %7p�.�dsRhcR,3���r]��
LAw�K��k�y�B@p��+�Cl�
rx�C7#��������c2V�K�8�5�4G;q�a%��������\6V�-�K�P����/�j��l���>���X��O9?���=�R?9����G/����������Ox <����@�7� ��N�#�0�7I�B��+�(g�y^��!���<>M��Q��Z�	`��+�~�!_=w��	V������H`d��LA��S�������!/f��������_e�J��=/R>m.�v�wz0�g3>6��A��Z�Z���b����l�����rW�*<e5�Jc��d����[=M��YMs:�����Q�l�F.�&0f+	�F��wx~x��&���zuk�!���P�5��:���=]����@*%`�}�iB`@R��F�����A�cz���&�S("�j�^x%Z�:�m��	Pe��Ip��%k�:	6:��a=�������v��`���J����E�3���{a��"���E)]��:��7����KyI=.&�;fzY����1B��w�*��������0��6[���ok�����_�|���be�uF����V��M��{���������z
�l����=��M0��{"�.B�h���$z8Q.�,W�M�"���rl�0+u��FJ�H
�kn
�����t��ZH���cD�����\����b������������O�[dY�\��Do�F�~4�0��U�[<�]>[�7d������&a,�F��- �����8���r�&�����v�}��&^F�m3:*�yg��F
kk�`��F���{��X�l���F�gkksike&Wd�v����i*��Q8�,Tk4����1l��wC��9�������lkj-��P!������L��H����:BV�V�6*'���wa�9T���W��J�� ����.��M1���:�g���,i����/:��TXY�%#�X�e5^���b��*�u�X���O��rP��V���=����C���l����%�Q��������$���x��xB�����<�>.��Y����f^��/(�
��}�g
�ql|Ks�����`���lm��eo,����4���t?@�	!`��@X�gP��Q[�����K�B'M�����glX|�s���/�YN��yW9����F2���d7�?��r��M���x#����`������_�Y��C��SJn��V�*�}�m���B��z��W�����"�����A�_��_�2�k$bR�8�_�zr���3|s<�e�����
��5O�s��IuB>�v�{i;0(�>�_�������h�c��g�<���y�S����,R$k��Z(XR��E�V�'�������PH$��p�4C��)$��s�|�D�N	�)Wp���k-��1��@��c��B���ZQ6�,��{
��F��/5p>
?��l�����O���w~���l ������/N�~�:��+���{�S����u�@%��J�<4��2NW��C@#H�nd�}�F�=����pI�*�$��������w��@�������|
�`�:����UM~7^ 2�Ox:���<�����uj��T�-�)^&�J�l2V��
"����B��,h
;�N}C��?�z16���l�4-2���Np#��,��>�D���g2�P�ws�K�� 0v���{���F���T��=���y�,��dTYU�X BB�
"4KT������wOk�&y���b�3H�N��|'=j-�#�j��&�DWS	GB-4t��~�%������O����[)LvK�������-n�s��W��+��0�a��
�	��>/���\����W�l/��3�o6�%��q�����T���n�����N=Xv� du��X(�g�4j�~s_�O���|]J��4�;��Y��K���kH=O�o�*@��mKD�<e�ra����=������c
�F��i���%�I��{r*�.����1_���5�bk
*��
g���O����~-�:��)���g+8��R<"s}Ur�Q�\�\W.���C���.YW��5|.������I6$'� ��N:�
.���:s�t��C%l�E=T��O*�@�5K��wN6�7j���zj�T�#�
w������M
M-���M5����o��oj�,�>��V�up
�]�K����T��'�������4�<�
�}�dG�����F��b�->��J3z��Ge6{�	���6��j����(���$
���1��6��Z�����: 5������D��T��i��	���<u��kqOh��2�M?
��H*0�G�eD�Lxv��
��!�7E>��X���Y��V��~��W��l��F#U�r����D�q�M�#Ap�QU� ������LhHr�w	
B��:�P�B�V�=���L0�F3T�8���
��k6���<j,11��	���g�	C����S�ys��I�Xl����T��F�������_(
Ie�� �j�u9+&��8,$��J��(��2�H��Y���)�O:�uuG�Yy#�B3H�xsu��!�2i7Uv�&�1�lD�	�u8���������j
����U��/i�y20A�(e���`#>��k:�"��P@�����?���/>�g��,��M��D ���~�g������)�%��FM��3��4���>�S|
����M��8D&�	^��zB�C����l(!#t�@��)S����U�!#������%��D��\m����]te�e�4�k0k
'�5��p��$r3>��(>h������GC�m�4�G[#$�� g�m�m~��������
��[��P������T�x����B�$�������os�����k�\2Q������|L,�����[�]��u�[�Z������o�
N�x����m�"�}�>YO�
E�x��N��w("��w��MH.�~���[x''�X���I}%B{3d�fH����X�~7����<�NU������9���5��D�tqzQ6[�V���sL�L0~Q���YX*�JV92o$b�2H{�����)��D�	]Y$,�.�������3�Z��d�r��(�����"��T����|��(V �3pW\i�G����P�Z��ZB�����E�7z��P�a��L��uP1P�d�����s�^��d��?0��OrH�t���C�h��/LQ��NQ/F��C�*��3��;�	"TRK{�����+{�����a���! �Ku�a���b��2�(w(�Wgq�7���8��Lb��_(v�L��c����w;��Ma��M�W��#���0j�rJ�0J��E�U���q5z���d'��1��^T����G��L8�������D���7^�+�?Z�j�g ��ab�n��������Lr`���c](��4���f��:{`f�a_\��f��l�,�b��8��e��'z����m/�a�s����r�?��q�u
�]�u�� p�W�Y�$H�<]�s��D'�?����8����W�O'�I$O��#d\xg3����u���-� |?m���A��G��B��q��'E6C%�	���
O�B�:�"�� �F����V�����p�TQaOT���
����TeJ,�h����:�L#���������BPFv����<�6m?�]�8��W�X���/��D��
Gb��p�iZ�9�
��E��Z��2A#�������h8��3��Z�V�������p+��.[���3�Z����!���L������
v����]�9�*�+��xS��U�V�5������#���%&��b�%�=.�k�:��C�_|�5����J�:��-QgOv��_0��2�X��if��6�~L���$���2QoF�$�#�CW�V��r��M�����eN'���R?X���'��d�a�F������i$�F�4������>t�?k�mjp�b!h��6�>�B/����ml���uAk�'�����&#�>F���LD�P��cW���k���X�!���������zw8k����]�r�.r�:�5��]�zv;k����=�����s�g����nzF����O�~��#�p�h����t�j��>��Y� 
Arn����T�_��(��:v�o'9�9�c��	!�I��6�4��zk2]E;l�*��!����n�*�=c���%z�;����n!�����vN�>ar�s��}�,�s������8�� �$bP��D��>��k��;��W�����}z�5�+�G?��@��c$=��wm���k^���Go�����=��5�6�~=�2i��?�gOD+�Y�'�����&W�7o�J�^z
%�M���khm}z�5Dl�����cs��z�5�lF~����P��^=�r�>r�>{�`<z���z�5dmf����5l>R�������w{�jK�^-�"6���5/�L���_C"��q�����Ok���J7�t���_�t�B��\�*�K�-)���s�	���`E��<��s�w(���mO��Z�zk�C��b���W�0D��S�Hym�P�B
����FN������`Y���
�0�M%������H��X��B������2wZK+�C�������+�y!\���MR�kLsJ�6bHF���-�
�����
]�T��`��i�P��r���Z��/j�����h�����yS��o����eK���Pbp����|����z.( ��~�� �	��9�CT�q]�A�u������i$>�i�������f]���	�7���N�����>�p��S\������J�b�0
��1�O�|���������S���O�$[h������"���$Z�'�k�e 2�4C��U�h���Y�1�
	��G]$"gH�����T�A�v��/��Knb��r�j����_���H�T'"�J�7��"�����)�y����`?�-���G:����;����1Y)c<�k9�
L4�A�D#1�J4�C�@ej��.�������0+���3���zD.�*C\���Q�a���XO���r������=�g�R����=�j#�q���9:��D���q\j�m��c��b������t?h��
�������b���C*������B������Ho�A�
��qtc"�K���T���e���w,�}ca�h0�f�"��:�������{y�V/����HE�be�9���;�l��6�����h��p���������"�v;�l�e���g�q���\8lK-���;�����,]+!W{�����l��1RRgZ�%����+KR�|�	��0~DVtj��7(������@���BV�C��Gf����d��e�\�R�tPx���%�x<� ���|��	f�N�V�^D����co8�G��<~��5�]�M6\�z\�����6����<5,����(h����)V��]��y�Lq���X���'jF��@_U�`%�������g�����e�RODx���S��=��o�}��[�W�Q�D��_�����	�Q�l+*�f_������M����c88B��y��2�/����9��c�	��h ��s�B��m���v+����U9���K��oXsQ�9)7�����+����w]�C�{(1�|�4���l������E��u���������H�g�_1Vjm*)�{�Y�����rm�+��yT�5�"�x0@4�����t�y�`��@�V���<�}�}����D�=V�%��/}G�-����b��/�4������E�f��L���������qBn�����C����e\�m��n����=�JTLx��J������j��K�b&�,e�o@���T�V�V/xm�G��2��/���4���TnJE�\��/��lrz�^}\W���'�x���s��g.w����������'�/�~��w]�?I�.�'+��<�X
`)�h�V�����L��d�4�Em���q(��c(O60]����mL`�	�&�tW2��
6�om�Q�d�����=y������������l���ph�%�
��7	�U�`�fQ�Y�%%*V�������1�����������'�X�ry���7�����
��
�������L]F�T�4S�}�a��&�����I��G��F��TZ��P��rb,)�kY ��:�/u����9aC,K?�B��}���3�K���P���N��8�:��:��{��Hz�O����d�Kg���b���8�����m1�D�v�����]����yy������JvRz���l��!�e�n�e�bT2��}���6^����]�5'�u�:�Or�g�O�9�$�wu# (+P���"{�*���Q
p�@�[�R�����H����������a-tFj�u��\���6��>�p+��m����P��Y��7�uo��\�7����t{RPZ��������[�I4xK��X���~������eNW?�O�LjN�`������*� ��{P_�v��N����&nA4;i`.qk'g�xh=s���B�Z��6����Is����c�������������Mk�tA�:A�HNB7���T�VRX;��>�D ��tA��ss8�"��"����M������f�������1&�������Sv�!�L���Zy�=�D���������N&�>�7}_2D$��$AD����� "Y���M&���;z'��P�d�����7���|�����A�]���Dm=�@�V:V�D�����;�t�JO�D���=�;��Mc'"�/�����&���2�\�/���\K��I-����Z,��'�XRCI-��Kd�gq@$��v.	D����R��E2��4<����}7�4�hf��R��>4����iA�3��3���G%�@cH�����M�~�:��h��d��N�`�4�����|��x���Y���>m���l�d���/m��xk��������������>o{��/��kht�I���{�D����jJ�X����}@���
��dc����Cf�<�b=����g����3�R'��z]�'���1fW�38�8��s��<!LQM~I��b/���p����_�k�qm��,�+|�Y��8PK8H���i�?a]�R'$!b���a�fU)��.Q^���
l�P
�.������}F��I~�=g�M��d�y��, xbr��rb^_�[��7�����X���9��h!nrZ�b{�B������5�6����B���M��Z���`�i����I=�?�p�4��_���C��[�'�++�i��Y��4H�gz�v���R���ww�Nl�%����.�$��l�v9��$�O���
@t2p���?�=�N#���<����=1���A=,����k�L���3�@S`D��>6��G���Y�'�O���K�.�Xy����{�1_p=����`AfELRo��v���fFA2�>H�x{0%e����.-i���*��f��	�{������k�'o.'0����(�`K6
�k�W>����29�|��o��?J�0�t
J-B��j���Ys9�?V�g�6f�F�j�R�Ws[s3E�T�����"�.l����[������_��	�PT�"�}
��/���.��
�a���*�(���@z�a�(:\~������"����f6�l��Z�_l�+�rC�L��/�n�>��`�$�N	�t��Ps�j�o�(Es��H�r^o���l����O7����j�����k!�U�� �PT�E��#����(!���^���K*x�q�Z�k�J��5%\�3[���."��M*.S0W�3T�����}P�F� !9��3��VR���u���1"�����<�M'�Nn��0��������s���+��p��y/�U����C��d��dU��j��h���v�?gT����JC����"�t�a(�(ks}TWtMW�l1IB$!\}
hs3N��q��W�C�������Ljg��d8��5�8�&�=.���|#Ga�Z�Y���Xb����@5��>

p���yH�����;Y�6��Qm�J�U���E�Q��g���H�CP(eHP(��i�!O�d��2=$��V������,��HRTO���A���:o�s8�1���]����Va\WK�8�|/���btb .(��'�0w3�������h��*�f�`�*E���/RV�?85�s�Q�����I��
���l���vu�PI������f���:rI��O��<��cX$�B=S{�`�X�����q�/E�_������F���h�������1<��p�����x*
%���%��X����"�{#�!�\���R�tR��������h&��vz��P8S����}�qGN�C��"��7O��Ud��09��������LSWU��5���@Hdc�VAnR����!cQA~Q�I���Q��R��%�EWd����MW�V�������Oj��f���6�l��Y�xR_H��
%��"�j5����v��O����x�q����Q���J�0��<�7)TC�Z�Q`�gO�<�������JPQK��Q��
�������M.�9cfr�@�������e�59�u6��g��:H�E��Nn@~�����y����LSC
m���{�������h�k��`�]r{O��������`W���[�������`���h�6������8[�8E��F
���~��`zi6������Z�p�GFWq��t��g��3��Z!v���H!��VK�<��[���B�����#W��zE~�����	?3�;A%\MgSc�z�#�$Q��5[������wg<�JJ�/�~���z.����?��;G�w��&���X���Upj��E�c7.�2���|qr"]9Ak�v�	8���;<��5;�v9�� 2�LH���xNS54�
��������cw�u��9�1�P�m��y�~�/	�h5�O)��s��-o"�"F�9��j�[uxV�x�Y��O{�����
��,-@��r��.��PVk����	2�o���7���8���Y��q������)m��lW��
��^�c����o�Z�q����J��n��G�����}���[2�������m����T�����.����+�b���b�6��5�t�P����'���j+�t&t���D��L�'�����r�.�q�
$.�4��!�v��#X�WO.��{�n<�.��$0���V���ft�����+���`�6�����#]{`e�D�%�)4E��B����!j�U�`����"���TF��-����;�]k�ub�����w�[�1
Lh	6��t�X�����[A��Y���L]Q.��t7kY?K��7 G{����E����+�L+P�\_,��c`.�r��z���G%p�kH"�o��� ��B3�]�=�F����~ec����V"]�edTS�[cVR�I�X���jn@��2�k�@K�G�p�����
�g@RS�`�!�j,6��2#����1�T����������D�o�@)�����|�#��^i\6q��"�O1K�����q��V^���t^���`xC�B�~_&Q��k��)$gS{>��5l<�`��X��o��Z���+z#�f{u�����G�����2I����p�Y���i���`"R�7���0n��M��Q�����xp$R�c��/�I����	��x��A�uir}6����r)"����B��-�G�t6��a��\^e���2kl��R{gd#.��1F������/WSjl9%���}�_����)�	�
����k�h*j��e=�������0�����a������=3�;�jcmc�W�O���4�����V�u��]��'������S���Kz�RO=��d<]S������c�2&����w�\�M5T�`
FH��w�AHYT���$�B�$O^k���X�S��%��l�y%�a�
��)S4��L��|�d��\����X�2K9������b�����i�20�H�%�8����l���`����&#2}�X1��E���y+�������
N�>��dFWU�E��Y���Q`�V���g��W�'G�k����5�+}�o��
�i9���3�v���K��|I��`u��,A7N�ZL�����D��C��!8��=|�xc�a�1�FI�_X��}���<���<������Q�M����Xlu�s�����(��)�DN(��K�����9��t����K�@�5�e+���k��.*��rU|�q��9�������#[���=C��'�������9�T(���-�R�^������ A�>T�C��9���P�%�����,�I�4�����b,�����1q�5����lJQo�����#���[���a��Ra�������������J/�"���0�A�y������G��������0�A;��AD�~"�eF�E�O������������4���;"�,@�G��,��l�j���+Bb�	|Y4��
!?Nt�:+��#��6p�%z�h)��$���*��lq�ZoH��Ja����E�C�Nb�Y3�����'���z��1w?���T�=|��*C/�Zi�N�n
��XA�L�{�2:��^�X��il�XM����8����������1KIB:����#�mC�ZQ+�$Tf:�e�]NiR������}g ���T��j���*ua[N�S��y@m�
Jop�Cu���f����xlE��Z�Jt���:���R�f&�#jD��~���`t�:�`��x�R��1{/�������H�B�B9�]�F
�z���6`��^��8�,x�-����T������U��A�����P��*|�i��K��Y�{\���{\�+~�{\/��=��\��/U{{�[v��]v�������=x^�1�40o�j)��]��m�4&���(??�=X�E&���@w�L�%ZsU)�D��]���T���T#������g�7N�%J�Q�����GAkF����
�1hU��c���o�w��ca�ar(�����.�^(���T[����%�s������E��)8jZ�Q�81���+��W�d� ��r��{c�YcfB(g���\�%?���J��:�K�s%%��	���'�����An��O�������U�
�-��};�?������Tl��]��3,��MG��V|�=�x�@����V�9�+��D�,T�f}�m�����o�����(�i���%|��<Zi�s���8�v�Qg(li�N�-�+a��������	��I����6'<Y����(l'�����s�l�l|�R"9o��pW)9���)���>���w�C��T$P����"T�U���/������"���p�YPm������-m���ro �1�3��
�~b�����<F�������+�.�.$���E���������q�+j]y�i]�m}\�x���c�b��x
$�������=�����Js�#��E�%���c�2��/m��Z�~�H��l�C*4�c��>�BZ1��l����%��#B����Q�=���3��:�h �j:��Z�p�k�L[���s�^���JxQ�J,�����Y$��*b�KM]��P���X�w���i��gWb��@I:��FdB�6��"�e|n�����N���"�m��Lz�������H���&x�����W��w��`�8�Oy�����w��o>�U<W�������'ggg��	`���6���S��sS�Y�c<���a,S�C9X�=�9�1�L���j(Ws�3�\�2���Z�BI���S�'����	���$3�)��
��d�"w������V�1��5��QM�v����(���q�7�n�$?=���ao�XH�����i'P�Y�w2����
��Vz4����C�h���g[�W�+�v��
��t��{�Bt|�uO
W[��V��y��8���=tj[�Fk�������`��L���E�6(��J����'��p��t�o�O��O��}������8�k�d@_��Iq��U�0�T�0�T�0�U��U��z3���������%xr�S�|$���i�^q/XT��A{��)U#�#����jm���>��I�#���&�W+������b5����M�d���2x����`3 Ep
 ��=iH?R�w����IQ��fV"�_j��u���N�s�L����Sb��	AY���W<Y9�T��2�^Pv�$a@u�������JB�i�j0�H@>��g�T����'Q�Qy������3"������ud��z����w������y.z���z����	����Oz�Y�S��d&KE`��U$P?S7��8-�������I����#Q��v������k'�t����M�����9y�Z��	�(����	�1�F��+�c1�W�%����R���+Apx���M�)�.��D���q��R[k���feclH�$)I���lV���.L�n��D�;5�#k\���Y�7��6���p��0 ��(��U�l�l����x�T1J?a�-�9���Zf�`�.��#~l[,V+��e���q�<���V�w����;�{�����OV���n�Nk�������iM��TK���1H.{t��{�3�6{v|6���N�Z����Qd�����dy�0%2;����Y�r;���&����#�:B���������{��:]�W.�����1P�f�Wh`J#3,#9������R����������)ih:p��8K������7������<���e@��\Y�c��$C��ue-!FS)U�B��TK*(������'����t�c�0}����($�Y�Z��Q�������6�M5������M.�%����Em�)�?|���OX��4��-�c�6Y��b����&��4P"�j����Z�+�����l*�,.yG0*���'����	��E�fM��P]����6g9���2U����ei�M]���f,�����tJ�e�/ _�++��E,�#,��EtlB] ,�F$�Q{�k~/'��d=��x&x9oV#W�s�%���������t�q~���[�b��%�h��[��0�bXF�����a`��V�%���D<�x������G��������wn�LP��C���0,���{�SX\[���1�:�,��=����J������E���.aGl������m-vD�V�W�2��������IA�����H4T���#����nbrm�-��<DFo3��!���|�Dc���T�Q����&!��{��+��2�Zx�v�*5
��l�c��
\��]����N��oP�w���>
��>����������C�>�r�����h���U�x����|�K��e����R[)GW���?oM�SK����
�F9��N�K����l�<1�+������#fs�"�����Z���Zwoh����n������$�}<����|�V`���ma��ArN!m<��g�����{o�Cj��.��2�(�Q�Q�F��w�_���xH:��_]�LJ�G��W���1W�@2�k��f�$���=�w�G3KL���"R������5{`���Z�&��@��?�vO�������#����G��l�Y�I��0�4J��(7Z��
cI���R���U��9�S?�H������������H���m�b��Ir��hy>�;~�W:�P{B��11bU���&��Vs5�& ��B���su�e>ww�[��"��l����������Fo0i�T���T��c���r��NYrY?A[34�	�4"� ��������[*:D����X,V�**�����������(�����zK�l���P*�	.���y���qi��I}V��=�CZ��;^�����w���h�����>"�Vg>�
��W�3��+~�#`��
~����U{E���;����D��*����=rF��r����w��zYA���8�+���X�0����{\C#��Ce^��*�S_WJ��W��������v|��I�'��
~�����3*m[�jx?�%���k,���c�=q���Kp#v�>P�8�Kf�����g��v�r	vy-Qu�|���o��P���.X�Vsm�}a�ycQ!�.���n�"��H$%��0'�Z����������4rq�e�P��������%�Bs��@#z�R����M��ZlI�S����e�H��muI(����]��EQ��:
,*��B���MU}�a'�����M�9"g+*�P���#�O5ux|�e��p8v������4X�qu����4�A|����6��5��l�gg`��H��0�@�{��y@zv	1��U��L/����GA
]8���b���3�
���1�a�������W�r���	�2:��t�-������3X�$��#�xa����������
t��}!r�K�}g2�J��	$mq�`l Z�v���vRdN���s����v���	#F^'GLV`X��_�����b�
KD�h{c(B�����F����=�w�]H��U��0�l_q���i���i���mZq��V+
[�/e��#��G��M+�~u�V��6�8��mZq�D6�8Bt���`Mrg+v�/�?[u %S�~%N���$������X�In2%V���&�>r�>}���8C�D��C�~.��)��v,�N&�z�pu�;�J$��7�1��
����9.�q���s3�l�Y��#�����W�����J��'2t-R�P�I�vsN��Y�c���gR�)���9b��rI	2�\U��F����R�c�Q2rr���I��U$kL�yi��=exl[3�jv������1n`s&@���{DFZ$���"[/V<�Z�Gw���RAd �xW�s���,VS2���Sgez�ZrZ��W��S1��A-%���jS��A2Q��IQ"����U��=����e��c<�<�H&ZYs^v�XT&Hm�O�g
m�%c�z���z����������
^��J�s���K����b���QXBH1�v�� �O�%
!wJ����S���N����� �S���������S�x��w�����=eW�?��P���mXq:����)���#�����(�1%�=�����`��8�������ZV����H&Y��>����)��~Z�i�}:]��#uzYO����<kq�E��<kq����Y|�g�������iV6�k�q���'�R�[<�_Y�	����m�*����g%j:����G�I��(��~����o����t@_�(;I^���ND��H�3����L]�d:FRh�p�%�2{I���H�GU���W���Le������]��%���Y�7���Y��b����1��I�����P;�M��*��Q[����	�����9O�>��Q��� �8�	�����\)-���8����p�uV��a�a(h�1�o���VG;���b>5K��B������_�����;�6*���;�� *�q�32�7x�M)�e([�;OnW
'���(�u��F|!�r,4�"i)!���������/��
/D�,]�����]�����k(�[/�z�����������~�>
�Y�sw@�.���j�b��x���!}RO}\f�~vp3gcY��C�l���_`����cGd����Ce��p\I���$�����I�g��.����H�?i�`Z�a�;a
�S�Z$c�FAzo��}��^G�P�OB��M��E$�����F l�+<�����N`�����/�����wT�l�����!J�T�S���'xP��$-L�vb��cL3Thi��8r1	�P�$������	�v�r���N(apt(��vK�������;l���������{]A���`>�jY��qz��T�k����\F���-��20���H�L�k����.�*^^�T,
�W��k^k�2������O�=�'L<�3�G��>[���+��s�@����]O������?���#�
�V����$�U��auNWN�qW#v(���7�*�
$�j���H�3�|��pEH�G;�@=�0��kT�}�����6�[F���}?���UM�h����cu�8�!�VB Ga$���wF�}�^d�Z?��W���Z'd�<�Y�|X1y���:��7��eL]��lG�G��D�J����m�H�������64�	R�!�}�u��L���r��&Q&	Z�^5�2��z0�h	�����S�M\B��xH���f���\�G�4kpn+�m��<�mJ�2F�0x](
���a,%���C�3����[
���ul��9��(���_G����F���$��H���!�B��}�����
��N���@�u=�����,s�X���#�P����N�e�����&6{F�Xj Q�o�
�hd������TgD�-���j�%���y�
�*�C�$����� ��Lg��6.v�]x��o�(�x��jr�UO�O�6�Z��EE�03-�����P��1�b���h���0�Cw�H��p+�{�8��tI�3V�����(��_2>��R6����3"�5�y�1�C��l��&?Yx]���\����)��R1U'�@h��B{o�zB�<�sy����~U;�����u5��1)Ij��z�U��-{�/�� Q#r37������v�q�D�(�,U�f��//�����-
>Qw���ZAVx����������oM����jtV�Y�P0;�(�z���������
�Td���g6��4�;_B��-S�M��W�__��{YYJj������������@���|���8�x8yf��3#
)���%�����y.L
�"���s9�C�����j;:��G���U�x��aQVI�k��n>:y}�/k���3�^�
����d"!n��{#KVIi �����R_@�Bf��9AED&"�����3�nA�q�b�+�+������`M�pS���������3
-vE-�t]�j����an�
[o��7�V\v�k�^��c&�\K���bc��I1cIUf<]=�)I��Z\�>n���`���d�J�����n�N��I��Lx�K�9f�'=f%�:�����O��]��-����_���@P��L
I��2y_����+'��q!��\����o�J2d��T�����T��,Z����v(�.H(�!t�?��;DvW|x����.�����d9["$��r���]�-E/�FZn�5 ��M;���$��e� �o�x��'2�����~�Xa�r��8��|�"\��E�l����~��XSl�;3i;:8�.)�I�T�tXM�h��t�A���5���LkVwJO�l�=�wWA�U�SN}���\~�C���ms1�:����U^���5���S�q�N�R?1~�U�V�����j$�U����Nw��i��>�SHJ�%��������+��,n���v����9v��0`��e~`N�r���<���L�52]����l	�,��$f���?�a���m����1i�
m
���I�V,SpD2L��OzD�`�v1,F#yUa�R9���s�^�I�d0V���>�$�Sq>xX�$�PlL�E�O������������PXs���'B����5��-��J��}�
�8:Qn������[y�Z��O=�o�+��,��#"\�x�(�K&|�K,zZ��:�E��'c�O�B��@�_���C��t!;(#�s(#��|�4FK���KR�"9��&���2$�E���I�������M/�Q���\hW�I��$zl |���R�@�����	4l����)���)~l<����q��u*J8�>v��r����������g�sE-L�N8{�R3��-P�0����V�T|��O��_�1���
2�` ��n�a.&7���E�q����������<&�8�����,�8h�z�����i
"����X�C�J����I����L��������q	'K��P�i��giZ�'K��~d�d:P���#�����#'w��k���G�����[�=������i��9�=���t�T�8�����Bm3��]�m~Z<Pg3�+��;�Z��/�������yx�����}R78�To����n�Z�����n%dW�;�T���-��"��T�����<�,�5��q7kj�*�a(�4?
J=dn��<�B��4��}�1x�/H����n�[��g��i�Q��9=���=���g�U%���������Hn�}�wJ6�
��[�_���������c��_)��������Y�M�����8���9UO�Z�J\j�dT�B���(�������{a����Z�R���y�;������������a�]��;
�ci�����,����~���7kj������� ��D���<���W��o�5v5Fc�
�F#;�>`�+�e6�����������bwx�H�i�EEA?S��.GJr���
��;��]cJ��8F���M`6�����&�
�K��V���'?C6N�m���c�
����t���-��"fq&.�K�X��F����V�0��NK�P"��p#�����ZV�M	���Z��d�?	��+k�^�q��m� y��#��D��nmv���))I+���p�n���5�gM�����w�F|��^�O�m;���[4��<�2�G^zIk��~���w^x���������c"����x�fa�-W��d���~0��L�"<�G:6c���S'}%��>#���k��t_��Uso��
vj��~��
	�q��=��n���I_qUaM���-�oN	DR�9t�;w����As��T�����^c,%��|�9�������'�,����o�P�pw�����0���2d����b�������E
���|��R��5������aC��4������+�;��������9u#Q��$���-�H/���f���w�~������(��^������3'�"�i����5�?R������s��L��=SPmFY����}u����\B�1���H�,&�>7���Mk�� C��Z����XCsl�rvfF��|y���V|���}-��G-��zY���HTbp�O��W�%����0fM�Y?��b%D���g�����OE5j�(�������$RX�Xt$Nh}��b��G*,��
�L������K�U���`�r���k�i��������}��y����?��7�_`�f#�t{|hq4�t��b������;��ZR��j�����9����Z]f���f
.�ur'�U��U��yM����~���.�t������|��Z2�9�$���H�'���4��y���p�t$������3I���}�?2U�(���~ib�z���v�(���3�Ny������0J��d����1m�$���25���|�eO$S
�cej��lH��v,tf��.%7�
^.
�uV����Kk���nc&
�N�s��V,�<A����#��$��VMV���J@����5Z[�\����`�B��dE�������ty��_�_P����#�{��v>�fC����sVQq��;��F�l��)�}5�����E�*�?`������s]-���3���*��B�)�R�p=I�����ul+_�� ��#���=�H,:w�TE�o�����seE��B^u�lI��+�x��GW����"�1���7o.��}K�-��Y�~qu��������#-���f�/�||���q!^1�+����O��~�������o��o��g��-_�]=�&������q�����+Cx�J��_�_}����7^���]���j�&wZm*��Rk{%TK��1�
����mm���\�1��PSu0F��k4�������I�;�_"Y"�1��AQa�����g"D�4��Y��t�ZN�cg� �R�0��ay|K��(������+��9O����V�pr ��r�]��5�Z��`��W���D�we
M��N��|�@[���.�I������A��^�����*xu69���S�Q��C�������f��Y����������]Q��:��XdO�()}N5~=�������V\�s�c��^;�����uH��cy��n)��7�>#��8��*�ip�%�����f��:���M�b�����i�},�@~�a��bE�p@5�v��e����f�#J/��V��s�J���jz��%X�����@�]��@�LX1�#����r�N�P\����@h!1f"`�w�M)�64}{X;�,���^u�p��C���Cg}��oh�;�<���dcJK�qrV�������f�g��-`�<�R��q���Kv����0O��a��.V%U�,��w����`��C����;$��0v��s��h\��Lo���Qx
��0��;?���{��o[K�=�u\��2xG9��R=Nj����}x���W5�f�uH�.1U4�|3v�"��Q�\�VJI'������?��w"��w��,�y
Z�0��d���L:�1���r�P�K���s���hR�B(�%����>���(|��Y�����;w�T�p`g�u#D��}��n�{i�]��`��<��2��w�;����`Z�U�R�o��Q��2�83=��c[4!����'�����Q�)�����9r�~C��`68���{��%!6Nvv7J���28���c�����7K��%+
C�B*��>����b,����38]����Z��n3��'m �`��@;-���u]��
,�1���n���~��$��������hg�;�;4FM'�������;�)�L�9���5���L�O�cgM�)��9^��O�Ey�.�����j��`_zL������nt���0'J�u�*�^�����ez��;v������Mw1��W�x�M��[�����O�)��yx����d��}��M��j;���*H��z��xr����w���v��SO6�����_��\ �	��&77#���p�\1�+)���/�KX�b�WJ���&��ln�l������1�y"APr����)�������f'wvOl�W��.�<���ORlu��\����0�%4;�o:V�
_cp?	���F��X��5yS]��uo�|�����)3b�Vl����U�*�Jk{�>x41�i8<����v/����g^���������r����W�O��'/���O�r�b�����Y3y�q;9��F�g�����|\��?�g�z�>ku���a����������vvv3�������Z���,��Ps��1Q�9�9��#?i�i?�&��j>���}E��1_�����7u�Q�I�L� ��|�~B
#>�����4���C�����G9���������:)���O�9�M��}���9K5*
#250Rafia Sabih
rafia.sabih@enterprisedb.com
In reply to: Ashutosh Bapat (#246)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Sep 15, 2017 at 5:29 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

On Fri, Sep 15, 2017 at 2:09 PM, Rafia Sabih
<rafia.sabih@enterprisedb.com> wrote:

On TPC-H benchmarking of this patch, I found a regression in Q7. It
was taking some 1500s with the patch and some 900s without the patch.
Please find the attached pwd_reg.zip for the output of explain analyse
on head and with patch.

The experimental settings used were,
commit-id = 0c504a80cf2e6f66df2cdea563e879bf4abd1629
patch-version = v26

Server settings:
work_mem = 1GB
shared_buffers = 10GB
effective_cache_size = 10GB
max_parallel_workers_per_gather = 4

Partitioning information:
Partitioning scheme = by range
Number of partitions in lineitem and orders table = 106
partition key for lineitem = l_orderkey
partition key for orders = o_orderkey

I observe that with partition-wise join patch the planner is using
GatherMerge along-with partition-wise join and on head its not using
GatherMerge. Just to make sure that its partition-wise join which is
causing regression and not GatherMerge, can you please run the query
with enable_gathermerge = false?

That does not sound plausible since around 130s are already spent till
append node. Anyhow, I executed the query with enable_gathermerge =
false, and still it is taking some 1500 secs. Please find the attached
file for the explain analyse output.

I see following lines explain analyze output 7_1.out without the patch
-> Sort (cost=84634030.40..84638520.55 rows=1796063
width=72) (actual time=1061001.435..1061106.608 rows=437209 loops=1)
Sort Key: n1.n_name, n2.n_name,
(date_part('year'::text, (lineitem_001.l_shipdate)::timestamp without
time zone))
Sort Method: quicksort Memory: 308912kB
-> Hash Join (cost=16080591.94..84447451.72
rows=1796063 width=72) (actual time=252745.701..1057447.219
rows=1749956 loops=1)
Since Sort doesn't filter any rows, we would expect it to output the
same number of rows as hash join underneath it. But the number of rows
differ in this case. I am wondering whether there's some problem with
the explain analyze output itself.

Limit (cost=83341943.28..83341943.35 rows=1 width=92) (actual
time=1556989.996..1556989.997 rows=1 loops=1)
-> Finalize GroupAggregate (cost=83341943.28..83342723.24
rows=10064 width=92) (actual time=1556989.994..1556989.994 rows=1
loops=1)
Group Key: n1.n_name, n2.n_name, (date_part('year'::text,
(lineitem_001.l_shipdate)::timestamp without time zone))
-> Sort (cost=83341943.28..83342043.92 rows=40256 width=92)
(actual time=1556989.910..1556989.911 rows=6 loops=1)
Sort Key: n1.n_name, n2.n_name,
(date_part('year'::text, (lineitem_001.l_shipdate)::timestamp without
time zone))
Sort Method: quicksort Memory: 27kB
-> Gather (cost=83326804.81..83338864.31 rows=40256
width=92) (actual time=1550598.855..1556989.760 rows=20 loops=1)
Workers Planned: 4
Workers Launched: 4

AFAICU the node above sort is group-aggregate and then there is limit,
and the number of rows for sort node in explain analyse is returned
number of rows. So, what is happening here is once one group is
completed it is aggregated and fetched by limit, now there is no need
for sort to return any more rows and hence the result.

Apart from these there is a regression case on a custom table, on head
query completes in 20s and with this patch it takes 27s. Please find
the attached .out and .sql file for the output and schema for the test
case respectively. I have reported this case before (sometime around
March this year) as well, but I am not sure if it was overlooked or is
an unimportant and expected behaviour for some reason.

Are you talking about [1]? I have explained about the regression in
[2] and [3]. This looks like an issue with the existing costing model.

[1] /messages/by-id/CAOGQiiMwcjNrunJ_fCDBscrTLeJ-CLp7exfzzipe2ut71n4LUA@mail.gmail.com
[2] /messages/by-id/CAFjFpRedUZPa7tKbCLEGK3u5UWdDNQoN=eYfb7ieG5d0D1PbsQ@mail.gmail.com
[3] /messages/by-id/CAFjFpReJKSdCfaeuZjGD79hOETzpz5BKDxLJgxr7qznrXX+TRw@mail.gmail.com
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company

--
Regards,
Rafia Sabih
EnterpriseDB: http://www.enterprisedb.com/

Attachments:

7_gm_false.outapplication/octet-stream; name=7_gm_false.outDownload
#251Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#247)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Sat, Sep 16, 2017 at 2:53 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Fri, Sep 15, 2017 at 6:11 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Thanks a lot Robert.

Here are rebased patches.

I didn't get quite as much time to look at these today as I would have
liked, but here's what I've got so far.

Comments on 0001:

- In the RelOptInfo, part_oids is defined in a completely different
part of the structure than nparts, but you can't use it without nparts
because you don't know how long it is. I suggest moving the
definition to just after nparts.

- On the other hand, maybe we should just remove it completely. I
don't see it in any of the subsequent patches. If it's used by the
advanced matching code, let's leave it out of 0001 for now and add it
back after the basic feature is committed.

No, it's not used by advanced partition matching code. It was used by
to match OIDs with the child rels to order those in the array. But now
that we are expanding in EIBO fashion, it is not useful. Should have
removed it earlier. Removed now.

- Similarly, partsupfunc isn't used by the later patches either. It
seems it could also be removed, at least for now.

It's used by advanced partition matching code to compare bounds. It
will be required by partition pruning patch. But removed for now.

- The comment for partexprs isn't very clear about how the lists
inside the array work. My understanding is that the lists have as
many members as the partition key has columns/expressions.

Actually we are doing some preparation for partition-wise join here.
partexprs and nullable_partexprs are used in partition-wise join
implementation patch. I have updated prologue of RelOptInfo structure
with the comments like below

* Note: A base relation will always have only one set of partition keys. But a
* join relation has as many sets of partition keys as the number of joining
* relations. The number of partition keys is given by
* "part_scheme->partnatts". "partexprs" and "nullable_partexprs" are arrays
* containing part_scheme->partnatts elements. Each element of the array
* contains a list of partition key expressions. For a base relation each list
* contains only one expression. For a join relation each list contains at
* most as many expressions as the joining relations. The expressions in a list
* at a given position in the array correspond to the partition key at that
* position. "partexprs" contains partition keys of non-nullable joining
* relations and "nullable_partexprs" contains partition keys of nullable
* joining relations. For a base relation only "partexprs" is populated.

Let me know this looks fine. The logic to match the partition keys of
joining relations in have_partkey_equi_join() and
match_expr_to_partition_keys() becomes simpler if we arrange the
partition key expressions as array indexed by position of partition
key and each array element as list of partition key expressions at
that position.

partition pruning might need partexprs look up relevant quals, but
nullable_partexprs doesn't have any use there. So may be we should add
nullable_partexpr to RelOptInfo as part of 0002 (partition-wise join
implementation) instead of 0001. What do you think?

- I'm not entirely sure whether maintaining partexprs and
nullable_partexprs is the right design. If I understand correctly,
whether or not a partexpr is nullable is really a per-RTI property,
not a per-expression property. You could consider something like
"Relids nullable_rels".

That's true. However in order to decide whether an expression falls on
nullable side of a join, we will need to call pull_varnos() on it and
check the output against nullable_rels. Separating the expressions
themselves avoids that step.

Comments on 0002:

- The relationship between deciding to set partition scheme and
related details and the configured value of enable_partition_wise_join
needs some consideration. If the only use of the partition scheme is
partition-wise join, there's no point in setting it even for a baserel
unless enable_partition_wise_join is set -- but if there are other
important uses for that data, such as Amit's partition pruning work,
then we might want to always set it. And similarly for a join: if the
details are only needed in the partition-wise join case, then we only
need to set them in that case, but if there are other uses, then it's
different. If it turns out that setting these details for a baserel
is useful in other cases but that it's only for a joinrel in the
partition-wise join case, then the patch gets it exactly right. But
is that correct? I'm not sure.

Partition scheme contains the information about data types of
partition keys, which is required to compare partition bounds.
Partition pruning will need to compare constants with partition bounds
and hence will need information contained in partition scheme. So, we
will need to set it for base relations whether or not partition-wise
join is enabled.

- The naming of enable_partition_wise_join might also need some
thought. What happens when we also have partition-wise aggregate?
What about the proposal to strength-reduce MergeAppend to Append --
would that use this infrastructure? I wonder if we out to call this
enable_partition_wise or enable_partition_wise_planning to make it a
bit more general. Then, too, I've never really liked having
partition_wise in the GUC name because it might make someone think
that it makes you partitions have a lot of wisdom. Removing the
underscore might help: partitionwise. Or maybe there is some whole
different name that would be better. If anyone wants to bikeshed,
now's the time.

partitions having a lot of wisdom would be wise_partitions rather than
partition_wise ;).

If partition-wise join is disabled, partition-wise aggregates,
strength reduction of MergeAppend won't be possible on a join tree,
but those will be possible on a base relation. Even if partition-wise
join enabled, one may want to disable other partition-wise
optimizations individually. So, they are somewhat independent
switches. I don't think we should bundle all of those into one.
Whatever names we choose for those GUCs, I think they should have same
naming convention e.g. "partition_wise_xyz". I am open to suggestions
about the names.

- It seems to me that build_joinrel_partition_info() could be
simplified a bit. One thing is that list_copy() is perfectly capable
of handling a NIL input, so there's no need to test for that before
calling it.

partexprs may be NULL for FULL JOIN and nullable_partexprs may be NULL
when there is no nullable relation. So, we have to check existence of
those arrays before accessing lists containing partition key
expressions. list_copy() is being called on individual array elements
and "if" conditions check for the existence of array.

The functions might have become complicated because I am using
outer/inner_partexprs to hold one of the lists and partexprs contains
the array of lists. We may use better named, but I don't have any
better ideas right now. Will think about them.

We could simplify that function according to your suggestion of
nullable_relids. Basically partexprs then contains partition key
expressions all relations nullable and non-nullable. nullable_relids +
pull_varnos() tells us which of those fall on nullable side and which
ones don't. Is this how you are thinking of simplifying it? If we go
with this scheme, again nullable_relids will not be useful for
partition pruning, so may be we should add it as part of 0002
(partition-wise join implementation) instead of 0001.

Comments on 0003:

- Instead of reorganizing add_paths_to_append_rel as you did, could
you just add an RTE_JOIN case to the switch? Not sure if there's some
problem with that idea, but it seems like it might come out nicer.

RTE_JOIN is created only for joins specified using JOIN clause i.e
syntactic joins. The joins created during query planner like rel1,
rel2, rel3 do not have RTE_JOIN. So, we can't use RTE_JOIN there.

--
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

#252Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#251)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Mon, Sep 18, 2017 at 8:02 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

partition pruning might need partexprs look up relevant quals, but
nullable_partexprs doesn't have any use there. So may be we should add
nullable_partexpr to RelOptInfo as part of 0002 (partition-wise join
implementation) instead of 0001. What do you think?

+1.

- I'm not entirely sure whether maintaining partexprs and
nullable_partexprs is the right design. If I understand correctly,
whether or not a partexpr is nullable is really a per-RTI property,
not a per-expression property. You could consider something like
"Relids nullable_rels".

That's true. However in order to decide whether an expression falls on
nullable side of a join, we will need to call pull_varnos() on it and
check the output against nullable_rels. Separating the expressions
themselves avoids that step.

Good point. Also, I'm not sure about cases like this:

SELECT * FROM (SELECT b.x, b.y FROM a LEFT JOIN b ON a.x = b.x WHERE
a.y = b.y) w LEFT JOIN c ON w.x = c.x AND w.y = c.y;

Suppose the relations are all partitioned by (x, y) but that the =
operator is not strict. A partition-wise join is valid between a and
b, but we can't regard w as partitioned any more, because w.x might
contain nulls in partitions where the partitioning scheme wouldn't
allow them. On the other hand, if the subquery were to select a.x,
a.y then clearly it would be fine: there would be no possibility of a
NULL having been substituted for a proper value.

What if the subquery selected a.x, b.y? Initially, I thought that
would be OK too, because of the fact that the a.y = b.y clause is in
the WHERE clause rather than the join condition. But on further
thought I think that probably doesn't work, because with = being a
non-strict operator there's no guarantee that it would remove any
nulls introduced by the left join. Of course, if the subselect had a
WHERE clause saying that b.x/b.y IS NOT NULL then having the SELECT
list mention those columns would be fine.

- The naming of enable_partition_wise_join might also need some
thought. What happens when we also have partition-wise aggregate?
What about the proposal to strength-reduce MergeAppend to Append --
would that use this infrastructure? I wonder if we out to call this
enable_partition_wise or enable_partition_wise_planning to make it a
bit more general. Then, too, I've never really liked having
partition_wise in the GUC name because it might make someone think
that it makes you partitions have a lot of wisdom. Removing the
underscore might help: partitionwise. Or maybe there is some whole
different name that would be better. If anyone wants to bikeshed,
now's the time.

partitions having a lot of wisdom would be wise_partitions rather than
partition_wise ;).

Well, maybe it's the joins that have a lot of wisdom, then.
enable_partition_wise_join could be read to mean that we should allow
partitioning of joins, but only if those joins know the secret of true
happiness.

If partition-wise join is disabled, partition-wise aggregates,
strength reduction of MergeAppend won't be possible on a join tree,
but those will be possible on a base relation. Even if partition-wise
join enabled, one may want to disable other partition-wise
optimizations individually. So, they are somewhat independent
switches. I don't think we should bundle all of those into one.
Whatever names we choose for those GUCs, I think they should have same
naming convention e.g. "partition_wise_xyz". I am open to suggestions
about the names.

I think the chances of you getting multiple GUCs for different
partition-wise optimizations past Tom are pretty low.

- Instead of reorganizing add_paths_to_append_rel as you did, could
you just add an RTE_JOIN case to the switch? Not sure if there's some
problem with that idea, but it seems like it might come out nicer.

RTE_JOIN is created only for joins specified using JOIN clause i.e
syntactic joins. The joins created during query planner like rel1,
rel2, rel3 do not have RTE_JOIN. So, we can't use RTE_JOIN there.

OK, never mind that then.

--
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

#253Rafia Sabih
rafia.sabih@enterprisedb.com
In reply to: Rafia Sabih (#244)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Sep 15, 2017 at 2:09 PM, Rafia Sabih
<rafia.sabih@enterprisedb.com> wrote:

On TPC-H benchmarking of this patch, I found a regression in Q7. It
was taking some 1500s with the patch and some 900s without the patch.
Please find the attached pwd_reg.zip for the output of explain analyse
on head and with patch.

The experimental settings used were,
commit-id = 0c504a80cf2e6f66df2cdea563e879bf4abd1629
patch-version = v26

Server settings:
work_mem = 1GB
shared_buffers = 10GB
effective_cache_size = 10GB
max_parallel_workers_per_gather = 4

Partitioning information:
Partitioning scheme = by range
Number of partitions in lineitem and orders table = 106
partition key for lineitem = l_orderkey
partition key for orders = o_orderkey

Apart from these there is a regression case on a custom table, on head
query completes in 20s and with this patch it takes 27s. Please find
the attached .out and .sql file for the output and schema for the test
case respectively. I have reported this case before (sometime around
March this year) as well, but I am not sure if it was overlooked or is
an unimportant and expected behaviour for some reason.

On completing the benchmark for all queries for the above mentioned
setup, following performance improvement can be seen,
Query | Patch | Head
3 | 1455 | 1631
4 | 499 | 4344
5 | 1464 | 1606
10 | 1475 | 1599
12 | 1465 | 1790

Note that all values of execution time are in seconds.
To summarise, apart from Q4, all other queries are showing somewhat
10-20% improvement. Though it is good but honestly, I expected more
from this feature atleast on this scale factor. I am yet to compare
these performances with the unpartitioned version of the database.

Please find attached file for the output of explain analyse for all
the queries on head and with patch.

--
Regards,
Rafia Sabih
EnterpriseDB: http://www.enterprisedb.com/

Attachments:

18sept.zipapplication/zip; name=18sept.zipDownload
PK
}{2K18sept/UX���Y���Y�PK�{2K18sept/.DS_StoreUX���Y���Y���_LE����������z)"�\�8�j�8���w��6.����]�1��5���/����F_4��X��E4)Q_|��DM����90}04v���;;3?nv������j]AP�;��-���5����,@����[�/))))))�]rS*vwRRR7���CPxX�2w�>���b������e�(����
W�S�A�a����E���/�X� ��]������6���y�jOoX���%�N:a�	���D�4��s}%�{��*���������);uU7���S�-C��<���by�������2�|��
�����Y:3,���YKs�De���T���g�W-}�p��a�
�>q����
�\
���Y���gi��U��C=�'f�K�������]x��|�W��4=�w������������6=����S�u��tSu��Q]���}�����%v$������Z���������iF�'ug����^i��3�p����b��8���O�]���)=�0'�be}�@K���{�����}����X��)S��{�8�:I��k~�M���
=C�K��	K]�yT�?�+s�	[��h���YQNy������$�?H��L8lh������������$�P�L�jB7y�]�K��X���Wv�#J5��t�^X`#txu��_�_+�x�}\u��i�b?��}�����KE��"5M���p;?�-�N^�^��!�Q��$X���Ex	����g�%|�����*�?���;���F�+�
k�	o�x����N��.��0�0F�8N���,&���1�O�s�2��o�y\��C�?�����H%^�,�Y��m�e`�v,�+X�Vz������^�9�vt7��lp��8*rR u#�
�@i��I��)pE�����A(����@������&��<�i	o-�����2F=�i)o�#������p�����[�P��m�;�����w���g�~��neH�1�pk�;�
a����Q��~���1|�8���1�;Oc
����<.��x��g���9|_�77�wl~v�/�X�m��K����Wr�^�dz����#��%����)F���8����$#%%u�*�����l��_JJ�,,�D#C�}*������b��"���+�������2@�����M���PK?���(PK
�v3K	__MACOSX/UX���Y���Y�PK
�v3K__MACOSX/18sept/UX���Y���Y�PK�{2K__MACOSX/18sept/._.DS_StoreUX���Y���Y�c`cg`b`�MLV�V�P��'���@�W!��8��A�0]�PK���4xPK
{2K18sept/pg_part_head/UX���Y���Y�PK{2K18sept/pg_part_head/10_1.outUX���Y���Y���m�%�q%�����fyCB�w�j"${f<���+���~R���"Cd7������o��n�pu�Yzo��U"���(d� q����_��?������������KY��W�����W/�����~yy���O�����h����8�6�-����|�5������_���7/�x���O��{����~m���:���?9����?���O?|������9�����O_�zy���/�����z�{������r��>����/_~������������/����F}��������?|��������������_�����~���W/�����~~����_����!����������w_���Q}���O�����?�����������������W����_>�����Lv��d��8�)����7����������L�������^��?������_����W/_>���z����>�������#�`��~��,�����w�}�??���?���o���?��������;J�����A�)m^6������_����}����
G6�����;����������zdZ;�9c;�a���w_���#����[A��������p�\�����gZ.�����6�X!�_�C�?��w����?^����}��O~���������;������w?}|����!��?������y�D�/�0�u��|p��9�����|�`������wg��n��3�'o��;?��}v��~����`������n	p�`�aBYAGRDzY&�z�|����4���cRv��7��A^����4�=���`T����6�j^��p����������P��������D�?��7o��6��;����uM:t]v�d
L�:���O�R"r�!����q�{>����E��\��kv���3z������������������'�����s|�r*k[^m���, ���}3H��Kt��&
9��r��A:A �]LWH<�b!g�m>=H'��Xq�"��_���C���  ��������O�~\���P����1>3����t&�9��3�'����?���>0��|�����n�>���^����������O�?x*�1�mV!���\�Sf��������}�]���d;#e��3��D��d�j>������W�#Oz��Hv���
�gO��|
iO�e}�����^]�gj�����p8c��ov��%��#��O|N�J�l�n�&��R��z:��7��[J���aj c8D���p:��'�f4������h�[^:L��L:��3�G��t#ik#�)S��sj���v����~4f����TeZM~�y{	>?���S���������|�_�����/�>�������������-�	Rv2��y/1����xGg�H�QE�,������>���T8n��D���?}�C�6`����O?~��{�G��;>������������;����}��������m�"/w����w���w�3�fIOP�s�Y�.?c�c��Je���MK�'���7�x����y��CU�b�����3���L����Hy~����>'�&�}#-luH=�y���a c�8CMxp��=��z0�<�<�.�`�I�8��������%�zO#V�9�k*����y�'��J|�����j����?g����>\���B>D?��v�������|����eo���E��D�g��@����k�z��y��^{����������10�#i�e�K����{/!%��:�q��<��'��`�}��f�u�d���������+�~?R��QN��4o��pV�8+���O8D���r��w[����K�z8Xvw�����	8��r�5z���sYd��L���;���j�P�L��|�^R������-{��H��BC��F�4���F�S��q��r4V��[�rKf%����j�Bj�c�fdlgoe�0�[�]o�=o�����������t��0&$��V6��H1vD�{+��#���,i�\�xg�B����g�i��}��<X@F������\qg
-f��n���{8�'��s���`��������s��������g
���:+y�\f�	�����>'�Aq@0%���|Ay����r=������h��F��&�������
Q����#�����2�c����M<1����D*�eo
i��5�u��s�(�������9�l�s�H3�1����m�[1�Ok#*�cC��N`o��y1f��d���>��U��`A�%�9g��5�{kPKx2�3�����{8+�`��q��5�u��s�$������{���	�@�\8I.Y#��X� �����>K�����)��� ��5Jq�(��|j
�v�Ys��9�F?[y���n
<EesM�Y���^x��<6������,���������kp�H'|2��qN60 �C.<��W�G��&�`��=>Fe�3��O�e<0Thsv>0'k�=��o�-��L+>i�y�5Q�'�i������,;�r��A���yl`L"��&��r�M��`�����-���}��������J�
�����Z'~��6�{��$��G��,�yNv�����;�w2���z	)=����q�2�	�v_7����e'���Dr*,�
\��l�
������v�\�k�n����D�'�p����(v���
k9�j���c��[�Pq�U��}-/���]�
�%�q������X���-�D�ng�x��]��������'��	�mQ����l*i)��S����o�[�^/��s����w	Q�G�2�(s%!����w��L��D?2
�2ho��y���
�A�!�<���}�p3������>�$)7_�d�Jj��/�f�
�p���?c��%�i�P�s�&�����&K��E�#J���}��"��s��wQ���2�(s!#���@�-����d��fI���W"c��b��1�\w'�C��{�-��#��@s���������b�l
������K���sC�b�������y�M]1HA)n�J��J����f]��)��H����#e���������:�����H]�#m��Av��ee+��e���[����A�Q�slk-K������	�m�R`���R7���u��P6���P�0�}���L�4eHS�J�X�#9�6(S�����g���a����64G"���D.�#���aCt��+%j.p������g���hAM����lS�����A�Ju���##������K(�����}�p���K�2=��Q�iT�b������,�V���1"T*���n�QO�y�M|,p�:����'����|�C�S��b����a�C�|������aT6���\ui�*q&@�%o�q�}�k���L�VehU�JZ�Iq(U��������J��I<�r�aK��Uz�'����7�
�>�lj��/��E��r�k������@�^����k�p��ce�Xq���ct�e�d[�
#������"�a�{�hezD+#@�2���������Bcf��v�!�j��x�4�<^^�T�%2��}!�%��pv������d���tU�*�4���}6�p��ue�#.��!�",.M�A���qs���2wYW���2�+s%���l#w_��2�Q�D�tC���%v_2������BS�[k�F�o�}gQ�z ���$�l.�f���b@
��`((���c��N���K-�������92�����UB)��s���+���H�vEW����8�>� fI�B�
�����c���5,���3V�Yd%���
n������3S�3����o5%w���-�9�}$Nz�)�7RT���_��S��.�E���G�g�K����hWt)��8?�����X|^R�.?+�t3^s���84�����K"mj�����n���n�918��}y�+��<��^� Cp��gjhWZ�f�9��^��7��r�]�a��Y�yxI ��]Q�vE�+��v��yd�����.c������!���}��H�I���e7Q�}��J��������j����n�k��A&#����9����/��+�����B�_�RtN9���AZo������]Q�vE�+��v�+ph�l�"�`�Wy�L��Q�i��yE%lx��q�]�$����r��G�-*y�������M�	��9�]�����T<�Zr����{����KU���a���4�������G�"�]I�2N�Y(���*�!�</o$g%.~1^!���r1j��p�\�n���/�&H��#5����H����!k!��
��@�[/g_TuK�p����?yT^���4Z������G�"�]K�r#SW�d���������I�6P9�$�Cc�y7���Plv�2�+K�)�iL79�g	�$�3B�5"���Z�����������k��j{GE���7e�e��7�IGC�]��hW$@��KiW�F��!X0�
M�����+�HK�/�t��08Sv0����"%G�}��O��5�c<Z7�hm�f�\lGRq	��r���J�*��Ak����&���A(l*�����+���H�vEW��H���H���������gT��������Hd���h�%#q��%I�g��d��1�A���6u��R�%�:���*�miW6�m����6����A*�<�����+���H�vEW����:2u��U(e&�A��K�*�gI�7�5�������R�`!�*P1X�KeIB7g�<��E*y�J���b!;�l������+,����h�H��,d���L4���A����3�wiW��]�����]����hn����OfQ�?�=��5:�3���7�9���w�iN�f4�������C�]��u�|C�����[~�)=@
�ph�������d�G9�G>Y�r��E`��}��fM���+�c]Y����u��~h��.��>;���	f��q[���������z}����4g_m�����J/$kG�H�/~���� �,���g�O������uv_��1��Y���1���o����h�����=��`]�+YW���r�d���p/J�Y����<��xM�
K��u��qi��C$ue'm��$-��M\%RW�������i�}���Y������&��y�4���\�;�-s%��}�]��������^��o��:���+_t�]R���&=jJf����j$J�0�4��ii������A��r����&#�,�8����QQ�^K�0|g�uh����&!c���8�w/�l���
\�+/��O#�}�S{��Rr��������/����{�S�<^-"������,`�%�]��e�R��a^�b9-^qd��#P��3T�m_��x4�R�������J�s��G�wqd.�����V�1V�AKw,|��Kcu'	�g���g,d��h��$�&�q�]�$����`o���o�}���R�e�����s{q9�
i�j�'M��K��8���}�����{qd{GV���^zq�����>�C��wHr,�~�'��UF��AF����k��pM��=��O������U)6��Z�U��n��^�Kp�"����`-4�.)���P-t���d������G�wqd.���G��������Y;�!Xv�8��zFs�"��<|j.�0a��e+��r�RB������
Q&u��YC�6���_J�C�����#�����/�h�bW��T�V�����h���{q�zGN���]�%%��{���|!1R\y������eo�k��1^M]Y�'�TR�/��e�\[fe�l��Ch"M�~�f���;_1h}�i�	��_�����g�|�}�4�k�q���4��h���^�����8r�^Q����*�F�R>��J���,Jj':������B��X�&�$���OG%�}K*�p��/8����8-�k��/�tH�T�Pp�������L��Ck��H������������9�{#wi�~
C3W��*�1�Q�-�+�]�{�x�!;���D���X�X�r�MM`��J�9d	�s�}�� ��h9J�&�Oxo[���y�P���D���=��=Y{E
�e���j}���w����j}���~�8_�&�Vh�g���] ��p4��9����r
�Ni��i�R]��>��Zo����\[��$�}�t��<�IJ.��@_�b�6��j}���w����j}^#9��dE(6:�i�qs�7������]�kf��a��n(����<�P���:�E���xE��A�b�%X��o[����
�F�]����-�bI�����G�6��j}���w����j}?�tU>YFL ����x�
�z��u���[c_X���MR�xT��Q{)�
F?�����������a�_�x~��wO���r}��0&��[_!l�hR�I��*������r�\�s�8W��b�0;�OV�}�A]���(9�f�����X��@F��W6�D�o]�%\�����Nd��g���&4�����ns�}���]2W'��?��]�4�A6���.4����\�t�HW��j�`����Q�k�U^IIPL<c�����_�U�	CH^
x.��6e�k�<���$8��M}���K�P����C'���I��b����)�9�mk,��x��#���k���]��������]J�2q �j�d8���zUVcx)�����Y�gR����<Dv����Az������goGd��\�+�����G�����O�����y���/�a#�Oy^�o�}����]��������]�$���=�lD�h�F����}�H{�2�.�2��i�:����>�xhz9�<���w_>��J��h.��&�g�w_@����<k�����T	�s��U�.������|�t�HW�R��	#9��z��e@zU����4�����r������r�����@*���%�]��{T���;�6l(�F�'���o��!�`���2��������rn���F�g�u�{�+/����6I�f�����Y�H�Q
C�[[�3�W�}z�y���"r���`9���o��dXW������/:����A��:y�92������:]11 #����]�����|�v�hW�J��vv��>Y,�2Y���s;9`+��1�N��b��&2r��3�3^dm����+���@��:�ZW��+=��7�l*���4I������9�kWi����\�g�W���h���K��=���]�+iW|��z�'KT>c>7�%�t�k�&��)7p�L������#Z�N���{X�4��f�Fi�
o��B���-��&}C�2hH���W����}�TH��[�����wiW�G���+%����@����r�5��r]$v_��@KJ C--���aH��~�-�!�t�����	I�`���P��~l���Uzv�?��
�JgT\��8�w����2��N�\�����G7I�wiW�G���+%�J��g_�DB
z��Vf��[z�+��x{��V�:�����^.�$��(��<
�a��#���������q����?8����n���@�9��P�E�D�����x�fx������=���]�+yW����}m@IJ4%]��D��t�[����]7�r������!��������23��t�.�����:^���"���}������6�+��W���6��������$���d��r}�v�{�+/@���j]���g�N���C ��d��D�rx��m_��f���?�����r�����$��������]m��'�7��oK�Bbf=����&)���V���Y�A�/����]��������_I���kG��8�xm*���93����&u�0^�h��-�,}�+�gJ�K����X����76�+4��� v5��v����]�j����B�w���|�nm���6�.���xW^�w�/������AF��O'>��n=���	��2�r"�(P-��y����x-zD�9���_���,ss��%������y�D������t(�dE���=���x�PPB6@�yY!���
wyW���
��p!�
���7G��+�Z�g��Kj��������gK�����i�8�	�o�}�JD|�_
��7Q"wU~�S"�������shhW�*Rs�atI��N�
'�R���Qy�����h��p�vz�� @�
��tNn��z����?���i)ZpB7GP�E'Jb�F���sQ�����/*��L�apC�����*�3N���!���M�	�m������`��k�����!P��X��z�5�/~�]�U�����*\�v����5G"2H����T��u(�km�n0K8_-���g��7�55GR��,IK/���"���g��lo����d8�]1�Z����:�������R��B���}i����]��*�����+���O��'Ai���Q$wU�CU�����H[��4U���B�}8k/��:���G��5����L�"�������J������9�:tLw��%>8)���=�}���B�vhW�R�?���~�0�$���nqD��2����/�I����������(���$3`���D5^o������a�>d���WO���ve,$g&{D���R�c�QT8g,�5EK�i�.�*�XWA�u�d]���*��$�Q.��a�5o���e���D�MvUd�y��Y�H�����BC�7����k�~-��[���N�u�M���&��[!�
t�HM&=�}���B�uXW�R�+�#YW���j'y��������������/�L*L�"S"�x�x�.I�f�x������"����z9�_����A�A�yM<�����$.�ir{e���ax�o���
=�U`]�+YW:��r��i*wn���5*%v_�W�k�/���ar"���sv�}q�_�
�!X��}W�U��2v�iOF���XW��Q�r�A��oStm����k�#��{�]�U�����*\I�2n��QQr��`&����>i�>?�=��f�r	�UN<�������&�Cj���dR���b#�O+�����f��
mXO]���c����U���
w�V�G�
d�p)�*�dk��B6�$!��[����^�����(�~�f���?�o�f0�Vt��Jt+������_>�#�����Y��o�����<����&v������n�hu���b��������WEzU�T�*������>�`�0�������z������"�;d���c�hl��g�C����!���K?��������O8�i���-������g�w�je�z-��s��]NU�q���*^�@������������;��7�eu=d��2�(����.��dt%Y.����F��$g�9���F+'��@�Sy!b�F�G��G���W��Ux���%Q��*
����$*�^�F���{Y��_��!���B��`����)�o@1��` ,�UbpCz15��(t�`I�M������W����%��_
4g��+�"t5����UgM�����cME�T��5��@�#������fi����L3�����N����uqb>�_�/C���~ �Y����s<bXn(���x�_]����=.���o�eJq����K|�u�4�zxC�x�&{4�(@��W���y�����&/����5�!.d����`�����G�c�*�3��N����D]���6.Y(L,���g��O��?�WRs-T)��"J
Gn�9|*�!F��]���.1*��QQ�/%F��h���*�
��?��������UX��
:RR��L.+�����U�!
�����4�� }'��C���c��-9�������4`��Zq��%n8����=&T`B�K�P|�x��+����R�VK�� ��|��������$�!�`�@T�5<��(�kX�M�.����7H'�5���G��[	,����{649���0���z��{��(@}�WR���9�x�A��Vk�&D�v��G���Yt�o�"5�t�_���)��m��7�E"x���!����7>[x�_S������#.���&��]tq��KQoj���b���N�R�)��_u��.��VrZB�m�M��:��"������|�-x0��R��G&�T�C2hL��A����'��%91Sv�W��n~����Z���"�wIN�Gr�$�x)�)��������-�<>	�I�_�`��]Ny����Wly��fY����:�"QF��Y�"fg�8�)�$��?\���h�����%�� ��D=����r�R���8N�R�S���9�\�3v��:��3������8��d>�Y�����k���f�Y��i���+�v�=yG�O�D�&�O�k�o����
W�W0�����Z���pk��j��oJ=~S�7�K[�9��LSYhA��n�'��FU���7�����!�UeS�e���#��qV�I��.���h�{!�g��p����3{X�����GHD�2�:�	b��;�]�S����)]Jp�#�u�Q���
��qv��T0*�M�l$B�? L��h�����r 8���T8��{�����j�@lb_Zf���r�w���9��e���R���N�R�S����W)��7�^�N�0��a�Fg��h	���Z�_16X���,}D�gO�_1��fTZ������������6�x/�T�|4*�������.�)�NI���.%8�0v-X��JH��&��?H��4*��h�T�Z�����������v���A��L#�x�"����lj	N�O~%LX���^��RY�@���:�6�%8��)	�������.3(S8�y�ZS�����j��h5��&���|����2����;Q�n��F"��|����+�����f`%��U���)�n�z�c��M����#8%�S���dF���*��p
���T�vH�����l=��F��8���<X*�GM}
+���3���Vxp�\�7s�O7)<��-������$$��_l��G;��8��^�KpJ=�S 8�K;�i3����K�^�����(!��.������I�_�S�e��,�����(�o*��������}GN�kKp���K��8�'��������-��]�S����)]Jp�i �����] *��e��������A��k�WF
�6��K����`-X�`=Pq��R/:l���5����\��_5���]�S����)]�2/������
Y���� Ep*���A�C��p!����xz<�f�����5={2�_sKp�-�Y����n��)�!�5lh#�G��|���{�,@p������=�@�Mo9$��R&�v2:EK���E?�0 p�e����q�cd �
[��Q���o��Z�p����.�n�]K�y�������kZ.��q��p�=�S`8�+N���
��1�
�"��Y@��P��Yr��9��E6X[V"���"�fiH��v�M���&�������N'����dP���k�����0=�dY�;�����r���N�J���#/t&�\>N��E^������7�5��k�����;�D�<�e:��m^���!Z7���V{��01�l*zV ����'�~*�	b����E��C{�`G'��]�S����)_Ip"����(LxT�,L:���5�J�����E.�Z?�KH��&��4�d���
IT��4j�c�k�]�>�c��
l>`8EFKk���u{��Uti����G������c8e�S����G2��{Q�JuA�8W����@�fK}4O�J���,o�f��d,�G�}d�Ik����,G��8���sO�k�pJ|5k�N�}IS{96Z����o�wN��p��|%��-�A~��>x��-y�(�����F���LI4�h�aLPA�h�����qvH��1Q��9���K���Cg$���b�H�h��
H'�*�����t]���2�r���N�J���a`�zP������jq������0�g��h=g#�uN�J^�����
$�P�S�S5m�b.�g�['��e8E��{�A���������7�:�%�n����p�=�S`8�+Nd�XI�ru�CPi)�����F��@����D�3��L�I���p��^����m"w3��
l�O�q�F�D�:�e8����nJ���sa?E�Z/t4
�����c8e�S��_]�%�fv��������E�	���ZQ�������N����E��!��"����w��X�X\^�?�'�,E��,���������1��GK�����c8e�S������ �����^^$M�7C�2���\�-�����I���4���IGW��yC�Y}V/�q��j���/	'k��K���@�g^���[��F9,z�c��>r��o�l�K������K��e����F?��f�q.�C�wA�^��&�|bFz.�IZF���_�T0�!�����q�#!'�e�n�lHw�5�����lZ���p�[z�k{L��o�{���������fK{�h�q�
��d��D��G{��0�s�(���ri�(x���D��u4>k0D�gv��{@x�vU�0&��e�;�Q�������,z�q{���o�;����<�Mc�'��*�9A���v��`�
�6�%/$�������*��!���	]�<�,��8�Y��3N{�z��\Y�����M���<���L~<F�K{����i�O�/�l��0p�%��])��s���)�����#�Fi�-�sAB������^��<�y�
)0�Ex���O79�����6Zl���|�S8�|���f�C]����w������N{)����
;��^��M�$2oyE��0d�s'z>+�3X�~B]�8��"��Z��a�\�4�c��������Sd���;�T]A6
���m���R��������G���f��^*��(v��:Tn��h�/���32�iy� �[������yr�Wl]���|k��g��``O[��*���lc^p�m9P>��kQO�����0<������dFs���N�cA�3�i��Ai�C�O5�o��5�%�.H�`0?�(�����!}��S
1R�r��9f3���(��������7�f�.8��D(�"��E���g���G�#��7V�i�2���>vZ.T�K/����������|Rq��G�%�;G��Nk�J�D<���X��-��*����f����{�L�(�i�I�:�w$Ha����Cm�jX|�h������p��������kQ�/��f��,��P[�C6:�8�k[{/q[�Q��p'��J����HWv~3o~��\@����8���m�WgrA�+;&��i� �&�u�P�#Z�.���c���j�x�+�������}V�U��K|����n��\������m�$!��TFM�e��`��]���6������"]��2��w��$��y�(��������7�#��XqB���i�(�Z�����vt�)�W�����a]|��:����>����������������n���������w����������?���Oo���O��]<���5c�O�;�����/�~��>�1�������O�|��%����o�}y���'�����������?|���������S��c�N��Z������)����,��}�x�j�q1Q�������;���S	������������8{��>9�N�g���9t�M��U�F����3��>��\�>�����_�|������n����S��c7%�A�"7��8dj�8���
�)��\�����k!��Q�Sc���x���������2V��Cn��8+����#�Nc�	=C�;vdo�9��(��$���:r}���|�j'��m}����~%�v���|��pWD��A�O�{:�X7c�9V!T	}�Y+�(*����)����P����>_�c�G�";0�����~�S���2O��6ymB�9t��L�H; �2�K��@o����h�� �b�r��������8J`���z8v���k����6$-�,;�^54@��,m�LG����o�|��������g��YC��r6i��&z���-e����+�YI��<�}��;�Qt��l��}����?���/���~�������_�P�WF�J����+���y��?��K�����Ox�����������bv?y��<�_�����2�/�����4�eU���O�����������g��H�s'�`���t�x��z��K�������n�yrI�,8@T,�*jq�bzNA�W�N��M�/����K�������?l�QM"��(�lB���"v���}zD��?��WL�O�
��J�
������bO���O��EV�}�luJ��EV��e
�<"��Q��8c��c�(����@�dbF ��K���
�_r|&���q�>
��CA��:1��D�c��U�����3E�\�=�n��(�e�L��.�bO��3��+�"������S�l���cy.	pA�Jza�,��(�G�=1���"���Ao:��sa\��������{����r�{Z���V+���X��{�����/�(��k"��Tz���=M��]���XZ(��4U���<�H�4��b{v��c�}5��;DM��!�P���2S�v�����;��P�}2������7�EIEJ���$7�Y��+zv&oS����j��?�BN}:H�M{}:n������=��`{J�=}��kL�;li0^f��Z�&48�' �Y6
��(:X���F���h���wG��3�J-8G�������i��;D�:t����j��!:�O���O'��_4sT4KE���N�Vt 5��$,)?�������Tf��|�J��F?��>�@���=:�x�^U�S.������������'f*���&�����	{_%A���y�	{���A��kL�{-�S5�'E����?pGU.6�YA��ct�J7���
����!<���>@�W /�(<(k�������Z�����gc���S���P��(=����_o������e��g!�N���U�i�"��7�0��\��O$��P���bZi�_�s�#�.���l���������l���������l�0�Q������ �A�|�"���J0��V+����3�3m�O�!X���p>P�W(�Y
O���O,�Z�e��M��K����v5Uz��K
���I���R0��7`����4��G����{
����v��,�����\��O,��P�X�������^/7F4`!�.��v���5��7��\��1��Z�a�D�x,�����%�x%�kJ�6�)]�[i�;q�X!b�xV���D|5q���0v���E�R���`��]*�%�A��)K��P�gN1N1���K!x�B�D�z'7�Y����r71��wp���w�O��S�Jx��_����S�]�]�Nl�����wp*�����5&���^g!��fa��r�:��&<��:�������#nF��{���G���Q�kV�/xH�� ]���^M�8�%<%`�>����]������kE6�`x7A���t�����>q��J��E,�"H*8>�9�,D�kd&��nO�J��M����"x��	o:�����v���rcD+����9��1��A���A��1���Y�#yM%p -��.C fi!��r�b���BA�M n������>
b���!_���h�_M�8�|�$	b����0��
�,Q�Q�PN�����
��ecL�T'q�
���M"��%�p:�}vA�0*�Q��z����E0����t<�O��B�p__����M�RV��`������?����T��EV4uK�s�JM/V<� �^� ��������	���e��#�GhSd6�V�N��#Q��5b�����#�BD���:��6���q����"�x@�T-�p�[�at��X�9�}���9�h�o��W������k8��#$�BQ������M/���>�0�m����.y��	sz�[37V4���v���"+��<:fN/Y�\4��\V������U��\���@����1��)V#p�qh��s����d��K*�����N����w3�G����B[QQ��� l��`����b��l�d��{l�����,��^[��9���F�1D���M(j�2��$l0��`(�OH���P���bb�M��&�nn�h|�����I����d=B�p~�����1h`=�I��(�X�����*�Z�M�f,�J�m���F���?����y�7�z���o6HV��,_��Q>���"+���T�.��'`p�
��1��{���k���z�=���Xb/�h���TZ��K��<B��E0���$�t<�O�F��5�=9����+�����p���"+�w�ep=[!�?7V�/(�J7��7V<� V�G�]�����^��[��^J"Z�r�x�|tfSfk�4�8��4�I@��"p����#+z�!
�P���!��E�a�"�!�0l�������u�r��p#p��fV7���b������Mj���eqQ\_�P}:�M�����[���T'X�$�fn��s#xp��g&����F`
b��"
���{nD�7����{�p�^�����Z��B��� ��*�9�G*��h�H#�C�Hs�L�9/g^�P�_�N�,�� �K�����r�-�,���n�F����j1��c	7`�Z��q���x�"�F3��|�Z�HT��� T��x���>=�H����S_�����v$^�H��EV4��h�&+6-ld�h$!��'+V���V<� VI�wM��H�o|�����EX������sP�]C����R��:)Is^F���NBX
����B[f�f���",�Q���F�[��Dat)��A�����NTV��+Q��c���,���Jb��Q(jh�|%��o���> ���B}z���_A�(��b]eG���hWMo���s�-v�D�GEX/5r���F��X�����)����7��>���C�Is���9�4g^���qJ��(-2����=X.��P$�) ��?����X�.��?��)h��nz�7�,��K
#R�F��a������"���Ao:��w�+/wz�[37V47���]���D����$�1��N/Y��Mj��'+6t��V<� 6�������5�N7�N^3�������3���z���!����N�����3�G����E.�tF��3�xu��e�^�>���
A�U�P����3��G���xB{q����y�T,�����[pF���ImV��Z�)m�����&���o:*��[����2��Mn����m(�����hK�
���[���m7xh�#����E4�93�qxs.�g�^�t�_�OQr�hs��IP�V��d+�y�:�z$�E����e��K�L|"&�^��`�*��  fA����v���J~��h�
�X�w�-8(�>/�Z�~���m�H�X:b�7�PT��x�Q�>=��2�i�bA����������vlh��v4:�=/K.-�K�c\�]��%.sc)�@��(��6�=pmd�O=���d4I�CUKs���9�bg^����%�w�����{�[�u�ka���)+�<����MrAY��3� y�V~I���-@��$�����%�d�x�d���d�PpyE���7��U����8��7�uscG+��0��vx{��>v(�t��v��D6�FX�K���A@A��%�-	>����Br]@���cf��!oY�����%J\�u�t^��^���1	�������h�nK�}�\dI6��c4��+�
h��8�����o+A�hZ��;^�d�3I���X�y�H+��F��4��C���/�A�>�H�m��������~�`��&�
����?����F�>������EV�I<x�	�^$�xA�����)��+W�#�����i�)����*n*AQ�'� �-�:uK:�cG�G�����D�S���%�[A�u��"^���j���K��=�����T�V��=��I��O��Q�
8� ��EB�9�B-��*����"���Ao:��G�$F��jX^����
���n��g�G{�%�K�'�\�0��
Q+�%��
���QA�����(�KB���*�U�
�������� }�A�z�t��%������_w� �h�$#���`���`��"p�b0��~�3`�e�Pf��2�")�26<.�b�����gD�gi�F�j����\j���"���Ao:���o��F7M��;��5sc�9�"��Yr��XAVT���r��h.�<Q��G�F�2������������==��QC�Pp��z���9�C=K�N���������;���9�p����7�����"�g�r�r���������{�:\��?��[#�F,�%���7�,�h�,yu� 
59����"���Ao:���������[_����{������^.�b��Xcz^l�h�]��0Q�*�^$�xAl�,�����A��$�A5|����������=C�z�t��%������_�������r�)^aG�EX�`-Z��pPz����F><h+o@��$E$���Z���������Aj��m
W���`�OA��xP�^��V*��"�fn�h�\��
JY�dIe����"+�T�e���\h�$�xA�z�`A�5y�����(v�rE����h� ���#�C9K�N�������;��H�]��W`@���6j�\dy�Jq������pW.31)��<Q��="�Z�o��`��g�^���U+_n�ZElR�0���`�OA��xP��"6�� P_�����"6)��r��"6)�'+�d���Q�����^$�xA����Q,�c��`<�r7a��#�U������%]'`I�����uj;es�K��G+�
�{].�9����W6�o�hrFX�Ay#o�>�K0�.�
��z#8���b/`ItV��&
59AJ�Y}B�� �7���A��F���5sc�>Qw�G{�9�13Y�Q��b��(S@�\���"�U�RSR�Tr1�Hs�zzzS���B��1�����P�#� �+�:�J:�TG�G��S�)� RH
"��������\�;�����Sq���2`�IT��l@V~-�3`xa\.�`��n�Y������ 
59�r9��:���t������-���l������3�d����{0�!�1�\d���18�L����"�U��`�Z�I�M���������t��I+Vj9%���t(XI�	V�yi:z=�t��g�jN�L,��6���`��� �4���q_����� x�\������vb�s��P>/�
8� �b������w��bPJ����7�]��P}:�M����:����8M�z�6~��X�t�����r�Mgp��4/�t�MgpL�l���Q���Z��
��h�%�"������xh][��F�
�t�B%������Ew|�a�n_
�2A�!!*����,�pFE_�h�e�
�#G����g +����2�aRFq4���F����Q�4yQ�A��J���"���Ao:��� x^AT������b� x��L��_.�b� p@�^.�b� xp_0]}���AaWMJ���)��c�Wn����A�I�f�Fcs�F�a5)�u�����}=t���h����r8,Y=H� r����?��)3�(}S�$f�>�X��?��I��Dv�IiRPk�x�=�%���E0��A��xP���k����"�fn���A��Q�jE<�k�������d}����D0�Fgz���Qa�jN��Px�[�2��=E��G��jm��T�@�FAjR��4)�y
:�z4��o12�� <�����2�
��r��Ax��k�:/n�>�3`K
�L������A��@h�|��v�m�Ya���&9D�&Q�5Q`W_�P}:�M���t�?�kM���5sc�����l����X�?���sz����A��f+D���69�0��S���:/k|_Q�en|P�^m8�ZAjR��4)�y
:�z4�:<���1����r��VO�EAd1R1��9&��(JM���f��3�Q���}r�'�.�pA4����Q�U�B���:�$�P}:�M�����b�q�^�������j���"+E�X9���EV(J9��N/V<� �jNCT�99�E�03:%�� ���!"��V)lr���!�a9������/����|�/��(m*#d�l��&����V���a��H�Z�,fA���c{�1�`I���U����-s�2.�-8#�%�`h��P������Qi�Y�CQ}:@�MG���Zu����*�������i��8������hZu���bGy����Ug,{��]�w���Z�i-�xWzG��&D���6�*�R����9c2<��������N{�����b����(�H�!kM�����`��E���0�����V���/
��N�P>r��N����3�,g�Y,��J;���&
5)	F�T�g�E0���$�t<�O'%�y�]��"�fn�hX�<xi�U_.���F����r�����\X-2�"���� �Zi�*���5����C�>�������4�8,���w���\��su�D��	%?��N�I��(�,Y��u�:�yq���:��^�M���{����,e9�"�}q���Qj�[8N/�A�>���A}z��F$���5scEs#���
0�\d�A���l����X�GX��mi}���Q�wZtB�%�Q~-[	�� 2%����������������^W�i�s��S�u|�AZAdKu�;\��Uo�\DD��?ZM���{�^ED5���K*��vo�a�O��
�n�Y�w�"�mj�|P�Cy����t����� �k�P^����
�����g�Y��0����Y�x�RQ_$�xA���
�&�J>-��`C��bd�[��}��7D�$� �aq������/�r����A�f���<D�5�~oj���"� �����Y��f#e�A�QW@Hp��oJ96��6[�
8� ��KT�YX+�(� ���"W}B�9Fo;��� x^	�������b� 0x�f�/Y�Ge
f+�UV��V<� 6��	���4E�k���]y����(�u��H�Fdk�4�8,�t�w���\��su�hi6o(��7|�5����<2�u�j��W�2���� �I���
h����E�
:�N%�
8� ��N>�o�o�(�w�V����� T��x���>����y���_Q�Y7���{c������b�Y�wc<Bzz���QAy����C�M�>/�nq'�e��e�A��R��m2�aq������/�r�����I�em�Z���H�(�E����+���Y���4��l���f�
h�;��M�n�Y�w�pD�V���e.���E0��� �t<�O�G\��Q��D����
��{cEs�����r��A���\����"�A^�"r
1�ij��� ��/M{>0l�a>��b�Ca�C�|���&b3)��-F�IyWv�mtj��"��@�
5J�n�h$ryl!bD��J�4���!��?�Ya�)�D�Ah��)���"���Ao:�����W��O/rk�������>�j��4x��b��<���B_6{7��qZ��"a����S��"r��2K�C��� �W(�T4{���������#��C�|�p�&b#��B+H"`�D�7)�u��"^�hQ	��
hD�b����Kp���.��;n�Y��LJ�����D�A ����"���Ao:��� �yW'�������AP�������
��5VZ\����h>�_�$�xA�����U�5F���D�'0��S�GQ��r\���u&� �!���!>b�W1�o1����EW������s�.�jN�6S�`q]'n�����q`���*��MXR	��y�
Ap�"�C�8��Ul�PS��K����"���Ao:��s��y�-��"�fn��_A����.�b����������q�4���Q�JT�:$��E�<Q�� 7,-������t�ow�P��S�t��������� D�9�A#�T�K� `��,��k�G�U����T��&!e@���+���U��^�pA��)%� 
59�P���E0��� �t<�O�A�S������������j����"+�Vy�J�M/Y���s����"a��b����z��5k���s'&h��=�6]�L��L�CMJw�&�;�A�^�]G�R�I���S,����~��"�I��/r(��k-����s�A`h��3`_��CJ���M����E�&%������B�&%�B]4)��`�OA��xP��&�U����En��X�hR�� 1M/Y�hR�+m2��EV4���+�d���GD�A��Z�]�����(U���-���5�/���P��]�I��k����A��A�Vs�h�TY���Uv�\�� ���i���
h� R�*��v�����`��e��l����x�"��&%�J��D�V�(J�� T��x���>==���8����+=.�d�v�������f��h��D"�_D$�>� 6�� �����j��M���-F��BD������������:MJw^���
���D$����r!�,�[>�;����s7��5�:I���
h.P-;������Y���,D��(��b8�l �^_�P}:�M���tn1x^���"�fn���b�"`R��(�[��������r��[�g+��"����F��6���_�\)�Zu�u�B���Ik}�A�"Y�E)�u�����="t��V�J9��T�3���"���Z`�V�UFU��}��j�.xF�^��}k���\��$��z��7���65�:-���V��`(��1�x�Q�>�-m��Mn������2:.������{r����Ev����Y��M��G�*Q	��Tz�@����A@�wZ4�f/��i�h����P��_'Q��K���#Iw��p����������*�p�\d�;Q�2@Kr#n@s��]�<�gq��PT5��4~��X��E�3c	{��X���B��:���t�����9�[�BY��^����fo�g�u���\d�saQb:�\d�m�pPD�^$�xAl$*
OI*���E�8���h���U�%�F��8�8����IT���t��H�u.5�n_�K
^.�^.���`��^j���8�.N0 ����(�i��7`�����vG������,��KTB@g�t�F��R�cCB��� T��x���>}a8�6����+m>���
m���x�*5��r�:q�L~z���Q�JT���6�C�U:���#V��a�5�5���u�%d��!��D��N�������G��/!"<z4�-,/���",1��@���F���<�)�ya@��t�2`����Y)`T�A����������D��4��>���E0��� �t<�O�(�+��P��������4��P�!��EV4�&�B�j�\dE�n��q�^$�xA��h�n�� ���WB��k��`H����S���^���?����IT���t��H�u�;E%*c�s�J��9,��{�����N���OFo^7`6F��}�	A%�11�����3�,��KT�E�V`
����R^�P}:�M�����<���^����
���+��^.��A<AOs�EV4�/%��E��G�*Q��n.m2J��*�c�4#(������y�������:�J^���I���N/+0��Jh�QB:4���"� �SPp��D7`�Hts�"e�>������
�����g�^��8��9�B�-�Q�Jz����t������b�J��^�����-�c�]�(/Y��b��m���"+�[S�)'+H��GD�4���bm����b6�+0\RT�iE�vS���aq(Q������%�����;�A�f�|��	)JY�mU���#��	&�F�!���?���	[~���C�..�a�ur�E{�Jc�F`j�&%�\���E0��� �t<�O�I����r��"�fn��3)��q�"^d��IYw���sc��Iy�\<� �A�I�R[�6�e��6��
�.rD6kDF�XAJT��$*�yI:�z$�D�^A�/�.��w�m-��\�%*�Q;�W�7`��Xg���h7`����j@b���8� ���1��Q����UbzB�� �7���/������5scE#Q������r��De�S��EV4�<x��"����b���������*�,�_z���J��������'���!��F��N�������G����E�1x�d� �h�G�L�^��!��K��G�)'nA#��9e6�[��F���.|������p����N��e�*qoB��<�S.���&���o:*��+���~�bo����V�GG��.����p���v��<f��� 	;�b�q|\��X�Z�"���G�L�G"$%,�p���h#�X�+�u����B�=
u��M�)	�Ju*w`h��iz�.���%|N��l�����$8���0HN�#��MI`Ji���M	r��{�J������BMJ�V
��lW2<>���$�t<�O'%�y-bz�[37V�S�N��EV�S��x��EV�S��_��|�DX+
��������^J"����ROlM��G�P�2\'X�K���#MwL������Ut��C����,YZD
e�0 ��9�J��E�����0Z~����+Th�����<���"��*
�Vt�b�v��~zB�9Fo;�����W����5scE���i�B�wo��{p��V�xW+1x���	+E�D%��R-�`�vxu9tD�*�5X�iS8��tiF8���IT���t��H�#����J�Q��i��.Y������d6����������A�\7`zx(�Er��!a������(� F���E0��� �t<�O�oy^]tu�]trk�����b�d&+�����b��<����r�{7�`.E�D%�b%V&dX�uE��/����;lE��4+"JT��$*�yI��z$������Ie�D���Vc:\B�h�:w�������A�#���@��A��t����g�^���IS�Q�E�w�pZ_�P}:�M����D�
I��5scE� �cf+�eV4"��R5�\dE� ��?�E�"s�(����F��f�'Z�I��"��s9Tz���f���.����:��p^�.�I�C��IT"de��N4.���",rmUp�4#�&�����b,��znm.0�h��t�|�09$9�"�F��i����F�V��j�4����t�����IT���"7����+����{( N/Y��`��d���$k���18��O/V<� V�J�qG[zC`jFWI[���H�Q�B^��D/��8���IT���t��H�wQ&%Ps)J �AC��E��VT�����c�hnQ�Y��'l����S1�'g�Y����X���B�<��}���`�OA��xP��<�km�T_����{y<���������C`p��"�Us��������F��G�*Qi�"����8���tR��w���/�{D��-!G�F[�P�2\'Q�K���#Iw\��eo1��c���"��EA@�";�x�	���N�r�!��b�D�{�"	"��3�p�"�F�2����6
5"�I�����t����� �F��l$����=���h�7�\d�A`p�&+V"��{Q� �s�%�xA���Nh]�A$e��H�3)#T-!���TV0^AJT��$*�yI��z$��yZ��U.��#���`��"6U�0���7����1��
h�q�J�p�k�8� �J���(�� ~���E0��� �t<�O�o���y������b��ep7[�.�b��2���EV����:W+�	#�"�U��:��5��Y�
OL���IY�k<�d�����bJT��$*�yI��z$��� H�MFW��\ J�|8X.��
���U���/e@��\
H���R4��2������7�,��KTRNj��o�P����h5���t������ <�\�l6�]���+�9^r��"+�9�-oz���}�LA��B��E�D��A�"QI��Z�k0����d��?�CR�=�� � *�u
���]x=Zt�ZYE)+���"�|�\�E�c��������!e@#r��6���qa�7��Nh�]f�)�t*n�Y�W��@j��6Q�aR����4���t�����0)i�O���

_7��Yp��X�gRbp?���j.�LJ^���E��G��Ii�=VW�A�n-F@��"�g�Sq�h+g�j�CM�p�&e8�A^�]G�RV��.����s��,aj�L1 *���I�� �t����B��
h����d�O?�Y����>����D�&����^�P}:�M���tr�W4��^�����������>��W� x��M�"����)5�HX� ���&%x=E�%�T�cK`X���V�RVz64���,� ��&e�N�2�����G���I)�%#�3m���)�4U\��l5g)'�5JFWG+o@S��k�R����-�O��F���&eB�1r�Dq�I	r�9�BMBT�������#�����[�W���"�fn��;/� I?�\d���yp4V��EV���|�8�(a��b��4�+WmA8d�-d�������2�*��&��&B�CQ�x�(e</B_�]'	!J�4���/��^G�E������ra@c��4�y\��'�K��4�y��)s�����ga�^�� iC��m��X����&���o:*�����XH��or������5N"~���.���dM��4�]dG��@�  �ov<
(6�9+��'O�=^���K�0H�ET�;4���O�����Fe�N�2�����G��/�i���nx4�����a�%S��T7:fF��uw-����u���a2!�~xh @&�En���Vb�@���Er���t6��9�@M]����b�EVq��by	�������o	���U@h��HX��%�N�v���h\�@(P�4����ctE�-�[{��V�+S,gS��P��q9q�6�v2h�rG�m�1�g��Q�|���YBq*V��Z�%�cK�v�y����&�W����)��*V�}����B]x��E�^�J:k<2AF���� .�fD�E+�C�
)�9M#iWn�K�@��o��r�^��h)��x���'�#		�����[��������v>'��W����91�'�m�[�<��7�,�8+e!ldI����'o����P9&'~ke�P��
�1��de�'Y�K�����[N�&Z*�nS�1YNd9nb/�W� d���"4E��B �e��N"$��`������X�`�����l%`N]��J�����\�{�����xk�P��J�����������J�F���!���x%A�����	�x%�Yx��dY�g	�k
y���R�S�3{�����P�H����#�52�r����e�'`�����[�]N>��2��^�y�|�+�(���KMK��{p.z��<�a���\�l	�cu�3>Fw�� `i-���\"Yb�������J[0����-�-��^��g3�������M(��mL��6���q^�������1�S���(L.��8�|��d��J"A�2;�H<��!�-��a��e�/X^G�n� k�e��;��u���4~b.��E��D�y�`������a\�?��w=^���������p�A��H$;�7^�g1����*m� ����}��"��v�l���W~I���
o��E/��9s�k��)P�\���g�C6����DZ��6
}��WQ��,C�*�(/H��mec�� ��a��e�/X^G�n���1
�S��"�t�:�����E�(��,}@�m�?����#f@���6��\�7cN$H��@�^��� :K�����F�����J�z�0:�����Ai���W	��>�����"hp�tAA�M(��<U��\`ta}Z,]�1�eE�e��%^�#��$7s��JV���RB����F{��<{c*`�	X���u�u�Q�� "}�uf^�Vw�1^����*\nB��'��D��O@�%v��3��������q�2�^�)����B���I����%[��	��`o�J[�[����������~�bu��t6��w0u.��/�~c!PD�v8P<� NKI�=����|�#��QG�(�d�p�����=�1��,�}���:�usY� D�		Q	jeff.�Q
W�B�����.�<��B5[�@�p���#������2�^�R:���Dt%@^;�N���xkP������T&8:F����?
nv��A�)��\0z��� �)8PXO2�x
X*����O!$�hi����$w,}�{��Wi��9�0F�*X�}
���b]|��9�P��;�^��s
:M�0����^or!���}���`H��B�s�dB0�r�U(s�*>��M{K�����
��I�#D�0:���4��}Bi��y�$A����E�����M(��@��x����O\�������(�����v���2���BV�������h�P��d��="NU,�>�x_�.��^�\t�2���2:&�K��-t{�!2
Km=}�N��`�@�m�'����`�y2�����.��U,5R6���/<��<�+=FWT��H��W(my��)����g�����c<T_r�6����YBq�XB�V��F�J<nK|2v�A�:"!H��	�|�y2g7���X�}*���^]|���dkr��dIh����.5��L(h��\;�$ls��B0��f�E\����s��R�v�'�^��"Q�����NA����+*mA(��+�����XH=>����0�n��8�e���a'c�JI�m�1�g��q�|4����YBq�XZh�Kr8��X�F/S%��m5��5Q���U
���q�b��X���u�uT��|��>��s9^�C�Ma���o����R�������J� "��s���;����'.�dKmQ4����%K���/4�8��m��atB�-��[�����c����k.(���W7�6��w0�\L�x;b4�Uu.����Yq�X*���W����.X]I@Ug�L�s����CS��O�2�����#W7g�Y�R��^\dY{a��@�#����,V�#��|ysn��B����Lu�����9OR��.��E,���F"-o$��P#�Z:�����xk�P�J��
�%�J��f.(z���2�`��UAA�M(z�-�&�9P<�"Z��TNE�D|��)���P
������+&k�/"���q�ve��U_G�n�p�[��Z-t�:���@���K#,
H:I��g�K#h��*s��bG���`<���K|���Q��$W4��������+*mA$��+����I���S�OU�������'o���&C�O��U�|X��|�1����s������YBq*Yj��P'����1�v�"AG� K�%�e��P��'�J�q��e��\_G�n�"��_�^�*A�a�
x
7���"����K	�Q����_�)_�_����?�xE�B��	�.���,ri���\"���D^�v�Pi&����������$N����E/"G�;�������W��6c�P�rr���\4�D�,�'��R�\1\A��������Kd��]��%-��*Y�}J���r]|��E�O�T�@�="���"Z�{��R����M��W�+��,��a\��Gv�x=�J1.�]�wA���B��%�e.d];��xkP�B���mS;�����c��l�w(Ybpw�p�P�J�\(�gD��Pe�r=J��
0z���-1�\���f0��d��e�*Y�}J���r]|����O�� ��B����J�9��N��!����R����J��@�z�&�w,�]����$�+YB�����*ihM��v>I����Ai�}�yup���9J�<��e�|������E�|��%w��E�|��eL��Y^�!w��[�L������������%��T�2�S�������(��� ��dQ[a��%�bS��4�(����������Z��0�/�.}��_��30�0.�]�+Yj������QT����M�`pB�-�[���Vw��/3e��a4����G�v6�nj�Tc�q��*����G?�$�H���%���bG���WJ�=�@e�u�{�4�xD�
Y�}B���h]z������y�(�f���G����qk.�QV+�@N������X�(�<�~lz��p!����p�A�^�R�{z�^h�bDS����*m� �����;�y5`���g3�"J�A#�v6��oip�*
�v��������P�����T��T2)��5���/�q�$��������&q3���e��a������Q��?���U"B����P����a.� �\����(����P�����������i�	��m���p�A�����fs�<��PM��C�{��atB�-�[���w�y^�1���f.(�D�����HM6/��S�\;�P�W�y�cE��<� �J�O�|����JZ�z���!�`K���HD
�fS���O�2���K�#O7�bx��IG�"sQ���u�/���4���7Y�a�3����F�_��|���%-��P=��� z�JEt�L�m��X�K�(N�;�N���xkP���U<��[�q��x�T;�P��hpT&��M(�z\:�@�a��g�i�!��X�b�2����_��R�u�6��6����b�S���T���t�ut�����Y��8������-�p�w=.+R&�PG����2����+j��9ob����QhT%��k����]�kU���������r�&��.18��4��}Bi�z\x�wL��M��z\4xV�*�M(�z\�il[�q�'�V���x�F<���X�i-h�b��R�����
����o����4b*V���U���t�u������P��P����'TM���|�a����I�8�X��D�M�O������z\�U �Ik�pwiD'Vi1�G���M�gh�b��1���D���Bi��4��1���n.8��4��/7��g��H�<z�p��36�������4F���%q[	HXZD6�/!�����-����S���O�2���K��O7��4���	u��[����`��odl�U�{*�f0��@e���������2�H��1���%�J�z��k��p%���x0_;�N���xkP�*"I�j�	��	�q%��s�����b�tQ������y!��",n�Yq�Tr�9��4��S�[���H.�{1�]���&�>3���T�}:���.]z]�yn�F�O�\��F��msa.�E'p�s��D;�@�����2�@��6��\�\D�B.������e�N%T���"�p!5>-����J[0�������i^��|���x���R�ip*
6����@;Q;�PI��T���2�F�����97��G@�J����3�w2'��Z�n�~1��L�t*�}]��:�t����3��>b>+�����2:�y�<�W�
��B.����U���@N7/�@<��W�D0��]��T*���o�/40��"wZ:�N���xkP�b������Lp{N�p����7o<(�&��6c���������@�,��gZDB�E�!G{�"�YG1�#k�c���YRXo����T�}:���.]z]�����!])��
������&�BD@� ��cn�bG0���xD�3C�\)����v�h���8�E�K"b����h�@"�s��J���� o�J[U�D��g3=������Y:�P�4x&��	E��7�x�D4R�('*�5��w{�WX���T8N��[�����+����Ve��U��k�����[�C$�'Z�r&��
t�T{af4�����H��#n"����W������"����!��"z�J��8�O4�:�8��a�C�-X�[���VAHK+Q'8Z�msA1D"���Xf�	���D�����b�IB�%U)q�x�E�����@y����8���VU��D��e����\��8fA�g<��x�h�z�m���zT"0?� ���{�D��o:?l��J8�8!���$v0�����":����.�!��b��!��87�D����:��:��?�i(#��������J���ww��60MmD����g:����o��&�f�����?c�8�-��0�hXp<M,T�&A.N�4	#��\Cp�����2�)�����WLu,��w����u������5�2_����G��)��)�,kt���Km�Mq�J����"~e�-����F=

��6���a��=n(�8O��/��n�/�s�����J[�Q`f������t8!QO�v>+���S��H����!�A!����OS�S���s]O'}���T"g��i�#R����7��
Z����%���y9*�,��O����������dX��#/�U��`ZA,6}e�H_I
^"M�������/�,�����$QKO~EM\�P���4�9N�0z��VL��]Bi��K�V�N�v�������J���E�lB�ob�=�������yp{�`������k	}J������'��:��"�w��3N�YH�;g�����>a�f�n�ud��L���M:7Be�L;�N]��dx+t�� �i�3�
@_�'���g���3�W���(�l�����/�6��u-QV��+Jt�)
Dyj����b��J[����M���\P�E���.��v�O��"].������/�E���Y;(�f�dV
�zb�Y��u���o�Pj�Z���g�3m4w�.�>g��-��n�����[�I�2	d�BYV��_�L�����9�6��B6/��Z6������[X.��(��Q$���i�*>��� m��x�ZW4�H����FOT��I��K(mQj���xG�
�w��	E_j#O�:��q]P��60x��Y:(�f�����+z�NM���SfYkCi]�r�En7�N%��������}��w^�Jh�L	e��8=\
����i3��*W���X�;�>i+#�����A`G��k!����	c�l��n��A��,��p5�h����<Dvk����"��J[�P�&:�I1������I��*�f�-�4��G��m�1&U��D��#����ib��2���L���������JA�J��MW���.	��X�S�l���?^G�n^�K��R>�`����-6����K�,}��5do���b���HF��b�6���|���n�^�R)HXN��@,����
���c�G����{;��V8Om|L2�����q]?p�M8�����	v�6��4�n������E��i���i��8���X���f�g��~�4������9������u�}��������fQ�'�.��xN��dXi�GU-�����&��@�\!��M�>�B��qX�B��mN1��<��W4dQ�MU!j����b��J[dQ�������\P����|���������ff��#��*
�XP<�$��$D}�����f�8������>�c�2����*��>�R������}���w^�IDf=mE���1�p�LL��"�Pp��(r�?�A���@q����i���
g�6��}���NRW4+��\�v=Qi+&��.��uY���R�������W<�z���+�IY/<[�(�&��^�a����C�N��^����b�qYi<OD������W�y�T��x���}�����Mk�����W8���?^���J#L���S.��E���d|ha��Z�#��7���?�"�����l����&�/E�['�S�c�A;�v�|Pi+����������Pg�(��`3����m~DX:{@��c]A�_� �=�U�����l����??X�#�W���G�i	5�[���R����'l.W"~��_��a{�`�?���o���4?���O_/?������z��������r�������'?���O_��������?������������9�=NX��W""���|���?���������������� �_��������v^��+��t���W	S�xpR��>+�!U*X/�����"��mq�p�4����^����wj8���R�uF(��X��XH��!����-����������r�hY�����C�`	�|ey�����<����MLN������J�K�}V����q����G�,c��e��!H�����o�������V�Q���m������������|��;�����G7m�aB��Cg�<�������ND�9�?>|�q;���?>n���\�}zj��?�����)�.)me�����b�pq0e	�"��i�����j��]F�z��;��k�������~�S�o��zK�%g���������|����L�����%SG��m�%�?��w�PK���j)PK{2K18sept/pg_part_head/10_2.outUX���Y���Y���m�%�q&�]��~�@�}��R�]�Z�e�b�z��	�t��{v������S(�:���W�`�`w��,2��x����_��?�������O����$x��_����������o��������_~cR�d�{e�R���_~��?>��������/��������������^�|�����S2�(�O��'���������1�����O_~���W/������~T�����>��_o�����?����>����~���?|�����������_�|���������^���_������������_~����_�����>~�+~n�x����y��_�||���O��i��?���?|�|�g?|�����z����?|�����xy��O?vW59��s*��8�f�>��������������%�OY�����}5�^��?����&��_����W/_>���|����>������#�`��~��6���������^����~�����������e_h���U�g�>X�DJ�������$��/b0����7d��v�{d�
_�C�S��n��^�H�v�s�vD�6����|��G�G?���7�&����g�FL�����Z;�

�?�O��/�&��~�����������{�_�~���?���w�>~��J�������������������/����c
�xJ��t���Tp����~fu����3�G���;��\�<���w���}��_�g�������v����^�[ �t��L�F������Y�������7������y���S"b��������'l�����~�nN��C8��m���e�I�_O����E�:e������S]��^��>V]��U��h���l���qz���u�!����Q�{:������W��5+�F��{���n�������o?���������{�)�S9�����Qy��
�����������N�a���*D��Q<<I�x�@�i�Q���I��.:=I�P}�lmT�O���N,������������*�Hd�W�w�1l�;���&�e��_=���`(��~�$����a�O���#+��k����xK���o^��s��O��?������D�p�_��(�'��f��6��t2�Q�I��*���2��D�w23G"��dk�7�j��Lv?�����:h��Z����~��5k�V�U��*���J�z��f�;���	���C	���4��R��z:��7���J��p4o�t�]�fS�G����Kq3�����)D^��������f���R���h?[L�������c����f������l�y���WV�[��T�f�.kJy��3Eu�����w/����}����>�|����O���������n>����j	�9�K1��xp.��b�~�v8~��t�����q��}F����w����_~�������?�����s��/?�����O���y�^~���O����/��o�����������9���$qT��C!����P}Nl�x���L`}�������{��S4����A:}�w��)���l���	lw�?�9����/a����ZRO#i�H����<�U��c9��s��tW����$��t����~��|�<g��#��x
�&�e��U�)94�NX2<aR���6��x�zW&dmHZ�6���'�����k����U�\�=
fLe����.�^��e���
�^{W{mO{������W'o�)�eo�p��s�0{�H��W\L�������}�,s�?^i���:YH��kr�T�v2��]��<����7;���Q�h��/�|����
����|D[�Mz����*��)�PVw�Qk�@e� �� |.�Q�|�rh&�����
��&�-�����������,E�&�9���p�-��E�����,�A�N�����VY�&.y��������r���-��L����he�w�����(��RYs�R���QV�\���s��f�e�Z���
dc����-}1�^Y[c�7�/r�N�d�7��b�G2�d����Z78�������(>LU.��$�������U��9P�pWYCOY�������G*�	���?���M�,zoFDw{e�d{��
3�
����n����_?���]<����Z����0/#Of�O��(���4����+�)�����	S�|����5�U��S�(�������9�l��7�	��C<3�����0r������;��!�������������Q���RX�hvDVk������z�h��O1?�3�P����l9�����}70{^���B"jR��+k������I@Y��'��4�
5W�Rb��s��X��?8(�%��
'��t�(�sf�O��7M�ma?8���m��N+`�?�Sl���u�1���������[C�5�� P��;��C��������f��W�p�L4�m����%�nTr)����5��r�2�J3P6�������t��\�������3�H�e>��St|�����4[Bi�5����N������X��N�����fI<%Gi���}����o�����`�-���Y7�A�'������r����Hc>�|�������t-q���g�������z3����[�Cy����9������P����9�x���nlK�\%��a�����K3L}�"�LeQ�JD��@@��<)�\�-n�������=���sa�+�D��p���� �SK�����YW>k����1 ������,{s�]S�� ��'7��1��h�s��w�P���2x(s%��)2P}��*�����#��lo��dx�����g.����z�G4T�� +�as��u@�'��e��I6�s�=���I����	�m�P�8����7�A��5V�����"p^c�%[9L}��LeQ�B@�1)����AAb�����|���8}1���\d���N��q������������9u�������*�]y?v��|��mz�����6�(�XRK���o��+�2�'�6I#[�\
�dG�.�]���A��D�\
��~d9>��N��$Z�k�@�j�>��b^P�:9�T�/��wSv�"'�Y�J���V���-��E�[]���2j�k]^����<���@#�"��o�z��|4ez�)#�2W���y]D�']>;�Ds��R2�/���:PO9;�����,��-�o��y��WD}y>�����|���~z��� �����rq�c��U_�4��F��JLn���1��.���`TFFe.�Q�i��[>�P'Rv��g����kK�?Rel!��_�
A#����������/9�'8"P��~e>��q$�X�dlB}L�!6�),��]�M e�%8Vb����G�����*�V`��XeRy����&����*-���$��-N>�"��-91k�����J��K��;���{�����,��QpX��6�RW
����Vn�}m]��_$B}����R>������Z���@���P+c5
����e����{�\�!��M)=�+��,)��{Xm���\�0OD��3�����j*�1_*�����
����pM���=�2��k��>l������#K��G]���+�C]���u�!�H�$��
-4�N���~2������|@]�P�)��P���g(��7��oI,�Ry�-���8c>��|���|��X����Z��	a�5������l`Q�s	�s�q��t���+���HvEW��t��s�'�����k����������	��m����-��2��>E0�,�f���G����z����>4.�EGA�����R�0Q�6�>U"Pf��;g�]�9Rq��LwqW��]���.�]��F�����j)CO��T�����F���.�E��!��&���,�:��f���B��Sm���A�0���K��Lx	sB�H�4H��_�H�����lq������+��HxEW����|.�,��7{�O��N��$��|Hg��]Xm,G��+�!]�|�Y~.o�G�(��6��s�x�9[���������**���R����V��^����l���w�W�^����^��e�,<!9�PE��h�R�/Q�*����\�W�>
m����)* ��CRa�_~�$Q���
	E��n�n�yE�x���T9����m?��z���@�T�%t"X�W�F���]���W$���+�W���e���������B�H�!��s��=<�n;�p������6 F�a/�9��a��oy�`���|K�������A^���r��zF���T=M��mp�i4�+�E^QyE�+�y�F��M|T�~aoq�������|.)���?��i�R������.���P_�������O����8���"���QaM^Y�u96�����V�����!�.��z�+@^���+O#O�B�^��v��^�e�H���N�H��"I8����:ev���*�c�p����b.lTf�r�#~�K-�
u~�`� ��D�����x��bW�;�w�W�C^����D^��#���4�u��}Q��h	�$N���{dy��Q��gd}?����4����cr�P���)+���D���E^Y6%��#"��nu��9���;	���E�tyE=�	 ��J����
��J�>������4G��-���J�~��l���C�r���W�1��kQ����N_��kp`
0t�uG�lj�����+lN�$$/����vn��-��i%���S�w�W�C^����D^��%��&�'J�Av�2��l��+���|TN�B��)���3�I��y��sl���TY��W�"�?�9���#��������p�q��]0=9�l!�y38i��lxe�W�J��Iy$aY4t������c�d��!@���L�H��f~!!��d	���~�����������D�`���!Uv+P���h>��|Wl���h���_����c�U��y���|W�.���pWVwe��]gG2n�OVe<��/8����
�%�����T���D��q����+fn����C�������
�����
�����=h��6i���M�.����
�L�F�N�waW����+{m������$���gv!�!�����%$E���du�SI�/A[�����<
�S�D�\�_r�A�
Gr��|S���P����w�����Qw��E�gB�����s+�*�8�w/�l���
\�+/��O�}	49��S����L!H\��J�"4z�K�"��$���G�tkk9�]2�Q;	�s�`w��e��
���5���8&p���g����Er	lX�6���G�����]Y��#{i���#��-i���O�U�����g"n�kRW<��0�V//�/���&yK��e(��Uan��3�+�g����jm����O�o{qf�
h�j�M�/��y�F���9���<��G�wqd.���G:�����dr��vP#3;{�f��-����Q$��fg6�g;�����.���y�B����]4����7���j0��z�l������$���As�Q�FT����OK�~�����G�wqd.���G�����h�n���o�D����M��j���y���3+Q�_\t_�d
��9���7�[�����W;b>JK��#�6���#���%���v�����0��'pfE�y=���^�����8r�6J�4uU>Y��-M�����@����=sXD/�Dt��<��Cp����������7��c�:_YO6R�T8�����>����#����ud��(]���_�3^��5���h����8r��#'pq�.�8�8�d����
�/.G�z	���F"�Ly9}5�4�.����+�+���^�3�+&1��j��6P<����#�!�'�#XV���[�$��A��.g����{o�z�FN���]Z������3��P��D����x�!�:��3`5�|�C���,+��n�d�)V���	��R���$�b1���9��nh#^0��z}����l�����
j@yG���lm�})��jww��]�^�	���K��������|�dh����\F#B����-�����D$gD�`>[�uA�����!��P�c���|�.%(�!q��k��8$����������B��p��l<���[��z��N�\�]Y���Z�������z�8�*J\���+�q�|��@��-Y��2�����| rq��bM�yP~���VF>�m��AW�X�����U�<���Sk��h���[��z��N�\�]Z��G��,��r�C=~���*���%=R�@��Q�I�Q�	��P���.��=B�{_�_���R��/������G]��w�ld���d0!i^|��4
����;�]���	������a��\>��������C3��\������vL�|���� #���0��e�P����_�������s����:�����.�+�@8�C���Q�Z%P�Z��p�Iwt�z�+'�r�V�;���|������|Cn_�r�:�/[�G.,Ln�)���/�C6	�U����y�J@����)V�.�L�����'���O���|���N�N��~�PX�X�����W���+�]9���te�H�U����j�����j-�| ��>q���O'	���l!~D(��P����#}��~E}
R�+Q�����+��J���
�9TU�4�c�z�����wAW��r�+wi���9�����((��$��������W_�G��eN9/GqL��������\��h�y��L��|{�-���@;_3#��A��u�i�y���/~-8�j��F�g�<��=~�]��������_��2a(Y>Y�[�p~A]���4�l�rx>�
��n!d�:�D���d@����d%��i��:v���{�y69��m�>����%��>Q,��M��C��[�h������|u�PW��6)���8����>����Q�Z�����]�<�
-[��B��`�1_���C'��,�<����<��i�%���
�O��A�����zqd8l�O���������F;�\���]��������+����#|2[.���Q/�����b>dFx>��p��p	�����|�R��vHlu=�vEC=���|����tIA���U�������P<�������Iu�{�+/���W����#�������P�;;{��1V@}-[����,�<�������`;�1��K����i����Y������R\}�;k�q���0��mW��<����U�vt���E]��������+�y �2����E�n�y6!����
���-��(Y��
�W�pY��.��l��d��
�0�� 2���x,q�mPW:���F�F�>Y��0��=<��&))��MR�]��������_���vd�K`}� �	��}i��v#R;
n��r���gs�4e�O�����60��iK��t��(R0��Ue>����|��g0O�o�u�y�S^�g��qI�
Y�Hh.%Gfx�����=���]�+aW���~�E���'����7;QC
���A��|�(�dfA�(�y�|���,7c&�g�������J$Rf�yI]Y������tfE����`���R!]RW��`�V�������]��������_�ue�f�-�J����)Y0�l�P��s\W���gL�e�X��<.b�.�F#C��s�/F��_pl��8��oaWH���/Yo��sF5�q3���\�����]��������_	���
�]�'�*�O��<�Z��M=��|S�DV.-�gz����|
j�����i>��xN�1$��e�������ny�]Yzv�����n���
�RW��+�����'[�r��e���]�������MR\Y0h#����2�����@�(R0hC�$�(��9��%���~�J�-����m���J�7�G�n	�������u�0w+j�����,�����[2�+i�(�
wQW���
��p!�
���55�6�,�UHA�LE�%�'�^�����nN_~?S��l�.%UF�!�zs��|�����:�Qu�n��_�����������bz�c���gmF�<�����C]�U�u�sr#�}�c��7%@����8����l���/��5������%3X�f��C��?}�|����aTF�x�Qh���>FY}-����g���i7T�j�����������C]�U���*�������������]����<c���qD��b.�R>�d��K�=��
�pC�i��3�C���v)f��<�~'��u��k�rf�z�d9^Q�m8�C��K���.�*�PWAu.E]�0�b������?��K ��@��n[Z@�
��w�4���%�A���~u/�<�2e	���~��F#���~�������+���[+������N�/������h�s���
=�U@]�KQW��T���
�oo���H�^t���vY�%�E�P�K���j��vY�|uKF�-��:�cP_��
E�.��w����^l���=�������!�����^���`!���&-��m�����C]�U�uEz��/>Y-t���
�������Fa�"��0g7���qd
�E��NP1���-����E���_*����:�'��E]Y��_O���}'�\��`���m*�8���wQW���
��p)������	.�^]l�K���C.r�{�r�Z�HX�b�wZ�d����r���Z'�{�2_%���.r�$��,tty�q�3���T��;�����
wQW���
��p%�J�4�n�|��^���NZ��&�����]����cb�6u��/�1Z*�H(u��g�z������/^0v�+o\��h�<g�������s.Y�`&�p���
=�U]�+AW���7�h!���_,�h�/HZ �-��Sw�J�e�D��+�M`�Ri�{
������p@�8����o�*WZsw�+�-�<7D�S�oFv�O]�K�����p��pmzh� ��
����H�����=����5V�'7o��U�c^]2���4\g��pocK���;6�K
���B^�uM�o�?i�n�<Us�H��Y��2��l7��)�0^���b^�U�RR+>�z���l9��f]�C����3�F��/d�����)��Qx����OC@�{����L�^��`AWX�*,�q+���,��s��C|�������wU�����xi�@3��'���������6����9@N�P^�
mBI�x��Cu��|��!-F�.1O���F���-[����b>�ig� *������'f�n����=U@P�KT|��W�^���&�-����R�L���@Q���,;��
������Eyv���!]�l3Ph�MKK���>�g��m�W��J��o��+��*9�^�R��K����x2{��(���B�������������]9���'��ao�0r,�;��F�2���(D���$�WD�H8����]�/8_]{��g9_������������bo��b�b#0R�J�(������|.��f��c0}�d%�Dh�3o�%�<Y�x3V!���D+��'s��$ez��cVD���f��R�
Q��V�B��E_c
�/v�]PT����(*^
��7N_��U_��M�Fs9��f2_6GH��Y�D�J����;\T����;�S�m&�%��(��{B#x���Fa�;����1u�T��{�p�~
`����A�*
����0(>!�),/`��6	|����y�������/�c��`�[v"����e7���m��c2_"���(�g��'�5�����S�����'�|��w�9�r��(
�������=E�S��d����{��+�3n���c�Wl�X6G��`���6	'�w�JA��ZHQ���U�&�8��_����55��b��;����E�[s�l�V����7��w�N��t�H�x)���X��G6��Md�?4�&�r�n��f��I��50hF��W����,f���1��[3����>Z�����M���#q�i-�������C8E�S���������B+5{��]c���� &T�<Y�Os=�LF��RV&-4Y������l3����=g�����vKx\_S�p������7�����r	��im���#��]�S�!���)]�p
4�B�|���v>���B��
�<p6�~Z�g���f�����Nb�X�D0:�������ae��3*x��5�'�G����P����*"1�������
�]�S����)]�����q�jT��W\{�G��C�H�~��
F0g���7yHHI�du':��BM!�h��P*��[a)������
�"�Bt*,	b�5�{,2e�[k�e2~���E8��)	 ����8�T�	�n�y�E>'��&�esdl��:$+�p��<��\��	�Q�(�!���l��L`�0����G�P���,.'m�VU��ymDs��:��8��)	@�����GR�V�D�1��n�^��$Z)_
B�W�KOZp�
@���F�d�[Q����}A;��1���24B���	�m1N�
�Z\��I�(�w�W���'�]�S�a���)]�q�a�	�����u�^�I`�
@@�}�f�2
�y9BX=�P���Np�Y*����6}��;,E�&�g+�N�k�q
))�B&,ZZwS��Q�wQK���u�l��qJ=�S�8�K��;�[����]@Vq>`sz�)���)�8�C�s�1� �qB�,�*���1G3���I[�L{�����S�"��B�"��b����!v�Y���`�p��t��z�$qJ�B������*��R���:��t�;L����9GL1
����]5ElNmX�`�Dc�P�t\�(+^Z�z'�X ���\��\��SN��{���
 �8��^��qJ=�S�8�K;�i3����{9�^���I�VY S6�C�9�Eq��C���5���y2���D-7��1�U��8�]�P=]�Ba[��LJ/)'�\��'��q��� bC��p�� ��9%�S���@���������C>�����KX$R
p���~HIP����D>:��,��(�q��������o4�+���#h��	�OXTbd�f�"^���)�]�S�����)]�5/t���fs�v�#�C�
	���6��~r7��

� �-Y[P4�(,�'x��;A��Y���5�(�R����5��.�8z�|V�fS�����]�|��{0�,s�����Sb l�?XI&�a�Lkx��Ho�$��
<�#�%�z�}�'t��u%�&i�-�!${�c� ��JvN���$���:y�7��k���{^1�������M4:O��B�r���N�J���4��E�2�=���f0n����L#�����;�m�0Y���v�(,�!���,#���F�(d�V���A��Eacr�ZX��. h��<+���M�w�N�u�P�|%�I��;�� Qlrpp�^aHEK�����$a��$r��	��*�����Er�����I�`�]��Qln�N�,�������j��
r�N��(vt�)�E:��) ���H'>����`�/HiI�1�n�ue20��*&�^2l���DO�By>^������������>����)���f���9��H�Q�V�������|��{H�,�t��"��H,����Wf	���C�F�	cu��m2y�����	��y2>fM\���1���	cqeE9�l�E��1�[���2k�N�}r_9`=��&;�
?`�"�r���N�J��Z����E�8�qi!���	@��"�4xs�
�p%@Xv�����@�lm�~�A:QI@�3/KB���	���"�*�7MeA>�K��pO��X
����]�S�A���)_	u2>�^/�����]K�nH��O��
��WN��%'���l���������@�����<�R��	d��J���"��I��b���������F������w�N��v�h�|%����~��d���M[	J'\"��I�<����z��1��q@�sq��L�IOLfnS�b<=[�pBa[�SHA-5�(a�s�O���I����C�p���v�=�S@;�K���<��
����!)�3��A�r"TJ��O�&��k��>�5_��19�V_c�d�s���t
�������aO��~�6����zM�q�h��|��{`�,v�W����C���\~8��!��|B����]qR�����9
)�k��Hq���A��c����~%�������,~���P����:�"�w���S����&^��F),zOc�_�l���:[�u���XHq(���6�A�C�Hq� ��v��I:����C�������$(IA�x���QG�����(����kk����a��������a�p�Kz_k{���7�����uQdc3�����������DOv�V��-��-�UrCT��)V?"���f��m���0ET���yAQ�y��������]�'�	:t��s����������Yt���=�S����{)�)���11z�����%���V�����v�����8`y*��{�&�9iJ{��n
:��V$s�Bi�O��J�����@�e
 ������w�O����V�T�e�-���-z�����Ok�%�c>nL��Yi�l�!�Rm��T��1s3��C�"-���dh.���<�I�������P�Qp�Nw,���J�bH��xX��J�C@�����b�L�=�SO
�Li	����<Yu�.1���L� @��R��� j�1P�$�lTD���fQ��,����dg����	}	��;���]>EK4�)nN������������3\i/�|�f�����~������
���I�D~|.%�s9�+Z����B�<��2iHC��Rq��)����<��<��-
�j-�����I��.7c����'���E�����P�o�+��`(Mv ����7C��%�%3����}��o��8i��g�i�]���+���iLc������ ey);u�)���n�PJo��D�K�Q��@[�q/V�+sq8i�B��_+�������D�}�6�s�s�0�kl~:���)���v{9�R���.57�5%���;R����Cn���sT��L�]k\p�p?�vmH�x����;m>o}^cZZ��)�]8T��c�D�_za����m��3������N�"���uc4���L�������:�9�%�����Xw�\d���Cy�[�5>03�`�9��Ga�e2.�c?'���3\i�b��_+�*���K�����n��zET��n���-��24��$2h��g�]c���eL���fa	d�y2������~~3v��c&�����7�"�K�P�r�����P�=��F�WJ�����������<��z����_>}��G���b����>������{��������������?����|�x���?}��ht������O�:����w_���Q},s�{�����O�|��%Y�X����w_���������������>�����^��1�?������/l����������Q 0Q�x�|{5,�*���:8�^SFt�P�x�|�b��O�~�����  ���4���oR��Y��E��S����R���O�7�>~�g��+���|�n������������Q�T��`�!
t3���;%Xp���Q|��Yj���G;�Y�����F��w���x�5�U�/�4�!&�>w
�X��M/��@�Sdo��\�";���
Q�����|�j%����}����d�n_�g�g_���h�|L_K��]�������zp;��=H������L=6�����l7s��)�)���,����~��qV{��+d���&���*���x�a����5���N��Q��k��#R��&2�����7e���#�X�G����o
�-���/v,�7W�����p?51�
�x,,��������_�����}���eoV;y��M�4��|�w�s��"���j�-8����3cG&~���}~��#�����_�S}����������D������6��W���z��?��K�G�����9��E����G�������^�/�|��KY�����_
Z��+~�����`����g�M`��B>��=����"�G��go�	�
	�CtA\�c�5�|fg�%84B`�"�>z��
3�_>~��_;������I+}����Iv��	�����'�_"�6�MQ}����
��i��@��O#�}s#GS����%s:�.������K�9�.������H�#	9u(��C��������
�8����`���9F��c>n����C���s(����Wc:���(���W4����������?�/�E��U�����b4fL:��[#1	�!�E�0���A	�:�rD�^YK�`k�Z��Qek�G���>��M[���������/�,����cvp[����h4��E{��>c��g9��)|�Ca��-����_�D������HJ���\�Y���%���������������h�Q��'Q�J�{�G�����g�/Y���Ol�&)��']���g$xz[�BY� ��g��]]�3y��]�A��N9�S�����m@}z�����������!�T��h�1
���&��c"�i����:e��1
�x�YXj���G����i����� ~n�H�Piz	G?���}g�:�:g��7���CgA?���>��N4ra'���s�WD���U���K#�$=����hJT�(��Mu��s���3�-�M�Yp�TjV[��w�=�t�����O�Yx�6�>e���y}���^�,�������W\�]n��!���s���1HX�G�������1He+K�%���,�lK��T�
�����,����P�Wc(/)���P�
6|]�m��lf�L�d^8����L���8T�@��7A���*�k���:;�n��5<mf�t�A��i�#ax��q��
�O/���Z3e �_n�h2�$���|�Mf!L�wup�Mf_!��C�x=�,,0�}��B��.W���29*��t
F��Z�����}_!�
�:_!������c@�S���sl~�U�)h�6��.�����l��6����f/��*-=�mgw����r�����S��������Q���,�+gr����������V��������{]%�@`^���
���$����a�� ���C��
���wW�c�4�5��<>Z�z����x�������x�����x�@�Wc �A
c�}t����-0����]�`z[����i����P�/�b�7�o��E��T?��r��`W����4�������O}:N�������t=����~��I%d�A�<
��I%��5�Q��d�+�i��|��/s�;X3]@
u,���1H�Y�BZs��X$^���t������j�1H�)�������K,]R�&Q�Wd���<��|(y'>���`z4� V��JM��6��q���J���?�,�	6����5<����&Z��i gx��q��
�OOYu-��b��F����<����5B�s��:�F�F}�Nd�@@�G���f���K=e�kUt�IS7�`b�(q��Kv�
�?��K�4.C>t�u.C>o.��1���d�Y�
��AC:�/�9��c����|>�X�4$��7�m%h2
E�����e��$8�?�}�A�������E
�'�H����D�i�P�^���%vGK|���A����Er4�#����0�*9��C��-r���
����,r��@��r�)�{
h?��2�r^X���E��k�u��M��G^�C�ZK����` f*p|����"M���c5n��%h��y
\A�T�G:#L�	P�B!$t�����C����j�y���): hp��B{SG���>����
��4`a�����s#GK����E�x�-A�/�-Af��"�Q�b��)�h�'6wk�7�������-�|!������#�t��9��4�QH��dq����_jh+�P�~q��;d�6���~u(,�#��U��#����$@Q��������H�C]��6�-�Y����d;���1E{��w�S1��w	���t�7m���^^�d�%N�h�/p(�����G��h26�_���j=}.��"��)|�C�RH��x�8h��!���w�Q�9z���h��F�8�2H��$�y�8�z���o<H�:3G�JM�����6�E��"���kJ�6���#bm���M��Z8��U��p����G�����j�/J� �o����t<�7m��+��<�Nk�8~��H�W^|]`h��ER�5��<n\$ES����-�E����i9\t�����<�KI��{W��x~�M��x�]�C�Hss�9�g^k��ae�8�_J0l�����u���q	p+g�I�l	I�$�gF��a`	�"�%��D!�C�������%8�F�i$��3+����#;���H����H�i�P������
��Gr��F�F���H`]$G����,[�Z�9}��]!��Fr<�P�MW�PP��G5�}���Ph��c���1�~��s��V�b�!����]��g�3��Y�������������a��/��	'*1��I1	��:,:����m;K1	��:,�M���� Jp��h�&����M��O�<4E��8o�*����X��M�������cvof96�EY9M��&�rHZ�9}.�/[�K���C�rOj�)ehw�j��u(/���P�P�x�W���!������g�3��}�����2�@-��Ny���R��"�P84�BE�5[x��`o�i6��P^	�Q)	���iWBY���e����C����s82E�CZ���G���>��M[�����������V�}s#GS�Uf�E��{#GS���V�nN���h��xvc��b��wy����G�N���B��U6�.��M���9��u�$�P�T��h*�y�:�z���3�d38La�m�`�n����"��@/&�Q�v7�l���$h�2��1�+���WVt�p	������?�7���z�D
p���[�:�C��xo�$��������r{�F�=p�'�JM����'�W�t�������1/A�i-"����^�����5.�K��|k����.�%[D���`o8�L�I��8d�4��Y��Lv��0�7�H��	�g8J�s���r�_��cd����hH�����-��Aa�|bC�)&AS��k�a�]$0�i�/�F���I�t`�Z�	T��F�H����H�i�P��DY��,�������a�=�8�����h�&�
!����F�F���H�wIr<�P,|sw���;����R2��nZ�=����h+�$�P2^��/�y�;�z���de)/q�/�S��?�/�i	g�/�k����2��I���/k�QM����$h��y
�q1(��$8�P�}^"j�oMQ����������)�O��x�V�>=�-K���������2{^����h2fw�z�����g�N�$"�������%���b^����8�A�.I�LlV%6����C:Ls�9O|g^���5�w��`T���&lQw��Np<�B>$���0�ilJ��#�u&�
��Yc�8�K��0����gc�����C5
�P}:�������LDTchSr{�F�=�]���ip�{���~^Ar#��`�,���BD�=Z	0
�.
f&�nK���D
T�����~�mM����C�K�����3���a�;F^:Q"E�<\g�Q�7�>�.�D�V��[9g�]NI\��.x���)���(=��l�l�8�AP�}i� �Vh�A$�z�����������A}:zk�����k	��=s#�^y�u�}O����k0���4�H��c	��V�@B�G=���hRR;x��A�;X������mNV�h�f��C�K�����s�����;�� Q�h�V����>��Lh�/��Y�x{��*&A{��+5#�AdO]~
�$(� �5^Y�$8�F�}O/>27gk��%�(���44D���o�&�������z���)�`KL��w��{�%&C�4�H�3�k��i !��nm���@��-yg�I\�v�mT���K�T��]w����C�K��������!�;v#�p����.�Vs[\@��2�H`S� �������U�Ia	�S�jq	��T�
�P�x��&3\��nDCvi���jc�7�]C�-�44D���o�&���F`]*�r{�F��Q&w����F����[=Ia%���{7���y-��������H���=)h�u#(�d@�K���F�l�!�%]GvI�i�����_h�l�W�p6�R����|��L�]�y�v��a�3�_N|�o@�b�\6\e
u�r�u#�K>d�N�S���J�x5

Q}:n���	������f���)�nON�;
.�b�F`r`p��ER����z^��-����R�R\n����]7���\b:E+��K�t17���������dv�z��:�T���C�J���7�`�H����p8�n�OJ������V��J�Pg��� U	G^N��nDKq��,��u#�B~IMACT����mB}zn����i �gn�h���{��")7�'G
�4�H����_�G����xh��I�8R+��Q�5��c!������N��8$����-�<��
��K�$���h>�s�sAO��E�5tGfm�!&A�F@��B	�����F��J��\��n�����U����|���|���)�O��x�V�>�+I�zZ�t������>0-��.������Ab]$G{I���"��Ey��X�-q�R�-��l�f�l����1^�.�j�F�Z�)a������#���,v�zX�:��y	�A��~�#&�t�_d
>MO�.����SJ��*��C0�opR�
P� ��a��q�g�=�%�V��1E�C��=�O�G���>��M[��t����a^�
e�����q(�����8��.�a�Z���(��E���C�r\j���:�����]�����j>l#��N��i���������dv�z��:����D"�����n���vu'b6%O�')X�6��R�����RE������:u�?e,�vAN�����������m�����>�xQ����O��x���>�z�uE�F5
����{���#�s��ER��Z�v\$�^���y-���z+�e��k�F$�A�����04�hmB����P!$�B�Z�u��t���^}����pJ�@K
+����"��0��YJ��{����II�DS����_�w�I��$�<���e���r�u#����	�Y���)��&%.�#AST��#���B}z)���F/K\Fr��F�&%e4t�G���$" �vZ���F�&%�U��p�HB�G�
�%I6�;���|�%��&"X���p`�j��x	�!�%]�jI�Y������q��GYc��CC����(����0h"eK�zX���EJ����r����
���mN���|�Z�����p	�:
��	*�S�:D*����H����P�i�P�n�=^���%N��s#G��<�/�?��Er�m�x�����h��PA$u$!���]9.����P��|�2V��C�9�+)Spv���4h��\��H.�yR;�zH�
C�
t���8G����~���@l���X{Y�������X����$&A������l�R�r�t(����$^"`�Zj��t)��ACT�cw�m�����%@[��(�ko�h�%����)��ER4����N���h�%����n�YkB�U�@%p-���k�{Uh�B ����-��C��k9�!����������������#��Q�K!XL�`nJ���"�D�{����h���E	�������{�c�����j�)�	p��0{�����PC����:4B��xo���Gq��
r�i �gn��Se����l�>�T�e��y�	��)�T�e�4K!b�� h��0�J9��Y���X���T`�(�.�x����4a����^�qi������g��J���4���j��)��~�J��e�\���Kp��L,9�A��
TPY��
GZ��
�ebQ��)jo6rb��@%�H����H�i�P�^����)���9�d"f����G�����������Er4�E���,��r<�P������RZ����4�D}�l\��r@hT|���v(�A�u���|��}=�_�DN��KT@{�@�)�i�*V9�&�n�i�,%AKSeU8-c�7�1)	Z��E�� '�Y�b_����5E���Xh��@����N�i�P�N>��5�~u �gn��'��f��\$�>��=V����")��EL���i !��n�Zj�Q�5M�"��g��c��S�8�A[BC�mCtvJ����"P{]�=_�e_O�W��J��(����F�3�l�&�����Si�`�-+`�	��
�S�,@D�
 �a�8�A�=�26.V���@�+�;�24B��xo�����X�J�\r{�F�����&MR�������_F�����lHH����te*6B|�x�&���#���p��7,?�v!�@�uE��|��}=�^^*a�e����GN���"��`K�sI�dt�:���5F���`�-�]H��hZ�O0C�MFv�g��}�fc��LQ{�����T(���)�O��x�V�>wZ��l�%�Vn�����0���S����8j��K�G���]�Y%�x��X�@�������xmM]q�s(r���Vpb,��V������j@��j/�z���/6H��-�pF@�����x�]d����^�>��] �>%��pg�7 )��P�����p��M�g��/��B����/�u h����$��=�OGo��`Z�r{�F���b��7�.�b��e	��G'�����#�	)� �Z��j����y���r�R�
<����[�e�v!�a���������r������0�
�9W���X0����4/%������A�LA&�����kp�B�+$`M�NN��n��{zKv#���):��v*V�Y	���;o�*��N���������_C�����S��O}
U6X�h�c���������i�.$�]u(6����kK�/RZK����HP���oQ���bk��CqX��+u����)��8����	p�=�A�h�H7��

���n�O�$h�lp�b 3k�1	Z>,H������%8�P4E�lV���)j{�&B�:4D���o�&�����5�i�����H����u�gvI���������%=A1y���R<�F��F�(���F"n��h�S�f�*$K��N��.��t�
wX��+u�����)�:F[f���ht���d8�4�E��x�;�n�cR�l�Q�w4�g�KpT��c��(��_������V���)jk6�]	�24D���o�&��W����7u���37R45���g���7R45<9 ���")��
���$��^r���v�Fx�
K6��$q�{ni4��N0���FV>�_o�C7�^�F��&���q�������6X�s�Ik���~�u#��� .�g��CJ��K��<7x�d/.A�%�Ce
/x�58�F���
������0�i��y$h���q$��U�O�r��%�GK|��F���yt��&s���G]$G�����Ywt
_�P������R��M��,��w����Q��
��L*n����'��?���'�y��^��8�'(�".q��(u��~�u��#.MA\�m.|�h�C����4���A\u���

p��p{��uj�.�X�q	�x�����O��x���>=��S(�U�@n��H� .�ui��t�
��)��ip�
���Y��s�2��5��-�@-PiK�7���Ql��t�n.���v!����s!�y��_��8���NqiP(�J���q#�i����[Nkt���4���*rI����xR4!$0U�Q������wn�5N�n��v
a�������x�V�>�N�nnTUGr��F��B��2�.����*Ba]$�Q'��g]I���C��[:@%�C�X�]�(v����@>����MH���;d�t��[��Lv��0�uR�E��t���x�i�6���v�����,���J	�t�0�Y��$�|���t�`p��������%����_8�B
/UN��{��t<�7m��������q���)��Te�0K.�b�K�Q���<�H�=/�@�_D�|�G=���i+��R#���w��;���������X~�$-��<d�t�1[��Lv��0�u�-E����B�
>��*���"�lI�F�_A��Z!1���������NL������z���k����� fK���X����5�R�@����A�i{P�c���z�j��������R8N����W!�=���R��G=�t�A���mrTk��w��l	�PZ1'�o
�w!�-�u���<��{=v�$�l�':��UY��oz<o��0���:�u7|�My���T(������'M��������k�����Ip���3[�����5���!���#AST��#���B}z���<�t�Gr��F�F�y�d�O������h2V]��"9}��M�O#	9u(6��!(�*J"���$�T�*�
�/d-j��bV�V����]Gm��S���Ce���H���4
}�P����"\�����l��7k�VL��*�+S��8����r�~�Z�� ����C�������)jQd9�)�<u$h���q(��U�O��*W�i$�on�ht���S[s�-!V������hiax��Ad=t(���2�� ����~
1��q(\H���|JV����8'����\��x.�yF;�z���8�(���o22����k��f����=�,�fex��+@��H���#&���U2��z-'�I_��}�
>g���������*A#T�cO�m���t�lr������{��O��e\$�^�y	��u-���)�j�1Kd�x��X�-K'��R,��6�>q��+���2R7~d7Nj��������:fK�����������B ��X����f�Hn�knP-��%h
7X��|Bj�Yy	��
��507���Kp��0{���N�=0E�Q�����MQ}:����
��nL]��������-��x�����{#G[����p1]$�Q7>�4�<���Q�b�qiXwS���F�%%�R�W��b��o���?qHq�������������\q�=�����x��i��p���s�JCq\�Yq��
�]i��6t6R4��k�i����p��{zK�%���Bm���i�i h����$��=�O�p��d�4��37R4��7l�R�,�:7R4�����L���h
7�� �R<�A�����q���+��������Q��Z������$�!�����������������@^�VQFy�]d=�&n����������	����h��Ec��`U����0[�'Z���j��8J&��44B��xo���������o���)�������I���������ER���,���B��<�A�|�F�R�� ���\j$J�v]���A�|8�5�E�Kw���|��:>K������N��(�e�vI������`��z|)
�V21+4��`_8}g��I^�}�0�@P/'�Yb�g�A�yd���O�r�L�A#T�����A}:z�I�;��r{�F�����M��I���_F��cr7m��D����gI�.%I����%>s9�*5���ut>$F�71^���Y���,�y�:�z���+?�(�����Jl�������E6	��8�m�h.A�4=~���5,�*D-.@���W���b�\��.�����m�90Cm�VZ�i h���q!��A�O/	�*��4��37R4I�g���9�����$�E1�$��J�&	��g���"Z��6=���X�te����.��(�����o�6N�����_�`��3����Xw����h���-n�������`�D���?��E{�D�1=%�E�N��g=�=���q�?��B�a�,\MA#T�����A}:z�u-u �gn��+/&���I��`L,�4�H��cr3��Y�G=��� ��`��W+�����T��7�i�N�6�A'i����_�`��3����Xw�A�����g�M\
�`���u����*l*�� @U�Q����4y���('�YbOai����j\��7\�:�B���o� ���B`])N�=s#E�T�'��g�G��
�LN�t�{5.b�"�H���RX��U@Y�����F�s�B���NE�����C&Fq(�!�������'������
�E�1�h4�.Y+�T�Q�$,��8
��Fb���������K`%(w��$�
g��=�%��j��]M��� �C�o4�MQ}:����
��h/�\[�#�}s#G����i��G��h2f/|���"9}.��E���C�RXG
�%������V�QX�G�
�.b�h��i
KHa����������!�;FFh�MJ��4ZLZ�"�o���"Oa�[Z��.(�A�KI�T�Ct9b	�Mw2)	��rH�|�8 ��C1^�����R��ZQ�S��E��E�|	���t�7m���^^�����Z����F�F�y���2�.���d��6��"9}��qY�(�:a��4>*�|	n��� ��9�%S�����1�F�����:�p��.���c�����Q����������C�[���;>L��-�nb
N8G[F_1	�Z�q0�����t(�����s8����#{e��q���o�&��_�
�5
�����-��~��6�3�R4W��������H�N	)u#VKt1J�`�����r���_Bk;���h�\�.��4��:�p��.���c/�dI#�E�K�6�~�.�hK�z}!��I� /@���h����/O+.@C;eUpex-��r�� ��^�:��on�PC;�R-���F�>�M���th�P��B�i �gn���N��9�ip�{�)>"m��������N�|���@B�G=����4��h�Dj��D��L��t�('�D*m��A*�+�u���<S]x=Lu�.�f��e���q��~NDS\�3��[�;1	&l��>����m}��M""i��Z�Qk%'�Y7b�^	n�Mq�j���
gqU��n	���t�7m��K#��ZG�[Gr��F�F�zZ�Ic���MJ�P�N���h����%7�$�x�����
v(R-�@}�"�v�J�3�\���M�����pHd�#������!�;�J�W�J8W����`�;��]*��4R�Q\���n�?g�H$.A����*�&�'��?�Kp��h�,|�S�:��}2�j	���t�7m��������%��������a��k��RG��h26Z^��������^|i}�K���Q�b���9�X�8,H���k�p�z�������}8�%��C^�p�e8�c^��qAh��^�8N���|��o�/���x�Re�`���K�d(���Zr�TmBi)	�^]7�����`���g�=�%��.��)j
xg��R�H����P�i�P���baA�2�����m�=�a�<�H���6�"��J������<�$�x��XY.9ZS�PN/+W\H����V�(29����Q�WW8d���\���v�����f�4�r�������f��\�
�'�z^�b4�6�3��A^��1��m4U���-C���g����2�����,��_��0����O��x��>=��WI�i �gn�h O��N���hX�����o�\K�P��Nz������x��XY.��q�VQI�5�����b��qx��&�	�K��4��:��p��.�Z�c���EM )Y��
�F��"LR�+U9�������%��E.���vYb��cI����)'�Y/�a�Lf���X���`#�
MU���t��7m��c��y��e �gn���/O�WM����{e	h^�H������]R<�E�L��}<W*�@��|����	-rb1�����p�^�!�e���2�g������U���m$����gM�`�C%�����qn�\+JI��XSq��(.A�������\2r�u#�l���X��):�^���H����H�i�P�.�\�y^bwd���^���|]$G���U�i^�mcQ9Z�%6|X���M{��Xy/-8	�%Lj��zP����Q�����D1H�����:��p��.���N-�(R"g /+_��v�!��,�_���K��)G�B�{����(.�s��/a|�f���/������m��BmJ����44B��xo�����0S��:��37R4)	�y�ip�MJ�'q�"*��MJ��kD��Qb%�����w���V�H�������
���Z�����x9�!�e���2�������Ct)�(����z��Dv9��~�MI��}�gX$H���`��5@y>+�!Eq	�!$H�n�?��Ap
��{�K�R����)jRQG�D��l	���t�7m���^���iZb���on�ht�|`�|j{��&c�,��SV�F��
�(�HB���]�S�%H|�,U`/'v�8(j�����l�@�E<$���]���v����u�.E[q�&~	oU�T���E��C'��[�y�5������5@�#���Jd�6 ,+|���FKp���{�KMn��nMQ������44D�9v'��M�O'/�uE,0
����
\�'GJm\$�>/��7�y	L�g)����+�%���v���6>d���L���������v�P�x�t�c�������a���%���O�MYO^q�y�_�i��+}�y�R��4n��V+�*����Z[Z#�W�*o���Kp����]����@%2���}q	���t�7m���J��%�*�+�r]$G��e�h�<{)��Fr<�Plx/sV �J�P�+>u��?S�<U�P� 
&�P�^��x/�y���z�~��^�������9G�E����)���;k��{3�50 ��@��'/����:�N�1*�������]�r�:���0�|?<l @&�El��:�n�����|gC�x�	n�m]s�,�^$�V�!�K(Z�K
���W���� h��]QiB��^���&Ml��G�g7����������ahP�r�-����`5��.���������vQ�@���b�3!������'�Q�|��^�u�����]x��I)V�K��O������g�dr���-w�p��W� �Q�����3�e����;q���].�J^�|���Sy��G�""�c�|��[���&�8���}�=��\P��808T7����q���>f�\��8 )�bA1�/X� N�K���.��L�A�y�q��$��*AH�7���eX't�K������S��^J�J�F"�����*���0:�:�
A�*E�/ZI���A���q!�J��xbp�E�:����'�"%�x6[���6ao�J���k�'80��E{��=4a��"�9S�6Jg�v{�C�V����GY��J�ZB���Z��4��f,B'�C���WWL&Zn1����-�}5��:jv�����"�Ie(y�8���E(�B�SA/�������C'	e";��e��G:���y>wiD�pi�^0pE}�e.kS��#*mB#��'�6��y�a�`=��K.��n_f�EpI���0j�.��>�Y!�t8P<J#N�K�0����^��=���e*f�he����3�*\�u
����]x9�1���1�����/��0����.#��'������>+{��A�M�������0��]�J\C�=7pE}�%��X�E��+*mB$��+�6=8���S,G�~I�%F7�
��@KY��[�����8<�G	��.g sn��Q�`^E5��"�J^c�����oBf>1����.�}U��:�v��
�����!8V�	R;=0��@*�� &��=��Vh�PUx��,�� ����
��\���4	o?q�����������J�0����M�#���}���\P��4�G���Y��������[�����Bd!���@�(�8�-���8���De���a&%o��<���-��3��[�u�����]x5����g�0B�l/��~`/����yBH��d�������L"d
a��Y?�3�w9a��O�}��|�r�V�R�$N��������w�����J�p�����b����	�q�^Pt��P]"��4�E�������V��B�i!��"�r�S����p��A�O�>(5��L��*��K�4��d��[�u�����]x!�1�����#��������|E�v�<^2#{��:�h��y���e��C��FF�n>wD+m��jGI��u��	(������J�0����Mn!��	���g3�-�vC���E{��Q>x�,B��B��.lsA2�x
Z*"�(Q��E��T�
������+y@������q��e�/^_G�nL!k����Y@`�F���r`/��P���hW *��fG����F�_�M;�>��U�1�����p�F�V�R�:��rE�p�~l��/=FWT��H��W(m�{���c�
��\pt{8��n�n'�|t��;��p,��QBqJ[jh���=�cv�
q�D�QA�$lh<z[��GF���e\'m������+QY^B��)�L"k;kFU���D�Z�.W��Q��a�A�DE$<Y�C����	l:%*B�����U����%��%2c����PheQ6����ctE�M�[{��f��t���g7�������pt;������y�1���2BqJ[B�6_!�d���v1;-�����B.�S] Nc�	�P�2�����E�����Mt$x%��t���&������^�o(��f�X(Xv��s�K�:���Aw����
u����%��%�L|�+�S7�
F�{����	�xk�P�,��&6y�Mq����n7]������/���R70�%�X���u��b���QBqJ["�?iKrwG@	y<=
��x�v�H���*���q(m�I[��Rv�u���|�����D���O9��\x�&�8�6����F����D��\V<��yv�4tT�ZkSu?�].�I[:���Pv��M9�2w�Pi&�����Y�%�kL�`'w�EvI�66�B��]b
�>�.�����)�~����Fe��%J
�\4�;��]0M��O�H�����OCe��N�2�����#c7f�7��JY�
��!����2�W
MYm�4U=�s��lh>�F!0)������1
�iG;��D+kI���6"Mo#��Zlb�0:��&���Ai�pi�k�'��h/(����;��� p�������(,�GD-k����S�������;���Cj8��UU����b(k��Z��v�u$�&a��zTxS��^t�zW<���Q���JV�|\�G
��"w"Al��#�5h"T_$�_]U{:��4����3����Q��i�*�����J����
��50�)�S�"��\pt��8�*����G����]�q8��(�8.��%��:�*���F�s
`�u(P|
c�M(��q��e�/e_G�n"'��OY�����b/���i/�r����Fyv��@H�F��6�:�C|"�l�
(�V�����.�h�-	!�x��Z�i-��K���6ao�J�\K`^!=�u�l�����yu���f��Z�co�E(�k	������QN����������[�8}����j�9�.��H�P�2�S���U�����M�~��Z�@�\t�']��=l�W3+T��I�/+��������������6�0`QZ+$�2��]��� ��n���1��z�|���[���&��aQ�u>�a�Cw�|�����E��Ag���������xQ��pY��6�"���b�C��O�.����8��Z���e\'k��������J�{��Z�O�/������fN�LQ�3<c���69cE�$��ga��J(#�����%d�����u4�������A����J���+
�m����iG�V�hi����� aK��~{���T]D8�E�]��9�n�B��rD	U��������4b�l�)[��Jv�u���7F���"���*@�����Q@�@8��Y,�pq]'��
��B��c�8�����OGp�F���Z�*>�rE}�O+"BG��#*mB#��'�6�EDU��M�b���U"
�v������O��}.w�E�oe�J���@� �H�����I,�P<B��\��N������:^����i��e��c�^G�nx�,s�����l/������0�O����$4Kv}��D�n ���2l���Q�����zUx�����U��)����+��,iJ%~g�1����D���Bi���0���J��n.8�=L���aQz�pt;����f�-���g2xm�f���f�L��S�R��R�,	V�f�4�S{asQf�U$���a�Cq��N�2���K�#c7����5@y
1K7�@�f�� Ag	���C�_W��p9D��h��
n����^Tdp�K����j��/��H��@���atB�M��[���f7 4h��*4��6sA����I*(�)����+��J���Y��-��0��_:(e��%+C���F�/`��Y�F�[����6obI4��A�,�:5�t_�.��n��a���i$E��I
��e]��-�����l��z��d��H�C� �����A(+ C;���%����2�V�R��yx��A8����O�0:��&���Ai��$����X��m�����<�
E`���v�������e�t�7�W����T��$i�������>j�9��u�|���"�)�P�2��L������C,%s�p���U��9a�k��^�5���KviH����2?j�	A@iv�%(�'������8wiD+b����>�2��b�0:��&4��}Bi���mLp����EbE���Y���������B��X&!�)+B���C:�@!���z"������RS��e�Y����-��!��FU,�:�t_�.��j��-���e(HXF|B��Nx7�fAV��@���Fv���;��;�N�
:/��N�>��!�K#%K�L��y����:�8����+*mB$��+�6;��z/�)������������)�p��8��l!����HU�%/��X��@������(�85-
�����	��!�2�������::H[��xn>1��L�$-�}���:�u�XK��������>A�+���z�'�N ��c�t����y���
G�o��h�~��-cQ�P/��� ��R.��
�6�o�J�E5���L����������[���rz��N>"'�u�����V��t8P<�!NK<0�g���������m���t��U�O-57��X�u*���d]z��qt�zD.g�^]"q��\x)��Bg�%��PU�a.]e)T.�ih�T�l:�;'|(��D���e��%4�����uBB��@����J�0����M�-�5��*>���h7/
.��B�U(���3�)�E(�m���r[��y�y(e����������n��3���N���py����b(a��IX���u�u$�&�5x34��8�(|]W����p���n���2;�NB�����q�@�m.��"�:*������ Z	KE������B�P	���NK��	�6ao�J��E'���t�l���K��������E�c�bXnCQ���E��I��Y��t8P<� �A�CTO�%�}^o��#��@e�<�q"��6�����eZ'a���������$yF��-������T��D#�%8�#��M`0�r;�!���xc�=dd�o�K�"���]��"\�CF���JN��W:�n��	�xk�P������K��f.(��+�8[g������B-B1�M��1�:(%����%��)I������yt�A�K����%�{t�P�2��L�������M�!s���	<'���e��6�����Q�7��z�(�V(�,�A��!��>����p|���V�RA��\Q���J���atD�Mh�[���fId*��	U���m���;�.����Ew��~G��C.(�$
�t���:(�J�
�F���Dxq>��t�U�R�<����������0��>"��k�D5��n������7SC��\"�C���O�a��W")���aG0���<��G|����[�F�c��&��|��`��#��d�L��7*mL&��-�6��YHGl>���h�
.u((��EK'08D��"-��B�m.T`�������#�H�DGq��$��A{ad�R`=y��=)�*X����5���y9:o��`��:��F�p���	���&*
9��Y����@�?�����^8���*$�<�+�m:�)Y�����?�2?=�U�+�o�z�xw�P�d����$3��G��1�B�������2x��{����F:�eh��X���Z�R��e��*�*�����{�A�nN��'Ca���U����]�;/G,�����|�����/�0>�N�j�F�*v�/�!#���}�E�\Z/�U�6��D-�T|n�N��f���?��JB����{�����xo�P�dcjQ4W�=>���h�q^b����G���,�c>7�G�������{8&����N�<`��T��>����dB�,�-���gix�&C����U����]�;�G,s0����x�fW6��B����:��*�z.�
�e�I�+~}/�j��s�	n�&����V��G�V���.fTz�����xo�P�4�����D���\p����4r;�J��G/_k�;V����i�M�����0��'��R8��m�pDW;�2�����[i��]������:�j���u��&7�������h�����p��&���j���'�� �A������b�U�x?�mb��`=�$j�EV�Gf-��a�F��h�{���&��Y��m>���h#+08�S��"md�A���EY�"�+���������+�%�wX
y3:ANY����E:��T$��M'��������}���w^�Nh^:�t��2QF�V���p'}��?T�)U.�#��>t�^'^�K���N ������1"�M':L��Y	��G�����4��a�F����{���f�14��	�:|fsA�����Y��t���lipo6�,B�I� Qz�Y��a:��2�i�H��q�	i���w�V�i�@��������$�c6�N
����x��q�/�Z��G<��L���@=0V2�p�5��A�h����]B<�qI
[�;�`K��oc��3��m&��a*��9pE-��(����a�D����{���&[�������3��v���"�Yl�E(�ML�#Ll�E(�����"����D%�It3��Z�x�rx3&��m�$��b��^p�����:I�j�n�uD�&���I"��o\�`��ae������	�5�����R"x��U����v	�p��nnF��p�I�T1����I w�[���6c��J�0	�,����\P�L��I����E�2	�;���E�$�(�u8P<�$��$
"U���#��0��,�#x�I�{RHQT"�*��p�1��W1��Bx������R!@E��|F��g~Ee2�L�����������L��BLR��D	��I���������v�G1���n8U���I���#��a�D����{���&[�f6�vl>���h�/�Q�[g�vg+��1���v'������x�Irt�����d.~\�!NK�:�m�u"���{���7�i�P��x��/������MK���a��r�P@���
��D��tY��
�G��
��[]������|:��<��L�g�6}h$1^y��O��R��->�S��<��#(m�a1��o3��6�D�mi���e������n�����. �-�U0v[��-�m�����A�v�� M���!��Rj���1�
]�����C|���~�������~���W������������_�����v�������~���?�L����/�|������������x��������W9�������'B?Q�������_�������O_����A�������2���O5����.���,'�F����>+�.��j����M�Y�<p�SJ��}�������n�����[�D<� -�]q��;�������f�TD��P{M����6��n��yVbm#�P&k�,�"��=��Fx�B�U�Ip)-�t�C�}��F���v��e��.��0��������V��W1p�}�H�����_B%��?��WP��N��`�dL�G3m?`B��ec:��s�>q,�����X����e��R+=o��7�\��zh@wt�z��(m�.)ml�Y1�����h��;S�&��d�3��<I�����D�0��#����_������|�S���dy_��o����������g3-�Q���U������n����PKz"
�/�E+PK
{2K18sept/pg_part_head/10_3.outUX���Y���Y���]�%7r%��_�oj��a����=f���hf%���ck��V*�6����Vg����_7����3�a;X�Lxp�������_��?�������O����$x��_������������OO�z������-�^�������O���oH?���~��_�_=�����?�������c���Y���\�%-?���O?~������9�����O_�yz�����~x�����z��_o��������W_~��W������_����F}���������o�����������4��7������O����_��������f�w����w�����w_?|��>N�n��o�����������������O?����+��������?}��jr&;�T�ur��}��������K��[��S\�2�������O���ov�����>}����O?����{~�����������,�u���#�����w����O�����~���������>/�B[�(E���e�>X�DJ���}
F*Jg��Q���?o�8�!������h���L�(/��#��q�	�
��?�����g�G����A���������{���WQ�io�h�muwo<�CW�M�?��������������>}����/O������Y�����������ww�C��?����/�k`��)�������0k������|g��|x�+�u���n>��#��������e�Y�{����E����7O�-���Pn����d�l�M�u���4��G>��lQ����m����mLt'~=��������7�4o�I���g����g��|Z������O���c`#_��o��6��'���XuM:V]����v>R$�����F��"����Q�{:����������kV������	��n������_����!�����{�(��*_6�n>��Y6P�#��Az��	:����.I��#N�1���%�f��t�	��MJ��c
����n����2�����A��w_�{���>|\���D&��h�b6�g�������	7���."���B�������7�)��7�>�����O��}�-����<��/��?-���<VD^�l���rs
*m�9=�i���=@�[����Y���x3��)��U�N�����5�f$���q��T��1Y�L�e}�����G�]�fj�Y%�b���Z>toT2���a������b`u����hJ�z���>�L�n+Q�l/\.:xZ�pY'2��T��f��9������q����V8[�G�~�h�|)�l���;�L�B��C|�����Gc�~Y�I�J�f;�XA�/�c�M��w�����??��O������O�~��_���^mM�����,��iA
��R����c�#|V��e����t�����q��}D����{v�Z�����O�?�?���#����=_������������������?|����>����A)(>�+�*o]������O���'��x���x�~c�_��>�G
��x�
�[9����H?+S���X��J��|��lw��S��,/���l�RO�-QV�qR��N�_58��Lw5�zLLj0��i��5���&N[<���F�ps4�	�&1a:��������f@:�BW�HHMF#�t�0�h��6��3�4�����������<�vVa�+mW�y��c�(�V����N��^{W{mO{������W'o*o�}�7Z�S�
�|8EH@s�������2�/~3�l���4���Y����������`{�
|?J�2���d�b��A{tB[]���.	0����?oyj����f��">�[vw�����	h����5z��b���.C�l���,q��`l�1���E�8��y3�^Yy0W��Ch���9����OC�&����<���Lqm����@����w���J�0'1������VWW]}O]����+�5w���cu�zyGe���~r�b��p7����BI��1�F�^��Zg��r[�x��y/�5w>Pa��Y\5����Y�56��pTq=_#uR�M�-W��N������9��pWcCOc�������Gj��S�QG��&5�<���k,4�j,��iEe���<��h{����K,��N�}p"q��zV���0�F��o;����X���y��X��:]g�o�K�=�4��jl�il��xe���}��X�WI�i���{2.�8������F�q���D	����-g,;�a������~��h{���(��>�<�>r�Nhlj�X>���������T�(��g64)��
��*l�)lP�t����H�|7��_���wxq5GlR	
���������S�f�F_�2���)������j����#bil%�t�E�b����h+�x���jb��b�������B�,
��!�V�|WesOe����+�Xk��6���)�[2*�)b�s�n��rzJ�*�H��^���F�;�gJ�������N9�:�,	��v��2�������6E���]2�����Z�* �I�1�U�o?X}�����o�����`�-�����#�%3Xj"�����L�=
 �o��e<
<������Y��%<�x�cKE3P-�l������	VT ��r��W6-D����\��o�Y���%�B����w!R��2)s%D�w�@�a�,nf0L�"��$��N'�(=c��K�����q�/{K^�<�N7��R��
<5��p�7��q�A��	�mR�}���;��WE��q�������H�����.>���QFe��GY�P|;����vFf�#	\yy<>s)��^�<[x��g�EG������]�iK���G�|��^�9���E"���i�Q���Hs:�7�AW}3od�E
hj���r�dF��]�����@�\���1����%3��d|Z����I��G��7������(L^�#%�����^���&�%H��T��{�:�W0e�:��l�}N�o��2X����y��]QdO;�
U����{��j����L�d�@����)�G`��������.�����(0������(Y@�gt��I���G�����u�48~y<�����o�a�Z���&�>�#����7����	A����G��nI%.V��*��P��B�����"��5l7��2s�#e��
���x�}��y�"���%bW����|_`?r6O1�0"����x!���/�sl��u��9��
�������9�1R����yBy7L+�T^���p����2=<��S��T�by��%3u���3�<{M��:^Q'P��wd�'����%�i������K��j�����[�Ke>�H��8��:���VH�09V��z���\Hv6�%�\~�ZE5L}���L\e�U�Jp�Iq���%�/��Y��6DC���{��%�����Vb^�#��eOz[�����mKf��������;]�<��qO��7L�bs��[/�6�.��
*�o`���iA�������]���a����\��2V���o]��ZY��p�M"����Ep��Z��?>��������/��q�$�c�)��D����_2H���2����o��B�Q���5������LS������
���2wqW���2�+s%���:3P}y���o��	��P���smD�e	Gg$.�/�m�k�l
�"�-���0���$��Q\}��]�V�3�~�j��A`�P���r�fr���twE=�	���J���zd�.��1U��-����*�&�R�X�fQ���I�l7�_��F�O����K�.��&1�A���3)Z.�N�GC�'����*�
5+��VZ6�'����Z)�$^���.��z�+�]���+�����C+�	�3�[��
64�_���
�Q�\��g����2��}��[�$ ���}��@`t6���N�w��1K����'~�7�r���(��{!����twE=�	���J�{�C�e�2,nB��<�����x���
�1���ea'������E"#A,���`.X4�}����P��o��2 �Y��R?�kM�
����e���w��*}����]QwE�+�wE��KV`�����%��h%��2��e@P���J"���krR���z��*g�P����gA�������3/��o����Te�����m?�{_�g
�xh��������_������pWt%��8=��I�%�y���zL(x�����-��Nv�(	��x����l�M��n�������M��D��-X�g�����N�o��bg�.-���`X����?Q	xE+�*�4���������"�]��rC��f������qa�#��v4J�����m��sVV��[{Kp�)�h9��C	���������'��^�W�������r���G��i��q4S$�^QxE�+�x�i��[H\P���a	&�<$���>U��P����
�����Qd+�B��������l�>�U=c&�wV>�K-�����J�������dt
1���6E������+��HxEW�H��E������7��59�fOS�f	f;����L|+��^a<��*R��t^�
�r��P9	2k-�X������x���T�.r�A��A"B��m�������+��HxEW���Dn8���E�r�������n���^�C�"|��d��E��oL�<����S��$C��p�/�g����<j�N�o���t��O�~����gq��_X`0����.��z�+^���+3�f���r�,�F�rQ'�x�
r�-<X\�=��mO��9�o�kN_
,`,[���Lr|��:��!T�s�����7��+����}�
���A�
=	�����Zn�����+�^Y���xeR���1��5�]�a�����4�x�v��|����4���)t��rBHH������@�`�Q������x��^Y6�KB�W:��R �K��w���Z���^���+�^Y���xe�H�Q�����������P�@�����x��K����\�d�`CS���1�^����������)<�i>�����&���q�|��S �4$M��3���`��lxe�W����C72o����������S����Z�K����B�)&'Qu�����\����!}"��m+#_�FVy ���mSGT��5udt��e�������(k�������:����H�+SG����_^2fv,Y��<_���3|��,���qE���3���}���ZP�,���������#{F���&k���6uX�zu���=k�MD1e�������SG�n���RGV ud/���z$Y�.�b%�cH��P����Q���d~Qy0�O'of�M�$:jO��<�.\C�I{�P�:�@�D���3.4��	�N�o�92�7��t7s���ZRD��=���g{7sd{�#+�9��f�t���r	K���
�K�Z�d�or(�s��
k�zv�E:���6����R����|3�>u�	u�=�DAy���H�@K��A�>�i,�W&���R��i/x�����5k=L���l/udRG�����yd���+U|=������V����T
j�B,Uu	d3P�y?�����b&���a�0�|9�sT���*L�'���C�����\}]�:��)Z.��P��gk�(U���pk�J�n���#�K9�����W
�����%3�zf��K��H�~�"��>
����wC����@U@�����c�L�ekG���g_���IN��u����+�Y;����� Ks��/������h���M�^��	�����#�C�rT���dvm��^���dFkn���\��� k/�W(7|
���U�>r��'�R���~�O����SG:$V�<y��}���>#`��Wf��f��+�'~�����e��@��]Z�����UY2m
��D��Mj?��Pz�h=��>x�4��X��#
�w�+�(Rtk�9i	��2��E�Zi���L�n��5�h�G���}�gW1q��X�w_
������}���w5����}�#��,Y!OO9)�t�r>'��Q���X���N&p���=;�yz��.�gKC�"-�3�Oe>�F]�/Q�r��%�1:��z�E��n����'�@8�����n�����;��}we���fh� E�yS�2ZqI�%��x�����n�dg���M��Je�T���C�7�����L.���������7sB}��}����-�Y��I"�+���"���o��q�n�����;��}wi������E[jdPR�fr(��P�$�+����M+�8���&�C-:ra���QG#z�Z���I�k��K����w���������������
<V����p��.���`WNv�.��C���dH�D������5IB{1�-QQ�r���E�H5Y�$`P�����y[T�O��
�+g�Ds����b?�%tep��/���w�p�-�R�N7����\v�`W����`&��%+5GB����j���5GhJ0�/��F��^}���V�Y������P�2^���UtK�s���P��N)�_��/;�)�[- �WJc��z�5���.���`WNv�.�]�6��e�
GJ���7>�6�x�_� �Z�R4������4gv����.�!�����	-:pK��l������+�FL}�
tV�d�2lO\�g��_~���\v�`W��V)|.�]!W_���d),�+���D���HF�_���d>�bWe�X��Q&/[��!��$~1^�md�oK�S�^��}��J�&����g~mD���$'�'���*o�����|v�`W�R��	#Q�e�
2!e�4�J������[��78��Z���D�`��/���0������^}#���$�����ae��|�����yh������/{�a�����UL4�{�waW����+i�o�&~)�Xbc��b�B��6�_��t����+(X�!8�&��g|!�!F��.��F,����V��J���A��X�	�=h���2��#���~���AE�L��o��+��]��������_������a�p�M�������,Q2X��%1��=-�;[��EG��R2�"���5��%G<^��b�e�y����$���R�i_��9�t�����Cx����*���Iw�{�+/���W����#c�X2xM��$�o��eqDl�=~1^�'�U\j����`�$�7�R���/�f�q,�=~��YU�13�F<;�������[C����|�m�]@Z�
\Z;�E;:v����|w�pW�J��<���,�Oe��~g�vc���9Bn���:�%���C������\F-_����b�2V�d�|�_tEy��}&���w�A���W�p�����+<���h�����[����+��]y���w�����E��\3('pK����2��r9*���B��W_��W/�h0<��6����|��H�xi+lM���m��P@1����������J�)�\�lr����p��.���pW^w���]i���-KV:�x���Q�|���[��u�xv���R��t.�f<�[R�����%#����}p��/�4kNH������4RA���Y��+��hEk��5���`��w�W����+)��{N�F�,�6��MLj�W���<!�3om��K7�+=�����.�������x�����S~�,fxM���x���z�%�B�n�A��6�h?����v��|x�{�+/��W�tv�W�dT2e��2K����H�n/�<��tk�Y���B�Nn���&����\�/���������Gc�'���]�������8��m2��Fh�l���V�.���pW^w�/������Al7��Z�Q)�]j����F}q���0�����l��"�07�r�s^����G�[o�;R_��d3B���>�����������6�Xt?��f�����?������pwz�� ��
��~JwU��!���e��mky3^z�x�����]N{���@��O���<E/�!{�N��[������),�����P�w�T��KQ�)`����%J�����B�f4�s���
=�U�]�qW:'7T}�
Z�j�~U���F=��Q_vf�V*4�Wzq�=e����W	�}����Q����/������_X�)�8����U�Aj��e����%�:��\i�����g~�]�U����*\Jx��@����V)�dl����&���p�����kN�<L�z3^�g����E�����������>��O_n#������>=�pB}pWF����I���f~� ��f�-[��S
�W������]�U�w����UY2��x�7K&�h�w�d~1��e��w�1�&�No2�/�x�����F�htT��-94`]��HI�������k���w�E���E�<z2�i�Dn4�9��]��*�����+~F��%�e������
��kjy���|-uimTf(J�J����D�0�	��
b<�B�G�yB3�S���������=�u�f��6'��:ZN��z*���.�*�pWAw��]��j(�
KV�gB��%��s"�o"*�L�*����i	���}6��X�fJ��
��hN�2^U���x�(��+��������-X$�]��hN���&=\}���BwpW�R�+�qW�d�R��<�g����MN��2�+���!�I���6��o��e<*�#��y������x�!�;��]x������ty�s�3_~��MbE�,+������������]�U�v���@���d����K#��S�(�-����Q�Hm����h���4
�M'��n�v����|��\�}������]y�� @k��?���0������9���W�
����]�U�����*\	�2nd� L�
��z�f���!��=�[���4�i�f�2-�9*����-i��hV'�%j�x<������la��x�B���l����nX��;��?�
|Z�v��u�b�Bs0W�R�U	������������@�J�^N��4����}P��[��Q#��	�!�Xl3��\��f����0���
|r.A����w���F��_���	�QV�.�*�PVQe/e�Jq ����|pA�9�y�
I�4
[�H���+��s�Fs�hy>dc.Mm%��>����\r��h��	�5��:�6����!8����b�����wqU������xiA3�(NX��f����i[]��Xs�r$1����QX�X����x�!>���*l�'lRn��k���+,���}��Lv�T��u�����R�(3:���bH�T�R ;.��Uv6�
`
s���8� Zxx
�$.[�82F3�4�dZ:H�XJ�e���K|��K��������=aq\���
�����b�@^�*���w�S�����x)t����G�W(��[�R�DJv'���������}�!y���0Z�`�V��t��y�z��&#�V��=a
_��V�������o��GVB�������.X*��RQ,�K=�k���#Z���=$��a08������B�x|��`6K`�?^�~3X���J���;��mt��6�(,��Z���I��)(����Fgx�]xT����<*^
��I�U�P���Q�L9g�������{~ac�:H���������*��I�:W$H���@����eK���C���g�5D�u���v���Gs�������]<T����*^�������S,W/�hi'��?��n;�9����)����
QJ�k	.	�9�K~��p����5|����{������V_��t-1�R5G�Q��h�WJ���.*�PQ�@Y���c��y�|x3�B��G[��vp�"�<_��\��2�Aits~��
�����Df�15��@4��z4�N6����������]�S�!���)^J4���,�Ky���]=;r�=��p3X�����,���q�9F�2[��E����-Q9`#��=p�iahr��h�P���vUX6��1_�Jl���!�<�+I�s�=�S�9�KaN�,����z��6y����`��q�G��Qyv���R���t�+;Q{e�x);zc������2�V��e��>�{������
z�b�M�=`�
�p���m�
)��QN�.�)�PNI��.E9�����DUv��!�8	�x0����[X��\4�z3X��Tr�n���sqH�C��5����5w�r����� ����^	,�	,
_��O`�^�<�oA�rJ=�S9�K��9���oxeW�^�9�.�`M���G�2�	Q ���!�e���<��DB(�/��i,�Tq+�J���:��8�e������kR�@&r��
J4~����8��)	`����8�d'���NkR�/���J���E�+������)���9��,Z���k�Cs���5g����J���)E��6��0�w�re�����G��Nw!N�qJ�t)�)�������lV�Q2��:H[�{{i��}CJ���e�X_���eu��9�Z�����YN�k�p
��o�b���x��Y��P�m�a~��E8��)	 ���'��#����t
�����H �S.��������S��������
I��8$��8�H�<*=�2L�[yqjN!��]!��n�=`S��6������3��.�)�NI��.m��N�����l�k�]��	iD��S,a����c��>���c(���Q�?m3��biP�h���������B�"�b���q,XJ�����N:��B�R�� N�R��Y��]]�{h�>��I4u�*l����4�PF������*-w���������4F�V��I�Z���B�|H��
B���'S ���F��SLqtk�t��z�$qJ����f $��+�!
�<�

%"�<�[���0�{�|��D����h���x�$A�
j��w(�[h��7���������B}�?�
V�#[m���fht3�t��z�$qJ�B�b����^��-�l�u(p�EG��2f��&rC�f���a0�hN�]��Dk�4�t��L��{�	}= rb��K��o�|�����WdV8������C8%�S��y^�#S"�hP������o;����/�6,m��uSB���T�&fK�}�)���*7�������knN^�s���5<�]������	�v[�F���.�)�NY��/������Z5�>}��z#!�3��;�O%q�Op�X��)}��`*F1C)�	l��dJDC@���"�|�+����@�s�|�,E���^�W �wN��p��|%�ISz����wHef&��m���k�`����5Z$>}�e0�����IK4�4��w���	t,��p>@8�0bM�D�V���Ew����� �c0��wN��p��|%�I��	�v�.��|xr����:��
iI�;�7M	�M�F;T��9}����i
��+O��G;r�P�������Z������6Y�\`G��]�S�A���)_	q"v*���U���;�BD���h�������R �JK�����fH8�UXSU��4h�����LM<bG`�X��m����|�q�����#g���c��r�Z�VR��Gg`�]�S�a���)_�q�#1��R����]��X>��*Gp�F"���/��8]*b0�}.��?��l;�0E5ui>��L<Z�wB_[�S���Ykt�������w=z�l2:n�{��{�,q�WB�����6s��:�zy��:-p�
����W�wuvNK�c'�
yv��Y�N�+������\.�J[q}mNl�6�0�[�u���KH�;�o��7-�����|��{�,q�WB�����`HF���
���8Y�����QF3=��8��A���Y7#��s���k��������o��F��:�(��	�maNpbh	;k|'��EA�_��d}����9��)����0'2ylZ'�������$r������3�R�����v���K�/��������a]�G(�R����5�'��9�J�Y��\���cM�' ��
�
W��8���9e�S��y��cVOg�[X��w2q��{��{��p%���.��y��h���������`������9��������t/�5�eKV*b�B����8���9e�S����p(�i���	�y�F�}7z��u�K�����G=��\%.e��4i��
���*�l���<�y���-����u�0�:s�X��x�bZ��bt�K�Y�{[��He�����K����&c��������}'���h[����IdZ`��b��f�OT���9�#.[���b�G�I��J{������6�~[X4���\'.���wbI�km�T�f��^��.��$l�3���t��t=&��Z�Q-���R���B�4uV������l����(3dY����l�U���3Mk�N��,6��#Bp}�%�G_hY�����>���������~�B���Kq�M0QDq0q� u�����C�{����<Z�O�����N�zb�	3�*&m�������
����=�EwJ������oE?��^�]�S��c�@�_z�yKa�y��9�������P%`L;|I*{j���F������c�*m��(|H�M��5���H�5GmpV6
���ET��B[��R
it
z_i{��7���R������c�k��G3AL�I��P�
�A�d��F�5<���P���|�G	�S�2>;����&��1�����y���6w���J�m&pp����z8sz_k{H��7���R��lj�C����
�fK�L�"Gm*m�B�O���dH���s:-=!�u^W�������yw�g��w(m���>��: Yw��"VK����)�3���(����=4T���J{%J�J��K� �^@<�Jt���_*�~(�U��Zok()�`!y@�����adB�����JXs����3Z���x�6F�����Z�B>���cS���x����j+�����29���������,-�-
����-�n������$��<�MJ,�(�~CC�M{���m`�^�(�J�5������kGJ�[��$�I����-m��a����E��>V\dT��������8�h��k�����bBe�����c�@�,D��>�q�%���J|�Y	JT|Y*����Z��3�]���t��,|���/@Cw}dB%!E`�����d~�b��_��:���Kt����n��X��%�[���=|B�����Bt� Y����kC��4>2�G.��@>>
����m#t��iWs���O�����w��|7+�q��,>+��7�2���[��[����&�i��/?�����`g�|�����~x������X�����~x���w����w������~~�����~������|������E���Y�O�:����w_?|��>�1�}��~z����_�yJ�-8/�{���w��'���X?����}y���o����O�;7B��5����>����������-��~�����x�G#�����o����2�?e��}�x�b��O>��i�O ^F�����7'P,&>����?S9���JE�S��o>}�����O���!����1��C��C9%���s���&�6m�y�f���B
�G���i>���6���Y�A[��qh@��������m@�������U�;�l�|�x�-L;E��H�K�X���)vrl�S�����|�j%����}����d�n?�g��]
�h���J[R{z����@�q�����h=%����]��������}���}[�E2��4��������8��#���)O^G��X�SS���T3�|��*�c'������5��lc����m�f��q�u3~�Q-��\�8��EM���MA8�WAM,Z]��(4~S"������O�0�kol��}�������������v�`O���6��������]8�������<�}������~�l��?������#�W�P������~����������/���WO����}����Z~�������7���G1�_����G����}��k������_
���+~������������&�������)L�V�����<����Hbx�K}���&��D��<Gq	�P)9t�8#'������>��b�>|����b���9��dG6�5��1E�1ED�:x~4E�����*������7�}s#G�!��!1A��v��&c����.�����nY'!�K����\�@����!���"���2g�9��sdb�y����t�O�u����j,��?�SU�[�����V�|�0�lQw��E���	��;���]!�4�
`��;��MF��Ic������� ���Y"2[+��%R�J�Hfz4B��xo�������vv������b�����cz�H�}c?��I
c��b���,��W�KH�R��!	["���v<�^�������>�b��yX��i�{�<���{�X�Wc,���I�w@
��
������^�
D��`���w*Zs�R�7w��!�}se������4|��>��m6�P��@��L��9�S�����m@}z���A-��"�_n�hTV��o�~)&�Q\����!��U@��E@��:K)��A.�\Bdyq�s/y2�+���������y�O�+�C_�]�+��v��;q�+���u������wbsiBl������Q�abRii�-6���q�WE�5�����w�yb%�b����>�@�����L%���+gr����������E
���&=|�����Uv��z���5"�����Ek���F�N�F��BH��z���A��'�h����r/�@`�:�2m��V����{���j�a"��e�X���5_�]�WdC	�.����}NVz����f�U�����E0�pP0�n
%�����{q��|��&�`P	������t��7m�������ooz�/7B40$Wb�jz�F��;�N/��@x�������/u�����P_?��1�����h�
T�W��6�g��h��Y��B��Y�
Ex5����Gj��B����������Ey�Xi[m��m��B�7���Z^��_����M��_;M�-^l�����f*e���6�����Q���E�����,�iP�.�Qy�H��El����,)�\]d����Q\b/��1_4-�Q�8/G����B\a�|�����]��e��,��eQ����&�����x�,����x�P�Wc(�A
c����P;�(������j��\*W{��{K
�\PJ�:���Wj��j��+r������ ��7���,��p�*/r��>g�M�����U�
�[^����m�����N/��(.JbS"���vIdA��~ZrB��YXH�l��W)���mv��w�v���
��&&^���t������j�1J�!������S}����oAR��B�bP�������m�O���������
&�����/Yl�����P��9�[��8l�-ZWM/r��>g�M���t���5��iz�/7B�U���%�Q_�b���M;mS�++�^}�*�y9$�x�����B��PF��A�X/�|����
��
�����}�!��:�!�7�����dD$�l�d�C�VyG�E8!���P��z�M�CL�&��9@����[���II��53k-�g^41ur���>�����\���&���qZ�����t��7m���^���4�!D�}s#G���<z%�(o��&)0v�AM�����2i��$!�
���x_�K|IZ�	!�T��!'E���hg7V:_���?Q��b���c�#������%��Q��;���&+�vv'J�a���	0Y^��B�(�1B����my���@�e6���%8�N��h.�p^-���!������hz�C�9v&��I�OGu1��z<����)��*�3�\$��V���qz�H�=�J��0���%{���D���h���/�-������@�-�3���Z[F�������'�\�i�3����w���V���hAf���9�A�>�/�n�����h��ei��`�������?_+c�tF����n�peF�U��R�%8�F4<��'�h�FP�NiP��&h���q$��U�OG{�6&���&�on���p]`�,��H��&ctc�$����|#�^��,���or���Xy"���*J�.�
�lzi��x)U[��]���F�>��D��x"�y^8�zx���$Z��c@cl�>�ak6{�]��Q7�FB���mTDEeR�f��$��f�
9��M:^�����)�D�m����6(�x���"h����%��E�O/(���E`z��37R4A����[M/I�%�J4m�DG�������.����+S���tSK�
�Z�,��^P]1�j���H��M��,�D�E���"�y�8�z����+L��b�D>������_����S��x�7�3�$h�7��)
_��0l��R��+�
�yE�7�����'�DC��Zsc�Z�D�����&h���q$��U�O�����;M1��{#G�����'96@ea9M�,�e>�U��2�:�y�G��R��m�WxU(�]��|_���C����j��7|^a���T�\G*i�����C+�Kd9��^���7�Y�8�H�����,.A���	\Ri��EJ���9(��a�W4^��E�2��h>0Em���JW<{}4E��8o�*����,h���������c����.���dl4���[�/���3f!�y>���H9^�P�����R���Z�h��u(L�
,�z�a��CqHCi���4�)������D(d��L�*����cL���y�_��(]�[P���q.%ASz��8`LV�Yq	ZB�����
��Y�b�I	f�pd�Z�;���7AST��C���B}z�k�J�0S�7�}s#�Q�\TS���"9Z	������]$G�#�Y(uL���/u(���`#�+�5�55���c��+���q|Rl��|�'�*�uL��<K�y=,u�
J���K6��/��T^���;��Z*�
[���i�
	��Vxql;�^Q
r��%�>��o�W+��%@����oz4B��xo������W�������b��<8�"��r�{
.{���MP�o���1���saD������2^��u�/���({��^�"!g�cs�1I{����:�Js��������I3�����3�����v� [0BN��op�R�����
B,(LTqM;�	����%�Up�z9�z-�eR>X����q:�����O��x���>"��dj������b�A�����ER�=�_F����(�����/� J9�������,��9�t�l� 8r7����qHji�#�4�	���!�;t!��e�4����������u��`��k���	��m(����R�d1	�6=ZA�����un�g���DD�����K4�I��r}4E��8o�*���������Mn����B�-�Z?����s	����_��*9Z�9�BZ�#I��R�b�����[�8 �����u
�#�|�c��v�+M[)a����\�xi�s����mw���N��4�d�'4�>�.�����\);2.(�IjH	@�mKV����������3�n����/�g�$^�Q��B{_�o�KYJy4B��xo�������)���"�gn�0{)�r[��\$�^�1����Gd��sa���"s�B�V���G�p\f2j��
�u=�lxm�L�I�n�[>Hs\�!�%]�qI�	�����,���5<S�{�B�
���]D=�L|����Ni��R�+���:��3�����%"2�qVpNz�\���x[+�� R�_����E�����x���>�d$�-Z��=s#�>#��{���^.�b�����
N/I��M��n�-/bF^�A����������0_tW�.�2�����MN*�
t7J�[�!�%]�oI�����0��4H�����'t��t�]���t������.�K�w!@$���"�f�	p@GU�tx
���u!��e@����tTl|X���"h���q!��A�O��*�`C��`�����������yZ��/������y.�ER4tT8o�4�JH�R�6=��B�9y��M���CG��J%��"��i��l�>�!�%]�iI����0��Q�8��N%	d���!��"���Ms�#�S���[ �r
�������� ]�f�n���%8�D4��Vo��6��q"<��A�X_�P}:N�����8-O��X_����{�������\$�^�18j�����;X�8�HY��:NK�#���%kq�R���Dd���F�&eL�DrZ�u��t���^{�q*#�"��������`�G"�-R�8!����a!b���� @1�n�F�)�I�
o�p	�z
�%��+����^�f���^�P}:^��6	��y�D0�#�gn�h���I
/�so�h�;�.�/I�x�7������^���2+�k��r�E��6��"H��~)FR�mi���qHdI�Y�y�:z=�u�)Y"KP���["���=�/�^�A�9]$`�Bx�$�� P�S�p��
7��2��,B(��p���g����2�e�l-Q�E��r�����O��x�&�>/��5�0����)/B��(�by�H��
������\$�����3�����/�"V�J��#��=�
�
Q��a"<� B)B���$�cw"�+�:�J:OSG����8��d{t�R���9n�9�.����E]2�d��Db4���U���`oz��e�7���=�p���W�c��+�x�)�o�^�P}:�����������	�GV�
�/x���")\4�Y����G�B��yz�����JW	�JWi@��d3l��8"H��������#�B�U�u|�t���^?]��S8�3��_
��>�/�q������
���M�9;�T�(+o�%h%�	,oH��;.�Y7b�YI������)j�"�*�����)�O��x�V�>�("O���o��77r��D�9�Er4���K����"9��"�Na��D,�K���RSR(�HMi	'zz�S�a������o0���z�W�u��t���^[]��S���vAC7�D l���.�6�%���!	<��;;y�h(_�l� �����8�K��+mN�?�BmH�'��X^�P}:�������$x^�HL/r{�F�&$��M�2�5��$~)�����EB��z+{e��k�_W6X�{I
��V�tL���5$AI�i8�W�u��t���^O����lH7`Z6����pH��u�F�����m�a����3�u&�K�6f���[d6��g��={�	�H��5nD����"h���q#��M�O�����y���_�F`��&)��H����k3I�&g����<�-p��"!�K��
ge�
5�����vz�P��`���1�M`��T\��8$���H+�<I������{��nZ���%:B�\��/���(	�	������<��6������n�U1G^Ppi�/�F4��&��Zc�7"�����t��7m��#{1��ez��37R�����y^fA����a}��Kd��\$EC������h	)^�F������������Y�B��9�

`�]���4>�W���+�y�:�z���C������/���?�/��V����L)�=�FX��h|�xCY�a��$8�C�=u�IA�
6v�-������A3T�c�m[���2����XO}��37R4�������I����I���f�G���2
��L���h�K}�{��$LD��m�Ri�{�S9: �J��T���Y�L�RW���+�y�:�z����D1��&�4�v��"�B��(�f\|~���h�0��4�R�*��>)$@s}����C,"���� ��� �V~��j�+uE�N/�F�>�M�����+1�n�`'�gn�h�+54f�BRun�h�+���v	s%j]�$��R����M\4��Z�-Wz�����E%����_v�=�C�J{q�=OSg_M]!��3G��c��,}>�.�X�q�"�c�h��������������+���O�Kp��hh+Sdy`���V�^�����O��x���>��V�y��������A�GY�"��ER41t����F�&���-$�x��zvJ����c�l[BJ��Ad�l�-�j��:��F��8�����y��]��Tt�"1>?�Ka]���u�H��0|nS���t�a��Z�T@�p��J&1���;}��z���g]�}��NT���.��
Ry�B���o� ���B>-��U^����M"��}��ER4.DP����")b.q7�!#�x���v���4'���[|���w��#�B������`��g���N{]m�=_�e_O-W�nJ��(t��pEx��
�6�E��`�!�\����G�����xT�[q	���X��,;t999	��q�����LQ�������E����F�i�P�^�j�	��)�D$�$E�7R4�H��I
.��IIb!��""v��n�Z�i������V��	�	�V���� ��R>oRL�n�a}�������������C9%����qc�=�/�n�K8�!AD�i����J|(x��hq	7���R����o��%8�F��;u ��L����<~���)�O��x�V�>�K�Q����Mn����W0��r8s������`�c[P +G{-0����&!�K������L.q	6�+�lpw
�z�7=��UCo���t��=����Uz��U]��Tu�6H��6o�\����t>�.��
�b=�K�������\*s�+.A[���f�f��.�Y�)�~�'�����e&���"h����&��I�O�#�D�4�����{�F�����evG�|IG.L���B_$E��K~.^�E���S�0W����u
nz��Bi�X�k6�7�q�:����]W����u��S�u�D�������6��\��E���+
�����v"��L�e2�����J�&�,(�I��=u%��ck��
=+�A}4C�9�!��E�O���������������$����8(�Ln�l�	�������s�D���>���3����i���x]Gz����!J|g�PW-!�DVz��*=���.�z*�:N�h��v	���}z�_�C�<���.�7P-)	/�;�{���Hg�C�(�V��G���g�����/�k�����
����b��	���t�7m�s�x�e��������-�,�-K�.��%�%�,G�W��r��y����/u(h�a`����8:x�'���8�7����C�Q���;,�t�~���^��zC.�p{.�X�6��l���~���NS?Wz]&q	Z3�H����T�HIpD@Ur�#�d��g���������X�R�__
Q}:����	���X��_�����)H,���4�9:o."�����\j�m���	)^�F����+\�H���M.������!���	$��a�{�a���jL�1�2'7�N����"�F���J$�"�(.A�w$��M[)	Z�AaK�S@V%8�F���
��S���7��
�Q}:n���	����Pn���=s#E�FPm*7�\$E�F`��y����h���$�9:{/s#��B�
���$���Y*�l/�5�������~�F����C/�]�E�����q�EP�YPN���Mxt�.�	���UN� /�����^J�����VJ���I��
�3���g�g=�YZ�V~���{���f]M/�F�>�M����m��<Op>��<n�(N/IAGR�Y
A���b��<�������K=��j�<����(�i�>6��+c�W�l���I{�����y������b�3����@�T�8�t�]����0�*o3�B47��p�WR&��@sy�����#��g=�� �qj���X��	��No����O��x���>=&l���i�%7��
	^����")&l_�x����h��yp')��{���X: "��L��=K� "lR)��T�6D������C"Kw��;�[�^o]'!Z�i@�*�t�_��#�B�h�x�[y	�TK`R*s`T�T�JI�B�����PA���g��=�%;:+"bk�Z���J�:�	���t�7m����"���8n�77r�������N�:|#G[r���0�]$G�p��z�CK��R�be������jEKz�S��$\R�nMTzMb��������:RKw�������(��l���+���4l��H`�@�����lH���nxXs
b4�bhMNX�$��8�K4��,�*��
5!	�"�����O��x���>�-���	��7R����g)�ER4iI���ER���>oy#��/� ���
/��Q�	�|�
>�G����xq����]�g��3����\��H�6�
|�e���b��E����' �TX������:~�F�	�7=�]���a[q>^�����D�V�� 	U��A#T�����A}:z�y�Eqz��37R����D�E��r�{
���O/I�Wc,Di�U_$�x��a�AYWaY����M�Sb��������^���	����:Kw������_�7�e
��'����E8��nh�"N��:BL��[W{�"��&/.A�����4h�!'�Y7bOai��pd�(�me[��MQ}:����
��Rb���0O����������e�5�F����G�<�A���F����G�4�IB�:~Cf	������RbH��P8o�3	+���Pt������_Gf��S���C]w����8K�p����M�,��vu'PU=NR�V��%&��*�P@�+�`�K�'5�K`6��pN�^��k�9��X�Vh�K8��h��"h��s�I�m{P����8�s������b����X����")���A\1�\$�^����>�HH�Rb��5k*��k�Ru���RvB������p�R�i�4e�?����QY���u�����$�,��A=1��,�1�^��5��R
	:i�`�1���UP�%q	�W�Jn�>�38\��n��3bgv#��)jBl��
lT�7AST��#���B}z�k�3�)�7�}s#G��<:�fc~�H�F�1z	�Lo���sY
��������������BeiQc?kqJ�_��.���l�+����CqHe������i�������8�em��Fw*�/�V���E�M������t)�K���Z�	�m�r!	��

�
V�A���g�=���Q-`��)j�}����/���>w�M���� N����	�����{�F���7U
������;��q�"^%E�x�B�yE���en�Je	�ZCZ���[�`K>�5�7��d�$�!�������g�������`���L�m����E�M�Q��8f27�'%�������y`6tR�i@
x
P,&�g}���2��Rwc��V�S�gz4C���o�"������4_�������i�����2���hZ}������\$E��S~E^�C�<�F#YB���,�?�^����nCa���a{���]��!��������������S�)�c3_�Mi����M?�u���H����������:����-.���=�fV�����$'�Yb�c�q�;�B{"����N/�F�>�M���t��j�4����{�F
s �4�����k0��f���s#�=�"�+�DV����cI!�\j�����;0�uj����d�ch�|�KSa�C"K��?O\�_q�q��f��~)���
��_����u�fL�.�%ha�|2����<���H7���@�
VN��n����(���1EM ��G�A}4D���o�&��GBkK(5����)��D��r��@���r��@/�E���/u#��_�Vk_���t��}�)v���Ed�l6('��8����qY���u����C"�,s��J��+�K�-�na��i���'��
�	�F�*�2,���\H��ha��Y;e���� �<��-�/f��S�Tb��"h���� ��=�O�9��nz�������a� e�X}�H��9�� �4�.��a��
m���"!�K=��� ��^	��5�l|�5��k���B){��Ib����qH_������������;t!b�����f)P��W�`���"�G+�
�����M s.@g4���mEJ�&	�s�E$+(�Y7bOa	��5X�1Em "(�;W���!�O��x�6�>�@��~��xd�/	D`u����")�@����m��M ��Y
'!�K�����7������[�#�������/���fW��8$����_�a��s����Yw����h���s|�TA���"�F`�������l�v��A ����p9E��K�w#�
��.���;9	��{K��n���)j�=�0�l��7AST��#���B}z����r<����9��!�r�}i��h4��Ew��>�YH�|$	9^�P�����] �|gY��(�A,
H#J�k��0[�7�������_�h��s����]w��$KA�o .����`��GX��!��r-�%h��!���3V�
�RJ����VOkuPzu.�Y�b�h������7��e��Z��[�o���>��M[������X����������-f�'�-r����h"���,m�K-��Y��I��B�"�����U�d�C�a7B������O�M�{"�P�CJ�p�e8Oa^��1V�Sd�,8��TwD�E8���o������f\�$82CTP�^��E��1@��%��ev���*�t(�����}8����`��R���M�����x�V�>�4e��j�)�����9�d%�5����.��MY�F�f�r�\$G���Y�n��,b�^�P��Q�Z������r�;��� ��s
�E��pHp�#��������;v(H�I"�mJ�u�!���"L$���)r����h�z~;)����HI���QY$<2N^�98�N�}G�;0D-�D�����6P:����L�i�P�;�i�u��1��{�F��I��4I��ER�����_.��AQ�K�R'b%�$�(V�e�������dg�`:�OP�}6@�-�CrZ��8-�y���z���}+�<�����H���`���|'����f���������kT��� o�0	�@���z��jn�N�8�A��,M��j��j<��P���/�F�>�M���t�����'XPmo��+//�f}�H��c�R�Q_.�b��<�+EV����p��j ��T�{��9"��er��iu������|��:>�p��.���C���I���e���M|�mX���"�A$��r��MI^���@
P�Y�x�0���JW+R� (�Y�a�t�;X�����)���E����A�i{P���b^�0����)����A�?�\$�^���y.-��{5��c�����/� V6K����������	I[�*��
.u�LVz�^����C:�p�e8O_^}�q	h��Z��E�@-5����E#����~r��Ie��B$A]��-2�*VQ$#� '�Y7bOii��Z��[St�����)=|S�3����H�i�P�.2"+3�O�������A�-K�.��MJ���l9:�rW!#�����Jni
Z��� ���,UH�����.0�R�Oo�|I;����:r�p��.�"��b�,�����,�\�|����"\���F.��y��$&A��������bdq	����Z��,�5GZ�9
�%�>K��)j[�,���M����P�i�P�n�\�<�����77r4:��k���v��&����]5m-V#-�"bU_�P�4�������`3�F��=�%G�l��O�Mo��fJ��8����\���v�����$��C��eW8	����vc�_�#`���YmPz-h��m�ih>y��o������-��g��WAp�:
�e2�����?����o}4D���o�&������a������
Q���n����hZ�
��^.��i���n�-/��/u#V�K��r5.�"��#���Z6��o��������F���C��p�e8�k^��1�I7�H�HL$�p��6(c���F�m�Z!	�
��G�9�a�%h;��� �N�=f�g��=�%'9����2+T���E����F�i�P��e��0M��3|��H���23�J���������f:Ls�ER4�'������HH�R7b��M�-\�����9��:��d���&h����s+�ER]���.�yR��zH�:5�p��B��������.�u��k��h���h�<�Dc���`oz�X��4`E�X�����$`k��jk6�
������t<�7m����pH]�	������)��
00�i���oQ)��
��H�\D�H��f')�����Jni�5VW�DTks���5��1@F�����n�9�W}r[���-�y��zX�:����h\����
[pB������`�_Q��h��ze
�����/�	`�fQ���6�<Z�����Y�5T��B{���4k_P7x4B��xo������P��	�p���37R���KiZ�Mq��{
���N/I�W�2y��,!�=���Y�Sx#\�N-E������ ~���Y���\SCh�&�A�C2�x�e<O[_m]��R��F4N��'��]���lQ��aK�Q6��,.������R�doJ���{�eJ�)�g���D�h.f�����@M�/�^�P}�=��m�s���W�rT��������R�4��������a�t�]�A�)���E�x���W�B#U0��&JD�_�Q��H�����6_�.�!}e���2�'�������.�����Q��*n���"\�����=�����v��?�
)2Y\��m���H��a���u#�����
��j�lD�}�vP�MQ}:����
��6�@���x��=|����r���I��.��e�Ga�"G�J���F�����][n�:�Wh�L��I�A>@�/���Ub�@���E�w���wC�x�'��u�u���"Y���9z8�%��%R�]~�@���Z\\�E�r���j�K�6D�V�
S-��O�2�����]�M*v%��Z�P�����=����D(����Q��B���S�5������w@@� �������Cp�P�z��(�y����(m�����ctE�-�G{���F_ol�b���n.8�����	?z�p%@i�(c��l��R�g�W{8�%������H ^����.���W��$
���oSe��O�2�W���d�����li"���"��+��\�
�j ����0h�Z�..A����v%��W�<��h����%+��|��z.����Z�|�7������m�W�h���y���[_Q�]�z���v��Gb���-�8NeK�_K]�$�������"�Q��%He.�qT`��
[�}�����]x	��k�%��-�$�D���+���*�=uSS��@����2Q��@�z�c�IY�Op�A������z��AG$��j��	��`�J[D7�5����g3}�����3{�	E�D���

s
%1����������<� \S��%��D��&U|�����	9�2zTNx�)&Zn
1����,�}���>�u�@��+�<j{�HK-�{���V�(S���rl��� �39��*����66�e3��$�����%:���Q���R:��]Qi"��^�������w������Q=����0�l�1�d��5jo�QI�����g�i��S����u����Z]I<���OO�������[<"L�,�>%�p_�.��j��O���G(c���41��
M���<��:�&
�@���Qo��Dr����Pg�����\�W�4D�����
�>BX��atB�-��G����4�2�2��y��\P����xZ��M(���lc
�E������(���x�A<d�
B�&Zo�p�������*�g^h��V V�b�b��X���u�}�����U��U��J�����0_ID_.%��*Y~�A�|1.d������#�$�TF����]��X���x����W������]Qi"��^�������xLq�_m7�F�M�*�n��8���k~��6�p�_����X���YBq�Y�z�h�NJ'NmJ��JKB�|��m���\@�h���8���a��e�/`�G�n���y3A=
E�|"��;;��LP� �H,���3�e�E�H��d�}�dB0d��+�9����x=��������T�h\��\�:��ctE�-�G{�����
�Sl���X�������XF����u���8�7���u���j��S�g�\��F<h���R+y�(�<�DH�ia���|b*o��[��rv�}���|"��]fs)��%o�1��K�(�a-4.<;�>�L�3B��)�����3����B�^>w�D/o)
d/&^h��/�:��*m�$>���x�����	�|6sA�?Pbp�/��&�+e�s������*7|�'D<�-��8��tT:R�J���$��!�'Zm�*p�q?q���e�'o��������3���i��|����>��W g����}������z�j�r�����Lze]�!'n�\����� b/o�\��x�A [�9H3w�Pis����������|������~�b�+�7��W0��g��	E��18��k����������D��S����%�sxe�Cs�t�Qq*o��[��Bv�}���jT��B��X!SN��Z�{a����t7���mjr!K�#�2�^�3��
���J_!��-����+�����&�u�*4�h,�H�������J[����	��J����^;|6sA1�%��v6�J�������M(����
���Yq�ZB�V��b=j��d�*6�|.�_�ns���fSM��O�2�W����^�P�`
�����JtWD?��\XID�Y����f0��@I��Z�l��R��bU����?�p�A�z�7%r���t
k��Y��*m� >���X�4�
����g3���l��i�hyQ�+�]QFrA�/c|YQP����T���nCQ��A<B��4z��$�*�G_j��wpe��X�}*���j]|��9�0�	�0��J�;C������%6�>��&�mr5��'�����,h��s�/���n��H~�p�A*�����h���;��.F'T��A|�?(m�n1��6�>����/��Kg�~�����lB1���P�����x�A�*��Iar���� <�����2�,��M����_2�*�q��e��Z�G�n� 4k8%t�u~T
�6�`'��� �`3���������?��7p:a����e�"GE_��%��e�������x��A�19$���*m� >���X�X�I�	�����m���_����$��M(����]s�/c�c��i$#^��Y��X����TD���t���;����C)t�ElC�-�%�T�2�S���������-�)yk}f{�������^��!�0�N��S"%��`�A��'O�*���`B`�Q�{9��4bT�t�,����1B��r0D�0:��4��}Bi�`�W����\P�4x	���M(�`)�����b�@�Y�����gi��]�#b��t��N;�tb\�T��6���5b�>3���W�}����X]|���P�k7��F���5��*Z��7{�����1�;��=fC0�����H�-��`���>&��V�y=��4�����
W4
E�ld���+*mA$>�+������J���g7�PF������c������|�1E���g���P��P�����
��n�r�+����i�9���/>1U����,�}���>�u�
����1����"�1����eC�t"��$��B������xp�O+�PD���`R
��x���0�gq&�7^h��i]�_j�[�l|�?(m�n1��>��o�������m`��l�)��������/c�5J�[�l���T.�j����"/z)��d=��������f�Q��ve��]�����Q�[�yJ�+	�L��rF���^��$""�r�r��Sonu%�9�y�ME��4wA0\I���(W��P��������v%��uP[W4\I��������F����J[�^�X�=z�H��|`�����{�1�d�bF��7�����YB��+	'��z�r����P	K��\q	���9�%�	�T�2������������$��
��-��|'��nM-��^�C%�kr������`�h��D{�9�!T���K)+�Ea��Cp�P��
�~����~������J[���
�-�k�����R]���S{�	���J�����c}�������e>�$����J��#{���
1�+)KrM�����&�d�(i:^B��Z�i��e��]��G�nzC�,s�%�#��i?G���^�+�;a��k��"���
���0�\�	(��`,��h���B�����$����)�z��+�T7%���1��������Bi�
�F�������vs�1VG�=Yq�3y��XA�l���8fcq��t�H�����T�T��T�0�i�0���tP���������������'���i��e�/b��G�n��k��N������D�w�o����n�:����@�����%Zza�����Bf@����].��Z*�����&�� ��F'T��I|�?(m�n}.ES&��I�z�����&hp��Jg�~�:����M(��	L�H�p�x�A4����J"�:~Z����JAXW1���� u�WSU��O�2�W�K��b7���yR���71kL��x���������KOtB��GL���A��m�'����G+�x���$R[������^�R����+�AJ#�tQi��>��E:(���)���g3� ]w|f��{A���bp-+
-7��O��YaU	r�%{�U�N*���J o��t���$'�Y�T�2�S�L�����(��C/%k�F6wh�!�����ED���73{���
��7X0P]:���L\����6��CS����2�^�R��d��a�>�^:�N����hP�JO��!��g3��55 �v6���Ye,(�����b��C���E"�y�A<��L�.��E�OHm9$5���"eq�o�U������M!���i��e�/e��G�n���X�2�^TVD��F"�������T����:5�!
��@gAyC^99�9�i��ec��z9��4���4P�|h���h���F(��K���� �J�������\p����9[��6��1:�Go�Y���Y5����YBq
]�I���.�3�*(����_������K�!�=7���\�}:����]zM�y��a��&�����F(�����^I�(5A��v�MN���;Z���p�1���('�b���\b���#�)��x�!6"�t)��K��	��`�J[�4��b��>����c#��
�g�PL�����v6��c#���oc����D���&7#�<y��PV���h��+R���T�2���L��������C#x�$�2%M��O�i����2Z��l�7h�@� N��Q�=MX6=�PA =�R$���e��%$�N���
��j{��atB�-�G����jB�`�l���g��
1(f=�������M(z�-=&�8P<� QKKK7�����cn�Z�d��R!�R�H�C5����b�j���Z��*v�}T��6x5��P>&{��i��^�%�4�,�k���.��
��j�fP�k�f�%���L#�����Cp�F����6����qE�${�v��)=FWT��H|�W(m�z1�9�������cX������p+��t���~[$�0�y��a��g	E<�$�E�3������1�G
\� �[���{'M�  1��L�$.�}1��>bv����z%� ���Q�|�9���&��:d{E��a�_I`b!�,�M=��i�2��P*0~��d"�����-]-dR;�^����h�P�b�b^q�S;|6sA��^�~�~	��c�3c�hI�;S����x�B4����@K��g��vh)�|sPO�jb����-���i��e��_��G�nq'��3>��&o�J��g������=dkx�a?�1�3@2�����B0f|V��=����������JR�3���1�3
msBb�0:��4��}Bi�<�X�����\P�V������{A1d\����x�lB1�]E�*��2O�:�����|���A��))��,���'s�{M�
��F��(?�C$����������S�:�A[i~K3�ae�v&��6!T!��Wiu������g0��B�Rm�
��4�qGG�RT���U��*��FOT��G|�K(mU��
�L�b�����P�A|�~h7�r�����s6B�lB1������(����1	�DIh�|DIqF��$�7t�S:�KO�h��R��Tb��e3�
��>�u��O��]����i�:���o��&G��#�|�G0Jc�>��n�M����
��7k/lb|^�M'=K:�[;�G�bP�+�r��o|�c(mYn#��m���gNO��DV�8����0������Co�1�W�+	����7|�P����&Wu�'�������������������Sq���]����]�;o�+��M��8l�A�����s�����?_��0"&"&�7A�F^�@E
��"Y�C~=���b����W4FLx�"�-K����b��J[�L ��)�+��b����K�D�lB1�L�R.�v6�b&��2^�,�����o)����a
���y��r������Yc<��$��&��J���l���7�G�nN%s���%s�(�8;��+
kh/�Z��5;��T(fh��t��������B0@��'����%��5[[4���"�rJ�=FT��P|�c(myE��M�I�9�=W]>p�M8�+�o�c������#p�x�X��X���eZ��$ExDR;V��Jy\���.��5
�Fs������X��l���?�G�nqG�K,23q��dZB3�f��@t"B��QM�(�A��x�����
��`�����s��'j�"�W�M,�K��!��?���NH���c�G����g;��VS�@���g:�2���j;z�p��#���m�1,��5�oX�����59�pEz""���� �z�H����6J������e��.^q_����7����#���MJ���0�W������D������c���^D����rg��f�
�q�t,�3_?B	��F_T��S|�S(m��A3�|�b�����bx�HP*(�W�P����e�"��G��D�"�p�x�K��Hh��p5��%�u������%��N����.�I��\b�f3�
��>�w�
_��������H:z:;1f9mYmk��T�l�7�B��1%<�
;�A��7@���m&��a*w�\�����[����a�D����g����B��L���v{A��z��",�v6��S=0�r�bt"C�l=F9P<�$AL�D,�x]~�=���I�`����+nH�g�6N��Lb�"f3�
��>�w�[	��4��\�98E4ib2�I�^��F+�!v��sO��'#��S�����vgb����6��	b�q.�+�m.*�J����b��J[,]�$���t�����_����.��M(�E���m���1U���>�$��$R�z����J��C�d)c��2p��M�����1��,?�E%������~T"��kK�zorE�4;����\fL	���/�@�*!Q�HvC���`1���K�y>��D��8�gpe���'�����t�Qi+:��n����b��tLq�3���b���n�lB��.bp�(��&�#G�\�p�x�N<d��8Z�chG���������!QF�u���I�opS��$f��.*q_���7��>5�����d0��'51Vah���F�C��o���@�!e]A���������7D�1<Y�2|�m��Ia"��Qs��?={0�?*���)mE>���X��������a���~���ML7�~��D�������)U��3�e������!p(�lE����_��NJ�����T6���/�=Y������l��|����E�pR��g_���������/�����?��������/?��?�r�����~����_���_����/�����������q�����	��(��w����������������o�sp��O�����b��UVXy����W��2i�������`�4A'�)����`��PT1�X}�������a��N}�-��������!�r�P��3�P���n]0�N������q�0������i��v�@�H�Bc�o���Y"@���Q�����A�\�NE���e��!?I��<��_����U,\|�+���_m���PIm������5{�|QIh
!O�a�������1-��������KK��K��/�s�x����������u{�S��X��D-��;e\%���X��?�z�a]F�L�<�O���O~q0h��q0k�N����oG��?���~�C�o��E��%������������fZ�G���Q���?h[���7��?PK�������*PK
{2K18sept/pg_part_head/10_4.outUX���Y���Y���m�%7r&�]��~���A�w�g#f����%Yy��O�f{���n6w������S(�9u�,�+��HL7�E
�Hd>����/����?���~�����?�R�������������_^^~����_~cR�d�{e�R���_>�?��������/��������������^�|�������Z�d�'�����w�>���o�_�������~������~�������?`������?���������W����������?hm�w���_>|����?|��������Wi����>��?����_�����������>~�+~n�x����y��_�||���O��i������?��������������O�����������?��]��Lv��d��8�)������K��[��K\�������j��������M��|����^�|��������w?�X>�?|���g���?�n�l�����w�}�??�����~������|����/���R�*Cg��(����}am2�*�<�ik��y�yC��
�`w<�G���E;�>e��Fy�U�Dk�='lG4l�����7>�>������|�����#�lr��(���7\4V{��O��/�&��~�����������{�_�>����_���w?~`�w�����������������/����c
�xJ��t���T��������|��{m������\*��������
�y����e���w����y/����-@�7���l���&���@d���Y���f����)�}�a>Z�tJ��[cB�������'MN������������!�������X�(U�
��b�~=���������S]���O^e����t��A���hl�)�A>�|`W�\~Hu��z���6;������{�J��u����-|�����������E�d����.(2�l kMV!L��1>?��7���_L�46y�X'�|+�����c(�2	9>��.�!GM����L�1&��o�57��MB�^��� ����y������X�������r|���Q��)�����X^���|�MB}�������7�>�����O����[��^~�R�?�?�����Ky���R����GV�Y9������~���ouX����'��f��Jz>���c�L|����g5�f&���?��_^t_;����I�����?���k�L�GJ�l��p`n=lv�'�;�$����`�f���t�Q6���{x\oc��H{���9m4O��}��*��.��l���?1�fsp���rl��5�����~6�_6,e������r�@1�V���l3����wK�x���`2�����>����w�}����?��/������O_~��_�������������s���{)�?���i#��
���99���3O���/f#��$�����5j
��_~��������d������/?������^���l/������������eR�F����{���:%������1���~���X��V`�C�;�x�z����Lx�����y��������0���zD���H:Y����}N(05
L!�YV`��r�Q`�i���8>e,�z���XP��h0��`�i0	h0]���7�q\��_�f���0�pb���s0�9�0 �k�y?��L|�����&���!�Ee>����CL�=���_���|����{����t��uX��~���������z�P���g�!EkGk�������V@{�u����<~�s���{��|kN6��7 11Y���D9/{c��Q(����~z3�H#�l��2��Z�q�������>'��5�����S���q2��|��|'&r��=�Yvw�����	(����5z�I��@d-������������7"����fUV>�NXVV��~�;����Q�t���S�o��@���*��VYq���5y����|'\#b�����-N�0e�w�����(��RYs'+}��(�����s��xt~,5�g&�++�
�dZ��S�)����7������.��������2�Wi��$�Q\YC�G��)���F���*���5�����s��������e
����TV�o�����q�O>���3�)+�F�XY��5�`FL�W�rs��g�g�y���8���&�	G9>�i���I�pBYc������Z
�S��5b�XY������
N���������Fe�W�����e��F�+���4�?!�'������
����0�����<y����V~5�s<Q���@1�g�7����}���&s3:):o��PNhkj�V6�q�Vm7�����b����c5\[�]mM=mM��.<Z����<��P�=���y���#L9����x0�'M��BL���!)�����ab���)^CV�����X����f������w������<;z`��d�h��wU6�T6�l����f�;�7ae�}�[�W����y6Hr3���-�Yl->��<���{��Y��#��vYg�L��������'m}?\�Z�i�5^>�dt���@/���`600<��)�^Xp�)%��Q�k�=�-{���/�o���ed�H�|2v��%���$d��������<��+��gl�x�-�El�hY}��lI@*��@�*���3����(��5-&���:,�
\�_nm0�$�y	G����]L��a��&�\��b���V���'�N%7'/c0C��P��r*7A�h�v:%�������%x�sT��,���"�M����0������)�"�,���������),�Mf�
k��[�_����{ez�(#�2W��y���|fg���c�q>~u~�}0_��E���-�263��\2I"���P���Wi��98�|-�����s.�'��	�m�P��s$�7�A��KQy�/�4��l����
��wQ���2�(s!"�����/0�����H�%TT�����z9L9�I�4�_�W`$��5����	#br���Qy?�'��<E�`q�m�Q��|GYN_�����L���@�E����:���h�����2=���H�K1R�,((���2"����Fa�3�W_�������U$�U[K�����s�|�s�gf�pHy������|1��rv/���:��-`*G�>�r�M6�����"��r�9�eh(T��wQS���2�)s%j����"�P�	!��������O�io�k2F����RI�,w�`����\)��|�xN���X�y��g�yB������
���Ce�Q��:�1R��5��rT���^~�p����2= �R��T�bY�W>�GEA�|MZ�������|MM�K���g��6���K�|�|H1��p*��HAnL����A�s,0X��SG
���j��E��������#B���Jy-���3Z}�B�LZe�U�Jh�Iq��p�F�e�i�nN[/��u���	��z���
������#�^�1������@w���Qz�,��1���f���������kh�jS����������w�V���2X+s!�
�$3���i���x������<���|����wvQ?���o��eK��m��J;���-�@��m�r;�D��o��B�Q������,,��2���2�����+swez�+#��2W���o3�$E��������X�q�
����2�s|��0i�y6�s3�^�0�L���3�[���/�K�����o�&>��z\�^�Vj3�~m7uD�����bv��z���FC��.��z�+^���+���]v��O�=3�'h��n��#�m
yN���NY�h��g
?��_��b%��/��c@H|���ce~6UuB}��R�pQ����E|b�?L�����B�=�]��pW$���KqW����+��������������V
U�����bf��
Q�&v�p�������*�q/�������z%���������4*r��:"m��_l.7��3h���<�,�������"�]��b�yd��,�XB���Y�����7����o��2���Q)���@�Y������[��ZG,i)Tu�w���b=}��V"����y�������+�Wp�\~)�;���������Tx��t��{wE=�	���J��G�����H%����m���b��3`����&�Y#�k��W����b�D9����/�C�Q��!'�������'����B|)�`�����������@�����F'~�.��z�+�]���+�����N�"����JOL����o�/��Ur*�J��%BW�������.�EA���b>�]�\PT4��'k�<�����[/w_k���U_�����"���y�4���������"�]��r#CW�d������4�CP���b>_��[�1+�0Gf�2_@QF��MK���'Z
L��/_�M��~�UKI\[��
���p��3-H�
���=�8��������"�]���4��-�Es6`K�3|\H7"�e�����#.��� E�=��5�VK(8Z�l�������R��%A����we|V~�`��	�C����/�oD���fp��|wE=�	���J�i?�f�?Uu
�V�9[��(P�@hHJ��l8\�GH]"�����>�MQUti��	=��B9�3(���2�g3�'���]Y>Q7EGD��`��L(w���l���5�twE=�	���J�k���Z=i�j�#�;.��y't3_���&�
c�����@�0��_	�k�B�7T�sY����������2,N�A-�
�smM�������KV]1h��f��C�wqW��]�����]����A��)�BS�g`��F�/�~:�g����/#q�Ew�������k���cB����s���0S�� ���v�Fk�9r�O���{�2�
*
�B�:oF�&�]��������^	�2)
=G
��#[������	=���`�<s��@�(���|�n����f����]�f�6���Wp"9��4OI���l'�����&5f���������J�Qvl���oNwe���lve`W�J��qv$�F�dp.��3Q��niM����r-K+���y�
.p���!�&eh���gq7�5,�e�RS�D���f��,C�	�=h����&-�>j�F��	��$���qu�=�n�`���lve`W���C	7E�W|����\2����k�����2��-h�k�|�a�����~����SF�Z������Unn��u4�:�'��M��Y�����?}��|o��U\��������8����H�+G�o����I�Q�Z��h�VIl�%`We>���0G�K�H�/���+��"6vL+�����\,��h|��`�6i�mGH�-�g��>��c3�\��.5������#{7qd{�#+�8���[=����ZS|2`��v(��!T����|Q�2�v���R��������&�U�]������-���|I�9���E5i��2��$�w�g�Z9 LcF�'ZO_6����n���GV qd/M�4������)==�ggo�#;�n�P={�a�����
H�)�D�$��4`�?XY$*���i�KTa�7�+���!���]�`�4J����A�t�p��h��������%��@��^�8b90����?o�RE�q�B�9�T��|��1@��#��S�B���t6���]� ��H���-���%����G]�6q������h�lW��,4�!��]�R t�G�6�����%��@��]�(%���/>r�T����7�!��4�-R�b��4���MCzx7�I�_�[��7�5T�D@�����Qp�����������GTH���v>�.l�F�v������s�G;��n���GN q�.MQ��'��Y�^����|��0���|�"V���|�"y_��*�ZU*���C�=�����o�����������N�yv���)��
,JQ�mR�v�
����m��ix�����^��	������)��\�O6��t|�]��R��/���\�ek��b���W�&���S,[�}��l����D����%�
�(Jk���N�o[���-�g�\�b���-V��M�	k�p
�����z}���w����z}>6F�}�'�C���M&�$��e>���
~	]QdWu�R���M��_���f>�P�?����>�P��^?Fv��z�E��n��h�&UD���I
w�����^����wW����1�� >a�E�-�%�B�
�54m�x>_�Tcf7z>~3��l9��aK�m�R����M�#�G�~����#�OS[�P��^T�q�����z��u�����
h2���
w�^�����@����^��D]�O���e�3�~��,����(-kv.Q#S�w 2�����<kp0���9#�>^}���XB�� ���"�{�����
	���o���&�Ai����}���+�]9����Z?��A��[�d� ���5���n�k��e���t7�(R�rK�}1_�\9�B>���:�7i_����E��LL�C�Z>ruP���2������?����`�Y��v8����r=��]�K���������1���[x��^P�2�T�����2�k�[�+]`
�h��&o������Vpe�R�<������K
��-���N�f2g��=�������{t�z�+'�r���L	�*��TC��b�T��!y�3fTD+��\� yv%�Z��yZ�O���n0�X��2HK�(��tO��{��,�J�����:�yF���_���~���r=��]�K���<��{�d��d��-�5��������g�|�O����I��-���\��_N�H6T�;�'o�\�����o&�-�z\}����8�����}Q�@	k�a���{�����+�]y���te�H�$>�E�#�dZ��4�����nS��z��=2��"�o�kN_���r_���k�(T�`k�7�O-yt�k������ 3k�����FY�����'��7�h���]�������MR��bwRY�
3�wK>���
�+�&o�����d�hH��]��p�<_Tn������
�
�����Rs���g��N��A��D*�y#�lW}R���?;�+��e!�����.���PW^u��D]i>�:��d�S�nS~�wo��P�i>lo�f�����`��{M�2��i�Q�yB �[�seA�
l�d�Q���A���%vm���G���9$6��	�N��]��������_��2&�=�%jp��J��VtiL&���y���b���K���%�g�W���c����O3?���x���%����&t��{nPW��%���W��U���������,b�vt���E]��������+b�8P}��B)�6�(��m���o��!�'T�/�9k�������[R���ZOL�u�oB���p%���,-�	�mPW:�y���W��ord���[�
%+�����.)�.���PW^u��D]��9�f�9e�:!
:�I}))�kj(y�!vf�l��wCJ��v�K���F�pO� ��\�/����:��v�9��-��m3;�l��D�)P*xt���^��l�����+��]y���v�}����^*�z_@��n�)��,����D�����wY[z�t3_S��K�����)m���i>?����l�.=�?��
�J���������i�������r�!������.���`W^v�/��2#O_�d��2�gX��7��D�~�!*�@�g=YR���8��J�����y��!�����W;�F�\ZB�F?{9��-�
����K��~����u��VH]k�`x�������+��]y���v���"�g�XO_���S�������������f���I�ex?_Z�{V*Z����]�zV����u�m�]�vuKUgl���l`�~1�Jz%�	vt�v�{�+/���6Iqid�P��R���_{�h�"��
]kJ��-��c���qT��rk���Yv�!-�����F�������|�����4[I�V�$B�w�g�t�Ssb��!���
waW��
��p!�
���\W�lD�m&ER�)��2N'��g��7���W���:��~���C�)1�~���f��y�i_.l���g��
xEi�m`W����	MR���g�������b�a���ywaW��
��p!�J�F��Es�t
@1.�vqVI�8�Z[4t�Beq�md������WZ�9�.�E�4
Y7�+�giQ��W~N��������I
�
5�dV_�w������\��.�3d����y�pvz�� �
��]e=�=wD��-�)����$jx>�v����)�����s��PV���r���F������Ow���
�����;���+8�y��&t���}-Z����o��\�����]�U�����*\���ad��|��K��g����(��|��������ULC@����l�B������L��ufo�k��e>]��$��g6�}B}[���q��k��s����sh�@]mx���=�����C]�U�u��@��|2�Q�'#gv��t~���f���
���l	����mo$
~�|1���rs6���8}�|�jd7���+� �%��{��������/�^�Y��b%]X@c�Hk��F�6�]�U�����*\	�"=4�[>X�#_�G���%X�s��!tJ���JP]���1"\~�b��f'�7*�Yz���V���^tZ�_��_�dM�b��C����\V`�_1����.�*�@WAt.��b�82����D��H�$�9-A����*���|~��O�_�Q_��s������`�������|��_���R)8������ +b^��=�� �����W�s^����Bt@W�J��Ni �F�d6���i�QE����W ��M�K�([;��u�;���c�Vzt(r����e��'�v���)f���t��C$���3������X��*O���Mwn����]�U�����*\	�2nd�������'s�iA�o���v��3U����~!Mg_Z����^3�7@</����h��s��U��]��,����0W6�
���VG{����y�~�K�o5�y��]�U����*\
��#�R>z����%'H��B���=�Z��/����Ta��UZDi+p�
��.��Xi�p�7�Y���u�� ���t����F��ac���:�M���wV�����x)�U��5�{*��0)]J���C��K�kk����1��������9>���cE}z� [�&D\aM��NoA��b��|-�
�Z��	�1U�.�*�0UQS/�hF���5����(;�����T�e2�F�?�P�����s���+#x2\ay2��\�,�.����
K�SyAb�v���Q�U�k�r+����y�xE{(�(�����������8,�K�n[��9�l��f���M@;
���MX��$N�,�/��aq����b���B�V����Zz������������(�Ak����)���A�������x7{��(������\{s#�W�	t��	�G���}��<�<�Sf����%P����W4@K���-P�;D�1C�<Y�,�������5Z����6����J������l0����.P*��RQ(�J=�
���6�LA��,�!4���4�������v��Q[}-���4�`��nt3Y����e�p��7{zO��?�WRs-2R�
��MF����";"���.4*��QQ/�F��4T_���
N�9��\I�`s`W��es�>�A�|
���3�B�!x�}-��=*��p�A<!C��	��~9_�"w/��?�^�"N�
���b�b�P�R,�-��kQ����*�<�J��W@�-��n���d���k����h�����`�L@69�%)h/N�c����k����.��g ?��W$MJ����]�S�����)^	~�D�����ZT�}�t6CJ`}���q6lB@|��(�P����(����������_��q�_y��P��(,��k�X�lR7�k��]��|X����]��]�S�����)^�1���lD���3�A�B(r����	z����ch�4��fQ�9@�@B_sI����G?�rB_[�SB��U_��n���4!-rT)�l��$��8��)
`������T�b��;K�6�q�l������5��U6�^`���'�2�D��I��o|F�q������\L_S�qB;����������WX�HY��Q�<���t��z�$�qJ�b�
L���G��S.s�����
d�`���I����Cs>9N�D�u�C�5����d^i�/���|���B�"����������zm��������.�)� NI��.���hpB�\�%'�d��D��)��A��-�sZ&@�T�G��l���
���&m�	�W���\�M��6a8��-�)DD�g}��W�����~FO��V����
{��z�$qJ�B���zS;:y����H��M3�tP�.�r���D��*j�&�����}��A��,e���}�9���q�9i�3�!��W��Mr��:��9��)	����0��*�$q�^q�:���@�,o
e2�F,���Y������4y������9�����XL�}��VEs��d��'�I-�)���Z\���,��������J����]�S�����)]�s�Cb��ZeJ�X���J�6W��S���EaQ� �@S����������rX^G�p�9��t�M��"�
��BJ���	����8��t�����%��Z�t��z@�$tJ�6�v`��p�!�����qV��#�d��k���������7GG�����4-�n�i���:�:!Z�V ���N���:O�Mi�N;���R���N�R��Y������@/�Jg�t�h��uW6k�#%/���'�e2O�
�'3��~�7�5q�j�"
����y�����@'���k���t�|�{�vp,T|�����.�)��NI��.m���@`��m1��ymJC6C��F_�r9�+����?��(,��B��������HY��y5��c�����	�m�N|gRz	9��#�K���K������+�:��)	���@��:��H���<����%��'p��/[���
Ai���r77M.��<�!L4����>�_�t�FfG\_���2��N��|���Yr�8�����]�S����)]�=/t������-a���aHg�}Zg�.�M�F�C�������8'e/���K�����-��.;wjO���.���E:y
��r�fPmt6Wd��MP7�j�h�8�E:��) �������������Z�*�9{	d"���0���Y/�!Z-P���V��()�A3�g��o&��:��:��a�ng��,��	�m�N>�V���m�#�obr�aXd�z���q��u�=�S�:�+�N���#�?��642]�����)�l�	������f�5<����ou����!��
�)�V���Mzv���l8��x'���y��\����C��>*�Eg��O�w�N�w�p�|%�I��y�^��F	����s|X��y����9f
�L>�@^o�;3�uz���N� �V�o{�3/�}=
@9��-�)�u �qb����t�T���"����;��H��C:e�S��D:<c��>�|����-t���A:QF�>D����?|�������J�<��X5@'*�7!Y4r]`�)��b�N�����GF ���	Th�������G�a�]�S����)_
t�#�����jc���N�)$P���le�`3�c�9x�����AT����L�
.��	4�,�zXr������9%�&j����Oc�S�U��Y_�:n��z��{8�,�s�W����z�^(�d?5����k�y�?���'�]���m&�,3��d�?c�.'e���T��?�B~3�\�����n8vB_[�S��������NI#6��Dk�i�f��w�N�t�@�|%�	0��A�TJ���[*�'�����s�j�.`��Hs��{�}�Rkc�h�U�>M[_/m���g+�O(mv�h����H�XNv�:AoX��h��|��{`�,v�W�����E���<�jRYm������9,�#p:	\b�
)�����3���!��-�h�[0���W��.�X'�jN��"�>�?�?��>Ev=c5
���X���:e�S���;vC��������;R"�N;����NQo����mb����^7�JD�����<�2���|n�W����'�
v�,�S�Ze]�g��;��N��w�N�v�`�|%�I{?���48-x�����B�(l.�*�`�s��?�,W��d��f���:�i-��e���������ky�������Z�N�W��N�Z12�V�t��Y�{[��He�����K����"��Q����`�Do�{u8�i)�B��.A�7��_zB�:�[�����@8 �}D����� 6�~���w����m��v��O,�}��A�����K{�E=��
��K�&�c�L�N@ky�W:3"�3gurI�5���s�����|������iP��u�������
�=@<i�ROt���=X�_^y����G_fY��J��<�������������w�pqK���$����@�=�7(�C��\&3�,���j	�	L�8Y�X.��?���������i�Qxp���b��%�71z{����������J+�|������������-N���7����gv�nB���Z;�!�D��DiUpC�V����n��q&��	)y�=@?�K��l��:��,��@S��_l]����W�����p��e���qe~�t���'��t�i��|��	��v3���! ���M�Y�� p\J�T��v�B�n�s�K;{��|@�������O�CY\B��A�l��
YdA�+mU�f��^J��7C���
OQ��Y����bQu24=\��$��f���&ap��i���J ��Zs����gsYg��B��������)Pz5��g�!���E�����P�o�k��X(Mv,
j9��w�s�XL��x���������V#6��d��4gNs�>���9�eD���3�D�u"�(m����^$��{�������=��;@8j�"��_+�������D��d����.(�d�����=jm���g6��mt����f6l!�~o)�wE�d��/\�s,�4]W��'vv������������Ssv@^���)�]8T��c�D�_za����m�����-�?�D	zE�������za�Rj���"]
����e�BBkc�s�K�
�����w�Q��D����w��Kk$��-j���X�g����D��>VZPT���(-���@��n*�|"�g������&�S��Ii�(t���f��F">B���:i��p��&���������h������Y���r�8�y��L&/��n��F�O��o��n�I���#'���u���?�����>�����������n�����_���w�����?�����������?������F
��Z�iQ��~������>��e�s/���������zID���������|�?q//������^����������c������I�R`�L��_g~�����y%�q3_�KU�<8��$�2�_+ d�o_K1����2�<1�������g���'�i Xw^9����s�g��=��(g}�]�h}�'��O��3���{y
>}7sL�����a��],]��7�����6���MS,�vit�;g������E5�\f-�(qA��������>�r���6�~�I��E6
�Y���5Y�;E��H�#���A&i4D������H�:J���mu��WOo������X��N�+���J�=b���}������s;Rin�3|�cm7��Zx
�����u�N
��i�c������?��qV{�z���TBCBf�P��,�H��(Sa�yl������C);��d��u��':�K'�O��(X:%��c���*u��r�Ha�V��~���Q����c+����������~x����]�f5��D�K#����{[�h2n�fy���}����?M���?�]������W�T_�����?���/Q'�k���_~����^~�����������������_���?��������u��������~��|�����,��w�}z�'��~�W���	�/���Y��]*�������7�>��Jky�����	�/.����E���Bk( �����_���}����a��E������h
l���Y!*�{\�@������/{P�c�%�Z>������b���n��C\$���=�\��.��H����@B�G=������e� \��F!��"G�����B�b�G&��Gq�>�A�]�A�y�A��b{��Q�O�<�/�!Zl���"�Ad��APLi�`_F�B�'��e��`_Z4���5x��Rj  �YbWx��NsY���{���q��t<�7m���[0g��q���)���C��3
.�b��<9R7j\$�^���R�[R��R<�A�5aK=y���&������0w�5�J������F������`���y[a_��8��&Q��}��:c�1@�V�L������5%%5����T6 ->��Sz�}��W�U0r�~�_��<;��aO���������lN}:���V��t�
8���pE9��e�c_x\#DhPT�������2�'��~�(�
�:K��+�9"��/�#�v�2��y[�1��,&���b�;t�u�;o!�����~�4�}:��p`����k�gVwgo�& ���*�����D��V.�mZDj��.�B��>P�_�?p�x��[���@��KI�����O�?x��_�c5�h��[��[l������d���"�_#��,������H_#��BS�I	�}�;X���(�(<YD�.w��|V*�\�"����8z���:���7�����Cu�^A
�+]9��"@��0:��i�������-���+4����]��On��.����Y^����`�Q��@�����$�iP�������"���A 0
�b���W�ip�{���q^�(��:����#��v	�����`1h3,�[y��?��F�4�B8t�u�B8o(��1�@���O��gM0J�s�v�{E�Y�V���UE��ol4%WM��IO�X��dPW���?�,����A��i����pH]�����O�Yx�6�>��^V�����~����q*���:�F������ {�J4�<<��N�x=�,����G�&����$B�b�2���������x�����j�12a�y�D
��WP�k��"Y�����|�7)JO�D�)ez�h��KM�D��p��W��-�Yg!��	v�2n�Qd����g�����Yx�6�>=�>����[0���r#D���A�1
�������u%���7B4�~I�V!n���;k��. ���2
�/t��+�6~��
�Q�":�:g!�7���c��Sl����?e� �����Yg�������Bt��������,���'������/-�,*����B���13�)Ekxg���{*�C���t��7m��SV^V����Yl����,8(+���(.��+a��Q_t4�V"9��x�Y�kd����K�d�k�s�IS�_��E`�� �MH0���-�����e���������\'#"���c%"N����`���
��`��#4�Z�K��6��P~���Ob4qM� ��I���rr�g���6������5��]INL#AST�����B}:�[6�y����77r4U���Y.�c��u�-���Z��>���]0�$�x��0z-�`Lcr)��Y����B����vZ+��0���(z�C�N������+s("�e��2���Ye�9�/���Lq��w)�K��?��B]����OJ�&��P($�p���C����t�o,:��6��\[�u$h��s�P�m�P�����F� �#�}s#G�RBc<����Er4��U#�<�H�F���z������X�!�!�������2���j#��"��?��NIK�^�CfHs3�9�g^�qRC[��E�f�*yvZ��!*��z��
���-���7G�&�-~���PX�vH@�!'�Y��a�d�k]��)j#|����G���>��M[���"����9��������P��0j.���P`vdI��Er4
����<���Q�b��$T,�"o��tS:j��)B�.��F�X�E���"�yb8�z���S$[������V��j6�E���t��u|����XL�6P�Y�0.<�K��h*�5��
�W9	�:{�H����j�Z��X�.X�:4E��8o�*����X���%^����77r4:\>p^>u�H�F�1{�OL���h�����C��FZ�J����j�*��P����l���MM����!{���=����3��>�����)t�,�[�6�o��`��:qE�d"�H[q	�f�H���5`X��R�
P���?����g�=��� =0E�C�0��O#AST��C���B}z��J�i$�on�htX'�"��QF��h2��!7��.���g�B�PL#	9u(����W���e�v�I�Vw
��B��z�s.H;�l��:�Is�o����_$��(�@������p��%
��|8)��K�$h�5e
2���5�"&As�)k^-��2��u(�IK��)j�Y�*��44D���o�&��������i �gn��S�br��N����S��%p�Z����S�br=��Y�G����R��������������c�@s�i
�zy7����\�Ji�3����H��K�2M���HtyX�6�E9������%�K$/.A����DK[0�<��A[��5z$Y�H��Y7bOQ�{J82E-�_�(&�:4E��8o�*����}hZbwh�/a������0�*9M�*8=���w����g�6{�<���Q�"����\��Sf�J�
=r	4��\$�L�O���!q������'�3����8,AI��@0���F�nn��v�m�X�7�[9�%I\�����������!&@s��1J*>�70����%�>��o��W+��$�7J�LA#T��'���A}:z[��.�=s#�^y���g��3_���c�]wlY)�j\�����"�z"�d����6�
 �^f#�JS�����
1I��t��::Ks����*��FI*�@���@�]y�A>��E8������!)��F�$h2�R�i�Jk(1	��K�l�
�Tt�_���r[&��%�)j��.�'��MQ}:����
��i//l�yZ��2���77r4:��k�B]$G���h�$m]$G��e��QB�G��p����8��s0K�����H�U�<��b/uw()/�u���<��y=tw�E����&P*#yvE��~�
J��&�$��[�FL��
�J��e��n��$8BJ8f4Zam|8^���D��%�V��5a	���b���t��7m��	K`]A.?
����{���S�U�����+�XT�N�����%�����R<�E�,���^,|Y���R�M�z�
�XB��\dd�U��v"I0�u$��<��y=tw��
�Dd�/tM@n�5w��.�>z�T���_��{�C^zQ�t���`oz�*]5����`�p���`�����X���7���A��:4B��xo���Xo����u���)�-�y���ip����X�1�ip����e	��"R<�A�J{��i�T������Fr}"���j��Uy}�f��C�K�����������;�Z:Y�{D!xGE���v�� ����q�jD
�4�@��0�Cm.R�=��jrG�����A
�e����Z�&Qh�q!�A#T�c�m���t��y�K�B���)���a������H��`�\���9Y)���h�i !���Y�k�ej����``���D�B�|:�d�&���4�%�\�u4�t���^��q"�d=���P�`���3�f�{l{R(��tS�&%@�ADE�qHh$�����B�gJ��$�g=�o��
��Z��� _Yy����O��x���>��!����������{���c���ER4Q���4�H�&�(�E� h�%�8\�;{c�u��J�B��J��9� ���CbK�����S�����;v!�h}���wp��3a�c��ap%9���q6a�%h��>H`��k��%%A���`m�
Ay�r�u#bK�t����)j�vU�@����F�i�P����D�Pr{�F���l��R��/����C��4�H��6���i !��n���2����e�fE�%�M������`��A���(�8�����,�<q����\F�D�1@��b���14q�&a���cQ�h|�A\����`[�X���-��(@ZY�*����%8�F4t��[�S��Oe�
��
Q}:n���	��E���������)�:�\���ER4�S�8+��ER4�<9X<�����[Kf*����9�R��",� �{b�(�m�
-�ErX�u�t���^_]�|J���tKe�����v��gx� �`	@�KV\��|Fu3�@RVkq�T*j;C��3CN��D�_��r`������44B��xo�����h/�s]`Q����Dg�d5
.�b��<y��7
.�b��X�N	)� V�J���dI��m1n�b�Q�RQ��J77q���������Du�z����I�O��\i���M�l�.�O_W�������8E�sd1���4�;w���u������D+���cc�����,W��t<�7m��������:��37R��209����ER��2��f��*�����/���V�wa%r���l��_%\��s~S_��qHXI�V�y�:z=u��N��*��]�����nY"���������QJ���79W�X�#��� ���K�?���s�g=�=a%$\�P�A�,J]����O��x���>�F�8���=s#E��'��QI�� �v��a\$�����(��:���Qb%���������7�%~�u��K�X�3�/t*m�)��vV�u��t���^5]��S�9K����h��!L�
�����L��e���Q����S�~��n����"���mMQ�����"�u h���q#��M�O���u-�i �gn�h���C��ER�/X�RbZI��
���7U
HH����T���fDR��S�{�6�A����*l���[��!M%]GSI�9���p��>���+��~I���a�J��7�X�Wv-n�m����s�l`�b4^�O�~^	��r�%8�E�i*
�[�X���2(_z����O��x�&�>=��8��u �gn�h�L3E������������t�
���)]��$�x����T"J:�Gk���^:���*{��py��3i*�:�J:OKG����8�a��_
 ��5���x������"�)�l�S.&qur��#�	p@S������8��_��hh*MP+��
����n�i h���� ��=�OR�JQF���) �e�N���8�T:��I�Og��u !���]i*�}��b���Je����dW^Z�;v!�!O���������������0$��P��%n��X��/���'�	:�PeJI�"X*:cni2�$h�e����#+'�I7���*M�
�LQ[���G)t+ACT�c7�m������T	u �gn�hj;y��RUYI�\��LRH����x�����"_�Q7�����.��"rFbc�J;�g�*�{�m/�����	Y�4��U���*�yr:�z�����d�T���_���i6�E�i�(_�+����������v����K��&)�2n��c�����w#x�VJ��)jQN�R�_���>7�M�����fcaW��=s#E�4�'TD\$ES��H����W�E�4%/��nm�FQ)��9�xj��1M������m�q/����^�Wi�s����M�E���d'�l�|�m����,�N�-�����8��DX`��hYAy]���C��g=���2E���j<�@������O��x���>�-U�.0�4�����|���4�H������k!jAn���1&�i��&	)� �O��C)�te���R�~ �g
��*�,(_q�]��
O{]��=_�e_OE�/�H�v8�M0�nj���"��@��t���b4�4������k@�g+.A��0�+����P��������i����6�aU*�u h���q#��M�O/a�OM�=s#E���*f[����s#E�������Iq@�C��"d	)u#�*Ok�%p��7;k�*:�]7��������8-#�9�Vy���<���.�z*�:TS�n_��#�:���`���B����l��&�(%A�F��JV���H�v�JG���F��$8�F�=�2��S�w#J�nF�k	���t�7m�������8�����G����Yt�\�LL--[.]$G�������������X�=�`�T��fC�6�5����`"������fIc�v(�=�u���|q�}=�]�)Y�e�Q�����I�"�u��:�]E���Q�`o���d7���`o���B����I�kp���|jv��)j
����<4E��8o�*����q�4-��G��
���7���Er4����&9�v���3V��y=��>?�P���la�)�$�����\�����mD��y��������a�������/������8�AZ*ax_�n��-�x�.������
���,.�>B�����E���2�����iBp�%�h��1Z���DS�	4K<�BM��%���e��t<�7m���[�k[������b�����I��,,�^����Z���b��e��R��zn-�DA��d���2��
����F�"Z�G��zSc���+?�u���|��{=U^��m�0 5�]x��M���]d=P���#^\��X���EK
q.������A�vANzN�I,��X`d[+��GD����j��{o�����������%��F�}7
L�g)�UR�����y�"_$��<;]OR-"�����3����kXRi-F��O�v� �'�(��Y&k��qX�����t�+����������~�8D����a�K�RB���t�c�Q����	!)�q��$h����i�
Q�
�p	��M�'_�7���):@IduK���MQ}:����
��S�]`�������EI��t�G�q�h\����hQ<;�����:���`��V��i����]���d�����H�U�c�n��@�u��|��{=_���,�\���J{JI����4����o
��,.Ac�X�r���M\TJ�v9�Oy��3\��ES�)�S�:���r����)�O��x�V�>]N�����d���m`������hy�xv�g9��-������*�e=���:v�Px�
Avm��h1p]^*P����>�[��$�=t(�u�=o:��1���,��0����*���v�_�
�p��:��4R4f�N}6��q����5@���$:������%8�P���
5�����������
Q}:����	��u��`��
��)J�<�I�
hZV��[�V6�*.��!������y !��n�[y��F��R���������k)��>Ey]�l����;t#�un�;o2��1�nEa7�)�y��Xq��b�/�n���� �>8�Xa7��#��8�%h�|�7�����/�Y7����ILQ�F����@����F�i�P�nTB���:��57r4��������")�t*@G5
.����KE��I8����~���yj�i�QkK�D�A�'�v���J��FZ�������������5f�8��0A�U���	{��[��t�p������E]�����[`��G����9V^�B�&�q��-�YW��\	k��S�F$�-Wa�.ACT��+��mB}z	��l��F�&"������/���H��5"QI�D$xr
�[��8)u#V�K�Dq#\tj�&�]�D��Z��n����MH�}�;��t��\���v��P�u�����I-!}��?�.��?�2��A�r�B6)�L�I
���		���eR��b���9�z{�K�W���l�P�A����a��t<�7m��+����8�������1LRDA�q#ES~�{�����ER45X���L	)� V�K��Q�9���Y���{�6X��
�6k��$-�<d�t�1\��\v��p�u.E���.����/���v&�����-���M�
K�"@R:Gq	^���9U9�z
��#���
����j�24B��xo���G a���r{�F��@��,]$EC ����aup��E�(���4���Q"�x��d��*�w�KA����"{��o�&/�ARZ��(-�y�:�z��:1�n]|j:������t�]d=��U(�y����6%&@�H���07("1~~���]�F%#�	�z{:Kx����B�A�7c�:4B��xo�����e]����{�F�����;c\$�^���Mk!��7R�����qHH���!��k�u���%���t	$@[��[�l7j���,��:Kw������_�@��CQO$����zO�� ;-:�Eq	0�`lY

y	��l���DA��FN��n����X�L��_{�q8�MQ}:����
��B�"_������_C +��<�H�CX�������h�M�R��#	9t(���2�%	�K(l_��"���Cam�!B'"���qN�o�?$����Z��$v����g5�,���t�q��P��~�u(J��B���Wn-��`o�,_�ui������K�7@�$�)��������t(�����(�������h	���;o�*����eaQ#1�����{���EA
��c����{}.��3�HB�G���i��O:�m6�OYw#|�����p�q��q�&�����:�K����2���f���9�r;� Op�E8B	\�'��NG;V8B�J�J�;���&����`
������2?Z����3e�����6B�-h�qM�#AST��C���B}���?���D(0��Np���q�-gk-l]$���y=(�����������Kk�rq
)�>�k�u�Gqp�n��8w().�u��<��=dv������i����3dn�9�.�'4���x������'4:�R.���S\�}��6j�2"�7M�p���\"7�|��!jp������:4C��8o�"��WsE�H@M�=s#ESx�08����")��	�c�`��")�j����e-�@B�G��������P��1����>v��kcT�B����V^���v"i-�u���<��=v���)`!E}��9�/�$,�)�b=����VS��X��nzII��Y��m�F>��_��hh-3mv7��q#�}ra�ACT����mB}z�i^�R�Qr{�F�;���4I�ER4j�13o6Ar#E�������#��+�%J��	M.�qY�(!Q����aJ��8��l��y��]����_�j��3����`�)e��A���b�
��u��p�8�h�+�	��?����pl�()�7���h���
��p�z{FK��DGV�����T�F�>�M���t:�&�����1�=s#��h���R�)�@1��6[�G�����l8�@B�G=����Bf��"�Sk��_��E`�%��7����������_�f��3����\w\�Ey�x�h�^��/AY���p �H` �GAj��	D�������5hX\���mA<Z��n������nLQ�`�H�{r��t��7m��c����y�������������I�p������.��a����RM	)u#����Vk����P��}.*pZ=1;�)�q���C6K��?�^�_{�1&"3I��t�g�OVla&	��~�nI�h����N*�#�f�`L��m����� �l���
��
�L�o=�.����O��x���>�":�R]`���)�("]���w��
��X�7������'���,���Q"n<�8�Y�,m��i|�������Z�Al�"w�N�.�!�������������������",�#����n6��F�,0<���6���m������%�|�'q	�]^�mp��������4,��L�����Up#�@����F�i�P�$�����*�=s#E���
�J�e��������0����RV���[=IaE,��n�Ji��W��n�G;�%<����;M*	��W��Ub�DrZ��8-�y;�z8��!^�:������Lt�_d��"�����C�%���P�R9W�
���{7����_�����g��=��a1����5��6���J�y$h���q$��U�O�����k�{���������v����S_Q�YW������h�`0;B4�HB�G�������N-�/�� �FgR�=�,�y+Wi#�p�.�u��<��=\v�	M����,��%f>2��~�����	��+���-���� �x+.A����h�-9�,'�Y�b�p�Q �L��W!�y$h���q(��U�O�?���3�%.#�}s#G������Gx�M[�l��e�h�#\-��Gr<�P��������G$��kh{
��y#��^�d���pHp�#��������;�JxY�D�#�P5�dU����5��I��,��-l�o"#X�h_�=��8�9lQ:G���G���aOni�:��P��agi�Y�F�>��������-�+���<��37R����CU
'�:7R4UW����.�b��A���X/�E� VJK�Q�Z����[�j.��Q���4��F�����%���2\�h�s����]w�A�(wD�`�F�D4����`���-��|J�iI\����ZE_����6q���t��I�8�A�}�����X�l�qyV�F�>�M������
���������d��VUep�
�K0K���[FbOR������X9,�P��D"�)�'��q��W���R�FI�C�p�e8�X^c��a�I�5��K}�FV�y����<�w�m�%$�����M�����M����"�7J'A	��{K�.�jU6����B��B?U���>7�M����j�<���o(����)�;O�V���")��+_�4�H��6�U.�P�@B�G����@!\��@�y��|�
|b)�2J*oh}��.��$��:�p��.���Ch��I6AS*�����6�E<��� `
7%#b4U��E�'�;��w1�TFP���Q�z9�z
�%���+��2��}��t<�7m��Ke����}���)�TO��$E>����2�M�f�T�)�TPU4I�	'��Qb%���ubIe��4NJ�����5AE�F���E/��8d��1X���u��0�W}at%��t�/��n����"�(�#��q�c���ku����%hX]�1|J��=Z��n����z����]I�6
�G���>G�M[��t���*������������������H��H���<�H��N�@�z�X�G�������p(�����������Z!;�Fi]���s�C:�p�e8O]^u�q�gMl���-���%A�E6$Jw��_��
b4!QS���8��hz��
�"�A���	p��h�,�][�o�P�����OMA#T��'���A}z���i�7�+����
6_���Yp��H�`#x��&)��H��k��4���Qb��$B�i�F$�	��>�2�k���YBl��� �,�uT��<u]x=�u��-�A`�P9��z��\��pR�pq�^�����+Y����9��� ��4�����9^��DCe��]��BmR��C]��e h���� ��=�O/��~o�u�u��)�����>3	��)���X�7[�H�&����:���Qb�����Pb���pbL�����M*�##�f����xR���2\Ge�����C[wLA�=5��
4hl�[?���"��(��Z�[6`s)	��P�.L�����9�k����X�X9�KZE�+�u#�T���7LQ��@���mf	���t�7m���^,���o`M�����c��2�.���d�9�Ur4�����]��wy��XI-��RK*��N4���0��&�	9�79�����CqHj�#����������k��$�SY(�i���_�
����-�F�7�^)	3l�JI> ���%hP����V3Jp����Z���|c�Z����)�i$h���q(��U�OO{��w�<��77r4:��C���"9M.-.[.^$G��<{uI���C��ZZb��%��iu(r�p����aH���$�x��!�e���2�������Cj)J!���-�r��ov8�/��KH�K|��7��R4`-��b�e0�Gq	��K*e�46��oq�%8�P�I-Ak���)ja�:*]X����)�O��x�V�>����2���Hn�����p��h1�.���d���,GN�qT�J�dI���CWRK]
�Aj�X/�DL`/'�
�6=9I������d�x�j�c���Y���a���Z���:�����,X{�_d
�X]�Qi�u�`o� ����)+�i="%��A��	,/�Ip���{fK���)j
����M#AST�c��m[��t���4���Hn�����a|���i]$�^��F����G[�
���^�k����:+��)��Iam�
�n�B������.\�7��#�P�\��H.�yR��zH�����e������&����d @���/l��U"�@�������n��n�����RG:���b�jV���
��B�>����sM���������Cm����#�������4}~7��Rz�~�A}
�"�!�K(�B�5l��W��Pp�Z��3AW�cB(��+���^l���KZ�	Gw�iu����6�F�8z�9Zx���9:���s�����$��YB�H^��l�!D�O)�y�
F�3K	��_%K�U���eZ'y����������>^�l����.����������y��I��;�XYBacP�fq�L��u�B���,w$���>����*yiChr(W�
��z�����+�1!o�x�N���\���	�}���ptg��U����N2�����f�pt��>��x.Q�����t��Xi�G/���LC�dz+��� ����X��_�u�����]z��IY�Kh�G�9�,���k��7��]�'�;���\[�0p+xK:�0�����23D&����������Wn���B������g;��!���_B�\m��	u���v�M>'��[���X��
Ei��7!K�x�B���@'JS�&�Q(����x�j����qMO�$��_�u�����]z���Z�pf��
�I\��{Y�[�
f���q����u�.��s�������"`�� ��4��~	)��+��~O��f�����H��W�1M�N���6����>�:)_���A�=���Q�a7��E8��j�����8��>���"4�A���;�Zu�<h�7#�(��H�?.2p�+M(���i��e�/w�^G�n�����&�2��f�k��^��&��
����m.��tn}��j�6�B:�}��S���� �������Z���A�$p�D�	�"B��^��4kmt*������p�Y�F;�&{CG�5Q�W�>[����(����$p<K(1L��r�����'��2��� ���v������ud+-<��b�i�f�/~�^G�n�'��f��������Q�=�E��x��d��I��p�!-��x�#$��^�V�"y%#�.���a:��Q��x�^N;��m"��xL��[�39m�W��
v�F{B��i���n(�]������s�P��o��6-w��#�_� �s�Ta<�8�����M6���b�s%k��3};3��fZ��������Q��8�h����="������	<�E�A@dh%#`�a0�z`4�0h.��)�����eLW�V��o���"'bE���&�N���A��?�1>��W$��m"g3'f�����d
��@���&�P���Z�6�@�,�8�/��*����u ����1��%�E�P�ks�s��Z�cC���N�2���K�#s7�iDQQ�f�����K��\d����6�8'���(t���������;P���1�6��ObW�Kc��z���;���fi�:!�������c��m"g3']?>�2s�d��	���s�,B�u��-(��E��<� �K�#jK/��=b�13�����7
��!�a��b�.�:��t_�.�����Ad�;kT��c������2��"�J"q���cR[zi��yX1�����G��'���ObW�K��zx��uw��B�}�:!������"[���\'r6sBq������_��zA6f�U=O���A`q�m"��I�}Kt�H,G��2M	���QAT�F��4�FY+H�1�P�2���������(��)���4��*.n��>���'VA�6&(��t�X����X�����t�XDJ���N�V�������oiB������"w:Hr�&�����F��O�1Kb�}��������������x�Q�E(�_P<K#UK���X�3�VG�7)�i���H���-��l�<T���T-�}���:�uc*/M#
[~�>�z��5��%�='��3)�pFA3�U��z�~��\TQ� ��.���ZZ�"��+�h.Oq�M�	�xk��cB#���}�����P\i���6Y��J#��������+�H�"��'(����%H������K��(�O�mh_�s��>�	M��s�4b�e��iY���u�uT�&��)d/ZSs�C9��"������dz�;q�8vQ���Om��k������t�##��g�=�K#�Z�hHR��
�w���}W�	�""��^���#@�'�����\�p�9��?�rn�A�E8F:r��
���?���n�2�YBqhYzl
kY�K����vv�4VI�zQ[ndJ����C1��N�2������]7&N��F ���N���"����P�HF+L(�V��$3��dq���)kjGR���F�]B��Y�#}���nD�/5�3AW�cB(��+���F`c�c�����p���E-��"�nD�;�}�G����%�<�g	��l	Sq�hI� �N��f����,����M�v�8�P�2����%���H���������[��n�g\T�	5U��=6��R�@+�\U�P��e\&��gu�4|w��U��xO{9Q���v�F�O��	�xk�crn���2�&r6sBq=�����1����	��Vo(��"a�c\7;
#��Y�JY�q�Q����3z~%�� Q?�hWmlr�����P�2�����E�����M2.��$��*A���w���������z�f�[rd����`��}�>�|\I�!����;�U"!a�����eP>\Q%�P$
M�m&��xL��[{����f�-nr�?�nN8�3����qd�G��T�jo�-��gP��v.��<�g	�!ei��"�1�&"������^K{-H���:���b�e��iY���u�u��&:�*���z�eKT����
���J�Q:�XaB��B	(�%��M�A
A/�]
���N���%W=KC_9�~����{���g�S�'�����N��O�1k�G����m"g3']�=���
�	�P\�%���{V�E�z���&(�������kqe���>O�~���,�FT�[�K��E�,�:=�|_�.��~�����4�� ����#P�e��"-,J�<�,�	W#�#�V5���
/�H]�Au/�G64}�������l��?�1�lh���
��gE6h�*|����lh*]��gE6�^��E��1�|R��w�>��\�Feg����,j��s������
Z�u�����]~�I�.��PCU�tJ[;��"PBu_[��h'(�7M�r*��FB�\-���g{��t������K#�����}$����/u*�ZZ'�$j��>���t�PLb�]($K���	���PtE������������a"�3E-si."�����`�#z|��� m[;@�Y��hi1T���T-�}��:*v��gEo"
�����r���.�u��*7I�RF$"��fJX�������x��?}�9wI�U��Z��G4��W�FIqU��A7�cB"��#��)c���Lb����	E��M�'��8zZ
���i�x�Q:��.Y���^d��x�D�F��l�d�� ��	�����D�C�vG���p�u�$�u-�:]�r_�������.�x�,�Xz@���n`/�w@P_���*Q��F�[���5���F�\i����,?<��C#�U��$]��
�,�B���g�����H��W�1���08��������������q4XG�e����\��"��������cd����p�D��Y��5Y�-}&���s���m���&W���P����,��������?c�M�@^c-X������\��#����T�#{N��*���"4���K)]zy�*��b{��~<��\�*ni�R�&���#�YB��:!&�����,=��3���N���Pt�X<�(�"]z����"	��.=�q��m"��Y��["�2�+	t9�}z=������{�db��g��HS���eY'nY��������5�p�Qfk�
�V���{���Zz�
��q��eT�6��6VA/n	*zK7��Gp�F\�-Q�r��5�����*�6tD<&4��}�YL����������Pt�IZ���,B�u������'���k�K���
O$P<K#\����:�V�2m��"�>iRm��ZQD2�5Y�E-�:E�r_����v�8�R��i��rU�r�����\d/"��RP�:�c���;���;H1����n�5�����~���YZ��z��AX�R
��D�	��0���<f�z=�Wm9�9�����1n(����P\O0-^p��M��c,^;��D�����pA�������(��63��@���������<���)�P����,�E������cA4�a���Y�>2�*���H�i���	�
=�Z����{�X����v�!)��Z���N>zj~<��4�"d��C����u�|VM��AG�cB#��'���ix=0�M�l�������X�M���4<}/kFA�E(�:
���Qd���C���:
KIG��d��Y�-JW����N��s�4�
Y�uB���d]y��q�����/����U�T�����]E���"����!�=�Y��z�)�S���r���N�2'u8���4"��)��<tD<&4��}���}�y�`=��Kh���(�*�,^�M��h�<�gi�!_��W��3�s������n#
�U�:>%��8�U��fC���N���W�+��T7�������*	��(��"�|Q\�[q]ZV���*��tY8���@*bj#:
�.��*WB����2���r���6tB<&�������}��{9�9�0#aG!xtN(�."��"n���
��w�1�F���OR����<��N[�Yb%Z�p�}��n�h��*� ���e�re��TW^G�n�PC4����o���
M���\DDPX�!(�����S��|&��Qs��t�Tg���h���eW�J\�&�x��A����S�:!����������bx�i"g3'��[�������P\�<�'�P\D]|7y72�e"�+�f�����������V��f����t�Y�1T�,��+�}���:*u���,+9���s%�]$1�
��F@��N�d+�b�!ViE@�2V�+���G���#�K"�5"�#��:�.�A�+@?`��!��������z��m"g3'����q��M��*����8O���������gID�[�<�U�-�>���2��jt6�!����E����V�u����H]y���5D��@�u��!����\d9 �JP���t
|@��d:Fq]a����	���
�.���V���qM�x�^��"�
~u"��xL�[���*�W��
�I�fN(��*��7
g��D+���Qz��N��O�I"O�IA%G.��o4���I����Y���Ut�������&'�!��D�����f�����w^�G$���k8����x`3�i�@��]�����j��*
XD=7I��BE
A�V	y���*W
"��%��gE$�������M
3��x&��x����;��h��h�M�F�tN8��hh!�G�L��j1��r�-��_0��`�qx�#�4�0M��U��	t�}|=/�4�s��s��e)�����X��l���?^G�nR�)����`*��"�VzU���r5Q!9���G�w�@0���5��Q`P��������4�n�N����h����>��!h�K��*�-�s��[�1�t��F�mq��Pt�,�wz�.��v�(�<B']�#�"�6��H�����%H��g@��NX�I�B��+��p,�(��gC)K��*6q_���;/�&��������uhs��A�5)������^����k�S�eW.��B��������)�V�m*�iYB��|QG%����(6O]��xo��crv��U;�'rfsBq=�X��(�*W*Q���{!x�O(�'��������Q�C�R#������\���c��F]H�rH��{�����R�t�as�u���Z7���h���D�M�����U������_Z������<��KX�l�-�P�H���	���Z�p?#�6���Zj�����uF}�DV�����+�1���x��'hg����9�9���'h�����E(����5���"]���y�K��Kx�L��K�!~(����L���QP�P.���M��|�8�*[��Wq��:v��y=.�E����\��%��Hw� (�r)��mG��$��s�g��G��[�T�B3�=�M':uK�>�����m6�
�KmAo�cF'��-��}
����s����	E�=���������������q�^��C��#��#Q<M'B�i���i
wX����-K�2z��6&�����%�x��/gw��dVT[BC���B�
G&#�-���\"�D\"��V�����d�[��p-J�IY��:4bNI��f��%�D��+�2	��p�.�&�����I��K�1>��Y��&rfsBa(��B�Ba{>FA�E(�z�l	O3����,������Ih�O�%2�A'\E����#^#�qI��Lb��e��
��:�v��]F�o��G$
�R��l�#&�;��kUSR!����kT�d\�q���m �Y�c�p�J\�.!��G����@�/��6tE<fT��}���E�}�������P\p]��(F����x���6Y��z�i�d7�Ovd���D#vI�>s%�A��9���	���K�&����Bo"N��Tb��e��
��:w�K	'J%<�LBmT<�(��������W�����D�%��3��"��~����&���n3����Q�����IM���'�����I��K�19����y�9�9���_,�������P\1��Yx���$cq��m	O3�|0���*6����������7*r���@�i�"�,��^��W1��"w��y=&�e���Q�0�qI���lxP��p�b�p
ox�]�������
p�r�O
�>�m&�����F�W��7�W]����'�1c��xL�ug���^�lN(������Q���7��zG����7��fGa$P<�$zs&&�P���M�.�S��'_�\�X&"�E[�X�
i1�����������b4����m��d�����`DI/�L$&�(������f���	�Yzy7X�fU�+�~"��m�p��Dj���x����%�,�i�����1#��xL,6�^_�D�fN ����mwv
����������� �Gk������#�c��_~�����Q:��9/�����?��4>]+2�u��kf�1o~�������������5h~����_O?�����������?����_N?��?�q;����~����_/��_����������|�������q���_���O�������O?��w����?~��w��������H�]�/otH9������R$�U�aH<EpMG�U}T���\��H�PD��1�A/�o_��������M}�-hF�#jT�j!�����[�1>��H��m>Q� t>N�������9n*B�yB`�=�"��H��I�K��+_�����; ���L<����#��'9w��������?��o?g��b�V��S���o���k�h��w��h�oy�MP��m�
&4>:V���|�Z�!��{u��u��qkM^Plp���S���Ji|�������������r=�BlM�w;��rNj�������Yo+=8B���/���������|P2������������������4�N|������`��=����PK��Fa��*PK{2K18sept/pg_part_head/12_1.outUX���Y���Y��]m��q��_��K@�����&,T"�$���0#�3�"���c9�OU��L�N��(u�FA��/��53�UO=����_y����������|��KK��>�t��^���~s���}����������Q)�����������3������W�Y�d�����wW�w�o��
�LT���?���^�������#~"��z����������w���}���w�����z����ZR�~�_
�����W�����x�{}}��~���Vk�^{����������"�o�^��v_���{��p�VV�%�a��)���sX���������w���__���_>���;��zw��U�-r1__�=\�4�g�e��ek�2��g`4ie�NWe1X�!����`qU�<�/���w5�����Q
�_��^{E!]E�����F����=rx%�>�j��xu�����_|�������;��V��������/�������������)��dU0O�9"R.`�7cP������+	���7|�������{��v���o����N���.����������k�������g�F�E��E8�hkcT�<2��Nk~������z�z�z���_w|qu���y�(��&BM��dX?��l���N�z�`i��\�~�����xV`~8���������_|��ow���/�������~���_��xuu�����'��������7�k^�������w�������[�Oo��?�{����?�����j���7V����F<�q����B�������|'���{H�o�����'x{��������_����t�
�-5�.4���v��0y��f��
�i�_�u'�@x�u��Z1�CK�p����{w��u��
P���J����:��V'��
��n���J`�R��gl� ���d'�g��V%���*�`��	*�N	�S��j�\�*��J��q��qp�~u��������G\'�pen�8Bh���0����5j*AQE'���3��B:���0Mu",�*����N�
hE'X����q����i
0��ND���O�J�;B��;�2*�M����,�n�F/Y
��~���RU�`EPh&�
����bj��%�iLT0����I�a0}
�	3}�G��bx��b,yM��2�����a�!���h����A�g�bl�k�q-c�k�e�O�z�[��B'��$�=�c{��7U�������L\`=GT�V�'je�����^��4�K�3�8����\��>m�K��)
Vk���Ql���1)�|c���q�;e��Q	L�F���+ag�R1|'Z�Y�[�����`c��B�hp��`�hZ*���D+����#XW���I��W[P�����?M1��l�Y0�h5*��R���('�'�''��q�P��#P���.�OV=S�e4+�S�b��a@���?R%�+#���Q(�d 	s]��c�� �����F�d{��M1�]0��0+����n��n ��X=��L������6�J����:M�$��	Vi�>mPs�:���H��������zN�����OU��8>Ul5&�0#*��b�m�����_Z����T���W8G%hFms���iT�d>R�5B+<�ma8��*���eG�ds��Q��_�d>�g1�O�W*�H+�T�Ai3�1���-��c�|J��v���Fi�J��Q�H4E%|nn1��K�SPf���:��Z��h�|��+�|F�2\u��O��gJ�^"�^G����jb�c�����1t����j>���+Z��6P���u�1`Tp]��h�Ds�^v2J�BcF�|jDA�a�b<�bl����J\!\5V9�]I�G )J<�L�F���b,�����gu��m8�����)�2���O����-�U�6`������cD	R%*!���l^�`>�g>CK�
u��b��*q������ ���t%�e�����������u��(k�P�c8�s%c���%����*F�Z�Sv���:�����v��y;3�zX��-����<\u�1^�����:�D�
Lb>%�g����b�����/u4��x�1����U�	[�A��3�9����1��A��l�)=\+�D�JdX#;8���>qP��OaC����BK�&#����,�d1@=�c��>�}�$��]���'c�j�����W�K�]�����-���e��
u���Dk�dWrj�X1��d#��P����65��}VL+M�2�d�g��K�M
clOp5������
���:�.�(��A�i>���bl����J����f��0F�Tp}(��O���#��MC1~�bDl��B����:�L��Z�Cv5bT'W�z�������B�"�2��>����!��Z\���J����������%��B��s�Miw�'�zY2�R'��z�yp
�������Vp�r8���z�f�8�W�
�����v
�����i�8�������`5*qil�R@��$��vW����5��X���k���i��j/�A�h�����@���\Y���s=����V��
$�����Fs10�����1tS��d>5������)1A�P���LnN��h8�^1B�B(
gc1gs+\Q�)�vTCA�\��~(���������	h��STb�Ji�l6Q��#;�,�:\����Bl�1p�|B ��48�:����������]�|�>7w%�iT��E�.��VS�} uH��-�<�
G��\�m�]��)��S'���2�Kj>�� �-�����\1�T��Z
G��\��*�K}�6�|2��lp����c�3��-)b�A�5W"��SS3j��)������Vpa�����<��B=�*�J�!k8�d��l���X�|&�{.������W2��]���v3��#��=��Z�A�3� DU�S������t�UB1����lE��J�W�����������
ub}��p'��'���>r�|k�bh�GH/�C>g)�*��I}8x�f�������|K�If�.�"���a"���:�h"���s�H��Bh.�3��=�3Du���2Y�dT_��l:���X�8������VH���

A��.<������M����"1�s1_5�*93'�B��yc7>k���MS#��*L���g���V���^����t�7� �������6#S/�U�(�T"��������|K&�h���b�c�A������Wu�������B+\4�������
� ��8�\��t�"�hZv!����q{�(O�U���=�!�Xs{�	}��"����%�I$��y$t�
\�O1H@��W��+.�����Kr��Vn�+b}\�{I�P���+j.0�)�Kn�������k�HY� 
�M��i,G�@�^�����m�C��ot4U[��)DI�;���+��Z�����6KnS����d��z23%i���y�$��t�AXr�b+`�
_/�d\�E+��(s*����{���b�m����P��A��-�x���?�r�9k-� a�m�lS&����a7hm6� �
`����hs�m��w?����rd�<�7l\����bs�pM��B��GuK`e�D����H�H���3��������q�mJ)�s��^����I-Bx��nn�=��R�qIn�XV��M�WC����������,��p3�Mm��SB�����*�0!�X����%��KrS����V�3���F
7}�����Q��y�����@n�����*��GrS�}��>�]`�^l��Kr3�������L
!:8�f.�p#i�}1�k�,�M$������gc��=g��,~��b�n���i�t�����&����1H�M���%�)���K����&`j@�Y'����}.Z��q�nB��������c�2!�\6�dlG�a�t~�������G5�����!�"�"�C-�7M�����%���b����!�Cd���l^<����@.$���j�����B�<������u�D�C.0�%�4zIp��g���Z�������Fg�%���u"����Z���^��I����)�>�B.@Z��N�P�IQ�[�J��w/����D#>�r��\��[`��L��+��@i'd�����1M�EY�%�m���D��u�Z���R7�����H�n��tp��K��?"9��F"6��D�|(+�b���� ��D�XN����� P}���P��H$G� ?���Ja��O�u������g<b��L$Q*���Azc�����>�C���
��f������V�����~�p����Q�a������W{�+gw����������w�g���:���r"��}��yy��|������|8b�hN]�?��g��z{�rw��>���d_k&��(,��(:m�H����U�KA�r�R��A=���h�4"���=m-�[
�i���;��S�BW���v��|k!}��?YH��p�l2�)$,�O2�JC��)$�x!�����2��4i����S����R�����S����������v}���F�����e���}�t1*||3[jtA�0Ek�HO�
����f9����I���iJ�:7v<� 
y�Fw������������(��*�b��U�������T���'Ae�����%/���4�����Zy?�tc��u������?uk�N��*@6Q2u���o��5��WN}@����B��
Z�H�0?LX=L����9����N��?z�=Q��K&M8��������S�?��E�������<��g"uJ-��]4-��3��u����<�_Z��r��<Y~�VSWA��	��.T��y
�W��g����B�&h< Q�P ������ab��g��G�g]���}��w�#����u<*�^hyJ+8��Q[�La����q��JZ��:�]=�-x&�G����I�(����x��<�����jT���#����R�S/f��a����^,(��w��<Leb���9�O����N ���%��f�F��TQ	Dq��$���U�m�:�t�i�>�dHuo��J<������K�LWC�J�)N�t���*j8j!����*��N��Tx�����GD����J}�}�@����
jd�}�Jd�zW��*��4���O����g��aW���)8v����^5�N|�W��=-�'v��B5W
F��'��W���'_J+���a��d����c�D]��/�:O�J:��wh�&�z�}���G�}�G�����%1%k��|_�M�D��AH4�=d��F#���8l
9�UAE<����d�}�})�`�e���X�������C_A�����a�����\:���������XP�O�<ztJ�m��5A9��sDZgu�"J&�[���&(���l~G����R��|*��j������<F%�A9��9v����j�tB�$��@|[�"g(�:�i�0	E:���A@������C��a�^�����"P����
�H���2��"���2����h�
�o�:[Y�����&�:%�*��+�TDb�4���Xbup��>�+U������1��}3�X�L�Q�S��8���4�r���b�3����Vy26J2�3�������g2��
T�7pB'�[GKA�<_?=z�bG����>v������z����6O�	����X�� �r�u�qt�B���JZN8�LR�+��P�$�c����KA�h\����!�pT�1���R*\?=IEZ�;	��P�����6����w�A%|��It�yT����9�W�eL�����?_�~<L1��uA-��dHe�M��C�w�38����jH��/8�����:�k����LGC=��T�������}}�5�G���Q	#g���?v�]-i��H�T���0a�\��[]���>�#���9���sP�������T������"Azz�N���A���� �����k�=>�-):_$y|�y���O�U��g�$;	Z.��������WR��3u����N%E��L.�em����?�wBO9A�U�'��R���]���R��Z�TM��GQ��e�������d=�#���UkL�P�/(9_�t������>���8�C�f���	�e��Oq�%s��M��/�9f\�B=�l�>%��m/�-�=_�rz�O��1�!��]����$(��9�*�#��sT�����f�S�k~����$��������i:_�p4��c6\9M>{�����-�&v���v�O=�h$h�+h�5�#�� �4=��$I����|�yTOk����5C*)1*�tE%���Oeo9Kf���D�2�R��"���O��O+"eAuj	eE}}��t�������~[�������a*�@�GTI�S%FR�R@,
��7�_qM�M~~��J���D��o���%��d"������oQ�b���_(G�P��l���]�������V?�+]����k	�����O�%yDbW�\gS��4���7-f���O�htJ�`d����v��<���i�����[�W�����9��a���&������b��R���LR�W�~��y.8�|_���w���q"M�����4�S�����5������__��\�|�>�l�W�eK����G�/�k����5�t�GC��"*�=��?�G����>�_PK3��,#)-PK	{2K18sept/pg_part_head/12_2.outUX���Y���Y��]m�7r��_1�/��� �U|����%���CpcOH�vu��\|A�{�8;��i�����8�����m5�X/O=U���_���?}���w_�����$�����������������'/o��?���:��HDQ������gf���W�o>�l������W���������W�G��_?vus�����#~#��f������������������������}M�^{����KA�T����~��_�?<�]]^�/��o��������\�{{�j?������7���W���?",=�T+��=�3i�>�
�������������������;�}����_�)~�<�����,M���X��X��}�GC�� O�")���i<<-�������|{s{_����{�������,*|z��O��@��j�#r�J"}��d�j���_�_�_���.=�W��7��G�v�����r.���{���������������X���y�����6]t���;���9|%��������3��}�g=�a��.��]�����������x�n����Zi�g��������C:yhp����'�'�~���������_����u��������#Ey,4�1lGs#�l��j�*�x�X\���������-��Of��w����}�����/~�{��o~�����������y���������g�=������7o���#����w�����w�����<}�����>���g��M���6����w�:}�;|�N���/�������w|�|���������z����>��7|7�s�j�����4y�R1`��N���b��t�A��T1lF#k�9��������������Zj�=�d7�����j��B��P��L�q���:������S� o�Q%\�TX�vk��8T����`�9Tb[�`��R%h�A+�?��HE���f��)�s8Tbc����p����Q�G�@����J�fn�8Bh�~a%�U�c�	k�T�L`��X��9SrC%����J���p��X�C�q G��u���&L�h���V�4
/��JD�*L ��u��<��S(��3HaG*�9H�H7T�� ��
'����'��Zk$�����hF;cc��:EK�X���DEHal]1Aq+���4Y�G
��b�mj1��&���kFGu� �	)+F���������$c[*��t�qg<Z~�+�$z�c�H�%4�X8,��p76U�����s��p�
Pp�qI���4��3���1���XA��� �1NV��(j�����Lk�����R18'���1���cX��\���h���tusW����b,�N�ed� u#8�Q�Uo�2s��F��1F4-c�y:�#��4��@�F�Z,���M|����S��b�<�0Z��f�|:�N�0\+�,��O%P�M�����[+F�-�X"��z�&~e4���F�,X9X10*�oo�J6W����Bg��fV2�c��&� k���+�Q+�^1�i�����������QH���X��O�E�bl��m����4���J���ckSVUsV2���(�AK�X�:�'f�Yalq.bl��D�c��Y���CS�X ��Vn���5v'(@��b+�������p�iV�D>��Y��_�1V!#�!��8�{���lpY���K���
�S���
��-Fj���b�0�#+��bPlj1��'����Tvw+��N(A�����]���ss��\Sc�|jC�L��WWY)01�EElF���V�y�u�t�8��]��!��8�����t�L9�18���1t����8�V�cu���Q����p
��;
�9p��E4[he��A�>�c�1���� |q�!q����bDh���B�j@Y�]I�8X�����,]-h�x��lK��S8�!��O!������
.������mI���t��|���@�"cxm9����8���������NxD^a��cjQL�?���$�LWq��
Y�����c�0/���p�����sd%�7���'���2��3�cu���:�!D����^!e������bh��b���,��^���z���Sd%� �l6������CK.�:Zs�q�1@fg����jR����s��tu{>F�1X�f��3��R+q��H�)T���<\���o>�����8�: �Z'#�!��!'��\Z�y��7�}`�ZIl�����'�N^��;���!' �Y��(�]|��c������5�V����0W����9���}�@��f, �32�_ij����HewNW�n�0b�����O, �^��
����m�W�*���C1�oQl��%�i��|ct=����e������<�'�A��3#����$��6�XAV��y4��.�P��1��1�w%�%�E�E������JSs b;!�!���N4O���9���e�I�Ek����|&+Y	�/A���
���H�wu�n��.Zg���O�B2X���
if������)K�J���2S�*�;�Y�8L��,0�fp
�s{W���EK��1��������2>3���c���nZ.���3���0ne�����7�*cf��Qv�~>�nc,�O��Z�TW�.��h<�5%�J��#7��W����C��lZ�<���0�J�Q�}�x����bl_]
-.�@>I�cVz��P�i_���g�W��j�[�n�|R�V�i2p������]	@�����'H��>7w%�iV��-��<���o�LR�������3�hY]u�S��S'���bX]�����mG��a16W[*Fa-������+�%��>��
!k8��FF7�1N�O�T)=f��C��	>3�4*����Qv�~Y^S�+t�G��|�P/�K�*�%����+�~8�nj1��ONa���Pm��
�ch�����(�op��c�	�I"�1N�����u"H��iO�:G#�����w|�e7O'���j)��G��*d4;�[�s1���z��Xw�t5�p��ty��+c�t$���o[F��,���C��^=#�J�nGs���w��l��!�
���>c����D�����|~��v6����V@��02�j+4����.�y�G�M[=.m����8����T��V���,f��,h�����F<��]���[��V�21��q��9���}�M=�[hE������Ti�����4�>�����0j[�X�/������@=q��H�
YV3O�Kw���}Xh��F�)��X��	�(���A�Ff�9E+��=�~�m���yz�j)�u�$�U����1��D��	Kl�{Rq�e��A���b+�Sz^,�a�ghdo�x�%������� +�r�"$���V~,�9�(��qEXb����H���O�
� @l+�����g��5�
�&;��}����
'C�D+����K��Pk:� ,�M��){Sgq*/Z���	>���d{���T���6�V��X�p�l���v���i��ik+���4f4��1���=�"?/i�0<�������%�I���LW���Z�%�4���q{=���'km.�M"�L��
�W	�B�I��rD[��V����C����ho�3Q�Lub�Q�M$�mn_Io���KlSv'c�k�Ul��A+��6�zD����M���6e���M�OV�1�N\E�,�m�qg���2��pS�\N����h/d,�e8�<�1��j��R�p3.������X�;^�
���6�5(z�M!4u!pS�����"�_���S���It�9{�ey,.����u���[������+of���h�=�6MB����.[a��	�REC)�R��Y��l����-����:��L�AhYSjJc��MG��%�)�����u�VH+zHqEd�1�����3���%�����q.����B���dE�X�zN����^�]�1��jl}����P5����K�`�������VY=��,P��@Fa%��%�-��zn*�����r-��Kx��Z�yZU!�d�7;M��m�v�!g��2�4z�o��k���N���
1�
9e��L?�p��^��p:�&���������M���N�Em!g@-��)�7}Tq.�s��!���?7s��HP��9mK0��%�I^+3���B��Z�1�Y���k�=)+�D��^���b���X����*����G�t���s��^����t����aH��]�!����=-�B�T��4�����f�FE�����#�<��b[�0h�I�!����{s|����6(��!���E"���:�~����4'��`:����I�>X>�����?���8�_��|������}�p���Ow���w	�����/���_�:��~z��N4���*A�~����7�������;}�/&�9����	3(��ga59�����#��aDUAa)�sC~([������Z=I�g�ZP�4�����������(KF����Bb��?QH�FY����}|��������BZv8r�QNz
�~��Z9�^wT������t!�����Z9��S��sN��$�����"�iX#�Mc����Z�
gi
�YzT�BO3dtA�0ek�HO�
�)�[�?���]-y �����scGAP�i��$m���8B5�<{��;�HV�f���GZpA>L;����z��8����J���*h�
�i0Mz�������;#���*h���<���1��6�*�q�}�}�y?��$A���Vl=�G
����K�(2�M�~�-l��
�k<g
n�_��^��+���v+k0�}����)w���A%FE�5(��,t��P�M^�	��&R�TK��rn���]-�&���~h���!K�Fv��}-�&�o�X|BAE0��UR(��#
��D}Px��$-�]�=\�~�\T_�f���:'l��T���������g��$Y����C�5�	(N��C����eg���\����'_�8�&Y|���5*��kr*��.����Q��+h)k��3�h�
j�o<����������k��)q���a�������lQ�?��k��e?\��r"*S9�O�koK�]��j����A*D2��F%����%	�!
+�)�������d��z���:J�)}���~�J*<RRSWR-�%�7I^�������,,�X����:��J�����`���I~h�6�U��kY]�
Z�O�*��&�U������,[�&�����$V1?[MF���M�u>�u�Q,�m��2��2!�K��yi3���%T��`I�eY�2y��k2�%T�?J�c=��<T|��Q����$>��D��:{�K;�����(�<�+��ph�����k���bGEy�d��)�`������k��yX+a	�
� ���>�R�5��o���{����G��
�j��\r�yHZ�u��H�r����T�MAA~�P=R�nI�e�*tM����>{TA��gz�����o� ������"A�L��}�eX�Q�^�M�d�r���1}%-�:�`f��V�T�dI�|�WIKyr��i5�3\�d���Q���tW
�VD�$Bp}oS)m2��s\�*�Tl�AP�k�\	�qS����T=QJ�� �]3fWrM��)�iN,���HBR��2��k�����R�#��a��� \�5�G_�8H���B��]�q��i���^�Y���I6��D#t
�]�5	U7����1���B#�x������&���@U%�`c�O����JZ.9�LP���j��" �{�{�Ky�U��0�bH�|�� �����R5\?5�X���������fnrM+�0��8�
�k"�K�r����X5�@����!���+j�9�YT�NuQ���D�b���)=�$�P�����+!)����n]� �t����&��P�9�������]OAK��>�K�Z\�����}�M�+�t��L-����D��#���N��>���0�Re��U��o�OEI�-� 3tE�NwwC�\�:J
l�����v����$^}^i4�[�N4��$8��DO7>����T/pD�&_����z�N��iE��h.qH(t���w�O����T��5�,u
+�]-��n�SJ~�������
�\��TvM&����,D���i�y�l�t����������0"D������B)3f�S\�LH��eOAK��E�Dt_�����������6=��CIG�
t5O��@��� ���z��v,�y|�����G�k~���7�a(�I
$�oj�2���>��Z/��'�V^B�-y&>�\P�
JF�e���}	������MyK=�g�L����y�]-1!��M>����*��,�  yWAKYST&��
Pf�D	����NGB��O#��L�MD��QI_AK�)*���~%��-~,(�J�
Z"B�GXI�c%�!�j8r�:w���rM�M~~��J[�
�����"t�������|� �Np-��q�b���g(?Z9S}����|�%L���6��v��-�}�\�o.�x�����X�2���j���`1��X�Z[#+t��J7tn�-������t����mbNcgIK�S�����o�TH�q�����'�$�H�F�I<M��v�NS�0Z��G��xQ�����0_�M��D�|L��W����������Gz��w���������,��?������������>�_PKZU":+-PK{2K18sept/pg_part_head/12_3.outUX���Y���Y��]m�7r��_1�/��� YU|����%���CpcOXK����]|A�{�zf��i��������%kZS�]�z���������������������$�������v_\��~��>zq{��i1(tJ�o���
zww���O����/^}j�����^<��z�{�~�����5X� *���t��������?�7��ov��^�\����~����wo��������}M�`��*����?.���0%)?����m���������������~������on_��gE��]=�������}�#������!Xo����>�si?�q{����~��������g;�}���w7/^?"7�����5KSa�-}�-c-*C?�<y�����������w�����v���{�	��#)pG)p�P{�(n�����&x
����~&���n����?�����w�/���������;�+����?[�;��������_o�oN�j
-(;)�1@���<��$�,?7=�H������g��oo�?:{F�w/���?�>�M�������������?�v��t�Zi���l�|~��x���n�:������u|��'���]�~���f���7/�nv�7���������'�b0����?-h�`fq��[���o�_?���=}������?�y�������?�������?}��_���������WWw��_���~����}�����_�^{���~;�~��������?��������&F�D�/c����>��i�l�/_!�����[~�|��=L�o��������x����������/w�a~\@A��i�v���4��b��t��A|x���Y= �'�������-1�T8W���^���Q`k�q��Q�����wmm��y��=���Z
���./2�
';�.s���*�o��P��`7��JX����
���Jl��:�p.��b�qpD%f��o����#��*�V��}����FM%X�J�(�b�<�P����i�ai%��Kx�u0��xif61�/7��)��+Q�0s��#h���*�3�����n�Q8�
��%G�Pg��Y_U�H���(fv!.�P������Y���DE3Ia��hQX�#��E0�#qz� �+BS��d5����Y����.�_E�/73;�8���1�IZ*���t�qg<Y�T���rHl1����c{��*���dWbTH��[K��"���S��]���^��4�K�3�Wqv%�u��e�jJ�cJ���im���-c�s2����9\��|:��eeE���n�J���T���,���B��*��Re&�4�IA>G��#�����<�
F�R#��^[�a�(�;��
�_14��)���%n���l\�	���,4Qi/���m��0N���b�2*�K��UO�\^�
��ZY���7�\u�q�J6W�������z��RQ2�1��NG��h$���+�^1lS�a�'���_D[O�{!��N��*��0z(����M��d>
r:��`W�'�=%�5+S�D��I4�R1�U��C��j�L=�.<�J\�	��
p3���-��Mc�|:k�r������RH����4�K�1�h�[��F%K���zF�
��bP�-��@P�a�v�J6'���,��K�� ��>)��+�����|"�AH<��J6��Z�%���0��9��VJ�(���
."mJ��>7��5�1���6��l1��:�`��v����������g�-�U(�|r�u������c$\���#�c�Q��}=�n�@���>eW-�������+����r��1F�%x��I4(t�c�(qV0�����S28��#�|cs���eT�p�X�\I�g��L����|��_��X�%��)��gu��(0U�1c=��K�OhY��p��z`����X�&Y����gh�c@�P'<*^��V��������u%�e���p5)�zW]c �����@��R���ah8r-�'���.JW�)�n9l��!D�J<�s�����[��Z��p,0^��w�AP|:i�1d<�s���ss���e.u�f�q�1�������n"�d�FLQ	�pu�z��c�0~
@�Dp��J��q[S���i���
J|�l�����P�c���b��B��
��b`�Ai����Jb��U<g>c8�@�������*�}H��
���G��0�i�v�B�����	c�dW5J	�!�U>��C1�oh����3bV�W���2�(q����Z�blNp5������2d�W0F`�1�x�Y�2W2c���U��d>e��\�a�^��@�t(�s�g�	k�b�L����b,u,��������>�rx��(�(:=x��]	�$�����x��S��JS3��
������Q��9���%��B�"��9���d%\���}���.����}�{�
.Zg����������"��p$�'��%��9�a�V�Si8�VfW��G%��W%��p����\�����*7����J���p���Hi!����@���\XC��O�����W[�T�j>�#5%�9������M1����>�J$�Z������d`�2�i��h8�^1B�B*
g����l�>9~�i5*hd����P�����%���'9Y�{�J�^)��AF��I��"�
u�p%��6
�%�p��C�8O�P�����*E��H�1������Q�[�(i����> ��eU�b�)�UH�JF��\�m�]u�S���N4g��ad�Vb� �\�bl�[*Fa-0rH��+���3��vR,�J���������D�H^�Me�C]1P&iL�lT(J�1F��%��5��r�nw�i�'GuC�"�v���2�5v�]`8�nj1�5��*�J�Bud4X"|Z��F�5���������O� ;TO��%UR
We������^lG�����t��;c>�l!I�:�>�u!X�D�Jb��w�om_�A-����p!��t`�pla�\���8�b��l	;�Yj�S���>�Jf�(�h��$�9f����8z[� >U��/O�MQl����F8����MgbxX�
)��=�`������������1c{���'���V�`�����j:�(��T�!�h��h�Y�Oo��T�
Ta^��u����� ��V�����.�������VH]�N��L���&��q	6$2���gmaK"���Vx���[�A��L1H��M�bl����c�a�h�9��\h�h3�%�z�S�����L7����e�_r����I��V�"*��nIz�D�K���$,�M�I���=��D�d\�
^�W\���%�����`�+b}Z(�IY+�
s�X��+���W�%��<��s���o����R�&ZM���X�x�����;C��.���h�]�?$o�SR��X�}��X��aIn������M�\9�`a�W��1��w��aIn���Y+�^�i�����Ka�Eu5�����KrS-f�zub��YY���>;<����L��%�I���LW��f�$��44%�l��h����-�\��2�S�^��W��H��*�vK8=l��ZM��B�&:�O��l�XI�H"�u#�4����}*�i�f\�������Ur����(��E�mn�<�7�����d{��LnR}���lI9/9��K$\q�Q�-�f,���de7f-�����UL��1Um��Bj�6�����v����1�n5(��qz���
M=H���mC�!�Vq�Sd<k����lJ���D�[������pBQ�+�J�
kE@�.'�X0���cA�d�m:��fX�Y�V8�rAV�;l�/�V,�Mo����S��x{a����O��i�Q�@�E��7q�mJv,cy]�N�05������}&�Zr�q�mR�iZ��z�&G�2r������p�F�_�tz������/G5PU�iz�;�4������#Ze��Y c�T�_�B� S���IM��Z\ Z�E5zIo���4������)�V��C.0�%�4z�o��0):��c�#�m�Q����3��Y�z������^���^�^�P�)��l.�h�ka�z�B���*&pAU�y�bGl ����u{�	-�,��'y����QHA	�%�r��E6eqX��k�BK���%���u�Zx�����|6������-M���d9��)�x5A���!'���~��m_�E�m(rFsOh����f����28��)?��Yl�����CD~u�����X�!(������"c���^���A[�"�������I~�[>{�����=��c�j���>�zx�j��
�n�%?��v���n�l�,������x�����������[%K�e�����O�'�������[}�/Ns��k���5��1������0���v)�,����laj��aWJ���������4�����w����|F��7OKu�W4���B�qj
��E�}������B�1�]��e.R�)��9B��md�E0�SH����(��A��U�����u�"��
�S�sX#��c�'{9w,���1��z��]4���d���A���@��5����<�C�E���QT�r�k��
Z�@������g�*�����������gO4��eu4�j|WAK^��4�����Z�(�b�6��O���<>uuH��<,"?������o���3	��O�����}�C�U��Kr����UOST�E�O�J�������X�x����k,��{�}-~�5�}����=(����"KY	�
jK��������U��@R���~����|��gx���(��b|9��*h�7Q��T5������ ��
Z�M�����u��8���:����|�����&[}�"�q������#-�I:AR��I������1gsAK�)�L�$h�ZR#t�RV��U��s����u%�/��J�U_SZrN.	*6�.�=L���6}
T)n��sz�a
����IlZW�%�Ds������:	j������|����2��?�0MI0�u%������Vm:�~-d�1Q0�h���JT���Pt�e!D�j�;	Z2OQ=��L���Q����^W�%�����(GK�&��X[�
Z6O���+�I���>R�H�I�����<!�����)��g^�t��s�en�aA�w�����.r(�hD��	|rl��/|�Rb!m3�S�(���PQ�TSW��%Z��l�=&>��:Z�u���>�C{���A�S�/���$.�i��Y��w���$�]h]P�XI�6�a)�����m���'X��`��{��n�
ZpM�+����J��1
A�UI���XP��&�r�a�������5`�r�A�������5���,]Q	�\S�#{�v���)���+���S�5=�z�9L�����'*�&��O$�8�"��d�����I�G�V?L�GHx|+��:�h��a��'Da-����R!�WGKA*�oWg����]�;*s:�&���${~�J�Q�k�KA�Q!��a���M,Ydqe�NGA]��qs���`�/����0t�������"�X/r2J�dVu��<��g���~�VY2�^v�1�Q�7hr%�d��Z�!N�mR���zW
����������'�DR���>��g�#�}i��Q�$� %O�$�N���&���@U�����C����q�l����*(JFd����m�'Z��@�^��Z:�NDDe�?�R*\?5a����(H�Y���Kt�K���gZ�j��mg<C��1�/�y|2r&7���xH��2}I������s�_��(
w�_s�+|:�"~L�D����!
J�����3�;_l}�dJ��>��V��-��JZrM���UTb>T�������L^�eV�3C�{6��}��F�#���s�*�L�;��g�D�+�?_�z��OdHYG�X�u~�%�d1�~�!=��{��_w=L��$��>O2��S/5Eb�L
�+~:_ox���y�����J�/q�q�TS�Q�q����'p���]��|�	?���#�����N����������V�o�RBd�aWu�*���X�:.a�$�:�����|{����Og�z�$�B����4�|�M<��f\!�cS �!i�����c�2��f��9e��0��������:����R�R����|�$()�Cg��ADY
�����O�|9���5��z���*��2w/u>_�p4��*��J��o��m���/�&��\P��:O�^��U��k�G ?�A��b��������G�O��y����q����79�}i��������)3+���	�s�e��#��O��O�"eX09Q�F2}OS�7E��@����T��&������ �#�$���|��=�Uq�/K��m~~��J8��d�e�]W\�.zDPO�����{^I_t����=*a��NA}�XV��mk+������}mlKa������������P���`���K�/���R�O�j��L�V��Ee��iGm����3���G���s��b��)p�
�~v��G�%��N�����k��S�L�����|�{�K?-����u2�+�v��50��w������^_��\�|7]�l��"�������{����5�t�C�Jm�-5�=��Gxdn?����PK��c�/+-PK�z2K18sept/pg_part_head/12_4.outUX���Y���Y��]m�7r��_1�/���`�")���Kb����4��vu�������Tqf���d{e�=1��K�����X��So��������_�y����xiI��>?k?���~s���}���������eI)��u�E�����������/^}����GW/�]��=\��
A����E����.{}{������F>��n����������~ww��������w���I��`��Q)L��0�$��'}����?<�����_?��|�5������~����~�������W�������~DX���aEZmmT.�LZ8�$�=����������W��nn�/��l�_\��y���������Y��;������*p?�|�������|D��a��;��]��]���8��on�j�[�YCG)�l�B@�C��
�Y'5
^
?r#��$���Mv����W�����������������5i��?[�;��������_o�oN5��FaRX�<�Oz8��&~���l��A�����~�p�|{�����3��{�g=�a��.��>�����?�o�������������d��g/���������4zcT�|��b`M��w<K���������7����yqu���y�(���S��S�dX�F�.0�d%F11���������^�~���{�h�_~7�����G������w����W����?|��>{���/^]���}�1�_��~���
���bd~�{�������m����������<�f��OwB��
�/c���N��wZ?K������������Cz|����?~���5?���|6�������2.h���b�B1����'�p��)(�W-�1c��g�~�8�h�l�]K�0��a�@s���j��
����9�D�Rl���{���B<��ME��Z�������A�g��V%"��*�*�r�	*�n&:�*����P��U"�������%g'D��9��0�U���`'�0fn�8Bh�~a%8`�'��6�
��9�������Jlm%��J��� �&,����*h�����I%����[	h
/��JD�9��x�Xu��S�H�B���S1#��� �*�%IAF��p#�u����q&c�����&�!C1�U�MRX�����M,��b����Y�pR=B�����cIkZ���5#��b *k@��)����P��iM~-cAk2�<1R���X�J'�~���J�c{��6U����T�i�g���|�d@%���x7�����c�p
m'Wbm���l'�'�
�24+����Q����b,xN���3����1Y���!�����n�J���T��I���b� U���X��#0+���]cDh���0��S#��zzE��p50�	���%CC1~�b���d8,�OB��'�
�V��,XF#r�������)C1�V��[F%�d>Y����+#��'	��';�|��*�\1L\N��(t2jpsU��c�QEJ��J���l��c���d7@Y�E�j��hV�T����E;&�z(����M�\2�`#;��b��O�\C������$�p%H�aK�XVuF�T�k�`�b��b�+!c��hRQ���cS�X0������~��$V��|R��<�$���t��d�|�Y��_�1�W1�c0
sI�\����-Kq�|���O�S��+!V!G&�����y��ln1\lj1��'�����v����1�j�Kg���b5�1����`��>CP�`1���8)��+�|F�2\5��O`W�gJ�^!���	^����5���c��Q�)�|�O�Ut�>#���S
��&�1*�.�c�L��B+�
%����|��#1|U�$U�s�J���\1"��JL!\������T)q+S�(�R��!s\f_2z�~�&c���7���>C��O��`4��,�U;����O����,�U`L9d�`���U�Hg{�����hvP��S��%�a
�:�Q5p�L�8��9�
���C1.�JB�x�.�U����y��yw����>�^���8�a\���Z�O{�Rt��=�W9n�(����SP��kzX��-����<^5h00^8�c������2eQ0����0S�4������%\vY�#�7�Dd����U�D�j@���x;���2��A���k�(=3\�d	�����"�q�j��_�4H�����$�m�T=c����UR���|\��+�� F������{��s�2�$�J~������e����Q��=����w[(��@�4���zX�0Ve��A�8�;c���m��@}F�����fDR��VQ���00��W��O[�>�V!�[O�� Q*��P�f�����I�i��[R��0Ud�zX��S���L�z*?a�P�����*��T5�S9��6�/�2�T�I$4�\�7���]�i�p�B�"�d����J[��^��G%3�t��ss��BK��
M��(3������gJWI��Z���^����i
�[�gC����'����ME�^�O���������hZ(�J����)^EZ�>e/�LB9^u��A}n�K��p�e��T4m@���(�I!5��$=s��"��wh����>��+2����8�=���E��B7�����MA����~N�H~u���[i7��;��l{
����#4-�q��lZ�<���4�EYZ��q?��C1�O���-�OG������>VYd#C�?u�Y2\�������T���^A��Nc�L|�"�(�����q���]	4�Jh��h���}}�g��^���S��u�����[�W�@}J9���FXW��n�J�$Xb\2������b0l����I��[o9
J�[VV{��!�:|^`jtS�q�|Z�H^�S
��PWvD��U2�x4�_z_^�.*4�G�������J�da���C��Lp��h����Z�e���0�����h!�:$��*s�'���oOp������t.�"��\�������)�J�Fb��[�F/���:c>�a?�Y�N�����hl.��s���[�Wc��~���VP�G-]V��
'#x��������;lK��a�&$���3{�<��� �u��������B�\���@�(
����)E�����l:����`�`'"3�*Z�U��B,�Kv&3s�L���f��Do��B�{Ne�H�|*���D�J�kL�����~zx���w��j�y FU+dB��AQ��>&n]������V��{^h��_B"JhJ�b67|l3���-����~�R�7
��� DI�ea���jl����d�a�&��S����Y������87"����"�lC�Kr3�e�S:W�B2'�Wx�q��hC�� aInz�T���C�W�+7X+d�w�3���+.�����Kr����W���T^V$Zi*B=Ws�-5�W�%�I�(7��R}���c���l��M�/�k:�3�M)�=��:���	nF�Y�����/�k:� ,�MB�:�=���d�Z�A���|�KR�M'�%�)��MZa�U��AfJ����qL5�����KrS��d�zu
��v�CS�*,px�LZk�	Kr�9�m����5����M��a���`����hsIn:��?Sh�R��d�������;����k�i:<*7S��4��3����e���"�@�����Vn�%���c�9iZ��x�Q������[ ��������w\��.Mr��c��j2^�����gV�8���1�����n�K��n��8�t"�iE����1Wm��B�n�%�$	:y��������I��t��)��.�@nz�o��*�p2�W<�����b�%�����2?����n������1{H���1/0#k�}5�i�,�M���1��)�����cAT�f�������%��1[�+m�u��\�L����.Pb�t�M\�����������c���3�=\ ��Z��q�n����`�����I�tl\��H�ts{���_+��"�R���#�8m������P���E�)�R0�DV��!s������@FQ�87�kq�\Hh����tl-�<����
!Jgz��71n2��L�o	8A/	N�yc`O�Yi��`�����Gk�E���u"��P�I-b}r/hE�(N�9�>C.�Z`[/R(��Q�\��I1e�dNNV��G��=�4-�,�K��y�`j.\��d$�B[P���s���@UVhI}�^��dl��Zg��
�l��ldO�oh::M&�-��r�S���8$#�S;���{T�m_��B�P���� Nmn���p���YuT<�N�v �m���'���:�������3A�9\���������6��!����>�{�Z#{�d#������>���*|������{�-b������������v_���a�������v�:4�����D<�v��L����gY+�2��w����,`~Ro�^�������b�C;-L�AQXq��,;x���OGT��M���laj�"�R�M���3m-�Y
������<���hTD|tEc!m��?YH-�"�FE�ek!�R��D!A>-/�������s����1i����t!�
�t�*C]�d�9B"�/�� ��B��984M
N�r�YZ���wl/n����]4L�Z2�����fz//�4�	�
Z�@n��.���Ec�SM��X�t�����<{MP>������W���W��'}��2��~����U��2�d������:�CF�T4���O�����C:TDGe����U��7�~j�I�R�����Y
V�r}�S�%���4Q�4�5�(�)�����O����g���8���%+K�B_A���S�@��A��wR��'�����%���|�P��>Zy����`uWAK��*��W�=��1N6
mW��e���/uk>�$�#�%��U��k
>��~���e=���������(L�UrMX}��
��z��:Z
��HE�: e�,%�������-��0��I�X��|���������������|B-8���r���%�D��b����y�bG�a��}�S)hrS7fz��*(��T�����<��
%��(Im?��4�����<�Q���HU;�1����D���7%�.N�j���*u*l�t�LQWAK<�����P�Q"�|s���zS2Oq�����D�LXz�E|�����Sx��P�Q��6�#LW�o���f�����d���G�tE��`����|i�0���Hs�X�U��y
�02��t��[�L.�y|o������.����TC>�)WCl�:�y��U����Tg��0a��eL_-q:O�J��4�Z��WGK��������'��L�����>���S�Rt��V�&t���8f
9�u���3�s����K:�R��uf��TM~�� r��Cy�WK���������&����KJ�y���N��\�w��	�C�:�cULC�-k�������SP�?Q�>Q'��()�+�(WpNOg��}����������'z����=�W�2�C�����@D���]�Wbu����P��=�"i��)]��+�M��7�U���\�7wf�]����n���DG��b�'Z
�@����-�pF��t�Q*�:4u\�����]�rQ�Q]��<SxT��5Np8���WFw������z�-�*O�_h|+�dofOAK�	�|=� ���z�u����S)l�'��W+:�D �������d�C��a%y��IG!�}����e\np+\n<$���A_B����	�uUPg/�	��/	A����?w�v��M��CQ��P)���XY�ll�R��SP_b�h.�M��j�mT EEV���vu��D�������bI]J6	
�#�K���)�&UP��D����.8�#~��D��}������(4]-��j����)i�g����}����?��|�O9,�:,1|
����@�|�QG!/����7
R�)I<������=b����Ja	��#��}� ���-�SIR�N>������]%-�&x����T�U��u�F��m��w�'a-�h����+�x����?"�q�4�ju��������;edn����2-�I����?�w�O9EJ+���?������\=�~�k�O����������XC%)�\��������!�_c��HF��#�l]_AK�	��.�:Of>I��J�Z���|��1�`2��9f�L ����l�
Z���\�V��"F�S��U IP��s;�����H"(*�]���v�S�k�t�
�tAe�����|_���>*�:��q�O>����|��1�\R�J*dE��c�B�GZrM��u��7��������B=��O_W�w`L*���������&V��(�5F/��2�fo�59}*u~Z)RI�
��O�����9y�U���]g8��d����#t���P'K��K���^��-�&�6??�a%�)-&�B��W����T������5��j1���1��[����$�����V?������G�R;}���Z��h��q���
@*t>Q���'65�BS�S����^���'������p������7u>�%��sn��Z�9�e� ������R���LRcW������&����d�h���H���+\D*~�	�
a�E'r�3��z}uss}�]����F-�T�{s�������x�p�"������+����#{�m?����PKf� /-PK{2K18sept/pg_part_head/3_1.outUX���Y���Y��]mo7���_1��.�U�"���^��6�\r��>Zy`�%�$����_U��I���(n��0H"�����b=���n�����_}�?�o���_�����>[��h���7���'�W7��2!'��_�����7�����������>�}rv~���������s�������<��a|��������)\��f��������� �=<�"!���s��g��p������g�On�����������������?n������^_��w����c�����wo����g�{���������O������~������������~~���v�|��}y�������g������p�z���Z_[���,1�~wqy�������__�{��������P�@��CrQJ�G�CG��B�Q��{��3	;�|x8b����<��E`��������0������bu}q�����?�}��V��_��{��^$�x�����w���		����{��c��{?����wy}�������^�����|�������w����������9
��u{L�o2�t�C#��Oo�Y����E��r�{�x��w�������������E�~f��H(rb^L��Q�{4H&��'Aq8���s�����Y������=����X����������..G�3��S�Y�u�2���X?Y��
H����E�p�����K}������3P���s_�}�����"��UF��8wIL�$���=Bq)�U�Q���=�dx0�5��8{�z�z���o����.wW����/_<�w� �G�q��'��]�8���F����}8��x}�����`��1��f�1�'���g��l��Y���*w�����w}?>Du�,��9L`1:	#��rn���sB�^`��}/��Y���a��0{�Gw06?���]������C��D���`A�|4��AC`�wu����r����( ����S|���*�@��i�!�kP�c�#G�H+u!���S!N*n�vJ#4�T��8�_?-n�_
_�,|5��9��� ,�V�I����h����p~1L8M
8:/G�5�Z�#���-�P�(S?h��S�%��O8O������00��JgH#
�$�hD\��_���`(�f�����C[��z�5`����v���KP1&��4�(��L�h�G�R�&�@����(S���P�x�(C>=�Sz����QV��6f2l��5*����i��
�JQ�'8=�vz��yMCYIr��54�����(KIU�g����;x�	���R�eO�I�����F���c�yz3�����0O�^�y�9Rh������B����C^��`����)��1���IH�RC}fu9��1��SK���(OH��r��8���@�e0(ALd0��&���CX������AP�A�c6�N^���z
G)�8A���U����	����+�1$���b#k<�������I^���/����S��O�r��(�f��B�f���������DYb���Lu/IU��t�tQ����XN9I����D+5f����0���lP���1s���0������2R���������O�d�G��Sl��H?`���xJ|~��\LR�aZ�����
�^����!+9q�RE�}Z%�����a�0@P���s i'�pH�Y�EJ�+���!�u��Io���<����X�@4�Z0'���kd5l��T�3�:����a�r@u��F��}j'��@�Ia���e>
	�u�@�_;8�)	���\P���4{B f�AC��
�{�0{�8OY�B�
���7`�28
�����J���G��3,0A������W.C����	�T�fY)����g�����3����,��KAch��ng\g�,J�N�L�'QM�C��9k�g�y0�����\i�Nz4q�4�U��R�D�C6s���J�!W�Fy��r�hO�q�iL�$�.Z�i�L��^���\��:�3S>}��AHM�}L�f�Y�lK�����T]��6b��y&�������5���� �$��CS�a+�Rs>}9��Q�u���{V����e�bt�w�;cubZ�9k����A]��e�q����9+�N.T��7�*qf��v��Q�z��������s�"H��F�2Dut��
�$����mL���c���f����<0D��r�����7;�n���r�"5���rR4��m�gu7�
7 ����b6H�������;(���[���an�*W\�n�WZ}M�(t�
��-�yd(�&�[�l��6�,�f��9����f��3�$�m�dL�S^��fv2��(9WO]�O���3� �0B3m^z
�1�������,VtQ����5�Td0�:�	���s�d�%���`3����YG�����*����J�%����<�}(�a�����s����sT����9K�`�&���{���J��4fq�5�Z�LU}%��t,xz��3�^I�`U�2J��Z�N�2��0�W�3�N�J�3}^�tFJ�"�vb#jt��s�P��0p^)���3O�|'G����(��[��9&�J�r�zW	s�3xf�Gc�Y���t�A�����T)���*a�@�yZ�I�"<�K$��M&NBL.T�/V�H ��
O��u^���Y��&��:-��k#:L�\��U����9 O9��hT�9B�7�3�)�D�,�j��J�����O9�������������y�*���.���/��	��4�|m�mU���\3yL��N���yrN�4����r=���4�v4��!�p��ou������P<��/�ZO�XR�d0��q�����i�Qz�<������Cq!kd��j��5g�/0��Fs��O4�����9i:f��3��@o}�*��5����&Z�OQ��c?��L�0�,�#���d��������0�g}��r�����]tz9:c����\�}�_k�Q����P�U�������<��u���4�Og�0{�4����d��1�@��6aH�%6�?�9���}Y�O���=
����J�s�mk��,���)�W4y��$r�l��	�1�KR������"�UW�Q�_&^i(F"���g���k&�q��_����K�"������f|�\$�R��8�|��'�Uc�F�5b!�>%'8�c��y����t�_�[���M�}�+�����'4�ce�2�q������U�����k0Q�N�TFQ��X�����tr[������Zf���:4
H�`l5*s�9��~�������+	����:L��L1j��!'Y
���cK+a��'�uf.����bH�`L!���R�f�Bc�R9���������c<%z�K+�U�u�b�l]�������J}E����4%zV���QB[j�0�2_RkNU��h�qEp�9M���Vo���7��l=[9�\���;^'��t�g�=�\],�9qc����^$�K����E����3����@�u�W�`"�&�^UD�*�� ��)��F�F�1�e�c	!�r%d4�^)�:������WZ$���v����{<d��������4�y2,�+
��g^�b+�(���<u�m��Bo��c<�y���e�Zh1v
!A�����W*@�)�>���<����$7�J/��dv�.�����b�C�a�)�Q���R��$���u���^�{8�\!$����)�������#/��2h�k��FUe�J����)��f+*USRv�#=��B	�MZn�'�a�j�!zj�e�-Rl�xg��g��Mc�Ub���+�yJ�L�I��u����w2;����t��D<=��3DO��eop�m�c�T,I����;Q���1�=k�*K���Em_��o�����Z�
!yJ��	����������9I�FSkE�uvc	@_1%zBQ�#@K����
�����z�Z��)w�+�D/�X����
S!A���q%c����_���D/O������t�p��0E�m�{�8�J�<���?�����j���fV�fO�d�R�c�J;�c���������������F
*��B���v���u|@��z��/�%�Kt:�D��!��"1�4������p=�*�!��=����35���%�ts�(�����}f�dH��'����d[W���v�D���\�������Y
y!�K�.�ctX-����}�	:����=���K�	�=�L��B�����d�ar���F��������%�S6�,����{��^%�jQ@�������nk��2�|#��C��jG��N��i����?��Q��v��cl�D���CD�-^�Z�t�2��g7/��AR"�Q��Rg��&�)����
��B4Rf�������cs�~.������������{�:�/�n�_��[�|�@\��}s�b�l'��^}�p;�������P�nP�(J���<�����Gq�G�q8������s���������w������W����w������;?~�g=�a��������>gH�������s3�A�va�FN`:'�q,�`| ���~�@3}�h�����8�����B�x):�����&�3����W=��;���O����6���~��\���>M��}�#����yd�>O}��}���??���S�������BO����4�������f<f
�������HS���/��?2�Q�'|~h�s}�����/�����mQ,AU,��;���$���@�^�&�qx
�:�f����Ey=�(P�'|x����4��e��,�����.�CY`=O�>>t�|}������d�����J\���X�'��Ny}�����)��������4�y#f�������z���g�OX�����r��[Y�����[�u�fa��*MA����[����^��0��������4�p�T}����Y�)Q�O�N�������m��%�������i���!�3l6�b���	K�j2�4�U�����{��zx�9Jk
�G���6Ap��#Jr!�P�;�[���~��������5y�q����$���<>G�cu��GN<��I�o�];�?Cn5����S\����{�<zU�Pl?��1�c���c����������e�/l����*��[K/���_����gs�b{����BJz�c���~L�����X5�k����d�q@�5�)y~�^��<	�	��^��a������������<m:��)~��#>�y�[y���\J���b���O���l��n
}����
�\�b[���a��(�B���Gj������{x�y�;6N+��-�z��f������IZ-�I��������D����T�<�qa�3�!���J=�xkl���
s5\?����tv	����T�7	�m�o��������Z
>���y(�":�'Q~����<	���{��D��)J* �K���Z��s�dG6���~����'S�^�<~�����'���u9A��n����%~I���0$�9k�S<O����J�:��8�u�����%��S��Q2d����e��C�
��.�x�j,���p�4D� WlKpc~�B���q�K9=I��E	���]����7��@=zp�uP����I��=�������2
�1�E��j\�i�?qf2�-�����>�]�s\�i��#����4���!��sg.��qb{��u)J���<��3��0�\�IU�-+�Z��D[D7�~�j�8n��0a�C3'V���6r,d�����T��M�e[��<N�������aT�y��3]G&��q�GJ��I����:��i�?q�N%���}�^6���Tq��J���I�����O����P��[�&.��J�2�f���P6v�2��.�uV�l��R�s�,*W��*��7�>����\'���J�c�mY~!
Uu2-�:����>��!�O3\*�X�8���y�����J�[���(M�g�.(��p]iW������P��l���Y�'�~L�C�����HhA�y)����4� u���Xc�m�O�{���6O���Nd���Cu�a�a�������@�?T7�>-�y����z����a@:�:��m�l����U���D�15W"Nk�Z���x�8@��[�����<�i�?�u����M�7UC�4�|������S��3\�� W��i������UWk���^6��l<�u���f�"M�eR����l����������������O�����TCGIF�RE�ikH=D�x���2-v����1������?���~�.�?W�%'��;D���,��AT��qc�.������$��ax���.6�&�����%��[�4����\]���
Y6J�a�������MOP������+���#�6n��cN���k���:����ME�T���.N��0��]�J�M����$���O�9��P����PY<L�B�9T]��c�)�I�O���.�.Z���l�u]oi�������7	{ho���.T~��,���Yu ������5�������D��b�%��<^%H
j�&�\�Z�'��^�=n�F]����Klk���m.�����I��C�0�q�Xn���g��dX����6�T��l-��G�?W��jZbZ���)��q����l�{TW��B��*�Y#�%���V���b5��1�������2���~��3--�>���:�`�>n���0v����fvT������>����>�y8mN	z�#���}���-Z?,5��z!��z�KO��)�`��
����iH� @�T[&8���}}{��WC-;j��y<��#��_c��i�?�u�RY���Cm�]�T�l�r[��l�����(�i��>�0k@�R��7��]!e���Z&Fs^R���C�!9_�4��M��i�?�u}��J���<>�z�\��[����PU�9��
���\��C�S=`����S��
1i��Ue1[g����wZ�������X���n�x?5b�p�@������>�B���l��DN!&9��VX��<g�kU���q��q�����6>�dj�N�������,o-��=v��9�]U!GY�m
��H��E���n�1����n(1OLq!�I4��@4q����ac�|�a���T��N\�����-X�b"m-��1E��s������v������O.�\7o���\7�o������6o]��R�~
��z���	s�x��;j��B~��An;u�������hD������.W�TIR^�}���	���������H=V��9�k#p#�>�s���a��24s�u����#�����d�����9=�~���0��������r����<�q]�gW�&,���8�A�r�~ks[�a;d���6�=j�	��y�.��WE]�����'�=<��68(�T K������>.R���:��S����Q�`eZ�4����T�)��T��'f����s\7���5�Y��������qs����f������� KZ�����Iu�2���~�~N�*�X�\������n[o�������R������*��5����G��IB�A�����;������%�	aiz��]��W"���SI�
�~���$��I����d�����_��~p����1���Sy�B'-=��U6�I�zt���#���N��Fm������NN�#������c�T�^H��s���V��b�K��?d�r���w��+Ydi�� �<h�TE���6(����~~�e����C�u���akW�H��w����`=,47��Ho�������z��|��^�\>�������2�:K����!@[*�����������G�!������,_���S��,�C��D[B
��_��] �Hn~����������G�V����������}��\��}s�b�lG$�������w7�Wo������`��������]>�����M���7�no�/��/ow��>�����������_�I�������nF��s�(��y}vyyq�b@@0��q��77����������{�MP���������[�~���PK1���@PK{2K18sept/pg_part_head/3_2.outUX���Y���Y��]ms�q��_q�,�������V�*���8�K��J��
�HI�����O�����,���ni\�@�G�=���O�<��u9���������?}��>�����Wz>�������n������"��=:@���hwy���/���v����D��>=9��p�vw}�n��2�������?>����{���O����}q���0����<����<�����g��?�?�?��}z������=;��]����=��?��~�?�����t���������xq���^���|w���<�:��p~�����~����ywq�ry5�����o^�\��=�o���/^��]_���������W.���]\�s��������g�'o��o�������|��r�J��J�1avI�� ������������PRA�\
xxx�r���p�'u��Bo�?�������g������������N�_�/��W���)F�b���C�=���Jp9��k�X8��k?����Wyy�������^�ZZ�{:�p~�z�/�������L���{N��y�>d�������������(����;y��f�����W�|��cs���Cc�^>�H�<�B�x��7��������T'������[���g�o�9�s�X��������/��G�a��s%��N�G_��)pN��1)�>Q)����"|8�s�����������3P����P,�|�^n�[ ����\V%q��������|�{�U����E�&y�7*(�h�s_q��������_w�����.�������|���������S���S8�Os��|��������K}{�������@)�s>G/�~������Y������_������ �D��3N`�����\Z0���0�9	d���(�?���/�2G�G9�G��2z{B�S����k��'p���*�D���`r=�Z1��j� NNnd�����N?��8���$�����;&�W�3�<�K+�X���MS�������$7]a�W�C�7���F|�R9>�<�7
e*�-G	��
��r����5����/�������&��_���A�8�Y�-N��-��1['�17r��|���e�w��^]
�Q|��Yy���D�P�:����~
+��� �^�QH�&4~����k4�~���8?X�)���i��14Q�!�}��9��y_
��q�(C9>�Sf'���oQ����(��nP&c�y�(�8>�fr��)ME97QN�\[�<Vr'��N����1&� ��e#���I���E#�5�d�r���������!oa��!6��(��|4���_����K�<�z@b�q��$j��������
�������S:>��'��@���[ah{�r��e��F�����uz��C�<�}��1h���a���������\��z���U�t��&�O<,9H�����1	����D�����_�����������r��r�J3��E,��D����kOH��W	3'��La/�)���Pq�'qr���h��Qk��Zs��;
��?_R��Z�����4�r�h����0�3``�q�u�(/��0O��D���������A����i0�������!�����6~��D�M���($$DG�"��}^%�Q����<a���e~��H������5i��R��� �u^�Y"���<��j��^�A�j�
�L7���^�|F^g��%�=>�S(N��0���s3n����r2�(t���0��J���iLI`��$�P�w�i`*�����
dWL>_/�u���C�<e��#�\s3o$�Qblq.��NC��7��6�q�H#���&�_B�l��E��e�i$�����:4����0�0���N%�XX��t	7�5c	���Y���a�i�$�d���L�2:!��N}Jq^'�=���4�C��%��6d	���]��V���W�$����n��N�-j�����%��E��%�H��,\�o�T��:
����m�27�P$�����b����bZ��H9u����~bWjOW���I���t��u���J����+�4
�B!���9���Y� ���+���
y��,qs��t�:�9�40��D�e	�\�P�%k�X�yF��o�������7x����@�Ya&/0�"�|V	s��4�i}^��i����b����)�Ml�� ��i���l����8��7��n9��(�9����FYi�.u`�4M�k�d�v@'���������������k�P=��,�����m�8H��M���t�=s��~d&�`�E*�q3���S�@$B�%W���&BI?��cZi��B������+��9�(H6{�P�?�Z�J�z��L�'Xg�����m�a��\�9
��E*�LZ������b�aP>|���3��gX��iP@l������6G mW4)���6��t@y��W�f~*c�T{3��A.��Lr��N�9�e���.m7%*%s����������%����J=����Lg��R�S����O�
��0M����U�z���dF�r����fN���rB����J��� �g&{��7��D�L5����������j�0t��qZ�&��2��_�=�Z����T��V�9����I�J9H��p��P��-kS��r�T���1�����?>��S����/'h��$DQ�xE9JLgDI0����zL��)��Wi��D��wA<�#���LR=�������L����4�[[n��xv�
�/mDo��y��_,��1F�p?�s�s��s�a��E�A%�Y����Q�u���;TL������ ��69�h�����`Z3p�;q�g��\i�O%J��)�N��VY%l�`���JC9��������|�MKsFn��g	���'W��3�5�K�A�,��T9��f��f�A���u��H*ndw�Y�R���<3����l[��������qB%��x��v�h�t�R1}sB��Y��(�8P-4;��x
���Y%�:���(`m,t�SZhm�YIL9����p*^k*SkU�)�����%z����LB��6g�#���vl.C\)7I�C6�����C\��[�Y�9�������W�A��)����6�v�D����o�����t�g|�V�2����/ku4�o��h�5�x������-#�����k'�`\��m���J�<�p�1M�X\,�v�������"��U���R�D���g0P�q!��P���'<M#�T/8Q�N�s@Gu�����O	�U�!
�"������:��3O1��tn)�k�n���`��m�h�<�uf.����08O0&DW�of.$���	�U���n����u7��1��<��F]UB�����D�
U�d��%��W�p|��<�;_���:�~+��u��M&�J���p��8Oy�`lc7l&��a�;/ �W�K�;^'�BX����S�'��Kuf����z9���A�U�Q�Jk�W`�gx^f��L��v�:�`�f#F*�U�O@>>�S���H�����&�S;��;��6'$1�J1����</������?����O�^p#
�+�����)�c� ��1,H�����?.^gm�m��B��c<�y1e��5S;vS�u����h/�R�������<�y1��S�sY����qF�����V���������]*���k��������������� ��<�Ly^�9����b[�V1V�Z��&d�f�_i\�;��2�y���Z3%!�K�x�KkBM�������;�U-3<o���cJ���&��*��v��c.x��t��<��XG^�C���[5����������ct���� ��T��E��U6��LW�k'	�=>�S����B��x�P�N*bfk�1����� e����5�L����;O�V��W*x�|���1b��#�7Z�8����W���6s�WLy�=V�{��^�Q���
��b���
V:�+���)��8���a�B��P�������7�utHx�s�|���Mm�7���/%���(���V�}�}�	;8d�s��[6H������4TB�����,�Z;,B������B�.�i]-�%��S^i
9v�?�������`���������+�����5�,���3dO�������=MU�(.��]�L+mdI�C~�����5�C^�.����w.F�)�tw{��z��t/f]�0�d�����c$��T�:��x�ar�=���|O>2�SSl/mPK�\}�������*A���La��;{^b[�W�o�d}	�����j����������H����S<x�B��b��vX��������)�8�z}�d!2������&�)����v��R�Pf��A��~}�����cy�WN����^�T��H�'}ur}�z/��������W'��/v��3���og�8;a*��
�G�!q��S?dlJJ>9�B"�3<��]����|�����{p?�~��~��i��N��x��t��������|��������Y��z��w\X�B������@������`| ��Z?Q��>y��n�c���7GB����-EKR����47a��2.~���3��X>������'��a�=M�q��X�8���X
�%��[�D�Mzj��g������~��/���,��TpaAUF]J������<���,[���f<�^x��J�s������Y_��lO�8��+���b	@szAn���y�Br{��\I��i������}3(w�<������=�x|�b${���2��]����m�����Z]�<��cw���>?`�y�����rM�\I��'*+��,���U(��	�}�?����X��.}*B�j`�qk1%R���,�9T��U��l	�J.7v�R�kf�,Rmr@���=xJ�������:����6������;��i�?�f��m�
vm6M6������������<���!�3ls�}���BN=����z������{��zx�9J��4����������NM�OO���y����=��4����H����s�2�����`]}3��'<��,��u���!����F������eUeH5����<�v�v@���*�+�N7Z,�����?�&�)[K/���_��+�gs����BN�ju��Pu��Gw�o��S�5�u�i��x?�E���������O��l��@��ce��i��s\7�]���|�z%~����\�$��O���Y������s]�1O���C3��+�Q�n
}*�����\���B~�T4��e�L��(Ag�3A���<�\wl��S[���d��A��z�r�V��y����I�4�����.�<��������Q",r�����"�`[8W��c["pJK�F��'�R��m���l����9����%�9/�<�k�t���y�����,�����hO���\�T�)��l�>*�
:�kj[�M��B�oO�9�k��*��\���.����|8+��7V��:�uq���D��:����W�OYl�z�D[���lu����5���	��C]�T-���^�g�)���Y�0�uo'K��-����K�R��y8l���~?�u)�*'�>-J`R�A;�k���z�GOC�k�x�6mR�EM�<���#���1�E��j\�4���8���P��,J�h�O	u����8=l��@?�9�0q�*�.gh���.f��}.�I?n.�L=2�a.�$�e+B��T��*�<O����:x��8�Y���cqL������P]�Z���t@n.�*�F%�����*BS�8�?Q"��g�~,������L���8��SX@_U���m�rU����=���h�	{�����a�s�9�� ��U�xgl�2o����n]��$4�l��R�3h�����M��l�}��:	����%�e��~Y�V�i�X~�0u����#+�p��fW@������Z�MEi:�?�u!��*�����M�A7?������<�S.�l470UBr)K��X4������7f�K�����\����D�l���<j�s[���Z�<��cuS�i�g�.p�< �gX�Y�7��M-1o���G����D�15W#Nm�z(�GqVlb]�-�<v��=M���nv��}\��<�y\���I�1��)w��q���ra[���M�����20[�e�G/[�����a/�������y�7����[7�.��|'a��SX��V���f;9����D�C)�u�j����:5��t�5�Qy�m�������u�q������9N0�%l��%�z�����V&�o]�8�M�!��i�~c�.I����.��G
�Y�an��#��77=A]�'Rcjn�,"��!C�G'F���o���Gu%�q�\�N|j������Q��k���i���<�������4�����a�J�K���������-[��B|:�?�u�v��D<qq��P]�_������7)���Lsu]0~�c��������b��hk����3�����
u�8p^��(�A���QZ&~�ZU{��r��uu�:��,f��x?��]I[�$��&a������C?�����$��[��{��su�`��!���z�C�����mm��c��J�S����X
,j^b]���<C_b���\s��
�Z\=��Um��6��I!u��������������W��E��������:�q]��G�|~0����u�V]����s
1�jS�j�<��(9o'F�l��������PkoN���M�/!�<��1���D��F�����9��^?.V	xs*�C%������;U�x��.��%3��6��(B��W�����J���F�"��_����&�s\�[5T��x���!���Z�u�VUg����s\�
���|��%L�53���c[���~�����,"������!�P�"�����y]�u;����*�EZ��f�IgGM~�W]�;����B�W���A�*����S�Z�|��3��u���T��	4�D:JRt�>�f����<���<�uS��C	�X�&�@��7��o��r�y�<�u��<)���'����$�fn�l�����?�Mej������������MOl-�b��~���v�����d�y��0��9��{p�<���l?��/����l���3{�����J�x��[���1.v�<��q�������)su�h��2��d�a���1�L��6���Z��q���\���q����u�w1��qk���z�?�M�c~�}����St���K!o��<$��A^��.���	�������'��(�lmn+�s[e^�j��H�)W���t�@~%6��6�y(��gN�
�n*��]s��?�%0u��5u$���?���lo*��i8�?(d���j{�B?��!�Y��n���b���uJ\1��1mn�w����4���E^��=T��o:	!n��g����������vyrh�� '�u
����R���������%����QB�0�7�_>?[s=�tp=�/2���k���8�r��p�f^Z��m�2�������k���n�x@�Vm�.@�u�������9�^N��w�Rv�����N�������fd:3���������f�������z�B��\[2�%�l*\��K��u�z���f�Ja^�����`��B�Xk����
���]�z�f,����*�z3��v�2w�z�
�����q�����#+�Spc�'x��|��^�\�����KX$]Ze"�����m��[����������C
��O�|�����ea�����8Vu�����������>���__��Ab����J_�\�����v�o���v?\�����q�����Y������w��{��Q0y�a�\�U3���h��?�{s}��n~��b����������~��k1���?}}r��(C,%�����}{r~~v�j@@�����w��>�}������3A�����(@����i�In}��'�PKmd�E�@PK{2K18sept/pg_part_head/3_3.outUX���Y���Y��]mo$�q��_��,Q�������9�c�!�'��-t���g�[�����N9�CQ��9�!���������S��]����~�������_������+=��~���v��������(f��9�]�]_������^��}�E��>=;��x�nw{�~���1�K1��ox|������g�\��r��������RvP�^|�v�����~8������W�Oo>����w�������������n�����?\_��w������W�.?���v�����������������W����wW����7����n��������������W�����q�f��f����W���0���>Dq����������������_}����{���Pzf�X���	p�B��P
~>:�J�_v���p�r���p�u��Bo�?��y���7�����>���og�o���������)F��.=0#|��z}������{�����p��J�����O��./���
��������o~Q�����o/�U.�������=� o2�+?���)�����/�[��o=�������c	�w�������=G�	�D�^>�H�<�2E@a�|�.|�QPNu2<��=�_>^��}�T<g���,�����7��]]\� ��\�OI������W?q
��������s!G��T�gx�����������}1���G?�b���V8����*T����K)'��H  ��E�Q%F����C�8x2�5��8{�n�n���/�o��.wW������g,��<O��#/]>\�W.��Z!�)���}8��xw�����`��1��r�(%~��s�r���ky���������?�0>��B�����1�/�fA��`����,(���C5���e�N�2=D��2z{����8���A��}��U"����`�V� ��e��q�{4��p��D�������S|���M"<N�g��|Q������a���4u���o��Kr������8������o���|#�#a�[.$X���R�B#�r�J|��zz|��~��|�WXW�D��J���5�|���o�
��:?�2��p�]b��o!,�	z�EOQn�j�P�:��'���
��<�a��0G�9��a��f�N[N�NO�`*�@�A5f�6�$�"&��bB��?.��:�8��������Q��`�B�E������7��
�JQ�'8=�er��!ME97QF����L��l�1f�N����1&�NC�.W[��Op�G��
gbAT�JQ���Gy*�
��d��6IF
 �/C�8d�r\��6��/O��.�T�@�D�cd��B$�FNS�u�i�)����&��.�!�������2�d��A&���tc�<�}��2��������E`��j�	+����*a������A���38�Y�8mr>i�����T�)���X�����o@��?y*W*��4�B����9�"���!�fN����^f������/N�b����4`5��Rs&>�����%�1��=��m	�Z��8�����9�3�$o���=��@���rG�sj'��'P�'�.$r�y��.�Af�&�J�&:b�mD@G�O���Fk��J��-�>������s����#���Y�����f�y�W`�wrz��*P��K�
$���r�,��	��&�G%�3��r����
��������M�L1�
���,�����N(������@����P�w�e7f�Qf����X�?���	���y�z��m�����r����,8[J�VZ�=x������/��E�9�!�D�{Mp�*0Q���I���fJb[���
Q(�����6Z'kf�U��y��3+�C���0l���YnN�T��	s�
.�:d0%\�-75���9�cWB�N�������q
�8�������@M@6�F����s���g���7�PN9�)��R���mF��N
����8�^%�)��<�OB�kYW��y�	�`�q,�!�)�`^�5s<}20L	������DefM�,�'���h	��)����7w��!�C���1��������y����lte�W`���!�_\P��^�|i'L�2 ����������*a�]��4�/����4|��� �������@h��N����3�;Z�	�oN���>f�r�-��]�=���4F�:��0
�kR�G���|���$�OX��3g���;p�)��!{�@��8�t�i��i{��9��s&�R�KP�m��3C�f"�B����k.6u��� �D'Gx&!�P�G1�4z$<���'a��KnA?�5bq��+����BR��mJ��
����aF
m(5�!�`�����"P�����l�F\�w������!����@���H�1�]h��]c"�q�%������>����Qel:
�����"��X)]�e�N#�������,B���T�5��Z
��N�R��@+uN_Dg���]�eG�}�7���f������e�0S�<��A���*P|I����
-%/����E��So'�aRF�i��:<j$�@�9GO�h:P��>W}����������t��^��������p�"���)�BXi
��'q:�A��8Z3!�m�3�����i��G���U`��@��a|s��o&�'w!D;DVaN+Mn�Mkq�d]Eo�aNb����r��������'P�Dz!��iM��]G���
�]��8
0���+0�|z��&*��\l�k���X�8B��Z8U������wH��i}���"���9da%e�El;�o�b;q�gT �qf�V���PC ���5�h2�L�RB'j���9�Lz`�5����f�A�*{Q:���\+�K�a Zz(�H��cI��M���S�!^#g/��R
���������f�|bp���P�YWJ]�,�i��z��V�@5f�B��s����]|��%�P|;���T���J�=t �d`�3iw����k������LfNxo>�Z����i<��Tt~\M�P���fy�bI�?0=>W*NR��`?�8ev���u�c� f�8/7���9���A���)����~!q�P��1ar��(���|�a�8����;����X�<N9�A��N�1yV�#7i��?����a9�����gM���C��v��.5
��;:�y�v,��G;F���
�u�x����(+���7N�q��X�l
�Ql�D����c���Q^��:p�`�I�nm,���S_���E����n����9t��g�b,d�Vq�4�&w��(�l����\���Wcp�`����%�\�!��v�b6�w�yj�����c<�y�K��U��������k��s�����B���y��8i�Jm{�����x�LY���������pz~��:��8�r7l���s-)w�-F��J1���<=�S����Sma-9��"�%��2��t�L-�_k�{`<��2�]h(���G-���73�h�3���c<�y	K
!�B����j�1��z�	�^)����T���hF���?�����"w��n�+m��>�����c�!��F�)���B�*��D+�BTO��T����lm�&{����[MG��N �O?�S��e�m��^���G=!&1}��+�8���y&�������gMM�:�V�������������2�yQ[����Sk�naW;&�_lO~���w�ye��b�!|u2����b�(�d�4�,�5b�}�E�eF��7�l)�v{�b��Bl�3����c.x��t��<.���w����ct�T�5���yk�>��N�����cq�uc0����{A�1�$@�m%+]��D��������w@����@�9S��'3:.�Zs��!R�:/1�\���]�����ii�l��J�_0@_1�y�
L5^A7zc`/�ot�5��K^1�yI��8/���3Xi�n�
N6�cXi��`|z�W�:Ox�Ku�\�vM���&2bs��*�N��C��L�f�4���/��Y����!�J��vp���2zu��0a���<T�r���_k�� �H=O&"bpINJ,�d3WS^i9v~F��������c�Zr*�Ck���Y�� ���j��r
�f2�dVK��t[A{+-dI���i+�T��:����"���9Mp�+�J��g���~*�bvQSN�-w � 4�d3�x�4��VJ�S����FS����0���������5>�J���:�<����������;.�uU�qk�!���S��=�!��x_/'O�\�����tx3��Y(����������9�Y�r:��|s�$P������ �(dQ�����-������������N�Aw)}yv{�f/?�������7g��_��(#����v6`�����E�p����x����f4�cL,_�I�8����������������wJ����������/v������;?��'=�b�����|�H�X��� p���63
]��"p~Y��?<�Z��X�'*4�O���%�����/��'�6�3���cx���gtW�
�z��r*�aO� y��O���4��)�k��b�6��>���8>&=7�o��@_WGv@���w��#@���T��z�#*��y4����||b���&�a�cZ�E�,�n��3������y��"�/O�o��	|���x�q1����tc��z�zr{^�\�(��i�����������;����yI{^���=I�4��e�3/�����Ff���9���|z�
y{���G,>/Z|�\5\�Py�dE{^������i�P�r�n.���I]�I�Vc���[��zpJ���g���"�LJn��GF����C�kf�,�Z������x��K���CoDx~�1{^�O��������C��~�v�\����������-1��e��C,f��8���5��K1�t@��h�}�[��r���<s����~X,�h���X�&�?�=����~�'����DF����I�9/gF��`�����g6<��"��������y��i��d(:�&����<�V�v@N��:�t��R�S8�0�������B�=�0#s���m�1�������:RE_W�o��S�5�u���'ZB���329��������=uJ�=-�qN���������qqV�����g����"���|���.T�)/p��q��3��Ku7�~(�����Z������CkXng'<s�=/}����k��pZ�O���G�S���y�IJ-�E�O�����i�?�u���y0�����/�A�wE�[S[1�P[8���cY"pJK������b2�.o-���<;�?�us���>�e�3dW������g.���E������&�3Z�R�ejv G!��Z-0n��(=������g��0������2Q�yci���C^����a�/����l�'Ln+���}[�>�i]�2�07a����'�\�'eK���Q�(�k]
��[bX�������0U�L��S�OsU����D�G`��A��5���jB���+�+��84m����$�\qc��S[!?u�=M��gq��Z��(.��AWK�=|�����=l��@?Q�'�1�R���G��2���������y��"��G���g0Q�"rk)����'���D%��1�u�<a�q�Y���c�vY�l�s��u&�����8����Um
��M�������y�3G��"����sF�i�?�8u�s�<b����k;�����5�������E�OO�3aO���8�k�5f�����8x��}����7v�F��n��$4�l�y�2������X��K
ec�S�=���$������jKk�}�5Mgd�X|?`��3Z��Xa��y�q��K����E�t(M�g�.����uy1�8�}�3���Z�������S.�la�a�-��,G��*Z."���	~c�b�a�3ZW�k����D��j~����}�l-{^���sS�i�g�.p�< �'-v����	�������S��i������SK��Y�x��P�l�O��cO�9����~m��w���$�����;��8�um���;F���2�Y���8)��e�3Z���a/����E��h����S�[w��0W��g������t�&������=� ��JBp���b�����<�����5���������q����9�Nw�ha�wh"
���n� ������v:L\�u�:���I�o��
B�;�?���y4����8��0u����uO�.���57fhiG���<x��T����]Z�!����n��N��^��]*	1k*��ba�-�1��~��DN���s(/f�58+SE�s�7����~?�i]�U� �'.�q�CnKM%a����@=fo���.����B'����p��!��5��S���DN�R]%��<X�]�xa~�ZU{^�r��uy����,D�
���y��f��I���n���P��x���c4j�yk|�����������Yu<�,����<[��P��]Isb��,�K3���]����gc����>��0���}	�%�G�O�u��A�6V�(u�������Q[���)�G'j�D�67	zLG�9���������T�h�bo��eW��4������Pg�<i���6!������{���i�i�"�/x���]��9O�4�����Lc^�Ew3��������=�d�\^7�I��E�����'��b0�+��I]&B�����c���)���,_�)�?���w���i]o��X��p���,Z3�Jn��e��w�����;[�D��1��nbG|�[zcjK]����sy]0�ED]g����
r�*0�[wc|_n�|?7&�P������A����������?M��&�dVm���G�a�Yp��yB*[c�B,:�?���f�S�:M��8at!�����<�c�k�������/���oB��hs[��~���_7�k]��'���<�i��p����`\`c�/j������2�Q�iQ�}[�y�M�gk�}�=��yN�����b�3�]-��P6�u}���7��Y�.0�ri�Q�������J�?������uC;�0��hMU2Q���6�'@��<e.�M5U�\�l�4P�B�l�ak�u)�X�[����������!���3����"
zD��l�1c��b�D����M�b���M�O	:T��9�l�J��v��P�L��_7b���E�G�V��M5.G��8�|������
r���	�zx���T��VS?�kN7>q���[��$�����7��}%�#
ef�kp�N
�i{�B�J��,sZ7������,����4l��������c�wi�u�nE^�E{�a�(��n;�[��9���s���Ku�eY�:���.���qsy��%��+�����zc!=z9<b�^���r��l������h��\%����Gt��$3�0n���{l���e�#��xq$'����
���f��i�?W��l-���G���TH2������=*i��s���P���Q�~��J@M�ys�z^�s�7��2P;�.����?�`����^��XB��w>�k���%�34��GQ�FtQ�Xi����
���]��G��,��a�kJ.%3�qkW/s��w.��*��`��_��Av��r4X
n����.�������8�6s�%����#��c�NaK����'�P(��=�ysD1�B

���lE�����T1D�8��<<�x�@N����������$��:����������h��� s���������v!p�����Y�����������I0y���@C'�����M����ooo����_������7�~����o��JL�����]w3J���B���O��.//.�O�.����7�����������{��"����P����.���'��?PKP**�@PK{2K18sept/pg_part_head/3_4.outUX���Y���Y��}]s\����~E�Y�UTVfeU)Vay��������}R`�A���h�f�&n%q�.�k���!�D����s��pr������������y�������/�zwu8|yysw�
`$���9�;:�����7��q����7D�>|yqy���������`��E������qoon����B��������l���<<�xL���h:5��L?�_�?}}�����/�|{u}��?���{po8���������������_�
~��������������������s�����o��ps��x{7����o^]�=�?�_����p���?^/��M������V�[�7���(n����x{���������7?�x{�Q��J"��8)��?��e��t�Ood`��OG,$���t�'u��Bo�?��y����W����������/�_o��{�=���)���(�����d��)d�|4#��$���~:��F^����/o/���������������+Q_�_.n���Y���4�n���1��Hv���FI>����N��}�-�����%��W;�������'��!"���s�D"���.dR$r��.�G!�?���N���S<��}��|���x1e��e
�?\��>�����r�!��x�Jo�>���s3%����&���R�Ogz���������? }���|�g���7������'��\V��X��O����#�� ���������;}�P� N��@}��+.��=�=���o��^^\n�?1�O�~B�18�NO�<&�����=2�][�'2�</�;��z{��wp�w����+(%~������o���;y�������W��i�a����a�;����\�0�D���r�r��>�O��9E����}���\j����rDA��|x�TK�P6��;��	�3@��1��������!�6=d~��`p�p�������Q������w�6�#�R<?��t�%c�+��p�(��$�R�����-a���B
r�o:b�[0�NW:!�C�Pe��r���o��4������4���z�����g8�C�!"R���M�E2�����2G���!��;	��I���	_�D����R>;����8��<�b��:0�X.������`����=of�9?�K���C�va�w�E��mQ�*�q�0C9?�K��9��`�����JtA��K����-�Fa�p~�
���c�
sH�3
"P�b��o����&��8�����r�fO���T��-�I���8�B�9�p~��j�����,S�2������I�����M�S���������[�j���DD����`���6�u����y!�@cj���[I����f!.)�@6|.�m�������q	�]���n#q0e[��g����s��6�s���q^�@��x�)�0
��"�� �����������_�a��G�R	q��<�M+��Q��E��		��"����y��L� c7l��,���DD��m��6j����4�B��e��k�O�s���%�6I�I��k�a�����K(L���7��+����S���Ph+O����^�i�l_/[#t!����a���rF�b��M������
����7� +�#��*�q�1V}9o�
��a��R
s\�@T��N����!�!��i�m����9�/,e�8	q����.oI�@�R2.p020nS
�4�2Eh����:
����7���:
���	���y�zGP��x���43�j��&�<���&aN#�6T�8�J����m�7{-��**���6��s#�B�f�����T��c���^���6ysau~���DVl�@}kN���VS����^�����\�$t!�������
�����@�_q�X�F�����$t"�m.PS��0���@����fy�6}s�r��.	$+�=��%tY��
������t�y�57���\��]�S�
��R&���B��W�L)m��S<2������f���u���fO��xt�=��Mk�<���.$m��c!��-o����Hty����m^�����1�_�����~.�HK�i��2_?��9q������4|F�V`�����*�aq�N#��M�&�D.����%�$��	��+P��L�p�h���@Z���� �b;��]�\ ML�y�m"W�FS�y@���@?E�����IL�i���3���l*o=l3K�`q�F\�q�Q����ffTF'V�0��#�h��p@�	5����sP#f�z}8�Z�4�@�C��U�a�%��/����vY�l�Xjg�����Z��@"��.��M��r:�56t��k��/�k���R#Cv���(n��9?�e`_k.��R��������p
��������y�0���w�Q�%�Y1i*t���D�8DqlZ��p�N#��Kb��+$!��XB�7�kN����)�"�6��Q�q�o^6���U����\��.�N��6#���$������&��i�f��qN:�
t��i6P-V���M��`����l��k�������oFM��0
;�)�FKh?���%Gfk�"X�a���hy_�S�4��I��?��K�c4�of��fy��6*W�c0�m�hn�G����
|�t	��Y4���^?���6@�hyW�����Bz��R{����Ut�����@���`m#�y�W`,��L�*��\l�k��+��5Xt��*�:�(�m����OxY�h'jPL�?D��0?+z���Tj���mN#`n�@M��^*&�Z����2���L���:Q���������p�!�+�9j�y*�����;��J�8���U  �8�
�>�H�O-k9&��P;�g�Y*��\����,��z��O���SE����8����0�i4jB��*:����������qA��vm���I�����G*0�V��P@M�va����I���6��i�cx��4�@,8M�|P�XB�[�������"_�^��Qm�q@@#��,F�S������-q�����,��2��R9��`��|�M�6�~�h(C't92��e6z����)4|�RGv�P�<N�r;&�1
� ��"���Q��K;�]f�
������[�����g6�J`�w��-�X�u��8@���F�yl�A4� ~����U���X�[��a�OD�����e�d����NfP+�x�q���6����Pj�nT
��,��M�h@|JK�c4E\:K�_0�tB��
��� J�\�91F��I�������>�Xb��3:1$�M�{����R��/5�V���~]8mv�Ol��
o�Y<�C�K���}-EL�?fGS�yC}����p~���B/������Ou^�B�P���i� �H��q������qma-9vA.����7��;an��0������E�S��B�:&e��]1U�[��' �����PjY��RW���],�+;�KOX�F1P��J���3���Y���]�Nn(d�����SfO�g�y���U�+��EQ~Mm�m��B���1^*���a��N�'o^��� (N'����F'���A�K��K�|�n����cva�n��;_h�3
(8������J�F��k��J~l��/^�{:�}�>��+K����a��[K�t����"�����~���ze)�b1�:X�u��cJS�)��f�����"��X�Z:�Yj�����9r����\�x����m�q*������y�h`��y�n�]�*,��
���
t���:�4t^
����������SB�cSaA]���k���R��h�;|W�ym.S�F�S��+"n5�dB�R��\c5��3�����"�J���������	`��X��B�G�7��y���'���	#���f.x�R����:?$�o�`��� .��m������R��Y�;��U�:`\5�(���p�w^��(�e�Jw�{�����J,l�>m���������c��	��a�$�bbY�Z�z����\���z�����K��i��CuB-��y�!�8`\8�������7$D�w:�5o
"�k���Y$^@n�=��8���o'c�?�o���V�0 p�Y�T��',b�+=B�4��r����.t�0�a�R�E��j�	he��w�Z��%��vjK�*O���R��g��j��%����2y�[���$�bR@nd�t����������Z�T<V�UUMt�Nu���������J�49Po��K������r��?�L����������v�� ���0!�����nx� ���"�c���[��K���.����>��O"]��������|KX�������]�x���s�7�>��&,����z$Q�\�O���yA��BX�}<�3=���\���}u����{p7?\~��s����A��yw�u���E?I�������W�>gH1����+��%@�!N)�u���P��A��r��d����,�<z��O!���]!����,���������yn���e��0=��g��s�{z�#��O�y�=]��}������=srk����^��m<�����"������%�$_�@�~�.������8����y4����||b���.����x6}�yJ�z�%T�����E_�Z�lO������c?��^�!TW������Sy�v��<�/�?
����v�q�^^7��
���EI�t�O����w5B�&q
��&��9=sZ�=/x~�y{����X|^�x-vv#�f�$���h���?uV�=]���I������B?� �'N��~-U���7Nh��D?cE?���2����7zJ�����@�Yh�Ya�s��p����'�^w�La������5b����\*eO����P��	UL��9�lb�U�Xp|n��=/�����fC���������tm':	�����7~/�������7��ku���y�����!=�?����>�'���E��k������S��F�1<	,������x��E���]����y�����ue��Bj�P�q���w�y�$u�-��u�����+�Z^�}�r.U�������h�\�<s����}r����C����c�~_�s�j��u�T��G\C
3��*��������>�S���C?��.;�E��g����h�L�����c��@_���ok]��'b^�<E'i������u���������u��*�W��u)9�5���S��y!�����u��iA��3Y��LEg\b�8�YJ-N�E������������r����:�5��	� ���j����"�P[����sY_���n]E?��Vt���WR=����u�R�9�s���xC�&�V��n���~|��G{��7�nHu�
$��(�42�v2���1W�B?<�4��.�-�k��:����O�o��w����uC+�����zZE_��p*f0�L{��"��}li]�4�48�s1����M��we�Q6lk���8�ie��)�t����	w��)����*�R�I��30t��V��0Nc�qw5
4��[����k�&�����Lq�X���D����������~�q��'\��������
f�����l��e��8 ��-�Y��va��_��-�'����>��'�.�L#"��b�d�lE��jf�O��U�
���3�	xj0N4��@wr�D�O�)t|�uu(�����8�o��QU[S��������b��0���h��@?�����~�qvT9�g\Y�R�4][g�w��v�8���&��������^��z�����oE9��������$4�l�yh������tE�7�-*;C��~��0����D��������Fv��������`��
�DZ�����t���\��(���~C�Bvs�F�nZ�,N�)�BUm!<s�=/}�e@��Z
sPghA.e=������y�����)������.��}q"��?EDs���������y���N���~lh]-��y]d\��;�-_t��a�)�L��'����~�k�2N-������y<d�����.����~K���~?|��!F��L@�����������\������	<��7jko�lG�����Eok�1��ui��&m��y]���������J�\�'\��%P+�"��8c��$�C�b��\�����GP��H�V�U�E�����~+�K.�����Yu�"��k2�����n]�<��mW��0q��E�HC�A����wv��\k�o�u�p
4t�l�8���,�T���uO���	�t�����_���Pm?�\^��M{��`�]����u���O�]�U��u��Q["���~������t�oi��B�bH����qF�G��9�c�+�I���[Zj-��k�3jfQS����Fo������.��CI+u���5��g���������.�-��u�8���	�z��x�s��{U�y��e8����uk�nf��-r��b4��M�"�m&�|�h]ni]v&KH�ZEq��EG���O{��!�����.�ni��Y,�qR@&����~�]���o2�����k��}��gg���~j�0s�o|��-�'���+��6�v�����8S��Y��Q[+���I����������#�#���M�������&O.������R��Zb���
:e5�0��>���D�P��������P������V�&�D_\2s���y>=]��Z7��`)��8���)�o'�nJ����Zy]vf��N��\5�v�y[Q����8d"dji]mD1�TkS2�x���%���6����E��u���J�����>�a�bo��e���UO-�v
����\5��b�B����B_]������.��b����u����q���3�/����;b�j]q�+�g���������e��t�oM����
]������T>����7��~�����7�J�i������l�-�-��#v����egr����-�z�'��c3Y�aw��yD�nnk�P9g�l=Od�d�D*�3������l*s������4�,�9����������v����q?���$`�yv�u�����?�������<��`y��0���',#�~;�;�$���8��~��4t`�8c��|��yJ+�M5UN�����?,��n�4��m�E�Z���nqu_h�����`��e�RS�[�!��HCi��Js|?y���� |_��,0�^�yg��0�����.����~�'�b����0��[��}[�=�*��8����~����<$�Y��;�<�<���fS�eZ�� ���;MC�6{W�)�ug�/�4����?�)��4���L�u�!�oK���
},q@����n������e����z��$���fo���t��U9����l�0�V��!���g��[��������t=l���HT����.�KC����oi���7����J1��r�#�A�7��e���z�V%���'����RNu�(s�B�]=U�a�����%�"�������D�m�����~r�������r��^����[%i��X��*i��JZ�-��M�tN��h���?�W�*�����|K�R�z!�����������T�t������\����Y�RR�:pp��Y?�����6��P�����v���
�4E�H�?�����r_��4��m%x����C��'���#���z���3��>
q>�To���=���`_tE�����-L�C����-���
�w��$��H�|��#l%g?�������^����_�W�^ �n~��������p��:���������~�p�� s�������_�R~���,���������#o�$����i�<O�����l��?�{sw���������W����?��������I��������c)�`�������W�?N���n�����/�������+A����&�Qb��%n���/�?PK��r�@PK{2K18sept/pg_part_head/4_1.outUX���Y���Y��}[�7r��~E�Yz���E�r���z���w&��'MuP"�4�Z����	�S�sPU���4j`[�n�t� �_������������_�����_~w����[_����������_?~z�.|p`(S~��5�>>������>����w���}��������==���1F��zc�o���������W,�������}|����o�|����~�'��d�=��;������������o��=~������u�����?><���\>�?.	]v&���w�����X�
�K.���g�6E���?r]�S��s�����������OO����yw��
�o>��������������W�~������?}��Zo(��!Ggl~�O���?�9�O���`��{#&D����'���������/���}w����������'o�?Y���������?�H��[D�1�Hh����.�t��O�,K����!9���wzV=<���O�����W�~�����W���+�������N(t1y�g>�@�;4���3���E���?������/gu|}R�Or������r9��Y��u���������������_�/#��;k�-��_/����w��`���T>��=������������$�����������������$������I��������:V��"�����g��6�PW����4�,�a����+'.��|'D}^�1�4'��N.��Y��c��1\��/k�`F�\�@>�6���!��'$��*�H"�&�@�0�����)�`4��eS�_�a8Zp�C�_C8{�p��u�Mu
�^��kc�\���� ��1��r��I�����p����W��`"�����FD�u����t���bt��u������%59O�Wx��^�p�uv��%v}�fr��
:
 _7���f����]��T��m�� ��a���~�jf�|3����o`�������z��&F6�N�m�kH���^0!z����b�$;^H�$��g+�#�7�������v@��k��������_���M
�,7w��k/�E�zFm����<Y\!���i��k�R8(
t�e[�@���M�u
	����V�L�5�"gQP�[a1h�>.�P�-VZ�{m:���	��p�6�Q#;���a��8���F-�5"��m!3#��D��)b�_	�����"�
����`�7��2N8�G2�(�������q\7�P��8n�8N%q� ��U�n�e{�K���p-a��G�1�����N�Y��B�QD�s�$1�n�����b���X��b��J:.����/��O�����H���u��� ���"f,<4b���n���-g3[%,��3�J�>����g�B�(Y��U�����aw����{�RD�Wj��F9�6���!��Rs�
��������$�+�������������pHy����Rt��
�m��N���z�kH$S��_�N�f�~9n!%>N�� �gL7�������_����.�dg1��hB��v�P��HnY;�SM���@�4L�&�/���NNR*�6�T8[���x��F2�_����w����g1A��J%�y�Qu
����s�j
�$��� ZV�d�R��E���)����vr�F2�-nN37Jd���.�n���*�'�nIFgIK��S����7qi!)���<
h���
��:�/��f���������F=�5$�s�h�
l_J����VP���2�[eT�d�&���D��H�7h$_������hF3v�eW�z|e�Qu��dh>~�L,
U1`�(��b'�7�,vrD4�5O�=�����:��	~F}��
j~S�i�Qu
����v�B��t�q�D+�vYnM..�=&�s��z
��p�6%�����%|��d[��������D�n������l3���y��-��(2yv�UW��c�������������N
lc�o9m6j��!a<e�|L�6�#10��.���Qq��ngbR�vh�5�1�J���k��qjE�8{�����Wv�P��H�2J�M
#�:����j��V��<��S	{F�Uz���7�����s�LmaZ�������6�P��8nmx)�JAl�QJ���hXX�7�x%l!g������0�7�����3%7�����W1��f���-��%�W�!�L�1�_�"�u,M�j-h�Y<v9� ���=`�u�)E��1�)BPt�l�B]C�X��,�5�����~���=�DBcb`D�)�j�<�����^��	3��
����de���z�kH$�����TI�,�-�b �4�8�,��B:6k�;������H��z�m�3���,�oR�u�m�C]C"Y��yi5-�oRg?	�"�d���Dk��&�v��c�]Q�?��y����mF3y�ll%����z�kD$���s��Rk�Y$���1�����A���V2'�����po�_w����aG)��2�� �����o^e�0�
%L����i�@�Y��}�����&vp��������2{��Lf�VD�dv��U4��F=�5$�U�^����c�\��a���Q
�%G-Dx��p�Z*7���@��A)b�N�&c�����z�kH$��P2�Jvf����Fn�++�O�
����"<2�)���3��@3������P���f����+��x
����G���	�z����@����c�;�Y��]�'��)=�))�z��������q�y����5��bI����
X*�,@U2�.�7�Z����w:#�����JR�����F-�5$�5k[Y�,�����I,8_bh�H���
>����g��&��"fXZ��.�������FD2������HfkAQ�e�64��R2$$E�>�������1��o����1���s�i�m�C]C"YE������U����W�h��U�c��t"����}�-4�,���>����e�G�m�C]C"�E��!����8���Fr������os5	����7�����^�sz������>m6j��!a����\�)�-�����=�����E��]���q|jY���z�6�I)b�R2��J:��6���!��&���F5��g����[n=%��$�����c���t:a?����}�����A:��f�������Ga�J2#��Xj!�w/�iO)������}=L����5g�&�8��~��4�z,��Z�kH��z����,y�F��+�!�VBS�<P�*�ks`3>��q7�����J=��a�<m6j��!a�Rz>�pK(�m�y���N������>���'H�n�M�����"f���W
�������D����|�fI|�N�\\#�Gce���1P�����&���	��z�Ls��"f�����PZ��F=�5$�[b��!8�e���b��`,�|
���)���b8��w��x���@�N@��S�p����f����k��A�SD�[�H�e�,t�I�����Df���8&�?�F}M�3jDSSD?I:a�"m���z�kH$�$�Am(�S-���^a}.��P�B����+-�!���<���g1��M�~��n����-Ol(�y���V��M��S-� T!r�CgC���e�,J3z�M�KH
��f���j��]/9P�6)�^�	:J�����,��4��&2u�p���(b��%F)}+��z�kH$��O;C����EZ�V�JN����a�y)�Fv��"�uR�V�RZ��,=���+��Z�kH7��,��%0+R���^=�^���)�a�<�m!{_�ee1,�1�'�J��i�Qu
�����;��g`+x���yN��=��M���C��qR��KY�z~���Jp
����6���!��R{�_���Nnq����'/���H�ph#���=����g1G$�����������J���\��cbc������g�P����If��$�?4��
f�R�R�����"z�������w�P��Hn�=������kvV\����9�=b����]tp��p�8r���27���Y��y��w�P��H-�B��+�P�]	�r'K�d�������2
��H7��s{��^�P��	�	�d�=-��z�kH$�����N������J�)�$(y�^�\<v��j$_���y3�'D)���e����f���*��Nt�d���"`gne�)�I�]����N�A�� 9t�=�f" ����u����w�P��Hn�=O�o����<���
�X�t����jHg�C����e�����v�<���F��z��P7�P��8n�p����#�RRcc-���l��jO����&H��72�_����ue�R���lM��S��6���!��[�b���SU
y�qW������6����cG��
ZOC���fJohE���Q�uJ�i�l�B]C����������t��e�����Q|��6rt�M����o����y8�����"flddO���\v�P��HV�p����>h�����#����B����?2�������������J3��}���Uw�P��HN�J3�t��-��+�v�92�C�ra?C���c[��n`[\g������"zo��He\d�l�B]C�8+���'Z�i�7�eZg�$,�^�P�j{<8�w�D-�s{B�x-�2�E�J�E�l�B]#�8��=+�QX�U
���W��Jb��S���������"�k���}���d��I)H.��Z�kH;���i�B$��&#�E9�Xce2��25;$��s�{z��xz�hB��S�X�%�\7�P��0V3pH�h����N�J�BX���b.���Nv�����������8eV��"z=H����w�P��H�*j���L�����
�[2��$�c21�t���Yh�����dH0g����B�*eu�Qu
	c��'�(n^�jf����WD[���Pn[�2��8�n�s2���(bv���^�m�C]C"�%�����?�iaX����dv4��B��.�|�k��lZ�E��ey�~Ar�m�C]C"Y����k��RA�\+r,�P
��.Pi=�c���3v-{!��Zs\��R���-���Z�kH��_�'�C��G��b*D��R�)��<������yR�������9=�0fDQO��Z�kH�~=�v�/1���)F^Z����#(Q{�AM������;"�lMk�S���H��W�������D���'UB��-f���r+�B`���3�Y��9�E�&r�Ld�X-�"fj�����6�����l���\G�b�����s�V��(��z�e��&�8t��-�o�^����k�?J3���\G�|�m�C]C"Y%��3��-������\
�Ey�P�1x�tl�zn�����o�V(�1�&���Yw�P��HV{���8Dkc��!��n ��u��;���S���-�V�����z�Pw�P��HV��dO�o�YU��]\�(_
dbPDC��K�@���e��1+=���|����Z�kH��i�O�EP0�+0v��I�f��A:#���a��a/u�.p�q\']����F=�5$����-W�,T5��gZ���3QL�eC�?{
5��K��w���ZZ~
�����������K��m����k��P�����Zq%���"����mX ������cU4vd��~u2A(QO��Z�kH�n=<O]`rk�����e~�����aNY����1���oi�Y�
(=\�!�mL�)���Z�kH����4�'����'�II�B��(S�Z�`<6/�`�B�����4����t����f����S/W���
�Dr^a|2PzC�s3����pl7�|����K2dFs�V^��f���Yqp�7������
8#�_,{���I�V{)�d7��r{�^�����Spf3aR��7����0���������"�,��u�B4����7]��/u�Q1����n�s{s?\/��M����e��i�Qu
�����N��g�x��)�J���9C���7"���%��OL��t�w�u�iE�U�F�:{�l�B]C�X%��2��('��&Y8pk�B��P�O{0��Cc�C���-�����[��R�L�3��e�^�m�C]C"�e����y�����M�k�. _���9����������2�Bd�X�,%ma�l�B]C�Xe��@��Bf��
�e)��{l!�Lj8���=��n��k�MKd��QD��p��E�l�B]C�����#c+E=;�R��\�)�X
	����x�2�[d��u�����I��S_���7��F-�5$��\=���!	��
+�� �Ec���`����1B��q7V�Ac�z���B�*��u�Qu
	cE�����Y� �S�������h�\����*�t�&z��?�[���9�������*9�i�Qu
	c��Xh�8��x���l�7A��!�M.E,�*���1�`^o�f�����"f_E~�l�B]#��YE��y��0��l��-���!���M�G���;��6�~z�'i�#i0;m��������X�MI
������jF/�~��X5���M�����sj���7�5�;��P&H���j�kL$�����c)����������E#Y&����k�����F���	��$�4���=2T�!u�Qu��e������n��i��|+!	����`��w
_�CG��
���2��7P��RD_LN_�lTC]cY��y��2�$C���������!��YZ���M��Q�P�,��Gib�R����\vQ��X�L�%WS{���o�J���k�%
�gl5�1��
oPl���r{j6V�Do*��S�l6���1��f�A<[D~�&�n%I��>��pZvR���f���L7 �T�8��r��M��
�|�*����kL,�~���n����"�v>��r��!�Nla���l���='�-n��c�Z�3���T�x�������������oSe~s���6�eRY��'�!�j��k�(��Z�����L3���>����H�lTC]cy�-9��vd�S���0��;�g_���t4��'B8�u�@����U��Q�����i!I��0=��6%��?�"������Y01z%� �n�o�O�m�s'
vBr�3H��0�7�[�A����N���������=�����7�_�}��p��������e�_���������������7������_~x�(�,�������
!���}�P�K(��X*��d�I(u����_?����o���}x��t��G!y[���o/;8��p����)����:G�)��D1/�x�]*�E)g��7�t���c7������������9/?p�G-_��i�w����a�E��v
�8�h�����?o��?�i~����w��N�����_�?|(���������~���������#tzA����N�eV�=+>g;5i�����L}u|�"�7���F�/#�_Y|V��Z���s�)�h�(�I/�Kl�&��>�i�"B���1<+C���v��e0F��(�v�3#������Ed�^B������4�9X��C����S���(7����_D�a��,�`�����\m'C�����2D��>�a����c�C)���,�������}�� �&���!��]��a�p�4%
�B�7/C~Jd��Pj��m�!������-�=p�;�y���)����Y��;r�>�N�+?���K�%
;�����0[+i�s�1[����(^��'�$%d����{����/"�������_�]&eu��W��Qr�!"&�d<F�$���$	&���Vy��tG�P�V����C�0Ia������L=V��?�!l��q�t����\�d�Au$	&1�I�/��V��#@�v��/�8$.`�� �����;� >�DV^�K{V��/#�c����cQ��Ai7wl��g��<MC���=Nw�N��`:j�;�E���q�������d�@��E$������k�o5cr�����$J��T`K��[A*b�d���d�cL������I���s�:��2����$�9��2� �����������ms�[����M��1iGyq`��
v�tG��q1.Bw�M!tR�^�:�]�N7��8��!����]�q)����R���)G<�1$i��� �>G��e-���!��!��+���Iv�7�|	I�`qIv9����Q�L6-������T%���nUc�D
�f{����	2��k�|�[L��|)��B�i'j>�A�W� I������
��6��#�^����X8�]f`!�1�n������YI��8�2IJv�2��)*��h2h��-�!$	>����YI�>N�q�V�����Q��F���[�*f.]�c$���4�>+���q!L� �q,.b���E�b�C���8�����.��}���vKvfA�Q*�d�	c����f+m��f����3y6kT�o�z
Fc%�$��tF�/�1�I�;����qb�:V^��]�TN7��!�<N&��L<�J���#����t-H2!q�X�Y�!���0JF����<+�9+�"#F��-:)�r�

B���
�iC����89;H���d*���[�>0��2g�TKL0H��[�C�wu,�f��fWe9d<�{2�!PVC{I��!>�sO�L������%+����=)E)++(�3�{D������������6�wbIb�����
�-J�������o�� ~H(-YA��N���PM�cVYA�UG/����$�#�VRm����}v�/���I�R��+�1���t9�Q5�#����T�?�K��N>N�R������aI���&�H�I����=�d���6���@J��AbA�o��$g"��Z��e�|��am��%#$��b�$�)#H(�3���|��8��`+�]��\Ow�g��|7��i�_D�����q������� >��y������e������� �t��E�}�l�E�_^l�d�����
����$� �I`d���8u]�{,�nLR]�GdD����(� ��$i���s�"���v'��L�6���|�@����$]�v��Ho�8��bO.���G����K&���$�BcH2����j� �X�=i1��|�1���pQ%1�$�y��f��8l-���}9���p���}���3���$���
����1�3���S\��d���?������S�[����$�Q5���O�+���m�05�e-u�S�V-�=���99�*�����2;��TG�MA��g7�����g%��8lN��{r���%+�y�����y]9`�w�q�{r&��.`RVdW�n��p�$fx�^61��$[���=9��H{��q�b���,6�\��!#����c�{I���W�c2����Y�E�5��Z�6�ow���${���z�J�jz]t���U����-F�>�!$�!������JgM�'q9�K1Pyq<H�B���(5��� ����PY�����,������
��������'����(�>N�i��<Nt�/�>����HQ[�F�~I���C�7\�8��p�3��.,K2��f�Y�L�V-�R��N�ow_�&�*#F��_>�*&�2����a��s���������vO*��+�M�X%�Vh U��a���$��L�~V�W>NHR��KS.���;@`R� ���D�q���^����CP2�)y��Sy��D1�����)��v����m/F�U�"/Y@"��Fa^��q�c����J�����J�5,f\,:�H�"ko{����n�:�=��fU���~������h��#�����hd��"����E�|X�1�Qjx�&z��:���
,��LOL��1i1�#4��K��=+F��(	}����fP��`��c�QB�;D}b��('Tq��]�"��7<�1�09m��A(����!�c/�u+�b������+��p�>0�����c�����V��r7���
��F�`R*��8��#}��kZ��k�Z��u�
��d�����b����t����P�(�����<1�/���{1��G����|j�* �7W�_9�|��`��/����X�&b���a�=��q�KD��G�l����B�1,�o'4f��kFI/��C�1D���#3�������O�w#jU�#��Z�w����/��s\3x�z1�����"��DxR��? �^�g�U������\���
�t�7��b�=
|R��@ �Fq����HhBj��C��a1���0�S��@H�F%��������	Q8�A�o>�a�Dk��bl4��U�sI����F�D�A�8;��/��{1l����\k�L��O�uW���b����y&��*�1��mCE# ��*4��� vc�����{/�C�&6���5����V�i4�n��{1�����{/F8)T�����4b���r�R���T��gr1����������\��nL:l�d|�bt�Qw0xr��H([<�b!��P��C���q�.~O�^L����W�>+p�Le.t.����b���{/F8II�EZ	M��wp�MM��q�R(��[v�����i-�X�H2���h�R�����M3�^�w��������Yh��o����L*�vvgg�J��"��p�"Gq �aQ������GE��C^nG��p���y��+�������� ���V�=f�]!/�:�x��n���r������aze�#��\��c(\�k�����)c����DB�Q�R����g\��t^+��'�sX����J��l������G�#���C�T-��v$���?{2��3����z��S���2leq�����3�@�q�����{�{3WT�	S�s-�p���vi��������q1M�Rwk^!��V~�*���a�b�e
��hr�[�KOr�3.��88����x��E"C0B�d����}�����7������lM`O�������O��}z`��?D�Ih���?�:�hM�����PK~�,~L+��PK�z2K18sept/pg_part_head/4_2.outUX���Y���Y��}K�$7r����n"�/�(3r����a��A'Z/Y�)c���\
����x&Y��Hk!f!Yw���;���??x������?��������|������Mw�x���7O����/�I����?�`�������o���?���O�}����>�������7�����!��h�����^�{�����������?�����?�����W���G�&������q��q����S���x����w~|�����:��������}x��[�r��|�a,$r���?����u���R����k!{����hSD�+���)^��������?������?|���?���?��������~����]��������������>F�����������/���"�M���b���YU�0D������������|�������yx���������_^/�e�7�������o<����["�)������\��M�W?a&4.fc Qb���F����>�wO��z���G���?�z�����z�A=�l�	H�EKp�C���k����1f�/�������?=���Y_�T�������_����,������k�_�z�����������e�����������'����7���`����|���x����|�"��^����_���2>��J������^������?i`��Zl(��s��=tj�/t����F-�S?�-M<cX`�B0���-�S>��F�C��O��l6�sny���C���0�+;@�`E�jH)�`#��>l�B=S���F�b/6`1?�0���7��hB3�����L��h��`�p2�
:
X������a�������_b�{.�PZ�cG�
����-nDJ�6���^����@g{=g������_����g�a&C�Q�V���ghx�T��3���-1��E�g:�:�?v�%v��B�bSA��,p�tz���z�Doh�W����r`v]t�n�5�H\�l�7[���|�7\[^41�*���:��P�i��)�z��,����!�a���%�%�B��b<2xc�����&���@����[���6��3%tS����'���A$;�n�b	��7Q�C{����n���o���N�\�xz��m��gJ��]6�,3�����(|b�0����(�b2��\�M����h�k�����������+���Q��cg[��s�%eF�����"�F0.�/04D-���0v���c�h$��:S��2�9=l�B=S��������p���O� .��Ur�.q��7��#������]��,�t!������!��;=l�B=S��?&W��T�<��'G.qdW��"2���9�C����\��A��:5`�E���F-�3%�[�fv%J:8%gr�o����8������������;���u1�z����zk�b���F-�3%�[=�q�
��x���
;���1�IjXr)��f����!���p]].��q���
����i��������5������
�\2��,{~Tg�������M��k$�t�]Q�J���tH���z�gJ$�R�����cg�.�:D'�s���C�Q�-����0�i�>w]�sM�=tj�H����Q�L	�V�s>����i8��h��d+]mI\dp�h��CGz��"�ks����G*��`{|z���z���*�ev'J�9K���q�Y��.���������x�.�l%V���/�E}���z�Drnl�o���
������a���R�V
��iCr���H&��gqU�����+��,2� +	���F-�3#����b3�LNIbD0t�%��]���	� ��S��d��)�L�=�uU`��cQ6j��)a�
z�A�q�n)������aH"�RR�0�n�Q��`�0��=�Y�5=�q���PLP}���z���������q��Z�p�7������j������n������,��Z=���XF���F-�3%��"����=yI���)�u��^M�Fb�%$y�v=������c�*�a���5�zk�����>l�B=S�x�!!;��DO$�s��b��Q��A�`<5C��8�;�c����3K��V�Z�-I����Q�L��6m�B0�t�y�R�)��9��k_���7T�&�CW��6�����l`E����Sz���F-�3%�[-O�[� �i��p�<N9��'������:t�8�!w��<`��M�)=��q����a������g��5��b��"�r�A:2 q�[����Cw	y��a|]�FV������sT����P��0n�<�i��RnY~�[�q��J�d�	�W���s��?��J�T�s��7�d�K�f}���z�����C%-�n�L���G:9���S��A5����d���e<�.��,�J+z����O�0��P��0�V�sP@�=F4��"��$�8AZ^!p����d�h����l"5�����@���q7Te<
&�R�c4�Z`S�G��������v���y��2����i=tj`��GV6j��)a���8�
�7NH-S�a��,yc��
$��l���U�w`h��:��X�"��J����Q�L��p�8W��^�7���D��(�[��@K��0��^EXI/��Zk�A�6�r?M�L	c�x%�3��-2��5�q���7Q�.Jz#���?�4�������aR1����W��W�a��P��0n7�������l�,�0F��3�E�hO�V������
���������+*����Q�L	c���V&,K����{��`��������S���*�O��=�5�V�w����z���zf�1����+��,�Xb��n4�����0�h/7�,��|d#����.�&W���'B��@��F-�3%�U��}Jets�
D�1������`�*�{8� �w�y���B����X���^s~���z�Dr����A��<���c��R��N�	���R^�����u�����J}P29����Q�L	c��&���?�o��j�g����#y�Z����L�t������zk��>���F-�3%��v���Z	m;J����.!��_ �-PT�����;�2Q����lW�����`��a����^9����%�i�XaOQ2X�M�c�Q���������K�������R���P��0n���Lx��E�-g����
'A^d ���C�G3>��q��&�6����X����B�>M�L	cU�c?W:�����Vy�7���#��8tI�Z���a���������4n6���/��>l�B=S�X�`�|"]I�Z�"�L5��P��B���I���;tA����L����=�@�z���(�A�a�������e�G��h���������N:*��5�[8v�w�-�t]���Uc��C���K^����F-�3#����GC�.��]%{Y��+N9I�I�/��Z�e�=c����3n�o��y�E�z_:5��i��������HR97���>H�OHh���H�X���#������I�4n���C��$�����Z�gJ+,:
���4��h�}� ������he="}���C���*y���[����A�AJ����z�gJ$��6D����@-��i�HAX��Hz�������K��!�n���GA�a-�QZ6��F-�3%�I��9T���>_�B��$�J��J��e"d	1|<�����<���<p�,T�ZkE���F-�3%���&�R	�5����+��/�
d'9\L��=�0�����V����&���a������<w��)v���F��o��v�`�b�u��B���������<Gy�&����r�>M�L	cU�2��$��%��x$y�q���}��k����)�_���Td5���'�%���>l�B=S�X���W���8�X7^6D��>
aIV0�}P�xh�������]�FiE�z��X��9?m�C=S"������Zn��c���R[��e0/�k���LpJ�p��q7���s+�X_�e�c}���zfDrh=�xZ�� �	�&[�E�,_�~P���r�#9�a�c���K�u(E�qDF���6j��)q��y�L,�n9![�E9�GA�Y�m@*{d���x����0���9'��W�����j1����F-�3%�U=/������;J�*H��\���K&���1��i������
�=��O0N� ���Z�gJ�b���G�/�2��
���������I�qs��?�.i�V�.�!:������eCRh���F-�3%��V��\�/%��[k����c��F�S(���9�����?���6Q7�+=��z����>l�B=S�X��Z���&�$�c�M�(�1[r�������p���O��	aE�,�d���i������g-����2�w�&?���=v5��H]-}���u��N�p���e�&��a�=���G�W|����j���'V
���
���z<uW����h����A�u=�Y�c������Wl%�T6j��)a��{,��4H�����o�f�Bl�����f�����������������lCd��>l�B=S�8+�XX���e7���G��
��I���q�;$+�ky.F�[�n�Vd�J�P}���zf�qT�y6V��5��8�a��<'f��(o�������'Ab7�W�:V��;���S6j��)a�T�G�E
��O��� �}�l%��ey(�cj�4;�s��x���x��W���&��2\6j��)a���x����3y�����B��c�X�+(��Wl<�9���x��kce��"�����)���F=�3%�QAB�h"/[jPEc��?N�,��Zy�N�����o��A�i�A�am��/���F-�3%��`�����?z�j`���Y���q,����	!����y�K�E�Zk9��
���Q�L	�V�������J�f<�a�P�p�	-^��������|��{�
�eI=����� 6�%�_6j��)a����u0��kh���*���t�|3��D�7��� t���
��h�x7)��&��>l�B=S�X�r:;Z�=���[e��tA�d�l��CO�c���"v�<6����7��c��<l�B=S��
�9D����d����7���hl0�!�j����q�w�q����i�J���&Z���F=�3%���<i*�n1c���V8���/�6��=�C������;��)�
����2[�
s~���zfDr�-�&s����)�
G�V7g��"v���t�9�C7	�#���u��#�����2b*k�C1-O�P��HV=i��)� �y��X�C'�/4�0��;t-����D��@��6���������F/�P��0V�y�BomP#a6�1��%6�zC�-
I����Soa'9u�y���":���*���Z�gJ�z^�'j7!�o]������Dl����}��^���`|]��d��+z��Q��:�S6j��)aL^E>yA�8����1�X������<:����p^���]�8�q��-T����z�gJ$������*��{
����yh�����
4��H��*M�@�Jz,������j
��D�a����K	ef
���mY��S3��P����.vJh���~��lk�1k=�����a����z6���������tvl��rRv���v��C[c��gvKk�y�C@���&�bsz���z��q��V�+�
��,� ��b���R������)X��)���������b+�X����F=�3%��x^�~�w+4��y<������y2����+;��il���R0KK���*3�tl�6j��a��&�V�u1Hl���<���kM. e�)�co�~{�{��lb^�CO]����*H}���z�����d����gbg�q����M��f�^+�FS��uw�����yI5n*=�S��Pa���P��0n�<���#���TXLn$+��G�� XnE}w��
�����u-9�X���V� �\Q[�>M�L	cU�#kRe`a����v�����
C���,���U�#���LsW��_[��R�ZM:&[k�1��Z�gJ��*���W���(p�<N���������6�;t����N!w���u�����,+�����Z�gJ�J�,�-��1E5S���q�SY�+�&�=��C������y6-�p��G��w�3�����Z�gJ��<p�TKb��M%B7l����
^��Z�1_g�����0��sQ1:i=��z����>l�B=S�Xm��xZ	� )&�pk�_6��b�Y�z�U����!�������M�k=��D���O�P��0V,���]
�7nw���q�"r"�����v��u��?�[���9�X��z�
�2����F-�3%��d^`�U�X �vg"��x	�/�+TX6[�[>8��V��nc����*=tj`�d����Q��cam����I�X^����9���R+�xdk�r:s�;��2�1��M)I�HZm��_\���V�����g�j��(�q�����M������
��6J��4b�����F�dW�)���j�gN �Z�`b�� )/�1����M���Z��&�u�����}��AV�8��}0�(����'-�<lTC=sU��7�N�fEU�nXdq���` �EWI���p�Ai����
�EV��
Gx�G��P��@V�y����	�
(���b�
��2k����c��p�Q��!J���Qr+M�A7�$)���F5�3'�/+z���^hm��vc���,U�� fE�l�7�C�R�*�eXQD?#IJ�f}���z������a��n=?��6�-X�e#o&���M���wd �;n*e�-�X������� @.�P��@V�y����t�x��aUOX
e�Tr���A��9�
Hr������YRAZ�Z�`d���6���9��������J�[���B���=�	m�<��{�|��E������E����\��r>M
��	����d-�D��B4v-�04���R
t2������h�0�*�9����^
�Z@�����a����b��_����`bD��OL.���:���B�NH������P����������������
8K���OOo_�~��p����w�>2���.������7�>������������!���s������~����HZ	�z�/��
0�`
H����g��lJ=����������^?�}|����{k�y]���o/{q�}������/D����V����0z���s�H���e�b����m�/)����gy���y��S�����E����.������0-��|�|��T�z�����������7OOo���?<���������?��b�����}������!�������$t�����bK���DF�j//�L���!�j7������#�_�P|V�x-�d���,�d�rt�]1�h�VQ��>�j�,B��t.<+C���9�2��������`T�p~��L���'>+C��F�li�q��C�p�_���E��C�a��]�@r�a��@{-Cr|mZy��c�>�z��$2�������0v8�T���Hi�CvQ�]���w9d�D�_Pn}V���!��>�8�0��(��P�H�$,�,���?��2����a�p���"��GRd����\����A�+H��<�9��=��+��>�l��I����v��vQ�C��B6�q�������?�$3�2;Hr%~�{�+�o��Y��%q����ki�fLf�sH���B:�0)���*���$(\�B���g$��_|�@��Y$i�%[���$�ow��$m&c2q -���6���o��7�$)����YI�Q����v��;�q(�5����
&��$LS�/�mxV�},"������$��'�$!
_Bl��l����{���{������v�<���#��Y����%��d�@��Y$)��$��5�TN
�Q��&3H��%�?E%I�������g%�Vnk��'	F��`���	�>��:N��.N�v����$�G(�C���������$Eg �������=�n�+ow4IyA�F�p_zm�4��P���n?I
�?�7��8���Y�q��8��`w2�u�&��C��?���k4Ah�4���M(��q��9�Ean=�������dW��I��J�6��$��Q��`��S�N�4���vy��'��R���(Z$p�/��a�
�*�	��zIr|������ v�m��a�)'��[�D�lljqw�����]"��>�g%��4�\$)��Q�FRiX0�����u��d}I��4�=+�>�I�^���g��\3Rz�ADe'��9�.Ez������c�L�Xa�������a��iFn����$��]$��8�
_Pw7��3��V�'~D���3�$����n\���������{�2�QxM�N&'���{*��$_�;H��q";�*n�8R)�fY��j����qw��Zv�d_��eP�I��n��O����A�80KE���p
<+�5/��"�i-�
d<L:�%I�6�fIJ���${/H�nU�	����Ri]I�9���H*�I0I��8p;�q��c�BT������$4e�i
A:�����4�$�C~so'3,c}%�	�~]�"�9s�u���>��3�{d�������v��a{=��e��������I�E���A��^P��e@��v�>���*I���']f;�7���'��qT��=��J�:�k��f����l�7e���J�@L|�'e'�SH���w��Q���a��:���I�)E�r�$����/���N
����3��������,>��2�����4K.H��w��J��\�<��cJ,I�G1���#%���]�C���9s���� i��x7�I~q����KS���I�I�7���,���;tG�J7��QyA.��D�D�UZ��i�d��d����C�H}��-^t���%q�^�G�(�VJ�a��$��w�`�J�H�I���/(���NF�L�}�n��a�����+18�����M�#E�7���ZT�~���$�Mv����^Yn�?9��$$����T��y)�g�d��"�W�YB6I�OF?�>����N��D�IE�'�d����W�Y��<;A��;r�M�����iPV��/q��
v/��1Nf�wvg�����Bv��R[�-p��n�6��$�Y5����@w�w�t�p<�}���*g��UKq;�� � o��-��	��E�>f��/f���q^BZ��$�G��������]Y;���#\?�r�z����+u~O����v�v�P�������oV��^� I���vr%���x��q�!�/�����uwg�'���#���}aY
�����$�s��:�I�'�����.�/v�d�H��S�y����3�B��9�4D���8I�<C����'��������^�__zz����?KF�{`��q2xEgca<�@��j�R�>H��I�I�X��s �1N�i��:Nt�����`X�o����]��E�l�w������g~]������CIJ�ViI���#�����#� ��WM:TE�SN����L�������w�i��s��N�t�c�vR�L��"��6;�f�c�v�X������a'�b���Av.#�$D���[����-��SO%�_=�g��?y���ia���{��i�&]�y%G��_�8������b���l��*}�G.KA�t�Y�50�{�h�;�D��Q�[jb1��#����"z!�V��I��.�=�#�hRqUTC�c(/5yc���d�0�v��n���^�����F�K�$�,M�N���H+h��60�����r/������$��bd��%��������K=�R������6���
�b���i�:1��y�aQ�U������&�$��u�{1��g���EXv!��Z���<�2v�1ub,��Z:��[WL�7u����0������R�Q�,p�*��C:q��M-�:AA�+Y��1�Q��`�P<�~{Y����p�� F���� �>���
���pS����������!�8�K�������(�.�4lB[����$�������M}C��Cd���`�!�D
A�b��@c�`�hb�B���jd%�l��7��8�s���`����V�����������I�0�N���$�����bd�xV�F����:���3��^)���s����I}�k4J`7cr���(�[����a1��{����b $~Y�����l����J��4���/u����V�Y��x,������� ��`�0�K�d��^���bdC�j���f3I�J�'��g���&���c����+��]����q���&R�8Wmf!�2��2����>�AH���2�p��.�Q>&4�i�)6�s�1B�!��}���i[��odx��������0N3l�K�^^��8��G-�7
I#
�z���}�$�$W���G1�K��G?���k�[�Bz�d�q�.�O�QL��a����q��+/u����=����b�>�a�(���L�����gl�t�:uS''i��5���(�0]� ��7b�S{�t��N�����cN3�QL�R����cI�J;����I�6]�{�;;��RFU�q����8B� �I`'i�q1��R������8l�#v���	���tA �&!5�������zyy��N��x3�����W���O3,��.��J=������[�LM����bP!������]����xQ��7�r'���
�1��&q���c��]�"/N���,�z�{��*2�Lr���{w��}��Oa�������SiX.^���{5t����@���}4s�e�a�eh�{me�U�)�4MKT�gt�8������FT�\��c9�^#L�������M�kf��<��3
�d�H�o�;z���(Y��/�_�}�����S}��2	����_�x��??����'Z�o��dy{�$���tZ���_�?PK�$'�g+$�PK{2K18sept/pg_part_head/4_3.outUX���Y���Y��}[�7r��~E�y�a@�+GHk�g������I��:$�x[�Z���7�S�sP`7k�����!8�&Og~������������=��?}�/w�����t���z�������}���_�h��Fc�or�����o������������_A����^|����>�|��U�R�&g�����^�}����,�?����?���������/><�q�!���o�'�q��?���?^W������/��������Z���o�o��������/�O�����B
.;����]q��.v�f@4��|XS��g��M�=n|����x��V����~|��������
����`��������g �?}s��������?>���/�\~z"�Zo0��>@&gl~�O��yo�` ��!�Ddb����d�����|����
�8�������_=�z��������O^�2��������7�_~��Y�!��#�_>cL�EXO���	�H&x�No	=�;�OO�t�}������������^�yx�F������2���Q����`$p&[�|F��>������~|����:~�(�{9��������,������K�_�������������!���_�����������������C��������������y����~����C
��������|���U�/�����
�����We=������[=���oY6;�PW����4�$�a���|�r����|�%�G8�|�{�M
x>�	h=j`��a��x�
�x���n��9�|�/��Z�kJ��F���r� �G�o�"n,��^�����~M`�������o2�z
t�a6�1x����_���
+t-����	i����l'��c�N}��p]�B��N|�Pf��_����)���y~w�8c!�	�1��g]��j_�^�y�y]�A������������W��&��f����_������$��w��q�����1��D:5t��x{�zC��@�4��S�'����.5���Y�����������l$�g@N�
�����.������5�) �����}��kJ��]��[�����GBf����cb{+���S�>��t�]la��.X��'~)����_����
���L5�"\�.L|-��lcD���{����k��/�����������:�A.I-��Z�kF;�"e�^�/�26g[���=t������7��
5�S�8���������H�
=tj�{'E����N-�5%�[���3�:p|�R;�C�r6���m�lM�H
�������6�f���yC��7YL�e�SuM	cP1�\-����1��0�lT���&����(��3����9�5��\,z���O!�Q�lvj��)a��o6�8�b'����V���Y�L&��s&����|����C<��f�,Z���:5Dd����lvj��)a�Rq�GC1*�J^a�?cg���hP6RGv�2n����t8���(��Rr9(�1��e�:6��e�SuM�dlr��_/dP�E����
g-1~]����*�	���ih�/Osx�d����PD_���OA���f����-MR�I���]u��a���7J���3�.�N\��0�����l�������
�,�XRbU7;�P��0n);����P*��"�>�c�8���H&7�:!���#��na�����C��F�=f���B]S�X��2�%��%	MWGz��X%��|
����2���n�`�:7OR�z��<J�96*��N-�5%�s+[�
k_���
�V6����R���{��7/L����`T�d����qC���B��Y6;�P��0����G�Pi�"v��n�mc�����I�e�Z�J�g����S4�o��3*"�!�i/��Z�kJ�L�bu�t1�����Gg�e=�
�|��O52�5��:��6��@�-=��6���-��Z�kJ�F:2��UX�x[#��m`�����	**��'�1���9�o�l�m���J�B�����������=��Rm����M����!��1D��S�6��������qH����&��)�V�������A,�u�SuM	�5y���0A�B�V��2����Iy �l�����aw��������yC���!Q��������qk�KR�V������O���6�l~�9�R}U��N�����x���%�aC�mG��Gy�f����-��%�W�@��t���0n�H��ww9\	��7�w��m��kMwJ}�y4Qt�lvj��)a��xG�5���k�/��r+�
�PN���O]����m������U[Un�\�r��N-�5%�[�I�R�$[b*����6�F���>���8p�����:�]/R�����Q��P\�������������CZJ���~�Y��KR�]�&C�����������mO�1m���1��-0.��Z�kF���s|���{o���� !'#EA��=�Ob3�p���o��l
�G��o+]��pcT�Vn^e����LPB�j�Oa"��%�A�c6C���6�����	�(l��w����BmS7;�P��0V�xl��b'Z�"��.f����Z1���<���������f�,�"[Pz�� ��B�lvj��)a����j���5�	`XS!����R$db�X����@��Q7��k(^�����8i�Z��N-�5%��U������h�.^��D�gI�8��j<w��a��|R.��Cc2X��f����-���U�%�!��F���8�!���dH��IYe�.
�w�T����w:��������@c��N-�5%�5[	�l0�`}d0�			�#	WE3*"�����x��`Y�h����'J\CJ[�f����A��'b�19�J�cv��
d�I����6K��]<|�"t��������&�5 Tu�SuM	cpCk�a7�n
0&�6*����	/���D8u��]���b#��z��x��Nnu�SuM	�p�^�T'w�U����m���{J�;TK��Y5��������
��>%�������Z�kJ��31�Q(uk�nc���s�����sd���a|r�pK������,T(�Kf���B]S�X����`
���f���a�S!��"���-:��������B?�Fn�����F��f���B]S��Qq�(�a��?���5.9����l�)��xj/��i��-��K�5�i=l51�2�nvj��)a������1�-������Q6 �f6*$������;����U�m���z�*���G�lvj��)a��x��5(<)�)U��0RQ��d.)e����r�O
���s�nlM�,��Cck�\�,��Z�kJ+�+��J�0ZE�h����|�2\�A�;u.��=L���wB:�����p��x(u�SuM�����evMi�+����=�����R�(��-TO���=�=��$DT5J[6�2�e�Su�c�U�^�������l+N-�M&�l�i���m
������wF���z�o4�b��N-�5%���]��I�����xA���@a[�*}t��
����q7�&	]��z�tVX�P�f������*,��A�V�
���>$
r���1�
��p;u��P��]��iC�m��x����������b�����Th%��>24-��B�$��A%���;Jc8�6�n��53A����D�
�U���B]S�8�\�3X	51_�@�#FE2�z�U�s�tn��_n��I<`��]�Z=������N-�5%����<����!+���a+�Ci���R'��z�v<�m�`|�}�����C�V���e�SuM	��$�[�x�P3f?v�@�$�4F�d���v���A����%�R�.��x�h(�V���������J��3��8�m���y���l�������*�����q��g�������?�"��	9�IL�f����*���q�7&)�\
�i\6��dE��#����@j�;���%�]�Ci=tj�,��e�SuM	����hMp5�!ln�����Ro����n%��m|r�x��q���/���lz�a��$�e�Su����xh��� ����1�����I��2���0�w�o�x�>^�
Pz�� S��"\6;�P��0Vi<d�
�,E����3
�*�����-C�51���*S���6��$�z����]^�e�SuM	c��K�<Z>��6v!
��e'���V�������t|�q��x������N
6+����N-�5%�[�����2Bw
�Pf{ahT�P�G�^Nm�`Jx�����W�6�'��R��C�+��B�]7;�P��0n#l�s��4��Y/VyXT��5N��<f������S��lx�n�
��o���Q$��_6;�P��0��xT�x(�R�6��xh��x(�-�������-����&�
=l�_9*�ovj��)a��x��e�9����,|�1k�L���h/%^���|j�{�>����9J��:5x~�x�u�SuM	c5���X���6*�w�4&����.	�l������a���G���9�t[���C��Z8����N-�5%�S���,����;��r�	C��?r��eU#xj�8����,�x��n����-��5�f����Y��R<��T�#�!�y*"��������M7�"�Y<G�y*�����d~��N-�5#�I5�Y�0�2*��i����H%���=-R!l	'�1���?�k���4qC�m��IE
u�SuM	c�\��M���F���q�Pr&l]���)�s�x�x�����
$�����A�=�e�SuM	c5�F��|�bB��5�qXD��&�����K����q>���n���B�
=t�		���T��N-�5%��J����|%�\��m���	��6/l�te��0��K��w��hm���M
�:���f������mcJ�?:�Ym�x7���*�R�N�-Z����A����n<�$�::W+��.sJ/��z�kJ$�Dx>����"�������Td��*���]���q���������L��c.��z�kJ$��<�����9h�P�8,��E]��EOl����p��;�jR����N�z�\���H�lvj��)a�ry9]L�[��\J�$���HX.a�U�������>�WA]2���"���-c}a��N-�5%�[K�������n����a�8�,2[�d�`�V�������;f�lM�z��nd�5�,��Z�kJ��xRT����
�X67����q�,$���x��q�w0�sg;�W����A&��Ly���Z�kF'��nl�a�A�!��5��n	�2�5�5�x�N]!t��[��q7��Z���Cw����n���B]S�X%��d4��h��W��7�II7������vK�&���0��5�VQ�aM�
k��f�����%���N@k�j�9��5-�bXiWdnH�I�t��������pA+����5��+��Z�kJ�l^��9�
6��a6/����p�c�:*��s��t�f��L��
=tj@42U�,��Z�kJ�+�"/VET0���8;�]n Z����S�x!�����xW0�1�S'	3��N-�5%����-W���]��8�;� l$T!�<U*�=;:�;����Eo��:5�%,�Y6;�P��0^sG>%\|<�v�*����8�cc��D�Z3S�S�`T����&���C�(&#t�f���B]S�X�����>���Y:��1�7(u�$6���?��a�x<�[���keJ���s	�/��Z�kJ��P���Mh�7��fy<��{r�]�6��:7�
�azB�M�9�zH�j� �
��N-�5%�uO^�F
�`�I����d<f���^�j�c<��w���5�����"z=X�g��O���z�kF$gE�����e��M
��<Xd S��d�]z�y���sO���@�z��*2��Q�*�f�����\��"=��gz�?fWk���(�U^��	[�2�;��0x7�3����Tz��|m�lvj��)a�2y�j{6��v0J�`b2(��=c�-TF����"����6�����o��S�AiY6;�P��0V�<�tSe^�!�kN��7I3�C��/d���4��6�w�.�]"����J=s�(L�5�f����A��]k�A5I��<.f���X���P�vDw�hE����e�2��c7,:v���`K[^���C]S"Y��2�B������F?l���
�t
��kbB{������[~M+�9���N
��	�~.��Z�kJ��<a��{����!��������dat�>��^�����T���9R\NZ�@��/h�M�f����jL^@Znc�������c.
�lVb���&����!�����'M���N
Qh�Y
�f�����_]�"J�]k>HC�"�?(Ac)u�R���]������n�����\������Y�K�x���B]S�X��	�v����m]�|��I������Ltk�)��l�w����)yB�z�����k.��Z�kF;�6���B����LB>//��@$�c+�cv ��R������$���=�n~s�cg[.�js)JG��� ����nV��v�<6���;�a�wj��qIJ{����&�~&~�4f��TC]s"�����JqE`���	��cm��,
alc���bD�����1o!+e,@l"�+Z���R�lv���9��U��\;L��)%�Wr��:)w�r@�-�x��@�wh�V�X��@��J]���{��������������Po�@�(������J�B�,d��j���wJ�Q�P,��F)������d��������uF/�����jn�1��Yg�h�0ia���uw(�P�P)���M<yC�q��TC]sY��Z�,H�s����m���&In$)j�&>e��t@�;�l*e\nd�Q��"������f�����C/1S�n���i���bWG!a	�����(�{�cp����=v�V�R�������oe�S
u�	d=7/��nT ��8����CB�<|J|��f8w��{����<��qC�0�,���������Mr2D���D�i��T�1����%�"	��ia���7�=��7�WC���+��������s�XD��?�x��$���M_�����������[��x�d�y�[���~kp%���S��/�������=bX�_�����W�~}������o^|`\��]��?����o����o?�����J1x��7Oe���?������
�e/e{�(�9�P��C�V����&������_�=����_>�z��������Z������}�������7�_����[����)~<��F�x��`_Q9_zb�X;&K������Y���9�?p�9e���r"��F��V*�E��a-Zs���� ���^�����O��o�{�����)}�����_�+�����O����~���>~���]��]���rOJ:I�$�&��"���[I:!��$bD�\�t��7��Z�<�L���������L6����� �"-1 ��J�3�R����|!��T.<)���1��m��H�F2���"�h�oH�=}��E��9��O��������uC���XD$��5��8L��>>��!8��Y����^~$C��Y�?��/��'��
����������z�}H��0U�8)`f����1�Id����'e�:\���C��}("��]�2*���b�C��8����@)��n���>���%i�`������}A�g������0[6z���������� 1r���3�Vd�~z���H2�+s�$7��U�bu��+��yb�����������C������A�IMe�����.��J�����B�$����E��3��II��t�hB���!�^���)d�Z����0�$��9$�OJ��bd�(��M`G/��s/$���0i�x�$a�\|Nm����}�H^���t%)�����>J�}��I��sH�z������K��t�<���d�%�#Q@�S�$� o�,�dq�{�&�V*&�[������M�������0���9�=���'%�6^k��'%I6��#���M�U���4��_�$��8B�[<�o����rG�,�d
<�������#N7���M&)+��(�#?��l�$��~%���'	���v��>�u���8�,���r�&��G���|��!I��I�9��I[Ac���~���8���Q�8�����t|���d����_��d�������$�l��T�i5r3 ����'�f����[t�]t)icIJu��wS���3H�}�|�$�������2�q����d�yE$����-� �9�n��{F������q�e.���������t�^FH'mO�E9�$����=)���I��8����e��J���F��$���B����g4~>)���q1�� �qla2HR�/l������N�g1O!I���d��$��
~h���(!�dI�e3Ol����Y��n���A��S!�Q�T�PZ)�v��(C.U��9�I�����q���{q�$��$��O�]:��
k9@�}��f����q��a�'���v�0#������9TOJr�
���az�Bf+H$��nLY��iGg�g��89H�����Vy��a�v���S���AT%��3������]��Y���U���MF:�$���,s�F������'��))'�� `L�z������'��lI�#�j���������a
��2jnQ���U,hoQ:(����0�KY=F�0
}�Pd��ah^U��$��3i�II�Fz���-�V���8h��LJO��%+��&�SH�ev.�d��f��*����� ���[������2����#0�Gz}L:f�g2���-�V*���,� ��>@����Z��e�|���48H�,����-��sH��8�t���[���e�'��`���ziG|L�q��Yp����"I����a����+)+��a�)�oQ$����W0�$�e�o1�>�MW��a������o�,���/�I0��`�
Gh�&��c����2�Z&Y�"�oq��j��^�������^��.	Z�E
B���8w0IJ��>��^��&eO��I�1`�H^
R�B*� �����d�p��%f�T�$���^���{RR��D�O"���<nT���~6��Vv,KMod�"�
+���q����(8@����)�KAr����;bZ:��M�x���&yq����F�~]9��o7�$�eKw������ij�qOv=�9��V��q�"H1V���!������yq��Y��${���s��W�����T�B�O|'��?k'�w��6������3�T�����z�5��l��`��$f�f�����#����a�!��l��R���q:>���$��#"����dl/
���t��Ryq��6��|7��s��e����}Ln�+��f�]I��70YU_
���3�����qRp:���0���Y���HR���nH���`d<�������q��������$���r����|O�{�$��|�@n�9�Y|�3���l�&��Y��E�|����>dV�JM���n�j�/N���)���b���|�$�Z5�P1y��x�+�B�lr���H���::�#����}�V�=�h����Z��9X5��Ow�;��?�$!=g������qb���:M������L(�1K��fD��z��({������tX�����t
#�\��#}�nI�Yx�$d����l/F��I*|���2]����5���8��F{@��N�^�DZ�b�� A�jte���=Ir�ewD�>A���MV��c��`4K�ymC��0��= �K�Gc���D?#��-KI�tR������q����
im�P4b���Wn���=g��b�^�����KQ����Y2��]�����t�P����V�!`�i� ��1Z~b����$������&��H�{���"�-��t�2*�T��Y��b7�N�e\�b>1����������0����q�{/F����q�A'OL#�/>���0
���G�"���A���*4M�����b7�t��$�'yb_����bB����:��O��J�(&���h�C�=`��
/�N�4,B[5�q���&�f8�C6�^L��jz����F�vc,����N��c�p{/�#���T��Q�G�X�F���lsja0>�nL^�����{�#>51ZI/j�+��3��%:�@ �^2���72?"����q���a�8��!F8��'�^D�h��)/F^%$o�B�0�]�G����b���K=cc���T�U�l0Au'�I�o>���Lk��b$�x���
���K-�����8�N*��b�����.��q�C�ph�j�:a��s��1��F.�UvcB��J�'�4ud���=e8�����!�#J����f�'&
(������#A>����#�ecf��B��2f���pv��p*�����|�b~���m���������FR�����z���{/&&wU���J�5����V�L��wI�^L��� ���~��%��{�r���64��=�	�wc��H��bI���	�(K�Z����A��
l8�.;�^�4]� 
��R2Uj��n�z������9�G�i�����]q�"},�_
���L�=!N3�8�#����*)WY�q����BT?��i�q;I��|DI���C^nG��p���<�����U�Qb�f<Yw���)��\��'��k:FfE�����i�a���s���Z{u�]�XZ�f�7h���2����C����X��k�0>�V.K$\{���$�#[=G����Y������AUO%�R
&e-�<�;����v��S\��2me8��.#N��t�r��-� �^���
�a�!c���F���E��\��$����3��
*�>��OU2�B��TR��4�<p�������f�{�"G�9�����~�l�H�gi>D�0�!�@Q����o��|�C���w�d��^���������_>�d��?>��#{Y��wi����_�?PK���g�+��PK�z2K18sept/pg_part_head/4_4.outUX���Y���Y��}K�$������n#��x��cF���v�a���aO�^��jc?����8�~����g"PU�����P���H�����������=��?}�/w��_�n����?�~�����o�{����R��C���M�X��������{������W����W�}��������>~��A��8o�i���������L������?|x��O_�����^}|�qB�M��[��?�Q�q�[��������/�������Z���o�7?}x�������������1�����������.tk3A6a��M
���E�""m<r]�)^���=����O�������?���y���&������/0C��7�o���W?��������cDO���T>�gz;c�K>=F
��9����]�&��"�!Z~�����������'����>|�����������Y2���9�����w�_>�G����}v����3��]��$��O�,D!c���1<���}����W?=������w��=����?)1�]`zx�)?e����[�1fo_��u����?=~����Y���������o\��w~g��|�����}��_����u���2��=X�e�/�^�������������������o_�l�p��y���?���2���w*���w�?�y%����������53XT�c�����C������������������8��.C�����|!���G8&M�0��Iy=j`��a��x�
�K�|�e�O�x��p���B]S�W0�D�XA��������]6�Jo7��5��w<�����������/P`!\���]Sb������-y��o@����K�E�^c�E��s�N-|#]]&��"�\XV���]SB�.T���k��6�/�W��AD_`�����C/S�����]��.fkV�_q������,u��}��kJ��&u��*z/w!\t��g�����Oy��l:5t�4�p+u���s�c��u�����AL�������G�� ���7C!�3C7���o�n2�=6�r�����_���M
�L7�@�5���OI]��r�����u]L�C7�B���Aq`�6�]����H����nn���0��Nc���R�K�z������UX'�N-{}��#8��
6�f�*>tl���oY6;�P��0f#u��[^X\e�U�e|�6�%�Ab5@6^;|��a���o��c`Sd5�4:68�<���lvr��)a��oN<	�~cHF�����C�=$��.�{���<�3���xi�n�o�U��7���A��x~��N.�5%�A��`rU��r���Jc"&�����I��	���1o�9�U���
>��1JT4��������q����Dq��LnuLicgH�����	�9����xw��
�Y��z�"������\�kJ�H�yZ`�EK^a���q�"4�nL&6�0e�����`��N
���W0�O�Y���F���B]S���4|��4eZ�8�Tx$�EDoR���3�����i�na��Iv��n��A�����N.�5%�[���TCt��5Y2"��]����Hc�dun_����=c|f��
���a"n���1H��1�f'���-Z�(�pIV����P�����H�d|n�
��6�"�A7��0fS�m��c���D-��\�kJ��]fU�D��������8��b"�UT�O�S�x����x��
>���q2�e��uM	��2�$�K����2���[F��VH(5)�P��Oc�����M�M:4D|���%��y�lvr��a-��������o���{x"������	�@WH�Z��;(`o��1��|�n�-m�[���B]S��E� �n\M<�A}�qvz��b��&��"F���q�C��F�$O%o����/6�f'�������Xr*��M'�6��4�&�R�RtS�,�;��/�����5��C��Mv�}_6;�P��0^#GS���^<����|�����O��`�u�_Xt]�t0��x�1�D�0���45:6Hey�Kq���B]S�x�!+F�lg��3
S�<��%���56�=7������x�'�����o�
%)m���\�kJ���[jE3$� (�`������0����;O�c�x<�oK�C���>MFR��������q���D�J�"���#F���+��%��2��U��xn�q���n�x��]��S|�F��(��f'���*�g��j
d����C��$I�b
&+�8�;5����m�4�^k|������x���B]S��E���.��`��{�c���	�K�41��t,p���lp�.��*[�|��I�f���B]S�X������7I�_�G��h����i�1����ZE��(�F�r��:i��#)����\�kFc��96(��#���q�����A��Y&G������o��m-�M^U�(>tl���lX6u7TQ</�m��1EV�C�4n���ci��2+�Fwni��W*���Yl���Co�9#YZf���B]S�X���`B�����Z��C��`M��d��rN�T�w����a<�:-(F�|�T����n'�����u�����nuY�*��L�,��J9f
����W�
�q�
>tl`��t�Z6;�P��0V}��]����M�E;��H�[:JZ��F@:w��a���0)+O��c1��f���B]S�������AgU�����(�2�$HEB���wpV���
���(>��"����nvr��)a��_������m��<5�A:��(	����Z�x|���_�l�|���'c�(�u��u�c�2��0����������Wd�4^�c�Ra�����m����
������v����g<�b��f'������zPm��������L��%-%|�uLp�:&r��x������J���c�(��:��������q��I�A��;���u�v��]*���[�3�������.s>95�B��3��$��f'���j�L�u�9�n��8sl66	�I"��+�;9�������P��5�@�0I7����������D�Pu�ah�PB ?lUA��)�xt�zU�t��y�C���8�<*>����@�]�lvr��)aLJ���f��a�`� �1�y`�RzT�Nm��t|7�������4#z>P��.��|�kJ$��y�p�r��E3����qg�2�YQVNKw��h���@�f�d�j@4��Y�
�B_6;�P��0V�<L���n������v�����CiZ�d�q�1�S�8��]�}7��u�����5:!���lvr��)a�Z`Q^��$�H�t��C��m�2H�w��������/e�]��K��
>��1�W�^,��\�kJ�@��5�"���u����	 ^�������=��V�=F>��@���������!.��\�kF�N�G��N�-�J���[`��c�8T�=]�'L��*��t�>{��������a���v���)�����|-I����~��+]� �eUN���;��-������I�$���VH:BM;��N.�5%�U,�T�{����
�J?������oYO1��SGA�������Q�`DW�����Y6;�P��8V�l�_���6#����.�X���qJ	�..
�x���a6�����
��@d��N.�5%���I;C��&��X(
�
��`��$�{50(�[=v���m0\4����;����;^6;�P��0nm�2K�\B �g����3�����Pj��2:5�0�Z9�0�m��E���Cc��(�f'�����<�D�To��ql�I�[�6XH�X�|M�O�?(���&u������COra|}Q7;�P��0V��(e�E7i�\�y\"N
�E�Fe$�S��>���$�*Q|�;V�Z��lvr��)a�"y,�k�����X,�i��R���0��<��x�������k�(����#=z��������q��I�}�jD��5+���� 'K���t�o������<�]M^d��m0��C �R������fDrh�<I��C��}��?��@�A A��x���U���'r����p���Zn���V$
j�������������7�$���,VZ�8L�H��V|nA&E6�x��
���0���I\���:6��r�/��\�kJ�H^ZZ��ec�Ic��0��N�Kj�Y���A:>�8t�<�l>o��w}bik��N.�5%�[ ����d��:�<f��i����@�t�Q�4�N=@���
V�
�!:��z�|�]�ll����������q�g�eN`�����j!�*2K�(��$�K�R!c���|���gu���C����"Z6;�P��0�Uy��H���� �<�d�vs��skP����p�����y�^p��ir).��\�kJ�@�e���A��8�BL4�T��	��������n�v�A�������[�W��N3���S.��\�kJ��6lQ��{>h���F�R*&i�Dn�Al�������H�������]�L&�&�f'����)��P���jJ���V�a�V��X�*M���q�;h��<i��.8���
��N.�5%��R���cmY��f�m��Y!��I_Y�W��;�\��<��X��+@zS��m���B]3�8��<+�}��
�l�x�F�V��%�J�/sbOcr�W���$O���
>��2�"��.���O�B]S��)�/�"�N���C����&�"Bf��F��s�x�x/�
���U|���5X��f'���j�
�n����L�F!<�$�X�I�5�T��xnq����b7�����
Fl���e
���v���)��*j-�g.�VI���P=&���E=���*)���S��O���@�iW��zG��x���B]S�X���z,����CT(��$�����AG2��E�O�@�0�-�c]�8��C�������e��uM	����4>^&�4)@>���"2��+aH�(7G]��N�|��+�c���T|��\��,��\�kJ��<�����90*��#_bV��@"od��Z~���t����+���P�|�����X�We��uM	c����T��<���4����U��e�P�v%�:�����] �$~��a��7�f'����$�����Y�)�Z���c��Kc$���
:�4�0��kf�*�z�q2Xz8��N.�5%���<I
*��b�6	$��� �B\n�M�juL�v{�8w��S�*z��A���f'����6��:��KK�V���Yn��E����.���0y8uv�=n��:���k$U|����
�f'���*�'�s��"�\^,��8�jv�4�m�c:w�yw�q�YS���
>t�<���2��lvr��)a���$E��x6�:0��xJI7��V!vU/<�v���u��������������w���B]S�XE��]Z�EgU�&�8��%����%E.95���w)���Q���i$U|�3�%ESDP���B]S��_)yQ*��q|�2���2]�����Om��p|=^���]�8�a�2&�Z6;�P��0��$�\{`y��0���%xtR�_:]`�X�sO���0��xM����i'�5��N.�5%���J��j��U���P��6�$Cz�t�����5�e�|j������r<�Dks����A��i���B]S�X�����V�Z��;�n���Hc���&��3����Nni���2� b�H�f���B]S�8]y*dZ`"��X�,�'f�Dy��?����F�x��+t��	�6�� � y8�������������.��U� 'c/�	��0�A,�c�� @2�6�Bz�����aSW�������>���+�;�f'���Yu�$42�RF����=�8���e�#X2V�%M����xi������%����Y����e��uM	c�U�5
���N�R�U'�q����G��R&�JQ��pw�����xI%j*>��1��b��N.�5%�[),�121�*�������&����t:�I��S�T�||0:�F���VL���dr�:/��f'���*���A�W�$]�))�����Q%e�<e�-�?�4�;��.����LC��.Q3yc�
���������WJE2P�G3EZq4��|�Mt��u����=��r������8��e�u�����������	$�n,}���E���3��^��yWCT�=uM�ww��m[M+)+n�=�������v���)������D��:N�5��������It)K/SR7���}*�o��\T=�4�Q2�,��\�kJ�	y�� �-��
O����u7hK��9�S[y���a�M�cUa���|��<����nvr��)a��j��T�<V�[��c=�)�1�R"�8UY�;w'7�����	y�������8)b0�f'����/0��z�i���0���h����b0I���'�r|������2��|�k�%��$��lvr��a�F���H��L�|�E�$�"�dMD/.���1��X1c-���d��H*�Au�W�<v���@��:�1fc�P��D��q�>-�.����;�^Awf��q�I����25#����Y�,��l�kN �h��`b�����u�|�O$l��Y�`��g���M�xY1c����a�}T�L���e��
u�	dT�ndr-.��9iD?��()��t%�H9.���n�5��k�(������I�4�,��l�kN ��<�*�}L��j�1"K��AH����$N^%}|$D1C�^i�@#z��j���������uL/�����lO4_9L2���h%a��
��,/TH/�6���e(����������qI��|G��q��6�0
�&ATd�o���k����q�����`S1�"�E��
Ft|�l��e��dC]sY�%26�y�V��v��1=�T����Gorj����#���D�����[o9�����$e��dC]sYO�,SKT�
����9<�"3������������Nz�t����c�L��Ud6
])3-��l�kN ��$'�����Z�P '�9o)H��r����j�%|��%�`��N�`�N�����XP�.���?]�#b��aT*��o�'d��R1�o��H��JKh���o���P_v�����?2��G`
|x���w�����8����W����8��??�}��������_>�<���<w��_��������(��6�}2Q,z�XU��|v������iD��>�����|����7��=�����[k�yS����/;6_��������C���f	uqFS�d���2���$��g���1Qm,��$��������?������\����`�����s�-eCk�c������������|��|����o_\N��o?<~������'��?�s��~���>~���]�F.���B�g)	%sJ���"��������J�����@�'�����+J�V}%�����s.����H:81j�R��V*�O�Z>�Kr�������R0�`�.iM�P�� WgzO��g��{I��giH�l��D�`��2
I����R��p����z|�Cp������,�pHCgR�^�,�[@��0	
����gi;J�TZ�2�4<��VQ��N
Aef����/�>K���P��V�,E�CH�������Dh8��?]�4����a�p����O�FHL��+����gC=�{E����tEw�A���B����Z�7���/g[����'r�����?%3�2Pr�~ayGJ���-�b�_�dH`�u��pJ��GHH&�U����#�F)-�`��{�1&)����Y(i�%���$��;�%m�~l����]�lV��:�C0��$���%����do���Qh�;��8�/I��JI([2V��1�$65�����g)��2!�����R���c��em�G��7qJZ�#Nw�N���t�Nw�O��R���2I��E�d�����P�E<�t�vM��%&�[���>i�����������m���_���YJ������ICJF���a�,Fj�(�9�|3@����^����Y�=gq9�2�$��v��8�`7Nw4IiA����|�$#�����]3����I\h�O{���
G<�MNf?=����O��|/V����Y��AI�{%�M`%1j-hhw������M�7��;|J�`���b4Y�)&�MC9�-_���!�x��q�@MJ�����������j�#kQn=���<:Q����,vJ�
���$�Z��kJ�80��;������"3D�U�����Y�p/��y��[6N�����Q�LRJJV��AiA2�mJ��$�=K���I��pQrr��3%m�"��e0��$�4G���3����g)��8.��$6��#L��Tc�v���������9n_�P��q�h	��g�0&�$\��NYF(-�/�)(���8�����CeO�
3����&b�8���C��h
J$8@������/(<y����Jb$f���4��7�t�y��}�%�����Ma����L��/��w,���%"���r������q�����u����	^�q�E����b�@�^
%8�N7�}�d~T���7���d����8����y�u��M�aV��R���x�UO�,����$�^NfXK����Z�$`��Q��d���������� ��Rr��0��ER�'� �����|A�X�RAq%o��D S.������CV������]*�*�$^5p����R���k%�f.�o���7�%�B(����^�n���t9�^5�{z�X'��h�r2��m��R�����CNJ�#0�{z1$�3�����!�5:���u�
�Y|ARk}%7<�e�m�U�C�'��4��
����fa[|
J�#|��6?��T�_2,'YQa�K��PfR�eD2�Ye�����������h������JraxwK�_��$�R���l'��c�`-����|����<,�� M��)�	�PT�^
�`����a����$�c��V��o�������-�_NI��q�4�����K�c�>H�>j_P��?%%u��C��7�_����O�Io��a���@�qEI�w4$�,���D�h#��_R����������t3_����v'�d@�6�Y�U=+A��-U�1�\��|V��?}|���$�`����qr�L�[Lq���M28�9��@�`H���8��F�	t�@��^/R�m$<��AE�R�&W�M�(�� � �[��u�^����R��l8��LT7����v��R��q$��)99�fA>Flo���im�*r�x����q��2��5sG���������`D;�����#����#������������R9�!�v�r������#<����dh7��3�`#����f���:y�h����8����C��5b���
P/���,�DRE�����$>���xw�m�$�sT��zz1+�C����	Ia�u�Yr02�����R7�����,e��D"'C���s[�D�d9yD_���8b��f����921�j�	)��d
;��~J�L?��nm�N��������)�Y�T��%�CAE��#��!�(���I���������lc%c�/�����&;:�#��F�>&-'U����Ij�/v��UV��������#��������9�����0��������qV�u'����O��8��_�v��=��i�/���)��Uq�,}��e���1���2C3*�E�@�/�\b������\3�9�ud����X�~����&	��6�'	����H����1��*Q��B���D����)}�g #��xw#�h����O�c2�T��K��tK�Y�x����o�1_���a[�2:���E#��dp/���,�'ct��zX�(����I:
��4�AF������#c��Z5;#�l�U6�M�;,97IOr g�k���1F]4�ar�"n/m[�rB&����=DoL�������S�
��(}dUn��aR���C�[12���eM�tr����`�Ho�q��J�h��z+�H2�T�i�6Xi�(d�2no�>�a�+&��?���������=�m���2�1(c�$�:#���
+��4LB�C]���%1u2����b�D�5����F���n#�P$OA��h��$h.G�$#�+P=<2;D�c��PK����a��U�v�O(<��k#S���9+��f �K����b�d�od8>���[�Vx��1	�s����I��4��{B�v�P�����h8��;�V�d.��B�������V��]���I�o>���Hk��b����*8�2��m���&���&i��d��_N��[1XQIq�l�D�U����~2�C������JoLD#��N�:�Z�q��f)rOPxro� �udC)[xR���V�����9�!`������������<<Xe����s�����X����J��O��Td��������-3�Q�������	�]e<�0��rS��W��9N��K�u`�����R�����a
�]2�Y6�6+c���=#�#dco��D�K��q�I,���g�U��8I*�t?Bo��/��i\�j�+W����x�(�$7�E{D�f����{�J��g\�=�S��fTq�G�wvv+��_E�G0��1�%(����$I<.�#R����"�*��L�+���lP�t�45���1��v
y9�Q;�S���.��J�J��4M������x��s����!�m-x$V�����YS��p7L��Zq��`�'�*�����4���Z�%��vyQ��z}���g�c��	t+�<�=�Q��n���<�U/�V�SN��%��
�XB��,3�?������^��SX>��G2�j�\�&%*3�7g\Lk_)q�����P�#���'�G�i�y��1������f'���&�9�����"�Iu��"
<��)ZB�1���}������~(����H�����?���~������j��C�b����&-b~���PKwY"/_+��PK�z2K18sept/pg_part_head/5_1.outUX���Y���Y��}[�����~E�Y:�y��$���=��c�X�'a�l��3��������WY]UY�"5Jvo�����=��/"��B�������=��O���������(}q������������O_�������`�rx��������n_=�����_��|�p���t����r6%'�rJ��/�������?�������|8|w��s�2��j����&�;mZ��������?�8|�����_���;�>�}/�o�?��t�{u|��������_�F����������?���������������~��E>>�����t���9�>��������h����|���������}<���������������#;(]����������.|9(�v+�i�E>������;q�����������yz}|�������:a�� )�l8�
1�q��b���E[;���o��������;������������ ��_o�ni�m�|Ug�W��:��`�M������&�9k\��~���h!���ko��#��0�x���%����Z���%l�N���������&���~��a<I�?>��|38�b�\����o_�����tD�$e���(�62$'\��#��G:]�'�N��d����'�i�������VO������:|}�����A<�����7��R���i/���\����_}��'�6�T������H6����������T�^�?qt��������$���|x�.N�����!��|��~��O�M8�o��'�r:z��YtQF��8��,ZkuT>���^���NO!���A��tSJ!��O������o�o��q�������n���_"�p�G�Z'��9�H�<	]]�o���y��������OI��46o�"Z��>����$�^� ���Y�|$q0�����l��I�)�3$�"��E������o[�����y��#�!��E���}Ge�������M=)��7/�Gqz���Ti�����O�����q�&1��5!s$��2��N��h�}��:nz!&K�&�Tk�*����	d,*��y��&�>o2���o2|��MjE�.��'���>��{��S���HL'��y�F;#$=&$�$��x�������C.�
��p�t6�9�\�p�0��I�I�Cn}w	U�
���$��]hj!z
��49v>��yi�_wU�P��yL������":�/���2�]��pk��n
]y���o|��?'J�
Z�r�1�����(��B��G��in^:����r_���l�^����u)�����4�h�������^����j�Q����N����~l��%�1*|�4����;oSw���j&�1�k��U���D&o�&y�6�7t7y�R�b����N^I����}O�L����+�^y2�������t�c�F�����t��5_��\�����^�WI��+O���|$��<s:x�r$����5�^Q����b�%���;=K��*]:��l���z�]�N�Ep��4��;�IN�c�k�j�*{������Jw��w��{�R�3i��y�Y�v�Av�|�]�U�4k����K�[��a�������pq�?������5��+0Jz�]��u������$w���yr���w���������=�(e���Nk"�\kv�u��wFK�#�f�%#y�,��bw�F��UH!��c���S#��r�I�L��W�W:�`������<m��V����o-{+��n���J��/�%CNj��G���"�A�T��������3)m[b�� -��W��!2��3��M�a��,��y��,�wr��p��H����T��V�:,���m��xR�I#��"��c��s��2S���C�X}���]<�����X��7_E���X}�������D��n!S��������F���TljZ��Rzmpm����(���m���e��W��O^�_t�6��9����nV���������n���dL�s)"�5�r�C'�.�J�7]eO�k�:c��=�������2s�O��Fm�t��Ch�yK�^�!g���5��6u�m��8���b[I����%�,1�S��d�d9t���7b��+�	�%�p1�y���6Z��6K
Kw�=��;oIW���@��G"���e���}`��z��A��DK�bC��2�{����Xo�7B�=���]�j���������p�+*V�@,Z��v�/���-E�����]b�J6����k��n�<I�!�.=�z�JVG���t�;��X�{�L��[���$�E�.=3��L������	B�r��
���}``�Nd�E��T�~�]�cft,����w��K�>X�&8�e���:F�,!n����Z2�
\p��T�oW��|iK�������ZzW������Y�R���
��BRy@���d �f��sc��0�-�w����6+/��,��F��S��wv�������
t�!��>4������A���Yf��?��Q��#1��j�&l�Q(3\z��Y)���}�#�#L�X����t+H����N�d��������Np��4��X�n9��S�Xk���_��H��+���h���H�`d��_>�����X�
��n�$��IL��G�nBr�DH��M���)6�����'UD�j���O(Y��3d�R�����O��i��eFFq��$�G�#����������6�@�jo�l� +�xf�+���~E��0������)�=���ev��6
���A&�3*"�?�a�j|�G�@������YhXz���`�b\������"��B��I��s
��9���"����%`���������0���I3�A:�D��hV�U[���	z��XaJ��N3�"}�"N,��u�{��>s$9x������$���3�k���}��>s/gXD��"z����>E�
���vp+;�{�o�k�H���H4�a�g\��;���]�����LvV��4d�������B�s����Iq������������9��P0>����s�a�DW�A����k��yd`,��:;��'oJ���b�J�X�PG&xd�O�����Px[��,����b�CQ��KX���fX�	OV.�9��V�@>�{�+:t@iM��
�(��e������,�:W�|�MC�	��2\0�����Z����z��n��>���;������f���Wt�I�$�v
=B��'�@v����i\������o�%�=��m!6%���o��Z�rxZ����R�H:�����Wc����:�h-,�4�v�g>�����m!���W�P2�$�c�s�X1a�1�1Md�{�����)
��:���s`\����_��V�P�m�{Q�}f�j��B(�<)�]��Z�O��i"e����C~�PaeL���:����y��Y`��d5Fv�Iu���J�<+�������s�Ta�zJ�{s�������bjG���!�}��4@��w^�P���MgPa��c��J������J�u(���)M��b2���Lz����M����D��Y��7��R2�6��@����kj�c9;z������>�c�y����C���~�Mw�
�:�W��q��{��e{����#��Abp�sQ��kt��Z6�.��5����8J	�^oNK4����zT-{���~cT��82�a� _4S�����e{l]9nw�x��B]<��\����_v�R�9�)��$cx�l�cM����K.�q�����J�����69��u`��o}�!%���g*�N����q������+�]$��>��$i��@��?�A-��(�Qt�H�@'��:@�4X:���&�_���M��l��d�^�/��"�?��&�����{�����>K�Z|������o�O�/��4zj|{������J��4��x������%��c�5�9�Y��4���]��_Q���k%C�����y4�x?�Y�A4k��Y���/������������8�2�Q�|4�����.����,t0����>��}f�h��oo����?��H�8�j9�����.��dK��U��.�.;Ex��3{(���#	��7S'����+t�����A��o�������������?
?����|1��~_�����{��O��?<
_����w��o�y�|��������t�_bx�u�������������~�~��tugJ���S{��+��Ep�����L���c����.����g�C/?9����mr��	'�����`�LlwM?}����i+N�sf<��t
i?������L�>S�O!!����W��qK�C�\F�j1�$�����5mS�.����-!-vB�qnW1��v �j����Y��E�L�g����;�_����%pf�_��N��h-g������]�9�'��v%p@�������L�8��;�@�.�opr8�m��>�@?
�&�g�?�i�.#p��+Et5���'��V��g?fg����"p@��A�7�����'U�,��=z�#��nR����/]!U�����@������L��P�)!��I��������$k�DFI�4�Q�K��M�����8c��(����y��P+X�a=��i�����A��8M>���V��C��3�o-�L�IM��x��K�#;����U�2a�`����3mV�K�$��8�8�8I���)s1d��
���x��,�i�b�e�n����^1n����D������L[�29���$7.�e�SD��0����]���@E��av~z&
���
_c����'
�rd)='8�����)��z��#�3mV�8%��C4�9�9O��|R���{G/�CWZ���'�3���j���]F�-G�)�����U�A����a����6+|zav�A��e�N&�*]b����[@	:x��e�^�S�����
������IVx��
�����L�>�L���w21�}B�����J`���xa@/<� ��m
_0��%@�x$���j~�$��q����6+|�2��v��yg
r!�R�Pi>dC�������r�g�Q�i��wL�rf,���OE�3�W���!�$�EG�g���1�
*
N�N8�'�<�����v\��AY����w�����](�RW,
��
�+��1O�Y�i����+}Xr�EA��'M�&bLII*f_9�����w�sN
��L�o9�������&p,�s��b\tdy��J_���`��e6�Z�@Z����`�`�:���5B.�wV���:2��^�����||h\udz���C� [s��w&����-�/aH$��4�F���.���95��E<+���AL��<�L�������Y�i���P.hu��Y�f��h����:�0<V)��S�k2R�F��2�iU�\�����|)|Ay���b\tdy��
�f(�9�D�Bk���+}�j�O�]����]ZJ���bAW�i���F]�
������	�y����6+}��Q���iAS�����Z?m�0���{�(���)}����,��z�������~�
u^udz���C���J��������b���B�����$����l-\�6��B	?]A]4����W�/Xc"��Y�i����.A�����������B�H`���	�p�W�����������}X�y%|��F��q����6+|�'�ln�H�_���	����
%�3�1������5mS�.�~LWP��pk<��p�-Y�i������ ��5��ZZ�[~_
F�uTz�u�o>N
�w�d��{��T������3��������L�>uR,���Qwh���h�pf�Z�I����w���B��� ��+�#N;�mY�i����.-sU����8��3-�3J%"��F�,��{�$��6*|2;+����!���*��5��o�Y�i���`.�::��i'!S�5�\C�����0;1��q���rzN����`.&81%8�+������#�3mV�8�e��2�7�i��)4[&�\��1CI;�����5mT�.��j�K2�����}I%�I�]���L[>#9�����j���[��-���g�O	k���65���I5�
��^��6����v� �]�����A?+|�W7L� Iix�[>_"����<�/O��oFWUZd*���^h���:�G�)����#�3mV���;e��H�r��Tl5��Ll��1���K�t]�W!\���4+w��*�"���.�����L�>�j����p�������[��@9�6����$|����k���]��3U�AcF�_�y%|:
�!|y����6+|,�]��az�vN����K�|n�K3]x�5�]U'%S�|�b�$��:����C'�a�W�6�g���H�T��B,]uB���V�O
�-.���P�$�\�t]fg�j �e�N�y^��0E#�7,:�<�f�����6����:���O����M��Y+|��hL�����U���*�}`-����"r�b�*y�����*�e�;��Y�+L1r�b3����F�x����!V�stU��m�m\����OTO��6/:�<�f��e\�����9���,O��FI�&�G^C��9�v���U�\l�m�Z�������<qO�|\tdy��
��u<�	���I��4&�Z�;��d�p��;����t]�o�u��D���nO��Z���,��Y����#�X��":V6��6S}Jz����6��j�����E�2A[��)��
�k�OG4��U���L��Y�c%��Nt�c�M&j+�
RD��F���J4�����UE]lUR������<y(
������L�>VR;��'���������o�}V(���"��}�aw��t]�W��K����J��6w���EG�g�����ZD��g��,�`"��%|��B��� ">w�sNW����'
T���r�����(/:�<�f���\`E��s11������h�@��
RY�~O�/�������C���g��B�t#�sXtdy��
��XO���Z�	�b�n�\���i��)%^�4�.|3�.�W�\��=f��U���T��EG�g����p.^D5�\P�>A�o��#&�b��v��N�������SC�.S�nk�K��������w?\��#�3mU���U}q�ma��2��G�_L��Yr
�A�'��������@.Y4=�����
�R�g���,��Y�� �����%YVX�o��9��� �RM%�c��j��U��]r�zJ�q���
^�<%zXtdy��
�#ymY�$1b�gw�	���ir	�t`��5���\r���	]�y^E;��=1.��E
�Y�3[M�[�Jb�-��b��L*�4EiD*������aNW�0s��D��N��zT��<F�
��,��Y��!\,Y#�E������a����'L1;��5������[A���4���n H
�d��#�3mV�x�x�	a9�g<������V+a5|>7�Q����$��������G�5���j)L��V��,��Y�c��YE�L�	fK&�m6�@'5$���k�;(�O��SC�.�4�U����y^	����Y�i��X���z���+9�Zy�@/ �����J��m;�zAW�dw��$�x^��,"���#�3mV���}�U�5*%}&5gd*���C%z9�������tU3j]q��T�����H����ey\udz�������y\I`��e[�NV>�!�igS��>�}A�%~��J���g����8�q�����*|��\r���h�X��lc���e��\��9��>_�\lr��)<��>%"�Z���,��Y�c ����0s�L\@�)4���$����A�
R�A����s��T��@.$p�{.�y-|�>��g^tdy��
���}&�2A��{*�f�jd{44�GA�g������{��;���
�����������L�>r���f��4=CG��w&T
3R�����j7;��h�r�<��@.�#v���
����3/:�<�f���\��:�q�'��J�����m0Ih���x�d/����s��<��@.C�L������A��q����6+|�����
��� Y�������C��YO�(��d��6��av^(�R�\t,��y
r!W�)�\�EG�g���y�gG��4��%�bh��-����O+�R�D������:I�
��� Z���.lP�����a����6+|���7��/H�B�����;�:���Q���]W���$��V+<��i�$���Y�i���A.��}>M�T����%}��>R������aN
����Wa\������������,��Y�K<�7i>��&�H��>��6.d�j]��i�w.�!|�2�W!\|�$J+<_�$�/��#�3mU�C�g���&I�?<���>%�'�����~��95r
�I��
���-��8�����F{�����#�3mV�x'��iP�.�R��l8�ctB�����]�f�P~��mu'h����\����s\tdy��
�:P5���(J$����XK?�<{��8Q����=��t]�W�\�c[9�+�Jy�q����6+|�<����KR	���nNlH�D���(L)�02��oN�l�e��P�\0Nq��k��5=�b\tdy��
�8KN���L�nH��R@�j9�*��ZV������q�*�K�@.&$�FI��w���av���,��Y�c �>i']��d�m��!�����J=�B��]�8]��Y�\|����y�0���l\tdy��
����,!O�s<���kNl@a��Dwm"q��+56���9]U�P�\�b��s���>�#j�EG�g����YE��S�NB������d�G;D;����Rv�oNW��%T K.�dvr������Y^tdy��
���%R���S�e���I)�� |��^w�oNW�g��"2|&����>#��3/:�<�f���*��TO�e ���N�2|(�#�I�����.��J�B=��E;g<�[���EG�g���Er�f�����C>��LI�6��������J�����U�������+L�r
A�����#�3mV��>R}'��,����m5QrB�>.$�� m
fv�������XA\���n�����+
i\tdy��
��$y������N5*�R}��1;C[�I�]�����+���+��Z�y�k�R�Dv�����L�>�s
�4/���S�9�f����P���!2���k��tU��XA\�"���M���nh�4,:�<�f��C\�#.6��vj��mhE\������q�����r/l��U%�b
q1a*����2;�'wfg^tdy��
��$D����,�jMl�bL�q	>�����;�eAWUP+�K�<��x^���<�c\tdy��
��Bv������)�LR�.k�p>;\�ja��
�����zw�
��p�T��y^�5$rq���,��Y�c��?n�J�!D�Z�N�Y��*)t�(���\���
Z+��IV$�����!?@��#�3mV����Kit������d��r(+
Q(�R}��[�u�}�n��m�<���K�S���EG�g����V.Zc�� |i��I��N��"�tPVd0&s�~)�7�^�U����Ez#����NKr������L[�������;WE)X��������Z	oXA��s��lC��!b�Wx^k�(0�C���,��Y�c #�X�g]�����7+0�(�!�)�/#����9����RrQ�	��������%T���,��Y�c C��;�����Uh6�N�<�aR�V$���hN
�w�K�@.����+<�fJ*�EG�g���1�**U�wj2?��:�>,*�5��0�f���{��95��2��T�\���Z�y�g�������L�>r�`L�5�+~<���l�	�WP[�i�%����*��j��3�
��>.Q����eXtdy��
_����C��K�S�����
f�D�4��y)s�.�o	r!6{a�
�k�K@�4@.��Pw`y��
��yKF�E�I��!�j	_RF`T��������@�>���
���aE)������T#T����,��Y�c (�V��Q���o�K��#3��
���v���]��U2w�������{1�//:�<�f��+���a����pI�4G�;+*L����4������>.�B�X�#������������#�3mV�X����>�~z�`l�l�q�X;L��������oN
��B�o�p	�5�Wy>������#�3mU����\����u���i)Jj�
�C+/B�?���{�����(1��z��V�^�_����s\t�y���o�b�09�bdbs2�(mU�h�4�&!���,�l��MbNW5��q�$��U+L��N`��y�i���p.V#��q.Q��d���.)		\k`���i7>�tU8�����cq���VJN��J�y�i����.^�(��Z��(�-eK�����A��$��;�zNW��c��"/��;���:��H���=/:�<�v��7t�R��uI���g��Vf��i��+#�k�������i���3y��b/s�EG�g�����.6&ar�9qb��UK���.�P^�����'������~���c�������#.gz]��1�%��#�3mW���"L������2�-6�����H��X];�{�oNW��c��#��]az=5��"��
��<��]�c�C�)�#�\�TnB~]�����e�v����=�2��BZ3�O�Y3�����"E�Q�wZu�z��J ����D��C+��,x������EV+O��g�U�.�sj���BXwv��D_8���f���������L��	������jI����:H�Y>�H����I��`�>�}A+�g]�	��������|){���}d\tdy��y�f����u��E3�`]N�|pI�S��T�u���_'����)y�+��^�9>���;�6���y"��_>���������/�o��o��i8����;)��E;:<[5���)����B����t��� ^~w�t{������C^���������}w��$������N���x���th�)��2���z#�)�>|��W>��thaa�MS�Nx;S����������I@������UaPB~�����9��s:��Lw����$y-�_v��R�N�0um������T
��������W�!!}So�S�g�M	S����O��e�K	�%���Z���OO���CK��tz��p����@��@���2[������w��	���_��y������%����;�:��O��>��[�X�������U>Q3(�fHr�!�W������zf��V%k��K�,���l�,��������.u����qq��EG8
�����oo��n�~>�/h�6��������<��0<w�?*�����G���������/���PK��.��,��PK�z2K18sept/pg_part_head/5_2.outUX���Y���Y��}[�����~E�Y:�y��$��^�m�X0�I�%���=3������*�+�*+[�F��-T��L����������'��������������i~���/7J��t����p������W1zo���k!�_�����:������_�������On��n��R�J��*
��r����������Q~������?�+��j��������~g���������?������w�������T��w���w����?��<��������xq��������a���n_��{����/�����_������W/O��������������||w�@{���7�#8�����������_�?��������<�A����d��O�I��lT(���S�rP��]����y���w����;.[�������n����������c�^X�n�qf���g����7����_���������������g~_��7O���6k�����ji���#Xc�I"$�oj�������5���c��-dp��m4"����M:�_��}	b��p��
�	em��5���Q�$S����:��'������o�g��������������o�NG�LR&zKb�#�F���uDN%E�9D5��I��pz�~�	ev���;���S�����7�_�z{{|������_�������{z�����9W??���������S�!�d��Q��~��H8������@w���Z'!�I!�������^}���xu|�G2,�����g�&�����~9���d�E������f��Q�p�Jzy:I/#}6X����tSJt��/<��Sz�n��=�=|{�����7w��������R��c�x'��������4v�e�F������k�>�q:��4���y��Z�.F�����Dz)���fy��Q���l[[W��E;�6n��;z�{��V7���/g��#�77�A/�����l�}�z�VL����5�yt���C$�!�i�$�����{B���I�yO�L"j�=�����8m��;T�M��R�=qJ5�����|����w�{��&��M�&��-���r;���$v������;�/�!;�8���'�B�DR)�����dOy�������r��i$�[j�>��k=�,�����f���Z���D�E��in�����k��+���yi��wU+QEv�������@��$T���*vNM�W�\���kC�^�rm��[��5J�LY�����c�7_kR��I�UJ�M['������RM�*�W������Z��g \ynH��
�������M�<������j�S���S�M�6���kCF�Rt�d^��2�pT6uW�j�Xa�*�6��sM��7OJI���Vo�n���v���89t�J�_Z�I������d�<p�7_��D��c�F���
	�0�Q��T���K
� j�yv��/�;�=4�R�1g	����_�����b�%��f�5N��yC�^�����-a���x*q���4m�#XE���Qi����l^��w^Wn*]�@�G�����;M���%�d���Hn��}����s����Y
d��2�.#�0����7��%S\xU�����7��5�&N�0���?6�k+��>��<ysm�U��`E*V3�R��|��&e�	a�^7z2+��G;�d'f��������U��h�M�f��Z
6q"+!�b(���/6*L9y�Xf�����r�l,�dw����Xz�}�1���>yR�$�x�U�tw�6��fE�&[��Xc��j\'9��?A$��*���b���g������|d�1��DT��SI=���l{�&�%�����>e�6�u��L�T�H�(sb��;���1�T���g~�w�mbS����X�'���w>
�h:y��d�����O%�
��i��4�
(i�&�;
��Fw?y[���!��y����G�I�Y���,d���nU�����{�ZF&	�i�Q�8$�"����~�]>��o�����-�v����*?����1��!���ko��S�~`�;O2C�6���w������;okg�����X�JJ{Ez�8E�$�h�<������]qF����!�;#�:���k��NrgD��6K
Kw����;oIW6�
]pm"" Z�x��o�5g�Lbf�����^����N����,L��D���hXz=b1�m;#V���mCvt��m�'8�
K&�t����	;���~�����]`�JF�F���QQ����-'��$�rp��nU��I.�;O��i�ykp�=�R�D��;xR�������P��Q�y�� �6+i�J2%�B�;�j
C���0q��''��M�C�\�����k)�T���e	r[���AnE�0�=1!Ff��f�[I��(�f����K�3y+�7�nP`'���k\S!�S~|~`��7�c�5q���&��O~���eV�AW��z���" ����W���k����|n,s�Z��C��
��m��s+)5�t��Ju:�'����=��,d������}�=S��*:cka$Bf�DW�b���t�R�V���	�?Xf�i5��q��VK�������>���E����
=��kI<,`Z�%}�������{_��t�
���KF�1��GY3P�}�[S��d����l_y��%K^x����,3�	��~�`Ik�����i�6N��A�������p�
��7��P���%V���E��8y�+5O����_Q�)���l�y�$��$F��2E_���
�I{>L�gTE4�}�@�t�;xD�?�c���������q�Z{����(,�Z��ld�S���q�������s<�� %����,���I3�A:�B���%s���oK����m!��$�%}��j	>�)2�%����`�3G��vT[D��&�t���Nu��`�3�r�E�m,�0���=����y�{�VvN�����-���)E��Hb�}����#��a��,vN��	�H������n�P��~��<�'��T�<Jr�.;������_�c�T�~0��a���E�����������U�����u�hn����,�����bd�O���:�"	g����}�\Y�u���5��h5P�	��$C�����;_��x(X/��6����������,�:W�|�M;7
�D�A5`2�B'%�}��5tR��'��9yu�IZ�<��<'��1�+:�$^�b���B�-�	~0����h\��d���}�2;��U"[�i��>��5Z�r������mA��#�c����Z������#���t(]0S�Xc���+~(�D�%���{��[���1�����E�u�&��8)YRg�s�l�C2�9v[L���������������-��p��yBF���kj<�s���=�s�X��f:���;�u(������<���>�}�gYkRE�w���j��STg��yJAD����7�j�Qj�}�vp��J��m���,����UL��z^�tF�J��(bI���6%�:��~S�6~q����4���w���Z�Z�K���$��{�40�GDuw�+�:�i��"��������p���=�o��AXu�VO]���7�9%|��)@�x������3��y]���#VD/:�4l��?@\-�������vr�!u����I�W�j�g�[������F�C�8,_��R�l���+�����]C@�$�����,~�#g�Gi�G��og.�OA�H��L�G��M�:i��]D���h�C��ew���[_�G�7�V���%#��!cP���>�]��)�#Vb����Y#��,�zY�Q�2�e��4jN�-���h�B���>�(����F��{S�=�,Y��/��2>?�W�ao.�X�z�,k�y��7^�9>=�8(B�h�������G�'}��x������O�<��i�x�\f����g�B�%g���?f��+{2�����;���]cnG�7�����W�%$�h�TpO�
Q>�J�e���S�byB$��+��<D��NPX����������}zw����#�����[�6��
�=��Y�&(�5��u9�9y����S���#]��o�N�c��W�.��_~���/�������������?
?����|1��~������{��O��?<
���?�w��o�y�|�������t�_�}�u�������������~�nd�tugJ��E���/��I��\�EG�g����R�t>T��������h���:�J�S�
�}&����
�����J�����W�g���&/:�<�f���CS�A�Yr��7���pU#oz�r�������fQ}M����<��I5��^d�����L��9[d�$�d.!s:R]K�`(���u�������i�2�.#sv����&���d�^de!sy����6+s�yx��6&�ca2�mZ���A��������5mS��eD����(�����DE������i����i#PDJ.����G=�?���T)Da�w�;��nX���$�/%�����vW��")�����i���9��'�J�X��oH�7B��#
lJ0K=/�\�6�_F��R�����pnW��(��I�?�1;�f.2?%I���q�����
�0`�Q1}�=mR�6%.^F��R����XqvW~���g�����L�����S���"�KN��N�L��y���
)'F�������)xA]F��H
��q+<��KiP�L�e^tdy��
_��D
}�\�����!L�t���V��=2�������i����8�s�g
�rp�='L����	�Z��0.:�<�f��aT0�E����T��>�N���$}(EY�����l�P�F��2�o9�M�@�����L��e�;�UG�g������)�Sh�X^��-f)�p��(?q�<�=�2���'/#~L��d�p�We_�����#�3mV�L�K|���d���)������!|�i�&W��g&��M��2�����$Wx^[�e��<�EG�g���1��� {�����M�dKC�[�}�tt1��3<s�bM��4��c���~*�����|�0h�����L�>\q�<���'!���#�?�W4*I�M�/�����S#�w��K�_�21��x^i>4��3.:�<�f���X�����
8���i���v>;��,�����95��2y���n>��+<�Z�j4W�B���,��Y�&�3B��tZ��-�4
�������K;psA��������`��3�Wf'YH���EG�g���1lK� �h'��L�/)g[>_ /]>Z��R��]�.sj�|�vVO�=�<�4�4"�m�����L�>rA���s���S�����
#h�/���U�R}�y�Fj���]B�*�F�N%���u�O*a�`�����L[�?�p.�r��v�v��t�V~����X5���=�� tU�v]�\l��u(<��<s{��#�3mV�������� H��^p�����qA6?F)����\k���]���{�����
�+���P9LjY�i���@.��%S�i�i�Y��M�rI�cV���u����l.\�6��B�>]�\4������e�7.:�<�f���\����5�e�9"��>��Aq=|����t���������}Xaz���Dos�=��#�3mV�,�������>z�XG��Z����y����t4��S;j���]���p.�\���dF?8s���,��Y��
Z�>f��7���"�����]:�F!��
'���}sj�����u����~����g�Hy����6+|���6��@���f���L�E
��������Y�����!|�� �+�K�R�&�y=r����Y�i���q.2�8V��$���y�i���6T#|D�/��9����
����
�>��|-�0L�Y�i���p.�:!]N��N�|�]��|>��@f�S�h�7���u��*��	�<��W>�6
�����L�>�s&���
a���C
�6�����!��9���g���i��w!�W�\�\�
�k�'}D+ <�EG�g����������[��ki>xziH�ir5���-���L��T ��,���Z�l��3.������)^�0�]d��	��Hi��&M�%R
��2��x���|s���"S�\��p���5���	���,��Y�c ���y��#!���&�b�W.�{���T�D�,�$���t]�W�\��B���yv�����Xudz����6XUr�%�2���\+��=x6���7q2C�s���M����g�l�IL��8��~���~��#�3mV�X�!�4
��)��s�^���
!y����5�����R2U��G/�>t����S
}�pU�j/�m�O�O8���<#�&;2�`�k2X��_���7 �=�2���=���w"��W	2x���,/:�<�f��$���\[�)��	��v%�ps��pr=h����UU��*�}`}��+�39!Q3.:�<�V�����s��>K-a��cle�\$��������2���}t����{��&���Wx^
*��~'i\tdy��
K8�4i>��9*����;��0���*��.|3��������
�+�����"2.:�<�f�o�:
���^����1l���0n�e�����t]�o�u�1�x^�6DzQ�0,:�<�f�����he����Ol3�%*������c�����}sj��L���E�*�d�y%|drX;��#�3mV�XI-���s���T���6'�E�!�N4tT�����.|��*�b��Z����j\1��Y�i���Jj�B;��sAi�)x�c�����X�������������������E���+L�R}Z9�w�N���L��Y��U�2����T��d��$�����	
	��i�q�<�tUk[�\�q�}f<�,�@�3p.��#�3mV��%�$\���Y��La,=L���,N��<���jn��a.:���x����yt-:�<�f���\,��jl#���@.��&�%��R�J�T�'av���ui�
�24�1+<��>����������L����F�KSg��oV��Hf����r1d���7���]���� ��K1'�y���CE�����L[>'ya_�D�@o�.l�lh>2?���yL�,�2eH{�9]UAW�\0�hzp9�+�'�P(�Y�i���A.h?6sI����#��c�St(�A��l���
[�*���������(f��#�3mV�x�x:K9�;����^���D��(��m����g-���
���
�����|*�K�//��E
�I�3[M���b�-��b�����~�
���P{a���
a�*��.��d<����pY�i��7�X2�G��E�,p{��Z�E�q}�A�}D���oNW�D���\Li���RRK��f�UG�g������c�r��x�%��V�d(3��h?��D������{�G��5���2Siht>�:2=�f���\����Y���d������0+:� tI��&�gE��!~���*��I���<��.F�y�Y�i��X����v�*���u�l����4�n=�����;�zAW�kw�e�F���J�Y`��t9�:2=�f��A]�0�&
��`�Lj���Be�AOUR��M
���[�U��u�S��+�3*���OW��i���{���]DX�)E�Y+��Jw�lGk��!�
�����K�*���e-�(k���
�+;�:2=�V��3��'Sr,�%ad~
!�F3���4d�� �C���9]���
�b��6O�y��p�x>.:�<�f��^��#����D������R(�#��{�63&���tUi?_^H�J']��g��:@��EG�g������g��)�e�u�lcm��(K2&j]W|����������x%�
�x^�3#��h]>.:�<�f��^�1'����e���m�]d"�!Sht��-K������S���er��B�hO�Yaz�s�v�-N��L��Y�c���&�H�T�%oU3� =��*���OS���t{�sNW�s��%��;
>�L��O��!���L��Y�c�O^\.up�������b���
���(\�,i�1sjX���������^�~I	�Vv��#�3mV�<K���|���M1�����2��i=���~��9]Uw	_A^�H~%��J�0�a�k�Y�i��� /J��� }�|��E�R������[��SV�>2f�,�����%��y�=�y5B�Y�1<`\tdy��
��$#2���	Rh� u��O�D��wcMB��Rq��b�95��B�_�w�-Y*f�W/A��
��L��Y�K<�7)?�!���H�Z�_�F��3����1�i�{.�!~�2�W�]|�"��W�~6	�)���#�3mU����E�`k�$�+msx_����B�'cy���oN���eR~��xi��>��J���W�{>.:�<�f��ww!s3�&g��4!5����_�2#�y�M��oF
�w�B�Pww�wtzr���31_n_^tdy��
��^O�5�>
[�����5�
��@��!����lN�%}��9���3}������i����6+��<����KR��(�ts�CD�4A�"9F1�=��oN���e��P�]�Nq��u/y)����Y�i����.���m����4���������&�^ ��an��]iNW�u	��3�]I��+�$4PQ��#�3mV��%z%t.�s��)*IF�lg�<	�����?U�
�v��t]�gtZt��WF=�Tc����6+|��y�"��}��\�v�Ap��`v�������qGy���:��
����9p�W1��G�q����6+|l������S�
Y<i�n���9	B��
����
vw��tU�]Bt��s���e&z.�q����6+}�b*L2z��I�����Z���Xc���|����������aD���.�<�T_��5	1.:�<�f���0��T[�e(r[����l�a4!_�O�]�UU��z�w�x^C<�}d�x��,��U���B��	�Bgk'���	1#m���wZr�K�����7����+�������v��r�������L�>���4�I�$K��d�������h4�����aog6����b�rA;��Vx^'�H��uy\tdy��
G��AD�F+��S���4��qt�Td��S������0���+��Z�y����q����,��Y�3<� OcT��N(w��fQ�2�d=���R�RY��������2}�����zr���E�	���y����6+�"�s��<t
~y�3�3�����v��R���aNW���5�����z����$�s(�Y�i���P.�2&r?%�Y������I~�B��82V����_;�eAWUU+�K�<��x^g�H:!�0,:�<�f���sII�Be�S��v�n(@v ��������U��oNW��3V(�D��V+<�5��UB���,��Y�c(�"�q��
%�"F�4����p�SB���{�aAW����$+�[�y%|F���$|y����6+|q���4�|��M ����@*���r�0���[�u�|�>��m�<���i�|\tdy��
k��5&����H���4ouj\�T��tS���{X/��zx�
�"�f���32P��Y�i�����"��;��������M|���`3�J�������UeR=�UB~��U���#4��#�3mV��]�sQ�u)
{�A0?�iw*O�PW���`����9]���*����]�y]�	#���#�3mV��������E�PP�^�V���"����x�^V4�����%U(����+<����)����EG�g���1�J*U�wj2?��:�>%�@E��h!Yg����sjH�e���B�8O\cz�r1�T�P��W��i���P.������C{��Chb��Ai��%���,��lp�.���g�a.J����J��4B��8�:2=�f��]F��!��%����K�#�,.B9�0��j��]�����7�����@b3�dWx^�dIG������L�>tA��2��=1B*�������������{~s�*�K���)�������Y�i����.!����#�C�4N���)�a^C&���Z����3�t]�W]�DX�yv��^��K��1��Y�i����aV��2K�o�36�l��d�FL�4@�9VWkw������K��.��@7cz��c�1$�����L��?��#Ae���JO��-�U���w��N$�T�"����������]H�-�.q���*��$=�Q�q�����*|J��.I	��.V�2@*&k[�S���d�����?�����tU���ya{L+L�����y�i���{����v1�P�v�d�6���~%�;J�O���s���
��%��Ei���^a��n��EG�g���1�����e�K������l��6����xQi�=�tUx�����c���Z��o���EG�g���1�F0�\]��A(�*eK���`��i�+]���;�zNW��c��B/��Zaz�$9E!�y�i�����h)L�$gY�3�fO�@�bn��<9��d_��s~s�2�W^�b&cz��"��'�~y�����+~������X��E/�c
6����Iab��D�"~��Y�95��ej��'������W�Grj�Sp\t�y���`��}1�hL=!7��\3�����k5 ��)�{�oNW��c�����]�z��S&��<�:r=�v%��^L���g1�aj�����N$�1D�n�*�=�2���[3�O�Y_�����:G��EG�g���1��%[��&I��f��v�E%� �A�%�z�I����������-^�.�������8#��y�i��7�<zyb���C��Q�t�h�.|�5��BC�5���}j��V��(��������_��R��A�]1.:�<��>�3�p���@��`
0-�u>�$��)�c��l�>�4L�����w���������M���V��|s�D2�:�||w������7�_�7��7����pR���;)��
�t���9YZ�����i��x!_~x|�w|/���y���{s����!��]��_�w���=�c����_�R����S�-�����k��BG���~-�J������-��4>��3��3���8�h��H��g� =_8�
�e���4�N8av��N&.�]8~?	@^����v�����(L�[�tv�:;g���z��~�W�!��L=�Na�z�
�����O��e�K	�%���Z���OO&;��)H���2�������$����d�|�����o�&�?��|}��y�oK����:�:�����>��[�X����<�9����Aa�@�N� Zs�V�S�T�L7��d�<{I�%;�g�-���������K��sNB�����q��y}{sww{���!��������������������Ry+
�o>����k<��~��g�PK�j]��,��PK{2K18sept/pg_part_head/5_3.outUX���Y���Y��}[�����~E�Y:�y��$��^�m�X0�I�6��zf�+��?�UVWFUV�H
��[��a&�iOvFF������s������������,��p���m��?���y<���{x�&F���^x-�������Q���y����:|yu�����������|p1����/�����{G~�����|8|w�K�2��j���S����~g��������/�����77�������T������/�/���\����o�o^��}��H�p������7�w�o�":���������������������rxu�z�0|�?�����{�]�N��7�Won��x�������������W�GvP*�Dg#�A9���V�j_*��?�V*���|^�W�7w����������a��z���xO����'���5#��|j��ha�,�'����3����������o�no������������'~_��W��7��6k�����ji���X�R���75�+�����5����i��28���6}��q�&��/a�����Tf�?)��(��x��}�LEJ?�������o�_���4���N�W����=�2I���0
G������AG���tB���Q&M���_yB�������l��=�������7��������:|���/������6q�C>���������8���;�"��O�HM/���>�H���������`H��$�K�����II�����pq�w�/���H�����w���o��|������H!I\\��h:�8>��ZI�NWI/�C�`�)�����1����L)����<��Sz����9�9|����������L���D�7��;r�:�
���/����$^*�}�U�����5}�H�8�?�������I$o�s��2�����2��'o�'�y�{HF���0�>cR!���=���m�c���#�~Z��")zr�4����MQ���v��-=B�5O�5/������$�������?��o_�������D�z��^RYd��};m��;T�m�8YCtM`��I:������J���Mb}�d��i�d�6�;2s]o�����Vf�=g��~�M�?���)�<p+�g=�>"I\c�3�?�J����d&L�I�����vH��S�I�N.�q����h��� �5��������M�7�l^���]�
T��%����y�.^����,�I�{:��pm�
��
������o�y��5��B{�������Z���,N�z�����E'l�J�A8�y�~�	�����)y��N�'h�6���I�6O:!��5�6�u*�R�����k���~l�yk����I�������ZUK�
���|b�&����$0��zQ�W�Y��������'VO'�$���yO��tm|4���*�'����k
��yvm�kn� 5�y2;��<1,+��'��VA���`w�������^�!��{��6����+��y]���u�x7�qm��X\�����v�^jX<�Z��J�6)���i�L�M�6�u�;�+��M ��l����;O��
������� �o�������j�6$���E(�Dn}�������5l����S|[�F:o?hXEF�u�����2���2�[�����k�6d����3BoT�����7_;�d�
�����w�RP1�<��	�+{*cw�F�V��-3�c����^&�M�\�b���w~��%�L9y����E��c�T��'=-���fE���.&�zmg$X�,0����������h�d���Ui���Qd `����<�A`SkX�����m����"��N��l��!�jV�J���N~j�'��c��&\�>6l�u7�L�T�(�g>�n�y
��pm�tHP�"��u7�M��[cY�=����aK�lO��������_>��6Z$�4lj����g�6�?p,z�t���u�O�af���[����#�aU��G[�����Ui���'��k};B�P�6I��=���v�T������Y�T[`a��|Y"
�Q3���kg$��q�������6��| �X �6��M��y[;#��C�������Wx ��K����<�+��MBg��-��t��}�gqk���,5,�ys�t�-��v���I��@i������7��3B&1��ch
�1t�}�U	1���L�������;�������s�Y�6�T��5�g8�
K&�t��z���p���%9e6uX��P�D�k��~�q��1Lb��"Jf���t�3�hC�\�d�w^G���U�I�I�2�����_	��@ow��F�'4,v����-���B�;�j
CA��(q�$��E���mC�J1<�WRn�aa����x���� �@&Yr!��*�JBMFa5����Id��@��[���b��?��U�D"���@���Vh����W|Xf��Te��H���3�����}���
��OA������DM�a{�8f���{���$���H{���	����H�����D���~���Y��LX���'����6�m�X��D��������Nh��0��@�'$]w�dP���<�����@��M�m%e�V�a����f���c�?|�+����6Odu"���!5r$��L��M���%6����;o4�N��ExVq�V�l�_jXR[�WN���Y�����9$�-`u��m���������6�(�zb�6:�����5�E�d��h�fw^���Q�	�!tm���;O_����I{>LpgD4�}	Y4g�
�3����f�`I��a� 1nQk�M8 �-��,B,�����w�Sg�~��L�I{��$����vAP��M���.�����Ee�\������r[,�"�7�v���N�?�'{8n�����`�3�(c��{�����;�;s��Kh���{9�"�6�v�4��F���y�{�Vv�����-��p[�Db�}���9��
����i���H������H=���W�����~��H%]H(�o57>8"�j����P�>���p�#$��^}l;�:�����#W�F:{h���,+)���,����.@d�Or�����&�&Xv����b�C���%�AnF3��rE��t��fs(J�{Wt��%i@�(���G�-�/�R�D���������"�y��2
��utix="�h���J������no5�x\P��,)��3�+J�$^Bj�)����+E_�EMc�4�(Q���N"��u�RD5�2�����z��(<)��II2�]�@V�����A���A��g��b��-C��b!��?AWQr�$�b����A����5�����Z�Z�j:S"�!�C^��mQ��f�D���?�8�(x.:q��W���X�3U�dw��j%J���L)��9 ��PCu��#0�w^+Q��"s-t����� ����Hu���#J���k������D�A���g]ww.�:�K;�.�vt��8p��L�T@����b�u�����' a�����)R���qJ�u�
3�4m�D!�?��T���DS�C��%��r��SC�E{��q0Uw�+�:� <�rv��=�����|�U����9��U�N��
�������3��9
�C�o����q��>��v�?D;CNibAQ�!���q��f�qd�7�(L�CgF�W�j�gL\����^r��'��yX����ewl�������1UBh�K�x,:~���e��A���R�v
�&����i�^
�&9����*�z��h��A!�����dS������}n<Q�4X��A�jf���q�ke
��U��.�q"�N�3Y���G�����r��(�9��Pd��FSL%/����s����I�WG�f{S�=�,Y�������`��G�[*'#�F�DYO�%c->�������������!���]=^�:�?�S�����~<�8x��1~H��'�e��0)-��9����g��R��S���M&��p�s�s+B��q��g���'!q�d�����8I���GR�zJ[�=@e��xQ�L�iNEq�)OO��\�����y|{����#��IT���4��=��Y�*h�D_���S�N�>�}BM;�\���x�
&_���?s�
�f������/��c�N�s�4���?v�����y|��������>��{�8|�����?����4p��7w��I��?�9~������3V�����O�o^����������;��	�:cz�����pU�UG�g����Q�t�?��:��V~��`��!��
�C,�C`&�S�����6*�<��+�C��_az%�h�<�:2=�f���SVo�xz}���`�_�:������g����M�;��3K�$q�1pnW� :���?�1;�f��3��4_1"\t
�#�7���$y���|B�9�6��G��R�i=a�8�+�#�������i�����|00��<��j���h���S��Q>�jM���p�s�i)���]����O�E?vg����"s��A�7yt-xB�\��[�&�>�f��.J��R��xP5��dN�Q�Tq�?=�3mV�B�9�d���dzb���%s�A��U��Ad�����k������\X��V���� 	�;�'v\�cw���\d���=��s�����:�A`�`�B��������)s�<2�2��(T�]��������3mV���$���c*'M'uj�]����Z����������5mS��:��-!��Vx^aYQ���y\tdy��
_�D
ohc5h<T��J��a��S�<�l�Ytd�qJ���LTNM�����������-�{�~���:]�fH��UG�g���1�J�V�!���m�9A>��CK�i�s��R]7�l4��
 j����G�-��)���)Z�y^
;@Bc�EG�g������9��B��
Q�oL�TA$����5����8��>�<��UX���z2�����(Y�i����*^y:���%#Ns�|HQ�T_J$�I�KQa�<Mxf�RM��`�#|K���Ir����9%0GC���,��Y�c���W����&$C����R}�����0������gNV�i��w��r�,��T�����$�+���#�3mV��-NAO���W-��T~JK4�C����0>jt����S#�w��K�e��z���^������UG�g���1@�G�wQAY��#K3��_G�L�	���9����'����M���%~��U�!j���k\tdy��
_���(��p2_�����l���g�q��"�Q����F��LQ�
�������3�r���!��Y�i���0.��������3����m�~����P�O�k��.�@�95<�3=+����(���J�+b�t�����L�>vA���s�yr�������*h%aU�]#D=��HM����UvA�l���<��������,��U����H��lW^��KT��R�l�T3$�M,S�Z`o
���J��
�b���j����a�	����,��Y�cH��	67� %X�C��f��r!W��`X�m2B�;���mJ�����1��%c��^G<���/D<��#�3mV����_rm�OS3O�[`���f�<�Dl�D|f����)~gJ��
��1	����R~�	���#�3mV��%(�zY�Y��#��[�Y����B�]'��R����oX�E�}�����+L�����8)��#�3mV�,O����<��PX�Z
c2������MSQ����Q�6��LM�t�v�(Y�����KB*�����,��Y��
[�>f�74���,�o{~*�8����
%�b���}sj(��$�u����W�^c��������L��?�u����;I8�K�'�U{������!���Y�����w�v���)E���x^e��;q�q����6+|�"I|�[����,>:�l��4�(s�$�����w��e�]*��"�/�1}E�a�����L��Y�chg���j��}�
#
�/X?F����Q{;�9]V��B���>t+<��0iUFy����6+|�2z��4�����:�`
�u��[Xa�3�J��Q�;����.������Wr~�j=T9�UG�g������F�\bk5���m����&c���<�^dTQ#�y����/h!Y�F��Uw	<���5.�����)^�0�b���)�S7r�4�g-���9��;~s��#S�]4��T��y^�53y�����L�>�v�Q	���9�J?��Tl��E�A�S��kV���/����B�b���<�������a�,��Y�c	k�09�M���q�Y�`����vD(
��sg	��M�;��g���IL�B��5��
9h�����L�>�p@��,|��Pb�Iz��|)b03��' nTb������*�*���S�,�y�$��~��b��m�/
�6�Y�<(%�X:��Ld�0��4���Ln������:����4+<���4�Y�i��W�
Ad�+�B2��6��OK��hG�����T8�����q���vS%��.���k����N���,��U��,������$���F�����;�!��e���}���.����r
&���W�^�t	B%$���#�3mV�X���I���n�Ti&�Mr����cN�?��.}3������
���
��D����{Zudz������<Z���=����bb������#����oN���V:��,���] �~���L��Y����#�Z�#:\6��6�}Ja~�Z-=zZV�=�7�FG��^l]_���i��+��^�����#�3mV�X}m��4�A�Vs��F����E�@��f��b�����B/���uPk<�2~3?��Y�i����k��"�3��i����o�����PaduBW���7�����6X��Wx�V�G����h����6+|��V���kNy��m�2�s�5�*Yw�sN�����'J�,cz]�%1}h�W��i����.!A�
�gbd��'����V8tV2&��E>��r_�E
R�5������/>Q�G�I�#�{���L��Y�c���Tc[A-�����D���$�^��Rf�9]���/CC����A*���T����L������F�Kz%����]����E�d��]�=�95��<���F�]j:9�+�O*a�Wi\tdy��
����/���-��2�F%�D�$?��Ez9���dH{S�9]TSAW!^�^\����tZh3v���#�3mV�8���q���CM������'%��Y�AF�T:�'�tQ`kWa^�������@fxV��i�����(����������&�:&�`�&�k���2Z�E�\y1*�5���4
����G5���p�5I\�($1���R�i|�)����^�9bN��s�(����.&:f{2����p�b\tdy��
��b���.Z�i�������-�����i�w�7���)�V�.����<��*y#,���EG�g����n���r��x��%�9�O��u!��)H�����tY^_�L>�I�
�k��"�&f\tdy��
C�8?�1���-� ��vA�� �c}D��R���><zN
�;O/yW�]Lb�gzs�>�����L��Y�,�gJ�;��%�c]��.}B{�<��d���n;�zA�nw�%� ��W����$m\udz���C�x�O���G����&����Dgq#Iu��b���[�E��u�3T��������q����6+|��<z��9F$~��R�m�5:�Y
���}�����/����B�X������N�N��J������L[>��.��S�-I"��S�$�p�9���}:��v�sN%|������Sx^%�����1.:�<�f��A]��#���R��Sh6�
&ui������3i�=�����
�BW�r�/��\B�IR�����L�>=��L>E2e'R�>T��\%o���� w���E5��uk�D�+<��OcV=���#�3mV���s�E����	������IE]��:�^�f��}]��j��E{+�Y�y���@B1.:�<�f���\��:�A����j�����m�����B��r�=�9��J��
���
��T�IB�9.:�<�f���\|Jcy�������n��D�A��D��x�2�s��0;�p�q.:k���>��}�����,��Y��,��V�i��K6>,���|~F� �$�,��.|s���������4�������������L�>r��Q�K���Q��T*����i}HGR�1���f�,�����%/�Z�y�����7,:�<�f��#\�����R-f�n��&��<!|A8W:���6��!|g��*�1]���������|D;��#�3mV���M��FT	Ms��k�|dyx0�~�	Yz����;�>w��.>D��
�+��l9�1�EG�g����p	�raQ���x���}�>�sU���:�]���H5�'�*�������<��Nk���v�EG�g����f.dk���L]�e!�&�3��n���3�$S��v��QC����!��\���������iE�|�����L�>�p	������Ga� n�Zp~:��-��������B�8���r�Wf�A�:����#�3mV��v�SA7tgS�K7�7[+���4A��@���k�95R
���B�p�hWx^���7��<.:�<�f��!\�%���=ci&A7��F)`n��:)'03x�;�����.
�*��!#��Q��6Jh\ �4y\tdy��
C�D����|N�2:%�(�I�H�&2�:�$�-����>N�evV#k`�x^	�U9�6.:�<�f��ur@,��}�\�v��
��?dk|v������]�8]T�P!\��S����
���g��`\tdy��
Z4�L�6��Y�����f'E#����%)��7����*��E��[�y%|dm*����#�3mV��#�b��P�0A���b���sfVkKb�'h�>��.*���E��bZ��zV�!�����L�>>�H�S1-���B�`+���>j1��>��v���E���zb�v�x^��#
?\tdy��
_�k&��r�L@�$b������#��]i��|s�����B���Wv��u��$�����,��Y�S\�H���O�$�K������F�,�$��a�^6����b�pA7��Vx^_���EG�g���q���qJ4��'���-���>M����H��u���w�YE�B�x�QU+<�R
�����!/:�<�f���T�<�Kq�v��FB)�����)��)�^�7�����
�����1�y]R������EG�g���q���c��&��N�t�
���}H�7X[�|R�U
s��<_�.&L�3��
���P91.:�<�f��!\�"��@�y�p��h�6�N#�j1��������*���%W����n#aDRm$�EG�g����.���\L��S����9(�&!1.��m�**�JU��oN��3V$j'�4�y��#W/�S+�EG�g���1����7
%��wM`5������q,�t��jX�E�c�p����Vx^�|Q8`u^tdy��
_��|)�>�����];� 5��j�vZ$�J�g�|�,��B����R
��U��%�Y�i���z�h�q���Y��'A�
�[��H�C�A��t+�w�^�E����Ez��^�y%|A���6y�����*|�O)"���i��R������f����sU��gF�f��.*���)E���+<_�����K^tdy��
C��`�����T��A�-��['�Z&)bb���nv���|�T!\�q�����N�{��b\tdy��
C�/���q�P�^�f�� ����`;��K����|�A��
����+<�4��!�����,��Y�cTS����dMx�u�%|��Ki��D�L�RY����sj�y���B�8��\��R�\����l\tdy��
C�`"���/"�0�LlV5@��a&�V�u�.|��hg�.J�����]<=������,��Y�+�����C��LH5�p	p�%Ea��b�G5 ���A)s�,�o�p!6{q�	1�y��,�����#�3mV�8��[2�,:r�'FH������|6X�RA����[�E!\R=�(��x^7��"�?Y�i���.p�s	GF�iSy�oJ�pv&�O�&��n����
���Y��x^i>�#�4_^tdy��
�R��D#�,�2�-$`6[U
�4�|YB������]T�T!\��n���o'z�bPJ^tdy��
����Ae��|?=u/�I6..��I��q��T�����oN
��L�o�p	6k�x^k>/,��EG�g���)�����3�m�yQ1Y����I\������T�����tQ��y-{L+L��5h���n\t�y���o�b�09�bdb�1�&m�J�(�����v���
w���E�k`�/�>/�����k5\T6�EG�g���1�����e�K����O����a_��F.i�=�tQ0���{�ca��zD&��~y�����+~��U2��*����^K6kj�
�R�5�ed��;�zN��c��/6
6��0��b��|X�EG�g����f.Z
��.�Y����ag�)�)km	@�=�7��~5��)f�0���E�$�1�y�����+~����I�\��nXJ���~A;���d�ppK�S~sjh����1�O�G��W+L����3�y�i���FaX_���$�RN���{��������g���{�oN��c�����]a�Z������#�3mW���$?�~��qA9�j������R�W��n3{�eN��f��B/���+L��~�j�q�����+~�b�>��H��3�?z���3S�X�����oN
���I���.Z��gzz1��������L��	�����S�F	�	���D�u�3dp&LNQ�E�O����}H��V��(��g������_��R����0�����L��d�������'�f�4�������$r��YJ�3�c�

*O��A�;���G������G���}��7����d�t8����������~<�8�������x����������H	��>(�|]Jk�.��9����2
;/��������{q������������oy-n�?~�������I�O��K�*EOM������������oI;�d"�����-��������,_%��H�~���S$��'� =_8�
���Oc������N&.�]�?�8	@^����v�����(L[�tv�:;���Q9�==�����:��>�mJ���d�����v��\��i��|��l���6�w
��?�L��y�x�'I�'u@��-���o���o���w����;o�o�m��~{�_�����y��{~�KZU!I��V�D��0Zy'�$�*�V�S�V�L7��d�|��*Kv�G�-�������77K��1'�E4�yg�*�������������C�a��������~:^����?*�`t��������W_|��PK�]Z�,��PK{2K18sept/pg_part_head/5_4.outUX���Y���Y��}m�d�q�w���f�&��K�B@r�8���k��'a�jH���ggo���[E�>��<l�jDu����Mg�,�����������������/������'��(}r���������W�����1�]P��rxz��w_������>~v�����������7�/L�A����J�9}����[��g��Q�����>>��������o� ��
k�P���������{���O_�?���o�����_x>>|s��������>������?�s�~}8��o���z|�����_~������?��{�������_�����������x|��D{��W�#|*���{}�����>=�����>��{>��2�':-��8(o\T|=(o]��_9�By��������wo����[������O������_������
���
Ip��_���B��������������w���p����w~��������i�}� U��Wm�r��1:�Jk����K����86��3\���V�I���.A���]�CRz�:�����g�����I�*��u�OR������_
�K�,d�������w��8�������������:"���,Zk�
���'T(��7��e������o�?�����W���O�����������z�����6���r����?�����99D��(��BrVCTizb�>X�'�gG�S�wc�\>B4F%�N�H2�|���Gxv��O���������������`�|�����T���,���:�x���Wx�Jp~�N�*�u�%�\0�;�R�h�O<��Sz��^�>�>�����__�=�o�%�$���Mu^m������'����-������}���t��������N8��O��7P�?y{~������1Y��:�~m��+D��4����;?z���1��X�c���S!���l����o����34_�d|��S"��?8T����>����}���x�{1����
��
����zO��76�m�2f�'����i��TL`=�(���{��&����&���o���
H�����7f�d��}����z�dvO���P�stl�� ^�8�%7�}�iCF��I�����F2�H�i���k�K��p	5���U�U
���o>r����I�B
��9|��5���@�?0���$�'���y�����\����;�c�6��{���A^Q��b=�q��[EJ���jRzk�O�	t��<] }�s��I?����������kC"�mHi��A�7���\}�I*��J�;O>
�zmXAv�J*�vBz�����y�K�o�\���Kj'�k�.��!k�N>z0��F/7z��r�������
]����H�	�;��Wb�d�
�|�a��;/�
���#�L���o��y�RI
?y8���EM�.o���/��;�VAv�y� ^7\I������j�������2�9r@����Mo������=O�T����!��cy��>s
��������K�k��������<�E��"���IZ�y�Fo���������6H�������Ko���v��i5l����O%��E�!)):y�+@a����V' 3�n�|��S�4f)f��(g��*�����k2������}�!����Yq�Mn�@�a���N���������#Kd�{��R`�7����@[O>`�0�9��|���F��y��a��5�dm�`����>�
�x`�X3>z`W4lrU`��t}�DT�+r ����/�������t��� ;�Qx>���_ �jW�J������7}Yz/m��{?�0��S��+��a���Od����92�N@���������A:�O.���b��k�xi���w����k�~����k�[��|�t�6���0��]�3d��z�
����G��l���NH()���J�<�!It�R �Qy�d/�y������m��8������"�6��Fr�UFf~�����a������18Y����2Y���w�����������t���s�#cj|>r�x��W�r�tuFH��K���;���CX�1������;8p7�yG��� �L%�����(��x���9#d{>b���d�
�XedX���q`b��W4,��X�R?������k�)�E���'��a��I�*}����@�#I'O�T"b��p��+	5��kc"\0�*����@��anU��I>�;O����y@�YL9�Mb���:#v��W�}�Q��[��;O�W�e��%�~+A�x�o���+0�D�/�R����3�9I�4^I�s
�&��!n���I��5��!�0�;?���+	5�q�]���@h����G� m1^`�JBR�{@/x?�n�'-K����[Fo~��e����YUvO��IF��
� �����V�b��x�����D�d2�&"�����8��+	5�
� �����l� ��@{��N~��
�9����$����y�H��>�$*��Th5,�v�g�����q�<��X(���Ah5l�X�m>���e�[��9�A��O���W4�5�i�~60��X���d_�
�����a�[�y��������s�c��5>�5,�-�����t����d4'LN�������_K��y�%6����q\&��K��������
�pq���;O�\�m�a�N��&���7�a��l��x���n��h6���=]z�D��xm�4�%��DOD���6��$��l����N��9�nw���b���V�b���DT����f����t��r�����2t��2��#������g��ql�� ��G���g�NxO��=��zkX�����p	E��y�,"��������W),�|�=�+;�{.o��p[�{)�"9X#l0��;��=F��3�p��s�W����7�Clv��$-�~��I�2�C�?q�_���9�w�B�W��/����nY>�Wr�]llu(���=�de��S��e,P9�8��%�:�7���A�p�5��5�S�z�,��uHFH
kh."��������B)���gF�|E��1&����A,�&$��g_h[�W�r�]C�q,�o'd�,c���lu(W�rO��9�zy�����H
�����I���������mq\�����wH��%g����D��}��E��o��V�Zv�DR�������mE�Y����c����V�Z2���3�~���*P��':�z�c� �+~(GE�]���=F�D�C�x�"�:�U�FJI�a^��8���D��1�K������='�w_E3"k"����d�p��Zj�s�������d+��a:;~���\q <�{��
��m�"���;_�C��z�M�������-�9�D~�7�Z�[DS?���0���p�E�D8�UL���,k�	#���G�?!�q|qJju���S�>��$�����X[�#������w2����t��n!e��,�������:�rtN�����p���P�IE�Y���9�k��8��������Y��\R+�������9��/#�����)`][zE�#��y{�|a��=����a!���&��l�W���9������/��|a}[��v3^J�y�����J�ov1��I���H��������(-���.E]B�E����RL��R8���w��ZE����*�r� ]Dz�������[_�G"���K\��92�#�u�K��V��s$��1��,���
�	�G2��v9���J��	��..Q�N3d��SKUg���������j�%��,���_��e�0��.��r#�:��d^��%c-�����������>?��#�������wG��5��������@�<���x�\}��b7o���!��2�"7~�����GSH6$\��n�����8���mKH����H'z��C�����m�d�tDMP�y�.��7E�
�^0:��W��o��~��s�zd�_a�K!���|~������E�qz�W��_�$���[yN=&��������W\5�km����s����S��?�_���Z��G��_�������M��{|����������2e|�����$����@bx�y��V���7�f^������sh[S��]R�W���n�7���:�2�P�F�����U��KY���������X������L'
�T��q/�v��Q����4�N�J:<o����)pqq^dy��
���g�2Y�q#���E���I���tcm7EO�+�[���]I��s�$�r�ew�j�Q1PSM�q�.�Y�sU�,��*F���t�>������cA�:��O��s�m������sU0c�$�;���EcZ�cw����N����db�l�G��0
�u0��B
r��_���m�^G�|k^��[v72��)�����8v����*s�1�9�sMQ�0OAid�r�,���  ?����nJ�����K[����F���>
9�]*�`w���V��'Lg����|�����q��e��H_����R�-mS��ud�e�D(�����nb����]h�2�?��IY��8OB1�n��j~���;���l{!���m�\����s�#��4�J���9(
���8v���%�4�I9[I]�;������R���]��������)yh�#z�����,5{,���A�������B[��:N�t�&�.f����'`V�so�6�bB	&;���g���bsZ���u�~,}����2�[�]cz����C����B��?Z�M:W�'��T�����/r���]���b��m Z���]G����#����)!���8 I�M��L/�Y��3�Sg�3�Pq6EL���V8c����O��[��q��u�����i�
���g��qJM��,/�Y���@:,�lw�>UL��O�����hj"��k�Z��������cWB�*����M~��U)��L/�Y�����g��VLCr����?���0��hMJO!�p�bK���@6�'����\��`�J�B2tx��i5���6+���?�?�*���60���r���|<7���OR'�w��K�iq����Lo3�F�s�WY
dz�����n�R�/�>�s�tO�:���n�����a7?��������z��%���-x�8������B�>�����-��P'��]�!�!p��#�C��Yi�s�Q'�r��gt��S�������g�e�V�^h��'�.���X"���gV~�x�u���['s�T��R�i��,���])���^��5+Lo�������TV�^h��'�/�3�R>D�\�~@�\�d5*���@���>��z���Q����4���2��$�W����;�2��V���#�S������
��/8�3v��C~	���+��>�tS�wh�/.B�9$���?kMq��5Y^h��'`/<����+����z�C7�����^>���K����M��R�h[���+k���+"p�������B�>�yqdn����!�M��������lG�WWq�_�}��m
���~�`^��^��7���M��nZdy��
����a_�h>'zwD����b�;W���Y��v�[������
���\a�J���s�CY
dz�����9?W�B����2;�������;���5�_8���m�����A�yq����L_	���1w	,��L/�Y��
\xD,�/%Yf�B7�I��2���ud��.�V�i�%u��u����qq�}%�����8�9�2��f�O`^�b6��^C�E���O����������/�]��������������0������RV�^h��'a/:�8��(��)��]����l��8�y����p�cK��+��
��Q����V��4[�WeZ
dz�������,@%�Nr�sy�a���<G1e�'��1���^�m����E���X����#�3&_����B��?	{�`�����L��'��w������P��/�7��F��J����$����&�@���Y^h��g�}Z�J�-����u}��QY���
	oM�G���~^'�`����=��[�g��4?�e�Wu�G�����yN�H��i0���
r��Qg�yB8�{K���#�_��������������6+���]f�����j2&v;�jk��'�!�W��+����������j�(��X��j�z>-���f�O������K�uh@�}���X���N�����KG
��M����g��`RsO2��>�����,���f�Od�O��y������}��%�a�K�T_�B�z�s��fK�I<p�v�I�����B�O����C?*|2�@�g������jwD�����[���5�J��$�^�t[�g�v^�S$�[��saRAh��@�������|�P
�0�Zm��Pvx�;��L���X��K��I��jw���~�6WLo�����f/��@����9�x��T�����/b����VV�n���s��y8���3������|�
�[��t���V�^h��'>��O#�A�<w��d����O2_�4E����oA7zqM���/�aV�7����R��@����-���%?5���0
�H���JC��^�<w�j���oI���V�����#��^N��
��@�����.��]�9���s~DJv���J����s~K�4�N����<-.�0}���Q�tF(��L/�Y����i���j�b:�~�r����n���+��oI7yqM��g�����^/�S��b �mV�D�m2��4_�+�OVd��|pF��q����Y��������������
�Wg�]�3�,�2��f�O��jTX����Y�D6F���p�9n8.�w�sI7�v
�%X�\Za�j��.]>�j �mV��SR��^l�"�y���Db������Zy�#[v�[�M�Xq-�����K�~�4���w�V�^h��'p/.����� �T�}�,tq/�$�!�;��gb@#�]��t[�����?v��+�/)n��N��L/�Y�[�^�*U$�����y���E�z"�;p�{2��qw�����T�����a��M�����zZdy��
����/N�����N�1	����2��|m��Y�{
.��z
���#��W2}����S�3�#�^h��'A/�M�m�j�Q�����<>�
z������O{�aI7���
��1�����Q~��}���Y^h��'��������������.�:F����C>�>��O7�x�
����0�%���.��-����u�G��J�5I\��������}�����s�/p���%����aI7�8�
��F/,O��6��-
0w�/��L/�Y�[`^�rh�����k#�z��������9� �|��k�%�T�A�z����dz+�����4�2��f�Ov��<T���H�j<���n���}�w\
U�A���K�-���2I�����r����V�^h��'`/>�S��th���)�����,wzq��U��o���K���u����b�0z$��c?1dSiZ
dz���������H��) �/��bN�q�%b�>�;���n*���K$���x�_��![UM��,/�Y���`�4��0���Wl�b^�|���"�+�3��������|U�����L���?4�2��f�O��������!e��\�������"��Ov?�������\�>�:y2s|�2�Y^h���%��%I�~�������3W�iP����V�.|�nJ�Byq�D���������_Ydy��
��D�&��w�T��J�o�&�����V���}���n*��I\m�+���d{���g^
dz���,=?[��XT�v�A�O�$������s��.K����m��m#�
�WZMD��.��L/�Y��g�	suRs!�~�]���C�����n}.����:i���^�a��5I2}���((��e5���6+�����f�R��~)8��G �S�D�)���{�sI7�v
�%��`��+���
^~��j �mV��%�4<xp$U���?^L� �\��J���u�g<,�c^)���^ V�G2�
}z������B��� ���c>MIw-��a��mI����d�ZWY�.}K��^��p�2�����E����Y^h��' /_�%��4��SEm�;\�D�\l��27����3���g�yI6�jb�����I���V�^h��'Q/�*W|?5���n�ir������j���+���+�~
�������\��L$�=���@����%������$I��
�����*��u.>��yF������g�����<���5-���V�������I�^���g�����;�;I���]����:\'��
�%hWg�I���'=����j �mV�d�27�i�&T�b����>r��s���n=�������N��}^������m�%X�Y�j �mV������/D�$�4�O��?
e����V?@��9[�m�_�z�^4v�L_�m��-�R^
dz�����O}j3��6���3�ep9��H}��&st�K�����a�zI�z�+Lo������*��L/�Y������S��X�M���l�4���:�I�$2�7[Z�M�^�A�XL��\j�DR�17[���L/�Y���H�z?�}���t����w
���9���,�g��'������A���@0}��*�|U�j �mV�D�r��^�_����H��3����%�������u�
���G<H��u���PO��,/�Y��Bp�N�Q	�f��g�"M��3���E����-��:�`�zq�x��1�7��w�\)v�i$�mV���"����P��w4�_o�=)b����k����{~K���;�������m%�W,O��#�:�2��f�ON7b�t�|�s�B>����I��y�-9���`�����M��a;�HD>L_�?��/�m�j �mU��D�8;#_�W>��3��"%�c`s����=�%�T���_����R���<����Y^h��g��q��I��H����i2T������x3+?��9[�M�~���p���W��H_rM���,���f�O�^�>�]l4���~��w�`���#����u��:��bz	��[�0}����%(��L/�Y��2��OV<�:��I?U�`i�x�f
�Z��r�%�T�/6�T+Lo�Y;���N��,/�Y���z�J��1�z.$���x	����zk�j��<,��r~���X�+�L_i�b��.��L/�Y���X�4Z�A�_��K/�N�_��68��A4��/gtS���A�`��n��&�N.��c��b �mV�d����)��</eF���/t� '��;��9����oI7��36��n�4�`�J����+�&�j �mV����27
��u�|?�I&']X�y������gtSx��`^lr*��7��I���i1���6+}q���4�~���!�B��ud�5���"�k�3�-������7�$��C�+�i1���6+}��Os���8�sRc�A�^L�<�!v��Be��;\��Mu��
��'��5���F>�����@����%9���/��&�,U�&����0Mx��I�v�sI7�vH�|#���`z����;��R^dy��J���X�R��<�4w2Cx�p�����\t�A��i�����n��K
��X�`����#'�����@����	��
zj3��h�����������!�t���z�6ZRG�]����a�^az+�%FvT��@����	��Z���B+��D�+����l}�Y���7�]RG���N
��2`���v:���O����B��?{�������I�ac���@�Vs9��N9��
����M�>S{1�<H���g�rY�M��L/�Y���r�\�=��y�����A�����,���Mw��'�,����s��9�����W:������i5���6+���]
�,��`f�6�[�G��s�`n4�3�����%��*���&�i�J��i1���6+|���f�,ej����blJJs�gbT���>^��nK���<b��`z����9�|U��@�����G<�h���P'�a��?�y�b���]���������������x���O�d��6�e5���6+��h����[���]�]�X�<��v<�%@��m�+���q������/�-���[�K@�?��j �mU����^����}qu��t�;g�MUmY���[O4��g@I7�mI�^���������V�]�V�^h�([�pI�����:c�����k����`
����M'�tS��k0��1Kp�
�"w��H����B��@��aLQ�������[��f��.W��v#tI7����[~z��\_IA�z;�r��v%P``	�.�������.���5���1hN�X1hV��%�TP�~���H��
�W��Z�+�N��\/�]	��_���R�ST�@(�~�3T��
@2b��[{pI7�[�7��\o#11�`3��r��v%P�q1)[� d����.��$	4E-k�9�����%ut�u��g	�'8���������@�����G<�/+����]9^������`a+����>���%�T2P�^��Qe���J�����j �mW��0���g?��%��������H����}�Iv��,������s$��>"��FbG���TV�^h�(1.J/
>]�� C?cO��	y�H���h�w	\R�
uW���@��H���@�u�p�i5����+�30"p����mx�t9T@��C��/�4����x��Z��}?����.����|���\�}�-vZdy��y��������*�,��v`I�����&����=�0��*q���,�)�3�W�_}|����pC>�Ww�$s������7�O�8�������lp���~��)���?�?���������*�)l�s��~�C*�w<]�W��=?�9>�W_?�=�?>|����CY������o������Y�O�L�z�5����������p'u6������"��O�\:@q[7�G��x����e���3���Z�����R�^.�V��(���C.��K#O�����?�����S���]�C���(�=^`>;��2���0����k���,}s+�S��������O1�1���k	��s6�+u��8=�Z�����������I���/�l��w���|I&��}�����
���-�o������c���~�7����t
yJY��Ya�����'<��O+����g��LrV_������v���{�������:�cN��s:p�T���c��������o���z� ��~����o8�z����s7G�)����0�R>����PK��X�,^�PK�z2K18sept/pg_part_head/7_1.outUX���Y���Y��]Q���~���7KUgA �:�J���l�''��'�Z���$�nvW��~��3���f{V���8�X����k���nW�������~����G��������l��������������"�G��Ak��/��n��~���~����W�=�=�xu���������+���&a4&����w��7�����/~��}{{����/����������}0���Z�__�����n	����rw��������������kY������'��_�~�������?����_��������wo.o��O�������7���7��������j�4�n�����2���5D������������.=u�����=��lZ��������l?��s���������w0?��_�����	�����)���{������G����@������w���|�����\m����q���-���Wr+S|�o_�o��?���uo���_?/����������V~��'�S������~����=&O���Wo.n���=��������{�������?��M~��'~����d�z!h�^��������Oo@Qt�/_�����7���q���`�<FGh�=�5P��Oy�4=%2��K���xep��=����ky8�w��]\]����9�������7���e?�4X(7���CLd�x�>$��p���j@�_E�=��=��=��Y��ps����{w��5w�����������^��+��E�j|wT|��9>������`�u*Dc@%}�p���~���x�n�n���o��^]\���v9���DK�U�M��k��|P==(y�\�y@�k��&��Nv.������O�n�����}�_������������?.���������O�&���l���/�%^
�Y1��?Yz�����O�7j��������}?�Wc,��PD��<@����<�XW�q�h���?�|��<��#p)S�-�<��	<���b`���(�<[q?U������|�5����I��
���P����eA��/����*�a|~��>����qA�W�F��������FM���q�8H�[3��DJ�'��nT5)5��	|�9�B�
���$(*��&�!��i`�X� �,^vE���O���Q�#��V��~�� ��#��U�	����G�1���CA�w���p�g�%��zX�{M���V�t�����	��z@�g�1>������~�P��u$�I��`Q����.���Y8�\��*z��{f�VtO�-/j����[Q9 ����U>���oR��
��������
WXV�h�����d)S?M���TU��8�cu�q�%�PT?>��i5UVH��1�[}�����&�3�������s1���x6&Ef���B��?`���`Fx�jzc���}9�F���7<��#��TZ����1e��^��C�/a(J��IU��Bj���&a��?$��|���
a5~T���(������+[��C���&�"����������Q� ��> ���&�+�Pgt=��$�4V��9��e��XqZ�Q��v�a�a�$
��[�-z>�]�	���"X�1�#_Z�����(�KdB�����cl� �Z�9�u��h/�n��6�����'f���Q���db�*�s���~8&���A�&����#�
Y���\!���g��Z7����<�H*�t�.�<�D�QU�;g�h)c���n�q�(��/S����,�"��A���fn��/{-Q��B����aM�������'T;�����N��!������T�=��\R��������=���C�]W��/0^�2����LE�z�5������R���.��x�9��[_����Zb"�Gt�pp���?$_�9��Dy���Q-�^`��	���c�o���f����N�u�%�X.�M%A�_�\�3����m���|���t�&��������Fw�<//>��=��"��U�|����`p��w���'
Cs]A��w6;����A���&��s��������?�g��Yx��W@��&c�FM�����A�kp�Xa����`-�G�s6�Z�����7�:�>$��(�.��/��-�qZ�#�A�e���f'�5��:����X=���J�Q�2����]v�B��
�/\�����L��d8����}	~�K�����M<�W-q�y��f�cJ�7!W��YtOJ��
h��LU���k7��:���N�x�S�'u����$�N�'�"�5�N��^Q��s�!Pp+�~Q��u���r�d~g�-/�P���-bNa���LE�ew����D���p��� ��c��Z\`��Y�=q�2���2�>Iu�;	8����m��+p������"
]K)��x[t����?rU����u����y�_���VFD1!���&{��������><h�O�u���'�:��
��Wxp���85�_�0ty�]�E�E���kw!1���}3��Z�@U��_mP���3�h����&�C$��y��I����:�_=U=������\�Z������ ��s��l�C��p������_�}��_S���A��f}��$��\U�3���(o�
[���rQ>���������N�N��?@��|��>���h�/Zy-KAm�����E�bU�s�{����������1�:=���[�<����`���<c��c���MVH���
�����s��un<��S48��F���V��|��6�$*2���H{�V=��s�+�~�������|�@;���LLc���6=1�5���^AT����]��$2j�M�a,�-u��Rjz>��z��s�$Z���k9z���J{�h��Q�7�|����F��^e2��j��At1�NQ�����mw�b��T��z��8��1O����E��Y�Q}����t��&C>�����u�;8�J������,R�����dlQ����zyNz�==�!����^�������M�jl=�6YW�y��'��Em�7���J�f���z�Z�#!Y���n���|xBz�b"�NU��+�.�^��&`�>��o��$A�:�`������b�T�zx�)��5z�
b����0�^��Xc �{�9�tZ�<5)fd�5ff_�LWp:����w�WL�vC�����"5z���4��/��7C�L�=a����i.-~Rv����Z����s�g�Z����"�Q������f��6��r����@�;��������s�x�<��f�Z�U��)�Z3��}�TfXr�8������f�Tm��m���0�^8SnkS*�c-���
zv}�hi=t��'[��-���aq] �>p1��:G'�!�p���$�Q������x��RY����+�{������
.�)��g�����%m4��s�	�(��h�l��gX^?����E��j��8��)��`2�}L4��V`���,�b���Gk�s��u��EB[��Rw������!5)������6��w�r�|^����C�6
�KMD�DU����FQ:8���_�������+ZT:\��h\ �D�f�p(�O����`�8��s�-:��n?.[��0^�m\
�i���zr:jr�eHU;���xrN��[��5]GH�^��2�6'� ���~Nl�{��������h1����|~�����B�6yc)s2�����s��v�f���l{�X3���	[���Q����V���?������d��'�TH	[��<��&H�
M=C������9�����aM[���6R!�i+�4��|��Hp,=����n�6�x��l���A�kAd
�!�h0���m���������������-D��k3�C�I��;a7�x-���6��+���Q���d4�&��&���u�������<��U��)O������Y�h2����t���������I�������}m��W�����U���9�M^'t
_Tz�h�u�r�xr��l5���kb?'�cF��X�\nj5l
� "�6�'@5��`J��0�
u�WY�;�k "f%:��T:Xu������k�)��Zd
���z%��m��qU_�K���5q�#3��t�)��h ?u�M����.�I[-������W&��[�z�9e5k[�d���]������)�>��R1	��4)/�m�8��S��.�m�y� �4yd�/��&0��v�� ���.�s~�MCp�XHa-�y;(W�`���O����jR,�����Z���4��IW�Q\����(�v�*�s��e�B���
n��W.JN�Y�G4[��P�Jp�S��A�k���P�JJ��zM��*�^��Chor��Q����T��pq���x��(����m���5�������tV����*a��`P�������Nd��$��/�_������ �{�q����_���n��������x}y5���r��0�!Q�NCUpt >�v�W���__����ps��rk�^^]�_^_���}��s5����_o=�W��8�R�	�b���8/�eM��$��H�K>����h0��JDO�j����O[B�D���eL���F���?Uy�
w��k���9����5�6���{������������3QW?��zsq�t������x�|�����E�R������P[/n�v��r�QJ)E*B�t���E���kY6
S�a�&G)���$	����-�uq��������mA�Ku���Q�&��J���cV��@=�>��_�o��o�v?������;���������"���[��b� �Y�'8�����=C��CO5�` �LbK}<��g�t���o_�=���d�_��pw��s}w�^���������������S}�/�G�c?�t���c��p-����N{���
�{�����=.o��������]�~=��5s��{Ty�?�r�l��1.Q�����RP�G{�Z�$*��0��n�@�c3�6h_z� _��-gKu�-
h�,�a~��Ri�T�R�6t����`����.u���������.5|�R��C��m��6�?]t�����CF�q�������\�~
+�TZE!����dg�u8z� ���^����[���b�i!5�%����u"��=�r��K��D�rS��	������6��K��rtS	]2Z�o������rq�\���C*o5�t����n��hi�1�j���t������~���|���X/+����E �H3��Vc�l����`��%AG�h��%E6����
�v�����f�s����8�����)h�=�(��m�\�`����A�0uC��$H��w��	o8���X�d+�v�����v��������H:cN�Q�mt�[�k!�
+>C7�$K�=�r�v
��Z�]�����=x�Y����n��kQ�n�W����,w��%r�J�W�
�%c����;�k���}2,�d�|"��4��]�|�:o��'d��Na���Z���C����������h����F&�v-���t���I����`��p�a��"s]?�-�,wn������%R�O7�����i�sEv��
4���d�&4����)�]�X{�Gz��g��,����"[�?�<�b���Hx5����wbuy��c�f�8We^4S���������������������V���������wA��9�`K�B�V'ZV�q7�.:
�����0���x��6�
cO�s�qH����6�bO����M+N����-w�G���#�qS�h��M��,$.�V���9��<n����iKy�T[;��e]
�t�m���e�x&�!Z��>i��M�Ks�Y�n��I�xl���h9�������h,GG�%���Kv-���W��zC��q�0�I�����k;7�L�����w]�=3y0I���m������rO�>����r���.�Y��]������5�b�HT���6Bw���9I��e��It���(�>�B���sN�+�7ZQ���F�m:�Y�+.a���4P`����Qq�t�q�DN�*��&�tXN�F�c��"��5D�����F�ct���G�b)z]�P�����Y��>�s�-yeA��FN�63�1���������[�=m$v���tY1:"�B��1%(��r��I
.E��9���n��M�fb9��]u�4Ia�f|m:g��o�bh/��8D�k7���L�C]EN��9�6�t���p$�"*�T��m������z��~{��+���o��;�/stW��Lh}�6.�t�\����`;T�'���oc&�c�py��V�dz*���sUoN'�Y��_��k}|J���G^�b\�e7��������i:/�@.�}��\�T3�8��V[�j��x�[
�p����`����:k���8j�q�;d��I�n-���8v������C�	Vk���`����F�M���1��M����t����i�N��������B�-�����P�>$}v�^�������N�9&�j8������!5����.��Xs��aM5h����6�tZ����d-��dl��M���y��.vS�AR	]�p�\�U�n��Lg2����yRa��^�7*���1����aA���������i��>y�)qa4�\/�[��M����{�_�A����6���6��k��
���!�^��F{m�����bk�L\b���]����\��0v��X��F�r����i�o��oH�����7�HO�e%�j���O���T��a����r������F^��'��&'I��IQ�N���N�����Nk������e��xe����[v^�B,v�o�:|��H�itn�VBe:K\��F��Y���{�<#��p9j;���w���	�/')�;��������u1�#�o��j���M����ic����q��:ZU���m��.�72'��/;�k�F��������f��-���^���d��N��'�:������	�f��v��������������+����1.v�-���������O�r�������JK9�����}��Z>����Qzv������7���_��8t�����D.��o���M��J����w#�@���������l\�'5=<$ �?��-�_���p�����<t�x�������� �69�t@\�����������wy�K��('�����v��c��Cj�ob��Sg@��	v���>�_PK7u��7u�PK{2K18sept/pg_part_head/7_2.outUX���Y���Y��]mo7���_1�b�E��������f/���],�����X�<���v}����g�9���M��q�cyZS�t���7������wo�k����^�����������m~��xq��<?���{�<Zo
Zc��&������/a���ww�_&�b�������rsw�q�2`cc2��>��E�WW����g�����l�����~��o7�����s_M�<r��b-��_O��G����������o6;0��������\���;��������>m�n������?��./v���������|{���Z?�B> ����}�)��_��5�m��j�}�}�����<�18kB�K��,�������`����"S{����i���W#��
�?m�s���������m�O��W7"�m����u
��n�o�xu�;�d�
	LBE�E��Pt��&	�&��T��A�S
4c�j����N�d�����������M�����U��������_\�mo�w>�=$�=_�z��w�����W�P~�>v�b����m�w�O?���W�������f�{�?1�+&�h� ��Ak�F��o�m~�t|���~���$�����tO
�1<F��N�!Dk��a"
&�E�I.
(8�<�8>�,��_�����n~�<����\��O�xv�;����U|w�`��1����Z�)X��i���c���A���7 �}��=��{n_�|{}�����������O��z������w�{����#1�����<s�L��u8��`�^e&F����6�S�����������r{��e���/�g���n�cp,7�#e�R�M@�2�1>E����.��<���[��o~'+�j}mA������vyg���c�5��}��G
0\��\W�c�w#��hz�������(�����<�^a�
�W�~�����!��7L�V��g��OVFo��z��Z��bY��������^h����P�$�PJ�ld��hK���G���+�`�`,��%*��j��cI�ilh�a��.law�U�!���qb���5����;�pg	�a����9�#�>C4����J[��EI���c�����w��	�����$���LJ���J��F|�A2�zF�`���` 	��)I���-P�������#:oI����c�58��|6�u,��!u�#��I�X��XV�����hg��|V,~�����c����F�C�j�8�%�����y���$���C#G��hm��s�����Y��������Q�=8�%�3�;^y�=5>��V
��5�0�M���W��5��������p(
�(�����.�q>��C��7&zy�2�S%�D
Em�(�u��������-��P��=��S_!��m*	�(���N��O�	������dd������d��pW|�7��w�	���~	���gX\��z��k�����dG�Zh�����<���0��
��y41f���5�~NP���q�+��������O,l���~Y9��\��%��MD���������'Y)B���K���*k���E��J�%V��uj���Y�r�hF]�52�����"���|�K�3�,n:h�G#�(�~�����%�G������@W�l�S��#	���f�-����"bQ��Q�f0�/��'�GGU�cQ(h��Vv1���g"���o-��)g6��P���:��e�{��5��BQ�;�z��,�
^����;�D����bF<��Xn	�J��8�%
g�u	���^��������f)���^�~�IEI�'j����E�1��r��~N�4���oC��' �4>~"�W���6�Y������rKF<����d��OD������g?�n6������O�|$��~N�(���S���l��<���
�f�N�}��GJ&���x
����h��q>���
G�?��|��m�mI�������3�_����C�F�{e��k��h��Oy��}���Q�#1V��=�L5F]��d����3��g9�U~` ��N�j����n�L�%y'�y'!�L���g{u�aKF����2��F��Jf�p�;=� ����.����/'*��2�t�_����A��$�C�	BXD��E+��P�V���A%�Y���F�)�dI���Z��NY��f�����E�:�:�i_b��	Pe�}�7�Z,`saLi��em2��� O�(���d��h�'j-$a���xa�a6�L��������z�O�jD�$��a�+�?B�e�E��,��bK�)�����.������-&8Qh���3R���tB����q���;���-�X��'��@Y�=Q���tV�Q����]�Y�	s�5v	l��?��hb�_E�3"u)j�?B�����W����~��R'$��}�q����4?6�v|������$���
�*�Z��IP�&��A-�?`3�A��y�7b�v_�DI�'Z]^`�����uo��5������lo�P�q��9e����y���X�5��u_c�k��r��N���n����J5��F��S�Q�2�CT���-��=L����@���~a.�������Q���*�]e+	��������j1���G���&�@���WX\�U��!5=�]�����g��YIk�:�����n�F�	bQ�9v�����3�y�g
��u�(QBV`�6�H9��Eb+�����E��y6��t���(��f�/��F��hS��]�)�S
Qf�����&,�� Q������1�E[t'��]!����>��s�	J9�!����:9�������]�p?������{��N�z�X�O���*�J��<4�R�v1���K4��v��d!k3�_����]��H�h�������YNT���it�O��c��?�v�f5�=�8&���9>1R�f�U`��F�Pe�'��xib����R�4_�eR�MeW[��>~�"S����]��^w0��A��'Jhz@	B�X�3���h+���E�X7����~>m���s&j?�������yu�W�����h%z�l��X��hK�q�7�������N��%l����m�"�4Kx������f�M���H���#�cLY;	���S������gS"��2�L���-v6'��gx�9v����R�RR���P��[�����{o�qkb�%|]��
��c���}[g1����z/p�3�����B�^m����52��Y�"�,�r��^�c�\p���c��Ah~�4Nu��%[{b�r���DN���`s�I�����(�f�|>�{�������(����F�@Z�|��d���q��L�5t�����}$��u:�>�1=l�Gq�R�lvt:_|{	�u���6G#��=!��>���`���~wnX��������Q���zR*��q\��M�2���1B[y�|~�b��������8��|ob@;?���38�|J�}~�M�R��(����6y�'Q9��-�����jwmv��KU�{@)�9��6�t�>�&a���s�C��SV=�\����[B�D\�9��������f[e�O�j��\6'�Z������\Z��VO���N\��LKNE��Q\��h[��{��^y���6��<������6B��4t�����$��7dU9�1`�q�q�^�t|{�1�0h>Gs
:f_���si��M�K�s��^��So����~������R6���lO�!��)��z��Y��@H�����!0.�,�Ve�P����M��6�y������)���a��>����*����k�(��c��3� ���U�uFO>�+������>M�k�3���C�?�����-��m�w�W��e��M��6�6:e�����N]�\����V�N��=�����V���<�6!>�k�c��'PB�������i���L�d�$���G��U���y�H�u�Xt������~�3���B�}dc�A#��i�:��$���V������J-���{��>�}����@(�C;�����(����,'�N�b��P�����H������?D^b:��
1yB_��P�h�@E>��������),�t���6q�NU��Ce����0��EN��z��z9�)6��p�5���d|�X}��<��[�����s�i(zX4����K��)H��4�V2���V}�������(�cba���U�v��g)5kL-�+b���������BX����wB�M��8�Xc
E�M�6,�����E
�4��6oX�5Q
p�c��@�F��\\LXl�����[0tH@�g��|n�F!�rJ�������XAB)B�����-PN8�=XtoG���a�{����W��>��~{�a{w+:D���o����o���$�{�i�������Yn���������xu���ZV�#oM���,�}��g�m�j���������������������]\�>l?m^nv`v������eu����(@�
���4��7��h����^���d�����:����J��q������:
��h�y�K����F���>Uy�w��#�����ew��z���q+����7����?�~%������g7/6z��>���W�������)35���'6���#���^GCJ)�YH���D����Z��s�a�&{-"fv@�&�P����_V'���������\�73:���7�Bu�����"�����l_��������|y��m�������xv�\���p�F\y��$s�g�D~����'�8	uL�ah��R0��:�����wr�o�7������w�9�wW���~x�oW^]_ow���ELw	��(��&�U�'L�Fb�v��n{y�������7��z$Fr��^)�z�2K�N�����?��n$.d8m�WG���MC|�L�AR0���9|w����9ae�#Y�LT��N�����mQ�HT��:RRE���O+*}���2)���UD�/A�T�k�{ZQ����T��A5��k�wr5���)�e�,�S��)�����F��q*O,����~�����*�ZX�hA��;�^���6R7-�x�-����u��S��rt����$���J�reH����+dqq���1_jqa���zL��+-5�+�����@n��(�8 ��<^jd2+����SKM.!a5���Y����!�S��!���W���W����_���U���`�u^-�5 �UV���^M��#j���Bcw�3�u8���j�?ZXr��g�������j�[�%F[���c�zS���m���VBw���2p�&x�Mh )�k-�	���v���22
N"�#���i�?gd���q���]Iw'�Z������"!H��V"�n"X�F?4�.P��4�7~%x��[&�n--�5�E^�]�O�	��(�AK�W�M���%�c��t�qnb�Ib��Nf��-�co�a�	�u���d���.�����!�~��q���X�%t5�$2n-e�4d>'
���&��W�-ql��V��pbi�5����<���,��-_n�M�2X�-�:�������/�L��%Q��$��9�F�\�����#�$�d���4�Hj�����8a��:i�.�����N�s8=/����	��q8�j�0�N��0��}���;��R�1����	^JVO�^G��W�xz(�� %r_�	��W�lxe����A�5���
����@���]��bwWw��������.����:�n��6<{��S�]�H�J������HA���=_i�Md��9
e`�	��@��mu~by'��5>��@!@'%^G��h

���]�=�T[9��{���Z���x���t�N�k^w�r���0�H�U���5n9��9q��G���O�����q�y�,.���&�,�u8����}j�������5�l���;��|��"�����V��[k��Y]��yZ/.�5er�g['�0<��-����HtZ�d0i-e�tl�(h�e��B�2��?4<,s���
�"+��Rd��X��t�V|�h��z�4����tw�K�>"��D����wx���e$�Z*����IO��u�������=��^
�I����uJ��s�����%q�FH���&�G�������:��9lXIw�{��<YX�������`�7d9��s^��k�Z�,�:�������r���@gaL����w��Ys�Q��R�^/Ibw�I�O�j����7lv>:%;��;����]m0Q78�"���V\n�*�(Bs�����~;r�K�jn$0I����[�b�7�Ky���HK������2m"��pm��i���R+5��ci��'w�i�BB5VJ5O0im��m
��������S?����0�����h�:�wxP�>x�����5y� <���d[w�Ydx �$b�:4rx ����1=O��$&^���0Pq����6�n�aX��
������d�B@���&������#�Y��#��Z1����6�����ZT�x^��
G�����A[�t|Z�(8c�'��ir�L�mSeS�����-�.��K����E��N\1�����pj�A�V�����N���8��^�9�T=/W*��2��s��4���5�������v_��������(dy%o<�����j��l�/�
��
�m�:�/wx k�2��R����3G{��v56����h%�'\\�A�K�������������2�`�J�/���mwz�������4F33�{�����=S�Aw����y8kbr��.���H�$$����&3�m�E}�W�I�O��Y��Fc��%����~t����W=q�	E� :Z��v��
�3��F�����.��sz$>v������a`u�����3������o_�t�l���C��{$rm=|�`����(A������[�����"��C��*�g/�^���~h�����~� ���g �7=5	q@�NC����Ph_S���F!NK�b!�\�OEl��]�~k���r����g����=�o@m�-2%�����>������PK&r�hT�PK{2K18sept/pg_part_head/7_3.outUX���Y���Y��]_������7KUgI�:�J���l��$��'�F���$�nvW��>��3���fkV���8�X���
���nW�������~��������}������/�_��vO^]��A�`M��������w����/_���&���'��?\���_���e6o<;����?^����F�|��p����v��^�y��/��_.����&$`C�O������x�z����_�����������W��������O^a/o.n��|������������^�������~������{y���F��T� ���_��*��\��h������i����?^��9�[�x��Y������!���z-$�����K��i���WG��
2~���^�����Wo�:���}+d������N�S���������WG.:��brF�!\�:3��:=&4�Q������xiH	���������+��){�o_�o���}��~��~=�������������|��'�C��������{���?�����Wo.n���=��������~x�������?��M>����E���^�v/������������r�,_�����7���i��H�Z�3!��1�d�b�0Z���h�0����N�|%���@8�{���/��pn�v?������s
�����W��|���"�`�ig�.���=j?�	�+��0��(���^�� ��C��=��=��Y��ps������rk���<���������^��+��D�U������Sd:����e���g����;2*�S�����������������������j����n^��qGKd��u����
q��Gx��_����7rr�h���������s����k�6����`x�	��#A�^��S�P���Q�z��r�J���'���:���`��+|�����_�ZU�����G��+��E�c2��)yqVh`=�Cm��!��rf�Ol��^
\�� B��0���=K��lQ����<�9E����(O��yg�\��wxT����x�5Os]#�����kb4V4��yA�8�ya}�����3��`����(/2�����d�s3������xaFM��%�Jq�x�b]��bI���B3�Q��� �����O3�R�(H�_�\Q���p��	p�w4�L�>��]�����SoIP����U:k���F����[�:$X�&�-����!�����<*���A0U���t-�WU������>*�TT>�-1�/
����m����j�i=
���D�5��hz���`'��f>>��U��|f�V���7h����<*���v��
���a������E����0(�	�|4����L�4�w����q=!�������'X
�9��_����G�A��T��3����D�j@�X~yT���dE�{���.4�~�&�r3�+6�/	�]q<� �.Sf!	�~p�"�b��1e7G��n� .J�S���qD1��6�|B������|��
^50�~ W�~-�pQ3�jzE��7������~��zA9*�����s!��/o�?S>��L�HcU���^�Y��y�UE�'��Q>��AtH�E��sM��Qop�c>���*+�E�!�d��b��g������^h�z��)p��U=�Q��t��d^1����c��q�1�NT9�o��]Q�P�)nOhs1CU��y��������}��Q��j�M���CMr�cU�;G��b�x�n��}���r?9�4�@��L"����9�%�����`���[���}�F@n��B��??�I����_��j��w�E,k��N5��S�Q��j���jjX���2����kV�N�cD>O2�
�jf�`�Fo�`u�8�+�Gx.:Gd�����zn5�}�Z��s��Y�,�`K{�������&Y��h���c�03�=N�r�^�sr(b]!����a�(R�a�M��1��>�=NOd���XW��$J
:�'���dV7q��T�����_`� �>bY�hG"�r_P����djm!QM�s�S��Y�!e_�n���4��j�y�MF��5=�0�8�>�
z_C���B�y��E�g�w-���X������	�P|����$v��G�&��E�2�{j1��"�5�?�8�x+Z ���)��vEi]`
���
L7�|��5]�0O�h�eU=�_q�!Q�X��\A�d�S�hjr�\�.d`+������joU�;��5��>j���j�3,$W���uq��`�o�]�])�.�h����t8�����n�eF��rBQ:�t����
�r-�\�\��$,�V7�>���tn�*-����"��� �b�R���tE��9�D��H����(�s��<�-��E��r���Y5��2 h����n�V���]�]{��TU�q����X��h��@G����NP������*49�$V��8O��������h �:��4�qyZ7�&��5��PI�����6�a����%����6K�r�uA��g�PV=�D�{�����<����Y�{�X�j�#�!A��-D2�Q�wF[-F�S����j�:.4�Y4n�a�����}�S���u��&�.y_U�������XDb���t<hR�3b0�%=�&��.U�<i]�����r1��oU�sT��u�Yn���bU�sv�!���3aY�;��crA;�B����=N���_h��K�f���H&��j�5Y�FUk�h�v]L��1�5bV���Vu����k���V���9�/?��CZ�U��^�BWR��������#��&��h�3<�!���
�0u��u�8�s����5�Z@��Y�"�P�[Q��XC�k����oR����E��]U�a(��"�b��	����D�(���� 3zW��S��5�)n|����z�� �;;�#.g�YI	6��X�_���t�ub�rfW�_?�'8�V)���SlQ�����4�*�8z����}P]n�9C9�u�M��j�m���G�����Wf0{
�A_R���gU������:��&`7X�$.��MWF[��@�v��l�~HM��DW������bH��-r.�&j�S�#��:��7	����g7�}�)�'a���x�{���_����M�	�vl�_�9hp����S�����U��tZ9��������aA��pIpC���k�n-��c�������9���\6��,V���d���mv�R5�����&��O�����9IGg����)�d�������L�������r�N�p�6S��5k���g�w���6�r���{�b�lR����Z��q�s�����L�$�b������|$�LXu��p��R����u����US0������E�Sp5yg��KC(Y���5�^��d��*�����RM\��6��f1�U���`d�����w�C�xN\��>oM�`���U�+�����\0����sU3�qm��Oy.�D��fZ��'M�9��Uu��F�9a��	�]���@������'�-���p����+����5���������D-��\7~p��U���k��n�58�s��B)lr�%��c�������-��Nm�SiWn>}����lU}?���a�SKTPy_�1g��W��9XC�	�q!_�@����,`�c;?�u�B>�����P3����ZFo�����*?�{�������Twn�<
k��B�Vg+�=��+�V�{���`
E�biM�uUgM��qm�h|�Cqe��hyQP&�|k���;m�m��\7_�f�V��+��\��ih��E�b
�m6�B�����|��	c����������z]g�"��NI3\�N�{�������!_���1����#�UU��p��=�qL-B�H,�']��Y]�k2~O�k��i�kC6�4��_�!{�}d����XZ�:G\��:g�k��������B�r�,�6�{hr���6���9���[i������M^����1S���5c��."?���Y!�::]�V�-����M��.�pv)c��
|u�d@��5����T��wv	�Z�02�zHG�aE;�9�M���.�r���k���j�|�d������@rU���3>c���@�t��J 5i��"6$C��L��nhg��>e[S�LtT���6)���4�����.���6
���W���W�s5�P����bhA|���������������BZ�	��]np[���������.�����jp������ZL� W���S�;r	�.s11X.���v"�h]��a	���>nA�.���R�����7G�:�R���?������F;����	]^����*��O0��~x�v'2D����o/�_���/��$�{����w���r���o"e�
����j�k!��7��Nc����l�'�m�������k���77�.���������������ovW�\��x���������'*���Cj,.T�;����9����O�$3y�&�0�6_~�b��8���H�_��n}Y����7���i�OU��D��u����1h�yw��F���~/�������O?<&���7��\�>����.{�/�=��3��hS
9����OoA������ ��R�Xf)z�Q�
_�����I�������;�$)L%����uq�������xl�8��t�'��x������E�>��|�}���\������~~wqu�=v��?^����+��\tFnY��!��tl�9����DP����*��h�����R�G6��e:W�����_�'����>����\�~/Z����g�r����������Ip���pD���6�!����5���vi����w����w=R���������8�L�>���\�M��S��\?#�ex�y�_1��a]b��!wLl��p���8�5K��'^�:Z��0#��M��%*�
�8#�'{R���K*}�������9�B�U-z������n����	�i�_ql���+�K:�
�6���%s�i�����%r��������`�hu��"�c&����!wn��+N���V �0V���;�!��!��Mk�Uc�R�%���
sr�|�\��Q�-��!����Q�+G���T�G&�����^v����j���7�K{dry~��$���b$�������H��"!#w������������[5qV|nQ�Z�J�"=�X5�`����Aqlp��!w��	l����5��a�����.Z5��f&��Y�#��l�8�+YT����F�0�jZ����k�j�{*7"wn�D3pn&x-B��i#aX�j��\�����:_Y<���]?�j���=��v�B����&������>�2�(G�����Hv�Z6�����Q�L��h�V
M��ZZ��i��Y���
�VM'�g_Nk�L��sZ!�
�s�������5R��*��.��C�ng&,���m�]Td��{]���0
�K�����x,j��w�G�n����"��g����������0Wd ''���]AK���,��-_��pW���o$�Sg{�L`�
6�ja���8��\�B.;�����������Q���3(n%�A'6)�w���M$a��HXp���at$���0�j�3��F�����`�b��dw��%s�mZ	��n��v��[5�G���]��
wqn�\6��;j%a���g�
+�!w9��;�Xrq��.Y
:FcE�V-��s7��+���-���m��t��C�������8��U�J�=?���IrA����w�#�I��D�X��&�n���I�����L���m������s�?_c�*^�=>���.������SO�v�&+�d#p9����j)E����!i����1�m�a����!�D�������w�NwLE'�k�;m�6�G&w���q$^��%qp��H/1�QL�������%��������rD1]����&�a�X�H]�t��6�.X6oN�tk��>�i'r���7l��'���]HZ����W6]�y����w�+��W��	a#a��5���2l%��V�9�T���](�� M�0`����N�*F����t��0-*5�o��LW���I��
x���.8���2��5y��e��z�P�fZ����r�Sv~rMe�Ax�;]�v�}�e�(�`[�lr������Y�Z�hM6]��{���.�������tK�^�F
�b����o�v�.����� ������h#E�`���*�:��7"w������|
w1i������\�v�EW��iwA����G&w�)�Q��(�c�a���h�.�!)k+5�$�S�6���b�c�g��uz����g�n/�5��E
�����������C�4WK�#����8������r��Z\�k�W���m�8����J�A��I]��N���k��b8T�U��tP�\���xr���\<;��i�D������<
���p�a�6�m:$������	L����7"w���3c����uV$6�
Og�t�YpM���t n��M��|������7=k.�`�m�]�����]+!R��4��	�M�d�5D���7B������'E�+^�n�KVS�)���A_yH�K!i��VU�H�mT�l|�J����|�H��X�d�F������_vn�"j`/i��F%e�!�}0������W�b;�6�C������r�����e��������'~%K����Ep��N�I��s[�}\��l��g?{�vn�!jh�5�Ku�NK��rq\	��:g�[��`��99>+%�]��a���l#������e�#���mu�?i��l���C��u���
�h5z��3��V���<�x�P�h��DV3�t	5��ar^w��N�[�i�0X��u�i���S���_�8�l���tC��
GN��z���P=Q��������_qo�������O>`#��4���M����~!�����7�4�p�!���:.`�����4�t��d���t��&��uyb7e���������vV���}����W:������"��~r\����/�PK�����;�PK{2K18sept/pg_part_head/7_4.outUX���Y���Y��]mo����_��,�FWuuu�q �|�s����I���EHZ2$u���_���L��4�Rs.����]n�35UO�t�fS������m~�����{�����o�/6?^����l������&;������O�����~��_��{�M���'��>\���]��~!y��C�O�??����Z���A���m6��\}�~��o7��.��K_��]0���b-��_O������_����~�z��{��x��������^^_��=�������������N�������n����`���}sy��}*oIn�.�_��wo�>�u�m��j�}�}�����2���������������C�dY/<�D��%������:�o ��i+�{���o._���`�i���F�r6&����?��������W��#��6Z��$U"
��<E�Q��E��;z����E��N��_��J��\������~�|��~��~��~�/v������7�7�<�o���/���������3�)�^�zsq�t����m���O����g��7<����f�wN�1�'f�hz#����������������Etn���I��{����&������"z2��|��n&��f:�@Dc���������������_�����n~~w��m���9��������{�>�����������_��[X�h�$����E�+��@r�f_������������?�������/���z��������k����c1���VO�9����E]/�d����b���p*�]v�������������/��m~yu��\�69�r��n�����e�$76���{�
�_����w���X��,����������������W�_��m`��B/�:���������}/��"�)u��1c�z�����~�=����1�������������&�����MAnik�����z������+���%���S�#z�2�������)�bA���,����I����<B���Xw������]�BA���cC~�~�]�owWU��� ��D���pFa� �����*��=�����9���Wx,8V
�� a���I�A��&�iB'����0����{�G���ojR���7>���� ;!�Vh��<�e�#�H%������6/>�E��#��&�C��)�E~#a��	
�3j	-�vPU���,��~^�=b��WF%:�a
��/!V���5������>����Z�~�L�c���|xH=��#��������b|B�yI�
�G
f������ZrM�����?	l���C-�r����i.���|t��i�t����"{g"
)d���s0�$�x`v��� ��=���T�I�������qp�����&Z	�\����,�3�R��&���Kg���1�J/��cM��S��| �q�j&�a�j��X'��),��l(����8�h�C������f��`�����&�;����MJ�"�����L���d�<��D�
:���b��w�d�Pt���O�]���j)�]K�[�x"��^>,o�O�E�����O�]k���u$h�les�u��}�� �A��&��X�����|,Y�%�U��x�}y*��%UE�����z���iP��Y����L��&��\U�3�Z��}~�����Qu?��iHr�HM��������$�d�u��6�������	�����	j}��f9��.�hBVO���T]t�������f|�Z���g!5����ol��~��08	���Mi�����E���jZ7���1NO6���J�hP��b2�
��EM*�������Q+%��q����w$|�O��)��6e	��?$W�i���1�5Xr�=�	�D�l�AnD�~�"��2j&���t
��O�h�3�b��@�����~��Y���f��lM�����Mx��e��AI?���vf�
)�?q��7��r
m��\~I�#��(o��B���sy�����NM'��hQ��#g@��Yq�6��0jO�O�Bo�5�]j*D�]���)��h��=��]�����xJ������8��
#F^�=	�I0TY�.�X]���5��N�"�����C-�w��?��,z�6o�dh�uJ$Y������>��z4a��>��z��;��`�=4��$W����E��B.�)-r�(�|��o�*��f�&9g�����"��#����r��j%�+��&������>@M�I�xW�~���h>�2����"��Q��-h���}��WB35����@�������

r^cq-v�yk���L��9��'N��v��z*��#b����f-��9T=�B3���&f=U�<�A�����<�$?���]{�G���~�������5�Z��k?!���L�u?Q��HbU���%�<��}	��b�A�~���<�B�Fj������L?!��u�l����[>
�OYy���l�*��P����n�H��G��A�����DC�����9?
u5�����;�����[�����3����V=��g��Yo`�d&W�<zT=v]=�']=�u�����v1��C/���������^Z��nN���C�jyf*�.��;Qh��S�#���k���]Y]���g�U�4��F���/�>�@�.�/p�\�~l�q������9�L��.K U���F,��yR0����dCWmh�I��:�!�5_�h�5�������g�b�K���-������I=�=\%�Vu����x��
mr��5����.jS�`��-����8�3�y+-7�P��&���X�}vr�G*D[�e]����{"gv�I��.�,,�4�%���y#�r�t�l��Y�Pd�2�[�0{����c]���q��5Q�f�E��v��(�c6�7������]�9:����3/�u��������8��)�h�c�C�<
u�\�<��r7O���:�K&e]�l��V�yx�G���x���Cz?f/#����E�F{���
�Y\��Czx����!;2-�n9��l��c�>6JM���P��$�)�.f'� -�����cm����@�U��&����f�ay,�5�rVm�h*l2�������36�?$x��,����c���H��#+���
nF�����������3Q���x�f)����M��!
��f.��g��,'C�]��D9R�Rn�3���Cr9����V��O����r�#h%���=�����L����Jm���l$Y:��,$J�Y3��{P�2����O�(���^��s6��\z�l���'6��K�:`x���z���\�s��N�-&��W�+&q�R�l�uD�����o"%����S-b�5�����[H}*Y���������a�9l��s�R��6N���|m�S(r��� �n;J(��x�f>'�����K�d���+��-�N�a��&5���*����V75eC�0,��V_�mj�5O.k�LM.9�����i\����������+��Rs�qZ�9\�<h��k5��-W��X=�\���z��;��"�u��q&��Si����K�9g <�����!U�'q-�4��P�[w*��� m���Q��=�mU{?�k#�l�����-�G_�3H�zm���=U�g��b���8����{��I���"Y.�5y�Q�|N��k�G�{��5~_�����|�Z��{'\�&�3�Z�<���[35��r
��p�kc�5C����6M���u��K+��S��;J���t�#���v��>����I\���r�l)����f3!fu+���Ok��4S�E40��}\^`��[�������wM�}���i�z~o��}���I�w��.�|�P�<'p]{?�kIg4fyLmB,a��?'%�1;M��9`M~��qmUf��Js����D48���hsD����i\��<C,��;�!	��Q��A�:��&���VC���J}m�z������Y;_3�v����%���J��;|�mOq�M�CU�G����b���c(FW���3@���or�����\hkM���K)@M��1����M����u����z08�"K�T���
Dbv��@�b_ C���J�'�K�54H�3-�|�jl���Y��I���vh�3U[�$����8G_�:��x�3MnR����F��bG>�W�Vz��6���S��j�X`g��1�������'�>��
��n�57Q�i���B�v�7��I���{��l���C�W�:�����_n��!���:X���Y|�F!���RA�����#� 1
����=P�P�u����&5��`��H����	����n�nE�AWU|{q���V~!��Or�77���m+���-���!Z�]��.w�U�S����y���i�O�������W���O����]no����������������n����^y�X����J�ah�p
A_�Sc�X�����|�'��������6\�Z��.<�����&���k t����/��{��������������}�q���`�T��oE��|�����������Z�z��������^l�w������?~��/����Do<+\��@��_:�SJ�/B��t�,��~>|�����Fi(^��EB����&�X�8����:�u~q�fxlA�%C0���PBG��G�_��0�N�}�u�_��y�������b���;��?������":�
7o����@��M2��hQpP����\�a�h��X��;��H�J]<�@g�tn��n^�5���d�^��p{�7����{�����<��+��������"&�����c����
��	���H�,�A?���ef������r�u�]���bAf�}���M'?��8��P���
�W���}��#�;J
����cl��y��>t�����&*MD��D���W6���\Q�DTw6�.uO�q~X�3ee[V5|��������UD��**�����D������G�:
F@�ul\�b����3��<�bkA�x�w�}���P}��� |]�:�6�J�N��������
QuA�\��nNR�oZ�W�$�����>��4W�1�������l:�#��������6��]���~%qy*n0	se(��.E!�����#���dsy�]ZD}� ��v\G�8g��2y�A�3$�����-&zdq��M�)���%[F".yW���l���AGh���1e8��$v8������u�bLhG���
r6K�B���Z@�u�0������+��V�#���uEr�VW��dtn�*�N��X��;�P�]y�X�4��3~M���nX��*��]qkv%��S�����Q�QS���V�Z�;��|.��T/(��k)�l��#�;�e�:x��+���(��_�&�~-���F�1�\I���s/�S��A���!@�,����5!����Qq���}N����A����B$�J����2{��[2��t���TcXIfLY��;SV��Q�{Dq������-s9op%�@���4�:����L ���c�a���^<Ep��;c�������%O�9bT5^����-cs�X,y�$[�V����w9��X�;x�kXI{g�OggsX=j�oWA3��`R�zS��*-D'�m�K3,���X�[���u�WR��k��-sT����l�,����,��oK���r�]�_U�~%k6�m����X�@L�����~�� �e�=o��Qw�������J��T�� _c����x�8l���o_���s��:�;^E����f�$����w/oWy���H=f��-��u"��������<����R_+�s��x?�>�<q���J��_'�o����g���9o�Z�w������u^np��@$��n-xg7g(���:����WJ������8cW�J&��!;���g�'�b)�H�C��,��oK������i��yw��b�zVm��{^D{K����J�w&pC�_,���k�����3�
����eI�uW�8h�0~�t��8/Wp�\��j9f-K6������x����<V2dS�&�p�}U��i1H���:~m��q�����2���2|����������J�Q��w�����4sBx��M�����W�
�9�%q5�ebX�L<�6��d���2O������C/\�/����z5y6��c�;��,p\���!��p�;^�'�!��P�]W�dW
����e
�R�B��H,�d�f�����}I�n�����Ws��tEb�0�J���J��2`n�J�x�������-@�N��]B��NL�[G��s�y���Mc)o�n"h��J���!9�����I$�J��x5>�qO+���jI��F��e8����w��H����i���l�N�����R��&�{��{:��^I���n�H_"���]o�R$<��p�%:��I�!�)Y���x�������hF���u���@�fq�)t��:��������{��_�mo<i~�Itv�T=��R�6�~�g�j��L�u��x���@�VKmpQ>��x�����H��n����u�{��J��x�x~�A��]���1���������{��tJ�J����$�tt(��y��	k���3V�����
���k���k�N���k�;���9	m���5������c�����{��R8��O<y	A`�C���{�p��Z�aO��J���4�}���C���1��R�d�>d;;��[]X@^�$�df��4P���\����`V{�f��*p�m��V��L&3:����D{K���
<f��������IgB������W����/������hL����:T��'B�����:e�0[u%=t�\i���0����L�l��w��W>+�u2�?c���D���o��K���J��a���3���2�(�'?`���4����������d�z���E�MB��P���`�����C<�v8�83,���xq�<��w����BEZ~����b��?��>t���[���5�����!����/�PKuz�/d=�PK
�{2K18sept/pg_part_patch/UX���Y���Y�PK�{2K18sept/pg_part_patch/.DS_StoreUX���Y���Y���;�0Dg�K4.)�pn`E�	�W���!��RP%�y��V�iO���_� ��3>����6�!�B�}c�t�vB�2���ts�:vc2]�J7��_�#��L����C�>�+�1�X��W�,��pp���?a5�!~��u���v����K@����nl�+����OPKj�m�PK
�v3K__MACOSX/18sept/pg_part_patch/UX���Y���Y�PK�{2K)__MACOSX/18sept/pg_part_patch/._.DS_StoreUX���Y���Y�c`cg`b`�MLV�V�P��'���@�W!��8��A�0]�PK���4xPK{2K18sept/pg_part_patch/10_1.outUX���Y���Y���k�-7r%�]��|S�C����pODk4/[#�-O8�Iq���f4�K��i~�s%
U�
���o!7��1�����*@.d�|y������������������/�/�����x��o��������������l�.)f��R7��^~������/���?|�u��_~����?����7�}���!�X�v���������~����_���o?����_�|��O�?}��G��?������W�����}���_���w�����|������G����?��������~����?��O/��K��_}��������7/������������kj;�_���>|�W/������������?�����v���?}�~�_}����>~�o������~��-:��T2up�+6����2�Y�b��������o{�k�A�_?��W�_��o��?������O������O?���^����?�G�����#=������`��o����7��������?������?|���E�d,.g�a2IK���\��6dg��>bZ�`rR:/6g����r29�`���'M��x��<��U���}�a��Y��>|���i�����>�h\Q�=<5���
Q��L-��q������	"�Ai��������/�����p����������^������L�����������O�������������<�^����GRa�"�.��2�W>x:���\[MKI�R���m�m)��~k_]�����ox��'�����0|���7����4���m
������������K���u�%e����S4����v}�6)[r���/����&4�=�����m�N����%F���"��4Q������������i�L:7\:�WcXm(!8�JM���;
���c��Yl�k3���S�[6��gmG��t"|��n���O�����wV��G�u�����19��rT��D��2����sd���>��[����-��S����H�,�.!��t��pyc��k��
������^��O�|��������/��t����%�|g:�{��`]��������_j�o��?}O���O?���g�}j/�����{#�R���E�`����wHG3�	���R�L����_h�;��-�;E���H�X������C:dL):��:��������k�U���h^5����L�R1���0�������k����T�8�hc��@�������d"8�\���Vc.M��9�Y�lv��R�*�����i�f�h��,���p�s����)�Cs��R�p:bE������t���W�8zV����������%&E�Y%�/�W����l^�o��z|�k�����{������_5*L<�h��X0��_6
vx�f������<��s���s��<26:t�CS�t��wnZ�M�Y�������*�����}������?|����O�����?}����v5�����A��1N�F���c�X��9��Z��d�?�s���%�����YZ�C��/������~�V����?�������?������������������{���������/��/+�M������q��*g��#��^}oZN�n)'��K�'���Sz��b�n�=��I3���������K��CuGT�Jd]t�5+�zJ:\�w�#�>��t:S��|L:��v�����O�`9j����	3��|3=��������?�D�-�/�����H����������?~���8ws0j�E�H�;��z'�9��\�4��j�����w5���9�:��iW�����
���o>�����������n����4���<bu�/
���J�7��j{������do_���[���w7��6LS�������~��������o^F��?�?��{��+�?����>�������g��������_Y��x�������O�!�#�/Y�����6�C��</nY�Y�{�}$H��V`�vu�>R&�<8:t�{:�fA�#�B �y��H�G��YV�b�y]W���?��2���������;YZ�qi�%������YZj���o��kX&.�]�+����%�'���:H��.=�'8Z)��r �������W���?�����)�sd���p����v������opY%��E�0���N���)[N�8)+�I�8���w��;i���T.�T��W����^A�b��A�f�=�se���g�����s��������/��nd����'e;N����
�1���<��B�/7�8�ULe���7��I�oi�2��}:'e���9��)5>e"�]����JQR@
��h���+g��������@hCK�e]��-�|N�Q��\����d���+�g�:N���(CH��k,���wl�#���k����$�I�S��>��
y>'�����;�����]�'�z�'~C\�������ecC��\�{9�u��<Y��>`u%#J���^���dt�=��jRR�q7��7����A��*dL�}6�����cm����)��Q�(	m��N>�:D��	M(rl�n���5�G�����N,Y5'a����B��I8V����R���z�V^��&i,-�@�����
��ejUd�������.ay�2�dq�4Q���/Z:��G�Q���4�ZX��FT*�9P��w�a2����{�'v2���ON�}"�������R��K�~a����<���I���_C��`Y'`�U�����*�^�v���^8�6�.s)�����&��2m{2�G�
�TW�H���k�SZ��&M�U�����M4iZ�x&�J��.���&�y�@��U�Jt�6��b�`�%y��SL*F~O��[o$\�M����%��J��,�D\�0z��Oy��z;�G>���[��K��	g������K�nKh%gS���J�I:a9��v�����}�\[:��'z������|���J*$u�B�K$�-�������xb��E+�9�s,����YmC�=�]�7�]�pHI[!
	s�/�m1��8uJ���!�aKZ��d2J�3p��@�� 	�c����
��5��G���I��r#�G��8��pb7w$����;���F��4��,c�%f��������'��4|�#M���7�#
�eO�E�eBS�2�(p�V�����
�7n;@ch��G!������F~@�W�[/C�i��H#��"�9��O�u���vd�,d�h5Y�h��.]�h#�2�	�Xs0��v��hj\	9��Ys���R�|G%GO�
�2R��
G���Sk�9j����-��A�9��Q2z~�}��`u�8��8&_��;�.g�r�'O�����>����R��P����M�N��4�1Y���+��DD��I9+��/��2���vl�
bpj����d	���z����QI�9e��t���:�rxK�H@�	��Be�H@v�h�aH�L���=Z�H����0c���V��bw����T;�m���}L�|�S�H_�N�m��U�$�al$S���K��*:��9�.�����X����T8�r�0F��"�G�#<�t.���=�I��������{�N-��Q
�Xp�;�c��#�]`�t��;���Wt�wEK�w��jVuB��V��Q�lg,*�q����Ji��
��&E{�\�(�����d��0`����@��f�.��~v�%(0�p�'2a�rv���9�_��E�F��#�	Da�r�>�{�&�T.9��G�'��L��7uq�gcR%"���$��QG^�]�"��~�������rA�a�������o�
�u������~b���rK���9���6����2s��]�G���)E'��V)F��B���AGI���N���/��#O�j�_��5e���+�Sw6<�6�K�y�AG4��nPK���AG��<�t$ �AGx9�%Y:���#���B�K��#���~��@2����V�S���!mw.�p.O��mDs"��6�J���;�Nh�[<3��.R;�Nw�o�_���5���Y��~�6:��3j�m!�h�h!sK�H�H,$�
h�wj�������8�#0;v��9B���O������wFi�V�����L�,�Qj�2�9m���X��yr��AG� ��ON�Y%�W%�I�Q�F�U�6��Fq@E��#���X��2�_�M�;,����fi��($a�$���v�p�~�$�1�a�����i��B���o�l5r���#�y�@N'<�`�m��9����n�	ot�au?��F���j)��������5E���|_����R�N���#q������fap���x���\�9��H�j,oi���!u	�$K��a3��F��sZ:��mE����?���"�Y�"���N?��y# Y�@Z+��F��X:�G���P;v��3�^�h�]�T�f<aa����XXmO�����Q|���o��<<��[��]�%M�����9!�y�P����(�O�\T���}&��b����������m|�D�Gy��A�!���I���K��GGb!�m��S��m�5Z��\0�;�c���B��W��%�b4��`�i��4[����?7�c��n��j\��c����K���>�W�(���Q�F�U�6��Fi�%�p#2���$��<OD-���F^s�k�����R}�n�7Kz�>���1G�d5���+-�3���e���Q�x��T��A���x#��g���i�,�rt�au?��F��%��u��7��hBUJ���{-rp��/!H�l�f���:� �p��1�\t74������ qx�|TlQ
)�29�gc����5�G�Q��B[:[��m�N���o!K
���KK����J����t����F�����v$ ���\jo9�^�(mb�I���YmU��Q!+�\���vV��6J�;��
8��G����>��$�l9���#�L��q��L.'*�{-|�s��33�QoU�%��;/���2:���Vm��F��q�6u����HY��z�D���BR��6z��]��6*��3,�������.�~Ro�4|g�4����8|g�x����\��!:/r��|-���}_��N9r�9bh��l��<�����m�����6�R���_1�Q���dg�,;Q���C!���\I.Hha�0�0��s9���Q�uoUjC��!�m��p������rL����|~�C4P��N�%
�F�V��m��n��N��c�79_ge��	�� ;ml��2*���J����d'��U�Y'�p����-���{���������5-(J^;�
��!md�^YY�v���H��|��f	���������i���]���1I����l������N?���%�Q�����<K�T��jG�H,	AvKg*�����{�L�hz�ivV�P[�V�q�
��TK�mH,�������s��(��FSA��uV1��	�<6�T�$`���;�.����2�r�]���S>����h����R>�#���~!���p�O�v$����;���F�H��q&wf��h�(W��1KG�$	�j�H�M3 �w	�Z�}.����Q9V�#mU;�y��P�7[�pw������v�35��Ke@/�W��"C/��Td����*Ld5bV��H2~�Z����w-J�&m]���s���,�������V �����}+�+��S����V:�	U����1�"��&�s�l��c
V�&
������k;��������<?L����.1�M�FVJ����U�F�)�.Jv�d�7��\���zt��1��J�K�B�L�Gv}cF��I����J�i�V��'�P}�C*�^;��\���eK3���b���]�������ke�5����SHy�9��-��;���D�0���?b����Tg���������Z��7�^��lqL�2�Y����:\6�H��;�`�����6$����iy��y\�p�S�Pt��E�l���'����j0]8��H�8���N�>��p�y}�&/`;������}X�my�G�rm�Q5w��t�9llY$�6�k��R���z��^�H�;"��3�jOb�����OO���F\{�O�Y)��*Q\{�O��*�z��'�n�W�:A�(�`�$9��0K.�NV�,N�V�"�q*%e�9%E��G��OI�_{BI�C
PR<�%p�/V�V�4yk(��]�����k�5��)��Z�K��Ppm��\�s��]�h�m{��ZL�����GJ
irj��E����l��3SR�B�g-����kX[OI=`i�"���OV��~~�>�E���M������+���0���!�������jE�P�H(�+������)�]3�������3�+
��wM�kS)����Z��{��:z��J# �[o��Qd>%��KO7�gQRQO<���[�d8��S���}����������DP�����\����z'%���tH���CVg�&�:���:�������^m#Jj��M=��7CI%z� DI!&(Yfyq�6g�B��vf:��z������]����x��%�{J
V[����?�AJ�x@����Q�]ZzJ�Ns�������R�9%�^m���e�P�_[Ob�����O�.5��'���B�:��	LY�vO��*g�������0���u�@I9�j�-���QRt�T>�z(��J2������n+7J���(J�()#DIa�!:��:���������,dj�gTh�)�2�;RR��-������._+J>����=��:qhX�1\�}|��2%e�
�U����X�)�hY�9f�G�K��R����k;������J���T�OI�7�]�lujs�x��`fP�Q=1�'omB�V;q�qRk3�0h�i)��$���y����!G�4������{�K;YG["8���6��B�I!?��pF�mOzf���fz�{=��
aR��q9�w2����+����>��0)`y�P��A=�I!k�C�������I��������q%�������c�d�����W���2����
����y�IqR6���";Ul�;�w���K�`U���@���]�7e��������1�l�.�����~��:��=�k.'K���*����,����#�N-��s�,���9KGb���M��qH[:�����qs�t��;5��j��O0R��s����<"���
%�]x,��Aj-�s1v.e\�}�C�2\�pQV���8;�y��aZ�����(��*�o,2^pS��\��Y!<���]p&Z�gH��(O�LS�����h����l�E���w#X��!��(����L\��l.Z^m'\����� �E��sQN�2��J�(�R)�;U�K�:�����<��,Z��v1\K���(FE�JJed�9�<#W��a��d-�����5���������
;���B�Q�����m�3
&�����^O�����
�tD����p���JDy���'�z���eVT#�z�����^.j�B�5Q����!?>2�5�q3��j{���6�����}�y��M2TTI�C�m�����BLT�I7��W�Y����@F�76(�'����k�P�ugv�t���x��U�����Q�KGdQ�m@C�S+�m`��_�Yu�H����0=|�����?Z'`�����U���&:��$T�%�9[tR�*��i�
g�P��%6��-^1{�����
H(��k�dH(7 ���l���"�8a��4	3J�e���V'H������E��l����(k� m�(yr�3�p^7'���?���gD�>G�,i������`����A���Q��U��l��)5�E��� CB���P&���(���/I�!p��qFPL�Q��2������+rG�A��
y�V)3g�{�x���H%O��$�K��r�=�:����j�����a<T0���P�Yz�a���T��:���T���ql�����A�����'�z����e^z"�G
X�Y��7�^�m�PY��n9�z���l�H(I��mHB���S���s�����{bR��}��)�{���C�*z��Ng�����*83vR�&�c>v���2X�Q��6m"��D���N�V����KK�>{��,,�
��wj�����"�}���:�sg�����o���Q��e���x������8������d�t�����cTP`���Z�7!���!��TU[��\���m�~@D�W�c/CD����(�sp5z'�mwL�@J��������+�g_G"
x�*u��Mx�BD�[_�-`C��	�_������<:�5J{��c"��n�b�)�VP�2��T��/���"�??3����D��a��)��������b���t8��t�9^����vDM[��<�B��Z��%�D"F���E@�%K��^0Y�< S������%��C"
��(��Zo�m�~f���^z�y=���Q��J�)"'�N�s>,��A=Q�0U�[O�HD1nAZo.��D����i��mw[�1����lZ@j���	Q��W��������
��ON�3H�c�
 ����c�}$��]8()����>Q(�]c���l�;�#�Ko.UH�V�-�h&��D�?�E!���,-�X��u[��]{"KKm*���zm#�$�VI����%�'���p�-�����Op�RL��:dKO�	:[
L@�'��`�_���U�����5���Fr����o�M�1M��
J*��(J*(� ��i�c�<�WJ��r�@lT@S��klic��k�G�Q�G
S*7��G����	z���_��{V�^8)���U,*�2�Jt\��� �L[�r�Z%��oi�2�Tx~l�	*�()����,������R�s�]���A�l�qoE��u�Z8��HI��V����F���q1�s{�������ec��S�~vM�L'k���^;��h_��Kz��[w� �I�S�8<��
�Q�����#K�d8���gk�$U����bQ����w
2��X�0�����E���
�����������uY,��P�d�����W���
����
����(-�Id��`S���d�N��;����*X���x�ora�cw�.R�q(�>U�$v��feQ8�{�T1^�QN*9):.����0/~����D�����;���Fa�W`���'1KvO���IU������/3��
����M-aN�@K<,c����9'L
Q*�
�GkU�������y�489-|pQ.��dE��5������z�Q���N*�pR<�1Wz����?�b:'�x�,tp�����d����sC���*�]V�����~�/.��Q����P��8���IA~m��Q����h-�l0��n��9���������N*>?L*��aR�S���% �����(�qSQ-���j+�V\L�t�v��hP=�B+b��abtV'X{'���������������	{�k6���2���:q)i����aj5����t�z'U$8)D��Zz'�9���j�f���'�z����9��]�'�z����rCMy2���T��P�L?du�h�c:CZ�d�����W��������
���qR�B)����V9(������N�v����[lT������`�	{��0jW\Q�&Yp>v��G����;y����~������`�&�|��t	{H-����YXj0R���k$�ep
�jN"V��?�t+8�-q�c�����#�"���t���(C?���ur����5nXc���N�B�oj�:&����^����LT����*����h��n��vM\f<\
1�Q�)9%��D�^D���
/����bF� :����Au���f��}��`�R��Z�K��z���Y���%
�l0����#+�D�oi�2LT������(��4��)���Bl��7N��������%Z���-�h9@�e<�.O����3P�
K��S7T}1��D������so�f6aDD9�w=��Zo�CqM�JD�Sg8��Q���sK���;N{r�����H����=�����jO��'�`(��3�^"*m	{�T�����g��7\��%�(I��mHD��������������P���:/�]��N�v|�<y^5;��"Ibw�QHo�<g:��t�q<�� [��,��v~��]��DT:J�;V:YZN��p��98�{"KKm*���zm#��3��k�����'��b�EJ�j=�'��bL(�����t�19����w�	F�Q�*_)���������b��Rg#'>��r;D���KI�%�_u��%��T��0�,�H��J��wN����L��(���R�B))�q@����HqY�))�K���A=��!e�eu�;)��SR�D~��r1�a�^�/r�S�a�
2�J���~��u?�PR���g�;x����)=JO���"w��rb��!�^���*J��X�P�	{����T�_�V��6X'�f^_V3l���-gt�����=AA������;3T3���b���[/��Tc>�LOw�'qR�H��[�?��p�M��f8��-��Y���DP;5szC��h=�N�o�|{�i�)��rR����	{�[��j�c�/�BM�L����j{�
9��������z�Iew��Hm#�WD�oh!�y�!��{5s��r���Q�G��v���I�L6T&/�;�>8JC��F�z�3����18����S�--]pT�K��YXj0R���k���
Ui����!;�cp���KG�+z	�U����b���(�v���tR�a�<�3Q��m�8��aU�%'��v #�U����2`����q�a����*BL���\��)�+� �%x���`L+����:T&�p�@�����(�k����*���E�gZ�^1�r��;���3Q�r�t�5qhyPt�R[MLP�niz:
G�oi�2LTy~������P��i_��<��F0�H��Q`�2a�5��[ ����J�Yj���AZ��"~WWx���h99�'3Q�|�H��6�h�1Q���CN������w����^���B��5j����oO���L��A=�J|�����f����a\�5�E���Z�P��u��s���iq�����>�%i{�
����Z��z�������f�����|�������BDp-/-��1Q��Uq�Y���G\d�Y�Y����e�'to�U�F�fi���\P1��'���6������6L���������%�'������jO�	��i-�/=�'�l��<�c��?�����5����>�pR&��g�_b�Gg@����z������SJ���+�2�:������������I����a����"-����v_�G���)n>!U�R^&QJz{;��(L�"XG�������%��~:����m`���1R}��(�1���

ltT�k���t�V�� BG���\:J}A?u�5<:�`"-�[y����y����(FExLt��T�t�rl�"�����[�Y��b�z��j8VLYo�t9�����b�b�o�-����_l�2��w�z[B~��w���n\Ob�r��F� �*e�z'�9_��b�U��U��wV����=��������f����z'�����������R����bhM����bt�=��j�Q������2*�s��Q�N�,?Z����v��;Wl��EA�>,
��p23(u�l�;������r�v:_)�'���=HF��t��u��taQt�-jGda����z��^� ,����+��#1Cv���(����g�Ua��"�?h��[�� �1,�#)���sPQ/d��2e�VvY\I~�����o`��'�Hwj�AHs���xq�A�W�c#�A�ed8�:��w�qFZy�8�FB���2��1�%�kq���k������/���U����s�b<��2���%�"��e:J���2�`���n�
)*�y�HO�j�k��\��/����<?,��R��2!�$B0acI������|����'N7o���<����l����%��B�o1�����j�]�������<
��,[��[����h���K{qo��i&�|���n^O���DT��R��������U�j=��A=FE+���� ����(������7�^j�A��:X(\8�v��M����j{�
�(��{j��{z��g��(rd��b��m=��{��Q�d�9[�m-���rF����e����(Wj�zo}B�[	�G�����X6�����#���q���YXjQ���kQ4���t$f��HD�����?Q��J-q�#E��q��t�����CQ�Q	R��H{��BA8YC��8����P%\����v�C�W�c+�C�e�b�|����a�������5��A,�����H��>�����v�X(L"���(������B1^��RJ�>�'U����Ai�|���j����J	e@\X��b7����d�~+�~�����u���4��J��@�
���,v�^�?�F,3�"�u�#2�C�/9�k�����G�Vs�)Q����AA�y�WW@�P3������fg���`�[N�9W���n�(�6���yE�:�h���%2���������<�z��Q����{'�)`O� ���'���M��z"��=���]E��\�{9(��B%pY]�i�`�G$U�lu�=��jrP�q�������JR��ut ����En���aw��V�5r�p�6'�t�=4��b�%���-���p�*\E��Q�v�������������H^�=�����Nm���n9}�E5�{�d�G��/�yq�'��n���k[O�	:�8CC�M�8�+���EDY-��v�e��d��12���.�k�t�_u�A6��r:���(;:�
�('��qv���t���e���
4S-��t���mw$9�ku�jQ��$�y��� ,J���x�T �l�����}�;))w����al�e�h�,%DSGD4���������-��A��r�@X���E�2�]&�kE�R�ARRy�3�~sSK��(V�~���d\[l�
�t�P;:#� ��@l�1a��k�[�I9V��kv�k���w���-�R���{�L\�;���/��sR<��[�d8�aA;2*�DP�W9��(f�z"��`EpR�7�^Nj�B���Q�ce��!�����VW�Sm��!'��S��S���T\RNB
i^d��������(����ZX���'�"�g	p5������;N
c��e�����;�����w�,-�jQ����D�����;�������@T��$f��	���i2`y�&�}�"�����&�P�	��X��w�	FjQ���R#��>`Z������Af/�o�#��qRQ_K���r?�������������B�zgH��8��j�����5!�A��B��w-[
��b��������F����1U����H��2����>�����������Iy��!'m�Y���V��I��23_di�2��~���ng�T����6m���Z.�;�#'U��C���)o����
P�����n`���%�V�H�U�1�%fKF���Zz-���\F��&����l�u�/k}����?u���)��R�
9({�Z�d8���+#8��DP��z��u�������:����7�^J�oaRH��Y���%�{D��x������6����n�n�9%��X��A���!�\&/;���E�W�L��Q�^����J�|�1���;77!<L�{'z
�G))��*I������SR����J��'���6������6J����p�w$���,YIdKG�$Y��d�r�%��+�%d��N[n��'�Q���k9=h#�)�!��N&-L�f��B�j���&dTx�E2dT�QA��2���8;��I��]bEGd��]�kl�5�M�b�Ex���g�����):U<�5i��P��?��
}�T�u~#���eH��r(UJ��4�����3�/���2*<��2~�~�
�T�R��v+$M|t���:�(Bu�`�����%Fw-�r���F�4_#����(���NNo�Q�cT�LS�!S1�r��}|N+�+��^���MvHF���������N��o$>J��5<�q���
��������0a,"�[O�#�����zsQ�%��VM�C�!�#��<�����v����T��m�E����������|!w����a���1��S{�9�K��/.lI"�����j������>��A����f�|�b<T8��b/'�J'����Z:"�Jm��Zym��j����!;��a�E�x����3,EJ��8��DA#rr�\�qPd9�V��m7J�W�p���&9Z����W���ITpP�U�8�pPq�AE)���$S�Y��6Y�d�@��%g+�Wi�+9��x��g���	��8(���T�,���LU���?�A���B���i�;�����E�|D|��o�9�t��j;������*>?I/X��(�����l7�9�YX���t�E�h)9UZ:Y���C��������Q���O��v9��-�F�n�����_.�y���r�z�noL.�q@T��(������o��*������n^O���@@
".�+/�������Zf��A�9(W�)[O���4�e��7�^*nT@!�����IF����2�2�MJ��j�P�q�4�����JZ��(:gC��5��7rr���a��4^y�MG9��'/";�. *;:|0)��l����@������Q�Q"*�(Z�� ����$G�55��{"KKm*���zm�PE�U���$f��	:���B��������R����X��	�=����+P�����f��9z1�����~���9��Y\3�p��CIs)�4�����r������JB�DE��u��&��.9z�!u^W<����{y-�`@I�V�(����uzv����q�'<h������^��NJ*�aQ��5���NeLIE:6$z�#o���F*G�~K�~�����))�|J�>�F.z�}lh�B%��)��q��L�{�����():i/�E�%e��ti!��v�FaQd,g�FI%{�r�>G
U�'m���iDI�`k�P���N.������t�z%��%e�C�2����w�9��V��n=�#%e
:F������B�Y��\�{)�u����!�3`�c�P����������^mCJ*=���7�����y1J���F���&3y��aw./����]�fV3�wv3��8�6RO��;77 "�i@\����gm�RR�������di9�����,U{"KKmJ���zm#���t^r��>VaJ
�E�,O���JuP��hS��?A��U�e��%OX'QR��$^5[g��|\o�X6*'� ������r1�8Y6*(�����e(�<�����9m�*���ac�*�d��
����C�x�W������?�2����#�\n��!���G)s�!9~y���1��L��GI��eM�gG���F�	�1�m�V��L���_di�2�T~>%��t)s�
U(�H��e����5���9A5�n�(��^�"���QR.Br�)����Yu]���;{���[��XB����T�KVaCl��
��q�
�%:����\$(�|��_ J�HDIA�>X���w2�QR�����\_�D����{�0c�����Q���yyW�E%�A����[��CV�0d�QA��-93��j{���6����nj~n�9%u�n����9��
�����%����;���d*J��q�9����v5:L�.�c�J����c���~��Y���T>V�CQ�p����TJ��0wR{"KKmJ���zm#J*����'1KvO���!�!�'����<Ag������'�%�5��2�Lx�Q���nGI��R���"/lHI�f4�t.eeZ�e,z�0&QRe@I�W�"CI�%U�()���\]�!��%���$�))��B^�RD<o����fRRx?���AF�EI9�I��������U+�v�(�(��QR�������a��:(�!��Bc���N������� CI���G�*<����T�E$��-^������T���	���I#���b���]C��"��6�����.-�>����(�`8W�S���
N�f��s�#";8^c����/"����[\�OI�$�d�{����z'�9_�^lf�j��������U�'���P��zsQ����FIEH)>du�����<V���IFW�SM��!#U�R��R����,v��** ���U��
j}���7�S	I
�l��r������]/}T�!������c�uI�Q6�t9{�$��U����N�t�Y:"�Jm.��Zym#��=�*^d�����GiC�Y���?Z'����%��?�(t�q��t���������z�������aXT��Y��"�c���:Ps((��)(���8���|
���
�R�����{sf�B�9����snrs�d�������*{
��,�Z�y����
W�v��P���IIz���$=�6���X������I��G��J?Ys-=���~z���D�'���L?�2]7��H�R��t���J��	�cT������V����w�G�	�,�S�t�^V�l�.b��!��Wr�hw��	����O�[e&�9��u������=	���>z�q=�~*f>�d#Fg���@�+aYT�2J�vI��B4e���vI�d�_��E��~�vP��*m�:�t��<b`f[]mO���F��r�gz�#o�2V*"J�vbCf^����2�w�]��
cC��L6�v��l��"{@+����,-�(ibQ�6���gmrP��7t�H�di�8�qJ��#���v�A�WK�m�A!�������9(���.��8���,���q�#E�H�TK�:���2��C����hjQ��!����0�Ph~�5���T�H(��{ldH(3 ��	�qN&�q.�q6U�x)�e�B��T�EUZd��������yd��K�<��y��L�����B^��P���|��2=��7o+�.W�#"J��	�Ob�H������!�����"�<���.N'�������I�\���4OA�HD1*�Yb�%_s��u����,�
"*�|��))'�nSQq�q��[�`�)��E�<*!q���k���!\b���wZo��G1h"��:���DT��(D>2���p��'�qPl=�.��/DT����)����������Q��Mt*�������:���3b�&[]mO����D�y�=5o�==O�+r�kQP���f�
����;����@F���K�lN�Qxo\t�`=���.$FD�{����*m��~��2G"�j�ih�,-Ek��tD��D�;���F��+P�A��(�8Cv�'%.�������\"��������!�t�:����Pp�F-�M���<
e���?Q�������������m�v@D�W�c+CD�e��0�I/���F�t-|a�����gK�C�l��1��<~?�^��_]c$����/*�*�F��FD������Z+�9�����CuO�7��_�[&*����2_di�2D��%d�/T�%�eK^���Q����=�����u��I4���Cq:X�����l�����E	�u�9�<�8DdPR�l6�V8Y#�����po�@5A���E�sS%�M1Z������}>�"�|�e1����;��	y�����X��DP�	y��%.��&Y`&�1!�uXF8�0�^"��DD�Z,�~���I�THkB����ct�=��j�P�q����t u�B��a�.�*UH�MH?Is�������h8#/Ksy�Y����������BG��{�F��S��;����,�������B��+v��vD���,�;���F,T�
$�jY�jH�"�Ro���G�� �M�����(�B�������&X(���9�ZL�Jk��>*8������1�a:1��,T�P�V�wQ���*
�C�8���q�Y�������P�g��]W�+OQF?�C/'��/�;�.:��p���B=L�#�n�Ot�Zq�X���P(��V8c�}�B!Y�
M�����!�\��l��NX��-��A�����
YOg�R�
z��)=}��)��Ur��Y�L���i3�%:� ,��SV���s�.��7����K)��]�U�q_���:wE+���[�9�lG�uP6O���wog�6�������X����k�g�����<��Pnl=�c^� &�ZO���,�����z/�p(k"X�:c4�x	�2w6t4����6������B��'�C���*�[pA��b�2�w�]8TY�)���d���4��9�cb�J�^%��[x�&/o"��D���E�Mq�vi9Q*_x��YZjPQ���k�%�������%�'���QF�?l��?A�H���'�^�	z�r�t���2�V�B#��y::��'��$��1N�&����#-65R�<�p-<������z���K2�TPRI���q����8���Vr!�P�6'x��F�w�+EDK���9��r�J{��fz�0�rs'x��Q�����h�GI��r����Iq��yR���^�-Con
�/���J*=��r�O��B���&�h�F�o����c���4�-��rz���*���[@b���������)@�E�>S��ZdZ��hM���:yF�Q..�d�m���v�S��{����,J�	F���a�z'�9?0
rd��T����\�*��S{"���(`e����\�{)�u��G0�VG�U������KPR��W���z���{5�O)���2�2
��>��{a��{�����4cN�Q[[�;g����]������\�m":���S[|:%��r�e�O�����A���5�'���6������6��`���hKOb��������[���tVJ��������t�J����	���F���h5��x�	��%���(W�\1l���s�J��<���TPR�V�wY���J*��a�Y���on�T�"�t���;�;��0Q"J��!���_j����)"U�\=O���E��F�k��PR��J)hQRt 
yHI��2PH��xUZ�KPR^�x���� CI�_ W/���,,c���;Z�]�_�Q��W�����.d��}YLn��~�@f0b�Y�X�����j?rR<�,�F[���>a���K�Y'aR�)D5��z�j�qR(J�K�.�������|����^����'�i�QkX���p���P����,=�#'U@�����DP���k�m����z/'��0)��1Yy(�YY�<Oa����T��m�I=^����"������|���!g�F�i�9[����RQ�q������0��vS��f{�������v�����e�����rR�&z����<�����'���6������60��fq�H��~g�>�p=�1q+a/�=#��u<�jO�	:s
]��������B�\Q�jG���]�{�T��C)�� ��':��jG�x-P������z���+2|T�QE*D��9�qvi-����C]��AR��@��������B�����������jF!R������_&\�����*}����~��(5�<�#�j�G���n�>9'��D�����>�<��r�J�� �(G�B��������R���YF�J
5�������d���c�QF�i�<o�����!!R���!�\�1n�MG���Q��Zd��
;�1����������K�Q�%�������GI�Q��*�k��;��|T������A=�Q	�S��rO��Ga\G�-�����Qe���������@��f�QF�8��j{���6������B����={!}����k��6����S{}T����*)�� 	}R�+DVR��\
+}�p�#h8�%�M&�D�G��r��a��nW�N:��Zi��,*�
��wj��
l���S������
S���jG�h��Y<��#�4Q��5��7��z
	�������t���@.���N(��*�D�Z��W7o��I(�j�y�EH(��I(��H(�3W��8��	k�\��D
H(��K���v��hg��W���	/�N�H���)�H(�Y��'����Rs����$��	��Wkcsr�����z`������zf�/������'E�+��$Jl�1��myzV� �33j
��������K��iL����1W�$Z�*�-�RG8��fP0������E�4��kW�.�a�^�P����[7y�%8(ZV��)+�% ��X8�e�����9c�5��v���m�PO�����1�jz.�����z��r�!�+u?�l�)������^m#�?^����:����B��2��&*}#O3ej��;�d(�����3w�lU;�N �T��C�0i����X��x����������gsWb��nW�#�b��KGdQ����z�V^� '���C�A
Id����9B���<�1g��/�q�y���CD����oo4���"�g�jk.�B������*6#��<����\�5�n�6�����pPf�A!
����@��iY�	��������U�-A�H���y�}����-���C<����X���R���L����P,��cDd����<f�9��P���#�k�/�E&1�K�~�������l�._nq	�B2�)"Z���9��-���M ��U��9
�\
%���gnT����U���G[[i#l��8(����yq���&h;��3��������5$m*uZQ���(�X^��(�g����,�����3K�i���vyy�0���s#�2�+��e���u�l�>��2[^}n�3=`uF�RP���rW)O����j{�
I��k���P�}@B�U�3F��|L,t
s'����1�0����Z�|�.������\�"�����g"*�U7y���Q"�����|8YZ�`(l�>��YXjQ���k(fw��z[�q���b|K��;/���Rj�*��wy�2�?�	�#"�!��c4��Y��,�`(Z<L%H"���FD:��%����z����2D�QV*������8���i�F�Z��(
x!T<r�[�L
��Z�(
���6�V58�O%�*�_�%x�
������D��^l���1-1���+�P�Vc�*���LE�I�{���dx(�|*�����)q�K����!^$V|�P���y���|��\��s�C�b���5�0ie�a� �T0�����G�Qd��������C�V�R
��v�Qbv�������=>���������I<T��P4���e8��'�)����+��A�b�h>s���A�b��
5�/�����Pv��������[��t����>�b��g3��j{���6������B��s��rA����
9�n�Mn�������t����
k���P�k��x�v�����x�PT��#a�M�e.��<�=�P&�M��fi9����p���YZj0Q���k2��������4�8KvO���4;}�����-F8�^���>��<A�'Y�z��`�He�B���u`��<��9�%�7�
V�<��Q�r�f�wF���(;F�
)'�H�8G��qFa�����2sGx��7:8�a��Je��F�C�PI���!�3R��X�����u<�Vz�������C�/C�s�N�5��.�rN�+������J�=?=���e��)-�q������ ��f�GiZ\5���h�_5��5VN��_$�3C�6ou��P�����������Z��Il1n�Zo=�Z��q4����.�u�Q$?�������*�9)D��2��;���Gqm��A�����T������z/'�n8)$�?du�`}�Tun��9VW�Sm��!'��S��S�5��P����?�����t^����Z��*�$�Ko����Go"���vQ�&�b>�������\����G9)w��,.�O����2:!�7��'���6������6�L��Cn=�Y�{���eT����=�'��o^�zjO�	:[��N$�����O0��'�,�QvM��(J*%�����*lQRz�n9��9)����e8)?���P��+<�N�Q�q6�Z��A�������d��/ZB�x!��g�n2J����:FI��
��������Ek<�I�#'�JQ�t=c�^��0)h9����yO���������N�?�����T� L*�g�}�>eQ��5��E	v��0)L��Y������s(�.&��
*�P�9������1N�_6rt�	������.�zN����p�zm���"6�=�������%8)���sRY���T@�����;����N#���DP���"���DP����Z�7�^Nj�B�M�V������C��WA{��8)I��m�I���T���sN�&)N��^~�g��&�?gr��;�������ds�aw��A���Ts��bw~.�9_��O7y���QN����+O��>N*�8������R���z��^�(N
;��$f��	�8��t%�|X{# �]��b�e�z�O��IA�����������
Xl�����,���a����U���c��&���qRa�I�W=� �I�'�j��87I'mD�%IHH�M��-n���qY]-=�C�������h�5u�Q-=��^��V�+C>��
}-=}�������IYO��y�MS,���5���N8��-��A��
�O���Ix'e�0�R��)QV��s���r�xE�����l����x��1N
��
+�iZ�n�:����]}��B#�����z�sRe� 9�����������y�p�]z[~~�i��������8)+���-�b��[z'�9?w��"��DP��{�����A=�������d�{9�����wy��������P���mu�=��jrR�q?5�?�<w/]�X�m��z����ai��7\�����>����P�`����f����.�/4��v��K��}�s��L���
?T����'t* ������|�%�b��,-�
8�wj���,3�1�l���,�=Ag�a���{�O�Y)p]iO�&nN��������<���O0�����[L%��WU�f%��d_����c����&�I�'_���'�T���8�:�N����%5�'�xv�s��iJ�I1^0�l����k=N����~4���m,�R����T��;��$��C5����@Y��
T�B�{�Z��%������-��7�'a��vb��NM
��/n#�BK�%�H�,`���Im� _Z,I��9UsFeUs�t�&��;5����vi��ko���MC]�`��Rz�����H0R��+��#%�j��hq��z'�9?J�jS�b���1J�'L�m��c���2
�*�0�^Fj�@����cVg &��$}��`�$m��!#�R��RQR���6��rX�����������hHJ��bB7A����()r�l���a������EI��� �;��	O�~��:���7���ni���������R���z��^�(J
���KOb������"\�YkKO�	�()�9��'�]�p�mO`&������r��28S��&����8@�+�lR@[����K�Ri@H�W��$CH�!�d)�AQ���}��S����k����)��M3<Z��%M������*	�D�bc��������v�
�o[��S���)�J�J�!
l�����&����WA������"+�~�a��/�F��l����M���04�>�<��c��
m�A�5D�Dm�X<G��`���2n-��_d�w�T��],f1de\6�c�^��������/��!��&@��?d��6O�qz>��:�����;����X97��EP��,�fU�'�z�'`�u������J[�r8�CV�J �E�g�d�����W���J�{��
x���QYJ���`G���[x������uj��sF���g;���'�,����{�i�X�V[Tr[��L�G��t����"���lTL�[��,-�
��wj��
��`�uX��-��	�����'���tl�Z�IZz�O��Qx������GlT�+�-g�7�����
���BJ+g�H��_u���tT����8�Y*�������-y�������w���$Yg��S<��\��3�j��U������l��]������}��A����U4*��()��{��q������Q�b���N(��-��A������L�p������$b�_�:D�2�u�>�������r��\h~�Z%2A������\��]h~W��U�����l��j8��JZB5jo;�?
��I�w�Zo���%�O���PR){<�"��w2������d���or����'�z���XfE=��O�������!�+�Bo��&��d=��j{���6d���^j~^� >*�0Rtde��Q{�L6�����x5����
B�!#=q2.��UJ�.$��A�Q6*��Q�����rd�J`I�����R���z�V^���W�2�KGb����R_I����(�����5	�YKG�+�������?��
A�����.vS��^�2��- Q�"���5w�v��TpP�U���pPe�A!
�����q6P\�9�
�|��������Y�]����;����1��*T�h�c~F���b<b������A��Y;wrP���8
w
�"�x�A9kC���3t���A���pP�[Z��U~�\��:�(�i�t,���`cH�OF,���F���������%&w��l��.�g'��w��������"k�o��-�''�9��*`���2�w�Q%�z��z�&�D����#\~��F����K�d8��F1���DP��Q�U��kO��2'O�T)����z/U6��H����� &����N���VW�Sm��!	UwO�pOO���"��1^��QW�&����;�.,��J��l������K���k�v{�L�C�<H\y	�G��r�������r"eYZ��M���Q��z��^�(,*����KOb�����n
	�=�'���h��<AJE�	:[�X�8>���?�9%K�*�Z^��2��@�m��Q�Q�%Dm�J����H��p�H���O�_�3R���0R��Izf��D�&��t�V�}p�8����������1Qu����%��_��;8O3��p����������F��Q8l��4�r���6��I���_k���e$����6���g�Q�N����D�g��>m�_:o+�TT
=F�@�����cp+N_Q�5�V��X�'r`��(�����G�&�C�F�wl�����i]z�5�s��N��y�G����{?��������Il���(D�JZ�d8�������A=����H�o=�#��i�,��l���w�Q��u1�>du�A��b�xG���lu�=��j�Q������!6*_�n�m��:�K('�yaU�g���p��"1 I����
j���.J�����>z�s_�{�}3�A6�x@GX�Z��n����r�����j=����s6���zm#��(��ROb�������D�O�=�'��4@��-cP�����t�����78��	l�+A�+��kQ���Q��!�b�
�@CHr-�����L�����2�:�F��2:��Q���P�y�.��`l�|�|,O+���D�u�����b<�+�U�-����KY4F
xqOr s�|��+�<�()sRX/+�N��C�(���u�\e��R�2IzXZ��%e�OI3]��A.�+'�'��0Z��H��P�.4�}��)����-������
���Tia$8�i9�Q���p�]Pt��r�%��2��c��l��6��FE�i��s���+�2��������YRN@7
��,�R{'�)�����Q�'�����b{�=�^�-��yW�'��KI���"�k�VW����f��:?��j{���6d���^�y^�y|���*4l���I#(f���As�pFyU��]��t�c6c�?[����3@3���_ �LoU(Z�Q6�p6wP�_���U�K�#��Cr��,*�
��wj��
l��"��A�����
�����kG�h���H-q�������s�:� I���FDEcTY9�d�Q)�@Gt�q�I����3W�<�e_���e�*��|�Um���R���~����!���bQ97�ag����Bz��
����%�H!=������"m����e��.�E�b������^!��Z�kXT����E��� �A��sP��4?�#�DL�)3}��2$�B�^,���_I4gT\���Nr�����z��l>sFp��PTE��!"lAuz�.���*�[��k��[a��$=Z���T��M���|��^z�y=���^ I�hp)�z'�9?IX��`-=�c��!�ZO�������\�{9�u��C��:�u�Jd���7]�IVW�Sm��!	ewO�pOIz�8�N)*��g0-��`&���S��*�����}@*(
����H�ir��2���u�8a	�Gi({�JI���na�h(TCD���YVj�P���kX%�e��j�������o���#���yR���E��O�b�%^:��44�p�����nV�rg�P+���eM!��[��b��!v���P�U����Pn@C9!�(KN���	����Do��P��V�5N��|WDa�P!�����ZQ�Ks��h������zx����kxQ��oZQ!=��r}	=��z��KB)��VT+
���F6j�Q?������dh(�t����K����U��r�%�y�!Z�l"F�����V/L��R�����g:���HA��L'bT�"�o�D\�y�`9<�����k6�,#�d_,��@�����[7S��|3=����C�(!��k;�������",[%�jO��2�xn=�#,g������}�6�rB��:����(P�9�s�����^mC�=���7�����%��<�l��#wk4/���t2
'`�Q����K��c��c�jt��v�R����ri��Y
^�Q&�pB��Nq���bQ�0�JR{"KKm�w_�$��3�#�_���(����tT����o�75}�`(-�&\����f���zr���z����}2n�e{�hU�����z(�`V��?�k[��{���(�����W"�p������� ����C-��SR�
<��qBI���CI�	%���x��l�4#��3D�T�(�K��A��79�p�XN���E��:;�Q?�M&V:�%�z�#
��tT��.�n������ty+s:
��{����R���l�A���������b�,*�_e��2%���TO%����P!]�b�����s/���j�2��������(���:DE	*�S���Y{q%>�;o��*�I|�����~��9f��Q���o���S�lT����� %��5�{7�������-[��TP��<�r�m�����u����}�w�lT<��,t\>�:�3&�����o����U�kmJF�����{2�<�Nnm~Q2d��,��]Vo'�k�EA�x�����"����&��V6��XM��|�5�#�:ro�vY�)�!Q�H��]e,��� ������6������6��X
���:+��5L�����u������H7[G(�g�X��y�?���U*��6�)�0��s�����8�7���������;M����g�t��4���Rf�����9B���|�hd���F�R���4���s�K�kf������|2eV��_3�d|�c�=�RRB��w8�4f�%2�G0g�����%y���3^�;U��:���6���A��g����P� �F����n����
������jG����ZCI��N�W�Y���,[�W��<�1�e�	���X�(\x�Ik9��i$6Cz�����=���<)������wi�-�������"���"�I\��w3���)A��:��r���:DDa������:DDa^��{kQ����E��GV�,���i�F��KR��k�������J������7$T�6Y����p��`������}���!"��������O�C���E�#
�9�qQ�m[vq�"���Q�zD��!�����QqK��=����	�Gm��iD��^��#�W������CD;�D�z�O0Xi�T{�����������[��L()a�FI�Pw���
M+y���!5��������	#E?����H���"��(�f���4�=g.�������G��U[�,�)���(�����q=��2�0�GDE)��Q(=���W�%6���(���U��Z��J��m%KD�����5f������6���F����Q���!���y�wV��5�(@5��*��d��*>R���+9���JG!��H����)��^���g:�������g����$$*%�s��t��T�J�����}:��BH��
o���T�G	6T�Au�G	�h���
�P��fU��Zo-��t!Q��?�:jpp}@��Wq�%F��WM��)E�{��<�{�(z���$�"#)D���a������.B�B:��)@_�.�Y�ob����B��5U��F�'
�O�(����lu��]e�
�U]�:*�Jk��Zyk��������X!'��a2l
�u���� l>w��o��E�t�<���OeS�@�<v�{H�����L|�'82�Q C�k.r�S�����?��Y���*+�Da�%��R`�Rx��2	�<����~2dhE�5$J�'���]Q���4��e�B[x��jK�����7b�i/T�����e<V���~
J��!Q.=+U��
	���
?��P�?P�R�����+l1���HM%�Q�������"r�Y����g��JHT���~�U|��+���m��l��m���-��@${	�j���N�f!Ql�&e�i����>+����g����%*y��(���{7��^$� ��{O�%#�����-��!Q|�3���[��.�_8�l������iZ��=)��F��WM��)�?�N�/�No9��|jm��������/����G���J�w�oEE�!-�m�P,�����	zH����^B1�z��X	�)��?Z�7���AEjU�����������V��,-Q�h�r��I��)�����^�rZ;�m�YH�����u�����-��T������p
s
�����[��TY-U&T��3.:T�pPE��A(�$����<�G����b���{�&u1��mT��)�uD��(��b|���n�����?���U�@(��i(�5��,5�#��G����������|?�Q��e+j�U&��{�w
�F$
�$�����Z�_��>pP�J�%��#��P��"O�e��Uf8��K
��T��}�7,�=f�,-/���(k��~�������|��^����FZJ�Z�o���T*'�HFM���iy!�zOuH�����������rP���j55|du|�g���a��8o[]k_����$T��=-��=���JJ��Z�+<����M��}M�6�)/9*b_~|E"`�r=$E�k�O`���Py&�����Q�JD%�J7[���~�g),�z*[Kk*���zk3����$���*9=�`��F(�����V��E���\m���S@���{�3�rT�vM)�P:m��SR1��U�y����+E�.�W'�T��Q�:�T�PRU�x�����9�{��,�`R<xS��	������:)�'x�%x��p������x��>�,x����\��}}@I��x���JO�/C?���!E����{�����tc����t(�����X�j�_�m��D���'�LEO]���+
�uJ��\�(�`=�,U��}����GQ�3\���{�<��Z��w%�����E�g��)Dv�q���q�uU����nq�R��zJ*as������t�W�,)���zMilO1��i=�kf��0�>�X���KI����� -��s�����T���[&
�����6����nj�n��R��Z��O-&���=�
��=���7Z(�Bm�
��}uv]%��`�+��Z�z����q'��	w��U�?���U��oLG!���e���b[K��l-�M(�?j��M,���,�3���JNO0(�1.����S��J�q6
��S������>~��2�l�����T[W��0�*��7�[�����}��������[J���e�u=%%��
%%�%�C���c�({��%x�>�|�E��
l���:J���L�{���h=%�x������ �Tc��x9�!S�R6{�2���<S�*���(xx�.������'K~P���_�*%	��<JJ^%�����}���L=A�x%����_��=�r�� c-����T�����P��%���l����F�
%���O�{�
����x�
�@�z�!�T(��Yz{x}����z2�D}:_J29=JJ���M����QR|p���bUP�()\G��M�n��SR�	������9[��S6��|;�������6���������J�/�II.�����F)�o'�A����$R�"YU{�E�qg��kb�1*b��5#F|w�*�RR�zt�9y������|36�����vOI�U[omf�2�n�V��JNO0�g��������V�#G��{�O0�*��U���?���^1�EI�J&�QR�����#}�P��fg�t������#7a���~��a����rJ������4�C�('�e��N�e�s��g���~�8�GCZ��5�3�3!��'��P�\�@_���>���(7�QEu�Al+A�~ �P�lq����j]�a�����6�}�����B��o��������w��������JP_�ZH�HE��!�@J��"C5@��e�k2����������Rz�
C����Q�L��v9��[��~���QJ�|��\_b�
)h����(4����tj���u�Jq%������:��
��x�r�XQ�r)��l�;J��l
}du����Y�}w� ������^kS6�}���_����Q���ck��](���D1..6����ORJ��nMjIGI��;��X�
���FT����(-��[���r�Jz�&����e���w6d�l�}��	�G
���v�)@�u4V�	(,����}��y&�87�&����n<l�0m���3��P
��#Q�eq���x��Q+[�,:,e��JN�b3���O8(��w�u8(?���%�l�k��M�dh�I��j�U��r�cDP<
��7�6������-9v��P�G-���N��L��C�1Q7 ��P!�x(����0��ycz�/+�n���
?��P���z�*$��v��W��������B:jD��
��
n��o)���E�`3N*�{�}a3���O�6��K)�xC[���l�-^dl����[��%��ps��=��
D��u�����%���{7�����7#�B���z�'��
��TP��,S��[��.�"������9$s�mB���u_fu�}��Z�Q�s�������
T[�Q��X>Z���	{ � ���$|�QZ�=\$C���T�qL*A#|M�*��������l����l-cX_���G�z*[Kk.���zk3����l�tV��	�t��Z��=�'��G�(�{�O0�*����V\�f�z9�b@^����l��4��IP��@9��#���w���'y�PR�GG9�PRaBI��(���}�kw������g(�T�2����JN��8�F���#�-����(���������%�,	�:*�tn �&�����t�s��uN{p)�k�Q�[����OG%Z.e.�2�
q��Xm}�xB�z�@�(�N�z����u!�y��*��H����b���g�a��z���4H��3���rz���
{���~�_���~qy�����F9��9z{p}+,�*�EQ�|<�*����N����nC�?�-F��+�kq���I���G>O��E}��
/���A��V��m6�Z)c�Luu����U�km�F��=��<�[�(|jm�N��0pA�B���]�'�A��9�7�|IV�i�A��q
Q��h;���3�E�k!��lTj�����Zn������rIOekim�F�Q[om��gM@����X%�'��,[F����������m��{O�	�$=^���s0a��9��������������i[)#@5�c�����%�Gy��Q�GG9��QqBGE%)s���`��m��x��9>J�b�
�����WkW�8]��e|�)�CQq��)kH�/�m>#�|��-��x#e�X�]7*����;�z�/gG1'PRq���?Y���%�NI���rJ
�R�*2����V����O�WJJ
Zf����}~4tB�RR2V+���3�tA����QRm�N�%�w����8S/�h���{�5�]7�A�����*����_{sR����nq�>%�4��e�&o��L�z)s�=����z����3�����z��##l�����]Jj?Ac�KC������MD�W���WG������^kSJ*~���_���H��� �`�$Rl���	k�	{��{0����U�"�����Rm������)o������Jk�RR�:�*������2RR����ko=����	%�Gm��i-&���Z5V��	�LZ�E�d��?�`��[�.��S��V��V�{��`BIE�sa�6�~o���;
�*lw%K�o��kH��Z[\���{F*��''F*M�� �i���i��'%�K����	�B�NW���0,pr�Fl�EI�����]�q�L��>
8��p��M$G�r�>`����bGZ��f��>��}M��-������u����E%�vCF�od�:dT�>e���I<^�o%�5���z6��(Z�;����F��e��LK��z
��uE�����V�&=�(A�Rkl-|Vu~�X���R>yS$T���QM�Lu����{�����n=����i�FU�(q2����N{����K�Ga����=1V�G��SA�	���5�\��.�6
��#�C�{�����������^kS6*}���_����Q�j��aZ|����vq��ok�Y�t��]�=V����E��� ��PXs��`����Z�O��t��#�H�}�Z6
�EA���S�ZZ��Q��[�Y&�������`�*9=�`���jzO�	+n��	�:jO0�*�^*�m��O0��fZ�������r��Q��?/yd�P���lT^E6�~t�I���	EJl���>����<����	��o�1>�m�����Q��6��s>su���l�\�����=�EClTH��c�Jp�j��B$9)lo�8{lT�Jt��V6��CG�e�z��_`L^^%�E=�����E)
�3JJA��L�W�K1>�<��3�*�sHP���J�-D�g��A����Z�-�Y2��xT1���Io�e.s��I�	�Jo?�iiE��s����Vl��x�G�p���[�f:���bL*�TP��,��X}�������}����(:�����GVWsj ���b�k���������T���v�U"� a�H��'c�b�:A_��S���u>�
�z�H&�QC���L)����#K�@�84�?%�������
��6�������6������6�I�-H����X!'��a2�T��:��W����nou���b��n����)�h���V$#n�Spi�	���x5���q��������;O����g�u��<�����9��@��)|S�h�K�
�����{
{�e�I��<r��9�]�,�qE>�U�\�S"=E���#��f�T8(y�;*��b��Q|�$:y�[�^I�@G��z���t8��u�=���5?�Rj�d���������y`{s$�x��]K�B�L-AO�����4{���}��{B�&�	��{��t���3���
���t�{�������`��������{T�[JB�[O8���
!Qm�����t*�cYD6��
�`O2B��������}�u�X�%�����#���e��n&�/����j{�MY�����z�e��J��dd������QK�	�.�B�7�R_s������97��5���->�����Jk�2Q����#��ekk�E��uT6��&L����&L�%
-����BN�W&�a��b��:����Z���S����l+)�����<��D���;Mg�l����cH?��ndw:�����*"���"�L���CDm��i��}#��RK�U��Q�L��%d��Bs�r@�({48W:��IA�\���f�������H�r>���9�=�Y>4��X@e �YDn}�e��\U���7���|_�<���ev����J�J]��`)���K5�h,�9�95�r��C�EK|��y�����N�����7�#/�WB�m~#����/T@i	q������2���Rs
	��w|f�+#���S�[QPI!
*���#m���\��r�����
�5

X-p���^��0�����Z�w	�rPg�GV'�4��S�+u�������^kS�|���_��NJ��)D��X�B
�-6���BTEmg�(�d��cTF���Ud�*b4��s��e�k��:�O	�r%���r���l-7
Q�8���S�ZZ�PP��[�Y���K�����*9=�H3nS�o=�'i��2�{O�	[��[���]�GM�xrpPR��q�
c�'�Yz{��Z4�r
|�w���l�RQuBE����CE�	Uu�(�f���
u[$o���Ln���68wT�,O�g&��W8��|f.E! J���&���>z��w��z>�B
�m�sp�������Y���PyS�;�R���Ml�A����'�rYND�{D�o(��
�r�u�����|C9�O�\|Y���L�T��-*����jRs���������Y���&T6��O����u^4����m�#��������y{T}����?�$��G��L�z�	XR���TP���,v��|����y�\����Wa����O���x��gV�,t�AZ�@�	6/����j{�M����OZ�Oze�3�6�r�.D��'����5�I�����F�*b�-��+���-6����2�G��6R���S4��K����e����
E\��S�ZZ��O��[�Y&O��y�V_U,���}�u"�����`�R�����'�'ls�����nq/����EJ&$�/�5��a��=���:K��B�s������������X~]�?�?{�?�C��O�fv�4gc�*�������%���?�&�+V���|`�Z	�g�f�i��kd8�����Cm�;�S��A�����9�<5�-�$g����������O&6���?���eA���r��X(��\q�vb&&��*�a� �>�{k/�U}zXy�y��-�,�e��{�d/��y��
��	u�EZ\ �OJ�+�#��6�%b�)�����H�j�:=+������I�-�IC�������n�S�~�W����UP�s����t���Q�����v-����qdF�O ���s���7)4C<�������j{�����m���'��~�{YZE�\�\��s�.B0������}uo�
UR`��
��W��M��C)���A�~�|�ms^�q�7��>���7��hw����Sb��Urk=����{����zk3��'�!���4V��	�	���k=�'�'YPi_Z*���	[���"����	&�Old�J~F�,^�}�SIr�`��4b���G�g����@�������7v:����ON�~�4���iF)��<F���X7$���y&�
���c\m5d`t�n
�.t��x*+�']h��	���+��d~@?��~b������(����G�-���G��j���
?��O���O�/�Vl���������3�>A^����.�T@�J�d�USN&�����=X������x�����g��G	�<Y�� g�s{o?���j�#���$n���Zo�������=��E?U��;��R�6����N��;`	��TP�sa��rOu<.-n�~�X��.����AP<|du�}��������e�@?i�^kS��}���_��N���*�a]�Da]$�.���}�(�����[�	{��C6*	�31zE�����!���#�d%�����
�����ZF��� Z�N��l-�M��?j��M���)!n��=�Urz��>��k�|wSS����O���c�]5Q����	f�w���-�)C����:f�,� �S})He����V���{����{��O�'�T�M3�4�?�I�}y$�;��8��[�|�/{�S\lu����
9��=)���p��x�pl�v]�����o����������vJ��!���	R�n���O&6��C?���O)=���Xu�(]��s�T�S��SK�	���A��H���_����j>'C��+%���'����!�!����x^m��|���M�Fz�lNe*C��+R�����@/+������K�Sv��'�D��������f,(���SA�I`!m��TP��^}G�~1����?J���i��ku�g���+�����"�k������~��������IK���N\q
G$��xq���������j��"�P���6�T4�=yg[,�k��D�E�S��_��Rzz�Z���_���zOekimB?�Q[omf�a�n�����*9=�`�x������V��#�����`�U�uH����O0������!��/D=�)��$rD�����xO��w��I"'y��O�G�8��OaB?������/$~���xS�aE6�����=��hB�����D�pH}���5w���c]"{�p�r}t�gxSL�Tw>�������.\��Ka�d�S��j8�m�R����~z���t��������uF�ey�P��i��o�gr�'�A�		i�&������Z�J�Y�P����r����Zq�T_�w�:^8��'��������\���{���E����������?��d3R�{��w��k�{����FX����y,7&�
�&o��&�.�O�L���j*�~l�6��]����iz�M����K~�Kz�|���M�lQ5�����*a�A����@b6���O���w��\�N8*��yWKA{%T��*���z���#<�c�x�U��w|�g��[��iw��[��$�������;�K)�u��r�������p�a$:��V��H&��%�C���%L��(�����-�=��p*����;N���7u�8!��R��_PA�zhe�N8�g��$��|1��b�U�o2]���H�4�y���!��:��n��$7�\��x���~}@8�1���!��N6���	��?*����J+�
U�pz���t��}���W{�����P���/89k������H x]{����$��=��j��|~����mq��9N�-%!H���������;���B/�^W����Y���������?!���oo���<E�|;Lb�}:K�����v�� ��SAJBdd_:�TP��R����_��.������������eyx��K���k������q���������S�jj�����E��6u��>a�j�(�!Ju�/�v�&r�u����{�v�RS
���1�e�c����)��rO!�o-c��E�JRT��l-�Mx�?j���,�YJ��R4V��	�n�5�z�O0X)�mhKKz�OpW:Z#{��L����oG5�����i�S �d7KHd��T���"�)M����s�t��4���R���KY�������e�mY�����y�{HP��(?w���d�6�������]�~�J;.1��3����t�?y�G3:M�\C��R(�����\��v�?�ob�:�S���x(v�������K��RW	(T)����x0�������b*Y-J����X��:[
c[��Ha����R�w���Y��7Gx������,�	{N��������X�?8oO�/�O�+�;a��jz�f:��;���%}k��������cu*��q	,����]�)�v�;�������Z:"�/�6k�����^kS�)}���_��N��|�im�6��a�W����T��	{�{�&�ay�M��7���6������P������&���A+/������t�������2�O��B��T���&�����f�{7����X%�'����n/�E��`�����5�?�@?���\�=�Y�]�|x,�������'k
�iC:'@��GgQ�a��8M�'��7&��&�)�O2�9�4��6���������U���]�3��Q�n�K\)]���%���������*�[��h��J��@?����L9�����#�J��K)1*���ob�:����������o��"��E_�{�w�����P�;����3�u�K��yU>��i���3���@?��-|0�����!>o����pI�����\���aD���czo�g.��Bbk?��'������$�-�������S{un�Nu8'KI{Ou8.1BH���Z�w�':��ra����*��G�T��s���/2���jz�M�'��#�_����OYIl�fvyD�)��*��K�}���P�Hv/u��C_zZ�:x��o^���h�Kr�D����y�k�]�x/���n��7�uT6��&�����&6���7����BN�C,�Bkf����������:��D��B�����&�v�p�V��t�3�
S}�2O
V+>��=t���,�p8���o�?:�Y�o��)+��COZD�k
a�;��"/��j���o�p{1=�
|SB���K&��>������^*&��Qry���Y����<�M	��{�S�������T<z;����K���
?��M��|S��A��,�.����xc�=*ZRz�v��2����������������IJL�u}���B�(!����FqY�'i2��v�;"���zp>��Sy��!����{��N�"�[����=���7)�;�$BF����t�������*��9����N�[:�����>V��]�i?2CM�D���*�[\h�T��wcP��4M��)��?�D�/�D����R�]
�d��xYD�m]��O��[�o!�")
�������Y������=8��o��S�+��n�^���|�pwP���U��R��:*�Jk���Zyk��\�r���X!'��a2lB4��Q��Z'�-<���5Q�����vQ��or��X7�	^��79��������J{�t���Me�7�����7�	�Tt�&���f<������}�`d�Q�V�e8���H4���M���a������YNu�~��q[��S���rWo�Me����G9;[+�\��N�������M|�)�;�ob�:|S�:�d���wBb����?�T�kQ�w�'M�0�u�R�>�\�Sw%����]0K����D��u'q���M�bw�����g���'�/�>��q�a[���'���r�����M�N2���bz�f:��	
�YzOu<'�E;*����%cIT��[��.���|!u_�>�:��`u�8��
Y]k_�����S��-������Rz]c�����U/��k�	���#�r�C=��o"'�k���#�v�����);�&H��<������t���#
��^��!�.��R� �{*[Kk����zk3�����i�*����}2n����S��J1r��=�'l�����?�L]��p'�����;A�}�^gCn��J���;�T�bu�:�����q�����~�J�u���Ms��8��r����8��*���^W�L���:�������k�����G�MRu���7��:�;�G��!�sbtdl,]��%�������db�:�S���xMi����1�{���^��&�n�	d�G���j5��Z��V�����/�BO�a�~�����9�������g�^�^�D��%������R:��~�G��^Wo]��}����:Lbry�N��L���:�������:�����{*��q���`S�[�_���!.������0p��E�L�EF��WM��)�T?�H�/�H�������X��"�R���}��FU04I����}
�`9�P2�<����p'0����Z�`k�?e���+^��d����yB��j������6������vo��D����:+���n���u���?��������>��b��p'��q$��J�u���4�sJ�d�,�&���]Q��rN���M��������&�go�&yH��:[�A�7L3���i�>��f4���������� p��1\UwJ�"�7mk6�=���G���|S���8����g}vS���v	 )1���y�R,��;}`b�*|���_�s�'�'������� ���G��?�*\[;�a�i)�%Rn����Z� ��~����I�U��M��>�/�k��"��"��z��lw�xH���4�}�������Gk?��'�����Bz&1c����\�^�^]�_bRA�I��QG=������.��|�qdF�59|du���� �T|�I�-����j{����mW��'��pr��fj�<h/��������
w�5YD�-yS�b�:aj���$��kbj����<M~
�������F]z-T����N.f1V�{*[Kk���_���f���D b�i��������
w\��8����������-,�^����/D-��"oI{����5q�������x�;=�����M�'��o�t�'7�����8�Y�#4�:��A�X*�JStslx����X!�=��
���1s�V5q�#|5f8��Qz�Sy��@?��~�����|��;�M�P�v�>Kpm��~z���t�'�}�)���v�����%{ub&'�{��O�i�q���G.�g��	r��<�#t,����|~9y��'|J��"�Of�����bv��-�p���9�\�j�U�Ia��W����t>8oO�o�;U
5qLbgo���TPg����=���t�)��JOu<.1B����E}�~rG1��������#b�bNH�-������^kS��}���_����O���E?�]Q�.�^���X���KEL�C������dQ��������O�c�����������J�O�'w-f�P��fk$����5��Q�XZ��O��[��!V�.�6���)���5	]�����H�HH�
�tv��5,��m�=-��	�d��EH�������m�y	;b�%��,T��8O����t�����{��OH'�D:Q�&��Y��cR
���O��$y����r�X�P0�^�R���p��^>";�/H'��m�zc�gr�	�R�]��D���I����2�v��K�����E�etH��Ml�A�t�������v��=�����/�]����M^M���U)W�n�9v.?�?AOV$4P�%�m���B�O�I�'6�J���coum�SD��l�[���W���N�&����G�����u|�@:i���$b�4�w3��I'�������:������{*��q�eJ�/X�kQ�%������Y��
�X�����.����j{�MI'��+��+z�)^�$�{:�����H���O�CL�`;Y�R�I{��`{p�$���	�!���
6�`��=�N:�k�{"��9m-C����d,��S�ZZ��N��[�����Rg�i��������	T���	j��K�,���<�@c�ii�[��O�t,�S��t<F�j3���&��6��^j
�Qk��0�����q�����~
Z1O<���i&�� z�0���p�4�����%�j^�(5|�����p|�������!�Tm11�����<�������S����_gA/O�\M��r�D��y���\[����7���)�O��'�GQqP���DH���5W��Bd%�}�0+��Jb�N��q��"f������5M�}�=�K�g>����c�S`�MU��Ko]��O5��e��K�S�Q�YCb�|r�U���#��'����6�����Tz���5f���t&�����
\+Q����uG
�Q����33F�J	Y��9)�n������iz�M����K~�KzO?��C?��~�LLv���.��VA�h2��T�z�!����z��V
[}LQk��
�@�Z�O���m<B��K �v�!�)mZ�[GeSimB<�Q+om��VLBP���X!'�A|��%������N�o���_M��u�]��4�,b���x�,�F8��������;?t��W����$�8!����p�!���p�JO2�H��iv=�5��d'p�`�{v���99�	����2:a-�g{���V�[Wop�se2w��||�`��G��W�m��b���2��o�!�
��+�;�ob�:�S�>��h9����A!��t���X�KG��j2�G.���^I;1)�V^���JV�E�� ���"�]��XJ����x'$V�����y�N#/���W���~��N�]�uz�����P�N&������\��
��{Ou8'+����SA�K`��`c�[�_������	�Y��>5��d]E��b�k������p����������W�uQM@�}B
�"`�dq���x
`{Oc�������=DRxv#H��J�/���c1���������gk��N���d�k�S��;��0[���������t������t*�o�:+��%�Jv���_I'���	�:��W�	�s�:��g�N�-M�)8������O�����nd����|A�;��tJ�)��'�)MH��C:�4G��f�d/'��:��U��)=����0��&���r~qB��e����p�*R��J�@:��(�b��dg�MS��[M���z���gWI���
?��N������ ��d'rfq����k~�zIvr��H��r��m*j��m*!�`�^3��RHM�	��4TG���e��V���vb��{�pNx���/o?�����z{�r�*QN���M�A�����Lb�}:k����v�^n����:����Q_4QV���ed�j���]�)�N��,�������u
��a����U�km�:��}��|����R�S[�p�����i��>aYv���Ip����N�C��%���E%n=��e��Qc��|x�5�����N�*��9��_��QY<�-k��l-�Mx�?j���,3�&�{����}�uB����`�RYPu_Z�s0�*p��O��������&��Pp`yB��,�.��T�D�AO�L]�?����c���h�?�R�]���y�f�,,U�p���I�P�;M���	!�����"�������gF6�\Bqw���n�	%�/�O4*��lv;�7"����eQ�In��+����O������D�Ww�+�O�).���-��g�~'��
g��O���*iBz��\���9���R�j�4���H��%ez&�~S��z�����bC�$;p�.��k���Jx���������[IvN!�N&Q�����t�O���nC��
�xN��
��SA�K�����w�����&����YG%��.�w�D�~������}���/pI�����
���.����dX�}����'��c�)_o"'�!�	������Y�����'G�	��"�^�S���AO|wp�fk��r0�#��uT6��&�����&AO�K,����BN����yy��Q��=a���O���=e�a��_e���=ol+g�v�]����Te����:��\=��SI���NyB:�=��C:�	���H'��������y���%�������\>�6����N��B:A��v�+�gu�g�h���Y�#�������$��@:�L��I'*���'�j&�z�v*ye��?��������ig�|v��N��=&(�����]��v�������x������'��t�=d��~���PFR�t���c����k*�q��K�Srrd�^?�#o/S�	�rRm�ig��[�7�d��$:)�����i�^�[��72��C�����i��V�P]���d���t��x���PH�DG'���R�L7�N��Z�rN�sO4�O��s�J������M������	����UQ@�h�j�:A_=Zu@�sDpA]��|��:�|	���%�\%��S�)�����fW�&���d�:�O���V���&�?!����$;�K�����$;9�[��Iv�FQl��PJ�������qv�{i��B���>�����i��{�+���r-�T&|S��.:|S��ME�o�i���\�e|����UAJ���.8�U> \W.g�qE��U��-�,�C�����Z4��.A��P@���]��� �r�7U����}7���4���A�=�	��U���Ml�A�o*�O��ey��G�4'�*K�������abhE�����D�d��^2(B��j�.�l������Ef�"������}~��8*;�F	��z]���4�|=��A����!�H\�7�[��|�o�
|�LbBH[��L�z�4`yx����:�����F�A�K`!�������7�GfD�o)D���9�C$��BM���V��Wm��)�T>wE�/pEo�lm�ZANu�<�E���.����W�
y8l�����|���0n+�|!�^s�C�S�&{�s(�V�qJ:�k%��
�7[�X�.!�X[OekimB;�Q[omf�)6j��4V��	����[%��S��J1���'l� z���������~b_������8���H@$D�PB���P9���OuB?�}��C?�	�T��'Ls��J	���Os����'�k��$p��h�GW%
��������Sfp
�I�l��f�KL/�����:��Eo:�\H�c��$J|����MIX�}~�������bU��C�Z�d\��V"���K79v`�B��nG)���V�9vUVkc�1^���f�]�X�������5o��(J����?�����G�N������'e��~�����[�N
��d�m:����c:��O���K|�C�u<'�6Tw|�_�:��������K?��p�dl�����=6���G`K}�wZcu�}��Z��O�s�����>�)<8|[�9���'Y|�-.�]�'��P;c�Zcv��"�X��x���������cqv��,�����WbJ?�k��w/O/[�H?��X���o=����	��Gm���eb2��z����}7��/jZO0X)�f\nzO�	[.����'��OH�M����"1��g����,�\JG6U�8��/�����?����z�I���IR#��L��i����+��������H.��V#N]b�RT����cO���>��!kV��X�^��R�V���C���6��O5������J(��u����������T�'���J?�Z\^�:�M�� ��/�\������ 2u��������Za��	.��hj�(,�����Z\<� b_��yb��lw����u��z�ki���$���d|��<��O�`����'��Q������n�I������\E,�	�=����sG�Yu8/���u-����qf���A-���%�VZ�������Ydu�}��Z��O��>��O~
����Z�(*K"2�j55�����}�o���D��r2�o]'��glv6�P�Q����2n���9�(����!�t��G�|����2�O8JM��T�������j���,3^�Q�Uz����}�N{O�	+e\����S��V�+��{��	ED�����v����2���(�d�O{�S���'7�����������rZjO���4��J�FA�	j��������=�g*�OYD�Z�EqI�� _�����s{*cx�����F�	y��R�,lp�?��.*A�=����)q���
?��O���w�,JM'/[Z�����z���%��=���p9A�SI2�h� �A�j�"�T�������>�g�q7�!t�q�����*�GD��"BA�w|P
U�~r�.�������O"!���n�SA���7`zOu8'!NS���=���I��Q_����K?��~*�����U>0��fL�����R`�4M��)��>�H�/�Ho��
=����DY��(L��_�e�,��UT+��M�){M���C'�2����(�}���������u��]��#{�fW�2O|q2N*��������w��V���Mb.�`���
9������rG�_�={u���:���u���s�����������j^N�����G�h�fJ%/.i��������������o�Z�v�b<>��4�C���'�x'y�_8	��9�%��_��6���h[D��w�R�U�����
���B'O�q8��|��&r��u�!.�<��JY��w��z,�=Z�t�Ll�A�o����lX�����$��)=�� }����n��m�;����X
�v�bu�ku������� �t�m����=�����(��&����$�Bz�l����ky�
��������g
���:���|�FI;�D�����TH��Hvm=�����}���U0���%/�\��}q�����7��o�7������
C�P��{_
|����6���������}�)U��)4���D�p�vi���|>zB�G�G|!E�+���TC���*B_�&�2���
��|���M��~_w�A]�!��J�:*�Jk���Zyk�tP��&5D�r�����������6���5Q�"�j�<���;�d|m�NE�{|S��;Ex��u��\��&U��%�gw��M�Gg8��Ma�7����'S��;vt��RQP��u){��j�|K�<J�|�8�A���`�D�������5�s�Go����7��0�;Ar��f�b�?�;��(�ka������&6���7���M����j��{&��7��4����E@d}�����x��(�T�.�?X5>
����@��7����T�[+�T�����������Y||/���^�#�]��������7ey'�D+q[�f:��7+�bb�����$cI.��SA�K����������M��w������D�|����\Mx�_cu�}��Z�N�sW4�W�^]�*��C�&_3T�w��E���}M�q6"��FcZl�'�k��	�2�����}M�A�**��y-*���t
Wy'���ZFy����i=����	��Gm������M\��4V��	F�5gP������"l�L�x+==�(��sO������)n�I,_����m��8W����=����:sP(,����~�?��Q�~��)*�;U�3y�e��������%����my�aw��<���|��%����������(���[�.�6:2q����L���)��N|�����j��up��������J��1��O��������.^c]���O�$9e���C.d�n_-�o��u�>X���)����Z7�V���_�{p^M������u}�����}&�>�;Q��J�l�^�@��Y������{������R�)������S�P�I���w3���_�����z*��9	,�#�z*��q�P`�����%���O���vb?�:�;j�G_<D��b�k������~���������uI)���8��8�y/����mk1x%T��+b_[`�l%�M)��[�	�����!���(h�x�n�S�i����m�n-���R�Rl=����	��Gm���eb2��{����}4i��?�`��E���S��Ve��9X�GM��z��J�$d�/z���i�-6K��%���Cu��~���R�)M����o�t��4���Z��4S�=Q%�I�,���E!�mG�>h�;I?{/2����mt	_)5���1�6:�����+�)��]�G��s�m�s|�F�]���/������7���)}�~
�/Ww�<{�U��Ial����	�z��1��v�����'������K!#�ae���	����x�B7�����d�~�0Z��i�~6g��f�EcK���=�����~>8oO�oE?E�l;�+�m:S�7���m����Iu<'y�ak�A���W������K?�Gf�o����:0V%�l;�����'M�km�>��=��<�{m����x1�����!�ke�14����
o���}�������5�zE�k*F�R�<��w0��)����O%�#�eW��P<��[GeSim�;�Q+omb�<���5[Gc��������o����_��a����Q���(���6����o
�`�e���7a���������WzD�o%{�H������o��a���h�7��$y#2�����
���Q���:�p%	D�(dI��F���<���\B�h$�p'�Q$#���&������&�	w�C�on��`��v���vR~�|*���M������D�wJ�V����.!Y�fc5�J��.�M�B[!��L�I�%��,���Q��(Yv���F]� �0�,�|m�N�����N���Wz�l&����7��I�ryo��D�N/�|SR��$���n�S�o*)u��D���w|S���d
*�w|S)a_��Q������,�>�:��B,�rI*������^kS��>�D�x���N�j�M� ���	��^��7|���=�M�	�)&��=�M��a,����)�DW����y���|S�G!��Q�TZ��M��[�)���N�&5���� ��wY�K������58���Q��;��|�?����x���%�e���C�S5��y��:�)�=w��B���<�����p�����o�Z��xr�(PD�@'�*������U�g�`R��M�E�M��`Y���~|{X��������Y������<�;yo��&��]����8X\�K���K)6&��}~�������\]��������+��lg�'��/�!zd/<+Jp��#��Q�����B�k��\�3(��^����M(�����������7E�h���!W�����|p��T_���B|ObeFj����t*T��WW\�^bq:��9	,��N�wKG�z���3�z���M�PwB����U�ymT�^����Z����6�����h���=�������p{��(�v��>A_�Z�Hy�D�$�����z2U����~����Wg�.��]���)���j�%�nv��z]A6�7f��l*�M��?j��M���\JmR���	�Z��a)o��x�"��z]��
�,>�o����d�C���&�Dj��z�����0���}�JY=�� d��|�{F�gw��M�Gg���Me�7%�I�Yh�$U/:E�����:
��J��Ia�I�����=ZF�h��n[D�����\�_�lyY�T�q5�7��2�7>��qs�n�O�L����
|�W���7�����rN�AL��U�{��&����Q{�����
 %��f�^��L5��� 	<l���qT}V��*oR�7T���,\r�sI)����!L3{b���$��[o?��B��7�[����MV�oJ5�*������|��:�z*��sR����z*����a�c-���.�T9'~�?�:�-�J����YV[]k_�����S��-����s��C8��h��1�����=�cMZ(�G_���b`������m���b��������,���tn�|i�����erBv"
cn����	��G-��y�kAi����BN�7���n��Q��Ky��b������V�!n���� '���ZR]��\�AN��yR]E0*��BFB����3�w8���t�?z�U�t���j%�U���f�RS�/�+T�����"�>U���@�����xt����B����e��%���,[�=
S�����t�#�d����������F>G>���TW���'~�!���I'�RX�0�v|cO\-���
�����	��z��+��v��(�%�z�3����5�~c(>��hbo��l2��s���9�8O���y�������h��[��~�tJ^#��'QB�z�f:��J5�JR]�����$/�@�z*��qY�x��E}�t�G��n������"�.�lP�v����U�kmJ:��]��\�I����MI'kbB�sJ0����	{$�x�B��Kgq���#�d�����9��t�[d��'��
��OI�:D:Ys��/[�H:�PCu��l,�MH�?j���H�b|�mR}UY!'��t*�����Q�H'6�m�twGT&�x-�m��_0�3%��k�S���O�1R�&���3#�my���F}.������v���N�������wj��
��T�y����!�kR/{�� h�9er�������(R,�����p.���u��7z�p'���"�������&����W�'^(#��O��q����N_�BH������J=��������_�������,�)1x����kN�F?%|�d�v�2���i�C�	�_�k�Z��������K+�&�H�\�K�(�Q'�0���H��W�	���t=2�zo?�K^Y�rr���X_b�H���4:ae�����g���%�_cr*��y�����SA�MY�v_�v-�����Q���G��,>������e��z
J��Z��P/(o�����M4T�*e'k�	%R�A=�]�'��R;cKNv�/aE��F;�����]�q��������{YT��4�p=��Rw(���/C-;��A�\�{*�Kk�L��5���qi�����PNO02������S��/.-�����`d���z����j��K�.nM�2���xvPb�k�KQ�]��@w3N���+;%N��8)��I�
&j��L��S����.�����;
�XT8)
6�����9�%�W�����d2CO�'O�J��I��'���PlD�p��~=�(�#ZWHG8�&���'�����(qR��yx�W;����_�{���`l�e]��!����o�"}�����Ia�v���{ZM>,d���-%�
#��O.�gE@�t��A�r��_P���De��o�<�z�yx�#������xmE�w�}#o{yq�:9q�q��(����L������g�mJ��H?y�2��6<�����#
����

JJ��Z�SR�|T�|��t���7��Ft�6����x_��P\�.�K����%�4!��-��N���+�����oH�n�Ey1���e�b�)K>�t����w���Yp�&V���~F^2����~F^jT���~FBE���q�3����Q1�Cv��y�-���q5#��@����Jk3���3"��� {%"���(�DDA���}�{�d�5��!h�6�����{(�<�Y\�X�Re��]�tu�R�����c����Rr�<�O�(��L�t���)��*�_��D_b����l�E���������%x���D��o��mG\�X�Q\�	�&7Oj%���gUOQ/���s<$v�e�U�`�Dx'�U4�'��;[KDY��"���S:"
9 ��-�|�u�BD�{O���yY���4�l�	�/1/ya��pN�RG=�R���e��v�P�K�J����%�t��E}����[J���Yvg�ClT(���[dx�}��Z�Q�|S�|�I�������(H��]�2�]^��O��V�1x�@(bDc�$��3�������9/��
�Q�JD��pd���/cl�u1���T���fT�_5��f���)�����PNO0�(pq��=�'�B���A$�9�#�}y{�`��(�WLm��\�{�F�4O�!zh��������^T�]MI�%~v��%f�TP�'����K�k��Y���*����"V9�Z�zyr�I�R[I=16{���'\�Q����Y[�����(���nk�aJIE�I��2�����*I+_�;Q����c�����i��%�m��Z5���4�	�j�I�Fy���+e��(���y2�3�_�j�ya������d�s<4�s.9<
:5�Kk����L�(S�i�#��gQ�m��h�VC.�r��X�J�����i�H����	]oH�
P��=���d,��X���:����v���mJj?9#���>2<��6�����|��%�j~��)��>j�>�=%��T�U�� [�-B��~�����.���5��B����.�_p�XMN������m�$�d�+�SR��y��-�����2PR�/���kzOeimFI�Usomf�<�e���X(�'L�qS.�pO�	Ce\����S��\�?���B���6Jr��FI�������v���2��T�����3J*��*G%J*�(����b��I�N�e��S���j��W^��<�#��`tn_I;���}C����7#cf�3e��y�w)�8FI����zH%�GII�i���9���<3���(��l�E������\^^6��*�y:%Ym
�Q��	r�����(_~�nl���g���y2��Ur�ET�����P����g���������Xz���b�i�^&F��j��l�U����>q�>%UT��x��+m��	UH��
�x\�n�/��KQ�S3��m_���E}���%��:���C���[JV5������Z����6���?������()5!s�
'����W[�	{�x�f���Q��/jn�@�,4II{���}K,����JkSR����\o��1J*%S��{*�Kk3J���{k3�������=��rz��D��l�d��`0T>���ozO�	s��v�/o�`��$����AA��?u�����B5\�$���+�B����x��%�~v��%�f�TRS���p����(��GI����������W���S���W���d��1���|BN(.�� UL�,8���V�8%��zz�PFJ
��{A=������o
����J��DI}`g�/J�T���rJ��%�B�O[{�P���U�T�#������������V
��Q��RBy����ZQ=�L������><6���Tq������)�\�RR�Vs�zl��m*ye����?��%�SV��2��	-��L�BY=\||G%��:����"��SA���M�
�{kQ�������'&R9>0<g%�K()v��{c�����6���?�����J��%��F�B���t]��O�c�^1>H�1����A/q�L(v�i����w�{�J=C�tSZ����1%����|T�z��
R��
	������6#�����6Q���}F�:���U���Y�t���
R����������
R<���u�]�k�(�Q>P�(�N�=6��i�F��I����]��J���n<�iFD��2)Q4#�H)]sQ^���U{���Yp�$
��5c!X��6���U�lH��$�����$vT]]�K�����p��F���lQ��KD
D*��|��%���y6T	na(���-���v6��DD���(|����&���E�w��X��`&�@F(|��^��m�&���<�(�X�����Q�S�s6(�Q������%��<�Uk68{���6�i��K|����wd���)��#�����H!]O��b��L��t=����5�x����Il,��TP�S3������m"j?9#����|`x��5���l5��b�k��������pM���7<T��=�p����-�-	"�w�Woo��>A~.xT�����x9A_]\u�
���rgj$�|��?@�����1	EW�(�����2�P�7DA������6#�����6�K�&�����"9�_��am����c��j�z�����b�'�+	����?��?�1/�*����
��Q"7SJ���$��k����<#����qV"�����j�P2���}���+D��h(~��O����%�Ic4���8G��+^�N==]8�M�G�o�Py��
���������,[~�z��'�eR������_�H�������g$�8�b���6�����e���-�	H�VO/e��u�tg�M�,��9�PE���Y�E�K����p(�B�v����tHu�{�7)5�z��NIG�<����?`�D�$D��!B�%D�����P����k/��J�1j���=��p�RJ�z������B�#
G������II�K@,�[lx�}��Z��P������I8TP
����R$K�#d8�]�'�A��;���$�i�-I���C��x;a*4�n�b���;{�>���zd�O/��������$�0�������������,V��S�N{Oc���`�Xd����ZO�	��E���,���<�@����-,�L8����IE�f�9)
e�#�IR=�z6��T	����@/3N���+%N��8���I�\C ���L��\z�o2�j3�op��R�=
�����:_����p�Y
�(�a��[������&U��(^<a��\�y�ky��PYr�����gZ���>���%N�|?0����� U�pR���:sb��l�� UZe��
�/M�M��%���D�,E���z9�u��hTii����[
�5t�RR�E������4�8-�G5�Zj����{Wt2���O\�OIeJ
���6����1������W�������|Ew�B�]�������bk�h�@?93�$�R6��A�r�#QsD�gT�v���?������`���,��%��HR3^E<��K��y��G+L<n�L?n��T|��u[��c��d�6����0�
�6#M�K������m.�33acy���m�4��"��������KR���Kg�}��/����2%��^���=I�����9��*7�zJqmi=��������z	 ���v/#pE����6%),�|��S������_�13-)fi���a�d�%�DJ��=I
^v���${����h���$�5���XG�k-.I����xks��g:HR`��y�n�on+�;
��,L��-1�y��]#IA9�i?b(f~I�����=�
@7QHv9D�zl��+h�~�I���U�����W���4t/
��n�
��nFzz���Z��lP��H��f�q�R��8���������Y>��B���H�sA6T�Z����nS/�(�s���/N��T��{�o����1��Q�j�l�=*�X���h�
�x�\�a=��,�[�|�F7Y�f�D(;[���y&!��
�'
�j-j�25��$�����M�G��7�����v��u(x�c
��c��C�tc;E��a��=H�k���i�z��<,Q��U�x�ij�hY2�v��������e8�dog�<��'��@ <L��n{��G�7�2b��-�N)�kJ"*�}M���51��t�)����8��S��z�l��!yX;l�gWW)�vv�^-�5tG(^����F��������p\������|*������������������e�#�����N������2\��_w��z;���Y-�C���)����w��F�I��N���;q���,��2���z�K_�^]{��J`]�����AAZ8�������������K���i�F�g��F�q�������s�����
c:(�	c-f
�������lr���B�����|�xM��q����lKV��^MG�b����\�D��!g���`����e�P765"n�I���2V����L�
J�T��nQ���������rZ\VB�w�:,�K	{(`�*������U��.P�=��:v��im�@�)����9f�q�����^���*B�9�>�O�����W�1��������!��&�yI��JI����5����C���4���(��LC��9��:���}�����}�����U���O2��/'�vd	�]#���x�Q���)ch"��4��1hLV@��D-[�v�hf����r�vy��E�6����O���O�:�%3�/����Y#j�����<��ww�PK��4�����PK${2K18sept/pg_part_patch/10_2.outUX���Y���Y���k�%�q-�]��|�+��@�7���/[W�)�p��b�I|����+��{wW�������"���������\����������~��������~�;x�������������o>�����O?}���&��bV�*u��������O��������"����W�����o_>�����\�Iy_�O����������~k~�4��������W/_����O�}�Q}��0���_����}��������~��W�����?~���w���Q��������?|��?~�������Wi���������7_����������������M����_�������>��{��z����?~����?��O����[_�����������������j�N%$�L��M!�����7�k������������/_M�������@����?}��W/�?����{���?��/��~����t�����?�I����������/���O?���?�������c�%kg\�*z�IK���\�����Y��1-�	%���3�x�b9�_0E���&H�x��<��Q�n��}�f��Y��>|���i�����>�h\Q�=<54���+mj�D�����D����������_~���������O?����?�������#������?���^�����������3^4^x
���%������E]h)����p[bR1F�p�sh��./[D�Ng���0x����e^�k�����~����^S�E�y{�1;BZ��6%7����7�_L���3��H����!�9����v���K�R����NE����'��������i<�
���h�*��{�����?��v�[����H�����_7$��6�X�*I�?O7m�/w9n����������A���N}o�Eo��1?���������7?���Lum���qt�����1����H�{�@)��x7������30�7�B����}v �%@��
����=�������@N�����R,��:\��x?������?��/�����	.:|h���hO�����%!G ��z������+B������w��?}O���O?���g�}j�����{��E����h��N�p�tt�Hq@�y�+M-e��B/�!}1�B�Z-(�����������(k�m����0z���X/x��~���Us�L�bV�O���N�(��!OG�g���9���$i�U��\���p&�iZ+2�%G��]�4�vh����LR��K�i�f�h������d�xmJ���-���:�9{ON�/M�z�.������~N[J��
m���-~��������z���l^�o���������%yw����c��T�NsZ�W����~�4������S�"����;�O}�$���:VB�d�5*���|��	x���������A��t�������~���>�?/�����_>}�������7_��9�|����A��`�k���)���n�OJ���9nY6:��/y^���}��u?4�������|K�G��?~��������~�#��_��/����~��O~������|����>�����O��B�Dk�M_��;7�?��D��=m)KJ31�/�-OPM���(@
A����}Qq�=��� G�A�������Zl.f��P����3�T���2��K�����}4|����YT
6�������Nd���4��!�@x�3�o���������_������_������?|��O������������z������s���*F�����pNw(zuQ�!�_"["���Y��Aug#��YsQ�U����>����?�6�m�6���,�?�u�NQ*�4�CE���^O��z��#���n��_� ���G&�o�������_������e�G�g���u�������?������~�����O?�1~������'�{.�������hHz��?�Zu�5�C4�����!�Z�:N��;�#A�i�������9���Y�G�D��
&�s����$
��"M~nC�����YV�b�y]W���?��2��������@�����K�4��4�!����>��O���_:6z��T���!;|s��8+D��!�oO�?���c�w'�A/�a�Ac�D���K`�lNX&��lG�p6��9��x�gn�7�'�IL�0Q�����0Qv�DY&*#�C�P��+�����N���x����"���1e�.�3Q��#:��e�I6�x��������_Q�/�=�D��������L��1�(���r�~���D%-�D��i�2L�}:e��6��
�&c��*��f��ls6w����zd�x���)z��eeg��I��NE1�)�a�rJ+��5�l>U���8��b�r�����c�|��<����k,��AL"���
.V��M�3��|3=����D�<���A��������P�e�^P�A�'��W�DP���u��\�{��u��i
��:�)�
^��W���^W���^��L�}<>�o >=g�bb��H�<�V��H3�l5�b��p�0� ����z!����H&�}d����S�e��u��2Q����m�'K���2)%e8�g�D��z
��w���x&S��+Y�dwG�d\N�[,�;8z)p�1������}�����M��;eGy:3%�A������Q���-�mQq�a2�l�{5Fv2l��QN(/����lU�-gU_��$EX�$�o������l�_;fD��B�����Y�t(�q�/�9�����r�����tX��X���^%��E�?kTr����~��O�;V�����M>��t�)z����+�k�%ty$��*�Lz��Y��S�t��j�.G�Ti�Zz�k\���8$�D��k#���������c�L��d�*XD��h��k[d�^�(Ow�	L��)�����2�0��VL��H�'��0�S
	�#��c��C�H/��e�W�bL���[r[�y�#f���@�P�9����T?���Ur�G��
D�'�R�V�T�9ai:���*��9!�(8"�'�\_�Ug����V����|�O��5���
[��D�G���A�S�z�,��4���y.keCb!���Cz��]�Q�3��rS2f��+���%x'�;��X:�^���,����2�r��o.��r�PG��t[^��@g��������1���8!c*o���5��2���F^(��98d�����r4�����_Z����O�'���o�K1�K���%�~s�����x�&���w�h�v��%�q��`2-���=���GQ����Y�2��~��������Xcm�V��<C�r�Tu�U�i
�x�XpG�qG[9�@����g�h.�Z����$��<�]+WL:��dO�+s)E�OV"OD	�b�_6��z'��:p�R5�N1�y�O�Z�t�(	�G��@�d.F?�����//���<"$SL�4E�)��$hS�S!�%���kr������A.�%V���z=���5$���!�!�)y�/T��k��0���2��Sz|�VCP�"��9���r/E�_N��F?w��#Tr���_���wL~�<��,$���B��G����,I���;��z��U1�7r�Sg��sG:
�e5��;�D m*������w���j\��Gt���<JE�����0"��i��#��o��d�(���j�d��0 ��yD��]����R� ��/������K1��<���*�Au���5F�<��Td9Nr�5����R5��Q��#ZM�ge�Cf��D��g����V'E�����/r���(<�;*F��� �#��3V
�	�.:8�:s��D�}��wav��W��l�
���J�D��0;�%z���#�/�<����>�����������8��f0�u"JK�4�
O'��q�����b�#9�8����bH@v�����S�-ma&��8���&|1�B�K���wP}���S���M0e�����~V�!q/�/Ok��qDs"�9aTvq���w����cS�]��H��;j�"[���@�8����c��G���I���mA�����QV�K��!���k@�S����8B��c�v@�:;v�q�(^b��
i��8�ukZi�N�p�S5.��G������k	�����Z�6xS�#U4kc�Ud|q�Q|5��2�QGQ�Z���$����^>���V��8lR�,������X������Oj3��k�&�j5z2�V���/�R���Q���P��f%;����V�#3��[N�K�0�^'������ C��W��0�8�x�h��-F�	����n�z��� ��Bn�(W���f�F�5���(�5���=���v�p�����X�2�}�f����
������mE���i@�M�" ��,M���d�����H��^�bH@��)����_	��.7!A�_-�T�{��uwt�)����0��_U|H��%s<�^O��z
���xx�@xy.�t!��^�*G�7���s��pW���+���
<b�]e�V6���\'�w51��5S��N��G���C����\�(:�#D`Cb!���8z��]��#�9���HC`v��O�G��n�lH����+��^5����irU9\�����#K���G.��#��+���e������a����7M"���8J���I�8J�(	ea����A.�	�g}i�
2��3k�G���~�q04J0�)�:%�����#B��,h��M���<���9��nbc�C�-�2bqrR��dtI�a������(=������#4*B��d!����m�bz�����f�$[g��8O���e]@c�6~c��5���9"Ln����-������j��I���V:�qm4QY������0G�4�M�j�x��-F?���H*U�!i{H���!�N����1�^�(m"�I�����������V���Y���Qz<�Lo �<:�EgwZ�>��J���	!�������Qo���7]�MKvz���>���~e�0w�����Q:
�M0�f!92G�U��z1$�z
��w���8���X���!0;v��;z�
��!
�9e�\�bH�w�I;($��2� �(��k�b.��93�gK��0G�SQ.�)G�����9����Y�9��(�0G�����Z����
$�������S �`(��m���BS[��hT�����P�g�,����E�F��������PR���h4/B��X�2�Q~z�Qv:qT4���5zK���H4%������#���8^���E��4'v�VY�`J�FxL�scd�J��xM3��8��\���F����`��n,��f1���H+�|�f	��������i���]���QI���	�U��,F?���%Z��]2��-��D]m���Z	�#�D�i�Z�S!�e���_�tz���*"�)&�r�s}�^O��z
���x���@�y����%r������)��B�R�wb���E9�����Ls�����{3��A���%��Qf)s���/$=�D���@�
���^f��zv�F�>��)�c��#�2�Jleo��;��5�z1��;f))����?������2��������J�&6V��R�[��km(4�_*~��~���"T����"�3
wl+t�E��
x����v���b���yxay>Z�K�>�'�����
x�f��L����
:\����J�5��Z�cQ���P	%P
��4JV.��HE�k9��:!�������<?S���EnU��fP9'}y�V�k��;TsD
Ee�x�&��Y'&�����B.����w9��kWT~�b!q��-�w������"9�b����dmGo1{e
�~������]�^������
�Fk�-w	t�D��N�$AII�N�s~�j�,�|�DP��o���A�A=��E|�{������MeKd���y���:����=��d���S}�^C��<��7�����~�5(��Y��/������{�}���S9��z|�.bKo�t�dl����`�n�	�-sM����<��T9�6��Vb�vi9rP�����Cda����z��^��_��U�$��������
q�c.`SX����e�J��c�]����$��B��(���9�w�r!$�b��;Ul2f1���L&�&�)E�������D�_{�D�M
0Qu����0�-�
���.7�dVP6��P�@qo:�p)�I����e��4v���*��2r���sX(��,���z���	�����j�J���~����PxY�����%���_0�WY2�J����-�jrjK@u����b��f���3��������FC�h��b���[R�e��%)�b����>o�������vjM6�Q
==�:_4k�������o��;��h(�L'D��d8%r	cM�i����9V�A�
�������w�P���-	m��:�����t#�;����T������@�N���P(�R4�u5���(6E�9�{���P�N�Z�g;y�aw4=w�2��y���h(�.t���m����P�����N�����7hQl�"K��i�������DW+Cb����$�en/u����HB�\rmR��J;�#	E�-��:� ��A=���3Ho�v��F�P(#�T5cl���t�����
�X(�j|ldX(3`��P>E��Pk��o^b�&24���{u,ho���Bk�����u��^����@iZ�f���;���y�s����A����j����(��m��>�e���
*�$�H��$�K�&�_�&�~O�~�a��/�D���3���-]Pyj���LoV�k�v�52*��������7*��i��rMR���SpbT�f[�b���/g&���MCc�Zm��.�AHtA���Y��K��YL{���n^�"��@�8M;�EU�N�s~9`��P]�A=~��PnD�B�DP��qx\q(m�\�{�(���3V!Hy���|��<���u���^O��z
�(�xxj�@xz^���U[!7'S�gn�9M��;�#ei�*9IP���=�cw��t)��[�H>w'iE���h3�D��Q"�p@���R���������.��%���k@E�S_��H*���M�%1Kvwp�O���VK�z�pS�,�%~�f8p�n����H6�q�K�2����w��)�[�>$_�x��CJ�e���DI�%e_
��%e���q��8�V��,*������k�9�R��K9_��H�3������y����;^��gZ�%�.�6����%�rPkw[�Jy=���T��0v�����y�:������J�>��r:����F�0#t\[y��ki�;�NPMy@�@�g#�����r�P�����G��Z=b��M�P;U��acV����:��r��x+u_�u���8����U��Y��S���i\l��IE/�Iy4�g�f�N�s>'����aT�DP���0d�<k�2�GN
O�%5���z/'�n��)?�uMK2���l=����T����������
���5z6	qR�hp��������;��CmT����Ca��D	`�]<wBe=w���*�}�s"
����R��~�����D+l8YZ��u�Cv�L�Z"KK���;��z��b��s�����%�;�8c��)�b�%~s�P�1 K�:�]�JY� �	w0����
���N��
��qd�
E�0i�d��I�k��m�n�I�W#e'�I�'����0�~�gH>���QL�Z�8H��E�*�+M�6�RH�&� M����b<����'���+^��^t%=���q}�^.jU�0�6��`T��@�6K6�7�(;Sj��<��A��rO��h�\�HpR�*���r�j#Lb���'������4���-4rj�xV����b�.��@�J��bT.��d�e���KI9T�4�j��:?L�2�?���|��c�sE��r�a�{>%��%�����b��|J
X�S�K�HI���i����2��
��5�^J�miR�+��:�)�����M�u������W�!%�S�S�))'V���),�B�������#��9��F�� vW�l�b6��FA��^�@���\;���s?JI��B�������QR�����,���^J���z�F���L�
k���t��R^�X�w�y)�����t�J���[��w0R2�Q�u�|TR���t�+���5�[j��}=f���u[�PR��@��PR~@Iy)J���:�q�[������%���(�Cg9��"���xU<K�cK�L�j���/c�������hb�,m|'%���=���6�h�0A�B��1��,�WJ*���3_�i�2��~��>��())���7�
���h�\VO���� ���J!�W/�QR@��Lt�����8�<"����j�YJ�������>Q2�
�Tf�)N������.��������U��{����$J�GJ�����u2��))��`����"�GJ
X�s�K�HI���uk�<�^J�oJ�.,�mw{+6�PCp:�n� ���^O��z
))�x���@�����`X&����)z���;�.��}l&�x6'�*��i�r�x���|�.�"�a6v@�"��RR�(!��r���YRt����j���R�%�N}�^#�4����jI���t�I�����%~��nHe������U<��m�_(��D�zU4�X�Zg{��+�@�`�V��G�vk�TL������
��A��
J*U�a�3���qNk��I��A���xt�m_F�D��S�E�&7�v����xP����|Ot�h�4��DI���z��V1���0?�kkr��C0kZ;����B�{�{Z��%��%e���>�f`�j��s	���}�(`��Z�{�qu@9�P�E&�����u�B;�q+�NL��>+��D��b#�K*>�����V}�j'���{�\�*���Y�&d(�p��SRA K�G��F���:�������P�DP��z�[�g��N���	Ou{��'?���T�T�=��:Ta+��-
.�MJJ���5����ajxa� KJK��������Ma���aw�{�/�)��T���]Z���=Z�����=�f'x�@��U�`.���T8RR��x������8��Z"KK���;��z�()T�z2���%�;�(�D�l6KEf�"�tw����gm�1�����;�(��/nz���� K*��D�7��� �{[����d�����V�7�����2Rq�H�W��(�H�#�����haC�4��eU�RD5�A�[�7���^S��KQ@��>����j��G/�!U���`Fg����N�lT����j�5:%[���6/��9��ag0m2l��^�� �F��'H�<?A�_%2�CFO	����V.A�QY�9��_���r=����<m�*��KF��w��������1N�6����5{l�����y�������[kt����|=����F6�1&�g���S �R]\P�A����ek�j�p,��5���z/�����������y��u�=}6J���5d���j|�y����c�^�� 4dE�-`Y�-	b���aw5A��.�[���Hbw�@���;�"����]H�f����Y�JZ	�G��x��C��9YZ��=y ��Y"KK�l�;��z
�_���2�%�Y����?���cVK��N�����1�i�!u}?L���&����2�*,H�r�m���q����"����A#�u���������J��I��J:*II��8���8;Z[��S��;is�3Uj<�����fL�
;�6'<<�tU�ol�I����
]�lK�@Y_�Q���R/#��q���^�yt��p�HdT��LY)���%u��u?�PR���J�����y�^�M�w��h�6�E��X:�j�&��E,A����R�&OW���-B/���J.�G�"�l�U6wE��l����l�(A�P`N����!�
���iX�~�)�n{u}j����p�O�rnT�DP�	R����,�c���8��5�^J*m5{�T
����@���}����&@II�^���Tz<LMo L=M�JYJF
���V��|�d��a�5{A���T�i�b�5{���x�Y��\���]���MlcL��~��JGes��('KK� e�rK'*�D��z
(�w�������t�
�3�dw�r�N\��������=1����r"+��������������U�RR����%H=�
����\��
Zx��H�8�d/��j��e�<`��P�3�Ix�M[c��k�7�(U1����4�r3Z�u
���"���Fk^Kau
"8�)���+z\�V~���=E���\��(A*:�E�0c��t9'�gF�_�e�2lT������1����$;Bym������1��N (H��m�9k��:���5�o\4m�\_0F���E��}-��Mf�x�D��b-������ A*#.��$�X�''=��������IlT2	R(zeA�f��@�������D�	�LE��V����%�����'�v`���z/������\�~�3��~Z���E��Q���u�z���k�F��#��"�S6*��E�5J��368M�N#�~���aw	>��o����F�K����"J`�H�'/`;�.����%�[�'�~����)��'KK� E��`Yy�Z"KK�l�;��z�<��*)^-�Y����?�:��b�����:/E9�v�x}vVN�"��f�X��`�Fy�<f����d,�a� eP�\���c���K�� UtTy5P.2tT�QE(A
���2�B��8�xMx� eZ%{��M;-Y�E����u5�;y.QD�)O�3��k�=����II�����L�*HY���t�*!w��@�({K��Q(A�~O�~�����k���� ���	Y��qy+
��&�)������4��g�S�6���1A����!���nU		��R	R�B(��C:�k	��/�pOIY�f�6�Xm����Ap4-����r��M�LU�;�KO7�wLI�V��UW�d8������\��[-���<���Y]�y��M�r��h�\�{)�u���������y@�h_�Sqk$>����T�����*����
���5{��PR����3��<wn�����U�P�Q�d�d��a��c�C0K�sO�$���EY������>H@?�GN�tF�k����r���
P,>vWCdU����z�n^��S�tE�nZ^��!;|s��h��Z
q��w�S���gC���4�����q�Z�E}m���j_g��(e�Z/S�&�����Q��/��?��tH?e�������Y(�kOX(�I���Y������1�0#o��Zx9��H����8�����Z��/��G�M�;Z=��Zx6/� ����5��~�b���^$
�
�a�����;�sX��P��BbsY�<��A������,�.��|��L�9����6:�%��%
v������%b�d�I������Z��jW�g�r���,d�����f����v���s�����sY(_���t�f-�������e�jm��q�
��^z�y=���E�L/�
bg�N�s~���=��f����B��BbT�DP�ez��=�5�Nj�A�!�K�!�3Z'eX(�~��Q�$���S}�^#j�;<=��7�BE�o!�����"���t�s'����LV�[0`{�75����STDy������z�}r���"{��#Z������szp��nW���Bu':Cd]��9�^�^�������
���?�Paie���'<����b��]������?��2->�y(�j�dE����B��OJ6le\in6/�y(�jtldx(3����Zrg�e����|9�%e�kx��U�t�<�/�����-�2Z[��/Cn�>_ju�)Bq�9<��x(�i�3+c�
���
���z���J��3_�i�2<�y~6��T*F���*#x�Hk:�7�l(4,�S\*\.F��j�L��bB��S\���7�jUjc�:_s�j��L���d4��u��P~��&�P!C�	q���U�����2��k-�i&�|���n^O���@6�ufN{�*�
�|*GW�z7K��C��
���a����G��0���!�E���ZwP����B�6W{xZ�Ov�z=���5������y��y}��J��h�g.�AE;��w��P�N��=@[�P�c��C�a�*�4��H��Bch�~H'�E�Q�p6w�n�s�������2��Y
�E�^��zy��@P��{X���?eSs1����B��v�!�,�a��a1���$�)
���J�K.��w\���rJ��X~�,/�Z��n����[�8(+U�G�>;�8���[�����#<|�C�Z��
ty�1�������i�&z��#��O������.z�9(��x������0��B��l�x�&����r������>_����]��UR��tb*N�AQxg�X�����W�x=7�#{�`.��TQ3�����F.�����Y\�S�i��t�D��H��7�*����_k?�.�S����Tk+��y���t�zVE�@=�T{5�d8�sP��VT'�z��R�J���l��9���y��k.����r���C^g�m$H�@G��:�9^W���^��$�}<<�o <='��T=�pS[Nj\�����;�c.�
���g:�Ov��1
����;��;���d�����<:&�%�%�l'Y����t��������a�Z"KK�T�;��z
�	����jI���t����lP�����,'\�
�f��A�^�(S���u��TI�@*[��W�3����Q`�NZ�A����m�����r3���
)�j��d)7`��#�9��f��9(���c�NH�[����he�������{Y�G�V������/w6�P	�9[���,���Q��2����V�2�E�C�-!*��|������������A��r�@F���e�F�&�J
��z��k����X��vm���|��i�)�I,[Y�\�m�����*�x��yj
*7y����.�}=�@ng�.�V��s\�~�b��F�	�����d2��iD�~���<���A���ZN�N�s>�_r8�6K������I�DP��.�+8�f�E���ZwP���1�3���	�-G�n��u�z���k�F��#T�"����(V���8Ln[���gM�v_�gj�5���]�����E���5dG�;d����SIO^P��(����H�s'KK/��YX�5������k���h�������!;�c�"��{�b������U����x1���$��iL�iQ������Z���yP��rd����K�U�v��pP�����pP~�Ay��y��
����!�����[eY4t�pe�t!�)zT]�<4@]<���
(��Ayum�tT��}1gc���
d^��A�>#*�P`��<��!�#��D�/�yk�W�LE�/����??#��b^�O��,Eta����G��	������-��)����k�TD������A;�#	����o3�=�C�����Dy�+��Tn��X 1h�����9�������s�T��F���,T���0�E��:���XE7�MVf*��������4K��P��84k.��,��r�\Q�?�u��)a�@����I^W���^��,�<6�o 6=g��b��V*f���!��L�v�E��6���'������^���� �\$����*�"��f��Z��~���G������z����D���k�
["KK�<�;��z
s���a��A�#0Kvw��DQ����f��A��5����%~g9Q���-L�����mm�G�Sl9Q��GRQ	��"�����]��r1�8�n+B*�*B*� $�q�V�2�ks�h�<% <�E���V)�p-LHE/�����K�d�2=�{��[:����BJ��R*��Q�)�JE9��	��@�9-��[����3��/���R*<��2~�d9��^��("J��5�|R�V��C����
Z6
�y9k�)�%����
��g
fE�Y�%r ����	������1���Q�w��V���Mvu�,o��G'J*�����Fb"�U�N�S�qa��P�A��k����j�8������YsQ�����8�G���� ���.��-���^O��z
)��x��@�z^��/���k�E��V_���Is'��K��A���b�m9�.1�/�(-����(D�:�HKs��RR��;OC��di�()����B�,���^J���z��)�uW���$2Cww�'.�z�h�����V����,�;���*�����w0��LR�VJ*f�B����H�.7E*���>�eze
�&1Rq�H�W��(�H�#�R�x���F���m1K(��@����p�)�U�&O����j~���O�Q�e
��%'���kZ_�=�U�D���|����|Rq.w��#�@����4�sH���^'l��^�� �F����;?E
�2��d��F,�X���MEu,����h��ruE�3	i����dlrt���:=qi)R){'�F���50v[�cr�3�R�
�T��:|����������$:*IdHa�o����p
8T��D
���D���B�j�,�3���f�E�����*�!�3��b�y��S<::J���5����!j|!�	��RR:����y���s'��w���@������&"�]��nE�������h`��Z�`?JG�#e�xfO���N�D��C$���O]X�5 �����kP�Gc����!1Cv��:=����b��������&o�'��:��T�<����?h�_��"����y:L������3'�s�I7���@Lb����J���I��J*	�Ea��|�F��i�P��Z�A^Tf�sy���~�
I�R�:�R�xF�UtLg;#��U���x������������(:����Sr2Q�
�������s6R�z�{Z�����D�P���|��8C)�=�&��$��1p*�~1��������\K����H���5����]k!83�c��&�k�sq�][��R�D����2��A2��2��{|��/M.������t�z��@^���h���������X~E�2��8�����n���;c!u�YsQ�%���kKCn�:�i�h���C���� �$}�^C"*=��7����y)"������-hc�s'��#��SHV\r�L�>Q�t�Iz��u)��E���)��aD�G��������di���(�Fb�j���R��N}�^C�X+(��fI���t�I�U�|�����^�*iN����t�J��H5����,MI$[d�L,��
���tT��8q���rY��R����T~5P�2�TPRY���q�i`���m�����t�����S�{M-'19�'��"����s�'Nr�N2��^1�x��i$5��PR�O��q+30���))Gj��G��t��)���O�E��� CI��SRN��0��.AJ*2�k�N.r�Q��o-�8C�iP�r����v�W��;���&L��a�z��B)
:��J�f[UN�V�����^�cqo/��OH(�t�cUN���X��k��TJ*������*�
7�a��Z'�9���<W�,����?���Y7����]�����Uk.���T�r����:�t��0��7��^W���^���T~<L�o L=���|��(��(����B��F����;���+��a
gKp��}����\.�g;��s�����lF����x��QJ*;��
������F�E��"K���;��z
r�2����S
���?�Fl���b��s�[��_DV�~����������;���V^"gGez�FEQ�5
��wa�2q��TlE����B�DE�U^
��UTT���x�-r1�tjel!_��~�v�g8���qjYoI�,�Ex�m��`%��*���I!ql�8�ed���JGEa?X�����Mm�y��1/�s(�
�]�y'�N�~O�~�����U�t�����*}����B���*�Q�k���#��J�TZ��b���J�v���_����O	'�Fqg��k���<G�#��M��wM���L/j���L��:T��������u��-@E��p�<���IB5�Z �����P���9CL�Y"��j�����7��L���V�
5����RQe���fp\y��h)Gyra�(G����	PQ��W�!UP�P����,Z���\Y8��2��v�G*���+VA*���K�����Q������B���]��17Lg�c�'b?JE��L/����fi���,r9j�����R��N}�^#��Y�jI���tT1�.�[-�;��Ou��[��t���n�������(c�{N���d��gG:}�E`����\��se�(�>�����(���))�kO()�I��(|��^�q��������>����^�xsiYo�Z3�Av�(W��W�n��3t���Q<�f�����C��S()���=�u76��RR"��f���eGek����������J���'SR�L��b����b64-CS�)D9J
�t������S{+���R��Q
A;�=���h�����&v�@�K��lLHs������/�*4A��(;��ae�*��n��H(G������I�T1�Q4�)�I�j����(`9N.X,�cvT�(@E�X"���Y#b�f�E����vP|7U�y���@�Q�� !���0����T������@�S�
QR��eG���8C	e)&Lv�v�UT��M�;p��>f_�+$|L�t�V�&�p>t�����X;!��~���� #u<�{�p��t�Q]���b��+�:������k��
�Y�w�68q�����(���C������_
q��p=�A�����?����#p��(Hd���7�B��z9��J�{ E�Q��=0��fF93������������2B��ft9�a	-����u����L��z����V_K|��IT�^%�C��,��]�����l�k�����ez
���`��*1������,>������^��5���	u��u?�PP���wq~������D��k��INl�'��5D�2	��S�X���TN��\�`J��������,�t��x�����Q�,�B���j�C�6����Z��c�����GO7�'QP!H��� :�6�d8��K�-�aKm�����3V\Q�j'^�P��&,YsQ��������y��8+�y4���t6t����kHA��#S�"���r�R�^#
��S�6P���������.b=t�p��I����A��p:��.��}�����h�1y�����9rPV�MI�fi9��B��TKdi����z��^�a�,���2�dI����� �^��-�;�����;�"�����P��^���	��0+���f7�oYQ��eE��������^��5���2����(�j�le�(;���PV�Ys��A���Cg�k�YQ��ny�4���R�YQ�|,GxQr��BB���(��H�����y4%}M��J�v���^�����>��P������u@C=;Sg��<��A�����������_!���@Y�\����C���*�
��J��u�����~z��G�����>Vv�I���
����QQ�����Q�V���5l�_QM�C����a|^�����"��=
���)�$��u��46�d8%4�t]��%��kF+��I��U�gXP�����{)){�ej�����
q�Z����)���^O��z
))�x�j�@�:����m�^#s�d���o5a���a�2���'��
���gsR��J�If~nI-����6�>n�^&b?JI�.-�n��,-=%����s�X"KK���;��z
De#+WKb�����O)p�b��A/cN����X�w���S�n�r��M�9����� U�Q���t�7C�(��XT@�("��M;*�l�RRq@I��y�EJ*(�(CIeP(g�:��}H���Q;
#N���>A[������R����-,�~���@;��l��gUL-��{IM�J*��T2[*0��fI9
�TH����a��W���z�PR�{Z��%��%��^�b�v�*a��`J�\P�PU��8h��fI+GI�g����B��J!�i�8��[��boN��8��rR�8����
����#t ei���Q������)�������$N*{N�����m�f��|�(��vA5������e�5K��d�O�����z/'�4)������qOF�OV�r7�zs��^O��z
9��;�������iRP/������;�w���N7@�i��X;�c�l�}�)�7f���q�E:��-cg;��9��	�6�]�������^%t�h���R�'�N}�^#N
�'9Ag�$f��:q7�5�-w`��<��A��p��@�����Uj�f]��'��`������;��Qy�����r���&����*�Z�F���'�jz�d8�4���'�q�-i��J+����/~���sJs�$�W���F��^�Gq�GI[�����9�"s��������#�N*�������.��a'�.A#M
�k������_�i�2�Tz>'����I�k��A�[7a�l0bi�tE
�4.4�?7b���%�I%r?��IY��|�������yarn�|L��r���>O
��g�j�
;y�yRA�T�AZ��]�\��S��7����Y������Q�	Y��:N�<)`��P}A���+��gSG O
�����\�{9�u���|�!��f��J`�L9{a�����k�I=�������T�b�{���;�����������0���e�J/��g`hU����^��N���r/h���Y�Y��J`?�I���y�����������o��%���k�I�S_��P�<+���6[�dw�j�*���M�D�;��3���� ~������b]��Q���s��=�)��I�8, J1��'cR�u����\��N_�V����M�]�������B�{1:U8���Y����)yD��=�SU0�{�����I�\��{t�xfm�d.�K��t��S�xt���z-�N*�jR	m���N�<.���]HEQ�b6N����{���d8����e7�t�k/�U��i�P����S�*Y�J�PoS��(S�"T+*�yZ9�T�<&�+��{F-h��:4LN��N��	l�_UM�CJ�[�A3��������|���n^����%EC*%U���������c�DP��=�r�Etl��v��k�a��'��KI�-MJ���C^G�rL��G����s��^Ou�z
��{�������^�3�����������;�>���f��{%�{����E��I�8y��a����	���ySv��(��9R��O��N���p�Z�e�^6���y�^I��AU�bH���Q����2�!��3'X�Ui1���.�ag9�j\�eF9��sf>35Q�R�e|����H��������gt�w�P���wE��*�eFy.2�u�=��?L�HH^P�,�5S��+d��3�x�&�.���:fF�bR�/��Yc��uE�;Y��W�A�����DS�03��������V���qB���{Z��U��B9w�F`���P��d�#���T�q^��13�:���jP��� ��\[=�����FH������TfP3z��L��E�d/�����]A������`bPx	�Y�&O{�u�m���6�������������_�g�P�%�z,��%�z,�C���k.��,T��tQ�<�uIQU(���8��u�z���kHC=���������z�B�|�F��(����G���&���,��RAH8\�)���2�cL0oi��4�oO�
?�$��^O��(U�yQ1l<���r�V�r�j��+��P����5pK�C�bH��~�CQ����!���Pk��������J�L*���}1�9��\�l(��-J�a�^�r@���2�45�E�k�,�����<�����"<���<��B�P<���q���/��k]��P���/�%��
={-\dC/��� 5������P�sK�m����B�_��~�u�C���]U�l��0J4`AhMa���B�zfK�/������'gC�+]��B�IQ?V��m���VT������k5�(����**�gO���[���sr�����B/��^C�������O�9�Y���eX�(�	� l����,AD���~N����(�(D��/�u2������lCuV��,��f������k.��D���z���y���
x�3"��NW���^�F<����]��y(��x��j5�w�X�����
)�����`w>�1�%h�LO�� t�8>*�E���vZd�$���sWb�
nW����cF���b�,*�:'�����k��4��Cb�����I�h��C���[�&���]�"{����?H�B�!�5�&�U�/F�z�dP�R��L����6]P�6o3 �^m;��	e$�J��8��0������<c%d���
��Q��^i�(�c<��P��[������W��v
��FBi���DB��&4�����>�1��(���$�{g��P��!������<_&�����x�H�Q��KKt,Y9�(�z$o�@�Xn����yT�$�5B�J��/�u#�**1�c5T�5��nz,x�}��j��A���$���o�����1�T������d���<-Q��.��
g�g�)�M/���f���%yh{`���=3Q�nzQ�*�\����rPf+��V9����2�
\�2R�'{]���{��P�7}�o��������	�*2�f��M��;�.
�
�:�eH&	b�Eyt�����4�F�p>v�
��N�-��I`?JD�c7=
��e�fi��(:�G�~.���R��N=�^#r�)�*3d��1��C����;��&�e	!�b���(3���?"�E����-x9�H�[;��Ehq�2&� i��c1���m�v@D��l�["��(+�
��gK����)�^y�3����Px��g��\����qM8f�
"Z,/�gUi�k�iJ��c6���3�:�l|�\�s'u��t�x����2�gG�P��	�\�r���BE�)�{���dx(�|*��r��2vQ�t��������w��4PK��HSs���\�L�yP@5^�&uX�M����P@M��M���*W����Lz�
U�	���_����rh�����Z7�Mo*u�Z��i-�U�Ek��u8�:N"
R%-��F�x&jGD��]gU�DP;"��F���V��{�(�Q�����{�a1�dih[��N�����z
���[������\��\i��P�v&���HM�	y�L�vGD�������ME�|��@7U�prsS7�$��N�>��Z'�(
e�4��j#�n��(OW���YV�5������k�U���bH��~�U���Pl����6�J�UC����Ms�:����x,�%��q�:�G�p@gb�����9��
��
X(�jp�dX(7`��E�����L���-[(� �
�o5$�iZ���S*�g+4���N7�E�Q���,R2����;H�������y4����x�������B��Z;�%��d(�����A��r������t

�2� Z~Q�Yi������C=RP@
��T�d�L�.�N^�-e
E����xt�����~��U�f�����xH�"�����J71��mF���f��{�"�x�4
v�@*T��@�A���f��|�GRR�DP��,���f���	X���~���@�*(�:c�����B���d����z=���5d�����{���<y��3�Y)�''���������{|�"C��:�	�R������P1���1
�K�s���������rP��AYz9�di�����`Co���R��N}�^#���Fy�$f��:�$\���f��A���kmk����UMn��[��`�F���E"��UB��4��\\���@V�J�EOV*�~�F�W�d/�F����(�K<���n�j�b�!���Qy�T�ou���qN����"���eU�W(��C���<��\�Iy��zR�<j{6J'
�W�bc�^S�������LVq
�svA�0�~/�~�a����(���R�����J���\�
^���+r�3���Pu���PBT�2��jp[���c�b�D��Y��K\kF����I��Q&f�Ym���23,���f�����t*�O#b�t6*k'�FEx��m��p
$�����^�@�!�U�Y���	!g�v	��B
O�����F�;��hm����@�m�p6K�(I���������
��d�M2d����"x��d��AwB��
!�`5�8
Bw�(g��vE������UH��~H�,�(��
�4*)��*]2M���"�J�4�;��z
|�B�fTm'��'�G���5B�7�w���wB�;��O�8��E�$����?���#���S^�>�B?�	c	t(q�gr��Y�=9*���j`d��0����D���}�v�N�]K#�O�NB��A��&�/��#�	Y
�],���"�0��������t�%��i�zv���S�����M���D?���j��)o�P1\Sd��	�t��u?��O���x!\8�����BQ����i{�M��G?�F�.r*�M��4-��
����2�FH�k4�l���������LV�2����V6�MN�r�(dX4�m�������\P���������|���n\�J���S�X�������p
����  �,���uKNl�DP;:�%eJ}V����K?��I^�d�C^g����C�������u�z���k�?��#��"��r�t!+�^�r<S���y�!�1��v�}9^�����)
K��}���Y:?��w��.���@��i�N"��(u8�{G����,-E�@�.y�YX�5������k�Ae���:�6���d9�Jx�w^��Al���j��w=�2,P^���*q�#t��'���^���GJ��U|�@��t ��T|5:�2TpPQ���aFc��.�����+�T�|����9���1�k����-��5��Yqr�\�����5��Z��;I�b�q-
,!�%���=���g��g��xA��~'�~��������@=�J���kk;PrFb�X�2P��qd5Z&��Bf~��������r6fl��rb���M��u�49J�
��Zvj�&��4� �r����u��JA��08���49b����:���,�P�%�z�'~B��j����x��iv����KA��G!d|����Y����S�l�((I���������
���=B�5����*4���
u�����4�G��A�9���]���w��]E'/%;�����9��)F���%�^�RP����fS��--}==
"�,���^���z����Y���j��,��A_/�k�b��������Z�Y�w����nc��1���kB��� j�P���T1�*Z�zU�J[vu;y�Q��89��Qi@F%)2*��7����%?�@�#��V�A�iU���Z����"8]�xG^;�����Qt-���'�.�Q�K�2t6^��A+*�������D���S[�2����_�e�2lT�%��2?��^%��W��m�h��b����*��i����������%p��mg�^�h
(�F�f��W96Yt/*a�����6~��~m��PQ�E�����L�"���4"N�g���AL�,��6����P���K��9-���CA�1/��d�^���o���z/��z<N]y��hw��6q����&�FI�^��lTz<BMo B=O��b����Ws>��t4�s'��o�������^f�o������,GKG�-�D�o�Gcn��L���?�F�cQ������c���)&�.�%���k�F�S_���3i03�S_,�Y����?-��b�;{��lO(|�l��t�����r�p#6�V��F�k/�<�7�J���\����N+g�C��_
���tTj�cA�jgC;����f����2����v�M#?��<C��2����|������P�b��}��W�&����T�%��=�����������NS�k������M"�b��z�PR�{Z��%��OI�t�$|���UB`�^eP���i��+*�4p�������	,WT��������h�()F��~�.�.��l�5"���*iI���v�
~�H��]�Y�&D(�|�_�����A4(�h��p�o���g�f��[���6u�>�:�[�E����Y���z/%�w�T�y]	����d���NW���^���T~<J�o J�GEF���<��7�c�s���k�����/�}y},
�%H]n{<@kB��������F�6jk�z��t�yfy��YT�5������k���6��UCb�����I���,�8��;�����K����f/~*�GQ�'C�'[�uPM��{��(Foy:33M!qQVu���E���]Ty52.2TpPE���q6���8[��|��\�Q�f��\i�56F2�,Q�Gx4�9������D��	34��	^���<��,�E��=�F��T�[W����(�D%������������� �A�_��*�#�.-*i��Q�RBv��\�l�E�����Z)A����]P�x1��n_��APV��2e��ZU���^�3������5�VY�-��
;��r��7k���H�^9
��/�A����fa��u2��9�@����f��9]`�T�DP��.�42��5�^�l:QH��y���o"�y4��=ZO��z=���5$����iy��iZT1B$:��J**D3wr����
������3yI�.����@��9u����PMM�)��|��DT9�DAB�li9I���c9\����R��N}�^������%1Kvw�������Y�w����z�i���j����@O8��SR�P`Km�g _�PRn�"��Q=	�d���2R9^���;y�����W�d�u>#��	#�7)Q����UsH*OMU0�/��g]�H��J�'�"/�
��|�9��SH��(��H�3��R���Z��hJ�����Q��(��M�<M���S��t^
X��������ZF2�/�~a��o~6���"z����l�����L�`�$n�C\&�v���E/HGlA�q����(y[���
1~h���G�TJ*�3 wt����:�6k��b.C��`���f���3��|#=���DG� ����������p
hF��*\��X"��fa%n��X"��f���qa����IGm[���(��:c�\�Y":s��o������W������������z
�^��t
�r[�/�z���a�]`�Si��7)5����.��C�����d��aC\`;���E+{��L�~��:�����V�������l�P��d�D��z��Q����5�L��I�%1Kvw��'���^,Y�w�y)���6K�:_�'�[3����������d�j��7��Q�^�2�t��Z���o�f.e|�y5R62|��QF�J���d����U�H�}��Uz�1�*�����.���T�1^�
D�mK1�<�X��x�V��TnkK2.<)C*��NzY�����c����2���Q��V�t����{���d8)�|N*�XP^���e
%7�^��1�g��2���������r��B[�����na_2�M�D�P{sz����n
�
��]�s>��(RG=`��	6�u=�jV��U���9���|���n^���r�)�%=
�f��t",K��0*["��O9����j����N}��>k��z/%e6J�k�����h�u�H_�Q!�MJJ���5����a�ya�y���2�1��yQ���v���a��4���P��Uq����Y�E�R����h���.���}��������~��:�])E�o�vi�;��*��"K���;��z�:�E$�*�3d��u�#��K'9'oy'�]'=��yy~7y�?�?��g��W5��J������(
MTY�(��p8S��f@��&�)]�b�\�r��s"��["��(+CD�8;.-��	��ZN�+�*\����W��^=��FD�ku<�N�x(Sa<�8�;�*��D�����,*��8%h_<���G"J�l��t���P.�F�!	���zM�JS�{����d�(�t"J{�������I�����&	��~z5r�TP�M,�<�M,���x�F�������E{�&w���E����V������nnC=_�Q������p�E��<J���vy����������$&*{&
���Y'�9����6K��D+"��Y"�G&���^GX�E���Z�Po��y�1�����Q��b������W�!e�O��O�z�X��V�Y�L�|,s'��k��r�y�#7Q���k������6���D:A��I�<B�
z]%���Og��19*���di�k�P%�
�Y"KK�\�;��z�8b4A7i�`�Vl�Z=���b����/.h4`�����5��;��`��?��l�'z��J��.��u$
��"�p��{gs1����
8)�j��d8)7���TrK��qF�U%k�e������k*���\�c������0����Ti_Gb�i�D{��t�+�G�T���i{���g!{����\sR������IQ�Fe��Z��O������N�=��24��9)4�����Wi�4���x-(w��PlH��iZJV�9J������)�?��5MvTG����c��0v�@M���.?z��[6F�)�*����P��������V�C'cY�w�0�Z��Y�����4�0G�O�����7yZ�$()7t����T��!������`8�K�3r�ZO�*a
�l����'�z�0g,�m����RR��0G�z�eu�s���;�K�h�()I��mJI��n��n�8_/I��i��asr��Y�|��{y����M��A���uPV���c���bwn.c���R�]���v����Q�X4����G%�$�kGda�mBH�QK�meM���r��G5��r�������J�#��#��Gl
���s�Yp4?k�9#���yoaFD�D����	t��k#����D��Q�G���Q~BDy!"
���6�^�V�&��LoeBD�k���/�fw�\c�Zz�s�V<��m�W6�*�O�(O�������d���^7�(?�1�j�3:�2�1/�����������[�����u?�Q��,�R����nP���1@5!��5���+G���%%Bd��������0�N����]�q�z���LRN��G�n�b[BN�����Q��/�����*���1;"�G�p9��;n�����D�:��?�����M�$n��p�'����Z�ZO�JD�
D�ZO�JD���F���v��D�?b�44]nY]�L�;X������WM��)��{��x�c*?R�m������'[W�;�US�}uu��G�F��Yl�'���Kom��q#����j�a���zy�x���.��aQ�v�U���,�*���:"�Jm��Zym�D�(�l�r��&rgq��:��W���V��:��W�����^p��pP���-*��k[��n�	g�����L�Si���������;L8���gd8�0���P����:�b�;g��
-�k��r�hr���p�r�&	z������Z�W����=�����U�Z��[T�� ��+E�q$O�_[����F�$���@����������*�����+E9(�$�E��n�B��\��F*����Qq,{������)�3�C���$	esu�it!��M�,V/&yB����
��4+���(���wyV�x3�^oECIHEik����V{��H�#,����A��+��&3�:RQ���5���[*�P�����6�@�.o����`�$m��)������*i�T4�!��!(���dr���bzN!M�Y��x9a��P����`T�H\��%��i���2U	�-%��2Q��G�5��ePLO��8���D���&\����f�����K����,9=Ag��k�'���'���p�so=�'�"����I���'�pR&�+'����E�8��EO����1hJG�������^�pR�GO9�pRq�IE��(g�
�8[r�g��]��\��������P����I\�X�xQ�V�+�D��T5�
J�5���G��
N*vqQ�A�qO�+�M�8(�[�\"TM�'�2XN�{K�~������T(a}�;q��p+��>L��$�N�]��T5��@�4SR���X G}Y��%(�=���u
N�}����A����	���U�9)&W�zm�6�Nv��(������?�^$C/���>'�@`R�,(�����(�,.B�z"���(������'�z
��>T���[��-'�����M���k�{���L�Le�X]m��^mSN*��S�/�S��Q�A�rm�=:��Z��L�R�N�v���a��H}��J/6�v/_N�fN���=�e�e�p�J�X���a����5C��,-�|y$_}S�Ffa�m�H�QK�m��Ed����9�_��
���v����Q�U���#���������:��gLT�4.PG���e�<O(�_)����A�p�K�D�8C/M�����d��4a���T�3�!�8������a��R'ExU���kv�������LR(S��_jQ��^z^'�y���W�i�Z�:?�L�d��@�<��+
6�0-`Sv�~y�����(�U��m�D}oi�2LT������l��r��`���l��e���~	��v��\���:��k�^�����F���cb�����m9��f�CXMD9l��so��C�3"�@������GQ?+R��{�p�z��
V��*t\��o�ko0�����M�Lo=�+U8��l�����z"��8�h��E���JD�enY�AM����TN�����U��mJD���i�������K��JH�� !�9�vr��;"�fb]D����(��Q���,��"����(H��������E��.��D����������QHfA�y��,-�M��?j���,�NG�uc���,9=AO��X��=�'���&���;��?Ag�����z��`�^��W�rZ����m	n�G�����c�VR�B�@],������?:�Y���J*�PR<��)"gG��F������U�F��Gl�JQ�$��W��./)��#�<��c���Wb{?�v�h�35��T�����������1@�Gj�e���B�J�{K�~������Q�*J��L���b��`�i����;�P��=�"��P4��p�h��	�	{��*"b}���t�sb�TE�z9�
������s>���PR��X�4��3k��������DI���|z��^��{�p�z��*�)))\���[o0��DX�^S�'����I�'�z"��=��fz��E����%E���[Vg\�/�,4	�[�eVW���W�����������jT�*��8{��(������jr���./�-�
m��MY������.[�!�'�Q�A����n�N�#�x%�]J*_))r�>T��������Yd��,-�M(�?j���,��eb/q�I���t�	\�?����J���h[O�	:[5X������E:�9+���!��GI�@I!t�����q��������*?:�E��*J�QRg�Kg�Cm�z���z�w�����:���3L()���z���rc�q��]))O�[��id���*}A���^���>=�Ot�A
2�b*�U�����tc����d(��~���y��
���)"X0�6����5��SM�D�`���::�
���`�	�\���:�v.4������]�=�g��@�<z�t����E�g��� O����q������T���?H���T��6���z{BlT�P�'�z��b�����'�z�'���v�zkQ����AI�chw����3t9]
��}0JJ��j�RR���Z~��:���V��2�k�v�9:�.6����5�6E���sN����������-��KH��"��e:�x��.%U�b��|�H�XZ�(���o�kOdi�mBI�Q[�m�������>�YW����Q��V�j=�'���G��z�O�EI����V�=�Y���M��r�M���nZi>�}!��C�N�-H*=�V���Q�)��?����zF���F�R���a�8x�0��)��(�V��\�_�����a�Y�*�2����5�i�����b8c����d���������=D���P����=���Q{��N&��[���OV�� �F��U6
�^ey�^�����B`MA��V��:*jC�2�!%��T��,"����P�Z'�u���e�Sz�N,�~��Pl.��������B5,2�4�Q[o���u�G��KlT���A4�o�i>��96�B��E�[O��RIoh]{WTZ�@�
�f�[o-��l���zR�����
8xd�������z2J��j��Q����O~
��D!�(�M@���
\���}�����L��h�Pq��]��LgH6
	I�A�xQ���Q�l!�M"�z6��Nv��sU�t��cB��:"�Jmc��Zym��X��
j�!'��a�L��Q�#��Nzm�x���_M�^}���<5���U�QX����MAFT��S:���b���z�L�'��cld�'3����
�}��0��Z���z����~�Wu�~U\��sa�H��~�I�~��������\��O�6gS;�'�#���d:�)g��o��*���y_��'����W�:��������A�~2��O���w�-��]b�^n!��9���'Fu�)W�qmu�s��O����
g,��;��������	���gj�q�e_�w-��g��zz����j��U���S�H	{�[o�����������}�)G�)��J%������"}u��^[O�jO�l7��'�z�'`�Qm��#ie	����9����r�eu���pm���$���j{��j��O��cj~�c:������f�nT0R��Y;��u�x����9�rT�/��4�u$�q�����+���d$G������C���+=� s������R�����v^��*5�d���9�_Mu�a�[G�J@��+��lq�+E���������D9(����rT����,%a����Rc��Sx��m�v�@�}c+�@�	e�R��zp�1�s��o���������jBh#X��3Z2q	�5%�Wbix{�!rU��MR�����g�*K���n��,��S�h��W]\��z�����b���2�WdX��-��A�������.�<��E.>�%���Z��*�aYF����a�M��y���Z���6��Ij�J���B�wEa�"��ko<��M5P��V�*���}���1w,T
�����=�>G	���^:���J���������7N�B��2��'���7��j=���t\�w��Z�oY({�Pc�gu���f�7y��b�������6���}����tLC�r���l��mQ!���'�a=����y����U��gv�v/��t,(����E��.e��PI����ci�+�YI)�uD��&D����f����o�r��B-�Y�~��G��,�[��[G�V��2����/��g�x(�6�r,�-*��BML�(�XM=s����B���D��Q�G���QnBD9!"
��z�<���q���o"
x�e�
MJk"
x���G��	
�����L�(������l�by$�~��r=eh�k�hC�s6�l<������QA�JQ�[Z����'�B\.W��2��B;���b�x�@����N������_�l<�r)�}��E���]�l<�:d��w����ZVQ�
�*W��}�����@�����6����3��{�p�z+JKhCa�m���`8�������DP;m(`Um��A��� \���ui4a_ ��G=�R�eub(1�6-
�(y���j{��j�Q��{�~�{:��2���Pm3"*�����p^�Q�����p�_��
zFi~�T�rq�v���(zo�J*H�y��D��*�iU�`i���LBX;WCJG������	�Gm���eb0k�����%�'�����O��1����tVJ�)��	�R��O��*����=Aq��L(�h�B���t�Pm5�jk�8�/���5A#�`��r�b���O()����e()?���%U�9�6��-r1�gqlz�g��x�/7'��Kb���B���F=���(���U�m*��R�/U��~ W���cA��Q�p���T��J����JQ����J��NIYj�]h���5W�A7�%[��L���zu�1��K[��o�����I�'�����m��Q���\���y�5g�DB���L|�w��\�*"�x�m�������>������co
"Q~���)�  W^�i�7����X&�z"���v��.�N���^�����qT���[Jj�A=|R�eu5�-�|����-���^����������_��c��_���Y@&
�P��g����������P������ ���
��<O��)�	��+w8��y���-�}����="S�R"�KKOI��gf���D���&�����f�S�6�z�f7x���"\���z�O�@�������N�'�l����\��{�J�GZ��������X��))�u�^��-���	Jjm�&�T��Q2�T�PRA�����\�9��Rz�k
+$�;J
�s�U<r��?��&*���8]�W<H84���(}nFI!�au<KL6��(�p��h[��j���&�)lQ��8�)�}����=O�m�H}oh�2�Tx�����V�L3�3��O����d���e�UT�2�c�E���������`���}~�Ew��D����@b��}�}xy��l�����~s���C#]+�z�&_������-�A���h"$/U�
�s=%��/�z"�WJ��P���DP�������[��-%J��Pt��������_
����N��������T����_��)�\����JP-�"�v��EIY�����"��9���
R%-^�N�}�-������K`����U7�V������������{�����	!�G-��I���t��j?nA���5]�`MQ��8�5]�m�k�^����:��g�Qd9�u���A5o-I8!�9����\C���J�c�����?��Q���"*�Q4�q���q*�i��?�r��4�(qMS;F�L.mld	��r�G���K�J�F����q�������oQ����aV�����L+V:�o�*>�q���������u?�0Q�?���ayl�����2����v[�<��e>�v�Q@�Ux,���6�BDz1���k {Up�)!�0��t� �f2kd�Z6
z,�c�4�]M������:���!9�z�&�V"�������(�(�������r{b������c]�j�95��dP��i������~KD�C7�N�oY�&C�ZyC����VW���W�����������n�"�JA�3��"����}��x(�F`<a��vu�����@��7����E���Pyz�4�]�`�v����d�l>��?����(m����=J�-]Vj��P��k�Xe��#,j�H�����"�����#�����k���YN��jBY��;��'4T0^Aj![���Z������
����_��4f���4T��P�G�8��PiBC%!����,��`����6�Z"�����"�\�-�"%���\v<�"ac�y�vU���3����>�X����RGCi_T9������Q�����H�����^�ve�?YZ��
�^����y��]i(������q�)�[[�XJ���PQ�$�F���R�**��i�������br�u���$}� �!��uU���#3������m�S����������4$h�4t���4T���A�=�j��p��'`��#��G\oO<aP���DP���o�[��[��-
��x(O���eu	A����*�P����Wm��)
�������C�(:�J�CE:�!������.���}��x(Z����+�4���x(P@���XU������Q����{0q�����Q�A�4XZ�=���,�S{"KKm*���zm3�����
�5"�yz������q{El��}��PI���F���j���c`��,E�Y:���+�"��oO���V����L��c^�c�'�T��O�2�T�0RY�����[�|������d��o���l<l4�����i��z������|�]qE:�5�����p>7�(��#���F��f����=�������^ ���?�Lb�����6*��a�}Y�=��LJs���s+�^
c�����9['P����
�,�bz�j��uQ��6�K*���O��J���G���-_�N��<��m��so�7y&]�rP����+��2�q��7�����DP��t��1f��@P�MU;��DP��(�����DP��(eC���UV�~�F�����'w��h{�]�B�?L����WM��)��;��8��ZzBzQ��By
\V�J�h-���r*L��r.��]L��zz�d��'�k�E�^e`h�����D�+�h��J�]��vDV��&<�5��&F	1jD�m�r���D��4��q��u��5������(��-���-��DBC�Y�E����&O���i�_B������"1O�����*?z�E�*����O<��>f}d�']�����|���U����^��t��\��=�(�Yy1d'��W
�,�m�v2><.��%�T�[�+.��e�&g�U��8D�_��H^G;$���|#B>}ob�2�Sy�|��jg��#���	n�6�InaD�� 9|"s��!�d��*Az�N���,-�@uN�G��$6:N�,�b=-i����
�p��~F-��]�A��k�#���(����9��^"�bZO8m�X��,��\���X�j=�~�����zy���fW��[����%���qy�e���5uXlu��j{�M�r�-���?����M��EUp�o�����k'�	�/�G�����A��O��6�����w�������
�/���%�J'�>��},-��y��Z��'���6������6-�G��m�,2COO���������,�}�<$��6����E�r�����O0��"J��6����3�%�]*������6!/�1�D����]O?�v@?�C��O4�����X�;A�B����Va�+����U��Ox��6��V�hC|�/�����)��?lx�~"3��O((�����w>1:���~�>��K��d�����db�"���_��)-��P	���!����%%h�Hj$Or�S�I�bQU�yG{J3�D��ER� �!�6���Q4���D�����^!���6��1�4��
��a����
=>��Q���p�z�~����h������������ZO��'u-��z"��v�k�{��E��~:�L�4b����::�z�z��6y����VW���W��~:@�vI/�k��HO��
�s|h^ ����^��xr�����l0����bs>a�y=��P�`l�z;a�=���?3Wb����'r�*3ZZ��'�&b�T��,,�����j�����B���:3���y*0����,b'�."s)�I%�2����D(����>�$�+�$��p�1�N^�]��[��j�>�4�K������r�k��%3!�����!���t22��A�@��L0�vv��"'8�
+���#��
��U8���$��.7i�J�m�Y��
iW���B�n�N�'�<���dL�kn�Lo���������sK������dH'�~����c��;"q���U)�6]Z(�}���o�C�P��S(bB5�#�G�]h��e��Tj0�����i���9�g���6*�&���9[3���������������z��?`��@�"�b��`8�������[O��(���En=�n���'�M�X�}��Q���M��(5P�9�>��$i{�MY's�5���=����y�4������'�	��|�����)R���X��P�p��5���|(�������J�������;�>��KK���AO�'���6������6�NI�\� 2COO����RZO�	�J��9��#O�	�B���gk��O0��C������!6�'�C3�)�,���%���^X��D���?:�V����
�O�nB��9�aA�J=1b����[�5r�4�+�x
�� ��=�g>��<�Q��Y���p��g��7�';�����C�m����7�
��`�G�hV���db�2��}�
����,��������]wI{'�s���lg��}4��K�t���V�����qi]�_�t��
G#�
?�U������P
���p�m���LU������k���������9��^�����x�m�y�|�z�	XB�'�:�(=W��z"��~�7D�L��E���G��R����V���e�'�Y?+$.���^����������_��N�'!�'��t!3�^:�O�=����������{�	��D# ���O�<���
A�.�d�Iw!|(=},-��S'�E�H�X���6������6��h���
k��7����S���'�=��'��@�JOO��O�^�� �c0+�p�U5���s{�|
s�c::�q�I.��w�)>S?�vr7�����������rb�Ot��@������������'��1mp{��L/���^�Tjm��4�����@��'z;�����?m���x�(������'�rB;�dP�s���X�5��>����|�ab�2����S^�?a&��Tr{�5��|��5�j�v�k�M��L����>q�4r��g�_��>E�!B�[�����=�����j�m�9{=�*��H��t�����#�����m�7��+����{#��b��P����~�z�����o�'w�>���=����������;Iw��W��~r�]R�\�q��}���6K��(~�i��JA$�g����V�����io���\Tf.6������]6���C���<}�K=]N��Y��8��U�O�&NK���	w��k���'�������z1��`��:�'��m��9�4�~'2N��l�e��4Sy�hLf���:m{S�KvS��Dk"��v����
�u{��N�Go��N~B8y!�	�l�1���Go����qp���C:>9oT�<B�Ll���`����4�Y(>��{����{���%����'\��O������8��":��*O�U�p����d'�>����<�����U7��6�f���qd��=T����xr��q\=�����-�k�y.�	_�o���9}w�����U�r���zM��A;���q����~�d�H��:��?�w�|�^�o��p�����N��j�O:\{��d���.i�����,~�o�&�<!B����M�Woi��8F/���^����������_��	�\�D����������k'�	�1�*X���� vIa��7kP�9b���u�������]��_U���2\Z:�)�� x��,,�MH�?j��MH'K�$V[Gb�����`���u����S��2�?N���W�	s�����s�I���R��bI*�Yv���.[�lM�Hv'��YK:�	�~����&�S�!�x�3�v�[bX������.8���r'���S���v(:��S2yE���&��&�]�oI���d��1�s�����i��{�]�k�N��X�����wYd�q�N��B�V���b%���QPJ(���4���<Av��A�GI��.%�P��r
���������������9\k�=&Z�c��W�Z!��o�.��Ro����%�)������E�
�s�1�F�Z�����I����'�:�.9�����~K:�#��|�lnY�������
DeM\lu��j{�MI�p�
���H�?���m�dg��0/t�C�];�O�}����w@�C�f=v�dgj96�#2g� v�dgg�c��b-�]�)\��"K� ��*T�Q�'���6������6��i0�k�ZD���}��U.��V-O�S���C{�,b��'���,�0�W��`�dGgbV������'��'�"n�X��$�?����b��8�����q�����~�B��9"������s
KT���k���1�iIvI�%���c5�e�C)���&����p���9kJKM4�/�O���2W��c�4T����|��"7�����BIv��X������vOdNg��"��Wzi�2���>�@e������rt�����<a�~��J������>Av��"�`�N����)��"n�;�/Q�����L�7����G+�
�����3]��~�8�;�[9vZ�~� @Zo0������vT'��������z"��v�k!��[��-����D�[Vgt)����iU��K�O��W��~��]��\�1���4���R�0�������%���}uo
J�X������u��[zgo��s&�����KK��-Be��v����xM�sN_�ci�����f)jOdi�mB?�Q[�mF?a0}j��Ef��	:�	�N�'p"6rz��J�k�10�c��*�}�����q�r�Tk���Z��vs�':�+�Dy0��5�����n'O�)��'�)M��$�r�O���	1�]�%:#�r8�9�_5�=2�i�S��=Cp(��joY�,�h������m�Lr�y��/����O!���������r�G�~��.���'�~���������A��<�	��X�v��u�L��r�9��2�X[����d/1�Y90C�A����\'�>��+�Mo�T��/�?"���;��X�O[�E��l������������g����qw���'��'D�"�������OgB��&����$�7�w�Q�]������[o-�����e��p������,���f4r�����6e��}�4��t�q��2�
�����>�n,��'�kRA[N�u(�Q� �5���rV	��9�Bw�1F!�����]}�y��F�]>�sU���X�����#���6������6�I�e��BuW`�����I�Q�����}��N�Bvn���_M����a��O�������o�����2��,-��=^j�f�s�^�7�	��t����'|S��lHU���zw-B��%�h��4�2W}s��h�n���"��z��$�Fo��4m�d�3�{r�����]g�q���?.y�%��;���Y~�)�� 
g�7e�������]�������u?��M�u�I{-"���#\�v����ILS��\=G������1e��K��	�b��+�����,�
i������y���`�kuw���Y��f��
P�`���;���H�]z��}�ID�	���������p�c���'��o����RO��/���7��QXg	�����g���(8�7�k\���I*���j{��j�2N��/��/:I��R�vE��Yh���jr���x
$��(gLQIGA�>�B�h_�H�^���PxE�&)A�q��R�]��r$�Or�^|,-}������g�'���6������6�L��fyX�'1KNO�k�y�1���Ad�����R�z�����A�����}�Q3����;�aR�,$O���&�%C��cG�
q�w*a��x��O�G����Oe�?�x'���
sy��8�t;� `�j�%����~ExU����

�-�-��?+�>��%8��R�>������v��wBl��?�W���(k��{�Sq+%�����d�����x	ay�P�n�������T\b�N��������3n�5K�Lg���s�
�h��d��5�u������P�Z�q�����.I�N���v�����\�?��\����	x�	���D��1��>�ZV����jePG���o�zkQ�����1��r������Z��5��;�O��W��~*�]��\�1����r��F.�$f�dj����
�=3��r�� tR�kH	�Z�#�J���H*���I��x��.�T:yq
V�+��y���#���6!�����6���b���:3��o���OX�G�����>"���8��?G���s�Y�S���P9z���Q�`�����=���Ai��H���b��������a�u=������)�8�8���v4��g6�!y�Lhllt�z1�w
:[q�aI�9;�K����Sm;�� �dW�P���!����A�
��u�1���-�m2S���A�-Y���G=ee��n�Y�������E���b��{:��;��^B	y]�[�3�N� k���A�I.�)"*�'�8�B|�+�Y����Fe��>�?Z�"{xD���<*rr���
Z����������R���1��e9j�y��Voe����SD��
�r3�X�l=�n�$���f!�n��B�����~I:[���Qr���X��i�P�����[du��j{��X��k��'��u26��&[�e������7��u/�+��a[e?Ru�c_7e�7$��{�#P��;[V�
#�����M��z"�&|�/�XZ��',G\o��D������_���f�kLz�z�����"�K���\��q}����j�<������`V��$k�r����wz'}����23��yZ�(������Nn&���762��PPFHc�\X���t��F�0Ag�_W�Um���^�w��,���� �`�����6�.�#���������'�S�.�dl��<�����8���tW��<�����&�� C?���'o�'�9H;��������s|��� ��'��!X���ji@�B��.����v.������q�(�dc��-(����}�Zj��������9{dOD�SpU����3nQy:��������" 2�A4����`8��Q�*;jA�7J.0���F����EP������[��5�h�S�eu�D�����B�*���j{��j��O��Oj~�O:����^�2����iO�9�K&�	��o	�"��E�����v����e���K�,���O����4j	�������3Z	�KK�?�Rs'[Odi�m�?�Q[�mf�4�	���'1KNO��'p������tVJ���'���7��y�o���t���@s����k#���k��`h��O�BH��=�.?;�vr;������������B�3}�:��E��h�*���gq)Dp�h�q0e����X
8���#�+����i��
���
�����l�?E���w.h;%}Q��������9� �?}ob�2��}?�.���u��g���h�m<�O���	�\!Mb��f�����t�|�hD$;�wi+�n�y��d���{����
��9w��p;�j����g�3�56��z������������$Q����z��\O?A8�3���DP�}oc{�EP��X�M�F����O�V�{Vg���#:�j��h��O��W��~��]R�\���x��|B���h^�^\�b�:a�5�</��t�����c�n_a��z)9a�.mR4��-�*N�.�d��w���KK�����eqjOdi�mB?�Q[�m��D�������,9=A�����5�jO�	:�'�y�� ��A�������	�=22�~B,��vh������D|4�\y�L~�e1��&����7v2����ON�~�a�(�/U�oty�*��I���0���9���#�f��V8�p�4���r���Jg�����%k�9�7�'��O9��F�^���*����������,�������u?��O���|ZN?�;���4��&=�xO���!���It�l	:�,�3D��� :�������)�1�Z����O1{Y=�i��~r��W��;�6�������E	k�����$$���p�z+�.���x#����z���RC��j�O���E_��j�]�����a��~K?�[�G�]���au��$W�vDHdJ����Wm��)��������c��9!�	iH�-`���1��l�"�I�]!��9�������)�	bw.-a��cNO���]��r"�tVP;)������7����'���6������6�L�i^fV�I���t�I����}X�����R�5�����U����`�o.R�"�+�~����K�y�����a���1?{��N?y�8��O�'��o�e�'?�����x�Y!W�sjG�X��(y�p�k�fK��U���"�������Z�n�V�_%T�������H�V����'(5B�������Z���Pn<�'+����u?��O�}�)�������vh>F�B�+�E�Z�B�@#f�-������������	=���P�����a�)�&T4Mo*�q����+������2����#j���m�}C�2�w~����� Q��Qu�������eX�~���^-�����������i��E��~�G�S�����UI���S:���]m��^mS����H�/�H����R/t�.X�}��yZ;�O�W�qD�"��
�~[�	��8^h_���GT��Uq��������'�����<����N��Oe�T��y��];"�Jm���Zym����H��:3��5Ld��\��#��N�v�&�GX���D����O+�3�'��C�jO�c.-���g0s}���v�r�A��O�\�H�E|S��M�Gg8��Ma�7!�	�\�:�0��C����F���L������2B���M<���KpV�v��Bz�A39@��4�`�#�����
�)�����v:L�N+����F���N������db�2|Sx�o��=�[�V�e?��S2E�N��
+UiM���,#_H�K&H�r
���,��R.2�SO-4����8�@��������@<���������e�E�����
�A����8b�b�z��\�7���A��IrM�:����z���� �zkQ����!��J������ou%�P�9����^�����S����_��N�����n���w���}���������6�\/"'�^���9�>�Q f������r�'�D�m%���N�*��Pgi�������Zk=����	��Gm��i,TM��xc<A/�d�Pi��?A�����=���7������V�n��O0��l�*W���Q���O�\��O��1��]�`�������.����)��G�)N��(D?a��#�a6������S�c�o�dGi��Y���X
�;#�����{�D�RbO�v:�9�� �;��Y��
�)�b�!)����P���y��{P5ZbO����'�~�����b�&X��31cM��c���S^�����:�#��<c�O��� ��2���,g��L���~"{�GbM�����uW�m�z�����O�TL���jo�OB�M�)]��d��	��!��z��\O?�����j�O��i�a4u��O�*�o�������O���,����:U�x��������j{��j��O��K�K:���**�sr�8M�o�����N�v��z8\\����^��[`�Z���l%��si1�(���%�l�u�)^c�R��m���>��6�f�����R��~���^�,��(~�z���]�>��������	�l�Q�66��A�m���m�]0��'g����<���l�����x�\l��
�4Ie�2�r�i-��&�S��9N2�S��OI��q���`��eQ��]��q�����I�*q�
��
��\�G:,	�r<�J�R���x��:*���H�#��g�����r��{3�~�����)w��-�������nZ�d
���$��H��i�H����$�p��S�]1������d�AE���$��/LN�j�'���A�^��=��o8
���'�r��~p��H�������{#��bA����H�`����H��7D�q�z�M�]:8�H��=�+Z,`��E����	PP��W���J������1e����U�9!���p��{�2{ ��+����&�|6K���E��QtVau��	z��c������m��r��Rw��������_��A����?j��Mlc�b�[���;���}T����u"�h��A���;?0U������P>���������S�<��{w�3��;����p�n���)��g�)O8�,���8��SB�K���W�4�!@�����V�]�r�����l�-��=+�5����CB!����3�����|S�M���
����iU�T�l{�-
�b}��>�yk�M��X�����O�Kyy��#h�X.-��hm������ Q/������Bq+�����z0��X���j�}&Nw�t$��X:��=����K��%�r5�b�5Y�d��]�����V����&�������������� h����>�	X�4TodP�}���>u�p���y�q������~�7�[�G��EoX�AZ�����"�G�����U��mJ8���h����p�S��B1O !9���q�!���'�>��S�������N�}4b�9��a-���Q���9�,�S�B���S��sE�l�����<���P{"KKm����zm3�����l����,9=Ag��k���`wK���R�V�����VK<������'�����R�s�d����b��L:�i�]�)���OeB?�}�"C?�	�T��'s�<��NO��{�EBap�)g�Q�"��_!�}=V.:�&QS��18�x��v������O�<=�A?�>��m�3.�����vhq)w9)�ob�2�Sy?�������|��f�h�8
@r�I�e[�)w%A����h{i�W��.�������VvQs���
w��k���~E��OH��*����'%��~��p�z+�I�������7���c�>��A��I�09��s�-E��K_�����[��|�;!v����r/��`�
��$i{�M��r�%-��%�;��om��;�U�I�s����|��e�����)�Sd�z�.���8�%���|��X��������.l�,�}�~*��'k>�>���~�(��A��,-�M��?j��MKOz�N�z���}J����������	�2���B������/F�7��,x�	����U��BEs�d�O�����T��p
�W�]�T��~���o������?;���!�'��`V|B�����H%I�O���IpN��v%��
}�k�h~;��lcT��yfdc��9�Tvu�b��F?a'|�~*��~���� K���FG��.eV[����e��n�X������W�']�I��'��L��*O��k����'����3"���O�>�r;Av��FMxB���7c+�)����B����b���%�����5��z���d[&��E��'�C�^���=����;�p�z������xbZo0����b���'��o���i����v�%������K���3QrY�|��hG��4%-�V�#^m�������6����}�����)i���3�]0dh!�#�a��>aw��$�t!)����*���O�����m���'�?!��B�]�1��?]����|j/\�������),L]{"KKmc����zm�Z���a�lo;�'�+��4�}X#b��'�������kO�	:[EB;�W���	&��IA����Q��
"x�d#��#�w#�K|���9������O�G����Of�?!�	���0���
{��3�����bq9�
cr��=�I����vY�&kFo�V�]��4���sw���g�g��n�O���P���\F�gF�TB
�CYJ��?e�2{��L��A�2�g���<����X$�a�����������B��S�C��3�z/�QCoi�-5�Y���Z1�'zKM��hm�$�xxg4Pw�)�so�rqV�������f�J;���~2C���OI�~� Z����`8��O��Z8�'����Vd���j�]B��MX7��/�O���h��eu�\��
:*{������Wm��)�d��������':Je�irz+k��y�Q�a��>aw�������~:��]`�3�k6�Bvdb�w�2�7=�[���v�~2W�qo������|��Jm����	��G-���������!'���`��mq���������J>ln��W���t2������'�$�
�
��;�C���{��O���;!�����!���t�R9w��R��)�i�Z�D�)��J�R
������8���9�\�~��*���&��fE�_�D@og9�Z!��?$o��*�_�N�#���1;����?���9H����N�w"���&�� C:��I'��K>!5��)Zx��K[�����0��;��W�E����3u�d�tg9��
���+� �1�9�*`k�O��g�� �)����9k�m���i���P94��,����h	��=_�>�$R��h�z�;G��)�t��d���Q��X65T;�:Iw<�y���[���_���:���LW\4����5FW���W��t��]Q�\�q�]*s��l��#�6*�����/[��G���,r�������b����%�}�f1�����O����w	'{%�t>.�?W��p
��Jm�E��	��G�����T6�u$f�	�����h����*,���#����ao�:'�"��D{r�"O�����\���*�Cb���{���R~���M'��7�d'7!�����l��� ����&��iE�5p��&� ������::�I���L�Y��I���4����k�H���Sq��=P:j����ew��{]vy���ab�2��{�p�:J�<Y��1+��q��ANY����Q�����"�%������Q�32�9��"S\E�F���QN`��SN*T����9h'��3)�����kd��|�:��}�)	�<9�_k���`8�2��
��>��xA�0�Q[O��.+��a-��|�;D��1\�eu%g��(�U
-�e�������6���}O��Ot,1��j�e$�a�(ns����'��[K��m��n�-��;��Va+�F���%�}uf�����Q�7/^Kj��7������U���|A1br�kGdQ�m�7�Q+�mb���S���r��&K�>��]����I�\Nl���_M���:��'5�@2��e_,�zV$hO�&��EX!6Ist��&?�����������o�R�Nt�����������:��/Lp��7�32N���\R��i�DT���!��+�v���.?�-��7�>�)Y���������t�geh:D�fq9���,���7�����/*N���� :u��[�W�E���\�wL.����m)n1� ���t��"LN�D1Qq��%s�6��cf�y�5{�o�%(���j�i@X�i3Q'��8��������|�x��T/�M^�o�A�P�m��p
�:i(�5�=1{1j�O�f�o�w�+Q��Xa���]����-��EA�����&���WZ�Sk���Wm��)����������� �U�yQpc���!Z�fr��;������-��;�`[N���3K�w'l_������.�����
~����N��GzF��,-�Mh�?j��M��*�`XkOb������A���������c
�PO�	:[������.X�&�N>9�E���D�Zqb�P�z�_g�W���1������g)��N&�S��72�S��OA(���/\=)����%�G�${p&�
��m�9���Cfg@�X1b3dnK$����4���d�6��p��ovH�j�1	�m�p'���0Y(���AE.���7���)�N?9oD��e�Q;��b s ������kz`)����>A4��tsJ����LG�g��'�.�);����V��|�X�da�,/S{mo��2����.r^�����L|��^���p�z+����ag+n��p�O��O���A��(*�ZO��.]��[o-�����ezG�]k�}muQN(
�R���4:�I��j��O��K~�K:�w^�~����v���������:Y��G�6�l$�{Q'XN=z:���;Q'�8��2n��R�]�)\c�2}=XZ�����KO�����	��G-��Y�S��u$f�	�FtP������v��y�����:��]X"y���:��g���,S@�?Rou#��������Z>\({'���JvqB:�=�(C:�	��H'��|�<�cs�����#&gj�q.����V��-39L3#�v��"M��GA
�S4�8k��2��C#��3���1O�=�.8=������*�^aW��6�T"�'�~�!���1O!��1O�SW�E��6���
�\���)�7�5��Z�su1O!r�����.ON���P��h�(d��#�~@:�~������7Gog�^�BK9���w1���R�)���NA�t� ����`8��N�r�l���n�$,�G��A��K��k6���}�t����J������j��`�g��EFW���W��s��=��<�����B9v�(�"����/'��[kqc��_F��g.�r��G���$t��UJ��Wg����P�j�
V�]�)^�������U��7�h ������R��o��V^��&1�"�[Gb���M���-�u�����q>��Id�����e{_��L���q�B��b>D���S�)��p��!�G����b�)M����3�d��4�����avu����XG�NQ@����q��X��P�g=��o��&��no�����,.x����~�\g�[9v��t"w��"�d���M���GG_CYw�I� �7}ob�2|Sz_��� ')\���[�[H�Y,�	�r����(s��Y�l�� � '���w����m�YL�k�@P���!wq��x@�[5����f��TC<f�,�x[o��������qw����,����L�d���ob,�D7����&`��PM�A��KL�}�����oJ����:��:�=20t�<�rH6/���^�����S����_���	'#V��"���`�X;�O�]�a[6g���G}(�.������D~�Q�S�r��3m����JmwI�t�e�H'�XZ� �R�F���#���6!�����6�/��Z;3���)������u��;�5(E6|+��i�,-����3a���&&���6s���f��H3C����2�N:�g<o��	�������'�Sv�aN��9��X-P���A!����/�|(+���2����p���H?s�'!�3�i� ��.[e�3���S���2mr���f�4�.(��
�Q���t�B���&�� C:��@��<�[5}G
v�fU�}2����S��������c�Y,4o����^�.91-����Fy����6G�"a'0���3k��@87����J@,%R{����]�t�C�7�ANZ@�)�������>V��'�}D�Au�D��I�4�v�z"��vI���G��E��t����7�-�3Z������o>���X]m��^mS�)�wE�/pE��N1K�N�`���H�(��jr��;�	��cg�b���}f]P��3$���N�6h\�YO'��E��.����������KK_������kGda�mB:�QK�mV��(�M��#1CN�]����n�q��z]�E�w�KR�]�:���m�����$��&2��Y�pZ��-n����?��2��2sN�����eB:�=�"C:�	�T�"�0��U����4!�H�J@M|�����5��l��2�\��L^�K��\@��t\4i�{f]������t��~�d��n�&��se2X����3��DH��M��A�t*��N:�>����c��n���k'��x.Z7T8
�T�2��1$�-����������uNo�[I��������5�Z�����if] CG.�j��G&5�2t|���S��N��M�
��v���N���;h=�~���Q�P�G�����vI���v�G�[���N�W5�����RDx	�_)�FC'@:I�^mS���wE�/pE'�NJy�6�,�y����t��~��>awj��������E������q���1�8���TQ/^�j�K:�.�I�c6|,-���&�����z"KKm����zm3:�S�m=�Yrz���P����x�'��a�����AGc�T�z��`&�D����S�r����1Ls~l�	5s�P��9�g�w��
��'���w\^�@������*���,�����m*�%?+�8��\��JpV�=�����}�]�;�~�EU��u��]R�>�Q����l��l��8(�(=	U�Q�����y��B9z:;�O$C�w�<�?�Y��U��/�������L����:��/1����X�	��x
�Ik���g��J�����Uh�+t�L-_�l���<e�A��3�t�x�xy���p���k�t�E����;��y���{N+SN6�����(FN������|����>�z[b,��h=�n�V0
5���_�Q{�w�t�p��n�#�"-�td�.2��^6��f|���N��o~#��r�hn�k(��z���;���zAD����������6��Y�@��r����%���g/���Hu�so���`}�))���Xx��'���6������63N��!���$&��	:
�6������t�
\�� ���Bq~���	f1Q%�U����V���~�6�	���'���:A��o�o�fFJ���e#DJ�)e��L�dx����H��%����&���5��&k�L�xv���T*\V>�,������w���8p�[R�4���5/�-���!��Ev�l�>4��}&�V����ag�/B��y?%/�Yc���d�1�$��x�S.,g:H[!�P��\FB���PT���2�[M�P�~T���:H������]��p������R%i��oS���G]�!ZO8'����w�?��G�h���f��FVc9�7T��U!���S�����Yy��x��'Yy['��V{���5���wt���	�L�6������'��'u�����������b9�fja'��m����;�y��
����=Z2%�sr�����6������nsR��*�x������K�DU�����w���Iv�&J����zv^��$R����G���6��y?;�`�)?���3&��`d�h!<t�m����%JQ�7C&���������r�D��=d+�D�ee�(k�*,{�BP�i!$���+]]U�� ��B��+j�]���P���$�7�<�D��z&��.�������h�U��D�A��va��a����Z��*�vI�l��0�-;�b���yz����s�����kl�� &�k�.��,�R��A�rg#Z:e���D*FD�-�V����Bt������vLD���K��'�0-AQ��E�ko�d:��D�{��?��K��(������`@�S��U�H��j�]FP(vC�G������������;]��5e"�S}n���i��1�4;	"J��j�Q�|S�|�I%�����(��vW����f�	�#�0/#o�^�R�c�D��CR�q�bwD��Z�
�3�e��&�����:k��������"�Km3"��{ms"�/l���$9�_�(|M��q���|�u��Dg�o���mr�-� ��u�Efx��B>����\!��G�6�/VQnFD��d'DD���B� ���rk����D�g�y��^���k�I����� ����Iq�I�.����v�F�+�hq��V4j��f�CD�>$J$�����!QH��9��z|��(!"�����Q���(�D(u�3�B.���yz�l|��R���t���m��I���B�t`����Zj}G��(����K��L)���Qn������oc�e�q���������%�.{�x�z+MO�:�������@H�j�P���v�%aE.i��DP�m�����x��f������Aa��ax�X�z�O�k������9���9u��93QAJ8��F�rm����w�?a�J�q�c�p�?B��G%���t<��=*���Q�g��R�m&�r<�-�I������j���Y]j�1Q��k�U���D)��#1IN�]��@�{��%��	������u���*yA�4��:��'L>2��D%Z���7���(_@"L'�LN-D<�����(?c����b�����BL�:�:�V�V��y�y�"�y�![2M|�19Y.����k4��������'.1 ���RgL�KL��C��W�
�����?��s�5�^�����^m#&�����1Q�}&��������[&����e��YY�d/]��J��m����`�d/]����e�FJ�D�#���@�/�%�F��BF��m"J����^;�������&����_���'[KD��'��'��Dn�1pT���z"���}�z"��v����]M�y�~���_����LV�~MD����Z�;�axF���R�2�!,6��^6���D������t%FDK^�
Z��YlW'�^���3�����H�����E^`��uUE�:Y�^�J��DTw:'�a�H�\_z�(c��H��=�����W���i��U������D9=A�>kkC��?A_��(p-�����3W�f�� /X�&YzX��R ������13���H��X���FIE�L�������
?��A��
3J*���W.�X;e�Bz��	�#���V?miY�������J3F\�\V�n1�g~�D���p��3�����6>JS�CI���B&�^I��O���j�j�{�}�E�����hg�/B�T����rJ
!<���>�����r�Qm�xZ�P����#Hha�����x#ORN��Afq{����9x��E.�N����p������gl!�e���z�mSN+3m'[�x�z+6JB��hp��z�]/a^?��?�A��K`���A�vM�+\[o-���T8�����o���R�6��v�������6���?�����J�
QR<7P�����zkg�	�+�����(m$���z���g������D7��O,�}��
W	s�R�������>���'���6������63N��V�@NO��(���	��t�
\�?����\���>�YJ���`k�9���QR%OC5l����P�B����c�3��7�8�����r������R�W�cJ�3`0�����Q�i�I���6%'��E5���4�9/�g����_"���	�cjn���|xV��%;J*��`�3:;?5=�P�H9D���j��--��v��"DI��))���N�;h�[�� �������'�kp�J����=1 ��Xd��9����h�o1J��g�P	�n6^���L�����"����k����dJI�l���T|�2�v���7��(�%()����
��q0��t�K�P�'��o���l��A�wM\�6���~MI�;�gw��2�B����e`��-��������9#��E���E0Rt5J��6K	�*D��W�Z����;�d��	���g���F�a}k�G�5�T���*����x��"WH���.S�C��B�#���6������61KM0�j�HL���6�=���#�5P��M�M��W+%XT�R[�9�L��A���C��M����P`�>����k�k1��5,�-<�8���o��8�4����c
�k�r�b�zE�PW��.(�Ip�p���B��+U��e��m�O�<s
p�9D����)��Y(�*�aQ�y�}���~��B!#�<���F"����n�Y���������x8F�{�����
�C~�U�m��FA�$&^�I���
J7��b�-bN3A��7���4�auX��U�f��]��2���
U����=��4����A	HF�0BtD��`@�sP��gta��sP<e�[�z"����L6�����k*aQ�1�-�C��5m�I
�%j}��9���i���$*�I%�i��sB�G�5���ev(���*I{�Bq��
�j���;%$&���^�6��zQ�����P�����#���6c�����6a����%�jGb����,`��u�������
�O6���"%���o>J�3���B���B�*�[$������������c����o�y�B����,�B���X(�5�4�C���QK����26M(�X�"��?V[*��=-���l*%�*\���0�'�yVC���H(�5�,��6M�b5kb�l��(����m�B����!*�	��s����J��eR�W����q�w�[�X�~�{���"�A#��] ���r/��kYC!�CD�/���9;W����Au��,��O��-eV���t��J��W�!x�4
L�Eu�������������kJ!����Y���i�u�Rx�zl(�)�^�>�ya��g�|J��HY�E5#������xx�����K`�	UtAm�f�y>������
3��_%xR��B�
�w���I�������OC�w���'�M�i��{�P�j�X�E9�_a74T��6Pi_&l�|����24���@���s�O��
���:!���}_MD�:9�����.OF<b��I��%�="��
{�Q�)����&Y�7��lP#~�R��#������
]�}��D�6!8�eD��a6��2m��( <�C6�&��R~���ED�="*�� �NDT�#�b"*��"r�h�E���bd�j��X<ZE�>�Dm��h#K�U��Jp����5�Z���^�q\�X^%��or��!��Z�I��F/���m�w%o�h:�\>��Q�i�����B��o:Q��DTR�Fv"
��^D��B��^��L7���i�\��6�y����������[�
���$(f[�k%�d���54u�g��]��,��jt2�Q���J�����v����b����"]�PQ���OE�*R7�2k��� ����C�=@IJP�������� J�?4�)3V�7�P�������o�t|�7��1�R��K��������A�w���'pK78(��vJ�����9���1���W�uz<��H{�">�r�%�@%��.��N��jG1�����WWsO�A���5Y*5���\�vq��8S��c�>�����$����6N:��|�0@�������6�����J����7|�H��6e��-�_[�J�T1P���8�B���k��4��wp�|���x���
=�t>}���^\���������������O�N���k���Dfu��t>��-_/_�.��~�u�dt�}>[��6��2"�k
�c��=P����O����cg�
����x��m=�P����x�um�MV�#d�s�m����\-fp��#r�x$/��)y,I������	��x����`<�L����*��m��k*:�/����xM9'�)%_Q�e�X�5����t�h5�)r�A7x��ijJ8y��f��x��'Jd+i��DZ�x��n!��I�GL�������s��V��#%��N����e�,�+�b�E�l�6��R"^P���
�����@����f�[����$Y����i�&]�u�����>�������mC?p0�B�({4�4�6����CeAM}����� *�b��(�X�����	|X;�'o�J��I>���_����Gw��<�61��f.��V��V&;}xF�Y!6��7�����nkA���;&!���p�|�|������d�����|?��m��L���&�����0�&����PK����V���PK*{2K18sept/pg_part_patch/10_3.outUX���Y��Y�����%7r%���8o�S�p���������Z�.M��<�R��*Z�L�������#�#8�w|�
L�����8_�����j�������_/�����_�	^��7�?n����|�����_}�����5�d]R�J[�n�Wz������V���o���O�1�~����?�������|����[�Q>�������o?}�����_������?����������?����?�����w����_���������������|���?jm������?�������|���y��_�������?���/�y9������?����_S�A|���������z����o>}��_@o��?����?���w?��������O�}�����o����?|�q8���TBR������r�wL����N'���S\O����_��>����w��o@����?}��W/�?����{���?��/�?����������?�E�����������/���O?���?�������c[%kg\�*z�IK���\�^���*>`Y8m4�e1�?o����r�8�`����'-��x��:�k�g���>���G�*�>������~��`i�|4�(�]99����i�����>
��,�J����������N^��q���~���z��o?|��G2y����~���?���5���?C�g�h��S��,�^�
�at�C��1���c�E9��/������F/GD���[A����u�������V�/���������
/��rT�-�G�����p�_o��>Z�iu:����YY������>�����O�;�����:�:�$�,���_���>f�tiS�����5�=}(��{����ng�%���8?7\��Z��l��U�2�����z����v��5��~m�?�v�{�&z�����	��n�/����o����_��3��w�Q{���#�K�Vv\N��c*9���w ����O���Y��N�|���lt]��]^R3��
�Q��Fm�.��;��K��R��~l���Oz��>}�}�\.t�6�Jm�Q{����-�� �J�_�=����P���}�6~����do�����z���6��������8���i�������W1�/4����O���]r	
HL_h�;��-FZvA������T��/4��� ��mZ�<m_>�=����?��j&I�P����1���/��-���w���cHQ�@�G~�
�\3��l�R�U��4��5��Kif�`�nD���\��si-��l7����e��2��]ZN;0w2��L}i�*S��U��n�-�l�6Q53�4���e���s�U��j\%�/\S����l��~o��V|�k���n�{����h����QJ��(o�KKN%�/[;�s����`�i>F��M�;7+x���h�^��3_��wx���=�.1�����_h��-,��o�����?|�^�����|��������o�V�k���V�E����V����rn���d*h�N�ik�f�as���K�W[3�/2���f_�<���o��������������$C�����W_��_������{����������/��/+�MF�����u�_K����2t�][6j{i=�pM�K`X{��
-��x!��ng�`��p��5�`�rO�#��K���uGX��r`����[P�-���X�Q[O�v�C���T�-�3���|�h��:���S�����L�>?QO��/��������?��D���_����>��0�k�����?~�B����9�s��ItM�<��;���&�2�^Lb���=k�m�Q���3T|�i���������w~x�����t[������nY���O�D������� �X]mO���~�����?����z�?�����C�va��������/����y�����D�����E������bF����h�?�����y�_����?������k�������
I/����{����6�C-������W����{�}$H�f�����9A�#eb<����`�r����}$Q�'D���-2��m�?�E���}��?��?.+~]��5�����E�M���p�����R[��[�m`�:fe9$`�I����Sg�(<A��?��Jy�q��(?G[e\\q[���)�b���c!������!;�@Bf����s
sy);�������������2�OsA�M3�����x�������E��*������6�VV���H+��`e
�;������-�/�����8)�Z/F;��!'���=!��$�����������J�>��������J����.�����\$)B�������r���]�-w��(�zZ�u����>�<�HG�X�j5BzV���2�c�|qY��h���#��4b��K[|0l�v��g���9zzp=��
Y���$�������t�g��A,�����b,���f�E=�Q�v���4/�����Q�	���
�����2�s�Lt����A�Q��W�����{��
x��lTBl���):��#�>����(��*�cM���/4r�G?���K�Qg�u�����������	�1a�%�e�t��!W��vk��(����~�COdk�m�F�S[�md�4�:�eZ�'�JvO��'���Yz�O�Y)�Fc�'���t���o_�����0):�
�Q�[�T
#�9��rT%��}�&Qn@D�W]d'CD��d�(�dg�I.-}0X]��P)�k~J��

,������IX(-h!�K�����:��\�E��]
�z��rGJ������O�5 2k�g2Y'�=�oX�2��{6��sp��'z�yH���O���)^���Ay'��e�#�e������AI'X�3�b�
��`��{)'����`�Bc�vH�D?����Y?�����K���i��kGdm���'����V�_#���z���3��~&������<��@����KG��r���}���H'��#��{��
S�'/I�U�������{��to��<a���R����xMxB���w�G������b4��U��d
������Tk��i�\O����I���7�6`Ri$��9�����FR��Cz��]��jh�9
�����1.N_�H�wF���*�K'+J�:�9G��Z:��GQLZ�����q�D��#
C��
�o"(	�Si#?�����������6�B�K4�� u4i[_V�5b,��O�VA�"�O,�����W����;�S�'X��%`�P��^�'��(#�QF��4��K(�����G#]��#�s�Bt�U��e���d�t���s�����*Z`��l�#e��A|T��<���.��o��Q6�y����9�]��h�#eD�U�b7A ��w�1JL:�%��~�,��D?����i�6����?�e��9�4�3��(O�W-�~&WS�k��dG��45�t$ ;���� ��3�^�h=���*=`a���cB��^w�.8#I;�m���=K�<�S�H_HR�m�gkR�uE�4��v�]�MQ���h��s7�p��T�Z�
�����]6�����S~�3���#���F�'�A[b"�#���6����e�6�����[:�c��c���9�i��(��)�����b�]�[FVn�����/�pF!��e�h�Z���@������<4T��'�Fa@�W�� C�i���LQ_�X�Q��]q�JL�F}�`��M���;�0ad,��#ka7��g�5�hd��i����g���>@��8"�m&:�8����T���og9{=3b����A�8
�'������h����EtF������o$��6�l�5Fe�).$�e�0s���}��c� ;�%_�pN��2A![67����i'1�v��g�KgM[w"K���
O���@NM�?�t������}��]:��l6��>P�H@S�)6��%��h=���f���I���.������jg�
y���w��wy����x�`nW�&,>�L]�;���

��b4v�����.*�-'�n����
���$�(ot�h{���~#�S��a�e��#���6����e�6�W+(}�R����|�������&}�J�3��$�u�i�>��Pc��~���F1A��b�Z���6s���]o��v��E���Q�F�U�6��Fq�E�5L�o���G�%��A�x#$C(��v�sM�v��f!.�0���s)��u��eC+)����x����%����d���!�"]#U6O<�;����Nx��
��A�7�O�QKa>o�H	|�����eLJ_Z�����5����U1:-������G���P&�W1���I���6�����&H�!�nm�������!<G���Z;��mE����C�M�"��F�`qt]:�L�ORHl�����&B���LSq�r���1I
�+ieIS!�%���������n��o�H��&w:Y�����6$����e|���������F�1�Vr��@�����w�'Ij��<���5�pG!��-����2���A�	�������t��p�v��U��f#���RM�]:Im���Zvm#C���U;�c���'�)T�$�b�Gt���6sgG�p����e�1_�Y�{g8�-����u���`��80T[�����$�(
�����d��4 ��P�Mr�}�'y�M6����z���6�%�8��|]>�9j�'LQR:!���L&(U�6��rH���Q��#�������-��J���-���!q��
��A�8J���f�{hq"�����z/�OT�97������C�sZ0��.��[m� ,F%�0 �q�eb���V��i�"�;[�x��3G�pg�����;��5���"�s��3)odp.����	��i�F��72��������Fi��N��,� #�uz�� �\���vV��6J�;��
8���F���.���8�n�H&-�p�@V�x}�Z��Kw*Fl�����j�5X�]�������Q:j�M#�f#�i#�������Hj�F���k�FY%�Y:�c�����Z:��'�c������*q�s~o�O����x�k�F��i#rjt
���1��7��(���f�(h�,E�K��Lfvq�rtG���8�&�-Q9�ZF��6���_�B������F�:p��{-��(��FQ���1!��h#�O�����E�%
�F�V��m��o��O�=��@����E�"��L(�f��9��'�*u�����p{E��F=�k�w�]��V6�������|S���������a�0�����9q�;��U��Z>�g�oD��o��������]J����J������fc�Yv�j��������
I	��Y��TPC�v$ ���0�^:S!�e����e�<`�&k�<���\�k��=�k2K�q�3���\)
1K����+�Rn������GKf��m��|���/*P��eb�>��(<��g�m��D�G��|HB��~#��%Z4��rjGb#�m�,�S��md�E�W���|g�tE��J�#
�%�����t����I�����t.��T�D�����%�x��X8;[��
d-�*c��Z�}�T�Ry�.2�R�KEH;;���^#���!3}���[
+\i�����R]D�-�"Z��&���t���6�a� ��4��!=�f*�T8���L��2D�-\D�\�6�I�k����pM�[Y���T�������l����E���t����i�N�Y��;_n���tMl��ij�����e[����+w����]��L�*��K	d���z�	�
8�b���j���^������x�r����s�����>�h
m���-��Hw��e�!~�z'�)`O\���KO��'T:B���A����5�3����4�-�	�%�!�3�;eY��i�t��M��W��o*�{��
x��	p��W��F��Pj�'
�0^�O�]�;����[xM�V��4~-y��8���;�m������W���~�|*��&�av���p���w�z"[Km���zm#
�b9��'�JvO���5�[N�������uKO�	:1n����-��B~���JDMe�����tN���l��b
���Q�ZX��$������������g���=a��!%��J
y�f��������6�������U���kr,6���wBp�<2��!����F���5���&��l��~?e���Bv�Z#85c����)���?�W��������z6�+�~a��o~�B��s(����P |�W)�,(4\cB�V�U�]��X.���5���������P��9�51*��z�J����EU��W��y�^��&z�!UD���z�'�C�?GO�g�Q�N<�����w2��.aY�P�A��(����[O�c�������l���Q�	J�SD_=du�D0�\_�UWf[]mO���Fl�r��z�#o��J�dA���Yy���O�d��aw�.*'.�H��[M4���B
G�,\�6y��awl�q-`6�x�q?�F/�^{C:�Z:6���ht�����R�9�^m��aiF�6��"�J`����g��*�m=�'�9��2��l����g�	W��7�_���(���l�3R=�R�'OWy�����������(7:���(:��(#���y�M!�V�c���h���7�W�+B�Z��F��|�o���%�A���R�/3������n��x���[r�K���LGI����+s89z-�1���
�H������vBI�oi�2���%(�<��v�����%Er��=4e���������2�I+�a)e0�R.4�f�!?0�[
5_K���}h����]�u;Fs4NO� g��Bp�������{�A�2��Zo���gf�������8�0��
jhp�Z�d:�T�B�z���?��@���DP�_t]I.��7�^N�lU��0�K���Eq�/,�t��8)I��m�I���T�����<��8)�����.��e�����|^d��
�DZ�g'�'�K1�}	�!/�}�s�������v���('u��{�_�r����THU���D�����;���F�	���lH�I���t����=��MM��"���(�'�p���c���#�}r��,��Z���~�:�P� W����R�b.+w1���N���)[N�8)+�I�<GN�4H\�-12y3�P[�I^)�����=y�A H����#gh��\����0>���M��Q�Z��8)�qR.�����4�������<$�������D���W�	'u��u?�pR������L��P3)yf��V������5)����b���C�NW�rR�
rR\������������dy;�#'�����
�JU��3�\+�w�2�����z`�DqRE	1+K��!S����������8��%8){,��w2��aE���QSQ;N���w�ZO��Ii��y�ro.����z�zk�J�!�3P���&��x6u�����6����~�}~�y��MB��&s�\�-�A:���8)�I�
�5��'o];�.���p)�1Zy�Q���L����xTP��=�QN���hZnH�mk���9���jOdk�m�I�S[�md����NZz�d��}B����	�&,(���n�~y����{��V���������O0���4j�d����]��4��St7b���nR���q\���(wN���);N�
8)'�Ia���u�����.3t����\�/n���kqDN����uDgqY�q<����g��Gg����|-o�N���{���7���������A��l�&$eg��|��u?�pR�������`���
�D��*��m�-��k��;���"T�:�3n�-���rM\i��qR@�`��Q|�Uh��/�jTQ�2�����|���SRN9L���yMW�a��C��[o��:W$()w���SRYPR���2�!����OI��k/����8��b,_�/"�]��)��Zo.����������:�q��O����j{�
))�������zNI9��=C�f�u����c�}���S�*E����}RPRPO'����;���=P�|�$�7�����t�P���l-=%E>�
\��D�����;���F�M�����'�JvO�QR�kJ{S�������������.�G���'��G��"$E��2���0�5�1-�}������\��?���~�H�W�d/�H�#����x��V�cj�X���A�o����#�*����G�)*��k5�".��U\��b�%#��m���p�����*`>�F�>k�Y�
�����{T�/vq��[#�R�*>�EV�� �F��g��p�������m`6������Dd��0{f�4���]������Q��dSrK�$�m�K��Xk�
����Z�E �\Ys��z���k���!�D/ ��n�����33p������Il��BR�D�O7�w2���)��`w[O�hO�Mm���$P���p�
q��E������9���CV����`�B�`P�lu�=��j�Q�q����I{�0��@�\g-�v�F�U�&P
2O6�v�F�R�s��~#�/���QF!$R�H�,���Q�($���r���lyL�������R���z��^�������Yz�d�E��98�����c���A[Z�L>�O��f��F
S���O0�5����*k��Y]cA��c�!�VP��X��n*�1�49@*�����d��0������������j=�_��;7=����x�\Z�'�kr/��V�QZ��d������2���8��1`���%�����I�=�57&�2N�+�@��IEZ�(�&�i-��w��u?�PR��R&\(�3�b�tW��4$�j���HU1��<�:�bZ���6�����FzUT��R.��T)mi�Y�.����	D~��l���0@
�7,���C>�PR��-����H�T@�;�����t
H�<n��'��HYDGF��j EX���zsQ�����mN���:��9{%fn�/�X]mO�����Tx�M
o�MHi)J
|��4�M��9�{��W����R�zA��6����G���Z
`�j���m���(%����{���r�mT5T��l-�
(�wj��
k`:Z�5M�{�d��}O~#'������qP��A�����q�1��w�	R�t���y�;U�����PGJG���O���7J����RRq@I�W�(CI�%�t��A�*�g.S���h��Au:Rt�s%�� ��("TS���Bj�/nx�WJ����:Y��M��_��|���]���V�7k�N��q�jN�(,��j����������J*>?J*e�(��xD�N^��)Zt���\�����<���C)�k�C;�.J�Pc��G�,��oZ��v,#�6���R�I��BSs����k����9f	]��1]��Q�aj�������z%U()�j2�7���tN�'�/s	_�ZO����T[��;k\�:�^J*n9{4E?du,��z>�B��� ()I��mHI��������(){��Hm�XZ�e�n�7u3�,�v/#'l]���8�.��Q_�����m�;�.�s�y�
��E����1g�&�������B�#��Zz"[KmJ���zmC���G�I����2R�,�=��}��<���Tv�Ii9�b����rdnsp�W��O0���I�SRZC�`���q�:�Yk�V^�K�o��1����i@I�W�$CI�%����h����yF��&<���V�|%Ex�VH�*������y_vmmrd��2Y]���������L�� ��<��J��T�M&��T�k=���Jq�bQ�@����$CI�oi�2�T�d���()~��P�.aM�+�t/�hD�	�,"J�����M��y��*�#t�&�h��b1V�Ph�{9��&SR	�U��{[T�	�()�H���[�P\c6�RR��-N�@��D�=L��j{K�d:�GIA�L#���DP�|w���c�"�}�{\����\�{)��%�����[]kQ]�k\"�s#�'Y]mO�����Tz�MMo�M=��JYJF
���Q���������w�G����V��
�M����Gg��B����:���G7������tW��~��JGes��p>�Z:J�Bz������R���z��^��21������*�=��>=��0ZO�	�V�#��D�9�*c2�Y�`�5���V�JI\�K��
�esm��������&���JI�%�_u��%��T�R6G�L�����)�k��(���D��^!�/JR�gh��6>���S��;es��6���+��/%B>@I���JZ�MK������*��2|{������3��/���J*����u����UzD����[L�����j@pD-�5N?'�2)��:����u��1%�:��Y�0���]F��TJ������b��^�EIe�����jo�������Yzzx=��JF J
��p)i����%�XQ7��EP�QRY��gT�����x�f�a3y������{�$��ou�oDf���-�a����T��mHI�������SJ*�%Fj*������{�.����S^aG�d��aw���fN���������{}�����G/2�G)�|����Y��o��>J��b��!���D�����;���F��A��eZ�q�d��}W��z�!���npay��D,e�����ZXO���?����&)�Ve����Z
�w!����m,�"1�Ge[re�%L��*J���(J�(�"DIa�S���>�`W��p�Z���^1����Z���~��������2_�Q�'R����������}%UzJJ�|��%eMZ����XbM�lTv[����PR�[Z��%U������()O�T���V��Z.)M�������7�;��U����DX��GX:�Eh���r%w����v�{@eQu:��(mI�t����t���i5��������Q��!g��5��z����p�g�����))��9!���t
$���~A��jGIryr-V�=���*���5��z/%����]���ou%�Z3���W��&]mO5����Ty�K-o�K=��^��*%����M��s����@���4'[�������6������������G�5�%�e�ws�NP:�U�l�,�B;���TjpQ���k�$-��4KGb�����	X�..q��u"E���8��D	����	���zP^���{�e�sPH��HhJ���]��.��?�V�sP��_�������'?������4�n�?��_��5���e����y��/����&Hx��]��������~�sP�
�{H����%�	U����l�%9�������������rP��2�#*�Z�6"����N�Y��BiI��t�5kXT�'�AA��-y��7��g��qP)�g8�F����r�[�I�=k���[��6��7��]!�%��t�8S��,==���A�"�������j������6�8?��DP�L=�����j��GE��zsQ�������l~���vt@Rm����.�$�����W����@�vO���PM"���"G3�%[��<wq��{q�����tH�[����,
� 7]�w�<���P��4�Y�p#'oc�=HD/���V"�vk���bT���tD6������j����a��U;+d�$���S%�kG�HDa���Y:��G"��*��t��
���X&�P��Q� yi��Tv�*: U�in,m��<�y�;62<��PF���i6UM��9�U�<G���V��.7���k���i�y�����Azo��3p�.o�T���US�������3+c������q<�
vd���
3_de�2�y~��D*TH�J����A��k���A��H�U��3��<6��A1*�-��rZ��X�M������
��sT�*���}��|�T�������U����EF�T����!���������IT���I�J��w2��k�/����j��F�4���A����I_g�{sQ������^�`@X�!��6�O�P�*�l�8(I��m�A��=S�<���<+�A��8�X��b&����Q`�-�@���������;�DA����`%3mt�h�G9����X6����O�C0�-=�����Nm��;L��R�z�d�G��X{�O�Yi@�,������|qR>�����O00G�de�
�9F���G�qF�b����?��8����r;�������������b�����<;�������N���6��j[�8�����P����VS/��3��jQ��}�O�=K-��#%��y�v����0����Qpq�6�ms�9�,u��u?�PR��������W��V�u�AO|y�(�(�Bsi�aQ(Q�,<�z�T"k��l#>26�-��	�Gc5�c��s|�������t�sUJ�{�y����|�V�a�����G����Yzzx=+5O��O�Gzd��L���(`i�������/9�`�mK�&
q&����0��a��KI�-,*���:�U�k��P�e����T��mHI���T���sJ*I��#OKY��1�����w�=%eh�n jC�$�����]�-&{�����������@���;H`?JI�N���N������
������R���z��^�P��*�^���X%�'����#d�'���t�/(�.-�'����q(ey�]�%B�����G�.��G��%eQ+)�@CC�lR�Z�Fw��%�^u��%�����lBt���|C��B�A))K^c��W�MTq3B�����3�x|�m�)�r�"PR�g�u��e�b�%��J��d�y�~�26�2Mt�_��|Y��,r�]������d()�DI��jQ���"��BB]��Z��F����qm�0+	R����l�8TB5^�F:~&�t'��E��k�S�.��������y��Z�����2�H��z�M2QR��-v�@�T���l&��	��;�NJ������'��QR�T^��DP;J��2�Zo.����z�z��Vg�a&4"KX���Hs�����^mCJ�=���7���g�E�()H�h�*�U9O�>v�%E[�g��������()�,T�=��������\��?Sqv_t"����;
�[��;�ZzJ
��6KOdk�m@I�S[�mDI���{&CjOb������:��������hAc�����d���,��O0�J��j��&��(�-I���R�5���d���
�#gv*%����Q�2��PR^��j�g_���P6:f�(��^kM=oj9��!V������:>�]^����b��z�x��-2>���}�T�KA^))e�Q�T���p.R���k����3g����d()��()�/|�DI�U&VN(����~1�\�����9^@�,`S�X��1J
<p��oZ����C��(�:�<Vr\c������~�
%��Z�?��20���E2�o
��-AI�S��?���^B��8���Yz'�) `NX�K�ZO�(`%	�:l=���9FXL^�Z�d�{))�EIA.�?du�l��{Vag��0��j{���6����n�n�9%��%�A�Z0�y�����\^gT��m5m"Z��J��AMr�Q��;7��QR��������(%���Q�.��dk���,BH�������!�N-���x�AFgU��M���Bv�]y�<����8~'����������5�e���/�Q�|s�k%=�UC���8$�\�U�.]�}s�s1�H��"*��"*�� UIzA�z�yN*5e�d}��Gx&��GgD+���V�mj%���%L���k���e$T�y|9���L,xda�4� �BUP��Q��2��m0��X�1oR�9��=�y�"K�~�!����(Z.����\U�QQM��������E)u
R���TD��Z����1)��+�J��-IP{�b�nQDak�m��P��(��\f���y
������(
}��O��
��p����cF�\��L�|"
��$��D���v���k-��)jO�������z/�Jz���>duF�/d��N}r��d�����W���
����
����z�B�|m�Z�������N6�vGDe\�A�&���N1"�\�G��aC�
b��^��s�P�3I`?JD�c9=rtM8�ZN��r���l-�
��wj���� ���ZOb����W��P)�'����
R�n�fy��Ev���
R��'e�>1��PE�VJ�n�-Z�;m��qD�s��M���FIi?���J*��(GJ*(�(CI�<W�#���L��O���SR��r�D�K���7F�������Vq+t��@l�YU�n����R�6�(��QRT����n,j��S��1�X|�@��C�fy��PR�[Z��%�������Dg���g�-�����-�T@��`�����h���4N9f����[o�k�l�������C�T�=r�[Rb���I9�r������t��c���B��j�5)Q��T<����9�$E�����3l�ov�9?�XW��A=��:���T��=�c�+��-*�LE����'E����:c��wk�<y��d�����W��������
��'�T�:HG��x?���(���'��"�a3��w�Go�`=��8v���>��t�R��9w("/��('��y)[P����qRt�)�����ZjpR���kY&�6�Bm=�U�{��>�8?�����R�N���{�O��*� L�z��`Pc/{T��|��ja����0��JQ��+�����YeT��I�'�^���'��T��r���*y��S�U�Ht��I�{�6T<�b#�C�S��%�0���)��z��k��R�+�W<��n*�._O����J}�J�[��%�2���ZY�����J~���F*_�~K�~�����9)�t:|�������f;���$�2n����d�����'f	vR��V\DQ���I������NB��j
���������\+
����LwP|�Yz-�?h:�RP�0p�Zo����L[>?KO�'QR�
HHaY���N�s����7=l��'�z����zk=�����"[o.����z�zh�����CB������^d3��j{���6����njzn�y������r����g���Y�;�NB�&<ZN5F���0`�RF�P�����I3���IHYTN<n�����RR�:��Vq���aR�_V�}��l-�
(�wj������g�l���X%�'�����c5jO�	�r���h���.L
f`������� L�����-hA���W$�������lP9��Ry@I�W�,CI�%��2�0�6�:�	���B�d���=���*��\��$Tu��w����t	
��Rd%y		)�-�g���-O�Vm~V����I!���1F������A�ff�!�	27�{3#+����d(��|J�����6 m�e�
�E�����Z�q��KH9�)��[������JH��rY����*�������r�(D@��mj1%kh�M��B�s���		��J���6H��h{S)�|��_ J�PRA�o{��L�|J��6-��������AT^���))!bt[o.���T�����:��Tm�WCdF:{a�����6����nj~n�9%u��Pm�XWT�����[��n,FI�H������M��%����M��E�;J*C:����A�F�QJ*����$������

q���'���6������6"�2i�<z�I���t����L�,=�'�����'H"���	:[
���� ��G�{���F�T�Z5Y/l,&Eu9B����C��RRe@I�W�"CI�%U���"Tf\�g��O��)��:1)�K�0JxI��W
�Z��HL���[�QlR��������pR�*&�n�xF��\��u'%U:J
���?�s�v%eQ1�5(�)����E&��
��oi�2�Ty���������(�hrR&nbR�
�I%
?��\�G��C����+"���8��3�C�Q.s���

��>7#�>�kU!N2�p$�!��*>A{�HM*���bf����/"����_\��I�$�&���h��t���t�W�J�VK�-=���F��Z��A=�I�\`�c��\�{9��qR�@����hzY��B@���H�$�����W���*����
����{n?��';:#�I�]u�)k{���;�~Z�d�9eA��jrFU+���5D�tf
bZR.�a����$��(#U����b8�X:Qs:�
�����R���z�v^��*���D�kGb���;QsK�B�J�a+�!�4Ov�j����D�-��M�6�QhT��q��C�J _��C�2**;S�Jr��l�b�\)��y(���x���|����R���<C���\��1����������j}}�Xk.��F�x(��+^BTV��e�u���W�h;��nI�8si|��PN��z�(��&��G���Q����"(���CAR��z���Dx(����C�"P]�;z�����j�B����Xh�'�E���(5����X���9�l���>AwT��'f�����E
���4�65�}��z�
C�P�4��Zo=�����������h�b$��2�pD��������C4�����A�����f�2�5���]��Jh�����;i����R������K�-�4�����iVW�Sm��
�����������
�2)����H�g/�v�C%� �D����bwA:�l�6�X�q��z��
��]b��Q��79�-��fk����g
�kOdk����z��^�([/A�RKOb�����O��'�=�'���h�|\��?AG����-��O0���E��F������w#����	k2.��Bs�c�qnh�3J���(J�()#CI�<�wi���[+�D�M�OI1^����^���Z?+{���ru���/l�f�"�~�y��Y���\+Oc�V��J���Y��@�B3?�<���>j�����M�'�k�W�	%u��u?�PR����wq:%�W	�����}��������d�r*7*e�%��P���������6�T_�^�C=RR�r\��X�b\�s5�=�\z�{���>��s���r������"Qgo��^O��B�OI�$B�W���tN�'�r��D����������:{3Q�5A*V^Q'��^Jj=A=���y���xHT ��l��������^mCJ�<���7���g��i�����h]�8�����O�>v��fr�@���o���;��\IR���7U��cw:�%V>��M:���SR�HI�m������4��a����'���6������6�L���o��'�JvO��'�&�������R��!�������U�2���?�0J��P�P���(�����	1�%T�y:���n�T��"���()���le();���P�T��i`
n]����s��R�� J
x&��]�3A���u�h���y���R�QRu���n�Y�m~%e;J�����:{��0���9��2
��i�;St��,��A����������#J
q��{L�fT�����)n�P;J
��Qs9���%�D������P��������yI�'�
6���j-dG���� ��3k�po��j�f��lO�^V{��'�E8){���sRI L�Gr*9���N�s~���m_z"�]�^D�#g����1[#���5��c����7aR������FaB6��'�l�����W��������
����jc��D=-�[Q�9�{�������	��
9	b��I���@6��Z�]����\�'pd
J`?�I�.L�&��l-}�����t��l-�
8�wj����h���9�H��~����6�����B�]�T���%�kO�	��=�xh�/���6Js�y�*��&_�MZ+	�lT��C/c����WWM�IlT�Q���wQ���6*
�Q) ����A��T��k	6��C�A�U^"d	6��Q�9t��n��D���!����I�u����=�F���"}
6������=:5��eH��r���k�����Q�[Z��� ������U_eQ���>�k��;�.g���J��f
Z]&f�u9{N�C�u�Z������*��^�%��h�b�eE�H��%�Zo�����j��,��-1��v<?KO�'�Q���iPPi������,����A�r�lR�g�=�c�F�)��7�^2*nRt�u��c2���]�O��VW�Sm��!�x!x�
�?9@J������t8[bdT�t<aMf��%I��v��+��}����L=�}��,Z�q?JF.�^�U�������:{M�����R���z��^��2MR��V���*�=Ag��k�]��j�=Ag��,�S{���t�J#�gs�z��`@Iy�l�
{`�Z�w�MH��
Mp�tm��o��ka�Q������]������JRR4���:������It��AuR�^S����6Y'��
��",��Q�YG6�K��(@
x����i��������T:�3w[�=]Rq�T����Z�c�d����"K�~�����))��|J��)k�<J�6i�dff��2R�8L����=g��&��C�()�D$^�t�7��o�C�\��j��(�r�'�H�V�,hS{��N��Q|T1�,k���7�>����Yzzx=��r�QEGU8�e��L���(`U]��'�z����lZxw#^5��X16�x�`�@I�'�w�(x�X�Y��5h�V�y����T��mHI=^���B���T�b���y�VM_	�.�vW�K�:�tF��|�&�()Dg�8�O4�":�NF���b�b�.O�Pj{��JG=sz1[>����SR��\����R���z��^�0r1���'�JvO�QR���P��?�i��w-F����'�()��oO��'EIi�q�
{�Y-J
�x�88�.��)���W\�x��Ww��#�j%z�e�<`��#U�
Ms���8%���_Mp�PNpV��k�E�f����-����.�!���j~]���4��Y����5���`�r� ��Z+v�t���z\J���W�l��2�����6*��z�Mg��RL�7{��.%�t����/���A�C���UAG�t=+����Z���m�����*x�Eo���g��Q���������&��_l3��L��v����|~��\�b��|6
�tM����;��������h=�^�
����5��=�q7h�����Qy����y���a�
`cJt������j{�
���k���P���^5�	u/�.������������(������
z����������w��D����{6
�}6]������Q� AZ�l-��G���KOdk�m�F�S[�md���8F��$V��	:6*��1rpP��?Ag���m���%�FAr��e�m^~�Q����H�sS���.�a�P*(^�z+27�����0��(/:��*����Qe@G�)�gW����]7���x����{�vQ�E"gxi���D��e�)���|f��������pY��NJ��9{t*m��+�)ady�$��B�I��)����5����d(��tJ��+�����n�h��U&���01��#Fu�G�*�\�������\�� ��]�JQ.����B/A�v�����:}�r��Yz���~��I^��U���<����i�y7���cR��Ot���k{��L��)��'�z"���a%�Zz"���D��O�`ko.���T��4dg�:h�*�e�2����#%iz�
������P�<e�^���m�Rc�A�"8�3s����)���]�+]�J�	@�
��{p<���Ma���GQ��
�u�B���}��G��r���ac�nw�#U�����tD6��\�;���6	EpKGb�����	���*�������u������M�Ux��s�����6��DD���-"J�q�^Y%�j���5���X��V��xx{}�A�W+�{-�Ay}�Ay-�Aa�y`���-�'-������Z�������������b<����Vx9��g<rP�&����3��~?�u�A���Zb���
`#�#��rJ���Ul(��g���"K�~���o~rXT�Rbz�A�UZ�����~DF���������{Yw�{?Q���A��&^����jd!KO��bca*DM��(����>,*�@����k_Q],�$=�=Ki���7Y��������z��qP�Qu9�l��L�|����[O��A1VY�Zd�z��+����\�;9���ZG��CVg�gt	efn�^�|J��j�P�����-�?'�L�J�3�$�ua�����7eq����
	I���hUnt��cw�?��tv ����z���Z`���c�-�H`?HD/���k�����QP���#���vND�WK�m@D%�~Z'������?Q,'g���Ev����b���'�'�G"�aS�O���P��5��*��H]�a��P�!���:�/����k2��nD��u���!����2BD�3�7YT�[��%�����(��R��%���FGG���C|�xY�U-*�"���x��^R��5��*	�!�d��������CS��A��%���V"����,��A��2�W��i��yI���U���k~^���c>�'��F�	4Cs�m�E��5�8OV�-�2���X�/B�U�"!	!�5�����������#���^;�
���(���3U����[���Q�U���*����i"��u�����t�'�<yr��'�z$��U%�������2��/6���SP�%����G[
�n�:c�GH&�_���������^mC"�����-T}QR�� j�x��t0���nS�����;�)�D��p�|=��m||&k�*��F������.L�q[�EH`?JD�c5=W�'[K��G��)LKOdk�m@E�S[�md��%\.��$V��	:��tr��^�'�]~�"`Q���t�����@�	s0���������j{���ZTye97�z��h<p,�\�����(�W��{+CI�%e���h�|�<����3(�.?�w<�i�<[�����c�P����5�J���<�K�3�j��b��(�C���c.v� �Jg~e4�8��B^K��e��0��F�>���q�~���	K&��J�p�6�{�X�
���-�<[W����P��<���z!�\���S(^.6�Q=��fws�s��v"�r��k���k�QR�.�	Zo=�7�����i�y?���C�T��&���L�
�d:�SR�B(�3���5�D=RR�F:��rO�HI����Ro.�����()����������p�#'�y�nB��X]mO�������E��[(�~.`^.h7�6��
���H��p��3eq����z��'TA�T�2��v�]l����H�Zp>����@��y�=6��[vm�RR�HI6?�ZNj�9U,k<����R���z��^��2i2�I��:svO��"\������tV����R��YK�=Ag�(�>���CJ*+��ekS��y��y��A}.BZ��[Q-���`N>�9%�^���%������x��^�9nuP�F^�tJ��4�gU
k:�v3$��	�aJ����r��\�p:7����@-xn�!�'IFy���3Z�Q�t�2�<�"��q�I*��5J*�+(�I�oi�2��{~��u�9)�J����[��bbIbQu�����[`.���1�Fl��EoZjb�.�Ky�;�#%T� b��"�_���57J�l���[o����s,�f���z�!��MJ�������*�))�D��;�w2���)@U�k�����Fh���z"�� �LMC5f2�������P��!�3ZghS*�MCE�8��j{���6����n�{n���yK�C�N���K���c�����w}���r{�&����]����z����!JJ��'|���~��rGJ�&g#!n��>]��(�T;"KmB��Zzm�t=|�/uR�#�Bv�]���C�7��~��gU�9������l���u��o](���QvM���a�F6t���2�G�[Y�X�d�r�D��=�2D�Q^��2�?Y��<���"%=%V�HD15\0��b[��)�XG"
x�l�E��o�Bzh�g�ds��:���g(�Z:�D�?Q:����8c�^�{"�f�, �+y�:J-�$g�������"�?�����\����9�b9�3(���J��w�G"
���|4���b�t��(�#�e�8������o�+F_gr��X�ew�&����'��<���k�u�:�(6*��x���e�&�JD�Sg�?����I�FeO�T�N��L�@laa���A�b�2��"�rO���*(,�P��7�^"j=A���k|��Jv��LN2n���1���jz�
y(��w���wz�C�$�C��jX6�rS�|���A]]�=2�S=�;Z;��w����kn��t�����}tp�F�kH����?�A��v9GO��*E&
�=KGdS�m�A�S+�m`����%�jGb������}M�]:��G��t�i���8��D1l�����?���+��Z?/��+��a�	��34�Xz���N�tG&qPa�A�W=� �A����4���d�&/����������&S��}.�o$�uFpR��Gx��V����]�p9��^��42�ZuP2�k�^pP����oN:C�1y�#����[E��������2
W^k��������d8����.�}�Qx�	��IE��21�i�WT��E�����Y�pM5m�����(�������ft��kQ�;��dT�a6���g^u�u�������o���y���m�Ci�i9U���$�4��������Y�PV JC ��w2������_�ZO���S!��'�z��"�������A��~^���CV��K�5�0��j{���6$����ix��y~^��\�0������B7D:"�.�vE�|��r,��}�����D��f���Bb�P�,�	����.��(u��{g���������K������R���z��^��* c�H��~�,Rg��8����7c&�jG���|�:����"*Hh@(*!Y�Q~6=�������@�������`r0TQ�U�8�Qq@DE�`(��I�8�^�v:�d��;C1^�#�=�JD�(�x�j��X�[������A0��u|(�U�38w����DT�������e�6���<���#g��M�<����
��A����O�������DE��z�����
j����{Y-�87Q��#�0Xp.�Z��[U�]��H���*R�#m_
Ua�p3���4�D�|
������l�P�&��"����v��qiS��x��_���.���4��;�N��<���ZO������	j�eS&�vYy�
m��p�`�@D�'�G<g|���K�����o?��j{���6$����i|�� �B��F
�!��������s���F�E
3�tH�2��w��*rQa`4n��Kb�z����2�!	�G����{������^��}���z"[Km*���zm#��d�{b�I���t�I��xP�'��������'���2nZ�`�Eg��G.B��B�|�M(���g�#�+��H������e����J�:�I��JF*�0Ru���� �2�.���!U��"���
M�>y�g�s�P�������K.�tr���L2�9�T�Lz��Y����2��,��bT����������}&�d�S����u?��Q�����l���J�R!�Q�[m�&��1j@ng�t�r-�����U
���:V�����v���Jc�**;�@�-�2��E]����Z\~�m����b�RR]��?d��-O�qz>��:��n������V���&�#��X��h=�#�,On�j�����QiK�CJky������6u��@������^mC6*=���7����Ee9�rSchm������vl0a�M��tv��E6�~!N��v��l��c�2vV�O�Pj{��J��<�n�?�Zz�rM.���D���l�;���^Qo�F-=�U�{��>��'�}��<���[�iy�hE�t�gzn���w�	FlT�+e�j�o��K�|d"������g�E��_����lT�a�x��E������]��}�%���5�a�7�&�D+uy��"�p�}�N6�*z �Y����kI��Q�O�3�]Cc
]����tj�Z� � ��eBQ�W�v�F�oe�2lT~>e��K��{��t��-Jn%3���gF���Za�bh�K�;z��l9�O�m��������rEE�:�g��Z�/�8��^@e��Q{��5�������|"lT>���/�F	$��$��g��L��=9�1���bEP;{�]ncuY��'�0�c���z/�wlT�Yj<t�OAY3��j{���6$���j~� 4*��Q����y\7�9K{}L*��Uy�������2�h�8 i��.#I�K�)��I�Q��4����Q����j���*�PTB�sh������M��
�N����P�Z&��������*���#��	E%�|�t��;�(:]9t�v�����BC�~��c��x�}
Ee��l�������<}���2�����q�����*BBQ�gV��<g����\��PT.dl�
E%��QBn�8�Q(���e|i��
4��E/���������(�$���}�3����c
*�*���h���9{-$u��u?�pP����u���3M��BQ�8�G�l��R>sE��V`��\1���f�F�\+�����>3��
�|!����q.\��3�(d9T��V]�-������<�����w"�y��.�e����T�h����/V,�lj=�c 1���c�2��P]������z/U6�(�G�YGa��8��n��)VW�Sm��!	UwO�pOO#��"��v	K����2�|������". ��X(*8���]���T�S�L���r~����7*�q?JD��P�3���>"���)������R���z��^�0"�*�j<�$V��	��([����>"�pcjOEv��tQ	s��~�w�	�)�X�V���y���q���
���R���7���s���r�b�RR��_q�������'�?�%U��#�9�Jl��A�9l�/pb*�Z-���������i�L����uV G����
O7}AZ��IIz�)))G��V?O���+m s�,=
I��V�]�e4����J���gSR�N����X^%����]M�bIz�
���+74�+n"��������d}������x���X,�c9��ad��]K��()�Jx�1��k�)�2p�]�Ai�u��z�{?��������I���))�D����w2���Q����z"��z�dA*�h=�c=J^��n���wRR�	����pX���iQf���Vk����T��mDIm w���?��(�|E�������afC�\�x�KQR���O��o5�����c
�����1��w�G7����Ns�U�
r��~��:^���V9������,mBXF��D����)��j��
,�t\���d�G��wey�]{�����T����'�G[��m6��<a�PR9��lm�N�lo��o%����Wc2�(�l��Q�������2�:�F��2J�EI�<Cc����4J*z�rz��Ed�A��|��$��x|)����Z�9�*��g�Q��[�FL��a?@I��rzY������C�(��Y
a�1O�r������J�<��
f��9J�9,���Ay���J:�y�k��M�y�����nE�`��F���m[�d�F_��w���
�0og��L�X3�����%�����1��
6��=��
*������S�i��g�����()'QNO'�FO'�N�S���v�����QO�+�GX6���
�LD���P��������RRf����N��#VUF�5�Hq�������^mCJ�<���7���GIy9�(G��Bt>���]�;�.
51Q�'�3��w�]�TKA�I�8y�a��Q(���i������y��RR��+N���r%UT`���'���6������6,t�U�(Q{�d�}�K�ws�D��?Ag��[�}-=�'8+}�*����?� q2��FIAU���T��()��!�����>o����S));�������������B�����h�����V��^���5�!��{3�QO�:����Rk��v�9TA��V�%�R�O������%��r��^����Rb���mMZ�3�}����d()�tJ��Mi>%E���`bA���Q�uZ��P��P��.�����bR�u��"�2��i�������8)���MS�k���ar�T��aR����0��w�D	�z���~*�|~���^O�����K,�ax:�w2��3���8��A=f���)�-�-�e*���+Wd_zsQ����#����cVgP�� ���;�� 8)I��m�I���T���A��,~��|!0@�|t~�����}^|+5����y���y0p�NE�(9�^�#c�m�)kD����1L*!A�dk�$����#���Xj0R���kHHE_E����
��%�P�g���?JHq���,*����R���j�\�0Q����l-�E��`�l2Q���kY��.v�0/�Z`bw���^����L�b�x���$�I�v��!��JG&
x���^!���/��I��(�9�����-P9b-
0Q<��X(�C��N����(���I�5\tI.��(O~���E��]���t>�(�D�oi�2L�{:eL�^T�^eP��� �����'���@uHY��[Z�s����#�\���T�f������������T�q�f�D��je2*s���.������-�i��z�[7S��,==��DD�(@D���g��;���D��B�������@P�Q�&-9�HD��td��E�������$�U�#��g��0�lLO��v?&b������PZJ�]���K7���/�0D�4��e"@DI�^mS"��wO�/pO����T�-e!�������n,ED�efx�h��|��P<>����m%Y{��"[�}��<��.u��{� �\���e�2����\���Zj�PQ��k�Y&�v�V�I����I����	�����	+��UbO����`�U��v{���3�2���|=r�T�=_/�)%eY.��:4�g��z��U5'7����?:�^���J�QR������M����w2���'��<��^V�I~F����))�#��!����FIe:(()����|�U�{�4������yV����9�����Z�����]-v5������'K~������z�<S�H�` ��%�naJ9��\h�u���P_�uY|�3Y��Y�cup�� 8�-[Z�b*�u�!K�����p��YF�El�q*p�^��O����()6!������;�$AI�K����QR<�����w1��))��\�X����))��O������ye����[J���Q�.���s>,
B;.X��^�%%i{�M))�M���M�����(���YAt�Ed�+�����.��E�Rp����4i�&r�b���/���n��H�{��2�>V;(o������96*�v����Q����0��l,�M�?j���b�PU�c��#�B�ClT��(�u��������E�q�!6*�d���BD�hZlT&AoH�i7u�C��:��O;��]�'�������;Lx���wdx�0�����yg���i���Dp.�[�C1w���GZ�B�E���z&>l�KNK�FU8�������������
*�	z4��kF��������o����z{XT�Z������d8��~����*.����]Ai��=f���7T��%�����raQ�����H-��x��`e�H.*���vAY�%������B!R��kG�/iV���C8�*i�~�������������(�(:����-���t����L-�z"�g{����ZOu�wu��c��Z�o9���E�D�euH�Vd��|�j���U��m�A���i�����Z����
:q4����{��W��E�{��7���Amu�lp�N� ��A���v	�#X�J`���Nt���=/�ck8(hn�����Xj�pP��k�pP��G����r�?sP������g*C�=T�������������'��P&�+�\��5��8������XS�X��f��W��*�� G*Nh�(��lC������%<>�����T6�=�e�s����[L����@<�C��>�������IE�!J;$h�����p���N9�PF���-�*���P����6� CE���(Z��]���R�TwLM+*���Ig��Q!?}���Z�-���r@=���^2���b���������3�3KR�V����T�>�
��k�����$�Ay�[��Q�"z����SQ��CY�"�+��.�s}8������]�z�r���Z��������]�P���[**��P���t��]- ���.d��e"@EI�^mS**�wP�/pP��������&.oqJ#J�#m,�^��1��S�&K��$���l������B���
���a��H`�e��9A)"�bg�R��z��]W�+�M��?j��M���!��u$V���D��
DN��8�
�*�j4��N��Bx���&-�S�x��I\��j�6�(=O
5]���kP$�jqV^��P�G�8��PiBC%�h�:�����ms;#B����M������*��#����f�9
p�
<:�F�!
8W
�i�;�lz��Jr��W�4Pp�B6�~�#9�{�������vem��V6� CA��@$J 
51u�Zvd��B��}^�4PPPSa�(T�
��D�`�����@A�M>!�j.������@A��>�e��K�<g���
��3��~TC�n
���\�z��?�T��.���>�@4���L%�N�]L��������AD�,.�>Q{"������������RP���B
�[V��	k��Q�~�������6���}�4�����J�����~���lM�(T�vq����a����!A���B!�{H`�o`���r^E�YS��x���.������2pP��(��uD6��&����&�r���u$V���A,��:��C4��GcqG��A%(6!�v�����5���V����qP%�)��qff�x���Py�2T�pP�G�8�pPy�Ae!�����l�|�YKKH�3��jW(��o�O�	e��sP��8���
�����6V�fOf)u�z�?����,���0��>T��f���JQ�0�`����.��PF��V6� �A������qP���(�h8���>�ul��Ai�U��:iiW�������J����.*��:l$P�*�9$YD\qS��o3�Y���6]2�����/o���`>����������9zyp��A����#���j���PY#,���DP�5����t�'�:��+���Z�o9��sPtw����g ��I�����j{��j�rP��g��gz�
�*�y��uF~t���^���g����3[������'[�V��\��o`�3��j!SI��
)����A�3P��bk��r�XTw��l-�MX�?j��M,��U3���{���g�t��2�	�����l��:�[O�	��
\���j���3�r2|��>Tt��x�d���	��@b/w���t^��Q�G?���Qe�F16
_
Os ��66�>+85e��V�%l��[�����(OwI�����k�������)E�	�U��*6QD��{+"��e���GD���a��p�K��1t��.�Lv����'+~�a���Q>��lT�����H�Ri��e���**�����(Qv�69}(B
�}i�|s�
��zf�x��������W������H��a�d�m����3�d��;�i��������Q��#.�AR^���^�����Nv������W/Q �X���!��}����%���Qeg�hKAN�
�C�^��*����VW���W���*�=��<�k}(m��(�Q��PB��GY�%���=��*N9���)�$���E$B@�@9�^�C{(��C-Gf�v�Y�c!�]6��5�����?��Z��y�.���D���&l����f���N�4M�I����	����>n/�`(�G��� ����
7o�'H�<�Y~��M�����i:Q�M���K��RB4*��Z��&�Or�$/�(��?����z6���6�R���4'��Lv���I"?��*��%<��A��6��(�yWG�/R-6��gbl6�G�
�!���ntt0��FE=�����������y�n���UfsW+O1����'+~a��_~���lWY�=�>���=��R����z�Q"���+/����E{o�$m5��_�m���-��%�`�����)Ni��X���GyD����1���5�Y���>*�?j�~�:�x�^�\/�QI"A��a��w1��
X(7�ZO�lP��bCuQ�lO<�}���+OG�G�7���[VgtFL*�<�K�^�z:J��j��Q;��.��O~
�)�4�h]D����r,�M��=$����Bo�J�"�C�^�[���Ra,��C���P4nr�����&u��{�4{��tWD�j��l-�]�Q��k���F���5&-�JO0&�&eKv����E���o=�'�h��io���O0-��U��Q�Dzp".ftH9��(��6�]M{e&t���Q62t���QF���if�&��q����p,���4�FYLE$8��p�-�[�'��(�LG���f������g�o�Qf��"���t�H
��x*��SQ��T��M�������:��OG���>8J���)OBg{i7eD��(�g�p�b���).n��x���Jd�H�N�gaJ�!8
���Q�M���&�Ic]�CE{�e����j_����t�7�n�~����K�(s�����%��!AOs��������h��kO���P<�F�mZO�lO�2����~�F��x��-�+��1��/6��^5���d�����_��^�Q��lcm�]Vi��o�*���b!.� ���������g-$%FV��kA6F�,C�!�j���\���D�9��s^�����2(EYx�pcjGdS�mBC�Q+�mb�&U��r�?��J���u�����a��:��g���:��gJQ��57��Ep�)���� ^s4���n��Z��N�'��cle�';����dtV%�����Xd�����'�r����U���tY7�Og�'�Bw�g��/�<�'����EZ:������s���h;���9���8-�y_P
�����"C?}oe�2��}?7����yx���#�]���w'�VTJLzA��)E�`l|��P�h(���q+��U|�ae���3����
��Y�B�.����T�d���:vc��b���z=�~/���~��>��r�$��0���R�w1���	X��Q�'�:�O��>��DP�	�m�����+P����N?��=�3�kl�\/�@pq�������6���}������r$k��\��
>��u4~fJ�Y��!��:���:������=�Y��6��Z$�$�=�����6K��Z	���=C%��Je[�E~S�3�z"[Km���zm3���������X%�'�S[�C���V�����V�������O�����	f�y�B�6����o�P��8�	U*=tB
���[Ad��<�$?�nBG�e'CG�	�d�(����G��v����H�3x
`���l=��'/@H1^�W���=6��)��^��!��i�~��r#%e�(].*�lf������=�V���EY)J�{K~������T�����U:��#��e'%	��r�vP[�
2
��*X��I#UQ9$
�1BS�J�B�������L�Uj�KBF���(��CU/�^?�mW\<{�����C|������
z�����z+$J���xW�Z�b:��c��7T�A_�j/���X���+�Zo-����������VWk�`�	�),���^����������_��^�E���Pm3�7Z�0)|<�~�!��7 �����bE��]��bs�c��\7��|���]N�
�����>����2Qi��E��l-�M8�?j���,�li��'�JO0��	��c����`�s����Z|�_<�`�9t�[��L8�H&�|L�iQ`�lH��g�1����d���.F�s	9�g���Q�'����S�2���pR^���y���A��2-�,�6�zN������_�	����W���}�*R�U��6+����R��G�P��{����`	�hT�I4��5�"��B������J�����N���I��y��:���WQK2�d����
�������_&��������h����Gc���m��s_�.���YC��83lr������#)�JM�=��
*k�����iAv���o
"�Q��1���RA@��&Q#��S�aq��\nQ��,24ZO�_,T�Q�'�z3��2��zkQ�%��J�{T)��:��j .t*\N�)%i{�MI)�Q���Q��"�E���.��A�=������=��K
2��M���W'�XI=8��2L7��~�<`%��P����/���`�%�NWt��[�EI=WCu[Odk�mBJ�Q[�mZR�.��'���*9<�UI=�M������:��z�O0������mo�������
����h�W-��"JvNJT��6��vRjmM=�5\�R�GW9��RaBJ)R*�xpu�39sMy;g�(�lFJ|��/��Y1#U���:���ik���P|P���Y�<]�M��3CSn�R�DJ� ��(�h��<��B�#�����14N���Y�bm����6� �I��9)m(V����*q�F�%���"Y��E�Q�����K(��u0�N��mB�`X�#G0L
��o3l�JL����:���2l�^;�i��i��B��n�~�#�B��
�^q�R���xQ\[���t
0R��-|�h=���r��U����H������[F*������[VG���Z�A���X���j{��j�2R���~��z�H�"���Pg���pIeg�.����Z]������f�_�i���yZ���[�{�r	;i�a��x���.#��Q���1[��HY��[�D���&�����&�����o�J=�Urx��}������`(~�_\\[Z^d�8<��V�����'�	Iy��&���M�����TH*��A����F���1�RqBH���(CH�	!���2���x�Y�ks��&$��B�8��p�]H�Yv��Z��s\���@���B�x��P��L�48���DF�!B���:�Rt����tP��	:����M������
?��Q�?���a=�����f�ldD118��5:�������b�s+�uF��`��Y��>�Q<�c�s������Y�����5i�{=�9�u�Qa0������{\Z����<��������+���[�b:��"9T�_b����l!Im=�A���[o-��lT�������-�3�8�:���I�,���^����lT����_��^Ii)6�@,�+�E�u��dk�{("��'�H�?��������O7y��,6��X8~j�H����B��lT<'���`/��1i����	&[Odk�m�F�Q[�mZ�����[�I����I�V�U����I{9�ml=�'+`��o���\�F�\x�l����������LH�B���:���Qw:��g�7�iBG��$CG�	����0�5���9�����hmW-����P�xd{��_E">*�j-�x���m����`g!)�O�m��J�$A���J%���e/�G����� �lIBhEi��L��2���,m�A��J�SR��<w���T}�"W���
YZ�bR�5���tA �X�QEE�R���j��$�&(�))FEb���s��5������}��k�u��t�L�3>o�~��V$e/]���}J*	H�$���t�]?�0��u���,��l=���c����A=�H���[o-���T��@#�[VGN�S��Q:��������U��mJI��nj�n�����Z��� U���/�����{tyMM	��z�]�I1J
�����^r�%l��]����7���RR� z)]l-����/���'���6������6�����z������f��������"Vzx�����.�-���-�R���%im��Z��w�O�l�e�����S� ����J*��(gJ*O(�,Uj����yF��F��X	�H����^h�LbvK���b{��e,���9��� @I������\�k_�����
J*O<���lO�%{/>M�
���D����SO���QR�[���%���6���v��U&�Pp���=����%���jT����\�������	�{���.�g�����L�
|����h�������Jhv���C����QR��0��Zo���VF<^�����[QRA���$V���w1��))`i&������b,�'�z��x����kQ����NI���VW T��\�x�����j{��j�2R������:��'$"U +�l���!��fi��E��"���G����|����*��q,kk��R�m�CO3J@�e������/v�3EW
e9��vD6��&\����&6�Y��Nj�HS[�B�g�$X��[G�l��-f�/"�y�?�(`!9�u��Oe���5���������;�����
�O�#������RPeBA��"CA�	Ud(�:��Ms4=��<+�8�NW�g��b�"����Y[6��Kx�F3�@\���:��G���o�O�x������s���u����a�K�?^��������
?�pO�}�)����V��q�\��=e-�gFv�4$o���|�I�e��=#^�v���=�[y�\b���n�tl 	�9��B�x5��
�o�^���y)=o���W{{�}YQ
������z�o�I�o2$,�t�R.�S�o��r��ROu8'	+����p\���[��-�Tv�	����� ._VQ�=�7�X]m��^mS���wE�/pE�C�����f!P������Y>~V,��e��t��WkR,��Q�Ny�#����(��p$'��NG	���S4��G)���e�B-��>V{"[Km����zm���P�q��EV��	����?���*�����qx���-���9Xp�_�O��L�7���an���n*�lc�;_�*P� �b��Z����5�D����]O?�?{A?�C
�O<�1�4��l
�Yq����4�'��"i�����D��,v�pP���s���(��~��L}2{bP,������)���\����4�;�m1��.�=�0�d��J��db�"���/D��V����Y�=Z��c.d�8
,H�D�z�y�� ���da�%)�*r��c��	����Epcg���� ��NJ�P��6��1�4����2$����Y���~>8/O���'�<��H�i�z��^*��m�P�[�:���ei�P�'�:���f���3�%�����H��au��4�By�������Wm�����|�������O��b2�O���P<������=�O��xT��U��3�����H�X�����X��	�6wu�M?�o�����2�O�&�<���Zj�������6���p�Mk�����'�'�E"�j=�'E�lU3n=�'E�7�����\�DF�8 �u�������s�rkr��*�{�[�9�ym��d&����762����OF(��W@�f�����$$���O����L��S+���?t(�*�v�p����w�������yG��%�����H?�{�I�J/�1�����G(fv1��T���Ll�A�~2�G?�<�	'R
}S��8�c+��9	H��I����� �	���i�)�(�*w����Z�lM|^��I�����g�5�
y��h>RM��K@����-��HT�3��Mg��;L�@�<c��z"��9i�U�����%a!�����~K?�#���w����Q.����2���U��mJ?��.��.�$�IL���^(����J�,����1��	�'r.� �@?�E[#��#�����@?]��	�+oD�}�~2gMrg�)�o-�d�bX'h��l-�M��?j���,�:�����*9<�`���r�����J	���e��?�`����=A.��`�|g�=����M
����Y\a�7����g���P�N�'��sle�';���X�<���S�db�h�3��i�<����%��w�KD
x���D���������h���nu���?��opP�"���u����y��J��o�i�U�� �Y�����
?�pP�}*��3
���CG����1r���P�&�u�K�b����Ej�d�t�	���)��������b
lv��3�kt�S����`���S=�a�></O��8�d8(�DVP�z�����:+$���ANJab�r����%a����Z�o9(���+�I�kug%�A�@'&M]Ylu��j{�M9({�/���/�pPB�O�.�^m�a�K�{�q�V(.��_(1�����w[�*��h�Aik���(�16]���]��3�P>3_l-#�2�	�%�'���6�������;��z���#GLV��bMY��'SM�o=�'�Q%�=A6�`V/$�
P��fF������B9�)�����S|��<��nBA�}c'CA�	���x�Y�6{�tOd1n�D��I���7���li�m1<��8������{����Db&�)�.]d[�Q��E?�1��0`��Lp�(�"*�8N����v2P7Ll�A�~r���x��]g�G��!��f���������!��e�#����"�A�e�V�B��Q��*��t�N�s�Y������r\��g��3(�}�/�[O:�]���#�'� .���H���RGM"��A��jv2��y	,�f�z�M
�����x��
����N��/��^k{��j��O��K�~�Kz��g��M���,@�z�PM�j/�R4���N���P)�qu�4D��	$�8���>#}�z:��=���^�*��8Dk��u�����V^��&i.�@�:og�1��^j�"����u�Z2mQ�X��l���~�?�|�QA����o�~��{]��J>�(a�(�MP�}\�9?T[�n?!������!���p�B�O�1�f�s$��c% �����G�p���3kJ���,*�f8Z��'��Y, ��p`�y�:����j�� ��������Dy��S�$��HG�*C8}ob�2���p��%��3�#��{����}�����L��t3�bR�� G��P!�s���F�)9�	��,��]�D�t���|�5�b�5�����I>e��}����]qn)��/�^��;y�'L�f���w1��%���
���DP�s�F�+Z�����%F�X�Z�o�&�K>%l^����8T�%���;�^��$i{�M	'�����$�rS�����@������}���������E��H
�����N�VA�Qq�kU��*���+���N�,�D����r-�\�|
W�����������K]l(->�/��J��w�#��jq�W�O^���E,��W�Ofks��	&�N��Rs�"mI����S�)!W
7����z���YK?�	�~����&�S�����94��v��dcY��v�].��VM�Z�9�l��Z��]���(��l���7�O&�c�d������<��S����m�1>�y�+Mh���
=M+��0�����{~�������.���B�A��i�}���X��V���h:s��`�P[�QH-��/�R�P9Z�9��8h6�wB>w�^��G��	������i��t�W�+^��'�K�S��&1��Y�]L�z�)!w�)��'�:��<B��jDP��Rv���OaO�#�$�[V��z>�YT�����"���U��mJ?��.i�.�Dq������e�,��Xq<�/����CV����~h*`��������@�����K��f�Z�G�.���v�X[�H?Y�����Zj��O��k�Y&M�.q�V]D����}ZX��������0������t�ZZ�����9Xp�����[qd�����'��'l��|O��Lv���[�8'�S��7�2�S��OQ�~�4#C���������z�g�k�����j�"I,z	�p���\�z����:�O;D���u�i���K�S������G?i����3� ��f�H�D����t��Ml�A�~��A��'���VM����U�B�Q,�� 3 -Ma�=.(� V���WkD�p�tr	BtNz��$�2��xi��/����5YZ����^��~���px�^g���Hbm?��'�[�vZ�~r�0���{���t
�O`���P�'�:���fw���x\Vk^=�o���g������Vg �[�J�@��/[]m��^mS�)�wI�/pI���$�8�u���\�P�vk�{po	�{�N�&E����o%�i{�����jW��
q�7����O��r�\��~�ZF���b�$�'���6������6�L�L��������I����	���!L?a��=���`�U�Z���.�rL��"��j�;������ �3��s�m��r�z���Syx8��iB?�}�$C?�	��d�'�f]<i�����0u��e8qRs����T�jO����']����V&uu�����9n�u2���V�<vO����H?z�m{A��5?$��O������4z��Ll�A�~J��O��>���&�i�*0����|~V���9d�!�'d�[����r�0L�O�}q������1����`��6����T�n���j����o���;�X��z���E����yyT��?	�?�$�nj���\O�2�z"��A���X�'�:�����j�kQ������
�u�[VgP?���v��j��'I��m�?��>i�>�u������u8��_���cxac!;����$�����a[���
���!�"2����p%7�����e�|�t�B����Xj��O��k�H>a.c��(b����`q�nq���`���u����O����y�?	z���/���K�db�k����
4�Ul��b3��u��)��g�)OX�,�1���� �<#	������L8e�����X�o��+��
��,����������hM��4?k���
i]�]c��G	�7X�<�N8�{�;M�:�X'��:e)�=%�)��N�������_g�4��X'z
��z�L���X��N���4w-k_�����t�zsa��+&j����3k�Y�
��'�!I����k���)M�_�~,=>�n�=�>�$��K�7��:Ih>�IDk�]L�z�q`q�D����%�0�6��EP�����z���[����t#E��-�3Pw���P��P����U��m�:���h���$�.J�N�.
8zd���x�8`��A�5W�Q��
V���	������?j���X'�yf[L��������J�!L�gpl-�Df�X�vD6��&�����f��4�&n�jDV����,����d��6�l��?��ack�:��gB���:�R�R^7u�m���}��1t���T�b��2a���.q�a���u*B�v�f��-B���!e!�P�>g�[�||�1*]P���I
� 
����T�9�8���:�0\,��5�S�����ve�u"�����	vZ�����t�����jW�J��2�����/4^BX�0�{�������u|�������s�qGd�^,@k���<ma�[3%�Ga��A���z?
��h��������.j�}EE�Y�]�yp��j��;���t*��oy�t�N �������s1��S���K�Pu)"��9������p\V�
�zkQ�%��.4N�w��
]Q�X������1pN��W��s*�=��<�k�)E:���:���m7�5�,�����t����=XRz(�E�F������� �P5��Q��&<�ty���7�Ac��������\�2�kGdS�m�7�Q+���&1�r�[Gb�����Y?�v��������u���>x��������0�U���Q�z.��S�P#��o)�1���N�[_�M��p����|��|?�����K�iF,p�*�Y���r�^�`W,p��[5��;$+ht���Pv���F���v"��4�������,�{�)�QY<���k0�s2s�W��2��t�"�(�t���D�&��_�v�O�9���4S��DW����'�J�S�L+a���\�T���J ������*y�r��`1.U ��|�(���2���p
����N��g���Q��#T���k��-���8j����<�����N4���m:��>�S@�	��u�'�:�(oS�X�zzKQ��X��^-Xy�i?3=���nY����<�Yl������j{��j�1N;�����O~�d��������4�{;�c�����=D9���D ��Tq����=D9��J9s�vZ�=D9���ZU/f?��b�d��Wro�g����e�v
Y|�o=����k����zm3��=>����*9<�������j=�'����'����z��2m�]0��v&�X��"�3�v���	��.� (F�F�W�3���3<��f�?��c#�?�	�d��'�fdC'�,�PZD�Y"�}�]3�T���VQK�;�Ed����x�k����i�!V�Nfh��c�oI���?�O6��O��j;�������,���%��v�?}ob�2��y��vy��G�����{$����p�$Hcw}�L��]|4�;�����b[O�9�������N4����_~yQ��k����v6{C���~*i����W� "�t<8/O�������I�~��������<{�[Ou8'�`JC
Eu8.1B�f���3�-�d��v���x���#w�!���U��VW���W��~2�]R�\��)i���'�SM���nTq�I����C�e�E9���Q
b�[������i�Q��.-��"1(�$$�}�~2��vF�]��ck�'0)(��z"[Km����zm��H���X�I�������3�[O�	�V�*�'(Y�	��
�����{��	��e�����"���������R�sC��.!c-�d'����7�2����OV�~�tR��h����7��,Ofr�����7������
�����s�G��]u�	�'�C�	�����-����D�����������N9_�3X��9SM�&g����Ml�A�~����%_V���<�,ow����t=@��ih�xV3G�����Mb���Oe�oS���(/�B^',u���Z !,���eL|4�c�m��s�O�����Ln�"G�s����'�sN���>���'��v���E�z���V��z"��Ai�R{"��yi ����[��-�d��'�{���)�tq�y��)���j{��j��O��Oj�Oz--^�D��.\h�}�/6��������c1,���g����+������W��>��B�����J��%���O��t��*�bk�*(z�"��Zj��O��k��?����z���c��>�k�����	��'ZP��m=�'�
b[m��O0����W�������q�b3�P�-4GT&u=�	_��On�?��c'�?�	���D���
>l��[�pJ�Y����*��a����t�����v�Ot:��9�����+��T�;f�����������F��F��}��<��Pb��Jq��J�<�x�������d�'�_�=	�O��{��X��AZ?K=`TA���l������h9@�D����T�4��f����?U�)|G�-]�F��c?�?9.����f��Ly�g���������������<��J��"�4�L����t
��;:9
+����pPb� N��DP��#D���[��-���L�5j���:rym���#m��%�X]m��^mS����I�/�I��'�����dGD�����=�n������M(�,�=m��va���{,�N�����d��	�����J�AJv����e,m�\�;"Km���Zzm��v%����tuG.m��u����v���.m��#n���3�'
��*2������fQO��&0d�[Ii��w�����u�?��^�u���E=E�y�MsiqA�@Ee}��.s`�)�.nq�1�(9g�1��~���}�.y�QO��b�[5��rj7X'?�N���P�O?��[������`��P���&6� �:��Y���Np~�D!D�{��-6�c���;�Y��\�(�\�b�X�u�Q����$I'�l��D��J�M��W=�2O����o�e���j����z�I������t
���I,}:��t�z��T��'�:��4B����Z:���x����q�g�[���AOtj�{VWb�q��O�^�A�s�4���������_��^sNVJd���������vi��n-A{���"���-A[K�>���W��>;��p����h�������Y�I��x�2�w*��&{��l*�M��?j��Mls	�l�H����0	6Bx������`Y�p����M������`�3�'dr�*���J�e����"O���}w&
��S/K��o
�)���)L�� %�d�BR�9��n���'z�(�8�
��U��
������m��{�dt�j�E�b��lp�oB}�w��p��UW�4��05:��l����Id�8�2H��Ll�A�o
��M:�q��/������(���M.�"�7�4���;�
�#XN�K&�����\\6��<����&�GI{��������y"��^���9�OE�p��"O�����	r
�No���$4�1���$m���y"��j�2��9��kK_�$P��X66����KP����.��JM����y��fN��
09���j{��j�N��+~�+:��
r���(�A�
djX������'��f>���c�1��H( I�A8��hm��z��.��"Ot]��9?��Q�����[Odk�mB;�Q[�mf�4�j"�'�JO0�'pCjODl����"0��	l.�O0�*����	�d�V��<�z0]�	����S,�A���g�rX�����S��O�G�8��OqB?E)���������[:xr:��?$,�VYu)�J�-�7B�QB���l�p�Qj9}�I�O��G����Jy4�7��8j��>����%Nk��Z��;����3�+2���&6� C?��5�M��]g�G�=Bzrl)��1q�'H�E�F�D�rAv�L����������H7@z���a��@<o|l��w�*N}�z�l��O5�b"��E�j�O�����~>8/O��r����Mbvm:?k?�S@�	X�9m=���L
�jT�����%���J�����Oq��,�xnY��������UIf�������6���}�4����~
���Q�)����v�|DK.Y��Q���Y_*���/%�=j<���N��~�����SV��}��I����K?�s�b����r��TB�*;K�tk�mB?�Q[���'D����X%�'r`#�+,�������,4�z�O0j<�*��z��`��/����'����z[�\n�~��Nw�O�Fs�W�E�S��O�G�8��OiB?%!��b9�����[|Pq���O,��U5���W������G]��Ye�LG�����D�p�5"0��+X���*q��	�f{Q�m��F?E��wmQ%�%�D�����
?��O��l;����U�=��������l���3$��I���JJ��(�4G�D����[jD;����f���vv9�c�>E?!����kg��t&�T��I��n=�l�t����l�:��Z��l��:j�A�I�*\aC����d����;����o����Ot�����8zr��$������$M��)���{��x����R����'�J�;;'��um����8s�}���4�h���F����v��R4~E�]��%�k�}�*C�]�3������v��k��$�e9�u^���	�Y|�^�����Rn�J>��l��E,������������&CB�m�qe�m��������`:����H�����)��g�)O��,��T|$���m7L�-I�+�f���)�J��h�op����n�F��3���\�Y9E��^� f�����7���.<]S�|�8-$�!F��*��"a����[�����
?��M��l���g�y|�`Z����LK�[����1LqA��@��������Nv�)����<�r�G��J�����B�� Gu���1������L�0�;��������TM����No~�oJ"�N4��B����Nu��������P+Q�sRWu��A�K�
m�mX<���M���PU��}�����z��q�S��������6%��}W4�W��pB��
w�G��J��~��>`�%��J\�-D����U ��J��|k�%�}B��H�mT�Y�.���%�X��bk��LV9ZD�����R��v���^��2
��q�M�I����I����z�'���bAe��V��O0�*p���z��`�����v�Z�U	�d��l;2�*�SL*����,{*������L��"E?%�/�i���u�Ly�@2�]�A*8���1���J��bf�j���?"��r���>-)�7���|5��J���1e��T�l;�=�K�q!���Z�j�������D���Ml�A�~*��;�"�m��=j���[Kq1�r�vu�����qA�<Kk=@��v,M5��"�3����'�2�(���������:��
l��O�a.����%�^����J��>8/O����*��I,�M�G�>��a%�7��!��u<'�����DP����J��T��~K?��p'H����5o�\^��>t��X]m��^mS���wI�/pI��������iE;���L?Y:��{�hGkQ����DA���%�������Z_�Br�h��n�]�H^�}�~*��'<Z��Z��'|@N���Xj��O��k��<a.Ya�v$V���D�\<d����c�6�����?�<a��6~�bW��N��U�����0n��f���/��2<�>��b�<��/I'��?x���z������R�t����JD'E��B\��t�S3��o�����C�[Bap}Q��]��6�B1&�h�e�a0�n&�{V�{���1�Ige�!gLqyjt�����Nd����:-�dw���DX'��_e�42�V{��"��"S�mVb�a]��p�U�y�d�8ka�t�B�k����e`���=���g)���jK�H�K(���U������v���Z`�����=�>���i*����'�<O�����)�"��>�M��V>�S"�	y9aC���R�1��Fh����j�H>T�h�����K�i?7QgY�|���!7��Lm@��l���j{��j�1O;�����O~
�D�A/%3n,]y7�( �k�{�y�����n/�,�=�<a���C$��1�'`W[D��������<����n}�gmn-c�S�TG�����R�5��Wm�����m�Vo%V��	���/V8�)o*t�'��U�<b�� ���	&�I�������e8����6��9�����b�L8(��ld8(3���E/F�����Q�g�f�G�ceiP��	q��,�w8���znoJ�Y}�Ep!~Z�S���u��2#Ek�7��lEZ�ywK�!'��
qOV���
~�����yw6/|���O�.�B��O�1*���Q�q�B
�P�a]a�!�."6�E2p�v�+�?ab9��n��}����~�1�d��k2�����\��w�P@G��j���Rg���O��6�A�S��0����z���~J�/�Z���xN��A�O�����%�s��a�%���Of���W������\8w�T�_lt��jz�M�'s�#5��#��{������&��,�cf�pa��q�>�V`E2����� �9��5~RU>d��C�*0�\T0�5F	����9+�{H�]�*C��3
�v�uD6��&�����&6Is���u$V��l�x��H�:��g�d���E����D1���X0�3��8����'�?���Tt��fM^T�����m�������5�dt���d'|���4��4[U��O�����L�R�q�P�
�t��ms��8/"D�d���%�'[$�&�3�b���uz�G}�bu�o����������3��H�Rs�lr�#*N�$��0������7i�\�	�����7eO����A"��y.4M��g�1����F4����CK',�`�t�5�����6�.�gi��N��x��v6k�g�|*Y!�R�^?��������k���$���$&d����t��m��I������d,�Q�����eJ�[��Z�o�&��MPO���@@
w>:����FW���W��o��=Q�<��<�(T���I��-�
�vi��nm�5��#}�~����>{���2>B��Y�����	�q�n]�]����&��O�����7Y�K@����R��o��V^��&i.-b���
9��
�2��E����N�u
�i{�6Q�v����8vJt;/U����o�M�'a':�T�#��C��){��"��M�&��3�d�&7�����j�:��v��Br��r���\�"��|FVLl�yV�}�68������4����'��`2Q~�&��	�����&7
;�������!Ms��E��C�.�$*X�IFW���
?��M�u����<��T�`����tS��`�s������hQ�]���;���Q2;�R�-`=g����mY�@�m�u&�gE.��rR�J�p���A�8M�s���|��kd�.+�7�K����7%a'���*�T{�)�^M�d��h��yKQ�s�������������Z�o�&�;��\�euF����TE��:�I��j�N��+�~�+z�+��
��G6)�u����}���B�����d>���c�;��WI�v4Z{v���m����`%�]��������Z��:
A'��=����	��Gm���ej����� b�'��3��b�����	+���	d�����b�c�����8'����t5������]�U�w���s���c�n1��'����7�2����O^�~�4��+M���BK1�Ym���:�L~�a���_�����xtP1�EQp�%�g���4�!_�2��d�l|^��K����N���������9��e�0����[��h'C?}ob�2��_W��K�]�Z:��������>�0��d�N���Gd����:��R^��uO��`7z��-b.3A��f���A�=���x'T�t\a����xf�S6Ne���z����+�������z���&1�cB�]L�z�)_�e�'�:�4B����x^�KkY<�o��~f��w�[Vg-��M�lf���j{��j��O��O��Oz�`�������Qt{��5���=�;iK�S�tM���� ������VbW������j���G����w���JN�����S*&JjOdk�m�?�Q[�mZq2(�6�>.>�.�`�;Ai����|���&��[O�	�������'��?��������G�����q?-�em���Mw�������g��I&�S��92�S��OAJ��cck"��g5&D��w��j�&��
/�#.#�T�Uo)������W�]�^���5�d�g��o�Oa<�4�Z��Io����PUvowy'�����7���)��?9o�+��=F���w�5'�##;@u�2��h�d�mgI���u~C]��!#��u��#��9����E�hO	���e�o�
�0Z���v6�z�Q������P�P2�Y�zm?��'�[�O^H��������N	y'q�
�C�d)�����M���(�z%������Z��-���L�m��������5(iw;jYlu��j{�M��p�%
��%�Vw^*�	���	0I�� �%���=���F�-c�b�:	���E(�.��.�7������s��c���K?�s�SNj/c���9w�$HO�����	��G-��I�]��Q�H���9���l�����s�2��P�]Y|���?��a����u���H'G;�c�)�&����w?��C&>9&D��6=(��XS<NH���GeH�8!���d�}�i�4����\�+D��;5�������T�3���e���c�2N�9�3���1��T����s\]��S�zB�`������6���0�^�>g|z!��7���)��bX�0�=b�5��{���Mb�B8A iA�A���t��N�U�e�NZ��1@> �A�O8�%Z�U�>.������r�����i�S�?`-�����{��R�)^:��? ������t��� ���j=���+���6VgEP�2���j����-�w�	eU�-�3X�tBr�xD���VW���W��t��]��\�k�'��H'�,'����2��{ �h���9w�N���|�H'��=�t6W� F:�V����]�)�I'z������\�<�p.�^�=����	��Gm��i�&3�i����c�S��W��?��T�'����X��	����b-�W{��`&�D�:�q��b�%��+s�qN��6����=�)���S��O�G�8��OiB?%!�qL3���4���5�V�"�T5�4D�p|�����<8��;���c(����w�����v�g�
�)��Ot��d�9��j�C��7�8��}����7���)�/�d�D��Z])��c� ���3c��w��	M� ��\�u���,���@��y�4�%2$�14~6��,O�B��VIl�v6��N%�5��A������V��������z�~�1O4�t��k���\/1^�lG�"��9��6m�hs�t�K��c���x�����.��!�v��8�iz.�m�������WM��)���{��x�����*i��x��X�<���,��9�u� V���
%>�����#F�.�;�Q�c):9N�/YA�[�.����ty�bW93O�����uD6��&������ms	�}�u$V����I��Oq|{�~�w���6�i��4�x��qb��Fn|�vn�cG?Y$�QE�S�1g�����<�����p�����o�2|S�fd�d4���o26>�f��0\N�V��igd�h~�E`��Q[\7��������Ku�F��fE$�z��~�7�1�)�Q�9v��i�"�,� fe��d����Ml�A�o����Sy�?�x��(�G�q���
����8��t�S��Y�
��~o��S"��*�4��y��P�WHze����(����F[�)B87���TWE���:�j��|���� �I���x��Z�b:�+�����DP�s����ZOu8.1������c��o���2��-�3�j�!����t����$i{�M	�|������x�Y,����@�;T��^�m��>`�9v�11���O����C ��d��I��(�=�P�D�����^��K:
7������e��K�VD:R��l,�MH�?j��M�`�K���u$V���,�wlq�3%L����?���)���y�?	r�	2�L:�{P-��7%���PJ:���|N��;��eB:�=�"C:�	�T�H'�f���4'eR�k��������55a���C�JX��v�L��l���9��,�cr��na.*��Icr�Q��
�����;zYYC���
�#$W�B����v�����O&6� C:��I'M�>�)!���mu��������Z��A��������v}���R�o�(��Bzr*�H9yh(�F���y���������[{�lv���}������Z����D#�+^.��>�������t���\O:�����DP�s2���N�\:�I'��5/F��t*{���UW�{�3:ni�P�S�r��N��W��t*�]��\�I������H'������u��,��@:ED�8�>D�H'������V{ �Xo�s�[OG��]���N��Z�;���!
��D���&�����f�i�I�M��W�*�c���?A��N�_�'H���\����	>�~���$��S��O9����{���r�rI��Q�8�
�����}fz�����p�?������z���T}P�D�L����V���t��K6D�D�B/4h���M�+�4���'��j��.����}F�N20��/� ��^��h��$9��a6��G��P:�A� ����_)��v6�"�C���l;�x�Q:�l;����w��� "JC����!S�A�r_� -���C��9Q�������6�i-{"�I���N�d��e�e��������G{N++TN����%**
PQ�4�>��bB���+�Jx�����%aY�P?jC�DNM`����~IE}�����8�[�gt
�&J�����s������������z����G%�:w�62��i�5�7!��n�1-������q#�'.^������l��d�(��%�o�Q���#<c/ �������*Y��V{"�Km����5��f����}b���
SR���a{�EL���������?�`�5l���f�������C��cGlQ���FD%�Z��.A����?�����()���l�()3���TX������.���FIA�] ,*��`8������
�E��
K�ct1t���W(��yi2����5-)��1`w()s!��:<=.��h�!��:��dO����ZmW��
;����	yI?�	�����7.��i	y.f�Z)�eh�5��x�_����QWg��0�x�>� �A��c��
�:%���Y-���Z�J-z
T�jR{�	y�#�����)yp�4�l�WR������@.���%�l�OZ��3;5�vT��kJ�|h@YU�-�+�)�<+MY����$)Q��m�H�pQ�/pQ�����
y�XM��@�H�\R:P�Fv)A�?�Q�'��%�7�����s��3��&'cu�om��(3�G��[���gQn��k�����k��M��f��P������lV�bCP[���<�5�T|SD����J	�:]��#��G�/��s�.���6��py6� TGk���.�f���&���7�B��qPV(,
r��;�\�^����%��V�����M�-F�Lm����������\�����^��m��]��<_�#��e/r���{c��S�W+g���E��=7��|!�����qP�����\�gaQ�����	�8��j�uH�m��(���E�P: ���������( ��(��������.�E�u,U{��6�4	���{|bk��y)y�^{��?��KaQ�+��mB?�X}L���(��v�����%�RGM"�������7T����A����;��x���������8dx���I�P��W��������_��NJ�=f�m�
���n����KV�{��0�����KGd��-0n����L��"�=�Y��\�Ym��cVb�&�����Q}0���2Q�TM��#���6#�����6!�0�(2�u$��LD1l{�����Q�����DT.����y�?�$/N��2^����<I���u�1(�n�:�`��WQnFD��d'DD����(��fJ6��w�5
�rj&l�����g�[y��������w�*��0z�V����]H�_�^D�>$��5"���P�#)]����Y"J!6����<DC	Q7�l�E��r�C�'��0Z��h<y��K��`�m;@�c8h� ����!���y�LB�y��;f���������u�<Go�\x���Y�1�����Q�W���L8����r�n��v�z�h/ZB(�t�^Xo��I��s!���<������D�<���z"��qI#,\Bf����&��G����~MD���n�Aes���Md�:#4,6��^6���D���������
bzQ�1��$o�9H�-Y��Q/*�����l���u)�E��/�G�d� �X��+gy�mB���m"�t;��(�b���'��+�=�����W����dTP�U�I,����QA�h�����QAA�F�����Qy�sP�?�����)q�p&Y������b�T�N�!�w�.������QR�gW�QR~FIyJ��:�6�������D�+8,�����4v,�Y>���U8.7=$e�Jr���rWp\�������Sw����E��^����t�5�U��Ih�lr&;~fx�]QR7�l�E����SR&-/��w��C��lh�I�����(����	�QR:���$��G���o����eXV�n�42
�?�S.()2`.�����5�'�1oA:���^���l����O�����@~O#�
T�]L�rCb,k�k��hX�zu\BwV���xjU t�zkQ����NI���x���5�e��s��5�������9%���G���G��F�QR(����!=b����dT�J3V�]���$�4��ez��� �(�kQ.��}�e������9��=�s����v���_j�QR��k�JF����Ab��`_��60S{�O0JF��CnsD���d�Q���	���	&�zSAE ����	#{����j����\	,��^n-�g�5��f�T��UB�T�QRAH2
s�����e�K�HKkD��a���*�����p���sU9����LR��BE&��2Hut����=c��PRa��RQ���1���)����@�N�&C�s�e��?����%��t=�����L�.Q�W}{J� ���J�n-���M��)/���]�Ee�/%���|h�O,Tb�����y���dT�'uV��^;��q3������:�w���������z}`�%%QP�)#�z�>��b��DP���(�����AOM�l��Mv-���T����������l���N+�Q�k�������6���?�����J��QRX���0vW��%J�U�W&Kb��aG� �#)��{�{��l��\��k�oSR��b�pg��_.()O;�f��{"�Km3J���{mSJ�W����X(�'))���F�	FJ�CUn{�O0RR(b���]�&�J���g3��(������]��HH�GesW�
����z�QR�gW9
QRqFIEJ��5��i��.���+J��8.�W_mc=�K8"{9���.7�7��LV����� 	U$[����B��PRq��B�gi'����OM_	<����blH�[Z��l�E����SR�=�B;9kU	���.��2Y#�D������
����%!��V�M\i�O,H�X�����H�����Q��H���uX%UJR�	��k���a�?D[�U �U��9�ze����>�^��B\OIYTlD��z�������z"��qIX������KkY��5%�ONoBQ���ax����1�����Qq�������6g��?������T��Am��f*��"xiU���,�bF�2�Bvv�i�s�B	�A$��s[�1@W�,2��lT<�G�%I_m-��991H�:";Km3.��zm3s�?�mZ?(�����?�����UX����X%{��T�i�����?S1'�$q�d	Qh-,����l�	�J���N�����x���~����fT��0�!�\gdR�8��@XT���_-�������0����������M�Q���mp	wx����)��u�������j<�1���6��,�io����(u���_�8��~�^�.�3�(|US�}
Em(#�A���;zN�)����BI�A<�?�8Z�*�����X�+dQ�	O���{��\�E�[�����S��2����J#)�������y�����A	�GY�����l��	]oHxy�c�����%F��X��xjb���d�Z��9���E�����$�2�)X��g�EvW���W���J����_��N�����k������b�:@��\��P�#h�_HE��(������� ����Q�,��mY�e�v��JC%��v����e��DFP��vDv��f�_5��&f��j�M+u$��l�(��J�O
�R�g%X���[G�l�Y��v��O8((!7��
�C|�T�<�
P3��^
(f:�sPy�A��}�,�A����Q�;��<��6RH�g_������9��#?55����3���"k�7��~�����#�w*�Z��o��}-�?��=����c-����m�f�BY(W�^�>g�U�����ag�/BT~?J������t��6����8��XF ���p�����\��x��z��j�4�����:���^n����Y���Z�����?�G.^��z���2�y����sPIF�<���6����aBe�#�e�z"��q�U��'\.s�9�%���~�A�����/�������q �����.P�s�������F[l�,�,��//�DJi�F����c�I����3��'�0�1�$��t��������6	E��M�7�M�I(r������D1�xh���G�����A6�KJ�-7Pv�9�JJ�GC�)�01�h��bVG,�N�z��MDQ}4�h6�J��KCD��8&����!O������
����(��4�3BI�����G\��~@���1����3\~�e�a7��$�z��D���cW8�����M�k�oF�����(�p�p���tV�����(�r��Q~���c��8����\S)����;�0�g��-��S&)}�r�%�8��WQ�)���7]����/�N$�
����"D��"����	�4��l��H���<�N7	��!$�e��P;`��3���������\w��J�x/m����Z[5,�2�D�O�m��
nP��Q"��E��b���1i!���'�0��I��m"}�����
��	6r����kz�&��"�2~]����(�g��
��n�:�PB!��K,�r���Pq}|��^;��O��[SF������(l�.�B{9t���@]Mf����=�S��p.jH�z[R�B\jw�]j���ED���t�~���:�� �[��6T'�^Hn2�awEt�|�VW5 ��2$��5�#$7�����$�;?�R�M;���5�����C�f����9��3������V!�� W�4��D�]5d!�F%��(7U�u%V��������_�SC�4hUF�n_{�R��}��"_�%����#^_����6Lc����}����8,Vv��|������]�����������uy���o������>�,���"�������]�����W���s>y��v����������m�pI�Y��_�;5u��6��
zh�w�W���Tp*�I���Q^�]G�0���S���U�W�B?��G���B��
�fE���6��� ^�V|o"�g2����0�4`�D���(��oS�����d<���(��]`�GB��$������!T]��C�qJ��^�(NM	KM�R�5F��1*��1��F�c����A,�F�
*�h�	V�P~�����)A��uwz	��b�=N%iZ�X����")��p;;*�Bl�7�l�$e�z��u2�\Z���3����`�.�7��&CJ��Aa[������ha}����,�/�#jH2����&�iP���X|�.�)t������
X��s����0@I�*��K� �	?��������������bYY�%�n7`!�Qe�n����B"O� ��Z�s���z�\�\>�������3�_���w?��j���(r�!���rP����PK��eKS��PK'{2K18sept/pg_part_patch/10_4.outUX���Y��Y���k�-7r%�]��|S�C����pODk4/[#�-O8�Iq���f4�K��i~�s%
�w
G�n!7��1	�����B�U�+_^~��������������w�K=��_�����/^�����������?����&��K�Yi���?��?���~k������D���_}�����}���wk|.>h�O=���Y���O�~�����As�������?����?}����������?��_����?|���^~������W�~���o>���6������?~������o����?���/�_~���?����������?|����~�����m >|������_�|���7��W�/���������~����������������g��7����>�8�U[t*!�d��xWl
9�}r]��\W'���S\O����_��>�������w@����?}��W/�?����{���?��/��~���������o��?~���o��_����~��w���������.J����U6�V�6���W�<�,l.1[��c
Vy��yc��.����Kd�?i����_��CC=z����
��?}����?�2�����|�����;�4R�����	*�e;6>o�.�/X �/�6���������������??����?�����~����d�����o?����z�?�k��?~��^�x/�x�)����,�^�
�at�C��1���iQ9��u�cS(6.GD�6[���0�[{�2/�������_x/����p�+�/�X���kK���Y�
�����=��E�=�-�6������_o���64�b�uG[T(�Y�O��������A���KQ���#g�|z�����6�[2MH�[�I��Kw���&���T�C{{����������v��5��~m���v�{�&z�����	��n�/����o����_�KL���8j��������|o�r���g20�l|	1*���%�Tl��AF[�
��7��e��2����6t��I��\�cr����A���?�������mOp�����@��\�����}xK�4���}��������|��P���}�6~����do�����z���6�������8������\�Wvt�����A��aL���IRJg��V�A��bL�6�BE2�(������A�$!2��*R���O���X��w?�@���dN&e]0\�8m��V��5s�;k��9t�����,�s�7��[Lg9�8]ZJ4�G���
��h7����A��M�:6���<�/\R4��9���K��^ZV�n���E7Im�p��W7��+k+��*9��}}�%�/\V����h�����v��k�����y����h���m0�E����4G���ll�������,�`���<]o����4�IW�����LJ�/4�
��Y�YW���Zj����-,��o�����?|�^�����|��������o�V�����V�M]���]cK�J��pC70:����'V��j9�X�%�����YY�C��/������y�����?�������?�
���������������C�=M���?~����������&Z�6}�_<�yZ��{)�����]*������_�`q��+t�9,&@7����d8=id4X�J�rP�gf��,;@�_�`����������P���e=G�J;9:t^���2���c��+����*j��!k�Lc>>HO�/��������?��D�k��_����>��0�C�����?~�B�{�4�c�����Z�`:������ ��|�NF�G��6TkEP�!j\Q��TX�_���?������v��m�6��6�}<eup��t�r�����h��X]mO���~�����?�;��z�?��������c�o4�<�����w�7/�?�?�������h���Q��O�?|�M��?}��3����������V<�x
�������O` �%�E��������<�#A����"���{��?�5!
�M~�s������T���-����
������al/5��m���%���}��?��?.+~]��
�j���o-z����U��tD6���@��Xzm�vYhif1���X!��m�PTa��v���~l�Q����?�z3 �tp��x��1�g;��}!{s�uqt�{}�7S�KC�
e_u��
e4�����D[��S�-�zB�jRy�����?+`	��&:&�O?����u)9���/K��R��Q0�)&@�����\��It���(�jd�����=g`~��r������mO������7��>�>�������s}������Y�e��Z�q4��pt�u���[��e��d�X{4��-�E�����.�4���6��Z��C+t��H�����ee�}f�-&�5i�Gp
	�i�����8�����S�I�T��))���2���S�M�t�b,�`��A��cM�X��;+>���\�G)����n�������}�y���FW�SM��!#e����
����T2���T�G]6dPa���@�Oe|.IX��s����u�Q�e$�������4���o�&��K�T��l����5>��]e�F���Fz���Tj�Q���k�$�%�-����f����Z:��{��a�6��7����b�&,�7��QH�W�0E����;�0��S����'k����0��r���;�
h('CCa��+�$����`�%����f������$���oaY����1A�V������K�n`�_�|O�NnO;i]�YC`��zdh��"�&bL
y;����!�7����=�p������-�=���V��������c��j�%5$.�����|������lL����� ]?LM�v���]����ju�������_H���
�ftF.��'��vF���Iyx4M���M1�F-'��	�y�I�fr�� ��D���������<B2���>D��L��tk.��G�$wp�[�����G~����T;�mH$��.�{.������92 4���B9uIo�����@��b��i+�ecrs����@*���O���{	�������������#��X\���w$6����;���F��Q�>wV��3���^^��s����(������4|�<�T���>���?
c�
2�<����x��A��J�����5�&����G�U���G~@y�4:��rL<�y�����4��M�k�_�hXS���H�c���,�O5��%�2�y�2���i�6\�]x�8�q��r�+_B��ZD���Un_Ys�Z(s�q��~�!���#���N��b�8�B�7h�b����6�Lt�+-f'�Kq���7�J�e����������hb��JA��mmt����L���#��v�o�e�)g�J�(W;���v&�{|P�L����|�����Lr����r>Q�/O���T�}z�.U�j�H@�s�rgx���
�(o����t����]	H���������jg�
y#���o��<����������E(�U��c���7���Jx�#dV�o�d6�������s���-E/|�7���#K���Hz���h8��;Im���Zvm#C��Q��GG`ul�;s��A�����9���SG�3M�
���e�o2�Y�6��)��0"AHR0�{��GE:���Fa��W�� ��o��0�,Dm ���b�3�r�GfX'��h����`���>��u��fL�B2�RZ� �`��`��%�K�|��Q�y#��4:���x#2��8,8�\������E��� ����F�����7�5v0:|l:j�[��#�"=���!�te��@q��K*��1!�k���������Vr��e���A���>��N����������������;y�p�����FZ@\�&0h[g2�>��fr��0>�-	H�C���Z:�����Kg*����z8"���f83�%	qf����T;�m����ex��a�Z���hMd6����Wl�������1(g�o+�
�4��7�]�QRG�GD��a�v�7�]�=��m���t���1r��Hl$�
x�wj�����R��pG`ul�;s$T��W;���Q�%S�#
��&�]�}u�����]�Ud�$��7��k��F�Nj�dO��_%�W�am�Q|���2�Q�FQ(���8h���H'����\r��F�%���Z<����16��������[nJIj<�X����}I���	�(�9jI�[�Q�C�n��U�"�X�5�(�/��p�(�����A�2�O�QKA�2�tA�M������/�-q�QF��K���Nj1�	RF�&�n4�]�rN�(�P�T�6�� Q�dj������N�|��xi�}��^:��mE2���'���"i�s}�K�����Fd2VCM�v$ �H#B�������GRL�B�4wbe����z�p����+�����d���vV��1�������c�������E���9����;Oq���w.j��!�0�|7��)����r��Um����Rj���c���e�v�l54�o$=cD;.������c�N-��1u�s&�X��������#
�����KG�3M�
-�����H#k+XY�7^E�G�l��0��r1:e�(h,9�����2J�z�I�2J�(�PF��k}A���%��VMo"g�`��f���X
qF��^��
�<���-i,i3#�i/�������V����s�6Jm����c���"bEP)K������Eh��
��A�6J���f�{h{��^�fv#F��u6y��n�i#��L!U������$�y����z��J��e�a��!�������F:��a�����mD'�Q�tnA�"�Q:tg���5+Yf�����O�;���y	�=mDH��t$ �����3�Q�(�d�]qOX��C41�e�t�������6���y�2���X�h���V
�p�eTE��Kz��Si1����\�r�{1�>����{�#k�3w���,m���F���}��t�Q0JC�x�Hl$�
h�wj������9������w��QR��V[X�3JH��:z�H�w�I������?4"����XV��-���qu(��g�6��W�h��Q�F�U�6��Fy@e��F��V�}��U�F��]��F�]�x���]�&���YryM�����fr�z;��*���g)b�>�(�U��T�x�6���O��V�!��h��
��A�6�O�6"D�6�(�A~��M�6�(��H�U��[U���<��7Bq\]1s[�!$9q�38b�N-���u��F���b��i1���P	���s���T[��m� ����������y1�e��
����������V����z�f6!Y\����>D��8Y�t$ "u�]F���Q>J-��'mcJ�6HWe�������\����X��Z�����c��(D-��(,�dq;��6�{_��Tr������^lJU��l1�5�
��M��J`���������Hr������4������FR��Zz��]���T;�c��#�uS}�9��wF�B'jG�3M�s&]:��G��(7�������f[g�j���i�s�=�$��k�l�Le@0�W��"C0��T��0�����tF��2�����Y�k#�����':H_�o-��B
zYG!����RH�n��yE��g��_��:A6��l*4���cM*C=$grP%��0��X�J8�x-"����qK�~�a�������G[�	�bL����f61_[��N��PUKt��+vW
�]K���c.�Z����W��l}����u����!�o�:d*�Z���|*I�����n�%k;RH�V�/U3�{���r���������@A�h
m��/����h
Q�w0�����e�B���^,	�~A��rOu���R�rp����>J7�[$����SVg�E�
m�%i�}�������W��t*�]��\��4�t��Omg�"a1��@�����7�{OA1�ED7R����n�&�����6CPL{��jO�\F<�=���2Pe�D��*}|���9��S�_�tD6��$�;��������Q�dN�H��
�9��%m����qj���w6�b�������'D9�A���(rn��8��0?.A
g)m�<�G�JE�kQ��#��c*���+2�:�����������������\nTM�A
�T���^]P���'�g	In^G�hf�ZRM�9TsKPV2�J�a�P�E��q*�pvT�a��v��t�-�Y$
{��cR��JK��kQ���T�	K�~���o~�:7M�t*���Y��	��~�����[�d!y�`���K)�y�{*�QSQ����f����e���(���MqV�U��&��B��o��\k�q�]]��n�E���Xz�)o�D��0=<���E	�;�$�s�N�L���.�/1��v\�X�|���v\rNX_~��E}����tK�*�SVgy���I����������^m#.������#o��J(�)�EA�����rM�����7�{.�����[�����k������R���E�����E�4-W��6r�O�>�E�o�^{:M�����Ci����YKOdk����z��^��%��U�'�'�J6O�q�x������t�1��e�'�'.��9pe�����*���3�c�r�\
Cbr_Xy�}.�fl���������8)���ld8)3���'�#�u���6N��Z��'��j����(�/y�N
��\����J�LJt?���i����.�v7a��cxR�8c:N
�aW
[c�Oc��zz��O���rN%_������z���d8)�KpRyzF�wx���U���n���C��Iy@���'$�6En����nK�*!�"y���-(�'GI-c������e�\Q'K7Q��^;��qDI��d8�K���I�L�=>K�gQRA���-���m����OI+q���'����r!���aZz"�{J*soY���w���(%en��U�UNX���� "��:��VW�Sm��!%e����
���9y�	QR��`R�v��]����t�6��d�t��z���G�|�A�+���%<����Xmg)���[|�-[KGIY8�S����R���z��^��2�����=�U�y��>��	�>�����R��\��'���b��oo���:IC�8c���T�L����!%�Pm�e��/��di7�s));�������������B��9s���G����h���=%��b�s*4	�����w{J���vu�[�CD�K�SR<>��|������
��%�2�k�^�^������8�lT\%���hy�PR�[Z��%e�NI9��0)D�F�v�em�$&e�{���S#�l����A�����x������^�M�t�6�]�P+����V����������������
�(�	PK��!S��Z��������(��()T���[�`:�SR���)������V����'����x�~��<�G)����D��)�3:�5!�������APR��W������T�����=��2����1"���������0P��,���b{t ��{1���"/��-b�X[�9?KI�}�4'����GIYK{1GI����R���z��^�+��������*�<Ag�t?��c�jO�	��Z�"�jO�	:[%\��87�	�Tp��3%��MV��4v�C"����D���+#e��^w��#�^���#����a�x�
����T��T��)��{�U~��](���*,�k�@qp�s���W��}13����*���y���<��$6��I{���Od��f��F��^(�X���Uh��a����6�=��"�� �:�,�W��o4�Q���X���ME����Y�i�C�\�����Cj8/��o,p�f"��K��X����c���C�kl�(�������Q���j8����[o��	6�z���lT���P�U��z�9��e,�|�����e,��Z�=������-�'�Q� o���:��N����r����X]mO����l�;���7���QN*g��99Ap�|(y���`w������t�����
�[Tt0�F�-��Um�e���E�,���{h|�5����>@*sdF��������6`�����6
���,������tR���=A�6O�Hn�K*z�O�H����-L�'F��1*�J�#�T�Ry�+����o������H�t-s�;������:�^���:�K�He�2���<��T�H�MBG*#��2������NO��t��q*f�o9{��<����(�:���X�O{]W�AJ��9{�������
Cm	����A�����i�L���"K�~�����s�\��]v��G����|��nq����\3�
j��T�����`iJq�G��$�
R4V��k�^�f^�.gc��93=�3�� �K�����k��v�QI=� �H��z�''�g��������()��a5�L�w0���1n��'������Q��A����k����J��$��a�x��Vg�NCF�Lq��W�du�=��jRR����������=�h] @*����d��`�9A�yE�;fhMN6�
v'#U����.��\E�;�B�
�y"[2'�}���{)�^�������2�j����R���z��^�H�-�+Z����X%�'�d���O`����RB���j=�'�l�b���v�>1�6�	�RLI�`V��C�fHI�8uLIy����(��&GH�%^u��%�T��0�>�:�F��d���3b����"W�3�N�5$+�""#�xv_�
/h#AI1^=3��V@���z����T8�����rnLve�h=W"��b�JM9R�Z(g�qK�~�������L�P�g@Iy��p�������C��k���=%�
���,��h� R�1�Eu�X=��i�B�P�()�'V�SR<V�Z�X������%"�@�\�����
6D;��,=���[� CI�C�8<��
QR"���w0���#�RR�'��QR4B_�N���v�����l����RR��l��2���M�?k����l�����W���
������A�����KN����8r�������r��W�*����um��l : Q[-�C�>���������]	�E�,%�����s�����Z���D�����;���F��i]:f,=�U�y��
&9�5k������R�[�@�	:[��'�H���'DI������������2V6��6*�r1�;�=���J*��(GJ*(�(EI�<�g�p�e�!>$AI���^V%��`a��xGI������J,&���()��~#�E�%U��M<HI�.J�k��%��-CJ*��zN�C
��������z���d(���(���GI���u"�n�9z���QR�����Q�B��[AJ�Q�BG$E���,�l���r=�����h�''��+45q�{����`�H��+W����
�V)8>K�'QRE���I���j���P6�w
���L�N���9,�!�N�����3l��>JI�[��E��:�"�\���B������W�����������(){��Hm#�����Y�<F:�������ej��U�;r�?A���5�.��h!����v����ti�.w����RRq����i��m-{J
I������R���z��^��]��jQ[:+d�o���LZ^�;|��d��:~������j�\�Q&C���(h��D
o��OYCK�cL2$���s��g�DTz�=N2DTQI��������cQ��X�Q�f�}w%�/���OE�C���N]�=��)U<|�n�X6[�{X�.4��{�\��:AD�^A�|�W"*������T�Js���v�f�!!�d���-��A��J���T��2"�#�J{+v����)H�P��K�og�� U���$A�go�}@x���P�^RWJ�J������XY�{�Xf��QVy�86�{��)KQ��N�@l�@�=�DS�k�`:b�`��DP��(�J��&#���F�J�]P��sQ%��-]�c��.���D����������^mC"*�wO�pOc���"��.JM[�Z?yqo��t��R�HL�TJY�KJ^�X�����=�����5��5��"�}��J{=s�U�[K����2�w|��l-�
��wj���,�a]�eZm��d��}�����tV��f�=A��O��*���'(~�>1���	])���j���A��2����)��(Q=g.%��T~�Q�2�TPRY���<�-�3b����(�SR�W���Iu+y�c�����x���R���E_+y7����Mn��[��dP2�9�T�))��7)|Tb2<5�"��e�D7
J�qK�~����/Pb�^w��"��*�&��&��)��b�5��
�^��&�"����$A�}��$�a�����P�Y���]����E��2����[�ju���-%��a#k�������D�s��,���~w�Sm��,=<��DI%# j�I�&,�I����oO��Py��DP��,����A���8�����>JI�'�Gy.N�{���|M����Pn�G/L��������T>���7��RRte����(]�����e&����r��
5���b��b�;!��ED���
vEN�M�:+�N(y"�YJ*�c�hZV�����O����������R���z��^��2i2����&#�B7O��'^'����J�Ep??����>Ag�4���2%��O0��X�0%e�-�{.�q�T4�D���.#�lM�*ar�TPR�UG��PRe@I!J��Y �g�$���W����=%�x��o�T�INB�x1.��3)e3�r�SR�K�Z��nz	<O��JOI�ed�cMZ�3���9a���EI�(%���u?�PR���z^O��Bt\���rM���  ��WQ#������NrQR"��[MA�
�2�	�
�E:�[]|uSp�^��aR� ��q��;B�}������z�����*w|��^����>G������s~��
>����>_�����z"��|��iis������('���]K�<nu��C����+)s�\�1���jz�
)�r�M-o�M=N�^��B*�=�g!��hY1R��X�����l�h�Cs���>���@����S����kh<J@���v�s���.w����(1[��Z:"�Jm2��Zym����`�#�B6�{�K����['�2�^eS��Vji�����?��G�G;[2��,$���*�Z����P�b���Zr~x[}LB���5�_��P���P��"$��r\T��_{;1Z�O�P�����n��Q�1D3�#�2������FUJn
���P��qX	qXf�����0	eu_Y�^��������5-m��.h���{YL�]����P',��A������$�.��z� U/��<b����V]:�����lQ�8�������	:M�P���E�6o��������Xqzk4����:��5�����5
�p�-��z�����[��-?��;>K�'qP�H����PI���t
��i��F��0���]���XAqsH1�K��}nc�y.������ �����"�Y����t�����W�����<������!�"=�	�xtU�'�S���������JP�b9���0�cw���_s���+�$v/�L�'R�|v�7y���$���{������Zz����f�����R�1�^-���fTR��PKGb�l�;�(GwH~����kF�3mHj���w�Q��H/����zz!��2e�V-�������*���Y�[�XHs���~qLD�W�c#CD�e��(�g��s!��C&y�=�E&-�5S�j&O�Q<>|��s-](Yw�����*!*��y������O����#�tF~�JD��el�(�Ko�LvS��i���YZ��e�
e���5��K��u��[X�S��DP���PM��R������=U�*G�mD��[Bwq�n�bD�z�@��Y�%���].��%�B��\��k�Vm�����Y�`�zk~H3I������z$��
9��[�`:�C��b���>��RZOu�X��l����Q�	�
B+�)�3��� *r~��]9�Y]mO����D�9���7��'�Y)"���/�R&�J���c��i��Bk��X�|/,��kF�����C�$��kF��]���	�t"jwAwP�]�����^3
���S��l-�
��wj���,�,�y�=�U�y��>	���(�&���zH�r�O��?Ag����=A��G�d���klm�i�fx�5AU�h��)��%�%\+��v@I�We+CI�%e�(�P��Z�<[��r�6��J}�^n��\^5�J�D����QEx����u�1^J�������x���{JJ�~��6V;�	z9"*�}_rf��iF�,���u?�PR��2��O����9*�A;�ib
�X�Ib7d�����e����u����bT��y�����~���C��z�n_���-���r�cNW;W���������d=2�Y2�{�4�(���=LO�g%�	��ID8o���[��(gP�"3�2j���MF��������!��zsQ���-8*9�x�V���/��9J�-NJ��jrR���j���z�I%��z+�1J<3��7�����K]�$�b��Cw���e���J�������e��t+0Y�,#e;s�r9�X���� �����Vj�Q���k�F�=$�#.����[f���>w����	X���#������t��x�@F�j�:W�b��P�nUC����9w��,gj{v�(>��r���;�
x('�CydoA��Th_cpZ"4
��������-t��L�|���xAi���L�3B��<��/�u��kN�	���y��vK3Jt�.��^#!]r�oBQt����������Qf�PTH���*��v%I\�����nsHe9nJt�ll��z9�rFe=�d2� ���s�1�Y��UYg� �p.
��Ad�0��v^�������S���Zo��&��(w�
�_ 4*Kh��z�l��;�N^�^��U���v�.��KYz"��KG�Yf�{sQ����;�C�Y]A�;��\b���`�$M��!��{��
x��)zQ(2
@��"����	���7�{W7e����u-��r�>�"����w�1�Cw�?t����9�@����^�en����qP�e2�M&��M���N����M�g��t�t$V�o��<������['`qU_:��{�i���Z:��G:Q	K5
r%��B��a;O8[\��/M����d��Rb���8(��g�e8(?���E��W���oaD����9e�|/�sP�_�g��YrM���Kt�^�����s�cEJ�3�mQ�!�O��|E���+�Q�`�A�+dgR��	�����2_di�2�~,����C7 ����e��r��L�T,=���|2v�8���}�h�$�T�����}��R���(v���d�J��}2���b��_����k����u(V��8[o=���$�?����I�������g)�|���L���<`!�\���>?/����G����b�4� �Zo.��$���B��'���=�]����l�����W������S���c�9�X(�PE��s��V�h]�	Er�p��Y�76U�/��T�c�bM���
v/�`���NE+�}���{��B�{��tLT4
�j��l,�
��wj�����P�2��U�x�
��DC��n�Hb�{&
��5�_;��]4T�JK�:���r\|-����EC9|NEC���py5�}L�rn��k�ow��^���LT�R,�F�T�<C�k�������^4�����S�a3���
t�"�xF���$Py}����<�
��������B
����r.�!lQ4���K��P�kV^�y���E��� �D��3Q��W,����	�WI��V6[3S;|/�1"�5��5.)A�UL(��
C���p-�d:s��7�X���X�:;]-�\��`�U�k��^;�i�#"
�PR������M"DT8t����h(�D�����s>��S�j�������#��X���|"����c�<��Q�V5�G��SV%���#]���������^mC"*�wO�pO��������Fr46V�����W����7����t�v�v#�4����AYVj���J,�����T�
�s��Y"*�K��
�4y��tBQ�?��C���ZjPQ���kQ�,���z�d�Q�($���tt1F�b!�'�i���� ��O0����UJ*f�B���00
�rd����P������KI�%_u��%�T
��y^����j[p����'7�b���{��4�:��(�ES���W
L#�g~p��u>]��)�Kx'(��QR���7�r��P���B�h(�%�K������5�����z���d(����`�G���UE��B�S����"T�$(��q]��"b����0XohgI��x���]�&	�A�G�`�_��]ZKNf��V�#����;v�X��P%pPE��=\��[��M�V��������I�T��ID�Y�����t���g�cAl�j=�}=�L�j=�}=!|�����('o�T@��SVg4]p��9�n�e����T��m�I��~j|~�'��b�Q��bW������LY�����a-c��S�
k��y�B\�!&������L����2A�,'���1������
V�2P�'���6������6�L:^
��KOb�l���O|���T��?Ag��wv��U�G�	:[��+3X{��`POe|M�������Z9�Bel�8p�CtAYE�V�b'��Tz�SN2�TpRI���yF��3$^Z\~����sR�zWIT�W��kQ8�8�='�����/�z���)��������
$4J'���Y�T���BQ+�ll�����a��Kv)�B+�e�����N*=�����O�sQ7�*-"y�WI�2j-;S��^��R,m3a";�e������k+�\lpf�fz��G�w�LonF�����D7{N
\z-�?h:�qR��C��8��/O.�4������z'��@�&�Q�w0����B��cT��������e?ZOu',�,l����rR��5��>>au��<$*|����!N2���jz�
)�t�MMo�M=���B�D�=�Y}�rj>���)�&�@�@O������]���(Z�$�>�n�9s|V����umg�����AL�E\l��^��.�����#���6`�����6J�����E��N>Z���z�H��q�N�M��������&J�,��t��"�,-E��l�(�)kxk��Q)�@�%'�q2e���9���Qy�B�W}�,�B���J��<#���)�+K� ���z���`��J�L��%��/���Z��<*!H�0�g�����h�9�'X��GF�xK40���#����X����B�gS|��u?��P��,������_b5�J��Y��21mn�BYm�����9�TJ���6�{���]D@_vm����Vt��'���~����Z����(|�z�W#�C�P,��W�z���H�P����@`T ������/�w0��I(�����a����I(`T�����G���\�GI�|�B�B8eu�����"ssm&Y]mO����,T>���7���PW>�6���g����h�������� �^X�([��ds�`w�z����j��`w�z�����
vw�h��Qy_C/f�����F�a��#KOdk�m@E�S[�m�FK+6�ZOb�l��O��U������O���h�'����l`���	��'%���^"Fe��������))�^(�k��5�}^�muU��DI�%U^u��%U�T���Ph"�6��]����j��()�W����I�N	��()T]���oU����K�#J
��+�i����F�i��U:J
��o�g52���Q��i����e0�����wB�����J�<_?J���Q6�R���*��j@�[��-�\���)8�����
m6b�Qu��V�
�xtK��!�y�[' E�M�����6T�.~Y:H�����!��zt������U�jo=�oy�S9�r���sR9	F�P��(wS#�3�P;)z�;r�����Qp�M���F���&��_zsQ�������x��J����}t�^'h����T��mHI��njyn�q���E��c0P����@7Q�'�z~QR-.���s���]�xK&��I�z���S1���p��s*�4�gb���J��n�`[���!��K��[����Jm6���ym�4��r=���X!�.0����KGo��T���#��F�z~qS����G�Q�_N�s��3M'�;z�B�y�5=�&0��d�V�lE��!E������B�_{�B�CJ�P<��XS�977��k��,��Ro��V[ V1���,��\M�C��kl+t�gb�Y(�:�m|-�8�t->�q��>=/C;~M��pGG�Q5�t@*sS1���u���DX(����B�2_2
	��pn	��lXSa�r��8�6p	*��U2����$��(�6��*��c��{,��7��|_,=`���i�KoU��n�B��U@6t���|4�Q�����zU�������w0��Y(`�lT����Y(x�!
l����Y(M�D/�������P�#�J�b��:���y.����k1��j{���6b�n ���?��X(c����5����bI����7�}9=��h^��� v�d��A&�@g�;:{>����NVM��q#@7�$
���{Q�p��t4�����Y:"�Jm�4�{���F�y:�����ZsW��+�G�
��#���N�j���8~WL����8:+��P�C%�����9���;YF�jd����9���P�h(��sldh(3���
���}��L'U��_P._��_�������P��Y����;m�x���K|lT��J����������DC���rY��������-�WY��u��P9_���h��-��A��2������������!N7��V	���N�CUT�b��|o�y������jD�;(=��n�yKlP��P<V\"��M.=��.��mE�Xd$p}y���Z���yH#�%��y�0�����,TB�y�����t
��1�[Q�j��,o�7"�]~W���������P�	��0�)�3��!8'tC>�
%i{�
i(s�=5o�==��+�ke�!XD� 8=���{�����4�������y��e���c�<���,A�^����A��Eyv+�}��2{"��I�l-������.��j��ZjPQ���k{E���6���X%�'���	�u]���	�?�	z�rd��E���(�/�adX�*E�Q�X�.��p�'[u�J�x|h���*�Z4bw��%e_u��%e��.�<{��<�v^D� �����^��^�rz���D�|�r��T%��Wn��l�����p9��/Ut�/��K�Z>�	J�v�"0���q�1��(����s�����<X�3ef����d()�K�_(+=���LQH����J9��2J{J
�lq���D��[1���<&l�����u����[���EF5!�?i�(��������;��R>���-�X�bz��\m��������_l��I%���������;�����<>����ZOu_L����X,��A���3
�����rR�.2��r�[���#���[��IVW�Sm��!'e����
�����jx�.�)��}�8����{���y]�k�q�QkC{��Z(�BY2w�����~��V��6]g�������������)��m-'e���?y����R���z��^��2�����M���*�<Ag���������J�r����'�{[��qy�]�'���<�IH��&���G�z���#�*����[r�v.'���e�]�������R�z���Z�M�K�M�-�x-�a���K��W?nQ��
����U���Y�L"	�(�y�����-@8js�;�I���J�lh���0)z��
�l�L��naR%]������z���d8���0����l=��P�W��ra��2z�x�>[_Y�gT���V����U*��v��
�&ed��a����S;N�a�Z
2Z�g�%���<�"�W�z�#��a�T$���X�������1�������$N*{�8)���U���S����3��=�}�F8�a������x^uC�	�MB}����8)�=����[�����?�'%i{�
9��E��[(���8���QX��?4y��`wqR�U��#����������E�R~���&���F�97Pg�9G�������v7t���G[����U���'���6������6�L��~\��'�J6O��'������^�<�����8�9�u�����n����	�����z�k����`�(u�dO�1����o�k�^	��5��<
8�W���$�I�'��R�0����y�56��kq��g�9�����V�p`��$'U���Q������'����= ��D���e@�=��J:��VZO�T����K^�-����n�����_di�2�Tz>'����B���*���:���K��NU����/ ��Hk�a!���������}�*��]2�
���������c��RL�Fy�aR��=K��^;���F�{����j��!o�}��������z%�$R�����z�w0��{��j���A�R��U��jO�K�stWL�0X{sQ������E&�	�+���/gI�f]mO5���������-~?d�rb��,
o��Z����7�{��D�lUV#c�Um��5<j��iF�4y��@�G�x��J�H@�e��^���������Q���vjGdS�m�E�S+�m`�|DPj�H��
��0	�Bkp������ ���8��D}P�rK�:�(.��\tB-=CP��E�8�J. ������@k��\���^�����E�]�������BqQ��Xk��T���\=�����qQ�spP��n��KF���qb0��R,x�\�[�E������w�
�W9�T���=U;��EH�P>�D�5�R�?Whm��w���L-���� �A�_ W/��qQ����%��JieHJ�&��A�r��W�{ie��Y9(W��8���6�v��[����A��GGw���Bw��l,
���aQ�>p�����HqP�A.}���4��������YT���|^*.�����Ay�h�/��'��qP4B��l=�=���������>�A�[X������:�#n�G0�5G� �$m��!	u���{%��k�!�R�M�	�'[��S�����B�ze�#��cw,T�k>C�DL��6��x���%�*'o'�����>(
9�GK/E��f!��'���6 �����6�1������X#��8m��#�����t�.l��K!�����5e��w�	FQ��R���y�������B��C{�jB{�B62*�ka�!^d����]�!����*Bd��d�<�*J��a�=<ku�N7M#�9�����x����#�.Eg�(�O���yI+^�V��U�$���ZN��hJ&����+���qJ������2���d���t2��+�2����*��=��B�"-\*I��$^@������r���(F
,��eo5rJ��i�;�K%
�baV��H��
��84�{���~LF����������� ����9����FI�Q�?[��Xz�9��V�qAM��%
��1VXQ����
�.|Ko.��dT�D���r����j��W}4ud����6$��W|wo���q����-_�P19����>�Fn��)�{���Q��ay(�������T�#�7�p^���U(����:@��m!�V�,U�QQ4A7M������2*�JGqOdk�m@G�S[�mX��&��/=�U�y�����7����J���'��O����Z�������LH�D�mY�� �h[t��=V�.�*�A5*�L'|w�JHy}LH�W+�{-BHy}LHy-CH�y�J�)x�[[������y���W���Dw7)%s(�1^F���d%T��|V�D�������f��R^w��+^�.��U�F�T �KH�N^��@�R�s�3����u?�R�7?9:*])-= ��Uz~��jc�dK��<�fOH1*������r!�(GH1*�k\s�r �#��{�e�^v���V"����RA�����T�0C�lV�'��z��,AH��:�~N��s���h&Qckl����OH���\����	����������z�zsQ$�n'(��S?eu���{�T�1������^m#B������B��cB�d)B�v��0Z��s��/����`X��%t�RKAm���#{#��4��tQ����IBjAw���`k�	)�����L�Zj;&�����6$�
/�i�"��y���J*��)�?AOH%��/������U���=A6�`#e��K�\���J�H
z���)H����\�59��(7J��*���PRf@I!J����h]��n_i�gZ1R�^��x�k���%��e���(���h}����q,���q-�Q*�������l�	��k���!��`��f�F����/Z�<�"��w���d()�|�(�����4��U��d�����
j���Z��N�R���d��x!�Qc
��X�-2K��&��\�y��B�U ���ZJ�Am=�\��{��6�A
j�a���:�z�!�L���k��95��%�i��z�D�R�w0��k��������h�����z�5C��j=�}m=�0BY����>JI�[�*@�SVg��Wy��J�W�9VW�Sm��!%u��5���Xm��j����	Ils���������w���������&|d��Jbw���c����Y�9J`��������gr��t��s��Mi=����%�Nm���e�dFV_z�d��}�u����u�y��JQ.8r�����Vi��9��w�	F��C��%��@�=��1��Ky~����A�F��a����Qn���U���������Bi{<���g2�VY/oj�i{�K��%rX�%�������jr@&�/���Qn��Q`]m=_�������L�>^�;AI���&s�k=��r�� )��`����%�e�+�L��	C�~�a����������0^%2�Kcm�F�nP��TD��:�t,-���1;u�H1*����D��
E3��������7�&|�mARH�����!�����q�cp#F
��j���OifI������z#U�#�IL���z�9���%W���A�3R�����'��g�0����zsQe����B�aw��XBJ[T��
��Y�����j{�
����[��c^.h9�6
�`�d��B���7�]�(�������������/�e��X�%9�^�&��,�0�Q�>�H�=#e�*�`k9���5!��D�����;���F�I��c�V?�`?x��>��i�tL#b��'���pCq��"?}�-��^���q�)����=�s4���?r�cN��c�^���[U����jN��1#�^u��#������y�f�g�o��r���+���Xs�C��V�K�Q9��������K�<:����}e=��+^A����M����`�\_Y�h�$�PbD���B�$.����"WP��z���d()���=�()��?&�7��L����9��JI�nAs���.���������n��
�Zx����5-����%:���J��()2b$R��z��>��,d*&�z�)�H��;���/%U�0�����w0������Nu��gCP&F���>q���:�<{��rR���B��)�3�.8�3-q���M����j{�
9)w�Ouo�O=�5/R�{�dh-��5r����,�
����H���e��jB"�{ol	��<r�����l��~������������'b�������V}��t��)�^�ZOdk�m�I�S[�m`�<��K��$V��	��Y_gZ_��:�<��JQ	^s�������9up�]�'���`vYK��	D��4��K	p@P�{����m������:�^���J��PRu�k�E:�J��tG�3R�������D�=�3�,�n]�V�k���!%�d�^@
��u��n=HG�=�S��}����|@GY�U#����RKLA�� �����u?��Q��t��W�+Ft�
�������!�]�1�����`�m�ld�,��5)���f��D
������y����\�)�����5�Oh����}�c������U���{1Z�����:����F�$�y�O'��s�=����A��S=2��t�]�L��=��$�zsQe�����k�����u��	)Z�5�'[]mO����l�?���7���Q6IEH��L��zp�
y�}Te��]9N�
b�H%�ZhG�rt
�EH��!F�DW �9?�F���9��x���Rt��_�����R���z��^��2->*��Q�I����������������R�e�,�8��q�����Cls@��O0`��KW��R{�[���6���s�1���"u.zr�T�Q�UG9��Qa@G�����3��sV����y�F�r���k���E���D��|Y��W��aDo��m������|�U�\�������wg�1�������0
�B����Q����[m������ CI��'��p�&<���UB��^e�7J?�3-�dTQQl9�B�{�+t�����
���
���:o'��w���
�o�<��%��5,O.���D��h�v^g�f��2R�v����S{���4��������YRVBF*i4�z�) #EX��5�yt3Q;)�
�/��������H�>X�e��E}��
�R{
/���hV�C�b��>za�����6���y75�7�8i/]U�mHI�Z'- )%�������h%N^+p�� �Q�=�����N!9J�)���i|�����v���]���s=�������
���D�����;������k2O+�$V��	zJ*,aKO�	zJ*���'w��RO�SRY��y!��������S}�S�W9�'=L
����~��l���j09@*����e�8`��P�����[RNJ��Tq^���ob(�V]��	z����>�AR�IVE������i�6�+Z�,Q��)H!}~�sm*!��(�@[�EC�mV��P���F�� CF����E�t=z����n��EN��$L#����&�`|��=,���xi��%^,��A��(������lM��.���d�O^v�ko9��MC-��j����l�����Y�JF�C�8�d���9ObNa�����t�/[	,�z��'��/[	���&!�}�J`9�l�����Q��=��Y]q�L
� ����k']mO5���\T<���7����.���x�`#�����.��
j��B
��AF�-7G+R��*�
eA��2y��@������ d��R�gy����{s���*{��XELG��l*�
X�wj��
Jz�\F�������� M����I��K~a�V/��Z_1�\��v�������QFUl��*f�	�o���Z�<�W��t��Wwx���^������S����<#n��sn���gwIUf��gq��u��J���-1����X��]'W������^��z�(�����������Re�[Y_���d�T��l@���A%7�<�YZ���~	���-�cx��������5���%�Q�-Y��5���<�m�=k��Z���e�as�_���,�	�����w���j���w��f����o(6�z�T�����#���AE�'Q��^�L�@���k�Z���v9zU��������v9z���u��E}��J�=�!#a��������K[|�lu�=��j�P��{���{z��T�
���|1��M9i���`w���F.g��d��`w��N�m����v�B���Y������gb�%��>G�[�����D��)��'���6������6�/@�<k-=�U�y���B�����
�'�k
n��	����	��(�BDyyg&����*z��hC.-G�7�BjJ0���6N�U2*���Qy@I�W�,CI�%�e()�g��S���.��9'��x��*e�U�)�k����^���Y���Kl������yy>9G���
M-i�/��<HI�>G��m��E�N���\�N���?6��o����X���J�qK�~�����))�.��a�*q�I�L��L�Ra��t�� K��6s(6����q0��e��v��BZ]^f�5�[�A�._W��QR%-�w��?p�}B$�����"�T>t��/@I	���$�C�z�9_�Xz}���%�1�XaE
"�{s^� �Zo.���T�PR����B��-X%��=��j{���6d��y/5�/u���Pf�#K.���g��z���p�|��>�RzE��}��u��aQ4���f�`I�,����v�=U\!o����#���6������6�I��uR���g��
��0�W��#���N����_:��]�b������?��
A����A���/C
e�bA��Cu��'cF���TpP�U���pPe�A!
�\�v�~����iy������h��Js���9(��T���v�mB$�|����W��k�6`����J�A��at�~�Ayr�3���3Z��-�L+�A=ni�2T�8�r���sPt�����F�UD��b���+*�X.�W�$�/���=�
%���PJ]QS,b�6u�[LF����Ew_s��:��*���X��*+q�.�8�Z������N$5�:�����
��O'��s>�X\j����9�h3v�������&(������>�A��NT�d����@'�����s�L����j{�
I�r�=-o�==�*F��2�.���dG]��.�
vE[k5��o�"�}XYo����6DA�.�"�R�>����S��Qe�����������;�'���6������6�L��rjOb�l���O������>,��`j`�����Uy	i��&����T,	�R-�Gr+[�
4��aQd��"�:������5�x�}LI���Q�_�SR��PR��"aQ/��yN��V��8��\<�ds�j��W�����c�l�[M��saQ_�m>�n����V!�qJ*���r��U����
KX:���H����A�\��E����J���gSR�N��,vv� ��4������Z�����H����da�(),[��U����m�NT��SR<V�jh!%A�m*%�s�E���[�9����9n�;����-PR�����z%���t9�����z�9_����o��"\SQ�)������'��Ot��7�n���RR���F����:�gc�`�����������^m#J������#o���Wdk8��F�
��EO6�
���ut��Dv���U���;�Gc�M��[?y+�`��\|��H�����^d�')����������,Jfp����Zj;������6�L�;h����'�J6O��O��bKO�	�V
\������m��W�z��`@Ie���Tb��FI�5���Gmr<�G��z9��m��O�����2�:�F��2J�QR�g���<��K��3���SR�+���x�%U��`�%������i�����l()����[�>jTj1���NPR���^Vk..b��P<����G�a8E�[���
U2�z',��A��2�����.`N��i��i�e-��s��J�fH�7Y�\r��$�:�e�5q]�bM�%d;�����PoFMz5����%��F��1��
6���))�`����}w*3m��,=<��%�$(�!\f�j�`:():N�s�ZO���h�P�V�'��QR�W����<��RR�FIA�^��:(5����X�^vk����T��m�H��^�y^�q����*����5�4P�hYHe��C����gA�.�B#��2�QTO{���H��Zh�O�8k;KG�.�������m�K��TJ-�]���N����QOwU�����
��w��*A��t����z4�`l0G�J6U�`=���s�����S����QV*�a\T��j	����U�)��3o�*��I�P�U����Pv@BY��3��1�����4qIB�<A�����{u-<����x|��u��:��J���tlT�jD�:>����}����E�=	�K��Z���P���ZP���[\T����E��� CB���P��4�������L����K�����
���bTT4O ��dU�z;1inOB1jdT���&Jh�)IN�|�b���pViA�I1�����������kW���^�Za>T��S������azxz=���^��&�C����s�A1���A���IZOuoO�5�KE��\�GY�����wp�����*��h�Q��2��j{���6���y����t��7��2�tP	z�.���.�
v��Q�$aM���nE�=U�>a�=���,��iF��Ha��OE�g�(���J���,������#���6 �����6 �0���_:+d��Qy��#��'���:��8��D���5�j�:����;Q%���BDYg����t�%r$"m�)����*IS�(7 ������!����rBD�s
M������!��*��a�2��+���(�su�ER���rF����D�r+`���� �\_J����Kr��h(�
���=>dw#���k����-��A��rO'��I�K����PP������6�]��
;n`�p(�E�ZZA����-�^+��A����jx����&�T��F,��N1�i��n����5��@���~�YB�{��v!���}Rb�����S ���3��OD�(!\��E�%�k�`:��������'��	�Vd���'��	�c^q^��\�G�(w.��G:euF'�f�d�����s���������]�������o���ic������c���t�}a�PX
��{u�(?>�� >����Zzn��VF3yq�;"
���*���x�vGD�?�9���%N����
y|l��h5��2Q�z���l-�h��s]�j�l-u��?��u�<�$��o��*9<A/e4f^�����zh"��	�]����b��k������O0J��k��zH�l�4�I��z�,VD�t�]�t���wGypR��H9�pRa�I�
=���u���q"�U*��'�'�kl����q�*�x��������GQ��B��@�
�Ul�6S~������9�,6�(��C/=���g�j���a��0��<���'���+��|������o�?%�5%5I#^�^/9JP���.�V�V�5EO�7��=��7��#�P�������w5����U�U�j��s��J�B�Z��T�5��(�X�� h��O1kPR�2,�An��hObD��L�X�9����:6K�LIE��K�]�RA=SRu^�6�q.���T�s��\ny��6�Tq�d���U��cHI��aj�a�5%E�����W�9�t#|p�S���Z���$:��,��"vOI�%T
��,��N}�>��V��U_Q^����T8'G���������7�5�j�l-u(�?��u�()�8����,�Urx���b�$�`������_;~���t���m����+()�;��&E�6��Q��c�TD)�t��2��~�3����8�����r�������Z:�<�9Ts�b�+M��G��#s���/.�e��B�:��t���l�1E��ey;��:s����|3���d[�5(�AI��^���M*���hH���h��D���k<�0F������?�PR��z=� �
������&i��iM���7����JK�r�������!��E+��qH�,���qR��I^�,��?���in�Tv�f�;�<����E:w��S����|}�^�^o�I)�F��>J�A�.�S���li����viR&-I�fVK����7�������-'�4)��[^W�]��Jr��$���U��cHI��aj�a�%���F��*��b@�*�c�������GA�4�=���]��9���tW1W�����k'��WBK3���Q��yp.-{������AZ_�WCeW�c�F�Q7�c���.VT����B�g�����:��Xt1v��P�?�(�6�U��x�?`�l��~e��/�z
���=� 9{�),���Q��M/
X��cl�tX�4`���y�h_�y��S��)�y,���~���o� ����MO���xnq�(5g�5��'ww�}$E�@��k}�B�.1

3�c�����#
��It2<d��h}g�������:,Tz���%�g�-��O�_\Eb!��BA������'��b#Y=����c�#K;�K��Y���LB	*�h�
b���+�];��V;������*��D�Uk�GTj��e$��'��U �J���^X����OB���(
��RA=�P�rRp�Z*�gJ�Pj�Vk.��$T��8rv�����Z�
�fS�Z�S���W}��!���������Q�A�r��(�RU�9�ue�#���(^�E����������|�pr
e��ME�K�������}�B��}��J�Z=���>���%F��p8���%�LE�Q_�c���t(�%���*9<A��'t��R��KW���������%cd��?����U���84n�Q��+���D�5O�{-�&�]r�^PR��@9�PRy@Ie�Z=��P(<�l����2��Z�\�ns�wM����i^W���e�CK����MyF���VO��4<�	�G�L��%�/�����)��-|Jh6�(�iI{7=�k�%���u��������B���)�T��9N�_��8\�>\���Z2N��_����+��d�A��-�2>,>�vyQ�����)Z�{(�wAIy�a����v^���yQ��zkE4D��%��H?�����K�TtyQ6,)H�N�.�S!/��\%�Z*�gJ��a��5��vyQ��%r^����RR����v�����|�����{�
�����1����05��0��������:�/�!�u��6�]����|��Cc�:��&*�]�x�w��$��A���5���l� s���	�:%���T�/[KGI��~�"uR-����%�G}���gFS���JO��gLU��Y�O�y)�<�6I�	:_e\B	]��?�H�� ����.�638��YR�6IhY�&y��f�G����e���h@I�V������3�F��5#����r����
{�i;q��4����s������g��F����P����<�s�AIQ�%��k��r>�q�:�E$8s@��wJ��{J�{O���CI��YR�K()��x�O�i�W�D>�+����8h�b���lZ���`�4)t�!���-��0�{�.M
XYZ������%T7���8TO5aC�-���2L�"<
%Vk;�����:~>L/O��8���&�Gd�M��u1�
iR�F�K�R�TP�4)`�w��
j�&���_g8���'E;'U�R�-��!�%���z���L��:^��:����S�����QQ��^�H���9�v7i��>`�c������:6������n����fIE�s��B�`�����vA���w9):sR�Y|4U�����
�=����R�Z�pR���y&�r��Y����Z��8���t^
\��Z-�'�|�Y>��������9�j[��������3?pR�-��G!k�l����n�W�T�1R.:�TpRE���y�Q�y�Ki��tgNJ��W����!4�g�-�9��~���]�7y2iR����U�vh$����|�I���_6Kk}�A����H9Z�l����3�����:�Ty?M*M��_CM�����3����:x���k�P(-9+��HI	*�Q#/����p�Q���h']QM[+���4�P��E\Y�����|$�
���7k��n�yS)�r���r��SR�d�Dk0��������?z���RAuW�&5���`3Q������5�[J������-��6%��C�-����:^��:��T���_�^�G�U��kI�Z���Ob��>`w�AdV9u�k�d�:`��{�*��e��y������e7s�!�>�.%U����|tK��Z��=g�"��VKek�c@I�Q_�cX��x���Kc����OjB{�����R����>��I1�'�|��=���V���*�Lh������61�����c)��zQ��1 ���I���������KJ���?�������g/()yHJ��0B�g>�a��{�\0�_/��n�Ts2�QY��f-x�
(�����)�0�R�|�^�l7����+��&�����*��|+�+�����BXB�zc�������'O���BI��UJ
_e:%��
��N����S���^�����"��q���()�UUTY���7&5g;�]���8��we��v��~s))B=��bm7X��|
�z=�Ii�v�[J�x�^^/QRY�rO&Q��5�b:���`�l6K�u��~�o�
���a�05k.����~��'����u8_y���������OI��^#Jj�:L=��_CI���������z0"Sy��>`�]���l.��7�V���w������u��a6���%s�d���3z.�MJ�|A�r[K�j�y+F6�j�l,u\R�����gT�$L�+��V5g��3o5��/T�#�*WC��j�i��b<�����t��-��F!�bDDoA��X7��Z��us{�%; ������!�����JD���:����L�<K)Q��T���R���6L!��D�
��������P5��)5��~~���*�e;"��y��M�$�x�+�OE|�J����P3�	5O&�����:D�}���Hgz�\P�^�	�J���"���$�s�������k��s%P��s+�e��C1y��Ug$A��F[#�����z#�mV;�CIC"
=5������Y����,�<�^"�()Q��Q�����T�'��X��RA�������j��v��oX������-e��z|9�t������E��!���uu��{u�({?<�� <�&��-�:o�riyY��������}��x(��x3�����s>�9�Em�J������� 5��f�b��^`3�.
e���|Xv������B\� |Y
�m��
�G����W��g�jh�����bX�R�gC���.r��EU
u�3
��F��j<��FqX"%ze��l	CJ��B]-op
jy:��\�
h(�cp�th(7���
%���������D'���|�����}��9��}�d������������<�_��`���-��L~.m�%
��=>�6i:S|t��zH��K�	�ZdM���{O���CC��K���_���2��l
��c;�(�����a��<f��v�K�����J�k����=cS����|��we���j6�>��/d�8<7UTY���N�7�R�~H��fm���4d�������V�����CT�b:5��K�5K��2g,/�f����I��
���j�E���r;e9���u��C��dv�BW/��Ci�^C���O�/�O�y(�@���%s(�{Xa��q��x(�b3
;���@-�8\�����V��B�
�B������:��P��%��t<*,���*�J���y#*,������
9�w<
���C��C��T�?�w��;
K��G�����<4	��|9&��Ca[�P.��J<�^�F�����g�����x(�ct�ux(?���%$!�yF:T�TR0Y!J~�l��������T:�����r�����x(�����_R�d��k��|�u<������C�O��A&9�����x��=�������b�/_.?e7#���=;!�K:�P�HP��4�By��3U���Z��������_gJP�g��mJ�myF�_u��c�������nS\<�>��?!1�Y�e�[��z�����z+����d��5�b:��S.��P���z�'y����=�����I���������P����_�Oq���./�k����Ry�������1����������R*�7q��<HEA����mpn��>`��P�u@��G(q|H&�����|���{s����A-!�})���OFe�6V�]"�w����bk��(t�v|�X
�����G=����.P[
�r�?Q����u�3��F��P�?QzY���"J>����y�[a��9��4
�	���U�rO��lB��ja�<���cxt��0 ���f9�Y��<�����7�4�:k����R�k�����2_��!!�U�t���V��9�ry?�]��s���|y4�7��p�Y��rK��&�$UW�Y��]r4;f�����u�!���D�*H���SZ�)�RZ��@�Mr@��Rm���Zcm�P4��R�])#!6�����GD5yW��M����=k��Q� N��������;�r������.�9�D��`8�ODE
�rL�^����������f������J�,�sY;�@�-����-�������-���c��
!�<���x���Q�~x~Axz�����{h��$hw��.�v���������&v��\m��D�C�G��A���(:��'`���KD�.���3!]l-�f��\��f�l-u��?��u�(b����:�li���tD1p=�'���]LB��O�A��=AG��$���z�J
�z�i�}
�X��uH��	M��������wJjn=���)��c�u(�8���%%���:�h�(��H)�+<d/�i��^�����b�x�w��}�~)�)��~5��6�F@98�R=�n))t��K�gs�h�4b�2��E���m]����%%�q�H}�h�t��>#e���Q*�^�4D�buK����sjE�*�0r6e����'&)u�@u���p���x������TIk�2�X[7�Cz�Y�3R.��$I�Z��ND#�/5�������/a6����Qq�*���d���4�b:^��������1�X0H�k�
j��2�t�n�\�o��3R�q�p���|M����I���)M��c�H��Qj�Q�%#EE�B�z����BsqWo��������E�H/(�L�D��X����?��G��>G�������LAe��2R��������ck�)T�D��R�Z�0R���x&&r�K�4V��	��)?g�~�4���x���7y�>A�WwE]F
�EX��z�#�(���$)Kq�f���4��CF���g�&��T��$��T�1PN:�T0RI�Z�LR���l��l�z��%�������xq)�~W��T���/��~�U}r�jgT#���/���Ku�|�y��J]��=I�����(��R��D����<�%���u�����$e��j�X�P��-f�)s�FO5JP�, >�7��\x>"��]�PQ������i�J��v�z����9���i������J5�_������i�����A�F����OS;\����[YR
�Q���B.���������f�����������z��b��`Wk.���T�U��g;��:dH���e�����������W}��!'���������QF��2��E/	�"�����}�����
@�9�r��(bw��,u2=}�!E�>��/1J�/�@L*s~��J�r=���[K�%�������Z*[KN���z#�4��^z�UKc����O��b$G�Z�O����%�@�j�?A��]?@F���'pR�����F��b��q1x��q,.AoO&��%n����������?F�Y���N*+5�����<[�"^#H��q(�*�+��m��K+A�pO���u��=�,fT/���( J��9�(3�'�;N�������%3V�����$�����(�����i�t8��:'����K��O���4y)��,���G�ti@E��I[}y��������m��V���I�P���'��k:�s)<n\pJ�*�=GI������c�k�G=R��i��k�s*�{�2,��SRY��9��U/�)��t�o�����f����+��t�RA=7�V��\���~KI�=M*�ny�E3=H(/.�b	�ms���W}��!%��������
R���=^�,���[�v�Y���p�������CQ�����p���7DjI�+�c�E7a�6�A5��RR��&Uh�3�>����=�jv�m����[KJ���z��=�U��ieKc���/�CG��l�?A_�GUk�Y�OpU�*-W��O0*����[;��X��xO+��0OR�QZ�l����2R4`���8�t)0R��H�iFb+Os^|S�����OH	\�p�o�M�-�L3���a��].+��zK
E{����1�_L���r~�fs����am`��V�><���@!t�Zfq���^Q"��w��:d�/gB�<�&�X�v�T:��N%����_�JK�-|�~��@;E��`��g�7&sPkE����j�7��(�g��.�(�j	�����-��l8�	���Y���<3����<�����
DT�T�L7�b:�QHl��:m�
���V���F�t�Q��������B������B_\��
j9e��&���U��Ci�^C�����b�AC=%�������E�����w��>@��ct��b���H�4���2�����������3�
�Mx.�w9(:sP�/�t��t*��.�}x5T6�:���:>�S��[
�r�?;fHgA����g�k��j���]4�i0��(��(��K#pC@�?�����j_8�7�)g3�y*���������<�<5���i����6��r����S*�_��b��#q"��o�w@� ���|y�_������>b��1=�u��<���/�!T�N,�|t�D�7BXK�����
����u����������2��I�������+�@HI)PdZ�N�Zo�.���0I0]�"�4$��)�����`���@��&D�;U�I���Z�]�ds
�L~����2����'��Q����|��N"��f]L�t?�(��������Ppm�8w�t�;S}C
�]?::OA��p*;��8 ��u��;�����
���uu��{u�r?-� ��|2��e>A�UC^U�Ov�v������l������e>�?
<q\Y�*bw�OxorR��o�|�u��tQ��w����e>���Le�T��:����:��85mZ��w���LD����`�!���2nJ�>AJ*�rx��W=��j9���	���*���M./X�aBaW�P�FlR���%�[���5�����c��|�I����<���p>I�8��&�u�#��M��*�u��J��<Y�J���������q�}&>��8*u29bi
�)�gI��O�t�S�n�6T�6��Bl�R�Ym�+Q�if�����*���_V���g�����f�v� �2P���9�����J �
<,X�!�YR[�9E�����SV�m����6��3r�grP�]�E>
����)��GIO�* [��TO�:}����yy\��AiT�a!��4�b:�'=	jh������U��VK�;3+���d�/9���P�E��
�A�(���d���U��c�A� _����k8��O1��eg���e�hK=uq�{�F�
����c�{��!��phv)���U�@ ��<����79���<�|�����|�����j�l,u\3P���d>a�Fu�jh���9�	���g_*.���O{����w���O�����jQ��|���R��]Sw��(�)���{������o��in��l���1*�:��0OV)�)����n��M�ei
�O�U��8��������8(m3_��.����B��L&.�v�� F�w��`�l�<v�-�������]�>�b[V��*���b�t�'�~��u�3��;�w�j�R�j�v����8���W�b?/���f�V�s^�8��������3s�19�	��47k;���!�I�����!�t<8/O��H����T���N��Yw����O��I��j������������x���k���-����J7�[^'j��!��v�������1$���P���Pt����Bn�<��"���!�2eq�;�)������?Mv�vG:�;>��3y�<`w�S��76�n�6V�]���U��[v9����B�<C�DT��.P2yk�c@;�Q_�c��<�P�_���JO��'~N������U�|��t_-�'�|]hs�b��+�
�lZLM|�	]���S.cr�+G��b
(��Oq��c\�O������On@?9%rL3��0�n��&>�g��5�
����pi�M�<�,k` A8B�XP)���d�r��@.p�RZ���VHQ{���.
��RZc����������6"w��B}��_����?��O�}�)�b�^���,�#q�7^����LT���K-]�[��Hw��N�+�%��9��L���{H4����o\�5���/
�p&W����i�9���]��f�':�L\�>9/�����l��O2��r�4�b:���`9�P�RA���Mr�TP����J������-���.x�4��o���KF��P��I��uu��{u�'w?&u� &�OJRO\&��z(�����}���"J@��1M���]I��zQ�L�O!����0��56<��O�P���?�s��[;3�����w�/V:(����R�����^��31���i-*�yx��?�YZ����������Il�'0��?A��qq�7��������P���1�!���1n�|��c=!.n~J�������'�cp�u�'?���R��L34�$tc������;��;*�Y��=��G��B�����6���:���g8$�6�43�K�����<Z�o��EO���S��h�H�>���:�O7\��������L�Cg�M�y�l@*cV���b&��hl�La_:
�O%�>:(@��OE/bfH��}4� �{/_u}~\X{���@c��V���L���.���,�~��]s�/C`�?Rs�I$j�I4��ct@K
'W���;`%Q^�Wj��QCu�Q���u���}���1-�3�C���f������1d������������=8{�	�g������l��_q��
D��)B�z��>�F�k3��f}��t?��"[�}�y:]���0���U�r;��C;�5)o���Q/�c$4��6CuR�Qq���1q��x������#�j�^nX�=��V�]��9��n��>�����c�)�� �=A���>�����Zwv��~���|S���P���\���Q$>=��8�N����S#p�IyBIh��ww���3g���)��o�]���l�p~�����B���F�[���9�J� x�\2o��SaG����w��:|Sx�o2�u��w����T�TZr>�Z��������m�LIj�v�� O��y/*�����M�-��9n	�9�%w������j����M"`*}���}@�Y��.\��?Hw
|&Q�V�b:�&���VT�i��^��6TT�TP�����J{W*sQ�������������a�^�N���d���U��cH8���h���%�DEKg��#9h	;4��d�.�vW�'�uM�q����Z�]������4y�<`w%<��U�`C��k`�%��Y������/�CCU$�7Kek�c@;�Q_�c���L(�5Kc����O��A�����R� �t��R��W��a[�����t'��Rj�]boJ[����'��Fsx��K2��8E;�~��)�G�)���D?a�	:�<����V�����X��5��v�_5�<����*��ZM�	F�2Td������v�f�K�oGo���=���-[�����������[��e��a�a���.��A�~����{2�Cg4{)��_J�Q,��g�$����-�t�owmc�{Viz������������D1ST�{Z=�hA�f��a�n���n=8���e��Y��"�z$��Y���if����yyT��?�S��0�$B��u1���'��6�������f���������
�f�~�?�����'����ZEI��x;�<���x����O�~LAL:�����QQ��I�sK����,�vW���{�K|��a�c��B��69q���)`w%<xoI��;�g���w��x.��v�3�>���>wO������R�����^�����?��}�lmf�,�'�����b�Y�O��*�\����	F�v�|[}�?�Ba'��0A)z5����OVO�)�'�)
������iF�(�����7����_����W���M���g��F�u.��>a
��34����v1��4�B~(���/����O���5���K�H�����!Y�=���BP*�����?��O�?hs�D�t��\E*��Y��
���� �25���^
|������b&rAg��%���C�ZQ!!?V�B+o��u�����r;>/���Y�l.������P�I�����=SH�������z�����?���K�.�S#�)���VK�O�7����G;���}�S\bn���w��~J{�]���^�7~~/i.�����9^W���W��~J�C��B�k�)�����x#�8j#f��>`w�����W��I�K�����u��t�>�����R*��I
������{��������e����Z��;��VI�P�X��O������\-���
9��k�8�%|1���5w��E������;�u����s�A�]B�Smq���bb��3~��d�����'��v#���|�����)�g�)H��D:�;���Os�U�R�!���ED�Lc{3�ZH'C�	�#���p���A:�dy2��e~����>���S�I���-��{g�5wAj�B@p�H'75g��\�����_'��s��G5w�->����r�E���P5���R+�)�f�yY�q)e�,�9�I�oo	]3��,�=�,��;��7�f�}Kv$v�������Z�����	����yyR�E:i�<��5t6�u1��I'Q�G]g�TP�s}�����tP����
������-����D�#��� "�Z�[�q����W]��!���G��D��%wA���D�Dl��b����}k���
�Y,h�����6z��[4�U��dpytq�.}����M�e������r����g���*�J���zy��;<�h5����
9����0�T���:��%OYE�WC�����oY����$'��^(+�e��o�����_������L[�&M��h�7���0��M4��H�o�i��iv-8M%G�";�CB=�,��-��p�8�o��z#��c�������v����C��I�����|u|Spn�ig80�I&9AX~��h���/�QI��{����7��|�	f��S�.����D����3��y��}�8
��K�Y9���r�R�RF������K�
|���������G�I���RS����g�L���7>��tw_��l>���e�K�N"O2����L�j:���`��Y*��A�X.7T�uP��R�������N���7R�@F��������%x��������W}��!�D�cQ�����.ie9y^K���4��s�����c#�x��u���c\�����(b��1��Jk*��V�.�t��k��s�[K����E������1`�����1`�����w54V��L6�?�MW?�n���C�L7*����)��-����K���~�������b�T�de�2`���!q�a���u*JYN2��N����s����q��j..�v��g��_oQ�pv)-��T*d9�N����p1���7_�N��r��������9���-�������������c�th����x�qv�,?$����u��/%�g����U�����g!�k��j��xO���{�]Q�����0�W�mn9^��
��v.�(�5��Z��h�� �)��DY��T�Gg��\F��}���B�&�������������?T7g�v�%c�ZS-������m�����y*��8�������E���||�r�
�����1$���p���p��x"�T�PVm8��z;���]����64��H�Kubh���y��	}�h1�R\.=�����K:�NY�,1]�*�d-�|�����R��t��^^��'��l|54V�����
�e5����	X�@��:��Ee�c�����T����tB��V���c='�IA�3����J��d='2��������I'��^�N��*z����N����C��f���j��vh`�����&��3�{�T�,=�C
Fc.=�w>�G]wy;���R���T'��;=q�{l��6���NV8'd��vd� 58�.��A�s����zN�I�Q��EQ������$Z<��S���?�-��2���U]����Zs����+��5�X�2�lH%r;�xZ������M+zN���f�Q���8Z�J����r���u�|p^�To��E
9q�D(�/���N9qd'B�Y*��9�74
5���d,��j�\�/������d��8f�+TE��>*��x]��^#�i�:=���D8��o#�bo�Io��K�����\�G�f�v�L�(bwB�x�U�`�;�b���$���Be�o�N�y���S�������(�9V-����k����z�D�1�uZ?.�\%�'�2��/��\�������	l��tY��iU�X��`����%����aQ���7�{A?A��Y��]��<,��Nr;�����������~�Z���O�]b1M�)?��?�]��I�8��M�������Z
8�����Z��B^#�	p����K���w/��#��O|�;�p4Tv��J� 	@i��+�Yqk���.��A�~���O�M��+ H�;��������Z��-�~+���7H���
���������wK��o�����&u�d��^0#;UK
d)�^���G.Rg�Z�p��F�S�������q�������<���w*�O���6�b:�'`���j��v%cE��X-����2��]C�����Ovog���-����o�%���>Zc���:^��:������_�^�O����?a]������u�Vi'�z@'g>�w��]dKk<��v)������E|����O����w�'{ng�?��[K�?q���%�R�Z��O���y&O�H<5Kc����O� �K�����2n�m�����W��Q����O0���"�@1Xd����l��O!B�Z4�J*{v\B2�\��
�'�cp�t�'7�������B���9-����A��|�'�%��_5��W�hgWh�d��[������������W8jj�)��V������!�VPl}�Z�0������qK�Q�����?��O��������Z:IKm����B������7��uB��Y���O�U���}n]�%��LV���?y`�[J���-�������v���"�Z-E��0z��+
����t����.C`��O
���I��t�����F�%a�j��v�$cIK�f��v��.��������f��y�E��E�'E�����W}��!�����������EK�I���D�)�<wq���'�^���]L,��]��l�
6�
������?�K��LK��F��}�~r��;���������_o�������R��~���^��~�dj��vle�	���'0*>rx��~�m��t��,i�-�	���x�M����J+�l��cx�D�.��S���R�L?�������������|)e!��R�a�-fI�z���
�^6��Br~�`B�1�~2f�b`8�
����bRH�p��py#�"/���'��O|��}��<�|Z��������*�*^��������:���/$���O�CJvo���k����>s��w[<P(X7��\�jx�D�U@�Bn�l� y�
<�-��[�B[�aL�y��1���"�m���d���x��'Ya��'5g
����yy\�U���sP2�_	�u1��=��t������.PM)+�)E�_���(X����rP���_�Y��^�{Hw����b0W?���{u9(?.�� .�����J��n�P������]�Y��.��P^�������"v��{��������]\����s���*�}��:]��7q�����>�#��.v5T6�:���:���Q���X!�s�`Ew������]GG��P�?��El���G�O�l�2O.��l-�P0lqg�����^�7

~r�]0O���8�0Oa�<�w�V �:���v��1����>g������B�1+�U8I ����g��F-{�fk�>Of	M���-�)���+�&�kJ�>�����E�R� ��R���.��A�x
�O1M'�\���?�om$��y)A��!]�^�[�;�)���>��b+�q��x���F-��U9��,�}�*�os��B�mDX���>��q���{Z��=�����7��:E�w�/%J�j]L�B�;��*k�Z*��A�4����j^���`M�Z�/�Na�|�f����JI�ZQC����^W���W��u
�c��b�k��))�������8���-����g���I���O�{V���uv���,��K�+`�Y'`{����.&i`�e��Y����G�Sg>AdR��Y*[K����z���<��(��X%�'�>�����Ve����d���dY����\�9��f=����r~B��
�
���Pr��a)g��B����s��8�����q������J�O�J�}<��C�
�������U�����TYD��SF��;������
���]��A�%�)^d>�p[��7q�t�{(������O���I��b�t���:�d������;B��G��:�Pz�����v�(���L����'����H�A:|\�|�������A��2���s��C��\�'v�,4}����*
t�r�K����-�9���x����NCw\&�P���:��,K����i1�6K�?'�B�;�Z*��q�j�f�E��~���S.���{��`5�^���	�&{]��^C�)�I�/I��jIO�.�����U����������2.���$*�}I���1����;����@�"�{��P���K?����x{�����/��s [�f����R��~���^��31�T������t�	���\���wI�~}�XTv��t�:�W�O.����S$��M�	�L�H���]��_��4���\x��S�16N:�S�OI��4p��Eob\�����T�Ps�p}�[�]|&I3�%!�Im�H��G|��� ���u2K��r*���n�O�����7�)�4l{g�� �5b��O����{���C?��u�mt���8�C[,`_����g�����Z\}��M�);Gj�Of)R&S��p���u��[
���r�~*����;��+6k;�S���`|m����{�N�O���e����;�Ps�[2z(5�b:����BC����������a�TP����T���~K?��~rn�����v��"Sg�:�|��uu��{u��t?$M� $����,*��~�nq�\�.����Z��m���K�jM�e?�����v�j�O���Wy��[vw��t�~B7�|��\d?Y�Z�$������1������1�~��L�M�G���Urx�>�	�,�+����O����9�F�	��'W�����	F�O 6j��@�����P������H\!Pq�}�[9�$�)���cl�u��<���R�Sb?���9���:����H�����P�H"�����dd��y�."�����m�v��+o�J������w����O1��*��5�h�����A��C'�����g����u������w���k���8��1�M��;�`�9��v����)�],�p��	 ])q:2���x-�s1�Gd��w�6�k�%��"r~�������j�s�t�������fiW���8�oT��$����ZoT�U��MS�7��K���F��`�PV��\�o����O�,t���a7�h��"���N����W����c��b�k��i����R�EZ������O��^�/V��<>h3�_�N]�(�9f��PV���?����.��N�	�a�W����?��u4�x.�#�j�]y'���Z���������Q��=��5��_�����k�)���Ja�����[���C������Eii���+�\������D���j�@������b��H!O)��J
rY
#�U�o���}�nTK�W�\��y�Mb�9����v("������8���a�`���u*�7���4�0��g�
*�[�2���[�����u�a�����L�k��VqtA���Q(jZ�X:�c
o���1�P�����7y�MQ�/����%L�a�c�va��)�29�)l��U�&���,�/��}�P�T����M*Ewt����S�`�0�R�����T(�c,I�m�
jwP2VD�Y�TP��r��������N���)<�[^�Gj�J���g{]��^C������b�����`�X't^�A�8{(*����}�;vb�{]���s��}�;D���.|���w���Z{��R�]�����PI[�E�;��ymI&���R��w���^�����:�%UX%�'�*�-���Z�O�������Y�OpU ;�We=���Ol-��{`O��D�FQ)@��mEw�N�|*�������������A��4�}�K�B��9p��v����*YOg�_�p��4$2iF/��+.����I���b���S����[6Y=�z�!�k9�F����Cl��CIs�{����?����B�(�C���|���<F��^�x�&M�t�2��wt�h���eH$Zy�q{KrY����db?:�Q4�Y����'|��~!���4�5����p��}P�v��^��'�[IO
��d��4�b:������~�]d*jNb�mK�_-��'�*j���-�T>���b�-����S�$ZD�X^�����{u��r?$-� $�Nz��:FA.T������~��>`w���_�dM�%�����lE����^�i��5�)J)��]��������|��Z�����.�����1 �����1Hz�\���+��Nzb�����P�?'=��Qv�����'��
��8 �L�K�JO���q��7"���2�
R!��6I��&'=sM:�����X�:�t����$�A:�k��Lsl-���`��wj��H|[������-�H�����]^r�8����������4���=�TL��dh�j��-��Ng��L^$���z���htj�n�X��I����N����Y'�!�T�9��VaZ��i=�*���iq����=�'k����}%��+Y�)���"����z�L�>�h�jv����m�`����\1O%D��|�^�v�{3�9�>=/��w�'2E�y�$"_vi��t�g�"��RA���f.6K�;3u��K�i?7�~y�h�
������xq,p�D�$���U��c�<� _����k��ll�R�p'�Z�l�l=:gq�;�'`iMc���;�'��,�����N}���������.>�|fb�d������e���Zz�'rdg�f�l-u\sO����$�df��i�&k���t������>���]z�,(�--����]�"p�kO�'g��8(���kq���F+n��9P���\RF5r�b��A�e���e�UK|�'|�Ns�hc�����u!E#����"bci��n�����(	����R2SZ��_ri1���*���}��78(�sP�v�O@��GNG���U����i�=9�fw7\���e�/�s4=�	��P���24*��m�l)�����M��$�e ]Y9��fB�"��+E�T��}D�vD_P;���������}�i�����e%g�Z�G%_4�'{�� �)k$>�$z�X���TH|b������QA��IyC���UA��K�a���L~�o�'��O�J�[^y��DIp���QC1���x����O�~HjAHz��D����:�-�p��%^K�������7����&]E�dw�&��\�=��E(�"v�@���9O(:���K?���8������K|����VCec�c@>�QO�c��),���X!�s����.���N|�k�������'M���<��N���$$>!*4�����]2"��,� �h6�)OnqW��tr?F�N�tr��)���4#G����H�|��s���p�g[EW��H'�Q� b���r|�k~��G�,R	���.z�lzV�x�tr����7�)S0#����k��������x�Smw���?��N�}����jO���v��}Q!�3"�i;H�+-�>t����H�A�,�����
4�����Y���i�KzX�w��D|dz�����&�a��X�c%���t�5H'w���I'�w����:�l]L��w�j�|b����Iy�b��~�?3Q��e}C������-��v�	�����![��&��C&}�������1����H���H���.)u�C���?Dv�C��z��b���%M��tW�c�f��</��U������~�FO���7�3�dh�����t|SO	j�*�J���zy����P#X
�r�?;&�W������N��o�����.*����O��F�N9.�(����p��$'�I�	�>��:�����](>+���n?�����������o�ZIN<����9�6u'��4����$��9�������}���a����K[�����W��#'og��.��GQ�
����N�6������N�����>��t��o�X�����79���\�o�J4
��/�����Swr��TP���r�[���;9�
z��M�)����N��U1qW������{�I�����:3b��9��IN��!K�S��?rn�~��M�2����MYE�� S���?�����������P�jNB���������x���k���-��wu'dx�[^�*��FJ����s���W}��!�����������F���yY	�~��>`wIN6�����qN�Kr��T�$9���&v��$���oc��_'��Y������#�r^������R��t���^��t��D��e54V��L:1�hF��:��tbX��E�/�����2J�W�9����&�7
5��H�o�c�	�^�0��6mj��O&���t
?F�A�t
�)h����RA�O�_\K��{|���5�
J.j[9'���Q�6��B�u
�i+us38��E�,Zo�b�6J������B�����9�)��ae?4J
����T��kC�u����?��N�}M��t�����9����O������^�A���4���*�G0�
uhGf�s���Eo����El	L><������yj�S*q��a�ZM
�%kG�S��'_�Uk���L�����<�^�����I�Mg(W����X���Z*��i�#�N�����2�f�a3�[�i;8=J��+���Y��&�D�*�W/�@=i�^C�)�H�/H����Z}d�k��*���0eq�;��cMr��'o]��z6	�Di);����QO�N�M����q�z���|G
[K����A��UKek�c@>�Q_�c���L|�i��*9<A���%���T�����7nK+^--��v���w��?� �	D8�6�b��������a�-��X^���[�J�����;�����?�Q���*je>y��v<��CV@�O�3"1��H(>�4H(YD����b�,����"�8+�i��cu��!��MB�������e>�nK�J;g�.�B��E�D6(e>}�b�t8��:����0�a���7�E�����3���9������������k��Q��jlL���0��'+���on�t%��4�1���mEj�Z�t�3x�Sl������M��������<���}
�O>�J�f]L�B���<�Y���
jR��<�Y*����Hy}W���~K@m�f���Uny���S����Sgi�������1$����4����Z_�5���v&v�����=������&X�"�U�U�^��w�v���-�j]p�Z�w	�x�}�o�y��Zz��%aJ�����1 �����1�JDE��iKc����M��IXXU|��}������J�jO��)��Z����ss��
%���P��m4(��:��%oM����q�T�1:N:TPI���i�K,Q�9�[\��l�����K��
�W�(���Q�����7)Q
��yq3���\AL\lB�������� ��Ew�a���7��;>�pk�D�'�-�\�)P�O>��A��J�gA��3P��O��-$"g���=���R�"sK�6�S5�{�%�g{b�6v�5/��{��{�%�K�]����p{F{]pP�M
����(#��jEb������y*�.���pPq>%��Y�3s1��=�b�CF�TP��%��Hr��
����XqC�sQ�����A��oy]��u�h��?
�8]��^C
*�K�/K��������,,6��������}���1��� �j�"t�`�V�����3h�=Q^��������y�w��t�����>w���.d�'p4Q
�M����G����O�\&�g���
9�����;}��w2,*���P�?�(`MX�gl#�'�,H�D����e�/��%[)c"Wm����S�N���8��Ny@:e��v�e�:���*����W!S���O&MI����NR��,aK���"�M��f�d�o#��yK�)�zO�4�tb��8dz��g�ZcK�&"�����u�����zO�L�z��qU�.hS���9e=adh��6o�~+�1���j��0y����ii������c��#�fA�y����l�Q��O��Fb����I��d���dq�����������z�p"��'��F[�7��N��v.�l��UW�����\�L���v��6����o	��>>6���A`���"H�x�
�����1$���P4��P��p�JEw�;��\�d7>@w����.�-��	������Tr6��;��}�i���}�p�g�q����]�'�����P�T�N���N�1�:�q�y~��Na	��T
u��p
W��#�Y�#��������i=%�6	�d����`���y�*04mYNd�q���M��~��I�p��DJ
�d���N)�V�B���9�[9������0h�#o���B������x����Y��-%�Ly��D}��!��_���N[Q��p���prJ���.��A�p��@��<�����;���
�����V�@y�ReWBY��D����#O;`��d^�U_S�����Z���E�p����@r!L�yM��kVS��~X���?3�`5k;�
�t�������'���N&Qr�V�b:�w�V�b��RA�J+}f�f����%�����2�]�%�h�y*a���
�����AZ	��6����W}��!�D�cQ�����S"�*;�8�R,�8������%�z�pg�&o9N���]��c����,�k�)`w	��q����6���]����G�W�}n-��8-�:UCec�c�:�QO�c�0�s������B���84���TC�S�%�~�����8:^�u����a���2�Y������'x���QL�����$s��L�����.�����X�����i�W�i�-����g2���p>5������4C���.�b8�8���==��'���+7mYU���Tz�	y�����;?T7KN��������],3�b���u�a����������w�4'$;mMm��j��X:��"j����1c�
����h��|%����������t����Os�a��dQ���v8�����?7��g��?�����������:E����I����Z�9��%����a�TP������#�
j^�J7k.���S���|����{�u��w��j������W}��!�T�������<��=��N�k�}:�pz����}�>�N�Wo��]������}f�[��8�D��"��uvr"��W�h�|�u��t�Nf�p��t�N6���VKek�c�;�Q_�c�~�}h�,�Urx��OD��>[�O�u�.*r���]��=X�f=���_�l��(�S��?��?RCAT�"�&4���3�'���%�?��?���������@B�\�M��:���4��8���n�d��~+���J]Ia�[=�38/R}�����+�Y���$/����H�&1�}����US����]�	�3�a������Q�?�r�_���e�,���[��������L�:�qY�*��!���!26��-���%#0���\��_y[�J%G�X����� [%Z3b�c:�b1q��������v�3{U����%.*)pQu�6��jB�{�`|�m�
jw^2V�7�f��v�&���6����/����3x�����o�v	�������v�:^v�:Ft���1����&B*�u�����L@��,s�
?`wiPh�$�G������]c'#>����{�q�h�9������IHu����=��c�P���*	-���R�5#�w�����8j�C���� �'����~�h���z���W�W�K[o�]��<a��D�,_�Do����r�R���������hz��$*C��@�#N��+[%N��8)���d�b�\G4�j$Q�Bu��������s���u� :.o�������6���� :.p�4h�����#���q���������A�,g��=��BA&�A����j�Ls��+N����Q����5y�<�
��o���@��Z�����/��S��O:O��F3$�~��,o�-|zP��3������l�|(�`B��}L-f��y��/�R��;���'����ie�f�Q�'X���J�T�	V���z�*���m��.�������[���x|Isf���`��b�:^v�:����� ��� ��4/*����"�o����<�F�[��y���o�����*�������<~oS���m;�����������&�|�_����B3��x�6��:{��<�M����W�1l�j�^�X������y�v���^uG��
b!wy����#a�N�(�l�_]�������(�s����(7b���s�*5�sm[��"��C�;��L9 ����g��>Z���5���T���5���QC�+u@�1�m��8L~I��E��_Z���6�4$�-~|�A���9���0Q7���������3`v�����J�I���m�G$��|��
�vx����7��Ld�y��.�l�m��J����R��9I"��&����5�L���Q��Ni[hLD9|��^��#S:D�����P��U�(�$�	e�bB5�(�~Y
�_��Dc9�P?*{f����[�i����w���r;�w��n9�\a�G���q1t5wD����1&��?������6x��_��s����{��9+���g^���v��#�q>v�s���0�}�������g[�������q2�6��D���6��������������1"�����1��b�@�������D9p#+>�8����Ze(WC�����&U|&l�qr����
��ef���B���;@�x��Tr���lfQ~DD��d�DD������u������S��\Dp%p��X��kS�������!
g� z�Y�����"7"\mE�9��R�|�e>j��6�}��O4��>,�}v!��R"�n�Y�%"�������R������[K��LTR�R����,�-'�x>+�=@v)Q�l��y��E�?�FD��Q��Z�P��YBs�RN�,�B���pJ9��^�����Tk;�����:���[Uz��dmnj���*(F�f��v��`�
���v�f�W�6�4{��&����o�q�g��g-���D��v�������1&��?������DT�����q^�x7����,
�ml��Z�ZD�E
Z���d|>v'E�����W����m"�t;�s��b�u��[��!�S-�����W����s�t�����PO��(~P|4n��t�*o��9�|_<A����G�,Z��?���'��DPR uW�`�������������&*[g�Y�TQR��P9(QRaDIJJ�:���w��a+��,������"
��Y��D����)�
�����F*{�
Uzu2-��9\f�3~�%������������!8|{��D���+J����Q��������;��o	S�l���BT�1L��9����FLT<��������t&-�5�*�����3$de���=��������?�\��vu�!�Ah�{P\2�G7k�����
s9�p��9)R���i,h���	���&XiCM:��y�XH�j�
jlF+*[sQ�����I��u�
����#�o���N��Ru�:��T�� 5�� u���I9d�J��+��6e5N
�����C�J�*�E����>������\[[���rjU��6'�]�?[��_z��L�+E�T��:F��_u�:F�����/��X(�'��W��R�.������t���r\� |������<_���Q|�j�`�k���EG/���Sq��K���M��'���'G�TT��0�(��s�[f�N
p��C��M��?�1?�c�J�u%�%4v3�2%+�|�����]��TC����pR��� ����z]����5�&��S0e��?�Y�%N*��zv:'�����_	��4QR���~1���qgd�<�<~����V%g�@#n}K>)�%& ������]$S�
�\�(�(��_#V;���CJ�z��Z����	j�T�����&� f^�1�6�9^L�|J
XyC�J��q�!z�PCQA�OM��B+j��f���5%�z��>��zk�A�C�z
JJ���SR�b��b�%��()YA�bd��~�>���u]���M�I��HW��nYU����!����/.&
���T<���Mi�T���t��2R������1������1j}���G�fi,��t
0�r^�����M��\�-6K�	�f���J��T&�������BI9����&Uh���
�?*�x���V��3���4�����rR�����J:���u��s���&�L���O������X��Y��^};_���v9<
��!���3��R��gpw(��QR���m��gc��I�����=M��v��G?���DI��))�|�QR5��R�����<1e�����u����V��A��*����%6���E�+5J�x!��hR����n�&��|�'�Z��6!�*���h�����*��,����'�K�TL�9�:�P�o�����$`t���������u�TP�cPiK6��%�'���>��7�d[��]���|�L4()U��cLI��Q�/�Q/(�l�}�����`2�&$�9�v�<y}�;������h�o����i���������|�N!
�������m:*������jk���<G1P�[
�����W���[�l�6�Yc����	y�h*~�*�y�?;���4����R���h�x�?�1'Z�������v����Q�/��������Z����e��	���	�G$TV����F]$�uF>q���r�Y����-�s�N�s:�aD�+��&�4:�����m�Z6��v���}^��v[�O9����Cr(�������I-/����Q"����z�\��]�%���K���i���� E>*������x$����T����O��6�v��JA���'�`s�>Y��=bn^�S[I����>��|T|��@���=E�|��������i4�M�G�����>�v�d���V��������2����������6T7�k*�yQ81�-�C�1��x��-4H(U��cLB��M�/�MyQ^�T�o����yhf��vW�g�$M2!k�A����J�[T��'�;JQ��J��MY���F�6���zi��P>���R=���f��/u������u�P���(�,��rx��R=O�P���R=<�U���V(�>������T�c�l��'PR��JI�����q2
K��OK
"���B{k=���tQR�s�LJ��()R*��\����^�E��C���.,����w�R��v���-/*�g�C����� �M�2����%E}^/�M������J�=4W���8Z��g�����n�Y�%J����2�y����3�A���Kq�r�I������=9���0-�����8���*o�g)��o���3BI%(�6��fzVh�SR���UW�����Q�/����������-����uLL�SRY���i����Y:�TO��hn�TP��2��w]<*�SQ�SS�kJ�vJ
�����Y������!/����xu��~u�))���~A�zMIQ�����Uw5���@��_��(?��{�_p��v�����@~���,�R��������I���4��A5%9\�����vCI[���Wo�vCI!�Z�&������
����R8�U�P�������z��hc}i)��QXk���^��%�=J���{n{��w��I��CPVw�PR\*��;�#��;h(����"\z��������`��2v���A�F$1h�!+�����r�]�C�w��Do����%_v�c'J*�QR�%�c�i<���Mc���������WJ�S��}��wT
p���w>I�2c�"{�m\[	c��R�PRqm��T�,/��#�����gQ�2�Fd;�\��:1R7�Y�M'F*�/#���UV��Js���SFQ���VQ��z���*�JL��t�����0d�������lj�q�;
��������$#eLD~=,&S�����x���]I�=�=�����$K���mk%�GP�e���(E��W����<��Sg�2�f��X�2/Tz=@{�A
�o�.#�j.j6��
 _�r&�����E]]!�����.�}"*��3���t���if;�c��,�'��];V��9�����pz�r}����-�-��.]+���/��Wjd2�O{H"�L@�{q��.*5�d��(�������G>}T�mG!���:�*V�K�
��J�%�����VM��"SfC��rN<��g����NY�5��@��_�]��������S4��d���?+���b�a�����j��.��iu�p<>�q�������������?������<������e����v�������C���}�����H�n�w6��,�|L����97J��>�kl�1���/�Wo�=��|q��o����kOcR8�
�M}	�M���������$1P��@��w����/�%?��W�n���EA^8���D�$������
�Z�\	�!�PD
Wx���H��Q����x�}>sT�.��q�.BS�������z����$�:��|��Q�����
9+�������z~:}���j�4���\�1~8�-'+���d�$�"��-99�TP����6U'1K�y+����3
%��%)Z���dj����_���{����e�>H0�J�HjJ���~��6qI �q��9$*�q��� ���K�a���,p��y�~�����#�w���������~'0!�,^3�/��4���Z�T�k���3�
u����r�z�����5�D������N��YL��d��1��L��d
m�����PK�������PK{2K18sept/pg_part_patch/12_1.outUX���Y���Y��}m�&7r�w��Y	�d�,��e@N���]c��A�K�`G3�hg�������|�m�4��XQ������y��U����_�����W�����~�����|l������O~����>�������;������
������P�����}���O_}���Wo>����e�1(:/��/a���w�~�~�	��_������~����|����~���o�}����������t��s?�x�t��_��_������o_x���R*���?~�����}���._����������=���y�(���q���wI%WkT��6���O��|���Wo�>~������o_��������_����xMWS������ka�Z���@������@a}�`�V6�V��O�����P�xO����\����P�
�����m���AZ��1�Nj�~��$����os���?|��n������?��������H���������W�?��f�?����_��*B�.��<?	�)����N����~x|�M��������2����S�������W?~��_��~[�D��R)��z�&���<�E~�B��]�����d�������p|���z���GB�??|��?���n������x������?<����~��������E�����tV8�{G���A+@��s��p�7��������=���\��?������?��o���������W_��o���>��_~���W������3���D^������w�7��������~|�C���W��~a������<���R!�_KE�������/>H�y�}�=���W��@�$d��C�����{�������� ��w��������4m�������r�/�i����u�������;F>y&x�
�/�u�|��S�������?���������_^}��$�����?����G��`���m9��[����^y�g����H+X�-U�#�5.��\a���m#���<�3�D~��h��_�
�A�`.�A�r`��2wO�rZC2�+JZ�w�s������k�Hk��=�v����w������1�R���v ��� ��~�����#�	��sK�L�2��
f�m�d�4�w HGI�Qf�/})UNWN�E8����Fg�����[@�4��i�$�������pD�
���^�m_%)~ ���!�>��\��|�0?�t�j�N�O|���55z�C���YK�p��X�PYa9�%o�6�i��-<�L��\D3O�:���y&C4��6����y�$�����
�J�����U& X!��U���~Ds��(���hzE4�����f��3�4]�V[�e�"�� ���h6C4/�
?�h��}�������u'��F�k ���O�m^���������U�������d�����"��]����z�����(�:%R�"���W��h8�0�:�=�<�#Z w�w��Q�vL��:��uz�K ��@�%���*x��������0��6
k�}VO�L���#4����m��3W�37�\�������<��
(�[���{����Iv�Y��"4���Y�FXnaYW�3w
����x�33;>b}��rP��T���	7�����K�<�<�s������6�2N%(�)�}�1�n<��V|�T^�i����U����3����q�����6h��@��b*�)��J	+]����N*��"zp7G��Z��j!�������=9��H2�Q�
[��	�$����Z]kh����/-g��<�f�m�Y7PN�����P7G(T	�=��*vs���sn��\�i��F����O���+�@p����^�C_��H�S��s��g���y��R\�����c�9�C{�;�n������:�;��\v3/���/5����������=�-���2�Y)��,���BA�C/��u�����"�y�	(�~�c����APs9*� p����{
�k�L��F9�vv|�;�n����="��"���������x�Q>�������X���x�3�QLrV<Gi�M,��6(
������>{g�Yd���cq��BPsY*g!����{r��!6�2�)r����w��8�a-%!��@x��@���xU�9����uu!�+��=C8�s@�i9+��B�x-�x�m�s62��=g�r�9o�s.���EP���:wO�m������ �% �;'�
s�$[�0���zK��!h	�����N�����h	����z��b'�g�������Vy(Z�2����FE�9�2��,�*5���2�r��M������\���S(����Iv�\�����s:jMl�Cn���s,W�8
\W��W��3k4�(����I0���Tm�;��1
�<������#�D�,t�ca��ZPsY*g-��S(�'����W�;�5%Y�@��N��:�Um�.�9#�]v3/A����U���8���\������p������9G!t:��=�w�kS=������6���?��M%�BePs�*�2 �8�{'��������K����s��(�a�����(���/{����-�*���\����F9�( �r�5��J=%Vo���������E�7>o���f��
�A�e8�����-�6wO�A�%��a������s��0�a����� 0*�n��f�� (a����������������*f��;�)�S�5=�R�s&6b��������f�
�U�j.�Ae�z0���=����b<��h�kw�9�n���f�`sQ:����23>�����]x?���\����b�s�wM8Y��GszaB���w���x�R2�79iu�b��@��
���*L5�	�2&��
��=9��KP�S�:�x�9�n����y7����6q�4�:��q������.s"D�g��[c�]c��h���V>H�E��j� �����8�����+���4V�!`.r"�u��=�aK�[���A��}���Iv�\��pN�l�E�*�s�#��u?��B�48�5���up��ny�g8���s��g�-��pN�C��6��9Kh�Q�� �e��`b+T�0��a^S.������K%�Goy�9�n�������9B)������:�U� ������`������2��{�4�*����-k%�j�\�(E��9k�{y��P/�e��� `.2���[{!zwO�4�/��"hBc���s��0�a��;��������,�������?8�.seD�g������R+x���`��%��>F��wM����B���4�i<����=��J���E�\k3�9�n���6{�� "�o]9M�
s�P����$�.seD�g�3>�Q����h�b2��1I�>FUWcR`�-#�K#H�v�[j�

��  ���M;���a���:�G���;G���Iv�\���9`�^`�$�qyh��z���$������`�@������h�{6���a���p�}����q��������X�>Th0�
AI�����-�H�����wN��:���Qj���\\6sKs��{��*���`�4���3s��H�����wS�����Z�s^�0q@:ZO)5��VKj�

��  �A�^�.��������L��wN��:�����A5qp����C{�O�gs�k�����B0W�At{�BG��P7����*��������u����UJZ9��[fAhu�
\i�� `.
r�2F��{�Q���q�7R�@��{'��s�
�l�Q�v����kh��8�K
�iu!�+� �=���$�[�&J1-8G�N��
�tt��+�sh���\��� `.r��������\5p�(��Gu�����s�#�p�9�73��6`.�X�`�uu!�+� �=C1A�b��q�����Zm����i����RN@�m%j��7���e(�Ux0�9�eQ�hwwOA��Bv>�	�Y{�9�n���V
"�1�~����7�9��\��eu!�+
���Gs��E)���m�u���Yj �V����j�F��s���}:�;BWxz.BB(z:e��B4������g����Iv�\���V#Gt�X���fG�zh��-����oY]��n��y��^��jM�b&� ��8���XW�A(���Q��,��pn�~��� �\�.���b�9��0���s�t���;� ^��d7�uX�qny_\�x�oL�z���y�:��I��\���JZ�-�I/���E	�_��N����������,�nYk'�[^�c�t���� t>�?��v���s@ ��=O[4{���Iv�\��`.��R+�8(j[
���Qb(X������B0W�A�{f�(a=��Es��(ap���l/f�
%J,�8��mi��'�WP��P��6��BC	e����)�����r�+1�wN��:lz0�-a~�����}m/cp,�J�[WB�BCI�g8geN�VR83mV+�K�����W����<+}F����8�pt�UW�I��~���X�����;�`���������������z�Fz��cI��V���R����g�Z~����P��N����X#��fi�k���@I�\;�@z]�������T�����JCc9��M��v����DQ���������}�:���=��d7�uX�������z<~R��9�%�D�O��\����3�h�y�H������y��`/��
0(S��3��������6r�*�$zn3�.�j�����zG~�OY��C�F���s~L3�q[-�����Q.6]����VB�r3I�g��t���=E����p~i
�d�'�X?�C�!��E��n:C�c����Kb���������6�N�rL�3����W������F�ku[P}�f�>��a_6��
2�Y��:(�t�<�32��iK�haV�U/�r&����>��-s���:�"�Q,x��:96�3�V3���<m%A
z�.jp�N!cEI��P�^R��s��(�a�XN��9-�*5��}_ZYm�`,�������ZI�=�J�AB��]HI��n��������Y�u:�s����pl��T:I��N���D���N-��)T�Y��r�G��[^��d7�uX+�3~�n0��`[������3H,����:I�=���
��M�6ZX�j�h��_��J0Q�Miwd�[�_f4�GOc;ILEO����4����z)��)�X���	��K����Iv�\��j������rqy8cW
�`���c�!�.�r=�~�p'I$yY��e�qYS�����>i����!��0�+X�L0�7K�X����i��r�&��t�E��Ne��_�.�p/vN��:��r.����A���)�G ��IgG��G���f�g$ADad.�<��Y�RzG
�9����������EV5vr���i��r�� �I�r���sn\8r���y�9�n���&�5�����GcHCc@+��=wQ.��\AN��3�yg@�D�YEV�i���U
�6�IX�-���B"lTVI=�*�3��`��4h�����8^�i���BC�������k����KL_�;��s5�17!�����`��~��wv��e�<o�1�|��l:j�������p�����3���`����\��)�i�0���S��������I-������F�ks�����������r��Qsr��.�r�C�g(�c����zZ/�q����mgO{2]�x}�k���s~��\��`��L���a������
�<W��$%������Iv�\�5����PB/��qyP��
���25b]]�
��~�p�[���NN�l�1���N
����}��L|��=T�`����\��)��r������\�����s��(�a-��C�y�(��-!��FX�"a?�.�r�C�g�e��j��3��4Y�7��@7�n$/���-st�0j���rW�+�d+�;��`s����4��)4�)@<�Czo�����Iv�\��P.��C��e<��c�`����x0�V�A����y�A?_�������r0�C#7J:�����9�B!�m��yLrl�����\����4�KC��r��X��/{�9�n���f��G4��+l�MyR�u�w\����.�r%%�n�������3g5�Z�$��o=��n	P���Z�%��F�R�5;v�����\����4m�DT��=�`�����*H���Mg�����ku5B}��}h!Ch���������B0W�?t{F�/��9�hM�?�x��.���i���?����BC��v��l_z�*0W!@��� yJ�S�9kE�q�����������kEsz�`�������6�}�g���zp�uY]�
�~�(a��B�����D�`�r���B������h���9� �����\�a�2 l��@���4�Snx��|4g���y�$�a��Z=sj�0/��_�I�������EG-�>ZW����3��BK�������S�J�f65����26Q�9�:�d7�a+;�a3�
s*���a.�C!2������Iv�\�5����|�j������F	B9��<y]]�
�~�p�5�����u�����:��y���P�Zy5���<�m�~87Xh�V(v.�� $��WA��{r��:t�y�JZ��4��w��8�a��U*7�tN	�!xyH�t=���r����p���������?-M���7-��&������8/�U��x:���{PS�cE�l�a�� la������=9��
�b�����I�y�$�a��Z���N�b7�FO�c��u&�>0�R���\iD�g(�i���<O��t�H���X��*LQ�TI8KYGKu��<��VHv.	��$�vF��=��9�(�#��c�Ru�9�n������������V�(Bh!5����`�4��3�z�!-}r	"��Zyv���F�������{��f0�UXv.�X�P�>�'����d5/��y�$�a����$~�����f~��LO�u��, j�/��\iD�g(J
Q�k�rZ0��@Xf��a����S4�B�o�V�@����I+Vh8��
�"��B��BK	+ir $��:w�9�n���:�9����9�1a�Y��
��.��^���XW������y�����,5W3L;�s�
�9�������A(G��A�	��� ckX�A�\f<������a�n4�x��V��;'�
s�JZ����>jmo��f������D8.�.��\����F�����N��� ��;���z4G�
O���9�����Gs��A`��@�Ei����\l��t/��5Hw�9�n���f�
#����������N����39�B��.s%D�g�0���#�����J����MOM!4`NR6�M�����}'���\��syX�A�����p6'y�7�<�����s��0�a-�S8F���:L�8��7t�+���qN^
�J<�n�h�&�����i%�8����8y�s���D�Q�����.A`��sy�� x*���{�h���o�����s��0�aM�>�8����9�z�cj�Z
?I*J����`�4	��3������M�A�(����l�te���\�A�^��%�P{��X!&�� p.
�� �;�,�{��rF	���x-��$�Q��ZGs8B;�S�u���|�9=��8\WB���3 tb���������OB��������v��Y�A�*rp�ZaA�\�,,��{
0���&���J�6������k�Z��E�m����zC�u4�������5���� �N��������i-�m#Z�>���:�A�f����`�~4�c���B���$�'AD��������)����>��d7�uX�;X�9�r�@��p%�����#�"�5�.s%D�g(g�Q���!���x�W�=�o�����xE�j��:Hn�����#�+(W�@�\>i���s/@�w2����9��!7�t
^��d7�uX����fU(�[gB�m����q"�8�&�.�r�A���8�����?J�&
1��	���u���0W��ty|u�b��]�~4�*�K��']�F{�-��j���P���lV�d�mz��$�!��Z�m��q���G��\���BW�?�{FI�,Lg��&�N�>�LP�8���A^V���'���^�/=��8W�?���'3�C����G����I��EB� �y�$�A��Z fD�!�3����j�+��>�K�������<������<T`���:b���S'[��;��,o������dl��U�n.������Z�N!�S�e<Z�:��d7�uX+U
C�4M�
m�CB��r2XJ��lwY]�
��~��4dA����'�����q(� ��M=U�yM�e�&1�*�<��U�n.��A!Uu�2��;�yf*R'�=�������!��6�����z���N������X���"�����&��c�Gq cY��C�a���T5��}�����
���%=8�Gq��c������
K{J��^��=����k�qZ�U[No��R:V����#�y��������(�F��0n���\1�����||�V�T������	��Kvl/��0�\��3yG�C����%���G]�/�L��:���!�U-��v�:9���8�)x��Kiu!�+��=�����
�1��8#�q�DV��q�r[g�b��S{w��U�n.�����u���A��}�L���9��d7�uX3S
#x�����#�:���>���uu!�+q�=��i�-��&`�X,B���z�a�W��6�2��;���8���s���Kup�A�7~kk<z�Po��!.1z�y�$�!��Z<k�4����<R3�
q`��odY]�J��=���� ��i�r�)���>��QR��c�M���7��Ksp���Ksp.�8���g����aw�X��y�$�!��Z�a�����my�q���8��ffZ]�J�n��4�QB~;�3�53M�8N��D���<(���`����1kw�T8n.����t������ �&��	�	�x�3�n���:�9������_w�qu���<�8�!�.q%�C�g���Y/����4�1��(Z����6\-7N�I�������w�Tn.����q�4~�N^nX�����Q
pM��:��)5 Se���xy�Jk�_5A �zCZ]�J�n�(��Gy%��2-SJO�I�i�%��L�9T>�q�(N�a�a���\���	�2:P�v�d��0$����IvC\���
�L^�J�j/r>�(�J6�����u ��Ny�gx����B~��5B�sXJ��)����4J����t���{;9�0�W�
~.������!����A\�����k����g���a-��!�qF�Zo��aSB�� A�h������&;t{DX�����$�����_��o�e��*Qp��_(���=Sc�F|�����|�o@��j��;��8�G����]=��d7�uX��p�^�_��6�2���8�D~C\]�
��~�p���JJ�0qx
3��a����q(�t���H7��6c!��o�s�
��o`�r�{��i.��-�8�3�n���&_� �K�a;U��������Z�D5�.q����AJO����)��q�'���:U�:w,�e�
h��[��8���+�?���sz��Q�N���
���d7�uX��#X�(�>i�g�:�)T�2���2����<�()|�p�����
z�Ty��VM}��E�N����������U0��o�s�
>�7pgC�����K�������Ivc\�53�!a��m]��H�Wu&>HH|�uu!�+�:=��s�� ��t1�d)�j!�Y?��J�RI��e�yp�Z�7�����X��q^�T)���3��O����IvC\�� ��`�r������)��7�JK�il�]V�����3,n��%�Y����AL����uU��������	��AcG�
����7���`yd���S��L��,�)�>��d7�uX+SRop�����V�����(��@Y�iu!�+��=CQ��)P
v����eP�B}TMLTuH�q�O���I��*�?���s~���{�D�wrQ7w���L��:�qZ��c��e#��p���zJ*��q������
���0p0��Jz��*��1n���7��j���3	O��V8d���W�7���_��
p>�N!Q�(��X�����IvC\��U;h���K��X8t�z�J�Y�����8tz�
�
Qg��0m~��,�H}P��-ms���#��|ta�U��B����BNo������)&�1��9��{&�
q�`�7��ec��-��_�(-r�%U��t��=�)������Y��g��D��O.�(���[mh�b�A��B�{�S{*��0��rz��v��S6�������Kg\����ke�F�P����b#^|T8��z��fy�T1��BW�7t{�d�@�<&��U�X�r��T����!��`)b����q~C��o~O���w
a�����5
�>��d7�uX����7P��������e71���� �4���3J�����k����F�q�{���8����N+
PdH���w�>V�<T�
a.�!��/���;z��m�Q~L9��d7�uXKk��L��gM�q#C��Q��8�!�.q%~C�g����F<����TKkd��-�v���*��,:�@���O~��V�
a.�!�
N
��w
��h��GU��{&�
q��7�0��ow�_����o�j���o]]�J��n�(I`(#�q�s��AJ��_z����c�(�u�Vj��aSN�	~C��o9��;�b���a��w�3�n���V��=%)�����!7e��q<��F��eu!�+��=C;�FY���;�0�b7��qng��D�*�!�Q2.��t��F��0��os�
!�7������<S��R`��u���5�n����q\��N���8U�Qcy����a\i�C�g�0&N��=-U��'�C�A!P6����|�5��QB�]������C�Kp9����`�9z'�8�����S0���/�L��:�q�cj�Am��p��!�!Niz�q���.q%�C�g<��n�����ES
ja���j��l�q�%�(�����3���j���BNp`&����{�@p�\�����g���a-�;��(��bw��uu�_4(�q��BW"8t{F�6�L�6��LU�U��x-�FS5���}�����v��;�!Ta.�!��!�N����7��/�9��d7�uX�3NTB
P`<N~����N�9\iu!�+���X3.hnV��]����_����t��@�&=%�-�GI���d���?�	r�������<nwP���3f9W��d
^��d7�uX3��9�c��<�e_/�"E2v�,���\�U���L��|ZM;��q���%8��1��eUV�Q�G�{�=Y�������`.��8��T*;���"qJt�zGs�0]����2�a�����1�`���uY]	�JD�~�&�'���G��q�F��M��������5
"�E����"���V��0�sq.�:p\�B��b�J;���;�N��9�n���V�j����2k�/���c���s��J0W";t�3�9k���i*���.9������Lh��>�h#�<pV���a�n�������9�V�r���y���a�p�9�n���V4g`����������^aeRP����s�C�k}T������D�|�m�tN�����D��C��*��x�OWe��
���<�����q%H�*�s|`����<��d7�uX�YFd�����_��z)��Y,�(��VW�����5L��r���`g)������������@�l��J��������\���?�s���P�;��0��r�0]���Iv�\�5�z��U���������	�^H�BKiu%�+�]C��by����4�B��9����m\W�C���.I-����}t8W�>���\N~�N��������)�e�p�9�n�������c43�p���	I6�����@q8����s����aB�l�N�sj	��zJZ���ms���Gz�9�n�����`.������S��4�Z��� �>��d7�uX��0�������;�;���zs����+S��J0W"@��F�#�4�~j��h.�B����-�s�Ob5���]��
�?W��������a>�@��`N
��������$�a���0G�Qk�R�v�~��M�@��,�+�\y�C�k�ef�F�u����N�����q�XO�<Y
� ��r�	��7�aI��~0Wa@��B���ng
O��w
3�-��G0Z�wM��:�p��3zv��	��:���e����<U�@���D,������\p��%}NYh�E\.�����*���3$����%��{z����y��������K�#o���?~�������O^�%��d�(�$H���6�~��'�PK��P��1$PK${2K18sept/pg_part_patch/12_2.outUX���Y���Y��}k�Gv�w��	���,r2�8�1&�Z����H��������U����]U�{�
r�a��=�9��Zg���������o����?�����������o�y���w������������/�3�9fc�/��,�������/�������w_H����������7^�����x�X����o���y�����O�}���<<���������������_~����o_}x�<� \0�q��:�s�������,�����>����7��>������8���~�������1�,��o_}��������|�����U��VK������u����g����{����?>�����o���A�~�w�~z��w�������^����/����l0�i������,����fA����2�W����������*O��["53ryB��C�%pE��+_��Z��Y>F�q�����d�)=���������7�_~z����������{O�������;�������|:�d��/Vz)L�>�pI�T<�+�k����^�O��~x|�M��I�b�?�������,������~��w�������H����Z�����}���3��>����-Y|v���[�||�����7�y��!��������/-p�����7|���?~������}�Wq�o�,w�9st�0O�#��	z���H��;'���~���=���|��'_��?=|����W_���_��W���������������z�?��g����'r���~�=}?~d���7z���������_��@����w�y��/~%B0��������O�u�A�~�������#��?���r'��O�������^��������_��<��_6�)��q�����\o�Tt�Y���_���%d�"���C�!���>�iy��S�����??~��������y���*���a���W�>���l�l���������y���3����)a���
s���E�����X����4���\Ba/=���������Y�d��\����&��sD9c	�,e�����Iv�\�5PNrc�����,(��^��]E9z����2����r��r��!�#�P�k��?��*�����W'�t�-v��I�5'�9�)^YpQ����
��
����������9����_"�3���i�$������F��.l���U���@i?t��G�j��K��4�i�i�.�G�a$����rH��g&�/�Kk�E(n<=�Q������wc�LW�L��3���L��gp����B����u�/�n8������9�����n�B��g������CM�6��3������6�Qbe��e6I��gV��3S�33���<�o���3�8�<��=�D��_��p�a
8�j@�����{��>
<�P<\���z]�Dg�3����?g�yJn��Q=�&P���������I��pf+pf���=$��l���g
���1H��4������k%���p�C`5��p8���L�Q�����LI&��Y�3�T�Yz%#�������3W�37��!:��8�����eB�c���s�/�n8��Vt&~8�C`�d9]��9@������T=��Y`�o%oY4����W;	����?]4_4?��1>��1��3�f��L�<��Q��_����a��Lh��0��ZOW4
�(:`�r,��@���y'�:*"����<Es�����E#j�G��Z��j����7G��h��U�:W�~pN��:�5�f
���C.upw�Xy��PM�*0�Q\]N���A�g8�<!�2�3��9��Q.�i�T�������1�`(��:N�+��[.j��r��Q���L��8��.�;z�4��1i���9��;'��r�@9����R�b����n��~S��6H����t�{zY�����(�FP���r�����n�A�:�aWx�������������\��8��F�S�Q}Y+4�%}yK��;'��r�B9�F�r�a�t;������\�����t!�+��=C(G�X��:@`��rJ2�X��:�e���*�)�)^�hi���|���r����:G���$Y��w�(�c��F��G������F�k��P�q�������.b���r�P4aP~ZNB����3�r`.+`��jV,�Q��rR�q��(��r�0.�����!���U�b.�@�u�uv|������;�Z2wb^��d7�uX�[��%cNmy��E,��c9O��L(�NB���3��4P��J�,�
1c�����
������P�0P�P:�.��S"�BIs)	�@I�����{�0��tB��1�s/wN��:����(��(-�w����������Gi9]�J�n����c��1����b��&���	�d���:znI,Ia�'��;�+*L1�� �L��C�;�`�\Jo4
sx��<��d7�uX���\�r�Z��.��Y����MU��t!�+��=�P9(���Y(G��N�9�w3�J�sZ
4Q�3���2����$�B`s	�@`��;����B�}kr��*Oy�qN��:���R�:��k,���.�z��S�%Pn9]�
t�~�P��#�	
����L��`�i�Q���&*������edi��������,zQ!6���q 6XI���n�=G=�9G@���,|�r�$�a����� �l>����\��;$q��c��B0W"9t{�`��9G0'�,�#)� `��wTT[�9�sQ���'�)�g_���5�U`�Bws��@w��z��9��6�)�yOA�s�����k���,.J�\�Y�.3CM�.�F?���[<]�
��~�@�

�s�J-�r��2u���=�.��f��Y_Sx�6
E�
�A��?���JN��G���3�8rVr�����;'��r�B9����F&�<&��C@�����\�����H��:�}�s�>S�b@y(L4�z!�m�N=����c���B�s��H�@`'c��S@9C?I���0wY_��d7�uX�T8��h[�2�7w;�"#��9,B���t��=�,���!��Y��I�[�$O(��n0X�FeNP��&��Xa�&�dsYe�� ��da��W��sLY�Fe��S��������9�n������9��-1��U��
Vb�
m�t���������9�B�)�������-R"����@��ce��o��9`/}�*0W!@��y @,��]���#�9'�������w�/vN��:�5MbF�Y=F�������T`��`B���.seD�g0M�d}����KR4g�Nk��FBR4E�=��S��d;XoIVr.B�����=��`/X� "�T����������ks���K�q�fr�>+���t!�+3 :=��`4 �7H&���L��K���[Q�_m
�����1��\��f���K������MRf��#����9+E�Lw�9�n���V0�����[6E�MT6���z�J	s�d��B(W @�{f�Y���n�50�Js���������uH����<oX�b,i_Vr.B���Yyt�1g��2U���s,
��w��8�a�h���V������67t��r��`��.�s��
����X���Vz���c6M����6����s���/����p���|VY�@��y�@(���q�j�dP	�D�y�$�q��Z���
XMC	Y0[nv�a>4���b&��.��\�����Ai�BeXN�9��ir��nE���qaS���x��M<�Z��Vk�!�� ��!4�:��9�����2*�8z?�;�n���V<g�}��I�G*���A_q�)�s�t!�+� �=��s}��|�&��sP{��^R�zu���(+h��+w� *$9�!�$,�\�:��)�i�(rc�[��U��s��(�a�h���(g3�Y��'��A�pDs��B(W"At{�V5�����$J�	�k�8�Z�F��i���J^u��D>��*+,9�!�,H��`7����RP�B#�c�����������ku!���r���,�����d��B8W����U����-��0��Y������P����A�����4|��$���P����P�5��V~tOi@X� �m��,��r�$�a��ZI��#FJb��aOPu������)��sO/��<�f+�s��AzR�������\����F�S~���Y+!����4X;XUxj.B��@PTm��)����J�^����s��0�a����}��:9�c���6&�]45��B0W�At{�3	Z�2q����L����X�fJ��� 8�Kk�g�re��"���s���Psy��ak�{e���9�F��'�S��wN��:�sAh�R������1�bC��GJ �f!���.sD�g�aT�$��j�Pv)��i���:��[�w*FsA8�5���^�0�s��5w�DFJ�k3G������;����R��������a(���0�����a�?��tn.$����<�3������9+���9�� �����z��AH�E���DP��I�VU�(Qs'J�a���%eZ�S���I+�=(����j��a�5B#x��,lm�x1���Z=�qQ89]�J%�������PA�b�*�l���~]�n%���7�C�)�jJ�E��&�X"��L���%�8Q"���s;��<@�`.����d7�uX����������@�Fm�<����B0W�(���:1}�3��D����ni��Z���^x3���VJ�5�s��5w�Dt5�Go�{�T��EIf�����0����3P���O�;�:4���VN��`V���\y���3cisP�vj�.��_��}�l�j�Z�X)�1;�Y�&�������V]�'�s�I�q��>���w
o��"�Qx���, �r�$�Q��Z����@@�|%��\g��lu �d6����:(���<�3P��a�p�r��4��&���=�J6���s���PnM�>����������$��<�
�wo�;��I,��`A(.r��s��0�a�`���s��Up�|1�m�r����p���B0W�'���^*��@�7Mp�2��f�a��r%E��D _��WQ�E�j7N2��+�$z�8�~:N���m���w
(���G�C�����Iv�\���9
��p��*����^�����u����.�r�q�~�pt1�Q
$�������j5����+��U���Z���saN��&�UM=WUSU5)�f��S@9��@�wN@y��wN��:��fuf@,$K,��*���r�9� ����\AU��3HYc��	f���TK��Xl!��-��U(zW�xc:��M�c�^�"����j����#<��2\��*-}9E�Q�Pj��z�s��0�au�s��L/,��^y��`XM����`� ����@������L������V��Fzh0�0�"���`��C����PWd5�\YM]��$O9{tOA#�Z��5���+�;'�
s�$�;F�i�9��*��Z��9}�������I�^��yfSa� �L[!l�E-$ys���4*s�+�-
��3y^���������OJ*����u��B���8���4a����Iv�\�5+sb�
8L&��:WQc��C���.�rD�g��7�h���������MMM�F�z�*�0OaU����0Wa@��]��$�Z���9��qEP!6G~���wN��:��I7�a=�+��m0!��Z�\`��b��B0W`@�{|�������Y�$JS�������SN�0G(#�{m����������
B�e@�"������c4�F
`.�&�y�$�a�����C6A����x�ip�:By��	$�t!�+0 �=��:�����i�}(E.0g�V5���FiN��c�Zza�����a����s���p�yytO���9B�IK�k�������k�����E=����Yd��/)�0`N���\���h�	$�h������V��V���@V
����v1�-����Es����9S�@��s�@(r��5�sO���|��;��<�������k�\�c`���2�}��6jsA2?J��:0���<�3�#�c4g�����I��6�]{/]��Q��u��)h�e2;6g*3�a���PdY���)Ds��o&�RT����9�n���f��F�����{E5^_��N�n���`�$����/DB)JZ�����Im�x�wR4��r��Bs��n:��]a*3�a��&N���{JL/�,���c��?�b�$�a��Z��Z�a�L���r��z4'��Xth���`�������c�n�;��zB�7-
��-��<�j"8s�(��9��w��r����0G� G��w
��j2qj���������F�ksj�p������n�?/�<���3��V�t��8���pp� zf��
2��A���8���S1^FEM-�f�:V
��
���@��J�w�������X������wN��:�56'F,��|�Y�q1��$������t!�+p �=��U-RsV��@����^�������mn�� �FF�����&��S�@��s�@�c�ptOal.P'���?��d7�uX3g!L�d�@���b���F�&��B0W�@�{��A�"L�GW�<	�c���cs�M�*���*0k�d�`�9S!A��$s\��d��Sj�b;��Q��
?��d7�uX+i��
(�A/u)@��K�T����t��H��YI�`hzZ���7-��vL�����+�U���Hh%��7 ��0�
���@���d��wJ����N/!,3YQ���Iv�\���9����������.�����b��B(W��������)�r�6�JTS��X���^���	����47x��p �\�9r �����{
0G���;�A�
��wN��:�s~��&����j{r�T�I,bD���B0W����u�W�N��ID`|��ne.�U�D��k�\�=e���>��r
��K�0
�u�w�)�h�F���pq�9�n����:�!�$����U�'��}��*����z����@t{fE9�	�I�o��������{,yK=*���9�
�]������B��s)�@�0V
���=�������G�(s�9�n���:�9��������,a7��j0G�{�����sO/��<���1���4IM��|�9���*sA5������EBl�~�c�����s���0�)r��;�����q!�;��d7�uX+eC������ww�T��Z
���B(Wb@t{F2bH���?�T��2ls�]r�X\(�aR���@'Q����`����s����
iytOy
�����������{
�/����YCd�o�����S�"Lt]Pia��B0Wb@t{fe@��n"�-�}�LSV6`�@M
r��
�sjp4W�@��[�@`��>��sZz���b���^��d7�uX��#�����o!�u&4�Q�i9]�J�n��4�>+���y}V��_`��u�B��9p���f��I����P �\
�=R �%���=����s|���wN��:�I��#Js���4'�q������Vi�C!����`����3�����0E��|Vhc���Z��p��9h<s�I�P��;V��V(v.�>�@h�$}%���S�@pB�� �D�S�H��$�a��Z�9;B9�K��F�����Ss\0�@,��\����A�V�������i��<�l�
&'�����2�n�����B��s)�H���G��9;�p���{8��d7�uX�����h����m;��2H
�zmN�fT'����{��a����3������0N�L��^�mZ}�]�C�!�ga�D��
v���p �\�=����-������fo�W����s��0�a��`�$�����������b'y@9{=]�J�n��-g��z~���b�Q��[��9)Mu����)�D�B/q/5������P �\
�}2�m��zC8z��r`M��8�:�n�;'��r�D97���^�I�6�Y��Q��_a�x=]�Jk �=#����f�.��L1�`)����\UQ�zJ��LB��$���Scu�m�a�R ���n�(��;A����q Q�=!����g���a���2���2I[�����7[O�����3�b ��iL�y� �
�<����Vt�}������E7a,��
����?8~�8��mw��;��`�!����y{�3�n����I=���vo���u��a%:�l=]�����ys$q�����iQ�#�G�3�2���L���8��emqy ����W�>���'�����k�t}�0��)9�P�x&�
q�JT�M�B�q������0�D��t!�+�=
��Ez�4}9�B��v'��t=Q�DSl	��j
1s7x�����\����D�eM��w�M��A��'��Nx��n����%�A\<�����l}�)�J��BW"=t{F2%��!�<+��j�FbEq��D����:�d1tbv����B\����r�:Fq�oSr{�jq�*9�cX��g���a�v�� N��
��m�[J���`��@\e��BW <�{F2��R��%r�Y�W����E|B�����3���5=~'�f��
����;8}��(��� N���C?Fq�3�n���Vg�PW}���,v'B�zG�Sh����� �Dv����G�����i:$J3�@��y���qJSU�����Gq����up�q�yu�����/Eq����g���a�D5����6v~<�*N
v�R@Q{=]�JD�n�������Hz��e"
�8CQ�V�������RD���Oi�KT����
����98{�8Aq�N���X;����L��:�Er0z����[�I�]:����4�l@h��.q�=�����k�@���HrS���rB)Ul�
!A���������8�
����88w�8�ro��)��a�@CdSh{�3�n����y9B�\���v���*�Y
�4	��t������\U8�gyM�L!N��VP7����a)=�8�<s|��Pg
q���Kpp�q����?z�q(�!9}3
�L��:�q�!����<��v��_��������t!�+��=CA������:r"��:��S
'��5��6�����<5R�7���w�7 ��y|�N���u,�)����g���a�0N��*��V��q����8��	��.�q%zC�g����;�,�	��q�o�(��z�*��>�4T���,��|�������������w
���#��X#,�y�$�!��Z��`�,]�Z������VK5(�n[O����7�y��(�E�j����M�h)��R����'uzW��eE��<5���W�7����78�wy|�N!�����`(����g���a-��C�q�������R�jU�G�[\XN���^�n��j�
B`~d�v��i���h�]b��-U��2� m4=�Loc�F|��������`�����w�j#�a����`����Ivc\��0N��8���E�o�)c�P�z�%�0P����0��o���^&�4a����F��U`��m������q�i������|���q~���o�~��Ms��B����f?��d7�uX��/F�9�a+������u9%i9���A\i�C�g|Z�Co�a2L�Tm�E
��4���t�x�����]s�E����S�~���o�G~��Y�~���d�R
�0?L8��d7�uX��%G�T-�y����,�u�CaE�(n9]�J��n�`�4Fq��o&.sX2U\��gU��98����a�0Y���@\���������(���w��*CO����F��L��:�����n����J��Fz���BW�7tz��c	��{�fA�`V/-U�x^��HTE�v!�+��}N���D��o�s�
��op����y=QLI�F}���IvC\�� ��`�b�K���}8����B[�u��M�A\������  �T���JTET2%��F1���l�]?j��j��8���g�*�x�W�
~.���
 R}�N���]�q�0�q�{&�
q�JT���<���6�B:�y����b|��t!�+��=C�c1�+��M�������	�����'��B�XJT�������
����7�#���S��9Fq�	lV5V����]����ka�r��V=%wUu�X9�[3��t!�+�=�	Q��c�v�+�>�2'�b�z��-�~xA
���[>���W�7���_X����S�Ty��X�� �{&�
q��T���
.�T��q��
�P�a��BW^����8���A�?90��S����������������=\�#Y���X�B��������=Vx�s�8K.�3$������{&�
q���7��eb����_�(�X�F���qOo��<#��Q2���L50�(\P:S�x�	�~t���*�J��m��B�������`0�c��)e��&2w�T_��d7�uX+S�b�,�g~U���`Z�h8(�4.�z�����1�7��=��Ya�tq�*0N���!E����66$Z�</��f,�Ua.�!�fYLQ���%�����%?��d7�uX���p�|�6�e��Q[-�;�=8}��t����7t{F&�)��P�i�5�	���Lm�q�76Fs�L|��g�����Xq�P�7����p�7x�6���;�bf2���u�3�IvC\���F����}�5�QAV/2�!���z����
���F�%�}�i[���K�8�,6"��8zxZxA��a��6ag�~C��o~8������Z�^��y�$�!��Z�F0�MV�5�����/��?�F��� ��o�������_r����K2�(�����_��R�,�Y�P2�M���b#��os�
��o@Pf
�)Dq"J�a����_��d7�uX+��j��ga���}���Db���Es��\�
��~�F��8ol^t5c�*�?b�2=��3
�f�
�L�b#zpW�7����p�7XgwR,�;G�SLC9�H������&��q6?�q��v����5t��s 8,�a\i�C�g�x�����u����0�B�e��
u�(�"�9a�@pr�,f��\�C8(v%�:z��nn\������3�n����Y;����8��B�"���.q%�C�g4���o���ik���oZd���4��W9\�i�	%����b��L�Bps	�Hp?+�f� N���pL~�3�n����I3��@A��b���uU}[$������.q%~C�ge<X�%�^�Z�Jq7���7��n�o�6��k�VN��F���
��os�
��o0����)@���bf��:1/�L��:�9��)Y
P����_������������r���
��nV�� 9���\��f�>����Pu�S����;��]'�B����`&����������(�/V8`�����s��(�a�@���g~�����.�zWUR����a=]�W�y�	,��q�C���T�-�3t=�t���:����E� ����S��#9�+$��\�+�q �����c��3��8@3��y�w��8�aM���CDG�J���b�4�����.�$�NW�����5�9 ��:�b����Q�n�vt�M}BN�����e�!7���*����t�������j�wPI!%��X5��s��0�a�������:$���^�z8'�*�-�+�\����������&+,FM��t��"uV>]b5��D�)/��a���+d��\�;�A�Gf����(<6�d��9�n���V4��y%+��W����wX���G���t%�+��]cP����)k=;\�����b���T�\������Z3c�������W���0W�<X����*����`�#.s�_��d7�uXsVN�HZ)�q���G��e]���=�q�D���������5��Ds�;:M\��h,a3���d�D�I�VHM����ypc'I�����\��������::��y�-�cC!�wN��:���Pcv���c��
��6�]��b�5��s%�C�k���e�@�������J�9�e��7�����]�Y��`r��kt4W!>������`�r�;`��ZU�q�8'�
s���)?F0s��|�MH�i�hNZ������`�����5�Y���&$1�� �/���n���aX�=�t�7�z��n�����`.���^v�(iGsx����<7�r�$�a��Z0���%]X���g�w���X�i�$(���s%�C�k��]-�r�M�(I��������N+���S�+vT����P����\��G����T�E�DT����s��0�au����iE�n����]V�s�=�(��+�\y�C�k��!���e�S`�������"S���2��%�r�%����rm�8=�.W�?��B���ng4Va��wJ����Hi����Iv\��Nh%O/[�A����i��r(<�w��,�[O�wn���~�7I�����Q���
oO�B��(�d A�P����z����o������r��������������S�C���4Z0��?����g�|��PKi�K��1PK{2K18sept/pg_part_patch/12_3.outUX���Y���Y��}m�%7r�w��Y	�|-����7Nv��F`�b<��;��gFq�A�{���f�m���&!���D���9������<�������?���z������_��|l�������~���W>�����P�w�{J�TQxx��_��������J�/>���O/^?|x���W�Z������/z}���o�~�}.������{����W�����wo�����{������w��������P!�wA�Rz��_��_�����Wo_}x��O�~��z���x��c�Y~��}����w�|�]���������n
^���_\
�w����}���w������7��~�`k?��?�y�}�G�����w^����-����a��
��&�V���o�^~+w��>��x�������7���h+�w�z�_A�`�#��+H���F�oN�1i���_$Y|K����^�������_~z�����o��������VJ��������o���z�;��/`�*��>�(�$�B�^1�� ��{%�����������}PF8�Io������"x�[�o�?�x���y��M�=�-�J	e�wh����g����
����u�����w�������O�`o�}�H ������}���n��'�M\�x���������/_�yx������#�E�������h����>�n x�9��������?��KG>������_�����7�}���������������/���_~������_|A?����z��z>~��������������o?������o�{���|��+z$�_KE�������>H�e���]����~$w:��!:�����y��YN��|M0��������?�es�q(��O���/�zc���,��c�t��/M�3�����"����YO��l`;�����^�����/� �D�^|�G%�?jE��|���G��`��m�m9c]�]���|�g��G����iV�sn��������
��{�!���S@��$XA��dp?���Y�t��\����.#��)���}�o����p�9�n����i��y��H�-0��bzW������o9e ������=C0��-��h�9�Ta��wF��Zh��b/��*��8�3,������
��
���g��E�<s�7�����H�7��~Ivc[������.l��O��X
�d�_�>z�P��^c� Xi�i�.�W���jO�&O9�
gz�3������z�����������5_S�~,��
���pf���#��������1'u�!�/�n8����$�9������)���p8��D��<e���!���G��Y�r��g!�p�hf�A���E3WA37����r���f\���YE�����~Iv�Y�5��������W�K4
���*h��h���e��"(�����(T.�5rM���.N?DA�����3�
��\8�C��V�y�g�K�
Wz�P�N�%�
g��58�����[:\���Y�����
&gF�� ���j�R����rMh�k�a,��
���p�������gpf
��r(��~Iv�Y���3���jE��^��g��Qtf�������,�[�[��3ptYc�����*����t��g8����n�j�L��I~D�s�K�?��d7�uX+:S:��3z��XW�3<g�p����t��.�gH��jeB5�Z�T>�������P��0��!D��Ew�N�(t5Rs�����wN��:�5�Nh
PJ�8w�Xy���M��V�����.4�
��n��k�o0�x'����r�C�)5@�����#��S�qH��t��[��E9���(G��L�S�8��
�N��3|,��p*�wN��:��r����������(�K�����J��:(���|�g���!`��h�i��B9���e�(Gq���������,�6CU�q��2��q�T��sD9cxX�_e�G2���Iv�\��P�����,w�m���L�+J�������t!�+1�=#�z��h@H�5�k�p�q`(��[��UQ�	VG���ji.��X���\��:,9��;G��\��>~�=����s��(�a-��Sf@-�"u�T����XPU3VT x<T���\�p��Ip8�����q��
�b9����r��z,L4��w�
��s�vp,Wa���u`� �9z�0���.��Iy�9�n���&��@9J����2����z,'�Y+<��.�rB�gd
q���c��PN	����U����$y��%��rJK�[��; �*d5���dvn��S�x��P�sP���Iv�\��3�u9U%��./3��OhZt�r�t!�+Q�=C�!S��Y��)��cu�6���u��
HuT���/4����N��
IA�%)�#I���B�;��5�Q
w�O�3��;'��r�����<��w�u�����W��B(W�,�{f�X5�Z�N��(���.'q7�ol�.�/��T������$Q����]P�X��G�P�r[��:h��������F�ke���@9����ww�%J�r�a9]�
L�~�0��Q��m�ux$M�p�o�>P�RE9���c�oD �y0}^U(
j.�A(
��QY�-{�@9E/����(
W����F�k����}�(y��w�Zc������z�����Y�r���c���\H����;*�PNI��n>�(�=�\��O�LV��
�A��:�����>����B��e�� )�
������k���-���[b�w��$�c�D�+��.s�C�gx`�GI�ee��&�1���@FC�Wj��.�s�*��\\����\��:�@K�E���#�iI>��`��PH�������k��1�$���=�I5���*T���x�������p[�8+��3+�c��ee�<����AX&gR�������[�����~�G�g�Y�8{�41Ga_�%	�Qm�y�$�Q��Z(g�a�E�*�9����:���hu��UBr<����rO/��y�sV��Y�+�M�Y�eB9���`c�!�#B�����
7���+�=����Wp�1�/yC9U
���$�a��Z0�G��(�a���GQY���%*�����`�D��ss��0��9�YUv����@J�PT���1�����7��Y�*�=�������r:��P��N(�������N�vN��:�5M�F�Y�{�[f����&��q�h9]����N���m��3L�=9d��?�e2�Nh	u�A_WG�D.+�z�V���ZK���s�����;^�S*�W0[�g���`�;'��r�f��!����?����U���r����NB�2���3�r*V��hf��VjU_����cP
�%���4MB�f�!n�E�
�A��?��2���&(��NifN����.r�s��(�a�X��((�����n�_�SV���.I:]�
��~�h��-�YCv^�!�*s��a�=��

����X��x���.�X�����\��>�@+)@�SS_b������s��(�a�X��.��4��qw�[s�n����\ieC�g���aL��M�X��43�5C����>3|�����^g%M=���+�=�����~���G�4zZH�4S�u���Iv�\��&����������,��0T�<�y�]���\����c�c�+�i��UISR�����N���������n"�����&k�����������Q�N����|�N�=���Iv�\��b9���%�tI�G���K��c����.�r�C�g��.�S7-c��x����d��r��O�(�b�y(L
�>T�z.�A�&�w��;�3�9
'7���9��d7�uX+�11� �e�~��e��Y�?���P�D��L\J���<H?O�$qY]����i���p%>�~���f�H9���+�=������B��n�N)c����*&������F�ku���Y/��c9����M�����P�����3J0��0�����b9z?K����r�54vvq��H@�`��?���L��`��La�};�<��80gK,�"BJ�j{�9�n���V�j��Q�t�v������}����#���u`��e�4�JY��&�$�I��V�A�sY�Nk��}~ ���j��WG�����B0s��������)�[�[r������wN��:�=17��o�s|�SCu.����j8gM��\������b���G������*R��{�Y�������E�,�r���W�6��3���K0����r��=����[��7J�����������k�\����V���k&����f]N�����3��BLZ-�����9�`Nk%,��������&*����n���+�+0W�&1s�ILa�Dia��=�h�[��@��N��{�s��0�a��9�<����X���e8l=]�
�$��qI��0/��i�$�B�����)��W�jBIk^�
�8����m���8��;Nb�$�.�����c"�CYNy	5���w���nA���Ap+a;�.��U�3�#w���`�4N��%��
(G	�4��\@����t�D�����c6+�,�g���S�'1s�I�q�����3�s�1���e\�A�=A��$�a�����2�u_�R�
�7���2�o9]�J�$���Ie�\�d��z���07����m^J��G��VVT�VM���@��;Pb
r������:/#%s�����X���/0�q�@��[?�������N��������`�<P��^t�cs>(��YI�q��2�D��I4js*8��J�9/n��c;��2Pb����@	s���SHZA��G��{�9�n����t��-�["w����AJ���t�{zY>�3J���I(}���9���l�����F7���
�RBy~�[�"�Fs�2Pb����%x8d5�����p\L�-^q�9�n���V4'a���r��c��N6���
���t!�+��{��=P��7o�
���(���nS�VA7Zx+NlA�(��9`�����;w��>(1��m���{
I���y"�<(����Iv�\���9�#d�!�qw1M]]*%�N{��B0W(����)���i��BKjS���2j������-����h��6���[��s5�QP�o���#�Y����I�������kuZ��-���������:�q�M<]�
�����i7��dr"��,�V��t[����V%�u�n0���9i��d/[Q��s5�QQ�Y�yG��sZ�����	q?�;��d7�uX�|5����O�(��+�9�XO����f�gd�$`�3L����i5�WG-��C�RR��4�����mER�����IM����=����,������:��d7�uX��`��
s��-�����Kc��"�.sI�~�0��;��,a������Zi��*��kDs�?��nrQ*}]R���c��l�a�� �S�������u�
��p���y�$�a����95@��yL.S�z�PH}��t!�+� �=�0w�Pp �I�:�Q4�P����k����B	���N+p'������0WaA��,[��t�x���s���ZP�>Dt������kEs�`A
\a���DijH;�S.��\����7�����Y-���Y�V���S����n(��:J/.�k XWj(�UHv.	�H����d�Vz� X��]�����s��(�a���!k ���LM�q'�a�%�Wxpb=]�
$�~�p�:DQMI@2M�������j�`�4�+��h�U���i�`��� �\�-� �P�SXC������(���~�s��0�a-��!;�|������EW��<3��S:]�
$�~�(,���w�����rV�rVM��m\/%����#�x������X&W!A��$w$Ar�&7�sO��j)*fAT�R	?�9�n�����sg���.3���P�Y�	�W@��t�{zY>�3��������irs,�'��\r��G�n:&&�K�a�� ����
	��%A����&�ZW���S�#���I+7�s��|�$�a���I�1���l�M�U�����Sa<���.s%U�n�h�'��P\7�A������&�jL��6��R�\�F&��U�B�psI�����6��wO� �92�����x�9�n����t�U#t�P���P�c��@0�WH���\���E��H:���w L���F���i-C�w ,�-�vp\xC_�-^q���*(W�@��w�@(r�)x���������zV����Iv�\���93B;u���q7���}�9�}��-���P������J������jszYx��pB�� ]���)�R%Y���
���@�t�m�s���P�;x��;'�
s���S#�z��r�x�_��D:7�xq�z��8����^qj���Y0���)g��f795
�F�q��>��%�P!����p����p�s*�Sh�j���A)^�(�;'�
s��YGh�5Y;�����:;��KI��t!�+p �=c� c�3���D��C�������U)C�k#�y�������js���ps9��	B*M��=����~p+��Qr����N��:���J����0k�����4�9g���z��H��Q���q���4�v�C���=%��`��/�R�2'���$\e���*$7��
� �e���{
8'Q8�������%���N��:����2����QK]�����3QV3�.�s�]���H@��^,x�����L�����d�W
�`r�r����M����]���� ��!}����SH[K6�$3�x��;'�
s��9!����YUm�h[-�{^��$b=]�J� �=CI$��s;Mk�� ��j����L`�M%)��V��
���0W�A��4W�A�]'z��c8'���;G�h{�9�n����B	�b���-7[e#A�����va�����`����3F�S�J��s�hV!��pLo]����-�t`�?��9���}�V���Z�B���48� hE	��=��9k�1Q�����;'�
sV�9�F�����,a7K�HZ�����5��sO/��yF��bR+}6���[���G����k+�SoBHC?j@����"�^�mB@�sip�A����=G���H�yq2���={�s��0�a��U
���Qo{;�.���i�teF��t���h��Q�di�g��i�\����|0�"������(I�mb{Q����0x5Th0�(�J��{J0Gnd{���vA�����U���"
|;�.f��R�a�����t!�+� �=�ot$�s��fbb��s�mTS�0�������J����*<�����W���{
�}
p��9t�Tj�y�$�a��Z0�`�f%&�o#���V
�C��.��\�������-�_g�!u�#4��d^+��m�]d���>����mA@�syp�A(��e��)�=��l7�����Iv�\�5��zDmN����63S3U���*�g5��t!�+������U~c	��)3H�,��e
����Il�\��-�
���|NZa�Tx0�Oy��/6���=�������8v������~���P��>������0����f�RvQ���`���������N������9H%�y0�JL)�y�y����rp�Z�A�\i&8����;�5����8�3�"]�����F�k�Z��A��c[x�TY�UP@	s����.�r�U����\T�o��m�<���i���C=�����5�2]�����BLPaA�\WAD�ptOa�a*+;�w8��;'�
s��68 g�B�>�
P�s!NkE��t���H��1���7���0m�����!��9�]u�y{``pe,��n7������p `.��u;�A�mh���).��,mC��Q��s��(�aM��#F�U��l�	��i����.�����B(WZ���u�8,E���-M!&z4;�t�X���jz�n~���l����X�`�P `.�Lu[��Z����;G���U��
���IvC\����
�W+�$?����K��|��BW�?�{��S>V����9J7!A���=(�U9ol��f�3q�1w�X�����K��q�����{���
l.P�k��$�!��Z����C�g�����`m��,����qOo��yF-�Yy*�����!���%�sl5'W'�{g�p�br�5���c�B}�s�^!.����9Y�@��1^��=����k%�a����L���K�B}R�I�qEh9]�
��~����e^�&��
q a'�I�Z�4o7��4�t�2�u��_�=���������;�u��F��@�����IvC\�590Bv�o���z��e!���x���8�����������=���3?�Q����J��8B�����������B\����R�9Fq�!��w'����������d7�uX�����3"��rf�����������:�z������� '��$/g�qr�8�./����E��4�9����wk��9`_�;��to�Q�2���;�v3#�x1���=����kEqns��0�]d��8�6xI�uT�K�A\����+�9%�3(<�,Q�kq��,G1��}Y�8�c�6.W��ohGq���Ku��qV�9z�P����X���Fx&�
q�LT���]f#������"��$Q�AO�A\�����!.	g�f�i:� T����������+�����f��%�cG�|�����< c���B�,[�$=�����IvC\��H����e��v��2��C,R���BWZ����8��
����M��(�tyA����*s[u�O3��y�4�X���p�\���G������B-���
��V���IvC\�� ��s�s�q?�j�Q��`��K�A\������!���p������4�Wf+�;�n���ke��X�c���";7xh�Bp�s	���<�S��X-�U����g���au���2������
�z_�eiA�*r��BWb7t{��
UJY�
��zCT��L���Bz.#�
Yhi�8��C#v���n�Gv��9���9@E���q���g��g���a�(N��*������NZc��������� ��n��EJ6�`G q,�� �7�l=C��Ak�������|�Y���a���s�
xd7Z��m�;�v����`A��L��:��n0f���QUb_TS��! ����b=]����O����6���YU�kKG65v��O���Aa��.m
]d��89��v�e7��������g� .�Ni��#�=����kA��Y��vw�R��
�c77���� �����36	�����M[E�C�i����%Y�����Ntd���y5vh+���n�#������w
Q���D����9��d7�uX����q4�6S����z��-wy=]�
��~�p-..���wjZ���2��n������=o#��Oa����B\���s�
X`7 dq��w�*r���������IvC\�5i�j
��@���N��Y�5�3������.q�m��1�o2A�Da���8�@=����zU�9�e�	�8*E���Q�
�����n�����)pT!�PlKNR�=����k���*E�y�*��`]/�+p�C��t!�+��=#�����/+�����fJT��-�������1`�������~b���p.�����\8z�44�a�c
��KT���d7�uX3Q���6����cU%��#B��t����
���(���&����`��*��y�C=Qt�^��*���A��D��n���<�<���y=Q�`��
�cu�|�3�n������*�|�<������+�eI���!�.qvC�g(=���<lq����:�q��E	�����t1�B6�Y�C#r��A��p.�����b{�NIi��[8e���L��:���i7��N^���*9��Z�S��j^U��.q%vC�g;��H�t�%l�	\�N.(���x�
�.6�5%���
 Gqv�e7�����bk��)���&�pRq�Z��L��:�qF��(�X.2wEu�X7h��Q/n9]�
��~�p?vT�xvqO���q��Sk-���:�Y��6���|�i�,�2�*���n���bU��{�P�����z���{&�
q�JT���
>wT��o���)��5wb=]���:=�1���
$q��+t��Yiv�����.����G�Kp�2��*��0����K����wGHX���d�5�|�$�!����������h>����Q\\J�����]HL��M�4�(�Qc��PfZ-.L.�(���[]���)��s�!0y�{[�vC��nGv��i8z��)!]TS���p�5�n���V�j�ML�J���(����Rl�0.�.�q%zC�gxL���+m����F�2G��&��Uc�7j���7S�|�����
�!��7��A��&���TY.'�����{&�
q��8p#�
  �}��������N�����n��`\��8���Iy�V�j�y��l�q��u�^cX�z���&O��U6zC�KoGzJ�Q���)A���?����;��d7�uXKjD��L�
�M}$Q�[�����.q%zC�gx�4���a���^�:���R#�^���i��H�d\�����j����B�����p�N���x���<����k�d��wY��}$n�-�_�m���z����
��q��������65Bq��� 0�~A����I���;*�X��P�7����p�7���K�Na0�rN��bV+��L��:����������,�G��1g(�>*���� �@o���f`��8Gw��Htk�*Bc�3��y4Vp��f �n~�oD�U�
a.�!�
�a'���S�7���3?�~�k���a���w�l��xU_P�xo�di��t!�+mo���G>�)�����o@�	�x��8��6��Y���#�����XQ�P�7����p�7xz����w
c#>�4z!���L��:�qcZ�Am�����!� ��8�N�����3t��`J0����Z$���4T^��8&�k��q���Lg�~C��oG~���o�NAN)p��I����IvC\�� N����o�������&U
�������BW�7t{��r��R
0k�W�U���>vc��������lb�z�sU����*��0����$��G��X���qI���L��:�9'�)�����[����=Ja��b�(��o���L��D��_O[P�;�L���-�m�W�:�)��m���x�wV�qJV�3A����@.�]=.;���i�����b����Iv�\�59�Q��KW5ww]U�'���������|�k��L�5�k�8q�*��8p�7n���j[(�����Z��������\�+,q�V�9:��v�Bq�YJXw�q��N��:��VE�9�+!_��V�`5f���&9��t%�+��]��P\�@�bg�x��2!G�T������e�r)N(�f�43���d���0��T�"���A�M�GR���;��d7�uX+iurD�����qw/]5f�L@�c^�r�������t2
�H����s�H������DU�k-�O����U:����F�
���`.����������%��T�u%o <��d7�uX+��z���m�v���F��R �q8���s�C�kP(Vf���eZ�,=V��U�|��*�=V���U�p�nU%+����8W�<����{d�P��^�
���<�9�n���������2�m^B���tU��k�y�����J0W"=t������^�<��Md�7s�]R�����*�l���������(Ya=�����Z~����
Y��������t�������k6!����W����������Y-s9]	�J��n���&�Zy���(�%�dRq��.�|�Zr�����=����N�U��sa��}p^f=���0����Ph�<���;'�
s�*����Fs���hZ=���u2�����s��������n^��j�	��Z�9��������J�
��c�H�eY�������9z����F4���A�������k��K�x��:�9�q�����c����s%�C�k��q���<�;1���2��;E!&oT�VV�w�!Y�{�~��9tV`n.B�#�~D��*H.�I+�e������Iv�\��a���8���������0�z�!D�-�2���s������.N�I���a��Tn��i�
��.��J����?d�z�cn%�Q*;�.W!@��������E:q8x�P���]+r���	�$���j�������`�[���N���s����@�:�����>����7x���UN���".�hx{�����-o���y5�����7o^��.���\��{?����7����O^�[J����:�����2���g��?PK�wi�1PK{2K18sept/pg_part_patch/12_4.outUX���Y���Y��}m�&7r�w��Y	�,�I�2 '���]c��A�K�`G3����:�O����Mr8�I(ita_(����S�<]��s����!�o���o������~�+��>��m��~�����>�����_y���sA��{����{�<��������r_<|����?�x������_�A+��A��?�{���O��_|F������w/_�x���~����?}���o����r\�NH��8�����U`�*������>����W/_?�|�������O�~x���o�{�������x|������7/�[���������wWkl��6�����o�=�����������~�w/~~���_�/��o�������_����0X-���=��P^�o��!����[�����������������tK�V-W�;�_�O�P9��������I-�#_$Y��O�6�����o����/�����
������G*�A�����g�_�y|����3�tj%4�_��*b�;���O�����0�^����O?=���|i����O�4�.M� ��s/-_�~����������5�.�'C�^�����St}��	6l��~��%�W�������'����=������|�����������W�_=���_�����o^�\�>�B��w�����{�>q�B%�6��p�p���I�w/_�|K��<o:�����O�����7�������_��o������_}��?���/�������;�����o��H���������>~�����������B������<��W����%��o���_�_���R~�C�`��{�������?��N|��7��x���!_L���_�{���lN��K�����`�;Fi��"��\���KM�3��C��*�j��	l������??���������o^��W%�X�������p6iUg[��o��?�,��)��Ns���<��*��a������c[i���:��^������~��_�
��
���(�(�lZ�sD9��hO>��;'��r�@9%�=�r���2�Kg�{}��
�U����QNQ��3��P�����sK�L�2��(�_w1�j-��Ni����ZPT�l� ��J�����p�)������8-|������_����auls8"�����~+�v/���p(��>��X��^�6�a0~0��#��:�>a)sDi�s�!U@S�YK8�m[�Ci
����F_���x����f*�f��y
h��P����i�20�F��=;��d7�uX��r2��v���A�@���
��B.�h��X����@�+�a>~n�� <�Qr��)
h�hv.���y)l�4��U�_��i�$���������o{�w�]gP8��x�W�3{��^�0���Z@!�����,)��NA����u�5zl��V����`�3��g
	'��Z�=���_��p�a����_�=
�R0�,�gx
����*�d����L+���M�(��Fg�
���0����c��U����3w��L>�y�g�����B4��i�$����Z��"p�nz��*p�N�YA9 ������ ���U��p>&�()�����+�8�8�s���3�5Rm�)DgV(�:�)��������kEg0���p�c}hY\���d�p�X
�9�^B��yz�n�t��oJN8�,�jA�������*���Z8�h�H���=�34�*�J=���J���Iv#[��Z���������r����O�����qg���PG(��=�1[�&k����[�0�n��Q��s�S�������kn���0��\'z(��{�s��g��e�j\CqG��`N�j�GMA�S������k��Ft�R���f�������s������u`��f�4�H��zr��^5>��@�Z����������V[
F@>o����(W��\�y�����)�i������Z
pN��:��rF�����e�1ux/{����AR�o�#wY]�J��n��r���p�i>�(��%�������*�8�X�{:l�p�~,�U�0�xG��a��<����!5!�����y�$�a��Z�z�q���>�r�x�s��|@���\�x��B>���������y�S0���k0���st�"pI(�]���fp0Wa�\�������9s�~E�`�q�s��(�aMz��r��9�%fn�W
Y��BCLY��B(W`"�{�SV��*~ei�G9z�����]����u���C<�L�\��"vl�T8	0��N��t��=�`�^����E������k��#N�tT�����L�O���B��������	���
�	�rV�)RLuV��
�o�s����2+p���J���^�`.Q�D�h
�����!����%g���F�k�Y�P�c����]�bl��!���l]]�
��~�0W^r�J����[�(��2�dN�]��6��9k�E���A�����&�
���@`@�'���B�J��6L�q�����������k�����9'p
�x���X��~�e&�������!���4�	�P�tK��'k������P��(@�&�OX�F���$�y�P!6�\b�
��SY�-�������������;'�
s�<�0��Q�t[�7s����Bq7�����h���h.��Q����^���\H�p������h��&n=�r�|�?]b3Y�*��Kx��)H�rt��0HX5��U���������k6��,.��.'�n����DU��U�����.s�C�g����y9=-����,�2M�^�I��E��
���'��U(0�
*I��G�aNs[��h�H%l��|�s��0�a-��nL;��ibOf�:�	��X�r]]�
�~�p�s��c��i#LJZ%���0h%yc	�c4g��5��K_U
��K�PG
��qvO!�Czu�s���J�vN��:�s:Z��r6�M��r�XU=�c��R��$��sO7��y��9�gs,J.�u��d�9�w���4���������
f���`B��p �\�*�^�Z��{
��6�������W��s��0�a-�S#��(�aK�>��
B+e=������`�����Gs�J��3����`��YRT]X	P[�=Z=+Bzk���{�*0W�@��u�@X����=�3@�fY���b�g{'��s��(�#J��O"������l�=���$�fZ]��$�N�p��������L�(��-��(w�K^5j,�F����R;a ���.�
B�eA�#�o���{����R�H���*Vv?��d7�uX����� ����Wm�Dir�"�q�LZ]��4�N��th��#���x��Xu��P�FF��0A�cw�����xal�UUhj.
BG3g7i��{
�s������#��������k�sN
�9>][����u��z��ms��uu!�+� �=�V!2�ZOI6�������������s�
a,w_U�j.B��@
TG�q�U
���V���Iv�\���9�#��r�����fn��y�������B0W�����BD"�uL��G���:�'�~+��F�(��c��q����|/5���*D5���D�n7��[�7�@�+yL�<��d7�uX�EX��i(�	vK���C��:��3!�s��B8WbBt{fe�s~h�4��Yu5��
�s���qAQH&��C�b���
���&����PG&�	t��\#g���,-�{'��s���l�R��J��@����8����A$���p�@����s����P�p��8�i#o�/����i����T�����dVU���X��P!�\*�:R!xV���w�1mu��`uM��|ZX���Iv�\���9���f�C���B��bp�����������W� fmNV�[2%��j�6�����cTc�����A��lU.����PG.K��p7��p<gP.F�w�y�$�a��ZU� �(�� ��A�o��9��:ZW���8�n�,#��a5�y0G��T!X8W[Cc���P.��X�Y�I����� t���r!taP�&��)�E�R�9��H��;'�
s��Z�����f{�z�+k&nr;����=�,��>�c.�3*g>3�rm��zxJ�����}1��f�O�v����p!�\.�.���YG����O���:�A�wN��:��;7��O���;���1V;�(�[���.s%.D�g8�b.�n?Q;�w���~'��S�|�p����Wj#�1�w����
B��B��<e�����S��c��m��s�9�n����5�����u.l��XQt�����q���������h�uJ0���)Q|z�zJ�m�9+�S�$� T�&Z������c�+0W�)�s{Jt��(2�G��5Za�����*}�9�n�����w��� �w���:�9�'�\W��BKI�g�x�6����Zya*At����F	�*��,P4���bb�*-%znK�>��X����S�J�I��9�5$���s��0�a��!�N9�YR��U���S���hZs�5�.s���n��t/	_�Q2��/��(����.����oY��[&�#�t�Sc�����=��DJ�|�����S�(:FL��	��y�$�Q���<?f�fZ��
�P��7�5��B(W�'���Z�����O�{�u��\V`Z��P!s��,'gnN��J?���O������7��^���<���I��~�s��0�a�����'1n+'�G���[9�D����R0W�'����6g���I[���"l*%<3�
s��������+����ZM�����'1�~z<��{
so���x�m0V�9"�wN��:��lq@���k".w�Q�P\w���uu�{�Y>�3���(P�c���Ed�������:�%��6J����V96�3�~3���<�'��u��
v�)Ds��D�����C8��d7�uX+��8Bs�b�Us��ycZ����@oI~����`��O����Lj-��&!��Y��A���^K	T���J��"
�+[�0�~3���<�'�|�����{
#��s�*#���wN��:��?BBc`�-wS���ku33f]]�
�$���B+����v�s�\Y�
������J��b�~��I,�T��7c�ILEZ����4GiM��3xtO!iuR8��$�S��;'�
s�*�:; �J����������H���o]]�
����Y�V�}���AiM�V�<�t��Z�VC���F.7N���r�LEY��U�4GeM��<��0�B(�AQ�����������2���S�?u���M���cw�9?c~o�8��5�=�VZ-��M�-��Y������CK(��/�8��2���)��"�i�
k���&y���=�����9�iv�������kRZ�bZa�}0�4�G�:/����\AX��3<��I����V�4�O�Y�L���Fs��|�N(G�&T����X�9S�@�������7�Fg�y�Th]��A9V�wN��:�y4d���d3��o�2ah#����P��������>�,�>���Ck�Y�X����
}������e��Z�\�a�r LAV�����i���>�	)=�]���y�$�q��Z��rH��+���7!1�����DY����H��a��3�����9m3Pb����v����B{0�s<��n��X��������
��eA�"������BC	�;�'����d�s��0�a���!� @��=8.w2�!P��`n�XW����3@0��V&*�0�o��"����dj�o���1^d�P�J{Z�*�n0�UXf.�XN
���)��q�M��W2{��s��0�a-��Cf{�(��-w1������Z��e]]�
,�~�����"�J��#l[Y�JY��n#�����K�}!�  ��2����:L����sY������Mnn��B�"�
G@�>��d7�uX���9��f��A;6��Q��]W�������<�/Y^��g�-N����\rk���UK�c����TIn�@9�m�VHv.	�E5���z��B	�����N@NO�y�$�Q���9��l�qS�5�dk�!�\}_WB���f�g�h���d��y�$��o=�]���fw�h"�UJ���[Zf�N����K��QM����{J0w]SZ���<�wN��:��l`��*0T��E��9�dE�������H����52�=�F�U t�iQ"]n���!4*R[P�����e��}�-��\�a�� ���)]p��X
Y@�>�r���s��0�a�hN���*���r����~)�n�Y���`�@���k�%J�Z��a��sx���	o$M�K���nZ���`��� �\�=� �):Ys�o��A��9K��d7�uX�oF���rKZ�r�1%BE����H��Y��X���Y�}���)i�K3����j�50����1Xr���c�X��p �\�=p l ��p�N�;� 2q�	/8'��r��YGH�.Yq�����������P�������I�I�N��9�U'��UY���5��:sVX�(��V����K���A��������Pg��uF9���z�O�|�$�a��Z9�7�h�^+��ef��q�/�r����H���(�)���z����gQv�)��1��.����sI%�d;V��VHv.	�Ax���v�)��|��eh/��L�:��d7�uX+��r���3W&^�c��Ws����`�4��3[B{>��sy,�$�o44��k���<.ikas�+�Em+;�a��2�u����Bf	�Y_�;��d7�uX��QM�=L�T�WL��W ��atb]]�J� �=���8��`�U  �Z�o�\&��
��G��������f0�U8v.�8rW����@�������Ax��;'�
s��'�#d�����l��2?��aN�	�H��\iD�gXm�9�� ����!N��Ln]C��-�t����L��Ax��:6i�
�r ����|v���{���������m���Iv�\��a�y�+C����%�z`��I+�^s��V�������<��V��������H�9�s�����%�t	E5=��
��I�-A`��sIx AX�������&��K��f;>�9�n���V�
C������-w��VG�:�q�O���\���)x�1��b���Y��-������t�����]l�C'?<��UH8������=G�����6�H��e��,�_8iU#�������������R��`]]�J$�n�(�\�5�I0�fQ����o���U�Pp%�)�1��]��*$�K��	����{JSZm:���Z�wN��:�s�cd�|�a��Y��>�:Ph���B0W"At{�C��/�gu[�8��t��y�<p���8g��b4=�1E,9���eA������>���s�og���JP�wN��:�I�W#����pNm-�17������'��H��\iD�g��V6�=�������P�2L�Z��t+|+�3^����n;fG�����A�S�����m���=G�S�
e���;`G�}�s��0�a��9!�������x��s� �d���\����m���V;��$�g��\��a���Ri�z���y.4�<�@�Z+4�K��#
B+V���{
�VP��Kq;| ��|�$�a��Z�V-Ds��m�����rC�z�eg^���iu!�+������q0�a��M��^�mcZ}�_�C��������Z��9`+����A�qDp	G��������h.J2�wN��:��������Kl_%w���:����>��B0W�At{fU����Y5mW:{H�h��jG�gQ����y(�39�{�X�
�UX8��O���V��w(|h������'�wN��:��rnD	B�
���7!27MCU@�{���P�4	��3Z@�9+��a�����P�����z�s�9���n��g�z�v0VH8��O����|S�D5�w�(�c;YU��$�!��Z�m��u�������b1V��BW`@�{F��/m,w�N�P�x\�8�vc���9.<��1����A��Jq�B�ps	N ������s]�D�-��pj�g���a-���t��y��r��_o��q��XW����;��<���,�i��N��$��ngN�N��(�
��/���f7�ul��U�n.������1}m���j����I8��d7�uX+Q
C�4M�
m�]:�#9vCk�(��z����O�}���4�[��eW�C�;%Mm���&\�I�Z�tl��W!>����
�����;��8�s��%���=����k�p���� ..w��
��`�V�����X��aUa�&�Y�WX������Me��q�;������k��hb�c!�BzpsIN�8��.��w���Q��y�$�!����0 N��j��m�[J��-r��y����BW`<�{FJ�X����C��
q(m�
�j$��(.�j�0��a�6�
����;8s��@��>d� ��a���Wo��g���a�(���jf���;q"�WT#�����BW";t{�y�\Q�,5{���~G��q��4V%e��W�/�('�Z�fpW�:��\g�c.��9v�	4�U�L�y�$�!����j1�Ko�����8�I�N�8����������"c��r[<�A�����4������y�N�qLn���*�
s����q��spsy�	�$�S8�Cv�	��0���L��:�Er�fL�/�v����Fb&��8JR%w ��A\i�C�g(�BE�)~���m����8z����J5*�RV��/��f���r\����r�;B�����;�D5��xS	��y�$�!��Z��t���m�op�����s;V`��������1�Yw���S~���&B�
N��@���8��NB
�L�������4R!8����}Q�N���eJ(�\����IvC\��!��0�����_�kpu������.L����� ��n��oG�LI)��(�	���%la�8��m�.~!
��|��7�T�
n.���
��4>{�q.���3�O��x&�
q���$�,��B/�Ji
���F���L�A\����.UrgQ���wP���q�oz(
X��q�]�$��mQ����8_a7���d7p�[��e��
��ill����g���a�rC�1sW��*��!�c��!c����uu�{�S>�3���g;hk����iD<%��*nz�
�'�(�.�[�h�;&�{�+�?�������.���)@�

JL)Q����=����kA9�3B��^�6������F1�K�A\i�C�gC����:�*77���_Bn�e�������x�*����r���M#��n�s�
��n@�s1f��BEU2��yv�>�3�n�����p�^D��u��*_��(�u���� ��n�����6Fq,2/QM�V��n��P����f�:�q�	es:��X����\v�/�X��)TT��D�<5���IvC\�5i�0����1YOc����*���<�!������C\i�C�g�����A���s���)l������S\]�1`wL�bJ�c+���n�s�
��n@����)@��������=����k���*��o�D�~`����q�������BWb7t{��cE� N�Y��Z��*o�<g��G9��������4%�4��
���e7�#���8��)5�p����5f��x�g���a�DuH������h��y,w���A\���� ���H��Mk��\/XJ�Z�<����:e�Q�`\��e� 58S���\z�?�yi�o^�Ty�Edp��G�g<����ka�AR��/�����#W�p�T���uu!�+��=��$U�|p=m�*D!S�8c9��'��(��$����5�
�c��
����7�#����2s�N	��&��7����L��:����7ph�G���13��Y����.q%zC�g��,���6�A��
�15N�eR(d5�����U�
~.���
<���w��f���%a"���IvC\�� N�qV�UM���Su]WS��������
��1i�hF!�i��B��/�qk[��x�QREz!��]�f����Y9e(�U�
~.���7J1��;�(�q��*+����IvC\��U;hx��%U�/�FI��
b]]���:=�$5%du�YR#��4��1S,kb���[!��Oa�Y0��w�q�Bos�
�Ho0��N���0
������=����k�T����B�c�y�o��Gq����|TS���@����i��Qudp��r��7(>�K]#@���jC������4����Ac��B�������`����)���D�U�V��y�$�!��Z�����^�Uj��$�|�� �X]��/�A\����%���U���q.�Re�#���44*�|.�'��l��s��X����\zC(�$�41�UQ��9M�(��{&�
q��8�#�
(0�}P�z��Y�Q�a]]�J��=�'p���FVX����q�x��[��W�E#���/�_�R6���c��C�����������w{� Nq��}�f��{&�
q��Q~D�j�����~#��F���bAG�������
����v�F���;Cjd���3l�Q�����U1`�\!������
�!��7�������)%�,�c-��<��d7�uX�� ������q[����I��BW�7t{�g�(�[�p���r���(|����.V(i>
�����z��H���\zC8���;!���R_?��a!���IvC\���8�G�)I��,7��)C�/.�`�������
��Q<�3�������[�T�\�8�y�<��qh��/.�����U�
a.�!�
�p'���S��l�
Z������d7�u��(.�Q'�2�*��i���kfp-�A\ixC�g����C� L�8/|�8�p�������^�DR����-Q�n�&f���\vC8����S�[�FV8��d7�uX��TTlu����!�!��M$�.�A\����� x^��Nr�,��oZ���\3V��7�p������8
NT+��0������g�������.t�:��d7�uX���n��-Vq{q[��j��<�n]]�J��n�(LLT
f�)�[��
��f������a�L�`v��%�c�7�
�!�e7�#��)B8z���25����p�3�n���f_������M�q?�����;�������.qvC�g��vS�8��iJ#.�M��/����q�QJ�gq�#�wjJc!�;/��?�	r����w�q�A%YL�4����;��d7�uX3��9�*�U�r��}c�*=
�)���:(w�*���K(gt`���8Y�q�j��|��zYU�N��8������h��p�sq�0�A����B�!��Xnh�9�}�s��8�aM���!�#~��+�A��Wc�R&�����p�Ds�v�%�c��J,nZ����7��/n�v��m���4�ol��[0k���U���`.����9�*%�Q����[<��d7�uX+i�rD��Pj����n_�����sQX)��s%�C�k,�H\{�S~kg���R�j+��fB]XZ��N����5�Nf��0W�:�������Z5T�Z%?�V�g��������kEsF�WB�^qw`��^a�\��0���s�C�k(b�.]BOI����>��x8�CA7�4W�9c�0���L�<Yd���?�s���<�w�����V7����wN��:��+�F$�����]�2������cy�uu%�+Q�]����l.�$�s���\�]N�h��(u�V� �enl'	�
��0�
C\AT�����U��y�$�a���5=f�*���;hXOZ��4~]]	�J��n����9�4N�� t:�3<	>�����TRx�9Vvy���h��{����#��U$61���0�<M�S���;'�
s�:��~�\�������������.
-���`�<���5s���4w��Y2$���� i��(iEs����^�b�����X�]���\����hN�]GI#��,��zr��]�y�$�a��Z0���]<`om
���������s�"�b]]	�J��n�PL��
���a���������YO��UA9���#��R��]NZ+�����sa���00 ��
T}?4r�y�$�a���0G�Qi�B�v������s�
�����`�<���5AH���Q�)���e�I�B(�Xs=�NW9^*}&��G��:t����r��`.�=���g��������U�o+��k���a5����Q������������r|�T�`(�����w��s��o���`U�(�rF��S���x8��f
C9�����/x���������"(Wt����}���������_�5��qV[V����������g��_PK��>��1PK {2K18sept/pg_part_patch/3_1.outUX���Y���Y��}]��q��~E�i��B�	 �X9����e;��*6�IA�04��c�����o&P��(u�E��n����dw6p�_����N���7��?O��������������%��=�����}~z����>���#��D0f�_�����}���O���������O_�����o�{������vh�rV>X����������gb�_�����?|�����Y21MO6@ t�_����������������W�~�����{�����o�����|��������o�~���w_�}�O_����W�z���o��~�t���y���?���YO��7��������7o?~������?�y�����Oo?���_=}���/�����o~����Oo���Q�6�����?�{���w����������O���O����dF���
D�����Ek,?����I�������q�e����T�za�l�?��K�.����w?��?||��/���Z��������b���}���M.��o9�1:�R����!���o����A�����~�����������?��������A�f����w�U��1��N,?���|��)���W�� V���7m��-�����h��}�{>�Gi�%�|i�6����7�=;g
N�o�n���I�$<�e5x9��u��������K��R���_�|����x���tr���Z�!�a�A��YX�#;�8�3��I��T��������^����0��?~�,F~����>j���_|���6��~�w�@���b���(�����g-W���<KQ�	�Cy���N'��$����_�Y�/59��8#�7��'�W�L/_�|�I��4���O��Z��:�B~b��b���0�o����wO���O���������������%��������C�l����(��+$G{j8<7��?������j��
O��s�����_��D���w�?�N������_�R�I94�|��v�?��,"y�bu���d_9����}�,z�����=�����������u��Y�_y�Y^d�7���>���[��?=��<���y
�YI>���.�h�UO���z�Oc�bY�	��!o�`cr,�a�����9��9�>��w�K�bc ��MR^���F�`#��p����A�F�����I���0��UlD���#e=��n6:q�&l��z������J����~,D�l�dw�F����[I>�9d���MT�)�Q\�����`%d�F��c��~=`T�)f�Q`Q���>/����AE<��C6s�09���������QB��F���"�@g��M�_��M�v�P�K���a���Nn$BG�H(Q1��x
��5�|BBu���l����>
�����(qGM;���@y�5�l�����%��.Q3u0���@��@��ktFrG��J�@����
@��p�[Q�c1P�(�z�ay��s�#*�M�����c��]0��1����O��#Sl��N��	:������I�������j8����;(^�����M��F����@��@�kt�h�@B�]t(a��TgM��b ���K�lX��`���{A~�^+@�����X�6�}�@#Y�L�(F�����`��u
�	a��+�c���<q�����,�M�\Fd����?��SVC��c�h����N�<�x�#o4��>;��X\��N][�u�6�34����]00�b��y�� k\���)�>�#GC%���UJ���1x�r�����r*����#00^a���Z$���(�J�\���-[�~Db���@���s�����k��1��g����&�u\3���G8�8e�����t�����j�@�.rC1�N���P�����@>���������aX�@����q�@��|��1S�?���N�GEb�i�
p<���_�W����8M��4�8e�%�FK&��Hb�P=A���+�@s����9� ?�p_��������$LWH(��<��t��e:������Gm*l����C���=3yXyD�3t;��H��U�$!��^yl�]��Czf�>��n�I�m%�L�b1k����&*��4�g�E	�5�l�b�k?����O�
���;�����`h�H����(��?6����'�d��������9�^��2���y��b�>#W=����v��J����6���H������i�^��S��Js?����8�De�I����9���&�	-zN�����������9/�[q��uh(�����8 S��C#��2J yD�<"�����
�ncS�Mh��n�nk�� �i( w>��\�]���z?�m��d/O�6+����HA��h�uhtnP��#���m���2��}�V��@�O�O�(��sc{�`�H.�
�ACq�4���|	I"���(.�W&�VCS���q�H����Q��a)F�a&p��z�9�nO{�PZ~&Zn�v���=�o����/��6+Y%�k��r(�6QY�Gy�F����n���z�%�F��eLE�S:�nh�S��|�8v�(�6��b�8�d�7p��-"�}g��&/G���(����`pv\P��s��)5����A\s�v�v����H|&8�]Hz��p���R&�sD���&*������D��
>{���B��Y��n_���F� y�@�rO�3���p��T�4wESq���<G	h�Dmd���s����p��,�u�(�ng����>8z9��U�y�n��LG���ry�Y��L\��2QY��7v�u�j1�
�����h,�����('D��P�I|���(�������#�+����T��6.�@���;-�pT������9��V�1�sT����cdX!�)D�,�Pv��=�>a�
6����Em���r�G���S��rc����#����@�5��C�-��W)�PCE'�����������b�;�����-�,��\�1s�{9��Y�l��Z�.��V��������ZPV�V��Va}�&���M�XE�4��vM� �����x�7h.����m��h���o���:8"�1a����/=��'�7P�����I����#��
s�'�]<����xqW��
W��]7��VF����j��1�P^����kr���O�c�6���+}@��Z��]s��X�s���8���<�J��aSpL����MT�)���0"���x��q������
�Ysn��$�
�v��.�8�C�qGPa�FU�Mm!MvO% ������^�/<G���N��������j����@����������qLY�Q��]��1��V��&�
8m��N�A��A]����.GXv�3a����Ab���I��\=G�$��;���qW��s
5����V�v�)�R����d�i����[y�d��_�Tj7�Xq���j���knQ�S����9�}�o�c�o%-�x-�$eU��&*��H�U��V�[[<G	�����!`�s��b��i�!&�D{�A�G�c��#�����(���x���Z)���&���DY���V�Ug�H�M���m�@��=f�-j4���9����c��J����A�w��&*���e�A��&�j������KnGL�	^��S}��>>\;8� 7��p\�ika�T�z�y���AvqQ�&z��D#��V�v�����jq������E�������c��s�<����a�2d0�F�g��N������=Gk �!V�<�%Vw��3�#D�V�j��K�:8�!W1Mj�����j`T���5�-�����K ���I-%���Y�7'�}pL�e�=�i�n�����.����p���stY����:�De��(���\�:kX�}-���>GV�$��6�8��H��=r��a���f�(W����[ ����+�T��Y8�����������v��[yB�:,C��e�b�4D	x��]�/��6+en��N8=�i���:8��u��jJ�I�@�B�u%y���Dc���X2w�;8�!��\�B��y��a�~uH�	 ���4FR��*�0�[����s��0��n#��r��4���?�C����s�<���4y����	.�6QY�G9�2����s�������j��}�Q5Z5�����O�Z��]��B�!G0d��!�t.XR%��"<��\����a�r4_<82n�V.�3:�}�������:�����y��������^�mV��c�:���F���S�#� �	���<��Xoj�cg�O��9
���Y��-~u�����#2p����!��2ws��x��:�I5�����[�/_�Q�U���j�X�����+�jm������[��$���'����y�����en���N�MT�)��[;F��EL*�y�6�t%������,y���'��0d��\3d$�65�O6�~���+s+	�W��L��/� �����A�T��B�Wt'�������k?�R������A�y��T�1j���G���s`#l����c4�[z��m�1vS�>����&�PPZ8�����������������w(��-�:h�q������n����Y<����N�b�1u��{�����%y����E�y���3��@RI�E5���:4�V��� �4M�6��\��>=&�����Uqn�E���C��#�1pM�QP�S���bA�Q��a���/yt����8�R�
�!�n�
��������\Y��d=��1�.�c.O�6+�����6QY��F��zI���J��wlB�r��n#�<?���U�����tg��64b��G�c�9�����Y���s7��D�Q�p��`|��bGB#7h\�� ���#��(�h�-���=�%'v�p�<���d�Hd��P
��&*��H�HN�#��A-������R��Q�d��s��������G);�<��7�� �io�a�8�k�^	�Q�s���-�;��z���<.�('�*�c���������s��L�v�#>"��v���<����!u�*c>�6QY��F�:����F�b��D4P�1�/�Qi�uV�A��R����`c��GPc�zx-y������o���f/t���D�7:��:�t[o}�+��(Q"�,r[v������]f^��mV���4	N���`�ul�a3�������lS�����*���j���D�����T��`c���h����DB���s�����a�7�W�7'�h����m23�+��R{gl����w'������]����p���o����b<cLT�9��a�tmN,~�S�l2�mVu��hQP���E���]������#:x���Gy��1:[�����iT����H:s��c��4Nn"�hQ����
��rf�q����wRX��<sp���S��J�7j�&�&�P�c�����akKY��q�����*ZJ��n�w�V�h��6t�X�7�����i��#Zx�����%iP/����Q����������S��F9�6�[��&�wV[x�������~�N�.
���p��4$�-<I����� �u
l���$�bLu��\���O���F��t��a���B[�����i��#Zx���m�u��`�����H,�)�b�j�����q�}Y�Z�ZL�QH��nc�:�d��>�.���S��J������$�h�u
l�r�P�$����Q��X[xBl�oo�b,���F����'R��������L�d�1���nLm��%��G������7F�a��d��7�m�N��� ?�d3+���S���w��\��mV��X��9�h�u
lD��Z�S��:�s:14T���6
�
U��h����"9�nB<��3;�����#zx���������� ��:�H>����7� Az��O%��~�m-3�l��i�r��i�y��";dh�V�����sy�YI���L����h�u
p��iMH|n$�6����L���y��~��A0�E�2�M\F�E��:=<tD]�����U��c�����AlE�N��`����������&�O�+�[+T[��Bu�5����wY�uxv)T_�mV�I��Nmu��p�u
lD�uRG7�i����H�"e�c�E�TU�$�	G�{48RG������ky[�\��"$�J����������c7B���(���Gy�h)���7�dts�5d���k�;���Y�g�������T�&�����A&*���]R�_��\DG�T�����h��j��:&�	L��H-�{��#oKG�����m�d��xL��P*���������/�CF�=�R�2c���&��D+�j�p���g����-<�3�L���sy
�YI�x26:����LT�)��5Q W33�����t�S5F	�MB�SL{�MPG���P��ku[��Zq����2�ZEe9�F�G���c^�D-'�
�	8���\�V��I*�6�y���=B�1�B��<���d���R��F���S�#A$��8�Y�,�WU�q�8&��p�k-�]3�4��f�a�p�L�6v�1t3�.�16$��w�9c��A5F����h1�T�5����/����Ji�n[�
�z��D��k^�}���>��
j���Ti��R/g��1QY��F�pE���x�u*�tsU��T
��
��yk}m���KO�CGPc��������-��]R&��t^����38��
�&TRu8j�a��\��)�#��E@���i�y�[�����6����.�x��|+Yu����B�m���:8�J���b
7&�v�T��$�U��S�<��Z��6m�v�7�����57F��+A�<��>�(yH�Q'{������xp�U����}<.���mM���"TP�	s.��]�>�sq��Z��p����2y�
�6QY�G�i�PY��"DD\�}���cJbb�=�tX����zp��c�r� �x����1)ty�Z2��<�"$!M�	I_<8F��p[�M�1d���>��~}�Xb|4��q���9����;��V�z�2�P |m����6v��<��J�+�bq�j<��wg�$�2�xS�Qn���'���mp�v�?��o�c�k�&�
���C9��tR([���E3����.�v8�V��� ���+��XFO��������<4f���n��6�Q�%k�6QY�G0������1:����b����,�b5��`����w�1�z�������0A>�~X-o�Q�tU_�OJ+�����jq��Q�y���V�j���p�nA��x�a���n����;��Y��U2QY���� �1Y%od�Q���i-����*�Q8�����4n��;�?�_i�����=��]z-�:��4���������tQ<�s�4r��gjq>��4�{��U��L�-��������x��l+i��d���G���S�#�s���5>�&�cC�u
��s�������y��<����G�c�5?����(��'V���V�V�:|ks?dF���#��9"�r�y������	��9�
�w�1*�C�.��6+	8�<�Z�k�88�De�Q����XUn!-�j��Z�:�&�����9�����$��;�?�_�cx6�z����>�cP`�V� ���<(����Q��q�j�o�����V��f5��Z��w-���(������n���c��5�P�d��N�:
w8�[�����au����>�����7;��j��x���?� ��2V��k���!t)� P���/��\���s�n{P�D8r�������Z����w�����L��	��n��J�hX���u����S�#9?� �(Ms�Bb��mv���eV'�	L� #�������.�A�a��#2�zv���$L9�k����g��3a���^�@��8pg�,t��������D��w-�Fi���dn0d�o%�<�
��F7�De����A�/�@V���*
�����������RV5������a��#2��!c����s���_�V����jK|�M�i�
�F��Gk�1d�L���\��p��V{�|��n�������sy�Y���rr���&*��H6jW�8�����b7�����y�0[N�]Zy:C�_3d���9��9��x
iX3)��#�W0v0���XVGwb��`[�"���LT�<s���"fb��a�>�[�c�m%MN��-�!�i���:8��3�����"%��K�F$D"����JN2^{
in�`��v�;C�_3d.Fk�>8�
��k�#�n!u+V~�������:a�\���&7�\�82������[�E���hw��uy�YIg<�V�7��d������p���)�����3YS���,G��<#���dW��=r����	G0d�C����v�9v��������8,5pt�_~A���g����<@&o������� 7��]�S������8^�mV��jF���S��&�q�S0TTy�\�Y X��D�V���Yh$�td�V��9�C&��	W��u���(�I�TBLl(S4�Z\���<d����$�q:� ����\�����2,��5q>�Z���xm.��V_�mVR��<����dm��N�r�6V���
����T��y���~X���#��E���5p�{���C&��	W'����"������E:�L��iNU*�^}0����rS��I%��n�V3�z���1_�9�v�[�g[Is��u�\xO�MT�)�����I,��@��������cU%�y"���J���l�8v2��L�����V7d�N��{��N�n�8�u^���}�<�
�s�)�U������=Gos�A��^TB|���o1d�m�
�O�h�upD#Z��2b����j�nX�Z���	h�y��{��
�L8�!�2r�\#�DN?���n�{Pp�q!<�Zy4�X��9G���Y1�Z���<^��b�w����
��C��n�R�9�\-m�������C�#d89��pOn�&�RW��UZT{��=T��M�q�M��`c� � ��K��C��V	�j���Vr��j��7.4�	�W0%!��c0���8��S�1o�K�k�j}ds���[�q�G��}��[�g[i&��|4,|�A&*���a+c��;�W�1)��xF-���@gWh��w�[T����L8� �	2(��������pt�s���K�9G�/�Z+V���mUT���f�	,���Ub�(�o���.��c��J�qT�0�h�u
pD�Q�er�`G�������>��j`}�8S������=t'B� � ���2�@�j����-�0�$&�V���jY��	2n\�ch���*
ys��<��f�-�
C�=6���[�g[I=G����G���S�#������Hd�}6Z��P�!�<��*���J�n����h�q�zL� � ��f�W��:�u�V�ul���d��� �DzS����j��mrT���Z,a���<�n)���q���c��J�u+����h�u
pD?H��8�9]F�V�`�w��:�����$�P%��2���A&A�	�SY���1A�{����[�����c���������V0#�]����\�0����e�re;U�o�c�o%�L�����h�u
d$%:��������y�1q����c�O�^e}��5F'�X�G��a��#�1�^!�����+����gS0M�(��|����9�e�D�`�\��m-�&�+�Fy��jJ��������e�]��.��6+)2��r��uv���:	2��ALupYAF�\'�;����Z�������(��m�a�:u�Pc���������T���%�%����'kbZ�#��:�a�F��f��7=����pAg���:�Z
iD�b3�z���n���b�H�o��8�De�����n4-�yx���vu8{����M]8G�������Gwc��b���7�ijme�\C���`@��I��4T[-������H!����y-���6��S���[V�o�K��n��<�Z'���&*��8j� (�n����:�L����ezm��.f����������� c�� �D���Z$CN���h�A�-�i����G#��
��������*��]N=d��E���i���>�Vd���7^��mV��p���D��m�����qk�m���X*0�L���!�}�(���dd	�50�+0w��� c��`�D��U�����W*05�v*��;�_>2F�]���I�����J�+��P������M_��mVR�Q�T'�w�h�u
dW��O<�M��mc����������g����X�8{ c����D�@��A-:w��Q{��E4�/X�l_~�Q��0�����������>�Z���3�Z��#Z�>y��d�gZifQ��/��A&*���]����m&����+�����+N�C���������5���;d�x&�+dd3�)&����6���(���e2�;^~�1�M�|mcN��8v=rE"�G����9|������o��y���g��TDt���:2"�A�4+M�K�Q�>j���R�V��yF~��MC�'��a��#�01^#��m�9�q��������mNQ|��h�&���
�j�5W�#�qe6��^�
���9D����.�x��l+���+06���h�u
d�4h6��N�����6m}CF�A�"c�Q���HS�
w�g��`�4��W�(��M��'P[E�Q|F\L�����)���W]j#�����g�6�d0g�������	Z{U`nq`�m%��@l�/�l���S #	���3��W��$[�&hE����U�;W�@����N
�2v80�L���(�[)��3z��Q�x��gLw�p��h]	�gt�<�n[���|K�l����*zz@���Z� �-������y�jDy�q���:2�+=���R>W�'�"cR��J�w�(�3JX(C��$_��md������5Fe���c��"�����������^AmZ��q���M@pm�~4-���Ds�w�b�(�1q�
���f%�3*&&0�q���:2z�3:y��u�o>�j4���Ml�@{t�p��Gp`��u���k���)�
�WQLn>#�W������%�0�I���
@]�\�PNb����k�>�G���]���gp��4���u��B�{���:2n��W��Y�D�Q�j����������q&���8I\�1�B���������������]d�&Q����(y����k�M�8dtl��?��Es�������ey��k�>�I�*k���.��3��J3;P����h�ud� x2�����\y��qE�[�Y
9�>�RB�g$�����������#V�>�E�Z�PMA0�{���Q���1l�QV��Mkg����V:�]2�k�q�-��x�!�2���n��-S����l�q&*��H����5��'�k�NJ��=����W�ta�-�%2-M�!B�
A��k
L�9nQ�n+��A�C	�������8���N;#7�����qE�;d]�\�.�E�^z�j���x��l+i&�JP]� �u
`D���P�;k�Fo['�d
w�k}8���*wS��d��Q�������5&����������>#��Cm��!2'A=���@��@+���$q#���Sv���4������zy�YI]�4��`���S #�4Jj�y�A��D��������QeM�4m���qFFwg�^;>������]Z;��S�`���uF�Rj������&W���:�9b���:��
e��?B�q'����f�*��A��G���S #�a��^b�4�`��A�q��Q��>C���b=-�x���2v(0|��)0�4�3Zd#��v�b����E��I�,���i���\VGPfr �&	gyEj�%��*�4�u��P`�>�x��l+��t��1��A&*���� 
X+��T�N��FoI��3j�h��N
]X��� c��GP`���8���qL}�F>�2'bk�����
�1�Ve��:M;���$�����8�����`���� �
������<� �<��&*��H���3����ra��D�U>�yr�o�Q���3�Z���W�w�;>��7������,��JY0��mZ������n��|��wG���rP������>2�������w��3<�4�m��n���3fA	uh���cLT�9�����������Cm��U�f7�V��L�����������#(0���-��\���B�rr~u�2zyI�����_<4��1���^�s`�b�<���NS0����e����8�;�/�6+i��f�j0�r2�De�Q��AN#�YkG��o:�����J���8���&�.�����#80���B��&d�V��*d�V�q���U[qZ���GF�[���n���h��n������#O�	�����U^�#��1�N_��mV�90�><�De�������\q�m��H}�$���e����Y��=����IGp`�
���X�v����:�SuNT�76d�;��z�����U[�Z	5�`���O�;�����9D�#��8M� ��90���T�&����MT�)�Qn��n��r�4`^90�����ko�`P�����������@�&��I��(�AU%���Sv���'9���D�
�v���P-N�m+������^���xVv��Kh�����$/��6+�Dc�/x��LT�9�1�Q*d^Bh*Nb�:)^���6o:x��iM4�����,�A�	&A�I7H0r�x���n4M*��Y$[4-.��o���4����e����<��Y^������w��+>�8������f%[x�3��r���u
d$�f���Z;D���r���$~�:}�Yz�fU�c�A��`�$�tM�a��]�������v@v�@j�iy�_��6�,��B��
�k3�]�~4��H��v��9Q�\� �
���d��Q�Q�v<�De������:���:WB���2U�u=mc�Z�N���:�t&]s`T��
;�����3F���g�Y��2Bt#Fp<zt�L�e�+�MX�Za�^�2z�n)��v�
�o��y��4����G�.�A2QY�FL~���W��f�4M�M�dW��2*%0��vDV�Q"�=����IGp`�5&.��P��a��1�yi���� B�[i�eu��2r���L�7*q��w�H�!��v��<�����i�90�NG7�De�	�>����D�.�����k!�J��GQ�g�TI4��:�����&A�I�9P����X�w�c�,�\!2�[e�|�Q�qF���6����A����I�h��[���/k+0�]%.��6+U�=��D��u`��
'����!h'�Zo�U���n-����mA���*B����A�&A�I�Md�5'@�1�����}��YZ���_�8��D.�WE�Z?�_����?����w��?d
L�]D�.��6+�nF_hn2QY�@�aS`\ �`Z{��n����e��se���bl���G�E��nB�������AW�(���%Y�=���2?P,<���q���G�`�e��&�H����y�>	F�f��Xv����*�e�cx��!�f&�NkRG�*�hG���s�cH���Yn',zG��q�oT2M�dt�&�Z��;��z�����o�������X�d9�V�����;j�S�6�������a�R���� ��k���k���kE�1Y�����P��63i�w���%7�Fe����`����i��[
K��2[H�y���_��5��S,���*����x�cq��m�i��a�)Gq�LX���@�Q^�a�[ccl-{.z�[�K�c8e=��k~�}H`v��\�mf�9G���8�Fe�)���U�;�`��F_�������ej���\�I1i��+8�}������8o�a���v}#h�q�u��0�t������s��
�:j�v���&]���~�����v��NBY�]�������I'���,����6*����U$����-a�Rm��������J�w:Y^Bsi3�=Y�� A�P{�x%���M��^
r ��P�#�Yv����W���C�x�Z�^�N������y��5�n���N�>�dW�p���u�y��^];�Fe��
"��x��������Zpo%�V��<���&.c��;gM�������8o����V���[��:�H��\�^��;������`$cp.��m��W�f)����y��e���'em��.��C��LZ�v��Z��m��N����F3�.GyZ�
����D`�6���1S�����E�=���v�1�G��59�3�� 8�n�����P�m[h-������q� j��Am���gQ^��b��,�B|-�E=�?`�`���:^��mf�&�����/�����S�#3V��Cg�Z�X�Z��+*<r�C+*�E�*Cv�������8�2R.����Z��f������Z�^<:��0�*�����Nt�[3�_��y�n>��I(k3:�2���n3���U[9H-�x���s�c���2Ji+=��p��p}��`�P�cbTK8�6&f��c�$��q:��cu w��oWd��k�:�(�urh������.�S��Vo�����&}��3.7�N�VrHC����������!�f&u2S��!%���QY�@G�qDh���3�T�"��{�(�
Tx��<��`��'�^��{@c�%��q4^r���N��JQ!����y}���vR��3Q��Q��[	�e]��G�PD��i7=�1a�� 9���~���1���cX�D��i�&�����Dz�4Q�
n0�������a���U��G9`D�������(d�g��.�:y�ufB�������C��`������w�9����?}�����8����<r��f���LQV=b_���?���O��}�Y�����~������w��#�/?|����?���,y�'���wo����O�;�XU�r������������N�X~��MZ`R1�����:�~����PK��6�;�(PK+{2K18sept/pg_part_patch/3_2.outUX���Y	��Y��}m�\�q�w������� ��E�I���$��k,���h.<�g����������C��n������+"��Z��&������N���7��?O��������������5��=�����}~z����O� �O��	8��_�����}�u�O���������O_�y���7�>}~������<8Y?�����o?|�^7�����������mJ<9���g!������o���^W�����_~��������o��~����?z��|������~������>?����~~��_����g�O�|:��_������?�B�������?>}�����O��?������|~>�����o>|�������??}����O�.�������o�����1�����o�}�������?|������?����2%�����,R��O/�$srQ�X �K|�8b�������o�V=2P1����2����������>���������_�|��������F?:0 �����_������cb�~t����G��������~����g�b�����7?�����h?�o�|��N���o\���K9������M9F��/��$���~�HDQ���f�9{>��<��o��[f
"?�Cb����Y;�� A�E�-�%���/2D]^nc����`����������'����K���o>}���?�{�p:���&h��zU��kfEtFf_0�G5�O�o]�_����O�������O���O�~��Em�v��/�������4����(H8�����?�A���  ��`/\f��:��)���?�����Ss��%8���|��)��19&Z��9�O�'�����B���{��K;6����o�}����������o�?}x��;�LI=���3�	�?3��"1l��� ���_i�� ���AH��2V]�������^
!�����!g���_�W�������_��;E�����������zj"��T��/{9�H���yw��``=����b
�b��[>�����o�����>�!z�����>��_��O���O����'x	|^2�� ����c��J��q����9�6Q]�$����4�(�5:��_���9$u
8fF��G]f
����^�TD
6��}h~�Tp�8�-�N�QO�����y�YC8
�wW��G�
�u/I]�Gaf��Z)��l���?40�����]�X�pp���#����������Q]!�@g����G�96�8�,%�F����F
�G��SV���HJ�8���x�4�E�"����z*G��
������@������zD��-R���",��9���h/S������L(��.#�����)��<�	V��$�����1@��$].��=��1�c��i(��B��w��i��t�>�P��0\	j�R'��xK���������b��Y�����#��e-$���n�g�Q�9���)��<�	��}p ��S�C`�����������ao\jH!�~�z0�O����=���� �� ���%cF����YJp
��>�����qZ�����-�a��(.�+|��(�_w�����#�g��/4�~���1�s�x(��S�C��d��^�t	�s����c��B)�[4�Ap��Z��k�q�� O�aq����=vj&+����nrA��S]�A�<�� ��p��9�Cz����co��_Tr�d�� /��E��',�Yt���S�C� p�TK�x�F�V���.�����I�G�����`�0Pn��r��z���"O11�P;��f�0xFj��A���s9�b�N�i��M��3�v�@�����c���4��'������s2��>J���O.��`JC��I0?�F:x8�G���[``:���"�ktYG����j��F�~��1S�70�����d��^)��C �.���x�<V)�3�>��l��5���'�k�X'Ll������
���X����r�.�:�<�| �'@���--���N)X��%I�e�U�8�����s�I�V ��P��r�� 
�	4"N������]�fd\�g
%/��u��!c�Oc�m���&u����Z;��M�t������B�%��#�����������?����G=q'-�Lny�C+.�����"�`9���[
��~_1J��A����}�~�e�%���%H
��]o&�P����y�~��c��J����B��@�MT�C�#���<��W�i0�s���lO�Q��G�o(���{q�1�(�d�pJF	FHl��`��K;����{���S��7�uop������Zu�m�
��V�c$+�k�����Oij��=G�KR��n��z����I��u��$�����4��(�y�[4J��s���}2���t
/d���d�p2J8%�`b��	#��/y�h9��tbGR+�~p��fZ�����H�m��I�������FF9���W(����2��%���3d��[I����/��k�q���zp��i�Y�L��.������%����C�<�t%+r�JJ�%%�PR��wpT{��Q
d�J[��K�GH3���Q�i�c��c�>�ta����*mt���OinW���wa��mV��oa5D=�<�Du=8r����^6���4R���� ���o�Y��/��d/��8���[pU�	WEap�{���@��*.����*�����G��g��������m�(����H
�`�����|���[�va��mV2��)Q'U�3Q]��Q66�
�j5o���i��j�#p��/��8��2Y���q�a	����SK6����"�0����9��^K�Z��_X��df��TZ����`��3'�-�>�V	5�8�}��3���[����@#�S%�6Q]������P�j	�	��9*��qAF� G����~<��F����8���[p[�	�%���V�J�y�&<b�L.�����
�q��}�V�7��T�9�:,��8��9G;�auX���t��Zp�����V����Q���b�M2Q]��aNXm_��������JZ������a��WX4� BS�H���8 ��[�^�	�%������9K����X�jc��8�r��G���d����X����cn�yjN����nQ�K|i�w�s�[��1�j59�4�Du=8������W��hnK���q8���h�h����z�0e�E'�0�l�p��	�au#�%��kn6��Q�i�#�+��u�9��0� C��8l�O�j�b��H�[v����c���,�8�!���JV�N�l5�m���F�����V2V8�M��P�a5gv����`NT�V���q��	�`���L��2��Z��j5�d)��6�`��7���H9���gZA�T��w�!��1�8�^�e���x��q]�s���9�a���J
�u�����G��J&������!�����qX��\,���#�>����Cn���S�L0
���C1�e��=�.�)<a/�_�����au�y�jSf:�D�������6�Q�9v����xm�9�=���n���r�V����LT�C���9'y��h<T����";	�Xw;�������l�V��p#n����!g��$Y�kQk�x"�s^�������=G�<G�0f���Zz�1������y8���E�������c��s<>����x�"�G��LT�C�#���ec�����,�`�5�z�l�����>����'�.9G0d�8a��irO��O0�������r�
*�����'`�i�Z4J��!�jo��/���}g{��]����#������mV�6@�2E%�6Q]�~C&�2>��j��a5x�9*�3R��}r��+�d��8`��-2p��	�Z
B�1�c[��3�Sp��K�)�<1�j;J����_����<V�a��Zv=f�S��7�������n������#j`^���1Q]�
HC�aXM�qU�����`a��4c�Q�h�*�dYvI��(���8`��-2p:�%��R�	xV�(���j{�V�����[��q�mf��s,�0����s��7������K�>}���p��,�.b�d��q���zpd�����H����~���j9��m�����k�J��.SPa���[0d��!�����#���'������MQ�������1�Vy��#����h�F��4f��U�-Zv�-�{O�4��n�R�Q��H�r��&��!�Q��9a5uf�A����m�U"u&�C���������y���KX=`��-2p��A}Pb/�`J��#��rh��
��Q\drz�z����V���6�T�t�Z���F,8�j���E9�A�,�>a�9����d�A�,���5y���zpd��H����Z�a��nmI�C��l��J�G=�`�]x�����Cn���S���������	Pc����P����b���g�X:f"}0t%�������,/b����|�;�#���s�!�r+C��
�R�m��ak��� c)��n����s�M_���6'
hp��z��K���!�`��)C�V]�@/��j5��-����gy��Ay��c�[��um������1��K����7?Lq�7�����s�!�b+8R	�m��t������/�����,A1��&p���a���KaQL��+�L�����!�C�V�#u�x������Y�K�[yl&��G�Y�fKXm�v��\G��e��zB����(����c��J��h}���YM2Q]����)8I�Zm�@��-����-?)\�%u����+�_���C�0dRX��=w��:8���OC��3d��L�<S�UO����"��
HYv+U�)��f��]����6+-��N�(�MT�C�#y�#<6*9V�	�g�I���qV�`),RB���V��� ����!�gf�`�c����I��l���3e��������9b��s$���m�4���.{��PU��~���s�����mV��L,cH�m��a���1}����ExB4�L�8V��&�RX�P���9������!��`��� Z�h�9��-%�H����	�U�y�c���c��k�������6`����.��.����6+��h����K0�Du=8������
����������ImI����8]��?�A+���O[y4~q��'@�CF�s3�I�IW�/���<�H��}����]o����AO�X���[q��$����>�K���n����XX��N7Q]��	f�I����9��������jER51�2��H?={�K��<x�V<m�1��j��
��9��E}b�O�]�V?��s�Ly��Dv�������_�[���:������+��um�>�K+��1�f%
�����Lp�h���zp�[{���<����9h���j��1�T�nmbB��I�8Zy��<x��dM�1��!�hCC����^y������1�i���@�������0�2����>�z)��#Y�����1�f���'���Z�m���zpT�LGc�In��Lu2��Q+��0��A�:Q��9�=��q����h��3b�Q��oG
������?�PMh�{��D��Q����5(��^�Y�7B�nMh0Z���V��xn�.��c��JK�Z@�Y�n���p��-���8��-��'J�{�Z���k����Z���A���-Zy����Eh^T#��>�1:ow^����e�^��(���#����l{�J�$YV&�H	����E��.�KA��n��~&YX�I�h�����(���s.c;rI�5C������c�6�*�Jn`����M��:���<t�^:��1��N�g��T��d�����N�!���"�4Q��Zx��Z�Mj��dS��F����E~����c�����n��i��BA��_Sd�����@G�i�������)��n���3�<�%�������N�M�,���H�V�E+���h���<����2��]^m}����c�������::���a}�16�[Uq�u���x]�Uyv�V�mV2l���L:A�m��1l��1�c>��������P�,��i1��#���������s���-�B��N�n}0���sLV�6���A��$y�w~��������L�s�6��dY���q���X��%�Xw�c��qoumV���q<#v�r+�*N���x5�u���zp�.0:�V[2���Z�M�	jf�Cnu����� 0�����A���-�B��N�nE�-�C��"�����B����=�N�0g�����q.A���iM�!](V����?�n����o�V�R�9>������	F��%��&��!�����eN�B�I+lL�.h��F�\�!��[R����
�n�Z�t�uk�Y��A��\�V�%�H.���V5|����	��t9����+���p�Em���[S��!��w�V�mV��,������&��!�� N�s�)���)�&8.��]�Y(4y�����3 {�rB���?���1>f��x�qP���c�������
�mL�k�F	��C�
�moQA7�$o�]F<��W��#;���x��r+��hAuV;��&��!�Q/�,���q�aH���Y�����j������hU�V��R�.�<~��Cg�n#�rA�!1�|R��:B��
������0}�i��/dR��f��0$A��E���zgK�2�]�FpTt���c^n�E<�)��l����V��x0k<�cM���Nd��&��d��V����$��~��C���-��N��'Q�A�U��������c�*�R��>�PD���%�r� Xf/�/S��2jy��.�����6+-����r�$���H��	����AuW)G�0��OV
E����7����7�?�n���3��f+� ����1��5�pSs�C������%m�������(}�6�Hc���M�����n%U(3^��a�>-�g�1/��";a�\+s�1Q]��6v��<G+�e�R�jqj�<B��d��
�l��(f�3�|���#�1|~���x��jB�������"X�6�v������(��������P}�-6��i8���u��Td���dv����6+-�Y�XOc�m��`?F]q���AEh��L�:��1x�o��6����R�y���[�c�T�|rMS���v�t��������05t�����,8O���d���c��jBgrn���tt�1��:�mV2t49���t`�m��c�I��
�u�������/�#:���� +!Y���8`��-2|"v�s�������2D���c����!::��Gqg�Qh�hV��J}������h_F�[}��
�������x�!�b+�&p�W��I&��!�Q�����u�.�(F$��<>��Qgk���[�$Qw��<<`��-2|��1�jj���,pa��Gucit��C�
<ef���$��:�[��c����L�
:��u�n��|�D��c���1�f%�W[`m�oN0�Du=8"�$�[/M��:�N����fe��S)�D
.���h�	0d�>a���S��j��8���&�FNV�jRdy���Q��z��7M���7�5�������]O��=J2wi�9>���d���"c=�+n�$���h�q'�cf���$��:�A
Oh\
�
�W^O�����-���8���-82|�����k-Da�j% 
~���d�Oa5�5���:J�G 4I�����{���c�����z56�=j2y�>��c��J����Gs�����@G
<�&�)&lEk��>DF�!���0�`@�p�e��]�< ��-H2|:D��j(���K$#�����X\�R����fU@�'�wQ��mwp��4^qb��e�3jt�n��$s�$�r+)8��n���f����G�Y��i�P���W��������eI�y��(�$�n���.Cdx@��[�d��$����h��q8svY�$�>hp�����W�����4��i�X��QJ��1���~�����.�]dy���6+)8���d��Wj��LT�C�#�8���T"��!���lV&k\�Sq�~�z5�]�y,�K�OY2VZecg	m��	f qW����?���X�@{b���RHH|�$� �94a��[�S�9���$sn��������F�0�Du=:���������[�a5(A�pW-�X���63aG�������%��,��[@<.�pf�
�/�_Z��2�������v��5���zu�v������X) �n��;����2b��n��2(A��gf����G����Ls�R-H�6�5Kg������')��e��O��t��L�K&��d|T[5��9
�ou���A�n�?lV�kG�[�b]-�e�L�.#-�BEj�k��N�����.�x|�YI=G#|��j�g����G��'I:�uUe�(�t,j\�������7>�f��:�UX���$�x�L<!��#l��������B`��`#�����NG���7���?O������F��D���X�CR�W�]/7�������.a��1�f%�W[�1�m���1�Du=8���I����#��r�����::o/Q(�*$��������L�G&�pd��C����X����gk0��E��)2���c����K���K%���K�i�Pw=f�)#�6��.�x�#�b+y�c,��2�Du=8�I�Y�O$�#c
��u:FB+��~9U�QH���W���8���[pd��L�]��|�8���^l�vJ>8H+�	���X\6��18�&���KE$��1$��(��]��Q�"v�s�[�
2X����*��d�������4�����6V�a��M������1b+82��x��	V���<	�M(���5�j�M.����%��X�1� �r�a�M���y�6F���-��`�������c��#s|�Yi�����]���LT�c�#��I	s#�r�y��a���|������P��0JnQ����F
�q����`��c�L��AZ��(w�q��r�����1�b5�0)!���0�:�8����c�6�(]"W���lf�[v+I�p����OT}�!�b+Y��26n=�l���zld�J�*O�j=&�VcP�H��!8&	
�T:y��o��fM����L�C&�2d��5��]�<����h�X+x\���O��\-��C�b��i*$c]�������1�]���Q[�Z��]2��p��l���9x
��l�����f��1:8����B�ji�������2G��n8��]�l��8`��[0d��S!o^Ba��Q�G��s����[����\�����	��c��D��G��*�QC�,%���U�a������>�x�!�b+8Z�QB���m��	����6G"'VH���nj�Vg	C�21��".�Z�8pO9�R�d�-2�������XhC���&G&~����Y�m� �L��~fX�bS��O�p@4�7�s�����9��Yv|�Y�W����Q=f���zpD�$w^s.��E��>��#G��\D��!����w�A&�� ��	����k�c�<��3�L5QB�����_@x�y�j�mLB�v�G�2&�Q����{���*�g�1/��~FJ�Fy�O7Q]����T'�=tT��rij^zLB��N�6{�9�>]m��d�;Fn����6z���	J�fS;!}���g�+j���W���9�u�E���R��m+�f�F$h�(�Zv�k�3j]��������mVZ�1'o��MT�� c������Ud�����0�p�K

���6A#UK��N-j���#���H���p�d���&��*�F��0����->#w=�����FY_v+-�<#]��Q���x|�Y��9;0�Du=2��e�hZ���&Y��6!q8FFcW�1������cxd�b���3�4��2u�!��������8K���FO��`0S�gA��mw�C��bo�`��+w���s|�YI?�hM����l���8k� ��M3C`���C2�����h,���w��\�q@��[�bO}F��(�E���b�3
�����������8�6-�p���t���.�����^�Be���>�Vd�]���6+-*<I
��MT�c �l
b�M�I\�Z�!E�V��2�S2�+Oo,����\9�~��F���#t�3��k �.T`�N�>c��j�VN�Eo�4d�r�������8�(��,�Xw_fm��n���c9��n&�6Q]���j����u�^��I���6-�{m�1x�?!���WV\�8���-�0�g�t	������ �l+�>.��yF�Z�����C�G�>E#�,jI��(7�]��Kp��F�'�x��B+��XX�&�E4�Du=2r�9-����r������g��Cq�h�����Q7�/dcXw�3�0r2��d�9��
�������S����
������h:����yM�z�4L�	�[v�!
x�1�M���b+��l�Oxt���&��!�S�M'S�N5�X��{4
0�M�".�kv���sy�8`��-�0"������Q�I.����Rr�VXM_��#�����k4E��K�\�
��Kj}�����c:u��2�����J�3J�M��.�MT�C #�����F�R���t������4BFL�A�sy����e�d�~�
FnA��t����MG�}+�� ��i����$q�������~��������,������;����s��b+��h�4���2N2Q]��$~���f�5���f��hZR�,��UZ"5K���  ��#����)&yu�{)��6��H��y5!&_)�Q��u%p^�18\����4���m�io�[v=T���w���e���f��3&}<V�m�LT�C ���s�5$&��$���f#��}Fu�PJ��~#.�����������tL:��H�!
�XLP_����7���/�6�O��~��' ���?@J�h��)�uy����T`���6+���&��#�Y&��!���L�g�h�L@0v��]P}�K���I�d����R��k'
80��t��1�_w��8�sb�3��W��Q�d�gq��,���F-#6o(
=r6f��6
G����!b������6+2Z�NR�F(�6Q]����'����(�(2R�zhyFJy8��M�
Q�g&����B�i��I����S�)j,j���{=DF�yN�c�������%��a2����?h�U��w�#9��<����W����,a���n����^��MT�c �����7d������3^��fu�m��!�����0~�q��I����3�$�q	.D�ej�at!��v�J������q����Z;�K)������_,�~�������w|p|�Y�:���F�v��*�MT�C #!��M[��LF����W>c�t�4���������MWN� ���n��I��h��2"D���#��.����cJ��y�`�������� ����1�	���V�z�c��>>�9���d#a������4�Du=2b���P�������H���IC�Z�z+o��U��(aR; ���n��I��B�����j��:��=6S���L�9���i���~@��@����%q�6m����z�]������"�>cV���6+-�hrZ���&��!�|������t@���1A����_&��3�*.���� ���n��I���5��_:��Z��5������t������������K)Ip���I�S�J�N��H���g��7}|�Yi�M'#�����LT�C #�4}F�h:�ip�����8FFR�1�4�MU��+�}�q��I����S�l�����(e<���P"������+y�u��6��J(�5P����K �tAk'�~+L��
�w���>�x��b+Y���Gq��l����~L)�j�V<���6�l��MJ*�t�V���W
 ���n��I�6�PW!K�/���AV����-�H_2F�*mR��k'�~�����d�aY���[��=��>�x��r+i4��,u��O�MT�C #a����F�E��5B��gd
0��I�q��1yG��-�280��tf���^2���Q�PL����+���3������q��A�.�W�y����/T����Kp���.���3��JMK��It�m��C�T���M���������#dT/��Z�a�������&���O90��[��4���Y�B.%v���6���i?����2�W�����ch��:���e����;8�;���6+����tk>�g��Q��INc
�����+���N��rmZh��#�.����`���oA���$�����,������o���j�e���=�  ��8�����l8��io�m��.��$	��Z�k��tY��A�%�>>���dRx%�Hl�w���zd�D&��Rlm��Hd�����
���N��`xd�`�-H0�	F�`lm;H6������@�Q���m��lp����0mx D']��;���;���%!t�u��o.N�.�xn������Y��C�m���fn��K�&�	F�4QoY
�=�3���7bfGKc}�����L�	&��`$��V�	/
*��H	�K������D�G��V�.�^B�K?n��F��d����^C�{�%�x|�Y�Wu_��O7Q]���9��!c
����x����\���Y��ER"���x�h�2H0�$�|�#�bw@`�{���7I	4�L���%���Q���i�e���@�a<L�M��KC�a����]���HJ��mV�z�K4�^�4�Du=2�Y�	�y�W��{��0hLMB��c��(��Q���8 ��[�`�)	&q���
qpI64���lM��W`���>c�<����Y�2d���h:yp�!��]w�0�C��A�3$��[)�`�=���GsU'�����F�k��j�Cu�dsrg�����z�'��Zu����=X0y����`��SLY�;���\b pQh��*NK�1������p��n�y�+�OB��"��v���*V�;���N���b+�K6���:��l�����g���L-N�t����>�sUm�h���;����C���X0�,�|���5����h�����(�&N���C��U� c���rs� \`�X�[��U��U������w��=>���$�&���
�4�Du=2n}��%����|F=��k����m'&r����������,�|L>e���r�-����,p&7��M�h���^����8�c�M��8��9���k�DG�1����W
����.��gp��LQ���,c��l�������K0��A�V��^��K��1����`�0�Fl�x���2X0�,�|��1��PoL�0=0�1�G3�fd�5{��(����Dn��M���/M=OJq�����.�`b�E���n�������t�0�Du=2N��� �h����vm�<�3Fd�{+3�����h:�Q��Q^g��~al4	�p���Uall��`�����'��������#�K5b������W^-�6�"C�Y����W*)��O�635-�?~@�����@����<��N\��ZS�%�Q*$�1�K�Ua���F#t0a�n�������Fe�#1����	5��}�sa }	����b����'� ^��52�/���z�aN�b���.,��C��L��w�p/��lT�C�#&��i��p�<��Q������*�FS ���|�^�l��[��>������0f
Z��R�&��e����F�������F�]{A����D��i',�����q�b��!�f������W]N�lT�C�#�Y���'��,�����C���uj�K�:��q��{�A�#�~��x�c#i[�7�1k������
��w|�<j}D���������._m���cn�c>��$!���w�)�>C�y��L�L�6���
�����i�P�P}'�2��|��>���9��w_��k�:9����w�&��c�pt<������;��L�[�*8�l^��D�����Sy�z�^�O}H.r�#.\��+�(����4��!�f�E�G,�F8�Fu=:R�D��%.W=[kwld_�%�����HJ�Z�C��-�(W�����c�pt<3 F�O�%ku�%k�c����R�HW>%u�Y�'�L��R�)�v�^��A�"+.�.0��=�k������O�63$����2�mT�C�#���������:;+�ZjB��b�kq\�u�5Ah"<	� T? ��/�O�1��}B ���1��`�5��������1&�6�M-�5��Y��h0Z�c�����w,e�V��C��L�w,����g!L�Q]��i�d�`��|��Y�V��#�4�a�T4tLIIg�}"�A�~��xJ���rM$���&9ce�������w~���Q���U���"�0�1:�M�/C��~��JNB]��q���'�p��F�*cC���l���(2�*���jG�1
R���5���5zC�&�#;�$��[���a1�7cC��/)�E�J��m��@ ��/u]���rC������Ar>��e�+y�|����vQ�89�������5'�W���lT�C��"����*	P�h��EF���_�/H��o�/sh�@6:��8`��/��9
z�]<�������D06v^H�&�qe��;�U��2��:�E��_�u@,��S+���z
�'vx��}*��11������L$��vjr<:|[Ld����U>.���S���0�O��E��G9�`dGvI���	�����c)�h��Xx�����O�����:T�	���������O�?|����6����b��e����:S�����?~������������������o���������7o>��?t��=�)�b�o�}�����*?��f����>���7�����������r�������
�p�~����PK���N�;�(PK{2K18sept/pg_part_patch/3_3.outUX���Y���Y��}]�d����~E��r� 3$�Fh����V!KV8���v�"g�3C�����@]���pU��{�v�|����,� ����������o����~��_�n�����g_�������������/�~���WY��&�1���������}��w_}��WD�x�����?�����������3��|��/��������e����������?~���k���db:}{����;��v���e����?���O_|���/������w�������|�����������������x����������{�?x����?�����~x������o���������?~�_����/_���|�����o>|�������wO�<���S��??����l����[�n�����o������������������dJf��7������Ek,�lI�;��fKRL�X8};b"N��YV���U��
������r����y������w��z�������o�?��>~����z��"�[�:'=������0zH���zY�����?=���7��?�K���o�����7��������;�)��x���3���|�����b�!����K;H���y� wV������'N�����;~������������D�j��A����p������QV��}�1�['L��������M�n)K��/o>}���>�{_q:9�MP-�	+H�-{�|��%��I��}��?�?|x/��?2��>}#?��Iw����o���
~����T�����21FbAc�3���Y��|f�W�{���t��l�)"Y\��/}V�G�+�*�|�}��?f�������C&�?�G,k�qu��|b��b��vn�7�~���������o�����Gwr2�<���Dt�����;�|*1���\8�x1�<*�}`tqzk8�]�*��}�����+C�"���=�������%X�J���+� ?��!���WO����W��	���T��{:�z��]�������rJ��=�Y��"3��oK�S`�����_�?��%BK�'o>��,�&�|���7_?���o��%��b;�?>���d��	������MTV=�����?�=�e�$��C������[$��<;E8�2�����rc;�A�������~�Pp�8���A~i��(��z���4���� ~��8"��^����[7
�7��y[�������D���$�iW�-D�p�d7G����d��p	���m����s�<�pT_�Y��d�~���=dt p�-�X\{��&`����.bq\�s\�6��<�-Lc�8j�
���D� ���}�xH�.R��TC,��)��ect�&�8���tr#�P�"G�P(�1m��x����G��r���Da�q�:
�4�CD�����wl����
�s�D��o8Si��0
�����+����4���K���oEY��@9 �0�%��%G�]�P�sNN<����#����x�K��<������6NY����rBou�d(��$���H.E��{��6�P|4~2�����@��@��K��u�@����B3��*��E*�WC�ai��k��w����8�)��S6-J������a l��+���#��	k9f&�4�8ec�Ah�J�`8� O�@`�b �_=9�*��OY{``����d������K�4�[�Zw���z,z7���������vm�x������X8m��a��9A��'l�������Rrce?���Vx��`|�@r�Ng�S�c�/0P�"O�+Y
�w�����r���Ej��H�<8H���r.��^�$�GT�� ����xS���A�b`\��%��wh���y�n
3NY����c��`�	%�lR�2w10�G;���0UR��j����~`0�!�Kt�����
����$�|��ye*��n ���X^k���M�����&��
\0����%'a�q�:J,<�
��4��H����Z���`�6Q�e�(�];
}�/������� �Z��YA�<u�t�b+�|�XL_�Z N?�Ct��4C���G���T��t�#����\T���k��H�L���2��V�����8�8�De�5��(�6�cLxS('_���Q������G(�}�'t�E���ut�?���]�z��\���jm��y��8��{���S���h_��b�@#����{�v�c�ID�q��'�~��k��K��o�6��>?���$������+SG���C�#��=�xj���lB��q�&����s�5�<_��Qp��u�(n2��$�8����[v�{�V�Y���-�F�;e=��0��������k��Z8%d	 ����k7_����v����1\g%qA�(��fn�����
j�q�L��c$���E/s��b���+8��cp��v����Q�dwIFA�V�	%��{�H�5����(��W����3Js���������K���Vf���/P��1nRv>?�����#k\�����2QY�Gy�F�������s��K\�:������sL�A�X�|'+��F�����.)X=����@���e�'+�9��9��9�`>�Vs���DJ�����S��j!����'��x#8�M�z��p���{�@!�i��MT�!���Q�� ��z��9��3�f�*h�h�Y����T+2��;T�U�]PU��'���(M7�����I9��b�YX���#��g����&������9v����#o�]��#����#lB\9?�������A���e���:8���k�V�yUC%�TMcV����������K�g�9G��7�����aq���O�y��n&�E	�y��������������9��]�5`�]�����2����%�����	��0Zn��r����������C��rc��qp�jgpt�I�V���dXB��r��7��U�7	�;����]�[��s���S��]�����
�x�X��c�j?����JzAY�j�6��m
�R�1$y��st�5Y���#n�9^���n%G������a&*��������a_�NO3	��{W��Q5�2�/NV�3�&�c����`�������Leg�������{N�d�1F��^PV��VG��5:b0��9�m#��>�R� ��'�����Ei,��j5n��807[Iu�Q�jh�h�up$;�����9*���V�\t��i]G�s�Z����6��q:����.�0�q�9GMvO%������A���������,��a����j�j����`'O�F�sw�1e�Gap�����J�6dp�z&�8�De�D�����V
2�.p&���ZKX��%�n�$��E��A�qVNe�au�~����A��1*'g�#
Qrzx+���B��m��C�����y��]s���*�e��9n��x�"s���"�9G"y?G���C�#�Z)�nXm��9JX�
-����E��J[X�hQ1�0d`�\2dZ������.��U�N�sA�;se=8�0�Z-V�|"�6=@���{�t:�����,�F#*^�s�ap<?�����
�9c��6QY�G-s�5�x�V{���0�����k���$���_HMt������c�!{0d�����q-b-�O$@��O�eG9"&�'�&�K� ����U�]��z�~�:��������E�������c��s<?���t�V�v��L�c���:8RZ�[����PfY9��� ���.82����:xC��'�$���������&�d������2A�b�JWVAG�/6��8l\�D��1df���l���Q�9�Y#��|gV�M�i���:+iXM�s�E��&*� �hG1dR���4����V��n�c���C)�����z�r[��C�`��%CFS
��-�����'��Y&���-�~�<��)5���`j�>p���](j\����i���YV��9��uV����Y� �2QY�G��nPXM�zW���k�A�jG�A:����9F	h��l��Z7�;��!��\�L�G^��I��0Fq1�� Ccd���
���NN#�b��������Y�dN�F��i���������#j��K|�����C���AM��C�E,�R.�:��j��}�^/��yLY����l2:��!���j����HP4XY���d�r4_<82��W.�3":L}������1���%����]s��#�'6��z~�YI�1�V�d���MT�!�a��8�����c�Iu��8��&p9����}�P���-~u�����=2p��)RD�sD�n�cwF��lj�y3��_���2���!��1��1/T�YCu��]s�Rz�dY�����27[�$Y�Zd���� �up����,s��JA�{����Ts��,s�	������������-2�a���d�8%b�9&�/"�db���[8��M���&�f"�� <������N�v�#< �h�V�+�����\�	h��&*��k�'�M��&p��Y�c���#+%���(n�~VU�6�&}������K�����*�8�aA�V�TKOl����z_>8��T����#�)���a�F���k7�
q�W��l��s�!s���s��jq����c�#�U_��SM�a��&�5����g�Db-��'�#T����G���c�!{0d��!��L5���3�x���*�����:����cY����Z��
s\�����i�j��>Gy7�kCdn��D�^�k�F���C�#�a}�:/�K�Z.<6�[o<GrA��D����1�Y����a�����a7k�q6�}����!��Nx.Y_C&`	���q6!���$�/3�n��0��Y-<�I���1\g%�.Q��jU����A&*��H�O�*(�"<A�f�e)`���h���V��5���Ci��v2�C���Ans��}0�����hF�����������9"��9�A����r oK��^��	8�v��+����MTy���:+i�Q�j�$.Pm�����v�[�>���=	OD�!��J��Z�����*<�w���������2@4�P�sd1�NR6ASbZ�o�W��a������������P�Q���:����|���������c��J��Qp������1&*��H��&��Vp����Wp���A9��?���tV�j�S����V���/[y$~1�	O@�]j�|M��Q�Q�l�L��}�c�qz�6O{���7��}�2	.C.����w'������Mr���p��4��>G�h�upd5&A\W<G�8��������Me;0�@5Y�I�;�<�G+^��(oq6CF�L�=G��-�q����Z����=�q�zqO�nk{3��dT�Cc�i�n>�If-k���MZy���:+M�eI"�di���:8���e�V��e4��G*C&�VY��#ea�Q0�v6���&��i��=Zy�����9}��}����TQ+B&p���}x�auWm 8�T�}���<������Z��v��#Y�����1\g��=kA��6QY�G1� pLI�������%�������O���J�k}�i�&p�����<xE�6�:�[��[�d )-3�*���iF�_�������)�p+�4I�iA�6�d�����C���0d���:+MM�An;�2�LT�!����|<�����k+�������,$/f��!���A�����<t����8T	f#B_�L[yb��c���!���=��4��������I�$�b�e����w�-
��&���d���:+)8r���y�MT�!�1�jO)��H�)�%T���6����%��]E&�1	eo��V�����[y@�h�1��R.������q&PG�^�xp�4P��jx��R�j_�*!����]s���	fk=G�I+��1\g%�9��j����h�up��	O��1�ag�{�3��RT�r����SC�M����C{t��y'�
����!��#d�&�CYu������j��'�����������O�k����0��c�]���M���Z�g�b��1\g�Iw�'e$�h�ulD�vlG7��4_K[2fr�� ��@|s8�#y�b���[�9RG������K�[���Z�����-�tQ����k�����Z��am����I�,og����
�����e������V�jQ�m�+Z��[I�1�qT9-7�De������:["��X��qbf�]ju0(S;&cm����`RG������K�[m]�T������W1v:�����Z=d�����a,A����M������I{n��~��x�C\����M�1��p������n)�n���:8����k��`&��9���-%!p�w�zLL[�NPG������K�[��+{���.{P5�#��`�;���`�����[�[�p
K=���a�Y�'����G�9�M����p����A��~L�|er�a��c��X'�2�&8N)���]�Ii5�@S�	�:)&wN�`c�C{�c��c�<�f��p�&�C����
�5yH���tl�n\��+���m-*HK�< o;�F��k^�}���6��~��V��cn����)�1&*���i��D�&��$D���bXG�Z���r
=dp��j}Pk��4�t�1�?��H����*uK�|���Ga��z�J����}x���:��&�p������0$!�`H[u�]�lqw&��Z	������~��V�y"�xm��������Y��1�l0k������
(�\����Dl�ip��ch~]�c��J����w��|cN�bDyM8�����c��(e]m�qY��nk2-���d}�2�Z�H��c����:?�����j����%m�����0��t�q�r�����#[11f�1�X�|H��A��ch~]��x���	2)t;y�z��P�����7p�^�����v��u�sY��n�O$�����%jg�b���%�yDX�M�~��V�4y���<�De-�l��y�
m����X�J
:1��O�	^�q���r4>I�������:8�?�����W�1�8p����a9��%�������/��1�����m�:�����Qn=d5������!S(VO���s<?���d5f�z���G���C����W��x� �#p�x��4y�Gg�JZ��#{q]����Z�;�A�_j�f��;����I�kY��D������W�����<������������*�������c��u<?���4��JGp�MT�!�18?�uL��2|P�ckG����:��udq�g�j�E��C�����Z�6)#p6%A�.]������I��1d
i�x��i�dV;���6�B�i�w�������~�Q�����27[i!�cQ\n���xL�QG'p�J$l�<6��QW�j��!���A%�*2a�^�a��=2��!�@���9:��>}P�f�U�K�v��4��!#�=%a\���)�����9��|-�b�E�1�f�w4*������:+M�c�]xi���:8����nm�Z�y�t����j�+��l�D&5!p�-t'|�!��`������Z�M��d��Gt���eb��N��xp� r\�Z�p��vU��j���9�o�[}���>�"#��&�x�!s���8��e����C��N��LF����V'�}��c4I�$��^p�z�d7�;�C�_0d������B��J��I�ke��m�����ct���*�����HZ�ud�)��w������������v��FG���h�up$�Ud��|�r5G9^�s���VshB��A�WS�7��E��A���#d�wo#1�L.QdP���`2�l<����W0�Ui��3q���m�o������i�f���=��'nS��B���J���"%glk�e����Gy�f!mt���OS�Z�]��Y�|_���gb���ea�2����{pd�G�&����	S�\
��j� �)=4CG#������82@&�Z��m:Jn9���$�Q��i7k���7��9?������\�N+"�6QY�@G�aP���Cm�!q��hV��tt�9�9A���(n�5o���!��=H2��$���Y�1�~��@���2Y#8��^����kcY�����m2���L2��$�v��q��W���d�
���J�;:-�(I��h�uttk�w�Q��X&l)]��I�D�-s:SA��U�L�Z������!��=H2��$s6a��K2��&�x1��C��,��#����imcYY��z�W�mS�Q:Q�A�Y��YO��_�@��v�	[��p���v���v�y��&*��hy� �h�Q�����)�����	ep���
��N7�:8�I&�A�	$�VU�9{���8	8���A���Z\�����Ok�bY-�i�L�N#(+i�$��z�i�:U�#�n#m���p����9�L�g���:8"��U�*�<V<H�W'��f���3��1��p���[x����	{pd�G�2k%�v:Jh�gjr6��T�6�����F�`�i��	�
��a�[O�/��C��Pf��z�R�Z]��$�>?���4��3��1�d���@qe���b1���H����9&�����Y�Q�FO�@H~�f�����=82��#�t�esGj�n<,�-eIGU�*8"�a��1�0�nk�JpqaPB�}��k1c2�lu�1l��827[IU�T�6	�Vq�q&*��H`�f������������8]a�4$�j���B��]�;�Hp�pd��p�#�la&(�=G��v�xT���iO�W���q��NG�j��qaE���{�Y%�f-��k7?��h��M��G�f+Mau�W��m����c��AK'��{�:����q������ ���������=82��##��6"d�7Q�<.�ll�>	�q��y6�+����c`A���,�f!��>:���s�1�fz�w2��Z��	G��������@���F���c����%���N��D�Sxl��|���U����S���c�}C�:�����=(2��"#�I��u9��~�Z�� ��X�3Y�W0(!���0���8����c�V��������L3�f�\��i���E�f+���)� �g�<�LT�!���Z�VWy"*��cR�5��$Z��c���Af�	��m��Q5o2�:t(2a�L���h��W�Q�F�	[.��=��7��&lY~����`X���(�n���oWI��8�Z%v����znB�9?�����Y��'�i�����upD�Q�e
��Q���b�X������|@9��'�z����zp�0d��p9EFE���"���;E���sS���!Cw�-�z4C��ks�����Z��DV������H{�N9n��27[I�13d\�G���C�������L������X=GO���e�TeB�1�����6m��L�� ��	^yZ��t:a�3�����2��{�2��D�<(�^���dG��j��mx�<��A	^_u%eO�� �#<�m4����:+I��y� ���a���:8��vVs���F�V�`�w��r��L���@���T1�
��C�	{d�3�T�5�����h��I� p���4�?�X��NI����pna����������a!X�Q��
;�v+����Am*�.�6QY�@F�Q�N�������R�KN}�2�H���8E
�&���c��`�D{��:y���A����F���Y@��
uD��G�����"c0i*���`��	�:}B{x�nv�qD�ku�z���3��JZ����G���h�ud+��.2�����Y��"#��vt}F���Q��NhV�������=�1�]"#���	��E�M���j���Q��WP�A;,��*	z����y�
���#��r�5�x�����t�jF�&>��\g%�!�r�%���~�1QY�@F����M�Z:J�����G`����3��D����oR����M[t7�/&����p%���V&�5�~�IJ����A��@�^����`����y[��w��3Fo@e��������M�0�gp��lqc�s7�m������f�r�N��W;�
�u���S�L�mh�R�k_+�wf.:��!��=H1/}F���<~AE%����s+O#��Z���q\�
9��67]%������h�Uo�]��c��k�1n"�x~�YI��X��F���c c\�t�����T`��P+0C���I|&�em'���p���9����FL����gt���@��*0%�a�2~�j��/���a��$�m��S��>2&��Gk�����M���uV���*E�������2�M�L�h����Fk��i�@X��;:���C�u�-�����{�a����`�I��wD�A"�6Z'W`���3��FD4��~�&�H��gQv���g,�������F�M��:�F+)QPs��2��F���C �wqLK�*���
2Z�FZ��v��$
d��g:g��P}�6�3v�0q2L��Y9��aB^�|O�hZYl!��0�/?������I��1'?��b�#:e�i���k��G��n�h�������yF�3�j')��&*���E�,�1q�3�!x-���X�<�����9�s/���l�g�0a�L�/���6�������A�� ���n���/�@���wY�1ZM\��*t(=�	���
)+��]s��#�S�M��
�f+)2R�&q��h�ud�4h:������I���o��:H����k��Y�:�d�"�&��L���I5�[m:�>��������I6CF����)���W]jC������_��4���Xv���FhmU������JMg��(Lm������ZN�J4��J �h:r���EY>����P���� � c�����K���������Z����*@?���t�GY����qyF�a�tg4����q$�Q�i�BE�X~���&C���:+)����&�8�De����(���*o�������A�=4�7���}R���;����9^r�*2F��<����k"������i����7��`�G��?��O�yy����T`���:+im�3o���h�ud�.�gt*�\:����?(>�R4��G����eZ�yF[t�p��{p`��u���k�:���hm����8����Q78�����mcwu=r�C5+ed��v�Sz�����x~�YI}�,����F���c ��	�}�o�NI�c�3�����,R����['�����j������%&������^�d����tq�6^o�!�� n��������xMs���O�v���t��VK�n��9?��������o�4�8�ud� x2����*j?����W�7����O�cc7�;���W80���q��D�u����!�)J���e=��!eu*0��vfZYli���&T����L.�?�7�&����:+)2v�7*�;�De	�����dt��Ii�3��v`	�u�S�#������-T�����=80|��	:��"#B��D��V]RZz�M���d�+[���g�&^���b\P���{�eu�����#�P�m|�k����yFw�����6QY�@Fd;j�K�N����������hZ����c���-j��������f�[����:�5
f��3��CtN>�zX?#�����Vu
�K�J4-���aY�]��1
!��E�m�����uV�:�S}F��x7�De��QZ;�%��i���!��1f	�M�C������.�2v80��/90�kI�"��K�R�/���6������}4;0
�%8L����~d�����'�N�z�nF
~�>�F���3��J�����g��A&*���~�>��h:��i0����uz�������9@��M�9��������K����SJ����/��Wdsy�����GY��k��VB����L�@�]�����������v���C80~d�����JSm�'�
�MT�!���A�V��Sm:��:�u&����+Ab���d2�;��v�����=80|����j*d�t�k�WZ��"A��C�^2Z+mRV�k'�~��O��2�3��o����`���� ���V:M��$���h ��ud$Lc�]�QxZ���$BL�g�����s0���'�FZ����������+s`�&�%@�u�Xfp8���u�X7�T>Z�;�����9������}�D��)/�v�R���7}~�YI}��g��F���c ��A�iq�'2'����V���h�E��h�xo`�6�����&���I�o��ICd�3��g��o�|F���6m�q`t��4������,}d�TT��e7���������:+M���G���C #��8�g�����#Q��7�����$(��M����o��IL���.90^�bj��Q�jz�!(������6ma������OtYW��dTD�n��������(�����Z��?��&���\g�	�*��:}���C #� v�x|)s��g�P��T�4���\�t&^�gT��
����I{p`�����/�H�����S���������(�fB0��3�"s��v���}������C$��#j��	2^�s���6������MT�!�Qn��f��r�i��r`T��"�l~���4����yF�T\�� |d�p`��t����L��i����\��$	�����{Z;�q`�����UP��/������k�y����V���I������v��F�d4Uu���:2�0J��K���Z��q��1�*s7��hRK
�3�9&�9Y���L����p`"�*����y �X�=����U`hH����i����M���w�u�Y�N>�i7+��Cj��(J���uV��$��d���:2�Q�	�yT�"c[��Pw7���G+�N~����[L;HL���.90JwMk���jh<�rt����(O�����!�euT�\S!sm����M#j���e��(}��GF��p`n���P�~F���#�6QY�@F�G��b�Sm�;�s%$���}�tG1G�l�M~R�������=80��b�M;��C�f4�y-kz�38=;�
���H�4���.��&���m�];��,T�h�lM_�s���p��lm������(��+�i���������Y������xjZ;�I�N������KL����2���I�5���i��A����T�� #7d�T�!pK��Q�����,T�G�3�M�k���:+���)�I8�De	�>��
��E�>�����k!�R��7��w^~\�k���;��&]r`�D�*Hb*�0!+j��+>�4��8x�>� �0��5�m�I+���a}��D&q��.�v�����e�E������\g�P�<��*dcLT�1���J�t+0�khO���P�M�%�Ze���kUr����w�
� c������KL����0����h����e�9�g��\����qoZB���&k��_��G��Sv��?dL�MT����J+7U�G���c ��90.���hZ���r��~!��<C>wz��T��u��c�
]�cS��U8�v&t�:�R'v&*��n{tf\������m�W�����V�;eY��k7��Q.��/�:3i}���7�qL�mT�1�Q��Q�F7���Y���4.����8���$v����wj���C��/��������Xe9����u
�f-���t#�k;�~\!+I0o["
B_�6�7!�`�v��0&k�Z�v���!\g&�{���,�f�h�utD�+��.KP[Ns��\Y"P������YX���6����C�F��:^��X��{�}��:4�L�I�@�W �(��0t���1��==����z��9��_dX�M�1�p��Pb�2�W��Y�z���::RY�(�z}URw.9�uj�"C��KsE2�A�!F��:^���@������!�B��a�;�|�<��|�`��48��������hm<�7Z;���;9e��7���pbn7�r"'W0�q���::"����j�������BUgu����c�h����N��\5��&������{�b��t�;j�vrS��:��zK�ak�F
����6� ���&��!����r�y����n=���0��!\g&0!��&u7g�8�Fe�
���������;��2!����cd6������T5�-����c�{����0:g����u\h��ny��}�<��;���,�#��@��T���z�.
��O"I��k�2�N������~����!\g&�����i0>�Fe!��f�}���"eH���D`�4F Q%��w$�0]����S;�!���@�KzL�|o�A����E���l�����1p6�U�v�5Myu����Y*��L*d��
��I���)\g�PJW�ZE����lT�!���,���3�Z�X�Z�yi�������Q�������	�;���x����r�$o�SW���P���(����*����Q��ull����K������;Y	e���MR_��uf�2�1w[�yO��ux�qT]F5�JO�7�4'\_�1!>����`k/x�(��������+�b�o���I�`����b����,�~
BZ�T����`+8�k�#,d!�\]v��������i���C��LdHw�
"�x�utDG��ZK�<FK�,�����).����%����N3���+!{h��d�{@�9KA�?��NX
qa��V��Pf`�4U���DYF�W����u��>�!dx;�NO�����f9�A�0��-��]����qV��0�$h�������1Q0r!��J���&�a�)��x��r��L}�(�������f
�������&:����Q�i[�����m�N�E<���G��������?|������g8��:h6�1���4�e�3�����������{~�Y�����������?���������~���O���N�<h�
����7���{�u�U���[&�*����������>�3�/:���{�_���Ss�/~���PK��!(�;�(PK!{2K18sept/pg_part_patch/3_4.outUX���Y���Y��}]�����E���X$#��ff=�53��k,�����n��R������2����2]����U�a�U��*�<��s��i��?�������������n���~�����������<=}����/�v���l���/|����>����{���o��������������������<Y>8�����>~�A6������������-sp
����F�_��q���p�o/+����������������x������h�3�����y���w_�����wO���������W~������/�.~���?�����/����������O?}������������/�.������������?�����}�����.�������������}D���?�����{��_>}�������O���?w6%3��b��%yN����lK��b4���H6C.��$�t��e�S���������=�\���o�� ������?_�%j�������>��>}���}�l2���������{��
��A�)@�~�e����M~������|��N~���������~�������O_�������m;c8���#���H1D�F��i������v�����|�lH���9t�����Y��O���F��!�i�����v�	���tv(���Y�(���1�X��3��������N������f������O������sA����B��da�`r�e�F+P�l�k�[V������~������1����t�QP��������o�A��j��8��rW�Y�X;}N��\
�H�@�����)xU>��-l���~V����&V_��R��L}��
&b���������qu����������a<�|�����~��_�~������~r'g��#i��
�������S�'[�xa+��dE�\�������2VY����/�>��2(B<����]J��~��x%��w������!�����������S1�_k����#`+�i>�e;XNi��g�V�r��K!��������?���Y`�,����"O���������|�N~B���\3�� ����c��JV~���x���MTV=�����?�=�e���;�[���������|�}��B~@�#�c'J��?YW�����;���G�^�9�p�K]p�,.��Q�����Xq���U����#�F��&M����
���8�\������H��HhwG
������f�X�����3QY�G�96�8z+��=�a�~���]dL���?��z+.�}nP��p,�%,���&3yH���a�`b��`�D�G���"�@o�M�a��M�v]E�r�@<��i�H(��;\GB��q �k |�y�$�'��6NY���9�`R�@
�k���.�v�v�a��:��~��; �G� ^� ��0��rI��
p�[>�$hvH���EY�A9!�@��[����,����N����A��� ^��s�#����l��N��w�	��u��$CB3'_�K$?��J�4/yM���{�`��`8�5:�����Ey�
�W5��lA|��O0,o�b��{\7���N~��L�&��N`Y�@���	���g�'��	J����S� %?��\F���y�	z�}�DM��(�c�XA���������x��p�-<��
�8����x_A����T�cA0�a��M�g8u}l�(��Xv����{c8�v��
����r��N��F^���)� �CTL&�%v~Q-���$;K��VO�C����]@�: HG� ]����S��`t���Y�d�\cv���#�!����q_OPf?����	�6-F��������k��ng@g\���)� aLN�[gb	�-��r���v��!hJ0h-e�I�N�=0�;�G` _;����$��W�������J ��10��tLYA�{:���Cj��i�<��/8�+��O��DN&�l��N���qI��Tg|Bbu��nV0�H&q�����;�gg0����$L$LG a�BB	F$�����-���Fb��6j?sk�q���]�7� ��	�S@��&��k�������]�Cgh\L7J%���|&�M�`"��&*���qP��#	��fp$ysi.m�l�v:k�R)�n�Wt��G���mt�_���]����f����u�0A�#���Z7���k ����HqX��v�z�����N����d�]�(�h�����K���9�f%k�7H�KM�MT�)��F�������&�:!S%p4L�y�Z��f�p]�������kJ���k�={����
sn�4��u���s(����62e�DG���<��VOQ��=t�FS�f����<�������%�xy�Y�*{L�1����2�De���l���1;��!-��!�n�9E��z��`��	8�^{�G��u()�J����f%�^�#z�Y\��,��(V~��(��L�D28�m��L�����kk��k7_���#�Rz�<�������:��rs����s���N���b^7��\��Y��]p$�F��Lf�D�nD����1�ALqW��=����S�d��`�{'V��Jx����������'R����>!�k5=4���G���Lp���.��6+i\-������h�u
pT��1����(�#8Z(=0�W�� g�ep�VY_��>Y�a�AXqW���yn��U��.8�}�1N���U�p�a�Y,�&8z��t�u;�D����%�N���w������.���c��JV��Z������MT�9�1����nX-��+��hUScVS��#E�2]V��=)���;�c���� ��k"���P�]�@�s*��d��!���M�EE�������9�|V�����w]�@����8�	5|8�}�����[I�-v*XgG���S��rc��qtsXM�kS|
���w+2#I������&������;����.�7�v�}��1�,�&��*����1y���#�a`�:p-�����q%���I��i��
"? ���op�Awy�����)8B0���&*��������6��:�B8���]�cH��"��ra��r�����C|qG_����������u�j�a��4@�~7��k���1�a���ma���1wC�`��0��V��knQ�|n�v�[4�g[I���qm��N�h�����9*���V��\7�(�S�"�����V�."9���qG0b�#�ew��5y�'G�Hs��%eVp�N
eY���+���n���>��~�$�,�y�8p��8�����'��f���x��|+����j�Tf���:8F��5:r4X
2�/����a��x�ra����*!����!��#H2��$��j�V���^Cj��5d�#�szx+���� ��I��R�����r����s�v�-�wj��9��O��
�����9GRuY�y�Bec���:8"l�3���&�Q��(�r�9��j�JW��!�c���%�=��|�#�����k��������1u��@t|��8����w� �zpX��j�Xu��t�T����D?Q��w����xm�9�=���n��-� `9�<�De���9�s�D�T��P�ao����(��.��$��2�>J\;C���"�p�"������z9��s��P������s�cdX�����ap�Z��A����]s���OuY[=����xy�Y���<�M�0�De�1m�-�{�m�L�r*���Ww�Q�j����'��9�.9G�a��#2��!#�I-���,�nX�*E�	,7?������Q��
m�SCf��������K�T�v�����#����B��1�f%�� �jP��,d��N�vC&�<DH�j�ha����sLr��s�M��@Oe�9�=r�����G0d�5C�E�)G�+�[�jY�������oy�<���5z6y���m�>r���s�K������i�"���V��9^�mV���N���h�u
p	����L(�<�����j�+��l4���V���c
�a�����G0d��H��#��.�r(���4��<G+�v��
8F[��j�~�����s��0��FR�D�����F�i�����n���#�V�
�MT�)�1��&p��3H9�;����w������� ��
��V�]f��C����W�#�l�9z��	��h��*�����>���[%�����s��n+1nm�l��Y}}�]s���;�B�<�����j�9���i���:8�$<�$��*�x���bg�_�acCGo5��w	�;C�_3d@��
2���9���N�U���[�/_�Q�U���j�8��V���a�@��>O���������	�o1d�m%G��eN����&*���#Y�H��T
2����\�iN]���eXt��jX��a��#2��!#o����&�/B�����X�SZ�9���2��HtM�-d��/�ucRn��k7��r��i����y���!�9�����8�De�����nAFS��	\��������sd�d��[�T��Bj���������G0d�5C�VM��{�}n�<$���R�X��V-��|p�[��emGF�jws�Lk���k7�
q�7�����s�!�l+ix�s�`y���:8�V���P�4
$M��&p��W26�����<%�#���0�8v2����f�(*W����AP�\�V^����s�^<8��j�v�N�^��
3��c�����v-����s$����c��J�sT��y�J�h�u
p�0��Qq�V���&vl�sL�dY���j�&�}�����mp�C�`��
��E+��	V�n�!�/�P�E+��L�4��#�k/��W�%���������
C^�����9^�mV�>G�j�<���h�u
pDc�'����Ex��n�e)B���*D��sD���DU�G��;�c�!G0d��qQjU!��Q��T���s�o6p�������9��9�3���s��)���5�5Qg��v�	�sL���\�mVR�Qs������MT�9��o����*�YxB����aW����XZ)q��z�p��;��a����!���A��/<!W����&��=G�
Ty���]28y�y[�}�+�HJ�w������]�^�mVR���*OR�jm����8l�`���
�����
���2������9�����N+����<��-l�H��&x�t�j+�������+�#��s�y�[�6���s�Si��wn��d��6�w�9^�mV�j5�ju���F���s�#�QcW<G�8����
C��x�Y�G5xNb�K�G�:�<pD+\��(oq1CF�L������cC�����%?���8��d�jP�����[y���h��k7�$���y��.�<��p���VN�i�F���s�c��[�m��F�j|d|e�D����Ky� �P�T%�T�pp�����<p���\�F��i�!��P���0.�C|�au�8�s�hS�����"�n�!:���6�O�V�
��,�E���n������"�h�u
p�G��R��f���GL����	N����"WS�i�&p�����<pC�6R��-��)4����h$z�#mU�/�S��V�i�
�pM�V��&�y� ��p�wa�\�mV��c�z��X��&*����� p4��!my����
��Q����f�V��87��]�b���h���Vo�J0��������'�V�	��_��H������s�����k�e�i_s�u?�p��?�	�`����1�f%�Vg�2�(t8�De�Ab�Q���v$���5����E�GJQ��s�I�	�<�Q5;��i��#Zy�������_.e�>����B��e�c�T�I/� #��8=G�M�y[K
:�}e4+��z���Ev����������n��z��@�N��D�MT�)�1$$<!����p�@N�|}�I,����W�WlL��9b�������N9i*a9�6�G�F}s0;����������,�32����?�2��dDw��7j�|�5���xY�Eyv)V_�mVR!p������x>���S`#��c;�QuL�|-m�X�9�`__�,.�/���Y�K���hs���-�u��Z��IlR������!^��r�1��#�bux�Q�8/��U�hR,��8vs���,�Xv�c|����6����8���}��l�9u�.�����S��v}�~�Z	��RuX�3��R���a��r 7��J���{;Z�x��-^k���G����/��^	2D���stc�=�X�d���d��_�
�hK=����!��fj�.���c��J
���j2�MT�)��5_K�?�	2�G���uI3]��	�1�)����v�n��[������T�������$h�J�1d�b� ���Z<N�S�sl�o��)���G�4�wKj�#��.���c��J����
V��&*���>�sL�dV���33{#��V������� ����C���	�l��c�~^�clLJ��e��,P?��.G�w�QbkG����������.GWj�u�ZT��<�����Y��yE�1��8�7�1�����fel�p�u
l�4Lv"I�7
I��o���g�p�V->�D�Y\;�5�Xk��4�t�1x?oH�F���(���x-�}��p9��Go��_�����G6a� �u��2?�/��0j<m�]�lqw&���������
~���4�c@���B�v���:8F�J��f�?�@�n����������'!,OQ��D�;�<�����l%Z���>acH���Jz8�����c��(e�l�qY��nk2-�5�2@}��s,�2�}��i��Z��p�����r8�0�De�1�a�ei�@�,��r��0}pd��	Y��^����;��u�����#�1x���J'������:
Su�(,�/����O9o��-����(}�V�Hb���H\����R�4���V��~��|+��A���e���ncLT�9����m�=�QkW	���X�&���� �g����e1����!�
����	G�c�
~���X��}_F/�����w�����/��������6B������1JcPI�y��Th���dv�/��6+)�Z�jo����cLT�)�1x?�#�x���un�Eb����a������a�5���:t�1�~L������V1e��H���������x\dBz�a����4y�f��0����Jkj;��[����c��s�<���4dP����h�u
p�.���@��'�j[8��~AF���5�����(����!��#2�J�V"��=��2]�au�Z���M���tQ<�s$9������M��l�_���y�(��GT��>��-����4�	�	��MT�)��G�s���5�
M���:2�V8����mh���<�C�	Gd�5A�y��������9G��`����-��L~���q9G��O9��m��	���"��u���[48�9���s���<���4����AXd���S�#�0H��R���iVs���7d'��pJ$�U��q���!��#2�� ��|��z�m��'[� ��9������AA��:*z8j�W�� �����K�:
cT�q���<� a�V��c��J�S�Vk�<�6QY�G�#4���*8�Y���xEw�j�U��u�V����]<�A&A�	W�ajl�����J
�CDJ�N�d\�/?�Hn{P�D8r��I+�c��_�aulLV-�2>� �vi�<�����#M��8��&*���.*�$�N�j&��`}�ZMz
��Yh��N
Nva�C&��	�d�wo1�L�0d$r�F�G�,k?@�9z�
�
���w�f�[�6���.=S�X0��l�]K��CZyh���
�������4���bm��N�>��%"�:(��(a�<w�ykw=Gq#����q�6�5�2B&t(2��L����"�[�1A���QK�A�?%}O�����������a���b�
G�'�)	�0�NI(�E<�?��*��1�f%Mw������&*���6��rs����8p��<u���"Y���#[l
qX�.�<�L8�#�92ZZd��g%�����BbG�E.|��
�F�z��(O,4ls9��
Q�������E�8����}J2�f�<�J���<�L�F���s���:+���r_��J�/�$"�5��m�2CuJi�%H����L8�#�92�<�~IF��Y�Rb�(9�����]������N`=����	�(���:������_���&�����c��JunuD�m���������O`|!j��h�Dqe4�|U>�	��!�5��;����;�xG&^qdl[Um��9v�)�c�ts���~�%������:R�����#F�����Ddc��i�:U�j������1�f%��(�0�xC�MT�)�� �[�����c������V��q5G���y����{x�����Gpd�G�2k%�v:Jh�=���V�#�/lY2���s�8)p9J�k��l)�������+Fg|�y��
aH�ksIf����n��JOd)p�Z-ty���S��G�X#�������(nXl�c��V8�n�JO�*G��<����Gpd�G��7]�##�n<��(m�0��H�
�+ F�)�1u[�T��n�T���5�Z�HC&�m�9�]��G��V��VC��&��6QY�G�v�hV�`�pd����N��}]L��p���������|�8v82��L���q�i0����Z��9�VSE��$�+���y4.����U��6"�>Q���I��y�n~��H��]��G��V���*eE�9	�LT�9��)p�%��#��#��z�#���E�[-�glB�=&l�G&����9�m��~�gvj cX�M�zy�5��h�1� Ss�n�M�>��Y�V�f��[����*k+8�.��c��JVGy?bn����!��@1�!2�\��g���!_�jff�T�	��'��73����L<�"/)2 P�f�.��=}K��_j��+����I�e�t56�D��.���I~)��L�� �{��A�'��E�y���X�3d�����2QY�����r����D�R�Q�K�,�h��Y��v���mL��Y�B�]�V�E&A���U�
�qT�Q�&g�����28`U��'WS�����D1u[58�����K���y�*�c����c`���1�f%���A	^�B�v���:8�(�2�:���������!�5�\���m���4.4��P���L<�!���h�&�b��8�"�N���g���!�w�-�z4C��ks������+�R��*��y�(6����r�o1d�m%�����-��s�MT�)���������i�g�����]W�,��r��������is�d��x�L������������X���-����m��'|s��Vs�#��Q��&l9��e�n)���q���c��J:�s=&����S�#�Aj���c��`���m�F}�z�8�Q��4�P�1���L<� /�	 ����J�C�������F��+�!	��<�q�j�uHB�6��sk���#)#c�-���,��R}��|+i1�M�Cn��N���d'�=4*Y+�����<��.;F�YM��� Qm�7sd�;��`���B��g�6vL\���}`�SiS#U�N�y�����Z�Md�&����lB������e�i���0����L��Z���f%�#�����y�MT�I�1nb�����
2F�����;[Kb��h�R2{W'(�Q��3��`���F�N����.���a�-Y�b�_:0�a�Fg�149ocp����R�����B���L���e�<���4���� 0�h�u`t[������>g�IHBu:�������Vn�?�8��Gs#uh1t-���`[W���>��&t:�"�8��5�[���xd�������1o���W
�r�1�c�v��Cnw��\��mV���
N�~���:2��8��r7��[��*��ft8IM�|\CT���&.:�������k��2(F+(*��C��1I� ��Z���q\�k�z��mj�*�k�S7�h%����]v����cnEF�E���n��"��a�/~��;�De�ik���f2�K@��8k� R�g4�����6���9����BA�!����������ZFH����Ei:��GF��5}>�u[��Tjy�
T[9�aBSY~m���3��J�v��A���F���S ����NgT�U��]XL�a��_��6���T[GN	��,�v��C��#�0n �7�)����+���t����D�����j
�	�@���(�
B���/�s7��k-~�����3���<�Jzs4-�[m��N������Fn3�d���Mk'N7�����ri:��+2��K�����#�0��Q�50TdT������&��i����	��h:�js
�����
���2��9D��s������o��y���<���F���S #pM���r�3Jt�M������J���B�J��	���g�a�"�52������L+�dQ
,�J.nl�iDz��i�[�e��h5m�Fd��`e�`dC9��v�!
�N�wA�[,�g[I�hi����MT�)�Qp�*K����[��mh��2H]dD2����G�������C��#H0�W���/K����ikHYo��q���/?��G�^u��\nW^����.G+0������Z{`n1`�m%F���B/d��N�Hv�-���L'��n5��x�;^0�uF�Q�i�W���N�2v0t��0l�@+��n;�\�	A���4���e=ZU������B�H���o���	��)|@���Z� �-���d�7����F���S �z�c�,��*���m�`R�_#�gH�i	�ji:���n##w0|��0*
Xu��������2�H�\9�9�~�iyZ��3�6��-���g����J��w���#��)�\��mV�����/�F���S cp4��Q���4zk;XU�q5�q�|.�����V1�=�v�C��#(0|M���/v������$����q1�V��^2��2nl�
�m#wx���q7�
�y��}J���
�p/��6+��i������ud�<������ �\���W9u����.�����F�g���L#w(0|��)0Z����=$q���0y�Is<M\ #�������QgS����]�0������y���k�>���*k���.��3��J5��gp)]6�De�%���"c����������W3������cwq;>��70L�*$���2OP������'0���e=��	eu
0��v$?����F�P���[h%���c�e����f��2�2�y���:0���A�����R�v�e��+�@��M����
�6i����d�0`�_3`�Nq��>b_�V�
�K,���<=�����eu��)�r��@k���A9�.�E�^z�j���x��l+�=;�@�4�De�����.;�y�:&3`��\��v��LH����,�=J��a���f�D�����5��V��a�@����!*'@=����i����5I�(M;gd���kOi-}+2�3c��n������;�[�4�De��M��v\0��`Zh���S�"��
I~NnFFwg�^;>����,Y��]��J��Y�I3���i�Czl�
L�|F	��qbj�7L�g���h�V�����q��X��gp��j4|c��3QY�@F���D�i������W�i'�t.��mU��;'}w�����#0|�����Y���%�7��2�[Nr2..�v��,��\�����eu�$��
��I�{gyEi'����2k:�u��0`�>�x��l+��f��i���:2;�����J�Y#�"�$���,�W���+��:0��C;��a���f��{b�SZI3R&M��8.�i��#n�5)����Z7J�Kg���F�7�nQ]���f�}����V���.�517QY�F�4����6�*��8����2t�����*�e���U�vq;>��7f��7U�X��nYP����2z>,\F7�T>Z�;�c���8������,+��y��"��k���CJ����/��6+i�[0���+3�De�*M�>K�9	e�
������@��LGy�c�>c�{`R���`��kL�l�2��q������(��!����.~��(N�0
c�!0a1i����C���q������4�D�<�������i���v���:4�x���Fv������4�D�}h$� ���^���~��&u(0�
L��������O���LH0i�E���r#���Mo}���N'���@15�\�������9�\v��A$�%��<�����uGy�cp�MT�)��� r��r����*.����+��$�v��`$A��A��������#(0�F�w�];�������<,��[�����T��%����>j��T��d���;$��R;��9D=>�6�� ��)0���V`�$�����LT�)�Qn��f��n�4]^)0d�*z�bx�
dTA	��������)�G�1u80�L�������6�V��3wMS=�5uF�
�v"���`�M�m����_�������]{	�19p�D���f�:9�U}�G���s c��D��89�m�J�N��o����������GNC�>��s%:�����#80����E>D�C{�����!���80����N��0N�[�Zv���w��'I�����'x�-*�����.��gp����6yz����LT�)�!���>�G��A��[�v���<c����r��k4-W~d�p`��t��au�����#�]��M�MPB����3��#,�����&B���yG�Mkg��O��kN�G��}����Vb9~�3F�g��6QY�@F��������N��h�<�y��K����8%
�lm�M���IL:���90�h1�@d�:9��A����4�8=9�
��Uy�u�MX�\a&�{��g�?����@��h���g[���r�jS�m��N���(�U�.��$�k�i�l_�1�Ic����-j��K�N��� ��k-	�@)�]h-���/��A����T��@#7h����z��`��Ry�w�X����vQ��<�����E�v�MF�b���S@#����[������k���t�'X��	`(W��WIV�-�:�mh��`�4�tM��!�U%�T�;!KN�v4��2n1!��^�@�0��5�6���_/�X��r�|>�e����S_����w���<���D�k�����8�De�y��I��J7��k���O�U��h��Q�(��4��4�;g�v��C�IG�`�5
&*s|>a 7���LO'l��L�WP��q�i�]b���U������!������v��?dL�]t�.��6+E�KWm�^	7�De��
�QIn��i�_n����~�Q�����$������G
F'y��F�����BW���	��S��s�"SZgk�W����n{tf\��X09D�m�WM�ue%� ����e�n>���\�9�^�mf�i����p1�{���::�4l���w��T������aU�w�4����n�C�F�p:�����5�hS�7)�,1��u��0v��RT�`��L��+�V��(����d-6K��B�:����%��a�?�����S�#0mf�<Am��N���DD���q�)���*�a�lw����C��/��71��B���N9���Iy����5Jd6�57Rk�sp���s������k~�}H`w��\��f*��"����QY�@GL�"kK:Ez�nUK���,�����6:����W�1�>������@���H[[���%$��E��������'R�#:�w�&�J�M�Z��+�;��g�q�-��;Yem�w��o�b6�I������YF���S�#�V����NLy�����v�>�s!%Z�L����gTM��2L��-F�p:� �h�v��#�
]+E(��B��+h����<��}�	(�<$}���cty�G�-��B��G���n3�������B�c���::����%&=[�wUgB^!�GG}������%?��t���:v�1��#�����i��dt��JUF�Q�P����&�������8?�68We��^{+��X�V�C��)��;UO���;�]J�W�p�����Y��N��6*�����h���h}�X�*<����V&�(IX'�j��a[y�{0����c�G��5AF��	m��:`���ZSE�tK~Q������cd6�T/�v�����W�/Xj���[d�&F�����n3��U�L.z����QY��G�8f��S��4����Z��Re�K�D������w�c#�}B�EF�p<^�d$�\�1�U������ ���ZU�^<:��0�,�����>��1z���o��k7��IJ(k3:�2���n3�s&��������� #�ux$U�l����7�	�����D���H%��M�G�)���������b�o�
���&V�]��H&�]����C�[���:uh�z��
�W2���S+��VsHC���������!�f&	��T,�����16*���F��ZK�y���Y$0s;���A%,b����������4�wx2��#���� ���})��$-EF�������z���(�������$��.q1�2M������f9��4�Wg��O9:)F�#`
Q����/��	��}2�p�}�z<&o9>�L}�(G�09EY���k���X���EB�I��4���:���	���	��w�)����������_>~����m2�Z��M!�)�a���{��������o����<?��?���������~#G�_~x���O���;X��%N ����|����7�;�V�����?{����{����b��E�C3�<������z�~����PK�u�n�;�(PK {2K18sept/pg_part_patch/4_1.outUX���Y���Y��}]�,����~E���`��`Ia5��3��5f���}2�R���t� ]����7��J�����l0I�3����f3�N��9��2~��������_��������//_��|����~������_|����_A
�#Y�����L��?��O_������?}��/������������*y�6����~�w�>� ����_�������_~������������	��/?�������n�G�������/�~������`�3������?|������/��������O�_�s����Y��j1=�Y]�3sJ�[���H'.+��c7��?^?����_��������(����G�������|��������������^s����VkF�d����`�A~g��c4v�F�����/]�����>�~��[�����~����}��qu4 ��������S�S�<�3��z(L��q�z��|����?���W�����������������H(O(���
=\m�uD����0���q�'�����������?���O�A��/.��������^�������+����/��������������/#��k���'_�G������ ��<i�����>}|��g����>}�������+N���ln��~�����������`9k�����96����o��7��A;�U���z��%��������_?|~����?��-_�\��/n.V�r#����J{��[����tX�/����������|�����/���	D���7��������������~�!��#7�}�3����+�����w�Qx�;�����
1x.����������&�
:�`�O~�AsB6$?,zy.K��*��>���Sg�/"/��c����
�xx�����a��I������A+�uzp��`
np����p���R��Ln��������<}��}���
�[�����Qx���g��k��+���
�!�.���$<PR]^>n�����s!Z��c��k��4���+��c������:=�a�f������<���e�j����z�z�c�'�2���V9���O�����A��������]l��.���p�1�^J�O�?5b(C�WY�����Y��P�;(��w�k�- ����l�Y���C�����2[�����8�+�s@&o4��p�.X��}�
[}VQ�j,�4e���B��1:��s�+CC����4��
��P��x(�[(�dk��1��_�\����1�uz(�����ea�>ce4�y+��P����V��y�/X����/P&0�K5I��A&����^�	���y��p��� �i`\��85��'d�������6����kM��9f��Ne�A
0� (�Y�LAB�,���]����i��h�P"9M�\��X6�M(�p �PR��1;�`jvM\FW��>�e��CY���h`jL�J�'����������Y��,���c�,�d6wC�A�kx�^�I��2D�T�	�c��7�,����dM����LY����51���Sg�0����"fc��WI�d�\8f��Ne���x�'��'�=������8�yy��6g�� .�4��f(������������M�Eo�YjN�|�4���K<Q�a���!c�������6N������Z�CM	Ij;_c��&0�Pv�P���-u�������^�xK���7�<��#�2�:n�m����
������v�0������P^k���^^��+��z�����Jbc2�!B������U[\�
���`�+3�	�&�/��V(���&�xm��^�-��j]�A{
D��!�&>����������[Wh���k��el��05�6�5�\sB1z)��0��/�����1\�W���^��l�F0�ss=k���9mb���g�xnew�e��:��[��or�c�m�@fd�*;�nv����d8��:AB���[��<7h&�D�oz;%�������	5;��dW�?�6��n8�l������o6��2��V��es�
e��:��[s"?��ut�����Y86��sn����(Z�[�P�'UK�.��X;����}����.����t�%��1�{nr�`����cp��!!��a)>�S���K���0��*���)��lZ����[G!h��g��z<����YXN^��&x��k���}�M��tW����EW�6���|s@mA�Q���lKw�MOh!j�{���������������*�[�	�o�-��6+_�������:=�u���ZFH����]r�c�mV�W�S�~�S�����P����=�!\qq��F7'���A��f�S+������C�X��Z���x�G=��(���!6t:T{��NG��CY����f�^��s�xn���_��\�x��i ��@�zn���� !��?0�!�Mk�o��� t��f���h��7@���6=��!+	�rm�~bN���Ojw=�@[����k��AI�q���V(������k������Y���8CI�g����t��f���.6��8�V���? ~�s�^�w<7�(��8�|�5,�0u9!�2n�7y���!6<7�}��[��CY������^���r����R���4|�����Cp��C�w	�
���i���RP�fS��������t4��D�p����
���]�.��
�oqUP�%Atc���N2Z�6��A;�uzx�
��^�n��x��R���$���T�7F��
����,?��Q�*������xK�	�D��*�[%��(�#f)GtF�Z��H<3�y|Bh�q0FM�0��Y�33�����:=�uw���[1�Ko�p���A4�v�T���������
��T�q��0��6����n���c�a?���:�j/I�=V8{��x�
l��&��"���aM��f��9h���m�q`������
��U`��:�`���� �P�#XK
�N���K�x�
�� #p�	��RSh��$��{���M��
�3jQ��W������� �
=GAYi�y72I?���A;�uzx�8
�1�����.������Y8
:��R.+���K�6�&��4Tt��l��� v���mf�*�-V����TLQ|������[)�07������\o7����:�Eu�A��BY�����s
�������I8
��M��FVm�� �6M�#��)7�ql��+�3�?�]h���1�>z�*I?��l����������>������N7�
4v�"��2q��BY�����
\��W�&�:�&a(`����<:�����G� 1i��7�R�c��7��2*�)=�~�B�]�|>���R��p�����	^���	��eo��2�W��u@f��BY������
r�c�m�F�����T�������������u��'��t��3���6�9��� ��~��w��h���Z���j)�?�Yr���A[z���O@�����;��6
��Q��BY������~����`v*;�tH2q�eq��Zxj<e)�$^Q��m�ArG�7'�������x��M��}f�F����&L���O�q�����7�7��VG��9h��Nn?���kPr�N��m�:�`~�U�Im���\���t�=�-��=�H���c.��c�WnX�wSM`�[p�oS@��y����MM�'�����,�q�a�uH������A+�uzh��	��&<��rxif�&8g
g��%��>!�#��jLj���u�������6����|}��~�-�kF�(���R�K����������&p����2�Ig�e���L;��v(����q`�����i������I�	>?u1k�8���
:�c����275��mKP��F1� ��e�J���*R���b��Q�6h1��k��	Ca<3zf��
q����sS&��es�
e��:^l�����N�Km���������r�p	6�q�#0���Q%�FMk,yy��`��B�rC4�����o>b�\)';��Y5{�_N�~z����Bn��H4�������	;f��	��o���0t�R�%�����/���/��3�Y
���08�q��� �K63��V�xgR��{�r]C��'
^��l�Fi|X�+f�j���!��70N�.��V(�����pcv�#���67�s�Yx	�>���P���v���+_���W�;����6h�26��s�,Q)�Z+e��[O�G���zn����g@��S�a|�������
Clx�hRN�^v�P�����&����G�m�D.u���M�]nA���cX�@�J
Va&��![���R�
�|sBp���-mxN"N��*U�7�\�7����������S��s�u��k�[���nes�
e���V<�
�m�{e�Y7���
b��������SJ�c�M����V5|k��N)���6���UsQ��w

NS��sS�k�N��%���C��cSPh���A(������:=�u� xP�r��@��q�V�2�]�<�T�5�����w����Y������s��m����[x%tn�G������W�����M��'TK�oA���6�X����.��V(����5��A����� �<N����+�R�KA!���sS��.�K��1����	��D�'-��[B�����A����A���}�
���c��{nV�#7����\����;h��No]3��A�����:�fiQ�"�v���Cs��1��}����N)
 �%
��c\7oo���3��uw���M����P����P��u�
���K������S
�Q���m�
�yw�e��|�
���TBA7X+���A7?M7Y�)_��!��x���BA��m0���������r3�~��&
�GA���A[�e��nS��_P��f^��C�S4r����A+�uzl��A���:��l�K�m�4����:$�����kA��znI�MQ�J�E�P��GiTzh��3�
�����sC{���Sn�#:��I���+��R�������n��&(�wd���9h��N�l]�_����_{�J/l���c�W~�>A/*����S���^�jE��\�k��z�V��D6����|��-��<����H��1�V��M�l�	����&��EO����`��l����A+�uzd�$*�1��}���B�I$*�eo\i����ntg���O�8�=p����C��7Tu����������W*'��1$�OV�6;�F%>AY��>��4,�b+�������:=�u�~C������6;H��O�Q)��`lNJy��0�c�op�Zq`�GY���D��9��6�I;�����/u�����!�QC�i{��{���!��q<��*�*���a�<���rJ����:=�u�~C�����;�6TE�E��!�^bB/���L[�mI>�"8H�����l��L���I�R[�r��]�:"��6ot��������l�>��9��a�u<��#����:=�u��&%,M��jr�c�mR�|�1�y��E
�������#K���<J�5����<�4F���*�w��rB'�����2�~xA������c[�J�2G>mb���4���ew�e��:^�?&Q��nr�c�m^�vT�����|JP���D�5�Cn�r��k��?��y�*-�R�`�I�5�a�)T���2�	l�a���sE~���_)T�h��qc�
��������:=�u��&&�/*�K�n������(�
������4R�@p���������o�[�9��UR���Q��Ae�O*��vCL����\��+	�	�}�J�����a�O�T��9h��N�l'�op���w���J�,�����?l���e�<�S�@�����U�������Z}J4>st�%W,.}�k�MO��F���D����3c��`8��Z��N|�1����������A;�uvt���@���E�qLTJ�p<B�JU>��J	����S��v�s3����o����9_�wc�"P�?[^����@^�B77���'`[�HPe��v��nC�t�!�A+�uzd�	�f$$���6�%�Y	�_F�YN\	��a��@���h�nT@h��8���� ?��J���`��g$�CVK�����4u�.=A��V���"���a=���*������:=�u�Z3���+�:�fa$h�^,)����aWq���T��rD�B��V��Bxs�21!:���L����	�N��n�^y�q0��g���$X+�������K�)��v(�����h�Jx�%����K�n���%zS��a�-�O)U�r��V�cI�a�c�@�7�sF�9"9J7<��H���A���T(��ms�����$��K���J����I��BY�G���@��GrmEQV��z�c�mN��*�i�Xn�_;@�M�q����x��������
��QsB�����>5�����NK	o@��-x��>���=��op��t�"z`8fy��;h��No1�6�	�@X�
����&�:��!&���`��jv_�R��M���<�4}�2�� ��2������	����:c����7�4�xy����h�L���%�~-���3nys�
e��:f��	���@��M.u�M�Lz�
m��P��v�M��T��:M�Q-&Dx,��&���b�6������}�]=a��0������x������Z�K����VAiB�YAes�
e��:Zm�Kxd���hy��1�6
-A'xf�������K�j
�)����[y	8h\5�q���
LU6$�y	9K���r���(���
�	���4��C�bm�A�u�����:=�u���%<���N��Ko���+-�	t^�5��t'�F�q���$�FR�z7C�-������+F����7rnrD-�r��j�R��No�Ohq[
L��a����+����A;�uzx��	�&'<4�j70�Ko����������R�V���.��9�R�I7��	�������7��	u�}0[�W����I��r`�R������n~|I�{v������k;�Wb~��A;�uvx����kv��f����7���@d �y0��:f�$pits�����@���)�rCO?,dEg�����>�TN�u�G���ganZ�`<���q�L����a�9%.t�	�es�
e��:z��	�T����R�@�4���Ym*��Z�1�����6�_���[�<����9�����\J�5w����w�x� v��
�cU'd�07C�x%7^1<���T�RE,s��CY������k��#�nXz�p��qx��`�������6#E�!�<7V������&���������0K��4K����TO�:E�sP�����x������f�)
��)LvX��}�� es�
e��:�o�����\�h����I�IA ���^_�w�& ��f4����i��}s@W&�xR����`!��/����� ^��:f)��7���c[GQ@��[�Xa[y<t�|��BY������k��C� {��R�`�$3Eu4��c��>�*g�X�"	J��������q;8!�I8������6�4�Tzr(�]eD���n���0����
��!6��d\�/_v�P�����(������t������Y(
�2��� EV��$F��7@�72���[C.en�&eQ���(\?��|y���F^�B�Bb�&F7
�5x��^7	\�]k�u]'����9h��N�mG�78
������y�R�`�,���7���aW/,��	�*k,���+o��sh����r����T�n���t�S��cU�ij����.^�)
��~�k��6yk��lZ���#[GQ���	�Q�K�l�PP�b�)&A�7������������x��	�J��	)K��|L�&z�G�w����/
������1=!(�)
�l�m��!�E�h���\6�P����#(����*.i�{!���l�
���BP7	��R�������$�$�Q��2�L��U��}Z&�.#� ��T*dU-�]#Eq��O(�������a�5�9�_l.��V(������kUhK���2nn�\�&�57A�t���Mp6u�����.�������^$gmq��#k8���9�/
�M�\L��n)AB��]-�%zz/�}6d#7^�7��&0�J2h���	c��`�es�
e��B�Kh���l���d�;�laZ����RE��i�>�����]F6u��	c|���!sJ��(_��C���fY�<s�6�a�����W���"�f����6�Om��BY�G����^���������g�;�l��t��vW;�m��y�w���v�j�M�:���K��]�9�v���8���K��ty=a��B�(��=�Wr���a[_G=+A�);�a��(�a���v(������������A�IH	�=�\(�&��$���op�1�J���A9�7�
�r����5up��v?�����%B���/i�	�4�+�/#(Y7l�a���"���;4���k��N�l)�����w��h���0	'��
��8���F�#�V���H��$���K�0��U�7'�e�!VK�.�V��|��F�tnB�1i�|�<����� qh�y���(U�	��L����:=�u���b/�F����.fa%�k����9�ii�q>�;Q�<����L�G����
iL�4��	}����q�������)
�F�5��������jpBj���!�X��l3��A+�uzl�8	��>�m��4�(f�$8gs�Bf�W������TIg����Q��7�J�LM���9���>Q^y�*>
61)<8"�r�gC6�������*��a���1O�1i��BY�G����^��1�>����A�i	�|��4��u,(v_�����T��k��Q��������q\.�w�v���c2D5Rsbd�4~JiX1T�o��l"�es�
e��:FB{���z�c�mB�w�&(��yF��v����K���
���� *���b.�*^�J�������1��U�J�Kmnn����-�>[������
���N_6�P�����#��B�zv�}6����FP�")���k�w���w��H#��P�?�Yz�RsBu%sk[BS�V��3�TOhU�tVp#n���mc��L[Ze�\3��1�
��Tr�lZ���c[�G��wz�hPo�$|LZ:P>���c�#n�X�"n������c�v��9!fB��/����S�4����]���k���9���>�=��#���#�F��1�FoR&�uw�e��b�I��8	w�m0�$��$��\#EoCe3�<w<72��`�o���C� tkH	Q�R#�&�A>Uw<72.�R����b�;��O@�~X��Z����k���E��\;M�gZ���c[�J�kV���?�N�`�,��i?G�d���S�`���hT�TT�i�}T��Mlkf%��e�Z�D�U��}�]oH����Jj<��(������x|�-�f%`3M�5�Z�����J���A+�uzl�8	�'a�o�;�m�p�U4X��7���.�)]H�6��6�:Zy��Ql8	�P���6�F��>'4w���
���!�Xg���O�l�	��sTd��
;�Y��2�n[��BY�G������G�m��Fr�c�mN�btA6�� ���"R�$TO��8��5t�4�L�MD�.)7��@[9a����r*�%����y����s�
��}lS�6����BY����������M�t�M�H�0�fl[d�th����8E�t���m�s"�w�hk	VG#e��7ja����6�������l���y��%/V�l+>kb�+�-D�����9h��N�l!���A6��1�6	�N��u����G��G�I�P{�P��h��1D����RF�{�w�I��6j�G���zz� ��r��C6xB<�OHK�����aS<�����9h��N�l!������;�l���VU9�U�$�
)�]d�r��~�H�D�J�:�x�����G�`�!Ua�U
)wS��.JM� ��r�gC6q��#�����xk��~�y���P6�P�����#��|��1�6
A��r}�k��
��+�:�5�2�������i�AN[�9�6~F���B[�����j�H�6��s�~������?�k�	RMy���PaQ%Ua�������[�I�kN���a���8'��+i
��O�������F���(�+����i;"!�����YGaBJw�v��JS`����#0�]G����q5#!�r��!6�*�w��BY��������w)R�����I8	$ ��I��K��G������:S9a_�0<X����9�~:�(Ta��	��?��s�[]9�����lL�����H�cIvX����'�GZ6�P���-u|���#��k�;�li>����>s��s��U�>G^_��[,�	��6�B������lp��IpG�(����\�����3����`?�G�Vl	Yx�k�����lZ���#[�FHk6�#}mP�
V���~�Y�H\�m�]Og)&�v=iY��4��m��HGMIps��lC�U�*��R= �S@`�F�075����<��sP�<p�+3(uT���9h��N�l!������N� �4\���1D4K�����?��SE� ��J�K�G�1��!#h�F�,Q�]�:N�/�yV���T�M>Z�+�4���6|����b#�_�Kc���T�Y9�ys�
e��:6BZ��k�c#�����Y�5�����������K:m�5��X���_oB��9 P�l�f�Urw4�tb4+��Lk��M]!��l+.B�V���k�)�X��/��V(����q��������N� �4\�M��f$������N�r�YA�`�<��il���l�7��!#�`BN�����h����}rBvy�rP}��Z;�����m�c����mb]���+������:=�ul�4���w:�&a#x����m�]4����l������Qn�jCT�������kX� ��E��U�>�����l�����O�lO`#�~:���E�vX���A��]6�P����c#�5�=�6��1�6�
�2!�����g�F	�!'��M�ZEiP<������Alf����}z@d�"�|�N�4uA�������Z�k��N�w��h��BY�G�����l�w�����A�I���+���xT<Z��a����j�5h
�E�JF�o�����p�E�IK�Z��}�-�#�rV���hj��g�l��!8W��[;�� /��9res�
e��:&B<A�t���D��u�\u:����������t��&�TJ�j��������n�%�\�Q�`����v;����n��M��������!h3|�
;l�r�0�����:=�u<���!����w:�f�!�z���+��b����R/���+A�L����A+g��	]I����%�#G�����,I����������xA����.5R��!�sH����*����6��p����QE�����&���L&����T��>�����aU{0S3��H����7ws�<����a������8����A�K���:*����		��\"��d^/o�L�/���(���z�[�r�b���C��� �����%l����@�k9����CV��I���pn�y1�'��TR�
���~���P������
p�������c\��$)obM�"z|�4CY��7���3!x+x�[o�P<HT��tlP��	�a_�R.8Dy�$��2c
P��a���5'$������}NO�G%@M&.e���V��u�
��	��c," ��o�!��2�[A����:?�ao�F%`!'�zT��� x����s����V�o��pw�UR�-h������V������������C%Pp3�f#B�3�j#y�|�j�Us���!=���X���i��I��������AC�u~��=���%	�I83�J6��Bp��t����h�z���W�����u����)~cx
�y�!*(QT�
\��9��#�&GV������f8f�S��'�Kk4L��,Z-�6��FM`����(��G=���
���57X+�����Y�
6�s��(�����QU-M��Qu�c��G�����e$8��s�����	1w�D�ej�T���/�}6�#NO�~|��JU%��[�n���es�e�����c��=x�[o�pT��rgo�P".P|'J%�w�66)� � hT.�����C[Y�a�e-�#�B0����i�zpT����p�=��[MQ�����R�5jAs�4DY����C��I
�\�"��0A)R��X�
��/�*�)i�(��ta�<���l�9!�)1�S��&�M�%TX���M�d����	����/�dmb�g�'}�4CY���������H#��7H*D6�9Jl��)�(�[y�3X�o3����C�0�5fd��D��n����1�%>eA�]�M���s
N\�tm�� Q���!�g`[G`p���6Co�Ik<9I��cF(����zdk	ZC�����B�/`�_�w:��������?~���[��%������O_���__����r��_&	�����������/���PK��f,2PK{2K18sept/pg_part_patch/4_2.outUX���Y���Y��}]�%����~E���`I �XM��3��u��c����*�I��V{��_��7��Lf��bEL��L%Uu����������������/������OO_��j�����������/�����ol��jEA)��6�O?������{���?c�WO_~����|����w??Cl*��T��~��~��_}�'��|z��?���������������h����_�����a�W��~�����O>~����?im��?������}�[�v���>��~��,������l���V�g5������"�(��p�����]���x��������O���)??��O�������O������?*����������+�W�����	L�VE���6�F��d:8����
: �����)x�����O��o|���������w�7�����B���`�?6�*,�+��Yb�/T����O?���y�?������>���S�'����
o/�p����;��)�!*O�����{�������??�������_~��e��O�?~��&F��������Z��WO���?=�}��}��=i�u�?�~���O����
~���S����}x�����s��O���Wy��������OV��_�������}��?-X��}����1��?l�`���>Y6'��W6���'�$�#�������N��w���}z�9��������x�w����f��m��|��K���P8��c�i����h��������������>�������������������������2X�����9�l|&J{F6��w�/0�	B���ml�
�.�1�f+�H�Q��������b�����A�8w�L����~<����~���a��C���9���.�l�A6�E6{���gd�-����A6;	�Y�����ny{�*��D��G�/����CP��4'��$���]Dc�R��1�>��wN�Zp�����s���h������V���_��^�2h��:Q��Vn9����*}i`LNt��$(����*������B�(�!���kl�z��b��Cs�i�
����B�K����T��?D�(\Q�_��A���(����w[��ZL(�_��^�G1���;�b�QZ��bn��8����rZV����ax<�A��
B1�Ms$(F�F����y�b!�[�+������ f�x�{���5w��s�'����?���6 �g@�2�������9@�i>O1��Hy�%���.�V��t;`$7(���4������>C���#��W��Q��C���\�Q���a���������uy��h ����A1��!�|���%��0�F.�P�)������Q��8���b>���1G(�i1�WuH�<3�yb�
��B�:`���5=w�y]�|b��a1�����1?����� ������bZ~t���|�<v��"���4m[���� ���3��@������b��XhQK�����Yi��_��^�G���X8�b:��nQ�Ot��YP��c�J�N���^���8��'!�Go���������E(��}���t�o����Z��|�8����^Y��P�a���7�b�\r�f���]ek)X��I/�uyX�
��s]e��>�1�'A6����&eq��J��G6�.O�w&*\�9t��t�]�g�
����
y�����C;C��q��3������k����,?��jJ����2��rW��������������c]�-��
2����P� ?��@6#�[Nb��s��y�>�a~�n���R�m���>#�I����_.e�A1�7r-���:>��a_
�����L��>�����a���\6'������a�-�!d�U7�P� �,L��s�	Z�n�F��	���w����v�����V�Y�����6��]]d��S=�����cqjd��Uc��5�g���8N>.'_�f>�y]�&��2=�l�k69�1�6@�[��{6pQ��W�;��p�S��DX���=��}���>�z�B0*�������8{M���3H�*�]��7hE3-+@�V`v���A�9f����y^�����P�c���
��r����A�Y���]���\��������M������"%f�1}i�U����lQn�Vd��&j��N��~��&F6��k�f���B��G6�I� ���9���.�l
]�>���FM���&f�C�l�0�&�b��L��_����'G�����;,�l&����>�1��n����6JR��N�P>SVO���z�����LPA��a/�@>g����}���<�5��Xo1�{���E6>�1�6�����?��K�D��*A� �Q�<1@��!����h�K��	���Mn����Gc�o���uF67��nZJ�4y������l�9���.�l
��>��Q��"�d��Z`0*K	���=�f�t����6�!*����F}e_��&[X7@1*/��:��\]������
��|l��7{n��fr�lNz!��[�5���3��lng��v6>�1�6
� �t�`��{�[;�> ��W��-
����'/[�9���&�T��{��l1�A �4��
�. �d��7Bw~�"��RD������<�5���Xo�lK2j�A��r��Zd�C�l�P@�"D���,��������y^��7
�\�����/"[��%�op���"���Q�&�v*�4���;> �l� m��;n��|���R?�L/�uy`kf0Au���@����8�����o�\Ltf��O��`q����@�D�68On%�r�^��'0QE��\]���l�c6��v6O�`�{�lK����:����`w(�l.7}l+�Cmv
���l* c�Z��>���d�����m��������
B-��]�n�&�X���X.��������m����� X�~����
F++o�es�y]����F�@m����m/��P�@�,�F�1Ub�x��v;u��S#���R_�tz��;u"�
�-�[�Q�X�E����������*3^��n�H���=h�D ���^��������8��~���:�&!!�Qf��2��"Z$��h��cD�����&tx0������20$����j�V�OZ�T���������cfhsa|G�mX�����6��
�E�%oNz!��C[�B��X.s�����:�&a!@�f�����o]�<*}�:�J�.�7���d��2�(yT'����������3E�����������Dm�-#�r'{���{��T�����<�54{���2u�m��|�c�m�tK���hk�[��nK[0^E#]�F����'wm�z���@��>�����@�����/CU����$�i_��Dm��`i{���
>��P�lNz!��C[�C�;<z j��?�f���h����C@�i��%�C���w�FN��p��;G+h$ed��@��u-����zo���&J�W9*���)RSw��t����`EH!��a��E�/��^��������p�.+�����j��4*y���P�fyzo�e���g�V���J�h���;S�����������Fj�pk���N��G�����"h��?l�@V�4,��^�����P����/�;H����,T�s��?enwm����gp�����K�A�i����>��k��
H��U[�4��*
�����3��r��
]�l*�r�V�a�le�l�es�y]�*���"�FYWu�MBEp��1�M7�����NlO�E���������Q,+�L'�g�#��v�Q���D��W:�AO=��x2�m��#T&�T~�BqH��-mNz!��C[CF�;d�G�����z��;�XJ,&
��
Ea8��D�>����L�(����!���>�e����+}V�J���R#UV�7���fo�p����C��U����QT(�������.l�u�u���Z?`.�lK���^``y���
>*igb �&>T�c�6��l]������:=���w�}�� ��.��.�}5h�8>�
At������9�hK��^�����p`g�#4�.�u���E�{���.��������p
�S���X!��i�P�C�T��M[W��D��51�E�`�(���
��R������"Ho@�����A� {R����^�����P`g�#��t�u��BEHMm^�������e=�k���S
A[eb���c?����&��UP4X���&�(�p������?3�q�:�h��m�����q�1RaRj���B^�G����5~������A�Y?t0�
��p\��b:������B��C_���s�FN�����%10�{������L�s��3��G�M����P�a�~�y.��^������}�9	��t>���}�Y��(2�������DM
�D�k�J5���$�^D�pg�f]���I����u��4��D�
�|����xx��(�}`�f��7�7�}�Z6'���������s�}q];���}�6���Q�����{�D+��5�e�{4�F��B��Ocn�p��9���V�FN�Z�(�i_w�N����������@��a�������9���.�lM�l�>�g���C�l��}�ib1c
��.��`�U`�a�  vK5whL����}���UX:M�6*Z���T����F]�j�0�6�6m�������%*!��es�y]�\����IPvc69�!���i�@�� �%y���Q�mhL~���c%�����g�����|�n ���]B2PX��^u'��vj�p4�+n��Akh��M6�C��/��^�����t}�m���!�?@�<�n"���A�Y�>,��\8�����Q����l"��D��d�]��%@�le�K��������
t��f��|R�����05�������PS&[�~�v} �;��:oNz!��#[����]��![�����9�1�6I���^*����7������p�'���8��J�
,���lP���Z��|(]�a_R4^b6~�V�����������Y����61Z����7'��������t��'m���[i]3
�&��t��2���YX�u���x>4*h�j�^y]TVe6�ds�}��;ui
�
���Q6��DeN�?��)=��$��h�s�=[��p���=?�B<�6'��������t;���r�G�� �I7��$?�9�L�'>�n?[���%O��/m�� `e�~6���u�����Q1PG��d���x=�`�v9��!�a<��&���^�aKr�*� ��9���.�l����Q�|�HeTA�E}�(Q��2s���l������Qy�����H�B6���A���D�|�q}]]1�P��F`L�����3���=�Z�I-�Uv�7 *��9���.�l
�m9�g��l�A6>�1�6
A�����#i�]�>Lw<�L,����I�J�4���� �q�����WEAwk�b �4�	�
+�h��i<�5�$c�����(w6I'��I/�uydk8���d��P� �,-�Z�=�
�~6]B�g��r�	�t���i3/"[� h��He�IU���}dc�M��,�U����]�n�`��hOJ}&��a����BN���B^�G���������.����A�Y8�Y>p&��:�=@���1��{6�q~��(�����l��>�j���_���J[��]%�[�ck��
h�
�{����� X���V�a����~��9���.�l
��p`����k7��r�c�m���M]:r���:,�v���T��P�LB�� �11���&BdK��X[�wb6a`K���<W�����Q��pd���d�x���D�E�-�����:�a�A��#�h�S�u��,A���hh�@������9R��M���6���l���4�EU��\��@L��i�oUA��y�.���Q_��K��y'���|ys�y]�n975��~�n6��x�8A����|�"��'�� �`.�����p������/
�s�����Z�.A�I�����g#��S�@�7�����������S�lNz!��#[�A�-��N��bd�*F|�c�m�t�������]�W�.!!<����������;�� i�*P��������}�ls�x!7��
[�0rK�`��-��f��es�y]�n91���A#^p��H�g3�����v�.D)8*II�]��F���1��*�cSbWa�c�`7A������J�s�����l�r������
�(3��I/�uydk8��Ax�������:�&� @��B j�H(��%f;���U@��u����o��ae�S1u���^������A���Z�/b(�l��������lu�G6l��$����
i�T������<�5�� �k���P� �<���'����j�c_S��)��I���+�f�����`�-���� ��I�P�����Oa��n8W�V@�<�l���B^�G����[��*|�c�m��#�w�
V+_�
{�4eI��+B��]���l��>H�$f+��@SW���y�`��5���l��sp3���v��-Q/m������<�5����H��7:��m@�g���_�����tk��"Mb�S�<	�� `�A��)���$��\��W���"��+J&��7�05����6��tL�~��F��"j�����<�5�� <�uk�|�c�m�<T�hNe.�-����{6�tD�	/R~���D�f��;���������f��o�� QE�gC~u����������m3�[���a�4�td�����<�5�rbWu��u���A�y�����/�n�h�]v�h���z�����������yF|��\��%��4D�����A���}���W�� X�*�[���Htb��;oNz!��#5�r^�lr�C����  *�_�12j0�	/��-��dX���.F�&>��|Lm�*2x&��`�������Z)�r6�J�����Qv�xdk9�p�fw���g#e���lNz!��#[�A�-�!��n����i8���
����e�P��
���K�Kau� �tmA6{g_���o��D��{����)f}��������������u*\��=�v/����I/�uydk8�� ����� �,��Mh�`A*�6��B7�[aG�8N�j�Kc�U�}�DD2 p^V-�h\7A�,�|�V�����a_
����A-�i��E���qw:��A���B^�G���@;�� ��d���Q�u)q���*��9��+O�@1JB�7���U��<���Rni��6vy�b���]2��V��1g����� ��}�=?����@�lNz!��#[�A�-����^?��d�����
��2a���|���B#W���&��F��!� [=�gJspa�C����UJFQ�~c��������`�m� 8E���-�M�O���9���.�l
��XJq��z�lr�c�m��J����l��v�>�x<����������(Qe�bS?[��p�x@�nAt�He�J?��nfdC?^S���g�|e�WV�a�Q�F��9���.�l
�v8����P� �,#Z����$���');�d���l���-�Q�N��8BH�Q��Q��.A�����kCA6�Q�����K-�h������
N:�w�����<�5������GOyRu�M�A�����p�en�����*�fP����W�T
wb�%i�(<!�EuQI��pF���������
�����^������YV~���B^�����Pk�/`��<LGu�6�	�h�E�,$��'����J������UEPHnPb�J���zTC�w��N]G����8*J�*Q�|�Ruxp��r�W�6o��:�������6nV�h',��^�������ch����:6n�DP5}�GaPC�,$������\�JS�!�?��C�>��)�*k/	���3�eU��4��t�[C��]�_������1�]N�j����pd��I���a/��lNz!��#�oX��.��~�N�lr�C���BBl�p�Q�{�;z�b7a{x���T����c��/"��3�&��G�����iAd��A�f���kf|
���B�����q��Jx*j���B^�G���P�d�3�l��dX��UX2��t*9r�m��5����$�A����3P~������Ue������c�DU����#YN�r���|�B�:�C������0���P=_>�y]�B}��#[?�3�l�� �$a�dA�vW~;g�G�&���4�d��A���
*)��;�����tu��g��0QHc��T�J�����q<%�C[A��v��a�P�t/��^�������c]�����
�LR��p\�"Ug�2Du�jE��t�H�8V��c�U��(�v�'��@oa�����r'9yZ?�~��`���9�����
[`3Rr`K��^�����p�c]�
_��Q#��,~���"6�0���}?E�ec�q�]���1�Q�w����]*��.#^����i��Bp
���x�op�����`���I�Z���I/�uydk8��>�l�[�8��g� ����8�EK�3��qz(O�Q_��kUF�9:*�� �}T*r`��x1��4�\>�X��>8�y9��!���Z~3�D1o�[d�	�lNz!��#[�A��uD2�g:��� ��$���<�U���.oTr�(�GG�k�"�Q=���%ue���&�~���P��
���A�������xd�pDe`��lT��5[���B^�G���P������d�����%f���u$�eW%��(
����t�9�A��pg ����o��x����&&�Id�z>p���� [hc��W�;?������Z6'������� ��A��t%I_���Q1�,�"���|��. ���E���<!��
��cmJ/"[��H2u�E��5�D��7*1�� ����8u7�7�g��{6S����w����l����y^�����0�`��d���2�)I���u"����$��sIK7��1}�A��� "K��Ft�>�@
��$���g{pl�r�WC��h��m������������)������:�����2�L� [�����fRm�����B`��lZ�������r�f![�@FQ��RuH�?R��M�rQN��r����l����A��0;~�"?�|�gK��^�����0������>�1�6�x~��
��NVEE���lA�M��`��1�G��^D�j�3!W4U��:���n�g��+_h�<�E�������|<�m� @5���^����Q�����<�5�0���g:��a ���N"F�hV���dCr�g�T�F������2�+8�#0��� ��.�Go��=�I�����-���������V�a/M|�es�y]�
B�R^u��g:�f� Fgd��V���Q6j��/� �����BS
�.�K6J��#dc3�h�LU1[��+���A-A�!�?B6~�E��(�r�gz!��#[�A[B|M��g:��� H
����ed�+��~m�qZ(�gh�(e���
A����eT`��\�
F� f#~A,��*�$	��N��g��G�
�@�=?��lhC�����|���<�5��� <�l}
#>�1�6	���LM��g���6*�\����NxY���h��@\*o�R�C��(�,.��`d����>�k]N�r�f� m� �'T�?�u���M��^�����p�����lT�t�M�A�S� K$�%��F1U"&����=fP�������+������JF2I���\���Ab�r�WC6D7��� �[��]
#�L��Nz!��#[�A�8��8(f��� p��6���P�����k��B��lV����l�l��@�	�9�Y[�������+CR%1��&��')�{���� ��1��a���n%��9���.�l
!l9�b��A���,z�F�����n���2�#��@Y_����h=!�l��j�����S�&���y*�6��/;6C$�;n��l�L�h���B^������W5}����I(�r&m�x>�"��A�GT6�!2m������/[�w:H����*b�B_�Ct�A�Ida�F]��![l� X_�;~�"�t�@�5��I/�uud�
!n)�
��L� [��� ��r��[ij�5d#8��N_�M��T��r�6�4�����
.:����hD��@��8I`m5f��r�i�qCA�1���W5I%/oNz!��#[CA�[
���nP�G���H����	��
�
�+�$�J�**��a�]��30_�A�|�b���Ai���HM.((/*X35m������R��"�^�aWR��ER���^�����P��������t�MCAp&K} I���.�H�C�!��,cN�eD|�cucEAp�Q-��J�0B�h��.xH�����:f�7�7��H�T~��gK�������<�5��� �����t��BA0B���Q0([Qu�IC$��C!d��`��EdswZ��l�\��<�T����v����+����m�@�X���u6���B���B^����7�5��S3�2�a ��L�f�N�-�����U�D��14�l�5�����/[�@�^�t��V���"L���5������
��z��3o�l�"
ix�[d����J���B^�G����2�L� �$geZ������v�a0�lV��5���Z�(�F��@�z>��z����h���$Y0)�����v������
���`�)F�6n�Q9/��9���.�l
!n��f�3�l�0����)�Q�0�b6NF))�r��P>JF}e (Qe����j><��#�3���p�t�����l-A�����f����)is�y]�B�2^���g:�&a ��]	��_4ha-s���(y�O����d��o���@�8���<6U�������OR�����_�Ss������1�������H��9���.�l
!���p���c�Uq�&�J�F�
��L�K������4��q^x�a�"��;!��Y*�(�
��@`m��8�z��1��1�0�Z
�|��H������PB��^�����P����H������A�Y(�E&+�2r�WR��OuAi���hRi����UF�;M�h�������g7�2�@IyC&�������aT�b���J����!�,dB<o���������`?���������M�nV���>���R�w����E-E��)�2ml��;��4>����{TF ����\��>k�NI1���D�i�:��v�^�T�Xj��tC^��7������P����79�A�6��:�F6hg�	[r{ f�9���dB�zQ�h����b!)�����_�0�j	Q�"8[Z������,�}=t�V�bA7��XS��{���p�����>���N��wnr���m>����BbZy�E5����@�����]E�Ho4T��y�K��
�Q�i���^��KN��QYN�j�Fo0����*�a�����^���=�n�I7�u}t��OE�S�n�pd�r��H��[))�0��C����n(��b�n�\�9vs^�$W}��4�J������"u��#i������*gT���8bO�2�7mN�!����k���h�.)�g���o13�,�n��
�x<��u*&5J��{��	q3��{j�����@���<��6��PP#�}T�7���F�
���/*jB�;~����f	q��tC^�G7l�mKNx���� t����E�L�d,��q����)�l��A����q�:�����(��A)S\\��J8�o��#V�:���y9���R|tk$Dy��?���V��9�����n���9�B_��=r����,�5����pj��RZ:�E*���)�G]��;�BP�.
o �IG�n.�O|p�r�WC7g�"v�Ip������\�����tC^�G7����1	r���m���%I���sY��>(�:��$3�\TA5��������B�F�pxY5��Ho�dg5��r���V���[KSH7�����MS����T�?�
y]�B�n[������ A6lt3I��j(�����]�8�s(�&�7J�%D&C�����w�� ������h7pc�@J�@��w�	/�6��`kX
E[�Nh}���F����9�u}P�-�������g�J�����������l��~~F:���������w���O6�7���s"	(��Ks{�~���PKA9�ow1&PK{2K18sept/pg_part_patch/4_3.outUX���Y���Y��}]�%����~E���`�dD�CX
`��~z��1�a��T����V{��_�d�$/3/�V^^`�������v�"2OF0��xy��������/���_�n����������z���?|yy��w�~����H+��?8m����������y�����7���o����ox�����o0���
�*U�/�������/�b���_^���O�������?~�����&�?}���������{?:���_�����~������?�Ak�>�!����>}���o��e�����������,?������W�6{%����d�.xP��?
�������&��v������O�������_^?�������l������~�h���M�Q����O������[{������:��~k��J.����E����s������s���^�����o�_~�������������Z�/-w^0����J�a;�.[��*����/?~���y����o?�|�X����S�J��g�h/���� ��"����s>�}����_����w~���>�=��_,w������7/gbt���W����_�������/W��.�F�m/Z����/����?����d�������>�����%����O���g�=����E���'+�������u5���������kW�t� �9��a�p�?���A/����o�b�������?&+������/�?������%3����peV���5+$����kv������Q�0��X����2����o�O~`lL�w�~���b�������?�?������O����/c����������6Q�A
����M�����F���,�k1�`�����0l�w��xy���x�x�V6��V���3.X�6����6�f��
v�
/��+{a�8���Nl�6�6{	�:�f3���o2�`���*����������5�2�pDtyu������.�A�5H#�VDG7����Nc�q
*�����\h�����F�����/��6�����3���^��1hp�����A�b����(���b�1 ��GfbU��%W[���; ��������;���Q����<BC�@p}&�v��[[@,@\��0uH�i<�abr��[����Y�z��y���1bb�1 �s�D�(�A�INT8Y{���I����bTo
���#U�D�5; f"o-�2f}���3����A��A�.G���"1\"��C�^��uz��h$��E���b���b���$��N��5	F�tH����G8�N�zkr�0��%0���<��?-��^#�05��'���6���; ���c���� �s@��b&n��0��, &� �cu9�u!���PEL ���������7A���1E1K��9c: ����S���x?3�y7�|�E�u��!@1�z��y��|b~ ��E�����t�t`���t2�n$&�����N��N>��PoM���H�y��h��ug����� �����"e���!�=f��Nb��0���c@,�b�oY+m�H��^�o�g�6�w�(�5�����M$�����gH��
{t=i���m0�L4��R��a�����������q���[�����^����T����L>&3����:��$�f	��oR�\�+��m`���~2�k�
l����]������Z�g����ie���d��d�������gf��Y��;n���y���Y�8����ll�+`�������l����`��a`3����E���w���`�����0����.�e�w�D�v
zF9D#R����}����l�Fv|�fZ��Q����m���#-[�8���Nl
�l�u����n9L���I��T�6���3���R�8(b��������V��y��
�������+�	��>3���������7����f���h�8���Nl
�l�=ll�1�6@G�|:c��P�h=�������A���36�x�-�&����L�>��X�B/b�@�1*B�1�����4��4a�I����@�����:=�5����#�Mb�1�6+��Wb.~(���~y�������c7W"6��f���S.���J��lQ����_c�H�u�.�>��8�j6�������]����:=�54����IEo��c�m�j��)��<t����m`�RU��������A]�r�f�Sk�,t����S���K�mE��Ze��x�
C-�����q�^*������A/�uz`k��Y/���66�`��=�7�r �A�K�����A���4�.������_����J"�N�����!��K(���t6OL7-�@��V��n�����,��������f}O*zK>H�:�fa�l�%`#������N����
�y�c����UE}���y$�M����u"6�M���V[���x��	gl�6������Q��N�=��A/�uz`k(�YG���Q��4,��e��Jxz`���3@N-D�2rX�a�Z<w���	l� r�S�G���{����(�F�Isi��.���M�����X�a/��s*jQ�B^���vP��r����6*����$��D��
��k��N��QlH��2�[�";�b�?N+!ir���hB��F'bs�!i������f6�'��
��h�
{�������:=�5�3�y Fl�0�A'%[�%�����s�&R�I*=����P�	6���y%� �CU�Pm���*O)b��)��P���m|�fu����_Oj7�5����A�/z!���m�v�y�H�&Flv��u�:M�`�X�^(�7�-�[8�A���m���5���r�xP��y���|����+����e�����=A�����!~�s���_1��A/�uz`k�vg��=U����Fl�0d�����1r��rE��4�J��#�X��	�(`�W���+*4�PDn}�VE#	��:V�r��M�2��n�v��Uk�v�^U���UQ����:=�5��3{��T�&��Q��$���6�
.N��IST7�x����:#��mu�
��������@4���������4+WqE���������yc�+�������9z!��[�<���b�1�6	���x�����B������}R�B�,`cq���j���0���#uH�F9���@�]�)���"6l����^�a�x����\�B^���y`2:���$���&��hDF�p��:}l�;/I3�1x�3�;�	lt�?��=������t'b#N_�$��[C�TYl}:`�O���y��u
��v���<����:=�5���<pwDl�t�����y�< \L�a�28`�A�Top�#2�U�`�l�uW���A����*�-t��H��+
U�,S���'����y`E5!��a�-}l��-z��:=�5��e\��cUQ?���N�<��@�xp1�����!I����ln�����Y�
�QU
��C�x�E�-r�\\��&�����2�����q�^�i!���A/�uz`k�v�<x,u�H�v�ARLB�!�������H�l>1�_��=�A�����t�S;T�����1�y��G�l�c��L�����apz��v�-\�-��^�����0���`���I�(gQ"	�H���^d�z���K:��MQ�E���j&	M
��<Q�m'��_(UPb�����B~<����i
Z���n�m����z!��[�<�;���H�l�1�6��X�(K"
��Hq��a��/�������!��j6�����)'5�����u����e����8�x��P������.p�t����vh��e<��6���a�!�s�&��%$
���K
��������
*�o.��<`|���Bg�;oR�����w�\l}6`�8>�
�@������c.���1/�uz`k��3��!Ju���<�#%F�[�2bGA�*����Gec5�e�4d�f8�ReNE�\�D�)XQzUIm2�R<;5W��>6h��P�p+7�5��4�z!��[�<������T��:�fa�>6/}l�8����z��*�`�/m�2�h����Y�����fg(X��������!�RIe��l������>���\�a/bC�0].z!��[���=n����Tt�vL�a5W�z��8����	EA��(�jy,=Gl���L������s�b%49�48c�x`��{@���v��<.}lz!��[����&I��,�@y�:#���,�36�M��T�����mE�Q�!����A���jRo�^�����g�J��M��'TE�m�S7�Vn�KEut9��H��y!��[����&��c�m�vgrU���x�}���e]�������j��x�?��&�2�t�3p��6���*!�Ruw��-�>�q�<�b�ii|�q��+���I������:=�5��m�x���c�m�v 
$N(������S��,���h�2���l����<���D�h:UQNZcR�
B�n�w�=]l}6`_�M��(�7���%|�8����l��{�A��[��u��4����H�t���&t�=8{�t�8)�%bC����j2�4P����@hm�y���e��e���d��'3�x��v���Wn�����x�y���v��{\N!�?t�f�H��,��QC@�S����]J�Ogl��Q��G	M����	x������d�x�y�r9���#��T����Fh2�2��v���B�����:=�5��m�0�������P�p�v�q:M�]�e|�.�
tn�������Y�����J��|(��'4�J�wU�v���a����l�Fl�&\����-����/z!��[#4��&oFlf�M"4��1�v��*�zO�I)$u�J�*b3c����D$.7�:U�ag���Z�48��U%J=��$<A�=c�:���6�}�>�tq�y���I��t�P�� �I�Dh�_�2SX"6�@�|����	�*T}lrD<�����}l�#�u���.t��~*�a��ZC����b���0�h��BQh�������T���^�����M����C\Q6�`�Eh�(��C)d�36�i������I��fXl���tw&<�aE�5�
�6)��m�R�zJ��������J[tk7�*���*�����B^���y�[��J�;T<`���i�$�"t���{���=�o������R%�C��b���MsE9���t�*�I�<�I�j
�Q��#7��A�-*���
��h�tq�y���������m��vz�L��^x
�Gv�R�[T��!@��,�&�����L`JU�9�;�&����G���M��&6������4A�n��
{�O6<��^�����0p�<x�R�Fl�0����*�i������s���xm�cZ������&�����T���5������Flh�\QiZ�=���x�O��-�@8'��V�a/bAV�\�B^���y�;��GH�b�1�6�@���v9����R��36�O�*CJ�����0�����t��;�������48��{P����NE�����B�eH��v���20�/z!��5��a<4���:�h�����h4n==���TT�J2��G#��+������������v��.�9����/�������l-�@�V��a�A�.����B^���y@[����sl`2
���,����{�EW������oT�I�Z��
�
�'{��4~�we����t�F�4���FE����
���"�04*
;n�#�,$x�x�y���m��M�g���Y�r�r**�m�:�*t� �F��3[
L��Ap��<� �v�����R�����.y��{����gl�2����
Tn���d@�Z.z!��[�<�-��Au�A�\h��uV�c3�r�s������>�����X��{V�3Jx���+.�S�D����7y�s����zl�2������
{�f��B)���y����0i����I� �e�{��[�~��0N-8|����k����36���*�]��������<���D���;��[�
�8�l���I*���]ufY.z!��[�<���{$e���y�Vy��������S�����36M*����A�*�f��y`��Ex�x����(_I��8�$xN���yb]<�~�i)���wz!��[�<�-����u�M�<�bR6�f�[��n)�v ���Ni�����7��_����Jgh�)�r�ip_1�M}��n<��63l�t+7�Q��Q�TQ���>/�uz`k��3����u�M�<�glZ[��.%D�S����36����A3�b�M�J�Ua}<H��@sl��|��S��-��h4�O�v�^UT��*�}<���Nl
��v��Dl���0�faH[��a��VF�^��u���{�/gl�j�
�!����&7��K���_Y��3R���4V�f�v��>���o�1�q���$
�/z!��[�<�-��Au3��c��������L0��8}�J������$�'k����9]�O���I���n�~�&�*_��Fm�>c����2����V�a��-^���4�;������5��e<lb�!���a)!��1���0�n��	�R��-����������M���!�fE
7F�����6���4���+��l-�@N���q���=)������:=�5��e<$4)Fl�0�K�`@h���vb�x�#[�c�v���l�j�*�4�J�f0tt�.�����y`�x=6�a�q�������5V�B^���y�������� ����y��56�
�)U:�(0&`C��Q�!��T9����i`���Z�[��N*
�iQ�M�P���Q.�>�����Z��L0�q�.��^�z!��[�<p;�������c�m�D�,D��F�	�����3�@�y��1��U#��ot����L���*S�����H9�c�Z��=u����x`k� �]a�
�36��<���A/�uz`k�n�<x��C�:�&a��!1@��2~/�A��4�E��Li��~�l��g��< 9�.$x(�{�R�TEc(r�z�>6��a.n;��*�[�a�A�]���x�y�����2`�~{���c�m���=%��18�qb�m��Ih���X���������AWR��i�����@��{�A�k�u�c�g6��t�k��8Y)&���;	��2	��A/�uz`k�n�y���=6�`��y  `	>%����{�gz)b������0H��j}�T4��Zs �S�y@J8�I���H�4'�lh�7���y`�����al���t��A/�uz`k�ng��=�7�l�1�6	���U""�H9��M���N�t�c��a%�f�E��)I�J=������i.�G���Z1g�C6GO�E[�AZ�g���i.�.�\�=���N�l
��6���`���\vsQ��MoB67��m��dy/�'���L�=�
��&��d�KU�nP� ^m�:YtRfCg�(p(���/�b��;g�/�>�y���h�=0RH�;n���B������:=�5����i��*����6��J_�eQ��6�@jVQ��U�d�
lH��I�T#�M����-`��j��5t��8�c�|�%��Qz�J2�>�]�}6`#3^C�o�8R��zq�~�@_����:;���{P�u������M����z ��`�9f+����v.*��22��������OD�M`3W��-����Z���7��C�6E�E��]>0�������a/b�����/z!��[�=��:���c�m�����~
�p���(��N'��������?�|����wC��E���s]��R�������b��[_>�-�@�����=�h�i���A/�uz`k��Y�l![��t�MB=!�����fA�uW�'������D���*��;���6�6��88���rR�Qw����y�I[F��{d2���{h���mAWn�k���s����^�����pj�.�f�8c�	ll�1�6	�-�t�Oa]�i��N�p�x��7jL���.lQ�Y5��j�R`�q��6�x+O��MJ�J����*t~�Y�`�l��N?�Aw��=<%��|q�y���Am����3�Q���,�-��)bc4�k7���;�(����=T(-�d������
b���X�
��C�'���.�sR*�qn<�'��m����W�a������rq�y���Am��&6l�P�m*g��
��d�T��-��.E�������VSd�q�z��s�����]>�����G��9�y1���
�x}��z�D"o�
{�l&7���^�����pj��#�	ll�1�6
� �%
V&w�i��������r����Q��j�K*�e���&�N�.�&2�7�E���'�z���E��{ ~{n�KE�%�U*�>/�uz`k��YG�{�M��$�4�L
��z�2&tm�p�C����E�7��!�AU�p�AHUQaH��q���c�J�"��J%���#6��l����V���r��x����T4_�B^���z���j��Q�,���"��u^z#��AO�)AqDd�1��
�}Jo[�6(qd�c��JO��dQ�l�c��+�	I@S��9��3��9c3�d��
{glQ�|�V�G��y�����<`���I��b �r�<��;!tRQT>��s��tIE��1
�A_m�@t��{�`_���D��U�A��9 p1�����A��
}��
{�2!H-�����-4��0�y 6la���IUQ@�e��&�
l^� 6-���363�*�A0��������G��� �9����>csO�v���Wa�q�^�@*�j�8���Nl
� l��{�M��,��9M���p�p���36��dfQ�����z������2��Jn_G�6+�J*�8��s	�N����g6r����f�Tsk7����$]�/z!��[�<��7#6��`��y�--i�1�alFw����lQ ��367F�(T���9_��y���y �v���q�V
�k ^�}:`O��y ����\�a�����gl�z!��[�<[�����8�faDg`��V���KE���I����e��8�*�*�K*�*`�]`���<����HE��x�n����2�������tq�y���A�2�#�t�M�<��S'u����G���^*j2�J��kg!���[�<�20ElX	�9��6��s`+R��$xv�x`�0�����T�D9f�8���Nl
� l����t�M�<� �;S�@��u}c0��(	�,x�%)����hp�i)p��*�
2���I)������^w'�u1����>!m�Q^�7�ElA���l�c^�����0��y�H**6l�0�H���dI�`���m`�{H\������������B��r�v���������e��"����.�>��x`��zP�m���F|�H��/z!��[�<c�b�1�6
�@��RU�l�U�=]TNN��c��V�`El�j�B8�
��i��h:UQAk����-�?��d�'Dl-���X�:k7����L�J������a�-��!�419��<���T4z
er'aoN��9h���ip��������SQ!�2�����'OadH��Dl"�41���h<�mfp��Vj7�6��e�����:=�5���e<��M��$������5��Vx�;�N�v��J*��,��lQ_m!�"[������
D�HC��B��A����-�3�0?����=B����:�|��:;���y����"6��`��0���D����u�-IM�����"z�����mLU4��
�PJ�������7IE=�nef���=����
��������M��rq�y���A�2�c�A�q���gl��@���vWA7&ip0Ai_�m�0�h�6���@�
/������(\R���e�R4SsE1��T��y��;n���"�\����N/�uz`k�q�<x(bc���i�h��9�a.���A��4*���^��H����y�Z�<1��J�'5FD-;�#,�(s�J��=N�����
���YDV*7�El.P������:=�5���e<���6l�0d����A��j+�A�*j���������w���6������f�)U�����EDnBE���93uU��<�m��j=6s[���hL���1������a�
���:T<0c��q�A��>6 �q�6J����*���l�?RI�kw���7��bD�|:c��e��'����A���[���@�����y�Ql�65;n�S�59M������a�����`��y�V�u.�A�>1
�+
.��*��B�UE��uj��S������Y%��lr��$��3�l���M�-�����������A/�uz`k�q�<xH��"��Y���5�gD+��N�J�X��U**�!���
��)b���������\��y�������[�<��n;n�KE-BNE����:=�5���e<Te���I� W|q*������izUQ��"4Y��EAwP��
��x�[-��W�T��I��+^��`hjJ:;�B[<`�*�)���^�����^�����0���l�1�6�@s��1UE����d�'�S��.���KZiX�|�	l�j�����+�(�
��<p���"���R�4u���[�<����7���3��{�x�N/�uz`k�q�<x�x 6l�0��������V���
@j������v��h}�C��dv�z��[�LL�
H�(^�����x�����.UR����-���/�]w�������#
�z�Dkqtc<R!�F)�X���z�4��M���	9s�����.d|,���k����R����2�rft��@���`����{5'�a���n����fZt�P
�����m�O��
-�J�v�Gli&������<���D���P~��:A���k}w�C��>\������X�|�6������n���*�Q�a/v����.�!����m�m,A�:�&!"��t����{X�����N�2�4>'����6h0��P����<����B���n��K��d�K���lOY�}6tsO�ZU�b���U�V�a������|�&�>7�u~t���A`�B�Y�2D9�nN��[)(t'Wqf�F��h���U��TJ��!�����]�U#hv�
�e/�`�j���s�b���=��P�������=t��bys��
y���E�������m&J�M���+�O�r��[<e���XQ�J��u�9	�dN�S��J�N�Tg���H��X=�yft�'�H+_T��J���a{����iB�8�����n�������n8[u���K��g&�3V��e8���L�"J���)~i��
FU��3��sF���
#u���<�t�D������������n�������-cv�������Q�8�����n�E����� t��� �g�zx}�LW]I��Z�~�#D&y�s7K�����uH�n�{�v�
�,F��yr��������n��y�m#���x�{��]������;��������6v4�Xu�M�P��"i�B��fu�f���&b���P�n���p�C���8�t_c7]��
9�t����g��2 ����tB@;~��������F+��
y��B�n[��C�n8H	�76��N�A�H^�kUA�Mts	1�3��)�q�
R82:^�O:���k^�Gon�=Qn������<������?���!QU��Z��3��_�z�y��bj5CA�e��~7�%���{�������b��_��j���?���}��/_>�e���(6.��_pr�-�����PK��
��/�PK{2K18sept/pg_part_patch/4_4.outUX���Y���Y��}]�,����~E���`����D�;��^��8�a�Z���a�^����=�~2+�bf���X�\�R����l �$@���_��_��_���?�����g��|�����^~���_^^~�����|c#y�@+J�?��QE��������y�����7�����~��/������������
.*U�/�������/�b��^^���O�������?~������?}�������G����+��������_>}��������OH�����>���������|�@d|T�,?���y�y�7{��C2�WD��u:�s�����&��v������O�������_^?�������l������~q����M�Q����O������[��Z~kt&�UQ��[���rg��|���s��v7�������������7�����/�������9kA[6���rS��!�4^�pp1�76T���_~���������w�~|���rO�?T�+����rwRp�^�k�� �����w��|)�������~��������>}�{��X�������o^�������W����_��������/W����o����N����������_�o��|I[��O_y����~�������������[��'+�������u5���������kW�t�xg;c�
/����A/�����b�������?&+������/�?������%3����peV���5+$����kv��J�/
H'��a�+3������d�����?�������>������?����w�~J�|����?q?~���U��o��k��Di����@��X^��.�6���l&(Z�L�}�c��V�C�2��J�7>ir�6����R/��`u���6���N�����E������:=������%L;lb�1�f'66U����?����I���(yisE����
@oE�eG]Ls���@H��"�C�	h�,������
��������f��
��0�]`�Z0���:`c���,}=f��N�c����������%���b&&3m�.bn��L���8���0��1���-�
bz��bG]����
>D���L���&oM���^�
��C2�� ��W6��v@BH �_��?����o@�1�s(�7�X��9@����-'�n���C����V�9������[�( �o�K�h�o��i~���� �3����A�A��A����{ �&��c���� 
���HL,:�`���K�#T�.����$T��Nr|T����XoMN�����\�	2o���(������05��'��a�9Ea���������:=�ab84��@g11��)�/�d��1R0Jpl���+EU{���� Fek��U@�b��!�1�S:Y���(�b��A��H,(p[������c���� F
���t�h��, �S��1f\��`{�Xp  �
b��w��&��zk�, fJv�u����#E����i�31��XhAJ��r��9~�������:=����Hc���0�y~�Zi���]�����&�yeA������k4��}���H���������g!M���+#���3m�~����A�~<��6��$d}w�n���Iw��~�tq�y��b�jql?�u��I����o`�IC�b]	�z�dFz5|����qxGc���)(���}�[��,|���h��`�zM���S��gf��Y~�G���m?YP%X��������xl�Y`��x�u���h`3���%b��ku�q�{���f��
��0����.�e�y.��)��`���g<�����l`�Gl�eh�(=��a�QV��e����:=�5�e<ll�1�6@��|b8MfM2M� m���>�v�3��El����<&����]=���T4�4�%���������/�����iv��ME��)�����:=�5�e\�c��Fl�04'������k�6�z�{��I�gc$~�\""p��w��	l�j&u��`T\���B��M]������W�!S���	=h�eh���7��j�8���Nl
%�6�j�u���
0����
9=�t�Q'b��He8����`LC����Pa6�����u�
A:k�B]���u�.�>�a_
5�@���>�yg.�������	�f���Q��$L����)b3AZ9�Mx�p>UE9)�s0��0(b����������nU��KM��AO��z������dv��WEK�*���������f�Dl�12��
3{�oLF)���gY�=(v�����X~���K�� `���%A��D�����I**��J�����t6?��nZ&�4]X&Wn�6 �����������f}O*�2��
��Q��,��A H����b���H�[:c�����nPU�����+�lZI;�Dl&'A���X�M��&6rO8c����7{n�k� ���A��������f���Fl��bL�u�Y��[��D:��I����c����M`����D��B����~'b�1Ip0j�G����x�`���JV��a/b#�9b��Z�N/�uz`kh�Y/glK*j����T<��y��el����k���o4�

TJ�A9c����o[��������g4E�����Q}�c��'�l�=!m�.�7���r�^��jL��A/�uz`k�f0�@�:�faD!�iB�����C^�7���Wq��*�-�1�*����"������l�����Y~�PRQ^(M�n�>����m�ZU�k7l#6������.z!���m�v�y�H�&Flv��E��R��1t��|��o�|%`�i�F�YS�/(���ZF�|�����V1����xg;�b��>A����kE{j�
[`�X[{�tq�y������=�PU��:�fa����x ��+W�l�A�����6(]n�l�j:qE���R�N��do�A��R<�vnJ��tk���\u�
{gl�������B^���y`wf<F��c(Uv���ERF������L��zgl!��!eVZk���n�l��_H��l����PJ��,p�V�{��u#}��f��#�^Q����*������:=�5�;�y Fl�0\�/�l�K�Y���
E&�qD����1l�W�B��0��Erv���4�x�v��6������SNb���ld<_�B^���y`G3��c�m����p�F#2��J��5�v#����LYu�����@o\��$u�T�<Pw"6i����c��p���b��[|B���+U!�Tn�m�pK*Jz!��[�<�;��#b���Kc�v�����b@�u2p��]���t9��8�� �"�W���A��������n���I+
o9������	}l�eXQM;n�U�������:=�5��e\��cUQ��f�ah~��x�2`]LFd����AI�)gl�c(U���g]n�����cor��A�'�
\E���I���x`k��2����vi�Q��A/�uz`k�v�<x,�A$x;�@�:LB�!`P�3^�	�S�<���D�2���1
�6\�O�36����������HX����*Ea���8�6�~������T��E\���������a���#�&Fl�0�vQ1���G�#�P�D��l��3�@W}l�(U6^��$�� ���d;����r��a�H�AO=�h<����i
Zuj7�ElAh���A/�uz`k�v�y�	��:�faN���$��la�$�u���R��R�T�]?������zl"%R@J����&��x`��#5{��_<p��|%d\�a�A�d�A������5��e<��6���M�<�	.B����]��@0=]�B�]
sqc"6W1��E���on=�v�3��?�2	���\��y����l���n�<-y���-���/������a���Q���c�m���X�t�����Q��NY�f�����MCv����*s*�1X�IC����H�B��d�z d���:���Z��4���
{}l��\�B^���y�vf<��c��n�A�c#�c��z��t#��h�*T��v�W��&��U���W�+!E��������Ml@OHEq��
a�
���v�������i�p�=n���A��,�:�|�j6�/\Q���Jnoi�E�J������jy,=Gl(�V��+E"�Q���*�[��<�m�=\������@`�8���NlM��;(4y�`�4�����A���,YC/���{��xr�`�����Oi�M`W�s�A���jRo��]�OS�\Qy9��;s������m�p�n����[��TE-�B^�����
����i�=�UhrU���x@:gl��)�J�R�n�u�j"C.�2ja�3@kn'���]_UR�N����g6t�&��FlZw��'4c�B��"���y���v�m�x���c�m�v\H�����K������d�tQ����Dl^_��e`#���Da4]�h���7Uy��s��b���s���~��!�5����36��3�r��N/�uv`�M��,4)Fl~�v�<A����W��N�GP&�)U��y�#6���{xs�?�g�@��x����<)�QQJ�T�=�N-
f|��o�=p�����w�F��[����y���v�m������1��~�v���(�6"'n����VE���>J[�{������O�����U���t��-��=�T�\���*o���~#4L�[]�a�R���{!��[�������]����\�J�����[C%ip]�e(���+��0qe�������&���?������Bi� =�I��9c�G����-<�\���p�����;09�����^�����M��B�f�M"4�e�����]��{@GhR&Kfip�R�>6�3�|�?��
����|g�((�f`�*Vd=���{�4���[��p��=J�e�Y�y���I�#4��P�� �I?��$��I�T<��
���u���HW 7��%b�fP���a�c���uR������I\QPt9���j���l��Cl�IE]Qh���l.�<H��������;B��������,B��Sz!�G�9�z�F�vE��Tq�g���Y?����U!)�F�}E�W�uI��c�XK����T��=�Z�IuQ����������x�>/�uz`k�~�<XO!��c�m�gk.�y�^�U��t���v
N2L9c�Agl��	�!�
B#X���TE9�2k�\5�2z�l8��A�-�q�
{��#6�8���Nl
����d���Y��20���^�� t��!#�{$��*��,�&�����r�fJU�P�8���H�	���
h��T�/
�7B�&�2��r��@g�T��B^���y�����d���c�m���P���FjZG�:gl2�#&u� ���]�s�������SUTN�
,a�G��-��/�%%��x��00����P1�*7���,4�.z!��[�<�;��Gf�Q��,����=4�m�o�KU�ve�&�"�	���!����S>1"��tX;�w#6+t
IE]������(;`8��Vh�i�r�^�&��Z.z!��4��a<���1�(��<����F���	����"u$R�JC50���cs�?���n��=�����+��B��ssE}|����[)���vSQ�T����B^���y[��%�9�����0���e�'X�����<0�Qb-�P�`�48�����{b�+JZ���L�K�]%q�0u�.<A�6���:#�v�	�6�\�B^���y[��c
�l�1�6�@:�BNE����N�
�a.���{����vpW��3����!�N�����<0+ip�{���gl�2������\d��C��y!��[�<�-��.��v4�faX�J
���
����vH��5Z��B5��1�����&�
��:��Wl&�3���@s����zl�2,������v��r*�.z!��[�<���=gl��M�:�&a��&:�48 [�_�=��\(�{�������Af�T��*�]��,+H��:38}��M�R<�;��[�
�8�l���I*�����(F�UQ�8���Nl
�v��H*�Fl�0�"����}IE���k������R�oQ���y`��-\��W<�TT�@K����0Ht93����E�a�+�*�n����(_�B^���y[��c�6�`��y@bR6�VT�
z�h��<^B�������M`����lA"��{
�^QR��A���x����<�g�f�����n��=�I���tq�y�����<x�x�Fl�0��glZ[��.�%r�S�J�`K#��j�����fJ�Ue$��a����=��������{��6���I�n�VEIEi�\.z!��[�<���=�m��l�0�D����VF�^��:glDi�Dl��}l����x�?or�.��0ie���{����	9W�
f�v|B�f�YUM?.n�k� X�=�A/�uz`k��e<��a�{��<�>$�mg@$�.#�(�����h�>6�FDZ���PW����y|��[��Y���}lTz8l�����/`�<��b�Un�;c3y�h�8����l�0p�<x�+*Fl8
�@�����L�P��t�=LL$x-%������1UQ�������!y�/ �+[��M���$ADssE�����e����;n���q�/� ��B^���y�[��CB�b�1�6
� `R�p�	m�Rs�;�J�[����I��`�W�KUQ�-������-3�$Q�����y`�x=6�0$��;n���(���A/�uz`k��e<��F��8��)�ov�go�2���N**�Di�c��0BC�BW��40Y�j�n���t�=B�T�&0�v��E����l���{`�<p������������z!��[�<���C$x6�`��y�"gR���E�.���3��(��=�?�������36_���a.BX�oP�������d�.�
z��������0��wu{n�+�h%,������a��y�P��u�M�<����<p2H���{�G���v)5Q�&�[�1�EX�<��<y>
	��������d����}l��a.��y���R����yX"6���:=�5��2./`{���c�m���H��P�	S�3���4��R_&�{��`�j��I��������t���Q$5U���~f`������q(\�1�v'��e|%�>/�uz`k���<x�A��:�fa$F{"�����������d����(�{��I�#U�s
S*�NE��:��������n��������
��2�1�,��
{30�<���Xy��:=�5���yp����T��:�&a8i{���"�����d�p�G9������h�� $	]���r���a�,
3�i����>dCxB.�RB��V�����jj�8���N�l
��6�������\���l+���-t�6��
�����I�����9UB�]G��:e#T>�W�q�,�)��3Y4MJd��T�����U��l�F�	u��{`��N;n�#Uy�L����^�����pj�
��U2l��.`c���Y���'eQ��l_����>��%d���R�	�u��l��6���nUgW��)�o�(�IQ�t�E�xc�
��������U�4,n�wqw���:;�Q�=����*�s���C��f�H�@�,D�������v.*Q.�"'nJ���}"�o����MlQ������-�>e�
RD6~m�s4w���/��)�|j7�ElV������:=�5�����Ml:�f��\���hGlk��	��"��8-����?�|@�j���"����}�
^Jw�;���h�w��,�>����j������������Y-��������f}��lall�1�6	��I���FX�N�uW��f{gl�����GW�&��z�o��6�Y��c�
5G{}���7h]b�r�5�`���0�O���2M�r�^�FBm[.z!��[�=����y��
�LL�I���M'��LK5k��S<R�Dl��%bscHU��6(en�����EO��1[����tg�Y��>����~����
{-��������������f]�
9c5
�f��W��1�����H�T����@�4k�&���\m�g
]��P���i�E-��+U�����C|��f�A����:�l�gjA��z!��[�=��:���c�m������%�E�y��`#!ElZ��
���AU�j�������_�V:���K%����J���s��b��������z�"�����q.!�����^�����pj��HE��c�m�AK*�L�\A��.[4����B�'.�j���
.��/c��:�.��Dg�G��O=�����Ei�=	�=7���&�����^�����pj��h�`���I��8�6���E@�!UY0�RQ��.�EUE��]��F��C�>6'���]=Hd��
��-�[��n�����
JS��A/�uz`k�6��P�.���fa�49��HQ��x�'�p�7�$Y������&��j�G�>��T����,��=�o�����xn`{�[����j�h��=NU�h_.z!��[�<�����`��y��x]�QP��,�w�����T�1&��"�-�i�
�j�.1DW�L5����(��Le(�f8��s@�b���3�=�n�=b��=*7����,�h4�����-4��0�y 6la���IUQ�5��M���H9��K��� `��AXSU�3@$~�z�AnAEC��#�����	��N=��*;n��/��rq�y���A�2����j����Y��8M����iB���El������.%b�Wk�M`���M!e�!X�����6�"��������N����g6��gla3��Usk7���AX������:=�5��p�yp3bc���i��2��"J`3�l�@8��3�h](U����y2�+x.���w�:�X�=�b9b���x1���-<�Z����6����>6c1���������a�-����Al�0�3��`+`����J�fJ�[�T�1U���R����Xu���0AE�|Q)D7u������e�+`��E�_�B^���y����H��6l�0$��I��2���2��7���Y��)�����[�<�20El�C�l�Ay
��r��>��NM�g���
��iv�����K*jK*�N/�uz`k�a�<x���c�m��2�34rx���}0opECbH;Ei�E3�+�� ,���Wex!B�+�����o�wrYc���R�v�;B�c��
�\Qa�-������a�-���TTl:��a�UE�"�d	��(�n� �%������F3�x@W�Yh-��kg(��T�w�$4��ku����.�>�����7��0�*7�����l�����:=�5��0�y 6l�0� UE������G&cV�b`�d����-\mP$����J�&M�*�h���GT�h��{d2�'Dl-���X�:k7�El�.gl��{��y���A�2"�����a��uzIE#1�J
�?�J���C(
���`�!�V�<�9��iic�4�"{_t�VT�(���7^h2lfp�������d(WE������Nl
� l�Ell�1�6	��[�����A&V��w�=���+*�4��Y������w	���lypua��E��U�\��
��#���<�"�w��6�������:;���y����"6��`��0����o��
k�����36T�����h4W�*x��- mO�����q�P�,?w���\��ap��{n�����+(�B^���y��������v�8���36B@��$������D����[G����������h���TE��H/��:�X�fj����)U�e8c�B{��]<��_�����Nl
� n�Ell�1�6
�����h�S����=uPVZ��E_
�o<�*���j�/�����Zv�&��(g�F�SGl�	�\��y�/(U�alV��A�8���Nl
� n��������M��,�#�15�:�	m�<�VEEjSfE�(��;�]o���`8�#�jJU4==6�ho�(x,3��������m�<�P����zl���UQ�8���Nl
� n�v}g*�1U�8
� eR�-xu6)cu��H�+j���4�7��M`��QF%JU��J��3~�7�t���J���Zg6�`��T0;n�S��"��\�B^���y�2��c�m���2�si�`�X�� ���J��A�PA(����uj��Sk���Y��		��I���Mto�v1����	����<0d�lQ���v
�0}�y���A�2�RE�H�q���y&�<�l��<p���PY<�M^�{�A�(Ut�l��l�&����0`�WQ����|���y��-��
���{��:=�5���e<Te���I�N��M�(��jU�����d�IA�)W18�l���Q"<�h$������"6���D}��705���l�-0L9�����A�:b�c�>/�uz`k�q����`��y������������`�d�|n��k�%�p���M`�Wt���bIE9T��(�lF*ka���7�Z��<]�RU���T�����/M7��B^���y7����b�1�6��PPh��.�����Hg��W����"y���A�*���M>d�����o�31�+��*�R����
�x�����.TR���b6�1�lN�A7�uvh;^a��aGF����&A7�#Ri�����#�mt����4�XsX��bc��\�0�t�X��/I�nO���YUq��F�5S'���n-�:W�U�~��e~�e�tC^�G7�����pW�v��M�:�f� xQ�`�A{
��[�0B����$h��3�f��By�!*J�Q�������:���I�!��d���O�n���*_,��e���a/-�a���2���n����f[t;�C��nl�A�6	���%�I�[��5ut'���f�����@�������y�K����~�R��*���UT����X�l��O�ZU�b���U�V�aO�����tq�
y��\�n�� �U��,l��b7�r�V

oL�2!��)��i���J��� gs��������#hv{@��.�O�����@�X�l�O�#T���({����Q��n���:?����B`�B��(	6�n���d?n�KE%(��
�r�����;��9	"�$�,��H�o�Ke��R���n���F�	,��)!�?l��(�����n����-�mi	���U��,�m)3I�X�%�����(���hA���FU�j��JJ����G���9E �+J�������R���gC7��tkg"Dy��?�6���4��x�
y���E����� t��� rF�zx)8�/^�+�$���fa]��������j�:�s7�{��v�g��H��|�w}^�}6t����f0�T����:BR��rq�
y���E�������m���I�Skw���S3���t�F����z�1�+���m����e�B�#v��dR� %�(����-��[�QH'����	���TMGx��:?����,����� %���f �BN�8y%�U�o�'�1
��A����D�����W���a�#�%/E�����lO�G����������f)�3���(U�uB��;�������:?���j�����m��>�d�A������?|����V�����������������_�|`����c��(�����N�3������PK��V�/�PK{2K18sept/pg_part_patch/5_1.outUX���Y���Y��}k�%�q�w���&���|-L�,�%	���'b�� ��Y������dTe���y�����
�������u2^'^^���/�O�������p��{���/*?y�����������?���1P�1'��R���|����y�������_������������}�����_�L�Fio�j�b��}���������M���|y����~��4^>k�������I���)������W/?����~��o�x}����_i����z������~����_���������������w��/?���o�����?|��������o��������^>��������w��P~�����'����_���*����o�������������������w�_;C9��
�����
��1z�b\
�UI>��J��b��>���A}���w���~~�����o^?��������^����8�u?pH�����X�~����O?�����>|xe�����w?~��7���?����=��c�P�U���)&R��~���3{�&��1g>@��/]���sn���n��??9���Y~~�=e��_���%lL*f*�Dr)������~�vJ���t\O����������y����#���w?����}|�a5�5�P��:XHS��tjz��Bp6(J��H�*��h��������n^d?�����}���_�@����'����?{��?�'��~���G~���%V�|}�����g�o7���&e4����c���'<���sq=��p�?�B,V����h}�d5��M�5��O����X���`�Fg����0�/���Qe�����#���x�y�TZ���fN�T-������x�	k}v��kcAXO�X��P[��v���N>:>��?$�?����q�d�|�G��mo���;����p����w]������~���������~�����/n����x�M��n��u�/���71r�-�MT>�WL���W����/j�&����.&���Oz�Rs�;����Vm��#��m$����}���W/F?�����g�k���n�������F������������{����k[uq{���B����9��3_����I���XN��^�X_��b�r
����/��&g��6�?���O����?�/��e/ZU��_�_�������S��?�_���}���v�x,���GF��^^�v�_7B����������w�������W;l��	��*�;N�=�W|y(o�E��t���}8~>�G�\���c���t���0�;�������
>|J�0�|�|�q�co���_l7����r�~h8����x*���O��q��l�9��t�V��8��+�f��=�j��%}�w��>!s�����!q��9K�\�`sg��\|�9�c7�c��w���0�oB�j�.r��� �{=zE��Xf�
�ov��8_������x{`�
`�;�6����rzE����NRd�U��}�����Oa
�9��V�fM�X'i�=�E�W7��WE�8�9$x5��?i\CC��g,f�ehflr�R�3,&rM�9)��9��FEa��
����O�(��-rtq�>A�,����P�)��!��Wd�8�7��z���4t���q�B��%�ZL�8�	�b��[���>s��/����h������I�@���O���<�.Q��L��
�]��_��>c14�S���+Z}A�j1�k��?]�6��%!n��A�����2��"W@�l�-�K4�-��,�!�DMRx����������1]�]�q����B�����nd3��|�-�.Vt	yD��-s����)��W��G����#~W��~B�?�\��S�K���w%vkx�b&d�/I��n��D��.�t����J~��
��2��	(��-rt���������j�q];�OR�������M�������%l3c�C�5vI1�3cxt�!��r������.���bD[�jko�]���E��WN'e�R:]BC@J_��-f�N��Uq��3,&�gty ��K�E~���Ds�]�Q��)Q��[�
����Jf�5_�����q*K3��[��S��%�%�.q��ljf�3D��Z��Qw�D]�2'3��0Vb���0��]���� �S��V6{4��<Ww9���x�3���!�a�l����j�i1�k�K8]�-�8W���������(��-rt�����>��������(���X-�]R��KtI��K��.N�2���afU}����K��.f���;�KJ�M}�V�=��%C*�j�(�����&��WwI]�}��X���x<�����{ZL�����A�t�.����As����rT��2��"W@��b�A��q�VH����a�R	M��:��M��!N������0&o�e\��5�L0q����H�]�����e��i�0�%����oq���2�\d���ZMD�#�'�d�r�Nn���"������t�)4�����\v�9w�-���|F�8}�<���	�&:]��c������c��d7��i>�h�(',�Z����0�}�1�>���>Fo�e~�A
���qy�8�L��������i
s���Q��B��8f=_����,Z�B��w@.�9��~����3:h�����/��N���E�g����<	>�G��~��{N�N�X~!�r�D��t��Y���>1ER~�lf���� ��l9���1��M���>�sO�����+AF5)?�K?$�q����'�G�I�s���jsq�#�Av�`6`�j����������' s����0��j9)�g@O1��E������d��i����o�uU�)���Lt����kT*
���
y����apX�BC��q����i�3 	0���-I��*���2��{��av�Z��sE�����A8[f6��X�a��r�4��=yF��I��797�����b���N�����9���@���;>� ��[�E;�(].r�)��Z��tPz��F��1�����]��M����:>@�O�9�'���XVZnq��cAx���
��]������=|_�GD�.������6-$+�V�q���<Y�t�����I�%\�@�aA �[�hW��\y�;yn�B���w|�<�Q��e����.���Ao�I��+f��e�qxu�.{�1���ep��V�����g�!`��0[��Re��@������m��mc����<��2'vy�g�J�H��b�	
��R��
H�V�qQ���i�@��P ��0���r|e����$��\y��E�=�&���>� �����E��r� �+�p��|1�5�I�yu$��e*
�#������� ��_��/`���2�������������o����9?���#&+���Z]��1ft<�HV���8sZ����#��u���r�!��^d�2�^���\y�I=����y:�o�mN��e��E��<���A���}�fU���l&��9��c5x*)g ;�����q���{��#k*_�m`"��(�^�Vb2�y���W�a�[&E�.�C�~7g1R�h�jN���a� �w�9�a�R�����r�c8��"�9#=�'��'[�����A$���Lt����D�L����������� �C���?|j���^s/���%����.����6&�2=#��Vs�5��C���	I#L.E]]l�2w�����j��)�joB�x�t$.'%��o����^��^����%�j9�kBO:��m�U����8��T�W���m��E.>��[)�������7d� ����,�me�c[�f�D����h�-m�'�u��=eL��TB%����KZ�N�C���Z]�iT�p�*���U����oD���0�C��Y�`J�m�i�)��&�����6T�C�l�8};��-X��m��E.>��*'���[y����p�cU,�nM�N���z�c9
||�8>��3�X��������I1�0�`F1�[�e�tR���UAX?E�.�s|�W|����f�r���9�tL�g��V��\1�$+#�����\{���=� im�Z���c����?jQ&�\�
���Z�={A��kw�E���6��U�������G; 3����[2��CW�s�Y
x~<9�a����'5������0�,}�V�N>���Q�)�h��/��w:����#3@[�2O�f�p����_Z|��7S,'rQ�9'�v{�M�dUGF���
{\R$+CD��t�+�O$��Y�'w+����2`�7����������R�Gjd�82{���7��B��<&%�����������s����[��e��\��������.���ZM"�.]�"���(���������s�����813�N�����	z{����k��|�jPX-q�e��E��<�#�������"�30{2�:�Tc
��[�\�����qd���������W�1��bm��s������lw��c�F�0��Z]�����T7�H�h�4�]���<K z@��vd�����R�J���bULz��D��<'
���l��mc�����|CA�{Q&�\�
���Z���}����L~0�&c@�x6vN�F�v�q4@���=����� ��W�y�8X�1���PW�u�/��=��{1xJe$_�6kmi\������&ZK;��^m�H�ud��E��[.��R(��^+����\y�)���l�y��;>��z�DfvQ&�\���A"�c�V�\�����R���u(�FR��<zb�m@f`�#3�[2��������*f�����Z��v�\�W��`���EQ�Y�+�6��F{�4�~H���O���j�tdX�H�9�!�����`��5��N���N�y6d����|��y�
`���(].r�I�\H�=2Ay8���b_�I�����Z��md�hd ����Gf`�d����V�$H��
>�����:����I��6�e1��V�=aH��5NH��Z=MN�6��������K����3����W��ZR��o��r"���x�L���F��k_��M�CN9��,�D��\z(�JY�th�����}	1'EeT��UA��j�vi@�����f`�l���+_ �'��Li��!��R k�c��5i_;*��8Q���1k?*�`���V/Zc���A�n�c3�A��Y���|+�TO@�F�)��(������d����A���7n�TB��(].r��9:��M������mX���ta��\�.�6o����ql����W��J��>(5�g��b�wV�I�����H������a���3�Kg�h���i->�<=��N*��,�Sh�V������\yNJ�m��^��=�o���T��2��"@��� ����:"B�-���=���wm��O�&�*{���R��K��K��>r��a���[��a�9������LdQW�l�-	U���Z�J�G%O�O���vT�+���\����p�C]�eO��\yN��08>�uuY��m�AT7V�2��"W@��M���:������>:�ra��w�S��:M���M=v�c`��1�[��v[�����`�<DJ�S�z�3����,�e�����{juq��l���Dg��x��g
�d{��8��0�����|�x-�`9g�N���sR���1���c}��M��/|M[��.����$�m<]����x����������Y��������<�8w�����V��(�4
z(�
�6�0c�]�m��z���P[Z���Z]�hts��<�-�����%����Y��s����K�*>a&y�5�'��V}{�AcaT�w|�A�|��}�(].r�q�"/���_O��=�Ww�>8�P��`2��j�M�R� ����Gb���dj�#=�0g�%D���n;����Fh����&ju1Gk�}	�7���j-w����<��c�����`.k��n$v��D.�<�����l~���w|�%1(�H�e��E.�<d��Q�bg>)m;�y�����J����D(�T���4����H��� ���uz@�SJ�ZFI�J��y�.T�z�L^���Z]lRe�0:���j�4������m�#1H�(���\�@1�~t��q!L1������<�<���+�z�obT,�Lt����Y�����C�����@O����D�A��b�h���j7��v�uU�;]�M*���)-fd2B��V�*��������T+uu��~H��r��E�vR�j����Ny���@����()�����EIS�y��KeW��\yN����l�ybV�����6�	J��e��E.�<d�^��~��z��^�����R�)K��:O��mtU�������:�3�e���:��f[��9I+!:���)�������Z]�0�!��}����m��w����:�� ��-g@:�\��uaz��D.
=����U�Z�������B�$f�Mt�����b$kz,��jcutc��"�Y���6��c�<7h�v�5V���j�����Op����<a�X�'Y����#7������,(��8��t>)�:��N�?�� �������<&>g9>�
%"lr�:��=
'rM�9+��m����~��Mc�7|�tJ-�D��\y�[�@'�H�IY+93l29&���T�����.��4V����mc5nV�V�a����x�������
�O��IaV�-���V�m�<�8K�m��i
����@��X5��9��Z��8�����v5��5������'��B=��|K��!��(].r�qX��a���[!^?�y,&!�I�1�P\�R,�K�b4V�q��t�Xm5����z��>N�i��\&-�fCc��j�-|�uuqHi8����	+V���i��W�gK=�<�5V{���y��F@K�~#o�c9��B�=�G������w����\�=2��n����\z|��\�
-\S��
[��%�Xy<���CP��P������t\g5m:������z�N������Y���'�cw��8i+�G�be2E��`��
��J�/o�=�j��
D�'=���j�:�A��=o��WipR��h|�b8��"�9]�'������o���7!&#E��r�+ O0�sg������V��c
2$-���bR��n���P��g�YM�uV�mg����X�*�����G�Dj�D�����Kh~yr(�g�Z�IC��|a�>Z�6���J�g�mVS�Y�-��g,s�&�5��P;N���sR�����K�j<��77q���2�E��r�+ O�(����A�n��x�f4����x���k:����g���[G��p1�J����1>*�D�c�o����F�y(O���&���6'S(p`�\�v��y����P���������3�O������L���5�'��cp{�9�5�
��N�����p�m��E�>��J����7��V�F����"v���yK*�Eh���e�j>�
qt��8�n�C�m�I}z0P��r�"����h��'�]�������������=�����MH��Z%C���T�Y��(����E���A�	
y=�S,'rM�9����h���v��;�oj=|.h3e��E.=du^�bg�����C6y�q��'�j���8W����<.����V���qX$�V�Y�g��8��Cl}�jkp��'��Vl�t����m������-�����] R�O����q\r|�r�,w�E�F���w5������<�<�
q�)�t���Z�eM2	e��E��<`�4����������y�{��d�����P�7���y\t��rh�9X\3� �����j/+�D~C�n[����Cl.2Y��`d��7�����h�4�]�a�<�zD�����z��Y�h�K��?��T��r"�����Co��	��V�w��	z"!���Lt����a�Ho5�?Z�,�0.��]�d��c�t���n�rR��qdtoE\�$��m�:�R=��*���]���&,
n��.~�n�?���V�V[�y��G��PGfj���&�8�����mMw���E���>�-���m�Lz���c[��2�&:]�
�c�;P8�5��6�p3v���.#��h^����31���qdt�� ��%�r�[���m���-�WR�Bf0)����-.�ju����|�8��N�U��)�����N����@������Zy�2�F�[c���r"����
��e8i�.��7�c�1IT6�6��"�S6�>j�zrF����@'��>�k��U��<7��q|t��@���M���`9vr
C�l�����b��|q%#�Nh�0���`8��&Z-��]�b�<	>�>_���s�3�tG�r9Y��b9����9;Jo���`��M����_�M.���Mt����%�|thd:��xoB>+�u�|�����=����(
�q�~Ki������I��t���P��]q�4i����K}U��M��M���-���ju�.�=~N���'��J}Gi���H?g9������
�Jy���D.
>���m�|�n7���������-�D��\|��L�����e^���=�H-�F7�W�[�g��H
�q�~�..c�QK�e������P�I!������%>���C��Q���.�m�#���V+����c}��R��8R�I�e�zx����N1��E�������-���������� ^Z��.��8����<!�jU9������y�p\y���L5���k? 5������8�a�����������tA%�����#���8����������i��I�9euA�j������t���H
���9�e�"EU��|HS'rM�9����d3���8D{�o����=z�D��r�+ ��~!ri��,�=�C�A=M������V��y�����H
��� !��8D��8DC�2�w�UZ�7�<y�\)<D�[Q�-n�6"����V���wBy��s�����v��s��6��@����4��]
'rQ�9����d�B�w�����Vd��eEv�s�J/W@������=&�[��H����x��J�����=)D�����h�o
�7A�awd����$����$��}�-Nb���C����|[�NAZ�Fs��!�u��Uk�"�����|�h@e��3�Y�3�;Pc����\y�i3�=�H��������{�
X��(].r�Ar�$�q!vDn!�!s*�s�d
�|E�����
�q�~Kh���������-E�At��Y��m�%Myo��~��JQ���az��Q!4X��A������}Gh=��=g9�NUh��)8e���]-'rM�9���a4����;�=�!�L �6��"W�fN��-dr��)c�q�<�`T*a��q��;oi�p��8
�-��O6�-�J�_��
����h��}�-����n��+����o��1aK�h�'e�������erX��&�K}�i��]���r��T!����u4�p"��p�l(
�f�B������C	7�6qMq����<+�=h����3�a��w;~���<lq�[�����}�<J���RDP����`���`���fD�����<v���{���E�S���x4��1�����j�4��(DC:y�A��-*}��_�L.��9�:�p"E�s�JoO6�X���|[�0|CC�H��.���@�����w�;7N���^�P���fl��hb�m�g���3�[>��[sF�������m��Z@�G�n[�Bqy�5���Q��]������a�g���i1�!	��� 8[F����AS�����7W�4�r"��|�l����9};��GiO��D��\|<_��M��j����HirN+�2��o(j$��
�>>����3`�Y�r�p�&�;�8`\��a�����Yn�/�����V��0���o�O�Z����U{�����_�����c�!e�_�b���8�r"�G�,�l�H�$�zL"j�i_>�h�.r�	��	���������)_�����^�v�e���'��q|�����Bn�Z
��p�0������*�r[����"����=����?�C�����}�j!���<�s;|B�g�\���7[��^��KE��3n��D��=��I��=��=|�������e<s��=��Lt�����6�`��YQ��e?.Z$�@��3Yf���9�]�����������R�6RJc�;��.k��<��Z�AJ��t8�*J�\��������
�j��=�es��q�#3H� ���2R�e"
{2�u5�p"�D��{nO6����L��o���������Lt�������J��:��#1��W�&���l�VF
]{���^d�82��%3�b)e�IFJ����#Z�0)��h����"���Z]i�<�p�/1�h-}��C�:�������s�3:~� ��z��D.
=����y���!���RRm��E�>�Tn��m|!:��]�14s�A
��z8�&6���A8�� ��3���=�� ���������	4��=q"�{�8��V�E�������j��vj�j!��n�����3 \���\J�O��<(��N������*�=�&`srw|�m�2�b���Lt�����zi�����&���A����2�c;��z����3�3���[>��/J=�A���Z�Ax��}S���")X���8��W��K]�v���V���=,+
�A�IE����-,n��]����\zN�*�=�XV�w{�����_5�!m��E.>.1�8�����K��(�zL��J?\.�l�E�M��]�
�g@i��4���^b�Y-�N�{�OY<F�����y0!�"�!~oJ����/��a���0��j�*���A�'��@]:J����PW��r|�S��a����&�i8�kbO<��`�h`Su{�o������Se��E.�<���[��E��r��y&�����a�q&�{�����6�3���[>�u��<��y�<�4�j�-�
�������R�RZf#��.�b�!��P���hmR.��<���[�3���Q!?g9�o���)����\zNZ�}{�
�s���N�S�Y��Vm��E�>!�e�4!Q��Gm���Eh,�c�1���>~f�3�4�Q�;��KT{�����f�|1�@�lK� �a��>Yge|D�.6q���X��uS�Zk�9�����[Gi`�Q�<g�� x�(
L�b8�kb�YMnJ����ud��o8D�9q��-�D��\y\�C��h ���Z
S��h�T�@����MG����}�<J�p�A��{�|�����&��M�e2S���/��������&7��������`h;�;��=b����{����W�aOOi�]�z{�r����"��^Q�S,'rQ�9����4`���8���l��-�&Ot��������[�0�R#Lm�T���T�\����&����8J�p3���E�1��9�!���Bn��[���$2h����6��Q����9t�-Z=J�<�x�����$_c�7[��#����I�g���\y�9���31�:��;}��5���(].r�q��Uw�{�kQ
U����[g�<��;|�C�������,��6�x�A�[�A��-��c� ��9�������L��A�s!����
�@VA���Z���2��A���s�3h_,�<
`��.�{ZN�������=��;�Ttw���/��e��E��;D��������^)�c����w�9�p����Z��6�Z���� �e�w\����V�KJ��e�a���hN��r����V��1��V��V�M)�B�#�,�0N;"��T�9�����h��dU�{O��\v�9�����D0���|[�&>'e�LQ&�\�
��@{��w�uJ������A*����&b<�a���qD��I�qt�[��842hC���2�;�
mh���x�K$i6Q���`J����h��oxyv�,�c1�����
od0���r��XN����H���
�8��W��-m(Z�J�E��t� ����h�L��U�6�1��)3��X��%�
<d�����A<�� �m���Z�����<@�2e��2>����I�Y���������F4tT�Z+��B� �l���Y��]�
`��~�r��4,��G1���]-'rM�	�xn�5����N����+���Mt�������!��<Xj�
����U.��Z�rC?�� X�q,��1_w��b;�
� ���W�P��)�0�e1��V��lG�b�OQ�����B� �'�;
OIe����fdYg<���)��(����v{�Mpx���oJ<1��M�E��r��F�e%n�u!R��
+�ee(��&�I���mq@`�#0�����N����k�����_�
�ciRK6�:a/��W�&!��(\ �V�@�9+�������`����Y�`g�E��P�qqO��\w���m�BJ�:���w�$�/](�Mt�����\xUypV�$O�G`��f@$�K�c=�y����p�<�x�A��1�H3����k�c����6ci�~_�����aS��}��.Z��q�q�r8s�4�~����!R��
���4�r"E�s�Bo��	�o��v�w��\�^�c�&:]���@�d�G�T��l53&mC:�l��(3�\�nz����q�1n���C��� �ac�V���F�o���6��mh�W���1��{�r��Uk��� ���t�G�S7��)�����r_.>�����v���5�'��l�(��k���E�vU9dE��t�+ �bi�Ne���D��p����Y�����������|�<��x{AL�1�
����q;��yFs+Za')��R�N�'����L�j�/=��P^�Z���tZ�'����.��E%�������T�4�r"�D��Ho��	�&��p���P���Pm��E.�<D�m%�m	����y"��eO}2F�xs�.�x�P������w�:��1��c�V	�x�z�B	�yR�D��V�QW��4����J��b���o�>�?A���]�\��po��d�
�*:)�)��&������\���x����6Jz���6��"@��.��@^�wcS�Rh�i%!����f8p��B�4 /H���-y�7T��� �IV�cC���S�!?����_;�E���q���U^����h5}`��:�>�(4���*y�[-�}ThQ(���)��&��p
��kP�3t����6��7_�e��E�; ��;:�v|N��`F��@i�\k��f^�-
��q�i�]����:�q�/���V����j|f��u�T��_y��&!^��;��Y
�y��a�z�r���|(�C�����(��S��=��>�������K��R�m��E��<9���XR-+���?�k�����r�������q��}�%���!�V����V���	�]��?��#zy��������a�`�����U��cOC�@��� ������3�De���r���v���E�G��<�������(�QglQ&�\��C�-�95�n���n8B���
�m�x"��[����� G^����:����0T���LF���N��hs|��"Q��p������������<��D[#/HAg�����L������e���D��;'����k�&EU���N���h��Rv(�D��\y\����^��x/�G�a���
�@�mmm��"M��tyA��809��<�!� �Eh���4��<g);J/{e��������UVJ<����/�!��:,�o�\��S�"P�����XN���sNs���6��nGY��;mm�c���m��E.�<��V���=���7;r��,���(��.�-����g@_���/Hw���#lK���a7����.V�� �
��L���4�h��EwN�H�Z��y��������Q@��[�h�����3���D.�<�����k�yR���9}�k+{3�A��.��K
D���j���@�z]HY�V��x������ G^���p�;������l*���1�r}w������(�
Q��x�5X�@�V����oI���'�;�� b���,gzD����
�%���D.�;�l��=�|��T�_��m���@
��"�D��\y8V���r�g���G4�k��� �����m�w	���� G^�����Z��'��(�	�:���ph!L�yP�^��E��
~���9�E�Z�z����S�#"������J��[.�_1��E���f��D�	<'���k�0����&���1�
]��.����Ti�5����\�,{�|�F���y�V+<��e��v��8���e.p�*j��R�nE��T��$w�R��C(x������P�Mi�������w�jm2���G;��O��Yyg~� ����~��D��;g�x6�(�t�C���
Z����D��\yl�Y�X8s�?ZM�=�	�S�|I�z�MC�}��3`.H�1�-se�J<"����3T����A�I����TRm�Z�k���<�X��V�%���O�I�10T7��]��{��@��3Q��t�aW��\y�9��a.@j��s�y�U�pW.�D��\y���K�-�<�nL6�q[�O��,T,�_`�O��k?o�4��q�y�\��u��<�o�C��*a`�%p���.�]&o��&�R���2�aH���m��s�h�C���kK=��.���ri�.��S����r"��s:�o�	(=t����[�!�����E��t�@�En /D�.���b�Hs�HRj�����_@z"���q�y�^�4�i�<)�ay�)�!�j�������9A���L��V��4L��{*�%������i��t�r�������<o���|Ap��\�\���D��<'�yn��	HO�{N��^�%��j�.r�q�{�%��58C��
�q���S9$��v�\]�����n���|{A���D��)R3n��
}�������� ;7	y���r�E��E�y�6|�����A�e�����X��;�t��x�r�t�K�?��]-'rM������kv2�U����;�6�H��h�.r�IdK:�9 ��V�H��2�+F��r���-��1��� �_���������pO���*��BO�I�9����7G��_����7	�X�Z��Q�!E�]�p@�'w��Rz�r��E9DLA%h:N���E�����s��F��=�o��W���+�&:]�
�cS�U�&;K|_A��8^8����>��bn\�:�`�g�_���/�[����j�m"=V�}�a���"�����:���@���C��2��	/Z������0��;����J|�r�O�7FJ0?�)��(����v{�9��E��8}[�����Mt����UZX�
iF��#��8$_�I����D�
W���<�������� ��*R�`iY�2#�]Y��*�]�/���S���Z��������@�9�V�R�{xW�Y��#�<���S���s��x72\!OI�+j��ZN���sV���� bu������[�1����U��t�+ �������u�Y���q�F'��"5&�k[u�f"��qy�`����Yb;D��[�A���������y|�AY����k�1����U���p^�'Q���-�X������D�i����\yNb0�=�&$��s�y\�92@�6��"@�1�+�H�C�Z
q�`P�E���[�~�C��A>�� o(�:�^6����d1J�4Z�L�y�>����Q����������#�1v/���h��wY4+�l������� ��,�~�����SX����ES,'�g�y$�"����#]���;�o�����qk�&:]�
��>t�z���x�cNfLb`�R+G�x����Kjs�<�|�A���
�f��<zLb�y��Z����@��^�{�9������n8�c"�x�^.Z+�������;�`��r�`���<As�d���\y�9��!1���54vN���Vj�6��"@J��,D)�����QS�z�0&W��^��U��l��� Gb��$��������������R�a`&��	�~e���V�f=����r%�h���i�y(�<�����<g9���:������ZN���N�ynI"f�������:9��6��"W@p&k��X��E�Lv��{I�B��-�k��9�y�|z�Ga�m������=�U��>��q�}*a�.��w��`�*�wQ��)��e��AX�:�`������m4Vw��FR���t�z�����+��v5��5�'����9���������m3��o'��@��^��8t����@RM:[��vj� [�a�����-J0z�d��8~�\u���`M�����
+������I{I��*.���6����A��Ek�9�>�#b����I���L�,�C��e?�t"���pN�ms�1�T4w���~�^ytD��u�+���k���kc��)�hR|�w�/��N���
���w�"����q��!4�|��
�c7�~|��&���Rov��X������9���
�<e�T�z��A9
~h���k�:�f�a�M���RV��)��&���h�9�&X�5�^���w*M�h�.r�M�]8j�����|�cThe��W���[p3�����8����8�'��8��������t=fV����������7��cBr��s�vI�<�|;���s/r/������P���T�h��f��N���s����d���R<���[����^��A��u�+�a
V^v��Z[��'���|��%��~������~7�t�s��-����4&�*��.�J���~�����0)/t��6�������6]��=jO��t�t�� �lo7���T,�J=G���N���sZ��%7���y;�������q�6��"W���g��N���(R���]R�U�
��w���%�if����q�l�
�f�nC������,��������8�=f��/AY~&js�F?�\�n.Z=MXq���CZ|�|KY9�������k�����)��&��s��lN6�����o[�W"7r�&z]�
�:P/�,E�����A���z��W6��=����(����������v�mX�.��c��G�9���SZ�o�V�4��x���V���;m���Q����P*��gL����>�QO���5�������f���R[|����l��K-�D��\{X��QC(���)�)����8����w�u��'f�=���q���9p�;��>4&������(&}G��'~()/+|D�.F;�{���G�V��=O�I�q��>���a)���t���
��>+j���N���sV�������(�����]����&z]�
���V��������x;f���T-��=����D�o��g�t��8~�\0]h�Sv]�C?}*5v��<)���egI�����a�w�����s�����e2V����!������Cx�t��E���4�r"�D�@'�����Y��z�oB����,�D�������I�p������6bje��e�<�!����xY�|���6`:�_�<�3��f���Md�-��c�n�bC.�L@����
ke��7�������?@���!�v�G��;�������v���|(�����,��J>��.S�&rA�q�_�'�����T���|�
�/'8_�Lt�������crevi����2
�~���m���Q�e�}Q��)���H3��o`������Ph���Cw��/6�i�$��by���g�|������{������w��� �+��������k�������&�lv6�vH�����r�S>�2iA8��Y;����m'~S�!��_:!����I�i/�6������`����
"�c\��O������S������-���q����$w��Qi�r���m��{�,V��/7��g����2x��?����vG>����P�[��J6+�Lhs�$����������O�Zf����/�����_����_���7�>M�����_F���^�y���?����O ��q�D���d���{H�f�Q aV���}����L&�>���=��>g?���������g,��^6�]�%d]=�5�?~�����~]>����+,�����'/����,o,�;��M���4����`����'?��PK�%nH��PK{2K18sept/pg_part_patch/5_2.outUX���Y���Y��}k�%�q�w���fi�9X- ���5�6��$����f��==k��~�02�����J;�)'&$LEW���7"��F�xy9E�������?/�7���s~���~yS��������������������sR:+����������������w���/?����}����w��"�!��6J�?��e�~��=�������O//������������������[�S�������_���������}�������~�5�o������y���O��~}�/?�3�������w����_�<��7�����>����/�������������W/�?~���}�����(���~�������%��_�����������?}���_����^�����P�[����PQ��C�,��0����7%R��������?~P~���w��?�����}���'�l�~������ ���a���Z�7l\����E���O������}����W��{�s���_�������O��������J�QSLNe����J��OJ��k���5��A����=��/����ze��o�w���C�����
Z�/��V�o���
��o�$"m����,^8��������r���Y��g��/�~�������l"C�?[T��B�E����M
���F�s0����J�����?�G]md?��������_�@����'����_���o��?��?|�����7b����o����i�nZ���"^����c���|�����y}�G�����sFSV�m ��vf���������>��k,�4}s���A~����*�5�9H���0)i^*I��!2z�S5����W.�fJ�i��_=��u����h�v���N>Z^������Y���q&�?@T�j�o��������S|��k���~���������x����������8���
�_���d��-��U��S6�y������U?�.��D�	����.v�;�w���9���3���/�����z��?~����?|�B���;�>3\�;�����~x����^�R~���
���?�����������,U��h4oI�,�x���_d�����!���W�����P1-�opx�O�z�3���R�����������������_����������w��e���s�|/������Q�������zy���W1��������|��Cu��o��������M�a_�+��?.���x7e��E����p{yp�N� �'����:�6P>�	&]b���<�D�0vyLy�y�c���Ol����\��w
g����-�m��^f����0y�.G��6'�C~C�0p�2k�����*�!{���<��m�Y.����-lb8��"��y�
y�S&l�|��i#yD��t��
��������3���{�g��(#�	&�l��#�������0�>F;|WV����=�Y�~''���E:>�!p�\��{�h�_�v=���vLQ����U�4�t4Oq�w�`�6�!�P����O	c��~��D��0�6��|�jN�/�����DE��(��-rt��(������O���p$$�!@q&�x�����u����.n�KsI�����g"��+'d�0l�^��3^�H>m�E��M�/|bHoOJ}�!�Pd�DM'��k�b�rf���c����+�����^�5���c��kw���������s����W�)�K��Z�1�.�$y���/����w�q��;������WkO���|?x��-����_B��K�G@�F���Hz���W�{��aUF&tR�W��o�)\���%���/ng��a���X�_������=�%^�.~���o����MyEQ��[��b������j�j���vD�SNy8���+������A�p��uj��TSc)�'�1����RfN�#5�&�DvM��m���eM�{����r�]��XN��R:]BCcU�{,F�dg�b��!�	]��^t	�������k8e�l���8g��[�Z���k���E���AW.�Q�I��*�tD�������x��U��K`��1�U�������L��6y����y6����h5�w����������2Q'������������6{,���{^C��0��=�%\-�Z�������	����)_�9[���CD	[�jTM�|�F�Oj�{%0��ZR�KhI�AKZ�-V9_�E�~R,��?���Z�BG�|o�-)���Y��t�n9�I��XQ�b;��M�O��%-�\H���b��|0��������[�5������y��{��%�P�����-rl��(\4s�����3��^�B����j$oyM�lZ�X�����!L^w��Tcj��@���\�%T����5b��I�:dt�j�yZ�l%���2�����LZ�DwDOdg�rxS�7�S��
n�����%�.��Z��C�0�p"���H��N~g��~��[��JwmQ�\��cP
5�����}����v{3#�BL*��I;f�F��0�6���{y��g!�W2|�����	����I�������n��	Z�0���M���9���1��
.�'�f�s�W!�=!�����������:A��E���C,'rS���h�������9��3��r�*����i�2��"7����+�Sd�����9��H��A��za�@����y���H���Z�^�}�1S�(	�]�C=����>�Z��6��
zx�MH��Z�N7���m�'=���|��k9<
1x=�C�b8��"�5A������7|��c0��#Q�\��c���J�@�]�!O"��5�h�6���-���C3�#`��<Z:����9�� �	O����	�KPj�D��m�`�4jNju�����$[����<�l�O���C�M��\AS�m�j
~��D��<I_�<+Z��Z�������iD�r�; ����B�����4���_��B�l���<��� O���#�Gr��J+�@�-��|,��%e�F��(1�1��u��Q��3"ju1�/{����HU�V�q�_�<Y��,�1���\�Y�M�A�n��D��<�^�<�|��C�
���m.�AU�I�r� ����-m7I[��|D�����=�4��t�h���������C@���8 �T	{��8-6�o4h�G?�RC:�7ly�g�J�H��bP���7@Zp��9.��H8��� �����r	��z^ZN�Vz��Dn�<�P�=�l
����Pm��u���A-��t��
���h.KSN��bj��b��:���E�� 1^����s�Wy:�t��9��������N/��3�Q�������P{�d���juq����e`�7�Y�;�.+1p��Y�����j����s�
j���N���.�0X�8�1����x����m�����yB��������
�L7i�4o�Deb��C-n��+�����7@�|�Y�
���=��h��-4�H��lS�u����(ju��r�xd0AC�f������yN�0X��`KB`��R�L3HS��[�b8��"�5�<�+��'����������|����.��o���3��$�g�~up �[kB^�-�gh�A����c#�G6 ��"�����Vsa�	1�(��ue��!�P[t7!�o����.6F��'-����Dk���C(���<��YP�����,��fP�`�Y���N����.�m[1����
���C>���BM�@���y"���H���[B�'��2Kq�d�0��=y:dtY=�y"�1Oi��V`~���	����7����F��b�v�����m��|��4#5ku5�<���M�����-������g9�R'�V����!��'���j�V�S�h�����c�yD�r� �u�(+����*�{��9t�H�)2�����{":y:$t��I2��"�f��1��F+)a�����21hk<A0*�������a\ Ex��2t�E���l�	�9� 1����Y�4#�f��*�6�C
'rS�����qe��-�akK���y�O����2��"w@������=�@�h�v��6��1��j�k�s
�F�pm#�����H�#��6@�,)���Okgq]z6>QP�/:I�����{����@��b�_��,m��g�����Lgw��Y���m�$}���u�����g��y5��M���l�������@>_�>^��yD�r�; OtR�3!O�y�u��q��A������il��0��t8�yf����Z��`������%�p'��xQ���5��O�6����V�Z�U�-WF��Y�wH����l�	�<f�a0�����st�T�m[����'��k�>.m���QY.���������!E��N���Br8M;��$/:~
��m�0���������'`��0�����h�#�
�M�[)��}���sH�����C�H[��(2Q��s���L�E~0f���t��W����'\�����G<��Qr� �����j���ZN���sQO����`�k���N_����e�:]��^]-�m�����7������jV]j:���F��0����#�rn(ki���!��7B	>Pg���mO<"d��|\T2�zR[��q��[DO6f��ZK=\W[m���-�8�����n�r��(q���r��P���|���y\��8R-��p�:���Oh���N�����E9R�Jx���E���2�����L�|���[����Gf`�d��-��Y��'���`Q/�%���u��k��)j[Mu0�*��28`P�5i�E��i~"{���	�� 3��C�Y������6��$���=�'\���\FE������f*\�6��"w����[�CZTp���*M(�*		��66�jGG��w���g`��30k>���3������g�+�
Pm8<�����A��&�i(��V�'�sg��8ku5Y}YoOr'������$a�v��1�&��sb9�{�O����qi#����o��5�$�\F��
t��
���"����M�v�����&������*�*��O��
w��Ci`��40kJ����f�N��
�%��5�.��M�Fe�'��02K/��<�U�����e�s��YP$&�}�#\�a�G�������|��}������_Z�y�����+]��'m��En>&G�\* �VC�[��"����la$[p�������j`�c50����������~���Y
3��������N��a��99yV��bo{�#��F������������g�j��Ji��Sh
*�qSd����|.J��h
�l����������o�:]��cQ����0f�Z��@�>���%{D�������l/5bs��Y�Qt������pqK9��Rko�Z��{mL{)���&���.��������� �:�RO�O���,�
8�W����������a�Fb9����E]>+f}c�t�
|�s����I�t�;��/��a�ha���W����
��D��c���=��m`��60kn����m�O�}&Qt@
��fy;p���2nrXG������8�~��7]�����9��������qf�m�
~������[j���ZN���sQ��&7���e}���R�I�t��������
��]qJ���LA��: >���p>�a��7@�v�
�y�v����Vj��:�z�RV	0�m�L�2mo���`�������'����D>E�y��P�z��9�����\�-�F��	�4Aa%�j�kJ��6�-H����u�����Q��6��"7K�+L� ���9L���=|��1"94���'�C�.:��a8��1�
���.������lPf���N�>vP��C)xV7Q��9f�O�A��.�{������;�3�g�p`Qe��Y.a���4���!��)�\S����)�K�m�|}n�_��+�@���ya(���������lbitV�����(�Vgm�8z��7����5�A�m�+.|�n��Q]��[0)6VC	�V�o�rg>����ByP�hS��#Z]M��������]��DJ�}�K| 3/���Z��P����<�d/���
"Fb�
��:�����t��
��?����CM>��~�O��2iA!G�� Wdm;E���"k�QdMTI�y��g*�O|���[Z$���'�
{����������8k5G:�6����6�dTE�����gQd�\Rv���@��y��N���s�e�����'fU���>_!O�3��'e��En�<�x_�&��U��:����<.�K���_o�h��m�
k{^��]WX��Up-�f�\�xC*��:���\�� �13<�%��@���`z'w2��
�V�q7�g��?� ��-������-?���Ky�5��*����>_���@�4)].r���U�m	�{p=V�����d�fF}0#O�N���D
������j{^y�},���S�x0 O���������-9��g��`���RQ��c}���'�D�`x��j������ey5��}�#���a�nm�!��'�\�n[�W_kW�p�������P��6��"7�sx.W=|�j�'9�d����l^e|8l���!���W�����cy5n��`5�O�d�Q	�_�h�p3����07Qb�Z]��5���{f����/�����i���#R�����@'T������!��'�\4��qix���N_�W� ��fm��En>��yj,u|�k�s�	�[@�
��c��#��0�q��jw^y�{,�6������f�?F���������
72v���l��uvqH���c��X:i-���h��{�sB������3|�n9��$WH
�5Yb8��b��{W6>|���jK���hL�
��t����'>�&Quj|U�E�;��"n�@��i~��Q��4pp��W��������/jnmrF'~F{�s��D=��������8hd���,#�)��+C�9�h��f�Z�������:���-��}�q��-��X*d�����V'k9��b�5��K��3����o��e
���
t���'��i��f��|b�����}R����T�%kQ�E6�qI7���v��W���j�A1�����~�A����bo'B�4��!����P��V�.J������ �V���� �7�v��[�Wx��Y��T�:��kD�[N���sQ��*�v���n^8}>�d>l�E�6��"w��[K
��}+Lw��3����Q����-��u��q���qn=>.]��08;����������Q-�qyPs��P8���\�o�I�7[D>��V]�VzBs�[���gyf���}�t�����z��D�	>��r�����/����t�|(�\*'m��E�>�P���Z70R�>�T����e&��E]�|�!�T����s���s��q��2mv�OO:Ka�p�ase:L�d)O�J!}Ug�����hm�[:Km���H�C(OE�R���c���"���r[�r���ZYMC'rO�����qe���,�9/|����
�KjR�\����_��fg>�W��L�I�
	�l����'����'�q��3;��7;�m���T�6;�`tk���x/�E![��U����oq`��t��Z]��{�k
x��`��V�sB��	���������rd@�V�tB��OvC,'��<��������:m8}cv�S��oO�@���|��Ir�)8��11u�1�r%������x:�;���5p���GZ�1�h�U��T��'q�X��4��m@��E��((;�au^�t�P'a�:�g��j2�����#z���rv��*�}�C	?h����/
���=��"���M!�+I���+��H�$L��
t��
�������:��_f�%�>�"�
������\�:��<f�5;�M��[��j��=��2f����F>��	��R���M������x�t1�^�Z�s
��������3��>�Q8�d�K``�~��Dn
>U��
�8^g��g�{��?�I�t�;���%�����9�#N����-�\l�Q�a��Vg`��!7p���
r�`��8��{6;.1�#4��� �v�"�wMqqU�����Xp��Bb-Z]M�^>gT,�
\`��i9�Zx�m�V���j9����E�����`���p���&��m��En>��H��
\��3J?�w>�e���[���<���u�
�y�n����fS#��?5��Y��^��"���n�� ��d��n�������q��T�������	���AP��Y��w�p��
bb9����5�K�6q�c�������Ky��aK��@���|,y>�'�|th�:�����S��8CIl��vC%�(��r��_� �2-�F�<!�(����Y�z���sL�����\J��Z]L&WM>�Pf�&�I�}v1�>E��OL't�����I9��r�4��?�11#�!��)�\�a���9�)�N_�Y>�a��
t�����n�d����^�������X�a�`@9�
�!e��O����Go����@�[�-����M��n�Cz3����q�����:n�� juq�M��|�6��0Ga��
�5��Y�1��/�
���rdAX�K�:���b9����5M>�K[8E��p���'!FB�<i�.r���*[�'D��������Rk����|x�U�3��"�+��z��_���6*���sr��x?�~��D���:���2eV���V�.3���IH�MZ�#�j�OJ'�����F��>����.Bk�T�b9�{��E���K�I�q�.��=5��&m��E�>��0��DT�jj�Z?�E2��H�4b���8z��7����5�A���m�n�����$��������O�a�^{7�����#��)��U������#����9a��_�x6�N�,GF�4����W�b�����)�\S����A*�i���������?i�.r�IZ�Iy��M?72V����R�%���04�w>�d�z���7�4];��v#���phA�h���H�_kw>�����0��I�TT0X�^�>;�����Ik#������|Ao����=����2�cY��X�����{��7x\���Z��E>_����Qg�&e��E��<�h�$��=Y;��@�Q
>Ee�4J�*�"gG�A���?�����
0�}�^,>�_��x3w�7��}1��IVn5����l�Z]M��#u��j�Z!\�a�N���z�����,G��<zB���XN���s�������q�����U���h�.r��l�TX�BvV����\��<Q�|N������5>kc����z���7`��T�2��������
o��C m���2y����Sr�������I��Q�i!�[mpB��_��`��{o��(#�]��g�XN�������[��t����.���6��"7����wf���s�c����O]G<���"�f)j��O����Go����z0��G?�B�� �R��q���;�>���[M>Wg�U���ZU\���Q��VWS��R4����Dm���Z�H[E��pV#)`�XN���sM����f���Eo9}���M����N�����4>;�3
|�����k��T����=7X���
L�u�
�y�~Mos��(M>�_�fb���S�_���!��C`�I>�V���v��E���E��������� XS:��X�Pn@W����X�t����M�'_>+zT,�-���'z����6��"w�b��'X�������{8P��U4�%�(Z���O����Go��
��0~W�1���nF����p�.�
��*u���'���V?a5��	-s��jw�ew>�����o$S��f�?l�r��gZ��i9����������KVQ��U��S8��Y�t�;�OHn��i%�I�>��O�z�u�k���n�$���7���Gz����`�a|U
�[�<��5��l�A���i����(��^���a����<B<$B��v;|��� ���1ro�\FOT�3�mUj��N�'�y&;��qe3�d��v���}'Ah�&e��En�<�4�>�zq�d�}��?�L�oz�
i�s;����<n�p�AXs0�����R�O�gm$���
P���<)
"��P�	P�����3���!p��9�f�����6H|�u;-G�����M�/�/GZN���sQ�����[��e�p�F�AR���N�@���|l�Y��k��$F����v�-�=��uf�
�8>���6�q�5����R�����-��:����2L�
��>RN��E�.����S��\[��2��:�9��-�//|�
����5��u`��i��Dn�=��YQ8>��v�����Fe�L�@���y��
]���&�����m���1L�C���Z�$(�����A8�� l�P��Qj���t�\*E��P�Fg�x#��$����J��\7���L%9�ZM"`��U��g�=b������d9��n���N������-}\�@�Q�K��X��z����t��
���������l�s��-����)k�n�0�<�������X
�#��=�����������0&��n�:�u�yBN��I�]���
����YkG�C�bE���9aziX�����>���{f �^=�p"�D��ZKW6���+���W���HF�I�r� ��U�6�#a�������dnQ��i	�&F����A� O�� ��jY
xS�g����:%m��6�pm
�Q0���!E�c�(>�����(���'8>��rt���t:"+(�y�	4�a�jr�h��n�Xx���CF����y�EEkR>y5:���������)�@���y�o.2:.�zY�G<���6�#x�:v����"��
C��A8�� <R
�|�j�=A��>��VXW+h���
nv�����	uv1�d��m`)7��g�Z�\�y���Fi��*�}�Klj��6�E�`���y.�����	:�]����y���wnR�\����ZJ��<�u����W�����
��-#�b?2���������b�%�y��{k,���nR���j1�!����+�#ju1��C�2�Yk�:V��m[�����r������lB���XN���sUy��� ��|�������4��6�&e��En=6��0{48
��yu!�@��{>��R�c���4�z�5���A8�� <�<{m1�'����M�7d��=��=��r^�����6T������uah�?�\A9��&���������,�@Wh�Y.��cEl&�����y.��Y���U�>_!�1�8�2��"7@��/���h���]j�'�t���C
��]jdd�<���<*�p�Ax�w��p��rz�~��_�����PVJMuD�>w\,U�����*�����g��R��1��F���A�5�y���ym�-F��`�N��������A��uf�������*dR�\��c��+$�v�F�kR�i���"�CQSmt4���3�q�Jc�� �Gb�v�,���>�������-2��6�����nb����vgd��Z{�#-Z���f0����xr�,Wh�}q�`�4�r"���tM/���f����p����6�%�:]���\��{Lv)/X�c����O G,I�@�xh\Mu����H"���f��kMu~2�X��=�������qp�p�Lj�o��%�||�yd�Z{�!?"{���&��`0����q���2�f����8�r"�D�x
e����`9�uy��+�1)����I�t�; O��/��32����%�A^*��e2R����%�8��0����H�9es�y�� �+��Q��b�G���
O�h:��(j�����*
w��U6�cRd��3��������Y.Y�@Y��.��c'��<�����/�rm�Z�|U��v���I�r����;%��f���E}�uV�LJ��!��Yx�W�;��<��h�O
j��-J������|=kc<
*,����Se��_��<QG�5k�~����7<'��4��9sg�>��z�c%Ii��l�XN������y\��ik<b_8}�8��GI�L�@���y� nJ���kK(�'�7&��[��q~`mA������[<A��v��._���^����% ��K�4�����_�My0#S�� Z+=�cA�O�,4.��K*�}�#4�R�8����P���y��i{\�l��O��o��0A����
t��
��Qi���6>��YHY'�������Y#C�����/���D��<T����OSm��
��sz�c���6_����]��_�������������T?�' ������g9����h�� b9�{"�U��yAHi1�v��5�`�� �h�.r�1����@�u�d~��m�U*��T5��S
��"� O�� �G_�
y|L5VD���4%b�X��1m�]��!O-/���_o��m���LZ=�q�q�\8:ywA�����r'�������P���y��
}\���*�
���F����h�.r���l����E��mK���m����YM�*jd��f��t��y�1����C�}A��� ��`9������5.�%A8���H=��6������5��CRd/S�%��� ��>J�,G60\%qq`��!��'���b�w��V-����6ko�(����N�������T������F��n��Q�\
�0�Y�%e�!��;��a/�����BJ��4����[���e��|m>Gw�4��{��1m(9�I��u��p�!�
��q����y�?y��!�>��j���.7d�+���j9�{"�EM�������
�[:}=���X�G'm��En�<�7A���-�"�.� �L�d�lj�Oh�9���<��x{A\�x�[3�<���mT�y���D���Iy_~��^ �������a^S�Y��v�;�B�O�)�y���q��f����1�"O	B�l�XN���s�`��uM�h���7������A��N��X\�b��l�R�!������"Rr��������}���A:�� �	<�:�������~�n��~��{��`Q�c����=��=�h5}`�e�<����$0@���g9>uhI�����<�r"�D�.A��uM�{�6��F\8H��h�.r�A�m��9.������ f��"4z+�Z�
E����C`��#0Hk��'b ��?�h����F�������<zn#���u�yR�M7#����>0���y��U�'��$Z"�<n�����M%�@|8�4�r"7E�k�y�5���6Lv���=O�*����6��"w@��laD,1�Smb|�~�� �/��6��q���!0H��
�Tn�Hs������s�M�<���)��n!J������������S�ZM��yl:y�Y�o�a\�)��tT>�!��)��k�g�`(-9,��7j����9�:]�����.�9���W&�n'���;Q�yPR�y�u5y:�<���0]hNK��� m��)���<�������Dm�5��p��B-����lw�\�xF��q����g9������6�������y.��y\��N���l8}��u�TUO�@���yli��b��������q�����&���I�:�<���0�	��
y�S�)�m	��s����1�H]�z.3���.v+L����IkKI�z>BZr�0U��r%���F����l9��"�5������AeK�����`O�@���y�h�������|�A��~����/,��x��b#�vn`���0H�q�
�kX^b����4{��yn�(��C���]��5��7���y��x
�Ik�:�q������ ���>��LJ�2+�����0�r"7E�k�s�5�<iY��p�Fm[Fv���m��E��<��D�t� ��D)9�>�A
o!/#����*��<�t�A��0����x�Cz2"!�����8���������kU���wg9L;��?���^|��%��1O>y�A�����Y�l��H��;d��l>�r"7E�k��<�k>yW�`����e%��<E�t� ����W�d�b`�R�9�M�VQ��5 �K������m��a���0Hk��Z��|z���UM�j5�I��1y4&��m����h�� ��D����z���UU�3b����pOzp�n9�lk�Q��3�&4�r"�D��C�5�d����{��)�6�:]��c1�(��ho��6t)���nT�e,��y��{�3�A��A:�� �9,�����K�?�u��v�Mt������A��t[��������u��-�<Ek
��<��I_��j��#� +����-9���=���{���y���WUk��>
��6��"7@9d)���PFZM�}�m�|.�"����E��<��w���a���0Hk���z��1�{v���r`S7���<���9����m���_�{����5ku)�CX�Dv"��U��������-G�Y���|�������y�5���0���
��cr����6��"7@��r����'�=��R�W�!]���'�~�M�cP�<��a���0�k�G���'�	����8>O�_d��!��[��I�0��t���e�*��Y�/���l[:!����P���r��7�M1$�7dGZN���sMU������6���0qE)����N�����+O��6��t:�u�='h����6�Ut��������0��q�5�����yR����^g�2�bE�Lz{�c�P����Z��R���e�f�ZS��,��N��� ���<o�9��
���E��b9�{"�E�<���Y����7b>GF[b��
t��
����N��-��9z����\��<S�<b��4�����7�%�r�� �Ga��6�������GG#��S�7�����c��IOCf&��5����M����N��=H�s�;$��a�yAa`�.�{,GCy.L4��!��'��k(�5;�w��7����*<!�6��"w��LIf�<}�u�q�u��8��YH���LL5��~d��a0��1�5���-���gs�"ZkLN��-���nwR�w�������O��C��R�V�p.�"{g$�p���Fn������-<���!��)�\����Q�hX�p�
x?&�\N�@���x���``�S��!��s��+R)��!bR��	G����0��������Ze[N���q(-�QX���>R?f�K���Q����a2B��9k��/\�<��n��`0���@�,G��$2�x1�iY�C-'rS�����q]s��s��p�:�F�8KM�h�.r��E�&�jr|��i7������6RB��R��j(.���qt��C`��#0�k�^���2��>H��+Y�������?��r
<7���&��������
�`��������K`p�%O#0������rd3J���"1d���C-'rO����������uN_������I�t�;O��4����1~o�&�-/p�F����@��c���O�� ��_���b�k���.�X>e1���[h�c"�O##Q�{��}�w	�(���V���K�|�%��� ��k��r���X�� R:�!��'�\�_���)$��l9}#��1����6��"7���+�HK�B�Z
�_���f����A��d<��|}A^��H�����q�H��"�fZB��[�'nH���D�[�"��������7�h��6+�7�vFA���MW2���#�����S�2����<�d/�����7�����N�I�t�;�g�![.y�^�����K_�P�A��o�A7�I��5;��a/����5{jh��������p��T:�>d��oNSU��6��^?�CSn.�r���K��
M'L����`��;-G���d�����!��)��k�g�^�KyQ��p�x�FR�#�@���x\
�E�;�U3G�����|.�2��?-�F#sm��|{A^�X��=�)�.by�k���K��.�A�yzVu��V�f����K��#�V�p6\6����'/.y@K�,G�&e
K��c��XN���.
y��Fh��,�����O�@���y�#�����QD���d0����G�a��9[yHw��ga��u�����{���'�P�a\X��CJ�7��1O��Q��]����W�ZmO@��U�m'U/����.��t���XV������N������m[�lI��-��Qc`QZj�6��"7���/���Ge��C���Q>�
�dal�y0�t�t8�������E]im?&w�������|���]�{����Vg�����������O����s��y�g�>���')�����k�[Lk!;�t"���pM�m�����?mx}�N��[�0Y�6��"w���V���$U�J��y���@'�<��P\}Y~:L����g������2�]7�	�)4f�����e�.�N:��:������Rc�=t��j��2�9��z�^
 ��z��l:p���	�;�J���P���~."3X�l
�/�
�^���0]�L�@���~@�h�z�u(����oXF��^%[�C9��t�������������ol:�r�]�'��}P�*%%U����0r	�V�{H�Pdo���*��{�{������0c�R��.)�1��=����������Kjq����N��O�2x�z]���0+Os�b�[}��'��V�D?Dn���q�n�����y���\��%�hN}
�:��z=o���!�.{7a��\���'��G�Z+5�5�xw�,x
<�,�3u����[����H���~.�~���1�����=�����m��E�?e�&	�N���(�d���rcs,�������4���Cm��8~���d5�����J�F9n\D?��f�-V0���3Q�����.���<iu5a.�U3�O)=���[���3�G���	�����N������Z�?���
��+��~�K��^��0��2��'���b"�o�x$����Ah����q���7�_��>k�k}��E�u�Wm|Q�f���������SZro�V����A��\����s��~�3M�����r6�c:�Nt����:�t"�D��~V+��'�Jl���������"�I�u�;�n�I�D��l5�@�O���\�$����Zw�u8�'�?���y���9��>�?�O��{���)�i�{���~��9�e|����>��4?eF���+�C
�Dv��=%�Y`��}����d��)�k��Dn
?W�]�������7(v�4Ga�z]��c�V����e�f-��O��A
�c�<�
?9��~:\����g�vP�4��)<��
f7��/�����2��(�[�V�����:K�3k�}[�+�7�v�������C�g:�j��S�(^�CL'rO�	�"�Y�XcZ����k�����/���^�w?�����JYT��f"ZW��[s�q������4����������<�yl��M�6�}�������%G���v6~`3���qF�(������;��{��x���/����������!v�Sj��o�l�,�v,FTi43�9�=�l"7�K�Q_;k:�r����u(��I��t����Y�c��������2n��|\������^T����0��������==��:��>���|���
B�!��^��=s�1D�����?|����'��o>�����^�����#f-�v��+�-Cm������i�@W��I.���.^�2i�:�;-�Cw�fy5 O-_����I>h0�:���[�@����p+�,���~z�m}�EW��7���������&�k���5p��b�����i�O��j��t2����[CE����'G����R](����`%���Kt#�$����������O�-3n�q������/~������w�>
�������}�������o�����'�Q��S��gmJ��������F|��|��v��O����~��C���)||�~�����}��}{,�G
��8�_b�n���w���������}��yG|�������^���lY�������s�SF�?���PK' yVH��PK'{2K18sept/pg_part_patch/5_3.outUX���Y��Y���m�%�q&�]��~�����������]���1X�'��U�
���������o<�L�u������������sND�#����)������������7�p������7�������>|yy��7�~���(����Y��7/�?��~m^����}���}�����}���w��|����_o
i�%��o������{��?cS��?�������-���Z����������Hy���������?���������?|y��o�6����������w����������e��W����w��^~��������7�~���,/��o��o�������O�����O���^�����?�o�_���3�����%���}����������������~��w_�w�r��*��`(MY+gs3�uI^��J�|
���w_>|��>������?�>�z���?�w���W?oPY/���luRN������������>����^���w?�g��?���~���^�A|�|�����c�P��F�j��Tv�5&%�TL�|S��k'm�v�|�}nz�Fk���O�:�������m^��{�%��J��L
���~	r�|~�7i���_�{�������������_s�Y�?���/���WY�f"/XHS���7Y(���>�}bHYe����)��o>}�����~�������_^~����?��?���o��?�W���/�xcS��V�������x�gzn7�s���f���rLZ��/�R������`�"o\)��Kk��
>�H6Q��m��?}�o�z�~��5:#-��=`��|�=��j
c\r��ec������c�H9�S��>#Y����.SNYZ�DO�X����h�v���'�(*Z����������f�W>��Q�{�����|X�����\�����],x�����}��������y������N��4��pt��n��u��������/�E{eSX����5��yu��6Z>H��Vuu1�{.��qS�����e�j�����H���o�����z1��	�'��0\�O�����������������y���/�����?�����=K�t!Z�G�}�3�@�ymP6F�?2�r"������+V,��w8���_�����~�
�����!���?��y���U��?�����w�}���q��/����O��6^���������^�������l����������T}��w����j���6�q_E��������m��E���p�|p������%��jj��h�|�����2�3>������c�����>�|���v[MD9[?4���G\O|bu�������=|�.���6'7>p3J��d��J�*�1���
|B� -����!(�1'�O��f=�r"7{
��
�XR6<q�|���QY��N����xg�z�����a��#!�a��g��N8b�����;v�c�����ubvz:�c���S��C5����@����mS���Y�ntV&#����^iX����U�����q
i���c1��J�e����a1�{b�E���26�#U
[wo�Y��E�y���8>�zt	��z���
�%���yk��������A

���C�d��1��41�C��,*Y���]��3]�P���E�~M~�.dx��^�(��|B�=��9]�����������2��3,&rOt���=0�*��~�.��E��n�;�K6���%��
�9
����2\�TX��'�2�����?^�^�V�����l����Q��+��|$?I�9���|��:����fbF�EY_�������]�K���ww4����b&F���-v*��'��k��o�c��m����bR(�R�2��"w�+e]�D��T��/�#r� ���]��BSca�.�<t	���g�X���85��b��T�<�y+�l�)�K��)w1��~����q�t���GaQjb9]�K�t	
]8��q��G���i�x�L���O����E���.����Ds�](�P��E��n�;�K ��c�Wt�hG;e&��PR�TfS��_5]�]�y�7�n�MM�yc�������������]������WN�E�q<�aj��Te�����9�Nd���W���K��=�V��(�J�����'��k�%>��S�z{s#�T����:��"w�
%pA�Q51���~N��������q�R��KhI�AK�.N�
V-��i1�����C�����Q��$p���h�_�U�F"��,J]l������������(��XyN����i5DGZL���������-��&���m�rT��D��n�;��/���d��u���8dx��q�Y��p�[�X������aL����T�j���;D~����6*j�x9�4�WF'�l��<u�[I�<�����:�:��
"���Y��	���~����\
Y��yU��+���C
'rO�������C���R�������������Lt��
���"j)'��U;l�A��]��K�r@�g�����0�s����y���B���2��
������/?�0c�r�k��0����FyX9��Z]�������*Z�?��BwB�9���t���{72�p�H��������z�{e'�<.m�xx�gN��@��r�'�D���|��NH|"���j�l��R��D-g��	n"�(�yfK����8���<r{��h���>�M����~Z(Dm.q�hgUD��������o�8!�1E@���r�z�6"�9jrS,'rS��&�y\�&DgZkn��m���*���E��t���C�>�j��g�dh�t�]L����q�Z1�S:��3�0�Q�-E��*���2���5MP	�|���:W��HhR�� �-��Z]��������"`��s�gT��P��s��,g���-=ZCS,'rO�I���P �V����[���F>�6��"w�1J�A���ZYC�
�|��DB���&���'�C.��3 
0��G��!K+�@�-��}8�AT($�Vi��G���.|�[�GD�.�C�(�I�E��E�G�������|:����
;-g�!ao�4��h��D�	>�]>����E�'N�l����{e��En=�@�[zp�8��z2������)94��1�OA��q��E���"`�,)����Fp���Ll-�g4j~H�Z�	�1�4�=������:��|B#���v^�j� �e�����(�����-���k����G���b9��B�5�h�K��������rB��j�.r�q��������fk��{���'k,$��?�d_�p��8��c Z�|+tK8���E�%1�#D��N�l���j���V��8O��Y#|�V�w������	����
��j9��f�����IS,'rO�	�lx������7>��Z�-,�D���|��B
�)�j�sG&��v�4�N�-�����*k3� 0�Q�G
�����6 �B��D����9��'{]#sL����h��GQ�����F�������p���)�A���X�W�8�����(���C-'rS�����qi3�dt�?q�|�W���m��E�>��&
og��q�O��za6)f�`Y���^�mr��r�C�9���<2`"�Qk�@I��a��sh� UEb��+h����F=#�	I#�.E]]l�QRR����/Z]�y
����'�>]�c�n��2y�T�"����N����.*u��`
Ee��}��y��	�(].r�!c���m�3]B6�����f[��)�)�7Z�`�D����#��'�/5�)M��Z��y��3G"�h6���d���=��-a�����F�.f�<�������.�7!�?#��� i��>���:����D��M1��=�'^T���4|���P��7�c,&�1��2��"7@����)l��Ln�=���9�J����L���yb�7��h
�y�fKk��X���n��������V���o.�����$���V�g�9��
|�W��2���<��	�xtLGk��[F���o<�����r�lw��Dn
=���>.mP��6��w��PG;e2�E��t�;�OHR�K�b>H��M��r�1�����:�-C`�|������l`�
��Z3Pn{�lw����s�m���9���m��Rj ju1�oT����J��h��/_6:�	���c6@q��\�f��d�E.�x�Q9�)��)�\�p{\�&P�-���|��*����Lt���'���,�S���(�/S|�/$�>���64�y��v@k`��5�Oh
\����5��(��-�pXa*R����y�'_"��V���
�D��� b�j��b��p;���v�|�U��Y�`�����"z?�r"���p�d�����76~����
��E����Mt�����y�s^$��v�jOu��I����ssa�$;�5����GZTX�j�Ra����8.���N2�]G%j9"=��F�����Z]���9�'�p�0�U��)2Vd/��	�=��5 �T�i9cI���=z}?�r"����{��	6S�:�;}Kk�/p/Z��N��D��B72A������_���������X�!���3�5����GZ���|�����6G��Qb�����!����b�*�jk��4�||�CF�#ZK=\Wam���u���Weo�\��0�k�O���M���������!>q��Me���SD��t�[��
�]�]���v7��un�!{��>�Y�����i���=���n�
�zK_�1Z�oA��4���Rw
����r�S������
��D�a]��"}��?�������
@�Nq��xY����
~P�d�XN���.�|6�H��gN����cb���E��t�;�OJz������T8��q�S�b���!+�>d��h������Gl`�����Yc2�p���q���'t�v����)�����
#��9�)�j��jr����D'�OGl�^�o�\��vc^ �\:?�r"��xMo���F���_��m�-j������Mt��
������,�n0��]���9
CJ�7E���}���������Gl`��:[r����6�,E���>�9
��I���AY����.��2�|0T#��|�G��|��t��A�l.��r��B�%&P�	;L���M������K�L�O��M��b(��Mt��
�����Rp��ZV�r���E���\�Er����n��K�����Gl`�
p��*^������^h��$�/���+u�i��l��&�9hE�.�n�v����e�h]��e>�>=��N*�}�3�;H&�dG�����Dn
>��6������M��R���t�&:]���P����z_;%CD 4�vC�S�F|�W�6�4���v�m`��6�[n6b������q����Gg��~D>�_��a��g\�$���8f3,8��C0�,Z�Qu����_ ����
�Y�~���q���Q������|.�������X����Rk����E��t�;�H��2k�%TSg?���D$��c���Zj��7���
�y�vKo����:�O�C>Q�����12�����CB��d�Q��P2C�.��u>��'�a�h�k����|�=a����
�S�Ky��?��P��<q��Dn
>E>[z��*e�WN�����4#/�D���|7I�[��V�Ni<�3;M��
�\��:;��~����;���=�7�@��Z�.�N�!���(�j7�QOop���g�n|3�[Q�����;m�+�*�V�7���:z��*�}�+�(jT�Y�m��O�62�v����qi����6�s��	
�j�&:]����2��Y��r=����()odv��.�1C������y�	�A�Vp@9�����~�ZB��\����M�v#��g!v���?�h�����fg/ZKb���|������gM�g9�f�B���q.�S,'rS�����qi�`����������K*h�&:]��C3evv��x`j^`�
oKQ�$b�G�Rk��1���;���mrn�_q�C���D�/Cn�7Em�����p<�v��S������0�|��k-�|��&}�SwB��u�qD�}��S��B����{GN�'�yM�b��� ��V���|��{�2�tQ&�\���P3 ��!�PsG��yq��C,�������=��yu�nPg����vO����T��<���J��/%?.w�n������\[,��b�������B�V�f��!IU��>�������{��r�r��k��BV��XN���s�������'fU�:{�osn������h�.r�!����Y�U�';��e8������}#���O��s�Y�������u�Z���r�c|�
"i����	������#�(��V;{8���&�j�(w�r���N>/��v���=(u�6*��Y�4A�Q����M�uH�f��r���4;�:�m��En>�w��3a������YG7�7H'q��K!<DR�^
 �>�:kw^��{����0��Op������B�V��"r�.|�@&���c����!�( �n�Ek��_VmO����:kF�Y��SEi�q��0�r"����n�:��k�WN�6�d��9�.m��En>��\E'�p�%M93�6�PY�KZ�����
��������Wg����v�*�v��;����h�����K��I�&�.���V��I��W!��aZ������������YG4��g9C�|L���:���!���=���	��K��`L����[����>�Mt��
��b�����p����8��(@,�����z���3
|hPgM��Y�c���h��v����q�M�A(���Q5����I�O�k�����CJ�V��G��V�%���NX��w>'D>��Yc|��y��J����l�
q��Dn�=t	�<�l|����|�&�� �(].r����s�R�A�4�*�v�[�h��un����[���&0�A�5�WdM�"kT���Y[��!d�2�a>R%���0.|����^�������.��el�'�u��D�4U�qQ�cN(���������r�Y��c������j9��B�5��K�����+�?�����D��t�;�F����]��S�����e��*�0�P�M����������"kz,���+���p�J{i� ���T(vZ�c	���OeLmU��0i8=��sapM�j�e���"{sn'�YSWd�-'�=���RL%����XN���sQ����&���'N�^�+L�6��"w���Irn�����Gq���0c��q�y��4#G������Xlk�A�qx�c�g�����s�����P���cM(�IUm.��	�1��B����u��'t�P7F����	�������GE�z�����{�5��+�_[�O��m�I�U�(].r�IVR�:�`�Qbn����t�F�[�pe��9��j�<�rt�9���C5�mL}��Z��tT��1�,V�I����g��C�����b^�>B_X���U��cxOE����@fM���B���K�"��d|�G�O1��=���J����2G�Mt�|����p���,�D���y���������i���	7����^��������<V������rh!l�,��n�"����A��S����38�IE���6�{��f�����EkU;� ��_`�u��oz���Y��{T?��XN�'�yM�B�f�\$��O��M����}t�&:]��"NSn{F��Y���h>��;:��=%�b�1� �|�t�=�����5sf=�h�����b�����L�0v��-5A�������Fn�7]�����d��!��L����~���*�}���9LQ60����{.�}\�&`�V���|����K��Lt��
���Y"�����j5���\2�c�Bcq|oa��8���t�=��fL��-�a�i��?����ta�>d���q)�En��.~e
�dU������p]c��:��`����,��aD�5��GO1��M���2�-���P�����J���Lt����f�$�0��Q�p���C�
��)��|�%iZW)f�LC���Gi@O(
�k�����
7��v��<X�U�(
&�<������V;;j�'���$[�����s��Qd�QP����r	t}��"��m�����)�\�l
l��K��m��Ei.p�2��"7@�QL�NR������#����+U��)�jKiy�
��<>z�g����h)��y0-����q���C��� �|d����-���7]�zq��!�y�?y��3*�}��>W����b�C
'rS��ff���6��:����yx�P����D���y��F�$1��I'Z�NH�9eK�����$���������?���o�r��m3��cg�
�\��f �1�O���R;_��bS1r������h��.�#>��>�����R������>��k��[v��.�ZN���sMK�������v����-2��w���&:]���s8���o�y	V�#���:+n��0�g�g���3���q�4���7��a��
h)�*�|�!��S&Z�'�Z]���n�GG�k���^�:{Ym�K}�g����re�S� ]����{���y\�B!���o*~=��/�D���y���	�`W\
����:�cEK��q%�Pn�����2���N�������qG���x{5 ?���l������E�.NCL�s��p[��D����=)����;2\�h�g������������y.*�~\��<&7�����q��v�(].r������
t���FTV��C���d�}�Y�>�c2�&���2$��}c����z,�5�
|����'��'��7Z8���Nq��aB���EN���>��Hd�U�	C{|�d�9B�i��r���U�lT�4�p"7E�k��W6�C}�o�|�E�� OQ&�\��������GG~�������
��=��rnEv"{�0����#��&����l����	��<�6�u-=:N�����/����v$#�K
�7�	��W�Q���bF�����"{,�%XG92�����N���sM����F����w>���Va��Z��.��d�I��B���B0�)�@_��#$%W�y�;d��yL�<&�e2����{�)�cB���0�m�m;$M����aJ�4���8�1ou@g16�Ek�pY3i:���wL�ry��La�MY�lm��ZN���s�E���������L����-E�Mt������S!p�x��_���<�P�
��
�\l�c��q=~�d��c2��L>|�]�2��3
{hF1�5%�~h�!s��&�������od�1%2`B�N�pR���!
�"{�N�'�����){o��W�Ic	M1��M�'\�=&>��P���6�1Jc���Lt��
���q��=�d����4r�c^Z��x{\�Z��C�!���0������ ��]��2j�9F��|���\Wd`�����S����s��7�k�G���U��)]G�	����vmJ�[-g��HU�2W�b9��B�5
��K�a�7�����t��<*�6��"7�����m��p�6fe���6������R��s������� �V�Q{�+n�+/�lxZo����CX ����������am5Z��@�U����=���:2��l�z�c9���`oJ�JB�p��Dn
>����P@����Yq�,�3m��E�>���$�	N�L�~��4����{����[��A��g�g���3��|�?��hEn����Hc�k��%p!S�g�g�I97�ao�9*ju����������Ek�;���|��.��3��r�O�w�^'L
�c8����5��=>�={���b:�{Q&�\���h+�@@�[�u~y<����|�*��g�i���0�3���G>��q+5�l��m�#���IBVi�=.����z�PK��E��8T�'��j�
t�j!�������<��3@���{��������U��&i9����5�	=�K��'�>�mN��=�����Mt��
���6ko��YQ=�e���dcp�0�[���������� ��g�|�\���R���l���`CN�gU�Ii���YE�0��8�Q��G"��>���������B�g�<-d:o����,,g1s��)��'�\�����1���12_9�I�[P!J�[�&:]������J�
T�q~�2��h������#������������ ��i���)��RF�4�,%�B_m����n?'�q��C9���8�}P��q�_�������	�n�~}���>�Q�za�e?r�P���}�5��!5��'N��o����Mt���l�@��`�r�b���a�p�!/�y��������mb�[���h
�Z�m��g�D7���	����h��x���	�J���Vb�D����h�E�#ZM$sY����t����N���{{C�T�b9�{�O����qi�����;��q|���A��N����@Y/��>u�����Z7t��������q�0
|��<f���l`@����~��'��ln������cJ����Y�������:q��3H����!��;���a��c6�����Y���*�I�;��pf��D�	>��>.m�%U�`{�o��� �Vm��En>.1�8�����K�M�m=&t��f��&	�	�9
]��L�����
�#��K�sZ��tJ�
�����&�'t�h�9�:|�Rq jcN�f>|�P��Q�V�r:��	>������MH��n94���8�l�b}��D�	>����-��M-�����8�m4��k�&:]��C^c���To����i�v��4\��p<mdi�fb���� �Gn�
���*�����)�?����W_#�a|�b�UJK�$�����1�B9|K�0E}�Z�\�|�i�Fn��*�}�3D|�+���#-'rO��hZ���6 ����_9�)��2�q�&:]���[ZL�\����Dm����~\.���O7�'������ ��o��t-q��_��A�3�'�j��;u����A.���Z]����[���v���\XppF���70�(o�Y�����T�R+�>�r"�����6��������������t�&:]����O
��t�@��p��<��`lV�$$��V�Vp�����0�7����&h�],�Z�;6i^��
�\J�0��uaV�jJ��f]��0���	�r""��Q����������7��t���������|u�:�b9����Ew>~��j$m������2Ia�&:]��C�,�}��X
2�Scz>�(W
���q"_�G��i��A8�� <tA�d9�Q�E>H����[����A�:�<�Y�xE�\��6�������^�Z]J�:Z�x����$_��7[���Wr8�v>�r"7E
�l�
BL*��?)u�p�m��En�<��]�P�=	m�.�����S�70��Ti�|�����A<�� �-� �����11�S���E���n��X��(d�RY�����E�hW�H_�Z���&7��AD����!v��|r��������y�5>������$�s�6���T���D���y�o�����*>2��'X�.5��������+��r�x�A4���������R>:�s��K� �g��\�P��Ehu��7�B�f�r�Tk�Z��!�?"{������1�YL���r��Vc�!�����=�'^Ce���M@�"�'N�����'�U��t�; OB�ta������vx��C	���H����:P%�C��A<�� �'�6R.����hd8�u.��#F*����#���=OH�$�&j����">�)[�����"��'��j;V�h�[��r���t�SJ���b9����5��<V�,7�uN�QH��m��En�<zJ�

�+}�9��:�O�*��m�:�2k#����p�V�x�At��'�n��R��=f*3�t�mfR�A����Hj�/&�&(�)�e��h�������<'4�4��Q�h�>��_i�SK�����'��k�y�5����WN�"��|<+�#�D���y�,�!K(1��_[N!���z���r}`1��Va�'V��A<�� �6�	��b;��2��"����g�=aZ�e/j���v�dpl#^�V)z���_ �h��<%��>�e�K��!���L1��M������em�C���m��Q�iS�2��"7�2|�[�vqp�3��NnxW�0c��&�����<Si���� �Gf��1u�x�?L���
�a�[�Y���K�
��	:a2��Wit��o���E�� �\�j������0���}�3Fs�f*��*�gXN���sU�m�d�,sR�9}[�P�X�h�.r�A����-p�X�y�%������� �/��d�!U��0���b� ����h&�~������H�W��X�4�����E���vX��Bj*��E��8�;.�gN@�Fc��MPq��raz�Q����b8���5�������P����O��)���E��r�������=lV���)g��m�CHe�#���O5�����`g�a��0�q;�(��A���<���+�61�x=kX���aqEm��<�|Avn��!�(��7��z�F`P���>�qT�'74�D��Z=�p"���pQ���/��T��3��E��#2^��.��`dL,��)����z��v�@J ��e0�n��`/8d�v��<���6�cRklcu�wz��<���t3J���zO�w"G��d�����c��e���_�z��U��?v:�T�'��r�D�"�������b9�{��E
�����C�;}[L����r/�D���yWbZJ����""��J�R|��a��J������y�qK]�[������C�&��K��cK����'1���u�.u�oH����1Z
���}�@����6��X
��>��gB�����!O���=���)=����2��{�oH�^�\m��En�<.c��P�����mHixt�`1�jk�Q��<^�y�mi@]���.H[�o�N7BmA�����x��(g:�~�'Q0�,%��V��8$�U�+�f��,]v��O�
M=u�J]�V����)�\�0��N���=�'�K��q]��k�F=�9��pRL:+ �&:]���0#S�Gs����eL]@hC��dL$�Ai%���t^�-
��y�iK]�z��D�q��a��0b�:�V[P^3	y��@*j�/
/y08�t���V��n�����K�dz�a��}�3|�Ux��[w,�GZN���s�-����x���o�x��RR�6��"w@��\�C,1H�+?�g��1�.�5>�!$�P��mK��tuAzB]`I�6�4���'�9���M*�l��k�xD�!o�~�y����1O�(��������y\:yz�\��}�3��d?�����D=�r"7E}
�l����W6�o���.��E��t� Y���X\�v��lt�������sH�]oR�@j'f���<����.QhN=������Q��*�Va��Y��5g$j����$���p���^l�l2O<#�����Y����A�����BR9����y.��y\� M�*�'N� ����D���y\i���bp�G��cR���j�f�?��4
��y�iK]�%��1�p�������;��<g )e�Z��Z�KqXa������UkKI�z8B��tX*�n9^O�0b��N���V|��Dn�<�T<�k����N�f�2F��[o�&:]��C�a���D��������K�<o��������UUM�y��<����������G�:���P�)�.�v����<�>�������i4����1��<��b����	U��c/@K`��,g,�9��G���9��)�\�����9|I}Ac��m�A���LM_��N��>�b3Q(�}5u������+wj���(l��!��� ��^���h��y��!
�oA
�A�{���
?k,�
�_����~<�tTY�yDk���]������D^����~V��Pa�OE�d9��"�5cy�5�n��������1I�\�&:]���x�T^�#�F�7j<�q�(���lhV���Za�&2���A:�� m	�5��c����1�Em�7u�0H)M�����RU-j�o����d/=�G���uU�����#0�����[�����������ZN���sS���6�e���r�v��(W���h�.r�q	UVe��F������i�'��"�6��9<G@�� ��� ��a�����6.��W-��CP����cH����Ik�����b������XB�Uk
��<��I_�����rx�Py��J
����=���{�
��y���������2S��N������7	e�����+���*���M��j?S<@��A:�� m9��T�y8���=�Q�:Pp����g�@8"t%�l��V��<�0�S�/u'�V�B�'�y�	,���&�EW���n9��W���F�)��)��k�g�a�������[����m[��N�����K�-�<�Li�ca\a�q�J����m ���:I��� ��a�����1O������a�����1�;���]h�+��-��m@�2[q���P�sU'�	1O�8�N��,gXT*�f��A�b9��"�5U����\>�����{���)C�m��En�<6���c(��Y�b�c�����+�e��f7�TOD��A>�� o9�������^�5�����j�����C�8��.j�oJ��p	�;��.Zk���e��	�r�a��]&���rF3R�)��|=��H���y.��y\�&��}������A����Lt��
p��yN���{U��Pz%���Z�RE�8I�A���y��<`0��1�-��VQ�Gj��V��6�geWb��s�p'�L�Y��_� �"r
������A:d����	'�������g9�1���Qi����w�5������J���������vR�R��N��$�%�
��}��EN���2QX�,Z"�h��gF<�|�A�2�|��g��C�B!��[	��d����v��;G��_���c������h�G���;#��[��1`$��}��3f1�>�y7�p"7�k�B�5
-kO|�	x��|�(].r��)� R�f'����n�{f�C_z��y��Bu8"����� �G_���dyjem9�W�x0/��P[y�D�������`).����x��k�����{�p��ZyrG_@|�
f�����8�4jy�C
'rS�����qYs��C8=����#���Lt��
`�WTZ8�
o�j�W������������p��isyx�;��|wA�r�����W���#%lj�c�!e�[�	��Y���Z����)�'�r�	^���@�}w��;�� ����>�rh^/����M���=q����G��sP+��WN����!��)e�6��"w@��c��wC�Zg�6v\Y`y)za�!��~i6�g�]���.�[�N�W�%�����.��^	J���<�V�E�6��!�X����\��;����3nxz��hK���g��hc'����&�H���y.�.x\�&�Y�?s���
��%%�h�.r�q�2��4.�CZ
q�]`|R-��Ly�5���x���y�y�]@�G���e.���#������9&R|:��B^8�c�Yd��!_���,Z=��C&���M��QM�q�G�`��r��<o�@��c�b8����5�<�tU;8t>��;~	��e��E�;�a�8���|���(�����SjI[H��5�3 .���-q�++�7`G�Y�o@T�6�v�!%�Q���N!~)j�������/�v��e�u4����<�#.V���r)eep#��q�N1��Ma�_;�^�]c�����&.�D���v(�
'�yN���=�q=�F	W�e$�T�����g���X����q������x�����J���g�D���+���Z�����}(���X�z�s���<O����������\.�%��H���N���.�vI"�A���7�l|����X��.��@�\�X��C$6�����	�����z�Jx$<���Y����
xP���g8�y6�������%�9���I�C�d������(�����4�U�M	����ej;���s/��v�>����CY�I����<SL'rO�I��mV��!M����]<	��%)+�D���~F��.�'�j����
����Le��p���MF0z@\��8~��u����<�k��Q�C�v�\�y��Q�Ng�k��6��p�/;\�,Z;����`��t�E'OR.�3��1��?!�(�M1��=�'\�q��l����k:�o��9>��(Y��^��^'N�08��v4)��D�G�XZ//5�`{��~������	����w`�{%�Z�g�����i�bM��_������u2�j��j��2�9���s�	dk/��Mg��OR�p�[�SL'rO����`��M��+4�����-*�V��M������v)5�@��������! �Y*���MQc�nf�3 1�_�?[�K��73N�Y��bbe���td�fV���PU��Z�}�j�����U���;$I(�7�vByu�^�^��f��R�m��"X�&YN���sQc�fa���O�]���-s���,X��>���^�e2\��� u�d�8��I����O�]��3���x�a�yT�^[�1�'
��U�P_:=���
����K��D+�����a���|R��?.Z+����x:{:2�:��t�c�#���h �4��=����gKg�M�v^��>"�;/�D���~�TM#<:���Ly��+U�6"���G##�auH3������g�i@�&�
~r~���������iA{F����2�L��b3},)�YT�VW&A\5����e�RV��v�e��W�����b9�{�O�fN�fa3���#�'N�r��
I�(}.r�a�Q^n}&��|2��Rk�G� �����-��n�����q�lY
�+�-����`���2��8���U[sD4��G�IWZU�Z]l�0���,���u�r�5���[���CJ�d��t����Ka	�w�v��D�	>��lV6�O����+�ou�D�����1eM������
�2]���	��Co��`
��u@�'���0#��/���-��+��-��WIu
�A��M�e���[�D���Q��}��������y��R�'�~�)�O�m�rXn}�n:�~$�E��i���N���sU����E�m&z��-���]��X��^����qLm����Z����s�l,F`����\�fF?���y���8��B������z���dU��'��p����*J�����axs��*���j�m[�+�7�v��O�q���!�3�!�a=eM�m��~��D�	?�.��
����U�^�F?�����M����{�!�����jud�S��Ft���\!��%3r��f����������<�y���6���o"�3�Zp`n�v����Vf��>d���%4?����F�w�����T,��"��H�
=���5�1�)�n��!���v��{����
hM�l6������������\�=��6�Qd�D��r���9��J�.2	Uce�y����6�*y��	\-U�����7�����5��c@���nz��|�q�
BJ������4�C��o~������V�����/>}�]}l�o��k����l�c{;G#��B��u����e��p���V������S�d�e��	�������fm��@��
���_�!C�YA���������C,������w~|�����,�yrG�C�Aj�6�f�.�����d�|������v,�s"�3�^](����`%�.t�<�J"���/�?�����e�m>��������o���������	o�O���0������w/��_��xs���	d3�<Pj��]����l��p[��U����>}d29��hq�����9��������#����Q	����A�5q�5�?~����_^��M	�~����������c���/
:�9�q6��n�r�������PK3&�wH��PK{2K18sept/pg_part_patch/5_4.outUX���Y���Y��}k�%�m�w���fi��x	+�{wmCk�X�'�=S������e���	fFd�����32����p��NU�yHy��r�������/���_��9���G?�����o��������������@1����J-����������_���7?��'/?~����}������?7d�MYi�?����c�|����'?bS��O_^������-���Z���gX~���=E�w�_��������?��7�?��������6��_�������_������^_���������~���o_�^~����_�����?|����������o�w�������|���O���7�����������O�3X��?k�U����w������������~��_z�����C9��
�����
�
��J�*���|�oJ��b��>���A}���w��>~~��~��7���w����?oPYO���l�gl?pL�����X��|����O�����w>��������>|��'_�_���>}~�?`�5T~U#�j��~��y�(������
)~���E����7z8�_������M?������/��~	C|��<_�W��t���M��H;�o�u�O�������~y�(�o�<s���7����������l"k���u����)*Mo�Pt����b��U���>"������W]�������~��o/?�����O����������~���G~���~%V^}}�����g����9{RF���j9&���/�R���8��`��lt*x����Q!�H6Q��m��?~��_�z��5F��p���A~��w�*�5�q���@E����L��!RN�TM����0�����p:��U�vzUP0����1������'N>:>4=�!y����i	?m�V��!��5��������G�xT���s�?�O|�	
�}���7/���//����/?|���_��)�7������c9G|\��7q��&
!�/Dk0\u��c�d9�`_���c��9������-���o�Y�?�������~������������`�?y��2��_���w�~��cX�oy��~���������o8p�.n�R5]��l���r&'x3�npq��D������o+ZL��k�/���?��I?�����]����������|���?+��_�_�������SF�>�_���?~x};p<���?��#���^^�v�y�X6�__��p������_��W����G�F~���WoD����|H��N������{��Hi%}o8h����{�f8���8��p+�|��a��������>�G`l7�&�!Fc�5��l��e{�U�7h��c�����6s���
�X������2��U�c�	�c���iZ�g9���d�/�<y��4�r"7{
���p�e�������|���L�@���|,���xz������E���Z.����&h>��������;v�c�����c�����N��9�#�d~Fx�w|
C�(��GO���������D�������U��5�h���A
��O@���c��Y�%M�Rz��Ki��D��1�7���������([�P7)��-rtq��v(���*g
`�w]t1^%tA���t���40����y�B���{��41�S~*M�Q�����.��Av���,Z�k��d�3������C~B�=��9]����
���-fb�O ���0�0�b"�D
��cc��q��U�9�.�(��-rt�$���%�Tf���G�����F���7�8t�t����_����a���6�+9"��-w	�D���9�?`)�m�+g���E'����(�'t���.eW���t
��
�]�����{,f
��<���!�'��k����c|,��^�.F��35)��-rt����o����j���6w��1��0�Tx�������p��ui���ZK1uKc����!��k-(����$/�r#��X[����1�^JcE���tQi,��%4x�N���b����V;�"?�b"��'�^�#��;�E������au�DQ��[���UJc��B���i���E��n��c��%v�%�/q���*����7���g/�&��y�����y6����h5�������zRja��
'�����;����Sm�X���X���<�b"���p
��GxqN�d��{��K�!"�jQ��[��B����`���|�^?G�������:�K�kc��.�<tI����7_�E���X���d�z�n��9��{#yI�t������{�������aR�q;��M���%-n^8'�{,f;O��������'��k�%=����$Gs�]��FD���s����gN�C�^�Vo'cF*�yY���P*�1�#�?���������|�����K5�f�	&�s������a�q�L����A�2:apK�����J�����[��O�Z�D:��'��e9�'7�	�`��o��1�P`��b|����v��w�#�����
���I
�Z��
t��
���1j�*�VGm���Nk��G�%0p���������������,�1z=/�m� ���y��[��w9D���:/�/�1YN�F��
�����u����(��Z-Csj{��ji��Xi�l�g9��d���J��XN����a��+;���h���*��N_����r����
t��
��1:**�Q�����I�Kl��kXS�5A��S���po�t��yLf����$9���
����[���G�A8����Q���{��Q�J�Z-R[���6��2�`
��v[�X�)��S.e�|b9����5����F��i�K�����j����N�����m*M�_�
|����a� @���4�3tE]��|JG }|:L�<��f
��Q���XF�>�8�H�C���it|�Uv3�h����V����q�"�xg�=yD�S��e7��r�)X���c9�{�O�����)e7�6����������N����9l�4��+���qN_f��Q�X�|�9���>�s_�y�����^��b�����9NA�|���1��u�SZ�������dvgo@i`���h5�c��|�>#�Y��-��m���K��V>����|��|9B���N_�(�-|D�t��9���i��q?:w�l��Y�(q1��?R-�i;0���	�����N q�3���O�1]ld>&�N�GH��G�CF�6�'x���D�.�P'��WjD%�VC��/#��g���[ �<����rYJ��1��;�p"7��k(�O�	1�:���W��`
y����.��8�e�������g�N�h>^�oz�G���<��#(�:���0�Q
�5�@�N����jZyi�+G��
���f���T{�e�$x/juq��<.��2i�w����
(��<�[��[-�g
���,q��0����=�'\�m�� �i�������3���6��"w~�)��8�Q1�[:J	[<�$�7��T��[ ;���t��yD�����Ck*{jn������Z��o[a>��n]�D;O?�Z]���-�[��5�����p���)�V�\�	��\���@�lj9����5�=�G��'[�������y�P��D�t�;���F�(��l�::�}�]8�*L��v~R�T�Um���6����h
�#M�5/	,5��mP���LM���i�r/�!�Q[T8!�`��������=bJ"3][�Z=�y����'�>����n�����E�[����b9�{�O���mE]�e�(�N_]Bp��P]���.���1�uO�6��.!�'E����O��V�����Y=z:�<��a��j�8�)s���G'��6��e�E��Hix#��XY_D�.����������f��&�G��	z�E����*�}��h��=���	q��>�r"���xQ���� pQY��N_��p���X�h�.r�qDV9���}���������P?�bk����k����Cn`�#70kr��]��C���.�p��A�f�|]_P�K�=9X�T����V1�.	����*�9�JOdo��6� 7���e��c��X��r �X���M��F���
�����-��Qtc��r�D�t�;��j�����b�n�M�JK4m)��TG|�14F��c;��<r�Hn`�(�������ygP��lg�>aIn`D���	zn7������n�()�Z�V��e��3��Cm���/����M(�L�s��B�C,'rS�����x�
O�*�K���Kq7�|�h�.r��$�<�4*G�����,������Oau������
����m��#w
��\H��D��M������l�e�rQ��[_�
{tN�Q�Z-#�!��"{�n'\���h�<������eq����Gb9�{bO�fK���F������cxb�(].r��Di��!�F�	���������k@ ��uupa�J�a6��1�GftY�j��e���cHcW'��&������G[���!f2Q��s����
A�2�?i�4�C����%=���.�
(�c��r��*�����)Z�i8�{"�E�=�'�p�Om������
�����.��D��B7�"s��O��x[n{,����M���G�����h
�#�
n-����E�Uoh�����y�&�H�������m��Ry�%�PD�Z+;\�bm�(�-h
�����n9�z��W���%=�p"7E�k.{O6G�dT�]�|M���'��t��-����_���'��~�[��������^/r=����4��q�5��}9Y��#��	��4����R�(���u����LW=Emg���[�W;�,>5k�C��="{���	���4�VD��,����0��8�r"���pQ��"5������W����)�
t���'%=�X{�[�g������p��&�\�}���������h
��� �6�����8	V�xCs��kQ����]q�Z]��Bw�O"e�
k��iB��UMnt�,h
��S�����`���p�X/�
n�N����,}<�H{rc�[�|M��>����(].r�!�G�B��C���v�
�,���s�.�i�d0����4��q�5���V�C�@��c�����a��Kk=x8'UYV��:������R���Ne/�3����� �����\&�,�&
�
1��M���}�����G�����������}R�\��cs���c���H���~w[�D4k�+:�U
~Qo7Tj;|�<>��g��W��M��6gd0"S�n����_�tV�A��1�Sg�z��^����9�-)�bH��O�%��N*�}�+wy&�/�xNo5
���M���r����`1�������I�m�:]�����d����s �wP�O���_K	�<{���R�!4���5���������q��b�����Cb$=X3R���8
Fu�p��wU�u���T��-�������q�>N�U	�vX�h~\Y���h9�{��Dn�=M���\�m�����c��H���t�����A�Z���Y:��}��+��~>L�������v��ylv�f1��Z�b�B4[�� Y�9��d��7����y"a�Z=��e������&��9}�L��Y����g���8`�l�D�%=�N���sQ���2��Rd}���=G(��2��"7�K���-r���������q�\����Qn�e��	n���a2p�1�
&k[c5��0��#�vNet`$H/�C����E�f9�(��Z=�����8���q�j���{����Y��>�%L�a����������	+�4�]�W�x��c��
��Nu�����t��
��,������\�:�B7p�h�&W�d���kK�!������X��,����~���y�{L^���A�m��oa����aN�����,���Z+\���y����c��m�;,���G��b��b8���5�m��>
�����>k�(].r�!�=��!�4��%����R�46l�
�����P���q$�Cb��#1pk���rW\�P�>������*q��8�B�����/O�'���b�By"e�a0k�4�����jmnAb�:�Y�`{�F;1t}�ap��D�=�d/��X"^sy��k�6���d��6��"7W������x�1�0��;���l]z�u��<��k�v��jw^[��h�f�'s|H����e��p2�?�8����bRg;����9h�rYx&Z-^�C��"{/z�	����&B�q�����	�n�����=��������yO�H����-Q��2��"7�2N�[J.�kh�����g8-����������n��jw^c�[7V��U�Vp����HZ6X���~�E��W����i��c j�p����3Vn�Y�q�����Ec5�Q	���r�������N������U_uH������u�/���t��
����������rXW�����xtjDc2�v�m=��c�q��jw^W�{��������{��^qD���N�g���jMn�M#�Iz����"��{�����"��v��e-��E=n�U��c�>�%���p���~��D�	<W���M�����W�$g;�{�&e��En<��\@'�H��wF�L�����UYC�5��������T��k�v�M�(�Y��M;�;��qq�(pp����
U�!G�3�����-5[��B�
��z���l�����5UG
�H�g�0?��Fy�N���s�v���m@ �Gt��������KT��.��8�A�i��8�k�r�I�C>b���w��P�H##�0��NS5��TM�M�Vc���-:���<�	Y5������#�nY5a�m��u�pH�;��uS(C'�����yW���<'d<�h��~���v�!��9�/��|�
1��M��.����O���
��;V9@S��t������S�B-X]*3��?�O��<��d�|�u�����������z�i�S��5������~w�$��a�I��b?O������Q�s��"����C�=';�E�LTXqQ�cNh��ES��5�y��'��)eq���w��Dn
<�4<�l�6�/|�������t���'�k���p���L����}�1���c�r&
�I>��<�n��t�����jzl��sJ`=9o���W;�$�_�<���|y�
���U���.�$�X�����C�9�����@TM����m���X.�_	�S1�!��)�\���z��s�E��|���I������t���'R���j�vJ�����I-��wm}���8�P�l���6��z3\D�jwA����������"����
	hA����y�?�<�d�oV�v��us�'L��b10Q|��r��j�:+�
1��=�']�]�x�9�E�H���
xlTAG��2��"w��	%Eik���R%����������,�B������#��b8:o1�����$��?G��4�
e�U��Or���oO?m$-��a>�y"�h��t.Z�?<��Td/Y�	D����Sa�����q�HC'rO������`���.V4/|��ER�LI�2��"7��M�����l�lr��F�h��\�b?�A��Pg)���6��awT[
g����]��C3�)���>�}��2�"j�p[����QE@����V�CR����;�2��>�������%~o�!��#�<�����	��;�6|�^���}>)].r���)�<	n�Q]����<(n���S�#�<���O����#/�G��=g�s��z"6I���b�KA��?����%�(�42����������W1�X���Z����d�!��,��K��R8U��,�]����b���C'rO���,��d��W�7|�G#�I�r� ���v1�P|T�^�$�%#05	6+���]�l�n�6R�p�y������g[������+�g4G��bC,��<.���M����6�X�Q�V�VoY�u3�g����� f����\.�������1��M�����5y��m_Lz���T���M�@���yl6S�-���J���{wv�����<��v���x���<�:���6��3�Xs�[��sQ�2n�#��RC����W����.{Ab4�|���z���y��/X�P`��i9C6��)(6V�����z.JzV��#�F��p��a�_����O�@���|8M1eI!0���'gt{t���p��(�yS���8�6��y�A`�����1G�s�R��T��g��5�n�� ��c����z��Ac��q�Z�7��,��	>��>_0��}�+����<�*�<�r"7�k�>m���m+��N_�������
t��
����F�$���='Z��I��;@cg>t����x4���!1����5�/�jn&�n�c#f�P��6'��sL����\z��Z]l*L�����������:�.��N&�l�#��r&`��.9c������|��&}<���8�"���W��I�HN�%h�.r����������#/�j]"�;�X�nf�[�����t��yL~�.c�Q+�e�����;E�=
�Z/�$�C��M
��$�Z]�6���n,
I���z}��e��g�%�K&�Pa�����J�TH
����|���y<�������Kz���Y�t�;���N�;Na�]�jp�C������(�+9�.�xC��2���w��yd~�!.:~��v��M�]���ml�m���L����|��r�f��8u�/�k&j�j������t�_��wT��Y���F!��S�+�M��ZN���sQ����f��G�N�.���_�X��I�t�;���C%n U��]�O�nQYl�2��3�*2��<���w��y|~�g�bP�q�j@q�����1��\���p���J5!�8���B�~��������V�>dId���	{z���I�N�,���
l���
.���M���^���
�P�(�-����������h�.r�IZZI��c6���t������>�a�^��Xv�p��8
�#�6&�:�����]2G�"	��Js�T�|1W{��a��h��\���p���2���Yk�"�������|Aj@e���a	�BYZy��-��XN���sM����F��U��/�������I�t�;�O�B���Ln!�.	3G�YY��A�A%\��!kg:���5����5����m�������T��g:�e�C��[C>�Oz�D�.�������g�Z!\6^�Nh��b�����rS��U�1��1��'�\u��b6��r�F��M�v+�@���|�<�
�[��Tm�����e��Eh���C�F`m���=��m���6���>��G�,)�����G���-,.��u�Yv�L�#��cJ����#�e�����CF�E�6�0a��>�y���-���
J�'9r�a�XN��������-�]:}������q�:]��C��2��|2�������~���X�:�3��}�2�C��;���7����5�A�h���O�N�0�nF�v�
��[s>�*��Wu���������Y��)]�%�	���
�vmS�[-g8����Nw>�����|�2}<��b�[N_���	��
t��
����4m���l������30x�r��=cV%���)
��u(�y~Mqs��(�>�I�V�(o���4��C�!6�-^v��Z}�Q�n��h��E!��������� 8[���X�8�nB��p��b9���O�}V$�����n��3i�.r�����b@>1m,A?�2�s��)H���@��a\h�t(�y�����g���������-!�A3El�E�������QfZV���>3����'�����.}���u�������3���) S�N����������Z��������j�:]��M�b���=��|���/�:���v�e��'t(�y������u�:�����f��-���]���L�z~��;|D���@{�k��Z-$�C24��u��',(�kZ�f����p���wUb9�?��3�	>�G��'�e�����2���xT&m��En>��i���Z-�'S�)���������K�p��E|:�<����8X-���)�Y�\*I9>��Va�\����hf���E�.���1�1G�S�j(�������8H�&~��[������U�b9�{��E�>�G�kM�/�����n��X.}&m��En>.������smg�|yi6��/�_���~�u�P��(��� �����'S�CA��)SX{�P�-T��H��E�.����v	e�IkEl}������w>N���!4h���b{�b�����)��k�gEq@������AbW��G�:]��c-:iK�����2D���_7E�,��6�R�0��-t(�ya����:��������'�iF��H��Vz�������oBi���J����pj�2�3i���ee7F���8 �c;-�>�_�������|�5S��G�,-Kn��k���)�/�
t��
�D�z���wM������2��n��|��d�����O�� �Gq)�D����O�|"g>�e	{`�|WV��A�>�h��]����L�c����<�&���P���]�s����8�1���Y��e>�"n��}���=���)����%����p����`Pa-�@���|\b�xr>�4�^�l�'�n�eK�C��?P+���>��p�Ax�8��6���:%m�e���Q@�`m���@��8��C�l���6
%kz�CkV5n.D���NG�Ev��;��:,(B����-�G
���5Gu��Lj9�{�O���`Mq��p������e�:]��C�	��mr���!;'O��x�C�2��gM��p`��n��A8�� <RX�uk��Z������Q���M,��|�m�� ��2�*�iV��b���N�rbfQ���66.�pF��QD��
y���h�i�O`m���P���|.���x�
(u���N��8H�4i�.r�	�MS�W�:e>Qc!l|0�]����=�����t(�ya��`��w>����W%�\5�L����ak�d���*ju��]r7G�Vp��Zk�������������}�3����� p��!��'�\����8���N��|�0���@���|\�IyD��cY�5Y-����
����Z���M��g��i�0����4�o��J���4��!@Vc��?�7��
aT�zJ�Dk]�z0�������*�
r�j(��e��tF��d8��������5�
Xe� ?�r"7���|V�u�*h��9_,��ThE�t����Zko9�q��9d6b��P�e��������T�H�0��0�����K���a�n���0U:��"������u�EQFU����{������[��Q��1������A�5�y��J��:���2�
���M��_�<+z�������>k����:]������2g���*�������{0^�{X"Pj}�)��c;��<z������g�JN�y!�� 
#Sv(��� b��Lh-j���o�O9�g�
k�[s���S����ZG�� ���!��'��k&|�5#Bj���
�q�x��O�:]�����R2��F����^���P0�`
��>7m�5Y��A<�� ������6Y�'KL�3� a8*btfYms�C�=%�:�Z����n���	��&��+��#�yN�-�b�������Y������[4�i9�{"O�����\�WR�p������\�~'m��E��<	d�� O��![�v�z�5�3%��4I��q���0y:��<b�h7�m��%��y�� ^���w�=&��[�<�n�j������ ���&<5k������<�V��`5�����Y.{�a��:�p���P���x��^�Y��h�-"]�|����>,�2��"7�x!��t�L]�f�>�N������O1�R����b�� �Gi�:�IA�_�]��^�8�6M9�X��������������:�81�Z�Z�d=�wAd�%�	S=��`%��g9������q�fRc9�{�N�����\�����}���Hi���2R*�@���y8�7%����VPHO^���V��c���Jm~`{A�����"���������viD����	��s���g�%a�VfOj�/V���ISR������D!]���2��9��g9�j��mJ|D\pC,'rS������\��b��_8}��H�\.D�t� a`����F�^;q�N�K���_
T���R�P��!3���D��<��/������
��M���<�5�y���pU�u�F\���6i�$��Jm�8��<&��n�v���C�'�<����1��=���R��� p����.|�x�S&�lL�@���v���`�#s	m�'���jL,q�I�r0v�>���;�x�A+��1�D��hW�=���R����4hc����Z��m����oTDg�Fp�w\�.��;���_5A���C�@4���B@rC,'rS���:��\��S5a]:}��-���-��N���h\�����6�������<g8��*�< ���:4���A��A<�� ����K/8��6eh��O������<^�����iW\Q�G���5����NZ#B<!E�����O�B�[�}��!�Tb���^1��=�'\����@?���/|��-�Qew�2��"w�l����:����h���z�q].��C���nx��G���{A<�� ����o��Q��i��u6��n�8��R:d���
O�x�Mj�/���Y����q"Z�E|���'�`gA]�"���Y�`>�a���dL�1��'�\4@�x�M�:�-qK������K�"�@���y;���V���H�}��x����6�x�.�a�� O�� �G]����1Nx��B6����Y��gp��� ��X(������C�w�a��eE�h-�;�B��.�y�D���"���36{e�y����h��D��<m�y<��-_:}�[�/��}k�:]�����e�866�%R������@z"�?r�O�����:��<����.���v#��>������
fI��V�1���]��V�����`f'��6i�|`��K}WhZR���o��A����4p���XN����%��x�M��^kY8}���� ��D�t�; OJ`6�KLmT��9=�.pI�2�m����9O4��m�C]���.Hk����1����t��!���
�y����<z �����<���F�Y�������O}�%O2K�����Y�`����^�R�c9��"�5�<����U�N_Sp��>U5k�.r���:����j5�d��Y>3��y(���PWmK��tuA��.��r�L��Md/�qx���4'�#�-�aT�S�S�����y2����MZ-{��t�,�����l9P�p��l�q��i��Dn�<��Yq�}j�^���F��9��
t��
��P7+�����W�|{��;�(����{�C�{A:�� ��@��y������.���_ O��Hs|�5#Q�mw-�!>@���6�M�m��gT�{A@:�}�3d0���N����ZN���s�(���iRT�n8}�<�(h����N���28*��W��bp��|95��[|C�����{A:�� ��\B��!��#'�H0kR[��<f!)!;�i�������2������o�!-�t�:�n9�X���� ��1��)�\�a�x�Mpn��l����pYE�Xgm��En�<�v��mh������\�E+hT,��TI9�`l��9O�� ��^�6�\�iy�y�3���r�,�����b�����d�����������1��9���f����	]�i�^�}T)��:v&��������y���y<����eC���k���,�6�:]��L,��
��VS���� f��2�� �+��p�t�yi��@�6�����r�BQ�A�>W���~�Z���r[!j���ns�X�m��5km����K"{s�|�4�����}�+ky����]�Fi9��"�5ky�5G���/��Fp���6��"7@�oJ����������"���,��.-�'�_�n c[������w���<��g9Oal��4S{�����y����U-j�o�����r5�Dg�q���jwF�� 0�u�����r A� �Q�a����'�\��x�Mp����/���a�k.E�t�; �KI�B��Qk#��]�6�����<F+;��z�?�.u8�yi�a���@�!O��p����N�O��<��
n��`Hk��X�k���]�����xD�6�p]���@���=O��,g��*S�B#���XN���s�=����<����wU�����
t��
��b���@Bi5u ����W:�2���A��<tLpy:�<���0�l�=�����'�|�f)�X��-�#�TB��Mj�����<���TLZ=Jt%��N�1'�T7��"]��{��Y�+]r��4�r"7E�|
��8�"�����y��3i�.r��7^.]������J4<�0(�p��.(Z����&Is�� ��a�����9O��Y�AY��D1�������Mz�����C��
�h�8�f�~����m���'/8�N�a9l�b��$���AC,'rS������\cN1(6���;*W��&m��En�<6���c(����C1�.{N��f!	1&L��j[ =y:�<���0@���y��M��lZ��2����nCr�oSyNj�oJ��QQ�Q��6�m/���V$����v���v��Q�R��V2C,'rO������\�`�Wv��+�!��xT&m��En�<�#:�K�-�
�f��p`��0���-���1��a`��j[�p��8���@(3ZW���fX�C�MgD�
y��<��+Y13���hB��<����V���U"{W$��;/8��}�C��B$Nk�b9�{"O�����\��������9������6��"w@�D�T���cL�J��c���6K�M�d���'��9O�� ��a����k�l�v�dL����*��3
b���{*�6Q���"�
AiS�
�jG����%��{���0�Z0C�,�	��h0���v��Dn
<��>k�6Z��|�Z���{���t��`�2Pk!0p|^�Mrt)>Y��H���% �Z\��#���t�yyM`@��O��-'����<,�/�s������@����m�3����z����0O^?����V�bo8��:1��Ma�����c��.�p������{0�wtR�\���'*M,��JME����G!9��C�l���S�Cc����8��������� ���r�{��4���c'��c�j3����5��B�_�l�Z��sv��Ee��������^p�
Oc/���*��,g���#�g�l��D��;W��<�D�����N�`��(_���6��"w@�bR����������<�Csqb/���^��@����������l���%��O����1>!GZ0�%7&����{�Z$j���#��J�u�j� \w�����Y�D[&
�X�`i6�qD�n��D��<�<�k�\�������-���N����]�FZFjx�B��X~�y�����]��_�����^��c/�k�������<��7'���q��}�y�1���~��L�X�c�Zd����G��dS�h5������[j;��z�^9��=�o�\�V9Wnx�Y�Hy�4�������y'�*�
��Z|��7(�'e��E�;�a��Z~F�U���9�.u�h�r�Rv^���!u��t��y�yM]@�Z�
v�S����4:O�;���[�9���������7�B(Xk���Z�,��(4���'/��Eat��"1�`��6�-2�#
'rS��������O���q���
�g�cV�2��"7�JA�PXBA�S;����w�EB�����a�|�
�����ux�y�y�[�<-�GSv]��\v���(3<�v�L��$�E����7}O��H������p�N�'�N^���������X����Hl��ZN���.Jwy�F�g���:�EoUI�'m��E��<�m�����ED��d'(��l���)��
��3<`!����Y���k���=.s��l��p�/u��2�9�}#�!Nx�\���\L�WjC���o7ku.��#r3��tm'4S/��]�FY��tF��u�k��"j����N������k[�lYF�X?^_���t��6��"7��0(E2������!��m���\���������q�����/���5E]	m?6w9����-�`'�+�<�
�G�y�8������x�)�R����9�&�3r��{1�����Lg�1�R)%�0�t"���pM�mu�1��
d^���c"=W�
����;\����@R�.��s�����m���`�Gg�!�c=��P�/���
��_����I�Sn|?�aQz��H���d����9��6������I�����������{M [���l:�I�j%�A��N���s���d�`���`��5����q�~�������v�6(�Rs����r�*����']b��8����/���5��&��7�/��
L!G~5-���Q
p4��%j=��u���et��s�)��-���a�p/j/Yi��t�C"~we��XN���s�l��`s�A�N�|���S���-=�2��"w����<����b�����OK�[����F���N7~C���<6�^k�)O����i�c���`������A{z<����6�'ts���h�o�^6����Y�x��3�L�\R9�_�'�qsb9�{b�e������E��p���
�x!4e��E��=e����aQ$t9u���ABC;�Cq�K��������<�Ys��*��=9?m:(�%�Xl�����mQ����l:���������{D��	� ���}J��_���I��3]�^�������P���{�5�zV���c�`6���p�Na�����>��0�(/W>r��
1�o����>K�j��
�z7����i��8{����Jp�n����J>�y*���fH4������Z]lS/x���Y.�������l������{9��T"�=�3&E���@�5)1��=���1���f����Y|���+���+��P{��E�?#������C}f��_I������7��L��n��t�
��������������'��]�Im=v���|(�R�6/ju����822*9k�*���<����NI}�.������C)��eS��l�!��)�\�o�b7����^���1X�RhuL[�2��"w�5�?��u���#�>#�7`�*�����}~r��t�������
~b
Ovd�q�iRv&���R�U���*J�������q`,�#���hul[2+���v�������C�g:S��K�h���M�j:�{�O���g�r�8�-K���</���h�.�_~���/A~ZY�<~b#FVzoP���Wl01��X�|����9�_�>���f���Md�g-%BJe��Vr�a{�PY�dJ!H����
�s��Y�wL��!��H�
=�������^���|(��w��a�+&�
�h�08�l"7����UE��3|������y��R�2��"�50g����.*	Uce�y�9\w�@�{�*�u.�*L9��4s���5O���{m�/�6n�A8��
P
^�V�@c�,���������O��_}x���������������{�`����K;�K���qe�m�-[>�2i�6�aTm{�Sw�
�8����1��l���'��]D�z��o�Bb����8+�,���~z�u}�EW����h���s-��:������)(�9���FZ���25��)2�~��Ms��YC����#���z)�.�G�����J�fe+�FZI������?���������/�������������7�>
�������~��_����������'�Q�z�����@W^�=~1����2k�O�G����#�����!�}{���������������V���")��f��7�>|x����C����������?z����~�Cye�_N�,�_�#b��=M���G?��PK�b�H��PK{2K18sept/pg_part_patch/7_1.outUX���Y���Y��}]�%����~E�Y�� ���@���z��x�,�I�i��[�=��3�_�q�������Y*2���aQ���[����8q��n��_�����������������/~}����^�������/�~���Hd9j�bP��/���{���?������>���d��������^��������ol4Fy�JU�2����o��?��B��������Wo^�~���w�{����o~�����/>��>��d�J���i������OBk��������_��{c�����������������}�����_����/�����?���G���Wo�_}���{��z����^���~%� ����?��������>�Ow��o��U�����^|���������4%&R��o����
`�w��_@��(x�e�����r�����^�x������\�;������?�~~�?��^��i?Z�3S~3�U0���I~�h�of�I�/F��f��7��?��&����[��lP4=j����uX���N9J�"�pX��e'����1KY���mS}�?����+�o�^����l�?�����|Dv��_���#��~������L�����D�|�l$�Z����>%�xI��:Q���^�Yr�E���e������������#�������>|���_�~s���������������^��x�^��/���������������������w/z����o���w_�O������v��������G���X�#V���������w�����?���_�2b����������H�<qy�������������5��k����!���h�����_V����O��������0���>����w�o~�,`B�d�o!�y4A^�t�-��1�d�� E����,����m*'!X��l����������A�E��h���(�}L�j7L9�OP�����.O��i���/��.�����_�������������o���|2�VZ�)8K:*6YK�������e������Z�h���N������O "_F�o����������o{��y/���^�x��lY��;���lH���8m/w����&�&>zL6;hJM��F���$�����������ZQ�������1��z�/^�x/q�L�}�~�.�����������J�$����=,�DC�EH�'��������D�`��ax�l�(�x>P^�_
�e5��6����>�
}�A���Z��<^o���������g�
}X�����rJ�o�����9�O�-�Q�y��^��f#	���)���~�������K������"���$N�S�KG*T�)��=x�<���c�1���;{�RDQ��H4�����c`��R��4&����8J���o/ps�<���`����6�~��5�l���R9���[��Mfl����;_���6�8#x)��Dqr]�U8��q
�q{�����$��&K!O��@T���'�\C�u���9�-�ZB�m')��������8�V��8#!�������3�\��u��$5q��������PLKf�>%�z�8�8����3Q�b)g���,����s����:���g�|��F�!�,������8��qf�u�$�R$@��g(��y>=r0�8c	�����[��5-h�pgx����I<?��R+��Z�N����:�P7��8����.�3�?g���C��a~f�?#�����	m��N�)M~�y9>^���	
�	{�L��� �o����`��?��1w3TYG����g��1"ypesF�	��	��.���ge����7��F�"@e;e�\�\��,#6p&6p&��3q-
�q���^���EN�z��Nef����`&hydxnesF����/!�\��C����8�i�SP��7%s#�����61j�n�o&�n����=�&]��[�#���� ��LL&�hl?c�u�ab�-#,o`�$=��X$�/�dM4 ����v'-v����V#��$����NN�)19������h���F�jDT�$�n���#b*y[V�G��x�h����o�~��(�����f��Xe~l�n�#W9_N�#Ms�&������T��~��O2���Fi��'9���cUH����2K����Q�B:[�ZV�tu�3"�2
����hlV�������K,"��D\*Yq����3�U�a��L?�G^W��,?��8Yqf]^��B���;)����	^a���RLC�����h�F�9��snF)Z�"���]B�XX�s����@����A46�Dc	7����EE>�[cp^.�g���>!�n����mu�G�,��������N�>+D��F�A�E�/���WTTb/�����Dk��k��Q+_2?�������#h���66{�������v��,��
������S��5VYGA�H�3�
4G���S�������O��}V����4�Or�u��X}��X9����<y�t���#h&�i�������^
����l�X�z�:�|�\��m�bfmPz�~�*�(�����������*����GypZ��I�g����H��x����h�A�T��Q�%`��n05��ln���{��_�C���AN6{���9�Z�
��E��D���$��S-����:
�8��2?N99Y�V��K3���a�9��w'����f#����Q�N<�!r%�,^�{����n1w�>6�A�X�L����,�=(����l�#���Z�Y�d{���3VYGA�[����y9X�3,n:?��J�����I�g����Hh1�1�0'����s�}tKp n��$q����k>J�32\#��Ad6{������%����0.�9[O%��k����>ri���D���*���j������RM�����y������z�����������@F/��ejV�1)�*��E���e���i����fsEo��Y7Z��,������~�*�0�C�O����D^�
*BhF^hd����iwR�Y!;o6?�G"��&~�&�s�7��^����} �����\D���I�nH���w6{���5��yy�i��lH���9/�6�3VY�A�d�E^����:=j���P!ad��I�g����H�;���$��yA��D^�?JK���}#qD�~��>&�,��dG��m���tg{Mwv���&kY��r��3W�*z=�Xe}������i�������m3��
HM�s�����IF��^`
���5Q|���yr�����B��ECf��"6�
�����lW�����&kY�%|q�B�P�g����>����N��9��X��^/'��CNu��}���[���3�y��j�����H*�k��`C�!�.�R@�l�^O��XT�#j^��v�{���5������Zd�CF(r
`8�~�*�(���t��<�A��q�7����[��ewR�Yg;o3�	��K���>2t����@��:m�!pEs�b�TYg���i���lg{�vvF+6Y#��{D��h����>A�EG�s�D��uAhF^Y�*��������v�f$��r�]������\���3$g�<*c~W��%g/f]w9�,Gk�:����S���>
�����l��������Z�x:WZ�
d������s����K�����6#/o,&�}�����Bv�l$����{V�����D~��T�mu����������<X��n6Ve7�mh\g���^s�D�&kYGy���-+A�g����O�/����O�V���c9��E6.�����*�V#i�o��K�ZCc��mP���	:.Yg63#`���uy9o���`"�!���:�=��v����<��3�"X�d�$�������:�X�/��,�T���S��#�@.u��ewR�Y�:o5�F@#�$����V��(�y����AP��s[��
�!-��h��>
�����l���N�z��-kYN��A��NC?c�u��������t��
����ab�e��}V�����x��B��x0�'^�'�)*����G�:D���"�cTk��/��\V}\g���^s����d��u���l��\?c�u�a����^����"�
���=W������u�j�����^��&w��
+mt�"M����w���3�&Y��Cj^
�����lW��������e�9��7������U�Q�����
}�{��}���c}l�.�����������H�y���X������^"/6��L����Z�� �0��8��R�(,a��\g���L+\gJU����H�U���Kp�\?c�u�%�#�$3�T7P�f�������;���>���'	|��%9Zzt�EP:�}B|��i�W*��2��������Fh�R��L{p�i������Z6��~���F��Xe}B�~�m�Y9�+�M��M�%���I�g����H�:�OB������Dy�`�!kG��Tz�bj^����Hj���O��L{p����T���eA���b�'������O�����<a�V�����H	���������
�y���e����Jv0�Y��>A"����1i��e
}.|'��j]�C�R��N{T�������7Y��99���B���3VYGA�@��Td�]*K�V�M�����5��;)��MG�j$�
S�q'xA���r�9ppU�'V3��k^���hw���R�>��;�Qq����e�x��2s"�G�	XrE�z���:
��k�O�9��w	*���O�q-�aHlg�������Vq�j�i� �f1��:��;�mK��T�yA7A��8�=x���
<�mH��;�Qq���������kY�rU�7�G��(�\c�u��r�;�
y���S����w9�s��}�*�[�tAv~�6��T��F����pm����qy�����+����C��;�Qq���;�h��H'#^/�'h�
�~�*�(��lK���s9�U
����2ee����N�>��mFB�EmC�G�������}�������x���_�����Tc���}\���������;jXf��l|����U����*�(����e��u�K�6��v�Y.tw_M�s�����IFB���|�y
VV��>F������M:���;.zY��x�\�mhy���kT��w���N�"��Z��$s��bs5T?�Xe}��?������}h�Z�)��=O�$D��=��+���	��T�7	\��������X��&�y�X�xS���i|r���)
lF�]����(���wCV07�-��m�w)�S?[�u�	�����|i����PSW��G)9��;)����7I���b�
z�Gb�\���9�����8s!���s�p
n�����[�8��pw
m1���������M����D�^pH�3VYGAG�F�&D�`�m�N5M�'�7z�������-��H}
��w���0�T�	����bJm����V��3���u���ih��=�����&8/\����Z���JJ[��Xe}R��_2��%*.��;)��
R�>����MZCCgp�iP���P�os����V���s��}���Y���ih��=�����X�l��\;�
P�5���������1���a���J�i^�IAY[M�������F�r���~�������g	j&y*S9��1�H��Rn�N��#���s���� :��DgC6.
���EV�'��5�b��UYG����g��_�M��T�oewR�Y�9o6�U��]����X���������2���os}B�)������i�4b��k���<g��,1l��Z��
r=-��m?c�u��}7��~�+��d_�wiE�h���I�g����H����vl������,I��Ip�6�9��}��,\\d
�`D�(�<g������=W��7�E�x�1�.�k�F�g�����
}"��]
��V���db�#�Vj��}Vx���mLQ������%�P�}������^���F���!��IV��2iL��Atv{��
�9el2�����Ru�*���Xe~"wt~BVf����n�-�>!�7�N
?+D��F��k����z�K�e}=���6�GPK�A���P���>I�����Dg���_�=��n��M�"�����������~Ig9M����u>����|��,-Vv�D�������6�\��T��-�
E�0$�]������8`��6������r���nD��7��~�����tK��\��G(�D�>#���U�Q����S�p���H��{�=G�c
]�>^v'��5m��F��z�R���SJT)�%�������<�-wy��q�����Au�{P���e[k����@5:�����Eh�k���>��C���2O����mW��]����v'E�5��V#!M����|�>|�[S�m�d��v��n�^r��C�'9�+*��!%w��:�=�����LP+�f-y��y���NE������w�:C�vP�+/�h��}�>���~��}V����4�L=�c�����z��rs"U<��l����t�;�S��
9�#�>
���������1��d-�b��uU}`e�3VYGA���_�K��W����y���<[��N�>+\��F���6d�����U���v�V_p�.�$f���������p�}�����:�k�3$�f��mkT����s����:
����"/9MS�9o�S�N;S�*J��+������mF����bQ<�8���������8�^��E�	K���l_u�|���dg���_�:���7Y�4����(,R?c�u�	�qt��ES��*����C�%�y]v'E��9�[��a�q'c,�����'W}�QkjOq'\��1�4�%<D0���i���lg���L���mk���{��y=�Xe}��~}^N����m}�76�
�#��N�>k��[�d�G�6�2M������j�KA>B����(�y�c�{-��Rqo���lg�v�_�"(��A/����M�:��\c�u�����������5������u��N�>�s���*�r���<�����e�����L�"r��I��1�<�bl��0�����L}����`;�k���o�g�m.��)
F���}0����*�0��G��<�w�V=��u���ZL��v'��5Y��F2
aFZ8%��X�G`����\`o>O=�M���c�$~��*�C�G����v�=��|�v61�%���"L��X-�q�*x��Xe}$N�'*/o`���T1�d����SG���;'�<~_�d$�Uu��Aq��!�BF!��3m;����0A�iM5H9)zq���{�����l��E��X��KdF����~�*�(�#X��n�����^�|�|s��#r���;)�����I�d����w��wx3��Hh��%�A�����U!e�t�t%*/?9b�7���������,��k��g	��w��3VY�Az�����X�&����������Fv�j$0���	��G����T��z�`t7��D�tN��Iu�gL�� ;�dg^!;��G|�Z����,��$�3VY�AJ�)��XO���a�����jQv'E�5��V#!��x�<�B���a#�lXJ�B�y�h#�\�6wzA���ex�#D6��v�=��|�vvh]O��E,V��/$;�����lc�u������3n����mU�v�)���"(�������V#Y�V]N��"3���c��(�����%5��LY�2�����LN~H�7�������-������m,B�S���CN��~�*�0�m��{�*���)�Wtl�}2SD_*��;)��q��	�D��>��L�I�s�u-�e������F�2�K?*�;-0�b�4��������l��f��l�kN6�F�������O�(������f-���x�o�}������I�g]�y��tQ��Y?���nuZ�R�BG�\�� ��9�J�J��|��%`���F��r���{�����Lvz�d.�>�Q
.�R?c�u�aq��^b�)��m���M���(����;)������`x;/�$*,��f�_�T��>�v
98�.1?|L��),�
����� ;�#��<���7Y� �
�v!���~�*�(��M�9H�MM�4g�r�k7��x��_v'E�5i��F"��Y�%����)���Ey;j��h��9&G	��1�%>��:wHg
@�����:�#�����������Pf!v ��~�*�8��o�N�����*�w��������Bt�l$�"���bQ���!V��)���\�C"��f	�f3�B�-	�VM������9��t�{(*�6YK����i�:�~�*�(�#'�[�'�!t��f�6����C��-�sb����IF%?g�=K�6��	���h�
C�����8�	=����ej8�k�t;D]#4h�a�s0W�c�������rGSQ[��Rez|���er��������p���ewR�Y�8o6�D>	�]F����oIkd����_3�0�q����(�H�%���=�����q{p��]���q����E��^�ht?K�u����uL5O���U�
=���)s|�����Fp�j$+��A# 4������a@/������>1Cz��d��%���9���Cr7l������ 	K�U�~�*�0��:�j�J���MA%�hR|��gW}��zV����d��&F-�8��GO�c���^��X��������yQ=C�!1��An{�����z�7YKcr ~c����TYG������%��Se�4F4�\O� "�^ewR�Yc6o5��@`.�?6��*��5�Y;3�
E��
=h�d��r���F�(q��9�Am~z�Zi���"T,���]?K�u�a�OB���������R������I�g����H^it�Xd|:�/oi��R]G���������i�\�3��� �������9�Al�z0��znZ�)�p�`v���Y���@O0�|��7�1�JLnU�f�����v'��5��F����Rp	��r����.�h���a�F%J,6A+
Rs�����`E�f�e--���=�����*�(��t���&G�mM��vq=(q�iwR�Yc4o5�U =I�%!)
n%���+ifv3��)pS]�]T	U8�A o�4��q��Fh0�����.f�L�d-�AP�*P�g���=��n.��,��P��Bhk
4'6/��B��y��H�������H�uC����r=��s=�Y��9�IkVb��a�4��a:s��3�V,�&k9E9���kc�Y���@�����d��y[K���G~$e���;)������P������=V�j+����fN���z����A.\��n���l���9^��=�q�7YS
��[C?K�u��)uc3K1W������F�K����.�sB����IFr��z��QZ.��,e��y�x�p��������qT:��i���l�x�fN`i�M���gJ�U�;Z���@�u����x�)��fVy�mfP��1�_r=������e}����A����FK���������t�;Z�]R�|�1H��XQ
M��4��q6s\a3f/coZK�'W
�s�a���TY�������Q�x����*�����H�����I�g����HV%T�H'�bi�[���z�Vfa3��������Q2����=
6s������	�oa��,�s�&��=�,U�Q���e�����*W:����l���sW��z����	�qB�����p^O(�����!2�i{=,~�z0��l%�EaH��`3�=������fk9�)/��I�~�*�0�cL���q�k	����-�%0�N
=kl��F2P3��z|�1�t�K��b0��./�pS��w���K~�����<��l���9�����&k�2��0��Y��RezBO�R&�D]S�OL�}�Ly0�ewR�Yg3o3�Q�3��C#�X�����f��N3���J���C�0���*>����
6s���W��:�����8�`h�W^��-U�a�G^���Z(+u:44)����)������f�n$�P\Fs�>5�[�q�b6���J��k�v�E��S*\Q�2Tl�@O���`3�+6�����m�������g[���@�c���n�[f��j�8R8-�B����C���N
=kl��F2*�U
e��$�4R�R2���l!��
�������!�SM�q~H���f�{���5��&�M�g���z���L?K�u��������&�l�*5E3L��<\��I�g����H^�u��C��{�R.Sf�������F�YbW(�^�	�@�FN��i���l��"�,o�*B�ZZa�N��1��Y���@����z�@,�PoB;�0��s��z�����������8�}����.�/KW��	=�s����0.(�[%F�l���9]k3c���nt�Z�6�.G(���TYG���_#������)+M�'�������;'�<~Y�d$W�d)����f���Y��u������%��!.��A���!���fN{����63������uux=����~�*�(�#��-���S�:������A�|�^v'��56�V#Azi���p6�
yD3�"��������7��pi�ZK��;�}45��i6s�f3G���3�����z\�;��Y���@O����rY����'�L3��^�ewR�Y�f�j$	fb�"�[�����1&�^�����D��Jq��]�`;��fN
6s�������^B�J����z������L�,U�a��b?^�Sv�`�Se���CP��������f�j$4�(�'_���.L�����_��k*�^9gJ�U�#<
�zl���9�h3���q�7�e��"^����TY���O��/*���_l���T��T��I�g����HQ��M=��.�tQ)D@�{<��1�B��D�����V^OR�J
6s��������'��&kA>���V�\?K�u������t����Hj{=r_��l��;)����7��!$�,7��>�
�Y�0����bFv[4�J�F!���,(3D�'5��i6s�b3�����[���?�G}���Y���@��������].�`���U_������������H���&�h�0�M��H�j
H���`���9\�x�j�&
Q)L
6s�������	�E�&kiHe���Z���Y���@O�~WT����T��E#{%�,��+\��I�g����H�����rf�63��Q����R�2�F#�!S.o'�s}���4s����`3�k6s�zL�Mk9���Ra��~�*�0���O*,���=�|��Ri��|��z���[�$���ifv(���@�	��s�w�43�V���������`3�=�����Q)O����:���h���,U�Q���M4S	���U�����r����N
=+l��F"�e�.��0������Y-lf�\zpm��.��!WA�p��3���G��k��Y�&���@�A���lc�u�aG�z)b.����`��F�M
�5���>Wo�����K����U]Lts��/�f#�o&�yC������|��.7{���	�#��Ai����5�f��6��L/�`�������~��<Dt��~0�&6������/�����y���������2��ts����\�d{�7z����-��!���B�!�F7h�����Fl��}��iq|P��&�;��Xe~��?h!�gl����6���B}DM�����y���0-4���VR�2��\���IT�(��D��C���c���7[��lt���/�?+R���X��`x��C�W%��lc�u�ac���@y&�q�� n���S)_��Y�g����J����X!w��n��)B�P�$����i�(���UcL�Rx8QD#S�n����=�g��,��2����e�H���~�*�(�:T�z�
s)��ua��L�����r��~��[�d�4iTh�`��YHC[��R���c~0\��?N��\��4(�����"�,h�M#��'��
�U�Q�����?rI/���J��m����'��������r�j%+z�KVi=8��TN���t�R����P�Y?N�&����e���4g|a��&:G����F`P����R5'���*�(��C��6�`�I1^/�zV�iz?����Z��;+���6o��Q�f���Ep�<��d�������!o��?�*_�w9��3�Au�������>��sb����U����*�0�����'�!��L��C+����wg��5��V+��?��F���|�`�rK&����s�Z�L�2�!QO��Rb a6�g������,d6L+��wT]��h��?�#��
sR����k�O�'F���;+���7o���s�'�)�
?8Zb/�y�jq��Q]��{g=J��t��#��sM!�1y��_�zsH�=n�2�mk�"$����2����*��c�]�%���3�t�iw���lFK��f�;H7O��OE���;����`'2�0u3��`��|�Oi�Z����/$H���/�m����V>�������_�]~��7���gyVl����Ge��x�����1+O��!��������7/�����{��������_�}S��Qo������~I�������E����9�����+�r��G��9\��y|��C[�zBZ����?�o��%F��og�	"�i��t�m�B{
�g����
�����n����~��o����~#W���{����_���ww�w���o�]����t�7��\�[�&Lb�2����.3*�M;;f���u�^~����[}~��I����-Kc��o�-|��m�E������[?�w.�����>||��`���h����O��w<:\���>�����E��k ��%�(6<���������12����
{-�{�Q���#c�-�����=B��5����o��z�c�����!��h�~����o�����l��Ey�s>&������6_}���PKc���2U�PK{2K18sept/pg_part_patch/7_2.outUX���Y���Y��}]�%�q�;�}�BH (,G���V^��Rv8��1v���gz�������p���*m�AN�����A~�<y�l��������s�����f��p�|��#�/.����������>>�:Y�������5��������?�|������������^��<������k�\P�$������w���_}!����_.������7�uw���w�����~x�p��N��*�������?	����>IY�#\���_u�7����W?�������_~/����><|��������_����?�����{�p��wZ����?�y���J�@>���W?�������O��]�����W���-~������?�}��iJL�����i����eE\=
�m�V^0@Y����_�>|���������uq�����O��\��>C��O�~�>�f��f)�Z�h]J�����o�C��~3�����S������A,'���%m�g�,B>)��XD��QR�u�^2���Y���oo�����N�G^�����_>f{����w�#2E���[��?_}���O��������\>S6{�l\ch�t0��b���V�������2sY����w���}����r����}�����������v�~���^��q����|�������_~9=J�������7���o��G����|�����������S�W����������?{y��<����b��~�e}+��|+O���}��_�2�m���d��'9R,O\��q:�&e��7!&����;~z�l�FG��NI[+�:�j18G><���U?�����������y>��q��#|�������LH���b����U����&���	S���\bm���<]���@����Y����rs����A~~��h���������1�������U�-����������7\�����{{����_���������������gIGe�*kY�����)7����Xe�H4_���0)�_j#�'�/�#���]�2�������^�a�������#^�1k����/�%�������n_E���Q�'O"�`�������F���$������=|��E\-��(����xp�_�x�^������ ����o����~c�C��}}����O+nQ`b5��Pv����3�����S�������h(&����K����BdY
��
��{���Bo����e����8V��~�*�(���wCf�������:XLrY-�����������F��F��DY	�}��Y��>.�D�i�Y��$�)��1I�4�G���*�ko<��x�)�+w����g�	��%��u�SY�����9<i:L���q��"�5l}KG��r���Xc8�M����f�u��|���w�`�&��3N�Gv����M��G�� �f��PG���7�3n�qW0#�WY��J��X<""��Nef�vNw����a�A
I�JA{5l�3n!�Z
3.I4�X���aF"�1/m(D9H�@�)��2$�A�%m�s�����2��2~���(U�5���T0��|3��SY�A�{�L��Q\w�X�0�F6gD�2+���(#���V;��������t�����"��9�������pfx����I�>��R��G�u�H_n��3���y6���H��a�_3p�(iEQ������#��0#���������X�{�����=`&\��`�K�R�D�3Ee��f���3b����$NL��V6g����qr:�6��>�LTnl;���6���Q.�9xa��2��2q��K	`�ke=��\�����f����2r��v1����_��e���B��N=���pw$slt
�T��d��@22�A���f�x�Jn�?�����=�&]U�j����G�b+�f������n�`��������������^�|�����U���^k$���ldp�b���<D�9��^�8Y�!I4 _��6Z�Av�F8����^F����� ��F���k� �*QF��X�~�*�0�c#uC��y8X�VL4�M�!	��>ewN�y��>�HZM��D/��}�
�0���1s���n�[�6�%2��Q��������4�f��Y �ZS>c.q~��5^z�*z��Xe~��s~�u��d��k�����*��Bjtx������D0^k$��b�3]X���������V]��&"K�3�ea��|��{�cl�O�`l� �k�1#��WY��E+d��pz���bc�u���[���.�i[�����9�Hf�I��I�g�`��H@yW�e�3y[�G�zY��T��f��]�rY&��������oh�i���,cs�2�^���*k��W���!hJs����:
�D���W�)����B�Z��O�Z���I�g�s��HZY$�%J6b!�-�%?��Grt:O��7����5J�*V 9�$�j����csE>	u�����`�|���*�~�*�(����C�=���
*�����f�d;�N�>KT��F�����qM������=:���/qtm\2��9�g���V\gR�&v� %�=H����l���7��������bcWU_l����>�}���S`�L��Z�iU�#�$���;)�,P�W	����^c�E9D�����"/��y��O����\rg��6V��&���i��4��f����*[��ad�~�Z��S�#�;��x���3VYGA&���X����	��`q���'���������sy��������*�.����3+PQ�>��$87ebb�I%�
�x�f��������0�=(����Rn���Z6N�,��>.gb?c�u�	.����sH�b[�f�Y#I	5�qwR�Y"4�5?(��(wfR���O*�OLF����L�r
��hkJ��l�������4��fj���6[��+�%O>����������:����K��W��������>~�����@t^m$D^���A��.����C#��i����{�������?�#�ab]�r��}dg���\������VY��8�:F�i�4u4VY�A�d�E^���������JFn�qwR�Y ;�6|������#0�J��F�4�Q�M��IP�qY�[�.�$��HMv��
�����l���?)r��E�!@�E���������x��|��r�q�V���-�	L*A�~��}����2��>h tt���.�.�O��M���v���^|���luR>U�����m���dg�@v���*s'6��������:
�J�K;;��U�G)�VS�v#e0�N
?Kd��F�de���
�L*�k�N9(�L�@�*����$V%�IIoQ��
�����l����G���\���H5�(�E����:
�x9c/OSX����&��DQev'��e��:#��k���q�b;3��c�����6�'�y���T��{D�sq�i���lg{�vvr���Z6���U�0� �g����>A�EG�3��i[����{�1�f��}�����$�Rb/NO����\����4;eR�A�ygt��,�#�������FM]��4��v���f;[���*k��ZW,��hm�~�*�(���m��>�m�$����D^���0M|�tV�����YF�QH�j��V���Wz����6g�v�@������L���*Yn��m���lg{�vv.(gWY�r��8k�[����3VYGA��_��'�����
�O+�Z��I�gI�y��}$�Q�6F� W$~�^�s����	�FLf��vv���(��n�i���lg��v�vK�k����������3VY�A���������T�Ybj2~���nw'E�%��Z#���>�����-	
�����A�����y�MUDQ�d*�1�[������v��lg����c��e��*p�6H���g����>��u�.����;��r�o�+���I�g����H���
������FA��Wi�jG^	5e��|���f���)�e5���v�{���5��	��U���(1-��7�e���U�Q����&���#��|�@�k�z�]�3�N�>Kl��FB�9kl��o������a��n�oG^�}r��q���[m>P�oR�j���lg� ��D����1���U3W_l����>^n�n��s������cRnK�=wU����gI�y���k��>�ik�
�%�bSM��NOy�k���(���]c��}(lB7����`;���R���i-MP�g9�D�?�U�Q�������t�R�B�Z-�����qwN�y��>�H`;g���
���	Jg�O��z�"���B�{��-�b�>�j^����*5���������q��eC����+o�J������OH�}�G���)�����@]�������y����YX�CTNo�64��X"/�5d��=��KL���>�I��m�>
�3��u�k��C=��(�e�
����'��Xe}���468����*��v�=:����S:)�����,#��<v���1�L��>!r]����������56 ]c]Uq��)�F������u��C�\���D�A��lCV%��h_j����>�t���r���������3|�qwR�Y����Hc�9Y(qn<TG.��u\�������iq^3�X����g�:L��}w���N�wLJ��Z6Z��<�����(�3VYGA�Gz%������Bd�!����I�g����H��p�r'e��w������6wZx};��`A��5/�J��@��mH��;�Qq���������k�U�lC�l���U�Q���:�
y���S����I��d��I�g����HP��P�I�����v�j���Ix��y��46p�xL� �tz�j��i��;]��a�D��EN=$~���t4VYG�1R���S=�U��5����9�U��I�g����H�4e���������I��������8��){�K��qD����2[~\����(����;��o�Y�lN����hQ�3VYGA�`M�6����s������H;k����^ewN�y��>�H��!��#�8�?�#����m��F-$�}��P��,o!,�%w�G��=-�S��iZg-��)�[-@k�1�3VYGA/�O7�!�G����pJ�n�
}_?������B�}���>�=�H�x���eAL�������3jrD:�R�g)�<�q�lQqw��������V��
B�5��AL��_��Xy��Ve{�x�-�N=Tw�)��t%���;)�,�W	Tg��`n�=��wS�9��>:Nd��
�����9�����&-��!.��s��bV�
�U��x�p��/���������~�L�Uf�|����������;)�,���6-0�=E��>�(+��bt��J�.����xy�:������}��-
��!.��s��b��<�
oZ��Zt��������*�0��:����a�*�p���I�gi��3���b�����C-����}(��yr�j^.*��������b9S[�OC\��!.������f��l������b"��0VY�Ac���<�+<�U�n��8�����;)�,�R^k�	}�U����[��y���H��
r>�6��d���3�s�&�$�I���tv{0��S��!������,1��x���b?[�u�a�{�Y���`�0A�1 �9�N�=D��FB���PS��1�`�f!�$��$�����>����D�@;����d�@�����tv�b��6��E�na�K����Xe~<E�
~P��c-�������I�g����H�������f:�P����xq�6�9!�Ne�`�W]^!l�)�Lg���-0�%��5�oZ�|t�9��Y�Z\�h����>L7����a�V����RHe���;)�,0�WI��b����������*�a���a����3��^�yB�I��Lg���-0�%��q��9��@A�\C�g���?�;:?!k3L��^��&S��)��v'����j#Y(:��;?{�!���i�\�k���x�F������x���G�`:�=������}�.��X�r�XrGn���bc�u���/�,���t��N�r[T�yh2v�D������48?rI�.7nq�*i���2�����0�]��(�<�6V=�M[�O�����:�ku1�^�p����I���AY��~�*�(�#�S�p�<���{�9%���#�L������.��H���L�<�f����e
��Y�=97�w���lY{h�V�5n���Av�{����(e[���4�4��R?P4���*�0�|?����#ul}�����l'8�N
?Kl��F�`���|g�����C��H{���M�Y������s)G<�-���Aw�{���5��K��\�����'D�U�����:
�0q7�3�<�\WR�u3�#�S�������@w^m$
�@4[���m3�!�? N(B���sJ�a��y�$���S�(���i���|g�w�����2��y
G��J����*�(�4�+|�)�������)�m���&u6o��?����j#~|��$���c�+c^��Y�/�sz~P��e�N�j?�\q�����g���_���S��g�E�"����`}d��Xe~��n���1����1i����<+x��~��)�3�)g����o��U��\�8��8�^���Y�RV�wD�����q��`<�=��Z�9_&��Zd�+O��zk����������U���Va��<G�9��s��}��)�5�Q�
x��n5��!f�O��'����(8*����Q�>��&��`<�=�~A��*����"uU��f�����U�Q��M�����2���zSf�'�,S~ywR�Y�v^k$��W����}��WR�d~���0�����-�b��@�I*U-��&U�����x���g���$���"$�]~��%����Xe}�#�p~q�mE�u��3Y����\v'E��i���D���Fq+�����&�}��e0�b�%�\���\"/hOZ ���[�3�
�������	��h�����H'���o�2��U�a��;�35y����(�M��fDk��>K��k���<)�O����r�*��^r�+?K�E�A6�Gbg�k�{�$�X,�� <s���{����lb�k�7�e1=�
��1"����:
�8����!X2]��"��jg��1�9��w�D������D��B���[�O�P d���T���1}��u��7L���~��Y6lR������;�5����j��V��$�w�Cl=����*�(�y���C�&.�������3�$�+D�qwR�Y�;�5�V�a����y�P�Z@������m�!��:g�]$��T����Pn��y�3_��I@7�U�"��'�Ygy�����U�a������8m�SbM���q�$��I�g����HV�.�}bT+o�>C�)��b����M�K�L�d���$���n����v�=����vf?�	�������v0��'o������~������F��U�:�"��:ewR�Yb;�5I��#/t�m=�B`$�}��#��;�p���E��w�g+�O�\�-o���
�3��v�k���<[Ze.�*#S��-+�]?c�u�������UfL�`[E����$�E`��~����ih��'�+���Jq���X������i��w�|5���&�^��:�\g~�u��kW���mc�x��/F�����:�D�����v�a�u�86�>V���x
��b��y�����������c�TJ��k�,�7����E7/��u��y�QS�$���:�Tg��:����������~�2oc�u����4SN��I�x������	�-����I�gY�y���.�"���
�;K�dK�+ZHzLiT��=��f�/	1P����6@�����:�5������Zd%�f���F��W?c�u�aJ�[�����V
�D�X%T���I�g����HVP9K�^�����@:��3�����]BN~��@7��\u�#�}Tg����O�����Qx����iz%��*�����*�(��M�9K��=�4%�r�k�y�����>�N�>K��k�dT@���
��3m�����>^��Y���slP�����az4Vg���Au�=����:�����"�����Fc��r�p~���:�����o�-�jr���������s^m$������aC���U�=�O�8�1Ljp�p�z��	
c}�n�4
�s����5�`��[g-��4�'-��*�(���/���i[�w�4C�?$���9�����,#�wcn�HJ���}����F�0�/sC���a�-��([Ke�M�5B��� 9s=�C�unZ�*6A����Rez|���ej���b��d�8
/��ewR�Y`8�6�$�(��e��������40A����`�r�^2�d*=y��x=
�s����B�%.L\e-�"$*�L���Rez��c3�#��m�DmCO��Lf��L'm,}��>�HF�%��%���\�J��������=,���������*��7��
zs���h��!Ef����R��B$����:����$�p�hj	*�D��bD���;)�,p�W�+t���1ry��v=B��i��`k�����1}3����� �������9�k��w�*ki���z�����RezX|���c�t�L�������!)����;)�,����
$U�!]��[��1���v�N�fC�rhCDi\��������������9�Al~z�Zi�����:�I�}?K�u�a�O@��~�����~����,��;)�,���I�%d��4���)��R\gP{*bajS���w�����&Ta�&��Ak{��?��������G	G��Rez��(���m����\��(K�qwR�Y�o^k$Sj��=���Ns�z}���!>�M�G�NbC��Wl������4�=8�!\A�1�N3���	���H$]M�z���:
�$�qf����i[�����PT>����I�g����H�z��vm���-����@���D��a
��B��
�NR�p�l=&n������9�����+WYK��)��9>/|���:
�8�U� 9S
C���I)��\ewR�Yb3�5�<t�(T�\)��b�0H|�^�����@84�I���o|U2va�i���l�p�fF�XM�mky���Cs.B?K�u������=#��I@���G%��4��Sv'��%6�Z#D0��K4�1�X� 	��������1m��uK-�uv�|���B3>6��q6s�b3{-��U���2���8��TYG��R76�S���Z���p!=����sB����YF�.����s=N���G[R�n��Q\'/QD�c�������qJal���l�x�fN`i�U�rh��]�Y/�TY�����p��n�4�S�Z"l;���<%p��z����I��2)����m��
�%�����y�� ��A,��RPS��r&m��4��q6s\`3�q�����|)��J*���*�0����QAl����M��,n=^�awR�Y`3�6�Q��/f��7��eH��ief6�
���.gOB�p������zl���9^�����*k��������g���=,/[���4eP�RED��R��}9�N
=KJ�k��`^O��+��"t��
�@���
��M���[^E�>�Z�6	�l���9^��cXm-��d631��~�*�0�cL���q�k���eI��<��N
=Kl��F�o�fQ�yc�����sy����K�g�A��������'6��q6s\`3#<�U���"$��c����*�(�z��t���k����;��*�j��z������4s�
��=�1�y%�,pR�����Q�-�K��U����B|l���l���f�i�����tQ)$J�0����:������C3e��_B�R��
Ig���;)�,��W)bhw��a�m�G~��+`p��@�5���+`�]�FqaC��87���9��f�Wlf��5oY*��p���TYG���_q��3�&��j�xr�i���J��yr��z���k�m��8���������$efH"
��
C�z��F{	�R�>��&��9��f��l�h��&����+u"��c�g���=��~=\^�(��m�A��h�#���?�N
=l��FB�){=�������8�8�f��\.�m��`4W����	�N��zl���9.h3�������H��),D{�Y���@���z�@��PoB3���-�:Q���������H����8qI�N3;e�.(��]IV�&�$q6$��Wd�c%�6E�l���9]k3o+u����������*�(�#�r�F
�}�i[SV�^�z���������e}����R�=�7��B:ky�n�&5������p�\����p�MRS����`3�mf	����[�"��lfgt?K�u�x��f�6����VG$�;�aVn�(��B��y�� ������'��y=!���h�Pf�f������ 
" �U�s���E�hj���l�t�f�>N�g�%1�0�G���TYG���S�����G=��L3CZ3QH-��B��6�Z#8!��@�g���N�=;�^�I��>�0G<��6���:y*W���`3�=������%��4oZ���'V�~�*�0�C��G��y����-�	=�������B��y��H��*�����[���.N(UM�����K�-�Y���N��f��6�zl���9-h3���a�7�����+��J?K�u�1�����R�	���J!�P�iv'��%6�Z#��\r=FE�q��U��" �=�?�����m����a�(mR�J
6s�������^~q�i���1��������Rez�c�GK�?�*l�#��^a~���ywR�Y`3�6�U���Q�+l>�
Y�0J<0��1"��f&#R�����z���D�'5��i6s�b3�����[�����BD�:Z���@���������/�`����6�����������H�BH�#�P��m��d��r��I��qI��y= i��dL��Jaj���l�t�fN�/�WYKYN����;Z���@���+\�L��T��Y#�:�,�C�h��z���k�$_�.72�
4��p�!������e��F
"���R\O����[@O����`3�k6s�zL�MkIH�a�����/�TY���~RaI7]�u��	���#g2��;)�,���	�u�
s��m��&�T0�+L!f�5�A�'+������p9���TXj���l�t�f�.V�����,<���#)�[���@�7��h��������2������0bW���B��y��$��"���m��W��u��8���s�
�w��wE4�bU�op��3���)����S��`
������7�U�Q�������4;m���E�,O�)�����s��>�JydS��I�n�j���F���7����]����r�@�q����8L �~�f����$����*�ye����|E�~���:
�x�G������N/n��6�)����Z��;+�,���ZI+��0��!�J7g=���UO�<��\5K����	���n"���{~��5�OY 6��>���@lF��{/q��g��?���� �1���uQ���=
�1������l^k%�n�`�L��/������D��'
�I��}����1A~�j����&���6�����f4$�UC�E�`�x���Xe~��~�=bI3����t�f�+�y�lvg��v�j+9EH��f��tI���~�T�0�a�H'Lll7V�)��d$�]���"�B0�����,�1�,�2����2�ZG���*�(�:T�z�
S)��1�mj�b�\����wg��%��Z+�\�����}deo���4���z�zq��	?���DY�=�	V����O���?�~$�-�`V�QLpI+gb?c�u��k��8��x.����������n��Y�g����J�,�jU�{?I��Rt��Je8n����i�|i]���[�O���?�~�����*�9qi�/�\=�U�Q�����m����m�R��~*���b��~�u��YI.U40�%�L�~� E�����oK�G#_��$a��r�BF�����,H7�X�Y�4�W��Y9A<���Xe~��55��8�I` ��b�q����������v^m%���q�n�^Zx��B�U!�F��#O����)=������3�Aw��?���,dVL���u.��H����:����g������:X��c�����z��~����YI��1h�����v� ����jg���qV������Y��a���d4&���Bp�$~|g�����D��f�������J7S�u�1�����q���s~��n���tFK�X�XWB�q7|W����N�cq�{{w"s
���Sj�3l�����V�g<�?�@W����rY�p���`]F���H?e`c*`�2�7���"�M~����[#(|�Q���}C,�ze���~y�� 
$�,w����A����������>~w��������F����gk���?�����Y	����IITC���:����oy��p�����f�mm�	�"'u������h{��L=A�<my�O��UV�>[��~V(k���>�t'��/�������_~���re����_}�����^�����_3���?D��^!�r�n����������(	6�,��C�.�q;|��(�\�Oo�3�����em,���������(��]���������_����'����1������g<9\��(�8��#W��#��)a(���������O�b���q�����=%)���1�1�����1�=A��"m  ����������}q`�ac������o�����l�����_��9N ���/�PK�W4��2Q�PK{2K18sept/pg_part_patch/7_3.outUX���Y���Y��}[�&�q�;E���X!d���A��Zy%ZK���'�h�ANh83������<@]��j����V!8�=����A"q����N������?w��7���/��}��3�/�~���W���|����_E"��f�R�xe�������W��?^����_%����/^~������W?���8��$e�+U���c���}'�����/����_�y���������O�������?��x����X����i��������,}�2�G��_�����Qo������v��N�/������?~�����x�����x����_��������6���~z����|�|�_��N>����~��?���}s�U�o��������������	%O��+�k��
!�c�!�h=`�(��������wx���������w����7/Z��>�^��i?Z�3S~3�U0���H�����SH(�o�n~���c~�������d���Qs�f|�a	c�
�*�G"���%y�]�,e������0�����W��?�z���������|D�(�?���_����?����WoF���A0Q������V6n������0;��?������&������}#���o�� K*������>|���_�~u���������n�{�����^�/��_N�R��_|��o~������o�������^�����o�����S�w���o�������������;���h�q���@y��<��{���������#���%��=���������k�=�M���q_c�x�Q�`O'��'T�ay��L.<���Q?������w����+y>������{w����#�HhSv�hBt*
c�����X~��7^��J���f��K��Oa�yU�6q @>��u�i��:�{�	%F7m+I���U=V[^���=X����U�����x����������/_��{�����G����J���2a\�N�O�\d����Xe�\4ow/aRr��F�OH"o�H�o���m������o{��yL0���NB�x�l��;����N�8N�����Jcd�.>��0y����'atx������?~��N�-!�(����p�'f�X������6d����y\�7v^��'��%����X�=j���$$v������WM���Y�x�T�~ZQN������#�����38�l��d#U�cO�rT���:�U�Y��y��~O7�O���$�&�X	�[��a6�G��{K?�A��%-�yk��=j����(���u\M��b�{L���#L�#>������;5��� zL<��*�]C��QA��$����p*��#��~O�f��h,�����p��?��l�e������L���X��������f�����M�fl������6�`u;�7�|��uD�W��]�n�A3|C3I��MH���d��;.�/x��S����u���|d���<K,�d)
�+�/�6���I�3���4#G*?��(DYH�hFN��h����]���dZoG�f\�f�4�niF8�� ����$rO��w�����L��h&N)n[H��������4�nif#:��$��l�f����f$&�jJ�P���f�P2�fB�*T�bY@{��o��?�f���I�>�)���`+���K�
�2NC3�-�y:�8/g%#��L�H3�4��1E����D���'�	�CS�	���e��xNt:�����#h&���l��oA����DF8Y9�n8�q���f4�SRN�o5L�H3�9���xVl�1�LT<l;���a���B�5��y��`��`�x�����[�"�T�/��
�2��2���v,c��/?3L��2�9,-��>Q��e��K��A��Lu���J2������?s
AD�����#�&��s[#���y��<�RC���2��8����
�C~��V
��VB8D�!#\f��NY�� ��@�$��=j�Oy���,��0G�[9NY/�NV�{�X��. 1Tl����e��?��~�Znd6�O����P8`�V�%��2NC?6R7����+�J��}�~�?�j�]�~��O	�@?&�����������D1����n����,*rqQ.��U��'-���#4�fAc,[(�Mh�$��9�s�����Xe��}��~I���v,�8�-o�'��i
D~������1�
�h���i����CV��1��Y�**����a"�6�d���5$����������>
��9Bbln%���b�	-B�b�pH�XM?��8
��h�e{l.U����M�}��������� 1���X��c��i;������
��)���L�Kr��2c��]%����4
��9Bhln�������&��~�����V(�Xe��~��B;�M��P��)4���7������ ;���5�G���7�-��*�������
�O��+.�])�T*�.G�����?67�c�Y�w^C��X+���{.�{u��2��>�a�c�����:UP3�#�`Er}U�]�}���[AB}.�t`����J�2� ���T�����$���V���_���8���i����ds�K��e�&�HbCu"�����U�Y������a�nXY�V���G+���B�Y������Jy3H�\{�����~dq�;GW������ZO1�
/��S��#U���H���4�������+[�oB�RL�;��{�������'��������5�ya�f�#J1!�<�.�>��� �}!+�����$|�-��A���W�ul���6[�0+��s��������!c672fs�j�U�H�?o�O��`�q�	�������3���ZI<'yj>'������$j�
�h�c%�a����d$��(ga=?�S� T[�JgsQ��:�,��=$���o6G���������.
����
���V���~�^��|�
��"���I��o���E�gA��$$�9�~��>��v�C���5������v�����P��������.�����!x6��gv�4�&���(���*V�g�U�i�'�ng�h�������,���Bq8�.�>��� �}`��`��������J�#�f�3��q^1��G�Yr���X�R���~lC�l�<�[�3K�E�	.J%PB~���b)����������������/U���-�qN�*_��5�����$�4�|�:�v�E����%�A�������{���B�<;Ih��v����6����]P<�W7�E��|g�I �Vg�a�~�g9���X��H��^I�M�;�avQ�YR<oi�~�O8�o��H��O��:��Sr�~X#?U��Q�Ug������6$������<[���MpQ���G�i
��Xe��~���_��s����2���y�]�~�%��@�����W��C�k��`�k�X��������z�>�~�;em�6t�{�OC�l��<�[�3?7�E!���SBEI�����O���Q��>`��.&M��c���g��e��6�Sw9�Fyk�5u�=��"y��p��z
me��<;r9�q`Z>g3���N���4$������<Cqi�&�,E#\
kg���Xe��}����(o��-!N/nV����Uz��Q����� y��Ma�lcBp�o��������vs�}���h<B(�����r��]D��!y�GH������u�n�Kx�T�5a�*S���
Vg�����KON?yZ����ONuBR7�.J?K��[A�~I8�s�c�����G;g���dK��0?���&���*�����OC�l��<����S���E�9����R(�;�U�i���~�g$*�SE��,1�.�����0�(�,I����I�?���My�s�^����Z��$��$��Y�:Nx����mH���g{+yf��dkpZ�\e�n�~`�q���}��7a.v��C����l�(�g����f������AJ7��x.�
��5��K�}��Qy���u�U�i��[.�������!y���gO���	.��)�k��v��2�B?���}|�3�xU��������D��E�gI��$�hG�{�U���^4|AtXy����g�$��)���\����n�k����!y���M"���E�Z%HMB���c?��8�8�v������/��zl��K�2��g�]�}�<���$?mN<�^��G>TI<{9���g�L-\{41�N?��i�Ha�!5�t������4�*Z��$��/R �����������'�ia���25�M�+�@��k�����I A����
	-�|�
Jg�O�
�"�G��c���r@$]5 ��e�i(���3-(�Q?�����I�o��Y��=���*�,���R~��`:/���(E,6�qvQ�YR<o	G���^N^�}=�����^p<���5S�?���9�y�	�6��rY%��~�g:B�L��gN�.�X�����`BZuC���U�Y�'Z��kC��0�~��q�+���uV+���8�(�,(�7�4v*E	������J���q���6h���YV�����^v�����u�#n�����I�5�v]E�&y�#A�cRxX��\��8����#�q���ja��b�sS�avQ�Y���$?���l{������s�r,�3?��>v�>2��q�	#S�o�]��q�NG\������^%�	-2��F�x�%��\<�2��>�;6�J
�d�V��p�����[p�]�}�.����	`��}�{�<��������=�2�����*�=���s�#�����]�g���Z�M���:����Vga�}7�s�s�'>�i�v���)s�i�]�}�����4$~u�a����[$	b��w���nn��%�s������G������t��;�^����z\S,J�>�T���
Vg����;��F��2J�f�������E�g��}HF�J����8~��^B9�A!����|jv
N�i�x!?���������|��;�^�{y��6���>���e���5�����O��[�i�)�����J�ig��S�z�2�&�<~_��e�U�m�m�Nc��xH����&m�c5jV\�]y|�L���`�w�|��;?�s��.��
���;n����~`�q�q�tcRz����q	�����P{���g�]�}��7��b/x�F�T�;�{�>:���c��5�g��^g�A�j-��f�wn���7������uF����`�e/�b���dv��*�,�����y�������*�J��U��.�=��A2�+�w��0)�S�9�?:Nr�[�A	,�L>�n=�N:�K�;7���1�u��#�7�eG�SFq��~`�q�a���4Ye��'O�|�iF>r@.)�qvQ�Yp��#t���_unkj�)���9����8��m��mT�V��=.���0�G8�����9�b�U��}�u.���n?��8
���U�MWS��5�������RS�� Y�f��p=��H�J-�X��Z}��K\�>x�'�El(���q�}���=��a0�G��������Mh�����R���Xe��}��w���^�A�������Q�����.�>K=���dK���N�o�Yh.���M7�!�`�������DLT��&�K���t�#���X�l���5��`������h���aU�Y�-�z�Y���`[OA��u���0�(�,�7��Q��*�,��W�,��O��:��`��W"lpI9K@�+���w �����:���,��&���W1�����1����������OTqdL�
�&��5����E�gA��$���$�����������X��q:Ix�^��=wC��C���d����<7��|������W���pK ��<W8�T=�
Vg�L7��pa��iU�MM���Iy���evQ�YP:o�*B(�i����45.� ��\��go��
#zt����hYV����}.�Jg>B��Jg�����E�����;p��Vg���;F?!�3L�jc��"Sgeo�X>�.J?J�� q)���#v?{���e]����v��Y%��%
���j��$I�q
��;B��n�����mh� ���e��&�Pk��Vga�e�e5y7-,_'T}����R�V-�k�����I ����������f��U��T�]�B��"����������Uz�=2?��tvG(�����^Si�:ZYe
72�Ak���2��>�S�47��)�.�y[_�������}������.�$��gsCY����>a,2�D��!Jr��Ch\�,t�jQ�}v�j�Rgw���-�S����*Z6�6���w�*o�g�U�i�'�~������z[����������E�gI��$��q�c�g���z����0SYF���K>�����ecg��x`�+w��:�#���V�L���
-B5$�E�i
��*�,��A�������}����[�$+
��8�(�,h�7��
�}{�������2�@:�[ ��V���cd��^oh�V��%�i���bgw+vF�"�Mp�k��Uo����)=�*�,���w����W�����������evQ�Y;oi�!/4)�s�)�xw1��r�L�k�de+7�%?���s6�"vv
��;B��n���0�	-�d���Z�@��*�,����v�B�7-,_�p�ig�k{��L������MyH���n�wN��p����������$	\s���R����=�}\C���P;�[_����MhRdTJ-P�[7�.Xe��}�I��U��.CN��TA��G6t�?�O�]�}��)o����D;{������
'��4c����B/�(�D1&�D��4�����[�u���g-�h�%�AE��U�Y��������-60������<�DY�\fe�%_�� eRn��+���<������Mz� ai;���+v�\z�ctUOw�ro���jgw�v�_�(��eC��Tg[��N��*�4�C���w�VB^n��m.����0�(�,�R�����3�,� ��&�]i��N�sC/"^)rwI]��PT�b�G7S�;�#���V���M-���"����M�����U�i��w�fjr�iZ�P�&�p�W��U��)����'��d��$a��m�Z�`���jE�#��f{���fO�d	�b�l�}�m��;����!v�7bg����p-a&y���(��u?��8����L���M���*%��f�EP =���>���'�d��	�
mv���0	��9[��<:O������Yg�2���r����7������;r��g��):\�`��A1�~`�q���)7t<)��7/,���%�y��E�]�}���[AB/��uFs���>�������3��'�Q�^�yE����s�{4��
��?B��o�����	-������B^�~`�q���g7AT(N�j�X�d4v�v'�����$v�
�����p����m*��2S�W�#��s��&]�E���������!v�G��������E�g�"C��O�H��2N�>��uR������NU��M��.�s�i�]�}���[A��l�j*3����(��>�H �w
a��� ����������Y����o���bg+vfX��MpZ����
W)�������n����2c���Ta�b����2�(�,9;oI�kB	�O;;�Q��K&��|,�H5�4L(��z�G��b�K��oh��Zg�X�lQ���+�u�$�	���%�$7T���U���h�]�G���J�b�"�)�am�N�O�]�{���[A�������������HY�~Y��;3�R�mPo)��L>V�i�iH��Rg+u�F�PfZ�B��p���&�~`�q���v�d*Gg&��������	fTH���E�g��yHH:����wv��8���Mr���������B^t��J��R�����7�������:�F��Mh�E�S�������Xe��}<������{j����t�}PTr��0�(�,I����!��=G#�Z�^��3��b���w1�������l,'9��}�wq�
��?B��IG�uQ��pZu�����JU���`�q�q&����c�;�:�1�����j�]�}�����$q���+��Nq�.�:_y	P�v�~Y���s�]V��
*����Z ���4�����$e�����U����T����T���~
u����U%]����E�gA��$yC��(��=�e'�-6�z������0�y�#Gf8q���8��|�
��Ci:�p��9�[���4oB������|��H�q���/�K�iZ�w�2Cy
9�:��I=�_�'�dU,]L��&����b4A����srkS�7���v���vU���b�"�p��9����rn������A)����T������Y�?M��Dj�|Pcg��g����f��&��f�������������{�I=N��0)F�6����.QOC��P8�p�r��MhY��E���JW��l��8�8y�:f�G���*���X*e��qvQ�Y�7o��~��������� ��`����p�T�x�N�d@���+���8����9!o����dZC5�<I�Le��l��8
�pGG
Ri����$��%�����avQ�Y�6o	�\����kg71=R<��\Y��i�F�*E�J�����*�C���94���is����#���� �^g�_��6Re��z�W�f��j�V����4s=�{�-f>yvQ�Y�5o���m�q���:����N�fC����/��m	��p��Z���b���6�#���-P���6�%�}�'�A+�gtC���P�����i�'��*���'Ea:W�qvQ�Y�5o����B�
�����{���C�S)S[+�o���"RU(�������9�k�1��p���J�{H|R?��8�H(���'(�i]�����ey�]���r���V��s��=	���}|�K�C(,�=�������=[�|�L�����9!j��z��Q������H�v���)�C���PO�;��|;2Mk�5�'Ze}��2�(�,)�����'���rz�dO�	��eS�s}R
�i��)|N������mN��F#4��Es�7'./���MhA����r��u���"U�Y��]�v����)�\.�v��M��=yvQ�Y�3o�)��P�Ev�W���$��*�c�J�'��?��,����9�B=
9s8B�n���,V��:ZA`��Lv�=�*�,��������i�L����/��avQ�Y�3o�bs��=�������u-F[���,gN���f�R.�3&y�:���0��
9s<B�o�����@�3hI�
u:�����Tg��R79�
�+.3���,���b�Ur�?��I=�_�'�d�-�����Y��
���z�%�u%������*�&�r������t�CSr�x��9���dZfZ�p(��5E�m��T������$@�c�����m���{�]�,g���R��a�V�tiCC�-��|�9� �\����u�B�)�A��"���r��^n�.QOC���3�9���sS�U�rrQ�q*Vk�F���PO�w��Si0�.ol\��l6�.J=r�� ����pE��������Ay�b�!�����r�G����������3�#���V��P�6�e���<��<�W{&Re��z��l=K���A��
h�m��7�X5��E�g��y+H����K��u=�P�cYLsEd�����0�����+���HC�@=
9s<B�o��1lF���P�ff1�~H�q�1�go�8����qlR:5�"�avQ�Y�3oI���
-�6�o��h8p�L����M,o:;��f�AE�mh$��C�r�x��9.��q<������I�8F�*����*�,�z��2aR��:j���hQ�B��<�(�,����D����&M��'��9��f���N3���QGI+G=�c$WI
]��">6���5s\P3�4��-�����`q���2NC=������,Y�I$4%���m�N=�.J=j�� 	{p'bcJ*���)���o^^
*���'.������+�+3���=
9s<B�o��
���khAS��(�bu��l��8����n�����/\�O������<���E�gI��$VF\d�eU�L=C%v%3���#�]3Cs
���\�9o��n�+���9!g��r�h�D����C��DN?���2�B=�]�".�h����J�R�6�d9�K���R���y3HF1�6:�L���
I�`�1_���b�����X�"sgl��Cf�i���r����������D=Y�E�K��!U�Y�G|�".� �+�P���v;@J�nOTf��ew�m �B�=e���|�N��P����,�J�����QAI�.�l���1|��ij���r�t���b���h��}�T�v��Re��z��~�.������4��`���}�]�z��O���,g��������������fcq/����f����~t����3�#��i���b]w��VP�rT��\G��8�=w�3���X��i�Db�t=� �)��R���y+H�#1G=�Kp�s%E�����!�3�3G�+������'�"gM�	v�����3�#���V�]�6���������%�R?��8������0f����N��fF�.�4�0�(�,�3o��L=B
����BB�#��*�1I��f�(�K�z��=p��s~w���3�#���F����U�:������fl�Re��z(����s��Y��B%K��p�����$g�

2�).����+]:q��T���nN�/D=���|�9\�8�%�i���r����,����W�"�S����BG��8��~��n�)t���M!�s&7,��R���y+HA�I�~T�I��k�.6�8�y�GM��l��>�k��]n�RC����3�9�3���MhA������~H�q��s=0mV��I�����!gf��9�f�Li�-��S���K=pBl
��s�(�d�o�"���~uO
��aOj���r�t#g��'���khA{�s����5��*�,������I�R|��z0mcxF�D6�/��R��9�f�����M�U�;G=��L=�\U�4��f���v��+��W�D�����9�fN�jfT����hi]������!U�i�'�~.��M�7�ZU�I��H��R��<�(�,�����6�����{{q�!���W�
��+�6�����p���%��P3�#���V���n��|�P?�~���Re��zd�w�zOz����v'.9l��5�.J=Kj�� �
&�3;�����Ru�Mv��XgV��l��Qi�Z9PP����+,5���5s�U3���,)\EK���r�:�d|?��8�89������yZePW����(��yvQ�YP3oI��.d�T"��[������pP����2����<<�U]�����s�����f�|����G������)v`��#y	|G��8�x�~�Q���+O���.����3��I?7o��P�2r%�����k���I���7)���aEW��2�:C�W�fF�=���i����%�f�x`�8�|P��)���������=~�H�z�B!���w�����U�gI��%���=���1[m�4LNRsQ��n���)W�������G6�=�3�n����#�gI�����3�YE)5[��W�V�����OPnnE�j�kk|����gW��%i�V�,*�@?��.
���K+c������Eu	'�f���kI��9t�d��]���m�����fg������g�e;�'�>�������7��d��z�>F�o�x����J�'��J?���(�7��&����3���r��vq����W��z�+��qL���D{X���7�G����h����2�V���*�,�=T�w��d[�1������~6_�����z��$q���S���Bz����IC[��|���|��3��X#�D�*��(v����_8�~L�-�m���d4jK��U�Y��{�G?�����WIU���f�hx��U�gI��%��m�K����b�����U*Cho���y7�|�m��/sl�������_8�~n��Q�c�	0d�r�G^}��2�B?.�nrC9���O��65��G����q�]�~�����$�c2����������}p���^u���[q���\>{o�����J�=��!v�������J�U�xh��MT�L?��8
�D�]5��9��V���:&@[������w��^���JR���O���di��J�n%���s�B?!�VY��03�!x����[�s�2���O��>�����*�4�:J�e��)����"nwf'
�y�y����z�����
%dL�z$A�~�#����}5[-Q���FbS�n}g������|�d2���d~�g|��y,#�FN�E�:Z��9�)?��>�2N�<&o�]�>�H�9:������-cb��.���8~*bI�8O�c��;��;��{��<�?#cV�PU@��3��1�6�g��aZ������"�x�8�`�h���S$��_�����-���ge$�x����V+O���b`���,B�����e?|z���������o^||��Ma�7F�����f��-����h�]dFn�{�k�%S�{�R����</v�y[sK3nm�	i� ?��3~kG21�����'���-O��f�	�P���8�
e���������7�������o��F��?�{����_���ww�w���o�������78���[�&oBNz���]��l���!�a�5N�tB����Oo�$������x����O���2Ix�)zTp��~�2j|�����?�|����xu������,��
u���&W����x�i������O'?�n��z,��i�Ex�e��~s�	OSG�[���D���xp�����^�x��������]n��,������_����O��E�8X9�xJ���c�������PKv���2b�PK{2K18sept/pg_part_patch/7_4.outUX���Y���Y��}k����w���f)b����c=�]�����+���|R�T��0Er�����<@=�}�j5��U�p����7o� 'O������?���������f�����_�y}q��W?��xw����>�:Y5����+k������6w�����?�:����|������>������Y�U::���_����w��W_��~�www��������������~z��?���������`M����Nk��'�~'f�����������������7/~��/wo����{yk��{��������������>���G��������x��wZ���?�z���J~@����/~z'����o?}����?�������a���������?����iJ�H������k`�THz4�Q)��R�`����������p���/�����Ww�����������?���G1�+y7����Of�'�HQ���������'3��"F>���d�����������A�O6(�jN����[��|R:,bR�����Z2l���Y��o�oo�����^^#���}z��/���p������c�����F�/>�x�?��z3Z�j�&���������-�v1:�l9R&��G�,�c�f����oo���������He�������r���_�a������������Y��������o��������_���|�O���k�������/�u��7�����[�O����������������'���h��.�[1����
���������FL9|�/Y���)/��|����kRF�x����������\��8������kc`&����U��O��w�����W�|s��#|������G���@%y�������x7G��L`��	���&E���&Mh�(X��l����z���x���4���7���Z@���K�?��Q�Z���.����<���������������t�owz�����7w�?>�L+-������M���U!��<�FE�~�*kF��Z��;A9E�����"�eQ`����Z~����_��c�x3��e|s���o��-kzz���
,���'_V������A�=����t#au�&���������;q�������������Z���/~���!�o����~b�C��y}��4PG)oVYO=,�D�:��]2A�}�x�M��{A�_��`Y;(�������c�c�@{�>� ^�d-�
����J��q?c�u���>�+7�O���$�&�$q��j���Q�u�����f#�K���3-A�6��������%�����i��[�)Fq������� �z�����C�q��Xy�k���LZ�������`���9J��w�t����En,+�����t�g��u�1>�]��n�f�u�N^AZ�L:���2��kyk`�V����QY�����E.VA�.�
7`�����|�M�����[��v�SY����u��j���T������a����0�f�V!�
34��T~L�P��G�AFa����e�}J��v5P�5P��2�e�
��P�E�>���6��T�iP&�^(�s7:3Q<O	����"��[��hy��@�\�P��n,t����vf�D�P��FP3�0#h��
��G��������[,e��Hb,�MR�����:
�P7g��0���asE������%��*>z�^���jK��v�$��s1��'Kr��3�3��	70���b)��9/@Q�s��Ne�f�H;�1�PZ�����0�3��(��d��f��9������iC���a���}f�3�3���K`�X
)�s�)(�����:����7���������+�L|��#�XJ�%�lg�(\���2U�)�f4�<x�>sl��Jf���&5�&�5���m%r~��H;V&�V�xB�����:�x��[2X��4�;���#��L�cR�����h�;-$��I^R�>���zX��>�KL Uz�f7G�k	1y��@�����������t����e����}Yn86�Om���
�$��H��VT���m��N�>6R7��������}��-7�c����&�<~^�d$�q���`����U!��s���G�R�}$�1�V��l�G�e�bs��,��M�)����N�2�G<EA��Xe�}���}�i��`��i��������N�����.�>K���F�d�G�
���}H���/�F~�|���y�!����b���s0������>
~�9�_ln��^p��&kYt�q���|�������O��[���&�i[]����'���2���E�g�_��HZ<���%�]������t�8�>����v!�JAh�#/�����8���4X�����a[b��d-�"=��Ev�r?c�u���{F^���"�=@����x��g/��.�>���F�lB��yI(`r���4�'��M�#�����U�]E^L�D^
��9�|ln��y�:��f-29��s���J��U�Y���}(�$O�*��f�j�e6�.�>KT��F�G��J��b|n���+��M7e2|�6.����/B�f�`b���i����dsCJ��e�&kY���Gn\g}?c�u�a���}X����.������y��E�g����H�}l�y�;wvZ9D\��`�M����~�yPV��3�V���8i�ip��\es�U�^��Ql�3�� �1��*{�)�3VYgA/�t7��s8X3�3,��}��������E�g����H���	E�?�o�%���GF�m>�o����#�����e���=Z%L��l��0�3X�U#���(z[\�����R?c�u�	��Q��c���d�����s����(�,1��I#�Y��TzH��#���������c=�K��������K�vg�.���@���Am67�f��u����\��g���J����*�4�C�O��U�C�PE��7�����k�]}������P����v.��HvfM��1��i
��3�/��
du]��]?
��9��ln��� ���Z6��o7n���H���:
�$�-����&�-����F������(�,��7	��a������a�%����2�/�4os	�$6�\�r!A�z��p�{��m���lg{�vc�x����Q�����K�3VYg���w����q�V�������5��	?���'ip~|bM;3~X\����Sf>O���KFyD����G��	�E��6�����]�;KtLq��$���K ^��l?c�u����xf��V<����^!A�m/�����Dw�j$�C��3$�\���Z�eR��X+u�J�92)���}LA������u/��;�#�������~M���`p�<����v4VYg�'��c��h�+�dby���v��e��6#	��(��U�]XQk|�`��{�8t�����o
P������9�?
��=��lo	��L�E��"��1~u/��b?c�u�	r0:��";m�N���->@��v��e��6#�Kb&��B����|��3�y�\�7��v��`h[C���*��4m�v5���x�G0��-���7��H�:��#��?��~���:�0s7����|�mE��������S������x�l�q���4k����$��Pv��u���nv�Y�44�R�g��\OX��th�g{���R��o��\+gQ�(��<��Xe�~B�~���d~��:Ym�<�h,R���������H`Ca��|E��6�Fy�\d~�f�y2/`~4�k4��{
�JgD�~�g{��.p���f}�\�����|����g��N?��K=#U9��b�����x���}>�.
?K���F��=Xy����[�|�1'���P�ahi;�,������e�LUK�v�1��<�#8���������5s���\�,Q������U�Y�G"�~o����q�v�;4��)B?ywQ�Y�<o6:@r?:�:�����Dp�����|9�8�7����T���\V~�g{�����!�&s�1	�;����{�������g�&/O�?~>Y`��hm�Wd��[�]~�H�[�d:�~@T������/�+�t]3�H���_�@'q���W�K��Az�G�����sp��3�"H�[���u�|�g������k������b}��&��g��awQ�YRx�j$�1$�^�K��GyS%����X+�L��[���!�6�X 2T=La�!58�t��8�h
���������E�~�*�,�jxG�If:X���L�vS�������k�����IF��"��J����^cf�����I���z��)b��T��\@@)��"���<��gZ�<�1�M�"�0D��!zq�t?c�u�'����#����X�d��y����������y�j$�6��x���}�^F��XB/(�ud
n\;�c�����B~	���%�~�g:��L��gN�n�X3�����B?%����g���?��nb>����*���t��+�m�����y�����*X �}���	!
��b�
9.�
��tRP��
�������e^)5��tD��n���f�������)��y����*�,�H��,al�	-�:Y16S?��x$��E�giH�V#���"��7fg���('���KaN��j��R�%��r����+�U4��?��;Qv���;��&��\��,��(G��Xe�~�w����SC\��V�|L����G]n8v�����V#
-6i�xg�1���*��L�r�0�j�����
�V�����c�i�����;����&����K�	�leZ8���g���?��n�g���:�a����:�U�/J:|��>�HZ������
F\$o���	�=�x?N\�|��\0�T�~�.j?����uw����r�6���AV �)��g���?b��uwSQ��^��m��k3���awQ�Y��o3�.�����v�
_������3��O�Y�|�-����QU&]3Y��n����;���<�f��HN�
`�G�s������O��[�i�)���m>&��3�_
�ywM�y��>�H�pS���%9���1�c<d3��\�hW�~t���2�E���~�yn����;?.�S�$j�f.b0��<����g���?N�n�C�t'�&�%��-I�?Bnc�]~
���d����w43��:�
9�$�Ce$XW��%������-�L��o���s���G���q���uF~�b-�D
��������Ve�|�*������9m�CB-
Q�z�1^p�]|������`�XwV�gR���s��~t�H��B����2b���SM�qf�nwnH��Rc|+5f]R�7���G1~�[�~�*�,���o�i�;z�}������~/V��y����� 5��H�{���y��b�tS#�Oud����q���Y|:=�^�f��;7����1���T��t�j.r��gxIr�~�*�4��:v�����^<u�����(�,
W�>�43���z����E��R}�'�h~8��C�����G9Q^�����#��xAj�k��&sf0zt��A:R?c�u������?��������������E�gi��V#���b:���,�"��$:��!�`���
ff�1&�~�J�&�]��
�3�x���g����uc� �b@����:�g����=^.���nn%�8^P�t�D���(�,�7i�6�}Rcm��g��@:lq���{�Q�L�p�Q[���yy�c�)7��|��d���m6Y�FP���R��&>�Xe�}E�
}��#�`[w�=q����@�]}�����������$V�%�19#a�|�+|�BF�2���*����������;��Y�Yb~�\<)~T�u��&���U�Y���
~����p��U75	�������t��P��G�g����H��J=y/������������Im0+����N�����}j�
�3�w����S��g�EP��u�%������O����U�mu��v����C�+�����y���v�:c�`�(���^v����'\�C����D�{�hw_�q$&�}\�����;�[����m��G5wx�r�1�����:���/�,����`�:�������������>���'	�	��*����eU(*�!Ee�yUa�������^�nU���{�~\�����;�[�1�^���n.��*�a�(�x��������S�p��<���{�=[�����s�]~�t��	�9�5o��0��b\�,s����yJ�g�C/4=���w�o�tgw��-�V�������0B�xN�kj������������v�c�{����Z90~��E�g����H�pA�Ne�.H��D	�8A�R����+I� bJ��*������tgw�����!c�Y��#K�GGL�v��U�Y�G��ntgh��3�k-P�u��	������E�g����H����v�� �l�(�<%R�Zm*o_98hz�����%�����;�#�����,gI��d.B�j��r�i
��U�Y�'���
~�z�m}L���g�y����(�,��7	y����/�.�i~���2�4W�}��]�I��<V�%�@��t�l����lgw������vS��g�E�k12�b�g����>��n����1����)i���V�������<Xy������b��q��^TZM����l
����Cq�����hr��=D~\�����;�[��|��M������L|(I�lc�u�	&�SX�|�!��mUP��	Z%��E�gi��V#
|��v�~�Fw�����>}�:���dA��M��c����T��.���;�#��nA��*��uk�$�cM��������:�x�uz���������� q�u�3�.�>K��[�$��6�&����\?r1�i�&�V|P!		##g�=C:�Rro���tgwKw��Y1V�e�p+mr��F��3VY�A��>>?���b�r3�����qwQ�Y����H�3E��j��i�^/9���2���b	"^is�8ntY��
Fr-��~����AwvG���-������������ �,.$i�������8����������M�IV!j:�����$���H$����U���&p��h/����5�"
������S�`���.z�=���Aw�G���
���T�W�e#���Op��������k�O\^�4������j���
��j�]}?�O2��������c�1�]��9[�^C(�|CAG4�����PgR�.E/�`;�#�����l 0�6Y���e���}��Xe�}k{�
OT�I	�5{a���`;�����y��P�G�=��d���&~Bm�;�20�����i��`hV�P]���1X�7������e;�<��m����F���bc��Xe�}��wC��{�muJ�Y#��<�t�]}���[������c����gh4���^��m]
�)**���:��}"����v�lg��A�����������R?c�u���o�2}������T� ���W�]}���[����f�1L����B�����7�%w
a��t�\�b����:w��!��dg������li��c_I
d�)�3VYgAy��i;[e���UPa�yem�awQ�Y�v�j$���>S����s�N/���sy52 GM�6Sr�������u������7������u��LR�b�"�I��:�~�*�4�m��{�������vW�>ZAbR���b��y�����"S`��dCStR�<e]f�����E�1	�,+oS�%���:�#������%&�f��x��O���~�*�,���6��'���Z<��!��Y+Wm��b�����,���H������;�]�l�xE��������������]/!X�����=���Au�GP��-��%�6Y�(es���S�����:�xJ�[���i^��V�M�!�Z�B�����Du�j$P����W��0wL$�����]��
r@�<�?0"uU�a�E]�7�����Du�EA�&katN�����[�3}���:�8�zS�����\�`�f���e����(�,);o5��f��0�c��
`?�@��:O����N�3�\	*��T#���4������e��V��UkiH��:��J���-U�y���D�F���*���[���s��r��@s�l$'gyT��~g���z���qHc��,��A�.C��{���`� ��9As�zH�^�d��^D����Y���@��~�Xf�M����dz�~�y�i�]z?�O2�nqMbB��;���b��aD�����a��K�������j�|��.���qGp�����S5Mg�Z�=�R��a���=�_����?m�X"5I>I<�����]z���dT�[
���>�@���$���Gn�O�g�J� y���x����4���s��S.n���aid=v����:�8y�:f�G���*�����9�?�.
=K���FW>3|�=���]��b(t�L%r�q����!
���B��t"X��$��p�9���Cj��ukI�������~�*�4��5P"NMA%�h|�����w��j�f#�b34VE�����=z��;E����i�=t��	��i��V%�=����mGp���=�(�M��
�S��V���m����=>r7g���t�L������� .~n)v��%b�f#)1G�Q�b�}�w������������
=�7rT('H�9��*��%��`6�#���-@�X+m�H����h@����:
�x�O@��~����+��r���%���(�,���)����v��������ZW�����y@�.yf�$P��f���jh������CFW^���,�������g���=�t��	
����T�M�� qV�c���������HN9�SK)������:{=���Y�o�M�	����1^�"7��-���=
Rs8���
�H�Y����%!-LE>����TYg���;N,5�:2mk����!��U4��E�g����H�(�)L����]U��g�S��I)��z���,����C=	&�����p�9����7YKC9,������Re�zX�����0s
CE��=�ZwP�w��%:�V#Ap��f
�,�;0�[�S+�r=����3;�T5&k�%.�@O����3�:3����M�besu��R�A��3-U�Y����=&K/L�Z��MgNVAKI���B��y��P�������X�B���*�t�dL���/�5H��T
�������
:s<��o���J���%��!��3��a���=.�ntfHz�K�*'a������������a}��p%@�DC�s�e�����?XwR��Q��
�D	��0�!���4���:s��3����&k���\�$o�Y���@��nM\���1������6�� ����1�.
=Kz�[�$v�$�M
�>��k�=K�mIs�:� ��,3=6O<�*\&���4���:s\�3�3<U\�����#!��g��N=�_q]BW����U���f��
eZc�]z����$/	������,�S�tz0f}�3�VZ�)A3#�%��U��b����3�#������AA���j-�����l�g���=^����i���J4��j����v��%���F���&	L�OH���	z�a�;"�7m���2�\�V:���K��`3�#������fk�u+{=t�g��N=���M��\�V ������9�<�"�.
=Kl��F��pV�@�i�43�p�L5���M,��:���Pq�Ha����l�x�9.����M�� �u]�C�����:����=�L�����U�f�(���3���.
=�l�mF���P$�~�{.2h9\���l���q����SH�� [���K{h���9Ag�tf�&���Xq�=]�P��s-U�i�G����9+u�&49��%��j�]{�����0CP�����{&���o^�*
��V��'��
c���aNa���@O����3�:3G������0Z���B?K�u�a��U��|���&�'n�c���%���(�,���	���?J�P��C�N
\Jf3�&�m�_yN\�a�q��S5q�]J\
:s<��o���8�97Y���y1���Re�z�~M\N�(��m�B��l�8J�D\ewQ�Y�3o6tu&�i	kv��H�N�U3�\����C!W�S��fIW��zt�x�9.�3'z�Eh[�n�~g����Re�z<q�&.s�+����2P���ewQ�YVg�f$���S�%�;���M\������8����a`F�3��U�.�HS�����3�[u����i����Y��U���g[���@9����e_}���������|_��kB����IF�_�C�z'�5�'��I��}�?�Lj����*�GA�\��U�9�"��t�t�9-�3S��NV��V7@�AD��Y���@��s�4s�*����VG$���r��<a�]z���[�D%O����w��<��Z�����AY���?��8�Q3��*&�=�GS�����3�[:sGf�]��8��]���~�*�,�z�3{�,'��%O�L3C!�da�awQ�YRg�j$�R���6�}�	��@�1��zL�������4:��N��P1���C�95���:s��3;P�"o��� F����TY�����=�q�>U�	=��|��v��%:�V#	� �KYs�O��Z��0��'h�M�����^����1et�V�]���9AgN�����M�����C0����:��~��n�)t���L�a.7q��E�g����Hh�B5Wz2;�z$(2��|����\c�]�z,�W��i�
Wj���t�tCgvi�����<�K,U5�?�Re�z|�\�Vi<U�VG$5��k6eJa�]z����DJ3��RH���z��}�)�*������p�xn�b���*�0����9�fN7lf��I���f-y�������}���:��`��zR�����LSc�c8�.
=K��[�$V
���A�{7�p2�DrUo@��"��}�&T����]"��9AgN�t��	#z���2�����g[���`O��"�����7�:V�J���#.r@:�.�=Kt��F��f�N���_�D�� 5��L\�� 9ELe
���W��{@O�����3�[:s�zR���X�X���F�=�Re�z$N����n�PK���$.6e
���(�,�����q2�����/���"��C���{�i��	-L�P
���C���E,,5���:s��3G��p�Z��fu��b��,U�Y�����z|���m�Bm ����,�:�.
=t��F
*"�oA�uf_b4����Aq�33�0��xH����!�YU3�%�2��g��>���:>�x�M�eH}�����\L��)b��N��`�f�+xR��>����s��>�J*�F.c�����
�Y\�81���a���������{V�`����4����%�f�x����Ev�������������=P��H����n��B�0y�]~�h�[����8��'��@|�R�
3\5e;�����3���V�hG���wQ�0��k�?K�f=�'�1��%�Cp������s��G��y�-s���D�ms���*�,Q��Z�(.�) �w�
�CCWR��0�D�=��ST.e�A��P�����lt���?8~�����Z�`^P��g��g���?^n��Co&�\=N3@Z/��}�l���*�,��7[�����������//���I�^�G��g��xMi�����3��4����#�g��l���n�2B�,�����U�Y�|���v�R���������j�]~�(�����cI��7�LJZ�=W�|��
?))�DF�G0������O���?8~D�m��x����`!�f��y���:����~X�y@�����W4%���m��X�]~�h�[�d#>�1��;����)Ew�T��"_H\���d���%�_�~<g���s�t�b;����Z�t}��,�k���?.�nl��'�'ok�������Z:��
?�����d��?�D�������:�\�M���-��6�w�PgS��3�Au�?����������>��@�,�3VY�������F�~R�d��"�
�N+�3��
?Kl��VB<���r�vnm/�Q�S�/��L�w+�g�Yb��x�����=���n���G��-�9+�M�b����^�g��N?�#��I�?%U����`���s�����,���JF^�Sp2xGA1A-�t��j��F�s����;�a#�J.�N\�Qq�)�'���;�����$Ro$n�����rJ�sB��������S ���m��'����q7`y@:��K,'�
�`���^q�z���X\����N�>�}`�15�	6��
�0	��f��k���h�Y���-����j�=�~\���:����U$���������l+_��{��7���W�<��K`JYk�������A���w�^���>|����Wo��}c���7~6����/=|B�wf���|��fV�P~��M�~��\���5O�4Slk[��V�W���G;b��v��L=A�|���?��6Y���b?+�5�ox��g�_���|�O���k�������/�u����U?��o��f��g���G�A����
6yr�{���GI�ig��$4E��/t����V��46�I�>lY��?qk(���
V�����{�g.�����>||��`���h����O��o<:\�r������
mFnB��$�bT�����#��������/=�������$�M�����G�_�l�����6����o��z�C����%DMr����o�����l������I�����8��}���PK���?�2c�PK
}{2K@�A18sept/UX���Y���YPK�{2K?���(@��518sept/.DS_StoreUX���Y���YPK
�v3K	@�A,__MACOSX/UX���Y���YPK
�v3K@�Ac__MACOSX/18sept/UX���Y���YPK�{2K���4x@���__MACOSX/18sept/._.DS_StoreUX���Y���YPK
{2K@�A.18sept/pg_part_head/UX���Y���YPK{2K���j)@��p18sept/pg_part_head/10_1.outUX���Y���YPK{2Kz"
�/�E+@����18sept/pg_part_head/10_2.outUX���Y���YPK
{2K�������*@��n18sept/pg_part_head/10_3.outUX���Y���YPK
{2K��Fa��*@��h�18sept/pg_part_head/10_4.outUX���Y���YPK{2K3��,#)-@��#-18sept/pg_part_head/12_1.outUX���Y���YPK	{2KZU":+-@���G18sept/pg_part_head/12_2.outUX���Y���YPK{2K��c�/+-@��4b18sept/pg_part_head/12_3.outUX���Y���YPK�z2Kf� /-@���|18sept/pg_part_head/12_4.outUX���Y���YPK{2K1���@@��7�18sept/pg_part_head/3_1.outUX���Y���YPK{2Kmd�E�@@���18sept/pg_part_head/3_2.outUX���Y���YPK{2KP**�@@����18sept/pg_part_head/3_3.outUX���Y���YPK{2K��r�@@��:�18sept/pg_part_head/3_4.outUX���Y���YPK{2K~�,~L+��@��18sept/pg_part_head/4_1.outUX���Y���YPK�z2K�$'�g+$�@���A18sept/pg_part_head/4_2.outUX���Y���YPK{2K���g�+��@��wm18sept/pg_part_head/4_3.outUX���Y���YPK�z2KwY"/_+��@��j�18sept/pg_part_head/4_4.outUX���Y���YPK�z2K��.��,��@��"�18sept/pg_part_head/5_1.outUX���Y���YPK�z2K�j]��,��@��U�18sept/pg_part_head/5_2.outUX���Y���YPK{2K�]Z�,��@���18sept/pg_part_head/5_3.outUX���Y���YPK{2K��X�,^�@���L18sept/pg_part_head/5_4.outUX���Y���YPK�z2K7u��7u�@��"z18sept/pg_part_head/7_1.outUX���Y���YPK{2K&r�hT�@����18sept/pg_part_head/7_2.outUX���Y���YPK{2K�����;�@��s�18sept/pg_part_head/7_3.outUX���Y���YPK{2Kuz�/d=�@��c�18sept/pg_part_head/7_4.outUX���Y���YPK
�{2K@�A �18sept/pg_part_patch/UX���Y���YPK�{2Kj�m�@��c�18sept/pg_part_patch/.DS_StoreUX���Y���YPK
�v3K@�Aq�__MACOSX/18sept/pg_part_patch/UX���Y���YPK�{2K���4x)@����__MACOSX/18sept/pg_part_patch/._.DS_StoreUX���Y���YPK{2K��4�����@��X�18sept/pg_part_patch/10_1.outUX���Y���YPK${2K����V���@��4�18sept/pg_part_patch/10_2.outUX���Y���YPK*{2K��eKS��@����18sept/pg_part_patch/10_3.outUX���Y��YPK'{2K�������@���~18sept/pg_part_patch/10_4.outUX���Y��YPK{2K��P��1$@���Y18sept/pg_part_patch/12_1.outUX���Y���YPK${2Ki�K��1@����18sept/pg_part_patch/12_2.outUX���Y���YPK{2K�wi�1@����18sept/pg_part_patch/12_3.outUX���Y���YPK{2K��>��1@����18sept/pg_part_patch/12_4.outUX���Y���YPK {2K��6�;�(@���!	18sept/pg_part_patch/3_1.outUX���Y���YPK+{2K���N�;�(@���]	18sept/pg_part_patch/3_2.outUX���Y	��YPK{2K��!(�;�(@����	18sept/pg_part_patch/3_3.outUX���Y���YPK!{2K�u�n�;�(@����	18sept/pg_part_patch/3_4.outUX���Y���YPK {2K��f,2@���
18sept/pg_part_patch/4_1.outUX���Y���YPK{2KA9�ow1&@��HD
18sept/pg_part_patch/4_2.outUX���Y���YPK{2K��
��/�@��v
18sept/pg_part_patch/4_3.outUX���Y���YPK{2K��V�/�@��.�
18sept/pg_part_patch/4_4.outUX���Y���YPK{2K�%nH��@��$�
18sept/pg_part_patch/5_1.outUX���Y���YPK{2K' yVH��@���18sept/pg_part_patch/5_2.outUX���Y���YPK'{2K3&�wH��@��g18sept/pg_part_patch/5_3.outUX���Y��YPK{2K�b�H��@��~�18sept/pg_part_patch/5_4.outUX���Y���YPK{2Kc���2U�@����18sept/pg_part_patch/7_1.outUX���Y���YPK{2K�W4��2Q�@���*18sept/pg_part_patch/7_2.outUX���Y���YPK{2Kv���2b�@���]18sept/pg_part_patch/7_3.outUX���Y���YPK{2K���?�2c�@����18sept/pg_part_patch/7_4.outUX���Y���YPK::5��
#254Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#252)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Sep 19, 2017 at 2:35 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Mon, Sep 18, 2017 at 8:02 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

partition pruning might need partexprs look up relevant quals, but
nullable_partexprs doesn't have any use there. So may be we should add
nullable_partexpr to RelOptInfo as part of 0002 (partition-wise join
implementation) instead of 0001. What do you think?

+1.

Done.

- I'm not entirely sure whether maintaining partexprs and
nullable_partexprs is the right design. If I understand correctly,
whether or not a partexpr is nullable is really a per-RTI property,
not a per-expression property. You could consider something like
"Relids nullable_rels".

That's true. However in order to decide whether an expression falls on
nullable side of a join, we will need to call pull_varnos() on it and
check the output against nullable_rels. Separating the expressions
themselves avoids that step.

Good point. Also, I'm not sure about cases like this:

SELECT * FROM (SELECT b.x, b.y FROM a LEFT JOIN b ON a.x = b.x WHERE
a.y = b.y) w LEFT JOIN c ON w.x = c.x AND w.y = c.y;

Suppose the relations are all partitioned by (x, y) but that the =
operator is not strict. A partition-wise join is valid between a and
b, but we can't regard w as partitioned any more, because w.x might
contain nulls in partitions where the partitioning scheme wouldn't
allow them. On the other hand, if the subquery were to select a.x,
a.y then clearly it would be fine: there would be no possibility of a
NULL having been substituted for a proper value.

What if the subquery selected a.x, b.y? Initially, I thought that
would be OK too, because of the fact that the a.y = b.y clause is in
the WHERE clause rather than the join condition. But on further
thought I think that probably doesn't work, because with = being a
non-strict operator there's no guarantee that it would remove any
nulls introduced by the left join. Of course, if the subselect had a
WHERE clause saying that b.x/b.y IS NOT NULL then having the SELECT
list mention those columns would be fine.

I am actually not sure whether we can use partition-wise join for a
LEFT JOIN b when the partition key equalities are spread across ON and
WHERE clauses. I am not able to find any example against it, but I am
not able to prove it as well. The reference I used for partition-wise
join [1]https://pdfs.semanticscholar.org/27c2/ba75f8b6a39d4bce85d5579dace609c9abaa.pdf -- Best Wishes, Ashutosh Bapat EnterpriseDB Corporation The Postgres Database Company, mentions JOIN conditions i.e. ON clause conditions. But all
the examples used in that paper are that of INNER join. So, I am not
sure what exactly the authors meant by JOIN conditions. Right now I am
restricting the patch to work with only conditions in the ON clause.

Practically most of the operators are strict. OUTER join's WHERE
clause has any partition key equality with strict operator, optimizer
will turn
that OUTER join into an INNER one, turning all clauses into join
clauses. That will enable partition-wise join. So, the current
restriction doesn't restrict any practical cases.

OTOH, I have seen that treating ON and WHERE clauses as same for an
OUTER join leads to surprising results. So, I am leaning to treat them
separate for partition-wise join as well and only use ON clause
conditions for partition-wise join. If we get complaints about
partition-wise join not being picked we will fix them after proving
that it's not harmful. Lifting that restriction is not so difficult.
have_partition_key_equijoin() ignores "pushed down" quals. We have to
just change that condition.

Your last sentence about a clause b.x IS NOT NULL or b.y IS NOT NULL
is interesting. If those conditions are in ON clause, we may still
have a result where b.x and b.y as NULL when no row in "a" matches a
row in "b". If those conditions are in WHERE clause, I think optimizer
will turn the join into an INNER join irrespective of whether the
equality operator is strict.

If partition-wise join is disabled, partition-wise aggregates,
strength reduction of MergeAppend won't be possible on a join tree,
but those will be possible on a base relation. Even if partition-wise
join enabled, one may want to disable other partition-wise
optimizations individually. So, they are somewhat independent
switches. I don't think we should bundle all of those into one.
Whatever names we choose for those GUCs, I think they should have same
naming convention e.g. "partition_wise_xyz". I am open to suggestions
about the names.

I think the chances of you getting multiple GUCs for different
partition-wise optimizations past Tom are pretty low.

We do have enable_hashjoin and enable_hashagg to control use of
hashing for aggregate and join. On similar lines we can have three
GUCs to enable use of partition-wise strategy, one for each of join,
aggregation and sorting. Having granular switches would be useful for
debugging and may be to turn partition-wise strategies off when they
are not optimal. Do we want a switch to turn ON/OFF partition pruning?
Said, that I am fine with single GUC controlling all. We won't set any
partitioning information in RelOptInfo if that GUC is turned OFF.

[1]: https://pdfs.semanticscholar.org/27c2/ba75f8b6a39d4bce85d5579dace609c9abaa.pdf -- Best Wishes, Ashutosh Bapat EnterpriseDB Corporation The Postgres Database Company
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company

Attachments:

pg_dp_join_patches_v33.tar.gzapplication/x-gzip; name=pg_dp_join_patches_v33.tar.gzDownload
����Y�<is�F��*���/) ���Y��J�JR��J�pEX ��0�]����$HI��U�vQ�)3==}.o��r�!����N�9O:�����k4��9t���������9��Lg�A�6xqLj�,I���o�d��Q2�6����O��&���:[.�8��v������O���Y�^�X�O���� �k ����6���A��}���]����/���8Z�����F�9�&����s�5g�{���7���N�OF�����]�%3G��=������!;�4bom�WE3�����0��29���o���������5aG�-@��u��}��z]�:s>p7=d�_���&��L
(�
(Ce ����Tc1l�����,�����b�<��,���&��0;����%~�r��9s�~���.�{F)s8���X�=�L��i�N�<�X3�)k\s�L�4���M��n���f�`���F��������6.s�\;THe��FH���������FjM�'��~4�$s�D�1���� �V~x��}5Gx!G�����%h���;c����m��]�j3(��n���ktW<��+�V>�-Y��T����,�G����Q6��,��de���m��#$KJ�2Zf���EN��tN�LM�@�<X!�Ht�&`��Ch�>K	 LU
M�u�%��ql���^'Z���(��B`	-h��3��b��$�j�5d��K�/
����#����{m�~����h5���<pqCoj���fL�o�������<n�������iw{�a�c�r�3���H�����>��w�1�g
G�k�O�u�����\|���'!�D���i8��AE��W<x�L�45��Hr0(� hK$2&�r�2-T�t�+�������F��"���P��%�IM��g��������G���������������u1mj-1��������8�~
S�s���4��W�/��Cf��om�����v�X���3�#�zU��&�R�7��@������cE�$�;�[��"5p�i����{�����E��A�*�E���i�}j�V�j��$s������Dd����E_�|
�X�S����ez/���vj�YN-B��\��y0��\���,_H�2pO��#`�+n�`��g���	�"��V�6�D�W�>���s���s?��n@�#���t���P���3�����F.�@������;��Gx�
Jk@����*/Z��
0����iNc0A3�	��na���w���`����"�j���z:+	b���A���P���~��,�<x�"���C
2��]�K�}����C'<���
RXL�Z�E��Q#�90��in��V��^��D�>�*"�w8p���Aa��(w�Bw�%j�����j���~����o���M��3[�k�9��om��6��������d|��u_���_qk����2)=i����uJ�D7�h�_����4�m��z>����v�#�#��k��e!{T65%�h�����JP0�i�12&�@��B�W����#�d�"�'� �O#��<����!�.�,�_��c�o����>��_��.���������a�72���[�xh�x����^�4=����2��w���?�����pK����_�gI�D
��I���z&��'0�*g�g�4L�0W�������#�2z�Rb^(��}�c?�Dw��2f����6,m��������^aPIq
��C��.B'���qG������'�m��}@3�}�,`�Z&��.H���'��)����Q���T�����y� �&�f��A�f(T���IZ���xB���97>�C�R���n�SH���R��>��+��
�>����Jv�t��8m*���l����������6;,u��<���"h��O7'WbG�����U5�F�Di��!>���b�����l��
�B�>,A	�/��
��� P$�
� �h���l�����GF]�!�u>#|�C��
�D`���H���rQpm�KP�oU%@�L#��+�ec������Q.z<��*D��HD�%�����$���|UH! ���`��1���F�0��D��i).U��@?������at*�JA<7�d�� ��\��8��z9SD�DdG�"��'*�J/�X[�S���	\��G2T5t��/&���;��,������m����{��M�'�h�5����(0��T��V���90�h��\]�X"�R��vO ���Z:��v%uZ�2��g�e�q���)��Tm\8�Gx���|����r:$���T���=*���o7
�"��p���Fe#����6���i;-g��e�K��'�;�T�L��M����!�,^"}A�
�(��b��#�
�S��WT,������K����6�,p�;��q��i����B.r�n�ezhy��,��}�'�(Q@��c�!�P�^:���Gs2U [��1���3nAH��;Iz)TO�.a
!�X6!=����LiL��H*P�L	3s��������]�!#�qvPK�D@��^�w[0CX��B�8Q���f�\Wy7Q������'k(��L���\#��P���v/s7M�h����3�O�j/)�&�n�oJVvHVY������c��&��1>#cD���C{���=���T�o$�0���w�$Z�Qz��?��04��))��&�=E�6�m������E+��V>P���r�f��v2g������(�G�.�H��d;?Mx0���.�VJ0���"�pl�)=�D�-�M�N����Z����mt�V�w�!�*]R)P��
������`�#�}���	6m6e`�%�E��j�����X�=p �!� ���}�4�T�h��^��p��Nw�H����EJ�Y�`C��:�C���w"-i&h�sOjY�����B�k?Q1SQ������Q��
sL��������H����b�f�%;Xc���2���O����4�k��mt�`�8�8��4� ~,�A��p���^~�):�� ����f�e�S�����_��u$U�v
k��"^�>x�����F���Q5]���J�$s�{��W`h����
��w����};�>����m�7��Ov<��;!I$.Xi_��T"��'?����?<o��K
b}<���P26��\���b�[����b��kY�a�^��h1O��i�~�-�b�B�����p'��~��-:~X����*V�����,I�4��� W�"�{[B������1
0hV��2`
s.��c��LC�% X/S	����X����`�!2�H���9���V����
����NR�b4�y�1
[)D��'1��.����[����c T�<��0���`'y>�f�� ��(-���^�Ddl.a��w�,fku�j2-���&I�M�-�^�=��%A�&W����6�L}Y����������T$Gy!��!������_#��b�7��UAOO�R��
8� �������	Y����Y@\���
!@�Q��T�]����OU��I�`@�\�c���'F��_��TuqdU0�����E������|������DW=�v,D�V��c[�,������f�3���hq�T>��$��i'
�4�.F���4Y-��X��������g%P%�a�fa�S_���K$*6��@j>���~�*$��J�\- ���Y�aJ?T?��x"Q��T'C��
�e�n����I��M���@	���RU	qzuP��!�J5�|ct@%C��2����J�"Is���P��)�4Z�������`���\��
f<.h-���#�ZA(�@Y��L�	��
�NO>FO���C�=u(	u�q.��R>��J��9�\M��[R���_���;4c���8�
7�[�'8�
7*�<��nu�J��Lw�QF����:[�:T�����`�[P�~GBZp��r��N�	���Q�5�B����W�I����iZ&^���?��+~�����t�\��Ml�$s����B*V1���nGB{����o�_���/�������v=7s�G�A��8����������P<���
\uL5�����?!.���SQ�g�%����������E��iq��q���\�+�����4Y��S���	�fA/!�;;����.����G�
t���<�6�Q�Z'X)���2�C�0����'��Ol��Y���N�t�<~��a�+6.���(����Cs�f�b��]M*�����`
x6��)pa������UF,�
��e���]�z��������?t����{J��<[���N'{�(H�!��$���V��=o��q�z���U�`9���y
�=����aI���	�Z��o��b$\*y>W��H�?,��_N���U�����$X	|���5�_���gi��������������]<E��A�<�Dki2�z2�m�S��Z�����q�='v_������%�Ae9��!�(m����"
��2��
��������[rD��F�������^�M��=��=�R�������{} Y>Gm��D���]Z	�1��
<���B_���d��5+����,���Eu��I0((EX$��V�^����7F��!�x/�L�;z�
�eJ�����6X
����8c�(,L�F����k��Y����l���e{��Il�`bK��l��dKy_���z��cJ�[��Xf�����z)5wD�."����
���r�
�G��n=v2�&RGo�� ���{G�������:��~+y�g�``jj��o+o��|��0_4���B"�2!`����$�F2g��������0im�<����tl=�-evHf�`�#�6l�������%��rN�a��)��7LH�im�V�&A��� ����+g��~_��>�#t�3gQ��
p��RY��iJ�iT=����	�{u�-/&������&����]��E~��o�8��u��YM���[k}a���h���Q�d ����,G�x_��R
�=����}SD�_�����<O~G��f"��c���	��:A�������.�.Fqk������W�5�������D��W7�����k�_����I��������W�?0�Q��/�74��{��k\��Ww8�g��No��V8s���z]����x��V�����2��^w��_#|������OM;�<�m+�^u !7�?Q�g������!o���B���[p~�������q@�;����8c���]L��\�i�^[[f��I���D���Z����9KJ�U���Ds<�o��W
<U��&`"�,��|�o������Lx��[���f�h���+&��r�������O������:�����kx�~kG��(0�'���kN�^�4�~�7q�Y���D�����I��z������)~����{6K�x���wo������3�������*�����a�8ZQ�s��������!)�=<��7<��p�V�p���r}r�x�m����x�;��W'G7'�����	�-����k�m��[� �q����������.~8i��Z �%�9;����m��>��������^��/]����0������i��A��������������n����E��?M��C���?)��-�{ur����j��Q���;Ik�~>:���Z,��6 ��11t>��&����%N�f
�-�i������~�i�4."�Q�V���8��������E��$������A��8�8:���$����.6oYu�c=ML�Z	��bjmd�Z$�e��W/��K�i�!1�j��z11��'��f�������q|��_���F�eG%_���n���������3��y�P2mk*K�(%�n���pp�I�r"�4�H����s�Y���=o4�t�vP�Od���0�(��:u����
H��yL�O�������Q�������^��|�_������:�xLHW'����o��������C�a>k����1-��B�x?iG�����)i,��
��������;��z����*�������"<���>�1,����c�2������s:�1y��mq#I�_x�z�H'�X�'����:�y�j��F��x_�xT�K+@V'&@�����TPYPi�O��)��m&b'���������D�����G������?B����Q�?�����H_B�_Qq��U� L�l�zS	�����6;����bJ�o0����V�(L�S���$k!E�\�9T�GSk�������|�}`�-����c�H[J�?
s�
�BdM��:����,t$���+�*��O�=�8"~�pNQ�`:�9h�*$7�p�Q�e�
���K�C����9�:�����EP��5��������yp����?@Nd�=��`��_mV�|o�����`�j�[����RB{%�����`c#%���N�!Q����2P�@�����p~r&��C�P:A�����61�)��
&&���5�`�x���)�$O��TYM��T�K������i�����b�z��������W���������t|x����V�	��'���}P���t�tB����O�S"��D}���"���m�t����sTo��[5��Po�[E��W�����g�e��^m�R�>��<��
�N��d�4
*�``���@���?��O��K��g���g@t�F��@��v�X������@0�db��Gy�Ac������N	lm_s����J�9��$����xj�=���ySj/FJ�4j[L�i�V����.�����>��A��r���S���< �B��UR�@�rp|��?�I
$���������5�0�
��f%?40k��Z���j����L������l)3k�%?�e<���<FY�11��)	04������	���������Cj9�.��bZN���A��F
����D�e�����6)e3b���p�������-V���d7�0� o^�g��%K��k�
B5�����V�i<~kQ���(���4������)f�p7�4�:d�����b��]�q"��}������4��64�1i;l�-������xa���q����t5������������M�(3�T�".��eEERa)�J
#����`�g]H9�:'�����QL�#p�����MCe���T�I�R�HU���8�`QBm������!�A��b�\u�
J��I!�q�Z)7�`���$fh�����_��S��V�K��<{B��I��I��D�#Y��h ���L��5�1�CL���|����:��L_��<��(���,?����Vi���`��Oj3����~��lD�n#��5�h��b�6E9��]�to[�%G�(1%�
��4a�(�j@�p�
���4���g��]t�_d*�~���!zM�M�g����&�6�l)BW�#:�,	��m������!���71�&�3�������E\� @s;�qE��N�I��6���_����V���I��vn��@=
����;nY���y
�/�E�$���K�������t*�JA
@�[�cs�O�T���+���'@\�9�B��=c#�P��L���]F�	����J��?u�I��Gzy�������WL��zy�����v�uZ~���"�n���o_�h�+����  �0%Q"��`����K�We������V4�g�F�s8�:H�)v
\����������(���X/�y�=S���/�����>?��	�oswpA�Z-��_�N���m��-P�8`��0E0���Lc+L��n6+��5�����������H��`P��-6Ze�n)*J���C�p��D�4�����Ij^����pW���tzU��d�;������*:�&���+���}/���^�t����AW��9�^��^���7:!�'G���LE��z�	���[���Dl"���yq��Re�(/~;K��8-�?�����tQ�UJ�i)�� /�����zg)U����G��U��p�2i�9��RRIPTJ*�JI�/e��S��-V`-��Y����ZF��KGp���!?����*�A��!������ZF"����5�6!��M�-����Mf��P�;��k����P�4�N�$�"Zx��\����#@��v��9�G+�>�v���g�V����u������Y�w1�ME�$�
$�L(��i�|	�=S��$��Be�fG<\QR�xG<�����q%N���9�����i�&�������B�./�d>P�&����s���yMg����0H���:��|�`�O�!J4�}NB2�8�J�F6��X�\a����%�?��ms�����tK����d��@�kM|S�6���!�	�[z{�0����:��z=��eF�g���(���`k����r+�b����n����������^�)4����OVd�L'����m����~HJB����Fn��L��4���/K�.OfKJW��6��^���%��luKhG�%�����H>�m�{��":����C�g�B�����p(x�Y����&(hp��������!4H����0jdS�<�b�
���4"�G�[���k��-n��q��@�w���(_�C�������"Nd\��X�'�s3�=E'������?b�c�&���d�����
R�uDH7~bekLJY��[*T��!:s���=��P����b�pt��F�R�n6�d�+�����}�
|��~]������lM�V+�HAw������%�2wE����gO7�g/�#���m�JqV���>�e��l3X���K~�@P0���q�&!��Q���<v=]8�����/��E`�XB�����@�iP�H�M�N����L��QH�"n����|�L��^�5!g�tP��xs�Z�T��5�eM}g�t�R��xs��Sdx<�5
dMW�4�`��E.v7����D\%�k^����1fk��n4�'�4�-�w��nr�>b]���t�I��{�~�|��������*���$L������u���H�������_��)`tY��h���XE����
��]����B1gZu�\���(��W{��<]������B�G��.�9��Z��q4���=yp��(L�' �cd;S�?]~������rh(JX��-������l���*K�dG@�a���o� �`�����P��8��V�MWN�a�m5���%uYB5��*	��hi6h���H�f�5�qJ#$�< �� 6�B�W�����*��R.Sk�� \�e��p�q��
�������V\��U��
���+j�c2u,�q?��6���
��(��!�@���~����U,���qmZ��f���m5<Z��g��_R^[@�<��DJ����(w�N/{
Da��a�!1��#0�a*����X14��0��T?��4�a
H�*���K��7��a�8����3�[�/M����D�/M����ts�|����|b���Q8�fL�.��i�y��MG@6.r���
�Vq/�WQo������1n���U��c�V*)��I��+X��C�#L����i��*��i�\v���mo�C�|���v;�VE�Lypm�D%����n�^M H^**?�+A�UO$AIG�@�UQ�Um\6Q�F�6�x��WH�������E��}�n�����#�3���g�����N�pXf�Bi,e�]�J��|����<04S��w-\[��Y+�`�6f��D[/�fW�ya�e���P�UC!��a��D���I�+3r�Q�-<\	�U e��x�������.:��=��;[e ��������*�2�M��D�pB��7��|#���yN���*���9�����9?�0``%�����������m�F��L��S��ofq�To��Z\8�[_�{u��A8�i��To��#������`��[����`>Z�ZP�L���:������n�p;��lP�y�q��r�c��
���6+p�u�>
D�r�M�z(��q��o��):e����f���^����?(����n�|W�Y��+�q
����������P�����|�b���G��ny�-����Q�W������"e������
��_�d��%���Z��%��b���yF����>����}-W��~�]6��D���(b)
��+y���������T�7�c���`�-C{�5YDX>���x�G�e.��]%q����?�1U?g�J��V	xO�2:�Z#���g��p>��
������=�K��B������ �����<Z�Z���joL���h;�������3��l-4���@�
j���ut�Yq?v��[F��m"��d*#��"���*������_��N�zj^��f�8��=�z~�5���Q�z����%0���`���cW���!�}6!
��L�M��������l.�����x����=#h�hi3���W�5\�=��Vg�I��
l)s1�\Lz.2���<Fn
�����~s�_��sdZ"Y�m��8��w|��{�;3������y���!�}���oP�}!����=��F����M�YfW�=���.����X�+;�q�V~T��P�[�6����`�k��Ts+�����;1��(�^��{�����������q��q�������=�|���a5c������Z��bN����6^��%r:��@�R3/����@5�b��
�2���*���)��R)b�%k�2z�6��v�q����aUg����������W����D�S�A����d�A�',���n��uJ�!d��(�����#��p�.�_]��^���S�m/?G��������>�D&A���-g#_����!��;i��=�
6.����Fp�����wh�F1[��]z7��/|�O�B#�J�����T1Qd�+�da�!S1���i"�?�[���L.
�2!od����
2
d������X{��D"�?[[���.[�(n2��qJ��l�^~=���=:�Q2S�8b��
���%j���Sg�m�B<�><k���� �5z�����6��I�����b���x���lJ��U�����_g�]a1��b�>Lgj/�@o@z��B*��\KabO�b�	���vp�����V�����_g�]�D���>Lg�P���[���T��Z
��T��Id������OT�m�D�B|��G��R\��wE�j.��m���f[_F���g}�,�E���@��`kI�v�T�I,��_�yj[��V�JI��s����J������zP�0�_����r"~�z7K�.&����U2����P�8�m����fC8L���YN��������'�{����6"�
�����CI���o�<G�s�g�(��v�LIf�d���S��s~����$�/����2W��[��7I'*��tk��tk����x�N���tk��d�Q����t"g��[{�$K�����2)�V����)��f�	�(%O.mO
d�(T}��}��5�O�O��V��BU&�����F#��R\�n����O!��gH����%	���Oy���"\u�V����W�W�x�F� ?|22��~5O�����^��W��0�m���U�)
������N�K��C������
fd�i`FN0#
��f����f��;�6�c����	�Xslsf�)���\|�eIQ����n/�r^��p�a�kW�Yrc!K9d���C+���[����R���rK}�����w�g����r^��;�%���,��/��$[j����l�&?L1���R��[�J����h���������T��v��OA����=f�>���O�������o�����]������V�!������8R�A�G����&Y�&��Q~��'2�v�$�k���0d�0T��m����M�M����	C
L�)Q�Z�4�[��y��Kk�c��N���m�OO�Zd�)���!��7������u��D-�n�u�-S3�-�]����W������n���CH�H5c���|�&[o�C������k)��XK�4�i��<m����iv�����"�@q�%������3����TF@{�-m������!��]>������v����d�
I��$�NK,��^y���
�9Dc ����D���$���U��/WS���cn�Ui���^w�����{�"�wV!}D
��W��
����0�4�6�4`��E����Bn����'p�	;�;,��G�����vS�rW�c�\������~���q���d{��N�wDth��p`}�s���7�I��Ax<��U7�7�VG��o��(v���
� ��!;L�����u���w�`2p�=�7�|�b{���g��u��cl�w���Cg���u�\����O��u��
�P}{���cg���u������Oz]W�=�vw�����Oz�u��������G���M�H�V����4�"�&K�7 ����,Jo�����X2�J���8}oO�������2��"���Z�R~��������)v���n����e�X,Y�S�>LZ����eM��Y2�&��V�ZP�Y�P��iK�UY����t���v}N|���������k��>���g����L�O,�����t�^J���2o�W�H�h�[��
��v�����Ky9k��YjRM�,g��h����.�]p������v���A��F���
�)�m����4�6�C���^\T���p=�F�S���)�_�w��	��j�&�������/kM��r.Tq) �Rp�X��})X�P���9����s)��+\
HY
lL�.�u��Nf�"�u��V�4can�����n���)��RG����R�Y �Uq��:��yN�<�+	�V.[�){�+_2-p�3c�y��Zq��Q{l�v�x�G(/Y��m���;8����dIJ���!��*?7)k/h�c>�q�3��.:�t|Gf��k}3$���D�A�M�Z��q�[��0��~��H����i�U�p�j&�M_����9Y�+�M9�-�!�3����Z��h|���Vg(
�^Am�`[���16��G������3������)`2�PV��{�����Yq�!�@$�L�Q�A�:GP}�mB��+!��� ����+�E��8���4�uG�6�Zw�'/��:�yd-��u�Y��Z�o���>p/	���V�y6� ��)f�8A��3��D�NIo��R��J/���q}	56��
F6�Txp1>���(���@~n&�����b����X�I�(j��$�	�X��������O(������b!$;�v����l8�S�1gO�X�����Vl������&+;e0�Cf��o��&������o�7�#�����z|��ax�:O�K��S�ax�N-m"@��'jF�x�Hv�J�!"��t�%�<�������J�e�0��V���2z����F��!��D&4���ST���![����L�"+p���
'Q���S�pcsc+��.��=07TW%Z����!�8����gtR2Y�xy�o�����Sg�?�m}��$�&A��d��$K=����j����������'O�D2
P�����0'?��4�2���4��9;�y�&��F����:39=����A�lGY��&e�,���RQ�<)�����U��{�O�J���=%�ii��5����������F=�2��Z
��np�	������Qi��H��������{�i�Q	��/��J�S�p5gYz�"D�J/+�9�>f�:�5����c�F!*i���UT�ey*��o1�4�@9�@����:_v�T
��6�Y��T�A����j��B��q�)��G����!jszi�>�E�
�uXb�����AN^�r�������c������X,90�r�QBK�������]oD��DL��=�Qcgj�j�f=]���GQDSy��6P�8b�����O#�F�i�<���F?+�a�$5Ub���1�-�
9q��eZ7�@��������d���\�����q�,��6s[N�9Lo�
�y*�a�bmbSrN�gvha
<v��������M>�$�8�R���,�5���:��{��.iO�;��=��Y?�����Z#���8�v��/�����>>n}������g�i@�ZR��ip����!B�m�<
�m�C����u6'��kX\{b�4�����{l������������g�����������:�}�), ���a�FH��mlO���������xM������.��8?#�Z��1}2jR&y���6�/�Q%ww�IzMh�&�w����,��\]��fB6�@-f��"�(�
�����Q:�����������)�*\���f^M���m�E�T�siGx��8�,Q2���"��R��:7]�PL���"�m����l�*�U������������Y������-���T����+��h����V�4�wx���c�c�����?����Q��j�G�A�`G����4�)������Q+�A�����:���Wu����c3v<k��.���w�B�L���J��/U�������z�V��k5�
8���0�d���L�w�
`y2%8]�1[�B�L�>A�����v+c��8?#a��2�-�7<�W���Z�k�n��p�}��nM�g�q]����8_��NKARN{�����jo��L:!�c�WM'����������U]
�w~��g�l��@N��).�/�h+�9g��0�����L���P������X���r�i���O�4�l�WD�x�
�6���Lam�V���;Z ���KG����<��grq����Ot���$G���]P������'<c���N������byG[����s�X������^?>��d8"x�d`�Hc H�D�UE�^�i��z�����,�^^�)�+LW
D�:���V��%���'��F��]O0�<��gp��l�^C��g���4���Mz���������(��e|�F����A� ���A���g������]����?��^7�F�p;����yt�M'c�$�V�l�������_#�����n�`xM���"��l|�G���i:^?L��Yt7_,�ip��3�\�������%��2��Mg�O��j�&���n~�a��v4O��R�������&'&�\&��]�������)�,�Z��0�d:��fQr�a���������.�3"�q��K��]N�����@�0�&`��
H�!�*�������w��oS*�%nn>��6��7&�����O�*�EP�
���S���`5)^_F��F��4��O�����A��2+q�-�YY����,|���b���!�3��!1\.&����?���R��11%��m�L��L�����;��ct5����_�F�]z����a���=�L?<����	!{;�n�����L��J�a�$�.�l��V�;&��iu���<�}�a(9���U��.!��$�.�a���R��P�>'��s��+H�+zo$dN"P*�����a"�H�yXx
[��r�	K=��������D��m�%�k��*[v����O�JT��a�p1��rH-���b�\L���mmr�b��1�����0������K�`+TI�.�.\�_i`����P�r�q-���1�����,lEq'��!�y�Z�<Dk��f>M�0����>��dP����%��'8����R����wCI���E��	p�����0�X�H����������.A8g���P��w�����D@�m�.>��	��5h�M~M��E
������uDIU`N?��?��ntw��SX��S_�����g����=��=��mh�I�WG��-h@�axC �����Y��)lP����b.�Z�o	�5��
{?�`�!�l��r}K����G������$�12���=�.G����c����������j3;N�����g�N����T���c��i�d�s/�����l���%��pLK�g��%�������O�t$���r o�]D�}�����Q����:�S�3&1,h@�~�Iih�H
`	P����3���5uxD��M*��D��h�@d
D��x�L�l���[��n�]B~�C���B��a:��@����l��������z���6��k�S�����2�(���"�RW�V��|�v�����N�|����wM_!�+��Ul~�_i���]_�-���[�[P8� ;�Xk*��[��ZW��
������p���� �ew���)���F��)��������|b�Kx3F�w	�&?�mG�*���H�/���POe�p#z�(h[����^��M��#i{�Z�*�~���P���E�-v/�{��{��)�u�*�e�Q�ZRg��R���)��S:/m�U���a�T#�"Y�&�|��Sy���/��Es��uY=��M�M�]
X���������ntJ9�A�J�E�5��>�Hp����tqMM�2�8�&�n��������r
N���v��<_[nk��@��O����(@���D=fP�\l����wpg���B�^�9�*Pc�b�wr'f����_=��Y�����3o-*P�r%U'r�5o�zMK�����DN
��]�i�U��W���������4��Mb�T)�D\����N�>U�>U���
���Z��T�*SeS4�6}��;�H�B"(���|�����7X�����������f�IP�%D\1��e�gc�FC�6��� �S�k�TwF(�����\��z�1]����Q��Zv�;St�	+:�z1Lm^ASf�`��>�������7l�,>�z��50e&U�
gj�a.�������5���br������SO3���N����K�r��;t�Z���u�-U���.���������Q�^�#�M��0%j`���hV,���HZ)������C�)?$��C�i��Hd�����@��j�����~���?zC����y��B�c���$�"$d�����������n��`��4���@k���g�(���+�y��Ow���|x���{��C:���]B�����]V����L�2��&��#�F��<{��h�`'�9�;��5"��:k���(hv(�����2�����s�zqQuX���0D���E���w�F@x\�C-���b@������j�9���s1�k,^���P(_V����UY��\�.�,�Cv�b�)~�y�����1�u>�~�-��}E�UjY�MgC�I��}����u!.�y��(��`j�r3Z'W�������k#��k��t�n�
[�w�?|��������9��U��{��&8��67��M��g��q�����m��mH��m��� q�6u����{D��*������J����}y��B>�����3��������vv����uw=��#!��w`Q�f����{=�}��|�!],���������{��*�����[&��t	���qq��}RC�
�]�p�����{�?���A�w�����q� >�S���2<�(�S�������}z����k��6:���������)������89v�����:<�:�~���}z��n�9����@5������B�
���~�&���n�i��h<��v�{��>�Z;Q���-pm��^�9B��G�nwww��F�H������|r���h����F�(�����~���A�X��}4���9j�{�4��m���"������1��7���&w�
,Vf���f2��W�]~h;��,�Mf��I�������H��Q�-�f�E�����~��v����gp����j�;�zE?H<6	�N����)-���dA�|��/�l9�N�^�q?�/o���&h��)=�`�H�Wt�!���*�]������;�)���::�T�M�,o��S���|Y�a�ZI�V�,`<��%%�r��v�\-� �WW,�+(���$�sp��"����}�?��[�
|��H�1��
wV�pG"��l?z3���\2�
S�(���v�
@�!��!d���'Y��&X3�j���P�5�"���e��.�/M!��OO2�H1�0~�4�|h{�9���5�z�>�*�EJA��W����: vA�H/3�"��-fRvH�@�}��#f��1�}X��
��I�>�>$�U����?�f��5�
p!$&@$�1�V�jI��wO���|��S�9���	,�60�}�e�{p���n�������
��#Y+ �(������.2��t��n���������?�(:<�v�.\_�&G��x���p��@��n�\<��~���;�t
��yc9F��qz���G��0[r��V$�6���
�^� ������.��([.V��N�d2[>y�d��%+���'�����vd�
3�	���`��n x�eK��3�)`��'���n���������<M����)�K��������6���K��v����l|k�����]��b����5g���g�&pb�u5���^�����wg�������
:���������F���lH/���;MD�\���s�iE&�� �5�a��pW��i���J�1������}�����[v�I�KL�q%XX�81)Y.��rH�DE�����d�k�h���"_���J�Q�5����2�T�*�)���
��w;�����v��k
��A��x�����L'aH��E�M��ayP��[��/#>_������&���Z��������e�\F��������^~L��lY�0�/0�%Jq�e�� k��������%�*���?��Ja�i-���6����=z���c����0�$,J����0��]����N��=6�\�������uDsnw;�5������O'�����h	L��������M�$=#S>J���5P(���	l$#��4�r�{��'W������Y)@��Im��0;0���FX�QJ�
��6�,�b���k���K�.���f����5���6P	f\"8
�������'\���$(���AF{LBi�9��F��ex���n8'��n����:@��b�P[2q����	h�P�����;�S���%�a�"�P���	}&����`R�G�!.�<_;~���ft����	�ca��I,�vt2��
#��o'�]]b��!:�ua���RI	��v��g�m�F	�i��H���+2���k��P�����]R`�Af$r%�3i�����O�m�� �w7������[��@�0p��2�����b��U��-�zx� ����:����J�����`CR�
5�^�CZ-����(��0r�m���/�������_N����	��r8�8�K���3"i����@�����y��& ���o�_����d��RA�THOx;�l���Z�>��x�\,>I8"���@��B�-�s"6/)��I>`�:#��,����,�>�u�C�����0)\�J�J9p[zt�9[�����z�i��y��� �^�	���j����{�'�� ���� ��.��6Y_�R.i�k�����	|���%�K��Mx�5�n���>y���:�L���DM6=�)��Zlzk,s_6O���B
Xg��U�A`�]����;�G���K�6U9Z�P���u��'�#�6���68���U�k���3]���
��wva���av�������j�t�kR����������H��I�4��B�1�Y��S��o�������U���T-�������m#z�2z��������O?��y��b������1@`D��dq���7K����I�Y���3����+~���i�z�,������r9���*�)������zm?�E{?k�9���}�XX�����7���{��8�E��aw{�{9�E�j��
��I�
��~��]�R��W���Azut4������b��*J�������w��"8���b<c �����jk�L�7�	���
O(��8����'��f�����8��g�f4r�$S*c����
o��yFq~z��O�p^&�e<�*W|��c���,II���c�������a�4������>5�"�#���D��F�S�A;�� �
�a��h�
�>�>��S�C��W����Q:�<=���:r2��P
��xx�x@��y|�B0��w�dB��l���+�}�}��iP���H���1^��'0���_^A��K*��� �6:��9\;9z��o���ft5����#�u=8�s�Q\��^����?C������Q�����Y2��=���w�� �����s�O�s\AI�f���������?��G�#Pw]��������
�^��\���)�
�W���{��;���=.��;�E�M�q-���^�y�k�j���N�����X�G��9��;<�Xd�~���Q~?K��W���l<]]�����3�
���7�#�U��:����|�:�p�X���Y$�
�S&���|���9��gI}�����}�>L����|�T���:B]r���3�^�86O�1����?�wR���;�\
���h|����}���L����!9�'wD'o�}�!	�@��N�,{��M�x/��[8~�>���<S�}����0h~�t,�e������>��X%�������Wx�����{wy����w,�N�Q:}���*�
�������}�Gq�-43�C���	�!)����[f���3��-j$E����o0?v���y5�mD�f��{��7�x=��g��p�����;�F�3:�Q�q��_N2�7l�4��U=���m���y]�a?�/�x���+#}�0g�F9�EJ5-�2�0���J�2%-pBpT���3n���'jl6��&�2�H���N�d��e�K��TE���c6�n��qde�WJ�h��%g�`��"��S'<��$���\A0���F�.�,&
���`F������9���6^���l�q�W	TXBx�o�WW�%�<��E]�������t�����2����+����%���/K^��P<l�������M>@��RL������}l�XE�V�1�@�K��ZV��FB_1��E���vc��6+�q$�
`����C�,��Ud\��N��8F50����i��n$g"��_�4��Za���|T{��K��*��m���@����67f �[�_S,��28e�>1�0O���,�[,����Z�S���vvOt�����/G�9�L>���r������<$Rh��_c���>��V������	��	��rsnM;���DG��=�l��V(�iy��zM����	z�*�pPD�`�9J�	6�����;d������R
����G��]tB5���t��$�����I�����c�l�G;K)~2[2�5��`���:���L2m���G��Z�<hGS�^ �v�P>� P�c���9�5���YT��|
����o;�B9Ri��
�{��F�E�"��������(>0��R�F��_��!�}�0y��JPO)�p���b�bK'Fx��H)Y�����UN��/2)��<�m�}�L�-X/�c8<Ar��X�����nw�Fz���H5Y��$"��Z��_��kJ*f�u�B�y��0`�X��0YZ����.m�"��j,�9��RAo����0Ox��o�x�90�rks�����\�Y��d���;��?�����{����Q����}bv�o�>0D�9�����������d��{��,�H��P[^P�������������T'�u�P�w�t�Q�����tM���cO����B�� ���fo_)��������S��(�RQ��.(��3��c8��6Z�5�>�5$����>/���!�XD�!i�E�'e;��[�A���x�%4�D��U6~Mjp�y9
L�'�^��'F����x��p�v^�FM��FY��cD����'���tn�@p�T����wwK�$�S�.����������O,���2Vbx��$�O���-�w�W�|�2:?={����������t����`0�+��!�T����r�xK��'���H��A����:`i8�0�\��B�)�W3���d���d�%b�����_!&�,�l�E�+��n��f�-6�g�����Z�49"�����{����������]joC!��L(�d_���x�'���'��^F�[07��D�Cy�?����4X��?a���@e2Z-!p�3f�tX\���.'0�b<]f*����H�-`��*��ID����?d_��������a�A�f��^A`���-OzIdu�(�!vOt�N�"�����g�J�K�8�i��T���+�R��N
������3�`���������3<b��a��b�+����������x��R���}�����k�_-��I�Va��|a�s1��$[�@I�������8�n�.�&e�#�(��zar��f|)Q�)A`��|N��0{dHo�pf
����X;@u�V�$`�4c5��w��$�:\�!��V��Y�W�T=Gk1x���q��s�Z@U���x4������jC�u!������'�5�� �����pL���=�(&��������	��Okx���{Ct� ua,���1�K���Of,"���1#����"jW��V����l��R.��w�%0���RVG2�\��'L�:	���T@�I���,�"���P�����G�m������2Y\RN���6<6j� �yb,3�� #l�R���C����m��W�����	��K�0�{��4�1�7H�lX5t%.��	&�����H���dh\#�&!�����8�����b�	�@�F%?�#�J� '��%u�Y�����$�R���/+�1���
�W������@t��m��~R�����0��fU��n��KV��6��:L�G�pD�W����N���-s��������d��K(�R7�v4��,p��1��n��A�O�Ju�O���$��e=^�K��\����
{�e�"Q��y�8�wH�}c�;�J�6�!��CP������c�Y$���a�d��oy���f�n;R��� �H�'O4X�p�Gf�FR��p�[�W��Ha'!� z'��f�g�ud�$d}��qd��5�/\�T$M�2����QE�$D1H��'r��%GERy���\|9�f�`����V��1�l���?���Mz��o�m6c�gK�X&�q�x���5�`a���?�H����"�N�t!#�7���
Dc��Tsn^����7�O����H�8*6�����~GZ9�!�
���(�3��ru{k��D� �"%�WZ��p4���#�}��-%�Fu�|�������'��m����x�G�'
1��?�����
!\����w?�w�4�a���(�u����$S�����K���������\��p���#��p�&�G`�G�Ws�4�K�N���H�D� PbW����o"���1U	XEb*��mKt��Wg$�B���]Y�����!����/A��#
D�����w�;��9Q��3���O����������r������d�JYk7`��_'W����o~z������O�����2���XN�-���_1��\��M�3�D��=7�@��]��'���%d��a��8���c��Jy��R�&��P�v�5,���`��U0O;e�����".���-���	�{�0up[�i+��(�������/?�����[��X�jte9�G�z�����?>����zd��R=�R��@�!	^�o�'��a�&�`�
L��������i��<���5���)��^����	������s�X
BK�cs����8M���������=GJM�s��%s��9��q���h��j��v�|�i"�N|!b�9EQ�x�0���c�����!jd�&�[�T�>��������P�zEX*�0
D���*����j�Z��3�$6p4�*k�'�`t;�)?1��D�4���9�1�����5O������7!�0:Bh�(h�u�������I�\T�g�� �r����q�C��PL����?��@�C0d���E�P�����I� a��Z���Z)��{�r��H\E6"G*4s�4�N���vC�X���u,�'~�d������$�H�K�Tc)�2��N�8��?�Z�=�A�i	a���8�y��
U���JS�
~���'���1d�"��9�b����]G�����M���ag��b�rnj#L�����L�H��y����Sr����~��T/�L��R�,>,{|��MW��)U E��-����FB���0O�X����
�`����a�}/��n��S���r�d5MQ<���c���?
��YM^{7+K��>����1�b�+s����^0�7_��H�����(�MO����B�2����������tz)(�9���(��G�^����?�/��i�	��eV��D"(�e�����G��m�!��q����x���T6M���u�R���1S��GM�"_
��He���d�P�/���[KjV�7���%p��5`�Y+�o�� ���lH���`R�4@�s7�#a[9�0��o�_O�g�AzY~$ ��,���f��1uLJlF�e�&����A��ut��~�����`�#��@bK#�`.MLkYX'w�-$�����;�u��N��aL����N%K��x��|�`_+B�Z��QG9b@�����;���v_aK�!Ig������C�9C �����@�V��"v"�������Cf�pB�
	�f8a���Q�@����+�d����S�i����Z���U������N���W(�B8����� ��/#E �GO~"��X;��%k���(@=t�:q�i��R�0P�\%��s�,��5!�B��86��vR���!j�����B�2zH_Y�+qDd�z �ch8�d?�O����
P�U�UL�G<N��/��B�c>��4a.�5�#�`+q5��-V�y>jB[T���G���rX�w��;����19O����T1-D�6e��8��r��W#K�#"����	��8@��}��SBu���j�}R����*jH��s>b�����0�	�CJUa-,�3#%u����a�m����#Gv~h�'�G��v&��8G���D���v��Y��t����ni�<��!���4�69P$X�c��L�����gY5���[�E�`�{T)kl�<@�W�[�!H���w0}X���SB���+��T���B���<]frp*C�t�M�[X�:�X�@E������G���Hv�I�!��|i]I�O��Q6�6B���F%>���jXptd�#LJ�&K�
����Ny�w:J�$���BZ&(jO�l�k���O�����kb�Cg���$2��P�'1'�3�>���! �2��V3�PDG���n G�mu����[�!b���w}�:��Q�����q{O&��L;X@�]<�!x��~��\W����W
@A�K��t=Q�F���������y{�	5fV�n�1��|�{1��2\��5�.���w��������*���p��fn�b����1&�]T\(@�	����{� ����>����*c��E���[3�`$�W����
W
�3�7�R�o�
;�<�v��dj)�1K��L�wY�6�t�_x��C2�'��� L�x����9d�f�-��F0N��j~�"�q'�J��%2�����:��������H_A�}1�����Ea�����
�W��:�cL�b�A�MP	�wv�[~cz��X:���Z��)�M3SY�t�m����0B�:��48�w-	Oa�n�2�����b]������sx�{L.7��V=���7v����0$Vx�UT�IT/�h�"g��Ea�}��0��t"%��"�V=���,@���K{�����ia�����K&�(��������[�H4T��I(�E�[B��Q3�bK;����Ge��j����Y��k�5������#�
�~ZN3DS���]F���P#�W"$�9��J�t�>����'����6�2��d���c���O�,@r�`���N�hV�1��"�f� ��%�[@�����
{��kc���g�������F�'����9Kq1��H��'O��"��
[���I��>S<a��r��}t����h�Hj�� 5��8W�Nr���p���(O��U!���)��4�����/?(��Vy�	�<�rk��"�+;*�����TN1=e���E:^-2�N?������1b����[�������s�kq�J=#I�!��u�1	�`�z���`\I��]����	$*��1%?L-G~T�'4�4TG��UyI�7�S����1���~���X�y"��LYT����m��!���@�h�5T����H�;]��*���&27B�I,�����L���������64_.}F9H2�`.���i)���8t������(��������xOz�GHa�&w����H2^��J�X,%��2�D�h��N��Rc��<��FO-A�AO���|��g\�x�������A��*y���k�&k3�������'IJ2[�.q~w��N ��!:Q�Ct��x����!3�c&��v��[C���`;Ji	hFY#���2�j��+��p���5���
b�Y���
�`�'�j���lY18�Hx\�����F������d���v~G���Y��)gR�Cs��.�3�>2������y��!��5����Y?#o�w�!�+�(I���,��,��G��,7�Gm�"$��H��5
��@x��W8g�0�C���I�����������*���e��������PA��B2^n=�Z�0L"u|���(#�T��z������z+�G�)���y�,#n5���Q�����	3r'9v�r4�j9��e]-����#�0h�<Izl�������@	%@!�,&`��y3�1�� f��������g8�\���\13��	��������E���|��,�����pR���%����z�%�N����1l���9�_UM������L<R�X�z/Y��+4���Z1F*t��]����^��x(
U2�!��3U@j0���B�#�������H�L�nT�/�;��Mu"B�Y���aV�9����o���z��a.��%�8YZ���Tn�����w�v������t
!E�u�e�� @������q~�%�$)E;���KY�b$-8} E��N(�g�t��N�C��^��
�ET[����X��s�oM<��U#�w�R�4��L�H�%])��Y�*.o.,�c%IH�)c��g4�1�������~$B��O�IKB/�7��.��r����%�0��-u��S�|�[W��D����S20�.9��FBUw�<f	����?3
i}[�L����U��#�����Lh�tw0
2/�8��c<7�WW�t��
������,�'����$�a���>=�8O
����u,���="��"IH�R�2#F#���raD-n!���G?�'�b�[���G�aY��������
0�����3Z 3$M��Nv{:n����.Q<�G?�O��Q�V�T������'���!�a ��vh���,��I�
0�2�Xa��M��N9�d�!�����"�����i�#p�z�!���kP��g�gX���g�y�7�4���Lw��U��`YU.@N���8��A(A�!�rS�f ��T\hi��Q
���s=��9Q��=7>f�O��1o��
�'�W����'�r�s$����Y6��������J�9��k,�bY�B���hS
$e��9Qt����Y���ba#�"	�$��Q����&!]�[f������/?rq�$\LR�
R
��)���)���|5��tF+b1��]��lE�$K��{Lse��`��\��f	�V����\�|��U�?tsmR��[*�L��������"��p��&����S�
����Tlf9��]���Z��l#�Tx|/T���5��Q9��9�(B�D�������cmc4?�]�I�x���>��Y�>��������X���n?*�4z�����.AAi���T�<�wc��6�Sk���A�lEs���h�!��_l�&EY������v ��=d�h�!C5�C������q^H	�\����A�� yST�N�6C�����*	ZI<�n��������� �{�]C�����x~7!�1�A�[<8	���V���S���<��lz;.n�!�o����+�|Q<o��1wtS��}n��m��zN��$��{���(uB*��x�C�x\�j3��������%���`���������p{�0k*�����9����{L��`����H�������2��@x<g������������
�h�s��l-��(�������.���������1@��	���r�K<������t-���Jk�P�G2��!�VW'�=,�Q�� U�K9�ud;&������\�v?P�t��h�(�1�;�7�`��ehd���pi|���<Y�����D4����G��q�7�	n�^�������e��Z�&��8k�Sw������A�'O.x|2?��A����I�?��iF��e�����Hi�������+>C��c��Z�\�<-PQG-��n>i�o�owGuyQoy���x!{&��3"7��4�M-
g�pt>M&�,
��>4m�����E��R3���@Hi:�{��#�RVp�x�|�����_w"��DC�s6�#a��E�������"��y ��9�Z`���A����g��BUu�`�q���P�4���Bj4<����e�5��}"m�1�����'>�3>}D2(Q`��L;l�/�����1�r�L�{O`�U'�����R'��5t���Dt�������pZ�^-����;���R���2Q�]���V*C��D�#3�4�Z}�zQ��tG���Ar������Q�����J�9Y�����/��
Yv/Fn�,�����l\=e�n��0��� �����0<����H�,�;	R`�#�HO�H���f�����-X��/����3BbW |?��3DU�/�����/���g��:9�{#NTo@D�����^g$��k�~���94u��*%�T���E����j������O�������G$�o��K�R��clT�0����~��j�*���'�GvE�[��$�2��$^�S�4����A	=5�`��)?B�]�,�� ������aO��b��|�����j~L&
�
�������B��e������]� ���b4������w�dXX����k�Te��wB}����	6	��_�sYh�a1l*�=��J!-��Dw�������2�_NnI.=�T	W�4#*~��<8�@��<�����������C�\(��B"�p!����
8=%��S�����\	a�����TT8�>U�bv��D����;�i�
��������e�~T�&~q�9�H%+��R���TD�����������'�L7�T��ey�i�!�����$ �r�x���V��jb���N��A�e*���6mN�����CK����������Hk�_X�F=��rs������-���8���<�@��P=c������JJ����=2��W���+e�La�)Y}lj}�,�(�]E�w������/Z�J
���	/+�N�/����'���H6�j�����<��4""S����b�N��x8��Y�h���j�P&�����M|yG#R������(d��� �#��C��8�7-�<x��%_�M\o��lX�1�2�EshY
Qp9_�*��H���r�)�����E��t�r�H��!��4)�n�'
�� o�eI�<&[3����7�3W|9\��U�U*�S�/X���=r���5Q�dE��KB��F�3�D����4���-��`���P��pa�����@J�(`��1@of$�����Vr���d,K�u1^]~\`z��['�[H������[���2
�����T�PzQ�5mm��P���7��Mw(����
�e�e����=&WP�Tb�T�|�6�;�>��]�}R��))d��1W�r,�
y�K?J��P���q�TM��+Y�e���c �����
�Ut�����;����J'f��\��/�y���^���nn�H(����J0a���N�����Q3$�'���Z�����N�'*�di�.���*E?��m|�����Z7�u������|�B�&���Z���{�f�k�W���������K@4����D���
�3�w�h�*i�%N���B����s��|}�U>��Y:�R����>U��"a���4��`�I����b���w��
�m�(W�t���lj)�/��F}x<;i�
"����1�wY���n�x�s�I���N![[��6�Z
��l��Sdre�W�[�	��f��1�Za�uc����-0`D1�����V~7�#V����M2��fx�w����A���Uv"�`��D�{7�a�I�SH��a�k�1-r��JMjw��yf����uaV�����u�yD�z[���`p��0��f��,v��,�:��-f����a�9����b<��G��u�f�&�b���\�<1���\��W��
���p�Y8�GQ�'PH��Sb	A��������4V��'��34�,�xH�vc�M�>d�x��=�a�B���G�������@W��:g3;�������5D`5��~$�p�O��25��������D��[��O�Q���w�S!'���o�?4��R�����1^�y���'�mag C������'����>��m���@Z�����"�G]5�yE��0���t2!���%c�8D��l�|�+��M��������6�4������-�m�����~F��4"A�Y[�X����Z�	�q9�9 }���	D\_24^	t�m��z�`|�s�-s|f���B��lz�0,*X�W�@���(8��%U6z��kc�t���A�O����w��V31$g)�IHc�*�y�o�_��/ja��MJ_$���hb�=�I�!]r7�����STF;��
p�,����6�Npf��P�3u�t��j.��e3�9���P4�(��������fD${ ��o��1�����yG���,
�����.$\���T���h��7]��@��=%�y�fk�=�h�����C���fr���%�k�#����������_���`k;�K,�*����p����K��v_F�\R�,����X��#j6�=`��&>�gZhf�T�M[����P�z�#Nz�I�ay�z������?��������1��58:����E=c����A����Q�������Wm53qPm���Yz���*/&����Y25�'��,c�P�PrQ�nQ}�.�3qy�����J���5F��	�
�0�G:eyD�����Dt-{��2�;�.4�3gR��W������Q|�:����2.����(������FLa�N�Gr�L2��j����e�i	�L��bgW�P#�owD~��|�%O_1�^X���Z<v��Di!�yU��Px"sr�<�x�(�]I�b�;k0S�v�c��jUX4R*���g����t�J	���KC�),Y���%���w�o��<�������C�����}����@����8N�0M1p�"�f���������5������2�~�<r�Xj��ql
@"�� ��R���\R�|��\���� ��=$�I�������hy��?K�����4h���D����2���G}�Q��o�g��tm�s�kh7�5�t�>���������(�3�y�,���#�z�0zrm$�k,-on�$N�)*x��EI���I�7��(.���d�d2�H�]����H3L���=d�.�&-��n{~�\�\R[	3���W%���Y%�O��j�,d�/r��`����!T~�����%|c����F�twt�����'�w�H(��-��������Q���|��-���=���>�15�|,'I_@�9w'�c���t����
�,���r�������m��9pS��*{��Y(����x^dK�������QY��Y�����s]�q��5�?�dG@)q4g�9h!���E'����m��i�u:Q��udP������~�a������c��P���?�F�?������o���d����L���K��j�X�G�	�,�7�O�����������}� �1�c��V�����u3���2���9�SF��jN}.De,�3
�]�@r������@A�9z��6�D���k��2	���4����f���O��@�(��l����W�������1	��2�I!��vA!5���M��Z'����
���d!W�� (!���d�����|�+F������[��=�o�cg��BH@V���y=~"���F�����v�Y���c/B�X,����Zq
����v���i��!n�mbLr��CO
8t�S��pGytMMft4qW,�R ��@}�LIY�Uc�! ���"�r�P �7	��0������C�%����JT��/��C�TW�#����du�7* ,+=������g)��0�o2�-�uBr���������H��
C9#�g��z���Vm�������G
$L?��i2[Hp�@CO�����"��L��RP�^�z���������jXL�]�BaS�E��d���"vkN�,��=��UW����vLm��2&����Ibbf���	m��v�]~R8kc�h�j����4���l��UQ��L����e�0cqJ����$���A��s*!Z���1�\`��g��E=W*�3C"��-0���{�X�I}n<��2<cHEpu��/@i�����	/W�'fmS��� �(��:��i��L���t����6�'"p��Q���@'3�@�_8J���[8�_�-�I���e�|�)S,Y���C,�a6���������(������v<��!�Fh������F��		17AbE�r��x����tp����$6F/�,��Bu`ye��z5��L1_�X)���U�8��]��Cw��\�)�"p,~�#��p�lW��������$~yE�^-���������N�!��)���J��9����}��W��?��������E'��~9=������o��0�����w�S:q�.��y8���Z_�����"����_���#O������.��rc��*"n��_]������������w�ly�e������,������gH3�������
�8�BG��X���*�����;�Y�[R�K������x��\������6(qu���7&��J�1:��\����&��Z���g\���a#v�f`��i���~/N�����Y��������1S�����	��������|�����/��;����X���%�P�G�+��H�>'��13�Q:����h����Y���Z	�lJ^��m�����@�#��s�$�"��`��_�B6����s�"�o�y����,s��d����S;�"���%3e}�TG������~_0����A�����f��dL-^��Er{��Q�O�*���fd���0
���VX`W}�;J^Pi�Bc����x1�`$������L��G5[�w���zp��x�^S�#����c\MR��AS�dzX
��m����5��'��K������;�v������s����^�v�/4�`�X��b�v���"+(��p�s=X��j����-_k����W��r!~�}T������D�-�z���<x��
�n��z�H��"�{��Vw`%IM���,_7��q���,����%,5l}"���~���>}C`�Y3v��R���c���G�4����N�p�����6��:��Z$t������wQ9S'��R-��������VvJ�R�v��5�������h��	�X�M�9���G��F�:4�%�0�kB�<'��_�W�W�������>�L�JjS+�s\g~�����6��2m���_�W�q�����cm��>�6�j�����j��Nol�Ff�� w��8�^�nO\������RL��-��`�w����s���03��N�pz��O�a���������hL�4l�n�O~��g^V<����{��N�O�'/X���N/�xqd.�x�����feW�4�w`0R��:G�+n�,�H����U���p��������0M����h4f��qFJ?�lyT�-m�����T�l~bw�5�B���"�
�P2f��@��� ����a��J�iG�0��v[Z���d�OF������H�;!$�J�XG�����O�o�J�pe�D����x���=~����������E�l���:�	�*���He~	r��'��o2���������f�����HUP,a��YPWn '�S��s����H�Xno+K0�-0w�������Q�Z�4��r�-Yu���j��p�8m���������5msXa����=�Vf�3&���}N-�Gj6!&��x1�gx3l�\��!r�4�4"�!5�������U�U2%V9)�<��gL�xy%v[	>�tC��K���b�j��0?$�5�������2H�SU%���3���2_3U�A|�����~�=�<F��\yUU����$��!e�����bF��3��XLp.���������1����^'�9�*^��Z
X���&p�G��.���x�@��be.�+ ~�iy�!�F�=��O������1N��"l���]����������)�M�R�DI?�v�.�����8P����:U���K�'�]%������G'g�h��9�U������~��'U
^��q��<�_!���/0'?�}����G ��(���c&���vF(��Oz�4M���y`b��ppD���������y��P:������"���&|����2�JA?*��$#�G��mBcTfea>4\4��F>�9I;4J��J����N#�������ur1f������1����w����2��!� 
��1,�A��O�^�Bi���<an�"D�|f��-H��P/�4j4�4t	���U6�����_���VMg*�$������d���ru{Kb��(�6��R�������F'�����U����?�Z�j�RTK����?�e�OG���e�.~�$����Ft�1d^�&����4�=��6�uc�q[�A������u���#1�nY?��e�DmDd�r���o6g'�N|w�=�3��
�������6���\]��u�FW�����

��!\+��1��X6]��~JjV���m�M&s�[c�����b����5���`$xXq*x�
R�/PX�w�.Q�v5����?;�w�)��E
�3nK�,��}���}�.�<rBzp4:<�13��:���aUI��(L��uP/��	a�������f�����[���R��;����T����p���<��C�m�%,����pcr&u]�� ���}��H=�,��~0�h��h��YZ������:�C!�x����oO_��E�"�����lg�~��c�=�{��R�|&�)g������h�|G\�wA��4iy��b��r��N!U[��0������;n�c}�:
l�w�dG}��!�������K�!}�v�dN��}���w�f+�O����<[	}
�%��%�����&�VB�Z[	����j���JtL����A-���+"J[��2�r=�����`���jC���h��NBG������"��/��(%���~�E�O���P;��&4��5�}X4ZbZ��D���?�?�������=��"��&��u��������dZ'A��pu�E�]�������w�{�����>�!���{p��{�p�U��~p�E��zq���;�{��O����)������H�Q�'�TV�yf�>���?��5lV�����w������Uz��%��n����A�{G�����(�	o����wu����#�gs�y�0E�%�o8��G��?R�L���,�������|����O�G?%�pm��������T�A��y��o���������������GA#��T9�7�}a�0��D��o..���ag�$9J���fl$H���)��!:���o���~����/g���;���}����G���������4c�ms��O�$#�
vnSzJ1K���5f7�Qp��NXi�=�f�x53Feg�<����[�������C2�n&���:e����Cl������$��R�Qn��G����h�ra�d��.N?.W��$����E�{��������7�k�Hov�K�`k=���e^u���I=�H�v������4%z�=L��>
S�Kn���F9d��)��/���"���u���S��-����?���%��]���!S)��n��n���:���?���PLf���9&b�@+�E	��Mc��Wrx���[&���pT34^�z�wvY�������]�bX0��D��R����F�	d���{��t����D��/�W'�O��(q"Kl�#��X(c��^X.��~�V$���s������a��ONL s�E|gG
W���[C��������|r}ck7�������,���������,Z��`LxzQ�����7>d��&��|
gq���vF��:?=�8�.N�;;������t���!)�[�������E�'o8m%���1��)������/����/������~�Z�6�H-4�:j�}jtY=]W=����g@�y�����E��-~F�xzv��"�{�?Q&�������=m�����3J�V�
��1�oG\�������7����-����/�Zx����T�@00�R	5�*��F��@��sOo!@��x�����:Wd�'�9��#eC0��Phd�4dC!�B�6��Z�����GEK=&*(���PLQh���`���(��(���"i��������{����N���Z9=������Y�_����
HjX����;�����w,1��P�������&�@�{\::y�=��o�����1�x�j��^���"�j���,R��)�p����%��;Z��c�����_��"�����
]<�KD��,7>����T�j>�|�X_�/���������=�� Y�}����l�%w7���j�r,�%�����h�^�'S,�b��*���k���9�����%37�4w��R:{���Au�g��=���=z�s9�?��5��l2����!_�PG�l�����,���l*��8�1��w6����6����V���������G�6?�^����w�F�
� B3�j��Mo@~�������p�����O���������K��v��W�������� ��
O��'�C_{0@�p��e���F�A^��e����)����nY"��>*�����a ��ww@�@}�������:�D�#�<�E�^����i�X/��E�z�N&������@�[1>	���]m�ts�T������T=�5�Y�
��8��.3�<KzNt�����M+C+_�m�Fk��N9��m�r��������%�k���k��5Z�������c�������wv�8:}��|������4������8����m�K�"�4�
����
|@jP���P|(b���Q&(�l�^>���Tb����7Y
G`H9"�H;��
B�
��ob�M��f@��7��j��2%��v**Lfz�V�i�<o+o�&	�T��6�0I�@&8������fX��]��Yf|���9�p��(��d��ii��k��B�R`��K�E5��d�
J�j[o��MB���o(�@9`<���c���e�����/� 	v$����� E5I��R)���$zo=����Yz��$%��+r��r��q�OB�X�=�H���^H+HWA��I��z"I� ���c��=��
Z�M?58�� o�x��V�9�*���������;u��a��#�i��E�/�����8z�-~�d7[�~[A�nf_L�|�xw�DvM��A�D��s��0��n>t*�]"�� �+�����������"���c�$o�����MB����A[��~dA�'�P�������q�b'��~=~!�������V�3��!�>��V>���	���WB�*2��3�'��B���m0���~������-U21U.�b�!{p�\��
C�
��r�s��sU	�+�������
��6(��a��_k>���� �&$�:�C�;����v������4�|l��;�Og� ]4:�^d��*b��XP�T.��`�!��&���O���zyK):d�Gb����2��v1�g,�X"��`�!�F��"{*!��-b�H,��T����"���%K��5���X�.TB���q1[���<�}�>k'&����D{�;C!�}�� ��J��Q�|�YJ=o���G��W]�Y���v6CK���sW]�����z�CK��,g:���3W)��v�#�1_J���'BE���/Uz4Ym)V�x-2s���]����Y-��=������
��)��H��\��{��
�(��KP�v�&zy�B�S����b���]�����'���"�vp������s����
�Z��2�d����i�`�����]���@,��m>������>�%W�*z�h=4�u��J���[<1:��g�������z��[PU��y^u�iv�=�)�m]����{��T��*�f�m���4��-;Kgj3i�Qe�:x��Wq�u��^k���kz���E�m��3�8����-?��jf+����m�D5�c*0}����o���iR<o�,N�����������%M��HK����>pSn4Z��,-`G��SR����:�u8�����D=��0���	�^�gi��+�������ng�l\Y9�B�B�lYA�p�B��6`jV�A�%��n�"w����b���
��>+����"�������������e(�������rZ�Tl�^���cS��=>���k��dh}��gA\Q��y��~�:Xy��'��\��TVUs0�@r�e�=��V{���[�B�����"��!��&u����`h_k���fSA�.$i�ZNE���Y'W��Ue�l��X7#��?�pC���Fjs����|���������|9x�	�>
��@��=�24���~����v�Lk��P�Q�R�/�N9<|�f�O��"������7[�FgK��9���f��T��E�)��O�,����/�R��U����l
r��=�q�[FR�*�<�%��%��%�iy����K�����!�U��#�F��*]�$��ca��<o=;�8=?9��}��I$a���=��*������I�${P
u�e[����cj��<�WekL��Y��[*?��?��W�A��w��F�MB'j�����h	��h�;�Qr�N��*�H��S�s��]�x��8��e��?�&{��E�vG��$��h�15X�T��,�2C��h��n�S�Zzj��4,�4��w^�Jm�+'s8���-%H����.{������f�(��
�y������L�S!������?�������5,�a�����
�b�����9��-R	AF��].A�pKy]y	VMR&>���d����B0�,A��`�]/&��-A��u,�fv?��+���EK���5�K���Lc��@/���%�����G6�S�����P�[���_���h��b��+~p�����k��~��_/f���s������^=���)]�`���r�xX��z�Z"k�lGN��c-+`�-��v
l��F��G��5��
�u5���WUV�}���Gu��^x������[�V��T���nk@��eh@���|����i���F1P��j����_�/���T��H��2���L6�d,���-_���?3�t��k`�k&yoO���������xu~��Btq����9L��p2[v0���8��,�7���|r~���
F`���'o8m�p��h�~���������{��y��<����/����r�]��h�]gM��&bTua�:�����*H]��7o���_Do��g�F>&��r>`���O_�+�)���,]$�t���I����hp|��P;���O������y�r0-���YL�q]�������f�EK���1^�	�����St5_DK,�Bl��M�sy�G����C��i������E��}�+	L/���9(�F��!;��pDCvD��T�h��h��J
�G�8�h�D4dA4dA4�����F�HhC���rG���%�7FD�Q��v�T��O
k����V�j��B��BQv�[�dD������*6i�i�MmZ()�!v��rp�C���v��V1&+n�^>J@�9V�{�h���sM�Vv�����������j4M��47��>�HzUV�_�f�T�2��T��� ��� e����W��\�D������q�P���qD��=H
\�A�+��(����7e-��P�����=���p�bwb���3Yp`�����fM����HVD�/�j��V?v�#v��)�,�B�����e ��XC?��2�% ������fz*�����S�d�l;f��44n��6���d?��f'Ro�>&�"�����)�.9b%?y3:�b�E�%^�(^a���h���������������/w�5[PP���O��^�t�qTc�J�9�,0���C-�
���S���_�5U�)zyg3����Z��K��\��V��?��]�ho�f3�������a���������i��Z����jI��{�\mk'|�Wn�^�������a��,�YU���a� *#
���D�PS��P�"�V���M'
H'
��^2w5����������
D�j��2��
���e�Kl�+;��QP�Q��5�'z��y���	��?�?��������E,�6pKP�6^��
��x��S8^��[�,�9y������=��:�5���>�e#~p[/�f_��m�u�W��u��W������������������+�����V����g��	X�[���{��T$[���V�W%�O����t����*��j���&H^��m������7�1��.Q58\��
���*��7�������zcv*(X5�����.�c5�QoG�u�����n��x�c5�Q���U����{�X�)���j���+�s[A��
�L�V����[j�5����n��[m���
%Ml��[mh���n��m(ib�
�jC��6���6-n>]��\�#�5��u��oD�u7����,���F�Y���6G��]!���+�����o~���i[�Ug~)�LC
�	����j����ea�I�)�Tm�^��z�k�k�(�J�J��������bMU�zE/E@��_XU��jz�
���eA:�@B��������N��<Xt�M��@u_eE_
B����[[
����K%kP|ZU��j��
:��hA��P��������A8�$y�t�#P
XYX�p������V���w����*D�Ob}���u�}6�m��1UVP���v	m��#XBkU���O�U|"���~�:U��b�^
��*����S���������^�Nu��D�:��t����������e#Z=0�Zu��f!Z���j���S}��[��9��t��Q�������e;Z=�Zu��f9Z���j���S}&�[�jc:��Y7B#����M�Q��*)�i�J�j����������=*)�i�������kS��P��v
m���_Ck���������?����e����3����!�,7�;�m���.�	V���0�{Q|P���V�U����n�e��2=#���9fif�����$�_�
"`���6X�4��z#�������A=w���7�|���W�����U����������*�M"���_(�x���x�s
GTT,o�*��I��<>���������|�t2���Bmw��X�#��`F�)�Y�r�����(�C��V�e`[Z�o	������t#��!uTH��:L��2��R
L-B�3�@B�BV����/<"&d��*2S)R��J�,���B�|EB�z�����UYb��5~BV�?���|�H�b�b��c9��J�buT�:*��Q�M���!+=��E�|�H�b���u	���r��*���d���Y�eK�g	YpG�e�xm������+T d��C�<!3��u�V�~������n�W?{�U�����(������!D��[A�����7��
���zM��BAVA$d���UM"����jKF�ddKF�&#{{��$K���dy�����6��4����i�M��,���l5�m�.>E���^��o�_S�j�F��4K��y��GPG�dPE:^���Ww7rF
?4H�R�*o�!j|_�G������z$@��/��<`�}|����s1�j��Ep<�'hw�)Yk�m��X��v��|u��b����W�I��������T}���>�bf�G^�#���m�����K�;���9^�ZO�Pv��|M��E����U6��3����8L+X����8E��Qk?<C9���3�Lk����z�9)����R@��)�&�U���v"��uW���)\�����p��n(�R7���)`�C6�!Q��N
�hn���je
�	'z�(���5Dk������y|�l���"��h��
��RP->�������
P��������O��(W�j�]�x����Q����@��*pwie*�H�����Wu�o���"��&������Wq��#���-A) ����Ig$�b�S0":y�=}��xPx"6k���o�D���3nD5l���u9R����m�6���o�l�^Fg���/;W��z��rWkah��j����*$q���3B������0P�A_��z�:��e^���<K��&����4�����Mzi�w�<d�E�D-9���Ka(C��b(��5|�]}$��O<��n�����y�.
��]���E���H����IY�������"�:F�H�E��"TY$�!D�f���z�o�D������]$|{�h^�K�cZ��:t�j.2�>Z��6��!��X$��U\�LN61��4#1����/m�v��T�D�K<���|���Y}�Z���L)Gy�]��QU��W�oL�+5�������t�^�&70�0��7<`���������0�����j
"^"H��W>z��s!W���N/m#<pa�V�
�VL�������w����9�+������J����S>z��s�W�b|[�������/W�
X1��#��#�3�+������+�CD�NBWL��=���D\M���m�^F���M
[��?v�+\�G���h�)rt�.��N��wI2�$�W�h�Mf��dv���f��(������N�(�x���^h/g��)�y�������)����&��&d
����(��r�].V���r�G��h�\��Z���j��Q�2��d�D�A��Jv����
�_!��b��X���������?q�rOP�I�?1
�\%(W	�U�;�{`�Mj�OJ+�&4�\U=����P��Y�<������>R����Hs����)A��q��e�� �h��A�e�W��1GR
Z�AX��]��~�z�5)��zI_���T�l��A8z,�\�J�8�0�QF��eQQ������;X��7���������iz��.*�������
e��Sz[���,_�9����>(g��R�Zu=H���++P���Lw�z�� -���rm.R����9���aXww�����J{Ds���=�;|�Q7}�����J6d�#,Y����P�����A9�Uy��S�����@��#x��W�#���^A*���R^UL�QN��������������63�r����SPv�U���f���!��]C
��/�7
:�j����J
�y������Y�x���"�����hy[���d7�X��f�^���q��:�at��u��n5���6w�6���Q`�q��t��_6bF���/�u����3]M]���� �T�����$��$��}_!s�:$�me�MH�1�z��������'�h����n�=k_�I�����D�%�h����!>����M5�$>6�&�
���_�!n�jmB����k��?�7�<IDC������aX��O"rV��V=����V��	^k5�=;}]Y�]�jk]���-��v�Q�U�Rp�[��v-vs�Z����^~���m�H���u�8�kC�^�#���&��F7�b���a�G���L�<-Bz�B��f�k�h���Z�������������vk]9��������hf���/����#c��6h]��hyZ�:��&��X����m
�������um�!�9��u���6rT�#�T��M��VL��6�F��]m��v�T�a�����g@n��V.�*���%�]B�������=�h�&wU;6��C������m8��	G�;�	F�B�Q���v��&�w���wu���CT������k�.�0i������97/R�U_��M
���@C�I�v��"
�N�d��p�d'
�
�a*�'�7��"P-�*�
� }u����.����W[3V�;J����C;�E/���.F��Ou�����}���/�������������5D�n���
�6�(��8��&�>��}���	�`z7�~
�
��=\�������A9�6����7�/|�<��~~��h��[���=��������A��6��Y9�Wy����~���h�Vx��=\�����A���}s�E�����m}s������51x�&[����
��^>&Z_��A��}��|D��u��kz��{�?RwM��A���������u��>��[<l}���z*��l{�����D%��u�]��e|Q�(���5�J��^��������e�u��.��������i�7�d-��t���iF���%��6��&�����M��W5��r���\������>y�$��j� N��{�����i�dK��:P�/�j�H���-Y�D�ekL��,��-i���1�hZ����6��xr���;�#v�2�����Zg�'�/Z�*�D��7���8�Z���������<a�j�%V��*������We9���Zi$m�(J[4�g�USK���Uo��P4C�9Q���W7�vuc�����[���o~�%N���Cn��]Wi�����'�]�<�d��������e`}��Tu%��t�j����Q|�/>#v�<��z	���/�������vx/~��]�����B�pJ����E��%dmTppH��EfP�%������J
�A)���H�����9A������ZV�J[������X�\�m%�����x���x�J����J� ���V�} �7p��\�7���r�[n�3�x�k`����O���7Iv��o�0���6�B���E�|�DN����r���gXj}�nW���n�F��!��^���u]�|wvJ-�n�VMf��[rG���&Y���O�/�\���c����N[ ������`�_X�^S~��5o�����/'g���g"�� `i�F��Y#��Q�Cj�����s���*�!���Z����O�/�7o��5#U��9,[�����U��Q��t�.�%��b�f���q,'����'g�o���b�3dy�������f�EK@��1^�	�s`����h��
���M-��N�����).4NIM�E�,R�Az�o@w� �is��!�1'��a�5�H��/l���x���!����F<Dv<������>�9<D&""��C��C:���*FG��Z�P�r��v	�������]C�/��BS��U5mhto�C%����H�����)�	���
�{v@L����j%I��!Z^�G�DRC2WRo����!��b�REjA�0,�:U`�x+d����9��uz)�N�p��U0�B\�q���-~+����$Iw�)�jcS�p�\�V�,R�y��q���y��^*���i��#���q,C:����)�Z<��.S���2���
?�h�t@�*=F�8�Z�Y�zA/
�LZ��<�6�?r��X��K���i]���y�qZ���`�|;�v���|3����8���~3�oZy�s����s��N��MZm)�e�����b�
3����3:��o��-��Li��G+A�M�8�UC�Q��}\�M$�:@L�w�x� ���Y���x����W�i�Qc�����_(��A�#�����n���*���R�#�X��T�[B���<��X��W�'��?\������*���:�f0�����Bj���cl�\d`.�091i��,��*bn�1v!$xo�1���=s�9PT�\?��N,������|�j�b���FV��
��5����k�[07���e�L���������M��k����:�G�j��l���� ��Q3�
�+����x�hc9���zq�A����v��d9<��zR��x��c�,+TF���#�"I[��-�n4����N93r@:�dK�����f�6+�)F������������q��zJ���j�T9�,JN�;�cv��{���Q�y�������;b�cW������V�!�����������38��h��k5~'�J�'r� ����!�s�j�����s��s�z�p�p�Zo��p�����?s8�r8������|����(�������8L�t�\Ln[�+xz��b�Fj��H�HD2��I{���2��J{�a����.�]���Q�a�{`����v�G=1L����a�������D��G^����5���'"����z<m����@]��/{Jp���'pK�vLMN���+��}�EA�+����~��q�zC*o��	�����������[b"0�[b*�=J�e>��L��d8��^��M-6MnL��t��M�����[)�m�����"�p���M
S`y%E�&���u%x]ae�nLY��X����A�g ����,5W���o��F��6?���A��?'��OU9*���&sw
9�7��jm�*`e��T�F�������q5 ��N
����(�w������`�PTz�U)DC���N+��qjA[X���^����-�PL;����i%R�c8��{
A����*��B]�NC��Jo�2
i��2h��q��AC��A�Ap�4��e�iH`������C��D���p���q��X���K�+��n[u�F���qA���Q���\����C{%��������A��������p\u#y���{�~;G���X��n`�����a�w���7��"{��
&g�y����-����8:p�} oP7�L����xr���P��n�k��w����#g�G�uC�����������cy���������Oz]W�=�vw�����Oz�u��������G�W���u_m��������w��-y���%���+����,��rEMm�nxNgqr���{C����vyp"���BN�
^4���������qf2���w���{/c��r��O8�U���=)�x"P�0�K�U>/
5H/}v���2;���p.�Q4v�U�+X�x�i����kC���O@a��PJ���AD��a�6��%�\��qPK�����Z�E���e��a�Q��J;��e�\��j�t���V
*��Z����m�}��_p9�`�����IQx\^�C�(�FQP-��q6P��"���'����LQ�6L/��p^D�=qV2W��<��|�U�������5r~�`��U��v
�%�^Yc���z������}�"�	��Puu��N�F�0�/��A1�� W,
����������W6���D������:�����|h��P&J7Ui�/|$
-�@��i?�u,
�������2�
=�@�S��t�K���U~�����j����[p�ES�81�������x�)^��Xvtk���m[M�VS��VS�	�b5�p��m5�[M��p���/WS���.����x�)��VS�%i��.,?���\g%�5����h�D�&�T����,��jJVZ���6W�]H��n��YH��U�c�:f�����W����B��5�`�D;a
��������0��0���6K/}>z�-�{�k�����BV�"��
��xU5��Z��4rG�<c����6�SM�����a��0�n�m��v�e���8F�@��FhJ�J��2�e�gM�o
����5(��Z�N5Uc���T�Aj�PJ���R����IiJ��y"i7�4����J�:��N��5�i4��Fi6Ki��r�#Qvm����e���e-��v�mW��������7���_�a�V��C]��=U\.S�zO��X���f?��Momf]=���Wm�^�s��<ph�?w����!=�i�����k5s��B�Y���Go��lF}o���;@/��Al;9pj������OCzh�X�G���j�������I���2HW������w�^����vr��N��[=tcz��%�o�Z,Hy�b
RE���)�����s%�u�[Ia]�VRX����uh��
�XI��)�
�r]M�v����%��>�%�Mt+���-�SY&mX��9;�0����jT�C�i�mi�k��.�4L�^�E5���j(���?���99�7��H����<
 �]��u��_D3.�
r�^sau������[���^>�����G�
���S����O��d(�(^J�L����R��L�
���(��h��&�@E�%�b�qD��l���R��+P����~�R�A@�l[�#��`F�)�Y���������#�"W��������*�@$�����F*	D���:*O��hT!�� ��B}�J�H�,BI RH 2H RH`�[��J0	��[��z���W"���-�[T$����I��H`��)�+9��@���^yX�oI`,�A,�},�;VI`��*VG�
N4���� ������"�>�%��Bc��
	�k�@�����V �%��f�+�����
T*������$���H`��)�+9��@���^yh�[� ��
�j�6Y���x��,����}��s�B}r�b^�aq����5��zM���BpVAg���U�4g�����hKgKu�����'�������d�D��=�S�An-P�>�-�HH����3z$�mqg��fy\5sDe;wT|����.�(w���2�����;2Av���9���%��vob�U������!zU�3����(h�<r�@�LU�&�+���h����\7J
�CTF�UpW�O9��M��\�Up���>�k�����������>�,�+b4�Y���Ma��3A�q��y���Jx-��GN��^��!�*x]h��5zF���.�����:^7D�	��x=h���jocn��V��&yn_���&������%�-!MI�J�LRX������#� !�H�F�2R���[�\"!}HD�R�p��j���4��<�����Z��(�f�'%��W�JQ�l����e��s��0{�d������E�{��V��=��%.\M�.^�*�4���5�[�-HfX}�4+	n���+�
���\p���-��V��M�a�K������-��J�����~E���7��p5aK��UziS�sO-�I6����,k�-�fr����M%zo/����r���^�$K3�
�����M�H1e��]E��4Z�S����d��b�!�E����S-=�t�:����5���?�!��eN��x[y$�}7K�.&�� �W�4K1��F���a���B����Xo(�e6������}er;����o�X��5�B��$rR���e���erz��<����uU�>'u�^����G/������!8:x�EI 5��i
��������((���'���W����,Z���r��*�XG��d���q
�{0�!
��D�:�[��%O4^���\�F'����?�E�$���,?��8��,�7���|r~���
^�x	������5n����(��{�+��<����/���y
��~.���XZx����Hja5���7o���_`?f_2z7�����!���|��'�)R�u:K�2f�b�f�n'�w�Np�'oO��������g�2Fd�=�������;���H���x� ��{��{���Zg�g��V��5`w����~�����
9�����3]�t!s��e��k������+.��������q��3��H������������������2�1�j�YlB�s�)�Llp����o#j]J��n�YS~��5A��AA]����5�u������"�����S��Le�2�}>�{{�]�XN@���q��D��0�/�Q%ww�IzM��G����:�����U�Px�J�E�c�0(b�Cb\�n���*����<V;��ZXN���r'��b�����
a��z�0����U�3���M.��#����E��StO�-��]B��?����Q���:�Dc<)�����,?��L!c������i��RT��|�v[I��G�0����'�����i����e�;�n�Mf���	����%��/���M42�PW�H����������G�����M��;i�<��.�x��u��*�N�>EW��k��[�[q�:U��R��O�v�oi�����R��<������UPA��4��:��KI�N����\h�U��B;W��^��z��0_-��W{��e��w��z��"2������YAF�|�Bw�p���3�h_�J������B�'E�Z��}�'~nW�g�b<�X%I����$[F���j[����d4Mk���d�t�_?�}�&�uVG^vuSVGH?s ��0���q3*2�&��J-%�&��/|����
d�gT��Z�t��+"���z�b�n�{P$��k�R{>����N{Mo�����m��h�����6v���oW��do�~H���R�,����0+	����n%1u�������d����U��?K_#�I4��/4^b����;h���;��w���z���V�h��V���M8A44(���	[���Xn��������4w+p���V:�=����b�x�������$��X��i���J�`�*jj��V�~t���?������9aY��}c�V�'����$T�p&�xX���
'�N<�5U$�I<P-�����]��@v�aI9���$��x`���F0l5���&�>��-6W����U�x ;�@v��,���zX����
W3�a����:�F�A���z	eb��Y�%�\�!.`��a1�������M�������
N_��7c>�k^^?;��k���j�|������2Ci(f
��fx5DF�
@�4�g
�!J��%�Z��@*��3�������z�Gc��!�~��M%B�W��:��T������t�+�Nyg���2��e)	��z����lh��Yw�Kg�f@�x����vgG����d��x�;"����o;"�����o������oZ� X�`��lha&q
���yD�ej1�0�k9v��+�+0��z���#
�A���vJd�K%�&�b:e���_D��f 6�Y	�]�������,CY%I��a���a��n����v�l���@,P��N�0cb_�C�a����U��)Q��9��g��@a�;e���1��X���������^��1��A4Z(T�����9?-oJ�1���B�+\���0����Hf�~N0��	f�f�9x ��tC���X^����)���^H/�{!�n�(�B|�zy�����{x���^���hu;��yH��,�������]����,lD�
A	5]\a8
���u�����p'�Bo��H�e���,�u�K"�`������=�/1���q���_�.��.���E���`�*�v��DC�G�*�%�����t9�{�zQ�#���`��5;@/#�T���?	�� �1H�3����|9�{��C1�<��`�t5;@/#*X��H�?�����cP�aPlbP�r@��1H�(���{�D��zY�	MM�>���P���Rq@Z ���4B^�j
��
�u��|�I��K|80����f�a)���7,	��iUU(��)�NO������4$�[b��)�Dx'�g�/��?B���ThW��@���6p_6���{yTu�c����0����l0+�����L^w�R���S/�?S��X���95�
�]S��x�G+-�?]�u|&�DuXqy7�@�1nT4�Z�Z`�?/9 �G;d�2��E(�L,DVDD.\D���c��x��FMO�{@�@���-PQ*iY�1G�@Vd����A�[GVdEV�B��&�jd�"Y=���#��= �J �@��(�4���APb�����(klR��JYce�=(kM��8����U�4d��d�?�J\��X&����C^��@g�+��������B���g��B��zU�R� H��H���qCs�
�~[���WW.��q�a�j��[tT��[LM�iI8�"%��nC����������'O�D2�'l�'��=a�O~��@���Y���I�$X���;Kb~�5�tU~L�����Y�c8�kGY��&��Y�������������O,�P)�-F?������������ (	}�nS� C��y���35�:�J,�����W������[8�
���U�����_��*�S?�g���O�y\_~X/�Uj�t�����Z����P����|���=���z�L�	V_��q��y1��D1j�S1}.�R,���4M��{�������b��.d�s�>�������`}���0{�X"��
����#��:E���f;�H���F|#n��,��x�[������X�_?�	���?����L)
�W� J�8��x�J��r�>v�q�]e��b����(��1iC��W]s�j\�RH����d���b��
��.0(���������7��rTb����L�@0�,D�B�	��\�����J�r�����Ulqc*v>�kOW�������R��F�-��9t������s��F�^9��C�jV�h��H����o �b��5��s%[���[N�s�?`���1�1�cxD�����c,�����F�����8���(7��8���}c����X�#!��PA;���rlT��y�d���M����v����0���t����:�_Y�
���Z�#.�7�h_����2��j���k�����������EJ��yB��4Ye��|�T����<��wj=��S�JG�k?�~���2��LW><�?��rU��*��������<DM^_	WI��l��`��T�S�+��27h����`��v�M�qV:�l�����IS*�Jj��T���C�@�~6��6�G;
51�6���@�;��y���[�����"����!�m�E���An������6��h�A�1�)�k+�m�
�P8["�l������f�t���N���y���k��}�/7�"����^��D��n:'D�}q���������h6��_Af��h�)�L?��K������������y��.��&`OO����PIm����%*������&|���O�=[fXV��a�Yw��9`w����������X]��~���{�K�J��R�b�E�!3�>t+'u%��yV9�,���H�Ja�O���L����/�Tw�]���!&���	H|�P���?�dZ�>f���3H����fh@�V4�O+E�*-V�N5���vF�+F�h�^�?���C%
c��������T�7��!�1F�V�P%�eX/������9�.;����S�a4(������6�#C��Mq%�pX/�UW��E��o�d��b�as�V�I��IZKl�T
����kJ�pM�->N)B4a�Z$����a-��?x �T
Y�:�+�o�R;zpx��K���
���_���TM��_�}�C�(�V
��S!���({i��Z-V�vfY��)��t��63U#"�j��T
��zDim)*x��x�4�,�?=�O����w�/�h����:^f��t��6���:_���X�qi���R����/c
/�:xYf�X8N�Eo��0��K����x�g�V���i������g��m�!>�`�UcsI�r������Qr��[�l����dD�V�6��hj5�e����1��x�`��>��2�O�����8�����G������p�M����F��� �3��V
OO	�[+�({k��,��i_���=��J��~���+�_�@��=^������L��d5������&+�[�
'��-�0l�Q�i%�et�(���VR�W�}_����V����Srov�^kajdo���q{�����M=l��}�8|,�4�*.�����J�4�����;��Lb��'|�cJ�P�b
R1E�
��M%LiFN��U�)a�9����a,�3[Ys:2�W�����j;�����
�e�
��%�D2��][�+��z������o�������`����W(��������{R&���]���/�J��})I�*s���2$���k���p��t=��n7�#�!7�n�@nv������c�����P�/RM��*G��)Y:v�
��<5y�(��ZN����/�� ���}>H���6��R�Q���W��U����h����/�e/��1Ze�\���P��}l�k�rt{-G��,Z��x��E���,Z;���-G�{d�r�Y����lZ�nOg��+7���?�)n�V�Am��K��`�
���&�����}0
e���,����w�l����k�
5Z��f�XjyH%Fp�9�s����uk���5�Ib�&��XjY(<AO�P����!#j�I�@�xR;@��p��k$0��Rt{��K�H�M��94�0_
��U��;����&���`S���{N����9c�@��A���[��u�_Y���q�z�g.���!j0�CX��K�SE=x���lg+��5f=��
��8>Q|jE��7���X�w0�-�F
E^���;�(�d�J���&�@!��P�k�#��e,�(�P(�y�y�M�o�2n�P��9K���b������&��B��Bq%*
�`�'
5��lcE�!���5�`�8���952�����I��`���
�t�-n�P�q}��d��l>�����O�[�Y6���h2�����U������p����)~!�o�,����rAQ�A�x�^zJ�����a�����5�
�m�����C�2���2�����2��p����������=�u����^G��������"�N��&���	���	0�R�3%R�.���2����/�LM�F��o����sM��z��@d����%oQ�*���e��u�[��uY�r��B��w�A�b/���}�=��0������
v�^���9��h\I�^V��.��:]_P�C�;t��70{_	��/[��[h�#���9�c����t�e�K��]/��,��qH:�}�z�.��WB������7��J������g�q*�)��`�H���!�v����@��\vQ�YP�~I:�\
bW�9�UP�I�:G�VI�UkN�vT�m�����Uw����]��v8p4���#���s�|��pN��@��v��j���BV��J��-)0�lK
��`����^�=��������I�qa�D��j�J��;x��ld�@yG_�]&Z�[����X��C���:#[zX����{���u}�H�q.�^��1���k�:�H,U~m����?-`E�M`c��>��o�N��?�`E�ac�����W�~q���<`?�eU!�a}]`�T/���J���w��$_ ��@���z�����NE�B�UT��pMWv�����Q��,a�L�C�)5z�+Z�T�)bXE���t%M��J�����z�k�VQ�(��QXE�#WE^��f6�
�g���a4��������GyE��c�y�}~������A��b5K/��d��,�%��b~�����E:M��F
���cp����."h�)L��N��#v?~��a����@��}z�u���p�Z����L��8���L��j��j�b����#z�E��|�jH���%��D0[�~��o%�g�I�R�H�H=�PY�W�C4B���|�`�����`Ev`9�?�9�������_��weW��((��O��`�PP�%a�L�ub1���M�a�B���$����^�����V[�H��^|x�� �~��Dopt��n�F����A�����G��"Q��<�CG�^���o�C�6o��Q�
�����P1�#�z G�S����8\;�b��S395xD�4���Vb�h��
z��J��`���eg�Ua��\��9���_��&OV��������BU��@��Z`�����m*�s�L��i�2C�W���M��a��Cz�K������?�?�y����=����nQ�'�T�;���q?����w�Mi��3�L�����
��h&/���#����^��M��>Ry�|��J�����*��glq�����%J���!Z�W�����x�)����%
dU��+�$�ji��I%����W��J�$H8A��Qz]�We��d<�c)Te����G�����z�����^B��L�D^��J��94��W�p���~�KY�y��qH(����f+����y=��2����}����1?��?��"�H�)"�����E���a��nE�J��|��������l�W<D�0�J:Q�S��\����B.�TA����qp��p���}�B�����Nn�^��^dYq�_�����A���0�[d�"s92�����v���N��`sf3�o��*�T�H��h��W$~{��$k��,d��"Q�i����6���et6��0�\[a��"�J�H���X��JW�%=��^J�L�e���Z��Q	s��mX�w�)S��RG���;g\��O	.8I�����	J��0�^���T��f_��?�)�
�D��
��������D���Q�7������YO��p�>��=f�SK������-�x�;��d������;�)��NG:��2��rV ��Pl.�+!�i�}��O��GK25].&�E��<=y
+�=B�Q	�i�'���u�����������B�}@�D�05/�G,X�1!T}��TO�n���HC���
���,��0k��N�������y���,�c�z�D�M�_�X�������\��Goe'���B�4�S�����J[�`�&��MwO=��%��F��'B+!~�]����<��	�O�j�����r~�t~'L.����s�t��hD��<c~	v�?��%�fM�]���{1E��P�;Q,��S���X#+#��n��#s�v��[{����3e�.?���s��W�0�)��<X��f�qB��:n��^&Npz��8��	��J8QO
�!�������a���o�ka�"�Y�`�cKl;K��'�L����
�����K6���5���cZ�{F�#%��L���V0b�E��%S���E
+�������V7&=�L��%�����-��1��5~��{8��r�C���x}/��@��Z2U�v��p$%�>�`�����<X3�xU���drBN
�iZ:�!o����Y����b��$?R%|R���m�����Dn)(' ����%�P|*���GR.��l�^>������o6��X����b������;���~�],�;�����V>r�2��	:��lh*k�'+{<�2k�0-O~�X�������h�nL�aW�]�-���M����z��Y;S53c*�9M�
�7,no���S�|��T�J��8�!��4WGK�(��\p|f��KKs�zjo���1nT4�Z�Z`�?/����"R'����N�*���T4�_��T�y@��DE��X��3K��1nT4�Z�Z`�?ocu"bc��6kj+*�*�T�KP1n]��e�F������X
n�q��q�j���!���������H�����%K���U�W�����fD����E�-�z�������~%���e�d�"��N�$�Y4J2��q�~�����4[Fc�<#�SpAK����t�����2�r�-'��������g'��'g;�O�<��+
����� f?b�c�&���=���$
�R9�zH�]�4V
)����5&��l�T|0���N��w��PeDU�=i
�UG
fs��Zg�'�/Z��2�^s��j�N������YY����\7�����]*�/�^U�33�)0q���-�M�D�
�4[,c(����y��}R^�mz��������l���	zL�b9�^W^TUO���
�9�./N�`
XhM���q�!|�u�_%�SK����b��2�e������CP�
����^W^zU���6�+��`�7x�ip*S��C|�k�s1i>l��XI�/�fv;o�����K]z�\�y��X�2���%���HG���7[@�M(Ld��MhPx��.��v�!���|��M�[�o`����6Xo;�H������8X��N>�����
a�	]�����Yo�@V���E��I��J��Y����<Z���)^�c�������Y��\W��.����V���]Z������K�J��3���I�U�i�W���2�`�W��K�F�3�:@���Q-�J��Au�K#��_�\'�������qz�yB��S���\V���z������aM]���m�tF�N������HD"R4�uJ�Z�������itO!����(5j���#�1�o�)�KH�I�v9'0�E���l��s��]@�I�v�8',�����l��s��]8�I�v�8'(�E���l��sb�ZuN���]I_�J�^�}Y�q���f���l5�&�i���-�,��gY4�E�jD��m�^��o�_S�j���M����� ��m��t�L/C���n>(��3�!,;��/�1*_�x������Iu�E����)f��T!��-5�d"��?��B��X��~�7I@��B���!����R�.g��r5�Z�~��������^c�A����#U�p�;�Au�M�<�~���:��������������*Q�V$~��^�3	�%�^��v�=�����i�g����@��n�����%��~;jr������J�'^��}����MN����o����H�UF��2��t��8i�� ����@��~���$����uH��m@��%��b]��O�I�D���X��w����9
�:�"��HR��
X+XW�6/��`�9�@�+�t�a�u���v�6����P	����Jsh�������Mz��3��P��4�a|;@^�{�N{S�j���N�5;m.�� v�1n��
�+��
���*����$�e�T��3���Db�HQ,ue���2��(�jL��8���[U����z��qo�!�j�}u��z�����4�L�TM��.���2��.���S���W��'�������5�0am8&d��h�a��rH-���b�\�(��v��]�����"�]�n���J�'Sp�p�wH��~�����N������)56~�������Xb@�JP���w6��������V���"Jg�+�`N�����h~u���a����xY#�!�I��;��4��a#{�<����8��fD�KB�B&��4�[OiTZ����v��j�^�6��k>[-&m��cG���h`0�N���4K���O�&{�a���l�
��bN���)p�Yb���n\Z���(S�'�����^\c3�w�X�\�,�#_��2���������!����+J��j���V�X�ziwr��4�!hI�����h�����S�P�(�_"*��z�8�>v�����(V�#i-��Y��lI�h��ic{��(�w����uv������w�B����/�y����G��X���6�@�^y��C���x�����"7�j{K ������u���<�&���w�$w�n��Q>=A���H���4/_V�@��w,��h��is���(�����������Wn��������l,�����XG��X�}��N�J����y<�����:���)^n���0��W����p�sJ�G�A��;��4�}���|3�;�:�y�������bH�Y��y<��ng��ac���{�z�I=��O=����\2��L�b3a'��%�����\����t���%�m����p���h����"NmwP^�P>,/|$����ue����t9���meYp�
+n�]r�%��-���NZ��YY9����#�n(Q)����1�:N?�k�����@5���	w�DrN�+��z#�9����c����[r�a�:om�QC���U���'a�.��V��3��	��#��R�/�����B��BO�I������*�)�f1S�I��^uI0`�����:_��hMc�W%���*NLd��61��i������+��IO
+#�~�������N�u�*�����UJ~Xe[.�d�*���/o������-#c[F�m�T'7�-�!h[�+\��\���*�)�ftT�I�p�r`�����:_�sMc�W%����o���F/s[�XMn����^����xB��B���C�����2�l�AU�u�����V��="`V��\}����l��?B��X��cc[�m�r.Ej�����e���o���O���Bjv[u�D��-V�j�O���]��46zUra������l�2����4�'������;�O��\H�s{h��~[�-�rP�u]�+��������'��oB���Q����������Y-�a��~8t��=�����j�������sm-b�f��W��/����k�s5g�Zx�|a���x/=�n�s��n�L��\��;,.7`���f9�����B���8UhNH��-��R�-�q�k���Z���E��q��v�[�/�rlU���V�+�B�9�q��yX����4ixb�)����=&�yG��Bj�����,>�����������D�	���
��n{�������a�\v��n��U�_�
�&dQ���,�Q���W (�:���%QO0T������C%�� 	�|�@�0�����DwA����P@�-��*9��,`�S��Co��7���S?�0��p�����=w��C=�	�	�`�T��;7�38�a:�G�"L>9�Lt���z��?���`V�0mV��`�4`����0
�t�&�4!Le�=�$q%ak��f�P�t�dB4�0U�������F����#�8�������fb"["��0���J��|��|���|z��|*��|���|���|:��|���|���|J�w����l"WUfr���5N�x���A@�m��_�<����U�}���������q-�A9-��
���*�$����
�QMx�Ua�s�����tA����!!�D���_||a����m$F��+�v�B����(���(��	7�C.������_���.�ghx`���E
�H�����x�@W�9�
M�| �Y_ZA1M�]410�^1$���J�m.�hhx�����E�H�@������S��)F��pZ�9dI��+��6��v��1r����6����E
�������������`����@�
j�`P��S��s��s�-�]���P�
~�
~�]��w_L�����Z;�6^�xx����v�m��5���b�PaG��������6n�7/������
�W+�^�C�}]oM�����T���!�>�c�M9��������W��{[���:[��
����{�
��Y����0����k�c/0@����
�&a�n6��Z��;Z]5'���pz<����(�[�>Y�"�ml���8_���v,�����eY�1=��?�ov@�&=o*�����^���J1���Uu�;�V5w���VzD+��E��[��G��n�m#V}�S��M�|�i�H��E:�Y��.��d������S����<%��I������X�g���?�����������'O"�&�u����1����N
�$[R� �4x�V+G�3�p�S-h���5F���M����l(%���A����:a���
0�����UZ(�*�J8�:;=y�2_(���Y���z�������4[b4>���
�����l��
"4*g����;^=��=E�.Bu'�{�y[�Y��-��Q�K	w�xa=:O������y��u��>���Ci� l�F'�o�!��3r��\�B�H��*���c�1K}�]\���e)��	�_�"}��X&�U��3���P�T[�,�1jE��)����S��/K�������Z����yv���r@ ��?�I����b��@g�-&7�Q�Dl���d��"<[o��z�����E���,��f���7��n�Y�y��"�[o��z�������������hy3�",<.>E�KS�y�Ho�R�D�i�La]G��Y��\W��.����V���]Z�����OK�J��3���I��OW��S��CK��C���^%:������)[.�wlrZG�^n����$�Z�BL��C(������>"���`��z]��j�j*?���Tk�P%|D�f�<���UHkU�x��<Z��(�#���i@]�N^��7P��g������^RgN��9	�.s�P����?�gN���9��.o��O����=��fN���9��.k�O����;7F����*q��i���e1��*��n�����xa<���4�R���e|E�f�u��|�ko�di&^A�Y��&]����]����;�_��+���x�����n��V�_m���=�KIw��=�2t:v���n��Dp-NvG������m*���d��x��	S�6x�c�����H��p��Dhf~�N7u�;���N�����9��K���\�O"�g�a�7���f���<�c�4���q�Bq��#9:��<^H5&��������UYX���RADb*<�9e�.��x��S`����j
��k�}�q[*�>��]���h�r����S]
�bE.�����������Z�t�@����������S�]�PxS+5�y�P�S+5Sy�PxT+5�x�P�TC�����9[�O�+��v�h���1�����m���>����M����`o�6+�l���OZ�
�R����&��g{��t
��(��n�fUF��l5�s� 9#Z��6�5��)h��4���tCQ�A�x�^z��_����qw��#V�b�Nyd<��<z�QR��?��h���r@���Z�7���x�[|�%x��j&�
���9ey��_w;G�rP�_��Uy_���j
���j
��,� "�`!~��`u8���Z����Y[k3����V����K�"[JK>Y~��+����<H�R`��q \��r��H6���U�o��%����f�&�'��XOt/���P}N�7R�Xy�����G����N5��UP����52:�������fFP�z�E� c(|�EVG�?�1�1q��
�A�?�1�1q��sP**yIJQ���j5�+/d.;yX�F�����*��4s�9d�AU�8�lL~������1&TuL��S
�T@h�c��y��y���S��<����1&d���<��yj��j��|e�����PH��>V-��?�"@}��
��?�.��X,X���?���@��e�(������JC�UE�Xih����=���f�n���`O����]-���
	pB�kj"?S�3���N|����`#`�|��|&�[��|C^�'<�����
+9������l�v���(��
	zL{??2�	�4
�\�\�_���K�% �������
>�YA@�TS�D���5�b�v]��_,�zh�i��O�����{���������c���qN�xzr��Y�z�^��!�rou���_�\�Q��h�H�Qs_��q
-#���B����i�q��(Hx�#�9h�16�����������
TTV[���5��d�9�w���B�������~N���&pl�q��B�1F���P����G�|bX��n���'/#��R�YR�	,����\���X;�	:���c�a�} ^>#bI��]6>Z;D8�\�x"���h\�.@��K�1�R���,5R��4��X�r/�_�'��r�3Y7CQ�L�nU�[��z����v=a�+�U��7��`�!�uI�@mgW��,�9.;��^g�\g�������u�!�:CE�����2����<N6|����%B�:sa��:����~��G��u�r"m<�e��H
���\&G�!�k�wD���^�\����M�e�W4��:EC������s_��u� v��x��"��"��) ����+�]o�x=�"�r��*�F�����
����^������;�2�<��2I��������������vE��*���w]�p�����".f��`Y�*���u�u���B?3����^Ns>��YU.e
��+�%�0�+�+�V*2W*����0����_Y�\���J}8i�*,6�R�k<@�g������i��Z�9����c��\�>�V�Vl�9h/�#
�M�y�Z���i��`�YUS.u�%�L�������=��P����J�9|�m������&��������d�R���.��~��S��������Se.���io��r���9+���Fi���z����j^�C��b��,QRA�f���}0�����>C�U��L%�T�J����V_����m���({*hz�=��t<�n�f9�u���Z5h�0�Z���b~��`mP^.�_
y0�����vn�)������������Vs��'�W��l��z�xI|d����u�Y��_�!���jKc�%��+f!�U��b�������N��A�_u���?�jf���g���%U4*�9*�����Dd�`*���1��Wu��Y	K�+�;g�����VE2�����y �3@z��!#*���7P�s��(���4�z�3��I&��9Az��m�KE�����{�������Wx���a����k��<-��	K��#������Xc��'�����zi�;�	N�m��a�>��aC$)f�3���5IR�I�Q=���6�4���O�b�~AYTC�^���M����c?l}6�>F?|��H��Irk���4Lw��K�Q�$���qX����}l�D��'!&�y	m)�l���r��e��<zD�j��k3D)�V��61����:���>8�6���~+��'���&��2q+IB�"v����}���p����&4[��EV�����al*�J%�Ei1�\����Y����xp�����}�$�����J��5��JS:���bw�.��b������.���1��[���X"7��I�)++]��.�Y�~]t��^I�lf�Ay��v��T_�k��7jj<�;��O�jI�x
!��a�%���:��=-�=�{�la���������K%����m���+���+�|�����������-�3g/6��T��n�}��>^�	�Q���2Y&Q�.���"��tc�7M?��h:�oE\~g������itq���)I�g��D��SzG���&Y���O�/�\����a|����^X+��@�~������y����_N���������1�����}ji0*j��(<.l��FP��sW���lu�X�����8*�	T�nQ��XkTz��1���������{v�L���mk�yz���n�	���E5:d�Q����������;F|'��r>����<t�w�"�����:���d��t1I�.18>�Q���O������yGs0 l���������d/���k�i
{{l|�����$�7��PE���g�O��}�L�D����#������(��}c�WlZN�G/��jbj��";UE����IUQE���T�]c1UEv���JT9�*�HU����k,���NUPi��"'UE�*rRUw��T5BU�UE&UE���T��*�PUd���NU���";UEKU���R��7�J��i=��H3��+��'�Iz����b�V�4��SW[�>�}kH�GqI-�i��S^�#���Z�u������z5T�u/P[� W�6�O�j5�-o��c�uz5HFX������C�cvO�-T�jk�w�N�U�P���R�C����z�0���0�O������3���5f��N�nt�@�����2��(���r?)�4��(v<��������K�M:M ]�
^����N��I-��n��a����Y6����$�^�����C��8���?v�_�R���Wl��23����OMsB�j�-&���P�,<p��5��t��~��[w�u��m���6\����7��d��|�a��Lk�iV
�#���Q&4����1lhG!�E�bY����2�F�
�>]*����������)�i�gR7�A��)��H%�[�*���%�G#b�.�j��R��9��*��p��}��}���r����}M�����_��h�����)45f)��9��U��s��k��7�����~9���T��5(y���_G�hT���+�4�v)�
8�
T����$�
L���oP�}�"U����>������xM�A���q3�)
��x$�n/�G�v����C�|_-�x�����.�y�C�|�9���Tkx@�GO�:`Ez�8�).���R�-N}�8��nW���i��Z�l-k��5�
n-k��5�����&���:�!���No�*������c���P���q��������������tb(��\%�����o���|Z�V��~�T���~����M�s�O�u$!L�����UE���siE��)��medK7J���^zR�n����xR���,%��jS3�j=�
��o�����:�����c"z���/%w���X4g��`�.��P�m�b���jJ�%�oA�s��>"c�V1z���A��������������m��j��F4���\��)dqy��
[\D0~�Z\��4���jJ��-.��X���Qd�U{..d�\z�l���,�[\Z�luq�rh�K+R���E+-.�����yZ�}s��������?�r��@M������9EF�1���;��\����u�B��K{�\\�;��h����sL�������eG�w���s���jJ��-�� ��Ab�@�;Wl�\�m����+��\�k���w���r���
�L�&��7�s���]}q���o�?�)�6���1���#2���s����v����b���v.��sq��\��U���U0
�BW�;����W��U�?�)���[�a�[���~k�������X$�n�|�"����ob�����E������(,-'{�x���G[���}=���	&����U�HI;�����v�g'��'g;�O�<�d�P����=a��`k�����b1.6M�lIq�������n*�5�L&c��Qe���k ��������v�el@��?���h�U�?�'�+��<�_����F���\�/���!Q�\�w��������(4&/��vG�;,�s)��:�JL�������|~g���^
v���X%M�Oo�0_k6�	yAw@r���N��<��?�C!��
?e��|����Ti��������(5��(���DH��oL����\,!6�z���I�����OJ�xi��!��y�7#��Ob�\P%�����I���d��{�(��P*U�B���,a�fbaifb3��F��_�f\Ah��a��"��1��
��@�l��E+t�YC���[&6�l8�*�&W0�j�T��IO�����sdD���sI���1{]��������t�~���x���t�c����hFH�f�����E9JW�+�
xT��Q�K0�8,��(�eZ�te��k��r�����
��`Ve��
|U
�jP:�>5R��{���N�P�4��c��&����%z�O�����)]!�
aD��$�F��y�,�E����Yjx��+�J����kw��� �W�W�����E;$@Jv|�EVO�6�<����Y\ IL(R!H[�I�o���U���TX	e�@���vU�{�W���i��p�DpBXLJ�V:������&.7��� sS_�cb��"n2Y+�MV�,�r��r/Cr?���E�~��e?O��UV�
�z���:c�y��%^.��V�����
(���n ���J���7������)p�DpBXLJ�V:�����ULh(|)�A������"}E�d*�W��*�tY����%l��~(U����l��~���+�te����+��fn�y��%^.��V�!��!��
)���n(���" ���7�+����Zp�DpBXLJ�V:�������ji(|)�A��������E�d*i�M��Jl���b�D,(�	��>'��~F�"����c����6"��M��&����DCib���v
�R��&~JE��Vi����� +C���	��d�{�1Z�H-���]C������v�{��^��/��5V�Tc7����Ug�����e�����XZ�����^>�������Pg4[*�����>��� ��P��|�����.��P��|�����.��P��|�����.��P��|�����.��P��|����X>�>Db�P���e�/;_v��8M���tu�����l�gI���.�4�.�i0��z�W�%
CK_��o����,L�I�W�`�6�8�M$�����%
�s{��[����Pn���t<��x��U�����]Z�u��3��:� 7Bt�,,�n������{�=�>��q��<+���Z]������/��������u�����v��In���C�D|=�S*3}R���1�U�TE^>�#'r��05L0�,{
f������X{��Ync��v��5{`�X���A�1�z*�X������D��2��;c`�yU{fl����-���	+rk�Q�-���[g�J�-*��=k�N�S��p*w�#����Gg���ut���Iz"N�����T�8�;����t��3Ny��9�T��('�9RepR����#^��)���?�5�7&jo�&��C����%0��������������>��5�����X��.�k�=��?6�H������zX�6^��+dg�\.n����7��,F�-��y�s1�]���p�����o��_�Y�9<~DVl�����a�����%��:�EM� ��KH����X��/1 �x��4#�����{czQa����������-R�2��O�i�L��f����e��������>f�
+
]Q���1�R>l��D���m!L{,�i�@U��(�R�K��6�
*B� %�C��p(��:�
�������f�(_V���N��r���eg+�������Q]�TH���K{LB
&a}��M���*{L"
&QM�TH���	{LL�aR6�yB��1j0��tj�����K��k<����k��5Gh���G���k�q�l��~y��S9��8���p��[�k�A-�j�����S�s���Mc��L�W�]�����/d3�"��0������L���0A1����TX�����5��kT&�����=���+�"G���d��f_I�e���������q���W~P�_}���jkU���O���k<�"�X��������#��UR�m�����&�\���f�f����
��EN:>����������j�����?�����-��h�-�j��D�n���9[�WeZ����C��x���H$�Baj���A�"|�X��.���P����Vj�����	?��:������h���.Z37�H$��V�����7`���N[]%x$�����e�b�s�u��Vn���Xv�.�v1�3i�E�������.��u}������:�����.�87�M$������(��.�G��d�w�u���&�ckg��]��S������Z��P�u�^�{]���"�^'�`�u�{������v�3f�dr�u��u9<�
X%���+�D09��4����s���%u�-���)R;�4T#�t��{b�9��I4��Y�B>y�KG��w�K�Ea���u`��;��=���6�@<{���-Zg��z��C|�^Dx���h<��`�U�[$�+P��r�64����(��tf�b/&�x�_ue��RXn�V��	���
��/c",�?������59�r<[��6��m����<oC��U�s��j���y�{�bT�����`42o�;�-HX��n6|��M��(z����qm��]��}w�f#P�y�Mu��h�`�?okp����m6|h���_���5ys�k��6�������a���u��m�~k����W����gS�F�my��	�<�
��/c���mn���U�8�\�m$���n����6���:�����[;�q���b~��|��`�?o+��-HX��o6|�h�m��Y$u�yk�v����m(�[u��]���U�4��������������5��4����^sy��
_��K�y�:���F����E�7�
���:8G�/B�o��a����`�k�|�����3�^��&\����%�r�,j@��U�����e����������5�+�WC����z�E�v*�5��6\��Z��:.�uv��Md�>oj���m=�5&d��T�3_��l�`v5/j	�i������W��ZDR;�h����$��m��[�g�9�"'U����:��������i�P���pu6��{���O������]�M��Q��Y��Y��l"��Ax2�?\��{Y=+T����,�1l���j�����������}�����$� *;��;������1�>X����3��~3}��Gl��O���Y^�$�X
}��;�I��
�,�l�����Au�E6S�x�J�B�X��*�p\��Dmw�%��]����L8���8���IG;T;���K��0��9X�<*��.�O���?�A�c���hA����a�d�������j�6�#7����>2�]fbs�k��!��{$��4z��G��1�e�p���@�=dL�	�:�y�?o~��Q}�6g����@��uNu:�/��~���Z�zvf����b1��T��������Y4f��$�3��Q�cg���{��5�u8�MOv:�}�������(�����tl�.�/����G-w�:�64�8O+���L)f�p�3�7S|���i�L�����V�Y�E�J�=|2l/SG��h'y"� TH��a�I`�\���L�I�&)�	
���u��c����d��`�n�`��Ars�������
���P2B�-6��W���$������&a+�e<K����v��2c�{��3
�"����z�h��!��Z���0vj�Zf3���J�`���������f=.5�YRl�Bb��/m�e�Y�
�	'$fa�).���KwL�tF������������U�i�����+���7M�O	�b�'����+���<.�^�����@,<8�U������.
����J�a;�AS�F�@����'���B�M�Y�s7M���?�7�nJ������7%G5�)���4
�
�����
�����,6�=c\���p/�s���z!<�B�5�^��������`-�����"����U[/\|qY�^����|^��z+������,��N�I�8=}���`D�!����x<���t���t�����n��j
���[0��+�t,��������z{��������7�o�~��4��������|�	9������Z�2�a�y��@� �}�gk@�M���D�
��'��|=��?3��OY+z��A�!�7#���>�>t$|�%������@x-�C��Z����(�d����)�����5O�x~\.��E_!�=t�����t6]}&�y
$�|��H�c��r�8����,���
~�\�U
��P�-"�N�W�}��~pw����w����w����wp���|�
(�Z�$��r+���gN�B]��*���?�����o5�<<��$�1���,���By�>+������S��K����������I3����7t���r��T
���	�	����|
�8�ag�,�����9-t��pc�RN�z����5����k�R ����['���������;���
���;�n�5\�>!��}�����S��
D�	��;����7�X���?������ @�y0p��O����������
�x�c��'��qoc��]I��f�Ipgv����"s��=����^f�Jqw6�:p�.rln��]�[�e��|0+�pO>����=F
���w��,������������[����w�q7��gVjh��n����/����;�n����C����-���8ru�����w��~����O�J�sV�AJ��8gK��/1��3��P��������CU�E)�B���$���w�*�QtQ��Z,$OH21K.����A-�9�w�{4����s������$�A�����$<$�����ix����<�
������^������m�V����?��6�os�yG���;�O�!��s�p�>��"��>���&�'�a���*�H��C�(�P�}h���G�>���Y��D0
!L�yXu,�6���3��e���5.������9w��d�����%� 1w8�;�tCG�-=?-��:���;�em���Bl��*�&���[Epk��BCd�s���I�!|���6�����@Lq�������+/����E������WU��e���
�X}kY���p6Zx��\�D����G���N~l~�&{�\�@sKS���g;�\.As�=+MPw�����=��o�Bu���m����b?�k?��KX�F�)-KJR��pU8v��-���Q�Z��A�_	�8�C�sjMo�C��?��#�d�N*��|8~{���4��&�]H���~��s���(����4J��k
/�D�N�.\"�t��T�Wk:dP�U+�iE�BdN"��1_�H��u���s�il��*��zvjlM������qg�h��i��R���b�-Y;�-��2�|��� �zP���*(pYUTUsE�<
��A��W8�J�`�,&V��������Ed��
*�%**V�#2������Rg���k�p�go��^�VW��g{���y�U���g=�,�y�Q�TZ�&[��|#?���y�Z/c�����t=Fc�����3�b�A�i4���>��?>�|v��|l/kh��
G�n��Y�OB#�W�o�����2J�
�kA�n8S�<~�+�u2�@wX��@�z[�1��z��@0�M�p�[$���C���@����u�}�Y9�e3��%�����	|�%�0iaw5�gbG�\������cYv>Y�iZ2dUj
@��
�(����{�Z��vZ��.�ak�������z���w,ro�2*�sHb�:�t&���:!�������������g�D3�E��u�k����������%W9��?�/��L��3%;bPX
y�	�3Ez��L	���5�������Y����\f��%q��"��54S�!�g��sx>4fJh�)�c�eK��(C���y���3Ez��L���U�)Q��LQQ.3S���mf��CZ�L�?/��
3%2��H9�u��w^��P���	=�'B���6�����[��9h���p���$���jQB���0O�O�����!��!�0�����$��>�V��p��(��[��E��V�U���>I�'{������0��1b�H+�J����?FC�
N��0�=��p����a������1?���c��d�;J�����5t,��d�,a9���11k��g���,	.�;�(�Y�F����I6�d���:O�qR(�(�c��@����b�y�!���	���^X��4�,sl���fpJ��&b3`�'-`�H�P����<�n�c��������D�S�k���s��v�
���p�{|\~�p~~������g�_Y5
Z���s��I����M��_$w�������S��(.G�Bc�8�����	]�b���4����1�Z�%D����x8G+,/���{��������W��#2n��z9�:����C=W[���`r�l�FU�;�sw���>"eE�z\�s�9-@����p)qGW>���2��������<8����,�\Y�X�aq,����t��������nc�u8[�|'���GS�k�,�X���M�[��m��_�_�v.7O�!Jp�������h��$�h"��+�c����E��m
����	.�B��m�8��%��x������
��h�u���RWi'4id��R�JQ(����OK��3hV[w]�kJ�uuZ`{�5�q�]\�5�
z��i�@�C���x��Vy�bE.���aM���*e���gB�0�R��~9:<�����kj�45DM
pSC��
����Q��2�����0&�����,���p
�n9)��D�gA#��uJ�n;%����v���$4[�K�(���**:�/A"��K5��]l�_w��9LL'������3�j�g����_VRj`�,��3�V�8�_m�B:Vuh�%�&��6m|XF��7/����pJg�0R��T���m5�b{j��g��_Ij�%.*27,nn�������`s��2�Y��������[���ho�+X���=�>;�Sq�T�,��`S-n����P`���=`��t�����
8�?{8`BS��/=��8�p�ZE�4��8��j�am�k�!��9b�����Dm<�O������&���U�:���u<�'U�P�x�$PX�O�0��2I�YU���P�����E��E�}��;8���-��GGr��8��4����}a��W\���P���%�l��
�:t�Uo\������ZU1�;�]BN,
D�C�/]�	�~{0K��8�����00��Vg��+����r%���S+iFs4��*�g����Kfn�x�z�#����'��F���qP�����L�I����d��h�������oh�R�:JW��J���hh
:!�#1Ew6uA$@Y�T3`���eq�b�l��.��k����r�UY����+�x���H���hQgSsK�W8������(�����v�S���b�^*.�������z��%��,��jni�.��d�����2��[u��7Y-�y�A�7}���]�	�'�}��%�b�Y�3�X��9�-�c4c�C�qFM��c�,�	�,��g��O��Q�4L��c����):7o�W�1�	
�g��l
E�f�f�D{�rWc�D��s/���c�� ���*�1��P*k<%V� �y1�E�~��'r_��L��B����:��/^����b�b��;�-�vp����%��|,m��|Y� =�6H���a��`��������<O�&�V�1�����iq��4c�a&��`*5k]!��8w	�����|���.�J����s��Vm�8�����e�7������q6��N��r����:5X�3$�gv��
��1�C2o�bb�����2X��lG�%�l�/��e����eR����0�3j�6�#�^��] |�3������ifG��Av���r��!��FK{��].j�6-Z�|�x�����'XOm��VK@���md���Q ���V�PvA\�	R�s���4q6+�a���� ���<_��PcL
��[������i��4pyj&9��i���st��gv�������~�i�69JGSW3���4���3���~���#�~�C�`�<������������%1��!���;_��:I���+�u����H�gw9P�g%;��uQ&e���y����{������R��ZD�sv����=8P��,�Dq��Y{4���+Hv�f;0��B����d��YG�����?~���&��	Ui���K�:�B�M���UK/�y��|��8��$EU�?f�b��Y����I���� MV��e<�J��q5]M`.�����4���a���u*�?/��C����4�_�WX��.:S5�����;��G���HS����/9d������<*��*�<�<�f����O)������fScn��3�
�8IZc��0���8����c>N��X@n��*%��*.Z�Zt>�"��)������uG�:�PP'Il`
�����N���M�<'QI��8��Var���#.�l��������b��7~~��?��p�������#�������A7��QzT���.
��B����A�BF���Y��{������E�~�h�E�%|���"�o��a��,�����mvS�e���m�I~U=�t��W�����,��!������Y�|:e�M���q����Eh�-e	e�C���f���t��&���T�@���<��{��%Ym����B�E�����k���I�����l�v[|�{�
�����^�l���u9�T��(�l���F�-��p���!*i2k�����k�#"�yU������x}��%8�����6���vq�(����1���8�4�=��.9���(���c����\b�����A�
8�>��$��=��Z)q���;�P�3T�=�rc8��{�$�����Zj�H��ei������2O8��q�T�SSXD��#��^,�b�Ks��b�K�����@y���V��;(tx��I������R	3����Z�s����RR�-�gL���tbU3��Eq�j���,��(�Z{ue�j��
�s=��Nf�I�.�7�\%���`-�*���*���X?oXJ��I�%����D^�GJ��*�����������%��WA�L;���e����X�l�V��(��Py&T�Q�����@iG��C�����\�m��0K�z���>�U3"�Z�R6�.@.W���*���YAe&Q��x�;��'�0]�1����\��:�Nu���X7��������lq�b��Fa��*#@Vj�2��	Z{a��[���^ec��e�A�5�*�]��w`�:k"�_�Q��Y$����hg��NgZ��R��v�e��{G�������gI������T6�sv7�Y�ys[�L���n
,��[�+���~O�+�Xc�&7�ZR*�
���������M����g�t�N�3��G��^]�ie}p�NR@��Y,�=#��&���E49���EW�jM����&�M���&�M��K�6\�l�@9?��.��K�O��]�Sq�����*������*(Z�9��?h�����R"������x$��]3_7?"g�B����I�s;�/�z5�3��Z&���@m�l�H����l�x�=C�G�BDj�Hf=z��B���Q�����Q-�:��,Lm`�A�8�������\A��Y-�?�n��a��O#��j���2�.�;�P�����"�=w����
�)v�5�OC��]�2h�L��M�G���D�\��|��@^��``���/��Nm�8�|[M%�AF2X���HE��5���fO���q
m4�,S��hXE��Pk�����g�H�Rd����+`��&]��&�� ��>a�p=��'�2fX���0�I���U����
������br�
S�w� �j:Y����V�N��P����!/JSO��JW�����>���L���{����U�wRL��R�E~��w��c*�'f�*��)�h�iM�RJA�n�ID���������2;�@��qd���6���/M-��F����Li�k���EJ ���:�Ugo�E��#�X�D���=w3O�j����l�u��V����jd���Zn%�6-oU�E�����z�T^gYS4+k'T���r;�1�*kg���*Q\������UK�������u�QmjT3�H.:��v���Wnp��WG��tY��T�2��%+�7�/���2���&k�7�6�<�^{��{��S���^}2���6d�����$jsb����DhN\f���1'��;���e���Wr����
�X`3��z���x���"��{]L��[��{]�{�7x��4�6��6l@��V���,E���6����K6�/i�]z���x��ty�CU��F��^jR���W��U�6 ���E�Qm�d��&/��k(�����r.����5m5��S�,�|I���*�xUDApT�\��|a!Y��l��d>O��	��lQ��kb���&V��s�U+1����%�s��FuP���:7^-@vw�������X�<�5������
aq��D������g;A�L�i"���mK�?kQr��rU�GP�T~Z��x���(<o��]�a>���\�pa�����!��T+2
�C��W�t���)w����R]��f��(����R����C��(^�s&����
�<gB:gr�.P�rN�k�4�P����x;A�L�k ���o��^���.��S�7��=U�S��.k(�����W���n��t�<U��U�)�����pM��4T���M
j��U�{��x���Y� z��������`��}V�&�C�NY��W�j�����Q����=X^���?|y��`y%�J��2���'{�[�>���
����u2��'%����rp^��
�Wp�������'�������g�b�=�m8�>4���_�����4���{��e���n��j��LW�x�����!�:�<��/�`#��Sr_3:�Z+2�JLF���nLD|��9������9j����&� 3k�q��
��Vp�K
�e"�}��|eK>{�,���d��I���H��H��\��j$�r�|A%W@uI������#ed�i3�7��p�B��j$�r�|B���0��^�%�d$Kb��(�ml����%�
�t�����C)Fw��*���l�����f3�� ��W(���L�������?��V�fg��l�w6c6�W����xm��smD��P����"e5~a�
>�,���=��Q������m,;����u��d7��z�y�N2��k����%�.�?��h�����:�������S���$C8s�D:tQ����AaxAz#�^��a�&�q�$����������8/�Y��Y`b#"U�_6��q������;�y�%�w����!��2���1��Ms��I�Lsv-�G��,����;�c������f���A�k
��zS�t��������S(n[�c'�W�_v��w�4��-;�Y_��3�
�(\�B�
�x�3�G�E@'zR;�����������
�\��ZV 1��b��t��q0��@��2T�v��[��F|���J��+Wy5�]:�uh�Nf����
s�Y
W�H�!�2����n������_��go�o^������Xr��{6d<hiE��o�\�c
�
�+��5��:��v��V$W��K;j���s#����l�-W�B��Gv�k���P�jP���5GF.T�jS���2��U�J	�U�*�5��c���.P�`%���t��]���,��H����VR}�<����F�y�c��e���e��@N�`�~��&�\�'P0N��g���`q�,��b�$"C�f�����&;�?�����������e����
]�����hd�����|Q�6�F����%�
2o���J��D�j
���	_��	����?�@w��7�j�u?8~�Z�����c�0�$VD?�@�V��u�)�_���,FxT�SA+$c\<c-��JKD��S0E��;���/q�C�1�16��l|���u6+�TO��UT�|���I`�x�{�����������w�����x)�|8{
���wL����N����>�s*�����rAG���$�u�����b���������}i/`j�/��3J!�t��k�Y��gj?��e�)"��1;��E���^���2(k��P���i(�o'��}�@U|��Te���++�Y�J��d��#�c~.����j��&|��d��8!^�J�>d��!�c~�����j��&|�G���$�:�N��Db�N2>`��$ ��`�����H���*5�����(�v\�=�5����e���/�(-�v���������{����9>;;��`���G18z������������#�6{[ �[���?�����YW
g�vdC �0��7<����C���ug��in6D�����*���]�$����m�;f�����|x����W����A������@���
"�����X�f� Rbh����P��n����a�X������H����er��5�����@�f�s���H sI�w���c�V�����m�%7�a��n��1���|�
�w�x���=���m�X]��:RcZ�!�!1<#���8������bV4�(P�N�L��]&�1�.]��&��@��N?�e�[g:��P����M�O^����8/`x�_��������s�����+� gm���x�MXW7mW�Oq��2}n��y�"k7e��b���F�u[�4�[�,w4�����o�~89F=�w���2H��7���4q=$�u�+�	?8>�
�X�dt��x�W�Y/���v1��lk��=Yd&�����_�/!���l��LI5��/�F��#Sm�g=��J�3�4�����R���%K��	XU��R��4D�*S��d�1�AC���9��0�,�i[���h�.ef&l
\Y��Y%��U��Y��T(�����y����Z#�_��$�]�	[V*�U	2|�����g5.?6!P�t��]�W�����E�j�����k��x�+^�(���xy�+F^��#��{�@p�C� GC	����),����/o���������~y9	N>�}� ��9.���t����������������>}����:-E�wprv
�3?~r���{����r���N�?�����S
�|��S6��#������t?�
�����/	��W>|<}���$�i����u���������_�|>��4���^��^f�A�c��
�/�����,S4�`�G^M����w���C�>��Q3�";�y\$��9X�2u2s��{�������W]�B9|�<B�����~p|��I���:x������������D��'��Xd#I�B���~AK�C�%G�gH)=�\�J@v����Va�
��F~�Lb����8H�2����*���w����w�qZ�cM��F=!; ����th~:T����6�0�c��&��L�	��>Au���9����s(����z������2Zw���)�=���G49��C�r�=�+�n�#K�
4p�{���6��-��N���W�,�U+W����6`�F� @��l_N���?�1�T��fD����l�j�y:��V
k0�a�:������j�B�Nxd����u������_��P-���eR�"�
��#�K8$��w�_�w�=�K_�G���sXw�@e(���X��(�L�)=4n�06z�#*!j��^�
�C|(o��[�p�Er
l�iu8�r�(�A��IQA/K�G��(s�R*�����g��`��[^���g�s�]�T��]�H���l
{0�A;@��Gp�, f ���Er�Cq%��,��H�Q�����pK���G��|�E(����$>.+B`E�8"����ws����3��5������.�2���#XW�>�%��`�^�Hj�@2��`��{���O����B0��tO�'������}���	������_oD�ZN���&J�0��Fy�S^�����d@�0%_^���Tm��.���.��Y��)	$���Z��:0��xA�R��vE��1�KQ�R����vW���U���]:�"�4���Q|��N�
�N�����=1����?��5�����gO���?����%��������=�5$0�{��#����y�Z/�Y�!�|��`�rY�P���J9��[�:�t�����9H=>�z������	�3Q(/�n_'�	��J��6!��C����P��n��\�A���[S��M��P��5-�[�yI.���l��*�����9���[s���]f_�fH��������}i�zs�@E}�� c�&7�2)�"�a�$w�t��B?K�h���u�/��F��3������{��R7(�)|����,x`�sj^��mlO��X�����f~����o�u
��)�iF���\�����GL(�q��bXis7hv��P?p�nN�7"kB���t�`%ie\�=.�7w#��f��
 �#lT�D=����m���~�|A?�������;��,��M4c�{�C���~BY��>�4	R���w���z���%<~��6���z~�  ��f�r�XmV��ni�����p ��|{$�!`�U@_���hX���������|[*G�Y��1*��4�l�1�0s�(*�YbRV�kl��Q@c1������cF���c�,�t�.��Lq
{3��pt"�c�<P<�[
k���TO��w��B=�A��V�x��+���R;K+=��J�RXiW�*������[^�9��@�bN`���[���cH���#{���=c���1Nd� "G��R��hT�lX����Z�5�599?-��s����u���b�b�������;���Ax���0n�s�6^���>�?�[O�AK.�
	>'���d�a��-�82���Z���^�=��������p��<�r����J�$C�E���=.��k��-��~x��1JI�B3LE��j����\~d���-�t�������4>E&���=x�';���XEY���D���]��^�cu�x�����9l�'>����r�Ct+�/��\��F���!>LB �A_�A|���f<��
]������1RWd0�,n��yL
��r�u���EL������N��G������+��������
��8�����m��yK.������!N�xYCU��z��=����2�+�,r����s����Ybq2rK~������*����m
�9E�������M�|����=�^�B���4V<v+3���`)M�u����/����p/�@nq���V�� d��\F�)[S����^����J/jk��v!�^���v��D����!�mT���(�=�L�V��5#u���y����^�6@��nMC^�.���^����%�b7�����5�o��C�	�r��f�nM�H/v{����(���i�����z��������#^����Q��d���18����S��9�Jd�2�?%v[1s��������������G�Q�7iz�]��)�p�*qV��FY�"���3+�L����|�I�o��RA���>�����B{j��=����Z�A/u6Q�C��mR:�����S��&Bdl&`I��W��x��J�6�j��^m�jC,Pm���r����
e�w����C!>63��S��
^m��P�W�{��f�6x�!��6lCh�OR�v����G~���q���\^�4x���@�<�8�Q,��
�W L�y�MU����}�CS#t�H��e�7���D}{%�+����Dpz%�+^��JD>4S"6+��'���C6x^mp=^�jHo�]�i��/�2x�!�q�2���U��`��W��`��fh���W�������
�W8,�����L����\D��e���n��j
�
V�x�����!�Z)��O�?�f�6��'$�5cl7~�hu�a��@7����@�:t���0P���g��t"�����OY�(���BR���	�b�t]{w�U�T�������[f@��6�+{��~od�e@�HG�}����Ip`X2a�n�B�V��+XF�Wk���v��f`�������:�e`G�E�je�#	�N�\L[��b��v���p����*�����+�/:���@&"�
J��Nv��1A*W�b_ti���[f��
X@V,8�����UA�j��e)j�"c�d���e����|i����Hfk�	�b���D�ga��V�6V�j����Eu\`Y���z��@;%y��!�?:��d����&��d~ta��b*T����xhk���D�g�����L0����QD��w��57���w|\��Z�r�yP�e(�`������C>����a��v.z|�:�� �p��0[;�U��\�v�[8��m���,���.}	,;�%�f���/(�s���`����pt�������b����������J#;��`s1��^�)ni`���9p�sP[����!�s��0O�.cH!+��
8~�?[����K+�bt���<�T�jP���4�TC�yp����:����NlO�e�,St���L�e�ny��)�L�-��2E�)�L�+�
�Zo�����Y�b����N�,Re
/���{������5�����_���X�|���]�uP��ndw�Bo��6�o�����p��.��An!�+��J���dv;�-$wr[�]ySR|�\Z`%��	T�8��.�}%]�Is���u1�-��Z�}�(������}�(�������X�srw�L���)���n���|��Tz|6�=	��2^-�N�P�B�����3R]q������cz��j�]{t��>a��5R�c���U|Y�N6QC`,^!�����?~�a�����VE�MD�L��k�(�QU��cq&T��=��=4�n��#IT�)r�8:=\587J84��58.J8+�M|M�v���s��s�(�0�F�f�/o���&�i(WT8����q����"�������s�����>���������O���>�t��|���S!�r_�`�Q0�4��d�0$s��K����������"�$jGjF����E���b����O�A��9�u��]���l����0v�N����Y?5���c:Q�	;!���'��t�6�u�~�%���%���d6��2p
%KYD��` �"�G$����P2�
i�Z
4Ta�cg�WcN�����g�t��%M�y������S<����������?���,��k$wa��nY�������v+�7��k$���"�&�PC'���Y���N&V�g2��<�:���p��Uv�Q��cOm��\n����mY@g�%M+�X��;�4�{�,�$��LQ�_��5����+�_vj�5�hl�D^,d�n�����*��U��cTe����l�*�~���/���\��������u�C�^n&��.k&���gyP�p�#����0	�I�W��������J���!���
�|�-|&e����N�l,��m�`c4�C�W���%�H����er�x+��({�{��Y��_�j'������]����X�w�^��vjo���io�����j\D;�w$Q�'D�
��-��Ln�������&,)���LjB������*�!f��z��������6��+���$������c �O�L�06�S�E�L���f��������w�f��q%y�pp���'���`.��"����	�{�,���^��n��OQ#�,i$<���`��h<��M���U	�l���QM��2����6���+��\�G�"[�5k�Yp9���X'H��� ��@ArH��m 6
��@�H�1S��n���>=~����1����w\.�Az
w��,^��}zA�M�v'lC����O����������gX9,��"X_]M/�?P��}	����e���7I\\�m|��i�|��]�M���2�/���!�������K5�Q	����K�r-?@�[{���6)�
�2��YW�|���@��dBL���@]�G��1��
y	�k������q�U@��F�������$�(��o/�U����_A��y^��r�����W@1/�y9O�9�+�����T����&�e
k��t����/����-t]^N���g�BHze1`��
��~>=?�58~��-s���D��@=#�yZ���R�(-��wprv
M�?~r���{����r���N�?���X�t�}�#���'F���������*xn���$c������{��
��t2H�_h�0���.��/��9��^��^f|Hf$�
�/���B�QA�*���#/�63���w����)x����iD.*�2�H.�s�ru�d����2�_%#����W]����P�#\D���~p|���_k+x�����������cD��!���G�4�U�;��?E����u>�p��~���<��F��('	����_�_����_������__�w����{���X��C�!'�$�;o���4Y!���z����/��)z1�#�w������K ��pu�8�� ����F��F�{����%�l�������g��s�����h�=��Y�� �^��78i��q*��Nag-q���H��������{������]�����Iotw�&�,�exD1%��q�lPBR8���D�Z`
�-�`
+�X�����Sn�T�\�
L#L.�j�|��������<����\�{�Z4j�2t�k8@Y-���%(#(����=hB�r��@���0Hk|w���R�,!
�G��c�{�E��	T}�>~��E��������tp�������k�����[p�+���������#�&wWp3�or���8�}������S�W��d,G���U��z[�k����S�]�U{��^^�vv���'(2�����A���&eT��WV� ��Xh��3���Oh��=4
�����xq���@�H�p����b;��,��m ��PZ���;hv�"S	S<bS�Q<)*f�+Cyi�*����D��=b5�Q#�X��('��V��l%���F$�*������"[�2"�bD������-��#r�ED�'���[�Vw�&[��mmU�R���n��� p���Ub]j�H�<[%��I)�d�����k�BYR�N����I���%m��}��-3���h�����~��r��r5B/��e�b�q$����C����.����,�\��jkJL6�(�
���Z���O1�6��u�)A;4|�;>8MEY��.��5|8A�O�7�""<�r�����Rk!�gx�v�h�,u���D�"b��U�ZX�����6V�C8��KF�OF�F2��T5��C�2�85x@��|U���=�K��U�I�����5���z���T���[q##�^Z���QG��Re���[FT�i��hY�[
Z�Z�	-����Y�2��~���0�������!<E����<�
6��y�sI�'jk��L���"��\������sN��o��N_�68�M.ue�5.u.������&PS4�Su�emHQ�2�S\�P�"Z�"���Ja��U�
S1�!eh:{����%�$��p(��d[�j]	���aO���.g�*6��b����/�.}�N[N��7`����j�L�9>Sz���M\�x��7�x�.o���K�������[����o���o��f��7T���f��\�f-	��@i����d&��<����#\�1����!�g���"s18��h�������yc��� f��U"������?�UO&����cU�����	����b������X$�*Sb�w��<'������@�t�S���P���uv?����l��9w���L\�xX�-�&��U�eI��,In�%����;
y���-Q�"_k�L���m�=H�:���7����������k
_��+���I��:"�uD�:��Zk�e�e_g����A��i���m|j�����,g�u����l��b�_�R?g:�jm�6�Z�FmDu ��^�p-'e-yCZ�}B�\O��d���3�����)��v��8h�����H��`���xVK�;�-SI���[��
U����-GF��h-*�-D��������S��^�u���P]t������byuq�Q��.�X6�F>����:���U�v���W��k=F�Iu����S��N+����g]bf�2_������^oU^M[`��\
���r���r�E4H�������e�=c��M�Ut�+}����o=Zwu%���j���n�f�lJ7>�W��I�gI��Yq�uG2�'��9F5����D��s��0���
�v��[6�n7d	v��);��E�7��l��N��<�L�����e���-�0�L��mb05�/1�n�Z�A��U�	��.�0a�5����aBN��dBM�<q ��]N��uU�=l�����������#���{���jV�:��^��[o�x���j7~��V�u����v
�0�s�%���3dl��P�#�r�������P]3-���h����R���R]\-%�����E�R����Y���%�`��"��o�2�����R���K	�u	��B�������`�����
�b�{4l�}�����KS�AR�s�I��=5� qH�����b��<t;!��y�`f�e)�B�i����K8�}����V��T������1�NQl4�����&���<��������y-t������������K%���K��3�?�p��a�c�3s1���{��9q~�(�#'�NW@�:?-���s��!Q;��Mx�l��ZG�.�������:"�������O�j/�|!4����@�W��-���L�%	s*0�6�:A��BN�#��!��
R�a��C6���u��@��X+,��Pp�_��tA�D-�� ��j�'��r���XEp	��\��
�a)�:*�#��U���R�������p���%<Cv
|eR2�A��9��~zc,'�m.S����#P������dq3����t1O���r��HV��QK�C����?�����C�7[�����T(dkmS�W����������[�(cU(��qo��������Y9��Y��
�[b�OD��F}[��&�D�d��*�'��<������[""G+���En)�#"$G!M����$h�G��H�p�-�W�jB����M���	1�UBLDb��'�Dn	1Q4#��UW���&N)-Q8#�pF9N��bw	�.�5��q
��cP���
:9d��s7�y+.sWy-P:��]}���,�wu���������;r�Z�p�,#���
U�o��w�
���`�N�V,����J�Y;���P���wx#�X���>[��|��2���Rz�(��)U>��9{�����0H�[Uk�*YU����� �Voa�,�f)�[�Ck�R��{]�H�4����1��2��zR���YUO�h�����IEV����w�
�zi��53�7{�q^��h���lO����j�������8Jo6�������_��x@��0V
��y�3
�]��������9�3���;�[���#�=#��QfHU�Mo���;�e���.oI����z���S������>>�&�����>>���75f xSc���S����PzCc.���O����E���Sa��f��L�����YX��Zh���V��������cUe�}���U���>V����8{�7 �;�XUoB�&�<(�	9�V�����]�����#�cU��Wgh,�e�B��^��[ey{���d:[i��$+e
f6Do�fa��9w��` �Vbo%.i%n�H,�����K���V5oC�6dc��������9��p�sf����n��������h��_�J�j�D������5��7q���*8���f��fj5lb�n)�a#�z��m�s��f3;v��btn�d���-����o������T]�H]�{�������0*�2'�eHp4!�uXoZ��p����og4�i$uC���T��T��T�i��Q�����a.mr4��~�}J�{\=�M<��;.f���R�#Nu�M)�M��f8h����z����6��h�3�4�������Xb���U����%$}��Z�A�>h�
�v�n|�b,�>h�-6h��A�9-��E�h��-��E{�}��Z�)+�7��5�����-��\��& �����_>|�Z�}���j��>|�D�>|��s��s����4�������K>|�4���sA�������
�pt)�>�q���F�76�����5]>��3�`F���}0�f���>�1������3�`F�����������}0c�4����!�����lhy�@c&p�����5]������������������������Ne@�����������]���~;c ��~{�w������g�������yovTqJ����:�2M���"��2X}]��r5]M�V�x�����!�:��1��y�	t�9�/�3�n���Rk������l�Kg�f�����P�2����w�����KPH�G�qM��kO��Y��xM5{PVW�x�����W�����v[���H��K��F:��C��P �CQ2"��C(��`�4RK"��D��,"{:D"��L�B�O��}���yVU������_�8��U����C(�J�&R>2a���i��>����V)6Z�v��O$�O$��a����6>9���P�'��q�H�N44��;$��T����u
�?P��6�������6�F]W#�C��s����<�SM�_���U�����������x����-���7�{s�7��mzs�$�6�i��uF��
�wuY�J�c�<���ry��F�]��J2�X�GC�1$�0~�EdeLC��2�a1Vr��"��r-|H
��mC0p��0k��? ����5D��UWW��t�El�5�H0_&y�#����%<�
	t�%���Bi�����e}a{�x(��������I-�P��IIj��0��v?�����wh��!���t~X��ptd�����t~T��p��������������G/l;A:A:Q����vQ[hi��S
<�rU�l!�S�d���!	;��
�HA�UA��a�&��a�,)h��L4�'4,�'
W%J���J�B����-'�d2,	��RW'���
��)�W~��-0a�C���}a&"�CN��]���q+Qn��4�R	Lu����P�+J�Td~]������G�fP�"c�y���PPv���@	 ���b�|9Z��rD1K����i�/S+�<'�<'��%k@E��k��$ew_���w���}#��s���'p��f\�\u�bV4{�3�����xWZ���g�c�(�d<��d�I!��xYP6f�A�7�6g�Yp����bbu�Q4�.:���Ak��]n���J�t�����vCo�Amv17!�{[(��
 �f`@,��
 �&a[@��
 ��a@,��
 6���Tx�l}I�.&�
��dN��N��������f���fUZ�?�5���IZA��aZ�0��B���]��
��L��#�p$ 5����[A���[���"<�w��������cZ�,���v��)��S�����Ur\����������I��M��j�tR���g�E������i������{�(�5���U���y�K��W4�V�Z�W��Kz]<�C��?��u!B���/��|��C[����B�,E�V�cA�5;a�U��X$.F�NlnU@=~�QSdc���`|)��
����^k����h%�x�{�v�������"Jv�����x)�|8{
���wL����N����>��`��(�������"�
:���O����&��/_���,t�n �/	|RL�����K��CZ�S�!���e���g�ARk��
UoM	�z��V,�[_��Rek�kF{������=n��k���f���V����O���A/����,�)")M��0YR�Z�g0P��C��0��PUe��ZT�jce�e�������e��������������_O�N�/����9>;;��`i��9	<����w��]�&��f���Ds�{���>4�C�>4��F�*�����*���&�*�����*o�N������v�A�3�V��Hp���r���pA���/���ar�t}�������iU���:���?�0�������\1J*>+��p�������2��W�N�@��7�'��Y��/l�������U�{�;�r)�[����;��c�-��r@�;�t�5�������i/�Q{��^?�5�o���!��vx�	��x���6��+ ��$�h�����c��N�L�Pr���]�F��GF^o����O�<��By�ip���'��R`�y���!T��I��?�{��7�z����x�M.���{�ei}�:oa��8K�����o?���^�B��b��P����u�8V����U���,��+�+������d��v1��lO6	t�^�W��Trp����n��2!,�4�2��K#����������F��AV(ca�n�42h$���&3�DJm��2+��r�e�@���T�������Ph�j��1q���S!�����9���E0#�{s�Cp "X����XXk��&4�:�8U�w�5���E��w*�������u8ry��p���p^�s��p^��A��p^��C��p�&?{%���X��&�e
+?�t����/����-�{���'��>`�V��m)�KC9����'��o���p7F����_
��U��N�N������O��|x]^�:���������c^bZ�P���c��V���<rv�3!�����&l��K2L�+>����k�PQ�1I�N��Kx�R����5�������vv{��1{�q(��Q6L���2�K�����.���AS<s����7�pb�}������)���r:�W�]f/��-��U2��������)��/�����C��|�ZK����w���z{�R.]��g�d;RYF�^�Q(=�Z%������T��\�J�av��������d��x��Wq�&+d��Y�V�Y�%�=�M�Uv���z!A���AG��w���'D�@?�T���uo qK�"c�.w�_��fU�h�ZN�N���C���	��}B�E����B�B��M�s��uQp/�(�%rsw�o���8�`Q����2a�3��hj��Ay�=����v��5u/��\�+a�~���`xP���Fm�
�0O��+�����F�� ZA���-��0Ou���6�E�a�juah��i��6�6D7��x����`a�P���r����(���Z��3b�.e��}���-�e#H3(��VpkX�K�@Km�e�����E[j�/c3��6�	Zj�/cJ��vP����'nr�f�;@;����'lTb��$�
��/��a�����2%~�����<�3�_��;�/�;�����#����9��`e8���X���Lm*=<n�l�G�U5N��"���G�3���hg!�,;��"�Y��Hes�j+�#^m�3)�rO��E�.H����r����|V|-������`;�eG�M�%�"��	�,f#"����Z�G�r��t�xGlB3�#���wT���0�8���-EOD�1V_�pQ;�+p�kJA����"��+.4X�" v���������X(
 ��Ar*b�C(�p�
����z��)	����1c ��5	n�"Hg�-�7��{����4x��N�q3!
?�7n�ZH�Z�8TJ�z��+!����t�P���%�.�Gr��;��G���[���A
n���q,��r,��V��P����a�����!=X�vC+������G�����������"R�-�����j���#Ki���}y��m����`�!S�RP�WI0u�E���F]X�Uj�N
�F�B�|�)�T��%���� q��l��a��#��#�+�n�Lz��#3���,�����:�|Sfgnw�l���R�
_��z��~Q�v�p��OW�Q�s?]�G�����>2n���F��6��k�������&�H[�m
_�
���~���nV��?����Q����d����_(��
�����2P�s��=�i+�`@��\����/�
�m�2�(�x�P��?��`��A1��������s����i+ks@���"�!S��E+t�I�d�R���?��?�(s�����b��?R���������^k��
��|g�Q����[�7��X>����]���~O�1,r������P�w(�X�j������b����4$R����	o�~��:����,�"�6�U��K��E���7��F�������>��Ej�n���}p�n����F����@���l�:=X�y��;l�5v�@G����:�@G��_>��:Z����}��t���>��:�@���:�@GF;����:��@�`���xVs�y�US5��^
gEs��/�[ez���o�7W�����b���e;}����5��q���V���h���B:����vGI�����d��Y��y����������a��`�-g����V��V��V�I������M���ZNc�C��tDX�����*C�\l\��/L&�o����Dr����b~%w���Z����A��L��H��<�)��M�Ut�+}��-���j���~�9k�\��?�r�S��3������8gI������"h H���@��R�����G��)�������*=pt�6$����]6�������H���E&J�'3[��v	�h�>�����c`L.<#���4�c���^>���)9�AL
�Y�ZH��2,�iH�,R�WH���URT	��*�(�Ek_-+h�c�Y6�Q��hfj����z�B��3 eX4��mY��9sq#�}z�gQ9�Mb
X�1��Q�������hf�����:�V�1R�E3��-�(�xP,:�Y��a��X�1��Qk��Z����hf���Y�z@�3 eX4�u�������g�!e����C�E�q���E�2�
�����m���f��:B��h�0��������V'>�a�
s`&@��'�#�O)� B�A��?���o-����#��n��(�[�n\�/V-g�f.o`���m`=���X��r��+e����M�x��
�;�F�x��e�����]f`�-�28n�G�����kYL\��*+��n�t@�:;��AK�F#f�2��3me���6�)@9���{o��e�q�^�`����E�
�4�
�
�X}�&k\d\���aw��n���^
�JY������Qp*	����6�s�f���=�6�>�h*����/�s�����=���>�h<��%�<�;f
k�(����V5���)\�q���N�9�m~�iF�	��aa��-���M��V�K���Zn�r��+]��j�-�����{}��w�v�\~�[-�
��4K���N����`��Px�E����Y�B�,��(mg��P�H:�h;��Bu�Am@[����@%�;S��}�9m����ON�)����������4D�Q"����y����NS�:]m���H'F��������/�_����|���5��h��t	 T�cq�A�7���8��+��{���"������s�>������;��}>���[�%�B�I�tD^�w����Rr��-���f��r�3��r���u�y�����x������r�;�����������%�9���Xaf���S�2���l�V�+�*���B_U��0����zA���y��U��//�h��T�
�:u��
��U
+���:Q!S��q')�W�L���k`0�9+~��V��S�Z��0j�k,c����������:Y����x5]�Sxs�\�)7#��oOSv7,�q�l�(f_TB��ZE�C�����M�W��B�a�1M�1t8�X=�(��o��$K�yVR��B*�g?K���:�EjU���m�kZ���p��R�H��*����i<��gbgo.:�Z���P5N�.x�3q$e��c8��#*�
?"�z�Lx����.cD��r�e)�1���������p����8�������p���O���*5�a�%t_��-+��y�K���L��;��%,
a�^H�2�81�E�y�B������y���&u��Eo�/z��
�Eo /zUN���4��5X4C����h�VX�

�aoD���7 �+c��.zY�J�"���_��0h`����W"�!"v��0OT�a�lh���_xEX�Vn���m|K�l��s�Ck%p[(������Nh���=�vzZ3����[c�)zd��^t��������5e��j�Y����n����F'j ��
�Z;k�`l�0���2K.W��]��A��vkP��3�����1�;C�3�;C��������w�xg�f���!���!�b��w��`��;Cr�������)g�n�{��GD�1�E����
�E��vkX��c�����1�;F�c�;F��������w�x��f�1�#�1�#�1b��w��`��;Fr�������)����{��gD�1�F.���f<#rO�5(
]�1Rpy��w��	�#�1�#�xy��w�TE�;F�c�3����������;F�a0��#�Pz��w��@��cd�����/"��w��C���5VHk���l9a�f�7�ki���.@IY��uu��k��UP)Wf��@D����@�\��\��U"��(!$��hr���?��92Q�i�FT��T@�^�e���)�cFO��h|����
r��]5h���J�1pS��Y�B�������p�#�Z���i4B�w��+a�hX(=
A�;+���N4��"�A�;}��A�hX�M
C�w��+C����kuw��U�YJ�tn�~UW���G�nD�-���	��-�'S�OF�\�q�eQBVL{M���M�I)���pG����] ���]�qK���<st�i�ae2��}[g�>�M���fgjCFU�\g�Dg����T|�sv��=�9;�y������\��vF�M���jL������BM�\���
�����������U����������7�KT��w��
���nD��������]�����}�mm�6��������6d��F
mr5�n��7�o���wox��woxW���wox��w��.Q��M�7�{��
ox��wox��w ox�_��]6�m��z_��[�&��L��4���K��8o�o��&xo��&xo�W��&xo��&x�/Q���M�7�{�
o��&xo��&x o�o��
n�d�m��������
�9��P��/��[���������������hx3�7���������������������������m�����g,�
�<��-���N�R����^����������*��������n@��%*{����Fwot7���������������������������{S{K���NW���i�5������"�����j
����$H�7��y��	���~u�&]�*�Kc�b7z�Xu�a��f���������	{�_f�'/-B�U�>��O�����C���;�~3[�[gL��5N�M6�
��/5����%6C��",�S��/���DX�.��z���5�_96;����S��+���Q�4{�C	�P��lX�B_`*�So,�%���C7��5�>��}(�>�h��>����=�#���D����Q
���iI��$�Gn��rhO7��WzE���5���se��@5�k��fgv�'��	%|���Z|�,�v&b'|Bi|Bi|rF>am���H�	%|B	��	k��eS�O$�O$�O��i�'�m|rR#>��O(��>>�:>���b��g��lq���ag�`_rM�K���}�5h�/f�g,��b�����������!M^��f����"��nU�9h����
���/�n�����l+g�@a0�����7��7�(���m���\"��G���Ts�N��V*
�%��z�Y�3�����d����#����C<�c{��c��u�W��z��,�}WG��0�2_(l�=�J��
��A���r��u������C"4�g��L�j�m)[
����f��-r�dl��h�|Z���8�(���2�$�*S�i�U���%z��p}^������K��$G��/)S��r�����_��!�&��8��Y��wNC��>��dc��-EH��o����r��+�%'����8�c��"#T2FE�%�1�|�rY1�1��ry2��els��Q+�9�A�!�1�,�r�4�45
6��n/�Z�i��t�&�w=JE�oa��a!��t	,t���sT�#��A��,bG��?�� �� �� ��,b���^�A�A�A�EY�^4���{�d��zT���z�mB�r����W^�3J{�t���+mu�N������	u����:�N��:W'���^���:YK:W+%n�52�!���Y�T(���p�5�
���tT�Q���F��zNzNz^���)�}�l��������US���=�_v{t�����W|�Zw_��P���m��@�J�
:/�������p�w����z�_�����SwA������#W���s�9��3�w�q�|���g���(�)I0���=,���8	E4e��2��HY���1��D�r:����l�b����-n�(,:���#R�u�����V�BGAZ��D��h��\4l��4�y4�"z.6B{F1�$�>
Q�
��_�u K���P�����ZQ�K~�����,jwZa��e7}�M_v�������nZ�c���4������S���o}��sS�0���ZVP�[�1�Mq��"op���58gs���+�hTP���1�Mq��"op��jp�8��)��W�
�������=<�05T2��]&i:]�SD��[�Jn���|M����l�{,n�e�Z,��j�C��}4NE�@�jg�S�`L����X��Zm�MXy�n���+j���Y��9L�*R���/��@+��D�X���z_u}�QG��Y9jS�u G�������T�+�u��D��0�j���gI9�}�K�<�:����7�b�N�H�

S�5����Q�B�4�� `qrT��2
Q���U*�L���,N�*��A`��@g�3v�Kbj��,c��g�R�z"+&��C��C�	�����D?<�x�?����}�����<"���s�|�|���D?���/��0������|��=���~8�8����0�~@_�'��9��s�	�����	���'���}��A�G�����������=go��9��2��^dR�t����\4#C=���>���������O�l�����~����w+y{���T�Y����k���t���Du�V�?���M����WWy����l9����&�E�9�Z�Lgr)�!�t��6nvYm]T���#�����i�E^�4�i������_W���v����<���%&�S��T\��T���T�ka1������L5�E���H{�J�c#�gY��}����# "b�	�r�G8��Z�?�6�6F5?��#�����2�������$3�g��~==;����������?����,�0�iN(�Z�x�����V83�}]�-3d&�P.����}��/>�e'b_,M
��w��M����F�Zjz��,u��������l�g���@}�=�M�{���A��k�����0��S/����-��wcf��i�����n;�=[C����?a6���E��G#�K�y�6r�����~��y|u�L��U�S:Q��M�	�j����#�-J&�$����|��C�q>AA��!�8B>�1�`���C�!�`�Z���k����M	��w;��f@����
�-l7�����v����t���d��<;�x|vL�L�P��d)L�."�eB^����]��e��;���Lyf����OO>�#t�L�.�����?�D4�mo]eb�y-7&l�<��E����S�.Df�p���P�!S�m@��y*�=��x7{�
u2�����B]L����h�!�BEq}���q���5<X�[u)��"<�J
�5�v�o]
���N���ktM��Sfv��Y{�����UA9�t%%�h+��.%TP<']I�ZQ6��L�b���+���.)�G�������v�e���{}z�����c��KQ��\,��*�Y�N�
u��/l�7�����,��+P����6��m��/@�����g���v����������!����OZC����5������?�Z�e�l���E��Z�I�)��HD�B�R��b�&U�:��N�R�Ag{��TMc�D":�
����yR���3�)V�4�l/�����N�b�A��[��UT3eu�P��{�Ey���u�\P��"&.��J��^K���j*CP;kV�`;N��*-C0�Uv5�����2"�����v��AUe��PD�V�� ����ue(��.���rMD���N��.�E����"^=$��������y����W�y��W�y��W�y��W�y�J���A��W�y%Ao^�G�����:cb��&�e
U����7�_�@���0��r�|>�C�V9�����#.?�����}��T��=EZ��=}���@cz������)!������7��_F����~pvz����'���xH`&�w����%���<rv�3�^��M����d4�W>|<}����#J��
:N�/4����}�=~����_N��W������a��o�w	I�"p|��J\Eyl��N>�{��N��5l$��:#x�\N�`	�W��n��x~��`J�t���T���eF�W�������;�s�������w�>�����%_}���������Z��R�ya�SDM��d*r.G�.�[�_#�K�_�_�����__���4"S����v_���4]'i��'���k`\��Pl ��3����3X[�������������� ��q���M!��y����?�� �Bd�p,�U:���?���w��Z������?K��,�4:EcN��V������&aB�e<�N>��q�������	�?�7���]�~���
�L�}�]E�(�K���bf9��E�R^�?������3���S���&�$���Q���}b6�=�'�f`��L��=2#�XU��:���i�BY���CO$U9(j?���6������C)�P������9�}���\��/�����_0'���l�s�<�R����[Sh��t%��T2��q���}�������e�#�8���X�ZX���c4���UY�2[r]IFay���
����TW�q-�e����|Xg�J�[��S+��gqKs������S/����[{��=����=����z�����x�FO=b�pK6M���}8by`'��H��C���!�1K.W�~�����x��Q����bv5����baMq��v0�?��"�1|aM�R4
�hI|iSN�{S�W������X�T�C.�Q�/��Z�|���4#6�_��0�����*�
"hU�G��n���D�����lX���(��Rj~�n�"h��E�%9�{����'Y���s�����=5)U	G5�b9����z!��/�6��E�h��I��a��6����U��@h3������������Vx��}���,G���!�����������s��P|O�s(��B�7�J�8B�w��Q�����J$��H��@�-����
���
���3��c+�.pA$�g0�(����%���8����G|C���h�'������C_�M���d�H�"_��z�"d������	|6��h&Ppp(@v$���@][al�����a�
X����_$��=c;fU(��������)�	3g(���@��'r�0F=a��	q%g����C�walEZ��,����B��G��q�zh�
�~��0�U�'��]"�'��@X������c$��'T�E������{"G�
�5Y�M�;"a����MqOq�|.������7A��z���\\Q��/����{ �80��������[�__�W��E�
g�8�y�{����'P0v��$��+���������^� pH_�WDzF"G���0�����8;��H�Vq7�ba�v�@�:���S�Z�l �R�8\7p
|=��<(��C�f��2�����+�����J������2��#8��[O��C�uB���@��(����0Fa(�����
�+��������J�(����B�+����sq�V?����.�q"d�$ JH��#��(p��G��/���y+�E�X��7Q��O���0%���J$��H��D8�Y%��*%�R�Z'J�"]D�S\����8���M��EmW�!�N�E��:P��/NE<I��1o	���"/	���`����q�W�������f��b�RCT
���t[�,P�{�!�z5����h��
��A��d4d��F.y�����6�@eSi���K!*��- ����JIo3��dB���$����	{��?7����Q��;�*;(:��5eI�l��>�(}�ME�d����Y�s�*�������:��q��'���O�QI�{T9�>��"}2�S���J���G
��U?��=jj�>��9S����"�����'H�O&lI�{�)}�Q]���J����U�s����=j��>���R�d����G��'2�
�v.#/���F^��0�1�B5Q%��8�5i�\�������^d�Z�)��D9��'���Gv9�I���g��pm68	�t�3w������,����\/��Qm��fdN2���,O9�T) �XS�6f�0��P^����Tlc�
�,
�U�1Q"���6�o#��M|�^fA�b��������7�����=�����=�
���j#�����"��k���@y�G�q�@�c����; 'V28���!�b��T����;86�y��c8����nti�f���
��i��Dl&�����.�-����V��C55�BiM�@�
m$ %9��'�*�����7SYf(���^�/
�]ns��pg�&A��nN_�b�Z ����a��G���M��_$w�����~����)�{���a�8�D��3����M$�q��	�"R�P�K�)����b����p���;,#������I'z�^��g��s�:�su��w��hT��C=w�+�T��R��s�9!��3��J�� ���R���B��9����(z��`�<��0)XV����k�'�����>���V�I`��]��U���Oq�.��Rb1J=��a�����pl��R�������J�=4e-�����d��eA4$>@�^4C���T[4^O-E�EN����-7QC;\�����.9pN-�@��1;�#���b;p�n���\�9�)�
;�I��Z�P(��=��)��C�2<"8�T7���~����m��P����������`^2��/U.���R�_�:6��"��T�!-|��0���&�������\!�t�=�~{0K)���1�
|��r�gZ�����W��c�;��=C�D_2s�)��n�����uY31�#��lb�2�}�����i2g���<�w(|��+�/��+V� ��H4���
�����0f������$������X�cS%F,<�Ve<j�P�,RP&�P���"?�Ry���0d�S���:;�^*[�Dv�Z�&�U�^*]�Dv�Z�&;p�^*Y\E��Z�&{�m^����.ZpAa!�`�I
�/	*z/��,6��b���(���Hm�j^�e�����8~����������G�A��<�:���_,��/�2K�t�*��]�,q��|�#���~���3���H�g�'m�����@S���YE���I�]���
�Z�Ou��:�0?.�a��yS�c���9��jk�U-��>s������_Mud
�)6�:s-V�lD����S��������1��� &��=�:,�nY^�v�>0�������Lv����/�Pf�48�y���j/P���mY�|':�1A�\#h]L//�5'z���ip
�
����wfp��W���*�
-�O����J$c���7Vr�J7PR;{�����Ho�Dr�26\`��>��^�����e;	������Np��)����0��������U�/��eS�Zk}�/%�G�����`e�|��J)�1���s��N3�sV��2����
��$-O,�������Td�u���?5��H�z^��������E�+��Q�I��|@����T6��[WI�,���y�d\��r9�����v^c����rY(r�Z�<�P��*��MU����e"!���b.+'%�A����3;��9����Q���
s�I\�|%�{���=cQ�����)�h��9���!�����;����3��t������f�E\Y��h��x|hm���>O�����r���I v���{u�-��!.���������=������EFG|Y�_b{�K	�j�YOW��,��0!�x������X�t�i�C1�r%tR�?�$3����Y�G��|O8{���t0�@����/��h��&�60��	F4Kw(��a��x4�y����;s;1�-j|���,8���������W������-s�
nY0|/�=���	����Gx^���:��S`>�9P�������Jm�/��M��I|g?�~���W����OO>�# O7�����f
�{�H�;n��m�SYD#;���l���d��I��t�Z��4��a�Lk�yF�4,�ge�ey��F��X�~aS��7�RA^�m�Xl�	��Ll�Q���[n5k��)�}����9sy��
�4G��oG{���}�C�'jg�O����pr����sr�&x��j�
�����{��2�Z&i�=�����(�\'�Y�-��������0>{v4��!���=0JV}������W�����G��n�W�x����{��q:YNo�A�����	|;�/V�`_��,.�
j)�^x������4���`�J����e2���~O����q�n��$����j=	���M|�������b2�{����5�{1��R��E����������������g#�n�_��b�m4[L~~K��c�>G��=��.!��E��������d�)�v5�-��Ts������
��;[�������������5�����<�\N�,�Y\$t:������g���0IB4���H�|?_�f�����V�nL�n�Y���,�x��S���8H��:^a�p�'�r���g4�:]���NP;��@	�����,�f��3j����������0�w`���W��.^^��Y�{}����O������O�"P0���3�GP�C��C�����i�@���$�X<3�kJqs������7m�x`�-�`r�HAK_�����4��@B��{S0��.����*�7f+��b6X-���d��g�E0��u���N��d��[��a�n0�#��F��@NZ"p�AE�<H�;�_`��i^-�d�E�
-L�i���o�����S�{����+���|�*/�3	�@�����@%����/���'JP����3n�����o?�~"?���?���[�lZ�H;�=S;�M;�d��v���tzv�y��7��a����#H���������1	V�J����(M��$�������.�+MA���������!���)wB�N����?��_E�y�@���r%�'��Z�g�Yr�����&x]"?��B&�g����6�o2�C7&��-�1Q��D�n�5����iGeFM;23JLVe�3Q�������������!fS��B-�d��,E5E:�,s�W���������u���Q��b��������Z�(��^�j��������-����6�j��no�kM�cc�1�K�i�;4f'�'�2�����o��!��:�)"��P��;/�
����X^v������	#]q4���=��%�E
��t=�c�,��
�7PI�p��N��p���l#�S�D2Y� q����^�����������������Gr#D7*2�CDr�a�.����s�4�eX���K�*��6�*�4��y|J2����7g9G�Fh5�]��(���aR�������Y��
k;�g��5
���T	���� ^�������
�U�p���f�M�c���u,�������&�/��n�����fr^�Z��GR������!��V@^$�g����a-�t_��_�A���b��P��e<�#�%�6�����i���h,�5"�0Z7�����c����O;Y�%SPDfc5�C�7gv�S�`���>!�FG�n��#md���"�6jKm�k���
��?Rb�(74��&^"�^���6f!��1pf5���X-��Z��j����-GRhYM�VQ4EuV�VeV5�jX-�"��V	��S6&aF=�O�m�������L�m�c��n�:GN
��3�v��h�	���uc19��������x�4�D�:�
��
�]F�(�
�
��
�6svL������!H����*�S��h �� �r�0	/�a���J�����>~l_z�9T��z!������
<�a�zQ�:��1��s,j�����7���a�+����y�a��}L��s�^db���������0JD(��wsl�)U�R%"��F��Pk-�k����-�O�o��e����v����
�/��8�G����������*1��@����Bh��`T^�61�
&�L��Q���"v�1��������|����sL���M��H2�9�|/�!x�7�?�<��b���]W����?���D��eS�?�=����>���L
�=^\^�~d�l��K������.�g���.��B����\�?��(�_���s!�\h~.�������Q<���g9��Y��B�����H~+���TF-�j
E��������vqk�)`[�{G��@���NhK�?RXe����6�@0����F�6zw�{�HK�Q]�`�,)����I�~�-F��d'�O���-�O��,��	��N5����}jj��m&'1��e��SSz��c��nP��x��P����u�,s�3o����Y;4��so�9��VY;4����o��^�QnhL�7����o��7��EGN����TA-'��91�s��E{N���m������Kn�Qnh� *'�[�����!��zR��v��}p��p�1���@��n8q�8��|����'H��z�/v7jD9�8T����;&��D��9��'���L��$@x�����������}���3Q7�����(�qA"=F�
�y�Ib�����}H����������oH��0�+o��vI;G��so���������P
()�������0S�f[;��@�eg�>�)���Ns�l�2�i��0���u��?�:q��<�����J�(9�e����h���Nw$��	�m���XX�g���cb��j9�������c�N<�W����q�.���*��]KM�c�n�\|�.����VN1������~n���P��"S��C��
�@YF��>�O�M�IH�3A�b����t���8f(��9[��G�O�_�>|�H��#^�~y����X3,���f�h�9S�x3���:L�Y���#�~�d�x��CAt�b�vGu�Y\w��I�
%���z��n'��pw�����y�d�y��X��i���E�a���;��
s�@u���o�
��e��38�{�(QY�k^��=>+HW.^b6W����!-�N�\s�\�L��@��fZ55~&�<O���	���l"�vs�Z�-ke������5g]#<M���9j��"�P|���m���
N��C����*^M�`��?�I�u��vi��%1Y�C�q�v�������v�m��
�
������D�#�N:�Mt�
�����q_*��~Y�j��-bP�IF���U�j��c����w�{�P��#-��U�-�VH��^Ou��7���
LA�s��&�t�d��E
�|��}x���)��g�Gc_��Mm_f'i�c�Y,jW����4XTy�"��E�6��H?@�����<�������ZC��4�/�1��<��f�#:���H�K ��:J�0�-m$D9��<���m)��r����W�V��0�JK�a0�`4�9�� L>����i�k0�}�X�$�oog��"�b��=,��U����d���9H&�8�����"�G�R�B/���/���5V���b~1%h�����k���r�'���i�Jl�+zi�b���JU�@�RG�x�.`T!�����E��S
��"���/<�FE��z�^=]\>],/��Hw���i,�Sen�B����k>���E���b9� �q ���`z0Zz��4KU�y���RD1���d���������4�A�x���9����+.@)���F�~����
3W�����K2C��mc���1UQ�����������^���$j��50������}r�3i��]-��v=UO���K���D��$����.�$�r:��cd$���a�	�m t�����3M(�!�m��J�BBr��p;Y}�������^��|RJ��G��|���C�R��#4.a��#4.�vm��P�|hIR�|h����|���G8�k�@]����SI+/����rCc<R��P�|���#�,�~���|���#$�hh:��e�J9��Lh���C�������k�u0�l��rU������8@�6B�q���Z��X�l�^]70�r��5�������J�#z	O��i@ui�3>-}fj�����g��A����<�"kGL�*�Zqj��[�,+��+�MI*���/������������~(���(Tk�[Q�#!�c�f����Z���%���IS1Id���5=
���Y��[��������R�~���Dy��Uk/�v�����1X9ng�	
\�O?��:��
����'h@E.�/�RuVv���:*;�����]�h+;�v+JRgeMu��C���j*;�[��*;���C=�m��'AL��������6��B�n������3����W�*UH����{����������hS�JcSg�K�(�P[��n�v�Zs��%���b���P�u8k�����XS%�
q�T&�PS(���������VW��������Dl2��:F[���>E8j3�z
5���B���
��I��O��q��,\�����.8 ��$�k�=_�f����M��<H��?���5`+�M�{~����$M��E�Z�� Na�d�\X�r���e1A���*U�����K�4w
��-������u�u%_zl��-Q1C���rF��vr����������
��u������u������������o�?v�\�����r��A�
���*?�����o����s�E���_���4���g�\4�V`Wi�0m��
5�KuDt�4/�Z�.�h��HzZX�BW
�b�:�����Y���xb�8����I�V�mg�X����i�`2��,�dI��9^'�dE���V(M��sMn ��Ab��/�R�ZG�n�c;![s��7	[��-�	Y�U�kwc�����a��$�c(��m9�k-�����\��5��5�]C`	���r����..�0�ck����
O�����~���q���t���j�8���wV��,�����OV�je�����D�K4��\&]4 (I�o<�OX~�9��6{��
�Qw�v_���>����`p`�<���$��'����<yf$�&��F�����L���e~5�\&�F9��������,-Eb2r�3)}�<��5�'��������g?�3��N����R��{EHFK<���
�E�@�7�A1����Mk����V��n>^�mT]�Z�������7�����E�3��K�L��]�N�9�}�����S��m4A������\�?�����"�s!�oh�7��
���|���_|J;!�j������B��"��H}O<!^L+�k��W������j���,7R��{����

��8�F<z�!X\^�~�>�-�����M��5���M<�?is�H�O�,����;����INi������'/8y�����'������mN����`z/���4�O������\����?��|��f9��.e9�l�����i�R�O��}���d����x5��I����O��4��N���;�K�a���%�y����l����K��5)������/a�kV=4ZY)�u'��������J��q:�:X m����p��e;�L{�6�/�Y��c�ak
�d8��,���{"D�+sZ��~jT	��HJ���S�=��b���z�}x���`@�
�!��� ������49��6%�)Z�H)%@���R[�j#�����d2�HIB;����0����`Lz�;s��4������U24|R ���\�b�Z5�����{��e=gZ��F�
N���D�$J3���#��i�W��e����!����]C"��*��G�&'��b����D5�����Vub'3r������?Y�������\Od���ue��([s�lT�+���6���u�������[����0������I����mMr(pkm�.[�z���m��7v-�y�9�m�@�������0=�"^�A�����7��j�t�|If�l
~e&~��z�3�[9��K��G-�N���N���N��m��b�v��h����7�E��ls���o����9���F��H�0���
T���r8zO=9�@����uT������q�FE� ��h ������!!����?/u�y��{�Unh������V����-�5	0�t��-D��-���Ew;&��a�j�_W��������%������[�_WC��j�J�jh\W���jh\W�-����~]5P��u54��a�u54���s����u5R��P^WC�����P����u5����~]
5�j�_W�����a]
��ZO�hb$�U3�qiKj��F���
�&J2|��A�N����U"���5���o�!���m@�V�������G`#���o�
��gA�3��]A��j���B���'�o6�Bqc��.� �V�V�L��6n��)�2#L�Z��w��i������<I��q��J�+�SI3�
	YyV%M�O�D�A�pI�y��\�$?��ZrI���FFn-
��!��_��FH��=�������ne���A4g58�"6I��Q�3�z3���D�f���]���Z �z���H�n���'��V������^~��Kp����g���[�7n`tJ�m)�68�r�^�A�����[0���[J���'�^�To>g1���X�N���0'�i����W59X�����99xs}��l
��2��M���X�^k+`m|O,���\lL{�V����l<dk��n��5���Fn+���� V�m�-�R�V��U[/\|q�A#�0�O�����g���r:K�I�i��;��-���-IyY��
:w}��w�����%C���	.u	5�[PJ6nV�[B���VB�V�!l��	�?�%��Q@7�~�a7����������;�?1d��	�o�������Cx5O7�Fx57O�����@x-�C��Z����(�d�����|�}���ou�6�B�T�p��t6]}&@cZ-c�A�{I�c�Y��D~3L�Y�N
A��Yr����c A?p:�
���K�������]�{w����YpA|'`������N�]�����4m�:��0��N��I�����'Ka��co������pc���l��}������Oo��s�:�	)�=��-9O+`�&��PXpX7�D#�c�m{ip�s�N�XbdX��@���
|�������"Y����npC��71�<��F��p_��5���ST��5
X�4 �K
��a�!��x}����������e`|�-�%����md�E�j��������5�����I���r!�@��a e���A�����t������[�w�54Fo��y7�?���OvEi���cw��NX#��v�p������WZ��i�"=���5<R���	B�����QR#]�:���d��x:C�V��y;������!�"a��Mf����",O����d��h��c�.�g��.�y	9h��^X��tO���Wt��MM�f`�;n/�]�h��W���	'���%'���3�T����#��X�s���47���q�^���x5�����f�65�k<_M+�W�����I��)PE)�}���S�0	"����l:�����@ij������)^����?���Et�����OD^�E{���.��Z�|��zf)��$�I�+��gd.�c�PVK,��M��0�Y��:s���3Z��}�]
�'����~cd�M�\/?q���u��)F�S�M�1�;F`�����	�u,[&&����l�}�(�� �w��^]�K�Y���dc6=�z���n�^���V�����=��b����*1 �;���q�6�����v��JZ���g�����\��B���^q�
�34m�?"^P��P��^���7@�Tp<
�����*�|[��e�|�u������9/��C����t�{���]�Rv��|N�Rf��v}��G%��"M�`�\
#��a=
�`
F�u�����6uZ�F!q��
di��F���'n���"IG�Id	Tkl@��h_�T���� f����\D`O���pKT��C����Q�@+��A1�Fy�~�������))b&������r���4��SM�<E�MUH�$Q��f�������	���c�d����,�,V�����2bkK_i���B����k�
���m����7�[m�A{�����@7��~	G���L���f�4�9p2oQ���@�r��Y����h��s��!8�'�;+��Tdw���e�C+q��!(9����4�u��C�QE���f<���=��Y����Rc�pVi��LM��
���&)Kg=��h$���KdU�V�h��p���0���a���[����<�n�����,7���a�)������Z~3��1:�a�i�
3OI���������-3~�����P6�Z6s�m�i��8��M�fKn/������6���Kr[���(���6�k
C�mV�����4n�ljq�v��`�k
�
��n���Z'q`���f��-����l�q	�
���
1�
�6>�*|
����Y�*��PHd}G]��_\�\�!���L1�����8�R��}
!��pl�W�n��L78��=+�s��l��:��, ����>^��������8rZ�%s�L�	�6HQ�?��O��~KW�
X������M��-L�],�CO��AK�r��������L��&��-���\����4��x��k��1)�����KA'+�{}��d_��bcm�����SSEB7,��e�x���q���d\�Lr������w�������4TL�������*�UZT
#���
�b#���r�����W��/��z�O��f�o���8���	�.~BSc`���hP(��&�`�9-�P$V�
g�|<;�xr�[U�<��d#���y��V�������g����OO>�#�BT/R�������'����&�)�����$������P�O�:F���]3���l|���o/`f�������67����#4��`��z�B
���_��G��-\m./'����O�E���ZJF����������m�+\M�����z+o����'��&�ety�"���������/1Vp���;���A@.���<rv�3A��W�cp6��%!H���������$
�#t�n�_���B�}�=~����_����W����H�}Tjl_�O>�{������������2�H.���NNI�G���o#�>�� ����0:T�$Bo�3����B�ZL�7��}F���	jb�l���rD��FT4g��R����9�~�&���	�35������Y��b�+�r3�r�x{�zr�6����������`
��*���^�1�(��5��kwK���!��(��3��BZ���Sn�������8l3�?�Z��D2��[�n�f����l� �����4�#-d��g��r(������Y�M�z��V��j�
rd*���NL�.�U��i)M*j�\��c�R��d�2�,���`�5&�����/nP	���
���/��n
��G9��A��.W����8�o/$��D�~�L^
�d��Av�z��0������fH���5N ;O��tr&J	Ix�P�+��X�.� ������	��
e����\� ��7l�
Cq���1���rW���^��GPci)�m�
3\}F��\WK"X�`��/-�����a����U�����)M����|�
H7�v%�j�����%�K�6���D;&l'�
���QvS��r�'Ju���L�����ew�������f��m�-`���m����Re�
������V���kum�FF���j��n�V�*_�`#L�Dq
m�N{zAn�7��a�
����>�d����	fj�|F;�e�ew�=|�&H6W�C�l+(�b�U�C�C2n��.������0/�����F����ii��>r[�m{5J&$*���5'h"_
�n�//���-T[(�\�$�n�&�/q�`�*�U/{�	�T
��0V�z�����y|j,��pV�`��X������T�5S�F��dp)�QTCe��5QS%��6�T�+MNI�T�9�X�1���+�(`;�6�X�Ae�����x�Z6M{M������_S�xC���)��8�|5���`&H��V�ivIq,c*s������K2{�^5���rg��/�G�,���F�{�����dec�2����Y������������L�b��k1�G�Z�����`�NK�0������
�V�a�����I������J����7��J-����F�_�h��2��J;�9@�����J�m���������h����!{%g,�zp�1�`��������f��?��f�>H��Y�GC]��w�W{]+7OO�j��42�X�G/pZ��km�)�Z*	�R��@
�V���VC����fv�K��{~I�-b]���a�s*
����
pi^I�msj��L�W\:���q� y��67��.v
o��5qc�k�����K@�����W�H.vd�u���KN���S>�5�;���z�m��|��Dm��j��!����y�i���������t;t�����GmS����}�4P���]t&�V-��-��B����Ry1;n����D�c��0��p���x�
�pK�H����S�Stj���@�����q[a�-�]�7��N������vK��F��Mp�v����-���O������&
n/8�����VkV������Dn����x���'�C��|��v���������B[��BI��9�7P�����-����?e@v���F5��}T�A��������2-�Z:�;��Fk��p|K����o����nAX��z��*�Z&�K���8���19o��������j��;��T�v��2�w��B���{��Z�����z���u7��������E���\w)(+&��l���[���#C�:��F����ZixW�
o�b�;�����Ti�Vc�9n��jj"�
���k��oMv�`y
4�����s�z�11#�s��0W)�lYz6MW�
�����sF?���gy�g�
����l@�
�3�����l�����t?�?�;!�S�4��>�NP�yq�@��l���N6^k!^���jF�r��!�gk�	1`D�H�������uV�����UD�m����y�����@�V�'\��P�k�1R��$M�~KF����o�
w� )�xM������!�������w�j.b�m�"P��T����!���+���|�e������W�d(��h�XUy��U~��V�_�
�a�s�h��{��x8�2x+hP�F8���Z��I\��D��n�C�����q�����v����������<WW�G����\o�����(���"u8b�U(�i���Y.c������p�b^&����#��CmU��N�S��5�5��������F����q��.�34{�XoaRm��R��������|;C���&��9u��:
��q��%�34{�Xoajm��T}f�[~�aF��lF��t���!e�k�&M(���Bu��:�,p���e�iy��mk�-.�������������?+K;��J/,MT��~����������t/�8,8��3�\���OI����--M�
`HK���������E8a�/&�;����P*C[-�+{��X��O����/�'N�;�@O����=C��C3D��L'�����������WU�u��Pa��.�������Gxn�@�	���p[:<j'���C����p��b��V����
[����=����[�n�����KS��� ��b�<S������X�sZ�B�mM������E��E����C%��M6Q8Ov�<��h�!mnqU��i�[�Sl���,fsr���RY%�I�nis�]�.?h����<h���
B������l���?v�����M
��h1T9����W�� �O�i�N� �O��
��"�X/�^A��g��O�g�wrv��������]rQ �3Nx�
3�7�������d8���n9S���c�q6���>~�|�U_A7��	�y�2#������/=��!7��,�F��P�}� �N�����*;��n��#��	w@f@)����+��3��}�`�5p�h��K�xQ�;�H��i�Cz#���,&�G06"kl��<�1r��U��3O?�f
����f{+�=��1#\|G����;���N(�}���lC|���;�(��&b3B(�������~���^��G��i1�P����N��H�}����a$���V����wB���K0�S �L�"�d��x�$@���@�����t�z��E�����xMx&������G�A�.#H<����6"_"�e����	�q��x�������)��DV�4}z�.;��4���=D�mhF�bzy	8rNx���"
�������;���6�W���*��6�>�X�����	�?N�������_~=��
����������i�w����'2l��!C�����O� ����rw4�CC$�}q6��S��(�/�T8m\��2��>ES����#H��s�1S�[�����Up��w������g�������n�F�#8�F@��\'���^���G&��gr�
$H�L��������o�h����_z=��~�?���~x�O�0��
z��,^�t/����6�^-R#jE����3\P�Q8<
��~�_������y�O���`09N&���X_?%�A�(��~@�!�=���1�Q�S�$�+���1���h��E"���dq����x��[��f�w` B�A������ z����i=�
l?�������@`P,1c���[\,����_��,���)T"��q����2M9i���*��,���t�<b_?��@��AK���
�7nr��0��/���jq��f��;���2����'�%�5�4�l�-��������N���&�/��BF���5A�2��F�YB
&��� }�!O��O�(?v�'yJW�;Jx�h����.&	��4�O/�=�~�
�i,�GM}�I
S������1Ap���?w��	��t�������0E���E��D4������4R�ac��
�����n���9�)~z>�T��G�
��r,�����{k�|��M���M��B|��'@�@�$���xr�>��,
`���m�"��t�H<#�H�)L�=.�y
A�G�6V���Oa�����	7"��s���e��D����K���=nTJ]��_����3:"L�A������B�X�Z�f�+n	��e��"�e�`��x�����;�������2�#��8x��k� ����X!�@�	+�%����2�x�R���O�������'v����k�r�������E�[8F��������0�	�)j
&�~���LLqN����+�%�����-pif�Da�b�+����.k.c�	 �=��(�`b����4��&�
��D���\�<c���t����6�F(�]w����f�k,���0<����,�+��Y|9��O�xz�V���t�]������*�7������������`�����u3�0*f��3�=`�������a��������
�����{��pd���^������43�.�U�� |A_3_�=�3��E�=�Q��j��o���E���@zv�~����9Z�7������&��{>���)8���k���pAK�h���w�ApT_������.O�^�a����q��`�/�� �HzQ��Yr%��K q�8��������W�������ox�{�K],���c�	�����`Lh�����Z�����S ���}�+�L/^����.��Sq"�%���8^��`m�%?���/�)Zu�w���&x����w������v������
���
���	��"���8�;�,��7`u�,�U7��rk7�0^g����
7��X���M>�=�H4��"�Hp_ ))&�g�Z�[�X�,^M��]9����H���������!nL�$��<�����N�N��%^��f�������%�����O��B��	��&�'H>�����j���f��7�v} \C���E��?!�A]	���7�]���[�,����j�B�`L������,��<1� �@A�'����m��������g��:����2a��[��v��u����d8x�l��"������]Kx�{�d�al3����b{��O�_��Q���l}Vn��W	t'�W��2�������W�@t��'���"Y1��5���k��h��61��
�~���{O���
���U!����w�=X��'�G4��#�9�������as��v�:������9��u����?�FP�
��_��M:�"�,�������}��,�|�L/�����WH>����L��o��i�������� QX�AN�
�����"���9
�)��Y��H��4aR�z����~B���	O��������} �DG�nxwyUl��
����=�o����m$���_���� )@f�'j�r�a ~8�5{�)|���G�U(����7:����I�$��=y�k���5@���!�?��%������x�����7�R���z�����@V�N�M���1��v�:zL��+ ^J�f
��g���P�\�q+���mJ����=Co����=;����.��U��#������$�gL����2���x�d�v�5��0����v�B��$��L�.���@[*Q�qc�|]�g��+��D�	����p�l��:���Q
t���Y��4���C�L%������g2[ S�
O88��6����r�NG��_��2���1��(tXY�����g���v4�M���Z����f��vO��]&��H��������&��2D�Vp.�b�$�%�|�>I�"#�l���j@��
.��bd����,��h���f��	�p=��Zm�K<�����m��g�\��J�[R������+�i<5	D����a���������D�3��R6H�vlLC�5�d�i���#�A�������0@���L���G4O��O�(���2&v��H�������X��1L�!M����s�`��=Z/e��9�����[����f���I���):��r-s�����RHg"���%r ���#��y��o�&� ��B�-��=�����]�LV������
_�Qa�C�&@A$.����t)�^���pebZ���s���8��=��Q��bd�dF<*��3�'��0K$���o�`����B��#�ht ,��	+5I+%�~N�nB�
��3N���4�� Yu����iN�9ph#S�����`����0������O9�ZR�N��ljwz��c�tX�:�m_������W��k��k��������k��k����<S������)Du��y9[�+o��.�-���m�x��I�U-,p�X\�|���L� �o`��q�%��)�8���=��#�(�+X�Oe7���)=G������at������A]��O������������w���t/��Y��f�N�������_����?^�')��@����	/g��"��w0��?f���`30D� |��V`�++$3��/�����'�g�L	O�W)�)�~?K�
�Az�z���h_M��%%n@v�h��Qp����
nj&�����O|���9F
�p�Q�t�:�x���K��Di}Ol�'���G�2��_!�6jMpA��_q���#S>Y�q�x��W�-����8��,\�����<:0�'��O��h`Ah������+���^�Pto6Kf��LL	.�R
��0���Ks�3�\�}`�6K���A�J������5,�O�S@L�o�u3�`F�;���
��L�����=Q;��o��G'>�����q�.t���7v�<����}�����������;���a�}��M:�����`m�h�^A��D7
�/���T��/T�
\��x@vb6eH�)�O?�� ?��U�P=����������p����������~@H�!�����Dl���_��Np,S:1�6�L/���~��)9C{_a���������\G�5��[���c��ND)����gma��#v0�4#�������E�{4���$
����������#z��������bz�z<��<����u<�
7���!�:1�b����9�E'"��!s���~��n��_��Va*�����:�������w)����?����<V��J� P��>&�����+&����@'� S�����I2�QtOf�&Z�3�!/���H�X%.��"eJ /vf ��2q�!���m9��?9f��e��3+,�����wW zPj,!
��� Rd�����'���7����X����h`������#�x #��i?��?�����ROV�H�P<��H�EM/0�����g�;���w����� j(n�a
@K@��P���VU{�b����RD?����G�����g1�@�B��  0'��<���KQX���f���	�f�3�<>\KL��%f �����tf����;��1�����}��j�$<���n���SA�`�Q������?c���:��&����X��+��^|i�bYz��a\�������s(����g���l����	�M������t=�Y�E��z���i=&P���<�x�[�iZA�_��")��Y;�5��}�5)�U��\��b�%�q���|>����I�R�������oy�o<���2�~�h������]�tm��SV����4�
�- �Y?]-�`����4_N�5te|�V��S�u|������V�������<����%q(������z�,��X���C��(�n3�#��c����=��'�����5N���+}�����V�t����!O�kn�

��V����-n�6V�c��2W������+
��B�4L�YG:�1������uNz�'bPR�����D!���u��?s����dL!at��������F$�?�H���P�`�����`���x��%�'��{O?2����@lE�����F�<���Ve��v�d�G�5O��-
��z���A^Y�F0��_���}yk���%�f`e����������;{��uU�`_���W���k��o�2�+���JAeE�gy��K"�,��ct���@���r0�0Nh��c�a�
��/���.��\�I<�m�8��t��	J�������,���F��H
c�a� !���q�����PnH�0
��E����W��PH����T(.�����9��\2�O�)�9�z�KS��D@X� Aj�Of���Z�
��r�dq5����0��/����{��Q ��$�EG��o������1P���<N��A�<!���fA5�hy�55���K(�F���h�9�Su�V�"��h���%Z�� �e�P <W��j�
P�s`9�&�%��C��PG<)\ 	�S���X��#���d�a�,�E
�EJ���*�R�v4b$�:��y��p|���T�������7���X�:"
�?_���!x=����B1�R��K��Ts�6�Q11k	���6���%#o�0���.o�["A������^l<��pn����4`&/WOgSX�&�Y��i1_%w�����@}��
��
������!t�d:�����p��
����8�u��Z�z�������,�0��$���rK����t��\�W�s�
�l�@=��S�������*����R��@"��
�q�9�����E0��W��a?71GMF-���[@���G�4����4@*�������5����'8��7%����� =.����#
	K�@�wh\OH3`���������5�]�z��~���B���_���z=�����:l$�),FT���6yJZ`,�M:���u�o-���0a9��'�5�p��w���VD���o�C�2dz-�P� ���l���6���l���UP��n�BpM�$�O���������3����<1R%�U2���FT�	����x1@*�pSJ�%x\����i�x�X]~�<��4F��"f�Xf*A�?�-�P���ew:��0P}P���iW�sU��2��D+
�J9��	`�p�������R������?�����w9N�=�
�A4��m26�eAIFC�e���(�YW��<���cC�AN:��x���v����I����2�[3W� ���(?������t3��,W�
	f����(=D���+A'=�O�@f`YxH��WRX����5���H�G�>��Wq�u	g,0Lj�l���HO�^����f>��w��?���q`~f&g��rD�R�D�C��<��>d��'a����+���n��.��Y�h������8r��4OM9�q�d	����g�&� :���M�"�y���C�5/�L�����1?��������r��,�$�L/�����sT�P������~nkA����x#x�)kfm�q�#d�'�����g���E�06��u����Ub�G���8��$����u�����c�����p�2 ��	�2�2"��b��U�P
0:�2PQ�ue�����r�Z����3I����<�K�G�����P���H���$g������u����?�������?��Q��?6q��?��0�/��.��Q���y�"�M�^��Qx�����G
��q�N�A���������p�G��x#�X�������,
W�=f���v�����������V�Op�	d�B&�KV�$;�`�����5�Q��d�Y�������9y����T�j��TK?�i��?�/c{`�@`���
�r������9y��@����\���/W4����v������#r���$�
��y����I�b�\K����g��Q7���1@����V��F���G=B��������x�&�������b����+,)������
������1nh��?����
3B�7��
����� �'�o�)���`*�I�LK���D�G�zA�ixv~
s�q;����K\Tf����qUL�`��8M�i��E2���u�
<��I�5�����0���w���z����n���?l��.�cx�kc��7�VX���*�L��3;���&���6T����g�@���v���-if�^�!�-Ak8�s;4N�FUS�,SRfc
�{��i���l��=�\��������e��Nf��"V��5xi�<x��J��t���|}z����ST[�QA����H9>����.�\B-%3�R��U���7�_��H�o����G��?�=�G�<Rg����0��t��l�!�
s���}�����|z�w�q>�2��`�dx[r'w�8��b?�>`���\��/��F<�a� �����*A�!k��2�`�F�y� 
����BAR������Y/\�
J����V��t+����_3���E�D�h�.;�X��,D�v[�K4���v����/r3B�I�U��������@"��������	+�cW�N�W'[t�|2r/���]O(��(YC	�NL�;�r�tB?����>_BNCm����b��is������1Q�v�X�������/��� �����J ����G�8|t{XP�
�P�����8�G4"�*�%U����G?���t��/�$��V�r\m��,�a:^&��Ojw��{q����(Q�1�N�#����A6��W�qYn���,�Y�]J��X\�T��,8/�`�N��z��f�iZ>��p����N=��t�K un'�<"K54-�H`Eg�`%��ov���o���>����rF�S��
��	2��x0�<�?{>����V�m�F�i�='��ywq�Dj�>��]�����I�R��,jM�%��\iH;��	� RX�H_�t�\u>L�����������`/��C�-;�3CwQ;y3�[Y5sA�?�5�$}���������~E��7o�����z���������'@������P>��(��?o�B��0�'�����.���hM��\�..����I��Q�s2f�?�~�0��������V	����������m�6�$�����h�����DL\��>��j�^���Z�H;��<�f�,X��0����L|�!���3��^�^yCR�`��]�Qh�B�&`�����<�)���Q8�r�,�B��b	���S��=��+�|��������d�2F1h�}��1�U@���5��*,���~�0%z8	�C'���4��Ev6�������!��f~\8�q@��V������M������+��}��U1����eg����$_���>?R����~����c�X�~2�������/����5k���5{E���#R�:+�����=�+�^���B��|����u�J5��O`������t�#�r6{�2����Q|c�
�'������&�Q���x��N���D8�AP|OWK�J�t����&6h�q�������8
tn�� ��(��d�����5���:�" ���Lr��|���NZ
�$�%^�\�l�����AA��^�D�
	Xp}�p��'E�����IdR�B�,�D 8��~�O���^G�~D
��: �&B����7]����@�Ip!�b�E}�&�������/�����/0����!��q�lv]��1�{_<B$�fA�����3z~��1>�c�rI���~)��G"�7����YP��e3�G4�@���@�S��������b�{v!>�B�P�y���?DJ�u�����j9��J��:d78��8��m`��/�<�v��.��~����#���Fbh�h.�Q�&�E U$���"�|85����t�D�%NH���u*>���B��=�Xq����*��Xl9|��(�R&,j�o��8�P.�*��K<3�S(�tJ��'� x��_��e���d�7�'.�l>eS���0	~�Do9�]A��e����s���A�B�9Lg�g��$�6�����)���pG�Fr_���]�F47�xk����?��������)	q�Hl��]<�Ij7�N�����~���p������Hgu���r�%�A5���Y��J�i�dq�����X`5Ks�E��
R���u1SZ($�f�@����"Q���y��		���Z��x���
3�,����d:��ef0�)��.N,
�1�	<=(|$c ����n��Qe~�h~�%q�X	 |]b��pM��_$�R{��xH����!�����;�������W<���1��'�/��}9�9��
`
	J��'s��v�/n/����[W����w4X$�m�d�`� �=����	`I���[�~@30�A������C�m2�"�g��}�g��D&
b^z�#I� +-��%����,����?��!��Jh��h�,�a��
�$wUC_�AF�8�)��SLX�j���S|~\o�Z$A�p���5�U��
�	�4�<����x�Jb�����^��Y�W�Hv���b:GU��s�x�Z_����y
�C'0�U�d�;�����bGT�c��Fvi�5��A�,>�p��dq5�H��> L@�u��I���W1�9�)n������x�q���o�����TR�"?h_#���mz�.����� �����{q!.���xE���������(�_&�j�/[J�L��Fq������E!&�� �b�p1��@���A���h���\� ��q�����KT�%���������4�)��s@#p"��O
�7�{��}���6O�k
�����Qs���y=T��H'Q�$����2:�,W]������L�4�<a��VH�������H���D��&��|w6���`��O��>@	:�"�������YK����Ea0�l����z�Z�G2�#��|!�������.B?	(�fV��E�(�=:�4��C�HPe��h"��]��������8@cU-t�u��`������)�����Cpa�OQ���KcR@��1�����_�XWF�A��_��Sg�S����`B�Ua�!D��J�(��Sy��@YX{!����n����}��sX�k:��|�M	D0��p�������pM\R��.�/����G�����������a����5T��E������}h���o:�"�)�eJ�IM����������Vp\�xt3���d�^NWp�������������*dnBo�6�+���i��\�����V3�\=�UKx�_.�h�(9
[�����c����qJ��`�rHu���&K��{����4�S�����0�hX�H��h��)������(����L�������#U(�	N�'����
w"�]eX�x�Gj�c��E�GIRa�<�in��}!���Xh<Z��.���`~�RF�_�a,���b'��00bl%���I�G3�<@y"h��W��k�+T�%�.����(J�dk��-��OE���z�"W�&���y��+��P�/y��I2Lp���i���Q!��(tfb4�MH��^c�l�Yp�b����B���%`f��p������h�Q�|C
��-�����$��������m�+�e�3Y�c�������s�V����n�k��%�	���mh���"�17]�$�U5i;_���$L��uI(���X���[(�U�?�sp��&��������	�o�T�
��	�b�Ppx��D��L�����w	@@��8�d8|!l�4V_���>�6J��y��R�Nq;{H�����������B��+���e�~s�F�d3[s���
����i�$�Y��q�L�NqvZ�
j�0
V_A'���u�����u�(���&w����z�.��+h��&r�5M���"�Y�P<�!�����)P���g4�hV��$�G�-,|"�[�~Jl� qI�!�?��#m�n.�e��96�Ua���%dz���	�o4<sEV�����w�����3xSz����Y�\��,�2�H�2��z�C7����x���lO"�@a����	E1�L4s>,P��h
Kt��
�9��I*"����h��	������hT���KI
�O�r�*ak��L?�������,%��W��<6����`f�c�����{>!��X�^�k6g~�-wjt�;����M���y����k�
j��M�~�����H��b�}�`[��5���}�D���Z�)*��z�z�����b}���B�G��
(�;X��#�i<'�2����)u��(��6y�����Fd�Ii�V������������ s|9��C&"���9�k����(������&R����.�4�\2�����a���q#yrN�d?��[�>D:@�A��\B&��'���_��0�6W����`(�����i�U�a)��s�b4:�b�%�p{����w��XV���d%��@����
���9g�
�J�<Y� a��?�4�9����s�q�Qa�o�<&:W^��Q��h�9DT�U�En��t������0�8�wP}��-���:/���W
��'`?z�\gH��0�n6����}=�3?B��t��q-����Gl�e>4T��D����a�mjx����7�8Xm����v�
����(�/p��[3�:0�8�����nS��
���c<�n�(a�V����0��X���n����'��5w��V���O�P���d�JF_\|�'(�������i0�nA7$�'b�
\h?�G
�����5�
=�;A����{�C|Y������|6M�����9�+���T'���z�>y+f���5�����=�Z&����3��O��d
�����$O���L#;�h�@2�eeS#
�}t3�'0eH1/^Ac�v��-��E-�}��J+T�akjJ#m��)��������n9��_��_������{5Kcg��&���^8�0e�zb��-Q��n�F��3�,N����[��^V��!�n2���c��P��?9e���"	=+��I�~P�P{��>���S����2 �t�gU��d^�X�i�E��l��n�Y��u��iHd��l��$f���!��G�����S|��x�J��2hP�K���]EZ�� 
�~��q�~�|/���T��'���=*��2+zM���g��1
O0o����H���rq)t���?Uf�����5>k�_������/��_������/��_�����es����
X
#255Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Rafia Sabih (#253)
Re: Partition-wise join for join between (declaratively) partitioned tables

Rafia Sabih wrote:

On completing the benchmark for all queries for the above mentioned
setup, following performance improvement can be seen,
Query | Patch | Head
3 | 1455 | 1631
4 | 499 | 4344
5 | 1464 | 1606
10 | 1475 | 1599
12 | 1465 | 1790

Note that all values of execution time are in seconds.
To summarise, apart from Q4, all other queries are showing somewhat
10-20% improvement.

Saving 90% of time on the slowest query looks like a worthy improvement
on its own right. However, you're reporting execution time only, right?
What happens to planning time? In a quick look,

$ grep 'Planning time' pg_part_*/4*
pg_part_head/4_1.out: Planning time: 3390.699 ms
pg_part_head/4_2.out: Planning time: 194.211 ms
pg_part_head/4_3.out: Planning time: 210.964 ms
pg_part_head/4_4.out: Planning time: 4150.647 ms
pg_part_patch/4_1.out: Planning time: 7577.247 ms
pg_part_patch/4_2.out: Planning time: 312.421 ms
pg_part_patch/4_3.out: Planning time: 304.697 ms
pg_part_patch/4_4.out: Planning time: 269.778 ms

I think the noise in these few results is too large to draw any
conclusions. Maybe a few dozen runs of EXPLAIN (w/o ANALYZE) would tell
something significant?

--
�lvaro Herrera https://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

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

#256Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Thomas Munro (#249)
2 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Sat, Sep 16, 2017 at 2:41 PM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

On Sat, Sep 16, 2017 at 9:38 AM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

On Sat, Sep 16, 2017 at 9:23 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On the overall patch set:

- I am curious to know how this has been tested. How much of the new
code is covered by the tests in 0007-Partition-wise-join-tests.patch?
How much does coverage improve with
0008-Extra-testcases-for-partition-wise-join-NOT-FOR-COMM.patch? What
code, if any, is not covered by either of those test suites? Could we
do meaningful testing of this with something like Andreas
Seltenreich's sqlsmith?

FWIW I'm working on an answer to both of those question, but keep
getting distracted by other things catching on fire...

I cobbled together some scripts to figure out the test coverage of
lines actually modified by this patch set. Please see attached.

I'm not sure if there is an established or better way to do this, but
I used git-blame to figure out which lines of gcov output can be
blamed on Ashutosh and prepended that to the lines of gcov's output.
That allowed me to find new/changed code not covered by "make check".
I found 94 untested new lines with 0007 applied and 88 untested new
lines with 0008 applied. The 6 lines that 0008 reaches and 0007
doesn't are:

======== src/backend/optimizer/path/allpaths.c ========
-[TOUCHED BY PATCH SET] #####: 3303: mark_dummy_rel(rel);
-[TOUCHED BY PATCH SET] #####: 3304: return;
-[TOUCHED BY PATCH SET] #####: 1515: continue;
-[TOUCHED BY PATCH SET] #####: 1526: continue;
======== src/backend/optimizer/util/pathnode.c ========
-[TOUCHED BY PATCH SET] #####: 3433: break;
-[TOUCHED BY PATCH SET] #####: 3435: return NULL;

Two obvious questions:

1. What are we missing in the ~90 lines of non-covered code, and are
there bugs lurking there?

First, here's an easier to read report than the one I posted earlier.
It's based on the whole patch stack (including the extra tests) from
your v33 tarball:

https://codecov.io/gh/postgresql-cfbot/postgresql/commit/19dace6fca0d9c2bca5022158cf28d99aa237550

The main areas of uncovered lines are: code in
get_wholerow_ref_from_convert_row_type() and code that calls it, and
per node type cases in reparameterize_path_by_child(). It seems like
the former could use a test case, and I wonder if there is some way we
could write "flat-copy and then apply recursively to all subpaths"
code like this without having to handle these cases explicitly. There
are a couple of other tiny return cases other than just sanity check
errors which it might be interesting to hit too.

2. What queries in the 0008 patch are hitting lines that 0007 doesn't hit?

I thought about how to answer questions like this and came up with a
shell script that (1) makes computers run really hot for quite a long
time and (2) tells you which blocks of SQL hit which lines of C.
Please find attached the shell script and its output. The .sql files
have been annotated with "block" numbers (blocks being chunks of SQL
stuff separated by blank lines), and the C files annotated with
references to those block numbers where A<n> = block <n>
partition_join.sql and B<n> = block <n> in partition_join_extras.sql.

Then to find lines that B queries hit but A queries don't and know
which particular queries hit them, you might use something like:

grep -v 'SQL blocks: .*A[0-9]' < joinpath.c.aggregated_coverage | \
grep 'SQL blocks: .*B[0-9]'

(Off topic but by way of explanation: the attachment name ending
.tarball.gz avoids .tgz or .tar.gz so my stupid cfbot doesn't think
it's a new patch set. I need to figure something better out for
that...)

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

Attachments:

coverage.tarball.gzapplication/x-gzip; name=coverage.tarball.gzDownload
�c��Y�}�s�8��������6vFNDJ�c��*;qf��89�����+%A6'������?4� 	�������$"A�_h������jf����{��g�/����-�O��=���}x0�wM���t�C�gt{�`0@]cp80�
u��O��O@�F��U�#�&������%������O�W77/o������� D�`�?�F h�������:8 �=�O�]�)������=>�1�X�";@c;��/�]�� L��Me����B�~�O`H�ti��v���#���.4zsu~zs�nN�����h�B�v���5B�,to�{)
������������I����!�������������������h���n>�]s��w�S�d���tK�1U� h7�������
��$�������� ��l��t�o�*����w@��������}+���m�v;hp|�G�������Y�5�&�^\�=�ods��Z�scB��43�ffi����G��^�����0�I���l&C	�M��
%���N�I�u�n��$�&'H9	0	����J@�D�v�C�3�!R8������zb�m&H�I$�������g��+Dt����2_�������h�����kB�w{�kN��xiu��#�_����w��B�Qht9��-�B
h�N/���!�}�z{~r�;%
N�1d<�����I��(�4"
�������{���cw���*�g��������w7��"�"	J�T[�P�X&��o����_W�l�_tu����@7��j*�e�x�#�$r	�z�T��A����a0�?s��w���g�?1�O8b{�]��JJo
�1�}��7V�6�]�ys	��~��8�v+ Z��k��G���Nr�L�/�4�1
�1���y����@gx���Q���h9������<4=�?U�m�?�����e���`�T	��F2�rP��A	��"1:�������R��"����'F��M�?���*���,9�h�{S���q������*�`����	1�Ox��jW}7�9�VS�������n��W�A,Ls���
���jn--�=@������B��1S[�xj�����j�gP������`��� ���3�dm�&�(~N>uCF(����;��C�iXu�N�A4�+�^Y��������pO����2��jH���M#�#>`��#9H�����mNRe�������}����$
��N���0Cz�G��p���&h�!/�t0�c`���h��{(����7^�A+5���k�A���(�Y����*}Lu�PU��v'�,�ff�f��f#N3:�Oh�YFx��%�tI?r����6�Tr�Iz���Mnq�lN����wI���p���'uY`���������YG5Ft���#���������Re7� s�R�������8����us���L��z��z��D��-�9V����U"#�\F��2b�eD�Q��E*#��j�-��H������L��D���P�Er	�O��OSP30����8�v�����Fx�w��zC�f@����������Ab�=�����e�'����;A'��We2�#[lm���(���c�������v�z{
�T'O����QkL[�Y��:��ruQ��'�N���b\@q?�|@F�]��Y�Vt�+G����N�������Br���'���2�23�
���A^-����.��	kdB���s�Qc�6��\5���5,�. g��81���'Fx����sS�K��1��m������.�����������{K��}z�S���P�@���{j�&',o29���;��f��$���mv��8[no���4���������C�>��<���;�Q�U�1�b�*f��o��8|�	�gW��x�}��2���l��Y��@n����xu����:<�X����:��J<�[�%&��f�*�O��bI�I7�A��J,��y-���Tj�m�#�4KZK-�R�s����H��2�L9��������G��TP�5�[s�(R�p��&����3S����#k������But�SE��S��h������T����E���.)��{~������$��]Pr����2������{��>���>�w���%�������t���%��n��GZ�y~�:�+M��.�+PJ�Y
���\'rd����Jr���^���W��E%�6����QI�Mxe.�W�<��df^����{������=�%*������������oy�)�n�v�pVJ���]7��y�'���?5EE���o"����hj��{�L��m��'~V�$�4!����L��������2�5'��9�w�)x�Fl�?���6��	34���W?��Pv qy��9yr�$
���=�c��)���i@:�=���7�"<��������g�<"�52���E���f*��WM�����#��� �W��1�]�o���������"�>�
l��G�}�N�.����,��#?�;����	]�Q&F��#�+�X�b6��c�����da/�PfO�� [$a�^Yg#���W�3�W�[	$��n��,NU��?���v.5�4�r;�t��g�)�����f�0��f�(��r�� %A���+�t�s��t��V5LaH04�s�#,t��,�5�4"� 6���<��gl-W�����X}wD�N������Y~��&���B���}�c�A�
qnW�@����G�C�����d�i2��q�j�myTr�&��\����L!C��5���b
	���m��jr�3��<�'O��
[�H�I/{�L�z+�����E�E�|����_U����}����~0�2���^1bS 3#
wgbS���e���P�j	�����	���R�V�e?��0��e{���^�	��0U�l�*X�t.���359��p]�����q������$"?�����[��>��H��>�����@��Ow�i'��A�[0���F~�}m��d�JOvz�{dh�C���^�~�����d'�P���:�;�r�1�����,��Y��fA_��=9�K�6r�>*�����G���{9n��{=)����+�������?c��!}9�������{}9s�]U����RerN�)'���{/d�WG��\/DC������������|����Tu2�~���u�PBrt6GK4v�/�Y�r���L�b�`������!�5�	1F�;��D���q���mUE�f���>v��#�HH?��7rR,��+�����#���$�����#���{�:k���z��!���l����x���y�E��7���1������?�G����S�*%���-�*�\��s��U�f%^5�^K\
^������6��1�,��I=���
K��Q#A�uUv�&�3W��+L������kF���7��y��p=[w�r�ga�B��[����`�e*��@���$���Au��^_��_���F>&�T�q�`����=�Z(����
y+hV�U�����D`,��S��������a4*��z�ns�i�/�����������<p���ex�:y��EC�fv���������/�����9�>�~��:��^;����f������t�pO�3�B�����q1�/r�4Znk����Fr������c��]{�;l����%rr��9w@wl$Fp��@�0�Y��"��������=WW{Xb���_��]nVx`��N��|��gtsO$�J6�&Sc��q)�4��y^�+<��AT������;��Z"�E��k��^��hZ#bN�a�	�f{��C�X��K3/�&�����R�i\Y�/�Wg��;h%&,v/i�u&;n��9���BV�B|����|�`Z-%n��Q��-I���
�\^9��)��HA
3�AH,F/���T�r17RlVuc(~��R!�y"L�i_
{~%���t��|W�K�T��b����&�R��B��������|�����%��<P����N��TM�6jSc����-�!��{��o*d�Ug7�[
U��d�b�����o����5>Z0�1�G�����=���
�LU����	X9��$���cE	S���e��n|��sq}IP����5BC"����X�a���V��n,:'f�K�N@Zj�@��`�0D��H�hV�����5���P���|	��8��-��U��X�JY��Zf���\��V���`<����ya��b4tQ���@����b�.F��5��H��K�UK�.g�:2(_�,���S�.s]�e�HWE����}���T�=��H�)ZP	T�N���l9��������
���=�Z$�-Zp��U}�-
���-m���2�G���;��R���U]���F7w-�j|3��������P��k�4��T}��r�c����/����K�B�J�Ru����E�t�rMs�T�4�������V������u��3�6������tN�j�JD�>��RV�Wr�@�k�^�)��zb�e�b+��f��V��5��J�9BP�
=� \��\��%�3s)"����S5
@ZPs���|�`���
X�5N,�8�����5�=Q�KkE��$�1K�L��������.\�T(�^����6���K�5V���������>p�I��*����-�q�R4���q�4��A*'���(��;d�7��q�LRc"�������t�.��_\�=�odS�g��E3P�b:�L3Slf�6���z�f��v���1�K��)63K���f=y3*sk
�>x!J?��r���L��z��z���'�i�qeaaE��S������-�3i���x� ��:�=K���{���v���������8m��K��k�E��a��jDA}��m\�4R������*	o�j�����U\5�����x��_��$pq������� [��W��i���>�|���Op��^����
`:n�+�a�;��x�;��pNzQ�v����[��^�:�\W`5�J���;��p~vu.q�d�^��c~^,�8LkN���%U7�
KtP�
����Tr����k�Tv�x*�I,�0�&���Aew���;��N�n���Tr�x��������`ZtL�u�xZr�x*�]�P������e^���s�)>i���g��O�������Or�x��]td��\���%�G�H��j@�jMt.�E#S�(��Bml������d����"���hU�&�*�\�?u9�G���
���hd�[��"nL�J5��5������J�
�����U�����FV�
t1{���zY�:BI���Y=)��Qiqy�������2��n�vI�i���j���|�nQ�����8��'�����^\�I*'��j"h�T�g[A���*����IJT�t�D��JV5�^^��bd�����A�U���s�W�y59�y��2�tB����e�����y��g�r�������,I�]7����I�]7���,J�n���\�r�����/>�����SrJ�uq4����T�zB8���������\(����D��D�����N���)d�++��
�������z���'�W�$�b2oN4>u�D	BQ�Q�R��T#�z��������OG�$���i���E�!3%���e�	�sSwp��9��J�N�V�N����[�P�`1u��x] R[��8o�DX.�	����6��s�h-m�'����H�p*d����;A\5���&��S�m-
u��r���w6�D��tX.�	�����1�y��&Yp��)Q���.��]�.��,85W��U���i�n6U#��Q#�Fa�S6��6Z	mc����%�Ucl��REN.T~��	��
G����4S
���]pE��E	d��@K`S
,� nI;��H��~J�
��� o,8�`�Z�
�.��B�3�� G�HP�)mx��{��������.8f��
�Og�c�{%�m��K��F�����j<L�0Y
�!WYT~�CT��|���j���8L<T���U���Zs�<�k}k����n�@����v"<v�Q����b*�w���}�;�K;����n���>�?D�_����]�����J��YA;�G���\��w�$.J�+���]�f].<*�v��]��-aT�-�zrF���WaLO�����C���x��0���{9������{E�8���z����/fH(����/�`<��'�X���,����V�����*����V��>����ca���~������|��~IT����Db<�^��Y��c�1���0i�|�����	�3�&��d�����@B�NP��/v��&����F�'��9�Hv�Q�z����<wL;�]q����;%���mU�|��������b�?������)4]��@�
8{Dl�._#��'4)�����&U+VqO"�;������V(q��D�5/
��������ea|��� ��9�!�b��4$<W!lZ���Sc�-����C�
�A��Am&qu\��j\�F
��P�&4�n�S�Z�o��iPE
Y&�2R��<�*��'K6�W�9�W�����1����������O�� ��
�&7������c���3�[���|D���;�vh2N��k�N�k����#? ;�k�:�����L�U	���,#�@c�����=�V
{I��n�\/�Vv�XB���VJ��V6B)=z��Y�+����A�^p]��g'��h�p�w���H�J����rJK��k�S����v�F���B~�Z�)�����k
iG�i
�TS�m4�Y�)��U��\SH�Q�)dJ�,�f��0�PB�!���JvkM!�����kMaJ5�)�fQS�RMa�QS�%���'������v	q�U���Il'��:�^����
�[�E�f`"�h	�$��y>2<	��V��
���v$g�>�L5x��I���V����o����@��8Pt(����K4�L9��vCI�j,��,�(=�zc1P��2P4��2?8b�G=���C���4�R���F�EV�rQ�HE��E���u���Ye�T�N2,�/���\�9�UE�9�P�Nu�#WT����@FW���q�2��K�BY��(�>QXUN4�I2�&`�4�rD�6�jI�G/���R[�����0'dJ]"^����QI/�%�G[!�G/��U�rM��LI/,�f���������6�����:%Zj��#��n��o��^�_w��=�W`��	f���$�������2������2%]7�~$M+S�u�H�L+SL��K+c�����2�����%ie(��U�x7h�e�p�9���q"�O�O�H
\�,�b�ToK(����Uu����X<Cc�a��*�'i��lsRzsF�@ ���^7��3��^Mbi���������	O�H����tY@&P�^�:h�G�:��ybbu�?=P#�J��@XT�����\��Lk�a�j:	�
�b&��������<P�%�%���5�T�V�����)�I�uy��$�75L�m��B�"���U����.��d#�f��J���l&j��������o���@/\���P{M�1:@���h�5��cl�9��3��`�
�4�8���C���;d��x��Ji2�� ���b����E�hv��3�L��+�V�j?��%����=��/
�9	�PWM(���U=zGd�.{�%�m��f���$�(y��M�5�d�i&����f�������3?r3������4
\Iz�LJ���>S?������T��?���5�4���&��M@4��P�]r���1evyb/E2�R(���Z�Z��Y�TR4Y�CY2����x�Z
a���z��J�`-�,��tY]�N�r��}��z����V��k}�������q
��+�p��f!������0��k�V<j�fy�XZ$�mD��S�����t���[��d����
��O
�H�2Z!�	�����C��u'!�� ��u�$��d�F�7�W�Cz������./eo���6���0�Q�����6��Q��-�~��N��j�j�5I��������Cu�o
��`�������/^Z����X�Bz��*���#3
������p�d�O����e���h=��7.��:�����8�):%�����s��T/�����;����&@V���G!�Cb�+.�����F�%�����_��P�F"F+�b���������W_]���(���\�4Z����qgG�����4�;R����[���d�E	��`wC�@^�
�F��Ik�1���?�]�lj��r�#5�_s��`;���j� �!;�NR3.��Dsi���Jm.����l��&u��m��NO<S����T�C2�8�����w�,7��~q�����M���#�	����f��b3��YOl��73�A��AMqP�tPS�,�����gY~��gf�3S��^��^�31�A6
����`��"�D��\��Bb�BZ�\F.;�Q��`�3<;�����]�P��[�=������LN�w.W��f��D�������<�����XowA�#����{��$��R�XzZSu���z4�j���bMVX��A��%���[�����oS��9��1y�;����oS�P:����;�9v���MU�o+h�������<1�7~�C�+ �����)&����K��}��RI(�H����#Y�����r���	�8l�k� a}P��h8GP�B��t��v�&
��&+���<�ys��h=�Z��-�����EP�����Yu�}��r����W'v�����Y�)y���@�z�'K/a	����&��? ���'s������JA5}7c�$7F~��
��y������U�����B��FR0(�k�r����O�W��+�"nh��*�iW�f�Up���sb��Zf=S�5U�)y��������b���N��C�N�RL'b�����4V=V����Y��c�)���kl^��|�D��'w��o����e�`��
�p�,�����6�������a��DS�1��Moe:�Y��}9G���:!��<���:�'��T�*7*�����uL���O���Q�ER���n�tO�vS�q}k����$N���o�X1G@U�`rM�9Qq���2"b�	���h>F�)m	
�:�P�[�TE��URJ�l��o;�Q	]����m
�����9S`HH����1%���{����R�4!B����iKK�l\1Z�d�D~����������^��g#6�[��Io��h'�RD"uL���T��0[���U}m�MJ����[���pcx������l�U:�J,R�<RT������W�^8����?���y�����x@�L�[U*k�)���"D?�-����\uj�����|lQ�i~��Z(�!�x>�FNh�;�v�c�������s1������$�k!�
;�\C��o�?���=���������I@xq}#f����B������}�B9}���Z�`Na<T1`����r���
p�t(�M�UU��&$1KIbV[�a	I�
p�t�S%�����}��m�I��f��]J�^5�PG����������+(p�1��Hv�z�0w�W�.���*�<4����B���m�����h*;�~���.&-�sI�4����?�~^���H#y$��G?'�VX~F7��4�����?���y�Z/^
�q���DW�r��������/���8'N�s�1)r`.@}�r�i.M}�����O}������J�i��g	I�>�R�i��>�R�Y�a��4����$�U�f��4�S�f��,��J}�Q����4s��,�OS�>M��4�����OS�>���4���\��4K�'%e�KW
$d-��c���B'\;,���z������L��@�hc�������l��|��y���H7
Zyz����F�r��6� ):��k�e#9�V���.2-��Q��<A6P�U��X�����1��Z�xfS$�B��_�K�hJd)�rB�<��/Y��JbwL�:����,�F���lb�^��;r���g�3y(a�G���^��j��3�gV���6��3L�_2n���)����<���o�T�����O��U<fM������q���+2Z��]9������)H�q�����&��vO�������K����*��h5�lN�Y�n�S��sC?���}����?.�TE����;I����<�+��c$���\C����\-��2����9ie�m�����YJ�P��^�ck���SU"����������o������vQ]��.�����@��3�E��7�7��))�o��/kz�}4S���	�r&�H�8�fR3��r���� ��Z����^��'h�(j��(�rr�GT��lDU�T]/YS��K��7j�7�T05'�xE�����,8Z������x��3��]�����5
�Z�a�4������e, *��.�R��N�&)*�3������^y8d����r�Ui�lK
���8�&���h9R�M��	���1n9��#Z�4���������M�C`��c��I��[4$���otG�2�����`iN����U9�l:���B���n����M�{/u���#M�@Z�W������p������H�|U��
��;���c?P=����D�d��l`vL��A���M��~��� �CNp�>r�?����������F/���v"'��!���f��2���{�^e�~L�T+��*��7,��p���f�=�������>�9���E�
��A4�� ;D���Ph`��Z��������I���kD���{k���w�7:���iBy�y�yO��34@�"N��&���.�=f��=���(B;)���	��,�����������)������-������{���~�����e"�`��%�-�8����s����/.��(�����+I��u���50����������#���_#f�
�O]z��$�VM���$[*FB1�{���Qv<�}X�yL���R�j��2�E��,��9`&YJ5g@���k5i*���'O��W:Y2�� �M$��)i��1����yD��S�G��e�?�v�[�a���u�h��2��k��"�-	s���-���_ D|0'gv����M�b�="��������~��B�C��y�v���i�.*N��2�]6����L���C��
�Co��}Ji�[�R���h���-�AE�fR�
.E�bS�s�����=�M,��W;�$+e�X\��d�3r���=!���"p�f�U�u��<5�"7�}���Ut��=9���w�R�����
)������7(�������@V��(�%\�a�������~�y�:���G�'�B��ZJ��1�g'Z!��Q9)��9qOzvYo��(${w@&���S�.����6��x���e��o�����A��!�1��X��p�-;��	U���h�}���t+n�@������	��r�)��&*�&Zq�"u���I<L�d��)vC�D�,�~�5�z����5�dn�'�c62�lgj���T�R�vc����.��GoUq�Z
���d�h)�ED"	&Q�h����-R�����%'�0�F[��j�	�V���Ml#����S��k
$�	�`������lja�������+'O���.#K$�%�&L�N��i��h�v��lq�#��g��l�)q\���:��P��1w�p��^�����"/���x���R
c�	�G/@pn!z���
#����L�r�p�"����l��*������p������Bc�V�Er���H��"m�.��'V|����naz���t�{A�2��=�{�=��(?�����.,c���3E�����l7�*��$pK�MO�e�T4�oZ�����-�"��,	d8�&���f6m��~�`#�K�c�2�`�'���VlTvo����l�S`���oeh��,��������ANV$���>�+Y�B�<�T)�\8Y�
�T��l��C�����!s���1�0Z�b�l0S8��@���}���~��&���n$���z��b\d$�R�W����o��!<�/ePo��`�r���8z,������FK��`�lg��f'O]#�7j�q��/������0PZ�I
6�*:���X�8���(��{�����C����y�s��3���*���[�CC�'m�0x��r3������}6mu�@��>��g/��v����B�EV`��|\�[v<�	x*i�\OS����vs�xf�@�6RSGj�_������4qu���x�Z���*�v9Q\$�)��9a��$�-�$�xn�J,'��m��vK@l)}5XVJ_�mj�&`��9,T�M���/3��qXf�Y�~��	a��BXcZ[�"�9!��y�-�d��'P�`�5j)Zf[@l)�����i��G����r1�+{[��>��|K_c��������O�]��+�^N`+Vne�B_�`���9����� ��������K}s�����|�W���e��9`�u�1����[�t3�m�l ��|�F��Uv>����
�R^�wK�A=*�}4��L�������_p��������a+n)�58�!�Bv�e�,s-�S�Zf��u����*i���6X[Jj
���5ic���{��H��	��XB�����f�ws�r>.��w����pM���A
�Q����;Of`���@�lwj��8�dQY�!�:OG?�������meoV_�D)H��R������X`]5yI��\��������P/u�d�� HA������)�>xB<%+�;��?u�/dM�AR���S�H���,����$jf�@z������@G�mH*u�������yy�-�K�����*6
y.f�sw}$��4�������^6���e����p�Z
S{��Y����^^�,����m�&k�D��i��La������^]���)�6�V?����}f������	�B�La�������4�^��ob����W` ��� �i��=���~6%@R��(�d��Oo4\-���q[r)h�t��kAc�-&���
YJ�l�OW��N���������5�n���8+~i��{�:�'Y����<����g�
_V�|{~���:��E
�(D����X��lK�9(�:6wq��f4�����N6f��3���nD���%����� /�f�����$�|���������R��_�4�$�# V>�r2=;��������7�A7N&#����5�������k%�������7BDi1���'��/���z$l!���#w�/d����������LB���:��|uyM81a��������s���?��[����;N��_���!��$�%������/���HY���h7a��g[Sd����������O=G3gv|udH��q�^�^�x��y����.nr�����I�lQl��bC����xb�x,3�K�h���-M0s�(�Bw�D�yY:.�q��}�O���V��Q;�@Blt���gZ�%�q��I7�Y��Z�A�����&�����6e�|��l>N!��S���)d��%�$_*#��4K?'C���(���|�-��(��b��y���b�p��,�V�a�(��@�n�UB"�`�G#�4��A{����i�@��	2��s�*�}v��d��#��UGtU������'E�:��A{������@H]A�B�M��q)��y8��<9�R�z�(����C(���a�t.F�Zj�Zz����1�aYr�����e���<����� �2:����,�*��^~�Z)^<��4���r<��$�L��3�$I�k�g�:�k9�T@��0���
��yx�i����e�b�Y"W�G���
��Kn�i)5�D*���	6),<�����@��| K"S���I����RXj��TZ���Z%���@9r�@� b�����S��0�S�	|}DC�>�R�
XKA��(A�\|��nQi��Rj5��F��F�M��^EB������PY"	��QHe��hD�4I�A>o��H��|P�(��Z���|�=������Ro��\i
nml�v�a>�OiJ�Ce�aYF�z�K,b���-�����!^)���L�#���@�w����e�h��@f�[� l1�-���J�]�Qt6s\zo�P��rX���^$������B�Rzk���9m��|r�����o��,e���V����'O���]C�Fg�yw-H�"��HT��P�@����CJ�\�!&O���^C�f�>��d!�>�I{��D��vX�g1�o�_��<$Z�z
��������K�U(wew�aY.�y#����*-e��Q96T��f@��Wv�����Wj��D+���CK��!�"/�\Z�����]�M|XVEB��y2��o)�5�$$�4��7������	��31I��*;)�`��(����<r��vRB�)���:'
U��br&�ddG����n5=V���(����*��eG�N�#�F*4��g^6��QP-EecV�l���8�H�j�P=I������l*�s���7��B�"��k�O�	���]G�7�X�e;�4Ib�j�U���-���N	�:;��1L�+d4��k;R�"��-��50[����l5�-%�=�������B<��fi��#em�K3&N���]`����	�����j3�[0�)�y��Jw�l�?���$�Z���vM���+�6��R*kP�� �����U�0���L7�\	�W0�<b	�D�S��Y�K���/+^���;:���4������(k������R��m�i9#�Z���>����D��������G�ke[>%�&��R��1Z�- izI��<v�]a'dK�������������s����e�e���c�lbKA�MmG[gR��'���R:�������kS�1f�R�1���;6��T���6o������xT����S�����h[Q������[�z=�@�lwe��������}�aY�;(n��e��/O��U��`�w�W5M�G?���~.P;I�|z����_3�0������?��3~sE�o]dnd+���ea�y�rE�Z�H�A7/4�#�mt����T�Ia���-^U�r����X�%p�R�we3�P����>c}���^<f���g���~�(�V�5y*C<���`��(�7V+���3V�Jd.�_�����O*n�{_?&��$���JS#��"�r(+/��X���8����2���wu+���*W3fH��/��2��_�5�T�m����.<V�UWW+a����o�+c/V8J�)]S�Di�y�:T��'�+f'<V�
q\��0Y*���_������<QDZJ`
-"5b��r5�Hq��������p�� �j��Hi�����i)�5�({U~�M��X���C+��	�0����������<����9yR�����,O�h��|�fQ����&�_�'_,��O���4%�7_�mSP�g��e�kzIM�73n �����TT4k�tJ��LVn�ff���?���|-�j�O�i�IdJ���B,���Z�����zj�%�}��tF���@h��I5���Iu^��+E5���4-k�6�����I����I��GD�����T���u���X)����*K��Z������A�x&6J��h��O�!$���&�<T>)e��O5���T�U��gc�x�4��f-gz
�WlY����2���Y��U��`-7f�i��xh��I�����������7W9�Q��W�ja-7f�g��xh��G��z������
wk�6�9f+��aU��f�ao�2�)��r-I
��r��]��=������j�k�YsB���Q��$kG������k�b���"|J60�����W�e�?Kn���3@S�)�Z���GR������YV���Xf=��)
�����T�!�Y*^��fW�/,�w
��m5)�d�-����$@k#���Q��r�S���).M�>T��$�C2?2D����`�L2�(����r�x����H�T��f��i`��QH���\^��4�<k�&�w.u{�`adTL�^*O�v;�:b���M��Bq1�={�&��D��R5i�J]b����G���hO�U;��-+���e��YS�s����=��~�����'?�-gO
�V=5J�����(�Z�M#�QgJ�������J���bF���Y�ISC�I�q�I��'�//Z��V-P��,\���9`8�S� ���eI���%��9`-�t�����x�����
���F)��6���������k�$���Z��
E�"�S���LW`#ez����/$*T�0�\��|��`�j��\���y��]�6@���j�)u�%Xk��{RU\B [�Yiq	������lq	�����if���k����%'@����{)�`��K��Y�=���X�d�(>��(��Q8v��pR���}y�����#)F�F��pE,F��8�����6�Q����=��?1�������	��������L>D���	�
��%r$���$(Z�pT���D�@%D������>'S��h������:4.k}�)_=,X�$��e�r]������ukF5���.�� ����}L��+�����c!�}��,}oG5:-
���y��{d*;M���J�/|-���;N��TM��
�*�(�n55������_%Ow�����]����&�B���/�D�^�I�$|�/[-D~.��u��RM����]v�T`t��>W���Q��,9:���#ve�
��v�5O��2ZC�H�,ri]5]�z�
W��	{�4zW��T���h��
u�)L��*'�@�3#^r�N��� ��p���b;���&'L��K�� ����(}V���((��&�P)����1?�U[�o��5�u�R�B��a�2�*��l3R���%E$�E���H�2�������,�|"|���|��W��\�/��������'Je�5�m�����b���jE�����9�{�A����jB)q�lY5rr�jn��vn�����	��M��4�=���^Y�s5��9�����y�j5Gk�X;G���[�$�&ii
��r`H�,����J|U�BVM��\���5D���k�A.h����os�#Tze���w����
��d^�uidm�Jj8�&�����-����0��*v��qB����k�Ik�9/b*i�.��-UT
k����}�Y��:����C�r�W�,��Z��zO��I��K#kKU��u)���4m�����zj6R�����t���T�V}M�%�����OT� ��R�[��Lo�n�����}Y>���YB��m�������NQ�1f��J�P�-�I)k.�V�(���<���R��Gu�Ht	m��jxI��uR�N�����Vg������H*��O�B� `����8$�G"�Zk���U�:'O�A�B,�������^e���O��~�J�p�+����s.C�<)
VOQ����*�,W���)��@Mo���+G���������@8�H��(�*~*/N�W��"O�]�\]�Z����/-h$�"�TPhyR���ltH�(T����+G���-K5���;�~�D#�o������</�R���3���U_9����4���m���2R�cb�r��m]3k�dK�>�O��BX}�����U6%����o}��I�$q��cb�r��Ma�m
����8�h���Z#i:����ba-�T���u������/�Jm
��KKm
�����h�,Q��BVO�v���4������50%3D^�k��M�OD�����6b-'I9m�m�,�G��*u�=��`K�����	�������s�	��e�<e����=l�"W[B�����O*K���=+/!F_��cb�+!�����������b�y�b���l���b��XB�>P,!F���I�����mi\EU�����������|��vD8@�y�J��8�a�+���'��W`�}suz�'�T`��l���T�)�����x\\^�������A��+0_0D�������|���)*���v�@�:�$�6M��:�^@X���2_?�I<��rm��m�@���/�'M
ss�_]�������d(�Tt":!�Beyu3�E��5��'Xd������r� ����M��@L3gDv���s�T��,�G�t�8�_�E�[/)��Wg�F���L��>Fq�Xi�O<�$SE~�Z�����z�<P�xt ��`3)3�����;��'�N��������*�5�f(f
�nG�.3���g��(��7��u�'_�<���*�0���2��#)�HS�!�{����@�C���a~�[/���<`��,���������rf���=��]���c��O�D�,n�83����a��=�4W������ ��:z��&����?z���� L�+V�1N�w2�]�
���W�A���*��i�[����<����c{2!b���B�����
��A�%�8�7C ��;���������BMbz���H�-��������?��\��&g �y�;�����)^WaK#�c=�W����e����-vHf5�p����Lz1���eu@�}�!���e	E������k������N2��x%����XjZxfke������x���n���7�g3��8��0xe9ll���������,�o�2h��vct���~����w��������#�gt{���G]������bP����P���{�H����}��A��[�����E�y�^���T��/�w����&�,88'6Q�3/lz'[���n������R���b�Q��'������=�+����g��]��CZ!���z�~u������v����,L��;�
�����v��g�&���#<yc96���-�v���������/��?z5�F_�;~��B{j����W���-����=��;r�1F��1b/�'<��C�	��_�gS<��F��W�c`��}�������W#+������v�XAP�6��R�� ���w�����0��F�xj����7�������*�����K_N�x�>~���p�?�W�o��>�Z�fFN!mOlO��)�YHGnH�5�N��;��+{�@T�/���U�������ge�����="s|R
z���8�r���v@�!�2b�7}�{����+����s����"�#ktO?#��!L� Rtk��(�{�m`M0L��q�AX��Of���B�g�x�3�<�h�"r�����np�CFz����#�q����@1����$X�X����/�c��	"}�=�9���I4���o�c��#<��o���%E�����=pja0&�����o�s6&v���-�3�OH����Yoi�9f�/����2�vvv��������s�'�)���0��`�����%<$�I��!�.�,��y_���Dw�K��;�pC�s>��O��I����������nA
�W�������J��c�h��;���-?n��������=��W{D��S����2�]`��;�0��~b���p��=/$�����zA�ObR�}Rl+��q�'��<�������'���9'�{$MB\2@�N�A��B�IX�P���|Lv�|zM�bI\�� �P�3Fb��!ES_�,0U�7����3� ��Q��w�P�����R��lFvFs0��D��MF#����[jj�����t���D��!������������LB����?�$
0E?H����d)#]@���6u^��I��c�3�D~�;{���kd��S��,��1E���q��-�����~H������ME*i�k0��YN��9�WWP��5�V����R�.�1��%K��s�<���a���J�N�TA���n���bs"n�����������q��Bo�$�C��'�HnHCd+�
wIN��#~��
�d$�����/��$t+d�(����o
W�n��vl�Wdp-D���'��x��fdWM4�Ep��:��XG������K���|�\GpS����c��y���Y��-�$g�T�K`�
!��^���w)n#�����2��(]5�ek�W���v
4�l���r2S"UdE�^�!�9�`MM
���&L�wB�P��v�a���7H��oI�(��B8r�9F
�`���"�@pr�����%��%5/�h=�UPg������?��.:5���N{���N�����#tzL^�k�� 
�� M�� ��� ���$�L���3I;��3I;��3I;��3I�i�#�z0 i�#�z�]����v=��G�������@g&:���>:��tv�����1y��{����@vF ;#����dg�3��	��v�3����@vF ;#����dg��H���dg�3����@v��I�>i��H�>i�'���]����v�n@�
H�i7H�i7 �����; �H�����; �W����; �H�C����;$�I�C����;��v���!iwD��vG��iwD��vG��P��;"��I�c����;&��I�c����;&������@�.P�����@�.P�����@�.|�X_P�P�P�P�PQQ�`�aRn��'�������2�[��~=*�����4�f����3�s���S��/�{����@8h
��L4�����|�4����^�L�i;
��5�d��S�jW
`�|5��p���[��
3|�5��0��bxl�
��l6�����iXm�
`��6������q��N:g`��M��	<7��&�����sxn�M�N3�xn�M��	<7��&����t^���f&|A�&��tv��I�'���sxn����������sxn�M��	<7��&�����_�M��	<7��&�����sxn��U������sxn�M��	<7z`o��<'��)wg���d��!+�Yj���3\����&�}�������/���g;����4��F���R�J]+u�����w`��K�7Q��	�x�S�,�2�.��^��M~�Io��n��Y�f�������;�����u�z�_���O���^�zY�e������V~�L>FS�
1��Y����f���`��c��C���s���9g�s;��2���zK�U�V�Zuk��D����!��a�U��V�[�o��]��%
�4 ;��nu����;^?�����r��
�����xK������Z;k������3�������Y&h����+U����+��f��i���������nn�N���?�C�m�����z{�z[[1����V�[�oW�O�F)��-��������6$�=�����Z�j����^�������>X����E��<�cj�x�hVk�E!�XA�OS����,>�:�
�0�����Z�k}���2������,
1�[(
.M�4=r!��8���n���u�C{"K�����\y��$J��i��5���Z�k��l��S��Qk`����Xk�ek�d���l�L���������N4Iv����R,IM��7�y��>��\�s���>_�>����
���3�
�������c���y���c�sO�[L,�������=�����
^+x�����YRV��Z�j���V�Z�.Q��+��]&��c_c�5y���b����V2g�>�����`da	�a�w��2���`�X��}�-=��}�:�A���
�J�A�����#:npdhF���|��m����z�3p�A�s���A�`�*\����H��jw��
����K��,k�(��G	K�q��]LXK1��B����=�r�P����kf�7���w��,C��(��l�f@�w\����i9+����V��z��k����$��k�	�!��wI�,���D�C9�$K�M58�>�\����hW��EV+����mQY�������VR��$�:���XQ�
���)Y���=������Q�W�����5������4?�����u��\P*��!7��d%��}��#���&/����=���d���Gd-|��M����He(���|���#p�b+MafO1]�GY�]X�����x���@���$�[+.[�.��I(�')\�Y���P�'�=�njxMM��	�u�.��X,��x9+H�@���,��
m���0�����h���)oiO������+�
m)�=�� ";8FlV��
���F�^8#H�u��~o���SZs�����&zs�7zs�7�6���3<"�?���v�o���k�[x�����!Q��VN=�u��Nt�Td�oN��5o��J��4m^�ViXS�YZuN�Z��9MZ���2�)S���LE2u�j�����g^���>�UW��?��x6M3��N
YnV�����P��%~�e3�mQh}�����?^\��������j�������+On��@$TCfG��s����|�[���''@w�A��j����?r*���
����L�L;��0�4K�$�!�@��A�`k`��Y�&�;N��<p��_\�b�E�E�KAV��
5)9
�F�/K��d�vv�N��Oj���OjH���Imy1.�+���������e��]}���}�W�g%}Q��L�S��I����v���i�1o����`|dfR��$���!���y����L���({#/rC�D��
v�o��@my:l��m�VV@�|�V�]S��`��)�:�X0��N�!s%�R����io�%Z~{������j�^<���O�x��������~j]�u���ZWo��.���U?�Z�jY�e����t��r����9SQ�!>Y`-P]
T�s���>��|U�\���V�[�o��]M��8{��������V�Z9k��.����.���V�Zmk��mZ�j}�����O_�B!��1B��1�"��X+b��Wh� �A/ 7`|w��4���x��z��&4�8����AN.��~����S���\QQ������.��[�0t��=�~K���7^v�:I��]k��[��D:'0����0��E�;B7,M��z��@��=3��[���!6��$��zXG��=�N���]�
�)�� ��tr��~�<�yt��1�*{��@���m+����Dt��s���/���1����2�H�@�R/�K\ wvvbM^Xx*�����E���
Y+d���B^��%����`[+b���"��X+����u�.�g���zV������1���9��������19��*��5�����JQ[��e�r���0��'���!x�??��yY�=��aH��1��0H�z=����hbcg\��*�X'����^��*�W!��J�j������Z�j��J�e���Z�jys��Nq�5���Zsk��m�[_��V�[�o��]�NY����Y+g���r�@��S\i����V�Zmo���6�o����V���m��{�q�r��by�+}�Q+i�����Jz��b}�Q�Y�g���zvUw������/����Z!��E�^SL�J�&@
;(���:^�x�����:~�fD�2�F���\"�9_����[kl�����{�;�`���#'������9
}�v� ��TGjh����V�Z-/S-�C���W�f��
��Fc�}"��J4I�T���J7��< ����	<0��h�,���C:���G��	L.v��?��D2��:qt5-�@�~||����@V��E4���o����|l�yOh����
2�������a:��'�M��p���R�����`=�b�"��7A����J;dY�r�YV����&���&X?	�i�
J�o�+�sigS��}[�sF\�<��c�}��v�in���F��|���;��!;F���_l�^��zU���^�����U]���wlKV��u`�����<n�;��J��w:o�V�Z�kE���9�B3�<���>�I�z?���V�Z
k5�L5L��Xx�����������?�����A���#�������>^�_�zy{sz��\�j������Z����j������c��E!�	A
�J����80������f-������V���Z�k5���V�Z��V����t�f��`��3U�vx�E!-���^S���fV�|�:��?��rWg�rA!,�3�dt\�:��A�zuX�!��*��������.qx���?9���{����~�Z�����@��]������u���Z�j��u�run�,��|�_�������g���my4�a���W����C4�x�����G
q���7�|k�C���1����3ZY�	~��0	��;�O����!,��y!j{:�c�
�����h��}|�`}������2������oR�1u?D���f�s�m�Q"�V�B�*�P��j��},��Vk[�m��]��M��4T�������ac\�).�a���������o��`�����N*o�r��%{8�g���$=�T�Q�s�����'�����z��k�^;����k������%����8j�W������ZKj-������r{��iZ�i��u�V���t~y������O���o>^���}���a4s�a��������'V��?�-�w�{~u����~����}����?�����vm7�C�K��^���QK�f�M�E��:r=w?�4�D]�����!�{ydF������A?Sh�{�/����Otq}�������4"];���&�0��i��.����V�Z�j%�D%�+���YI�3�(�F4�!s�5)��tY&������^��u[zT�2MZAk��V�ZA/QA����I�^�C,p��t>��6��!Nn���U���l]B+p����
|M
|�!�oBje���V�Z�ke�De��Ch5���V�Z
��ZWk]�u���ZW/�:Du�N��5���Z�k��i��Q�oU%?O�o�;�zM�k�^������[�\�s���:wKu�$�w�-�i����>�D����-w;�q�C�`|5��[�����g*��'�3w'���N\�
zU���^���{p�m����Vk�h��<�d�|C����r�$�VM�%��M��M���q���q�4)7���%	��9H���Z�k=���������D]�9�i�%K��������B���������Bj��Pl!�-�g~y�hj=�d�����0v��]�9i�6\X�xg�v������mgLh���#/r��+�"����������;��h:$]x��3�����5�'=�0�o
iGv ��E3�s�+Hf�}V��{��]B�H����wh7��������s�S���]�"����	%i��
��,�S?~��=;�>'���Kz)�K�^��R������'�-�������j5���V�Z
�*�d|9u����gNtg����c������F�K7�����O�n�l���B;�cCa�����c���C�
"rP��&"g�)�h
��r�����O�j���Ad�1vp��_SolOyW�9��O���F�,�R����^��{������^V�����%.+;;�/J��<}]��%Y�&`��*�[3��u2���*zZ��7��������
^+�e���=�	���������_��=?�����1Fw�3�v���T���%�w]FD+h�����
z�
:-#�^�/7+$r�CD}�q�����,{�w��P�/1t�!8���uF��)U����K�^"����D,o��Gb��3�`dCM�&xjg�K�l��#kh;v��AJ�i$�;������@dQ�U��d�HzY.�&<��?��~t�G_nisr��c2&y���zm�k�^�������?X���ih������YB���C������(1��0Vz�M�V�Z�k�U�V��Q������V�Z�j%�D%����-��n�bg^�C�
1
F`���M���v���Cu ��c�3~g*��t;�|�q��Xw/K+{���z���^ �����H�Ae��c�=�
�[�����E]7��`�v����-����^��+���~��������Bj~;$���)�lA�����CXLz�h~;�
��A��p������z�m6z����^-�j�t��[�.D��ka;�B���Xp_�'{��x����
�y)�s����r�x��_�n>xc��?�J�*]�t���J_�J�=�S���2�(x����*����5Ir��>��������=��H/<i=�����Z�k=�t=Nt��C�Rd�@��I���N*��2��2���ZGk�u����P�PR��M
�E/�������c'1��i)�y��pP����1���x:�|������"���9{�	�=1�@/�bl��������dc[!K�Z~�����S�����F��H�{��i����G�S���*�4�������hh��w{��
�AL��{6vL��#HX4
�P���� �'�{�i���@ �@�T7��apq{d 6>��e�d�Kc\��i��W���V��(� ��
�����}|��2���;��y�!+��@�*C��_���%��f}�|Awq�
����96)eo��e
�t~u}q}s~�������'����]��ze�+�2Wv������by�.�7���zk8��z4��{��
9��*/x|�f5���$.A��,��]4%;��>��B�0�����5���^g��K�^"����%/�V=9��;;3�K�w�/��cc�z����2�Os�q}.�AT�����N�N��?{}��|�Ew�u)���� ��	~�m��#g�{�8<YZ��bw��.i�����_�p�i���G����lBDN.�4e��w���;��4��������G\!T���r����M��C8p�6��=�N�����k�?�� 8�fx��f������x���c���dGV^^��S8������D�dN��%v����8E��m�������O4���wB�f�c�&���������K�C>@�]�E�]r��t�'�D���^\��}������KV�Jo�6@o�6@oV�
��� �y2~��(����]�s����5�_���
^�����/�W����i��4�����J+ YlA�yYX�������I�%�^�����mz���5Ok^���{'��k����
��4h�x;)q�2+����}�|�,���8:��.��G�8��Q/��^E�������|~���%�Q�4�c��D��9$��*���n��#f�r�v��]}�����h�:YG!��j��hWR����k���=D���^��\p�,�=0�!,U`��a������m�(��z]z�� &'RKv�����S)������a��,x���U���dgxrr�<!#\�M��J��t���<~���H�i;�:���>�~���9X�C{�(����@N�!����59O-���������MH����@@��K8��1d���VC��'r0�4��%!��{������t�$�c����V#3�����g6�5
�M����������%�0[^�����^�c��x{'���YX���dL�b2���iu����
j��S����2d�f�D�Ro@��P
{��r� <jH�:�����s�D,u�����cdM=VT4�/[�!#�}�2�2�j�vE+B�N�u���E�
%�	>gv-�����GQ����K�y��7�v@�$sB�����4�2�3�UOw�zb�a�����C����7�z��7�z�����B���"!����G�*[":�G�;Q����)i�h�L�Z�%�P��Z��v���t<�k0MnC��$H����wg��q4��frw��ta���^6a���sXoV�������,0�Mr������[����6O������M^�;���1����
��yi��gl��<qt�Ex@�\Z�|
G�p���tI��<�S�����7�|>r��|FW�t������8���jM�"�
�;V����`����y��i7���������7�9�/������4�!^�HCS��$k����-U�h�s@�@�_�`>����Xp���
L�
�y��K0S4\9���h��}�z��+�^9���W���plb~8��`�%.O��^S�vuk5���V�ZM/QM�K^1v>�2�g�
We�M,-0��2���S�B���^,�b���[,��l�
��������-�����9��6Q@]4B��� 
��_!L��S%��� ��6���KMKSL�����qD=I�Oo��N����w�8���;D��y�D�*��	���1��C�(zE�+�^Q���P��n� ���C�����cz6���������K!\�������Z�k5���V��W���x'	���%��^��������D1Z�k��u���K��;;|��l�D+���[n]�\�b���.��x-f����Jl������~��Y�g���~^�~��x[���=NL�Z?k������Z?�*�Q+Y�d���JV+�%n�Y y��X&�/�A���<�400e^�#�0S���kW�^���W�2,qe����
wvv���<e������h�Mg����]����2�� �����'Z�k��V�K�oC|���{j���
h���C�����D������i�a?�2	�+h�?�F]���\�r���._�.�)��J�*M��'�����t<&�S��?���851�IE	��:�K��ajDV�w�R:(��������_k��W��*L�'����������AhO�6��X���rcY����{�
�wM�{Cs��a0�L$���'�m0�b����<9���2����{w����7�)?���C�I��iO\�Kq�Do}�,7��'��ID�3��f[�w_����c;�{�l�V����`f~{cc_����rW�I�g��dd��N��H������+��T��Ola��S�)�����VX����i��K2�i�l�tj���N�W2�y�<�AHc�|�s�l�� �����2�.��S���%��AKT�W]����S%�(C5�+���J�������s��R��/���Q��z��h.��z�*<$���d*�G����8�g�QM;��o������ut
:+a��6&K�4��|�w�E,K�]]��:#51�np#t���MX���+NS>����S�w��L�lv��ZU6�;�4
.g�{Km���0��
�S������,�"Xh�����H�dS���'V�T���xA)��)��f�I+G�<�gCcOOT��Q�tuE_�sK���,��r+����|�L����X��1n������Y��|����K���*���A������	E=��P���/N+v�i����zv����3_W�_��|Q��L��i"��>i����L��������H&w�����2K�>zvD�zutMo�+�$W[)�:��_e`��k�b�v���X=Xs�l�h��\��X���`��,�S(�[�S�G�k��I����V�t��/���U�X��S�8r����{�����%P��24+��b������K�G��������.���Xq+	jUY���/+U�
}��Os��Zpx�\��W�%�l)K����f�r������my5�Q�/�l|aP��'���.�qSxoX��\��)�����i:������ �gLh���I��#m�,��`���7n��(R���������S�arYVW��
Q1e�w�I�(f]I��t�[x�����zU5Ez���y�4��k�T�.�J��8\��;�]�o���uV�5��Qh_''���g�T]H�	���y��
}�G���cg���B��_ ��d�����v�����D}�������o�]>y|�>�N�d�W"6U��F�uv�V���GC�Py�0qi����;�5���������
@����5!�`9���0v=�8���?�Q�Gr�Q,[�Sv��%����.��P��N����@G������(�d����I,����@�kC2+]����p�>J������]�9�OG���������b9�?�����M]'�AW�������}������|��X>��u&3S� �I���3O�i?�k�Y�j����aQu��jT�[p�W+�.zo�#��%���W,�1ZC�S����X�N�^��
�"�w�r�N�dMn�������+����k����W�:���4n��+�h��G��:2���5V��9MS7�$��*8�T(_+�u��E-�%CQr���8�O�b�[�~na�����l^10���R3s��T7N��y[m��iz��x����^���.Fh1B�Z��b���W�;�� �nT/7h���m�VV
�i��@z���8W���{m�����������~{+*��V6t�B�;�F|I����0!}``F	Gq��M�+���4��V��gW��2-�*��o����,��s���4:{��%L@��D��H�""E��mx�����M��`8�8{�&Oq������{�;DiC\4��DZ���@��F���R2'�eS^���<Y���E���N�sJ�@���Ia	W�x��Ec�g���v�@y\�����8��O��ov
W+`��m������h����[S����I�-\4\�2����060�+�?.����������~)���{�A�R����-�a�2_�`���ZW*Z�18���`������p����
z�\��^c����
^��~�����@�O�������n��3x�3~�p���l�Nq+\���&z��$�����A�uOZ��������x�=�h����:��0�h��Q��D[sOu����R�Z4�Ds�a���y���@W�+��
t�ti� ��
t��=C��S��B��Eju��%����Bx,<���������^�������~>���C�u7�G���0Z-�F�y]#	u6bd����VY(	j��a��`x�����Sp+p��/�4o�9TE�����Y�,t:�9����-������q����#�non��(������7S-\��1Z������5�I�[�>��uc��g?��R�[�5E`����v�(2����
��2]`�&V����{��J�5G)m��3����
s���	����>��_�K��i%�J��^���m�Z����[���v�&���@��F������bA��x�(nH|�N����YB�U0c]Uz��.�*���Be���L&fL�I��`������+����c^�\�ue���Y�1]<��~��R���������DM���1�t��j*�����U�c��Df�����
 �NN����'O/�O.�=�?:{���	���Bg���y�J9�m���U�lnF<4���^a��w������4��t�#g<6��*�Zu�?�{7���jvs!b�3�KLl�)�d����\��c�G����c�
�-:P�15��dly������JY cQFV�����h���"5��j��c��=��������� A"�D���b=��B��&��d�Y���bz�(���N~B��3�\��������"�r���Y�,|����"%��}0����O���*]�TA�-!�L��uC`-�X���Ac�H>(��''�uJI[E�4����}�y��w���V��Q�,L&���{f�K�i
o���_t5�tU��f�. �2��tO��a�1f�	���k���z�*4��&��/���N��qLU�;���<��RV>x'?�e�+���-�O �{��*��l��������=c��?�4�!R�Ad���	"��<��>h������XR�%���[�,h4��{D�+,�^��[�,d2��/����?^v�T=p���G.��u'��X�]�R��
s��~��Y�\��2�Gj�����
��Fcf�����(�r�*����N�mn��-a�%�21���47m�-�hN���G>��K������`���e_���7/��W�N���&6T�Zs;x���:���6�s�����p������������3Xi$���\���y�[Q�sW�3���X]w�x.j�?����0�F|����>v��
y�z�W�,��������p�����i�3��+rS���C��������.��E����F��+s����7y����"�P��{���
�+~St��:<�i]l'-��pN�ku`�'QDU@TQ��
��G=�Fq���>R,��{_�V������N�Xh,4������_�������^�J��QX�
&}�2��2T���y����^����yPh�)�X4����B]�����)�c�.������%�YRL�&�gwr����[e2��(&��
�IE�V��[��jsSE]Y���Va<&Q��w��k-k�C4���n�
<�9�����?'3���>P�5���`���������hY����
���*�CA�m9��7�"��h����b=Bc#Z�������d�K�Gx:%��m>��s<�T�0s(��p;_���o�aMBR�1?�1?��Gw��;N��fp���D'�@t�	��`������hO0��Qy������Y�,|>����z��
�-on0��f����f�I=;F��^�T�#���Y>"/(��+��,��'���6��	��H�vbu�c��u+sm��BaSX�2>T6��s|5�v��M����`��dW���Pi[e^dE�na�������US[y��"�O:���f��jn}ef��;v���f����;�%����G�����
-�.�!�7Xp<�|������-��
ui�5������S�Q~����Ts����1n�M�;:
�t����*��T��et6W��PLo��c�ug�l�/V�H��V�U�{�����e�G�g�0�E|qzW�2f��'����MVW6�U�T�V�%���a�o��D=>(Z�h�E�!Z����Oc�l}���?�?:��X,��
`���I�OA����z���X,��J`���l>,�/hl���H8,���aJ��]������|�
Zd���o����W�+��~��`l��n�(�+&�x`���o8�k'!k�n�]6�EnV���Xx,<7<n0���a	A��VP+��6���"��d��
d�Y��@��@���aC�5�������6��A�,H$�����D�L�\�.���SZ>��B�o�a���o�(Ep,8��
��8.�v8$����x�����L[��j��]>�-vI��r7�������'_�P/�1�+��W�������;��?S��f��-;1�������gg������M�������wxV���TF_��\V�5�����Hy���-1e[w�%B�S��q$,��\��B
���V�
�&4���lZk�'3��}c]�~Lq��
��L������/���D_}�����+
��f	�����X���[�����v0���`������}K�F��a��b��v�;���i]����F�q��[�-�b����'1���<v�1����<R��z�pV8+��
g�g��;n\�-����F_�C��W�+�������C5���`��.�3Z
lL3M��9�jZ"E�J������ �������J�������P��4��VX�o������c7�k�1M�C���a� A��@`O{r��&�j=1�DP+��
j��4���b����e��&�nU
e��BY��P��R���TXln��B�M.��
h��w��J��&��1�\�}���:?�b ���F���vQ/6����+g�j[�b�]���-���'��"%DJ��)��.���=/��sKa��:��������r�NY�V�lSIF5/=j�]���9/+ReO8;��.t%C�r�a��M�I�
�.�X��V�3�I����vfK^��q��?y�N>�����,�
TU����$��l����q#��i^,u����W�X������������u*0��z�V�,IA+�MI�c�.��m,0����(���\g#�]U�TczF���!~4Xc����Y.w��V��qY�:=3��t�r<%g~�x3��G�/8�p�u;������L�d��r�M�����V���H9��"�E������'���27/��<
&]�s�S�t�M�n�U1�<���|0�4�m=�c�r���
�V�nP���\�O�������e�tA��JW������N_���6eQ(�3U���9g��dx�r<eP������q5���?��r���i��N�7������>k�H����N;��
�w4�S&�������Bix�;�uV"�%��R�;+y���H�+�!z���G�U��M�M1�!����fa��Y��G6G3��4ybb�
~���=�7������gnaT�I���7cn��	O�4�*���	G�Ad�j-$�4&'���(���4d5�f��7a����z�u��:���0)�;�'�UR�&%����2	;��C36/���M>��Y�U���Wl������-�(2Qd��D��o//_)�H��o�r0-u�5�
�\�^�5`'J':oS�����Mm�"�"����X�B.�{:�j��35���O`jK����,����]��N�7���Vs��2����L�(�U���W��"_����.J���`A� ��������]�X(,
������������)F���0��e=:�����?��ZcxQ��������9��@;6��yj����ml��R_����#,7����|�r���������Rg����n�&���H������%������h�BkM�^7\k����7Cg�`�l=Ry\^���V��-�}
g���tB�6�ti9�\�g���O�B���1=;����q�/�"��"E>�|��|L���k���+h4��{Dsg���-������A�G��U�{��
���)����B��N��������}�"�[�TE���sL�9q4���9{
s3�(��D)_O�65R������gs���m�yX���6�^�	�v���1c�%�*e�;C��vs�l���G��6�8{��N�uF��"<�(<^��*o��G���
�0AS����^7���s���q�^��P�xA� ^/�?��)�\l��s�wl6:�J���w�t�0��ue�����L��Um,�W�2���I�jn�9�F5�ACS]-`~���'�\ue*�GCk�=��@g��IM�%{i5�&\��\���jR��H\��6h�\
c8<��
������,���J��{O�CiH)/�����4�g��E���90*u��pK�������Jrx|��+}�F>�z��S�"NE����n������l���u�:a����c]g�p�e��.P��`�N���2`p�4���Z~�5fsm�������>��aa�0\.�3��"��������X ,��{�p�`���6E������+��9FF	��h���k�X��(sE�(���.`�W��c��/M����!�L�0D��yZ��|���T��O+��:�����4�����x���W05���0;k�mY��U����"��bc-WD��O����b;�
�]:stX/��P����\���N�\�o��������G�}�U��n������0��k����wx�^�T�x�&�M���[Se�A�s�A4>L�SA�PN�;
�1�&l���k�.�|��P\(.�3�1�a9�����E,7�*���Be���MG��O[�x��	a�
����=k�d��2��e����!��}��C(�:b4��T�����Bg��~��`���D���-O���_���W�+��#~���:��l����B`!�X,�o}�a��H>$�D����/���h,T*���{�2[���8c��
�N���bDZ���B��e����(F�f�w���da�0Y�,L�#���/�r��8Q��C,����
�xf�8���zF[��w�^�f��p;��:����5��'.\��zx��&���l~�|A� _���SI���I�����MV����<������&�yp@����gb�����}r����~���QS[Z?79W0h2�u��+��w��-���+��[�-�|�]�n��P��1��B_����=�w�0���@Av+���Ytc���Y�,t�/�1���r

���JZ	��?�L��B`!�X�W��N��q���DMH����2�����nUz�B�,���2W�A��j]`��[H�[T@W	�riJ���G�LKh�L��(�F-���W���"i��ca�k]�����Za�-8�]d�-�m�"D.�\xo�����)�����Er�.�
x������j��!j�������"Q����p��Py�X��B���T2�m,	�]�5��Kj����:��4�%��vR�u�[jfJSQaJ
�1�����7aL
�a�����C�c���+��u����&�v�����X~����ie���H&C"�D�����O��R +��
d�{�8�d4xT�y2���a�@��j,����V�������&�������}�?2u�l~pC����.aZp^N��S9F��O�!n1��X�����=������y��`�����IqV�j
�����J�A..-�g�%D(-�J��I���2S�nP~�+���
���`A� x�~`��(����
���o����}��l�f1��N2���&sI���)]yDk��z��/�BC���)����S��"�����_��O����)���l�[��D
7��bj+��W��Bg���Y��m�����8�9�7L�q���,5&k�=��W
6�QmK�4(��z'���J������\\S5����~��������5{(���
"D4�U4�������[���Vx�G��}a���h�8!��(�d��j��0w�vu����B��,�r���B�=��!����J���ea�U0c]Uz��.��\�PY�,T*�*c��&�M!T*���B��L�N
���f�$&��d�����:�VV:&N�=�(LqT�^O��!f�rO�SQ:�`��aak
�G�b������[�Y2�p�aMy_����J<����R:nfe���vh������X���k�
��CS5����a���j�nK�8E�4i$�h��h�Nv �
S]���O�O�'�����Q������3���8�~v�~^A�9���f���3������B��gPa���Vv���7m���_?��o�������C19����k��y�J��������m4�`�`�	��ws�6fIp`��n������<��{���2&�p�V�l�Y)�z?��h'2Q�#�G���
���xE��y52�=&���$Z9&�������ui����>}����q��`|���Y��@V �G���7��X�a��vi�����>����MPn��,���D�u/��i]ftv���d����o��S�P��"C]���r����KW.c�)wgHT�$���Z��|�Y���b�#4e��{�4(eJ��w���n����P��-8�������n����y�e��3�����5u ���G%\i�=���{����6
u���M)�!2Yd��d���������7S�S��&���',$���Y�,h4����� �w|�1k��
������T�(�:����a�����7	,���X@, �mX��ia���E�����]���@X ,���;��7,��tg��M�,P(�����5�`�%���fs��w�"<���g���L�sd=;���%���X_���W��o�����T�T�G5��ZY,$_TF�k5��Fi����)���)02oD��:�yp�Q�)���'n0K7P)�>R�i�ke�W��*N�Cuaf��:�
�2�RI�*��J��EYj���O���������sS��ya}����D�y�����EZ��w���I��pe���/��Q/��bM�.V��_�]������Z�E�K��\����?v:�1������;[y�����p710��YV��(�\n�&Ws������3Kx@j�jx�+�+��[�f\]��@�B�z�����Se<0�Q�XO]��"0�RL�����a���i0[r<��
��	�|H�������WI�#��(
�(���WEa��� ��e�o0W��e�&��
z�{E�?)���I�	�@vzy�������3F�)���D�>�������9z�������g�k���Zx-��+�1.���,��
z���=g����s�agx�p�N=���x,X,������2�Jh+��
m����mc+~������]��m�7z�S������Lu]>�n6<��xr1R����Xf����T�����-����bK/RFC�/�����}`�Q��n:K�O�^���'���W���lAY0���,�{]�y6����a�����O}r��E�M�Iz��|t��,����er=�y
�Zxn3�E��|��m��S��������x�hEq!�B����L\��U��I��$}� o��mT`�5��YLG��B���
�9V���n�ln�`�kj����4�����!�qm���9�O��LL*����!7t�mA�]����-v)����,x�H�W7��j-�"�8"qD������[T3���bj+��8����Bb!�~u�^��ly��c;:���&o-�n���:A�))B:�[!�����t�Wv�
�&.�����{�?�v6Y��%������E>�������@��/�����-9r�3A�C��!w�0z���]X��W�Tu����J6����f[�<�,0��0s�u&g�3P�
���>Tup$��8��q���Q�S��s��\uE����U�'�!L�S���	-�����d��D=~����G�"E<�x��xt*����&	���
�A�uMZ�,P(��Png
���e
Ax,<���c�����M
��� b��@K0J�h>���_\�M@��^t(�#5��d����)g��.�J�������P,1���9!rB��^�� ����*��qE -�H����y���E�B�>+��Y�,`0�+��^�s����`"��:T�$��S()�J
%?<Jv��jz��R];8E��6^����S<m��mD��y9�U��X�������~�"�?7uW����l���wU�_s��R�=?�*x�y���G�����"u�u(l�[�e�9���E_I$�H$�H��{�?��g=b����Vu��	v��bD(���=[��P'2%�9Q
�YI���c���Xx,<�/�/(��`@��^'����������O�_�]<��,T*���{6�����m�&Y�sW�*;�1��K���������Tkq�j���B���K�����ky�*9W��`]��SdT�L��[!�V+��	K ��[}m(/TQ���7��������Qn=1�W#t �������(��`A� X�1 x�h��B�y��v���*zX<$�D�a��z?�c���V`+��`��%��[`�����-����l���U�*p�������'������=�}����!u����Z���\A� W��f��TF_	;���Na���M������f�OH�E,a��TX*,}�E��w�c7R�Qa*�e�v�	$o���]'�������b�2���B��.-Bf�{�������.�wS�w�M�&A!o !p����������M����H���DW&��TH*$�~x$��@�y���	��n$������
i��B�_'i;���(�q�etb�����t�
v�
Q��B�_'Q�������O�.�H�9��2+���T@* ��Z+{��!�S�)���g����Ha�0�W��7�3f
3����_/3_+� �mG4KI���$*��
{�����"���u�^��C�8������6�F�zg A*8��
�b`]�	�f��#jk��NY`jKW�8Vj���%;h��2WUX �\��Z���9:u��TN���4���!�WyM1
�+}��-��YQ{�����?�*{�8��7�(Q$�HD�^E���;���*�Yet0cV�;$F��������%^X0-�L����N@w6��l�1@�����&<-����Z?�[�)����<`��!�w���c,��p_�/���?�����3��3��h+�%�[��0�x��+6�.U����k���z��>��z��&5\����e���N�xBrp�2?���hq�>����cn�O��t������jp�_�E�P���FM+�Pz���4x5��y�vv6jb��i]���%���:��>��J�c�ve�
����2��gsl�/1�[�:�I��p�!�*rVW�)C�V�����"�Q�#�9��Y�H)�R"�DJ�QJE��QP1<Z�=i#��O�;��PX(,��R�`����~1��<�)�����j��Z@-����������*l�Y�W����a��`x��u��)M�����J�j��v���<7M���QP.(������q��c���^���������^Tq���[�-��+�;a���w1�t�����f�Z��D{�}b2������s����R5�c	������ho/�#�V���"M���1�@�~J�
���fA���M�E���wXT��+R�{::z����(�O�R]��9k��"i�jx�>��V�sWjY������=���^��M?������Jg�f��t2%\�x�+�+��-
l�yC
k:�C�:�9�
�K�\U���,t���k�0lk53Tz�Bfaj2��������p�����(��FM	c��� <'P�N�_�s�]Y�E�551X��������J��]z�ip�RG����'\��������pk���y��L��{�������z�0M��6j`�m�w�Y|���R���]��(w��G���@_Qo�[�7�^ROa�T����-qY�c������Q\s?���(M�P�7*'J�(!���"J�;�ONE�0���e��1'��R��t���{�����5��)��,�\`e��SC��F�?TZ9&?>(���-����6��Rx)�^���6�K��,����BM��Psw��S<Y��r����h3m�������U�*@�n�Me'�d=M�y����L�^�=���Lth�
e��BY���%(�_RYG�;�^\����NN(�^��Y��pV8��Y��|�����e��V+_��'��V�����K4o�aMu����t��2����/;j4z�,�u���w�,*�7�W���Bx!�6�1|��B���BL!���J���0�L�s�X�5����WWs�
Q��B�_1QS�����0�)���Ba��Px>�{�@�Q7'����AS#`;i���%�%ga���q}�2!�*7���K����Nk�a�����c���&���8h;y����^B��vcp�a����f�����H�����R6UJA�����Jh���J��J�����Y]��0��=��b�������f@�������J}CG]�A�\]���	�>5\m�*��P�&\���F(����TkU�Y���(�@�y,ML�F-]�8p�����s���5����|��W�-l�-��_ ���(����|1q�V%4Y8������,�^��kl�=�����;�G
�GcW����O���\�����F�\*�T�9�8�pS0|8�-���0
������+Qk����y���\L51�X�4_x_e�n9�5�t%�Y���n�u>�Q"���k�'U���K��A?G�O����z����6a?q�nla������05������r��|a@�3w�`pru�����Gg4���:f�>P~0��9���a����\��o2�L];��"��S|[M�����
���h��u��)Z�h�{��h1G�����xs���8�C�L�<�H���6�v9��\J�C���_�/��'���sQ|YB/F�2�;1���S}��6(�[���U�*t����5�)����P�*D�
Q��7�t���S���Tx��<�8WI=-�Oc^)�)�V�*D�
Q����P����9H�+���������
���Be��P�O�w��V@+��
h�}���j�����=�z����	mAG~�2tTG�"G���o��m�te�����4aQMl^�.�h������`#�x����j�q�I[>���0@��ge��BY���B�A+�>��c���m�.O/.�??x��rs��q"��R-j�Z-�@�,��
e���[(�vj�������T����?���t#���ej1��iA����S��L��������=����5T|
�]j��)I��Aa��������m���>�����������D��8q"�D���8��B;�7�R������\[3V':���W���a�KU��Y�n(T�����i�?��i���k�d�clf�jn0Q�Bg��E����v:5�W��E����t�[���]�g\�����S�Q0�� �&�J-�-Cl��*�\��L��z������?+�>�|"�D���'2�
�L�"tBU�Xz;�\+��
c�����W���Hm����o����Ba��P�P�O���Y�^*>z���(�6�;����������{7Z=8Q�`]�.X��_��?$���-�zc��eb���L<��������%Z@. ��q��W+���%6l'[�o;e��1!!BB�����h��\�;~3�J/��ah��}jRC�>6�^�:��+T��195����j�]�PW�g�K��%�@�MZ�
�qY��n3�m�u#������!C��S`4Z�Bd���m�u5��*�m����YU]a��Ra"w��e�V�ya��,W���q��T�S��� N�J�,N�����B|���#�����D����$���l���h��
4�!�C��H�_��@���|3l�/J6v�R��U���Gov�����c���q�u��c��.�OO/�_�}7~��������hS��Z�U�*@��J�:����2D�R
)��B����d�}���+��(��Q�*�]����
{ppj�C�yaf���;������Qf:5Y����Z]z���[���T:���D������{���l�����m���������+��[�kS��w��4�������������k��2:_c�a��A�h����sh������H��.�X]�{�0�s�����0,�Pop���{�$��9�V����:V�w>����i�,���'�~O�K���+W"�><q�������Z��-D�'��	�>�a�����������)��c�M0(
?
r��JP�ygm@�d��e�����9oY#8�W�>�%7�1~�������}�%7���U�*|�(�:2W[�f[���-���n�lz����m�%L�`rP���-�'��]Y��-1��3�'�n����;F�m��$K-1��X�����[rVW��5HG!33�y�� �_�f_M~j��O�y��@�#��l8����-'q����{��p�>�W�(P(~�P�I��	��tB���tlC�M�
N�9&w��g��������A
b-�
I���*��:Zq5��.�M}����Z�����\/�@
 ?^@������Z�O�'��},�k��4��j[�Cz�
"'kt����&{��/�?��w�u�j1���N�E���)�7++�o��O�'����7xK������
�B��*�JM�t��&�	>���T
[�������/)]J,�@N�
G�UM���QSW�9����
�c$�ens�������TBN!��S��Q��e*psc�b�Fs���g�-h1E�br�,�T����
��R����G�Lq,_s�����{KUk��5����+�)�7��nKl��:e��ceu�L����ya��E��~�1��\"^��O����R��
o�X�!h��\J�j�7�;6]�Tp�����������klk�`�
sm
��Z����Y��?<z����0��c�p��r����%��~�����i��j�w������
������,S�/o��c�*�%^�c�(��t^����1�������8�.�]�i]����������-�8U���W���%����X�;��t��*FX$F����|@[V�1��]���>!����O�O��'�L?��C��|�������`����#k�����������U���
�����T�F�q��k21-��
�
�d��v6W~�+���Y3����B]��@Z�M������\`H�=��v����
d������Sx�w���������ba��x�,���y��E2l���@������o���p������W�$��G���_�_��j��Z�������-�j�+
���i-�Z��H�w�~�M��b�W���I�{��W5:iE����<���Rt)�k��������M�&k�D��e����B��
�b���)����dV�]�V�"�^r���,+����|�4��<�:��C���YL�2���t��:6����Gc��������j��������Kl}�^,��(9Y�:mA�gf�,tY�jx����������(�Q�����)�,�y9u�����6�����	����D�&�{(�����	��������3���1t��`:�������bDB��+��x�����a*��sqt�r+����R���x3�3���{�n�����g�"�;�Q����������o�*Q�<��)`M��%)?�1I�T �����������������o����G�0R��_�>�!��o�7(��p�j#�d�[�b=���)}]E/���R;T��2�����������{����_�c�����u������L�+s�B���HL+� )�Q<<�I�-�9���Y�
��#�O�~�nte3���I��0�G7�TW/�����_~��N�T�����Qn��V�Qx#Q�7�
��1�g��y�\q��xQ�.3�h/F��OY��:�s��\��0�zA�+���`p^��L���w@59�~��_�S��6rt��^_V]UzM�,��g�lD�jK8��<��p�x9r��CcJ�^��h����\\�y�V���t���j��N�8w�p����]���<�u��~=L�>��?��:������M���S����}=��6�����1��1�Y��c
t����1�������J����'�i��[5�fY~��g�N��Ak<����e��mRY������JP\��2����s��+�����.���7�w
�GC~�8s�9<��N���u,5�x������+?��"�m.������0qm�V| (����4����5e<�r���T��?21���7�Z0���yxzyvq��L���3�*�X:)0?��p8��i���E/ ?Nl.>�&&��v�����|�a%}e:�QQ9sM��|���d�5���Gjc�2��f�����
��rf.'�Y�A*w`�Dt����(+�����_������C�!�&�
��m�����'��
�7�zj�zl���;v��]=)�`j�����Xn��%��xjKP���`�m<0���[Ev���p����L}��C�:�\����_�?�zI��GH/���/��D�Z������}����nB����N0H	�����#j���z����I�#�+�p�f\��~���Q�������z�'6����'�MF�g�z��s.��dz��QX�^-�����O��6E���%� E'��SI��/7����7���F
NASV�g%(�--���D�� 'fS��t��W����1����^���_��RpWG����\c����9j^�;d6||����32cDQ��K����p���)�Xdu��Q����UW~�3�
����Y�]x��p��/~�Q�"x4E�(Q�������Z�t���������2�n��������6/��W��'�f�0����	>�4��&?m?]�: ��t�n�����������j/059�t��p���fR�r������<'H%�?��__
�8�x�{�7'b����Y��Tk�[�,�0�����)4����kumMGd�����U ���P�"����O-���R��O2�?����5��K�>��y��A�?�IG��Q���|��L���*���Zj������^��}�������X��;>L�|���L����Tq"*��e��f\8.j'�JTB�I����c$�4=�4��Gb
�1h�	S"��oA��oO�r����7]�����J�tn# �@O�u��yp�x|�W�/;K���#2eF2��W#����EM���+��/�o�7��6��d���O���l���
x��W�V�,Q�k����zD�L�����+��5��`��u��o~�,�z���,s{m��}�����jt���Q+SV��c���������q����8�xz���f�Z��+=q���������G]N� ,ce��0Y,����d�@of��l�F%���&~%�.�))��kW�l�r�Y�o'>�������X����Q����'I�y�Y��c��q�D��'��)�R�	������o~������iW^��fzO���J�������I�>���Fa����Y/�h�4���T��������>��Q���	���mG�7<��I}��O�Mx �pt�w�kr�}�o��4����O�KR����������_x4i�:
�S��n���8��K����:
�p�d*�4��/��%O��}#���/��`�;x�������b4L���i���w�/,�i�Z�]���V�o�c�����
�n��!>�M��N���FvJ��8���	�db��V�}�l��������Rs����g�?�j�e_����X���4��T#�g����,F�{���X=I��8��76�r��$�X4��cB���2iElEXVn`[�zD�*#Sh�������hZ{tn�tZ���E�(m�U�^>��vk;��3�'�����!���sZg����pd��w���r����3:Q�~���L5B��Y���or���o��i}�yL�6���F_�	��� ��L�r1[q�t:!���P}��@���hR^�6&���o����'�^��q����M:��d�CcH�'Y�F�z���w8{LY(��q�7a�����6�T&�Vv{��k�5����R�U��������6OR��k��G�,�8�8�F#�������"Q��R��:�$u�sW��J�1 MxWlB���5�s����(�v�<����;�E�H�Zj�����|L��-����O��]\�=����i\���C���U�^�Ik�a��1����R�/6��
9�.i<�*��a\)�:��m��U�����ebP��Zh	�G<:�K�<�1�c�o�l�0�o��[��Y��Z},F���?���j
�0��n~<�u�GI{�*�x^��;R�U}���SF_�<����`��d3L��iQ2�K�a����1����'��W��%�4B��x�G����k<MxjF�FsZ�����
781��h7�����I��8�cR8�h(���d1�����\�C"�
N�y~����E��{U����v�/S�T��v\5��#�<~(��d��p���#C�xa����N���f���_�����-���y*��J<O��t��zn\����7�T	����e��p�&�hM}�W�{�?���]�ju6#�4�Wm����4�M��.���+����
�a��\���8aC��-��(Q"�DD���� J.������j3U]�*�F��O.�F������J�[D���9y4/�hZe����v��?�z�|���"\{L�lp��;CGg�L��;�����5�2�L�uu��
����Z�F
�������I�Q��
k1 �t�"�E���'
�z\c����C(]R����NqIH> ����	�	ny�dq=~�:�OS$8����n���kT��H�<
��������N�)��!,����Matt�D��r��^)L.�����&IdE��c:�n�)��H�pT�-���rR;QC�����v�i�$�`���O�/Q/QSN�Jgu��I�����ar@�-G��1�����j����;����hzq�j��?h<�|~����O�Q_����N%9����po8&����f����%������\4�F���8>U����Q��� ����A���n�k��8yW��.{��=��kzs������n��	�J��
Y��z��l�o�gp�.�L���[31����p4��������>�+=Y���h�}���R��8l��;^!�f�����9�]������&���o����mr y��vx��Xh,4��������H#���?�/���=��;o�����{���;X��^�����~��%�Zv�`��)�[�#������s��C�v�����8_z�������_������MG�7/l�������W��7+���A[o����<��N�����N"��~��]�1��mj��f�2E^�#�G}�O����'pq�;yF������(��8��(����8��Er|r0t�75X��H�69X���k��/vd��;)����H	�F,q��.��r�o�mb�0����u�/�������m�8�D�m���[�-����=�l���B6!��M��1��S�
��vt����e�-��	�>���[����(��)�8��9X�y��|��-�s��}�L�f����'#uk�s�L��=T��[�����[�fF\�E�+f��KE6R��{J��E�XS%�~�R^.�3J��x��MS�BLS����m�jh)_2'��
#	:6c��Yr�.8>�
|�R�I�m�D��i�8e�������:DH��!!B�]_����p��,��	�w>�P'~�W^�7(�0�ek{/�E (
?|�I�U�X��T}c��!s>��O��������	�i���ie�����n(juE��6���#��	���z��YSR�
S�rMi���OT�d��]�>�Y� S�)�����TB{%;#UDg�c[���)����@�����a� ���o�7������M���a��uj�6IA�K$<R_	��BB!��@BJ����{�J�)���"��j��������S�)��X��S����I���i�E�Nl�+������	���Bg���I����	:��%�T
*����[�%e�2A����CY';z���^�W�U2vlA�M�Y����J��M�M?���m����7���diO	�����[�w��.����)_�\h_������3���{��t�g�0Y���zi�v%\h��Lke�_s9�V
�����ak������j`����������k����}���O��&�8��&	�_��������MT&�+o���g����b���1�X,��8�]�*m��������m=|��q�W�x����}ZQe�����|����u���8�^��W�u�i������2��`�Y^�A|��=��?n��8�p�?��+B�~��6}�������G�y��]H��S���Pjx~{A��k�@b��,@��Z:������}��%*��\1i`�968������M����G������f8���,^k<����N�$]�^���S�������IL�����jK�����jfp�[�;
�j���H�d0�"=��A	��U]�����nenW���}�hQ���{���A�-Mf��I����	vkeQ�����4jag��r�#>l�6SEd*��7�"�l�_t��	�$
NS���{�O�D��1
I�N$%+��!�x����Vj&qN��
�:x��ib��;
�L?��$�8'1L�z�j��h"H���;�"LB�HR�=A������h� �AD���=�4��*�\�[���V`�o���I�K7M�%4�+���.=�G��F�ve�&#yT��!��@�m{_�m�Y����\M�B���M��
l���+��Z�R�'&Cs'����s�f%4�5u�w�.(9Bi���
����H�>"}D���y����e�"�iF�-B`!�X�O��v�=��]���W��G�����Bj!��ZH-��g��pV�?|�Y��m�)m��b�8����bF����l�2���P��Me������v�J�z])��v-+�Mu��(lv����1����`w���y��@^ �G
�cZ��z�i�~EEZ���Xx,<�K��f@��
OEt���-���Y{��8����b��\lYQ����[���v����T=��*�[nQ5~-G��f(w!�	�����m���������&�����aD�I�����7��m�����p������~���}�7~s���:����/�����}�����w�w�w5�b�����hce{ci������5yO�=}w���6y�B_L3�,Oc���r�)���z)[j��Yn�#oa����9��P�b�7����,���lRS���\,U��j���]S
_qe
��tK�:h�'���%��:9�}c�R�`������(����B/
�+��/S��y0�R��1�A4yL|B+�1�	�7��Xt��=pu���e��`�����(��U�r�).������Yr$g����K�����r�J/�l�*��o��bf�,�E^����(R7����)�"w���K�o8��.L9U�AbT$2Hd���=��n����?`�Z���W�+���s��`�A�'��%.�#j�-?���&�5w��^U�/]��3[�{C���`05�^���-���&�nN�yj7^���U�}N5]O�R��b���(r�-^�$"H�)HR����H�
_�+��
y�W��c��l�G5�pS'`��B���c���Xp�_�J��TU�h����������0�C����fA��y�hFU9z�\��t�';F}��|xt�
����,h4��{D3r9���d��;��@\��!E�k1a��c��~���T���i�@�������g�"B��K��N���!��w�H��E��@���O������J���;��
{S%��.��Pj�&�Y�j����!4��������������tzJ��{u���2����f�G���|��#�5�PI5��!������):�3X�A����%�-ma*�SmM��}IV���M��KS��������8q.�\��������;}�ON��� ��
����/*��@�>��l9+L_!�z�+s�2*���t���7�]Q>l��vujQ���rA��|��t��t�� � ��v:5�)W����j5����s�:FD��d�6RB�eQ�l9R��}�dM��fu�+>��@Sy����Fa]m<�g�!)c����n���5��X����k?.���e�*�/x���]am���k�����#��G�7�?��"�DD��%"j�+�h�A��YT���m��{���^a��;hM6w��{V���q�������j~r��O����'
w���2~��\/�������7z�Lt��*w$���$|�������������8����d�\-��v�r9)?&�����Tk5Y+_S�~L_SHZ�1E1��������`���n�����6�R��=!q;%Rn�.�`��L#H���`��m����N�����Y��t+�����M7���z�f��,WS�7*X��$u�wY��������A32��0����Op3�ug����.��:�-��nR��0MwqA"�G��.�
�������QZ���|Q��3n�u������=��3j�}��{P��w��W�P,���
|*�@I�a����xsSZ�Qo:/+6����O���(L���]:��\�
.�t�s������.��5l��"��\_[����\���U� &��R�D�T�>v^��b����g|���MU.��XQ�x��u��-�rZ�w5�&iX�Z���������������jZ?S+\��IM�Z;�w}��R�7�\w<��������>v�PLN��6�J~p������/91��;/j0yz`���8VJ�a�*P���C����������G�S�<�J#�xy����<}��v��}����|���z�3����Le���%��x��y�7b�����������f�tOCL9}p��c[�w��D������_l��c!2��k]��%��B�3�r$�:$;dG����=��2.��(����2�?e�[���A/�NL��_v���iKI$V�z��brckZ�����vb��c��e���fo�VU�JjQ��S��S�M��c]�J���I�7'RC��H
�"5������&��������X]Ju�K�jg�_����e�������BD�����d�6�&d(�RT�s�-�����!�cKG�wlgF������������<�y���yc��@��T������8�%��d�	�"H-i�o�4���?��=���#�N�p���%��|�B�l���_0�����S{<Gi���X4X���)���!L����7"�>�8����q��-RZ��Hi��"��+�o�2 0�����z��1��7�p�+0�k���}&�c���Xx�GSpq��z�Q�o��	�Z�����qDZj[�X���2���hv��6�l=bU����W�A@;h"^�����@'n��,��TGlw�Zgb`p7d]�uy��*"TD��W�o��E��%����RW�"��+�
{G�,�O!��=�����8v�X�p��:��|����p��^p;�0�;��yPX�/�\8e�@�D=�E�L��NnG�9{�i>*k�c`�R�a��`q�'���q����)���4N��ry�0�deRYe@��`l%%-{�E�i���5�	�ln����M���d�h�b��A�X"E�8q�GqSfsn��{\1��df"(��{�����r���m::v3?9��	{=���$bbP�nt���74KyJmx�Gi6q�����e{�8jQ++[h>k=��y��x�	[������#"GD��9�$HWpp��6�u�aX��8
��z��%$����B��Ir��A�
�@!�M�g��,��=��[�-�`��
��c����'O/��������g_=����I=/+[�N5g"���1en�\��e/r���R�bI�pqmw#YO����7���~�?P4��CYd� bA����=�����Smv�a�S4��G~��k��������Sc^�n&����!W��_ /�������hR������Z@-�P�����r09�X,��K`NV��K�\�W��?�R1�w��*��>����o�w�O>S_?���:|�D={��w��.����q�����(M=��,c�����s�*�^O�:����E����U<T�T���1�h��5d:��mQ�X��`��dW&?��5K�����aS�4L�a�Gn����D��T�C}�����p�,��a�����M����-�����2_��_c}�Xt��;�8��_*.dO%I�`!�����"���:a�,V(��e�x��G�H�V>ikt�f��+��t�g}p�����N>���;�c��)�'d�a��G����ij���v7�\��>vy����h�z
Na�W��4��O�@�\6I>���H��K�d��ubV����	C��8�p���0��	v5���jk��.����X�6r��������NW�����Qjo��Ti$w0*x}�\b�������*z6�����P�x���x�����o�C��V��R�u�R[��L#���_x���=�85Q)����|���^YL�B�����'�_�x�V��W��
�3O������}b��
�b8�&�Q_��^sTb�
�P��X��;����!0�6�3��qy��G��#�f6
uN�[��Px~��L��������U`wo����-����W*r�_>v�[{�Z���\�������#�JD| \oX��]yv�'~;�;�a6
q����7���lR���������'�T��:�X��xt[��������������� Fka���R���8��%fu�����W��k�S�+W��;���^p�#o������Kr�L��Kt����/l��r����TWQ�M]]�1�R]n5J]F�����E���|�����~������g\sjE�%n�k���B��st�EI�;���Q��&��BN5�������
������vTz�����p����w�P�/<K���3��@I2���u�Vz�T��=���.>��Vp�@[[��|+����Z���b���l���.J���k��
x�*,�?UJX���b�r}����e�4u�U�0oSq�w:����XW�/����!�	���#��`�u��>����"-��1�4���I�y6�� �gk�h����������s
��b�,�\�'��e,���[�WGEN`�X�1-0�
Sd{(`�����a�hTp��(r�������������tw�B�)�?K>d���H�.�����_��U�wx�F&��4�c��&���<�Q,����1���)��Z"�3��,�O�����d���9��;��qAU���^
�-C������f	]��*�����������d�
Z�����WG�/�~��"�9m��[O�F}��m��@x��fN���(*MVf)#������KDn��
I�_���-A��}��w�! �������L�^�a�3(x���Nc ���f�qx�O��gK��\f~@�8`No�����Ak�a�A�7
n�.�&���
�����(7 �[�c)-dj�NAG<�R3|�ARZ��\�Z����,����^��x @��<35����U��Z��$�2�}�V�30��
�M����bm�A��(�?��1���"�W�nT���n*)r7�^���N�G�vlv�8���fY�rm6���,o"h��hc4����G�4��H1�4.I��YB��3��{76�� ��C��7W��w�`�.M�	����"X���QiGc	����fc�b/D����1�a�q���+/��o�CG���!jT�
�j�sP�]�����:7�4 ���9�(�s�,��H��k�'�
uh{�)��;�L�>~�����Gu2<C���6�0�8�f�%�>�/x���i4�������T���L�;8��'��Xpd��+���O��=P�(M#�p~L�7-&�����=i��5�O�O������N�\�1�t�1�������s�W�D�9�M,38�
>��O������f-�4��h��NU!�����)��]b�`e0}Tz��w������
����pA��}�nFa��<'��4�<�JZ���^a7���b��{����V��-�J`����a66,��������\Q|��'�es:���a@;?U:o�C-���&n�y�l�V;���;���$p��u�#0[�fs	��p���4	��H[w{v���z�:=C����K��FR3�}<�x��xcx|����:L9��y
D�s�	�}������b|�`R�=0^t�K7:��)//��=�x������ c�U0�&��*��������qzf&[�h4>�����������o�=�v�p���A��t�w�����us�D�V����}�>�P��e����b��U���k���SM�ZZR<I�39�]fh���]�����s�E���\�y�L1?ci��!4N(�Pl�@�]��q����*������U#�b�z�xKu���k!ZK������2K~Z�d���k����5�i���}��u�i]E�������f.Z/M��^����I[&�t5e�U-�i>4���=��N�o��I��%~ ���R�Z�������-Q�Ec��1���b�*�j,�cI��h��@T��T�_p�0?������0���"/e-��`���\^[]��e��x^Kbw07�-�Z�H���%�,��l>T�����6d�8��ID������e�/x��k��
�.K�����NO����X?���Z������]
Vd������CL��pW^���;~�V���I�?��[���
��QD�F�D5����>�/$���Y��a��24^�&����U����n��2����Y������$`����8��h��L�����9��5����������D}������8�s���
�c_K���X������:�����$6�3"lBn�
��������e���J��QZ��EC�Y-�T����s�����y��j�zp��f��):6��� �L�~�!������^tC[�b1��$�\y[��Q.��h:��c�=��k�::�FZ�Z�S7=���0��2�"�2�����/R�9�=�Z��3f�pt/��$�����{������+�|�	]���n��w��J8Y�������������YK6�>��r����9�W�-E��L�y����FP	7Z� {�OKT�g�5����(l�QU��;{<���E
��+��[/R:�������g�����q��q7�O��y
�"�zx{�m2����I~����4�D�9'_����miR��e�:���*����t<a���6+�_u?��������?�@O��oT��;>�8:;?zvQ�����=�������(����X:W����U��U)H�
7"H3E����d�E������\P�'��������y��	����v�d�Z� J�u�/�,hl
����9��^��G�'����$	���bk��u�}��i>��-�L���M�z��i�����,�-���{!.�}^�-D^N�(�gy�z�T�������(��4srP��e �����WO4�-�
�:�������G�v����(C�N���"n��~W�W�*Z}��;��1��z����
��f}��tqw�F����
��S�S ����V�q����Q�|J ���
�kK$�(r@����(lq<���t�(V�G\Q��=���&Y4sW(mLnf^��2��4G�&�m@6�B]H���cn��B���-��C]�h�o� �fBQ�|1�]#W?���_\.�u#W�����]S_���@r���KmI�����`�����}~�w|�*3|.g-D��`'�����eU��&�;T�X��?�r%
E��$5����zL
8�%O��>��C�m,A��UBn�f��k�gF�������s���^Dxx��z�k�u�h�iu�kpt=r�~!d��R��nH�H��$9��A�D#��ia�����JjB$�R������j� m����@@I\�������r-���`P	�`������^�����������`�;�W�E^ N���5V�&��������1Rd�Y|����r�C�S]sG���G[L�*�[
����G_�C��"E�8�����nK����F_��YiJy��*�
�KMA�~�?�k]OR

/�j�:����v�.z��:gU���[�J�GZ/�(�Y�������AC�������
Q��eel����`�L��������� {Z���-J�����ik����d,���
h��wz����`������ua;|�u�����8X�:�-�C��N����B7�W��NFm�0�Ss�������V�3G��[k���N-i�SZt���o��V�6�p�ok��B��^c��s�V(��\�mI/0h`�Y>��,1�H�/a��*�L�o�|��ro-�&���+�D~���{(�$��4�Q�<U!��J�?�~f�GR�����T��U�YR=!�(	Dk�k��G��X��ZP�������y%����WNKg\�/z6:>y���o�F/�>;�����p�Gf6p��+�lta#�7d1k���_B��_~�E�DU'4���Lud~�-i��FN�p@V�&����P�����?u|�glJ�Ot���(�~�B_k;ux}9�8�h����VM/����e�yR��k����q��vHQq�H�������Q��Gy��!�t[$s]=Fo�ai�*���Pn�)�$�u���HH��A���8,TK��!�j�8�k��S�����[���Z-Qct���h�����X���V$
�Dx��|���������8D���I��-��e���;��Vz+o�N��PgA���+p���n��7�>!/GS��;�Rk�J�GIc��S��u�x��'b/�x���w<���6����p�Z�*+�F�<�3�5n����zG����jI{t�
�s�
-���W��!��)���EO���Q37�����f�����9�������U��*�
xP����\O��^��>N�ql���
\A

�v
��0��������������F�SW+������G�������>��}�������x�iS��p��zx�n���F���������x�-8�l�����9i��,�)��
T�k������mX[�8��/���6��F{xyi���T����&@_�qu[)S�������RoH�uF�����1�3�����f]��7��Sf&h	WU��]���'������j��K��?�J�a0�OL��Qs{���e,dds�2����T�qa@�H�3����C!Y����B�w -��	S���"��;(F�����$U�����}Z���[Q��_���T
�����������O�W���?�����G���/��[?��oa�j�xn��v�
������HZyU(�VI���eqH�
���amVD���N7��t��H��-��p7�>���b�O��Xk.a]X?�

�������!	�����	mS{��F���c���e�0z]��,������e���/�g�
��"���P�zH�W�;.�	���kY�0�����A�3���v���]2K~�(� =����`+�i]��]����g����/>���sdm���>���\��U�e�/E<��"�U�5��4x�;E��*�h�������c���e�����}|��C�����:�px�-�2}�����k��f��#�w���&��+���dF�����}Va�X��k~���tE���+9��4��ESt�~~�J����?[y�O�����)��xU�C�{��w�7�o7�eim���x���'�Jsf��7�z���k�������%3�{]
:�/^��V=��uw��19��S��
��������V�&��T���������9���M������
~=�+8�[C��j!�2�d�|���$Ys[^����:�&iy~�HU4m]+�Kh�{cV��_��6R����R-�U���pnGv����!TD� 1����q��F�;V4@�Z����n>PO&�\�JhD$Zi��A��������������i}+K���Sf;M
�e� v���I-��5Un�3G�?y��g��������r�	���lu�P��u�k5e�x�i���[�`X�J� ��S��4�Y|8�����jG}��x�/���F��$��A�FG{��Y�U7�2D4������6T�W�,E�0h���OL�&lu��5��2�4����Z�!jb���
>���.8�+�j�������
��B^�X!Xa�>Co��1���k�����FO?����<�����}��%���:*2
/	xms�!g�b��@���������&��qN��:���|���V=(R��_G����01��;:]�ob��Fc��x�\C�����&��~eLzl��n�����Z����Z�hi}��d��W���Q�5��;����svo�l���
J}���-^h�gcR��lL�W�[$W�jd�J�y�a�Y!Hi��C�A�Q�rZTmg�#�v��������[lJ��=�[^Z�6���3lc)����op��t�W���{��Kd���HN?��y��������� �?o�bC�7X�f���ge=&"P���(�;:��Yk��|���J�T ����#v7L�~K�Q���2=b��I��=Y ��P{�c�RX���
�9�B���\]��e������ {���p��p�_��������w������*����j�;SK�2���]X��o��b�T`���,_3q����~.�-Q��Y�/�,���\��H��
U���z����&w	��a�m.MB������5�����c���e�m�c���?15_������my��A[���%�����e��U����M��&�h�]b���;���j���]��Q��
��W�i�^b�y�6�f�	�/+k��T^9��=\�T��a=:�uEGc&�e���C��T>��e����[�e-Kf��v��� *z�v���U��eCeM9��j�ra��++�y�-��3!WX�L_�i?��ozons�9^B^F=U�R���}�I*.��%�������~%o
l�c	�!,,,|��y�`q���q��M���B�s�3����w���b3���V_|-����\�e�I1.�B�2 d���@/������=�J{�U1�����H������ko�i,�V.dz��)r����F�hp��MYS�������.F���0|����&/M�:2����9����������<T��������$��`�MIb���d���4iJ���G^���*~��=�7��l�]�/��d�~X
+��
m��������o���GT��)dS���M!��H67��� z���/55����6���B]�P


~�p�V?�j���o@��[���?>p�W��(�m�^�^�^��{�^���UM<����0v������<��h�XXXX�#Xon�7E|�:gM���b��}�_V&I�I��hZ����e���^�^sH��b�G_���)�G���O&Y�������<E����>�] �i
�M2�����J�����F�l�z�B^�b��jz��Q��
���zp[�C�g�"$NH��8!qB�>���5H:���7�l��U%.!���|K�F�����������$8��=z�������9�h���r=U�'����=�u��fo����}�U�>����Oc������R!�BH����7����#������������o��7 �����`�)�U�U�U���W�1��7��/�)F\%���6D�z��6Y�����w������N)@.@.@.@�/����#��<:������QPWPWPWP�_u��_$�W�n�
qe�wD�u=��wZ������ }�F��$[�����"����������}����(K8�Y+?\\\\�hq�M;�h���C���n�nD��L��2>���]r���������+f�B�
*p�T`3���A_A_A_A��}��!�[E��F�����G"���(�*�"����dt�*��e�%�����W�&��(��z��'�'3Z�cDHbUB��,�0Jm~�cn{��A��^a����4D�*(��>`BlR�c,�7����P����&E{��;1�����nd��s�F�e1�r'�N����{$w���x�X�a��S J#���@�?�+�`�`�`�`�=b�7��2G�2�i�/Y9��4��m�8���c��AgqjT��7�u���g�
������������c82qH%///��%�����I�h]��\%���[������|hW�l�y=O�(q����DWT�y
�;�`�IS��l������C(X�T_c����]���j�DHE>7������ud����n�!�����p(iW���~[$���l����k�E�^�]_��}+�J�*!TB�~WB%`+`+`+`+`{�z>{���R���8v����Um����J��3��s5ML��hz������Ql�t��jk�����������#ZW�xu�������eeeee�e7�����S�������������,���)����l�	[*�(U���A����x_���]�*��G�~��:�j�j�j��U�&O�M�?id��+���U����(�
�
�
�~F9��8�5��m,�z�\����lYa�<�	����8���q���"I�`R�Aj����*�.��Vn��)&��������$���E�!8-8-8-8�!dQ��SJ���2�~�@4�Z��7�#���p�L�q�����-P.P.P.P������Lb3Q�//�_������G���= ol���h^$������w��L'���3���O
Z��0�� � � � ��"r{��x��[&��c�(Cx���<������p6��j�FFFF�F���.�F���G�2�8J��MQ�-��+�{������D��N����"�-��/�v��������M.��Z!�Bh�����ib�hR3�����������������766Hb��|����>}�?{+%���i�D�"9Tb�$^�Rg�Bw��������8����IRX��+J�_��;O yt���?�gv���n43��)\�P���RO��-���T�*���b��d�@=��������l��Mf��`�G�(���&����X������^�w����C�2��SS���S�&�fzf�!�iUG�G�Ia"&�����M�\[W$��R]Z�;���]m�[���3����gy�FG�\��|��������B�����������I������\I���.��yZ8=NM5�)� ����!/������o�/1��6�b�mG!a3GC��&�>��2����v�"���:���-���@����y�]9���E���d0�}9��[~�(�������b�7��i���X����yu����D`&h����`���a���[�4n���U�-�����7��i�k���"�M������������V��:~�iil_u����\�������W�������:Eu��n����w���l��{p�Q���VUr��[���&q3=����r�:�m��xUq�t[��bUaN�/{���	W������6SW���br�S�'�.yH�������S\�U����aO-��+Z���N{��)\�F���Y��(g�m��,��^n��/7Q�_���)��r�}��:��
�,�_
�r#u�rC�rwk����f���1_n���������@����y�4��gMB�u7��b���{�W����^^������}���R�]�����SW�Y���M�H��u����~���|�[�JZ�f���������C�{vf��f�g&*�\�}g���?����,_�,ZHY�g��������������Wt2�%=��>�t/�^�o'p9X��>�s�L�����*�����o(�^-?h;]���QG����B�7���E��-�>�]~~�y�p��/G����*�{������
�	B��P[�i��4��������[���nB�F	��:��=�U�&�%��\�Y_�<O �+.�������}<QQn���dw�����J������1������H��r�S��4�nzen���-k�]���*T��j�E��im�����n9�%��t������[��6�O�{�������t�W����S�ZH����
����d�d|���*���[}�a�hX�����:���5�f���}v���~IP���<7��f�7��hW
3*�b�C�?���z���E�������J�9��o6[��1���� ����G�Ig�5�P���=�z����)
%I����]#�j��(,���s�|w���H�I[
[f|��E��_I_�p!�`�4p�(_�:r���m��/����q��y��TKBp�z��n��iE�u�D����k����y�xn�������
y�UC	�/#"O�o��gt�����U+;�{��-��-m���7}=T���]����U��\]�m}`���P�;�_�[����IZ�������>��o�����<�o����h�$��{�$��n��//s	�+������_�?���/��������?�����������O�z��W�?�����'��|����3����x,�+�2+�[�A�������GU�~$��w>�P��I����@I����|6�F
tc���:f�
�U�Z|�/��">3��p.~'����*�����N���7L�{@����{Za���7,\I��������,�xeJC�^��%���FM@/�(����Q�.��W�����?�6w�$���^��\2���"����**�#W��Q�RkU�;��]�^�./�ch�7�M�k�Cf�M�S
V����&m�������
�����$������XGW@iZ�����V��PH��j�33�a���*����9��P����r�lfRv���
�_�pQ���Y^f������}���$��V��z��/��_�t@��I�1W~��.�,7��dX�+f�2>8&1��~Iq��L���ln?�x<����X�~G�T9)�>���l�����`06����S�����L�^���V=Ydg�$,V���S<O	�a�3��h5��z�~@{&5+�]#�a�{E����%t�������N
H9U���v#�=�y�g�l�9���&v�D3>/�y���D����.�2��^f�U�:j}g�%h���A��������P9}�[�^�u�B�� ��6H�(L��&�=��C�i\�T���s4s��
�(F��%������9���A�t������mex��i�"T\�������<�8z/�
c #�����dB��d**S
'	@>�v�,�v0�tw�����"���U%�I�����s������g��V�$/6��L;�M�E�p0Y/a��C'`������`=���s��Bj������J#�����0�*^��N�&��9��%��T�����)P;��3����G���0O�8���k����S!�:Q����
6%__���7�&?�~��/P��D�1�:Cw��
��d�Hr;S�SXG�}*��s��/pR�v)����e�~��~9>�8���n�P6'�h�^q2��	�gX~�3�v��`���"*������*t����DPex����-� O8�ld�00��$�~��Z���3�K�#W�Du�.��u�uq�������F��
<���<s����s����S��-"2�l����� �lDy`�i��X���O8Y���u��IR0��2�N��D�E�'��t0��}o�rMo���_+����EO�7SPeb(�$�]P��sy�^y�+�k$�;p�]�A"��{��cg|���nm\T��MX�������s�K��
��;h8E	�� ����Bog�����M2��Y��	�#�Z
��H�-�UG��<	K
�n��EY��I���m��9<A�H��0V��70���������!�$�@�fx�ad��1rS�mGI��ac8�,u���>��LJzJ�s��/��8�a���O������x?IL<��jh�����!�[��	�Wz�"�6C�P���=��������L����
�'�%����o���d�0���"��������������?��0�>��}��a�|��,�#Z*z����/P�zi2WM��YB���!&�=���t������w��1�1_�l�y�z������c���s�y$�,JK�����>�4CG/���~::���H�v���^���0JlG�����3�y���vt�deO��<���R�x��mog���U(/��+�7���<Kl��Y��?$S�Z�t�^'/�]��jE.1A�-�++ ��4O�jrK�,��������No���y������t������Xf�{���,O�O�G+�1a��H�t��`�J����6x��-�/��=�x���}����>���_~	��-��%��m�p�}w��������^~4zvz~�_e_���g�'����+��'4���{t����}�$�b�'����pKd�)y��������S�O@P|yxv��������=E����/����}7�F����;zvq�4x��������=���H*W	�A����>6OW5���Q�/a�����4���g����
]������c�H�J��xK2r���Iru�K���t��M������D �.��yj�'�H����f�R��Sy����=��k��M��@��SF�&�^0�77�8{��_�o�a&%%#sM\ ^C��p����3ey�����;�T����eOY1�7}}�"N��&}�;Xm�Pg�Ki{_^7����~_ ��$�r����^�nwy�����[4��>�ms�Q[����R�����u�t�S8K�"I�p���T���_��I��U�������iS~���z��0��k�:K�"������a'���e#?��������;}]���&E^�4����Pk�T���\�4�T1\[��q���p^�����g��F��j��B'�����uf)�k����p���_��������1������(����
�v>���3�>4�M������Rj5�v�[�9x+�@;����#����\t�=\�G���p�S��a���.K,����dW4���y1�w*�_l��CQ�gs\u��$6��v����K8j	O�?�yY�.]0���VE��n���+��k{:��}�>�|�C>4,�d�O�d�O�������{67XV
���������SN���t1�W�+z�M5��,cSp���j����0K��&28��M����.?N�E�$����^`3��(g.����9%z7rD'����'�B���!B.��\ll0����N����miiii?N���b���?����s6�t�l���� �,Z W W W W �!�M����^��� �E����a�m��S�V��
�*���&(?�jQo�4Cn�y������ �r�u��'a����B6����'��7G�x]��b!;Bv����s��e{��Vk��k�9mw��AlAlAlAlA��Cl�^�k(���w?|D{�����c�Z�Mu��*�~�v4 ��}�qu>%[:2����
MCOK
*���xr�=a������H�"!D�^���:s�d�:8PgG����dg+h,h,h,h,h|�hL��]\yn�N�X�_�@����yP�nH]������S���t������|��oq���G)H(H(H(H��!�W�n�a-L�@�a�a�a���Pd��G.�
�
�
��#�v������%�����-K��uv�#��mP3�s�v��tBj���������S�t����k8�r!�B���!��F����eX������k���'��B����SK4tK������F�0�������������cX8���~��l�X��4�0�O[^���������b���4AbAbAbAbA�{D��I�:P�0P�Z�\�
66666�367��m�n��9��`�`�`�`�`�}b�����j
�z�3	�k*c�|��``z���������+�`�n��alp��TD!��������,xm�#----}�M��a�k	�X�*����'v6D���5�n�]���!���+SX�z'�-�-�-�-�}�|�?77*���H���?�;�vxxxxx�O������T�1pGE�����&M�+�g���n�K�T���������/�����i
0��M�2�����K~3��3�1����N	��j\:5/���7\
>Z^��F��h
L�Kk��B��M�I�bo�33��0:�Q1�tyYh��pE���M=d�U�;`�3|]��-�3n�u�,.��0��t��A�\K�?�"��h�>�����V�w�%R�TgC��/��2s*3&�83��Z�9�se�l�o�~��85����.������(��9z#����%�(7!�B�4
i�W�� ����h�_��������]��)Q��#���Q��i�������������oddddd�Q\�����I�=�z�2�Hg�iC�o�3�,��U:�Ua\Yd��#C���M�P�������$���P�ua�u�C|�%x����+c$�Z;��r���'�j)��g�U�#c?T���,n�`�:O���X���/Sl������|3~���!�a=��tvi.��Q��h��x�>y����Rz��v6�;;�K�_f�n�����.����>Q�9���!�������77`60�/��p#z��d*\.����R��1�����Z)W�WuyL����g�A����'�@a;�������:�7r�Za������x���2��u��:�S}b^;��� ��u������U:�G0����d.�)[26�6L,�l�i��������D}�z����dUX^VH���vUMu�"96&��:<$�1���}����	����y;	*�Nl������QP{�K~_���	�;����/��������v�:��\]�\fy�u��&*�z��]~��������G5�C�9=���c�nd�k��iI��)�`c�)"���W���qi�D�<��ire���0DA#NghI��,�
�g����*��-8����i��S���&g�\N�O',�������E�������w��\@�Q���^��pn�G�u$���p�M������X�����������b���r�g��NA�0K�j�3�<�a;�~G���c�Y����k����MQ$��X�8*����7�
���w
����ZLMV�87��y�#������0l��N�K�8�f�v;L!O��{��v�PD"����$D$q�os�����n�X�5!zao��zzzzz�#z�:wwwww����<A�,----}�r��b���n���C|�A�0���o������r/�J
�#���/���������O3��#�-�-�-�-�}�b��+�W�)��:@������
����7�-w������������5�����Y�%��;vWh�{tTTp[p[p[p[p��q�c�A�a���:Bi�e�e�e�e��{����S��0000�C���,�+L������������}��x�����G�]�T5������_jy�bK/�;�Ze��)t�����y�n����YW=�C�MT���s�!�V���.�k�Ar���ysO2��yb�"�)����qW��������jka(����p�����
N�|UQ>��"���-g'qEq�0���_���C�U����}�Q}C�Lh�]=VT�y��~���
C�B+���P��9�����t�X�w�z���o���>��
��9�S!�B>�|
��(�';���+(��#�5eL�fB-�
����� � � � � ��y��������g-��C�����i~y��3���J�Y�Y�Y�Y���P��
z
z
z
z~����c��/o��/<.��W����m��i�p��Ccl!	��������p���V���du�r��S�Y�l�<B������k��W�������p�����~�b�]e�~���7�OL��!��)�E]XMa5����j�sQ�:��K���D{c�S�c�6��.�������(/3������!�"$��0|�5Buf����&��?�s5��Wk�T��TN�A�)����'��������c�^�,�M�CY��O}ff_`$�0
	-�C���!B>� 'h���l[jP�'������������)**�>?��5;;;;;��(D�.���3����KFu$�(��~�����(�R����� 	����
S�e�u��E�mu����@�@�@�@�@����M��l.�n��$s&���V�o�s[mg����������v���AU<g���L��*M���5�H��&��zIq'g�����������Ml&u�5�H�sRHc�e�<�
U�!zS5�0T�yU&���$�7�b���Ti/xN���:IQ��{��6�
U�J�n\���w��������8qJ[�����������b�jh<�@��q���FU4��Og��v�V_�%h��_�,�2Be���*��������_
�
�
�
�
������J=�qs)H+H+H+H+H�?{��T���E�YWWWw��%�����������e_�$����#G�Hg�{*L�>��y��B(����>T��Q��
WC9�Uf��#4%1jk@E�)�������0�#3��bLC1�r��a�S�����t�uj:�s����=���N7
37.�-����;������B<A�Gc��h�����N���8S:��R����H�"M��As?��qP/7�����.Q��8���&r�u�n�\��c���i	�$ c�Cea�.�1��0R	>�R /X�o7�sJ<�)�wrzq����S��J�Z5+�7�lNC�<KoT�j�k��Z������+����y��T��S�����bP�����o�p\�kJc�p[B~z�nK ��Z��C�(�(�y���Q�t��eb\4��r9�?,[�����h��+w'd/c=���t�`0���%p���������a
�"���/��||���R�<O���%@Sp)��p���Y4����]"��
�
�
����n3`�X[�?�V���������������]P���
BQ���A���r7s�����[��r����ooooo��V�Pdd��I��xZ���on<O�C9���I��C�n�a�a�a�a��%�So�Q�Q�Q�Q��8�^OKKKKK�KI{���g���p�*y��yC�K� ���|�gy&���'��J�H��h����W���d�w�������N�:5K���i#�L� ����
�b��"�������`e����:���}����cu�v������i^�1�����y��2_��Q��*��hc��p����~��R�~*�N�:!tB��@�P���J�<TM�����
v=��vG�-�������������.��3W����qvq4:;z~xq|zk�2AWAWAWAWA�.�Vj_h
vn������jQ���Q�
���46�K&7j1M�)�#�wY4�D�Iogj�1.�����R?���NES]A�7���M5�k��Q��ed���MZ5/L��s��p=E�XK*�IT��8�;��F�f��4a�\�-�Zg�iwY�gzZ�2O����pX��(!���"�EB��	-Z�m�ssc����5�X�0OFk��
^{�U�T�T�T����R|,�@�|fjG������"W��~Gw#Tr@��(�3`�����������M��?���Dn��kph��&;�8���R�[\	�
�
�
��i����_���]��W��y^�_�_�_�_����/j����0�5�������"��|��W��+���]������hBP[P[P[P[P{���4��*d������H
�>&p+p+p+p{'�m��<Pqb	W����/��$A?�U����������I��d2���u�3�L�������X�33n��;vn�d�D���,%�N��u�cf���=��P��
T�9QIya����`����RE�y|�@��N�G�e�2��T�C ���o���u;���j�{���v�Ry�����U�a���0jl"���i�hw�X����T�>���B#�������)*�L���i��n{Dg�+�n����BU��
U�*T�T�����1�$�@�@�@�@�;��6?����O���a'k}	Y�dU��}��~}k9G�����xP��-�9�N��g{J5� ��q��%��r���>5���P o������t4��f�F+x��j#����t�$�B�4�iu�OP��2&TI��P%�JB��@���7,{ixxxxx]^��
�c��E�2����E����x1��Y��<o�yc���������������*�����@��
�
�
�
��7�[�md�ll�~����vu�*��9�|A]��0�����<���V�Z�Z�Z���`M�[Av��
�)�@��s������"!9B��q�:Ow���"�9D{D�Z�Z�Z��]�M}��l�IoEF�� ;���8��C'5�����(��K��wJ��t3X-dj��r:������2��@W*7��F�����?��9+���9�����e��/tte
G�y�Og�y`
k�7�_�-b�E�+6@�R�E~��z�T���#v8�h� �}���P���/�����:r��z��PM2��mi�����GgC�4�����o8���R9��1U��u��vG6���~�T��V��}����lw<I
o�@c4PxU�_x�����\���2�V���y��TyWo���h�<W��wH	�
����!��$//���;p��[$@��7v��QO����3C�~��Q����7;�����Xa�vW6�����pc�`���-j�����L1*",�<�T�dF������*M�_
X����>V����/8c4�t��EE��Uo���Jp�a:���B
�N��w��E
F4�.��~cH������O��M�I������������I
k��`2d��e�'�fI���mV�JG0r4Io*
5D�_1��8M��6,�M��$���G4�Y�����
Q���T����-����D51T��BpP��L��>+�}
����+�#�0�H
�y��6p�8�2�z�{XXja����ZX��Y�������0sS�w���Y���%�U��^>�(o����v�#O2�3������N��SW��hs�3�����*�*#b����'���hw�PA-�3���q�T��
L�����P#�R��b�nx����	��;l|�m`{=K$��(�M��?!a��TxB�a3i�q�+�y�!x*x*x*x*x���hO�t�Tdn��l
l
l
l�)a�{��L�v^`�����c�S���4�N,J��z��Z0 �3�+P)P)P��B%`%JHG�n��O����I�{+�)�)�)�����~=:>y�@K�e����k� 	�
�
�
�
���5(F������[���~�SrrAZ^%AGS��������u�0
�5>����(Eh�6%5��\�����f����+h��A�u�iv���"��!X/* ?�M���FW����@�^k0!����
M��������F��0���7��L�78v�(3����
�A��L�����Q�����jr�G�\����^���.���<Q���m�
������FL���d:�Y��3I��'Fw}02�J����Z��A2�������Fy��d!�B�� �)	�:�r���(�	�
�
�
����(�	�
�
�
������������(�	T
T
T
T��*Q�#�D�C�U�U�U�����o67L���^1N�b�SR2��`�7o�QQ;N�
�N�j_V��lM�U;��388������\�
������//a����������+����d�����H��4y=����W���r3!��@�@�@�@jR��U�Eb�k������A��<��g��+�u���������

�^�^��O����� � � � � w���LTTT�]T��&&&&��3���T=~S�T�T�T�tm~
�:����%�*�*�*��~7�>��������w0�j������� ����5�<+�}����.��x�)lz�.\�S(��@�cii�����h���=�l(^�����t�B�����UA��a{+��~'xF�F�G����Sc��o�LST �r����$�zS��Q�0��4��j�s������@q^���!B �@�xG.~��m���*�	�	��#_�1>�M��q2'Z���yU&�� ����]���-W��������&��1���sA��Wd�%����e���:�l{��I�4��"c�oO2U���h��
�M���MwCu�0����Q'i����S�!0B`����S�Q��h��x��Y%�AA	
C�IE|��@`X��2�_"CH�4�~N�a�����4�
�_-0:3�T�_MTi�=G���d9$ff����8G�e^�B{:AN��R�����DP(�P� BA�BA
��"��\�������U@�W�W�W����vl�M�:�p��x�P�NQ������K���N�TW��+u�w�H����N ������n2��c���5o/}��0^�`���?N���(��RRl:�{�*��V���N3��pH`W`W`W`�c�]�[W���^�H<�Eoq���L�~��v�:~A���cT��%?}�k���$s��D���*@OU���r�<�u��
r�� J���9f�9q�h�_F�i�)'S���w�8����B�=�=�N��+�@�]�&�*�*�*�*�zA6���R3���|m��Qi������������w�b\��6�n�\���	% + + + �����h���,��xQ�m�����y���SE��,w�D�4�Q=e�]�CK��M!�����"�����~	D�)�
h��r#@�������+�|�Z�"���^�6�<)%UZ4�C��m�z��~AbAbAbA�����-E{=��qr�X���u?>��X"�
�
�
�
����ntD�����|���u�n���B�����������`wo[]Lk���@8�M��,�.�0B[
/T`�C[P�w��������w�b��G��{� � � � ��������Av���
�
�
�
�
���]m`+�L�CV�H	+ + + + � �fsso{#�������<cg�Q��	�����9�O�M��"c�|B�s0B��7�$/"+����y_/�w��������|p������4A�p�nG
3���Ps��i�����@��,N�&GS�m�*3Cc���r<�f��A
UT�����h��l����&N�O��P�K���LC�hZ2�I�����&�0+0�x)� �v���������	t�}�P�n�]qa�jMJ��}�i��$�AQ����&��_�����������Q�IB�y��4�����9
�c
�Y�#��N�V�����QG���Z�&���T[����g����|f�4�Uf.q<U�I�!k���Fz
����x7������(�$�N��-'M�1��pMMt3�@Go�O�j�j|SmI��c�ci���><�)�2��p�q�`��*N`K�JhG����6����S-/������+2E����0�v�[(����W]�#���z�d������v	���g��"�mo�r��������/V�Q��JX*a����K���
��
�����2�;���s��
�
�
�
��E�P��'
h�F��]7�A
�&�*����������]����uE��l����?*�$�E�Al�C�F��Z�V�V�V�����hOO��tnM�(�v:s��
�
�
�
��-�-��K���o�������Qq��U`-���b����hffcS�����^CE�%�,�,�,�,��^�lQ�����I��*�{K���Z�i��=�hI����c
�����1�_���'�}���|U���|�:t��k������$vT���bS$�&��if��
��;����]�U�%BK���k���%����G]���h9))))?R�p���U�*����8Qf��E�1o�3���SQ.n�(IF*��u��x��Hd�g���w�W����?B~�	���"S��!!(*(*(*(�.��l�6��u|D�����ul��LnV��Z(RNo���(_D���tEy��h\O�!d�i��7q���%����I�!=�-�����)�N>��34r}pL����&kY������.������7%4UP?�4�5Z����yl�Cu����(�<���*h�[��XC��o,0�v�~��]����:�w��Cun��^1���
�K��/?����/p`d����o\�3�d�'�l7^~����Y�
>�8�{����5T[��������S�f���:�S5��X����3�����X=RH�i$lm�������+P|���.�>�o~�$��M9��8g;���er���mR���=,��v�>���2�7��[�Z')e,��5J��(X�����ud�]z�����rfx5x&�fD�ud��/6�t�X�f�d[����aFr��Ui��!�l���4���Q�[|r�u5V��3���tm��EA������-��R��c}����������P�7�������=6��S��)6p���+��UY���5V2D�X �jj���^��I�}���[��@f8l\��~
P���:��%T;��Q����x�S�k����,�{���N]5��aW\��>����N��^z���@��9�~���>�5�����5��T�HG������T;���|�+l��%W�P3��Z�k�a���WM��;�+�l9�9�8�6C��m�n|6/k;h�6�e��?3E��g�<����n�,��r@`2[�;`+(/��$�����L��-S�8v�'����<kL��x�K�%+�=���,�hc�0K
�
	U:e�n���C����P}���^=*0���H�F"7����
2�i^J��f��mf��m��w������wvdF���Cjg���.��tVf��_�4�'q�%���{dt\��z����<q���}���>����B��5t�3�k,����{8���bk���/���
h��c�)#��)���F[[[[�.�{h_�����!2�b�.�*�*�*�z'`=��n@�{
H44444��\a���+;E��������hP�v�s����gAgAgAgAgA���3�����yn�=��c��#?�RZZZZ�;��E���
�Z�?D&Ic�,�
�
�
���w���Q�:�=� ����TW�,n+�+�+�+�{Ww�-�_`bGGGGG������g���k�c����T����(��[�[�[��;�-���|��u!2SSS����]	�����
�
�
�
�~��5�p��Z{�����������M������#&Mf��E��Z::n	Dw��}���2�����[���p%4�*t��oj�^Y=�t��j�:�����E�o�oz^n{�����=B{���O{*������\��(������8S���4�s���!
a�W�������������'�3_���Q0'������0�3G����\a�vEw��=�&N���%o�%3t���B����y�9uQ��%��>z0�	zVK�C�'�.Z��O
MoT�]����D����eH5��d�D%����-TGn��*\�O��K"�=y�L��%���y�$��{e�6��C��D	�%$��%Q
/�U���|Z
R
R
R
R���r�yb�3�W��v��#�
��A�b�7��a#�y[TjRRR�?!��D���j[��%���[��e�'4-*���oe���0:������u�v�����=����O�3�����n�n�<������C�&9��P�8Bq������0L���Mp�����|�x�;%x;�SD�t�t�t�t��^L�0R��$ z��-�AQAQAQA�?1�n6��*�	3���&�$�y�6������>������rptvvz6T��0Q~�%��Xeyl������_�C��B_6��(���nnnnnp[iTG�u#�����_6��T+U)U[R4fPeUe,����9C�c
��vI�������"xJf;����:
���(�0�AW5q�Rj���u�=!/]�~��PJA1�Io;!h�(7��2�CEb���?��+�����
7��}��*�e�������y>��w��: ka0�*��T�ql8���\^�������u��4�!7�O�)���N������1����\����^�N�i^���
�c2ZA�T^�J����_�I���K�8��u)��P���t���_���h�FO�<	Y�,dY��GK���e��m�ToX�h���O��P'����u�p%~�����Ya�a�(3�L�L�L�L��� ��h���on�G�?n�">��t?����hn�9_����?�ki+o������������R��hd�+��*c�?:��Y��(u���k�Y��g�
���|�&&&I�����R����������M�"4Eh���)@=x�<���X=U�:::::��Ng��9W���htv������D�T�T�T�T���x�/tqEd�H�/��(
9U�46�x����[�bbbb����Z����@����t>OF�Yt�7777����eZ�Y_�{a�b8d�1~��l�/+nY�OL����
��
,,�@�@�@�@�*�����@��%e=����2�l^�����UT�M�_�_�_�_�_���vl:QH��mu�a3��K��6�@���& , , , , ����PK���'[����N������^�
�
�
��������BLQg����x�
v�#��9�����K:�p��TA���m�CD�����������l�IoY�m�����X��3����yb�xa�[�h�@����=O"�b���$?3�HoVKs�Z_�.�G����xH1h)���k^��"A]7�7�n������4`&T�K,U��Y�d����.�1��MC���B%��C��d�y�bkire���������Z�CI��.f�4����������`)�K��P/�^��W�
[���dl���L0������
��=u��\�B�:M�LH\\\|�L��^��@���{}�\l�
�
�
�
���R/�9
6~�^��E�(o E8P���3Y����8�:��+=GW��v
##r5jwwW�K��$N�t�o�P;7Q����>��kS��,//}���:O���)T
��.��
MG9yy��fW���:����k��k�	Un�"IS(Du4�L�"�Nb�����Q^�NJ(7)Lr9u��eMT���R]Zc���
c]�D�AiTy��j=6���O^���]���1�cB�V����b=w}�����������
�x�g)�����G����MVVVVVW�j��~e�U������������i/��B�aL�\T��C,�N�����
Q�\Z�6����H�3Z�����e�[�[�[�[��^�����F#���8�N,�������g���7�������:�����b�1���r�����=�3hs�{�����j�uU�������u�F��R_>��m�������(P��^H�
���N�r���u��L^�yI�����Ri���D��/��N��UK��Sy�XXXX^�^����������YYYYYW1���-~�@5]��2�:t��������@�@�@�@��P�F�Q�Q�Q�Q��7��7��P����<6���q��:be9��XY���%���Q�"��_:*�����j@��{�}//�6���:O���f/1t�6��C���6�T�����Y��4444�G�����=0�##�n	t
t
t
t
t��������p��-z���yJL-	eE�*�*�*�*�z�n���v8mHPPPPPW�j7��&z�|��$���g���
�����K�\u9|� �V�@0N
W3�Y2/SMV��������WB��k��+�lYv���\y��sc��g��W��'�i��G �VA6�},@}��*�+��`���P����G���V��Q��./XB�s��2"dd
��
GQmI�7��Y�BZ��P�V��<�s�mJW�m����y�h��q����o�H���m�rZ���e�}����v�j��7i~98:;;=�eV�(��������������_�C��B_����~�kU7�����1s ��y����L�#��Fd�\
�r)�r}��
����
�H�+b�|N��$w&���U�g6�)pe������oIl���3T��.�La4F`sq,W���:US��JG����8����,�0���&���U���h��%Q�q	INM���#'������.1V
��I��G\���|�F��E��2������.~y�/6>��8�
��6����$1i<����V��+�e�2L�_�h:_g�\mc��wbi��F��Q�gL��zIDg�����g��Jm�� �0�xZ�|����9����zVq)`j�%�	�Zp7�
�����Vd���|9��Xyv�k��<���W8���0�q�#���9�+NQf��zc�z'�>�f�@��Y�(����K9�U��cJ�Y�z�}�����.�Y�1��K'I�8R���'$�?|<�j-���}�n�^`���k0<���n #������;��[x7�-�t$��d:I
�i����xi��P�����\��^a���|���S_���y��'h��a����;VT��GT��w�t� U�(�w#��x�������~���}�wz����hl����Fm ��=R�{��8�6L��k"��m�c:Zc��V���z�j��M�l�ef�{=^��h��s�04�%����+�o�sA]E6?,
<={G�����"��y�t�&Y���C�) M��;|��V�'p��O[�����`��!d�N'0m%��!�hB'kH�$(�	���IX����e����,t-��0�TV�����l�H��#�0���{����c��5�):��%�!%���D��YN�H�b��r��Y��@h������TUw�6�}���g�-s�F���*tWW�q�|��Z���+RC�hT}���R��\�x!w��?w��
���
u�s���T�e�y!p�_��H��y�r�]����e����2��������>����B�UAC�
��C�n'2A��|����������t��S�����&�.K�R	�;�����^E�K�6�[ZIy�	�V?��/q�	%����j��%s���,���y��1F�rv��B�����^���1W�����W����-V�:�'����$��Y���6O�� YBD<T3z��{��l�n��v��t���S9'����S�R��~]J��ACY[�i��Y�D�+���$@�p�L{?
����������D����j~8p��'�q��O��N~z����g?�������t�?v�Rc�����l����;zn<�[4��x"��ul�N����FJ�Dvy���7nW>�!I.�.�
��~*��������Pr8�+�5z���O����S~�c��]^�81�)K,�k^�g�J�����n�Yvo��2#�Y����[Gr�Y�����o�,�F�����86��u�����&��$��q9���k���{k�����Xk�'?���������k��WeQ�������~s+�g�����'D+V�3b�\��PA�5���wt�Ee
��87�zg���?����U�1G����v����%�A(yU)/V�4G��B���~>$+Y��q�cmLiRVoRm-D���E�h5�,I�����r<NL����1BR�
�Q��7���A��4NYv8Z)�F
b��-�1���S
��=���f������TE*���sY��6\ m��S�����u��@�2Y�&Q���&]�D�����Q=:�]Fk�����[Y���n���������.ntF��e]�S�O�X��H+f�m�v]$+���
����v��NG�4�+�.qP��'-}�e��u��+f� �ki��d��:�j��<�#?�"����������/$���[�����Qd[�mI�	(�l�y��Jp��^����F^���Q�L5�x7F�������
�p�y�}}�`���U�E@�L����H������A[��^�1��j�����[�]�u�#�r����K��������O�������������������k6z�~����z<���&U�}Mk��V~5�����	o��T�~�����������5��'�e��v��Z��7:�h�?g�d�#I���^'N&�P"���a�>����im/J��8|���r���7v�GO��
Y��(i$��E����h�j�%��_�v��K�s�zO�Mc6%z���{�H��[��Y�G�W�	�O��z.4�a}�=CF�;���%��J���+)�����h���U�����V��Q��x����0��t_}��`}E������kJ�5���_H����6�z�5����}M���&[0���L�>;��
*��v
f��`6�
f�"�v'�]p�w��[����<~
�c�z�<M�b����c��{��������u����!    nQ2�yK�.aO�r"�={cD�Ad���4b����?=��� 1H�� �m����)8
N���48}�� �9V��`5X
V��N-�j/(����V��aU������3�#����x���Y�?���B$�`e\��tr����d�"O��d*�Y�m�u��?ec�Y���7=7Ub�����(��Y�������'I����;��>����.X�)5O������+�"N�����Z��69ZO�MQ������O,���j1�3���
���T�Z/lcu��TN����{V'��'�����iZ��S������B�9�$�'n�8-��,Y���0/b�K�-���ZOw����9.�du)�SYYYY�!dy�,��s��w}��0��f�l�oy��v����80��R�r��� 1H��&�5�
��`3�6���y����}�n2� ��-�>�`;����`�R����x��g�x�M���H �08����-��W����1x�����, ������
Wr8]>�}�Hp�A}���������\��
�*����:y���s�c�~���O�G����������^4��,���#�fJ|TOu1Z��wL������M�:��ib_E4���4�)�����~����f���2����]�I���G��#^�n�t��D���h��zVWv�I�S�d)�j������<�9�d4��lJL]���eI��W��V�O��T����N5�w������v=���_V���T){����
I��&c��~��QV���X�5+(������l��
��4�.������<�k�������.�g�	�MS���}mNcs�1�fj)���B�vKC���
zM�(HW�`E����W��������}y$5�p�R� ���R��X���%�������-9��%\=�2���_36E��<XI;�I�!Wu��+,^a�
�W������^8ip\���������n������<���U��� *�
�~�D�`��h�w�p�%��6�D���!h~~4��X� "�"���E?|b��������}����a��bB��m/�4�+gz�%]c�^���%x�����F.,K���ay��@�x��w���}B�@ �@�M��{���}~��TB��� (
�~~�`�@����	���v�h����&AH@=P��>?������2�(��P�v��5GD��,W�i5��}x�e<�o�y��s�����}�=~����?�|����z�I:���I�;|���l(Ua�S=�Rj�Kc�
M��� �3Q����������8���������-3��	�����si�6����_\F�gN�>�7�-���Rv,��������4-]��j�h�5����y@E�t�qHz�Ml���5Y�Q>T������w������)Y+E���4�4.������1��y��I��gS��dW�B�%�����/ZG:K���g����������$���,��,~�86�F��
��q[�sK@��,�z=�����tY)3N\�s[*�"P���H$/7ZK��3��%����i�"MD�����){�z��������c��#�Qz=��v�������j�uY��;(��K7y���
9F�~��G�9���V�W�=��v�\���3";��S���~I}��w��#���$�����]�����YIv��3YQ^^���mV>�U��I�U�K;y���wY�2���<���l_r��i��4��8��Qz�DM�F��T�L���v�E�u�O�����:�Q�yL���o����4S���h�+���K���F��J�5$�b��FF�e�W�i��n�)�r��r�o�	���n��&�YA:������P�1\�N�b�G�$6L�o+�~��^�eCY|��2�@��s�b��7<��p�r��l!B'#*��Q���JJM���.f�v'�F���h K[�:��x�#����"��=b�r�M
yr��W1��S��(��l}����:=�2�G���v"�l�4y��P�u�U�\��3���'	�h��^����uLw\��J�I9f�(>�c���epB�7U�������]>���#]��*m��N�*���� ��,���1'Z�F��y���V�p��6f��79���d`��	@cUu�#"�tt������a�P�$����B��*��l!�C-Ch�����;2�m|2[�c��7��_��0����>�2�F�1-s����v�pvS����A�I��]]p/��l;9��$�\�/��NlN��
��x�b�����.�����^�1����������&�%gE���V}���z�3/t�����$���o�[���F��^��J���_�D����~e'���9�py�����f�w�����?�m�YGOMW�J�^{�J�5Q����9jZy�O��"��I&���2��N~P_eK�E~^KfI���I��2������tyo"���KEBH��:?�~��~����>P��#A��RG��K1|O�����2���;a���Al4}��"�s4�=L��E��yxiC�K>�{I���t����\����A��%�L4��h�3�W��
%��9T�@"�r\��U2!2�����^���=��������������$>�v������V"�p��R����Y{�{��5��D�s�v���l�g���6�^_&�5"]ca�X����9��sVZ��+^tR��2Vv��	Z�-U�%�l��
a����s��%��k����:�~����������Z��k������Z������vra�+�#�d�}$@�d���l7,��]��P[I�om������S4��y^���5x
^o��3��A��Y6��M�@-P��7T��� D���s��gxh�Ac�4�o��6'f�AZ:�US9[U
 �m�S1l�<F����
S��Ti�/���x�������&=P�����C/*S�b��E���.�!n����_�W���C@@@l\#w�=d�^F�n{U���U0C�����rD�������;�&
�~����9/q�;�xW���x�j���B���l0����}��������D=��������#%��H���9f���V��BR�� 5H�����{6s������]S�a��N�
���{�n�5����v�h�����[GX]@�t�MC���x�����@�r�����m,g���5��I�]f�+({��gw��5�c<J�xM�G����@a���>�C��1�{��������[?�Mk� �^;�w���
y7�k�6�`'ry"�G=	sA�Iv<�b=����es��%k���^{l=[��G�����r�Z�d�*}�m�]�S���W�4q�R(�}����`�=�\�R+����'�����'����a�l{��j�uM"������?�$O�l[Y���{�~�����������S�'�tm'
?}���Y����]�6�f����8����~��f��Q�N�;�����`������naP�LQ���"���wf
�(�Q���GA���z�����vv���\��]�w���[[��c��l�	����'���~��t�?����DJ�*-m6� d����{�j���9�@`�o��d{�W��8%g/��u����%QaUp��o(���*��������@F��iypmm����X[���n�<	���Z��l��Z�>�M�V�����X��(�`�9�7j��2�T����2E�C�I�K":���h����N���|0�@�b!�f�
��cD��W��s����H
E��m�"*zl�mH�!��s���b�F���		]���(e��`Zu(A���Dxv��
���r���E�U9��+A����_p B	�Q�Vn��F<�0 ��b%G!}0�GF��h��E�c^KK�G�G�G�G����_��{�^�������=��`0���e�{�`XV�u����������\K4�-����.q�vQ������������S���CE��K���#3����Mp	��[��,t�0cU^�vQ�y�YB{���8>�)= {�j�%������]5+��
Ios<P.�z!������y��FI��x6V��@�r^�����
�D>�9�}������M�B�6��k�Kl�w�P�<	\p�2��N�b�[s�a��l���Z�I������G=�q4f	���Z����\�m���7����@k�]�.�u�����+*��dl!'[06�uP�8��}[��MNE����nt����������������+��>��h~��`����;1^���Q�#'�&y,���kDq�ZT�t�Zr �w��[1����c��=����3m�i>>/�Ij�\6��7�h*�W��'�������/�_��>C�kkk( 5������%jE�nK�N��FN��:���?�a"��� 7�
r��H���P0C� ��_�����N����-+a^�X�qD��3���U���Q#��n�Z�6�
`��6C�+g��>�q8��?��/�������n�mK�����8a@����-V�B�Gv�b���~!����#`�v�]`�*�}��}�����"��R���n�������a@�"�?%';�C� ����HL���v>=g6�-�-�-�-_�l��I��;`���-@�n�#M
�&1p���-p�)9�0L����q*������e����L���.��i�T]�y�*ty��F��w�a]�8��YRY)��W��7j���ZG����p�,q*"V�S��~�����c{��I��;/q�U��FP	S����rU���c��6eLe���q����fq��� u^?8��m\��X��fBe�����xv�c�Sz���qFYV���d�aS
�����c���J&�zl��$}G!��'!g
O+3/����K[�C�����
�
�����������E�qT���r��y ��p �B����[���`/����f�w!��p������C���W=�*�U��]p�w���yI����Z����_P{���7�[��l�
X_��{��dYAV��b��3�o��C	)�o��#����r����q���S�I.��%:,�4�NN��%#g�q�8�Ole������j��h\rD�-o[c�0����H��e�b��s
�Pr�gl��F�����?&Xh���*l3���Ek�U.�N��Mm�$�rJW��c�{��q�0����9V���t������:�<��P�H�#>fj�SSQ��R�$Nj���v���*|���[��`w���������+��q�E*�A"Y��Ry�sz�ac����GY���������*D�=�*e����8�WC�.s����P�\Zx6��CE#h�A�_�R$��r�&��N��X7b�	��}o>8�6�>�l�i��
�"������������V~z��"�K���;��/����&�Nj�D"0[�
��0�U���3CeA-���y��?�A����#�]��������������6�o�[�6�^�u�x��S������.���|te�]�/���q�����������o�[���w
�( 
�������-��x\�����\�h,�R������K���cqi��'� �^i��y{�������e����La���)u��y�{V���j�(>��N�]�2���<0s�Z�����_x��W�3u\do����4�����aJ^D?^��C���������O�Se�Yv�����a��(�R'��[�A�Y^��W��7Ya�*����)�;;�^R[��x��2�Wdly�u^��e�7V�����Vb�����\J{*�%�L�
����%|P���K;qd�!������-��W�a[�L�J�X��'	��?����G�o���i�����,1c��_���~r����������pYU6w��/�<�mo��U��j��d������ I��E�3j��vR�3;!b���>~��74D9�����6@x��1��(GS���_�����u����`�|��P��k����2�_��+]��t�s���.�f�n��~�"7CE��������be����p��\�4���T�"=���yV%���,
�N.�x�����(��L�0�Z.w(K}$U�������<h�Z)H�c��~�S���1m_;
����06'_L�����:K�Y�����(��j%5����F��5T��mmi����P ��l�Q��%�;hY&��g��I�H��/R���vi����W'IU�D(����������fe���!���)��XoX��'K���A8���)��d�o�����
H2��q���u����oTO^���,�1��G�dK�6l�*�ufeM��MY7����@����(:jF���4�3C��
zbDmjR��+��q��Y��������z����A��"������O���v����H�e�,6����6�g.~�y;FQ��Q6��[�=4��\����"��L�O�z%��3r%	�f8���a��:r�b�#n-��D����d��.�6���������;��</���\��
�~8=���G
�]��e�����w�7�T�\F����������u�^@���2��h����K=�~{K'F0:4�-~�}��������u����������l���&�>u����D�>A`F������$n��Y�%�S�����&O����� ����Q�)�����)�z�_���ks.����}4uZb��{��������H��2X9�$��HfG���k^A(��I��M���j#�|c�m��!�����+�~[e���t�[zg������{�XIKM=�m��3����B�}.���7��>�V�����z��\�~���ET�:�����U���R�+���J��������5,�?�*v�y�.�Uo���3�@{�}�Q���6�z#O_����"�����d^?�E�W����u���G�LQ�T[V|���U���,l�?Ti�VN_��������+W�����}�6�e�s��Ot�PV{I���W_ l��-�e�x-��l�F�rWs�����m,*g)�:�$~�m�<�Lk-����^����q5�k���@IW����W�gAX��t;#��'UR��������&;4�*L��G��v��)�PU)
����-��I2J��NRj�y�tEJW�M���m6��1�eH�\���"r�#S]���������2��t�|Y������8)����>c+g2��@�Td�#����~3�~r���p�Z�g�wp������!�~\t�D�0NF���J�����)m�����M�SY�v+(��&	��X��)}$w�Q�f)�gw���%g�|�<��V���x�qS�]��}n�J�H|�]5�1k����X����hoW�r����l��}�����b��i�v�����ma��c�d3���1�5�����������[�6�~��MA*k�w��K���%%>����<|��?>��}����~j2���~�L-��7�N��V�������z���_��[�����<K����i��~So��������ilL�e�Wj|'>vo�������2�H����=�����b���(%��0�R��TB�����������������������3�}�]�@W�\:'��~�e���V��Z����Z-_��T����%�B�5
��}����Rh����^�@�:������(�i�
��X��?�e
��IC�����$xF�`��/���`�\@9���A�n��B���$m��.3�.�[�������������e��,t���U������4��~�Hk���Q/6A2��Vqx��j�-1d���)O���Ys�Rx3'��6�^m3d���w�w�]a�g�|�f��+�P�"[X�&��X��&���ZVI�hN��k�Ej�g2�}��lZ�,H�i*���3�"�t��C�L�)�q�1������a4P�+C��O��b��;v���S�[#F�
�|�&g�*����9��K9���\)�T��r5'��M��J�NwP���0#���~���3]�g��Z(���=�_|�]]�����a_YG���x������.���y_�_�3����Wm����h4%������y����/��{|�����Y�� J
���,�!e4�����1=2T���w��&B�#�+4R<i�Xw]��8*�#J�����;�Z�����]�5Dv�6�*wof(�.������5;�w���D����K�������R�nr����II�<U��b�����*��\��4'����q��
�+��c����>}��?v�q�q�fa'Y�g��f��e����� �@�+���3��OY>|��mOD���(�����`�F�N���*�
���7�j���u'����bWs�[��o�����ve���;��������^����9�XsW����&v�������,��p��(��6��.T��C 
 
 
 
 
n�,}�A0����]�to�`������%9������0�A��S�Vq��4+�n���n�}a�����|��Ms��������E�7�j�B��v�l�]-do�����u���������n���$�����c�������b�[���L�L�L�L��LkmiC�e��&��` ���� ����W���a���+�z�5������}���:ku�A`�I�������L�����1@|+�C�&5��6-kC/��c�8�$����?��������	���k7���z�^��zo�(,G��Y��J�����v�
����*�:@��Z�[�HR�~�~/kXc��|[:a�N�9U��ch����9�"�E3e��`f]h�����6����j���s1����6~�
�����hr��l5���6��<�6��G�}uY_�k^���X������N������
9|����(/���g���K8���E�r�c��C�o;$i&n=OZ#R"P������O��8UG�*,G��l1m_�3�u�O^u���a�li;���|Q�A�����c@���q)�F�� +�
��� ����+[JB�|_���)|m|v9������m��$@���IzO��X'���Z��RtI���5`
X_�-��\�UW��]@�t�3�{�=��[���2�(�����DQ@�<D]|��8�[J��	����4AKM���(N�NL3H
D++���w���X������{�:�P�QN����lqf��M��h����Gt���#H�o!� � � �>Ci$Q3�
p�$93�&� ������u�"����������3�<�x�������*��V�hZ������	���`+��Y���Z=[�._�V�l[���(���8�<�
����+�z5��,���AUPTUA��Q5��IN_������� �;�6e���]�Y�6k��_�B��,�R��2�B���j���QS�dKU�������JK��B�;��U;"<
�����S�����o;6i����~���@>����%���<����
w�����@/��^���n��,�s(t��X�y��)_OS7=[���G6�
��/��DNs��������+E�j;;M���xU�����<(�������>�����~����d�y<6���SF���E,��������Z�N�����T�;���v:-aR����q�#��������sX����$������'�N&�U����;��`�U���-���Q|��:���yt����������iAZ��K#��R�	v��`'�y���0	9�O�)G}��S}%�)�
�����p�Q��%�:�LT	���*�*�M0M�=������I����X�+�c5w�A)�,%�`����<�)Y�+�K��?9��Z�u����]�3z��L�����e��Y��z����>�V���.���u�?�T��}xbaRRRR��*w7��aG�+�-0^��a��(o�f�a�>I���dU�p,���]KG:HJk�'������q��B�-�*���P���gG�WO_<c��3n?>z-�!2 2 2 2 2.)2�7��:���m�/�x�� (
���'�ok��Jf�BX
�?Zy��0������v�4���������X��'�����
�z(���B�z�J���6R�0��o��r�f�TY�����F��aU�#�8I�)�\3����Y\����p~ �������'��Q���?K>f�uJ�M��W�����K�k�t���1���*�����i\��'F���B��P46
R�%��~b*���Q��D�y�j;R�j��W(�c^�P���!i�k���(~�|R>�7�L&�����]�H��p�~��Z�w�Ou�����,�)�7:����P%W*/�)}-�l6���*.��jyNW�|��������
���}���FZ����7z6�'+�#z0	�c�a�y�w'��;�u����n$�z�#�������k�6�&��N���v���J�y�p��5*�:��v���>l8H*}v���`L�i�"��0�����{sr
��?g{e_
��O�"�aq(���
�I{`��Q�IuJ*O�EN���j1�2��g���6<yFiV��T����v8(	������+�_���'gU���]w��'��F�9'2�`���=�?0��F�O�I�������d-S��G$��g���.)o���������"X��v��4f��F?R�mTW����JK!���)�g��>��JM����d���?kq^����Xtt5��$��p��U��	�D�&�_��������@3\�:~�������!����y���M-D\�P�AN����X,ccu��Z��w�"s�7��Y�+�rufmSZz�C���d�=��uT�)8�]n��ddyu�Q��D�3�q�m����j�,cq\hjd�~���Dl���!��L]�CL�y���w��y�YD[u�J�����h*�~��<.��Bf���8�Yo�Z]H�Si�,��*e�:�����8s�����TL�qYd'v���?CZJ��� �N}5��b|CJ;�jH�i�1I�8H�_��?�j��*l:������Q*���6���KJ7�"�x�g����r��5���nQ���I�J_�1��^�i�}�^���v�	�MX�Z+�L�R���������;��x�k����/���-�Q�E�Y����{��%����~��)k�����KgI�H=��%��|(�}���F��j���n<��&�i�����������%c����c���@����~�I�U��E^Z���l<�YM�e��������j��K����s�ig��Aw��=:�U�����Fa����b���4H�����LYn���I��g2��������Q���Vs���o{`�+��I�S���&qh��>�$�-;���U ���c=0���TV����'���� �c��`�3�">�K��"�SZ.U��{�\����7d-�NEcZ�	)�� �������D\�M�m�x6�C����I��V��h��
�F���$_������y��>u��|������B<utqJ}q*X����3m�l���U=WZ��5����z�1y�F�7l��e�S���dA��]�K,fN,�c^)ZVVqj��� ���<�K��]�A�����L�Y���������}����F��d���j�&��f�G���6�O7�y��T9�E���A����M�S��&��5�E"�S�pG,�z���Y����ux5e����((���~p]k���)^�h7���8�;�)�B7����g�_��c�LMn����/��� ���~|H]��}q������i�9��B���&��{�4��m'c�E����vlvYl)R��Xs�T;�}��+��k������ik�XC�X�0�P��f���?;�3n�/O��Fw�c�[f��.����ZV^��"8�����j���#��%[�V�l[/��N�,u�����h;�&�p���-p{I��W�����F�*�
����U���I,��e:��A)����2�Jk��i��I����������E_�K6�)�a���k����h���������.������[�h����`XV�`��6��j��n��GU�D;���`X��{�0L���������W��E��,tq��4��R���Y4�����T��������k>�`_'o�8<I�^T�Z%�R���J�:��g�c�V,�N��Q���j��mWIhq`�<O�!���qvjVd�mw���[g2����))��e�o���K���\�T6-����#�um�F`�LAX*�E���z&�4+8L�Rk�,hV��!7*���J���TR�	,���B��6:��LUA��������s��V$�����Y����P���h5��12��A��EVr�����9+i"�Ec.N.��P����m$���)W�d���2��,453��Jq8�;������8YY�9T�0������i�K��WJ<ID���aeZ�/�[L&h���kI5�:4�N0�b%nt�+�f�����2p}}o���<w�������c�����������b����V&.0$���F��LWg������B�PY?�	��qa��<&�0��O���{��k� 0v�+wO2j����#�q�{���E]���i���lN.Yq���z9�o�N�0��Z|��o���-Wp���;�at�0���&�I�:������	Mj�{T}[����/�S��N���K�$j��g���I���$��s��9��^����\&S}�X���\\�� �������,����?z���C�������X������7~`��z���V�I=&���^������wg(�_����R�)��I�>&���%*s(�P���v�9�� �[���������s_{m��{���kd��]i�n?��x��	5���{�r�{�O�DD�n�������
����
AAAAAq-Aay��$���`-X�n����6����/��G��`0��7bp?X�a�{%��`0|�k=�}��������}s�k����];L�]��Ee��V.:��v���F.�4�!�Yom�z�H���M9�������\bQP����eJ���J)���8 �A_�:�A)�������"���G���op7b�������w������ � � � � �n&��p�tX�
%�25�<X������������R�D��t%B�.$$$$$��%�������*�CN:������7�	����#��w��b�U:[ak<k�B�e��I�� uv��g;�F�k�f�<^�g�+�x�W���6�)��~���}8Woh��Yi����|����V\?:Y�Qq�����M:�6���f+yt�K7Ap����9����V�>�t+��n���X�.������n,qm`F���V��<��f�F[��4Ym�(�6���7�M|����
]�v����1�W�P8P�8"��R[�JpG��v���Js<k�.��aC�P����Z���-���]���ru^��s��6��[��i��L��S[��?����h�����C�������M��}��� (
���_A�����e����j����w���~�pk�0(
��_������x)�O�=l_��WyG��z����������Wa����Y������.4�?t�p�B�/+E����W ����P4��������Mz������K�����������waZ�z��[��_:�@0�@����]��Z���
���`�����`*�
���0�w0��=�b�������-`��u��`��l[�v�e�zZs��p��k� h��'�,��8Y�!�x������1p�����a.��H��0K�����D`1X��`��Xl�"(����[�s�\0�s7���J3��.��m�;�U�����.�{wu�������?���t�)5��"�w���=��>��g�.�uW�������l}Z_B��� �a@[���������< �_�nSo�0��LL55��X����?)ts`z��mA[���a��T�wC��r��E�y��X0�c����s�����,��9�����fZV�:q}����y�U,1�2� �`�����������@,�^����
de��,8������8�We���e�X0�co����lyf��K����r���3�0��[��������+�
�^v���@,��@���o�h����+�
�-��y2>�[�r���~|v4y��������� ,{v
���^?��{e�]����UZK��k�j�Z3���f�7]~ZxZ`����������#^M�>�<������US��IH�M�4�0m�|����Y�����������t���7;��T�f<��v�}����������������F�Ty~z�����qNh;�v��@���3x���+r�N���s;�������`_m���^���c>X��fB�g��~�>���t�tM��m��
5�����f���_�����P�Q�����*I[�h)���
{��o�o	k��9A���"`EHJ���2n��C_bh~��������R{�I����ly���Qe��lI]W��9P��V���s B�{�����=6�*(U�S�c5����8�i�)�EP�K� ����";��h������ �v��-���Tg�����nj���������P��>G	�
�0a(�P��4E��}
��r�I
 �2� o�-��?��	i�gi��SH��d H�%y�g��@2�$�@�� y{��1��*q�|����
6�����/��q�����z����n!����/��y��w��������C��x�x�i�
���V�����@��s�se$�4+A����&>N�YLY�J�f����X�u���V���j��J���[��i^�f�fA�pF*K��z�"8��I�T�Z��uZ��.������l����=�[���������������f���p�(��b��V�Cj�����a0���[gp�P�<z0��`�m3�kR/Mx����/���~; m����}A_����m��������b�QPl6�-nr�V���^�]�,��R�T���1�����F��� 5H
R�����G�z����eux��� .��n����RE��c�wG�\�?*�2H&L�����7��[��r��hG :���� �-���}��eV�Lz��?8����~�Bt����CUk��;X��j���8��5n�l���	i0�3�0_S��y����@-P�q�U
�P�e@�S�2������}�z�^���e�v�E����|���bF.0����������a7��@��,?��.��~��(�j@(
��_B;�e�Y\�p���0 HR��/������B�;�Nwd@�o�p���� .�����tt���3���Bg=�; :������p�P�5@�	�Nwf�`�?10o��N|$��@-P��G����m���� 3�2t2��q+((!�������@�-�%��y�mU��}�K��	HR�T ��C��^��I��r7}b�`X���F���K��.�\\��{A!aAX�����O��>�X|�k�{{�[jO��*��R�t�"]�b�Z-����B���4N�U�Ua�*+�]�`�3q���e��*�)�gqL��U��<��HM���T(���k�E��<���x�t��C�T����m�l���4��W�	����.�
56�������[���4�3`�?�(�3y��h5����m�8Uu>��=iE����RV�=+���'���	^V�$��UB�c�D�on���Y�>T8�1111x�bp��=��������& 3�2�� ���LZ���T���OI�VQP�Y���qz�����%#���K���Aj���A��C��zB[��`0�`���Q��q������`/�{���eo��{�^���u����yW������0(
�����)��
���0(
o���V�`X���u������;��[J��w�6e�'R�����c*�����{�j����1e����&�	��C7r���Ui�������"�J>��N�Q�2SAXV��H��������.�~����3;q������(N�Q���KQ�cK�N�lw�����)�R�J��G{��8�$Y�0�'��L��fct�>�$��+=��VAB�H�2>�����96�B���B�s�CR*n�vS���*��~��8������;T(�a8�AN�~�����
�YL�l�p��L�%/�!$..�J�SO�,�L�R������7��SGi��4�I\���� �5����9���H)e�}{��TLcE������J�w�T��f%/��+����.�����v$��������������L��/Z#Ce���f��V����]�D���@�?�+���x�����HPv��A�N�m�L�sI��SAd�NQ#'nL����y��IP�������0���R��l�s�3��y�����R��������F��&�s��s���IT���1v��YP�O)*
�V�e���G�2Ec!Jt'7_��R�� )�3V��I���R��:����������i���o���%��������J��[��P��x��jG�+�`����ALCu�P'	!!��<�ta�
�����=|o�4���!��^RT�g��������|���8����C�|KQG�����O��o�`P{����0�����m��8���m{�o�������H�Ce]�"��`��/���9p�o�5��s=�
� �b���@e�4��_(��k7a��	0����`�����w��:,Ln��b� �o
���sV����I��

���0(
o������w��pT
���0(
��[���[���/�����C�5w�g���Ab�$�o�@���r��%��`0|�Fj���K0��������4�Ac�4�]����%h�������j�p��(qG1
��hb=t��EP��=���O$�������������y�����H^j�
mB�T�����N���nK���EV��1�-�
���x0SS�dK����G�Ui).����������Qp$�O�����P�W���b�i���.ddddd��%���~���[�
%�25�<X�������������,Z��c�
R-c��:�,
�8T��g��"m ��"XL�eDDDDD��E�u��8�����<>cY����*#�q�"�y��lx�bB��	l��*}�����X��7po�I����������!c'�;��~����A�;�������d���0������A�M�F���*�����!��c]N��Gw���=G��X>O{b�Y�_���R�{���
��� *�z����0	b�8��\_`��S}�J,���@����^V~��l��P"��i�T�VQl�i��;+��I����X�+�MV���a
l��q*�����r0St2�������9��.��
�L��#�5����Z/������J������r������6��4������5����������-O�E^�.����-xf3y�mn����p��%��r�v%+<Yy�K�_����A2��3�wA��>��"""""�Z"�O��Y�q���U���,��f�h�o	�m8�]�����o�1@��
�������1`���-�|���|�=P���x��R�~&;�c��5k9�_�D�F���2UU�����J.�����X�I�L��J�����EP�smT0+�����)�����u*����Q���-��z5��SQ:H�Zje�I�O)�T�H���5
�E�~��>�%w��3T:�n����=^���5�1�A��X�Wo�U���������&�p��99999�������~E^��/�����_��,���������qA\w��%�)'�N�����C�rd���>�=k�
�j���R�������o���sMfN3��Wwu�Q���bb�����m��;�R�#��P�����������v>N1;���
;(���H �!�!�!�!������?t��</�w��(��^��1������m�H���v{"n�%+9�5�!C����� m<GY��bY�zvt��'e�"H�84~O��O�����FlS:Vm�7��7����������������}��!���O������P:��'��*�x�hQ�R��($.Vv�\��.#Y��3:P��=r����3��I]�E��T���Z����f�� (�~���� ���f��J����[��.\pc+�V�)]�f����\���p�����JF��W������}^�j�U}_��W�1����\��>N�4,���i�T����X}5Vb`�`�4+u����K�\r�=.���������p��UJ/OmE}E-���nc�R�C�����]������������v�uT�"���}�`�g1�<�o��{��k�5�z$W��]J�)�~�S����n]H�<��$���@���"�E]��=x���k>���(=������������n=l������A����'�lF%�Oy��jZ/������6+���p�6�����F3oT*��'Aj�T��>��5<*���xN�KI�K��q��|B98*md��j;v�%4�h|��jAj[�,-c�=���8+��xk77�l���}=�&=��������<
��m��<�P�V��Y����+��?c�.	�tB���	�s����z,�>�e���_��W��PWZ��HV@F@F@F@F�/������d�M_3yfL<��*�t��x'�~���b�p\��epy�����M�u��-����)x
�~�<}/a����$Qq:KX�=
�8���b4��h�y�;�dB��y
��_����x�Q}p�wN����_��m���@���V|��8g��6w�N���u^�(� ���f�]�<���������Gj7�;�i�r]�,X�h����*�+�E���1��X�E���A}�<Z�`��P HUV��qP^���!��w�!/�i0ec�T��e�����9�����M����.o d����9�����)Z��Je���������\E�T'?j7����s�g1ozK������/�������m�<T�m���5T-U���4(���QIy�d*=j/t��|�|�|�|�|�U�E()I��m���IV�����"+���F'��D4�y5�H�$s,�so�5�WD:��O�`�K��C$+/}�$Hw���C�d]8�0������/�V���
�gP�X����z��?�6�o)Rcvmd���������4���X�8
5�YJ���K�V�Z,����x$��^@'��@*N�0�g���h���q��,����d�*#E�
~���ZR�����\��$eG�����JY+K1�:�6������t�v��|��a�W$n��)*���>����bm�~]��Hq��*
�����t��Q3q=�S���vc�X���4�:#�>��jw�$@I��%J�����E�^
l`1X�������w�c���x-������Z�;p~��qp�C�����E'��q����r���7/=�G*]y�X1� �$��P�\��0VU�hcw�*#{U�Nx���6U���y��G8��p�W���Q�E����F�������������<������u����T���v�(���SX�m4��V{��g�}~�����rT#�������G*�����b��]���~7���U��t�k�Uo�����������8��e���l(��~E9�`���H�{rT��n���}���8Ne/��@oY�i���e�UVY#��x��x����r�v�����^���}�j��;sYd�1o�SF��y���qd	����.N������������1��KI[�����<d\[�� =4��[u��f��v���NI��������;�����}�p����L�{�c��K��V4L"�<tNsR�pC1��q1���H�v:�$�f2~�O�1.����H'���o�l�|q$�k_2�xL����Cu���5����
J�(P:�V:��������++m��4?YA3j����Ks�&������'?#�����:�3����l�]���[�?7u��e������,�C�����\]������jM�/q6�5�'�M
�+`,g	��71�n$�id`�GE�?8K�)���4;���T���L����+9��&�V�W�M�
��Y��ea�J.�������>��'�r�����h���@�9U5�!�=A��kB�{KTG�v����8��e��*���@����*���<�W�d�*��}��zF�c��z�^�w����5����J>^U�?Z��t~���`/��nV�����B/��������u=��j�Z����`��n�
���3�:������,;�X�.�YF���4d*i$v�n;j���w{G���#�@6�
d��9U�Z:������:����P�V���4��#��{��[Yx��s`N_��)��NDnq�*���"6Yj�������\�y���->���2���{������P�|;���#]����{����F��R-���Q���`�Sz}�L����x�dS��Q�9�����w�����g���E�I���g��`0��B�H����#u��s7�\���F>�,��������]��_����N�w_�4�]��z�^���<z�D����.� b� �?�O���ZV�E	m{�Kv�[���aE^d�=��l��]�p1����vx��z��;��(fEA��/�8�_6���[��k��y�����:��bp�_��/������qh�����D��ap�D-���6�<�����r%�*��?>9����g��~��'����b&[��H��lB��W3x��:j0�q��m���{����V���X�t�t�t�t���h�J�a_�����N	 �2� ���i�{����#�a@�o�Q�������D�P�����T%�*��X�7����?�3��a�����9����S����@;�~�h������}���}\���j�z�:�o;���j���a`�7�/�Q��J��c����r
��1po��V�`X�
/7����#9U�XY�N���hW-�r�� ����6�����f�/8�+J�J'O��V{��LG������g�,S{E��c��3w�b��X�$���z��X�>��s"X�_D�!��EuD�A�/���mo
�Y��p{�yl�':I�{IH(���4.W*��RV6�c	y�Wt��%�L7�\�i��Y�z���~W�,�B��Ck\�}�gn������'����:�V��YV���Q����GN�L�������]�&�%(U�j4 ,vU2�S��;�����-h�����.
n�e�,X��JZ+�����L��w����-x��������m�%�x���V�I`��#N�4 /���� ���k�"(����[_l[��lo��J3��>o[��k��.�{}����TY[
�<O�S���<}oM���\�U�9���*��,����$[Kb��z�^����,�i9�"1�rF��;?�
���+�z#�w�����{��[PT�AeP�FT��~��]lr�K���U�r�h�'&x��e�Se����OI��'���8����P��Fs.N����W����3Y� )`OQO�w%Nb@@@@\Wl_p�-"�����l��d�A`�.�����
.����,8���7�l����]@�t�B�9�6������,8�����sogN���-h���7�m�r�p��	���`,�n�������U_�W�|_o���U/!���eAYP��ek5��aAX�����xh�V�91v4t{l������������;|R�9����`,�^�������)p����������T��e$�03�d�9����,t/�4(VJ�2^�V����5%�(��������k���(��*O����,�B��l���iV�Q����r;F��[�� �8�2+�D�h�z6�@�A�U��D�m��YZ;��<3:������i�&�taO�U&N�%�HI��X�(�^c�e�$jJu�D��lZt�s�g��4m��~�������[��I0��u�\j��P��*
�PT��H7�V|*U����$/t^h���PW�E�l6��7��fp��z�s�{��8�e��A���|�A���P��5'�7�J����4g%-��
���:e�������������M�V�)���p��u+jv�����[� �)����
����O^�\�=���T.@P��O��������k��_57i
A�t�U�s.?�NJ��h�v����)���������D�����on�\� 7<[�l���\6�@j�3:t��(W�{3����6��6z�,+v�����������C~��k�?0���l����D���E.����
/U�37S��r�����[;<�x~b�$	R�UF�*d��S�&?�<|o�4���A"�^ZW�(�g}}���?�DoZu����9����5�?]�,~�R�>GQ�
�"�E(�P�0e�:9��f!�E�w����k����KV���3vW�2ng�7��������������[[mq���s�@yP��A�KS�2��R�|_�W�u�|��j����F���h�+��������.��I�6k���p�w�]pws���rkk+����0�I�M9Itz\�G�;������/����W���
Oti�/e8��Ks���Exj\�2o�e����l�e�=������=�4�As�|4�w�z�mL�>P�@1P|�(�+��E��`.����`����[jO=q�a��+9W$�[�	���3gQ�bJ��r��T��D�7Z2
�Aq�)�Va^M���e���,+��9�&������@������Q�'��v���56�n>*SdK>=�Y����{�<����>%E���;W�/.�H��������r�Sw�������EVhi�D�����������Y:H���e�Y�D�5J9�V������?=�N���xmS|�n�#�9��+���kr�J��F��+��e�KjO�Y�U`��,��NK�����h��TS�n��w!}!}!}!}!}o.}���T�v_7�+�
����^������z&��1p���&���K��#0��Sv:��;V�g���^~�%#����o�]����Z�X�2X��+�8*��x�j=�8�C�I�
�W��J2��c��?}3y��O���x�b���OO���� ��dlW[�m�����b%KM��4I��'��^�*zKM\2*����?��-dh�n`�qv%ds�|�?A(>lnT%��S-�6I���D4V��>��Z����;���>~F��x}����������������UP�?Z��sp\�p7�����-�y���	X�������;��=�Zc������y6�}�X����������-�)�%}�$���5�m��<�*^�I��Z�u!*I���^��zqm����2��	%����s$^����m�[�5��t�5�}��nP?������o�4������o+�����}��25+���)w)�d���L��<��Mm��*���#$��Q����z�mf�?�?�?��M���H|u|�0����^~��=j�'$;��>UH
nu]A�g�_N�f����5���8s���^.M{���r�*�]�����\����^��h��uv$�$�K�p�\���$2"�jW���"�8:				�i	�bn��^��D{/_��|�M����W�)��qt��+�C1zr�)2�8��3�|K�q��A�m�/o�[��o7�[��iy������-x���7��{�0LS���0u���ko)�C'{'b�5PS9�uv�������;���36*HL�Lp�#��g����e>&V>�x=;H�<r�c��l�����I`.�uDd�5���c��U�+�P�rk�(fE�P�q�7�{���"t�a-���~�������a��.��xA��j�0n��U�o�ec+@��C���/"
���aI�p|�������+7_��|�M����]p�w��w�]p�K��������
��fo;z�^��z7�^���y9Tgx�yA^��y7G������ .���� �&�����	�����.������f�[��<�
����.��nnu��u�����no��
���i���;&���6�\�&� �
���q%������+����O�re�&���W=tw��
����|��D6����P/��D���	�2��������Hbs���Ah�)��`1����))))�	)��FZg/w���y|F`h!����5�<C��oMB�4�:�l_mo���+�K��
��n*    �� J��F+��� ���	�Dw������$���W�t����Cs�AiP�����gV�6�OcC�>T��?{o����$��M�
\��1�P����n��������s������X�H"�F4��{��0$%?I��t*e����n4�&�7�~����K4fF�eZr��E��
s���������%��BQ��PT(�*���B/�hA�����jS}eU0+��
f_	���{o��O�nQ:+����z=)b6p[�{Fe
�������w�&�Y*gE����K��)�������5�3��Z�Me���ja����=����2��U��/o��Y�����+��'LL8������LI�N�,bR�!C$�H�+Q�����������S�/�_��v�K2$W�����8��5�N�I
&P�������_�:�&������D��OT:RW��+�����;�������I��}�[���\h.4�_)�����	�Z����p��V:��Z�������\Nm�l��g����LA	���uS���~�wP�;�����Z�K���J�%��s���w�Z����E�T�13�7�9T��P���xv[��F�051��,T��N-����-��^@��
�f��TV�`���R�b
�b���Kh�����dXC������S����F�
�J���4�&��fh&h�b�W��<Z���6d1�bW�c�U����v����N��a�v�~��1���X�D��1*b�*,]�<�q��4��R7��y�-�~���������������s3�����*==��G�:�uMOz��T]�fY��h�"�ED��l�l�]+��
`�W�������yV�:�&/&��`V0+�����`���P�,��S���	���j[Lm���t�v/*g�e���c���Xp,8~�CS����������s��]��A>Db�f�\�t���Y]LK����
z��fr�W�a�2��l%��G�C���-3�/
���������zaTJ��ME;��������e��7W�*t53���T����[�X�^���ov��{)����J)�
��Vfih�p��^zl1�e�������p_v��c����x�])xx����xv"�#y"�D����w%"�S���M�G��x�p�\��l>xb���9j��@�IR�Y'������w(���hA� Z�� ����{�+��
b�bY
F=�t����Z����A28S �����=����,0���;0�4'��[nl�����pW�+��
w�������_?�e���'�tmk��?����,������/�
���%��
{���^a������G�=�M.u@.������N`,0�/�\�||����o�,G�~c�0\.����WN+��
b�����X`,0�_�+���+s��
v��]���������<�6����('�ea�0_�/�~r]�z��^A���*��R\��tVw7�Jx8!��W�+��"�w;�F$j�W�+��^	p%7��vc��z+��
m��W�����nA��VP+��^j_���R�u2��t��"��Om������V�.re�g����:/��rB������.�xBGcU�������#y��E�V�S�S��*u��f��Sz�,�TLn��f�9��>����&�����jAeni�Z�0�Zy)������zf���Z�eaC�Mt������A���u���,�M��=��RW:��~�u��uQ����t�����p����h%V�WX�G|�"���n�tw���$�(�WU���=�u�����%�O�����aV��r\����0l�C�W��u�����'rR���I��������C�a0-b8���y����]��@W�+��
����-L�/I�A�V+���z�_C�$���N��r�������F�e���h��9��%�{�&��������l�]�XE[�9����\�O�l:}�_F��X�H�1�}� 2@d��������e��/[:~���~v�������c�����������;���_���^a��W�{��]�j�-��6�	���_����
���fu������8��p�6��^u�����'����;Q���pX8,���������w�L�+��
q��WE\�<T��.�����~��IBW���U�*t���o[$M
)c��������������o�r;t�7���/o�sWO
3�:�t�0��z1��)L��
�x��g�O+���9����Ox�Q+�r�J����?�i{4�2#<+��T,@F�+�����>���S��?���jxD��=�r]*��!u���s��U�����{�(4������I��x<��t����%���!�2�\]��>��~��"nx����s�B]*���)�5:0��N����s[fts��,+7���G�����]�RS�*V���W5�jB�)���[����<�z2�;kor����AU�~,�F�w�e����1�?@�������1�S���N/��b�������:��r�*���>00�|);]��4VzC��{�4�&�,,7�un���pzSl����{���;j�����O�G�wx�����;���./�V�v�v���Q��+����;��e���������p���������'����B�G�X��W���4�muRvh�,t��o*T@G������+���%D.������}n����-�B#��1t�F�m�������\E��1P
�6�@[d?��ut���W�B�M�gM8z� 6����\b�.����b�X<�o^��@S'�Y4"����P��r
�~p98%3�w��j��{�g<M�<������VeXWxte�u���E��^�+=���R�k���9�s)+��W�R��9�^���S��}��>7�zT`�}�a��
�-�,: �|���jQ�`�\�G>�f5"�q�>��%�wPm�����`g��zoS�����4|y�s|�KK�Q?�D{����#��6������q�Ru�1wv���=���2���TSxsI=.��t`��X��:�C���bb�fLZK]Y�>�6jyd�WRk�R�]�
������b5#�g��s������7)�
.��:�AG����������\��M������5�7~��Y��3�afPL�>R��boA���f"�f���S��~��Ue�{�C�0��"*V����s�8<���jb.�ln4�*b��Z#_�����86,Zthi��z��4��)6M%6M�i����k����b!�X,�%�_�y��Q,����y^��:�R/��{��=�����.M����>���V��� ��LL�g��j'KH�;����E���������B��f#��o��C��0R�v,���7�`A�"0D`l���H�h��i�jz:��r�l������b'��sH�lY7 �=����s7MVzB:�-n.���Ka�sp�<8�'��GE�������x��-���0t�.��d�X�56��4��M��M��Pr�Y���#�G"��B�i��w6�gjwD
��@X ,_���} ���F�+��
w��Wb��y�Ba}h=!�(�R��<�]�u��	#�l�wLf<�Q�|����Y����4�-Cp.8���S�~9��J�����2��g����
t�-����;��Q�+��
z�W�o�7�w�6��>SI����`A� ���T`*0�
L��{{7��k���z�,t	���v�A���3�������{�D
1^ei^�w`!�.���T��"W���:��^���n�N�Q�{���<�1jKf�86���Sy����t������|=�p]+�uF5]p�"�B7������
��KS��S���Nj�A�f��<��mJ��p^8/���q�������4�hf��z�5HV�|���2>�����36L��F�Ja��RX��g����V9u���bY�����Qp?���w���?����������c��x�7�j?�1�e},��	�>�u�^s��y@!�bP�@q�S~ �/5�
�c
�����`G���`8��H�v�V�d��(��X��c�
\5nB���m���U��p:�H���Rb������� _Vf[W��( ������gL~�,.?La��ll���ssQ 
��0��v��zYa0WB�n����?[����F�����1����������RUc�N��H5�������M��T��=���
��������(��WM�"x7�
�m�&�U�����OI+*
ZN�&�w�eE���?Pz��LA���������,����N�5�C9����Ogx\����^'�Z��Y�����n����m��Hw������������
7��(s�\����&
I'����A���9��d�K���c3&I��7�kOY���v���l��=9���-���K�.E9�H�#Q�~�Q������ly�,�xy��'O�=?���G'���E�pO�'���7 57IW�&�(�S@��\��M��S������TN��������f�������V������7Z��f�EL)�.���G�����%4����aU~!�N�GPA� O��Q!/)�1�oJ`��[�o#=�%z��#��A_�<N��e;�:��@��d��B4��9�u�.FW�����������%������_t�V�!���&m���j�>.o#�L{v6��e�*J����|�me����G�ECi��[.����M�w�p���*��k����y�^J������F�����xO��pYf��k�![A�ps:��]G�H���v��gY��{]�.NO�O?� �A����A8<7�w?&��������b�	�y�.t\��]�v��
��%�J
%?>J����$��
���`O�'��}�k�~��a��M�]�mO�'��	�~����)(�	�e�?���P��S
C�{nJ������M���a>j����)��{f����TT�\��w9������
�C�=V�!n�����UX/
���p���F���9|�a+;��.�����*q���0`e���>�$���T��.�������sV�Uart��`dL�<�S�F�f����	u��B�z�Z����g,>�6@C��[�����@,����m��{9i�.Vz�A?�����h?F�|��q����1�_VW�)�h�`��w�.7�2�����xX���X51t�9�o^�+������t6�+����
�~�T���G��$rx�������1�MVC�$C��n�x��:�zD����F���;\�r���.�|���2�.
?��6�(7iW�QVK������/�W�bU�5���tI��n��/%f��
�+����
��t%���V����i�������
����4��.�������Zx-�^�_��x�#���g��q��|{��}���t�O����+)P�$��
�,@��e`����w��M��\�Q�K���,q=@��(Q<v��"XJ=u��5��)�WQl:'S���	N��ai�gV$P@\��o�
��GTJU����E�HL��$�vI@uG�:���KP�������t�L*����7�Q�����v2��T�g*�0��r�g�r��5XtC'$��Lj[�M��XH��jy]q�5�x����=bx-�O��?�[��1=���;��m�M���
q�E��*�UD�����h��'q���{��������;q,	��[&���D�*D�
Q�0D}�Y`)�X
,�0��0�y�FmBJ�-G����.T��S����X$^���M���!D"���"Gs��`���|�������?}���C�Q����E����f��@���s�����b��4��af���S���G��d��!��!���.F�.����o-(h�)]=�7�@a�v��Bt`q���;��7�x�	'��;rWC�
�d��w�J���c3l��=���d�k8���y��s��+g�f�/�b��������xD6�kqW�o���e��=	�w@=A9!�qc*s�y2�
��i���3�Efl��wS}���?����*m~vGy��oJ=�%p_\nX�X����udy;�S����~
����5:�Qd�����8�F�Rxf4��`�S�	�\�p�/+���S`�H_��Z���c�B��, gB��Q�������<0��qA*���w[��d�?��~]f����?���G�2������1
	�

���1����S��-�����#�������)��:��Pt@�E|;�k*.�]��`W�+�};����OP+��
j�o��W5�	o���[������^�!��
d���Sj��B~!��_�/���3���
e��BY��kQ��cBT!�U�*D}
����/	F�*�d���^a��W���}���[���V`+�}+���@`*0�
L� E�V�*��pT8*��:G{)�0$��Sj���X����M����:p����}����
]{sD7b�)[b�O11�X���	�0�gK:�w���Ua9������t]���2�s�$�5W=1>T6�	��{�Z��+t���!���
�Qy[f�y�[��
�Dm��T\�&*MV�P>T,���4���K
�z�����eq&,�z��4S������=�y������s�P���z��C���@]�$
"
$�_������_/�������[ �I��E�"?B������q��`��>� �����:BJ���u��G`-�X��om	J������F��c%�_���W�+��b�.M��p�`�X|�|<��S��2c�R���s�h�N�wu�����R�������{������Qx��Br��p��J2[�����Li�`(��`��bj+��e1?���ba��XX|1�6v�V�83���O��C`,0�������O]3��W�+��
~������
d�Y��[v���:Tr/&t�F���,�����������P�[�M����
9k"@��Y9["p8����-gK��_�������-���w�'�p�ba��XX��Y,�p�^a��W����+�pY��@V {����A�Q(�M��hw`w5�dm��%���d�d���lA��:���"���;���G����]��pW�{��*��TS]x�eN���4����U��[5no�v�T!���%�k��5����1;���M��YOeT��47��=h���O:(���U��x���
.%���2��s�����aW��PWf�H�N�������c���1�h�P��	#*��J���qm��E��������,�N��!//���/BM��5j"��S�a��8:�,�uG
�_�����O��@Y�,P�F(�Q/&�m��Q���[�����a��5rx@��EG�������A�9�W����Bl!�[�}���bX�����`a��f��"
���B�whC&o��e<�8��z�$:s�Te���F�28��X������'�����deb������F���\7�
���7���5�U\1`�! B@��5
����:�iRn`y�>m��1F� c~�4�A@���W�8Sx��}���d� �A$�H��YDG�����qI�����M�]������'F����Wkrgg���7zt�^i�'����e���8Y&�0a �@��5
�Att�d��7�<7Y�+U^�g
��o�~vG]��}n*M�����m�c.�KxIq/��M�J�]aY����#rG�������=zInI����g���Y�|�p�s�w�B��
��Y�,p8���7T����g}W�ln��2%i.
X#\.�����+���6�:]����X��O�2#�:�o�%m���9�.Gje����Q�.*��`�l�y�Z�m��Z����eG��0���,&���_��vY�c|.���lY�S�H���\t����J����{���*U)W�����G���l0dX,�����o8��'#P�G��],��FjR�U�0�XXG9VBE�G
�n��y:wu�c�e�3���`hF-��vb��M���*f��j��8�v������(�}�&zR�1�q8�z�}���+=t����P=�Un����Np4x���=��,�
Fa��nk����Kp�&�&�,*��@h���i[���x���2xO�6sx)����7�GJ=�����C���������+��TVua�C��B�l�co��+sH�j*
Z�u�����s��z_����Bsle���t���pC��)o�!�(�(*��8LM�VP��*�9��8Yx�P��x����>j1��J��fLL/O�J��1�]�����2]��zu6��x�]YOn���[�CCp�oP�
���������x�y�Co�c*	#�A�*(�����R�[�xN����n���\��:+�D^�]��nH��,�z�k������#<_VhO4C�E3��Z5��^bQ����`a�u���L�nM���p����t��.�l�}��7W�bCJ���B�����=:F�����K2V��K ,����*������M�!�X,�k��Eq2��:���W�+��v���>����J�CUgA�_j{���!�s�
����=3�z�M�%���C6^4Ff�#���pA�u"\�-�`���q�����&Y�-��Bi��P��Sz����L��WUz=/��,4����Ic�;�QD��H`{N�9J�"0����|�QO\�\�xz���pe
�0e�S�SW>�X��T�	�Z`����yK���Q�����h���������~�Q91)��<����hxJJ�)se>������o��_�9�(�uk��"�����K�eW���y���G���c��qa�Y��~���:���5q�bpo6��T��D}@�����O���
�,�jL�+��
}��j�_
��r��=#iHa�8\�4or���aP��c�Gi3�S�o�BQ8�nX"��s}�u�{���b�>�'5�)��Q�����*���trS.���,�5����L�9�b+y���2�SaQ(�.�gp9�T^Vb?���7W�M�X6�C�LX��� ������A��(�����!���_��NmF��H��������.��w�.h{�1�by#\@qK�����C�"�G�}����fV�L��6���4��u����3Uf~����H}��"�E����v����:�.��h�����p�c�c;�����K������%�w�\W�)���gs��u��)����Q47Q�H��Lx���O�o�jwx�o�{G�;	D��\��������k�/������
���:\X����5���������-�I�x������U~��*rX���a���Do�W�Vo������uG��F���>�?�����w�6��v����RfW��S���>�����[�w�"�D0�`�$��]��)@���u��xLT{_/F'�^��P6������C[����ABm��J�G�����Z���BO������t3)�GYE}y��6��S[�0v ��l�Jx
�6���f���_(��9��nq��-���%��+V"�DX�k�-,qj�0����Ba��PX(�.)���BT`���W�+�}���
�g���V`a�0X,~�>5��!d�R,���q����������`��
rE��w(/�|�8]Hg����c�1
��o'+����|2���A�!BC���w&4�������������9T��0����t�:�Q�L����|�|���"�]�HA��[�'����i=yd�3��p_�/�������zU�:�����������[���{���o
7>x������d�=�eK��Lw�zr�b���=�A|�� @��w b�tQ��?[��q_�Q [���`|��KH����?�w��2���5�"F�c]�0�H������x���FT�g���[�����EE.���������2w�N+� ���1G�mL8(nR�P
���7���M���n��q��������b���:�#�P���.i9��b�m���]m�W��)tK]��B7��a�nx�r�����Q�3/
�r���"���C"�E��."����jbC�+B:�St�A�g�����b�- ��������TA���"����t��F�k�d����\V�z6�[�I��(��J�R��ec��2�Q��Z1�<4E��t����Q�'��������=������Jg�j��4��!�B��ob���C���d�������~���%�P��HC���rA�Gp�S�@@����s5����p)nl�Li*]DQ�.��l�jz�+KS�pS��x?/Q��|��t�������)�e(,�	����l\���X�����o�Y�gn�������qe@��<�QZ�ly>^b���o�8�1
��C�N�(�S
a=�g���
���]=)�!-/�fC5P��%a�M�� ����:�X��J�
��a^���S/�T�+�����������Q@����W��.KZ��e��pI����tdAh�(iN�y�4���e�sK��;������q��I�+;U����J�*!�������Q���qa�.����e�����������Fh��p�}��[j9���� R�^u�PX����1���Z�st�a�OW�,%�7�����s���[�B;)
G|���!�Y�����,�J���*���564%����i����r�G��|��v��Je�6�e�������
h�	�s�.�c��^��kp+��fZXZ2[�m�����k��bY�r�(E��Z�
���*�3�22l��bI�-L���>H=���W�/�4��/�u�/��X�����_���_�8y��}���{>���������{�%�0V�:��#xi��e�o�#D����/�_D���~���0���0gST�1��X����O�=;bt��=
6��!�_��p[�-��NnG���]�,�xmtHk��@/�N+t�k��G����������#���v>�����A|�	�e��@X ,��A���h�����qxw���\)�U��@U��e��1��F~1U�����U��pU��������p�%T���b�.����������@V +��^��6�0�#cov��`�,�x���*����S����Z�x|o��JS���Z=}���H�;}��>�q\����]���0��d���$WE��I	%���c%���sK*�&�L��G����*m9���n�[y<f��+�o���9z7B��%�[2�������:bWH��^�_��x�!��a�������#`�����d*�9Y�WU��)��\�*���H�|��J�[e@7L3��C���&�z|m��i���L�Q���IO[`��I�{C�������*/�o���������a��#3O��xw�6���y��$��k���/x�f�=y������+J���/�����|��z�Y����q�Qu���������sj<�C���Ti��_�����������������t�C�,}�I�j<J-:�v'�������)�����b���o�c�i���(�����x\�]q��:�e>&T	�N����]N�����)���f��+��^��]��J\��Zm���F���9IwA)C���,�Y:�������j$D��_��k��{N�i��.�=
��������Wit�|�����`�O���{�}{�Kx_Z�
�k�S��xn�Z�LaR��_�����3�~�Q��U�1W~�RVml!���e�����6�
"�K��2�����JD]r� �q�������	����0w��%�=�3�p3��Kx�L������������h*��_z�M>��&q:+����+��hc��K�g?������}8<�#R>��N�SY�+�(���q8���9w�9����<y:~���O���1{������y��a�;����	��8�;��NO�� }�)�-���[���#��/�������$�D��I��g���+F����s�c��j��b�!�������_����+H����w��T����k��>�����~u�#��������V������i�BK<��	k	L�4G��.�[5��:�@�S��t"M����������[��t��v�y�;���r�+�@���(t���!u�+vuGtq�����|�=��SWw+���0���*��I8�������l�*�����~���p�����[^0NNO<��j{��� 4a�*Tk
&�@�;c��2��
���W���C�*�~��|A������Z��%:i�[�|�i���r<�8����)�wQ��o�B}�C�ls
cW|����~�`K����v^NT��=:�^}������ 3i�W����)`�]��-��qq���ZS-+��R��J7�.HC9D�.���3�T�.�l��u�0�3���
��P���uR��P<F����!j}�~�����*o�����5D��������P�J��Re����a�T����kLj�W�("�[b�b�%}L]p3v -��S�(���(#�e2�4�^}�E|��hc<!L/��?0�G�i���,����[����$k3��)L45�h���L�ZRt�d��!
�
ks�V��y)�uP��wm1q8�U��q*������5)E���/x�C�VhIfA`e�u�z�}�����z���#
C�������N�������5���x��h�1�:C}�����]��c�b����>����2W�'������w���=*-�D��&&�������d��ANh��%�(�L�cJDC�;n��p�2��-�G��'i#���KD��������)���AJ��lX�%o����Y�S�����xe�i�6�.�~o����Key4���;�3z�Y�\f��`�\ZH����n��DO

�������Eg�8W�����)�(����I9�?!#+�l�@:����?�@n�2|��[�p��GC�[}�6}��*����������n_�e>�js�,�*:�86���GU,FvrML��_�b����hM��;��V�-�>��������t��*[��
R}e3fTM���8��Yq4���������0�*����B7��<��2�����v1
ni������G�IaN�7k�Em@��A���k��p{���e�N�zm���C��q���
�#':�pg�/��Ve��$go_Nl��$�y����%i��ef8��V�b,�(M��h��;����'��d-w��s�@tP��O��I\,^L�y���s	#�t��>.|�f���Q�lei:�	
� 7S]�6����X!i��tK�Z ��rgh�V5�����������(���)O�o�(i��f&{,������^�7�)/Z����zaK������l��]a�zZ4yoHk���q��1A#E]6)a�P�dI�4�A��L���t2�(�����1^2Q&k����}6p���A����@e����K!?\$�M'������6KtF����������3��>�Fj'�{aAvc�5�4���0����b,e_��{�F��
[?��&�rH��m*}-����=��H���L��q��qa�@6^�j����B��_WPXg�qer��c�^����������������(n/��4Y��H�����$=��x����K�^����e6 ��u�_H�C]����Z{\�d�Z?��|�/e���@S�N���e�����I��h���2C�� lf�� ��t�PAQ���^�4?��H����7���M�MF5�z�`z?��'�M�X=���B�3J��j�/2���L�U\�GAGn������G)����������4%��fWvjguEb����N���:����S�AW�.W<f�l�������dM�����,4����L�2v*N1��BA��w�������)�
�nE��;��0G�F��;��OL s�<E��/�CF�;9GS�{8�/�t�t��ck%��C��M�a�_��+��[�K�h0�q����@ ���8���W{Q/E�H���$~�6�i��c����T&�QyzgO�k3�
�[Yc�VW��S�eKGCZS0_��q�`Q�����N���@�]������x��������7x[�(	���p���m���bF7&VwQn��/�A��sj�r>M����49�&g������4P���������	�R����}�ZEQ�{���g����������A��v��XH,$�����'y�C���������Bb!��XH|�$�(@�wx��^�'�����m���jd����>U+(������C���;��/��$���dA��!��c��Y�,L&������1�DM$������dQ����da�0��`�+F��
u��B]���Q���
�FDaa�0[�-�f��_?2��W�+��
y������A�+��
v�W���3�u��B]��P������[]��@W�+��:��4s�@W�+��
t�
�������
����b������Bd!�Y�|-j0&#9u���M)e~����#���W�+��
����6�s�������0'&>�}�9��
>&6��o1S��%9kg��5����O�"�����+W�9k����`6 �9�f��L�����5&Qm��G��^v��
q/�j���B�+W�cQ���q�Vc����+��`W�+��^5v�|����D{
�u�q���)k�����?5I]=���g���
�o4�n>e���WAW3(2%�b>d�qN�Rv��>����i�2��Y0��W��9'���F=e��&tR'�X��+
����y����|��)�n���my'�t?K�PO1����EJ*����R���X����a�Y6���3�+Y����P�5�-��o�K���;e�g��9���v/�n�5������{��]�o����2%�7�;���`j��<�Rd��d��"�?��I|[R�D�g=����9����y
���]_��M����!II�����[�v%�|��4|�1�����@�+��
z����;`{�v�����B`!��	L�)����=z���V�C�C��������0�1X��o��-�
a��]�.l�^��8���}�{��O���.@�B%\H-�R����Kj��@r)�����
~��_��5���@V +��
d�Q�e��(�K]��	�������L���a�/��\'�u�zw*����nG�t�8�T� �u��������P��&y]�o�Q��<�q�����{���^����~�{�7���/�e��2��C"��a)#RE��H�*�?���=>�I=���k6~O��tQ���w:l�
����A1�(��L>�
�����ma4Rg%]��(\�
��N�p(�`��P����^�^��i�^�UIE�8�t�:AqV$ 3�v_�Gt��SI�+{���
O"u��DG�t@��m��;$G��+3��������"�D`���%���s�u�������BS�����J�7s j
5��B�?*5���]��6]�����N�.�Y\��7upS�*X�
V�a���1�
�5�\�W�����>t��zj�z��$��������p����F~!�	�d��b����9�xI�<��PW�+���R����)�7�N:��NA��s+Rw�=��&P7yl�R���Cr��q���ne��|�����N-2�X����������Q�����u�~jKvFe�V)���J��7m�n��k���}���^��Q��C}����\�32
����)1��Z�m6o��m-�R�
��TW�Z(�j��y�:K�D/���_�
�#�TX��Y�1�_V:43#x��.p��Z�:���\y_\���3��vO�kr
av6����]��8�x�SC���/���x��x��'j�
q�dJO��*�5Ma!�.�:E��w�Y���n��d�4F`�����n19,�f�V>M����P���M�����xE����-��rU�SyTN��k��8���Yr�nEoWx�[����0o&����~?�<E��ez<O#�-N3�c����i:���5���y3�i>r�l�\H�U<i@�K�{3��nn���9�&*��x����w5>�.|9!"x�����������.�h�q���|�!��,t��g
$�n�!�4E"�D� ��H���-I�����3�/�k��}���qu�����	 @$�H������j�o�b���������dM���Td�����zu����n��
*�I����2�t�]���{2������u��{�f81x����`u�eUQ�����j�u���Ex�Ql�]1<��;�`]��^�M���5�rL�����:�Zbu�\���W�
�����U����M�9Y�^��e���0[r�{2��m���r�sd����b�t�|#");�l	�Q�L�&�����;��#�:r���_#�{s��]��3���?G[������K�����^t���#@��PI����lg�6< ��t�������2@�\�K�q<�����t�=�V���Q8��������\���<���-,�����q��F�+��#�4*�����;���]P��}�����v��]h�)���{;�w���cE�q���[���j�����i��|a��@�6����x�J�������<Q/d��7\.��DlV�7�U0�+k#5���u��������VH�F�r��6����|���;��\��~+
��5�z�>�� �W���t0�[��DB��>@2E�%��!�eQ/�zY���^��F�#o�k�[n�mIK���)�k�%k���#&�S����`�]>C�L.S�������7��Oa���6iaM$��SH��=TB��D�$I$�uJ$��K3�uB��Y(�M_	���Bb!���:I��c}���-�������\�//�B��8���gt�����DN�{��	�P���p��-�����8�[�@EqM�����07,��F�6��B�n���q��<�=����%�^�C[��7Mjx�������c��-���gV�����K����	tF�X���'S��H��[F_*fZ�E:���i�c����y[��_j��|�O�r���2OG/p�E�K������3]QmS���y�J�3Ar�B�J}��B����9I��GR���q���L	������e�{{���,n0���78��]M��{��=�l~�U0r:cAE����M9sha^�8;�Wiv��o\�&�:/���?�>������x��Gfi�>O*�m�CQ�����Y�)U���r$��(G���MW�.�sb��q�y���+~����k�B=$o��h��6�@��p5H����Enx�
��(-�`b����OQ��LB~�uTO�����J�oEu���C�8�����N�ZiN��s���
�����?<{�d���5��%�lD��\�&rM��[���@5�a�^<��n>�'�vx���������d;p�	�/�@��i����/.��d���.T���&/���v�����!0�/�o{�m����������6��T_6Tj��EC�	��k&c;���>�2Ms�F�����G�6dD���F�}Lj��@���Q�s�i�*r���X�
:���W.����;@�����b�?�&�[��{�����]2q}�9��h��k��{�H;b<�p��z���*�
*��~�j�
BP�����A_W-�O
;�����x[C���c�KB8,�cwS����B��/�������]��g7�]�d�29�r�H
s:
�T��PmM��N{���M����wy��k��[��+p��o��a��R��|�w5�7���{TF�QlU��{����O3`Z�*-���x/���) �QF���/����7��o���W;�#��	�+�f���87�s�87(qn�;~��
�=/��]��:�u< �6��q(����/�5����ba��XX�Gc�����k�m�A�8L]�,�@�M�}]���4����N�5��N��dW�8ta�E:`�.r^&��K���!Wv|���0=�>�^���f���^t�0�,��_��0�p!��0�ty.0�J����|��(4����������\}ZS	��<�SM���
l"$N�}���9'�'��Z��y5��v:}�)������a(�t����h���z��)�J
�Rj W�?�T=��)Gl����K�6f��_��t�m6SW�t��z������T�.�"�E0�`�|�����E�����'�q��=�1���p�~HU����;�;���
K��K�*
EM^D��L*����BH!��L�V�|����+�4�,�����f��k���w�����_��w�7��_/�Q��O-��%�%�}A��;�Z�w�2uxt!����
���>:�����4���'_�fG�������������_�����w�a���7�7z����w���^kz�W�/|y��z6����;)�L�?���`%#���|������G��L_k6p�$�L�,����>�5�W����e\v�����k\����u�v�>�7@�/^>��f��k�J2�M�y�����#��q6��6|(>������{�m�����b-?p��b~�_�vp���X�ak-Ph+ ��l�aKk1�scs�����X}����T���Aq1���VO��]��<�1�����o��jw��i�������p;�Gad�uQ��f�/M�m�L|)�p���'O?�O��2���m�At"���_���e�}�xw;1���v�k�N,1�y�`�cN+������z�����,&d1!��	�E75�e��)/c����y�ct�x�!�~H�M�<�l|b0}�v<��@���
��aT���1�r!Z�z��	��[�����os�bQ���C����
xFQ��w%59��)�}�Z�9/W�������l'&����I_�9��J�9y�d���Xl��x���O����:��y���^`�w������.�����t�K����^v�����3�b�������B9��a���5������CU)@z.��B{bzMt��w����������]-gc�w�1�%��]5���(���=v����z)2;)�^�]�e��!��h�i���c�
���
����~mb��}I�K��3^�4�w�#,VA�e��_�/��_'�%1��XH,$�o7�V�:���{��`X0,���M������1?��|�-�z�����f	���M��l�)F6Rh�>�;E��t�u��`]�.X����y���f�x�L�b����c^�6�RM�+0��tl������<^�����odS�������9���4�C�)��6H�7���*S�s
B�.�=��IT�7e�D{W���+�B����j��c����e����
Gt9�����d������j��ux�;��f����Sr#�yv[�������7��)���.���������t��U��zi�_Z�P�MM�g�7����%��t4]��8��kL*oobZ���"�����8L��&��B�3���E��R|'����e��na\����FPsS,���������V���4���e��jnAC��0GX*�:Vgg���#�w���4�Y�;M��p9mkl�i&��>�ad������R�4��Y�Lu�4F��Z�����3��#Gq!=���-8�zYP�+5�L^gf�[b�xn���g$�"���C�sFj5���r���^���y����U,��\�������1R�5g-����}H|���X�o�k�%K/�\icP��Zb-q�Q��8t�}���tbf��_j���M3l��-p5N8�m~n����R�vG}�������<z���)O��3�;���:���.8�Y|���*�0+��Z.gY���-8>Z��������I��j�����bT��O�����z����4�iB��,���e�_sfp���Rx��[���LE�0�
�_Dk����P_uLC�x�e7�p4�n�8�$F�Y��~xDq�Q&�r���r�+�@��y��a��Fpb	���d'VnlC�x/�'U�O�j����a�M��,�����������S�Gvl��Z��VV�]%AP���W�}F�K
�f�Zj�N��~�����|p�V���u�#4��X�	��}_H�3�%s��YZW��w*H�u�N�`yp_i��Ge�x}�y�}����
��(S������P��������'�	�Km+��u� ���K�N%��i4�4�
��V?u����e]-v�����������{>xpns��Q��)��s�g2����&�b�m,%_�x�F��������x{����H�v6E7�{�	�=����_����3�c�y*��v(J������p��G��G1+�nZ����3�_Co���!-v��4h{S/)5���?��b���I��=�c��G������<$�u�O�q��^�>)��Ob��j"a�%�x:*��MN�����/�����	6i�������rD�Q��TNy�:�6���4�A��/��L��-�S���[4b&�1�2�ib�T�����P��b@��cc�EYXWt}�>��r]���RO���FA>�i�}��:5�������.iuT����b�XhE�W��|�S[F�l��8�������TX`^z�!�qvq)~N�Uf��-�&8�($��&�e���=��T x\Iqcpcg�f���(�����Q�����L��.sq����i�@�#_��I/���1�:3El�z�����w1�����#D�#�H���T\r|�������U�9����d���R��0�@#�b7�W[Wfx��gQ1���*��(U����j��N�`����(�h���6$�h�5
����>��
[Rg���*��0x+,�1���1�����3p�P��(G1X���
������4����������b�u^��n�RdbK��'�(����0�G,FP���w�/�@�Dm�K(l���BM���l�
S����'��I�o��8�P�~�����~�nk�e�C������
�=de��D<��z�����I���h�s	���x���x!,�x&��5�A��T����qciK��.�����;���B(v��AE���MK��C�i���:��4�p������I��I��

>���^Z
r�e5�|?�����Df-}�z5����&���)S|r!��68F\-����E��kaO4l���O��X�<�m��0�+��������W�=��k���	$C�DRz�r0sU��(��/k�%�2j��&��&����hv�0�����?���A��`�������Z0�(#�	Ty�(�A"y�6�ZQ��xI��	��3�	3\D��	�V���8h�<�`pT�|��P���Zd79�	B�B��Z\�W������zI~�
�w��<ue2�4�����*$]sW��3X��6"(���B]Bbv�����:-��O�8>�R����e'MEkY[~��>��=��<E���7�����
B�q��������$������+��V����.�b	[�7��D����$-V��If��@�Y�`}���5>������3�	�f�l�#�#����:n�h�x��>$���o��e���iP��5������)4�Y_����~�������p��n�P�<7��
����0�b!z������s�h�]u=1E����h�m�{�9f��2��g���<=C}���u�w������j�������bs�bX/h�H�!�[���Y���l��!��!���Q����������
3z�/)A7)r�H�u��#����h3a�4���!�<���Q/�Y#�����[��/����/���u�/�����\a�0W�+��F�6!X�	c_O��5F�}��e)��S��:(�[<�
+����=�Y���P��&��`65��y� ��pQ@�n�3����}
�QTF�����{��y�y�v)��6PV}\
��TB�n��u�t��<����:5�'\��v�ajM�ct���z���5���t��!�%SA��{�AYx��E&��GX�t��o�S48�"	���<���.����������?|�d|���O�=?>}�?�8�?�%#YY&mn;L�������j�l�r����
C^�O�U�f�w;��TAeb=���}p��'��=!�	�%�\����l�)���q���~
�w�`����������<�|�)s;m��Gw\��6���V�����m����/�4y!��x4��-%.���-�4q;	C�0���&�^���Z�;��Kv<��=R?|���'O<?<9��hD�0�
�E1�}�wx7=sL�c�Dq����c��)����a�!�d�f��-���f^���0�|�^�!�KKx��D��Qt3������B�h����a��l��k�(+���s������?����S�2��J��h�<������HzG���}������m~�Nh�yF_�yG�����������HE��S��<pz^�;~�nH8��;��������Y���?o8�@+��~Ar�8�����S�8�gi���^�����\r�!��5�K-�fVii��T�h��s��l�gu��X-r��.��-���~���I
e���k����|���t%=����RR�h�oO�R�s�'�I��1q��g���V:�w�1���������s�����m�����M>�h�"���*��y����������>����!���e+�3�Z��Y�����&F�+sh���d���1���F�U��������2q��T��7@������Wx7�K��pc/D$��wi&�.�}��l���\,temx��9�d�gE�i����A���5�'1�4�h��p_�)#_+�C���S���������Jlb�`���s�
P��X��i�5����8����7$n���'��MJAI��a�R��G2G����&��g�OD��������)(��.�Q�h����>�����Mw�w0u�k����dB����p����>�y�qJl�N������H�f�|��M����������T��T���k_��{A���+6!�	�MHlBb�>��o�,WgH��t�/���������o}U;��Xh,4�����lU=��������Bb!��XH,$�N7���7����#UA��#�����x�a��%��U<Q�]���q�q�������T�~A��_��Jx/�)j�
�e�Q�+��
����j�6&���
�a����[/��0Z-�F�������2�K�r
p�f?&�T��C&7��x� Z-�D����Mn���x��
�b��
}��B�k��`��w��W�+��
y����v�S�-�z�����J +��
d��h�c�!	��sy$��.{�|�A�:����xy��}���D��������&���.j��l�rq��������o>d�xdvz���yE#��{�~x�t���Nhtj�?���S�TRTJ'm
����|�N���+��7�U��rq����5���|'���rc�;�@cK���v�@!��+��G�P)��z�rBs\����Azi����������f!�+����=h��]o���+#v�W����8�w�N8TKZNk6��l�����g1� ������T�x1��a���+���Kx�b�*�wcS�v�e�#KY�������Z%�'�^+f�T*��������y�<a�0�w��KO���	�u���u{t:�@�f�qk��	��q���?�(���6����H�jbC�+[`�z_/8��>7�����=��������:(��`��=����
��<�<���PT�w��31�;W��T���&��61(�Px`��������ta�0]��0���F
#�����#�Q�e�2A��eG�R/L�K����s:m��~��+t���;��'��~|'.����{���������Ot}��)�����r�6o�y3�I��9��))�q�O]�I���a��Ry/
�5{NE��P�"�C��{�\��G���T�C��o_<��=�'�m�u�k��'a���.>a/��T*'5(���w�v��2e�_G�������a�zg�80���]��#.O���3�+J�����+�*\L�)��-�N�e�����9�u��+��]����S�snx����p��k��E[U+?5ek����v,	:1^��r�1S2���U�����.���^rVp,��X�������0���9�+��;�X�5�%��F_��^������0�S��t�a�y��A��g�0�+ah���S]#%�b���{�R�s�R7��O��/l����`9���?Lx�o`��Es�,5����X8<aA����y7������ ~�^��z1q"���aiHL����o
����(x��O������5�\�jA��~s�w&sMo��l�����n�9��/���ldm�H�;` b�w;���Wu�Cs�ez�n�rE�(y��&����=� �w��RQ4�i�K����c;����-8��s��^��18�i�9Y����{ �JW96BQ��]�=�~N\n�
n�J�=�'���;��)M,��IV����os�qw�C�`�@� �l���i"��g��$�`v9wm�r`�G]!&aR�;��R���L��?�7&O�8�s���-q� ���9�|���:�[!�:g��2�~��&wIg��Evz:
d
z3���"t�Vt��-�f�Oa�`Q��a���V0�<X���P��fcP���r�)�f�R���F,��C��M�Y��91SK�Yt����OA=8�m�#�v�����C�I�-u����:����L5$l��+�O>�E�(bE+����zls�I	�~�5w�$Q����g�����<�e���2���,�>���^a�����;q���4��������Ou��@X ,������MQk�,�=�����h�|�n
nU�_��Z��{�h��M�f'|c����'NY'���]�.d�_+�wRxi�1;���a��pX8|���?Mg�a�E��IwW��pX8,�FS��S���:J<~pG
�u�����|DK�9���X`,0�^��	���mI�EFJ������y�E�����c���K�D�$Nn��*�d�G������`����O�=�����XX
2�����i3>��5�k�����uU�9�����gs�QTD��l�g�6��.��:��3<��,|D=��BW�����Exj�����a]������	%J$�H�k\.�!��}L��������Xn��bA���ZQ��`V0+��
f�U�}w@�qv����b�����E�-�Y����'<�����t":hR"�4Y%t�Sw�aba�1F%��LA� [�-��fd������s���Xh,4_;��XwVwT���}�XX,,�����br��$����B}Vn��l#��[*��������J�)]]�4�����1���j�ns�t�J��S��#��&R�jCA�sG��
�"D��F9���7�#=��6d�d����^�����|����r ����K�Y����Y��"R��}���;����ea3��o�SW�M�Ak.l{��K���p�"7Dn���!r���np���Y}���DV�'��}�k�a��`M�&X�(��V��YE�
g&��nquO�)��6���8�O��;��<�����O?�������\X.,����y���� ���B��b�Qb���h��@�����!��������^A��W�{��e�x������"���Bd!��y��@,����:5q��B\!��j��b�Q�S�V�B�=iu��)����H���y�c
��p����� ^�1�����"���F�r�z'(6���fo0�	����{s�F�7��K}�IN��r(���������8��W���l��[[�0$��D+9��o_�(�2���S�D=�_����ln��2
��P�0a� v� \���%�����������n�Wn����o$��A```���QOmbkT;���X�i�&�������*��b������N3G[�Q�/{�2�V���a��X%����h.t\b< Tj�|��[V����xo�f��6J�ZDq�
��N�����J��e��r�FI����v�s�������B���ng�L��}<��e���Tu��R���P^L3|���S|>����(h����GIt���^��1�~�C�)
:<��s�([q�����L�@�S��)���|8i2�C��:��������U���l[�=���wPSSGcZ�U?�Qm�^��KAZ�!.�8M_�������q�2�a+����VUX�������������S�3�6�V��v_��#M*�]
o��U�'aK�+*��^��EU�����}�z������b+;�Z��'��^�!?�YU�*p�Z!�|q
����_�����TAl0���mVP�����@�)�b;L�x�6l*���M���f[�
:��5-X�xa��������X���u��0��9��\��\�tFd������,.�Z�)n"�	X������{�(��E�/9���D%r(����'��[��gG�<:9����4�������g�O	��IJ
Z��N�F�n�p������:���2��4�K�?*�Pz�r�g����<��a�A����X$\q�n<1�Hb�ir��\;����ryn�Q�B�G0x��h-;<���}j�z�.J�n�������T��V����Hr,�|wZC�
UN��A�N/���@"�9�����s]0��pZ����Yf�qRU����<X��6������]��"�, ���e�����<8     �!�& + + + + �;���s{��M1�t�`�wJ�=$��������W��
����Y�N�J9��r:SZ�ZgDKq����9��pIZ��H$&��v%7E�'�>E/�O%K�3�e�Z�?J*���
�X��|����2*��q<Q�5�8�U�2	�V�A��	�n]T�T�r�S�����\[S+����9�XiKR���h1P%'��+v������r7L�{GW�1�^+h�����+�#_��Ret_d��D�z���u��vOcw���9-����&�1+R���f�j��%jJ3@��t��?7����75��WZ3$Ieu�f����\q	�F/�^���{����P�]����������wt���~�n��2:�S��9�V�-�l�G�gt����)��� ��d[��=�r�.���N]]]]}��>`~,�+�+�+�+�����15:X��`�*0&Y����������;�_�@o��4���+��x��h�uccc��
��v0�������������1�����H��9��C�����=�\4����
V�����KUf�'6&fC���M����D%T��q�x������^���}�_�Q:���	�|e�YA>%�uSE�����T�E�;xd�%�;�c��s}9��/Ft#��������;���S��<���Q�@�����H���������;DhBg�W>���D}��@�@�@�@�@�N�����wuQ$)�xm��$$$$$��H�
X.�b�a�a�a���0�������o�C�F�Nz�0000��&����o ����PpXpXpXpXpx��
'!w���"�H����������F�Q�b��R��rrrrw�����������m�<��������Q��c���\ZLv��O(�	&_���L�]JjgB�������sn�J1�_N��k��=N�>R�����cE����sP�����3��V�9rx��;�w=?����:	c���\��X�yz�������(<�{����-F��=G7N>Vv�)���D��0��D�DEA@v,l�	}}}}}�'�^�H ,,,,����AZ&%�
8r�:m��}S�x�)������8R,z���,h-h-h-h�kq�/c#���o�b�b�b�b����M��
��'����������;�������]��z�Gt[PXPXPXPx�(\�����������;��Qn�2O\>s�\�\�\�\��b�o{{wn����B��(M�����,1�t����c��l��z��t��%
[2��j]�D%����*���l�
��
�t=�j
��{h�ahB���<���z����Q������X�E�!,�Sa�6r9�.0�������|���KU?9�NgO����8���{
DDDD*]K�H�����p�*�4��(��(�����UAzar�<^I�L�+���(_7kX���L`�x�#�W9�sG}{���=���4�o�Vf��D��U���?t���z��������N�k.e�y�M�����b��W���t�����0��l��_Bau?�X���
?~�C~���x4�B�zH�}��~�^�����w�5,���b��<:/QL��S�
�x���U���I�#��+�VoJ�_*[���RX�W��s8��!%�/�R�'�OHj�zA�/��R���VL����t��f�"%J��	��"���%Ks����:��a�P��e����sCtt�N>�v8���,�?�K��B�[x���*�S�|�����N)H��
��)�Z�X��J���k�-�<
0B��J��v*�-E����^�U�����|k��T�i2����[��Y!�����xhJ���9���ytmf����w���E8��"k��(�v{��	u	�f��i���h4#�r��Q�z�H���5�{x����Wd�DFa�2�Z��h��",,,�kw�5���U��M�X�X�X�X��xEtAY�W�W�W�W�w��[[�2+3=�����d4��Wk&U3SPR�/���H����6@�4����P_%���N�'�T�������/����kkrO����r2��f},��������S-�@��4�R���4H�6�v������,T)I�o���'Mz�7:5�	
��KE/��K���]`����9_�6/�;����*�V���|&=/t�Tw���y^�8���*~B$l���.�"$;vS��p�d��|��m�����kQ�4�i�����9Mc�TY�4G�]9(0Ac��Pt���*��2���jjl1�b1T���SAgAgAgAgA�At>X���0�H��W��rk����oh/(�*�*�*�*�:��d�WkWT33333{�����H]@:�4W�������/
��Uf�`���SD	�B�t���������N�-gY��>��T�����0��W���q������:[��,Or$�A���W�#�u��� ��vc�z:�(Dn���W��w`,�P��C���2����c����5�����O�[��VYN���wzP|�jD�X�X�X�������F&�f]��
��c��'��lY�&i]�U��sS��L(�,t��A��+d�*�YLA���z����82�{�
�?,�w��V�x�����~j8�������3<���}�V����r����)�9%V����3Qd	�f'�N������&�6��6p��s�2Em=���E�Y�Y�Y��S�f�i��[�XAJAJAJA�!)�|������GpSpSpSpSp�7Q�d�IAJAJAJAJA�~��9���oNNNN�^����55555�B�����M���-���	rtJ��*����s3G'�)�<�Iw���3��/�6��w���w��������w���>���0�_��?���z��?�-tMY�I�n(�f�
���?���G�������*z���\������c����bhl�j�(Y/z	�4��:d1��9����)���W�Oir���d3ITL���H�Qe4�:tM}����z�f�y4_j�+��?"�e%l{����\��;sa�4[��P?�i�m&�5w�Ca[e��1���c����:���I��MH���'����?~|tv��9w����s4�L�;iVD��������p��}N������ .C�>�x4����������>��-1�.<�����[��H�%YyiuQ��w�.t���d�ikk�oa�L4O��������ed.���c��;�ma����-f���I`�^��_�L���f�5��;t��f90�4M��H#��5�(��>�@vla�[������,*���h(�h���i\�sK�������x���M����48hm����z�n��N;��Yft��_)#5�Lp����3o�)��47A	U�)�6:MZC4��z���@kn0�]�G�_N�����������L���,����B@���(�&�C�DtG'�VE|�W%�������_(.�[���������0�,pu����gG''�Ehv�N��}�MT���B�t��[�*�{����T	���B=:����k796����{�+�)�tz�F���T��i�������8j��<��n|�E��g�wJ�w�CYN�n��T|+����q�	�-����������J�s�a���0�m#5�:���a��������j���Ox����m���tr�,�j��	:f��s3f���:O�<�AE��}
*��&���l=2���Ye5:��i?�D�d�B_�(��i��"�hYYX�������&���� q���M��5<M�y��T�E��|ETh1tV��|�����k����&`f��T��*�<5�vd���v��_�8{����
�?����'Dzk����	��y�!��R7�Ib����4���4[\�(��6�O�����p:��>Eg����\�^����yl�%���!RUK�VT�s
O�����I �|'		���H-
���>�BWP}Eg��.%)r��&��� ���`����T����l��3��c�z������[�oa��L"8�R��^��=�G����_��s�<��n)����UE� %4Y}$�m�8�.����:
��3�%�Z��l!���f�0"7�����v�g��o�&�����6L}��:������n����Y�����C!�Uv�����~�������~���?E	~�W�d���A�����C���y��[�gz^���<Ll�xH�������z��i*�y,�M�h5�S���AyLg!u{�KL�6`��gr�J������S�s�X�m��q�������R�I������}�z��^dx�Is�%������axS����4x�*]�z��B	$��}P}�=x������d�zg���
�����up�j����k���v�u|j����h�\_p���������k�H�������\�_#��n��K�zE���Q%���i������Y�9.f���e�`$[�G�3T
`N�a�\����������'�m����p�����'�q#w7��Eak��(�8�P����<'et{���[�W�����5�vD�����{���U����������mK���nO<>%
��tz��m�
�w����{��&<#���ZU'����m��F�5�x������i�[mX�S�����o/s~��f��M��q	C�lO<=�z�ai��7_�����4m��PP�����'a<���������o)���Ep6��/�=G�����5��Y8".{�<��y��[�.���1�B�%�wZ�;,�]L���E���W��co�AW�5��Nk�9%��I��\q�TR�SlK�-� 
����f4y���LQ�N��8�:����8�	��>M�]Ed�U��1����W&���@=���B}$%\McP#���h��4����J��YN�����A��4�x�g�_L�3V�}F
u�k����q���W?��r���P�Y����������Z���%����`E4���:���kQ��3;!
�������G^e��~���a�(�#���TZ?H�h^}��r�g��l�if��������b��S������B��xx�r*�aw ZC�V]��D������`�>�A�;�N�o���F��=��d�����A�z�u��7�coe�9�Dn�lE���w���>�P�����$��s�>����,}}V�+�w�
���
�M�Z�d��^[��Ud���Z�S���#
��P��}�n&1�K��C	O�$y�
[��i����E�:w����Z�'��8���O�M�8�P�=���{����� �%9R<yP��s���-vsUtz�w\��s���p�{jB�e��d�a�Z�h�J�)i��������dU�?�z�����v9�(�i2G/i�m!��p���A�4��^���@���Ms��%�$��a�p���<���<K���2���3��������23
�)��Z_��W9}�j{u�y���e�X���z��~���X\����v?�����1]@�+�'���;2y�L��>c������!z�7���3�����
�?b�L����zGF�f�}G���I�v�M�������U�}e��Z��������V78���������7P���������38b��]�bNt:9�6�v�q`�k�����\�=�E��y�v�C-�!��I����
�E�����}U5���r�U��1����/Q�����M�t���g�1Qvj�4������������fu����%����/q{��Mk|�;��uH��3�Ou������?�<��A�A}�4��x��w��2Eu��B&'8����|��o;[-��8y5��l:��f�Bs���L��p:�"s;�u/��P����ibI����M�M��G�����A����^���6�2���8S
Z:��|�������--����^�>��������+�R"�r���J�OB�c7z�j���9�Y48�NR=y�U���)��K�x[M�����FR����TZ�ah�>�,��vjfu�>0����=7,&{���+c'I���J����8�N
^���I���Mw�@�������R+� Hw�{��9�h��v�@I���-��+��p��q���>��)��vY�;>^��5�	>�?`�YX���UE=U/��i2���x�6C�39��F�5$ v���*���T���;��j���e��������I�s�����>����+���_	�-gI*d�<Q������d�}�n��C��RL�\`}����2	����Fi~r��,MB$IM���������g���|�l�y���5��h�)�0T<����Gh�m*���[y7P��[2�/��^e<=��M���';}.�tM��J�0nY:QY������Y���v89�	9l��a����*>>>��U����;�El����
�yS�ye�;�
q��������������t(����w�WP�+����C��P�>V��]{���r���}(�n���.����.���7��{#Db,���:z�xW��.�2�#�2��������y�n������)����|B���'��������<���q�V8����� �����
:"��bP7b3��$r�qp@�h	G\@p����pppp�x~_:%h�_�_�_�_���/	���o(�8���B��|o@�`Q
.*I�.�@�&����hE������VD_"p/p/p/p�k�������k�l���i���U���-�n}	���,�����|��p��V�|�K���,�A�
]V�d|���CK�(!J��0^�ir��Pq����Q�2[��p���[m
�E���B�t��������.H�1	^������������4]��y+F5��p�1n	���\. ����&���=�~��m>;�Gcr��k��+_au����	��!���W��p�_,�6�����'���������G����?�q��(4�M.��xGk�)d����,p���p��v���	��&A���$���,���S��6��)��>OA�_������c�e�"�CHAo�|���9��O�����)A
1��]���hg��7G	���(��i�c�rI������tF�����M��_�.�R��f,���|~���#���A��#�Trl�.�N���;aw��v|�B�1���%���8�T��h���u�>��6YJ��l�[��8��mO�����w�_2q:?y������|���S�0�_	���?h�?��5E���$tAC��w=B:TQN�+�~tIZ��P9�us����XX	�>!|B��K?�~g�����������L]���2C�"88�W5/9���a��9�������n����"�/�/�/�/��a�<p"5�A�������`�`�`�`�`�1y�<
�5�l�R���{R'������L��v2��K�����/������H���&���E������{�{���mi�S����Q���e�d�Zs�s���	�.'\N��.��G����lllllw
�N0�2�x�r�&����N���$e�����������������1�3���+?���"D:0��bNQ�#s�� �>C Y Y Y yTe	3*�.�������������K�9����2
�������^%�C�^��������0�g���F)ia�=�uU-�D��0	a�$>F&A��]������'���������(�zZ����##���11�l]�w�E�3��h>�w$$$$?J�l	�c�V��UQ��o�|�	�>����q�6�����e�c���B@V@V@V@���!|��wN��z�z�z�z'�5B�7����x3��u
�
�
�~�X���������AW�twLSk��s��!����3v�����7��te6�m}#@,@,@,@�Q�'��+D\�6�Vav6����������'&:��$zS�!�����\d����k�������e^��&F��;�+��T�&Ml����`�i��o#���c���s������������Np��Pk�0���y����Ac��M��9�8���Ily:M���/�o`��.�8�8"W�\!r���\�q_���lw�S������v����6��X`P���[
�G������D���v��x���Pou�Zo�o����zby��B�s���6Xh��������nos���{�J;�gwj�6��!�0^ai:t�|�-���GI����90Z+*�y���F����1��u]����pQ�x�q��N7�a��
�k]���]*��(M65$�J�I�;�k���&>�-�F�U�����#�C���Xs��Gy,��o��;�Zp���k����M�hvy�96��R���N�O%���Jo�?9���<��_U�Y�Y�Y���C��-���1>N��78N�E%������#P
�����Q���*b�������W/^M��e�t�D��P%ihf������s�O����{�5x]x��������m���L�2�b��7ed
��������Qn�2Oe�(,PX��@a��c���wn���ki%��H=��J+|L��fi����L�&)Tnb��YT:S�jfp�p���.P����W������,U�Y������q�S��p��
�����7ze�Y�0�� ��D|}�����w����x-~�� ��5���|�u2�{S��<@��:eoo��d�'o�\�����B�$o�<�`���T�<]�I���la��yk���3�����Y_*���5>M��2��9���
LV���He��a@��c�j�2���~mT��s�����!~C���fd����%pj�[:�@���A�@����Q�!�V���(4���!(�G�,����K�Q�eU��0
�`3Q6U��a��51d�X����x���NWT�
���*T���jT��b���G���?j�$���X��G��c�B���4�0�u^*�J�����(L���W����*�)8�����*R57�����0O�x��r�/�E������tF3vgodWQ��T�"Y�d!��H"Y�P����x:Fq
P�����.��#_A�Ym��ih�T�%�����G�x����>���f��k����Gd��XoB�z�����/���1��d��m�-�|�uE	9���)|S��.����	�
�
�
�~���QW�|:�K���2�V������+Re@$�S)"��i��ZIu���4Y��3-QV)8�v�Q0~��Tm�F��UV�6��@#kn�ii����h�P[���
�}���S,%�/�)�S��O�%O��R��A�������_h�'���j�>��*��UC����j6��(��k{����e-�w���^��{��f���
�Z��B&��/�:W��M	�ht�VQ�8TxC	�A4#Z|�H��V|q���q�/��-W}���-��]���Wn���Qn�o*��$�`�!��������������n���Zur�q@�D��8���6<�1LpWpWpWpw��;"������������fh��+HJwq99���\�J[��I��A�F�\���0�D�l�Ej������A9<��Bc���4��>�����&�KLN�QD������Q�-@�#��149I"��x���xm@Y�������/@���:6�ay�xQf��x�QYjm�|#��i���Q��)5��2<�*���'O8�p<�x��xhq�v��&�������&�	�	�	�}���!q������C�E$R��9���m��p�~�P���s��������k�l�����5�3����(���&��`���B�����,����}U& z[G�l	@�I�&@=}��e ��'!:����h���|����.u������i��M����	��E]�B�)��u����������������@���	�9!����������n�n�nt��(a�(F���ASd�4��Q���rO�n^�:WKS,��	�]���~��`�Ch��U�<�0 �W�u�\�&SW��������X����������<��U��/�(Jn/��������������p%b�����������������
�~}�D���i>����?p��H���������[L�Z�Z�Z��SAjO���,6W��T�#J�P����������{�kej\�`�����W��EM�:4`���w�`�c�}7��59�N:�)�Mf���_���D��k����|��/�r�����4��-��xjtv�x�%��I�;n�]1:}���t���,
�jh��9W4����n�9�,_�o����Tc��U����F�`��eBn�W�y����
�B���D|������~8������������P{�a� ���<�`A�7
��@?�� ���^��*��U3��+�FA�U����a%�J>~V�����\\\\��p����	�(:'��JXL�xQ�(nBn?�n^��?��k:\���c��?|#����b����]
=.a�W�����1���n��2~vY����W�^�����$7A:O(���Ph��7����\�GI�����~�?�����xmL��t�E����22���k�Z�/���������U:Xp�/J�Rx$������������&����+�������}P�?�����Jd+��D���gN�U�t�������t�:(}�(��)�<!Rr?%///�C��@��#>(��u(N��d���t�������/���f��g�[3�feq���Y�18^����p�RK��p0�)����������E�I�2
)�s
��ex�H���i�q6b�u���3D��������C`b�c�	|��(��0a4�h������uo������U����'"��
�
�
��{G��D����,����Z%�����w	�QR�F���$H�'�+�+�+�+��K�Em�c�(_�i�0�`1���H��m��������N��m>OCC�
2|���M1�Y��b|!�����w��#������Sg���[Vi6�0���z����|��:7*��Z��2N�$0VE	�A;�xi����J���9�C�y�&J�!�[h}N�Lo�v&D�o:���G9��r�����-Qa�L��Ma����*Q4��W�h��[�i@8�p �@��v��F����2�����3C�B=g��0��9Tez~��
Pjp������MtU���O{�b{�\�cP����Y�v��"�E8�p�,�����N���@��lb!�!�e#'�i�i�i����4����}�����b��u��r5 �,�,�,��CX����:�L��
�Zng����w+����������������D����T$�
�
�
�
�
��t��T�5�lqi�@��	��F�� 
�Z-Z�E�(V��X�"X`����*[�en&�y����NT��]`
��C�Vi(�cxV,��PS�i��7ks�)�[�Y��ry(�C���������^��EV886��@]�xAaAaAaA���N8�I���u����w6�g�D$��� �;Y>��]�<�~FV�f����&EI��5~�r���G���g'��g�<8}vtr��E������%�W
s��Q[ W W W w��[��?�D�.K�k�
��W�px�.��������oL<��'ZE���*���_ ���gO��c*�$-���2���9��������Gya^G��g���'�WON���_<W_|�+FQ1�>f�'}}}���Q�^������O^�7�@��|��A�@���oO�`��"�8:�����{����@����V�\**8�K�,L:	��a�Q�/u�W��&��r�����H��d�<������0~��$1��VS,��#�O]^;�`��v���n�-���`�an.C}�2�f����8��z���xy6����	��3���6�m�����y��!	|
|
|
|������b�N@Rn�mh�z\�"]���
���fYC�Z�[�Y��A���9K�i\.�0�'��2��;g����]&q���Me�i����5$0��O�����w����I����*�v�����1���)T�qS)�(�j"������}O,?�Gc6�l��~,(/(/(/(����V�B�\*{i��apu�l�t��s����4P.tXM	����Rk�sR��Y������������z#����/�#�p�rL�1jj��5V#�G�1>x����.�����=Tw��_~)�-�-�-��G@�_]P����v����d2��#���_�:h��Ev�C��^o�xC�*E��!au���+��Q�9%�^� L��Q3A���D1�M^�����K)5\2+�t<�>��-������Q,}������]'����Tlk���lq���a_�u8bVU8��	�}��i�;��������r�&$/O�h����H@����'����B��cZ�~������<���d.t�{�E�l�}�g3o����?;�L�����B����_NO��}��������,^�f���w��I{.OS��1����[��H��Z(�"u���:Tj|L|�_)���������G��o����z�cTjg9D������4��03�|Z��A�"t:C�V�mJg
��m^�{<�Qr�?��(�T-������n%�*ViO��99���zI}�(W�1P��$6`�GN������&��]��n�dj����j���t�~YO��9��="����D��8��o�b�3�eV�k��EaA��� ��o(�bF�\8�3�!�&x������*��'<!�V����W�o����"
p�7�����>r��6�%!1n�z,���s�+nCU����[w���`�����QQ��yY������}'����4�|L4&�5��M�+z�p����"�z
<�������Y�����_���X�����e�`$\L��p1�b�#��if��0}Sb�;9c�n�x�^W�)�l��tAE���)�-����{{�� k��� .CcY��C��UN?��p���]+�F	���]����E�����"�E������_Q!���u��8M_#KY��]�*���6����3�E[h��J<v%�.8.8.8.8.8��L���Hd&\\\�.���i�z�Cc��*)��%�\Js�E[-�q�x�%���E�z��K�Y���?!��4�1.��^�5"��^�.Q�R���6������%4%-�A�;�V��Z�
ak�q��SN�P7�uC���y�k������G��5`�a,
5�����T]Ga��b��2.H��t��������������Rd��
H����4�!����Ml�=��wJ1_u��7$JWPs$�7=G����J�_�\�>���4��-p�s�n�Vr���,���4��g�<��+���K�l
�Vy
�,�(0����#�sUa������:�CWv4��)�{0�@����1�fHf!.�O>���|����c���'�{l�����<��S��:��`���QA��!G�zP,aqYL����p�r%��u��/,Z��GK$ ����������-��A�,}�tN��pV/\�����LK<��71�zq]P9�s�V��a��2�f�dD��
M~\4�����$H��2X	�.2l�BQ�`sd ����5lN����������Il,�M���F^���?:9}�������P���]��7�!&>��:N
�gi���be3Zq�#4����=R�7��Vh���b@=� �5`4?h��B ��!Zn�9����q�y������"�$�z��F<���;��Q����(��K�hm��
�7 w���4�$�����!��x�_\�Mu�"t�zf�nG��`�_2��kh7Z65���>���x���\��k6��C�u>�r*�����������n�3[4���� =�vM|�O�8�cT����m��v�������/�d6U7��j���(�F94���S94�B;���A��x�$�����)$�q��lBvHvd����������0
���L>7(+�
O�]M���X���Y�^�~5�*�(�(�(��B���RcBG�"��F�@v�����{.?nt��SmWI�(s�p���}��������.8,8,8����'����'�
�
�
�
�
������`@�4�$�&P&�2u��������W��?V
V
V
V~�X�������
4
4
4
4��143]���&N��'�^�x5Q��In�t�D�������o�?����^����0�k���H)��-�_�a�Y��yd���B�V��c?�J���3���U:X�=����N0����}Kk���CAb�u��'3�����*M���^��=�����k1VX������s��h�=>:bF&�M���7ao7go���b�+x+x+x+x������{|�Y��^�����U�~�����)�������O�k�����I�cO��$ �����2�4������p��c��&��oS�v��S�����R/K��]V^�o2����9��\������r��?����;�|���(�i0�E������C�Q|-��:������8:NO�X
'��E��!��X�����?���$a4k�0{�0�a
��)S��f�m���=���:�W��]g����PT������U�����
��{m��f��}(v	�&D�0�O����7b���j�tV�@�/:w7������������f�Ra�0h�����/�c���&�z�u�c�[GG�����j(�c�aZ��UY�b�T*��/�d���u���
��FiBq+�@�s������ry�Jz��wN`��*OWlmT�1uFK�>��-/Jsj?
�I)s�z(�������dZ�V���:���~�A��0uO�Q�yI�Pn0g�`�<ck`$M1��M�0pH��4������|"�Px��B���w�1�n����%>D����=�H��0�d��	����{��w�����N���b6��.��8��>��B���1�D���t��=�7��~��1e�S���'�5�x������#S|i�����d�!��M5z���[:�����Y���Ng���E�lcJ�wt���	����t��t�2#�~0YId$��DF������<��7�)��(y����$$$$�d��9k�;3:�����G�Em-kE����$~b,�1wQ�@m��4v��b������p�>�d�������8��2�D�|5�1 �&=����^��?�I�+)�f������g�1S)���2�08L��S,��W��5p~+pM��/�G�<v�:�d��t�ih�-�v:��B���u��K^�<I+�LV��@�$�+�b������Y�����?4ZK�bL��yn9��YF��U�����:k�)����u^DhaB4~���+�W���_�w~�7t}���N��-� � � � � �
�����N��
�
�
�~�p�
B9p�C�(J���Yke_0��WF�	�#,�+z����M�'������{�s�!��wR_ I>clG�=��o$���6�x���_]��U��Co������4s�
w��W2�9}�?�4�6��B�eAt0|,;k2M*GR��}���w�����Jg�Bg'T-�*/YE��H�Lm��Y�.��(�kS��5.~����[%at��p��"�c�L�5�����`a��hAOO�F�IUm#�WnQ7����
>�Zz���|Mv���~�*6������_��<�/��-V?N�YY�A��\&4�B7���[V���A8cB�����4
�7�7-���d�k�k����&)��&��kS�!����4���r�8�m�+�"�;��}��b?�{?��fW���I��{����`�y��F������ ���V_n�	"e��\
Vd� �����-�G�i4;��\=�ob[�SoW�����%e��	����C����~�4���w��[U�F��0k�H���*��l��%z�B�z���Q8b��h�i�>>O��7��2�j"�U��Z�{,�z1gk�qm��@�zW��9q����wF���%��T�G_x+�����}���z1��i��n����y����E���K��)��#��&�
}_1����/%"M
RHh5���I�x�T�����%z��F	~��!��"x���e��N�L�B<���{���:^��^��T�w�evH_���.��9A��k��{i��2�+�����u�����
T�Sl���kiM|�&���Gx��\{�?�|&nL��4kO�q��w�@�H�+�+��{�[�q����l���r���\���D�&��:�O�XYpF4q���`�����C�W����j�a,e�`B��
�z�����c���G���!VF�w���]0�i�5������5���[���s$�xbH-�bZ�YLQ�G>�5�����z}����c1�=���nG�)���jp��-�zW�������a�>}��t�Q�r���Jl�!�MmcC�<p��4j`����\���L��0���W������#}*.~��F���mm�h�?>m;0x���H�����|���^�L�L�L���2I!��a"���29GUk�;Oh��!��(�f��f-�&������c`��h���V�%#��(�%�'�'�'��G�����!(((�1b�����k�k�k�k�u���<���SD�������f�^|K���P}��������{��7@>Uw�cG��"=�)Y��j�@��"U�=6,�MU����"��!��m,�MM'�����Q��7��r�6�����~��-��<�q�W�{�A]Z�J����.-�@gY�US�f����#�_�0~a�!�G�
%�n��F�Y:��Z����"b;�
i�_�W��"7���)U��C���c�H�\�P��ja����W�����>q����Qn0�U�������|Q�y�+7�����#}��GR��t�:}vtrR9Ru���-�?F<|�q�?N���Q�M�M�M��c����T�DoJ�w9�#����yM�����]�J�E����6��z�C3�
���SDo�(��i�����c���m�~��U�o�"���Enl���1OVQa��)R.��2����j3w�9�
���L���pf���uS��=����T�Hs����O��j�w��!��_;�)��]06^;��C!a�1����ri�Eb������'��GgO��_�x������~Apd��x&�p�7��~�F��������m'#,�	8�G�����Pe+P�����q�[".��qB_%%RK#E4��QK}�$p�k1>������"���q1�8�t�A�8�����{�wU� �u����c�D��-.m�����@�+�l��b���8T�3��2�rvC���*8F����C�Xn�2�5������u^���<U�^Sc�a�B]h|��K��`,���i[@��&&
�G]1��S�`�����9-X���{%�����]�;h��U�	A�"��2�L�h���r�/i�����%��fO��2q�d�@��</�J���i�c������
�	."�[.�e��Vp=6l�����4-�5^�L�	g&��lF;	��|(!t�������������oS�8��0���KCX�a4�`~�Q��q�8����s����XO�Mbwn;5G�P+P�46��X��F��/����g��)�
���tOT3W��G�	���?��A�(w�
�ttFq��$E~��Y����������A�`\U;�7�Mj��CP�
X"���
������M��������^^�a;���������_���}�����k������
��CXn,����?���QW�'�/^��l�V���9Jm�f,�SH*`��0���V,!4���-qZ�C&����H�n
/���R�$�o2��b�P}(� ���3}���	�F�/���5�@g(������S_|�>&����3�&����
?���~;�T�������nkG��	���14c��6����v[��������Q�����%�N�j��Vu--��,jC�g�2	�z�����%�:Z 
�5@��e������"9��4�Dass��Uq�f�12�(��H�y��+����&��/>8�G)�� ��T���*���������Nb��1�w���0����#6,�4���>��;;B��{O��n�Z(�T�,�\
���o����(f�E�Y����&����X0�[������_��F�`�U�[�s3+x5�{zt��f����K=��z,��4����(�N�Q�����-,s4�W,��1�R�l1*)�����Us����.pHCt��}U��}������+� ��z��c:ib����zK�zzJ\��a��B���(��
��{y���MEXZ@�Q�F�i�7CG��!�
��0i�K�
<�(���]C8����=!?�oI5�c�Q�P�i
��z���g������g!�����������7'��p��>�l�'rR\�a�E���Rh��@�����������<!�
�P��Q��.uN��tZ����n�R��:���=���^c�i������~ �}�]������������'O�7>x�=��}����������+���OT,J�fm��tz�r`o�{8���U���$Q/^}�����������.�K�nc\z�����R��@�p4�>�������G�g�y�L381-���]���6M�K��)����a|��v��c��43���������je��h?z���T�kw{B�B�V:���c8��G��RA#���U, �����\�����wo�i��u�&p(����#��:���K�jm%xs�VO9������rJ����8�����J
����[���
�X�� �Du6�`-�%t$6�'��Hi8��!�ec7���R�<�+sK����v�8�����R|J��A3������������WQ��M�APR�����q�:��{����a�nIC�~��:����`��s�A��A�t"�U���{�w���.��~�Y���L�d��\+��c�9�v�%�5�m?���*����S��N�
��-�tm/��*�������=�_��jmc�5������y��>Q74��)�[�%�9+]�*=��=�����!D��a�J��y����7�N��9��K��,��(��I����t=7�����yGWuo�=���m��j���l\ �Oq@�'_�J=�2�Omfr���^R[��	�*�n.��6��^/*k�����F���������[?q��pf����qzN9��3��9_U���f������T��eP�r2����d��]��ix/�����/�����[�I�G��#��@8��,��:��9��,����������M4~����a+��
�/ayQ�q%x��d+��x[�=�����r��uD�������K�����^1@c6��#�� e����2p�Avju��v�������w�m��>��y=37OG�����\5�+i-��_���T_��������5��l�������%�j���g��$-�*gQ�@W���z���d&�0�WL��r$��qT�����%��7��I8�e m���o.xf��{�c�v[�,1����'�rM`	���e3P�.�&L�%6]�t��N��"6�	���l�?�������u�����#�:�J�WG_��?��?�������?��=|
����{�������>]`������S��WP�+x~�������}�M��M����7�����"�	�����_����^	�����`�	S�vjh%.Z��hX��o�LD��s�}��?!Z�:�U�H���$2�C4UI@��|'FwI6x�K`KjP��u+diC�u�)z���&*��a7��\./�l�����s�,,	"�}f��s�V#:�8��j�\{H'�<-�
k��N������K!�LK��j|D7���Ynpt�.{�T�oxj�I��2yMJplw3�N4�}?����/g�'
g��	^��������s�p�ASa4�Th����������9���c�Cu-5�c���#Oa���8v�a��7+Y��;Y���n�������]��	����4�i�?'&������'f��O���F��JY�ti
���g�-1�V
��y��)���Y8
�W��k09'\�Xk.��/mc�)�1`����u��.M@e�~��G�0�zD��G��z��zF�5��M���������X(�A����E�4�#��E�q&I���i���;�>(f^C������B��]]]]?Ft�D`!)))�i!$K�'�ani�k�rH:�u�r#w<�`�eF>p=��k^^^�Oxk�4-t<��K�M�M�M�M����k���zP^
^
^
^
^zx��B8���*`��#*#�)�)�)�)����:���{OOOO�;����!��'�s�h����Wb����R+�r�����S��Z�|o��HF�r��ST2?�����'$��/m�MP����8H�S���8(�����k��ics�z������S�imC\�ZZ|+�1,�V��eT;�Ja97F��i�*D�P"U�Tqe���tb���������W��VVg�	��mK��Agf�8�_%��E?PD!��?(�`Fq��\mJ#���m$��%y#�J�5���d��V	�Q�qyP1��}n
�G!S�(��L_`@��}(#�qu�>�E|Y��E�A,��4E�[\�����}^���XW�l�>�.u����0�:�����)B���]7�1*}d.L���]�`��*���7n G��cR���N�������9"]�~y��DO�{�����>���ce2~U/��))
���1{�fqiU�������DGi������:�������x~���P&^\��sp��k+(���{>����,���(��M���I���2��z���f���B�~V�vp��n����o���<9:~^e������x���9�0������o1��Sk�9NP�vBe�*j5I���C��2���h�6k��_.�D�m�t�TR������=S'�u�����@�X�:�����Q���
Y�+G����"O�<-����yd�:{�`�H?�
k�V:��m�m���;���1�J#��H`Ju�X�Y@��!S(��R/0P�*��JiK�!L
���PRq�l8C�8?J�#������(��6"lD��5/{�7$�����!���NN��!~f�M	%�R�&���������s(+��i��]A\A\A\A\A�+�&����������:��%��l�C�g��{��LD�d�u��Yf�mo~Y|q��L��7������6��<�����6&�#���������iY��\������)�.�X�u�t��/tc���'{>9����
�=�p�7�/s���U�������������Q���$�vPq�5Ou��e��"�A1�o
�g��S���"�SM�!��G�$�����\�v`����KV�����{':��V\kH������� h$�������we��k�X�Ky��a= C���k8����I�����_��^ow#�������w��F����2I����7Y�W5�
��N�����y��I��|�����7KUu���W&`Et�(PF��ZK��T��_ki�E����
���G����"/$Z�?������a��I����o�f�s*kO�v]=��ih&�,��r��2��J��Fn� �W���a���w��������$LW<�}P�_�K�������M�fL�J�G#8���=�5�iM�9,�V�8!���,MB,�FY�+�u�v�����`f���4�{ef����cS:�VYj#jK��������0Y$��Os��<2(�p����#S[2�=6����-1MRAi�Z,�(�������{ ��l�u����=��p�7ztZC��g|��'f��*�|l��f�{a�G���,:K�}j��[4�l9��uw���l�T��c�a�����w�RX)F�qLT����Y�*?U���?T�,��=�Qk7�_��_�����	��^+�~����R��(��|K�������O���,�$|a�"�����d�G���%VEfr�-���-��t��uEd���&���S������a����	���,bW,��+r
����-u�^�f��=�	WsL��G��Kc��a��/�;R���{���<X�(�\9{A���;�D����(��q$h��=b��D���hun���V����ar=|��@G�P���8L�p�9�x���V�;�T�<]7��6�N���U���H�kG�p#N�K����H��0�R��Vpvi��
3
���B���!�6�M��s����2���*`���#@�����;��DP�S�sA���$4o�-@���2�����������U-Z��/�1��s}����/���u�WD����f���e��x����o�A�z��)�{�:E��^���G�k��ucG�Kuo���a}n���t�r��d$T�v���p�v>d����O�#�C42T�Mx�pG@����F��a{���[����n��:���`��h�����t������<��zCi	�y�PV��}�.W(�������)�z�t�V+}M^��Jn���9�����z'&�SQ�����5�� �)H*l�[��2
a�{���`H��bNO�$G���"�Y7sA9b���������'w�{r����Kj�}�A��N�]�p�3^Am��&��oa��h���������j���������������2�4�����="�fH�".F�5��4A��yJ�u�d��I�+�l������bs:)Q+�����v|YU����6�������c��s��\%����D����)bA8)�/;��t�m�6��
���latXD�[���v3�W���WT�;Hp�G	H_:n�=x��X���-l�)b�4��	h�r�Il����$:7��t�yG���xA��V�G�8��:u�"��t���~zb&�
m�$S�i��z]�����Yk$�M���fWn���*3Z�
�����N�h�~�h��o�����L��4}���J��������'�QuSGuWjgA�P&��G\��N��@Mm����WgX���|��i�*�����������'�g���g/O�����yzmo�=k}98U��o}K��U���=�Jv�O����t+Qw����`Rf�r�M�����4
Q�y�Cs^����!�S�������I�� �.�YA������+h�99��~����Z�����j�o��^c�C[l^?Vu��� �F,>�T���a�*�.��S��]cI���y[}�n:J����j�����;S=k�P�����w��
��X��f�����O����x2�32�"������p{l|UQ
_����(�����^{�=��9��:���>]��(y���\��"���������S���U��1,��U'I����Zw�:��4�����������:��n5we���C��y�=��*�1��E����`��c�j�	�8��w"����0(we�z�]�	���LD���h�z46�a:U<�K��
�������(hX�����3���,LG�C�qJ���)'m����VpT���#�u��Z
��j��J[��kIcA
ia�G���z���2�I=��}Id�8ZF���LT5-�t�����cS������g�?���T�F5�p@l���	~��,�����G�����.��b<����^�q��]�z����I��B�v	�!Ijt�EX�i`�ik��������=:=��t"�����^��`�mZ��e��w:��E����sm����p~����I��~�~��iQ�w��k����Zm�$!�4F���*����]~U�_�����j�pdQO2��RJ���)�����������J�'�#�UY=��Xj�k�A�D>�c��8>�wjRgz@����2���������x�:f�((ZR�{���!�z���*J�t5,��D�w`�����v[�����������_kk�����OSi-��J��
�c	Mb���
XO
[�6�G<f�![�V��Q8�//H��������_���O��I��n������P�F|�^#>�kd*i���4��G�	1RO��^�4�����y�u�����6![l}Wk	�L��tx�(4�w��s"�������������e}�V�A^�L&8l!���mI{[���8L��$2��|�m���f'�?��u�U��uT���M
M9�94I_t�M�-��d.��9�ZI�����;<�$�LQ���V��C%W����s;���.�j}�����=(�>������2>�T~�9����mTH����k�T	�[�.�--W�v&���A��������Pj},�Y�k&��@P4/��B�7t�i����I���]����Fc��/j)gx���2��Y��[6��z�u.xq7<D������%:l�^E����A%�U)���
��vu'�� �V[�l^c?s�~���L��v���j	1l����U@Z���!��O��ADAj���|."�q-9�p�l�-��5������E��5�S����
���0�/Ug�-�����sK�)�.�:���|b�t�b���Z�Z��%�QR���?��9����u�F�	���V�&�*1�{�A~A��������x�i�d� �H�l�����Y��������,JX/e����k�#U\fxV60���������_�r���_�:;>;~�\}����z���'�������h�f[+n���?=}���B�8#nDG!w]�J[��	U .��$�e}��G�d�L����6i�x��d��X�S��Hh@��	��"
��\e6����@��]p�40�!Gu�s��:��D���9�9
�����\��C����P�F-C-y�y#C'�;��4�q��aS�]�]�k��f�`��0�z��%����r�����'�%�N��,[��]�BdP�M(v//E]�
q6P�C'!���7E{���3Vs ��b��#�+�S
�D��s&�/n����������c�{����t�lS����pK���d}�{�����
B�YcMu�����T_��X'K�u�l��M��f����:DW�ZjF��@�TM|����D�|s����������w����j�o4\����o<��$�m����G;O����:�����5��6���u��������N�j:��������X��P�Y]���b{W�!vM�m��Z�V}=KgL[}��v_������l���u�y�Au���j.��C|��&�j�A��A��������0�Y6h��,|�npeG�=�(����_W�����}���w��m�H��[�h��Dve�o��6��qSw��I�\�����{t(�PS�BRv��|�;/R�-'v����&6_@~3fxL���a��s���[��
��^�9<p�x���X��VAu�cj��i��AW��W����@����}�����i�Nt����Yx�=����|4h��������%!a���!=��>sM�+�GCaY���;������r
�X��*�J�!5h��u�3��#���������Dl��q�4R��4?���qdR6q��3*���[�����_t�W!���C5S(h���^1y�aP�x*�������F/�$
�k��S�S��K(4�su>����%q
��p��X
lj;Z��W�-��+�����Y} �����~�T�O��(��������u9p.~���x����6���E��������Y%8�M���4�"��K�I�B����y�Zr������V�t?;��5��w;e5�����>F0]���9n�sh����
Z��de�����N�a��]��w�
�;��z�Xv����?�z����7t��v> �H��hH���\m�-_��W
�����y������0F��^P+�38}�NR7<t}�~�6*��O���$>�u��G������j����nW���-��;-use�l��z������'~����qP�^�`��w��
.P]��W�\�_��?r���5�~��'x�sT���9,��Cn=�����*w�u��^)�+3.\����FM����u_�u_�u_�u_�u
Q"�JDU��*U?����c4���-X�L��dA� Y�,H��Z�����\�@���k����������]�h���c�F��/�c*��P�.t����7��!�
a:h�:�-��k�Nvi�a��I����m��p[�}n��"�4j�tf����Bg�������^�Qy�x:i�c�A*����]�L��F'\�C�������h/�<�#N��7��(tJa�M�L�E�,��j�mM!1/0"6�����9�����yaw��+?3����<�y�o�c^�����?|�K�m����B���"���>�R��;�S��������=�.���D��d��d��O#ck����46�����#�
7]1%?O���
.d�����&�d��At��z�IxE��&�%l���Nl�{%4��=;�����y���b��1�RY����Nx�>�_�m��]W����X�*��Q
�f���A��=*|��0h@�sIQDq�A������('�I�����~��6������3������@�W Tp���iOQ���t�n�wE��	���������B|!��?pj���!��B`,0���j�{���l��Q'
���B��Ijm���'�({����T�,p8��(������:]�h�F'�B��������R_�M!�`@���qX`����l�.l���W�}�]�'��l����:[o�^s���q�En���%rK��M��/�j!��
q��B��#�R<-Lc�*��8��HZThF�?�3���"�$]sy�I~E~O�����(q�%1�W.���V�`�5�7��v���_�<��T/��?W?��x��K��&�<�Z
�������mU�����IXG���S^��3�w��C�/`�\A���.��z���K��������K�H���t���*��m�N�3��$:��LwEg"��H]����������|8z�Fr�%�5&L �!5I35(f7�*4���?t,$����w��4�
z�����zuoo/L�������],������e��.���g&K0�����������c���Xx|kj��:��M��x������s�H\	!�1�b~�a#M�jL��Bp!�\~[?������P��!�@"�1!�~=+0�Q����9�r86+�<�)f�+�|w	�9��g|x�@���+m�@^ /���o����G�B��J��u�<*n�.�S%9����GS&;�JnZ]�)9�����n�������7���>��,s�z���c����v��F��=X
�C'�,�e��<O4>�i�������*N��6��H
�")DR���I�X4������<��^�K�R��o�������wW����[���Vp+��y���6��t���[?��`�'����Y�u�����j��J�3�z��V�8�N��(=��b�4<�U~�Sz$T��]�~��AS���k[�
�|����J����E5�*�6^j��;.q-��Am�D�*�O}?����z���g��� w�gM�v�.����%�~�~G��N�P�j����4�f����{0��c\�Z�y�qoiF��v.���(o��f#fA�����7G��/Z���01���c�L�SH����_��tA��=��o�X.�"H�u�g��L�,��{J=�+�R���Ei�y�t�b���w�'4����mwwW]h�K
���Dk������L���cd�i4� ��II����g�c~����Q���@m�MY��"���%D��/���+;��3�/W��(��t]�Bz��tFV���P�P;s��	0����[(���:��O����s�,�b $�{X�k���M�W_����FkV���h�U{w�C{�s ^:U=���~��?2	�5n�v��(��}�������a�|6��0�����y��9��V��|�������e�������kKK��r�����^]���A������[����w$����zO��s�iA�gyCo.�Q�r�����)���."/��g��
�oAyi������0;�_V8"�J'�n�e�[������q��Tu������J7�
��QA����Ts��K�#�19(�{�1�����C�����[��k}�����i��o��3� QUu�`�Z��G�����o8e�h�@�����>�&�~�r����9��yzQ�M;�5pn���u[8�P�m)�[�t����8K2��mNx�����Ov�J�h`8�����6�.�>���=�y�jR�����T�+=������P���bt<����tq���x{�U�\�c�	��3{�h�+g��G�=����q����(�4�����#�&��y�����67�|r�����G	�����OMF�h������u3P��)����B�q�w���:��d�Y���Zp[���tFa���,���d>K�����}0�B����eUW`?�N��)NBQV����o���'i��0_7� �p�x�jn��G��f���A'
y^	��/�c�Y���%�jBoI���W�S
�	���V����I6���`��3�E���>&�.~~��4(G�{��o��Cl}X��=vf��"�#6�h�:�?p[�A�f3��io;����%`>�J2N�zt� ��zUB�A�^`n��C�����4�o��]������}#D<f�f'j�=����HT�_��E�{ �8mG�o��tb�nE�%r�.�~����G��)�����	�����*NE���9�E���gj�
U��YTm��o���|W���$#C":Fk���*���LXU��}Y�Q�
����9����N@��I&Y@|��:4�J�i�j��U-�
��N�SC�1I�d���^1P��'��z[���)j?���p�����z��������-�8IZ����GI��z:��pV�N?P�����Zy[������_���O��`��_C+h���0��
����ND�D���M�knk��������h�?�����;��y����+�����/amm;u�C��������R���j��o~c�l}�Y��C��?��w|��F�.��3������L�k��}^-�������A���7�H�[a����b�6��� ]��Z�+�z����3N4M��`'�U�6l0�E���l�|��\���\�����0��H+g�&��5�Vh();��h"�S����Jo���6������;P�3\�c-��g'��U��0���FU0���g�p�O�EL�3���S�h��+�����R�M�w���,:�����o�6�r�j�K�l?]�}��+�������.�p[_x'Q��r�r�iU�#�cy��-�bT��>�Wh��yCX��&�.�����/��H�mY���08��J�*��so%��Q�p��kGH�3��_yab��\W���W�U�����Tpt�=N����*4�h6���+�A�
�H-�@�����[OA���I��ET��O����pMm��W+g���s���N�C�5&R&A�>�&��f�����
/l~\+�3�;�6e��8P�q��
%����v�@w:9����Q=z8���{���7�'���4���YI��'�#]^�����3��]��d������&����Y��:���?W;�wr���`X�����+O�x���� ��>�h�\�z:�O���G�uojX�/�������Os�7�����s��z���`�S����-�3[u�Cm[g�Vo}���0�&T
4�dQ���^��Z`B_�J
�>�	m�����e�z����`��\�ujt+'��c~��y�yO4��WW�uh=k����
�Z�WH�A���^��.�l7��\��Z+?a�����~��o����w�38��_�5���r�|��yrz����Z����s�\�����z�N���;���:�5�1�o���i�b]B�#do��������uA5�0���d}

��KG�6�5�G�^��O��w�z{�'~���p��o�H+'k^����v�����f[�����������:��4�P��x�k~z�U��4��0���@��Y��4*@
�����|�@�����A��c�N�=�v��v�U�*J�����TT�����9,����{�I��Q��h����qQ��+=px!�W��o@�������-��fgxB�w`O>�9t9��OSE9�#��p�3��J�rQp^��X�e8*�o���`Qh4�@	E��C�����8g�Oe8=�������E�s����k4%�4v~����\g�����8"�7�U��u{N�m�s��3��7GW����Uj{�Z���yJ_�,
�(��>�����l�U��>��p)��^x��$�lC�������6Z�k%�����
3@���Sg}�2C�?
�0t�V9���yY.8�)��=s�|����t�X�C��n[���v��\�XZ|t�W�M-�7���fW������0�1��vz
������[kq��e�z�x����Vt�f�u����]}�+��b��DG)m2#q������$$h�HN������v�[�)U�W��3	 /P��AA=�S�VmE�:h�wd'�5��@�p+t�e�quab���,KY�m-�i���L��*��}Zk��dym�l�K����u����6�i��E���y����\d�.-]��*���x���ecum���fX+'V�m����%�Q�E��Pw
��9�[��]6�\�y�������o��)j�zHJ
��/Me7���N���*�9h�GC���}��Ms/�SJ,��j996������O4T`�_��qaFnxcK��e���%P�2���7t]��2N�����
Zg�%M�g:��p��Z���j�pX[���E�
���kK	����+�@��q�{���i���!�����6�s������,E�Q���,P7���m�Z:XJ�MF����^h�j({ns4�E`�G7Q��'��UyI��~�����w����f���<������
t��t��������LAeH�"�����K9����]b���S�}>�tF"<���+����o��`����q;���&�P��������
M;�(>[����^��M4��P3x�a7��sO(�9p"
��P�CW������w��AR��D�L_4�0h��4�@�&G�����NNVGi���f��o�/������������D����������g	>y����A�!�Q������]\�
������4N9'��S
'�U����g�A���	M8P��=
;7?"��������Gn����_~<8>`�O���: ��0
2��6��M�h�A�Ns?H��UC�K�sx�:��%2�T��)��������R�F��L��s���ZKEa5�b���hpn8�������If��n���?�������]:�?i��������,\n���g�����"�}[	P"�'/]c��k�Agus��;�O�J�7�[����M^��{u����/��������:����>@��.-��uFh������L�P�*o�����n(�Q�Jt�U�*��������Q�h�������������b2x(��l8��3�}4�7:n�SI�7^�4�u�
P�T�U��/LB�����TGK�&����Mw��t����2��k5��-;v��z���������|B����Z���b�|L��M����1 �7�{V�\ma�v]���p�L�`Lz������;���{%r�]:���*����~�qW�@�����=�?�����!���74�����9�@*Y�C���jdU7.����h�m�����#?=���)�����?�h\�����J&�����>��i�D
P"f��kEy����a��Q��P�>'��U��e��
!.Y ����S��Y�zF��N����Z)�~@��cO������]�%�,_�����Ak��$���~��bw�v���h^Y�p�_Y�Z���6eq]��5�����ZY�0sk��}�e�Q�_��hv���V�����$�.�(	lml	�������	�M�
���J����m��y�kl����h���d��M�����y��+���yo�f�_�t��J��[7h)�j� �\9��vp�z�B?�/����-J�	���wO�>�
�����R�.���vb���z��8��HV��#h�w5�����$�xn�����fnm)vj����WV��f]��Y����V�f����.��M�����-�o��n����6���5a�P���f�;"���Z��ux�eUw|��D��e�!y#���?�NBN3�.��t����z�-����R���v9��������/S�����9��a(}��B�y����|��7fzo9y�5�����#�"�x@��>Y���[]W?OX���5�jnM����s�R�v�������|�-"����KW�CA2��F�sh6+�Yax�Y0�iQ����VL�L��I'�1��m������Yg"�x*x�0�icl_q��GM�7��;I�Q��w�.��(vI�p����
�7/�e�Eh����NJ��h��K���
��X���D G�T[����C��-h�2
}Fl)���M�T�S�v6/'��a]�j�TCyf�Z=u
�I���N����yj�f�/E������������K�=�+�w�����7V�������P��H�yx��t�Y4�jkaj���.N��������ee��������R'n7�V�)�7��O��@�m{6(m�'�������@���p���M�����2�������^~b���sX�������y����i�����yu��'g�3@��<-��
���v��5��c���'��`�]�A'x
w7����l�qa�[biM���{��C���n�P�������W��������f�z
��~�m��� R���4��n�;MA���[7�K����f��.����b�[*��P�L�Vk�~&.�A�Y��"8����K[�V��;tI����]��i��B(�uz4sj�NW
������m�]�����E?����?�~>8�����1��(�9����������\5����)4z���-�Z��3�<�9��Q����e�'r`:������cC{�#������������kxiw{���k�1~�w���e����5��}	U7�����C+����
����?���i�|��II�G������Z��I�;%y�$����{w
�I�k�O.M�"T����B���zgf�:����������!�]�.D��"���S,4���B�[��%������	/��Bb!��XH|s$����@��
���W�+����+!��\H.$��2�?7V���^`/���:�ot�����p�\.�9��-��z�I��`��c+��?-�0�)Bn��&y��x�
�{W��
�����yD�#�HD��"E"�nyErh�K�y�`�S-��#4����k��^Po�<���g����y)��`W�+��
vo�K�qqq�+#�����Z'X���������rA_����z�[�}Y����t���LX�k���tC���w�pW^�����+-�o����^�
�A�����s�V�s5���]a���w0���<J��x+(Z�}��'����V����Z��3o��>����R
w���������g?����:nv�Z������'���"��	��:a	+�Y����*���Wz0���O���n�2���o����v ����v�vz��k���,<�pS�)�n~��������(�Bu�0���O�'��m�LV�8���Lx�{����N��A�i4���t�/��\��|�������������a���BS����#��[?�	��y����y���>=9�	�u��MB�'��R�cP�l�n��pN8'������?�.��=s��n@�9�9L����Z����&���BM�D�c��\~tg't�uq���P�)�x
<?~x���R�I�lB6!��m�v11�V}�������N'��m�p7R��������z��~�O���m6�Ml��JM��Lg�������p||t<P�F���g�0�V������w������h���!;�W�[E.9��S*��i|,���o�w��y}C�.��
bee�T]����N�����d�;�����=��L���)M�w��_��|
�.�:o�|^���)�yY�k�K����*�i7��������,BB���M�9��~"�d�#�E'�	�w��;�����i���>�����|�qk���m��MQVGY
��A;�/�"���q0UoM���Q,�>��o����F0�=]N�&H�m�������iui�Z�u����9�?���O��	�k�JsT�,����/]�FZ�����Rs�������"H�D�-�`A� S�����!p/e������(To��������T*�������K�&x�m
�:]Mf*R�J�
�u��M@��"��a�]�q��	�x<Z�p��}��x�~�N`'��m���O�&H�m���O�'���1������?��V���������u11�����2O����H*�qa�TN��t+z��1a�)r��Le��9��Y��U,��hA� z�|U/��R67����pY��:�W&�T^$�S$e��HI'����J{��	������K���rSxy��y����*L\QFRT'�C���B�%��|��i���Y��I��*������uUDq5��C��W
�8JKI�"H�
R7	�0��F�B��J��N#R>���`p�R]�j�2�n��3�0G���*+L��	�A��<`Y����I��`��[
c����m\3/��C�G6h��x�y�h��lN�9�7�����L��	��UJ���
h�GZ�*t�
]?~���N9P���C��l1<���)?���`��{x�>�����������I��&@r��j^d�3O�&p�	�6n0~���vT\��?t@J��,M./�G\�S��A�S���vjB��QTb�jk����N�h�E<���kv5	/u3����%����T���J���m����A�z�?��z���������"�-�e����TN��z���c��5�/
����[��Z�rc�w�%���h�5A=VM����'t
.?��S}2J��X@
+�J�
�BQ=@PQn������c]��~h���i�|�x��o������`�|����O~~���������G�}�g��v���Q���_i��]���AtVx����v)RK���y��
�4.X<*���v�Q]���s��ig�5����AM��7W���G/`Cm�4����+J���>Kb��l����-�����5]C������&Q��T�"��46ac�+Js������������k���������u7����0�����){����i���s[\����]�9�-eP_�x����6
s.����o�n��Z�Q�v����E8~�������"���:� 1�|�U�18���T��������wID{�
���Xl<:��:=�U�a@��H�&{�����%��I�'���Nf��9���Z�EvhX�����iI����.2 =>���������`S�z10��G\�����D�N�b���o@�~�c���UE4���/���?=�<;|yJ�C�x��6�q�T��(���~��T��(P���}xJ}�o9�dw��������a0)�gH�fn)���O@~#���\��4����/��&H+�����}&�9?4E�Q��t�S���/T���NP��Z2E�#n8+�������Q����NO�w��;H%��3]�j��.���'t�<�,ZFT�J!pBG���x
���I-���.�4?o�I��r�Sa�,)�
��5�	
�b��@���un�>�6SQA��T�nj�}�JP������\���+B2W$h��S*F�D�gPD��Q�.��n�h�	�O��+D�%����.��^��f���Z��?OA���
]%��������^�<�������������Zb�~��zX���8
�t�3
�Z��H��q9�e����pp����o�d'�}���*( ��@R�����������A����B�t����/���9u��p� M�'���dV��#��Z� ��t�G����s���QPqC,
J*��������1R	���4.�N��<�}�$����[����`��x&P]hx
X��_���+��dI~Ap�Dx����r>
��Q�M���+�8��(�)4j�.�I�(�����6L���~�9@T�
�PL�a�\�67u5<hS�����iO�|���+��c�����\-�����(PQ(�q�@���,��e8�h��4B�C���[mA�&QO=��s������}����u��W�Y
��Q�8qf������<77<HF{�_�~��T�pS@T��������J�����|����@�r
�o���vt���Lx{���0��*�4�pD�6~�������j�xT-�V=�Q�vS�t W�-m3�Hk4�@(h�����S���|��$k�)4�1N��������>:�0���#.�3f9>z,Ft�F��W�������Y��l���#��@����,33S1�|C=�����j�}c��)S���,�4J6��v�Xl������� 
;Rg��TZX?;��"Z��9��eH'���B�R�c��Wn���CU�Z���\��P���/�4�QEf�G�L�3�4����9ZTUE9�6N����O5|\� ����U�S
M-�����
��Mq��7�h�/pP1��[q�r�>��l��� �x�b�LQ9��r�,u������l��\�H��Y�
�
O�51�QSE8������Y{�9{`��e��N���F�O	M���ukn��R��Tw4
�8hO�}��v�g�b/g����5�a;M��D����X��/��>��\��="����b�#h�6-�@I�/3
�)u�g���"�����k��s
������2h��>���:��i94���?@��k8P�����������R�?�q-�5Z����f-�V�>�:k�8���H=��4��yrD�����/i����KMMA�K�&��L����f&,�EA�}~���
!d��v	�Y��j��<��	���Zzc��v�>7U8��H(��J�k�[J���(;���DB3�3	���������Rw�C�����
����m�8+W�)����}�8���}:O������T����<�d�_�J��}��Q�����|�)h=yR�>�'���u.������&gDh\mRh~�����o[�_��������k�6�xm{�[����'���d~�]b|#�=��"f������wv:�[�9��?�k`������gv�j����=�{r�%������ar�Y)������Z�:�b��M�a���������������n���-��Q��Q{���Y���z��P�U����
����/�����_���_����~YqDF�f@����	�n
�ci;<jh��������a��z��iY�����;�a�I��3o�����8�N
'����;�I��P��q��>~L65���l��)�h 	�-�s}������%��� W�+�������m-J�
}�:����]J�&��Wm~�qEV�`ug����g�����1�6�����`�;PB�VW�{!�[�-��Dbx�����Bmy�h^Hv�������1,�J���e���r8�L����e�LP+���	�^o���V����0H�R�������������1��B�;HL��Aa���s���=6�o��`R0)����6L6�����C����N��?I�h��z�(�=NW�~��\4O!��VH{�H��p�}CAW��z�5.�O���%[����1r#��M�
���|rp<�}���v�j1��K�^���`a�0xL!Z��i���U;��N�$���5~�z�X��(�v����� �����<����K�m��z���u?��`#�s,��9�/{���6����L%//8��^$�(U�����;��F�_j�P����#G$�H��P��B���lB<!�O����{s+)~����h!@�
P�P_�e���Kd���vpW�5"<�����,����^��6�����'7~� ��=��<*}�3���r>��&���MD%a$^N6BY�(20G�n��p�ec��#(��D�h���tS�,��9(r�n���+�(B?���O������9.���O�'���	�Q�+� �	�`�<��(�w���)�4���PLL��Ja���n�M�.v"1Q�d�-���>��{#�e����[0H�e���}����]�k��0�E���T�s,��}�d����Q:���������z�����}rY>P������{)�W�i��:F�S�� ��u���B?�����{N!)!X����*�
>�G1�0�(:����.*����.V
��kR=�X�������7C�6��
?N0��f����PP�|T��?6Y2���	��i;2.�������k�RC
��Q]"�Ac������.|R���F������������+�mRV���@f2+����p;H}4t�h��1���u���`�^���|�<3��]�A���e��py��:���^�+��-��BK�����M1_��	������7[[eU&V?��T��_�����x�s��8�>]R���%R~�'�����Da�0��c�5&_��X������[�������������zsfx��g�Ii��f�M��d��j���7�]�{�.��
y��5y��P�'M��z$1�L/'��	�U�L��qd�^]�8���t�����O�B���*@�>��Fy����y���(�mZY���} ���<�*�:(�[}Yl46����!�C�����B�k�`a#���PX(,n}`��Z�QxgE�����*su�U�g*5�0�L�=��3�uW��%G����b��rO��}���G_�'>�t���*1��������<����
��pa�_���H�S
�����yPQYX�3�%Qa$)G��,�b�ws�����X�E �@��a!������E;����l����/�/��lx�������;��pQ�(\.~�\|���f���BG�����[k$���(\.
�$�>�1����oU���G'��|cx�N>�9���0-�����M�������wUda�0W�+�����V���9�-��-�Z���p�`�PS_��_B�/���?>���sX'��	�>N�	��K��4�:�X�^�����a�R��+���u��U6�t�F��Nt|������;g��S�S��s�w
�;��+�D�7���8�7��.t��4\x(<
7����J�(�8y���C���0�(`^�j��N���Np��J�	����S,����4T\��O�G%S�YY�
v��]ZQj��}����sZ�����I�p��>{��Il�����DcAA����r�-�������ve�
cJTQv��#�}8�:�	mH&�
�_^l)�t����,�R=��iT����
}�������q��.aS���/��]J( ��d-��lq'�N����;����	�5Ng���N�.�C���p�x����]�S�V�2��K�&��Y���e���5�V���=��A�(R��t�@Jv�c�{6|k�����m���U
��M�awS[�#m�q�y�ngC��$�

���MCa�G5{7f�����
�����4���+A�8/����E���������
J���FXlJ����H>A� O�'��L���r3LC�
��M��|r	W8����K��]����k������+�B\!�w��.dsF��]g#��l���P`��0\{!���"��BA��p�P�j!�1+^Z�UHY��
J�w��L}��-��	�����o
�P(n���KH������l�QO-�����K~�c=`����U9���c%���4�ta����\U�*'F�	�	�;�Q�u���(���N�,I5E7z����B��$<hH����"6M��:��iU	����
��a�����+�n�lQv(�KQ���,�S�?���IS
�Z��'�tb��E���xz�cU,$�D��V�a��x6�3]Dq����{��R��T������T�t�������,��uc�a�����������0AU�����M�6�@]�!z={���*��3���;�bG�K��E~��3���
+�-d���������o��������q�r���e�����<M��)����7�F/s9Io^p\�A�)�5��������T����^h/�1�C]b��h��9��,/K32���
�k��.�)#"*�>N�cxQLE	= iBE�e��m�dO���S��V���BATQ6ZE�/K]T��M^�.G�(P(
7�D9��
�?2	��fQ
��a	�
�8�c</4

��JCo(
����;�Ck.�����Ky����C���pCy�Gy�^
D�@�(D"
7��[�4�������~#��`O�'��m�z�iR��},��	��zG=��L�&p�	�6
n����~�����?���O��a���eU��:����)p�Nsi�c�U����2�
�������9�R���i����Q,�G�'M}����������Mp��I�K�}f���NS,3���UUc]��v���	�N �fy�(
 ���'CSS=�D������v�u{���Z�!��:�V�'��m�z�M���Ny�<A� o#�����u�K��e���_�E�G���c:<�$�j���og�Z����z���gb������Gt�;#��	��w��� ��y�=���b ������.V2���LU��I2u��3��PY>������R���3G��#�������r�'1�d���*��(�(��� ��b�3�b�b)2�,**��B:�gI�U��\5�C�6�5����It^?�������������`t��U��F����{�`���M�F���8�*��3M�%��2��Z�gf6���s]��9�79a Wt����-`����]��3�t�����B��N���A���h����f:P�a��_�F*���'�O�.x,J;<��+R[�shY���`j+�����b�������)s����8bp�v��m=�2a�	�����s�9�>��
���pO�'��m�pb����}��Ltx<���f���a��SAa�����&�������2�(
��
E��r._5!��'����������Wb�|��	.\��$������@5���i~��|AY��|�7�;��/W�$����]:�����.?��q�'K��4?��=z�:�]�SZCn�0�=���7�u[\E)T4Y�6�������5�����.h�K�[�W���������Zb�}��d�
�{�:��M���������/��r��F�E�
C4���0H*�������O�'��@��L��Y_�	��{w�{0�5������O�'��X�]g}���2�(
�	�[_4����D2+�M���6I��x���~Z��{��b�'�\���)����Ys}����b�+^��x�g[�Nx=KLc}q�����/^�/���M���$����[_��
��\��8�����z�cc-6����I�k�N������2#��(3��tx��>��0��4���KT��H)�B
!7��eUD�>]��	!��PH($�tf�4-���
�����

��B���a�����)>E�^a�4[4�M��L6e��@���r����~��y?���n!C�������[I��R\��BZ!����W�����]&I���Pp(8�LZsc�F�l��@A� P��$##�;X ��BA��Pp�(h>�T�&l�	�6�m���Y�0�M^�o���=��eJE^(3��o���E	p�fO�
�\��]�2{Pq&��m��l6�����"i���t�[�E��-N|e>�m���C����=~�go���orA�������I�j	���K�������|�������Uj�j ������}�lC���?��0���8nZ:����o�kJK�j���������I$d�Gis���?D!���7�g��Z���qY����k���_�/��X������iT�
8���.V���O��a�CC�z��U���Bsy�NA-L����3�����<�	z�RP�8����q>[AW�;c���K���a�J4Q�j��/)���pV8+��ng]��Ly
�t:������VYE����Q9�\d���AI��pG<��/�/�q�N��LD����R�������V�Q������)<
'����������������������e��a��5��1��-��)��%�����<4�>��>��>4�>��>6�0X,tf�&�#��?����}�����N���q4�� ��A�4������Z}f������z�l	�C}���3���_�/��_���GC����!<�����Hy�/N�Q�����y��N����ja���]Y�_U���t�B��<�O����iA� W�+���+rGy��b{��"l�T1��Rb��XA� ���h���5��Y����j����%9n���tU�=E�3�:�+E��R��|���(�&W�)�c~��u1P�zP��.�_�Ws�1R�4JS.����"���$��PM�
�+�n��(���DU�;�2����<���aJ����,�;�L��h�a�E(bpUe��N�z��}U�&�P������W�EQc���m��M� ��R+�6,t</Jm��KVw�MD��,����'�Od���'��T? n6�	��TO�������������=�<;|y��U���@X ,����L��RL��R,<���7��Z)nj������z�)"��y����&�r��pz
����.|�"w�c>>�����*��}���0q�;E������w�<��c�=��yj�!��|������flt2�����s���?tE,��>n9O#�[���K����rvu�w?[�Y��I�YYE�Xw3�]?��B'v�n�|��/�G?�����ye|���lr"D:�t� ��z��w���f\
�T�i����|K�m�q�Y0��j��`�WM�� Z������"`H�6�B�9�}�M���}�&S@�D_y����70��&B_{4�T����<J��#�_7������J���E0/������
0G���������@���!m+��)\a��>5g����i���>AY
��VO�����=?���}(��p��
�/k.�[�n�����<t>?|&L&
��w��A��c=����Z�'��'y��e��@�::��1Na/��Jv��KJ	Ot|������;������^�i�Ecn���"D"�Dxg��4/ ����u��X7�
`�X,��]g�z�F1��Z����Bs�u�����T��b���L��"�����`U����4��o#.�n�
��4p\O��j�GS tYG�������XGn*�}�a�v�K�b�a�
�`\1��I_��d�d�������(�?��O*������[����f���yx�bE��6�)n����:�7h�;������Qz-J**g���q��V���':Y�"&�>DN��9-r��6}tdm�@c��'0Q3������`a�0��lb�cT���Ui���n�`����]�[�I��f��P�������������s��>����2Ui���9������O3������g������nO����R���W��#k\�/������7��]����9Wz��`Sh���P�)x����_��b^�8�Naf��L���qY�5F:�r/&&��a��y�B��Ce�,��Nf�)����!��:;�&�pAH=~�BD�	(P$�;I@��^�
Jr`��c���h�JMf*��[_�o�WH+��
i7��=\= �f�������.��)����������W���U��>_����UZ��2Sw�D��(�V��U�z
�R<�r����c�F#���Z����T`*0�L�.���~@T<�l������K!���%.�3*���c���Xx�>�q��$G����@6����om/h�tJ��Q��`���h�7������*j��x��?Qq������!�pY�p�����g���Y��p�r������:3Y����'���g�'�G��� �
G��w��4���v���:*Q���.*�
C���3������=Vn2UU�B��PA� t
�f]����&�>��O����r|������i��M���4��
�7�o����(�=�w�i
u���XtX��	����ja���;S�\�ClDh�Z����/u�~tXP����j>S�U�W�9&/�Ri�"S��AOc����b���@F�R�Y�P�x�c����������p���[':6�R�G�����d���f������}�R�� �MD��!"B�Y��(�g�O��}��O'��`A� X��3)%}�`��Fl{��VH+���LxxfQ�G�$��0QGp1��esv}��!��]�������X�j1��]�6�yf^��X=D� A ���Un28�j���	��0����`�q�-��;3-
����c�����y\�j^d��W�*h�
Z����7[[[wn�����z/~�������?�������K>s�b��L�"�,VD�����������_���4��Y�O����~=+t��(\X3K����~��.t��<;5�)9�@���r���2:Q���������X�Ph��k�U��z�]��DIT�r1|���+��������������B�Xq���U�y��S�C�����w`c-o�G�����_�������dtR
{y;����T�i���?7��$kt�����	j�J��L���2�U���"J�(�ot-0��iNNHVY�(���./��j��	���0�������2�~��
>�O��8�c���g�>�J�\���~�}j��\�qo��B�}po�#�S�i�>��'0���0�S0n���O5H+�4�)f����c���c��2��B��@�T��9��	�������R%p��g��Y&r&F'e.��o��Rv���~1��y�"�Z;�'# �X��H�:���C�����c�)��ZSAGe	?&+8�z��G_�	�����%��������OM7.���Ws:�@7��K�;��?:�'�4nwV��c��_G�����V��e��Q|LVS�%"��Y�8���|_��|�"�6	 �~��(�J����������=�[�qy�%����v�O�x>�D���Q��h�f���Q|���j
���`q(�����&��*�@���~�i��G��`��gI�KN�!>V,�i���f��A�E���5��&��3�������U��+~�����zs6xo`���)���/U�R��Ot|�.&]�P~��
g]v������{J��fE�U~��(�*�r���f_�
�VA�a��h��%��9�T��8���g�����lA��:-�H�2lU�n��?��y�[�{�i���kR.�c������O=7�,��RW����!5�����Wx6��q����d6b�����9Uu�5}Hl�_H�yP5��<�Z�F�
�/��2�x�[��}�k�Y���������vw��	�����R�g��(�@�L�TW4��?6t0
��X�J�%��`����~�4����n�1
x����F4W�-�����P5\w�y�����:���D��4w��,��N�	{���z�e�����?@Uv�G�uw[���@������y�E��r{���������������k�3�+d�[��5-d����j�g������$�c�`}!}����;����/��2���az	4j���iT�|���$k���
����r/���+B�����6�-d@n�:b��Hz�OQ����6')�m��.^UnI�[�mo���l��g�!����q��7�x���c.��P����7���U���U�hy�������e��p�+��r���f=�W�_^m�Gm�W���e[�5l9k�JV�"hh�m7;�����s����[���Ns�w�������%�)�G)K�~���a\�<����&���lx�|���J���.��?�8������g
�
��(w;�p����;�	1.r�+��wa�z-<�(����K�p�Piv��Y?������p'�J�*IO�g���*���'
��U����H��<�?98�V���U�E�|�J�?�y�@��l2��:?Xy�q���/����z��{N�����Z�a��>�)��P�h,)�}���?���'2]C
g'@X�2�O���k�������j�!����(�]�u��v����Q��N,��}M{�6���3��,�)��5�:P�[�t@o{Xi����?<�n�����(���(�M�O�o�l|E����l�$�k������w��k�O�e��3��b��@����p||t<P��<�u}�dxLfv�q��K�w�k�Z+������?9dqkK]�����$.|��u�Jr'b{#x�3�����T�4��u��5$���m�_���878��cZ�������t&�n,���B����&��8���\q��B��z���?��8��������jw�y
-<
��#����|��������5��D���2��V���a���Y�t�USkU��T�e}11�^g2�**^���-NP��PUT����|�7��c=���1|����
����BG��\}��ml���E�w�5����g����_�U����5�*��'gyV��IM�p���\������}��q�-�����je������p��:d��kBm	m
g3�7R�����&z���*����-`[�pl��NQ�;�B�Q��8�Wl<��%���T@�����q���n��"�Sy?��6���Q�����-��0�/O0�`�.������f�@s����CG��0����Q�V����h��u&\4�k��S�����V����28�*z4����fo��#k	V%�N�g�Xl7tdVu�V��.����hA�C�������V���	j�������N�<��h����cTs����K�H�I� �>�N�S>�ji1�z���t�u����k���!/H��H�>���7������q�^�����h(�b�����VS"a���������U���u]{&�PC�����������S)�z�/3��k��������3+D�b�D+��,�l(����\R*�1	k�0b�]�
�Y���:��z���eUvo�����><�-�W?��b�5�n���n��N����H�N�^��uy �0��)p+;*NB�<V_���j��m��).#P8@��g|�������=v:hzPY*�#����f��������������Lx�`��rw�o�E#���������lv���7�y���1�������~���@�{���XI�
t?�/u�{X/E_4
|3���@RQ��>l��d�2�xu�}��r�3]D0�[�t�%�mI�G�����0�1��.������*h^f�9�4�����;���?x�gT�/������L��tb��i�2j�gC��������������ML�����xb�nL!���U�.<E~s���Ou��L�D�����q�q�u;��_�����[g�Wo�������>�%��wB��j�i~��$�+����2*wM�(�>2�l!4�'�jR�o	�	�����6��?	������f�g�%�z����\����6L�/R����)��ZDV�*��[�[(P�Xn��X�g��e�r��!(#j���0��k���F�0����.�44��l_b�vzGP����N�rZ:�v��U��S5��-��>��K^{�=`oo�^r|m���X���Fpq��z�������4=��n:w��\���Yn��n����	v�[��i�;���S�u�{�I���fy�v���u�[����`�T��{��v��!�ap�����z3��d�"�L ���r��.F�K�����������/`
�t���p@����l{^YG�j�S=i�@�q�����=��S��%�)�[�&�b���t������02�B#�(���(5���$���	�����Y������t~n��F��q������>��qm]^j]��b�PL���\�:�n�(
����l����j&m�Hd[��u0^���N��6B'y/3��o1[�������S���������W�\�M�N�
���y���w���g��5�W,����j��<���_����k��\�v��yi�V�`��/f�jEcU,�<�pzy����_���?&��-wcqu�6����n�k�fyi2��N?��n�4\s����.�(�E����r7�I>��]O��ZT��U�f�r�r�EH�o��f�]!���P��C5@�R��������#�R`X�d��m�O��C���y�����.Y_*��'@j��F�a�����/?��2h��f�T�m��&bXs���O���������9�yfM��g9
[�V���/W�����}4��[����+r�����{�q���WU�����0,����E�-��| �cO���_~�G���H�D��@G�_=Fn���J���r��V�~�'����G���k������R�������[*h��{I�H��4?�*�;a������D{c-I�@,*k`��(iwVt����
MIU.���#�z�����������-(�;��o�`��^��n�2���Z_��������bl���S�������>�D�����~f����F=?Z���/sI�����Q�����J�b:+��B����*���,��5�>�:\{�������4���5�;M�������&���uy��6�����,�3�'�.������xw^8���P���v/cPY����.�s�P�=^q9�zF�L
S�Ws�����Ou5,/Nj����D�b8b�\P���8E'ZJ��R���0Xo�x��O����i6h:*
t�el��s�[�>��L^���z�������Y�l+?���E�����[k��~����D�����Z`lQ���Q��LMeJ����Ac�@M�\#�S7<	�$+
���)�LI�� VtJ��K���h����'��\@���]&�����s�M��Y��`w��
�0�k��2��t��@�H:�;����`����X��%��
l����m�!���=�r�^���F�����f7d��gk���IJ+@ ��w���b��Y;*�xv;���Bb!��XH|����3`9t����� W�+��
roR�E��fH���K�>�I�!�a�Y�,@�Q 7s5}?��&�X��=��r5���J���vo���f��F
���6X����=���b��aH�y����
8�Q�1�5T(a���z9���I�p�d`W���=��]��{����X��J.��A-��W�AyG��^B��Z�;�_FoK�6�=����-�W�����j>�����d@����a��
���=5v�X�|qb���Q�������c`�I>��(<&�W2
��(�x��&�]b�y������2dl�0�}8�� �	�a2(����pPs@�!2��`��4%E�|H���5(��x����!q����s3��w�����x��p���s�Y�wH���2�(�-��a��,�T��t�88D#��Q�>�E�o����D���3#� pqp��@�H��p�FB����1���� ���� ���A���<hI���?������s�]�(�c���a�Va�V�#,	F���q�]��_��K�)�|
>��O��b���\*�
@�<@)������ �
i��B�$���AY#���/���k'��$�>m3f%�0i��_�H+}����t�e4�$�Ye�S������_PF���T�
`~��(�(U�OXD����a��<��T�D��b��y��GQQJ'�Y�1���VpQ�XP�i��t�|!n���3���U]b��uZ��^��CwA�`���_<����)n>��Y0��}
\?1���.a57�V14�iQ
{��Y�	e����-�n��Lut6+�|��=-����Z��g��_Mtl8a<sFY5��-�g-�f�?,WS���O��Eh������*�}��DSR�������$)�[A����.V6%y�p��%�0�:�����������T'��b.Z����d�q���jb�d��BY](�&Uk����"��>��{Q�����hdRS_�Z�R�_����j<O1��E��O�2�&�}������/��q*O��j���j�I�eG����y�l|]>%�3�Lu�����}?L���z-�*&��m�(u�j?����vR�}�g��FJ�&��<;a�T�|��;}<Q����E�Y��������NaO/�.n����|����.���K���� l-����������ZR���"cZz�<%n�&�$<�r����M�a
zlI9w���m�A(�&|�4�\@��b���z:=���<�gp�[\P�Wz�&�b��l�Z���N6�����UJ�[?�����;�
��.J5�J�]fQ��=��8��FX��Z�����#��G��J��:8�_J��0�P�A�Z7?K��3�C��\t�X�c��	�^,(FB��J����K��Rs�HX�\9Z��]���%^�H�L620%�n�#z��;���4��b����2%�)�LIdJ�S���z��:�c�1v8l���i�o��lY�z�5�o����k*��+�t�CJ�ms�g�����HuvZM���oeg���y���~�d���4~�J�{Z�GAr:�<���>�)�*Nh�:����h���s|��\�U�b�o�������y��mB�-�Z�Z4k��@W�+��E�nQ�b���S���)�_\�BkI���Q�����*g�����IT�g���8qY��xY�6�F`hU	:�K3[���Z�^���V3��;�|G��?:�K���|�i��%�#~���Bp!���#x�-?
���@����I?��~�����Pp(8�|v�?�|�)�c�4$��;	I�( ��W����a��r��O��_�]���B7_��S�)�����5���y����
�E����bNa�Hk���bH4���Fa���n�q����~�M�C��mY�2
��w���<���)0�.�4o�$[p)�\�]\r�,��E�1 MYG���B>����0�����d:Q��-�Tjj�(m[�S��E��8�L4��Q�����<�S�Q�>��W���|����b�f������:K�r���w7C���bb����_�XH�9�y�A���������Lg�9�5��hd�>Gc�T��A|l)�Y������6bWQ?.��Z��ll2~����Sh=���*�`C����@Q�
S�Q�/�D�|&�I�f����U>��-��'�K�a;��_5�1�����>��R�"JE��=Q������x���t�����:�U��`U�z����
���O��JA���.��My�e^`��_02�m���
����}�d��ta�s�UV�y�T��`�gz����B�H�q���t�
n��R�S��3���S1��bDm�a�P)M��`��y���\�!��]��?RF�Mq����O�� ����"���,0BK/�HE����E�4
�w�6T�o������6����r�����.�����G����H��
��H���`��vwE��
V�w��P�
������Y��M���E�H
$�w��q������'�"������_��@S�)��{�D?�U�|�x%2��d
2�����Lg�dG�'(�_L�BG������qT���Q�*����co���P�T��+F��7�a���6�
��������c=X��X���TH-�R�ER���z�g��F?��3J|E��i%���-�>R
)��B��HJ�������E�"����a�f:K�|`�Z��%��PR()�����Yt5/2�l
	��B�;H�7[[w�Zg�e	��Wv\�S�Q5Q��J��B/�(�1F��	_i�=,�w�k��W�y9��t�.���jbJ*���~{����~}�l��9���c�>t�/�-t��L�������`��|��p������<7��7<��.L<���
�.}z��Dc.z�����m	��?�����Q�W8��?�R�����j�I��Q��u�������*����7�Cg����}w�����������n$Wv����Iz�������Nw{{zx ���H� �������@�e���]'�v�A/�����(m ��6i�mh��.��]h@��~v���b)�R�����B�mt�[O��yQN{�
��yoYSg3���Lc����!Y!���;�a�0s�9��{O�y6�����f3��	��8*���8g\�|� ���c�1��q�pl9������e3����c��G����e�M�1��i�4f�!��3d�Y�,C�7��e,mNE�V���!��.�	ap�����0���~/�)4b���/�T�'���^�3�Nc^0Mf�}S�������?���������[;e&.����e���-2��Y��e�2k��7�n0i��LZ&-�����;�Zf-��Y���P�>��Z�e���U8k����
���6�Y_al<����A����pf83���sf4#��x(�q�D��"����]3d�6S����fj��1��A� e�2H���O3�A�
zE6���q���De�4���5�BD�Gy,/��"�s��	�<Wz���W.���c&��lH���$c�Y.�8+�-��@���I�i�������|��=Ss�;����h�l:kx\�[gLgX�4�!����C�]E&r�5�Ge�qE�1|Q����(��(Z�_K]`R�L"*�|T�"�8T�\��P��h| �'q�4Q����z����(���2�y~k-����F����
�)�|0��q��:U�po�Y�K���Kj�@���Z��^�t����K7�n�O����Avp��E�pbQhV:������ C�!�����������n�v:8�����W�q}�Z�:)h��Z��z�/�Nytp�gd22���O�P�j����/U���!dHg�o?��p�"���{���lt<�tq������[��b1�����cas^E����1���A��'�Aw\�"�d
2�������Q7s�9�d~�\s{�����c�}���;�<=��mWLC�!��ix�i�����hc�1��=����A\L�4�����&2?����v���mD��&r����3�>#tf�-����I"J���m.�T��5�����n��y�������C�=��]���=��F��F�sI*hD��q����V���Y�����T�B��ZH�p_���8���T�s/�����"/�e���zw�c~j��+��T����''�'}��Lsf�4����P�����O��>]��e��=����c6eu^������������G/��������S�m� #�*E�q���L�"�R-p��8��>w�ey<�S���ix&���0�mS���_���lt�^
�Az[��9�a��H��<��7����s1�S2f��tPv���"<����f��J1��W0�M�����?�����Oi�:�)K1T3�3P2��~��������N��hYI�1gp[���=����o�1�pA;�<,'��R|�����n�k�C�A~��f��_�{�(�
K���3J�=��2�dZP>��F��j�����;�n���
��S��t���<W�Vo5\v�^���i��x��G<X��b5��LV3?5�.G��%�A�C��@�*�#���q7!��'�}?�6��Q�eT3�������fno<_"��Z�����e�2x�b*�.�2)�����	/�����e�����Kq�����e�2p������7���\F.#�����;���h��e�2s�����f�Y8��������e�2x��;YI�UQ��X�I��e�2i��w�3��b!'�D5vY�����k���`
ln���J)�%��Jl:4����,��0�����
_E.�v.��h_ \5��r��l�
\����������L�
��$f�=���o��
�o�p������Obk^A�0��[l������}�A�o�������m�
�������!�?�M��q�����]�=]�X�8��W�r����-�wS�W������1�MQ������ra�._~�����J�}�|X�@������V��, ��l�����W�{��e�/������^��������9�+�>��t�~'�\P���d��Peps���s�|�wndG�930��1��(����*M�M�Z�K]�1-y
��
Q]x�:�n��n����
����������������qn�}W�5����k2���5���x����]�u]��.���h}��y{{`�T���kjA�����R�����he�Q�Y!����G�h)�i�������������D5���l���s�H�����7pX=eJ�'%�-5>����[\�=���L3�I�1	����b������������Q���(����Y��U���j���
��x�*������
�����#q������O���R������Vu���
������L�7^�4I�����w���8't������/������
�}�=/�����c^<�|��-�L����� �rX�*ts�~M�/J??��z�b�3�����������g
3���La����f��f3�?7�����^/��3���v����e�2z?S��������^f/���c���30���La�0S��)\����>c��f3�������p}�Q�(f3��w����?+����e�~n�����kQ������7Q^��z������*m�
���%o~����^}.:p�/�����J��qAw>�o��f�
�i����b��a����o���%��W4|�E���E
���X`=���������8X�%s9!q�������-o�gz2=��L�{H�ko`g�1��u�����kmg�1�s������6c3�o�7�����U[�p8������L8&�	w	w����:f��YwoYW������9�c�1�iaM�,��'C�!��c�}��|�'s���\c��?��{���DEg�4Q����>���*h����G_�o����X�bS�9�O;0Z����%��[�m�0��qV&�(�Z����s�$Q�w��d/�-!��.��s����FM��fp�����!����z���[k�|/�c���0���
^@k�G�3������R��en�f�z7����i_�B-��t��7bP���
�:����C���PyP�������G�����Y�$���/��8�N�+���(#��+�o����MU.�����z�������Gup��\�K�w�*�'�,.$���{w������������F��}�H��F�Fc�Q��O�����6��.(���.�d/Kn�p�dr|Fw�S�F�K���\`��o{��Wm.�2/bl*�O������
�������.<t�\�.��Lw�6l���;
����e�2}?W�������]�.C����B�u
�n�:g�2r������u����;��I��e�2i?s�~����[�-��3��G;��y��e�2o?[�����0���-F0#��f�
�=^��w��e�2x��{yD.S�)��e�~���������u|����5�nc������]����m��%���j����v�[&�����8�RN�t$�r�2���9�u1��[8����.��9!?�&x�^F�*�aA��Q1��Y���^���->Z�&���M�����?�����^���Y^P��%�����e�	��r�E6��{�z���i$f3.��������/�3����;,�>F�6D��da	E��*�6:PM�\��<� �c����|��n�� �Yr����8��x��e���T�`��+*�N�0���S���L@A���?*za=�R�Il�M�
da�q��|���2wMv���F ���X+�n��2�KA=��\6E�]����~Z�s�l�>�:�dF�pH�5I;���b[5����_T7��&5�+�f;6FU4u$�����m6�h�� `��.�n��=�9t��?�����iK~-6�C���B�x��HC�&��K6�>�������tt#��^]s���D���e�TaW��>'.G�zN	��C�wue�������_Y���-�#���,�V^L�:����[�|m�_�q������
�����<>���=�2a��k�)�����c�� <Iqu�[F
�u� �������J�[�a��,e6U9H�����v�Pra-�����m�6s�S��
�q�~P7��i�4d~�4��h� c�1��1��\����A� c�1�9a�me�1�d2?#�!1�B�gBoi�-7�Of!��Y�,��X����Kf"3���L��������q�8d2?'�������`2
�����W�(a�1��|L�O�|������n=�ez�84�|�����r�t�M������e��w�~�vls������dB�~lv�c�Y(�P`��B�����f�2i��LZ&�5�{o�,mF,#���e��l�:#���e�2bo�����3b��XF,#�&�]so=C�!��e�2do���k�zWZ-��A����R���Y�,s�9{c�^��
��y��e�2o?H�]����Q��e�2j�i9{���*S���Te�^��M/�����^&.����e�~��AG��8�\��� !?B�e���@���%�$5A�!o�c��E� ���;{F`Z3���L�����N�Y�,C�V�v[�"���6����e�2qo�G���!����de�2Y�5#���e�����F8zs�*,����|e�~�t�Y��,��o�,S�)��e��������R�\0f��Y�,c��0k�g-g���A��e�2hy.S���Te����Z��}�����I�����I������5c�1�d�C�I�+�-_}n�
!T.��\0��8��B�,I�YZ���KN�,��{������L=�s�
l=)}�Q�+�J_>,����i0��y�C����^����UX��a,8Xp�������'�3��{�=�����u�g�1�x�{
������c�1�x����h��c�1�x���<����c�1��7������.��/�c2���O���x�>����	����Pt�s��>F����	�o�2����������c�}��;�v�8���N�?�������{������u�?�����i��6����?�����c�1�>a���
�|�%�����c�}�����.6��
�����c�}b���i,3���c��k����x��!6��^���[[[b u�<+�8U������HHq�1�[B�g�d��\LJ]���P��X	Lq/K�w}�}<U�,�\����P�4���zEE���k!�\�h�
����Y���3�iv������{����+|}��������f�������'�y�.0������>��m�����h7�gk���WD89>\���-�^�m�6C��}c����d�V+���z3��n�����e�2e��L�P��*����i�4e�2M��7��Z~x��U�*c��zo�GoYUe�2S����1�,SY�- ���Pe�2T���C�6�B�������he�2Z?��	��ku��e�2`���6�},_�^���he�2Z����zL�F+����he���W8�a�2I��LR&���i������z	�y6	���������M8��!62�<����*����8��_j�c��T������+���;��4"�4�%$�rs�����8(0�P&a��Bi�x����,�����\i�4e!�a�G@fD��'Vy�����.LJ���mE�Td�,�������I1��r��m����,��s9Q���_MY=t���������8�b�$�����r�([�]!V�R���\��m"
�+���fb`�[E6�\%q�)EP<�0������(�E�`�_��*��f�r��wj>�a� F�PZU��"B����M��x@E�i���9;y��/�����|�$�0��E�e���f����$[6~e���98��.��8B�,�I"��	�����8�^c����=!���>&c*���Bk�������}6�;��b�����YY4[����W�%5U���<�~�d�9V!�,��p��5@5�C�\�D���K���]&�~��b�$"K�9��,��Pr�<��\T�����]�*���i�*��$G2N5��i�u<H|P"�`?c�����9�b,�BH������>F��.��:}X��X����czU�����w��^Myq�w��azI%ir��VS�J�U;�X\@YB	�<�p(!H=��9SJ��`1��*v�0)#�\�20!O�TUma:�.e��	S���5�2�7��W@^=���fVS72�����,�?�@,.�J��3�'�ak�]��;RcR���B^g`*2C�gT�p?���~��b2R`MV|%��'�x��
~md>�~�\��� �o*��O�M�'�V"�'�G�$T{H��)Upr�
(����#`9�3|;�?����e���A�B4��m�#�:v����*0����	1D���yD��K�6����)s�?7L�Z�-_]��>�8�b�.��G��������C��M�n�x��7���������:&�V����>W4<�Uj�����B/��l��^����������:�pe����[��&�����	�`b7!,cX���as�2���)�lt@�=9��}�B@���t`�x�F���VZ��oltmnt���V�����O�*Q��w�������qSs��2L����(6�Ra��r���
�7���P]_B��T&�����M����l���1���L`&0���wLui��r�B�.c����e����}T�\f.3�����S�66Xu�j����8L]�.S��������"�c�)��i�4e�����,W2w���-��LT&*��z�Y��6:��@�����sZ��'��no�DB���1O0l�[���`�Y�h,6U����z:=T��Y�,s�9���P��R��VMv ��v���E>�
q|X��q��e�~ni�Z��H���<)8� ���pe�2\�W�!�����������x(�*�z��`,up��mm�xa����e�2wo�]\#�
���W��z�g$z�s��W^��Ma,/���D	t��N�O����g�a"K�����21���H���I��j	ix~
�fl���O����P<y"��k����Dy6zG%���rr|��7, �t:i�|�Ca�1s(�z�(��)p{!}���<�����.�j\�.&����f��+�9��v[&{z"����e��N�&�����
,��;e�v����b!�B��(Q��#Dg�����.}'D�q%���r���\I@�~wG v������i�crh���A:W��)]��/]�
1t�	�E�ZO��0�0�*� �Fc�#���pt�k����&�+��UZ����o�7����Z�!�Gl����Fs��0�^��7(��Q�t�g�K��F����u��B�vp;���D�N6�3r�ehS����C��Da�I�p���W6����AVG��X����Qk�+�(�4���������������z�wFQ�T��mH��7��L C�8��J�Yn�THT�:�\;#F��������Y�w��j�x4&G����A����45i�,����U��2�z�]6��>K=��,F��x�6=u0'�i�n��L��KlrFm����5fh�/���K
yn�0���K�6P@U����ho��.I-��0
��Z�J�w��,J�G'��w����,����b_MN����|]��9���/*�84g, � _����1����@�����]�������C���������p����N��\�gr�����G:Q�C���������J�n�g���������N�H��8�`|����g�@�a<*s,ED���2��=����*����W���*W�$�lG���c��\%s�9H��s�q�
J8����d���Y+�fm�`I�-O�y���6���d0�V;|8�A7��1��C��qz����Os	�%�3�I���r���l�W7
�Q�*xT��
U�dTa�*��t6=����l���e�2d�7�����[�-��a�lW�+2F��Q�(ct-����$�����Z���bf1��Y������9c�1�e�2Fo����bW�f���?=v^�L���Jf%��Yy/Yil���/Z
FY�d�2P��ko�8�V�d<]f��$���0��d8�L�B����~z-mUenLm�\����(WC�� �i��r���1��w���z��
!Q�C<�b����)��fk"Y�fe��6Z�[D�E.��R�	�?�i��^�i(�:Fa��)�?#��d
�>�n��������d���0>W����U�-��6��q��.��-����e��oV�����i�{���,�E,�X�,��ha���p���y���dxW�2��2�m�n/Vo~`$2�����Dr~G#^��������wN>���������L"��0����?a@�����L�K�n��4�a6������p* �U��M�6h��e������#�����bP���2���O�$h��j�����"�Id�phQ��"�9��0��	�\%p�z���i�3������_2Y��LV&�'F�/����@��q�������V+}L5��w��I
���:e3�W<C�8f3�?1��	<��L�~��s�{�E
gi��@F #��i!������^pp�w���~����������S�?�c�|p����o��.���pwo������'��)|ep��<��xe�2^?1�Zv��a}s�4��#9PI������Y�!��y�<��x�������tc��k������
s�����y��c�1�>1�������7�����o7=�����c�1��/��$�����~B��#f������������3��_yt��c�1�>1��$u�ON�O��A���8d�z�N�������=��r0�<C/inCu�j�B���� n���Ke�k���y\ ������{���LX&��F�/bL��W0^B9�!49��&jX�Rl���c�1�>1��f�F�0��IG-Q��e��g�E��x���Cm�����
an=����1�^Zr������]H8N1Q��
2[;��]�y[�!����t*s�$*	�����VM�|%e��HJ�����G���S[��*�Q��m"g����e��{'�3�Q$r�-�s��"���O$y�H��<Wg2�b���pF%��Q�	��S^Q7LT:*�V_1�y�<F�'�<<���]�p���p���NUu�,_��e&"���i���,h�k�G����#N ��h�E]���aQ�!��y����V]h
Ql�C<L�����0�s���l�rYd�9��_�b�w5��N���H�&	9�E.C<�2Wh\��B������{��)0�d��������Z[�����*lyh�)3���xF��N1������m���B��-SLb&1��I�$^���k�8��?dG]�	�z8�`N��Mc��m�p`�2m��L[��:zocO�$�
I���?E�?��?�=��uS
�O���*������:�������1�2S+I9I��r4�"�\�u=?��%����� E�o��V[Os���q
,X4�h��h@�[A�)k�r&,�	��e������&����Wn=UaPE`�2h�Z��@��I�_�iK����"(V@�21o���[�-����� ��?3K���Rf)��&,E��G��De����2)���4�8��w��e�2s�����0�Z���.s�\e�2W�����jeT`�Z]3]��LW�+���t�
����`�2W���U��
�Z��4��`�2W���U��
�������s6�4e�2M��L��������
�I�9*����a�E� ��9�BY(!S����/d��P�%Rkab��,�1n�=���B���p�/�!<5�D�
{�8����}H�+��,W�(�DAz����&�*�#�p{�4::��2���h�yw)��*dgg,*XT��`QqQa���L�	��W!'Dla����m���n�c���;��X{*I�f�+��� �8��|+�&S� Q��@��^"����J#c�fRu��
Lx&<�	����g��+��X��Kq�w�U�� >�O��"f�2f��Y���t��v�U�fO����SP�Q9��1d8�&a�9w��'�`F0#���	������d5A(��DB�������e�2yo��Vs�jb���C�~V�W{��%�|N�����;��*���,BZ�+��0	�YZ�8�B�O]�����"��+aJx?����t��UJ1,�b���,%�(�&�sO�X�+��~F?���#�w�qZ����_H�!�8�7�$3R�Kr��(T�����_2���lF6#�F��b0��S]�*_��G��z,���5x%�z<���C�Y���r)?����
���t�!��[�jD�A�m��[x�&���O�o���J�?U	r}x|)2Zh��"��bI!���;�=�����Wd/�-'J�����,]9,p�����L���g�Kf�
O
02��'y���%��%-�t�d
(-Y���bJ�,s�9����Y�Jw]��4!��r���<f3��w����7��2�4Jk-��$����`e����LZ�Rj���R)��Az���}��9<��%e�$s�9���w��\�M����M�a�M{��F�{B��
�����i"�j��,����k�{�<���F}(b��i)1��X�c�DZ���,DQNe_���S=�����zn�1�Cl<T��f�Y��c����>��V� �sr����� �i��2o+�n���w�����������6������.�����U}a,�X�����m��E�q`��Fg�{T�
������8"���c������� ����`�e�ZV,�3�$A6��B�����% �I!5m�!��n��'�	�Z|���%K
�,)XR��i��k�5��DQ��P&Z5o}�����Z��T�f!�6K�q��g�Q6�7��(� 4�B�b�����';���`�����������h��wde_4�j�
��,���t[�a;�en�	/YM���x�g�t��
��m�O~yy���.���hN������q�������|?�FP[Or���[�s~\p����
�������T
�������y.�Z�dZ�IG�N�%��D��a�|H&HT:*�][�-N�����AST�C{0~L j�OzS��?*z1{���x��2�*U��*��	��V�T�.��U�a��-����wV~P���Hk<�����\���.�~�V|lg��H����0%��������J�>�k7��>���$h�i�"?��������S�7��H�n���5���v�H����+�*��E������_�4�����8����nz�����*Q2��8�P��,+E�C����_Z�����xHr�<a����x��WC|�������1�2��!�s%b�G+a|3Pb���Gb�Kf��p��M�b�dR�b�f�r����
z1+�9H������P���,���}*�u������w����!�bJr:��i�k�����Q�nC���P�iZ����|W9�p��K�*��j��c�:����zq�jV:�*vO�������~=��HgW��;oG��������"?��*�����q��_�`���V-���_ P~���x�Gy�l��?99>���D�����.��C=u��V�������W	N*�Jn~���T"�7�qI�]K���\�{�x&�c{��j,�#�1��1��=�c�Xq����.����Z�>L=�S��w��A���k4G�D|���z�WL:&��I�i�n��L�g��^�b�1�{����=K=��;:h���xc�1�o�o����9��c�1�>)��h�cA��\�!��c�1��#�*[\�`�1�m��Om����#����a��z�����
]���������"��1��i��,#W��*���oCr���Pi1���"sZ��-
�_@��t@�}��B*!��S���>�[geAGd��_�bL��Mf2������b�g9H��V�������l�"c�yv���Sf(��mp�����1�eya|yV����0V��Bqce)[F]��,��px����d���k��C�+E�d2'T���(�Ee�o���4�����'�&����(z����_�8��8�I9R�C���j���
LO�J4G����I�O)M�2���r�"4*�ow��-�L��P�E|��<45J�>�Z_�B�k��T�\$
bn�����G����2�����)��)����K�26���&�$����@��g���uG��()��������P,KJ���9����SWT���<[�C���{vV�������'���tu����	cL+�7�B�E9
0�R`�2i���<����Yc�$�>���Q~���~���^������_r��������t�W�2IT�6�'fck-������[B�~
P��������
k|n.p��e�co���*����J5�/����,H�� �������+�I<�r|Z���FsY��w��I\�]��MzU�f��&?�
��[Em3��������@����]��c�'2���k�7�"&!{f���e$�m��
�Z�L�W&�s2�7�m�s��2Tf�T:22�!��z>L��n�y�F��Wq����k��'��^�v�����j�k����&�^F�_>�z�-��"�
�C�\V�2�_��?^&�m_��-�>j���2	]�5���v3��M/+�0�H1��h<����!���Q_����D;�����dn�������6p����5�k-t��Y�z��wk��N/�N���]��Q6�F43��4�9H#�x��j�4��������a��4��!Uqd���@#	���;j����5��H�[��0P�S��l�����p�U�n�4�7��-C����G%U����ZD�7�%��k�0?�{�}W]9�����n"��U_\7	_j\_'���@*��5�����}��o�/����jq���N2^C�����:�T]w���k6���\����A�<~�|h}����c9�S_�����o�*B����f���s��h{��7�����6��A���6*"6�w����@ �}����l��
����e���b|y��n�U�RxI+��	�l_�Z��D1�5���u/��a[_��J�x8'p8U^g..��|�cY���:�I5<fO7������R�(�P9M�+��
�e�����$W2���)����y���M������=������*�p&��d�V����,���K��|�.��^������j�a~���Zvq�w}7���`=�|�>"�����u������x�X�:���������t	��=P;��b����9��U BW���{�
���������������f��e�=���%�^�����sUR�g�&����ts�t����n���tNWW���c����Q��s���z���mP����n[Z�������I�b��G��=b\�����Di/��G�?��
z�d�,
`;�N����$T���:W���:f�������X�z���T ���
�b*��->|��Xz���F~��X���Zm�k����^��/<WI���j��/Z�1�X���a�d_��d�b�V�!�����A��J����)E<Q[!����/�t
^�WSQq��@j�po]������r�Vn���p�+�>	@4�������I2���=_s����[��>T 6��*��
J�p����+e���V	�V/�>t hKR����Az��
��N�D&\�T��<����`�Bmx��y��C
�T��0��b���%#�i��=���O�F8dh�^y�S!H�0�\d�X.Y�����D;��;��x��R�Bw��1.Z%���I�&��+�����������vN��4S&&h"��B�8����Icl�.>a�E1���oF��\P���J4e��2h�7�oX�@Xj�B��y`�E��@���Yh��0�/t��_���	M��C���|e"7[n���E���ZF-����I�<��~C���?F��Y�u�eH�
Z��Y����
�����1���cc2hRY�x@F��r�w��oPs#��y�j�b���K[���u���kKy��q��m�v\����:��r��$H��*�2g[F��>aj���MO���
��M9a�T�Xh�kl�_hBj�l����f�����]�����3�5Z����5C����g���<��Ct������9�UXTJO���N��t��
CAP���IF�`&���4���p
E�H6�I��mk��B��-�%�U�Y��j����gg�������3;���Qf�G8�3iX�S�[=�����r�S��S��S�����\��|`
0[3�o|��G���4�~�Q����Q����������]g��������<���y�[/��_����K�<v�teg��N��������;4��e�����x*X����I>��&*���n.���L���S�S�-�L�;�����<�78�y�d.�n#��xl'�g�����U�T���U#N/b�l�n��7�����?U���g�Mz�q�*�i�n�M"N���PY�f�4�:�_(���a"Gn����Q�x7��hTZ��Q���6P
��}^c�y��9���I�y{3�--7]y�#��ot������p�I����oa~��&�QfV��*�M��&�[g�[�#������0�z���D51k{��Xz��v!~��]�
��s�Y��
���w��Q������Lu%�	�(�q��9J���(As�Op�J�?�'hj!��X1�
A�d~n����-��.�����j���[��h�r�h!�k5IZ�-��UZ?������c�q���4K�v����\����d�jG�������C��C3|�M��I-��xo-�ns��i�X���b���F�tk/+������:��L��k�d��8�����F�V�.�����������6�V�y�i����Ig�~����sm�������k����������������������=���Y#�p"�.hTI��A������0f��Y�������31��LL&�gL�km|f^2/��������l�bL2&��I�d�a��q��d\2.�q���}�%��a����aY�0�V��&������dz2=}z^�m���dD2"?cD6
��y��
�����m���VKy/�Z�{I-��t������hv�]���p���K�L�(�9HT�_Q�������mIU~�3����l>x�D�oI�}�d����7�8�M~I�U��{b��v�M|Z��_Wbp�.��]h������x��Y<�xd������GV{���������F�t�D��q$��Qi6��^NTb�:'�y�
�J�xv{.mtp�T��YY�������*<7��f��Lq:2w��v�{Y:L���GF�Y�����(�RiQ.{�-�4X�8bq�������#d9�}���X��ZlG7��b"��g����5;��C��|�Z�31�iZ�%.H%r���;��������g�3����������P���0���Lb&1��.g��m�54�6d�/<����
��a�0g�3���0o�������l�����n��g�3������6�'�^����f3��w����0z��^F/�������)<So�b'	���CkC����"U*R��!�a-��fp3������j\��{��L`&0�	��C���><���f3��`��xa�$����0f3�������Fu���-�1��q�8f�=�����W���a�0f3�?"����'b���� ���B;f9��H�h�Qfl>bN3����i���s:����jf5��Y���#��\����y>����e�2{����Pt��=?��]�.C��{���m���|T>�wCk�v|�w|���`�,8:>�g5���Df"3����K8���x4�����e�~|��<V9{nc3������`������;�Yf3����`�.���'g���^F/������x�=c��_�/����q��~����0������J���r����k{��5�?D�x�S��&�)�B���l(�4~[*���8B�4TB%j��B���b�g1z[����pt�r���}|$���xP?��*��qH;�����95������,T#J�n�C�<�	�
��p����T	�+����9HTo�n<Z=�C/
������k����A�_�6���Y%h[8����w��������,�Yh��f��B�.���	�<�V4o0���bF1��#�8����9A��d�2~��_����WM�����~�O�/�����e��!~7h^^l�Y.���1*�c�(���R�\�#��s�}!�$����/��,���	�Tyb��8���HD$	O���aXN�C��qZd�:NYoo���a4��/'�@`����G�(Fg~�$���<p~�x'�����e��5c6\g�2j����F-Nh�p�M���y0����f3���h�N���N^����\A��k�a�0C�!|����f'YY����h�������;��MoA��,�4g���L3�cZ'�)1�i!�L�Ta�7�l"�2�zO��t��B�I����-���:��"�(��FY��cy�����U�I�|�I&6o�<���!��Aj���Kr�[��k�f��P����&�AJM�������	J�	D�<U�!������2(|������j���_U%������q���%p���SO�������r��?J|�%M�,4Yh���K��i������_�S�Db3����;E1�"u��f5�O�yVb��,�z���
���9z�/�u2��A����X��h,z��P����0��Y�(`Q���E�]�w:�Z������B��i$�<'g�����������a�0f3��x���X����������/�O�^g���y����@f 3���������b����`T�:�z8��&=��@m�6C��}��~����s��1&7d:���/�IN2G����z��|�DW�TB�$��a��+`-R���������/�_�<��4C�X[��P�5����H�Z�������7I�|p����i��&���8��
�"��1����6l�@�%�#�<,yX���a�s��e���[��0d�2~�����������u"����2�pM�����(^7�A�rY�5c���%1��� �]6�c�Gn�@'a���dS�
<�C��e�������'I�XL�W(���:K.�\,�Xr���j�!6��@FQ0Q����-��-��h�v0��o��Gv\2@����oz6��.���7�[���u�fR����,�Z��`O�t������6�g>3�������s��?3i��LZ&-��.I��QJ�k�q�Q��Z0��i�4f3�����}���v��eE���Tf*3���wH��N��2Oi����e�2q��wH���67����y���m�A(���2��b���c��q��)��*�!�XM�:���"Q&���7p�%�y�X��.���?������g�3����mtm�W�.���(�S�E�,��xD��x�Yl���,�(���=,hS���E8]��S�Sb�3���pF�"�����@�,�I��_��W�Py,��i����4~[*?\�S���}���+��~f��ux���Gj�����'*,s
#xq����0�<���|od�WV�2�G�����T�1�#��h�E����;���k���W�8�Nm�k�k��f�2-�����h\Ya��otgcnZA[��AY�7:9j��t$J�"q�j�+�No�M!*'��@��a"���q�K�m!��0p��a0WhH�<��4�,�Py�/�t�y�1)�K�,�2��<:�9�&sz��Q����Q�l��y�5������W����=�����[��T�X� XK`-���������k�a����&)�Y9�:���2N���������i�$O(A���r��NO43�8�]BbI�Q6�d@>%2T���F����>OG1�Qh(�0�������x���>N��srk���M���$b�
P�t1���SCY&��t���������#Fw)��W��E^��>3����e��%s�Z�e���}\���}H�K�l����!��Y�,c�1{�0{����39�2���Lf&3�c�t{i��v���,yS�<c�{�,Y(�P��B��9r�S�)�����{��������+
��~4Ro�@]�d;���Q�F�vGA�]�\��������C������>��\���C�}�~���������������)a@�CV&e�g���h��%��cD��=��hs���lZL��	a�~�8���%:��:2�+��s�aO���e�y�cA�
{������>��4LTy^&�@&�9�����_�Y9�<�o��D���9�����u\����L�a����y�y��0����lZ���?*���������*�-Z5�xD6�EJ��b0���{O6o�����$�o�7����c���.�����[�Dq%	9��>H���?��B	��S��N#q��q�%�h.�Kz���e�]�[�\����#��/�e_�T�D�sz�{��|7����0)����=�����
�L�����g�t�������`��`�ma�fXH���h��s����f t�D?��	�����+@���V�Lr%������n��7�F�d���XL�H���0�\�8Q���8���ML���'a/���JhU�����M�P�>`���DZ��,1N�e�E��P����b�ll�>h��OlNp�-�X�~���d���7�(X�M
b3U3�D�����r���K��EJ�9�N�w���a��_8��������x��]j��a	sB�9
}]S�I��Bj�IPQ|G%��Rk�MKSZ�6��!�|���N�>D�GP���4PB�+r"Fh�%�g��i�H��1����������/f�~��Q����_��9-�^@��a�y�&Y>Gz@�d�_;�!Y�t���mt0��0.@����1I��JG�n�v*~:�}&61d,����Q��=h�@5)�)���92km�8I�2�%��f���Jnf!jE��:�~Ww�no�P�S�����n����'b��7�YA��)�������D��([6��L���h��R���n����F���(+p>^s���MB�1�c��qU����C
m���x�.���7PJ�@n���=G�*9�+.b)T�����rr|b
zU� 0v�OUa��TLw���`�F���f�x�r���4n�������_��9<N���)��a�����)-��e�*;=��x����|"
`��`k�z�m{��H?��
d_��PBiV8�RIJ��8+�R��n%�G�������B�@M���D?�R�S�C��Z��3�j���d��2W��\��}�!El�fW��C����I�����7uA���=����A7K����L�k�\�V�B�1�[����yZj]N��64��������p��[�U���>�������)������#�EUW��>�T:&����Dy���f!�t�<�*y�}oz�o�3h�q������H��Q�B�i�`����@�|�+����7�F�f�no����8)H��-}��$�o����%��������L"�
tVx?<W��j�r����(���4�3�Zs�mi���c!�BC��y����E:�O������0��sb�t�(��@�0N�0�ha6@����K��SQ���6��H�Jjy@���	[����f%T���8���*��_~!�CsR�fh�����"���LF����C�1q���K����'��Z<J�0��
�P��:����x����$�D�2�����(``�m��w����u<�N�,���R���������4+9��"+dB�t��-#<~�=����A�f����aKu\�jxe~b%���W}��8���b%�M��!�<(1��[Z�k�k���{2��%���nS����p��T�q�x��m#�[��N��2r���?i� ��*���*k��Mf|���Z����ad������(WU7$��h�S��XE&!�9�F��g=D�6��Z;�c���y�:���XEM���!%�����}��ZYA�7H�����(������=
�S�c���
����4{L$�3�f�0�:I�J��iB�Y�&��;�L&3979l/WWV���!�d��5,�P��9��1i�����0��������
�tP%R�j�CZ5���jP_!�zm���i�Q�����\��P�~0(�x�������q	�� �e�x�O@��hy�=#1�,G��+&7�U�EY�/��Em]if�1�vXKOc:(-SO��09��_cn��B�!v�'g�U��P� ����dpBaJ�{�/����������Vj��.c�(���/2R1��*s �I�2�#jQ�:�p���b�	����:R:i���t*_l��4�#���CL�1���X�(+(�<�0����vr�I��������5�hMc �F@J��?�o
��Z�q(a ��d�p�4���!us(����(����jw��x
?E���5u**��p�l�d���fv(O�ujw4"��zk'��t�r5�7]��Q`<���-��@;C����
O���8�%(�����,����*�.��|�L+��cw��!�n�G��4
0��i�I�S��1�TY�����.X���L����[SJ(K�Z�����Z�W�F��H�FK��+����+��i6�C�f��wq&�o4oh.#���'�Hb�g88����R��/�fE�scG��I��Gc$�iv�GF��fL��T�|:�U��
�H��I'�p�4F���ffT�R�;N��8L���<1#y��z�q�AYIA��VF�������-(��a���gB��!6��oT�0�/��c]��"'8uj��z[����,���%T�n�e�R�lE�<�������f����2�v��
�������n������jk:�����0�|��c~'��x��
MO����������4�g4����D���������l�{���Q�l��V��������P�8Nh0���B4��%������1C���F���V3�&�ra'K{�^�����m64�{�?�!���Wn^8>�_U��zw��������?#��YX+IMF][��
=�\o���1�����s�A���O��L�������-��B�t%��,GU�2V1uXA���)h�p;	� ���;�������\[������&!"�S���t�fX��)4�j|�h���f��t%���[�����x�1������M���O��;����P����F���Um���V$����19��d������| p,����h43v�������)��FC�C���0�,&�O�%;DG�Ns/��h�����QC���55������0�O�}�v0��8�2�����I��<�U��V
h��FL������h�v������</	�R��-*���A/dz)�S��(=r�Q���c�2��c�N���r���X������-���wJtl����O��f��[usL
�M���jl��V5i���k���0w�ccw2��?���|ai��@�R�U�uF�{�
,�+,
��j�M��Y�1�4��������B�J�����Ya�>5Uv.��;�K�M�|�I�e���j(L�
4��OM`��.4�'���J��U�4r���kq�W�~�OK���-E��W�]��0
9�aq���SM~��sX�SQ���3m���F5���og\���)n��wI�����Dm�5���}����L
����k������r ��L��q���z4���6%o�u��������.
�'P	\��.���7�(�+^�"N#���7����}��K�k�*�����@g���7qu���������??�;<-�S�).�u� /'j���B4�fsQi$�#�_B�d�������u�������%��D����'� ����2�Q�I�!Lg�!����R��K�|��Y�c�Ye�_Yt����Y������l��vq�Wk�Q-�+Z�J�X�E�%E�"Bsf��^��5X	���@`�9���6C-#��(��Fj$_�������we�R���(��� ���:�;���c��a����Q#p�25����$tMz�K������
���=��
a����*]��>(&�U��
N�����2��>wv�8�U����6�7�7�A[)�}<����d��_z4����n��!<���u*���]z�������d��*���z��L��.N��b�p�'�TJ��|��<B�����kM]������@�A�qX
+��)-��������;�ye3�a4 �he���fB��C��
��.�t�3���VM�F��'I$R&/�=��y� b��1~
�j~�V�h��l�3Kd��ul���������,����������o���g���7d�
������s;���.�-��2�Zk�9�������a<*s3[�AU��������g��b�����.��8e���"3�T�������u��d���}����n>�
)7h����cS�����y\����5a�����,�_S���3���NU����y�tv��`o���������
��o��]m�Ap���N��D_n9
MAQ����G���C4=�i����?O�_6#&���A��C��Z�d�#lqF�6�����d��@��ZN��f�tl��N�E{|A���2i�e!!����)(����Q�iG��S�GR��pEb`�����$_t!�����c��V��2-�<D��i�������G�um�jk�uQt[(��0}kzz3���&�
�6�G��Ua������{�����B3�D���O��M�K�M_\��$���r�62}�q�y�Kx*!k�t/�`4��s@�k����@������+�������3���}V�!����
���7�,"���$�\��0C+�P���K#i�X�b��
���\X�'Z���B���`�8���W�x�&+B�,\�"���fY����O.�daF��
���mCY���e[��P�-�c@�5W��Y��'����`�]�Q�uInCt`N�h����S9]�tT�����0 p���2y7I�<�(Z_��~�,8=�������
�P�j���F�;>zq�2xqp���T�,�M�QO{P����n��	���4�f���l4�����<���������P�����W����T�GF_��0 ;X��������/yT�Y]f.�5�icD�&�r
4�Cr��Z0N�����.���G�8�������56+_.����?_��O��NA_�@����L<�i<����f�4��`����3�F}�V��;��Y;�#����r�O6M$�H}z�9o����~$C6���PqY���^h�<e)���d���I`������<KA�Z�%��S5����=���N�v���'x�gv\����?{�2xv���������������
�a`���Wi�9-, �Y����0�T@
a����1��$3�flz����������h����D�\�Y#R�����*���EG"��(U�TT�.�E��8[�9�����o��l�'E{{�'P+-��������M_|�[x������26���$��}���b�����8�[����=�_���x�vE����m��]"�5c���/�\nn����xY���2�Nc��h+C����W��o����{s%/�fuo��'Ao��M�Z�~�r`�W3�����ke0tu�_���w�qL���nB�6�������xRk �r�~i�K���kF{0R
cn��sS>P�f�-hC�����o�:�-�}��F���d��D@���xo��1b7�\�����/����mk>W?Lz�_�����K�� �3�������d+��*�]�i�����Y�+�v\�IBZ'��H���?<�[L
t	PFB������x����*��A�.b�vv$Y����,���tk��F��S]u����xfFh!z#��48Wj
�^(
�k��pY83��?$2��:���n��+JjU�����p����&���
H.�A8IPS�h=�����b��DM>8M���w��4�>�������)����D�Q����|�~��KoZ*��o�N�i� caR��ej�S�i��(�9�X�*��$��U}�4
�0��;��F�3����Qna�������'�����u��b������|m:����;�S�]�`�i#l�V��h�S��]/��v��������Ti;�a&-��E�ESn]]|a�uO���m�#����2����KO�d������&��,�e�E��t��6:�=P0���}���g�������o��������v��>F�w��~������;}����x����ge<�~	�.���������E+B@8Y�i
���{�V-�f�]U�N0��$n���������z�Q�%e���m��%�A9���)���nSD����Q|���v9���r������ra_5���8
h���\�}p����84�8�F-�`�Y�3����y���������,�0re��*���=�=�_6��B�����=/����Wo��|���!��]TQ{Y�8/d��.���U�S�t1I�Tgn��J�������������'������4�A�U��?�O��6�N��Y���#����>N�v����<oV����|��+?���1���2A�H�f?F��%�F_��%�kp���ev���c]5b���b����}K�wi�+p�i��g�������o�/���f{)���:nVh^����-�X�{�U��-p�N_���w�������g��4�H7AgH.����}�~;@{��G���A��������Wgg�-�.�������l�\����4�%d�>@6N�Oq�?��Z�'����'/�K
���){c�q!�l�C4Iv;Mca�l��[f�\���j'������(�!�k?����o��o������]���n9�{?�������p31Ys��D��7j���u���J��b������/��$i��>lA�~kty7I�CS�B����{��_^>;8�=�g�l�t��f}5������v�L��sF���0�=~M}k���l��)k�,,'����y�s�H�:��6�=��"����1x|��PA���$�c��f�>��C5-������:Z����I����mu"�8��Z�xh�FT��NwO
��
������>P���"G���IISK��7��+���b��*�:������J������T_����M����h3fk&:��;��i�eu�h�q
�4��e��
����'g���[Mk��������Lu��Nv�%��(5�uR�j������Z�-��y�d�Z�QR�	�.�7�;my�Z$+u`-���]�A��h����������mc��;���gj��9�^�;��m���o#�I;�y��msG"U������~� A�(;J��{�i������u9=��kA���������V���e�s���}�;(�v��>??��:�8����{���X��uU�g��
i~/�����gM�����	�BK����>����_�g�5��'����#8�����_S"�~Z����A�g��(�P��.��@m4��_�}�l����������&8������o�z2<>���_.S>��'���uP��w��������V����,�����g���_�3/In�x��c|�w�$������B��n�1�k���0�������e6A��.E0����TpDc�g����*��AXuzM�/I����?������xMF�7�In�����*��������=���Y��:6=�����HkI�����W�������,k�%�G���"�u�O��B�~����`F��?���4
a���!�����|;�8�7H�Qiq���7���]��p=ml�y�����8���C���
����.z�&��
������)������u9�#����N�z������{���D��-A�ib�@���Wy(?�e���B�q0@LYd�%:�)bk���.�YJ�AlF���w��|����.���'�����F�a��q<��4�:>c�#]��R1}����S�\-�]N�3�
����bd��v��+���}rK��[:>;���>����A��5f��ah�g{/�}6��~����b��Z/�^z��]4
t<g@��`L]��w�i���9�UC��5���T�k�Q����������������0�!������m����kc��D�#����F��Gjs
�:>�8�yC���q��b��A���|������#Mm��Aa��b����S##��s[zk��E
)�%���:R��dC�Rt�M��
x�e,����5�l��;P�0�V#�W��RP 
+���q��-���������rs�b c��6m�S��gt2��J�-��p�6��H�%#J��
�	g:R"9rc<_����o��N��{��	`��N�A����x�jt
gc"���
�g��Z���w1��6���t��h=d�v��t$C�~�^�Y1m�2����!g�L�����h4�����Ph����B�����`
N&�����J������u!J�E\(C�����*2����[��n;Tw���Q�M�c������0Y�6�(��V.���0�L�]�Jo)�*`���N*�lM."�e�rH���Y�(�@���
'�Bb��TR��Y'�V�_C�p�Q4��k��t����>f35�] �%����#�#��C��S�l&�&���4�����a�tg{)��	�w�6�0���m��(��7^pTo���L09a�2Z.��!�6�������G��#)4�;:vO��i��V��3� 2������RT$�B���Go/�����|Vxya|������.kt��_�;�b-�kx�3��usK���`���W����� 2��B��O�'���;�����<f~�1���$Q3����C��V#��{G�t�<4I�������O�X��"�}n�;����0�c��:/c�P�����)�������wp�rOP����Ehu��w�|�xd�\,8���!�������Z�����r1`��m��)MQ}����X\S�.BBt6��c�G�����!��T#|�h���"�o����m*�s�n
�j����u������NO����[��U>�;�9F�O��_�X8�a������S�P������E�;��>
���u%�N���w�q��5]�Y�����s��+����~~�����{���_�{C����?���O�y�T�z�>��8�:n��\�"��{��|G�������5T�I��������zO4������&U��K8����+��j��]���w������V�ES��M����
�v~<?C��FuP��Y�V���;x�}=iTI�T�h��33�r�0;�;>��[�>$�C�E}��*�\�tB����%4%�w��B�8 ��.�/����>�*���_&�����\tz?�n��2���5�'�'��g.�����u�JT�jG-8+>8�-�a���9
�g��B��sy��S�|~|f"9k<�uN�R�L2�����
����j''f����aa\[c�?��t�~}y��"'��jVW]UamMm���������{g� u�.�����$��`U9Q��2
�Z(9<9;��zn���������p
g����cY<%��.e�^D�	��M� B�������y�eb��ha��v����C�����K��	K��kb������eWO"��\X�1�p�yo("Yj��]�y�s|�v�������:.c�<�'�7�i�n�/l���x���������X���$��%3A���=���k��ce�p'�@n�5K���
9���T���p�
*\,���0�i2;U���[�5��<Y{���6�c�J��m�z�N�AV`��z&�cj�����������IM)�h����;�������u���>r�Of:@E�Nt�8����;�O&;)zd������y���M�A�����B|���V��g��9;c����CrW����~EA��j����0@���sJX���]�_a@��_����)u"F����1��S��� %�	�y!_���=�����n��M�7���`�f����p�uMdE8�
��9{g4#u��j��>K�������VG������c�k��	Ts\a��m@kz�c��F0��[i~d�O2SaU��)�z��x]��'���3n&x�H���Ci7L� V�9B��9s	���C�2d$tQ��.S�j����k]q:��C�����-b�w:8� �`�E�`��FW�-5�A��7����u�C�����d����s������������������2���>z�Ao���EB��
,�/�r��j��5�k|���rE�d���<*��?�x�A���+c�kN�vq�-`����v�`����X�R/���V���u�E��[���9�$���w�5k�8���?1751�mS���I4�����v�w�|�����N1'<����'��yf(��X�B1����������M���@?����'O���G���wf�����q��b�d����~B/�J��Sg#v�UZ�w���?;��Uk�d��n��rV����n�h�~���y���
��D��/K:�e�&l���/�����������x?�>�*�D����zb=��}f��T�fK]��y�����M����\�CM!~�z	o��������~-*���)��!�sv��U������]q�E|��������w�gi��-�)7��x�O\�r�P����S�����/v�e�����8{�/��Y��
N�Bk����n�������O1��2��p:�0�
�zy����F�i���������d�?��,@�qa^?�=��x��e������I�<m����������NH���|I���^�kGo��r�����^VY���������w-W���+7W�������N�@��d}�G/5L��F�0S�5&K����j��������ykW��	�+�C�RG1D/�(��	�I�pM=��8�1 ��r�f�����&3$�
-*8�9�?r�f���:w����=�^�[�P�9��jQG�8�����5Kx�
NK�l�f9���( -'()v;Ko����tdj����1����bd�vk^���|���R��]8����j8�f1��L��0� ��:�4ZyiW�7j������Q�w���$;}���6��3h��<�����h����R4p����xwD)%I��^�I��3���GrL9>a$����W�1�0�G'$�?y�"G�Ys/@��><���q>f��
�s;�m����#�cX���������VX��-�<�Wo��-P�d��,�-���e���bW���}���U��#��@�G����a��~�����V��U1Z��� �z�5Q��7�"���$��xM�tU���>��stXC��O��%��`�?[����������z
}��dub1������Y��)�����sa�6����]�������:�U������-������YU��u��m����Xk�>V����+v���k�?��Ga)	V_V����a�d�PL�pQ���kN^_G��3B-�-Zm,���6�����u����c��u��T�.~:K��yQ���h���&��c���"s�����C�az���S�+�l�f{Bggg�o���30���(��Y����j��
-����vn��`C�&�M0�1MQRpS����^9�kV�=���'z����@�.N�����q��'|j?@������qG?;���|	/�04cg������g��������o;AL0o�@A������O�s�9.R�t!d|]�9f�������b,S0��]�0!&p�|vs���vc�'�'���a����=g��v���BIc��=C��+�{e��Q0������B�#2�%�]�<H�}����9��-������6��N��������
����J���3r��Ne�p3���������c��[���b/�����QNp��tC9���k�y�?�n�J3w��������H���{[ef���{�]_�
[e�6R�)���������Y�i��o�x|��� fT1�
�����$m2cyA`����C���������"O�����=��~�M��C���
��
��B-lz��9�b2�#���Fo6��7�v���UI
c�%���A0
f���x"��y�����<��J���2�p���<�k���lP���<��?��0�'�=AE0#���:�9Y57eS����9=�J���If���]fS��M��}q�co�]�c�_�LSo�
I]���u:����s|�e�_�/Kf^?y�|f�U�9��~�(E���E�=f��5��;�m�xw�s�]V[>���7�NW���>��^��t��D����JT�stq�E��� �)5�����m�`��Ym�D������y���U�����������rE�����Rba���������e�Y�B2���)���� 3)�Z��l��a>5c�ag1�M7m�M�$'	������9��#�ts3����D�t��8�����m��@�D�s�������H{�9��B��_9��tkG��.��@�)���m��4=[(�E]����wiY�:�uk��;z~]'lbW]����Zi+�]~��j�yW�a2����a�T���q���:[L
z�����g�+�}+��E�1A`j{^��J��#t��Ln�m�*��^;1\�6�7�L����\�Q4�n��������LZ^+j��J
�b�m�=l�����=��������A&$����� �:dQY�V4�_����7s�.��;��gth�,l�wd^�x`c�6c���3���q�{~�j�*���>;�U{��F�]����:"��x��E�l��x���o�vm�
V\OVl��Ti�������#�}��U	�v��fQ���<��
�>��'�����q���rVO��;���[������L�R<�P\<��	`���A��*X�������W/�r
�
�X��#��}��l�k�����h���xg���T�!��M��B}8]voFX1��})3�<��bF[)H�}����	y5<��R���S����A���_t��e�o�\������`N�c aN,�7����_l)��'����~�����
L��4��B�����R�����<��n0z��/�%��4�s�>�����zz��i���S��������4�	$��C�$�����>;���g0�aU�����Fm���l��	�Xb^S�7WW����1��b���fA�����
���W�s�U��_��s��d	l��}z���R?�OxC���$��y:)]h��4\�+��L���W��{oQ�$��KgK���~!K��`�f�4a<K�J��WQ���|���Y����;������{�a��n�$����'�-����s-�[��y*�_�~:�����������������^��e�&�D)�F���Bp�R��h��9%���o~^�V��i=|��O��$b��{lh�i�Dr��7��)��4���h@��) {���uiER&��{(P���%�C��v5j��5���=�9�Og.z�^�\~Q��EG	n��KF#(���tX����
X�����t�0�DUP+��m4��� JR�]��Va?��9�+�+fQ0�������v�������s���������
M���	��L;j�S,�W����<���Q�����?����$��G)}�z���O�3�!���`:�_����5����['h����RI3"��D�`�C�z1���W�����14���_@��?`S�<F�y~��}����q��� K +���#�Ph�����b����@�8�-�\������~�lsRl�������+��G)c���@N�)�j���CM9�x���1�n�����
U�C�W�B�m��atn��=p
]s<��-�#���Qy�d�3s�����UV��W�8��e�0�$�C\��9����}-�8�q��N����om��:]U���{P���?;I���	���f�Z���8�]e��h�\��g��*0��Y%������\�EL�ffn�X.���9<���;���u�������!��#�Y���P
?��(�\��������5F;2[1�,����e�<�'a@���t���d��nw������������*S���`�q�A�_*���"�sQ8{��
�E��2a�Db�� ��1�e�m}�[A�i��1�,��C���{6����a\�qs��>B~�����%"h�����M02.ZP�3����)�*	z�����y�'D8�&���-9'@��q�/��,�c�D������!���L���R�~M�1��Q���-W��t�B(	?�9����=\�6�-X��T�i��r�r�7�UR���y�����g
:p��oMS��1�~58jU����g�AQ�r\m��p��K����XW8��k~&/R���A��}��N�5�����lM����@��tP���w�.M�C����p��P^E;�V�.
�l"��t�Vf)	��g���C=M>�����Wzap���]�����0�r�0��=Z�y��-���P��kE�F�����������<gV ��Q���
��QN ����������B
��6?+y�� �V:�t
���!f�
��N������1����/�8�H�G�~��k���,;�{�]�C.XsC7����&`��f&)�P���_�xh�G[�d���F�T!#�1�U�A��[�!4�C�c#4�3��"^���W���_ea`��-'���a���(]����������pY��e!dh;r���i���xS,v	5�=ocP��v��%�ve����,p�|�:M�}�?���1N�r�D���C�z�&J���(7gq0�ER:�rP[v��4	F5w	��6��8dk��)r]�M�7Mq�����9��PT��J����G������$jr�)�7jj)(O��e��������e����HC�����O��4�;�)Cln
�l�_Sfl>�R2�|%�-G�V��bF��zD�Qh]����Ka~vu���9����m ?�!����*R%5�AqBqX��%c���P���.��o-��&�!�%�����a����-��s��>���$%���@��u�L�w���n�Y������\g��(F���?�1��Ori��jr�����A�gl�L�W�a����(/d#����9�����4>`g$EG����\�e\����HGh�������t�@>�Qk+9�f��<b��B��H������[
faZ���x�L�Q3��*���*WY>�,B|�^����[������8��K���+R���s*�=&5u :`2�����"�
�@�l/����g
�������jg��c�)���������/<+��n
,���p�+�b7L��)��G�mE��Yf/�u�	O%�����Jd~�xm�O�J�������8�����
��N��!�_+�c�Z����3��&3+�'���S�W;6������)q�=���h��w^��\f�A4�g�����oa4#��4�(*������e�X+�h*�
wL@Hy�\,� ��g���<�L1#�.�aQ��:m�}�i�S��xE�S|���(���a!�y�?�s�K�$3�R�n���b ��1&������i����{����Z6�[`��jq����l��z���9�a�|�;;a��T����`zj��d~i��V���n:Qc�;��������YU�r�����%k`�\0�����yKj =` Ur4��`��=E� ���w{���[=����9�bc���uO�1�@\M0�Y�x t�<Lf}|m�s&
q�d{��b/�
�Q)��=W{`L]y`0�0���o���>���!�j)^J+P�?��s�C����a�Y� ��j����|����M?���F��`6DQ��-�����u�h������Yzw�K�A����&���hB	��O�\9�0��A�2c�9�f�YU�����$>�X��k�G!��=��
��l���6����������h�����h������!��Y��&������h��kb!s���j�����zt	���z�-O��������	��Y/������.l����7�63;���LiDs[]M{�����TQU�h���2��6 �����0(�!���I��l#����o�?�$���{�+���2��q������{���U;?��<R���^9��=�fj�?!��Ic���������yo�������>��4ut�g"�kD�I
�i6�����yUC��N�Bt�;A���wY����A1��97�h����<[�AcvP�Ze�	.;����<������a�8��=6d1�p��e�9x	�����L|�����{�h�M
Y���&��r~u�!���DE�Z
���GW*�>%�D&������ek�l����n�y��?��t��������lM�v7�������E����8��:x��������)3������`����
�j#�]\����nL'T<�{j����2�&�v�:�g����ipp�(�������>��rP�u����yex~tN�@�W/_>�t��a�-Hp~:�a@$v�y0EE(SV�FL��w���#k@l�-�W�%Y��y������
��(���gB���3���o@�F]0�
����&�;#�C4J2�m��,rp��G@�,���>%E�|�%�h� �[�P�(�kDy�q�G�(d{J(���-g�.�S��&������N1����7�|X��'�\�,�R7�� �B��w0���03���/��^�
���a=�YN����q�`�(	F1������&�7A6�#[�`#�FV;��R���Z��K�I�
���:��(48�h��� ��5d/aG���Z�lV��\���_^<�6�W/XZ��{|��I�+F�D"B�����x�)G��9<#Z�g�1��
�$r|�M�^%����O���"�w��^q�U��1aq��a:�M��3�=�B���3��9����%�*��V��D]]�5���(&c|�_����`�o���k&������\V6�t��qZ�v�Z����H� }���g7��V�x'�[��O� 4�-o0u�����'Q3,FS�S������v�[H�s��a3�������rw�/E��������0v��L��E���_�K~�{��1��^��\���nQ������P~���� �8�(�.�p��;����3�d��$��b�����U��Z%
��yVQ���/#���(�7�[��i���q���x��f�2��"_�c���{�ST�JP`���2Y-Z���FkIB�s�D7n���di��I�8���%
+
x�r�
�9���������?�)��o)��Ll�&�f��_���c������H���#�u�0���K
^,`NP�v���+T]�Q-%e��]�\#�����f��N�Adv�U�\2����
�7������Q�l�r��N��s����D�B?�'��F��q�Cj���4�"�9��e���+Hm�<��$�"g��bc�������l��������
	$�i(��	QS.e�iqy��B�>y���}m�n���A)�u���� $����Y�s�"x0�!����s�65����r9:4���i��o0��)%�9#���x�K{�O���>l2�A�� �4���,R&������:(�3�(�-�7�1d�7a��Vb]���
�*1��Y_���32�F�Uv��������M�u��K��ay�>^��!�l������q�Pa5fcJ�]�cb:~)AHe�}Z��@i7�����{�N��W����g�p�U��6e���\���H�Z1�i�!�H����c���7/�i�L�_l�����@i��B�]~{M��-?S���g���H����i�+W����Q>��1�PWh4�u=�&�=��R��g���q?4'���9f�{�&��g#\6&Z��2�4W�Z�#���@�������Y��u
�<���Q�q������I�$K�Y�b�3�Nc�����qU�x}�>C	T�"��7�����p�ip�d>��o��`��a�3���
�X���[�4Gv5�������58V��R�<��7�����a����Y�fN4���l����7R��4f�D���Y�R�W���1v;�D�H�x|I��iZ�95^��X[���k������N��h�q�J�y����pa�U�k�S���P�D���v&�Gb$���c>#�����>h�f������52v��e�R�N��� ����h�A���lo%!���Wc,�<4���c�EMb0���hu{
.��m����l�Tvy�D�F| eN�����U�I����4�5����w8M�jf`���gpTf'�E����C�����x�&���hS��1z}J�BV?��9���wl�a�px~�5��+�/����B��h���u�l�e8���l������������j��������9j�j���3My{�`�)�8���8�/&���&�����m����l����y����G�g���dJ^��++�l��s�9��9�������z���fQX��B�E�d��9�Yy�����=sT�rgP�6E&�z?�1�����	xP�Vl�v�����YvD����� �7��En���w ��S�ZQ�YT�����}���j�e���d��p���5?�k9�\����>m����������8c�,�S���f����+����y�����v�~Cg,��{@:xS�\�����@�@����<����|���
�����A�������;�����n���������?�R61�h[H?�`��r�0``�m�4kz��	�B�v����&������|�����aqm��b���J��,z*�5�|t3R��wm3��Zq�W�NM|�rJ�n�����ye%�m���ITM2�jd��e����\���!������x1����dg��kQ�e
;��O?��@�j�T��M|��]pF��g���#��9>���3�%k ��n
��-�(���������HIc�����w���/����G���(��f��F:�a�*�L���7~�O�Z��nT-��@�fZ��"�6��9Y��dgST��X��";�G>l���D�u?�t����J�q��y�Dp�'�r^�r��i ���f�EkC}j����']�.���`����7�V�hNY�b��0��}��Dw��M�f~�G��}��YT���:�w�������G�p��~�(0Q���K�B2�*g��i��<_:2���'_s=�g��K��9m2����A�2����m�5'�Y)��>�@�RCB?%�&��q��Z�Y��W��c�^-��������w\-�<��������V&O����QcF�s��Xg�X���zN��������n��N(��N�����[�����gEe�of<����5gM�9*If����$�*
�.���qk�k�������T���iu����P�����W6�I�9�K��s������t�3��}��0J��l��?�%���7W�=�^sx��������:iEz�$�D���7����	��V�h9&�K6�t����z����G������`��`c�d`��9���=�0)K�<� �o:�����'	G�����^�sz�s�������hf5@mKFZ_��8��c18��\���>�z�3l�AQ��a7_�s��p�(	7�>[o�����o���o���"�as���px�
1�:r^��z�}�^����+���cAo���UU�Q�.�j�v�g3�����P���6���n��{M����@i�$�r��-
������Y-+)#���S�T��8O�����+k���[X��!^��VO0. ��/J2G�s�������'7n���y�����`��gZY�A����,��_�;��[��eE��)eU����W��O1�"m��Q[-��	O(������B�*���0"�'>��D������j�����N��:��~?�JW���Q�+Q�>������������=�5���{�"��f���6��Jl�����{������&1,I���eq>����I;fw�:����_�����>��S-<���q�W�7��V�
<�����Y|T�����	bTg�p��>S��}�V8S�:X�i������C�S^e�V���X���������P����-��1vK�2��������aO�Z�����
�f�,�d��z~�
�6��Z�x62��v�*��e�?k=�����f�tJ����
m^1�0�rZy����s�xE���jU�����Y>'u��1��?��m'���e�z��������^��g�#�F���S�u:?>S���ts�m��8B�����{���*��!��z?�H2���+��!�7��������0��H�)�}�Es�G�Q����^��"�3_�����{S�t�xN}�����o(j���J��
�Q�g�����:s�vv����e���K���X�[�7r�)�'�z� �o�W]"<L7�O
:�k3�[	��EB7��OR����������9:o�@B����~������n}~�E�X+���l?�%�GJs\y��R3v��o�k��V/��]���]��I���ss�Z"��U3`�& .%���,���,����}3��[�'V�0�
<��er ���N���[���}��;����������@�Z�R\�i/0��[�'���l����f�S�M`�'C���R�.iP�b4H�9".F��n��R4�9��R�����}|^#���1�Gb��f!��'��s��):����������j�3����UC	��C�:JV�;�5sE����U.b�S��t2�Q�������/�}��FQ2���C-��
g��\}�&.����q�����]�������������4 !!��Ye���������Z.s�T�}���6��=�-S.�lw�ob�����r�lzU�� ^���n���)�fV7�Z�J�M��H���p'?��V����GA��������TG��PD����`o^D����Ub{�{�h�ok�3��B ��T�p���za���;�+H4�?�*�*)����{��V~c�yBQ��tPS�K�1�@P�V�z2]�6N���s�U�I���2������4�0��7o:������}������d �����)��<|H����9X�]�D�S���b�/�5?���d��1%���I@�*����,R�E9<=��1���P^W���">���[}�jy{T=��r��BS|e��!��/2d��6d%>���l�=t��>Qd����}����P�_'����(B��q���(+ct�^��6��)������]�+>>?��i�75�'����/��h:����T���N#���Z����g��s�imc��g���a�cs�l+�3Bep��D�e�q*�o-
�]$�>�5s�T9$�E��
)S�
�
J�o!�������v�!
��������k����X����?60oF�K��b�����J��lQ����������c��%����u�d�����.�}A~v�W)%h���b�������c���$$�=�7��y���H����Z���fG~)��
]���~����Q����B���WpG�?���e+���a��������1��9���`�%��),Y��;��8��r�_��M���t��{������,e���$9��nBB���*\)SP�
���C��&��X��^�%pfm���j�R�w�)9�=8�����$�L��`8����|�%����N9��Y����o�d��`=�}Y&QX'��|V�?d��PH�Y����N�%������y���d�_���n-�B�s��_�����A�'/O�d���S��������ae��y�k������n9S�gy������\�����
B�6�{���jOlB)#}�s�!
���6	�Y����<�}��E�I��?����R�����Z�E�OY4v���)�yb��������l��d�Gw�c��G�f+�/f��w{9y����������y�p���8;�t���j�2��;��pX�$���(�j��0�;-k���.ft��g�z��,I���FT�-INW�~#�Jc� Ls�0�:��������Qo��h^�&�����E~���HX:�$��D�������5�Q|0���We��������@�k�\v�#Yda��~������0������������x������o���"�Na�:F��;x�}�$�Lr����+)�(���+�LEN�������6=�
����eN���_�~�t���h�G��g<��+2/����~�L�N�����j�A���d���T��2"��`{���}��o�x���N%�$}�N{|Z-����PBN��+�+�A�#�c��
� �~���~��������~'�:U������2�nU4@�oQX���8�9&�F&d�LdC�LG;tc���=�G'k��R��f���y����n����Y������(����A���E�p�]�OX0Ti�3ez	`�������yZ�{����r~1p/���\�������?[|��eL��T�����?���`���T�������t�U%y��(
A
oC��,�����Er��y������l�j���X�7GR��j�������-���uy]���Yr��+��H){��P[�P��g�eF��23*�3C�������i0����\U�[����e�:�^������4�k��w����3��o�-c0\
�F:���w!oG~B��
=cE�;�`�f���n��~�_#xx�4&�x����mG����:4��f�8�}8v�pB�N�����164���%�7���P_�~����Eb�D#��y���A��s%m�{L�9�V{�o���?������A{��=
�O4��8n�I���!���2"+�4��'r�U��W����X:�H����u{���(�&=)G���%05�dH]��o��fW�\
�h<�H�xdz�%>�czI�9h 
]|��Te��h�Hxe�x \�0�8�gc���IFV ��h���\8�z�j� �a�V�GY�9��m�(�-���Xz�$N����+�GMq�C�H9n��EuB.�R����H�y_�.{��.�O�8��8V�-�I�^G�y]�({�"�rX���,��]��U]��D?s(99�"��(J�m4X���{����Kz`�0��������z<W�S�������������'����������g~N�����YRC�j�IT��BX��y8��� ��;�Z�Lq�<xr�I�u��))�e;��\����J���9\t��9H:(��	T7�t�%D�/�����_����r���!�]� K�Y�O+�]�y��5��	���X��P����H���-�h�5\�|F�N=�)�f5�In���%�<R;;;�%P�zg���8�*����XA����.�������x���r����Q!�D������(���W�e��������g��?)L>�x���ZQ�#�����j����A����b�wRm
����MGiA�o���1�|��#J�c�Y���
������s�l^�;�l9c-��~4�������~y���u����s~A�0����x�|�?����c����Xe����p(KQJv����u�������V������|�K�n�s�A�h��0kT���+��D�����j"G��V+;hL���I4����l�[;�	&fB�k+��y�2��mH��uZ�kx1�l�P �k�K;��C4����_���t����_������]C�b��f9���/}�8���)�H#����.p��f�L}D]p�T[��eQ �"� �U��x��������AlL�7����i�	���i�j8�J�vg�A���J6KZ:�h�������g�������lb��q~�F+d	�z��{V�����NVn&K���Isdj�./1F4C�=Y�w]��IJ�����C��H�n�.^����z�GT��N�f����)���BV]����u�a��	Z�&�16`��U�f�]2�[�!��Q�u�0�k����|���+Uf.U.���q����sF��'�R��P~���w�g��m�����>�6
eRjl���F������-%)��)����n=B��y��xL�m�]��~p=��3�[�{~��`���h���,��b
T��>H��q�C�?�[7hSN{ r�\��7}��������DP��m���%�n�.~���R�#?�������3x�\�q&��K��h�� �9F�"L��`.���s>��88��A�^�3(�[�x� z6�U�B#�"p{j�b�D6���rH���6����;��#e��b+�7P��HQxs�}F�~��~y���R�^z+��r����j|0-]�eej�gp6��P0�7Q�9;.$�v�9�fi��6M����*B-9����F~\���%�Kf>H�#�^����
I��>o��x)�r�?�w��=�j`�]v��������f�y���6��2*]�}���2}O��i�M	�_�u���}O�4a;�:���}8
������������L��4%:*Beu���f���Bo����R���s���V������%������wd�SMi�����f���U�Z����|G��#������1o�1a��>|��g}n��������uTZ���U�Bc����k4����f��G���Q,����0�����"{v
s"�P���!a��\�P_y�F�`v����}������]e��g�U����!�B��?�v/N����a���_<���u~��/��0�4����?�T�mW�98|�"�����"������������P��i�Y���m�k5YY
����F`i-m�l�6�E�h?�^7�9B"it���@/�v���W��e�@WyoVx�E��5���Zg_z�;;����Aw�$�/����7�i��a���������[�A���6����2��wu�*P�+<�����in����;�:��`������v�����Gy�9
:���&��M�.Fu�9o��������7/&���b����g����Nv�	�h���o��9�!��&�x�0�i����u�������fg}k�����{�����%G�<*���Y���Gi�������p�]������7>�����/�|�`Rr�VI�s�����J�R�FV���o��F�Mg�9=8�<����0��f}�b�*o�w�V�s5��U�&�j���y�i�.0o�o;?m+�Dcz���+�,;��%���{��l��A�,�E�����~l�c��e�y��PA�q��=���yq����o1��m9�p,p����v6�t��y���Tws�B�4������H}����D��%ja*o��d��)�3������R9�&��WmdAl�lq�zrH��:jq����*�gLr;��/`���6���tp|A����=�o)4�������,k��S��.�p�!������9��!��pa�j@8jB�NN��A�p;og�I�������4���l�-�:�O��~��q����^���U�~�MIES?�#2���� �+�8M%��s�;��������Y�����zC}���=\�:TWH�P3�2p.����{���k��A����!j�4���G���YI7Cba��
$F�0<SXZ�$���l/1��|��_��%��w3:rY|/r��:
�h�y�O;�(������GW�(�X@��)W���d`+�
���^J�$�I,��x�����a�����s�H��*/�fWYk��:v�l����������*�xvJ�}6��(�c�s����2}������V�
����
��g�.�������U5,QMc�jZ����X�m�SN�]1��fS�'��<5�L�(�U/�7�zj+Fe�]j�eq`����B��WY��Afl�H-�B��@O�7�����x�f[�}m��s�`G��*��F�&�s.@����jw���b\���6�O���pY���,:��������iit��O&�O���k	0l:(�Fqz�!h>@�P��
M�q��.���9
@�O����-�q�o���d�hj���T�h���Tt�R����_�����!�.��^,��V�sQ�j�b0�gG���� �����F:���g��xx��������7?W��
�O�s�
-��b�[�U�b�Z����}�%L�ST%��[����|�>��\{��u�Sg�{��� w
��3�;|��D��]G:g��S��G�B�Ml=t�J]�s�N�������������T�H(�*���i:A�����K��R��vYa	�<�X���{�g� ����~��
c���w0|���H�\�AT�.����@��`�x�cr8H��H$��L�L8�^1W�����%��tU)E��=J���2�4�xcE�H���7����f���,�N7�u�D&��� �r�>o}{q������������6�<�-?Oib�[�i'�`=���Q�d���V�E�Xs�(��P���r�M�3*7�xI02F�����7g0"y�d��E3�_C���>��WN�{��L}5�z��A/+��#���Ev��������*D+b�\z���@5����"���9=��|>�������+s�����A����A��+�#F�����+�;����f�~��Y��6�����m�`r�zp���WhO��?@�g��5���#,�D�A�b*��r�#�Y�ctvvjC!Jrq����AX���;���j��)n
<��
�'DQ�z:f��B�Y���L�~�U%0��K����1I{�>/�������H�	�I�0ar0�b����$P-�A;�������T���������S�^�Gl~�w�
��1Y��k��l��I�������a��h�������so'���ed��Y:�H�p4�_��_�?���6t(�R�9%�<}�)���i�����7(o\�!��>������_���#o(���>��3��L��(�(|Iqk_o����w�����l$fE���O�K���`�ZeW�v x�T���0=��=������ �.?`����-V|K����o�e}�	��� :m���c����H������/q��>���/���GVTg�P����3�7q4���8
Q����������5s��j���Z9$5.�r[�Gm�3�TW�Fpi7T�2%p�FPB*Z_pw��*�L��E)w��'��@�A�b@A[PO���ER�15_�����!����gbH�mW(q�T��LG�B��<��!���?�4�~����j��IV���/��!~�d$�y���5��|J����G���2��!A�0H������Cd"��������7|Ri&%RI�J���8�,�61����X���MM��=���������MP=����I_�\_��	@�d8�ij���|��GS��������Q�{0=�����))��E�N�-�Q��A������:����
=����{�~��-��S����B�_��+��F��U������-(��-(��2-(��R-(��r{Pn��r{Pn��A�=(������{��C��� �{��C��P�9�{��C���>��R�{j�����_��Wj������^�kx=����C���g���}��>�lz�=���z��lz�=����C���g���}��>�l�96��g���}��>�lz�=�����r/����{�^@�P��{�^B��P�%�{	�^"	P�%�{	�^B��P��{�^A�WP��{��B�WP��{���r�@�o��7P�(�
������r�@�o���P�[(�-���}���r�"����P�;(����}���r�A����wP�;d3��9�Y�y��������������/hphthxh|h�h�h�p�Z8H�=M�������j�P�p�Z8X-�W����&~�C��1k���p�Z8l-�\G��C�zAs���k���p�Z8�-�a������Ql��i�_�@�p$[8�-�fG������l���^���/pL[8�-�k������m���pl[��d�/px[8�-��p���c��An�(�p�[����/p�[8�-�vG������n���p�[����5���|�|�|�|�|�|�|�|�|�E���1��1��1��1��1��1��1��uI�W&~Ak�'�NZ��>i����������=���_�������������������������������_����������������������������$��/p��p��p��p��p��p��p��^=G�Fd������5����E"�=��F��5l?��c$H������[��������������],�`0���������ka�{|�}uuuu���>��m��I�����h$�����,�b0<{�m���`����c��G-w=���):��y��"=���������W���@~M����{��M���)W_�%�������rm�S�b��:�H�4�E�*��|iK|�����$�YE�8��d��=n�{lQ��-Jo!����!P,P,P,P�F(e���������(������WP�k�C?�Gi�����0���W����D�;Nd��6��j6t�������^�d<���n_8v��
ljVDx�~�~�~��5B�mdN�J����~��A=A_������-�_�%�,�,�,�,��VG���c?�S ,,,,���2��E=!�+�+�+��V��w�i<��.PWWWw���!�6I���{����V�?d����}�Etttt^':cL���\�A���]��`1��U�N���?.�Ew��������+o���-�.�.�.��N|�=��sf�4����J��V�)�L{��m���nW�71����X���tYR8�-��4��*�����z��r_`�dk����J��~
���PA�[���R]6�dd��M����G�1�����������k�_
du���X��Z��Q���a�c�c�c�c��u�1�M0L`U]BQ1#o��_��'�(��0�(L��o�/U�GV}p�-�s������76��5*K>�\�\�\��uB�������kTX8c5y�z�Jc������69�5Q����������k�gp��Ht��������k��E��������v����������.�+�+�+���7�
������"��w3_�]�^�^�^��ub/:����W�G�������{����,��Q���l�jb��1e
%�)��ol\������P�t������fz�vz0G��z����M�Iwc����'�lx���S�#
�\�\�\��/��,2�
�
�
�
��l3�D���d�N���{�?~/���{�.+��l�
�6��+��_.�O9�l���)���X�LV�\�\�\���9c.&��YQe�$�d-O�FNH&�,�&�de��}F��g�����1W��
�
�
�
�~a��x�����8���d+aY�%��������������OV@V@V@V@v� ��E�;����G/����U��Q|��F#�d�l���i|�Vjp��LJ�$�F���cu�7��.��`s�?�Via-X������W�<������6a�C�<a�����;����*�0Ho�MCgI�r�_��%��V�?T�l������d��s(K���uZ���K�S�c�#WO�.E�5��d_�}m�v)%L�W�W�W���%�_���.FcSAcccc��F�z�,&�Q���� � � � �zY�[	�
�
�
��l+Q�3=D���p���x����$vxV;�	_P}9�������|	����E~q����6#`.`.`.`�������f������v�����<
m������������-���WHHHH��i���������������m���C��19��v1��hn��������( + + + �F�u8��d��c���I^��N��c?j�����C������`��N�E|�d+��D��J��[���	�
�
�
������+��U������O@�W(�c4�1f��Ko�|���.������O��<<<��Y�b����b
'((�?
[�1�!]0Y0Y0Y0Y��dddd�[��1S{��yY�E1Z�yq���Y���M�������?�
Y)�;�����������������k�����qI6�m�b?���*�}����{��`2�toW���9��&A��y
�����0��.)n}�������>�B������h����V��.�M&���D�����x�Qi��xc4���m��
1��8���f��[lpG�v��w�����&���o����tv0��~��a[�O�����3Q���7�NZ6��9x����46F��d����O��h�����P�n^����E�_'��-�+�=�P�7'P�t-�,�,�,��Fh~D&=/��c���*��d���QA}���`2���i~y���x�Q0{�����>�(]��|_���������r���ppppp�	��/T8w�T��������������G����"00000��h����`�`�`�`��5B�(�pppp�J����U������%������Efhhhh^#4/�c�|�`�`�`�`��5b�(�pppp���>��U�@�@�@�@��*6i�����Erxxxx^#</Ps�>AaAaAaAaA�5��(�pppp�
����������`u�`\�,�@-2�@�@�@�@��y���szq",,,,�FE������T?}�hC=Q����J)K�m����gU��}��O�9OA.���~���T������^#j?�0N�7������@�@�@�@��x��x�Mc7����m���O�������D^����^#H?���l
'�+�+�+�+��F��7?��"	l,�,�,�,��h+P�Wnh�������"-....��%W������������
�9@��,�,�,�,��FX��y��������&Tn*,[-����������qY��....��p���\�h���������8YPYPYPYP����M�e�EZ\\\\^#.?������C�%"�
�
�
��w76�b�/��,0?��H�"((((��Q5QF`��^'S��2g9���������&Q�nn~���z��
h��*2������������:��jY,�m���Q4�y����������������k��x���z�����T�����e�c�F*���$HR]�atfo�]5���kz$��4���?V����O��0y_�z)��eMy�qA�����
�T���?�v�S9�4�in<�[�T��_������,�!i���E	
1�a7��������(�buy��^���?������BbC��k�CLv�e��.^1�8Ro���z�O����;��f�����|��h��3P����?I|����Z/��V���V������1���<���}����9-��?��;��P�z�hc�z�a��We���)�ZQ&6��= 7n�~B��3|�����
����J�#��M��M�~���\xj4O�h�?E�|��A�f��#�\�x�z�����0 �M��
�r���(��6d�� �f�]�U4L�Y
e��T�S����qm��������BX
��nt?3_A�h*
w��;���9�W�=������i�*���c�u��-��w�P���Rh���`x������k��:��3xup���������y���*��dE����9�M���w��
O����g����c�_��`�=��'���L7{~�>��=;>��L]Pyz7�u���^���u�)�\�ROp��Q�S;�������?��M�p��[_�xaRGT�W<��b�p��E87�hW��3���G�8���Go2��%P�d�REX����X���sA��!��q��i|idkf!Oh�v4�`S�z�
;�Rj�D���+��!9��yDf���Oh�fXrEL�h�o�t�Y�+�����?���0S��A���Q�������$�1"��a$���.:������#�eX�=�n
b��`�3���b�7�]�x��:P�8W��krx������.���wx�m�F�'�;��;X�W�W�W�W�w�_5:*8	����{�g�I���&@���������e�w��u�����eu�7uT�(/���d>�o3m�"�2F�U��WX��h�b�U��L|/��z���/Fi�A�!�lk� 6�?��6�P�z3Zx����P�q���������_3���W���
�
�
�
�~mL���88S��
S���$���Q�hK��w�z��0�U���F��J��Y�
T4S
d:C\����
���x��;P��W�x���������3�;���
�z:P�
����i�U���A�i�#cA�������3�Z�2�����2�q|�6R��fC���I.a�����i���?�P�_��$�
��6�r��$��ZM����W�����:4k��E������
���v�=8�m>�4��LrUE��>Q!��SoY�fmA!m���������]Q��$i]�Z����������0}T:�i3(>(z����RNm"C�!2��k�!2c����n��N	�_?�R��������������Z_
ZZZZZ��s������j��#����q�\\\\\^.g��tC����A��j��u���NW.Mv���:;$*mwww�5�����?�!j����'�c�%�h:��R7>��t0}�$� I����ys3E5�e����O�\n�k������@x�8J�@��!j�&���GP�<����h+�u%O	O%�t_����NN���B�!si~{@�M�W���&��l'���v"���������7*�	���Z45���������D�z/
�^
�
�
�
�
�����-�A������W�x���7����D� ,,,�F�`������WjQXd_�]�]�]����.�����RD_�`�`�`��/$�^��+�+�+�+���E_#���+�+�+�+��V�5���;��"�
�
�
�
��w��~l�+��@�@�@�@�:!�H��������a��I^��z�T0X0X0X0X0x�����������"2�N&�^��+ , , , ��@Xc�3�Y����]#�����a��$���h���+Q�9�R���u�MU�zq:�����i���d��ya�*��1E�����
������J�p�c�����,<�j����$�|N�p�3	B_%�A:���]��������N�q)��(Hd���Bv�-��[���F���<������D�CD��,���O����X��$	.'w�+��P��1�����������j9�M��`�����{P��iSo���m&��|hzw�e�p�����a�m��_v�!d��b�;D�+` e�prb��
P{B��,��m��<,��������2�}yc�<98��Oo� ��+�����W�|�&:[z&c��z�<QW�?'�KT��c�J,�����������E�#�/�/�/��F�����B�,�u��H�W�o�����������8��;����+o�������k�^�;�E1^H���������������TC���G?L��)/�����5���q0�\�*��0�8��4�����)}F�N�~��?�*���(�g����%j=�c��a�D�A�x[=�Jn�Q�_�_�_�_����66����g��I�?���������z�����%��v�C����F�����v�Id{T�rrrrr�Y�;S���bu$i#o�N�I&�*���0:�bo��~��#h��cls
�~�(���'�W��=}Tz%�q�7.��7Pe'�M�e���C6�8�2@@V@V@V@V@v������$�uHw��69��	{���=B�k�M������	�2��AvG���%R�K�7I".��rV��	&>V�F ��!��i/�`�w�9�����K�����`}�MS��(I�c?�u���b�����������"8��D����	��H��>L��kB@W6��K#\G������2��^��
~��z��	���*.���@��;�[�]����
HS>�H|�5e��]Sv�5���y�1���
��'�W�~���{76�G��$�)�������a�����Yo�X�G����k�v�����A�^�\,�+�+�+��N�������m����o��s�n����r	���A�?���#R�����H������c@�*5���O�;�T���|p���:�g���#������E~������U�y4	�0��(�us)Z{!�	�
���T���Q���B���T�_��p�.}�8�(�2��T�KzSW���N^��1|\��M��K���R��>��?Kot��*G�����S��1���Z���
�H1��|���R5��T���g��lK>6�U�_����_���%���
@��z&'A^A^A^A�u"/f�`����i��;�?h�8<�����g�[���5��X�ciii�5�4@4
�9@��W�c�^�^�^���j�Q@f_�
�����Z�������|�9@o+`���;����^/Bo$�8��M��m������m�����������b�2�>�V��������w�.�xn����;�C�`�����k�dl6e����:D$1g�h��	���������=�R�{2���*�)�,ZC�n|�o���
��C��b�� �@�]FQ:��M��ip}��+/��-7��B\��
���j��l�)���*I����V��,��nrq5�������xr��bSv"��d'��h�;�m����z���3Q��
�
�
��x3������e}�a�M��x ,,,,� 8���     ��da�'�?�����N��-����������_P$$$$$���a�n��Sxxxxx���)����������
&������������F�I�b,�+�+�+��n?���6HG7jSc�G/N�f���	�
�
�
��|)���K|2q�?;;�ceP����c�eM�����W�m)Bp�������	��Q�"/�&��,=�@����������kFa��X�DL&}}}}�\.(�������mo����_S�b�}�&JNN�}����R$�����#QI{�^<CbH�1,�&>��^k�)�M ��_�-�a0�5�9��(!�dom�w�S??�����0�3�� �[���I���/_u:���I��q����v9���$��*!�W�7|�vC�A�11�A�g���q�w�G��o�����'��}`�����=���	�����*W����^��k����A�r�� 7��0�WS�R�]Gr�M���d0L�f�03��mA[��m������g�=��4�Zv����mp�qf����AaP�?��
����/�����;g�<8[���i�,N;�0 ��p�����W�o�`0������U<`����e`X��X�{�F"���{������q� 1H�� q�$F�5�$�@��d�Z��`2�&9LF�5h�����#�v-���w�p�)���U��>D[�@7�
t?�D9��01����ap�\F�5������c����0 �����`0�o��U�5��p
.���2������_� ���|�K>"�Ab�$�q
$�@2�$#�L��d0�Kc2"�A[��m��k������W�\�oq:y(�����k��GRaLu&�~Au� @ H�����d���=��t����jV�\�d��NML�h��g��{��&W�����r7�
p�
�[|���E�b4��.]����90L�����lg6�e������c(3V�����Xn�h����=5~}(�O�_��v?���P��B��"��� 1H����I��n H���/����d0L�r���n��mA�G��Zpw�������&-fv�}���n��~��)i�B1mP�A��Ea��}A_��}l�n����'3�0��{Z��PD��|�K>�L@b�$�g$�@2�$#�L��d0�Kc2�L@[��m���v��w�-`���m�^q;��v���6��*���Wd:�m��qb��i�"�k�g�%�Z�*�������4|2��#e�jn]>���{r�m��|t;E��;����&S����s\�udg3jx?�S��"�GS���V������t�Z�����c��N
���z�Tm)�E����Q<�M�����&��-��Ki�m��C�W&����DO�E��W&/��������qg�gvI)2��;���F����h���vh�\�f�,�:��3|��;Ck��HN�i�����*�M����jC �d��m�������A�����\���$ k��b�"�]��_�a1>������y,�H��P'����yuz�?f��Z����q�Fp���yp�A�{�������[3�Cg�"7����k@��D�Ad�I"�2�����&�k����n����2��������W[��i9E�p��o0��t@o��Fp��Dn�o�~E��"�Tjs�E�^�@�"c/���A��T����6�<�|��NmG�AY ����H�..{}	;]��U��m#�nf>�3C=�-���Z��6�v���\�����������i�=�2N�,1/X�(��j���l�����`=Ql=�z�F�U,4��jX�oZg�:K�t���)��"3�#:�HrVTi:(HZ����[4���"N��in�9����%n���e��j�T�"��yM�>(h�Vv�3 �!�!�!��Vr��b�Bj���~���R�4(
J���t�����I1�tB�-s��� 0��
',#��<KLZ��~]n;��xy�j����wq���{���3�:��������=�Yw�r��r���)@m�d�Af�dn���J��m�^Y���{���ZL
����Pg�;�m1��I5ww�l5�`.o]���}���Lm���qzgG����QF8�c����I����3���L�v������uhu]��^��T��!9w|�:��l]o���8��JKq�F:}������dq�T%:7YiE���]�m�wkvc��A:r��7��������U7d`���wuuv
�������`g���
���.��$p���4$M���R�������y<3u�D�X��#h �r��Kx����j��"d�2�_t�.q���E�(�*�V6M���?GV�s�Q�8�=�G�&R>4!��3���M�$t���yaX����o^
�o��l/��%���y�.���M��j������D�D�D�D|������}MF������1)�;|����zJg�J&#j�R;�"��c�:�����4I�U�y��������RR����,���M
��H�������c���(R_}���Y�*�*�*�*
J�Vk��M�"T�AeP��R��|{�^��{d���d�	4�Rn�r3���N���g5���Ur��O��s��v�X�7_tk~�'�	Uj�����au�':�nE>h�W�%�gk���8'��QY�H���ji�iH6�!g���tYi�|-�`�������~����+�F�������w2�����z���H��D�*,���3��Y���@r�3-M�$�r���dy�����'_�B�~�
�KG�A�W��I+��%���_(\����t���"��������Uv����td:9�8�I�����f�i`��XttV#g��H��]��@XH?g��~v,J�����,�Yr����I�vY�����KUL�eW
o�!�:2,���f����AC3��)}�i�.r��!M���~sBm�4c��Y�3$�6s+�?H%H%H%H�/B*������S3s�j�qnZ^�q,�>[������$�����k�1++C�@?���s�D�86���<�+��I�eKJK[�$*����,RZ}��r`���$4d�&n�i<��zq)+vdS=��4����{��������&��"u!�--a�����7�=�v��a�C����q�����K���d�1��UZ�6$$$$$��H��,��b������C-i�I|H�[�h������tp����q�����qp�s�3��l@4
D���h�2�A`��s����c$2��f�h~L4D"�56��2�,�Mb��e1�iw:�?�~ ��{"@�tI����~��2(;���gggg_�8C�q�p��n�^x����)_��A��[��^�g�~�4
@���op���i@�~tH�s�A`��<;�]��;8��3���p���
:�3�0�M���z�����J�������B
B
B
B��j�!p\�}����7����.��6I����"N .���� n��}����|oG*r	��`C�_,0���w�B
���S�x6Ok����]��������a5w6�v���$	�������N+N�V���f��:R��X�^��l����j������xU�#�0��Z~01������H�p��X������~���:�����]j�}��>S��I�4�������zr�5�7|kO���}�t���#xW��^��6]".T��()�X2;[�����=q�)h\\�����k]X)��O��`���o6�l�������>�i��5���I*n�%\1��,���H�z�[�B��Mt�@N����m���./<`��[��hZu��`������.�=������X9~�����ahm����������r��
I�\<ISko;�����p�,�S���Z���fdZ�-�g��+y/�o
�����$Q��?�^��Z�| �����u������wk�xv��(���� �.�!��U��}��T�y`�>�0�uv��=I�q������Y�7��Q��[���w{�{7��3����C����w~S��rk]Z�I�l���vX95�<0l�q|@�_��\^�Y��a�Z���Gw�d�����X��j��Mj���l���unTT�gR��D �����d�.@{w����4�����0�<�d|1:���zX�c�U<V�
��I�w�s=�%��9��P�`p���y��y�SG\�85.W��3I�k?���L�BJ�t�9��hy����8(�7H��C7���r�]�pA��/����]��=[*;�%�,�������C���
�88����x��)���������1�Bg����O�1���d0Ln��?{K����*g���V�t9�K=91;5#���������3Y�����
~���A��f�^7{{{�{}e�L���qJ�K5�b�����+���V�z���J������;���/O��{@i��+��V���t����)����V8` M�������:5SSk�#e?�+���8H7_|�n6��Gll��.�����.-uUfff64o{�6�l����F��y�f���Ax���
��������m���W1�j4���;U�w�������]:�=V�&�B��K|��M�j���|}�����C|��&��&��|�9\�`+D~�*)�bLYy�J�*�E��y�����[c�2T��]�gBc������u&�9vI�Gq�,���k�v�����ff�e8�����o����v���3L�09�����Y����a���uG���b�(4�g�|���[��kK`P�AaP�Q
���\0�s����Fq0}���2�.���j��]�l[��m���l��\_*��}W�M�i����*��fv�������h���I`��a`�I��7]����(��b��I���~�4�?2���@o��}��z�+2�)r�������s��I#{|�G>!u�H�Fm	`�����1��U|j%��<��a�3M�g:N���:�V$��k�2�S}�@��m@�9h�\���O�������
��/�Y�7���8W1���;33����PQq���8�{��s�R�����O]��Tr~,2K2"-����HC�4���Bg)��a�r�����*f<��4V��SU��\ �?Abc������Mb'���.�.^u�./j��H}����b`W��������^}m ���x����, �6h����^��,�H�y2�:�T����j}\��@��4v��Z�\kinN��]��H-�&�d�F�j5.R?���t�T;Q�Ki?���Wv���7eg&hZ���-z3�B�r���^P��y}LWM���W���!ic�.U�[e��n�[�*�8�q#�.�"NVB�T��Ko.��|��,��X � � � ���Y?��4������|_���|l��bS�>s}��	���fq�;����f��_"���OY!P#r^�FV����y�F�qYD�-h!�{3����L>-]{��sSZ�b��y��������-�+�t�:�f�����b4=���R��fi�Q�Da�4w���^��#���*,��%Y�9.�U�Q�������f�qs�F�o����.Ok�N+�x{hmB�p"�KK�|�8�4����
�UZ�0�9AM�$>t�p�E ��U��bF67/H=#�hI�qy%���������i��z���oU$BI�
�)�����A�:N8��w�DN�ol3�Us�]�K��P�i%��S�p}JHHHHH����5��~�z���vsj��������'�����"=�S����:�[v��-eJR���/*�V&�[��3��U?HHHHH�����p]]!n��0(
��
��CMj��l[��A������
�f�c�8��|_��m�+������X�^��l�*����6�5t�`1X��M��u�.��Q����`bR��#�'i���F����I2�0�s�`n�*F���]�=��}���{����������M���n�<[���8Y�����U.���x��
�g�l�����|9��8&��;��Z<������u};���vQ9���8�3,��Ng�A�:�'����NfF��d���{��,�Emz��?�C����YqT:��l&��3S�6T>=������^L���v�L�_�����l7��pQ����l��6Q6T�b�1���M@�q� fU���d�������3wP
�� 1H7J����f�
�V��� �g �k�nM�@0?&�C@�G�u��g!�<5os��%�K8���zO�	��b�\�C�,����n�x7$��U:]�"�����ql��q]Wj2���/�F�li����o��933�-�:��>�a��C��:����4fEZ����\�RI�6s��yu7t��[3}H}��������d����a5�q�$�t6Y��lS�#��y�d�g|;�����E|o`��Ah��B]�P�\��3�>���s�|�VllI#�� �������������j��n������l�
�^���h0�n��~B�3��Q���%sTd�TM�	�������`���-��]��r�m�� ���x�W��=���e��E�Q��!���5���	���6D��x�����[�5����5EMdF���{����L�v �R|����7�����k1y�nv�~1�G�����;��43��x�z&�8�\tC��u�;>}����e,�
�$����?����;�t�����S��#�(�	���V�A��!�H��RU��Z�<��/�����)�w��������I��'�O�A�Y8��-�f���Z^k�l��9���)/��,W|����}������g�$�c�d�t5*�z��)�*���r���z���1��o�<�]��|e7�,������w�������>���{�HW]��b�;������H�Jl:1�
>q����o��/���)e��e�v?��6v<����B��������aT���<��j<�1��3h����A���ks�_~Y�
�B$�Ab��A�jz�7&���L��<��p���'���UyH�G�5uHp{�3?��%�B~��mNmb�ldZ���O�����=����+Y	��~y�V��h��W�������Em��-mV���_��B�_�\��n{��7/����i���|��������jMY�u�Ssd�������u|R�j��c���J�:������c9�����Qt#��� ����{�MI
dN.T�����������p��\�)���f���x����V���������F��im�����g���5����������yw��
l^n�]��[+mB}[m���E��>N�����|�����'O������;��WO�?���}��>���CO�����2�u�����^��ccx���/�++���������S�w�WS�����y��c��.R{��$�_��i�����:�t�����v����Y��)h;b)��a�+�������C�}77�x�|jj��B/�l��J����D�i�F�v�����������_���Vc���t��M����]�����g�b��w��}���3��?�/���[�>7?8?f���v����������K����j~���no�Y�����JL*q��?���+���+���[�{��Y[�U�E�{�����y.���:2/�������Sc�z	% ��PB	��p>����N�`.����M�����an�������s�����GSYa����5@R�����t�{sy���lz���7�� �Y���M������#Y�3���������8�wK�Gb�<��W��e�;���Ao��Yz�<N�Ag�r��gm5A�l��j�?�*33�L������x�S���\Jy��x�����{�kA����;�� ��7�����*s�*��&�euSU�t�q��3�<��s�x�Y�������s��Ra��-{��h.�J&�-���(������lV�n������ykFIkv���S�.�;'�X�,/�|��v����L����p��qm��DDDDB�"�������}��*O��Td�o2��H(R$t"����S�yeL���3�=�:�yfs;������o��TBK��`�;_��}�z����s.�|^���?�%���YZ>������?���>���������������^^�_��	�����a2���1h�(�3�Y�S�J30)�_�H)� �b��a;f��.q��8��X'e`�{�G�o������L�cb�7�*��sZ���RE����i@��!.���>LLj�xT%�X#�_��br
.���2�� �w��k��Q��`0�����lQ�J��|_��A��cOl���A��������Rm����$����xSsF�x���z��@23���|�����;����K���FB)�p��G�����*�����v����]j�@2�$�Mj���&�Y:��:�j�&�b,5�9j|f���~��u���g�3�:���s�t&;�F�p����&#�3L��b�(�?��&"�� 0�0�	����|_��I����1�qA\�}�ep�g�Yp�����9����r�C��8�3eS�9���Q�b���q�4J��H��������]�;��]e��A�����1��L���9�@�g��3��H=�m�������_�w����(��{D������De=]�\n3����=��@vp������d���R���s���aV}�+s�����8I�����2�6�	}�v ���q��$��r/]��
2��{�ggggg����Z�5OUT�*���`�p���31�0w���s���Z�����37g�������?#x`�v��Q����������
p<��b�(���Qlgj@���y�;�y����/�_�p$��Al�n��?�T�+-�q��qM��8��c�8nTI��l4���~zmV�a4�i18���p��y�L�+��$��Il2�{`��a`�q���s�:
n}�U{�;�|F�Ao?sOm9�A�,���fY�+���8���@n��}���%w%��pU�e3�'N��3���� �-�'D��"��S�����nF�SM[�g���dF�&8U�}#����3C'�r���[[����V�u�.��f����yb�eH���t�������*�������*�n�[����Y:�������������+fS�p���������\<�U�d�%�G�>��|�i����������o������N�l����kO�Ff�_�q��\�FD^�(��#�V/�2w&M��G���3����0n�)����i�%�h3�E�#��1�V6^[��f:Ih���)
y��~AZ�Ek���-e���>��kH}�����^U
�o��m�]oDz��o]%Z�=;�c���i������#
�gK���\�4�c$8�;o&&���R����L����B'*2��MW�Q.�djJ�������=�%mK��F�KnG����Cy(w��.u$�G��;�?��#<�T�)���8eo��3�V�����$>��A�H(��.y�V�V�9G1�������)���M���mu&@��<Aj������
��d��<��J�j�Y��B��9�-hg���C7�/2����,��B�����2����g�E�&	]���)�<�x>��$�%�0�91M��l'4[X�,%�F���B/���sz�d
�k�1�t@���N���^�8���� .��qy�������LqK�u0v���.�����QW�?��~>5Y0�{'�|�*1��l�]��Xk�ND
l�,��I��5`
X7=E��	���Yn�vp�x�mz]�
����0 7;cf���.�V�4� /���� o����	!`�e�gf'��A��2��[e��6�=/��uG�wu|���~����>�>��������GGJ��@:���M"�+=-8��f��<7o�Aj�������z;2&r�3�N���DP�G>J@�	�Q�i�7�#��&_S�U��D�M�%�d��c��"u�$5��tJj������f����_bV�H�&:��s1�-W�f!��S�g?}���Ah�n��<������/�����Q��.�������������Iy���Z�|Ga�b�� 6�
b7i���r[�t����@�#�56M���T>�	�rN)Y�s:�<Xe&*1f�i�]��5p
\7��`�P0s��'���"
F��`t�Sj�l93�������D��a��_��&�{����� ����f]g0�'�@/���@o��������V.t�|�Al��"�������|�fp�����p�M��
��Ze�;.?&5�}��.(
J���4(�(��?�)g�8S�'�1[��apn������S�p�i���DgC=)��\������Ru��p�u��j,�������us�]+}��>�V�
x�/\
_�[��������:������J�)Kf����0\nWZ�W����#���P�������" DF�D.���� �#�}U"���2��s!���� o��
�<+��	�p\�I����?��v���gK�8�u��/E���'VK"��e�zZ���$x��+t���������lwy&�pcN�[�������^�Q����>��c���������]���I:��|�������I{dS�������"[p��pZg�������jh�;>� 	��y��yY[|jF�b>��������f��4�5-�E-�*�E�X������(5����|{Zro	��SR������
wK%?�u�./�����-��HS=3b���2�ie�9;q>���I����d\�����"��M��2wt�C�F4t��v��E��-Gr��*g���.7��J���������������7pl�4�k�WB������K�����Q�6�3R�������\��H�]���l�c����_��+�������3Hd���@n�M�-F���/��fQ���b��	�'L�0y�����M <�����i>�'���d����#�a����3�P�O����c��~�5X���`9X� ���Q���Z;�T�~�_������&Z��� /�������LW�����b]4�q�i�������n���$Z�TfLLz����T�l�aC�XJ���#�2��=�n����������H;����.��;����6T����r�����W��������b#`�����j��������W������l��f�������W����������p8��M�{����Ax���_������������au�d0L�a�-��a�M���f����F�[*��W7= �r�\ �Q���>�I�7+�����RW~�VI�re�O�����amA�\7�v9�PRDq:��q�T�TB��\g�4\�t�����f�u��q���B��>/��<M����.h��q�Q����
��&\1~o�
f���j=�
�!�jKX����p�mF���
J�����������Zz���)G���tL�����Im&&���4�W�����|��s��>�SyD7<�7������C{���_g�Xi��������K�u��G;\��}��ly����wO~����R`�7������y��P���^���3��(�vUb���"Q�l�/9�/��Px���7�@��1g��8&�pD�0��������s�V��*���:KS�[wW`?[p�e=�pa��
y�8�#��3}K�+�����"�>��"�s��X����'�W�-���}���fK�OK�J��L>�mn����L���|^j�p�[��Q9��i?g�� �cJV�����r`�_j���V�:|�Cewy�C�(?���/Q�����u���o$R��/d���qff������\�-��Q<�9�U>�"��Kn����d�Oz���w`X�I����9�v�h�Q����@<������	���:>�4"P����T'�����J���������b_�x���9�����
��(��^T�},m������G�������3���tN�����u}���nv��*����� ���[���^���&S�`������������mM�D��d
^f�6���&��5����	��q��VuV����*�����y�gz�
C_��r��������jKD1=�p��1�i�0j���K�Nxd^H;qb�\�W��I{���-/$� ����?>�9�`�������4(_���������DVqX�a�UVqM���
,��yo���*F�^�z9��6x!PT����N���/rG�n�,M�*1$�G�=���r�O'�����RRRR�I)��gv$Ro^����<�*4�>I�	���_8���}��Z���An��ArW���x%#�J�������F��AdD~���8��7
��u#�������,�\��g?����_���{W������`e��M��+��O�_��vi���"����=Q���[v�s[-�av��~��P�����D��@L	6%p6��}q��,��C�@�@�@�4,W���R��K����j�03����A�F	]���.�j�%1�v�������l0��f�uW�P����b������6�����48
N��Mk?��<U��:���8�qY������hY�flj�_�7��]N���e�n�i�rVR5hW7�^�o>Ni�.�������K�m��������������jL$}q�-rG�L?M"��mQ�M��������X���Z��J�&=>��L����\]�d&&&�G�'��@�4�e�2'_����" ?���U�{V
����%\������LRA�_���yH/V��+�^�M��U�DVm����p����~��mu�r��+�����u������o}H���{_��/&,��`��	�c�"r�;�?'"+����s�T�!)���7$Hr��&�C���|�,��)ByT�}8��|�.�<�}��I1��Kio�?�������\w��|�����0(
��
�u�u,[��6M������s����?va|V��q�X=�'���<+��������wpv=�8~�[����p�SO��C@@@@4-���J�������
8��c��i��)�����p��ap~s���������{�^j�f�,�7�
p?*�[[}���83f��:�=nsa��{�"[�}�Az�����������}���>�~<��m`���M��|b�m1��O
����3>5j�ip��A��%�����md>m���������;g�?�9^��p������=LB�|��A�]������l}������\�R�����a��&���������&?Ka��I 4
B��_�y" �^>�.����n��%��<��Q{���NyE��?�Ty���������t��Z�7-�h�}s�j1�GS�����K��H�4b����r�-������yf�Y�s�"�k�/��@�%������\?a_�����(�)n)N�U�S�i���HG>��QO8��d��$eM>���s���`Y���G#���������d����q�j����E������q�i0#3�E������f���Rt
�D4,r��
�yt���
N{/�o����.7:R����������S:�lGCb���;Sv�wd��������:�a:��H��vy8���)���t����?�*]e3��������"��vL�B5��^m�FSN���Nx�d��'�*���\qj:���X��DV��������ex��9.�E\�����_���h5��*��v�&�]n�.��Xm�����/X�9��E���S�C�
P%W����vA?e�ge�$�R#o<]X�kY�7K�e>���C��|w����K�������
�7y�gK�\_�������=i��Q�ohA7�
y���������7/Z|��l��K�l����`�K��_�7���l�i~Yi��g��)�Y�/���^���/�\�>K :���t���]�O��]��)���H����V�wuuy��p�if��+��n}�?e���@�`]~y�����������2�������H�6���[�������EE���qb���i�	!���!{��n�~�������Si���]��,�����(�w���>�5���N�a��w������4���"����1�vf������u5d�f��S>�kz���Z�'r[���������x��'M9�����w���[0E�'���m�����``���~L�1����Qm�R���I�2W2��]{a���c������4YIq�����?"v��&
:������:1����+�]��T��~���x��>dsg�H��������������hPr0�[�yf��0��)��������00��c�5��^�������GG�8��� 1H7Ib��g�>d����{���g�H�8��3����[b�|&u�]�J��X��"�M���F%v������u�L;�;&F������.F|[��B;gN-�|�i�=��Jf�rB���M�����?����7�~�^���WP�@"@"@"@"4*8������I�������y���`1X7��rOmr�?������`0?>�o�{W�/���������;kJ���.�.^%��>����AmP�Ijs��*�=)�||�?��\��!1N�B9�|�K�eT��QK{��9�\����N1[���f��pz���s$�A�E���<���1�����"���&k��S}z��`��kdKj��4}�?s�O6�"�������.�
���C���Pm�vS�{`.
��fT<!����`������>QN��@�r\g�d�Wq�z<6�\Y9$�BB��6�w`&-=���g�,}��k�S�QO��"���J��h���e�����I��5���i��W��%����2�p�Ux�S�H&���y�4�����>&����4*�����J�����B�R���%�}�?f��.����Mr������\��^�]�N���g���9/R��$��a��*_&
�,����Lt2���s��u;�,���Y����F��:d����9�$f�bt0���3�����H�+uDn��i)k���q l l l l l�6U���~����\*����L���hm?�4�@3��$��������[
�_�'<�x�\�\��^�|��F_�0�������f��V���.X[�Z��	�j���������f�����G���?���D\��ep�Q.����y���}������8�n/��Vw:��Ku��Yy���d��=9`}�8����Y&��|��/��EY�l��G��:K��A]������H��T��*�t��kEd���T�i��B�i<�I�oE.}i��d��M��|�����K�����yD�/����^��83��'�i�2H����q�����x��5N��w��RG�1�a�JS}g���6��(�����A8@2=�MY� VK�*e��l�!_!_!_!_!_C�~E��tp�9�O���L_'��d_����d^	��5���X4��l������S���Y�ZZ�?�f��p��2�9H�R��N���C��u(.��C+�}����g�ra��c�@}_��R�3)�<�U�N�����,���j�Q#K��Y��.�tn3Q1��%M��d8���T'��:�wqT�D
����v����X�EEE����js3�����z�/�RW�o����4�EB'�
Lw�-Jj����)�&�
��[\B��O�g�7/_����H ��:�������E
�l;������"�^E�|�:�����������[��*�9���S�v�{�7_�h}8���f�;K^�0I��gT�)z�d���y�V�Be6�qf�]#��f��VUJ���0L���IGl���jh�z�sCM���b����$��*�K���(��L��K��4k���c��stO���\G�6)&K��{4�:[V��E����
0B7o��	.��;����T���[������uWyk�����dr�/x<����IR6��t?�5�J����"�������W���#��s�l���n��K^*�������g���U�f�e��6�����5������mGl|��GVuv���'�sHIT4���I�e���_����9����hwD�M[�]M2[����G|S��m�k5��p��r|���/[�����e�Y/[�����K��y!���|��$�@�����������zR�{��ac��O1�RK��M�P��=�������e�<bpr���t��6K���%F)0l��h�XO���C(�l���3�> u���b�	,&���r4l�qb
����B�B����Yz�^��B/���z	�?�*����gHs~;�p��h�Y����/.���H3��u�dp���{��������8��gFE�Kqa����kc��)6��ff��(a�z�l����R�c�������9�y�����W��y�>�`�l��;!����B���)��(-�(1Y=\�F��|������ED���'}`�B>��4���p|�M�����lT���1aC}��+C���$���W��Q�5�G�p4
�tM��u�<����td�H���*U��Bu��/���]��fw�.��H��{}�z������,#���.���Bt�.���Vv3�P����$�e��������������v�h��W:~����f��� ��8��bj�xL+��J�O�/^>�=�S������z��jlyBC��y�����|�C����mB��W�O�g����|~m��OC��1w�0Y�id\�IDOp:J'�v�\������P~E��WD������������~�Vw&���?/o�Q���8?s����^��*9�*	��w�
8�����TL�0=����&�g�Gdz����}���v��` n���Uyb��fZ�\d�J+	ZT���.IYc����E��{M��g�qef]��mg3�g�L��y����e���Rzi��j��q���r�����
hu���J2]�F����e�P����������z����6�GI������L�oP���]�^�QP8���>+V��0]�����+y3�:_�Q'�V�B���^��q"i~J7]� �����t�s=$���=�����$6)�{��v��W�zU�o������R�6���4�s�^���8_�q^�@���'�,!O!O!O!O!O����g����/����e���������/�$�;��-:��l���e����12����gc[�dY�#Y�x�^�&��]�6����N�\R�M9�E���^*Z��L��y����
�bf�� 8�9��X��k���q�������!l�R��������.���T���1�>��Ft{�d�&����O��C��2�5�x��*<���#��2222�IY����.Oj�S�K;i����y�����k�����Q��QJ�R��Fu�K�yF�=Zo���/!W_V��;�L�:u�e�zm�]
q������y�T���s���������*����/���4Fy���M��B�oo��Q��)�An��n��;���	��������)��3����N�]C���S�� 3�27H�V5��>�����N�18�tn���������$]��h�Y@�T���>����>�����m��NHG������Y%�{��
����<(�?�4<�H�D]����������I�H'����?�����r�-E@
�kBbS�|P2��g��#]����������hxP2}���
�AaPn����$0xb��a� �b� n�UA��$��b�,����Y�n��W&1��}D|b:��a`��p��o����X�'G��:��~�,!b>�y��/G���M�������u>��������z��V$��q��+��<��V�dw&���8��;m6�es�������]�Wq���*��q��=u��m�R��b�����M�!o	�{��L=�
l��6�����p��:�=�4�]�Q���0�c��A������x��9&��/����
��U:����]��T!�A����/���,��5`
X7
�*�f0�M�i.V?�XKPT�AeP��)t-��+�`\E� "X��e`�q�\C�Zpbq@aP�%�����
�}A_����<�_�k.gI�������|/6�pug���k��~���kUJS�Y�� 1H?�E�#-j�N�j0�3��8���Z�Pa��x��g����I��>��`�����8���L��������U��0�����3s�@
P�u��.�~R����F8�<�U�:����N�����:�}���f��h��ii�����l��?�G���������zc���;�0*���Y����J�*]�t�����	�x���O��'��������0�����Z�bm���� ���[���` �D��id�a5�S��G���������b�ZQ�<+F����� Ns%��h������|�� �r��Q��������oP��M��RF�R�HJk(9f�`�sM/�\�:sf�S)+G:7����,wW��1��h�n��,�]�Y.��}Z�Uw��`/�Z���/M���m0��j�zp�{y|s���eD���>8Ck%���&AF:IX�2M���'����e�j[�����`����*�����M����pHe�������2	ou�W���!8!8!8!8?�����STo����������p��l��6����������� 5H
R��_@6����Ac�4��1�7�� 0.�z3�*�����D���#�7��2�,v,�z3(
���0�7���/���7�/�7��5`
X����Ab�$�?;�Q�`�f��K�6_To��g�x��xF�f�0���cTo�j����A��To�e|P���!� � � �>�TC�f ������o���?X��(�<T�����7��(�����+}�/Z����3�7����!���)�DN��(��'d�@��"��?���#"������|�c�?���k\r>�^��3~CM[�iP�}H���u�')��+B�����!��D��[MU�����N,;��lr�����G��� �6��`3��(	!>]�@�4
@?F���`�x��R��_����IC�
�+�,��������b�,Z���AXk��l��*���
|3�J~��R���JN[k�!ye�]�O��P�i��L>�8��n�)��T}�obkO�[o����^g������tvqJ�p���=���h�X��U����9�f6)�?��4a�y�!��`�(E�x���H�^44"DIY�0�2��"J+���|�����V� ��/��uiN[JCd��f��3���������W�W�W�W��������
l��6���v�#<Y7a�y@��������c�	����/�����c��@c�4�A��i,+�Y��y?���2�.����cp��#�AePT�A�Gf���)�2�,��r�X~o^*P�AaP�Y
��/�����/��x��M����5`
X�
k,�(��/�n$/�
B�� ����>�x�0�3��>�^�x��g�x~�������c�0�����2P�5@
P?�=��
�>`;�W��*S�^�{����A�W���@�W���^��k��Q����������_K�q�y�=@������`�}3�@p���.�9���R�2��H������;�Z��Y1��FO&-fJ���#�K���Q�dX���`9X�(���qjJUw�^3������U��t��d�Af��A2���R�p�3g:��Zq�s�k��rWq5��/���!����]9�_���,���>�O���k�n�]�=TL������r$����i����y�S�e����9�V��kd��Dt,����n}����S���z����q{��{zV% 8�v{C���-R9�l��Z%�]��l
�W�}��i<��Xh�H�w�m :!:!:!:�����zi �����\�d�+���\����_��������������U��|�/�=Mf�D|{S �M��ld������������?8������������7xs|u���:I�I37i��*��Y���O�_��v�H���������H-�|j�\���u���D�S����L�[���b�t*��uaZz�n���#��������D��	:8888��p�����3�&�W���*W�Y8)Z��N�rZ*=�-���������m`�1Uz��M��`_���{Tw�'@����Aj���'�=��K;�f������0����k��iX��4�*�����D���O��2�,v,��4(
����g�>]��������/��x�E�i����XW!:1����Ab�$~<��L�3�0a`��k�"��3�<���(2
��1`��a�"�5@
P�_8���"����C�������|4
�B�A�A�A��K�5��` ��V`T&�k��y��.�c�=q��$������c�T�@5P��l�J�,w/��5���2�����o���)��&�P��\��_��m���vvv����=umr��%&<��q%��qb�N#���Qi1�,��8�H�����v�>�u�����q���O6�����q�������E����C�*N���,T!
��wZ.��Q��'&5�|��DwZD�H������dG��G�*��W���s:'��L���l�,���u�L;�=9L���/"�������yfi��I
J9�wE���|'\^�3w�F&{�j���������
��������
J��P��x<��h`����I��������1p7�cv�vS[$����S=��T���{R��H]���C� �2��$�[&��v������v��<I��((7�LK����h��j��v�r?��v���X�J5N��I1�<��y�O��j��lPT�A�F���M��_��{�^����D7n��a���o���L�h����:��o������0 7�w�, ���l��o��]�*{8�[�_��w&Uw:)����������O:�Sb�������,2������Q#�$&�`n���#�IC�j���q��������|j1�~��s���k�f�i��s����qkt��fF�\�!tu����3w�	�qas��;I���)�P1�Xz%�$�(��B���
��4�f�:*���y������D>�H��htO�!@K�Pm�5�A�A�A�A�5(�Bu���n6$��*�x�y_���y��f�w����(�]�����q`��8�����'�����u�_��|�UcZ��m%�4�A�i\%dyi�Q���?GVv,���������S�r9G��j���#k�v0ocG���v���T<V��9:u`oY��W���~��@����H���r��z�"v����.O��bQ I�����?�tEk�*�E�X������2�$a�43��~���%3��(�bn�����j�+W�B��c�����;P,����}����M����w5��8�s�kw������LZ�;�k�Z4�/)���k�~����Us��o��������h����$6u��e35�Q<�G���o�����Kr#4fq��4�y�����.Gn7B2������y�Y]/����j���I�U�|�cRE�4@����M���z���a�kt�DK�;����)��;V]�-�w���w*D^�4�m�ld#��N.O{�����������������;�8z����?-Z#��$f���W������)�5���k��G>��`f8�����ML���
�t�;]E7q|s�\]��W������9���g��;��������/�N�z�������{�z�c2c�4}����Y+�?��|2�79��:
`���y�g�
��x�;C� �UCe�����G������������'��n������uTH�*
y&�A��tW����]����������9}����_��r��m�+
��WQ������U�.��0����/O���c�?y������+z]�;��i��_o�wq���R8�%
ko��`��T�����ZI$Z�������ng�
�J��*�;���W$+��b�h�~��o<�6��R�13�$�3��d��%z���B�B� !�H�j��>���yb-b�u_��DM��7����\i���+���,t&W�;(fe��pMo����^���l��g
�W|�k��=y�����^���=�#~����7����]]�!~������|W�?�?�������p��������@gG;[�W�m���3)}#��V,<{7�8g�}N�	}�3bs��^r6u���&|�TIf�
��i�!�uj�
�A:�h����%~����d��[�w�������Wfy��2�]��Z���K�.&�N}�PO�/t�*�~v����9��������,yC�b���x�x��,�-%���AB<8�[2�{��q�Bw�!t�
�?:��LX�7?�?K��T��'���}b��[!��z�	�sM�"�-��DH�YHO���V���z��Y�����J���-���%M}�q����g^eu��E�5Ud���r�$YV�v4��,�L�`�1YF-����)E.����:.N�Ad����{�	��?�������V��\�������X)�1Qo�[��m��^�B��5s!��`/����M��$���MPg���<�q���m�q:/��ij�}�TiK
w���q~v��]���wX���z�@�t@or2�l$&����w,yr���&�|_��Q�~����g��7�W��{����~�������_9��[�|��M��`����5;����U��tX�a+�A|�o���WIO��^S�DfZ�b��z����8��}W�IV�8����R�rnN��}}pv%��Dp��OgB�-;�Hg)���xp�I%8����������NY"���r;�*x�CX@X@X@X4�<���K�UG���sxW2�3��Q(��b��aM
Ga
�%����y������������ 2�"����T�0�G6�t�V��S��S�#o���H��d �Q$���J�L�!��j�������;�����]�o��c�dv+Sf��/������6�<�*�+z�[��-{�U�@�[����w������������+���b{P6�@;���
+Lb'�+Z������L����v������������,J%q��7�
z���w�j�2m}I��DcN^�`��� 0����8�cb2)>q�@W���Jx.MS�tR���JZ�����Z�/�s����U
_�����E�}n���/U�K$����t������;��,G�O�|�����Qi����B�I�(���/��6��}E8�,�L��P�P?�-��U��e���N�-��)i��zt�S��t���2R������i^�?�x�%k�g6:(����2��������s����F��m�:/f�K:�Va8�v[����-����
�g�q��Q�%QW��������jau�:v�dy����g������g���
���"|�qfF��P$�k<��]4���Z}'~����rkm^��p9�5vUG��R��l��3���p�:I#��'��H�R�~��]��N\�\��r5��B�6��+��/)���S���!������d@������W*����86��R/i���.�m���a�5��94��i�$����k�S�{	)���e���&2��`"��L��h����^!���V���
PT�A�G����	�s�\0�m�������-��p���]
x������^��3��"�k ���x �A�w����=�c�Fs"/��P=�'E��0��:�3�)������h�V���m�����9�f����q�h�<�w&st��E�D�v�j���Z��om�v����S��u���t��/�|�������q��{�@�p��=ju�N��u�3��;R�?����P���e���wZ+5�)� �E$�6����,��K��fq.���iNo�\�	t���#;7�}�?e��S+&TQ���1�=�i��d���t����Z9���I�N	�\nH�.��y�w���|�������9����L�����x�G����']�[U��s'�h�Ir���(}�+gr%�v�v��5�,
���������*7�#_�P.:A��Hhb�S�����������-��b�G#�\{�l��I$#��I���O�_��viR3�������-2c)4'�_��]����������
�!�?���%����q���'e�������.5o�0����q#=7���������oCh�o��C��Rg��W�.M[�OG�����x��(���q�ai]�������{V�K�\���/�{�/��\�^T��+&33��r�?���F�>p��^m��<+�������e�|�>�-_�{���o�W}�T��q����8S8&��B� ��r�q�T5I����N�*C(w��|��[����\^�����JP�������wD�R�= 3=�(�@��D���#D&1Hd�8�|����]��c�1��=��������_��d '�7%����� �#!���`�Pk����������&#���<3weZf�v\�3�i�wVS�-��#2���0�AGf����V�I�=},L�=���h6o�X�sU_U���m�v�[��4�`��lbO���_zc�T�V�j�z�6�X�?T�E��9�7*�������UU��������3;�C_^����o���������p�}��O���r�4�_�������s�������k��U�NPj�W�O�JF]���X�L���n��!��F�4m9��a���������V/��>y�&���f>m9���s���Q���0�����[�=��6?�*���w���Z����/�n����,�����M*�l�K��C���iY��T�~��,Rh��������KU�@�ZlSv2��L,����K���n���G�:��@�x�����.Kf�@Ga�-H��q���T���1<
�?(?7[�B��n��X�j;.���>����������o�`�V�l-52��X���s�}8z�o������w����m"
`3�1l������<��
��&l	e�v�IGJuF�Dw������6&}j��Se�|�"{Nn��%�O�^X��q���i^�bXI+V�����kll�AW`;g@��~���!�Z�
��N���� ���	0���H�|��J�q��2��	�$�n�FU���:J�� g�F��Ju/2X�����<G���O�L� ��2�E����1G������i�D�q���m�*��8L�oq�o�$3�M���d:���Z0'�1���~�����Q����e���b|d��f X�J���<����{��yg��>����;?���p�+k�E�ku��]u��F�����L�-����8c��f�'��x�9������,)�:��V�=�c����q���b��l�f�t�U��t�h���~�~�:��}�?��Zj����Z�t�v��V�S��~-�~�
���2:
�4��_6�4�~��:������l��Af/��]rXg`""2������C4Kdh6�
@��:P�MOj�Z�'���LBj���qm(�'_���u-)����s�~�?����^oA*��2v�e�
g���lTs�,AYI��������.?�I 3��	��(��`m�[�z�37��C%�����^��Z�u����qW�������������c����/�^0j�����v����c8�����O=� �w09��X/_ ����1E�T����7�G�*��� ��O#��{�9q+E�Y�.�����>�ID����F�� ������7x����%�q��/�ab�{0���k�RM��*k���A�V�����;<��A��[e��j�4i���!����Q����Gl��]]	�WN�n���;/��;fq�X����Kg��/R�sn"J���]R�{7�+���\%���_�Ul{�`����{w��75�a& h���3X0LtF�#�-�0}�p�7��6�2���O�����A�,�=�Pw���f��%\R�����#l���'���r�HS%�c����@q�I��s������<��YB_;8c��QCr�J�
�a��T�`2�����Y8�BE�F��
���Yf��f�f<���>��$��u�������Z�U\��{��o��zEO7��t�g�m�r�;�,��b)��$K������	
?'���el�$�.�l��6 ����ERa��3��*������i�������,d�o��P�����I��c�i�����H���(
������h��q���c�0e,��"'���w���4#��������#d��?.����Uw�|Qe�=��T�F��8sbQ���7P�VD*���B��F��#M/E��:�*�-
hAq������;86���u4�G����S�����P`���IFaVA��B��w^����e�d��S��Q��F|��>���/@�����l����P`������
�Z{Z�
�����k��r���7Z��'������F��� 
�3��F:�@�|���M����fe�Et� y2�)$KY�C��k�(�@�$�+�R��tU��3���"1Lf���,[�:� �[Ng1��E�������p�v�,��.�+�����w�n�������W��G����L��9>��+���U7W����>�
��������H�;��u��"M�2ZY���� �� i7D�{�{�[���������U�d��V�:C]�
�� �B���YL�Ut�Zk8i����Q�
��=�&�E�o���SPi�@�P�K�t���N�^����mZ8�;61}���a&y�O��(:���~�������^�$��������C�A�i� 8�{�d0����(��3���y�G3$VQ�K����2E�g��kk�@dB/4���U3;�(�t�h5�Z�����i�E",i(�,�Y��k5	�g9��e[�����ky�����aJXAN�S����`�4\��Q]0��v������Z1���3�Q�	��*�����A��VV����LT8�(4��8�tf�d��hj�JWT�0�$���K���,E��/�b�)�b��B�M h/f�X�lg�T�*?��(�E�{����z������^k_}�?��zkkk���{_%�k�@�<K���o�J�R�+� ��Q��"�~��g;�� ������GmA_���:����%j�$���B�����zVZ|��;nt-��k���X�s@��G:��^$`�(`�]�g�#�/�/�/�/��B�_n�Zz�wr|�>`�R���5:�������j��� ��`���Y������6P�BJuc�d�t�%�V/�\�plh~~~~~�:z9*K��~���@�@�@�@�@��@q�sYV��'�	r������1�TTTTT�Tf��n���x>���
�C@�O�l�������^%P��_���:���t1dE������E�ou��T�L+���������~h���q����u)�S��h���]���$a����m�S����xZ��`��u���������5F�����0u����V�y��&;�����8��"B������@.�����V/N�\?�"L�%���������D���|OG:GS���]��6��[_08��?,w&%c�b����������9b\�>����<�
_����8��P������0�(Uh�E][�����U���|��	������6M�0�����6ij����]D7�>n��w���v�8�x)���}����N���A��pD��m<�#:��*���-�����I�e'���<�\���5N���������~�'����y2��������-�V�2�����V�1����1KJ��/��+L�����P,��>o�z'����a��=9::<���	��y������!����78�z��BO�/a�������bp
��������}���iv���?�Bz�Rd�K7qWU[BK\��4<��ez��;�A��4������b�e{�R�)^�x��E�)����q��
qQ�����?=��D��
})U�����?'��jjj����$�9�(��(:$-GV�OK�N�Y�Y�Y�Y�yu�lbC��������,���U������"Y�I	����Y���1���;��)a<��~����,j�0���[�T6�A�&z���~[����s��%�q���S��P�"BEVHE���s�������������F�<�jy��>�Xo�~`#dX��U�[�[�[�[���������_��3~��|���0��S�[M?w���d�����4�=?-X��	2J_datqo�fU���x��<��4M��C���3J�G&��6�$�v��A��ya��U
ZL��)�P4E���BD�E;�u�����������L��O���V�����2 ?V�������F\^3k�KR�T,�	��^BF�J��q�n��:����d����Ys��'�-C���)h�%�\S����uM��/�=W7I.T|�p�^�|N���tj6������� �M��I8��S�����'�~��E��hXd��M�����}����fG����@��E��U{P�"��s������Mk�����)9�����������g:����Jeh��OUO��i�+�m���������z'/��sU�Q�����MR����G*]��rNd���r�S�4-M������.8��u��,e��#�cz�D[�����j�
�1QMp��S^J�R����n�?���!xo�����{�D?�N316�FYSgi�C�=@���7��j�VE��b(�F�������|���l6��v�����k����\��'},Ts��� �%���K{vU��Y�'��~���lv�&��� ����#S�d.y��������P7
���1LP��p�u4��pk�#S��[ps}����7�lNT�M��y��U���_]v���s7hT���R����`����9O����	a��Xi5t<����S�=������\e��/��c>Y=e�W<?�������.^��"V��Kk
`�fd0"`8��(<�����DI0���c�$�l��xi�ws?�}q]F
�a	��x�M�\��4�R���
`{�p��C�o���8����T%���b������0�A��e��a�O_�0��3HE�+Gn��&�p��s��
_@�o�:K��0'� ��elnV��y���4��}�����>���~��d�� �[t�|���l^���]��
���r��i���M@|L��o�=<���'z�����0���O_?x4��Fo��o�zb!�PI�������c�G���Sv��a8��z���S�&ax��[�Q
�I7��t���>�c
�q��v�������5���bghe/�����4�Q_�������'�x��6��� ?3����?��	U���	����.\;\�7/U�M��!	�`S]�g���H�u:4[P�]�=��6��o>�.�;K�0�'b8��~���{��*/&��������N����n��e�����VZ7L��~|`��(�o����$����8���1��>%�����_H����w�x�1F�<&V�d"$`��R���E��i�
�o��$������`H�X����z����%�9f��V�g�������n
�)����-O���7�?���,e�6��{���If�1E|��u@��;k�
��AP�4�A:8��T��|�����d+v�jE)n������5���h))6��d8����=$�4
V�|�a���S(������J�K��
�C�W���HGQy\	nQ��9�	s��P]a�)��d�yk����������Ra������k���b�J������x���+I�:G�D��a�X�Z
Q���;$�y���k�p���}\��^B`�Z�}��9=�;'��������e���/��`���b������z�����
`,�g�(u�ux������<f�z)�
d��C�
��y�c���(��9��L�0����kD{R_{���I��V�rb�+�"^�������;��1<���*@�/m�_sq��W�^�
���+�(p�pk:��j��m��'g#�����:�NR�;������8|J�a�8���d�X3���>�@7�l>�p�? ���FF��3V����.J��d������������`�(E�	���3�k�e�7 ��QF�����u|��*{��T���B$fSM�k�0<�9q/�������a��i��[�px/�}qw�&������H�0�
�3D���Bo:1[�R�F�����	
��M��2�2=G��(��i&*���L���S����O���#��,�S{�;*�:5�����j����/�nq���LZV*�����SW5�YZ���>.�ZfE�d��M�y��N4,#`��}�m�F�-v*�s;�V�mn�S�yJR:Ig�vd�]1�w���y�*��#bfZ����vb���X69����v���d��"�A2�P��<n�f�H����g�����[��wB��C�*^�o��g�V��D��D@R`;A�2�lb��
*��
�c����ip7[�
"��q�y�3@{#�e�%ZA8��9�>I/X)��G
���.��T��� ��7�����8��w�
�������s@�3��q��pI�J���!t�XSNJ���~*1[����T��d�`�mY�V�X�)��
i���Z�V�S7��AUf��_�I�*5�BYM�!
����>���4��t�|Z�����J���c�p9�%J��c�5������jdgt(&���rh���K/�%p�@j��J]*��^�]�������!a�k(����avt`ut��1C�����5��-�%�7 ��`�����/[��R5�n/x��6�QJ�z$9�# �O�,�\������{�T<�t���%�I`Q��@��@]��g�y�k��}����6+.�<Cl��X
JI6'H,���H�*>���f�_q���">E��I�K,R�5��Y��	}x]u��{��nOo�������c�@9�N���-�|�cl_5@����%����^���[��}��>1��7��E5����7�O�~k�:���hs�d�L�F3�Y�XCT xD�a�"�c4�`REcD�ggJ=��h�sE���>��T-<#c3cD���X�U�"��m8-���'�w������a8iZ�Q�x]�X�b��;|�������������U�ho�xP1��er�<��j���o��H������p�H�S�$~m���a��fM�����'�B'k0��7��z���;�]��]������?�$<(vm���P�}�$~IqC�
���I�� ][�[�����S��$n����M�����&_nsq����*�Ei�K��?���T.7�X����8��3=x�C���N�_}��/�#*�{s�T�zC<���v4)&�gQ!�%>�J����P�b�����������5����S
F
����f��_]LU����}Z,�VY �����4x#�~|`��.�j�_X���|-|��u.�C�Cm@����B0�k_�.�['��������������P�t���$��oK�����AQ�e�%.G����r�����C�����@����R\0/a���]6�4x�O���\W��K��u�Y%]?��%t���a2�	� ���"�6/��{>wnL�q�Wu,���:�]~(���?���2��M��Xv.�2e���H�+���,�������R�Uq`�B'��C������k�����W��	O���K����������x6�<%�j�WU������Ao�>)s��1m~o���t��2��������%>�j��\�\���n�B�7[j4	��
��=	sL^�:;��
��������{b�H����'�
8x��kE��?�\wH6I�j!T�`G��_������/?��v�]�q�p������lJ��*��/���C<ypP1����h(j�������;w�����l_������{���<��1��Q���;<v7�F�����TO���w��������s���Ru�z��c28����ly�h�j	��s�s���9|�5�~�?����E8�W�#x���Xt�&U�@�r��������5��OAy)'�<��&��kkTG������C�����7�Fz2(w��L���B������|6%����Xl��h�r��4H�i�����+�p��/�p4�U��.�.���[OO����zV{����h��FF�fjh�Z+�����T�u����W�'�a���g
u�>�Y}�P_�v��J���.?Wme��{Y-�(�/���q��"�M5��&�"����������w���zhy=�JK~�o����_1��#�m�w�����:�����,���.Y�n�d��Sy��w!(F).�1���W��u���V���f�^Z��YA�7[�����a@g��Vq�&��tUO&
�(���z�}x�Oh���}B���D��.G��`�A�wILf�5���+F��7^�@)n�����[�DB��B��L����S��������mY�b��d??������@K���U6�����ZBEnFD�ck�II���q�U��T�����y�&);����q���4�8f?*s��}<D��n,a��uO����R�?<��Oy�/��Bqg��<�g���x����,��_�C���Q��K)�/����'�'fm�~38������;#6�!"I��\�1����uZ�����9�����CY�
Cze����\��Y�����?'����X���-:(�`G���h�(�7�G��A��]��)���G�_5�~������;��%u5u����������c{���frz��y��@h%o����|����8�������0S���`�:v��0s�^E<nR�~�����8�bu~�'���(��y�����d���H*�R"�jf�!+�S;������^���t�C�������d	gH��N���\�d��/�v�2�,�!Z��6osq� ]�Wt��~2x�4J ���^����a��h$T�d�O��aOZ;�Z�a�q�1u�	?C�F+�^��E����:����;{��A�t�����~��6Y��j/��O��!������{�o�6�z����]���7��rz�i��
mfM}p�"��w������K�:y��i�G}�9�*)B�`uHF�)4�p��c%Y�	j`�Xk�34�y���_�����I!m�
jGV����g���W�^��PW���>��Cx�*Y
��W��v�(L^K�=!�f�=�kH���Rv�gg��.A�eH�\ 3�%I��5�b���-�R�ow#50���
(0��-���Q2��/� �a��)	��8,�rw�|�;=�k7����pUN:���������?�{���l�=q���z��gj6v7����>��n�G����e�W/��jw�p��]�2���]��Aw��?a�m�s�h��}�������`��qe����xD���{��{u�������������dW�_�GW}��sP�TW�`����^Y?L0Z3���Di���������)u�]�8�{��(+�r�I�����:��%��������W�'E(��G�x�������x\|QD}2$���{��`�����u���`�}��7<|W�)���N��_���#��4��b A����l��h��`�t���!�f������Za�J����R�5�`����q�2�����`���`��bI�����|��bQwQ?W�O�+���H�
 ]�h������~������[c��s�s��1q��f��S�L>+_>���]]���|����%�w������x��6�;�f�_�����,�&7k����|���n�^B�����
y��&c����i|��i��n��'�N�l'6]���q��-U��~����yrP�o��`��|u=s��z]�l�*����>\J�._m4�[��l
�d�������yF�`d��^����2>��d�8�����$�f�)����<��B��'0^F��.��R|@�v�R�m�����i���
��Q������c]�b���
�o�s7�h��*��/��1��6����Ldv1�$c�+Sdm2wWOlN���'�T������z���Fwo�0����}`C�S&�� 6��z�PC(0��a�Z� C��"�9i���X�}��������8
k6��	3.�`�yg���:�Z��(v|������&��j"M�?�&�%��RG�J�*p�{
��v�y���"��}�+�k�^����!6rF��_���Co�9�4�)laP_{0��*���~PkT��W�E�d3ke;����qBJ�F	?��0t'�Hc�J�u�iak�"/L�?�v����o8?�����)�]W�(�+�KR\�(��W�_>�}�2�f�.�~�c1O�
)'����=a@�������7�-������z�'�+V���
�
K�N-�yi�n���;S�7_8Yq���
��;�N��[����dW����7�gS�t+�h�K��&�,��}���}��z��Z��ug���X���1��{��%�F���y�� ����ak�<��/u`�Q�`��	�^R ?�V
�"�*�r�|�o�#�%�|=����7�����?�w�~w{{-\��F���(��6��� M���a��N���2p��lO��U��tz5k���L��`qM�B������N��]40��m�����KM������X�{�[���r��e�U����Wc��*0�n_mV������>V}�(�K�������8l���X�>�Z�����+?���U��P�Q��*��%��c��|��k+E>Z�x~������3dE��J[�����w�x����������x���n�Q(�h�>q����W�9�=z�j�L�H�}�����������#*/�B�h���RK����1���j�0F/�����\��	�^��- Wj��a���C�;j��1���m�;<��H@���15��.t<)�4� ��Xk������',ck���Q�x���q��C��g�5���8((����4Z����)�u�3L�9[$bnO&�-,�#���@GBw�%���*n�,�Y�`K������h����w���s�&��M�a�3�"�M�p�n���W������*Y9n��m�c������R�Q>�Q�-M�'c��F?dUGF��/��_V-v�9�E�5�SD]�w\�`�����=uwQ�wO��e ��gD�g�m�y*�wh]
�Zs�_7
����"�����V�9`��6���v�\��EC"��@y��_�*^�8��v{��3�'\�7��c;���a��h��� _[���d��UH�H�Df�o�r	����EHq��N�czS��*I\d;�0D����6s�F���X�&;�B[�@�!4�C������7c�1gN��73���������l`���s�M��N��i��b`@��St������$Sr�dR�P��L0k��)k���&s��B �/�<x����
0^�#|�;�YO5J��:a��oq�t������5�&O�h��B�����u����	�Dd3��j���CEZZ�)���<����	GD}r6/<g95�L)�o��Z�v���9�*�������x�v5��Q�6AZ�������
�];��
/Q(�3���O���~�;���.�/���vl�,�@���NfI�j���`ZEi;����ETbx���]g����+�0O���e7Tib�mB���D�d��������b:lX�{�]�eU��:��k������t�s��MC���H?�����	���}~���9N��q�?)*��.�������7W���n��b��Qz���� ��>\G��1���j&����i`>�N������{t�� ��-I��'G���Y{���C�<���F����(|�v��	j�U��:�ov�2�*����t�,�6U�_o�~Q�@���P �*�]�-�lOU�Z+?�T����*�E0Y�x��gQ���n��Y�*%��~�>Nu���]6�������SC��
����z�F_���
��L)��F����6�����z�����8���6��v�>����9t���[���g��g|p���a���^#�A�&VB-�����!yR��l�w��V���9�F��������R�,��r^�z�:o��;�R��������&�����f#.�Pb
Q�~z��.:�b��i6<s�`����%��u������p��{A�x
	�D�h�f���*2=]��d����v�����r�������g�+�BEX
����|���w��������	��{m?�~��mE����)���"��>����0Y�������'�������Of�,��m���\>�k8Ng9��fa�y�^�|���������d�y�s�c���z@��&��q0��.u
5Nn 9���b!.U��j5Uk[���#�z�ZOT�+��Z���W��7�@J4�H�4�PJ5�X�mC�m��mC�m(�
����6���r�P�!�{�b�P�!�{�B��P�!�{�=P�I�n���j���}�v�������j�&�kx=����B�v�g���]��n�+A9��.�U����J�0��?����B�v�g���]���Cl�A�v�g���]��.�lz��={�A�G�3(��=�r���#(��=�r���c(������c(��a�'���=�rO��(��=��B�'P�	�{���r_A����WP�+(���
'�}���r_C�����P�k(�5���}
�����r_C9�:�A���r�@�o��7P�(�
�����'�����8�p��l?��~���'�~�K�_����������
��5q�������S��+���j�Z5q���ZM\�&�W�!m����k��Ek��5q���nM\�&�\�����~�����k��5q���M\�&�a����|L����l�Jz��p1���M\�&�g���v&~�k��Em��6qY���M\�&�l���k���63~�����m�7q����M\�&.rW�������?~�u)�X������n�z7q�����o�����C�k��k��k��k��k��k��k��k��k���c�_��o��o��o��o��o��o��o����YD<��I��N'O:�t@q��q��q���a�/p��q��q��q��q��q��q��q��q�����/p��q��q��q��q��q��q��{q��?���6��6��6��6��6��6������<���Id�c�����u|�����^J����r�R�E�[�[�[�[�{u�]g��
&u�����������
��5����m~k�
_o(������J9��p������m�_WWWWw�:����4n^�{���&,9w�F�A����|F`���M4���������������}8�P9��g�6�F�y~'sw�t�5����
F[1�|�	�*zcH���3�5�_�G��i`j�qm
���R�  �w�w�s3�LM}Qg���>UvWF|�m��Q�����VOm�3��a�6�&y��k~�n���4����V����M���y���fp��b���0O�<	�$��\��
�
�
����R�^XXXXx����r��_����6��"��������_g�
�
�
�
���'�qxcc>�� � � � � ��6]#��y���Bp[p[p[p�6p[l*||||?����N�����T����������H+�+�+�+���G\@���('M��M��'���SL=
1)�o�P��c�~�����+B�H�Mt�i�����#�,8.8.8.8�J�]�G8,W���{�������L
�����0��,c��!f���1N�{�P�����T��a���p�2�PJ�3�CuzA5��zpW��`�Z��lKQ6�Q�N�����)�������sx�\�h*��rQ���Ze��5K?P5=����&���
0W
&T����G.X���>�_���E0���y�����q3�}�}
�����:_�ud�0a"]5���:��������['�= ����G&2VN����v��f�����F��;��`:�c�usZEw}2�vc�]���LfcA�W*BY�6��L1l�����7*��+
%��/[���7��G����S�M{��v����grC��b���?H'�2j�W��b���9"��)f�j��/������~��3�t���o�S��*`/���
l�T�1a�Z��~�#��#%�M]�g��e:�LT�w������{������u�%������q�T�Q�����[E9G9��:F�9�j����yFS����tw���w^�� ��[/����+X��
����1��6���s����r]��[�Og�I�1�M3�oD������������;�	Y������B���av��x6(��{�S6���O�I�p����������G�7���+A����&��r�b�C��U?������R�n���hH��N�h��o�"�x������-�@����n�y3��pJnDe��N/`:(�3m]e> ���C����)E#��~������4i�� ~���>6�����������m����v�$����Q�y��3>Lx/��7�T���w	S�(")�,���4�������d\{�����$�1@|8��j2"f�������s������`�����G�2�)��yQ�i��1��5�M�$���
�pOb6t��83_<5�I�����#ho�vZ0����<Lf&7�'A�y�Y2�T��/`6�7��&L�P	���v@��:CI�p�of��tx��"M��@9���x�O
=�����t��b�,Cl,KK�	�O�OE�@�y��,���<�"�k�pjS��N��bC�:f�5iP�5fp�����MI`��T�����F�1�o�Uw�#�7L���}@r�[�~��W�m��T���
n+�v)���1��4	#1v�fSa��Jh����P/3��=��(�C��v�*N�K34WM	QFA�Y�)7��r�!�n�e�i-S#�y���q�Q}��&��������)�q��,�`��q���t��3�;��;Q�t{�+;b�}`W�k��
,t�����^�X�����^���.�C���psP��??���g���e�7�:��=hD0��<5ep^�IY$C��9V�����0�a
�Z8�^&��\�����HC��S��4���M<�P��J��@]R��	L���&}��$�
�BX[����(��1���1��Lm�r `��gx�����~S�z��������q*�-�%U���rY��G���v�PU��4�s,*�����<�d���bf��n�4����1�j��FhCX�
D\���).����,�I�$k5I`�B����.��@�G�5��O&!�J�3�|v���`��j6���}?�(�3���y���������/D��E���7��D^D$<otz����#��	R`�g������_��)k���VR�a����&�"�qba�F�����5-84��1�8$��h��:�A��dfS=q`m���)���v��F�R���{Qv��-'��
�Y|$�PK<�4���8����� �	*:���TZ����e	�U����z�2-Nu)���.����P��<i����.|K*�=X��&�dW�j����*���A,��J��`w_'����#�`c��N�������4v�����z6�c�~������������/T�\`��h�dp��p���4!9y��fS7�����6W_������i�@f'����u��8?c,�f.�,���G�^����-�L��"���7l_�8�6�/�W�
7��q��0#
�3��PAyv�@�a���p�[Q�Dj8h;�vF	K�n���k2�d(�[�$3�b`0S��G_=����G_5T���|@J�l��>jM�������qlK�wW��l��B��|���l��!o���9�d��a�_�����W��t��T'��� �+���?5�����s����:]�����V(��0����?�.����aM�*����a��~@�����.6���.=,jno��\�{���7��y�h����������|P�jo�p���/�>��o��Aw;
�#�]��\�(�b���
�\���,����	3B���Ef�2Io ��O	,�:G
)��'�lU��N��|+&��Ju��������uz��.t�o�.��Ks�l��=����CC��?(�p7a�^~;T�y�y_u�
z�I������Z<��P����ISH;��Z��0���C#�-���,����7���*��e_�J��c���m�Lcg��8��g|��5�U����,d�2��b�,M&�hE|1H�-z��]���H�o��e�K��Ap
S,�������`+u�Z���
�Q����"�1�{�T�<FK��S\�N�d��I���X{���=k���!����f
[�I�#q�
>g����+��s. �z��	,kf�<v�}$�M���m���\��l4
!��� ��7�::ManAF*1�����Z���M���p=��m��2���M������7A?
��������B��B�����n��HjG�c�c�U�8���,A[�+�k�VCA?����������{������$������{A���p���	<~�!�]!z���	�����X�������o�����|0L��<HrcE���,L5���@���D&9�(�V
P�=��vj�����|����"9�����@+�gs-F�Zf�(��:r$w�C��T�����D|$!Y��N�

�8��������Wc���b��=�)�����.%����/�/����E���Jl�]-P�?���>���-�6���VZ���.'�-�B��d�,t��Q���
��%{4�6�������0_,xS���]Zb�#+//�e�`7�U�@9�6� �W�yqh��������`�'���w��m��Ny�����r��/"��f�i���82���NM3Y0S�iQ��E�8�WL43��zVW�6����
��ba�,?�������z�7v����X��������iM�������@-����b5s(���A2�e��v���'#�^��9X�^�O`�]aP�Q���u�����o�J��~Q��y[.�^&V���>UXB���\�|��m�K%���j�G���DRG�n9<rfo�"���fbe��	�d�@�S�"���0��q��e
�-�L�J_�r6> �N����� �Q������Gz��R����&dp�j��5�&y�
O���/m�=�p-f����
r�I���
8���P��zjg����74�$���o�}���kQx�
*|ID�zaJ6p�6��)�)ann��`���F�W�I�f�Y�����f}�C��������oy�p��+6���n5�x�����d�,F�t�^���}�����K����e�l��5gZz�*��_��Kj��Fn4Y6���f�c�q�M��J���CKv���K�B��R�%����3-*]�V����+�/����"p�k��2�jw���H���+gz��A�b���F�M����@�������X�"���i����qYf�{�`
NG<,�����)��'�O��0���dk��q@T_��
�kzNm3wN�Ws�INkV�����2�S�
|���_����F�]�������-�a\1����D��i��D<M1�<[,��9r�<>����S�l���pi�=����-��s���$V�`N���V�����V�w�� ��~0��p}p������s�T�(��g���9NdI1
aCY����l:r�>1:$����R���T���[M���Y.��Ob����
g��l7�J�":�$���F����"`g���^�����lv��6�R���'����*h�:����� �/Y d��L�6���E��l�;�W�r�m�Y=P����=GTz����)\_3�������B�?�`��L2����;�w�)��w�l�&�������~����P�j���MY�-U���+�"u���=�sf�U���B.��ul@�6�TK�?
��@M������sY��������gmg5��RV�d��f��0�����y�=s��*MQ�iu@�RQ���O7���t<C���U��"���9#��o3tpzqp�6x@w� ��T/x����=��G�$�d����=���7�J0����9�T��4�Q��m���0w�-�n=����.F�����I�r������]{�MP��^Es���>u%�x:�!>��5$�������YG0�z&����e�J�/�3?�I�> `'p���d0-�[���U�����q��Z����� ��5����N{��~v�EQ
f�@��Y�E_���; �TD���'�|A��.6Vk���j��w��b�s�'V�
�������~�(&���q�rN"t��*vh%�`Z�M|���S��;���R?���{0���L&h:�OI
�[�`���z��g�}������S����c�f���cx�?E�RF�������V���ip�
|�JN�~�����k��n#�������lT8���j����T�M��!q������"����uF�����y�r�0n!n��Z����~�%j��)p��gs�u^z��s��wO��������5�{��8T�-
���gA>���!����0��!�]���_����/�� D��S=��U�NhI���T���F+����j�-N������{����"���/t7�[.B:�R�;<���������������������Y��������;���H�n�R�W������e�����^��%�� �S��s3�0O�M�G�����~x_#pY���]z�L�?�3:|^<�ZG��u�\��TY�;�S��9��rx+`@d��-�����L,�H�_0��%��!���VI9�}p������C�u/��E�)����=�(��>~���?>���9T�����=D*S^���)!��p�s0Lv�BF�\�eMMrnL�4���n��A�������(��6���D�	�����(S�i�8�����	�j��-���q�pW�6���2���������X?kzv:C�T��������5jp2c.��)]&{c�[\&(h���H�kq\
#�/�p��*L������U��z��;��(
��s� �t�(����M(��O�Jw)x �v�JBs����\��\�������)���E�K�nR`��io���%�$��-B�����������P��Y�s���(�2�.~������oP�ZnM"Jk��:�K��L���u��A�o��P#*3���p�aX���c�~�k:���|���g��i�8G���^9�b���{���2�X�X-����P�9\N�xH���"v>-�MJUU��2P���[#m����lP�7j x�D��
^�l�j|b�������4~aIU9��E����0�(��b�"E�k
��6��%��+9��&��nVr��
�s����v%�'�wZ����;wp�����-s���� �s{#��r�O���]B�x[� ��"��)�U�>�����{�~)��5^A������]�{�[�">!E�<�������_���
YP����v�fJ���k���vh<�AnSS��$��urD��`��������eh�o�C'a,���S=&?j.������%�Q���/���t���vo)5AC�|����)i��|�K�F:��i�7�����������6 ��
q��h?��?)�����l���i*�&�,e�D�#-�}I�)0�
;P�!���[�2��Zc��.�q?�5
(���/���f�)�X2�Z&�W�y����@�'���pb��=v�r�p��W�	~j]�P3������g����~sT
]x:#����Ar�61�d��
�z���v m�8�v~f9���vP�Q�w|��cs�o�*����4�6i3�ZX[��31M���Z
��6p�i��\����BF��S��jir�`\rz���+�T[������H�H�xwQ����a4�����\�D�o���)P�cZ9��R{�T(��Y�AcKR7P�����[z��l�a�'�)C����Ju�E�XQ:cIb}k������l����JT�
�3@�o����~��L��t��
�k����o��Y[$d�@�1�n�.�	���z��(�
��kc����,Pb��-�O.��aBau^��t�nn������h��lp:���B�G��I�F)�p>�$4�d#��L�]�v���?�$�8C����\s����%��\���H�G-1�8�1��xc���/��[�
�j76.��2�0''a|��{Y����u����
)����9��Q�E������g������������z�A\g1J��
���6-��).xlhYcI?��������I�w�����k�p���}<go���Y��aBaZ':�a��f��=�hV68}B��\����3|:g2]f��<����`�;����;{��A�t�����~���x��u���iW�	�x41^��H�S�d���l���|��@HNq��%��
�Y/"�V����tW��d�����U���x
�!�	�������HT����������{*�K��z������o�i-�sL�^��k,Xi�t�Y���NF�&�����X�j�Y���DR)����?��;�8Rh��>��o��������zm�1�_X9��,-���G8����u���J�$������a(D�Pk�o�x�B�|g�IX�92
W
����?)��p�[�$#��=u���0���8����F������$��B��5�)
�5�p���|4��e[�c��'�7�����[�ZC7%*r��Pcl|y����E��_,�<S�� r�� �x:I�h�f]yN���E�����Uz��#�m/~�xy�����J�TI�*�S%y�V������gz Z����Bdgs��51�w:j��9���������}YOWWWWw�)�M��� �a$����Q�������+�j�����+�������	�u������������'���0������oQ�\iNQ�Fr�:��HHHHH^!_�����vqsK���+�2����G�R��AT��@K>��o��!a�>	����c��
��w&,co�k���
ve��������e�.}
=W��%��z|ZU�W�j�rm9�eg� �o3����&�6�"9�Pqzl��,������j+�V��P[������*<���:��ql�.�D���������.'����H�[�[�[��/����������o"ttt��	��7�`�`�`�`�����M$0,0,0,0�IaX��������H Y Y Y ��&$������H����
�:+t���*?"��K2�[�I4Lf��P������o]L��D����U�� � � � �� �x	�����M]������?o"�a�a�a��O
��M$ , , , ,�D�����7!�o�M4o"�&:+tV�������Vy1�/�'2��x
��Tm�C� � � � ��KJ��+q)�+B��=qqqqoq��H]]]�o��R$,,,��`��	R�"aaaaq)HHHH��	�}�"��*�"����[��!^F�e$�WH��^!������g��>F��z���	vvvv���$	�������.�A\A\A\A��A�[�.����:v�W
��F�o��=5��~�1��O/x 5V_�hors�Ej	��o��MUl��
�,�����,9�_��4Mfy�l�-���s����Y��:\�@���K���"�
�$L�0M�4��ivACm��9� ����x�I?��=E�8����kk�����|�����S��L}����0c���<����L� M��Z/�����B5
���V��D*��Q
�2)$BH��!B"V,Wh:5I���Dk5H�*)���_���p6�Y9L�Z�Z�Z�Z�z�P}�����y��G��:��u���i���K���7������c���    _%��5`yv��[�;HR�������R����M�)s���A�Q�u�����s��K�FI���Y
��lQ�B	�%J�bJ�T�y�Z=U����Q�fy-"�fu�(��C(c�{�x2N;C��I8D]<��=$t�~7���[LRP���� ��`2��x�E6���S����?H
T
k���(��_O�����/����7V������g|��V:�O�����-�`�e��V��z��}�
`��k�7���������$��ff�<��Z=����p8"�j�v6_O�8�t������p#�����@��du�O�����kO�����������@����;�QP

����s�Bko}A�v�dVN�T�q������P�I:��[
\3�����2�e�w������E�a^�y�e��KH�
�
�
�
���"�mx�=?7�����-����@��>>><c+nnn�O�1O�d0��Y��
�X�i�i�i�i��O���/�+�+�+�{;�[�2D��u�������������W���@�@�@�@�
!����e��6L��@&d������W��dddddWhy�-J�����0�7I,Q�����?���(ha���
�<�<;	hhhhh����axr�Jd	U&�-�-�-������*    ����*J �@(�P��
U&P,P,P,P,P�J���h�Y>6�t00000^!_\2����?|����:�^�����=kc,X,X,X,X�R,�X�js<��N{���@�����hg��h�G�����~�?]xvUK�Ev���VhI��pd|�9x�}US����5���}���+�X�p�B<�x
����xJ8o^^^��V�w�Q
�/T�w�0��^��
,,,,���,`�P��
�
�
�
��z?��S��z�
�
�
�
�
��R!!	qqqqoq� ,��������������X_�/��z�6k����XxfAbAbAbA�"�$<����]!�V�%���?
a����:B�Y�Y�Y����U����-JO�4��\�Y�<����G��W��U�$����6������tY��)e�8�����U��%�����f�i��
��Y�+���T�����.D�Z�Z�Z�z�P}�.�h��������(D,F�)��|C�Z�Z�Z�z�@}
=G���0�T��:�
p\�����Xop'V
":AmAmAmA��@*� �M�~��Q8 ��~�DZ,C����?������5��^�u�(�eg*?�j����L���������o����N�)&�����*fq��AF[��$��28.j9�J�O�������������*�I�u��#��=��Z}G#����+U���i�D��>%��i�t6�]#���u!�i��A7���2�Ge�k�k�k���u�9�yi�H�Vg��N��U�yD��
�#�f*O��{HR��_ga�kB���.U���Q��/�Z ////�B��|�������_6� ��$G���
F9 v_(,��A
����C0@U8�c������
Jo8�����U
 @(�P������eN����,��#��ZZZZhM�;1:��t������9<9f���M����?��d��k��6�0S�*1^:z�N/H�Nga4\����>�,,,,��
�,�D�!H-H-H-H�� uY������C X X X �+>�E�!8.8.8.8�W��j�G�5���w�(@����?�����@��z��i������W����f��MF���\(m$��F`����T}�S���������������������g����]]]]�R��la�N�3V�I*i��?�o�`�'�i�~m%�2�	�T6L~��
.�w�������!|\|���L����#G�����TjeA��� ����������������G�"�l7�8����5a������Xg%0O�x�P�T^���L�a8�����'��c?�[J��ru��3�i2G�����Cg!<����(W��Ls�(�A�1�p������s�P
�B)�R��Bi�68������*X4������P����{�~pz��7!%
�D�-�-�-�-����w����iD���Z�q����&o4����u����R���������\i��&��0��*ga<�R�3�����
����V�4��Q�LP�M5
�x����Y���y��/R����<�"����,�"��c���J����Lm<c
�t��=9��?���5���$�@m�DH���� ���jp1��@Q�'
��A�&��,����_�Cj� �f@`��*�w��!���X��������T�
�������l��1}:m
M�|���1�QS
���{�������T6;��	&9���Y�������vf��dH����?v>Kxv��
`~�\���Z��g���k��)m!�B���y��B�N���,�i^c�
����^���~�:���������J�X�%qqqqoq���nK��A����*	5�4���$���U/������E���m����'�S������F��31�4444�
4'(�g����bn��T�.�a4Ku��IJ��7a����;e���������^!v��MA`A`A`A����$�� ��q��H��+6�,[j
�,�-�-�-��j�&�L��N��o�i�$#��8�������������A��5D�!`+`+`+`�B�]_��YDQ�L#��*�6���`�f�f�f�f��b�������j�r�Lu��=�,�0T��jH~��>^@U������P_U0BwFS�s�LFj�d�8��o��W��i�`K�V�d"rdj��b2Lu7�U���[X	�@}����g���)"Mz�p
rE=�So���P�A�%�$jF�G��G�����5Q�=������UqxF��	P:��R�9v3����b?��..���"�H�)�S������?���:g�g/��F5���6~AC�Y�Y�Y�Y�y�:~���v{/[�^��a���s�z��+����	"""""��"�(0|3���,9���,|���d2	�!�/l^����q�
���5��`
��Y[������w��t����Fmfx�0�o�,��L����T���a�ka�+1�j!�B��P�O`H��[��i�#�:�`�7�Pg*N�yp���\A	�ncEb&{U�;8��J0����+�M�GtB��f1��|O�j�bN2�yi5M�&PUa�F�(�P�dQjX
	}%��{���tf��[��#J�	��R��@1x�y����rt�j�nPz�R���
���yc����
���B��d\�?|Q�X�f*��c
������ 610�1�%O!O�Y�F�q��@�(�
�0V�]������a(E��,I^s@��P�19��33�f�tcQr/������0����mL��J����%�//�N�fE��T�R-�z��Z�g����+����*������T]
2	� EdG~���8l����F?7�����x���3��C����u�?�
����F���W�q��A���*.�u
��������#����5���,S�X@����������(ldddd�6 {}��e�?����+n�,�Ek#4Hh�� �Aq�
���l����Ee# + + + ���k�l��<�5���)kN�$Z��Z��-3���Z�����i��A�3�s"r��)R$�HH�
I�q~6���T�,O���l���?O��:��E�_YYYYy���
��O9����w�e���nk����g��B�D��(8-8-8-8}�=eG5�,J�n�n�n�������e�v�PM�/U�Ae�u�pAkAkAkAkA�UjDD�!,,,�Y�:�I�O�D������	p���5U�A�����4�.��vFN��wsu��(1kJ{����Q/b�&#����#q|��6�rd�F�7Q��=� hG/(+��6��o���)4!DB��	!B$VH$�D�"�+�+�+�{[�?�+�� 
&S���0��0-����i����n���T!!p��a57������^^�<�t&s�����
s�T1��Rp����c��	z�c���������	:$t�3�C��`.u��@a�����FXXX�����=i���� ��4���B����������	z����p4�#u�m�������������4�kk���i�G:����Xg��F�^2��r�I�<tw�~g��z}}�`�����oN.��S�T����AD����������1����_�F��U����]Q�4ue���J
��w�k:���s��?��HB��.	]�t;t�:��5����y�k�����C�\�\�\���1� �|�m^}�f�����i�����������
�3+����s=���a
�w�^����sZZZZ'W�w�� N�pD�b�g*�R/�{���F9�FI�f��#���~������?�6�J�[`�+4�y������e}�Ogq�7P� �Bh��
�B+VhA���G���(�h� �E����W�[A�w%����F�����65�q�+��'Sa�����o�y�����������������,3���3LLLL��<Z����i4444�4fwNc�G�q&b�'�+�+�+��B�][�Z/�������b�<AhAhAhAhA�"�<��O����
�
�
��|�c�s��y�q.����������m���0���
�
�
��>o��w�9���;	w,�,�,�,��b|��t����+���3�t��S7����q0:�;#�-�-�-���Q�����B?�s5�t�Vw������
cJ����"
F)8���a���cG3���s5��K�r�r�r�r��B9g����T�KA_���W}�`'����6�����]Jd(�
����o?��eQ
-�Q����?T���d:rM����-�L��M��I�l��R=M���@\b	XXXX����������_d+��,�u*�s���������b_�x���	����N0~���Xu>>>>2|.�:|����,9����m?��y��I�e!��v�����9��~��Wd����>�G
����������ks���6��J����\#���q�:o)��h������U�j�Cbf��!&)e�����n\��#��|�[�V,� �t�/O�`�;���x��2z8�?���\O��c4����.�������6�7�u��W��7���S����ye���U�H��^��L���&��F�>�e2���;w�<����>p������x���������h�_����%�\�7���/)�+��(������d��;�#��_���������OK��J��`}��J��J��*2�����/v�K�W�{�E�S	G��H����j��v�%F��L/����*1Hw���]��GC��T�tQ�����I��\/����/���z0Gr
|8�h�0�G�%���*�}��z������g���^�!�2�'�NE����`��8������.s��#�Y��0{�����9>z�CG�/���.��/xKs�+����d��B��5cK�q}�p�������d��D4����W������SX����X��R�h�I�lDm�*=��.e>����UE�t�L5�As�j�����-^�y��	>��-�ER&sY��������$%.�4��$pT&K8=��{����e�`��3�g{�%f�)~?8�u�����o����6��
�hg������;�?P�#UL���1��"w�,������zs���^?�{�l�����^��(Fp@efg��3G�_�^?�s�!>�?�����]���;;�3�r��
��%�C��|4��T�u1��c!�C��R�j�f�H���=��]tW��
X(Kjj����WO����]8�V.��	�]�'���'�Y���Vz\��@�����!���r;��Z?%U��r���[��$E6�~t���	k����\�(������v01�������w=��Q*��*�z���s��WFrIkf}T������Amm���m�g�o0���SNg2Y��1Ta0dk������o�����L��lXs�V����q����fS�*�PP�S����-������;�v;�\����0"q����1���� ����VEB��\�W�hM�P��x����.e�*�����
��e���.�S
��|�T��lJ=�3��@�:�<�`�������C5K*7iam'���<���4x�o��e��� �������-#�vhS�KX
S�'�=9)����*�r
gK���\�9(B������������&2=Wn��v�{������LM��f2t
�W2�����
��7�U���<����_u�b[L�r/���b��P�B���[u�����������O��iW5����I�IR�cn���J�������_l�^���^��V5eK{�2r�%�n��fI|��;~��Ch�P�m(�z-�	pj��VS��rQ�����u�F����:|9z����<��+�=|���T�.�
�53]��R��Z����FZ��3aMF��?��]�pfZ����� X���8N�������/��v����9�(��x��u�\
���SDq+'`y���I��g�,a���i�;:��^�@Z���/9���S�g��1.jl�hAK<o��d[XYW��S;����v&�~2v�d���6r�#���%�.!v	�K�kq	��oR�/�}q'^[q����Vx���[r��+��O?��Buwkkm��U�>v\,<Xx|5�����]������|[[N������������H���4�����]�F����"���w���T�f��0af�B�
���K�
'k�����5��E�I��7U��b����H%�;`��&yZ��`�d�J
���ZpK�/�"�6lL"vRh��z���hC�����mA*7��s���-zfa�u������G��}~4������,�X���c����H��#���3�3�v0�2�2�2�~��+s#{q��[Vy������}/���2�2�2�2�2�^�L����<|����������{u�k�So�/�����W�6!��o�(��e�2�2�2�2�2�^��������E���9)�J'�8_��(
��%��R�5�������M�fr�}M�P��:xy|8��4H���
�6
��[���r��[w��Z�$9�YJ
���<M�O��Si3�0�J�������J�2�%���1����g��8el��n����#<�\QP��L?����r�Vf����&/�A���#�����v'R�9ff0E;CL�X�[��
�`{��K���|X�H�0��W�D6�������J�t���[�:O��b��I5��5��n�M�9o�^��2�4�B���h|c�&>��pp��1i�<������n���Ef��m��yB�a$�C�O�ekR�DC�d�M�3,[�w��8#n�k�M�{;�/�����1�xHL(�
[�e��>�EJ��������:�����
�����d����jX��*JZ�l����#�q����E�:'	�'b5�<�@B�
����j{hn���1����<h��"Pxh?p�Y6Y��4k5,��%<����}��?��}j�8V�����@K1�t��O)o�]�����������V��������{���7^�>v�=�F���U�Aw��zy[[��n�>�F*dmo�D���!�@F�O��29I������A=K�d���NV;Y��z���>k|v� {��������5.��1�+L�Or�g��r���@��y���0���5�%��%���u ��bC}�$w#?\I&n��H,5�|���� H��F[>�e�Nr���Dg�,�q�gB�l���o��ND,k����Xc|���� `9�r������Z�f��>3�&�2�2�2�����l	�&��N�7gz�@pf���=��Ng����{���U��%:6��S[{o��7 Q:t�-�w�V���g���C��} O !������P��SGw�0��xb����q,�X������E����R���n���	���J	�1�����-\����������'��M�t&���U#[��j��Z5+u��r�h ����`+�LV���V���r
�^[���Y�RZ�[�0YeDmj�KQ�P����d�'���
���\c�}K�0�.-�i 6����W+���B�=m���%C5B�-[��h_��vC�l���
S�$�5%)��^N�@�����A�?��FJ[Ic�5BG&��yBt����uLeE��S#6���e)���{L:_��1�~`�[�"�f������U�X���<|	�:�z��g��^�+P�Hba���Eo�IaM��/d�5�D�X�%q��iM�O���d���d�(B�d��������T��iaX<2Ig�<�/�e[�������+��X�b������S�)�qO<��s;"���T����+��
��c�� ��
�+��
�.n7A�*����&h��
�kp��P�����Q�AVv�d8���E���Q�A�"�����w�z��?l}���|���+��<$E�r�!HO=
6V�SV�t�&��'waE��__������V�W�X�o��KP��M�c.���qJ��&<�
�5�zg*���M�y��/�d�RE���L@���J�D�3�_e����c�i��&9f�Z}�=~�������;����NA�&~���|�y��@�#3[\���T�[�{6����c���fhah�R�v�7���}����.��;��/h��S���@��K�
uay?\_�4r��%x��-7����	K�n�������7C]{�mk���Z���y��d"K,�3��PF����vb����];�k:%��6��
&�k���g�Y���tdq
�V���8I��X�|���b�U��K�;��q����c��h�<����C����}��Xz���K��(����
�{p~
1��p�*�E�}�������=���!z64E��h�S��.^??:x;����,���V���O�e�)]m��Eal�+�I��w�{��GKA��V<I�%<8���K?z���w��F~��c�Y�W��p�wefI��d�+]G�a`�d"�2*3��"���oPI���E�Qx��j�*�������L�����6/I��>��s3�_��pb�;�;���t���)��P~�Kq�����|y��w��3_�I�u�g�!CS��G�_�\��ri�F���P�_I:N��4��Q��z�C>C>C>C>C�U(���#�������]�|��>O=�2�2�2�2��eYB�| ��g���T�X1��_���C}�~��QT��5��I�/��r����V����w�&���rH�?�&��s�
;Z���D)d�����>�n�)w?�_r)������]�`K�R��X������pn�E�gj�����@��������|,�X������o�T�lTV2�v�A�A�A�A�3�����dr"^�:�y���fr��d���9�'+�h7BU��f��j�c��/��&Yv��p?���
L�O��&|���{���+�i�)
`���	��g���jZ��J��&
V�b�U3x�S�mUb�T�fTC7��.�[-cXsxE%�*�j�a��yO/;'�E�e�:3�J����G�Qh8?D3���N5�(����1��":�����Y��4�sH�M���e��,�=��6�v�Dk�Z��7k���fK$#�0��&�����c}]��)�4)3�c�������[�a\}���@����$��T_���64�\���.���BUB����e����_��#�v�8eW6\����/P��M��P�6b�,����.��	���e����k21�����<��%q��d�V$i*��E-d9Wp=��w��~������
����63����vf���"�'�i	#��HW��,jX���aQs	��sw�EV?[��h��yi����Z����^g���{X
m�N����JR'2��P�l�i5�(K#+���g��h��[61��aq�������dw��E|#�K@�c�f�f�f�f��|���t�?dS�4�h���d]m+1���\Da��I�
l�VYh�s�T�L���g��5b�m���[f��u?�Rx*��ri+BTb�uF���D������Ie��J8�w�ehA�D��Si�j��v%!Z}���z.�Xq`��9����(��8\p��� 7*���a��9o����pjb����o��6���)[Aa����g��O<<����u*c�� �y�;�����w%X�@CQ����T���G���`������M%i��qa�q1�[��.E\�6T�����Y�C����=l�2��NT�}K4��@�!4t��b������k=�-���.���Go�c����(��7�I�2%���	��qT����
3�fzA�]�hk�o�t�wX���X���J�w�����wA�|��� F���P���@��.�����k���$�����l����f+G��}n:T@�f���
.(
������w����9�W:8�G�����S�O/t!��j�ym�I	WOG�O~�1�*�#���Axc!�(i�:��?���*�>dRb�9�A5'C��4+h&b��7a5|q���5�i������N�g����g��]�|�.�z<��������o�3�W��Bk�B/��m=����^F5�_ ���#o�����8���v�T{�z�F���*����3lm9]�E=�����iS����f������'t[����������6�k�a	��%K�3%H�~�p���[�[�[�[��K�[R��f<i�F6�'+�6�$@���=����7����n��r�Q�Q�Q�Q�Q��(���^�m�dF^F^F^F^F�
L��d�:����*@�^���]�d���=q*
���tK�c�3�gMB��Z'�s�59Q���'~%6��.i�J!�pb/lO�U���t����P�+/��!X`��`�����x@#K�wv��'R�[[[[�@l������*������j������X�(�(�(�({���(�������������������������$��([+-��D�E��xfe�����1$&�R�K��K���oE��![�����������I��C�J4/���kP�F����V4�@n�Q�SY����j(��
�������~��!�:R��X������8��]Q���h��E[i��M{�� &H+��������7�o����g"��p����5���&�q�a�a�a�a���ax0��k�Qv(������Li�D[w�v�+A��
�nxX��@"����_`���'w�Yg}5S��"�FU���j�V�V���s���;�����iN�����}����F�[����* ��\�K�:{Z��M����%���(�&��+�H���Fg)�.��w���'��+e�M0��-l����U���z|�����]UL`��]9K"��Fp{m����_&�u6K1�5�)B"�-�{S����Y��m����|W;�$@"�\&%�o��R�K�i���BT��/����/h�~�?�`���)�����W/����;�6:�'	�x����:��#i8��UE����%�6u����B}����[�����1X*���tf�#�����.f�����f:���.�d����T��,T��|�s��{�&p2
#znV�Y�f���jV�?��!"��aDn��j�����uAJ*033330]w��U� �x�t��a�+��������*����x�(�\�:��G����dO\N�����WEX����*��.��#r	���e�V�D��,+%�G���(����!b�6�	9Rl+������!P�`�RKW���t�6d3�a��Jm�:j��kc��M����-�?�=�r��Den��~m�$�x?#�cB�L���|,�S	�����Zv�5c:X��~�BaO���9]���V�-�u�i��;�B��!e'q;���������"�"l3�$p�b�V��RM�������vs�
��5��Z"_�mf��kGS�$D������o��y��xk0��������<d��5�LX3�"5� �XOQ~G��n���g��a�qG�|�V�sY�[�xOC��:���Ro�Q+:������N���44���fQ���NLi�z����E���f�	_��>���7"���<�����`+���!b`�	��g�2�e<���To�g�Z�'+��p�p�p�p�������`E���U�`�&\��j	�88y�����C�O�u�]`�\5"?���Vy��GW,[�e�!��3,g���y�����;���\��]u��x�s;��6'���vv�����`����lL6������_���Z��z�gV��X�7��Q��O���xF������#
����(��3�	��{+��K��/AM�g����.+��8�4�sh�5$y)�li�$N�1z0�������k��H0��o�hc�c���e5��2#9_TK���!j��6�Q�,FY�~�
�tw2)��7���o^�`�\�Y�Y�Y���Yt6���F`�}��)�z�B�
�������_"�n�V O���r[q�����������]��gw���Q���U�������}��@,!XB��`	��K�O[����1�2�2�2�~����|�z>��2�2�2�~��6�7\|R+�j��Mk�"_	Gc)c)c)c���[���=y����TE�b�d�d���Uo�QNt)�*��j*1y�J9��b8���B4T�1G�-u���%~LT�j�(�t��&K�����@�KGr��,X����g��6��������������2h3h3h3h_�f��'�'�'��{�����2��|`���*����W.�����]���+�Tu�r��\N�����5�}��+�Q�5�VXQ�n�e�_I��g����u~<��Y�?>��GW���+>��C�D�U����UW���8TV�.:���@�j�E�����F�Q��H��eu��P��h���??�@���N�:�W�\be�Q�Q�Q���G{K�t�d�+��Q���,K_]�3�tWf���-	����\�S�#�&������������������A�����������x�#��o���'���ndI���%�u�tb���#�}�����?&�O����)z��e�q=f�\O��_/ \0�:�x����L��cO�,0@�MS,�XX��ba���
\^q��m��l\:De�d�d�d��2�3���Y��%�mQ����������7�yf��&D��s�������_.�G9��(T���H�6�P���>��������������k~|�AS����0q�|�z"�b9�tT��qI�/���l;a����%K�+����}��
j�
#1#1#1#1#�e#1���Y��bw�>��V(4�.SI�;2�B�@�I�3�{<�<�����C@k�?��n�n�����X��m�Gb0�B����r)Ng��d���_T���V��MJ)W���B��n�z���t��������Y4��C��S	�,������a�����*���-r��J�a,��l��X:�tf������Hg�����U��QEW3��fcPfPfPfPfP~?Pn��O�����]�I�Z1b=��N��D_���'���G�y�w-��K1,�3��o���{�y������'OW ��������{eee������i�pg��aJ������o}bm�����h�H%Re�����2�$oS��t�J	9k����Y$�Hd��"Q�H�d��������������dXs�'��(�����������~%��T�2URV������v�
�|b%�j�Fi����2�3�3�3�3��/�{���q[0�2�2�2�2����C���H�G;�^�:�k�F�Fh��������$���uStl��X#�@&��w�[27�#$r\�p�[�P���3f,��Ta��M��M��F��.�B����O
x��E�����p���7:,X6�l�6�������(�7�~��~o$��L����{��Fp"�>��{C��Q���	B���e5�M�2#9_TK�>��K�poI4�;��<,�X��c1�b�S������k2)g7���G�����������V���B8c]��J<���/����9�vK^i
�����a��������������i�iFnFnFnF���$������������Lq-�����H��?<{��]����������������������l����Vy�>LG����K E�������;,wX�|���9N,h�t<��������D�����	�h�����c)�Zg>N��x�j��UL��h�����BU{d�XK��)$k���������c}d`e`e`e`e`����M�����K�����K����x6�c�#����C=���h;y[�a-r�5�0<{%a��`m�����}w$�!��d��b�k��������<�BxKN��w�����	���O�K��}�h�2R��Z�z���,O�2�T*��HhX��h�"�����p���"�W���yV�"3�� 1����?����"M�tT���j_2�=�,�X���c����$l����������}}}}}���D
�Od�'��z���3�.S9�da���	Z#GA��
�y0�}�+e���r?,�o��n��#�?������4trC�6�B�Wk�.V�]���h���j�T���,[�SHx�p�D��j�F�E]E�(p6~�s��,��������.��:Ku��,�Y�_�T������O^)�1�1�1�1�k���������S���������k��N�����g�e�e�e�������]��;���_���S�����Y��7Tx�l��R��X���)(���,,XX���R�EO}��I��z��^BE�������?������o����k/��H��O��O���@�!���$���J��c��������A�kC9 ��%5e���(�������.l�-���$D��j��"]	S/��d����Y�C+f�	�"E�@c�`�H*4�A31�
��Rj~�C��OW��%&KL��,1����Hu�OOOO�=�~���{`��!�!�!�!����(E�G9���1;��:��PL%zW��U�V��T�b��"H�6	4����2_��(0sO����D�����&K�.��~kv�L\UF�������x���0/_\Na�e�e�e�����l�`dd��A��)��Z)22�u�����d,e,e,e,e,����Y����K_Q���.}%]�*��Y�e�p�J�f�(��F�6dF��Jg#�M�	#������Y�(�����]�E�6d��'����O�
8�����C�P���>��x{�^"��H`_��`)'��E�N]����Q�	=��K�����|)6���N��;DY�o�:~�^4;�-J���������:���n�9��%�]!��/�8�#��
n?��+]��R�~��!yZ�-`�=4u���v���F�('y�Z�x�\N*4QTj�y�*�Q�8r������_�X�|������r�Y�3l%�a_/D\A�9"X�`�����O��!��J�B/��4�d�����(\v�0'�(_�
�0`3`3`3`������3
��1<��QX2333�p�X�P����D�)��D�M)OEU�rC8�2�2�2�2�����6^�Q�Q�Q�Q�Q���D��W��N�f�x;1�2�2�2�2��X�������.l%f0f0f0f0f0���BL�{A�$����FeFeFeFeF�
��A�aFXFXFXFXF�
�n�c�Uc1�x�������I9<��9UY0�������6iEF����/�bm�^]�"�IsYNe�0��R
��E�RU�Kjsg��f�I��-�L���"��$�L$'���q��$t�I���>��/lD�A��Qh���V�p�}����z.��Yx&&���L��K+-2M�Q~�-���J*�2��$�{%'I^K�Vi�k�/Td��8]�pe������u��n�_��;����*�\_����.�][&9�#�xZ��"pX�0`a���Z�e�����DG~��h:��B�*���}��O�x�x�x�x�x}�x0�$iM*�����W�Xz�8�-
�[�4O��g2��TU��CZN�T�SZ60��T�b[8�9����+P�P�91���������,!��}��=��;U��8[:�?�T�����
*�<$�/��"����tq����Hl�jx$�V����?mR)��VE�_���Vi�2uy�N�}<���,�J��B���c��
m[62��Le��!5u*oe��u��#S[�K���nm��z�v���PV���
�B�biS�FU&ie`b���d�9r0{�4z�����j��u�m����y�k�c.�a��n�a��v��t��oe���E��[��G�~���/r�l�@hbs��`���6��VX9`����+Ps�/.�8�8�8�8�8�q8l�`����W6��;
���(*��� �Z|��6���F��
����[����X�7i����,���\������a33]�Y���q��z��_k����un'b��������~������/k���S��7K�>,}X�����]��:�up�}��c�dfff�(�W�!���.:�G�>�����[{��P�rWMT�X����>CfC�i,��>�
}����K�,�����L��H�o�`���:|9z����<�zs�����
�,7Xn\�F�����b���R�������p/`���<&���o���~�WG����?����NT=mk+������!5�����YV�9hg].^q_���h�/_>?����I�d,u�e,�X��,c��+��-_��26d~B[�zels�W+c�=�g.�W�ny.�][G���'��"&,�������%����n��Sy���p�u�����VyM�T���bC6U��J���s�~�^��.)������-J��e����5���t��B�{��*O�����������6����/�������iY�JK�!�:�0�J��D�O�a�}�"���9QR����(�2��\15�(7�#��>�R|���ZkA�BZ�<��a�effff��L*�&���{�vAm�2�2�2�2�~-K6�X�X�X�X����&��Sr�8W�B�}y��,#-#-#-#��"�+�'��j����?P��jxm��P�e�e�e�e��V��A��!R(W��3�~��#�q�q�q�q�q�,��3�Rd�����������������-����+�+�+�+����k���e�����A��K�E=5
��5_f��6b�U=mk*�Ql9i�uG=J=4��,%���(�9V=]hc&��r�����++�P��;�1��D���NZm���D�����.��cJe�������#'2r|�I���m*K��,��V��81Hd��'PkC�*��'T���B�����D��$*\9c��g4<�8�ip��6�1�2�s�
/,��}�LM��]j�It"��!�%Tu�m�Bf�n�i>�hMC�~�V5zho���?kUI�Wn��1{jie��y8�6���w<�vn��&]�����cv����t/��Y]d	43(���u�b�CO�\�|�'���1��l,G���G�w��d0!0�����Bnoh;��A�Rx��J].�g�6�,!��j�Y?6[3���)�iK���+�M������,1�JD��yaF�95���8���ibs����.��Y�t��cOW
f+�jM+hO��K��O)�,�=\��kTx�ubH�C�����b�Jc�������%�s;/�?pW���b��q�Sw��h/j"��T�y�WO��I�t��$�}a%.y:�#4����I�|Z�'�:
�R�@�Hx�YR��=��]������*���������?��dB],��c�1�p"�y���u��y�C��K�J��}�e�+T�<����8T�ZP}�R������zkW����3��.*��t
��Y��}f\�z���)�H�*K�j8�ui�KX�$��jd.�j�G���`-��k�����L�%�����������Pw|��@��K������������]E/��j��&��z��
0�������:���y��;/�y����,[
c��<	� ��1\�����`�$T�@S`J~������������}UQ7�����E���n�E���p�o
��4`�@�@�@�@��q��i����$>IW|�9J�����HT�����%ds�(O�R*NIv�i���de�P��'x�h&�l���?;�����,�|y��4�]U��~�n�F��FAAA�/�����L��}�*�w���6�2/���b����Ul��x�r��c�=�V��������]zzz�KO:�}�����P�KuYJ��6&���b�f�>QE�:F� �?���s�s�s��/����qA��������=�#;���z�11���?���[=���A;�\�j6X;X*������dA�����Q���9�����	E�KW)�4�G��:A��'o]z������������w[]_��W����x��K�(%%n�i��u�TZ�o�����
��]5���r�&�`9�r�������(����"�eW�����G���^�J~o�eC*�������r�I^�$�d{�`���:�����l�QS�3��L+�z�IY��^�2��mH���U��$2�\P+���2�D�L�L��A1��L���b	��%��I�&�t�i���Q9��`{o�������o|�X��4����M���R�9.���q��"�(�B�f$�q&2�nDkR!�ED�q42�G��P.���`1�pb,�X���|!�����?\
/(C+C+C+C�5���-�S�;5��k]�}A�b��g�����LD�	Lpx��n{����01pn�Y)�����s�5����0�=�m���`C[��|��Hz(>#Nd�;��4Y"�a�^����I���������!R��d�ko�,'���ZU��G�%��X�m�Y=��_��.*����1�c^?|L������@��@r;A���g�	�4��H�I������0�%���r�2�
^JT�DW�~�H1���Rd��I����-�R�/���*�\qp��D�(2�#��x-�t�����Zc"V�L#����5
�!�M=6�
���
���F�L�k��0�d��l��� �f*N����>���H�)��b�����:�$S�����������_-*��������)'m��������F���}���g������T���6����QS��j��O�V����z���b����ie�6�����J����M�������U	mD'�w�%&h�<�^X�������Y�j3�?
�����,dY���e������ %�z>�P�t��BO :�tn�/y�������o�wAO�rrrr�K��D�3A/a������i������}���i?(�<���������w���x��[�"�jC-"����X�P��5�H`�2�e���!C��x�l��u��.���T	��OX�����~���Ph���"���������L1�0�%:�0b�{���_��7/^��K��%����9�(�~�e�D(�\��5�
RI��c(�@g��1&�����^gDo��)�x�x��9o m�F�b,����s���Y���u�d
�O�e�Owm������T2�|�*���E�S�����X��le�����*�t���zzzzz�f[#�o��B���+n������m��
n�v�{_$�����������A�;kmh�6a2��WwI�����������K�����M|����k���ucx�!�~���{�A������C`#�������
�8�!Py ��G�]�O�~�����������*����s��S�����y�sAc_����:�����v�6.g&k����(����qt���[k�,�YP���ru�?����f�}��4>��`���qbe("�#�������ktISS���^p)3�
�s:�E�"���<��N�'?����_g+��j:��H�
���rV��j��JEt�
z`)��v���S��� bA�����DkT"�;��
I�}��S�����/V��a�OQ��
���->q�����&��
��ic�8(����\]�?k����O+i��*O�e��1,c�������$���T��K�3Hh��H����C�T�L����������K-aq��cl8��]
���>��m��qh��j�5nE�0-^��'���p��f�A"�L.5����S��f���9?�aoTT��"��U�,R��%�p����!�oq<s;-t��L+c#�3�*C�9(8O).�������TsU��s�`�p;��5���E]�'H�=�i�p��L��I�V��^T���e�������7c=@�af����6~�Cw�����5Z���R���*2Hf�b@���K�u��C�LzC$f���cy���M27���4�pq�`�8��S����%Ms�g�R����SY6�'��J�I�T�H�MK��A�,X~_J�^��7�8����v`:f*�aty]�����N��v�����zy�t���P���U�fG�wsd��.FC<U"�����w���ww%���
��-�)rS%���y������$]Og�$�����`�N��F��*��'��X��h���H4���j���������jXcR�[��`E�������+����J�Y{%�K��-����	���P���Q���������THE�V0���[�n^v���$�����%��[Tx��f�_2�����W�9L��w1O*8o�}r���.Ux�qS��4�h��wo�6���t��
Y��v�����t\�5\'����g���%6����7N&!2Nl���oGf#U�;V�Y�g�|V�/����E~�����#��Q;��D��M%�v6���p�{&����~�]k|��
+^@<�������m��_`Er�����&�����,�nCD:�����������/�_�,~����.�����������I��E;n�e)�atetetetet=7��DZ�����7Z9�c��������zn@���WZS�X���h��H1�����Y�u�s����F���z���|x���)�f����KL`�O��PE(!��i,*XT��`Q�-*���97h�7��
<����N���.k�����a�U�H���d�e�
y��9xy�N;YYY������JjW��������J�c��o5����
c��^���H��3���6�IS�uZ��/�l�����$������-�}o���#��}�,,XX��`aq~[y��f��j��)��UU��w�a�6�������������x��5�U�3�Y^;x����������k�Z0��J��7l�Gb�f�f�f�f��&�=��*q�����R��?<+���VnuS4B`(��������O#�v��w0U���32�����`��D��qc��`�f����eZE��-6G��2n=����X���i9����{�|?�,0Y`���&��V����~����$M��Ze���fhk��YV��O7l-_:������:�j������������/���!�!�!�!��Cf���0�wr�7���;���L���G���?W������
	�(��E�,X���w�C�5R%�qL�JS��2����Y�4G�0T3T3T3T�;��������;��������;vg�a���i��s�����eD����.s��d	(X��	���`��=��S���,���^[���AR~�.���`�oa.���*]b���3�
������,^�q,�
�Y�V�O�.�=#��^�.�1z� ��vt}�F}�Fg�p�(�hS��vs�y]�aF]WL��n�S��i�������i����iz�r7uV�a�k�
�6��@��)"��xDC�X������a��HM�<��
L(����!�!�!�!�=7hc�l�]�YWP�Bf���� B��mD�!x���A�r�Gc���������������	�w�X=yT��7�
�������B��t;��6���N�������%���"���[n_sKF��"�K��V��U�O��B���j�h�]�>
o�X�<�a����~Q�]�^Nbypj�jw�����a��G���fv��!��?����T�md�*`��k����K���$Qy���#������ZF-��X�����qC��MB�l[�M���6��V��j��M������*t����U����v�F
�dPBQ����]Qn�ov��/�&E��
�M����������*�n���PHj����:�o�N�����LVp�}��sx�As�'+�x���hEu��V��E/=�����}�O��N���~/.�?�S��-2M�����p��a�Y��zC��'I^K���!L|�RA%��p`�{�`w����;O�������yr����}|�AJ%���}T2���J&+��d~�J&���d�jQY�m��[����:���?�h����A�N ��s?%U"n���������k�
#�xZ�(D�,X�0`a��	���9zM�x���}��,�p���?j�x��������1�1�1�1�1��<O*���������W�X���rw�����m
���K��!���]����s9�����������:���An�c������i���|�����C�����yt��d���m[F������C��TA{���,9A_�I��L,���
�jF�*h��^U�ZL�dj�i]��$��YR�o�\�
s�"!��������8nj(��&uUcS���J���mCJ�����xS�G	(a���%K��N�a}��L��������/�}�:������#���v�>r����D&%��Te���o1�,�>#���������W���������������z%���������(pt���+�����x�n'��e��_<��g�X#��
�2�F��
�z\}�OD���]�c�*Fn��{�������������E�C��>�[��.0{�����o����>��l���Q���W���������%�A�yy�?o��^�9^5��3�����7��e�����A��������Q�"m���l�>J�����=~�5��w�i���.���u�����+~l&'I�W���dz:x~t��h(n�E)S=-��e���`�|/����M�@��j'�QX��.uk��u�#���e)����qS��7��M4.��p":���0��)��/�|E�!jZ�R
U=��|\BH�T���O1�!A��1���G��T�!55�m�OM��6�@��R0�<t	'p-�S�������\N����:���������;:z�b�5����3;��v�3��A���*�=	)�!�w��5��G"���v_o���[��s!UZ������j����-c����@�&��!7�E�JN��uU�e��-,[X�\UH��^jq�,-���$�MR�46k���i���E��U�H~��RVuY<�����z���,%R����1�m���g-m���_������Yw\�]}���y�M���/�]?��/�����`}���K��,��~<L����Z;����1�^Y��q��Tr��_x;�� 
�6O�Im#� [R,Ayi������7�v��T��(,X<�x8�T�u��_2�-��W��;[��\w�v�_�_�_�_��s�oC��k�b`e`e`e`e`��z���(X��fk��������n4e�_,C�k�n�n�n����X��P4V���*M�����d(f(f(f(f(�-)%-�o�����L�R�I�l�H��Q�/[r�/���A�n{���b�!��3������|�u�TqN����	3�eldldldl��1�|7�\5Z0h��-,���
��b�|e���>'8D�Q�w�����iL�
Y[-�:����E�K�(��Hrz���kb$����:����w��~oL�"�V�Y��xa����JTo�7��O#/dG1����a������G�<���i����
5a?�<���H������4�p��O{k.��h���yes���
�bt�K�0�D��������\b�xP"��������c�F��F�j�*�74^���\�Y'�eB�(�,)�V�}Z
�|���I���Vy6
e���[��pS�UVaX�a�U�+�����BL�!�s�0N3N3N3NB���&�	Z��^���#Z&�#�(FP�o�~���$�D�X���o5��}������H�yk�u�y4����a�u�c4�hCH�w�[���r���9�s���3�6�
�-�64k�$���q�''�����.0�bk��%H��'(2�?���:�60���xp�Q������a��u��<��q�N��H�KE��Y��r�	����t��������'�Of/#�d���f���������������{����0����������zn|�jY�c{{���[�W�����-K�
xB����4�13i�f���S��@u���{��}��,!XB��`	��B�p�W�W�W��+���)�N������g	f-fs��'}�K�c�JJ����[g$�a1�b����#�N������$H��������=���`���������Z��/�=�\�0�K\T%����������P�G��,X.�\`��r�*�����;�=c�d�I�;�����]e��L5I`���:`
@�{�
�M�D�bB,���*�f���j�������������wM2�{B��O��,�H�����Z����\�h
�X�8�.1K�1���D�A-!o��~|�)q��e�IPi�(�B��RLk�WQIoh��#�K]b��%��u�^�(G�y��``f(�*R����T����M����\�	<�|k�?cAO�B�����_���N4K�����xz���a.����`�W�S��m3p�������?��dc�P�,�Y��`g�����������affff��L��&���*�X�X�X����X2_0�2�2�2�2�~ ������<�E�)���YFZFZFZF�E��������Z6�����2���&c-c-c-c-c��X��~.�_�U�U�U�U���u���P��-����k�������?�=@g\�-�P�L�T��2�.E��v(5b�l��v�R]xJu�K���xO��2���kE�0*��C��q���DD�T�DF�`{�j���J���Ha��"�E
��)7�`�\�V�G�?�	�Cq�1�1�1�1�zal�:�>EYY�`�A�SG����{
i�*<$���^w�%��6|���G)��,l5�n�\Le5jU4
�^2�����?"���c���
{Xd��d��"���
�;��
�_a�e�e�e����;�c����/�����{���0�A�>.�DN\�<[a>���I�}/�W�2$�m;�P��s��[��:1_���������v��{�-t�x��p]��M��>s�B/�������9��PO��0�L�u�n��V	V�M�La#��)u%N����y�������q�����SA�Rq�U��I��'x�a1����J��e��r��.��5�{$�W��b�jm��\��t�!���1
���x��>G��SR%�v[�����~�?�+���>!*81��
m��{�����)�~����g��G�%
K�4,i�G��C*D������� :C-C-C-C-Cm���~L8��Qn�����uIv$q:K(JvR�j\f�$�W�t61��k"+�%'�	�m?���j�@��GW�'�y�Q�Q�Q�Q�Q�\&[M�5��k3>�V
�4�4�4�4��eTEqeP������Is�J�Q��+z�������������`� �Y%.+>����8>����
�	����	,�X���c��� ���rv5�g��j�����q���~(�������44tn�0��,X����������yyyyy���[�NKK����>n&AZ��o-4y���A��C�i��>��am=~��������ug�g�g�g�g��h1������c�������=��1�6��C������7'�
���N�3�

���%���-3�O�.r�L��Jg�����H
Q�7��� |���F�	����u������[]|����68����0e\gr������p�df5��{��V�E�Z�il�a��R��K���v��^��=��0�2�2�2�~Pd�t�Q�/��zS�T���X�BW+�H]�8����
�4�4�4�4�t����TQw�s?�j,<�y$�_�J�������_[��T�*��B�X#�Z}�LdH��e\���eFH�oE/���P�rWM�BU{B��b�����1�"�t)�X�e�5(��\�

O-���p���h���q]�[Iv��#����
�������m�*'�v�� ���BK?�~,�X������#�PS������.�|#m���J#D�6��g���g�4��4��K.�,���s�#o�� i�r�R�v0�R�DSm���cdY5/���w�]�q[�N���EV�iF�R��<Ky��,�/%&��oZ�g�*z^���!�!�!�!�!���i�����<J�lD��_zf#w�r
��0��_�\����tZ
��le�E	��
Y;E���-)��$*'��,�<�����M]�|!�&�C ���"������H��� ����c)�R���5�rvS������ikS/e���5�`���	������2���?��G��??��>�8m�)f�e�e�e�e��0q�s���Zy0���Xn�oc��R�&Zq
1��5��)a�R�K1U'�R�Dc�5_�|�l��<��1�
�;� �|��k8�������j���^c���E�2���J�V����>O<���Z�SuRJj�6t���)zo��n�-={{+�S���K)�R,�XJ���,)r���y��g��1�2�2�2�2�^����!��}���S1h�m��D�`�`�`�`|Y`q=N����4���EA5�&�OUF�S�
�A�6\,��?�2
�!.@��`��X
��B�o�
�%xDf�o�iYYvPf���Lf�:�j9K�,	X\�$�����`����N�Sz-wo;����4)(�E��7�5�5�5�5����uk��Q/d�T�}?}*��$W�+�����������{y��eK����V����5���$��5Ok��o��R���M4�f2}+@���|E�Lt97��F/����p��`����6����������*���>�%KS��!���,�	�%,JX�\GQ"�Zg���fwJ.
���%�$
�v�`�,����a���y�U�����x<G!��I0�.2�Uw��-�'o�cU@�����g�f���������Z@�"k�y\��Q6�5�Lt��DwSm*J&
�M��U��3�kw!�e�Jy��C2Sx
Jd��m�����5��C1���D���&�	���?�R���<��%�&k�,NX��8aqr)��u<�$���a�f,f,f,f,f,�R,�
������/��YERQ�����Wd_�(?���mu/�99���R�Q/I���a��O�Z���%"����"�B�x�Eu%��*R���f��<�J�dKLP�^�2����Cv��_l�>;������~pt�����=1��D�4�Ix<K�>,}X�|��������Mh#42333_ZnRU���Q������#��z�O	Z���&�<�N��|��5jZ�R�l!�5V���4�V�W���������/��do�G4���/��c\�5.�z����ur�d�+}�-f(�
?�p]1���7�����FUHH,R���}$��&���\��]��t����v����cy�O�j��������h��%r?�s��h����;���1d�
C��UVaX�a�U�+Sa| �'VaB-�O��8���U��U�����2��~��x����o�������:4�������:�����yE���p����+~�����g$���q��A'y�dg1��UF�*=���+A]n]����8[����X�DS���S���i^g��o[�t!��	5Bn{��;��*�TH]�~�mU�
,��(��X=:_�h��I1D�Q���3�d�R%�����v%�����c��>�yV<�������7����A:��u��P�exexexex�N��ymx�P����v%��6$>��
,X*�T�R�LMM"�0%jDFo_�+���q������4�����()�T�r���f����������{��{��3�3�;�;�;��X\��kd���x�:��1222~qhHp��{�
��y��� � � � |5 ������`���{�
�������{�o������5����6�L��bI���ik�����gBFzFzFzF�������w��	�!�!�!�K��F�����we��:�����������hd�]K���
�z+Ry���T2���D�y����U����L���k�}k8�Y�}!�������WyX�]�u�2l��s,����}x��
/��;���l��������hc��v&6vf6��r�����n�6
Z�]��u?�e�k��$Z�yB�H��H�����b)+���3�J��j��ZV��2K��_�=�F�2�1E.RNU����3!L)1t[?����UW�}\�z!�|)�5�������+��Vs�'p�Zm;����DH����0��o�Geu*�$A����s���,2PPJ}��<����Ji��q�����z\�+4�M�]��h���������)�A���z�emMMr"G���F[?c.c.c.c.c��b��=D������U�T�T�T���S������������zQPmh�U��$�2�p����������zah��H���/�e���\�Y�CefSZP\�XTTTT��R~}����MHw%m�I�5*�a��s�
1�2�2�2�~�������[@S�Q�Q�Q�Q��s�
5DP�P��~+�&�(FWFWFWFWF�XK
�}_�r���1U'���O��C�����o��eq$��>�!�\WBM0�S1nQ�62KN(����Lgb�W�
��u�&������lO`�K�1�"�����Q��%�ZB'��"�`��q
j+*�������4�����M�,�X�$bItN=��������3�2�2�2��Z%f'���1����u��Q��8<yg<��k>��Am��+y��TV�V����2d��|�`<���m�	�a��Q`i����K�sn������j<�
�*�*�*�*��y9-sTw�����F�m_sU5U+��m����������02�i5��r*	���p{MgE8+�Y��HJ)jC�O�*H,cTZ�Ii9�l#G�����y���8b�R��a1���4h}
�K�x3v�v�5�P�St8��:M��z,���<w~
*���|���s��mS�8�������I�������FT�"<��{%�g�S�u��[&��y���������L������������V��J�/!��$jq�'H�5x�+�=C0�I�LE�a�^�:�v?��L�P��$iU�T���=2?(C���M��,�����������t|\�<���Vri���yz��xnI�G�yZ�D\b8G���t�
f�<��Z������sl�.�O����[H���'�[��Z������f�fF�50K����QU��&"4���Ch 1��fF;���p=��>]���Zg�o���Fu�K��h��"�\]C����4������fp��d���t#R8";��m�O��0Z.����{_�%��8.uU�����IR�cj�:����.t�zm��&`�������6\���M���������ai����MkxI>p�=l���|�:��$���N^u`�ja������9��y�j�I�^�2N�r���UJT1�:�P�v�����
R��e��~H��WVp��	,$�j!������?m�����ds����������@�@�@�@�@�����7;����B������33333_�!���s�E�O����;���A>������3
3
3
3
�(��I-�����������|�JS
r6S00303030_Q�zT�pQ�TC�J�����!�PV������V����\�?�t�	��.]�7^j\n8����Lf��������P��v���jb��F3
�C����m���+}�b���0�?�$7��m�_{;�����oZ�:�?(3����x��B��)RW�
�����T��G�k8�N���ylE�x6��[�����"��mR��r�����}b$|��3�3�3�3�3�_	��q��Z/&�\�
���L�z7c0c0c0c��;t�qrs\% 6V�&u
��������V����j�f&�Qbf[�X�>B|�m(��x�)K�m���B�U������iA�n1�Q�z�n��A������Z�������N�3|lG���B<�<}x��u��;
�?#D�*�v=zD�<����N���Ax��F��[��
�d���im�zaM�7,���J������
��mf��{f4O�tF��z%C��Om�T������Z�j��D-x�e����N������E]����!��m����`��$6����9AB��Rfu��� $t����m�k8)
rH��,8�N�R���1y�l&��&E�R��ie[i�m���="�3<T��$���f6'����%]�<Y	tW�I�Q�Q#�6���9Y,d����6*N��d����a�������9j`��S�bF3�jf�N0�����~5'n�$�J�
�F��}�w�9��v%������q�8�PS �X��y�$*�v	�}u�������4����,AG�`c���,I�J���*|`]���z�+y�ZP�?���(���0��pA�� �j���g��6�)u���+/��O-�5'��6tl�?�����;��&���=A��w�<��K��I�O�t�[�=-Sz�.6m�	CI�����W�"	6���n��J�S]�z�L��h���E��0y
��Mx �3��$c��jI4�/��'�������H�+�������@�W�-{m��TG�Qs���_��"�����z�� ,��S:�;q$X�����%�z��%���^~�����?6��������GG�z������+vp��;��+���������Kc����i��H�6W�a�a�a��Ogs���ZS���v�
��AV�6[WN��le+>>>�>�1���mq$)���?�L��8{`$���sb!F����'uA����mk�6��Jg�a���(�������N�Y*
wt��������lu�Bx*nl���U�����:�0�Zl]���
�+(�@���`�A�]0�����
����^7����,�������D����W`���c����m�z������o��<�>J�%=�kv��7.e������YO��$u^��ZwLsO)�>e�K��S2Y�d�L�
�uS0{e7�)�)�)�)����M�����<^����1�1�1�1�:`������+��2���*qf/�K��Rb�N6J��,����g��s�!���|����{��;�f����yx�����w`��{�_���q�Om�����������&�
�����~!?��w/����mV�+���J��)2Y�������&c��|J�/3������������
���p���0�W��~��8��O�D�zAN�_J]/67�`(��TMv��B��7�nG�%����P���v������?>;8>|�������8I��"���J���e��Y���)��]���o�7:{�L��;fovs;:<W&M2�v<��������5�5�{N��0���<���|o��R.z�cm��e������/]�r���Hg��m_���o���r��h������mI����+:��1��d[q�8�3G�����������;��""���L����K7^H��lQ����Ilht7�������3W���h]�}i�v)�$4�e�����N�����R,�}��W���WSE�q��4���j57_����$ +�������v�����[}k��x����FRoy�R�z�l[������L0�����Zs���~�����UY7�2�A�5T���,�g��E�	�hB0�yO.��Kuu��f �aww�]��}LU���E��r��^�T�lo~Vw��B��oJ����Z���[����A/0��5
"�a����V�`F2)�MT�.��H�T�^�
B,�~�����PlT���ZO,�q�0�:�3/N�YL�p^t��	69���D���Ms^8��(��Ac;[���b��6enU�H�uN-�f2����<z��m�4>k�n)������l�����v���I�h��m���Ul	���������~����P0��a��8���	\��?8PJ����o������'�7�=7�gRV�>��6>�o�A\/���'���x;�F��'���
������.h�x��rp��"$!��E�������x��'��~��y�c�c�c���
�[4������Ki�Y����a�����[�Y�Y�Y�Y�y���I��H3�� ,sV�����R=��6d�	��I���O�ll�����]7�D��-P.P.P.P~PN�����q��~%"W�ju�}tq��#��Eu�uG_75�}�����DO��Z�Aq�.y�qj�&7.� �o��q��q�G�4���T%��+�D��=���W���M�u�N�X 5s�x^��bE6��y�m����)�!�*�������3
l��:�f���������#������U�������_D��<y"��(�(A�#���h�-��P(�r�*	....��(B���*�a`�2�#�vu
�I�����[ Z Z Z �s�hb�����E��������MV Z Z Z Z z��������U�b�[�;�k��`D{��8���A!��oM�`�{����iuy
z�����'���1��'qz�.��K�1��iug�w����x�;_��/�&����	���nT=�Mb�)2��z4�h��V1�"N�T�0L�7��&���d�����@3^P:�����b��8�M�OWk��"����,��Z����u�0T_EZ�
s�r�1�z�����E
��RW��������}
'^4fo���P)G����o�nqu,�}?']���J�4J^7���D2�������?�0��_m���l��w�� -
�(�P�B!
�U*5ZcpV���SD�W�W�W�w����76��B�k�s@
��`�{�,q���]��*r�d�@@@@����0������
�����������&��	e5��$�<�L����&�S8�?��|�nP�3n�=�o�+S��(�5�/�
���������$���{����U7�v.<�Fr��CW���}����EJ�*�s.�g����(�Wx�{N��*QT^�bj��	z?j����'��A	U!�d���uZ���d}rS�����r�`)*ry������)O���Q�"�0i��Y��"{b��	�����,�C���������a5^�o�J��OQ~Y6*&�4�F���ai�3������q�0f�N�g8��*���k�n�h9��<�����V��/�g��M|
���~Q���[���
dA���U�H�D6���k�4�[�J��� �7�_a;%La`MJm��f
K^��5�;�'���)�dw?V������f&��hf���S3[�k-�j}}[M���=��&�V�Z��j[��$�-�-�-�-�}�M:��}�H��������4�|N{��Q�����ft��C7���	h�pZr���A2�p�A���M��H�S��jH���������(�)`+`+`+`+`{jv���"����'���@^���mi6���|�4�6ZA����F��Y�,��(�!�U��_	�J\#T�%�3����e~6����OQf���l����\�o���}-�ft$4}@������R}��S����n|���H����cN���U��\w@7���Eu�����K��9p�Rl�#\q���:6ka�r���������M���/S=]�/w�c���s1���KL��^���8������s!]x1��g�������������uG��P�\�,��+qG�g3�*�3�������J8�o��zw���J��0��5����\5u[�!��4��?s����m^�s�g�v�c��>��9���?Dpx�����[�Q?��h�Q\�c�k��3/q�F"�`Lg������y����
��Tn)��Yt�`b���#XHT�[��1�c�����C�q�g�+�!
��d�2N�X�(�k�BDF��Z�:����	��q�e(��x��{����#�l9��AW&��_�w�`F�7�e��M��5����[���N�,|15�x��4�Rd��@
�[��66��_�?=V;;;����_��/_���_�TC���:>|��Wa_E}�������}�L���4���������=8`���n�VkX���s�a�D��3�0�4�qFs�.�J�!3vR������a��o�9�}q�Qh`��!�+���i~#��f��zH��RpDZL��v�e��Z�V�}��T�n�5;��4��H
k7[4��	�������� v%v���K�;�H?H��P��(F8���@��*�}����
�����H��
��W��}l�d�^�2����.�����>������]�MJ�T��YSq���I�4(=|l�3�8@Syb,�����{���6zCI���Ek&Z�I�)��%h����S�����b��a;���iZtuC�5J�n��_�}����{��=0�~�A����@��:}��a���^�|F1���z�N�{�*w0y��zP��X������-~O�l~e��xxj���
�/J�U�U����0���(�!���S�����M�1���_�4F������\\4`nI����#�
E����4�����]���������$f���
��p�)���v��L��sK�������F+��UW�6�r,�x�n`�<#���v�m<���x����n��BgD����G����w%n����������*����k�[q=��	�\��G��y�7�N�3]��W���v��!U� n�d����_�
��������-�(M�B^#�����+>�5�k.���V��mo$���A����J������X=�lp���tSM�FM���Q	$��\��f�sf����*T��y���zCw��=�;�?��'��G��A���(�
Sx;���Ss �����^�;�I.�S�)��LMt8M�N����m���tjA�s��3��:q����Y]*�S�����_�^)U����X@M�.�~f;2n�r
���q4C�.��3��]?dN��Ao�n��L�
�����q��Y�ui?HX��[�-j����r���M�z��m��,����9��i�pE�V���_c�����'I��}��%��h_����T���_]��oQ9[��kb'��xHV<������������~��HAfAfAfAfA�+/%�:��B�*����%�,�,�,�,�|�f�G�W ��/�,�,�,�,�|���.��T��brS�8Q����7)Z�"��5��W�Y�Y�Y�y���Nkkkk����:�����AI��4��u�����}����*~�[�9	��1����P	M�9_���9eT�]y��z����by�:>_M��E��)f?R��U��7���]��)^v������\I��b����{��]S/��H���oG"v��6?$�7��Z����.t>��QUB
093�<i�b>�is���b��@��*�;��a��1�Le�0�WQ�"����YV��<
� 6[f�$����`u�zK�/e���Y�b���v(��Hc0_���,��G����,C�%�����8�������zly�aC]�i����B��|��f��%���;[np���(a�����	���~���N�viB+�Z��J��cc���z�z�(�}�O`�	c*�t�Y�?�e���4��J������M
��������4[Q�C�����o,���
�|c���$����j�\x���_v%�FN)rJ�S��R�xJ�j��ZT���y��S��>�|S�������?�xs���U?��mc�����%�z�\p���z��#�Oy���Bs��	?(U7��P�4�H�%�!��H��Y�#�.�����m�yL��� Ge�2�'i�X�A��A���
_-������H2�d"�D��S�5�:O��%�+�+�+�+��F�����Q]W>��! �B��>�P�8!3Y7�z;��<�Id-���qa�E�x$�l��|S;d�GK:�C4��&%qXT�z��"�"8Dp\���vQ�uuuu�u��vA^A^A^A^A��@�y���������������}
���������������Q�2$�+�[o��J"�9t����@r�;a�a~���x\!���.���U��
�������s@���Agj.�e�hX��6�=h�]�J	���������]�Mko�����l�l:�1B�c<��*S��"���OG�e.���TI��_�Q�����Q�	��Vp�����o<��$�X����4p5�����%
��4����w���xo�0����c��~#����H�����7660y4N�+Nk���v��Z���������9��T���r�}}�y��J�d�}��.�����Q�E�\��K��+m�L/�X��6�X���bbR_�Q���(�������-�V�FB���'�$�����n,[��[b�2����_6~i�O]}�����1�������q�3��g�&03F�����4+�U���hug��MWR��xX��yk+�4���	nQ���%����G��O.*wW�G��..��Z�Z�]����T���XV�_�@�	
�|t.����7���&PVOb.��F���UzW����1��~j�-n�C��t��"2!J���m�k�M��9�(�����j�3�"*�k������dzU�IO�+�L�rCdP�����>�����1;I=y�~~u�rp����hX�����r����9���G�UN+r�,�W~Uo��/q�=i_U��G�X�9/��e�\��;���e��}[������s�Z3���n��|4��f��vm�C�nN�������2[��tI!�����87X[��M3����0��,c�@`Y�-��tH�,x�DU*��	Kh�x��c���TvpY��O<�vok&e�	�}����N�0�b%jM��l�7|#T�Z����K��fv�����^��W-�U��}��,�UK���{.�9��_�,	�7�������zi0�����yv�%(KV��5/���
d
�\�|�gj�+����{��7��x�o�]`n.�~�c_���INSC���P�Oi�-�]���,���6���� ���m4p�%�O]U����:��-�)���Z,$Qub���/�n66<0~���[:�����!��;������73Ia���\Zd��{��]�\
"����t����a��I������u���
�����������J�[��'�G�&#t�*n��/~:r/pwS���r�%o���'c_p��3�J��T�]�~�G�/���������/*���?���^���?��m���������]�3�r]`���l��=2<_&�r4O��p�N��Q��>�B�-������l���qG�Ko��?�k�)�Z�����%��\�o�\���
���>�F���	�`������8��e;�
:�/����s�I���	N���'f�o����w�<�{
KN��~���{��x�i����h7��)>���t^�B��#���S}������*�E���:?�J��g��X}�T��K�����_����Bm�l�V���s���tU��v�s�����E���S�yG�:1&RX��'~�������A��5�s_y�C��m�P-�.�kF$�I�s�~�������$�ED�:�k��_B�%��*6J�>�l%7�f�g�/���]t��t�JP�������D����`4�����NnE���41q�w���|�c�0�i�IO0a��#���J���b�Q�p�d����at�t�����u`V���tG��6��!��W��J�dRB�X�3�-�:�d-k��r�����,��q�qW�.B��s�����Kt@B}��=��${��~(�Z{�.M�k�,��*��p*s�T����7e�,�	���������82w�w4����O����O0CD5t�2-b������g�Mt��X���T�3��4������ua��k<I�����}����]^����[�1�d�}Q9�N��P	_�����XA��������5V����X��������>��&�	8X+;s���	�D�e����L�������Cz�b�������(i4���"�X[��L�������q��n�����������&�HN�%)8���k
���#��L���4�&���;�AZ���-��/��sgz��b�{y|�:/���T�t��9��*j����B���X��=���}��0������.�ey���}��y�t�����w�A�p/0��ZR�����������t��t��X�b�Yv�J�P#���
�����m��p����k��t�J12�����8��z#Q�������7u�����;X��U�������I`�������r�a�}��M;�jW�E����0 ���1?��ZQ�x]�Z������1;�<9J�G���D�d��r�G��������Z!u��qX<�P��|-�1�"B�dqe�1��L��PIdP2t����*f��~U�E~��W�������]��7�f)�����Wq���a1up{B�XW��er����6XW���B�#��R��o�NO	��WB�W������R����X���S�����j��2��v��v/����]�����n,5(V
�������d�r�.4�9�
��=R+�3�b�x�p�����u��ZJ[����lArx�o��V���[��"o�)o��������h�5��~���xHF��
r�����M��8R�����HV���{�!����p&�����&�b�a�(�"�V�Z�tf��yx�����A3DKh�~��A9w}E��
*�3bbL��7zq��5��CP���1���+w���LpT�*��0�.���_OO�,45����������@;�o�^�Mg���(4�n���f$3[[|{�?���O8�My2�����l���VOS����m��:�:S�U��LjwW�
����<�s�\�����
�x���� 
���v���z
���PO�r��������M3����3�#{o�;������������/�T�v[|?�}�J)���m6�3�ir�U�������^}����������h����IK�0�d��U�����Z^y�����!���X�z��C��_��:�e�����'���iw�b�)�!�C������
����.���>=Y������r2���U���������4��:���K�Y��.S��:��Y7������n��/�����o���������w�:^q�}�����w�|w�._KKv�uL��c�y2�ZF}9jh���������@��6'��0b;{�0�Y�5�������J�8�'�DeAL��}��kq&eD:��A�E������L0�������n�����Z��&��ON��4��\V�������lr���S8p#��JZ�g��������-
1�E�����Z`���MM~�^*g��Io��@��z@�0,��	t�V��0v�b����h�3_�0h�?S� ��T>�8�%C�a�	]8.-��)�U�B�UM
k.3<��VE����)���������!���u�J�h@�iU�x$���rv�r��[�1�S���c��>��4�"�~�W��:�tw���u��&%�e�@�|l��$4����#�3J �6"�f����+��?�\������v���>�G������<���X��;��%����������D���Jl�u������~��]nAV������A�K�Y9L�a�v6����",X�8��5�uM��������h�rY�'�����B��why���w���wR{j��E�,3�x���S;F(�������Lp$�
�!��Pk��(m��X�����'~>�����w�[��_{{{{{���N��N5������2�}���y���O
�</����s4�c�2q:x��C���
�����J��ul��&��bj�I|��wb�SD��]1j�n��qah4�h]���vg`{�h��gQ�y@���\���m0��k���:z�:�)&���}K��D�9%�W�x���uL�
�����E��-x���������5���G*(
=��F=\�"1��R��8�2�bCS�@`r��t���g)��~<x�������#�s�&��Psk�q���3vw��]�l��C�u���5v�4�f/��/��.\�p�k�n���O~���
W�1\��ou��((���z���
����( ���a��>f�F���#E>�=# ��'��6�����>���yq~L��c�H�2"eD��Q��?���Je�C���"��Q�m3�����J.�L)l9��������)���NMdJ�;AFd��g������8N;���y^���q������WHDI�K6���������n.g��8�Q���i_c��n����.��hd�;��IC����
+��v���>�HP��"AE��]3�O��Q`��:�|e�H�&�y�����i�������c�
>���O:j��r$����N�g6��t�cn�s�(�Kp�1,&X ��.J3��">����E.b�S�����eNS3%%��P�[�������UIT%Q�DUUi��R�N�e3J�~v@��
}���k�H�F�F�F���	�w�L����,))))���q�6�%()()()()(��s�y�I�I�I�I��
&�L�IAIAIAIA��()����������R��y��&�����C�G�G�G������U������j!Q�+r9	��:��#��d��2�����
�
�
�~6���1�I0�Y����������,�z� �\��a')))���8�c�G�G�G�G��6>�>�O�/:��?Ox��v��f�����
�$_mX�ITS8�b\�,4��,�c�.�B\H������4�������2	��)NO<����D�||g]�������M*fO	*�H"D>�|��^��
d_��~�q�so�o���e�,�I��K@bE�Y#��(c�B��8�����71�vE+�Hf\4)�b;���dF��I�-a�����_'����O����K':j��(rP�J	�J����t����DPp!^�4�����Z^|ET�~�~�~���S�7:4��e��
;������Z:�����c]��XWg��g���>C��owB�_�_�_�_�}��y�2Z�����W�s�]]]]��������w6���2
�7��y�$F��fA�%��q���G�Tp�������)���8���qbU04eQ[�1g>��[
M
O���7L��BS����~��$�a�����A�}C��o�-���P��8�\e��X�����Y��m>"�D��t�&�m���9��m�������e�o����wZ6��?����>���diQ~� �Q�%�%R��".����3��4��
���2����'GX3(rM`����'H�&:H���)Cm?\�T�q=�����`�0,	�g�)-<�b^#j���=��iC4���(%J�W$�Rq�qP�"�B��>)5�:���_*"J�n��Z����Io�DOtZ�xp&�
5���������Q���"/��(wwg��g�kZ�Iwi�M	��Y����=!�;q^m�;�0o���q��Y������F�_��.��%`rxnhb�.���8��&PKr��1(~��Y}��7'���f�[]���o_�����S�n��X�P4,����+�M�����y
`��mn`����3���3S��T����OR�����z�VC����!������s����fI������U���������YP�`���I!%�N40��D
l�F���Q�w�i-8��xK��_j��]�U2GGG?Wm��a��?��k�K����,"�)�)�)��������hk���[�1xH(������A[����4���o��a���n��+�"�
��J���v��%�]p]p]p]p��u����
U�D}q.������V V V V V ���D!�����,��vx��~|��k)9��X� �*��HP�����FPF�
.t���$����`����P�����*(�������+S-�r��GZ��@��0�`R>��A}]m�z7<l�^���%�A��"/����6>��qI#�7��v���OMMkJ~�"\�����<�]�=����=V�_���ce/��,��i-S5,[�.z����1W�~k�k�S��e�6K
���n���O��Q]��OV�'�j������)������L��������Pj�bb�Y�
��c�1������a�"��#���y��?
KYw�qV�]���u�&�f�,,l ��S�_?���oCq�2g�_?�:����vG�0�j��.c6�VA�Rv~/a����\#��D�'�dr�E���H����3� RCh��������'.q�%��<5���^i���hda��
��DO��vi��L��sx�D[�h��:�<(xh��q���6�I�y]#z/�cZS��%�����o[L�	Ae�����a�0��2�09�A�y�>;N���UAIQ���V�������*�1�LCf���x������wh"��>}�l�|��������������_�?���������-\��1����l�a�:�Y�����\P��pn���������J�_>�<��������U�\;Z���O�Da!^��(��)��)�����\#:5�	�<Q@�\�
�i����������t����
�%�'�>J�'{Vk�H��@[���A�O�d6�e?�6`+�^���^�M�c����7
s�6D����x]�Gtbe|���NC��VF��***���$�����!�]�[R�>�����"�f���v�����%B���]}-=e	��Y�3�Wi}=]�i]�95T'��>Xy���P�
�[U�.
������45S5��"u#��V2V�����^��C�!@1�s�q[^��V��X\#q��H�PP����6*����e������:<e-�S�������L#X�,�Pi�������@���g��Eh�d��H�B��0�1�N��0��P�0��%�2����F���i����4��&���������)����+7��'uK}�(.[��D�D��b8������_����|�������E��n��~��V��^\6R�^�~Z����]��y�+.o��BO�9����P�t�^��>�X��M�eR|����GC"m�0��f:�#0),�ubN��[�*a��9I�?�����;����>������Fw�b�;!��@D	DT�(���cTv��h��*�&�3��!��r��.���G����������M�Dx��V��h������~���[LTA�(7e�X
��d���M�������p��C�<Y�� �3'��y�,�z��Pc�������uz��$r,�_2�F�,/��GeJ]����PV�	�y�8v���<�R��N���HaZ�UQ�x\�k�D�Ot��T�,�K����&�x��f%5�2��L>iM��i�s�f�O��������\����)p|�$��G��#��*�/C�����r����y��B[��;5�Y��o��`<�8��}i
P��%s;��
h����Lp�9�����:�W�������������2��" .YZl/��������u+����H�FX��4����8�A�J�X���b�2w�����0��NP
�x��Z�bX�)���M�'&��-|=M/|�|K��g�"�wgx!����<)�1�E��G�8�uZ0����dst1�.(�u���F��j$���S���z��{�+�(^6��G=a�D��[�Q�Q�Q��F#�\?��+u'i���1�IG��F�5��;s`,�$+8<�S��$���Q?�����i���+/���;���� �50����3p�xGc��\�Z�Z�Z��Fc�� ����2@�p�	


�l8t$1������O5�]'c9��):�6�/Z�� �?����
�>���a/��Rp��S�%D�g*�6�97�{����}P��MFd�-�8��L\[�6����o�"�|ot���Q{��G\.��.�I����UA[}�
-��+��Z>29��a��1�������js��a#�F���&l0��!'�b��1�I=Q��K[*��-z	'�-&��-���x�dF�m����E��$#�{�F|g�������6AB�&���Ly{��������:W\���e�������%��U{�f���tneW�+�N�$�$+|�|���5
��|�4�L���\��FP�u.�\g0<�3��7�>�����+�5�]����eF��������s�6F��LL�X��!)BR���($���D!�p�)$WB��ls���,����Q�V�Q{k<hm�KqP<��R�����u2�U���.�z�1�;��0�$��p�5*�a���]���7b�0���O��������
�#�z������U���St%��7q��"�����v�����=w}�t!U���7Y �.��V�~�����������%w.�qW�����,�_H��� W��a����5a��w�s$���p3���r��[�h]-`�D���9����5�+����.[��5F�6��O�RWR��bg�g����/2����:�+�Hy\�{~��;���_����v��V�z�;�����g��bKN�hmw��N��\yJM��y�)A:�����
���-�dH�C:Z�js7��;�
���f4�������������X�R�?q�����~�E��-f9�a{�- N��k^�20����d��3 lQ�=�����;g���N<�����G0�Ee���"��-�,�<1�d�.�����,<4Q��`�__�b�k�X#>Ek�(������@:s��"�������|�8�Rg��kP.�+�+������F�����S��(�Q��L��%��}j&*��S�2����3�RDSu�R�l9�>����x�6H��qg�:j�A_�����C�-�B���x�y��U	�����_�x�j`�/��)�Yz6�"Nn���z�y(�����E���x�-���-<��Bzr���Q��hLN����q��z3���3�`�7�=,R�v�?��%����3]��]	�s)W�"����>�$�?��[=A�������=���\$�������	\$F������'��\�c]��>���|(�����P����;��?��n�.
�h��g&)���f>3�.�&�vR��rU�����5��Z1�~���FH��lVM�FF[D7	85�����_�~_��y��������z�4�� ��K��
����y���uMF�����	�_��N�R��>��D��8��N�%����&�4Fi5�7���6��-=s6y�1H��w�A�
c��$���%l6m���n�0v�Mi�-��[�J���@
amM� T4�(��O�u�m�1vw<����n���d`s<���E�%98���=��vY+���(:�.�	��C���L�QK�S�c��������.s�q��D��5?��e��&����S�m~4b��&RR�P���/�[��9e^���`:�he������}���K��Ip��#��!��x�	�#��&879�����JO]��''���Y�z��p��*5"lI������VH��dqq��j�^D���Q�WR&�\��
�-���epa�g�����q����	����
��&��
��
.��	v#��?�k��IR�Q�s��L���~|�.��b��=�
�z��h+�?l��(����S_�~o5�Y�����H�E�75���)��Q�u�n�1'�u2Om���w2����r��T#-b��m�4H�_�����&�A3D���GL���!^�Bl�|o�%Wm�(P�qhgW8nd�
g���i ��I9i%?87�e��3�����h3�G������&�=b�)�53�QCg~��W
��������tz
��cY�.���������*N�I#���L��R��*$]P�����QO�K�I��c�#
M��,�k9&%:��,aD���D6F?��I������h�56������)��E���6.f^� ���|�o�Z�n(��d�:�}X���P�>`�yG�o��6�ccm��tH�������G ���h�aJ������=����d�q��?Ek[���k�R}�]@�t�����o��S�������0��G��I��^9DE��m��3��n��soy�Wt]t�Zq���a��-���E
���Q�AX���8��fh;�B�Wk���,R��ii�Q��k�?:�W���[��C�ji��\I�dw)J�e���Qt�����^\�����p��K�Z������8�����L��F�-D������7H
s������F�d���FJ1��K���I0�������{^�	��fMSj(~����6�^�4*�mp4�t���[���9?9}J�BJz
����?
R�0�����u+�3�+��<�X�� �]�<���l��k%��s`�����V�2o�o��x���l�-+�&�pu��	�����4��S�uQ��p�d����C��6G������8U����c���o���~�n4U�����w@��I�	p�)W��j:���5�%�z���c����>v�/KW��6�4i(����z�bc��+[����N��Qd�)�h�2���Y�kz�+�	��0�f<q��Je�c�����d�'c(J������������-�������-�qq��c\�W����?G�go~����/�F��H	�
�
�
�
�^e`����)U�s�s`[��7[=+������|U9����H����!���*���L��3�F��yu�4C'6�U�;*E�=�~NU	+�)b�()����u@.?4�CW�F������s
���`y���g��D�8�A�-�����d�E�i�3��Y*(
�H�������6Z�K8���2=�G':s��b#h�� !i����4�*m�k���tt�����yI�������lF����=��%%[���4��1)o2�6���z>�H�o
<���=I�XI�fp�}q{��7�oX`:m�6�=�������\��sOr�7�0'�q�V�{^p6�[%����Cu�A��T�_h�=u#6I���]������U��|�����4Po�XT���������9ziC��A�����Y��p.�	WOT�[M������C���LO�U����,Nz�d�e�i2��y�6�C+���=}l�A����S��G����y�R������c��US�`�FQ�j�J��X|s
���{Z�G���8���_��������/�d�����������G�h7�2��<�b�[�wBeep����������������j���:\�rW��$�.�M4�?��P{���h��u��F�)��]��3�����^A?���)&�B�;���:AWD?s����%�^>��1jF����#��C��`�����o�r��m��:����q A�C�2Kq+)�6���)V���K�G��E��b���W�<�A�A���]U#��Y��� ��C{��o��/�~��m�}8�vg�����)�QgkV?���!�y��uH�CbZ�u�3��Q_<�6��M04g�8itZ������&K�e���V����>�e2����fjMl��u%�}X7�O���q�h����B���Ub����8!I�*(�	&���RK9����,D-�H��H#�F"��L]q�P�w]H��
R�^������ppppp�2pv����/�T_�0[1�****__������`�H����� �������m����8=h>X���]����@�
w���|��;t ��`{����l)~=X�*�������:�
����^����ZM�K;��c�Z�4G�:���k��WO����R��@!8OE@Q�(��"@4����{�Le�0���z��T����n#y��*��H�b�^81�X�!�`�\ee��������;�Ox{l���t�x���D(��YP����Gc����Bw����5E�&�$k��-'g�"0E`~.�J���i�<�Eu1��r�K ���`�����������b�(�.u�l�E�9�)n�>V�*t8N��K�Y��[�-�DL�
�r�z�/Q���lJ�!��B�1|[�S
������'�	�a����~#�/���[������T�(���Q���� �9=&RF&I0Z��"�s&+�I������=�e�b����2��d��@����W�g���$�vES��1�\�y�a����;j����[��^"�<�G�'"G�C,(���U��g���g�hK�"����!�r�W��j.��h�j�Y�c�q�����.4TG��Ff��[�E�6a4�-##���R
�H��=)1��g]#�u���_v�t�e ��(>����s�.��6���]�+wX
��{�C���V�$������Dz���5�;�/�������!�H� <%�h�U8 ������:�8��������50�tV�{
$BD��"�".%���!l�s8x^���	����*�������������W/��1��E�������������~XF�!`+`+`+`+`{�*3���u:�*��H�	Q�M&3R����e[~����AD���vB���!?E:�A�v	+(�+r��+���mS/�7�Q���_���1$H�>�x��EeI���g��m�m���Gml9/
�Kr��A�
�s0�S�����Cf��^�����}�'~��q����<� m��Uu����?�Pz�zO!�"E�x� E(�g�W�������T�~��>����o���C��5$7�Q]�����gWq:l�]7�w}�����_��^�/G)��]�"c��"����>�Q]���.i�t�����	����s
�`�d��#��n�3�|����T?8j,�������h��y��p�����:��vQ��*"TD�����P�������{/^�^�z}T��������������W�����AL���-��4�Q���l]g�PC���F�
�Vy��`R�K��5���O
q
�#x����h�������\W�{N����x?ta�������|�V���x��e�Z9D���!#B�S2���r.������L��i��r����7��*��_sr��g2s�y�D5\/�G8�� A �@��+�+�w��cS0��aJ�3�eD#)���@;�h�4� ������~�\��IY��/��@X��q�*�S�`�i<�I�T���AQX9��$I$�H$��I��n1���/z�pv$�w�S�;aT?��G��H�
"D*|�RawN*l:��~���	�����_��['���l��7����n��k���s��'�UAi��n�.[��-Q�������m0��csT�qz����qd����v�K����`��Y3��$Q�D���K�����:�f*2�nP>��}L��,;�z�z�z��+��E��20M��;H���~��$f�����(��H�� � � � ���
�
���0��A������x�cW�(P�o�U��"Um��Z��UR��h��&��
�����7\��?�P������}�$n�'"OD��<y�~|M�������?Gi!J�����������[:xW�����������E������c�2.������}������.����h]�_����;�Y=�2|%]sU���6�h���C��t��|V���T��:ABm��B��T���pb��^��{�m_������#RG��H�:W!u�h���e����,�,�,�,�,�|%���m��Q�F����h�q{\�~����JSPPPPP����z��*DV��B��Qn&>��s���sO���w+-�M=���e
�����v1{�y�y�y�y������@����;��nY���\�1��8������HB��EHr�77D��i�0\]
D������N#dO�"�� "@D��r��DD�����D����������j3���r�{b������J��w�6��$�Q�W�W�W�wm�2q��X�+````�`1�rA�]�/R��FL	�����b$PPPP�b#�76�`��@��1"+.��K�v�����f�,h�
	�����_�����1��K��w>�����TGAcAcAcA��r�e&+���d���rP�������������j��w����f�v����@]��o��3mU��&
m�J���������g:����������7S��s"2Dd���!W&CZ2�����w��67AZ����Q8��fr
����������13���$�`O��:��G|�)7���Y��a��Z
5r_ac:��D124&�g�D	<����I�����/ N0k�x���B������/,V�`���2��U��d���5"jD��LQ8���-������k/
����j�~n���o�j������� ���N��MB80�-������i9qG���u���8��0	J���
������������L)�����
�$�����4��)T��$P���l0��-Q��������xA��'-Vv��z��"ZD��h���������������u��ica8
��)0,0,0,0,0�FnYH�bM$�p������mF�����l��,�,�,�,��^x��UzI�W-g�������������\��Q������(�m1HM�{�;�*	E7pppp^+8o����f9�y�L����	����n]9�N������!�T��P[�AA�LY��B?����L��$�2������~��!#������������^�*���*g^�����u4����Eh���!Bc�Bc#4i�eG���V��o|����oK��j���z�d��i�y)���8�$Gj
�"�_#J=
��O-&�p�n��!"D�1�W�v���#�)h�����/���_/�HU��x��bV���0���6�(��I��GH�.R@��H�������t�w60a|[�0B�k��\�1�� � � � ��Gj�����p{�%�� � � � ��G�3��X7����j��sg�Vq0�q-��
�Jt`eR�~/���������;�Hk~`?	R��W���E�?�rm�H��Fi�����J���&+�I������=�e�b��(@�7����D��&K���3"gD���Y��Q�E8&���4��0�wO��0�c8�rhu�s��D��(U�`�W}o�������766:9�O�-D��y��s=`�A������������T�B�19)N�"��~�'�3�nnnnn/��R.B���'H+H+H+H+H{�%�lK�}.f��]i��ed%$�����m�a��
�{�@�G
D�z�r*=r��C��
q�B��mPxTmv�����P3��_aj����q�����������n�H�>_�[m�f\�+L��y��\�!���o4��	p��L0oi���R$&���O�15���Q:��b�
�WU ���b(��(���G1D}���{u��L�������P�"Mt)
>���������p9�������ETTT����������/0+0+0+0{Cav���5-�G������\))��Q�{�
�#r���Q����e6��������I3W{����'
�
"D(�P�V�4��y�'
����v�8Fu�>�?�o���D{�&P��9S���f:�G1���l1��-��rY���u>0q��f���bCD��"2n���p��
�wg����?�f�
�
�
�
���V�z����.���V��3W��&mc��1����3l�'���������������ux��c
��+}���x*����1����^����b�{��_�Wi9�?U?��oR~J�]�]�]�}�����M��j���^'3�@6����p�#|��\�^��mTDDDD��������S5
���v���������8
�c���4����%�pA[A[A����N��X	\\\\o$��{�������i��5���~>��tZ�Xc���I�6��f��������2��������������WD���������	�����X���dc~Vw�b_��UV�#��@�jhL�j���P���L���'7^����X�a�i�������U�2
	����I���a�$:���V7�������=ku^��WN
qy"��E�\�\�\���C9�����3]�W�DE:4y���t'Z��^������Y�?��=AhAhAhA���\�E=Q����za`��&����,��Ea�#6:�R`Q`Q`Q`���b��V����������E��m���&�����������I�"�����^TJ����i���������:I��$$���:|������~��������b-||||_�-�n��^�U��S�
�
�
��/=��-�k������#!,,,�S3����� �y��7A^A^A^A^A�u����P+0+0+0+0{Cai.��rE���N���V`���&
&
&�\L�/�Q�6���V.������o^��K����T+GZM�$���|s1Q���;���U����q1�g���<���c��a����f������{25�v���j��CVy���F|hG�W��Lu����c@|�G���� A$�H�g+�� ���7�H�H�H���#�qi�H)��G�G����+��������Z�����������>����]6����w�3���9�/5uE�������%RK��H��(��m����'��L
{7�qj"��''�>	

Bs���D�w����{��$���u���������r�#l����z���|������a���?�-��2)����n�V\�G����7���w����ME���J���uh�"N�U�Q� ��2��E�%�-�o�B8��>l]c�MN�����fy|2.T/�R�/�����������z��tb��N������� C�	�l=��4�eo�b��<
���ip����g�/��<�;>x���i���0Ou�5YO�?t~���mM	��*������i���V�2��3��Y���^�3�����Il� ��)�Z�LS���LC�uQ�-tj�a��W�!s���2tr��j�V\_�p��$�R�z��e�`��G�#�������gA��J����\�
	�0�IzwM�,n_�%c�&v�(���=�?t]�:����Y�2���i9A6���������f�E����a~���V���?�d��lANH~���������):
���U�� �9l� UVc=��Gw���z���l�������a��gS�k���`�
�������M�S�������f"�&}������
�����w�|����C�6��|��IP� ,LNd�!5�v�b<���yB^n������@\�)��p�����G��6:�s����+u3���6�
T��6��4��T�T�bNa8y������ �<	���������o��tFy���M�"�L��=��[��{�
���d��q��-�<�����s�����;����O���������#�p�����}�C:)���}u������>���#�������X�}�6�o����xp[���D	����De:�G�F�:G�H�jH�����b4E@����n_��>���uZ5v-��D�~�����+gj9S���F����}g�����{��o^P5������P4�YRn�@[\Qf����Ymn��1��@�@�@�@�����U.rSSSS���2����)���h)(((((((xQ������hhhh�y���3��#5\Hr�;���0z����D�i��d������xn��3U�"H��:���K��93h���):�������fI�5�5�5���k�J��Q����&�^�����#]kp���W��|���{x����������$����Ta	�)%RJ��H�KH��+&@UUU��������v������w.�������#C��w�]���5��c���?j�)������V��j�|����_p�7�6�;�e�o�	:M����W�[
vP�!����|x�g>�[���p��{t.w��[�T������hm�:��Gy"����kN%i�>��)��r�0��l�DS��&C��;��u_����O5�}�<�O�i��k�5�t�R�1��;�O���d�J�-�67��BY��
�c�V��n����Q*2%�@�E�,��XI���t'�q+j��S!5���=��������X�l''''o"Nz��E2� :�������c�� ��z��N�xxxx���R��tY���p��\PPP� ����+��������'�s��n��7��EOPP�S@�`%�����/��������n��*�&�&��i�Z��� *��������n�������n�U�=K�\Z9��+*r����L��qYq�QVk�����b�K�En��o��7���_5�M�*��fqCe
f�*�"�Zo�Z��{�u��0Ch��
��8��-?�� If���6>I�Q��c�F:���>w�9sg�$d���T_vR��u������H'E�WS�������L��qY9v��Agu
��`�q��Da��jt��}��R��;�m0��o�����c)JW��_��UI<��6���{Wo���*~p�y�nQ��[n�������T��U������'�k��[��
uQ������'�x��Zq��h���zj�&�m�fb30C�����[��j�rbua��]��}|	4�XRX��I"��
V>,�>^7)W>��-v�j�7]:V��~�2Q�Z���3^f�R�7>��j����j�����z�(���������k�����V���^E4����.�)�R���$q8Sy�h�/6Lq���p��/�gn��|���o��7����.����� ��� E�N[�g�e���D���'�)��������`��p�S���a�\[����0G��
Jl��ib,c���X�����Q��>��V����l�h��),�LS�5"|RDK�����e��B��et;��z��"�����(u-�M_9���t7��O�P3��l����Y�HD��^������Y����������4���2�������?l���2�eO_�y�B�]�+�B��%�[rwks�4@=����iq�wV>�����w��v*a��<�6�Q;
�?tn�t3���0��"�D8�p�$��Z������N��'�����VI1leK}��@�@�@�@�@�:��]	X?t����Y;p0000��`��,j�;;;jX��o^�u��U�H���vH��������'�������^B��e���|��{-E	�8/�,�,���b���S���)���x�����?\nx�:�	��!��n�@�@�@�@�g���PE�s4iAAA���]��/��	J
J
J~�(9����������`�`�`�g���0Mm��`���Z�\v������u@�7����7�������M��R�g6?Z��s��@\AXAXAXA�����O�uW�X�X�X��j��s��X#^��^����y4���]0��@�@�@�@�M���bb�e�M����������Zk���^F��V V V �S���+�H�VPV�d�V�p��L�������=�<��xRN�W�K�f���IOT����R��|���#��ENIhc�����n�/�&��Q����]�kN��;b�R$�N����o>�)�d�B��:�t���I�g�\�sj�ov�F��"S<��3_�����T��W%:��7��R�M�(���"�u���w����`�a����W����o�������O�$�����F�����o�7��%�E&�$�!�a�{:Z#�t���:p^>�:�A�?Sv������w���Ts�=��[�\��:�3����}|���Dt���!�h����)�PV�AF�F��_�R�&��y��1�Q�#03��
F��|3�n����9�L]�u_�R{�_�o�{���i����I���x����5'�r����P1�Y�L���3�W��SO����5;w��*�v1�E������j�d����$e�h���v�f�/iL�^�0f��cj��:�2�Uie��\Mb��]t���X�������������M������������qG��,�2$�w���6��p�
3
mFz������fX{�:y�H
|�i~�����2,��(���Q�ssi�b��t�S��n�����a%��� �R�,t��j��x�����=�����^-���[[B�#RA��H�
k�
>�C0V�(���c�����



��76tbNz������VH.,(���b���K��T�iek9��t�nm�N����'��'jN�W\�I�n�{�{�{��u�}����m�_�_�_�_��5�����:�����Lb�^#
S�I�Qt&����l��J9�2�
�������F8���*?Ub4��!@,@,@,@,@�N �|��;��������eCf��n�rd�N�^��>��J���F��4mDiAeAeAeA������ouG�0M�r��t!����Cu�Q�IOt�� ���`hJ����4����
�?1g���������BK�<1|�&-�����n�u��+p'p'p'pw������D�ilu_y���7��8$�	���^c����y����Q���I��.�=�@'0��.�!s�������z�b�3�d
d
d
d
d�C&����wm��.*no8�[���L0�Gq���9x��x��� ���Z1l6 �{�y� ������w��~U���e�:��@O��8'��U����<����~����~�Fr��:�:k�b�x�p�o��OB��B����/�<��N�������'g�������R��{[����.s�.
���L5qGo~<�?����TO��L��m�z/�&`�|��s�;���������I�]��j��F�;_�1=U=��6���?x���R��i���,�}|��Cmf�����^��k��+��$Qzm���|�P�E��]1F?�PH�+[#C�w�'V�|q�~�{���I�byP����
}���N
:k*:F�.MFeS��n��%hL�[�oE���
�[��.Gq��@�Y�Y�Y�Y�y�������W	����vGT�cj\-3Rq��C�`1>b���:�g���5yA$�zGQ�ng��]��2��p���6y��8=�����W��j��
R&�?)���"78l��w��z���!�������4��:���]�7���4�0�I���Eskw�
g!j�Q��,�eEY��RD��,Y"�n��"S:@������PPPPP���Bp��	���_.����B{�����{�y`���J����j�������Iyh�c'
�L�c�|M�����>�%��"BD��rE��l#�O�[F�cb������_��]�V�o�U�����
K�#��W}����j��%��A��������Pw$�Ud��V��.[�P�b*�2���R�@�Z�Z�Z�z��}��PL	�X��,Q�K�W�f��R[]���kb-���N������*�]riE,�X� b�ZT�UK���XE�Q�q���(?���y�	�_8��c�����������j�W��%Th�.�,�,�,�,��F|��������@�@�@�@�@��Cs�����c32c���L���0�%�:"�R�j�j�j�j���������_E�����^#F������]�����M���f�P��/|��H$*�d���_��3|=��0��z5�s[Aa�XC�0�x�mK�'�%"G�[55y1�!)�/���eR;{U�w?�%LO���L6�e5��:w����/�C�;{0W%�ap^���{��N��������L�s����M��0,kB]#fI_R����!rJ���Z*0
���W���YpoN8��`T�l�\��S7v��h����9�j��I�|C��j4�(|3�I�"q'76�a��Zmoo�}1�@�k��9���k\CF?�I�AGn
�s��f�/��6%~.��u�"��/-Cmq8*�*W�b:���g� ��A��k^�w�:���/3�a���R�T��i������f�T�IP�kp�Y������W�-��5��8jX�xk��{A&����0��'�b����qcH��0$U�:�oZ}��9�<Z:;J�X0���4M���������m#}���)�}��^�n������M���M�����{���H�bC*ZQ����<H�,���?W��$� ���`0cS�R��R��w��wXr��0�BG�T2K�hVw���#��d���,�g"!6�w��d�F�4w�5,s��=���WSj��lD�u8�'�+|cj�������3��^��-�%k� ��i�������G���x�M�U�����6��F���tj_[
������-c�[yI[p�V��A������yF]�/^�����>�E�7��w2O����c���;9<�������H�Fb����+���Sx
�;j����F:t�����A���;���������E<viM�spi}s����u�?����F
�_����"������}�K����5"F�.>|1#���r���Th���}�r���RNa�ZS��\Zy`�H%�8c�L$�`D��[WI�l'���c������W���d6�g�CO!���;:��6�q=jD44V0����`Y1fE��!�8�;�G5�����_�7�~����y���3BFE�����,�dau��:=�6� ����6~mGy:�$�8.��)'�of���'���I�n$�����|K��J}�j����r�iD���jgo�����8g#^S��k��@�"�0��["����m������� �l��S���K�YF���j��s�H��H�s4T	��0��D���������{H����=�K��JK7�y��{���M��yT����_;}l�<OU����(����N���*�@�,�xH�������!h��d�9�����h����rV����\�	��Vg�gt�IF#2�]���?5�N$��=�]�����{���%�;��aq�����!��d+���b�G����#�y��V�Q�GH
r�qij���`��O���8.�x��?����E�,�n(�vU�`���YB�r���q4����AP����
h=���i��+
����Z����q�_�*�>j�����.���cKn�S:M�N�O�=����O��COH0Y���Ao|���z^QM$���;�H��s;�9{��>V	O5tK�������:Wi��=����9��J�������	���"X�`m��	��
Z�~����:�z�9����
X��2�}HT������J��p����������9�0��Is���w������H�wlV���1
F��`4�yFw���Q��|�����6��/��{���V6"����<���I���p��g)��<��]-�mxF�l�������LI������*y�.k��t��0�<|����W'���������O�t""""f��A���.��y�lu��|�UcI�y���OHD�k`�/H(�G�cl�����
��+���Nz���>��o�V[�����'��(M�g��Ba�mjG�]!���s���.�!X_����������f���T~��aXxw�%Scg�g�)�����t�#aMR�^���A*���v���'b�`o��NB4w����O��������g��,��S�O8��d�t���l?T���f{�7]����j�z���}��a�I��@�|m�u��"��b9��2� ����t�>�l\��(�����*����������>����f����s���jG����'�����n��O�����g�6y{���9��L�_h� :����_&����t������o��\��z���W�M�'Q��2i��x���:�[j�xI�q�Y�ml��We�nX��`pYH
��=@����^�oN��x����_<?}u���)\U��zFz��$����p8��
Bc�F��b8G��r������^M]l�.�9�C?X�J*��	���uL�]��8�Z����,����s-.�F3��'���������wtYi1���+�H�^���R���3/09�
1q����5��:��CB�	�������Q�.�Gq����,"9�8������d7%��Y.HOT������� �U��l$R����,��T���|����3�r����\D ����RY�nW��M��u�/���WoN��8z������K������������\-��g�� �a���!jh�O>&��E���0gkl����i�-��<�]w1�2sF:>�]����&��`�f���rB,����pn
)��_����?����`K�j���3P[cJGOS�*�6����/��Y�z�v8��_L<������z:8��}1�>'�����]V��'����z{zxR�:���8���9��\n�x������/�{����T��4�-��{���)p
��m�6���
�����D��K
`)X
���`i�������m����ZC�����Zq��_W*����������=Z��:��������p��U�	9��6��w��u��y���Vl�*_�Th8����
��}���L[O���+3333��	��|9��&����j0�[ly�������������_��������E�)}S�\7��\
���7ad\w<�;n�e_�:|�Lu����okNQ�n
E
/��yim������%~���J�[x�yG=��K����}�������q���)(n���6�T����CL19��p�E���.h��z��7�X��h�:l�3��05/
�3\�
acu�RrN����Q"�q���U�Q�z����u����u.3�� �kX�vGI���5�����S��s��(9����k�J��b3_uOJ��{��.3�E��*�s5��H��U1~�'���4�9?�9��T��yUR���*�D
�.J�	"�I����nw���n��K*��MJgR��K*�)�>�����%*�P"�:���T���=�[�����W�p�
���Z!�*��u���R����(.�K����_����m���~fe�B�j}�t���t���Z=�P B��y���Q]S�<\h��_��/�S��� 
���( :���{JZy6�����Y�j�2�\��j�s�8�����I,@��R::���3�:���
��~��w�W)�k�y�R	���*��Nh��GWo����/����������n�|PA�_. ������,��kM���%�.T��+�
����W�����3���k��a��E���P\�/���o�8��5*��mi���]#��������a�8'<�[�����G�Xy��x������=_mO��V���A����������+�A!�wr
��t����l-�J��\��pU��M��B:-�h���@2��:��Ah��U��3n���0(
���B�����_��oM�JN.���_����{����00���&0|����2�,����a��-o�� 1H�����o�������-x{�]����z[;���Q��QO�@4
D?0D��- 5H
R������V�4�A�J�Of�8������E[?�����08������"
��1p��V��D`��g�x�o�7�_,��2�,6�C�]p�w$w�D~8z��������-������q�?6�L1����Ai\.&3J�������r������2���u�����������/]vb�n��,l6(��1�;,�X�%K��<]AkO�`����V�
{��;�ae�X�X�X�X��X��Zx�Jn������Y�/��~B$7��,�.+���-�6��`��cs���iOL��'�<�~fU~������4��R$���D�9��OH�'
��%���>QZ��Ln�*>+�_��j�LO��|��n���x0��o���a!e�9q��t��k�������D�D���2"J��<�N
f�����}t 3��i
�,��P���N�F�"-b��w�~�ju�ah���~���qX�!	 	 	 	6+	:i�ea����$����� /��Y�6tp�z]9�Ac�4�7N����PL��2OD��3�L{��_ ��1��L��A��y���U�� ���M�,����QCD������X�:�I��/7��3���"�j���;�]	�����iiii�Qi��@�-`����-k���B�w�]p����n
&��b�Z�A�-�
���� ��BMd!&YK:S����%H���o�{���t�h2�y�j;c���}79c.�����1������=�����p�y�r���C�.�����
[�=����D�"cc�5
��L�L
��B^#������6�
n?hn���wk�v�]`�}������g�Yp��g��v��o!��-��G2~G;B<��t$���:`(-��J������w3����d���s����R��jZ�/y��?��L[3gK#�����*���"22��\]\j�K��)5h��j��B���|W~�ce�E�o�(�Qf����R�s�:�x%�z�[m�\Hm�1o����:[S��QJ������F����
Q�������cs�pw=�VsmjH����,�4��mw���\�����X>��(�����s
��F�e�B�t8�6\�|�za�������c�u�Va�N�z��Y��&���v���?w�����/�G���4����%��#�kJ�3�2�n�Q9�a�f�W�C�\�"M��$����
���m�����1�"����Y&���Go�bS����iF�
��	m��N�v'uW��N�tCU��k��4�����`��X�e��C�m
��������^M�U$v������mj7�j��@��V�P��gC<��`�D�)'6X��C��|����4(#=A�NI�R��J�'��}��%�����p����N����D}�g�����:]8y����v����f[����V���������t��,�gv�h�b������������N
�====E�X�� c���9��i��e�l�3��-���q>/<�����U��R�������D�������`k`��6��2�[��~�_�w�v�e�����x��Fw��g[d�����|��\1@������3�\�I
N�������Y3F&�����x�����P�a%����BGc��V�����3����=^]����n�v[��UR;$h1e�=�;P��AAw.�X���3��T��������h���h��v
"��D;=�����#V:��uk���~K�Y;g3����'t�7�?�_����/���:��2222os��}��r�D���m�W9{W
����Nr�\�n��ub�������q5�w�|�?O�w:_�{�[����)vx%�;�#��u�f������SQ@|���X8.����/{�V�>��w��������d�7k���>�]����� ���b����1o?s9j��[�z_�>8����������/a9�g�p��������������[QZf�Dq~De�}���B}0�}�ke���|���r�����t�n��w?��{�������a������UF���z���,r1�2-a���j����?lU�r�������fG%�����jU���e#�BF��Ip���(�d�1v8�P$j/��Q����L,)� |n+���m��$���������x8r��K���rS�w���.��>4�~h�f-LJ�1k�.���������*q����%�qJ+��f�VV����T������b y��;��l���A�60�J|Q[=���� 5u�Q��@��Vwb��t���~��De\YSD�L��x@m�VP����C�����1�d&t{���1�rsAR9������@�n����\rM4��ef|�Xn����:�]W�t�������j7Z<��Dj���������D�S�4�� S�V1�&���$�.����r�Rr�����f���o�{�Z���M�^��Q�8�l�t?s�[{Z��|���M��Z�����K��I�L���(7��v��q!�����6��4rd��������
7������p�a`��{v{%�xx����yc�{�~�dtupQ�9���W�<�J�'��k�Md�1��T}��`��nZK9���3������j�z[��h�N�Y����������0��7����C� N��|�c6����4�mD^��*��6�-�x��Hn�v��kE��u��.����#�|�T�����USH�����7����C{���x"���.og�E�2]��at�"Js����7��I���3�<����D�&c���y`nD�C�|��l%O,���1=�
��^ ��p4b�tdD��I��%������r�.8�6*��n:Q9��d~����j����{?J������4�#u���D�V�WM.�L�6V1����j7�FbL�/7����CS3�������*)�A�~5�9�4
�c�Z�����w���6�M���kN����Fk����0������}���11���Q��tzn��\2ine��_��O�C��������n�������;hs�[2���J*���@��8h��u�C�����H�Ux��������i_��$��+���[���!�u��I�,?����G�W��'�,9x\�0'���p�qMEa��/g#�g������Z�;�1$����G}��������1jN�hVn?�����l������G�I���\��s�&[�J��A�]��*wR�`���+f���nn��{����ya���Tj�u=������w��8
N���48��%�fV������!O������/-�}�%\���9��h0������9f�g;_<����g�q)��&�k�6�8�c����<|����W'���������O�t��`�v�}�`��^���ra��sxM�W����.�GE{�����>�a�>T�
p���V;�gcFjv��;��*aiiiii�A5�����)1����#%�v���[)�T� ���h z����go]���#��_�W�|��|���v]����0 V��{��s�g�����v��6��
��w�n�A�d0L��/���-l;�� ��1`o�m����$u,b�l�, u>���M�6�
n����f����/� +'��d�GZ���,�9>���*���[���������/O�O�li�q@��@��#f��t#bp3Ev��B�l��h����Wy����L�U!�9;5.q����1���:]�g��QO?*��:p��5`
X�WX��h� ��WK@����4M@��C��I��H��v�Z�K�*���)��|��p�ww]��-�`*�
���`�:��W!��f��_�5@
P������K�o:�^�SpG������������{��K_���=AO��^��2<�V�x"
s�^oxio#��&��.F�7���Kx	�7�C~}�/�A�@:�����^��[��u�<S-+�}���������_�_�O������f�7aG���z��m�jl�!�l��� ���t{����+;�����fH����K����Zi-Km�y�M-��^avdI��h
��� )HzOIZ��M�ny7�t�^������+i�e�pm<��D�����/M��"��, ���?��I[�'V7qYpn*!39�����Ul_h�+$��n�����t��q@|31�V�$�����p&XV�������ZR����2�V�l���S��&�^��s�\0�]�������ZF�Q`F��������DI-reD��Gy,�y�-�~��w/�
d��3-�!{�Q�kW�t����I�9������/�]�yi��O�"�5B����=h�i�Is-�w��'���@W�t]�9]��W����� .�����B��G��PL��2OD���3�L�~��
m=.S�_���4�����u$�_�{���i�yI�������Ui]�eu�o��P�����u���E�}�IX=`����������FgP������@�1AL��!&������v��54�����?CSw�]p�s�����!,v�Q"�i�V��<�����j��,
^���Z�
`�6��I`w:Y4��<YgH���^����v
�M:�D��� ��%ot������b�,�oe?t�r��$VQ�n4^PT�A�K�[^G�V�����u]+d<�3��0;3��d��6�	��n���~�Z����,8�9�n��l�����F�v�m�d��Nv�x�?��H��[���K(-C�x{]��Q���u:�f�L��l�/d�Ku����Y�N�[6nUm(I�rUH�
U\N\V��F�Xr��"�������#mP�f"Q>N=o���2-�Z�������pMt�D�i�aUCm�%����y�J�~�t��.�c-���Q2��ImO8���4���f��;G��������T�FPWF"�3����@�l����=����9��J�����MM&�� �[��t�����0������������?B������q{���z.��(�3y�lk���$��*q�H���`����,!a�"-��W�K�aJ��I�k�EY�q"�dT��H�!	G|t�Vc)�{jo�l
-�����7obk*�E+���c��$���~#c�'�l�.'k�I<y�~�t
?�X%e��D7�"V`�Z+�u�I�Rnl�>^>IjWjv�[������WG�4�i[;��	/����)?X��:����nS����	���
9�{ ��,��G�0no+�A�������U�z��qH*���\����D��y"E�}����:2�G]w�J�=�=�=�=��
wp����pQ���k���-�!�p�����p[��l�=l-i�I��s���Z����x<��T��L��
|?(���C������ 3�2���&3o�
��b_�>8������������'��������`,{��{H��L��4���oN��x����_<?}u"=�`�����L����������-PHS�F��k��B�y��� ,�~&�e�v�F�����[��/��~F�]��~�RmMQ�����-`���L;6
���>
���Y����P/������sU���EgJ%B��<���R#�����B���*gw����9���s�?��UY&3�����o8�B��\$;�^.U��9���]��v���.���}�\%��W����p����k��ce�k��]���~n�������L����mk
~��y�]{����K�=�U�[��|��.35������l�B�2s�����[��2�-\��z.��@��{�N�T��[�%*��ns�Vs�V����������7���-q����m�������z����;�m}4��HDQ?7�~�� :���c�t� /�_�p(����:'��������g/����WN���!h8��=�m���(���<����kJ�[�	_�N��\�����B.���,������Y���p�k
�LO4�X	B�A�A�A�A�m�>B��X$�=-d�W������2{���x'���h/�m(�c��n�G�z�������y��w3�0�O�o��oH��a@�7�����\��{�^���+��������w��;�@1P���h�Oyo��b���D��=���\���a�|��g��.��������g���+~����/,��������P|{iD ��p ��Ul�d
e0x���pT�AePT�c*EyQw��d H��
�>���8>z����#�.1��(sU��,�C���4
@��t\�HGa?`7��'k�����t�Ag�y�t��C�A]P�uA�
R��|���C����a�^s��H���0� �6<���J�
��ts����W�Q�M��/�C������o���_��2{�����>o��^�Ev�j�U���<k��0B�<����?��V���xl=����(����n}6�8��������=��i��(������[����*_�cG���A����ke8I9`/u1�}�Z7�U���a��,�����D9�E^��������Ic.��M�'�
��B���5nj��`6��m�=?��M����z����ur���,"��������y�m�?x�����H����+���\����$-lK_��>��a�}�1�W�^|��N�&��>��9V?�s9�����.pW�U��6?dU��l��w\:��>c{��N#���]����_��=n���v�������o���[2�&}�����"��>i���Zu#GwF1�#~��>&
�tl?g*G����@���T�������gf������j
�A���\���o����p���j���<�$�LV�&)B�XL>0�����N>\�lK�J�w"��z�
�NS����KN���S�V����Dn���@)/gX�����[���*Y���g�������Z-��T��f��������Ds�'B��?}wWAU@TDQ?��.m���l�t
u�+�������������i���7���������AiP��l(��;�%��*�����_�W�|�|��	F�t]@��l���U��Y�Sy��2����`/�{3������@-P�����C`XV�`�	X����Gp\W�Up�J!��v�s��������������{�����m��L���|?�_�U�~���w�������v`6�
f��`�ta��Qc[�x��a���_�Y��T{~��:m����g��`2�&����O�Bf���_�hZ�V��huV����XV�U`X�!V�t�$������+�
�^�!�d������+�
���M��q�b�X ��b}���W�+�
��@+�z��������	�Z��j��O>4(
������B��������I$��u<������������vVI�z�{[� O��Q��C�����^g"�[�wz��,tI<���GY���d�^W�'c5��\��k��M�����~�W��?:�����tm��B*BBBB�[i�����z�����p���/J���\����!AR���'��D�M�9�e�<�eV��@uPT�4�W��;��0�$KX
���`��f�5���`'�	v��7����@)P
�n&��
���������3���m���������^��
��`0�a3��3����*�
��������� *�
��� ��D����|��*�l��2��{d"v�U@PT��PmR��r@H��^
��@(zBm,���X���}��WCtx�� *�
�^� ���W��Hc�lc���\�~�@�t�
�"�pf��z
9)���
*�w�Ad|��2��%!d�<���0	t�e�M3P�n6�nxq\_����7�R���h���+��X�Y|���3h����=iB q�#�=����:����7#����1 ���h������#�X��th�%�<�����@8���f�$����4��uq�$04��3.����4�2��U.�4�qD�'�<�:����e���*��bC��bd���sY��3i�f	��._�w����
����:��_^0�._���,�[v9��s����������Mz����4�z���}VfQ!x�*��������3X�&��sqe��TQC������?B����^A������f���X��
��x������0i����I+�z�����St�������lJ�;�Xv�F�!���������u������l�y�,
�d9nH�R��� T��3~���������/(�p�$��,�x�Zi�m��m�}#�d"Wi���H.�5bBc|�qU4?���2��F��<.i�����pY�{�y����1Q#���U���-w����x��Z����:����T�&s��iw@������oH�h
l>4;�Gi�r<y�
6 ��`�w6 oXi����5`�&a���L�vs�&���>����A����88�`9��~�sy��f��8�f�`��M�y�M�`��M�W��	���1xo<��[9\�6�r

@�4}�:;�����j����V��Q O�h��f��.���C�2�.�����rya�P!u����w
Z&���*�%qFAk���A�[�u{
��v�c5�u��@��Ag�t�:76���` ���
"�������1Pf$I��y���3q�u����F���?%R#�iC#��Qj<)���j��/�&H�c����Y������x�����\������M�@O2�+I����'��4��,F�D���(%���c�����z���4�%�6QR��&�d&|H#�s��I�~q���hj�����'�p��CY	�>�������������;k�6QaJ���
 ��0 |�6�D�@0|'�BF����*��6���/��A�����p�R��"R���u�M�E6Lm�<J�c�8�7�c�����?V^%O��`1X��`�fY�E7���z$��[Y'�y��U��7���#T�AePy���V�LT�r�8f�h������bG���>,��2�,�%�����<��
F��`4�Q�F���
�}A_�w��%�����,�(��"y�yw�(:�BGci�����E�[d�Z�������0��X���e�u_���?9�W�������\��sy�O��� ;��Q������,�����/��n�����-�7T����Xg��_�3��\G����
Q��MbUROT�p�$���O|����X�f��#��	h��b��q����uQ#����\G�}q�P��I�����7=@����.I�8������Y/�jS�+�s�C��^��4�s�����B]o���Y@~�b_��w��~���X���`�g���J�N'�'�������~[��s�e��<{��4��M�����Ag�t�A��g\��qYpS�0L��I4�������;���`+�
�� 5�����x_P�
�oOq!>�e�2`���-`{���������0��O�iT�Qve�n|��]`�v?�^����b��4�E?�f�Y`�fo���*�������*�
�~�Pun�>T����`*�
�^���
�����d�]����� -H�^��a�Y�+p
�����8����lu��8�L��������4M�xy����H��]����r8M�ZU!���DZ�Z����k���H%3���F�����������HR#�e��lw��������0k�7MtOp��'�EY���Y��N�G���<8��oF���qj���d�ug�����Z��H�zx6��7�?{~��7f�K�l��9P�������M�H�G�M{�u�0�cx~h���o����xX��7�b���I8���=y�{����~�~�~�~�R����$��#�	�����hZ��,�PC{���p!�w��dY�n8����0(
_a�YG�Qp�:G�WAQPEA�+Pt�
��n���G�����( �.D��t8�z���z��:�ua��E6Lm���V�hZ�_�Z��������w|\W�Up\��'���=+O?z����-x����RX[,��
��� )H��E�w�� =�X�&J9�D�WG��,���!��iAZ��]_g�u����x���x�r�{���fB�,d�S)�*lD.e���*�H��7�,�f�~���	F��`4
F�mWX��m����hZ�V�h]��� h�?��YV��i�%i��x�
//��H���qV&�n��a��(
��u����gB?�J�r<�����efx._��z4]�V����{.����+�_*#H8�D,�s��������|H���i�4n��������?��Vc�7pN�'��v[3�E]U�!�����o���W<J3�I�qI���s)t4��&��F�o6~�{M�_l#�����'����z{zx�����s����1111����`S�c5eCR�$���y),���)~�$�l�TKj+�|�'T�>j{���y��
*���2�|���������������F�J�_��{b^Y_2%��i>��)�)��:
M���l0{Mf�"���%vD\����3&Y�%�>��9��]�-�����p��H���O�<y�+3��O��%&Y��R\�)G���T��D�eb�V�<��,�nR����7�HD|'K[LS�i�r<13�������+��r�v�%�[�}MM��C%v�>"�W���lA���q`�wJ�`�Z{��qv��xIJ8�NsRm��m��[�ou�
�����<����E5P	T�@��C��,�f6a�����;���>�-��M�wN^"3`�����s���P�~�@;���,iWD�~��������E��d!�{"WB�o�����]����!`�y��������}1�2���x����a�)�y�7Md�g@?g�NRY,�1�w�p�9��m�a[��K�����t����\����v�`�����1*�l���H$r��n_�
Q�v�u����cg��n�XD�
�R�e��OJ��(?P�#��-8����6����~��gH�F ����?������'���c���7��;�������y��n>\��o��g�7���/b�sR�:7�}a������y�uUg{���.B����3��
r�r.*�����c��a�I�����qcR]������y&�����l}@��������#N�C�0�l�
��D�	/�E�����,����H�:9]��P�"�u��BDXW8���8��"~Y�u���9z�����q4�zq�����[;�����`?���O�~���B�6K�J����.��^�����r.Y����Y@�d��AvU,����@-P��w��`X��	`�mH9���H��-x�������:�{�h����.���w+�j�Z������R/=|_��|o8�eA���p\W�u��G����0��I���;��.����>O��� ,��� ��k����4�Gm����`,��dQ�������7����p�Sl-�wK<�n�&�bd�@6�}S������*�
������T��[�����8o!'��27mWf�8�#��V
E�j:a3�5������4"�L�+����\j�!�m�[��(�U����"�2�\wU������uK���L�_xgU~4�|�Ti���(��A&��4�fg�����A;i��M��{�����\��82�(��ITw���FDY��S��,}'��7�"f/����=�9}�{�:K������t,�z�������Lse�u]�)�h"���9��oIUar07��7JL�T����.�[���8��w�}�Ey.Y)��#/�����V�U�'��`b�X���A�����*��/�����~�%�Ld�F���hp�sc`���6��R�:|����0�q���I9�l'����fMp\�&	p�Egg�\E��E?��*f����t6Kb�#5#�
=B^���L?s�����)h@&�>�1}"<4�a56R�?6�C��@����|d�'��K��/�����j�;h�9��@����D5:�B�T.[�l����C.�����WG��7�?}���u_rV�n���&�'B��,������>�����	����2BP2s�JB$�����?j��#��x|�M�l���.��wS�xQ��f=!	X���s����6��~���)����X83��h���n�b^����V����R�Km��uYH��hS�����%�4I�b��?o�$��&�!c��������B�i������^��e=]�Ggr�d�#�"�*��w���Z
�}�o���G���B����p?~o6��^8�������k�����]w���E ���\!V�]w��Z�y�����qu����m��{�>No]��~���������������T�2��+@:�e4}Kk���i�)�"G���a��j�"�^��I����{�3�1�x���eQ�	�|i2�����D��-����.ew5����&���r#��U'>^�X�������#�N|�1~4l�6R������.��I}��B�����H_-h�������8VA&*�7����sH�afF����p��������k�����<��J��t��73�& ���/�U��S�6������,Z^5o�o���'�'�������G������wO����u& ������f�����Am���_G_:�{�>��m�Q�����)�~����_o��N��z�zI�Fk�*���H+7��r���� ���,�d�6�-����8��n����/�b�l���l�z�kl.���55�nL3��H�J��I� |�a�_}���U��).���G�*��b���G��b�q��s�e�8��������T�3[Oh6��7�/������??�5����@
���vV��@�";��C��m��x�[�Ys�i�����M<p����t����W�?����hCes�I���oT8_����E�����]7k����f[bm���Z���K�5�|7����%�g���<W��zbP�~��T��r��v�����������������1����(?����07�H;�2��,�0����0�Uic�Z~�}]N&���~x�<��T��Ll���c�%�a;4���&���s�r�����Qj��\gls�Nv/��3v�-wm��^���G�:�0i�����hu��5i��am����kS�Z��:y���u����"����X�e�`����,��7�(������O��
UN��w�q\E�s�jx�g �}��(��vg2?3����7^��t���10����A5��������F��g3f��^��N�(�}T-�9���7:�V�i���U.���_�~Z�F��X���O��~��^����.E�������B/r<��T����:%���?KWI��4��4%��K�����s��jR3�����]4?D�n���(�6����m���xX�>�I��B:
/�	'<o�s�c��������;����b���}�v��S�����^����n��xQHZ]�^����{yt��A�:��N�����W�V��[�N�����q�?�
�������#�I�����
<����_��{sj�}s~��><��WD�~R��h���t~��(uA����=V��HK7?��G��g#1���u����p�����qewzHG��\8x;t�� �^���G���7?���w����[��C��:�N���g+HPv>Fp3�+kb���e����������}5�/k��W'�����w��bk+��5lY�f�j�Cl��l�n��tr�7��`iamt6��<Q���7`KC4 !�(��\���_<K����������VSkNz�#t��w��~��Rcx]2��0��C���&�%p[�K������U��?��n��[56z�y�����uH�#�4�6/�n�UAS����r$���^+4�08OQ	Wkc�A���?�5?�%mO��������]/�����n���n��r?w�xK1�����8�DO�������`Yf��	���Z��L��4����zq2�i�����gr��z���n���<�������������_�� )H
���W%i�"����(8
���������O����)`
�����`�=AO��=������u'PTUAUPu��Ja}R� H�u@Z��:��?�h�/3�j�E�\3�`���-`���k3\B�w�7[]3�!����`.����z�,� )H
�����
�r\�kEn^�W�x^���V�-����)x
������xZ���?�=����	x@Z��iA�KIkc���a��ATDQA�u�B�U���98d�i��i^����K����, ���j����{p���s����D��s*����������^����G�3�8���:��A���<5�
����*��U?�%��Y>��J�Ms�""��U\���H���qV&��6��U9Xv#�3�3)���-WH]f6G]���/�(?aH�{������+s���k�\'���s���<��E�kYB���vE3�D�Z'�����.���x�������b�=1����������� .��.���f�(��V#��o8�xbSV�������L��v�;K
9���
�N3a/q����H���\�����V�2�2��.�j��#[�P2�m�����Sq"8q���>����kcr�E7%-�/�k�O�Z�-�<����l�nN��R��da���
���J�m��9�)�9�p$���������|<�F�4�b:�<Y�}�h�c-\��\j�}������r[�#�$��,���M��p��l.b�����K��9�|�s/m-yw]w��������)��IYN�g'��\�3w���	'��X��$QF��K�[������w���"8m#��Y�aZ��3�n�W��������Ht�����6aAO�?������b?��S�3�&�\�����jpM&2�w�u{"Th��K$*sE�]���g2�J�����]
��|����<��K�j�o�~�iQ��Q�Hi�N�D�����Pc���hD�R73".M�x�iI�!����6��iN_����r��'�EQ9��rm�+�Q}��*B���y@|��-Z����e�i����W��xY����#�@:���1�
 ��0 |�b� ��
���	`��;����08�)�o-a 
H��4 ��'���j���7�PW^ 1H�� ��$q�j�K� �b��NA\�"��&�v�s}��w��X�!��� 
H��4 }7��WIl�� 0R	��e`X���[��N��L�c5�a%��` �N��-Q����+����0 ������������Z������m���\Lfq�4.�:$$$$$�&���3Z���.������	�G��8���2��A"������-��(`o��/��.nG�_h�5b���>_D��oy�inz�=��N����V��k^�t�e5juH*��;���6����[�z��%��X�:Ml�wV<zDMXLvE���j�rm�.W�+��cs0^q�4U��W7,���d\����[��Q��Cmk|���n�k}t����AK��-m���������j���t�6�
n��������������� 1H��w�{Q��^��{����f�@1P�w�#F;�>����'���7�jgg�i@��kH/��J/GPT�AeP�����`0������m�` ���M��-���|�oY�w+�@4
D��D4"����.���u�cn�h�IV�o'������7[�N���1�f;�/��x�����K"��Z�^������o�5n/]U���2���)
�l��y	i��Z��_e��Nh��(+��A8��=x�����Rm�<��n�����k������\����1
����\�O���O
9�&���O�<���X+@���<��C��f
�������o���;��������s#����:����[7l��p �='���i<���9�nAAP�R����@!P�C�8�	���}`����UAC@:����9��������*D����N� "�"�����~����q� ` ����������6�
l�l��<�Gc��r�ImTq���\�������]e�m!��)W����lh��1���jv�op���qp�[����L�`������f|
�M~�V�r�0�73���s�\�G���."mM�]|VfQ���
6���s�u���L��|	���W���JC�z�p��LdA�MqH 1)T,��Z{+>n����+������,D��'�B���R���:��>� ��H�c-����L�*j�@�+3{�H��5���k�zP|}}�-?�=W7��'Id"1-".�+���~���/~��+��#���J&������
B�u
�e*E�D��E�4Z�����BO��1�>�H�E���������G=>N=�R���[���(�:�0���C/��}�\[��%oSc�$��<�o_���������*���]�Ck����+ �+:u��q7
gUr��BM�%��KS"}�t�!�L�����)�	�p8����6��a2u��Kj���&EzN�jY���e<nX����}�%d�%W�y��h����������&��rq��-.%x��U0��n�wyq��������B�_���{��e�����������|I?�|9��%��(��I������o�5��o����/�B
���K��X�m���z�Y�G��Y����q)=�L�U�0��e���V5��?X�Ka^�����,`��`�__ �Aa+������V$w�N�d+m�:������\}�U���N�
�gn�.���3|����N�8yP:yr���L
�����@��b�_����pM���l���H���
��������H�����f���$��9�X��u����1h����M�xo�a-8V���
���{S���+�
�~Vxee7�n�sl��e������V�����[�"�"�C�+(�V�uuA]P����e�Fy"
9V�%b�	��F��OtdR=Le����x|!����r<����-���00��k�^�c�y����I�=��4��c���H��yQl��s(K�B�'fV������s\����`�f��~��wb3�sXH9OMX@MP�5�,��X'/�K��/�y���R2��b�� &�	b^}�ngS�-����L��Q�G��xwqK�nck�e���H��c����N��&���^.�r����V8����|qqqr��Ic��/���N���������I��*0x���1x|-�H���#�"��1h���1h�)/
0�{���lvQ����IX��h�6��e�%z����k�~��T���
 @3�4�@���L���n���k������:�|.e"��c�gi�a�vn'v;���\*Q�g#��~5�@���n��:�^����.J�b�P,I���?�f*e.�T��F.q�iYs9:�8�����b�UNaQ���,�0�u���z�uY��h-�i����Q��^'��"zLd7�k�{�U���y��r�D�AU�e�Q��_X���-�U�:��-A
4E�~�E����`�J.x���D&�������2��������L��>!L.�-�N�uS�V
d�P�&�m�?�/����f����7�sUS��2�5���-!^��!�!�!�!�����R$T���0��g;eY$h����9�K�SI�f��!��8��������~�x-��q��j^������Z����.d)�?NNHHHH�{)U�^���j��Z^��,(����l���T��;�h8gb+L;��O1���N����j�akm-�j�������>a1��M�F�[��.�0g�u�����d;�����5�bB����������Y�����J�t�S�g cnK�������~��p���J��imL^���*��tl��.���j0�lxl;mUk��$$$$$��B����%�WobX���t�`>`)X
�������?sJ+�?��5����_���/���%	~���'��y��	�\fC2�ySng�
��n��
k��AN��9?or�4��r-�)���T�n�����i�
u�>@���LO^w��}��z����<d�cnT��N �2<2����K��.��\V�����lcU(�&�'�p�'���d�n��WG,�%�������?�4fs�Z�"T3 
H������N�x�����_vk��8����JM�3dYAV�u]�.P�������*�
�����JjjT��T��_�W�|_���F��w�K�R�,K�ci�4��WJ]�~f*����*�
���p&��Z�h�.�n�Gm��'��_�kk�D;8c����]X��*�YF����f_�LK� �y`��o`��X��P6����u��lX\B<z���-`���S��l��h:���>	��hZ��
�O9�k�i��Je��i4��!6����he�{bP�4����t����R��2���An���o�sfs����#�
���`*�zu��|n���?��)p
��W�����y�k�H�����+�
�������j
�����fMC+�
����*�zM��A���TUAUPT��a5�Z�;x�71��+�
��kgL�y��9�w?'�fJM�n�@��M&�'QL?Z��zg�Yp�g�Y�l!MY�TAPAA�5	�akkogK��������,�����H�!��S�!`V��(���<I�3����J��%�L��]
f�D!���*�t�o��o�}�[�H��B��;zUE����.
�-Z��UYj_�1,�X�yj���W
�Z��1=Pn�4��<*RUj���bl�"��*M�F{Y��W�������:]�����f����Ap�VRJ�pm�x2�f"�kIp.����q�zL]�r9u]��u3�E�@-
_��G�w��U���l,��v�B�kx0t������*��F	�]���{k���Rm��N��0���#���#�&��m�����5/w��q���O�(��ow��~-��Y�����c7��@��7A/�����_.PO��BE����P=��"���y�j��-�Nm���Nd�j�S7��=	�?��)�A�h�yN���Ag;�)�Y�a�������U�N,��
'BJ	����9�6w:��^��D/�9.j
��S-�9w�%��~���2���*{�b����c�:��V	�����v=�fcm��I���e1�
�AaP���T�k'�~�_��~o��M����1h��2�F�/��yA^��Z��~r0��x^��z��d�����WJ%�����p�v�[H<4�@3�4_O_��e2� ������Df����3��yp��o�&r��g 0�� ���j��_��|�[������`1X�����k�a��`0�oO�j�60��`0����O���1`���S���#+7~��`0�7��V8�`��0��p+�H��0�����������8*����
)�i�q2F��8��F��:�'o�kU�;��4���d��K\�����3>�]��'��\��9�    n[<lu
i�"����� -H��2i?lm��p6r�A|�Q�@��������U�X��OT����M/�B��} �T�\�����]��&=�������VZ��G�*b��a(������F^��f���B�E��&
j}x0{�U��c�&Js���SUj��(��6EDU���������#Mh����W	�k����nnR�w}�,�}4�d3�������Od��i����{e��Ri���%<>:]�QU�H�Y�s�H�r�o��H��D�j�m3��Hu���������������H��E�^�������$��,_�E��/�V�:Y�����������+�sAj�p~sV>��$t��&���h2y��x�����@�r�\ ��XI�A`�A��H����-h��h������-`�>,�"!<���nx�8�����O-�; ������I����`;�����p���q �nq��sY��Uv���iK��h��U������^_��B&il�A&ERJ�u��"����������������p�&=&����l�S������\����F}�^��!�bF��f�q�
	����Ey�����H�r,�E\��4���5���)<-�g��#1N�FF�����W�l����G5��G5�f�q���5�GD:�{�UK;E���zQ����pW�P���K���]��	j�E��QPK���*�2�|'
��4��L�����D�tic6d����vc1��6���+���q������D[!O�q=������I��V��+n���?>z	��5����d��.����&���)�<��d�z'��Q���_�w��F�F��zy�w�]p������n������.��n��v������`���0��oD�&�6~uA]P�uo��{;�$�����S:������4/�}�HP�OS���YJAp���.3)�
�����C��|6�I�~�1�*Y���n)���Z�F��,8�'o��RU%U�fR����i��4�='���YE�����z�u���6*��Te�
2Id"{��]]���N]p�8of�������{[��6��c�u�f��n��?C<A<A<A<}���?l��l��>y�����;N����_���]fs��x�����&�U�TYz<��R�����1@?d�{����Xc�0�y��:M�&S�/���s�o0C������/��Q�.x/�^�J����������F������ �cid����G��B�}��`�`8�����K�{�9(�` ���%��(�<%�(�����r���6���v�]`��vS�o�����������|'�O
}�- 1H�� �
�u�E
y�)|f�`�4��vD��9�_5"E���5
��w��E����W���� 7�
r������[3�yA^���y�v�:bG�,�8=�1�R&"Z���H��J��GC�K���4b�E�p��fi�����0�c�g���Ra$������yR����8���4W����tH�BU�MUx@��;'R���oiE.���R��[j��,�{F�V4��~u�m�l�6)c��=�fUS���g���m��g+2���D�=��M.������gg�*�C�Ez��m8�k������ce����o�E���$���-\��l�T�m����	U��Tg%?�3zg6��0�z���n4��~��*B]kM�A&]@/o���������3UG�ZFY�

��l{s+�����U�^�p\�{����{�Z/����
�Xn���r<�M&�9�(���q�]�mS�m2v���[��{?3�Z��_bG��Fp~1N]`-��������(a��x�
oV)$�%/vmEmL�E�BM[��;�
b�� 6�}5b/�U��0����lB�u���-h�����i��/�4�V<
[p���]<@-P��@����Q��<o����PjAZ��i�J�++�`-X���`�MY;�E����%���m�����/�����:�{�r2�Y�BFY6�r{fx#G���B?��G;��o�#���u���;���"��M�M�b��
.����Z���3������<�,s����
���}�_����-~V�O���AT������W�#�1J�R��zX|-�=YN�o�{������N�:�[Qp��IV��H���#������s��eWp����r%�+���LUQ�8QDyS/�Ob8��2;^�'�:;����>��u����O�U.����>���S��(�m��K�4^���i�"����,O�<	X�n��D����"�M&����(���u�8H8jK}Vh#'=�
~�^��.pJ�T��)���|�*����T�c��t�9^����l��|�	�N��H����1��1��/nc�
�b�����m{���F:�ijF��0*���Bms=)J���{��6�k]�|%E�w]d@R�r�$<tCQ2OdR��b�u��������nBH������'4@�� ��+���yz��]{��0��D����fX�U������������Hg:��
���H^�%��Do1P��(�V&;�o7�M_9\-��]���i�yFk0-&��r��t9��sc+����(���P�8��q�����3�����s8���7���$�'K�z�	R7�W�S}�����w�"�%�V^��:�U&W�8[4�!��hG5�,�����KEr;��y���'�	�r���T���{��"��2�U���F�
<����am��O�gL��Z��;)�
JE��UY����'��c��
���R�re9\�K^L�����*�~���T�����{�$��	F}�f����hA>�Ma����K����A?��Dg��b����;�6���'���L�6����?��&�H�����=`
j�F��1����=AQ����I���U����q���/Vx}#�ow�5$DE��!*BT�!*Vf�c������l>{O���2V�'�+�+��{D�������^��$M���$O�������w��wSd%qXW1
��F��ij���B�,�f(S������:PUP��#��>�	�-�@���!�hc/���_~~~~~~�6u��r������������!�
!���n��������������=�
N?������������f���Nub�����w��Y���
_��e�KOQ&L��C����=�:�����|iR���fW:3��9�����:&�J��@��
�:�U^6�0�P�:�Q������Q�p��*�..D�~��,�+�����x�R_}%�H��#!FB����Z���V��wJ�����f��U�j{L��@����t���$�t-�ys��vK�����������m��j�;VE�t�9��w����K�+WA����s�c�z� i���C�z��*M�24�*���r2W)~����5\k�#�5Ej'��%��,�s�\A�P.H[fr�d>k����)�sE��J�{��\���Q���X�����A'�+k���8�m}P�	��1g��gY�����>��	>(uV��a������z�jV�O�h}�"K#�)��~�p�����m-����O?!��C���/n�R,��
�~k��������-��W����\u����W�����g����{d���������i���0�ASbG'�|L�k�6`�[x�i6J�.b/;���f��^/�V\���n�-���wa�%�F���!7HnJ�aVXct�����#J7���G.$����6�Yl	U�^�q%a�GE�@3;��{T$����3&+#zhL<��[�L����7�f)�Z�:DY�!-�����A���r��	^�z�<(2���6Hds�-�~�����}��]s�L�1�)�}9�*���HKw�����hb�O��u��1��R�F�Cwj�����?�!�'�\
r!��u��7r(���A��|�N��}��Wff�6�����,�CC5������[���+X��H�!����}E?�;n�Gu;���r���T��m�-[��4�Q��~������O����k��J"l�y�9�bl��Z�CHq
��z�����kC���'����-�>�
$i5
��vgG�4�O��8�.p���{�����s����/`v�%l�����3C����o��r��Q����
��)�u��i���U��F?C���=��*A�u����\$Tot�j��g�U������7����/��MCnE�`uX�%H1+�����FXU�]s�V�4�Q,r[���=-�A��p�i��	��|���$�a5gHT����
�`��ys8c��	s��P�{	Q@t��u��XA��qas�g���n� ��*J��Gl.�5;��X�����z��m�"�hqw� ���~��<)�w��������?
�0�)3eknCx�E�~D�Cg�p�@p�>N�����ZD���'�����%��H��t��h�������_[.�r!��\�}���nd
�J|�Bb?2�����h%�����K��/ApApApApA��7��c^^^^^��+e������w��m��)=�5�<����q����%/u��j����������_>���`�`�`�`�`���)���bbbbb�bo���������~Z�Q�p�����w�O���������-
�K�b�W�W�W�W��O��Xt� �*�*�*�*�zdEw����U���+��PZ�
�
�
�
����kC��I�V
�3j��v���"� � � � � �������>����F'����B�&�=�L����V���$���A��~�)N��9�h�h�h�h���Ct�I�8K����tRi�%)LO3�oI�`%~��3�Q�FS	��d@}hj���jw@5�Rx�X�?������z���9eJ����rZ��z
9���M����-���mx���.��.���������s_�K�r|����|�?�2}�&'��S����?�>|���������-�����]%��d���n@H���;�+�]"qe^�N� ��NP��-�^nb����s���)v�b������K��m�t�]9������uk��K�,:�.�F�w�E�k�~.�XK����^|@���������S��8�E49����)[t
Hi}��+qD^X��36x�
��k�5���>�j6�y�������2�'��Xg�e~>K��u5���4��M��`I?��81w�N<��:��N�����]����X���UJ�	�QF����e��@�Z�}e�R>�bwUNd�=�@^�&�\��k���n0tKq�6���� ��5L�1��:��B��(g+��C�v�W�W���Jn��k�0 �=�O����?O��a�I]�y�jnkl����4(Y�GwQ�^c7�B�0"��S��a�Ykf���������F�6������&�!<��4cj�a9*�[-JQ�]
�N;����w���<qI�o�u�
B���r�J�P��w�zs�
���U%
�N����Vw��u�h�z�L����8����k/��p#%����7�>�%���7����3=&���]t�z���a�x^�8���2�L��0��C��qW���������CY��1$=�@���86���j��
�@@�,�[v����5m�aU��&��no�3����.\'	�DYJ��
�������	A�T��Xp�-�������:�e2���&����D��m��D���� m�q�q�#\U�����t2�n��E
��Ap�W����G���|�>jGET�����T�����g�����=a<(>�k9�3u'K���(����up���\��_��z������<��5���q{o��7�uI�674��]:=yK��O`4���LFGGg��a��H�����L�S�C������O����Yy�z��,�x���Cw���l�e�����78s�������_}n���$/>OI��y\z����;�������N�����;K������������������F.��w
�q�C��
��P�	'��5�)����g&��D�����o^���)���%y6��XI�F�(���c��	(��M�k�j��P�jZ��s��
Z'A�q���#	�0Vh��wKS���Y�����An����X��2e���,i����T�3��v��.�M����#�cg�t�`(���0M#
,�O���Y����Z�Ht�_�N�1`������so�p,��!JHN����.��]�Ur��5�����rV�U�g�j�����<T:����%����+*_������S���p���+����^|���&pAC�����U_y��3]�lla������i{+	�����[-f�����yW*G����[4�z�]3�Cm���s{?LJ��S��W�zt�
G<�i������"�6�@p��/����� 	�s���x��K</����A\^wa��	X~G��R�m�{��7�z.�3j�~�R�a5�qh��r7(�o��2������������<o�e������>!�����L6��F��p�"d��m��r�6Q�rB`-�o�&9�G?H_�����hQ�-��g�C'I}�3V�KL�S�6��a���<W��"^0�����,o��Hg�-�@��t{B`f|nv��FG��Umg�����m��J����xm����D�LtPK��x�8��-�{��rEK���/xL�������53�j3���7�o�W����vb���q��hp���X�p�C��-���~��>��-��� {Tr��,1(�xN��^K�{ke���c��bs1��]����0������V����"�+�o)jW%VER�����<o�So<���g
��pci��jy����~��R;�e�Ef�c�,�9d�E��{b��/���('�sv���i��gc*�EC����
(F9���:l���+^��w�S��_v^�>gL^:���q;���^N�nX�����Xm��4��B&7�����B�hS������>��T��0�����K�e?G���������&��q�vK��C���}�;%��� ~nJ
s��do�,��<4gn��a���
���~�>�Vu�����u��Y�0����0G�wwN������a�D!o��s����b�j����0��	�<��[&%���q��KzU0�$��d��_vy,�\��.A1����{)����;w`���[��OQ���#/��%�%)������Q������k}��1��N������p��
�Z	[�K�f�� 8sQ�
{tnG5�y��|[E��`[=�=eY��U���^�Ec����O�1\K%^��H������m��c�� KP��+�r6+�b#���g���T������!^�����������SE�����_��5��Q�R��9�of����Vj�2���	���CD��x�Hg`5�F���A,��Z�&�-��Tr���u�q�`
*][�2��b��z.�tsE��5�.R���a��1�����p��ng��V7�7v��T�9�M�Z:����F:��]C����N�:KV���5�
ej���;7��5�vV�Qy�����;TRU��5�c�!^[���i�MkMyh����7+`
�$�|J�rBk���6S�f�?3��o�\����C%�@�[����,���?B�Zlx�������:���I��(�M��%������_Ypo��k�-��m�[��������*5��ofI�4�w�n��V-w��dv�w���hu��'iu���:J��f����I���������%�������]>%���AY�L�G�^���Q�����{����}��}h�n-��R���U9Q2�BDC?G6���F�7>J���C�FF��2#l�w#�����_���k�k�T�D%��������	b�d�FB�^��v���;/�;N���z�[nz*
����9���E+4`�������T�~��PGq�`�5���8�70�}�7�K9����S-�KD�r����i}���8�j:���2�F�7��F)�7e|���P wW~���t�%���`�#�0���E%�������������������Cm��
�R����Q�����Q?d���g#_u�p��?�y���,��F�n����&����`�����z������b�V\��Z��L�Ef�k]$p�o��x�,�����*a�Oe`"=�L�W))-�vT]���e�g�S�����������w��e�'����,�0�4P����yM������8�X���>Luj��?u��p�*y�{��Bt��j��$~����-��P.v+e�^�r�����KQ�����5�.�a1���C��o�!����-��B�r;,24)�	��g�P� Eg`Qx�y�0KY��W1}ip��tJJ����������-���Z�:�n"p.��w"��<�|�[������������<�U';]F/.��#��%��)B��4���Oy��N��eW�G'�sNz�w;=�e9}�{wP,�?���H��1"7�����u�����C>R�(.��|������>�4�����@���
����.h���|iY����5�im��i�5��>>�k��v����>i��|3��6*$�D]��^[���U�S��^��[S��
X��>�`�6�v������lfHua-����bG��0>�3�-�������Z/[wZ�dw���[���n��=-:��9��S6)@
R���g�u�ta�,:��>h���<�*�t�O�U~��1��,O�7W(�dWUU)���Pm}���^O��������������&�~uZ��R9����-�ON/��/���{uxy�������1�����,m��
*�1�C���K.t~�������t��#�xa�����.K}m���Ma^tn{����%�`��@���B�D�CT�|=���V+�K�q�c8	����G�YDM���,�qol�v�B������m�2*Q�Z��]���g!�px�.N����Z��f�-�C���:���#�l���j�]b��L=�aB�����.n�q�V���0�������-�!!����vtF�w��}�*x~|���5����o��@)Zk��h!�E����(�+����::;}����RY�+|D���;?�1���������].=7�)z�	�� �����Yt����|/!�K�����|�D�I<#�g$�����*������-%��y�F�dx�4��8�'��Im����&O�
9����hu�k.%Y
�j��r�%G��fj�"����l��u��Z�:<�
��%�Kh�����]�0�Y������H��i�i#rOROwTiK�\�\�\�\��A��nMW�h�h�h�h��G��.��'�.������Pd�����S�����/���\�\�\�\�\����&Xo�h�T{�9�����C�m��������������!��C��L���>��C�=�=��$��(���Y���4�$�x�gY�q�t�t�t�t��{���0X,,,,,P��:~=������W�?$$$$$~$^�%N�X�X�X�X��!�x��I�`�`�`�`�����*
���hz_��W?-�\yF�"v���%{�����`���P�.����0H�3��c�%�.7��)eN1>�
���@b���Uy]�O.�WG��]���8��K���HU�����B��g���	�D���� ���eE=r�L���k�<<��kq�y�=J��H�EA.��2$$�[�)Hx
�$C	[F�N(x�$
}m���X
����K�}3��h�4������|��^�'��i��
A�*�1�4
7�������	�6F�������������o�V0W0W0W0W0��1w�f��'������� � � � � �"��6760
:������8x���Fafrf��-x,x,x,x��xh��m��o���`��	�I���j{O`Y`Y`Y`Y`����{�/�,�,�,�,�|���\a�5`�>,,,,,?&,�[���@�@�@�@�@��BqM���������fa�~�������+#��+F�<e��k�ycm?�#���4�"�9Z]���O�ko�S��Y�&���A����x��i���,E��$�L��JG��]�T�m���r|���8{�0f�l��j�M�:�����i���>U���a<��k4)���r�\�����h��o����!����z.��B��0	a�����~o�;PX,�a�a�a�����������@ea��*�emfff���:2�S����U�.(,(,(,(,(��(\a�K.t~������������4/:����[_�������^���:d:BW�dY���y��(�cAgAgAgA�GB�L�"�o����"�,�,�,�,��8��|-U^=�������4�-����������0�M�u6E)���w}hhhhh~h�t^dv��r������������#�q:��������������i���(
}t�_�!0,0,0,0,0�80�����2/v�
? on����&�+�+�+�+�����qsso{Sm+?�^�Q��!>n(uD�����e����LO3mt�5��(�bt�����������/�/�<�{�(/�8�A�����ch?/�Hqr�L5(�k7h8��J0/za�����Z��&c�{x3u�E����1L�4-�ij����.'�b^�0�3���pu��R����m�U���ym�e�"�0Z%i�����}��k/1�Z�&�a��+k=Yc=X�ur����g�����iln
�Mp��Q�gL��vUZ(��PQ��]+y�A�50i��'0�r�
���F#���ps���+����o�V%��"Y(�^�f���<����<!���B���M�S���Mt��{���7�A�]�%��N�Q��Q����ut6���1�&��N��x�gf�6���'�+�5Lr@�j.��K9���mnT�U���<P�w���^������vq>��������>�G���]QF��$�����}��In��h�U9v$��K�>����k�I:��/b���N�0�F��s�s4`����\g�?�\t�y!��m��a#�Do�����"��F�gi�;�
)�T��W����
��s�fW:3��i�-�&����T��
��P���;a�_��i/0~���Q��5Rk'�l���s�!r����*�a�3�<�j@�t��"��Q%��4c��A��Q���H�&Pi�� B9�u��L8*�)��NU����4�gX�����)t6��a��
�+q��k�8E����q
�hej�E���?be��B.<��@���x��-FR|&E<�F ��M�t��Q���8U��F��D�>�4`��o���Y��}1����5j�(�_��]��|Ka���v����=���&`���B�YxB�(��=�$�p�,=���F���h���z�����P��
��g��J|G�`����V
�a�o`,����u�S������|�����f�6�Dlt���C������*�����^r��n�:7�U�W{����;�1�K�o��z��w;/�9�	�.�o�bsbc���n~�L��.������\+�|���*7���p�
P�����������G�e����+�C����C��xJ��A8B:A�|�x������x����	�)
bfU�0#�~N�|(���P7�&>����/rF�&��_<"�s�YyAN�`��I���9�p�_��7��p�O
�J�?
���!�B��!]��t����2�#)��������������h�w�v�g\�W��!��b����ni��x���+�" t3�:���V����0��N?�[kK�1�5ow/;MsM!��C��T�P�� %�1�����J1���>��� \NM8��92�
t��S)�1P�$����&��`��iV[v���&np9���+iI�g�`������pN�D�S��4�x��zm�5�^�co��WGE�@&���]zN�u��52��Z�����{u������;����k'�3|Q^#�Y��pV���:�,�A�:
�3�Kf:���%F�
UQ�"��	�Z�)��� H����`�`�`�`�o3I~|L&]��p�f��*e��c�4~1�� 4�u�������������_���r������p������	`����L�77�,��	����|�� ��?���.���K��Wq�Vy��_���e��%�����~�....pYb�O#�����l��I���Nz��U�$���j�;K�y���H`J���\hu�z�G��UCj�M���.���d���T7B->z���D|��C1;�J!����&���#%��xI���_��r�{)�)��z��$H5������l�+>������4��G�4�Ie���p��W����Q�w�q�G���`\�J���q�C�J��>�&-;��������vm��7��?�����U�������6�[�6_����:a�a���]��0�
��!L�Su��T�>k��}k�Q��j��-�co������W%p
N���Np��U���Tf0�B����R�������������t<5�?��a���/�B[k���<���������\,Z}U�W�j���t����.�'��@��"��4��^-0x
�����%u��Q�@i\��Z��*+��|@�$������X��.e������c�����J��KL�Y��Z�������`�W�S�@M�Q���OR�!$N���UH��3-e
RJ`��u�V'#D\�a@L^�qJ���},��eh��C@����(��C�{�a�y�3����0�@��D1e��^�"<jdS?�����
�V�-�:����	�������1<R����c�F��d���(��m�3J������e��a�G����+t��3���@�_��kzyT����e�������k��q������(T�����D��o/Gg������0I������os��\�V��(3�1�R���y��g�kIiY`Km���@1�f�l�y;&p�)e.&�0$N���6���[������:~+qU��
Pi�y-LT�_���1����a��3Uz����.y�*�#j^"��3��*v<Y�'����(n���Q�{�Am�S��Z������iG�Ij�-�N��L��B�����N�������V�&�C�"fk_L��_��-���Z����r�my���������eo[����]1�gUo^��]^��}����.^l@Iz���|��u�N�u�� �:$�e{��
�H�G��BpKu��o{�5r�4�}�Z��H���k��.�O���f4U��������m3Gjpw9!E��l�z���
��+U���E��u������n��;���%od=��~�%��}�XSW��&�|��!5�xar��sn�w����#��(��f�W���Yu����������6fG3J�����}�����o����w�bg �������o4{1i�����X�e��������A~���8<�a�������gY���g����W7]\]n�kq����.o��MoX���}l�����f������I���QyX��	Y�������`�/�>������J\���.R6Fs�+[��M85����^��[�v~��7L�k�y�8iM[���U����Oj��6?��.��h��_�'bTE����g��n�X�F�}�i]Iw�-�uu�kS�r�-{�*���D�|w����Q}�B�m��x��G��,�=w�f��)[^��i^�6:���2�����T����
��ni��M�G�����>���+��������u\�y�b�\��t,�j�[Wk_�W���������C���gvd>�y�_�40�����~no�jv���j�r
�;���-=��Y�����ne���|�M�������%'�?/���oJWM�P��
�}�[�
??��	��0}m��K����z�����gw�N�������4��`����P�dI�]F�?��bI����5�Qk�������r�����N��I������'?�Y�s�����_���v������.)@
p6������3z�v�����O�D�Y~:���|F�t_##P�=P�k��	^���as�x�����^]M��.?t�����{��U���m������G[\N��B�����2�y�]N<&�3�������f?ONn������;�����-�O���0oG�Q�����K���u���`���6rHK������n=��e3N&��.���g�-h�����K��%@��D]�d_/\WU�:k��
��������Xm�u�N����T����^Z�&�r�������M�-����7m�F�����Gm���rW�5���[3n��V[Hq{���n�}V��5�����/�oz��=M�^�����[��k����J<|��R�YIG�u���
3�>	)�"����.��BQ�����MqsP�vl�z3@��w��]������:2�LS���o<|6l-n�Qm��/����U���},��>6�x�9���R`6V��q�85�O��{5����717��y�_h������MF{��dK���)�� Or�E`���v�m��y���f�
��cg�&��>4'd�������a�N���.��pN=:�~hc}X���J�2�!��4��Q����Vh-�Z��m����=�_��
-�)�*�����i�y��{��Q:�����������t��)����������^��[����oG����������"�)*D��"��	��f�>���nk>vFXp�W�,��z��B��D�>��"F�]uY}w&.�.��]5"���y�B�,�&��}�K���
�P���z��1E�����,����!q�P7��'����>�i�a)b�'@oH��������%�5�!L�/���7Q�:��;���r0�����;�M��|��X�crk��QY����)
�z7�:H�-t��[��KpGS�k1���O&W8��"��aA�.�
�8[[��9����9cT�H#���B��E���@,@%���a�CN���*
����@A3]�����n��bT1`Z��F~��=��=T�]��S������ 
����PP�g��m�"����-�@"g6Z
Q�	t!���s	(Z��*u���Q{�����9b6{��,g�jz?���9����!F�0)�'�D�A�F����G���.L\������,�j����iHi��������Gv��t!Cy[7����$2�6��jL�Y^��7:�	�"���=n�.��"M��v�z��0
�,�
�d���`{�?���/�d�0R��n&��������OM(��;�k�k�����k)	��X��)����[�;8�m��$�/�{�'"�eZ�I��F���	����b
�����������M��
s=e����O��ez��I|Hu ���0dI��������Q-�g�~�fRk���O���W�S���|g���(�c��X>W�#�~���j-e8c�r��^W�%Vnp�NE�O�z���U��w��O��^���h���S�7����<1����b�`��4��h��s�������������i��z���v�]o<���0�0��Oh��������
���7�I�}�5�����o���?��^�|���������?��_����/�C=��Q��)0�*t%.�,]���F+�i0�U��_�����}�X������]�z����O��2Z�>��0��pZX�����F��]���^u���{��z��?��5��>�09lZ��7Q:�"�
7��k��r���?�����2%����	s�Ev�E�(�����u�������������)�d���������<���l����w���������
 
_Ny:������g|6{���"�)�J�z��7��!JcK�k�\�%Aq���LO��]{�jqf�^d���?�nm�m���{����8�w�:
�Vu������.��Lg��O��
��	��,�S���:�E�g�*MQ��3����@�*Q=n;���I�O�!?��y�{>v�<�t�<�K�G���l���vx�}ic��6[���������;�OEg��.��@����y��/�ez�1\���_�����x�X�����V�����?��5�\7D���v�����=�l�j�����m����O����?�������@'��f�w������n��K1��>�^U�'�O���1wyI�B��Z��O�6����w�����Qgv�JW���\��OTG�A�]�k/�fVTo�Y:���:���rn:R��9�x7(���s�"E�/&��M�Mp24�
���xlU�re�"�c����d���������>zV��d�����Q|�3���U�0�F�k�tD��X��5��%4�C�~�5�Hd<RX%�k,���\��_;�1��@�Liz[��$�a�=nr����E�4�����IQ��I�5{s�I������c[W�iN������j|�e&��������a4E��D�T~��;����Tg11S�����l|��-�$j{�C}�cA�0M��do��2�`JA*�#3���� �$�#l���@�!��g��fW���_�4��hE�(m$�q�v^��P��3$�cM�f�j�I[}`�W�X��(�E#��zn���,����w�+����r[�����q��K�����+���
�~U�E�X���Ik$�T'j��'<�
�<M{+;�Zr�E����5����(x<��i��j�<]�j]�%�
���?k� �I��=�$����K
����mfj�Oc!�@������������I�%�CT���=�����7�LB����A���P��Q�P�s���Y���Zb�S�����5|G�Z���g�"TR�<����%���2����	W�����nQ��<����U�����v�m��x��{���xH{<X��U��@x'�<�k���(�
��O  ������	<��:��s�jW;��pl�����:x"��l���s��{�_�m�\R�[k-��G,�����i�
�Zm$�7w��h��������4n�X�����z��6AfyV�����@1��0u�y����q�Q��g�O5�K�������i��K(����1}�!]��B���oc�i��1N��^>)G�}��k�uf�[.�s��.KF����T���L6`q�3k�����P�$2���J���v�����_<%��[3G6:��=�E:H���X!��h�X��O,e���:c�pE�k���#K��4�N�DB��eV�YJ�H����qX�n������Lm�>!y��x+�R�H�Q�PTP��maU3��5��b�@�L��hSK�>��E�[-Y��
5J2p���S�FvG~�	����'���������KY#�y\n�Dw	E���Z%H�AQ9+m���EN�y6�1��7���r��$�$~��� �k��Ea�
u��n��b��X��yM����g�m�����v�����k��5��c�C>8%�/Q�ur_B����%�{�g���|huf��k���k��]��c�_A�����G�yq���t������;w��z�V_���������������8	��/P
@�B�{
H��.W^��G������fG����r���zr����������,��Y����F�.���}zT7?
�z)�R�������K���^�A�Y�k����@JW
����u��2;�AY%�ex��������j����u�������o�v��x�n9m�}�2���:����at�G�_b����;EEQ�	�$�^D� ����������L�}#q�A��j����>u�3`���d�K�7�(�F-��E:��2,�l�Jg�1Z/��Jc_�v�C��i���&��4��F�H]�X_��t��Y�kc�GN��H���7����pT��`��N1�)��N����]�������p*~F�f6�6�:}�>��2{W;��9�Z��������vq��q���/���$M��M�������|
��&�������=%����3U�����rC�:��z��O�F����w��^���:���X�����EFS*N�[iR�� ��'��0��%����z���?D�P�����������m��v�O=Q����
�����_��9?~sxy|��������Wg?�~z��������W��wo���?{�����r�!�����7	�n��^�si�q}��z2Np��������86�@>�,�Z�����ol�cb������=HJ��Ch����'��Vm�&��x��m�iI-nuo ���#�)����;$����S�(�~�Ec��Zu��^�����T���*NH�d����N�W9-b�i�����{������C�fa#�liu�����d�j�� ��f�����^Q:sf������}�nO6��A�
QSk7��:����>b	�#��U')�C��h��]��J&��Lg�5�y����C�az�m�k�:|��jR���S�q!�*cR&��x��]u*���M�����T�6i�gd��$�5�����U	%b��V��
)�5NK\@_j!T��z�Y9�<�>�2v`��}:�A������J��f�DR�����"����d@�p\�v�����@�![L����c����{\cN������sy
�JN��qP���i����.2���7,{�E@l$�4�L�'_'����L�eM�0Z���G�b��P��PkCk�pF�.���=|�F��iaH]�k�����2"	\�q^x�#����hk����h��-�:p*kH��� >zi���6;-�4��� ��}�g�Y�Y:I�qs[�S0E�pi�a4�8�pLE&i��)Wc_�~1���Exe��h�i~&u�Do�5v?4���X{X�|T�����zI�~F��u?������
�O�����~&\x"������`�Yp�I�^����>�-{�����Y���w<z�+_��76~�r�Z�5�z����|�oN�6^����S��iO�b�ak�b}cs9r~���}��;�������4�,�d�������97�iz�^Z����#>pI@�t�����x`Z�k!�\����YO�j�M�DV��i������nP��.��;Y*u���1����o�d�4�. �������xF9��)eK���Oi�wC��Sb����Tc-	����{�h��+bU�&`TPXW0����V��0VP��C���H��cm����o���6l����������y����.����9�dz�&sS�l@��|�E�/���7�j?m���������Y��aW��f��q���CS���%��j�W&�����J�����	�<������Np[��9��w�[v�+��<w����I5��z��0H����f��$[5O�F1��B���w7���{�c[UU��j�����A�����Dr���W:����GXw�n7t��
��U�V��)������������k����nl��������c�k8=t�f5�#�-7<H{��J�A�Z�z{}Wr�8�-cc���V,��,a�^����~������#��5������������_�}����h��a��s��h�J�m
����>Z�^���Fq6��Y���rS�T��p�����XWo�k�7���'�%VBl���\������~�/�Z���X�"K-��>G���%��j�-&������7�������U\S�~�i�m�*M���
���������e����,`�,���qw�Yy��
�Rf�2��'
�;�`E���w����c�'���bXWU��l!f��5C�)�����}�3�_�����=	��-��I >��c�k���cO{}R��>��+\J�&�������g7�����#����;M�6���t��G#��L��Q:��'�����o�b�w��^y���@��b��k5���I�V�!V����dkjr�0d��fs��pp����@{��F�h�����]�,:.��O_�f�f�f�f���u	p�w�'}�k�ow�O����3�@��Z����7�WCF�Y�\
�tPy������vQ����']����>lW����$��O��'yH��
��`5���p9/�N������`���Q�*�?��~���G�>����tX�p����*����o������(���F@o�����|���q�+��\
�'/^�9����*�e��ZX&R��t���3G�'���gZ;K@/q�������Q����U�����e��M=��k��0%�_T��H�}����O�*�����-PZ5A��}!"^!�B��d��`�`�`�`����"����z�����0�766(0IX�Z0�B�J������Vz�_Q����>����	�����[f�B%�6���z6����u�B�]�*Yt�"~���qH�h�;����;������'u�/}Xg��O#�e�������n�c
�dr�!�\�PGAe����
��u?�[X.a���\�r=���6��_{Nz��&�S$�&O-U(=��|�
d�-����jv�f;-!:qQ_�*����e_\ c�y�)��BF��r�%my9���G����q �aU�C�f���j+����Ba;�.WuN��La�(�Q����E�O~��{@\�"d��-.;Q�#��Y��;|Y�X	xxxxx�x/>��Y��Q��dbYY��I4/-=v��#6��%������bpv�������m�;�G�)%����
������#� ����+�
jjj���_���
Sa�yh�a�j�����V�'o������-X��_���G��������57���V�%������ ��^�`$������0�Sx���.�[�~�4��+�����i�l,����O����jP��):i����b'9�)�����y�u"e�'�x���#W���?��6#�`	���*Ma�I����SC���t\���!rF��E9B�+�+O���^�a��p�T%��p!@)����\v�����S?��u�|M�������=
�!qm��T��oz�Y���
�����J�Ig�~�"�W�#H��S/��Q�k���������
������N3DO36|��E��+�i�?��/w/��]��k/;���l�_�@�Ky�{��=|�����T�����y������`Y6pm�KT�}��������~��[���P�Q�ch�����w����L�\��e:bw)J1E���������=A���2r�������c���:�A���2�M���<��j;]]��'e"j����������)B��F�0O��I�������$�K'{|~&m�y�_�7��m���@�;8��z����*�����(I�[Sr=R"dC���
!�>�Q�o�C-S������8$)*����t���Mk����p����z����E�����O.�	�e�Lr�zB���!AaAaAaAaA�;����:N(��o�j�o��&w�3t6�����"R7��
�%�RRRRR�R;�s������r��&���R�!�,�,�,�,�|/���9O)�X��}T�]]]��,��lw���tr��+ltqqqqq� { �s�����ve�
�
�
�
��A��@o�y�I�|�FN�jA���%����6��m`�9j�Z���Kj6C���~t�e�L�$@;D �nV���:��Y�M�
��NF���r���*l���i~&�:8P��o��]�WJ�*�Oj���^��e��l�
�xy�9��[z�O������Z>3,&U���]c���/�����s������Y��)�=#����1ci[^�<��w��r<��0���N8:�����/ye'k�,y@Zm��tey�+����_a��d�gr���O�$�(�t�i��
��B��5�Oj���=LLLL�`�������)���M#/��|�����L�16���o�?��������-������s��o��0���/�y�N�s8�_��������/����H�tE>�6�HN?�����go{��~l�r'��������D�oK������~i�&����^��
W�u�K3�%z�N�����[
@����o�����O��{�C/R�0�a:�1��ht���?���1d6�)�}�v7aN�kG^B�����u�������������)��d�����t���<���l���g�w���������@�/�<fw��f���b/��N��}��gF���{!e��������{ad:3$�<��3	���:���rI�`W4�V����N4>%���w��(�S��A�Lt�f0J���`K��ZZE{4�$@�<4y��������q�7
f�4l����� [m��Q���{1�PkY���G��wy2
����-�,9�A�W�tk�a�&���zy��-��,������K�V���+mgI���
�Q<n�K<�R�`�t�E��}�����Ui����H�Mm7�K�z���^�mhr�Y�~�����y���|���N/.�ON/�?�}�4xwx~y� �O5~��W������P��'y�F�[
�,�c����6��y0���|��Pt�������&M����Z3����+=77yUC�j�5@l����+t���hwk��j��t|LdN�p�8���Ib��G@��5������U��8�R�����<��`E�q�p���x�������H���*���U��5EnF�����l�����`���9��7H�<)���	��,g�?���k�N0����FjK�0��X9�����_\�vM����g�Dg�[R�-n��W)����$��~�j���k����6���S��Z�8�B���:�>U�t��?]sd�-.Q��l�9@k�q��4��"b�����R��
}E�	����������8}�.�!�G-�NNa&u ��J����ng�i�l@@b`�"h�������x8�X�e�hw�F� ���n}�2_����LB�&���V��n��.���;Y�t��_�t��JOY�S9���3/L����+,L���W�z���*��4�)+�y3	�����BOs�$��i�a��h0K�+T8�!�	��b�+�|��s1�#����
�
a4������A*@:j!k��4U5�	X��r�(��������V��_������D����4��6P>�*��]|��bx����{�|3M��k1��M����P�7@!�X���`GK�k�'���wlw�U�=p�2T5�RL�P����_6mH��)��� G�{A�t��bN�;C�C�p�,�l����d�VhJX��dnB.[8�E�����I��t��n_G��h���F�u��&����"�����E���r�E����*1R�=����
�x���������������������3����sl|�a)��)uz�V0Z0Z0Z0Z0�1z�l;����Jt0L'�mC��}2��yFyQ��`N�0L�
��A$���>~e�r��.������R�,G=�yM�R�
F�K���3Cue^2�$N��d���G$��S���0��[�����	"$DH�����K��t/����H�t<�A-�2>h�������G�4i��+��/��������m��d�n�$����L�q}�������p~cCg�L{���g�����Y�c�q�vt��x�����������rp��������W[V�2�f�����g05PyI;���A:Z����-T � i������k���6��$�e�=\P�"�fm��B�B*�T�R������j�6;������F%6���ppppp~pp&�2�H����&B�N����VV�h�h�h�h����2�e�����0:���������/���U�9�G��������������8�"����C�����}oKm��g:���������w]q���p����0�����	�wc�	��G����~c�c��1��X�S����F�hv��p��K�0Q�I�O��2'�0�H
��(��a	���.��E-��3����1L_��k2�D�c�+Ig��|�(
?8�y�{� G��{p�>E~���mT�=�L�6Bm���j���:���#}(���jA\��&��B�h�h�h���h�]�A/1�}�;=vyQt�Q��d���]:�vNp�>�9�<=0�;/��hq	��]�G���)7T����9�F?v^��HH��!B"Ef��w��s�o����q��k��P���Z�����j'a�z�&�r5L�i�,�'ea
��-sb_uiN����B �@�!yW�1�_��X���+ub����+�Mq�%�,�,�,��������5�"#F�����
���	�����.#5q��&�����R\��a�"~����}@���0��(ji�K��M8����������p��x#��@�6�F�LS����X�#�hma��l�h�h�h��_F���_�N3�=�j]��L����������S�*6��I�������Q��3*6�	RR���h
��J]NB^k�<���M���Y856�Wc&^��B�\�%��B&!d�:�L�'chg
xd4j��~��E������P�^a��2�D����l?
�.iB&HX_�T]`�
7
�1Am�g�J(8>ob#���|q5L�����r��2*����$��U�����@ei��&ff4~5E�������	��'�O���oc��x�Hg���0�����x�"�2�!E�@�DYrrrrrou��^1zQ_�e;��
�
�
�
����E���YlT�r`U�����H������������
8�*��%�l0Xx�|���$�����D�m/���*�l��$�CG)}��M�4*���>����y���!�<?WXR��4&t��+����)
��X��g�\|o��7 �� �����G_��C���
!B6�l��l8}������iz�
�
�
��V`���^E�=T��B��������=t#b������Q Slk���YHJN�\;E��(����K]��/�������L�3Z�����m�)�3�h���Y7$��Dg�;��(c�09z�EU�z	�	���������J��=��c/�����pb����>z���������{!@B��	���
+ig�~�G�����Z�j![$���d�`���U����}5,r$\��8��#[���������$��1���i�R#D�H )���P'�����n���z��N�=�Z�&�������x���K��c�Bb��P�w�����_}����M�"4Eh�������/.�=)�I�.����}�|�j��K��7:�	p��D�D�xw��xY��[����W���*��*R����(BYG+�!99�aI��<z�8Ru�O�@����� $CH��!���!����}*�\ [!�V�V�V�V�����"���3�L=�~KKKKKo��5G-�-��l�`V8VAYAYAYA�;r��_V`T`T`T`T`�.0�,pKk�V����s#++++{�%0��s�^��j{KM����!��~�B���EM@�V�V�V���xP�RWB��0XD]�(�+�+�+�+�����NG^Fs�[�[�[�[���[?L(2�����������C���%�����z�4��
�
�
�
�
��qr�C/�}��B�_%��_����TTTT��+��J��
�_C���^{Q�1E�WAZAZAZA�[��u�!�������$�+�+�+�+�{W�uj����.H+H+H+H+H{G������ X+X+X+X+X{'�-�jX;�����Q�?��~ �+�+�+�{K�����2����-��d�Y�Y�Y�Y��[�S?b@��\�\��"��Bc�`�����@^�t�t���66O�wDW�������o�	3�����Y���<�����N��������S�]��J��0[K��������e���2oekI�������������IE�e)IPVPVPVPVP������yQD�p��Y�$(+(+(+(+({7��xf����F�n��e�������v+`+`+`+`�I`;���v��M������4��S�k�����B-MpVpVpVpVp�v8{h���^'�
+�*�*�*�zle2�Wm��:�'d@F�a2V��YL�`
�c"���U8R^2���dH_z
�������������W�s\ZZZ�.>���OF*����a���gZ�^�
_r��P����
�L�9|�A�I���F�][���1��Fk�y���������cD�Z���^WWWW�;9}������+Q'uuuuu�,m0
3#�+�+�+�+���r��+�*�*�*�z��,?�K���fc#���BO�_1��#P+P+P+P{;�u(��r����W���������������-��}.h+h+h+h+h{K�vQJ�k���b�����R W W W W �N��q���#���%��]�g�Dg����
���hN��~�e�L�$@�]��zc�6j8w��s��B��$4,�xf����%�����~�<T�.�|���U���$����\z�f���x�a�at!��Q�������X�F����9c����~R�X&��$
���n��������@y�IU�)���we�	3oi�S�$���DJ��:�A%E<�8������1=���cM�d���+�/��&�&6�P3���T�����$�����Vj��5����*f��w
��[��qx��y�����4��2`��h��Rn�Z]����I��f������P����l9gM��N%��Y,>������=h�|���:o(wr	7��e�/�r�>�����&������O�6*h���@���n�X���5uX��>��?���o�������;�t����F��N�?�(z��-��W����x�n��_���6n��j���l'��j����@��0����<��u��s�}���.
��l�"B�����������/�����T
���H���*�`�0X��xY0��`U��m��Z���y���C��U��Mz[��rCOV���2kJ�T�b�`���_A��b���
#����m
qT�~��i�`�{a�FYSJ�#��.)�%�P�����k/K��|s�q��@�@Q�����^�fh14�fa��^���F���'M

u���j��ot~�_���7Cm���\*�R)�J�T���`)���}�(�������������m���ds���r�p1~!�����������|9�x�X���N�@�^����&����>�m_=�`b����������n�~�~�~�~���~���[A��[F���l��(4�*=s��<�F:�.y����Ea5<PC�k@�����i��~�Ki��N����o��_Bn�M��~������>b����E��0A��0�����-�+�+�+�+�{;q5!.��g�y����������t���

�������T��X!�V�a��*�*�*�*�*�zp���h`����bATATATATA���a�w���V�h������Zyh����U/D\[	�eK��t���HU�Eks���Y��h������"M�dL�	�L)VXi��F�m8!�,-��Y���>U�W������66����[;{j���P3#+���:U�7��e�wA��*$TH��P������X����u���o��b(�+�+�+�{�o,�~�
��/�T�T�T�T���Aj��� -�������:4�������
�8��0�29�0x�B�&��'�M��.�����}�c7�Y��|���
�����~��]��T������$f/_�����g'�D�!�C�!w����4c_���c�+r�:��x���$�t%/E��T'M��Xj`K��0@]h��D�"�-�-�-�}���E���k����K��;��Er����Krr
����u����(5����-������������
�N�Q����}iT	������c��wEE������a�_RRRRRo����FGXj]�U����*���\W��C��C����� ��9t�{x�A���	��@�"��=��c�9����C����c��pppp������z�yC
)������������F�o�W�9=���[V��J��\���o��o�~r�j�����������oo�EN^^^��
��s����ubD�7:�pAO����ut�j���k���EZ�T�(�����`v�C���^�6�
��?nZE��.�$���,��� dA�����$PN�|||||�Yro[�Q����-����~?3N ���������k/a�KC�n�0��o����r��]I�X]^^^������o5��?������X�{�|W�%�����Y�{���{st�WV�����R_�=�]K0��������������L�2!d��0����h>@r����.�,�,�,�����D{Sk>�=���@�@�@�@�����0��T��q�Pp� ��,����Z����4JQ)1�^��@��4����r/J�fW�C���������V^����:z�1T���<���L��5���!���M5B�X�&i��
���R��G0����^��y�"�����Fo{�VrJ:����$���R����G�j�j�j�j���	�EX���.j�7�:�y�k�P���A3�N=5�uR��.���:����l���w+�^����q�g�����%^��<���x�|7�|����Q�o<m����W) -��<�b���O��L��"��y�i����Ga�w�t������XU��a�����C
�1,�����ho~jr��������hx����#(7���7��F�fe���W%�C��$+__a�7��.���$��Nm3��_�)DH��\!�Br��>h��
"�VV���4�)�n�ks�-�9����������������<�nnnn?$n�PL<z���K���40������J3f��3�X��X�g�����Z�a���JdR�R���/����md��i7O���H���
.m��)���������}w7��}��>�=��J�gv����~�,��6����d��3��ID�F����$��OU������g��ObK����_
�*�.QP,����Y��.���J�jA������BJtj5�#����zv~5~~x9������a�������m�(xq�������?��5m�/������*}K��2gS��a��e�n�k&�R�=�f��C�]���n(�,7�jYR�/���
�t�U�����T�U�X�K�SS�)<L:Z�]��D��]W��Wy�~r#3����fmr����~�7UN�sh�W:*�/����*����h�e�[x����
[�Y�g1�b����-�!���[�-��S�����<'�5K'����n_o?��%A�?5.���JQ>\��g6n�=,���oSK��E�$f�R���������k�J,0y6�����/�UI���7!��Eo5<>����*O��W�[���i���!��SK\E������\g�3�?c�����Ty~;���P�.0f
�u�T�vxx����0�7�26o$/�������N��v�!O*��/�F��#��$�=J:��Gt�OL���)����{?�[�aKQA�L��?�#s��j=������
�K��m��BvN� �R��VR
��0��j�H�x#Y�d�-�����-� 1p�-Z���|/�#�|��sY�e:�.��#d��X��$
��
����a{7����ea�N:8-�K���	cg�?U2��H�,���U����3x
Q�����,��\]3���d��r����'7)�K��#����y����"���YBye�1>F�W:��u�U��.�\�9i�,���\�;�� AP��	�5�Z�d��@&�-�����{V���3�Y�}��p��� x��fZ�O��>����y� �����F�~(�o�>�]�|��&��6��wQ�W������i�u)����������.�z���)M�?��0���E������l����j4��blX����7�_C��+:��?��P/�y��+A+f+kd~��f���JR|�������u�i�j��������j&U�S�����[�s��Pd
[��<��%��xJ�NM��k���>��:m�`������	g�0Qo�@<�{��Z$��_?N����`l3�\�3=�0]/a�����eS��)VJ�W���CI��?���n>�*K�^��1PX'����{�>�U:VI�6W������'�O�g�UVI&-h��W���7=��QR��;��@����bxx���<����ngz�;�1�.������r�a3�XD��7�B��f�A,T��i���T�66���9���J*�"f����]�o�����UQdn�g���N	��E���6�/��T'X4<���oYnj�i���jJ��^`D��1�0f_���d9Sn@;���|�;_>L���v�7����}��Y_f��j6����m���|���� Z0c����U�@���|�w0`��\��A8�,u�|q�����&0DP��|��ve[������
�/�\�7P��C������
�E��Q�c������^a�E����Hj�'Q��Y���+�;0���B��cXNS!o���E�:K=S>�3�:��v����VZ���7��	�?����Tk+����c:�Hx�~��d/�Vj���Y_��u0�K~nP~=�`�*s�m�����(��0�9�� T��.�`S��xk���Z7�jB�#G��`�/�O:���"�|~x�+��7V��z����ZU@�H_=N����&~��`4l����}�z��D�`�����&O����DM}���$��-`s�9!������9��L��^} ���OP�����������q��s�.��V�M�Z�b8	=�F[��i��R��<�n!��IWS�!�jc��N=��>7:y|�,�V/�����`�B2J����5����$�UP���N_�]�g�������W�0������o�iS8�r���'"�+e�9qx5�/�6my'\�Z{Q�`�7D�},��}����(�!���)��k0����v?j����0BC'R�������(i
����k�+������
M8q�&�
����*T�.�Zij�dMFp�PaK�
�T#hc�6i��A�2�]\("K`0��8U �;c�{ �q<U�~w6
��a
����vt6qj��)��`�'����&fn��+�a��������������Yd�N���2��/�0��A��Z"!H=�{���^>��+��Z`�f�V���JF>w�`�X���5��P�5�~b|�!<(v�
����������e��O�F��;Kx�����p�?8��},TKh�P�M�z*�h��G13��G��#~z0o�9�u�����L��4����'j��A����J���ab��[
���5���8��=����nZ8�5����
������E�}�J����_���\���������3�����\��������������q���������BT�p���';��r��0����;
M�/�=�����o��9������}���K9��cWi�N�|��+az�i�j�+�X:Q	)��+b�Rl!h�V�(�v��k�����+�����{$-�i�����pWi����Q����������]�����hm���G[����\�N��I4�~�*���~oolt�]��qD�r�2N.jjE����L4�i���\��A(�9_WT]�k���pk�~�C����"[��!��g�~	���0�����!p1.f��a�o����<�s�D����;�[T�q*U�{�g����Z��,�������b��a��lR�5��:WS �������]�2�-1�{��Yx�Z�/�KC R��>F_������:"���t���J����_x�{[��a�������F.������0�YB��*d+�u�0���(a^�h�{q��{
�B.��;�5D��_7��-/��8�s;z�d���o�t���n�4TtOK�����{����i����@��u�n�JC�
�0�Po��D��qG@J�.�'��������&�������)�t�������$����^��8?����_�mxq��Q
^�<k�g�(�`^-`RO������$���'x0��ZHE����������p��lWo'	���pF�d�<����������q.����D�������|�UW�J�N��;��	o�����}�������F��Mv�z�D�
���W������pc���D;���R������_��=����
��MM�kD��P�`
�������7�*�km�����]�nMy^�ZSR<��F�������}*�E��M��������"}|K�@_J�����VU�&�X��>���j�J�IR�DoD�#�3�7����
����6���5U��_�`�����%7���Z�O��4<d������
������j���R�/x�_�oT�Q5_���"��m��8��8u8���^k^����}���7/�����a�.t�$U}����|XP`6�^MI6|������.���G�&����Q!�mU��k�}��R���u�����P����
�c����L�gfL��PQ��Sj�q'�OE�Bjm�l���A�����wx1$\C�s���!<w�L���%?�������:2�S��n�o%��^km���{��7�7����|("��p��ket5o�<��m��4	H\�F�*_�g���oV�!d�/���iZ��F;�}����c��X����N�b�
�1?X�����������3�-8���F�8���~I�v%��oQ+%��"���GPq1��V��8���V
�����
$8>Q]
�u���=�{�*�u��������������:�"_���Xc6�������6�H�G?���k���>hvJ<X�~��6��������O	��1��>�����2��A���l����"�5�y��O��S�dx�_H���k��9*�>�r���0]�j���R=
��^UcD�>=>�g��7+v!�nn�IM���n��g�����BA?|;�����bo�[��)���R���8<x<Y�����������Ms*A#�${�&���f�}�3~w(����1�\�Y��6LG�&#��������y}:���qz�$�����fG�S�a��P����L��')����;8
*Z������kk�����_���6�X�
#:����;����;������\9	sh��?~�OZBu"S�l�Z]Km�.��I@����*�}���F�i�Np�j���iv�������f�<���6�V���dqKY��-���d&����O[��]Z�����VKq]�0�4�j%������0����)i��y���:�t;J�����3{q���Li��-i��g�Vij-�i���i����������xA��\��w��n�����Fo�]��K������*;I�����U�����8��A<��T�L�-��k���`@0������*YF��Tv�����Q�R)>������Q���B���vZ�������nQ[V�I�F����!G���+������vFy>�������.�����H�aPO7���?��?dKo�CI~98u�YFk��:@�&m���#u�������]��x�t*�]�=�����������0@���
��������f���Ey=�A��u[|\2 %v*8���X��<~�#��R�5�N�����e�}������@���{�Gh.�o���m+fbUy�Rodm�q��su����i�7b�o@���]1��Y{am}v�Y�B�����]>
���ZdK�u{���S����r)�\���z�����~�?������o�}7��;Jx���j���h��������U�W����z��x�l^��<��A��N�;w�<����%����L���7��[w6�����;k����L�����o.���W�������Bm����a#~9{H�(���y���C&����j<�>�b��R5�j��Q"f|����S��Ld{{���63�{�9����/����@&�~�������_A���:-���"���"=���5w��7����Z�06���i��4tHZ���g���3c���S�J�:�V���G�4Q�0�G���1e�����"h�a�����j�!cq�}�������I�j[S,���:0�2����U��%9>���W����}�KPn~�D��;��c�eX3.��Do���Ky��W���h�Cz}��#uq���R���������6������7��ou������e
������k���{��y����R1xB����������_O�|(4�	�G�YAw��-M��F��iS����e�Z{S�v����?�^��*���x�)o2�M���t��Li�	�g�b�D�����.z�����E���oM�x�A�0�=���v_����|���Sst�D����T���$�M}����b��`��"�E�6ENli-�bt�����z�7�'�����}}x�������Wg��W���},	��!\92��O]i�������� z���^��MI��>��g�"�E����2\~{x��71�a�E��q�],Uw#��Z���4��:��jA!���
�pB>-�*V&E�����dv
^��VZ��g�f���^����@���/@���C!<-��{D�G�o[�c�9�t�pw.�I[L��P��^r���%�\fKhf�L2�#�:W�C��r�}�(�T����A���V!�=u������v��b�}�a[�w���]>�$���J��Wdp[B���$iS;m
r����wo�L7��%C��K�'�]-�k�
T6�0���u�D������	��Y�[�0�����)����.-x2w�f�a&�3�e<����G~������JX2����Q�g�1�;j������YO+ T���S��M��2��B D��LV�pQp/D�+�q�W\a�
����\������[G�wQ�������1~����!��8���a�K�8���3�Kd@�bF�%�t�>"P���}�����>E��pU��J�g������,�,��Jo>�$�Y�z6e��=�!��R5+e��/y��v��"M���1�m�&���A|�&���4�q�wz��5�F�
��AJps�����e�UUj@XR��������������u�}�s�����E��(�
S�~���NS1^cX����L�m��
3d�f�B�u:���(����J;����Y��5�?!�����p��Yg=����Z�W��A��,g3��a���!�`��s4IL���:�v;������&��u�^�D
�<f3�a^����cN��lc�1���G,��^~O��5~}g7���f�"����lf6oU�4��Y�z�)(L��� ��,k��
���I�!��fH3��[�4���tur�^k']spR*yM?�|O�*%|d
�J+fc�n�a�4UV��rY�:y�k���b7XagY���e�����z�3b��n^'r��Hf$3����m#��9�^m�����|f>3����h'q�?���Sm}J��Dc�P����!�{("�q�p )����R\g�w�j��
�����e������>����g]Y%�*qs�Q�e�|���[�\��w�����W��u��	i+�C;;_����6�mt�{��j��<XT��bQ���S�>{���(���w���A������fS��XS>�GQ�m*2Z��x]q�<����g�3����e~��S��u�����<f3���������������]a�:������Hn��t!��U�	�a��IA�7�(�jy�
�$�q���(�����d!5EI���@
z���bzy��Qp4��_�3[~
-����N�(t������y�R�7@{R,�r�	]�q�n�J�Yu�q���&W��	�>�|j�J��drQP�:�����v���-�����%��'<b������j���a]=�1eT�������/��oc�������1�M��I���V$e�����>)��Oh��m�>���L�0�������1K�����Gk],�z�P�/�^47�uX�|{��{=�����p�)3|�����
sn�����p��ic=(.k����g�t���������:�_��1�o�,��
����c�1�~_(�,1����8}: ��y��c�})�{����
�f��FM���eC���0wnl���'G�]�?�'P;<���YP	���hO���NF'���@'�0m����~�S�,`F�N��\f.3�����r���E���;������R�9�Y�9f���.�����V�6z���V�Zk�y�
�w�2~�
�7�2d���3tW����\J��.,4�Z�������������oe�^����+O��q+1�=.���F��g��#��������0�����&�G�aI���%K��_Vc�0w���]�.sw�>�����2o)���H��
�~r�v�\9yG�����;u�2�=��q��g�o���8��������h�M�2z�~�T���&��K��c��	s���h��o����
�(�`���<��:_>?�����ud��L��G��n���:l�A���9*�{+��c���t>��A?�ka�)���7|�4z�-��u~�R�������w,!YB��d	�r�K&;�vw�
J��.pO��=Is� �(HyH�D��
V����Ag;��6�

�O�{g��(5N���Q�F�r���$~V�����T��%Et�+,�U~����U$����).�TE��.�?%@��$WQj8����y7�)24\T�T$&�9��:Nm�rs�{���t�dU�R,)�Z��e�R<��o�����!���%nq�����jv����pI�N����C�pU�����Q��W�Q^;,'W��XYj�8����-x#Tj���h�:(,�1jg�������6@�eb�����X&�Ld�����_v6���D��c�1��q���p����������%4�E]��r>����0�i��C4��;V���C����C�x����q171i�'�"mx�l����9��=��w�2��/�q���Z�"#��o���m�����c�1�����;�W�x�h��b��l���RZ��� ����C@c����)��`�����������LF&#���9"�����L�"������J����B�d.lXs���\Iomt���u��&of��W���:%;%��g��\F.#��@.*�-n���|c�1��o�#�H�\�������h��c:�P����IY�c2�_{�����>4!2��q�8f���ww�����9��c�}����?-��A���@����c�1��r_����T�*sNq��c�1��0��[	�3��Gz�T�	���)U�z��o��%&j.o4\��&����g�S�8'��z(��M��I�Z���N�.\�������9��EO�^,=���C��xn�0$�o��k�X*�.�]���X�~����16��rjN����	|���������p��d���E�RI��F�[����	s��RbX���.�+�����n$TVb���oGG���rxME��nC�%1�7*�;e�)*$N�u�_�Qo�B-
������b�Z��b:a����$27���?��������ei)[F2���'��#�R���_J�Vl�d)b�C�����Tf���>9}��P��H�i�)����G�3:��k���4�����Q+�U5����,�!��
����d��R��]�U�a�Dl�L���:cQ�\�/$��
�?.����QQL��mH��'<8V����I�[,|L����~7������3U����K��jE���<����N�(�����V�.zx(,U�f�Z`(*,��{�[/4#������|�g�;���������]c��������7�����r��!+�[]���RA�"*�M~T�;y�����a�zb@�HIR���=��n�(��px|�8s�9��&��1�n�G��:j<����_�/����M�n0D�(���8��7u�t�"����7X��4��Z��N��h�$���SL/����;�{-�+*}4;�P`��B����	�����W�	�1���� ���a�0s�9�E�7%������e�2w����H�>9E^-��\%��{�=����Q�&��cJb!�)UE-rjRE���p�N���z����dZ�Jsi����T�����Z*7D?�Ee](!���>
��?J�PQ�<�����E�u?I�8�h���G���
:�]�yC�V�qQ��W��4������}���^�������s�������
�S>���S��\���E��g��6��-��J$�����Y��B��
����	�@ut&�l�W�o��E;�H2,�z���\���zF������
�0���������6V_Q�3�f���Fjr�T��P��[�p��������v��������%��N�*���m5��R��6��-f���~�+)9P�S����(~��<��������{w�V�l��A�E��X�� ���������<�����x��z����]��F�8�p�p�;�KO��>"W=��k�T�P<����;r_�S,���U8D
�D�������NJs��v8h
aY
FK�;�G�Z���3U	���H��?��O`������\���j���X����C]�i���~���[(��������d�#��8~�U
D{w�@��S:g7�������?�\��/��SctT���'ub��Z�`=�<]���;����?����� ���=�J+g8��[��a����Iz�B��F������y61Y��]��1
����	������c*c*�E���#��i[���SI��QR�%rP�g.�JI�5�{���G��?5��9��D%Nq�l����Z�O���\[K%���X��__^�]��g����x����C-�&S[�����,��$�j>M�}��R�=�W1����o�������j��
�"a�	��VD'��y���H�8�y���<�y ��n�Pf���y��e�n��������n@�`?���x�5v������~���~�Fm��5r�,!XB������@�{���fF3����h�N��T��R��f@3����FLX��X1a�xE0����df23y�J�}�^l�`3�����OP����La�0S�)���F��*��e�2s�����1ws�
n�N0l�W�Wo��18|s�H�T�p��
NN2���/cI��>j���_X�MJ=��K�����
0OQ���Y�I��3#��;�8tiWT:9��h�^�S�;=]���r����r�!�hC)��������Mn�����'T|���jX�z�O�=�BY���cq��n�����lt�����a
�����Tf*3��o~G��,C�Rt���{�q�3��w�3��i��'�����]�}������_�/����E����3��z�c���"����1p��Q�&��;��[���x��J3�7wU�G�k���U4�k�$�E����������e2�\�#L���yux5�����C�u���B�j����(�$�B��Tff�����&�����1����R�%�2��5���C/z�X,54
��nl�@�]]^]�9����vZa^b�
����R.}�`]���n���D��x~�T��e�J
���MX3���/��z�n�f)�y�z����hW/a`�ZqkJ7�4�$��	��_���gM������G��Y���~�O�pz�Ov,�t�P��*�xt�~���*e����
3c���E�:=�����XRo��n��1�v���t���P������m�:���cv�2���zE���a;��~:h�����&l��?�T��EM&�XR����,����7�W:�De��Ny1:����.�u��]�p��T_����_m���NK�b�t9�M���B���HY#e��5�-j���^d8S����e�2u�H��]��1So)6;L ��Vl������)��+��������+��(a��_���)��Po�����{:'�7����SC$XLa��> :������*��m��8�vt�]74��?z�}�=��[����0�It�+zX�l
�S����l���f ��"���<�C�B����O��:�I���C]7�*K-�w>�81&�y�;��g��e��P�,3Yf��d��2s��O|�ul�r5�F�+�p��A�CPN��-��~�
�����`���V<\�����Inn���O$=(���)���#JQo	n���zhA�����������h|1:A���X��xa���e��0\O��N3Y����,�*G��hE��5��!��M����<<���r����OL�/��0M����&K�1����qT%J<���"gS�Q�18#�z�R"������M����NCi�d�@����|J(�q�4�aN5L�nT������!O�_���]p3u�J�$��y�k���Xl��O�
�� �<�:
�l�Bu�Z�F����R�)Yf'o���n�h)���^�����-`�lRJtGR��hy)S����'���CX��e1�b���'����
s������L��i�3o�yZ��f^3�����u3qxK�[���������La�0S�)�E
g�@�����0�6Gh=�Q&1��I�$fos���r)s1x�0�6��L��+^&���W�=7&��&��3�]!����
[#���k]�iUz�/e��}JW��t�-�n�{���Xm	��|����������������N�\����^����Pa��B��
�m
��#��������J?����|f>o���Ae{��btrv~5>��vt1~=z���c�53���Le����,D�_�=�K{��U��J�d9S�Uc�0C�!���=�n���������j_����B�V�Q�Q������S���	��Y�,g��-PGs�\���/"1�+e�g��[�~P"��J���Q9)n`��yh�kn�=�XTW(���c�	�R�����d�Dea�j#�������#�����-	���\%�ZY.�� *����v�L���y�g�v`�H�{����8�������*��a�p��o<���f���Z����Ik��9C	K8�p,�X�m�R��O���
�&P�0c�1��&�?f(E�`�~�'��������e�2r??���h=�:;��6����q����9�h�����e�2u���:������Py�_;7\�/��_�/�������O���q!o{7"������(��f\3���+wv����3��^dR�M�t�������^:t%t��<WD��vJ�����:�3���s�9��S����;@��c�1�x��j�����x�\B������RY��������01�8��gW�E<����Zr�����Q����~������VQ<o����-O��J~#�T���9��y�|*;�E�)����s�m!,BX���$"w��K�Te�r���r����q�2����Ai����m��}mE�'����vL����`�����1�Q�H��4���{N/�P�����x����K�'*��Q��K���T:���R�Y��|pM���@hJSL�3�;�����l���G;���6�"x�
K2�d,�X�m[�u6��LA���f3���0n�A�nJ�,?�������a�0f3��c�y����Y1f3�����-�8n����;;rZ�����Z2����df2o������yw���������l���33�Y�h]����ha�J�}b������2��J'L��}�V,d7� 2`����K���TY*�Rg��
������B����ie��1����SCZ�]�)*�Jz6w\p�<���h�3���3-�B.�B�,y������g������P�h157y���&��!��Uc��tgxG|t��B������7buD�}�{�X�(fQ���L��{���[��`!�!v@w7�(X�l�'*�_"��8<K�,X
|")������?�t���������:a�q<�O	)e]�4q�10:LH�[���j�EM��K����4�z3;���}�LUI�1���UI�LI�:[�D�l��;�
�V��h�&o��:�T�G?����S���*G_�8I�V��I�T����i��z��M�`��[9z���'}8�T4'Eg��4E��zW<��gUZ��'X�8N=.���L_�fcy��ij(o���7x�:�[s�n0	8LV���T0��&�I�~��z�jY���Y\�@m��S[�J{M�f�F��U43��h���?��~�7����w/3�L�{�-\}&<��e��cD�[EU����@���S���"}�1.f�z�����y�M����9�Z�t#���: �������2m����Kh[nn�����eQ�A���oz{^
����ucI��]���l��VEaJ�;��c�C����&?U
[:l9gcI����3��U��F ??��+7�T�T���j����L�T%~U����K81/�#�o��#��GN����x���^UE�qS�K�&?���.��|1�vN���;���?�d��2�}�=G*;+b$X����Y�b�u(��>?����������	�	��c�1�>?����T���`��c�&�#C�!��c�}	����L]M�Q����S����d��S��>1pH�m�}��~L?���s�_����9�`��g4(��6�3�p�{Y�����O}�T���~����zh�L�+n/���=h�����c��[�a�:�=�:�`e�2X���.]Z�k�*���hWZ[-(���p�k�}6e�~�0�4��|+{5�A)�+Y�M��~;���9��Y)ev2;��_;wZ�#`/����;�U�Mk��~�#��d~	��3Q�O��OD��OOE=��~�;�73�}��/}�����)��)�4vU8$���&�� ��S�|�4d2
��_
[DT����M�i�u��3�A����:!S�)��B(��B\;��*��M��:��6f�5�^�����LJ%����<:od	W`���e|�M�&C���fKgv���[:�����uC-������Q)������wf+����l�r��C����#��x�?z[���I���q����p�[���0�m�H�H�m?�&��$3V�������dT2*�(TzC##���Hc�}�Hk�c�|���oF����������6��,�C��=��q7��q�iM�1����w�5I��}C���qWbZ���:���q���a��	���bdx�
i�����~:�2���psN�TJ��pmi?��h}������a-	s�E��	����?��:x��X���BX���|sz�'������L����'oF��CGW������G/�����.��.Fq��)���������O�����|4@;��?�8��r�]�-d��Cp�P�~~<{O��X<���
�bw��3BP7���>���P������s^��"�H��`8����b��Z�P�����<��/0���(��b���O%(}0h��~�*�?l����g��?v�u���*sqX���R��4e|����GgC���C�E�4�Q�����c����K�z����@t6��A�#a���6V�Xac���W��uc;k�������H�*9��D3>�q�Oa�����PE����j��Mp�d	��0�m,��N&����f/�}La,e(0Nq�
�K�o_��b�\�Q~A��Z:��^���s
��!�3H-����*�0�w���@�T�V���p��:%�������RD�: .]��P���+�BB�q,�X�����d������#�2����Z-��A����+1�Ie��Cw�f�2U��LU��=Tm��$��X;��#ij�����	n������y�i��3��a��a��������ml�ppCh%&+����de��CV�O	z,2�Z-�A<b����z+����te�~�����j�%�����5n�V��+����te�~]	��V/�,z���;��z�����w����e�2}[��!����:���}�PS�k���!��u�k��Q8���0���4l�Q�U+O�F�^+��J�`���Lv&;����d���N�\�;*�>����2U��LU�*S�n}������u�d;�����X���1����;���l����1�������)m�_��|���N�F�m�������4�SjMT�����6��~��It�h�Fm�l�5�.��z����������<x����QZ�4b�2R��TF��6g�O���Pf(3��71t5�_��������]6f.3����e������\�'�)c����e�2v��nN��J��B��*'�o�[����Y���6g��R4�S[�S�R:S�$��**��������J*�0����J��(M��

n�p��D��.���4YRx��uZ������*c��K�B�������	�`e����D!K�PP9�C�^��������%����x��x#�����p��(�a��\�����1c,�>;1��3g:�E��#X�rf���t^T���\�s��!.0��Y�P��(��`��~3����~��O��A��g�3����fF������U���i�@��^}���'��a��EU���t�@�T��<fBR+3�������Q��P�'��3���y��o����o�\Hdo�G��&�B�j�J��$����V�#�e�q�}?`��0�m�6c���Ul76YX��s�s�J7��f���5g����i}Q��������z�q��������3`�
��,X.lS.xJP��j�c���"��L<��4���a�0c����K��HP�;^;-�7������mo��M&����o�7����	�54�������%�3�����'�R��a���oI�x��/J��D)6�P���_-E2����%
��i�s���7zY�x��,����%��,X��d�xX���a�%��F<#����M!��]�������F6#������e�_�����+���}%���������X��ta�����%]6:�����bh3�����ko����*�?UjLa2�7�����v��\Q�3o���_bj�����Au�or����m|���������~vF$&O�*UpR:<�e`���{y�e��R��������:�'.o#���u(5p_n�H�*����
_[�W/*�J*J������9�A����o}�����3Z�.[���6cn2\��S{tZ�����t�<Y�CJz�p.n}�{�:�$hJ"3���\g��)&�.}[Y���4�����,��Q��f��M��vV_u�Beg�.R*
L�{���B���7V�(fQ���E�E�/;e�He�vWS8j���L�}�3R�v�6��df23���L�-���#rk5�O{��~�p��������L[�-����<��(zz���z��
���������!��<A#�Z���_;oGZ1"���%�,E�x��
�n�g��]_�7����eM�g��r��@�J\����	��	�s��7d��,i��S�����A�J;y��fCk�uT��5M���h~���f���*s:d������uW�P�r~�����;��/�?��������'}�8p,�X������f	 ���V<}*�M[���k��y�W<Mv�pI���>=>A��p����/����J����e����{�=��y�E���i/���v�J=�������a�o�y�����X-����������)�L���h���k*�o���L]M�Q��%��������K7�*���t3y�.l_�^�4����l"v�2��J��}"^�>Aqhm�P���J�Kh�
&US?U8�BYES��o������*���g%Ff���%/��h�6�����>�VC{D?t��i�8z�b||z9�����3�����<��|�c�[=$��
�_�����o�n�p�/�"��� �{�	���A�Jo�_^�>�JU�B���J����J/F'���R�2�!U
�}L�R5�p����J��R%f�����
c�����!�m� ��Ni���hx%����8���+�Tq�k�Uc����d����m�	�Sjx��i���+�����!fj�\�"C�syI���pr�)��c�6���6(�XJ+=���;�������*OK�8;~a�bi>�O�|�g�9E.��1:����6�k�e��X�cY�e=���mo�J^-H[<��&���z)t?`0�u\�g�ue����N��D�FG+�1���_�:'k������������������������G�gA�������F���+��W�ud@Mpf23���Lf&oU9'�<�����������}o@a:3���Lg�����C���
�y;7U��5��q�R��t��	�`�"_�I��?x*N�������\��~?����]�s�E��@�x�!�f3��[��g0���?I��Q&1��I�$fo��H_t3D�xR�,O�Up�e��V�1]����n �:���i��fZ��h��!��e�2d�[���B}5�����e��-E"s�St�f�0���8%��L0"�B:8lc��
��b ]�O��V���������q/F�����
5h����:w��������/��"���s��#
�2��(h���<)F�m�D�y�m�x*���F(g�6t� ������7��l����
�B��&M�,4�'4���S~G�5����`ot�3����|�$|�D2�0��z=�7�K��A�^�]�utQdT3���jF�Q�#.}�[:����N��{�_9S����fj�-��JwF@�����5C�!�f3���:_*Y&����);���E�%RO��
~e�k�1�[��X��
��W���rf9�|�,�x�]tH��%l�`3���L��j��N�F
z�������t�&"9&��o2��6���O�k�����<g�3�����-j�-w�5�g���(f3���Eqbr��jC���(��a������q'M�/D�e��J�B-�E���O�b*:[�%Bs��
,X(l5�`�������t:�'�^*�c6�0���bF��Q�YSNt�*�FU�>��f@3��h����bF1���F�&]T��d�M�>�pf83�����<�w9����c�1�>?��g��;Q����_y�����3��

q5h�����\t�����u�~Co�l��7S�f��N����g�3��[U|{��$����)���%9���,���Ls�9��i��h>1&�s����y��#�C��-���rF9��Q�(�&��Q��4�������\;�����P���K��T�����r���ii��H$�/�s��Lq�z��vn2%dY���QJ{��f���:�2��VX9U��&5��:OT���(S����T�-����H�2�)�ws;���������b}������n���JP��N��q?����/������2d[d�c�����c�c�1��q�8�d8�`�d2���/�Md���8c���Xf,3�?1����:��	3����`f0R0����X��e�2C�������|��(#���Hf$3�?5����u(�q�2c���Xf,o��{�0M�I(]�wfvF�{���|j�&������Q�������6L�rf�3�����b��I2x�^�'����G�;bWL*������T������j9Vo���Ex���2���`x��~�bjJ!�L��\`y����+��+P��U��oXXb�U:+d������^f�"����$A�OL	O)L�����,EnRL<v��48������\ywE'�P�Y��j���o�A���S*k��v���P{�������u�e8�(a�)������
�s�g1�>�����D�@!bww����w����q����i�l�%)KR��,I�(I����������z��+��W�#����|f>3�����s�����H�;�u��m/ �:<RYx��������@�wrj�;�7
�>��_���Px�T��99a�3���L~&�vx�>C�!�f�� ���j�7,�1�`3�w��2���Lg�3�y�t�t����J���Dm�0s�9�fo�T��3�$w�����hY*���~���bF1��Q�E�~���F��bf�zU�2�9W�V��*{�����`i���+��j5��A�EP��T�=�<����t�P��e�0S�)��"�{�/�����x0Y�
��g��:!Wf���(��6�t3���m�6c�nl��h�m�c<�c!���O�����w��g3��c���0��V����L`&0�	���^F0#��f����h��2�f3���[��;�_eV��=���}"!v��?�W"`��IF$#������4������b(��&�����0����Dv����������D�j7��'�>�V�S,�&?�����.-��6��Lh&4�	�["��\�3�e8����>�#(����P<��������df23���%f�h�zK�$3���Ld&������u4Z��s�W�I?��@9d2���U �������L^&/����=����_��?��U������O��lV��t*'�F�r���g<~��������
_�������4=�!@���?�����?��?��x�i�{�T��������s\6��s�1������G�{��?!�f=�������?U�\
S8��?�R��N���0��*�2Q���sS:�Fyd�e�gs'��@������
�`��"�*3(���Q�)(3���T����P\�\lc��7��qn��\�G2�SS�Z�:ai�/F�W�/�����N}�l�<���Z����i�=jw�����G;;���$�R%�/���a���2�����Z����H�n�$��GsW�T9�3*h��@e��l�I��l���2q��t23�G���[WJ�i~�UEi�;O�e��'�[�T����w�����Y�,�t�g��iz+�=��6��B��4��G�!��8��_p����@��������9c��������4USqv~u�����.�/F���Z��(�G�z�z�:���$��U+�l��Qu����������g����������U�u�T�]�l5�
3��u�C[�]'o��C��[8.a����s��+�=g�ru������(��Z�z�I_���.�zQ�|���o��x:`�H��2������[�0��x����r������j��^/�Jk�1%S�i))�x*^�^�9����<�_�9?�_^!Nv|�2��5��/p)��xyvq4�^���N�g/_P%�5���1Y5�zg�L9���J�Qrd?`��b�e<����}Dx��+�v�r1CI�s@T�����(@��K��IJ������h����Me�6��k�Z�)�s?S�`�����Z�7 �������w��/�~sx��\��O]^�]��'�'/�^�9=���������jt���a���O��.�����ds��_r����q��������38���O��=��_����dty�F`��6{��*4��^C�l�1(!���
�V�������&���n�f6)�1z��W��,��O�8���*�C |�U�0kH�'v	V���*
z��0Z3�6��u~�7�C����������BO)Wy�.�G$1���� Al�Y���]�E���� �a����J�T'�U��H+��n�i7|{YU�������1�<��[8A��V�o���x���������Lk�k������r ���RM���x[<[W�u#V$f^�/J��cg@���jBq=T��j�C������s��V�t�������{Q�u�#���r��!pba7F�mx�T�����9����P�����i�]�,\+a��5xnx$��N�U=����*��]I�<e��;��9}_m}��R����H�y�	\H������<���lw����.dy}o�B��"�Cg��g�T���!v�t
5+���Pb���4�WC��
����g"�.!a�Q�y��q�
X���r�#�}@���6���F������B�C�{����zy0.�cZ}��LA4w���}��_��>��M-��}e�^30}����f����S�c��T�X�cq��1k��4^/[/P3!��`��x�4V�s(�����[��	�P���&/���ue�����y����
�[�������:/*x�*��b��H(�WH���>�����F/���B�_�^�]w�Q����z�=��h�co*�NdN��s����z�[U������������t�}/�� ����iS����S%�(�*���4�v�gm�������}�L�i�=���/����k/]t���v�\���p�z���{:����>���S����o�r�-����{J�	w!�e/�HR���
-7�C�_��z�f�0������x2,���w��gt[�r��`��8JV�
�z�o����G�fJ=�GV�������E�5�����q|X�������q�O��w�k��c�&B�nP���&+�Y����!�?�i+������]L�>�w�)������J�?V6|��@�fS���PO�u��Z�	��D�I�����������^]U��^�[����[����L)�}m��K'&�&�����z��F!1Q����C,��ls�]HA�Yj*�����p��1MZ�aV�b9r��h\r�
�������WLBQ���R��rJ�R������dm{��'��)�X��e�����l�C���T���������H��F,M�u
�pXo����K����W�:=����i=[���%uN���;��?�v��M/�F_��uf"~����0�qr;�����4�'0�I���H��`�6�`�6[t��e�����?������$�8f3����-�����x�]���
��>����B�.em���0��z=�~��������e�2r���m�%Z1Q���l�f�3���I��e�G���lw��|��������:k/����Rex�ts����D
��� ����T������y��D[����������3_OtP�s����+�:�1�����3���]���vRI2)��B��2�nG{�l5��"�/��.;��tN��>�kb�|C*�\N�V�����*���TXR����"���JqEy�J&sQ��?�Jju�>x��������=�����&�,4Yh�������{����[�1���La�0Sx�
�oJ~*N�O��L^&/����I���&/�����e�~2��
���������t���_�/����u��j����/�����e�~Z���a�_�/������t���P��<�`0�����<Srz�f3���������!�f3���4k�G��N��L_�/���������;�`�2{���^f��3�V��[�{3y��L^&/�w����u����]�.s�����t��0��ey��e�2|��O��f2���L_�/������T�u���V9!�;��U`0���>�SU�<�g����`0���6��L��*k��\�9����S%��(<���*�I6�Leba�V��0����AQA���v"5��O�Ai�2�+1��5�~(�f=�!����Y&�1T<DO�B�K�'����Z(����P%y)�+^�9���V9*	�#�JY�f$5XC�k�]�&m��xh�����VM�<��P���T�1I�O>�I���^BY��b��@��
	��U_C�e��:�A�V�qI�����Z��::fJ��JTpv)&pyxS����������R'��&���5��)V�s!&��#o����H_T��$��\+�r
���]��T���v������*����To���J$^�}
]���C���OO��,��~M��	��1�Y<�Ih�)�:���f�`��fh;�P]u�r��V�Q?V�E��gaP������B}	|�9tw������$f��b��R[��8>���B�����r������is�c�mCU�t�'���/I�mX�V%�����4'
����Yk{����1E\��w���������������������dt2>�n <��W_�*���r���*���t����SU�� A�c�2y��L^&�v��.s�����a*��ja���-���������'����3���Lf&3����B�V�=��+d����������dtt� f3���-��A �\��������+��_�/����m�.��q\��%�q���L�Z�3^�c3����������<��"q��e�2{��������x��������`0��E�~�+$�����\�B8e������m"W}���8l[��Q}�V�,K��1o��h�,��A�Xg�3����z@sD�k������������8�8����9�<|9bX3��k��Va�n��2�je�^�bU�F|��<�m�����}�6�Nv;����`g�o�wi�J��f3����wM4�u�*����.)��*
�B�sU���\�3�2D�qe�DZ����tP����B�W������������
���0>2�mIE?���_����[`Itf�t����U��SS&�1�S��j��|�T�RY;�-�����Ea�b���I|��z���aa������A>8��e�b�e7��iG���.�O��;�1�O�ts��n<��"dcHz�T����b`�S�Bh��;}l������}
oT�;�tq���V[E�.C�:�������5�6��@h/FR��F����a=)P��TY�m��A�W�-�^O+�������a?Qb���`����^7������J�xg���$��KYi"�0e�Fs&'
*��u����LC
K��e�-idu���/���Ir��8�r�VeS1�jR�bL�;���P}�._���F�kd���7�!���gG�V���/y����Df"3��Id� ��a��gy0i��������/3����&��>P������������z��Lt&:}�D����L�sCV�S%����FKvU�Q�`�!�Y�9��I�rtu��/hZ�E��������:�
Cd�"��\�f�[�;��Z�R@�'c�-Y��RM��RL�u{���NFC��\�10�`8e2
�&���d�
�)4��S8�7�)�zsD���"+��J���������B�LA���������uOC����k,<Xx�����GWt�&��Va#���o���d�� k��:���J�^��t����i����N~F��j�-���Kp�^����F��x�7<��q��T�c��/piFX\��k�p',+�y�T<��6Ca�3�������
�����p��R70�Ot3��d~�|��� W7^�PSYea�E��,�C�
}Z:����+����u�:c}����(�BFg@�@�s:����/�R�
M>5��&�;���Kc@;^��K!��

X1�oNN��tf:3���L�����e(�������Re�_����K7'{�����"s����"��� �K���;����*0�2�#��������bt2~y|zx�Y�^�{�=��a�U�#��"�)��S�9<��������2�A��fP3����q� �:�����[��k�3���Lc�1�x�4�,��!�y����-�MJ�eC�v�^�k%lUR<�\V~��D&���L-F�H�d��0M���z��/r��)�_�4����q��{v�\]������
,X6�l��l 7����[�\VEaJg��_��Q��1�Q����g5��\��J��_�r����aO>���E�uD�%&�H=U�k�7��h�B�t���!��� @K�B�%���0.��M��1��B��!B,��-��,�nwC>��4	9l��IZ�����i�������g����6�E���O1�yNL������o{?�-'jeI���}���3�""�`��6��]��H�N������Z0��f���g}r�7??�_�=�8���l�Pn��2g7M��Sf��x���O(�y�����C$Tn��]����2��Yq���_�7���kb�������k�|u��aQD+f8+l�M�X7�ls�)R]x�b�e��x��Qs�.�eV��+@�>o��b��(ux�?��������^�')e�K���b1FjqT�3�$�Pi5������kn{	W7�|ou��3f��U2SIntAi���w�]bc�N_�7�N
�c�Ugu�5�F/0�������|������R?�_R��_��	����K�v��H��8�������~^�"��'P�G��4�����U�Z�9m��#{�;�w�v��d��/h���~��JW�<�bt8�`���G�50_4%Q,+E/mu;k2��7N�<�|Qy��h4�5:�:���{O�>�.�?p&S*�����A�-BBP8o�o��
|��	�����s�4p�`B�D��9�Q18,�W�h��u� O<:)����������r�����0�x�������,
�(���Z>v��~�J�87��� G�Q
�x���n��ZX-�V���{�JO,�7_�T��#�����e��@Y�,P�%�Q���(�  �9�#{�*�f��z��&��{�=���#��y�)O��%���t������)��
�@�����c��C?4r?��/�O�'��O�f�[G�������XP,(��!��-&�j��~��Ua[�7����S��srBT���jA�NQ���v9�������a���s�4�����5'����Bb!�c���En��J�,@ ���l��Z/4���B��[y%6dja�0Y�,L~L&?=PGGG\�ifT��f����4�U���g</+2�'��1$���^H/�����^R,����da���1��$X�	��z�s�'���?���P��n�^��[�	������1��*�y����������Y
�l��i�(�F�ja��ZX�[]�!|�"�s�K|�Z`��.���(8����������)W�iC���PX(,
���3�����+�>W���2�v�5t?�3!��Y�,d~T2��i6]�_��4zr��PY�,T*����.�����-r�IH,$����Ab_��e���q��_�_�d�Y��@N����(�,��@��*@ �����B�:�M��������5��	���fA���Q�����4�&�l�`�X,~�j�j��pX8,����jjc8��@Y�,P(?���}S��D���hA�c zc%8��J��Y�,p8��u����2�a��`X0,~�f�)���j[�[�
��d��Q�l�oty�znv9�_���ba��XX�(,���,m�����c���Xp,8~g���O��70n~���bA�c��������7I��V�$(����]�����|�g~E���Vy�&FU�����ji�<��y��;�Ub+�����h7R��l�(4����B�]��v����dbnx��Y�,D"����Gd�c#�����]rA�u��X�+��
s��;d����O7�g_(WM�5e�,L	��)uZ��;���([(]�~��;O*P���jfJ���TF'3n�=�>9;yy�"UYA
R������	�sT���s���� ��)M����U�����v�Y�(mZ%&U�?�V�����2���C�c�Md��I3�3T�b�Ot}j��������N�v�rso�!5Feeq4�����47�S��?+�7����7�[��W�����N|�����S>����_���e���;��������vg�Xh�W�:+����V�����~-�v_�W�e�������p�Jm��WS|@]�W�"_a�R��"14�0�4��N�����9�!,6���'{�M�lnF4Az��$��d4S�ft�����s���3$�/��^z�U���V��rg��0�5��rb��n�/-6�^m�Zj��#4Q�R�����V�����8��O�~�8�G�/Kin�k��)lj6�_�2�������{�Q��Z���5s=�>p	d~����S��F���S�*>��Ml��9�-�g��V�.��$����,����4r
&N��|�T�.G|C�g7�/�,����7c����������������H�'7T�fQir|2�2m���H�*<����MA�za�N��;�0fs�8�xe=R������/��?���[��@.�s�^����L%��Q�G?J�&<��O�"s�l8�8�[�t�=����1��f, GaV�F�D�-J�(��D���������8��k��8E����]�l�����N[�������y���Aa��@������b�������� k�G������U��6�S�L�q�r8��x��#�
�M~'/I���jw��<+U�pP�xb�������3���Z
��#RI��H%�J��JH`�m�L���
x�^���Kvk��
v��]�����r6�?�l��w����?S_�/��i��pZ8-��=��d�pZ ,����\c�c�DK��g_���W�+��=|-:e����L�+��
~������= ^���3����-�y/����Xh,4~gE���L�"���^a���Q��xC�s����W�+��>
}�U�3���8!�X,~d���I��3�^a��W�+�}4����)f����`�X,~,/L_)��
|��G�oi�oty'v!��W�+�}����?`S�h�>���|�\�c�k����R�,h4��^���o�s8,�@���G1�61��t��l,���	����\/$s��W�+��>^�FV��������;���W�+�}���:;����&����`A� X,�=�g��36�
m���:��rBa��PX(�H��I�
���a��`x���j���P�+��
u��;�n�~�����������v.�-�na�L�|�\�����L���K���p[�-�n?������d�����L���a��@x���{z�����T��)��������7
#9F*�v(����}e���E�&U���/��:0_<5u]M.�����J|mNm�2�G����mg(���E�$�zA��^P�C���]_���^l!}�
��pB8!���je���;R:���H���N�o�Q3=��;��z����	�������s���r���G_��)o���p��5�9\�}g��)un��������y�P�5N�����)�CW�j��sf�3Z���E���Z�-W��jbr�������.
�UG[��*��d�7����j�A��i��[���#�G"�v(������j1����-*wM������U���������_���_U��4s��j��m��/?od�u��
����z����X_�
�iU$>�E��Y�e�\��p��ex��TZ%xUK����g�3��K�H����CZ�qb����"2L��S3�)��qW����+��5J�Z�Q�������#l{"	E�$I(���_�����U�\��=�����%��2ib����D}\��:���m�H���
m	���Bw������N����=d���x{~zq�����#5���t{�@�s����G���R�*�@��9��4�WY	���W��X?��@��FOkA��[��n:4R�Y���V��U�����{S�.aK`<�4S�WgNV]q5���2�s�Zz�M�F��0�
&9X�\,L�iA"F9V"�DX�������h�j\�����P��W�����0�Ek��q��Q��]���d��H�.+R��)�!�����m���0�A��`q�Z������~d���A:�GJ����MS[RK�B�BP����oV�x7�����J�����CsE��QKh2;;�9�:>�'p#���A�;��%�c�e3}�?��o}�5��%���]��[��H�C���'O��&�8�@q����B`!�����@M����z��^A�������:�
��
&���7bk�
}��B�]����{{Wh2���'�/W���4�|������A�����iG�,l6�w�f 3���Q%6�q��?�w2E?�����2x������6X�{f���m����Z7b���S���Tp�o�SRiO��ooN�H�e�<�W3\Wk��TA� U���k���
G���Q�����m�k���bd�
z�����w[�E6�&q@Z�QDE�y�O����������:���41���]�����
�(Iq�GJ�3�0&�����!w!�nU��U5p��h�<����,�^�(>
H\����9M;%�i(�T]$��J�U�����3����Y����tA:�o7�x���{=�L`x��M����v_�?.�r/�YD����'��s������<��-��w���S��B
�&�t%��Pq�:�i�4f{�$GjR�pY��r�g��
5��4�R�d
X)�J��������.o��4��N�$#Q��3�AtR�;��[�'����z�:��x>��&�}��R�Z�'���@��pJ��� 3�o.�"p��1(bP���A��'��~�o�=���J�K37�	��l��9���������R
��������Z��<�I�)��)X�V	�5�����W����sf�B�������,�����Hb~:ss���S��p� ���\����v��e��l�[���?���"xD����#�g��G���.��7����V|Xca���_f�|3F10�vK"���3Uj_q�,�d�ra��\X����W��Qk���G����N�<���X/����u���|�lL����Z���}0�p��g�e@�/�V�A���?�-m��4���$��j��jL�>��
-jl���i����7rx%��l/./���A���?"~v(~��6Z�ha��65H�5L��g���Y���@	��R'������������'7!���V����S\'�	����fn������[�P���}�Z����x����
�U�[��Pd�a���v}�DBa��2?�z�f�su~z&�|���\`.0�%���/{4�Z8,�w�a������2����S(I0f��3_|������F�P���|���Z�����)�mV�s�l8Jw�F��N�2�p��D����#�g���frml���@X ,��h��Z�Z���g���y�|~����n
Y�[�~S��	<v�����G1p�������[8-�N�w����6$7;��]���4b�B���B�]��T������Jsxc	�0�~]��/~�,O9���&|�S[��7��-Q���L_8
\L��I�C^0�����M}l���Z:���9�!l��G��>����%����,r�l�}� �_ ��"DE����m�|��X~i���`������Bq��cQ|
������4�/F*!��Y�,d�%���d(h�{�B�����$7s��nG�?�^�7��{�lk�}M-���:�J�'g'o�^���w3S���;���6�����pq��k�������o�k�z�ST�����)V~K�+��PN��^c)h[�+�{h�J�Vu3�T.��h��4�/�����\�� Z�h	�%��\lD���h�z��^[B]^�����yD�sT�C����L3�]����CVz,@)*�]�.D����h���R�7�����J�����:'lt 6����fa�O�mg�u�.+R�t�%�%��R� O���c����4<Q=�����������\�h!��ZH-��)�Q�nXL����b���_�����}����
�yM���!�?�7��4�o~K������N���\���w���&�o����~�����?��9!rB������	���N�V����'^-u�q�2�T���)�2�����r��*�?q�����H��I���A33�QQ���Q�\*�GJ}Uy���>��X)LD���3����T��7��� J�*#t�����@o��.�*�x?��j����%=3<k���������3���@ �����2���oN����eO;��\{S�\}�Yp��i�[x���&�M���7o;oaT�@��?68�G��@�0���?����8�_���AS`�7�e��d��^�k�V;������u�����o�_����%�K"�D.=��K]6.�����[��,�U�^]0�Ug�������I>��e��8*	����w��N�����s'q��_���W��X������G
�k��D�����"��&irB��S}������-�vx���F��R�*l�
[����3�r��M�������Ao[{���?������u;QG������'�"=Dz���!����:��S�\�����$0����;?�M-A��c��v3z��G��v��L?�����>��^�>:�)���)|�Rj����
�q�P)���U-1��6�Y��Ja-��8V<�H1c\����B���B�|T�N��v�z<_<���*��-�D��_4�z������W���J|Uh��z���v�E���7��=YV�
�Qw���$��������G��:iQ�����X����n�"�Et�^ts�"B����<�zq�xl7�;g=���/�#��B9;7 �0Zci�$T�&��.���V�����k�p"��w��\�����@���2������ ������4uj`'���\�*[RC�t�,oV�p�����\[,�cY;1 �A�y�RCb�B��T�/Hg�Y���DB��{\rM@�b�l�����b�Dq��gjn���
��^��U�����g�d���Yt/��Q�����J�l��)�R^�Az;��}���������Z�<����{zn�{��~c��C-�+P}`Ssk
C�h6i�����NLn�GC<���D�#�mp�LB�6�z7%S���}��	�E�I�w�*(�=Sx���ozu#��[dt�Q�u[!70Q�Au�������L�4+�����lIz�[��zQ���2Ke^����E����K�O�����G5�����L���`C�.�-3,a���a������R�c�O�64��"���*�+���9�:1����|6�|n��/�S�����Q�f?������f���_�r�N���<�7w��g��I/z����gE�!*&x��������_��a�3Po�#����v>��������oM{�i�q�S��nO���.34WNKch�w�x[�n�p�Q����'�!�2�`H�E���+��������^aN���@cD�;c�D�I[8P���B�p�����jbo�GSU���7�*@�L(2����o:���7��/b�����/����]�7�^p����4�9�h	�v��{Z?�������|�����
��\i^3�+g�	J������VRc_���dIG�V/�Y~Ciy~z&�V����Ud����J+�um�r��b6����]��F�c���oX\=_7
m�$�Y��z�~����5���UH9��g��+���x7T���4����cX�����v������h\���Zl��������3^����a������b���m�:�}�8�� 6�'����[Z�gd���(wv����d���������y�'� �l��������i���x���w&�����G��h�[Sk��U}�,;E�]Gt�uv��PU�+C�wJ��Hq��������?���T���
��������D�x!��p[�-��5��>�2�|D~�
7{Q����������w����^k6X<�D����]�qi�`k�]��x��v7qW�>l|5��������p6�������MD��
"vj��c��IP�������I��BR!��J�+K.qsT��'������Q�L�����%F��&'w�Q,��0n���I2��Fc)��������m>��������c)l�x�
w��
��E.�\� ra��O�Of�1<�{��G�����
b�2���B���9h��HL3�����\�W�C4�T��B�0�i���Iv3��m��`�Q-/��sd��\�����
��Z�����k�.	G���q\ �H���O������'k�NQb�hP��g)F�NW�jNw�"D�x\�td���_R�?��JJC�����W����;�=�Z��}v�\j?sOTb1YV�q`�C�����4��(=P����jr��L&��=�qXT�tm��{��8A��j����Y�s?u���*+���p�2�I^�$��R���$���Q:M�B�����<Y���b2
~	K����'WW'g�����g#u���L���-["�Dl���K�d���cc�
lJ�Bc���Xh,4~�E�}Gk\��u'CESe��^��q�u~�5z�`���V!-g���g���N��o!P�����CXw{�!5I��G%d����"��L�
��0
�����:&����3~;?�"�E��.�$\J���P��q��B\!�w���a��Nti+e7��{eA��tR��F��F�4�w���v��]��g1������>�R�_��'�~�z�
�������95��"�hKy8�g�a�a��f�,f�U����}����>KB����I�/��N��UlQ����Y��N�#F"�D�0��0�km\z��l�����._a��V���qu��,�&IUb^yo@�������!Gx23��5���*�_�c�d��ei*���!={��#P:p
2�[���bM@�%�!�CD�����@�#$�X���B\!�w��m{�v�0�o)�����n��"����t���^VD�3���3�1�CsW;#'�X�,���z��];�%`��p�JB�"5)-&��������JC����bS&�tm�B���,��.lq����5��F���P�����>������h����vi�qC�k�Q� .��}��^����.+����r�+4]a�g�Y�p�%Q��A�"E�x������?o*�^��~{�rCQ�'0�
L�[`��L�f-0�
L���m�U�v*�T_��21��{SP��1�O����AXx��>@���kA�&���
���ce����K����$K�0�J�!���3G>,6��xU��Iuy[��p��lM�yN'/�#�Jh�������!+�A.1���Y���+��i�l_�Tj��a������q������i�fT��QH��B���+t�i���A#����l����*��8����lB;���������f	?V6E3~��hjw���g>�/m5�M�6���!<��:������q���QB:U�y4�KG����0Y�~^�j
�����v�j|�Za����W���.��$��L��.�����b"m�K�n<�.`�
���C~���7?
SjO�����pg��!��!*_��H����.*��z�.���DP��1�����t��EC
Q4D���V=�C�o
�P���e��k3�w������=�Va��[�-������
�.��&�Q>��)pH�j!��Y�,d�5���W6M�T]\���9�N���N�z�5�zQf��>=&[�:���o�)K�[��f���2E�M��~���'��������z���4�]��P
g�]<YTs"D��������q�,Z6��6��5=���-����e�E8�4�����!���|L�G���z
����76�?��q[ ��y���*r�FX���C����.&��s��		�``B�����9����Q00���
A"X�P�)���%
X�85�}��k��r�$�D�!�b������{���(Q�D��$�q���_A��O�'���C��n������w
q&�#5xg���)-�N�?ov�iE+rF��E�<A�c��9=�4�����!�Q�:R��������7p*����C�9�$"ID���m��6��;3�E��Q����pY�,\.?��"�\!�W�+��������uf�T�BW�Hg�X(�a��:��Gy���0:���j"P�"X@�Nfl�o���*h��4Y�)Xpo�t{_��,���>�XWD��L�"2e�2��l�f�J�!��W�+�}�T������8Px������$�2���XX,,�w�b�N}z����!��*�'� ?��Q:�>�_�v��i��@Z -��9�#r��St
�8*���E
��c��.a�w�9���9�8���K�Nf�|���/�J���Xp,8�w���-��"�f���|S�� Y�,H$�B���)����S��.{M�cs<m��.L���i��1�usPo���X�������
d�������T4�[w�l�	j��ZA������w�"����V:�
n��[�������m)��
m��B������>nO����S��a����1������\]]\���UQ����_LJ�N�_-�������AM
������
�l����y����1%�R����%�:����c�����e��<�����M�U�C��f(���Z���\���I�fjJS�
�.�����T�#MY�8��Ixl+�J��e��Y��{���n�����4�����)�9L�4��J�Z�'y�w3J���YS���,����aGs��p0e�D'w0*�J�iV�c:V/���r�%3UV�4���.J��k�xj
7�>�z�@���T�bKIU��<'�R��Sz{]z*�s���U�0X����)�8����=����gS�)����b����N#��jQ����H�~q�����9+`�d^�X���i������~����D��XR�)�	��r�b�	�����[
+g|���2����;���i��M�)�V������.�N+g�21������g�V�B����%O�;�����&�{�	������e8L=��|��d8R�r����i�<���[����.`v��8i��CsljK���|�#NVZ���<6TJ8	��\��B�3�~S���y�O�^��W��|5�}��=}urvrs��Y�
�����Y|{���ei�������>-�)�/~�~g	^����s�W��_��u1h��	l�Vy$RVO/�k�1i�i\l�����% ��l�����l2U�0,y�-f�����[�}�<���Vj�)�df��8?*�=��_�3~L�!Pd0�����O0��D1�zN�
���7N����=�q����hh�������i�@W�n��=..�%�Wa��_e~����f�a�������9��k?�:mN��v�#��[~1��:L�5.=����������@���.K�;,t�\}������A8���8�S������Gp�6����/g&�������fp����v�.������)
L��nR?�����>��B����x9���i�w�_=��L��6����x-�������\j��x���[����������dm�
��m����o��a=�A��X��x�����z���������f��)����V</��vL[F��evK�)=$��EQj)�t��=+C��(�T������L��O�.|�����M�~�z�][���G��{��u��4�Ij��9�|����D�n��G�A*b_<t�(aCW7'\�/����t�i���_��9�$��0�&Q�G�%e6����n�z�
���5���9�
a8��saT+�e+3HK,~�����vd:t��f$�a����D-�M�����
s������aX���WX�T?h�����A����h�*��������P��Z��e�����:�����2�l��I��-H�� {��L�]j�bg�)|\&(N�{���"7��������c�s�<�
u]�,��L<��9�)V�!�g���;����K�1<5��5XG2~]�i�EP�H:�������w:��[`����UN�P�roM&5���	���#5�Q-�h������'P.���1~���	#�'�[��5+�@���o����������C��N����' ��N)�<�����������4���=0��XeT��cY ^�/���!�f��I����Y����X&4��qr:K{CM�'�hj���(5��W�(5l�����`h�r�����������LP�o��"���
+���"C`������L��K�=Q��V�7�4��V�R�P�R����z�uF�Z��q�_`���7|On�uOiU��i%S�?f\O(t:��jk��.`\a��w����]i�x�gY���m*F���fM��F������/e�v�|,8��g�'!A&&����JM�&��b��$<q��0pZ���~T�������H����'=��3Fj��C���a{[.���W����b�F��XX7�W�u������R��!~�6}�Cb�����������?Jmz*B�Z�����#�Gs��
��G����7�[�/3��-��%Y��'��P������fm^��������;6�[��(����59��Z���q
��=���J��.�UmU0�5�Ik��XX8�t�p�����hw#�%�85���cbr�<�U3�+R�h��%�@�K|�U�=�j����[�-XpaC+���8�E4�(�!I+Wb
n���"�y9�,
py��S���^S�G��c-/)�6p�������MC�W7�7��'��7�_������n�x�2�:n�\���u�qy�9�nZ�;'�\�E~<�
�����M
�]��b���M����^Wp�p[;Ai�k�����0���9�ZU�c����}6�U~�h�\�F���r��6AqJa�hrl����Z��%|�x�����IE����ST.a�j�����5J.����\����\>'>&�KJp����L�l�C�	�Ul)����!W8�����kSG��1�q��W�4��6���c�tO��������][ZS����3����aq��6���a���p��s�?���?#Ok�{\������#����m�l����MN�^|����]���~�kf�g�*y��������%���d-mk���^��F�h����N��X��TQ�$@�}�Q:I��B���	=���w�V�q+��
u�0_���z�	���Z��������d��O����|���e������t�:]����:��<�����l,lf���������k�B��i3Z*[�������=.Qq��c�	�Q&��,/�
�B������q�}��6Q���q���7#�&t
[�	�?Uz ��b5g��O�{��;X���+(^.�w�id��`N����7_��HK�1Oh2�z_��[����&/\�$3��_�����9�/t�6X�=-F�1��� X,t��r)���{w`�;G�r�����<�9�(����*z{ru��b�>S�;��(��U��	������{����m�do���<��-b�6�'���q�2>�;C��,4V���~���u��k�-��^������Dl��Ewp�/w����S�_H���+��[��<���
��:m�����u��$�C�����4��K�(�|G��>W�k�:����/=pO�(?��Y����
����mco������B}�sz���w�xtO�
a_���v����1�����5��$kZvY���R�j�J���|u|v�����[����m��%�[|�\��Ub=����9[D�EK�T����4�l����*��o���rx���N��i]�x��b���'YSzv��UC���o<jOW������������j
�Y��-�������K�[�M��A������`��[m��~�!e����������6=y�������<���V�@�?
���������sb!�Xh�Wv�i��tbf���l��,�6#k�y�Vqo���N�wEL^S��J�K�-���g�j����h�����ife����FL
4B&�����X����Z���9	z�����"4���Y�����3A;�mz9�:��Q�:�����}�8�u���z;���zw����n�'nPs
�m�M��f���G�y�M�G�������0����,a#\�a��zpn(lN1
��s
*n*nT�������C���S���Q�)�]%���a��K��F�'��g����)V�7����hU�:�]k3'B�Go����]�d,f���V|a���?>
�� �Q��7����������`�3����b�Q������6��xj�*Y%hJ�	����&i�b���{Q��n��s��h��~����?x(p�?�?oxo����������F��1R�?���3mk�������=�����h��3z�$>���f�l�}��MB�����a��#ds��
�^Jlz�� !���-i?|���/:�����E0M���q���1 
���d4����d���1�+�p��2�c�����tc�&�5�h�F|�:���}jq
�����wh�4��16�9|Tk��z������V;3�N�!��[yc�3������>�J�x�&�����%�nLg���t�
{��#�J��:R�5�c�����c�P4����75�Kt����}�`���*��[�A�w�(a[Kq�f���]��Gm����9p���mJvNm�~���7��&����W4��#��
{��$�N0�u��i�nK[�=okW���3
��������%a���3��^�U�
+[�q5E�?�����/���
�����(��rp��"�S�t��a!��1����6Me�9&�����-[AE���E�Cw�3��@G�?v�$H���=�6�5���������t-���&>�^��V�8�}]�>CV����F���aT���W��c��Q�&L�Bn�~������w��a������G����mG���W^�3�G�=l�#�hc�9q4��������>6���d<-��YJ��<��]�������>V�{��+��i��^d��iF"�{��,��[���,?����e_~�������9������97|��Y���kKv`B�;��7�S���4�})uqs�M�d�.������<0������5���Dk������l�=�&�H��&�����]�|�����g!:�+�O����D�#�-9���\��*���a
1��F�T��#o���)���%2�{p�����g�
�f��cr��_�o��G�����4��y'��Vs���S���������X����1�8P�����<��}�m?��d�����O�����y�G��7�B��~l,"-s����yD�\X����1F�4������9�����>|�m[��\����FHT�}j{z�(AV����
A9%cA7�`���/7���/�e+�O�;�hi��lg-VcX���Hx�,6���I_
�E��f����<�g��zO��-�3:�������W���;zJ���`�<K�V����E��&��[������qs;1h+������L���������,��v���eL��C����ZB�<�g�O�f�n8���g�
��L�0��W�	��s����Y���JW��`�o�����������u]�&��e�U��?q�^�HaS���������/����6��~U��rp�!��D������~"C	AosKm������:�P���w��eK�����p����>�����n]T��Ll������Z�_����:?=�����a	\��,9��[���l:�&������<�v�E}X��w�hCx���Fi�
>	g��%(�����o����<���Z��<��;8�94���$��������VbC���i����;B���e�Gbde�p���9/����H���24�NiKl���w0�������G4�Y{(��}g($8����&�}���Z?V���6�6����+P�H�A�8���*28��Ew���e�_�G�������W�o�j^V!>�	��m�����)��Th�����6���+|�����hJ.0����_��6^��cGw��{H�m�c[�Y;�}o*r�d��J|x�����B�7��� G��9�l���(@�-2:K�!o��Q=�V�������'��lIN�k��5B���W�O�;���c���7�W���qAH�uf��+�tl,B�k�V�R�sN�4�/':u��AK+�M�z���V�l�ZjA)�Z�?�8��	=u�p��A/n���Ba~��e�F�����s�i�?N�f�-44�w�|�#5�V�,������3�j��W�D�z������6�F��������:��c}y�S������o�j� ���Z���i�}}=��e���-L�����s�a���p�0h�oY����Dg�:�S5����e�����������1��fF��"���4e���|_e%5�8��-yt��7�wB��h,���*�5v'�G+J;��Z�8�q9LF��yR�aFa�)����\����!��$a��q���)�z�1��c����Yf�7R����
q�1s,	�~r�u���NbL���7J6��/)�:C sm3O�67D�]��*�U�%�������,��R����
����u�������uc�y����d���j3
[������n��_n1=q6����w�{q�b�d�	34���7�7���h�4�.���P��:���7�1����
u�q3�]��FWj��4��N��-��&�X���)g�n��hIq*I�'��$�����iY�=��:B�#�	��h�z���S��H�(���Q��pX8,���Y����=;��-��?�
z��^A�#�7�U!��
y��B�G!/[���(/�8l�b�#�X,�������c�	Z��+�H���eN0,�w�a��i2B� �~[/��}l=0>}�^�����~lU"r
�Y}��]�X�T��F9e�@��|z�����M�C�4��hiQ��j�xA�����7"oD�������h��;��	��C��n��g�v�H}V��r5FqMyV��q�Uo�|��s5d�p>����f:5	��r�&��������J�e���S�vY_<�4z6�������g��MM�)EV�F5�7
S�]������V>+BT1{=������(�QD������b���*��@�+��
~�;^����PV�&`����������I�UsS�.0�&X�#�9[i�T90s�`�$$�����I9�8&��,���������K�|,;J�����9��)�����L�q��T��3iaD�_�z��b3��?�Aa^�����;����PS�gY���C�z�B��b��z?��qB;�%�tL��!���.�����{IY.8���U��G�w�{/�|#�Le�bY��P��C�c�Y2������T������3����>�Sh��\�9�����������SQVE��"���g���o�qm�f�Z&U\A��:h������2���p����M�uL��e�'�N���������PX*��?Gu�����m��(l}<5��hqdL�;LeL���V���}wLiW����W�\��\q_��Yu
t�����3&�M��V��
���s�x���1��.������R!�8a5�d5��@���,�V�Y�(VRnm�tM����������������z�M�'���#��V�������������|Jl�8w��%�l��J��scc���M�����6&��pN?��u
�
�o�Vq}��ufBL�5��{�gp���VH�qi����ZU/5[��47�r�PV ��u�b���J)}Bv�	`5�T�i��I�N����Z�������`p��L0����������������������������+.W��P��:��-�T��q�`\��-��,%D!�+*y�=<?v]A����v�)7A�l1�	���j�������k��������������%&�����d����!k?;�����?B �|I������/u�T9N�3�MH9Y�1����)�V	�h������"	���.?���W5��������=���P����{{k�v�f�C'V�@��d��2����l2��s�w��Y��|���l)�u8��v���N���N�c�����Q��M��=�<�v���L��������bjC}{W:n�]V�v��*�{x���!"�g[���� ��%�2�H�x\���rT���`A� X�K�*�C�+p�ZO�����
���bA�#��y]��L��/�d�U��a��7�;
���?�.[�)�k��k?^q3�
����va��}�lgz_���u~~�������.�^���XW���������W^����&�v���;dw�3������:���K�%C9������q��A��A?8z	����|a�0���}~Z��@~w�$��5<���3nCO�_����+"k����h��q�_j�e�$�"7Dn����S;OKkoU=�
|�_��N��5���}��0��7��-�p��8������E��O����fsX�%����F���C�O����Ro,��c��:+b�!���b!Yj�"A�E�����r���2��TQ�����(�P}4�r�1�,�.FC�����s���0�v1��
���&�9M�����
k�6�9�R���+W$�H�ok���<���T�����#��7g�)��������.�B�}�o]FBD����7 ��WJ�.]��������{�@\S�1Lc��$Ib�A���2��SM,�(l�`h��c��Ef\F���)Er�d�%zK������Z8��+�rz�s:�G1&�����(�<��6�K�E�%�;����!k�3��?���%H�C,S��4�p�Y��0Nd��c�2��W��:h�c�������I`$_2��
�@�C'��7�����K�4DK�(��-��\�������)���je��g�@P��$���o��P-=����J6������1$�/�N����AE/�@��v�oYAg���WM�+��
xw^�U3~L�0i��g��9,\��P��U�_}}yvz3�>}syv2Z�4r��.���&��5g�lmU��N�r��"�������F�h�}�,/���s0�]M�"������f��;j��b��q�[������1��y�fVGm����`��W�v�&1���h����V���Vs$41��e���e��(0)X`�6Oa�G�8��/M���>���hp��=	t�&����
�* �����
4i���1��q-�}}�G�0�~]�/~����k�9�z���;_�us��QV���(�yKZ`I�5���_�<2��Uz�?����E��G�1���]��P�������������o���2�F����K���.����~T�e1�
�Y��%Jo�(�g�$+�`H�J���b�gpiks���fU"���.<���i�����������.����%�d���E���a/��\��&X����q#e>���[r�uv[���=����I?�e������B�?1�@��rZXn�#�������	�F������� �N0Q|F5��V���N�����8�l��5��z��WxY-M�
�����~O����T+��-9rg��I0�#J��7�"E<�x������_���W�+�}$��
5~��_����q���]��pW�+�}<������_���W��h�m��
}��B_���w���K�|S�n�+��W�+��>z��m��En�9������>����p���qV�$�e=�����7��N��7����$�a!�B���
�/��1����&!4h�R�j�M4���2���G�qQ�-�`�O����u��-0�����q�J��)(���������w���/J������VyVd>��sv���Wp�H��>��gI5��;|�6�cU[/���w��V^��M�I7���8����sRa2,L��-�j�6�B�.�~g�"�'(�JG���o1��He�l���9��:Gl�K:���x���7'W'�z7�������1`	�	g���h��~��U���X�a��}H���U��R��`[�K�8����������[)�B��qirX�t�������Y1'bN���9s?�Rfo������[?��-�M�; d�j�:jJ��L�k�^�(�O�P��JJ�C
"���<f��So�~��Q�������_���Y�g���qTOY������9���T��a��"S���k#��Y��p��*)
�l��t�B�"�D�� A�OX��N[�S���)FTm KA����KX���H

+4�ql���K�q����UX���������
�[����pd�T6�%�6&e�����"D>�|��B�;�
�x���������������(�
n���K�6���
&����a��1��[�t��)W=��[���nRL,�ta�0]��C�wM,K����
Pb�F�s����Zb+�V�����[���)�1�C	���������3�p�n.g>�q��5I�5�tb��"RC��H
�;�Xm9��gv	�!?lx��������b���X�������	"&DL�PLl^\��[�v<*��os����-���R��
���W/,�k\U�g�E���e]����lWu���U��zP"ZD��h�����P�EI6ee_�,D"�wJd��wg��w���]��N���f���l���M���]}�r�1�����fB������N���>��/����a!�B���E�"�)&t��%��UV�o9c�2���YhJ.at2Svm�3��`�q���!W�C���iib�����,r��T�U�}��u�#�#���M�����
��695(X�i�����[a_!��	��p�N�x��.S�1��S�K5"�DN��9�C9���W\4<=��UTj�+�x�~�m�����[��o%rkO�VC��FZ�G����n��qb�:�=��Kt�	6����p��s���O��@��!_�E����o�e^�{�}_����A�������o�M�"�q��F�C��<������/r?j0�#�o�3V����7��k��A�7��>|�z����d0��C�oyQ�)
u����7p����Yw�u�p�4����^s���L���R"C�_��n�0���w�,(�A�S�����j��������gb�mW�]�vE��������M^�Bh!�Z-�~|B��8�;�Bg���Y�,t~|:�-(�=?dT:���Bg��#������k[&�$_)�X���������w�7#����]�%�q���&o'�?-����$[����4�eU(�����^�CE0p�,5�����,�$*�2R��SNc�NU��+�2�~&�-���s�	��O���Ub�w����2K�V2�����p�
;�Z���P��Q|&��N��_;W��j/v�����j������z���Y�qi��vK���
�@�j��S��M���������f�G����%�����|���'�~�im��ZC9��Q��y2R�2�/�DJ�/Z$�Hh��"�w�m�R��j������:�|�� 1�D�i��Bd!�y�����b��}��QI�z�S��?'g'���Y�,t:���A?&��	/{K6����2iH�[��Y�,`�-�������_�"���M���r�#�y���4�N
���G�{�s|(����xD���xa�0^�[���z�1I�"NF���?�R�Op*8�����T�z��y��v��%�H��
i��B������0[E��
b���*��-��pV8+�����lCVQd�X����������)�f�\:��e���=(�I�g{��H>q1����*��qF�~a�9�-5&[��EbQ������`G[�E�!��n07>�7Q��p�":�M�e�2u�.��O0=d����V��F����;?R�+h���B����S��Y������P��sMNV��:��{��t����>���"E.�\��\��8b}�X���s�}0����@���v�����	����`���W�+����o�=~L�[H!����j��U��}|{���:-�Z���B���S�N�!��
s�����`n���,���5_S�=L�W<z �^P�����na��[�-�������zEyXqWuk�N ;R�h<�
*����S�2����
�<e����a�wI�B���e}=]��m2����%�L>
'��r���'������JK"�Bd��
�"+�!=����dzZ;s��������m��@[�-��	�*�����b����
��}U���=�"?�~���a��pi��p�^�p6(�\�"�A���>C}��$���m�G�
���+��0b�p����y�*�J�������f���c�J�$�I������������RM������*��;��zvk(��O�����v7t����o���[��S~�������to�5�!��V�{�X����Y;:�7�q}p���V���C��=��P���1OY�-���<|A������GJ�PtV���bu:����'K������g�qJ�t��S��'�����y�M�X����
�jp7����1&N��J��h5�h�<�����I3\��a�����L9;�af�^����Z}��04N�����u��5	���������&l����<��:9�>=?>�3���Y�'>.�&�"vE������}��N�_]�C2;[�1����g���Y��O���Wo/��_���-m�]�,|>������mH�$���I���Om���min5z���8��R-��9�������@�)���I��XY�0����J�yx�ll�3!�j���h���&����c�dGCd���
"v(��Tk���ZP�����%���C������M��������W�����Vo���{�3b����]������6����{��M[�?b��'�������<��v�%&l�
��H�w�*(��=�+�^
�x����GY��'h���yG]��DO=A�����"�!-!��_����_������5I�;����;������r�?���]��UMX��?�(l��G��u�}����>���V�N�oN�_��$�@��}���B?h
//��%�C���"�E������������{���"h��p��)R	T���d�A
ba�DK���W'W��?�4r�$��Fl�P������Q���� QD�d����B��<���a���;s�~p��������l>�,����i��O���1����L�b\���rS�d�Y������
8ua��~r���%O�A��"KE��,Y�{Y��t�$%	b��������
h��C��l_���p�����7J�?��L��:���s���Y�,d2��A2��d������N�"���i��`��Mv�F��z`m�������q�l�^���i�x����a�0���h����aej|2sX�:�
�}0����E��D�"�Dj6l9�Y�95���>n��%`���O(��=������ck}�
x�I>�N��6�J�K�H\!<�x��I��9&���#b	S���/o�r��
U��h�)[����V�'g'/oF����p���]H�s��A�h�;<����;��t��mK�rf"�D��0���hK��`�"���t��
���dA� ������*�P|���c���x�<��-����/+#�w!��XH,$�!��������]m�UZ�����z���H9K���V��}�������Rteo[���Bx!�^/��!��@��&��c���X`,0~Tg.�x3����g���Y�,x~$<O�ei+��O���@Y�,P(�w��D�u������Vg���iZ��j��*��{�	��Gj���s���K���>F����f�����F9oN���f�P�6q���d6��QW2��0��8��\� rA���%����-����PC�O��y@�Dp
��c��n��-��__\�?����Vo/_��<��������
�c2�N�
����j���_b�Vx�oA��������]GXb\%W�?q#�\�3�RN���(p��`���h��r3[����<�3X���p.��2{����N�.��%Fk�6��c�Q���:Od7<�pSb����*D�� A&�l��,���/'B*^/��Q��CT��Fdi�T���m�{m
c����d��I�n�:u}=��e���-L�A�~N�~�h�@xz�������,��� ���3��)�e�DK�H�$"ID�?Q$��G�rf�I8�Bb{�-]4�5������-�d
��B��_�\�<
K�W'g'7'\fei�PM�-\�%��ii�a�T����.3^��(p}EU-��0Xb�s���ja�'���7����/|�n�Z�0�"�D��P�&Bm�B�,s��opw���$w4����Y�������[���5��}����
���`P��No�Q/�9y�;uqyszqN������������J��!�U�4�Qo�G��-�^��ND��":v+:��6�X!��s}g��Jh&w��v�O��m@P.(��w���:�Q�E����u��ae��sW�^�\�ga��[�-����\��k������5���
U���*5:�3���po6���4e))�E�Z���s�Q3�
��R��-�t�r�!�CD����R�c�k�]��9�uB�^�,t:�wH�M���]��`W�+���R���d��h���Z/<�����������3�pT����[���Y�,p8?&�K���_�\�#�0X,�����gm��q�g�l$�p�4wl;F�e��6����w};G6d�	���K[L�,����|7��H1���Q��Dk����Nt��U��*AR����%RK��N�����!+Y��e�E��S �X1�?���h��NY.V�}e�����G��Z�� =_�����MT������:/�}&�������]�Z6������^�],LI�����>���������������n���V������I%u��������e��2�9;7j�h�4	��v�n<��N�vD��4i"�d��dp����.?��W#�D��v%o9Ue����U�3��0��(~�������}�������Ei�M(��-��na���?��w!���>��bR�j�����<�\���]	���0r����Z�v
�������)���w#e�[]d	G���@�$�=���y^-�=���O�Ea=��`}�,�q.��T,oI	���XB[.��Q e�9����Q�8l"��4�>�����~30����%�S>-�ShZ?������hU5�Jg�0���~�t����AA�����L����)/
<��'k?����M��M��V�|>�����yC�a��������{�
������mz�B��L��)?WS�������@7�R�����������Ag������g��K��C�h�;\���n��s�^���n~��L0��c���w�Hay<���H�{���7x�����O�2�gu�&<����=$M�~�e��O��8r|������7��f>1�`�I#ly�j�"K�����m�J�[��T���)�����I���Q�-{e;�}S�� �����d���<��nt�D��d��Jb@_N�>��Q��0�'=�(g��;I�TZ'�^2������h���Y��X�c����K&.y4|�a��3cTf�q�����^@
]d�V�z
�������p�����i�$mr
=�fP-V�
=�/6U����sS�2�d*��HBi"C~v���X$����������I��<��	I��5Efg��'��Eu���Wm�v�rl��pq1������(��MB�~�V����b��f4N?��DZ�����(O����Y0�`��U�ME�k~��W{��m�
�^^�LHb�u���E�N���S�f@)�0B�='�e/|+����0���o��3F��(�sf�1�O���BH��eA_r3�1b
��Fh���&�y<Y�	��,�)&�b8}���:��]���L����sWn7���Qw�aZF�AAJYj���~�a����j���3t2^Eq�]D?Hw,��\��*l���
��h�m�G���k��<0�����G���?9�~7zK
���w����������cnE���|�DsRE�'���	�����������[�)�EnY^�N*�T��L�c�0�8�D�����E_�+@��U�@��i�z/0�
������P	��_�8f.)����+�j��6�{�@��Q c(���dd�y�3

��]:P=��[����D�D�0��.�EJ��'~��9��bD���'���|�YJ�a6��Sh�V��R�|��V����I�&	�X���g�'�*S��C:�����<C}�Hx���<..c*������EW>�r���A�b�KQ�C��c`�'�+����
������$�<?��HsO$g*Y5��I+��/��Uzj�D��<�1�C����
���9��'\
&,"��XD�%�eL��`09d�A������L����"I��7�]�s��(��8�<[�10��
:9Hw�F2�\�(m�&IB�	��/D�"��D,I��f�p�����������^#O{��WK���<05X���M[PPw���(�&D;�����_us�^����,����6??]�I�,>Kk]GU(a�,�[5���B)d@���s�X�c�8�2��a���?C���J�*
�~+9�64�������c�3�KF��\�����i�}��Ow�>
��:]��"H5M��N�,��0��}��.%�����m�������!p�R������R$���<��4g����>��%)�"�E
1Q�0�~W�7s3[Y�(�+K<
��<���Er<�7?c�F 	�>�8Ix�� h���te��5B�L�-����$�E�j�,�g���9"�D�������m)>��s/F��S����9�d[�*�������<��;�!T�i�,%������x�T6T02��AuT��K�h��4|�5�L���q1,r�b5�/�����2_dD�v����	�����g.��4D�d�,5����9�������r�K�XW�g�����6M�������
�x$��F�����@_���>�Y�����u�����<L ^b���,[��X�`�������������]R�������-��/g.�6���K�G��Hm ��w�44
m�=�i���y�>m[�_[�l����C=eoAmz��|����
Fm�
�0k�������=2���1`��u�:^����U��=C��V��>��Fvk�'�������9�(���%0��,���/�gqO�6�H�I�"�M"R�'FJ`zT/G�B0Y�h8f���O�d�� �8FxZ���%	�<1�+����K+D���B�������t
p�����"bmg�#�'�nm������V�c��Y����*\�$���K�XF�A�[A �	-���I�����poO�,��]f`oA��'�kF��8����n�a������1�z

XD1��x��}c)p�!H�!���D?���T�3k�����+#_���_���a ���������4t�g\��$Q��8�<�9�YOp���D[��1)F[Y�wcs6O��`��#n����������\8�S��V*�K{��x��2F���X����.`�X����k=���pG�;"��;�~���B��] )\;��H�u#�
v�kQ<]���vw������#h�����j�]�-a&'����
�eD���s��G��G�z��(���
��5���m����"�Q��;z�q��h"�a��G�c��I�1L��x��7�uW?pZ,��x�
������%����E�K)��y��b	� ��q���b,o#�����v"<jP������kl��~��rN�G��Q��;?e����Ss!���������� xv���O?���o���]���������W�F�M������Ww����w]�l5%���L��pG�;:|�tX���w������mGm���:?D~Cd��a��x��e�B�5Jde�Q��8%'N��"�z�&u Zn�����QF�����m�m9=��Ya+�k4��<����p�xR��t��,N�<�z=d7I�+K���@����u7�0�����gZ�tW���D�I��6��6����4K���<[�;AY��"J%������4|���ac5�U��Hm6�?9���B�j�5Sb����u����ir�"�A^r�D9�>��|L�+�j�%g�����=H��|���8!�1'��t��2s�I<�Bq����Xjk����!����y��IXF_�E�����&*,9���l�����	���N�l��������OYYI�����:�
��,���=��f��2X,Oa����st/4�DB�$��e�&�H���REe���������Y\��6RF)���"s$��)]�xJ��9	�
����2�$�`r�$0k����W'�9z���+����4�T����o�&��#Z��?yw8:9|y������R��q��q���k�����0�7��A����<����@�X�_��m� �H>�ID�,�Nw~N����MnkG���a����5&0w�|m��Q�@��y�^���9�y��OAmM�5�Y�����F���M���K����f��7!']��0�{U\�o��o9���#|g�\�K�D��\*�b���LD�|a�V��L�[Q�q����&/x���mH(+�������-���:�t��=�p��j�����nF6_>};:~}|�~��k%]f��������s��Le�9$�\�*��>Y��(����9���}���t�0r���&�2j�_^N3q^�n;���0�D|k�l��g�B��P�����q)	��"<�+_*�daI���TQA��(�u?�u�����g#R&�^��<y����h/�$z.����zt�n�5��s�%�������m���i��E&�gO.I�8�����M�j���"�=)��������w�M����
D���^�hr:��'dAw��I����JXNQVJ�2tq��D�v�\S����!O?�����l�	%`��5��^�i����$����m�Wt��"�Atr��������T�:%.�q�[���7��~j�<�j�+>�oW��kn�vx����i���O�/������G�r�W�R
�������q;�Uv"�
FI6��R��4���,��A'�BT�c��������za�o.�
8N8%5��~�����:�e��������~�����;h�_���?{�N����!2E�LMK�<����^x������&J
i�N1]$���#��8[��(����t��Z�������uC.���C=���K(��^4\�S���h�Z�d!;C!���;��j\�w��GS�qM�/}���s�.��\�����
~I�eI��4F�������SM��xq�-����b9�GAX��)��H	�'CZ����_������k����v�(��e�B!9��8^1e������*C���+��s)�J
G�yLs��8�y��Q�M����������1�L4��(. ����2.C����e�|p���)�E#���H�h�m6db;�{.����.y<���a`����f�@�1�)SJA���ii�n3�&	{ 2��q�=,<EmH����+�0�0���mf�����le�}�����������B?����)S�� ��������bw���W���H����Gj�T}��|����W����7����	��Ng��r�#���*�ossF�'\jd�����I�����R
y��-�2<��*B��t��Uz���^S��>4������h�����*n�{Ms�s���i��{U�<��qw2�Q��2)����z,/_z0��TZp0�h����!���Wa���L��q��%�~�&�;����9���C^�V�� �'����Y�p^Y�����Y�X�-qni��AAY�~ko���	��2*�AB&�����h�,�\hY�e*b"p��n����a��y�}�F�'\����u���*���/T�/�dI�KX�X��/�1F��Kv�=���h��B�L�neS��.���d���RK.���4QB�y��(h��`��?� V'���u��ric$��%Q��y��1J|�����`uC����`���	��G�;|���?~o���%S0���������&��
���z���|�����k�`�[����:��P�N��P����:��xf.�n�>�q�Dx/�B �1KS@4���g&�@���e
��xDM�#'!��eIr������V��g�2��4Z��3�>�K�md �j��D�wyt�v>��cp�����F~����t������������p�
��y���H�M����|�M����%�%��<���I]�[����
�[ns�.����� �7���V~��w.���]qmE��j9
-�/�U9H���v�K����J�����!,�����W���bk���|YY-�D�����X����\X����nH\�5��8B	�����OZW8��F\��7��(��s�������3a[nI��j��Z��sh��)�
�.Se�U	U�{Z$����`�$�%��r�+��<�����9�q�����`�8�R�BK��|Pe\8��b��k3_��[ME+�k��
�,�V����5����b�\�.�^�8��������
FD'Y�VLb
����sJ�#8��d9*������V	V��K��-!�g�$9��H�;����R?T�3�"�J\"���x����l��`
'O�"�q���kzJ��N���j�<*�[�d��5���� ��CM�HX�I���$�#������jv�`���
/��Tl���t��8��0������B���l[L����2�D���"�+�fMKZLU���/��N�kQ�3��}k����
�,��TO�~k��cf"��&0�s����4>F`�����F8�e�ga^�.CM�+s�x[�L�&q��fY6!�-����A�Q'����������=6�������r8��.������L�u]�[Cx�������.��#�A�rG��� �	
������mGm;j{����j�E��n#b`�X�lp��L��1^{����!3�W��a��x�cq��;5W�x�[�����1���t�c0������v�������Mm�����eWk���\������%_���bbxm�~~����+
�4W���������^sp���I���	/��M@v\��r���\������n{�J�����K��%��.��@��u"5�PE6aQ/N^�<�o�U�������2��HSX�4���e��r��
�5E�q�����L��[�D���������6�yI�]T(8�v��v�H�5�H#��O���d��ST@��� *�qtT����9H"h��P3�M+�m�;�2#\�@��Y�n<C3�����(aXA� D����b�}�qt!�JI�$�Z^4�d�i��^��cR�@C� �x ��j����|f���~���������^_b��iu�#�������G���s�Z�M��4���������	������z(ad�����F6_���m$774m�W�n�-���u3�����f���h��%�b��.���G+2��F��j�N��y����5��$�NR�$�NR���vD�#�����"�4�}�@���,��g�g�2G��9Q�zK���I�5����q��q�����/��$�M�*�S^d�F(�MQ����+G���T)4���u{�,:f�1��Y�%��8u��#������[���CO&�S����*��G�,,��$�Wb�+:�����)6xG;��X@U��-j�����S���+����$�-�K���������o&���9%EQa��E�x��0��-�ct
jS����R\E�����^�#����>��1����6M�� x����o{�m�=O7�t<������E5H����.�^IK���a���?hUvT���^p��.�F��Gk�;��PBo2)!��Rfq��7��&��%�6r��`�;�k���?����W��&�y�n��_�Uj8X?^�����+��j(�8W�|aR����#���T�C��3�d|#�r�8������'Q��c_+�<���(?WFiUk��E�j��"����k��CX��k�����7�>����G`�\�L�
H����\������@�: �;R�=��C�,!�n��������jrq��)+O��6f
'��������]42�!2Od�evq���4�/Q�8(�<>]��1W1�����3Om}"���_E��,���vi��P� �B�CUW�p���.�+�j=���p�L��)�#�����g	iB�L�X���h\�f��l*K[����d�n�����3U ���@B��Xl�������G_�9W�����-iv��	�[��g-�q����Jv�)��jY�,>V���'B4����Sp-�}�bM�k�I,����|
�`R�G����v�����l?�������x������|����.�`#C����F�np�H����,^��,}������W�_�;z��>��a&��i(=
ag���z�+o*��t��w�]n�5AHX.WP1(�1����U���G��H8MZ�a� �����t5������m�Cr���!%����OT���A�'x--.1���s��j�2�Hc������=�������	�����Xf
�Vb�H�����[�qv���c����*���K����f)|'���6[3~W�����p���O6Q�\����^Q�������r�������{R�)h:���K�F������A�'u������F�"tS������zk���^�;T���'I�@����,�]����;��Lj��E=���~���Ch0����<�p@t�Q N�����B:G���T����ga>��;�\l2��n���(J����i�c�5*���7����-���Z�u���t��H�JZe��T�� ����#�]��f�d���MR������������g����k�$��\�^�����m��9���=�m����K�@�����C�!�.D,�gt�\�2K�7���B����h���]�nN�k�S��DI��/����4��Hq��)W4��aj{C�F4K�)��>L4�<D-��'�Oh�*R� ���2D�Yq�t��5\���w���k1�X��:?���y��4���-��pP;NQ�Y-q�P���&�$������;5-��a�fx�j@�4�AX���w�����v�^3�/�^4C�Q�.�}S����H�1xwz2�����,R��a9��w��a������M���N_@�L//�K!!��\h�"h0�S�t��U��
�z��N@qrD)7$����a)P�
{�f��=����^Q�Y~*l�m32!�,��
��z��[F(��W���1�[>���ba,�}S��-����Eu=\�&��C�	i y���za����v��I>`�l���C��'��7���L-<vl(���M���L��8LF|N���"��0K4&�K����}�%�F��sa���E2����q�%uZ�D��M��8���H����*����<���03N*N����w���	��H�e/��n�)Q�����S��a��:�u��&0���A��e��B�����c��W�>�h���#@P/�{9q���)Ip���J�hfL���q��'���e�i2����+�7u��>Y�L	��3�����m@|�#�Z��h��}���6Kn�5�����������)�����t�b�l�}��i��$b���T��iy2�3�Yc��z�e�X�-�D�	��J
�?X��"��
�h���P���x" b�v*Y��2@�))\o�����Ch�EbU�[B3K�u��g�~Xd0k�m�2N���ev�Htk�}3�.�����L�4B��:V���IU�v�t�A�R)Tg��q�IT�����q��L:���!��s�"*#�0C����I���%�P��b��0h#FB��������|jd��27M���Hh�b�B�uR��{�q�	x���d���5����9#E��0>;�UJ�����%u���[U��X9F�2o����7��-�(6����8Y��Ndsg�7�
h�q���\;�Fqg�:���J��\����Q��{���(�-���Q�V��q*�i�!�T��r�i��DP\M4�����Mu:�^����w������=����G�6��O%������]xk�A���'��(K��b�����Y��W�����	��p���9+K�b������� -��5�D����)|�&P 0�(%���o����)��l���"k~��W�^uh��01�N�\��-���.��r�T��A����!�e9-nn�v����h��P��1�^
zd�
�	�)����r��v$v:�[3>���II����<@#.3�9 BfK{�dx�}]�$�d����A�9n��0[�Kc#�'����~�Y��k���OX�����
�h��CaVYr�fs�?[��ydB-�;%G5q/���$K�u/���v�K��~$��I�p4�����2���-3��G?~W������.�4���U�I�M�O��n�l���$1t����vH+]�2�,���.��d�(4��������/�.��0�:'e��x�����Y�
���H��H�����G^�),S�2F�����E��D��}��h�(/9��>�m�k�N�}Q�[���+��/d~���Ot�f<]=I."��.�+��/��h�>��-����kP�::]Nyd��W@��%���#�����o��F��6����hf��m=��2}����e/�v�+('��7`FE�&YTH���p
����L�/�$�����j`m�hZ���b:���T��Io8�Q���[.UNK9d\ �@T��|������o��>����1%$��+A�4��K�]h��p�L}��7+������H�0h����H���~\�s��g[x
Y$��Nh[e	b�J[N��(1��@�����������&���m��%��j$��85���T���sB����O��3���(�Q������s~���[X�x%�<��?()���@��.�i�������~�5�M������s1�������E��,Z�T����h��|"M�����7�V���o����R�<*�R��`v�F���h#{i�"�RF�
X"S���H��^��G�%�DP�!�@&A�Lc�e�x�BI	B��!�\Q^���t!�p}t	�y�� ����Ad��=�5?uCQ�}���g^�D�~���"�[9������)�s���Rs��/����I<	9����p�z��}}$�09T����^1{��4���4/x�0�f]��k���������������D��S�Wh!BK�����P�A��<:��� ��6
�}a�Zz�����Q�$���G���������x?x���[VT���[����9p�656A����Qz��}����k���u��n�����)�b��2F���������<�����\$�)P��M���k���;%d��������6i�{S�0.�����!�W��l�C�axy�G�`��(!��m��iC�����>�+�!�oM~�v��������k��S��%P<�
�������=��A�M5����[_��<y�0vb��>f@;���B+�	S��`%���� ����B"}!���AP�����L|�]H�z���dR���H��;.fa�F�u�n��"��{��y�H\�Ih�%m�&H�-�����I$������_����p!:V(��*���r�]~^d"�*u�d�������g��O�8�]ag|?�}�����*�g��Kn���r@����U\����v��J��_F���Y���\���H��L�t�������>:��_��jwF��J��X>�^��:���D��@8�V��QS������2�:���d8�{���'�:J�bY������X��r��V��*YV���1t�L6�U��#qT��UW�M�.h�DB��e�y����b��s��s������
E%�L�ld���ZSH�I6*�"�sWz����Pr62���j������y?G�N -'��!�>e�{|p"��������m�����/���lw�`�'P��g�?���J.��F���y�n,[w����Q���@.�r�['9s�DO\���y�K���;�f�����p��@�J������ga��H�R�0fb���l�������_��G��F��=���p��8#���6X���TBU�ydG+��"�E8��7D���������m��O!�B5!��O��(�O���LT��8�}*1j,���k���R��`p�������D-�/q�ji��X��9;n��_o��
z�pp:�=���0�1�x3���h8�����p[
��*���V�����[M}�j|V���.�&M?��]�
tH/��0���`E��5��P�:f�=�{��}���N3���),�������^�|sfQ��"n	_��`�C��l��A�+�;?�IdG>��N7	�U����_M}��9n�,�O����*���A���mG��t��������d���83�E��B���H�oX�l�pU��	��V�r�����>��������b�2BV$���
��?b~T�ME���3�jN��tJ���h�g�K� �!���2�YE�(���Fuk����d���h�>��c�1>`MW.����Up�*8u����SW��#���pG���5K]��e����������#�y��sG�;�|����K�P?�������pG�;*�Q�?�
��2>�>��$�(<j�=��rG�;��Q��*�!U�
��x�H~��9��j�k��_�|����H�BU&��Q�J!�DN��s�P&��
������������[\������Z��3e�%	�<�b���!`&�>���������c-k�XK�Z:�r���l\���G;I��jB�����O�
}y|�l!��=B�T�T�$����A+��+l�B���W�Fk���	vL�c�����vX�9��S�%�LdP���������6�d�xlU�jvoEi���w$�#�	�H���1H
�kR5.$U#��X��s_�n�YS��B�.�'��pWXo��N���6�����b<��d���k�Cs9���1���t��c<w�;85�kM5:FG�;
�Q���\�$����n2�V+Q���pN4?����e�'h��:����wA��:~������F 4��*�vGz;������v��U���t��#�!�qG�o��mU��`��|Q�_�g���vg�4���J�i�����O�"gU�V.E)�<�3�7�)��m���M�L�Fj)��v�H�Wcg�
�����@�X�T4]�c.�iJ;G�a����\�%(��)�)qF�yct�6/���������	Bo�3_��%��[CK�Zi�Q��4�"�/�S	�.��xk�q���u� f���VU��-��Hr����P���o�\m_G��9��Uu�C�bIGYD2W6u��	�\���������z!������t��.�:g�8I/K���(��H2�M���g>���|�Pa���+���s�v�����e�X����?�--w�<:QQX�QnZ0�>�+�-
�M�����%[TS�y�$������.����:����:��N�+!�������O�o��s i.�gBk�YG�;������w���G�k$y������E�WD�l~G�;2�����wd�.-�A�{1-��!�qG�;B��?J�n���
+�O�;�����fw4����6����s"nB�W���vD�#���_Hd�������:2����wd�n������lGd;"��;������E��~!]�OQ't*9���� J��	h�z����Y���O�a@���<� �]
�1��Yr�q�~��,N����nVD%}cgB��$,�A������/������
��h��������0���WD~�|<Cp�g�[�������3^	B���8����d&��$Y.��9�1��<�.��9F�D��y�b5&������:?�S�z���s�M#����� ��p��������i���$��E�H�;��#9���9J�Y��=3���XI!h?J���d���]Ou��c�`8��f��
:G���e�[[����m��w��������mM��~Z����z�o	��Y���F��d{�_��w�"!k~��Zv2�o���[�:(�(/{t��2��(ZO��(��K����C���4���,����l�������g�����c����A�yGR�za$�>���~���?
��{*����d��5���zhe�� /M�,
�m�ntW���"���jg�E����w���pp���8����`����^T�$p�5��������q7a����!�<�B�.���������������a�V�x��/�.c9H��q�����Wq����B���'�Q�>���Gu�^��imts��rE*��LM�?����Z�rM�j~�x%�U{�!���I"�)�Bpo�+w�	�
��H�����rD��X*���c���������C��-��3:D�/_�vww9�P�ms$���� =�qpm�o]3]��]��K%&�����>�/H7��`Z8����e�A}�5[E������t����n����r���#������Q,��B8FL��=%�g���T�3���8�4da��Rdw�t�$�a�c]p$�g�����YQ�U��'�!H����������J��E0/� @����qB.��U6V7�	����y\b�%����1���<U����������S4���iU��Y�P��~�v0������2E�5'PG�Q���f��"K���J�p4�
�
�um�6���
��1�>��@�`;�Fe@�G��S�=x���� M��,Y�����8i�_b�R*���t�xa���E8&d)��(~$X�����C�����/�^��i(T��PjdQ�w���-n���Me�W���������Q�U�E�,t��EX;��;yDzo�E�Z0*
DE������"�c?z�Ao1�,����s��;�m���%,,�1�9+&���,�����(!`2���L��(��&��4s������4��b"C�����"K���) `n����wR5��~  |$��N�i^�/��<�;
-'��=	���xvFW���&�|�4�BG��%)�����B��N���_xK������}e ��<������{�IE%�cM�o��d'"\itA
6�ATMIA2A�
�N�+N��
������?�!�t�l����������?iACla��2?�8����w��4H�4�ST.	I��<��g�x���x�����w��G���<�-`T�ja����Y�be��y�����Wd���:�A�p���uf��E9�cm�33���RE)��x�*���m�����$}YF����~ ��P�Y��$�-U6���a�E8�l]f������d�fI���J �t4>�U�A�C��
��18!i�V�yT��e�4��d
	Y���:$��>���
x�h�J�Q!��*4�c@��%��(�cY��)��&��"R,���A�\�%���:�[o����������2F�<y �`]'����gZV�1�yLLrAR����`�g����0��vQ�Y0��	V�/�5�@���:�z��y�+�7N�,�O}�E�0��E�������|6�S�Q��Q%{��VL�o��{?��%9v�;�i�%[[�0e�~�����Pr|�ls��H��M���9���D�����z�c�����@l���!L�r|6���Z7��ysxrr�rT�N�e%�#������v�4T��S�SS&��`�/�>�D�&M�4�%��V�D�=��%���*1�����/G�/Hx��y���[����B�a��!�UJbaaz��c(����������/����-�n�W�mh;����-�����zB�$Kg,i�(6#a����*��GE�
�y{*��������wK����o,C~���%���l[�}�.k:��?IR��U��1(��,d�c���ne.siZ��lkn�M������U��k����_[7&_��k�V�C���[SO�����_�{2�l�!���!�E������������]F�z�[�c�������P�� ����Es�����0���Z+�]S��-�t����F~(d�b9�,"q����Qq�5&�����yH
�5��]��VU{�J�B���v��H��x�j���O^�
X��BV��m}�����������@��N��LC5 �2��������^��e�,��,���|�#��Cc%���C5A�x���~PKu���l�$�J����q�d����-aaNh��I�0B�T)���#�8(��$E+�h��\5���*�	��~�����U����;���Y�?Xw����$2&�g�����4���"5�rL�E�9E{1��I$�����Dp��Y���)�PVM�A���4ei$��,!�r�8�(�e^fK-
��4*/"�1���n����z:=��#��~4%Z'����d�/���`����G�[ogG�M����WT]���Rc��Xu���\�������o�������@W�����o���+��w�����`����G��#��n�]���������r��D���:f=�+������z��7>��@��qO6�o��lC#�XFy��B�[����c����j
�m
�W���pK#%x��� �?���&)���,/�"�Um���:F���n��;.���.�l��o(�y4'� B��p*S�*G�T8!��l^j?�6M���2��c���K��	�h����Z�����>%Q�$
�q
��Md������~�X����0���<>g&�%��Y���*4�_��U�9hT%��gS�������F�OyFjc0� �M{�"�z���\tE�w�_��;�tb�;�����	��������$-_�����c��&�WvqU�����9GO]�_~i����Gl��_�~?w4�6�����[�B�m����@4�S=��(�[�2��,�@'[�>gC�,�N��-rx�Y�Q�?�d�=\�
h<�?+���f�H���t�)/�5��
�{^��Nx[h��1E��V��?@t��b4�
�v��	�B�z��TS������=0ZO�L��77w��Pc41�5?VF���9�8�Mj��Nq�v��}��e'a�"��{��N�����k�F�0�j�4��d9�`tK.�'���Yq���zM{��b����<�ht��"��m��`x�B�I��+c�Sm�t�x��(]:���-�[	��+�$�x.�H�~�,T��L�	��9+������"d�hxv���y����E�V�ku 	7`��^�|��m6�;�R��[![��D� <d�t�e�j���"
�r`��W��t���_D`��5��J?�.��K�I�,]�����q����qY�A^��U�xB�JD�pE�g�Y8Bv��'�����[0,�XK��"����a���}�}�1�����f
w�I���Yv�$��M*�4���
�i��
������Q�\6��u�xo*+��N~�K��
B��,Ub�p���(�WD�������hg%
�������6�j�Hr7�,�H�;�&���O�[v��'"v*���~��1!\�ae%��(��R
�x�}e^��@R����6�a���
�DrK`��T!;!�K��re70�W`�#���.��"3�B�07v�bV!r�b)��t�y�`K/3g0��Etl��K��	���B�T�lEe�7�L��P�1���
-�����T<�
�i���b{������C�%�����r8Ai�T�_�g����l=��e�8��9[&a^m`��jv��X����H5�0�H�"��s:c�(��"*�]
��an����
��'fX����T���K�������>�k������;:�6�����	��zP�M��7��n�M��h�
��Z-�>��6��OW�6>.L�i%5�s��s`&�&>^�&2a�$-+���*������a5
8X1�����6���!���	���W�L*�}Y�z,#:���6BV�jx��������Q7w��U<�����������'�����E�����6�����G���g��[sS�EWl�������C�������}A��Y��!2%q%�c�oU4k�����y�F,��P�/D�u�;.c�3/w4<���T�J���n<��Wx�����u�������a�y�����|�-����&��l����������-�*�1W�g�0��q�D$�����3y�t	���k�q'Wrc���	a��R�O6r�n�p��D"[�w���:om�Z������j�g����8Uig���&��!��#�
�|�������9)��+���Mp����^���g0����'��?�o^�����V.����Q�����JC���
�>�� Lu�ZY�����s��I�.h]�2��(&O��;�%��*X��x����Y�|X����3$���,�Vn���}5z���/����N�5��C�6�@ �������TcnXh����i������+�yp��}�>������!�EH�!��T]p
����<�J��.���Y�~0����/���������&���^��IZYG��j���k��i�k�������<��	4[S����J�V��Z�T��L}n���W�>;�U�����/W�Z+T��T��^��6e���VMK�
�F��-@�sU�
c^\�u�$�&����yz�<\#����Y��� d/]��}s'���_�t�b&�\v���*_p��AF�4#��P�m�Yr*�8���"[$D�S���<u*_J������
?F��0�3��1�Y�jU �z���P�����GFW�R�����-�����Z����?_�;����^�Bv�����U���M6}��.r@u-�^�Y��b6�Ru��J��B�oS�[�a��W��U93�7�:����KpGa"�����B~WRS�����A��#*��U!6\Y'����4��
k��Uu��������pi��L2	��3u�1-�������$�����O���h�r���ua��HM���F�����{I�u%o��Z!���P����z5<����o_�p�[u���8R��_��<��/��"v4���������!���q����,��NJu�+�i��� hs�5��gl���;�b
���q�M�C��f64���k��\��r�|�ty���G;bVS��Z���?������7�?k��`)�aCT�2�Z"��jz�B�[��k������l���w�����"�x�J3A.p��>�SA�����#.u�"g������#j8'���<�
F�L�A�a=!X}
s)���ps������uf3��W�y���+3��(�z���e�5��L������n%�by��
vc�f'��1O���+���Gg;z��L�����$�I��XQ�2��K��"G���_�)�
X�|�vB�18<)	�6��,��b���T���+K7�1���������6�zjq���,�-��Cd����>�f(�)}�B��'��f%6�	��Egap�x{� ��m=0q��<��erYW��y�.�A��D����{�c���xF��+�aXF������f��
M�N���3�u������-�����|H�*���K�
�97��z�V��������7���wy�!k?��gI�7^�B�:.�M���*9����m�6�i����������[���D.�o������
l�tQ\�7�#�P.�v\���/�Njt;��!�]����;�a=�Q��k����z����������;q���;:��������y<�R}��i62�6/N�������;'�X�	�2�=����t���c��2:�Lt��+C��|-�@cN�����F5C��m��MH3,�;@��
���p�U���.Q@�T�*-�g��I�&�+��%&�Q����h�K��������}uC�g��i	��o���-�����r�����v�J�<��{�~��t���5���H�2������I��
�T���t���5r�Y^���s��"����+��5�gCl�j)���������1�X����R���p�$�����\��{�N5����\�$�o�
7�����~~s���(Y����}D�#�;N��KcO8��Kh��7�n7���O�����>���h(�����~v8z~x�������5q��o��)����</f�/��I���ub��<�J�K���(]�\�u��
�C�����vE����+j����w��J���H���gL��<*�� o��Fl5#	��Yw��r�	
]�N��	�d<��(�����[A�;H�r�]�[1���/�R��}Tx��M?X�y��zA�����E��RL�r�(�i�.	�R��T���x����Hd��[�{�r�u�q5p5������q	��>o{u������;��&�L���:G��M[k���,������G����#��I�,K�I�;w���RR�x}���L��
��*O�[�%?�2�	���R����fB;;;��^�(�t�C��_�GV+��NU�	���dq)���h��NH���A����v��.��^)@��6�a&���)-f��5�Y����-n�����	��j���R���a�Nq�[h��pET��� �
��Hw�����qj��2�� �����jL��M�n�#)���I|����8���<>��2��3��b����E�nc���Gv2q�9�[�3l_e5l��34kE7�\dXOD�O�
��ls��[[�)+j�������g�z+����y��y�� �1eW�k�U�� ���������(��#��PNQ��������GU�>�.�����%������O��#v��.|BO\�;'4[�P6��b9>�A;�����[Wp��T�9U�����xZI�_V{:B���
v�
��/ZK\!Ir�v��;�+F�4������\�OlzUN������W��J}��t�F#SY]�]�]��1".S���o�v��m�_�d���v��7��nC��/#��\����Qr��
��	]��ZW
����r�7������
�O\M��<�}�����[�����I�@���MMK���{�a��F��[��A�� z�^JBh���~"k��\�w����;��\�F�[��t[PR�����\�	������0����t�s�L3v%.�����cl�L�2�^��hc�#v��,^���b��!�B���c�oe��N�|��W�J���X�J��W4��N�.�B}6�P�%��,�i�����<�,��d�!�B��6Us��m�MQYm���|����/`u6|��ti����r�A9��.����u�4"P���?������}����x��&9l�E�]|�5��%E��C$]��7�A��AWz��h3�LE;��g���&B8!�v�@z:W���9����Rz/����������b��]w`��/�������4�z�|�d��KV�@Qq��+"k�gcb�#O~���"#G��t�b�EB����_��!Zf	�����D��a:���}�`�vf���[�
���|Q9�CgE
;ko(�n9,��hf�������+���.V/����m�*T���������=!��$�z�����j���{�>{s�7���5�a{�y�@"I�0>�"RN��<Ds��r����^���������G��S�e=-���6^�#u�dFLBl�X.�iQ[N��[*��y����
��P[(p���/���s~�6V�����Ro��"�C�e��AC��!��Bq��'��5u������
��}���y<��6����FV`��t��-�U�P��^]����{�W�^�=K)�8���|�x���W�NM�	'�:���g��y�O����U#��w��.��-
=h�c�)������
1~���
E���\KBK�����f�F[������'w���v���cy��5s�\&W�7���@\�DW�{��TS<]p%_�L��v��N��
]���"�$�,���H�?q������%�m#��
z������1���S����,�O�}p���0Z�-!���U�u����@1���B���� 1�5���0<&��m�`
��)���v�B����&�I����N�/�mp�6��zT�p����
[�}�/��p��_i�\~�X�c\�^M!��������1�}R�
�D�����eBQI�@f����Y��CV��H�]}���$p�)��=p��j���#����6r5@,�a"�O�������C9|�C�=./������,�I��{��L�p�@}�O������q��a�UHg�k��\���Ct����@!iZ$�6�?��`5+� )B{A�Do�BF�����PMiU ����S��fX�m���nm��@�$*k9�TD,����P�A��[����u���q�]��9��yri�l�x�V��������K��Gs�q�����$������rFr9*PIc"%��e��H�������`�>�E�����_S1+�G{��	�$�N��P���fv��l���>�(�9�����\�0�EZYA��v��������dN�5N�-�J�*�/�����Y;��| l���vU��%�p)��E�7���U�cx&�a�����zRA���X�Az	�!�PF�p��v�i\�O|�������fJ����L�b�$Ke�z��]>i�:���X�k3���A��JN�rW�K�:�Y�bew��W�[3���E��X�G���O��	f��b�c;A���
�NDne����zv�A������KG��c�p�R�apzYF}�!o>^��\��U��{��(����B��P��Cc��@1Z^�����u;������
,A8%�������0mUf,`c�=i�c]"�J��PX�O���:F���U��01^�������/�["4L���H>��4J14��������Fv�_��P������c��Q�=��4����lh���Z��~[�*���`kx���l�V��~�gl���+�|��s���p1��������X}oYQ���o�_�iO���b��7�;h���[[�^�����{��>LfYN�j�s� #��-mV�4�S<'�%�����=��,W=n���6t�&}���E�!!�l��q�O|��e��ilR����v����'=S�����^�A��RN����y���C��A�+P��*V�"��`��x7�y�����]q�����n�����G`�o����L�l�������������7��\��RV�0���,x}�|J���U{�q�V'���m���}��v�
mQjoSk��p���1��[��3����z�5�����WY��4;��;2�V�Z��$+�I�\�<��>�b k���m"�����{��l-���92�0���������u;}w[l$��[��U*tm��������v�
_�p�]H���m���v���F����@]�P����2��ABE���lx�Of%�l����otn}Y`��	[#q����B�w��T��]��2[�a!���iU��[(���p�$a��������V�)�-)���S�V��[�,��wF�P|�Z�/�F���<�O�9H�u���8�����[��D��t����Qw�-g���H�~����A������w�*Y���x��Z{G����H�H�E�����gE�@�����7G��k�}#�t��&�����P�&l�`]��1z�������cuC@����I�H?��n����
��C��Z��*�6:��������k���x�kIM���1<��}
�1������6\����3
�p;��$���^Z�S���F������qQ5e�\��|�i������#_3��u�
s�6��Y�FH���ll�1&���>pT1���XbN�]W�c?�A�_�Z�7�z������E���z��7�����~��t�o������[�p�X?�	A���R��g,�D��K�e9���X��5�^nKy�ls�Eh�:�� f�UpE53l�}�)�6�h���y��f����+�����Kr%��,F���]�ZT3�}wx��Y�q����<�E�������/�A����AO�'.O��<F,��������d���������W��m��
���:9_��D�[;@6�`k����l���~�mP������*PoKXT�*P-�	Z��q�$���-��U������92ZT��O��)uA��m���Iy>M����K��$�;��S�jN���3t%z�]���(��
5��F��W���x�>e?k�T������U9AGR���*�h������U�"�YA(�G��"�����U8�+�YB��iT^D��M���i;=��-���kE�$�t��l�yJS�^�>E�e�������YrO�\s������1�=����H03zG�FKq���5�JFOCA��E1�0�B�`Y?�4��zsM��yJ��)��+1m�X��pG�[E�	�@%�T�:��E�%�����>���#SZNKISENG����O�B�a�7`�4-�yd��.�l�����'�&�`�feI����^AO�qN���v��t��g�%���	o$**����rQ�c_@��Nv$����Z -�18��J���c��dwV:�����r�@].�5���x�������������!N��5ygs�����f�B)$�q����4�{$�/���5lo����O�l����a$���+P_��p����GC��m��P��c�v�N(a�����ryb�_��=�G�d�:$��
�u��JS�N\���0��ytgK��h�?K���zq��k�*Jc�6.�c�/�Q8����a*} Z�`��M�,]��]��p�|�'.�:����4�����o�BTw�	�h�|���y4����;�'���F���=�~�D��]�ji��wH�h�c����uX�����5*/�����g�n}�z�u�z�t����D��,�S���!��!����_k�A�M����M�����M.]�r5odP�16n�jT�X��������kqG�'6ak����W��������gGo��?>�+Vy����3�����L�"�5c�2Ku��"P�}�����$��
!�&�Z���ut�����<��{��F�h��QI�������dx�\W[��]�����L[��?�''$a��V���?���9o��2���r�v�3�6��PpJ�h1��l��L:���dp��&����qCq�q�������x���V�)��X����Aow�T����d�Y.k���k�&<��.8lK�����g��Bh�e����]��{����V7���k_��^����G��'��H7�]=�M�A-�j��=i&
�v��Y�4��������W�P�;?k�6?G�B��/~��)Mi���?��rI��J"�-����l��*r�'�yQK3�^�W����q����*R��(����.��U�R�bN+��yH5SKTptw���1�����zL�D�
H�N���Hb�p���v-V!6�X�a��@�]h�`��c_�A�����a��A������Fk#�N2���xk�d�`5�k�c��:��_`���hR��.��u���FyE�Z��jW;X[���.�s�F��
R8A���w�0$�M��q�!��-c��O��B��k�Ei����%�xK��J��%�+����_��P���'�R���a�l�e]�g���/���t6�1�g��������A �����6
����"��\V$Ef�8��������eil�4��e�hY�i�f�aB�|�;�J��C(7��)��#b�C���G��Z�YDA����
[:8���$b}�mc~ws/_�m���ms���{�����S����T�O�-������������*����G4�qM��+�����^Um�kX�'�OkAa���KV�C�_��k�>��A��6A�����1�f
|�r����[&i����J	�E��dm��!��nF
������@ �RI#p��h��9���T5:i��U�(
x����U�z�����ja����W��o�F�ic�o�M��^_u��}��K��6��o�-�}-���m��M���G7���\�:����4�f���>5FT"<�vH���'q��N�(<�DF�e�8��_��Q�	��4�h���T\�gW4BJ��?S	GQ�kvD2�W�U���z+_�(r#`Y��8��.���D��l�2!�d���P�#�73byHz��@N_���x�����&��lK,�gB��Z:u��~��Pd�3�K�l"�����fYL����t�~_�7���B
���{&b�i��!BD1`d�cZ[��U���H�7]����e,2b��zO&�8�Y�q^�N���RU�D������m�DSv���+Z3�0�����p,o$�}�z����q�U���������v��\P�U��B�wX�/Q������E��Bm��E*f��>��>�X���%?ZK*��J
�~G"�S����I�Bl�:j�x���{�=��$3���f����������6m���t����f�����������Hy�@ �c����*��Brn�o�~�b��f[������v�O�������g�
6{;=G>���i��0��4Yi�^�l�������}��/���U�:�zw�?$��
B�[vt�V6
����-iH������z�iq3w�1����_���S�w���	*���a�w�P��X���f������T��f���&s�W-�F51-t6'��-F��c��D��e*:���Et��c����h��-������������w��&^��x���;�5<����ap�|||||��H����������^�CzeH�����������{�|�����������kL�u��Q����'��o�'�O�#�%�
���S"@��!��M��� 4��!��<Bp��9_#FC�0����!�:���Oh�'_c�������������������7��7��7��7X��
��
��
��
���$��{��{�P���C��o��o�=4,�������������������}P���V�C����������@������������c~���3E.=����������������������������`f8���G�#�����~p?�����
������������#�
qH�=>Mn��/pP��_���8,������_3��g6��
qjC��6��
qrC���|�����8�!p�
����M�NQ����8�!Nr���,�8�!Ns���<�8��w���g:��q�C��;�q���G;���gd�8�!�w����?p���5�)sk�����'=�Qq�C��=�qq�C�'>����\���|g��3�������p�{8�=������r�����m��3��{�Sn&������o'_O��|Aq�{8�=����|���|g��3�������p�{8�=���|��������C��?�Bz	���������e��/p��ng��3�������p�{�}�QQW��{)~n�v�q��M���P���w��#�9�������N�z�zs��5���=xm��?PG�;����w����1��7���E�������7������?t���X��s]�8��\��|Gl;b����v�v
b��A6(������v4�������zi��6�fo�D�Q��rw�����^C:���5:"����vD�#�7#�7,=�����vd�#�������*[Gl;b����v�v����:��Q���v����k��7m}����vD�#��]'RLZ1hY�h"�t�_Dq�*������]tt���������_I��������n�]���fwF��wt�������Xp���E�(jGQ;�����thA��zmQ.�;F�zn/09�2�q�a)�)�=��H����Lwd�#��������V�^����������nW���=]��2�e����Nok�p[�k��N8��~G�;��Q�R}����"�U�xTf�h��%H0���=����1���5:j�Q��Zw��f��w���L���f!*aj���p��y7or�fn���ox��=����BAB�I��7��;I�-��4;����d�V�&��S��D�IH�<�1�\t����V���[�G�9F���"b;�"	���glL:���Q�+��Y�,��p��{\,P|�-K�Nz�x�,�ny�����	��B�q�L�v�Kv�.����~����`I�����$./B��yx)�5�(-��#��0�mHN�Q���,e<&�h_n����%��n71���{�0s��^S+��/�T<(`�G/���G�8z��;�~���n����y��#dG`;����vv�$":��Z�Y�>	I����w����%��*��F��(qG�;*�Q��
�o������G���Z5�2*��pC��m 0��:���d�N���A`�	.�;�1q�F0��q4�������{��������wd�#����x�����N�R�D�{	��#�y�����,���B�����Q��JwT�����.���~��\��wi��K�����C�]$�w�������wG�W������OD;�(/:�sGK;Z��������u�BG1;��Q��bvs��R��h�/�*�#�9��iGN;rz�1��:�9��VCkp���K��/]��]�:����
�hxG�;�?%I���E�(rG���Sd?�h*�������7G�Q�@F�iq�{�$�iX��I�*�H���2�/<PB�q\�4���4r��gy�R������N��.)���u��{G�oF�W����_D�����E�3A�o��
����9<99|9z}������/�oUCGO�g����Kb��2"NA�B�z$�`A��������jX_b)R�dt�e���[t�������=XIW�'�Wy�	�3�I,��4�
J�~XD��f2B���6O���Rz9�A��<�,0�$CY,U$
RGJ_�����&v����8�
8��V��<u	|GP;�����v�����m�I�E�R	I�n����G[A�B���k�g�<��a�y'(��ky%��@4624��������k~����Y�B�bR�u�_�-���X�BGAX���hcY���~L����%C���%E�B��#�$�0N�Ks_FA)\�d����2���q�$�n�C��0Ul)���y4.([x��t��wf��Q���"$�aL�l��/����B��h��"�RE)�Lp��<�o.�����7�#����l��Kv�p�[������:�HO����D����/G�/�q���~Pd�EdU(���^.F�F�*�����8�"`�i�e��&��b���8Z��n�2��8�	 �E���Y���jC��O#�b������8V/��� g�KA��5�/�
���5��i�c�z�I���z{����� |x:x��Uo�p����~E���n�^���E�E��MH��
�kp��������?�K�7�Ge�_>E���,/�z���!eU5d�"P���s����k�Hr�������>����>n��{)��#��)���������y�lD(�+��xz����X�W�EI�e���`{����O��i<��������w��%F��V�E*.�h����m�����?���{��D��e�5��[�|��������t���4K��,)���5���0��;O#B6z-;�Q0_��.���s�A0,��)�s�E����R���M��J1 � �s��h�J����v6�Dx1LE�Ey9��/{-��<����tF�8���nwE�>
�g�d<����3xq�W����4���9e�9e�4���v���u�N�O[[ISZm��*.���NO{.�H����i�5Y��_�0��3/�)m1�
��:F"���,�sY��ea}����Rg�������>��w��u��h��Pl��J�B=�b'F��)f����	;Tj��K346��r�7�����(!�����,�i���@��h�O#q+A���@vy��,�,��_O����%��a����&r�������%��Q���!����"6���a�oR����Y�@=���[����5��m�\�����3{���S^K��e�%\���*J4o�����J�fqK�JJ�V�G&��c�m��6������+�u��dQy����d9Q�cy%�d�sj��C	+�@3���������e����<���F�`0�W�wC��C6�!�8�Y���m�V��+�\=w�d�tQ���6M����"�g�V�J��[SX��>n�QC�[&�$_F����i2f�bk�X��C�q��[���$��l?.��a+X��5�9������^� 
�>�K*�����N������������������5M~<~����o^��9~u����������G�����>C�K`vf�������\��&��	sSC�+nC��V�c�mG����o9���� ������M��1�+�w����+ED���x��=����y���P6Y04-��	���~g�KC)g�8���j�l��K��^�I��G�+��Tey�R�o���W�����"�������6���=|YG�uI�S�v��������-�
��y*$���S]Oy��x���F��ruaj}e�����$���
�&U 
��_���\�������^mQ�;a�X�R�m��5�"��"�����5�<_�I^��ql��"�����������o`������3�'�P/���G�Je��
��<kT���Ys����i5G�]��F!��l(a��|��e�w�a�_�z����:��+w�o|��}�F��M����8������/Z�q
������Z|�*�������N�n����zl�"R������^B������R��[�X���A������wZ�?�j�C�������0;v�#�2>da��������;�P�pva|�����K����h]jQo��C�������w�9��w�:��������:���XQM0��h��"6���$�QH�I#q�J�]:�6���>�����	��=����V:
V�����'���3��L���'ghee�(5oMj�==�����h0>R���rO���N��(����\^���*�q'�g���z���,�v��2Z=e���Sv�������'�{�e���\�@�Gmf%�$U���j���s��QV�b\-
c��'��x���d������*M:e���yBX��K�L�3�����v��%W0�5�A�7�=�oz���������S��{9
���&�����\�P���[��S$���U�e��Zq2���=A������m����/GgG���3����n���>X/������'�`%��A�}+^�������[W�(H�����"�����,CdG����N(��$�����98�=�2t�f���h8��L�g�����f����}6O������P�=f/�������O��t�rt���:I&���L@m�����a�8� vI�x�Jmaq��b�f�N����H���OkH;;�����{�j?�n������Y�,C��ug����[�v��wVP���l��v�����A}{<w\a}59�N>"�h���=qt��t!����N_����|Z���-�pT�$h���"�m�'�d��dT��_�_�8�N{�*����\2;����Ph5����;���m�$#�u����z�v�����f���~��$����|Lb���T>�QD����'�x���y�������9LtY�UM$���u����]�^O��</�K��xAC��?�������� ����@v����d,��yZ�E����U+��:Gpu�M#���G[[[�5���k5����<eS
����q��+�T��Uq���c+$�R����CJ�� ��zRd�*
KL��k��'��
�}�Y��ai�I��#V�<e	stE���������A�g}��b�9����wEX�����d!����~��IQ�(�����c�x,�r����g��n�i�d���37glu�{��Cq��[��������V��m%?�|�-��T[���S����e�����t=��fp�.b����~����Ng�/O�|c�rn���
Q���l���jee7���vgn�fqI���:��U���D�Vrl�!��"��U	��4|`5p
��!������m66�����	}H�������7�_7<�/�d�`�.s���,����f=ZN1�U����!M������{�A�]�����r��:��h�BAs	�E]@I �}��pv�������[��_kV�KW����h�����?&���0u�,0�F(� ���ep_�N<�a�q@����[�yn�Jzj;�z6��7N��,���k��^����������}����m��R���W�&+�W��0o�L`smmb�A��w}�w�MYr#k]��$Df��lxu��o���F�@����^Lh�W��J���F]Jx+�L������b���������(��qcb%�2��6��$�=C�7I��U��3ZP�mZ�3oZ}f���s5H������%I�p��E/�/��9|iye\`�"����9 Y<7������36����,�7��I8��P���u��&S�A����Q���M��
V;T4
)������4���
K�]���cX�y��Y�o�T 
Vr�f��.XP�t���{��=��uk���F�v��~����e���{���^XV���<�N�
�5���r�g!h#Q%����J[���2j
�k98b>4�Cu�\���w2��w[���^���w�MV�����;	���������V��E>!g��!/X0�>�2��y1�q�j?ICZ��O�6������6��~���2�5�hLg�&lV��i�vZv��/����m7��SA+�l�]6��s*��cE����_�B�$q��#��l�
���W�-�DQ�QW e�y[�(�ZY�� �z����`7�m����1���W�:d�$�Xd��bm�ve��N����6���K�x
:�"�Rfi���yTOiHi����A��C"�K�!k��Lx[@���^veY�����}	���#�n��L�}W�z�.�(|:�t1�~4B�����E��A�;�.���]@�����*3��(�L��<1%�C-�j��F�9��#~{`9uz@��� �����m�e����O����O^E����/(#C�H2�Yry�T�4`����,���L$���e���E<�z.M%�\��0�dV���d6 ������2f�d��a�I��%+��0}�����s���p�`�XI���%_�[Kh"l.�����4�.��!CA<��F}[dF��m2�A��(���sBv��g����f��f����f�J������B��%�n,�H��&$�}WU������;F����O����-]GVDm�� &I�(��%�M�N���}��{���Q2S�J���jn������,����aAkB�m�A�S4������#�%� �f�z��{O6��� �V|��gw�W������V���
��y�r��+(��,[��1O����0��:xkDn$aEFh���5��d�Y��b�}�Ks�������V�2�\�v,c7{wv�C��|�a��ik��V���2�n�/f��j�g+���	����U�q��rG]yW&�����D�(�}�\�6�������_s}�$��m���V���z�WV����e1�P�k����#���#��{�M"���a�o�WTQQe.����,VQ��(%���W~u^�nli��{�c�E�����qc��H�I����0�*M6���9�;��O�����]����w
���n�~D�E�RI*1�M�
�%�C~Mn�K'�\8���������ud�'������$e��hE�j���o��b����uY.����u��c=�s����$w���N8��.W�jn���E}����H�wY��'�7�|�t�~e
�3���r����rh���� �~Z���L�5B�'�Dd�&�y"��v��y����{�X]>R���M�P�82�SE���i���b�,�6Y�*�����[��O����\�MX�V��8���V����z�Y�	YzO"��58o�s[@����T��	w�s�[��GK�����%���v?��5t{����4����������s�4b�O�m j"o�"��F)���nM���5I�E_Nj��gMj+��4sR����g,�������a�AK`������,������D|%���P9V�4���D�����@�8A\��t�.av�cgw/��O�7�"�\J�a�1�r��1!�l/S[8�y�I�c���[�[Q����@��5����TU����#��$��������L
S��X]n}6?q!��$�	�\������:��?��G����d�8�?sz4zur>:{���������4�hg�e����+�OSC}��{ob��L{d�^�Hr&r���n4�i�o�D5�����W�a���Y���eE��u����tBl���HB�a�P+Z�@��Ck�#&y��#�=��<a�IQ��56��6�������%���{O]����(��[�	�#���qJJ��X��d��$���o��[{�L{tWl�Wy^�	��n�
���K�r�����s�����C��P��� #;1��n_[o !n47�u�����oX8g�8!��}*<cciq���U�6f�\Oh�[�|��2����-=�YF��b�tt��-u�fY����4lC���TM�,[87����k(��.-�����b`e�W��SM��_�o{�\j�]�$��R�_���^��9��s��CX���z����>g��^�T�U���������?�lDl	���t�-�!&[�x�%���_s	����d�a)�;3M��h�rJ�h!;r�9j����o����>�(��k����F�u]YSw��;������J0vJ�&����@�����!j0E:xS{�"9���~��w��{�?��������8;?�Ya2/w��;g�-��V
��H�,���vW������qm8a�Vu$��$��,.2�����S���K6��JJf��R����� ��q[�����n�������C��~Y�������F? Q-��Q:f�5�k�o��$z���rE�Tg����!��)/$z|�p��%������0f�
�EO�LD���8�*�'�a3�<Vv�����k���������S5���k��U^rSN��2.+D���%��'���\1��{a���X-hbD�c�M���)�l��>~-�(��M�����u�����}Dr�>r�`�?��;h�E�`�iKH���;�d����.Q����_K��[��@�<��������e.��R��q�O����ZL����AB���'��<�D��!����zf	���QSocUY�\q�^���D�/���d��MiP+�$�Hu�(E\��-����c9����8�u.�-�$.��[�y���;��������b{znX� ��7����'5?��c�	Z�;����vX�������)�?8���I�^J����T�w�Lc�
���Uiu��:r�=0���bWW���3�P� �X��!�i�)��b���E"�GIZ<U���jv��#
36\�,�Xx�.$�FV+j�l�/��W7��X�k<~�}�y�X	�!;����x��2��S
.�r-���K�8Z�B��f������2�Nb8���B��Kf|����L�����(�������!I��-P��dr�u��(D>c�����
����i�{��Z�!�S��;BL_8���H���2Uz��N��Q��ptk����S_�w	�Z���$2qA$����G@��#U�����+.�2���{u'rb�HP����I�D�����T�����:���E3��8�i
ei��p�L:�.�����:�
�.e���o��S">�����&}�/#�d��;��V��;xO��{��^(�����{w�s����J��E������KB�: ���{q�����L�M����Z��M����x�5�����)W�C�7�
�1P���:C�gw����$�XVp-��:��/?c�zc�j�4*��@��
4�h��F���i4�Sb���3��?s���
x�6����-�[e��Syc
h��bk�q', b'�6*�p��.��^WV`�<x����y�V� *U4��h�BV���`�#[.��R��
�7n�w���� 7@n����
���Gp���������:�p�����-����C�
0��l��&#��5�k��w�M�@6�l��d��p��5���~!�!ndfD���S�h�
?3�t�y����MI�,����$r`��b��Hz���kh���E������;���jKaPRl��R����F��(�Q��o����a��]!uM,�	s6����B������!��&[���\�>FR��$L��7�%���l�h��9�_[�+3��nd�~�\����D�@
5�0P���+����M������&"R �8%	���C,�h'~u0�����R�?J�0�u&4N��%�%�kI8'��{��>����
������;�����$���FT�A�{]@'pTC������w�t������C�Y�F��������s	����G���^LI�#{]�_�w����K����w���)���B�������E6��V>q~#w��C
.\���n��V*Y��Mo����y_<���K�J��?��p���"!%QG|��&q��\����9�r���%r�/������{��S3��U^HA"��K.�jsE��u�g},3��"N��
t-���?G(�<�@+u���A����Ls�:��~
��%�9IU�� ��iB(��y�=@���5�K�;�����Q>GJ������n��#�+@�Z�`_�t����Hqi��wQh/�J_q~���������;�o{�6=���n�8���~�v�\�I��*�R����P\�$n+��tX��������4�AS4�AS�a5����F.�j���V���zw��jI��^[s8�q��������Z�?J>gG`vW*y�:�K���*�22u5������n���c������^���0�!0�0����**w�����+>�fY��~��Y��k���?�r9��h��@�t!�r�����C��y���,��EVl;��>�v�}��t�y��@���q��f��sR����_�zh4j�r7~LWl��f�:�*F�TQ<�R�e��\���Sa�)&]��z�\�Ea9\������.�,��J���0-���XG�f�[�C�PU,��+�*S]��\���&��-�����2���(5K��6��4���tb
�A
��V	a�D�l��v�� �q;�T���h\��"<�W7�_j����X�5u�r��Vzw���G�ojD`����~K{�0j�����c�i<'�b�k?��e��C��E�XI[u[��Y��]d������'|Q���g{
���������c�Q�9�(���%o���W�&��Y^m�c1W��`����3(<g���D�8���s�m�����L���<"�������pH�l�.dF|=�|4���`u��pEi)���1W-�Mj+���(z�����W�/�<?�����w��Gg��k�x�����/��:�������G���|~tz���#���
�����'���n����������u�Z���^�Kk���Y���:{�F����[�!��-��`�U���y�P�(����]�P��o�
x���������G*\=]�E�iU_$
��Y�g�F�.���5B��d�p��
��t���I��P������,3���_&�?����#��)h���%������,����������� ���v���_]d���aHpT:�^n[���IO��V�6�@#�[�}oO��_!Q;b�n������'�g�X�}�|����'�glO������ =����������^��h=�o=s^��b�{��(P�@9��{�u�U:���-�+�D(�ha&Mc�/�l1z�4b��-��OO����OZ�=`{�������������R�
�`7��=�.���y��U_�y
K����]9�*�b�����\\3��q��������:���h������gl�}��4i��H@�����O�������*Oa�
H�0 a@�?�g���n����gl5d2 ^@��x�~�g}_���tD��0/`���8+�l������e�	��H���b�SW=�� @��bV%���6�Z@��jT#X���;�r�t�����n��N���o��|������'��ax+G���T��x��������d�?����MQ�T��e�sf�y��W��
�?�Bt3^���r]r}��L�N$#����#.��3�lrJ�y����Q�d��g�^K��3SE�����f�t�j1L��?R������hn}�y�Tv!�j<�r���l�V�<�������W��*�K������i��+.��q�998m�+AF���I�m�6�c���c�=N���l"
;= �x����� �J4�$+��s��*��j��(+3/��������~D�{dK�r��^�br{z����W������|	��?��8r��,��;R�;�iM�@$sBY�����y2��<�����A'p��v*�M�������?h���k!��x�1���K�k�s�&���z�6�����+�zh��I�TI���t�mmc��\���.t���8���
��������N���C:���5���]\0�2nT����.�U����1K�1n�Wy�	a
�����HI��:AR�)g��^�B����Ji���Y�\#���k���x�����#X"��Y�����,5cB�K�*H������9��l�d����w�Q��4Z���tF���7�#��Yf�f�)s��9����\��eAs�L���@I����L.v��+��g��M_ ��b>��n9�s�3�0�}M���}jz!��w���:��1'`r-3�����6��P.�������&�+
���Z:����|�_��bOz�#rw�:t�T��������q�-.k�LB�������?�q����s���e���v	���$1��R��B&�)q���w��B+~�H�
{x\��k|R���K�6A�~q�����a�����F5��j�O�������FC������H�c���q�F������!�{{����k�C�'��u����������s��{��aDw��2)
�(�z���[+q�E���pa�����u�c0Pk�*2|�N'�P��F�Nk����]f�9���Y��/�_�_jQ�+����X�ym
:��H����"�J��4B�gL$<NQ#���r��I� �J��E9�����N�P���l
 ����C���p�<N����a���Q�Ei�+[}Z������4���b$4���x�����I������T���+�����f?����4r��"�|�+u� �%���&I�����^�����%3:���f����.����}���6O������R�����e�t�����X�Y����A7�\J�C,1��_�E�YR56�64+�2|e_x��DK����y^jY{�Fh���UWn�N�� K���2���9�P=^��CC?Z?�x�Lf$�q�<�M�21�[������`Q��"�����)3��=������������i���M��"[�����s'U���W~;3C1n,�$������/m��91�f�E������NW����,h�o���u|�<>�
�s�(�p�I%��W�����8���>3��I�Im]N�e�^��u]�A������!�d�}Z%E��*i�TNa��
<���
<���ON�/��CPL�������xFd����1�Xe�
�v�����P�R5.�ebY�Y�$�A�,"����C �<�#y`m�H@$tX�!� d�|�,)Ygm�/Z��J�(P:�����:���3���
��7�o�{u"T�T��q�(+S����Yl]{SjM(Qr��!6�DW�2:|������z�@YbX���Q_NLlA:I����N��������.L��7���
����I�M����8���e���|�iqt=��/��2P[�.�>e��X����3y�d���B�?�ckN�Zz��%L�Bg
�Ht*�������R��}��y�~���4�@E����.T���]�����E���
�������=��5|�s{Xv����7AW��V<��67�
�%���z��Z}��q=����_Oa�9�H������+���y��1���_�����0/�
�������'e����7���3Z��b#��7���l�������Vm������c���@����I1^�3���r#���S��0��:O�B�F�z�v^5\X0��~o���1��u1�f���fk���n������/��DO���b��
Fu~��L���8:O������^�c��A�����BG��!5�tL��9�_�s=%���?��hX� ���?�kj]������0�*�S��.��mDg�i�5�;X$��@�%3:���G��hi+���qpy6��zF��FI���btz�r�����{jR���K������9~��~�{.���<�'���m;��g)�ecA�qZ�e�]�����g�h�'�mr!�����GoX�R=J4�������t��4��b5��\F��]=���/6+�1�<�^��r��u��|�S��E�7`/u0 "�;�����6������Q�>E}���^���A�����<��^7�B	�@�0������$��KM�/�/M3:C�?)�����2)����u-1��*��3�LT��A�Z	�PI�Tz��_�D���$�����y6I�]i�!�0������|Z�knr��+�e��AM�c���]8���o��Q9[������w�H�F��������
�pj;3uo���x��?����aTJ��Z��=ol�Wfh��b�N<��%�}��cQ�����e�O32eE8S�R��� ��M[F����H����J���V�j�du8�D�{��Im�V~
��I\�Z��&���ivvN���p�xj�/�_�9<���HL�P7���]k�����8��K��sc�Q�L���f�U���Q�S��&���]F�Ea��J��)�
N�XY8����n9o,7\������'3�.�Z�1_^:8t=h��i2NPRu��Osq��!�d�/�6��A��9
�*P��Z������>��;�gQ�w�GG<c�Gz�j��v�.N�|�g��ox�P]5Xo����^z�X#wd����=\��dZTf�(�>cO���;��G�k�Dp�'\�i�G�����l@d}{��5��������������5j�b~]������e\L����l?un��kK���r+U������"F���}�A�.+;K����
��\��D���*,����s���E�]�����2���&����T.>��loM1��^{��,�L�v-c�g0�B�Q����LiG�-�=�,��
���������E�'"�m�_���"&��v��
�������G���az\��_�O��|��#���\�+��nv}����I����$H���U�����S���b>^	��l9�:J��p�:�u���_f�~&cS����9@p������4��J�&U�k_�� �^!X%��
`�6��}��/��!X6@o���z�z����u��'{Pk\Z6������y�m����c�d�������>�OvT���6��0�$��	q��������Z@�M<�v��*��Ma5	�$P�@M~�����9l��`���o��`7�n���H���s�7vR�@7�n�?"��#���%%��\�b�-y����7`o����{�����r
�q���J�H�o��^�7���������u���5@k��?"�����?���G�h�x^��"K�l{�_^�E�I�yk�����o���|E��5a
�gW��!p��D��j���>�}���������4�[�Y O3
e�����v�l:���������������GS{'	��Qt�/�$����<�[Rw���n�p���+TuO
*��lj�N�t4-�*�$3���Ey��Do^��B����Ui��f,�N��
W�<�8/R.����V�@������c��)R��I�/��8!��}t��2�h������������s�b����$�N�^'e~@+q�T��Ds�kWq6I�S8���qK��Tf�s�f�|��1��>���c`�dT@���E����u/x����\���8��[�@�e�v8?3������W&~{���q����1���>-�=�E�W�E�<�|t��Fm��1�8NS�.����ra�2�!�C�%������iP��2���j
�FR!)y��r)����h���>C�`DR����������8��Hs����J�D�"$����k��Cw�QSv��o
����#�q��"����(+BhH=Z��9���s���m��������%j�{�%4��$�����./�$��u��������C.x"�,��/��5M�q>�{���LJ7Q���1a����UG���\���z�Y����S�������)��(
����#�H��*N���AyS�UU4���*N�����~�����r4]���gI9�'���R���Q�����������f!���g�x����k;��C������}L�g��
���C�N����b��Y�������0-���c�\�zH������n�77�uC��*�j]�-�����p^i(
����'��~��	j��e�l��r4�%�6Q'*G\�Qa��A��(=�x2'�X������D���������yj��5F)������/��	J����}�0S��r�������9��x�x6�(�h�j1c8+�{�i��;#�/j�p�3�Pg}V�*;g���n���ucu�kG���n$�-K���.�{�R�g<4�W{'m�r`e|l5)Lv��:�*�Kg�9~�U\D[��������X���z�EU�S�K��,���M_�9-�U\^��ES��]�Z��%�0����7R>�d�E���<&�����dQQ��|i{o[q���L���i�`�(eA{��$�o)��6.2[�E�U�F<����H:��6��k��]���.#�G�:�����C�����Qu3=Z|����Em3����O�i�I*P�����[)�hA������c������%qW������9����D���
�zo��tt��k���hL�tr��N�qM�����Dv��3n�{�%I\������E�h�\Y����\!hkV����������=�W�D��c�G:����8���&����R���Qv��y,�������Y�>�/�# 6j��o��Qo��j��REuB�t4/��eO��k�>T��A|�~A��m��Fy��,�� ��lc/�������{�|�o=}$
]w��\�a��H�������l�Y��Qk����0��P�MY�|�%y��/@�<
J�cy�������3������y�)���4D-41qZ��v�Ai�>�����X������&���%LON��F�E�KMJ��#�$��3TVJ,U�.���aE�����*��	>F*�Eirm���%xy����Uv��%���zd$z�����J�^�9����Z
�\��U0VAV�U�^,q�x2I������!������o3�r%�v���~E]>��4�=g���5�+����E,[�8���85��������:��1$Vu2������T\��d�]����9�^-�U%�vz~���l������*�^J'�����d1��,D6��&U��ak�qMP>:�L�_tJ�j1�r�<}G�e�k[��N�M{�������(�X�j�+
=ftmgW�[:(���%Q�|���&4d��v3W��G�)�B�])ek�{R�+�P�U�Z�n��30pWP��B������9:�/�hW5
�Y�+o0]��tX��(X�ux�.��	�CM^~�E"*��'��H����~�.��RQW���bFT�'�-C���,'�f~5�A)��P�$Zr9c�k�Xy��c��Gqb�A6..i�h��x�gF��>��`E@����v���p�}��n�j�����i�����������w����}e�i�����W-���K�	�X��'O�j�z-z��B�V���wvt~�Zk�F���%7��@���M�cf�Z��A�,c__��
AY��=��Y|C�����+�fd��E���x�Rp	��M5������o^�>:==z9��������X�|A��D_E���B�}Z�QeB_A��452NQ%e`
�Mz�/����?�E	d�H.����o��_u5�&����&��5��`o�-*�rn�	���>��_�X����V��������VP�����~I��q���Ume�6{��A��Uo����1U.�,"�]\[�ZL-_�m����H/�$��+��Ii-h.��[�Y��-����^�}���(c%�x�.\�m�,�Nd��b�|R��Pt�Q
V�m�\�3�yp}����d����Q�*����m��]� �X������m���mr��+Ug��Hy����-�*iiA�*�s�-v�nZ����`z�k�7�'��+6��D�BV�?|�=�B��:�Iww{����t�HJ0�ov{O��1o��d����~����*��AIz��=����^Q����ixfR3�����z�G��^�5���s9��O�Z�-���Ed`����9@y�0���=�O����������Ji�EEq���f}j����5�����Z�r�mYa������\�bI������=<{�x�c����~�Hs2��`�8�^���5kO|Y�4�6-&��H�����[l��D�������Yij0,����t�+���b��E��lMI���V�3���,)KgNM���
�rN	��_$;[ �Kab,��sc�� ]V�X	�vajcz��e�;���d�M����j��,8�RvZ��W��
�6�L�U��3?[8~]��e2v�5|B��l(���{w�.��,.��;����1H�]���e�/��D@�L�G23�)tji�n9[�~�^��P����}"�.��y�$�N�I6�9]�ji��~�X[��kn;�p��'��Li1�1��~���Q��$���3mE�qFl�I���]c��j,�nt��N�!-������@�������-	�e��!Px��A��1�!N4�h�+�i�����L�G��D���GXQ,x�K��,����1����K�c_���j_��a�"�`t����n�w���$��/�M�D:]&\jO'	�T�4/�q�Cas���7ue�D���zR�r�F
�%�`i�^\�v�b�:����#aD�����dy=y�
|�;���n����W�� ��m;p�)���6�JE��e�_�C�����bs��)I���!�����������q\��I��{l^�%����r*K�������E�)��,}z�z��]|!���Z@�V�"IK�e�����A�`���^�!<5������s�%|��6^p��&Z����]��M@	I����*�T�^�)�d�\������s=<�-Y6�p��}"�������
�����zQ�����?�����A�Y7��#�R�R�H�`BP�GV�-��C�eNK����>%v]�~(����h1gy��8��PQ���X����_�+�����5S��6��"��H�>Dw�gDf�i}�tg�1�H1WQRC��d\�igI��|s����.Z�zA��:�w�T���6M�l�|���t�����Y>�/�q5�Y6��,�����_�����M�rcwu��U������v�-����G*��gX��K�0��������R�H.�VD����j��0���;�7-�e�u�:g�n��0 8�3aW�>b^V��������:����n�'�w������������>���t�������W�����e����wG�Nk�D�,��t����\�g���p<���#+��>���;��a�J*X��M2
�C:�v��E
yP`�1��0���r��It�H\��<q��i�4K����;��M��;OK�B?
��V�:#	E�B�'@���[��,+e�l>>�%kj<Q��\����A�q_wv4
����Kcf��U�q���P���QP(��`OI�	V�B=`�eK7�1x&�l��vD��g��P������F9l�Qy6�fi��N�Hg��g��jP��?�Li���r���EIXL�.K_eK�T��8���R���������X�uV|���a���y���E\��o~��&7�GZI
?�j��~�9j=����?��P?m���N?�}m�k�K��//�+���7�e,�}�>�	���N�a�&:���`A���w��f�)a�����m�����P�ZE�#m*s��>j	�]�1�M��f�����-9OX�c��`q��$T%�v������SQ���#�����s,��lt{���U��>�F���x������|c��7Z����)	����)�Z�
K�F%�8	%���-�<����
��)C-���	r����!�q�����*4�b`]�q�'�-��m7w���+����4j��^��7b-j��&����h@h�������0�\��������F4.���>2�@H��H}���J���t�r���B:�D������eSc���&h���q��O�J:�#���5�bQ�f�����n��_}��Lm�b�X�ds���K����*!:���
k�qG���E^�+?�a}CbiD(�p�I�[���������B����?&�[Q/�*�_8�/���WZ��T��wA��6�6\3g7�,3q,��|�19��w��X�j����a�����X�h~��(��T@��x3��N��i������-�-�+��������E���Q{����5�45�����*�X��������d����8{u\���)�6z0��G�;��:~nGF��K����%\V��3
l��F�d���\�q��������a���%�����{�*wv�\
u����_��<31���*����#��e@^Jm��6P��hX������6n���	���{z�yO�h.�P(�&X�&��'�`{�w R9r;>���g�������)��>���Q�>�k�3����5�1����V*�q������kEM��&��2m5}$���Q���#�V����P�S��>T|Ix��2M5��Qx������t����h����L_��Y�j��;?W���d,���xm@��O����x6pX������m'%�h�z��}1/�PO](��j�LQ �?�<?�8�?sz4zur>:#�sB"�s�7�����cgLq>�m���I��{��t���d���t���~�vU?{N��g��<�2�����T
�02�a1��8c�t8��r���Ok!;�L����(�8�'>����87u&�~�(���y"�
���b9���t�D��oW��[�c�mh�hf���)�6��[�8!�y�%"x'���n������Jb��������/��(�7	�?����+��h������DV�)���-mp���0O��������$��b���7�����2�S��,�$������ck���X mI��w3����
�Cc�������w�V�.�a
u$a$����j+�WHFT�����u��[������\<�����]�[<��)�]J"��:��--���M�OS�����:�EW�+(���M������W�E�`����^��e��B���&�!^Us�K��o��2I~��wI�$w�v����n�4xc����>H�
�����iW���E��l��Mts�sF�s�����3S\�|7�(a>�]��
nF���{����F�&�-�A���0����c�q'Q.4|C�� �.���n���bN���2g|C]����|
�/������Gv�m��	L��~���������F���cSO�����n+�U��Po��]�5��#f��)Db/�$���m��z������X^���P����y���N��G���<��|�N|z�|�|�C����Qw��uX�j�m9����%��w0?:xazj'�g~�~�;/�
���{$n[`�����`�9�[^l/����j�	B��������&L�X�*.h��@G/�t�:����C�^�����r�����/��Sq5�n�8�Z������i�j};=�����:|D�����uY�+�(���E�h%�&��,<S�,�3��t�y��%���i.l�|���7�$��tJ�������$4�I�hv<%�
�j�d���f�AkP���1p
�����.,��S�����{���*3�_�h@�o�~�`�����)��p�}i���7�E�����P��u��.���b��g���l��6qM���x���N���#��2?x��7 F�\�=�"��V]q|���r�E��hY�n�����)z[��>q��a���Z����������I�8fy���S����O��M6A�l8Zu= �e5���Mb���k���S1��� �K��B�q�6y��X��@���rc�P=8�O< ��f��H�\��fI��������"��?G�a���4jl^��3��.U��l][����t�K3�4��p����h����aS��G��9y���
;�]��@�f+�����^UBVG�F���"��1�� �l*��n{^ Jg���G!���A���q'���L����|�U�w��w�$����W�Z�;h�>&Q m_��$���]5'.�����bS�d��m>�^l]qR`�p��X?�U��)
��}d"X�Hw2�u9$�m�;]�-���l7��QVt^���@�u7t:29������C��Xr"h��Y\UKI]�@�=�b�@L��=6�x'	|0��1(�&9JS�
�Y�W�~�s?�?7,!O"������H���G-��/�e�����_�;���8�����-0�GF"�D�����S^�%-�����Qnw-���F�AwRCi��W�(�i	�����>J�k�(=��J���������W/c1�����%�c���-$�x��m
��/�4�:�F�}<���7�������Xb��E���+����
����X�8������u�������uA�~��������K�v��k������o��z�,	�,�������1�����A���"��:pk��Q��E
������^�����?znjt����+]gOR�]�cn��������O����v������\�ww]o�`5���K�)8�Z�G��"$�a+Io�a�
�2���Q��N��8������g}]��$��c�������o���Ll��9m�������$6��r�Co9�H�6�P36Va����lZ�[H�nre��8*���|�i��FHE�tE�&������K��:f�;AR!�����o����}���B�;���9���LK�y�����h����$P5)���]<���r��M�D��E��@x�b�������+��a�R7WB���H� ������wL�d��(��
���O��f;H���x�s�Y�o>�/?�l�B_���J���/��L18���"�C /�.�?1/p�����Z��5�vy��J����rc�}�?�Y76��-��yo�����2	�?������*������*��x�Y���� e�t�t�`0JJ����v��$k��5���C#M<Z�R|M�Z)=y-5x�Kh�����
{��%8T:��_�8�2��	��7J�����e��a���92�/Hh6*�����9_���%QE���yQ��~�7Y5�X�a�6����E]���&k���U7n����w.���t���q�e����Ooh�� ����V��O6C�s����y,PW2�{����^X��_�;�!!�l�\:���Y3���u%V��}	gf=�	�#��l��B��b�E33���=������=�D�����lN�?;����$���0��ju����%IJ+2[)��&��d�l������*��D����tP2]q\c���-xMn5���`�3������3i�V���������]��TNc�,W�kun��@ow+H��#m����j���N%���g�_����x�����@A~	�O������+n�gXP�7��qk�����h�#��?D��v�
�Y���n9;�~�U�@�����IX�V�����E$�� �Zc��i7�����pW��w.�i����vCSi�]WL}���08���WA��������GeC�a�~(9:�`�|���q����;�����������B������bD�S#���N���S��g|��<|������j��S��J�z1���3�;��������=�ds��D?k�0��ga�+��=�_h��F^<����	��=/��K���#f��|0��9po�p�3�rX���e����-���V=.����7a�3&��N_^b~���gC���:�V+�%�E&�������(�����a�����W�
i�'���������W�|�F��Iue�k���O��G����}o��)�R���G�D���8�g��������M�S�!��>�T5��Mr����w���6O�����f:ap7�Ev�PsQ6����l��jdw��Sc6����o�4`�m���3�f�WFA�]q��/mx�l�#3��7�s�������bq�)����C��/�y��u���T�J�!��b6sT�S���]�41���s�W�I"@d�.�a)	������%�=����y��p�c�"������Z9Nb1����i����
kFp��e��Njk����l����h�����Q�$�[bF��-�S�3
�������]u��m����u��^���z�~ q�%� ���M�=:l���K#�E�5�;�z���r���q��g{?����B�����������<������0EQ�j�6�w%�H�jRr�����
�r�Q��c��u��T�R��X�E���p������I}a�������9'���x��[m�=����mf*z�4j��*���o�X��+��_o�o�k�Z����H���E&�����@�i`}����aY�)�r�96����<�����%���P�I~jl��H����I���o��8�MEc��d^�����We|"
>�����G������z�n�[���DI����Om&l]�M�������W6,n��P�E���A�%��hxULGU:v����G�?)�zS5�u����zW� �L-k�Y	�k1%F���]�n�$����#uV�����G�t�[�����cz<x/��������$E��d�")��/�S�(O&�/k\t�j3�6�5����@��x��w{I��M���
y���V�gM�b�ODE6����E�`��u[�O����x�u��6����fKf;��IF-��z	#���-R�f����w1�.{})a���3������?��z4�-��j���b'
+���.|���p	�'��|���i�g��is�q��M�������Fqr�<�*�F:�O�z��_�9���0�y��W'?���Mc:X�e��`|b�S�g�#����<�[�8�C�k>�����*�n����Bv����T��W�Gw
6�O6�gM��n�>�@[���� :�+�q��v�>��~��vK0bD�z���(���i��L���Z��,�������� �{��nT����<WF{�@�(6���J�\9_b�)E�4Po$N��\�=y�����c��J�'N<����[����������{<����g���:�>��7�K�9������H��KV�8�y�/����J�J�j,��n��.�Y������K��@	M��+{I��+����u������=�{Az	����^f�!����C���Aw�u�R�d����4"��������n���d�����9/�U]>��hK��,]8�-���\"�+���l�X�����z���4)��@��)���M�d��7��`)Ecl�c��z8���]���G36/��e;AJy����e�_��d�s+���tby'��J�#�A���^�g��~��K�3^Tt4��h�EQIm���YjT���:x�Vo����3�����G����:��o�'G��G#�����G��g���#���5#���~U��J�&�������*&�y��"���5�D�
���F��L�p��g[.A�W��+���C���t�R�2N��bw	���4��X��B���"����G���/7N5���S��	�e��a!���=Y{uUeWOC�����5���r�~�&�/������5�4�E�Y,O!
�(�=B�/H����	��3vA=�/E�L�O����n���~��T��j1�)5��� �Y����|>����������L����5$�������i)��?P�<����c�li*��NI!�.��\\lqkkG�����t��l���6�U����]`�b]���d~u���_#�����[��[��Z��q���vX��s���7�u��e"�%��[s!��j�3)��Je9H����u�#V��i B����j|�����i,�v��^�9�5�:D>Z������O�����E�y#����CKLp~'���*�����a���cZU��GMq���;����������23�	�/�����8��>�
g��Tpy�
�aWBO���UKk�H�X���%W����R$��Gb�i79�\����,����x��M���,;JP��H�|���x�GM��lJ@��`Nj.��v�$u���9������/�}#e]�xC?��V������z�=V
'�ac���-�0��pM�<EC�)���1���j�����6Z�����[�h�6��'�/�D�t���
Qw3���A
V<����z��F]����d���K�ct��8
�&���o�S�J:8n�|9��8uA������0�_67j��[(��������rd���E��[C�<w�@�t�v��$	��W?�L�;�������"�QYR�����4�B��A��v_j������?��s%���
��J�is�u��g���]:���?@�]O������zU	�^w�e�i)�V�n?5����������e�]��n���x��[�%}I�K������$�������X�-�y�"�Z%���jb�.?�S<vu�)wc�4�[��sD��Z����M�3��Z]�I���=3�����j��l.=��a~�yU;�XIl�ki��2h{�K��U����U����m���N5I6N���8+t�S�kz�GK��j��h �fq���X�F���	��)��n!I���^[���5������M;��;�G�|�>��{�������j5����Y��������
�u��e�YS��G�@�P���T�����Qy9vF�6�jCF�����8S�$)a�P������!��n�{TM'I�����Zh�_�x��jPwI���-w�5`���T���������;t���,0����:����g4H��i������k��MT��21�	d�T�$T�k:J2��	��~������)PO�N����!	P,�Nb��>r�H ���l8���:�xJ��g��v�w�6��B=A8p�m���������I�HQq�mu��@�R��-���k�`5x�!���/�����'��N�?��w�����/��/��������o��o���!5R�!5R�!5R�!5R�]j��~��.���v��n���R�]j�K�>�v�S���Aj�9����}N�>�v�S����;��o�F�G_D_F_E_G�D��#<��4�����Fv@#;����hd4��]�C�hd4�����Fv@#;����hd�����Fv@#;����hd4���j���#�v_P�/����j�����}I���v_R�/1j�%����}I���v_Q����W��+j���
s�v_Q����W��kj�5����}M���v_S���(��kj�5����}C���v�P�o��7��j�
V��}C���v�R�o�����[j�-����}K���2�:c�w��;X���{������z�`�w��l
����������
��-�
�I�]�M��}b����!�j��b����!�k��~�o`����!6m�]b����!6n��b��_�����!�o��b���!�p�=b�����|��6r��b+���!6s��b;���!6t��L��=bS���!�u�}bc���!�v��~��o`{���!6x�b����!6y�]b��������!�z��b����!�{��b������|e����`�w�����]��.�|{��=����b�w�|���|{��=����b�w�����]��|1�f�
��|9�v�����{��=����~��o`�w�����]��.�|{��=����b�w����7�����]��.�|{��=����b�w���_2d�
��.�|{��=����b�w���_}���76��2H����V����u��J��"����\xaT��]���"��l_R���Z�%��"���-�5�!�U9��hO!�H��3uGg��f:JWtQE��V����
U�B��L�{�:��S�N�:�u�(���x��� �������7�?��4�}�Y��0`8��}�0�I��
�_��#�b�U���!#
r����4�'4��P�N\���_�W�8�q�������UV��d�Lp��t��/����K����g���Ge�orU8�v3�y��R~�7�6�q�j@R��p�-�!%�*I�9�8�����^r��n&WyjRv���4�z�nl-Q�:M�@4��0)����=-R!�F���x>���ej%XM�9nm��@��~��������Hbr$���f�%��@W����:�H�2V��:��>&�G���H�$�H��k��?$��H��!�;~]�=A:.gY�,3c+�H������B�U�M��Q�}��	�VOR}I��K����$C��9�����|��d#��2G��|�<��y��iO8]g�*��$\v�(����{�Vi�]��wv���m!4��q�)�4��_��[�\\V�<zRi*)�*G[���L���daU�%"V�I�l
ee`�;����v�^�0�
���b���%e����2�p�����5-��'��QI����|�C\\G�r���1�TC�7���S�������G����~�Q���������z�I>=W�\�{r���BQ��o=�G�+!�@A�O���<��6�m���	���*����A��m��L�r�j0�,1s����$�����G�������<p��p��}����^�_��
��9�������4|�Ps�<�q)�d��-��8%-�m�2�)g����4�=�|�������Y}��rq1N���94�N; p@���������!���#���J�o��p@���j%�~�+�q�����1p�M.@Z��i�~���U�;�B\K�lu�W�h$� U�E����S�BB"� �Zr!��8Ptlb�rp�v}�/�3pE�4����4�����^\��K�"G�5������E�����i�����*"����-;=?��x�������4������F���5����R�]�)q?����'���������� dn� �*�FIE��cTH��|qr�y�|��������GX�M[1
��]����^��~�}~�=�o���?�j��E|���zki�VAw>�����������k-�A��
N��e�@�>���r�Hz�����Y\I�x�I�Y�\NF��v_���S$����}�r������2�=
�D`';���FO������+L��z��(P8��}��"��-a�m����~�0�8�S�?���$���30�R�>@x������v��u�������������[:��&	��=�{@�{Dw�??C�X�zN�~�0K��~
�x�\9�������$����I�����9���s�D�Z&>`|�������/��SR������=�f�Dj��)�&��A=kf*TGR�,�b�8w��$����RJ�8	)��'�������>��h���!�B����\��(V�;�wf9Q����$7��G%�_�\)N?�S��@��*�H�"�HE>����������i�4�Wk�����#����;������Z�}J�k�)�mi�4�Y6^��_B�@�|��6�@`�	&�@`���l�}����������;7Y��,��S<KH
B-�j1�0$��Lt������j�]��r1c���[��)OGF��4�+�G��{����a��:�L�3�������m
H�`�F�3�k[��9����}�]c�F@������gu�K,�U��
��`�~u�20��8p|�p�N~iR�����_�_=��?=?F:��������GA��9s����P��L��#|�uVd&0�E�����*@��������)��ey�Hf���0\���5�#��6a�������_�j��py�c<t"��@'�t�>���Am��������o��[����)��&�6f��GhA~�3�8����D����9*������w��m����L�k������u�#�9er�@?"��.��e�3-.��q�C>I�7���*i�����.J�|~��I�n�<������g4f�E^���\s�J}j/��(g�R�:K��H_��!�Yq�y��Lh<4`;��9;��J��o���h�>�V��F���z����c86^�@|>5�q6��G�qB��TvC�;zJd���"��[�ZP����;@�=�R��[�D.qy.���8 p@����@�j���$H�|��,�f6$"�b[��<D�GPYt��9�����/\�\�r6\�	����������9�H���=bF��8��1L����9>�����Q�{.H�Qt$~Gqv�S���������nI�_�ni\>
3�.n����t��),��U/D6��U��$�
���ZO�j�(O&���o�����,���(�?�NmL�"������a��Q!���dzb�u���iTWf����F@�}��'FsM��~�������'V!���c�EY�{�}n3�	=�*3�k/��M�\�'����D�KW��e<���T�����=��g��P7���v/����S�%�FvaJQ��L��4ff��_P�~LR?}.�����m����U���G/���<:>��t::?��u�[�>o�
��q���oQ�?����Y�d�:��y��4�����/�]���{Fs��?��z=������}w�1�V�,�y����=N��5���/-�E��t���C����F����7��L���l���T9������8�_>2�#N�Xs�3�^�z1O�12���i}0z���w�n�U\�h\4�2�
Y�G�x!������"�p��j/�$�h{\ #]/e����4=8��hv%A��yy,6t����#�R�p��@��}����q���|���}��o��=�0�P"����)X���8c*���^�L���5��*��K�L���s�Qd��������2����g���K	W3��++�}�%�{���P���'#�RKd*�LT @�P @�Z@@��!�T�8�����0����t>���H�������p���W�_��*z���q��5�
���
DV����:�[y��4����i	�%��@Zi�G���7f�_�T�.c�#3A�: t@�������\�eTV1�{b��U/ o@���y��/�>~�������S��2�)����s��"�Wy���e��fy���x��}��]���*�w'�o��|��;����^&�F:�J�N�K�����x�2����������.)�O��d���i�%����2x�I(P��+��eW1�T
������g�����.{���v�f>A��e��d.��JRG��H�����
�<��Nyi]�mm���S42�HY�[�*��@��
�*����TmD�
��%��;{Q��9�l=���%���g!_o����L�'L#�{%a@��y2)�;�{��,X�8`�}bq��E5#�f�q)Y�������,+��L�0�>m��������F�:��������������&����~P�8p�<��=��&����Y�m'��s�i>8$T�ub����)�8�X��X�P�����D�g���@�(P:�t@�����s8o:A:F�k|G'H���SM��8��z9z.�b��k��J���g������z��>E����z>z�z~������������	$#��@2��_�qK�]��k����U�~
hp��1���]��Z��d��	j�u�R��Jk���+�W:���I%P�@
5��7�������
])|���V��~%����%/���z�L�|�*�F$�0\�-)�{����m���/�)���R(�i���q��Gi��H�,'�Ym8����	��$��{cf8�Hv�h�\~ZmL��\����}��J�+n�*d���L.4��0�@6@6@6@6t)z����e����/����������E���U�E<�]�:�a���u
�z������9�a����>s�0h��J5gl1��	�QK2_�J�TR9��������)���H����J�����*��w4Fn�(�)���o��N��bD����J�U|���a�5���j&!�t!2u�21�>DBS1��k�+L����Y�N���������EC�|J�|���Z���
��<����G��v�jn����J�2�%����U�`RM%�=H�D��h�������.N����q~b��T�<N:���T=iW]��fV�q'f�k�.����7��n@m����w)�?�������r�T��	
�h�zV��W�)��f�S�v]����Q>�e���~?����Z4��*�^�1�$��c	�odV�O��V�#4��W�A��<����v=���+���)N-�a�m���a���-��U]z������$�U��C{P�W
�u��+W����[�N����������A����������X{�����ds����F��5�,%�`ml���R��TvA��$5��q,H�>�oX����������R~��N�1�S��d��S����d H�)�[i�6��$��d@�������q<�������i#l]Y���UF�������-��i����]��d*��������`�B�����z3�B���qq����+����"��j|��Z�f�h���S^�<8��8����:��eg�x!�W��Jb��'y��J5X�����N\&�������kK����eF�<�����@n��knL�GQ��/�Y�aQ�jV�E�:���)Jyh�l!���v���-����y�2��<����H�f����%3:����Z�"��������(f2�V7h���C)v(������T8�ZU-Ui��������^�s����[n��~]w#�[������z�H���Dz.����L>��3��Z������I��et]x��ndxF}]�#|�Z��=,��@xI��}��V5?(&PL��@1�b��b�Zn!�z	����`/��!{k�r*���
�*���_�wg������i��d H���.�L,67G�tmJ�*�&|���GM�~�����X�$}�!�
z���7��e$���ie@P��?��K���?H<�|�o|*#'�
���RP=A`�w���v�>�`���x5��X��=��` ��D0'�

����c�I���l_�C��*���r��qnOP��3�O�t��1p��q�8^��Z�a��P�`0�����d��'i�^/�J��O������_M>�0��cf���TaK5��������r����������F���������h�=:�j�����P
%O����)7^Hne����k���'GT�����FY�M�T��J/��*f�s�+��1�HNpz��)+��� -~�q�j�1
IIII��$�����U�L�����T�W�!C@1P�����0�Q����V���`1X���a1�$�+I�@b�$�A����E��x=����f�h�w��FI���5�(�0�3��9�io����[��X�lH������e������Ah�����O��F�J�7���'�m��$��)�5p
\�]���J�ZQ�����>�Q4�Ac��KS�E�\�:s��1Y9�WJ���������k�K��r�D�bCT����p�y:tM��y1Rcc�t:�����v�G����v��7qPS�������,N���+3�E��Xxh"�sn����i������R���L��Z%�7V���-)4g�RP��#��_�b�:w��K}f2��,RY}���7�*�sA9�4��PVO&�����D�-�S.^����[� ������CY������Y�<��\��r� <���7�^"X?�5x
^������'�\X������N����:������/��-C����:L��t0L���D�����&��g�� ��L��f����1�/h2��k��/..����\�>z����i;��7��b;:h����}��'3z��1��M�I�Z�7=��oW�[��5���5�<J����x�'��j��m��	�woX3'iv�n�R���}=:�*�S��QA�f1R�,�u����L=���I�Lk����������V���&����'�z��z[�������fP2����E3�����O����;�����S�w�[Os����d���r���-��"�(V�D�#u���N��"<�k8!]��4�;�S�W�/�$7V���L���D|���������
�!�\7�V�)A�FR:r�j�5a�O.�����_Es�{Gy�-���)W>����,O�2]����B��P��H~7��/c������Xl���Q[c<(>��G���:B*��?�7{o��CG��.��CG��/4�� To�}
;{�2���e�N3���F`���_���7���oAa��P��|����$;rv���Z����A��0�����N����I��+J�^�V�� �s9S���=��a��1:��=��RU������l��|����������FTr/��F�����I��������]�q�A�]
d�@6��9���"S�+���=�e������w�i���Tl���@3�4��A���-;�4�@3��)�u�{=��s�=�+Z���S��1@w��� n��1@��.5�!%3O��z�e�o�6?~"�_?������G��Ak����������a1.����!-�b��L���R���~Q����������IW�j���#���
tL���$������"mHHHH�N%���>
��.��(T
m�{�^��S�r��G����(�8���zs���Cv(Vt�G{��t�I_|R�K��^������*E����?/��]��6C��AyP������$`��o����/�����wf��K�k�`�c/�,�\+��a`���p;-�T���IZ%&*W6P8�p���TKl}~�^�������zZ(��@�����������dP��2[���������������EEJ������lN���O~{��l��~���#�������{�������Vv<������gA<��v\���l�=$�X�����f�U��XW;4�:]��2z�����j/��h!�.�m�������@L�S�n{������K��������g�_}uDf*fe�u����������e�K����^�u�hs89?��pFx�h�l^����+��FeQ�[.�-Cs�B�n� 7W��m�G�����}*l�K���)��X�l@7��oi)[�F�.~�2���	?]%P���sz���H^�,���$�{��*4M���BM��5j��.��^��~+�������3�<��s�x�`�����l���)����P�e@P����dV�+g��v�}Z�x�� H���n���"�27���/����o��U/���3/'�a���X~].���~��>���o�"�b�nN�K������$q�������U��;�\�xyeO��K����L������ ���\�yL�l���0�e��?����Zhn�������h���_���$��o���/���^=��UIU��V������_�&��}��&~�q����7l�t�S��*]�A���(�(���cN���0�S����`Kddddd��D&4���
��w�<�d�d�d�d�d�T2l��X��8�(��b�(���D�m
��-v�
�#�A���a�u�?X��t@��oY@�d�� ���`���j�0(����6����w)F��d��VV9�s�B��:�C8Qgj��[3����*�U�C����8�B�b�Lu�>tG����`B�<[W���S�2N�b���������(���S��Ck���U�p�Z�|	2��<fy�Q�dn���fJ�a�w��ARK�bg��0apad�
"�U%��Fu2�!�ei��b?NS�����Jet�U��E������r����qh�Nq�;�dJM�9T�5�|�Y9N?A#�la�����H�g�c=�����p��}T��q/L[����
i�J=��6������zK4�F�"����
��Ah�KB���z=�P���I]��*���~��.�%����@�\-~�����yqz�(���������r����?7���Ff:�R���n2�H�*�Q��<�>�z<ddddC�����4sC��3EH��}������AiP�KJG��6n�s���I� .���xk�zZ�w�{�j��8�8��j
c*��5
�4S�jLK�Ru��G�����������e�u)SY�Nr�b1o-f�!��I�I+J@��
����;�r�{�UhC`!��Ax�K�S��L�Y2���0�p�����%-[�R��1�?��IU?��,��6��1`�!��*��>n�[��n;�mt3a�7�~�s�b=���m������d0L���N����	�K����` �;E0\��g�|��|�����40
L��t�����i�����( ���h �CDS �U���h��`/����`o��}S��������X����
�*��3S,b~�B%z�UZ��5V���T�R�r��e������u�\��U���$�564V�<�Sb�Z:N���GB|�����-�EJ=Y��T�VN��%wgJ�tG1�L�B�p\�5u�h���o���U�&M"�s��'a`�qf\�vz������a'&�Q�/��C/�L����&|.�p+��3����
g�S(1�~�L����
g�����qPy�i9Qa4�st�������(��,�V���!<�}.�������Hq��)'��1�^��>�G�������sE2Qa���w�M��{�����p���g���k��x9���D��a?,_�����[�gO�A�E4��,��h�9����^o;����^{Q�K����#��8��l;���]��o�O�6��` ���] �E`�`x'1�����L�V=d���^f`����'\��L��2O���	O�S���*��c��������*<��s�<���{=q �
�#%
�h���J�����R)�Y�<K1
��i	��Z	|��UZy$����~\������D�At�C�sJC�-�/�jN�l\c.�y�W����hH�J/���~����`~��JUl_��Nl��1:�&>��6�����fx�����������W��������K�o���q��=~"���m;{���G�EW�[[f�k�����_rN����9%7g>�b��+�C������.��~�|�>'L������������{�P���V��W/>?�:e)�>���f2O����>���a<z?���� �'�s����vWO��g�c�o9�]��^�������������^�}�{�>���G����4�[8HeN������T��m����h����U �����`.��i���N�����^�v���g�x��/�w���v�?�-�����8�]���S	��q`|���6�i$�Ab�x�$������@0��M�Z�#�B�qA\������.������.�����r�\�my�����~�N���)�oU����0�d@���pk��`0���Gy�[L%X��`1X��>�C�^��M+�<������3�0���}�4���a@��
����@[��mA�Ni��rk$2u�2Wd�x�J�
;�(��r�Pf3D�8�=�(`��1`����1�j��b�,����F
�}������J��/��v��A��W+�4�J��LuF���'~�o�foe�2I�H��Q��7�
z�����S���O///.��2���+��T���u1�?�������}�z(��.
%V?�.��\�TK�,����~���X+�ED~�|@�{�������C��~U�?�oU$D�N�t�A����^��m>�>����~���\E[l���5{S��x��j^�^�F�#��������������5sJ\�	�>W"��*��|"��O��T����Pn���Od.�0l)���e!�S%�uFx9��(����ldX�����P��e=���"����)=��mQ\=t�8��.�^;�aa�Sy�����U��4�K3?}]���e�������u��M_�b�����r�^�N��n��J�o����Hm5����L��6�$n��&�a����{��q@Ig�Z�ZO���[�.Y�	e^�����q��9�q�S�����R�0�U�IJw��������������]
����z{+a{T�����o9����*�-����w�^u�&O��
VO����+;�|^DS�pK����^y�opT��R���-7��R����~o+�/���~��/��'I�����������;�On9�Eae�u�����u�	���������..?�8�\���4�;C��\�^�3%�"���W����d�,g(-�!�Q�%)���i��������W�P�d=)�<*������g��nG�s�"���5�������E8"��k�m1�����k������,�V�L�@h6`O��QM�=�>�����RWm���D]������X_�������Y?+�L�1F����S�65��1��s*|�GP�z��UC�����9�e��S]���j|�
�����^/M����m��w3b�������z]�>
�~�<`c��,����O���4�?��-�,��f������J~�d��^������Ey����V�����N�Z������N��fh��Y=5��S�w��m��m�z�O���r�q�Tke��?�^mL�Z������o�D���n�5�}�'G�y���q"���iu�����L�w,g73����"����N�����N����Z�������?/U��z���d��JKt�3�`�1�C`��B������R�k��$���u���9�WSB��N�I�#�g�[��������=���e����%���!�j��R�!� � � � �:S-����E����!��8�L��T�~��z��O��m��F��@A2A2A2A2�D2=:�������q���S�Q���G3�W��M��IB|����>,v�%�����R�s����J
�c�8h����L�-�Jn�\�����3����F��kl���������q%���5#9�"W7��)����U�$A�V���O�������v��}��D}i�TYT$�����������]��O��k��4�_�����"s7�����+�"mMVz>5kj�rg�6����6Q��/\�?Q����_�$�In
�|uI&�;d�t%���8��
��J�-i�}�o1X�a4��/�q�D�q��Y�N��}�
zQx_^NM���'�7A
�S�H�y�BY����v!�O�]��
�|�>X*V/�h���I��>jC��j�
�U���&�W�00����1����1@���_��+��d2��a`�;������:���&Q��K����g�hVv�v�-��)�Tf~j���[q�^�l-��SJ�h�:U+�:����xnc$�`��bwnal�L�n ���1d�C��t �������~��`����-�-�-�-����a�"&K+����}��O�$�@2���F��s5�x�Le���d@����/����21��?LL����Q�t�fI��W��J���Wf��mp���mp�[n_���N��l����/6���Q{��qdw�����V�Q`���i`�#{G����~�S)�5���-@��i@�HG]Z�)��3����D��R0��J:�\\\����Nm,�j���_�V]cz50
L�����j&u;�LS�i�.]%o�%���}�����y���8 �k��j>�-��n;b%�UCz��+�U�;�~�p���A{�~'������.���W�q�>���z�����
��P>/��|���k���`=X�����>Sc���o
��� =H�������s����]A�>��Az�����o�A'�GP$�S^$2��T�����M������r����(�*t�b��%R72+����I�8M-���J�����n����Q��6%V�<�6��Q�**�D�0���ic�����N���Z4C���j����2�p�rAJ1S���T7�F��L��g��q����(���01�9/��2X��c��B�A�A�A�u&��K���Ne*z�^��zw�������`/������
2�=��9���� /��=y+�3���@/��;��D�NeC^�$����/�����9���Zy*��/������N-0���Q�g�����_��|��j��/������nRu[�K���rB�c��z�^��+��M�/�a�q~qu���+���e���:�J�<|q�(�����J������pSSf)7#3�d����pP��Q���BE&sv����k��2;��q�'�������~�@�,G�C������P�P�P�P���M��Tv���/��vl��!�ua��2�Co�j�KF$�]�����Aj���N�V����)Y���{o�����^8�}�[��(����*7\�E�B���X��WS%��r!������	W��*���@��!�
*q)FJ��f*=�#��bA'�q��-�j�%��:k��O���	����������K42222�f!��^�3�g����� ���#*�|_��c
?h���:�����~�������.Ud���G���d H��BrP�{k����T!�������,��b��#s
q�f�F5eNd�Vvn.b���?���v�X��_1��i<#����y
4�
h��vW
4k�����R$�XP��9�5���'u�,^�d�3�?��g0���h0�3FSLd��G�31�_i�0k���/��vg�X�o���x��V#�4�@3���b�Bf�������0�3�0w�7u6q����L���"���>�k$��������9Z����2`
X��u���Py��o�[��3�r��/��k���E�X=�����B�����W�q")�;*�
�i���k�����M��KUd2�ff���L��C|�b�^�s�o�����uNm��l���X������)k�,���������(Kjf���T%�Qr�o^��S����tS}���c��s����sec�5��Z�[�L�-��J[�l$������f��-b��W�����"W�,���_+h~�+|o�������jn��c�������<�cF����R�����5��7s�������
~V�5��o�n��W��������Uky��F�����B����
��
��j����I����mF'�y4��uvev���3s�l&�~��o��w�a3cM�O���t,�:O����hQ�J��\����������}��3�;o����X
���S�f*����A�k_q�wqj�3��+w�~�Z+wcoSX�-�����c�V�T��vY��o�������R�hU��J������L�_��f�U:����f�R�eV�l����
_��y�����~�!n���������_P������$�V�9��� �����$FV����l���wr�Z>����G��{���wy	���^���=+��_��w]e��\qc7:a�F�+�D�L�������5h�2<�*��,����\�
S(�&��5B�_�9�]��S����:c�Z�1*=9��~o���:]i������r�,�0��I�Ly{4Uy9�T�����������VT�g�r�s�b.E��+�����p��X��@ �����Kqu�����#!�r��,����t4z�?Q�t��������x��V6���.��O��%��p�"&�����k7���Y�/tZ*����t��t<_in���Iu��~�)�9K��.�����r2�-]���u�Q?W�(N��@7:
����7%���7W���e���j�%V��9�������1���=*��
����_�^^5������
�H�~%I�s��������C����?o�����ldL��U���&�c,3��CM<S��	j��@v��\�c7O��,V�Q�r��&�M��	�+O�;�Q�)
_���%�$�v�5���.�����]j�B�g�kdQ����<����~��Kze�� �T�EZi�������0�r�4��nX�u�L�}�
b+�Q�l����p��+��E6�6�}�����l�<��o���EM$S�O���-��}�'��t^�'1�[Orziwx�TeXL�����pI��`e�2�Q���������iFxQ�x�2�G�y��~����i��O]�0S��O�7��-�(�Z�,�Nw~�?���������rh�!��;��p�~��2~��u��A������7���0�3u�c�W6�Rz|u�q��#����cn�����h��Dpa�z��������G����Gw�4��^g�_'h���'�������^:���y�l�H��X>�i��3��D�h!�+��\#����EoD+����(G����������]Ubu��*Y71���S�Q��e�+�J<q%���SYjf�>�!�h?7#���TX3
8%��=�1�p�������/
���z
�$~�<�9B��>��,�0l���`B�����*�T����J��<"{�T�i�����l�w�i_}����m�+��V��D[G.��V�Z������o�-��A}/r�n����S-���>�O�I=��w�%E���z1�#$J���%��$@P���������M3��)5sZ%
�M��~������*��sE�
JR�ayB�Gdku�V�����U��8*=6�BAiZ_F���*���}�vb������mK���X��7?�Y����N���'�����^�H7W�W�	+M�(A������~�����Xpw�x��a+xX��
�/���H3[�s���E!��[�������dtN|����]+E��c]��vi�w�xBO�wp��<u���^F;�?��Y��m7�v�
�8�LT�	���r
5W~��vC�Tw&��FB�y���������f_�:cy�p:U�ow&�a>(IY���>���QIJgT=�������m3���+��3vZ����@����}�}�������UF��`XFF����6���9Z� #a���f"�Al���{}Q���aC��m���i������z35K�E�����q��Wf��|@��}@�^����.��(��00���=YK���U�<�V5��B�mY�AiP�������5{{(D���������)-mn��v(`
X��5`}��bU��m��cF�;��9��o���_x�/���r�xg�}(0���80��tp�����+���7�
x��Uo���)�)���`�]f\Gbr�?5�-F"���s�������U�2O8o�UQ&�,S)7C���W��XNR���t&t�m��cp^�����6���s�Q���A2��/97�U���sn�>"W���;���4�I���N����e�*������aaaaa�*�h������vD��`/�������)����iDLU�f'�i@��������b�X �b>b��c�����`+�
��[�D��D��^���s��N��XZ�bD��[���zGc��U��[�VZ���"�y���[+1��dZ#gE�!7I��T��/��8hl"���FWK��������:ZW��
Iq1%�z����C�^�\��5�;��BVAVAVAVAV����a�T[�lI���`E,�1@��A��qo�y{e �r�\ ���[���+��� ��_%|]��/�&	p\��ep�^T���o�Y����.v�%}�[��o��������V5��J.������(�\������������"�����/����������X�����?��)~����@-P{/�%���>TU@PT��|�{t ^zS��T�b��T�(1�~��J9z�o�q.�h{`7�cDV�� 4
B���*��	�_�W�|_���o����X
��2K�<6e���[�&E�Wef�?������NEn<�hW�������9����qX����.}�~����'���S[������_����Y����G�YR����HN&V�)�J���QVN�������	/?����#����\����������<��_�����|~�������
��W:/m�������p�x|����'�%��������I�KOP���pHEf�uY<JL;P�xkJ�s���_�;<3�����i����/�A4�u�p>���5�=3#����M Y1�T��w7�gJ�0	s�{���\�W�i��-�d����Z�1Qkg���_�}q������<^�����L������^�G�j�G���w�7�����y���v>(��pG�{���fa(>l|���`�]"����������L�.��uw���}��S��;���0���,U��C�D���U�m���	2/�ja�q�1����Nz����)i�)��=�(T��UN���/��������������P�I�����4$��B��'�z��J|v*�8�H��.
}�S���Ym���^ ����1AL�Jr���I��oI��t8�J�������2�A�W������7������A����7�����_S�#�r��/���������7���;���z�-�vzYq�����J�o��.����=���l������_�aaM�Z����[��;=`��k�]��b���2����.��'�S��<+�����/�*@CPD��\%B
w�,�N��L�m�<A.
��g���������I��+ur�-�L*��O�H:����
�]�����[�S;���-�$��'anu�V'afuB+�g`��}��_�}&��gy�^�zA=�L3�����{��J����WB������GG���>�`�u�)F�BUB�L�?��#�`q����r��}���x�6x������������<�a�����S�'Jvw2��;�N�9���h� 8����������R�	���
������ 4
B����c{����T�B}W���:q��� l'��q7���-���e��H��t ���t�2z7 
H��t���^
��-`����v3��K ��0 �m~�5c��j��x�}e���f�h���9�y�����}D)M�&�(��b��C����g�Yp�g;TycHyy_zH�X�gu�!E���8v
&����"lzh(_��f[�'o�3��w�-F_�^T�Z��R`{�xs�����.�����8���ux.f
��}�5�D��*�.�5��9e}��.�����#�����<���z���;��P����C�?��c�,��
��1p�%�WSk�?��6���{P�����/�O�����
3�E������uM�;�l`X���.�^�@�ku�x^����T`&*�L^�s��Vz%�`�S�7�����2;��
����!���<}~��jxq��������>��m��z�g'/O�G���j�z���������!���7y�2:��1`�����	���\ �r��.���@$S%A��}Y*\+�xE����������c2�����a�B�M,Y�M��u���n�����3�c+c%���1�j0�����������r���XfNq���}�(D���"�A���40
L�;�t��G$��Ah����ba�����"�*�w�f�_b�i���a<��c��{v� ��8?�������[hp�w�]pw����k�
���0�z�������X��00����0����a@�a@x�.s�]�@aP�A��Ca�P^Y��J��a�0���8�V%~��26�����00���/��������o,vz�6���.�����s��*�|��|�`��0�����{;3���D��J�*���uJKs�*��#���2�fm����w�7�
|�����Uy�,���0 �;���S��� /����;$����/���;o!'
���\0�sw��������.����,������/����1��#��o��x����d�����~�_�w���M9�������{�1�T����:����-Bzm����u�AiP��w�$+{�,��8�od�����_���KW���F$S�\��Wa#��`0�#k�����V��2s
�A`����tnM�u��&���`/��C��o�|_��|wgz��;������_��w����C0��`0��0x$���y����0����������E��00��9W@�1@�s���SIi�_�:;�~u���1p��q�8���va��x��w*�P%�tnHv���` �ay"��.���3_��|w�L�E�"@�t��@wd�<����w�^��zwelP�N���@/�����Pf�ej��`0������@/��v��������8/�s��B��?t"�X�N�f�?UB�Se��y��T++m2]��1���qSS�-
��a��S~ 45,E��J�0V�<�[}M�:K������p��Dz���b��#��B��^�N�vUk�qK2s�����:9�C���X���}� � � � �����z-�>��;0��00�%�{�����[��i�����"�.���F.���;�8��ab�E��Vj+,��3�>��]���^M�[��D�0��,'�gS�\�(-�YO�u"��.xy���O��o�
��W�:Oa��g�p�����H��������������'d���bL
����F	��������53��p��L$���L���(����&R�`��"2��������,�m��A|�[�uod��l[��l;�m�[�|��?_�^��O��xu��V���W'�=?�=+[�>y������gW�+��O�:�������/�..�������xY��+�]�������0�u ndV*'���Dhy@�������;��V�BH���2�(!�THk�����A���O�c���y���J���o�+��X/�S��Ck��ki�&5������B�'H+H+H+H��������$���'~��������d��)�����2�.�����a�����]Y�P
��`0�-�{��a���Z2���������p�{e����)���4(
J�LS���O����s�
E��` ��(3��'��4 
H�	�l[����T�e�y�Q���O///.�A�[��I4��\�>~"��>�"���}��dIo
t�����Fq(�X��MCc��2�������j��LQX��������r�������G��99999���h�n_Q�jMI����\S�kW/$�L�dJN���*|S��Vp����]H��	���������B8��V��?K����$��#Ay����v/.&�����p��>o����T�J�b
�L�I�����L���r���g�xf��26vu,�l�,�V��|!!!!����n���J����?NY�����*��$��v\,Bsm	z��E������]� ��8vhTR���{��K�]�,�bE���C8?���Z��V�<�@�T"K�'��-�Z����5��*5]�d���[�5��h�I���jUP3�
���)dddddY�����<���
�� #���}P9M�\�n>��`��e@P�����#u��Z?������b� �;�f�����!�E����ej���f$6\6�k������^KeFF,@�t��u���J&�~6�<a��$�I�"'L�00��b�V�z'��K�ED 1�� ����}���u����3�>��]�������.{R% k:3��{�s'd�U;�%����<(�����BB�6X����MO��oh���y@���=��K��p\�C�qkUZ����2L�*��h�|��*�	<��3�<w��?�]\�:����|"fd��F��e.Jv��TyM����Q������%S5S�j���8(�wL����Z0�j�������E�nqq�9��l��X�\a��UA�U��F������8��f;~����"��V���#���kG��c�x}�����'$y(��a���f��:����b����Te�*E^�F��M��L/yHU#��V����S"?h]ka���|r$�7J��U
]^Q�O�H�!��o�}�L��\	75e�
�b������6n�@�}G���'�)��K�O��))))�>b�����2P6�`1X��`q�,n���G@.���@n���<��z�Mc��l�N���$�f�Zp�HT-l�.��a��o������V���(����`.��1sr���w�G��Gd�����IHA���I����B��.U�Y�|P����B�����=R���L{+]�_d2�������=���~���;G�{+��U�ie�a��Aj8X��`q�,�j�0t?���)w,4c�4�A�i�I�b���%����1`w	�Z5������-h�)m����� .��#�������zyyq9r��e�yO�q���A�#�d�-w����'��6�Z�B$fV�LY�]��'O���#f��Hs�gL��Fw�g&w���giY�����s:��&�qp�z|.�y8�%���toK����|�k�_��i^��<��lh!�����zT���c}6�������^�6
O%��J���zf��1��u������T��a��@<���[�ms}��\J3��k�����X����O����������a�)���_���b��L��t��L��������>�����a9�E�.�������{�k�+��or�=����������F���+�	��|�|[�H�M��>?�J�?�������y����W/N.O�_��N��j��]������lT�e��\F91�0\l�~��e�l--������k���*�������2���T�}dLFEv�<��+q
l�����8%m�a�+��R���>����;��[x�.�C����d��A��~G�����[gQ?���_}�+����>�{s���=M��G�������,�q6#�����K��^c���J���k���X�QHz,r������D���w����Z�r��j���z�1S�L3e��2f�]:�RJ� B^�5i�A"��DK������ �G�X3w��7&��'X�L/���Rn*�i����T�r��
������Y��*���br��[t����C�p��|�����S��?BJ@J@J@J@Jt(%�Tx�����:5j��+�����)X�jaf���T[�����f�����Z�(Z��s��'1��R� ����*
�T�{n�Z-���q^ODe�4B��UxyN���\��A�q2�8Cy"�<�-��6�f��D�4v�f���Mj���sN��L�����1��U� �(('%-�W6
Mef�W�W9�����6�D���E�BXBXBXBXBXv),+cv�z\(#!s���]����V�EHti��n��19.e�������lc&�/�~��u;�}[�{*s*�+�?u�RRRRR�>��?�W�����]R���� 0��$p]jn4sC�])�����v
��2���.��[�n�-�I+�\�%����oV�ds�L�/������	�t�i	LO����(��r}�{��(Qq�ac���Q<�)kgM��xiU}7�����~����H�k���u��T-J����%���m�� ��LUr-d��P�L�3�D��������b�`�,��z-/b�����������<|������SVP���)uLK��{eMu��S�]A#�x�~�;����t6��^�N�����T�+U&���ly�Ch��D�J��ipQ�n��Hf�WyH�������{#)��C�[������w�3���J�
O7\?�N���0���f������N9+���^B�<COo�oV9���~�������^�L��
y�T�Y�]�/��C+�/����,�����������W`�.Wyo�W��8�A�����C���	�('PN:5��R��a��)p�+.tz��&��o�|�4�Ts��k����.a��y	_*5����	���:��
��7���A�V��a[OV���lw�q�����;�Ql������8������j���O��T�c���F7$�X8n���^�4�
_������o���^�������o�����n:�o6T@e���*T�{�v��o������������Zk4'a0��\;�2����L+���8������u;��}��.��Wm�!l���=�&L��@�@�@�@�t(K8�����?/�H���VQ&^m�u����y�����������fYli[�A�,�Z��=`�w:q �;oHO'�����#5�
v��`��L
s���w����,)�N��W��	b�����Z����K��V��d:��j����T��f-����`]B�*�R�R��V��+2��Y���@���>��]���T��c�\�~�m4��9�a�>��=���&U��^_�&���8jR�O�C������'q�)��wm���&y��I�o��aS��&���gcv�����Y=�{�*Q��������H^Z���;!�"�~(:���z��������Ke<^)M�V�����'��'/���������5�/1�'o8FE�%VH�R�	��}����]�st��0����mi
�t��
�t����I�.��`�cK(������w�,T�eF��xD��nJ*6�!�l���J.re��9U�7.�4.����r������9.�)�4R~���df�#\4��������/O/���������o��?�)�)�)�����b�&�+|��Tr�#a�R{y4A�UY����<��Qx�Z?��K��;��wvr���4���� ���1H�&u��)�i�*�Ir�>����j������K��z��
����1[��yG/�u����W�o�uA�_?u{�
c�8n���[����%P�p��W]�T��{V!W�-��C�Jl}��*H
��� �o��o�Ke��P<������b}����T���b����ip�SN���L��uN)�.O�_������<|�Ab�$�wEbr�>T�51��g0�,�����������]`�v��.����j��B8/�/��)_�^�S���q�c�Y&�1TU�C��U�ju���q��ibr*�g�u��u
6>{8�2��=��^���<8��F�n�\H+g-B�e�`��1`�{gYM��ap�w���������P
�����e?�{�0T��`2�&���������_��|w�B��`/�����f/����
0 ����X�#�8�|� �4&�z�@1P����J-�b� ���cAF�$�Ab������8�WV�R���@�d�{ ?::�^����3H,)�;�[3�Tg�U�1W��*�i7�b�e!F*3�����M��9p����G���\�|Y�"o���d@���8H���-(F`X��`�n��������]���?�Bvki��:��5p
\w����e���Jy�d����^�j�:	�w1��I��n��#���o�{�����uu���`��`�	���!���H��@/��;���@/���@�n�K��J�*E����<�i��0����FZDT�0�/���*O��&�1@�S�<U����}A_�w��u^z��|_�w��-�D9�����.������,2��x^�w���Yv�����|_������Q�4x9��/����E&������` �i�3�1~�,|�{]~?��ap�w�a������,���N/t
��`0�����A`��'��KJkUN�rc�9
���0(
����tnM�u������/��c��o��`�wk���w��
���c��`�.��|���08�?�"��[�x��
 ��0 �� �������F�@1P�?�9w`��1`�^`<��|�TRZ��������_��7�$�@2��$���_��|w��tC�d��!ms���a`�m)
SPR5�{�[�)�(��00�����C�L�qt0���� ��������;2e���p�{�_��~wixP��/�����n�e��Q���08����{S���=�/����o����=:���(�d�3���E�����?U"�Qz���w$�����j����ZS(��r�
�aB���mz���}�t�Aw��c���D �U�^�\�����"1��|7���;A�FZ>H�S)���A:��53n�O�w��j �U|t8>1�����?�d��a�X�2�Z�?�f������O��8&c�p.5#cAR���2��B���i>��TR[a�Q�p[2�f�"x1"K[�.��)u��q$�ge�d�
*�N��)���
M������D�4S���������t����?�bbbbb�S�; 3 H���� Y����d��&J�zS�Vm��AiP���}�Z�9#�1�"![���d6"I1f2�[f��S���B�������&�RWR1U�oD��J|}~�>�pS���'�9Y���V
���?��,Q��2��L_���C��U3�LVa����^��lq�d����v�����t��$g*���sSnH�NT����/#�� 5b*o��:!#u�S����_7��f�{���c====��&���
S�
gj6
��7�������s`����N1���O��n�f�_�����2�*���r���[M����b�,��w����e�E]���!��2:a
r�q�YI�i��8~$�WS��$2��3=��+f�!7�7�T5�Ss1*�8�;5u����D�Pd@�����jN�� ,\�h������c����s}�������CU�����D#M����[b@g�t�;���:w���0��p�^5Y��l��s;�4�@�{Csc����#;Y��������=U���O��V�����T:2ks+��|�O�+���T��gT:��t��R[�6�iu����~��
���/��1r�2Pj�jeB;����9pH����M�u`X����W.����|��������T^4.)��R�:A.+�R�+�;�2��| ���G~ md������*�#G�s���L�39z�LpI&���B��P�H�[�Q��t��O�\��s!E����ke��8���
�e�8��P�lM�N#'���%?�9�U�(9�H�Nn[����bJ����c12~�I9�-7>GW��jyX��pT�D9GI����	�NT��S|W��Z�;�
�i��Q��,N�dhanb"Nn-\|�8����������M�F��]FsO����)�e��v���v!�r��7��H������F��a�{�������`��J�)����e�L�#����&��a:��LE�T2�E���zQrSuVe(��Ayy���;Dy����u+�h�M� ����������[���FBnW�M:|��.������Z	�h%�J��t��h�0w���^����T���U-T�AePT~�Tn'������\I&h���t�M��1�����08�!��=�$MWV�z�/2Z��g��D)J!���72�����k��kX�:�I����e�E.g:9,��X9�q�G�	�����C"J�?�L������e��,����TC��#2V	^�D,LY-/�:�5�����5�%�X���XT	���Q�Nk��/bB!gf�v���M�8��d�$��R��-Wc���d���u��gZ��|z�^9%����f1��ddddT�2���^��C�Ii#��S���$�A�nM�wa��cA�����Y#���L��40
Lw��m��m��P=D�Ad�["�2YP2��f��h�R(
F��`�n�f�J���6+��.�����n��
�}t�G!��--��9�H?�{B|F{��T�E��yPXE�(�B��Rg�hJi@���&��;
mQ7������	�3 LU�'N���R�+tKCc�s�r�E89oR�<$G���V]{G6��s��1��-���&|%��V�4����L(���V���	�%��������d���Bb.�����>��L��k�)���vj84&|���fUnR�b��0��M�������<�BjK'SOr�������]
P����	O���[��g�dXd���oLvCu��e�=r�i����m�a���*��j��V
����.@5�j����O5�����m��ko^������$(V��,T�e��pd<���T�B��E���o���^��:���3�P�o�{��:������,�W5���z@%$sx�/�p��y���	�����4jZ���f}��}�+�������/O/������:��uVv-����=`�����v5�����"0���U�\���
/����}h�kN�"���yP���t�qo�DA����1p�/���`�x^�W�x��>:�����K��B$�yr��yPu�FWN�\�"7U�������P�����|HV��6	'�F��|1|�pl)g��w��<��s��7���Ig-s
,@�t]@�;�V���[��o��l�ui����*��P�
����������`����������>h�, ����B��`,����)��k�
����-`�v[v6S��	L]"���@.���@nG���Wj.���.��)p�\W*�qA\�{�r������U��u0��/��v�T[��a&re6lR�@�r�\ ��w���`-X���;J�I�z=J���T'��.����JT��������/�{�����{~u������-@��+hgTS�{��@,����X��b�X �����a�T�R���-@�v��N+�Z��hZ��+g�i���-@��W�XY�'�B�p\���:O�k�[eAYP������(TM_�W�|��V��X�`�{�4����V&�, �����C��#k��� ���lG�-2���p�j�Z������(��������B�`-X���`����t -H��� �.H�]RZ�r���e�h����-h{���skJ�s�e1p�g���8;,����hZ�����V��V'A�x ��-`�v��8����,���@.��=re�����.���;t��	�, ��Av*�P%�tnH�<��[��n;��7����w�"8p���m'�
��C�L�`-@����6|�>���,0�f�����Y`�f��l^f�ej����-x�v���,��fnX���}Sze���� 2�"��?����/@U������"���.�{/���������
�x�3���+l�����F��JOx�tG���^�)?�v���5��^+�����h���� 8��� ����Rg���N������mV(��>�C�M<��`2�&���b�x�Bz�^���_�m�$ME��DMM�*+n���SQ�-d��?{�����$|?�J���SJ�fr���Qe�y�#�������E��F�������U�/d��Q��t���~U(�Z����Q2m{���~g^3����k��
�s��*q���*s�uQ�
��t�aI����o�{���x-3����ff3��'m����{B��V�!�U���C:WFF3���La�0S�=��gpjR���+D3{���^f/����[��K'���N1)��A�q�8f3�o�hD�����t���;�J5Qe�R�sA+|d#s�eb���3%�n*]�Z*�MU��k��Cq>-�\�|����'�*s"�.����c� ��_�%5�y�x��fvD"���S]���2����
,(n�1o���[�-��q���n7���jx��K!7S,��k����i�4c�0�|f03��f���'��=:�WL��d���~����T�(A9��
kDj�C6�h����o���0���Ly�<S��(/N��~}r��\���1R�%��U��9������Q�����(LC\�t�NhU���t:�&�)1�)�-
�� �s�}PP}�y.��m�D	=���������H�)p��hUe_��uvF<#����	��x�L9J��I��0�HCF����{VX��j�FU���{����(��o�7������)�j��#������<��������Y�J7�������Q1�/�2��p���1��9n��d���L��|`oB������Tf*3���7C�R���#]�W�+���zp}��{W��(������L9U��I���x�{<)R��zV�Da�����;w��Z6�g�
�����4�^a��5mtF���D_��G��gG���y�>���
4�Xy�5�V\����b�a7<>��g*{R8���s�pR�	U	�m�d*zz��B*��4V���/m��(SN�\��{����� ����dQ[(x����p���}wm�Z�	���?��l;��z}�����#��,�X���cy����h��x�
#{�O	"��5���M������2wv|�����w��Df�B�ADT��8!iB��*�����{���i(2e-41�.Ad�(�Y�d��04Q�����w 
s2w6S���p�a|�MB���bP��T����ZH�����k��s�/�$������^	'.rS���Z%���9_{x���"�GJ��Y��H�=�2u���7T���
�^���Q�j/�?�9�JS����z������-l]�:��,�J�������bU��35����*��
�	�#Ufq/��$��LS�X��)��N����?�/�\[5�f��RC/�;�sS�!�������2����$3�����P��c���"�^��]�|���'���T���#�#zH�����q��Wo����6�����
+8���9�C�Te#/�� ��F�a"3���Ld&�v�������&�&���90w1B(�&b��L��N�?~��6��p�����\�$��cz3���Lo��6
��B�df�����e�2s�����+��0s���\f.3����h���;<a�WW�=��
yH(����f&d�<�$'���yXt���y��s����T�i�]���,�m���y���$?����J��u\ZV6nWo�r��(��b��B�\k�U���T�Zz����4�C�7���^+�b�Y��x:����l/��XO�b�C[.?��3�����P/������M�K�p7�.��|�k��%�:�&�N�eyiE"�i��^���K�s�@/<\��{�q�:h�D\�����>���KD���2�e6�l��,������G�����?���l��0!�/Av���K}� �\�T��=�@@�c����w���Z/Q�{��)F!rTXgfa�����]�	����N2q��L\&�MWeV�r�-���^�0C�!|� ��31%���Ec(jQ��|�^%�!���c�1�x�J��@#��,)*Z�1�#�.��D6��uj�z�0f3����	�W���8�s�����/q	��TP����V�4�cx���>d��(a�R�y��HU�-9������X
[�X`I�C��>�c"'�E�p�����%K�,!�+!v0���+���{�S�$�E�a�DEN`w���0��K.
>j�/>m��'(,|@�/��A��&OY�:k���
cQ!_�D�L����/U������,�C�!������0�B'*�/(>w�Av(�O���5r��/^,�Xxm]x��i'"�!1|���r2���?r�6'�2C���P���"O_�#x�s�Q�>�cb��9��q�8f3����W�?z*���b�.�c��G���[�>��{
���]���`�u��b1��]1n�?�V�k��r���&�Gx�=+�S�������$0�t���#,g���~��!(�a&q�Se��gF:'�P"s
�l�L��X���HUb��Y�T�*��^x��>�\����v;�KW3�F�A�*U(��@1���agBX����$�DC��(�O��x���{��u�F|���������v���7��?�������'=�V��<���;�l���?�\O��o��KP����=9�������I��B
���e�F�{��S/:w{t��������g��*N|����e����~i3��c��gc��`���)�(���������
��
��?C�����*�|�����u�(���U1��%��#(>rH��
+Wi_�,
������i�m�Lt&:���M�w����7�<�������4Ca1mP�5H��e�El��P_�<��'C�
�0����O� Y��L&��>��C��u|�!Ug�����	)�����}����C��C�aKs4,hX���aA�EACV�W�����Zf-��Y�E��/�n��+u��.�������S����Nk������������(�l�a�P�s_J'�o�c���@<����	^bC����B�U����VnR��9o�p�2V�|'���R5)k!5�j�=H�45�����)f�%7mi}Yu�]zM�h���$�7G���2��/�*�2�c.�����]S��K�}V{�g#�!�}hi
��5����
>����r�B�{+x�����b�b\��/)��Sl��0�&9�`����������O�#0�q�W�<�1|�T�E:Ceh��t�FX_�����w�Lb�h��D�n���sKW�L���b�K�������K��������k���/>>o�[Q����|�m�G��f���sxpl���S��8e��9$d=h���4�oRXO��^H�l�)���u!a�}1UqdC
����I����A�����K�;����[�5_;��pxxwhb*��aA]�,�4�c86n�{Cl�P�{"r��D�
t3���9�3��!���t��a�>��`�
�[%3�q�����&�hZ[W�;C	��L��o>/�l����"n+5PF������s�F�7]��S�6��������@��{b�9������Fp/|�{>_w���K�)��Q��W�a�GcR���=�/������:Bc�����������q�1�RR�������{H�nPueK1�#����kw����q��_��J�E&�*��#�RE |����opG�L���p�������p�*~gx!*�����}�\K��nGz���a@@gZ�5u���L~��U��������eu��_����
�m����S:$>&�N�t������XZ'��/-����-Hub��Q�1�0��������<���$�%y.��H��D�����l�/�9_�K�ON�!%E�����������QM8(v�;	�Y�w����H@R�!�8�2����rb�g�)j^1���U�+�~nJ��fk �R_�2���z�h@�b�E�:�#���[=����������cv�`i����K��J���u-���Y���Ll&6���Eb�����\H��j:.q\�Y����*})3�N�(�����D�h�{M�Ti��8+�kP��3����Y�"�qj�{�f��R��Jp�fe�����d�����	�,'�('�%�B��.�"�~�gE�U:
�xZY�'3\�r���Zf�u�4f/�2���Y�`�"��D���
)�kN9�tf:3���oAA�b%���� %��Zr-�x��3f,���u�UT�2��i��fZ��5���_/�>d�<��f��e�!��e�2d�[�r�;m7;��*(C�!��fH�
7�(�#��p�l��j����T��Jh���Sb���SgL`�3�����m�&t�����<���D��zg��:X�L���|����/��]"m�����6���N�P���u�\F�&l~���*�Bd��&4��������p�����i
�.�<���9� ����O�"��_���_tG����22�����[Y�����*7���� ~�t��Q�����7��r���\/����������*13����������4	��27���a).�����M�FR��a:��_�P��bO�735�k�	���)�X�\�0a�-2������������P�t(���x��J��<��{d}��Q���*5*\9X�r(�5,/�$�IZ��CeM��nt��W_�0�o�B[���-����������9�5��l
`k������x"#���fD��x�����o��������8&Xw!�@���~?Vn���B�d��9����m>3�{�)>g���<=����)������`�2�e�6e�T0~��_���Q������>�5"���^������S��VL1-�_5m���zWMop�VF��P�I���������T���M��p��lp����G(�jG����
N�0(�<����y������0�������k���a���W�f��������
�<3��)�Lyf*����{���X�
����A����	�i5�-�2.��NJ=��%�3��)�c���������"���7����{8{�S�3	�G�L����g�����O�����`�v�������sqrz�
3�G��?�}���*$8q�B\*�c*KZ�����zbL��aVd��W_���������Jf��t�������	��4������|�a���aA+�������I��@k���=l+l������}����m������_xe2���&L��`��k�CYpQ8��^q�8�S�wtY@D�Ph�% )}�4���{�?�5�e�2@��@���!���&U-]F��Q���P���i��h��c�1��w�tj�u��?~�k�-#��}F#�������g���x@�-��R�&�E�>8KkK�t�HLW�d2������c����1��c�}X�S�����kI��S�	�g��/e0��|L��|�;��)"���|W�=U�A� s�9��p�����6F��������r3Q.��*�����6-���&kfJ�����>y?��\I��p���1��pN���*b�J�'T���!3�V�����	��v��\|LU�����
<����h���7�<�k���M�2/�K�VQ���=S����	�g7W`��q�*�eOU��Tm������o��KY�`t�f���h���M�f)m1�Z���6�vS�RQEh���?�ea=�Zb������^���\��*��|q��]�F�=�i,��Hz�J�
�����P37���m����!��d��Q���d�d�{�
-u��mB�����6��$�f�
��
�6��k�����M���Y��icV���@��sS� /R�� [S�������+mG�>,�LH�tD&���	zS�H,�-���(�Pg��FQ!�:SX*fn��+�:����@!��x%P��	����
�Ty���EEIk+Pp��p	�)&�85s��-0���I��R��:����5��DTjf2>.^[�c!�i�$�h�X%1�3dj����ol��a��B��"E�[���&6IlDd"1���Lc�1��i����:�'��D2��'�h0b��q�8f3�������6^���]�.c���E�b���M�Z1u�b1��A|� ��V��<���l���a�0f3������E�����A�
3��b1�x� &�7r���U�H��]��/9z�����e�	�&S6�y/�+��T)��5TOL�>{�(��P�M�t�hRJ������I��g�3�����r�p.d)g-DOdf��i�4f3�����V	=�Q7������5�T��hgS���'eL3���i�����oR���Pf(3��[�2�7�:<=a�
�/������-�E'-��
��R�bf1��Y�,�];�d�$f3���L��H�*��U�Q�(f3��o�����*���G��l2f"3���L��9f'[����l��a��=+�j5�z���qL]�.S��{k�mev�w�Lu�b�^�c4�|b(;�_��$$Gt�Le>���&���T���b���VA�	�v�s��S%�d*J@6V�1�91^�LIyT<��>��4!�,f�������3��F������L���S3�'� )��4f�B�2k�x���2VX�c��,��l&u��a+�b�CZQ�f���gBr6��uN��=Jk�m���
������5�}2�/��E][�}N����%K,��?�����L�q�3����xM��f"3���L�����]Q<��|1E/*���W����m)<����t���u�'�?��~:��;������e�2g��0g�R�.��yZ�hd42�)�������L��t����LL&&���3$"
�D�`����Q��OB��$����l�kll��<��� d2�_�R��K9Sy���TyJnF*w�B�2����������P|\/��O)E��K���I�t�{j/[����d���4������X�{dfE���*�>Q��9���f_�g:OTs�	\?V"W��t(d�%ieVeN����;�j��_�?f�w5�T������Z(���/T���L����w�:���K��c��1����K5�j,�V���vw<9Q���`�!��y�<d~�<d,f%��Y���+Ss�����U~�6t��e�~���������vA�����b<2�w��-��xfG2MG35�r��EV���LH&$�����p��T�U�*0<
��7��C�S=���Bf�������
�?N���*T;�����e�2`�(`w^�9�R	^bX2,��������wG����HF$#�n"2x�{&c�1�d�56��.�#�c��#g�zk%��S<���F�����KPE��7
������U�0D���C�J��Q��%<(�R�t������.��3�P����2�R��'������V�6&��q�qF$�yn5P?�u���J��l��E�����2_�bZ�U����P��/���iW�]���������T�L�)2�����N�bf.�8S�W�a�3	u������s�e����o���K�U�X~V������Wm�
7
9��i��,�YNo�A
D��[��*]�td*{�yl	�E�4;�P����|`���[����+�;; ��%+�s��=����,i����|�[D���n1X�u�N_,8Xp��`�qS�������e�2q������*��R��,�X�����
�)k�G�U<�a11���y�<e�2O�G���'�L���A��OO������l��_+�����jW�$E���5�4BC��,��I{��lLg�3���|7�\����
H2 �H��
HT_ww������,������,U��A[A���3���QI���!��Y�����^iz`������ft/���AC��lZ=cN2'����;��v�Z�Dm��l:el26������g��R'1�M�
�=�95�g��K
���������m-��{��S��?#Z���������zP5����(��B}���c��eEa2��8�K�H����Vw�|���S�,[wZ�{��nMg2z
���z�������QJy��V���R:���B�V�K��<Q�2i���1�B���**$W��V�:[+������o�����Y�u����t�O���O�?����S����#����V�H�V�L;������K�q�Tw�d�t�{{ac�����w�������*�y)��w�bG�t�C�������k��	�vR��WpXD��`�"bYD��rL�;GRJ�[a���`���k�L�P�&U��LY�,S�)��]�,b��[�c������]%����dx2<����m��i��dZ2-���h�,��a��M�&C�����G��.-��SC����NF)��Q�(]F�����3$�I����W������rJP2�)�N�����"E�2Gg����T�A59����\N��%�	����'Uk:�2��'�6�h_u�������2Y�TN�l�N�U�_6J�=/�����z���|�f@l��,�-�S��M�DnRlV�"�
��4�����<���������c;�#9�T�
����J(e�kF��,5\2�����L%���J{��9�Do��,�#�E,1����Xic���W�w�}��`�$0�/������	cb
ea
b�2�����~'xQ�dj�y�v�E�+9�������b�*��
�T���
���
�E��)� �qwJ1�Pj����5)���]�yzWb�{�������O�S��/�}Sc�����o%����F�T���T�]��aa+��%~��� P���%�����<m|�(*��&���Sw0�_�M��H����:��>�E��������uaA}�@Ye��B��ycz�~U��p����� ����
�*�]Q������o�����FOk�<+T�e�7�����vXnNB��&)��"����f���Z�}�W���7������T���4E�;E�m��A5�%L/���;��%#KF��,��d|[	q��LZ&-�����?N���N""n��)�����xY��:��bB;���q�Q�_��D��+WF�E?Z��p�8K��P��E�+���k�&�^c�g�������m���J4�by2ITA�$��Pml���V�9����������t	��^�r�b/���T����$C��X�U�������m���I�X��������(jJ\OiEP�3�N��z�{�ii������O��������:h����4���x��.st�������'����Z����qGei�S���6�?��ajT�*�HE;���~E/�_��P�/�V�w����v�������^�����{�g�}�S���N�Qw_}��/������!���&n���������+�T���j��_�������M�k��C/�f5W":��G������z�?z�U��n�v�x�n�hm
��o���_W�^�x����)�����Q��y���4�SM�'���'Kq�b�wC�m
cX�����g���b.a�m���}([�s���`���������R�r�4����'T�X���Ev�\��D�n
k;�C#O��Q������ ����x�x�z��1)Y_��e������-��4/)#z9�R�g�<Q��>|�O��tR��H�1�9*��EJ��3��L}Y J5��r����S��(�l�l�"Q(��!]x���$Uq��|�����p4�F�"m�
�A�	g?��7@���E�:����k)
oRtG�$&w:�����I��������3%\������Z�%�_��vk!6���������T3E�K��'(���BC+�������fj����C��*��U5�
��.��E��o��������v�4zz�f_-���>����y*K�z��_f��[���;�v:�x���&��$�y����A��>"W&
�[�'���������KBgU��a��8���_��s(����^��4��r$�g\���p�Q�g���~5����VF:J��T%.\PY6���Kjm5���Bfl�����%x�����h��/"i��a�^�6�&:s�_�u��z��#����<����}3A����T�?g�2IZ�f��K8�R�HP�A�����2��g�U��������RY��\�XR�E0k�=u�P��5�����f����������2��������Kx�}P+��e��B'/�=Cb�Ct���W��p��]ST�4�L�j2�	�x��R�_����"|?l� ���C�So7�:����4|��3���mJ�=?���3_�B����W���?�������@�a�J�bj��4B�z ��G�������L[��Ih�
7[��~	���������M{G�����������f�)>~@���wh;W�G}	�]��f�Tg������XuP��T���zl}G�T�[�M�R�V��|G���6�I��~��
-���>�
�����(���A���������*�h`"~dx����AP7B�������_+��f���{���d����k�h�Z�-b�l-d�7�O�Z6
���cu��p�a���
�����w��6�g�N���&��c/9~d����H
i�A�s�z�,��7z��c�#&&$����r�z5�B[B��!��H9��J�v
PR�X}h��o�`�����_������]��Z��4��.��l����w���F��`�]��p�ejy�b��
����'[c���Z�5ok�K�V�q��
�z���,��5\�
��������>����0^�����{�C�wZ�F�Z�O:������3��Q�&���v�����f��	��M9������f��9�H���L���D�m�l9�4{��f�������n���ryI���lH'���Z�@j@���[����{��S�����F���8Y�Piuo�{a�$S�I�~��OOH^S�.;�
�+�=����p���O4t:��}�$����{��!47�iy+�����cc��S��Gf�>�w�}�`u7�?�����n_��z��w����S#
hD����a�xN�.�oXiM�}�5��x�Q�����z��'�?]���8 �l�I����Yf��
�2��(
_�Q.�2��,�l��A,�"B_3����Q�(���"��B��5}�����P�����J|_ks��w���B��o.�P.|�u�������N@�((��~�1b�#z5y������"�� ��B��2����#����m��L^,���
/�����_��T�����"}H��vdU	!vhU�wUQ�f I���������������[v�q���J^�X����[Hd�'������w����i���/xSo����7���M}kbx�vt�w.���@g��5���xL\&.��{�����n�NdN8�De�2Q�<Qo�1�e���5���|f>3��8�o�k
����s�V����}~��I�e��,��x#�<�'�����dxv��?aT2*��JF�ZT^���y��d^2/�:/[��{�VO����8e�2N�w��wZ2$�I���d����Xi��=�-������+aB5����_�R�,�����}���~&�w���������`�=�o�����g�3��&�����X[�c��%�{�#�mm`�2E��LQ������Fb�p�KG7��e�2N���;����7�R�������~/�)��a�0e�.o��E0�$S�)���@�W������J%��n����M�u&�B�N�S��2�S_��rs���W���U���G�b��E�
*��e=�0�6�������q�B-�P/�RQ:F���K/����U
Tc���Iz^���b&1��~J�Ii-M��z)���bP,3rS��P�a���K���M��q�����,-��e8�$�H��,#YF����[�	��a�����yE�8�[��>_^*8���h�?1!@?����H����g^��u#/cB,�������`~�Z~����hN%�\�VQn�T[��K�0l���66�o%����Y��Pa��Be�B}Lb���g��Gg�'�'ON���?�	��(���d=�d�=��1����������"�{o&Dc�ro��M��).�-�k��Om�D��M��:�8.�t���)]��1�;=����*U��cq��k!�"w2��Sse���Ce�?�fqfx.!��;�RAt4�i��`�9���#/e����[���
��yZQjt��Xm�"��K�����E��#��:��v]�����
��2TP�l�0�����3��NS8=���������?L4���L�-B�|��6�vC����[�d����j_�C���!��-%�S�	VK���������#}��I�l�1^�*h)_6~g0y�N�w���z:�'��E���IW�z���g���z��9
��o[3p��CQ��{���`f03����G����Tr�(�d���Z��hxE��B��/���jf5����q2���Lc�1�x��sXFb�2l�[���7����E�A�Vi|M3y��L^&�{I�:������G�FG���a�����gmg�?�����e�2|�7ec��o�����qcEo������-o��'Qf��n:����l�Y8�����P�M��6:��z���z)�-������33c�3���!���"�����SH8x�� ,]X��ta��aK����j��1�\cP���Xw����`�/�����}��VP�E������K�5��4h�8F��U������"a�����e����]���hc�1�m�/�:���*O�nK��J$2�0A�-������LF��6�m��Q)~Cw��������aw��4�
6G���R.������Lu��tg�������
v7��O�{e��UH+,VX��Xy�
�@6���,�����]�3�/��!Z���AP<�)�i+�Ge)r���TL�j$��(Bw{��J�4y(�m/B!,BX��yE����o����]�.c��{��
t��c\.�����	���r��>C������e�2x�7��������U���L�,WM�h��('���g�$\��j}
�����w�sR��7���?��9��V�y����|�k:5��z�J��T�b(G��{rr
7>9G���w��������X���u8n��xv����������&\*'5�@*��o�/�c�[��Tb�R	K���(��wo��������3�B��=>�����0�XQ��J�=��Rw*	NS>J�[�l8�����D�dEKL���NzVK���C�#.�E�l��
��o�$k[��-�W<[�u�ed��B����JU�}�����������wH�~+��!��
�Z�9�����xt�b�������5���$
���w{C���T�5hE��;�~i�/��<��9X����(,�������� &������b
UJ��l��CBv/q)�J�n9�������O�=�y����i,��#�����7�+A%U�9���/���v�pmk�0k\��M�����?�|��f_h_uI[�Lu1�I�����-�����i�����A[���+��X��x�������&k.��$�/�m��q�R?��6�m���C��q�H�f�������\W�CF�����n���mchavJ��y������_����K"��%�����P�qd������3\O��.����u9;�����	Pdd,
e�@�%���
�Y��B�����u�������vM�P�Oq��0��F�'�����������'b<��O�>��m-�������6�B�������[�-��y����s�l{�3��L]�.S�����-�[�-��a��������e�2u���C]�,c�1��e�n���2o���[��y�]J8P�������Y�,s�F���U��T�Qb�E/�;����n�r�0S�)��P)����X�e�2p���T{{���������6ei����g-�}���+'���b��=��K�,-XZ���Xun�D�H�f
3���7m$i�g#	����e�nU���H���AFz��v�TP�Z�(��a��fX3�o��g'�VQ����f7�%_�v�H7X���y�m��~�E��
��:��I�$e�2I�O���l�x[�\�l��db21��w���G�3�=�����/X�dJ2%��w����9����{�|�N��&��������I�����Y$�%"�~4�1H#�$M���@����d���N1~���F���t{����m�_�,���e��������Df���J��ZR1J�OW�`[��R�� �>m$X-����*H���$��fE��p�EJK������r��.�?�����k�5X��3O�/�X������%���4��������aA���[_�*������o6a��6?`Wlb�o^�n�a`�+�_���"q3�lMiB�������59�	��a����l���Z���&E~N��#���w��n����{�X�R��J���*s~(��\�����
��U5W�S�OW��������v� �W�K�=����&$6!�	I�����}��.�N��p��A���11�,K�8��z�u��_�fO��p���Mf3��������;�<�i������=Q�?X��2�p|/$�����v2���_�/����;G����|��'R)��A� }M�����ci�(�<���W�+�������b��`�2i��L�wwU�:�^��s��+D�l����`w��
l�C���E�~�j�iiD[�\c����0��p	�}�I���/]������l�bk��Y�S�v\���z(]������S��4����kZ7����~C3�M�������^t7{��]�������+~Je����V�RM�ar(/.Ju�d:J��*��������O��o�	}���S��'0'��$0�����|���?����_����F�����:YBUfU^�
��e���������~O~���������D����!����$�Z�_3R$fVd
=��L�nP���~)�NU�.�[���/�R�<�Y�L�(
G�
.��R�:�3x�rI��SS:mr+�bQ����dO�������C� �x��Xf�{u�2S�h'�V������L]��_�`�s
�j��#�d��~���NX������'?�<::?yr�_�����L^�<�o
���j]����nS��&?����y�U��)������a�����J�[9���9�v��\�/.F�!�=��T��3�BM�<�}'��VO��_�)^��<��L�|��\��N;�t�=�S������u��A{?���d��N����b��j"�+�����$O������y�'�g�B�Q�+��%����+O��X�x)(�t5\�,A��3k���W}��X�G��s[�����K�O���j�����PZ�,��b`��r�c���sG�Zf#���M-���N�	+��Q�w���vv����i�_�$c��4����d�h, ���K��&����_��IL~9�O
h���/��������u�A�lMj`�wX�d�t�.RP�P`�r��~��	i�����_N�{|<::;;�������;>��
�R�6m7�+�a@B������~���/����W*��L�(�`�:l>B�[��)�U���<�P���~�P}6dK�H�umq:p#u"e����1��U�e�
�	���*y!4
6�5����~1�d�Z/���Y	��aP<9�$������M����2��m�� E������I�T����r���,,
�1:U�	��P���|�j��P��1�B�����7�+SY,(�W8)IV���!aT����T �` @��q�H�
�)}�X��TY*�0(�DZ�$meU�������H�Uu�������>z|����'�����{�������&�>��x-�������N�{�O�b�G�k�����n��zh���7
Ef���3B]���3����S�	prw��n���B�4(0���L��a����Z~�!1X�����/h��*Q��{�)�u*U�{��XLPo�/�o�Yd~����1pe�;�F��Q��2{���>����������Z�V���CW��]T����_��`�O��vOA��g�d���Sn� T<�{�%����W=��[���x��pG���F_
����Y�y������tT�/�kW���-��	��:�_��q	����5���n��J�������q��m�}���f�C�vFD�4���,���=�����-JBrc[�����}�p����y�����G���+S�����[#�2�0n�0=z����l���|�^�+���+��_<��Hs)�V�2��:%�HA�l@W�?���<��AD�5����
�w��ih�Z������#-��/pB���\�������WT�.`�j�u��5��ZP�@����e�������u���WC�Q^{��^�W��
M��F�������Kv��xQO?]��
mz�5�t�����#�T�*���^O����n��p��HT?��Y�l��.������u��7#Pt����4A��T�N�+�&���"L� j����hNr�?�������>>:-������G�������'TmM/|�`7�i����A}l��3\0�4�4�����4A*M��%���+�+h0��T8�)�Js"��@���uxj��������O���k�m����a�M����/��t�/
3-�5Q�Ll�IV����i��y)�/d0����<�*K���)#7�X�*����J��J�R�����8D��sx������� :�n��^�U�X�_�
���m�u��mc��U��6���z�G��%v(����E�?���������k��b���d����a�"h�".v�}��	q>�/��P/�}��X��������F?=|~r���3�T�rf��i*������W8����r�(\I��������G�:���`���3��)�
�5��Ef�K�p1��]�S2c^������|+X�aFfl%�)����P��I�����J�Xe�����n_�
���p~��)>�Ij ���K�x)�]~(��TF�%]��^]��b��q�����!#s����*�
N�M�<���8�o���)���m����/e(_�N�I��R==��L0;)�T.[k���G��������n��Q��L9��XP�5�*���N@@�u%�@G���������0x��wm�|O�*�@q�}�o���[����B���/`D�������*'��^�Gyx���������D=;���
%����L����C/�P���Lj�Ln�$���|.\�jh:�u)��X�<�O��\���L�*,�"�N���f�������O�����+3��
F`M�����6+b�}����cy�����O"�C������:Vn�@��R�t�������R#���[����Lxr������ZbQ�S-����6|����
�����a�h�M���������{-S��-��C�|�h��T~����Z�G�F��)�5z&���75cx���� A��D�7`bGS�L��j]"�F���'CogjS`#����pRi�jM��Z3Z�q����F2
���o�Lv�-�������.��g�|a_��p��#��5�D��O�}qNU6��lKev�f����h!z]�b�O4�����ah�j���-n��K��e�)�����3z�����; ��?<���J	l�Rq2��A�l���F�j������zK5S�t|TP��9.6���f	�L�_=�&7�&�E�hV�S����=K���x
��`j��v^P���'E�=Co���V�x|~�}G��������:t�������~����n�G���^2[�p��e�0s�9�f��=x]���9S���V�g�B�0��e�2C�Cy�,Co�,����w���(Mi�)�,{v~l��S�
.�\3\�0��T�Z�eA\�j����L��:V5��@?K���k�T>Hv���M>O�
s3��S�h+��X75V�aA�ww��z�D��%K �@,�XmQ�����(H�33���g������_b�nh����'e2d�0Xx�p?[���s�9�|�8�@8-j��2AS��.�p�t�:;Y��f��W�%[
�����O����i�$'���2P9E{
���,�����Md��$����p=��c����ur���L��b1�'��J�������$�F�U�aR���S�L=I[�2�i��������o(���b��
�|�kb���AU����1���i~g>J��i����\�o�h��YKa-���Z
k)[�R^���~�Ig#���V�����m-��i(y�����[�>��oC�7(���q��cr�-+��n��'km��}���,HX�� aA��������a�0s�C�pDU:��
���C���W;5���nG�2pH;BghJ��Q�����������2z�.�ll�{M�/h�+�;b�*�Bj�jA{��4�M�aK=����La����f*h�#�;�W,�X^��by��e���m���tt/��
�V���fF3���|�i>�S�dc<T`3���h�.��xtv�������Tf*3���[������x�>2E<f3��b�A��1��LB�2>��*Y�H�tpzQG`jm�1�(��N(:D�y��Nu��*)���g�c�'A@�XU`��-p� ���H���Y���k�N,��Z+��QI�e�������q�LU��C:d#>��.�J����{
��S��7>���z���hp>���0�Pd��(�%:���iM�=��^`�a�Bx�& ]��J[2�=�X������P��T�,RY��H��H�Y�0C�!�~��YF3����h~��L;�w)���A���"�L:��R;�'���i��/J��I9(�X�%�ST������2U'.��0���0@||����;1Yi����8��r��'���Y�zX�_���jmE���a��o��������������#�^a�"�E� A�\~F���
�K��0�H��\�0\�i�b1�x� F;�N�c�}�p��v��
��6�0��f���T&�����������3w���]����4��#kU����T��������\���`���c�7��d^'{�y+oH��cr��n�k��g��i��K+��$�0���^�ci��y�����K���x�F�_u�l,B�d��U�`�B���}��c�l/^��*�pi��t��dBy��v����\0����-�f��U��?��>�7�����L;t�HqX�:��XU��?c{�
�������!�0-j�e��l����������!V",��WRv/e��p�uE�L��Y7�o�$�����=��r=(mo�m�5�o���sU/��G3�>C]����u�
�g<��q��i���
�����8�i>�pw�6dY6U�v�e���������)�����NY�/K�\���<7��k�=}�l,��m}o?b#����*3�j�SS������N���B�Q�u_T����#N�����O������r��N��u�=w�!�?}z|w�~89=z<��U[��]"�,��|�������<�f��U>V�X�����j�s�9��e�2g�h��F�p�LP�^<�I�	sw���{��'��P��%�a�Y��?�!��g"���^���d��������IJ1���������{���z�M�is�
C�h�r}���?��9����xUys�W��!0`�nO��S�TbN�&�=-Bo��"��=�
i6(}���=��:��������bV�#��/Y	rEv�l!,H:7����c��`��^-����s����-���*S��
��~�_��;n����o������h�����:������}���|�O�c&{��/����Ro��j����$�7w�����]��)��)a�����2���&���K����'���x#B��*Q�[8����������`�q��KE���D&S���	0�/�*|��Rh)9CcN�����J5G���u4�%��N���u
�y�����[��N�UT��dvM��7����T�Cc�f��W����W?�Q_��h���X�$�u�������1�h���kS#���p%BY�f�UhV�Y���

��y1��S>&w+AF���L' T���>-_L��j���WE���[R��.��g�3��[�};��S4o��2����c�1���xL*���U�1�4��F�>��L'V��	����hF4#����"�k����_�/������=��jp1XK{)K�>a3��`��xR�	�M1|�_�/���K�_$0���L_�/���{)�J��`�����e�2o������P�^f/������u��L��e�2}��L���/���0��`0�6}~1>�v��W2c3��`�m����@�R(4����Py:X97��i���pf83��gLF!��#JU�������S�R�KUN23 ��g�%���rL��-����rQ�T�bJ8��tM>1��
����x���M�G��}�B%Zf�,��5w}����l�����������>�+�(S����
��=�;�R��K�"[�";;�T�)�������;Pe��T��GO�?=={�����F�O~:9���������B�zf/;c@$�V����A�?�?hde�$1N��;���B�>��/�����z�B���Dj|�X��
�2���G��'e+�|���k�	���9&�y�^b�R��`�WCi���FkA��<��9fR/	7�d���EB�ezr-dj�8��O�u(�K��d�fr��v��S�&����^�U�	�t���O0�����c`X3�_�i�S�0�����1�Ia�g�B�61Q*�����&�.^A ����*J3��]P�]13���"	/�7�a���"�z��<��g��r��>��m�}?�jK���7�0X� �B�D��1���Lj&�6I���Y�,Cv��
	�0	5OF��i������,=���)Q��k8ke_����<t��n���z�$�������#�D��dw�9����g�3������<��=RY���p�M��L�&2��b1�x�v����l1q��L\&�-���#k�E��`(W�U���8���7P>��,x�H���@����~�0���
S����3�����fx3��or�C�`|�r���DL�Ke����kt� ����5���	��fB3��j�����z����3���D�������w>A�[ol�xs���[U���g�3�����(l�q�t��Y��a�~��e�2}��L�������=�N������]�.s�����ww&��U^J�3�N��06���3���e��-���U2cE����e�2q��"�f���@f 3���W���T���I��e�2i�B���&gE�����m��������@q��E�r��z���x�mm�������/k��b�v�V��8�_}�<�d�����$��(���3n�����D`��N0���g������k=?
��� �`��q����
��o]�+�Ne��o�W�?���bc��>��f�r2����R��z7������k���oS��/���	uzg���>�z��F�4��3�
��<[�)�����_�{���US�nM>�(�6������NP@���F�����������C���7��o�b�������O�[:�j��3o�w����R��:�h)B2O�3�'?G���
6�N��3�����"�}��}3��A���D�����d������o����S�15�k|��
��4�:]�&�Hw?U<x7?���o>o���K����B���eN1V�d`�p3�����/V�Uf�7-~�����u�?�q����|Q�%B�����/����V��~��x�4���8}���ojJd3���bF�6������f3���[�S~�edf43���fFoWQ��-|�_�/���{�{C.�Lg�3����t��r<	N��b&/�����}�zq�cV��f3������B]0q��L\&��#n�8��ps��������������'�y��X��6�Cf��K_�9��<��w��\R��������Z���[��t7����vJ����k�9����Gb�1�x��x����Q��c�1��O����&!��I�$|?Ix�,9�C�!�~B�f��0���@f�{��
�g�5�s���>r-8����J��"S��r�y��l������r+�����/�)�Q�1;�[��t���L���<�1�'�X��A��:����V<�*����3:����W<�5�`\<6���9������y����N�!����KO
���gs�G��XOb=������8�.C�!��e�n�g8���n��o���\��UQ�rU?��J��v[�N�Zf�C�L�Z�>��o�j�W�	h�>��)���T�9���E��HM~�	�8��d16��RL��x����7�.;c�P�\�`!~�I���Q�3���V��yQ�X���8�Tx��.�xG;��*5V�O�x������J*���\Q�3�DU�t��X�ar�w�M��<}|tr*�q�F��W�M.6���T����_���1/
��R���B�J\�r!�O*���R��/�V��bD�n�w}�����1�c���<nd�3
�EQ�'���J^�5Oir$A,��, !B!��B��?7�*�* /�s
�t�p��-	B�LoF�$�d�3��~��v1`�X���`Ez\=3s�r��:�B��k#��z[�][��������k���?�/H�o��rl.�P��M������+�l����L�'�B|W9T���R�B�����'��J(O�y�����d����������^�Av�iFpWh�6��l�~����rY�o�������;;O��3_��/)�P����&��n�����	����C:���P�����*'��+0`=��=z������CK�<�|tc���C.�y'�z?�2
4�lKv���uK����T����h5�WKd�����N�\�jL���F%C�B?6��N8+17���y�l�M��f�C�^�	�����
��z�:G'�����k�G_�������u���9���
���^����g������Z���Zg�:�B�����U��^�L�B�K�c���d�W[�c�2�;1�����_H�2;�w)���E��U�ks����1�|�:�^?�(6b�k�G������������JD:�p�C^J�a��{lif4����R?��E��UO���'��������'��GO~����x=���/���
���j��B��Xd]�n'4���l#��P�[.dn�?B�>�8������E����j�f�5����/�)z�[�U\a-MU������<3���>t������
J��a��g	�}��?��2�D�����l���h^a*m�9������S�L�H�x���}
�:��0\kx��_+�V|��mUK��MV�.�
��Y
��#��3�`���t�=l��&���nF�%CYm�	>M�1oc��&�����$n@�L.p2����I����U<G��=+f��5�6\�
����t�5J}{>���s�R���^�dh3��m�������#D�������	�4�6�����[C9"���P�o �-���T����
5����g���ug�|]O�}o����O����j�~��mS� �Hj-K,�T,�K*�T������5��2#���Hf����Eu���������RE�*�)����td:~�t|��o/1�u2w�#/�5�+����;�����K��j�����e�2x�:x7y�/Q�z�m�S�)��yz7y������dD2"�4"������U%��r7��'�O�8Gs@k�D�hk�D���}[�=��u����g�3�?x���_��}����c�1����x
��Z�S�0s�9��kn�-���>���O
b�Qml��S�)�f
3���
����G������fF3�������e:� ���a3���`F����!�B�bQ�j~�X���As���2<�%eb7(�!�v��a��JkF��*��������s���H�+FI�9-��$>�c�-��)2x��7]'[>�Wa(�3�Q�i����r�����)�M�P[�������l_Ut6�;9|����K�����P�X��L�����V*�o�/��.-�K1C�a>i��)����������-��:����W�g�\���T7b��N���,
�������x������'�[�����L����R�L�>b�T.�+~�Y�S��)fT��g����NON�J��P�cu]����fR��>�s1AkH�Z��������������B��0��+L�|�v�T!*�1S�hlC��w�U��3y����<{�x�\�+�����vy��
Y��b�d��i0xs���w���o�I�{B�N����Z ���GONx|�����6�a��{�UUZ'3�����k�V�v�y�Ja�s�G�y���������1�$���0���h���|�fG�:thu)sW��k��qrzz|6����!����D�l�-��)��O���+�h"P��`Wfpi��S�6}3=hR3�������K+���S���|��4M�(\C�5~��Y�g��5����}2��g��������"t\�,\3�;"���$M�<�GI��z�}���h]{|������w`��%�O�(�I']��wnhi|k*�3�l�`CZ�
�����Bn��	N$|�����@�o�fM�}�JF�W�����t.�X����X��g)�R����J�.���O��La�0S�)|����y)�$��0�*U�+��bf1��Y|Kv��R�^G)n�&����dxaj3���Lm��[�6��A� f3��������$��3��b�-i��F}��+��$zV������L�N�=�L�{j��93����g�3�o�
��E!�;$b�\�1��i�4fo9�cp�^����f3������bp�N�f���������vs��{VLp�����s��X9;���MM������"3c�����}�>i�{�����������"@[�T���.�u�
�_��ir
�����������d��R|���8'8��Xu_�eS�S,�XN������a���f3��[�,�Z��<�n�Z��2���@�6�&�����c�1��q��m����dB���1ic����.gj�6XOL��9Z��@*���|\��soMW�b��I�:K�y���<��4�=�..��N��27.\%t��rl.�hb����B�b����^oTK:�-�V?V!����i����iU�*����b������k�@|�3���cq������[��l�r�gJN��-����������VP��I�J��<���8S35���
�&�:�H��K���Q�����U��K��
�o����M�sYS���N�s����8�.���C�?���/xj���e%�J��[����J����7����|f���#8Z���;���g�2C���P�+P�uo�d���L��rBfC�e1��L\%�l!���*L���Om��1�H0���
.�K�����j6[�H���<��u��q��}�>c��b��n�h���a�iq�����e�2{��[`�[���e�2~�w�o-0�����e��e�����>�	����e�2�2�n�G&0�	���?������e�2w��}Qog73��f�]� ?��B�limm�]����s-�e��V1��{s�{�����r���dMQ������u�3�`kV�S��[����Z����x]����0z�TR�V_��9|8@���9�������k��P�MU:�k����?�����+5��N��,�:c�>�����=�b�������#8�q��TZ��_5���a�j�v��ak:����{�kt�+��ku���~R�4��R�64\��p�����S5�U��yUf.�ggO����*/Ub.r��J)�p�B}#�=��?D�{t�\^�����5��
������7�m,{������jcR!e;/��+U��J�{e�+�Inm�����Dbx0�h�����E�z��;�J,	������P)mrt��,G���r}��N�X��]�l
���}�U6��Z��d���G�X:Lt\>��Qm}d���'��3�S�(�Q��r(��{���s���=N�l��� �St�_Z��2�Q��qf:�TQ�^��F�)=��?��5�X��0�~����C}��u�!���8�Rc6p�f���$��X(�B9������h2���R�����������;;���4�n<A\A\A\A\A��;o;����#�����I�t��^I���<�
�����S?�������g@��7�8����<!e�P�:	czk�!�c��L��kjd�;�J����7,�	lre�L��SWj����S����j�����T~5
�@���8z3��s���y�f:�F�I�>U��S�hI�J$5N�L�N��&4.H�Q�*hl���U>��LgQ�X;��n�I��'c�v�/h����u��0��r;�m�nnX�f�9ej�����0��)��Gg��Q�C���?���:���p��Q��r��F�~��-G���`q��1��RG��WH�6�������{B��~�ts�\����u)��)*�������oD}	�������~�rx^=�k�W�w���~�E���OnKS���YK��T'�z��mu?�D������O�O�Mi���R����md���44)����`A��>��������9=��X����Qo�����Ch�85�[_�[����������}���]>�g������'Q��@�,\�Fses���Ux		�|�/+$�=�l,1��fU��XT���UG�&�lC�ZD�BJj��)@��O�&>�g�����.E��|RO��V�����*�SG��=p�"�W�������2��1�8��Y��I8ig���?��Q�W_Z���b��&�n�
�'�m���T�Q#�AUa���+��^x��{x���dw;�ZV����1^
�y��a#��),���?3P=���s�= 3�\}��\��!5�J�\�����bb��
KJ����T!SD,���<X���L�(V�@^�B��[T�
e�)��pj���z�
J�*�3�:>�O���W&���W�H;�#����?�L�d*��W��/�#^:���,N
@l`i�)���������i`�W�9�'����f�l6���]+k�A���9Zx�}�=:Jov�Y�C�����c���o��_	x�v������uU��i��M�I6������r0J����{��;8w�
�_�R�u�p��b�x��!&�;>��:����Es�E�<��#�R�P������Bf,��P��0�D�T4�-<&���^:�odL��2X{�(��2JSdu���H�a2�b�y�xOO��^�V����f�����f�}�6L��O'�%�*�60�)7���)
~T��6�ffo2�������t�JG�
85�D���5�E�^���,$��1��-���dc�&����*�8#�w�Bh����a���m��(�5	`dm�ieE���kB2��eiba�Z�J�S���N���E3B�>7�+�(�K�t��R��}kU�U}�hS:)�����]����y���
�����;�/
}�����%�yP@+cRIi���c�>.����4�o�yV��j�D�)"	���&�&:��������S~-Rzf�4���)�<�=��xR*��!lv;g� **4g��We�7�j|H����������e����t;��Q�g���0��������I�\/N���
^z�%����� � L#�L6�lN�CuX:�rW������L#^�_����Q*(��4�V��
�A���eu�47O��,4W�&I�1&�����?g��we���������FR������k��C2�T%|'3L���H���X	(��~�`�
*�p���3�����U;!ZpM�����Gf�H���� e S��{]{��Y&i���*�������>}����C�q��l�6�������R-x
���>X��������g}�&���C���}�vU+����\c�����j�Ww��X������c>�������z�n������Q�m-��[Dw��>��g�4d*p�N��m������
tk�kg��e�Q�t��z���2su�1vC�j��W��8G��g8��n���nff����''O	}Q�O�	Ip80���G��&�M6�,�%
��7��:`���SQ�u��/����}��.�+_����������HlZk�N���0���k��q<
�v�"�	�_�� ��]&h���	Z!��������oj���\S������}�5��K��}w��J�>����iZ���������u�^���������YV�)��F��G�[��Z��&��G[W������6�^���/ZZ���Wa���VS�7��TZ�
�k��A��rkn����s�M,s&��SR����H�����: �*0�id�����+�O�*�f���s��M�)*���#!�F�C�4*�y%xG�2�C�w��P�"<6�l��	H
�'+�j"@��H�t9���\�tk�r�,��[,��[,��.��K�zb+Ju�v]�/c�]R�P�ccc��wv*F�W�V�V�V�V�v19��#�@�@�@�@�@�6��Q���a`g3��:���V��>>�l8���PdB�\�X^��8�X{��(;�@��x��Ma��:�����Ew�c<�*���d�M�"}����:;~���85� ��yJ�=!?��Lb;BK���"�������8���qff�*��I�G	3{uz��|��9h��k"J����nc�[���������?u{�Ar5i�K>3|1`/59J��C���)J0�����Y��-��K�+@��QLf��E6,��������X5/�����ko��`eu����E)N��U��.�������1~6���F|��x��.
�p3�
�kw�
����<^��U<�u_N& ���������d������c�!c�����81�;	}6�~�)�/V�?���<Q��P��v��J�D����L��e�-o����W8�����ra�4�i����d~����o�4��B�XD�
_8"�X�������"�o#x�[��5���:����Ep���������Ov����z���%���������6aX�����>W��QxpPH����#�E��������:`���@3pL����������+S�J���x�;���N<�4����p��!�C��d9D�w����0M�R�B�M�6!l��	�k�b
�7�������G9�������oE��<�[}D�=�&
��a?�~���� �q�,
��/�����@q
�1��r���n���[��%|�@�
��7o��e�@xK���`�|���A[�4�������������p�ex�*�a�U�����O�N_�zvv����-����������V����M��9#�N����(�n��\�J��,��d2�!x"����t���f
�O���Q2�8`��������[�~T��aW�NPU�|�5&P�0����BA_f8�sx�3����^�?c���(
�'Ve0�����Q!E��{~OX�s&������o5Av;��~z|������������I�?�z!��D�F#�����U���f%Y�U��������8��_�L����|V��b03tA���C���(�b��7fpH�-%F)�R�0Ja�[�dU)q�:����fCqn�	G���|^`������x}rq|�����J
�a���"���gNj�i�I[���AB�sgy�E�J����*��v9b��T'�v�������'f�A����HCs��4��g�4S�c�lC��b����
�3��.�������.������y�y�
�.��DC�x5��E��� ���%�Q���2����
�8��G��M����h�qp���,���Q�������B������wj����z�D�Y�e
�&�d�n�bac�I�������y������AS��P�3��_k*I8L%Gl,J�Ts������s������������v�����Y_�-�]����M���N��|����%��}-�FZ.��p����Al�L\KU���JJ-�>J���L����������+�N��*��x�R-�-,�:�/�0B�:	���6M�4W����9��sS�D528�X}�DvX�<��S�$�	�������:���Oj�MYN��9��H�fQ���/R�H�"�oQ����C���y�)�E/����W{�-�`�`�`�`���^/#��_��������V-��3��������f���K����AaAaAaAaA���|�k����89XXXX���Q2�o�z2)D`�������&��QYL6�$fy�Vv���S�b�H\"�6�DDDD�����34>������Q�W��\K��,���O&����d*�Y������E�d��7���7�a�+*����d���M��P�b����2��������O���DfO������~�;z�� �F����:�c2Q�������
�]ft��h��&�]%L���~������/����	NR���8�A����t�z���c�`��e&��d���e��X4POlF�N��H�������n�W6
{+<t��Ks*a�������t�t�������������?8��\�Y�������9���,�&MM�O4}`R~`���'-���y�y�y�y���GsD����@�@�@�@�'u�`����D�"�	�	�	�}*��v�t;��J��x��G�.��9y� � � � ����U�{@���C�:]>'2@@@�O�_M�7��r�2�U3�^�P���9<W��������.l<C�l���t�t��XG	�y~��A����
'���J����?]0���%�



~�H	��_
_�<zv|x��wzVD����1p�&�}�_�Al�-(((��@ ��
���=�OpOpOp����&���'(((((�7C��d����z�z�z�z�.��]�E��qr�!�'�'��	_=�1�X"�u����Y_u	�qMA@���a���>�-��u��)����4||||?a�EW@;r����W]~X��S�W�0�/��Y���L^~_*�<\?��9
4oK&�����q��al�I6�rc:��:�<�Al��<3�Q��K��)���n|�Sq�DY��&��H��zf������M���8���M�fp;v^D����8[�;�R�����d�.�=�����C�&�qb�{Y��x�#��c���|%��B3���������:t�<�n���@�����C
���uL�yy�a��i���1��<��qNE:���K�����<MM�����a�L��
3f�I3��oBLF~����|��y��[������R5O�������Y4���9�(����mz+�~��NE`_`_`���}�1{�N���3�l���^YL�t���3�x�X82���R=��|z��c�qoC�|��O
������Vm������U��yh��ey&���vs��0�JNz6����O����!|���8	C�����������������,6����c���=kYLF�����~���DrO	�����4*����������[�����4�����[�\o���f��a�a�a�a��m���Yo���_��]��W�v����������[��h7�x��Q���l����aj�&5I`<��^�0/���]�p*��
���c��J7:�$T�y*.�c��>z�B]Y�L�&9�8|��n��CU."�����t�%V������
��<�Thg��(�g#�b�L���;�Rs��%%�gX,UQ��/��,���n3�S�0vV�M:��L�y�������
!Yg�T�����d���Ik����sj*{�Nb;�C��u�����u�8����$uZ�m����p�g���&�b�>t"S�C��{�*U�0��C���^<	T 2D����������3<}P��������+&���h2�SRh3R
y�8����}�X]|������Q����n%cQ���{�+�.�.�.��7���zT�DnAeAeAeAeA�m�rg��ngG��1��*���&*=�>�9�s��T3���S��*�u����)uA7�HI�x�9�
�:2�]���������W>�PP/��_����b=���vFoG�<�q�g���z���/U�:	cn��q���s�(�Tb}hG�mZ���m����/��F��T#p��v����vvXT�04@�+U,���S�U���W	�^u�����	���Wu�O������E��=����4|yJLR�-��-�f��&��Q�\����!���l>S�������+W� \����-�on�����a���+���_��p���;�D�D�D����R�3%�� �d
��|*�(�(�(��Ib#�ak��Qh����2uh-[H^���iE�/7�����O��x;|�X��^��q�����T�]��U��|n��t����>��)L4:��u�]A��,]������v({���	.1iH�FQ'���AjeS�����R��!���05�c
�/��(�T5�����Q���Z�J6����N�G��g�}�y�����*�q����6���jxK)RL�(�fM�o�����T��R?��QDI%E���EI�$K�����wJ�[>��{��S�O�O�O����?����2	��`�4�����������Y����f��g�A|�b�O�
������M�w���`�`�`��
���_�*��O��_>�x��������Ja
�4:��M� x+x+x+x+x��:���k6)06�/Z�M�E�v�^U���
�*��BCW�������(�6}��exd�L�gL��S3���n�>���b@N����W��D+���,/;���i�HN��E��0����+�E��Kd��8�=��G/h�4�@�%��$���B+�4
0���=����wAlt���0E��&�+$�[���� c�ez���~�������6��ja�K��F���Vcd��F����Z?�!��������$@m��|�[����A��C��;=3���K<sQ�^�c�;
a����biLupI�f:�qI�y�1��{�,<��"�2�[�	������$!��H"IlKSh`��yv|:<9������WG/��OO/�IC� )�bq���E@����� �G�y�+�kUo��%"H����G�������:���D�����h^�i��M�QC��e_[`4����(5�R��O�j����3���8~������B~)d�oJ��O�jJO���"48~����\�st����fNZ��7�F����U��������+������O�q�nE�=^_u?�>9y�EW
��`ye����mMR%��{�(i#����DHR�/�}�Iq�Ci�HcA�]����
�OoXh�:���i�#:A�����7�@�u�F�C�z��#��QzP�YKQ�YJt@�b�b�b��o��Z!�aL��*w�Y����an�f	+l@���a
XA�do��
�������qjL�NTRV����o�p�p-�+�+�+�+��
�}+�*�*�*�*�z���y����c&��Y)#�I����R����L�K%gc/zw�++JZ���0��VY��J>G���dy��P��*�������+��Hk�obn5�=��L�y�!Xck/�y�8�L������^�a�L�X;����;j����`��� 5�FW��(tXD���;��	.J&���u
m�����Q��I_���|�W���qV�9N���@�-�Cw5�� b��L�$��������)�F2���u�������W��=p��.{�,4���8����z]�L��<v��ct�h"z�>+�r��M�������]�4B�"_j����Jl��][3�A���]��H@2����2��D-�M"�P���#
���f'8�*�q>�Q��%�
I��d��)���E�G��G��n���<�����j�~H�=*�g��o
5@���b9�i��O
��1�������a_?�������2X����Q�i����PL�q2�j�oq���������h�m��5�h�H��� {�&��[6:||���V���&����2�x���k���Xj(�/)�R�wM�_2y��%g�}��2����&�o�@�Y��|�I����
������D��;tvp>1��n���'#.�q���L�_xbw��q��z������ig�V\�Y��@m���<���
I�e����NggmI�����l~��Y%�����)�N(��]�%Y�T�Yl0<S�Y��C@����L�A:1��)���]<GD�\�a7�Sy��ef�W4�
�Ay��3���a`EX7xu����~��a.��0�"���9G���G��(��_E���'�Y������q���NE��
������������3:
����:&6�/����kv�f)����������Ur��e*?����V���0�g��g�������{5v��iO��L����Cu�q�����#����L��3��}��onz>.�j���/k�d�rZ�8F��Q�;���1��dJ?�)m���@,	�k���l��u�A
[���T���er?��-����uO�6F)�T�����S���fj�j��|~���z*jI/��+]�V4�Y�c\	�F�2i�I����&�S]�x�F���*�5��#��&W&��
25cWx�S�	EIf��	#� %N���l���'��k��>����X
/���M�����V�tN���iV�Q���_M��>n&R��0G��M�B�u�EP�hv{������N&��	��M��H`�O��K�3�����������8��f���Q~x�u��E����v�
��oa������4&hG���84�-�U"�|SD���>��������C����x�
g�N��sn]6�Y����$�b�?�h/x���`u�I
��:C|%JT�Xd�}��iGAgK�]��E;���9�nT��i�0�4����J&��t��n���(���7��R���$^o�L�����uCS_Y��T;���q6��,�"����+�d_�j�J��j�4,FYp���8,�l�y�x�
k��y�����y�������������I�2[f��O��@3�^���/6^�A�:�h���������������sX��s��_?t�ea��`(�`j�#�,���_F?O�'���3Y\��:S�m�?6
a�������m�8�|�U��I�L���\�]
�8��� ���X�A����iJ�jJ�d�S����[�����}R
;�/h^.�&L<�{�	��1���@_2 ��8�������e�'�Faff�|�-��U1 0�v\�n��>z?O�oi�n-M��!��c���[g
�v�Ft���y��09��5o�nTH�4(������Q��[@���i���]#�	� 2<�4����������n �.�eb>�@���������3����3�3��x��f�
�Fz�����Q��l������l��	��Y�U�����+_���_���j����p�z�D�zm���k*j�.~�XF�8}��/�>8�0�������O1@,�!~�k�W��LhM8�h�P2��vJu�N���tz9�$=|����O?�zyr|1<>=�8><�::���p+�w�6�k�p@8��T
��(�l.p�>����;��r�j�����>�p�c��x���������O���������:��@����j�
QX�r`V6�<d#����*fV=U�����uo�0����(e�g��&E�T���*U�$&Da*�;�����qJ�;��o�Zuz�@��.X��T'�Q��|bH�pd�q�5�6���G�R��%#�Y�(�����8������)1%�L�0Bg��GA��S���(uL~L9�
�8��<����Q��v�_�&E��Sp��rT����)��XqH�Dc���<�������/����c��X�7���M,��J����w�&����L�vp�����~�fwrD����������0*h,h,h,h,h|�����*��J���6������������Y7��)"�� >�j�G6�2���1s��&.s�.��?�d��f=�_�)�3�c����%V��1�'"����[���+O��w�����1*U���cRz#
���4�_:�����
����I@a}����m	�!\I��p%�J����+Q���������&��0:�U�D]���A�����	��S����>��0���Mo�	��[9��!,BX���m�2R-��%V�V�V�V�V�v{h������tK�V���+%f6���3s����������=��@�@�@�@�@�}�^X�18���������t��Z�c�c�c���H�5(�:�{XL��������������GO��r��0����%�i�a[06���@l%�~��;;;;?B�l�_�F<<<<��������wFN��������������C��� � � � � �6��^��	"""""�?"�s�Eaaaa����j|��������b����z:7�`�`�`�`����`��f�c"qD��`0������:��eidy�����XEo	�0���8Z_Y�8�'��9^~^=���
��$�}���Q�4����h�)�(o
R��Y�M���cD	����1�L�B)0{J���y�2�H��?TQt��b#��g����U�2@�MJ�9�4"6��9C��f:�
��J\�)���8��v�M����Jf62i���Lqj=����q\g�����IP��*&�H���3�9W$�����V)��vcp+l��uj��H��V���%�$&�
W�ar6�^#mN���������%�p}������o1W�
GE���+d�j�~������3J��*0	��s��K	���*��F>GjT���+��������e���]vb��&�������c�q��c�L�i��<v���51�'�@�����������f=���PG���`�`�`�`����r��
�S�ot��c��y@�W�W�W�W�����[z�
�
�
�
�~0�
lr5\Lml��]oW��pW�%!2S*^
���;J*�f��T/�@����U�(,FFFFFoCH���`���J�@mmmm������=�0:k�����q�������������v��������
��$�`�`�`�`�6s�
�
�
�
�
�n^w@�%���+MY(G`O�B����b`������*�+�+�+�+�����"l��������������V�75�������������/fffff��o�;�n�!{���l�����d�4&�}R!,�^RRR�#D�
1��4o���D��K�+�
��@�9p5v���4�+�������=1��H�b���p��E�0i�a"��rn��S���D;�{�������{=dN;rS�O<���E<k[PSPSPSP��E�����I>+����l�X6��+P,P,P,P�k��dy�x�����������p������ljR��PZ��eid&����4�@*��)�/�.�:�><���(��K4���8�����G���&�(���*gg��5
s�I���&���	����)��r�����]L��@z��<��GF�tLM���&6Su�R�:���H���2.�*����S�	<s��1�{����@��q�Yd���;�sD����v��)N	��7�u�[z>70��r�,�u�����<.�,_��i��
-L�85���� �q��|���s�@U�^��fmN���+���[����9�4������.������s����P�`��VL4Ilj����_>�����9����[����!����m���
��p��=4lK��w�u�H0��V>����$+o/��~c�xZ����|���P/�bk[}��������]��V����5��b+��4��~�����qi�\1���(/�V��P�����I	��E`�K@SZn����vy��Q�D"3�P4+���4B��J��gO�����)_���_���a|�$.'���Wa���Q�f����R���=� J�)�p�������;�
/V���
��G95��INMrj�S����v����o g%��pJ�q��������������,KO����;�6.,,,�!�U�
�1��P]$	�����?X��}A\A\A\A\A�m �b�Fu��@�G�~D�V�V�V�v[	��N�a���Fh�f~C��������������I��h.�V{����0��B��-<E�Kq�````�VJF��f�0d��������p
wK�������������������U�!����\�	�
�
�
�n
ww�@����]N�^___�����4�,`+`+`+`+`���R��O�V�V�V�v�H��<������Y^M�>��������9��/[��	����������U$b
��cRgfvc�M������(�ui��.3�3��P�_���'�qN�EM�)Rj������y���g65}H=��F������af�$l1Nt�L(��(�����2I�����bF
b{|�V{�&�������H�ij�,���F;��5�a��������Yz�����.��Mlh���J�\����~Z�p���!s3P(�]
m�Q�;;���������<��=�������8^�r�N�O����Ho�<7��3U���xZQBN��|)�B&Dd�SA��u��U���|��[�y���Y�8w�27,a\����#hc�q#gL��|z7��0��)������C�?V]|���^�4����	�0��"���L�d83
��b#kH�����������'��>E����w�w�w�<�a����x4w�����y6�����0� ��X��A���3}i��������*�	�H��������<������GOt�%2T�����$l�Gllll����1.�h(�'�'�����
�v{{{{������
?>��#��4��6��f\w�������P���.��Bh���	���Q��.c�����<q���_bL��L����9���8�d�	��h2������#�
�3)-�^��V���gy�Eh���7Y��x90�����`�g�+��+���F�Q�aH�~6vV���#e���<���P���=j�u��75��};B4(A�����K8(�/��K���
��m���n7_^'��d>�2�L��a��!�������f�����Ukl����g=���e���+�?�|�{�F��QT05��rS2��HMp����`�����s���F!_{;
E�@�������:�Q�%����HAxX��
6%���5y�=��^�s���Zi��5JX_�z�z�N2�=������'��7�{�s����`*3��h��F�4a�f�<�R�o*\Y*�u\K���H�e�k�M��2����s$0�/"��+}<O����	���)�a)�k� q���%���y�A�K��>� "��Py��<���bk/A�@::Kh�s���9y���rOb;B<`�gQ�V�H��J�Qa4���L���,��G��F�������?�ZE���AR�3�tv����hU�����������s����S�G�`��<�}8Q�`
:�<��B�j �/�Z��	F'K��D�b��������2B{�C���D	�UO�]�>?=>���p�.Jt3�Da,�]<����m�,��c�
�U�(+��C'�}D�k@����
�2zsP������j����c��h�s�����&f�A�x9��� :��j�>�����)�Y�nm>��MH��uO���gg�?�?�P�_>?�8R��.�����Q1�0�?�r����p���������t�����R"w���s� �,{`#3����uPt�W��3t�us��@"^6+��YL�l)��3�D���7m�p�e��(v�?B���0Q���?���I���z�����v
���e�"2E�t��J��D�>�!Z��5D�BdMD;��59���^*�b���vS�<�>E:vZ���������^��(�>��l��5�8����i�� �yU%'N���c��@�T�:2VX�(�Y8JEI����"�/<�,�p��<���L�i0��K�r`d���U������"B�*���������B��)^�,��D����i�������M:��j�d������uY��Cb
g� �A8�68C�+�����������Ef�O````�m���Q������������6A��+I�(�+�+�+��]��V�]��5���<��M��S����Kk�/<b�X%<<<<<��k���c���3���&�\ZZZZZ�?Z��is�B�6�fteC�q�`�`�`�`���'$�������6��a4�loooo��Z�T/p��.�a�)�`v||||������KL	]�\*"sD	�"W
�p)FWIjAI�g�npP���Y�[�[�[�{+
�?17V������!0000�N�ZL�Va��R���E�v`�)� k{����������H��)������������=��[����&����I�Z��� B"....��Y�X�X�X��S�V=�Z��zz�"�h������*H.Zf�]�]�]��?A�|<���h����I���'a��=�*t�K�^�bt�H�P���$���/� a1�F���uY������������V9���������qj{
��%Ll'���������H�K��p��"�s=A������^�/1+���Z��C�p9�r����m��!�i���O���r��&K�`��'����+ ��\U
���&|e�rr��<'ze��8/�J���Q>��|�i�������X���
I}�E�I����S��p�,�C����X��=K�����&u���7�I�\l&6�����
��Y���Vo���!�=
����������'�(>@�����+�o��>H?��$?@�j���]>�������U!�����q�3���at�@1C+�aK�������)~�W��bc�I��]�k�O��Z�����"����Mjp��1���I�!>��sCg
9C�B�r��3����J�k\��"��F�F$g�G0o�J*�*���i����7I&��$3<��|z�!^�\��!.�~���\�s��.^y�������0!��"L�0����x��#�{+��Z��U����0�|�|R5��G��t�f:�Q2Q3��9y6��:%�"�8�Y�'�$]m���2J'K:D�������,,GX��a9�r���zog"�B�B�B���
���Z���N#�(�(�(���"�mU��s�s�s�s'�����!�h+�����	����((((�(����c���j+?Rd���RpNpNpNp������4%����f6Y��`*Iaaa��a/jV���,5��o�<
�� �5JG�^�{e�e����N��nG����|nRz�����a4��S��
^9������c
�@��8��}���#:v���1��:�����K�x8��k��tS����7�8O!���%QF���<;���&4����5XG>�S�F����L�X;�����������`�L�3��Y�=(�s1����rgB,�!�|,C���\�
w0��I8��t�+#�2��v9�00~��.�6���E���0�i
C�Hu������1
/�5�2�"�t)/�>I
�:$�%�n�h,��s��ZW�r�w���0�jb���c��a3�`�����A���hh�*�������v%��,�A���O-��4����������(q�2��AM45k�/W3��a��OR��~8F$������q�F�K�E���
`iv�������3��?�v�8�� �a<�[��(+�����6�������V�!I�)����k��n�������6mg��N����bqG��)Xw��{�������\�I��hX�-���Y�Xx���B��&������!j����E�Rxjj����(	R*��h��e�G\gD{��)�j�lM�v1��b�&��C'���\U��>>����P	���M���MB���+EW��f����hM3F�K=�8@�(P���4A���}�q2�j�U�1
P���&Uo�����c/-G���$���m����M��cjIp���^#�~WFA�E-�����e)K��X
�,
�H���t�6���J����oHc,=b��X�4�����J�xR
O6��x��A�?iX�����.
c�j���f�����5p��z��00L��u�'���zC���v�+������@V����US@T�-�J����R
��\��/+EV��6{E�i*���&�������y����V.��8���
�y�Xa��d��[���Na������Z�~_��>9�5#k�%�.���W+�b�K�\�\
��6��������=�������d"�E��6����{X-�B����g��;Y4�.Y5������R�{�������}�9��,����{�D����!�eu�W���^�F����*�?]��[f��i����z(����
SUNP97�iy����D�+������~������r
69�
�����{Nw�����[���������������ER���TkQ��T��2���L�G�E1�q/�f�y��9��^���;����t���n).Vg�m�e:�ec�`�JL�-:��*}�b�(��A����7{�k���{�y�V~m��l�>���~3��8���:=>iMv�(! �M2��]����@���W�K ��S�Q����A���8��ozp.K�,�q���	T���a������L�q�W�G�P����"J^�70�l9;�Rc���|-G�����8F#���U��8AKq�e����_0.4�._��'4����9|�!y
��F�l��	��������m��l>F�l��|B&����94*Ol:�o9=��wMsCrpk��6��9>�@��!�^D�[J��yb�8��n��0��0��W���[�L�~�������@�0�u�����c_�����e��<pja�Ke���.^����Ts1���P��gn��(����q���T'�!��1�bX���2K�S!;��k9=Q��Xx=����t��=O.+�3L}�'Sf��:S5|O����
d�3T�VBQl��~hh��.|U���L��T�E,�~�!���u���������9��g@�>�
E��A��1�Bo�"
�+���i`_2n���K}�v��NM��n?�GQw��r���Q�����C���N�_,3p�s��K�Q��;D6�!wH��	����� �~_8~m��IF/���d��:��&cO:�������.�0��./p i;��c0�Q����� @�������Y��p������������`
0�`����(z�3W9��B#�����$@��x���s���[��C��!�����5�����c_�$����n:��������Jh����:�!)����X�3�5�
��C��� ��]���_"|��M����K;(���q
���P4g��u��+7n�qn�����R��lq�+��)�M���]?��c�{j6��v��i���{�tP����~D��b'���(���.�	�H<��*\j8r��F�W

;�2);`y�G-|�+����,���"�R�
������\^��}�a��F�}����D���Kd�K��a��}��������P��)i7���
z�����H�7B�t��s��$/����+�q<�)Z�<�QV����b��8`��-%'���U4���Z�*rD'!�����'�2r�17����S
�G�,�AX0mx�HAo]��{�J��Y�U�j� #��Ra��S9����!���
�Q�����������~s��:,���SO�����=�Tc�*�<�(2p�n��J��:;�1��]&��`:�-��j��!&�����������G���N���zqxz������(r��������i�lq�'��b�A�dI��[�2�\�%����� ��<��X	06�����%X��Q�
 GR	�7v��Z���{�
�3{�����)]��t2�@��sC����A���������rp���E�Hb}��Cb��G��?J)�G����Q<J�v$v�qQF^���Q�`BrG��d�)QZ�S$���Y� ��t��<1�X	�.#��~�%k8��% E!�E��R�P�#������t�"�����@����Z+���&�F�M��
����#^6��"������e��8���1w�%�\�x>�#"uj)"��q��$�
b����#<�U��HQRDc�R��.sDu���g��[)S8�pL���1��1K����1���%N5�P]�j�/
��u��/��K��������m��w
���n |�.�pK��Y%j>�E3�ZH�]dk9H]r~�YM��,����U> �4�&�ry�!6�l����eb��������d%Q�����8a�|�w�&��v�_t�����^-�3��������o�
���5.e��=�Y>�l���
u=~T,m�Iov��U��w_4;o;oEi@��D��4�u��
��!]%V���D����4����E��<%5h��7�������h��������X(Q����^�h�Zk�?�7.f�;!)B[��
Ml
���y������}�Y=CI�7<?y�����L��.(���wf�O��K�*O�� -?5������O�E�����i�T<�Vk��>j,�����U�n�n���m��W�����.�)�v��9����0�"�'�t�P$xZ>$�8�l�+M��Ktz	��l��)2�����0���l��$��%�f��d�/���^����q�����9zHx"@�6:����3��w[55����3��.(Dl��W�;K�1�2]�1n���X�G����>��u��ns���n�Ze��VV^[e���U�'�E�h#H�k�k�p��K�v=��Poad;;�+�im�ma�wke�m���}���������������A�s
�,�N= Z�NvS��n���u������:g����f���0�#/��W�I%���K2�)M��-��	l0�d��mn�N����9����j���'K����2�&?�{Xf-1�cj��������z����������oc��{
)Ic8I��8�����D
��g���3o��+*��~{yrx|�#�'�-�����p��^^N�s��Fm�N1����gL��D����*�e�w��#B�I��0�����9td6B�;2
�'!�����Kpe"
�k��KZym�v����87����[�0��S��g^�++[�%��G��a�f�J���w�����:�+��Q����;�]y�p���%e#j�WUs$$�/��f.�����/���G'��g�����v��hE�y_�;^�;i�{MU}��4p.~�aC�7�2j����8z~�����������������/x���:��y��6J�&�>Z5l8L(wL�b�o�O�y�)jXD"���X}��-}����|�
�F���l���_�}s��	
���e�h���|@_ g;����WL2!����ktk~2��i,"����sI�Q�%S��5`�{h�
pvO���l9�jq>��K�L��X�0�M�~jg��7q)C�����(���^�2�o��&���E����C�t�42)��/F���s������Z����������]�������=L�@�@�@�@�@�{C�N�T�K�/�V�s��4������v��k��$@-@-@-@-@��@}}�AZAZAZAZA��FZP�x��f���G�%����IiN�.��7��&uv8�H����D�\P3�*?;:v4<~�������������P��"%�y�"���-��F�w-�������R���O�"���#���(���c�U"
�18�t�a
�}���J=K��Y����������|�#��G>���|�#��G>���|�#��G>���|�#��G>���|�#��G>���|�s���1�����
blame_coverage_on_queries.shapplication/x-sh; name=blame_coverage_on_queries.shDownload
#257Rafia Sabih
rafia.sabih@enterprisedb.com
In reply to: Alvaro Herrera (#255)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Sep 19, 2017 at 3:50 PM, Alvaro Herrera <alvherre@alvh.no-ip.org> wrote:

Rafia Sabih wrote:

On completing the benchmark for all queries for the above mentioned
setup, following performance improvement can be seen,
Query | Patch | Head
3 | 1455 | 1631
4 | 499 | 4344
5 | 1464 | 1606
10 | 1475 | 1599
12 | 1465 | 1790

Note that all values of execution time are in seconds.
To summarise, apart from Q4, all other queries are showing somewhat
10-20% improvement.

Saving 90% of time on the slowest query looks like a worthy improvement
on its own right. However, you're reporting execution time only, right?
What happens to planning time? In a quick look,

Definitely. The planning time issue has been discussed upthread,

On Mon, Mar 20, 2017 at 12:07 PM, Rafia Sabih
<rafia.sabih@enterprisedb.com> wrote:

Another minor thing to note that is planning time is almost twice with
this patch, though I understand that this is for scenarios with really
big 'big data' so this may not be a serious issue in such cases, but
it'd be good if we can keep an eye on this that it doesn't exceed the
computational bounds for a really large number of tables..

To which Robert replied as,

Yes, this is definitely going to use significant additional planning
time and memory. There are several possible strategies for improving
that situation, but I think we need to get the basics in place first.
That's why the proposal is now to have this turned off by default.
People joining really big tables that happen to be equipartitioned are
likely to want to turn it on, though, even before those optimizations
are done.

--
Regards,
Rafia Sabih
EnterpriseDB: http://www.enterprisedb.com/

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

#258Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Thomas Munro (#256)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Sep 20, 2017 at 9:44 AM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

The main areas of uncovered lines are: code in
get_wholerow_ref_from_convert_row_type() and code that calls it, and
per node type cases in reparameterize_path_by_child(). It seems like
the former could use a test case, and I wonder if there is some way we
could write "flat-copy and then apply recursively to all subpaths"
code like this without having to handle these cases explicitly. There
are a couple of other tiny return cases other than just sanity check
errors which it might be interesting to hit too.

Under the debugger I checked that the test in partition_join.sql
-- 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;
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;
covers get_wholerow_ref_from_convert_row_type(). But it doesn't cover
a couple of lines in the case of nested ConvertRowTypeExpr in that
function. We can add/modify a testcase in multi-level partitioned
table section to cover that.

reparameterize_path_by_child() coverage is hard. It would require that
many different kinds of paths survive in lower joins in the join tree.
It's hard to come up with queries that would do that with limited
amount of data and a reasonable number of tests. Me and Thomas
discussed about his suggestion about "flat-copy and then apply
recursively to all subpaths" which he sees as a path tree mutator. It
won't improve the test coverage. Like expression_tree_mutator() path
mutation is not that widely used phenomenon, so we do not yet know
what should be the characteristics of a path mutator could be. In case
we see more of path mutation code in future, it's an idea worth
considering.

2. What queries in the 0008 patch are hitting lines that 0007 doesn't hit?

I thought about how to answer questions like this and came up with a
shell script that (1) makes computers run really hot for quite a long
time and (2) tells you which blocks of SQL hit which lines of C.
Please find attached the shell script and its output. The .sql files
have been annotated with "block" numbers (blocks being chunks of SQL
stuff separated by blank lines), and the C files annotated with
references to those block numbers where A<n> = block <n>
partition_join.sql and B<n> = block <n> in partition_join_extras.sql.

Then to find lines that B queries hit but A queries don't and know
which particular queries hit them, you might use something like:

grep -v 'SQL blocks: .*A[0-9]' < joinpath.c.aggregated_coverage | \
grep 'SQL blocks: .*B[0-9]'

Thanks for this. It generates a lot of output (970 lines over all the
coverage files). It will take some time for getting anything
meaningful out of this. May be there's some faster way by looking at
the lines that are covered by B but not A. BTW, I checked those lines
to see if there could be any bug there. But I don't see what could go
wrong with those lines.

--
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

#259Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Ashutosh Bapat (#254)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Sep 19, 2017 at 3:17 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

- I'm not entirely sure whether maintaining partexprs and
nullable_partexprs is the right design. If I understand correctly,
whether or not a partexpr is nullable is really a per-RTI property,
not a per-expression property. You could consider something like
"Relids nullable_rels".

That's true. However in order to decide whether an expression falls on
nullable side of a join, we will need to call pull_varnos() on it and
check the output against nullable_rels. Separating the expressions
themselves avoids that step.

Good point. Also, I'm not sure about cases like this:

SELECT * FROM (SELECT b.x, b.y FROM a LEFT JOIN b ON a.x = b.x WHERE
a.y = b.y) w LEFT JOIN c ON w.x = c.x AND w.y = c.y;

Suppose the relations are all partitioned by (x, y) but that the =
operator is not strict. A partition-wise join is valid between a and
b, but we can't regard w as partitioned any more, because w.x might
contain nulls in partitions where the partitioning scheme wouldn't
allow them. On the other hand, if the subquery were to select a.x,
a.y then clearly it would be fine: there would be no possibility of a
NULL having been substituted for a proper value.

What if the subquery selected a.x, b.y? Initially, I thought that
would be OK too, because of the fact that the a.y = b.y clause is in
the WHERE clause rather than the join condition. But on further
thought I think that probably doesn't work, because with = being a
non-strict operator there's no guarantee that it would remove any
nulls introduced by the left join. Of course, if the subselect had a
WHERE clause saying that b.x/b.y IS NOT NULL then having the SELECT
list mention those columns would be fine.

In my previous reply to this, I probably didn't answer your question
while I explained the restriction on where equality conditions on
partition keys can appear. Here's answer to your questions assuming
those restrictions don't exist. Actually in the example you have
given, optimizer flattens w as a LJ b which kind of makes the
explanations below a bit complicated.

1. SELECT * FROM (SELECT b.x, b.y FROM a LEFT JOIN b ON a.x = b.x
WHERE a.y = b.y) w LEFT JOIN c ON w.x = c.x AND w.y = c.y;
partition-wise join will be possible between a and b but not between w
and c for the reasons you have explained above.
2. SELECT * FROM (SELECT a.x, a.y FROM a LEFT JOIN b ON a.x = b.x
WHERE a.y = b.y) w LEFT JOIN c ON w.x = c.x AND w.y = c.y;
partition-wise join will be possible between a and b and also between
w and c for the reasons you have explained above.
3. SELECT * FROM (SELECT a.x, b.y FROM a LEFT JOIN b ON a.x = b.x
WHERE a.y = b.y) w LEFT JOIN c ON w.x = c.x AND w.y = c.y;
partition-wise join will be possible between a and b but not w and c
as you have explained.

In this case b.x and b.y will appear as nullable_partexprs in w
(represented as a LJ b in optimizer) and a.x and a.y will appear in
partexprs. Depending upon what gets projected out of w, the join
between w and c will use corresponding keys for equality conditions.
Since the operator is non-strict, any expression which is part of
nullable_partexprs will be discarded in
match_expr_to_partition_keys().

Hope that helps.

--
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

#260Jeevan Chalke
jeevan.chalke@enterprisedb.com
In reply to: Ashutosh Bapat (#254)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Sep 19, 2017 at 3:17 PM, Ashutosh Bapat <
ashutosh.bapat@enterprisedb.com> wrote:

On Tue, Sep 19, 2017 at 2:35 AM, Robert Haas <robertmhaas@gmail.com>
wrote:

On Mon, Sep 18, 2017 at 8:02 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

partition pruning might need partexprs look up relevant quals, but
nullable_partexprs doesn't have any use there. So may be we should add
nullable_partexpr to RelOptInfo as part of 0002 (partition-wise join
implementation) instead of 0001. What do you think?

+1.

Done.

- I'm not entirely sure whether maintaining partexprs and
nullable_partexprs is the right design. If I understand correctly,
whether or not a partexpr is nullable is really a per-RTI property,
not a per-expression property. You could consider something like
"Relids nullable_rels".

That's true. However in order to decide whether an expression falls on
nullable side of a join, we will need to call pull_varnos() on it and
check the output against nullable_rels. Separating the expressions
themselves avoids that step.

Good point. Also, I'm not sure about cases like this:

SELECT * FROM (SELECT b.x, b.y FROM a LEFT JOIN b ON a.x = b.x WHERE
a.y = b.y) w LEFT JOIN c ON w.x = c.x AND w.y = c.y;

Suppose the relations are all partitioned by (x, y) but that the =
operator is not strict. A partition-wise join is valid between a and
b, but we can't regard w as partitioned any more, because w.x might
contain nulls in partitions where the partitioning scheme wouldn't
allow them. On the other hand, if the subquery were to select a.x,
a.y then clearly it would be fine: there would be no possibility of a
NULL having been substituted for a proper value.

What if the subquery selected a.x, b.y? Initially, I thought that
would be OK too, because of the fact that the a.y = b.y clause is in
the WHERE clause rather than the join condition. But on further
thought I think that probably doesn't work, because with = being a
non-strict operator there's no guarantee that it would remove any
nulls introduced by the left join. Of course, if the subselect had a
WHERE clause saying that b.x/b.y IS NOT NULL then having the SELECT
list mention those columns would be fine.

I am actually not sure whether we can use partition-wise join for a
LEFT JOIN b when the partition key equalities are spread across ON and
WHERE clauses. I am not able to find any example against it, but I am
not able to prove it as well. The reference I used for partition-wise
join [1], mentions JOIN conditions i.e. ON clause conditions. But all
the examples used in that paper are that of INNER join. So, I am not
sure what exactly the authors meant by JOIN conditions. Right now I am
restricting the patch to work with only conditions in the ON clause.

Practically most of the operators are strict. OUTER join's WHERE
clause has any partition key equality with strict operator, optimizer
will turn
that OUTER join into an INNER one, turning all clauses into join
clauses. That will enable partition-wise join. So, the current
restriction doesn't restrict any practical cases.

OTOH, I have seen that treating ON and WHERE clauses as same for an
OUTER join leads to surprising results. So, I am leaning to treat them
separate for partition-wise join as well and only use ON clause
conditions for partition-wise join. If we get complaints about
partition-wise join not being picked we will fix them after proving
that it's not harmful. Lifting that restriction is not so difficult.
have_partition_key_equijoin() ignores "pushed down" quals. We have to
just change that condition.

Your last sentence about a clause b.x IS NOT NULL or b.y IS NOT NULL
is interesting. If those conditions are in ON clause, we may still
have a result where b.x and b.y as NULL when no row in "a" matches a
row in "b". If those conditions are in WHERE clause, I think optimizer
will turn the join into an INNER join irrespective of whether the
equality operator is strict.

If partition-wise join is disabled, partition-wise aggregates,
strength reduction of MergeAppend won't be possible on a join tree,
but those will be possible on a base relation. Even if partition-wise
join enabled, one may want to disable other partition-wise
optimizations individually. So, they are somewhat independent
switches. I don't think we should bundle all of those into one.
Whatever names we choose for those GUCs, I think they should have same
naming convention e.g. "partition_wise_xyz". I am open to suggestions
about the names.

I think the chances of you getting multiple GUCs for different
partition-wise optimizations past Tom are pretty low.

We do have enable_hashjoin and enable_hashagg to control use of
hashing for aggregate and join. On similar lines we can have three
GUCs to enable use of partition-wise strategy, one for each of join,
aggregation and sorting. Having granular switches would be useful for
debugging and may be to turn partition-wise strategies off when they
are not optimal.

I think having a granular control over each of these optimization will be
handy for the DBAs too.

Do we want a switch to turn ON/OFF partition pruning?
Said, that I am fine with single GUC controlling all. We won't set any
partitioning information in RelOptInfo if that GUC is turned OFF.

[1] https://pdfs.semanticscholar.org/27c2/ba75f8b6a39d4bce85d5579dace609
c9abaa.pdf
--
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

--
Jeevan Chalke
Principal Software Engineer, Product Development
EnterpriseDB Corporation
The Enterprise PostgreSQL Company

#261Rajkumar Raghuwanshi
rajkumar.raghuwanshi@enterprisedb.com
In reply to: Ashutosh Bapat (#258)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Sep 20, 2017 at 3:13 PM, Ashutosh Bapat <
ashutosh.bapat@enterprisedb.com> wrote:

On Wed, Sep 20, 2017 at 9:44 AM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

2. What queries in the 0008 patch are hitting lines that 0007 doesn't

hit?

I thought about how to answer questions like this and came up with a
shell script that (1) makes computers run really hot for quite a long
time and (2) tells you which blocks of SQL hit which lines of C.
Please find attached the shell script and its output. The .sql files
have been annotated with "block" numbers (blocks being chunks of SQL
stuff separated by blank lines), and the C files annotated with
references to those block numbers where A<n> = block <n>
partition_join.sql and B<n> = block <n> in partition_join_extras.sql.

Then to find lines that B queries hit but A queries don't and know
which particular queries hit them, you might use something like:

grep -v 'SQL blocks: .*A[0-9]' < joinpath.c.aggregated_coverage | \
grep 'SQL blocks: .*B[0-9]'

Thanks for this. It generates a lot of output (970 lines over all the
coverage files). It will take some time for getting anything
meaningful out of this. May be there's some faster way by looking at
the lines that are covered by B but not A. BTW, I checked those lines
to see if there could be any bug there. But I don't see what could go
wrong with those lines.

I have also tried to find test cases in B which hits some extra line which

is not
hitting by A, with the help of results attached by Thomas in
coverage.tarball_FILES.
It took lot of time but I am able to find some test cases. which if adding
in partition_join.sql
increasing no of lines hit by 14. but for hitting these 14 extra line
attached patch is doing
900+ line inserts in partition_join.sql and partition_join.out file.

I have used gcov-lcov to find coverage for files changed by
partition-wise-join patches
with and without attached patch which is below.

*with existing partition_join.sql* *partition_join.sql + some test cases of
partition_join_extra.sql*
*Modifed Files* *Line Coverage* *Functions* *Line Coverage* *Functions*
src/backend/optimizer/geqo 79.4 % 269/339 96.6 % 28/29 79.4 % 269/339 96.6 %
28/29
src/backend/optimizer/path/allpaths.c 92.3 % 787 / 853 95.5 % 42 / 44
92.6 % 790
/ 853 95.5 % 42 / 44
src/backend/optimizer/path/costsize.c 96.8 % 1415 / 1462 98.4 % 61 / 62
96.9 % 1416 / 1462 98.4 % 61 / 62
src/backend/optimizer/path/joinpath.c 95.5 % 404 / 423 100.0 % 16 / 16
95.5 % 404 / 423 100.0 % 16 / 16
src/backend/optimizer/path/joinrels.c 92.5 % 422 / 456 100.0 % 16 / 16
93.0 % 424 / 456 100.0 % 16 / 16
src/backend/optimizer/plan/createplan.c 90.9 % 1928 / 2122 96.3 % 103 / 107
91.0 % 1930 / 2122 96.3 % 103 / 107
src/backend/optimizer/plan/planner.c 94.9 % 1609 / 1696 97.6 % 41 / 42
94.9 % 1609 / 1696 97.6 % 41 / 42
src/backend/optimizer/plan/setrefs.c 91.3 % 806 / 883 94.3 % 33 / 35 91.3 % 806
/ 883 94.3 % 33 / 35
src/backend/optimizer/prep/prepunion.c 95.5 % 661 / 692 100.0 % 25 / 25
95.5 % 661 / 692 100.0 % 25 / 25
src/backend/optimizer/util/pathnode.c 88.7 % 1144 / 1290 98.1 % 52 / 53
88.8 % 1146 / 1290 98.1 % 52 / 53
src/backend/optimizer/util/placeholder.c 96.5 % 139 / 144 100.0 % 10 / 10
96.5 % 139 / 144 100.0 % 10 / 10
src/backend/optimizer/util/plancat.c 89.0 % 540 / 607 94.7 % 18 / 19 89.6 % 544
/ 607 94.7 % 18 / 19
src/backend/optimizer/util/relnode.c 95.3 % 548 / 575 100.0 % 24 / 24
95.3 % 548
/ 575 100.0 % 24 / 24
src/backend/utils/misc/guc.c 67.4 % 1536 / 2278 89.7 % 113 / 126 67.4 % 1536
/ 2278 89.7 % 113 / 126

Thanks & Regards,
Rajkumar Raghuwanshi
QMG, EnterpriseDB Corporation

Attachments:

partition_join_with_some_testcases_from_extra.patchtext/x-patch; charset=US-ASCII; name=partition_join_with_some_testcases_from_extra.patchDownload
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index 9fec170..ab411b6 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -584,6 +584,215 @@ SELECT t1.a, ss.t2a, ss.t2c FROM prt1 t1 LEFT JOIN LATERAL
  550 |     | 
 (12 rows)
 
+-- join with aggregate
+EXPLAIN (VERBOSE, COSTS OFF)
+select t1.a, count(t2.*) from prt1 t1 left join prt1 t2 on (t1.a = t2.a) where t1.a % 25 = 0 group by t1.a;
+                          QUERY PLAN                           
+---------------------------------------------------------------
+ GroupAggregate
+   Output: t1.a, count(((t2.*)::prt1))
+   Group Key: t1.a
+   ->  Sort
+         Output: t1.a, ((t2.*)::prt1)
+         Sort Key: t1.a
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, ((t2.*)::prt1)
+                     Hash Cond: (t2.a = t1.a)
+                     ->  Seq Scan on public.prt1_p1 t2
+                           Output: t2.*, t2.a
+                     ->  Hash
+                           Output: t1.a
+                           ->  Seq Scan on public.prt1_p1 t1
+                                 Output: t1.a
+                                 Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, ((t2_1.*)::prt1)
+                     Hash Cond: (t2_1.a = t1_1.a)
+                     ->  Seq Scan on public.prt1_p2 t2_1
+                           Output: t2_1.*, t2_1.a
+                     ->  Hash
+                           Output: t1_1.a
+                           ->  Seq Scan on public.prt1_p2 t1_1
+                                 Output: t1_1.a
+                                 Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, ((t2_2.*)::prt1)
+                     Hash Cond: (t2_2.a = t1_2.a)
+                     ->  Seq Scan on public.prt1_p3 t2_2
+                           Output: t2_2.*, t2_2.a
+                     ->  Hash
+                           Output: t1_2.a
+                           ->  Seq Scan on public.prt1_p3 t1_2
+                                 Output: t1_2.a
+                                 Filter: ((t1_2.a % 25) = 0)
+(37 rows)
+
+select t1.a, count(t2.*) from prt1 t1 left join prt1 t2 on (t1.a = t2.a) where t1.a % 25 = 0 group by t1.a;
+  a  | count 
+-----+-------
+   0 |     1
+  50 |     1
+ 100 |     1
+ 150 |     1
+ 200 |     1
+ 250 |     1
+ 300 |     1
+ 350 |     1
+ 400 |     1
+ 450 |     1
+ 500 |     1
+ 550 |     1
+(12 rows)
+
+-- lateral with VALUES
+EXPLAIN (VERBOSE, COSTS OFF)
+select count(*) from prt1 a, prt2 b join lateral (values(a.a)) ss(x) on b.b = ss.x;
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Aggregate
+   Output: count(*)
+   ->  Append
+         ->  Hash Join
+               Hash Cond: (a.a = b.b)
+               ->  Seq Scan on public.prt1_p1 a
+                     Output: a.a
+               ->  Hash
+                     Output: b.b
+                     ->  Seq Scan on public.prt2_p1 b
+                           Output: b.b
+         ->  Hash Join
+               Hash Cond: (a_1.a = b_1.b)
+               ->  Seq Scan on public.prt1_p2 a_1
+                     Output: a_1.a
+               ->  Hash
+                     Output: b_1.b
+                     ->  Seq Scan on public.prt2_p2 b_1
+                           Output: b_1.b
+         ->  Hash Join
+               Hash Cond: (a_2.a = b_2.b)
+               ->  Seq Scan on public.prt1_p3 a_2
+                     Output: a_2.a
+               ->  Hash
+                     Output: b_2.b
+                     ->  Seq Scan on public.prt2_p3 b_2
+                           Output: b_2.b
+(27 rows)
+
+select count(*) from prt1 a, prt2 b join lateral (values(a.a)) ss(x) on b.b = ss.x;
+ count 
+-------
+   100
+(1 row)
+
+-- join with rank
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.b, t2.b,rank() OVER (ORDER BY t2.b DESC) FROM prt1 t1 inner join prt2 t2 on (t1.a = t2.b) where t1.a % 25 = 0 order by 1,2,3;
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.b, t2.b, (rank() OVER (?))
+   Sort Key: t1.a, t1.b
+   ->  WindowAgg
+         Output: t1.a, t1.b, t2.b, rank() OVER (?)
+         ->  Sort
+               Output: t2.b, t1.a, t1.b
+               Sort Key: t2.b DESC
+               ->  Result
+                     Output: t2.b, t1.a, t1.b
+                     ->  Append
+                           ->  Hash Join
+                                 Output: t1.a, t1.b, t2.b
+                                 Hash Cond: (t2.b = t1.a)
+                                 ->  Seq Scan on public.prt2_p1 t2
+                                       Output: t2.b
+                                 ->  Hash
+                                       Output: t1.a, t1.b
+                                       ->  Seq Scan on public.prt1_p1 t1
+                                             Output: t1.a, t1.b
+                                             Filter: ((t1.a % 25) = 0)
+                           ->  Hash Join
+                                 Output: t1_1.a, t1_1.b, t2_1.b
+                                 Hash Cond: (t2_1.b = t1_1.a)
+                                 ->  Seq Scan on public.prt2_p2 t2_1
+                                       Output: t2_1.b
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.b
+                                       ->  Seq Scan on public.prt1_p2 t1_1
+                                             Output: t1_1.a, t1_1.b
+                                             Filter: ((t1_1.a % 25) = 0)
+                           ->  Hash Join
+                                 Output: t1_2.a, t1_2.b, t2_2.b
+                                 Hash Cond: (t2_2.b = t1_2.a)
+                                 ->  Seq Scan on public.prt2_p3 t2_2
+                                       Output: t2_2.b
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.b
+                                       ->  Seq Scan on public.prt1_p3 t1_2
+                                             Output: t1_2.a, t1_2.b
+                                             Filter: ((t1_2.a % 25) = 0)
+(41 rows)
+
+SELECT t1.a, t1.b, t2.b,rank() OVER (ORDER BY t2.b DESC) FROM prt1 t1 inner join prt2 t2 on (t1.a = t2.b) where t1.a % 25 = 0 order by 1,2,3;
+  a  | b |  b  | rank 
+-----+---+-----+------
+   0 | 0 |   0 |    4
+ 150 | 0 | 150 |    3
+ 300 | 0 | 300 |    2
+ 450 | 0 | 450 |    1
+(4 rows)
+
+--join with prepare statement
+PREPARE ij(int) AS select t1.a,t2.b from prt1 t1 inner join prt2 t2 on (t1.a = t2.b and t1.a % $1 = 0) ORDER BY 1,2;
+EXPLAIN (VERBOSE, COSTS OFF) EXECUTE ij(25);
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Sort
+   Output: t1.a, t2.b
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t2.b
+               Hash Cond: (t2.b = t1.a)
+               ->  Seq Scan on public.prt2_p1 t2
+                     Output: t2.b
+               ->  Hash
+                     Output: t1.a
+                     ->  Seq Scan on public.prt1_p1 t1
+                           Output: t1.a
+                           Filter: ((t1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_1.a, t2_1.b
+               Hash Cond: (t2_1.b = t1_1.a)
+               ->  Seq Scan on public.prt2_p2 t2_1
+                     Output: t2_1.b
+               ->  Hash
+                     Output: t1_1.a
+                     ->  Seq Scan on public.prt1_p2 t1_1
+                           Output: t1_1.a
+                           Filter: ((t1_1.a % 25) = 0)
+         ->  Hash Join
+               Output: t1_2.a, t2_2.b
+               Hash Cond: (t2_2.b = t1_2.a)
+               ->  Seq Scan on public.prt2_p3 t2_2
+                     Output: t2_2.b
+               ->  Hash
+                     Output: t1_2.a
+                     ->  Seq Scan on public.prt1_p3 t1_2
+                           Output: t1_2.a
+                           Filter: ((t1_2.a % 25) = 0)
+(34 rows)
+
+EXECUTE ij(25);
+  a  |  b  
+-----+-----
+   0 |   0
+ 150 | 150
+ 300 | 300
+ 450 | 450
+(4 rows)
+
+DEALLOCATE ij;
 --
 -- partitioned by expression
 --
@@ -1094,7 +1303,8 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
                            ->  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;
+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
@@ -1111,6 +1321,234 @@ SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 LEFT JOIN prt2 t2
  550 | 0550 |     |      |     1100 | 0
 (12 rows)
 
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
+ Merge Append
+   Sort Key: t1.a
+   ->  Merge Semi Join
+         Output: t1.a, t1.b, t1.c
+         Merge Cond: (t1.a = t1_3.b)
+         ->  Sort
+               Output: t1.a, t1.b, t1.c
+               Sort Key: t1.a
+               ->  Seq Scan on public.prt1_p1 t1
+                     Output: t1.a, t1.b, t1.c
+                     Filter: ((t1.a % 25) = 0)
+         ->  Merge Semi Join
+               Output: t1_3.b, t1_6.a, t1_6.b
+               Merge Cond: (t1_3.b = (((t1_6.a + t1_6.b) / 2)))
+               ->  Sort
+                     Output: t1_3.b
+                     Sort Key: t1_3.b
+                     ->  Seq Scan on public.prt2_p1 t1_3
+                           Output: t1_3.b
+               ->  Sort
+                     Output: t1_6.a, t1_6.b, (((t1_6.a + t1_6.b) / 2))
+                     Sort Key: (((t1_6.a + t1_6.b) / 2))
+                     ->  Seq Scan on public.prt1_e_p1 t1_6
+                           Output: t1_6.a, t1_6.b, ((t1_6.a + t1_6.b) / 2)
+                           Filter: ((t1_6.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_1.a, t1_1.b, t1_1.c
+         Merge Cond: (t1_1.a = t1_4.b)
+         ->  Sort
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Sort Key: t1_1.a
+               ->  Seq Scan on public.prt1_p2 t1_1
+                     Output: t1_1.a, t1_1.b, t1_1.c
+                     Filter: ((t1_1.a % 25) = 0)
+         ->  Merge Semi Join
+               Output: t1_4.b, t1_7.a, t1_7.b
+               Merge Cond: (t1_4.b = (((t1_7.a + t1_7.b) / 2)))
+               ->  Sort
+                     Output: t1_4.b
+                     Sort Key: t1_4.b
+                     ->  Seq Scan on public.prt2_p2 t1_4
+                           Output: t1_4.b
+               ->  Sort
+                     Output: t1_7.a, t1_7.b, (((t1_7.a + t1_7.b) / 2))
+                     Sort Key: (((t1_7.a + t1_7.b) / 2))
+                     ->  Seq Scan on public.prt1_e_p2 t1_7
+                           Output: t1_7.a, t1_7.b, ((t1_7.a + t1_7.b) / 2)
+                           Filter: ((t1_7.a % 25) = 0)
+   ->  Merge Semi Join
+         Output: t1_2.a, t1_2.b, t1_2.c
+         Merge Cond: (t1_2.a = t1_5.b)
+         ->  Sort
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Sort Key: t1_2.a
+               ->  Seq Scan on public.prt1_p3 t1_2
+                     Output: t1_2.a, t1_2.b, t1_2.c
+                     Filter: ((t1_2.a % 25) = 0)
+         ->  Merge Semi Join
+               Output: t1_5.b, t1_8.a, t1_8.b
+               Merge Cond: (t1_5.b = (((t1_8.a + t1_8.b) / 2)))
+               ->  Sort
+                     Output: t1_5.b
+                     Sort Key: t1_5.b
+                     ->  Seq Scan on public.prt2_p3 t1_5
+                           Output: t1_5.b
+               ->  Sort
+                     Output: t1_8.a, t1_8.b, (((t1_8.a + t1_8.b) / 2))
+                     Sort Key: (((t1_8.a + t1_8.b) / 2))
+                     ->  Seq Scan on public.prt1_e_p3 t1_8
+                           Output: t1_8.a, t1_8.b, ((t1_8.a + t1_8.b) / 2)
+                           Filter: ((t1_8.a % 25) = 0)
+(71 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+  a  | b |  c   
+-----+---+------
+   0 | 0 | 0000
+ 150 | 0 | 0150
+ 300 | 0 | 0300
+ 450 | 0 | 0450
+(4 rows)
+
+SET enable_seqscan TO off;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                          QUERY PLAN                                          
+----------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Merge Left Join
+                     Output: t3.a, t3.b, t3.c, t2.b, t2.c, t1.a, t1.c
+                     Merge Cond: (t2.b = t1.a)
+                     ->  Sort
+                           Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                           Sort Key: t2.b
+                           ->  Merge Left Join
+                                 Output: t3.a, t3.b, t3.c, t2.b, t2.c
+                                 Merge Cond: (((t3.a + t3.b) / 2) = t2.b)
+                                 ->  Index Scan using iprt1_e_p1_ab2 on public.prt1_e_p1 t3
+                                       Output: t3.a, t3.b, t3.c
+                                       Filter: ((t3.a % 25) = 0)
+                                 ->  Index Scan using iprt2_p1_b on public.prt2_p1 t2
+                                       Output: t2.b, t2.c
+                     ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1
+                           Output: t1.a, t1.c
+               ->  Merge Left Join
+                     Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c, t1_1.a, t1_1.c
+                     Merge Cond: (t2_1.b = t1_1.a)
+                     ->  Sort
+                           Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                           Sort Key: t2_1.b
+                           ->  Merge Left Join
+                                 Output: t3_1.a, t3_1.b, t3_1.c, t2_1.b, t2_1.c
+                                 Merge Cond: (((t3_1.a + t3_1.b) / 2) = t2_1.b)
+                                 ->  Index Scan using iprt1_e_p2_ab2 on public.prt1_e_p2 t3_1
+                                       Output: t3_1.a, t3_1.b, t3_1.c
+                                       Filter: ((t3_1.a % 25) = 0)
+                                 ->  Index Scan using iprt2_p2_b on public.prt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                     ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_1
+                           Output: t1_1.a, t1_1.c
+               ->  Merge Right Join
+                     Output: t3_2.a, t3_2.b, t3_2.c, t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                     Merge Cond: (t2_2.b = ((t3_2.a + t3_2.b) / 2))
+                     ->  Merge Left Join
+                           Output: t2_2.b, t2_2.c, t1_2.a, t1_2.c
+                           Merge Cond: (t2_2.b = t1_2.a)
+                           ->  Index Scan using iprt2_p3_b on public.prt2_p3 t2_2
+                                 Output: t2_2.b, t2_2.c
+                           ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_2
+                                 Output: t1_2.a, t1_2.c
+                     ->  Index Scan using iprt1_e_p3_ab2 on public.prt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                           Filter: ((t3_2.a % 25) = 0)
+(51 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.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
+     |      |     |      |      100 | 0
+     |      |     |      |      200 | 0
+     |      |     |      |      400 | 0
+     |      |     |      |      500 | 0
+     |      |     |      |      700 | 0
+     |      |     |      |      800 | 0
+     |      |     |      |     1000 | 0
+     |      |     |      |     1100 | 0
+(12 rows)
+
+-- lateral references and parameterized paths
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+(SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+                                QUERY PLAN                                 
+---------------------------------------------------------------------------
+ Nested Loop Left Join
+   Output: t1.a, t1.b, t1.c, t2.a, t3.a, (LEAST(t1.a, t2.a, t3.a)), t1.a
+   ->  Merge Append
+         Sort Key: t1.a
+         ->  Index Scan using iprt1_p1_a on public.prt1_p1 t1
+               Output: t1.a, t1.b, t1.c
+               Filter: ((t1.a % 25) = 0)
+         ->  Index Scan using iprt1_p2_a on public.prt1_p2 t1_1
+               Output: t1_1.a, t1_1.b, t1_1.c
+               Filter: ((t1_1.a % 25) = 0)
+         ->  Index Scan using iprt1_p3_a on public.prt1_p3 t1_2
+               Output: t1_2.a, t1_2.b, t1_2.c
+               Filter: ((t1_2.a % 25) = 0)
+   ->  Append
+         ->  Merge Join
+               Output: t2.a, t3.a, LEAST(t1.a, t2.a, t3.a)
+               Merge Cond: (t2.a = t3.b)
+               ->  Index Only Scan using iprt1_p1_a on public.prt1_p1 t2
+                     Output: t2.a
+                     Index Cond: (t2.a = t1.a)
+               ->  Index Scan using iprt2_p1_b on public.prt2_p1 t3
+                     Output: t3.a, t3.b
+         ->  Merge Join
+               Output: t2_1.a, t3_1.a, LEAST(t1.a, t2_1.a, t3_1.a)
+               Merge Cond: (t2_1.a = t3_1.b)
+               ->  Index Only Scan using iprt1_p2_a on public.prt1_p2 t2_1
+                     Output: t2_1.a
+                     Index Cond: (t2_1.a = t1.a)
+               ->  Index Scan using iprt2_p2_b on public.prt2_p2 t3_1
+                     Output: t3_1.a, t3_1.b
+         ->  Merge Join
+               Output: t2_2.a, t3_2.a, LEAST(t1.a, t2_2.a, t3_2.a)
+               Merge Cond: (t2_2.a = t3_2.b)
+               ->  Index Only Scan using iprt1_p3_a on public.prt1_p3 t2_2
+                     Output: t2_2.a
+                     Index Cond: (t2_2.a = t1.a)
+               ->  Index Scan using iprt2_p3_b on public.prt2_p3 t3_2
+                     Output: t3_2.a, t3_2.b
+(38 rows)
+
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+(SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+ON t1.a = ss.t2a WHERE t1.a % 25 = 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 |     0
+ 200 | 0 | 0200 |     |     |      
+ 250 | 0 | 0250 |     |     |      
+ 300 | 0 | 0300 | 300 |   0 |     0
+ 350 | 0 | 0350 |     |     |      
+ 400 | 0 | 0400 |     |     |      
+ 450 | 0 | 0450 | 450 |   0 |     0
+ 500 | 0 | 0500 |     |     |      
+ 550 | 0 | 0550 |     |     |      
+(12 rows)
+
+RESET enable_seqscan;
 -- 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;
@@ -1238,6 +1676,187 @@ CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005', '0002', '0
 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;
+--full join using
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM plt1 t1 FULL JOIN plt2 t2 USING (c) FULL JOIN plt1 t3 USING (c)
+where t1.a %150 =0 and t2.b % 150 = 0 and t3.a % 150 = 0 order by 1,2,3;
+                                        QUERY PLAN                                        
+------------------------------------------------------------------------------------------
+ Sort
+   Output: (COALESCE(COALESCE(t1.c, t2.c), t3.c)), t1.a, t1.b, t2.a, t2.b, t3.a, t3.b
+   Sort Key: (COALESCE(COALESCE(t1.c, t2.c), t3.c)), t1.a, t1.b
+   ->  Nested Loop
+         Output: COALESCE(COALESCE(t1.c, t2.c), t3.c), t1.a, t1.b, t2.a, t2.b, t3.a, t3.b
+         Join Filter: (COALESCE(t1.c, t2.c) = t3.c)
+         ->  Nested Loop
+               Output: t1.c, t1.a, t1.b, t2.c, t2.a, t2.b
+               Join Filter: (t1.c = t2.c)
+               ->  Append
+                     ->  Seq Scan on public.plt1_p1 t1
+                           Output: t1.c, t1.a, t1.b
+                           Filter: ((t1.a % 150) = 0)
+                     ->  Seq Scan on public.plt1_p2 t1_1
+                           Output: t1_1.c, t1_1.a, t1_1.b
+                           Filter: ((t1_1.a % 150) = 0)
+                     ->  Seq Scan on public.plt1_p3 t1_2
+                           Output: t1_2.c, t1_2.a, t1_2.b
+                           Filter: ((t1_2.a % 150) = 0)
+               ->  Materialize
+                     Output: t2.c, t2.a, t2.b
+                     ->  Append
+                           ->  Seq Scan on public.plt2_p1 t2
+                                 Output: t2.c, t2.a, t2.b
+                                 Filter: ((t2.b % 150) = 0)
+                           ->  Seq Scan on public.plt2_p2 t2_1
+                                 Output: t2_1.c, t2_1.a, t2_1.b
+                                 Filter: ((t2_1.b % 150) = 0)
+                           ->  Seq Scan on public.plt2_p3 t2_2
+                                 Output: t2_2.c, t2_2.a, t2_2.b
+                                 Filter: ((t2_2.b % 150) = 0)
+         ->  Append
+               ->  Seq Scan on public.plt1_p1 t3
+                     Output: t3.c, t3.a, t3.b
+                     Filter: ((t3.a % 150) = 0)
+               ->  Seq Scan on public.plt1_p2 t3_1
+                     Output: t3_1.c, t3_1.a, t3_1.b
+                     Filter: ((t3_1.a % 150) = 0)
+               ->  Seq Scan on public.plt1_p3 t3_2
+                     Output: t3_2.c, t3_2.a, t3_2.b
+                     Filter: ((t3_2.a % 150) = 0)
+(41 rows)
+
+SELECT * FROM plt1 t1 FULL JOIN plt2 t2 USING (c) FULL JOIN plt1 t3 USING (c)
+where t1.a %150 =0 and t2.b % 150 = 0 and t3.a % 150 = 0 order by 1,2,3;
+  c   |  a  |  b  |  a  |  b  |  a  |  b  
+------+-----+-----+-----+-----+-----+-----
+ 0000 |   0 |   0 |   0 |   0 |   0 |   0
+ 0003 | 150 | 150 | 150 | 150 | 150 | 150
+ 0006 | 300 | 300 | 300 | 300 | 300 | 300
+ 0009 | 450 | 450 | 450 | 450 | 450 | 450
+(4 rows)
+
+-- test placement of movable quals in a parameterized join tree
+EXPLAIN (VERBOSE, COSTS OFF)
+select b.b from prt1 a join prt2 b on a.a = b.b left join plt1 c on b.a % 25 = 0 and c.c = a.c join prt1 i1 on b.b = i1.a
+right join prt2 i2 on i2.b = b.b where b.a % 25 = 0 order by 1;
+                                      QUERY PLAN                                       
+---------------------------------------------------------------------------------------
+ Sort
+   Output: b.b
+   Sort Key: b.b
+   ->  Hash Right Join
+         Output: b.b
+         Hash Cond: (c.c = (a.c)::text)
+         Join Filter: ((b.a % 25) = 0)
+         ->  Append
+               ->  Seq Scan on public.plt1_p1 c
+                     Output: c.c
+               ->  Seq Scan on public.plt1_p2 c_1
+                     Output: c_1.c
+               ->  Seq Scan on public.plt1_p3 c_2
+                     Output: c_2.c
+         ->  Hash
+               Output: a.c, b.b, b.a
+               ->  Append
+                     ->  Nested Loop
+                           Output: a.c, b.b, b.a
+                           ->  Nested Loop
+                                 Output: a.a, a.c, b.b, b.a, i2.b
+                                 Join Filter: (b.b = a.a)
+                                 ->  Hash Join
+                                       Output: b.b, b.a, i2.b
+                                       Hash Cond: (i2.b = b.b)
+                                       ->  Seq Scan on public.prt2_p1 i2
+                                             Output: i2.b
+                                       ->  Hash
+                                             Output: b.b, b.a
+                                             ->  Seq Scan on public.prt2_p1 b
+                                                   Output: b.b, b.a
+                                                   Filter: ((b.a % 25) = 0)
+                                 ->  Index Scan using iprt1_p1_a on public.prt1_p1 a
+                                       Output: a.a, a.c
+                                       Index Cond: (a.a = i2.b)
+                           ->  Index Only Scan using iprt1_p1_a on public.prt1_p1 i1
+                                 Output: i1.a
+                                 Index Cond: (i1.a = a.a)
+                     ->  Nested Loop
+                           Output: a_1.c, b_1.b, b_1.a
+                           ->  Nested Loop
+                                 Output: a_1.a, a_1.c, b_1.b, b_1.a, i2_1.b
+                                 Join Filter: (b_1.b = a_1.a)
+                                 ->  Hash Join
+                                       Output: b_1.b, b_1.a, i2_1.b
+                                       Hash Cond: (i2_1.b = b_1.b)
+                                       ->  Seq Scan on public.prt2_p2 i2_1
+                                             Output: i2_1.b
+                                       ->  Hash
+                                             Output: b_1.b, b_1.a
+                                             ->  Seq Scan on public.prt2_p2 b_1
+                                                   Output: b_1.b, b_1.a
+                                                   Filter: ((b_1.a % 25) = 0)
+                                 ->  Index Scan using iprt1_p2_a on public.prt1_p2 a_1
+                                       Output: a_1.a, a_1.c
+                                       Index Cond: (a_1.a = i2_1.b)
+                           ->  Index Only Scan using iprt1_p2_a on public.prt1_p2 i1_1
+                                 Output: i1_1.a
+                                 Index Cond: (i1_1.a = a_1.a)
+                     ->  Nested Loop
+                           Output: a_2.c, b_2.b, b_2.a
+                           ->  Nested Loop
+                                 Output: a_2.a, a_2.c, b_2.b, b_2.a, i2_2.b
+                                 Join Filter: (b_2.b = a_2.a)
+                                 ->  Hash Join
+                                       Output: b_2.b, b_2.a, i2_2.b
+                                       Hash Cond: (i2_2.b = b_2.b)
+                                       ->  Seq Scan on public.prt2_p3 i2_2
+                                             Output: i2_2.b
+                                       ->  Hash
+                                             Output: b_2.b, b_2.a
+                                             ->  Seq Scan on public.prt2_p3 b_2
+                                                   Output: b_2.b, b_2.a
+                                                   Filter: ((b_2.a % 25) = 0)
+                                 ->  Index Scan using iprt1_p3_a on public.prt1_p3 a_2
+                                       Output: a_2.a, a_2.c
+                                       Index Cond: (a_2.a = i2_2.b)
+                           ->  Index Only Scan using iprt1_p3_a on public.prt1_p3 i1_2
+                                 Output: i1_2.a
+                                 Index Cond: (i1_2.a = a_2.a)
+(80 rows)
+
+select b.b from prt1 a join prt2 b on a.a = b.b left join plt1 c on b.a % 25 = 0 and c.c = a.c join prt1 i1 on b.b = i1.a
+right join prt2 i2 on i2.b = b.b where b.a % 25 = 0 order by 1;
+  b  
+-----
+   0
+   0
+   0
+   0
+   0
+   0
+   0
+   0
+   0
+   0
+   0
+   0
+   0
+   0
+   0
+   0
+   0
+   0
+   0
+   0
+   0
+   0
+   0
+   0
+   0
+ 150
+ 300
+ 450
+(28 rows)
+
 --
 -- list partitioned by expression
 --
@@ -1304,6 +1923,86 @@ SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM plt1 t1, pl
  574.0000000000000000 | 574.5000000000000000 | 1148.0000000000000000 | 0011 | 0011 | A0011
 (12 rows)
 
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                         QUERY PLAN                                         
+--------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c, ((t3.a + t3.b)), t3.c
+   Sort Key: t1.a, t2.b, ((t3.a + t3.b))
+   ->  Result
+         Output: t1.a, t1.c, t2.b, t2.c, (t3.a + t3.b), t3.c
+         ->  Append
+               ->  Hash Right Join
+                     Output: t1.a, t1.c, t2.b, t2.c, t3.a, t3.b, t3.c
+                     Hash Cond: ((t3.a = t2.b) AND (ltrim(t3.c, 'A'::text) = t2.c))
+                     ->  Seq Scan on public.plt1_e_p1 t3
+                           Output: t3.a, t3.b, t3.c
+                     ->  Hash
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           ->  Hash Right Join
+                                 Output: t1.a, t1.c, t2.b, t2.c
+                                 Hash Cond: ((t2.b = t1.a) AND (t2.c = t1.c))
+                                 ->  Seq Scan on public.plt2_p1 t2
+                                       Output: t2.b, t2.c
+                                 ->  Hash
+                                       Output: t1.a, t1.c
+                                       ->  Seq Scan on public.plt1_p1 t1
+                                             Output: t1.a, t1.c
+                                             Filter: ((t1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c, t3_1.a, t3_1.b, t3_1.c
+                     Hash Cond: ((t3_1.a = t2_1.b) AND (ltrim(t3_1.c, 'A'::text) = t2_1.c))
+                     ->  Seq Scan on public.plt1_e_p2 t3_1
+                           Output: t3_1.a, t3_1.b, t3_1.c
+                     ->  Hash
+                           Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                           ->  Hash Right Join
+                                 Output: t1_1.a, t1_1.c, t2_1.b, t2_1.c
+                                 Hash Cond: ((t2_1.b = t1_1.a) AND (t2_1.c = t1_1.c))
+                                 ->  Seq Scan on public.plt2_p2 t2_1
+                                       Output: t2_1.b, t2_1.c
+                                 ->  Hash
+                                       Output: t1_1.a, t1_1.c
+                                       ->  Seq Scan on public.plt1_p2 t1_1
+                                             Output: t1_1.a, t1_1.c
+                                             Filter: ((t1_1.a % 25) = 0)
+               ->  Hash Right Join
+                     Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c, t3_2.a, t3_2.b, t3_2.c
+                     Hash Cond: ((t3_2.a = t2_2.b) AND (ltrim(t3_2.c, 'A'::text) = t2_2.c))
+                     ->  Seq Scan on public.plt1_e_p3 t3_2
+                           Output: t3_2.a, t3_2.b, t3_2.c
+                     ->  Hash
+                           Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                           ->  Hash Right Join
+                                 Output: t1_2.a, t1_2.c, t2_2.b, t2_2.c
+                                 Hash Cond: ((t2_2.b = t1_2.a) AND (t2_2.c = t1_2.c))
+                                 ->  Seq Scan on public.plt2_p3 t2_2
+                                       Output: t2_2.b, t2_2.c
+                                 ->  Hash
+                                       Output: t1_2.a, t1_2.c
+                                       ->  Seq Scan on public.plt1_p3 t1_2
+                                             Output: t1_2.a, t1_2.c
+                                             Filter: ((t1_2.a % 25) = 0)
+(57 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |          | 
+ 100 | 0002 |     |      |          | 
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |          | 
+ 250 | 0005 |     |      |          | 
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |          | 
+ 400 | 0008 |     |      |          | 
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |          | 
+ 550 | 0011 |     |      |          | 
+(12 rows)
+
 -- 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;
@@ -1361,6 +2060,172 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
                      One-Time Filter: false
 (14 rows)
 
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.c NOT IN ('0001', '0005', '0002', '0009') AND t2.c NOT IN ('0000', '0003', '0004', '0010') GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+                                          QUERY PLAN                                           
+-----------------------------------------------------------------------------------------------
+ Sort
+   Output: (sum(t1.a)), t1.c, (avg(t2.b)), t2.c
+   Sort Key: t1.c
+   ->  HashAggregate
+         Output: sum(t1.a), t1.c, avg(t2.b), t2.c
+         Group Key: t1.c, t2.c
+         ->  Result
+               Output: t1.c, t2.c, t1.a, t2.b
+               ->  Append
+                     ->  Hash Join
+                           Output: t1.a, t1.c, t2.b, t2.c
+                           Hash Cond: (t1.c = t2.c)
+                           ->  Seq Scan on public.plt1_p3 t1
+                                 Output: t1.a, t1.c
+                                 Filter: (t1.c <> ALL ('{0001,0005,0002,0009}'::text[]))
+                           ->  Hash
+                                 Output: t2.b, t2.c
+                                 ->  Seq Scan on public.plt2_p3 t2
+                                       Output: t2.b, t2.c
+                                       Filter: (t2.c <> ALL ('{0000,0003,0004,0010}'::text[]))
+(20 rows)
+
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.c NOT IN ('0001', '0005', '0002', '0009') AND t2.c NOT IN ('0000', '0003', '0004', '0010') GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+  sum   |  c   |         avg          |  c   
+--------+------+----------------------+------
+ 137700 | 0006 | 324.0000000000000000 | 0006
+ 158950 | 0007 | 375.0000000000000000 | 0007
+ 169600 | 0008 | 424.5000000000000000 | 0008
+ 229600 | 0011 | 574.5000000000000000 | 0011
+(4 rows)
+
+-- test merge join with index scan
+CREATE INDEX iplt1_p1_c on plt1_p1(c);
+CREATE INDEX iplt1_p2_c on plt1_p2(c);
+CREATE INDEX iplt1_p3_c on plt1_p3(c);
+CREATE INDEX iplt2_p1_c on plt2_p1(c);
+CREATE INDEX iplt2_p2_c on plt2_p2(c);
+CREATE INDEX iplt2_p3_c on plt2_p3(c);
+CREATE INDEX iplt1_e_p1_c on plt1_e_p1(ltrim(c, 'A'));
+CREATE INDEX iplt1_e_p2_c on plt1_e_p2(ltrim(c, 'A'));
+CREATE INDEX iplt1_e_p3_c on plt1_e_p3(ltrim(c, 'A'));
+ANALYZE plt1;
+ANALYZE plt2;
+ANALYZE plt1_e;
+SET enable_hashjoin TO off;
+SET enable_nestloop TO off;
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+                                                  QUERY PLAN                                                   
+---------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, ((plt1_e_p1.a + plt1_e_p1.b)), plt1_e_p1.c
+   Sort Key: plt1_p1.a, plt2_p1.b, ((plt1_e_p1.a + plt1_e_p1.b))
+   ->  Result
+         Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, (plt1_e_p1.a + plt1_e_p1.b), plt1_e_p1.c
+         ->  Append
+               ->  Merge Full Join
+                     Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c, plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c
+                     Merge Cond: ((plt1_p1.a = plt1_e_p1.a) AND (plt1_p1.c = (ltrim(plt1_e_p1.c, 'A'::text))))
+                     ->  Sort
+                           Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+                           Sort Key: plt1_p1.a, plt1_p1.c
+                           ->  Merge Full Join
+                                 Output: plt1_p1.a, plt1_p1.c, plt2_p1.b, plt2_p1.c
+                                 Merge Cond: ((plt1_p1.a = plt2_p1.b) AND (plt1_p1.c = plt2_p1.c))
+                                 ->  Sort
+                                       Output: plt1_p1.a, plt1_p1.c
+                                       Sort Key: plt1_p1.a, plt1_p1.c
+                                       ->  Seq Scan on public.plt1_p1
+                                             Output: plt1_p1.a, plt1_p1.c
+                                             Filter: ((plt1_p1.a % 25) = 0)
+                                 ->  Sort
+                                       Output: plt2_p1.b, plt2_p1.c
+                                       Sort Key: plt2_p1.b, plt2_p1.c
+                                       ->  Seq Scan on public.plt2_p1
+                                             Output: plt2_p1.b, plt2_p1.c
+                                             Filter: ((plt2_p1.b % 25) = 0)
+                     ->  Sort
+                           Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c, (ltrim(plt1_e_p1.c, 'A'::text))
+                           Sort Key: plt1_e_p1.a, (ltrim(plt1_e_p1.c, 'A'::text))
+                           ->  Seq Scan on public.plt1_e_p1
+                                 Output: plt1_e_p1.a, plt1_e_p1.b, plt1_e_p1.c, ltrim(plt1_e_p1.c, 'A'::text)
+                                 Filter: ((plt1_e_p1.a % 25) = 0)
+               ->  Merge Full Join
+                     Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c, plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c
+                     Merge Cond: ((plt1_p2.a = plt1_e_p2.a) AND (plt1_p2.c = (ltrim(plt1_e_p2.c, 'A'::text))))
+                     ->  Sort
+                           Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+                           Sort Key: plt1_p2.a, plt1_p2.c
+                           ->  Merge Full Join
+                                 Output: plt1_p2.a, plt1_p2.c, plt2_p2.b, plt2_p2.c
+                                 Merge Cond: ((plt1_p2.a = plt2_p2.b) AND (plt1_p2.c = plt2_p2.c))
+                                 ->  Sort
+                                       Output: plt1_p2.a, plt1_p2.c
+                                       Sort Key: plt1_p2.a, plt1_p2.c
+                                       ->  Seq Scan on public.plt1_p2
+                                             Output: plt1_p2.a, plt1_p2.c
+                                             Filter: ((plt1_p2.a % 25) = 0)
+                                 ->  Sort
+                                       Output: plt2_p2.b, plt2_p2.c
+                                       Sort Key: plt2_p2.b, plt2_p2.c
+                                       ->  Seq Scan on public.plt2_p2
+                                             Output: plt2_p2.b, plt2_p2.c
+                                             Filter: ((plt2_p2.b % 25) = 0)
+                     ->  Sort
+                           Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c, (ltrim(plt1_e_p2.c, 'A'::text))
+                           Sort Key: plt1_e_p2.a, (ltrim(plt1_e_p2.c, 'A'::text))
+                           ->  Seq Scan on public.plt1_e_p2
+                                 Output: plt1_e_p2.a, plt1_e_p2.b, plt1_e_p2.c, ltrim(plt1_e_p2.c, 'A'::text)
+                                 Filter: ((plt1_e_p2.a % 25) = 0)
+               ->  Merge Full Join
+                     Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c, plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c
+                     Merge Cond: ((plt1_p3.a = plt1_e_p3.a) AND (plt1_p3.c = (ltrim(plt1_e_p3.c, 'A'::text))))
+                     ->  Sort
+                           Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+                           Sort Key: plt1_p3.a, plt1_p3.c
+                           ->  Merge Full Join
+                                 Output: plt1_p3.a, plt1_p3.c, plt2_p3.b, plt2_p3.c
+                                 Merge Cond: ((plt1_p3.a = plt2_p3.b) AND (plt1_p3.c = plt2_p3.c))
+                                 ->  Sort
+                                       Output: plt1_p3.a, plt1_p3.c
+                                       Sort Key: plt1_p3.a, plt1_p3.c
+                                       ->  Seq Scan on public.plt1_p3
+                                             Output: plt1_p3.a, plt1_p3.c
+                                             Filter: ((plt1_p3.a % 25) = 0)
+                                 ->  Sort
+                                       Output: plt2_p3.b, plt2_p3.c
+                                       Sort Key: plt2_p3.b, plt2_p3.c
+                                       ->  Seq Scan on public.plt2_p3
+                                             Output: plt2_p3.b, plt2_p3.c
+                                             Filter: ((plt2_p3.b % 25) = 0)
+                     ->  Sort
+                           Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c, (ltrim(plt1_e_p3.c, 'A'::text))
+                           Sort Key: plt1_e_p3.a, (ltrim(plt1_e_p3.c, 'A'::text))
+                           ->  Seq Scan on public.plt1_e_p3
+                                 Output: plt1_e_p3.a, plt1_e_p3.b, plt1_e_p3.c, ltrim(plt1_e_p3.c, 'A'::text)
+                                 Filter: ((plt1_e_p3.a % 25) = 0)
+(87 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+  a  |  c   |  b  |  c   | ?column? |   c   
+-----+------+-----+------+----------+-------
+   0 | 0000 |   0 | 0000 |        0 | A0000
+  50 | 0001 |     |      |      100 | A0001
+ 100 | 0002 |     |      |      200 | A0002
+ 150 | 0003 | 150 | 0003 |      300 | A0003
+ 200 | 0004 |     |      |      400 | A0004
+ 250 | 0005 |     |      |      500 | A0005
+ 300 | 0006 | 300 | 0006 |      600 | A0006
+ 350 | 0007 |     |      |      700 | A0007
+ 400 | 0008 |     |      |      800 | A0008
+ 450 | 0009 | 450 | 0009 |      900 | A0009
+ 500 | 0010 |     |      |     1000 | A0010
+ 550 | 0011 |     |      |     1100 | A0011
+     |      |  75 | 0001 |          | 
+     |      | 225 | 0004 |          | 
+     |      | 375 | 0007 |          | 
+     |      | 525 | 0010 |          | 
+(16 rows)
+
+RESET enable_hashjoin;
+RESET enable_nestloop;
 --
 -- multi-leveled partitions
 --
@@ -1687,6 +2552,42 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE a = 1 AND a = 2)
                One-Time Filter: false
 (11 rows)
 
+-- Join with pruned partitions from joining relations
+EXPLAIN (VERBOSE, 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 = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a < 450 AND t2.b > 250 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+                                                            QUERY PLAN                                                            
+----------------------------------------------------------------------------------------------------------------------------------
+ Sort
+   Output: t1.a, t1.c, t2.b, t2.c
+   Sort Key: t1.a
+   ->  Append
+         ->  Hash Join
+               Output: t1.a, t1.c, t2.b, t2.c
+               Hash Cond: ((t2.b = t1.a) AND (t2.a = t1.b) AND ((t2.c)::text = (t1.c)::text) AND ((t2.a + t2.b) = (t1.b + t1.a)))
+               ->  Append
+                     ->  Seq Scan on public.prt2_l_p2_p1 t2
+                           Output: t2.b, t2.c, t2.a
+                           Filter: (t2.b > 250)
+                     ->  Seq Scan on public.prt2_l_p2_p2 t2_1
+                           Output: t2_1.b, t2_1.c, t2_1.a
+                           Filter: (t2_1.b > 250)
+               ->  Hash
+                     Output: t1.a, t1.c, t1.b
+                     ->  Append
+                           ->  Seq Scan on public.prt1_l_p2_p1 t1
+                                 Output: t1.a, t1.c, t1.b
+                                 Filter: ((t1.a < 450) AND ((t1.a % 25) = 0))
+                           ->  Seq Scan on public.prt1_l_p2_p2 t1_1
+                                 Output: t1_1.a, t1_1.c, t1_1.b
+                                 Filter: ((t1_1.a < 450) AND ((t1_1.a % 25) = 0))
+(23 rows)
+
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_l t1, prt2_l t2 WHERE t1.a = t2.b AND t1.b = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a < 450 AND t2.b > 250 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+ 300 | 0000 | 300 | 0000
+(1 row)
+
 --
 -- negative testcases
 --
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index 21734f5..d8f818a 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -99,6 +99,27 @@ 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;
 
+-- join with aggregate
+EXPLAIN (VERBOSE, COSTS OFF)
+select t1.a, count(t2.*) from prt1 t1 left join prt1 t2 on (t1.a = t2.a) where t1.a % 25 = 0 group by t1.a;
+select t1.a, count(t2.*) from prt1 t1 left join prt1 t2 on (t1.a = t2.a) where t1.a % 25 = 0 group by t1.a;
+
+-- lateral with VALUES
+EXPLAIN (VERBOSE, COSTS OFF)
+select count(*) from prt1 a, prt2 b join lateral (values(a.a)) ss(x) on b.b = ss.x;
+select count(*) from prt1 a, prt2 b join lateral (values(a.a)) ss(x) on b.b = ss.x;
+
+-- join with rank
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.b, t2.b,rank() OVER (ORDER BY t2.b DESC) FROM prt1 t1 inner join prt2 t2 on (t1.a = t2.b) where t1.a % 25 = 0 order by 1,2,3;
+SELECT t1.a, t1.b, t2.b,rank() OVER (ORDER BY t2.b DESC) FROM prt1 t1 inner join prt2 t2 on (t1.a = t2.b) where t1.a % 25 = 0 order by 1,2,3;
+
+--join with prepare statement
+PREPARE ij(int) AS select t1.a,t2.b from prt1 t1 inner join prt2 t2 on (t1.a = t2.b and t1.a % $1 = 0) ORDER BY 1,2;
+EXPLAIN (VERBOSE, COSTS OFF) EXECUTE ij(25);
+EXECUTE ij(25);
+DEALLOCATE ij;
+
 --
 -- partitioned by expression
 --
@@ -163,7 +184,28 @@ SELECT t1.* FROM prt1 t1 WHERE t1.a IN (SELECT t1.b FROM prt2 t1 WHERE t1.b IN (
 
 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;
-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;
+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;
+
+EXPLAIN (VERBOSE, 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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+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.a %25 = 0)) AND t1.a % 25 = 0 ORDER BY t1.a;
+
+SET enable_seqscan TO off;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (prt1 t1 RIGHT JOIN prt2 t2 ON t1.a = t2.b) RIGHT JOIN prt1_e t3 ON (t2.b = (t3.a + t3.b)/2) WHERE t3.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+-- lateral references and parameterized paths
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+(SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+SELECT * FROM prt1 t1 LEFT JOIN LATERAL
+(SELECT t2.a AS t2a, t3.a AS t3a, least(t1.a,t2.a,t3.a) FROM prt1 t2 JOIN prt2 t3 ON (t2.a = t3.b)) ss
+ON t1.a = ss.t2a WHERE t1.a % 25 = 0 ORDER BY t1.a;
+RESET enable_seqscan;
 
 -- MergeAppend on nullable column
 EXPLAIN (COSTS OFF)
@@ -211,6 +253,20 @@ CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007', '0008', '0
 INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 3) i;
 ANALYZE plt2;
 
+--full join using
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT * FROM plt1 t1 FULL JOIN plt2 t2 USING (c) FULL JOIN plt1 t3 USING (c)
+where t1.a %150 =0 and t2.b % 150 = 0 and t3.a % 150 = 0 order by 1,2,3;
+SELECT * FROM plt1 t1 FULL JOIN plt2 t2 USING (c) FULL JOIN plt1 t3 USING (c)
+where t1.a %150 =0 and t2.b % 150 = 0 and t3.a % 150 = 0 order by 1,2,3;
+
+-- test placement of movable quals in a parameterized join tree
+EXPLAIN (VERBOSE, COSTS OFF)
+select b.b from prt1 a join prt2 b on a.a = b.b left join plt1 c on b.a % 25 = 0 and c.c = a.c join prt1 i1 on b.b = i1.a
+right join prt2 i2 on i2.b = b.b where b.a % 25 = 0 order by 1;
+select b.b from prt1 a join prt2 b on a.a = b.b left join plt1 c on b.a % 25 = 0 and c.c = a.c join prt1 i1 on b.b = i1.a
+right join prt2 i2 on i2.b = b.b where b.a % 25 = 0 order by 1;
+
 --
 -- list partitioned by expression
 --
@@ -226,6 +282,10 @@ 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;
 
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM (plt1 t1 LEFT JOIN plt2 t2 ON t1.a = t2.b AND t1.c = t2.c) LEFT JOIN plt1_e t3 ON (t2.b = t3.a AND t2.c = ltrim(t3.c, 'A')) WHERE t1.a % 25 = 0 ORDER BY t1.a, t2.b, t3.a + t3.b;
+
 -- 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;
@@ -239,6 +299,35 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1 WHERE a = 1 AND a = 2) t1
 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;
 
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.c NOT IN ('0001', '0005', '0002', '0009') AND t2.c NOT IN ('0000', '0003', '0004', '0010') GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT sum(t1.a), t1.c, avg(t2.b), t2.c FROM plt1 t1, plt2 t2 WHERE t1.c = t2.c AND t1.c NOT IN ('0001', '0005', '0002', '0009') AND t2.c NOT IN ('0000', '0003', '0004', '0010') GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
+-- test merge join with index scan
+CREATE INDEX iplt1_p1_c on plt1_p1(c);
+CREATE INDEX iplt1_p2_c on plt1_p2(c);
+CREATE INDEX iplt1_p3_c on plt1_p3(c);
+CREATE INDEX iplt2_p1_c on plt2_p1(c);
+CREATE INDEX iplt2_p2_c on plt2_p2(c);
+CREATE INDEX iplt2_p3_c on plt2_p3(c);
+CREATE INDEX iplt1_e_p1_c on plt1_e_p1(ltrim(c, 'A'));
+CREATE INDEX iplt1_e_p2_c on plt1_e_p2(ltrim(c, 'A'));
+CREATE INDEX iplt1_e_p3_c on plt1_e_p3(ltrim(c, 'A'));
+
+ANALYZE plt1;
+ANALYZE plt2;
+ANALYZE plt1_e;
+
+SET enable_hashjoin TO off;
+SET enable_nestloop TO off;
+
+EXPLAIN (VERBOSE, COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+SELECT t1.a, t1.c, t2.b, t2.c, t3.a + t3.b, t3.c FROM ((SELECT * FROM plt1 WHERE plt1.a % 25 = 0) t1 FULL JOIN (SELECT * FROM plt2 WHERE plt2.b % 25 = 0) t2 ON (t1.a = t2.b AND t1.c = t2.c)) FULL JOIN (SELECT * FROM plt1_e WHERE plt1_e.a % 25 = 0) t3 ON (t1.a = t3.a AND ltrim(t3.c, 'A') = t1.c) ORDER BY t1.a, t2.b, t3.a + t3.b;
+
+RESET enable_hashjoin;
+RESET enable_nestloop;
+
 --
 -- multi-leveled partitions
 --
@@ -297,6 +386,11 @@ SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
 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;
 
+-- Join with pruned partitions from joining relations
+EXPLAIN (VERBOSE, 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 = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a < 450 AND t2.b > 250 AND t1.a % 25 = 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 = t2.a AND t1.c = t2.c AND t1.b + t1.a = t2.a + t2.b AND t1.a < 450 AND t2.b > 250 AND t1.a % 25 = 0 ORDER BY t1.a, t2.b;
+
 --
 -- negative testcases
 --
#262Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#254)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Sep 19, 2017 at 5:47 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Done.

Committed 0001 with extensive editorialization. I did not think it
was a good idea to include a partition.h a file in src/include/nodes,
so I worked around that. The include of pg_inherits_fn.h was
unneeded. I rewrote a lot of the comments and made some other style
tweaks.

Don't look now, but I think it might be about time for the main act.

--
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

#263Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#262)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Sep 21, 2017 at 9:12 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Tue, Sep 19, 2017 at 5:47 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Done.

Committed 0001 with extensive editorialization. I did not think it
was a good idea to include a partition.h a file in src/include/nodes,
so I worked around that. The include of pg_inherits_fn.h was
unneeded. I rewrote a lot of the comments and made some other style
tweaks.

Thanks a lot Robert. Thanks for changing comments to be more precise and crisp.

Here's set of rebased patches. The patch with extra tests is not for
committing. All other patches, except the last one, will need to be
committed together. The last patch may be committed along with other
patches or as a separate patch.

About your earlier comment of making build_joinrel_partition_info()
simpler. Right now, the code assumes that partexprs or
nullable_partexpr can be NULL when either of them is not populated.
That may be saves a sizeof(pointer) * (number of keys) byes of memory.
Saving that much memory may not be worth the complexity of code. So,
we may always allocate memory for those arrays and fill it with NIL
values when there are no key expressions to populate those. That will
simplify the code. I haven't done that change in this patchset. I was
busy debugging the Q7 regression. Let me know your comments about
that.

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

Attachments:

pg_dp_join_patches_v34.tar.gzapplication/x-gzip; name=pg_dp_join_patches_v34.tar.gzDownload
#264Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#263)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Sep 21, 2017 at 8:21 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

About your earlier comment of making build_joinrel_partition_info()
simpler. Right now, the code assumes that partexprs or
nullable_partexpr can be NULL when either of them is not populated.
That may be saves a sizeof(pointer) * (number of keys) byes of memory.
Saving that much memory may not be worth the complexity of code. So,
we may always allocate memory for those arrays and fill it with NIL
values when there are no key expressions to populate those. That will
simplify the code. I haven't done that change in this patchset. I was
busy debugging the Q7 regression. Let me know your comments about
that.

Hmm, I'm not sure that's the best approach, but let me look at it more
carefully before I express a firm opinion.

--
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

#265Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Rafia Sabih (#250)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Mon, Sep 18, 2017 at 10:18 AM, Rafia Sabih
<rafia.sabih@enterprisedb.com> wrote:

Limit (cost=83341943.28..83341943.35 rows=1 width=92) (actual
time=1556989.996..1556989.997 rows=1 loops=1)
-> Finalize GroupAggregate (cost=83341943.28..83342723.24
rows=10064 width=92) (actual time=1556989.994..1556989.994 rows=1
loops=1)
Group Key: n1.n_name, n2.n_name, (date_part('year'::text,
(lineitem_001.l_shipdate)::timestamp without time zone))
-> Sort (cost=83341943.28..83342043.92 rows=40256 width=92)
(actual time=1556989.910..1556989.911 rows=6 loops=1)
Sort Key: n1.n_name, n2.n_name,
(date_part('year'::text, (lineitem_001.l_shipdate)::timestamp without
time zone))
Sort Method: quicksort Memory: 27kB
-> Gather (cost=83326804.81..83338864.31 rows=40256
width=92) (actual time=1550598.855..1556989.760 rows=20 loops=1)
Workers Planned: 4
Workers Launched: 4

AFAICU the node above sort is group-aggregate and then there is limit,
and the number of rows for sort node in explain analyse is returned
number of rows. So, what is happening here is once one group is
completed it is aggregated and fetched by limit, now there is no need
for sort to return any more rows and hence the result.

Thanks for your explanation. That makes sense. I forgot about LIMIT node on top.

I debugged the plans today and performed some experiments. Here are my
observations

The join order with and without partition-wise join changes. Without
partition-wise join it is
(lineitem, (suppliers, nation1)), (orders, (customer, nation2)). The
join (lineitem, (suppliers, nation1)) is executed by one gather node
and (orders, (customer, nation2)) is executed by other. Thus the plan
has two gather nodes, which feed to the topmost join.
With partition-wise join the join order is ((lineitem, orders),
(supplier, nation1)), (customer, nation2). The join (lineitem, orders)
uses partition-wise join. This plan executes the whole join tree along
with partial group aggregation under a gather merge.

The rows estimated for various nodes under Gather/GatherMerge are
different from the actual rows e.g.
-> Hash Join (cost=113164.47..61031454.40 rows=10789501 width=46)
(actual time=3379.931..731987.943 rows=8744357 loops=5) (in
non-partition-wise join plan) OR
-> Append (cost=179532.36..80681785.95 rows=134868761 width=24)
(actual time=9437.573..1360219.567 rows=109372134 loops=5) (in
partition-wise join plan).
I first thought that this is a real estimation error and spent some
time investigating the estimation error. But eventually realised that
this is how a parallel query plan reports, when I saw that Gather node
estimated correct number of rows even though the nodes under it showed
this difference. Here's the explanation of this report. There are 4
parallel workers, so, the leaders contribution would be estimated to
be 0 by get_parallel_divisor(). So these estimates are per worker and
so the total estimated rows produced by any of the nodes is 4 times
the reported. But when the query actually runs, the leader also
participates, so number of loops = 5 and the actual rows reported are
(total actual rows) / (number of loops i.e. number of backends that
executed the query). The total estimates rows and total actual rows
are roughly equal. So there's no real estimation error, as I thought
earlier. May be we want to make EXPLAIN (ANALYZE) output easier to
understand.

When I tried the same query on laptop with scale 20, I found that the
leader is really contributing as much as other workers. So, the
partial paths were really created based on an estimate which was 20%
off. The cost difference between partition-wise join plan and
non-partition-wise join plan is hardly 1.5%. So, it's possible that if
we correct this estimation error, partition-wise join plan won't be
chosen because of it will have a higher cost. Remember there are two
gather nodes in non-partition-wise join plan and partition-wise join
plan has one gather. So, non-partition-wise join path gets the 20%
decreased estimates twice and partition-wise join gets it only once.

The explain (analyze, verbose) of a parallel node looks like
-> Parallel Seq Scan on public.lineitem_002 (cost=0.00..168752.99
rows=573464 width=24) (actual time=1.395..3075.485 rows=454464
loops=5)
Filter:
((lineitem_002.l_shipdate >= '1995-01-01'::date) AND
(lineitem_002.l_shipdate <= '1996-12-31'::date))
Rows Removed by Filter: 1045065
Worker 0: actual
time=3.358..3131.426 rows=458267 loops=1
Worker 1: actual
time=0.860..3146.282 rows=447231 loops=1
Worker 2: actual
time=1.317..3123.646 rows=489960 loops=1
Worker 3: actual
time=0.927..3130.497 rows=475545 loops=1
If we sum the rows returned by each worker they don't add up to
(actual rows) * (actual loops). So I assumed that the unreported
number of rows were processed by the leader. Is that right?

I might be misunderstanding how parallel query works, but here's my
analysis so far. I will continue investigating further.

Any clues would be helpful.

--
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

#266Rafia Sabih
rafia.sabih@enterprisedb.com
In reply to: Rafia Sabih (#253)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Sep 19, 2017 at 2:58 PM, Rafia Sabih
<rafia.sabih@enterprisedb.com> wrote:

On Fri, Sep 15, 2017 at 2:09 PM, Rafia Sabih
<rafia.sabih@enterprisedb.com> wrote:

On TPC-H benchmarking of this patch, I found a regression in Q7. It
was taking some 1500s with the patch and some 900s without the patch.
Please find the attached pwd_reg.zip for the output of explain analyse
on head and with patch.

The experimental settings used were,
commit-id = 0c504a80cf2e6f66df2cdea563e879bf4abd1629
patch-version = v26

Server settings:
work_mem = 1GB
shared_buffers = 10GB
effective_cache_size = 10GB
max_parallel_workers_per_gather = 4

Partitioning information:
Partitioning scheme = by range
Number of partitions in lineitem and orders table = 106
partition key for lineitem = l_orderkey
partition key for orders = o_orderkey

Apart from these there is a regression case on a custom table, on head
query completes in 20s and with this patch it takes 27s. Please find
the attached .out and .sql file for the output and schema for the test
case respectively. I have reported this case before (sometime around
March this year) as well, but I am not sure if it was overlooked or is
an unimportant and expected behaviour for some reason.

On completing the benchmark for all queries for the above mentioned
setup, following performance improvement can be seen,
Query | Patch | Head
3 | 1455 | 1631
4 | 499 | 4344
5 | 1464 | 1606
10 | 1475 | 1599
12 | 1465 | 1790

Note that all values of execution time are in seconds.

I compared this experiment with non-partitioned database and following
is the result,
Query | Non-partitioned head
3 | 1752
4 | 315
5 | 2319
10 | 1535
12 | 1739

In summary, the query that appears slowest in partitioned database is
not so otherwise. It is good to see that in Q4 partition-wise join
helps in achieving performance closer to it's non-partitioned case,
otherwise partitioning alone causes it to suffer greatly. Apart from
Q4 it does not looks like partitioning hurts anywhere else, though the
maximum improvement is ~35% for Q5.
Another point to note here is that the performance on partitioned and
unpartitioned heads are quite close (except Q4) which is something
atleast I wasn't expecting. It looks like we need not to partition the
tables anyway, or atleast this set of queries doesn't benefit from
partitioning. Please let me know if somebody has better ideas on how
partitioning schemes should be applied to make it more beneficial for
these queries.

--
Regards,
Rafia Sabih
EnterpriseDB: http://www.enterprisedb.com/

Attachments:

pg_unpart.zipapplication/zip; name=pg_unpart.zipDownload
#267Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Rafia Sabih (#266)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Sep 22, 2017 at 10:45 AM, Rafia Sabih
<rafia.sabih@enterprisedb.com> wrote:

On completing the benchmark for all queries for the above mentioned
setup, following performance improvement can be seen,
Query | Patch | Head
3 | 1455 | 1631
4 | 499 | 4344
5 | 1464 | 1606
10 | 1475 | 1599
12 | 1465 | 1790

Note that all values of execution time are in seconds.

I compared this experiment with non-partitioned database and following
is the result,
Query | Non-partitioned head
3 | 1752
4 | 315
5 | 2319
10 | 1535
12 | 1739

In summary, the query that appears slowest in partitioned database is
not so otherwise. It is good to see that in Q4 partition-wise join
helps in achieving performance closer to it's non-partitioned case,
otherwise partitioning alone causes it to suffer greatly. Apart from
Q4 it does not looks like partitioning hurts anywhere else, though the
maximum improvement is ~35% for Q5.
Another point to note here is that the performance on partitioned and
unpartitioned heads are quite close (except Q4) which is something
atleast I wasn't expecting. It looks like we need not to partition the
tables anyway, or atleast this set of queries doesn't benefit from
partitioning. Please let me know if somebody has better ideas on how
partitioning schemes should be applied to make it more beneficial for
these queries.

Just partitioning is not expected to improve query performance (but we
still see some performance improvement). Partitioning + partition-wise
operations, pruning is expected to show performance gains. IIUC the
results you reported, Q3 takes 1752 seconds with non-partitioned head,
with partitioning it completes in 1631 seconds and with partition-wise
join it completes in 1455, so net improvement because of partitioning
is 300 seconds is almost 16% improvement, which is a lot for very
large data. So, except Q4, every query improves when the tables are
partitioned. Am I interpreting the results correctly?

There may be some other way of partitioning, which may give better
results, but I think what we have now shows the importance of
partitioning in case of very large data e.g. scale 300 TPCH.

--
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

#268Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Ashutosh Bapat (#246)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Sep 15, 2017 at 5:29 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Apart from these there is a regression case on a custom table, on head
query completes in 20s and with this patch it takes 27s. Please find
the attached .out and .sql file for the output and schema for the test
case respectively. I have reported this case before (sometime around
March this year) as well, but I am not sure if it was overlooked or is
an unimportant and expected behaviour for some reason.

Are you talking about [1]? I have explained about the regression in
[2] and [3]. This looks like an issue with the existing costing model.

I debugged this case further. There are two partitioned tables being
joined prt (with partitions prt_p1, prt_p2 and so on) and prt2 (with
partitions prt2_p1, prt2_p2, and so on). When join is executed without
partition-wise join, prt2 is used to build hash table and prt is used
to probe that hash table. prt2 has lesser number of rows than prt. But
when partition-wise join is used, individual partitions are joined in
reverse join order i.e. partitions of prt are used to build the hash
table and partitions of prt2 are used to probe. This happens because
the path for the other join order (partition of prt2 used to build the
hash table and partition of prt used to probe) has huge cost compared
to the first one (74459 and 313109) and a portion worth 259094 comes
from lines 3226/7 of final_cost_hashjoin()
3215 /*
3216 * The number of tuple comparisons needed is the number of outer
3217 * tuples times the typical number of tuples in a hash
bucket, which
3218 * is the inner relation size times its bucketsize
fraction. At each
3219 * one, we need to evaluate the hashjoin quals. But actually,
3220 * charging the full qual eval cost at each tuple is pessimistic,
3221 * since we don't evaluate the quals unless the hash values match
3222 * exactly. For lack of a better idea, halve the cost estimate to
3223 * allow for that.
3224 */
3225 startup_cost += hash_qual_cost.startup;
3226 run_cost += hash_qual_cost.per_tuple * outer_path_rows *
3227 clamp_row_est(inner_path_rows * innerbucketsize) * 0.5;

That's because for some reason innerbucketsize for partition of prt is
22 times more than that for partition of prt2. Looks like we have some
estimation error in estimating bucket sizes.

If I force partitions to be joined with the same order as partitioned
tables (without partition-wise join), child-joins execute faster and
in turn partition-wise join performs better than the
non-partition-wise join. So, this is clearly some estimation and
costing problem with regular joins.

--
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

#269Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#263)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Sep 21, 2017 at 8:21 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Here's set of rebased patches. The patch with extra tests is not for
committing. All other patches, except the last one, will need to be
committed together. The last patch may be committed along with other
patches or as a separate patch.

In set_append_rel_size, is it necessary to set attr_needed =
bms_copy(rel->attr_needed[index]) rather than just pointing to the
existing value? If so, perhaps the comments should explain the
reasons. I would have thought that the values wouldn't change after
this point, in which case it might not be necessary to copy them.

Regarding nomenclature and my previous griping about wisdom, I was
wondering about just calling this a "partition join" like you have in
the regression test. So the GUC would be enable_partition_join, you'd
have generate_partition_join_paths(), etc. Basically just delete
"wise" throughout.

The elog(DEBUG3) in try_partition_wise_join() doesn't follow message
style guidelines and I think should just be removed. It was useful
for development, I'm sure, but it's time for it to go.

+ elog(ERROR, "unrecognized path node type %d", (int) nodeTag(path));

I think we should use the same formulation as elsewhere, namely
"unrecognized node type: %d". And likewise probably "unexpected join
type: %d".

partition_join_extras.sql has a bunch of whitespace damage, although
it doesn't really matter since, as you say, that's not for commit.

(This is not a full review, just a few thoughts.)

--
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

#270Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#269)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Oct 3, 2017 at 7:48 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Thu, Sep 21, 2017 at 8:21 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Here's set of rebased patches. The patch with extra tests is not for
committing. All other patches, except the last one, will need to be
committed together. The last patch may be committed along with other
patches or as a separate patch.

In set_append_rel_size, is it necessary to set attr_needed =
bms_copy(rel->attr_needed[index]) rather than just pointing to the
existing value? If so, perhaps the comments should explain the
reasons. I would have thought that the values wouldn't change after
this point, in which case it might not be necessary to copy them.

Right. The only places where attr_needed is changed is in
remove_rel_from_query() (useless join removal) and
add_vars_to_targetlist(). Both of those happen before
set_append_rel_size(). Since parent and child join should project same
attributes, having them share the Relids set makes more sense. So,
changed accordingly and explained the same in comments.

Also, changed list_nth() in the following code block to use list_nth_node().

Regarding nomenclature and my previous griping about wisdom, I was
wondering about just calling this a "partition join" like you have in
the regression test. So the GUC would be enable_partition_join, you'd
have generate_partition_join_paths(), etc. Basically just delete
"wise" throughout.

Partition-wise join is standard term used in literature and in
documentation of other popular DBMSes, so partition_wise makes more
sense. But I am fine with partition_join as well. Do you want it
partition_join or partitionjoin like enable_mergejoin/enable_hashjoin
etc.?

The elog(DEBUG3) in try_partition_wise_join() doesn't follow message
style guidelines and I think should just be removed. It was useful
for development, I'm sure, but it's time for it to go.

Done.

+ elog(ERROR, "unrecognized path node type %d", (int) nodeTag(path));

I think we should use the same formulation as elsewhere, namely
"unrecognized node type: %d". And likewise probably "unexpected join
type: %d".

Changed "unrecognized path node type" to "unrecognized node type".

"unrecognized join type: %d" seems to be used everywhere except
postgres_fdw. So, used that. Also added a cast to int similar to other
places.

partition_join_extras.sql has a bunch of whitespace damage, although
it doesn't really matter since, as you say, that's not for commit.

Right. I will remove that patch from the patch-set when those tests
are no more needed i.e. once we are done with code changes to the
patches.

Attached the updated patch-set.

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

Attachments:

pg_dp_join_patches_v35.tar.gzapplication/x-gzip; name=pg_dp_join_patches_v35.tar.gzDownload
����Y�\�s������_������[~���S7q�t��8�;w;EB�T2�o���~� A������;�{k�$ppp^�������_�(��~,�}�W���3���dv<v?��W�����l|4�M�O�''����_�I�O�r?�+_-�<U�m��z��g��R�����Q���&Rr@
�j��Lr�^��}� ����2������|zL����{����\��g�JO�'��Ipx6[��3�&���"<Y���g��x�&�J��d&����1�;D���02��0���l8������l�����0HW��g~.����O$_AS��L�N�Do||8�\�_e�??��x�����Nud��a����@�[��2X&��B�E���(��M*T��b?�o�~2���<�j(������(V"]�|)�X�'bN4�$&]� �/���]^<{u	��K�+U�U����T1+������
��2XFq�M��������!���u��(�E�A���/5��Cf=��mW���a���Xg�I��J�����#������EFd��
������ HC�+E�$�����.1=g>y*��ZB��#�$�O_A����8No���O��E�����:$z�#(J���������`.e(���v�a6u;v�A���T���W�(H�Et=�����?���L�zA��~�A&��i���;:^��)���'?nGq*���L�#?��SU������;�),��������K���vH37�=<:���~��������aWf�����������I��3�n�����\�q(3����L��Cb�����d�:���5ZE*]�K�6����~k(�����*�|���(	�"�#�V�������W�=����m��9b�ZXZ��~G��������~�^��~��`�[�wb�Ql� I�j�#�G�V}����E��;�n4#*b�Bd��kbM���"�*DT(Du{}q<��K�`p����b!��Ay�5�����DI(?�`r�8O��`��	k|rtD�s��v�o�����i��O�F<�x�O��$%��,��W��>����X
���b���$x<r���nk�����QL�V.�M��O�x��!���NO?w�(|�������y_|��"��2~���ye3��1�?7��c�buN�$����.�!�����Z���4�>^�_%EV����c������k�s�uSdz�-��xd����6GC�?+YW��b���df�2FJ�50R��Df�(A�@�eq�hQk��,e"�$�!0z�F��h%~d��
�X�*`��FJ��M!�F~(��U
�j�Q���eGIz�����F,��\���^��yU�
t�8'�	u����5����>�"�j8Dt�dA��&yxpk1�C�4�YM�@ScE��F����9!�����FV

�^�<)EV���<],`���VF5cq]����=���\W��
��]���2Z���iv�[Z*���b�N�8��������h*O��q0?����1y'�w6��<��������S��sR��2��0s�d^���r��Hm
X��1A4�y��9�����{�P�!������q���-t������e,�[��4�`�1�*�� �H&\N��uu��'+��7�����|%�>Hd����y�n�Y}������}�T����������tz��Mt��}Zw3��<K�D�D���AJ:KebE)�fe��X�y}Dy ����NB)b���F��!���WV����s+Re�lt�l+�8��
G��!�+#���B_]Q�H���X�nsi"X�
����jP�lJ�n!����@2��Z���\��*��K��\�UJ�Ti�u�7:���6�ifu�������6��:�v�������\Q��P�m�8$���:��/��EvJ,�O����D�F��4�M+�P�NS�*�J/����j�D�-����������"�&��PH]������wkF=+�}��9�h� HE72�Xx��@���Di��u�z0b�e����N����r%G��dL�+`��|��������>o5
�� L�������r�q!�j�NO1%�!����FH����JM�Hn9:�JW����4a=�PK�����;==�6*#,�
�p�%��jt� =a
28p���W&h�9m���>P`��m�n64�`~��q0���,<��l��6d��Q6G�#\���(��]cL/�E*di���K���!�)}����=��`WM% �������7Z�@���0�d�����P�#��]Sj�D~#��cj�S<O)3�b�J�I��$�q��g�.��%��F�]h���IXg2��������@:��%h���ph[���k	��������Z�V�)���9���
_D�Yq�t%
��r��f
[����f�r��5�[�g�''��px|�8��}l�Ii��7����M�d��q4%KW2��n=L��Jj������un���O��j9I��X`F����Y��w�y�$���Ee���78�/���1O��w�/��}��y��������w��3VFO;�"�z:R�����.�<{������I�����X*V�DRB��}���"%����������7��P.G�����S�g��c�}K�u��0����E.��,���4������2�^RVm���4���?]�?�E���s����b�
=l!
Y��bQ��������E��ka-�����#�?������^�ul�������<1=
����ro����y�3���Q[��-��U%-�����'G~�L5��gd��,��a�����d���������&�z����������������! �GO2k/��U|�x�7��BA'��y��CW���H~R��5��{�i)S�!x��9~Y�r�-��.a������ l@z`2���E?0�o���d�?V����
�u��k�Z��9�y
��av���}���O2�#e�y@S�8e���L8"�5�xp���'�������|�9�T-�O�uJ��72�8dtH���dk�,e��2����	������GJi���e������4�a����gPdDJ5���b5$���;��A~�W\��O5�
���
��y�O�����+���d�f^B����PSu��4�\V��
V�"n8��|KK�t!^������Z�	E,�����lJQP Q��9a���'ih�1�Tx+BDZ	�o�����;P��sB����k�|4��'�������6���g�s���1�$�	�5���i�7r���!�fz���|?h������	�XR�G�J�����)�\1 $�k��H��T4���eq���k�1����d|�&7��t���L�fj��8�h'�jw:y��DZEf/H���I�jHd���u+St��A?��(���|������������gd��m�q�[iBP��2�/M�{��M��FS�B�������� k�3K�80/?��/�!�qf��x�] D��2��@l2����)7��rv�������%�-k� ��9��/��tqW�h3E����xGx:��g	�"���Q�_f��
�)(����5-���������� H����\����[%E����^�jiM[���s�Z��)q���E9���.n���pivZ������y��u�As&T��M�!e_}���+�A�>I ��yO���UU#aZ���.`j���)�c��F��=��nt�T��8-l��c��3�tg}��%�V����S�#:1P>ri?�$��H
�a��
a2�oj�P����V(U���+��7/^SvMy�g,��,9�����Y�Pd�MSe�F����g�yGAD��-��I��K����.����/�b�jp�w�{&9��j5�F	�Cr�WMoT���\Iihe�m6�o��|�����#G��{~q�]�|���x��{�����m"?�E����tN@)� R`�m��%�wL�L����K -D�F�,W�)x!����QJ���?)����j����p��<5hu�����L�4U�?���Ee��c��sN������U2�s��f�`��R����z�&��_=J����U� ����3�VO�b��%�\xI��66x���^��'�[E_�	��!���VC
^���\�7o��x��?.�y�.�����yq�!OKr�g:~��-4�f��uP���7��I4���u���6��o@v��O��+7��*�b�g�cj��7W�S�@��@$B_oV{���F`
����iW�B:Qc����"������1z�W�t�J�,X�����o��{�R�svgQ�jgJ���I��
��<��O��{��Rw����\��1���Oi�i��g�����8��'�����A����k�r�k
��iv��Wk�'��
���o���F�����[I�������%�J8���08���Jw��i�:����19#%���d4-@`�G�$�}�9�|�����v��Kz\����Q#����L�)0����fL|�C�:��|����H�j-�q�@�1�d3.�B���;'��7}R�"\�&��3�~'�vO
�Y�U�G�(Z*[o��h�)Ee����)��as
�`{Q@����Ih��	��H%xr�*������M��#C����Vv����+�T�����vT�a��q)��p�|T6b�	b*
x�!�i�KM&�,5=��/2[��sz`kds>���d���������
�G�������
[��b����K:�P�4�i�F��'h�D�w���2�@C��Jy��u�!8@�_�d?������%R����S�����s,����.-}���e���Q����x=�Y -\%j���4���]���A(�����'�2�G��C��2���(�9��TW����|��V�l���
I��L}i� �����^����z�:v&���2M}������i���G_�v��?w����=�*���x<��g'�0����P���m�����7����W�,a�_� b�^�.��zT��k6~��y�E<N�G��x��lS{�������m{���"����X����:�����9��S�k��i��/cu�!q����������[�7�����Q�m����OW�O���"K�������|(��M�D��c=�$�� �\�2�r�������k9����y��5Vg�|Wo4����B�2��Oh{tl���Y�kBj��	���]�T�5�L�m��A��	9��f7��9Q������u/v{_�mo��m3�z	@��|���{+��������%���e��^���x�Z/������'����������=�{���dJ��.S)k�=�;�DFH�2���0��:n�C\��������C�m�p7C��!J��s���6�-���I*�(�G�?���g ��� 6��	q������Uhi��Lj�v����~�����$���zC�5����^/�d��
ET{����S3N�fH#���)_R����$�/�Y["����-Yp����$����z����8���%�.{0����]G�!iv�U�	b�-�_�!���
��H"����yfYo�{n������-wO�_8az�	+��I��5���S,R}?�k������0�AIn2�{�4����y!����P6��x���:�Mu��p:l��6.'��B5��$���%�������@�Y���w��s��B3��
�S���<��A:���f�<�W�U�n3��V�.o(�MXfm�X4ZW�2��EU�I�
������?�i5jl4��Nc}3�m����I[p/��x��������[�7i��d�6M��wi�����j����&��W��V]��-*>/fom9>��w&�0[�me*`����C���V9��_��p����V��:�\Y*'A�6���.����"2q�v
�"���5������z��H�DM�Wu���4}L�\�!��[���=��2-]�4�v�%��7����b�6�p���wo������WL�}Z�%����N�8r����WV������ ��Tx��6��s�~fP���!���9s��+��leK{��tA�j�;��m���|��a�Qo���;d��`4sD����-!fx}��P
@XeV+�f��0�{N�F@���+;��?�3r�K�Dc�����i���e
O?��l�p:&����$9��������p����T&��D�K�5z��]����P�A��C�13#�s�C/��C�*���[C�R���c#�B���;��T�Y	�l	AIn��,�mx�c�G�M��0(�!%"�
>Itw����SKX'D�ay�����.5�����1e(�\t|5H����27}��s'���0�M�SJ���j����,�i>/<��1�����qf�U<�P��Nb�j����FN��8&��1&C�G����8�.�6�����>����
4��i��M�(���M���0���;�1�������4�dd@8����b���,vl��9�sdnhYi�����f��C�:�V*�O�5N�!
y-,%�w	�0��;���SdOJ�A^��
u�,$-��J{���ON[��0�a�?�%��g����;7*
��`��%v�"��$�AC
��HN��Y�i��(j����G$�ZhM�">6����\���L�T%�?���Vb;������c..�JRYJ�O��c�fl�L�B��g�
���#��3�$y�hs��"�Z��t2��V����a
h�Hf�o�V&��a�dbm�^�k�Px�/2�|����Z��L�`J�nrk�A��?yrK$�&$h���^,@0�����jVg-�Bw�R�Q.�1")����,��\�;I�2o��1����T���c���V�J�{Z6��/�����Vy�;��Y{���7+'*��Y���,B��b|�>v���|�^,�I8����ba�g��$*#��n�J����$�[��|�������rfc��;b���^\���]C r#��s0�0Y,��5*D"^wi�������m���W���> �r����`��������2�>L�]�gc����-�9El�����[�����88�=�;6dnn���6	p5�M��cB��ZY�W�����p���QQ�i�D�*���[PA�e����<��I�~�(�������3����R�9i�^1�
�-��)�c�Vc��-��_��)_K�m�,(*�,ue��H�����������!� #���Dv�Y3�4��
�$���d��>KT�t�@������{9�0DW��wP�1�xf�����~��0�3��%nZ24`JGt2JGc���|��P����l5_p���	��� >���$��&^�Q)��	�G;��8k���6���l>���������OL
�����s��H]98��$�7}�
]0��s_%Z�-��3bz�~����"���E�N�b���Qy��rS��r!����"�F����W8o�0�C�p�I��������~�a�2/K�E0������d��LA5�`�D��3����b���.�uW�8�wR��"3��AafV��/� 
�d����Y�OV�PYM&Z-�A>��AW��y.����(O\'9dp����x~aP��e�2�v<��0U�	 f���x����g8�\���\����V�������� ���v��,|�N�����J�?h;��Dz*�m�����Cm�X�;ic���WU������u))b,2�z/Y��Vpe_�bU��#1�q�>b��IP9���;�q/-�;R�V������
J[�f����7�A5��{=C�)�����e�A��[�s.��%�8�Z��n���[)�;w3g�I��Q>����z�RG����IK�J�Cw��8=�c*��{E���9>aM�U*�1?)���tB	�|�@g��Txf��5v��P^Duu���x;���s�������k��_
���X�1�`R/�'��.�B����[������d�x��mNb�?�#WN���D�Z�)�WX�!A� �7��.���6����i����2A�)� �fL�.�8LOI����HhXS���!K8O�o��aH��"`�K�T1�����g39����(����;N���_^��q7(��UU���Y���'�$�!���"��H��Vq�,��O-��A6�x��
$����{���B�0�!�7�cE��O��Y
���Y\�X54j�>��r3��vJ#,v��1�B��I4���pM�7I�!�F�K�����E�'[�$TY���$H}OQ�OV7�C��@�Nh"��$��IX�C��V��om$����"-r�B���OS�!$T=�9{��"$Tx�Y����z��s��y���
������D�x�:��Rr!�3��V���B�����0s��I����Z�j�I/�����0�0�����������	\�6�^<2\
K������s�L�/��Y(6m�/����)�=����-p����Y�������!@.<	��D����cs���fa#�,	���Q���1!�B��������/?r~�8\�R�
R
��	��	�5���������i"�<I��k*$[�@z]=*�$��@9��f	���MRw��J�n�/<�*��]4����+��jW�]�4[c��x����W�OdYJ)�j�0{*3��bg��w�V�%5;�4?�a�s���EDN����2�P��q���3�<�F�C�%�K�����;v|���u}�������qQ���?��i����QY�6���5	�V9�gc�7p����A~�����>6ZmHj5�):�IQ������o�X~�Z������!kekY��8���/�����Vd����$AQ�;��A��+
U>��8���1������3@X����0��&�����vB�\p���x��:��My�t�5'iL1���d��^9[�v�]����u�%�����
V���v��9�������"������A��&�f�!Xk3u���5�1�:9�G������Az�X�5��fYB�����P��������K*u�w~-��f4f��B�y���-�w5,I��m�A�@s�^{-�j�B�Ev��b?����(����@+N��
W���p��(�4�K����H��>���(A��%��|��R�
ud���KM �<��\�v�KU;~p4X����b0d��0%&0\��q���+s5�64��"�^�H^0����-�������|��}!�kb]�W��uOhW��]�z�����'+����������l�4��KZ�,��N Fb&�Qp�����v�l�< �����cXh�ju��HA�H���8���E�����L���=��e����3����T��[4O(��Au>�&7,���?4m�����E�.R3�Ht �4]��o�
����]*�/]*b��<i��������oh�zN&r@"d��HP�����s�d:2#��jN��yl8������0��^@�&c�4���\AjX:���N`���Hz����3n�. k�h� t|��4`P����v��_�u�qcE�L�{�[�U�&�k���b'!�5pu���NZ�_��-5�t��Z�EqU��T���yqe���::H_-������ �!�p�!d�(�h���a~t���_��Q�0��dE���,J���Sp����������!�8��I����r��j���;�h�aB��\L�N�����Hv�A��!b�*�c_�G��1�I�7��F��xp2M�h	7��1�4�A��:�!�E#T1�U%�\3CW�X;�.��1=W>s���\��Q��������K��7�	�vd���z�
&{�:�Z/������M��Lr��5����z���4���"�SJ��7A��#2-��C
�r1=�By(sM9�D�$���.�IY�Iab�VX�y(�S*����A`�oI(��vH$�^��f��,���D��a��*@���������������\?�Km/nx�D��\,zR���C���U�v�%��G2c�e1@G
5x�G��c�v�P�Vs+��2'�
Q[��u���s��i��B��	�X�"�J�.��G�������G���*]>����d���_�ZSF&3�d���b��������B�x�D��=��se��������,�xt����(�W�s��F����Oj��Kp1��� rf�%p��)#k�c���������&��
\�,����GQM
��M���J��n�vV
�B����#`�in�5]�@�9�����d�q>������.��I�*Mq}"�!�nB�q����N���c� Tl�8/U��|�T6�t(6{��
�(s"V�R6�6��m�%t�����T9uT�����0J��D�Rd#Z���J������IV��[�?�8��Z��$�7�1E�!e(0=_Lr�I2)�&k(�f��|>�\d���u���/oi�l����w��a�R��ldH���i���w��L�Y������MFR�
>��o��hn/!"
��+Vea�5	:Y^�Li�G�'gI���,��j�MV��#���%�iDt��19�	4!��L|��p~Jv�V��4L!X�`���z�d���5��dk��K����&.�8�;W���5�h�`����%l �f�a�����@@A�5�1`���}�����\h%0=�~�d�k�������+L|�:��B��� �u���"��Q�E�]���*���Rrgs���m2���^�~�15��*L�nY/�g�UP��bV9|��;�!�������F��	a�*��
y�s?J��l���q��M�U(��e�B����N�L��*��`e�AvX� ��d~c��*����'F5���us���J_+�}�>;���RvG�����!^0&�����j�'P�`��,����*E[��-��P���Z7*�������|�P�&�L�Z���{�'�k����<ne8% !���Y�m��)5,�yg�4y�4U�<��p u�v�y�Gp���E���2�^�����9Q�&"a~��4� �I����#������
�m��U�t���\b��/��F"}>9�x(>��hu����C���p#z<\�a6��A*A���v�KE�B�&�b�	�\9�����Bt�mL�VH}���{xx�`�z�����������^��&K1�&��:�h�h���FB��:##�Z��Q'3�3�q
y���v�2�}��B�I�� -4("z��3��6�|��ts.�0Wo3�N1\����	r�c0���d�`��������z���0I;I���p�$�l��j�<c,������#���!��\�
�6_���<@n00rs�}�Z?�@������\Hv/P x���E���J�T�F��pI�G.�o"�!K���a
Z��?�/�}��~M������
�:�.&W��������>���L���`�����9����R^5#����N�{��h8 �.~�E�c�
fs���'�mau!�������~�Czr�|W�6t��P-�J�\Y����
��6jx���|2���0.c�8��d�|�+��������!����4��i���
�m�	{�2�����9"�X[L�B,ty-������T����5���=�g�:����m���
|�k���BV���B��ly�0("X�W�@���(0��%6��kc�t�7M	8�_�N����u���3�$�*I�H#�*�9���?���0��&���+�f4_��������m�Z�����D�+~T*Kv���lDco0���_X�1�CR�L+]E��C��n�)g"�U�$�Fa����7w5���}����C�Y'b'a����-��6f��!�y�hc�!Q������9aH�tAO��"�`[�&`d��re��n`���\vx�
{ov��7�qfU�`�dm�t�%}%��*O`��R[���H_K*�%k�B�:�rtx@�R�fkA����� O%�t�q(@}f�NE���pO��#�6���;�cK,��)�K���yop�i�^
SE���"��7Z0]��`�z�_	{�V��1���������7�6�����2s��X����%C�����u#L���&`�������#��j�0E�}N������mE�c�Xc��KawU^h�f��h[�b�_��2)��)r��UFI��cUE�|��L�	,��I���$�ZB�|&$�F\a�A�Kys/���U�c��������D�(��g�_���G2R��-�V �
�
Od*���e�<�\�~l
&j��}�>S�
��JY�m�y���X)A�rayj�s�#�����CZ�O�;=��
<�Ias�A�}�� x�{������;-LR��E�����/����Tb@�H��J��}fq�	��$0K>�["�q�l�"��/�
$�yX�#aEH������F�s���]z�1N�g_S�<���JH�.hBQ���>c)�e���?z�~�%�����HM�N����AyR`�\�<E-��g����?��W	*(�F6�������p�4Y�"��^�;�<�$�c���2�y�6�&SDBs��G����;����l�����R���V�*`�r�0)}U)�U�5x`�&�����D���/h�k��',IO��F���1������+�A��EFI��j��`��|����l8��`k�P�Kg�"���15p|,I�@�w�����[%$��;a�(���s:����Lt���v��n�@7�VR���9K<a�E�(�E7N5B�J����w�N�t���q����g7����9K	����Ib�%:!���Q;����g��Z'�����/��C'�����Gm�k�|�^@1���H=�p���g'���(��$����V���K�		j�8�V��*�,�7�����).������A����e�OZ~Ox["�]��<�*d��XN��9����@'<�6 �y9A;��y�1�����==�6	2�,c�I@S��;��
��>�$�i�dt�0���L�|�
&��%z�?&�4� ��dr��R3��$��u�]�H���]OF�r����"2{H�}���J��b���H����K�������<�����!�����'���#���|���T�!P��d��"K�^/��:�+��*�@�G'�E���oCu����
f�.���a��O�kP��=5��E��]�I[��[�K��*'�tEP\<a�w���R�/�.���&'�b�kXI��:]���Ny�Ddk}�9J���������������K
���xrt��I|�)fh���p&����|Lh.��c�� �����Gt���Z��6{s����)a����9t!�*�����?f�E�S�s;�A0KhQ���+N��9� �����d�f���,E���fYJ�	���eA;LMm��2B�{�$M�	s}E����U{�P?)�8gc���j���4f��l��UQ��,����-1b������#��$��ZB��s�!:������a��gJ���+������
��H��^9 R���fu�R�|�.�`�z��gL�9_�./���N��Q�Q��M���M�E~���+6o���"��f���t"xgS�fW`X���t|CQB�"ZMn@9� \�sC��a�"��3�8�=+(�Y�8� �~�Nz�����*�&n!�.3�x��D�Fh.��H�_�&4�9>�nN����F(�W�&K�._���|�����!eh@���j=�B���Y���8�����gy����2Vv�����{�
n�+���{9��|8����E��������S�yLDLw��� �/�+t8����+�����W�_>��}8=�������_����{����O��?���"��|���|~�����J����������t��?��@�&��5�]�qs����U�����?����� ���/WW���m�3��� �������/�������h�u���Sc!y�!Q%�i���cW���2�m	��K�����x��\������lP����5B{��+���<��~o-��<#��8��F�{����t{���1����i�V g�[&1=&{n��M8����7��|2���� e��x��X�C������z�e�-�>B���"Q�N�\���F�����<:T&?gJ�[\;5&���)�n���( �Ms:Tr~�|0��wha9�������I��@�<�2��*<Y1?3&��TZ�K��`V�L�O��H8~B�p���f�@=���1y���Yv3S����j���hz9jc��Pix��M�
,�����s��~GI�*-Sh�c��c�/&���V{�u���V��.W�-x������W�H=?)���Bx��3K=���&�@r��u)�`O$W�,������n�wXx1�'"{�;�eaE�d�[C������*�=m<J�j.��P�Cm���V���B~��S��W�N=��
�Hj�5�9�.dJ�`2�OE ����0�_�p����A}d>����$@<cZB�J��em-�)��`���L��D�jAB���}&q�y"�����������
�DI�L�=�l�Z��,���[e�S?�<y����[��,fgW�2�}t����C�2p�h�F1_���1oYo�&�
���P���������/�6�}�|u�|L%�[��
l�cdy�O��H�V����%f�XL~�:��V�+��S��=b	������qO\������9�RA�o�ip��QG��fY��Y�~w�i�������*OhL��N���d��nE�����e���OO�����+=�f��A����,1���q�����_eW�4���7���Em��n��������U��7�������-�������r�9��)��0PQ��Q3�5�2�Oy�����k*�i���Sn�$�(�x%�Ob���:i�A���<1I���B�w��� I1	*��}���[�w'��%1���g��!>�7�������,��$
E�{\��������Q� &b�w����JRYXE5�!v22Zr��_���I��
Q��l�HUP����p)���%�������T�a�du����3"�+�����y2�
���o����?U[c�����3N����+��4�6�B��9{*�{�����|�������&D���s;��a�*����xQ���!�Q+6mHM�4�2,x��,��@i�!%�0��U�b�����'�w���b��,��3$15HK������_�+,1#;��5����(��������$�K�
la�U�`�C����M���|1�y���b&,&�9��B2z����x��o1}
�q/vvb+Yx����&3W��w<J�h}�2�5 ��4���O�������"d�d�m���"l���0�<:���������S*)Z>�v;v�K�����zb�\	�s/���G�������K���R�����.&�A�.0F%ShT
v]�������&��@��K6�@,��������������R��wFI����y�;"�� q���z��u� ����TDu~�/~��� ��{pH�	do�`�+�HPt��`�l:MV7
��,�8��:�1�4�9�q)�H��#VN��mN�����"���%P���~�������������>��|���hw4�G���3'UV�z�� �%�"�����hy��;@�@	jC=���m�|��d}^������Go�2����j�Q���n����d�_�������q�x\e�'�=�7�O�9M����G�b�?��j��VTK�m?��������pt��<_VO���J����DG1A��r���nH��Pu	U8����]��x����3�bH����e�(�c�b�������}��=yu�����c<|��|~X ��i3�������Q�rty�������hB�&�>�Z���.�X?W4���|�G����1�1ZP�!"�����F�	�5���*����A��K���%��#��@|v��zW�>[��c_>�AT�-?/?N�O�}�.�rDzp4:<J11��<�t�aUI�(L#�u��������91.Y�V�o��[�,
�_�d��0q�8\!P��9SP�<ec|�SG�E�1��6�kf\��|L=���LM�a��i��b
*P��YR�������5C!�����OO_��!~����CT����v��`����n���A�S6�i�V5��m�������h}H��w�+�;�W��\���O�t>�u}��D���VD�[�7Q��z	��h��"��VH_�]�*�K��^*��]���J���+!�UB_Bo	��F	}9�%�Eu����U�e�����u���\%txPK�z)�
DqKNr�e���j&��3�������*O����%"��8�@-7�E�7_��Y���>����+�l��I�A�4���}�5Za\�������vv������]�
�r|�/�u�����j`�=`<�0��/�I��6p����>�{���z�7�n?��^�;�v��a��I�O�@��Y������x��/�C+{�H��p�b�c�fG����(�g��qvq�;L�������.�=�>�?���w������(����<G'l��w�g>g�#���9���]`�r1��o���>����!����B$40��|�}�O�nw�vw>�G��s�_?����
��7��D@wv�=��VTj�(��/@�@��M5!.�1�iX������� ���q��sV���
��Qd��%t+h90�wv��fpnf�G���@h��������(gI$5���L�j���=���=��"LJ��3�7Z�&��3b�v����m��%��>^-L�)���}���i�;��Al��1������,���\U������&��p�����t�v;d���-�&��C���-����G���;r����������[�.�V��4����4I�0:�������<>T0Q�ETP-0� ������SF
�����&�!PDb�R�}Z��q+�"�Z;"b'�,lM�a������%)�����D
WY<���L���J<v\]?Rg���31����Q)(�vM�m���/�!�v�^���U���6�]sX<����H�7�a��������W����Fr�/(���IQ-1!�|��-��8'w+�#I*�����������'XC��1��6'S�S����X��� �%u�
A:�����.k4v����&P�0~v����Y����X������ �c����4�}�YP�D��j�%� n��r����<�rK���Ia��Yv9���l4�&�b2����2���f�B�0���-n��|��]��D�����>��JAv��U:�Y���wg�d� ��&[dL>8>��k����,4���~r�?�_�>.�cyZ����d�Z�7:@Fy������R>5�;u��|�����5h+0BR1�����n����������k��2����f��lp�w�l??H���R�7V@�L�E�N(9&i��=9��R���5]���n)X�!2R]�����:�����$`y�A^�g�M.^�	�����rO���qY��[*c�3�-��[@�g	/	���9��~F�CR�����(_0e��P��=T;��H���$��K�I�UpG�8/KL�ux���Ay=��d�YQ��� nc�p��$`Z��\:�	�	-������}�����(Q����h]'1B���VHM�	B("A�ai3M�[.������-3���&�oj�D�,�����[e�q1�_Z�8���\vD�z��%��3�::FG�Z�D�(�����L�D����n�Uw����z$I=�|=�����-fo�.������(��[��A|t��\F/�5��s��D�x�D���c�2���#D���Z�+�A����Ax��O�����3T��mb�S�����k\���_w�S�>;_�9lQV����3���l�"��9UpN�P�	{��f�t�<�[���w�J��y<�ft�����Au ��-e}�N����<��;�c�����
���?`8����kc�P�.)��,�{A��<�w��?�fl�7�n�f*B&L	H�&�8OL}4~M
Wx��:��<�vz��IA�D,���*������_����?��^)�	� 9�i����Ta���,��e_ea����JS��_2��,���%�/WJ�N��R��,M�L4��br�m�;n���x������~.�3��V��j-�M���D���s���p��5M�GC�-�3J���Li>��0=z�\1wR#���6*��%tjA�>�)l��vvEV�W�S����t|1���87L�Wh�F��-r��2��Vf��'���p�r>�����P ��'�3�ZWgC��H�����$������DlADu74��m�EPK�K\�����&��!,�Nx��!X%w��4�1y*?�z���a2J�����"X�D%c<O��V���fIV��QM���[*�v�����dh�D[���1�!	����q�,��gJr\��f�Y*\��@��*�:�[�k��l�#0q��
Kd�:-z\�;�@��"2�Q$9VM�<���h@��QRCD8�4��,`&q5ayN�F���V�����C�����c�R-���UE������?��f�4�<�:��U��%KbDV����?J�F��j�N�Ew�E�e]d����T�4��J�����qr�S�vN��&!j0���������R��2�a{��0VAv��t�0�;>�$]��8a�e�61y���G�O(9\|���u�`�Q�H�"��}�&�v5�����R�\Q�T�G4���l@"ep3��N�\	�F����f7��)�2$!
��f�$��	��H����|D'����<����������^�Xo�T�{�7v�7t�7s�7u�7t�7t�+��d�����+�&N���\�����z�+��:���� ���8@�j>�@�����@�	��3	o�$z��Kx�!��7?1���(�����'�4�1�����p�����(D~j�T$?5�M�����c������	��e7�������T���^��Kx�k�Uur�����d-��������������d�J�]�Wo���������o$��>NSW�}K��M�`S�1��)����?��B�w>����i?�AV>|�w@��:6�K�E��n|r�d�+�#,l�D�T��r���g�������W2����R
?>��YG'��L��Aq�h������8��, D������V5:$��%M�
�BJ<��'��!�.YzB�H�f_�)�8�P��������9��v1���Uo�Ps+�$�D	�h�����4�������n\���Kmg��0*P�)������R������������<���W��� �/��E4����������:4�w|��%
�s��Y�T�E�0�s��7�?�����k4yu;q��d�y/]����a&x\��R�8���;Pd?x���$��5�Fj3�Ue���1b�<
!~�#6u�`�@@�g}4t_�.�}�)��<12��.'c�r|<�9O�L�--y�|F�'�P��5���T�%���lB��"���m_N���DmI|���N�"���2p��������rm�h������/�����8h766PX�G�~86��wp�`L�,E����x��x���g�P���7��"��o9h������tv��|�a�BvU�b���o����'�dv��'�OS&k�>��������,�����e
���t�c�hND�.�#gt����y,��x��L��Sm�Q���n�^x��E���;�'{������E��������������%��9W|����=����������H}��p �S������^��\�	�3%�Y��l�Y|K��3E��$QP�@�9�qSa?v1����~D<9L��9M�Dj=L[� �P�(W��i��<G�\*����r(x���������Ku<JM����% 0�#����xW��rW�,h�Qb�$o�*���1'��o��MR�#��@e���,���������d~D��I��/�R0��]������ &�4S��#d�C�K�X.����*>���P�/��|�w��c�iC���";����tO�`����,<����=7A[�5�������EVW���Z)(�B���p�._�,(�:v}��"��o*���d��F�EP�����\u��Y�o�[�5�	���NP�>�aEs7��*���)����o������O`k��\���o��FP����W��������:����Nb�����"��5��4|��7�M���1[^�W^�Jx4��d�^o����]�����b�����h
�*�Hq�^��!�����L��%���8��&J�
��JL���t��8gH�#uQEnD�A������%���`qB��&�3�����Vw���h�d��Z����Tf7���&h�X-k/�/�(R�;���,{�YCH��U"HTB	�2����iw�}�.���:#�+��u#�������hh$,mN�g5r���u�b����Zl�}sU��W�b��x��Y��f��*x>�����L!f����FcY�5��U9����a=��z�"5�i"E�+vB�@�HrB�����1�����4�����&c�J����
�re
fn5�y%K�%�@A�3�1�v��D�r�����<0��
�'n�+��������^h��}���$DZ���e@M��NM�
F�\�O�
&�M�������G����$�>�z��z��#b��e���
O�
k\3��e���=�	���M�#<Wyi������<'!����W����kH�K�|�"nx%�1����(�@$�%s#Q�����?$D��0+*�8�&D5V�����p�������Z1�l�R��I�!BV��\gS��/x>J���oH|*����QN���0�3<bg~m������cG�?������ff�����t2��e��-�>/YT1�����!�33�g�u�)9X���7�b�7�|�����43�}E�������������)��D�`�K�#2�����k��[�F��w���i�A��C��$����4y/�\��q��A6|9��S/$����<���3������%]bV��T��!�c�)���s4��W���M����-Z�/�xBW4 ���/���W����z��+����M��V���>O1���{�1!Iac�Y�W���X���;�S0JU�/�7��V�b�w>o��������W�����D�a�k����2��1�,�`���7����q	��	~f�lV���Z�E��P�8�6&�p�3H�V}�A8���S�w�sO`�/�9��~��c�����wL�iz�� 2nz�wf����\�X�H��VB�I�'8'��Pd@�Sj
���xJ\�l�D��:�wU3]u��}�*��K�u��R�qP�?B��f�%1��'-W1
�t����7%���*������ ��`Lx$bl�`cG�bP�w�+t���5OSv������~�O�i��
�\V�_DC��X�J����qD��BmT���C2���y�T���<$>d��Z[AF#z���B���2q>��������
z��{B���M�-��G&VuE�3���$��C�/>��z<
.K�Ja������������ �0H�R�#IS}=Y}�!��l>wx����k���8%�w�N��9�((u��+����?P�*�WDs�����)A�l�!\����2�d5���	sTR�����������~z0>��G6p�9>�9>p�������(���?K�g>$������E1/k��Gm��������{����j��������R�\����q~�;�6��r@r��M���^Q�3��������E�I����#C�v�I��e$��$�����nS�m*��]�Ws�*����o�Lt����;����P��{P`��:���0�G��z�f���^���..���r�v����ar��zyz�O��4;����G�����?��P���u�A��u>b�?z�����'��K�a)(|��� E"a��������)�l�$6�x-��
x�}���$�(�6_� i����4N:�;=S_�,l� e������X�� B�Drw�7c�� |�;:�{c�;��.�$���z������_.�z�������x����
�R=��}3�zO�T���"70��L����
��0��G�(�N���Gi�������hd�|���Rj����i�BL�{/I�v;y�=�\�,��~���"��_O/s|����O�/��+ �����}����{������G���[6�������k/!��������St~���Sty�X�P+��4��1��-0�/������7�o��C������������oq=o����+������pzv���;/+�p����9�t��$�l�hM�
����Od�������dZ�%����E��?m4q����=�2�cY?���g�o~x���������5�k\����/�h�n��A� ������!�IxwH�����Z�L{��x���i������>�����S�1�O�e�K���"t�;H*�;��Tt0!�8yw���������Ga?J\������	�����Pb2)iLS7�&M�iZ��L���4���L�SG�������mN(c�N��Q��z��7�1�������;�z����<��m@�d�W����J�G�U�?�������wx\��M�?��|�^"��m���~����������S��'���.z�=���^'��/���������_!Y��J��c�����a�X�!�������R�|����o�YV�2!(�@l�������Q�u�M'�}yF�cP]/����K�*�vS�I�����'��i�4��3B����c	v��{d�������G������?J���?�G�.����GGxF�rF0n�L�L;J�.���S�h<���<K$�d�=}}�V��F�H�e���Vo;&����#�[�S����DAE�\�9<������s�++e�|J}`�m���������t��.�[(!{��������,t$���wI���z>���ciEP�a:@9h3U�n
���Y��A�.��&po�������5���D<H���e�_C��n�����WM�?����\�.����f���\���6��@	��`@Jh������`#%��QG�7�k��!���6��$�#8���x�	d�.H����M)�O�0���������7�C�$Axj�Jj�4�J\�T����N�E/�N}�G�*��������������e>><;k+�;�����>��}@����\[6� O	�������
h��N��X�B���x+&~b���x+(~���/BT��z�J�f��P�.��2����h'�niU|,��*H5������?��O��K��g��g�t3������v�������W0d���G�6��g�R������;%s����S�>#�f��%|I����^�/DSj/��d	�<��of��Fl5�~8��RZ�]KR��7z�IB��(�$_(��J
B_��1��6!������w������f������Y+����T-��r�Bb��9o��{'5[J��k�W���t�cC���H���I]HH}��7����4k��\�/���RZN���"� ���X"�2JI�bA��r�Jq�H�X\E����s!@"��d��^�7������������P���7"�UV��Y���51�q�d��d��'�M��9t�CFk�������4���}#��9L�G��HZ���7�V�s��7�y3a�����u��t5�*�O��� ����M�(3�Tg��l^Qa�����
Q�F���P��(�.���P��a���R�G�`*+��zO
�RE�J/?��7%��h><8�����
�I1^.�v�='��3'�$qGl�p�f��!L��k������_@�S��V�K��<{B��I��I�����Od���O�IJ��]^y��>��|*��i����O��E���������Z~��l��8����yR��Fgo~�1h��$��W�yEC���Q��M�UQG���^qd�"Sr�����&n>
�3Rxp��	�J�/�[�������pf�uDVO�t�� �s�m�OZ[E���+2�!���	��c��H;��M��7�f�I�7����~3�oZG�q��N9!
Y;�,�'=�����y����RtV�']K�����4Con�ey���5L)�/*����� �>���P8�����*MH�L����w9!U�s�	I�	I�'$��	�5���V��:3j��A��h�e(�P�=z����S������^�92%z���+��S�<�}��u���-?��h��^7���6w.������A�o��lO���8�.V� �q��Tw����9���j�@�q-p�������Nh��b|
�����3q�	��=�,��c��`��	.�_�<�K�)@6�������%J���&�9M"��+�#<s�:�4����M�x|M�4$��/&��/�v1~n��V9�[��2��#�Pg��G��tbMW�L���</��AQVx�U�h�	�*{�����^exA�{h����+=��}/8��^oj���O]�!��{�9d�zSSW���	y>y�<f,��E�S�����l���b�	��?��6+����f���Tv���9Xs_]:��+������h`����t��[Je�i)��������|��G�`@��]J
	�JI�AQ))<��L�����5X@k�%�/w���L!.@������������h	W�^�mwS6�h�����@��	V�)4���[�6YQvB����n���@U��z��<�gk,?�Z�x��x6������!R1�>��4����G�%�������,�;������l��]SR`�`�	��5IU(b�c�T%U�<������J��w��;W���+q*�f��.f5+[�m�6�w����]^��| ��
�id�������t�e�wq3Y����de}��<��Q*����8��LZ3���`�P����,V�+<4��u��v���m�tk���C~3�2��Fc�T�M��opwBF����C$LX�_dpmj�ez�v��7t2J,�+�ZoF^n�]<)���n��NTD}zTs���)4���� ��5IUl�=�vw�R2�n�O
��@�62
�%��.������U*�������I�r}CpG����W�O�t$�����{��������^�9u
v�K�{�g!����m�Fw�I����B�����
�F.��=m&������9�?���]��]�Eoq����j�k\���te/�^jn���8e�q)'c!�H�O�zB'�����?R�c�g�Uk�"�e�M_�Du�!���c0�\��$��\���
�o���|p��5�����X��)�Q����M?;����Ba���oY�����V`��]��6�n��t�����B]"kW9����zz�!�z���s}c+X%P����V���,���Z��8f_��A�
Fwc�����/Gu�e�z:s�t�M�_�1������(�M���4(=�l�*���*�8S&g�X4���xs�o�)��kJdM��O:(�y�9M�g���������O:{)�{�9��	OzMY���'M*X�w�������a�W	I��7�7�|o��� �E�y6����E��Nt��HJ1�@�KP�^;	�u�a�������Wx�0i���v���M���'oO?�:mQ��$SV@sF0�Q�o�	�e0��YtL�s�b�Q����S+���B1eZu�\����(u�W{��4]����v�B�G@.�9��Z��G5x��=�w��)D�'"�#r��O���(��:�����)��
-J����&�UY�${*s;�.�9�����.��%mw(��Z7\9w�Ev�TV�������$�.|����!v�"���_���9t��?�L��9��W��U�Uj~��_����3@q����|���a%����
+���C0;a�YWY�*�������J���������g� V��Dx�,a$�Z�'L'7�b'��k�W?�mDo��!��f
=������bv��@K�����y�pw����a�����0�31�0
F`��D�u��C���Bd�R�X�|4�9�@����m�(��{c�
����kw��n��4�K���4aN������n�S��K�G�x����>�9��>71�i��
���P��*�%��@�������<�6M}��j������J���x���
�a������
Wcx"M:I�2�����G�~{�x�J���]Zt���*�g��k�&*�����p��j@l��\�W"~��
@�H I8��������h���J6���$8�8���	.W-�	�`��F#| g��&��/4������3v���2�Jc)��2Ej����<�����I��"4^�����$�Z�������m�[]%������C�W
�Po�p+c�;%��L��Gw�p%W�]��	B�����%�\�$<�{���K@�k3�9<d[)�U e(WMm"�D�pb��7�t#�66�VJUH[�����E8a�Ny1����IEz�ik��h���O"�����z�6s8g���M:\8�[_��:�� ��t�}��CI������(�tu��Vn��,hG�V���4��&�x����/�"�c��
3��0��S�9`~\�Q�f����!D��^~���T�GA��aU��V���S�\��k`��=�U�M��2�X��CY�J2K�r%:�����W��6��hcs����4T��hN�-���Q�QT�=b2�e�H@�>#����1���=�t����V�s	C�!��${`�(^���O�{�(o_���p���
n��;<�XNC��J���0�j0��.�
vl9R�L�eh/�G ��'R���T�Z6��a��Ubw`<L�����. �x*�"Z%�=]�����p5�:1�:��������:�������u`~	�ZlXcz��+�����i�S�S
�J�7&X�N�N�`�in�`��C�������
b���u����~�g{������W��TF�O�Q�YU]$ ��a���_������*���q��{�/��`|��������(JK`@w)�:E��4�8�eL�_7!
��L�M���O�71�=��}����oy����3#��hi+�����>�WG��������V�&�nE�q�k)�a���6�o���8�WILK$g��6=���p��=��3#L��GN�s��@_��o��
�/&���7C�����8��
��,���9������r�Go��2�B��T�<$����n��S���r�S���(?�4{mj�~#Rn��������^������w{PY�Fa5c�����{#t\1%\���-V	���9��R����;�Y1Y[�~�M~[�����w�����5F@�XHX;��vk���s��j��G�������|��)I�9se>d�^�'���D7��*'��0Y�(�������p��//_h/g��)����_"�D��'�p~"���qJ��s�/����`��^��hO��
����0����D��f����QL�S��Mz���P�������+��a8(U�BT��
09��z�TL�S���N���E��O�LI00UL�T�
09��z�T,=&"#�)���AZ:���,��nxpB��l�^~�����(�)sq�|f`���V�����e!�������lu����\[�����tQ�vb��XK$6�A0*o���p���}����w_X����Cs�����(��^w��J��j)����L�������6Tq��j�s�������/�Hy@�������@��wKz���*[b��8�/~CU;�D��A�����*������ ���A�{��(�� ~W���L�mK��?�0��6��s���,�&�S��0���V7{��$�D�/;<��Ru���$�9�zbdL%�f��E,=�r���Fg��q�o_�g����&������*�z�E�����B�������u�#�h���<H����Y;��A�H�B�����P'"�;Q�'�sqf�(��v�LIx������:��/��\����$�o���8��TI7��o�*��t���t�����x��fM�qgMr�(���p:��?���?�Q���'�� _&������<E7�,>��)���I���T_���P��*q/�����J�,�+��M�4F������:�;������DgfYH����iW��Tp�
[����^�_t��������w��<i���3o�z�_��A�nW��t(Pj
K�Ck(���w��!��5Zu�c�m��4'�4'�iN���T1��4G�N�=��1��6��w�Sm�S�4�Q���Bmh���^�U�Q~��R+��L��F�v��#7V�(�8�s��J�������<����Q�q���,I�g�����k��$_`��D���N�+��X���w�S;QJru�T��7�[c���R��N�A�
m@���O;�)�����}����E=��X}vO���W�I�z�=`����o��E�B
=������5y����?����8	\�cI�K\��_���`�z�,q-����KX0����Jn����y��+g�c��NW��M�OO�:x�)����]�{:��z�����|��������������+��������l5�� R��X�|f������?��d*z-�~ki��<m���m�r�f�GH8�)@A
��Q��p�����q�7���`n3�����������������0Hl����>N��tM.r�_����I�U��<l��$d0�>NPd
��iD�!},�R�$��j�}���*-}�A��.>_�a�Y�*���!J�T!T��&U��#���f� 8��>�����,�R[�����r�({l=k7�����:��<��w��<F�#�s�v��#�������q�[wO��n@$=w���9_���A��;z�~{��[w*oP7�����I��������A��t��;�]s2��=�7�|�Rw�=�����y����1u�;������Cy������`�'G����
���������[���A�� is�{���^�WwO����vwp����w_�z�u�l�0���>]����KH��4[��Kt���g(��]}���d��"�oq����#QP���qe�;����j��*7I��!c1LS���_��ukY��98�X��Z�>�[w���e�w�92�f�z��-*�lH�������,$W�Uz��R�>/<�jymVh����s�l��q�3����i�1��R�����������n�<����m�7u�y	#/oM.C<GM�i�C��\�%�����v����P������E��F���cTS4�����4�6�����p���:�8���0@p9NU1�S�_�w�����4��
�G�VH��B��
���m�����[!�n��[!qo�,�o'�J�[!���V�UW�e+�}0��\��S����'��������13�u�|6�����g�v������8��yU�z�tv5�t^u\IX�p�AM�#.8��i�~��;�K�j�!{BG9��q���4���`�d�����M��caO��J��T�E��9���r�IY{Q+���N]���t�r��Yiw����t�l���GYh6�j=^�n�;���k�K#I*��?�	dW����a��7}�#��x��7�3gH��\<�?������]��'��=�u����d����s�c��:e���g����<��C?!��{��+�����89aY�3�a���3d��� n��S���fgH;{����e3�=���>�Sg��{=h>��z#R�������S�1�8�'���$A�,�w��%���Q~�����.�.l��04`1@�C�\�N�����<*%�.@��������P�.<`d��DE�W�c_!��Js���f�/I1�R<we4G�	,�$��,j���,&�,���2���,z�������b��bj�b!$;�q����l8�K0oO�H;��V��:3���a���de0�Cf���o��&������o�7�#�����zt��ax�:O�K��S�ax�O-m@��'jF���Hn�J�!"��t�%�<���������e�8��V�Y�2z����F��!�S����8'4��iRqZ�;���78��9��sr���
�D�w>�����&75!7uBn���4rceU����9}F������D�7^��;������Y�~�1�A�h���}�,���������������'O� �����|�����T��H��i�-W8;�y�&��Nt��Euf|z���97C�F�%���Q��}�/���b�n,�Z�L�#��/��,�Sr��������>�/��b*U�e�)���r��f2���%�����y$�%'�_W�i�Q	��7���i%��w����,�G#O��
C.����f� #���a�Ja�p�mY��f #XFL/
22�MB�}���-U#~�h�i%�x����~�r4C��i�)��G���1bszi�Z��|��z,16��V���W���+��0<�����7x(�_2GL/��F���dj,��!b��15S�i�v����Z���FO�i��B4�g�F=��l#�Zs�����c��G��o�����g�!IM�X��mb���,@C�A\���"�A�Hr�s^�D��)�*���9�yn�+�������d��F�[�@�j��Pb�-bl
b��af���c�����|ug�o�1'���2�z���\=��3����vI{����-�)�������.��I-��A�k���_�C�����}��3���cN#����N��������m��Q�l���4�(���8��^��������!-^�~����i���3��� :-���l���Kz|K�:������5��	s�M��\��s��v�i_]�
9��|����Bl�����P�2I
�#m6_�Q�����$�@�I��]d��|�.&������M0P���[cQD��A��q�9JG�(;������]L�T��*����=��jr.�(�*.����Ks0�#|�6��lv!~�E�Y�duuj�����(���W�JU�2��k�%�c��W�*v�����-���T����+��h����T�4�w|���c�c�����?����(�m=�����e�#���S�������s����|tk��:���Wu���_��0N<g��.���w�%����=��_����z�����z��X��Tm�4���)&�PNde:��V��)�qt1�d��2��9w|{������-pcQ~F���%[Zox��#�e�,�f���,�d�#��6�u��
{�1_q��
�����������|q�/tD�3��5��N�7������/�U]
#������^��M�k�g[��T�3Ze�x�PX�S�s(���l��.�DiZ<z�S1M>[��$^u���09SH[w��k�#��&�b��	o�`���3�9�����d��{��#���>(��j������1�RO�M��sd�I�����TJ��}��w\V�s������^2Q2��>d�D��W�s�6�_� ��{E����iJ��UAQ�{a�U�i��2���@p�����	�g������-�+���CP6����r|�_��9��ABry�^&��������������+�����nX{���h��M;�hnG?�O����h:�� h�-���-��A�)���-v��+��p~�x34S����~�O��j�q�Z���b�M�0�x�����P���r�+��ae���3�Z��h���n��KL�����	����(��b2���I-�*�d��b�	�~3�����l���C\�e'����p:�������F��!nt~A�A���z���1�!����I�7�lR���s���v����d���g3�&�t=��a���`Rx����H�X��vJv(V���ea���`������l<�8��.���2���ux�I������Y������?��x&�����*_���U� f,L���6}�@���sg��Y�	]N����/r>Gt����Oz�i�r@���E���,�r���t�4u��&aX%T�C85��s-��r�[j���VG�>�������xs^f��
r!]L����>���('���s2��ch^CJ^�{#!s�@��[�r
��X#i\�a�)\��-&�8L�Jb�;L��	_���T�Mj�l��7��?�"���f
�M�V.Q�%�r�Z.��T�66��X#y\�m�9\Fk�k���� �
')��_	R,X�+m��r��_NW'�r�}C!J(����Vw���1�����c@�f����|���� ��q)K��|Y��cS
(�(�<�Wx7����[�Q����/�34��	���q��L�+0g�[�,@�*��h:�4�0�����g�;��~I
o�_s�j���&���@'A([B���/�/���n�?v�L����4t��V��34�Hx�|�^��6���$��#���H�a��=y����7
�5���A�]�gZ�9t���$b��nr�����1����x��5�R���GIzcd��c{�]������MOx��3�����q�U>�7�8�tY���������N�-+��K�������V�}_vu����{.�7s�;(�x��dOG����.���9:��7P+K�9�lxq�mk��9:c����G���@j`GX��0%N�_�PB0/���#"44h�Qi�@�$D�ar {���<���q�"oq'��v	������y&|"q�D��s[�oN�g��E������C�n��<^2����5�IDI]����J������H��A���uN���D���L��9��W��U�Uj~��_i���[^-��Z7�$������YS���	-���d�U���x6�E�3����(+�K �L����&[��.��n�S��	/�Q��1��K5��am;bT�[����$�k�Be<�����F��*�o�%����1�v������?�!�����������}��scx`B]�J�(E��)3o	na����9]�v�.c��0d*�a���W���Sx�3W3��YO�e��ykHd4�6ew�������E�64}�A��3	D���Q`��t�E	��Q��W�tn��]g�k"�|������r����S�x�]�
����z�c ��'�I�y ���v�+(4G1��;��{v��[/��e�1o0�;�+���x.���,�g����7	�f�����r��\�Q�����:�+��qK8��z����|�����NM�x��$F,�"I���b;�R%��*q/�����J�K��5x��*K�4�4���S8�����R>K���lp�	F�j�{�6�{L�/!��I|/m��}���#"��N]�S��<���^s7z���t��k�GO�2jil���L�i&�����0uy1D-���a��T[2�O����%s�8x��Z���%3��T8Pss+�jh�n
�o}�����������f���NMmh�-���y����������j�����?�hl�����e8��l5�P"VuWEG�bAD�L���,)�XSM��h��DSKI$2���CA
j�PE�tc����_~f����z2io^|!����	�	�4f�0������c>����T�k�z��dc��vv�2v����W��tW�ZQ������Y�����fu�c���x�N���4��g���5i��q�|�x�G#�v�I���w�96�kL�������;fA�Cq�r;���Z�N�����E�y��Q���S�0-�G���5���=b�����F87C��fp�Tm3$���T��w3�k,��{38g�|38�W��I���x7������(�����\�����!��0�Q���g��-=W�S��e�TP64��&���iL�P7�rjSNEi�Q�����;��Z75_y���6�x��^Hg����Ez7�#��Y������nX��@z���D�����t��|���E�w�&����}��}$m�6q���o������{D���+������J����}y��B���������������vv�a{=���jxqKBo��"_>
u��l/���g��!&E2b�=\����|��@<[�a ��'5�m�rp��{�p�U��g����C/�����?O����?��]L�z���C��^��KG��U���^�!����Ao4�_�����b������xt0�����E:e������0������!�v����/�T���9B�e���������b��/n�]�����ow��T�st��1��C?��H0h�$y�v��S�����������9���O�_���gG�����@[mL!k�bD��O�(����'���'Dj�8�Zfs�3�����"�}����+�p�0H������+z��xu@U��u�l���������|����z6�]Mn&��/�A��e�)�%F� ��h�q��4|�M����:R_�h��O'��t}���W�,|�����@�K�|]t�A�y�=!l������"����^{��_>�QX9~ppy��/�����,;J�0��5�H��eI@�4�uR��IP}��L����r��Y���+>���oW�I>���'g0�����l��������h>�>y2ZO�Cb_��l��r;{O�}��A��M(D��	x<O1\_|F���YV�������Z#���W�)B�����,��U��Q>���'x0"2�*����x+�M��"�S����O��}�^��&\���O;�+�O�Q�;���q}��x�Sk��J��r>�d����F������[���%�X�`('H�Wwz:��JkZ]/���k�wC��K0rS��;�YI\~���uh?��Z������n�\e��7F�N����Va�M"�	�/����l��������j��[Z����$�N����"_� �!���-�0��5�����l�y�M�y��� �A�a��b5��1����[����/ -�,q���-�
~���[�O����|~x���oO�g�o��v����_=y9L1�9��sxJ��5�S�	�|�p�{�������~?$��6�������'n��G`�i��,D���+^�__h����w$}��w?��
z,
�2���Z�gg��:�O��b���=G�t��j!m._���oCH��TK�Z��\M�j�����rD#
V��.b1��������@�����cD"V�e�7���4pa�������c������HA>b�[��D�H1�0�X����D��;��������?&���z��9I���D���t��]�y�&�y�_�����^n��'~F9��@b������QZ�\jL�v$������|���/�s������A\�h��l5�wL�!���X'����V�'����.��G�I��Q�zE��^���^���=9\�-1M\�K��-��z1c}���DiIx��=y�l!*>��e	b�m�Q
@ox�<���U�����I��� �{~C<��(8��`�zm������W��Cj2X������}Cg	��7o_��r|��w�h�	�m�0�R��w�x���|��������pz9Y,W�)kR,:�Y'��M~3���1��=��j�����B�C�E���v��T%��8i=!0`B�����T����J���Y|y���O��������������l��Qg�v����g��W9����<���v1�Crx{�q(]�\PB?�2�iX=�=��t��
1�5\����	C��a���{rF!Q���[/O`=E;@����@z�0��\�B��!IJ���
$��Vp����}X�B�H)�%:
��"������gU�������t��{��M{�������E���]����� ��z���(���G�q��{�8���F��x���ct�����^�<=|��{��G �e�T��l��vl��������p*��'7���o����j�����rg=S	�%F�$� #d���3���d�_qYZP �������a���x�y���<�?�
���h����/�j�����i����k*�V�(���#���i��T�������I��r|P?�t�i������K`���0q���t"~R<A��.$�%��W�
-�o��c���C���`��w��~�fs�D�������/��t�����G��W���k`���u.8�����A
���L%1��
�[�.s!6�[�������n���xb�
�Xi~��������D�*��T��m�z��V���d08������A/��@� �5�.��p9��2 (��������#�[�������q~��[�Z����q�9�k���%�t��`vb.��1L��P���|�������o��d�����?0�,�����,E�2�����l���+��a*���_xT]b<��^[~O!'.>����z�'w1����)�W�c u|q|�\�1H�,\���T
RJY�i�vz�D��Y���o�,^9$��'I�^�
�Oq�19$=k��E9)2��oj&�X�js"�dGSv��E��r\�X���5>���F������N��{L@59����	i4���3����2�sv>hE���'�|�����H���v��:0�~q�q���CO���m1�>&���'�{TF�>a����F]NV��	��3�w����y����N�T�T���g>Gt�9U���k�iMjQ�Q.<Y=��L�B��������7T�B<���Ry7h�x��:)����(� ��E�p����E*i��O�5�=���Xw�4�B��p�L-��Z�)�?d#��
�xr;�������h�}��������n�
�Z��I�(�����|*Dp4��3���t���������%�r�����������ii"2|�_��sq��?IT���$��a����0��
�([�C ��B����n����z��������������$I���\�MfT�g�3`7�a���������3�|z�[��..���{��������qv<J{���M����?�_�����?���pg�.:�e�_���$y�=���}��1�"@!WA�d�I%���Q/�Mb���KW�n8�p�@��3�M�����x88-	�uWl4��F$3|*�KH4`��������)�'37��Nf;��Hb��_N~�,���N[9/Fr����m=Y0a�w~ye+z�}�$�\]\�����8-fIG3@V@�D;�y��iN	���U���c�w�,������a�P�SF$g�c0����d�|����CT��,�7mx����p��������{G�I�q>��"�(����G2���S�pGN��SB�`X�.��L�,w�
�!��!�J�$Ba+��PP#VT�8���c^�d^��O#������.��1�=����ps0sC���}BUx")}A ]�L������o�V��?����;s�>:#�0`��1>h�"��D�2Yr���l��������5h���y�)�	�,�
�-Q�����D-$0f����>��
B�S)��zJT���#�z����b1
�T�B^cB��c���.��A="|����#K�"D/���7��`Q�1�����{����� ?H/�����R��,
,r��9@���c�c���/1��X�W�Z�l&�f���^�5�p	�1�B��=@�q�v@Js�K�]���k�qFs��1�=��O175�o�����n�k��G��G�~6�u�S���l����=g���g�&�0��ap0_�����������������:������&�E|f���;�'�CRH�4����O�xl	��1SR�v��p<�\E�� �#��9�
#k\	�Y�D������jH���9��F����l�k�h��_����W�!�A�WV���a��	�����|�M��B���=�����J���5%K�������=(e�-�����x���br����x���7`Y�\&���=����Y~	D�r)^��`	�R�|�xdo��$t������c�D;����s0jz�<�:�v��L�4	+-Qq �0H��o�YN�'m�
����>����|���w|��`�<`�0� ��p��5��w�#"m"�������"�gd�G�t>�"9siUC$.M��D������	���3����6uZ\e,7Y&ae��6�,��)d����$j5|V��Q�bF�����O�J@�E�\�F�����m��S%T�����
�|$����b���>�e,�z�f2��(�%��"��>�}�m���>�N@���QJ��V��P���L�q�,����!�\�x*�rk����C�K�pL�$fj�<Q�C��n'�]]d�$t���A'!�^^�)b��������(�Myzj�}}sIV}��^��(n�N��.9���+��z��������!�Do�?�}{{M��<���.berp�#|�*���0�C:{���F����UT��g�'<�)��=1�b�jQk�6��ny���u���_�O��99�}d\����M��N�RX	�A8m�D�/�~"���bo��X�����1�@����"�Q��Ro��J�i�z�����(�&���2:a��s���k)0W�9a�W�S����d��]������6�[�I�>V�T��������������J�����M�����A�^���^�6=k��n���49��f�����0i$.��p��s^���%��k}�O.`\
�����@F
��|$
i�*m��M�����i-.�5���.��'+�
W��;�3�����C`X'`����s9��^*��H����'�M��Fu�����ri7��W9mjA�kSKy�9�����X*�G7�!����I@,Y�"�����o�K��)sI+ 8��b��'��r�
x�����0��]��w�.����w{���?L��|�^�?={u:��7���_�|8g�3���8[\������'�gdEr�mP�WJ�C�?����e=P���Fh�l���a�]u�)��nl%��������{���D�Ib�����|�����W��������c 5�����V�{��^���pt���G����e������Gyz4�������&�����	��T�{�|p��n��!�	������X��P���D����y����%�aUp�QH2��E�q�i���o>��~}���/o�'
~w��{�����g��C�?����=>N��'�n�Cm��%
PvCT������a�b=c���qf���a�9�#��y�_����s�����
����z���������Nk����g��g����4*d<5���>f#i �����N�UkAOE�@>N�`c��5>���0E��^�6���a_'���#.	~�����
�TGew�|f��)�C%�	�-��:f��P�;��1������0_^������3Pi+��6�#�v rJ�����+�k�]�gTJJ��3�mh�F&�MB�F`���mH�D�.(�!�7��kR/&6Y����O�]]O��,����:����_@�/�����/���}��;� O�l�
M�k���
+�������e������95��B_�r��2�E;�������~�{�x;]���� _PA��|(�a�\Wv��@�������9�N��
����kY�&�/����� .�])	��a�I#�
��B��:�<���G����;����G���1�o�'��$}A�����?��.j����X��*�v&�9�H���Y~3���1�H�U�p���5��d��wEo$��+fy|�%W4���w��H\Z���46���W_
�n�G7�4��4�@j�P������H	������8�\���-�,��<.U�/�xr{�����"/����bE��~��]c��=hR���?�B����H�1���?b��_�B�R/R��9�li��$#�UZ����>��9���w�*�L����|Le5�Ay
��k�^HD�D���)fE���(�Yc+[�!kr���G�^�nC�
���v	i�t�
;���0+�bB>\���������j&����&�a�.�}u��fJZ)_�����2������CW��=zi����J������8��`b��m$~����8�2�..4�j��Mo@~�������p�����O�����'&n^�bb�_������l�Y��4��:�_&�C_@�@KY��&��VmK���Cak�%�B���H�f���	D��������,��}��?�	-���Z$���)-�����H_��K������hu�zI%ZNY��vI���|��B�$�~ �?���(�Eu���I�3"sUFLn:	Z���05Z-t�)5�@��h��7wCB���!�A��h�^�1�7�E8�#�Yq����Qt����CMUW�i�3
���q��5���

�3i��F
��px��
�&Xc���pG�P�mzi�`��VzP�o��~�b82��"�����s80�$	y�o���~����7��uT�xVV��z�.E���D��z:m��m�Vl��O��i��,������m%�������>�+�s��&0��O���?�����a�EJ�=R/��@b�q��	*(9�]��|"	������0��?��
:�M?-<�I"�=�@����� E5H��R)�$zo����9z$%0��?��
:�M?mHR9���Tv-
�T��
��A@��6H���db����Z�A�����}E�=0a+UmN���������8���b8���@g�fl���;�?�"�^�����z|I�6��S#���X�i��e�����u�6��i�'�Sv{��S��)2����b��J�M�ox�$����{L���
Z�7�R�I0~|8hk5�
/(����;���`"��A� �����g=�80��U�L�gn�������h||�S'���
+��������:c
�s��V�41��r&&c��%�-1xN�Z!�[q�T��\+���4�keF�_���3d<��w���H,x��g{?lX�#�0A\Mp
ux��=nz�N��>�k[U�iz��^�O���x�A�htF�*�0��U����$gP9�Os
��`���3	���2Q�`.E��H�X*`)#�]�
	K$��!�4�`j`�y���0�1C8�JX���nWaCB�	�%|�>�5���X<'!]���
����~�T�J>C�5�	}���h�,
�����>��R����xK��ZJ������u1�R�n���
l=��.]o�-��qh)��C���K���J��4��\G�������RR;�K��&�m�
��g��x��q������G�1���]�!����!,�RR��i�?��"?����jh�W�(�9!�K��*�}P�+T��� ���W��n�l6���6�n+�P����B���f��w�3��s3�b�@7�j K�����m�w0��z�'���$:\����f��V�M��v>�Eo!�O���"���"����yh��%�u�������Vd���C2[�'�y�uvD	������LhT9_��A��*���;k]3u���h)�F�N���<8����-W�V5�O����U"1��X��v�6�~wD4)��u6g����u����SP���IQ�iiAY�h�#�F���ui'���������)����E+�D>�D� &#	c\�6ui��*������u�s�lZ2�r�����	d=�E+������fd��p��]����"�}$�v�h�/jz7Oa�]\��U����;u]�U0y�s�W^�������� lJh��'���p
!���N�,�**X��)���N�lHA<d*hd
��9<@t�u�=��V�����[NBE���������=��n&{q�������dS�.$q�ZN�����+#���j.rL������1�P~��@#�9��Y{��z�J��
b�w�^!����>
��@���24���~����v�
Lk��P�Q�R�/�N�|�t�1?5W���_��~�j�]-.v�
k�V�WkP)i�|>B�������(�x\������2������>2�"R��aN�M�M��_�_r|��G��2�-��|��L�A��"R������������'O��3�'�vQ�~���4��+ru�P^���5�qJ����'�
�r����r��[*��������c���}��!��"tP�����������v�#	%�T��r��*?U��T�ZEe_|��b��^��������eN<SRX�f��R�e�A���*3����~��9e���P��D��J������6�,��k�Vm+�X�<�w���c�u��U5+E�m�\�-����y*�����s�%��
i\��T��6�����NR�}z9J����������.�`�`Ky]yVMR�>��D��FoAm�
�)f&|�>�R�|�����l����R�}z9�|MZ����x���z	���/����t<r��������!���J��C�o38\�������9�^��;�z1��?�+,R��^U
8�F1�K?�7,RC;��QOXG$b��K�d�?6���Xo��v<�=���^���^Q���\w*�*#��X����[�u�
��dm����`�B�`q[��()C�U K�����.S���%���DU���$7F�-x��ZS��F����2�����"<�t��.��x�����M�X��=3+��J'Z���)�
�����4"�0_��l���Fnc�1[���E�|rv���
`�g'�~8m�p�]4j?K����������y������/���r����C4����$�&bTu������U��h]o�}8=;Go��g��>&��j>��j�?���	v�S�)��Y��V�p�/&�������q%m4���;y�������it<K�R�������������H/��x�g���g���r�@+��Bl��u�sy����f�1������^�2{�w���5�	�/m���\�$�F�2�%n@K�-q���2@K����������Z�����8-qZr���x������u
���T��K�o����1�v)���4������J��B��BPvw�[�b�g�X_rY[4�4��6/���@�7������)��*�d�-�+D���|�����|����.������Z�B��&i��f8�����^U�����<��L�0�k0,Hi*7H�{,��?4�>���2)muX(��hr�inR�)��g��BU�����U(�n�q���n8a�?	��K��,82@�Z���v��$���XY�hV�������;�D�A���4,��/�4��-�X2%_�x7�s�DO��t�a*���K��c�YaOc���]k��.��[l��z��)��h@xT�RgZ�]�b%?<y3:c�E�%^�(^a��\�����f��MG������}�������R?�B"����:��������P:���U��PEk1W���H��,���r���M�+8����%���\j�;�2e�"�+��������^q6��[��?�����9��J�F����+����>r�G���v��}�&��o��\�f��R�Uu��F
D�N`c� @�x��k��
D�*����B��B�#��� ���MyI�c�B�h��N�P�IzH�S�����%�����c�Q)(�R�5�'z���y��g	=��?��������_d
X*m���!6^��
un��A���=�V�ez�N���������mBZ��
��*���z	��J�oc����?�k_�8�e��O)Op���'vn����h�Azm%>��~!�������wgr��(`+����J��$��A[�>yw&�� �� ���k�����n>Q�������J��<�������"��7�������zcv*I�h���o�]�z�i<�����8���%��z]�z�iz��eK5 ��`�+��q�SX�o��vmw�����4�A[i�V�ii������V���n��_�4�4���n��h+
�JC�P��V�����4�+��6�n>Y��\�#�5'�u��oD�e7����,���F�Y,����s�C�;����{�f�����*��t+��Z����'S���������x��I�$gR�Yzis���-Q����
�*���3'ablw�5E���%m\~�v���*����E��"A��l%���t�0���!�nED��*�j ������b����^*�����)�T��U��5 G����"����,���z�#����M7�8"����uG�H��nm���zG/
q��@��8�� ��Z���c�vSe��n��v=�-�Q��~�D�P���L�+��� X&s��j��V�N�!���X�Vm�^y�&d��Lt+SmH�j��H����Z6���oT�zg��;@/��{S2�Ga:�����iH�j��H����Z����lT�zg���;@/��{S2�GaR���6&S}P,���=��~��&�(�J����R%e5ISIYM�TRV�����G�eu�PI��	U
�i�Tu���{�����HU[v�z�-�Oec�a����[�����l7��DUm���.�	v���0�{Q�(iJj�]�"\���@��r�z�����9fifo	���$�_!
"�&i��i
��Fx������z�o��7�B�����i��U����������*�M����_�����x��
GTT�o�
�#�I��>�������} �t2n�eK���h�bG�����|���u��l�DX\��C/��Rx����
�������uT�:�0q�hSCd���J%0�Y�x#Y� ���|�� ��#���L�HI�+!��	�-��+"�����Dd��Y�-&�+�W@d��C/���[ED��m������NUD���J�Q���mj�,�Y���"o$"KD��Ed!���U@d%�d���Y���
uYtG�e"�tc�������* �������Lxk����_���AHZ���_�����p@V���EBh�������r)�/-�o@����zM#���A�Q(d���uM$����z�F�hd�F<�F#{{�U�����du�f���s�h�����E�\N��%���r=�m�/>��/_��o�_s�j���u�������@������"Vqu{�QAg�����C���!%��FxF����H�����\�d�0�C<�G��i_����^�Z�1t��=����d�A�����R�|u�J1_G��+����t`�%�6E�����/�������yIz�&�H*%^�������:6�iK��{��5qZ]��X+�T���tj��G�L+�������U���+�K2n��+c���2������z��(4g�2|��J��4r
a�Z���.�p
`�f�o1�f�n�%R7���1`M%�KIdc���=�m��Z>�^%xg���0`--^
x���0`3z���V�H����P)>�������P��"�����O"�(�j�]&x����Q��)E �j��42�5�4��q����f8*���a4y3���
B�*R0B�TY�$�X�8�D(V="�����������g�V����M�H�e����]r^�/GJ0=����Z�Qx�._���v>�%����
�S/�Z�Jb/�tY�`2��W�$na��R��v
��[*9�H��P'������g)���w�WA2�I/-������	��%#o�s+�f�W��aU"����f�E_��x�������$z�B��M]�������_s�����m��6��A��M"#t���:�M��M2��I
C�Dv�1?����M2�/2�#t��dh�!�I��<>�uY�c7���&!�"e�h�^�M�8���I�f���f$f��9|��M���JU�(�b	OO�`v��������'h�l�:aJ9J������B��b�p]��N��-�T�C���7�'��c�z�v���I��]����&#���{���?gq5q`h����DlY�(�*b��YY�CwL_��C�c����$���1���?�q5q;&�uziq����z���3`=8b=:
�1�c���9��cX� �$v���>���I���������A����l���cz���^1��~�����V#��|q�#�b1��lvA���Wh�����dv���/a��Q>���5���:����/��3\���x������O�����&��fd����)����T-�z�!<�r��5��K�r��\��K=�(;�F��#<h�Z�.���_%�WI�W��U�u����_��?q��'��$������$�*��8z�;�����	MWU��;��)`|)O~"H�m~y)Ohc�y������)A�����e��f���K���+_	*N!c��tj���Jp�r���s�II��K�Ywx��`+�^��+b{X]l*1N� ��F���CD��j�j
�`�`��'���s���[`W ;W�7�������3�,_�9����=(o��b�Zu�K���/+P���Lv�z�� ,�����6���`"�������0�{:d���h�3��tE��>�����G�M�
%r��,C�,5�9��g�sP�nU>#��u/��:�Pu��^e���8��W���:�\��
��=#�1 ��3"�yF$l+�h����L�h3�Yu
�n������T�=���{�OC{�k1�EQJ�F�\�g�kP�A�[Z�e�`���|>�Z/�l)["�}X����j6d�k��0��X�z����n���F���n�8|0��v��U:��/1#�������l�CA���	�>���d\*�{����("�$�����"��uP���X[�bc��=�F����QDCv��v��a8��"F��(��.�S��t���.�<Ts�Hb�m��@A��:(�oV�-H���f�����i�?]�(�!C�PCq�0�}	G����S��(�����c������=}]Y�]�jk]���-�uv�S�S�R���k]���F�k�zH�0���6s$[��:�~���1G���au���i�A������7`X���k5�@�P��������
Z�m��6���
��-4d6G����Z�"�|��yP����6aVZy0����,�12�h�����V�E�GpZh���u�-�6`]�[z�[�2�#�Z�n�k�GAP��(�PVG�0+�<��3bc���:��u�C=>��m�/>r���rU���n��������[��'�m��XW5��(��
3��p4&����F�B�Q���v�c��S:��\A]��s�J��:R�b�x�Z���U&m#���Y9���T���G�)�����
"5�3Q	%TS.��B�F
�A2��c�PC�F
Z��0g�����"R,�+��(yu����>�K��W[2V�;J����c;�E/�"�.��Oe�����s���/�������������=D�n�����6.(�yp~�M�}����*�'\N����s�T��(�"_U��wL��i�}��ay#���G�*�O�l�r��v��vU���Y:*����n�W��U~p^��0��R��<����U�~���N����"w��M�m}s�|m}s�kkb�xM�>���!�BL"<��V��}��BD��u��k�6x�?RwM��A�������N�9�P��-��z[_=A���z�2��j������.�a�2��e��u�8��G��>�����k�2�:jm��v��B{{h���<��E~�/��8_�����"��������ku����|�����������}��	��j� N>�{�6�H��i�-Wd�u�P^������Ld��Z.Ycr_/���%M�}7fM�Rq����^�����A�l�a��A���'�[+�D�jqX"N�T��Zd<p�����:��*�u�
�y`�D��l�Hk!��F�����E�NIP5�d����VYE2d����t]��1u�K������}a�#,qbW�/�����J�$t{��w���w�[�S�N~�����a�S���g���� �F���^|A�����z	���/��SQ�x�R���_|<���`t�`'<L�R�f!�R�+!k�������<��/a`���dm�S��
J1�IP�EHe<�����>As�����Fv�Z�������9�;�r�[�w��to��V�x���x��B[���8��u�z9�hrfK�o��/��������8;U�w^g����s���|���)f���,���'�]����j2�a�m�������4G��aI��:;��
��|���Z*��V�&�U�-����l1��m�������7x7`(?;y��i8�XS��g�@k�o���-�~��r�����@����,ma���xkL�j��Z�ru`����a��zH�����������s��~��f�j�����C�������`?eX�*���l+���K����q,'���w'o������<��g��YJ@w�������d�V@�1^�0�s �.�������:G����Gg���c��sR{��EN>�/���]�����&t�`��Hb���a ����M�p�T���
����0q����`8L��aj�ab�a������=�a��C:���*D#;�<T��h��]B���'q����t�PFl��"�����pUM��,s�$�I�W��0E:!,u��������i��T�$�Q D�K�H�Hj@�K�-�_�0��RlX*H-h�V�:Y!�
Y?njdh�^���o\��~���� �\�s�o�p4��D"��2�Xmh�N�����*c"��}#cA��cA���
cdu����H�6�%��Tl�r-�t���jCY����g��2�2}"�D�#cb���A���e&.k��n�>���X���e����������x-��J0��H;��M��7�f�I�7����~3�oZ���h���5[o���6��r��lI�E��������t��o�[�CY�6��V4���bq����^w���.K$hu�1�c���9@��a�Kf��b�<9�_���bUc����F_(��A�����Jn���*���R�'�X	�TB[�DO�s����:����O�5P����+�T����h���DBn��W��+Q{��������h��x!7� 7q@nRr����3�{�%>���k����:���u���
MAn��W��+U{��`������T�������r���[f�T8�7��[�)r�5PT���hVM��1;t6�0��x�j���C����/dm���w!����8��Y&��|��I}��U/N��r�P�aR�
'�`���P@��h9�DA:�,W����.�]VS��n}�OC��������VQO�]����J�4@�S�U� ������^���^S�������{����zM����V�!�����ZJu���R�����V���Q��d=�m:L��K�C0� �	s����0��`�Yo�%.�s�sI0gZr�Z���%�%�s��i�f!��g��~�k_���8��t�ZLnZ�*xz��a�Fjs�H�HH2w�Iw���2w�Jw�q����.�;���a�an=p`���;����xN����A���O�g�7���� ����P���3!����z<mN���@]��z�pVI�/�*�������	Jw����zW��g���Q�zC*m��������7)�w��[b!�d��R�z�t�z��L�.d8��Z�WL6M~H�B��������V������J���uGaM����%�)�����*<kA������2U6�l��t������2��VKm���N��bh��F���>	YW=N����rT���M��rToW��b����@rlP�=����B�K���`�:6r|������Bt����B������!��t\�V�c��&�Ed���)�;�Qk�e�qGd��=��C
��w�_v��!	�!�����P�m����S��]�4�
z8D�8h��!����Sp�8��D�>���@�1=m
l���x��y�����|��T��u�=6�m���I�k\���1�������:tWr8p>�y����
G[��Ax<p�Ww"oP7�|��o�c�z�N�
��>q�;I�s�����7��&��c���u�
�[����1:��} oP7�LN��N=pr���P��n�k�ww����#o�G�uC���������[���A���4u�{���^�WwO����vwp����w_�z�u���)����@�k���[��E��p����{���������2z�L����+j��t�s:�������F9��
���D���BN�
^4���qi�|��83Gz���li���TF����n�	��
��������� �W��4� ��������0%W�Z��hL�UI�-G��{�m5�ym�7�
	(,{�	�qxAD;������ru�C�6������F�E���e��a�a���;���x��Q5�������E"�T����6�'z������(�i�L������:FI\%��Qt���%>���Ne�R�az���"��	X�Z��jP�UY�U������5��m�O�*����}
�rP������yc��r�HR����	Q'G��D�p"
�"���r��0�?���(L$������@&"=U$�� ��(Ldt��Cg���0��B3���(L�hE
�� ��(L�b�X���e�r��"�����%bw���n�����n�����6�c�3J��E��s{+)�J�7 )�����z|�VR������?I�Z�HR�l%�[Iq�p����WR���.�'[I�VR<����k�o�Y~$�+�g��J�kP\%*-�T�rM
�D�%�)Y�9����X*%�=\	�v#+��H�f#=0������%��h�^���]������!��	*�$������$sU�Yz���s���\�
��g�b��D������m6����;
��<7����j"���
�����x#d�6�Ajw�^.)��c���h�D�D+Ck`�Zp6����<*���p�)%�T56(^l@�%F��4��YLS��21MI�#O$�F#�&RTZYHZ��	���9���0���#�H�][sG��e�;��tY�0��q��u����������������rh���S��2�)�4L�U(�h�S.��f��C���q�f�����^�#������r�����Qr����jZ��B�Y��8Dn%�lF}g���;@/�� ui�����$y+�6����4V��RM�HY�:��$��-�d������ �~����.��W:�u�*o�����#�P�bQ��(S� !C�S$�+)�K�J
���������.C+)����b����DR�������E�[N�}Y[n#������m�S�&m���y��n@��D~�Q�.��E���.�6���0�{��j^��j���;���9����WaJ����<
S��$�p!�"�&���i
��Fx������z�o��7�B$����ik�U����������P�Q��J/%�!�K�� �*x��EauEC�59*�.aK��k��KH�@�b�>2\����
��m���K�3ZM��E�[�lN�\�!�j��^����V&r$r�'r�&��uT��?���&*V��C�"&

L�((����T����V@������P`����
�Q`tG�e��dc(�������:*�������K��"
L�6H��O�r�*
L�Q���C��F�(0u��P��Z(0d.bQ`����@�����(0D�_4�
(�D�o��
��~�R�B�"
���L�nV����LX�?��Q�	o����(x+yz�r�����d�l�?�k�����d�/��\��;,.7`���f�V��i|�V�:
����n��5���bm�l��z�b���:{{�R������6��8�����e��������
<��I�B���:�L���jFE��;	,>R��u�
���0��]�\���|~K��{6�^��$�9��
�Mt����;D�
q����r��!-n�!�
�T%n�� ����	�-�u���?F@e�Xv����N��`��_�
�#�F�J1����/�������e�^����C!��D9�����{[���z�����ZDT��Bs�����0���wp=���p=���!LM�l
�����>�6�6[m��]�����2�]�x�]������$�%�)�S	�I2K�3����$$���^Fr4�c�|��K$C"���Cr^�T��X��X��v�6�P�e�<��
���D���_)B�-��e�}"g�2�0^�7<��m��/�U4��~s�Ws���J/�v;�-��g���"�F�m��J����2nE��7g�p5q[��UziK����z����1`[���G�{��_����y?\M��(k�^��Y��E<�C'f�,��x���\o�rcK	���C��*[M>�9���8[�K�
T}K��:_����.��:G�|����W�%�]�?�3����>�r�c[wA��$����&��=�Gb0�n�3����(�VIo������M.�e6]�x��F���a���BN�b���]���-��������I�^k��c�85��s�8H&^��q�p�L���c,���=�R���*D��z�_Z�O
���F�Q����	=4��$�Zp�4�V	 R�zS�#�
���)��,�U��?*8��+�\y�XT���<Y��fE|H#����m[��\�-��"��4^������������?��V�&�U��6F���:[���'g�o����������N[��g-�����������_N��r��!���y`���=��������I-��A��������s�����K��&��j>�A���O_��:E*��g�"[��e����V�������\������O�K�����1��5�4{�6�o�|8w�{���D�%8#�������}z����~�\���"��z���'�~��G���k��Z=tC�k�^��\���\�o�������+-^�~��������3��H����4���X���X��\����)_j�Y���~��S4�����^���u)>K�]oM�j;k�*H]�����u�5�u������B�����K�w,e���}��{{�6[�& ���4Y�D���b��Wh����v:�/���Di��fW�]L./��Bs���$9�>n�"�8� ���i�VAtDF�q�	h�����w�4��b���AJ��b�9�d_a
�+���V���U���b���Zw)���\6.��i���fs�����*h��T@$�����<�)h�����>d:%��L>j������j�F�VU����?�t����"��~�^NfWh����U������f�3�|���D����H�<��C��@������NZ(��������o~[g���gt9_��X�
1�Z��8���g�"�g���������=�����3x����Y��+�Lm��c����P'N��j.4��Yw����V���=�P��W{�����"_�'���w���s��&�p�^v�-�!��n��`��0������!<��C�!���M-����?�;���1�\��i;k:Y�P6����2��*M���h&�5]����=K_��k��a���b��h�g���p�:nFdB��D�\h��$|��+_�p4�q��K��Q)l>[��QS�A�H����+�B����H����������6�X4i��B�_���`�`��7��j�7�?�S�l*a���5�4�J�������$�^�_2p��HM.�GMP�P�9��|&�����x�F��3���:{������;���ZL��hZL����Mx�$-hhP���w2\5
�0����R����ej)`��J��g�RZ�������_O��\���"�@!0�M)g�� ����`j��Gg���#���)��^
��h��U[�Y>��l�I2Q��L$�p_�FI<�H���YSE��x�GRy$^���7y$n����r��D��y$����5z&�U#0�~���G���jj=p�\��#q#���<�H��#�_��x��(\�d������6�
�����HK��J,�C��:q���Qd���pm
��^&U.�48}���Q���Yxy�L']��;�������R6"_����5�
���/��P�5�� (5/���������x;���%��<�Ih7��]-sJ��lR$z�:[�S����l:��M�������$�:��"����>hg�?����A`�;:�=Q�B|��;�y����;8�������wJ�a>��Cw�i���j�L����-L$NA:�4�hjQ�Csvj�J�"�"��n&�j-:�ptJ]]Z"7^*!7���)��i�� :L0�0����U��N�l�2��Q�������\����e �<	
U�)cfL(��}�hP`�x�AU(h���rJ�����P����2N����CA�AAjBAZ��� F�t� #-d��F���l.B����	
a�U�����Et?�����9�o1f�^5�l�$q�,�5��nH2�����z��+����^�^���a!�R�<�������=<��@/�B~
��=a�8Je���e~L�jr�6�Q�b@BMW�F��0�K�<cl�;	fzK�#e$����W����.	��2��B�[�������71�O�2qA�v��E/*L�[2T���%2�0UI(1����I�#�s�7d�;&H�P��2"M5A
3�	�0MJ8ix�d��b_���9�X*�0� �E����
�<)r'��3!(��j��T(�#x�=����^.��L�,����f��K�(i�F�8�)�g@E�����iE��������$�j�>�oX�`|3������
K<�oZGU��DK
��Say0�	~/
�����x	���I����o���4�5��/)��M�
��K� �^U��w��0���a
�
�7B�����.V�U�w���gJ}*�:8�����kJ��z����7Z�g�HT�@�w#lj����V��9���(��.1�"1�/1z�D�`bBa����ImpL�!�=�p�4
��0I�L4��cV�J���X�����s���^���X�V7�U��c"b�50�X8��L|�L$%3�@��YQ*iXS��fM�
�F`�������>��`����i�)���5T'�!����'F��`P����e�����$h���z��9-8y*�(F��~f�(V�We�����Y�74g�������Xz�@q���:�4����G��%��4���S,~�7�{{r~zv�vg���'H�������g�����)+��b�<[��v������j
'�(?�p���)�k��4pm�\��I��r������$������$O,�P)�-F?��b�������@(}�nS� C��y���35�:�J,�������nV���-�F�vN���[���/��g�<TX���g���O���/W��a�hE��S6":�r�u�6������O�@,���z�g�d��`���w�������6n1���bs+�2��~L�D��w{�{~�~/���������/�|��Jf�r�!�vi��]��O��k�A1F�
u�^�Z�k$ZSH#�7�cp�b��-�R�\b
Y,����Q�>S������W8wQ�"��B�*������U�qv��2D>���io
S�6�)zY�j�v�1p
L!�/G��f��*7L�:�����G`j*�%~pN���U�0LgJ��qd!�BM������V2���m�7�b��*���.d�C�i�*��)���A]Z
u�p�|��x��$cN��h��B>�W���8!��������������~����	iZ���lY�-g���%�wL����!���:��^�h<������By�P�(7��P�8<E������r(�
���"����S�������&�����r(���1�S�}�U}�Pb�C�Z��W4�o
���_[�����_��v�����:z���Cc��h<��K�c�^)Iy^Z�[���Z��Nk]IU�q���i��D;������T��~�k�*��b�U�E����1b��B�J��a[}[%�Z� �Xx���A�&
,�;��8hJ�YI��������)c%�bS��H�ah��0���
���bML�������BGV9 O-�c��v|3�����J>�@���!En������5��h�Q�����������j��C�l�V�d|�:}�B�/Q���������!���"����8?;A��v:gD�}~�������4��� 3�}F�G3u�c��N�K =���7�[��br����������f�U��()PA����0�5�[��}z��2�������{���'�~����:�����u��G����V*WZ�����������[9�+��z`�Y�S�x��G�U:'v�U>�f"���~9R�Yw�Q�~�0�|O��{�4?��K���c�k ��A���<x5����|^)Z8U��X!f8�DWg�N�9<�uz=���\9���J��]!�5IE�,>�����02��0�TY���^=�8���2��M
�AiE�>���I�02����V����^�q�[i\����M�Al+�6�h�]�;[�����pMy\��\
������D��E���.�B��0Z������e�7z�R�F0!����N��3]�������J����=���j�p�=uF�����7��1������%u�������������5W���QZ[J�
\&N�L4�,�?��@�t��w1.���a�Gu���Z�����\��z�j�R����r[J�
\�N�L5�L��e��c�8y��2�2.�VT��0C�:}p��XV���=��2o��C���p��zHdN�o:X�-@n�!E�
~����m�+��j0��y�C3$	H�p}q�2�OA��L'�l��H�\�>���m�~U�0����u@��j"���J����^�.X�W7��M��GzU	�Y��M)����",���*��<m��hU��c��,�n�+h���sh�as�bM+'ZFG���R��J���a��*�v���_�^�%&g���F��n����p���z����qN'����aLC a�����1��M��A�i�����HH������6���2RR$�p��T��f�}��!%�6���i	v��VF�����UF}+k��.��_��l�t����t�h,���.�-��__<@��#�������� �3N��(��������;&���\���o�J��]	I�*S�w'2$��Lk���p��t3��n7�#�!�n�@v��<���cy����|��_�L5uz�=�NI��s'��!�^�$������G�8�9�<7z�G���xW�6J�8�jc�������w%2���SB6�O������a+����rCR�n�!��v���A�$�|�!��v)���Iy�C����=rH9l��K9�=����S�/��Vx�]��������3�k����
�Wl7��n�.~��(�����\�*��
S�CO��+�h�^;��J�C*!�S���:�=�KZ�F`�Z�Q3�D$�<�0+-�'	��:��i��F��ZpR3PD,��q'�C�5�9�|)�=�
�����&���o�/&�*���W�M�o����i<�T�c����S!��?]�X P#�C��i���.B����E��9��#�P���B:h&
�s���zig���������b�]�����1�'�O��x��!�0�$8Cx��q'
E^�����D�P"�V*h�0IJ� t�DWp�%�
�2mcM��&�7�w�P�J9�iO%��k��D=LR�R��J T�1�@j(�A���5	q��c�I����#7'Ff���(���S!~�n"��-�4��2������>YI�����E�\N��%���r=�m�/>���4\�M�k�_A���|���9Z����%T��W�E ��\�0l��}�"R��k[�9��������d�`j�m&~8{���l��Y�m����:Vps���#��������j�_e�\�Da�����?XH�����SgJ�fBVp��^&��~#[��[z�|��$��J�: k/�
.�x��T�W/'?���:?����3��������f�Ke�V'��_�T��o�#�
���1T�Jr�r�t�_�	���z`�����W����}%0��~�Ru�B��W���Ce0�$�P/�\B�=�r	]fQ�c�aF�+�+Rva��GL�l��|�����T������:�SaO����_���<Llw���
�Z��fu�U������P��j�xd_A�m&�<�;9i_�s�-���$)��y'�����}��v8�4o���wOa��G��{
[>�]�'{���
XWBk�����-*������jr9�����5�x������(	��J�J��;x��dd�@xG_�]�ZlZ����X��}���:#[�_����{���e}$�8]�/�X����~$K�_S�W���"���$�1oFX��L��?�`E�{ac�����W�~u��&<`?��!��|]`�/���BD�;�D`b�I!�����X��Hh��
������=��!5*��%����wh� �F�tAK��4A��@��.���#]�S�"�H�����\���**�UT >�U$_n��� yv���G��=<<[l{x�W�=<���Y��wx���$�.����f��dE,�/��7\`2�B�|��������w������i�}��	�?m�0��]V e�>���O7����6e�9SvN�-�Rb�Zb���X4R�l�������(�E�!�=N� ��h\���u�B��@���
�T�z����7�n�H�Ho��-:��_�������v�����O�?_�~���"EEQAoBeDK��z@/9w�9w]����w'�s�J��Iv���8�o%�,���#���PsAJ������J���T���8K������zJ�z�h�a��y���b�k���l���P�
!���9��rUw�`���Z���I��#B��h4���Fk�U��"�*Qg�4�C���/��Y
�l�w��dYM��4$�����&+�UY�Hj�}4���'7����l���*3y��$�0<�7�Hn������^����;�MzQ���U{rt���������n���hBk��Igj��eop�E0iI��H�q�=M���mR����JK���*A��!P��gla�����K���i_�Ro�A�m/sB8���U�n(s���
�q3'�D�Bn\_hIG*��(�$�������"�*=%��$KgU�\�9	l�}T�9)
���hedb*�:	y}��(�sl�����������Um��qp(���MVn�,!�8x(����f�?>N�������������qG�����a�1�,[�z�<Lq��-S���WJ��8��~e�_	`a��*���Z�-_��/�/��TQ������k�9�jTw_-�������[�K��E���e��~��)����-0���>ln&{#�Dh6?bV ��V�RNu�����yE�w�]��v1���)���X�����*�@o��[2�&�VX'�������2(�i���y������ 3 AY���N�pTB�}��]a�Tg�����W��S�N���l>Ai��7HR����u�gU"����0��a?����������@4J�1���Vg���F�4��'y�����G���p�kG:�N��=��v�g�w
v���N������s��Yl.�+A�9�>^�����%���B��j��<���%�Q����#4���M���[��!b\^��>�z"sX���#,�X*>PV���h7A�`�!GY���o��_��>P&S�Ja���<�^iV���-�cK�56P{N}����
5�,���DC{ZH�Z=�tiDOM���fl��{�����	:��qo�yz"���8EO�j�����p���&�;��)���L�w��r���h>�����3Z6��Q�1��v*?��sG������!�;1E��P�:Q,��S��F�� )#��i�% s�q�0������+e��^�a�����J�]$��&�"aB��:~��^&Lp|��0��	��J0Q��g*F�Y&��\M���{?�7Bd!A9�`�cGl7I��O����#���K�'�l�']g:l'���N'���'FJl�N7��V b��/$dS���E
+���j0
�.���q��_0gP"�scs�E��:���Vi/5ZNw�Q��d]i}Z����.���$�Gh1��gj�n�*r�����������H
������l�^*4
K�#U�'�A���&<%�lL���,�<5�Y��S�<�r��f������O��������"�&s�-��p�����%	������C�����b�s��l���.eCKY�?qX���Y��yy���^�������vc�
���8��kVG�\����E��\�����s����i�T��	P����j�N���s��%���QZC�Ci����QB[�f�����./�%(�
���q'E����c��%R�pPL��HTPL�N�
���P4�_��P���m��9��X��3K�;`�I��k5���yc���������tYS;A1�@1��bZ�i���/��`P,��	���0��h���u��������GNnU"�U��?�o��q���_�yu(�\(5#:_��-�n�7|��^>���Wvq�_�l���t2&���([������#Zd�+�#��
���%q��Z6�_]��|6����j2�L�~�N{r~zv�vg���'H���n?����R�#�?�y�\��Gv���!W*O�A�/��B���r�����>n�X1��$����:�����;�:�UG
fk�A���'�[�F2Z�W����CBM����8T5++Q
�N�eqs����Z��"���U�?#2#�2'�))�G�w�6���:�3��X�j�#jG��k�+�M/���9��7��_\����.���u�MUU�]��n�\�SQ�����p�"6ZS���x��O�T�U�>5����
*�O/3X�-jl=G0�� ����u��W�(�,j�1{����Fo=m�
�)f�%|�k�s)i>n��XI�o�fN�`�����K�z�Z�����B�A����[Jn������+ ��&&2J��&$(<�S7q�v����w�y����7p�v��
�����y�������a ��;6��&d9V_���Y���������d�~[���h��sj5G��f�1��|Le���b>����j�����v�z���[�!o�M�2���M�L#�}�f�{Q���x����&�����0������njTK�y`T]��Hs��-�	��Dl�Mf�8���1�M�[��5/�+���TI���D���.�%u[9�Q+��� ����l ")��:e��i+s2j���dO1����(6j�m��/��w��Y8��$[��G�f7�l��n�b����-��c�Yv������-V��[���7�e7kl��n��b�7*s����N�zwz�e��du�f���l=�f�i���o�r9���h2C���n�	��|A>��~����k��2GWs��P���*��*��u�����X��_���0��$�`��|��������&������D1��X�
�����&���V�

�S�~c�m#�$�2��������w�e�5����jy���������z���6���T��~P����7M�������2B��Kx�[rv�^*G�[��Y�sz�$��z	���%z
OqW��2�B��1F�A���*�O.�m���&���:�|Y��$��t`�
��%6��V��)�nLc��$es������2|������DB��T�&�?]�K�+���]"��@$�E�D@��H���
u1bw*A]h��8��.���$u��u�ub�R)M�����OG}n#�N:�n���e������>�4��v�{�
YY���0#(�O1@S6 ���Ux�W�t0���}��tZ���m
R����|@|%�U�P����d��*�w��!�">�H$vD�`�+K!�K��O�T�1Ac�p�Vo]	��
�����|�-���>�z�/�r�B��.�]�?����.���c���Wg�'������O�M�64M��0zW�D-����j��S.Q�M��&j����Dm7��K�^����	�K���D�.	�.��K��N�����Rll�����$��TbH�J����w�z��M��ON+�pz��������#�y��//��3�S0/k�P#�2C�p15��\�c�AD��y*�3>!q��fD�KF�JL��4��:�4*.Rz_A�ZC�z��U�L�����2'mz�cG[��T42�ti�(���Dn����nX)3!qC�����1
�K,u���K����e���D��e����1����elt._�Q��I�����������!���0(Jy�h"l����^�&
��X����1`I���du��|4��)r(q�&,��z�F��;��nV9
+���]U�,o�
�$Y�B���3"d��ETD�:'F�^��<;\!�~��Ks�
�!���;V�,��4PiP��0����u����Wa����	����������dWT�I��N=m�?j��'����iV>����*(�s����V��6w��"����Q����kzY��#����fdci���s�9Z���;�~�T�4(�I��C�t�:^��	��Yc�x���.�*m>����)1N{��&��������,�p���K�f�!�fU�/��t}3������s��L���T\����_eJ�	;�9V\���:���+�?�8����2�mR^XF���qj�������ay�#4��p�+���Sg�W�ok��[W�q����n��l�mD;��eAe9���.k�h��L����j�Mh?�k�KZ�B��k����u"��*����ToD8'UW�F����%_� }kc�J_���?F!	�tf\�J��y��)�b�f%�����*��-��O���J�,f�:�6+.��X���>V�+;mhl����d����T6ziS��&���"=���D��2��9��<��TY����7��J����l�O�X6����1�r���q,'�c�'7y,�LB���W��c�v��U�R���Q]'��]��^5���|e�
��^�<4��U?����^��\��&�@�"]C�������:C��s���8���*�:�U����V:�"`V9�\}����l��?b��T;�S�XN]���"��c9d��e������O���J����:���X��X���>V�+��ohl����n����
l�2����4�'�����;o�'�X.D��e�o���X����Ky�9(�o�c��D?
M���i���X�w����Z�C������>kA��uM���.�kr���ZD�f]��C��j�(������k������x/=�l�s��n�,��\��;,.7`���f������B���8UpNL��-��b�-���k��Z���E��1Rv;�-�o9��{bW������!������S�mQ�4<q��8S������2��R���$.��UP�pt�:c��p6�yx���m?��@!kavd�2���0��BJ����@�	^���K�h��D0��m��@�n�g4p*V��������C��P�L>�~�	Jue��@C�Zh("���?X=�,`�S�!��`��7���S?1�p�����=��C=��9���)�� wn\g��j���b:l1���3�]���!��|z��YU�����E#�CLG��i` ��
!�Ab*+�'�+��{�6c�z�#&sF�STA������9��Q=�td���m	+m&&r� �0)�.)������)�v��@;v*�;�����g�N��c��������)y��}f����\WY��v)7��"��2�
�n�]�������q�B��6��6�������4�������J�R�"B�������Z/!L���^�H��WP7$�s��{x��_�E�r{{��7��H�\;}��d��
��dZ����?�K|X/2�a�LV�~�����k���wH@6t�6J�6J`�Q}a�d0�*8���,g}i�81�����{�3Y
'Vrl��D�@���=�pu>���"��7�|����b��
���C����im#�m��"�[N�m#�=:_��k�n�n�n�n�N^
F���=D��mP����j����6��X��*��X�
~�
~�]��w_M��X�Y�}�V<�X=p�}"oC!��.T8��q��q�������������:����j����b�u�o��m(�^��
'�6����z(J�m��m���jro�u�Xg�u�!�6r�\�4���q�<�0|�y�E�{���b���9|?�~�r-�Y����mYT8=\a�G��I��J�l���8_���w,����eY�1=�X8�?��bMz�T
��`�{��*�|��WY���x[�����Z���!��X�wr�����9�F����roM3�}�)Z���"���%�f�6[d79~5��~�������d����+d������?��������;�O�<AL��:���c&�)h
�l��pAi���V������kL�P �Z.Yc�k�,���%HI��u�N��w��8O���
 �����u^��*�J�e�==�p�2_(�����j��=c�Rt�]�\a0~;��8�7�s���Dh:T�����w�zT�;�d]>��N�w���(�[j�D�0���zt�ZA��_���� *@�} ��1���Ad���dS|\�t+����p�R�"�#�XH'�I�,�]tq	O����GXa!����C��Vm��VC�S9lU����������6����,�6���Ejh������� L���f�`���}����[Jn��N��ys�����yv����`��7���9r����;o��ys������w�����8o��������Z]O�3���h�cnj5G��f�1�[t��U;�
]�g��w_�7�����Zo��vk=���iyZF7P���i��O�	yZ�0m���Z�Z`��*��E���$-�M�r���!�u��C��=�$P�	P���ea��6�E��^Y��P�VmPL��g���j���������?��i��#/���G�e|D�`�>�+��E:���L8���,���:-��tZ����P7�i��n��b?���}��M��t�����6-���lZ�����O7�i��F����
q��i���d3��*�������0���x>�;i��d�9���`���l5�H��8[�K�
�]�O���'����o�)�b�_��|���w������Za�C����D�J�� ���?C������fp=J��dw��,�;����X����/�3a��Xc���;���O����o���.{G���Q5�A����n��5�"RV`s���<��-�G��8��v�0P�0�������)�d]��1�uJ"���V
h���L�gA8����2��t~�,@�U��BIATr-�O!nK%����kTz�W�����W�8Ga��������A��(^��4����f���Tr�?���C�r�?����C�T^<��C�6�?�>�+�))b������u�]�{:hep��9�D�sv{�~}�l�h�G����k�=e�
���4Y]��|�7[O� �B����|����Z�GtMd#/���d����"��2GWsXn�eK�"������}��8n�?v�N]�z�)��G~�G��d���u�eq��cRQ���<�����V<�-��8jK1�����	y`��;���J9���Z����;Ok���Ed�]HE�g�?F���:���Z8Q�&���f����|'��JE��� |���W��k�y�l���M�<@�l��6O�l��}�4�����[���M�O0_�c=�q��[05���S����*p�]O=JtVeu������?f������$��I3#(#����cI���4��4|
Rc
Rc
��5HX�4|
Rc
Rc
��5H���U
B��:'���,�������aq�m�x��TY5����q$�Pc4I�E%+S�����1��SRuL��S
�T�h�cJ�uJ�uJ��S��:���cL�1�j�������b�-������PH�r>V-��?� @}��
��?�.�B,,�����?Rl �����^M�]�ab���"T�5�Xo�����0tT�s�X�H���jB��.�b�0!���%5(����1��_dn#�X.��*����9&/��K��J����j��f�'A:��nXd�=&�}��O�D/MB 7<��W4��JJ	�A����l�_��6+"H��j
���j��H�D�.i�/6A=�H�e�S�y�{po�[_����zL��w��LOO.b�;KX���?$XN���5��k��1���)>�c��9��cdCkl�$���1?�	/vD>�B;eS�������	^�3����Jj�}����_H��cQ����B:\4FM�_w�����@�����
=j�#n��$�S�������P/��5/�[�T�K�� �V�(���u���'J����ne�{ A>#bI��}6>Z;�8�\]�Sy4.�3 ������)E�l�)��4�k,{��W�f�WnV�B&�f,H�����v�p]�{x�]�Y�
e���Pe��
���0S����������g��s\���g����>K�}�8�Y��gI�>K�}���,1�YRc�h6>z�#�Y�!~���B��:�,XS�?����x�>K,�6��Y�8�Jl����J�m	��H����2���^��I'zY���e��!�{N��������:���h�Yo����	 ����+�]o6x��}=rzI#jpx���m $���8�YbZF�[�*��
z���i��2��IWT�Ya����x��
��Vv�0�����^�P
Hw���y��)3��X���5�c'�S�R&��S}�\�Ss�&����;5q�T����l�WF����p�����-:���wju)@���n�o����v��|{��@����"����)�JM���BUQ�oJ�j�F��>���M�y�k.f�7���*ALZ��-���aV���;m�������0�CC#��(���*���/|���z�����Y	2��N��(���]�a~����
sV�7X�%)�/������u�A�Pb�^�EI��zs+{o���a}�b�8K�JR�"�T�w���+�A�)�	�T��{j�x����>�r6��*�j�6!bh���C����`mPA.�_
y0�����tn��l����������Vs����W��l��z�xN|d����u�E��_�!���jsc�%���fG����B�����L��F��Pq�7�?�jf��Yg��%U0*�9*e�����Xdc*���1��Wu��U�Ki��3�n�
�a���!�gfd����v��J����������,Q=�6?���1eS������b+���#K����zq�$3�H�[�_�|�[O���
$(VB� P���]U�]����5��e6#cu�$���Dt� ���}��eNCj�������vgv����N4PZ���k�Gy*��)Kf�#�������1oNL��n��C�^��m�9��x��
�w�O<-I	a�����(PqIJ������d�7�8�K�_��~N��� �^��&>������
�Ac@���1��$=�[���y��@]�S{q�{�sI�<��
���`?K�
���'�P�s	UG��!WN�;��t(������4-�~��b%�R�m���x��Vk���CwN���=���������iIS�L�F����0���o{�����f5�@�FH�dY��-���L�;:.�8/����+�WV�O�\5�������2���rg�p=���*���8����T������*��i������>b��l�'�Z]�q@@v jub�C�Ta�N�|����G}�4��j�~���qO�������8f�B�����M���<�}z�9-�a�7a�����	~;[�+���
re�+�\��}�+���s��Eb%{m���}��R4�pd��t�F�l]-��
X7����K6�fS�k~

x��������Ytq���3t4�nM�k���?��K��|N�����o.��q4~����^(�[A����h�?}8���������U��Q/��w����):�L=j:�����Z�Zd ���lN�V�@7$c���+Ub-Ubs��5TQ��@�k1���1i���}F(�WSe�^No�����G���J� �Z�H�"�h����g�����d������N�����x��w����|���e��F�l9�V]�����\�������o���P@���{���������X�t���n!�-Fp��!�����
��VE��-��oh-��vL�����*�QK��(]��2�|*�����H��(YE��}���zU��[Uc��W\Uc���o�����UUC�j�]U���j�]U�-�W�X��j��yU���j\qU�����E��{YUiU��U5V���zU���j�XUc���W�X����U5���kVUL��C�"�eV/���
��'r'�w���u5Q���+T�WC��&�GuP��}
��y�}�I���3���7�<�k�������#��{�������=�����u��Cj��d���aX�����;!S�e��RnN��n����ZJ:�t��}j��g�8��=O�h�Tdy(��5!�.qn8�(mNEQ&�Lh2�=�V�'�@�\	s��2�dBSO��.I6qQ�|<�=������
������[,u;%K9�R&T��7���s��W�����+��ja�_�U�p���&����S��8�| *���Ga��v6\R��*�����	�2���-Nl+����r���{k
����[,|�q� Ls�iW��R��9��2�q��/��H�a#5�<�y�*�������A��-���x��3V�r]�F1�}te����sC���N�H%�k��c�}����������������}	�}y9����D��D�}I9����}��zv�goZ���#��v
��Y�}�}�����
D���oP�}�G*q_F^;���������x�g�.��!��!�}C��[�}C���z��s���������@9���Zoi<T�4������E+��#���Q�����b���}>����zsYB�?a��G�qwu�5�������#}�-����Xjx����G�.�=���u���Y"k,;�5!����}gP�`v�%�Dg��`�r�%�g��1���"���qlE����1�z������
a�Ur���]���|�L�����(P�������v��K�x��#*a�~����*tP�]:VTn�y9W]�QF��F��^�� ��3�o��A�>5�*�!�cO"� R*��w�����:������b ����svO��Z4�3#�M'-�[��=/�(��4Z�|s�\�0��c,�P���O�X5�bqr����&�.�F�n���U�wl?�t���erY��r�\
F���\��i��GA5�Qo�+?�������l�~��\��s��g�O��������N.�����1O.���&������t�4O{�K��������e�(��4�sr�����������;W���bq���;W�������v��>Zur��\��`��2���\jF���\n;W-xTc�6�a�L� ��a�J��+Q�\��s%��+��\I�����\�G+M.�A����zr%�w.5#��O.�#�}�������c0��;W"�\�j�J��+Q�\�n��~�N.��K�h��e�s�!�����s�A�^kr��\��QP�i��o�������>D����#��H%�����<�I�;'�/��,F���p�DTx��|/���������h���>�6����cg��}���{zqv~����/�KT���#���	���9���������,KWk����0���8�C%<H�dB�	�L���H�9Ks �s� Z�B���z
�aggB��g~������x{��{��Uys�}a��)�J��u���~���M����^f��<��Lh�w|���G/���������f��J�N��0�h6U�
�442����'��@���\�����<em�|i���V)��&��d)Z&����-Q��f��-�5DO��K`l�O�"���f>)���d}��1A�c&�9&�c��I��������\sm���Co�F��d��Rf�i��f�
31[a&�fb��L"V�a����Wy�]kX�h����'�X�T��kE3��4��� rS_�c|��2n�#��M5V�"b�dD��~,T�I��b�+~�����t5�8r��W:�<���H<#��V��r!+��jF1W�(��%����4WPW���J�9 )������"��
�t�a�����6��� rS_�c|1�2n����M5V�"}j��Q��~,TtJF����<!?�\��&����t��_�+��xF�9�tTe+g%�rV�U�J��Yly,i�+�J�+g�����MJ8@ 8!,�%e+�bX}�t������4Dn��x�/�V�M�"k����JW�\�{E�eL��B���/V���	���JW#c4��~���u^���3��i�P.�+��r��[�"oj����jy��J�HL��I	'����l`�S�����bB@�K�"7�U<��+�&]A�j�Tc�+���"a;&�c�R]��+]����\s���f�Co���i�Z��W:#���sZ�����J7�;�V���������Z���(��jIR��	a1E()X���k�3�ji(|I�A��������e��+i�M���Rl\��a�D*(�	��>'��~&�"!���cR�^�6i"��/��'E��G��R�DiG�j���/��)E��X��R?���4qW�E(�" K���RDjiG�jB��/��.El�����R>���s��Eg#�:����e',;�_vB���"/$!��
�!4�F!4�:���P�OB(�X
�!��V���@C����Bh�@en
�!�V�]!���27��j+�����P��Bh5�@uWmu��M!4�K�H��u"�BhXv�����/;��t�������h���7�Y:�eQvw��V��b����h��Ui���V/��7���i	���*��p��mD�
6�M���hJ�(����/�Z��O�b����e�����ye]�]�\���YG8>��#�	r#F7�������7�
{���������"�Q�����)�����R�|?������@<���8�-wl<�aG���;����!�1�}C�^u!Ie��C8������`�������`6{��X���������6��a`w�[�f�����������n��L���/c��3�1��W�g��nx�QRH�Z���!���m�n�:�T�lQ'�Y�Up�����S�#�tG<:�d:���
'��8)O�����q�v����@3g�L��9�T��('�9RUp������#A�W)���?�5�5&*o�&������sK`�!Z��?��<S|�ck������p�����=6�?��H������fX�6]��kdg�Z.n����7��,E�-�����r�]���p-��h��������;<������j�����	K^�-t*���A��r����h
_|@�x1iF�'�&�	a������{sOU;
�u[��dA���(sY��%�'�E�O(����B��l(+�2teI \��XH��������0���m]R��`K.�r,�V�@7�	�1@���P6�
U��Z1��I�Y�BQ���%��b#eR�Y��N�'��=##_��H���K{Lb&�?L�&NMh�=&���&5#&4A�����?L�>Oh�=&C&C��@��w��Sw�'�v��b
���a-c��(][q5���.��t�:�#��������5���\�j�W����.�:nt�O�V�1/�
�$��''y���9L�p�L
�7
2CC\�@8!D"R�Ra�����xj�W�LF������/G�L��R�b���})tVp?��3C�t�s_�A�~�]-������2?��C����dr`�K�!5��G�����:?��#L���-z�8�u���2�jRIP:�����Bc���[N��rb������r�G�[���J[N������~��
[�WgZ�M�!�r<LUx�
�0^�:e� �"��n����.Vmu������.�mu����{���i����Ek��x2�nu�����#�����:mu��&����)���a�)[��VGc���������Y�0;"[+��D������.�����j�:�cc�O&�Y�hf�Q,g��G��e�w�u��&�c+g���a�����`+��g],�:y�K��.Q�u�n��~��:��N�h�Yg���3y29�:{��G���or�u��&��f���u��9����e��;E�c���r$b��*=���O<��*�;4���'7�tT���'�$pX�W<��l��g���5����i�:����{���"��-�D�iL��WI�n�`/AU�O2@�i��%��#P���0�^L|�b���4�RZn��_���e~��a��6�������N_i���|������y��m,��X9o%�\v�j���rnv���
��SQ�F�mug�	���
��/m��n��E/W�������2o�������[ih��pF�����"���m6|i(���_����y
�5�y��V�oY�0��T:�4��l��s��}�53>v����y[�cmA�2z���K[�C7o��o
�J���e�&�~����D�o�~�r�~a���s���5��v�������� a��a���-L���FW3O�J�V��4���m,�[y�e]���U���������l�S�����wU�F�mu��	���
��/m�%�����'q#��}�"�M�C�W�Ih����7��0����`�k�|�����3�^��6\����%�S5 W��?�-������0��m��*3F�����P���t���DQ���Zn���
�s�Vp���b���e����|!|[���L���{����M���E��?�8-C�k�q
9�e$�s����M$�>_h�G���?=�t��T��^�sl���z>�
�O����,���l�[����O������]�M��Q��Y���wZ6�F� <��.e�����@�mZV�6uxL=f��X�]��]��[�agL��?��r�f��uubL�6p���L���LC������P����DK���r�>��SI��r���w��T�[�g��W�[�SZE��YNTv�^B]�5�;��;.�3�����t���p�C�3�r�~�s��<k�G%@Cp����i:��wV"��q4`nE��X��K'�%d�����TC������}����*�9\cL�(��#�����f'x<�|f�--�KNDv��� czL�=��������NG�����C�^q����t\�������^�zqf����b1��T�5;��Q��h�xiI<�3��Q�cg���{����:��';���`tw���{�|�w:�}���|������}�i�Qj��Eu�P�s]���s��P|s}��zs�V��Z-Tf�N+��a�
u��G;�c��R�o�p�L��:]O�`BO�U�����U��s��c������F`�n�`��Qvs�����������X0B�-6��S���$����bz��+�U:[�������e*2���&<g�E4c��m-�����jdj����j��X.v+5�a�/Vs��f�������fI��
�~�_������	'$�
��S\���KuL�pF������������u�i�����k���7M�O	�b�'����k����<.�^�����k@,=8�U������.
��������=���L#j��^����j��&�,����&Jo���|4`n
��&���7G5����}�O��K��u)���i�{���p�
B�� �GA�1�����?�s�E2x1�A����l��K//!+����2��K�^Co�����j:����e�"EO_�k�$X�m��7��:��?!#F��~4���.���t��BGg4����5x:^���^��.N|{�������o.���i�������t�9�����[�-Z�����|��@� �}Ig@�M�o~"������7��f�g�����0oEM�8�~#���%�7�c%�cG��J�K�hs�WR>�(�$}��>I�#��������N��Y�����j��.�������������[4Y�W@���p����]�U�W�,���
~�\�U
��P�-"�N�����~tw���������y�?�!����[E��:�*%��J�������
Ete��L]��DTB��O�A������L�����d�����Y�SZ��J
f/	;���w<b�*$����o��7�r��T	���	'�!�����p���AUN�9N�+sZ��i��8-���u��W�5����k�J ����[7a�w������;���
���;�n�5\��>!��}�����S��
D�	��;����7�X���?������(B�y0p��O����y�.m�;������}@�a����m���-M���������"s��=����^��Jq6J8�9��]��l�%;2�
6�W�����"��������G�����������=�r
M����{����RC#eq���7��^?@���p�_�L���X�o�����U�^=��Ib�e3�cj>�1(	�Y�!�N����#��;�\���j��g�8���Cu�e)�\���$��I���r�����Z�%OH2q�\l���Z�	2����h(��u����h��$��Ait��p�\�����4�J�&&��/�,����W�o7~&�F��������O>�-��������F��;�O�!��3�p�>����&|��M�M��>2��>TT�>��`Q6������M�d��v��4�0����Xh���M`���D�\�i���/���:���^������e��4�
U�����X^fK��C���(����u��R��+M�So�����2��5�������:$A��}vo�$�b�j���1�IV2�a�W.^��e�E������Wu��e���
{y���,�BD8-�e�Q/���Z������/�d��(n)�!h��b��%(n��rE�������	�~�n�\y�>B�����b;��0�[�+X�F7+0Z�������p�
q��
<�p�8��������$qt�
�8������!�=G��<�T�}������Wg�?rir����L��[BLD������Q��]kx��t�v�����K5�<�J�)'�J�Z�N(�"sil����D�>�#^����Lck_����U��5�*�Kv��V��E����[d+�^�u�ld��0����YST�rD�l��`a��P`�������~�y�J�p��j���XL��E+)���	������.TKTR��'d�2���?(����@���J��doU�^�VW�k��
vX�qU0�����%�d�M��t0�z.��!n����f��L�6c4�nZ�?+F��h���)~�oE�O/>������+��������qQ������[9�]�p�S	������7���+�<~�+�u
�@oX��@�z;�1`V=ip �&w��-���H_�G�����������������r�_�F�w���;O�Y���P G�T�����cYq>��4-�:5�|fYR@fSG �^��vJ�I9.�ak����%����|5���:�7��5�9���:�t:���:!�������������`���a)�3�)��m346�6�x�Jgq�g�N�q�<S�#���e�X9S�������)JC���Y/��u�)2�Uf�X��)bX^C3ER?3�������)�~���yq�<S�vkf�&Q���j3%�fJRw�$@�3EF��LIL3%��L���L1��dM����gJ"�:�u����[(�����!���LoiR�a-L���Il@8�Gn��M��(�o��0O�O���OC�CZ��R(v��N�HL[A7������n3�CZ��Z�XO�I?�Gh+����
�k#F0���2 ��I�+��a4$��$��
����Y��l���45�6a��9���|�&�����t:�@7�"�����CJ������f������2�e2K7�\���d�N�����'�����<�=w0\l:���y5����~�����4�N	���o�����a���~��'��s�������q�bJr��Uu.`����XA|Q�������I�E@b����/|Y�(h��>7`k�ba�~3������7�=
����QZ)�Z���q	������J�h\�c��(�J�(��
�+��A+,/��g{�����M}����7��_���)��X��1j*��9��Q]����]-����HY����\�O�}aw#\J��UOa�����a4w8�'�$@����+k�UtX���F�)�b<T5@T@�6�X����wr��<�B\��8`n�F<h*����o�������b��p�8<<�(����W+O�>�IQDn�+y����E������t�P�p����K����&�:��8-��R~+s]��H����4�P{	)`�(�Z���$���4����N�
%��>-�=�3�q�]\E�
�
z��i��0��
!`�|=������\4����MW�U�\�}&f
sa*%���c���a.SS��!jj�����`Uh�������.cm��g�0�elcF�(���A��R�y����FT�7�T��vJ ��y��s]Kh����)PhUVt_�DZL�z�����_wz�&��o*�]kt��a=4�b,��W��%K+�E���>������m�����Xy���U����"Y�Y
�Pq�
#UMQN5�JX�V�/���,z6�0��*R�.qI���dqs�����Vv��E�-�K�gU�S^/3h�,Vn�=���3V�(��{�}v����{�
Y�C�*�\��)Nwt�p��<��J'l-�#@�'���P�=aK����?�p�\��4w�p`�<�4�����62Cl*s�Ff�M�	o<�O������&���U�:���M:�'U�P�t��dPX�o�0��2��YU���R�����E��E���98���-�aGGr��8��4��������+/E�V(����J6k��M:��7.h�KQVQ��X@��.!'"��Kf����Ja%�����$9�nL�2X��LhuH��EO��U4�9��
�3tEd�%27U<b��abp}��e��rhU�gu%Gf�eR���&��9WV�g����b�Z�T����}��a�#���NH���EQ�M]	P:�M|��bfY\�0[��Kqy1>+�Z�5��K����S�e��v�G�:��[*���'�}�_J�r
:
	lg?�[
-�������[
m��7Q_�z������B���M���^*/s���P��z�B���0�w���M��a��	\a.V��;0�����1�"A=E3>�wat��:�2����|�����K��<�H�.z��s��x��6A���r�����L�����(w5�J$����.>��B������Bma����Sb����BY����x��2�7f��&�.6���.|�zT��5K�����m	����o_�/���X���|Y� 5�6H���a�
g��������<O�&�V�1�����iy��0c�a&��`�4k]!�?w	����X��}j�V��b
}�9�P��<�a{Ugr����f��i
l�M��S�;��g51�N5�	��c�b<�cL������X6SS6]k���!���M���p�X��pZ%���3��������*��
����%�5Elt3;Q��2���.��������Z�K��jY�S���i�"��3��A��`=�=dZ�Z-=������G�L�[E�c�%q�:,H���:�S��y�;��1����|�c5�q4)�'Nl��RJ�����Q���V�Xi��Dr�����(��-�-���1lr���g��i~)Z�����m�Gn���.����t�H����Og���%1��!����_��&[A��V>����Z=�3@-l������L��c�Y�<�a�X�]���([��*v����=8P��,�Dy��^{�4���+Iv�g;���B����b��^G�����3�VOCz�����y�R�����F�����A~��m^��%$�)N�U�BUW����7@�J�_f)&g�N�U�F-��u���z�s	=�E_��W�
#�/X�S
�y!=�������R��W=��C1�P��|N
�4�����_`\�z�rW����v�����<*��*�<*�_��}]��J��J����XD��~�#G�zL��fT��?p����0;�Sb����u�E�#A-2)�2@�����;��@��*Ibk�xd�t������s������*�\�X����!0ia�y��>�.��??`�0����s�������A�0(��qP�C!nT7���(����y�gh�&�$ecV<��%��w��Y���g
��Y�aZ��:m-�������B�s]��o���/���lSO�U�0�il_�;�6{��[�C^��>_+�J�t.�l�g��_J���:[��������"Y6��eE`?+?�R����;x�<��K�������T�=��7��'5�gO�����m�>6��:�C�m�t��W=�<rf�������	��<�!�_���
H���6{�#�������r)!��mI���KpLm!�lh���8��Q�5%[b6��q�iz{l
��8���H���c�eLk�
�K�^��uG��|j�M�Y	y��8*J���x(�J��d�1�w�G�$�����\j(I��ei�����e�p�M�2���&����G�'�X�� ��
biK�+��F��X�q���=:�d�K�W����R=��n�Z���n�n�(��?����
��Qs*������Z=ITO�r�F���"L�����\�n��<I�������]���V�����"l
���y�R��&�W�_�y
�t�PLt�b<b'wz��+����*�v����6U%]��Y�V���H��Xz&��������@jG��GC�����X�m��70K�~���:�U1"�Z�V6�*@�(P	E�I����r.Q��x�9���<a��1����L��:�Nu�s��nv�Am/�V�����q�6��5(�UE��)�H2���Z{a��[���^ic��e�A�5�*�]��y��:k"���Q��]$����hg��NgZ��R)�?����2��.=�M��3���f��o�|S��6�n���Ms��L���n
,��[�k��WvO�+�Xc�67,/�5����l���o��&]}sv��Z�F79���Gv�^_��ie}p�^R@��,�����4w��&�p�h�
�w��~�h�m��al�h������n�e�����s�����2�Q�$��9�E>���"�����VA�B�1k��)���*�L�v��cl��_�Q�,���|]B\sD���$����&;b/�z��v-��"P�-�1���=�)���g���Y�Hm��G��V��"1<��j�^��B������v���>��:��+����V�On�`����O#��69��2�>�;�P��Wge�z��5$I� �S�\k�����{�e�2&f���0���*�&����
>�����@c�1/��N��8�\����� �,
mcD��x`a�%����$���RMMa@K��0V��^j-�tsV�0.e������.h�e\l"`�������������fa�W!����Vi�N�6H0;��"[���*L��� ��dq3��SXE{U��������(E=U�*]}6;�����2!�����
�����P}�<��*���GW�OL�U`�S��6���%����	�F��*�������t��
 �@sY�4�m�&�KQK���k�F���L_(R������M$�KQ��f�%cl/���<������%��[@gc������j������U�+��5�#��z�5E���vb9�k(�k���v�Gb;C)�kx$�#dhy	�W��U���0�M�j��E����5��
n���H>K��iJ�)�A_��x{��~{�2�Am��{�j�����-�����V}����>�A c��T�Q��O��� B3�r�;+3b0����</[=�\��>��P(�b��h������.A�`��u��so���u�����{����Wm�������YJ�|��m�#��lF_R�����	���
���B����������J�U�� ���E�Um�b��=&/��k(�����j.���a5e5�)mJ���QeI�*"!���Q@�(_XH�5[�8����jB"?[�m	�X	����U�s��j%������v&�Y�J�^^�&��%���:Z�������c�	4����V��\&Z�����W
=��*NAf6l[���E����9T�������T�	
�=Qx��/����|�
��Y��uJH:J�+B�G�:V�?r���2���'*�q����R_�f���t��^�_A�R!{O/��I�s��B&�������%J����J�U�����-��B�l����5x���<UAqS_�S<UA���2x��(���V)L��S����Pu��!z{AY����J�A
>+ojPK�����'����J1(O�g|VA�����7��*u�����T�\X�l�����
j����<X���0k(�K�rP���n���G�6��Z�)�z��V�kH��]'(8A�	
NPp����O|'(8A��
����{��0}hj
^�N���/���
�CH.����Et�.���`�N��,ZMon�������_��6��9%�C�"��"S���Q`T��	��D�W��cA�Q=m�h�Q.��5�O2��8.����u�
��}IB��Bd������b�����8�#d��i3�13�Zq�(�Z�d	\N#iT���H������?�IA&��6#�0#�W,��F�.��4����H�{��8��(�%�$���+�R��(�}N����}F�)lO��g�3f�������P�/{�����-;���V�fg��l�w6c6����MF����FQ-+�C?���HY�_\{�7��������*���lcq������R�'�y����:���~�����K^]�?��l��H��:Q�������)w�^�!<w�$*tY����A�yA
z�d/K��Ac|�3����"4�tVZ��6�6=LlD�}��Ft!��08�tz��_bz��%o���h��"-��[D5�UP�'�3����
�3�Xc
����*#��,�+�(_k~�&�����.B�m�q�u�T����N���?��D��i�[v��4���g`�(]�b�
�xJ3�G�E@'zR9�����������
����^V >��b��tz^��\V s ���]��V�2��;%�V��2*�Z�+'�m�I���W��1��tJ�+�)����Iv�uO��
��+���h]k�������%�AK+��z���S�U\����i���-�k���2���.��E����h�V:���\C5�����B����yP���5GF.U��)g�j��B��*f����_5������FPA��X���,]&s{����|	���J�/�����5�h{/cLP�lvw��V+ ��`����Mt��O�`��M���m�L���������IBN��8���t��~t�oG����^6x9z������J�F&��6���a6BG�e-���y���C(
C��Pc�	_��	����?�Aw����j�� :}�Z�����c�0�$�KD?�@�V��u�)�_���,Fx�a����1.���b�%�Y�)��Uq���rJ��?������Cx��E��:��@��@��*�O���b$�e:��L��x�������������x)�~8
���OL��g_D?�x�TB��-s�E]�v��,�F���������5C}B��^�C�\���N+�0`��V����j3��q��hOA���9(��T��U@�!DJ@�l�����X*��-��64�r
�*0��WVb�|U�)��#�c~
.����<�x�	_�A4F9�F������^ tl���q������	/0���?��`�;A$��1*��I.l	2�����Hdq��R3���E��Gi�����E%].�o�}�Ei)�;g���/g�gT�;}�������?�K�}<�����_^����_������`�"q��t��ul���H_)�>���@vn�qox�QsU�>�}g��47k"�yV`��Q�U ��
��'j1�E[����
Tt����z���B� ���/�p��������YR.m�� Rbh����E1G���ic;v��d�V����!uN����u��T�l�j�&�z�h��%_�82W�{�X�}�wj�ne�.�q
�������%=��[n���/��`�.��-�n�j�F�D�ii����\�/���,+��",T����(�Q����(��nS�j
���l���z~��)�Z��u�s�1+��9�D`����������3���?�^}�@ �mi]�J	r��}����uu�v����-���M��)�v[V�:�H�l�X��I3���rGc�y}v����W����w�XF��p�����*s=$�u�+�	G?}8�>��X�dtU�x�W�Y/���v1��|k�E�=Yd&��%���_�/1��w��S0%yh_��_#������F���t0�-!c����z��uj�Us��������X|e�@G<�1�h:�#Kd�V��"�K��	��+k9����?�X�
e�P`c�?�x>���������F�BWz�z��Z%�jA�����������&jO]�%x�{^}��^�����~}
���@"(%WP ���@B�((jD�q�F�|�D��c(���rn�x����/����[�^���D�>�� ��
9����t�����_���o1)A��!}���uZ��������������7��_FWW��At~v����G�W�R0����r:3<1"����DH�����	{���x���g����$4�#u�n^���>�}���O�?�E������w�r����V	w	�`P�)a��#/��F���w��\@�>���btyv��2����s�0e�D��k�2�_g#3��{(�r
�By�7!�w��I�
&y�ZI����w�.N|{�B�E�s�"[I���VE�sZ���_b�q���C�u,e��#�����f���O�Y�W�t�F�l��)��j]���<AO��;�8������F=&; ����t�:������|�a����M�7�<����b�B�9�V���c�����g������}%��m
���3j9zD��*?�/gk��v�,9+����5J�t�X���Q�R+Z@���e�����
X�,�;0���t6��LjL�U3�i�v-�B�z>�K�Z�������LdX��s�f���*Q��h��mm��"�$b2��N �T�8::*��E�%9�/'��xH~��p�p���/}����������Plc�	�Q:�pSyh��a���������{	~(�%�!��K��������|8�RT�A��IQA-K���IQ��%U��B��]���Jn�>c������z���P��e`��`���v�V-���^@,@4��er�Cq-��,���H�Q�����pK���G�t����no�u�� �P���0����S ��a����_�������B0c�p����Yv���u��&$���V�q����$^A~*3�O�D~��m���u���[��t�������z+��rz��m���~l�'�0�� LaJ4S��� Ly��a���8u���Z&M	 ��F|�rd����%RA��� �/r���QA��/I��_!je�]�P>$�����y���,I�bC�U�T��w&���t��zp+���b�\���vn��C>T��������������G��#����y��,����u�Mu0d5�,r(\��R�A8s\��2U��������@�C��^�Y������H�W�|�����J��6!��C����P��n��X�A���[]��m��P��5-����$x+K��o�C}�J��^o����m���|3$�_�%����V|i�z}9GEu�� c��n�U$R�y��������*���E2G�B��q�#�5S��(1���x��T7���J�t�B<0���ro[��X3����'��_fw����v������4�I�L.3�����Q.�y�z�����
�����5��i�Fd�B{�ZG��V���q��L�����6�t���Q��i��mos�#��	e$�����dA?;4�,���	$f
�	e
�k��Q$HM��A��������������,���:��`k���|���|���m�O�@��|{$�!`�*	��HPm4,DB�a�xL����-�#}VD�:F�R���m2F5fNEE�� &U���4��~�G[�{����},��%������6S������K<z��cO?P
<�[
k�����$���?�A��V�x��+���R;K+=��J�RXi_�*��,Cy�������]�X"��V�$mIG<��F�t��"m�HG "F�x��[6��|X��GW������8+��sg���u��-b�R������7���Ax���0f��@�_]W��+_O�KN�.�
	>'����a��,�82�e�Z���^2�=j��������
p��<�r�fcL-C���c�xfwOKvwq�%���k��F)�cQj��i�Q�WoT�����U��E����4�]����S������g!W�(k]���q�k{���Su�N.A:9:'����v��P�q�nE��>�+:r�NG��%"$�E���pl��i��(�j7FV�V��<��� ��dq3��SR������58]-bR4��-?u�e<�&<�O^]I.]R�^�gmX���^��`lc�,X�>,�B�t8�
�M������T	 �2e���Y������%��Ybq2qK~K�����*����m
�[9E�������M�l����=�^�B�T�V��Va"��`)E�u����/���W��G�X��v��{:�|-�QkF�VTu�v�����2������]
n���=<Q{7��~�z[������#Q�R��f�nU�� v�;��(���h(�����;�����w$_����U��c��{8��-VilF�V�bw�����2������]
n���=<�{7��~�z[��+��~#�Co
1�9e��S�rA� ���Q������-%m
e8E�p8��p8�Gi^��Yhw��S�0*qV��BY�#���3+�L����|�I�o��RC��>t�z��CG����K;t��j������o��Y5�%��44"c3+������(�
�P�A�pP�`jCPL�<@�a'R�}<w�7T
�	�'+
�����jN��6�!
jC9�Am�"�;����,Pm���r����j
������c�5M���i��+(A�0<8�Q,�

DP t�BM]����}�CS#T�H��e�7���D}%"(����D0%"(A�J���)��Y��������68��x5�7�.�4��WP��`x<�6p��*X�T��2��*���*C����$o6�-0���m�$�/��+:&Qrq��.��t���t�u:�e�jzs��g��H��p�3rP����1�?~�z���w\���NG��]�Fw ;��?��?'��H,���x�S�5*5g��x_�2�7�8���]�A� U�s�������� �H��5��~odR�@�HG��z���I0`X2a�n�B�V��+XZ�W=��u;.�������U�z�N
����r�"��v7~&	���E��O;I�N�����*��s��(�_TR���d��/*��6�RFq*����H��B��]T�b�$`�U�g�rW;D.KQ���e0�bH)��}7;�#0�R]��Hzk��r���Dd��*m����u�c�gbQ�XV�l�A
2�^�@A�y�����4Y@1������%���]�/�
e�a�7����-Q��P)).@��%d�*qg+^s��z���=���f��7A�����"�K�?��89"�\��ru��Al��a�v��p���� �p�K������X��]�XvtK-����({Us���`���pt�������b ���������J#;��ds���^��4��s@���9��4��sH���94��U)d�����	�g���xec]Ct��@5 P����jP*�����jH��.�PTT�;c}�������"�+��u�\�����m����nu�6WdsE��i
����z��W�M��m+b��*u�L���T����_��V�n�j���'v���f���]�u���mdw�B��o�����D*�{�K�v�[������r+��r�]��V~W@��_*��X	�m�,N���v_K�w���z�*��rQK�O5%t��TSB��O5%t��T��[����.��
��+4��o@����6�	�W�����Y�����z�t���b�\�XTe��������;zD�Y{B�b��"��@��I.�Q�[!�;fN�Z����dK5���B�?������������kY�F�����p�L������3���
��	�X�m��A���S�qtz�::<87*84���f�G_���&e��1��hE�\���|�7wH�	h
����'�cD�e:��L��x����o��\
��D�KQ���k���bl_�}|u�t���S!�j_�`SP0�r4�?d��$
o�%���������u�M�#4#lU9��E���R����O�A��9��u��}���l����0v�N����Y?����c�IR{!���'��T�6�u�~F�������`6��2p
KYB��h ��G"����X0�
i}/�����3a�k�1#���e���o����&�<[����������vO��O��_x��c��DG�:P(H�����������RzH��������H���evM|��^-��?%R�*�X���F��v�����BW��FI����@�9c����mY@e�%M+��P+v)��YIBU���_��5����+�_vj�5�(l�D^,e�n�����*��U��cTg���l�g{?Ac�K���Q?�&'j9�}{�Os�����j��:��D�3�<�*\���&>?L�3��r���8@�>�_�m]��F��
�|,|&U�<t���-��Q�������m�9��Z�.�?CJ�^_/��t�Z��F�7�'���%��%_�8�V4c��������z_���k���`��i�}z(�E�s{Gx����/�"8,�&,*�jG���%%R�cXHMHP�U8�3�o�
b������z��nS��j
V��l���z~��)����u�s�1*�N&=d���a��
���E
�;Qn��q%&I8:����OH0���g��YU������Ya�`d�	�j���B��s�3��?4����h���V���i�c���^F��v��4�v�TK����b�"6&������� y$�(H��s �
�FArH��U f
��@������g�o�~xu��}��W�e��w��,��2���j���N�����>�G�Y[-o����rX��E����^�������q~7_]��I���m����1��h����7�2��zt�����^�����
�������~�r%?�@n�=Zf_���|iw����e>�����N2!&�B@����;���P{���V�QA��8�*��N�`I�D��~���6�����b��__A5�yA�r^����WB� �9O�9�+�����T����&��
�����7�_@�����
tu5�^}:�B�kF�Pj�����W�D�o��A�T���zMd�s���������/8���,�'����	��O�_]����2��Zw����O��?���m�}���XZ�#����OdD~x=7ac�^��+^����{��-��uH^Dh�0���.����N>��^����(����6L��:�KF-�hP"�����H�>�{��N���*���2��-���j:�!SH�9��-��u6�^��|�C�]���e=�Ed���GO���C��k�`Eo���tq����l�h�|�=��Q�+�e���e��O q�E������_��s��a��s�_�W���_�]�/����]�?��'�]�y�����9�����P�������CN�Y�w�t�F�l�L7��z:��d�C�bQG�������n�@����f����h������X��C�b��(���WdN�:��)f�1*�#����~�� ���D�k)�g!�b�B�8�*
��-1s������������q/���@��Tb{t��7�;�	�Y��<"����{�	(1)jW����b�@+���0%%V���60%>`2�N�����4���R����/�n�����!����R�����������/Z&%KHP&.P&��44!A9p�r B�M�k
�5����T)�K����#Q�����=`�U��>s?�g"Ti����l�U�D���V���i���f^���\c����b��v,
��������5������\/��:Np1n3�m���uB�$�	��h�>���V�Zp�������^����1�nG\P	�s��C�b�Q�T:h}8���*�U����d�8��vY���)0���@a���.^\�?!��;B7L+|v ���-�$}��.�����.�/�T�O���S<�)K�f�KMui�*����@��=b5�Q�X����jm��%��gBr�Q�"�m5��V�L��	9�,)9��FEKl����k����������f@�^����V	<���t0&�0��C��fI����U�/MJ�8��N�eI�8�/%'�;�t8��y��9������~��;b���3
5� �>������HX�IV�d%GV]"e�Yv�������b�P�x���0��+O1�6��D�����
���FS�;�����y
_%N����G������ eb>��Z����9J$+��'(�e��X��nU���b/q����FD�����G���+#Y
�����
�A
>�
)_W��r�IU��9)�8v8��=��O����������1��2�t
ef*hH�ag/f=Q	c�x+�e�oIh�j]:��B7������RGma�%��E�s�{�,u�
5��6��RW�a7��O���G��#�y��\������sN��o��N_�6�M.uU��:�Gm���]�)�)_hYR$�t�W����u��*>���Rycw����!e�;{�����$��x(������;�#������g,�d��M�u<���;��U�;9l9�����"�T�f"0����S0�l�
��`�	���
��`��D/o,�`��E+o��&o��&o��x�@��`���7W���%�7(�q����d{�G ^�l>�����n������Z�/�3�4����b9���VDo6�J��_��g/~��d�p��K>V��H��hqC���}���v�EO���J��j��Z���R�1�NU[�b���W��_� c���[�4pf���C��LO@��Z��|]X��,K"_��w�|�4[��y�Vl���'���A���'��V��;_W��5b{
_������I��:"�uB�:a�Zi��E�e_e�����A�������2�Q��Y���4x�+�l�^�a/E��3
P��KkP�f���:Pu�z������!���>�M�H�#e���3������H��v��8h������h��7�t������$�bT��Y����lPf���f�[�J�DZQnf`c{��h��Ra���`mQ��	:j�EtG���8�(��u�[#������G��eu��;i�54z��H8�n5��j*����(�.x�3k�fRu
v{�Uy5ee��}5�f*�qx�j�m�h��L��>.�,���n�Z��.y���p���0�Q��	������mdS��i��}FB=�V��E��Xw$�~��k0���h�%b)������nm����o�(�n�
��
Sv,���'n�<<w���{)^:�\3u��r#VaBF8�d��2!�����ll����5�wh�2��zL�Fu��	���k��W�	e��	�����G��pz�U�U���y��������5�&u�4��� �ou����#����
�[{7~��V�u����c�?$��Bv1��T�����z�\���02���k��tRm���RIcq�U����DT���(ZI��9�=�^
�[,��}�����^[I��%�W�}	��B�������`����g�R��6�vh�T��)� ���C����
�2	�8$*��|�y�@�����<v03���r��4�N`�%�>�^ss�dx�e|�u@q'�6��9�n�	!6/��;�y!6���������}^�t	�� |�|I�0f��On8f���c�On<f�4b�7�{��9q~�(�#'�N�@��8+���sg�!Q���%Mx�l�ZG�������E���MT	��%\��h�~�
�dvN` �+�����"SMqI��
L������	;T�I
����V����'lP!#5>NX�BB�!�.���](�%��$�^m��8uS,�R�.N��\��6,�SG�r5�
A��S��R5�8Z.����eH����BJ�?22#x�ABm�eD��e��T}��A�U�,n��y��.�+xs�\�VHVg����!nY����	�g�g��!���Vry\*�kmS�W���������;�(bU*��q���������Y9��X��
a�-��'"�%��-O)�yu2Ob����d�d��_$n�-	���5x�������	������S���I���SI�}8K�������Ezd�C)buBLb�����d�	1�[BLB��!c��$!��SJKB��(�����R�.n�%�W���@�k��Z���C��=w���w�2wu�{���G���e������^pW�7{���kU�����:W�5T�;��
�a+���E;�\�p(�k+
��;l;C)��a�h|5B/%|��(�b�d�-U��Q�3
S�xXss�T����a�~��z�jYU������ �`a,�z)�
[�C�j%����~$_
��gf,����������'%V��Z=)i�zR�}+�����$_
k�gfF�`46����������Yk��*���8���P����f������a*� c
<�����1�����<�tG���~+������2D�!U:6�9K��,����=]���S��t���!>5�6����TO��W�O�B|���`j,@���+��Cc04���F#���O��H�E���Sb���f�(L�����YX��Zh�tk�U
���bm
��"�!V5���X��bUE��9�=��^!V5���	�e0!�lE�����u+�8B!V�l�pu�����P�wG!�!�����U���Z�I�����L�J�������,l6�n���J����M�9{a0�qE�q6c��l����m9��pr8g��p�l)������3{;Z���I������H��1j����Z3�6��U������L��mL��6l�����;wEkvn��W����[�e��p�Fj����j?"�w+�N-�;5M�0*T2'�2$8�J�������3� o����vI��p16�63U50�5-�6*�����f G������������S��S���b��o��d�q1��7�T2��M6�AM6�����������h�H��6��nb�C�b�WE3OZ�A�!h1-��E+��m�	A�-��������;-ZA�!hQEZA��@�����WV�`�5���]�-��\��" �����_!|�Z�C���j��!|�B�!|����h�B��~�K��������X
Z_��F��/��j~�&UJhd��!��h��Me};���+3�`���C0cf��!�13��
�B0cf��!�������!�13�~�������F6X���e����q'v�z���;X���;X���;X���;X�����Nm@���������]�X�~;�!@�~���j�~���{i�f�4���;�8���t�9�2����=D��U����n��z��:���,�V��[��<�d����l&R�� �xg�5g�V\
�W�m�v�����^p������`��ce��_�B�<�i���L{����x��T�Eu�m�	�������bo/���h�:�_��z�����X�=�k�P���>*�0�X;���D�C"�!�J����@�D�C"��T�'l�e�������V�s5������|��N8����f"���}�c�=�/��>�Z%��/��	�D�D�7��x�2��'��|��'�����:���)�&�f�E���@���
��3zEm������jDh����~�T��Bb���k���.�Q3f	�jM���L�b�>xs�7'xsl���7�Irn���{�Yet�,�p|����QZ��}���+��-/0
���uV���&8��!���yDV�44�`�<�J�_��^^��	a��m�����O�y���V���*�K��"��Z�H�

�t�C6��x��LR�?�������e}a{�X(z������I��P��NIj��0��v?U����wl��1���t~\��ptb��	���t~R��p����g��g��g���G�m;N:N:^����v�-��T��{��@���)B"�������R����j$��� )��R�l��S�$4\U&V��������@CT�"�P!���d���K���SI>j�B�Bj��en���:�X(�W���".DDm�b����
��
��}�1UJ`���R)C�W�~��|_������G�fP�"E��%�r
0m���n��%@�	 ���b�-rB9!���i[@�40�� � � z��D�|-���������s_�~X�vo��z���y��}k����������#_X�^x������(�_^��$��v$}M
�V������0������z��Zd�/,:���DG���P�(��E@gw��>�+����v�/���ov17!�{[(�� �f`@,�� �&a[@�� ��a@,�� 6���Tx���$xr
��K0'U�x�l�\xgC��dEs��*��q��B��$-!\�0�@8��b���]����L�
��p�!�4����[B���[���"<��w�����w��cZ�,���v��V@{_!�~�m��n���|U����l�{-n�e�^,��{��Y�|��68�>��t�oG���F|��n�z�2OtIu@�����*�\k�jsq����;��]�"�����*�7;9�U��X|.E�R�nU<��Q��[*�E�r����V�c��5Y@�����������\��O�����@p�V�L���i������D�|�{}/E������������������,��<�����S�SDzQ���CVw�n����r�����B7�EB���'��	��������;���1e�k�Z����8#��E��T�5%����Y��_��Jek�kFyU�����=n���������U�#��	�|=8*��{(+z��HJS�+OrYR�Z�g4���C"�0��XVe�}/*d����2�h)G��?��2����\t�B������/g�g������������������>��Ap�������XN�DV���G�
>��C>4��F�*�����*���&�*�����*o�JI��������Ae�m��g$��z���B� ���/���ar�����E=;m��\���@�-���^,���%��*����R�W�����^_/��t���j�&�z�h��%_������*�qyro���B��[j@�c+8���
�-��cK�X��-E��}�������VKa�V3�����E9-���M�B��]f�)�n��M6_��=?��������9�?F�a��Q�����������r�!��I5���q���B
���R;��6i��'xo����a���M����'�&�H!���Y����X�_���}���)��/t]-���3�&�t��+��@�*��~�p}���5
�~��m0�:�]L~?_��M"���������\�/1��ws!�|C�V������tvB������N� K��0|7Z�4��p�E�s�xJ���rV����B�9X,%�������?��
�����(b���G�B��<�n����#�����$r<����&'��Z� >7���(�T������6������p�~}
t����+�pA��]A�:�XA�:������`���M~J\+	���M6�������o���.�Woa����$z���#i�l���2���tv������[<����v����x��v����3������Wo>������������������1�0�a(�w���.V���<r~�!�/�G�&l��2L�+~={������Pc�D����>��w�E@k�����g����z���E������a�����]B70�}8�hv!������W��{s'���_��3�f��2����j��l�nf�t~��`��t��� E�k��2�0�����}����V:z���������K� ���9g�=�,�A��(��B�������P��\�R�aq=?#'8n�7���2K�.���h���]�f3[Og��lv8�����~�'�����B����F�>&�����Flz#V���-��9���3��L4��D��r��u�{t�
W�NL�Z.V+���Q�C,�P����qn��.
���xn�
�&I;���b�TMe������"?���@`����VoT���j���=�F\�A���O����L�I�V+�����F�� ZN���-�[lR]���
lIm��Z]��i�6�
|����;Oy�lC-l�w`�XN����3�1�����=#v�J�n�����^5���2�j����Zj��b���m�)���������_��n������7h�����v���*�a
d1M�N���K<::*��E�%9�/���xH~��p�p���/}�����������(�c�b@
�����r�#>���q�
�@�����=���|B;�qg	�Y,u���� �T6��[A�jC�I�eP<�x
,���@j�O7�S�6�����D{`nc��E���3��7��,P���\��c6��}�Ez�(�$���$��9�	�w��;)�[MM�Z@)��2�'!�K��S������8�5� t�BeA`�S;mD��IhxN}@,
	F��95��!$@M�����B=�a��W���1����n��3����i����p��A�bo'���������n-$�@J��t
}PB���T�v�?*g(�G��u��#�E����#�F��c��� ����8�Yv���u��]���M��EL�����_���.?��h�=#��|z�y]^��A�z��5���v��J���h_^��yk43+?�~�����@�O���B;��Kk�
���!�hZh@^��W�rL�xY����
W����9{D�y�x���I��{������Y�<����e�q�v��i+����U��WE���nw���tU�,���}s�/��#���>��~k�H���
����hb�������U�@\E���9�fU����	��	�����D��D���bk_�����������q�H[�
_�*
V�~�T�n��@��r��r�������=���_�Hm���W����M��H[Y�
_.�����[��H�%r��r����!��!��C���z��s������������H[92
_fg�I���#X������|.t�O����y��#������a�o��c��!�c��%��>����>�,q��HE�v��{{���������g���9xD���_R�-�`���Cpcn4#�Cpc����� ����h�gpcPH�{���f6k�,�<������B����G{��!�1:�@�F��Wt��V��@�(:�@��C�ct��!���
��!�1�]t|����E�c4O��e:�\}�mUW���W��C�����p�L�[��6}Ez�,�Jf���YX��W	���Z[s��
j���6��*���X_jw���JN1K�. �U�^�wkk%��#m
�l5c��Z
{Z5��H
3H5��M���ZMc����P:J"������+Cy.6�����\&�o�%c��Z|�����;GI]-��A�h��L��H��<�)��MW�U�%���S�:��zQ]��s�������'��3f���S�8��j3[���$�h H�����p)w�y����b��������*
=pt�6$����]5�������H���E:*�#3[��v��h�>�����c`t.,#���4�c���^=���)�AJ
�Y�^H��*,ZhH�,SeWH���URT���*�(�E���54�1�,�(]X�05���T?r�^���*,Z���,J�����>������&���p,�M���`L�
�\X�0N��HY�E����������E�E,�(�8�,�
���sp,�����hL�F
X�\X�0g���]?�^��*,Z��lYtHYt�������c������]����q����FX�6���EHJ!�Y�^�
H��Xk��_��0V�9�&�����#�O!� A�A��?������t��^m7}P����W��u�Y��+XMW�
��p<@k��A������r��:o��U�~'#�����l1������5`v���(��f�_8�����b��TV�X�/w����A5Z
41kU��z\\\���aw��0'�d�sXd��y��"���T���/w���i(5�=
4��T1Lz\d\���aw��n%��^
O*Y������Qp*	���l���C3h��M��?;4&�������CsZ��Md�?�h<��%L?�{f
k�(F��W�V5���+�*�d=�%��d��I�-t'~
�%��l�&ljm��,e��j���IW�6��r[V��j������v/�9��Km��6�J5��S �%9X<k^cAFQ���3��������vf�e�8�?@���5��p@�m���(*�=��tQ�IPg�-�
��?�"B,�U5����DG[�����m��k^#~ #���N�@��8+���sg���t��+����yj:�o�j�AD �&=���EfP��4g�v�q/��|��.99��k?���O���{��,�D_�O2w�V{����S�9�U����*t��\?�CI$���#T�S�����Z���hN�����o�NK��������j��;�8�4w����;-qaqJf��"S��5'FM+�^jO��[�W8�T[��p� 
t�U����9��z�s��V~yNE�^�\U���#�S���0W��`�4�T'�"�U( �%�IJ�U.�������\���U/:rQ��\��^c9�hl�p��W�?�����f<����b��7'��j��-����S���
s��8[,����Pa��	������qpS���U�1,9�)a<�G�'�q�v�K����`� e:+�r�l�����_�VU�z��!������	��*aF��U�./M����3;{s���b~B���B������4�,��I�x��h�	a������k�/��u#�Lb�g�����'l�~dN���K��$��I4�4��5X�C��}�h4U��
k/��"GlY	��[]�0o$UB��ag/ni�X�b��UY���/!���:����-�����@�7a����.zv��Eo�-zq��s*L�y��E�������w@�[a74D���
���L������%d�����;{qK���E/!�^�|�����<Ii|����2~aa�Z������-e��|���a,��m��R��J;a��rg�P��i�T����n�i���=��z�}60��{K/
>+��K���#�w���)5��n�@�W>�Xl�������`4
�,�Z�w5�{��Ai�
���+8C�3DO��	���)�+8C�3�.��!�gHp�gHp�g�f�R���1B�!�b��)g�~�{��GD����E����[��H]���4t�H�#�1�'dp��Hp���#�1R��	���c$8F�c$8F�c�
��)�A�sp�����1@��cd�����3"
Ip��]#W��l;��������)��c$8F�����	��r��c$8F��#�1��Yp��Hp��Hp�Xa#�0hp�#��1#(�r�����>�E�	nqHp���
im���'l��6|-
�W���()*�FW����)�P�re���D��JD4����-�3��G!��G����?����G���fD!�%n^-GR�{���2��$�=r\�1�y���d��7w����Z+1c��������Ae�'>'�NG$F^���i$4b�s��+�g4,�	�������F�
;�HBc@�`NuEc�
�IBcH�`�3uEc�C�|Uv�^��aZ��#Ki���������BD��F��\��p}��|2�}2���z��u(����k*F�8F�("M*���;�����i����j�[�@z����L��
������R����I�h��6;S2�2�:s':���������s�;�D��\�=��8����vF�m���jL������BM�\v������=���=�e���=���]�F0�T�w�����E#���=���](����}�mm�6��������7��F
ms5�m�I0���
��`x��`x����`x�w
��.P9�uW0����`x��`x�w�`x�_��}6����z_��;�&��L��4���K	��`�o�
&�`�&�`���&�`�&x
�/P9��uW0���`�&�`�&x�`�o�n�d�m�������
�9��P��/��[�]���������J4�>���`�f�`�f�`���f�`�f�`�f��hi����^���w`��Y�q'a)F�f�`tF�`tFw�`tF�`t�������]w�{0�k�F�`tF�`t�
F�������v[k��vf���������_�����i�u�!*.����Et�.���r��gY�����?�Y+3<$�c�&���a��T��/�q����c����c�Vq��!����v���.@7NVZ�R+�},|��"V�%�����w,����r���t�Yg����B@Ps>�+�Q|3�\��">��y�����|W��P��p���`#�S=JL9XA�
qN�"h���C���z���S��zc	},�>h��>�@�����@�X�}�F���{�'������O�i��O�'n�O���V�������vf`���	T-�F�m`v�aW|b�X���������jg"v�'�'���0������a�����>��{����'�'���x��I���a!����>������K�{����?��vv����T�1Z/�/F�F�Eo�����<���p�������c9��k[���b�PY$���Z6eU�U����k��`m���-���[����#�����G�U�fs�`��V)R�%:�~DZ�(4�����f�"�\���G�U���:��xA�:v9>A�.�'��{�<=\�Z.0Y@5��G�7�"_Hl�=�R��
��A���r��u������c"4�3���\5����[���f��-jr�Dl�%i�lZ���8f(���"�$��P�i�U����z��pu^������K %G���/!S��3�(���_����&j[�8��Y��wNC��:��bc��-eHT�o)���r)���b����jy/�Xg��U����r���E�%FB�ZV��F�%CFB�Z��1�l[�\rf$��e�(s��q@�:�FB�Z.�1EF�������K{{��>�4]�I��F�RY�;�uF�UX�F��]7���TA�� vB;���Is���*�=#�=#�=��������U{N{N{^����?��J�����4tTY:jn�W��rQ/���{Zio��UR|����	u����:�N��:W'���^���:Y��\�P'kE�j����B�a�<&P��=J���)�}���tB:!��tR7����1���������n
w_!0='='=/�y���jOg��#������(�N�<��j/f��w�|;P�<T�V�}>�
u}8����������~=���-��8��FCe��G�r7^_��+s���8���t�n+�7��qeQJS�`r_��XB����q��h�"eU6���"[)��XJS%���in.��7Z��VXt(��F��k���g�M+S�� -_v�uE44��
��
�@^
��nD�Fh/AC+�WDC#����K��
�U�0�F<,�T�*CE\tJ�;��s�����,kwZc�e7C��Pv3��T��nZ�c���������S��k��Y��IN���
F-���-���&�b^�7@�z�j�3�9o�C�%y�4���E���&�e^�7@��8j��$��K��PTU�X��]�*���.��j����V�V��&���'�$��M���m�L����QMpH�����,���w��>�	�����=�8����	+o���SQ����5
 P�� VE
�6���he^C���Z����9�h�5+Gm�������V5��z��.Q�(_fU/��,)��/|q�G�����M������JAa2���;?�R����,N��P�@ �����Q���A����J�h
���#�a�$����2������'�b�~8f8����~�����O��3��g�����	H����~`;G_�'�a�v���O��9�>��#�y����O�C����~�����'�!�����~x�����s�|�l���~`;G_�'`�����|�Q2�&pv���q6c[X����L
��������P��v�������5���?1��>��� ����;���=EG�*���F	�2((?E]���Q�����By��������������)4#Z�-��y�����M
�v�2�T�_���L3����.��f3�����p�q�W��p�q8�8r������])"�4�)�r�	
�TB1W�B1�2�B1'�ZXL�;���SMS�6�'�^T����AVec�'����p"���F���������F�y$3Q�\������`�����/g�g��z���������q����'���r��!�)����6!���r�CW�t���#������b_B�K�}���KSC��]���_@o5�����e��V�W��.��y��Z���p�\7#�s�D���e�pWz��BF��^.6�����3�<H����v�c���������oQ\����|�hx)5O����{ �w���h�^_/��t���N�h�&�z��
`~��#�-*&��$x�Fq!!�!�8�� ��B���P�X�C�G,�!�k0A.�p��QdJO������YO3 X�-�FAf���$tae�]f�)�v���M6_��=?����,��������1*R�&=D
��<��e�]��E��;Q��Lf���g�>] t�L.�����?aD4�mo]eb"	y-7&l�<��E����S�.��`���S��\��q���P�`�������T�30W�m@��y*���+�6 j��8���(�.|Z=�Zq��GC+u��R�)����[�l���Rx9%w��]{�=Q�N���IOPd���RZ})��r:�	J�=�V
�/%�S<'=A�ZR6��L�b���)���.)�G�������v�e+��{}v����W����Zq�XF��P�����*��P�k��V}�|��O��������!�l��|Z�]L~?�Q'�{l=�D�@��,���������!�I=��/�Z�/��[��j�RTPd�1�z��)O�jEd&U�(#EKFf�1�T����yt��2:����jZ�N��S��)��]�'U�rt<:�����E�R�/Gg��S�*�)���x`%�LZ�-���gQn�j�)T������R*����/���#�����@�'cP�6G0��Uv������xmc��8M���9�CA[%Z���J?S����h�V��5Y"�;i{�d��+���t���+�~}}z�+>�`^	��`^	��`^	��`^	��*(���`^	����r���`_�/t���M6�<T������]^����WW������0d��+G��~$�����W�D�o�b�(��nAO��,u���~����k�y����������7��_FWW��At~v����G��WxH`&�w��R,K��y���'2F?������G/�h�|���=����(ES+�8���xb��]������N>��^����(x���6L���,p�� �
��U���6X���w��\�iz��W�|�Y�����s��2���\7�e:��F0%g:_�P*eI��2#�+B������;�s�������w�>]�����[}&�M�=#�	�/����3���'���
����\�4�n�~M(/Q~�~=��R��~}~$����J���^��B���V�le��b3����%���>�����}u~������jz�]��P����`S��o��-���=���E�2����G��\
n�;�#���d�w5������ff8��N��Q����<;���d�|���S���k�<)��1��/�����@w�
?u�|
O&��>���������C��������)����g�a�����������>	{��A�S�9F���X����r��d�2h���H3��Nv��QNF��ebIO��'�����Wd��~L��I���f��O�L�BCj�u�$�>����.����I�$"��*��;�`��Zr�U�$j��v\1"rj�*g��l����!���,����r�Eu*jU���CW�Q\�c�r�9g*���c\KwYl:m(��R�N0��
��Y�R�j�:�����8��V�?�H>"Cs����q$�����O�8�����M�g����X������R��P>��gc���5��;�� =`j���&q�B��'^q�/�����.F�� R$4�-�I]�Z�-�o!m���@�|o�����V��WkX��z�y4J��%�Z������`���� ��0���ZrZ��`*��[@�$�=V��6,q>,�sc�a�4?��3Z�af��o���������U�T�k=��Q�B�pTsj0 �S�[��1��(f�������m^<��7L����[~r�6����o}���k3>a��'G\+,~�>��3������m��k��������J����9�pHb�?�G%n1�;G�����J��������-f���
����r�����c���qA��g4���(�����%8���8����'lC���h�'����C�����d���<_s��9�8z�<d���9��q�q|6��h�Qpp�Av���G][nl1����"n�
9X"��!�_���=pc;�fU�����&�����p3g���!G����,n�����_'�����G������i�s27���
�����������Y��f������|�W�#~����<����
��#~-�8������5�&�������9��h����o���pkO�9�o�{*���3���x��q�D��s���g����|	5?�8�#n��)�A��.�-�w�#�n���}����.GA~6�3���1��>�y��qn����B�+
�ks�q�u�����}��g�s$�jp��Q~�A������_�n�wC~/�V)n�8���_?9�9��,���u�!���>���:8�i�q/���b�{*?�8���)/C�;������s�Np�;��9�e}^��(�y��p�Wa~E�x�?��w[A"��:^"�Wh~E����{�������������=����x	��{�1�%c���D��y���?oy��K����q����6'&���a��ZI�Y��'7�xy�_�xY�_�x���/q�� ���������Z2��r8D�	�(�_j���T���qL\�-Aqu=�%��t�l�x:z#�����>�7�Y�T�BS<D�(K������2`�WC^r��z��k��JFC�lkd���q)}Ll�T6�f��1��r���l��|�T�63)L��!t/(KB)(���H}������?��OtG�Y�
�������,�3_���|>�ME���g�3d��U�����e:H�q��g�=	��F%|v�r |�b������g�*��g�>��U?��55	�j�>;�T$|�����$�g.l	�j�>;TA>;T)>saU��P��������CM��g.�
�j�>������3A����y���,������P����I3�J:��"b��{�k	�J�������u=����'9����9��lp����q�������y��s�F/(�Q������d&+�,O9�T� {y��b3Y��)�b�G�r*��|�a���*��H�}�@���M�&>�SX��:bF$�{�h9�M��G}e�]�ze��Q���f�y+#�:�D�?���k����2��C�q�@z�L���E�9���!����\(�I������c3�G]6�m+l�F�FmIx/��v���MM�f6�|����m��r=o9TSs/��d
4h�PFR��r��l��.� ��+��T�Jv������pga6owVkd�1���W�����r�6[�=��~l���2��>�g�0���:��G��\��1$|P��>��h"�+%�(��R��)��2��n���;K)�	wXF0�����I'z�/G�Y�y#W�j���y�����rw���j��*pxI*Su.�'d��Y���_��$v�YJ�wX���2�	N|I��7�7���'��EEK��q�Zx
�H�)��4�Hh����%JY�oY���R��,$��s(z�~g��c����[]�W���.kY
��^%��
�&�Y���!��F������p?�!9�s�;�\G
�Lp�^n���9�T��cv@'���v���(���{r$e7��&�eJkI�\��Z������.E���H��R� �Vk���7��h���?��5��lh�8���X�P��T�0�*Y~����r�]#Z���a6�C�M`��E�/=�B|��{���`VR��c(����b��.u#z�k�h�W�2	v&}{���,�D�fSD\���EK��8z�AN�
�������J~htz�\�9�^6��C�����G/_�8{����*���B'�X���+�O����c��m���[�����lY2��L��$E�D~��� �+3�a���p�#;;�^�Z�Dt�Z�&�U�^�\�Dt�Z�&:p�^�X\E��Z�&z�m^���.ZrAa!�h���Ph�d�8�m�Lo2��LW@�M���B����V��%Yv���t����^�����<��_�E]����~����$��,KW��"n����x��9�KD���D"$V���'e�����@S���EE���I�����J�Z�Ou���~���~\���A���r�n�sjk����^X�}��F���[�?OudJ�)6�:s-V8�lD���4�S���������1��� &��=�:,r������}����JC��2�U����XC���\���i��]�@��F�U)K����d����u9��Z���e@����\E��T�7zs�����~[^g��e��>�/V"Y��MXc%��t%��7?�rQ�T�fL$'(Sa������E��Kbz}���������8�
�Q�t�'�n��V�t���B�b����cj����A|��VZ��KJ�����Qv��l]�C����@�"E<�1zNq�i�aU��|�3m��%iYb������J���������*�#X�YI���F��F<����F'!���������]O\%�|�/���q}NP�����J\���Y����b�e1�e�jE,����	M�rY�T��/.�	Y.��sY5)�
|���&��t��G)k7�X�5&q�����>���>��>RV��5��Q?'C��8F��?�/	~'���w����3V��cS���k��]������[1���x��<[�Q.57:	�n2pb�J�%6�����Zp�WY�����/K���_�R��Zf�S�Uj6+`"L�6�v���!�thJ��G4�\q����<	����;l�������	�_x�M:l�|�����C4M}K��R��	-��;P����x4yT���?�����>����9��;��p	��_��k��^��9��e��=g�`d�"&�_��~���N���O?F�C�#�(��(O�T�G�<bS8�9(��C'p��v������$�I�F[��V�LA��T��M�
~��h��w�|����`���I��t�Z��4��a�Lk�yF�4,�gU�e�Q�u��d,I���RR���U�$���r�,6����	�J&6�HZj�,�������>���(�\�o��������#����=��x�������^�"h���\������(�~��|#b�_f��l�����f�l��|�.7�,[<���������8>}z2����!����%��:O�<���?�#:�%���	�n�W�x����{�6��d9�E�G���������b����`��UD�A-��q�x�}�f_W�-X}�����b�M�����o�d�j<��?�|�-��L��t}��.n�?�f��L�������3�{9��Wy+��:���.�Fw73�Z~�w:�vWQz�e�Z,��f����o+�.���<�A�/�K�z���Q��f�e2KW+��z
[a��~�'|�Kcw�0������]=�G�+x��Bg�}���`�,.3:��d;�������a��h�}�}�~���tS���VG�#0�zq|g:\~���u$RL8o�h�����5&K��q-7���f_�k0��3��0$P��:�t4����-���}�yBz'�������0��`�����^^��Y�{}��������������"P0.��3�GP�C��C=�����
+$����Y��xf,������?�6`D�)��3n�E���h��g��&�t�R ����)��_��%jp����\l1��f��u�|
�3�"���U������������$�`�G���hu9i��y
��(�C�~�M"�y���A�Pd�
(�01���b�����^���������gX�����Py��I��X�<(F*a���>��P>���Rd����p��&����t��h�@���u���n%�iexD�9>�����L��!n����g�����z��=��z1�D��>��\��`����`��V�r���G�h��y��������o���`����l��X��������?D��"n�����K �V����`	�
-����yv�����&x]"?��F&��7���6�o"�c7&��-V1Q��D�j�����hGfFE;"3
L�e�3Q�������������1fS��b%�d�b,E�"�"Z�92e.5���M���~��(]h�Y{�?c���V6��������v�xr"oK##8���Z8����F�o��Xt���
h~�F���dX9C~���m�3��A�:E��*w���yC1)���nU��Ba�HW�
*�hG��n�z��Z�j3�c�-��
�7PYm���9������(]�&�������G���/��c~2�w�|�@#�@7�-����������0Xn*��a��4�h�F�����8��������=
-o����`}7�5,�(��������;�
;�����e6C�������3������MS����N|o\A���yk|�h@u�!�"��d��avc���:h�����������6�C������W�Ve���l��5�R �
���D����s,�%������"(;���j�u�Lg@xD������uZR�:-��%�#B�MS(�=K:������%Q�E6VM<�(ssfw�0�����2et��9>"SF6�Z*so�����J\�����L��������"&%�1U��3���T�j���bwV����l�<�B�j����)��Z"�Z,�Z�`�X�j�Y-��ZLX�O��,7��RmC�v��F�3-��
��9H|������&h�h�	��
�Mc�r��������x�5�D�z��W�{9���%*�G�6s�L���+��!H����,�S�o :�� �2�0/�a�
�8�*�����>~�@x�9T��z1������
<���_Tv�z|���K��{-G�������pi��J�_M�f�g[f����!8���~�i�w�{`l	��a��P��wsl(U�R"��F������i�^m
���'�7a��p�vD;�FRQ
��Usl���I�vrC��_RrU�@a ��@� 4�b0j�s����C�(�ds����pSl���h>�h��9&1��&�f$��h�������m��m�Z1ke�nj��Kb��E�A"h��)�����t
��jLqz��/��^p?��l�?�.�7�_��C<��j]p���s����}.�<�P��p���b��X�\�>�h���t�0�0����"��o�6o%�[	���R^
x
E	��������qk�+`[�G��@���Fh[e�`�=Ff���$l�a
����}�
"-�FUi�+dI)�~�or��m1��&;�|��'o�|��f��O�yr~��7��SS�x�h3��DT�6�����'�8l�E��7��E�X/*2�z�6S�F���h1o����"m���:gQ�j��&�E����&�o�
����d|����M���&{����E��:UP��quN����l��c5'*[����>'��o�
/����91�pbL8�O����|�n�n�b3u��L��
#��}���/�h{��-�{�c�Q!�Y��b�U���r� ����=�$?���J�G,J��:�[[��7/�y������2$�#�����$F���>��������>ioH����s]��V�i���sB�>#��I�-lA��~������[�Ia3)la��> 	�]v�<���Y���{�Fks���M�n\�8�C�����q�:Q�����\fR��������TG���#A��R\K����u{t,:[/�7]��<:}���g��=3�>N��RQ\E���b�(��m�m����E��m�����(�I������4(2�J?�_��H�e�O�C �d���4a=�,�=��G��-\���<'`xZ��I����O�1���p�_^$
$69���v�h�9��x3���:L����7dG6����������F�t���f���w�4J4�_J����ND����qmw��Z�|[�&�:��Yq��l��b�qw��[����F��o�-����38�_�^Y��DWS��=>+IW._b�W$��_����'bnjn<�s��s���M��	1�!�y�0!1�������Z�p�6���f����O�#�y����Y�](4��m����u�'�7�!�
���:]O�`��?�I��l	����+b�&�B���r�%�G������|��/qcQ��G��pf�:�~09w����d�����r��-bP�IF���U�r��#����w�����!GZT��*[D���GG���Y=�o�*�K7A�s� S-*����G-��3�nNQ��p>j�Rn*��;IK��M�� �]��"�R;`I�K���l`5@�z�4m���yf������ZM��0��w1��8��b�:���D�K ��*J�0P-m$D9Q�<���l)9:2	3P�����#a��Z�`��(s��A1�l�����
�4�`F�|���Y������e4�r]�t�L���*��^]eKF�!�A2)�9(��l�H?���V�za��|el��M��)9@�7��_3 ���=�����R�������(Fh�T$*uAKg��rF"��&�M�P4��0��@��=&b����id�(������p����<�i����O����v(���e�l�<:���������3
����h����P,UM��G�K���R�:	8g&ont$���/��|�7��Xs�xi�4���-��nrsJQ<�e_�J'hkSg���j�l�V�f��'��j��%�P;��N7�]h�������#����UB�iwp$���vU)�
�p���$��>���$����a-1T-�2$D����tC���9�����UOK	�����d�A*��Kg{����I)U.�����eK��X�|����X�|���Y>b���$I���\�c���.������AC�SIk/����tCa<���X�|���#V,�z��w�|���#&�hh:�e�Z9��Bh��!\�����[���
��	p�Ov��>j�&0+���t��C�Q��i<����N1�r�)��e���yD�����^�j<
�*-}�����C�0�LZ��5>�oz�=�c�!�q�d�����W��l}��&��6%����R�/4��������!��b�Pa����CBt�b�~(bU�l'$�4���(�b��:��z"<����/������0?��������rx���Z;��qq~
V���t������N?�}�E��W���"��U)��bue��Ge�X]�A��kAeeM�nEI|VvPT'+;������C������CL*;�Qm3�>�qbrQMv��Wp��F	��1�RoM�g����f*U�����{����������(S�*c����[P�������B�6`rI�� �Z��	�^�������
@O�L��B�$MM����z��;�]����cSV}"���������OQ�c�\O��@�0j�\�����~����/g�����dv����Rj�F����l��	E}�U4�G����M��F��
q��������s����E�^D��(]�&��:���������e��TE����*s��P|��K4;��@U�PU��E���
3|m+g���*����<�
�+O�XY7��9k�������b�mo��M����c����K��������y�����6����q_n/��/���>������Z�n��W���X�#��YI��0t�`G��"haMU) /��u�-Q����1��	��8���I�����g�h�o9���6��Y��x����Y��f�5J����
�$z4HL��%UJ��h�v9��
�f���o�M�[U�����nT n��W><�>P��m9�%��Xag.Uc
�q�}WX�$~]�������fy��G����p�^^f�����7��
J�����3����r:��T������ZT�,�_����|I&L����%����l�I����g��^���^��������C��Fk������� �/(?!O`�\���Z�(��<�e�f�0e��|&��Q�`2�������/����,)�!�#<�"�7���[#z�������A/�~��0SP���$�(����d`�����P
�^��S#<�����fh�-j�Gt��bn��r��N�-�HV�����$,Z�am\Z�`:�����$��y�o��>�G4E���GT��*�q���s����}.�<����~c��X�o�������3���,l���/�}/��K���������Xy�Py�*����,g0���b#�����E����0���>#��-��^p?������p�f8i��p�f82���W�'mVX����=wNdd$�U������ 8����} e����]N����`z�.���4�o����l��xo�;~!�!:�@��r�;]�r~�X��3����p��+��D��B8�����&eT�~�������X@_���Z�$qV��]���G�����*�Z��X�b.�|�{���-�f�C����QwB��Z��[
�b�[�C��#���z�;�<�CX�/�!����c�u~a�bm�x��5�R���YZ���'�V��"1
���"�!��|��N��#�����������z�o8����
Y������1����4S���RR�Z���vL�F�;E
�Hs�#%	}�� t�5�A��!�
������)6�mn�gq��
�H�+)W�����'zy��w8oQ�s���m����1'�&Q��d��LS���Qk;��(��>v�P����bMN�}�~�3��j�y�#���������~��z��
�d��N���6s=E�R�7��.u�����U-��+f�06��]+%�W�����w0�������d�n�v&9���vM�p=�|�5����<���5Y a�Y��la�~���h�����7��zz8��d�h6��&~��|�3�[:��K��G-�N���N���N��m��b-w�z�t
o�5�(��o�|�6zO�����nT��T����Z�PE����p��|r��*���UT������q�Z%fL-j�l�x�9$Y������=��s�A��
E�?�V��R������&�������������N`�d��"�����u5nn]���j\q]�����E����U
U|���v]�+���v]��h^Wc������u5���q�u5������j�e]M�u5��X����u5V���b]���j�^Wc�����x��j�YWc����M�d�zF2&mI.�HU����d@I���=�d@�L|U%�;�a ��)��o�^S,[������T��
�7T�?b���r�Sc�<T��<pO��n����v9]�@���V����m�&���eF��/�RJ�LK�^1-��9O�$�"�Et�K��N%-HK$���Ji��}�&��I�6
~�r9��<�l���(E����F��C���Y��<a}{0#����������(�jp�El�&�������d�>h5����vS�������n#��-�C4���X�?�K��z�E�.��]Pj��m�v~oq����)���D��d2����8��/�'��`:��[J���'�^��7���rMN�}��ng����m����,{	
5�='o�o���0oj�����h����������:�������v
��f�![������I�5r[A����m+��0�l�\*��<�@/�zA���K8X0
	��|�T$>]�1����L�4����'8����%)/k8TQ������b�����D�o(�<��.�pbj@����
yK��cl%�he��6� �-����y�s_��t�����{ql~"�A����Ao00?1�[Q>�����i	��@x9O7�Fx97O����1Gx%�c��J����$1?2��Z��|�}���ou�6�F�T�p��t6]�&@cZ/S��
�����f����p��2tR�@v�`���5>�	����e�@����u����������wp���bQw��=���������c������)�t�O��}��=��F�����spc���l���������o��s�*�	)�=6�������lT(l48l�C�������U�$��/���:������n"9�4����[j���l	�'�Q�
�w����@o��}���b�Y��5�Q����,X���c
Yv����?<�^r���P����[<:����@�/jT���%��<]o`��oy�
��	��R���y1����.>����{K:O�����x4�E1�g\����(��ty<�]��E�S	������7�D0,6���+���4�
���`Z�y
����i����(�i��H���M~G2�U:���E^��y;K�����!�2����
�3�",O�������h���%] ���],J]�<r�*W������Z,�/�
i������w�^\{x���/���F��P+N%�g����TGrC���,�3hn1Z�b�[�j�����mj��t��FV�/��Q�
��MW@����gbF��$H���f�1�t��������55�M
AS���_�{����?�y���F�{S�������R��Iv�������\��&����T8���Yij��5
��t������D���Oh�r���F�����/?1o8�7>a�#���)e&���"0�C�t���:-�OI�g1��>~�c
��<YN�?�K�Y���d�|z��*�����3���6e�-�{�7�ho���Ub@�#vT��lm����]@	����;(�Y���%������^>�������xA�?B�z��AV���+��L��Y����a_��W\��*A���Bz:�?�O�G�Wr9��\��7���T��+�]���Q�v�X�"�`L����=rg�
F�u�����7u4Z�B!q�Xde��F���'n��"I��ID	Til@��h_+T���'����pD������V�B���b��(#��'���������
SRDO��Mc��A�i���l�y�����RI�j���
4j�A
�549���e���d'�EY(�0X��v��1d<�����0c}�Z���������6�}��k6��c��U�1�� �
�x9�_�����S�8y��F����g�-2�
�y.6+��8+=���k��B0������=�2hY��
��{*�o9�*�p�t<��3Y���sZ��P �0}{+�g�����4���0�]�<IY*���D#��WTx.U�fXE�yW����ak�lc�[�@�gI�n�{T�~�����2��XJ�/=����(�mXzk��S�d����}��}���_m�����Y�F`���9-S�5���b��e���4��f���bEnk`S��j���y�a(w�j��v����M-n�n��p���A�Sm�;\�l`���"[������`9&a�^c�!f�a��g;��A��q��]\E}
����+XC���)<��<��70S�<��wb�����&r_A�*![���,�(�
���>C����:%�%���&H�;?���+����6��G�AKF2{g�m��A�2�i�|"H����:���|�;2��g���>q�kK�����G�R��'��GG�v��J��6��t��\��zZ�e:���~�����v����[����>������bcm�������"�nV�*F�\���e��I
�v&9W����?�������/D*&����_e�U��(#���
~���z���������/��z�_mP%���b	/����&*��[���K��E�B	�4�D.��s�2 �Eb��qvq����g������E�#O?J��fk�o��������^�Y2:����O����n�>;}����StOB�~����$���}�T���_�K���w�L������S���%��l� ~��5����b�_��1���`��f����g?�y�&���p����D�>�/"`7�J2�F��Og�~��Z�������kL����m���*�	~]]��Hx~v����G@�+�\*��.���$"xbD9?��������	{���x���g����$
�#u�n^���B�}�=}�����������?f/��$�����M�}|����7p��?�� �8`�/���j
��d�dqtn��6B��#��*�h�C��A"�&8�]�I�]��A�i����O����� A��9����n�����6�~�O���	�3}�LN�Gu��0��>_������-�^3������x�o6xm&�(�B���t��W?�	����pw�j�������9��-��us��t��w+s������~�$���d���
����}���bA'��q/:�)�GZ�{��.�	('��'��Y�M�z���	��J�r��/1�S��mu�hZJ��W���������[�����[��c��X?sq�ZP�/o�$����vF�)eS���#]��A����~�����W�{z�*`$�D4��{�{�����g�a�#��Xx,�@v��Y�d
L�����h*��X|S�Q�(p��G���B���r+?H��
[uCA�: =��X�j���_��|����. �2��3:U�:/�`���2����v������2����)M����|�H��v�j���o��
��j�Ye{���y�F���)�A��������&������;�?6m`���j�d�t����G��Ek��E���v��m�A�U���Z]��Qu���{��������:�
�6Q\C�����C��M�Gk�ru9 ��!Y��peA���9_P�k�i���_�	����P!�
������|(y��A������4������,x��po����V�����9l��Q2!�PE������|)��a^^���;�[(�X��l�h���8q�EU����=��;*��b�����(�4q�<$�jl~��H(�1�o����b�*����GY
�]��DM#'m��5V&������r:���b�_UW�Q�v8m
��
��Bx;���l���
.;a��S�xC���)��8�B5���`&H���V�ivIq,�+s������K6;D�jK��
�3]���X��5����}�_d<4E���Hd�:C��@�������[���\������z�g
S�pZ����@o�[
���&���6�;*-����oJ��g�#p1~9�}���w�*}�����*IvY��;&�^u K#�<�����X!��~c��:��WA]��Q��W}l�g�>H�wY�GA]��w�Wy]+7�@�z��21_�G-pZ������l-�r�~A ���UyT��ah��HSnfo��X�F��%�v�u��F�6T�m'[�RSI�]sj��L�W\:���q� q��57��.vo��������A�9����c	�X^s�`�a�������>����T�X:�#!Z��S������[7'*�@TW�
����5��9�*r���e�!
��n������GmS����{�4P���]T&�V-����R���������(��Z"�1j;�;r8n	~+��B5��<�w{	tjv�nCM�����y.�F����3�����)�+i���oD��'�F����r:����mp��|o�����C���j�fP	�x�o�J������.J2=����+l��*�����l(�����}�s|�K� �Q����L�������7po�6��4�����D�U7s���h�(�[�����-���B��z�c�-�k��
v/�Xjq��>A�-q&G����s��_Vv�~�~��Q���zX�,�5�r�c�:�����q�Z~���u���~��lE�2Fiz�����{�6MT�-�����f�`K����.W������X���)C�'7U���X3E�[>�������gFO�����
,�O����5�s�^�9%f��z���:��-K@�������p@�����b=����LU��~1��L5(�:�2[/�h�'�G���O�NL���M����������8�Cov�����x�7J�7^@�Q�hZ�_8dw�l-� ���"b	0"4]t����/p2r��k�����:����Q(��!
����[�|�8Fr�B�����o�(��b�]U�6�������i-�@��]�z�Qs��m�R�~���������=)f,�����d���hx��<���\������tC�V ~�P��*���5j��vQ\��� �kQ�8����+�%F�h��e��G�[>����su
zx�GN��t��@���D�n=����T�|���]��'�FI�M{��T��Jv��?�g�-*������
���k�=����v���G*����8Ka���=L�w0�v�K�9��T��c2�O`o��@������#����Q����.������zSk�n��3�o9����V���Z�5�������4��Z$�3
����t���y���=���Y[v��[\��9v+p���eIyV�v��U^X�����H�=�sA�i�Q�,A�qXpg�>=�R�{3��a�)�=[Z�8@��&C�e%q�+
w�@XLvw�1(�T��Z8S�����������_�O��wx��l�G�{�N�hfG
�N��'W_ZC�U5������C��V]���-�o<u������A��txx'���M��=�[
�x�C�Od'�n����������u�/dwt��n������<v��!�x<5@�7�K?�EH�����M�f�(���-.'�*q�l����y�?��y�DYi{��|K��:�b���e1��+���*K�vK�[8�b�Aq�=@�As��4	Z�r���5��������nL��7)=v��P�D������ ����j���l>Yl�k@���r��{�3���>��C;������D��W����D�9�q1�p���Z���{p��Bv�!dL	���_���Q7������������`��a�����%&����>�A�!7���r��6@�wv~�}�T���u�D�aG��2#H��tG\!�v\(������!��/i�iD=�"���\��4�k41Yc�������#���/soZx�)7+8�$��[5���32��wd[������@n��������/6�����hj�7�����=��h����x��^�Zs4��.��YZP�����<=��9�DR��J������m��M��n&@F2�m�Lo2 �LW@��M��t��z��E���OxMx&���������/Q���O?��p�M��|�e�j��i��x�����{�V�3�YuV���'��S�O�}k�CTr!��b/�WW�#����`~^\�����3����wf��m�����u��m��jo�����'��xY�����\co�dH�`u��I��l�4�;���c�6N���
����[��!0��8x wW1�04D�8�g��<���"<N�����-���M��N� �Z�H������������N�����O����
���n�G��#8�F@��|�V�%����mVk��9\f�,�����dq����d���'���K��t1�Z��
����������'���O�������Qr48���hX��(�_`��+��*�}O����:�N&����/�g���t2N������e?���%�Y�$z����m�yt��/��hv`3?D��F��)`����fO���d��w�D������o����~�.6Y/:�~���1�t����������09�|������z
m�����a���?�O�[��	�b��F�?WO;�3x���7�k��l5���E�!�.lu�����it�y�� ~/���<�%]N�� ��'��	6ZN�������|�Gv8��F!���W����?�������f���A���2��P����r������vg����tqE���u��-��F���0�����O��!DK���	���@A��Z����j�-q��)��=��;
��	�_� � 2Yt�~Ci
�uzA9��_�l.
8'���tM�c���:��'�
`�u�v&������^�G���1�-n��-�`�4�$�s��������`4����s��P
�5:%��s����'��@%s�o9�1��3��M�.���|3��4����}��"�e_�@L� 7v���t���_M�#]f�
��`re7��oO�_�4Z�[<t�H���c"w�A�K@+r����:��"��7����C�t�z��Y�%���]>���'�[8��Q&!G��1��A��f�F��a���w�a�)i��G+�0���\�N�]4M.�������Jx�)�<���Z/7��f��y�	���u:�"�q>\7�4�T=]f;*2�6`��N�;"��O����)>��t�u�������ql*�@�FmV��f��(�g�.)">�L�����6��=��5�����.���������HtB�s <�8j��@;f:	��>yJc�F��r=�l=�����bW����p\���������'Y! 8��{	�=Y��E�����e�CW�h��-��[�i7��N�"���w9�����t�Z��&6R'��f�����U�XjN|��m�*`
�U0,�oE�<����/c�7�tYLc0�����X���[B�|�A��d���.����I����7��r �U�\������~,�����o�p
lnq���<F���g��A��}z��%�z���{�e�d�����z��#3@Rj:�j����)��
l���:p E��p��VD�^�o��5����g
q���G8�h���E@����@<�/(� ����O#(��D��~ �\n&�l�/�f��������
�[�&;a��������L���B��
�qe
�3�W��N��p����;�^��F�%"`�;�B�#��F��L��i�M�Q����k��~��}T[|M���`� ]C�g�(��88&vM���V��
I�`�t��H�@}�5:��N������1	��,:�`e�7@����M�������k����T��ND�,k��5�|,��z��U��@�o�0i���"RO��EFu��W(�e�7��E�x�X�{hN�
}-��@�A�P���,�^��H��S�y9�'����JRA5x
e}����E
�'p~���Wx�	�0
��#�l1&�pK�����t��>��!��0�Q���1vu���sT��JC:pt��������3K'pAu�ks(u-��
7LF�x�
BY�fD��|�ZI��u=�7P���P:�i�Y�gf{,���v������b��@B�0Ra
���B���P���(���Qh��O��t��N��j3��'`z��N�<���v�����77`9>O�?o��������,��^}Ng�cNB�d���y`��
:����N�ZN���������v
4���8����D�W0\)���,z"^���m�����A������c@�YF'��C��`�~�}��Ah�@ E7:������~o�������.�./��Y���`2�����tt<@����(Z���>�����	�;�#p���g�V�U��S��N6�u����������w����q���X!�����t���7MDl�N�E4�{�����v9�2d`0�6�Bp�hP�� �e�jd������E`)
�=!���t�'
1�&�K����w�������^�p��l��x
���tI0Y�U���,�j�W
l�r�Zf���������~����%2��$z���}��� A���%�+2D����P>��,�:�
�L�f��&U !I^�����?�@6�\��������3������"�8d��0��.����TIF�����`���d�Yb�A�S������m4`��-�C��9oa^���@��k�Wq�
4B���_,o�*�S�H�B�G-����%�4��$�e1����A���`�.cb\���u���&�uz�t5�fX�RP�jk2��<}�����3b� ���1���������g��|>�I6J�����V��"D(����H�,�l��;h�J��V�'�
�B�0��6����M$T�H�2��)"���
����n��b'R���K���[���V<,����C������&V#p{
�z�$����W���T����Fz���4�
$cs�{���*������@���V��C���5��pu)]\����0Fs���$��eAk,��E��d�X�usz��������'����P?��^9�s���h�d�a�(�i�N/�����)'u�2R�7�yoUl��6��l�����Fl7S�6J��v35l����'�����X����?��]P��CZ����Q`���I0As����-�7s�g��[��g�z�q����?��W=��/��|�����!#~�:s���H�����g�,`f�Dd"�1dE�<����<������C/{��������~�io?�������w�n�.R�;8���2y ���9�^8��K��8t�K��pS��\rXO.9�&���K��%�nr	����[�a�����VN�d���A�4G���2���+����<O�C}��Kp3��+��d��	�'�}"~�����O'������V�	�-��Bzn����w=���
s����<zl0��`�z�e����Cb��;l/{
h`�F�~6�0Z���4�g����-&gs������O����mm�qH��?�Xg�6c�2R�7�kc"���q�3Y�Vd!��g�|M�NAo�7�����W������g(�����1k�7�-y�%����%m~��>gd����
>a�'�V�g�����w�o��.���x�V��nH�\��);���E�?�}{������Z1���Gp��;l���b�-���@$''��S���:[��r�+�s��O^�w����-$N������`��
��~� �o�{�WP���xR	��`	F*��"L�:��%$�A'
��/�d�G�$�r�	&��dH��=���i�e/��1�#�!K ^���p������`j���� ��EZ�,�XhG��i2J!��L����S�hOz�E�0�_f��D:(�����$<e�=�j<=:@�0C	�0��dh�xh�}����1�vX03����X;
i�lXx����=*�F�a)`V���Yy!�?������@gTc|�X���C��P��~6�<b����m4��m "
�8��=����h"��������g|���i:���6����������}XC"���0�7�72H'��r�d��h@&���$���SU$���Z���PX��0H$��a��U�Jo�8�nz�����
�:�p�1z�9��Y�!����1�����z��5gl���
`#��-pshL_��0
YO������o4�-7&�g���'�1L��)��.�(�pB�e���}/�����Bw�IE1[�:��w����|sC`�`��3��Eg2
�%P2C<�c	�*X��^��x�����+�(,'L�V���X'�T���4
@C�H3{W��Vn�����H~��u��V�p��z�B�Z���py������
#271Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#270)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Oct 3, 2017 at 8:57 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Regarding nomenclature and my previous griping about wisdom, I was
wondering about just calling this a "partition join" like you have in
the regression test. So the GUC would be enable_partition_join, you'd
have generate_partition_join_paths(), etc. Basically just delete
"wise" throughout.

Partition-wise join is standard term used in literature and in
documentation of other popular DBMSes, so partition_wise makes more
sense. But I am fine with partition_join as well. Do you want it
partition_join or partitionjoin like enable_mergejoin/enable_hashjoin
etc.?

Well, you're making me have second thoughts. It's really just that
partition_wise looks a little awkward to me, and maybe that's not
enough reason to change anything. I suppose if I commit it this way
and somebody really hates it, it can always be changed later. We're
not getting a lot of input from anyone else at the moment.

Attached the updated patch-set.

I decided to skip over 0001 for today and spend some time looking at
0002-0006. Comments below.

0002:

Looks fine.

0003:

The commit message mentions estimate_num_groups but the patch doesn't touch it.

I am concerned that this patch might introduce some problem fixed by
commit dd4134ea56cb8855aad3988febc45eca28851cd8. The comment in that
patch say, at one place, that "This protects against possible
incorrect matches to child expressions that contain no Vars."
However, if a child expression has no Vars, then I think em->em_relids
will be empty, so the bms_is_equal() test that is there now will fail
but your proposed bms_is_subset() test will pass.

0004:

I suggest renaming get_wholerow_ref_from_convert_row_type to
is_converted_whole_row_reference and making it return a bool.

The coding of that function is a little strange; why not move Var to
an inner scope? Like this: if (IsA(convexpr->arg, var)) { Var *var =
castNode(Var, convexpr->arg; if (var->varattno == 0) return var; }

Will the statement that "In case of multi-level partitioning, we will
have as many nested ConvertRowtypeExpr as there are levels in
partition hierarchy" be falsified by Amit Khandekar's pending patch to
avoid sticking a ConvertRowTypeExpr on top of another
ConvertRowTypeExpr? Even if the answer is "no", I think it might be
better to drop this part of the comment; it would be easy for it to
become false in the future, because we might want to optimize that
case in the future and we'll probably forget to update this comment
when we do.

In fix_upper_expr_mutator(), you have an if statement whose entire
contents are another if statement. I think you should use && instead,
and maybe reverse the order of the tests, since
context->subplan_itlist->has_conv_whole_rows is probably cheaper to
test than a function call. It's also a little strange that this code
isn't adjacent too, or merged with, the existing has_non_vars case.
Maybe:

converted_whole_row = is_converted_whole_row_reference(node);
if (context->outer_itlist && (context->outer_itlist->has_non_vars ||
(context->outer_itlist->has_conv_whole_rows && converted_whole_row))
...
if (context->inner_itlist && (context->inner_itlist->has_non_vars ||
(context->inner_itlist->has_conv_whole_rows && converted_whole_row))
...

0005:

The comment explaining why the ParamPathInfo is allocated in the same
context as the RelOptInfo is a modified copy of an existing comment
that still reads like the original, a manner of commenting I find a
bit undesirable as it leads to filling up the source base with
duplicate comments.

I don't think I believe that comment, either. In the case from which
that comment was copied (mark_dummy_rel), it was talking about a
RelOptInfo, and geqo_eval() takes care to remove any leftover pointers
to joinrels creating during a GEQO cycle. But there's no similar
logic for ppilist, so I think what will happen here is that you'll end
up with a freed node in the middle of the list.

I think reparameterize_path_by_chid() could use a helper function
reparameterize_pathlist_by_child() that iterates over a list of paths
and returns a list of paths. That would remove some of the loops.

I think the comments for reparameterize_path_by_child() need to be
expanded. They don't explain how you decided which nodes need to be
handled here or which fields within those nodes need some kind of
handling other than a flat-copy. I think these kinds of explanations
will be important for future maintenance of this code. You know why
you did it this way, I can mostly guess what you did it this way, but
what about the next person who comes along who hasn't made a detailed
study of partition-wise join?

I don't see much point in the T_SubqueryScanPath and T_ResultPath
cases in reparameterize_path_by_child(). It's just falling through to
the default case.

I wonder if reparameterize_path_by_child() ought to default to
returning NULL rather than throwing an error; the caller would then
have to be prepared for that and skip building the path. But that
would be more like what reparameterize_path() does, and it would make
failure to include some relevant path type here a corner-case
performance bug rather than a correctness issue. It seems like
someone adding a new path type could quite easily fail to realize that
it might need to be added here, or might be unsure whether it's
necessary to add it here.

0006:

I have some doubts about how stable all of the EXPLAIN outputs are
going to be on the buildfarm. I'm not sure what we can really do
about that in advance of trying them, but it's a lot of EXPLAIN
output. If you have an ideas about how to tighten it up without
losing test coverage, that would be good. For example, maybe the
"full outer join" case isn't needed given the following test case
which is also a full outer join but which covers additional behavior.

I think it would be good to have a test case that shows multi-level
partition-wise join working across multiple levels. I wrote the
attached test, which you're welcome to use if you like it, adapt if
you sorta like it, or replace if you dislike it. The table names at
least should be changed to something less likely to duplicate other
tests.

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

Attachments:

mlpartjoin.sqlapplication/octet-stream; name=mlpartjoin.sqlDownload
#272Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Robert Haas (#271)
Re: Partition-wise join for join between (declaratively) partitioned tables

On 2017/10/04 4:27, Robert Haas wrote:

On Tue, Oct 3, 2017 at 8:57 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Regarding nomenclature and my previous griping about wisdom, I was
wondering about just calling this a "partition join" like you have in
the regression test. So the GUC would be enable_partition_join, you'd
have generate_partition_join_paths(), etc. Basically just delete
"wise" throughout.

Partition-wise join is standard term used in literature and in
documentation of other popular DBMSes, so partition_wise makes more
sense. But I am fine with partition_join as well. Do you want it
partition_join or partitionjoin like enable_mergejoin/enable_hashjoin
etc.?

Well, you're making me have second thoughts. It's really just that
partition_wise looks a little awkward to me, and maybe that's not
enough reason to change anything. I suppose if I commit it this way
and somebody really hates it, it can always be changed later. We're
not getting a lot of input from anyone else at the moment.

FWIW, the name enable_partition_join seems enough to convey the core
feature, that is, I see "_wise" as redundant, even though I'm now quite
used to seeing "_wise" in the emails here and saying it out loud every now
and then. Ashutosh may have a point though that users coming from other
databases might miss the "_wise". :)

Thanks,
Amit

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

#273Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#271)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Oct 4, 2017 at 12:57 AM, Robert Haas <robertmhaas@gmail.com> wrote:

0003:

The commit message mentions estimate_num_groups but the patch doesn't touch it.

This was fixed when we converted many rel->reloptkind ==
RELOPT_BASEREL to IS_SIMPLE_REL(). I have removed this section from
the commit message.

I am concerned that this patch might introduce some problem fixed by
commit dd4134ea56cb8855aad3988febc45eca28851cd8. The comment in that
patch say, at one place, that "This protects against possible
incorrect matches to child expressions that contain no Vars."
However, if a child expression has no Vars, then I think em->em_relids
will be empty, so the bms_is_equal() test that is there now will fail
but your proposed bms_is_subset() test will pass.

bms_is_equal() was enough when there was only a single member in
relids but it doesn't work now that there can be multiple of them.
bms_is_equal() was replaced with bms_is_subset() to accomodate for
ec_members with only a subset of relids when we are searching for a
join relation.

I am not sure whether your assumption that expression with no Vars
would have em_relids empty is correct. I wonder whether we will add
any em_is_child members with empty em_relids; looking at
process_equivalence() those come from RestrictInfo::left/right_relids
which just indicates the relids at which that particular expression
can be evaluated. Place holder vars is an example when that can
happen, but there may be others. To verify this, I tried attached
patch on master and ran make check. The assertion didn't trip. If
em_relids is not NULL, bms_is_subset() is fine.

If em_relids could indeed go NULL when em_is_child is true, passing
NULL relids (for parent rels) to that function can cause unwanted
behaviour. bms_is_equal(em->em_relids, relids) will return true
turning the if (em->em_is_child && !bms_is_equal(em->em_relids,
relids)) to false. This means that we will consider a child member
with em_relids NULL even while matching a parent relation. What
surprises me is, that commit added a bunch of testcases and none of
them failed with this change.

Nonetheless, I have changed "matches" with "belongs to" in the
prologue of those functions since an exact match won't be possible
with child-joins.

0004:

I suggest renaming get_wholerow_ref_from_convert_row_type to
is_converted_whole_row_reference and making it return a bool.

Done.

The coding of that function is a little strange; why not move Var to
an inner scope? Like this: if (IsA(convexpr->arg, var)) { Var *var =
castNode(Var, convexpr->arg; if (var->varattno == 0) return var; }

I probably went too far to avoid indented code :). Fixed now.

Will the statement that "In case of multi-level partitioning, we will
have as many nested ConvertRowtypeExpr as there are levels in
partition hierarchy" be falsified by Amit Khandekar's pending patch to
avoid sticking a ConvertRowTypeExpr on top of another
ConvertRowTypeExpr? Even if the answer is "no", I think it might be
better to drop this part of the comment; it would be easy for it to
become false in the future, because we might want to optimize that
case in the future and we'll probably forget to update this comment
when we do.

That might keep someone wondering where the nested
ConvertRowtypeExpr's came from. But may be in future those can arise
from something other than multi-level partition hierarchy and in that
case too the comment would be rendered inaccurate. So done.

In fix_upper_expr_mutator(), you have an if statement whose entire
contents are another if statement. I think you should use && instead,
and maybe reverse the order of the tests, since
context->subplan_itlist->has_conv_whole_rows is probably cheaper to
test than a function call. It's also a little strange that this code
isn't adjacent too, or merged with, the existing has_non_vars case.
Maybe:

converted_whole_row = is_converted_whole_row_reference(node);
if (context->outer_itlist && (context->outer_itlist->has_non_vars ||
(context->outer_itlist->has_conv_whole_rows && converted_whole_row))
...
if (context->inner_itlist && (context->inner_itlist->has_non_vars ||
(context->inner_itlist->has_conv_whole_rows && converted_whole_row))

I placed it with the other node types since it's for a specific node
type, but I guess your suggestion avoids duplicates and looks better.
Done.

...

0005:

The comment explaining why the ParamPathInfo is allocated in the same
context as the RelOptInfo is a modified copy of an existing comment
that still reads like the original, a manner of commenting I find a
bit undesirable as it leads to filling up the source base with
duplicate comments.

I have pointed to mark_dummy_rel() in that comment instead of
duplicating the whole paragraph.

I don't think I believe that comment, either. In the case from which
that comment was copied (mark_dummy_rel), it was talking about a
RelOptInfo, and geqo_eval() takes care to remove any leftover pointers
to joinrels creating during a GEQO cycle. But there's no similar
logic for ppilist, so I think what will happen here is that you'll end
up with a freed node in the middle of the list.

In mark_dummy_rel() it's not about RelOptInfo, it's about the pathlist
with dummy path being created in the same context as the RelOptInfo.
Same applies here. While reparameterizing a path tree, we may reach a
path for a base relation and create a PPI for a base relation. This
may happen when GEQO is planning a join, and thus we are in a
short-lived context created by that GEQO cycle. We don't want a base
rel PPI to be created in that context, so instead we use the context
of base rel itself. Other way round, we don't want to use a longer
context for creating PPI for a join relation when it's created by a
GEQO cycle. So, we use join relation's context.The code doesn't free
up a node in the middle of the list but it avoids such an anomaly. See
[1]: /messages/by-id/CAFjFpRcPutbr4nVAsrY-5q=wCFrNK25_3MNhHgyYYM0yeOoj=Q@mail.gmail.com

I think reparameterize_path_by_chid() could use a helper function
reparameterize_pathlist_by_child() that iterates over a list of paths
and returns a list of paths. That would remove some of the loops.

That's a good idea. Done.

I think the comments for reparameterize_path_by_child() need to be
expanded. They don't explain how you decided which nodes need to be
handled here or which fields within those nodes need some kind of
handling other than a flat-copy. I think these kinds of explanations
will be important for future maintenance of this code. You know why
you did it this way, I can mostly guess what you did it this way, but
what about the next person who comes along who hasn't made a detailed
study of partition-wise join?

We need to reparameterize any path which contains further paths and/or
contains expressions that point to the parent relation. For a given
path we need to reparameterize any paths that it contains and
translate any expressions that are specific to that path. Expressions
common across the paths are translated after the switch case. I have
added this rule to the comment just above the switch case
/*
* Copy of the given path. Reparameterize any paths referenced by the given
* path. Replace parent Vars in path specific expressions by corresponding
* child Vars.
*/
Does that look fine or we want to add explanation for every node handled here.

I don't see much point in the T_SubqueryScanPath and T_ResultPath
cases in reparameterize_path_by_child(). It's just falling through to
the default case.

I added those cases separately to explain why we should not see those
cases in that switch case. I think that explanation is important
(esp. considering your comment above) and associating those comment
with "case" statement looks better. Are you suggesting that we should
add that explanation in default case?

I wonder if reparameterize_path_by_child() ought to default to
returning NULL rather than throwing an error; the caller would then
have to be prepared for that and skip building the path. But that
would be more like what reparameterize_path() does, and it would make
failure to include some relevant path type here a corner-case
performance bug rather than a correctness issue. It seems like
someone adding a new path type could quite easily fail to realize that
it might need to be added here, or might be unsure whether it's
necessary to add it here.

I am OK with that. However reparameterize_path_by_child() and
reparameterize_paths_by_child() are callers of
reparameterize_path_by_child() so they will need to deal with NULL
return. I am fine with that too, but making sure that we are on the
same page. If we do that, we could simply assert that the switch case
doesn't see T_SubqueryScanPath and T_ResultPath.

0006:

I have some doubts about how stable all of the EXPLAIN outputs are
going to be on the buildfarm. I'm not sure what we can really do
about that in advance of trying them, but it's a lot of EXPLAIN
output. If you have an ideas about how to tighten it up without
losing test coverage, that would be good. For example, maybe the
"full outer join" case isn't needed given the following test case
which is also a full outer join but which covers additional behavior.

Yes, I too am thinking about the same. The only reason I have EXPLAIN
output there is to check whether partition-wise join is being used or
not. The testcase is not interested in the actual shape. It doesn't
make sense to just test the output if partition-wise join is not used.
May be a function examining the plan tree would help. The function
will have to handle Result/Sort nodes on top and make sure that Append
has join children. Do you have any other idea to check the shape of
the plan tree without the details? Any EXPLAIN switch, existing
functions etc.?

Removed the extra full outer join testcase.

I think it would be good to have a test case that shows multi-level
partition-wise join working across multiple levels. I wrote the
attached test, which you're welcome to use if you like it, adapt if
you sorta like it, or replace if you dislike it. The table names at
least should be changed to something less likely to duplicate other
tests.

There are tests for multi-level partitioned table in the file. They
test whole partition hierarchy join, part of it being joined based on
the quals. Search for
--
-- multi-leveled partitions
--

Have you looked at those? They test two-level partitioned tables and
your test tests three-level partitioned table. I can modify the tests
to have three levels of partitions and different partition schemes on
different levels. Is that what you expect?

[1]: /messages/by-id/CAFjFpRcPutbr4nVAsrY-5q=wCFrNK25_3MNhHgyYYM0yeOoj=Q@mail.gmail.com

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

Attachments:

em_is_child_em_relids.patchtext/x-patch; charset=US-ASCII; name=em_is_child_em_relids.patchDownload
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 2821662..78eec0a 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -5680,6 +5680,7 @@ prepare_sort_from_pathkeys(Plan *lefttree, List *pathkeys,
 				if (em->em_is_const)
 					continue;
 
+				Assert(!em->em_is_child || !bms_is_empty(em->em_relids));
 				/*
 				 * Ignore child members unless they match the rel being
 				 * sorted.
@@ -5796,6 +5797,8 @@ find_ec_member_for_tle(EquivalenceClass *ec,
 		if (em->em_is_const)
 			continue;
 
+		Assert(!em->em_is_child || !bms_is_empty(em->em_relids));
+
 		/*
 		 * Ignore child members unless they match the rel being sorted.
 		 */
#274Robert Haas
robertmhaas@gmail.com
In reply to: Robert Haas (#264)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Sep 21, 2017 at 8:52 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Thu, Sep 21, 2017 at 8:21 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

About your earlier comment of making build_joinrel_partition_info()
simpler. Right now, the code assumes that partexprs or
nullable_partexpr can be NULL when either of them is not populated.
That may be saves a sizeof(pointer) * (number of keys) byes of memory.
Saving that much memory may not be worth the complexity of code. So,
we may always allocate memory for those arrays and fill it with NIL
values when there are no key expressions to populate those. That will
simplify the code. I haven't done that change in this patchset. I was
busy debugging the Q7 regression. Let me know your comments about
that.

Hmm, I'm not sure that's the best approach, but let me look at it more
carefully before I express a firm opinion.

Having studied this a bit more, I now think your proposed approach is
a good idea.

--
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

#275Robert Haas
robertmhaas@gmail.com
In reply to: Robert Haas (#271)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Tue, Oct 3, 2017 at 3:27 PM, Robert Haas <robertmhaas@gmail.com> wrote:

I decided to skip over 0001 for today and spend some time looking at
0002-0006.

Back to 0001.

+        Enables or disables the query planner's use of partition-wise join
+        plans. When enabled, it spends time in creating paths for joins between
+        partitions and consumes memory to construct expression nodes to be used
+        for those joins, even if partition-wise join does not result in the
+        cheapest path. The time and memory increase exponentially with the
+        number of partitioned tables being joined and they increase linearly
+        with the number of partitions. The default is <literal>off</>.

I think this is too scary and too much technical detail. I think you
could just say something like: Enables or disables use of
partition-wise join, which allows a join between partitioned tables to
be performed by joining the matching partitions. Partition-wise join
currently applies only when the join conditions include all the
columns of the partition keys, which must be of the same data type and
have exactly matching sets of child partitions. Because
partition-wise join planning can use significantly increase CPU time
and memory usage during planning, the default is <literal>off</>.

+partitioned table. The join partners can not be found in other partitions. This
+condition allows the join between partitioned tables to be broken into joins
+between the matching partitions. The resultant join is partitioned in the same

"The join partners can not be found in other partitions." is redundant
with the previous sentence. I suggest deleting it. I also suggest
"This condition allows the join between partitioned tables to be
broken" -> "Because of this, the join between partitioned tables can
be broken".

+relation" for both partitioned table as well as join between partitioned tables
+which can use partition-wise join technique.

for either a partitioned table or a join between compatibly partitioned tables

+Partitioning properties of a partitioned relation are stored in
+PartitionSchemeData structure. Planner maintains a list of canonical partition
+schemes (distinct PartitionSchemeData objects) so that any two partitioned
+relations with same partitioning scheme share the same PartitionSchemeData
+object. This reduces memory consumed by PartitionSchemeData objects and makes
+it easy to compare the partition schemes of joining relations.

Not all of the partitioning properties are stored in the
PartitionSchemeData structure any more. I think this needs some
rethinking and maybe some expansion. As written, each of the first
two sentences needs a "the" at the beginning.

+                               /*
+                                * Create "append" paths for
partitioned joins. Do this before
+                                * creating GatherPaths so that
partial "append" paths in
+                                * partitioned joins will be considered.
+                                */

I think you could shorten this to a single-line comment and just keep
the first sentence. Similarly in the other location where you have
the same sort of thing.

+ * child-joins. Otherwise, add_path might delete a path that some "append"
+ * path has reference to.

to which some path generated here has a reference.

Here and elsewhere, you use "append" rather than Append to refer to
the paths added. I suppose that's weasel-wording to work around the
fact that they might be either Append or MergeAppend paths, but I'm
not sure it's really going to convey that to anyone. I suggest
rephrasing those comments more generically, e.g.:

+ /* Add "append" paths containing paths from child-joins. */

You could say: Build additional paths for this rel from child-join paths.

Or something.

+       if (!REL_HAS_ALL_PART_PROPS(rel))
+               return;

Isn't this an unnecessarily expensive test? I mean, it shouldn't be
possible for it to have some arbitrary subset.

+       /*
+        * Every pair of joining relations we see here should have an equi-join
+        * between partition keys if this join has been deemed as a partitioned
+        * join. See build_joinrel_partition_info() for reasons.
+        */
+       Assert(have_partkey_equi_join(rel1, rel2, parent_sjinfo->jointype,
+
parent_restrictlist));

I suggest removing this assertion. Seems like overkill to me.

+               child_sjinfo = build_child_join_sjinfo(root, parent_sjinfo,
+
                    child_rel1->relids,
+
                    child_rel2->relids);

It seems like we might end up doing this multiple times for the same
child join, if there are more than 2 tables involved. Not sure if
there's a good way to avoid that. Similarly for child_restrictlist.

+ pk_has_clause = (bool *) palloc0(sizeof(bool) * num_pks);

Just do bool pk_has_clause[PARTITION_MAX_KEYS] instead. Stack
allocation is a lot faster, and then you don't need to pfree it.

+ /* Remove the relabel decoration. */

the -> any, decoration -> decorations

+       /*
+        * Replace the Var nodes of parent with those of children in
expressions.
+        * This function may be called within a temporary context, but the
+        * expressions will be shallow-copied into the plan. Hence copy those in
+        * the planner's context.
+        */

I can't make heads or tails of this comment.

--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -23,7 +23,9 @@
 #include "optimizer/pathnode.h"
 #include "optimizer/paths.h"
 #include "optimizer/planmain.h"
+#include "optimizer/prep.h"
 #include "optimizer/restrictinfo.h"
+#include "optimizer/tlist.h"
 #include "optimizer/var.h"
 #include "parser/parsetree.h"
 #include "utils/lsyscache.h"

Maybe not needed? This is the only hunk in this file? Or should this
be part of one of the later patches?

+       Assert(IS_JOIN_REL(childrel) && IS_JOIN_REL(parentrel));
+
+       /* Ensure child relation is really what it claims to be. */
+       Assert(IS_OTHER_REL(childrel));

I suggest tightening this up a bit by removing the comment and the
blank line that precedes it.

+       foreach(lc, parentrel->reltarget->exprs)
+       {
+               PlaceHolderVar *phv = lfirst(lc);
+
+               if (IsA(phv, PlaceHolderVar))
+               {
+                       /*
+                        * In case the placeholder Var refers to any
of the parent
+                        * relations, translate it to refer to the
corresponding child.
+                        */
+                       if (bms_overlap(phv->phrels, parentrel->relids) &&
+                               childrel->reloptkind == RELOPT_OTHER_JOINREL)
+                       {
+                               phv = (PlaceHolderVar *)
adjust_appendrel_attrs(root,
+
                                                         (Node *) phv,
+
                                                         nappinfos,
+
                                                         appinfos);
+                       }
+
+                       childrel->reltarget->exprs =
lappend(childrel->reltarget->exprs,
+
                          phv);
+                       phv_added = true;
+               }
+       }

What if the PHV is buried down inside the expression someplace rather
than being at the top level? More generally, why are we not just
applying adjust_appendrel_attrs() to the whole expression?

+       /* Adjust the cost and width of child targetlist. */
+       if (phv_added)
+       {
+               childrel->reltarget->cost.startup =
parentrel->reltarget->cost.startup;
+               childrel->reltarget->cost.per_tuple =
parentrel->reltarget->cost.per_tuple;
+               childrel->reltarget->width = parentrel->reltarget->width;
+       }

Making this conditional on phv_added is probably not saving anything.
Branches are expensive.

                /*
                 * Otherwise, anything in a baserel or joinrel
targetlist ought to be
-                * a Var.  (More general cases can only appear in
appendrel child
-                * rels, which will never be seen here.)
+                * a Var or ConvertRowtypeExpr. For either of those,
find the original
+                * baserel where they originate.
                 */

Hmm, but now we could potentially see an appendrel child rel here, so
don't we need to worry about more general cases? If not, let's
explain why not.

+ * if, it's a ConvertRowtypeExpr, it will be
computed only for the

American usage does not put a comma after if like this (unless you are
writing writing if, for example, blah blah blah -- but there the
commas are to surround for example, not due to the if itself).

+/*
+ * build_joinrel_partition_info
+ *             If the join between given partitioned relations is
possibly partitioned
+ *             set the partitioning scheme and partition keys
expressions for the
+ *             join.
+ *
+ * If the two relations have same partitioning scheme, their join may be
+ * partitioned and will follow the same partitioning scheme as the joining
+ * relations.
+ */

I think you could drop the primary comment block and use the secondary
block as the primary one. That is, get rid of "If the join
between..." and promote "If the two relations...".

+ * The join is not partitioned, if any of the relations being joined are

Another comma that's not typical of American usage.

+ * For an N-way inner join, where every syntactic inner join
has equi-join

has -> has an

+        * For an N-way join with outer joins, where every syntactic join has an
+        * equi-join between partition keys and a matching partitioning scheme,
+        * outer join reordering identities in optimizer/README imply that only
+        * those pairs of join are legal which have an equi-join
between partition
+        * keys. Thus every pair of joining relations we see for this
join should
+        * have an equi-join between partition keys if this join has been deemed
+        * as a partitioned join.

In line 2, partition keys -> the partition keys
In line 3, outer join -> the outer join

"pairs of join" sounds wrong too, although I'm not sure how to reword it.

More broadly: I don't think I understand this comment. The statement
about "those pairs of join are legal which have an equi-join between
partition keys" doesn't match my understanding e.g. A IJ B ON A.x =
B.x LJ C ON A.x = C.x surely allows a B-C join, but there's no such
clause syntatically.

Maybe you could replace this whole comment block with something like
this: We can only consider this join as an input to further
partition-wise joins if (a) the input relations are partitioned, (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.

+        * Join relation is partitioned using same partitioning scheme as the
+        * joining relations and has same bounds.

the same partitioning scheme

+        * An INNER join between two partitioned relations is partitioned by key
+        * expressions from both the relations. For tables A and B
partitioned by
+        * a and b respectively, (A INNER JOIN B ON A.a = B.b) is partitioned by
+        * both A.a and B.b.
+        *
+        * A SEMI/ANTI join only retains data from the outer side and is
+        * partitioned by the partition keys of the outer side.

I would write: An INNER join between two partitioned relations can be
regarded as partitioned by either key expression. For example, A
INNER JOIN B ON A.a = B.b can be regarded as partitioned on A.a or on
B.b; they are equivalent. For a SEMI or ANTI join, the result can
only be regarded as being partitioned in the same manner as the outer
side, since the inner columns are not retained.

+        * An OUTER join like (A LEFT JOIN B ON A.a = B.b) may produce rows with
+        * B.b NULL. These rows may not fit the partitioning
conditions imposed on
+        * B.b. Hence, strictly speaking, the join is not partitioned by B.b.

Good.

+        * Strictly speaking, partition keys of an OUTER join should include
+        * partition key expressions from the OUTER side only. Consider a join

I would join this with the previous sentence instead of repeating
strictly speaking: ...and thus the partition keys should include
partition key expressions from the OUTER side only. After that
sentence, I'd skip a lot of the intermediate words here and continue
this way: However, because all commonly-used comparison operators are
strict, the presence of nulls on the outer side doesn't cause any
problem; they can't match anything at future join levels anyway.
Therefore, we track two sets of expressions: those that authentically
partition the relation (partexprs) and those that partition the
relation with the exception that extra nulls may be present
(nullable_partexprs). When the comparison operator is strict, the
latter is just as good as the former.

Then, I think you can omit the rest of what you have; it should be
clear enough what's going on for the full and right cases given that
explanation.

+ * being joined. partexprs and nullable_partexprs are arrays
containing part_scheme->partnatts

Long line, needs reflowing.

I don't think this is too far from being committable. You've done
some nice work here!

--
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

#276Robert Haas
robertmhaas@gmail.com
In reply to: Robert Haas (#275)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Oct 4, 2017 at 11:34 AM, Robert Haas <robertmhaas@gmail.com> wrote:

+        Enables or disables the query planner's use of partition-wise join
+        plans. When enabled, it spends time in creating paths for joins between
+        partitions and consumes memory to construct expression nodes to be used
+        for those joins, even if partition-wise join does not result in the
+        cheapest path. The time and memory increase exponentially with the
+        number of partitioned tables being joined and they increase linearly
+        with the number of partitions. The default is <literal>off</>.

I think this is too scary and too much technical detail. I think you
could just say something like: Enables or disables use of
partition-wise join, which allows a join between partitioned tables to
be performed by joining the matching partitions. Partition-wise join
currently applies only when the join conditions include all the
columns of the partition keys, which must be of the same data type and
have exactly matching sets of child partitions. Because
partition-wise join planning can use significantly increase CPU time
and memory usage during planning, the default is <literal>off</>.

Not enough caffeine, obviously: should have been something like --
Because partition-wise join can significantly increase the CPU and
memory costs of planning...

--
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

#277Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#273)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Oct 4, 2017 at 8:23 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

I am not sure whether your assumption that expression with no Vars
would have em_relids empty is correct. I wonder whether we will add
any em_is_child members with empty em_relids; looking at
process_equivalence() those come from RestrictInfo::left/right_relids
which just indicates the relids at which that particular expression
can be evaluated. Place holder vars is an example when that can
happen, but there may be others. To verify this, I tried attached
patch on master and ran make check. The assertion didn't trip. If
em_relids is not NULL, bms_is_subset() is fine.

I spent some more time experimenting with this. I found that cases
where an em_is_child equivalence class contains multiple relids are
quite easy to generate, e.g. select * from foo, bar where foo.a +
bar.a = 0, where foo and bar are partitioned. However, I wasn't able
to generate a case where an em_is_child equivalence class has no
relids at all, and I'm out of ideas about how such a thing could
occur. I suspect it can't. I wondered whether there was some problem
with the multiple-relids case, but I can't find an example where that
misbehaves either. So maybe it's fine (or maybe I'm just not smart
enough to find the case where it breaks).

I don't think I believe that comment, either. In the case from which
that comment was copied (mark_dummy_rel), it was talking about a
RelOptInfo, and geqo_eval() takes care to remove any leftover pointers
to joinrels creating during a GEQO cycle. But there's no similar
logic for ppilist, so I think what will happen here is that you'll end
up with a freed node in the middle of the list.

In mark_dummy_rel() it's not about RelOptInfo, it's about the pathlist
with dummy path being created in the same context as the RelOptInfo.
Same applies here.

Oops. I was thinking that the ppilist was attached to some
planner-global structure, but it's not; it's hanging off the
RelOptInfo. So you're entirely right, and I'm just being dumb.

We need to reparameterize any path which contains further paths and/or
contains expressions that point to the parent relation. For a given
path we need to reparameterize any paths that it contains and
translate any expressions that are specific to that path. Expressions
common across the paths are translated after the switch case. I have
added this rule to the comment just above the switch case
/*
* Copy of the given path. Reparameterize any paths referenced by the given
* path. Replace parent Vars in path specific expressions by corresponding
* child Vars.
*/
Does that look fine or we want to add explanation for every node handled here.

No, I don't think we want something for every node, just a general
explanation at the top of the function. Maybe something like this:

Most fields from the original path can simply be flat-copied, but any
expressions must be adjusted to refer to the correct varnos, and any
paths must be recursively reparameterized. Other fields that refer to
specific relids also need adjustment.

I don't see much point in the T_SubqueryScanPath and T_ResultPath
cases in reparameterize_path_by_child(). It's just falling through to
the default case.

I added those cases separately to explain why we should not see those
cases in that switch case. I think that explanation is important
(esp. considering your comment above) and associating those comment
with "case" statement looks better. Are you suggesting that we should
add that explanation in default case?

Or leave the explanation out altogether.

I wonder if reparameterize_path_by_child() ought to default to
returning NULL rather than throwing an error; the caller would then
have to be prepared for that and skip building the path. But that
would be more like what reparameterize_path() does, and it would make
failure to include some relevant path type here a corner-case
performance bug rather than a correctness issue. It seems like
someone adding a new path type could quite easily fail to realize that
it might need to be added here, or might be unsure whether it's
necessary to add it here.

I am OK with that. However reparameterize_path_by_child() and
reparameterize_paths_by_child() are callers of
reparameterize_path_by_child() so they will need to deal with NULL
return. I am fine with that too, but making sure that we are on the
same page. If we do that, we could simply assert that the switch case
doesn't see T_SubqueryScanPath and T_ResultPath.

Or do nothing at all about those cases.

I noticed today that the version of the patchset I have here says in
the header comments for reparameterize_path_by_child() that it returns
NULL if it can't reparameterize, but that's not what it actually does.
If you make this change, the existing comment will become correct.

The problem with the NULL return convention is that it's not very
convenient when this function is recursing. Maybe we should change
this function's signature to be bool
reparameterize_path_by_child(PlannerInfo *root, RelOptInfo *child_rel,
Path **path); then you could do, e.g. if
(!reparameterize_path_by_child(root, child_rel, &bhpath->bitmapqual))
return;

But I don't really like that approach; it's still quite long-winded.
Instead, I suggest Stupid Macro Tricks:

#define ADJUST_CHILD_ATTRS(val) \
val = (List *) adjust_appendrel_attrs_multilevel((Node *) val,
child_rel->relids, child_rel->top_parent_relids);

#define REPARAMETERIZE_CHILD_PATH(val) \
val = reparameterize_path_by_child(root, val, child_rel); \
if (val == NULL) \
return NULL;

#define REPARAMETERIZE_CHILD_PATH_LIST(val) \
if (val != NIL) \
{ \
val = reparameterize_pathlist_by_child(root, val, child_rel); \
if (val == NIL) \
return NULL; \
}

With that, a complicated case like T_NestPath becomes just:

JoinPath *jpath;

FLAT_COPY_PATH(jpath, path, NestPath);
REPARAMETERIZE_CHILD_PATH(jpath->outerjoinpath);
REPARAMETERIZE_CHILD_PATH(jpath->innerjoinpath);
ADJUST_CHILD_ATTRS(jpath->joinrestrictinfo);
new_path = (Path *) jpath;

Now, I admit that hiding stuff inside the macro definitions like that
is ugly. But I think it's still better than repeating boilerplate
code with finnicky internal bits lots of times.

Yes, I too am thinking about the same. The only reason I have EXPLAIN
output there is to check whether partition-wise join is being used or
not. The testcase is not interested in the actual shape. It doesn't
make sense to just test the output if partition-wise join is not used.
May be a function examining the plan tree would help. The function
will have to handle Result/Sort nodes on top and make sure that Append
has join children. Do you have any other idea to check the shape of
the plan tree without the details? Any EXPLAIN switch, existing
functions etc.?

No, not really. We may just need to be prepared to fix whatever breaks.

I think it would be good to have a test case that shows multi-level
partition-wise join working across multiple levels. I wrote the
attached test, which you're welcome to use if you like it, adapt if
you sorta like it, or replace if you dislike it. The table names at
least should be changed to something less likely to duplicate other
tests.

There are tests for multi-level partitioned table in the file. They
test whole partition hierarchy join, part of it being joined based on
the quals. Search for
--
-- multi-leveled partitions
--

Have you looked at those? They test two-level partitioned tables and
your test tests three-level partitioned table. I can modify the tests
to have three levels of partitions and different partition schemes on
different levels. Is that what you expect?

Oops, no, I just missed the test case. I saw the one that said "inner
join, qual covering only top-level partitions" and missed that there
were others later where the quals covered lower levels also.

Instead of "multi-leveled partitions" it might read better to say
"multiple levels of partitioning".

--
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

#278Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#274)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Oct 4, 2017 at 9:01 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Thu, Sep 21, 2017 at 8:52 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Thu, Sep 21, 2017 at 8:21 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

About your earlier comment of making build_joinrel_partition_info()
simpler. Right now, the code assumes that partexprs or
nullable_partexpr can be NULL when either of them is not populated.
That may be saves a sizeof(pointer) * (number of keys) byes of memory.
Saving that much memory may not be worth the complexity of code. So,
we may always allocate memory for those arrays and fill it with NIL
values when there are no key expressions to populate those. That will
simplify the code. I haven't done that change in this patchset. I was
busy debugging the Q7 regression. Let me know your comments about
that.

Hmm, I'm not sure that's the best approach, but let me look at it more
carefully before I express a firm opinion.

Having studied this a bit more, I now think your proposed approach is
a good idea.

Thanks. Done.

--
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

#279Alvaro Herrera
alvherre@2ndquadrant.com
In reply to: Robert Haas (#269)
Re: Partition-wise join for join between (declaratively) partitioned tables

Robert Haas wrote:

Regarding nomenclature and my previous griping about wisdom, I was
wondering about just calling this a "partition join" like you have in
the regression test. So the GUC would be enable_partition_join, you'd
have generate_partition_join_paths(), etc. Basically just delete
"wise" throughout.

If I understand correctly, what's being used here is the "-wise" suffix,
unrelated to wisdom, which Merriam Webster lists as "adverb combining
form" here https://www.merriam-webster.com/dictionary/wise (though you
have to scroll down a lot), which is defined as

1 a :in the manner of * crabwise * fanwise
b :in the position or direction of * slantwise * clockwise
2 :with regard to :in respect of * dollarwise

According to that, the right way to write this is "partitionwise join"
(no dash), which means "join in respect of partitions", "join with
regard to partitions".

--
�lvaro Herrera https://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

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

#280Robert Haas
robertmhaas@gmail.com
In reply to: Alvaro Herrera (#279)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Oct 5, 2017 at 9:48 AM, Alvaro Herrera <alvherre@2ndquadrant.com> wrote:

Robert Haas wrote:

Regarding nomenclature and my previous griping about wisdom, I was
wondering about just calling this a "partition join" like you have in
the regression test. So the GUC would be enable_partition_join, you'd
have generate_partition_join_paths(), etc. Basically just delete
"wise" throughout.

If I understand correctly, what's being used here is the "-wise" suffix,
unrelated to wisdom, which Merriam Webster lists as "adverb combining
form" here https://www.merriam-webster.com/dictionary/wise (though you
have to scroll down a lot), which is defined as

1 a :in the manner of * crabwise * fanwise
b :in the position or direction of * slantwise * clockwise
2 :with regard to :in respect of * dollarwise

According to that, the right way to write this is "partitionwise join"
(no dash), which means "join in respect of partitions", "join with
regard to partitions".

I'm fine with that, if others like it.

--
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

#281Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Alvaro Herrera (#279)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Oct 5, 2017 at 7:18 PM, Alvaro Herrera <alvherre@2ndquadrant.com> wrote:

Robert Haas wrote:

Regarding nomenclature and my previous griping about wisdom, I was
wondering about just calling this a "partition join" like you have in
the regression test. So the GUC would be enable_partition_join, you'd
have generate_partition_join_paths(), etc. Basically just delete
"wise" throughout.

If I understand correctly, what's being used here is the "-wise" suffix,
unrelated to wisdom, which Merriam Webster lists as "adverb combining
form" here https://www.merriam-webster.com/dictionary/wise (though you
have to scroll down a lot), which is defined as

1 a :in the manner of * crabwise * fanwise
b :in the position or direction of * slantwise * clockwise
2 :with regard to :in respect of * dollarwise

That's right.

According to that, the right way to write this is "partitionwise join"
(no dash), which means "join in respect of partitions", "join with
regard to partitions".

Google lists mostly "partition wise" or "partition-wise" and very
rarely "partitionwise". The first being used in other DBMS literature.
The paper (there aren't many on this subject) I referred [1]https://users.cs.duke.edu/~shivnath/papers/sigmod295-herodotou.pdf uses
"partition-wise". It made more sense to replace " " or "-" with "_"
when syntax doesn't allow the first two. I am not against
"partitionwise" but I don't see any real reason why we should move
away from popular usage of this term.

[1]: https://users.cs.duke.edu/~shivnath/papers/sigmod295-herodotou.pdf

--
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

#282Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#275)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Oct 4, 2017 at 9:04 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Tue, Oct 3, 2017 at 3:27 PM, Robert Haas <robertmhaas@gmail.com> wrote:

I decided to skip over 0001 for today and spend some time looking at
0002-0006.

Back to 0001.

+        Enables or disables the query planner's use of partition-wise join
+        plans. When enabled, it spends time in creating paths for joins between
+        partitions and consumes memory to construct expression nodes to be used
+        for those joins, even if partition-wise join does not result in the
+        cheapest path. The time and memory increase exponentially with the
+        number of partitioned tables being joined and they increase linearly
+        with the number of partitions. The default is <literal>off</>.

I think this is too scary and too much technical detail. I think you
could just say something like: Enables or disables use of
partition-wise join, which allows a join between partitioned tables to
be performed by joining the matching partitions. Partition-wise join
currently applies only when the join conditions include all the
columns of the partition keys, which must be of the same data type and
have exactly matching sets of child partitions. Because
partition-wise join planning can use significantly increase CPU time
and memory usage during planning, the default is <literal>off</>.

Done. With slight change. "include all the columns of the partition
keys" has a different meaning when partition key is an expression, so
I have used "include all the partition keys". Also changed the last
sentence as "... can use significantly more CPU time and memory during
planning ...". Please feel free to revert those changes, if you don't
like them.

+partitioned table. The join partners can not be found in other partitions. This
+condition allows the join between partitioned tables to be broken into joins
+between the matching partitions. The resultant join is partitioned in the same

"The join partners can not be found in other partitions." is redundant
with the previous sentence. I suggest deleting it. I also suggest
"This condition allows the join between partitioned tables to be
broken" -> "Because of this, the join between partitioned tables can
be broken".

Done.

+relation" for both partitioned table as well as join between partitioned tables
+which can use partition-wise join technique.

for either a partitioned table or a join between compatibly partitioned tables

Done.

+Partitioning properties of a partitioned relation are stored in
+PartitionSchemeData structure. Planner maintains a list of canonical partition
+schemes (distinct PartitionSchemeData objects) so that any two partitioned
+relations with same partitioning scheme share the same PartitionSchemeData
+object. This reduces memory consumed by PartitionSchemeData objects and makes
+it easy to compare the partition schemes of joining relations.

Not all of the partitioning properties are stored in the
PartitionSchemeData structure any more. I think this needs some
rethinking and maybe some expansion. As written, each of the first
two sentences needs a "the" at the beginning.

Changed to

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
partition schemes (distinct PartitionSchemeData objects) so that RelOptInfo of
any two partitioned relations with same partitioning scheme point to the same
PartitionSchemeData object. This reduces memory consumed by
PartitionSchemeData objects and makes it easy to compare the partition schemes
of joining relations.

Let me know if this looks good.

+                               /*
+                                * Create "append" paths for
partitioned joins. Do this before
+                                * creating GatherPaths so that
partial "append" paths in
+                                * partitioned joins will be considered.
+                                */

I think you could shorten this to a single-line comment and just keep
the first sentence. Similarly in the other location where you have
the same sort of thing.

Done.

+ * child-joins. Otherwise, add_path might delete a path that some "append"
+ * path has reference to.

to which some path generated here has a reference.

Done.

Here and elsewhere, you use "append" rather than Append to refer to
the paths added. I suppose that's weasel-wording to work around the
fact that they might be either Append or MergeAppend paths, but I'm
not sure it's really going to convey that to anyone. I suggest
rephrasing those comments more generically, e.g.:

+ /* Add "append" paths containing paths from child-joins. */

You could say: Build additional paths for this rel from child-join paths.

Or something.

Done. Removed word "append" from the comments in merge_clump(),
standard_join_search() and prologue of
generate_partition_wise_join_paths(). Changed the last comment as per
your suggestion.

+       if (!REL_HAS_ALL_PART_PROPS(rel))
+               return;
Isn't this an unnecessarily expensive test?  I mean, it shouldn't be
possible for it to have some arbitrary subset.

All this function cares about is whether the given relation has any
partitions which can be simply checked by rel->nparts > 0 and
rel->part_rels != NULL. We need to explicitly check part_rels because
an outer join which has empty inner side in every pair will have
part_scheme, partbounds, nparts all set, but not part_rels. See
relevant comments in try_partition_wise_join() for more details. I
have now replaced macro with checks on rel->nparts and rel->part_rels.
This would change with the last patch dealing with partition-wise join
involving dummy relations. Once we have that an outer join like above
will also have part_rels set. But even then I think checking for
part_rels and nparts makes more sense than part_scheme and partbounds.

+       /*
+        * Every pair of joining relations we see here should have an equi-join
+        * between partition keys if this join has been deemed as a partitioned
+        * join. See build_joinrel_partition_info() for reasons.
+        */
+       Assert(have_partkey_equi_join(rel1, rel2, parent_sjinfo->jointype,
+
parent_restrictlist));

I suggest removing this assertion. Seems like overkill to me.

I thought it was good to have there to catch any bug breaking that
rule. But I have removed it as per your suggestion.
Do you think we should remove following assertions as well?
/*
* 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.
*/
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));

+               child_sjinfo = build_child_join_sjinfo(root, parent_sjinfo,
+
child_rel1->relids,
+
child_rel2->relids);

It seems like we might end up doing this multiple times for the same
child join, if there are more than 2 tables involved. Not sure if
there's a good way to avoid that.

IIUC every pair of joining relations will use a different sjinfo, A
LEFT JOIN B LEFT JOIN C will have two sjinfos one for AB and other for
BC. For ABC we will use the one for AB to join A with BC and we will
use one for BC to join AB with C. I agree that we are building sjinfo
for AB twice once for joining AB and then for A(BC). In order to avoid
that we will have to somehow link the parent sjinfo with child sjinfo
and avoid translating parent sjinfo again and again. May be the parent
sjinfo can contain a cache of child sjinfos.Do we want to do that in
this patch set? We could avoid translation entirely, if we could use
parent sjinfo for joining children. But that's a pretty deep surgery.

Similarly for child_restrictlist.

Similary for restrictlist. Every joining pair has a different
restrictlist. Otherwise, we would have saved restrictlist in the
joinrel itself.

+ pk_has_clause = (bool *) palloc0(sizeof(bool) * num_pks);

Just do bool pk_has_clause[PARTITION_MAX_KEYS] instead. Stack
allocation is a lot faster, and then you don't need to pfree it.

That's a good idea. Done.

+ /* Remove the relabel decoration. */

the -> any, decoration -> decorations

Done.

+       /*
+        * Replace the Var nodes of parent with those of children in
expressions.
+        * This function may be called within a temporary context, but the
+        * expressions will be shallow-copied into the plan. Hence copy those in
+        * the planner's context.
+        */

I can't make heads or tails of this comment.

haha! My bad. the second sentence is something left of the code where
the child-joins used to be planned in a temporary memory context.
That's not true any more. Removed the entire comment.

--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -23,7 +23,9 @@
#include "optimizer/pathnode.h"
#include "optimizer/paths.h"
#include "optimizer/planmain.h"
+#include "optimizer/prep.h"
#include "optimizer/restrictinfo.h"
+#include "optimizer/tlist.h"
#include "optimizer/var.h"
#include "parser/parsetree.h"
#include "utils/lsyscache.h"

Maybe not needed? This is the only hunk in this file? Or should this
be part of one of the later patches?

I think 0005. Sorry. I will move it there.

+       Assert(IS_JOIN_REL(childrel) && IS_JOIN_REL(parentrel));
+
+       /* Ensure child relation is really what it claims to be. */
+       Assert(IS_OTHER_REL(childrel));

I suggest tightening this up a bit by removing the comment and the
blank line that precedes it.

Done.

+       foreach(lc, parentrel->reltarget->exprs)
+       {
+               PlaceHolderVar *phv = lfirst(lc);
+
+               if (IsA(phv, PlaceHolderVar))
+               {
+                       /*
+                        * In case the placeholder Var refers to any
of the parent
+                        * relations, translate it to refer to the
corresponding child.
+                        */
+                       if (bms_overlap(phv->phrels, parentrel->relids) &&
+                               childrel->reloptkind == RELOPT_OTHER_JOINREL)
+                       {
+                               phv = (PlaceHolderVar *)
adjust_appendrel_attrs(root,
+
(Node *) phv,
+
nappinfos,
+
appinfos);
+                       }
+
+                       childrel->reltarget->exprs =
lappend(childrel->reltarget->exprs,
+
phv);
+                       phv_added = true;
+               }
+       }

What if the PHV is buried down inside the expression someplace rather
than being at the top level?

That can't happen. See add_placeholders_to_joinrel(), which adds these
placeholders to joinrel's target. That function adds PHVs as bare
nodes, not embedded into something else.

More generally, why are we not just
applying adjust_appendrel_attrs() to the whole expression?

Usually targetlists of join have Var nodes which bubble up from the
base relations. Even PHVs bubble up from the lowest join where they
can be evaluated. If we translate reltarget, we will allocate new Var
nodes for every join relation consuming more memory and then setrefs
will need to compare the contents of those nodes instead of just
pointer comparison. We use this code and attr_needed to avoid memory
consumption and setref's CPU consumption.

+       /* Adjust the cost and width of child targetlist. */
+       if (phv_added)
+       {
+               childrel->reltarget->cost.startup =
parentrel->reltarget->cost.startup;
+               childrel->reltarget->cost.per_tuple =
parentrel->reltarget->cost.per_tuple;
+               childrel->reltarget->width = parentrel->reltarget->width;
+       }

Making this conditional on phv_added is probably not saving anything.
Branches are expensive.

Ok.

If there are not PHVs in the query i.e. when root->placeholders_list
is NIL, we don't need to scan reltarget->exprs. I have added that
optimization.

/*
* Otherwise, anything in a baserel or joinrel
targetlist ought to be
-                * a Var.  (More general cases can only appear in
appendrel child
-                * rels, which will never be seen here.)
+                * a Var or ConvertRowtypeExpr. For either of those,
find the original
+                * baserel where they originate.
*/

Hmm, but now we could potentially see an appendrel child rel here, so
don't we need to worry about more general cases? If not, let's
explain why not.

By more general cases, that comment means ConvertRowtypeExpr or
RowExpr, nothing else. A base relation's tlist can have only Var nodes
when it reaches this comment. When a parent Var node is subjected to
adjust_appendrel_attrs() it is translated to a Var node for all
varattnos except 0, which indicates a whole-row var. For a child
table, a whole-row var is always a named row type and hence gets
translated to a ConvertRowExpr. Other kinds of children (subqueries in
union etc.) can not appear here since they do not participate in a
join directly. So it's really a Var and ConvertRowtypeExpr. I have
modified the comment to explain this.

+ * if, it's a ConvertRowtypeExpr, it will be
computed only for the

American usage does not put a comma after if like this (unless you are
writing writing if, for example, blah blah blah -- but there the
commas are to surround for example, not due to the if itself).

That comma was unintentional. Removed.

+/*
+ * build_joinrel_partition_info
+ *             If the join between given partitioned relations is
possibly partitioned
+ *             set the partitioning scheme and partition keys
expressions for the
+ *             join.
+ *
+ * If the two relations have same partitioning scheme, their join may be
+ * partitioned and will follow the same partitioning scheme as the joining
+ * relations.
+ */

I think you could drop the primary comment block and use the secondary
block as the primary one. That is, get rid of "If the join
between..." and promote "If the two relations...".

Done.

+ * The join is not partitioned, if any of the relations being joined are

Another comma that's not typical of American usage.

Done.

+ * For an N-way inner join, where every syntactic inner join
has equi-join

has -> has an

+        * For an N-way join with outer joins, where every syntactic join has an
+        * equi-join between partition keys and a matching partitioning scheme,
+        * outer join reordering identities in optimizer/README imply that only
+        * those pairs of join are legal which have an equi-join
between partition
+        * keys. Thus every pair of joining relations we see for this
join should
+        * have an equi-join between partition keys if this join has been deemed
+        * as a partitioned join.

In line 2, partition keys -> the partition keys
In line 3, outer join -> the outer join

"pairs of join" sounds wrong too, although I'm not sure how to reword it.

More broadly: I don't think I understand this comment. The statement
about "those pairs of join are legal which have an equi-join between
partition keys" doesn't match my understanding e.g. A IJ B ON A.x =
B.x LJ C ON A.x = C.x surely allows a B-C join, but there's no such
clause syntatically.

Maybe you could replace this whole comment block with something like
this: We can only consider this join as an input to further
partition-wise joins if (a) the input relations are partitioned, (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.

Hmm. I meant the second para to be read in the context of the first.
Since AB is inner join A.x and B.x are replaceable (I forgot the
correct term, identity?) and thus A.x = C.x implies B.x = C.x thus
allowing join BC. But I think your version of the comment is easy to
understand. But I think it should also refer to the way planner
reorders joins; that's what causes us to worry about every join order
being partitioned. I think we should redirect a reader, who wants to
understand more about implied equalities and join orders, to
optimizer/README. So, I have changed the last sentence to read "That
shouldn't happen, though, because of the way the query planner deduces
implied equalities and reorders joins. See optimizer/README for
details." If you don't like my changes, please feel free to drop
those.

In the code block following this comment, I have used shorter variable
names instead of accurate but long ones. E.g. outer_expr should have
been outer_partexpr and outer_null_expr should have been
outer_nullable_partexpr. Please feel free to change those if you don't
like them or let me know if you have any better ideas and I will
update the patch with those ideas.

+        * Join relation is partitioned using same partitioning scheme as the
+        * joining relations and has same bounds.

the same partitioning scheme

Done.

+        * An INNER join between two partitioned relations is partitioned by key
+        * expressions from both the relations. For tables A and B
partitioned by
+        * a and b respectively, (A INNER JOIN B ON A.a = B.b) is partitioned by
+        * both A.a and B.b.
+        *
+        * A SEMI/ANTI join only retains data from the outer side and is
+        * partitioned by the partition keys of the outer side.

I would write: An INNER join between two partitioned relations can be
regarded as partitioned by either key expression. For example, A
INNER JOIN B ON A.a = B.b can be regarded as partitioned on A.a or on
B.b; they are equivalent. For a SEMI or ANTI join, the result can
only be regarded as being partitioned in the same manner as the outer
side, since the inner columns are not retained.

Done.

+        * An OUTER join like (A LEFT JOIN B ON A.a = B.b) may produce rows with
+        * B.b NULL. These rows may not fit the partitioning
conditions imposed on
+        * B.b. Hence, strictly speaking, the join is not partitioned by B.b.

Good.

+        * Strictly speaking, partition keys of an OUTER join should include
+        * partition key expressions from the OUTER side only. Consider a join

I would join this with the previous sentence instead of repeating
strictly speaking: ...and thus the partition keys should include
partition key expressions from the OUTER side only. After that
sentence, I'd skip a lot of the intermediate words here and continue
this way: However, because all commonly-used comparison operators are
strict, the presence of nulls on the outer side doesn't cause any
problem; they can't match anything at future join levels anyway.
Therefore, we track two sets of expressions: those that authentically
partition the relation (partexprs) and those that partition the
relation with the exception that extra nulls may be present
(nullable_partexprs). When the comparison operator is strict, the
latter is just as good as the former.

Then, I think you can omit the rest of what you have; it should be
clear enough what's going on for the full and right cases given that
explanation.

I liked this version. Changed the comments as per your suggestions.

+ * being joined. partexprs and nullable_partexprs are arrays
containing part_scheme->partnatts

Long line, needs reflowing.

Done. Also fixed a grammatical mistake: contains -> contain in the
last line of that paragraph.

I don't think this is too far from being committable. You've done
some nice work here!

Thanks a lot for your detailed reviews and guidance. I will post the
updated patchset with my next reply.

--
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

#283Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#277)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Thu, Oct 5, 2017 at 12:24 AM, Robert Haas <robertmhaas@gmail.com> wrote:

We need to reparameterize any path which contains further paths and/or
contains expressions that point to the parent relation. For a given
path we need to reparameterize any paths that it contains and
translate any expressions that are specific to that path. Expressions
common across the paths are translated after the switch case. I have
added this rule to the comment just above the switch case
/*
* Copy of the given path. Reparameterize any paths referenced by the given
* path. Replace parent Vars in path specific expressions by corresponding
* child Vars.
*/
Does that look fine or we want to add explanation for every node handled here.

No, I don't think we want something for every node, just a general
explanation at the top of the function. Maybe something like this:

Most fields from the original path can simply be flat-copied, but any
expressions must be adjusted to refer to the correct varnos, and any
paths must be recursively reparameterized. Other fields that refer to
specific relids also need adjustment.

Done.

I don't see much point in the T_SubqueryScanPath and T_ResultPath
cases in reparameterize_path_by_child(). It's just falling through to
the default case.

I added those cases separately to explain why we should not see those
cases in that switch case. I think that explanation is important
(esp. considering your comment above) and associating those comment
with "case" statement looks better. Are you suggesting that we should
add that explanation in default case?

Or leave the explanation out altogether.

Ok. Removed the explanation and the cases.

I wonder if reparameterize_path_by_child() ought to default to
returning NULL rather than throwing an error; the caller would then
have to be prepared for that and skip building the path. But that
would be more like what reparameterize_path() does, and it would make
failure to include some relevant path type here a corner-case
performance bug rather than a correctness issue. It seems like
someone adding a new path type could quite easily fail to realize that
it might need to be added here, or might be unsure whether it's
necessary to add it here.

I am OK with that. However reparameterize_path_by_child() and
reparameterize_paths_by_child() are callers of
reparameterize_path_by_child() so they will need to deal with NULL
return. I am fine with that too, but making sure that we are on the
same page. If we do that, we could simply assert that the switch case
doesn't see T_SubqueryScanPath and T_ResultPath.

Or do nothing at all about those cases.

I noticed today that the version of the patchset I have here says in
the header comments for reparameterize_path_by_child() that it returns
NULL if it can't reparameterize, but that's not what it actually does.
If you make this change, the existing comment will become correct.

The problem with the NULL return convention is that it's not very
convenient when this function is recursing. Maybe we should change
this function's signature to be bool
reparameterize_path_by_child(PlannerInfo *root, RelOptInfo *child_rel,
Path **path); then you could do, e.g. if
(!reparameterize_path_by_child(root, child_rel, &bhpath->bitmapqual))
return;

But I don't really like that approach; it's still quite long-winded.
Instead, I suggest Stupid Macro Tricks:

#define ADJUST_CHILD_ATTRS(val) \
val = (List *) adjust_appendrel_attrs_multilevel((Node *) val,
child_rel->relids, child_rel->top_parent_relids);

It so happens that every node we subject to
adjust_appendrel_attrs_multilevel is List, so this is ok. In case we
need to adjust some other type of node in future, we will pass node
type too. For now, I have used the macro with (List *) hardcoded
there. Do we write the whole macro on the same line even if it
overflows? I see that being done for CONSIDER_PATH_STARTUP_COST
defined in the same file and you also seem to suggest the same. But
macros at other places are indented. For now, I have indented the
macro on multiple lines.

#define REPARAMETERIZE_CHILD_PATH(val) \
val = reparameterize_path_by_child(root, val, child_rel); \
if (val == NULL) \
return NULL;

#define REPARAMETERIZE_CHILD_PATH_LIST(val) \
if (val != NIL) \
{ \
val = reparameterize_pathlist_by_child(root, val, child_rel); \
if (val == NIL) \
return NULL; \
}

I added do { } while (0) around these code blocks like other places.
Please feel free to remove it if you don't think that's not needed.

With that, a complicated case like T_NestPath becomes just:

JoinPath *jpath;

FLAT_COPY_PATH(jpath, path, NestPath);
REPARAMETERIZE_CHILD_PATH(jpath->outerjoinpath);
REPARAMETERIZE_CHILD_PATH(jpath->innerjoinpath);
ADJUST_CHILD_ATTRS(jpath->joinrestrictinfo);
new_path = (Path *) jpath;

Now, I admit that hiding stuff inside the macro definitions like that
is ugly. But I think it's still better than repeating boilerplate
code with finnicky internal bits lots of times.

I too do not like hiding stuff under macros since that make debugging
hard, but with these macros code looks really elegant. Thanks for the
suggestion.

Also fixed some lines overflowing character limit.

Yes, I too am thinking about the same. The only reason I have EXPLAIN
output there is to check whether partition-wise join is being used or
not. The testcase is not interested in the actual shape. It doesn't
make sense to just test the output if partition-wise join is not used.
May be a function examining the plan tree would help. The function
will have to handle Result/Sort nodes on top and make sure that Append
has join children. Do you have any other idea to check the shape of
the plan tree without the details? Any EXPLAIN switch, existing
functions etc.?

No, not really. We may just need to be prepared to fix whatever breaks.

Sure.

Instead of "multi-leveled partitions" it might read better to say
"multiple levels of partitioning".

Done.

Here's updated set of patches, rebased on top of the latest head.
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company

Attachments:

pg_dp_join_patches_v36.tar.gzapplication/x-gzip; name=pg_dp_join_patches_v36.tar.gzDownload
�&k�Y�\�s������_��;��l�%K����I����1�s��u:��$6�dl]����� E�J����S=H`��]�~vz5w���kD��M��T�G�����1~N�}���
F��x2������1�x$�;'5�L�n"�#W-�4V�m�������?����M� 
��s(���`�
�RF��7��~�1P���I�����QE���������wO_����]/�7�L���l�����=���dr:yr"����t4�N�|�&���\��������b:l �'�R�H�p�8�3#���#�U��O�^��h�rS�D|�m$�1����q���D���Q�q�M�^�D������k1���"�*�@E�@�ksA�j���[D��L�Y��� ���.*X��������;
��
q������J�3�.e��HL�f���W)��O��>\]�zs|�J�*U�e@��
dbE�x��+�2�kN���B��
S�F
��u�L��[�a(��_��G��K��4���]5l���$#a)7T���l�;�H��">�z���D��=7��B6�}�x����I�B��@�0=k>i,��J����'�,|�
���.���h��<��{���D/}���\�$3�C|��%0����u�l�M����N��@�=����2�yq4�]�.j_���a_�Z
�����IF~�j[:Nvt���1�9��v=�����`&������n�o0:��������t�g�?Px`\�3~��B��b�D��G�Q�p���X"W�v
:.I�|l�m���A�[��'q�������3�O��K*&A]GC��e�	s����=G����o����2P^o�y6����
���@�sX���������SQ|ya��r��m^L�����4��k1u4�m]7�,��kQ��dw?U3 ���g����o�����ML?�� I�����~O���@��n��v����	��,�8������
�
<��
*Q��Q[L��B�:G���f�����{[}�t��F��^x��l�v���`��#�;9A��n�a�o���d4l�l�������@�����]����w�"��E�-���g=���h��khq>���R	�\�p.B�H�p�����e��a�ud�T;�M}(�9���r3'o�`3�����x�X^�0L�b{������t���8�7��QN�Ed�%�P�����5��`�d	G�\��z�N>Xos4`����u�P,~�+b�V"�@�w���@��\#�<V�����.��-o�P�FS��I�hO�z���J&�'��!
R9d^��N�h��P��%	��K=a UA0����8A��:�|&'�A��~�>��j��
bKHm�wB�R�����@{ ��p�Hd����P2%XK� W��
������al��y��	�X���_��(�+2?K	�������&�}9s�0�%.0���lf�-�W� {=�j�s���]�Y��
&Q���������[����j�X����wb�����QO����<�v����GeG��;��M�a���6�4������U�������b���:a	Q�B��E�?�9��[��J[s���	90���X�Y�	F?�cL$0���Ofw'�t��2�f`eKp	6'M�I�"��R���NO��#0D��+r�IBV����fjM��+���mCHw!Mi5Z5,�S�j�.��ikfj|��<�IZ���r����t2H��H�	:�sD�'?g����D������]���1�,H���V�3fb9w�����C�F�����<�C'�+L�#iQ1��}_�<z1�\���:�A5�1N��L7�sy�o��Cd�Z( l����	n\��MU�/=!r�w�W�-n��:�Lq��rB$�v�C�Ub�9Ax��5P(�m��F��fHl���Ek�$�R]t0M��	y"��������e_��x\5a��	e]���r����8�e�}H�)r��,n���#@���N��*n��E���+���Yy�
�aT
N��%HAd�n�^G�X����n?�D"���X�%�����)�W�U�I��Y"��5&��\T8�Cp)��#�H�^��H)���j���n��
��H��}D13��h����d�`���+�$U	�j"
���(�@r��d���,���c�v�O�R!]��1b��.{U-�FD������������]m���t���������pr��6H�
�>''���a*�:�����{6/��q�i[���uL�����Oq�;������*]����a�V�/�,_X����c�r%����G}�ES���o���z	�)E���S5�\�s	�C���!"M���)B��X�R��#�����e!��;@���|N��'�1�jQq��+��a��g��{�=�����V��.���E�:�����d�v������w`�V7����4������	%�\�`F��&����ni�(���K�E������7`B�/�?�~|���yw����������k����B�:�����e�&o��T&n�F1�L�,�>�1F�H�w������;9~��iL1d���o�a�P��L"W�T M2[y����@�P�
���!c�4K%�Lr�<e��o��@h�R��x1��n��l�?�?����w�ky���f��2���<�B�+�CE�x�DI2��7y�c�&��������=|�������[-���5����s��S!�-���[�z���D�c,�_n��/i)�[��xRH������M3�E�6�/���^,s�H�~P��d��m.�����������8��z�Y;Q�p�L���m���� :�\�yJ���j���� ��5C�;�i.S�"��:�fHl�a�,j��A�p� �������`
0���^�h��37T��mM��H
�h`,R�,���`v��-}��/2isu�w����q�w0 ���x����U��z����6��u��,�M�m�[�w�2��',����d���e�XiJ�����Hf�h��%ri���Eg�I��T��*���gQH���1"Z������?Gj��.p��C��p2,uM�c�~�&F����u(��������U��2�^��_r����.��%rC*�H,M��e]p�T�6X�D�Ni:
H��}������A~�KHr�K���_����������H�`D��a�����t�N�P��������q�=d@���&>�9��uv��F���������"P�3-B�4�� �Um�CfdWyG�����4����`�:v��~2&�+���,]��Q����` `8s�1���4d\�&�{���oi&����-�0
pg�f���`"�"f/g��S.�F�&x� �
���F�fY�b	:�"�S.$�i���vU��a��[w�+2gikB^�F�����yk����n�����iT�/�q�L(�f��Z{��X��@�����
��2��M;@�p$@8����tm�l����z��+O39��N�igt8��F��t�r��r��-�f5v��2��(~h<���/��k(�D�Y�
A��X1��W�VIa3jzpPRB)���VB�p�`��bJ� ?s�A%*���JS���d��\c��Z�F��PqV	��*�jc��XB���������b����%��$"_P�������������0�=�a�h�0��!��q2�����G���F)
��K���_�iS�Q��.X�kpHX�D�QR�������?�o��]��S�#J��9xc�@Q;���@�P-����:���T�����	��O�� ��u�0����7�z�#�������K��I���X)$���34��P�����	VNr�@��U� �d�|"9AG��*e��R1�W�{�\�d�z~�L0��������t#!�{,���f���vD�B����wQ�,�T0�{���k�+��6	w��q��I���n��e-�;D�&WV�t��Z[���Cf�h�(�1�� �M�b��s�������$���r�.{��|��SQ�|&��T\��RV{�q��Q-g����I��1<�X����_���!.����U!
t)d����vw��B>�eLu�XO���
�\E�e!!�z�X���M0��L�{{���?�>8��^|�{�r��H���������fL��C����!P�M��
7--�rlA��M}�x,:UK�l�X(7����c[f_\����X��H�J,A��w,B1�C����(saB�r�����A�Q�G���,���q�

�;�;_
c`�_q����wn�.W[gXw��v�\=;����y����pz���]��H=X���R�dx��?N�E��<��}#~|^�H�6n������X���������R�g��p0mI9)n(�������mo%Y�wJ�h��t2:������������E�AYmIG�1�>�WUZD:��Aj����\���K[���nK:��C���je�{)��P����|�����\l�H�Jt����Jz`O����t�H#����Bo�:����c$���4V^������NG��j	�@�Lej���B���R���eJ�sJY�2���C��$d�z*���6�Re|���s�Q-����.�j�;����J�K��]L;,���t���\�\�?����z��nHp�jW����'��Ufk���h\>�S�C9w�����H�����#���RME�K�r�� �O��'�bD��T�����M^*�0��.�C��]5+���+���\cO0][@�r$��O�~���;�r�}��������-c���W:r��8K<i0+R#���I�w�|v@��g�+,�ehs���2��)f�>�e���V�� Y/�m�Z�ZJ1��}��*�,J��|�(�O��O������4�u
���_�����~U����<E��9�*�p4�w���it�~U�������-�np~}�����
�%X�7����y���8�:Wq�q}����{��x��CH�~���M�r�<l�}v��xk��Y�
�����m���9%��y!��/��|3Tkdo,K�O�#w��b�"+v�kG�,�"u���y{0��AXs����
i�7��B]i���>����d(1��O�	���m����]J�lM��Q�[Ctv�7y3����L�|�v�H�5���W����/�]��U��m��B��
9����
6{N���,o�~��n���mmaVCC��i�����v��H�oy�J�S����+�,��f�>�9�����x��SxZv��&;�'#}.�����KL�+C�{�L%�=�}��v����0V������j�?���/nc�R<M@��1<���j:��T$)���MzsE_���sL�!�9���0p�@�G�=�������Hc�6�~��o�s�v,�3UD3C�t��^�Q|������Az��!W��q������������"N��	����=����gm�T0���M���hz/�-)e���T������'1�Z�i�]���B�l�9��"�5|��v��E��~�������y?��<W����)���`���A��~����������.f1?���r���_T8��
�f���68�
����AWVw�h���;VR�|�8�R���K���Q��]^(�O����'*�j��o�V�����s�������(��LJu��dr�g`��6�����+�*�82��RY;
@�}��c�+�u#h�F��
������z��#jo����\�*��m��X�R��},m����p{}{����+�p1'���v��d�6M�V��F��� *����n�����Z�H���3e�,�������
����e��6V%��4�����?�A2t��~k~L.���O�n���$�qd,7z����s�{W������ �UJ��T#����7ooI����)�����(�H�x�G�q�c~�f�M�}��$q�"5<�q���oW����P�M��!�>���.Z�4���5j���g@�/KiT�[.�OG�4� ��'��6Y����N�F)f+]��n.�,
T��mP��Oc7�}�|Wj��S�o�g�dQ��h|C�3V-�?�:����VZ��A�O��A���4[����Kdo���}O~>�08~�~9}��Y��=�G�B�[���3�X\!��G�E����cdZ��?��A���T�<����?�
.>e���J��>�r�i!$�V/I����������������;y���XY[�������j|n�K�c���S�������8��s2�����s8�90���q�3�@�m�o���E#rOg<���1�9�I��d���R�����O�	�B_:-P�.{�'����U�h��.�*z�2��'��N����I��fn�h���H��Z��a.���~�#��V*��	5��F����l�����l���P�1%C�������z<k�BL�#�|�7L���yR�A�m�/i�<c�<��q7�3��	~�8���Qc��@���0��)?�TK���F�����X��#w�����q�<�����f��jh�EL��s����/Q�E����v5*?$����}l|�����aS��!�!�@���a=��3\�c�$gm�U�.�f�Cx��Y������=WZ����
�6�l�l��=����iC������b�gg).�������
��LH��E=1�X1�����iV�[���Q:���\@cDR��R��
sE��R����9z�1�P��64�����V$�|������U�v�B~�r{����)'*D�Yd��,.���|�>����l��/�I8����bI�c��$*#�ln���CO�m[���g�)^b��._��%c��.b���+���(	��[; ��� ��<x��P�yv���thQQ	��O~`a"T&���������ox�R�������'�G0�����5����|��,*�1&�G����t�2v��A%�������Pr�K��sqp*{�3v��/�u�����-�������/���C�����~?��W-����|���aH{�n@'�O�3x��?��i|�[W�K:
|���#=�/,H�@���s��_7�~��3+k	,��9�������XM�S�o�^?s9�r7/���K�G���?�fHi�j
���������J0;[h�6��H����](����!�;�m?W23��w�n��50����&�����R:�B�����W
x=���.gs�wA�*�U���h>#�%[�`�H�3���^>H�8:� �6���l>v���Lg�)�[&��hA���6!�SVNq2���M65[0�c\�A�]��34�R�S�I,��D?~�$+���/��e�5�P�������e�=�87�G��8���p�Q�=T1��d��p
����LqG7C=1�D���C0������8^�2WM$a����H(V��>��x{���z+1�SEgjA����,#nJ��Act��-���y���j2�j����t���g��,����$u�!C��/�Z1�AP����#c���b��~&����G�� ��0�f����+�%9�+��`�j��A�4�-��6S���8=U�t�#���K��}?w�S[Ckg+��s��ML�b)�,T�P�T8�z�Y��������S6.�v5��[&�1��A~K��x���&jFK�
&8�=G[�b.�y�i�!�eew��z�5[;�
�L{%UuK�MHpK�e�-���?���]mA:w=_�)U�
	KNP�\��:2�^/�[bHP��8��i���P�(-+��#J���@{�P�I<Q /��(���e�N���b��x�������;0��+&����-��4.���FM������E�tT�*���#3?���9�a������n�fZk���0K�!�h�S�;�[���DC70�l5��$�<�I�����]]s���l������v�����)�f�FAf��d�QB�r�;:���W��\U���fa�j�0@����e�B�rk*��8�	����%����"!���Y��.��A����D�RXH ��-�X���-�2~8����p#4�
��<���!��v�Hvl�2�/���	!�nW���$K"�F�K��;c�,y��P���J����4��dy}3�O���nkG�b���B����`��m��F�����(K�Td�A ���4��n��iz����L1�����<��854Ou��7�>3��F7��O�(�O���CT.�qd0��A\6��@.�6Vf��(��^e�����r~��b�cs�c�`���F����}�p�$��?a&�S@3a���������N����S*�:���Yf���E3
�M)3)��@.���(����c�j��fa#�$=�f�(����}�����scB�����_~<>;aLD!U`5(�t=k��?@k1\�������sH�,Z������;TN.CkW������m�����������}�n��5����t�N���Z�*���k�wb�D����n���f'�0��l��t����4te�&��g�2,v�9�gP
�[�l�Q�J���[����X;�i��_���������-������
fwi�
����:w��}@X�>���5��Vy�/��n�zY�I���lEs���h�!���U�&EY�����v`��{�j�C�j|���ml���x0�Z�xd��a��b��]�%���Y~F�s�'���Q�i��=���#o�8Dz"�@n��S�q2�OD���
�����2��������{{I�K�q����TV�[Z��My�N��:W�V����L�e(u/�~�15k�:��z]����+�B��$����Trb�@9YYd�2��"(^�	��pR����AC��������vp����&2J`(9L�� �#j*s5�Z��F��^��/aq�h�V��e���{��^���������u?=[����z���T����?/X�`k�k�E��O�����J����!�W*�qp��b8�v�l�<��f���c�k�u��09���� b4�np��B6��~hp(�u�2Qci\���(|��&F���f�(A�$bpX�.���T�Ybe;�9��F�(�O���l�����H�F��t
�id�1�,D��TN*b�Y��\Fe��A�����hk����f�2�,Ws�bL���Js�FK�A����y
$vI+P2���4�#
�.B�J�@w!����K���H�XG&��>���W��^E@�k��&F������
� ie�����n�$�&Yq]���6:W%�V%"����)��H$Z���MG��a!hF���Ai�*�!�XQ���/�����R����w��,��E>���L�B�0q,���j�%T0P�p82-%��������T	hB��	����E�Z�2-�\��u�bB�����"�6�k�y����-RJ�4��ac��$�q�/�b^y�-F`6�����){�;L@4�|�XTY�Ay*�	�(Y��D�>�k�^a�^4�ru�iA����>�~.��8�G��6��'��Lp%���,+����A!Z��,w#=xa��U����Rsq�T��#�I���f��u�RD������K�NN95�P��Y��%V�y`��a��]��)�E�4@:��<������v5�h4L��3C���
l3F5�/��-�5�}���^J@���\0mvf�s�.X+��7`�=fN��0V������P��nm�gD`h��.��GF��`q^���S�TK�������E<B�f�]�h�y��=�)��#1��m23^������.�%]�����T�%d3�:�0�2T3����U���j?iw1OC����Jk��
$����
�mW����t'�����n���]�j��n^��TH�J+�!R�S���
��nn�8~�g����;*/��u�\P�����l����q��.�#��m[���v{)����r�-5B����/IK�D�\	I+�1��sp��{{��������mTTJ�����n=]����|�\�]}��b0E���,^�%da�X��$�@�W�
��iA�G����W�����c��Y^U��Y���N�;}��L�D'�K��G��p|�eY�+��:�+%�g�5�%�'H49�B�`��0
����-LY�;�����c��n���^S�����-�mC@m�"y ���qk$7Fd���J���_��b���d�(i��MR���Hz�	�P�|���;<��,�1)T�o��Bn����K�a����vK�t_���I����������#I����O�7�������>*���G*@�\>������&���e�C�(����)��,�8gc2�+��%|��]���������&��-�Z�S�#>�s\��eFp����R�+s�`���Y��Tu6�p
8��r�)��9�R��1��������IM=gU�
�����&������i�eDG��[F�{,��1����d=�@���%�	�k�J$��L+f6+|�h���q���W�3�Z�N�Oj���>�A"{��U\J�?]A�T�(F�D�B���3���R*p��<����Ki�JJ1~�r����T���<d������#+��j1������[-��g��Tv!��I�dq�XL�<���vs���t��������<	B���R5���V`�����E�"D�Bw�?)�g�NI#g�4��i�q��:sE=e�N=7J�AiD�R�(Z�B��J�]����1�U�u5������Z����{��IM"T�pa�r)d�8)�^p 5@L���.�)%e�����
M0 ��f����Zj��;�P
�����MK��5)��W7c���i
>�����h�F!"
.gKVea��-/�S�A����X�K7���l���vN���`y	*�HC�2���Gx4#4EL�d!��v8;�]�U*��S�/X���=	}���5���L������2]��K�8a�2[�z�6Zn0����6P�a�a���'�@@A�6�	`���~L���G�,A�A����n�����_���I���}���5�)���B-bh�;�U�^��/�6g!��c��2Y��*�\�5��xY���)�������>VW��V��F��a��$�����Y�:��TK�)�?��2c�R��8Z��Bs-�����"	�U6����#g�d�#�����i��s_�
h,&�9w*oCQ7�� g�A%����L�N�@)�%B���g��Zh$���N�Sd���rK����#!�7S88�j���qZ���:����6b����s�\w�dBy����r.�72��
�S��"����!v����tP��f�aA�f��"b��^X�c���&�r�_�'*�D$,�j�1��R�y�^�TO:�i��\��By�s�+��`�#��F��s)��H�O<���8Z�")���)RvX��!=���3H��Rq
���.��cj�+u�LX����`�V�T�c�Uhc�Z!�u����5`(�!��zu-�t���3��^�}���d�~�(�9�w����k+�������<%8�S���z���eL
>��B�I�� M\5�d�Q�Vy;��w�4�6�m[o3rN1�����	��3��?�s�0v�����C=L�|�`��sx�k���l��j��,[#$F�Q k��Am�l`p���6�O�l�������L`������4{���i��@��1C�2�K�*��e1�!G��c bL�@�9��p!�y�����/���������t�	0<�ia����O���v��E��J����/|�y��*���v������� >��R.�-�&Z�Q�����������>���Q���k$V�G0['p����:7f��R�m>�����
*%_�~/,3��<����	
���]�|R� �e�6���Z�������y�[�m��������keEJ���%�Q@A�),Fe���7�'`n.��C�4�`��<A�AH�k�D'�T�Q�:�0�k��������hH��B�>v|B:�4fia�����+���&C�At%\_��s'���=W>)[�G
��Y�l�<�'
L�xA�{D���(���A��Nf���������O��X	L���ui�r+>����O���Z�t���g�%l3��w��?e�A����l>��px\��pG"��
�Y�1: B@-�m!�+�?sH�����1��"�3%�������QE��?���/hr_<��/Y����l,v��	�������&�28�`JO�3�M��B���f�d����ED�N��3n@�U�l/:�HO6-#bu��f[���l�O6,�"���=by��P"�
���\����"���
�q)�KD���� ��?b�T��S�g�
:�D�M[�:����Kj�q	%w�2�C�Bd����-�������&G��������j��^�$oj5gjN*�}Eh��r��j�Lf�����T�����P�����Ffk�&(�a�5�z>)Z��P��&TL��BX(K���Wy�� _�����=��|����{+VV����L����9g��*
��v2��e���p���Z��l*��F�}J�v9��=�R=V�����g&�#%1�_F�+��D��j���>�-��k��q��L��+	������l�":�� �5��	�h����T�[z\yt� �!Y���0.��*d4�0����������x	��	���uZ�k�X�����T��#j�1A��[m:��z"���#x�7*�$40�E�?�v�G1|e[$d�������Mj:����&��P,H:Do�i:�wM�����#L�0�~��,�F�6`*�_�D�RyBJ�*:(��L�8�o�g��H�`�r<`��5]�g��1���g���(��9�����>��T��2�n:�$T���7M5�H�O���2�9R?`^K��6�l:$�a�y.V��H�4���={��]bL-;�2���p��Lp�f�7��J�7��J����n%,hX-�(��Q��z�<���X�	��k�I�s���Q�������
4.x��_
��)�.>V9�+[��2I��;�1��[>���o�W�#M(�x`��4�X��"�9�`��JG(:��J�`R��aN�����a0�,��u*���hE��;X���
�E�:r�N�����O�������,Pi�yv�������zLE�9M���S��E;:fM�����8���7�4��C��*�_��s�g����t��r�N��������f�[�����h4f���XYh]�k$F8'BS��	1�h6��bK����'u�g4=��<���.�1[����Wg����<���X��
��5�5�;
i��3��)��2@S��[��l/�0,�G�
����m�i�'�������jaB+�+#g���a�`f���@�F�5%����>$]�NM;�]_C�]�?!I:/@[�;0o-�l��T�
{y!�G)�69�E"���,:��%B�����ba��(��!��G�<1C����&6/O#���3�������4Nx8L���RW8�.��SH���t��L�gg���>!!�$�,����bq�[���d����2���dBm9��0�������������M0�����j|b��-!�i�+<QD�+�9����K�����\Hs=�.7a��d0�ctcF��VWB�?�_���>u���:�@��
�����e�,v��,���$�{m	���P[x�<���2w�=�-m���t.s���l3��Z��0�2F�+E��+5dg��zf<�,�*������m8n*��N��@c&���No�}�	~E��g!��l��1����g)!�����y6�]N������&��.��2�x��<�C,���b�}N���0������n�g����(��o�f����Q|~���K�8������
�^��" �?����4{#F��5�_0'�K��������<�2�J�$E�i��A���g����-{%k��O/�~89�pr�����'��������oO�~~�#��l	�g��lv���	VhM����m����y�/�>�{47��rc��*"�>��?]f��
�����{��	B�./	��df|o�\�{���y������������8j,�"�7���F�?:��i��/�����rU>�������\��'��A���@�r@�����������}n!E�me�
����5���a����{{Y��?�.����@���Lbr�{n�.�M@j5��i(M~L�i��x��E(���	-
�_B����%��	�.�41����P�H�L~���7�v����F��f����W(��x4{�@a���r��KR����1����{�?�)�b���!g����&:E���sf���G���XCI���(D>��Y<����iz=��7���<���������j���
�2e��F�?�nn4����U*�h�ldn ��|<D��r��S\l%k�b�����
�!-_^��AD���*�G/�x[S�Bw��>\g�l-4�����.�XX�>����'�v����@�v�wL���0Q�n�n���/#E���x���\g�+>��1�]CO����m������%1(����2��m/P~�$��.�2�r��*��#�#�!�`�T���>?OM�-cJ!�pq<�h��dbM,��2N;i�7�$�����=&K�y���@�6���C�3Y�(iM"����T�;L����	�`��4e�/=����2;"wz~����p��6Et���\�3����L�m��J�0��`�\������
�M��*	����NOje��l�68qD�2�:�Y����&~EKJ���A"x�W��@>�k9���CE�-�{�p�&��{&�~@��+.P`��|� �;{4�K��rq��1���	�y��-�0[ �	u�{���A4��;^���_~�e��O�$nEWzBM��}�����x%�0d_a,i��m ����M�ML�	��4>jhs�������]�?������Zx(�1P�j�(���8#��b�<*��6jn�M��P�l~aw�5���Y[�)7�I�M��a���}�����2�'�L���R��p��L�Lb�E62�eA3�v���y��S=B�$yd��G4B#�*4Fh����D��� �����b��{����J�$$����U$P�kY"@n��%���t���D=�K'�UA	0��c#��u��a�]��k���T1Sf�evTy+B��G��P5��Yv-lN5��&`u�+F��d0��x@~T�B9n��lc��"��
��T���#���T��&M��cT~]���,���p���#��+�����"�����0��U�U��O�����!��R�1���H`B�K6��R5<��y	YyT:7:�%fNC�a�f����"������~[�:+��KrEO`��J4e,���fg�#pp��S�����,�����	�\��6�
RJ�*�o�X
XW�&0����3���2J�g��X���
P�h�B��'p�m��S��o?�Y2�9��[�,9�/��F�����%wra%E���b��T�������m9��a��e�x	��,�E-���]�����e;��a�.�A��3���%r�od���bF���b������l~�T8>����O?��E	<.��5���lo/���,��D�5�A���a���C��CnR��"\��f���y���mL5�hD�}��F�eA�c���p��Nil�EPzci�9�j*���P��79���<#���g��������~���b���~�I�y6\]Bh"���1,�~B�U�n�!��h���G�v B���h�%�
usM������!w�|�Zn�>:������U���M�fa.�u:�mp����2&�m���(�|pb�G�
n�Z�4u��{#�����voE�����w���m�K�G^�^��p�k%�$:���P#&p].��1�4HU�P��e��{���{TV�����x:�{���)�GY:<�����x�tx������������������a_��f�u����������|��h��F� ��{�j�Yb6�4|S����:3��+J�M
���$hA�]	������9h4�[#�0�Hq�{�R�/@X�w�.Q��-��'�8�wI0��yZ��S���t�y�q�}Z��la�Y�#�����aB����������J��Ga��:�-�Y� y��V�e��v�=�����7*��� ��;�sD�!<c*j��c���~L���fNY�����9*(�L������+4`�|JP��i
2�����������~L[�����v#~�k~}�m���� �����ZA`���3��w������m���,�@@��MZ����8_m�Sp��wS2���������ken�w�lD}���hq���)��B�"�8V�\��R)����VB_6W	�x��:K�5J��i-�-������6K�/h��c�[%:$�J����hu�-��L�3�%��:�������,Ty���,"Dt���Z�g�lw6����CO`C�1;�%ml%0�:�r�(����y|4mu�����[[��y�����mt�-�v:���/j��]`�v	�-���o��"v��{p��;���;��?t��~��������^��C�	k������Q�d�����she���
N�����h��,�p?MG��(>8����p����xxp���������D����y���b��[P�����Q�CJ�$�����~�[��	����{����[?�`��4�q'zK"&�A��g���^�t�Ig��j�7r�>�������?G����4�0�6������P�HDQ�l�|��M#o�Y���������-f�E�c`K�����wI��n0=�"-g�(��@�@H��k�5�1c�2�:����b1����T~bpd�x�������]*�[6o������g~o��u�yk���y%cGm���Q�vvn�A#c��D��{K�?���45�^1��r�#?m��k������m��-w�YBJ�q����B���s��������=���[r��N�������4�_KH,
&0�������?C���i��fZRj����@,(��e�&���1�a1e"�C~0����"����<[" �(���"�Z["�n�&���i@)f��3d�������U!Cj32Z���
JxZR]?���S�����B���@�&���N��t4�^��������2X<|���Y��_�0��W�E�h��5�$�����>�|���h�eD�h�I8;�[��X'(+���]8��&}��{�TFw�����iN���t��-���4 �E�9�� t��kr�����U�Zi*s%�O��u[s�
r�Y��+
r�;f1�OR����M��uH?�!����*Jumyn�� ��aTA��iz1N��p|��b<��{/S���z���`0�-��j'����F��/.�'O	�R����o�R��,��!��5_$t��q�������v]����\���%����y���@������7��+���~d��V� �"B�_`��,oU���1��'{\PFW� �wp01��A���&�UO�������<�+1,|�D:i�(�$��^��d����PX�A�K|�i&�����)H�����rd���?,�������~������!o(�����h|��O��]�]��6D�e���)�A.��{���/�h������wO�7`+�U��|���A�AH���= 
��6>��?|�P�6Y!�VA�m]�`�r�p.n'�fv���I�����,
)�P�eCV�&��ex�ZJ�4'd|Eb�{����~�BceV��W�~Z���=w��
�M`A"!����6�4��"�(m���l�y��5�~S���;jt����2�b�_Z�8����bD�z�����3�0�:�FG��}����(�����T�����n�Qw��������o���|W����7d�yd26��+��Q������Q���7:<O�6�������eQ�������#�K{�$���R[��%X�P��� ���&(�9g��TE�6�P�"Q����7����wJLaH��[�B�90c�o��V(����g�T@�������n��w��q��L����P-�p[7��
4�$�������`LZ��+���-�3�v�,�>��ye�UT�K��0�Z�tn�;��K�����Gn	"����t+����T����6��{�u�p���\�	�i��n��
�:��������MJH�O���F�^$��G�[?,�p�2�������,,S�Zih�K���%y��$�eKq���>V*`����)����2@L�v[������������H��d�����NU���T�ae���0����`�i�#��3�H���Pi6��az(:�b�$��-L�P;[J���] S�x)���Y�*��*�*i�����<��0�i�-[��a*��ie�5���[b���Dp������0_�0�"5_N����'-��"��-PVwM�C�~���hf��4��~;�0CX��t0�m+�ijG|*?�z�o�a�I� ��P�0%,��AW$y`Q�8}�/�*�����t2�Z��
�^V��8���E����r�����d�1p��'5��i�
��E�	TQ�R��%�M^��X�c���eVXb���9Y��NN"����
��v$9VM�@���h1���J�
�!���M������/~����`n�"?�3����h\{VbVb���t���4�<����U��!KDV���TJ�z��!--��l��@�����U%���*��*V	����'0t�r
�����'��N��<a-J.m3�Nf��c/���*�.��~������v����vxP�r�8e����P��({����`�S��S@��<��F;zm�Xi�Z�@��{��f#�PJ6D"�p3��N�l�2����f'��)��&D���������H��L��_�������������QI9������u�������fN��������s�4��R����C��I^t��_�WOq���p=|70��[��������3� �d���.Pq�x�)�4�o���G.R������j0�$�(����~�`4��;:���^�S��"��Y5A�'n�����?,2�� *Xz~=��]�I}N�n����t���;�[U+�}��zN ���}A/�����n0��N��v�������e�Y�N���c:7�]`��
3����Q�[�m�;Q���OY�����~
yw�ZF��J�.u�R��@�~�����{�����������BL�IugSe�c�7����H���>�`5��V��B���o\���zC5yQQl/�<KI/9h'}d,c��)��X����JE�l�*����6�+�����D�M���|����I���T�qB���j�c'y6Eo��f��D�o~����WHL- Rw�����f��!���p(E�l������4S��l�"����@� �U�"���t~����@�a�U>�~%��zsA���kpn��"��5
m��%���a�O`��I�X�+���� �Y2�hu�6�t�8A�C�a�q�cBSp��b�2�0f1�I]<�M�	S����x}a��H���iH]�*����������n���B��d�ZspI��9��j���������=x�s����������(/�M�@�c-Q�MR���n>�DaB��;�!����������~ ������gg���,o����I%�=��$��w�������b�8�[T��jT �}rvr���O��p������g���q�G^T5W6*���-���th�U�
O�N�<Z,iio^�JFt��:.������vi�G�[����U"~��Z���N�P:����[���iH��v�e�S�9�����
a��.��z������Y&�V���(�W��'g�d�y�%#�WN��R3�+���^L3�:l�[?�bv�W��M<�=�0��
�#��
`�iH�<�b�+������jXpV
���u���)�3���76V[_�0��5���<cA<KC�Q���0&2�}��[�������7�1���'������|�����n�G���^�'���I��������7d\i����������������p�Dw��~�}��
�6��~�;�Y��f#;��={�'KmDL����X!^t9>�.IJ��y<�&KyNS��w�(����_��7��ph�3�X��C�|1= f�](��X��P4�K�	T��IT�GO.dE���P� ���/D]��Qj�c�s���XK��'%mE������	�1�f��-6cO���?MF��*�R;s����lV��j�0��rW�r��/�����(.<�B:ph0��u<������+O<1�C��%�O��^S����\�ssId/��l���,|�CkmRyY����Y���2����"�yC�O�V��S�W%7�{����7�x-?���i�i�+i��SY�hM��<��-�5z�<�������V�R|%'�o���%��k���=��/�9�$<'����m��=��tq��>�0</$G5j�U�<Wyy����e�����il������`4�Y~�yp0�y�j��m0>T3�S�	m4�sZ�u����A{��:`h�^ccq5rK��Bj�w��}�
�.N�u:�}����q�dU4��:�U�0V�aX�+�"uh���!�<��S
,Dn��q���h�����W�C��^�K�a�]�P!4^^�����%����B������S�(��_��t�)�"��]�����`?4=���E��m�%N�e�0�rh�[����Y�������2����z@(��k��P/xj�i�%3�S�v�
wYW������W��0�0�R�%A�(��6LU\�������Q
����$��8��n�_~y��O�fFl��Fm�q��K:LJLf��W�"�R��ni��)L�c�IH���d<�
�e0���>�7\��,�RH=�.`a'��|��8��m4�Qk�O8J�6z���"�VM���<�9���L��6���?�d���f�F=�#�b�E�p
�o�{��(R>��~�����j�{��Z�Z�K&�Y���bV��E���1�k~%�� o����|IF����c,oS�Y���@�'OH^D�)k�����O�}:��o3$�&�b�e�	��@
W����$A�����r|a1�M��q0gqR�+/������3`�+��c�h
��r-�`,���Y�X�.�rE���Yei��Z�O���4�@�@>i��mB�Qx�{I�����;EOVb�)��!���%,���UM��	���AW�k�;V��-xP���G^f�|�d�S����+y���*]�&���-g*4VT�Nx��P
+?�&��bIZ����#`��-+m��5�3l���5�p��f�{{�^�KFYr�C����m!��7��a[O,'��{;*>���@��'�J�_�e���"Sk+N�SW�u�LU����)�>�/11�	���o���G&X�"OE
��XL�W��X<
��#��m��
��CZ�Os�x,�B�P_�}��$����'Bw���1���y���[�G�2�� ��'��VQP�(1[�A|~X�@��iRI<-Z}z��ku�5�����V�%�n�(8�d�j�:��^vp��7Jz����,��V�+��VSR����h���-U(�N5�=f������E�I��}��u;3�[��M�����z 6n�e�����z�������~���/}L�75��RzI�������k��.PG%�Y�0�Y]�K��-�1��k��nSV�*�����3�*����f�R���0/{��}���D������X!pa���\���w�����{��M���<=L/.���~�0>�'��A���(����a� K��5��=[e�� z�
Y��������������*�I=�(|��� ����7"h8�]�>���C��������q���R)�6[,!-����{4N"����d��H��%��,`��K�c�#��(��i��C��a����:�R���p���_�B��|�@}��;�e�:w~���I����"*�I7R�+B*BDE��)�4�~�a�s�a��<�
�����K�����X%%�?��Io����� ����],��.���������EFN��������_	�����F�?}����/��9$_�^���k��o�\+N�
r��,�;/OO��N�����D7�e7j�������6����H$P�~�Ktz����V�m�dpC�y��?�OW��,��������������'oN^�E�6��8����i����z�K��:}�V&�t�-Z�D�;"�lGc{�q��c(�;���!�zz���w|�K>9}�_��&E�?����m����H��j�������x���������4�x�=����
z6B�_�H���|�c�4f����{u���w�;����=��������������(��b��a`:�B`��i�*��L@�����qS`���il��10���ilSK���������*|���<��	V�a (yw5�@D�'���~ys��]�z����2�W���q����������e�7��������wd\��M�?��|����m��0���[(������|rz��E����E'z�#����n;n'�=b�������_ ��J�g<Z�E��a6_�����?����R�|�����3KS�29h�������#���l5��G{�� ���^Z���K�*������'.�OL��:����'`�H����;��]��2�/�\�����_4�G1����>y����/ytHf�'g������� v"�J2k/V�7��s�����[V��,�J����m�h4	!����>��Q5L�������sdR�M�}��WV*��)�A�_���������V���9�] �P�{�����?��m_:o�	 ;S�|��M�]�X�O�m�6P�L���y�a�m���	�[�C���e��� ��un�������7��u{�C���� vC���p���f����\m���mx�=��?��AKh������`����Q����\*�CfQ���H��Lq�N����S�B�����gX��+����%���7�}�$Axj�Jj�4�J\�T���NO�E�=�V}�Ck+��:_���������/�����8�Z[�$�Z���{ �{s��X��#Y����'(>y�l�]���
�k"��QH�o��BL�o�^P�U���_��B)����}��f[_�{�����-
��o�X�:�(�^{{q����<=�>>��X"{7\M��!,MM�������i7 �Ggy�A����������5w#~��w3xc�����/^�j�=M��@H��B'��)-�[M�Vb���������C�FG=�O=�b����B�!VR�@��tD�?�	��������x�_���� ��$��6�Z�X-��%j����c�y�M�[��Rb�^K~�hN{=y���cB�	�``0�	�`�C�r�C�f�\������\B�)��3��8H��=��_F)�S���R#V))�	����|�B�bq.1������y9��?�$5�n;W�����BZe����E�^���R�J�_�'�x"��e0e��#0Z�;%�� ��J�����)_�A�?�F���&�v�H[q�.�_��M����
�aX�n��LC��>� �&�,��7���P�]���8���H�K1���0���UW���bE�u�>����aL��S��c�q�����bOL���p+U���rc�zsQ�m����sk���]�����9q`��s���s��c�q�Z17�`���D;9�YB�v}�hsj~��v�Q��'�P��OZ�N���5��|,���|�OZ&���A������,���{��-�}��>����}Z���luc�8�#����6!����g����I�1B/_���+�����"�����cK��HE�x��l�I6�A�)<�B�"-g��H������pf�uDV�I)h4��UnS~��z�*��]�h��gN��{�v��v���8�7�f�I�7I��>�����#�b5��9��I:� 
(y�1���&k��d�7W�\A�KKW
����6,��_���uZdX�����-�M���p���p'�/�������j�x��M�Coo�����,u�pAT�y�y�!�J�����m��V������%��s�K��1fqg�9��9����9��s��a�eN��G��4Q�4	���:�I��&�9���s��V�_�Vt�<W�sy�3��>=�))G�r��tf�����f��fJd���O��D����[������F��py�n���{����l���q^����=���-��������v�$K�P��k'���t���>�M�k����84,8����P�>���;uQ����"�����K�aN~���O����u&��H'��77%�cN(��Qs�����R]�T���{�	�'&
�>�R����j�4�ec�,�8���nACB�'zy�����n��[/�����V���y�h����r@� ����I�id��������r��2�����,E�uf�2On9O�g���d6,�����&!3X�7r8������}C������`7�d�-��d��^�Cv=��R�>�5!
����4�7�;!wqso��&���/R�+N�#p.S�z�G}'�DZAs�yq�N�xRT=�{nP|'*�:�r�kv
��>���'��7��*��V���D��RR*�����"�O��Qs�S�r
������U*�M//��%��I�bu����}����C�t�m��w�����z
����Z�N�N��u��,x�Y�����M`���7���w�ChuUSG1ld���f"
�(J4"�����U���S������]���:����������l���G���0��&���4������(��GB~L�t�lq?������U����R���v�X��$��X���
�ok���|�%�5�,���W=w����(
��C������-�g8��+�$7�at!��F�3��QyM�V�=��;�VN���5p��]���������d���1����V�������V���,��e[���/���:a��1�dM���a���:N=�9����;fa�,1��LT�-T��4�\$�&U'�U�q�L�(��h7{���(v���YS,k��}�@!���ib=S�����5��}��K!���Q)__��w���5�]}����z7
\�>jj ����H�����1�{c��2�N4���l1�Z��t���H)��u�B*�i'��n0����g,�8L���]�%a�������/OZTy�&�����y��qB:��.�_��9V��(��K����=�l��2�:b.�H�<e�F�A��w!MWy�|������{�K�G�i�V��P
�)nO���
D�	H��t��g�3��B?�JCQ�PZ�x�z�����	��H����XQ��#<� ���!7kH������O�m{`��(�4*)�9��T$��^��U(���f��9Eqq�P�j��X�x��cN��Ul~�|��_%�Wj��Af
7+�mS��vX��a%�������JVX��UT�
��h�j�"\�:����T���
���OcF@�����<a:�QU�k�B$�M\�����]�C���Z���Jk������HY�������z�����f�<�L*�E����0�i���P10���*���.eaH>U�������E��l�w���-��&�bir"��&���r��_�v?��Ypx������1� g��i��M��o`Ne���Vq/�WRo��W��i�C'U�U�&��T<�E���W�����������7�J*�)�4e�]8*v�����@?����N�UQ>S*K6Q�F�6xx��W�����~% ����$���I	 ��(�*�&��d#XH�����	 �3����2 a�h����=�6�_������$�v
!�r����FKd�)R�%������M����G�� y��X����=���`���10�r���P�UC��!��X��AImW&��c�Yh�����.S�x!������M.:��-�;[% ���@�����*�2����T�s8!1���@��^k��<�VJUH[������?a��y0����IEz��������A�D��S�umfq�Toq'�,��p���,���To�pB�����$E�wUo���+��i(�n��,�����be����I�p���$qK����if����#�C��C�W�dT�Y�:�+|� ��[1d6��Q tX�A���&��-W��Xioz�_k�����6V{p_����R�\��k`����U�Mo�2��\�X�w+���>��wC�nh��h����e���%:�������q���b5�}��?Gs��4
�u�[F^�	�_e�,��A�I�#JO��Ic�Z��/cGB�Y�'�2*���2K?���=dt�o{s����LC1H���Q���S��w\A����y`����X���y���l���m}a�;F�\Q���TH7�c�au}0������H�,�vYs��Rk����-zW����q?��>����p��4'H��SY��8h�``kpVJ����:v������ui$C��� ����XW�	��A��<X'X'��JoL�N�`�8�������4��������XWc���fG��7��!�o��F^
��O�Q�jJ��]��-�!C��i�c�y�K��c����rh4���G��J�+#P����:4�P�u
�i\al���nB0V��"�H�5�\+��G6��~��V7�wPU��t\���Ut��N
�g�����v�_c�A�-e.F��H�E���k)�a�ooq�� ����z���D���l����
w|��[�;3��[}d�<��������A_��B�AY�{H!�\!=)�
���{*g�]0�����A���rUY�B�gU��'�������S���r�S���(?�4{mj�n#R���[�����^�����}�w�_Y��Ga5c�B���{-t\1%\��\-V	�z�8� R����[�Y1Y[�~��_�����u�����5@�X�KX;��v�k���s��j����������\��)I�Yse�g�N�',������e��0YgQ6[��2��p�>�]\<�^N���m/�F�����O��>��EP�8�G���}c�k���N�jiO��
����0����Dt�fq���QL�zS��M:���_�������)����?(U�BT�z
0Y��z�TL�zS����O���E��K��L�70UL�T��
0Y��z�T,=�&"�����AZZ���,���n�wB��l�^~�����(�)s��|f`���V�����e!��d����lu����\[r�Wu����	!�C!,���x���mJ��U�����_g�]a1��b�>Lgjo�@o@z��F*]�\KalO�f��&�vp�����V�����_g�]�D���>Lg����[���7T��Z
c��7T��Id���?����r���j
D��
��#�D),y��"m5g����i�s[�F���g}a��&��S��z0���V;{��$�D�/;<��Ru�s�$��9�zBdL%�f��E,=�r���Fg��q9?��f�g��Lt�"�,���P���c����VC8L���Y������r/�O��f�m��
�����CI����Xy+�����$K�R2%�9;�Svzb�*?7�s��sW��k���7�lX�R%]���I�Qi��k{��kK�w����#g��k{�$K����9�']��'Yj���8���I��r&%-O��5�O�B)y�Hi{R �B��*�/��F����e��{��*eKct]#���n).t������� B��X�,��K���ky�+�E*����`�]E�����Z�f�?|22��~5O�����^��W��0H6���(��JM~�{h
e���2����F�NslLs�Ms���X���2�q�4�*������I��91�9��9qNs�Msb��$h�]Z�5Ms������7���=^j���	���b�kW�Yrc��r��>[��R�g���kK��.��-���"�B����q�Yb)���Z��\�/�l]D�y4/�/�~c�g�&vrD)�����7�?��Fy�nB��:u�*��c�o>nGpO��G���cG�&-�������{��G�zMj�Q�>����Y��s)������`t��[��@(?~����I�Z-�k.Xl[���������`�m���-X�����Ur��mX��x���=v��d9_����������p����yc�nOK~[g��[����Yw�65������|il�b^\�g ?�N��+p��j>�"�3�x�G�v��	D'�S�k)��HK�4�i�&<m����4�?$9���5P��FI���������{�!���xs��l���G;G��E$x@�����O�Ab;���q�@��kr��'%�Oz�<5�~p&���������DC�X�*�I���T!*��yUZ�����]|6$���n���UHQC ����B��7L��GF�
:���w|}D
�Yj%y
�WB��y�R�8�|��T?����0�����qo�c\���q�;�=�o��;":�Wr��>�:����
��k� <�[���;�7�vG��o��(q����
�`l�w������'oP78����c����u��
����o��h�Y���A�������8���Y���A����w����Cg���u�?T��o����Y���A�G i���o��n�UwW����w�o���s_v��uw�&a��}���9
�� �g�tI�,����c6�������KfY)2�����?b��y�"��|�p<.��V��_�&)|88�4�n^��
X�������%��E.`���u;�Z�p���%�lj��
��2���
L[Z��Br�]��l-��s�s���f��J�
<����-�<3�����d��Nj!����Bo��L� p�[��
�&v�����K{9k��YjRM�,:���b�-8�m�������mG��Ph8k�ZfA5I��X�����&���8��U��Ku�Vc���
A:EF~�*kB�v�1`�Vp�>Z�B�b�V��Tm+���W�
�s+�k,�
�}+Xg�|+X�W��
q��;������+[���q��`�2�P��xJ>Av���J'��0us88f��g���.,�PY�����8��{U�{�tw5�uN�\I(X���BU�#/X��I�+~�	;�K�k��{|G9��q���4���`4e�;����N�y�caO��J��T�E��L9���r�IY{A+���N]���t������=���V:	H:��Yj6�j����[���c�KcI*��?�)dG�������7=�#��xV�D7�3gH���<�?������]�F(��Aqu���}�d����s�e�}U�2�_����^������bN]cV���}NV������0�1A��p�<���A������^9C�9CI�m�h����5���4
��zGX�������S�1[�����$Ab-����$���Q�o�����q6El�� �%8�pg�u�MzS�v Pz�hT���P�q0��@����+���_>*�Js���f�/q1�R<we4G�	,�$��,jQ��,��,���2���,:�������b��bb�b!$;�v�����8�K�1gO�H���[�Vb��v���}v�4��^��/1�0�Mjq��I����oZ��s�������Du���V�'�����b��F#P��<q7'�4��CD���:"LLx�	�	��e��a�;����e"�*~���:mDB�26g3�Nh��������>�-�qrcsrc�����
8�
o}r=�Q�MnbBnb���������D�s��4N7�^�3:)�,g���wRo�Y������>bRMR��bP��i��<��'n��������7[;�=�dB����=e�?��4�2����,],8��y�&���u��Ewf|z���99C�-�k�GY,�H�T��'���Pr�*�zK��[�WY���:-
@�5�Y}�lk�T6�J��S�K)�#����&f��?�KF�e�#uHZKN�&���"�l�p�o�)�J�S�p1gY��"D�J/*9��d�R��mT���C��!*I���UD�e�*��o1�4��s��2�J��l��cF�H*����Fv���*�H�M�=2g���K�$��/�W���cM�m��s����������G��!C!��X,�0�|�YBK�������aoD��XL��=�Rchj�j�f]����GQDSz��m���H�6�Nn�x:5RH��q��6z����f0��3���~�8lh�9�/��gu#	I�<|�����!3%U���?� �Mze�����r�����azk(N���J��E�MA������B x��}v�9��)��0$�(]d�<�0�D������9���=f�d����g9�T�v�Ok�X�����C`�/��O}|�z.w���M�2��j����4,�)�-C�:��yZ�r�4-
CJ�l>^0�kXR{�2$�������>�����tD�eu�=[u'����&�%�y.aA��}
{6D��o{J��}��5%�k�UW~��ui�����Bl����r����
%�#m:[F�,Jon&��<��M2��<�^f��||q���	��x��5EDQ(�i+��t$�r�G����#�S"V:�
��)l�8��l�2�����*����H_IX�tz.~'E�Y�duuj����
(��y�W�JU�2��m%%��C��W�*������-�������+��p����T�4�wx���cqd�L�����< �MgQ��j�K�A����a���eS���-�\3j�38��Q�����OZu�0���g����y��n���#c����a�K9}���-�>V�w���H���\
�b2	�(+��[(O'�����u�o ����3��q���Nk�2��u��37�-��z��y��,�e�4���gY(k���y��U�q����Mm�� ��w�Z��.vg��l�#b�9��a�t���K{o
L�rY��0}���{����	�������l�����sJ@�o
u
��z~�
��BO"��(M�G�|*���K�"L�U7���S��`pkPi�vZ2�h�`Op/&<6���i��<��C��^�K�������6qA@9T�d<�����8:m�F�#O��m��T*��cM�c�
\��{��|(��������H#���4� �DA�^�m��~���w�-�^^�)�+LW	D�>���V��9���+��A��^�	�<]�GOA��t�]B������$����|5���G��1
�;�a:������~gV���YA���vvv�����v��^� ���aD�'����p2a~�h�.F��
����)R��:&[��0�����F,�fh�`��]Z�"�d����8���nf�e:Y�<����/���'���T|���f����r5����uz3�Y��z8��]d`GA���MR�������]�������	�,�XMG0�t2 �.����x1�Lf����-�p��!�; ���Q4=������q4I� �+�D~�&UZ�w?<#?�o��O(K�F���tJ��>]�ap�.���|�VR$�X���Kv(V���ea�+����������^��2+q�-����N;!;��zrg���&k���>��,����l�.��R���0%������.�����[��St1&{=;�����koo�����N������)9X�3do&����ij'}�&aX%T�85�����V9�sj���V�_�L����6��g�9/��d	9�����|N��(��r�9$Z�4� 5�����9�@�LZ�r
��X#il�a�)l��-&,�XL�Jb�[L��	x_l�/�\k�l����~<�/���
R����Y+��bw�D-��r�i�M�X��<6���6���5��c�l�E���r�+m��r��]NW'�r�}C!J(����Vw���1�g���c�R�OW�I�KHoBD��lT��A�*_K6'��XWJ8J9��
%g��e��X�L�QF@�R9���rw�~'�������?�e��Oc%d�;`���=Sf������U��x�C�u�w�,�������K���������v�W~�>7�Cl}��F��L�wc=	��7F�KK(V�uw��jO�����'�0vO�14p���#V��]S-r��<��4e+*=���N�-+P�{K>���p��K�)�$,�%��K��bu�%����C�Tu��y��,:���P+K�:�y~�mk1�<I�'.����9a?����9R=�-�����%��_�P|0/���]�4nT��H=�����9��h'��I�e-|K:���.A?w=���Kv��qL�~8<;��������)�
����v�Yf����Ex����RS�����2��U"��o4'**��r���2�
c!`�0�;�_��W��W��U��y�fv�^��.h����CKl�kM��>+�X�*�U�[�����VIg�b���@d����j���l�:� ����O�g&t�G�s���"j��`m�bTy�1��}�F6�Be<���{��F���+���K��)�c$�����s��pW��M����7����-�1�K�A9X��3*x%�� +�������	�>���:�-#��8�-��rZ\�#JPE6Zf��`��Q1���1� ��y��mKe�~�%S�Dgt]��w���!S��������
��������H���,�+�[C"���)#�C'���|�6����S:��I ��������6���
�G�u6��6>���*]\���������)�r����f���^j����[�I�� ���m+(4�G1w�;��szb�[���e�1�	�;�+�Sw�/�"�,�g^����	�f���9���v���FQW�#����.���*����v��u^W�uj��k�$1`�I")m�Y�*��T�}��5�/Ul_*k��KWY*����im?���\��,a��Y�|�g�EN3zV;�]i|��1	���3�Nby��=�����!Nl6��j�< ��^s7:�u�t�k�GG�2Rci��n��n&���jn=��[-���z�7W[2������%�c;�
Z���%3��DXESss+�jh'v
�k}'����E�����fb��NLmh�-j��9���������j����E_�4�g���r�2n�u�]�(��U�U��X���� R�%�@iJ�	WM��h�S�T�p�P��(TQ)�(��������GoH=��7���A�����F����3�T����|�1���'o*���y]�G�6��|�2v�������tWs���l@�C�Y�d���y�R����<Oq���y�I�
���+[j��:��b��G���[��N��#=O
�_c,�Xg�6��2�=�e�"���Ld�t�\�nRT�c,��z X��"J��W#�5��!����u3���!�okM�6C��q��;7������7�u�7����W��s3��+���,�A��&��]8��.Bf��l:�LZ����E�i�i�6��N�f�>�
����)��<@��m��h��P|C�����If_�]s/��h�	����v�7i��'�L�i���$��d��,c�<c�$��&��W�il�k���{@���*Q�������?����x��
��X�w��+
aM����������asy^7���&:���l�H��������]-�H�{0�&���rWYfx��%����5���a���A�����n����n������{��:I� I�!�����tE��.�V����}z����+f����E����?��_t���I��{p_\$I|����0K��[�%?d7Q� �t�����,�T�,:fs��P���s�7���F0n6��$s>������������UF��n��,DL@#��gI�Y/�v:����a5�[6Z>�������?G����F���@[��4�L�D��E�E~mm����)���-��Y�3@r{��yF�J(.2.D{nq��#d�����Kz���q@���q�t���ma|k\<LG��N=��,������O�wO��{I���(��;����C�����S�����:�/�������h�:��O��+�>����(�����p
� 8oG�]�_����&��D��$�v���q��?�+��E�������^/��/�,p��&S��,��N�n;����G�o���:������|6#��i6y�dO�I{+zt
3z6��L�����2{N�g���G��xr>�A���"�,������O�?���8�&j��\�'��?GW�i��������:��C>�y�W��z�����4��W���a6JW��d0�
Zv���
d�W3�����#��4��S�y/�^�~>��g'm�Z���Q�����'�=�@�d��������l4N!&
N6%�j#�����(��&P�H�W�tz2��FkZ^�g��+�-���1���z��9��M�����jF�8W�v�*�V�1�;"���"4��'h�R�`�Y�O��fv�����l	hM�k�����<[d J�Y�N��R�
��g�E:�����<%�& ���`�\d�lhi�=�gL���t��M�����\f���M��Gd�Z�?>�~��������|{����#_=A5L1�9��3x�_���	��x������!<9�q���m��Q~w����j[a���f�����J���Z�>��&Z��s�g�]����_f��e�����i;��j��Dl��E�|��v�B'm.������oCH��TK�Z��\M�j����r#
VQ]�b
VQ_��
V�a��
>;��,XM6��o���i��.�A�h�,��������@����uR�Z�G��,�/�h"K������������D�t���g�?��:H�.�(��`��OG:|?9���g[�	��a�5��|cZ�AF���l�Ih]��S��2����j���E�f���m���N(�&��&�	x'q��)h�Q����e�������"�+*$���@�w����q���ma��i�kRz�h�-W�)��C��� �H"��$P���`Q��.�H��n��rh:d�������D0}M���"��52x�����T@��
�I��Jv^���8`���;�({Bg����o��/3r��w�Z�$xJ	�P�Iv�hL��|�����On@�6������)^/���:�f���1�b�Z�4E������r�-�����|�������#�2v���LE,����Bq�]t����Nwxq��BEu��ST
6N����v���g�?���7s2���f>�Crps�q }�lPB?�2�iX=�=�Y:`����,f�e�������N�����H�@a��V&��YO�P0�e/`o�� ��+��	�3|wr:�9����%�=�����lrR���T���|u}�Y���j���A� 1���N���o�B�o?����~6����{~x��8?&q���������������%��v�g���n�!�=�/P��
d����-$�{[[/Ws�Tl_��	'vM���}	����'=���j����1�#d���3W�]���HYZP ��������&*8^���!���k�9)Z��!�K�������-l�������k+M����R�G�?��d��9�;}/vv�5v}!��J'7������j,��!��l���'�3�_�Amd�^�2��)9����(]]�	x��u�PG0���F�[�����}��s�c�o�k/�a�$�#]X\�\?�Q�����R8�<`*��,TP����2T�#�L�,u�� �%����0X�D;%���u�����"6ud�����m���Y��'�P�st�����H����������B�������B����L���o��E�2�X���-�w��Ac����Tg������KeR|d���sj�l�()�Y(��pW!B�gT�R)]X
]#{�@$�X0�c��,%��+r"�e�����m>	��s��p<��L+;v������/��#O��"����|QdTj@8�_��1����]Wu����d�I�RH�|p8��,�5���R���:������`Z���Cx7�$n��	������eRRz���r�������0�5�3����3=g�,�	���6�g�5����f8C��h������y�A5����e�����hX�-F�`�N1�Dc�a����,�|/�:�+��py��:K�sq:s��������M8��j�j
J��R	^���tTp7$��b�'1����)5��EYn�$��C��y�sD'��,dS������V!�������{�)�b9O��%���T��Gt~�4�|�v��N(r*jlh��H�*,@/*8ma%�y������^���;v
O!��j�����2l�3J����Rf�9���cB0u���K�����
X�|o�j�o���B|X�5�R�n;�#��;Z��n����L�X8L�������(�W.���'�O����?9�<S�����A��N��h#��.�,U6"�����z������M	��<�}Z~��@��;�.��)�-�9���ov	R�����_o���t���F�s�.z���0�?��G���"�.�����0���������"
�$�y�y��0��n�Y?�:�?=����
����C�&�J4`_������@j,�����d$�4�E��7����C�^!�Q�Q�C�DhJ���._�c�EH�f0V]8���q������x�E��$>�������?�~`@��yi3��o��`��A����-)�������M��h[�a��YR�G���K�#W��$��iQ���o��1�KJ���Zg@sJ}6����R�Q�|�e�Y�t
|����L4.Kd1Y��t1���#�������|�M>��(���^�3@a.��W+����)����sF]����}�-�y�P9 PI�D(,�'@�5���28�&�.,�L�b4�<�>f�b&�	��`_2yQ��j������.������3�F�
R#�;��lQ�����������nr������R��,
\t����v��G�?�8�.?6_�� v�.	=8]]#�<G
�5����&�������XP�Q��+�5�8�Yi���@���1�9����W�c���3�i�(i|kA��p�l:�M%�5����~�<>�������7'����7'���������������!�����6u�
��|�����
<�`��=&�[��`����=�A�:M��Wq�E�����jI�
����bT�f�r�����<�3��H�:��N~���w��,z2%�*UF�� �=�,a���t����%���M��O�4ymE�:�G�N�Q�i��	]M\���%�Z ���S���/�W�������;�{�U�k��}2����P	Cp^2�b�t>O?S��P�"�M��b��������^������Z�}�=�C`��zq,���NE��if��Ra0B�L�"3�(�I�B�G����g3up����a�M�I��"�-6����[�/�&�P��������X�0\�a6�M/�Y�Q�HJS�� 7�h|1�Y��L�E�9���}������v��6;����*?�v�%���D����f��V�5��R�J@
��/BX@}?�}�fQXQ&q���4�h&��@��?�ATb}�3J
�N�z<�H�*v#�T���*�����X����B2�J�(yI=��BVJz���}*P�v~$��N..�
���f|9%��U?��CN����5�3�I�MdI����`���s��Em�bSb���zE����(��My~nE�Gz@��6�O��'��sA�X0%-`Rh��������>A�����E$��1�(�$����t�^��C����D�}��=����@����+x6�,�d#;AX~����$.g3�/���.��R&,ZCfk@2�����L;�G����R�8��/�M��Z�57���m��B�����@0��
����[-jHH�)-h�S���A�8>b<H G{O�
f�X�;����m@���AQ�"���/b���/��'���]�^7���"���x�"�e${�����.oG[�[,�P�r�2KU��I��5y�0��+c	���8b��Y�f�]\8�� ~�e�0���A�~'C�x?&���s�v�QZ���,�Y�k�����1�����9�R2�����R�|f#v9��!M�@�&�B���?0�r<"�L��%�L�
������O~nE����zk'���F��~?�*�_��`�q�_�D�(�����S;Ik��H��"c�i���{T��['��=n���U�����Z�������!��	��������8u�~��l�����Ur���t~��o�����<Z���1S��L�	���~��^!�H
#5�.�c������A@P���/����w���vO`�����h�m���l���������������3��������6.��u���^v�?LFCmu..�q/;:���~��{�u�������������g��gI���;���J�
J�,"
Q��"��gg�������2�_\c�
)f�3�>��O �9����-��?�������'��w^�xrz�#�������{�kP=����P��tA#�]���:�����l���|��83]����9'��n�g�p�mm���muM��4��Z}L���q{��JVk������������4�c�=���ah ���$V�UAOE��IN&`�B�9����|��}��
y��&�2�������s��B�j$n&x
G)U���Q>3~����
�S6�v�P�T����U���^t�.�Q���<��T���+�M R;m������o����k��
��8��fJdrF�$TI��8��������u?@���x������Z���7�eP�dKN��U���}��9����^��O���
��Y'D�g�������L�}�xm�BK*N���_+d�l�SsD���@~��@��e�y����04[���K�n0 ������d?ec@���JMo2��:��Y�p3��$X�
�Hw��4�#���sW�c�<O/��������o�����d�YTH�R�%�vI�K�����v���7�'q�I�����?�F�F�2��A��9��Lr7r��L����z�?#�����Fvk������C��o��H�#[����G�0��Ko���1(8��+9hh�������Lni���i�b+���lAJ��?������	a�����\����B~a0��v��c�^������~��Y��>h,e���>�B�h��H�1���?�(�w�B�<R/���b��X�iL��+�~�~�>���9���w�2vL����|Le5�A9
���e�$Z�������d�Z�#*����l���[3r�{��
X�#|�bl�t�
;����0+�b��{��G{�R�rP5c�l�V���|�>�:�W3�V�W��C�����j&���j��G/m5cc5[I/M�6HIo�8JQ01��6?�Y�N��C
��R
���������M��;�&�7=Z���21a�����f��������<����:I���:�D�2(w�k2�k�6��W�1��������$j��j�������7P����v��Nb!��c�|L��z�n�"���|���u;�2 �����i��Vw�W���]m���.�H��*������*yQ�Q]b�y��e�����M+A+_��Fk��v9���r��������%�k���k��5Z��r����#������Jw��(:}�|������4�����8����m�K��L���fm�Q�<�j`��	�Px("���Q�(�l�^<�����W"��=�����RDp;�c;}�M�r3�$��$�M�~��oZ���ge%J���RTX
B�^�&�fi�V�*�-��09�z��w��N]��H�i���G,3������I�(��$��ii��+X�B�R`��K=E5��(d�
J�j[o��CB���n(�@��x�5v��r��q�OK��@�i�%���ke2HQ�+�TJ8{��[O 1%z��yI��x�5v��r��q�O�DN{"�$�]K��$Q�$�$�������A )������Vn�2n��Aix��F�HU����?�m}tl7��3@G��{1�����f���O/��]Z �� ���i#�}15������Y[d����pj��9
�dd�n�>�+�]2E��A�W�5R	�	��/��l����I��~/D��������V�����O2`!`1��c
�L��3��H�3����D�V�+`>S���c;k`u�3���
�I%����?Cp},m-���`���u*q��R9�1Q����'�
���0K*�t���\U���2���gJ���:�m>$m�<��s~?�Y�#.?A\Mp
ux��=vz�
N��!|����F�����r��>�qt�htF�*�0��U����gP9�Os
��`=���3����2Q/o.E��@��X*`)#�]�
�K ��!�4�`j`�y�����1}8�JX���nWaC|��%|�>�5���X<'>]��y�Q1Y���<�|�>k���7��t((�����VJU2Dy�����o���G���.]�*��fh)�Q?��q���m��T=-�?��t\u�:W)�������/%�@E��F�����R���j[��k���8��v�l��j;��"5F5�7����X0��@QJ�[
��.����#���D/oQHsB����U�C!z�"�nW(o�'A����,��� ^op��m��\��j��������*Y�8-fZ]�z"�������@����6����`R��&�^�$�_M���f��V��T�f6���@x�
&+������"����yh��&���8'��u�:9��4�=_g�O���vvzD�������LhT9_��^��*���;km3u���`)�F��1��3�w��	S[���jf+����m�D4�c*�|����o�n�hR<o�l���)
���K�_LAy��&E�?��eU��<���b���H�PK�wv�G����w���e�@�������Y������n�Xg��Co���2�I���12zg&�t0���C�=8��t�d	�����B�;�E��H8����p_������]\��U���5w��|*o��k'��:��W���A���dOvu��|$Z_��YUT�|Sr�Q��������T��VUs0�����{�?�������
�A������]�w���=����dSt�8B-�A�b�I��nUI59&��P���n�?������z��=�@�A�xA�x�;D/�D}�i�eN�t�}���Z?B�F	H;t�5Wh ��W)��S>>]��O������{��[�^��%���acs����W�������O�,���3_�����w}������6�������T0iK�SK|SK`���W���_=��Cv��@G:���E6�0��aa�����9>;9=~������H������(a?�c���%�Am(����j���8�f�l�c��h�`�I�t��#-�kq�U|�
V��k}��`r	u��������YKP����v[J��<3B��U"~�����������	�$��	7�K�-��;��8����`9��V�*��%Tf�7���Ms�VK�F��������KQ�mz�x�Q[��d#`H������b�����V��e��1s���o�&���p�������*$a)3��k�x�n#[�;H���ea(kRcZ����]n�����������}2f�����
���<NS����}�%�|���^�l�����R�}zY�|MZ�<���4&_�z	�&�/���kZ��O���K�B\o�����"�7����E�jP`��_����-����C��6R��^U
X�F1�K?�7l����k����H�Z1�����e��-������{�����M���d]��nU�UF|��d�A)����<�1*����k�-�J�n���d
AR�+O��7���q)]&�!�Ke���In��%X���kM!X���Zz�xSK����������+�M7{`���`�wwe00��?+y�h���'d+Dg�?��1����65�����|t����_�O�^��&L����O'�)�
���������!?|��7���i���7��|`�����=D��8k��jB�z��������.����u�~����,z��<c�0�1n����LV�����[�Y������<]f�E6g�V�����Q��I��������y�r�4Z���g	��u���������"Zb/��h����������l-	���W����f���)�=�&�"J!���{����L|�'>p0����6$���-�Zh���5�Zl4k]%���$h�	h��b��wh���`	mHZ�Q��O��@�;x|@$5�i�CY�����Q�,�iQZ�$�,�!}-e����-F��.�b��O�mj�BN9�m���cR�\5e{V����Ez�}�X��q;�!���5���p�]]�_�A�5/�:h��in�S}��)�VX1z���SY����z����r��!�{���s���u�Ii��B�LG���Ns{���OI��>�����L�Bw+	�pKGp�	��I��^��d��2V���6s�$��������E��d��������!
�
��k�7�a��|E��Dl��)����9�&z*�����S�$�l
�-3�
{�de[��f��b3����O���$�G5(u���A+�p��h��e�x��Q��f���=������6�t�-4����|�������7���<mv����aNF,L��(�P����E��d�l��{M�n�^����}-�V��R����)�1]����.o�7�
����E�a�0�dw�
�y�D��4ZL�m^�\�����=������+7I/}�sg���0�]����ko�0j uC���D�PS��P "hV����b�b)7{���l�Kz�BD+�v* ��M��@
�*(w���.����l2�:HA��������>��X>�������!I���h6Y]O��C������n	��a�e�|�P����{h�x��N]�s��	"�R0�Y�[\�uHkZy}��F��m�[}%�������?�k��8�e��K)�w���'tn����h�Azm$>��~%�������wkr��(`#����H��$�	A�>y�&�� 6��h#	���%A�������7�1���������"��7����-���zcv*q�h���o�z�j<����0���%����z�jz��eK�GQ����{�X�)��7�j��6;�k�A�UJ���4t#
]�4���jvw#
�HC7���I�Ml��ih���n��i(6���n���F��HC�f7�,��Y.�������:K�7�\��Qg��F�kLf#�,���r�G���!�r�w�K3O_��sm[�g~+�LC�����j����ea�I�$��3��,���g�9/Q����
�*���3'~bl{�5E�k��%i\~a����*����������%���t�0���"�nE��*�j ������b����^*�X���*lW��U��5 G���"��Y/���z�#��#/�nq�+� ����������^��[���q�A���n������$��
3�l��z[h�"U��	<��5��7,SUA�L.f����	��|M2�[���,�,��u�T����F���L�4��vY%w�lD��_�L��,D�w�^Vi��d��tt#S5���L�4*��vY%w�lG��/X�L��,G�w�^Vi��d���t#SmL�z�X�{!�����&�(�J����R%e5ISIYM�TRV�����G�eu�PI��	U
�i�Tu��6{�����HU[���l[������m���7o����`�)���j��vaM�se���bEISR[�
���z��M����0%��K3{�����B��&�4X�4��z#�������A=w���7�|���V����U����������*�MdW��/d\�GZu�����#**��J����mD�KFB��@a���?S:6���Biw0�X�#����VS>gA��:[L6W",�Z���mY)�UDd������\�XEd�:�X��8L��!�XCd��Z��g���,VY��Ov�PE���U@d�P����Y���
e�YpG�e"�xm��������+ ����W���[ED��m������NTD���J�Q���mj�,�Y���"�o "KD��Ed>���U@d%�d���Y���
uYpG�e"�dm�������* ����W�����OjC�IQ����n�W�z�U��AWo��Z�8�r�VP.��%e��i}�|}��~�d�`�U
Y)8dU��4,����
��G�����n�2]d���xyMg��)�6��$��?n��b1�M�x-V��W��s4G1��9~|����W�,Z^e�,��E�YuD���F��<Tqus�QAg�����C�����Ui#2���ui$K�j�Jz�G�$��"�#����/�t�E/W-�Xt��]����d�~g[Wc��Kuc��-+�\u��t�^����9�)���g}�;���U^6����6�#��x���OK��u�U�������iYt��b�tRe#k����2�`��vnC�V���<�s�qSf^����t=y;1����b@sV+c������oM#������oS�lF�V�W��D����2��d�)��0qb@Gsk����V���A�W	��*�!XK�W��������"���
���]	�P���P>�S��*PR$��_��B`��r��&�e�wO�������X�w�&@�����F�6�^�Q�lG��]�&���C��Wp�"#8���-A( ����Qg$�b�U ":~�#}�=�p>c�j���o�D��)3nD4l���}9T����m�6'�����2;���f7�^v*��9�����$���J��&��!zUH����)a��mWz��A�����t�
u������y�gM}��iH%$�G���2�8i�*�@�Z2tF/1��@l�^��PV%�k��iv[�DH���>��<�I�g-tL���e�q��z�Iz57	.~/l���I/
z�n�c�����$1�$�*��0�H`�,��+^��$}�"�a>|7	?A�b�4����cZ��:t��kn\|)C@�����ol�V��.�OJ65��4#1����/m����T�D�Kdzz����������<A�e��	S�Q�x�?�
Q����q�u��[e�6|S��������y���i��u2b����t��rG����z�������	$������K;�]�a��X���c=8`=:��1=�c��9��c<�/~$t�������/$�������A�A�;����vL�������w����9�;�����`Q����S>z��s&�T�c|[����:�If������{�L������n�$9����YZ,�]�N����j���e4��gd���(�������N�(:{�..�k/���	�y�������	��O�+32~���?Z���\������\�(��`������U.V���r�Z.q��� ������j�;�������b����D��������75 0~�~����'���(�*�s���JHGs,��,��he��&�����-y�
0���'o���.n#�	m,0�4��U;%��<n������5�Czi����+��S��#)�Z�AX��]���z��jRs�b_���T�l��8zl�\�J�8�0�Q���eQ������;X�7���������qz��.*�������
e��cz[���,_�9����>(g��b�Zu�I���++P���Lv�z�� ,���rm.R����9���aP�t�����JgDs���3�;|�Q7}����J6d9#,Y����P�����A9�U���S���j�@�9#x��W<#���^A*���R^UL�Q����q�3�#a[qG[��Mf*E����SPv�U���d���!���C|�C��n��jT�EzF�E�����l�>v�:����uZ���%"��e�[<�f�^��<�\�u����M�z�jtm�m�zm����i�=]��
��3b�0�})���������`�2�@^�����j��"|L����)(�b�\E����)6F^����`�9m���E4d��k7���/�(�bD��"���M0u�J���Ra�}5�D>6����a����"�f���[_����0�������"2t�5w����?����H��V��7'�*K��cVm�k7�����N9���]
T�u�k�b7��u�_��g][`�f�dc][g���6���=r]�obZiaxc-6�
{t�A�Z�D��"�!7+4�l��v��ok����-���k
���m�k7���C>\�<(�P��6aVZy0����,�12��h�������E�CpZh���u�-��`]��[z�[�2�#�X�n�k#����Q��rm����`B���Y�7bd��h����I�x/7�������
T��v��6[�+�B"�"�}����	K�����nAi�Q�]����7�hr
��`4�!�%�Yl�|l�zJ'���+��S�Q	�V'BJPl�\��w���M���6+���J����8%nT�QA��x&*��j�R��H!0HFpxj����C���t|yu_�T�ECw�]�$��_Cv�}��jK��tG	���|hG���U���[�i��������O^��E��|�7���=TRv��h��??\A���[����	���=��_����iz?�|�
��=\�������A9����y3,o�_���_��)���[n�v��p��*r;K%���~���������Ky��;[
�n����������������7�W�nz�	��o�C���o�ymL����'��|C=����A����jbPC�o���h��wM��zo����41r������[���#�����Wo���"���F�����g������.�c���/j�8��V������B���2���L6�Z�-��B5���n4I�a�N�yv����([`���t�^g���3���Zx�U
y��eo��NN��l�<z�(��j� �?�{�6�H��I�.����P�
/���c���L�{Q�-�1���=��&��3��e������,�������]��|�<������g-�������8r"N�T��Zd�s�����Z��*�u�
�9`�D��l�@k!��F�v���E�VIP5�d����VYE2�[�]��pqcb7���{���/�0��GX����[:����%p��I2��uR����Av�O�:�q������NUw��Iw�F��B%�9��+"gL���%�����?�������?���.x
g��T�N8�
�X��P��"��p��6�88�Y��3��"#�� k����mP�����(R�� k����O�l��ir������j�6{�^����7����p���p�U9���m8�{�������#�7p��]�7���P�j�+�x7{`���NO���W���o��~n<�>lB�0�9����	��7�|9^�gS��
?G���r|3�"j����^���}�����Z*]G�4O�m�xE���*�oG����>{Mv����w?����k�5�`l?�V	�����+����������z���t�����-m�g��_����,W��Yk�W��k��_�����>���E����k���m��r6��l�?�z��1�J��4��KX��8[�\����`�8&�;~����=�= Sly[�%"��?��]��("��Y
L�����<Z`�gy�Est�Y�E����cF
�2�����y�d�{����L��'tp0�b��p�j
	������v8���al�Ck��p���Z�7���0��al�al������a��C:���*DG��Z�P�r��v���Ot����t�PFl��"�����pUM�Z�PI�x:+b�W��0:!,u�������i��V��(���E8&����z��$���
RZ�a���N���BA����=Z��"���a�_r*���Aw�����r8HH�H�;L V�*��2�f����"a���EP+��W/����N�P�4e1��X�2[1�\�G5�a���P���b�Y�����L��"Q����X+>�W/��A���Z���[��������B�����rZ�o9m�����l%�h�����A�|�����7	�&�������V^�\�d+���6�vg�V�Ki��p�$��"�������7t�����������>Z�����^5���]��u�X"A������d���]2+��������L�U�{�^}�����R$��������Q:"��&G��^���*���:��K���8<H��>Y�@���<8./R�N,�#��i���^���X���>�Bnl@n�An���X����qE�-Sc���7����B��������>���ev44���^���D��-��rr
r'�&�&�M*Bn�1S�L��xCnI��@�5�@�Qr��Y5���h���� ��A�
�+M�����������^��l�E��e��.d�,��-�'�-�W�09F�b�Byd�iH1�*���)��@s�L<@��@�d�Xj�K]���<&s��%99�����Z���zB��u5T�����V��=a��w;����tG�]V_��cv?r�k*����z������8�T��*�����h�Aj5~u:K�'w�M�)�����5�k�\l���>��6���s�
����\�����V�g����ws��bi��C���Q��
�}A~;P� ���r>�n����c���fE����dfE���CefE�����%�C]2w��/�a�A�{`���wfw�=��X�'�A���O���o���#/������6�3�����O��#5P"�e�M�2n�\&��Q��R�tG����)�w�(������0
ToH�M�72�pq�:%�6y|K,���X
�C��nY��1����//��%S�M�r!�mm����lu�u�V�j[�������&�yoi�X^�(����Z����+�L��)��<�Cr3q:�����R��������/������O��c��g����	�*G�\��d��!��sU�-V�X�z$�����*��(��t\
��c���G=���;(D�l;/��*�����n�K�l�8��-�_�y/�����&Z��wF����J8� :|�p�e� sR��XO
u�v8�;Uz��qH3����Ct�����m
Z>��C�L��C1	���V?���Jo�a9_�������\�]a��p��st���4�x�m���1Wp�+9�[w�uw�
����k� <�[���;�7�@�k���q�8�N�
��>��;��s�s���7�����c����u��
�[b���q���{_��nH�����8���Y���A����g�`�'����
����={���#g�G�u������N�W�]�vw�����N��}����������W��v_m�������w��
z���A���+����,��rEMl�nxNg�9������Q��g�vyp"U�R!�q/��z��4o��z���=w�i�4{�e"��S[n���C__]_MAE�@
e����|]j�^��P�Oeu���p-�Q4�����
�#�=����6�z����
��� ��������ru���6������Z�E���e��A�a���;���x��Q5�������E�T����6�'z�����(�i�L������:F�m%��Q<t���%<���Ne�R�az���"��	X�Z��jP�UY�U������5��m�O�*�o��}��W������yc����H�HuF��+D�(�T�����(��E^3�x%�0
U�_���(�@^Y�Z�/
�HO�i?���QF�k>��| 
��M����(E�+R0�^G�0��T�V�f�AV�����*_X��K/��V���
�o����l�ol�=I1���^�>oo$�I�$�2��[R\�o�H�7���V6��� )VCI����x#)��FR��J��0�E��x#)�H�����K���3�DpE�,�Y�r
��D�%�*Q�I!���D>%�5'���K�D��+�l$cE7��l�{&�U�c�R�$����r@�����]0d��0cB���Do�a{U01�d��6K/}=��#�{�k�����LV� ���:{U5��Z��4rGa�<c����4�]M�����a��0o�L�Z1H����%ep������@�heah
�R+��B�4�G�3k[���j����
�������c���i�v�^&�)��c����h��J+I�`�:ar� �� >��_B�;����kc����.s�;�.kF7;n�����V1�~��<��=9�Fmt!/�Ta�L@h�=
�c�+���K6���u���bf\�YzY���pH��v����!9�i��#��K5s��B�Y���Gn$�lF}k���;@/�� �i���o�$y#�6����4V�����9#e!��g��#��U6#��5�����Cs��4N���a���C7&�~X��}�	��DLA��(2�9E�����t���.p+)��J
�2�������!+)�>A�!U�+��l9Zt��������"�n�S��M*�dv��_���0�],����|k�"���.�6��� �{��j^��v�u{o�srro&_�)���<�4L�e�B���&�d\�4�z#������A=w���7�|$��V��k�U����������P�Q��J/%�!�K�z/��{��EauEC�5Y*�.aK��m��KH�@�b�>0\����
��m���K�3ZM���[�lN�\�!�j��^�e��V�r�r��r�c���buT��?���c��+V��}�"�

�
+(����T����V@������P`����
�Q`pG�e��xm(�������:*�����W��[E��m��m���NT���J�Qy���*(01P`bA��v�P��\���DA���&�Q����h�P`�z��}%0���B�JE�z�(0Y
��9es%:�
(�z��G�&����������>��7����������={n�����8�r	����\������Z�����JA8� ��RP��a�����j�u����:�m�un���J�z��7dm|uq*q#�
�e��oI�@Bj������R��p�K�]���T\5�����*���K�P��0��]�Xf�������{6�^��$�9���M�����;D�
q����r�!-�C.(��J�H�^�A����F�����h�
�*��C���	�2�K�
�v�^�b�AsO�{�sO����zE��9��Bt�)��r&(5.�3�w�Z	��5��i��k=�_����F��H����u_�R{�u_���05�Y�����u}\�m�m����8�{A�����f;&Y/�gInKpS��<�d�g$�#��H.Hp9�����h�"���H�Dp�<��4������L+�N��B����D9���
A�*�U�R�.6k�fY��g�{2�0^�;���m�w��*�G�9�E�	9��[��F����~���l[����A%�
�{��[��=��7RM��(k�^��4�5�]�/`k���8����w��+�G�9�G�	�e��K[���{jOr���
��&�2h%W��\�R��ww�iv�.����k�.��x��E��*�g���.��U��	�bF^���|�1�F����s-=�u�Mm�#���5���?b�	���>,&�G���Hz�~�����3�/��"#��4���8����r*p�zC	,;�y�8v�����'
z��x��2qjX��q���X�m�����q�
���/�Y�r�U!�@J�Mv�RX�4*����&�\�H��a�%�����)���Z��BQP�%�O>\eQ�
%�A�Y�^9��I���bE����kV��4���a
i��A��� �
���H������Itv������i�J��t�&��F��t>�J���/��g��^��N������NZ����Ze���x����F>~����t�A�����7�;k�}k�ZX������'�g�#���������l�l�?�z�:�^f�l�.��"���E���zGGm��cR����7�o���2�g�e��m�i�lm����pf���>��>KpF�Y'C���{���������"�}EmuY]}v�����Y�]�Z���z���,W��\��\�e�b�r%��+�/WR�\=��I�����k,��g��o����Gw������e������2	=���h2��c���=R�P|�t:����m�	����u�W�ZW���O�2V�W�:��g.e���=�R��R��F7�|9�����"C�O�t���Y���L��y4��Q:����l��/.��Bs���$Y�>v�"�8� &��i�VAtDF���	h�������i,������
!��~�����U�����/�#:���Ec�nRtO�
��\B���< �MgQ���:�D#�(XJ��7�OY6U��o��E}��J\)(|�m������j�F�VU�����������<��~�Z�����1��!�����/���]44�PW��Z��k5��
88pW�G��O��U��;�P�N���9�a���J'�����l��c�6��Zhq�O�@���� ��y-����-��S �XJ��=C��PW���
*S�wY]r))�	�2����j�]h������9v?f����bw6?���	QO�[�D��Bw�]�|';���o�n6�7�q���v���r��������ix.v������v�'�$���5/�Q:=���2��2N���h*�5]����K_��k����]]�����O@�*���u�����D�\h��$���_h4�qn����6�.���oP(�-��w
D�c-Gj���~�i����2�u;M"M{����n �"����j��N���� �J�ep�af%aU��Z�V��/�[UvX��:�RT!���F.�h��_h����qv�z��w�����;������)����)��pNIR�P�`�d�j�*`a��+`I��+�IN+`���V:�=�����f>{���g���xA��������r�`�"jj��V���4��}��e;���,Q��g�������(o>��7�	DV����#G�yXk��<b'��k!���<��� ���<�SR�<��?v"��y���F�d�j���<���#��\Mr,6Wy���GlG�y�v��-����N�f<%skJ�x���cR�q$��%Ng%��>Dp���Pv��(����c��6y�F/�*����Po�(|R�,��~���k���P���vyC	�+d��P����jG�
@�4�c
�!J��%�Z��*��3��)����z�Gc��!�~��M%B�W��:��s�-�)n��W6���&�qehg�Rh���{����H���������I�3��-�����!�q��x�["���Nn["������7��[s���7�^�\���Z}��E��	H���M-��b�a��E�\�VdW`���$V�EG��V��MKd�K%�&�":e��M_D��& �'���*��.���U��4J����������� ��������@���e��	|�b
��<�
M�b~PPN��{�:
r�]�i�P�p(H4(HL(H���:��H�.d@��Lu�Q;k-����r��tB=F�zU��z��a��`�r��e�,��1f�n5�l�$q�,����n0�����O��	����Y`�X�����_�^�E���u;�����g!��Z��0n�����2?�e������F���P���a��`~8ZK�<c�w����(G�H.�55�`��]�e0_�D���m��%��kb���e����K�^T���
d�m��K4 d�a��Pb2�q1�.Gp�\o �xw�����f�eD�j�f���a�s����������s��T��c0^2�����yR8��SgBP�!(� (1!�P8 G�8�;�IC��\"���zY�	IM�>���Q���Rq@R ���$B^�j����u��Y����K|�7����f�a)���7,	p�iV(�,)�,O��!���4��[b��%�Dx'�g�/�%?B���hW��@��)6p_6���{yTu�c��������)l0+����['��X�W����F�)��P������B�)�WT��JK�O�h��#Q�E\�
���w\4�Z�Z��^� ���X�����m��	��c,���1��H�Dx�e�4hzN��L�%3�@��YQ*iXcc�bXc���&z�:��V`�]�Z�$BT����V��c���3�3��D�[fE��1`M���516X�Y�&V���0k��YkZ�'�`���P����C���A��w�'����J�b��o������I��SqD!b_�3cD�b`�*C(�$wx �����9����O����+
W�����q��
8���-)����$�b���������������G�E2�'���=e��?�^�?`K$�XB�M�t�D(m���%��j
��(?�p���	�k��4p��b��&�������������H�Xl�4T�[*�~(�C�b�������@()}��Mm�9��5^n��H��*��r�B�_
v�2�n�4,�s�W�b��Aw3����:u�=��W}:Lu}��^��@+H
�����k�K��F���z|�b>�@����=c%C��og��k����eT��q���>�[)���wc�&z������C�{���]����}����T2�����K���e�|"��l��Z12l�S���kn�C�Dk`
i�7��|�.U��e\B�KL!������ ���c
�����p�
�.S�Sh�B%�P9X��$��2Q��'b��2�Ma
oc��:E���V;������Cn��S�H�&�aP����05C
��;'�a���s�3�s�8�l
�&GrnJ��N*@��6n��T��M�Y�s������I{��A]��.)��j��>�C�^<���	1�nh4��C>9��f�_qB��s�c�c�khBz���&�up�S�ld��l95�,_`�������	<�!���:��^�h����K�BycQ�(7��P�X<E���.����(�
���"����S�d�����&������(���1�S�=�U}eQb�C�Z��[4�'�ne����������/�f����}E=�0���'$M�������J�@������������ZWRU�]?}g��2��*�+���������J:�mU}�y%ay�������-D�V_�VI��'H+�5en���A�K�-��qV�n6�6��0iJ�XI���(1P|������1mCx��P�p����T�#������2H];�K�`U_m%�m�E���"7Xh][PmhR4� mLs���h�@{E5���!�p#t2����j���(p�������}���{{�8����N��yv3�R�p��|8{y���C;��>�W��<~����f��	� Y>%.@zB�7�?��&����E������D�X[>�*mDI�
�������Dn	����k���63�:���:������n3����}V���~w�R��:�T�d��/_dF���N���X}���g�O��Yr�W������49<�E��"��u����1����8��q�����dZ>f��SH����f`�O+��g���S�JA�b�SItu�������=Z������C;L��A���X=��+�f�a5=����j�P\Id�z��X�\����:4q4h%�w����&}�ph5�)�$��
�������M�l��V�1l��j�H��Hk�-������pM��)���)�&"W�r�?l����a�����!�2g�`���`BjG��z�g�R!Yu�K=���i��y�{p%�J�8����/#�^�h�V��?\'���E����ev��63S#"fj��L
��yDim)*p[�2�������c<��:N�Eo�l0��e����2 0k�>XfG,kSp������J����m)*p�X�2��2��ef����]��KS��4gX�Q.�����2;bYuf*�����y�`�<b����IZ��>�9�������dC��k��!�k��W�)�`�%C|�C3$�H��q}u�2�O���L'��:*���&�|}��m���� D��P��*o�Pc�%(o�P��-Ev�t����)��W=��J��~�K���� ���K�b�����Z
V5�>v����V��&�N�j3��9l�Q�i�D���\j�ZI�^=�m��n�Zq�������B���������o�s_���7���D�9(�� $TX`Qyf9� �iZ�2�w�?1):}T��=)yC
)�
)oXLn*AJ3|�>_��f�s��$��ca##kNFf�������_m��/~}�a��R�v��5�Ldz����/��������S����WZ��)'����c��Dl��������"��U���-�!��2�{{"C�geZ������eu:�<�y�u:���#?������� ����g���[���tJ�N�;1�
��<%y���XN=���/�� ���s>H������Q��*W��U����h���-�%/��Ze��\���P��m�k�rt�)G�[�%��{)G���H9,�L��X����E��'}�����I9:]u��"��
�6K�p��1RXu��.�z�*��M�����|0
e���,����w�l����k�
5Z���6����J���y���e�o���X�|�'%�1��R�B��I����h�>�2���
'�D�
Ep�e�8�9_�N�|�|)i�I�7��KA�
a!�+c'�7K���4b"���O������6g���!���ykS���Kt�e����;��?s!���9��H��3U�C�M|�NVQ��.Xb���`����Vt�c�i�k����A����"/P�XN{,A(�]+��z���
U�v=�+X��	B
�R��6����q���;n(H����'������&�B�BI%*
�`�'5��g�<�{�8�o�2���x��R����#�?�m}�t�����@7����e��)O�Mg��)9Y1�x���<[,���"O��j��*��q�i���������W�"�.g�rAQ��*��2;���k�m�#���(��m2�7W��14�,�{L��-#�O����m����m��_�
n���p$}�i>[�_^���t�i�(,�`}�5�R�p&g�u�L��,�d�}�e�h�7��{K�<�k�^V�V�����P�E]�������yn��y���
ys�����xv���2�\'��_�T��o�#�����1T�Jr���t�_�	���z`����W����}%0�~�Ru�B������Ce0�$�P/�\B�=�r	]fQ�C�a�+�+Pva��L�l��|�����T�����}+t6�����V����r9���^�r������:��/���u��!v����]5�3�9��r��Zs�����ocR:G�V�Uk�+w���A��|�w��{w���0���9?�^�%{��V
XUB+�6���
*������jr9���������B�%FI�)������,��W-BFn���V��X�u���?���+�^YgdKw+��=�vO�����0�E����<B��~�_;�����)���G
X�o_��7#���C&X��|�"�����1oFRX�+I�9�_�����X_X�8�����!��E�0��	$�Bv_��WN`Q�"M��**k8��;j�H���H���
D&�����=�-u*�1��q�sh���F�t!N��rEJD=��f��@VQ��(�����"/�r3G��Ev}7������b��������9<�����cwyhj�|3_M���&�/�K�D���������e4�&)>n����;�(z��,�V��=nGp��{��G��yl��+��{�����kQNTK����)[���X)�T-�V��X4R�l�������(�E�!�=L� ��hl���u�B&�@���
�T�z����7�n�H����}�����`G�a;�?1�s�����-I������@QQP��@�����K�]l�]f,�zm�I����NE{��]�f/!��[�+��G��H798�\��^��D�x��n�F���}�@�������@���4�<�G�n���o�C�6o��A�
���
U��	�z G�S���
8\:�B��R3)5x�d��F�O+�i�f^�rZ%�L�fu��7�Ea��\��8���_��&MV�����C������$
��>�f�g��LL�E:YdM����?h�����^
�
6�����I�S�����&���vS��=y��������{�n�����kBk��Igj�-�e��E0iI��P�I�]M���mR����JK�c�U��||��@)>cK��-_���7H����z�o;��L8[R @VE���IJ�6����T�q}�q �PO����6s�"J����L���'�`,�UYsE����Q%��4����e. �Si�I`���V�������Bx����^�V�sw�C���}m�r�$`	���C1>��f�4���QL��������[�v��w$�/,��2������<��0���~�������W6�&��vTQ�������U����*(�z9w�0������e���=�9rr�z	S����v�l�%���0�a�7���r`��%���x�oHN��`s#f2���h�r�K���g�+b�=��z���i��P������G~�-��y�f6��i6���:���J:K���D��JW�%=��^J�L�e���Z��a	q��mP�w�)S��R����;g\�����������i[�_/YHI�fR�/���U	{F�)a`���o�z!��5�?���@4J��!����Vbk���F�4��#y�����G��oq�kK:�N��=�-�v�g�w
����N������uW�Yl.�+"�,�>^�����%��,�c$�I5����`�b��d�i��p��:E���K^�� bl^��>�z"sX��f,(+��_l7A������w�7��/�Z�)�)o�0�yi�s�4�CmP����%�������>}{XH��}�P�P��=-�Hs=�ti@OM��Efh��;�����	Z��qo�yz,���8E�;j�����p���&�;��)���L�w�����k�|��	]mf�lF��<c~9�T~<��t�5)vU�C�n�A�Cu�D�X.N)_`D�b��������|�����<��V<H'j�<��B������7_U����J/8� i�7
]�v���2a�����	7N0_U��z\�>S!B��0q�jF^��q�"+��%6<�����d����	[Ra�	�D|b��}������{L�tl��}l����t��m"V��ZA�A�A1�o
^����pQ
�!�M�A@uc�C����3(�A�����"K	c��P�����-�;�(n������>��S�h�^
ER���
������57P�9nN&��������@�������l�^*4
J�#U�'�Ah��&<����������4g
OE<��H������K�'?��|���M�<�4�=����9��{��]/�.�����wn+>GY����R6������=Y`�5i��'�P,��o�G��P��n��aW�]����������z��Y;353c&�9M�
�7
�7R���W>�\�e	�uh�����P��#��m��V�p|f���Js�zJo���c�q��k5k��y���cu!bc�����>M�Wk�k*h��h��7(�d�����w\4�Z�Z���(&�B$*(&j'm��VPL4PL����b�(�,��f�K�s�b�t{�;.w�f-s�?7��� r��S���zU"�������Xr�����Zk.���+w�|7�����/�������g�Q�����d<�tg�0]������c4O��dGd�e4"��0:�tB�����t�y
#����x:Z�~�R������������G�"�J����H�����d�bI���%�!W*���|A�/���E����5&��=�R�b����I�����C�U�$u(��4������8k�/��S��j)��'�T���K�CU�����\7�����]*�/�^U�33�)s����z�x7Ii*��l1C`�����>�v������R�����[g��&���+����v������j�����r*
�^�<NS�Fk*��O�������J��&���QA���e�r�E��g	��$��������Em0f�0v�������S�4�l��/w�s.�������D��k�������>���g�E��7P��Y(3���,��aG���7[@�MLd��MHPx��Nl�m�C,Xo[��������������`�m�#9XoB:�S�`�o[�0��V>6��&d9}V_�k��Y����}�-�����U6�M3rN-g�<��}��&QY�w?�M�=��Z�uc�����l�����[k����n�t�'��o��Y�^�D�q1����1Ax�"��O`����M�j�U"���i����:�O������,���w D�) u#������8�{@/U�9�+�6,�K����������}�����^y ���J����i#s2j��tdO!����(6j���#_1�oc�S.��I�q�v9�0�Y��l��s���A�q�v�8�,����l��s���9�q�v�8�(�Y��l��sl�ZeN����I��N�^�}Y�i�������t5���Ie����b<�.��4Z��t��1h��9~|����Ws����E]�`B`�F��F��<�i����bq���i'0a`����`��|���biugK��r�����}��U;�RC�m��&�����|'j�����p�8e-D<<^�R�C��,cs�\���������m�����X���f~���u�'��������I�R�Zsq�BF�tt��~aK���K�(x+>{N/����B/�X;j���S�U����P�o�Q�Wb7��������
�v���?^��/+��x1���;��7�d�����U7E���i����lN|�pP�4�^�o��_R��@��~���$���u�u��R�K���� �����[��.D,b�B%���su�EPq���s�.Q��@l^*�	�:s�PW����m�I�C���P��w��9���8�s'�!+����f���)h���v:�
/�����������Njv����~��c������DY|��
���X���H�l����
}#��1RKY*���L�$J�46w+�V� o����q�[i���@_]������2� S(���s�c�ZF��y�G� �xk������I����'���6!��?Z#��\������\�(����vc����n�����g��C����%��]��~���%�������Sll�$��=�sO��Q �U�*�s����Xz�Y����>��E�d��+B�!|���f��)��	����5B�!�L��
���L��1� F��y*�3>�8BI3��%�_�&��4��X�4*.Rz_A�ZC�z��U�L�����2'��$t����R��`��H�Gi�^%r�������fB6��dA1�ic�8\�X�&��V�oa��������F�I/c^3�w���\�,�#_���y���;�U+������+J��h�o��
�^�&�����[i1C��^�[)��b�Q3����a�DT��q�}����Y�4P�D��ZtY%��y6��d�
�����Qx�A����{M/������5�_���������Jg	m��J���0�L���5N?������-��K�J�/��b[���
�
<i�������G
��u���#�����|Y%E���$������;��?wB2G�:w���^�s���Cbb3����_��c��c���X?w*W������!d:\�s�xi�rc�x���.�*m�\���SB�8**���M���;�5��Y���a�34��&�MCz���_G���z���=l���~�bXo3�g��gP����%#���*6v��-9Vl��x��F��/�J��nya�6./,�����8��~y�Q������[Z����r�����W�o+��[U�q����l�ol��E;i	�dAd9���6k�`��T����j�uh?�k�MZ�B��k��;�u"�IcEC]�7"���+W�����+����1G
��JW�C��Q:3.Z�F��V������J��L3��I�dJlz�L�n���W�J�j3u�D��O�z�X�\�+;�il�����[���l���z5Mx95>6zz>)�	�ae���s�u9�1����;B�9(��Gs��V9�<�����l���?B��X;�c�X�m��)Nn�X����cY�p��r����p�b5���N���8�&V�j�O���=0�46zU�������l�2����4��������'�X.D��34H=�?�c��Te]��JsP��J��G�*�r��/�S9����C�G���h�rb���X��Hm�X����cY�p��r����p�b5���N���8�&V�j�O���]��46zUra�����l�2����4�'������;�O��\��sgh���XN,�rP�u]�+�Aa+�9O�C����Q�������x�Q�rVt�/�+g���Z��|��&W}�/�kr���Z�|���W��/��}�4�Z`�\���X��/�������[�<7������8�%����r}V_�c������o*��3P��D�`�
��`g�������xYD�E�n���|q�c��+v�J��_)b���y��u��-J��'6�k��cR�w�L,���Cv=���+��
8:����J8��<<�p���qr ��0;2H�������~!�j�K]��/J���%=
{x4�J��6C���z��g�s*V���O���}�C%�� ?w~��Jue��@C�Zh( ���?X��,`�S�!�o��3���S/1��p�%���=w��C=��9���)�/wnRg��h���b:h1���3�]����!&�|z��YU�t���#�>CL����o ��5!�~b*+��'I*	�{�6C�z�#&sF�SP^������9��a=�th���m+m&&�� �0)�.)�������������������������������������qz�����rUe%W��\�R�����w���wY�������=�
�o������Z��G\K{�D�<j��+�J%��5&z�1���Z- Lz.Vs����Y���nHH�c��B�/>�����v,�&�k7�+�v�B����0_��(��93n���^`p��������-t�?�@�>���.lh�M��M��������`�Up"�Y���
�qb�������g�N����������{���\8�
D���o�0�-��hqNK5�,)�u���D��l97Dn��r�D�{p�H��&��&��&��&��&����`�{D���v
���U_�M��hmq��(wCQ�&��&��vm��}3��B�g�N�M<�Z��B���N�M��5���b�P�D��������&n�77/T�[�D����J/�����������zU�n*����{_����(�7��6���i��M��
��`�
����[[�=�-h�c3c3�y�a�����wOC���a�s�~:��Z��;Z]5'���pz<���(�[�>Y�"����k�q�J!��X�;Z5G���cz�1������������

$U
�nG���[����������#Z�C~.bU��?Jt�h���^���h����N�yv����([D��<�I��uF^��7�_���'�y>^,���22v���W�����������G�EL��:�@�#&�	h
�t��p��4x��V�����kD�P �Z,Xc�k�,{�%HI����v������0O���
 �����UV��*�J�e�99�p�2_(�����j�e	=c��St�]�X0~3��8�7�s���Dh:T�����w�zT�[�d]>��N�����(�[j�D�0���zt�ZA��_���� (��} ��1���A8�p���	9.C��g�PC�T)�Z�U,���C��.������Rs#,�E�P��LH��	g�]�!����Yhc���
hS��A�f�_�B��l������V��Z
J��@�w��f�b���}�@g���
;j���7�l�Y�g���Q[o6�z������!��,\��fa��7?o�Y�|����[oq��������}�-����0����4#��r������l�Q-3�A���l��9��j�k����fkm��}�Z����t��>��|��������)I������yVK�ZP]
I����-��;�sZG�^�G��I�������QF�m}X4���.
��fa��T<~����,�J��*�~y������*<����Xx�f�Q�GT
�������H'o���6�3�g;���jg:s����1�v�3��9��ig8s�����1�vv3�{����ig6s�����1�vV3�w�!N�V�*��l��f���tz�d��JW#$�s�����	����,3�s(4��f��r����Q������>]es�����3�a'��yE����l�i�]�,?�
���mv���V���wW�2���G6C�Q"�'��E�����:��E:Ydd��	S�6x�#���8R�?��N#�����M]�����j>8=�zN ���
���H�YiX��-d�Ss���<���i$���@,���Fr>t�/y��bL�U!9���_�$��>z��R@Dj
<�9e�.��x��S`����J
��k�}�q[*�>��]���h�r���@�:�\<q��YZ��k�k���M#z��nF����������Vj������Vj������Vj�������X1M1Hk��_4W������A+�cl�Y+�����s��;g�D_9��7��\��)[�h�K�KE����h:��NW�	H�����y�XPa�4Z��tM���(�����2�j���j�E�3Xn�#JPE6Zf����}��8n�>��N�/��	���?��g�	2���:���8Z�1�(��oW���id+����v�����t��{
E��������a�T�W�lU�����Z�e#�Z�$�B@�,��������p������I���A��b+��@��R�-�%�,����e��{$[)0��q \���6O�l��}�4������;�5���`�h����d��`j����y#%�����;�z�����T#\YI�^�F�c��qx��fFPFz�E�����TgiX��$�$�$�k�4���$�$�$�k��A)���$)F�NLU�Y^y!q�����7���*j��jJ3��������h��
JV6&?�[�cL�1�����S�����������T]���u*@=�1���bcL��)��S���l�+��PC!�N��X���h���7��t:�2(b�`�?l����H������}z5%w)�����P���b���[�d��d�f��b���rL��.��k�B�fG�����������/2�l,��w���r���P��?���h�V7�DU�w5#=���t��"+$�12�~~d.
�4	�����_���K)%@�R��_��~��Y�@�TS�D�V��5�@�&�uIK~�	��E�.�������{���������ezM����cz�r��Y�z�^��!b9�:�\��Q.�(�N4G��(���b����
rck������q��(Hx�%�9h�	(�ZH��Y�R&x�
�p�**�-��f���B���Z�������1j*���U���m\6���F �=��J�;����N�=��
�B6���x��J%��0
l����\�^g�)y�t:�ylW���e�3DK2u�����AN�t|y�	$V���L\��B�Fc���Yj�L�������^X�R5��r�2Y7CA�L�n�[��z������z��T(SV��*Sn��?���JN���)@m���}�Wq�����,6�Yl�g���b�>�]�,.�g�����>��}��g��B�����}V D�g.�P���>������1a�*����K��k��~�B%�,���J���-�����e�]�,s�����+:��:YC�������sO��ut'v��p�����[@(~Kw�W*�����+!D���!�U$������v!!��q����%v�edy�We�XA��[�uU&W�C5���0+,��9���X�B67W��].�w�y=+�W0��=���<�������o�r����*r)�U��.^���	����Sc�N���[wj���s���,�r��]o���he����S�g������i����9��1�j��\�>�V�Vb����BS��P��1:����U4%�������o���C�1In��$�����v�j���<|���944��h��b<Y�B�Z1���>��~�k���5���v��E96��
�S��s�0ger���W�2�������Z��d�%��[+���7��w&��]�W(�
���$�*I�:p���<���aOIO������.����,g�N��R�m�V�;��������2\Q����m�+H���,�:<q��9*8�l5�zz����V�����W�����H�Xg_]�m�?��67�\B��lv@Z��
D~}I���x�o������`���J��*zSK�`T�sT��{�����6�T�O5b�K�����&2WPw����;���
g9=3�@�g�����#*���7P�s��(�Y�4�z�3�����\ =�i����g�kq�����a�rv�+<sg�0��7(�Zx*O�0|��y���s;h0�.�����~�d�{�^����'
�6��AX���}�JJ�%�l�@M��p�tX%��M/
�������_P������I�()a��F�
����F�)�t����������&��3��&��I��a�(��G��>2C���l���`������7���k���CY�6-�rZ������ J�l�L�}L�w��N����w>}w�{N�f����$�"2qIB�"v�������KRx&�~����l-S�N/�:4�lc	�Q�(�/J���
�W��b�\�������O��k$	�&��DU����T��!4-�{u��,&l]�����u1a]ts�ZK�f_>)v��J{��=���_�<�W�/��uP�/���;���ZO�{u5�x�i���S��Z�%��c��a�%�J�:
���x����&[X���6���u��������,Q=�6?���1S������"+���#K����fq�$�C�
�<������@�QHP*��2A�j�]�����.X2'��|l]� W�+��A�re�+�U�l��������1�a����2K�������u��ut�XF�`����/�,�M���)(�����������������8h0R�4���`%�&��t9��.��N/.�\��h����K ?�T6����?���p����O����	$��^�&����=Qt�z�t
o�5�;��@�kQ����>�nH����W��Z���~M-j��l��\�bbK�c��	���P����l���v'�'�O��k	��A0��!��E	�����/.���(!��;����������#����+���&�g�t��V�r�������=�(OA���O������J7!T���c�m�=u���DuO�B�[8<����ct	��5:����`�[�g��ZH�=��Fy?�U�1���Q��ev�L�
���� Q��\�XUc��7����U5�����UU��yU�����*>V�X���W�X���[4���zU�P���kW����kWU}��U5���&����j�XUc���W�X����U5V���bU���j��U5�������fE���^���8
�O�NL�����j��E�V�,���)M����>�#�8���X�N��g(��o�x0���}��UG�S�-��R��'{�U��G�hy�����i�!��4�+�9\=�wB���M���T���S���&�tX��C��:P-�0�qz�{�V9�4���<P67kB�:]��pQ����LB��d{���O������e������]�l�4���4x.{=$@��I@���#�X�vJ�>r��L��fo�����=�0s�/�W���
�l�����M����t�q��@Tp	�����l��`�U2�UKQbe���[��V�����Z]���8����X�v�&A����2���5�sVQe���Q_�����FjbyL�HU.���[q��"�7Z�#��Zg�(�>���b����6�9�W7���}�+���J���)����5��Q���Wo9�U)�%�����r	�}��}����r�S?R��|��������7v
G���������,�
(�
8���7�s������T����v�go#��#��'��]�}C�}C���l�����"�
��7,�>�#U������r��7����x<�fil�9��Y�V=�Gr%������5�����|L�w����>�>?��<���Lk!���U�G�,�[�K�Q��&����)��]6{�=���
�5!�&D�Xv"kBdM{������&�tKL��@���tK�@�cdMKE�}�������	c��X17&������#<|����3�X����=��Q�����A���*b3�J�dEGT���>�qYU��2��t����r�����T����#���@��
f�=��}j,>UC���D�A�T�����ieku�y�%��@����/��:��%�h,|fF��NZ(�6X�{^�QP�i����4�baLc�X��4*�o��6�jr������+�M.]������'�&��~r��!�����E�6�� |�5�,�����jL��&W~�1���1?���n9�bq��O�f��OI��;*��\�3����=b�\�G+M.�������i���&�����'����>�QP�i�����1�c��+w�X�s���+w�X�sq?h'����}���������er����� |�5��v�Z�(��4�mr%���@",�����;W���q�J�;W������+�����V�\����!���J|�\jF�W�\�G���GA5�Q��+�4`��w�D������;W���������\�;������v�2C&���w�R3�����r��j����(��:q�o=h�}����G�A��J�G}�Ry�'���wN�_HeY�<�IE��������^���������,�}:l~�-3T���?��>N�������������/QqN(b���oJ&�B���NBK�c�,]�1o"v\��6����H� !�	�$�2�s �s ��@,������h�"~������e��	�c�i�K�>��y���W�����1�;�L�+a�^�}{~����?4A?LzE��&�X:2����U
�U���.w�3^��#��C�*Eh8�C�l��T!�p4D����&����?Or�"�?���9��9��Z�H.��ss��@Ph�R���DIT�/����l�=�/��y>���������������:�5���������'	���J���s���jHv��e���J��g����E+��l���V���
3�Xa�-##�_�ftEh|�a1���*F�0�n`-S
������P�����M}�����IW��7�X��L�1����Pe'������yB~���������_���8�W:#���sZ�b����F��\5�D�f��,�V��\A]���+������b�PR6��)���JW�?�P�����M}������IW��7�X����1�F���P�)��&����\s�����Co��Y'9�tF�i���QM������Y1W9++g������\+���%�&�7)������"��
�t�a����
_�t����1�pZ7���U��+]�s9���1���|�X���'��:+]���z���:c�y�3�H;��n@�p��t��n�������>��M�*m"1m�$% �S�����N1�^V���	
�/i:���W�_����t��qS���H���������Ju	�_�t���s���j�y��J'��k�v^���3��i�R.�+��r��[��"oj+��jy��"�HL��%I	'����l`�S����X����%M���*�B�q��x�%7u_K�q!\b�����'�*���/c���[��|�*�I�{���p��(���O�)J�y��	���(��QT�p`�"�J�8���]�_�X��,u8FK��y��	���(���]�p��"�K���j�r�������l�������~�	�K����P�+�������f@E>	��c!4��P�[*sS
��B��
����)��PX!Tw��V���@C����Bh�@en
�!�V�]!���27��."�B$�C��
�a�	�NXv���Dg�*[E_����|1?�of�t<����n��V��|M��j3������^��o��3��,L��U�,�����l"���-��.Q���_z�*I��6\+J�����w�������v��K��p"|�;XG`�F�n�����{Io���{'��;9.r�gE>�pQ�KS������~������x���q�[��xh�����GvJe�oCjc��,�`��B�����p"��=S��#M�UO�l�������c/�=�m,|w���<�f���1�=uO%��0�;����_��wg�c�5�*��-�0�6���f� a�Cn->1�r��u�����<N��F���=��
�jG:28��xt��t��3NN��qR��W'O�T�3'��f�8�1s����Q<N�s���$�58��G���R��cbk"kLT�fM��g3do����C��2���:x�����t}��5����cS�;6{ll�?�����w����]�\O���x�\�F�@�Xf���4�N��#t�����|�E�J�Cf����Wd��U�0�^-~MX��n�SY��"X���kUl�Ek��b���I3">16aM������[�{��i���2� ���F���,q�8�,:}B�>6��,`CYa��+K�R>�B��
t�hW���i��0m��:��[r��c!���AM��R\8�X���n�������O���z��et�(��)�j�Zvv�x=!��)����Fb��&X�c+0��aR5qjB��1I�$�0��1�	����aR5�yB��1*0��tr�����+���=����k��
Ch���G�����8�|�p�~�C��Al��4\�����eP���67u��q�#���h�y9W�%y�>9�#����a�P�fR���P����	!����
��.\~��P��Ze2���|��9zeR������X��+���������2�c�������j�_n������r�G�'���X��i<�X-���Y�`b��n��k�����t�W��H��I�F^���X�rb��������=b�r��V�r�F�M���[Nl�r���:��o����a��#�T����)�y��.v[],nu�j����.Vnu�n��~Pou�#��N�h���.Z�T���v��}nua��]�i���@01�U�LYlu�N���:�n�%�.�t&������Zq�%���/���u��^'�V{��E��x2���D�0��b9�<�X-�����@01[9��Pm�N��[/h=�bq��{]"�u�r�Kt{��v���u�G��:��N����a����<�X-�����@01�@7����{���%��-���)B;�4�#;OU����|�y~W�h��Y�\>������<U%���0��a�g��?��@�g��9/���k���������^m�$jOc��Jt�{	�2�jM��,A^���Kg�)�b���eW���r�^��$,��6����d����v�J�V��3x��m,��X5ocq���y+)��CU;o-�s�{�fT�����`42o�;�-HX��n6|i��u��,z���5��~D�y�����Tv�JCS��+0�0�����mc��K[�@9o�������hp�9��X���~���������e������#����3��0����k�y��
_���y��~k�Ut.�6��D��&�~�(�[��X��������e����3��0��m-��	�|�
��/ma��5��yRW��:o����8ocq���-��f��J�f����v�g���f6�����`42o�{�-HX��o6|i�/�����>�qV���o���"NB�����p��_�1~��\[�K��n��j|���E�--�e����Z��n��w�����m��W�1��W���RE����'�:d��rk6Um������u\��4-��`}�����zP=&d��T�3_��l�`v=/j���ir\C�k�q-#��{���l"	�Q�B�?�����1�����~�r�cS����V�~z��
e�>
Wg�����7=&}��T�m^�Rl���z.�
�M����4�G��l�p)[�e��Pjn�������c�y0+�.�������r��"�;c��������w0{]��c*}��3.g���vz�O$��z��
by���a)��]��'Yv*)�\���������|��L��*x�caJ���q1�����K���u'0x��p�3vq����v�w^n��x�����`
��h��<>Mg���J=<����hA����q�d����8���j�6�7�O��>��]eb3�k��!��{$��4z���G������p���@W=dL�	�:�<��7?���>w�3z���#�s�S���7�_�����T/�l��[,��@����f�z2�|�/-�~�z2�|���?c���\�C��d������{�b����N���������x�r���2 �B3JMZ��Jq�q���o��Zo��*��B���,��i��#|2�Q��>�h'�^<A����0� ��<�I��/���t������`}��-�X����`e�9���%�y�����������>�%�j���y���}�8	��<;���f��t��V`��&{p|���)��I������fG[�h��!��Z���0v*�Zz3���J�`���������f=&5�YRl�B����6��������C�	�s���c{�R.��; �����x4xw]bA�������� b�MS���S�����I����� &�&�������3����K�k���p���K�)���3�cO6h*��(���A�$��Z(�I=�}�&����1�'
��B�3��{��M�Q�or>j�F�����������o]
/bc�3�� �!����0�Q��AB�v��6��\k�^�~��7���zA���+����"������
���[�l��,���2��d���C��W�<	VAt���]�N�3�O��Q��M�@�������.]������E��j
���������s����Oo�qp����������~&����"������?"�t��c+�E+���1��2 �h��/�l�����O��86?���������7�����	G�o������f |�$|�H�XIx�
�c��J�������'Ib~d����s����3K�t~]/��E�� �=t�����t6]�&��
H8���c�������*�*�e�5X����[�����A
�E@����*�v �����������=X�;O�g�=�=�~�h�XGA�DYi��TW�3�P���ly�������JH���7�]���C��r���,���Ry�>+�`
B+PR���%aG�A���GL�R��a���
���T�P`�*��0��B?��7��.��@�9��i1�iqeN�8-��������������S��y
#U	�v=~�&Lp�n��{��w����1~�w�m���'�|������p���H#A~�����K���'���#��E(3�P"���	�@��0o!b��-t�\2����H�`7l����mWR���q�����6�^d��g�3���XI ��F���"���k����-�dGf�Y��{������)R����;�h�p~�$:��������c-��D�M{��M
+54R�?}|����DQi7�u��!(���Y���^?�\������$�_6�;�������~�����=��O�c��>��Lx&	����=T�QY���%�OH�{��n[-7*�.���\R��$���6�i=�e� ��NY����Y�������
�MB^�Fg�'�!�5o
�HL��ob��K���~z{xE�v���o�[	������S�������8j$����dB�;��}`��}(��`�'����d9�#��CE%��}�a��-�az����O��~�nG>LC�?���6�&z�L������Vx��b���#�.1+�Eh��H�]��N#�P�QzK�O��U���<���
����[W�-_��D=�v+	�|-�<�PY�����Ct�g�FI�-��h�,S�d%S���EY_F�Z4(��*-yUG~\��������[�R-D����[v��	^�%���������M�z�������v�/v~�\��V�,W4A����-�+� ��G�&�P�w�#�M�;_Q-��C�	c��������at��e)@	*�
�������	'�c[+=��+NGw�p�C��-~��G�sD��QIe���o�?����E��ARbrp`0��bn	12�{��FI�v���/�����B"�.����*M��*�je:-��S��I��1��Ki���x��c�2��}Uw�`W�����8.��NZw��/�Vn��{I�������H+��fMQ��b������VB������+
�Q��
*M���V�
@kb1�J��\�N_$�/;wPR�PY,QI�Z��1O��'v��D^����_+�;��Ua{�Z]���*�aM�U��j��
�T�y�7����D���o��C<O��eJ�30����X�ii4��i���U4/����?��tq�Vl�hh���G�^��E�OB#�W�o�w	�e�N%X��v#�p��P��q����)l�a����
����Y����` �����wH�?8�J#}��&����b��v�KV��T�	|�e�`���<�g%bC��R!#�V��d��d���D������dI�M���z�:�)'������V�o����2�����Xd��^�P��
�7����Pv���X�b��C?�sT#k8�O��bH�����h�������h����+�����g8%S���L)���B�mb�L��6Sbn�(
�"�f�<o�y��(W�)bIn��ay
�qH��{7CSc�����?�Y���LI
D���e�D9S�������)I���9��*3%1��d;3ER?3�|^�=45fJ��)�t��0�Y���o��c�z�O��3m��I���0E�r�&��4�IB7�#�������<�?�#�>EQ�i%�K�< �I";�#1m�����b��E�X�i%�j�c=q�'	�����3>*��m��������'1����������*��C�g��N�M3l����9�Xsl�9��������l���&�gKX)k{J�����zz7���,�L��,��sy��n�
:Yl����8)E�����@����b�y�%���	��(^X�{4��pl���fpJ��&|3`�'-`�H�P����5<�n�c�Non������S�+���sK��v�
���po<>�8lN�/z�=��}����FA��|��[�������Uv}���a�Qh_4%x��Jq�24�cHh�~��%-W
�(@��h���D�.P�GA�~P�]��
�Xy`yY�?��dUn�_7����7�r�u�M)W�j���QS������rw���j�M8|D�����T�r}Z���R�8��z
��e��=����&8�%��,�\Y�����X..5�O���)��������:��V��C�����T�s#��0�ASa������}�����]�3����9$@	�.�Z�x��9O"�� r�\�;��8�/�|�5@����r&��U/�]r�l@6���	l�i���[����'GR]�������KH+E���:v>%6�t�Ym�MuJl()6�i��1�A�3���:Zlo�����H�h�9oK��id�W�W���|��~�Z�r���31S�S)Au�����p��HM
QS��4��B�����Q��2��V�^�6f4���)DM��!%����,hD5{�N��m������9���f�|i��v�WeE���I��t��?��
�u���`�;��R��F���C�-��~UI��Q��RXT�Z�������X���+�M��7m|XE/��/�%���p
g�0R��T���m5�r{j��gc��"������K7�z}~�ne�.Y����4}V�=��2�f�b�V{��m<c%�R��G�g�x�+N����E9�b��M��tGG(�Z��#`��t���;�8p��	E���������@�5P�JsG8�6��#oAC<;ll�a#3��2Gld������4��Y:�n��ZU+�#-���3xRuI��m������/���U��.5,���/Xd�Y$=@���Q..�b�v�q$�}��s�Jsa*������R�l����-�d���n�����z���e������rbQ �!�Ta&�����V�X�ZJ�����*��X��V�+Y�d�ZE3����P�=CWD_"sS�#V+&�G�X)�V�A}VWrd�*Q&�J�k�K�seEy���n*V��EJ��(=�G+f?��](��L�P���%�U��a,���wX(f��e���O������r�UY�����\<up\��nG~�������+�p��G���(�����v�S���b�^*/�������z�%��,��*n)�.��D�����2��[
u��7Q-�y�A�3}7����	&�C���b�Y�3���J�-�S4c�C�qFM��c�,�	�,M�g��O��Q�4L��c����):7o�W�qn�Z/*����D��:�rWc�D��q/���c�� ���*�)��P*+<%V� �y)�E�~{�'�/�xc&Kmr!�b�y���G�]�d�M���P;������b����
�^@���Rc�l���L��pf�xOK����[aRl�c[����'�3�fR��J��B|�s��9.�*�5{��viU<.��W�������Wu&W_�l�;�������:�O�s�yV��Tc1.�p��1�*�3;�����q���e35e��V����K��d_.�+�5��UR����1�jy��	l�������]R[S�F7���N ;��b�����l���T����?��Z��-�>S�P���	�S�C@�������`k�ij{��<�U;V�]�������3<u�����s�akSi������1V�G�R���}8!���A��!Me���a5 ��vON$����}�B��r���>
�&G�h�zf�1�����u�q����|�6�x����Og������t��z�(X",m��%�m���`����A1���{����Y����.�$�9f!���sv�5��U[��U����1a��K������1O�GY��WNc�L��d�z�]/.��	�*F��uN/�N?��k�44��n,K�W/U�8
�n4�@�i�W-����^]A�A��$]e+T%q��,Z��	h�B���,��L�i�����e:����q=]O`.�����4���a���uJ�?/��c����4�_�W��=t(&j4��I����f0�\�K��b�B�Y����Q�N�����G�Q��Ge�k6��+��B���])��75�6���y��HR�i�������g��8��b�b�uJL�U\����|$�E&�S�=x#�?}G�:(QP%Ila
��c�N����yN�����Z������09d&-?���������L����>{N_���AS4�E��0�q(�����&}�P%?p2��
��2��l������^��2�28�L!�8�2LK�R��%�=�"C_uY(a���mv[�e���m�����F:���+sg�foVpy���<��k%V)��E�m���`��@�uZgKYB�Pv��^$k��"���g�Y*��zy���#IQ�p�{9�jQ�'����#~�������U6z�-���^qCg~����.���g�G�,��PTW��BC#��GC8d��U����x�f�t���=SU.%�t�-)^��q	���"d�
mz�G�5J��d+@���;�<Mo�Ma�G��Pi��},��i�U!v)��k�������OBm��s+!��GE	���;C���,7�C���H�$Y���B
%i��,
���;�#����i\&U��$A����dK�X�R�A,
b�t��TK�!n�X��BG�Ls��z3���X*����M]K�����-�Q�g��xZ�T8jN%V5#�ZW�'���"_����WW���|W����
v2�'��R��r�������*P`:U���?oXJ��������]"��#�Nb���BRl�G��No�z���UR%����{�����W9��jw��SK��R;�7?��H�H�h(=3�k����f���`[RG�*Fd_����&P�*��2)�\T�%�1�o1'����'L�8f�����)��X���nvn������#���* [�X=��F������5�I�4���Ak/���`+�V�+m�`���3��[E�+#�:O�QgM�s8�����$:��}������L+W*#����w�]����Gc��3{�E�����
�o�����M�ins��������|{y�0�����	�ca�kl�����%����0����������o��pVK��&G!`����b��K<���Kj������A4�����cM\��.���CM�-�8�MM�!�4���m���6�t~.>>�=\F>*���0������Cq��U�p����*(Z�9�c�5�a��u%�	�nSp�
V�<
�e����K�k�������y�B�c�DcG��T/��?C��e��Qj��e;F�}^�g;�{x�U1��"�����
u��S$�G_[��+�Z(�u��^���.�6q��[��q������j�������r��i��c��&'�^��'zag*�|vy^&����_C�t�!:���&h���G^-cbV���	�����h�+���O��#���4V�������#n�5�jj�0�����6FD*���[��=�Kr�1(���$�t��a��%���O�q1g��Rf����K`��&]��&�K��O�>\�h���iF|"�A`�h�6�4j��#o/�����T�� ��N���<�U�WE
j(H1�x��R�Su���g��p��x� ���^+!���*;�N�	�W�����.�}te��� Y�;�m3�	_R)h���m4��b�+���/O�����	4���J�vj���}�0���La�=����"M���p���Db���k�-Q2��b�����-.�9[b]-�ut66��l��[	�m�[�j��k^s=�*��YS4�h'����b;�6��hgx$�3����GB;B���zU�[U1|A�������Xt��HQ��������tY��T�2�%+���/���*��&k���6y��=���m��n���>���S2F�Nea[���d�
"4#.����1#��������c���"y�B*��F��^�.����
^]?V}^����
Am�-�
~��-H]�J�����w��<0b}�f�%E����	��;����y�*T���?HM*x`@��_UjY�\[�f*m�c�����B�l����R|4�V�PV#1��f�t�K(UV�$�"���������da[�����<]�&$��E����� �?�Xe>7
�Vb�}�Kng��������un�:Z����������X�<��@��I�i����e�5z�MN~�����l��4df����^�|��C�����OK����<����*��>�g�0��\X������"�{�c%�C!�Hq
*�
��2gJ��)�U)a�P^HLG����/�D�2��D=g�+d�����1=\��N�k�4�X����x{A�B�k ���o��^���.��S7�<U�S�.k(��J�rP���n����<U���U�����e�pM��������T
��Q}��*��D��|V�gT0�
>�}��R��,|��+A�k��e������
���y�_��<XA	��2x��(���V�O{��j����u���I�a5����EPp�����'(8~��WPp������XL|��
C�����5��t�9�2����=���:Z]Dw�r=]O��t<������!�:�<��/�`#��Sr_1*�Z+2�JLF���nLD|e�9���f�F��{�[S� s����	(Y7��*��$t�*DV�^Y/�*���y��c;�qA&��6#3#�w����H���4�fA�(�:��}�����#�d��i3�	3��p�R��j$K�rI��(|9���|�#���X2�Jb[��R*`�`������$��P�Q�g������${f=c���i!���%���n��8����j5lv�}���~g3f�~���d�k�>o���B9��/{������7x�Y�-����Q������6���an�*e~�����m���H?�G��9{���E�S��v@������o��\�rG�%�s�K�2A��������0�7Lf��T
4��9����+B�Mg��q^h�l����FD��7�lD'��	�#K���y�%�w�\�f�!���:.�B��ET�\%zR9��]�����1�5�p�|�2�{�b�����f�w�aarXoJ��"T�vWZwJ�m|�D�Z���N�����9�e'=K�K\����(V�@��t03|�ZTp�'������
����@��}�e�C,�+PLW������e2B�Pm��_m*S-��SRk��/�����rR�����,�/q�YJW�D�!������jP���d�_���������{k������[2��"O��o�\`0Z������vm�B�vz+�+=���Z������l�3�i�5�P����y-�
����J]sd�R��r����(d��bVJ���UC���[htZ������e2�7i�Y�w�������y�;xQ�����1��f�w�l�2�
	��o`~�F���
�+�����,Z�e�t�X:����Y��$�d������&��'G�r�����e����
]�����hd�Q��g�(mf#t�^��o
��(�:��0$�/�5���U��p�����t���`��_����e��z�
�;�
�Nb`-�D�s
��hUY����������b�G��
Z1��k�,VY"*����q(Z��9(��?|�c[�q,��>�g�[$���]	�~
O����{N.F[�������o��.�����D�KQ���k���`��>�xv�|���S	�n��)u9���X��M�W���77��{A�r�"�;���I�Z�6_d��L�����=IU?f��(S��RpT��);���B�vb���������5����_Y���U9���K@���-�5��Z@��0�&|��U�8!^�*�>�z	��������b$���/s���_H�����O$��$����%��3�r2�!��e�C,K����]����Kz@�f�t�L�1�e������o��_�S}���?�����O,���(FO���:�bFv�\�*.�H��������w�_��lo��������	d����5Wu��Q�w6�Os�&��gf
�Y����Xz�3[�����@E7�����o������9�gO�^mn�,�%��v"%�&*��kst�O�6�c�XIl��p��R���f����Le�F��o2O�W���_���#sE�w�����{�f�`�V�����i{?����]��[�����b�F������`��k�N���v�iH���b.+���,+�B��[���%J@��Q��w��.���@<���k��o����e���t��>Fbc�0'��r�]q^�[�1~&�!:�����K������� gm���x�mXW�mW}Hq��2}n����"k�e��c�d�F�u[�4�Z�,w4�����o�~8;E=��w���2Z}�{�d�nV��!Y�[]�O8���E��W���$���J���za7�����X�/R�a�"3YO��?P��}���8�����)�C��Rl��2��u^�d4BU.8��il	{��8]�+��S���[��]e��U��]��c(��:���qD�aY"���e��\�LOX\Y��Y-��U��Y��T(����Yx�����F���UE4*����*�U2|������g5.?6!P{:��.���������*]�L���k��"(A�(��x"(FA�P#��@0�c� GC	��l�+Xp���_��	�go�{��z�}���A*
r\)f[�7������_���o1)A��!}���KuZ������zg~��������������{]�_~�x�0�5��7����N�O��#�?���*zn����$D.^����{��&	M��E������>��w�E@�����/�����f���e���>�a��U��]�5�e�F@l�����?�����K��~S�.�@{^FW��t��L��p�^���l#b��u��PN_(���&$��=��C�$�_+	�y�����Oo�_�y�h��{�Yd+I�\���~NKC�K��/.<�Bz(�����zq$z[�����2K�����h���9e6]�������)|~'�59�4]�����~z)=������ �x%g��1)z�z���������X��s��'��X��h1��c��|�{_�h�A[� ��Z�Q������Z��/��� K�J4p�{���6�7�b5�cT����P��`%n�|mV�����4�M�;���@��iZ�]��k����O�R���s�oj��V��\������J��'��~[[��H0��L?��4�"���
�~IN���	�%�_��;�/�;�#�K_����}�{��z�4�Xl�`�F!�T7gX>z�#� j��^���C	|�4������"9E��4�3��{��~�CTP�R��nR��|I�@���g��`������������^����.T&e�X��=�����U�#8�Mb�E����E\Kpt��8!�f�@���r���@��q�������~���e%�DB"`:�~?w��8cFXc�����/��"a���?�u��p�]���f�"�II?(���Ff�� ?�W��J�����ll[v o]~����'��'?����$���|n�(%���� La*SA�
��xa*S���}���"N]of��ISH���y`�q	�T�v'H���/bT���K���W��_j�"�	���t*D^)(!K��Py� E8���I��?>��J�?��5���������[��������t��|p���Cc�������s.=i��7�t�5d�mSYM �
��T`���s�L�nj- ���8�������yV���3�w&��U"��������`�MH�����}9�m�[�+{Pf�*�VW�s����j�DK��8/����-�[�P�R:���[=�:s�e�%���W|�79��_�8�^D�QQF�0����i��c��0}���j�����~�����+}�/��F��32Jcf�5�(�
26���=������������$������Io�W�}�a>�������0%2�hR7��1��(~�e���!��6�q�f'�����s�ps�Y��^��������w{\�q��f��
 �#lT�&GZ9k��\������~B�	�gGu 1Y��M4�?;�C���~BY��v	R���w���'�f��%<v�66�'�za�  ���l9_��+_��4B[��S8�>�E�^�~��J�+T
�Pq�7�tc�oK�H���Q���8e��Q���FQ�g1�IU���=�F������=zueK�`��kw� ���� bo&����q��J�qKBa-��7�������'6�y�jo[q�v�Vjgi��ZYi_
+�K]%��e(o��u�Q�!��+K�*����!�h�G:����6\����hD��/u{�Fu��^���u\��P����2�=w��Z��"�,u�
q@�m�{�yt��]c�������u������4��d��
���s�>qXH�v���b�#s�]v��k9�%���f�,+Z��H��:��+7o6��2�� iz0�gv��dw��[���(�v
m��8�f��&5x�F�9���X�[��0>�MC�E/a|�L05�/j�j�Oqr��������MA��.�8U����A����s�m�l�`�����VD_�c��#G�t��\B �A_� �Y�F1�v
���vcdEj����2�L���<%�i�p�:[���"&E�+j��S�Z��m��������%��e|��uj��e|	�6f������2�( tH�S1^�P�D8��iO�b/SFy
`�%n9p�{�[���%'����=�-)�b��������S�-�[9=�����\�����,�\@�1Oa��o&�
�RTZ��������{~���kWZ}��P��b�f�lEU� jQ;��(���h(����Q;�����w#^����Uy�_Q�8��-U�kF�V��bw�����2������]
n���=>�{G��~H{[��=�o~�CQ�b��f�nE�� v�;��(���h(�����;�����w#^����U��b���18����S��9�*d2x8������R��P��Q���Q�`��x��u���vg89��g��)��:��^?���z+���D��
-5�I��CG����=t�
��CG����:���!��6)�U�_���JC!26��O+�
Am���PeP�����6���#Tv"�<$�s�zC���0~���@������jCP��6�C�-�Am��9�
Am0����]-I����P1�(>�]���A��k������A���3(U�b��@B^P ���U �-�<$9�15B�<[Q&pCXOH�wP"��]A�JaP"����D�����]��!	�-���j�����WCz��M�O�!�����`gP���^Ae*���2����2�O\yH�fc�3lAQh`����N"�2���cr%����"�K��)<@7Z��Y���w�y�J��D����������b�������7��������CW���E/��q��I/"y .@�u�J�Y*$^@��L�M�!����w�lP%H��\���6v+ ����f{�����[�@&�6�Q��^~3i�LX�[���~�
��F�U�y����l{=1�C�^������\�v"����K�i+�@Ql��N��n�/*y�
$�\�1�@��TDE�A."���Jv������Q����$,�|���B��,	Xp@U��������R��"c��RA��{������FW�"������\��+�,�J������b]�X��XT�%�U;�j����W2P����/MP���o`bxI�1�GF�K�B�x�����(kK�?#�AJ��P.��F	��J������"����qA��:�8��MP�E(�`��H���C�����0N�H;=�\�v[��E����2�v�{;�-���n|�%��r������C��=8�+�^��v8X��E,��2&.!v�XH���(p�8��v���N*/�\4(��<�-
l�8.p��-
m�8�.pM:vC
Yq�l��sB��*f=^�XA���3P
T������
��!f�����A5����X�nubw�m����nu�6WdsE��n�+���[]���\��a�_n0j�^���y��b����J�,Se5/U��k{���������+��I��r,������Dt�ydhw�]��P��7��F��h�(��+�^���r�������Jf���Br� ����7%����%V{�@-����������4w����o�\��SM	��>���9�SM	��>�E<�3�x�����j����7 N�F���
�+�����,Z�e�t�X:ePB�I.�,
�2�Hu�	�����=���=�V1�N����R�$�������3'U����u���c�����������k��At���,o#�fB�_�G&�Guaw���PY���{���6R� I��)s�8:=\��N��
�
�����i�N��r��~v��M.���L����;���4��
g?�1"�2��^&�|<{~v��c W�w���R��p����������/>�s*$_-�k�l

F]����l���
�d�m:�Z|=��)�I�v�f��*]���\�U�)3(w3�@��.��o!06�M|�1���)����;���cC`L7I
�a/�����D����������`P#q���`C���:�f�R��`)K��
�XB�H{���&�!����@C�v;v&�q�1f��t�L�1�M�^���g��~=�8�#�*:}�������?��}�B�����
��0�d��k��`��Rn������3�h��x�]_��������DJ_%K�3�H�.���r���v�Q����6Py���l�y[Pa�@S�
�<���F
���a�@�PU�����f
�uc��J�����d�*
�7�KY��#�6-�J�`f��Y?|8��Y��O�X���~�e�O���Z�o������r������%2��#�
�*���������{�\l� ���@i[=��js�6��I�%���EE���b,��m�`ct��h��K�/��77��&]�V0���M�	�jaIf~�W;����%lo��b��W{�����}:���}Zi��q����D��%*���	�J���Q!6aI�T�R�h�L��[����%��C'g!��-��`�Z�U�6��0�]��v
$��?��9�
�Q'���YD��Dd���"���(7����$������%�%��Y&tV�5A���MV�+c����)j�#�����������t��2Z�*�U�|f�X7���Q��]�)
�]���>���X������`<�:Ar$H^�� 
��@�Hn�Q���@r����$�p;9��������N�/���z��V���1���Uf�^Pm���9�P�������`?k����b���V�T��W��+��{�@_bz7������7	\\�m|��4f�m}]�&[��R��2[��k����QC�U�{79��o�C������G��k����/�N]t���[�5�I&��Z�xtG�j��W���*�<*hwg[��i,I���O��R���*]�_����k �9/�yA�r^��J(�� ��;g{e������y��d�\�!?�����Kh�;{�@�������G��^�0B��Rc?�_����}�
R���14���QOKg����_��(J��t��]�C�����]����2��^w����O�?���m�}���XZ�#�����dD~|=7acO^��+^����{��-��uH^Fh�0���.��/�N9�������,����6L��:�KF-�hP"�����Hg��{s	�����L�sP���Uv=�����$������&A��t��!�.e@����"2l�����w�!���r��7��}�<����K6F4W>��q�(�����T�2c�'�8������c�/a��_��l�9��	�+�I���.���c�.����.�<�����g�_�G����XP�C�!'�,�;o�N�U�F����l=�e_��!z��#�y�����EwK ��pu�tEhA4��K����F�z��~1gK���+2'z�a�������{J��A��O���~���q�Z�~��u�Sr������CK�	Do����Z�����t[u�Kn*�=�E������,fi�L	`d�=����+�e��L�u��Lqm����|QU�0K���Rm`har������d��^nV��GCp���j��e�h�p��-���%$((P����@9��&/4���Se���H��������X���0��lU�����3���@�o6�*a"]Lt+~~��?`�g3���~�1e�TC�	ca;�@]yd����n���mA��@'�����6q��:�}�����Q�J{��V+�
-���ThW{y��[���B��#.���9A�!C1��h*�>jRF�*peU��`��V�,��
������C}�0NJs/.������>;�oG��e��
dlRKl�~��Wd*a�'�T�)���%�s���&���4K@K�F O�������	I,LH�Vb��U�[�3!��	��J����[b+P&�P���[����a��%�rbB��H��d��|���d3 Q/��VE��m�v:���Z�!�R�$@����*���&%R�G'����$R��������O:J�<����[zl�Q�����H����a�g��e�R�q$��$�C��#�.����,�^G��zgJL1�(�
���F������D��Q"_jJ�M_f���H���������'��I�z�#_D�e\�T�21�Yj-�����%��N����SD,`w��[kE�����js#�Ys�������H�������u�_F�������+S[�����]��O;�_������A�S�sR�olu����{i�J���234��������1R
�����$�\�.Zt��Y�
��~���0��������=E����|Jl~�+��I�'jk��L���<�|.ue�z
z�9�c���y��Hz�&��*F�K����6����	����/��
)Z:s�+Z���G�:BDcaV)������U���2��=�Ht��v�N<��D[�l]������hoa]��3�l2C�&�:�L_�\�������|���FV�`3���L�)o�q�M0��M~�M0�X��7W0����7�x�7�x�7{m�y�ju�������l������L�Hd�=�#/o6�j�HU7
�?�Q�h-����vn_i_�����O+	�7�X%��/_���?P~2Q�u��ZG��J��!@^���}�G;��'H�U��`5�d�yN��K�A�����W�h����~��/�����-f83qE��_[�'��U�eE�.,In�%����;�Y�V�-Q�<_+�L��S�A� ���_o+q����Y���=��j�U�k�$�d�:�|��|�4������2^
���A��i���m|j������f�u���h����������Z��5�V�QkQ���W=^�IUK����C��&w$����b��XjG�|K�gR;Rv��	��vK^4O��e:�R��mKWG1*��,gCUs6(��Hk3��E�v"��(730�����A��T��Wf]��(�����"��X^_r~��:�����Oog�S�����:�������c$�T��Z5��pZCu<}��5�J��:��������2@���@3��8<u����h��n��[�U���Iv?]�WQ���gK8���
���k���W��v�6
�)��4^�>#�^d������C�;�a?B�5��n4��������6l�A��l�o7dvb�);�RE�7W���x��/x��:aM��0!#[2aL��]��`j66_`B��;���W=&l������k�5�������c����y�@��wC8�����h{��������w���������j��:�����[m�x����?��u+�������[�aN!���l*C�NmU=r.g^Y���5�J:��6Z_o�����*���J"��XZ_�$~�����H/��-���Q���K���Z��+	���R!\�s�O��N0�AI��C���
��?;4l*f�q�T���AR�g��D
�H�z>��\���^�m;��|]I9����ac'0��D�w���U2<�2>�:�����M�w�����b�������Bg[���Zl��/H��P�?>N���q�'73r�1�'731���a��8�O���_�k E]����������������&��s�h��q�t`���D���H�]�&����
���S��?_u2;'0���y�q�F����$`N&���C���*��a����	+THI
����6����
',P!!U
�
F�c����.	�����m��Q|��)O�	�U�~�U.�`����b9��X��p�)UK��Z-
{�Y�2�f��W!%��<� �6�2"��2e�	�>u�(H�*
O���<]O��9Y.V+$�3��R��,p�������3�����V+�<.
�����+��yq	vf}����?�*�m���VFT��������b�Zt�0�������Q�����<�:�'�J�Ih2O���/�����	C���hv�[�CB����II@H��D�$	Z���$�>�%AK��[M�"=����	1�:!&�J�IhBL�����-!&!�f����j����)�%!
gB���t	I)v7������k��5h�J�lP�!����y��;q�����@��#w������S��	�Jg/����=w������YFj�+����\���i����X�X8�����v���������F4���>[l�|�Q2���R����)U<��9{�����0H?XU=_�������VKK�k��
V��W����������AW?�/���33Vb[fTUOJ�����I	���4W=������^UA�/���33�q0�ZFc�h\f{r��5Vcf�l��F(���eM��I���x@��0�
��y�1��]�������Ysc���w�wj�GhwF�}���*���%Uu|�l���`I
���z������c�SC|�'��+��F!>U�X05 Sc��S��1�PC��V����}$�"���)�G�Um3&V����,�bO-��{���5��k�C���XU��bUC�j�U
��"�����`g��L���l�2���P�"Vu@�
���W��Z6D�:CcI�[(�������U�*��c-�$S�J�|&Y%SpnCf�`�
�s�
s�`%V��V�&����0�������	�1gU6�`C��l��89�3k	g8g��p�����=�������r�k_�ZM�h�����@[�{�x����^�ma�n�V�6���R�b�c�n����5;�cW�+F��	��-���Y�f#��j�S����{������|*�|M%�G��M��n������F;��nh��j�������j���Ise3�����S��S���b��o��d�q1��7�T2��q��o*n�&����&���}�f�ml��Fe�i$�{sw7��!h�����'-
H�����C��r�6����X�C�bZl���
-������"-��E{�C�bZ�++Z0���`�.��	�l.rQ�X�X���H-�!|Qx�F��X���h�y_4\!|q?��nh��Ew�R_,-�/��E#������5�A�*%42����H4���2���s���C0cf��!�13�`���
��_!�13�`��X�XftFC���
@?�`�`D�po#,�������L���G;q=]������������������mh�6 �
��`���`�.C,X��� X�����#�~?r���4|3cl��U���u��}�f_W�"��:Z]Dw�r=]O���x�E����C�u��c`��6��s�_�3�j���R+������l�Ke��e/���B��X0����2��w�/N!e��4E�K�=����^<�j������6�����o���q����nK�|	�/ji��Hg�u,��5`(KFd`�T��Fj	C"�!��T�CbO�D�C"�!�P�����2X��`u�A+��x�U��L[��D'b�X�D3����>����|b{��o��v��O"�O"��f���I���a�����>������M�Z�t�d��jX���NA��J���T����Z����u5�?�^]�G�X*��!1����N�_�
�����_5�&��A�o�L�9���9�@����$9�QL����2�OP8��eu�(-�����K��L���w��&+�cq
I�������<"�`�U�Mc%��JB//�����i�6Gz
���������q
Q+x�U��%]c[x-W$r_:y�!����<�
t�)���Bj��en����=F,�H�[�������T(|S�$5UU�i��*�P[�;����t~L:?�]]�?:����t~B:?�]]�?zn��s��s����������� �� ���Q]�N��ZZ��T��\U [��!|guHF�N)r@�B5��pU�hX�I�h�)K�*�
+��
�IB�U�R�!�R�Y����b����L�%VT��$�P!u!5E�27Y��F,�+�w�P"�6[1fd�B�]�[�r�>��*%0��S����+J�Rd��Hz��y��
T3�A�"g�y���PPv7�� ��c�^�v1�Y��9!��@���- t�ej	�����d���Y��yAyAARv�����~?,_�7Bj=�k���<����5�R��etl�YQ��/��J/���]j	_�j��/�Ga��X;��&�`+ceA�PXU�xo�x��h-2������hX_(t���"���\g����V��U;��wCo��7������-v`	{3� �`	{��- v�a	{�� Fb	Sq�}*��]�_��	���%����S�i�v.���YD���YF�V��8tG
!lm���f�V ��p1�p��.�k	�j&l�	E8�N�A���-!\���@x@pD��C}��^�����1-P�ov��V+����J���Zg���f>�*�Jx|6�=�w�2]/�N�=\��B���x�q
�F�?9���'�^#
��~�j�I�'��:�~M��mz�5|���d���]���sz��?xA������v,>�"f)b�*����	��
��"q9jvbs����[��, k���PcK�lWH�_��'�FI�M �B+Y��������o��.%Q���D�KQ���k���`��>�xv�|������G�_C[|�v�H/�r#���n�M�6�_-����X�&�Hh_���<�1�[s�a�8��#��C|�T�q�g�A��H��������b�5K��+^[�lmp�(�j������
����[<#����Z�>���G�p�eeAO�Ii*u�i.K
Z�����Hd������E��7V6Zf-�h1�g�\�����.Z��;�<������r��������'�v�����G�u\`������8���"��h.���Z�����N#j��V�i���V�i�O�VKi��J��$�j
�AIH{��2���U��3��p�\l� \���@z�09��js�������iU.Gu���
 ���H/��R��s�SR�Y)�+�/pDNon��M��T�z5�y�Z��������V�t���<���c�H���
�-5 ��[eh��b���B�A��"�����H{�������	��@}�[�������&h���-��@�Z	�6��A��]��v
���?��9�?F�a��Q�����������r�!��I5���~~��!�_p�B������7�{c�0xo��&xo�@?�
�E��y����R�b,���O���pv�zz�]��e�����,��2��r;�������?\DrcM���yL��o��/�d�H%)��z=�B�@%�KL����\3��P�U|)d>|9��P�|�����"�e,���F���!�u��,�R�.�����`���r���G	f*p�fz�Oy�4{����?m��9�1���9���E�&<��$�����3&������>��Mh$>�8Uxp��5���U�~p*���^�5�p�
:\��tW����V���g@*�pA�3!t�}����Jb,���r+?�t����/����-�{���Dg�.>`�V���(�KA9����g�F�o���p7F����_r��U�;O�.�����O��.�|x]_���������c^cZ�P����)]���y���gB�_EO�M����d��W>�v��}��CE��$���/�}J����������_�����������C	~9�a���-���n`�p�
�By5�3g��{s	'������3�f��*����j��l�nf�t~��`��t��� E�k��2�0�����}����V:z����������K� ���9g�=�,�A��(��B�������P��\�R�aq�8#'8n�7���2K�.���h���]�v3[Og��lv8�����~�������B����F�~O��K����F�z�[�s��qg��hV��f��q�����	�����9 �\�V�?��&�X��h�����>a]�1*n����M�vG�����$7�Q
FQSE~�����������0�z_�b]	{��:���AI��k�+�b��V�- ���*��A���k[�������U����Q��44��4m`��
�Mw2��4<+��Z� T��f�����gRc�����{F���L�����E�jie\�
n
+|���f_�8n�/�RS~��=��'h����)���?hm�'n2��;@;���'lUb��$b
��/�����xttT(!���Kr2�_������2��~���_�R?�/�;�c�+H��Q��&���jS��q��G|$;U�x.� ���{B;#>��v����X�,�;+7@��l����:���8����x��X4������n*��m|�g�{�����>����g(;�o�HY��/�`��l$�1�����Q.I>�)�I>�s��X�wR���0�8��R,eOB�1���pY;�+p�k*A����"��*.4X�& v���������X( ��Arjb�CH�0�
����z �(	����5c ��5	n�"Hg�-�[7��{����4x��N�u3!
?�7n�ZH�Z�8�J�����+!����T�P���%�.�Gp��;	�G���[���A
n����q,��z-6�V��P����a�����!=���C+6]~4f��\{F��������"�Q��HAk�5�����4H�������hfV~0���W%(���$�f5�v�Q��h��SC��������*���������
5�s��l�H�*��=���H��#5�<�y��o�l�����5�V�u@��Z]����+��.����>�Y��+�(��_\�G�m�}V���&`�v���s����i+�M@��Z������s���"�'������	��������K������-�;�������2	6��U��������/r��r����������z��s������E��;���z����67�\,�2���B�9��K��!��!��C��C���"���?,�~�#>���+N
�;��{���rd4�����~egG�@o����\�V�&w�	�
�G��1�c��$��po�|�C��RWK��}��=��}��X��!�����������Q���Y��-�ms��P5?��[4�<^!�17����hF,7�����!�17*@��!��������<�@���l�:=X�y�;l�5v�@G����C�ct�������z	��Qt��!�1:�@��C�c�C�cN����::>�@�h��7�t���<����im�����9o���V�����m�����Y8���c{��h��ba�����7�9���3m�UHG�����(����b��]@*�b�:����J�*V}F���j�X����j64&�f�j��l
�����C��t�DXU�_W��\l���/�LD��K.������XEs%w���Z����*��V���;yS(������K^���n	t������e��5s5����OF}g��-
M��q.��f�6)8cI*�@������R�F� �G��)��s=�o5Uz��zmH�9��jd�$TL%-��u��tT`;Ff�d=�3�.�}z�aA1&����\XFt��iT��;��z��%S2�����f�� = UX���lY4�,���4|%q����UtQ(�z_-kh�c�Y6�Q��hajJ���~�B��= UX���mY��93q#�}z�eQ1ZM��X�1��Q�������ha�L���>����R�E��-�(�XPp,:YT�����X�1��Qk��Z����ha�N�Y�~@�= UX��u�������e�!e�!��C�E�q���E�2�
�����mP����:B��h�0��������V'6�a�s�M��g�G�BA�������o-e���#��n��(�[{7�����z3W0���vXO�<x�����3�*Yc��j�u�:����NF�+��i�b���}��k�����Q���h�p<|�����%����_�&K4���j�h4b��by�����ik����aN��<���<x���E�%{��-_�=4��Pj�{h4b��b�����X1k�����JJm��T�^�_�����T����6�g�f����6�vh*L����/�g����������x�UK�~����Q�e���j4I/JWpU��z�K<:�d��I�-t'~�%��l�&ljm��,e��j���IW�6��r[V��j������v/�9��Km��6�J5��S �%9X<k^cAFQ���3��������vf�e�8�?@���5��p@�m���(*�=��tQIPg�-�
��?�"B,�U5����DG[�����m��k^#~$#���N�@��</���s����t��+����yj:�o�j�AD �&=���EfP��4g�v�q/��|���89��k?���O���{��,�D_�O2w�V{����S�9�U����*t��\?�CI$���#T�S�����Z���hN�����o�NK��������j��;�8�4w����;-qaqJf��"S��5'FM+�^jO��[�W8�T[��p� 
t�U����9��z�s��V~yNE�^�\U���#�S���0W��`�4�T'�"�U( �%�IJU.�������\���U/:rQ��\��^c9�hl�p��W�?�����v<����b��7'��j��-����S���
s��8[,����Pa��	������qpS���U�1,9�)a<�G�'�q�v�K����`� e:+�r�l�����_�VU�z��!������	��*aF��U�./M����3;{s���b~B���B������4�,��I�x��h�	a������k�/��u#�Lb�g�����'l�~dN���K��$��I4�4��5X�C��}�h4U��
k/��"GlY	��[]�0o$UB��ag/ni�X�b��UY���/!���:����-�����@�7a����.zv��Eo�-zq��s*L�y��E�������w@�[a74D���
���L������%d�����;{qK���E/!�^�|�����<Ii|����2~aa�Z������-e��|���a,��m��R��J;a��rg�P��i�T����n�i���=��z�}>0�=xK/
>+��K���#�w���)5��n�@�W>�Xl�������`4
�,�^�w5�{��Ai�
���+8C�3DO��	���)�+8C�3�.��!�gHp�gHp�g�f�R���1B�!�b��)g�~�{�GD����E����[��H]���4t�H�#�1�'dp��Hp���#�1R��	���c$8F�c$8F�c�
��)�A�sp�����1@��cd����3"
Ip��]#���l;��������)��c$8F�����	��r��c$8F��#�1��Yp��Hp��Hp�Xa#�0hp�#��1#(�r�����!�E�	nqHp���
im���'l��6|-
�W���()*�FW����)�P�re���D��JD4����-o2��G!��G����?����G���fD!�%n^-GR�{���2��$�=r\�1�E���d��7w����Z+1c��������Ae�'>'�NG$F^���i$4b�s��+�g4,�	�������F�
;�HBc@�`NuEc�
�IBcH�`�3uEc�C�|Uv�^��aZ��#Ki���������BD��F��\��p}��|2�}2���z��u(����k*F�8F�("M*���;�����i����j�[�@z����L��
������R����I�h��6;S2�2�:s':���������s�;�D��\�=��8����vF�m���jL������BM�\v������=���=�e���=���]�F0�T�w�����E#���=���](���}�mm�6�>������7��F
ms5�m�I0���
��`x��`x����`x�w
��.P9�uW0����`x��`x�w�`xX��}6�����P��;�&��L��4���K	��`�o�
&�`�&�`���&�`�&x
�/P9��uW0���`�&�`�&x�`�o�n�d�m�������
�9��P��/��[�]���������J4�>���`�f�`�f�`���f�`�f�`�f��hi����A���w`��Y�q'a)F�f�`tF�`tFw�`tF�`t�������]w�{0�k�F�`tF�`t�
F�������v[k��vf���������_�����i�u�!*.����Et�.���r��gY�����?�Y+3<$���M�gW�l"RU�"�4�)v���Uo���=[����l�V�^�=^`|s��8YiJ�����}��X��<����[�
�:���_��g�M~��A��|�`F��s=��Tn��Ad\<,J���]��C���
�/���N�(1�T`�*�9a@����b},@��
J�KL�v��%��@�X�}�F���c{���c����c�O�i��O�'n�O<�>��}"�>h���>1��nZ����^�o4������'P�Dd)���Y�]��|b�2��+����	�X�X����'�6>�ER�O,����O�m|���D�D����'�6>��T�O,����O"�Owp,Y���*Z��3B��%�/FSE��h���
�����3�L�g@���3v�Zf�}��&�mY3��Be��C�j��qTeV/0����
����F�v��Cn���~s�x��r��V)��%��~�Z�H5����i���\"��G�U�0s�*�IV)z�>b,���;�������c����x�kU��d��{����|!=�}��K�r4H��iF�~��w��w$�jtL�R���xl��C/�r���B�l9s������������=�iy���������B��!V}&w/��S��y	,�v��2/Q ��Q�:��LE�����o,�Y����l����f)��9
_������f��!Q)����"��x�W��'����8�c��""T1FF�%�1�|	�jY1
�1�	�jy2
��els�����9�@�!�1�,	�j�4
�56��j/�Y�i��t�&�w5Je��`�Wa!�Gwq,t���sR���	A��*b'��?�� �� �� ��*b���^TA�A�A�EU�^4����*�BGT:���Qeq���M�_M��E�\��.�i��m:WI���:W'���^���:Y��\�P'k{���dm�suB�������
�����@y�[�d(9����p��
�	��tR�I���B�`zNzNz^���)�}�l��������uS���=�]v���{D��2��;��������*�
��@��P�[A���6����^��N�T7��<�����Z���
������x}�����;�X'c�1��t�8[��E)MA��}-��b	M�WDS�I*�Y��U�L��lA�(�c)M���+����&�hycZa�������
FK�%6�L����|������F4l��4�y4�"�
��
�_
�`oD�F�/AC+�WA����TJP��q�)f\���2\���vRv���i�?��e7C��PvS�F(�i����rS�br���O5;���f��&9a^�7���z��c����yE�`���q�8��I�W�
���z�c����yE�`�T��q���3���CPU	�G`�{|va<j�0dv��V��b�B4[}[����z3�@��Jx|6�=�w�2]/�NG5�!E�>��D �?����&�'G�r������&��m7�vLE
��o�4�@u@N.�X)���_z��y
U�^,t@k�������������:���S_[q�P����Di�|i�U�Tx�������i�:�?��7�r���S�*����@����J!c
�F�89�R@���?vgFU
7S 0h���*�)��4������*��~v�ZJ����	�������1��>�����O�C<`~@_�'�!a;O�'���}����9�>�l���?�p�}��>��#���~8f8����8f~@_�'��9��s�	H����	���'���}��A�'~rP��F�����������ma��/3)|<{~v������D�KQ���k���`�}}��� ����;���=EG�*���F	�2((?E]���Q�����By�o������������)4#Z�-��y�����M
�v�2�T�_���L3����.��f3�����p�q�W��p�q8�8r������])"�4�)�r�	
�TB1W�B1�2�B1'�ZXL�;���#SMS�6�'�^T����AVec�'����p"���F���������F�y$3Q�\������`����������z��������'q��������SC(SZ�ymBvA+��������G"w)�W�}	�/!�e/b_,M
��w���c\|���/@o;���[�_j�4��Cj}���s�D���u1������]���s��f���A z��/`�l� �VD` �������#�ncL���Fq1�~�"�����<qB>b�����~��yzs��n�u�R:Q������"6��%��8���P��)!�!�8���B��F�Cq:�BB)b!����B��8�p���F�)<��X4�vf=��`����-l7������n�����h���|
��������h���t���Q��4�!RX&��'/k���/��9�����d2E�??�t��39��k^���8��u���$�������T&�;;O��sc��[>O�:sE�����B���o�v�S���\A�q���P�_�|���U���b�����i�8k��

��-_J1�Oz�Bl
����K����IOPv���Di;e�S`'=A���Ji���r���'(��@[)���PN����hI�2�B�%�^�l8L|��q��rw�{K����\��������NQ�/y��z��V��b1���Uf_�n����������?\DJ�o�lC���+��|���~8N�N`��zz�����Y�/1��ws=C��zh_
�_f�����r����vc0�D-S�����L�NQ4F<"����c6�������+e
tv�I����G�\�S���8O����xt��=:����_���G�\1T�S��-��J����[(l�=��:����S.��I�Z{�TF�%�_P55G�;kV�`;N��*m�`�#h��*l��T����*�
�q�U�s�<��J�AY�~�*C�t����k"�Dnw��n��W6wW��1YW>����2�W|^���+���+���+���+UP
�����+�y�)����P�����l�+x��O���y��������������G2���#�[_?������������!J��[���5K����_r4��Zv��]����?�?�|��=�et}��D���.�l~��f"|�]> ����G.�&c����	�	{���f�����������R�0��n�����'&�w�EoO������s���������	�9�a��k�w	I�"p|��
\Eym���>�{��N���)�����
��Uv=��%�9\U�B��.��M6�)9����Ra(K�/�	_����F�������D���{�������/��3�m���O�}���(���?y�DnM�<w�r��%p�kBy����1������#�$
�T�@,�����t��d�(�O�9�����,��@��g�����������_�����}vU�C�s���B&}��=h��~�A&�,
��eX��to=�����Rp���(��&������63�i4t������������6����t�2l�]��I�����oD���#V��3�kx29����:yFQ\�(.9�9�x-�������y�]pl�A�N���g�D9E�c�����<����!w�Hf*����4cY���T�`�.�Da*Z&���z��AY�	|E��0������i�j���d/ 4��Y�Kr���8^`��[|��tK"�����K�#�nM�%7NP�L�FP�(j�#"����r�k��-���\O�b�jaM+G�QT��Ve��l�1t%��9&)7�s�r[P]9��t������a��,U�TcO� ���-���#.�O����ou�S1��#24�:��G��[�T�#��
�-�4y~q������\�8!U����a�Yv�&�tG��L�p�$n[����k"N��5�]#�����#D�����5�KQ+0�%�-�MY8���Mz�[^�*��b
PU9�F�?�RkQ�{�'��C�e;#:j������Vg0X�J<�;Ip�+�s�8��1���T��������������IL������U�T�k=��Q�B�pTsj0 �S�[��1��(f�������m^<��7L����[~r�6����o}���k3>a��'G\+,~�>��s������m��k�������9�J����9�pHb�?�G%n1�;G���9��J��������-f���
����r�����c���qA��g4���(����%8���8����'lC���h�'����C�����d���<_s��9�8z�<d���9��s�q|6��h�Qpp�Av���G][nl1����"n�
9X"��!�_���=pc;�fU����9�&�����p3g���!G����,n�����_'�����G������i�s27���
�����/�����Y��f������|�W�#~����<����
��#~-�8������5�&�������9��h����o���pkO��o�{*���s���y��q�D��s��������|	5?�8�#n��)�A��.�-�w�#�n���}����.GA~6�3���1����y��qn����B�+
�ks�q�u�����}��g�s$�jp��Q~�A������_�n�wC~/�V)n�8���_?9�9��,���u�!���>���:8�i�q/���b�{*?�8���)/C�;������s�Np�;��9�e}^��(�y��p�Wa~E�x�?��w[A"��:^"�Wh~E����{�������������=����x	��{�1�%c���D��y���?oy��K����q����6'&���a��ZI�Y��'7�xy�_�xY�_�x���/q�� ���������Z2��r8D�	�(�_j���T���qL\�-Aqu=�%��t�l�x:z#��
�3�+|��p�4����x�JP0�n�C�9*1:d�������5��A�<���������$/���R����%�l*�6Qcv��@Y�5��T��mfR�t�!B�^P��RP8�������/-)�����������CMY�g�h��(|�������g.�
��@��
�t���&��|{>��J��P�@����3?��U���5|��~
�jj>;��)|v��H���k�3H��\�>;�)|v��.|v�R(|������Fi��C��g��(��\>;��!|B!#��mg�0L�}9�"��Y��-TeQ���cY�f��t��E��G�"#�L����9�?���za�cOr�i�es����$,��O���'mk����?���^P>�Z�I��9�LV<�Y�r��R@��XS�6f�4�US^����Tlc�J�,5�U�1�"���2�o+��M|^�� 21u��H��F�r���?����������=�
���(VF�u��%�)����Ge�]����'��(7����!��rb-�C���1�P�{�J��<u�f:��l�V�����,��^�a�<�#
����$lR=����E����z�r���^(��h�����$�����j�]AdqV���(3���e������l����$�<�c�3���|��G��8l�.{TY�(����Uv}���a�7+�u8%x�RO��cH���{}��
�D2WJ�P(#�
S�Oe.I�0�	��R�=���`�_7�(N��_�6��b�F���\]1�������X����?U���T��\�O����G����1H����:���y�eN���Eo�n�O.���|��z��P5��S�giR��j9	,��K������)���:�YH,F��P<�����	�:h-5�{��*|�b�C]��<��J�YDM��D�EC1���M�E�1�~j)B.r*�fwn�����.T��v��sj������N`+N���9�Qs]��H�n�M��� ��(��BI��	��r:�1&�#��Ju�Z��w�������j�������u�9�@G
v`%CU�R����d���C`c�-SxtE�h�������6��wE���2��R����YI�w���T����=���������Q_�$�����"����Mudp���-����9(9�+�:f���W*�����sM�bz�����/z�r@��|��Mx�*>�D��
�c�F��X?I�0����>�Ub��SlU��c��e���2!����q�������!��������{�j!�ij���V�z�r��=k�����{�bq��k���=�yIp&��h�A��@�I��V@������w�2���b3]Q�.]
q:/S[���d�y{��%�_x{zy~q����/�Kui��S���?���/�,]����i/KL�1[�H.at�V�3���X�����!.^M���.'=�� ���*Uj�>�}{~�����q������U����U[��zau���_n��<��)U�d�l���HX������O���/�*��N������4*�����u��b���!W�+

�dW�� �b
ezNs������v�5kI�V�,��w��=�.�����hQs������j}S����;3���+�my��-���������n�+�t�[(�]�����B��7c"9A�
.�b��!.�^��S�C�����IW�*���8�u����3nbe'S�_
���p����^R��NO��ge�r��R)�1���s��N�
��m��{��i��/I��&.���V�,��_��V��R�J��4�<4��%|4*8	����-�.7UM���z�*����~9oU��s�_.G7V�2���j,�\S.�y.cW+b���O�hR�����Pt}qO�ry����I�nP��x~f79��s�P>JY���b�1�s�o�$������������E�)�d��9���1��I�;����3��t������f�E\[��*��x|hm��)?O������r���I v���{U�-��!.���������<X�����eFG|Y�^|@�"�
��2��
�R�YaB��0��5M
��3�@S�<�9����l�I$�G�a�����fOx��;o�q�`�V�X"�i��X��T��&Lh�����r7U������|��Y�}������9��e��,���]�K8��r>_�������Y�-���#s1A�vq�</w��.:��c�?�9��������A�y����#6��'���b��=t��n�����>]" Oj7�����f
�{�H�:n��m�SEDc~0`�S�/?�SgOr���sX8�u���
KgZ��3��a�<��,3�z�u';`I��M��r�,�J%y��K,`��>�$L�V2��F�Rcf�U�]n�,���~Gq��tG���o'�t�>���kg��O���pv����sr�&:<�������r����a��,���
3�e��j�9����hl�Pg�b_�������M�4;��
�x0��Q������S����������w=��
�
�6��t��c���f��,�w� ���1��v:_�?�}���h7�����!no��/���*��P�T�^,���|�{�-��6@���p��'�O�E7�I4��o��������)��WY��|����*o�*]��W@�����v^���Ng#��*J��LW����l1�=��
�?��sO@�BA����B�^��u����~����
b���GX�y��	_�����-��g���eW����
�6���g_��)������N'<�����Y��h�e1�e?\e_~�of3�T2w��Q�L�^�Y���,�t��S���4Z-���t���mc�E�
:�>�������#	�po��>��/`v?�v�u���I�������#�a#��*���W��-w�C�^��Gty��2:;�x��A�
��x&��z�r��4:�:]a��}�5�������V�9������7e�x`�-�h�y�-}�>�$���Y
$��73��b3�B
����pc���-f����l1�����|^S���_�4���.[��py�d�[��9��n!'-8��"}es��/�I�4�s2�<�����&���4Y�6�s����9��/Oz{���.@{�����D_ ����bT�vq����n
��
(E�7kR����O���t��Q���V�V�G���#];�M;�d��v���x~q�y��7��a����#H�.��������		V������h�-���{���/^��_i
�=}����L�
�&���;�K<>���ct��(�v��x��,`u�=�+�>� ���B�z]d��/xh��%��kd�}&Pq����l�A�&�<vc��j�b��L�_��8��vdfT�#2��Dq]&J05?��	,KlKlc6��(V2QL-�R�)�)�e�#SQ�Rc`�z���m���H�����w�3&[
je�lk{���nj��''��420��[������n����E�hl����wh�N�O�e�3��Oo��9C�?t�S�?��2q���7�����V���
F��hP�E;zvG���*W���l��n�X����j����nP��mD�
6�M��5Hh7<��>�U��������C!�m���F�n�d������rS	�
���iD��4��%H�v�&�F�m�e�ihy�Td\��1�a9G�Fh5�[n�(���aV�1���-��������N���.�4,n�R-h^t�c�x�
��7�[�F�3��� �8G��X�
�A�_M�]>�����9�������*C��Xe����!�aV@^$��f�ca-�t_��_�A���f��P���e:�#�%�6�=�����iy��(,��pm�Bi�Y��t�����,���.(���j�!F��3����0��@��)�#@7���2�A�R�{����P�U�������e�
E�D����v1(���@��b�Y
&��:V�����Z�f5eK��JVS�UMQ����b��b��
V�w�j���b�j~��d�Q��jb��&5��i�
pl@6�A�s���pn�6A�=@�M�lTm�(�C�~M�E��S��&��S������������P=������`���^�=
A�_te��:Gx����q�	x��U���VyD��c�#���^��I��}g�V�������s��c��X�6�k9".�'�o6�K��W�j5�<�2��?&��F��LC��`�cK������<��c�@�����W5�@�
G�M#�jk�6?��	cW�S�#�Y5�
�j�?���`ceL����������
��b���Q{���pl7��02FY&��������b.�F�!G[��90��g6�6#���D����`md�l;�l����Y+wS^\V�(*A�M��Xl���SXV�`�c�35��xq}���1?g#�1w��y�����y��W��{.f����%�s��9��b��+��=�������D�����8����,O�h��X|+�y+�J��<���R��k(J�/=L�����[�^��}88b"-u�0B�*�c��12[�F	&a
�h�F���`i�6�J\!KJq�����m�Q�7�)��>y+��7K�|����s����p�����D�1$&���wy���.=���`s/*�P�Ug(�*��zQ�9�������LEE�y;4��iso�9��V�vh6Y/�e/�6��x+�P��o&��f/�j�o5�����d/�7���JN��sb��De����9Q��5'��9Q��x+�PxAdN�w����c��~R�vv��Cp��p�5���@g�nq�<��}|1D���o������
Q�"�����c����t���I'�i&�T <bQ�u���Z�r���x�����LuXl��� �����<�$1�w����>�nW�h5��I{C�7&_���ZM������9���D�na���(T��f|��L
[�Ia����I ��3�	��z\�8�6Z���}h�u�:���v��Nw���R�l-p��2��\f4�e\�:d�	�m���XZ�g����c��z9��������N<SW����q�.���*��]KE�m�n�X|�.��Oo+'fE�@qOZ?wU�e���C��W�!�r�D
�,�B�'�&`$�	�� g1��	�=:�l���8f(��9[��G�O�_.>|���1|G�����2 Q �����hl��GS�������\_�a��u��!;���WO����?D72�wT7��u����PB���J����v"�w7�k����H�����5y4���o���Udfp~k���\�2�?T7
\}�x��n������9�������%���X����YI�r���
 ���J���8s�Ps��������nj�L�y�1��	��D��f���X�����	��-4���Gx�I�s�~���B���nc�w
�378��qn�g7�z���'���	�Hr�?gKh��_�59D����/�<�n����|�����~��� @=���3��A�������]�%_��`�8�n�
O2jE�:����_�{rU9��*\U�"j��8<:�]e����|]P�\��*�c�*hQA}>jA�9ws�����Q��*pS���IZ���m��A�J7���KjX���<`�J��i�/�s����-��j�N�����N��N�����%JX2	�WQr����ji#!��:�y�oeK����H��z|�m�	3���js �@1��`��d<�NW��Q3���u4����n6���)�����e:��V����:[2"@
a�I!�Aa�f���@�����=��c/��(�c��h��_M����q����������U}��R%6�z��bT��JU�@�RG�t���`T!�l���E��S�c"���/=�FF��z�Y.���l����-mK�T�[��j�",�Z���q��C�X�/q8��;���);
�R��o^p$�Q�j.E���sf��VAGrQ��l<������|C��5 �����A�~?����&7W���Y�%���q��6Uq����V�fjuo�~~�L��)Z��S��tS��^�y�>98�i��]%��vG���lW�R� 	'�)I����]hI�:�1�C���(CBt��I7��L�JxH}[��T���\};�MV��:�t��W8.��R���/�z�P�Tq����G\k������]��#V/J��/��?�.q��M-j��j4d<�����J��I7�#y����G�^>b�����x��G�Y>b�|���s@]Q��#<+�����o�?�u|/ �0 l��g�d�k��&j��!�M���9$(�`��yz������ ����P6JX�G�*���5<�������glZ��>�
�����\�����?<6"GL6*��p5���7��a��hS�
�m�!%�B��,�o�9��R�*6
��+�1$Dw,����"VU�vB��@��.��+&����Y��'���9
k�Bp�<��9��Y�<��Y�(���Z{�������`���M'(p1�<�xyv���c/�/����=y����Jq����+;�>*;�����]�(+;h�v+J�����:�X�!VTv���Vv�5�bR���j��������h������&�6J�6����zk:<�L-�X�0S�R����>l4E�]6wD��T�%.���be]F�F���a�K6��b��H(�:��G���mTz�d�%�$ij
V�����i�ru%��������H�u�v�u}�:px3�z
��v�Q���n��������~9�-��� �H�?K��Ez�7��'�EW�t�6�?6��5`+�m�{~�����*�n�z�6�t��&�����W�[ZV�).(Sux�}����'C�}.����U!BU	��[wl+T���������Nn�$��7��<�bid�8C��sF��O:T�;�����6������-7s.�"�^.nQ|4k���n0���$z����}Q�������������
�jm��S_��c���J�f%Qk����
O�d��5-T���C�Q�D��"�@�X'��<�S#L$�Zf����N���i�`2��,�fI��9V'�fE����(M���'7
��� 1m��T)Q������N�6h�m��s�7alU~�n�vC�Q���"�^�@�<�@��k�� n�0zc���T�5�K�5�]M`��u�>d�JJ�K��1��kn�azu�]A�/��0NW((f3��Z(�<��;��T�S�j
��'kQ���aR^�%�0Y.����6��'y~�>��6{���{Io���_>|�a�Bt�a,�g���������<��j1��i�����H�
G�M��i�����F9����*��?���`���P�\�@��\�@��O�n��	~ F+v��`���?�LA	6��t�3��s�^��Q��+B5xCx3�MqP���B�C������y��������YK;W��"Y�6[����h}��qi����*��V�t����y�����av�MP������=�������D�\������~c}�1�o�����@�j����b������/���O������c��C���x�bn���,�/���?��"�Q6����s���x��C���~��8|<[,���I���MkT�I��x�p��^�2��Ya��#��=8����V�P��S���������Ap
�S�v-8���B��]�Lo3����QV���m���
x���@������t���Uc.�N���}���m�e~
�<�I��/��yP�	������c}��Nj���yX5�n	�be�wk���j��c`E��(���}�K\���U�VV*zD�	9�j��n5��}n����<��G�����a�l����b�����9��u����pK��fiU<��<A��*��Q`��A�����{�t����A��}\o��^�w=�C~�A@�F�n�"H����e ��4���������*-�c�6R�)j����@�c)I�cO�������_���� ��b����z�N���I���rE��A`jx�������e=gZm�F�-NCq�m�Mf��n��1lj��3o��h�cW�5 K��(���W�w;�����W;"[��N�����������O������k3���h_
��6��n��9R���Ut�����:�k����j����v��B<��;Z�������$�������g�O���ac�"�g�s�&$:k�-����u��5:u�v3[Og��l��������r�xK�YI���e����i���)��m�S��#S��N�mc�����-���F�)�s<��*��*��_S��([4����O�P�pr��*4�^w�:�\K����E
��-�?�� �>������t�9hW��H�'�J}[*0��S� �$�4R�S���vp;�	���2_�>��X�������v]�+���v]��h^Wc������u5���q�u5������j�^W5T�����u5�����uU��q]����������+��X����u5V���z]���j�XWc���v]�5�jL�U?����lV�H��-��c�*�6��()���a�(����D�c�5���o�!���m@�+c�e�����#��������J��A�3��]N�`j���z!����������.��H�S��*���~�I��r�������;���WLKn}�<I��q��J�+�SI�	���R�(l���2���a��M��\�$�0�/�$JQp+#��n���.�oV#$�GX����m�n��t�� ��u��I���s�z;����D�f����T���#�f���H�n���'��V�����^v��Kpe���g[��?X��n`tJ�m)��8�y�� Nc���-�N����Rm;���Wp���,�\�k�����D=�|�1��&�^BC
n������_r�5���@�79x'�c�z�����=��ks�1>���B����z��6$��@sgGy��V�g���h�
*<,[4�
�� $���^���
L�F�a4�0U���V����,S'M��u�)�n7o�H��U��������=(�-��J�O0�K�������p�B�r�[�-Z���
 gKx$8`^���t�D����'�^��Hz��l~f��O�V�����7B�oZ�3^�����^��S��!|�^I������1C�$I��r���#_!C_����[@�Ko�z�%t0�M���	����t����6c�Y��Dv3\e���<�]<���z����@�~�tz��;~�G�����~��>���g�=�=���h�XG��t���z@����ik���|��8{�&���w�-��'O`��������\@�X�`?[���?3����������_��iBJs�M�%c�i,�

����hdl�mo=����9s�s,12y��H���
|���-5F��U�����(���;ob�Y�7�@
����kh1��P��(`Y��p,������,���	C�E�8nEw(�Cm�-etl[ �5��[v�Dj��70C��y�
��	��R���y1����.?]���{K:O�����x4�E1�g\����(��ty<�]��E�S	������7�D0,6���+���4�
���`Z�y
����i����(�i��H���M~G2�u:���E^��y7K�����!�*����
�3�",O��������Fy�K�@���� X���y$��U�xa���=�X_�
�45����������<�-��_��&�\��V�,J7�(RA������b�1�Y0�g��b���f
�6��
����)��`��|=���_^���$����*J!�����B�I�@�o�dc���7SS��!jj����X���+<�YRD��zL�=��F�72����od�7�����N��l���
pF��?6�e���������JS���Q@��3��\��� :@��PB��C�7Z��F�~���x����	��a�mO)3a���	b�3\�'���h�� J
>�A�����S�d��rz��]��*6�$��c�Vy_�7���q�o�)��h���c�)G{S
o���3������`#�h�]l��J�����A��j���(��h�������u����G��������d�9�B��t~�����e�}�u����$�9/��c����t�z%��{����p���H�����Y��h���*�	�4jQ�N�#'pp�!�`�Z�^n��zSG����)'!��@V�mT{H<�q��&�*�t9�D�@����	���B���Xp�`�}��
G��I
Xn��j!�p��@��0*h�2"(z�HO�����;��0%@���4�P�4���w����H��*)�$������A���� ZC��;��XvL�Lv"\��r��hl�kC�Clm��3�W�����m���m3�7>�fl�1�]Wc��p���%	~=UK��w�mt�i
p��"���@���a����y�����_CpO�1uV���6���A���V���CPq|�Wi���C�QE���v<���=��Y����X�0t8��}V��|U����I�RYT&�o���s���6�*
��N}��8X[f���B<�H�u3��R�[��4�����R�}���f�Gath���8X[f��&����
�[6f�j;����P6�Z6s�m�i�*8��M�f+n/������5�U�+r[���(V����k
C�kV�����4f�ljq�v��`�k
�
��j���Z'p`�����-����l�1	�
���
1�
�6>�)|
����]��*��PHd}G]��_L�\�a5���b�Y'��p��,4��
BT	��R�,�`	F�np,T�z�$��)�,U�7Y@b���>^��������8r
Z2��;Sn��
R��O��Ab�����-X������<>����;�^�X����M=����>�]?9���'U"�������2����@/�������x|�{}/E�� ������v������[����nX�Y9�,�r�~c�
&)����\�������^\���?i��O���*��X�5@y`7n����67����_��~��D�C��j�*a�Kx9��� �04Q��_�X��Xjn/
J��	$r������
(�t���;O�8��0���]T=������j�f��;X����;����%�����}�D ��^�t�������N�=	M�S�"����~��S�:�t.����]3��^6>Ny7wW0�U��������7�ym��bG�����
j0?����=��go�js}=��>]|/"`7�J2�F�����g�F`��h�^�j���&�����?z��j�_F��k ^�_~�x��c�����q�<����G.�&h��*zn����$)^����{��&�B��E������>��w�EoO�����x��������b$	t9}djl_��>�{�����������et�]O�����,�����Pb����
2���P�/t��	�`Wz}cfPb�y���o,�<HP�{���)-���mDE}=,�M����gr�L�@?��Q]n�+�������)7�(G���L�'�k#9^��
^�I9
�.���"�����qDv��v����s��;3 �{�8o=h��'�2������8�2��@-�?�#����B�q�c�zl�X��pr���r���2����r�I�w:��alVtS��E�yb�v�R�\�A��K����r[�>�����n��/l0�)e�. �V�d%7�;���>��\�����(	l�/��n
��G�T���HW+r���8�o/$����~OO^�d���Qq�z��0�B��8lr$��^�����:+�L��T@M%v��b
<
�H8�U�<\n�����a�nh"�[��0b�]m#�#z�+2���� �R^��[f8F��\�%�|0U�����N@�2�y�qU�<99���0����v�����Y�>���Q��Tm3�lO�c�v� O�(U7�:(7Px��PW��Z��;�|�\Zu���
��6c]m�l�n�]����h
���V��5���7�����]�k52�=S`�Xv�p��W�Z[a�&�k(�u�3�{r�����h
[n�.�{H&(\Y�`&g����ZtZ�����m�d{�;T�����!��:Jbpq�~tA�/�fy~��0,�o3�[�+b���i�}�v��j�LH,T�7i4'("_J�n��et���
*�41�-�d�%NlQ����eO���J���*���<
;M���c�����b)JnL��fk��F��dp)�QVCe��5QS��I�@�f���"��|����k����W��j�N�B,��������N��r-�������@����T����p�pJ�4N�P
�z5�	�='��`�]R�����nf��,������R1�B�LW��(���t
o��7`�MQ6� ���9/Pw��56r���2�=����F�k)�G�^������,���a�A�G"+�[&�V�}��I&���
��J�������R-����F�_Nh�����J{j8@�����J�]������W���(��c��x��VH=x�o���b�UPWubT3�U������]��QPWq�]3�U�D���3�^� �L���Q�nd�-7%[%�\@�_HbacUl~��&�����=���vsI�b]���f�
���C�����TRi��Z� ��"�t\:H�wz��l��]��{�'n4`�aPm��lp�X*���-Xt�rGb�#��{8�.9+�N�H�� �T���A�c�����*���}C��#uM�a���kg�q��v����3�Q��(�`��(
���c���UK�Fm3����s��//f�-
t�����&����[���g�P
w$���^�����P�w%:m|������w$v���o;����J��8�v����0�o���N�?�w�c7��(l�����ob�Z�YT�#����;�fj�����L�i���
�E��Jnjb�)
�t�s%E��B�R:r���t"�w�0�$��
�����'M���� 7Qw����"-�Z:�{��Vk��p|K���������nA����~��K*�Z&�O�{K����19���������j�?8�vT�v��*�w��\�����b�rm�������|E����_z4[���Q���.e�����MUK��}d�Y'��h�}���������(V��k��	�M�&i�0�L������&��@����S��tg����S��md
����sN��������N�g����j��=+�r=g��X�yf�=SU��}�_���=S
�������)�I�������;5+A���c��9+1��"/N��������d3�����wT3�����:[�'H�#b��@��M�%yg�����\e��e�i����3ottT��b��j���*_;������ )��[2J�t�|WU���$e�5=FrvZK�(�oW��m�\�j�jE���T����!���+���|�E�Ke���;*���%�*�%�2��n����k�PD��? T��������A���]��&'H�Z�gG��r����������hy�'3Sc���y��A����\�������H���<ub�*��5V�k����(��i/9���W�n������E�PY���S�������c[�����H��0�g)��{C����&��})>g���[vL��	�
�)�;�\�v���]>
��q��%�74{�X�`j���Tf�-��0���\6#Qk�R����5`�&�]��u����;�;8�����|8k��5�K#��6��n����/)�������K��w���{n �>-5*�%H2N#����S�zo#P2�0%C�gKKgh���d���"�pE�N���!��~�+������hn���'��������^�&��Q��������QC���������PyU�����:�X���Clgc��O= <ww���d�A�-���<(@S(y����.��P��	��?�a'��j�G�s�����wae*�=�]*v�3O
�������iqv5%|����#�8hw���	e�J<,�8l�t��yjB1Q�C���*F��d����5-mY��������J���������_~P���y����
B�����l�=�x8v�����MJ��h1�9����U�/��t��d�(�O����*��,�^A��g�����wvq������zz�]��9'|_3�7�������d8���]fS������b��
p'~�����"��n,���0������%&����>�A�!7���r��6@�w~q�}�T���u�D�aG�3dF4�����B��P����
6��Cx�7^����z�D��M��i�o�`xT�d��
�[��O0�l�����i������T^��no��'gg����m�{���+�rl��s��_a�;O!J��	�
���2;�-��_����{j���^��t)����B'�E�������a$���V�%vw=��h��h���t3�0�i��ez�1g���]
m�����&,
o���k�[0�/N�v���/��.]F�x���+mB�$��,KW�\N{x$��cv_����h�"������Z==)���}Z�[����6#x5��9'<���j}NW�����3[,���-o2\n��>`U{S�'��8a��[�o~���+x3&C�K|^Hz�!f���������q�7l��:� ����������!��?��)�D9���p*�6.�o���h*�wr)�rF2�����ov�h�\ �v��N��x6�t�W��uw3���94R��s�����(9<���f�F���e6K��y�^NW��N&���p�y:���O�g���������������8���W?�������%G�������A�e�������z�R��������.������?N��W�������$������0�N����xG����1���'������(������E?����4{6���=�v�;$���M���|�ct��z�I�s6����'?��GO���Q��f�O����o��6�����"��x�'�	}K�?�_��������Y�s����&z�j���O�9wH7������V�]=�.?OW�����:��'����b�B���34�F���t�};�jcZ�c3*�����������n
6�5:�\�s���t��'��,]A��z9�V��[��2��K�.��Rk���Hz�ls����t�d����K6�����CAZ���x5���@��-2@���n��Y� ���s�f������J��h%@��B��t}M�f�[X�(�D�H��g�O�5�L����]���:����:���L"0�����9hS����W8��0�8"��#%�t�d�������)����B>F���WW�W�#N��y���z���7�l�#��c�_�:����8�����o������ cGE��L
��#��tHh��3�D���wFe���7��`CP�*h���73�=��}�����2��.�;�T�z�Z������ww��
���`��D'tA;�Sn��4G@;f:	��>~Fc�F��r=�l=���������)�)��:���e���	F��t�^�O��o"��uf�� :�=��~�M3�M���S������]N�x�\��V�<���A���C��l�w����
�K��O
�
_�C�@�
��P��?c��!�`F(^�0of����`���/�������&+��s;LK���b~��]�0�`��'������5Mf��c1��6��x3�S`s������1��\v����;4/��#���~��,{$��v4]��3�f>$������
�8���o��t���R�yg)@0psz��M�@���	x�y/3�o�p���[��'�a��x�_PR'@$9�/�E?/����Bl��2��f�I��E��v�3�=�J�x�b�Dc'��?_�9Z�)��$�r;@a�"�4�"c5dwJ��/��_�:~�WB7(��)X
A�6��e���:x@Gh��R�~� ]����}�+���kzsC~���z>�G9N��1�+���fh\A�#�>��!�!�D��t�v������+eL��9��3�NY�
���
d<'!�t�����������8K����r�*K��^gz��:P���=L��/���3(b��B����
{����z}�\,�=4��6����j���8��Yz�:����
�j&O�ohl7��Xr�#x=��3�RX��@��	��!��hB3L�|��,[��m���XB���dEe��l��n
�DM��BD��d�cl�.�#:���?��2t���Yvl�3�_�N��j���&��%U���	�����A(���H��P+)\�nf�
���J'4m "����l�%;R�n7���nV,���H(?F� LA�W��
xtE�3
-��)�.��a�Ym�p�L����\dw���nx�����-X�/����/`��<�E�7��?�����w�I�]���:/k:�"�9<<�D����q:�,�?,��@��o��C��LD�w�T�?�������u�]�����O��LP{�0�D1��,�����p������O��>E�����A��4���
����0�=�#��G��:�N�g����xp�E����������������������GO��A�[�=�.�c���V�bw�Y���/��N���}��/�����C4��}c�.�������
����f|���5�; ��-�P��<&�	����]`*D�.�Be���������@��'��.���RVf�b�c�9��W�>\^��.�`��m���/�X���.	&k�����H�eP�J��_MW��&]^��`��>��g�P?�[f����Yt��t�
��l=�������"@�
�P�L+�>M�gO��$yQ#`�g���#�_Vpi�.j��04���^�����x
���Fb�P=YDP�_�=�b��f���sLU��
[��[����n�X�aK�xE�������Pj�����������[�M�=��ax��G�Wu��G�����S��gQ�����1�����
If��h��_�W9@��l��/�_��&����{�8#�B������o!j� �|�,���d�t�^��hE@�+B��)������-�Y�cV�
�T\A�d(�F }��2p�����`�|@f~6ED���!�:c=@���@�D:31�v����a��q���:�z��\���pnO�j�$:��8�*��"X�8 V��H�������c�N9b`��A5C�[C�Y3�Yh�
Y~��p�Fs�.�����2�h������Y�,h�E���6�������.bN@���?��4a���+t.\aQm��JL%<������8;���_E����!����T�F���a3��Q���f��F)?�n��m4B��D��\{�k����z����wHK�4� �8�L0�1	&h�����2�f�L�Bq���y/9�����P��'p��?�o��T�:d���s0��{��M��Y}F�f��H�@VB^CFQ��3�i���~�mo?��������������|����
�q��F�"CR3#De�@��hs�3�ph+���q� �����V
����\r�M.9�%�z�K�����5���Z[�a�����xoy�4+h�K�-d���W$;a�y���&�0�f�!�#V�	�:���O�D�<������������>!�e�PHO�M3�N����s�a�S��p��G�,��;�^fY/�+F������y�����640f#lB�Q�gYW���g����-&�s�������.�I�� ������{����JF��L���0F��Q��d1[�����5=:���`z�,���7W���<P�;$c8��o��[l��K �!?�+��f;L{��$��=�+|6��O���������w�o��.���x	�V��nH�\��;���E�?�}{������Z1���Gp��+l���b�-���@$''��S���:[��r�t;E`�'/�;�;����'������`��'��
A����
���=�k��Z���6"TU�,0j�KH~�~��_e��L��H8\��L�����)�{��9��^pKc�GC�@�~������/��"N�WQA<�;��8Y����g7�d�B,"�����e�����$�a����1T� tPbAq)Hx��z`�xvt�p+`�"^a�S/�������C#b(��`f��E��v�L������{T���R����0�a����J���w�e}���	�j���+�P����<E�����|��7hS��"�5���(�S�� ���ra*��C���4���r_�����	&�*��%f
� �����5j�n�b(��P�&�E:SX���	i��:O����!B��5G�� jI��
��c�H�p�����L�E�,�`,I)����T�$F��@.r/��
$.�L2�a���}�����:�&�M�����$Vn��b���!M���3p��S������H$j�~��.w�-P�����+Yv�?�����}�D]P��,%�A�>v=k��AX
��M�����(-ONN��Xg���E%��������	4�(�hG#�,j�����yN9�Y��3qJB�����b����f�����
�����t4d�h?x ^ Q������l��*�0��������i����2+s�rF8��$�;�r��[.���^��h���
@w�7#����Q�����O�$zWW��j���
/��
6������F�Z���"�2����X�����e�r���;��HPKf�G��NC�?���9���w�VT��@'���9_`�(L�Z5���N�>O�>�x=�������-Fb|#/6���i�GC[��Lb�DS%���4�R���8�4���"�9��u:[e��e�����
W���p�+\�
W���p�+\�
W���p�+\�
W���p�+\�
W����W���
#284Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Ashutosh Bapat (#283)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Oct 6, 2017 at 5:09 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Here's updated set of patches, rebased on top of the latest head.

In this patchset reparameterize_pathlist_by_child() ignores NULL
return from reparameterize_path_by_child(). Fixed that in the attached
patchset.

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

Attachments:

pg_dp_join_patches_v37.tar.gzapplication/x-gzip; name=pg_dp_join_patches_v37.tar.gzDownload
�m�Y�\�s������_��;��l�%K����M����1�s��u:��$6�dl]����� E�J����S=H`��]�~vz5w���[D��M��T�G�����1~N�}���
F��x2��&�n0�N����I�+S����U�,��b[����}�6�����s�4H�8��Jv�A'X�B��Q���.��gT�drR���i�tT��p<���������\��'�Rx��d:��F�h���sO�N&���''r:�OG�������H�����~�	�/����y".���w.�xfd����o�rd�J���i�����n*������$_�"�`b0x2�?��V<�7n��o�K��_�]��x%��_E�@��h�qm.�UM�Tz�(��I1��Ds���B� t�p]��~#u��T]!nR�2u�P�x&�E�,b��)��"_�0�*r�)����������[�RE�*�hu��L���p�R&s�)��A�SS�aj�H��A��r�� �����k�`�h�v���F7]�����|�b�d$"��*3�-vG��T�^���H���f� ]���x/�A\14��T����g�'���ZIP?��d��OW����a�!����Yp���!u�����^�k�d|�>��|_�b����
�)�q���tH���7_�=/�f����E���7��V�!����>���UmaK����s�)�7G~v��gw#���d�s�?U�
�
F�0 �:;�{1��,�
��r�/�~C��{@������h8�.��K�����A�%���
�m��4{����"}����{|&��y`I�$��h(v�L=a�Zx��hrV?�M �[���3��SR�yU�V�(}K�X�}W��`*�"/�|�Cn���iQ?���a�t-��f�����e�!q-j�����j�~��,�XT�
w�C�:����$��W=��������;�����7�1��������\�Z\�gU�XA%��:j��"X(�B������Lt:�|so���n��"_�o0�M��nw�]pd�~''�Bw�m�1������3���m�
}a�v��_�����Yx����Cp��RBD<���������g����~
-�g�cW*��+��E���6Z|��,��!,����jg����}"g�;���Cn���lFX��B��`�&�)]l�BR�|��W�b��t��:�)�����$�*[�0�����,�H���Y����m��~�.���OqE��J�(����H�k$��J �<V����E�-��\�`*�;)#��T�#�[����0DA*�����	�M�bJ��${p	�'�*�\�['H�[������#�;�/��G�Vm�YAl	�
�NU
�����h���g	�����|J�k	��J��N�:8E� �@�`�~�81 ��>����R����������}_��,LE`�L�`�E<��Yv��,�^�����2�|�zV���I��as�"w!��V7{x�*��)9���r���v����d(����h !�.;���Y�l�{�?�����������.�,D+Xm�!K��R��,�!\�0-���"�W��dxH���Q^������L0��c"���|2��8��5 �A6+[�K�9i�M"�$���<tz2\�!�0�_�����L5�4SkJ=\��
m@�iJ����`���W�uY�N[3S��t�9Mb�
$,w��D0�$���AR��D@�M�)�#=�)���D%J/d�M�u��ygAJ�o�*�1��Cf�n�
4��\��g:]arI��E��������U��Q���q�5d�����|��� #�B�aK�6\���Ops�*un��|�	�3�s��lq���if���"����;�	�v���B�lK5r��0Cb`S�-Zs%q��*��i"�����5��V.��p����	C�tM(���%��`�����-(#�C2M��p7e)�p���p]u�'Wqk�.��$^!������V��Rprd,A
rx/����:���xD�ft�)&90P%��,��\��NQ���*M2/��V�1(t�����Ka�t@���WDJ��OU#/u��TTGB��#���F����%��5.^9%���H�Ui�U�D��#��$��m�\`�����`��~���
����+u��j�4Z ��e��lT���jC
�c����nw����3o��Ajh�h��a89�t?;P9��\5��y����8N��'��c�
��~��H�q��@�E�U	�����w,�
��|��8`��Z�t�+����=�-�� �D~o����L)�-]p��q'�����K`:�%Xi���pO���b��.���,yp7��f��s��e<��iU���4\i�
�==�L�s�����|6�����v�U�-�����f�'C�+Hw�>G�w��Yg_��9�vm%eH(��b3r��4���%�~KsE�\*,�4M�����|���x���w����WW���W����F�h��Q/T��~�o�^�i�&[Ner��i����b�Cc4�$x'��8�
���G�f�C6Xh�����<�$rH�$�����:\	�
��0{�A2L�T��$'�Sf]���
��|!Uj�3h�F���&o���C���~W���'�/�lY+#:�����-�B�1T��GJ�4 �]}��<�h�^�ZH�KX���S�������}��"_A��q�<�=;����<8�����.�B�~5����f
	�����E���'��J��n���4�Y��a�����h��20W�4���MVy�����'���=���	(A�cH��'�����M�����\���������6�i�r�Y3��C��2�/@�so���v����K��'�
������X�
��}����V�)sCe
��TI���&�"�M�����
f���W���,�6W�0y����I�x���'�\,\�H��:�ih��^G���������}�.#}�b�(�I���\6��������dF��M]"��M.]$q6_�t�J�����a�my�DJ�#b�5�I��X{ �s�y�Gp�\?�-'�R�(1��7!nb��o�X��L��yi�\��.S�5:�%7�k�-0�+["7��R����K\��@%l�eHt���3��$��7��nx��X��$���|�Yjn@�__z+��
�Gv;��IL�]�$e�A��p� L=��C���n�3�c|Yg��n���Pi��.E8�"�@N�P/
2��Q���gFvU�w����-H�0np
&�c'H�gc��P~���Xpl,k
�3G��(OC�EnR��{��f��
��2�w�h�� &R+`�rY9��m�m�'
"<�Y`l��.����,2Q��0�B��6�(lW�����u��"s��&��27����R��2���������Q�`�!3�����h��b	�����J+�w�l�7�q$�Y���30������5;S�����<��h�3:8=����`�>����1�����L���	����������D������fI$*1���b�$�^[%�����AI	���n[	��Y�}&�)Q���A>�h��j*M�Oo.��BLs���j)-4'B�Y%0������hVb	H�;����������<���|AY��BJ"g��b��s���0���q�[� DG�������7j��4�*.�z~��}Ly(F)F�`���!a�!FI)�f�Wf���q~|{�\L
�(i`+3���1E�<^��L�\��.�3�@D(z�R-.�J,7&D;?�r��K�
G����_��a�+R�/��&Ej\c���*���JBajl$W#'XxX9-�E�V����aB���!c��YWJ��_�yr������2��o���&�:{�q��(���kV�%C��Ye���?DQ�@S������~���!�$�9�S�Q�G�Nf�1�g����1�\YE�DH�SkmaO��Qp|���y����l��(7m��
�9�/b�����g����hhJ��e��OE���(�Sq�s|JY���G���_�^&i���c�^�f��~
�������b���(����j����B[�`�1�ebe<���+�s�1��L��J`���7���3���������z����������l�@�V}Nw�0c�U�r>�,]��rl2�T�ii��c��m�C�c��Z�e{��@�Q,���2�z�b�<|�e<�d���Hy�"3:�]���2&�)'��l��%qt���)����������0�A�7��{��v�r�u�ug��h���������w��
�g����j����j�-�I��������P�qxp���,��7�����q�Tn��	i��>`����;��i�*��qX������".I�l���V�u`x���vZI'���H�������xY�����t4���c@{U�E������G�����}���E(����s:T���VA�2	�y�Q�W�	�������k�D������\,�LN���1.-q��+�����,?F�n�JS`�U�X	y�tD�����T�&H)$�i,%
<_��8��E0*�<��]@B
���H�/`c)U�7Z�;7n���
:�B���3���N���t]����BYH���������i�����p�����v5p�jx��_d��<�����S:���<�s7�	��dX?
>��a�+�T��)X"�dL~�(F��Aj�_�������R9�U�"8��aY�5����*G��������s(g���������2����(�z�#�������"5�[��y�W��g`���y6��^��17{p�-s*j�b���[�N�j�����r)�Va���u�s���������{����BO_�ckb��/���)�����{�E�[?��`�v�L��������������E�[S�+v������!v��+D�`������������\�
|4��AFx���A�y/!���dJ7�G`�i��u��=���
�gav4P[��GT�1G����������Tz��P�Ad���,�?���y�]t���������E �_����a�=D,�f��6���tL�
u�9G���n��8H���� (+��l�)���(>@����������o�f��+
o!���>>���Lkbg��I&�_��p9(������}��yr�O3l��0G�Y�V�����q�������g��Y�ac�pK���8�~� �+��'9�WpY���}ts��u�8��C;����NGLvNOF�\.5�/����W��v�J^{���m�$i!�a�"�U���*��7?_����x�� �6cx�!Q�&t��HR@
������T'����C�?r���`��>:��{��a��)J���m����G���:&�XXg��f�5�"o�X��LE��������C����5��}���C_�E0���$��Q{�'!|�����`�?������^[R��;('�,��G�KOb������=����s�gE4k�0������2�\��?V�~y��#���S:������O�=��
+)�-?�5�\�b~V���������p��t��+�imp��'������&�Kv��r�:p����!�JM��s��P��N�+�OT����,�����+�����i���Qz������.���*�m�e!f#?�W�U�qdX��v��2��%��Wb�F�������A�7M�heG��$����U��f����X��X�t}w������������bN|o���-�hm(����5���;�ATl�����3+����EEg��Y"5k���=��-�m���^m�JdQi�?���6��d�1�����\Re)����}e+H���Xn�.-!��L������A,���B��F%2-|����{�����p8�Z��<�)��I��a�'�x�'>�'k6����HAw(R��'���vU��n�u���"������R��k�������_����'�4\L���ivAx�O��	l�pk3[���R�V����\Y������5��n�0�����+�4�����������g�Zp@u(a�Y���1�)"0�6���5�i��':�����.�F���<�|�ap��
�r�����{Z�2���?�g6��B(������������)~ ��^���y�;>�u\|�������}��!��BH��^���{�U��C���-2���=w����2���N)�1M�K���>����3������?�{q��d`Y�
~�pbs8`��5�7��g,$������#<�!��F(*�&��x�q#c�s��X�M}��G3��-�l��tZ�&�]��O"�qc'1�X���] �T�e�N����*��1���q����\���0Q����\�u�.G2��Tjaj^��4�5�Y�8��#
�������cJ�b	a���U�x����`G:�ho�F��1�$����B_2�Vy��y�3�n
�g �0qL�q��
A�:3�a�AS~���
9��E-��0�F���	+�c��y�7�i9��I�����i��E��_�R��99&�jT~H������YS?����CC���{�zlgg���2I�"��9�]4j����$/���)��	�z��2A��Zm&��&����{�I������k���9��R\�u;RW�������A�zb8�b|�3�o��.��s��t@�a�����,2?�fQ���I�,sw+s�"�c*j���lhJ+h%�=�HJ�����gI����������k��SNT���&Y\L��!��}l#g��h5_��p�Y���h���ITF�
�&��Q��"��@�#�@1R��u]�H�K �`1]��
3�W��sQ�q�v@V�3A>�y������
8,�!���� !����D�L������e)b������]7��	�O��`o��k�'��WYThcL��Pm��He����J�)������x�<���T�@g��a_����Q�1ZV{A��H_ N���l���I�~���Z��+���
�������N��g���+�������3�t�:RW/�Gz0_X�:��	Tt�0��n���3fV�X�s���]�a������7�~�r��n^Vf��2�|��d�������1�����`v��Xmvs�^�!
��!�P@���C�w�~�2dff7L�N?��!pk`*G�MF�%��t�1�tSC��(�z�;#]������U���E��|F0.	J�����Lg	_�|�6qt�Afmv3�?�|�~?���S��L�������mB�����d����lj�`24���J�
�(�gh�~���X�M�~��IV�#�_2�
�k6��Kt����{qn�"�qt�����{�b�=�`�=��3�����n�zb��,h�#f�`�I?
p�\e��0H�:���#��P��S}�u���ck�Vb�����F���YF��n	1���\1[����x	��d����3��/"�jq7�dYS!I��C��^��b�"��-$G�z������L1;����A�a6���-;*WlKr�;V����������i�[�?�m�N'Aqz8���,lG:4H����Y�~�t�����VR7����*��
�RvY�(��pf�N�B���[-�!��l\*��j<=�Lc�3����q�k�M���lLp{��z�\��*)�T;Bl���v
�0!j�v*.��J����������[LW9)~4Ia���t�z��S�H������Vudh�^����(p'tp*��8��,�QZVmG���#����>x�x,�@^�uQ�)Q
4� �6W���U9��)�/�y�)v`�AWL�p/�k[��i\Ne�������3��f��U�w�G"f(~J��s���1N�����>'��a��CJ����w��Zqu��n`�=�j2AI�y����EW������A���6:v��"n��E7S>8�����=�:����^wt�Kc�����
����h�`�:���7�>�`��T�q2?)�K��M�5DB��!�9��]����s�I�B���@��[J�"{�[$d�p�-3��Fh�syv�%C�q��62��8en_��`4B���H�7I�D��Z�(��v�)X�����������i`����f@�*�s'���B��3��������s�����oc3�Q����<�@�9:iN	��C�����O�b�	%y��qjh��1n}f|��:n���2Q4d�N����\H��4`���l���\xm�<�x�Q
����<U�s����������L���3��bc�a���p5H,��L:��2f�|�o����i��P�_w�T:4=t._����7�fD�RfR��\z+�Q�����f�4���F�Iz��>�Q���a� o
��� �%����x|v��>�<B��jP�%�z&�:/���b�B�1<��+��.Y,��!�{�q�w��\���ES��m��q%w��U�I/��lk�}c�N����U$;X�<����,K);����N�a�3��q�~-��i��2M���BeX�\s����������!��?��v0��.����
��cG7�[\���������j�J�u�4|�/���j}�ukBC���_�A3�������������1��jCR�>�M���7�e}k��������5���Y+������`~�@����{��/��,K�I�������O�i'L�����{���G(��q��D�����&�d���\�%����e���A������������gi������3����6�u��7�&�s���P�^l��cj�lu ����%��W:"���aI(������8r���&e��EP�8����
�K��Q�3@���A����C�Md��Pr4�uA�G�T�j@�������_0������+=�&�K�<�z��v��uA����~z��5�(�����+���^��������)�T��Z�,�C�T�����p,�^��yHc��Y-@�0�"=��'ar�-��A�h����
��l^5���P|��e����(
WeP �:?0L�0Wm�tQ��I���4]��������v�s:���Q���!$� �'��������^�$�c�Y����T�8�������$���Dg9�_��l�-��7ddY��?�� M?���B������	l��H�<�V�d���h�G<]�>"
�X��B��%��������L}�/��5�������M������A��j���5��
H
M���D1gmt0�J��JD�k�%�7R.��H��_��z��B��F����U�C���O�^���C%�	��/��%���Y���|�S����a�X�5I��K�`6���p6dZJ"�mQ���5Q�����*a�t�0eZZ�N�	����,�33�LEXmr��B�70E[,��vi��$�8�IB�8�_���<[��l����S��w��h��B���P��T�Q�w��}4����j�hb�����2�%+�}��\
qz�:m�#O�8l�=��J�G�YV���e�B����Y�F"z��P��W5����
��AF��k�&����"�$G�)9��4�?��r
j 4��M��eK�L���m�2��
�S���i�t��y��#'�j��h�x�g���	�f4�>j�_i[6k��Z
���������`&���,���]�VN�o�{����a����G�����������$��\����ZK���;��l��{�e=�1�x�:�����@��A{�S��G8b�'�df>�~����]�K�*�G��0K�f2Du$�a
d� f��{)�"Lq0�~��b�����	��<�H��-��P�������N��o�������
N���D���P�VDC ���1l�A��q��� Uo1wT^8�����K-��Ds�!���9\ G@����{���.R2�w��[j�O3_�������Vc$m�� ;<J����y�����������E�02�z����	�j�����Y�`�V�aY�DK��*�kID���$���������M��%����� 2�������X
��w�b���5�N��4E�LA�����4
W5u�WJ2
��k�K�O�hr����ua6�s�3[���w�w����{��>���8�5�17�i[��6���E�@�1r/8���Hn��>�s���A��&�17��Q��)1�.��C	��Zt�����wx@�Y.�cR�t�\S����!����h�/l���n��&�/�����
s�	K
F�p'b��� o:;�A����}T���T�*��|@
4��+�/M��17���0Q41#��S��YXq��0,d�W�GJ�+�X�90a���/tM>3F[����G|�;��������a9��
W���\!j+��k��l�p.#��rSh�sH�d�cj
�)Z��'��z��jds�G5M�Ge�������/����X��cd+��z,L�d[K���L�H���V�lV�,�8�����%Z��f2��4��f����A}>�D�4q=������@�Q��������g�)$:�T�"ey�i���2����b���L	S��igy�h1�1.FVZ��b�9�5�ZO?�&��BD�F��T��py�'r��� $,����o%1\a��7y�*-��j
&_[��J��O��"E�����R6�6��F��i�����T9u��z���zn�6���V�NQ�����������cJ���jP���uI��K�������D�^���9�R�&qRD<��@j��F?\�SJ�P�����`@|/���E!/����kw 0�� ��Q;����kR�'��n��7)&�|�S��A���BD\�����b+�[^�4��M���d+�n����h9���&�D��T����e����hFh����B|��pv��@�Tfq�,_����z�\/�5j�
������%g�e���(q��e����m��`&���l��+�@�&�5O^����md!>��	z=���u{�"LY��������/�?�	�"�����-���B9�k
-Ra�Z��@w��
f�(
_bm�BP����e��MU��^kL���
S�S�K��+�V9|��[;>�������FiI
�+��/t�G���S��5.2e*�4��aq�����ZF:�e�E��lx[+dG�X��G*�1�r��p�XL~s�6T����n.�A�t�JT9s���"R�RvK�x5,+!�:!��H�������>gY���9�U)F6B�o�pp���������u�3O�m�������>�����Z��m�\od$? ! :��}E�	��C���4���5,���������E��{����"��r�M.T����OT��HX�	�0c�����"�@��t��^���{��6�(W���G61�d��RX#�>�x<;q�:ER��S�����Cz<\��g�]�"���]*����W�.��P���#�^��� D�������B��>w?��k �P0C����Z��0�Cg>9����VM
�:��Q�s����}#�Vd;#��
��xJp&��������1��| ��P��26A��j��(z�~�,�v�9�ji:m����f�,�bR�gd�f��N��`����q+3�z��� ���i������u�h����3X�FH�!�@����2�&���!Sm��
�6[��#�!XE)���EQ�-��i��	�1����!c�8eB��U2G�bC�&�@��t�`s�B6��(eUa�_�),,6 �_�-/�-���`x����QE�����,a�"y_�f��YU�/��%_A|"!�\[JL� ��!���g1�|>��'�H���`�N��	[un��?���|�i?TJ���^Xfy�)�0�E�|���AJ��m���P���#i�(;���%�C�a�+}����(<��K�9���SX��&��o�O��\,��i�|�y�&h��b�d?�N��D%�$ua��S)-	��� ��q�}���t6�i����+��3�W��M�F��J�$��5�N>+�k{�|R���
���+�
 y�O�~��
Z=�����Q:�������.['���O���VS��~����
V|�������������K�f4��.J���.��||��������D�+~
:�ct@:��ZB�B*�WV���E+fc>�E�fJ�_+�uE����H����_���x�C_���i���X�$B7���e��=M�epr���6�g���������3<�tAO���:��ug��>�\�^t���lZF���o��<��8�.l,X@+D&�5b{���#!�D��5��+e�E�/%J�R>��@���AN��,�����4ft��D��u�������J.�|e8����Jw	�Zn���/(7{M����?����}�B?�FH��j���T(����4��,�����4Du-�����%!���m�����M<P���k��|R�4a��iM��J)��P����� ��A�@�!<�E?z���dM�l�V��v��m���7.�s�8\xU���d*i���������y�TH����2���r��{��z�
?*	��L�GJb8����W s����iw}8["���t����W�s		Z�m�hEt.<7@^k���� ����@�����DAC�$�a\ ��U�>h�a��=�?�]��/c�n�
��96��2B�2�$�e+�`�G�&c�,7���t�	�D6�'j�G��oTLIh`���L�
�b�8��H&���O)
�K�5��t�;7L�	�(X�t��D�tb�$� ���G&�Va��)`YR�m�TL%�|�l%�����UtP����qR��������.L�x���k���>�c��}�w�Q4gs����8
}i���e��trI����5n6�jN�b��e�s�~���,�l�2tH@'���\����2i�=P_{��-���Zv8e��I�rr��l%��oJ_��oJg�p��q�JX��Z�Q&����
y�1K��pq��h������-�5�e1F7h\����'S�]|6�*r0W���e$�"�w�c�Y�|,I�@���7F�P&$� ���il�4#E�s:�5��Pt,b7����n+p��ns;�9�`�Y"3�T>������I%v�^�5j*�(u�:����O?8�F���;9�Y��<���2�#soIi;$4���B5r�+��	��vt���)�!z�.:�Kq�GuohiLk��
> U��H��0��sm�R�+�^?��}
�;{�d�,7j�9�h4���������H�pN���b��l����!�MO�2�hzpcy0Y
]c���7'���Y�	�y�����-'�;�k.�k�w�kg5S��e���Q�Q�^�3`X���B���R��O�
�]������V*�WF��7w�*��"?���$kJ���'P}H�2�0�v4���B��B�t>^���%v`�8Z��-�����B��2R0mr��D6�5Yt�-%J��%eG��
��1�Q�IBS��yb�6%�!M:l^�FR��g�g���i��p���4��px]�S���cU��������e7�}
BB�I6Y���=��d�,���8��e�u����r�I7`���������9��`����	P����[B��Wx��<W�s��%%�\�����z�]n�D!�`������)����.��q}�
O�u��j�|��+{�*Y�njY�#[If���+�����yF�[e�X{�[�����\����f>���a�e��W��1Wj������xX��T��iQg�p�TlG�9��L������Z����B���b�c���9�.RB�91�_�l4����=��MF�]L�ej��_y��X��k������a�c��?�������Q���|�����������fq�=����s���sE@��E�=�i�F��'kz1�`N���7!��s�y�?d2@�I�P�4^;���z�������[�J����^~�pr������_ON�28����w?
������G
F�������������2k�O����;��_�}��hn����UD\}���~�����������]^*�������������g�����?��/�q�XE�o04��t����y_d�/��y��|:�/�G���,��O�e��"���X��)!Gw���B0��)��<��+6b�k6���l�������a�]���Y��Ug����&��]����j���P���.�����9)�P*T�0ZT���}I�K�!�]�ib�s�Q���P���	oH�T�	)��5��w��P0�0h������
����y�c�����JBS���K=B��k!xOMt�~S���������������P�|���xv�����z<��o���yz}����1K�0G<e����.0Z��h��m%�T������@L��x����.l����J���2����CZ��!��X�U��^�3���"���

}���)�Z,h:	�S�%]^��$}���O:��a-�%�������$�a�����
<�5�'^F�&w��(����lW|@Ecj���W[)�����sKbP�=�eJ'��^���I�3B\�eH�>U+GfG&8tC�����A}4~���[��B f��x:"��,/���X9�e�v�n�I�s;�{L������bm�=��2g�:Q��D|O����w��7�s+��5h��_z�-89evD��������E'l��>/���gb��K�0�d���a4X������m��x���U8U1�)�2�����$mp��Peju�9���C�L������D�����|�-�rvIs��[l��%�ZL���LN���uW\46����RA�w�h,����*=b�??���M[>a�@��������h\�w���������g?��H����������o��J:a���X����4@7�3������i|2�����m5,�W�����i����P�c����QN#$	�qFJ?�jyT�-m����^��������k*�i���Sn�%�0�>j��A;���c!e�O���@��;J�HE!�4���ldV��f"������z�I��(��h�F�Uh��"�I��wmA:��.(�v}�NE�'�^I�H�Q���H���4�.D��p�K�W��W�z�N��`��F>�����p�����#��b�6�f����V����>�j����Z��jZ�4L���W�j!�`
������r�Q����En3v�����3F�����M(�O����NG�Y>����#�G1W����QE$j1�
�a�;
��(K5�P�	;^C^Q�8-bHY������l�K�jxL�
���<�tnt�K�0��(�|��%��E6L;{{Ig��(�.uV���� �>��c�h��Xf�)�&��G��:�`�m{Y�a��_��r�%l���$U<�����dM`
;B�	Bg.�d���y�2�����$�4��O���b��O�~��d28�sJe�tYr_���F���#K���J��O)F�����#e�r-�����(7���Y:�2Z���\	�%/�vV���]����g@�-K�0���Eq�
���q�\1M�	�q/���p|����?�~4�	��x\H�k�����^��gYfQ��k(����!�_�4�����}E��>���e�����(j����78�J?�`��j!�H'��xy���V������s0�T@����K+nr6'uyF��9���O���	������&���'?���>��l����D�E�cX�������DC�i�.;���@�6�M��<JP�����
��
�C���l��\}t�/S��m�fk����\��t���|u}�1dL�6��Qv��������'�4)h��q;�F^[]�����j)���i���?$�������VO���J�It���FL��\��Pc

h��0�J��1��-0����9���� t�����S���tx*Q����������'?��#�3��
gw�����0���{{���������

��A�5n���(�lvi����YufOW�*7��
�I���*�����_s�h8
�F�a���L���_��D�j]��=\[�OO~q@��`���������������8������� G�������������Q��m��4�
�u�[L�fA�$BO�������{p��oT��Asw>��*C:x�T��96"�<7����Q����8�u1CsTP /�.�Q�%Wh���
���[�d �C�!Q�'TA!j�������o�F�����"*���_;RA��{OI7���:PC�g8s��UM//E��;oY�!�������]1�q��V���-��d�'����;n'b}�"�������������|{k'R.m��E�q���T;��R��/����l�r�l%�%t��;j����ZB[T[	}im%l�<_��g��JtH����A-��&*"�[2���2g�K��u���)��QY��4	�YD��@���\����l��p����<cv�K�
�J`uT�NQ�{	|��$h"�����������"�;����*[<�t:��_�8�����*6[�a)���E*������=�w�;�wz��&�����~����N��z��:a�T�V�6������r�p����^�Iu~��������������Q���~v���~���}���{u:���QLVs�y�9�~H	�D��9���� �9����po4��~���W���?�Do�B�4}���y����N?�l}X
�F��g���r|��������������������:*�(
��A��\�i�m�<K���xQ���,��l���Q����.���
�GQ���%�h��{
��7f�RfX�;~�@,��������O��l��Yt0��KC%q����9s���������N?o
Y�A>�d���}�!������#hd,5��X�uoi�u������+f@6^.x��-��`M�_����c��N#8KH)8N�p��C(:�v��q�U���#�~K.���9>;9=~���a	���f�]����gHc�@4<���L�BJ�����������T�4�>,�L�y�F���P�A���gKd�E�pcU��PkcK�����DY6
�#��5y�L���T4��*dHmFF��XX�C	OKj���u�y
��C[�����h���:�I�B@������]�� :���!�[�����6����&���J����?��������� 9����Q{�����4	g�~���deex��'�����}/������`X�3�����n��%�]���9�:��ysM�������!Q#Me�d����nk.���A�1�^{EA.t�,���IJ7�����Q��'� �x�XeB��-����?3�*H�9M/�i�!��[��!|�e��_���f��s�[�$�B#���������)T
��=��R���;;�5������1n�~pt@>����;�:�+������9/Z�������f�w�w<������dASD��,����jSY>���d���
#��� �>9H[;����	z����[���p%����H'�e�$�����,>2<
+����"�v��0�$��}��| 0	�^�5�@������%�C�t�\�A���{�����1�
��;���_�������bW����,��=e��#�e�}���b���s?��)�l%�
������>�7iA�P���R;��X���
�&+��*�����WN���d���Ay=4�x���P��E!E���l���dR�/RK)�����H�|/�����Y�`����J�C�::���:Z!��,A$��v��&�F�]�����-3���&�oj�C�~G�����^YfR�K+�>X��_��Y�:5~�fRG���]�}��YEq5|�J��=�=�m;�n�Q6_�������*�����2�L�&�w�6�q�u~t>����F��i�fWR^SQ2c�,���!�:��{�~iO�D?C�^j���w�S�3���3�,���h�FcCJ_$�\�����NI�)is�a��Bh<&b����
E��\�L�j�S�P]�_�m��n�?N��)=c�o�����n��U�F��7��==���A���c������u&����e���7����*wI�fA^���Cy��|)�Q0��2"��-A���R�ne�q��*_����Pu/�N�;��;=
������VAS��9����I	I�����H��d��s����T� ��>��}��e�^+
������$�����l)�:��J���45����R���n���q9���y�w	\`��3��/��jA �cq��"1��zT&����5M�ax@y&���*��y>LE'W����������bgK��W��d
/�39+�Q��U]%�3���G��1��ek��� Le�;�,��xxKL�@9��������P�����������e�]D�����ith���q���A��^�o'qfaK���n��m�9M��O�gP/��4�2i�
&�$��R2��$,*���%YEc���2=�N&Xk�]A�k�*0����"SxX���1��2&N����&�;���T�H1�*JU���D�	C�k��t�"0q��
K���5'���I�x=>�!C��$���	��|
 ���T)�!1�Y�i��Y81��`��q�]���
W�����bFC�S��k��B���C,���n������Q�r��7dI����
@�Y��B6���|����A3u���$x�TE�TE�*�2[5���UNa�9������}�'�E��mf��tu����X�����o�w���;�q���CRN�l���J`wEb�����w�q*���Gv�hGC��+
Q+�uO��aD��J��H��nf��i�-[�b�������!�!e�@���v#���X9�����	��������^��{�w[2*i#�z�c��S�����������������y��q����Q�z���`��8��r��k��)�������9z��y3�B<���?c��*n/0e��&�����E
��SS
��d���4�����{{GG��p�K~j�T$?5��&������1���E��DK������?����
�}����v�|�j���_�	�Z���/�����p�
F�)�����;�V��;k���]U|L��;��7�af7�bS9�}K��M`'JP��)���]��O!�._����B����Yj�������W���^�<{u�:]���<��L`
�LXbL�]���@:�����w��
Rh�#��K?Qo�&/*����g)�%���,�el6�#�Pk#�V�_��`��~BE`��Cq�fu������V����iU��O�b�<�Q��1N��3\-y�$����-�,<���OwV��
����A���|�;���Ap7d!T�h�M�}2�f*6��!�\�y�T(��
Rd5_��O>[��5L��� ���\o.�v�1`
�mT_�������u���8l�	�0	KrE�ZS �2K��n�f�N'(x�;�2�vLhb�c
.ZW��,F8���t�i9a��8~r�/��Wq2
��R;��4�=������SH_�l[k.�v0��TM�zs|6x����~n���_����#�#��	a3�a�%J�i@
������(LHvg� �����������??�~�����������-��<�D�����n����t�T�gv�JXU�j���O�NN_��	�0��P��,�?:.������F��:����0����������G�% -�����T���Z�E���?b�v�.-�Hyk�~�*C�or�O@����J'���w���;
I�}�n��rj4|u3��!l�eY��@�t��B�_2���JZ;%��S���l3o�d�����Xj�r�v��ifS���t�'�C�n�`@�J����f�Y�wb����7
���X�{���B��PR
����6�.�!X<�|r���jK�+�����t���b,�gi�� ��AB��DF���xb�p@�����&7f�U�dxU4p���\��w���������$\=)����10���+-W���tX5������N����c�/ \��f���/}G6��cdg����g`O�d����u0�+$��.���%I�Q:��d)�i�p�N%�#�2�s�����|F��~��/��l���%��Ku��fp�1��}�6��������Q��@�7#����u<JM|Lp�\P�kI<����AX�R� �88�`�l��E�f�������A\eU�Ajg.�����2"Q-��Y�J�BN�b��rbP��q��PH
&��'�"T1RV~��'Ft����I"�k���3��Kn.��[�m�3����xh�-C�#/k;��>�8�X�w�b�[D0�c������o�"���&ro���������g��8�2�e!mXq*���)������F�9���4y~]4���\���d���U�D�{��t�g���5����D[Q�Z�-�����.���������F���J���!/���<����:>T4���[�!��F=��<F9/VC���
��j�}J;��FyN��.��6(�c��]��kl,�Fn��AHM��������	�N����X[>�����R"�*��:�vcW��
�[0����J���
Y>��\
�R���|���+w�=��k�#*���������d\� yY�2]�z�����~��>EW���+5w����'3����Z��������&�[mq��?�{�3�{C0�sBesmCs�O��#
�d�tj����.��>�~Bb;6���fB�RJ�$�e�X����J��<��;��#��������<�M��/�Y�i���-����:��xIg�I��lX��jS�Z�|��-MT<��� xL=		U�����A�Fx��g����/��Y
�g�,��5�o�'t����#jM�	g@	���B��TD���	b_�g1'���	v�f������6���b���z�Q����B!�
y�U�E�g��O����ZMco�S�S�r��>K�S��J�H>�9&y������~a@5y�/��;�0{��m�2K���)��hB5e-�q;b��i�O���m���\���2|CH��
����$h��9�R�/,��I;�,N�p���P^�{,y�3@��`��ZU���7�� ��EU����=�l�#�RU��	`����`�'-[��B�1
�~/iz�1t���JB43�����"<�����:a��4��
s�~��7_���1���l�/�L@v*S\3{`%�5V����d���L�����	�Ja������b<_,I+���rl��e�M>���u����Y.:�lo���z�� @��|(�\�-�q���Y��#l��%��|oG�'x=(����B)��L��^djm��u�
��^����pr2��'�%f#�=�P�z�-����[�	�H���������C`�|D���8[��sHk�i��X��k���_�$�z��dC��z:{[2?~xk[��]�������*

AE f�x"����#"M*�G�E�O�N��#��2��*�$�����V-�P@g������FI/������p%��
aJj4#=��������e��&�������`����1�Z����N�cgFzK���61��#U������r��T/{{�Qv��O�����)����xCJ/)��t�x��z��c����d;�f>���y��?��Vb�;�m�
PE�7��r[�"����W�#���e�����H|�P`]+.����+�7��������qa����O�����";��w��}��dY��xx����<[S���U���W�����>�<K:����,�������'
�	r,�x#��c����*��04�x-��
x��G+*��k���	�G�$b�����J�9�>�d_���&Z�$=�?�!���}���V�>4v���-�qk|�q��hQ�E(�]���7��c_v�C�q�O~��{�o.�b�ts!�"�"DT���:�R*A��W�}F8�@�~�����0I�{{�$9����URb-�#������I	��8��������=����{�N�_d������}8*���/�.��`���o��p��?�C���E�����<������� �����������$:;���Itq3_v�V
^l�hHo#p�!?��D����D���~:i���J7��7����t��B��}89=�^�;{/+�p����Y4n��1�S:o���;������d�����oe�8@��EJ��#��v4����=�2�#Y?����'�z����������kR4����_O>�&Z���t�����ON����������QkIsY��c>O���g#������=&Lc�x��{�W��_��~���X�������;q����_��-����"�(�O�C+�:���B�Y���0M�`7�I��V0�����V0�t�L-�K;��gl���pLA�`u��wW��G�y��O���7���E���?�} {�
�W|��K��xo�^&{#:@�;��_�{G���T����i�G��>�6�Z���B�^����'�'�X��Q��^t���?������v��#V������������yq��u]d�f�%���#���.��_�~1�4ej�!��F ����)�;�\�V��x�'�r�����I���"�n�?qY�����?����~v����I�S����� 3��%��}�A�Ay���?��G�����G�dFzrFn�\�L-�b'���!��b�yc�87�oN^���a���1��4�������F��{H�s
U���a-��h��=G&U�����n}e�r;�Rp�E[�^�~J�|�m%��s�r����@����s���e�C�F��s0�����$�%���D��i���T!�)�f��a�`
���Z0�'���Q��
b� �^�&(;�j��ps]�<�jz}�b'0��ZwA�G�mV�|o�����������(�CL����J@]�6,�=�ux����e��9�`��o�$Y�����+�N ;ua@*�-��J������m�(X2�o}�H��Fq���Jc���JU�{���X���C0l�8d���<���~���|o���"y����H����j�`�B`��17]��%=�y�x���g���%����0Y�&"���1�FL� ��A�FP���_������+����^�w��m��u����;�N���0��X��U�j�������'�����L�����S@:�%�wc��t���������
�L�vb|t���@�7J^p~���Npnm_s7����y7�7��x@L������V�s���t��$�*t����b���a%�Kim{-�O-@``=�ot����Q,
�_(�b%���/�GG��c�k�-(;����'����
R�I�~h�����r��\��Kh9N!1F��������-%f���W�������(K=&D� 9����� �o0d+?�i���j��].Q�%��	9�
����c	�e��8��)�0b���p�����-�-�B���q��^������3IR���s����(�UV���Z�u�-51�q�d��xB�'�M\S��;��5���QX���4���������#M`�
�h�m���w���5o�T~��@� �E�6!�4�J�jBo���i1z�?�E��Y���?y^Qa�����
#�Xu%X��+Vt]���nl���a>:��=�G/o*+���zO
�RE�J/7��7%��h>8���x��5�I1^.�v�='��1'�8�G��p�f��!L�����%�i����6��G�nu�<{Bu�����$�}ZCL���|L��}�$�e��$�	-���=Z�'���2=�|���Q���,������V7���9B�<Oj�����~�Z��#��5�h���>0�p�)�*�(>��K ��Qd��
��4a�Qx�����+tN �rv��t�o��+��
g�_Gd����6@SyZ�6��	���"[@
�����}�p���Gn}l��o��c|Cn���~�����o��M�P;".V�I�������������j���\m@Fzs���T��t��OJl�b����]�E�e|�O����[��|�w��=�	?���&�'���p>���[�{��^�R�
D�9������Z@}�����)n���Z
^R
��1��${cVwV��X�S�	����:�q����Z���yTiNeN��9M�s��ir�s�j�1���a��!i�@7�s�8��9�������Q�rT*�nKg��]h�o��i�D�`9���I$�
�j�?���hD\Q����V�h�wz����(���������b���+�p
��ngM��:U��v��<L�N}�3�����Q)X�@���;�m
�C9��S�h9�+�{�`��d��7���4�=_g���t�y}sSR �0���j5���n+�EI�����7�pb��0��.�k]��&@OcZ6��2��X/�4$�|��gN�=����\��b�<�Xo5+����v����{!�2:����V@�|��=��/�(3����R�^g&+����$|F)Mf�r�����`2�,q#�c
X��{�7��m��vsK��|M�n�e?d��]/��]�pm
(Hc|��r7��0�a�<x�"�i��t=�2���|�w2J��4�����T�'E�����w�"j��*'�f�P��3;<y����I���m�=�It;(%�"����;,r���5�=(�0�<�.)^�r���r�]�/��.V��; ���K�'{:�O����{��,:���P+���u��t�X������	�=<�
�z�X�|>�QW5u�F6��m&����A#r_���]��];EoI����%j�c\`�A��n�{�	���;�����9lb�Hs���?�;�rI$��$K��3N��[_�X5��)u�noG�kLr��i�P���XK��Z��ZC�k�}�s�)�La��@�=d=�[����~��yK�Kr�Frnj8C� ���n���
�So�t�M_G����}����<HV/�C~��m��=9�olM�2N�^�,�Z0��_A��V0�CN�������������/�=�7��c���8�DE�B��KC�E�mR�p�_Eg���B�q�W��b'k��5�����'��<��&�3E{zM=YS��'���=����eM}gM}YS��'M*X�w�������2�����I�k�#�7Flo �D�Y:����oI':������X�.���v*��m?{������*��Z6M/��9����E�l�L9X���1�'���,���e�1��cK�2���Z^�#��)��#�2���S�h�d_�q�t�G�W�.
)��T{��j���������q{�A���
�M�(�=s�*D���4%5�������k��@[�����E~;�#
�|Xr���k��h��4��&����H���2k�#[MeAr�.K�%[]%�r�-h&
�S���1�P�1���0�;�_��W��W��U���Od�pCA��6u�Jl��8Vb;�Xk*�d�k]eA���
����/���c�O��M���XA���Qx�4f$�
����U���+D�����Mh��5<Z��EOa������]|�i��5))���Gj��;\`��������\4O)���.��������h0�R&���Q��z�(��Q�
�Fyq���!�bi"(��&'bi��/�� ��n�S����G�x��;H�rv���������TV��i�~ �Fq�z�g�q��>tR�Yulb�J%�CYt(Y}��zl��(z��1q#��r�RNSp���b��I8H$P��.):�d[�3���d�l�j��w�,z5 y��\�W"n��
@K �9���������a��J6����;����<\�Z,�&�H��3�a��%�WO�_Mbj��)G�Na��m���Av�"5^P�]�{^p�$�L/x��Pm
�g���������z	��Jc-�=��^5����o��veB.<&���+��
��2%�"-��k,���s ������P�^�	�� �JI�)C�jj��@%:���!����F��Sj�TU���Mi�\�������+i/�T�W>m}���wI�Y<1�[W�f�L�w��B��z���y�N��'4-~���@R�yWP�vL�����b�����}��/(V&�hl�D���K�T8��f6(��=�8�N9����qHFe������z�CfS�B�ea��[�j�N�r%z������W��6��hhc��e�+�,e�������_�^����V(������z�_���9}74��Fm�F���^��lA]�������+-�x-V��W��s4GziA��]��e�����U����d��:�t�����0�����2v$��{�.��n[*����~��AF���7��|.a�4$��dL�>?�)x���k��w������%N{�G�h��I����ct��5��K�tC>�V�So������bn�5��(U��
�x��w���S�c��'�Js�T�9����������g��p>��
�c'X;�[;X�F24[
kL�m�u�`}`4;��ub�u��Y�����
������K�<���gk�N�m�u1���:hv�Yq/q��B��m���0K�A�Tux���0��U��b2�_���1V����0:&��_�g.�F��/�z���4�2Ei	�C�X���������Iq�&Ca��)���ZQ���b`�~�`s����lu�{U��A�EK[YE�_��py��:Z�l��5v�R�b4���\d�������!�6�k�zk�8�W�MK$k��6���p�'=��3#L��GV����-@_�������/$����4B�����0��
��r��s^?I��8z+W��-�zV��}"`n?�Z�;��J)g:%�N��#J�����6"�:(��o�h��e(������~����~V3�)���)��B�S�����b��'���A� ���x!���A����@�����Ua\|��\���-Yc��������Ck�~���:1���6,p�]
���`
��E ���5W�}�A�${��l��:�_fa�ueS��\ ��
g�����s���|9�6��k�������[\��sPz��
}�7F����������+���\p�:A!:HDGl�����d�7u���s����?m�:��"9/@��R�,DU�����\�L�$�7�L�To
Z�>0�$l~�{S��MU������L��Co"2������E�l��x���{'d{������\o�!���2�+�W`�Q[0n��:{m��P�!xH��Y���V9�����%�x�Q�.j���?�b��g����$?\���i�j�u���<.F����tv��6
���-o��%����o&X�j�nW�j�8[a�����j�u��O�<�H����tv�	qo��%�n{C�-q��0�/|CU;�D��~�����*������@�K� �=�O����+�Vs&?oK�F��0��m�/�
p���f!�m2��?��[�k����ObI4���S;*U�:WJb���'D�T�m&�^���(�I�jt�H�c����i�{6��D/��"��
�^`���>�����m5���$�~��H<�n/����l�Naq�F��P�{l)9���H������\�Y[J��-%S��#9e�'��s�9W(?w%I�&�k|#���+U������fK��gK��d��|W<	N;rfM��gM��(���p��3��=��VQO���\���+gR��][��,������'�.T\}�b�BYk�_���P�Z�*��P�4F�5�Y0 ���Bw-\�l�|
"T?���b���d!a����g�r^���n�
v�U�
�*���h��'#���W��QZ�N���u�~�\�d�����O�P�������P�1.��!C
�+k��4��4��4��i��i�-�Ms�b��i��T{�c�m��4'�4'�iN�����Z�4+�,)�|��>���V�+���/F�v��%7Vl)[����J,�z��,���<��\�R�+�.��Y��%�r}^����%���Et1�G��b��K�7{6�aB`'G�b�n�*}���Yk���&T;�SwP�B�P>���v���{���<v�k���z���>���~����F���z������8G�B
=��
FW��5~
���Woq��a���2��������/Xl[0k�a��Zo���
,��Q�Z%7����y�'Kk�c��N���u����[x�	�����7������u��E-�n�u�mS3�-�������+���~b���t9����c,b>���x�n���@t�?��B?���M��i��6M��L��C��{
PP�)j�4�~:}��/���R	��7��V���s�lXd@�t�j~�d$�Cl^`'TH�&9�/pRb����SS�7�`b����!��a10	H4��%P�b�d�\M�����W��o>z��gC��?�6�Y��5^	�*�Jq��J@�|d��@���A�,x���G�p��V���}%�����/e�s�G�M�C�]a�
s^
z��:��;G�C�s�v��#�{%}�������A��H�����u8��cy��awt��v<�g���A�p
��~��}Nz��{�u��Yl� <��I�Yw_��n�AJ��v<���u���
�����N�u��
p={{89t�}(oP7�C���v<���u��}�6{��v8�v\uw��1��n{�v8�:�e�+oPw7o������l�����wi�y6I�4o�"���>f�(��Y~���d��"�oq����#VPx�g+���w
��2j�J�Un����c1LS���_��ukY��98�X��Z�>�[����e���Y2���z��-(��O�������,$W�U:��R�>'<wkymVh������l��ry�3���I�1�,��2^���,�&y���tg�����mb7��X������&�A��&������l*&�����v���kQ	���v����F��eT�4��E-��m�9
�s��IQu�9�TG�a�`5v���4�S�`�w��&k�n���u+��[!�okM��B��
q��;���������u���{���W�
�s+��+�
���>W�
6+�	U
���d�����t�S7��c6�|&�����"�E=^��s{�g�W���KwWS_�T����u�-T�=���>���������D�V��w��.��A�K�h�
FS�s����t��<����$IJ�Z�
���h/.������	_����N�J))��J��}�o�����`��Q��f����m\�^�%�);�?�4���{�C�BvTo����~�S�1biN�g�JtS>s�4�N�����:�Z���h���Qg�^A6m�p[=��Q6��P*���?������?�	)�d�5fePi^��d���;���!3D��	7����NT?z��!���3��3������6���X�]��A�����p��n	�^�?u���|l��M$��=k�N��{K����M�waS���	�@]��
w�\���7�Q)aw���F��>��p�#$*��������4gz@?1o&��/�sWFs���2I�����M�bl�b
,�K�������YL�,&�,&�,�A�k'1
�.����T#p�����_��m%�nw{X�gaN��u����
s�������M_��u�:�z]���9OT��Ia�i�qx�:O�)�Mh4E��wq�NS	:DD���#���G�P���XP��,����[&B��w_�����F$t*cs6c����9�+Nk|�3[��'76'7�Nn���P�����'�S���&&�&V�M\��x@n��J�XQ0��H�tc��=����r��{x'�F���8���o�#&�� �$%�*%�������|���9>;9=~������H&�,~���S���=�A�+�	�1�����
��iR
��Xg<Xtg��'���#0���b��&y��b��K}|R,�
%W����D���|��{J����kP��������Le���L9E���9B.\obf�#�dtZ�=R�����ob��*2�6*�W����7�$=uWs���h B���R�b����]@F-��F5��9dh��D�9\Et[�������K��8�:!��������?f4����H<hnda�X9���b�������#s�M���9�4�Or��|��:,1���Vo?��n([��/0<�wD�2�/�����'��%�d��,~��F�L��Tn��-5����}a���y��xE4�g�F<��l#������Q#����G�l���+�i��*1C}�����a�������yV7�@������H��2SRU���s�
���W^��-���
������
y���X[�������-B����g�����B����C2��E���1
sMtx�N�����K�c�Nni�!��Oujw��������N>&��������r��Z��,s�k�;
N�B���2T��-�G��-wH��0�����c���%��!�/CR�=�iO�����kL�aA�ZVw���Qwb,��],ab]�������gC$��6��t��W�ZS�Q��]u�W�Z��{�_�^-��X�.���I�P"8���e4����f2���1
�$C����e��������M0P���]cQD��A����9JG�(�|�Y{�\<R{1%b�����������V� �������R.�������%J���wRd�UJVW����1��P�����z��T%�*���QR�I:4I}5�b��*|L�B=)M9�1���w+��K�J��'o�86GV��z�����te�����04MA�f�OY6U�����5�V8�����u�������U��~�x����g��V92f�{0,	F�T��wz����c�z��X��Dm�<���)&�P��2����t�0��Y�B��/>C�����v-c\�(?#q���-�7<�W���[�K����p����nM�g;q]����8���LKR�{g����bw6?��:"��coVM'��������-�U]
�w~��W�l��N��).���h;�8��� ����P�p�P��g�@a](�$r9���x���b�l��+�d^u��M-09UH[���k�%��&��b�c�	o�`���S�9����d��{L��ho��A5h�A�C�	��h���&h�9���X��VxJ��?�<6���9������b/��(��4�L��J������M�W-�x�N����e���tU�@����o�m������2(���������|���O��%��|J (�L��`1���W�,z�� �p�������{��w�`a���m��jkkgg�����h������F�|�,�Y
'�����b4������"���c���cx��	i�rl�f6>���/�I6Z>��O��f6_���3����r:@�~�Z,I���l6}�.W�h8^^�7��!��������Ev�~>�$�X�y�L	�5=�}���������t�H'R�"J�?�����d6�-����������A���z���1�!G���I�7�lR���3���fo����diD��M������G�bLH���h� E���o�d��b5-�_F�������!���������.���2���:������'w�[�o�����#���r>������b�.�S2=�O���R�zn-�5�>Ec2�����������Fi?����==�>>����9C�v`�:���v��i�UBe?�S�iYo��>�&Y�iu������o3�~&��"]M���|������?�2�| ���@���@�
R���������u*�p��5���f�������b�R����$�����(�����6��������~�����B�?�� U�m��r�Z.v�K�r	-���!�����c`3m�a`3Z+_CK=��V8qQ$�J���`!����Y+�����tu-���1�����llEq'o�!�y�Z�<(5�t5�d���&D��F�,T����ds���u5�����P_��Pr��oQF�����$e4@/E�c
�)w7�w����mK~0��0X�A��4VBf�flK� �3e&�x��\U^��:4\�����
H)��$�������,o'y�w�s�<�����nt���z7��@{c�����a�Yw��x��T����{�c�TC����8b��5�"�.��HS���c������5�����n
'�x�T
��K��_*�{.V�Z�x�'{:�OUW[9�w�����z
���t�Q��w������|�"
}���#IK}�#�����\��\���
������u��A�F�����c����y���v��,�d�Q��������s�3���L�dw�7 �����3�k��*�i��;�p�i�m�ev��,\����:-5��n�.[.SK]%��Fs��b/�)�p-Cq�06x��cN��Ul~�|��_%�W��jf��A@p����-q8��vh��T.��B���2a_h�e��j�n��t&)����D�)o/�����������~fB'x>g�� ����-F��S��hd�+T�S�=�=ltj[�B�m��"�1F����j��<��jw�8���h�{s�����S��`��+pP>��W�kr������l��0p�cA�����2�j��#�"�.g���:�tUd�ev�	(�H��N:�w)���Tf�Y2%HtF�e�{�1�Z2����;���+_���������_���������5$2�X�2�;tba����n�Ok�>��S���NP��(��y:o���`}]g�Kj����x���
@	�1��x����/'�}+^n�k�������p;���D��A"�-n����Bsqs���[;�'���x�QV������b�0u���)��Ry��]�ym�P�h����S�ym�kZj�pqU;rJ8��NK��.�jG.Y�ueY�&I�vH�J�$��v��u���K���Z��R�������Tq���	������)�����(���g�~6X��1�g������k�@����>c�$�7.�������; ��f�=����R:�5w��^w�IG��~t�+#5����V��f�N�����u��YL�'�qs�%�_;�
[2�1����%�X2�N�U45�0�b��vb����w����[�=>~l�i&v����V��=����mj��9�������]��Kc{v�}k(W/�&\�����X�]���K"�YR���$�p%��+�&9%�HU�)��B�����N���;����I{��+����>a$8�?��M�1/xa��g�i�x��b�X��U�k�����/c�o�+����Nw5��Z�T=���I�1�,��+E:�n\������g���M����&��#,F�x�<���h��D�:�����5���u�h��-����XV,�{��t@�N'���&E�9��R���U1)r�$�{5"X��������JX7C�b�f��Tm3���W��s3�k,��}3Xg�|3X�W��q��;7�����+���A�l�I�������!d�px�������]<\���jA���7���l�1���@L�I��*�$�������	�7���
H�d��O�5�Bz����{�I�a�~�F�~2��d��_KB�M��;�2��3vO��m2�}��6���Y���m��U}}�l���#_Y������k����������������^k�6��us98��`��0i�O�4�}=�M�����b@h�
��l�{�-w�e�g�]�{XC�
 ��{p��;�������&�����~����N�9���!�����tE��.�V����}z����+f���������~���?<O�������"I��^?���Y���,�!���Q����d	���g�1������B����!��7�q���� ����hv�����||��2B5v��d!bQ?K:�zI���'��������Y�?����9���5z=���&��f�� �G.�/�kk�O���O�@���h�~������^�3�WBq�q!���p����E!kF���<�^���r�����do�[C��a:��t���f9��o6
�{�N&�K���DQ���v,�r�U�$��?�v'�a}q��G?OG��yf|
_]����i�nE	�l^D�S �y;�����8���6Y�$:�&}�����#]>��_9.:��]��z�E|�f����5����e1�vw�	D��?z~�����|=��EO��aGO����%{�M�[��S�����d��&O��s�t8�M=����Ab_D�d��v=}B���D�4�P;���r<!p}�9��O�_dU/�H��?��6�)��C�*~�#�8}���Y�:dW�Q�"�>&�����T����W/�V ���)�����������{Q�
����>=;i���z���?`�=�>��	�'���8�$],f�q
0ip�)��Pi�uv5^@��5�rD�`�b������7Z��j>[]^�ly������C��Iem�����V3�(��*��0W�h�"�a�9�.h9����?A#�%�r����|�_7�V�l5gK@k"_�x��t=p���"Q
�
�w:_�|��WU?[,����m��)���0Q�M���"#dCH���?c�>���{� m���7��2��lmB=>"�z�a����_��NO�����dg����a���1HD��S����M�����������~�����l�'����>D�V�z+X�E7��lW���=�����0�
��s?� �z,`<�2��.['���O���VSF bS�,���?��:�hs��G�B���X��B���h�U��E �}0�#Q����"S��r��U�,#W���`�@d�j��|�n�H�va
�@�g��8=�76�@_"���n��E�"�8��F`	|�GX�����4�E8W���$R���/g�8������ABw1G�u��:����	��<��O8�3 ������
0Z��f�`;HB����|��v/��p�P�U�� .B4��_l5�wB�E0!>4�N�;�S�OAs����lo/����^�^Q!A�zx���\4x��@lELS\���8G�l��OYD2u@Y@�%�r��[���hu)E��w[��C�! ��g�h%���h�>=y���������Wo�M�uP��
~������YP/@�:KX��~���x��������� �SJ����O�SGc���C�T��|r����x�X�&T�H���z1/���0��nF� �C�j�)"}����n��m���3V%���!������f*bi�e%���������v�����*�C���R�q���~�C�=�=������N����Y�g7��������f��	�L�����K
6 ��`1�/[�����u�_8��xFE�
�U�2Y�z�v��Y-{{�OA�/X��O0���������l$�.������
d���^T��e�������xw�U��v:�!��vz����6.�����~�g�,�������a�������{��{p�Y���(:^]R�o7~�<����C��2�j�@6���B�����r5GN�����pb��-=���[-X}�C0]l��*a� (��12@�N?se�u:�������m!oM>��i����y[���6X������G�tk{{/�}��������^Q�6���+`�@�+5|�#KJ���S1���"`g�p[c�r|P?�tr��M}
-��R�>�j�6�(~R\1s�U01�FV�5-�n��c������u0���[Zw?u�K8k�:�%�	�Y���9g:���&���VK�?���0����<J��z� �s����B���-.sA�;r�d���Q��
2[���L����A�S�>��X��W�~N��!bS7@6������n�{���}�	�:G����$����8��?(/�L�,�(�k	@����?����_4q(��u]���}�	H1���a�Lu�y/:�*Y�DQ&�G�*=���&�����"�
w"�yF�N �����5�WnD2��?F��Rbj�"'bAp_��Y�?����M8�	�q����`���O�L���>��x ,���>��E�A�����cY|a]�u�P'�1X?J���(����#�R^S)�+e������0�	��u�K9�wsO��>������(�_&%�'!�(�X�=�M�	Z��;C�+;8�s��h�@	�8k|�]����`�3�������?��K��T�1}�]\V�����E��`����M4f8~~����"������Q�K������0�3������}�)�������6������.��E�IGwC�/�Kq����RcP_jPd@���NR|;T�a�g>Gt�9�B6�m�iMj��`Rx�|������,��t<]�xM�?�pD��J�A��k�~��"'�����t ����2��V��}
����x��c��2	��9(/-�&k1�D
���,e����
�<&S7��T+h;����E���6�����q|*�W��[�)1����<�?���u9�V���P��u��D�=--�������(k����A�����#�3�(m���Dn�6B>�B�Re#2��.�����j������n�������$����b�;�R9�2���f� ���+�����M�Om�?�p����wp����lx4�/���;���~��..��/��pM�����w;��
���v������C����(��,8Do@��D���LQ�xa����No��JFRKS[�lz��<���5�8AH����!���5?FY�t�`Fc���jL���
l��[��L��)�\�� ���V����6#j�����L��_^��R�;^^��:��Uf!�%%z����D;rEN2J�u��������
�u4��g�9xP.����ZV��	�UL��GM(+ ;�D����@�%�J�)al>2�'��-=)����#@��b
�E?��(|��-9��"yI`)=g�5������*���dI��x��QPC+*�cl���2/��,F������cV,f�0i�%�������h+���H���=Sl�� 5��S���Mh�^�����&��p��[{.+*�����E����h��{��3Y�����c��h	b�������5��s�p�_C������gr�QH���Q�����b_��S�����d8�3�C
��5?��i|=����������D�XP������T"^�<�?�7���#(�1m���zs�_�xs28>==������NN��"���jS�w���X������c��hFL�I�3�����x�Xt�������J@��*Fn��/G��
�38H����O���l�z7;��'S��Rei
��8�vA�|LW
�@�\��J�T���I��V=�3{������6��@����h�i^R��1(Q�=�_)�B|�ZZ{-P�x���7�X5�~��'S�����1t�%c�)FI���3�M
�/"��*6������}�5~��I;�%���9�X���A���Td��&a`&*.#���-2s����-�!�z��/�w6Sm�9������*��bs~�e��m��P;�.��OA����f�������)E��4��r���c��E���\d�cm������X.ng��h��m�Y��Sh�]R]k��@�n+=�iV�hU^3�,���`(�"�����7m�e��H���f�	����D%��?��0�d��8��_�b72He �	�"�\
��Z-$������{)d��0�=X��uk�Ary��������Mh��SB�
P�C��1$��zP�=C���D��.�	v�I�;��MQ�,6%�M
�W��L;@������V����h��d�}"�?��S��&�,�!������O�pZD����I��Jr,�o~I����;�Zn��J�����S]M?$�H���g��2J6���'�-O�r6CQ��nm�b0�,e��5d�ta� �������|��\|,�}������_��5qQYsC��=�f>.dq(|Ax�
���k���F�t�:����>����@��#��4r�����``&��s:
 @����<��/����"����"�}b���E�u��+r
��WQ)b]F��__�[
��v����R�uk!�.�T%���_��S��2�@���#�!��z�AQa���������[�z_�wb0���crO�Q�?�hg�E�\��r�e��miKs��8��(%c�����/��g6b�����m�,�\� ��s#; !�#���]r�����=�X(���&P4��_a��vBxkj�o�����E
�����K���<�Iy0�������/2&��|�x�G:�u�0��V�x[P�>]�<������M�
�P?/����<�S�����/�?��]!�*?N������M�n[���)�3.����M+�'��b��0R��rI0�^���|���}�zW���`������6�6[�w��v_�?�}����*��^�O0SN�w;��m\�����������(�>�\\�^vt��G��E�"=��S�Eo	G������C�w�?P����YD ��mE����^��ik�He2�*��R�<g�}g�@ps���[�_~~�����NHs�>������G���������zD�7���C��F�FU�u:M/���|5e���qf��Y#�sNp��<�.� ���:M������izy���NW�6�|A
����{{{OYO��Si��H{2q�=�0�@ q'I�J����J���L��	�B+r$�,�k�z������}MXe��%.	y������	�H�L��R���w�|f��)�E��	�l
�:�H��M�;�B����� ]�����y��)�4Wh�@�v�9�M����
�q��)8�q*�
������I��LqzW�
�%&����~������%�k��z��o���*���`���]�f5$4�sh��}��s?��9:;!!�N>�����]������L�����T��Yu�.V���>��� 5���y�������o�ah"��7�����` @�%,?��9��~���-f	`���d4#u|A�8��f�/�I�f-���ii�-F�/f����� y�^*������d��������<���K����7�x��������o�O����9��|��*��e8��b�s�#���n��#�pM����#(~F#�T�����1�+���(�~���G�`���la����R�%cPpLcWr��t;�_}5����8�1��V�����H�>9�����`�/�!��v'P�Q���`(���,�����H�f�%v��jy�"��}�X�F��}��������c$:&Q��4�~y4�^X���li�$���3WZ���}	"-s2e���e��l�%w7���j�r,�%k�nH�D�W�	aE���(nGT&�[�z
X��f�>d��u�XG����J��v�{�aV\�?�\G��������j�����&�A�.�}u��f����fQ��WSf��L�s��,o�^�j��j��^��m����q��`b�m(~����8�h���y���z�?��w�M� oz���db��e%&f�-�O�'=��#yJ3(�3�u�>��t��7dP���d���m�!���b�oM7$�OI��q�0�����o�> �y���O��BL��z�������EB�'z�-���v!d@������"}��V7�D�)+��,��]R�Hy-T����
�gU24��D���h�$�)�\��y��V�V�*;L��D�rJM;P�:�S���!!KV���� �k�J/����#G8�#Yq��l�Qt����CMUW�i�3
�o�q��5���

��4

����
x�9<��M���PD�{���Q��6�4x0�C+��D�7{Z�g1�!���v��v���8�7�f�I�7I��>�������J�P��������XM&�����U[$�Sar�����b����������]�Xft���9js��Q��I@�����W���"����z�j 1�8P������:>������P�����k�k�-����}�$��K �e��d��$Vr��p����@bJ�,=�����k�k�-���6$���DI"��xI�IRH/ �{�	$I�@R21c��c���e��S��>"������6'Ue�����nsg��`1,��b�3m36`�����^����@nM@P�/(�F2�bj�{
>�
������=@����s�����\}lW��d�������k�j2$�^�1>�0r����A�^�6����m������d�B�b^���#��[g���g���3���W�*|��3��v����g0><����JX������X�ZX��
�s��T�40��r&&c��%�-1xN�Z!�[a�T��\)���4�+eFW_���38u��|H��9x���~X��G\~������-�{��z�j�C��mU����c{�>�}:��v���zU�ad���!>c���r>D���A�4z�I)g��'e�^�\�>3����T�RFv�
�3�@,�C�i�����"{*��-b�p,�����������%K�}�k0��xN|�P	�*��b��aSy*�}�4&��[o�=��PP��?���)���d���g)U�BK��,�W]�.�UJ���R��~^O��K��8K�zZJd�����u<�R�#M�#�1_Jj��JI�PQ)���JU���b���3�q������_�v��Ej�jlo�+'��&`���,&���;\(�	��G���^����� 6������B�^E,���P��O����_Y_;�A�������p@�v�k-Ss�U��qZ���0��D���]���,��m>������>��+L�I����u��J���;����lv���>LVD��o�ET���2���M�k�qNho��urX�i�{6��D�0e�����4�/-��3��4��r�z)����Ub]�w��f��%��R^���c8gr�s��\[��V<�u�V�h��T:`�V��[�����x����q3TS��_�����>%M��HK��D{x(7-��K8�������4��u)G�T-ZQ%��� &b1a��`���KK�T��8�������ef����c(d(��L +�`.Zq-��{p5+����i�w�;��wT�w�p��
�����]?�������*��k��u�6T��U�N^9-t*6K�b�C��)������5�H2��Z�� ��`�<��v�:Xi��'q����)���`p���-�`�[�#>0�o9	�j�0vw	����C{�i�U���.$q�ZN�����+#���j6rL�����1�P~�oO#�9��Y{����J��
b�zw�^>�<��>
��@��=�20���~����v�Lk��@�Q�R�/l�|>|�f������������Z�FWK��:���V+������)���Y���g�$J-Wy��~����wmt�mi�`��0����������
�/Y�z�#��V)��t���l�a�Q/���o}s|vrz�fk���G��3���vQ�~$��$KK<��P�
/���c5�qB�����x�b����b�GZ*��������c���}���l�Q��������������$��Syf��-��D�TE/S�k�}���I6z�n��n[dkw�9qLIa=��rB
��Uf7K��o6��������*B��c
J+������R����N��j[�F����	��M�k�+o��Y)�(mc�<�m���M��S�4l7�/.���UH�Rf$J�����F��w*������P:����p%<S����1�-�u�-X5H�d��%���y*���-�-X��K���-({��-�����}�b���H	����y��+�iL����M�_�����<����������"���J��E�o38\�������9�^��[�z1��?�+l�*,j�6���	�b�.~0o�H
9l	<��F=a-���b6��)�X�Xi[`���=p/��{
�{��p��(����e���R�o��*x<cT��y��j[��
���m
���
0V�,�o�7�R�LC��(�U)b���K��%���B�N5�G�������E_����7t�W��n��f�s�"���``�Y~V�8�/OO�V���xC#b2�%O�mjxE���*�oG����>{M������NZ-Rz'n?����[�*C~��o�������o~=���;���{��q����F�P�Qg]�_]P��O�z�����Y��y��a�c���/g���������3Lq�M�y���l>��N;���x;�����������i�<�-���������EgW�E��^���<K�m�M'����<Z6b�-�2��k/:��g3RL{HM�E�Bf/�{��9]�� �2O|�`�%��mH-uZl�8�b;�Yk*��h��J-�hI�b�b��@���b���4������������=v��&�Hj�n��8�ICa
�0YZ���BIZY�C�Z�n/�aK[��) �\V�
���f��r>b��)�8�"�j����1Yq������"��v@Cv/_k4[��������Zk^(u�$
������S��*�b�R5������'r��)M�)Cv������Gi���V��B��&���� 5`A��n�}F9,T5�)k�^��V�������������d��)[m��I2P�����f��P�_?9��sC~D��JoH��)��HC?���%S����s87L�TQ�H���Ix�[f��44n������b?��f�w����
H�jP�LK��*V�����#)�,���x���s�{���5�-m:��[h|A��;��-�
(���'�o��x�
�8�1H	����X�yQ��s��
����R��/���������Zb�X�����.SF+b��?��]�ho�f3�������a���������i��Z����jI��{_mk'|�Wn�^�������a��,�YU���a�@�6�<���^���H�@D�6����)�)�:R(0n��������>)��Vl�T@
U����8UP�>-�]b�_��8du��2��!P�}�W�|��7o��,�C��%���l������lA�K�
��G���2�@����������G�
�L���D��`h�~��~������l�����z	��J�oc��}Z��yq��(�R�<NYO����w��P���H|<?�J$>�{#�1&���>Q�F�������UI�� 62 }�nMTAl$A�F$�uK��#�y/D
/o~c6+58\��-�E�o~���[V����T�`�����:����xF���a"�K8�u;�����F-�����o�+����SX�o��fmv������4�Ai�F�ni������F���n��_�4��HC7��h#
�HC-�Plb#
�HC��4���6�n>Y��\�#�5'�u��oD�e7����,���F�Y,�����s�C6;����;�f�����*��d#��V����&S�����y���x��IX'gR�Yzis��s^�\�E	�U�UgN����k����+z)J����*lW��U��5 /��"��Y+J��z��a ��E���U��@�o{����k��T����U4��&_� Sk@�$;E��^�Q��2G,G^6��V�A�"q{����k��4�q����X��gc�Fo_�u�-HL�f ��B�-���ZE���xBk$62�oX���`�\�"��aZ%:��d��bZ�YzY������s3��L�!��i<�#��J�j��VL�V���Y�������u�T���F�j��!��iT�#��J�j��V�_�V���Y�������u�T�I�F���L�^���B"t���M�Q �*)�I�J�j���������&=*)�I�������k��P��fm����Ck���������?���
����o����.���vS��U������`��>2��������(����t.����aJ��#�f�9/�'��(�M�Bi��i
��Fx������z�n)�oz�H�_�&V1��JO�CW?�K�UD�����_��x���x�s
GTT�o�
��I��<>��������'�t2l�eK���`�bG�����|���u��l�DX
\��C/��Rx���b�
b��c�����buT�:*?q�hSCd���J%0���xY� ���|�� ��#���L�HI�+!��	�-��+"�����Dd��Y�-&�+�W@d��C�<"+����,�� �;�����,QG����%��Y�!�R5u-D�3�@D�(�,���|��E#���J��f�+!��	�-�*"�����Dd��Y�-&�+Q+T@d��C�<"3������~���{!i���~������>�(���!�rq�����\B�K��������Z����J� � �Rp��&YiXd�A#4�A#�r
�����e������*����S�m��Ie����b<�.��4Z�������h�b��s��:�-#��Y���Yt9������T����y���������/���J#K�%��FdF����H�����\�8I�E<�G��i_��Z�^�Z����c�wi�Z�����:�o�����[V��:��_�&����snS�M����w�����l*���m�GR)�{G��6��X��-����������3�Z���F��S;�?eZ�������
D�]y�$����2��+��z�0vb@Gsk����V���A1W	��F�!XKW�������"\��-���Ma{eXS�fS�1`��������Y����F��5U^C����M�]��]E�W�H�7��P�
�2|��"��T��H�5!���� �b]M�����;*y5����Z�.M�Lu�4��ml������
8��M^�w��T���\E*FpJ��[�P@"�e���H���@Dt��G��zP�|�l��;��T�4�Sf��h�&��r�S���lN0
�e�ev���np��T\as�_�SI����.�L��C�����/;S�!���~K�@%}����;�yq_C�,%��<�NC��JH&�6��e�q��U6��d��^bn����*��0�J`�,������� )}��5xt�X�Z��4�����|��:t��jn\�^�&)k�^��$2B���#�Ib�I�U6Ia���Y��W���I�"�E��|�n~��3�:i^S����.�u�&���$��>R��6��A��$���]���lj��iFb�-��G_�dmqK�D5�2-����
fg�;=+1?��y���&����4�>~T����3���rM��6m��*�W��=f�����x��d����������H5/$��+�G�9H�	;}[��v�� b�R�T;��zp�zt��czr��sPw�xp%^�H��)�G�9_H�	�1���K���Fw���_��>��!�������s�w�a����x1'�;�|���L"�&l���N/
"Muf���-����
��������hI0rt��/��X��$�����2Z-���h<=���0�gQ6������Qt�>�]\<�^NI����������2�?��W4fd� �=H��mG�X-��%j��Q��������\������\�(G	�A:�E�A��Jw�{w�_�>_%�W������7�oj@`�$��=�sO��Q �U�*�s����Xz�Y����6	M,WU��[��)`|)O�"�}�]�F��X`i>���vJey�"u�!�k����f�e�W��1GR:�~��v��^9������$���������1�J�p�
��.6�'pax�k��"�v]�S5yw�
o0����������9]T�-�+������@�����3QY� sd��}P�nU������3mWV�'A���1�jgA X�+hC�:�\�
�	<�s��g�������+��������g�w��3�n�
���4�l�rFX�Ee����U=#��rv��Q��;I'�v%��sF8�*C�xF��%��6T������0��3��K;#��g�G��������T�6c�U���&�-��L��C�m���44����(H�������r
�>�ayK���}<�Xu��eQ�$��-eKD���X�xV�f�,t�y���\������2����b�����F���{�J��e#f��aX�RXW�M�/"(1�5��e:����K�q��:9E��$�s�SP����p[kRl��^�g��(s�vO�'�h����n�=k_�Q����E�%�`����A>�����j��"|l����+(�b]E����)��^����`�9m���E4d��k(���/�("g5~`�3>&nc5���k���oN^U�p���X�n�k�C��rTc��~�Z���n�Q�Z����������������um��+`{��X���� ���Zl�0,��`�������E�CnVh���u�-��`]��[z�[�2�#�X�n�k#�|��yP��rm����`���Y�7bd����k5-O�P��������5Z�����������-4d6G����X�FA�3�lC�:��Yi����k��o�����&�k����^n2 ��5����m�fm��W��D�E���=�h�&wUS����l�03��	Gso���6��h^C(7J���n��d��N�q{WPW�d���N����(��q�B�I�H%��mV�y�"�X���qJ�� v��@
D�LTB	��.���B`����.��Q��V!8L�����������J�)H^]%�����;����U��@�i7���h�K���A���Bu����_��d���m�"o��7{���f��~��$��
���_o�{�}���	���~:�$r;�{��W�����rZ�C��fX�@��{����S.[��������]U&�v�J,u�����}�������w�l��C��+qU9���7(�S�on����~d���4_�\���<\��OnA��zH/��o����f�:8��m6��������]�ib���11������G�y������WOEy��`O�u��D5Q�ym]f�0k_�2Jq:j��\���C���w�e��y�l�6[h��jl���h���<�D��"�g�Q����7�<������f�ky�����|���������y��Q$e�@A ��m
����,],q���P^n�����'l�p���/Z,Xcr_/{�%M�};fM�RI��e�%Y�7���A�l��ynG�7'��Z+�(Y�q�D�N���������{�a�J�%T�Z`s�Z��U���:�BX+���fEi�F����jb�RS!����d(�&���u����.n,]��s_�a�����	]!�t�-��K�*��d�����+_'���R��ur�7��s���$7����!�6Js��WD��J�/�K�7��VE���My/~��]�������p0J�<��E*�`%dm�qpp���gP�EFp
�A�F9���3	��Q��	fA�FY� ���k��d-;`�m��fl������n8�
���x���x�r����p����
\�
�{Go�:}�o09���7��W��n��f�s��(�;���������x>%|��0a�s��9�j�o��r����k~��W���f�E��aA��<=!�*:;���	�T��Zi4�.�n�6�>���U:��~9>={}�������~:ig�k
���~n�hM���W��W�O�?����3��	'K[6�'�c�E=X+Y�6,�����VQ����i���}89=�^�#/��U�����ls�">~�v�c��.�i6O����q�������qL*>~w��/��{�{@���,�<KDH�|�,:�/�%PD������g�y�$����*�������N�����eX{��?�����]�� �9O��`���C��� L�����pW�������0����Vo8���a����������;����t05�U�������j�Fc�� �������������E�m�)C���64�������tV�"6�D!a
tB4X������0$�1V�$�Q D��p$L$5 s%���/Hr])6,�����S�,o����52z�N/E��7��P�
�T���� �\���o�p4����tw�@�64U'�e��@�1�E�>��1��V�1�^�K�1\���,<Ri�be�e*�b��j���y��������U
A�>�E�J��1
�V|V�^�K�2���}O���A9l�,���%g������r��o9-��J0��������&��
��o�M���O���7����h�V�5[m���6�����lI�E��
+���7o������o�eI�|��d�)G�j�;�u������D�V��?Fm��$�&�dV��)V����j5��+�N�4�B���H��#(����tD�'JM�c%�R	mU8]u���<��qx�~�}�����E/xp\^�R�>XfG@��&rcu�b�_��c}l�����X���	����r���[��.�	�o�-�a�\s
��!��}����hh
ru��_��c[#+�&�&�&N�M4�M,��T��2c����������M��k����:�G�j�������A<��&PfW,��-�x!kcQ�oY��� ��}���]�2Y��[VO�[d��ar���`�����bU8iS��
����x��s�
��x��,>P����YyL0�t�Krr��7�?��Fy���5�
j�TQO�?F�r;�{��=z�v;�5���z���>���~���T�;��g���!�o�qN�5�U�OA�_����j�44�t�rO�D�S����!�k����sq}��m0g�7�b�Y�
����3-9H�����9����b+����-������R���v��A����||�"T�����7���"I#
 ���&�u�"���*�u�!KR��d��/_���,���93���{�9��O��w���������G^����%R��m�g����O=�6'nGj�.D��n�"�e���L$y;�&'#�@���q�aSP��Q����_a���J�jod����uJ�m���X2�-���%��yc&�2_^��K��&7$�B�����O���F������J���u[aM����&�)���PP���� z][	^WX�*S6Ey:V��f�tP�Hi��6F�U�Eo�7^l���K[����U���1�@U���z���]C����Z[�:���H
�_��U�Q�w���[����z�;PwP�n�v0^�=Uz�U1DC����+��qlA[X���^4��5jM��C1���_���pHAt������A8$�8�6U�����pRw���+��f�A/��ma8���|
n���h��b�?���~^���0���r��=�=i7U������U�����1.h��8�����c��:�Wr��>�:����
G[��Ax���Uw,oP7�|��o��(q����
}l�w������'oP7$������9�;����
���o��h�Y���A��29��;q�����y��!�m����N�u��
�G{�~;GG����
�>����};�t;������P���};�t�����7��kNQ��r_	�����
z���A�f����{���uWF���Y<�����2����Bs���{C�������D���BN�
^4����i�\��03z���li���DF����v�	��
��������@/�W��4� ��������0%W�Z��hL�UI�,G
�{�m5�ym�7��	(,{�	�qxAD;�q��-����m@-�J�����
����.�wp�e�Z��jP�UY�U�f���5��m�O���.G�Q��x������?t��0J\�x�*i+B1Jx�i����j��21
�E��*���2G�������F	Kk-�kD�`��U�����e����ic���
�e_�>F���WW�:Q����cQ���fj?�JDa��|�=Q%�����p_F��*��~��/
����|`��@F��*4�>�Q��W�`���Da��b�\�,���s�9U��XA�^"t����m��V�
��p���{(�bF���h7|��H�7��5H�e`G������o$�a�l$��AR��.��I�FR�=�������a��$��FR��{w#)��$�kg�����Y.���W�JK$U�\�B*Qi�|J�kN4%+-�J�r�W��H��n6���H�L��&���fI�%��W����B#�5�`�D�a��������0��`b��\Um�^�ztmG.�6�r�A����PAh%hu��jp����i(���by� �
6ih��H�A1b�� qa �����b�����K�"�9i7�)�"������V�5��i4<��g� �JI��D�
�)�C1��������LLS'��I���	�V���4u���)@N�A|4Ls���w��?a���Q�}]��w ]�2�nv�f�}[;n�bh�0�y�*{6r����B^���r����{��*W4�)�lzK3���o���j����S/��L���9tCrh�(�G���j�����������2HV������w�^�Ab�8����I�Fm��!9�i��#��K5sF�B�Y�$�Gn$�lF}k��;@/�� �i���o�Ty#�nL����"^���AQd�s��s%�u�\Ia]�VRX���eh%�uZqaC,VRx}�hC�\W��r��f����k��E���:e��T��6��W��yC7`��X~�Q�.��E���]X3lt��A��*P���;�!���������L�
S���y�i�"�'�����M�B���i
��Fx������z�n)�oz�H�_�&�6��JO�CW?�K��D�d(�2^J�C�J�^<��,T������Jk�TT]�(��#d�x=>��\����}`����-*�y����6g���9������=B(p���h�J��"
��6�����r�*
��Q���<�Q�
�-(�W�T��E(
�(0VP`�[�~%����M9PI�+�����-�-*������D���P`��)�+QuT@���C�<
,���(0�� ��>����(0QG����'UP`b����}�j�@��E���&

Lj�@m�x+������J(0`�eK�
��(0�#�2Q`�6Xes��Jt0P`���+�Mxk�'�E���}�3o,��%�g��A__{��,�}}q�^�Aq�>���1����M����pVAg���U�8g�!���h[g�u����&���%���x�o�����T�F�����������
<����B���:�L����jFEe�;	,>T��u�
���a���l����7��
.���,lN��kIs���he�+�+xw�^�L����n>BZ<�\6P�S��	���8.��&x7/��RC���b�U���N;u�e<�^.4���*�8����0n�����9����s����^S��LPj\�g^�l�\Kk���*:�"zH�
\�kv�^��v-����0������
aj���/\���������l��;�q����K
vM��vL�^�������$O%x&�,	�H�G���\��r${#x���E�-�/���>$"x�i8MS5V�c�Vb�V��Z�B���r�c+��U��~�]6l�����	�\�d�=`�Vw�_����.�U4l�~s��Tr���J/�v�w-��g���F�m��J�~�p��a{��3n����Q�*���?htk|�l_�����qX����.�W4l�~s��T�5�Z�����9��"�����eM�e�J�6K����A��F��2]�?�9&��(]d�
T}���U6�f!�]D��,�g����/����c6������Z.zl�����G�ILk��U�j�9}XLR��m�����4�=_g!^��EF�;i�{�p�Ys�)�T�<���Xvn�.�q�.��'�O����e������ ;�N�>�e3:����=H_��.���B������"���>ziT�GOM
����A��K�y�MSh"��7�:���Kj�|���^J����h�r����UA�:��'�����i��;���m�4�k�A�q�������������7'�����F���M������|t����_�O�^��&�l���w?��F����4���_��_�?��|�����<��p?g��o�-<v�������Z��wNN�F�/�������">~��u
+����<]f�E6g�V����������w�o������d",�b�c��������y���>��}�c}�����N�
�����������uEb��8����������������k����Y��}�bs�b�r���Jj/Wb_��x�z^�����Qw�X�C
����c���b�s��R'|��g�ez��O�dj���}�{6����,�t�5�W�ZT�u�������>�k��e�f��u,e�\��e){��������n��r"��O�E�r�(��l
�(������hL�?�t4O���":_\ds���!��I��}�EDq(AL��
�t������c�����E;o/��X��)�)B,��f!�]�4��8'.[]_V
FtF���j�����pY��"�9y@������uV�FdQ��l�o���l���������i��RP�?�v[I��G�0�"���i�S��i����y��n�O/�1c�CV����_����hh����c�z�j Qpp��
������H�wl�<� �G�3rZ���}�N���������m���������NE=;A>��Z��=�[Z���@v���A{�=��b9@sT�6N����RR�&eb5�a������Uw��s�~(�V�����l~���"�������s��&�p�Nv�-�"��l�o`�L	������!<��}�!�I�M-��\�����U�O.vIi;k2^,�tzN�-ze��e:�d5w�Tk�����;��P��:�#����)����:��U��q���	.�� ��DKIx��7���h������Q)l6]�!��)��P�[Fu��t!�Z���$u���n��iep�v�D,�4�n!�-��@�7D0����d9��d�	A6�0���5�$�J���#���$&N�_�����&�u��&�B��,}�\&�����x�F��2���Z{������wX��k1�S�i1�S2t7������~�:&�����/T��pW��Js,V.��V�����tb{f�!�5��|�����������`�?�g)(f�	����D���L�{�iv=���#�v��aY�Fa�X����Q�|'*o8�<����G�<b;���Ty�N��B�y���A�yX��yX��Dq)� <wA�������y���Gb����Xl���#�#���<b����#�[�;��(R�xJ�����6"
�����HK��J,�}��:q���Qd����m���^&U.�48}���Q���Yxy�L']����������6"W����5d������@i���C���K�v�T�g��S���
������C����J����9%t6.��:[�S��Y�l:��M�������$�:��������>hg�?,�����g�[:�Q�B|������D�m����D�k��;�o������oZ���Z)����g���7M#�Z�����]�������4��I�Z��4\�RW�����J�Mz5Dt�����,��M@�Ob%luU��]F/�ei�$m�]c�5W�Aj�k�=�{OB�B����
�n��;^yP
�b���������u (�r���4� �P�hP��P��AAu����6]<��F��\�v�Z6����M��z�0����C�
g����������Y��c���j���I��Y���3�`2�����:��+����^�^���a!��<�����v�=<�W_/�B~���a�<8Je�W�e~H�j<]���(Y! �����G��p�.��y�8����-Q���\Fkj^�2>P'�$ ��`�
�n���bcKh��T?���5�����0�o�P���h@���T%��d��b&]�����@����1/�C����4�<)�p���4!(�������|9�[��C!�<��`�d5;@/#*X��p�m?�����CP�APbBP�p@��!p�w����{�D��1��p���}V/m�$����<�@�I���3��'��;����5�.��o�a	���|�R'�oX�>|�:�*(PV'XR�Y�
�C��{i�����%KH���N��_|K~����)��pI�v�Sl��l��������-��aM�S�`V����N^w�R���S/�>S�#P���)5�
�\S�����������:>�G�:,���~S�1��h�������DA�v���|���8c
c+ �.X�k�c��������i���x���Kf���-��T�$������F�KM��u`�������I���X-��Y����g�=f".������RIc��%10kbl�$�&&fM��5qa������ON�|���:1
|��?6��J%
�LO(�u����^��@g�+��������B���g��B��zU�Ph?H��@���qCs�
�~[���[W,��q�A�j�pT��[R�M�II8�"!��!�������7[;�=�dO8��?�{���@��HX���d�b�P����K4���uQ~B��)u�,�1h�����uM���=�/u�#�;'��<��Bi�`�T�PL�������g-�PR�b{��rF�k�������Ub������fe�=��iXh�D�*��6����8f�1A�u�J{f���t���re�V��V�>a#��-�Z�j����Y����|���=��z�J�	V���q��y1��T1j�S	}.6�R,����4M��{���������+�����}�b��/�d6(�co����%��D�!����bd�P�����j������o����\�/r���>��BK���1A�����1�1E���]��9��h�J��r�>v�I�]e��O�:0�e������
u�^9p��v�1p
L!�/������*7L:��K��`j*�$�wN���U�0LgJ��qd!�BM������T2���m�7�b��(����g�}�i�*��	���A]R
u�p�|<���x�$bN��h�+�|r@_������c�&��&��&������'��MH�� �(����rj�Y��(���1�x�C�16u���T�85���������Qn�#���(x��14>�]�U%dQ�C5E�Q�7���#aPMuSA;T�dQF}ch��?z������*h�j�,*.�7�hO
���_[�����_��f�����:z�a��OH4����1[-���</-����i��������^�~��4�e��;TLWV*��U��5W�tV!���"�J��1y}!\%�[�������P�O�V,<k���Q����Z4�����lJmRIa�����X�)Qb���7P��
Gc���hG�&&�������|GV9 O-�e��v|=�����J>�@���!En������6��h�A�����������j�-�C6l�F�d|�:}�B��Q���������>���Bq����G��f2�(�>;�p������v4�}"� 3�y4��g����A�|J\$����o^8kM���u� �������P��|^U���D��1�)���v��{����mfXu��eu��=f�#w�f�Yg����v?d��>��r�uh�\��E_����O���I]���#�*�:���.��9�'^��i&rxf���E�;���Oc���	p|O����u��|�p
���l�^���V4��*E�
��+�����l;��#�{�N��?�+�v�RI���r�zPW����jz�	��������2�������o���th�h�J*���a5M����jBSRI.�z�o%a�;n�6���c�\��f�nm��[8S�5ea��25\Sf��S
MD�	�6��XA�h��/SC�e���J������;���t�B����z�+U�2������2J���qv��_F����H��V�Nf^��(�����em$f�FD��X��/����R*T�2��e��e����x��u����p�`�G��?��e@`��}���X���2Q�+Q���=.M��R*T�2��e��eR.������
���pi����:\�����ev����T����!�6�x0���9���(�}"s"�ia%7���e��%C��
�Q�DS��,K��J�<fH��������e6�o��N8�uT`�#Mr���5\�&�W�A��I�>�[T�J�� PKP�Z�D�[��$���}uS\7�z�W�0���\��+�_�@��=^�B��c��m��j�}�~����^M���f�s��X�����f��|����z��
��$���h���z����������������a�o�����sP`1�AH�����:�r*AD���d���b*Rt��8�{R��2RbR$����T��f�}��!%�6���I	v��FF�����U�G}+k��,��_��l�d����dk,���.�-��__<�A��#��������� �SN��(��������-	��[%6*D$���q[�C�_ej��D������-

��M�#��ty0����t��aG~���9��QA�'j�-�TS��J����$�:wbn�yJ���=h��z���_�A�s��|�����%h�d�U�6�O����� 1q["3J^Z%d#������8�f����8(�$��t-R�N'�H9,$J���,R�N�o�rX����?�H9:�C��#O�p)G�k�rt��8�E\�^m���,qc���j[�\��T�+��Te7I?�`�B�sY.u���)���'
���j�J��lb��!�����A���&�[#�D-��N"J�c����R����I��4}\#dD-8�("Nj����>�qNs������R�|��o�7���z�B�W�N�o����i<�D�c����!��?m�X P#�C��i���6B����"��Q�w�g�B:h�
�s��zig����������b�]������2�'�O��x��>��1��;���q�
E^�����X�P,�V*h�0�
�� t�zDW����
��3mc���&�7hw�P�J9�iO$%�k��D=L���J T�2Oj(����y�5�q���e�q�����#7'Ff���(���S!~�n"��
�4�/S��5����Sr�b�����y�X�g�E4�F����U6�����p���oy!���E]���,�:�tUd�ev��9.V���|G�/Q*�m�do�>�+ch2Y����a[F��N����"#�_u��g!������H���|��9���g��2�8QX>��lk�,�d�L������Y�
�������od�9��^y>8�$��J�: k/�
.�x��T�W/+?���:?����3�����
{��f�Ke��N�M�l�:_�`G�����c�����e�	�|�.����8D� +P�@����J`0������;B/o��9��`\I�^V��.{���������W�W����}%0�~�Ru�F��W����7��V�l2N�=��2��r0�����*�k-���uT�_Lg��BC��5�#�
jl3f�s4o��]���kgA����t���,���W�*H����������(�c�#`��s~<���K�T
�<���V
&XmP�	dT�A�G��r����9�!�[��9J ���S"�]�UYb�Z���.��W�~�����A4!4}W2������V��{j��z=hY_;a������y,�lo�R�v$K�_�S�W���"���$�1oFX���L��?�`E�}ac�����W�~s��&<`?�e!|��.��q���M�CD�;�D`�/�H����z�����NE�@�UT �pMv���.�Q�.,a�L�C�)5z�Z�T�	bXE���t!M��B�����z�{�VQ�(��QXE�#WE^��f�
�g���n$������6�GyE��csx�}}���.��� �f��f��M:_��h�|1�]G#���h�MR|�����w�Q���Y�>��{����g����o�0��V a��w;������6e�9S�,N�-�Rb�Zb��]�h���|]�A/U�Q�'�RCN{�.A~'������
��L��&�K"�> ��Ce�o\��ao��>:&�_�����l�v�b������=[�~��_=�]�����7>�2��CA=��������X�������k����$��^B���W[�H��nrp�� ����n��H������G�*��#u����L�Ei�y�����5Z�����m����X�&$.[�'�P!�"��@�6�\��p�t@��
�fRj��4���V"�h��
z��J�� ���eo���xc��q6����,dYM��4$����&+�UY�Hj�}4�v�������t���*3y��$��?�7�Hl�����������;�MzQ���U{�h_�������[}������;���~[(�n�H�&`�����J=�
���Ui��,��C������I��
�R>|��,�[�D	Zyo���U-��-$�v2'��p��@���v}���Vm��9�$:r��B�@:R	��1'qm��E�^�U��)O �X:������g��J�Ii�m�G�\@v/��������DI�C�{���=��������rO��d��I�2���b|����i���������{	���?��#�Hz_X7l=d�e\���)x�aa*]�J	��a���l�+,LP����Ve�������C1UP�`�r(��a����7�${�{~s��F��z�e��v�&K��;a���o�y�����K�����������F�
d�o�*�T�H��h��W�~{0��8k�,x���Q�i���.[,����lv��lRm�u���t���A�N��$8��Kz�O���	�*'�����P���f�
S�Z�����w��:���))8N'����	J��0�^���T��f_��?����S��v=���B��k�7��h���C��7���������iG�3��%]�������t<��z�{2[��H��-�lI�#�|]�cH9�$�6��\�WD�Y�}��O��GK25Y��H��j?�����Q����4���u�������A���B�}@�D��4/�#,�X*>PV����n��!@�GY���o��_���S&S�Ja���<�Ni����-�eK��5�S{�O}����
5��-��D}{ZH��z*�����Ti� ����w�����'�c7�����XH%�/q�w���5��qGM�w�QS��/������y����\)���h��6Gy(��r���,x�-�6kR ��f���)������b�\�R���40��I��uS-��K���y���x�N�Xy(K�p��t��o��M��^p�A��o"*&���&��e��P�	n�`����}�B���a�.��$�����~-DV$�#KlxlI�m'�b��%6<����p����
��cM�m%��������H�m�����
D�6 ���D��bJ���ae���LCt�|���������7gP"��sc��E��:���Vi'5ZNw�Q���e]h}Z����.���$�Gh1�=�gjn�*r��L���=Ms�0�mY�3��:�Th��G�O
���#�Mx�9<�1���1H5��i�"��x,���su5[��O&~j%������y(Ri�{n1��us�;�.���^�]"�;������V>|����1:��lh)k�'+{��2k� +O~�X�������`��s��:<�n���/�#hq��,S�vfjf�L&,5r�*�o<,no���S�|��L�J��(�!��4WGK�(��\3����C������zO�����q�j�2��)�?(��B�*(�j'K�J}
(����P(�T�6<A��oP,������1��h�������1PL��HTPL�N������h��8@1)��PtY����7(�������w\4�Z�Z��n�mA���7*���D��������_�y�(�<\(5#:W��
�n��|ww_���+=?���t����x����a� ����h�N/����hD�/�at.h��|u����(�F����t�4���:����������G�E��v��r��	�1�����=�mKLC�TK9$���_+�nG�kL:@/{��b�|���v��o��
#�
 nI�P�W*h��E;j�99�p�2_0�1�x��R��O��:�����fe%Ja�)�,nns#4{Q+�T�_0���gf�S��1%��h�n�&�TP��b��CP-}D��3]oMy�����s��~�Md�W4�	���d{]ySU�t�9�3�T���y*�����T`)���)�/���OMl9o��������^�[�A16H:-`{]y�U5
(��`�^a����[O���i
�z1_���\���m=+�����i�mQ�}z�[�X�V?o���Pf��/�Y�%x��ZCXo������(	������O��z���X����q�7!o�y}}�m+��z��Gr���t��������a ���|l�M�r���n�z��:{���h7Z^����l�9�f��Z��yv=���M>��H�~>�f{�}�Z��Zi;k��Z��u�����{)�@��O�!�>i��� �^�b<��_�c���Ev���)#.4�����DT�?7�\�e�u���#�C�Y<F�?�@�tS@�F������q���^��sHW"mXR��%u9�Q+��� ����@�)��:�y��F�d����>��B�A/Ql���]G�b:��&��\������<r�a���9~��!��e;������q�Y���9^���Xe;s�����q�Q���9>��������6;i����������xyMg���j2I��,����g��x6]D�i�X
�F�c���s��:�-#��d�]e�,����:���T����y�����G��1�N`���7 y��Q�%������&���}��X1�vX�
�n�R#M&B��s}+4�N����o�&q�Z�xx��?0>����Y��f���_-O�����r��Q��V�����#U��O;�Au�M�<�~���:���������������Q�V$|��^�3	�%�^��v�=�����i�g����@��n�y��'g����5�'_V*?�b,��w<�or��M-|��n��[��1'q���t��8i�� ���>3�P'��I�OW�b�
��b��3uA>P'=i�jC]�X���JP��(�����"I]�*@]�@]���TJu�����Q�������[��.D`�B%�s�*�q��NzCVV7��?�
C�S���
��tz^��;��a�y�;�����A[������+�_��*��X�?�$�e�T��3w
�F"�c��:�T�}	��I�j5&hl�V
��*A�J�U�����o�����G=@���eA�P*�K����l�������hA�������������O�+3fm0B4M�F0z[�X-��%j��Q.V�����j����Xm7v�K�^����	�K���X�.��.��K���;~�������I��{��$��@��$�U�*!��=��&��'��}89��)��W��C�<{�..��S�0/k�PCD�F;p25��\�c�A����Tg|�q��fH�KJ��M�i�+�jiT\����t��h�V�6��k=[-eN��I���+/�������V��,�J��o'{�a%��l�
��bN���)p�.��MR7.����
�%���7����^��&f0�����|YG�&'e��76Ww��VV+�]�W�r�����X�bM�%�����b��%���R������fN�C	�4~��,��5�<���{w��i�X�����Jdy�l�%������>��>-�"��91j��^��a�/�k.p�4�]�b��c���H�z�-�3`�W�k�,~~��W;[A�^�6_�����!6�x�x'�SO����	�p�sG���y��J���cI�G+�O�;w<F���d��u���5�r��%����fdci���s�:Zw���;�~�T��+����C�t�:^�������^��	]zU�|�;��8qTT����HS;w|?j�7������gh��M8�*���U���f����_#{�X�5�����fR�.�S��b3��KF|�)Ul&�x�[r��L��:���+_�?�8����2�m\^XF���qj;�����Ay�C4��p�#���S�3����V�
����V�-��r���[�v��?��(�r���m��vC������P��~>4�����@5���	w�D��*����ToD8'UW�F
���%W&/}kc�J_���?�>	�tf\�J��y��)�b�f%������*���-��O�����,f�:��+.	�X���>��Wv0Z���U�)7��S���-L�j��rj|l�
�|R����������r$c�Se]w�JsP�_��*%?�r,x��?�c���7T���v,�����eS�����3	A��^�������W�J�jFGu�D�q,L�z�X�\�+{`�il�����[�c9��e��i�
����+�5TO��\��sgh�z.,��c9���w�������e��U��W_��r,�C����c9����8����K�����3	A��^�������W�J�jv[u�D�q,L�z�X�\�+���il�����[�c9��e��i�O������w>7��c�9��� �����X���*���W����V:�s����	�7Z�0��}����b���0_XW�i������uM��0_X�����0���u��1_XW�i������
�1_X������
xn������q�Kx}�����n�,�����T�7g�
�	�2��:���:�r
c_�#����1�����%��,�vuW�j�0Q�R0�6�7.�4�[�&
Ol6%�T5���0���XH����z��'WP�pt�Xc��p6�yx���m���@>kavd�0��}?��BJ����@�	^����Kz��h&�0��m�u_�n����4T�����69���J�CA2~��0�����w��z��P@�-��*:�?Y�*!�CL���g ��5!�^b*+��K*	�{�6C�z�#&sF�SP^������9��A=�t�b��ig������CL�������~��FL}���S�@L�kBL��TV��O�T���m��PGL��#��
��}I�a�s�!��z���T'X�>V�LLd�A$aR&�]R���O#��O��O��O��O���O���O���O���O���O����3���-���J�6K���/���u����W%O��{xJ�6q-7q-�n�������x�}Wr�J�kL�cT#��Z@��\��za#]��]A��������_|xa��X�M$F�<n"1Vp�t���+`<��IQ�osf�X/va�����3Y	�U[�����|�!]���(��(��G	t������DN����������d5�X����m
Z�����p��`�_�4�a�[����&��jYR��
���`��rn��l9���`��|�B�M�M�M�M�M;y5�.T���7A�j�5��V�8w����UQ����M��M�;����f�����j���xx������k���yk
�W�p���������'�M��o&n^������	�W+�^�C�}]oM�����T8�7!��Z5�}Qjo�_m�_���W��{���:��	����{xZ���f�f�p��5�� ����
�&���t�9��,f�w��jN�eQ��xp�EP��&}�*E`����B�|�B���,(�w8�j��ea���c����5�yS)���
�H����^eQ���mUs�+�k�G����\����;~ ��&��&b�������$%�7�D��"�g�Q����yt�������o��WON�|�X���ed��7'��?�������7[;�=�$���u���GL:?�d�bI�i�r[�<V�1����J�@��X�����Y,�HK6����m����k���a��3t@��������iU����zsr���e�P�-�Y���z�������l�$`�f6�)p�o���e�7�,�t��5��/�x�����|���o1�m)dQ"��F�.%`�����4��~�*���AP8?�@�c��p���r\�t�����p�R�"�#�XH'�I�,�]tq	�����FX~!����C��V��<�VC�S9lU���������6����,�6����Ejh�
������� L�0��~�\���~���-�v��[o6>�z�0�������,l��f���7Cn�Y�t����[o~�z�0�������,��-'#8{���h7Z^�a���iF���,�g������,Zf��.���4�s���Z7�J�Y����l�����-OK�J�}2
��I�!O��S� CK�#C���^%2������)[.�w����~���2���j
1�w![�,�����h2�+�]�u���5��x��;S�YB��UZ����k!�Ux�����h�r������u;y�N�@�+&�m|g��vzq�9��t�8P;��c@�,g���s�9���p��O;��c>��f���s�9���l�8O;��c<��f���7B���TU���L��t+���D��w�.��FH����)�I�%��Yf��Ph�5�.���#�{�t�-�+�w}����=�d�fd�N�3�����9���(��Y~��W;$������;
����#6d:v���l���Dp-NvG�L����u&���t����Y��m��5F��!�q���?�F"43�Q����vm[G�|pz��@�%v���������[����>/my>
���H�m��2X�	��|�H_�p!����Br�3��VIda}��o����x�s.��] ��N���<�_E�)�%�R����T�}������z��+.Nq5��u�x��_�".�w�����F��?����O%w�C�M�<�,��C�O�<�L��C�Q�<�l��C�S
�b�b��l��h�\'�U���V����V��9g7���w�6��r�}o�Y�f�S�|��� ��>��W�t6���&�fE�7�l����i�X
���q#/P�]��e��<��"�.g��PG�.��l���=O�'�r�q�\}l��:_v�G�"d��u�eq��cRQ���<����V<�-�� mK1�����1>������J9���Z����[Ok���Fd��H���3X�C��C�i�X-����X[k3����V����K�"[JK>Y~��+����<H�R`�M�<@�l��m�"�,�Vi~[�.�oEw4k6I?�|��uE���o���O)�FJ+���w�(�Y���F��

��������F����������V�#����#����:�5H�5H�5H�� i`
�5H�5H�5H�� )X�RV�IR�j���V���B�"���1o���U�2Re��f3�C����U�8�lL~�����bcLq�1��N)S���)1�)1�)��NIc�T�z�c��1�����S�_�V//��W�
�}��B�����j)$����-n$��t�eP�b���z���b���� �jJ�R+
(V�b���j���Q���|5�`O����]�g�h�0!���%5���
�4��_dn#�X.��(����9���������$nX����jFzr�#6�F�EVH�cd����\&zi��9�����RJ��
�����
<m�"�4����n��11jz��M�����b�#@��]f?U��{��u���AP!�����q.���t�"����P��C�rnu���_�\�Q��h�H�Qq_��q
,#����-=�����Q��bK�s�,�P6���_�6n�L�����
TTR[���5��$�9��;�-D��Ec�T�u;���u(���lT�C�@�{�-7�$wj�a1�{����l�]��"�k�JtIa$�JCE���HS��t���������g��d.��e����������H���Q���1��/��$H)2g��P�d�a]c����~�j&y�f*d�n��d���*l�������1����P��
U��\C�	3���-�S��tW��,���i%��Yl�����bs���}��Y\��bs���}��,���<4��
����@���\P�LY�}�����c�LU�k��9�6���Y�
8�JlY.�#���%�["QcW/�.�zY�&�e�Wt��u����9��37��Vw��N�f��f���8�&�P����Tv�Y�uWB����C��HQ���K+�BBz��rg7K�@���\��8��^;*�Z��L���j��aVX@+sb3�u���ln��\�	��zVV�`(�{���yx�)�3�[[���4�c'�U�R&��S]�\�Scs�����;5���3����_Y�\���N�;n��,6�S�K<���8um}���H��Ss��c���!�,R|X�����9h/T���D�V�ct�l?�hJ�K]s	?������V	b����I�a]�g%���6X�]y��
�shhC���x�T�T�b���}�W/����37k"��-�T��r"l��������a����s�$e�e�]Q�W���!�b1J��(V��Yone�L�u�2��Plf)SI*U$�*u���zy8h[�=>��
��bOm/�]�[��Y�z��}�V
�&��w�/�__�
��e����!���V��
=Y�ux��sTp43�j.��8�
�yc����[����M��z���*���6`�[mn���u�� ��2���� ����������<1��T3#�p�U���T������Y���c�m����j��^�UV%,Md�����w[w�'[�,rzf�����kGT�/�Go��;��Q��2^i&��g�?h��@z��m�K�����{�������Wx���a�[oP���T��a��%���}�v�`�]�1%&�����t�L�����	N�m����n�w��!��0�K��R��()�(��J�i�^�%�GI	��,�!R/m]�&QR���6�F�
�>�^S(��%��5jW�M�M��g��M�a�(���QX����}d�D���'w�&�y	m)o���rv����mZ=����?1#A����+������Vc�|}I�|��6��.~+��'��II�Ed���HE�E�+}����>L��55��Z�`�^du,hn��&��tQ2_�����,z�����������*��H*M���$*_C�4�ChZ*v��b�YL�,��i��	�b������.����|R����.�X{��=�.:yb�$_6���<_n�mw��������j<���i���}�$K<�jI�XK<�hu�=-����M���e_m�}��~��������6����x����{w�F�(���������)GH���}F���g��%����Y��@
���H�[�_�|�[O���
$(VB� P���]U�]A�re�+�\��re���.^�H�d��
����O�Y�F�lt���h�����2����p�}�f�l
~�OA�w��]��^�G��?�=G�A�����t�+	�3������ty�vzq���
G@��o>^�����O��������?N�~:��\eO ���"�7!�o���������Sx�������Er]���Tm�	tC�7&_�R%�R%6�kjQCe���[���N�����}5Uf����;�=9}r��\K����E
��-2H����x~q	~��@	A�i��^��d�N��~~�X�/�7�<[��l����l�O_���Ey
�>}���T�����+n+��[��O'�{����a7��K������hU��b>���B��a�4:������D~��%�(��g�0o���~�Y��U������W���U5���q�U5������j�^U5T�����U5�����UU��yU�����*�W�X���W�X���[4����U5�V�X\Uc���W�X����U5V���zU��j�^U�����fU�d�04+"^f�"^�@��i~"wbz���_W�,�^�Bey5�NiR}T�Y����|������t��>C��}�}�������[l�:2���h�-���|=�S�:�=ZG�;��XO�N����^1������7X�h*�����F�Z�5����JG���9�jq�a��������IE�����Y���������Te��&��m�~R4��0'�(SN&4����d�����s���!Z�M�z(���R�S����/e@��0{�/�<��q��|�����fm�e[�
7��l����?�������K�|v|hg�%�����Z��+�?l����R}_-��
�����=�����7	�4���q��,u�����*w����6R��c�G�r�o������������8cE)��Um��GW�Q�q���17$�+_�4�T��L9v�g����:��}�z�Y�J�/��������H8�KD�K����s��������g�}�����k8ro������7��7`�o@�o�q�@��������~��5`���>{�����78q6�R�R���7d���7�o���a9����}~,���s��������A5Kc;�)��Z��Q?�+���]�	,����c�����7�%����}�9wWgZ�/}�z��<�g��R]����6��>Oy�������_W��	�5!����Y"k�k�w�
6a7�[bJt�
(7�[b�pz#kZ*b�K�V�G��L�������x`0���\%�����_����t��������*]mW��T�'+:����I���B����cE����s�Ue�:n_�e/�R�m0������Sc���R0�$R"��|W�N+[��~�;.��,�/��|1g���/�Ec��03���t�B���J����jL�%�7��c0��Q1|����U�+'W��\�nr��h���N.?Q5y���K7�\&�E-*���`�{��ey��xTc�6����������w���;z6��X|J�\�Q���R��-L.����>Zir��<]>N'O���7��� |�>���\���jL�>'W,�i,�8X�\��s���+w�X�s�����A;�lw.��U'���e�L.�����f�{����s��GA5�Qo�+�DXa�Hv�D������;W��������\����}���2��0��'W�{�R3������?R�<
�1���\�0��c,�`�s%����v�D������v.��������V�\�;�a 2�L.�;�����&���U��Fy�����A��C����=���T�=������=����s"�B*�0`���N*
�ND�g���r^�>�}=��f)X��`��l��:vV����q�oO/�/N�v���/��sB~S2A�2g�8u�X�e�j�y��&���|�D��	�L� A�	�1�	?bi$tD�A����V�6�,��L��L��X����o����*on�/���!e�\	������������	�arp�+��5����	�����P����e�v��S���^��"V)B��if��
!7��!��F�6���$h�y�y�������/�Q��*Er�����,��@����<�%J��}a�,^�e`���	|	���I_�6���'�u<����!&�y�7��~,T>I��|Ub~���k�MUC�s���(��l�U�L<#�\�,Za&f+�$��L�U�I�
3lq�*�0�+B�c
�-�V1B��!tk�jX}�h�4����4Dn��x�/0T�M�bD����JWdB��������*;	�_�t���s���FG��Jg������g���JS.d%4Z�(��%b5#�d�����
�
�^�0$% �S�����N1�^V�������4Dn��x�/fU�M��W����JW�O��4������N���4��'���+]���z���:��y�3�H;���j�l���V����Y�X9�-�%�t�Z�}�,��69�I	'����l`�S�������P�����M}������IWd�7�X����q������X�����JW�<!?�Y�jd�����t���+��xF�9�t��v�P�p+�@�Mm�6�Y-oZWi�iS< )������"��
�t�a����ULh(|I�A������"}e��+�W��j�tE���W$l��~,T�K��b�+~���k�tU��s��W:1�\��Jg$��vN+��r��]���c��J7yS[�}V���Eb�T-IJ8@ 8!,�%e+�bX}�t�R-
�/i:���W�_���t�#-����X���;�H��?!U�����B$�#�TqL����&M�S�E��}b�H�P��(���]M�T�E?���R����U��q��&�*���Rd�{�1Z�H-���]MW�E����R����A_��=V��c���l�Ug�����e���N�XZ��$��^!4���(��Pg4[*�Ie�!4�� ��P��Bh5�@uWmu��M!4��
��+��:T��@
`�P�@[*sS
��B��
����)��p��"�B$V
�NXv����e':KW�*�:]�����|3���Ye�w�l��.��h:�V�1^��(m��|������`a�����f,�F��`��}n��t���}���UI�d(��ZQ:�_EL���W�����]Z�u��3��:� 7bt�,,�^�Kz���w�;�=��q��<+���Z]������@8�/��������S����r��CvD|=�S*}R��g1�U�TF^<��9��	�
&i�z
f������X{��Ync��v��5{`�X���A�1�{*�X������D��2�;c��yUyfl����%�4��	krk�����v��3N��qR�5Z'�)mn8U;���Iw��3N�c�q�p����d�*8y�j�18�4s��t��3N���qR�#U'������=�|���[Yc��6kB,>8�!{;��5�|�q���@��3�>��������
�������c�c���4|ln�[n��5���z�Fv����6�'x��2�������t:W��]T���-�V�2����"+v�����a�j�k���v����f����\�bk,Z�C<^L����	kB��6��4����S�N�d��)xY���4�\�(f����d��������>f�
�]Y��1R>l��E���m.L{,�i�@����,�����6�
jBg�����B8�
tCt�Vn}R�G��PT/���G�~��H�Ts������	�k��H���&5+'4���X�I����S�@e�I��$��I���	M���d��d�����m��P��P��;�/��_��]�I��>�X�� oB��?>J�V\M�������3�N�b��? ��r
��.�.�Z��v���K�����Ec���B/����I�f/d��"<�0������L���0N���T�TX�ft��^�����*������=���+�2G���d��f_	����������1����~��_~W��rku���O���+<�<�X���dHMs�f�j)������c�t��^3�g�������E�N:6����������+��X��p?������}���S7�m"|��rb��#E�����x~H�S��B(���N$����u���bq��U[],nu�r��u[��z��1ou�G+muv����"�L�[]�s�3�3`���N[]-x�����e�b�s�u��Vf���Xv�.av1�3a�%�������.Q�u}���K��:�����.������v�%���g�Yg�f�j���f]-x�����Y'�j{�ur06?��xA�Y��N��q�K�{]�������v��>Zu���u��L�L���^g�f�j��\g]ux��)�Y'�u���N.�l���N��i����y�JO�<�����D��*���M.Uvx��*	����>w��j?m��x���_[���m� ����x���j�'Q{�`�U�[&�KP���Pkd	��T�_:3L�_��/�2
������� a���a���M�%{��g���W��:'����8ocq���y��6V�[I9���yk�����5��w�T��y[�YlA�2�u���K�����e�������e�#������-F���V��.\�����?o=�H=8n�
_���y���4ou�F�{�q������[�=��8��7��-�o��fq����y6��hd�V�X[�����0l��������[�����s�����&��6��D���\��_X=o�\�f/s�����F���?ok��-HX��o6|i�(����������y;
.>�y��V�oY5�fU:5��l��s>���5��v�]��y[�knA�2/~���K[|I7oeg�I���z�H~����qZ�D��+���2��;�#���*_�~xw�L~P��
W/"oi�,��E
�����p�|�;v�>Noo[����Qm�r|5�*�(��<Q�!k��[��j�������Xg�i�D����6_����1!������r�eS��yQ+�O=N���r\C�kI���~�eI������%��O�9�<'������:���������l(K�Q�:�?�����1�����n�r�bS'o�sqVpn���M��>
Of��K�z/�g�*Ps���=�MS��Y�w9}�G�}���y����Om������z]�S��
��p9,����"y��{��P����DK���r�>��SI��r���w��T�[�g��W�[�SZE��YNTv�^B]�5�;��;.�3�����t���p�C�3�r�~�s��<k�G%@Cp����i:��wV"��q4`nE��X��K'�%d�����TC������}����*�9\cL�(��#�����f'x<�|f�--�KNDv��� czL�=��������NG�����C�^q����t\_�����^�zqf����b1��T�5;��Q��h�xiI<�3��Q�cg���{����:��';���`tw���{�|�w:�}���|������}�i�Qj��Eu�P�s]���s��P|s}��zs�V��Z-Tf�N+��a�
u��G;���	B��������M��~�����l��
����l	��
++������-�H����n�mo��}��/�`�P[l������I������6�W��t��G7����TdL�M"x�4��h��0;�ZF�'�:,����Sa����\�Vj�x^����o7�1�Q��b��&��1�Ee
44NH����������p��p��O������������[o�:$�^�O
�G��15y\8>/�>�
��
��Xzp8^����KUO] Oi%�a{�AS�F�@����'���B�M�Y�379M����?�h��2��M�[�o
�j|��Q�4
T
�0�&�����Rx��1�p!<���p=�A��n
b��+��q~�2X�d�b��������^]AV�!ge4�Wp����g�?f��t��G'�,E���J��I�
���}o��u:��BF����h:��}�Dw�r=���h�-�MWk�t,�]��^��.Oz{����o��o.���3i������t�9�����[�-Z�����|��@� �}Ig@�M�o~"������7��f�g�����0oEM�8�~#���%�7�c%�cG��J�K�hs�WR>�(�$}��>I�#��������N��Y�����z��.�������������[4Y�W@���p����]�U�W�,���
~�\�U
��P�-"�N�W���~t���������y�?�!����[E��:�*%��J�������
Ete��L]��DTB����A������L�����d�����Y�SZ��J
f/	;��g�<b�*$����o��7�r��T	���	'�!�����p���AUN�9N�+sZ��i��8-���u��W�5����k�J ����[7a�w������{���
���;�n�5\��>!��}�����S��
D�	��;����7�X���?������(B�y0p��O����y�.m�;�����}@�a����m���-M���������"s<�=����^��Jq6J8�9��]��l�%;2�
6���oO�������GK���'���������k��&�n����hrPX�����������@� ��Hs���c&Aaw,������q��\�O�P�$1���15��������\�p�V�_|�c.��P5g�3I����:����w.�}B����w�j�QtIdH-���'$�8O.��N�A-��w�z4�V�����wGP4T�m���4:�E8	q�ykhGb^%x�g^��P�����+��?�|���J@��u�'����[_nG��Q#y��'����D8���CI�>���&�&�a�d**���C�(�P�}h	����&|2��t;��`B��yXU,����&0��e���
.������s�N�v�Y	/BK�?@b�2^w��*��[j~Z,��%t�!\o`em���Bl����&���[Ip�k�����B����F��C�>�7J�m1@�E�e���$+�����+/��2
��Ai|Vi��:���D�p��<V�Z�j!"��������H�z-y�������l��������������g��	��n�\�y?b7q����K�m����j1��M�-��5,����-KJP��pU8v�����N8Q�Z��A�_q�8�C�sjMo�C��?��#�d�J*��}8}{�����G."Mz�����sK�������4J��k
/�x�N�.\�t����Wi:�dP�U+�iE�BdN"��1_�H��u���s�il��2�#?�
5�&W�q	��u����h�~��r�l%��K������FZt6k��T���,��
LVU�\Q��B=oPi�N�RmX��U�h%�:vB�"!|��������b�J�����yB�<��%����5�Z)����
����*X`M�W�k2�
�V��T���������&Z��|�0���y��,S������f���MK��g`�H�t���y1������������c3xECc�V<�����3.�}���B}+G�K.#u*���������w����#|�Na�
��_o`�5��'
���7�Cb��T���x|0Y7>`��_���X��T��"@N����(����v�I>+3
���
���bv� +�'3��%BV����� K
�l��O���a�N�8)�E6l����}��@�������X�"�f����:�T��Y�N���V� ��{�Qw��Y��|LCZ6,EwF3���m��F��O\�,n�L>�)�"��gJq� ��l+g��t��s3Ei15��y��3EF��LK�p3E�kh��C�g��s��3%���1�"��gJR �n�,�$��"<]m�$�LI�����y��(W�)�i�$��)����)������1S�LI�#^���z�~�[���"���i�`�-M�?��)����6�
����M�I�%�������1�)b��H+�]
���N�I�i+�f<Tt��-b��H+�U���>I�g�m��9Pa4`mc�F��V��>�y�8����D5Ta4�=��t�m��a����&��0���c����w���Ng�&XD7�<[�rH)X�Sb�������Y]e��`�Lf���c6v�l��b3_w�?�I�(��>O�B�=���.yEM ?E������g�c��4�S45���=i�Gz��2�-��_��Iv��tzs��&e����\idU�Xb��;V_�{��q��as~��������_E5
Z����
�Z��XEX���������|�
c�B��)�{�V�����xCBc���.i�R�GWFc�}%
v�<
����J<lP����2��^ '��rS��*�yD�
������nJ�:Vs�E��
&w�fhT��c5wW�n���#RV����:���x_����qt�S�-���q�=�_0��/	P�f����h���rq�Q}J��OU����=��l����7����:����
#�<�����{���~��1O�!Jpdp�����@��yA���J����i������b/5�3�\�z���g3������N`+N��������\�=9��*��&��,�^B
X)
����)��q��j�m�SbCI��Ol��z�Ad��bx�w<D�G+ ��xCX:_O#��z�"M���f���z�#�g����\�J	��E�����FX����@jj������)6XZ-�����Rn��6������2�1�Y�O� j��)��D�fA#��u*�n;%�W��v���%4[�K�(���*+:�/N"-�K=��]l��;�h��7���5:����l1x��JJ
������z��
����UL���6^A@lb��i��*�x��~�,����S�8C����(�w%�h����SK=�Xc�M�����\������Cu+;v�"�����*�)��4K+����ho�+Y���=�>��]q�=T�,��`S.n��;:B�8��E��������L(������p@���G8B���U�;�8�y��ac��!6�9b#��&��7���Y���Iv�����Xi��&���*`�K�Lo3(��7`\�|�e��*�@v�a���~�"c�"����rq�����#��U��T�S�Hg_�f���"d+�DoI%��u���u����(�(WU,���d���f��
3A�of������P�L7�U��J&�:�X��'��*��
hv�
��"����*�Z�01�>����H9�*����#�W�2�T�\]���++�3d�vS��
-R*VG��>Z�0�
�BA'��`������.��
�c�&��B1�,.S���������k���n�%�^�����2Ut;��E�M�-�^�P���>�/�E9�����-�c�Ry�O�-�6d���/Y�dQoTqK�wY�&jfv/��9U�R�sV��j��K
Z������&L�0���0+���y�\U��o����b��0:h�g�LPdi
>��}�����a�|K$p=M��yS�z�s� �zQ���P�h&j�H�Q��K%���{�Vs~!pf\W�M��0�R�X�)��q��H�,���C<a|��3Yj�a�C��X=�`��%�uh�������E����SOf,m��
�,f�g�
d���x�3{��{Z�Tu���
�b���b����<�G���0��e0U��������qyT���>�K��q����f��m����3�
@�bf���46�&��)~��C����_��q�����1T1��1&|L���^L,��)������v�_��&�ry�\��\8���e
}��]P���N`{�gv��������"6����gv�ifs�}f�tfk-��bw�,��]���l������ �WO���2-O���[�LS��@����"��
���T�P�D������D��[[��Js�L�D
���8���'���	)�W�D
i�(��L��i���{r"�g���S����~�i�69
GS�3���4��
�����~�6�#�y�C�`�:����������E���x�ai��/�l����~+�u���X����6�Zv��uQ&i�1��L���{������R��ZD��	;u�X�(FD�y�<�J��rke��$;���zq�_L�W1�E�#pzAu���_���	�tcYZ��z�R�QHv�i��M� �j��6���
��'�*[�*��?f�b�O@�J�_f)&g�N�U�F-��M���z�s	=�E_��W�
#�/X�S
�y)=�������R��W=��C1�P��|N
�4�����_b\�z�rW����v�����<*��*�<*�_��}]��J��J����XD��~�#G�zL��fT��?p����0;�Sb����u�E�#A-2)�2@�����;��@��*Ibk�xd�t������s������*�\�X����!0ia�y��>�.��??`�0����s�������A�0(��qP�C!nT7���(����y�gh�&�$ecV<��%��w��Y���g
��Y�aZ��:m-�������B�s]��o���/���lSO�U�0�il_�;�6{��[�C^��>_+�J�t.�l�g��_J���:[��������"Y6��eE`?+?�R����;x�<��K�������T�=��7��'5�gO�����m�>6��:�C�m�t��W=�<rf�������	��<�!�_���
H���6{�#�������r)!��mI���KpLm!�lh���8��Q�5%[b6��q�iz{l
��8���H���c�eLk�
�K�^��uG��|j�M�[	y��8*J���x(�J��d�1�w�G�$�����\j(I��ei�����e�p�M�2���&����G�'�X�� ��
biK�+��F��X�q���=:�d�K������R=��n�Z���n�n�(��?����
��Qs*������Z=ITO�r�F���"L�����\�n��<I�������]���V�����"l
���y�R��&�W�_�y
�t�PLt�b<b'wz��+����*�v����6U%]��Y�V���H��Xz&��������@jG��GC�����X�m��70K�~���:�U1"�Z�V6�*@�(P	E�I����r.Q��x�9���<a��1����L��:�Nu�s��nv�Am/�V�����q�6��5(�UE��)�H2���Z{a��[���^ic��e�A�5�*�]��y��:k"���Q��]$����hg��NgZ��R)�?����2��.=�M��3/��f��o�|S��6�n���Ms��L���n
,��[�k��WvO�+�Xc�67,/�5����l���o��&]}sv��Z�F79���Gv�^_��ie}p�^R@��,�����4w��&�p�h�
�w��~�h�m��al�h������n�e�����s�����2�Q�$��9�E>���"�����VA�B�1k��)���+�L�v��cl��_�Q�,���|]B\sD���$����&;b/�z��v-��"P�-�1���=�)���g���Y�Hm��G��V��"1<��j�^��B������v���>��:��+����V�On�`����O#��69��2�>�;�P�����2�=w���[
�)u�5y@C��=�2h�J��M�G���D�]��|��@^��`����^n��gq��VSKt�Q����1"RY<��������_����A����0 ���t
�h\/�}:��9+G�2�L�m^K4�2.6�^�X
}���zF���eL�0��a���D��a�Q$�y{�-��P���A��t��O�)���*jPCA���C^����s��>��k}�������Z	��U�wBL��R�E~��w���+�'��*��)�h�iM��JAk�n�IDS]A~D}y���M�9��U���S������k���5d�#��_�/)�h�u����&���(F^�l��1�{�f�V�hq���j���������gs��Jhm[��U��]����Ty=����XE;���5����WE;�#����5<�2�����J����ZG��F�P���s�hG�x���pu$����4�����/YA�=}I��W��6Y��m���@���mkOu�>�I\P�� �1�v*������'CxW�q9���1�������c����[(R��f4
U���u	^��E0W����y��C��D\oj�=hAm��6lA�
V���,%E���6���K6�/)�]L���Iw�cU��F��AjR��ZP���R[���"��6S�h����5�esN}5������������6�_B����$A��U�(�V�/,$��-v����j5!��-��M�����*��QX�S�k�Xr;���U%n/�s��d�G���F��J�1��N�N+��}.��slr�����e�� 3����������*�GPUT~Z�����(<o�W�}�a>�������:%$��!��P+�
�G�SP�T�>��8Sb�L��J	���Bb:jw����x��} ��q�$�9S_!�LL�����%�p�_k�����U
���j\�o6|[��<UtI��������
���vYC<Uz���dw����j\X|�:N����,��k�]�����75��
PP}��|V�g%�����
>����W�Y��H��:Ud�#�^	�].,k6F}V�`5�C�
���
J�5����G9�O�p�J}�#Vd�G��t}HJ��5���.�����'(8A���'������<g�b��m�>6���_�����i�u�!$����"�K���z
0]��Y���w�y�I���/xyQ�����!P��Z�)Wb�(0*����pc"�+�������6k4����������vO�@��iW��$�[U!����z�W�����e���2q����I��cM�F�.��4*F�a$�������� �O��L��T�+��]V#Y��H�
F��a$���K�D����V����R)K[���'i��R��>#��'	��3�3�fLA`�W(���t��������`U�a���_��;�1�ko�&#^[�y������|�[���/�����Rm���V�Zv�~�e��8��sSW)������h��D�q?z����%�.��ZW��J$�N�(�}�f����;b/��;\�	��ge���������a2���j��1>���MN]�m:+���B�e��&6"R���e#:�WMY:�M��/1����7C�e4�q�b�-���*(������Z��d�X�1�[w�c�����f��5�k
��zS�t�������S*n[�c'���_v��w�4��-;�Y_��30N��@�zB<����#�"��=������hHf��f�]�C/+r�`�]�b�
=�@L.+�9B�j�.�j+P��h����Z�|�W-���Z����gi|�+���R�%���f��T��
N�$;��'G������[�.�5��hG�������yr=~K��)�*�L�v��k����[�[�\�I�v�����F4g+��N[�����Gv�k��UP�<(mV��##��H��3g��E!kD�R�l��jM���B#��x���W�.���I��r�����lo%�������n����1&(X6��[f���WH0^}��6���'P0^	����g��.[����ID���R�$!'CMv�6�]?9���'�^m/�=m�j��f%F#�Z��?�Ei�0�#���|k��Fi�!��!�|y
�1���b������������s5�z��-���Co���Vtk	�%��S�
E���:����/n|G#<�0�T�����X�d��Q�,��C��8��A9%�����b�cq��!<[�"�t��J �S�xb��'�sr1�2��^&x}<{~v)m�=�n� �^��.^�??������������J�?p�eN���������(��m:�Z|=��a(��@h_�r��q�i�L:����"[mfr?���I"��1;��E��*_���
(=�H	(����4��K���P��_�!U&���Jl����0�^btl���e������/0��=����"����U9����K���-�u8�V\�� �&|�#��Bl�`'��"1F%?��-A@F��@����,.�bYj�?����(�v\��5����e���/�(-�v���������{����=��8��b���G1z���3����Wqy Fbe��H�.���:�f{[��N�vdM ;7��7<����C����A}��5�<+0k���*l���������x���*������xx�\l� \���@8�x��js��t`�,)��s`)14Q��_����~���;�J�`+�K�/�:�77��&]g*k6z5�y�Z�����F�+��K,�>��;5s��}����M��a������-�`���u0Z����G�c�_#u"���CLCb.x�sYq�dY*E�bV�(Q�N�B��[fw)�n���m6_���8��h-�v�s�1+��9�D`����������3������>]"�����^�9k�����n���m��C�����s�&��Y�-�dk$c6R�����je��1��>?}����)��%��]/���3��&�t��\�r���}���.����-�%�]U2^��p��9�]L~�X�Z~�jC��zz���r��KL����|�LIZ��b���������'��r�9LcK���-��*�^�d���u��:��*C��*��*C4��}�#�����,��F�Rfz�z��ZE�jA��j��*��B�?�����;�l�5r���*�Q�������V	�Z���D�}�?=�q��	���AWv	^-��W6wW��e����^u'(A�
D���@�+(A�0

���@<x� ��8�J��d�\��[?�����K��8{���������GRi���J1�
�Aw?�_����}�I	zm����_r��Rd��g��;����g�o>������������������)�1�`��w��t*fxbD�8�����W�p6��%!r��������5Ih�'F(�����)}��.��/�N9��fw7�?f/� ����
�/�����,S4�`�G^FM����w��\B�����btyv��2����s�0e�D��k�2��d#3��{(�r
�By�7!�w��i�
&y�ZI����w�.Oz{�R�E�s�"[I���VE�sZ���_b�q���C�u,e��#�����f���O�Y�W�t�F�l��)��j]���<EO��;�8������F}Ov@��K��X�t,?
�+�8�\�I��8�3n�'x�'��j��s,�8!=�b�E�y�S�����JF�
�0�g�r��"�U~H_���|�&YrV����k������	���V�,���(q��k�b-X�w`�/��l�����l�fL�"�Z^��|��j�&�}S+�5���RP���T]�T�h7>�����:�E�I�d��/�@��qttTH���Kr2�_N�/���2��~���_�R?�/�;�c�����(�b�t0
�����9���IQ�7���P�J�C��n%�-�)�u��p����������Z��u���=�K�����=����L}�|�E��w�J%>�w�2)���(��,��Z�P��X�h(��4�,�Z��#X��	�6�hrM��#��hzM��+�f�E
(����$>.+A`%
��a���K�@�3������x	K�`���������z-6�IMH�A�62� ?�I���T
f����dc��y���?��.?�@�?�����V$����s�D)&��HOa*SA�
��h��+SA��&L����q�z3��L�@�������#�K�� �;Aj_��}���_���B���P��|HPm��S!�J!@	X������(��4L�������V�����8&-�?>��:����|������[/�?�5$0���?�G�s�I�t�Y�3�!�l��`�jY�P��_��p����e�tSk�M��$�X�������	�3�(��n_'4(U�kmB����u���n#�*�_���2�U9��J���e5��Wk$Z���yI.�V�l��:�����9����9���.�/�fH���K���}������ r���0��A�h��N�H�������OW�U�%/��d<���X�� Gx96j���Qb3��`G�n��)|�����,x`�
5/����'�f,�UeOz3������7�f��)�iF���\f�q�E��\(�8T�1��9�4;.V�k���������:����%�����
x����5��m��a�r79��Y���*�G���H�?;����~vh�Y��QH��������8H��&�'��~�?�7+�-�����?Y�Xu���e��b�]�r���"/����!/:��
H�C��U@]���hX���������|[*G����u���>�)�d�j��6��>�AL�
v��14
h��8��(����+�X�K.]��ym�t{3��xt��cO?P
<�[
k�����$���?�A��V�x��+���R;K+=��J�RXi_�*��,Cy�������]�X"��V�$mIG<��F�t��"m�HG "F�x��[6��|X��GW�����]���_����p�����d�sU�Zns������<�B`3�F������������'�eWp�����B���gC���[-^�a/�5�eY�zFDb8�y�\�y�1��!FI���h<���%��������G��kh����(5��4����7*����G�*���O��al��.z	�Sf��i~Q�Ws|������U����n�8���p���:V'� ����o�d��G(�8D�"�b�9j�#����
����r86����k@t�#+R��H\~@��d��O�))xO��C����1)�_Q���:�2m�'��$�.�f/��6�S�T/�K0�1c,F��E�C:������&�a�@N{�{�2�k�,q��K�s��,�8��%�%�IoII�f��������`o����	��}���@�m�d!���y
+^~�0�uh����:��P��m�+��#`,n_����B���5#e+��Q;��A��@DmECA�.7��A������b?D�������>��(�n��^3R���_����n
�A�V4��Rp��������;�/�C������|��B��46#w+�F�;��A��@�nECA�.7��A������b?D��J�k~?����7
�����\�)T� {����(F�m�������2��E8�bs8��4���,�;��)�p�8+�M���������N���XQ��$��Uh�!O
e:r���\��#Wx���:rM���DuY��I����OV�����}ZAmjC��r(���E8�
v0�!�
&X�����!	����������B|lf`5�\P������6hj��Amj�	�G�6�BhyHR�n����Ga���1���\
^�4x���@
�
�A��
{"(:�������n��!�����x�����2���zB�����
JDP"����D%�������,I�l��P^PG�U��pm�~�+�Ae0<T8��P,�
*CPt��AM]��}��C�7��a�B��6t��i���(����_�]�\O���:��h5����VJ$��p��/f�6��'$�cl7~�h��a��D7����@/����@v(z���Nz�X�q����kTj�R!�*�xe"oq�����d�*A�t������[@��6�k�~���
��2)���������I�`��d���z�x��+W��4��z���v\�`�������:��`'�e�jE��n�\L[��b��v���p�Q�;T ����Q����"*
r�((�_T�m�$���T�E%a����������`I�����^5��v�\����`���
R���nv\G`��4�z���V�
}]��daU�V�%6V���j���:.����T�d����������}i��b�}}�K"�q?�0Z_2���Xo<�5GY[���RR\&�r	�7J��U��V��&���/�z�����!�o��/Ba�
D���,~��qrD�����������]/�l�����s��An��� �u�+ /�0�����<��Z<���_	P���������/b����1q	���:@��-@���\���FvRy���A��)ni`���9p�s�-nih���9t�sh���R���e����V1#����
�����j@��7��Th5�
1��@�\����v��v���msE6Wt����"�+��u�\�����m����n
��p�Q��j������V�`�U�d�*�y��}_�3�D�
��P���X�O
��c��>|�%���� C����2?�^��6��GCE�T^8�r��� ���%�m�u�V2����������))�T.-���jY��}���.����tUxK���x�jJ�������y�jJ����,���y�sv��V+ ���X��q�6���'P8_	����g��.[����)��]HrA�cQP�aF�+Np�O����	�g�	���v���m��'��Go����9�j_V��-E���G�����\������ex�6*�Z�=2A>��{,����6��&�c-���:I�~PN�������������pwbxp\TpV�E|M�v���s����mr�F�f�-^��!
�&�)(WV8����q����2�������K����=�>���������O����>�xv�|���S!�j_�`SP0�r4�?d��$
o�%�o�������M�M�#4#lU9��E���R����O�A��9��u��}���l����0v�N����Y?����c�IR{!���'��T�6�u�~F�������`6��2p
KYB��h ��G"����X0�
i}/�����3a�k�1#���e���o����&�<[�����9�W���tO/.N���x��c��FG�u�P���y'��_������r+�7MW��D������B
�-�W?%R�*�X���F��v��������C����=���s���e#�����K�2V��V�0R0�
����2���5k8�+�W�/���&kVQ����X�*t�)�i�Ub�0��������l�*�~���/���/�~�MN�r|��~��P��;�D�u.���gy�U�V��M|~��g$���bsq��}N�J���!�V�[��X�L�(.9x�6�/*:�k�`��o���s<D{��\��8��Yf7�������o2O�WK2�K��q$�h�.1`{�]?�������N���>��J��P��h���$*�/Q�_�EpX�MXT"�>�
�	KJ�r������D�p�g�'�B%�,T:9�n���������|
������S �O����al����:�����"J�%"+��)��D�Y���$�����g�.�`.i��2����	�{�-o��^��f��OQ#�()$<��6g�eh<��M���U	�j�3����Q�������Mi �J�����(`��ElLN��#�	�#A�"H.1P����@r����0����@�$'�������O���pv��}����e��w��,��2���j���N�����?\D�Y[-o��/�rX��E����^�������q~7_]��I���m����1��h����7�2��zt�����^�����
�������~�r%?�@n�=Zf_���|iw����e>�����N2!&�B@����;���P{���V�QA��8�*��N�`I�D��~���6wW���b���^A5�yA�r^����WB� �9O�9�+�����T����&��
�����7�_B���[����Dg�.>�������6� ���������[<h�
�4P���Lu�zZ:��<��%GQZ�������P��������������{]�_~�x���5�n���������G.�&#����	�	{���]��������my�(��[@r�2B��I��w���t��9���f���e����d�a����]B0*hQE�"�y�e�fF:�����K8/>��`���J������,�L �����L�7�z���uyw)�/���a��=��C������y�����Oo�_�1�������G�4�U�:��?y���u>�pS~	#����eX��~M(_QN�~�w)�p��w)�p���w)�q�� ��<���?B�X��B9Yf)�y�u��52�nf��,�����E��S�>.�[	~����+p@��_Jo��7b����9[��f�_�9��{��=�������S�=������
����
��T��(��C\��@��Z"L z����J�v������]rS���-����d'LfY0K��dJ#���&���p�]9,�F^`��-�`�k���X���������X:�/�j�@�K�/��� �U�r�r���8z�Ku/W��(cG����h��,!A��@����x������e7y�)�����*S�.EB�7�D���",����Ve���}�|��P�5��9V	�:`�[�����>�y�?�[p�)���M��(��#�&��p3or���8�}������S��	��d,'���U��S��Z1Vh��/�B���{�
W��]qA%<�	���GES����P�2��V�+��_���/��enH��tp����qrP��xq���@��0����|;��,��m c��Zb���hv�"S	S<��bN���x,Q��-�x4	���Y�Z�7y*��E��VDMHbaB�C���B��J�	�
LHFUb�����[�2!�b$������-��r�EB�'K��[�Vw&��z���*Z%�l{���D����*���%R$�VI��4)	���>z8�@,�%	���������|��P�������c[H���O���Gz�4d��<��/��*�#a'Y��Yu��e<f��:Zl�;Sb�1@in���6���<u�$����RS�vh*�2;>MEZ��.��5|�8A�O���"",�2������Rk!�xTv�(��t���D��"b��U�ZX+����6V�C��KF�OF�F"��d5��C�2*85��6h�|]���=g$U�*��x����g���>
z�J���c�#nD��K�T�5|����!������`D%��R������%���u������RW�^�K������_�
�)��60��S�`�K]���H�>Q[+eJ��)�s�+s�k�+�9����;}E�`�4��U1"x\�\L�ag/vM��h4�|�emH����S\�P�<Z�"���Ji��U�M�b�������G�SW���w��xgx$�Rd�J<��D{�r���d�J6��<`�b���W}�������,7��R���4�gRO�x��+o��&o�+o����`�������`�	��`�	��`��k��#U���&��\of����4�ebG"��Q�xy��pW{@���i��9���@k����H��p�J��E��6~ZI���*@O~�r����������3��X�:"�V��
���]�{8�Y=A��*%�q'k�s2,\J
�HG8Um��E�__e�l|���-�6�n1����+��2=��j(+�uaIr�,�|M��Q����l���Z�e�����	_���z[�+�|]�j����	(|U;��"_3&I'����	����k�	.��}��R0T�2M��o�S�LG��g5K������E{����~�(t@��.�A���Z��@@�=���ZN�Z���N��6�#����K��R;r�["=3�������Nxf(�#X��y��,���rwl[��8�Q��f9���A�-GZ�o-*�i-D���1<��
���J��2���E��&������������S_��ll�|z;��:�����v�������#����P���h>�������K��!T�EH�)�M��V����:O����,����-�]D�Uv;u������`L���j�����>[�Y���V��F]K&,�po���i�M����j�	�"[mf�_b���r���&w�y����`w�7G���a��e�|�!+�+L���*2�����������x��s��	k��X�	���	c�����S�����8���?�8��1a�&�_#�A�_&d�K&T�R����WUWE����O����dw�h����wW����������jK�#�7�n���1_�[��/��8�0�s
���gS2vj[���s9����`o^���V�I]���zK%��EW�/�VQ]����h%��N�,�DzI4�n�_��J&_zm%����^I8�%��
�Z��~J,v�!J��H�vh�T���aS1�������
��?;4$j�$@������	���~��B�o������J�<��;���x$��{�����������m����h����&�����g�0�����:�Bl��b��ExA�%����q�%�����?��������?��������@�
C����}�8���:])���L.G����D�dG��4�-��u@k�{�{�&}D*�:4Q%�gWp
���^�0�B(���9��H���;�4�L5�%s*0�6'�P!'5;>NX�BJjV|��A��� l�8a�
	�RX0�P0�[�FtI���HN�l{�����M�xJM�"�8�S�rQ��N���*�SO�ZJ���hQ����2�!5��
)� �����0	�1����)SNP���GARTix��O��z������r�Z!Y�������e�c�~&l���	�To�Za��q�P���M^����K�3��������U��n{���2����~�Vg��c��+�!�����T����<�L�I��<�U2OB�y��~����$D8N2��%F�K�R"$'DHNJB�N%zP'I���N%���,	Z���jB����L�I�	1�UBLBb��'�$n	1	Q4��uW���&N)-	Q8�p&�KHJ���I��_Mv\=�ATju`�J�����c������E�Jg��O�>�U���Np/P:{�]����#��U
��2R�\�P����*|��Hk��r����N��4X�������.|�5������a�b�����1�T�2pF%�(L��a���S�O��A������eUe~n��ZZ�<XX��U�����*l��>,��l����|),������2��zR����XUOJh�����IeV������
�|)����1�����r03F�2��3f���0f�`66B��F(k��O�>��{`bL��l ���)�d�
f?����������{�S��8B�3����T����,����[fk�tKj�O
��}��������S=��^!>5
�������!���������2�P�">u�#!qwVN�=B�j���0��'�7ga{j���,�!V5X[��5���8�X��bUC�j�Uq�`@�;{�X�`B&d���l����{`jW ������X��!��K��B����������VY�kq&��V��3�*��sb0��U���U�3+q�W�7a$����h����M��9�Z�!���`C~�A���YK8�9��@�sfwv��I�h�V����_�R�j"E���U��j���8�kW�bl�w3��1uw�����w������r\1:7O0�m�����5��V���������;�\��4���P������hB(18�|XoZ��p����og4�i$uC���T��T�����T��doN2x�+��
@����F�J�CO}O%���Y��A��)���S�|S�pc6�-4��_�6oc�6*�M#�������%A��_�<!hQ@2-�������k��&-��2�C�b�f��hh9-��E
!h1-��C��^Y������w1�L@gs���������~��Ej�����-0Z���
���E��C���
���i.uC#�/���B�b)h!|1�/A������
�pT)�!�q���F��6���������C0cf��!�13�`��hh�6 �
��!�13�`�2�B0�#0�`��h��3#Z0�{�`/��u4f��<����
��`���`���`���`��oC;��W�~�w�~�wb��������������������`�������������4���������"�K���z
��N��,ZMo�����=s;�G��HU�����	T���Zq5`�V\��e�]*�w.{�=(��_��\���Q��C|q
)�`4�)�^2�q��V��VS-�U���,��{c�k�����v[���H�~QK��F:��c��X �CY2"��C,��`�4RK��@��*{:$��H�R�O��}���&��Z����c�B�f���':�h�&��dF&������� ���k�|���K'|a|a|4��O�m|���X�'�q�D�n2���{$��T����u
�?P��6�������6���������>��R	�v��'��u���T�WD��%��14A�2}�e���	�����z��'���b���f��}�����/�;Giq��]b��`����(���7Y!���hHb<�D���Y��h��m�+1~Uzy�>$�M��!8�kX�O�?%��-�/8�k�Z�����/����k�"�+(����\�/�/��o�3H��LRS�/s�����1b��ER�R|<&5�B��:%�����O��TA�����m����c��q������m�'����I�����s�����������.���������E���v�����R��x����Bn���;�C2vJ��������@�JM�E�NY��pU�hX)NhX�O�J�
Q���B����-'fd2,	��RO%���
��)�������0b�@_��K������1#cx*�r�*�����T)��W�J�_Q��"�}E�;F�[U��A
�9k���5������Yv�9&�@�� ��r��	�����m���,SK�<'�<'��%k@E���������/\|��a���R��_�������:�/�c[��z�|aTz��RK�RW�`L~y=
������5)[+����0���{s��Gk�����(&�E��B��8X_����:��V��J���	��z�����X�m��3K�����0K����m�3K�����0K�����S�]�����]L�5f/��,T��M��s��
�"���2��"P��;jak���p5���\��9��v1^KW3a+N(�	�p���fn	�j�n����Cx "��s��J��^�i��|���e�Z�}�T����:���7�	T�W�����Y�����z�tR���g�E��x��k�4����=��Q�=���U�M�<�%��ko��s�����%�.n�0��� t��������0����V��c��1K�UA�XP.G�N�nU�<��Q��[P���r�dYv��2[
g�Br�G<�7Jzo�Z�2��^�]|<{~v)��=�n� �^��.^�??�����������,��<�����S�SDzQ���CVw�n����j�����B7�EB���'��	��������;���1e�k�Z����8#��E��T�5%����Y��_��Jek�kFyU�����=n���������U�#��	�|=8*��(+z��HJS�+OsYR�Z�g4���C"�0��XVe�}/*d����2�h)G��?��2����\t�B����������������^\���?�����>����3]��������Ds���+���-��|hvQ��c��N�<_��L�|Z�ZJ��U*
$9Vk JB�3�a��:<��`4���bs�� N�����A�W��.��\�M�r9:��OT��(�F"x��R��������J�;\��#rzs��n�u�R,��������}��|�����������[F
�Vpl�	����*C+8��-b
:��?EvF�x��^X-��OX��������6A�n�����H���|
�����S��N�������1�����GFVo����O�{1�L�it����O�)0��CH��_��E����	�c��{�7�{c��xo�,"� ��[g-��BcI}~�����S��K^��^,��g(vMf�f�9V����U���"��+�k��X��`2tx���~��'�D*I	/����*9�@_bz7���B��4�*��K!����������F��A�(ca�n�42h����&�`���u9���k���s�<X>J0S�S5�[�C����g?P��i��N���y���->/"G0�|0'I�x=�1�MN����A|nB#�Q�������m�����S�>����A�#W������t8��t8RA�:�	��������Vc	��l�+X����_��	]vgoa����$:�t���J6�hG�^
��>�<�5:}�'��1�����#���yzvq��?zv���{����z�=�.�/?]������}�]N�b�O��#�?���*zn����$�T�������{�*
5&I�-?x	�S
�]��_>��r���nV�^J��)&_Xm�%t������P`����)�9�����K8�.>�����6+04W��tV3�f��^p3[���l�%��u)R^_(��!$����;'�������w�>]������X�	&|�9��IezUF!g�j����eX\���%�:�2����9�q�f��L�Y�w�t�F�l��r���z:��d���t�.���<E/d�4?0�HB�NW4��{"X�_Jo��7b�H�R��3P�;���D�:L4[/��['�'�O�pE������b�B�9�0!8�2E���	���^�Qq�����l���8z�(vL�T&���j0��*�CP^6���],���A�����J��o��e4J�t_��X���h��l��>lT�/
��t^�b��&�U���������������i��l�n������Y�6���z6��4�M�;��
�^��3b��d�6>���/�U#H(��VpkX�K^��6�*�q{h}���������M<AK-�UL����A�hk>q���80���mw�?8a��@&S�~��M�����B	�_�_��!�����������������~a���>XA��r<6a@,�P�*�[ G>��#����p		��x����'��w����RgI�Y��Les�����6���_�3����Y
�V�tS9�m�K<+�K��6��^��>C��}�}@���|��<f#!����\���rI��L�N�	����x�:��2��������b�(�xB��4�<���H�XX��^S	B,TVq��:5��F$@������B��a�
�S;B��$hN}@,�	FI �`��q���Ip�A8co��a�����n��+�v���	i����p��B��
��T*H��%�\	Im���r���(Qw�?�[�I�?�o�:��Rp��G�c�e��h�Y�J��j���[�t�]
����Z����1���3R�.�7�������G
Z��YnW���AZ����u8��F3�����L�*A�^
$��4��)�s���F����b����|U)�T������ p��l�����Gd�G�W�������Gz�Y��q�#>�|[fgnw�l���R�
_��zU�~^�v�p��OW�Q�r?]�G1����>�n���Z��6��k�������&�H[�m
_�
�U�~���nV�?������P�O8�OD�O��_*��5����m�����g���I���U��`E��M���|����,�(�8���?�s���������-zE�����k��������b�.��������$_"�)�Y�R�r�?����a9������]qj@�����`���#���evv��+;;�z�n�(��B��4��O�W�?�~���&1���{;��r?��Z������s��'
�T�o�����\�}�Z�~��w�h�o��G����%���	��
��!�17��F3b!�17�H���!�Qbn��&}7���:^of����P�s��a����+:Z]~���C�ctl|�@��h�Kt�B�ct��!�1:�@��K��s��@�G�,�!��A:F�t�Y�3����Vu�Lk{5�=�y^������l��W����d���E;}����5��q���V���h���B:����vGI�����d�RY��y���W��W���0��0�V3�������U���0��0�T3}��dkh��4V
H
��$���J=��2��b��]|��`"�_r1�H����*�+�s���r��V����0�4�����B��t�^E]�J�=uK��_���/;g������}�}2�;cFnQh:��s��6��I�K"P��4��p�r7�y=�/�H1��Q}�����G�kC2����U#K� ��`*i��D�]���12�%��h���v�����10�>F��2�kL�:��)��>,��1���P�5��������d��1eQv���+��[%�@�X���B�X��jYC�S�����ESSJMN�#��������>l��4�������,����h]�����$����������E�dJ���Y�^p��*,ZX?lYt@Yt�������c������]<���!�Z���j�����EsvJ���c�E���������E��E�,�)�9�,����]p,����mpLm��m�Z�]X�p���R�E���0�t�������:�Qce�Cn�?;|8��dd����k)K����v�GEy���qu�X��������t���z
��#4��-���W��/W����)\5�w2�X�M��_<��]f`�-�"8n�G������,&.Ie5���r7Y:�a�T���@��V������L[v��
sPN�9�E�����,2.�K5ly�r��9�a��R���@��O���E���Yv���VPj��������j���������?;4�&�������CSa����?;4�%���D������Z����g��6�b({�nU�IzQ���2N�C_���0'{�O:�h�;�38,��f��5aSk��`)SUU3.�H�R������B��P�7��5����{9��In^j��iV��'�Q/!���Y��2�����Y�F�,hO��3��(s����N��Q��z�
h��FQ��tw���xH�:�mAT���1�b��yw��%:����D,�m�\��#9���u����y�N��;
��[_Q�&�S��|[Uk"��7�@��=��(2��7���8��+��{���Ev�)��\���}�u�X���wfy'�*|����*�KF�F�������8-W�K����J"�EM��r�f���:-�Gs�n�}&+wZ�=���WL�V���)��������i��S2s���J��q81jZa�R{�-�J��a�r��*��Si�K����W��u�3��w����s*����Z
l���E����U+�!�:Qa�B1.aNRz�rW�/U���%���EG�z���Zt�
^����(Gc�h�C��b���@��N���<]O��9Y.V+nAFw����&nX�#���bQ�>��
#�N���W���������R�a�1M	�1t8�X>����S��^�%]��)�Y!��d����X}�2��B������vuL8�T)3���w)xi���������N�;�����O]`f/���.`���H���'D�O�$��,^�~	f���g�>#-e4�?ac��#s.�_:'�u8N�y�����*���t@��J
%hX{	�9b�Jf��R�y#���;{qKC����E��2N�}	a�������l9w^�"�	C^�t�����.zn���^�Sa�#�/z
��/w���
+��!:��TX�dz���E/!�^�Tw���[
,z	Y�*�3$����IJ�;��
�����+��-�U��o)�m�s.th
c�ne�Zw�P�	��;���NOk���=�vkL3E���������)8��[zQ��XQ6\�^D���N�)nt������(�b��-F��Qx@f���=��9?���~
JCWp��\��!zBgHp�gH9^��!u����q�,8C�3$8C�3$8C�0��r48g���	��M9C����="�����-���|��_D�j����+8FJ��	�=!�c$8F�c���	���h�Hp�8`#�1#�1#V��H9��c�ep�����#{n�}��iH�kD����f����=���4t�H�#�1�'dp��Hp���#�1R��	���c$8F�c$8F�c�
��)�A�sp�����1@��cd����/"�Hp��C���5VHk����8a�f���ki���.@IQ�5�:D��OY�*��+g�t ���Pz ����$�ly��'=
��=��%�����<*��4#
1-q��j9�
������� 5%y��	��
�A�/��f&�\���m�Z�7��.�u-*�>�9q�p:"1����N#�4��=]��=�a��Hh$
��PW4�h�iE�s��+�hX�MC�s��+C����k�Jw
���YJ�tn�~U�7w"
�7���������������Q.W�C\�CY���^S12�1EiR	 �?�����oWH��vtW+����;��d
gXU�4>/G�������OG�t0�����Q�9��;�Y4�F�%��\����':�:w������^��3�o�P�]Wc�o&(���j��b��|�`x��������.�����4��]�r0���`x�w-������2@�����{mk�����X�w]�g���0�7Rh���nN��}�W0��{0���~0��{0�k��w��������=��h�{0��{0�����2����m���bw�q5��f&]���wE^J��|�W0�|0��~0�|0�k�&x������	>���h|0�|0��|�&�`p�'l��xU5��^V�����o}q@X�����
f�`�f�`�f�`�W����3|0�3|0�3��F0�3|0�3|0�?DK������^�@0�;���h���;	KA0�7{�{0��{0����{0����`t����+���]�F0��{0���P0�7otF���Z��3�L�-~ln�:]��L��+xQqq��.��t�������t<������!�Z��!���l�=�:�`��B��1N�=~�z���w��*nw5d;�������������J�Pj������^�����Xx������V��Y.����>�l�[�_j��{3�o���yX��r�>"��aQ�<������n|l$v�G�)�+�W!�	bP�4{�c�X�^oXPB_b*�So,�����c7��h��>h���h{�}bO�D�}"�>q�}����=����@�������t��~�W��}�1��l��<��%"�H�
��2��O,����W��XYX�L�N�������F>���1,�Z|b�X��}|bo�cX6��$��$��O>���1,�Z|b�X��}|y|��c�����W�������.�1�*�/F�E��h����m����g�=Nu��s�2s�#�4ym���P,*���U�����*�*x�_��`m��M6���e�4r��}��c��}�[���J�l.l���*E��D���H���yV?��R��KTY�H�J�c�cA�6/��A�.�'��e�{�����^��&�F����F_�����G_����Ab}4H3��cN����#�V�c�zL��csFzA������`���c��<�EMn��-�$��M��0��u^��P�7
��3�{�@���K`��S��y����R��%d*�cf%|cy���>��D�`k'U6K��i(�Rg�Tl�4���J�-E�Y.���r]�8Y7P-������902Z.�0�����H�U��Q ����d�H�U��Q ��-c��K���X��b�3�Yg�H�U��Q �������_T{i�bO��g���4����Q*�~����
�?��c������*���Nb'U;in�y^������U{�����
b/b/b/�"�����TI:������*�CG�mB�j�^.���^uaO+�m��J�����:�N��:W'���^���:Y��\�P'k{���d��\����W�<������'C��w\7����U�N@'��2�N��p�2�s�s��2���M��+d���e�����W����{D��#��������W��,T�n�o����J�
�/��������x�w���5�����9>����]�h��~���U�����xeN���:����m����:�,Ji
L�kaKhR��"�"NR�R���f�]d"E9Ki�d�^9��%5�F��
��5���um0Z�,�ie*t���N����F�6�a#~���������h��%hh���hh{#6�~	Z��
u�����P��^e���N�0�b�V��b�<����e��Nk����f(��n���
4B�MKw�u��"�;4��~��q��7�<7�	�����eu��e��$W�+�h\P�s�1�Mr��"o0�&5��h3�$��+����G�c����yE�`���J�>�����QC�!���e�ZM�����j��F�����W�����Y�����z�t:�	)Z��8�%���>��0��<9���'gx��6a�m�y�c*j��}���rr��H��&�����k��b�Z�}��5G��f��M����������R_���%J�K�����[�%���/N�������I�w|�"U)(L�v�GU
S 7����Q��$��;3�R���A#X�U)M����@wp�9�`����UY�����Rr�DTL������������~x�������>�	�yB>�l����?�����~`;G_�'�ax�;����	�1?�O��1��1��?�1��>�����O�C�v���O����|@>�l����?�{����7J�����=�fl�t�{�I������K����n� �^��.^�??������g���Y���):�WAgm7J���@A�)�r����.��~�C���_OonL��O��rl�]����}�hR(�3���R��R�d"���e}0�uQ
]0�A~
������j%h����i�����
m�J�yN	�KLP(����"���!��8!��b*�a��e�j����?���
�P�F���*��?�%F@$������p�7�^�?�7�6F�0��#�����2�������3�g��~=�8��������^\���?����,=���+��B�B�J��h�Za� =t�O�,x8��K����B�K�}	�/{�bijhW����������xz��,���j��P�e�Q6�P�3@n��&b�}���A���L��J�|�C���7�������~cf��i�� Q�.vlw��Qtc��5��������/���#��c�_�������ev��3�����d�@���/�~��E�����O�!!�!�8���$4BB�C�!�Jq���8b
�!���6�L	�	�����Y�3�iK�e�(�la�y��.�,w��.���@[���k��o���^�G�v�s�e}���I��2!�?yYk�w}����A����D&��(:�����K�.����_����'�������LL$!�������2�����y*���,��y*�a�+�6 n�<���x�{�
u�
�
��=O���r��D��]�s���O��!X+���hh`�n�R�9Ex�bk���__
/��Nz��k�'J�)��;�	��=�VJ�/E�SN'=AI��J!���r���'(��@K������B,��2e���a��%e��#���#�[R���l��u��O���pv�z}����e����,��2�
u��/l�7�����"��+P�|�gr��]���������qu����+�tl�}���8������C��R�5�2{P�����+EE���'j����VDfRu��1��P�ddv�IU���G�\)S���hN����$<:�
����yR�.Gg��S��)��](U�rt�<:����?l�VR����Ba��y��f���rAUO����+�2z-9����9���Y+�
�q2UisA[eW�`;����o���G�V1V ���4���#8��U����sU����6m�]�%r���wKf�����J��������������
��`^	��`^	��`^	��`^��R0�(��`^1 �+H
���BgL��d�\�C:�����0���-�����Dg�.>���'�AH�������/�~�N���P�M���<�Y������1=����������������/���u� �8��t��#`�k<$0����)�%���<rq�3�_EO�M����d4�W>�v��}ooG�"��u�^Fh<1���.z{���O�����nV�^<NP��&_Xk�KH
����W�*���h�u����7�p�^|�M�F<�mV`����,����"��v��o�L����=�
CY|��H�����~�4�����$z����O��?�=�V��mS|��|��K�gF��������&r�h2���#�/�[�_�K�_�_������__	%ix��pb�W������&[E�|��������f	�"><��}<���u�.N?�^O��������/�2����AK��x2)dQ@�.��L�#x��o�'���N�Di�7��]
����l��N��S4e�ol��0�/��Y._�3��as�'O
�x�����#:�����O�1_���!�����3��RFq��XX��g�k	~��WE��3����c�rp��u��x��=S� �)��O�f���D���E2S�G@f����
�v�('
�P�2��'���U��O�+�M�i?&����XH3T��l&{�!���\�K����S������[�
xn�G\�Y�tk
-�q��d5�
FQ;���\�3\s�lAu��zRKTkZ9r��:�*VfK��+�(��1I���3�����1���,6�6��d�r�{j��,n)N�PqQ~��X�T����q$�����O�8��������j�Glhn������@,���r����zp(���gc���5��;�� =`j���&q�B��'^q�/�����.F��!R$4�-�I]�Z�-�o!m���@�|o�����V��WkX��z�y4J��%�Z���K=9
�(�/�A�aPKE���:��"T�	���I��^����������QL����0n�<h��E�%���Nb��E���r�z_���������S���B�����oG1����6d�%�h��I��a��6��������f<`�s=D\��	�_?9�Za����V�s���OF��o�\+'\����Vb�=�!�Cs���8*q�0���9z&G����V��'����E�o1�[���o��-�����__��[w��?�!�&G�������(��%�����7<a�r�5�Fs�?���xX8��o<%�'�'������D�A��3�!�F��Qw��{�����G3���c�~N8�r�rc;����q�o��q�
9�������!7�bnD��6y.��H����9Cn�
9��G<gqct����:��D�l�8rp����-Ok����>�W�<�p�|���\�"n4cn�������j��0G���	n-�V�h��x��k9���}���6����5���<FC~M�~����[{�/�}��Sy�'�s�~�� ���'��C~><�Wn�K8�����q3Lq2�w�o1�;��p����+|�<u9
��������=������(�p�}_����_Q�]���[�#~�8���+<=�#�U�w��������J$p+��{1�Jq;P�A�$���A�A6d)n8�p89��y���!O3�{x��������S�Y�Q��_My��8������C�u���!G�/��� 7Fq��'��
�+
���!��s�
/���B�+
5��s~��V?������q<d�$�KH����/spF��'������y����X��7^������9�0�%�>�J���D��x8�Y���*��R�Z�K�<]x��_����8������ym��!��N�E���P�����$�c�ro	���!/	����`S���i�oh�A_�����9@�.4�CT
���t[��Q�y�!�~5�%�������A��d4d��F&y�����6-AeSi���K.�����Q��JEo3���B���$��������n}iI���Dw����,��j�>�E[��7@�3�T��\x>sAV��PZ���nX����7�}�����oT�g�*�g.����)|v��-|v��C��P�S��PS�����L��CME�g.^��@"|������&H��Ct��C�B�3V��5J�j�>;�D)|������F
�
I
o;�a�����)=�b�o��(�
�O��4S���//"=���`�t�/^�!��~^���{�sL�,���'a��}�N>i[����9�n�����L�<��If�����cM�*����*�1������*�x�.�b�Wf���b�����}[�Tl��:����#fDr�7����d�y�WF�u�W��)l`��G�2��CM���(AOa��<(#�:��?�G��
Q��k�����b��T��=��{86�y�ec8����n�h�f���
k�)i���o&a�����-�������C55�BiM�@�
e$ %y.'��V��
"�;�"�NE��d�.{��(��f��po�&A����/|��=Z �`c�u����G���������|�
c�Y���)�{�z���xC�e����mP�&���Rb��B)U(��*�pI��YL������p�eS��*��Dq�7�r�Q��7ru�����'�����.w�j��V�����2U�r}B|�e8��UN�Ab����y���{,s����(z��`p�xxr)XT��K���������>K���V�I`��]��U���Oq�.��Bb1J=���W�w�M8�Ak�I���U�{��������U"���� j�5 Z/��n�m�-��	�SKr�S97�s�u�P�p����K�SK�M<ft[qZl���"����'GRv�^h�\���DQ�J�eO�^�����1	U�$�j
�3���m�;W��������:R�+���j��W%��u���+�`D_�4�}(�	,��(��� �A�/�r����J��c}����\����nD�~����X&���o�������l��#��u~�hi]G�A9��^A�1{��R��N���k2���&�w(|�3�������`go�T�Y%�WU�d`4Zu��I�v�Y�`w,���#�b�23�-K���	9�������TDteF8��nudg��KU��NS��D���K�K���Y��D��K��H^_��D���K�3y�E@K.",M�U�
���K��m��
�bw��3P��y��*��$�����/q�������������_��KCP����/���|�e�j�W�M{xYb��2Gr	��h�"���H��j�����p�h�?����Pv9���u`a~�V�R������������K~88��T����vNm��|��������r�����L�2%�fSg�E�
��(]0���||�T�4u2&�|��Q�'W�E�[V�����P]ih�X&�jU��k(�s���<��`���Y�H��*e�?������u������@����\�W��3�*�Fo����_�o����h�<<�D���H�ow�X��+�BI���/�^"��	�T�p�s�q�����^�����l'��N��`T9���	��=��(�q����(;8��Z���h_6����7����ptzj��8+[�����(>��H�qL��SwZ�oX�n��c�L�}IZ�X6q)����Rd�u���?5�
��zV�������E�+��Q�I����nqu��j:E�W	�,���y�b\��r9�����vVc����rY�s�Z�|�?xJE��\V5U�����xB��k�\VMJt�_��3���>����Q���5s�I��|s%���5���E����(bM�'C���P�%��/���K��I�w���!|������5C.���oWa���Ck3�VL�y:��d�h�K��N������n��
q����\��B�U��.3:����C���Tp����Tp���
�����]�ij�%�A��8��)W\'e�3O�� >����|��4{���y��(�r����MS�����T5aB�f�����*�E�������c'&�E��9w-sNeq���*\������wv��6w�rnY0|/�=���	������y��v����#����|�eT�'����
��)�?�����8Mu;��~~��	xR���m�U7S��;D2��p�l��*"����2�x�y,�b8{��E�$������3�fnX:�v��$
K�YUg�i�c��;�K�/l����faU*����\b����$a����
4��3��b�r;e����;�3�����8:�;���3�q�#^;{}~�����S�^��}0����t�?���`D���f��V?�P��,��V����f�Ec��:S������=~�l����n���A��U_��O�����������)�n�W�x����{�6��d9�C�G���������b����`��uD�A-��q�x�}�f_W�X}�����b�Mo����o�d�j<��?�|�-��L��t}��-��?�v��L�������3�{5��Wy+W�:���.�F��3�Z~�w:�vWQz�e�Z,��f����?W]�1F�#x�
��_M���"]�������d��V���8�R���O���60��la�?�?f?(�z�FW�����<�]O�,�]\et:��v��g��WG�,��,��*���|3�����;8��zG`R����*t��d���H��2p���j�?�kL�n�,Zn�1�9��N�`�?g�aH��{t��h�}�[�������Nz����0�wa���W�o=�
�l�����?������������"P0.��3�GP�C��C=�����
+$����Y��xf,������?�6`D�)��3n�E���h��g��&�t�R ����)��_��jp����\l1��f��M�|�3�"���U�����w������$�`�G���hu9i��y��(�C�~�M"�y���A�Pd�
(�01���b������.���}y���s�tw��u����$�Y,]��������wS(�Hm@)�x����aX��?N�~:�H4~��]~��PWW����2<"����m�A&������������{p�A��o�A�t�?���.EOH��M�]0eG�l9�V��^4|���JS��������0d�W@6�N,�I^����������E�C^�k�`�C��\	�	�f�����"�]|��C�.��X#�3���?P�e
�7����U���(ve�X5��V��W�#3���&��2Q�����gxN`�Xb�Xb����D���b�h1�"O�N-���2�[��&�nS�?Dz�.������1�RP+e[��F�vS�_<9������zX-���v���dl,:Fcs
4
�C�w��2,��!?z������C�"������������f��e�*����P�0��@C�
,����;�^��V�����d�ot��r�-TDV�w���p���l#JW��l�.�A�@��Qt��K��������>*�)�
h$7bt�&�<F$7��Jhn<7M#Z��Q?,Az�+�0�4zn�/sOC�[��"�*X��q
�9�6B���r3G)�����q(,@�l����~�emGu���t��aq��jA����W�7�a��0P�a���8mY�9z��X��m�Z�*hr��)��,�������`&'��Uz$��*��nf
��2�"��7�k����E�����75{���t�-�Q.1���y����N���GaI�����h�J{�����6>�dIG�tA��U1�����.L��y�h�L�q�����
������-E����7|���(�n(�%2E�D���	@	e�BFc��j05��Z�f����b5�)[*��P�����h����H���+X-V�Z�CV�5�V�S6&��z��T���5���L�m�c��i�#'�D`�s��	Z�m�g�h�D����k�-�%�j
4��2����^N��d�������}Sv%�
�i�f��+K��9���'��3L�K}�5��#�`���;i���^L���;��?������h?����^�qa>!~�9\����W�������o�1q62�_db���[�D,`e<���[JU�T�H��xjm8rm�W[���I�M�"������T@T��y��(cR�����W�\&P�4;������6�c��D���1�2�\��6&6��p15�9�2t��I�=�	�I�<'��7k� �f��g��V�Z��������GQy�Z�l
��b��'���0��S������������9���K������3�_�Z�s1�\�.a�K4�1�/\y���}.�?���%��pD�(�9���gy*F�o��[��[��V����T��B^CQ�}��`����}���
������(i�#��V�+Xe����6�H0	�h�F�6�pw��HK�QU��
YR�������n[����N!����[!�p�Y2�v�^�+��/����,�'��!1����5�t�����{Qi���:C�V�6�����^��T�Ug**Z���9d�H��x��YT���C��z�.{�����[���d|�0��0{�V��x��^t�D&{��NTrb\�c5'*[���X���V�91������[���"sb�CN�5�N��"��3_�%�����L$:�t���q�����!��� ~�>\�nT�rq��g���\�����s6eO:�O3y������������������D��f��:`������7`��'������!u��F�i�O���1��B����j�=&�����	��'�v[Po�?@)���7��fR��L
[����Hy��)O���x��:�����\�8�C���)������u��@\�N��fk���8���2�q.;�:�� ��Hl����=3u������ml,ON�(t���j�L��Su�TW����X*��h�v[0��+�v��z[9�0+:�{����2,3M��L����$Re���?7#!MX�9��GO`��!d��1C1�	��8B|�r���oD��;b�����M��&Gc�]<�rG�$/�L������k�
��M�z2l=��� ��1�����.����$����W=w��f��a\[���E��<�V�����sV��"�0���Xs����9���Q�����vK>fE��������W��,��T�b�f��J������U	��W"�����������p��l��uS�gB��D�y�0LH�&bn7�����V%\�M�ll����l<���Hj�#��sVd
�/tc�kp���	�M}�s��<�I��/��<����O�G��9[B�4������!���}�\|��Qv{����C_E}�K�XT��u'�����L�]���/�b����9w�Tx�Q+�p��|����`����x�r��U���Q+������*�oV���J��
EP�;�TA�
�;��Q�����S/����T�����N�R�&n�,jW�����XR{���%�X
P� M�}a`�<��m9�V�v"��]v"v����x/Q2���H����5TK	QN�1��x+[J���G����l��H�A��V3�q0��(sP&��q�u��5�"��>_��q�ww�ivM�\�?-��M��������js�L
q
�5��E�'4����^{1_D��0G���jJ������H��~����c��*�!���K�4T���:���3Xt��
��c����(EG�zX�X�1�
|��42z�����pq}�X^eK��4mi�X������V;aa�2|��Kb��r~A�������`��H�i(��&~��#���bVs)B��3�7�
:���e�����l��Bt��a<��w
���o]7���(��/�%�����3�TE�r6S�{3��gj5O�w���H���.4���K����NC��*���;8��Te���JI8ALIeN�BK�����0�����F��@�L��P�g�P�C������������n�� ����3���q����*���}���������G�]>�Z�G�]>���,�z�P��|�P.��v��K��xhjQCU��!�����U��L��0��G�^>b��+��X�|��]>b����44����x��Y!����.x������{��a�8�'�^{5Q���Fh:���!��@�4�����
�������v��Q��<�Vi�`D��	5�T��>c��g��V]&-}��7������8b��P�x���ig��YVWD��Tho�_)��eY}�Qn���U�Q�0��XQ�!!�c�f���Z��l�lvI]1Id���
=
��QX��[����q���R�a���D9<|������8+��l:A�����������{�|������U�EU��ge�X]�!�Q�!VWv���Z�DY�A��[Q���I�����C������C�����~T�L�Of��\TG����6�QB�i���[���eja����J�*%�g%��`�)������#������,q�+�2�6j��
�\�1��35GB����<�-�o��S%�-q�P&ISS����^g�N{��+y�����U�HuFz�c�c(��S�����9�S�,���%v+|�&��������Y�ny�v�]p@��YJ
�(�{����=��/����h��������[!n��3��|�VYt������+�D6YgW����������LqA��������^e�8��[p�f�
�J���c[�b���m�W�vr�} �'��iK#��2g��3|����Q��������<�n��s���rq���Y;���u����&���6������`��4��>d�\4�V`Wk��m��
5�uDT�4+�Z�.�hxZ$-�i�*�����%j�x1��:����a"��2�v��u-�M#���f�6KZ��:�6+���FiB��=�AP�D��is��J�rM�.�vB�a@�l6��-�	�`�*�{wc���
��a�����
�\�-q���+���j�A^"���j"+�����!�WR��\�,�y��_�s����
�~���q�BA�0��Bq�Q^�YN����WS�>Y���E�����/���r�����$���
<��[��,��+�^�Kz���w����wx���h
p`�<���$��'�	W�y�LKE�G�l8�l�L���d~5�L&�V9������C���"�2r��R��xwkDO�0Z��<���~�f
J����#�%������2�x^�������o��b���]��M�E���n?^�mT]�Z�������������E�3��K�L�W�}����<o����������h���/\�?���}.�?���%��b��X�o������~c}���vB�U��-������%�{��B<�V�W+/*�W�4s��f9Yl��!�p���9F���g�������K��9���bqW�N�'mZ�N��C��6������
��~q�g������������}���� 8�t���S����k�I]L��ez���������m3l �C�/:�@���]�w�KU���pq�pZ��S�x�h�/�S��M��~������O��U����wR�$����wK0�(��[�V%W��+R�E�~��_����zh��R�#�N��Uk�u�!U�s�t(�u���>Rv��p��e;�t{���/�Y��c����[
�|0K�����	"F�U9�H�s?���D�@$%���S�=��b���z�}x�B�����"7btCAz'��/i��7%��x���V�h�S���NQGG���HIB{*��
������
��hwa���67��8wJ��O
$���+bW�S����f?8��(�9�j�6bmq��m�(�h2��t���`S����yC�GK��D�Yb�G�&'��b����D5�����Vu�Ofdv?�m=�X�}�ne���]���E�R�7��.u�����U-��+f�06��]+%�W�����w0�������d�n�v&9���vM�p=�|�5���<���5Y a�Y��la�~���h��������zz8��d�h6��&~��|�3�[:��K��G-�N���N���N��m��b-w�z�t
o�5�(��o�|�6zO�����nT��T����Z�PE����p��|r��*���UT������q�Z%fL-j�l�x�9$Y������=��s�A��
E�?�V��R������&�������������N`�d��"�����u5nn]���j\q]�����E����U
U|���v]�+���v]��h^Wc������u5���q�u5������j�e]M�u5��X����u5V���b]���j�^Wc�����x��j�YWc����M�d�zF2&mI.�HU����d@I���=�d@�L|U%�;�a ��)��o�^S,[������T��
�7T�?b���r�Sc�<T��<�@��n����v9]�@���V������M'���0�_h�������bZr�s��I�E����WR]�JZ�0�H��g��Da�$M�$��Dm(���r(%y���x�%Q��[��0t��4va}�!y<���hF �o�;v���[EQ������$MjG�������}�$j6k�%��"m7q7{E�F�w[��h<�w��zx���%����]�+����=�������u�S�oK����d�{�q+_�@�o�t2%���j��O$H��#�o>g9���X�N���0'�)����W79X�jp{N�^�����a������;���km����euX�����������C��!���;�8�k���>;�
|E�VP�q`��� Tny �^�����WWp�`4�1���*H.|��c]Og�:iz��;Oqv�y{GR^�p���}������A�m���Pj5x�I]B
�����
��������Jl���7�mA8[�#�������
 
���7?����D���e�3��``~b���&|u�������r�n���rn��
�c��J�������'Ib~d���87�
�Bg%����]z��#�.����l��M���^����e�#���%���*�e��������e�k|^=�������k?������{��1,(�?�!���\E��:�(���D��/w�M[S�8������S4����;�h)�=y���������2������'����O�����hUNR�{l�-O+`�'��P�hp�4�D#�c�m{��i�_��;u�c���k�Dr�6h�;�Fo�1Z,��%��H�F	74�y���ij�5_C��f�J��F����`I��>�5d��O���(z�q+�CjKo��(�c����Q���� R�t��2����h�d�H���0�r���A<����t������[�y�44Fo��y/��?���OvEa�����2-J�J�&,�%�F��$�a�q>_ae.�io�
���kx�L�M��4E�M��F���l�;�����EH/�����YJ�T/��WY�;��d�n����pay���������6��\���=����%�#!�r��~�������n��6��	�|�-�����m��j=x6a��}��dQb�qF�
:�Mu$7����Q?��f�5.6k���&�P���N��{M��id%����0 ��tTQ
y�}&f:L�*|k&�NG����HM
QS��4��p^��_�a��"�w��c��y^4�����TF|#c�qE����uv��f�5�l�32���Ig(�%��&�eV���f��.��t��-h��.��Z�zx��2�6z��O���O���cn{J�	;�p�L�3��=���E���SR�Y������$;O������0EV�)&�8�c���r�a���c~�M�eF����M9��jxs���������G�E��b�~P%-v�
lV�g�F��F��e����34m�?"^���P��^����� +���j^�����m���/��+����� i�y)=������#�+��C.en����D*������Z��D�\�V�M0�Q���v�9����#�:�r{���:-��O��8A,�2�n��C���7	�P����$��4
� Hx�������@���n8"�Oj���p+T��C����Q�@K�A��Fz�?m������))�'������r���4��SO��<E�MUH�$Q��f�������	���c�d���,�[�Fc�]�2bkK_e���B�����hCm~l�����5`��1h���C]��c���/�H����Z���l�CO�h�3���<�
���.��F=w���x�����~@E�q��Z�<������[�J3�x�*�L�������2L��J���Y���25
��*Ln OR��z�2�|��KDU�VQh��p�{e����2���>��G����������q���=%��K���7�<
�C�����2�T4�8.=x_lh��1�W�1|.D-��A����nsN�T�iMlr6�Xq{�6�5
������X���mF��v�m^k�]�Z=���1kgS�����+\k`m��T���:�X����l1�5d+X�I�l��X�n���h����N�k���p��rWQ�B"�;�
���`
��:��-�#�:��/�c/e���W�J��ze�K0�t�c�����&9�N�f��b�����1��xp?6��M�o��k������r�m����=�s�m��n�j=���L����w�O������=��l��.�	�����=�������� �)��z��w��8&�������x)�a p��?���Sl�
V��2p<U$t����Qe�H����l�0I���$�*���G�������H@��x�WYp��*�����q�_l��Y.6w�w�����EE ���WT	3=�X��y<�����.���0�6�Rs{Q�P�'M �v���o@�X�k�]�y����o��!��������lW�5����JvG�am/�,������%)F�"���O���pv��Ih����Q\��^����
����+�siT����	����qp�c����Y���O��\��Q��k�0��;��,WP�����7���={W���It���#x���V�0��~>�<�5kF�
W���E0��������3TT�2��^������������+�T~�]�S�iD.���<rq�3A��W�p6��%!H��������5I�'F(�����)���.z{���O����U�nV�^#I���#S�`�4x����7�p�/>�� �8`�/���z
��d�dqt���s��Gp�U��F�
|��D�Mp�����3�����w�P}cI�A��s>FMi9<�4h#*�3�a)hm�L�@?��g����@��r+^a��}��O�G9Z��f=9_��B�l��L�Q��p)��M�~�#����(��;d����s�y[�A���?��)�V��=�a�	�jI�i�d_o�����c���N���^t�S�����]�0PN
��Nc�����.R���z�
2_b@%���������&5p�~a�1O)cu��%+�!�������0~�����_�@I`C}���pS4>���G�Z��\7�	�x{!	�g�{z�*`$�D4��{�{�����g�a�#��Xx,�@v��Y�d
L�����h*��X|S�Q�(p��G���B���r+?H��
[uCA�: =��X�j���_��|����. �2��3:U�:/�`���2����v������2����)M����|�H��v�j���o��
��j�Ye{���y�F���)�A��������&������;�?6m`���j�d�t����G��Ek��E���v��m�A�U���Z]��Qu���{��������:�
�6Q\C�����C��M�Gk�ru9 ��C�0A���39s�������7��l$����B��
�}��P�������}i6��;6�aY�~���j];>�L�#�s�vW�dBb���I�9A�RRw���(�;w��PP����n�&�/q�`����/{�
wT
��0V7��Q�i�<>xH��6��K�Prc�H5[3�4ZU&�K���*�����*FN�R5k�L9%�S��t\c��d���T���p�b���>�&v�-�k�6�\v����:/�����S��q:�j0���L��9)�����X FW�v3[Og��lv�^����rg��/�G�,�kx#V��P��xh��1����u��y�������������p���4*^K�78��
�&�f��43:<Y��2���M2	�Mm wTZ����5��jY�G�6b�rB�,�����U��S�.'5<V����
wL���@�F@y|3�+�
�B����x�t[������!���&��}�H���������!��$�Vn��8��)db���Z��p#{m�)�Z(	�R��@���`���f7�������T�,��K����4l�(��N�����J���j�`��tq���A���kndK]�
���=q��
�js�g���P���l����;��g�C��w��X�t�GB�y�RUu�5��nNTV���6�R
�kr5s�U��_;��C��������[���@�{�Di�����L��Z"0j;�	�v���y1;nQ���D<`�v0v�p��V<��j�#y����������+!�i��\>n'��#�k�f|�	��S~W�n�9����
N����3��t��	������Da����}�j�
�����@���16S�]�dzL�-W�.7UrS��H�8Ph��+)�8�����A�������c��Q%do��^mT=i�G�=����n�0i!=��Q�,�Z����[B���|w�\�vZ����^R���0�}��[�L����y�ho���V�V3��!�������TY�k����nu�=k�k�>�����+�n��������e���\w)(�'�>l���[���#C�:��F�s]�4�/u�wE��_S�xOn�4I+��f��|d5�������*.�;��X>��m#k��\��sJ�e��w=�u�?[���MWkm�Y����9��z�3S���4�C�b> l��jPfue�^N��OzON��������Y	�Tw��Y�9yq
�@����go'��o�o���������p����Z<A�D�.`Dh��,�;+_�d�*#�.kMFu�y���:P�CW���T��q��d��I���Q
�������m$)��1���Z2D�p5x�z�n���W�V+>4�z�4�	5�]�_�{,R�X*[���P)&l/��XUy,����w+��_\3��"��@���6Uo
j�����59A��>8����+�%F�h��e��G�[>����su
zx�GN��t��@���D�n=����T�|���]��'�FI�M{��T��Jv��?�g�-*������
��=h�=����v���G*����8Ka���=N�w0�v�K�9��T��c2�O`o��H������#����Q����.������zSk�n��3�o9����V���Z�5�������4��Z$�3
����t�������=���Y[v��[\��9v+p?��eI�pV�v��U^X�����H�=�sA�i�Q�,A�qXpg�>=�R�{3��a�)�=[Z�8@��&C�e%q�+
w�@XLvw�{8P\�e�p��EsceG?1`�7��|�8m��5���b�&=���"���O������j�_U-���B���`;[@�x����&$��m���NN�A�B�{��p��r��N����
;��U�=����_�������+S��y�R�C��xj�4o.�~N�����)��<�Q�A�[4\N (;T��`��a��d�S������W�0
�&�u<��ii�b6'W�?.�U2���6�p������(�����nOh�^�~8gk�������� �oRz�F�9�������}A���&[E�|�����&W��f	�
�g<��}<��v������F�������D�9��b�� �y���?��� �!��2C����'����n�;�����_�5tc���<����`.90)���a�
z����.�k_�H���������l�$B�;��9 3���x�@w�h�����l�a������F�c� �ll����H����"&��`lx��z�x�ad��e�MO?�f����v{��=9;cF����lk�{0����]1��c������
��y
QMM�f�8P�����m��j=o��+Pk������K�f�:�."%Dnn#�����/���y/D(�D�t��	P��Ls�.���9�t�Rh+���^7aQx{��%^��	}q����/�Ku�2�����/\i�%_fY�Z��p��#�4�����D���DV����I����b����\�������5��9�q0��W��s��o�����bq�+�my��r���������8a��	�?�B�x�����X��18X]��B��1 ��t4��D���>�a�`(6��xx�/8��UL?
�4��(?O�'��O�S��q�� ><DS����#H��3�q�t��}��E������v����a�����x���������:&���GGG����?7�5r�.�Y
�����p�����t2���������}��?C-�}�����o�dx���}�I|���I�xx||���Qr4<���hX��(�_`��+��*�}O�����^
����� yq�<�����N���a�<����*9��'q����������b0�����)�Q�S
X!�7J�gc���3h��C"���dq���k�W?F����D?gc�h��c����q��h�u>n�������v
m����+�����������!��(��u:��(�qo����j���q�tC�]��oU������t�A�^���)x�K��.6+dA@�O>Cl����A��g��S\�6V��?6S��\mno��z��~��`�\������9�|Ig�|���������l����-����Y�!�������6�>��M�L�k/��d�hz
��:��}���WS@��
����!D�n����E��-��<'��+`�*�zL���V��)t/N���i6������L�t�z��Y�������m�����L���?�$��9N���6U�<�~����0�#��=R�N�OV�a?��������+�ct�LzuE~%<������Y����z��V=B88f������y��H`NI:���z��
2vTd\m�$��N�;"��O����)>�I4�:ygT���{��6.�B!�6��z3C����7�K���(�a�����
�sO��p�x;���zzw����?Ov�HtB�s <�8j@s�c��0�`��g4�j�Z,�#��#�8^��Y�	��2�q����3x�[x
�`�X��H'���d=�A ��^g�A]����s���7�4#��
L;�:�(�@���W�5�l��cl�N�<1�fy�����������U�<
4�`8
%0\�3&8�f��e�f�.�i�n���*�@h��/8�c�d���-�W���@����}q9
�:[.Q�dvPh?s�j���7c86w�{LL
#��5�`� Z�>�C�O=��
�g����G��nG��=sm�3@Rj:�j����)��-JW��:p E��p���7�W���w
d�-�y�����2���'Zz`�E��|�V��E%uD��c��Y��b��+��~ �\m&��=]4�8�aG;�����*6M4v�����5��U��JB,��)�J@3�/2VCv��A����B�����%t������E��Di#�_�����t�&�(E����{�������-��77�0}����3�p��d����k�V��8���!["=BM�
L��`7:��/�R�$l���8����E�J�@6�sbNw��Y��
�P��;�����(�������u��������di�B�PH=�"6-�=�����-�������b�Cs�7hc�~�����3���7���:p�0�f�����V�vCI�%��7��3�?�,��k
4H��9�bJ�^��&4�4�G����� 0�
��%m^OVTf��f��@��)D�Nf9�&�29�s��}�Ci(CM�N�uP�`���9��E�� �F}
hRP�ZRn�0|�l�����4������f�o�0�h��tB�!�z�,��X�#�v3[O�f�"
�-���#`����q�p�L��GWQ�i1��
�"�a�9V��fwO��\��Ev8���?�����"����v���^���3:���~���������"���+�����N�ZN~��������n
4���8���D�W0L%���<z*^���m����A����#��N*�2:/�,>|7�l�8��,��P4�L�nt:�K��=.�����s0b�}4����4{���<��Y�?:: 	m{��{���_Oz�~�������b�9F+��j�)p'��:�_����D���w������o1D@���7F��	o����!���h��H+X#���r
�*��`rm����N���B���+T6��	��1X��jt* ~B�q��(e`e�+>&����p�!���e�L�B�����/�2�e����`�Fz��Y5�����t��n��]������~���`�o��E��JG�`L!�F���i	n��a1 ����������q6���HN�56y��O?��e�6���� 
Cs�W��������8�����o�!�|���E��u���-v1�l�n�<�T=��-�E{
��f�����W�y�N�z���^�q+����~����DO�#�1�g���xU'��y4�~]��>yvE������}��d�x���>�uz�t=�fX�RP�%jk2��H}�����3b� ���1��������r���r:�I6J�����V��"D(�����Y��~/�W�"��8f� O�J��m�Gm!�[I�,���d�gSD�)k"�3���jq$O�3Sl����68 �xX ��.��h�_�5�MG����`I��9?�#�	� ���b���z�
)�H0���#��T3$�5�Y�5L���f�!������k4���R���Y�/3`��
�xI��e���X��<nC��L0�"����1��OH��q���m�r@��u�6����Q����^>M!��SN��U��o�����L�m4_�6Si%��n��m����fj�F#��Odj��~���������}��t�A#
���S�`���Q�;(So�`�:!��G������i���z�_ ���&{I��CF�:;s��WH�����g�,`f�p�Dd%�1dE�<����<������C/{��������~�io?������w�n�.R�7$53�@T&DZy�6�?����r�a�r��nj��K��%����C_r�����M.�<~Xs+?���V�������7H��I���B�^|E����y�`�o20`)n�@�<be���#Z;��$�O�����q�����8K���R[F��4�4��Q�$z
�<G�:��y���2�����e��"�b�.�m�������{hCc6�&���z�u��}6��X1.�br>�<�=x�������
2�����Y���d������cdj�
M�Y����*_��S���
�����l1{su_���C���0�c
�&�/���&�2������o`�����L�������a#��������~}|w����������L` ���t��?��CY��^�����xy����	zxW����&�{��.&�A�"zDrr��9������I��!G�A�Sx�R����nM�  8p�|�� @�"|����{�p��C��A*�5,�h#BU�Q��r������7�����Uv��������U�8��{�����'���#����4~�1d	�+����Y<_��L-�4x���HK����zv#�@F)�"��iA{YXvJ�IO�h&��,C�B%������,��V�gG�f(!��=�������<4�� ��f�|X�Nk�a!��
�qX�G�aa��8,��a�!��0�{/����K{\���[���������P�L(�1���S�([O�����6�(�X#���?�p+��\/��"�9$~(M
��,��a�z��`>�R>Xb�`���
��\���@-�RP1a �o��P�c0�u��\�0�F����
,0�"��:Qs����0�t,� =�
��Z��a��t]��i��D�B@��.PK���@b��
��"�2`��pA�bp�$���<~�G�1�j��#j���Xk��Ob%�f-f|�L��z�=[<�}�m���4@�6���?�r����������e��3�]A])��gH� �R��c���a
�d����A���� ��������K�u�Y^Tb?��/y��@��b���v4B�����/�������l���,?�$��N�^)F<��h�X�����L�OGC����u��
����B	C��Z���������� ��0',g��|�H���.��*���H��5����yh�@4p�>�p�1�;�\��/*
^��H��N�'puE�����M�����`C^o���kt���!�*�+{�����
��Y6(�)n��3m��aVy�1�4D��������
y�iE\�t��J��5�F����U��x�$���$A���A����x`�(:[��a$�7��bC���y4�h�$�K4U���LP�P,��\!q��J�<�/���n]��U&��W������p�+\�
W���p�+\�
W���p�+\�
W���p�+\�
W���p��q^��@��
#285Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Ashutosh Bapat (#284)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Oct 6, 2017 at 5:17 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

On Fri, Oct 6, 2017 at 5:09 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Here's updated set of patches, rebased on top of the latest head.

In this patchset reparameterize_pathlist_by_child() ignores NULL
return from reparameterize_path_by_child(). Fixed that in the attached
patchset.

Sorry. I sent a wrong file. Here's the real v37.

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

Attachments:

pg_dp_join_patches_v37.tar.gzapplication/x-gzip; name=pg_dp_join_patches_v37.tar.gzDownload
#286Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#285)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Oct 6, 2017 at 8:40 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Sorry. I sent a wrong file. Here's the real v37.

Committed 0001-0006. I made some assorted comment and formatting
changes and two small substantive changes:

- In try_nestloop_path, bms_free(outerrelids) before returning if we
can't reparameterize.

- Moved the call to try_partition_wise_join inside
populate_joinrel_with_paths, instead of always calling it just after
that function is called.

I think this is very good work and I'm excited about the feature. Now
I'll wait to see whether the buildfarm, or Tom, yell at me for
whatever problems this may still have...

--
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

#287Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#286)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Oct 6, 2017 at 8:45 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Fri, Oct 6, 2017 at 8:40 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Sorry. I sent a wrong file. Here's the real v37.

Committed 0001-0006. I made some assorted comment and formatting
changes and two small substantive changes:

- In try_nestloop_path, bms_free(outerrelids) before returning if we
can't reparameterize.

Hmm. I missed that.

- Moved the call to try_partition_wise_join inside
populate_joinrel_with_paths, instead of always calling it just after
that function is called.

This looks good too.

I think this is very good work and I'm excited about the feature.

Thanks a lot Robert for detailed review and guidance. Thanks a lot
Rafia for benchmarking the feature with TPCH and esp. very large scale
database and also for testing and reported some real issues. Thanks
Rajkumar for testing it with an exhaustive testset. Thanks Amit
Langote, Thomas Munro, Dilip Kumar, Antonin Houska, Alvaro Herrera and
Amit Khandekar for their review comments and suggestions. Thanks
Jeevan Chalke, who used the patchset to implement partition-wise
aggregates and provided some insights offlist. Sorry if I have missed
anybody.

As Robert says in the commit message, there's more to do but now that
we have basic feature, improving it incrementally becomes a lot
easier.

--
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

#288Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#286)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Oct 6, 2017 at 8:45 PM, Robert Haas <robertmhaas@gmail.com> wrote:

I think this is very good work and I'm excited about the feature. Now
I'll wait to see whether the buildfarm, or Tom, yell at me for
whatever problems this may still have...

Buildfarm animal prion turned red. Before going into that failure,
good news is that the other animals are green. So the plans are
stable.

prion runs the regression with -DRELCACHE_FORCE_RELEASE, which
destroys a relcache entry as soon as its reference count drops down to
0. This destroys everything that's there in corresponding relcache
entry including partition key information and partition descriptor
information. find_partition_scheme() and set_relation_partition_info()
both assume that the relcache information will survive as long as the
relation lock is held. They do not copy the relevant partitioning
information but just copy the pointers. That assumption is wrong.
Because of -DRELCACHE_FORCE_RELEASE, as soon as refcount drops to
zero, the data in partition scheme and partition bounds goes invalid
and various checks to see if partition wise join is possible fail.
That causes partition_join test to fail on prion. But I think, the bug
could cause crash as well.

The fix is to copy the relevant partitioning information from relcache
into PartitionSchemeData and RelOptInfo. Here's a quick patch with
that fix.

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

Attachments:

pwj_copy_partinfo.patchtext/x-patch; charset=US-ASCII; name=pwj_copy_partinfo.patchDownload
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 3a8306a..ebda85e 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -702,6 +702,74 @@ partition_bounds_equal(int partnatts, int16 *parttyplen, bool *parttypbyval,
 }
 
 /*
+ * Return a copy of given PartitionBoundInfo structure. The data types of bounds
+ * are described by given partition key specificiation.
+ */
+extern PartitionBoundInfo
+partition_bounds_copy(PartitionBoundInfo src,
+					  PartitionKey key)
+{
+	PartitionBoundInfo	dest;
+	int		i;
+	int		ndatums;
+	int		partnatts;
+	int		num_indexes;
+
+	dest = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData));
+
+	dest->strategy = src->strategy;
+	ndatums = dest->ndatums = src->ndatums;
+	partnatts = key->partnatts;
+
+	/* Range partitioned table has an extra index. */
+	num_indexes = key->strategy == PARTITION_STRATEGY_RANGE ? ndatums + 1 : ndatums;
+
+	/* List partitioned tables have only a single partition key. */
+	Assert(key->strategy != PARTITION_STRATEGY_LIST || partnatts == 1);
+
+	dest->datums = (Datum **) palloc(sizeof(Datum *) * ndatums);
+
+	if (src->kind != NULL)
+	{
+		dest->kind = (PartitionRangeDatumKind **) palloc(ndatums *
+												sizeof(PartitionRangeDatumKind *));
+		for (i = 0; i < ndatums; i++)
+		{
+			dest->kind[i] = (PartitionRangeDatumKind *) palloc(partnatts *
+												sizeof(PartitionRangeDatumKind));
+
+			memcpy(dest->kind[i], src->kind[i],
+				   sizeof(PartitionRangeDatumKind) * key->partnatts);
+		}
+	}
+	else
+		dest->kind = NULL;
+
+	for (i = 0; i < ndatums; i++)
+	{
+		int		j;
+		dest->datums[i] = (Datum *) palloc(sizeof(Datum) * partnatts);
+
+		for (j = 0; j < partnatts; j++)
+		{
+			if (dest->kind == NULL ||
+				dest->kind[i][j] == PARTITION_RANGE_DATUM_VALUE)
+				dest->datums[i][j] = datumCopy(src->datums[i][j],
+											   key->parttypbyval[j],
+											   key->parttyplen[j]);
+		}
+	}
+
+	dest->indexes = (int *) palloc(sizeof(int) * num_indexes);
+	memcpy(dest->indexes, src->indexes, sizeof(int) * num_indexes);
+
+	dest->null_index = src->null_index;
+	dest->default_index = src->default_index;
+
+	return dest;
+}
+
+/*
  * check_new_partition_bound
  *
  * Checks if the new partition's bound overlaps any of the existing partitions
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index 93cc757..9d35a41 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -1825,13 +1825,15 @@ set_relation_partition_info(PlannerInfo *root, RelOptInfo *rel,
 							Relation relation)
 {
 	PartitionDesc partdesc;
+	PartitionKey  partkey;
 
 	Assert(relation->rd_rel->relkind == RELKIND_PARTITIONED_TABLE);
 
 	partdesc = RelationGetPartitionDesc(relation);
+	partkey = RelationGetPartitionKey(relation);
 	rel->part_scheme = find_partition_scheme(root, relation);
 	Assert(partdesc != NULL && rel->part_scheme != NULL);
-	rel->boundinfo = partdesc->boundinfo;
+	rel->boundinfo = partition_bounds_copy(partdesc->boundinfo, partkey);
 	rel->nparts = partdesc->nparts;
 	set_baserel_partition_key_exprs(relation, rel);
 }
@@ -1888,18 +1890,33 @@ find_partition_scheme(PlannerInfo *root, Relation relation)
 
 	/*
 	 * Did not find matching partition scheme. Create one copying relevant
-	 * information from the relcache. Instead of copying whole arrays, copy
-	 * the pointers in relcache. It's safe to do so since
-	 * RelationClearRelation() wouldn't change it while planner is using it.
+	 * information from the relcache. We need to copy the contents of the array
+	 * since the relcache entry may not survive after we have closed the
+	 * relation.
 	 */
 	part_scheme = (PartitionScheme) palloc0(sizeof(PartitionSchemeData));
 	part_scheme->strategy = partkey->strategy;
 	part_scheme->partnatts = partkey->partnatts;
-	part_scheme->partopfamily = partkey->partopfamily;
-	part_scheme->partopcintype = partkey->partopcintype;
-	part_scheme->parttypcoll = partkey->parttypcoll;
-	part_scheme->parttyplen = partkey->parttyplen;
-	part_scheme->parttypbyval = partkey->parttypbyval;
+
+	part_scheme->partopfamily = (Oid *) palloc(sizeof(Oid) * partnatts);
+	memcpy(part_scheme->partopfamily, partkey->partopfamily,
+		   sizeof(Oid) * partnatts);
+
+	part_scheme->partopcintype = (Oid *) palloc(sizeof(Oid) * partnatts);
+	memcpy(part_scheme->partopcintype, partkey->partopcintype,
+		   sizeof(Oid) * partnatts);
+
+	part_scheme->parttypcoll = (Oid *) palloc(sizeof(Oid) * partnatts);
+	memcpy(part_scheme->parttypcoll, partkey->parttypcoll,
+		   sizeof(Oid) * partnatts);
+
+	part_scheme->parttyplen = (int16 *) palloc(sizeof(int16) * partnatts);
+	memcpy(part_scheme->parttyplen, partkey->parttyplen,
+		   sizeof(int16) * partnatts);
+
+	part_scheme->parttypbyval = (bool *) palloc(sizeof(bool) * partnatts);
+	memcpy(part_scheme->parttypbyval, partkey->parttypbyval,
+		   sizeof(bool) * partnatts);
 
 	/* Add the partitioning scheme to PlannerInfo. */
 	root->part_schemes = lappend(root->part_schemes, part_scheme);
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 454a940..945ac02 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -74,6 +74,8 @@ extern void RelationBuildPartitionDesc(Relation relation);
 extern bool partition_bounds_equal(int partnatts, int16 *parttyplen,
 					   bool *parttypbyval, PartitionBoundInfo b1,
 					   PartitionBoundInfo b2);
+extern PartitionBoundInfo partition_bounds_copy(PartitionBoundInfo src,
+					  PartitionKey key);
 
 extern void check_new_partition_bound(char *relname, Relation parent,
 						  PartitionBoundSpec *spec);
#289Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#288)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Fri, Oct 6, 2017 at 3:07 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

On Fri, Oct 6, 2017 at 8:45 PM, Robert Haas <robertmhaas@gmail.com> wrote:

I think this is very good work and I'm excited about the feature. Now
I'll wait to see whether the buildfarm, or Tom, yell at me for
whatever problems this may still have...

Buildfarm animal prion turned red. Before going into that failure,
good news is that the other animals are green. So the plans are
stable.

prion runs the regression with -DRELCACHE_FORCE_RELEASE, which
destroys a relcache entry as soon as its reference count drops down to
0. This destroys everything that's there in corresponding relcache
entry including partition key information and partition descriptor
information. find_partition_scheme() and set_relation_partition_info()
both assume that the relcache information will survive as long as the
relation lock is held. They do not copy the relevant partitioning
information but just copy the pointers. That assumption is wrong.
Because of -DRELCACHE_FORCE_RELEASE, as soon as refcount drops to
zero, the data in partition scheme and partition bounds goes invalid
and various checks to see if partition wise join is possible fail.
That causes partition_join test to fail on prion. But I think, the bug
could cause crash as well.

The fix is to copy the relevant partitioning information from relcache
into PartitionSchemeData and RelOptInfo. Here's a quick patch with
that fix.

Committed. I hope that makes things less red rather than more,
because I'm going to be AFK for a few hours anyway.

--
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

#290Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#289)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Sat, Oct 7, 2017 at 1:04 AM, Robert Haas <robertmhaas@gmail.com> wrote:

Committed. I hope that makes things less red rather than more,
because I'm going to be AFK for a few hours anyway.

Here's the last patch, dealing with the dummy relations, rebased. With
this fix every join order of a partitioned join can be considered
partitioned. (This wasn't the case earlier when dummy relation was
involved.). So, we can allocate the child-join RelOptInfo array in
build_joinrel_partition_info(), instead of waiting for an appropriate
pair to arrive in try_partition_wise_join().
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company

Attachments:

0001-Support-partition-wise-join-for-dummy-partitioned-re.patchtext/x-patch; charset=US-ASCII; name=0001-Support-partition-wise-join-for-dummy-partitioned-re.patchDownload
From 4bf8ca38719aee730ed57a7f14522384b1ced7b0 Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Tue, 29 Aug 2017 12:37:14 +0530
Subject: [PATCH] Support partition-wise join for dummy partitioned relation.

Current partition-wise join implementation treats dummy relations as
unpartitioned since the child relations may not be marked dummy and may not
have their pathlists set (See set_rel_size() and set_rel_pathlist()). Since
children do not have paths set, they can not be used to form a child-join. This
patch marks the children of dummy partitioned relations as dummy, thus allowing
partition-wise join when one of the joining relations is dummy.

If the dummy partitioned relation is a base relation, its children are base
relations as well and we will be marking base relation dummy during join
planning. But this shouldn't be a problem since populate_joinrel_with_paths()
already does that to an inner relation of left join.

Ashutosh Bapat.
---
 src/backend/optimizer/path/allpaths.c |    7 +------
 src/backend/optimizer/path/joinrels.c |   24 ++++++++++++------------
 src/backend/optimizer/util/relnode.c  |    5 +++++
 3 files changed, 18 insertions(+), 18 deletions(-)

diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 5535b63..b46fb5b 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -3261,12 +3261,7 @@ generate_partition_wise_join_paths(PlannerInfo *root, RelOptInfo *rel)
 	if (IS_DUMMY_REL(rel))
 		return;
 
-	/*
-	 * Nothing to do if the relation is not partitioned. An outer join
-	 * relation which had empty inner relation in every pair will have rest of
-	 * the partitioning properties set except the child-join RelOptInfos. See
-	 * try_partition_wise_join() for more explanation.
-	 */
+	/* Nothing to do if the relation is not partitioned. */
 	if (rel->nparts <= 0 || rel->part_rels == NULL)
 		return;
 
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 2b868c5..1578dea 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1321,14 +1321,19 @@ try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 
 	/*
 	 * set_rel_pathlist() may not create paths in children of an empty
-	 * partitioned table and so we can not add paths to child-joins. So, deem
-	 * such a join as unpartitioned. When a partitioned relation is deemed
-	 * empty because all its children are empty, dummy path will be set in
-	 * each of the children.  In such a case we could still consider the join
-	 * as partitioned, but it might not help much.
+	 * partitioned table. Mark such children as dummy so that we can add paths
+	 * to child-joins.
 	 */
-	if (IS_DUMMY_REL(rel1) || IS_DUMMY_REL(rel2))
-		return;
+	if (IS_DUMMY_REL(rel1))
+	{
+		for (cnt_parts = 0; cnt_parts < rel1->nparts; cnt_parts++)
+			mark_dummy_rel(rel1->part_rels[cnt_parts]);
+	}
+	if (IS_DUMMY_REL(rel2))
+	{
+		for (cnt_parts = 0; cnt_parts < rel2->nparts; cnt_parts++)
+			mark_dummy_rel(rel2->part_rels[cnt_parts]);
+	}
 
 	/*
 	 * Since this join relation is partitioned, all the base relations
@@ -1361,11 +1366,6 @@ try_partition_wise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2,
 
 	nparts = joinrel->nparts;
 
-	/* Allocate space to hold child-joins RelOptInfos, if not already done. */
-	if (!joinrel->part_rels)
-		joinrel->part_rels =
-			(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * 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
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 3bd1063..21fd29f 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -1746,4 +1746,9 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
 		joinrel->partexprs[cnt] = partexpr;
 		joinrel->nullable_partexprs[cnt] = nullable_partexpr;
 	}
+
+	/* Allocate space to hold child-joins RelOptInfos. */
+	joinrel->part_rels =
+		(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * joinrel->nparts);
+
 }
-- 
1.7.9.5

#291Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#290)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Mon, Oct 9, 2017 at 2:05 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

On Sat, Oct 7, 2017 at 1:04 AM, Robert Haas <robertmhaas@gmail.com> wrote:

Committed. I hope that makes things less red rather than more,
because I'm going to be AFK for a few hours anyway.

Here's the last patch, dealing with the dummy relations, rebased. With
this fix every join order of a partitioned join can be considered
partitioned. (This wasn't the case earlier when dummy relation was
involved.). So, we can allocate the child-join RelOptInfo array in
build_joinrel_partition_info(), instead of waiting for an appropriate
pair to arrive in try_partition_wise_join().

Wouldn't a far more general approach be to allow a partition-wise join
between a partitioned table and an unpartitioned table, considering
the result as partitioned? That seems like it would very often yield
much better query plans than what we have right now, and also make the
need for this particular thing go away.

--
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

#292Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#291)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Oct 11, 2017 at 7:47 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Mon, Oct 9, 2017 at 2:05 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

On Sat, Oct 7, 2017 at 1:04 AM, Robert Haas <robertmhaas@gmail.com> wrote:

Committed. I hope that makes things less red rather than more,
because I'm going to be AFK for a few hours anyway.

Here's the last patch, dealing with the dummy relations, rebased. With
this fix every join order of a partitioned join can be considered
partitioned. (This wasn't the case earlier when dummy relation was
involved.). So, we can allocate the child-join RelOptInfo array in
build_joinrel_partition_info(), instead of waiting for an appropriate
pair to arrive in try_partition_wise_join().

Wouldn't a far more general approach be to allow a partition-wise join
between a partitioned table and an unpartitioned table, considering
the result as partitioned? That seems like it would very often yield
much better query plans than what we have right now, and also make the
need for this particular thing go away.

You are suggesting that a dummy partitioned table be treated as an
un-partitioned table and apply above suggested optimization. A join
between a partitioned and unpartitioned table is partitioned by the
keys of only partitioned table. An unpartitioned table doesn't have
any keys, so this is fine. But a dummy partitioned table does have
keys. Recording them as keys of the join relation helps when it joins
to other relations. Furthermore a join between partitioned and
unpartitioned table doesn't require any equi-join condition on
partition keys of partitioned table but a join between partitioned
tables is considered to be partitioned by keys on both sides only when
there is an equi-join. So, when implementing a partitioned join
between a partitioned and an unpartitioned table, we will have to make
a special case to record partition keys when the unpartitioned side is
actually a dummy partitioned table. That might be awkward.

Because we don't have dummy children relation in all cases, we already
have some awkwardness like allocating part_rels array only when we
encounter a join order which has all the children. This patch removes
that.

--
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

#293Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#292)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Wed, Oct 11, 2017 at 10:43 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

You are suggesting that a dummy partitioned table be treated as an
un-partitioned table and apply above suggested optimization. A join
between a partitioned and unpartitioned table is partitioned by the
keys of only partitioned table. An unpartitioned table doesn't have
any keys, so this is fine. But a dummy partitioned table does have
keys. Recording them as keys of the join relation helps when it joins
to other relations. Furthermore a join between partitioned and
unpartitioned table doesn't require any equi-join condition on
partition keys of partitioned table but a join between partitioned
tables is considered to be partitioned by keys on both sides only when
there is an equi-join. So, when implementing a partitioned join
between a partitioned and an unpartitioned table, we will have to make
a special case to record partition keys when the unpartitioned side is
actually a dummy partitioned table. That might be awkward.

It seems to me that what we really need here is to move all of this
stuff into a separate struct:

/* used for partitioned relations */
PartitionScheme part_scheme; /* Partitioning scheme. */
int nparts; /* number of
partitions */
struct PartitionBoundInfoData *boundinfo; /* Partition bounds */
struct RelOptInfo **part_rels; /* Array of RelOptInfos of partitions,

* stored in the same order of bounds */
List **partexprs; /* Non-nullable partition key
expressions. */
List **nullable_partexprs; /* Nullable partition key
expressions. */

...and then have a RelOptInfo carry a pointer to a list of those
structures. That lets us consider multiple possible partition schemes
for the same relation. For instance, suppose that a user joins four
relations, P1, P2, Q1, and Q2. P1 and P2 are compatibly partitioned.
Q1 and Q2 are compatibly partitioned (but not compatible with P1 and
P2).

Furthermore, let's suppose that the optimal join order begins with a
join between P1 and Q1. When we construct the paths for that joinrel,
we can either join all of P1 to all of Q1 (giving up on partition-wise
join), or we can join each partition of P1 to all of Q1 (producing a
result partitioned compatibly with P1 and allowing for a future
partition-wise join to P2), or we can join each partition of Q1 to all
of P1 (producing a result partitioned compatibly with Q1 and allowing
for a future partition-wise join to Q2). Any of those could win
depending on the details. With the data structure as it is today,
we'd have to choose whether to mark the joinrel as partitioned like P1
or like Q1, but that's not really what we need here.

--
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

#294Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#293)
Re: Partition-wise join for join between (declaratively) partitioned tables

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

On Wed, Oct 11, 2017 at 10:43 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

You are suggesting that a dummy partitioned table be treated as an
un-partitioned table and apply above suggested optimization. A join
between a partitioned and unpartitioned table is partitioned by the
keys of only partitioned table. An unpartitioned table doesn't have
any keys, so this is fine. But a dummy partitioned table does have
keys. Recording them as keys of the join relation helps when it joins
to other relations. Furthermore a join between partitioned and
unpartitioned table doesn't require any equi-join condition on
partition keys of partitioned table but a join between partitioned
tables is considered to be partitioned by keys on both sides only when
there is an equi-join. So, when implementing a partitioned join
between a partitioned and an unpartitioned table, we will have to make
a special case to record partition keys when the unpartitioned side is
actually a dummy partitioned table. That might be awkward.

It seems to me that what we really need here is to move all of this
stuff into a separate struct:

/* used for partitioned relations */
PartitionScheme part_scheme; /* Partitioning scheme. */
int nparts; /* number of
partitions */
struct PartitionBoundInfoData *boundinfo; /* Partition bounds */
struct RelOptInfo **part_rels; /* Array of RelOptInfos of partitions,

* stored in the same order of bounds */
List **partexprs; /* Non-nullable partition key
expressions. */
List **nullable_partexprs; /* Nullable partition key
expressions. */

In a very early patch I had PartitionOptInfo to hold all of this.
RelOptInfo then had a pointer of PartitionOptInfo, if it was
partitioned. When a relation can be partitioned in multiple ways like
what you describe or because join by re-partitioning is efficient,
RelOptInfo would have a list of those. But the representation needs to
be thought through. I am wondering whether this should be modelled
like IndexOptInfo. I am not sure. This is a topic of much larger
discussion.

I think we are digressing. We were discussing my patch to handle dummy
partitioned relation, whose children are not marked dummy and do not
have pathlists set. Do you still think that we should leave that
aside?

--
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

#295Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#289)
1 attachment(s)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Sat, Oct 7, 2017 at 1:04 AM, Robert Haas <robertmhaas@gmail.com> wrote:

The fix is to copy the relevant partitioning information from relcache
into PartitionSchemeData and RelOptInfo. Here's a quick patch with
that fix.

Committed. I hope that makes things less red rather than more,
because I'm going to be AFK for a few hours anyway.

set_append_rel_size() crashes when it encounters a partitioned table
with a dropped column. Dropped columns do not have any translations
saved in AppendInfo::translated_vars; the corresponding entry is NULL
per make_inh_translation_list().
1802 att = TupleDescAttr(old_tupdesc, old_attno);
1803 if (att->attisdropped)
1804 {
1805 /* Just put NULL into this list entry */
1806 vars = lappend(vars, NULL);
1807 continue;
1808 }

In set_append_rel_size() we try to attr_needed for child tables. While
doing so we try to translate a user attribute number of parent to that
of a child and crash since the translated Var is NULL. Here's patch to
fix the crash. The patch also contains a testcase to test dropped
columns in partitioned table.

Sorry for not noticing this problem earlier.

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

Attachments:

0001-Ignore-dropped-columns-in-set_append_rel_size.patchtext/x-patch; charset=US-ASCII; name=0001-Ignore-dropped-columns-in-set_append_rel_size.patchDownload
From eca9e03410b049bf79d767657ad4d90fb974d19c Mon Sep 17 00:00:00 2001
From: Ashutosh Bapat <ashutosh.bapat@enterprisedb.com>
Date: Mon, 16 Oct 2017 13:23:49 +0530
Subject: [PATCH] Ignore dropped columns in set_append_rel_size().

A parent Var corresponding to column dropped from a parent table will
not have any translation for a child.  It won't have any attr_needed
information since it can not be referenced from SQL. Hence ignore
dropped columns while constructing attr_needed information for a child
table.

Ashutosh Bapat.
---
 src/backend/optimizer/path/allpaths.c     |   12 ++++++++++++
 src/test/regress/expected/alter_table.out |    7 +++++++
 src/test/regress/sql/alter_table.sql      |    4 ++++
 3 files changed, 23 insertions(+)

diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 5535b63..1bc5e01 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -950,6 +950,18 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 													attno - 1);
 					int			child_index;
 
+					/*
+					 * Ignore any column dropped from the parent.
+					 * Corresponding Var won't have any translation. It won't
+					 * have attr_needed information, since it can not be
+					 * referenced in the query.
+					 */
+					if (var == NULL)
+					{
+						Assert(attr_needed == NULL);
+						continue;
+					}
+
 					child_index = var->varattno - childrel->min_attr;
 					childrel->attr_needed[child_index] = attr_needed;
 				}
diff --git a/src/test/regress/expected/alter_table.out b/src/test/regress/expected/alter_table.out
index dbe438d..cc56651 100644
--- a/src/test/regress/expected/alter_table.out
+++ b/src/test/regress/expected/alter_table.out
@@ -3604,6 +3604,13 @@ ALTER TABLE list_parted2 DROP COLUMN b;
 ERROR:  cannot drop column named in partition key
 ALTER TABLE list_parted2 ALTER COLUMN b TYPE text;
 ERROR:  cannot alter type of column named in partition key
+-- dropping non-partition key columns should be allowed on the parent table.
+ALTER TABLE list_parted DROP COLUMN b;
+SELECT * FROM list_parted;
+ a 
+---
+(0 rows)
+
 -- cleanup
 DROP TABLE list_parted, list_parted2, range_parted;
 DROP TABLE fail_def_part;
diff --git a/src/test/regress/sql/alter_table.sql b/src/test/regress/sql/alter_table.sql
index 0c8ae2a..fc7bd66 100644
--- a/src/test/regress/sql/alter_table.sql
+++ b/src/test/regress/sql/alter_table.sql
@@ -2390,6 +2390,10 @@ ALTER TABLE part_2 INHERIT inh_test;
 ALTER TABLE list_parted2 DROP COLUMN b;
 ALTER TABLE list_parted2 ALTER COLUMN b TYPE text;
 
+-- dropping non-partition key columns should be allowed on the parent table.
+ALTER TABLE list_parted DROP COLUMN b;
+SELECT * FROM list_parted;
+
 -- cleanup
 DROP TABLE list_parted, list_parted2, range_parted;
 DROP TABLE fail_def_part;
-- 
1.7.9.5

#296Robert Haas
robertmhaas@gmail.com
In reply to: Ashutosh Bapat (#295)
Re: Partition-wise join for join between (declaratively) partitioned tables

On Mon, Oct 16, 2017 at 5:03 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

set_append_rel_size() crashes when it encounters a partitioned table
with a dropped column. Dropped columns do not have any translations
saved in AppendInfo::translated_vars; the corresponding entry is NULL
per make_inh_translation_list().
1802 att = TupleDescAttr(old_tupdesc, old_attno);
1803 if (att->attisdropped)
1804 {
1805 /* Just put NULL into this list entry */
1806 vars = lappend(vars, NULL);
1807 continue;
1808 }

In set_append_rel_size() we try to attr_needed for child tables. While
doing so we try to translate a user attribute number of parent to that
of a child and crash since the translated Var is NULL. Here's patch to
fix the crash. The patch also contains a testcase to test dropped
columns in partitioned table.

Sorry for not noticing this problem earlier.

OK, committed. This is a good example of how having good code
coverage doesn't necessarily mean you've found all the bugs. :-)

--
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

#297Rajkumar Raghuwanshi
rajkumar.raghuwanshi@enterprisedb.com
In reply to: Robert Haas (#296)
1 attachment(s)
Re: [HACKERS] Partition-wise join for join between (declaratively) partitioned tables

On Tue, Oct 31, 2017 at 2:45 PM, Robert Haas <robertmhaas@gmail.com> wrote:

OK, committed. This is a good example of how having good code

coverage doesn't necessarily mean you've found all the bugs. :-)

As of now partition_join.sql is not having test cases covering cases
where partition table have default partition, attaching a small test
case patch to cover those.

Thanks & Regards,
Rajkumar Raghuwanshi
QMG, EnterpriseDB Corporation

Attachments:

partition_wise_join_with_default_partitions.patchtext/x-patch; charset=US-ASCII; name=partition_wise_join_with_default_partitions.patchDownload
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index 27ab852..045b4c2 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -1337,6 +1337,160 @@ SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, ph
  574.0000000000000000 | 574.5000000000000000 | 1148.0000000000000000 | 0011 | 0011 | A0011
 (12 rows)
 
+-- test default partition behavior for range
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+-- Join with pruned partitions from joining relations
+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 < 450 AND t2.b > 250 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_p2 t2
+                     Filter: (b > 250)
+               ->  Hash
+                     ->  Seq Scan on prt1_p2 t1
+                           Filter: ((a < 450) AND (b = 0))
+         ->  Nested Loop
+               ->  Seq Scan on prt1_p3 t1_1
+                     Filter: ((a < 450) AND (b = 0))
+               ->  Index Scan using iprt2_p3_b on prt2_p3 t2_1
+                     Index Cond: ((b = t1_1.a) AND (b > 250))
+(15 rows)
+
+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
+(1 row)
+
+ALTER TABLE prt1_e DETACH PARTITION prt1_e_p3;
+ALTER TABLE prt1_e ATTACH PARTITION prt1_e_p3 DEFAULT;
+-- N-way join
+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
+               ->  Nested Loop Left Join
+                     ->  Hash Right Join
+                           Hash Cond: (t1.a = ((t3.a + t3.b) / 2))
+                           ->  Seq Scan on prt1_p1 t1
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p1 t3
+                                       Filter: (c = 0)
+                     ->  Index Scan using iprt2_p1_b on prt2_p1 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
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p2 t3_1
+                                       Filter: (c = 0)
+                     ->  Index Scan using iprt2_p2_b on prt2_p2 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
+                           ->  Hash
+                                 ->  Seq Scan on prt1_e_p3 t3_2
+                                       Filter: (c = 0)
+                     ->  Index Scan using iprt2_p3_b on prt2_p3 t2_2
+                           Index Cond: (t1_2.a = b)
+(31 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)
+
+-- test default partition behavior for list
+ALTER TABLE plt1 DETACH PARTITION plt1_p3;
+ALTER TABLE plt2 DETACH PARTITION plt2_p3;
+ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
+ALTER TABLE plt2 ATTACH PARTITION plt2_p3 DEFAULT;
+ALTER TABLE plt1_e DETACH PARTITION plt1_e_p3;
+ALTER TABLE plt1_e ATTACH PARTITION plt1_e_p3 DEFAULT;
+-- 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 = 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)
+
 --
 -- multiple levels of partitioning
 --
@@ -1663,6 +1817,73 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE a = 1 AND a = 2)
                One-Time Filter: false
 (11 rows)
 
+-- test default partition behavior for multiple levels of partitoning
+ALTER TABLE prt1_l DETACH PARTITION prt1_l_p2;
+ALTER TABLE prt1_l ATTACH PARTITION prt1_l_p2 DEFAULT;
+ALTER TABLE prt1_l_p2 DETACH PARTITION prt1_l_p2_p2;
+ALTER TABLE prt1_l_p2 ATTACH PARTITION prt1_l_p2_p2 DEFAULT;
+ALTER TABLE prt1_l_p3 DETACH PARTITION prt1_l_p3_p1;
+ALTER TABLE prt1_l_p3 ATTACH PARTITION prt1_l_p3_p1 DEFAULT;
+ALTER TABLE prt2_l DETACH PARTITION prt2_l_p2;
+ALTER TABLE prt2_l ATTACH PARTITION prt2_l_p2 DEFAULT;
+ALTER TABLE prt2_l_p2 DETACH PARTITION prt2_l_p2_p2;
+ALTER TABLE prt2_l_p2 ATTACH PARTITION prt2_l_p2_p2 DEFAULT;
+ALTER TABLE prt2_l_p3 DETACH PARTITION prt2_l_p3_p1;
+ALTER TABLE prt2_l_p3 ATTACH PARTITION prt2_l_p3_p1 DEFAULT;
+-- left join
+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                                     
+------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2.b = t1.a) AND ((t2.c)::text = (t1.c)::text))
+               ->  Seq Scan on prt2_l_p1 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_l_p1 t1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.b = t1_1.a) AND ((t2_1.c)::text = (t1_1.c)::text))
+               ->  Append
+                     ->  Seq Scan on prt2_l_p3_p2 t2_1
+                     ->  Seq Scan on prt2_l_p3_p1 t2_2
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on prt1_l_p3_p1 t1_1
+                                 Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.b = t1_2.a) AND ((t2_3.c)::text = (t1_2.c)::text))
+               ->  Seq Scan on prt2_l_p2_p1 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_l_p2_p1 t1_2
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: ((t2_4.b = t1_3.a) AND ((t2_4.c)::text = (t1_3.c)::text))
+               ->  Seq Scan on prt2_l_p2_p2 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_l_p2_p2 t1_3
+                           Filter: (b = 0)
+(30 rows)
+
+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;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  50 | 0002 |     | 
+ 100 | 0000 |     | 
+ 150 | 0002 | 150 | 0002
+ 200 | 0000 |     | 
+ 250 | 0002 |     | 
+ 300 | 0000 | 300 | 0000
+ 350 | 0002 |     | 
+ 400 | 0000 |     | 
+ 450 | 0002 | 450 | 0002
+ 500 | 0000 |     | 
+ 550 | 0002 |     | 
+(12 rows)
+
 --
 -- negative testcases
 --
@@ -1868,3 +2089,24 @@ 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_1
 (10 rows)
 
+-- partition-wise join can not be applied if the different partition have
+-- default partition
+ALTER TABLE prt4_n DETACH PARTITION prt4_n_p1;
+ALTER TABLE prt4_n ATTACH PARTITION prt4_n_p1 DEFAULT;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
+                  QUERY PLAN                  
+----------------------------------------------
+ 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
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt4_n_p2 t2
+               ->  Seq Scan on prt4_n_p3 t2_1
+               ->  Seq Scan on prt4_n_p1 t2_2
+(11 rows)
+
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index 6efdf3c..e6eb64a 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -261,6 +261,39 @@ 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.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.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 prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+
+-- Join with pruned partitions from joining relations
+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 < 450 AND t2.b > 250 AND t1.b = 0 ORDER BY t1.a, t2.b;
+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;
+
+ALTER TABLE prt1_e DETACH PARTITION prt1_e_p3;
+ALTER TABLE prt1_e ATTACH PARTITION prt1_e_p3 DEFAULT;
+
+-- N-way join
+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;
+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;
+
+-- test default partition behavior for list
+ALTER TABLE plt1 DETACH PARTITION plt1_p3;
+ALTER TABLE plt2 DETACH PARTITION plt2_p3;
+ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
+ALTER TABLE plt2 ATTACH PARTITION plt2_p3 DEFAULT;
+
+ALTER TABLE plt1_e DETACH PARTITION plt1_e_p3;
+ALTER TABLE plt1_e ATTACH PARTITION plt1_e_p3 DEFAULT;
+
+-- 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;
+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;
+
 --
 -- multiple levels of partitioning
 --
@@ -319,6 +352,26 @@ SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
 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;
 
+-- test default partition behavior for multiple levels of partitoning
+ALTER TABLE prt1_l DETACH PARTITION prt1_l_p2;
+ALTER TABLE prt1_l ATTACH PARTITION prt1_l_p2 DEFAULT;
+ALTER TABLE prt1_l_p2 DETACH PARTITION prt1_l_p2_p2;
+ALTER TABLE prt1_l_p2 ATTACH PARTITION prt1_l_p2_p2 DEFAULT;
+ALTER TABLE prt1_l_p3 DETACH PARTITION prt1_l_p3_p1;
+ALTER TABLE prt1_l_p3 ATTACH PARTITION prt1_l_p3_p1 DEFAULT;
+
+ALTER TABLE prt2_l DETACH PARTITION prt2_l_p2;
+ALTER TABLE prt2_l ATTACH PARTITION prt2_l_p2 DEFAULT;
+ALTER TABLE prt2_l_p2 DETACH PARTITION prt2_l_p2_p2;
+ALTER TABLE prt2_l_p2 ATTACH PARTITION prt2_l_p2_p2 DEFAULT;
+ALTER TABLE prt2_l_p3 DETACH PARTITION prt2_l_p3_p1;
+ALTER TABLE prt2_l_p3 ATTACH PARTITION prt2_l_p3_p1 DEFAULT;
+
+-- left join
+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;
+
 --
 -- negative testcases
 --
@@ -384,3 +437,10 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
 -- partitioned table
 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);
+
+-- partition-wise join can not be applied if the different partition have
+-- default partition
+ALTER TABLE prt4_n DETACH PARTITION prt4_n_p1;
+ALTER TABLE prt4_n ATTACH PARTITION prt4_n_p1 DEFAULT;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
#298Robert Haas
robertmhaas@gmail.com
In reply to: Rajkumar Raghuwanshi (#297)
Re: [HACKERS] Partition-wise join for join between (declaratively) partitioned tables

On Fri, Dec 1, 2017 at 1:36 AM, Rajkumar Raghuwanshi
<rajkumar.raghuwanshi@enterprisedb.com> wrote:

On Tue, Oct 31, 2017 at 2:45 PM, Robert Haas <robertmhaas@gmail.com> wrote:

OK, committed. This is a good example of how having good code

coverage doesn't necessarily mean you've found all the bugs. :-)

As of now partition_join.sql is not having test cases covering cases
where partition table have default partition, attaching a small test
case patch to cover those.

That's not that small, and to me it looks like overkill.

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

#299Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Robert Haas (#298)
Re: [HACKERS] Partition-wise join for join between (declaratively) partitioned tables

On Sat, Dec 2, 2017 at 2:13 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Fri, Dec 1, 2017 at 1:36 AM, Rajkumar Raghuwanshi
<rajkumar.raghuwanshi@enterprisedb.com> wrote:

On Tue, Oct 31, 2017 at 2:45 PM, Robert Haas <robertmhaas@gmail.com> wrote:

OK, committed. This is a good example of how having good code

coverage doesn't necessarily mean you've found all the bugs. :-)

As of now partition_join.sql is not having test cases covering cases
where partition table have default partition, attaching a small test
case patch to cover those.

That's not that small, and to me it looks like overkill.

I agree, the patch looks longer than expected. I think, it's important
to have some testcases to test partition-wise join with default
partitions. I think we need at least one test for range default
partitions, one test for list partitioning, one for multi-level
partitioning and one negative testcase with default partition missing
from one side of join.

May be we could reduce the number of SQL commands and queries in the
patch by adding default partition to every table that participates in
partition-wise join (leave the tables participating in negative tests
aside.). But that's going to increase the size of EXPLAIN outputs and
query results. The negative test may simply drop the default partition
from one of the tables.

For every table being tested, the patch adds two ALTER TABLE commands,
one for detaching an existing partition and then attach the same as
default partition. Alternative to that is just add a new default
partition without detaching and existing partition. But then the
default partition needs to populated with some data, which requires 1
INSERT statement at least. That doesn't reduce the size of patch, but
increases the output of query and EXPLAIN plan.

May be in case of multi-level partitioning test, we don't need to add
DEFAULT in every partitioned relation; adding to one of them would be
enough. May be add it to the parent, but that too can be avoided. That
would reduce the size of patch a bit.

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

#300Rajkumar Raghuwanshi
rajkumar.raghuwanshi@enterprisedb.com
In reply to: Ashutosh Bapat (#299)
1 attachment(s)
Re: [HACKERS] Partition-wise join for join between (declaratively) partitioned tables

On Mon, Dec 4, 2017 at 7:34 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

I agree, the patch looks longer than expected. I think, it's important
to have some testcases to test partition-wise join with default
partitions. I think we need at least one test for range default
partitions, one test for list partitioning, one for multi-level
partitioning and one negative testcase with default partition missing
from one side of join.

May be we could reduce the number of SQL commands and queries in the
patch by adding default partition to every table that participates in
partition-wise join (leave the tables participating in negative tests
aside.). But that's going to increase the size of EXPLAIN outputs and
query results. The negative test may simply drop the default partition
from one of the tables.

For every table being tested, the patch adds two ALTER TABLE commands,
one for detaching an existing partition and then attach the same as
default partition. Alternative to that is just add a new default
partition without detaching and existing partition. But then the
default partition needs to populated with some data, which requires 1
INSERT statement at least. That doesn't reduce the size of patch, but
increases the output of query and EXPLAIN plan.

May be in case of multi-level partitioning test, we don't need to add
DEFAULT in every partitioned relation; adding to one of them would be
enough. May be add it to the parent, but that too can be avoided. That
would reduce the size of patch a bit.

Thanks Ashutosh for suggestions.

I have reduced test cases as suggested. Attaching updated patch.

Thanks & Regards,
Rajkumar Raghuwanshi
QMG, EnterpriseDB Corporation

Attachments:

partition_wise_join_with_default_partitions_v1.patchtext/x-patch; charset=US-ASCII; name=partition_wise_join_with_default_partitions_v1.patchDownload
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index 27ab852..f83166b 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -1337,6 +1337,96 @@ SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, ph
  574.0000000000000000 | 574.5000000000000000 | 1148.0000000000000000 | 0011 | 0011 | A0011
 (12 rows)
 
+-- test default partition behavior for range
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+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)
+
+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)
+
+-- test default partition behavior for list
+ALTER TABLE plt1 DETACH PARTITION plt1_p3;
+ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
+ALTER TABLE plt2 DETACH PARTITION plt2_p3;
+ALTER TABLE plt2 ATTACH PARTITION plt2_p3 DEFAULT;
+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 GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t2.c
+   ->  HashAggregate
+         Group Key: t1.c, t2.c
+         ->  Result
+               ->  Append
+                     ->  Hash Right Join
+                           Hash Cond: (t1.c = t2.c)
+                           ->  Seq Scan on plt1_p1 t1
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p1 t2
+                     ->  Hash Right 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 Right Join
+                           Hash Cond: (t1_2.c = t2_2.c)
+                           ->  Seq Scan on plt1_p3 t1_2
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p3 t2_2
+(21 rows)
+
+SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+         avg          |         avg          |  c   |  c   
+----------------------+----------------------+------+------
+  24.0000000000000000 |  24.0000000000000000 | 0000 | 0000
+  74.0000000000000000 |  75.0000000000000000 | 0001 | 0001
+ 124.0000000000000000 | 124.5000000000000000 | 0002 | 0002
+ 174.0000000000000000 | 174.0000000000000000 | 0003 | 0003
+ 224.0000000000000000 | 225.0000000000000000 | 0004 | 0004
+ 274.0000000000000000 | 274.5000000000000000 | 0005 | 0005
+ 324.0000000000000000 | 324.0000000000000000 | 0006 | 0006
+ 374.0000000000000000 | 375.0000000000000000 | 0007 | 0007
+ 424.0000000000000000 | 424.5000000000000000 | 0008 | 0008
+ 474.0000000000000000 | 474.0000000000000000 | 0009 | 0009
+ 524.0000000000000000 | 525.0000000000000000 | 0010 | 0010
+ 574.0000000000000000 | 574.5000000000000000 | 0011 | 0011
+(12 rows)
+
 --
 -- multiple levels of partitioning
 --
@@ -1663,6 +1753,64 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE a = 1 AND a = 2)
                One-Time Filter: false
 (11 rows)
 
+-- test default partition behavior for multiple levels of partitoning
+ALTER TABLE prt1_l DETACH PARTITION prt1_l_p2;
+ALTER TABLE prt1_l ATTACH PARTITION prt1_l_p2 DEFAULT;
+ALTER TABLE prt2_l DETACH PARTITION prt2_l_p2;
+ALTER TABLE prt2_l ATTACH PARTITION prt2_l_p2 DEFAULT;
+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                                     
+------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2.b = t1.a) AND ((t2.c)::text = (t1.c)::text))
+               ->  Seq Scan on prt2_l_p1 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_l_p1 t1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.b = t1_1.a) AND ((t2_1.c)::text = (t1_1.c)::text))
+               ->  Append
+                     ->  Seq Scan on prt2_l_p3_p1 t2_1
+                     ->  Seq Scan on prt2_l_p3_p2 t2_2
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on prt1_l_p3_p1 t1_1
+                                 Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.b = t1_2.a) AND ((t2_3.c)::text = (t1_2.c)::text))
+               ->  Seq Scan on prt2_l_p2_p1 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_l_p2_p1 t1_2
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: ((t2_4.b = t1_3.a) AND ((t2_4.c)::text = (t1_3.c)::text))
+               ->  Seq Scan on prt2_l_p2_p2 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_l_p2_p2 t1_3
+                           Filter: (b = 0)
+(30 rows)
+
+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;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  50 | 0002 |     | 
+ 100 | 0000 |     | 
+ 150 | 0002 | 150 | 0002
+ 200 | 0000 |     | 
+ 250 | 0002 |     | 
+ 300 | 0000 | 300 | 0000
+ 350 | 0002 |     | 
+ 400 | 0000 |     | 
+ 450 | 0002 | 450 | 0002
+ 500 | 0000 |     | 
+ 550 | 0002 |     | 
+(12 rows)
+
 --
 -- negative testcases
 --
@@ -1683,27 +1831,27 @@ CREATE TABLE prt3_n_p3 PARTITION OF prt3_n FOR VALUES IN ('0003', '0005', '0009'
 INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0, 599, 2) i;
 ANALYZE prt3_n;
 CREATE TABLE prt4_n (a int, b int, c text) PARTITION BY RANGE(a);
-CREATE TABLE prt4_n_p1 PARTITION OF prt4_n FOR VALUES FROM (0) TO (300);
+CREATE TABLE prt4_n_p1 PARTITION OF prt4_n FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt4_n_p2 PARTITION OF prt4_n FOR VALUES FROM (300) TO (500);
 CREATE TABLE prt4_n_p3 PARTITION OF prt4_n FOR VALUES FROM (500) TO (600);
-INSERT INTO prt4_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+INSERT INTO prt4_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i WHERE i < 250 AND i > 300;
 ANALYZE prt4_n;
 -- partition-wise join can not be applied if the partition ranges differ
 EXPLAIN (COSTS OFF)
 SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
-                  QUERY PLAN                  
-----------------------------------------------
+                 QUERY PLAN                 
+--------------------------------------------
  Hash Join
-   Hash Cond: (t1.a = t2.a)
+   Hash Cond: (t2.a = t1.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 prt4_n_p1 t2
+         ->  Seq Scan on prt4_n_p2 t2_1
+         ->  Seq Scan on prt4_n_p3 t2_2
    ->  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
+               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p3 t1_2
 (11 rows)
 
 EXPLAIN (COSTS OFF)
@@ -1868,3 +2016,24 @@ 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_1
 (10 rows)
 
+-- partition-wise join can not be applied if the different partition have
+-- default partition
+ALTER TABLE prt4_n DETACH PARTITION prt4_n_p2;
+ALTER TABLE prt4_n ATTACH PARTITION prt4_n_p2 DEFAULT;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
+                 QUERY PLAN                 
+--------------------------------------------
+ Hash Join
+   Hash Cond: (t2.a = t1.a)
+   ->  Append
+         ->  Seq Scan on prt4_n_p1 t2
+         ->  Seq Scan on prt4_n_p3 t2_1
+         ->  Seq Scan on prt4_n_p2 t2_2
+   ->  Hash
+         ->  Append
+               ->  Seq Scan on prt1_p1 t1
+               ->  Seq Scan on prt1_p2 t1_1
+               ->  Seq Scan on prt1_p3 t1_2
+(11 rows)
+
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index 6efdf3c..cb39485 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -261,6 +261,26 @@ 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.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.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;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+
+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 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- test default partition behavior for list
+ALTER TABLE plt1 DETACH PARTITION plt1_p3;
+ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
+ALTER TABLE plt2 DETACH PARTITION plt2_p3;
+ALTER TABLE plt2 ATTACH PARTITION plt2_p3 DEFAULT;
+
+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 GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
 --
 -- multiple levels of partitioning
 --
@@ -319,6 +339,17 @@ SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
 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;
 
+-- test default partition behavior for multiple levels of partitoning
+ALTER TABLE prt1_l DETACH PARTITION prt1_l_p2;
+ALTER TABLE prt1_l ATTACH PARTITION prt1_l_p2 DEFAULT;
+
+ALTER TABLE prt2_l DETACH PARTITION prt2_l_p2;
+ALTER TABLE prt2_l ATTACH PARTITION prt2_l_p2 DEFAULT;
+
+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;
+
 --
 -- negative testcases
 --
@@ -342,10 +373,10 @@ INSERT INTO prt2_n SELECT i, i, to_char(i/50, 'FM0000') FROM generate_series(0,
 ANALYZE prt3_n;
 
 CREATE TABLE prt4_n (a int, b int, c text) PARTITION BY RANGE(a);
-CREATE TABLE prt4_n_p1 PARTITION OF prt4_n FOR VALUES FROM (0) TO (300);
+CREATE TABLE prt4_n_p1 PARTITION OF prt4_n FOR VALUES FROM (0) TO (250);
 CREATE TABLE prt4_n_p2 PARTITION OF prt4_n FOR VALUES FROM (300) TO (500);
 CREATE TABLE prt4_n_p3 PARTITION OF prt4_n FOR VALUES FROM (500) TO (600);
-INSERT INTO prt4_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i;
+INSERT INTO prt4_n SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0, 599, 2) i WHERE i < 250 AND i > 300;
 ANALYZE prt4_n;
 
 -- partition-wise join can not be applied if the partition ranges differ
@@ -384,3 +415,10 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
 -- partitioned table
 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);
+
+-- partition-wise join can not be applied if the different partition have
+-- default partition
+ALTER TABLE prt4_n DETACH PARTITION prt4_n_p2;
+ALTER TABLE prt4_n ATTACH PARTITION prt4_n_p2 DEFAULT;
+EXPLAIN (COSTS OFF)
+SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt4_n t2 WHERE t1.a = t2.a;
#301Rajkumar Raghuwanshi
rajkumar.raghuwanshi@enterprisedb.com
In reply to: Rajkumar Raghuwanshi (#300)
1 attachment(s)
Re: [HACKERS] Partition-wise join for join between (declaratively) partitioned tables

On Tue, Dec 5, 2017 at 11:04 AM, Rajkumar Raghuwanshi
<rajkumar.raghuwanshi@enterprisedb.com> wrote:

On Mon, Dec 4, 2017 at 7:34 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

I agree, the patch looks longer than expected. I think, it's important
to have some testcases to test partition-wise join with default
partitions. I think we need at least one test for range default
partitions, one test for list partitioning, one for multi-level
partitioning and one negative testcase with default partition missing
from one side of join.

May be we could reduce the number of SQL commands and queries in the
patch by adding default partition to every table that participates in
partition-wise join (leave the tables participating in negative tests
aside.). But that's going to increase the size of EXPLAIN outputs and
query results. The negative test may simply drop the default partition
from one of the tables.

For every table being tested, the patch adds two ALTER TABLE commands,
one for detaching an existing partition and then attach the same as
default partition. Alternative to that is just add a new default
partition without detaching and existing partition. But then the
default partition needs to populated with some data, which requires 1
INSERT statement at least. That doesn't reduce the size of patch, but
increases the output of query and EXPLAIN plan.

May be in case of multi-level partitioning test, we don't need to add
DEFAULT in every partitioned relation; adding to one of them would be
enough. May be add it to the parent, but that too can be avoided. That
would reduce the size of patch a bit.

Thanks Ashutosh for suggestions.

I have reduced test cases as suggested. Attaching updated patch.

Sorry Attached wrong patch.

attaching correct patch now.

Attachments:

partition_wise_join_with_default_partitions_v2.patchtext/x-patch; charset=US-ASCII; name=partition_wise_join_with_default_partitions_v2.patchDownload
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index 27ab852..a60ba7f 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -1337,6 +1337,96 @@ SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, ph
  574.0000000000000000 | 574.5000000000000000 | 1148.0000000000000000 | 0011 | 0011 | A0011
 (12 rows)
 
+-- test default partition behavior for range
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+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)
+
+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)
+
+-- test default partition behavior for list
+ALTER TABLE plt1 DETACH PARTITION plt1_p3;
+ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
+ALTER TABLE plt2 DETACH PARTITION plt2_p3;
+ALTER TABLE plt2 ATTACH PARTITION plt2_p3 DEFAULT;
+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 GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t2.c
+   ->  HashAggregate
+         Group Key: t1.c, t2.c
+         ->  Result
+               ->  Append
+                     ->  Hash Right Join
+                           Hash Cond: (t1.c = t2.c)
+                           ->  Seq Scan on plt1_p1 t1
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p1 t2
+                     ->  Hash Right 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 Right Join
+                           Hash Cond: (t1_2.c = t2_2.c)
+                           ->  Seq Scan on plt1_p3 t1_2
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p3 t2_2
+(21 rows)
+
+SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+         avg          |         avg          |  c   |  c   
+----------------------+----------------------+------+------
+  24.0000000000000000 |  24.0000000000000000 | 0000 | 0000
+  74.0000000000000000 |  75.0000000000000000 | 0001 | 0001
+ 124.0000000000000000 | 124.5000000000000000 | 0002 | 0002
+ 174.0000000000000000 | 174.0000000000000000 | 0003 | 0003
+ 224.0000000000000000 | 225.0000000000000000 | 0004 | 0004
+ 274.0000000000000000 | 274.5000000000000000 | 0005 | 0005
+ 324.0000000000000000 | 324.0000000000000000 | 0006 | 0006
+ 374.0000000000000000 | 375.0000000000000000 | 0007 | 0007
+ 424.0000000000000000 | 424.5000000000000000 | 0008 | 0008
+ 474.0000000000000000 | 474.0000000000000000 | 0009 | 0009
+ 524.0000000000000000 | 525.0000000000000000 | 0010 | 0010
+ 574.0000000000000000 | 574.5000000000000000 | 0011 | 0011
+(12 rows)
+
 --
 -- multiple levels of partitioning
 --
@@ -1663,6 +1753,64 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT * FROM prt1_l WHERE a = 1 AND a = 2)
                One-Time Filter: false
 (11 rows)
 
+-- test default partition behavior for multiple levels of partitoning
+ALTER TABLE prt1_l DETACH PARTITION prt1_l_p2;
+ALTER TABLE prt1_l ATTACH PARTITION prt1_l_p2 DEFAULT;
+ALTER TABLE prt2_l DETACH PARTITION prt2_l_p2;
+ALTER TABLE prt2_l ATTACH PARTITION prt2_l_p2 DEFAULT;
+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                                     
+------------------------------------------------------------------------------------
+ Sort
+   Sort Key: t1.a, t2.b
+   ->  Append
+         ->  Hash Right Join
+               Hash Cond: ((t2.b = t1.a) AND ((t2.c)::text = (t1.c)::text))
+               ->  Seq Scan on prt2_l_p1 t2
+               ->  Hash
+                     ->  Seq Scan on prt1_l_p1 t1
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: ((t2_1.b = t1_1.a) AND ((t2_1.c)::text = (t1_1.c)::text))
+               ->  Append
+                     ->  Seq Scan on prt2_l_p3_p1 t2_1
+                     ->  Seq Scan on prt2_l_p3_p2 t2_2
+               ->  Hash
+                     ->  Append
+                           ->  Seq Scan on prt1_l_p3_p1 t1_1
+                                 Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: ((t2_3.b = t1_2.a) AND ((t2_3.c)::text = (t1_2.c)::text))
+               ->  Seq Scan on prt2_l_p2_p1 t2_3
+               ->  Hash
+                     ->  Seq Scan on prt1_l_p2_p1 t1_2
+                           Filter: (b = 0)
+         ->  Hash Right Join
+               Hash Cond: ((t2_4.b = t1_3.a) AND ((t2_4.c)::text = (t1_3.c)::text))
+               ->  Seq Scan on prt2_l_p2_p2 t2_4
+               ->  Hash
+                     ->  Seq Scan on prt1_l_p2_p2 t1_3
+                           Filter: (b = 0)
+(30 rows)
+
+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;
+  a  |  c   |  b  |  c   
+-----+------+-----+------
+   0 | 0000 |   0 | 0000
+  50 | 0002 |     | 
+ 100 | 0000 |     | 
+ 150 | 0002 | 150 | 0002
+ 200 | 0000 |     | 
+ 250 | 0002 |     | 
+ 300 | 0000 | 300 | 0000
+ 350 | 0002 |     | 
+ 400 | 0000 |     | 
+ 450 | 0002 | 450 | 0002
+ 500 | 0000 |     | 
+ 550 | 0002 |     | 
+(12 rows)
+
 --
 -- negative testcases
 --
@@ -1868,3 +2016,29 @@ 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_1
 (10 rows)
 
+-- partition-wise join can not be applied if the only one of joining table have
+-- default partition
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
+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_p1 t2
+               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p3 t2_2
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p1 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_2
+                           Filter: (b = 0)
+(16 rows)
+
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index 6efdf3c..5b96012 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -261,6 +261,26 @@ 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.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.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;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;
+
+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 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b;
+
+-- test default partition behavior for list
+ALTER TABLE plt1 DETACH PARTITION plt1_p3;
+ALTER TABLE plt1 ATTACH PARTITION plt1_p3 DEFAULT;
+ALTER TABLE plt2 DETACH PARTITION plt2_p3;
+ALTER TABLE plt2 ATTACH PARTITION plt2_p3 DEFAULT;
+
+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 GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+SELECT avg(t1.a), avg(t2.b), t1.c, t2.c FROM plt1 t1 RIGHT JOIN plt2 t2 ON t1.c = t2.c GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
 --
 -- multiple levels of partitioning
 --
@@ -319,6 +339,17 @@ SELECT * FROM prt1_l t1 LEFT JOIN LATERAL
 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;
 
+-- test default partition behavior for multiple levels of partitoning
+ALTER TABLE prt1_l DETACH PARTITION prt1_l_p2;
+ALTER TABLE prt1_l ATTACH PARTITION prt1_l_p2 DEFAULT;
+
+ALTER TABLE prt2_l DETACH PARTITION prt2_l_p2;
+ALTER TABLE prt2_l ATTACH PARTITION prt2_l_p2 DEFAULT;
+
+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;
+
 --
 -- negative testcases
 --
@@ -384,3 +415,10 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
 -- partitioned table
 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);
+
+-- partition-wise join can not be applied if the only one of joining table have
+-- default partition
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600);
+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;
#302Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Rajkumar Raghuwanshi (#301)
Re: [HACKERS] Partition-wise join for join between (declaratively) partitioned tables

On Tue, Dec 5, 2017 at 1:24 PM, Rajkumar Raghuwanshi
<rajkumar.raghuwanshi@enterprisedb.com> wrote:

On Tue, Dec 5, 2017 at 11:04 AM, Rajkumar Raghuwanshi
<rajkumar.raghuwanshi@enterprisedb.com> wrote:

On Mon, Dec 4, 2017 at 7:34 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

I agree, the patch looks longer than expected. I think, it's important
to have some testcases to test partition-wise join with default
partitions. I think we need at least one test for range default
partitions, one test for list partitioning, one for multi-level
partitioning and one negative testcase with default partition missing
from one side of join.

May be we could reduce the number of SQL commands and queries in the
patch by adding default partition to every table that participates in
partition-wise join (leave the tables participating in negative tests
aside.). But that's going to increase the size of EXPLAIN outputs and
query results. The negative test may simply drop the default partition
from one of the tables.

For every table being tested, the patch adds two ALTER TABLE commands,
one for detaching an existing partition and then attach the same as
default partition. Alternative to that is just add a new default
partition without detaching and existing partition. But then the
default partition needs to populated with some data, which requires 1
INSERT statement at least. That doesn't reduce the size of patch, but
increases the output of query and EXPLAIN plan.

May be in case of multi-level partitioning test, we don't need to add
DEFAULT in every partitioned relation; adding to one of them would be
enough. May be add it to the parent, but that too can be avoided. That
would reduce the size of patch a bit.

Thanks Ashutosh for suggestions.

I have reduced test cases as suggested. Attaching updated patch.

Sorry Attached wrong patch.

attaching correct patch now.

Thanks. Here are some comments

+-- test default partition behavior for range
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;

I think we need an ANALYZE here in case the statistics gets updated while
DETACH and ATTACH is going on. Other testcases also need to be updated with
ANALYZE, including the negative one.

+-- partition-wise join can not be applied if the only one of joining table have

Correction: ... if only one of the joining tables has ...

Please add the patch to the next commitfest so that it's not
forgotten. I think we can get rid of the multi-level partition-wise
testcase as well. Also, since we are re-attaching existing partition
tables as default partitions, we don't need to check the output as
well; just plan should be enough.

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

#303Rajkumar Raghuwanshi
rajkumar.raghuwanshi@enterprisedb.com
In reply to: Ashutosh Bapat (#302)
1 attachment(s)
Re: [HACKERS] Partition-wise join for join between (declaratively) partitioned tables

On Wed, Dec 20, 2017 at 5:21 PM, Ashutosh Bapat <
ashutosh.bapat@enterprisedb.com> wrote:

Thanks. Here are some comments

Thanks Ashutosh for review and suggestions.

+-- test default partition behavior for range
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;

I think we need an ANALYZE here in case the statistics gets updated while
DETACH and ATTACH is going on. Other testcases also need to be updated with
ANALYZE, including the negative one.

Done.

+-- partition-wise join can not be applied if the only one of joining
table have

Correction: ... if only one of the joining tables has ...

Done.

Please add the patch to the next commitfest so that it's not
forgotten.

Done.
Added to CF: https://commitfest.postgresql.org/16/1426/

I think we can get rid of the multi-level partition-wise
testcase as well. Also, since we are re-attaching existing partition
tables as default partitions, we don't need to check the output as
well; just plan should be enough.

Ok. Done.

updated test patch attached.

Attachments:

partition_wise_join_with_default_partitions_v3.patchtext/x-patch; charset=US-ASCII; name=partition_wise_join_with_default_partitions_v3.patchDownload
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index 27ab852..d4c875a 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -1337,6 +1337,74 @@ SELECT avg(t1.a), avg(t2.b), avg(t3.a + t3.b), t1.c, t2.c, t3.c FROM pht1 t1, ph
  574.0000000000000000 | 574.5000000000000000 | 1148.0000000000000000 | 0011 | 0011 | A0011
 (12 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
+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 GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.c, t2.c
+   ->  HashAggregate
+         Group Key: t1.c, t2.c
+         ->  Result
+               ->  Append
+                     ->  Hash Right Join
+                           Hash Cond: (t1.c = t2.c)
+                           ->  Seq Scan on plt1_p1 t1
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p1 t2
+                     ->  Hash Right 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 Right Join
+                           Hash Cond: (t1_2.c = t2_2.c)
+                           ->  Seq Scan on plt1_p3 t1_2
+                           ->  Hash
+                                 ->  Seq Scan on plt2_p3 t2_2
+(21 rows)
+
 --
 -- multiple levels of partitioning
 --
@@ -1868,3 +1936,30 @@ 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_1
 (10 rows)
 
+-- partition-wise join can not be applied if only one of joining table 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;
+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_p1 t2
+               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p3 t2_2
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p1 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_2
+                           Filter: (b = 0)
+(16 rows)
+
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index 6efdf3c..9ff990a 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -261,6 +261,28 @@ 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.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.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
+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 GROUP BY t1.c, t2.c ORDER BY t1.c, t2.c;
+
 --
 -- multiple levels of partitioning
 --
@@ -384,3 +406,12 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
 -- partitioned table
 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);
+
+-- partition-wise join can not be applied if only one of joining table 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;
+
+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;
#304Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Rajkumar Raghuwanshi (#303)
Re: [HACKERS] Partition-wise join for join between (declaratively) partitioned tables

On Fri, Dec 22, 2017 at 3:00 PM, Rajkumar Raghuwanshi
<rajkumar.raghuwanshi@enterprisedb.com> wrote:

On Wed, Dec 20, 2017 at 5:21 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:

Thanks. Here are some comments

Thanks Ashutosh for review and suggestions.

+-- test default partition behavior for range
+ALTER TABLE prt1 DETACH PARTITION prt1_p3;
+ALTER TABLE prt1 ATTACH PARTITION prt1_p3 DEFAULT;
+ALTER TABLE prt2 DETACH PARTITION prt2_p3;
+ALTER TABLE prt2 ATTACH PARTITION prt2_p3 DEFAULT;

I think we need an ANALYZE here in case the statistics gets updated while
DETACH and ATTACH is going on. Other testcases also need to be updated
with
ANALYZE, including the negative one.

Done.

+-- partition-wise join can not be applied if the only one of joining
table have

Correction: ... if only one of the joining tables has ...

Done.

Please add the patch to the next commitfest so that it's not
forgotten.

Done.
Added to CF: https://commitfest.postgresql.org/16/1426/

I think we can get rid of the multi-level partition-wise
testcase as well. Also, since we are re-attaching existing partition
tables as default partitions, we don't need to check the output as
well; just plan should be enough.

Ok. Done.

updated test patch attached.

The patch looks good to me. I don't think we can reduce it further.
But we need some tests to test PWJ with default partitions. Marking
this as ready for committer.

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

#305Rajkumar Raghuwanshi
rajkumar.raghuwanshi@enterprisedb.com
In reply to: Ashutosh Bapat (#304)
1 attachment(s)
Re: [HACKERS] Partition-wise join for join between (declaratively) partitioned tables

On Wed, Feb 7, 2018 at 2:00 PM, Ashutosh Bapat <
ashutosh.bapat@enterprisedb.com> wrote:

On Fri, Dec 22, 2017 at 3:00 PM, Rajkumar Raghuwanshi
<rajkumar.raghuwanshi@enterprisedb.com> wrote:

updated test patch attached.

Changed partition-wise statement to partitionwise.
Attached re-based patch.

The patch looks good to me. I don't think we can reduce it further.
But we need some tests to test PWJ with default partitions. Marking
this as ready for committer.

Thanks Ashutosh.

Attachments:

partitionwise_join_with_default_partitions_v4.patchtext/x-patch; charset=US-ASCII; name=partitionwise_join_with_default_partitions_v4.patchDownload
diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out
index 4fccd9a..7bbdc36 100644
--- a/src/test/regress/expected/partition_join.out
+++ b/src/test/regress/expected/partition_join.out
@@ -1338,6 +1338,77 @@ 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
+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;
+                          QUERY PLAN                          
+--------------------------------------------------------------
+ Sort
+   Sort Key: t1.c
+   ->  HashAggregate
+         Group Key: t1.c, t2.c
+         ->  Result
+               ->  Append
+                     ->  Hash Join
+                           Hash Cond: (t2.c = t1.c)
+                           ->  Seq Scan on plt2_p1 t2
+                           ->  Hash
+                                 ->  Seq Scan on plt1_p1 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
+                                       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
+                                       Filter: ((a % 25) = 0)
+(24 rows)
+
 --
 -- multiple levels of partitioning
 --
@@ -1869,3 +1940,30 @@ 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_1
 (10 rows)
 
+-- partitionwise join can not be applied if only one of joining table 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;
+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_p1 t2
+               ->  Seq Scan on prt2_p2 t2_1
+               ->  Seq Scan on prt2_p3 t2_2
+         ->  Hash
+               ->  Append
+                     ->  Seq Scan on prt1_p1 t1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p2 t1_1
+                           Filter: (b = 0)
+                     ->  Seq Scan on prt1_p3 t1_2
+                           Filter: (b = 0)
+(16 rows)
+
diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql
index a2d8b1b..5d5de59 100644
--- a/src/test/regress/sql/partition_join.sql
+++ b/src/test/regress/sql/partition_join.sql
@@ -261,6 +261,27 @@ 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
+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
 --
@@ -384,3 +405,12 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI
 -- partitioned table
 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 table 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;
+
+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;
#306Thomas Munro
thomas.munro@enterprisedb.com
In reply to: Rajkumar Raghuwanshi (#305)
Re: [HACKERS] Partition-wise join for join between (declaratively) partitioned tables

On Mon, Mar 5, 2018 at 8:13 PM, Rajkumar Raghuwanshi
<rajkumar.raghuwanshi@enterprisedb.com> wrote:

On Wed, Feb 7, 2018 at 2:00 PM, Ashutosh Bapat
Changed partition-wise statement to partitionwise.
Attached re-based patch.

The patch looks good to me. I don't think we can reduce it further.
But we need some tests to test PWJ with default partitions. Marking
this as ready for committer.

Hi Rajkumar,

partition_join ... FAILED

The regression test currently fails with your v4 patch because a
redundant Result node has been removed from a query plan. That may be
due to commit 11cf92f6 or nearby commits.

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

#307Ashutosh Bapat
ashutosh.bapat@enterprisedb.com
In reply to: Thomas Munro (#306)
Re: [HACKERS] Partition-wise join for join between (declaratively) partitioned tables

On Wed, Jun 6, 2018 at 8:11 AM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

On Mon, Mar 5, 2018 at 8:13 PM, Rajkumar Raghuwanshi
<rajkumar.raghuwanshi@enterprisedb.com> wrote:

On Wed, Feb 7, 2018 at 2:00 PM, Ashutosh Bapat
Changed partition-wise statement to partitionwise.
Attached re-based patch.

The patch looks good to me. I don't think we can reduce it further.
But we need some tests to test PWJ with default partitions. Marking
this as ready for committer.

Hi Rajkumar,

partition_join ... FAILED

That made my heart stop for fraction of a second. I thought, something
happened which caused partition_join test fail in master. But then I
realised you are talking about Rajkumar's patch and test in that
patch. I think it's better to start a separate thread discussing his
patch, before I loose my heart ;)

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

#308Rajkumar Raghuwanshi
rajkumar.raghuwanshi@enterprisedb.com
In reply to: Ashutosh Bapat (#307)
Re: [HACKERS] Partition-wise join for join between (declaratively) partitioned tables

On Wed, Jun 6, 2018 at 9:21 AM, Ashutosh Bapat <
ashutosh.bapat@enterprisedb.com> wrote:

On Wed, Jun 6, 2018 at 8:11 AM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:

On Mon, Mar 5, 2018 at 8:13 PM, Rajkumar Raghuwanshi
<rajkumar.raghuwanshi@enterprisedb.com> wrote:

On Wed, Feb 7, 2018 at 2:00 PM, Ashutosh Bapat
Changed partition-wise statement to partitionwise.
Attached re-based patch.

The patch looks good to me. I don't think we can reduce it further.
But we need some tests to test PWJ with default partitions. Marking
this as ready for committer.

Hi Rajkumar,

partition_join ... FAILED

Thanks Thomas for patch review.

That made my heart stop for fraction of a second. I thought, something

happened which caused partition_join test fail in master. But then I
realised you are talking about Rajkumar's patch and test in that
patch. I think it's better to start a separate thread discussing his
patch, before I loose my heart ;)

Yeah, that would be better.

here is the new thread with updated patch.
/messages/by-id/CAKcux6ky5YeZAY74qSh-ayPZZEQchz092g71iXXbC0+E3xoscA@mail.gmail.com

Thanks & Regards,
Rajkumar Raghuwanshi
QMG, EnterpriseDB Corporation